Merge commit '627b31f8141c9789ca44a5dc384c8c6ac30ecefc' from 'release-0.9.4-pre-rc' into 'stabilization-f69489'

Change-Id: I6a1257029f8aa12ec3fb4e3ce86ee3d42e40a131
diff --git a/README.md b/README.md
index bf05c6e..e1a2d5a 100644
--- a/README.md
+++ b/README.md
@@ -85,11 +85,12 @@
 
         http://localhost:19001
 
-* Read more [documentations](https://ci.apache.org/projects/asterixdb/index.html) to learn the data model, query language, and how to create a cluster instance.
+* Read more [documentation](https://ci.apache.org/projects/asterixdb/index.html) to learn the data model, query language, and how to create a cluster instance.
 
 ## Documentation
 
 * [master](https://ci.apache.org/projects/asterixdb/index.html) |
+  [0.9.3](http://asterixdb.apache.org/docs/0.9.3/index.html) |
   [0.9.2](http://asterixdb.apache.org/docs/0.9.2/index.html) |
   [0.9.1](http://asterixdb.apache.org/docs/0.9.1/index.html) |
   [0.9.0](http://asterixdb.apache.org/docs/0.9.0/index.html)
diff --git a/asterixdb/asterix-active/pom.xml b/asterixdb/asterix-active/pom.xml
index a6f6a38..922a0e8 100644
--- a/asterixdb/asterix-active/pom.xml
+++ b/asterixdb/asterix-active/pom.xml
@@ -35,6 +35,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
index a810576..bca90e4 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
@@ -70,7 +70,7 @@
 
     @Override
     public boolean equals(Object o) {
-        if (o == null || !(o instanceof ActiveEvent)) {
+        if (!(o instanceof ActiveEvent)) {
             return false;
         }
         if (this == o) {
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index bb76204..4adffda 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.active.message.ActiveManagerMessage;
 import org.apache.asterix.active.message.ActiveStatsRequestMessage;
 import org.apache.asterix.active.message.ActiveStatsResponse;
+import org.apache.asterix.active.message.StopRuntimeParameters;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.memory.ConcurrentFramePool;
@@ -40,6 +41,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -139,7 +141,7 @@
         shutdown = true;
         runtimes.forEach((runtimeId, runtime) -> stopFutures.put(runtimeId, executor.submit(() -> {
             // we may already have been stopped- only stop once
-            stopIfRunning(runtimeId, runtime);
+            stopIfRunning(runtime, SHUTDOWN_TIMEOUT_SECS, TimeUnit.SECONDS);
             return null;
         })));
         stopFutures.entrySet().parallelStream().forEach(entry -> {
@@ -157,8 +159,10 @@
         LOGGER.warn("Shutdown ActiveManager on node " + nodeId + " complete");
     }
 
+    @SuppressWarnings("squid:S1181") // Catch Error
     private void stopRuntime(ActiveManagerMessage message) {
-        ActiveRuntimeId runtimeId = (ActiveRuntimeId) message.getPayload();
+        StopRuntimeParameters content = (StopRuntimeParameters) message.getPayload();
+        ActiveRuntimeId runtimeId = content.getRuntimeId();
         IActiveRuntime runtime = runtimes.get(runtimeId);
         if (runtime == null) {
             LOGGER.warn("Request to stop runtime: " + runtimeId
@@ -167,21 +171,21 @@
         } else {
             executor.execute(() -> {
                 try {
-                    stopIfRunning(runtimeId, runtime);
-                } catch (Exception e) {
-                    // TODO(till) Figure out a better way to handle failure to stop a runtime
-                    LOGGER.log(Level.WARN, "Failed to stop runtime: " + runtimeId, e);
+                    stopIfRunning(runtime, content.getTimeout(), content.getUnit());
+                } catch (Throwable th) {
+                    LOGGER.warn("Failed to stop runtime: {}", runtimeId, th);
+                    ExitUtil.halt(ExitUtil.EC_UNCAUGHT_THROWABLE);
                 }
             });
         }
     }
 
-    private void stopIfRunning(ActiveRuntimeId runtimeId, IActiveRuntime runtime)
+    private void stopIfRunning(IActiveRuntime runtime, long timeout, TimeUnit unit)
             throws HyracksDataException, InterruptedException {
-        if (runtimes.remove(runtimeId) != null) {
-            runtime.stop();
+        if (runtimes.containsKey(runtime.getRuntimeId())) {
+            runtime.stop(timeout, unit);
         } else {
-            LOGGER.info("Not stopping already stopped runtime " + runtimeId);
+            LOGGER.info("Not stopping already stopped runtime {}", runtime.getRuntimeId());
         }
     }
 
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
index 1df9020..453ffa0 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
@@ -18,13 +18,14 @@
  */
 package org.apache.asterix.active;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.asterix.active.message.ActivePartitionMessage;
 import org.apache.asterix.active.message.ActivePartitionMessage.Event;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -62,9 +63,11 @@
     protected abstract void start() throws HyracksDataException, InterruptedException;
 
     @Override
-    public final void stop() throws HyracksDataException, InterruptedException {
-        abort();
+    public final void stop(long timeout, TimeUnit unit) throws HyracksDataException, InterruptedException {
         synchronized (this) {
+            if (!done) {
+                abort(timeout, unit);
+            }
             while (!done) {
                 wait();
             }
@@ -74,10 +77,13 @@
     /**
      * called from a different thread. This method stops the active node and force the start() call to return
      *
+     * @param unit
+     * @param timeout
+     *
      * @throws HyracksDataException
      * @throws InterruptedException
      */
-    protected abstract void abort() throws HyracksDataException, InterruptedException;
+    protected abstract void abort(long timeout, TimeUnit unit) throws HyracksDataException, InterruptedException;
 
     @Override
     public String toString() {
@@ -86,7 +92,7 @@
 
     @Override
     public final void initialize() throws HyracksDataException {
-        LOGGER.log(Level.INFO, "initialize() called on ActiveSourceOperatorNodePushable");
+        LOGGER.trace("initialize() called on ActiveSourceOperatorNodePushable");
         taskThread = Thread.currentThread();
         activeManager.registerRuntime(this);
         try {
@@ -106,7 +112,7 @@
                 done = true;
                 notifyAll();
             }
-            LOGGER.log(Level.INFO, "initialize() returning on ActiveSourceOperatorNodePushable");
+            LOGGER.trace("initialize() returning on ActiveSourceOperatorNodePushable");
         }
     }
 
@@ -128,9 +134,4 @@
     public final IFrameWriter getInputFrameWriter(int index) {
         return null;
     }
-
-    @Override
-    public JobId getJobId() {
-        return ctx.getJobletContext().getJobId();
-    }
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
index eb43d10..1f3daac 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActivityState.java
@@ -24,11 +24,6 @@
      */
     STOPPED,
     /**
-     * Failure to recover from a temporary faliure caused the activity to fail permanantly.
-     * No further recovery attempts will be made.
-     */
-    PERMANENTLY_FAILED,
-    /**
      * An unexpected failure caused the activity to fail but recovery attempts will start taking place
      */
     TEMPORARILY_FAILED,
@@ -41,6 +36,10 @@
      */
     STARTING,
     /**
+     * The activity was started but is being cancelled. Waiting for the job cancellation to complete
+     */
+    CANCELLING,
+    /**
      * The activity has been started successfully and is running
      */
     RUNNING,
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
deleted file mode 100644
index 6633810..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicy.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.active;
-
-public class CountRetryPolicy implements IRetryPolicy {
-
-    private final int count;
-    private int attempted = 0;
-
-    public CountRetryPolicy(int count) {
-        this.count = count;
-    }
-
-    @Override
-    public boolean retry() {
-        if (attempted < count) {
-            attempted++;
-            return true;
-        }
-        return false;
-    }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
index 5e26ae4..477bb5a 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.CountRetryPolicy;
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class CountRetryPolicyFactory implements IRetryPolicyFactory {
 
     private final int count;
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
deleted file mode 100644
index bc6f1b1..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.asterix.active;
-
-import java.time.Instant;
-import java.util.Date;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.common.transactions.ITxnIdFactory;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.DeployedJobSpecId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * Provides functionality for running DeployedJobSpecs
- */
-public class DeployedJobService {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    //To enable new Asterix TxnId for separate deployed job spec invocations
-    private static final byte[] TRANSACTION_ID_PARAMETER_NAME = "TxnIdParameter".getBytes();
-
-    //pool size one (only running one thread at a time)
-    private static final int POOL_SIZE = 1;
-
-    //Starts running a deployed job specification periodically with an interval of "duration" seconds
-    public static ScheduledExecutorService startRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId,
-            IHyracksClientConnection hcc, long duration, Map<byte[], byte[]> jobParameters, EntityId entityId,
-            ITxnIdFactory txnIdFactory) {
-        ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(POOL_SIZE);
-        scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    if (!runRepetitiveDeployedJobSpec(distributedId, hcc, jobParameters, duration, entityId,
-                            txnIdFactory)) {
-                        scheduledExecutorService.shutdown();
-                    }
-                } catch (Exception e) {
-                    LOGGER.log(Level.ERROR, "Job Failed to run for " + entityId.getExtensionName() + " "
-                            + entityId.getDataverse() + "." + entityId.getEntityName() + ".", e);
-                }
-            }
-        }, duration, duration, TimeUnit.MILLISECONDS);
-        return scheduledExecutorService;
-    }
-
-    public static boolean runRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
-            Map<byte[], byte[]> jobParameters, long duration, EntityId entityId, ITxnIdFactory txnIdFactory)
-            throws Exception {
-        long executionMilliseconds = runDeployedJobSpec(distributedId, hcc, jobParameters, entityId, txnIdFactory);
-        if (executionMilliseconds > duration && LOGGER.isErrorEnabled()) {
-            LOGGER.log(Level.ERROR,
-                    "Periodic job for " + entityId.getExtensionName() + " " + entityId.getDataverse() + "."
-                            + entityId.getEntityName() + " was unable to meet the required period of " + duration
-                            + " milliseconds. Actually took " + executionMilliseconds + " execution will shutdown"
-                            + new Date());
-            return false;
-        }
-        return true;
-    }
-
-    public synchronized static long runDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
-            Map<byte[], byte[]> jobParameters, EntityId entityId, ITxnIdFactory txnIdFactory) throws Exception {
-        JobId jobId;
-        long startTime = Instant.now().toEpochMilli();
-
-        //Add the Asterix Transaction Id to the map
-        jobParameters.put(TRANSACTION_ID_PARAMETER_NAME, String.valueOf(txnIdFactory.create().getId()).getBytes());
-        jobId = hcc.startJob(distributedId, jobParameters);
-
-        hcc.waitForCompletion(jobId);
-        long executionMilliseconds = Instant.now().toEpochMilli() - startTime;
-
-        LOGGER.log(Level.INFO,
-                "Deployed Job execution completed for " + entityId.getExtensionName() + " " + entityId.getDataverse()
-                        + "." + entityId.getEntityName() + ". Took " + executionMilliseconds + " milliseconds ");
-
-        return executionMilliseconds;
-
-    }
-
-    @Override
-    public String toString() {
-        return "DeployedJobSpecService";
-    }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
index 5fe26bf..13e16f0 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
@@ -48,7 +48,7 @@
 
     @Override
     public boolean equals(Object o) {
-        if (o == null || !(o instanceof EntityId)) {
+        if (!(o instanceof EntityId)) {
             return false;
         }
         if (o == this) {
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
index 142e84b..e01d0a7 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
@@ -31,7 +31,7 @@
      * @param event
      * @throws HyracksDataException
      */
-    void notify(ActiveEvent event) throws HyracksDataException;
+    void notify(ActiveEvent event);
 
     /**
      * Checkcs whether the subscriber is done receiving events
@@ -53,5 +53,5 @@
      * @param eventsListener
      * @throws HyracksDataException
      */
-    void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException;
+    void subscribed(IActiveEntityEventsListener eventsListener);
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
index 37120e4..2d36bb2 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
@@ -56,9 +56,8 @@
      * subscribe to events. subscription ends when subscriber.done() returns true
      *
      * @param subscriber
-     * @throws HyracksDataException
      */
-    void subscribe(IActiveEntityEventSubscriber subscriber) throws HyracksDataException;
+    void subscribe(IActiveEntityEventSubscriber subscriber);
 
     /**
      * The most recent acquired stats for the active entity
@@ -99,6 +98,7 @@
 
     /**
      * Get the stats name that's used to form the stats JSON for the active entity
+     *
      * @return the customized stats name for current active entity
      */
     String getDisplayName() throws HyracksDataException;
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
index f37b2e8..a52f01e 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveRuntime.java
@@ -18,8 +18,9 @@
  */
 package org.apache.asterix.active;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
 
 public interface IActiveRuntime {
 
@@ -31,15 +32,15 @@
     /**
      * Stops the running activity
      *
+     * @param timeout
+     *            time for graceful stop. interrupt the runtime after that
+     * @param unit
+     *            unit of the timeout
+     *
      * @throws HyracksDataException
      * @throws InterruptedException
      */
-    void stop() throws HyracksDataException, InterruptedException;
-
-    /**
-     * @return the job id associated with this active runtime
-     */
-    JobId getJobId();
+    void stop(long timeout, TimeUnit unit) throws HyracksDataException, InterruptedException;
 
     /**
      * @return the runtime stats for monitoring purposes
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
deleted file mode 100644
index a010984..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicy.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.asterix.active;
-
-@FunctionalInterface
-public interface IRetryPolicy {
-    /**
-     * @return true if one more attempt should be done
-     */
-    boolean retry();
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
index a946337..82b1d5e 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IRetryPolicyFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 @FunctionalInterface
 public interface IRetryPolicyFactory {
     /**
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
index 074f8f4..6f43c64 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class InfiniteRetryPolicy implements IRetryPolicy {
 
     private final IActiveEntityEventsListener listener;
@@ -27,7 +29,7 @@
     }
 
     @Override
-    public boolean retry() {
+    public boolean retry(Throwable failure) {
         synchronized (listener) {
             try {
                 listener.wait(5000); //NOSONAR this method is being called in a while loop
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
index b31d245..d33e1da 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+
 public class InfiniteRetryPolicyFactory implements IRetryPolicyFactory {
 
     @Override
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
index 1596c17..b8af32b 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/NoRetryPolicyFactory.java
@@ -18,15 +18,17 @@
  */
 package org.apache.asterix.active;
 
+import org.apache.hyracks.util.IRetryPolicy;
+import org.apache.hyracks.util.NoRetryPolicy;
+
 public class NoRetryPolicyFactory implements IRetryPolicyFactory {
     public static final NoRetryPolicyFactory INSTANCE = new NoRetryPolicyFactory();
-    private static final IRetryPolicy policy = () -> false;
 
     private NoRetryPolicyFactory() {
     }
 
     @Override
     public IRetryPolicy create(IActiveEntityEventsListener listener) {
-        return policy;
+        return NoRetryPolicy.INSTANCE;
     }
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
index 39b905f..cb9c61b 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
@@ -82,7 +82,7 @@
 
     @Override
     public boolean equals(Object o) {
-        if (o == null || !(o instanceof ActivePartitionMessage)) {
+        if (!(o instanceof ActivePartitionMessage)) {
             return false;
         }
         if (this == o) {
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/StopRuntimeParameters.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/StopRuntimeParameters.java
new file mode 100644
index 0000000..fbc41a1
--- /dev/null
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/StopRuntimeParameters.java
@@ -0,0 +1,50 @@
+/*
+ * 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.asterix.active.message;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.active.ActiveRuntimeId;
+
+public class StopRuntimeParameters implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private final ActiveRuntimeId runtimeId;
+    private final long timeout;
+    private final TimeUnit unit;
+
+    public StopRuntimeParameters(ActiveRuntimeId runtimeId, long timeout, TimeUnit unit) {
+        this.runtimeId = runtimeId;
+        this.timeout = timeout;
+        this.unit = unit;
+    }
+
+    public ActiveRuntimeId getRuntimeId() {
+        return runtimeId;
+    }
+
+    public long getTimeout() {
+        return timeout;
+    }
+
+    public TimeUnit getUnit() {
+        return unit;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExpressionToPlanTranslatorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExpressionToPlanTranslatorFactory.java
index 5db516e..c29dee8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExpressionToPlanTranslatorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/base/ILangExpressionToPlanTranslatorFactory.java
@@ -18,7 +18,11 @@
  */
 package org.apache.asterix.algebra.base;
 
+import java.util.Map;
+
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public interface ILangExpressionToPlanTranslatorFactory {
@@ -28,10 +32,11 @@
      *            providing the definition of created (i.e., stored) user-defined functions.
      * @param currentVarCounter,
      *            the current minimum available variable id.
+     * @param externalVars
      * @return a logical query plan.
      * @throws AlgebricksException
      */
     ILangExpressionToPlanTranslator createExpressionToPlanTranslator(MetadataProvider metadataProvider,
-            int currentVarCountert) throws AlgebricksException;
+            int currentVarCounter, Map<VarIdentifier, IAObject> externalVars) throws AlgebricksException;
 
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
index 724b284..a37f802 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
@@ -18,10 +18,11 @@
  */
 package org.apache.asterix.algebra.extension;
 
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,7 +30,7 @@
 /**
  * An interface that provides an extension mechanism to extend a language with additional statements
  */
-public abstract class ExtensionStatement implements Statement {
+public abstract class ExtensionStatement extends AbstractStatement {
 
     @Override
     public final Kind getKind() {
@@ -45,10 +46,11 @@
      * @param requestParameters
      * @param metadataProvider
      * @param resultSetId
+     * @param executorCtx
      * @throws HyracksDataException
      * @throws AlgebricksException
      */
     public abstract void handle(IHyracksClientConnection hcc, IStatementExecutor statementExecutor,
-            IRequestParameters requestParameters, MetadataProvider metadataProvider, int resultSetId)
-            throws HyracksDataException, AlgebricksException;
+            IRequestParameters requestParameters, MetadataProvider metadataProvider, int resultSetId,
+            IStatementExecutorContext executorCtx) throws HyracksDataException, AlgebricksException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index beb47a8..94d3461 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -47,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -59,6 +60,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -120,16 +122,28 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
+        ITupleFilterFactory tupleFilterFactory = null;
+        long outputLimit = -1;
+        if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+            UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+            outputLimit = unnestMapOp.getOutputLimit();
+            if (unnestMapOp.getSelectCondition() != null) {
+                tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+                        typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+            }
+        }
         // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
         boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
                 jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
                 jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
-                unnestMap.getGenerateCallBackProceedResultVar());
+                tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar());
+        IOperatorDescriptor opDesc = btreeSearch.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
-        builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, btreeSearch.second);
 
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
index f06fe45..a24a83b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
@@ -85,6 +85,7 @@
 
         //get dataset splits
         IPushRuntimeFactory runtime = dataset.getCommitRuntimeFactory(metadataProvider, primaryKeyFields, isSink);
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 1d79455..fc899c6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -135,8 +135,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalLoopup =
                 metadataProvider.buildExternalDataLookupRuntime(builder.getJobSpec(), dataset, ridIndexes, retainInput,
                         typeEnv, opSchema, context, metadataProvider, retainMissing);
-        builder.contributeHyracksOperator(unnestMap, externalLoopup.first);
-        builder.contributeAlgebricksPartitionConstraint(externalLoopup.first, externalLoopup.second);
+        IOperatorDescriptor opDesc = externalLoopup.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, externalLoopup.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index eeb6688..7b5eb83 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -120,10 +120,12 @@
                         jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
                         jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
                         minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch(), frameLimit);
+        IOperatorDescriptor opDesc = invIndexSearch.first;
+        opDesc.setSourceLocation(unnestMapOp.getSourceLocation());
 
         // Contribute operator in hyracks job.
-        builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
+        builder.contributeHyracksOperator(unnestMapOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, invIndexSearch.second);
         ILogicalOperator srcExchange = unnestMapOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 949be23..a80e5b3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -102,9 +102,11 @@
                 builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
                 dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
                 maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
+        IOperatorDescriptor opDesc = rtreeSearch.first;
+        opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
-        builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
+        builder.contributeHyracksOperator(unnestMap, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, rtreeSearch.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/AqlCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/AqlCompilationProvider.java
index af0ba73..548917a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/AqlCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/AqlCompilationProvider.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.compiler.provider;
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.aql.rewrites.AQLRewriterFactory;
 import org.apache.asterix.lang.aql.visitor.AQLAstPrintVisitorFactory;
@@ -30,6 +31,11 @@
 public class AqlCompilationProvider implements ILangCompilationProvider {
 
     @Override
+    public ILangExtension.Language getLanguage() {
+        return ILangExtension.Language.AQL;
+    }
+
+    @Override
     public IParserFactory getParserFactory() {
         return new AQLParserFactory();
     }
@@ -53,5 +59,4 @@
     public IRuleSetFactory getRuleSetFactory() {
         return new DefaultRuleSetFactory();
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
index f658be9..f625343 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
@@ -19,35 +19,39 @@
 package org.apache.asterix.compiler.provider;
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 
 public interface ILangCompilationProvider {
+    /**
+     * @return language kind
+     */
+    ILangExtension.Language getLanguage();
 
     /**
      * @return the parser factory of a language implementation.
      */
-    public IParserFactory getParserFactory();
+    IParserFactory getParserFactory();
 
     /**
      * @return the rewriter factory of a language implementation.
      */
-    public IRewriterFactory getRewriterFactory();
+    IRewriterFactory getRewriterFactory();
 
     /**
      * @return the AST printer factory of a language implementation.
      */
-    public IAstPrintVisitorFactory getAstPrintVisitorFactory();
+    IAstPrintVisitorFactory getAstPrintVisitorFactory();
 
     /**
      * @return the language expression to logical query plan translator factory of a language implementation.
      */
-    public ILangExpressionToPlanTranslatorFactory getExpressionToPlanTranslatorFactory();
+    ILangExpressionToPlanTranslatorFactory getExpressionToPlanTranslatorFactory();
 
     /**
      * @return the rule set factory of a language implementation
      */
-    public IRuleSetFactory getRuleSetFactory();
-
+    IRuleSetFactory getRuleSetFactory();
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 70f8f92..6451b6f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.compiler.provider;
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
@@ -30,6 +31,11 @@
 public class SqlppCompilationProvider implements ILangCompilationProvider {
 
     @Override
+    public ILangExtension.Language getLanguage() {
+        return ILangExtension.Language.SQLPP;
+    }
+
+    @Override
     public IParserFactory getParserFactory() {
         return new SqlppParserFactory();
     }
@@ -53,5 +59,4 @@
     public IRuleSetFactory getRuleSetFactory() {
         return new DefaultRuleSetFactory();
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 02d79c4..32f8001 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -194,10 +194,11 @@
         }
     }
 
-    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env,
+    private IFunctionDescriptor resolveFunction(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
             JobGenContext context) throws AlgebricksException {
-        FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
+        FunctionIdentifier fnId = expr.getFunctionIdentifier();
         IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
+        fd.setSourceLocation(expr.getSourceLocation());
         IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
         if (fnTypeInfer != null) {
             CompilerProperties compilerProps = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 93c7d79..6749bf9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -32,7 +32,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -58,7 +57,7 @@
     }
 
     public static int numberOfVarsInExpr(ILogicalExpression e) {
-        switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+        switch (e.getExpressionTag()) {
             case CONSTANT: {
                 return 0;
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
index 503a631..d7e05b3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
@@ -60,7 +60,7 @@
     }
 
     public static IAObject getSimThreshold(MetadataProvider metadata, String simFuncName) {
-        String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
+        String simThresholValue = metadata.getProperty(SIM_THRESHOLD_PROP_NAME);
         IAObject ret = null;
         if (simFuncName.equals(JACCARD_FUNCTION_NAME)) {
             if (simThresholValue != null) {
@@ -103,7 +103,7 @@
 
     public static float getSimThreshold(MetadataProvider metadata) {
         float simThreshold = JACCARD_DEFAULT_SIM_THRESHOLD;
-        String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
+        String simThresholValue = metadata.getProperty(SIM_THRESHOLD_PROP_NAME);
         if (simThresholValue != null) {
             simThreshold = Float.parseFloat(simThresholValue);
         }
@@ -112,7 +112,7 @@
 
     // TODO: The default function depend on the input types.
     public static String getSimFunction(MetadataProvider metadata) {
-        String simFunction = metadata.getPropertyValue(SIM_FUNCTION_PROP_NAME);
+        String simFunction = metadata.getProperty(SIM_FUNCTION_PROP_NAME);
         if (simFunction == null) {
             simFunction = DEFAULT_SIM_FUNCTION;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index cd217ab..f9b5c38 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -63,6 +63,7 @@
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
 import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
 import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
+import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
 import org.apache.asterix.optimizer.rules.PushProperJoinThroughProduct;
 import org.apache.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
 import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
@@ -181,13 +182,14 @@
         normalization.add(new IntroduceDynamicTypeCastRule());
         normalization.add(new IntroduceDynamicTypeCastForExternalFunctionRule());
         normalization.add(new IntroduceEnforcedListTypeRule());
+        // Perform constant folding before common expression extraction
+        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new ExtractCommonExpressionsRule());
 
         // Let PushAggFuncIntoStandaloneAggregateRule run after ExtractCommonExpressionsRule
         // so that PushAggFunc can happen in fewer places.
         normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
         normalization.add(new ListifyUnnestingFunctionRule());
-        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new RemoveRedundantSelectRule());
         normalization.add(new UnnestToDataScanRule());
         normalization.add(new MetaFunctionToMetaVariableRule());
@@ -350,6 +352,9 @@
         // We are going to apply a constant folding rule again for this case.
         physicalRewritesTopLevel.add(new ConstantFoldingRule(appCtx));
         physicalRewritesTopLevel.add(new PushLimitIntoOrderByRule());
+        physicalRewritesTopLevel.add(new PushLimitIntoPrimarySearchRule());
+        // remove assigns that could become unused after PushLimitIntoPrimarySearchRule
+        physicalRewritesTopLevel.add(new RemoveUnusedAssignAndAggregateRule());
         physicalRewritesTopLevel.add(new IntroduceProjectsRule());
         physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
index b9b5340..8219bcd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AddEquivalenceClassForRecordConstructorRule.java
@@ -112,14 +112,16 @@
                 LogicalVariable fieldVar = varExpr.getVariableReference();
                 Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(assignOp);
                 if (ecs == null) {
-                    ecs = new HashMap<LogicalVariable, EquivalenceClass>();
+                    ecs = new HashMap<>();
                     context.putEquivalenceClassMap(assignOp, ecs);
                 }
-                ILogicalExpression expr = new ScalarFunctionCallExpression(
+                VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+                recordVarRef.setSourceLocation(funcExpr.getSourceLocation());
+                ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(recordVar)),
-                        new MutableObject<ILogicalExpression>(
+                        new MutableObject<>(recordVarRef), new MutableObject<>(
                                 new ConstantExpression(new AsterixConstantValue(new AInt32(parameterIndex / 2))))); // Every two parameters corresponds to a field.
+                expr.setSourceLocation(funcExpr.getSourceLocation());
                 EquivalenceClass equivClass = new EquivalenceClass(Collections.singletonList(fieldVar), fieldVar,
                         Collections.singletonList(expr));
                 ecs.put(fieldVar, equivClass);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
index 5a70d3c..8bfd120 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.rewriter.rules.AbstractIntroduceGroupByCombinerRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AsterixIntroduceGroupByCombinerRule extends AbstractIntroduceGroupByCombinerRule {
 
@@ -43,36 +44,46 @@
     @Override
     protected void processNullTest(IOptimizationContext context, GroupByOperator nestedGby,
             List<LogicalVariable> aggregateVarsProducedByCombiner) {
+        SourceLocation sourceLoc = nestedGby.getSourceLocation();
         IFunctionInfo finfoEq = context.getMetadataProvider().lookupFunction(BuiltinFunctions.IS_SYSTEM_NULL);
         SelectOperator selectNonSystemNull;
 
         if (aggregateVarsProducedByCombiner.size() == 1) {
-            ILogicalExpression isSystemNullTest =
-                    new ScalarFunctionCallExpression(finfoEq, new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(aggregateVarsProducedByCombiner.get(0))));
+            VariableReferenceExpression aggVarRef =
+                    new VariableReferenceExpression(aggregateVarsProducedByCombiner.get(0));
+            aggVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression isSystemNullTest =
+                    new ScalarFunctionCallExpression(finfoEq, new MutableObject<>(aggVarRef));
+            isSystemNullTest.setSourceLocation(sourceLoc);
             IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
             ScalarFunctionCallExpression nonSystemNullTest =
-                    new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isSystemNullTest));
-            selectNonSystemNull =
-                    new SelectOperator(new MutableObject<ILogicalExpression>(nonSystemNullTest), false, null);
+                    new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
+            nonSystemNullTest.setSourceLocation(sourceLoc);
+            selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest), false, null);
+            selectNonSystemNull.setSourceLocation(sourceLoc);
         } else {
-            List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<>();
             for (LogicalVariable aggVar : aggregateVarsProducedByCombiner) {
-                ILogicalExpression isSystemNullTest = new ScalarFunctionCallExpression(finfoEq,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(aggVar)));
+                VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+                aggVarRef.setSourceLocation(sourceLoc);
+                ScalarFunctionCallExpression isSystemNullTest =
+                        new ScalarFunctionCallExpression(finfoEq, new MutableObject<>(aggVarRef));
+                isSystemNullTest.setSourceLocation(sourceLoc);
                 IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
-                ScalarFunctionCallExpression nonSystemNullTest = new ScalarFunctionCallExpression(finfoNot,
-                        new MutableObject<ILogicalExpression>(isSystemNullTest));
-                isSystemNullTestList.add(new MutableObject<ILogicalExpression>(nonSystemNullTest));
+                ScalarFunctionCallExpression nonSystemNullTest =
+                        new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
+                nonSystemNullTest.setSourceLocation(sourceLoc);
+                isSystemNullTestList.add(new MutableObject<>(nonSystemNullTest));
             }
             IFunctionInfo finfoAnd = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
-            selectNonSystemNull = new SelectOperator(new MutableObject<ILogicalExpression>(
-                    new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+            selectNonSystemNull = new SelectOperator(
+                    new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+            selectNonSystemNull.setSourceLocation(sourceLoc);
         }
 
         //add the not-system-null check into the nested pipeline
         Mutable<ILogicalOperator> ntsBeforeNestedGby = nestedGby.getInputs().get(0);
-        nestedGby.getInputs().set(0, new MutableObject<ILogicalOperator>(selectNonSystemNull));
+        nestedGby.getInputs().set(0, new MutableObject<>(selectNonSystemNull));
         selectNonSystemNull.getInputs().add(ntsBeforeNestedGby);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
index 5cf6769..b0c3bdf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByHandleFieldAccessRule.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ByNameToByHandleFieldAccessRule implements IAlgebraicRewriteRule {
 
@@ -65,6 +66,7 @@
         AbstractFunctionCallExpression fce =
                 (AbstractFunctionCallExpression) fieldAccessOp.getExpressions().get(0).getValue();
         ILogicalExpression a1 = fce.getArguments().get(0).getValue();
+        SourceLocation sourceLoc = fieldAccessOp.getSourceLocation();
 
         VariableReferenceExpression x;
         if (a1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
@@ -76,7 +78,9 @@
             ArrayList<Mutable<ILogicalExpression>> exprArray = new ArrayList<Mutable<ILogicalExpression>>(1);
             exprArray.add(new MutableObject<ILogicalExpression>(a1));
             AssignOperator assignVar = new AssignOperator(varArray, exprArray);
+            assignVar.setSourceLocation(sourceLoc);
             x = new VariableReferenceExpression(var1);
+            x.setSourceLocation(sourceLoc);
             assignVar.getInputs().add(opUnder);
             opUnder = new MutableObject<ILogicalOperator>(assignVar);
         }
@@ -88,24 +92,33 @@
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.TYPE_OF));
         typeOf.getArguments().add(new MutableObject<ILogicalExpression>(x));
         AssignOperator typAssign = new AssignOperator(t, new MutableObject<ILogicalExpression>(typeOf));
+        typAssign.setSourceLocation(sourceLoc);
         typAssign.getInputs().add(opUnder);
 
         // let $w := get-handle($t, path-expression)
         LogicalVariable w = context.newVar();
         AbstractFunctionCallExpression getHandle =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_HANDLE));
-        getHandle.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(t)));
+        getHandle.setSourceLocation(sourceLoc);
+        VariableReferenceExpression tRef = new VariableReferenceExpression(t);
+        tRef.setSourceLocation(sourceLoc);
+        getHandle.getArguments().add(new MutableObject<ILogicalExpression>(tRef));
         // the accessed field
         getHandle.getArguments().add(new MutableObject<ILogicalExpression>(fce.getArguments().get(1).getValue()));
         AssignOperator handleAssign = new AssignOperator(w, new MutableObject<ILogicalExpression>(getHandle));
+        handleAssign.setSourceLocation(sourceLoc);
         handleAssign.getInputs().add(new MutableObject<ILogicalOperator>(typAssign));
 
         // let $y := get-data(x, $w)
         AbstractFunctionCallExpression getData =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_DATA));
+        getData.setSourceLocation(sourceLoc);
         VariableReferenceExpression ref2 = new VariableReferenceExpression(x.getVariableReference());
+        ref2.setSourceLocation(sourceLoc);
         getData.getArguments().add(new MutableObject<ILogicalExpression>(ref2));
-        getData.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(w)));
+        VariableReferenceExpression wRef = new VariableReferenceExpression(w);
+        wRef.setSourceLocation(sourceLoc);
+        getData.getArguments().add(new MutableObject<ILogicalExpression>(wRef));
         fieldAccessOp.getExpressions().get(0).setValue(getData);
         List<Mutable<ILogicalOperator>> faInputs = fieldAccessOp.getInputs();
         faInputs.clear();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
index 19cd4b6..9c478ee 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
@@ -47,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ByNameToByIndexFieldAccessRule implements IAlgebraicRewriteRule {
 
@@ -100,10 +101,14 @@
         if (firstArg.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
+        SourceLocation sourceLoc = firstArg.getSourceLocation();
         LogicalVariable var1 = context.newVar();
         AssignOperator assignOp = new AssignOperator(new ArrayList<>(Collections.singletonList(var1)),
                 new ArrayList<>(Collections.singletonList(new MutableObject<>(firstArg))));
-        fce.getArguments().get(0).setValue(new VariableReferenceExpression(var1));
+        assignOp.setSourceLocation(sourceLoc);
+        VariableReferenceExpression var1Ref = new VariableReferenceExpression(var1);
+        var1Ref.setSourceLocation(sourceLoc);
+        fce.getArguments().get(0).setValue(var1Ref);
         assignOp.getInputs().add(new MutableObject<>(op.getInputs().get(0).getValue()));
         op.getInputs().get(0).setValue(assignOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
@@ -148,8 +153,10 @@
         if (k < 0) {
             return null;
         }
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                fce.getArguments().get(0),
+        ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), fce.getArguments().get(0),
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(k)))));
+        faExpr.setSourceLocation(fce.getSourceLocation());
+        return faExpr;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
index 6cc87ad..069ba49 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
@@ -36,7 +36,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -51,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 import com.google.common.collect.ImmutableSet;
 
@@ -205,8 +205,7 @@
 
         LogicalVariable aggVar = agg.getVariables().get(0);
         ILogicalExpression aggFun = agg.getExpressions().get(0).getValue();
-        if (!aggVar.equals(unnestedVar)
-                || ((AbstractLogicalExpression) aggFun).getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+        if (!aggVar.equals(unnestedVar) || aggFun.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) aggFun;
@@ -217,7 +216,7 @@
             return false;
         }
         ILogicalExpression arg0 = f.getArguments().get(0).getValue();
-        if (((AbstractLogicalExpression) arg0).getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+        if (arg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
         LogicalVariable paramVar = ((VariableReferenceExpression) arg0).getVariableReference();
@@ -225,8 +224,11 @@
         ArrayList<LogicalVariable> assgnVars = new ArrayList<LogicalVariable>(1);
         assgnVars.add(unnest1.getVariable());
         ArrayList<Mutable<ILogicalExpression>> assgnExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
-        assgnExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(paramVar)));
+        VariableReferenceExpression paramRef = new VariableReferenceExpression(paramVar);
+        paramRef.setSourceLocation(arg0.getSourceLocation());
+        assgnExprs.add(new MutableObject<ILogicalExpression>(paramRef));
         AssignOperator assign = new AssignOperator(assgnVars, assgnExprs);
+        assign.setSourceLocation(arg0.getSourceLocation());
 
         LogicalVariable posVar = unnest1.getPositionalVariable();
         if (posVar == null) {
@@ -252,6 +254,7 @@
             opRef.setValue(assign);
             assign.getInputs().add(aggInputOpRef);
             AssignOperator gbyKeyAssign = new AssignOperator(gbyKeyAssgnVars, gbyKeyAssgnExprs);
+            gbyKeyAssign.setSourceLocation(gby.getSourceLocation());
             gbyKeyAssign.getInputs().add(gby.getInputs().get(0));
             bottomOpRef.setValue(gbyKeyAssign);
 
@@ -266,6 +269,8 @@
             nestedAssignVars.add(unnest1.getVariable());
             nestedAssignExprs.add(new MutableObject<ILogicalExpression>(arg0));
             AssignOperator nestedAssign = new AssignOperator(nestedAssignVars, nestedAssignExprs);
+            SourceLocation sourceLoc = unnest1.getSourceLocation();
+            nestedAssign.setSourceLocation(sourceLoc);
             nestedAssign.getInputs().add(opRef2);
 
             // Then create running aggregation for the positional variable
@@ -274,8 +279,10 @@
             raggVars.add(posVar);
             StatefulFunctionCallExpression fce = new StatefulFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+            fce.setSourceLocation(sourceLoc);
             raggExprs.add(new MutableObject<ILogicalExpression>(fce));
             RunningAggregateOperator raggOp = new RunningAggregateOperator(raggVars, raggExprs);
+            raggOp.setSourceLocation(sourceLoc);
             raggOp.setExecutionMode(unnest1.getExecutionMode());
             RunningAggregatePOperator raggPOp = new RunningAggregatePOperator();
             raggOp.setPhysicalOperator(raggPOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
index ab359bf..1eb3e75 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
@@ -19,6 +19,8 @@
 
 package org.apache.asterix.optimizer.rules;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
@@ -62,7 +64,7 @@
         IAType condType = (IAType) env.getType(condition);
         if (condType.getTypeTag() != ATypeTag.BOOLEAN && condType.getTypeTag() != ATypeTag.ANY
                 && !isPossibleBoolean(condType)) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, condition.getSourceLocation(),
                     "The select condition " + condition.toString() + " should be of the boolean type.");
         }
         return false;
@@ -75,11 +77,13 @@
      * @return true if it is; false otherwise.
      */
     private boolean isPossibleBoolean(IAType type) {
-        while (NonTaggedFormatUtil.isOptional(type)) {
-            IAType actualType = ((AUnionType) type).getActualType();
+        IAType checkingType = type;
+        while (NonTaggedFormatUtil.isOptional(checkingType)) {
+            IAType actualType = ((AUnionType) checkingType).getActualType();
             if (actualType.getTypeTag() == ATypeTag.BOOLEAN || actualType.getTypeTag() == ATypeTag.ANY) {
                 return true;
             }
+            checkingType = actualType;
         }
         return false;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
index 7c01ebd..5849d88 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
@@ -42,8 +42,9 @@
         if (checked) {
             return false;
         }
-        if (InsertUpsertCheckUtil.check(opRef.getValue())) {
-            throw new CompilationException(ErrorCode.COMPILATION_INVALID_RETURNING_EXPRESSION);
+        ILogicalOperator op = opRef.getValue();
+        if (InsertUpsertCheckUtil.check(op)) {
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_RETURNING_EXPRESSION, op.getSourceLocation());
         }
         checked = true;
         return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 29a8e77..a2c500a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -37,6 +38,7 @@
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -78,6 +80,7 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 
 public class ConstantFoldingRule implements IAlgebraicRewriteRule {
@@ -93,7 +96,11 @@
             BuiltinFunctions.FIELD_ACCESS_NESTED, BuiltinFunctions.GET_ITEM, BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR,
             BuiltinFunctions.FIELD_ACCESS_BY_INDEX, BuiltinFunctions.CAST_TYPE, BuiltinFunctions.META,
             BuiltinFunctions.META_KEY, BuiltinFunctions.RECORD_CONCAT, BuiltinFunctions.RECORD_CONCAT_STRICT,
-            BuiltinFunctions.TO_ATOMIC, BuiltinFunctions.TO_ARRAY);
+            BuiltinFunctions.RECORD_PAIRS, BuiltinFunctions.PAIRS, BuiltinFunctions.TO_ATOMIC,
+            BuiltinFunctions.TO_ARRAY);
+
+    private static final Map<FunctionIdentifier, IAObject> FUNC_ID_TO_CONSTANT = ImmutableMap
+            .of(BuiltinFunctions.NUMERIC_E, new ADouble(Math.E), BuiltinFunctions.NUMERIC_PI, new ADouble(Math.PI));
 
     /**
      * Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method.
@@ -220,6 +227,10 @@
                         return new Pair<>(changed, expr);
                     }
                 }
+                IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
+                if (c != null) {
+                    return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(c)));
+                }
 
                 IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
                         _emptyTypeEnv, _emptySchemas, jobGenCtx);
@@ -235,8 +246,8 @@
                 IAObject o = (IAObject) serde.deserialize(dis);
                 return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
             } catch (HyracksDataException | AlgebricksException e) {
-                if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-                    AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Exception caught at constant folding: " + e, e);
+                if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+                    AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Exception caught at constant folding: " + e, e);
                 }
                 return new Pair<>(false, null);
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CountVarToCountOneRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CountVarToCountOneRule.java
index 46c441a..5205672 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CountVarToCountOneRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CountVarToCountOneRule.java
@@ -81,8 +81,7 @@
         if (exp3.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
-        if (((AbstractLogicalOperator) agg.getInputs().get(0).getValue())
-                .getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+        if (agg.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
             return false;
         }
         fun.getArguments().get(0).setValue(new ConstantExpression(new AsterixConstantValue(new AInt64(1L))));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
index 85ee0ef..952c12b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class DisjunctivePredicateToJoinRule implements IAlgebraicRewriteRule {
 
@@ -117,6 +118,8 @@
             }
         }
 
+        SourceLocation sourceLoc = select.getSourceLocation();
+
         AOrderedList list = new AOrderedList(new AOrderedListType(valType, "orderedlist"));
         for (AsterixConstantValue value : values) {
             list.add(value.getObject());
@@ -129,14 +132,19 @@
         Mutable<ILogicalExpression> mutCExp = new MutableObject<>(cExp);
         IFunctionInfo scanFctInfo = BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.SCAN_COLLECTION);
         UnnestingFunctionCallExpression scanExp = new UnnestingFunctionCallExpression(scanFctInfo, mutCExp);
+        scanExp.setSourceLocation(sourceLoc);
         LogicalVariable scanVar = context.newVar();
         UnnestOperator unn = new UnnestOperator(scanVar, new MutableObject<>(scanExp));
+        unn.setSourceLocation(sourceLoc);
         unn.getInputs().add(new MutableObject<>(ets));
         context.computeAndSetTypeEnvironmentForOperator(unn);
 
         IFunctionInfo eqFctInfo = BuiltinFunctions.getAsterixFunctionInfo(AlgebricksBuiltinFunctions.EQ);
         AbstractFunctionCallExpression eqExp = new ScalarFunctionCallExpression(eqFctInfo);
-        eqExp.getArguments().add(new MutableObject<>(new VariableReferenceExpression(scanVar)));
+        eqExp.setSourceLocation(sourceLoc);
+        VariableReferenceExpression scanVarRef = new VariableReferenceExpression(scanVar);
+        scanVarRef.setSourceLocation(sourceLoc);
+        eqExp.getArguments().add(new MutableObject<>(scanVarRef));
         eqExp.getArguments().add(new MutableObject<>(varEx.cloneExpression()));
         eqExp.getAnnotations().put(IndexedNLJoinExpressionAnnotation.INSTANCE,
                 IndexedNLJoinExpressionAnnotation.INSTANCE);
@@ -145,6 +153,7 @@
         eqExp.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
 
         InnerJoinOperator jOp = new InnerJoinOperator(new MutableObject<>(eqExp));
+        jOp.setSourceLocation(sourceLoc);
         jOp.getInputs().add(new MutableObject<>(unn));
         jOp.getInputs().add(select.getInputs().get(0));
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
index 01b37a4..f340898 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractDistinctByExpressionsRule.java
@@ -69,7 +69,8 @@
                 continue;
             }
             LogicalVariable v = extractExprIntoAssignOpRef(expr.getValue(), opRef2, context);
-            ILogicalExpression newExpr = new VariableReferenceExpression(v);
+            VariableReferenceExpression newExpr = new VariableReferenceExpression(v);
+            newExpr.setSourceLocation(expr.getValue().getSourceLocation());
             newExprList.add(new MutableObject<ILogicalExpression>(newExpr));
             changed = true;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
index 4fbd632..585077f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ExtractOrderExpressionsRule.java
@@ -65,7 +65,9 @@
             ILogicalExpression expr = orderPair.second.getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE && !AnalysisUtil.isAccessToFieldRecord(expr)) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                orderPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                orderPair.second.setValue(vRef);
             }
         }
         context.computeAndSetTypeEnvironmentForOperator(oo);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
index fef2323..6b03d32 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FeedScanCollectionToUnnest.java
@@ -63,9 +63,10 @@
         UnnestOperator unnest = (UnnestOperator) op;
         ILogicalExpression unnestExpr = unnest.getExpressionRef().getValue();
         if (needsScanCollection(unnestExpr, op)) {
-            ILogicalExpression newExpr =
+            UnnestingFunctionCallExpression newExpr =
                     new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
                             new MutableObject<ILogicalExpression>(unnestExpr));
+            newExpr.setSourceLocation(unnestExpr.getSourceLocation());
             unnest.getExpressionRef().setValue(newExpr);
             context.addToDontApplySet(this, op);
             return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
index c594ff8..bd79a38 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableInt;
@@ -102,7 +103,8 @@
                         && ((DelegateOperator) op).getDelegate() instanceof CommitOperator)) {
             for (Map.Entry<AbstractReplicateOperator, MutableInt> entry : replicateOperators.entrySet()) {
                 if (entry.getKey().getOutputs().size() != entry.getValue().getValue()) {
-                    throw new AlgebricksException(ErrorCode.ASTERIX, ErrorCode.COMPILATION_FAILED_DUE_TO_REPLICATE_OP);
+                    throw new CompilationException(ErrorCode.COMPILATION_FAILED_DUE_TO_REPLICATE_OP,
+                            op.getSourceLocation());
                 }
             }
             return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
index a93b732..73da2b4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
@@ -37,13 +37,13 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Checks whether the given parameters of the ftcontains() function are correct during the compilation.
@@ -99,7 +99,7 @@
         @Override
         public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
             ILogicalExpression e = exprRef.getValue();
-            switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+            switch (e.getExpressionTag()) {
                 case FUNCTION_CALL:
                     return transformFunctionCallExpression((AbstractFunctionCallExpression) e);
                 default:
@@ -144,8 +144,8 @@
 
                 // The number of parameters should be three: exp1, exp2, and the option
                 if (oldExprs.size() != numberOfCorrectArguments) {
-                    throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, fi,
-                            oldExprs.size());
+                    throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
+                            funcExpr.getSourceLocation(), fi, oldExprs.size());
                 }
 
                 // The last expression before the option needs to be copied first.
@@ -184,8 +184,8 @@
             ILogicalExpression firstExpr = exprs.get(0).getValue();
             if (firstExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
                     && ConstantExpressionUtil.getConstantIaObjectType(firstExpr) != ATypeTag.STRING) {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName,
-                        ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, firstExpr.getSourceLocation(),
+                        functionName, ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
             }
 
             // Check the second parameter - Expression2. If it's a constant, then we can check the type here.
@@ -198,7 +198,8 @@
                     case ARRAY:
                         break;
                     default:
-                        throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, exprTypeTag);
+                        throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, secondExpr.getSourceLocation(),
+                                functionName, exprTypeTag);
                 }
             }
         }
@@ -216,12 +217,14 @@
             FunctionIdentifier openRecConsFi = openRecConsExpr.getFunctionIdentifier();
             if (openRecConsFi != BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR
                     && openRecConsFi != BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR) {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, openRecConsFi);
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, openRecConsExpr.getSourceLocation(),
+                        functionName, openRecConsFi);
             }
 
             // We multiply 2 because the layout of the arguments are: [expr, val, expr1, val1, ...]
             if (openRecConsExpr.getArguments().size() > FullTextContainsDescriptor.getParamTypeMap().size() * 2) {
-                throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION, functionName);
+                throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION,
+                        openRecConsExpr.getSourceLocation(), functionName);
             }
 
             for (int i = 0; i < openRecConsExpr.getArguments().size(); i = i + 2) {
@@ -231,13 +234,14 @@
                 String option = ConstantExpressionUtil.getStringConstant(optionExpr);
 
                 if (optionExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT || option == null) {
-                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName,
-                            optionExpr.getExpressionTag());
+                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExpr.getSourceLocation(),
+                            functionName, optionExpr.getExpressionTag());
                 }
 
                 option = option.toLowerCase();
                 if (!FullTextContainsDescriptor.getParamTypeMap().containsKey(option)) {
-                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                    throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExprVal.getSourceLocation(),
+                            functionName, option);
                 }
 
                 String optionTypeStringVal = null;
@@ -248,22 +252,25 @@
                         case STRING:
                             optionTypeStringVal = ConstantExpressionUtil.getStringConstant(optionExprVal);
                             if (optionTypeStringVal == null) {
-                                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                        optionExprVal.getSourceLocation(), functionName, option);
                             }
                             optionTypeStringVal = optionTypeStringVal.toLowerCase();
                             break;
                         default:
                             // Currently, we only have a string parameter. So, the flow doesn't reach here.
-                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                    optionExprVal.getSourceLocation(), functionName, option);
                     }
 
                     // Check the validity of option value
                     switch (option) {
                         case FullTextContainsDescriptor.SEARCH_MODE_OPTION:
-                            checkSearchModeOption(optionTypeStringVal, functionName);
+                            checkSearchModeOption(optionTypeStringVal, functionName, optionExprVal.getSourceLocation());
                             break;
                         default:
-                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, option);
+                            throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+                                    optionExprVal.getSourceLocation(), functionName, option);
                     }
                 }
 
@@ -273,12 +280,13 @@
             }
         }
 
-        private void checkSearchModeOption(String optionVal, String functionName) throws AlgebricksException {
+        private void checkSearchModeOption(String optionVal, String functionName, SourceLocation sourceLoc)
+                throws AlgebricksException {
             if (optionVal.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)
                     || optionVal.equals(FullTextContainsDescriptor.DISJUNCTIVE_SEARCH_MODE_OPTION)) {
                 return;
             } else {
-                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, functionName, optionVal);
+                throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, functionName, optionVal);
             }
         }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
index 511ec55..895746a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyEqRule.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FuzzyEqRule implements IAlgebraicRewriteRule {
 
@@ -88,6 +89,7 @@
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) exp;
         FunctionIdentifier fi = funcExp.getFunctionIdentifier();
         if (fi.equals(BuiltinFunctions.FUZZY_EQ)) {
+            SourceLocation sourceLoc = funcExp.getSourceLocation();
             List<Mutable<ILogicalExpression>> inputExps = funcExp.getArguments();
 
             String simFuncName = FuzzyUtils.getSimFunction(metadataProvider);
@@ -100,13 +102,15 @@
             FunctionIdentifier simFunctionIdentifier = FuzzyUtils.getFunctionIdentifier(simFuncName);
             ScalarFunctionCallExpression similarityExp = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(simFunctionIdentifier), similarityArgs);
+            similarityExp.setSourceLocation(sourceLoc);
             // Add annotations from the original fuzzy-eq function.
             similarityExp.getAnnotations().putAll(funcExp.getAnnotations());
             ArrayList<Mutable<ILogicalExpression>> cmpArgs = new ArrayList<Mutable<ILogicalExpression>>();
             cmpArgs.add(new MutableObject<ILogicalExpression>(similarityExp));
             IAObject simThreshold = FuzzyUtils.getSimThreshold(metadataProvider, simFuncName);
-            cmpArgs.add(new MutableObject<ILogicalExpression>(
-                    new ConstantExpression(new AsterixConstantValue(simThreshold))));
+            ConstantExpression simThresholdExpr = new ConstantExpression(new AsterixConstantValue(simThreshold));
+            simThresholdExpr.setSourceLocation(sourceLoc);
+            cmpArgs.add(new MutableObject<ILogicalExpression>(simThresholdExpr));
             ScalarFunctionCallExpression cmpExpr = FuzzyUtils.getComparisonExpr(simFuncName, cmpArgs);
             expRef.setValue(cmpExpr);
             return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
index 003cff0..d814ba0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -329,12 +329,13 @@
             switch (joinOp.getJoinKind()) {
                 case INNER: {
                     extraSelect = new SelectOperator(expRef, false, null);
+                    extraSelect.setSourceLocation(expRef.getValue().getSourceLocation());
                     extraSelect.getInputs().add(new MutableObject<ILogicalOperator>(outputOp));
                     outputOp = extraSelect;
                     break;
                 }
                 case LEFT_OUTER: {
-                    if (((AbstractLogicalOperator) outputOp).getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+                    if (outputOp.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
                         throw new IllegalStateException();
                     }
                     LeftOuterJoinOperator topJoin = (LeftOuterJoinOperator) outputOp;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
index 27b2712..683d29f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForSwitchCaseRule.java
@@ -22,6 +22,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.asterix.dataflow.data.common.TypeResolverUtil;
 import org.apache.asterix.lang.common.util.FunctionUtil;
@@ -38,14 +39,23 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
+import com.google.common.collect.ImmutableSet;
+
 /**
- * This rule injects cast functions for "THEN" and "ELSE" branches of a switch-case function if
- * different "THEN" and "ELSE" branches have heterogeneous return types.
+ * This rule injects casts for function parameters if they have heterogeneous return types:
+ * <ul>
+ *     <li>for "THEN" and "ELSE" branches of a switch-case function</li>
+ *     <li>for parameters of "if missing/null" functions  (if-missing(), if-null(), if-missing-or-null())</li>
+ * </ul>
  */
 public class InjectTypeCastForSwitchCaseRule implements IAlgebraicRewriteRule {
 
+    private static final Set<FunctionIdentifier> IF_FUNCTIONS =
+            ImmutableSet.of(BuiltinFunctions.IF_MISSING, BuiltinFunctions.IF_NULL, BuiltinFunctions.IF_MISSING_OR_NULL);
+
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
@@ -80,35 +90,67 @@
                 rewritten = true;
             }
         }
-        if (!func.getFunctionIdentifier().equals(BuiltinFunctions.SWITCH_CASE)) {
-            return rewritten;
-        }
-        return rewriteSwitchCase(op, func, context);
-    }
-
-    // Injects casts that cast types for different "THEN" and "ELSE" branches.
-    private boolean rewriteSwitchCase(ILogicalOperator op, AbstractFunctionCallExpression func,
-            IOptimizationContext context) throws AlgebricksException {
-        IVariableTypeEnvironment env = context.getOutputTypeEnvironment(op.getInputs().get(0).getValue());
-        IAType producedType = (IAType) env.getType(func);
-        List<Mutable<ILogicalExpression>> argRefs = func.getArguments();
-        int argSize = argRefs.size();
-        boolean rewritten = false;
-        for (int argIndex = 2; argIndex < argSize; argIndex += (argIndex + 2 == argSize) ? 1 : 2) {
-            Mutable<ILogicalExpression> argRef = argRefs.get(argIndex);
-            IAType type = (IAType) env.getType(argRefs.get(argIndex).getValue());
-            if (TypeResolverUtil.needsCast(producedType, type)) {
-                ILogicalExpression argExpr = argRef.getValue();
-                // Injects a cast call to cast the data type to the produced type of the switch-case function call.
-                ScalarFunctionCallExpression castFunc =
-                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
-                                new ArrayList<>(Collections.singletonList(new MutableObject<>(argExpr))));
-                TypeCastUtils.setRequiredAndInputTypes(castFunc, producedType, type);
-                argRef.setValue(castFunc);
+        FunctionIdentifier funcId = func.getFunctionIdentifier();
+        if (funcId.equals(BuiltinFunctions.SWITCH_CASE)) {
+            if (rewriteSwitchCase(op, func, context)) {
+                rewritten = true;
+            }
+        } else if (IF_FUNCTIONS.contains(funcId)) {
+            if (rewriteFunction(op, func, context)) {
                 rewritten = true;
             }
         }
         return rewritten;
     }
 
+    // Injects casts that cast types for different "THEN" and "ELSE" branches.
+    private boolean rewriteSwitchCase(ILogicalOperator op, AbstractFunctionCallExpression func,
+            IOptimizationContext context) throws AlgebricksException {
+        IVariableTypeEnvironment env = op.computeInputTypeEnvironment(context);
+        IAType producedType = (IAType) env.getType(func);
+        List<Mutable<ILogicalExpression>> argRefs = func.getArguments();
+        int argSize = argRefs.size();
+        boolean rewritten = false;
+        for (int argIndex = 2; argIndex < argSize; argIndex += (argIndex + 2 == argSize) ? 1 : 2) {
+            Mutable<ILogicalExpression> argRef = argRefs.get(argIndex);
+            if (rewriteFunctionArgument(argRef, producedType, env)) {
+                rewritten = true;
+            }
+        }
+        return rewritten;
+    }
+
+    // Injects casts that cast types for all function parameters
+    private boolean rewriteFunction(ILogicalOperator op, AbstractFunctionCallExpression func,
+            IOptimizationContext context) throws AlgebricksException {
+        IVariableTypeEnvironment env = op.computeInputTypeEnvironment(context);
+        IAType producedType = (IAType) env.getType(func);
+        List<Mutable<ILogicalExpression>> argRefs = func.getArguments();
+        int argSize = argRefs.size();
+        boolean rewritten = false;
+        for (int argIndex = 0; argIndex < argSize; argIndex++) {
+            Mutable<ILogicalExpression> argRef = argRefs.get(argIndex);
+            if (rewriteFunctionArgument(argRef, producedType, env)) {
+                rewritten = true;
+            }
+        }
+        return rewritten;
+    }
+
+    private boolean rewriteFunctionArgument(Mutable<ILogicalExpression> argRef, IAType funcOutputType,
+            IVariableTypeEnvironment env) throws AlgebricksException {
+        ILogicalExpression argExpr = argRef.getValue();
+        IAType type = (IAType) env.getType(argExpr);
+        if (TypeResolverUtil.needsCast(funcOutputType, type)) {
+            // Injects a cast call to cast the data type to the produced type of the function call.
+            ScalarFunctionCallExpression castFunc =
+                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
+                            new ArrayList<>(Collections.singletonList(new MutableObject<>(argExpr))));
+            castFunc.setSourceLocation(argExpr.getSourceLocation());
+            TypeCastUtils.setRequiredAndInputTypes(castFunc, funcOutputType, type);
+            argRef.setValue(castFunc);
+            return true;
+        }
+        return false;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
index 748f326..8c39d12 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/InjectTypeCastForUnionRule.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule injects type casts for inputs of a UnionAllOperator if those
@@ -68,7 +69,9 @@
         // Gets the type environments for the union all operator and its child operator with the right child index.
         IVariableTypeEnvironment env = context.getOutputTypeEnvironment(op);
         Mutable<ILogicalOperator> branchOpRef = op.getInputs().get(childIndex);
-        IVariableTypeEnvironment childEnv = context.getOutputTypeEnvironment(branchOpRef.getValue());
+        ILogicalOperator branchOp = branchOpRef.getValue();
+        IVariableTypeEnvironment childEnv = context.getOutputTypeEnvironment(branchOp);
+        SourceLocation sourceLoc = branchOp.getSourceLocation();
 
         // The two lists are used for the assign operator that calls cast functions.
         List<LogicalVariable> varsToCast = new ArrayList<>();
@@ -89,9 +92,12 @@
             // Resets triple variables to new variables that bind to the results of type casting.
             triple.first = childIndex == 0 ? castedVar : triple.first;
             triple.second = childIndex > 0 ? castedVar : triple.second;
-            ScalarFunctionCallExpression castFunc = new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), new ArrayList<>(Collections
-                            .singletonList(new MutableObject<>(new VariableReferenceExpression(varToCast)))));
+            VariableReferenceExpression varToCastRef = new VariableReferenceExpression(varToCast);
+            varToCastRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression castFunc =
+                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE),
+                            new ArrayList<>(Collections.singletonList(new MutableObject<>(varToCastRef))));
+            castFunc.setSourceLocation(sourceLoc);
             TypeCastUtils.setRequiredAndInputTypes(castFunc, producedType, inputType);
 
             // Adds the variable and function expression into lists, for the assign operator.
@@ -103,7 +109,9 @@
         }
         // Injects an assign operator to perform type casts.
         AssignOperator assignOp = new AssignOperator(varsToCast, castFunctionsForLeft);
-        assignOp.getInputs().add(new MutableObject<>(branchOpRef.getValue()));
+        assignOp.setSourceLocation(sourceLoc);
+        assignOp.setExecutionMode(branchOp.getExecutionMode());
+        assignOp.getInputs().add(new MutableObject<>(branchOp));
         branchOpRef.setValue(assignOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
index c3e036e..f1b20d8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
@@ -52,6 +52,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IntroduceAutogenerateIDRule implements IAlgebraicRewriteRule {
 
@@ -66,11 +67,14 @@
             throws AlgebricksException {
 
         // match: commit OR distribute-result OR SINK - ... followed by:
-        // [insert to internal dataset with autogenerated id] - assign - project
-        // produce: insert - assign - assign* - project
+        // [insert to internal dataset with autogenerated id] - assign - assign? - project
+        // produce: insert - assign - assign? - assign* - project
         // **
-        // OR [insert to internal dataset with autogenerated id] - assign - [datasource scan]
-        // produce insert - assign - assign* - datasource scan
+        // OR [insert to internal dataset with autogenerated id] - assign - assign? - [datasource scan]
+        // produce: insert - assign - assign? - assign* - datasource scan
+        // **
+        // where assign* is the newly added assign that adds the autogenerated id
+        // and assign? is an assign that may exist when a filter is used
 
         AbstractLogicalOperator currentOp = (AbstractLogicalOperator) opRef.getValue();
         if (currentOp.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR) {
@@ -119,36 +123,62 @@
         }
         AssignOperator assignOp = (AssignOperator) parentOp;
         LogicalVariable inputRecord;
+        SourceLocation inputRecordSourceLoc;
 
-        //TODO: bug here. will not work for internal datasets with filters since the pattern becomes 
-        //[project-assign-assign-insert]
+        boolean hasFilter = false;
         AbstractLogicalOperator grandparentOp = (AbstractLogicalOperator) parentOp.getInputs().get(0).getValue();
+        AbstractLogicalOperator newAssignParentOp = grandparentOp;
+        AbstractLogicalOperator newAssignChildOp = assignOp;
         if (grandparentOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
             ProjectOperator projectOp = (ProjectOperator) grandparentOp;
             inputRecord = projectOp.getVariables().get(0);
+            inputRecordSourceLoc = projectOp.getSourceLocation();
         } else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator dssOp = (DataSourceScanOperator) grandparentOp;
             inputRecord = dssOp.getVariables().get(0);
+            inputRecordSourceLoc = dssOp.getSourceLocation();
+        } else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AbstractLogicalOperator greatgrandparentOp =
+                    (AbstractLogicalOperator) grandparentOp.getInputs().get(0).getValue();
+            if (greatgrandparentOp.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+                return false;
+            }
+            //filter case
+            ProjectOperator projectOp = (ProjectOperator) greatgrandparentOp;
+            inputRecord = projectOp.getVariables().get(0);
+            inputRecordSourceLoc = projectOp.getSourceLocation();
+            newAssignParentOp = greatgrandparentOp;
+            newAssignChildOp = grandparentOp;
+            hasFilter = true;
         } else {
             return false;
         }
 
+        SourceLocation insertOpSourceLoc = insertOp.getSourceLocation();
         List<String> pkFieldName =
                 ((InternalDatasetDetails) dds.getDataset().getDatasetDetails()).getPrimaryKey().get(0);
-        ILogicalExpression rec0 = new VariableReferenceExpression(inputRecord);
-        ILogicalExpression rec1 = createPrimaryKeyRecordExpression(pkFieldName);
-        ILogicalExpression mergedRec = createRecordMergeFunction(rec0, rec1);
+        VariableReferenceExpression rec0 = new VariableReferenceExpression(inputRecord);
+        rec0.setSourceLocation(inputRecordSourceLoc);
+        ILogicalExpression rec1 = createPrimaryKeyRecordExpression(pkFieldName, insertOpSourceLoc);
+        ILogicalExpression mergedRec = createRecordMergeFunction(rec0, rec1, insertOpSourceLoc);
         ILogicalExpression nonNullMergedRec = createNotNullFunction(mergedRec);
 
         LogicalVariable v = context.newVar();
         AssignOperator newAssign = new AssignOperator(v, new MutableObject<ILogicalExpression>(nonNullMergedRec));
-        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(grandparentOp));
-        assignOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+        newAssign.setSourceLocation(insertOpSourceLoc);
+        newAssign.getInputs().add(new MutableObject<ILogicalOperator>(newAssignParentOp));
+        newAssignChildOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+        if (hasFilter) {
+            VariableUtilities.substituteVariables(newAssignChildOp, inputRecord, v, context);
+        }
         VariableUtilities.substituteVariables(assignOp, inputRecord, v, context);
         VariableUtilities.substituteVariables(insertOp, inputRecord, v, context);
         context.computeAndSetTypeEnvironmentForOperator(newAssign);
+        if (hasFilter) {
+            context.computeAndSetTypeEnvironmentForOperator(newAssignChildOp);
+        }
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
-        context.computeAndSetTypeEnvironmentForOperator(insertOp);;
+        context.computeAndSetTypeEnvironmentForOperator(insertOp);
         for (AbstractLogicalOperator op : opStack) {
             VariableUtilities.substituteVariables(op, inputRecord, v, context);
             context.computeAndSetTypeEnvironmentForOperator(op);
@@ -162,21 +192,25 @@
         args.add(new MutableObject<ILogicalExpression>(mergedRec));
         AbstractFunctionCallExpression notNullFn =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CHECK_UNKNOWN), args);
+        notNullFn.setSourceLocation(mergedRec.getSourceLocation());
         return notNullFn;
     }
 
-    private AbstractFunctionCallExpression createPrimaryKeyRecordExpression(List<String> pkFieldName) {
+    private AbstractFunctionCallExpression createPrimaryKeyRecordExpression(List<String> pkFieldName,
+            SourceLocation sourceLoc) {
         //Create lowest level of nested uuid
         AbstractFunctionCallExpression uuidFn =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_UUID));
+        uuidFn.setSourceLocation(sourceLoc);
         List<Mutable<ILogicalExpression>> openRecordConsArgs = new ArrayList<>();
-        Mutable<ILogicalExpression> pkFieldNameExpression = new MutableObject<ILogicalExpression>(
-                new ConstantExpression(new AsterixConstantValue(new AString(pkFieldName.get(pkFieldName.size() - 1)))));
-        openRecordConsArgs.add(pkFieldNameExpression);
-        Mutable<ILogicalExpression> pkFieldValueExpression = new MutableObject<ILogicalExpression>(uuidFn);
-        openRecordConsArgs.add(pkFieldValueExpression);
+        ConstantExpression pkFieldNameExpression =
+                new ConstantExpression(new AsterixConstantValue(new AString(pkFieldName.get(pkFieldName.size() - 1))));
+        pkFieldNameExpression.setSourceLocation(sourceLoc);
+        openRecordConsArgs.add(new MutableObject<>(pkFieldNameExpression));
+        openRecordConsArgs.add(new MutableObject<>(uuidFn));
         AbstractFunctionCallExpression openRecFn = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), openRecordConsArgs);
+        openRecFn.setSourceLocation(sourceLoc);
 
         //Create higher levels
         for (int i = pkFieldName.size() - 2; i > -1; i--) {
@@ -187,17 +221,20 @@
             openRecordConsArgs.add(new MutableObject<ILogicalExpression>(openRecFn));
             openRecFn = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), openRecordConsArgs);
+            openRecFn.setSourceLocation(sourceLoc);
         }
 
         return openRecFn;
     }
 
-    private AbstractFunctionCallExpression createRecordMergeFunction(ILogicalExpression rec0, ILogicalExpression rec1) {
+    private AbstractFunctionCallExpression createRecordMergeFunction(ILogicalExpression rec0, ILogicalExpression rec1,
+            SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> recordMergeFnArgs = new ArrayList<>();
         recordMergeFnArgs.add(new MutableObject<>(rec0));
         recordMergeFnArgs.add(new MutableObject<>(rec1));
         AbstractFunctionCallExpression recordMergeFn = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.RECORD_MERGE), recordMergeFnArgs);
+        recordMergeFn.setSourceLocation(sourceLoc);
         return recordMergeFn;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
index 5644a6c..97e2174 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastForExternalFunctionRule.java
@@ -74,8 +74,8 @@
         IAType inputRecordType;
         ARecordType requiredRecordType;
         for (int iter1 = 0; iter1 < funcCallExpr.getArguments().size(); iter1++) {
-            inputRecordType = (IAType) op.computeOutputTypeEnvironment(context)
-                    .getType(funcCallExpr.getArguments().get(iter1).getValue());
+            Mutable<ILogicalExpression> argExpr = funcCallExpr.getArguments().get(iter1);
+            inputRecordType = (IAType) op.computeOutputTypeEnvironment(context).getType(argExpr.getValue());
             if (!(((ExternalScalarFunctionInfo) funcCallExpr.getFunctionInfo()).getArgumenTypes()
                     .get(iter1) instanceof ARecordType)) {
                 continue;
@@ -92,14 +92,16 @@
                 inputRecordType = ((AUnionType) inputRecordType).getActualType();
                 checkUnknown = true;
             }
-            boolean castFlag = !IntroduceDynamicTypeCastRule.compatible(requiredRecordType, inputRecordType);
+            boolean castFlag = !IntroduceDynamicTypeCastRule.compatible(requiredRecordType, inputRecordType,
+                    argExpr.getValue().getSourceLocation());
             if (castFlag || checkUnknown) {
                 AbstractFunctionCallExpression castFunc =
                         new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE));
-                castFunc.getArguments().add(funcCallExpr.getArguments().get(iter1));
+                castFunc.setSourceLocation(argExpr.getValue().getSourceLocation());
+                castFunc.getArguments().add(argExpr);
                 TypeCastUtils.setRequiredAndInputTypes(castFunc, requiredRecordType, inputRecordType);
                 funcCallExpr.getArguments().set(iter1, new MutableObject<>(castFunc));
-                changed = changed || true;
+                changed = true;
             }
         }
         return changed;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index 2695337..4044965 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -23,6 +23,8 @@
 import java.util.List;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -52,6 +54,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Dynamically cast a variable from its type to a specified required type, in a
@@ -164,7 +167,7 @@
         }
 
         /** see whether the input record type needs to be casted */
-        boolean cast = !compatible(requiredRecordType, inputRecordType);
+        boolean cast = !compatible(requiredRecordType, inputRecordType, op.getSourceLocation());
 
         if (checkUnknown) {
             recordVar = addWrapperFunction(requiredRecordType, recordVar, op, context, BuiltinFunctions.CHECK_UNKNOWN);
@@ -197,6 +200,7 @@
         for (int index = 0; index < opRefs.size(); index++) {
             Mutable<ILogicalOperator> opRef = opRefs.get(index);
             ILogicalOperator op = opRef.getValue();
+            SourceLocation sourceLoc = op.getSourceLocation();
 
             /** get produced vars */
             List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
@@ -209,13 +213,17 @@
                     IAType actualType = (IAType) env.getVarType(var);
                     AbstractFunctionCallExpression cast =
                             new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fd));
-                    cast.getArguments()
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    cast.setSourceLocation(sourceLoc);
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                    varRef.setSourceLocation(sourceLoc);
+                    cast.getArguments().add(new MutableObject<ILogicalExpression>(varRef));
                     /** enforce the required record type */
                     TypeCastUtils.setRequiredAndInputTypes(cast, requiredRecordType, actualType);
                     LogicalVariable newAssignVar = context.newVar();
                     AssignOperator newAssignOperator =
                             new AssignOperator(newAssignVar, new MutableObject<ILogicalExpression>(cast));
+                    newAssignOperator.setSourceLocation(sourceLoc);
+                    newAssignOperator.setExecutionMode(op.getExecutionMode());
                     newAssignOperator.getInputs().add(new MutableObject<ILogicalOperator>(op));
                     opRef.setValue(newAssignOperator);
                     context.computeAndSetTypeEnvironmentForOperator(newAssignOperator);
@@ -240,15 +248,18 @@
      *
      * @param reqType
      * @param inputType
+     * @param sourceLoc
      * @return true if compatible; false otherwise
      * @throws AlgebricksException
      */
-    public static boolean compatible(ARecordType reqType, IAType inputType) throws AlgebricksException {
+    public static boolean compatible(ARecordType reqType, IAType inputType, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (inputType.getTypeTag() == ATypeTag.ANY) {
             return false;
         }
         if (inputType.getTypeTag() != ATypeTag.OBJECT) {
-            throw new AlgebricksException("The input type " + inputType + " is not a valid record type!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "The input type " + inputType + " is not a valid record type!");
         }
         ARecordType inputRecType = (ARecordType) inputType;
         if (reqType.isOpen() != inputRecType.isOpen()) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
index 0fa4d30..f4d8419 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -63,6 +63,7 @@
 
         if (sameDataset) {
             MaterializeOperator materializeOperator = new MaterializeOperator();
+            materializeOperator.setSourceLocation(op.getSourceLocation());
             MaterializePOperator materializePOperator = new MaterializePOperator(true);
             materializeOperator.setPhysicalOperator(materializePOperator);
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
index c41601b..4a75cb3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
@@ -20,7 +20,6 @@
 
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.FeedDataSource;
-import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -28,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -67,12 +65,13 @@
         }
 
         ExchangeOperator exchangeOp = new ExchangeOperator();
+        exchangeOp.setSourceLocation(op.getSourceLocation());
         INodeDomain runtimeDomain = feedDataSource.getComputationNodeDomain();
 
         exchangeOp.setPhysicalOperator(new RandomPartitionExchangePOperator(runtimeDomain));
         op.getInputs().get(0).setValue(exchangeOp);
         exchangeOp.getInputs().add(new MutableObject<ILogicalOperator>(scanOp));
-        ExecutionMode em = ((AbstractLogicalOperator) scanOp).getExecutionMode();
+        ExecutionMode em = scanOp.getExecutionMode();
         exchangeOp.setExecutionMode(em);
         exchangeOp.computeDeliveredPhysicalProperties(context);
         context.computeAndSetTypeEnvironmentForOperator(exchangeOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 6d53c13..e123715 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -27,6 +27,8 @@
 import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceIndex;
@@ -73,6 +75,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule matches the pattern:
@@ -115,6 +118,7 @@
                 primaryIndexModificationOp.getAdditionalNonFilteringExpressions();
         LogicalVariable newRecordVar;
         LogicalVariable newMetaVar = null;
+        SourceLocation sourceLoc = primaryIndexModificationOp.getSourceLocation();
 
         /**
          * inputOp is the assign operator which extracts primary keys from the input
@@ -125,7 +129,7 @@
         newRecordVar = getRecordVar(context, inputOp, newRecordExpr, 0);
         if (newMetaExprs != null && !newMetaExprs.isEmpty()) {
             if (newMetaExprs.size() > 1) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Number of meta records can't be more than 1. Number of meta records found = "
                                 + newMetaExprs.size());
             }
@@ -146,7 +150,8 @@
         String datasetName = datasetSource.getId().getDatasourceName();
         Dataset dataset = mp.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                    dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
@@ -156,7 +161,7 @@
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = mp.findType(dataset.getItemTypeDataverseName(), itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.OBJECT) {
-            throw new AlgebricksException("Only record types can be indexed.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
         // meta type
@@ -195,8 +200,9 @@
                     .getAdditionalFilteringExpressions()) {
                 filteringExpression.getValue().getUsedVariables(filteringVars);
                 for (LogicalVariable var : filteringVars) {
-                    filteringExpressions
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                    varRef.setSourceLocation(filteringExpression.getValue().getSourceLocation());
+                    filteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                 }
             }
         }
@@ -207,6 +213,7 @@
             // Split the logical plan into "each secondary index update branch"
             // to replicate each <PK,OBJECT> pair.
             replicateOp = new ReplicateOperator(secondaryIndexTotalCnt);
+            replicateOp.setSourceLocation(sourceLoc);
             replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
             context.computeAndSetTypeEnvironmentForOperator(replicateOp);
@@ -270,10 +277,14 @@
                         secondaryKeyFields.get(i), secondaryKeyTypes.get(i).getTypeTag());
                 LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
                 secondaryKeyVars.add(skVar);
-                secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(skVar)));
+                VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
+                skVarRef.setSourceLocation(sourceLoc);
+                secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
                 if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
-                    beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId))));
+                    VariableReferenceExpression varRef =
+                            new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
+                    varRef.setSourceLocation(sourceLoc);
+                    beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                 }
             }
 
@@ -283,7 +294,7 @@
                 Mutable<ILogicalExpression> filterExpression =
                         (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
                                 : createFilterExpression(secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
-                                        index.isOverridingKeyFieldTypes());
+                                        index.isOverridingKeyFieldTypes(), sourceLoc);
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
 
                 // Introduce the TokenizeOperator only when doing bulk-load,
@@ -306,8 +317,9 @@
                     List<Mutable<ILogicalExpression>> tokenizeKeyExprs = new ArrayList<>();
                     LogicalVariable tokenVar = context.newVar();
                     tokenizeKeyVars.add(tokenVar);
-                    tokenizeKeyExprs
-                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(tokenVar)));
+                    VariableReferenceExpression tokenVarRef = new VariableReferenceExpression(tokenVar);
+                    tokenVarRef.setSourceLocation(sourceLoc);
+                    tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(tokenVarRef));
 
                     // Check the field type of the secondary key.
                     IAType secondaryKeyType;
@@ -325,8 +337,9 @@
                     if (isPartitioned) {
                         LogicalVariable lengthVar = context.newVar();
                         tokenizeKeyVars.add(lengthVar);
-                        tokenizeKeyExprs
-                                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lengthVar)));
+                        VariableReferenceExpression lengthVarRef = new VariableReferenceExpression(lengthVar);
+                        lengthVarRef.setSourceLocation(sourceLoc);
+                        tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(lengthVarRef));
                         varTypes.add(BuiltinType.SHORTWITHOUTTYPEINFO);
                     }
 
@@ -335,6 +348,7 @@
                             primaryIndexModificationOp.getPrimaryKeyExpressions(), secondaryExpressions,
                             tokenizeKeyVars, filterExpression, primaryIndexModificationOp.getOperation(),
                             primaryIndexModificationOp.isBulkload(), isPartitioned, varTypes);
+                    tokenUpdate.setSourceLocation(sourceLoc);
                     tokenUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
                     context.computeAndSetTypeEnvironmentForOperator(tokenUpdate);
                     replicateOutput = tokenUpdate;
@@ -343,6 +357,7 @@
                             primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
                             primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                     : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                    indexUpdate.setSourceLocation(sourceLoc);
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
                 } else {
@@ -353,15 +368,18 @@
                             primaryIndexModificationOp.isBulkload(),
                             primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                     : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                    indexUpdate.setSourceLocation(sourceLoc);
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     replicateOutput = indexUpdate;
                     // We add the necessary expressions for upsert
                     if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
                         indexUpdate.setBeforeOpSecondaryKeyExprs(beforeOpSecondaryExpressions);
                         if (filteringFields != null) {
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar());
+                            varRef.setSourceLocation(sourceLoc);
                             indexUpdate.setBeforeOpAdditionalFilteringExpression(
-                                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
-                                            primaryIndexModificationOp.getBeforeOpFilterVar())));
+                                    new MutableObject<ILogicalExpression>(varRef));
                         }
                     }
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
@@ -383,8 +401,11 @@
                     keyVarList.add(keyVar);
                     AbstractFunctionCallExpression createMBR =
                             new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
-                    createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
-                            new VariableReferenceExpression(secondaryKeyVars.get(0))));
+                    createMBR.setSourceLocation(sourceLoc);
+                    VariableReferenceExpression secondaryKeyVarRef =
+                            new VariableReferenceExpression(secondaryKeyVars.get(0));
+                    secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                    createMBR.getArguments().add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(dimension)))));
                     createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
@@ -393,18 +414,22 @@
                 }
                 secondaryExpressions.clear();
                 for (LogicalVariable secondaryKeyVar : keyVarList) {
-                    secondaryExpressions.add(
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
+                    VariableReferenceExpression secondaryKeyVarRef = new VariableReferenceExpression(secondaryKeyVar);
+                    secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                    secondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                 }
                 if (isPointMBR && isBulkload) {
                     //for PointMBR optimization: see SecondaryRTreeOperationsHelper.buildLoadingJobSpec() and
                     //createFieldPermutationForBulkLoadOp(int) for more details.
                     for (LogicalVariable secondaryKeyVar : keyVarList) {
-                        secondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(secondaryKeyVar)));
+                        VariableReferenceExpression secondaryKeyVarRef =
+                                new VariableReferenceExpression(secondaryKeyVar);
+                        secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                        secondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     }
                 }
                 AssignOperator assignCoordinates = new AssignOperator(keyVarList, keyExprList);
+                assignCoordinates.setSourceLocation(sourceLoc);
                 assignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
                 context.computeAndSetTypeEnvironmentForOperator(assignCoordinates);
                 replicateOutput = assignCoordinates;
@@ -420,6 +445,7 @@
                         originalKeyVarList.add(keyVar);
                         AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                                 FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
+                        createMBR.setSourceLocation(sourceLoc);
                         createMBR.getArguments().add(beforeOpSecondaryExpressions.get(0));
                         createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                                 new ConstantExpression(new AsterixConstantValue(new AInt32(dimension)))));
@@ -429,10 +455,13 @@
                     }
                     beforeOpSecondaryExpressions.clear();
                     for (LogicalVariable secondaryKeyVar : originalKeyVarList) {
-                        beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(secondaryKeyVar)));
+                        VariableReferenceExpression secondaryKeyVarRef =
+                                new VariableReferenceExpression(secondaryKeyVar);
+                        secondaryKeyVarRef.setSourceLocation(sourceLoc);
+                        beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
                     }
                     originalAssignCoordinates = new AssignOperator(originalKeyVarList, originalKeyExprList);
+                    originalAssignCoordinates.setSourceLocation(sourceLoc);
                     originalAssignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
                     context.computeAndSetTypeEnvironmentForOperator(originalAssignCoordinates);
                 } else {
@@ -440,7 +469,7 @@
                     // nullable.
                     boolean forceFilter = keyPairType.second;
                     filterExpression = createFilterExpression(keyVarList,
-                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
+                            context.getOutputTypeEnvironment(assignCoordinates), forceFilter, sourceLoc);
                 }
                 DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
                 indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
@@ -448,12 +477,16 @@
                         primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
                         primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
                                 : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+                indexUpdate.setSourceLocation(sourceLoc);
                 indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                 if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
                     // set before op secondary key expressions
                     if (filteringFields != null) {
-                        indexUpdate.setBeforeOpAdditionalFilteringExpression(new MutableObject<ILogicalExpression>(
-                                new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar())));
+                        VariableReferenceExpression varRef =
+                                new VariableReferenceExpression(primaryIndexModificationOp.getBeforeOpFilterVar());
+                        varRef.setSourceLocation(sourceLoc);
+                        indexUpdate.setBeforeOpAdditionalFilteringExpression(
+                                new MutableObject<ILogicalExpression>(varRef));
                     }
                     // set filtering expressions
                     indexUpdate.setBeforeOpSecondaryKeyExprs(beforeOpSecondaryExpressions);
@@ -463,6 +496,12 @@
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
                 }
             }
+
+            if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+                indexUpdate.setUpsertIndicatorExpr(new MutableObject<>(
+                        new VariableReferenceExpression(primaryIndexModificationOp.getUpsertIndicatorVar())));
+            }
+
             context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
             if (!primaryIndexModificationOp.isBulkload() || secondaryIndexTotalCnt == 1) {
                 currentTop = indexUpdate;
@@ -546,6 +585,7 @@
             boolean afterOp) throws AlgebricksException {
         List<LogicalVariable> vars = new ArrayList<>();
         List<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
+        SourceLocation sourceLoc = currentTop.getSourceLocation();
         for (Index index : indexes) {
             if (index.isPrimaryIndex()) {
                 continue;
@@ -566,7 +606,8 @@
                         ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordVar : metaVar : recordVar;
                 LogicalVariable fieldVar = context.newVar();
                 // create record variable ref
-                Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(sourceVar));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVar);
+                varRef.setSourceLocation(sourceLoc);
                 IAType fieldType = sourceType.getSubFieldType(indexFieldId.fieldName);
                 AbstractFunctionCallExpression theFieldAccessFunc;
                 if (fieldType == null) {
@@ -574,11 +615,12 @@
                     // make handling of records with incorrect value type for this field easier and cleaner
                     context.addNotToBeInlinedVar(fieldVar);
                     // create field access
-                    AbstractFunctionCallExpression fieldAccessFunc =
-                            getOpenOrNestedFieldAccessFunction(varRef, indexFieldId.fieldName);
+                    AbstractFunctionCallExpression fieldAccessFunc = getOpenOrNestedFieldAccessFunction(
+                            new MutableObject<>(varRef), indexFieldId.fieldName, sourceLoc);
                     // create cast
                     theFieldAccessFunc = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(
                             index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX));
+                    theFieldAccessFunc.setSourceLocation(sourceLoc);
                     // The first argument is the field
                     theFieldAccessFunc.getArguments().add(new MutableObject<ILogicalExpression>(fieldAccessFunc));
                     TypeCastUtils.setRequiredAndInputTypes(theFieldAccessFunc, skTypes.get(i), BuiltinType.ANY);
@@ -587,9 +629,10 @@
                     int pos = indexFieldId.fieldName.size() > 1 ? -1
                             : sourceType.getFieldIndex(indexFieldId.fieldName.get(0));
                     // Field not found --> This is either an open field or a nested field. it can't be accessed by index
-                    theFieldAccessFunc =
-                            (pos == -1) ? getOpenOrNestedFieldAccessFunction(varRef, indexFieldId.fieldName)
-                                    : getClosedFieldAccessFunction(varRef, pos);
+                    theFieldAccessFunc = (pos == -1)
+                            ? getOpenOrNestedFieldAccessFunction(new MutableObject<>(varRef), indexFieldId.fieldName,
+                                    sourceLoc)
+                            : getClosedFieldAccessFunction(new MutableObject<>(varRef), pos, sourceLoc);
                 }
                 vars.add(fieldVar);
                 exprs.add(new MutableObject<ILogicalExpression>(theFieldAccessFunc));
@@ -598,6 +641,7 @@
         }
         // AssignOperator assigns secondary keys to their vars
         AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
+        castedFieldAssignOperator.setSourceLocation(sourceLoc);
         return introduceNewOp(context, currentTop, castedFieldAssignOperator, afterOp);
     }
 
@@ -618,15 +662,17 @@
     }
 
     private static AbstractFunctionCallExpression getClosedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            int position) {
+            int position, SourceLocation sourceLoc) {
         Mutable<ILogicalExpression> indexRef =
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(position))));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                varRef, indexRef);
+        ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
+        fnExpr.setSourceLocation(sourceLoc);
+        return fnExpr;
     }
 
     private static AbstractFunctionCallExpression getOpenOrNestedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
-            List<String> fields) {
+            List<String> fields, SourceLocation sourceLoc) {
         ScalarFunctionCallExpression func;
         if (fields.size() > 1) {
             IAObject fieldList = stringListToAOrderedList(fields);
@@ -641,6 +687,7 @@
             func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
                     varRef, fieldRef);
         }
+        func.setSourceLocation(sourceLoc);
         return func;
     }
 
@@ -657,7 +704,8 @@
     }
 
     private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
-            IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, boolean forceFilter, SourceLocation sourceLoc)
+            throws AlgebricksException {
         List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
         // Add 'is not null' to all nullable secondary index keys as a filtering
         // condition.
@@ -666,12 +714,16 @@
             if (!NonTaggedFormatUtil.isOptional(secondaryKeyType) && !forceFilter) {
                 continue;
             }
+            VariableReferenceExpression secondaryKeyVarRef = new VariableReferenceExpression(secondaryKeyVar);
+            secondaryKeyVarRef.setSourceLocation(sourceLoc);
             ScalarFunctionCallExpression isUnknownFuncExpr =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_UNKNOWN),
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
+                            new MutableObject<ILogicalExpression>(secondaryKeyVarRef));
+            isUnknownFuncExpr.setSourceLocation(sourceLoc);
             ScalarFunctionCallExpression notFuncExpr =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
                             new MutableObject<ILogicalExpression>(isUnknownFuncExpr));
+            notFuncExpr.setSourceLocation(sourceLoc);
             filterExpressions.add(new MutableObject<ILogicalExpression>(notFuncExpr));
         }
         // No nullable secondary keys.
@@ -681,8 +733,10 @@
         Mutable<ILogicalExpression> filterExpression;
         if (filterExpressions.size() > 1) {
             // Create a conjunctive condition.
-            filterExpression = new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), filterExpressions));
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), filterExpressions);
+            andExpr.setSourceLocation(sourceLoc);
+            filterExpression = new MutableObject<>(andExpr);
         } else {
             filterExpression = filterExpressions.get(0);
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
index c5cb6ae..926df0e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
@@ -64,12 +64,14 @@
         LogicalVariable v = context.newVar();
         AssignOperator assignOperator =
                 new AssignOperator(v, new MutableObject<ILogicalExpression>(selectOperator.getCondition().getValue()));
-
+        assignOperator.setSourceLocation(selectOperator.getSourceLocation());
         //set the input of the new assign-operator to the input of the select-operator.
         assignOperator.getInputs().add(childOfSelect);
 
         //set the result value of the assign-operator to the condition of the select-operator
-        selectOperator.getCondition().setValue(new VariableReferenceExpression(v));//scalarFunctionCallExpression);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(v);
+        varRef.setSourceLocation(selectOperator.getSourceLocation());
+        selectOperator.getCondition().setValue(varRef);//scalarFunctionCallExpression);
         selectOperator.getInputs().set(0, new MutableObject<ILogicalOperator>(assignOperator));
 
         context.computeAndSetTypeEnvironmentForOperator(assignOperator);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
index 561afb9..02d8f14 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceUnnestForCollectionToSequenceRule.java
@@ -85,11 +85,13 @@
         }
         /** change the assign operator to an unnest operator */
         LogicalVariable var = assign.getVariables().get(0);
+        UnnestingFunctionCallExpression scanCollExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(argExpr));
+        scanCollExpr.setSourceLocation(func.getSourceLocation());
         @SuppressWarnings("unchecked")
-        UnnestOperator unnest = new UnnestOperator(var,
-                new MutableObject<ILogicalExpression>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
-                        new MutableObject<ILogicalExpression>(argExpr))));
+        UnnestOperator unnest = new UnnestOperator(var, new MutableObject<ILogicalExpression>(scanCollExpr));
+        unnest.setSourceLocation(assign.getSourceLocation());
         unnest.getInputs().addAll(assign.getInputs());
         opRef.setValue(unnest);
         context.computeAndSetTypeEnvironmentForOperator(unnest);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
index 2fe6460..7f23775 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ListifyUnnestingFunctionRule.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * In principle, an unnest operator produces a sequence of items from a collection.
@@ -104,25 +105,35 @@
             return false;
         }
 
+        SourceLocation sourceLoc = func.getSourceLocation();
+
         // Generates the listified collection in a subplan.
         SubplanOperator subplanOperator = new SubplanOperator();
+        subplanOperator.setSourceLocation(sourceLoc);
         // Creates a nested tuple source operator.
         NestedTupleSourceOperator ntsOperator = new NestedTupleSourceOperator(new MutableObject<>(subplanOperator));
-
+        ntsOperator.setSourceLocation(sourceLoc);
         // Unnests the dataset.
         LogicalVariable unnestVar = context.newVar();
-        ILogicalExpression unnestExpr = new UnnestingFunctionCallExpression(functionInfo, func.getArguments());
+        UnnestingFunctionCallExpression unnestExpr =
+                new UnnestingFunctionCallExpression(functionInfo, func.getArguments());
+        unnestExpr.setSourceLocation(sourceLoc);
         UnnestOperator unnestOperator = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
+        unnestOperator.setSourceLocation(sourceLoc);
         unnestOperator.getInputs().add(new MutableObject<>(ntsOperator));
 
         // Listify the dataset into one collection.
         LogicalVariable aggVar = context.newVar();
-        Mutable<ILogicalExpression> aggArgExprRef = new MutableObject<>(new VariableReferenceExpression(unnestVar));
-        ILogicalExpression aggExpr =
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> aggArgExprRef = new MutableObject<>(unnestVarRef);
+        AggregateFunctionCallExpression aggExpr =
                 new AggregateFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false,
                         new ArrayList<>(Collections.singletonList(aggArgExprRef)));
+        aggExpr.setSourceLocation(sourceLoc);
         AggregateOperator aggregateOperator = new AggregateOperator(new ArrayList<>(Collections.singletonList(aggVar)),
                 new ArrayList<>(Collections.singletonList(new MutableObject<>(aggExpr))));
+        aggregateOperator.setSourceLocation(sourceLoc);
         aggregateOperator.getInputs().add(new MutableObject<>(unnestOperator));
 
         // Adds the aggregate operator as the root of the subplan.
@@ -136,7 +147,9 @@
         // asterixdb/asterix-app/src/test/resources/runtimets/results/list/query-ASTERIXDB-159-3
         subplanOperator.getInputs().add(op.getInputs().get(0));
         op.getInputs().set(0, new MutableObject<>(subplanOperator));
-        exprRef.setValue(new VariableReferenceExpression(aggVar));
+        VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+        aggVarRef.setSourceLocation(sourceLoc);
+        exprRef.setValue(aggVarRef);
 
         // Computes type environments for new operators.
         context.computeAndSetTypeEnvironmentForOperator(ntsOperator);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index d6581a2..32fa744 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -25,6 +25,8 @@
 import java.util.List;
 
 import org.apache.asterix.algebra.base.OperatorAnnotation;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -140,6 +142,7 @@
                     // create an assign
                     LogicalVariable v = context.newVar();
                     AssignOperator a2 = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+                    a2.setSourceLocation(expr.getSourceLocation());
                     pushFieldAssign(a2, topOp, context);
                     context.computeAndSetTypeEnvironmentForOperator(a2);
                     ILogicalExpression arg = f.getArguments().get(0).getValue();
@@ -154,7 +157,9 @@
                             context.addPrimaryKey(pk);
                         }
                     }
-                    exprRef.setValue(new VariableReferenceExpression(v));
+                    VariableReferenceExpression varRef = new VariableReferenceExpression(v);
+                    varRef.setSourceLocation(expr.getSourceLocation());
+                    exprRef.setValue(varRef);
                     return true;
                 } else {
                     boolean pushed = false;
@@ -217,7 +222,7 @@
                         }
                     }
                 }
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, a2.getSourceLocation(),
                         "Field access " + getFirstExpr(a2) + " does not correspond to any input of operator " + topOp);
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index 13ff0ee..cd99c28 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.IMutationDataSource;
@@ -45,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This rule rewrites all meta() function calls in a query plan
@@ -204,7 +207,8 @@
         }
         // The user query provides more than one parameter for the meta function.
         if (argRefs.size() > 1) {
-            throw new AlgebricksException("The meta function can at most have one argument!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                    "The meta function can at most have one argument!");
         }
 
         // The user query provides exact one parameter for the meta function.
@@ -218,16 +222,21 @@
             if (!dataVar.equals(argVar)) {
                 return false;
             }
-            exprRef.setValue(new VariableReferenceExpression(metaVar));
+            VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+            metaVarRef.setSourceLocation(expr.getSourceLocation());
+            exprRef.setValue(metaVarRef);
             return true;
         }
 
         // The user query provides zero parameter for the meta function.
         if (variableRequired) {
-            throw new AlgebricksException("Cannot resolve to ambiguity on the meta function call --"
-                    + " there are more than one dataset choices!");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                    "Cannot resolve to ambiguity on the meta function call --"
+                            + " there are more than one dataset choices!");
         }
-        exprRef.setValue(new VariableReferenceExpression(metaVar));
+        VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+        metaVarRef.setSourceLocation(expr.getSourceLocation());
+        exprRef.setValue(metaVarRef);
         return true;
     }
 }
@@ -268,6 +277,7 @@
         if (!funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.META_KEY)) {
             return false;
         }
+        SourceLocation sourceLoc = expr.getSourceLocation();
         // Get arguments
         // first argument : Resource key
         // second argument: field
@@ -286,13 +296,18 @@
                 functionIdentifier = BuiltinFunctions.FIELD_ACCESS_BY_NAME;
                 break;
             default:
-                throw new AlgebricksException("Unsupported field name type " + fieldNameType.getTypeTag());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported field name type " + fieldNameType.getTypeTag());
         }
         IFunctionInfo finfoAccess = FunctionUtil.getFunctionInfo(functionIdentifier);
         ArrayList<Mutable<ILogicalExpression>> argExprs = new ArrayList<>(2);
-        argExprs.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
+        VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+        metaVarRef.setSourceLocation(sourceLoc);
+        argExprs.add(new MutableObject<>(metaVarRef));
         argExprs.add(new MutableObject<>(fieldNameExpression));
-        exprRef.setValue(new ScalarFunctionCallExpression(finfoAccess, argExprs));
+        ScalarFunctionCallExpression fAccessExpr = new ScalarFunctionCallExpression(finfoAccess, argExprs);
+        fAccessExpr.setSourceLocation(sourceLoc);
+        exprRef.setValue(fAccessExpr);
         return true;
     }
 }
@@ -321,7 +336,9 @@
         // Function is meta key access
         for (int i = 0; i < metaKeyAccessExpressions.size(); i++) {
             if (metaKeyAccessExpressions.get(i).equals(funcExpr)) {
-                exprRef.setValue(new VariableReferenceExpression(keyVars.get(i)));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(keyVars.get(i));
+                varRef.setSourceLocation(expr.getSourceLocation());
+                exprRef.setValue(varRef);
                 return true;
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
index 233171e..f51358b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullPositionalVariableFromUnnestRule.java
@@ -63,8 +63,10 @@
         ArrayList<Mutable<ILogicalExpression>> rOpExprList = new ArrayList<Mutable<ILogicalExpression>>();
         StatefulFunctionCallExpression fce = new StatefulFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+        fce.setSourceLocation(op.getSourceLocation());
         rOpExprList.add(new MutableObject<ILogicalExpression>(fce));
         RunningAggregateOperator rOp = new RunningAggregateOperator(rOpVars, rOpExprList);
+        rOp.setSourceLocation(unnest.getSourceLocation());
         rOp.setExecutionMode(unnest.getExecutionMode());
         RunningAggregatePOperator rPop = new RunningAggregatePOperator();
         rOp.setPhysicalOperator(rPop);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
index cf5088b..82c0d4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -221,12 +221,15 @@
             aggArgs.add(aggOpExpr.getArguments().get(0));
             AggregateFunctionCallExpression aggFuncExpr =
                     BuiltinFunctions.makeAggregateFunctionExpression(aggFuncIdent, aggArgs);
+            aggFuncExpr.setSourceLocation(assignFuncExpr.getSourceLocation());
             LogicalVariable newVar = context.newVar();
             aggOp.getVariables().add(newVar);
             aggOp.getExpressions().add(new MutableObject<ILogicalExpression>(aggFuncExpr));
 
             // The assign now just "renames" the variable to make sure the upstream plan still works.
-            srcAssignExprRef.setValue(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(assignFuncExpr.getSourceLocation());
+            srcAssignExprRef.setValue(newVarRef);
         }
 
         context.computeAndSetTypeEnvironmentForOperator(aggOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
index ccf7ccb..3c97ec8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
@@ -285,8 +285,10 @@
                                 LogicalVariable newVar = context.newVar();
                                 AggregateFunctionCallExpression aggFun =
                                         BuiltinFunctions.makeAggregateFunctionExpression(fi, fce.getArguments());
+                                aggFun.setSourceLocation(expr.getSourceLocation());
                                 rewriteAggregateInNestedSubplan(argVar, nspOp, aggFun, newVar, context);
-                                ILogicalExpression newVarExpr = new VariableReferenceExpression(newVar);
+                                VariableReferenceExpression newVarExpr = new VariableReferenceExpression(newVar);
+                                newVarExpr.setSourceLocation(expr.getSourceLocation());
                                 aggregateExprToVarExpr.put(expr, newVarExpr);
                                 return new Pair<>(Boolean.TRUE, newVarExpr);
                             } else {
@@ -328,6 +330,7 @@
                             (AbstractFunctionCallExpression) aggOp.getExpressions().get(i).getValue();
                     AggregateFunctionCallExpression newAggFun = BuiltinFunctions
                             .makeAggregateFunctionExpression(aggFun.getFunctionIdentifier(), new ArrayList<>());
+                    newAggFun.setSourceLocation(oldAggExpr.getSourceLocation());
                     for (Mutable<ILogicalExpression> arg : oldAggExpr.getArguments()) {
                         ILogicalExpression cloned = arg.getValue().cloneExpression();
                         newAggFun.getArguments().add(new MutableObject<>(cloned));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index 769894d..b779c83 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -25,6 +25,8 @@
 
 import org.apache.asterix.algebra.base.OperatorAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -125,7 +127,8 @@
 
         Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
         if (dataset == null) {
-            throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
+                    asid.getDatasourceName(), asid.getDataverseName());
         }
         if (dataset.getDatasetType() != DatasetType.INTERNAL) {
             return false;
@@ -228,8 +231,10 @@
                 for (Pair<LogicalVariable, LogicalVariable> m : varMappings) {
                     LogicalVariable v2 = context.newVar();
                     LogicalVariable oldVar = access.getVariables().get(0);
+                    VariableReferenceExpression v2Ref = new VariableReferenceExpression(v2);
+                    v2Ref.setSourceLocation(g.getSourceLocation());
                     g.getDecorList().add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(oldVar,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v2))));
+                            new MutableObject<ILogicalExpression>(v2Ref)));
                     changed = true;
                     access.getVariables().set(0, v2);
                     VariableUtilities.substituteVariables(access, m.first, m.second, context);
@@ -275,8 +280,9 @@
                     }
                 }
             }
-            throw new AlgebricksException("Field access " + access.getExpressions().get(0).getValue()
-                    + " does not correspond to any input of operator " + op2);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, access.getSourceLocation(),
+                    "Field access " + access.getExpressions().get(0).getValue()
+                            + " does not correspond to any input of operator " + op2);
         } else {
             // Check if the accessed field is not one of the partitioning key
             // fields. If yes, we can equate the two variables.
@@ -294,7 +300,8 @@
                 MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
                 Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
                 if (dataset == null) {
-                    throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, scan.getSourceLocation(),
+                            asid.getDatasourceName(), asid.getDataverseName());
                 }
                 if (dataset.getDatasetType() != DatasetType.INTERNAL) {
                     setAsFinal(access, context, finalAnnot);
@@ -319,7 +326,9 @@
                             return false;
                         }
                         LogicalVariable keyVar = scan.getVariables().get(p);
-                        access.getExpressions().get(0).setValue(new VariableReferenceExpression(keyVar));
+                        VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                        keyVarRef.setSourceLocation(varRef.getSourceLocation());
+                        access.getExpressions().get(0).setValue(keyVarRef);
                         return true;
 
                     }
@@ -368,8 +377,11 @@
             return false;
         }
         AssignOperator a2 = (AssignOperator) op2;
-        if (getFirstExpr(access).equals(getFirstExpr(a2))) {
-            access.getExpressions().get(0).setValue(new VariableReferenceExpression(a2.getVariables().get(0)));
+        ILogicalExpression accessExpr0 = getFirstExpr(access);
+        if (accessExpr0.equals(getFirstExpr(a2))) {
+            VariableReferenceExpression varRef = new VariableReferenceExpression(a2.getVariables().get(0));
+            varRef.setSourceLocation(accessExpr0.getSourceLocation());
+            access.getExpressions().get(0).setValue(varRef);
             return true;
         } else {
             return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
index 4c0247f..e64889d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
@@ -18,12 +18,15 @@
  */
 package org.apache.asterix.optimizer.rules;
 
-import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -89,50 +92,20 @@
         PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
         LimitOperator limitOp = (LimitOperator) opRef.getValue();
         OrderOperator orderOp = (OrderOperator) opRef2.getValue();
-        long topK = -1;
 
         // We don't push-down LIMIT into in-memory sort.
         if (orderOp.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT) {
             return false;
         }
 
-        // Get the LIMIT constant
-        if (limitOp.getMaxObjects().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            // Currently, we support LIMIT with a constant value.
-            topK = AccessMethodUtils.getInt64Constant(limitOp.getMaxObjects());
-            // If topK is huge, there is no reason to use topK sort module
-            // since the original external sort's performance might be better.
-            if (topK > Integer.MAX_VALUE) {
-                return false;
-            }
-            if (topK < 0) {
-                topK = 0;
-            }
-        } else {
+        Integer topK = getOutputLimit(limitOp);
+        if (topK == null) {
             return false;
         }
 
-        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
-        // This is because we can't apply offset to the external sort.
-        // Final topK will be applied through LIMIT.
-        if (limitOp.getOffset().getValue() != null) {
-            if (limitOp.getOffset().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                long offset = AccessMethodUtils.getInt64Constant(limitOp.getOffset());
-                if (offset < 0) {
-                    offset = 0;
-                }
-                // Check the overflow case.
-                if (offset >= Integer.MAX_VALUE - topK) {
-                    return false;
-                }
-                topK += offset;
-            } else {
-                return false;
-            }
-        }
-
         // Create the new ORDER operator, set the topK value, and replace the current one.
-        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), (int) topK);
+        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), topK);
+        newOrderOp.setSourceLocation(orderOp.getSourceLocation());
         newOrderOp.setPhysicalOperator(
                 new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort(), newOrderOp.getTopK()));
         newOrderOp.getInputs().addAll(orderOp.getInputs());
@@ -145,4 +118,38 @@
         return true;
     }
 
+    static Integer getOutputLimit(LimitOperator limitOp) {
+        // Currently, we support LIMIT with a constant value.
+        ILogicalExpression maxObjectsExpr = limitOp.getMaxObjects().getValue();
+        IAObject maxObjectsValue = ConstantExpressionUtil.getConstantIaObject(maxObjectsExpr, ATypeTag.INTEGER);
+        if (maxObjectsValue == null) {
+            return null;
+        }
+        int topK = ((AInt32) maxObjectsValue).getIntegerValue();
+        if (topK < 0) {
+            topK = 0;
+        }
+
+        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
+        // This is because we can't apply offset to the external sort.
+        // Final topK will be applied through LIMIT.
+        ILogicalExpression offsetExpr = limitOp.getOffset().getValue();
+        if (offsetExpr != null) {
+            IAObject offsetValue = ConstantExpressionUtil.getConstantIaObject(offsetExpr, ATypeTag.INTEGER);
+            if (offsetValue == null) {
+                return null;
+            }
+            int offset = ((AInt32) offsetValue).getIntegerValue();
+            if (offset < 0) {
+                offset = 0;
+            }
+            // Check the overflow case.
+            if (offset >= Integer.MAX_VALUE - topK) {
+                return null;
+            }
+            topK += offset;
+        }
+
+        return topK;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
new file mode 100644
index 0000000..b7eb8b3
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
@@ -0,0 +1,231 @@
+/*
+ * 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.asterix.optimizer.rules;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
+
+/**
+ * Pattern:
+ * SCAN or UNNEST_MAP -> ((ASSIGN)* -> (SELECT))? -> (EXCHANGE)? -> LIMIT
+ * We push both SELECT condition and LIMIT to SCAN or UNNEST_MAP
+ *
+ */
+public class PushLimitIntoPrimarySearchRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op = opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
+            return false;
+        }
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        context.addToDontApplySet(this, op);
+
+        Integer outputLimit = PushLimitIntoOrderByRule.getOutputLimit((LimitOperator) op);
+        if (outputLimit == null) {
+            // we cannot push if limit is not constant
+            return false;
+        }
+
+        Mutable<ILogicalOperator> childOp = op.getInputs().get(0);
+        if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+            childOp = childOp.getValue().getInputs().get(0);
+        }
+        boolean changed;
+        if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.SELECT) {
+            changed = rewriteSelect(childOp, outputLimit, context);
+        } else {
+            changed = setLimitForScanOrUnnestMap(childOp.getValue(), outputLimit);
+        }
+        if (changed) {
+            OperatorPropertiesUtil.typeOpRec(opRef, context);
+        }
+        return changed;
+    }
+
+    private boolean rewriteSelect(Mutable<ILogicalOperator> op, int outputLimit, IOptimizationContext context)
+            throws AlgebricksException {
+        SelectOperator select = (SelectOperator) op.getValue();
+        ILogicalExpression selectCondition = select.getCondition().getValue();
+        Set<LogicalVariable> selectedVariables = new HashSet<>();
+        selectCondition.getUsedVariables(selectedVariables);
+
+        MutableObject<ILogicalExpression> selectConditionRef = new MutableObject<>(selectCondition.cloneExpression());
+
+        // If the select condition uses variables from assigns then inline those variables into it
+        ILogicalOperator child = select.getInputs().get(0).getValue();
+        InlineVariablesRule.InlineVariablesVisitor inlineVisitor = null;
+        Map<LogicalVariable, ILogicalExpression> varAssignRhs = null;
+        for (; child.getOperatorTag() == LogicalOperatorTag.ASSIGN; child = child.getInputs().get(0).getValue()) {
+            if (varAssignRhs == null) {
+                varAssignRhs = new HashMap<>();
+            } else {
+                varAssignRhs.clear();
+            }
+            AssignOperator assignOp = (AssignOperator) child;
+            extractInlinableVariablesFromAssign(assignOp, selectedVariables, varAssignRhs);
+            if (!varAssignRhs.isEmpty()) {
+                if (inlineVisitor == null) {
+                    inlineVisitor = new InlineVariablesRule.InlineVariablesVisitor(varAssignRhs);
+                    inlineVisitor.setContext(context);
+                    inlineVisitor.setOperator(select);
+                }
+                if (!inlineVisitor.transform(selectConditionRef)) {
+                    break;
+                }
+                selectedVariables.clear();
+                selectConditionRef.getValue().getUsedVariables(selectedVariables);
+            }
+        }
+
+        boolean changed = false;
+        switch (child.getOperatorTag()) {
+            case DATASOURCESCAN:
+                DataSourceScanOperator scan = (DataSourceScanOperator) child;
+                if (isScanPushable(scan, selectedVariables)) {
+                    scan.setSelectCondition(selectConditionRef);
+                    scan.setOutputLimit(outputLimit);
+                    changed = true;
+                }
+                break;
+            case UNNEST_MAP:
+                UnnestMapOperator unnestMap = (UnnestMapOperator) child;
+                if (isUnnestMapPushable(unnestMap, selectedVariables)) {
+                    unnestMap.setSelectCondition(selectConditionRef);
+                    unnestMap.setOutputLimit(outputLimit);
+                    changed = true;
+                }
+                break;
+        }
+
+        if (changed) {
+            // SELECT is not needed
+            op.setValue(op.getValue().getInputs().get(0).getValue());
+        }
+        return changed;
+    }
+
+    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, int outputLimit) {
+        if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+            DataSourceScanOperator scan = (DataSourceScanOperator) op;
+            if (isScanPushable(scan, Collections.emptySet())) {
+                scan.setOutputLimit(outputLimit);
+                return true;
+            }
+        } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+            UnnestMapOperator unnestMap = (UnnestMapOperator) op;
+            if (isUnnestMapPushable(unnestMap, Collections.emptySet())) {
+                unnestMap.setOutputLimit(outputLimit);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean isUnnestMapPushable(UnnestMapOperator op, Set<LogicalVariable> selectedVariables) {
+        if (op.getOutputLimit() >= 0) {
+            // already pushed
+            return false;
+        }
+        ILogicalExpression unnestExpr = op.getExpressionRef().getValue();
+        if (op.propagatesInput() || unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+        if (!f.getFunctionIdentifier().equals(BuiltinFunctions.INDEX_SEARCH)) {
+            return false;
+        }
+        AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+        jobGenParams.readFromFuncArgs(f.getArguments());
+        if (!jobGenParams.isPrimaryIndex()) {
+            return false;
+        }
+        if (!op.getScanVariables().containsAll(selectedVariables)) {
+            return false;
+        }
+        return true;
+    }
+
+    private boolean isScanPushable(DataSourceScanOperator op, Set<LogicalVariable> selectedVariables) {
+        if (op.getOutputLimit() >= 0) {
+            return false;
+        }
+        if (!op.getInputs().isEmpty()
+                && op.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            return false;
+        }
+        if (((DataSource) op.getDataSource()).getDatasourceType() != DataSource.Type.INTERNAL_DATASET) {
+            return false;
+        }
+        if (!op.getScanVariables().containsAll(selectedVariables)) {
+            return false;
+        }
+        return true;
+    }
+
+    private void extractInlinableVariablesFromAssign(AssignOperator assignOp, Set<LogicalVariable> includeVariables,
+            Map<LogicalVariable, ILogicalExpression> outVarExprs) {
+        List<LogicalVariable> vars = assignOp.getVariables();
+        List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
+        for (int i = 0, ln = vars.size(); i < ln; i++) {
+            LogicalVariable var = vars.get(i);
+            if (includeVariables.contains(var)) {
+                ILogicalExpression expr = exprs.get(i).getValue();
+                if (expr.isFunctional()) {
+                    outVarExprs.put(var, expr);
+                }
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 1d7a61a..364816b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -262,13 +262,16 @@
         List<LogicalVariable> lhs = new ArrayList<>();
         List<Mutable<ILogicalExpression>> rhs = new ArrayList<>();
         lhs.add(outerUnnest.getVariable());
-        rhs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(listifyVar)));
+        VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar);
+        listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation());
+        rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef));
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList();
         for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) {
             lhs.add(gbyPair.first);
             rhs.add(gbyPair.second);
         }
         AssignOperator assignOp = new AssignOperator(lhs, rhs);
+        assignOp.setSourceLocation(outerUnnest.getSourceLocation());
         assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator));
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         opRef.setValue(assignOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
index 7b46b39..21990c9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
@@ -37,7 +37,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -185,8 +184,7 @@
         }
         LogicalVariable aggVar = agg.getVariables().get(0);
         ILogicalExpression aggFun = agg.getExpressions().get(0).getValue();
-        if (!aggVar.equals(unnestedVar)
-                || ((AbstractLogicalExpression) aggFun).getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+        if (!aggVar.equals(unnestedVar) || aggFun.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) aggFun;
@@ -197,7 +195,7 @@
             return false;
         }
         ILogicalExpression arg0 = f.getArguments().get(0).getValue();
-        if (((AbstractLogicalExpression) arg0).getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+        if (arg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
         LogicalVariable paramVar = ((VariableReferenceExpression) arg0).getVariableReference();
@@ -205,8 +203,11 @@
         List<LogicalVariable> assgnVars = new ArrayList<>(1);
         assgnVars.add(unnest1.getVariable());
         List<Mutable<ILogicalExpression>> assgnExprs = new ArrayList<>(1);
-        assgnExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(paramVar)));
+        VariableReferenceExpression paramVarRef = new VariableReferenceExpression(paramVar);
+        paramVarRef.setSourceLocation(arg0.getSourceLocation());
+        assgnExprs.add(new MutableObject<ILogicalExpression>(paramVarRef));
         AssignOperator assign = new AssignOperator(assgnVars, assgnExprs);
+        assign.setSourceLocation(agg.getSourceLocation());
         assign.getInputs().add(agg.getInputs().get(0));
         context.computeAndSetTypeEnvironmentForOperator(assign);
         LogicalVariable posVar = unnest1.getPositionalVariable();
@@ -220,8 +221,10 @@
             List<Mutable<ILogicalExpression>> rAggExprs = new ArrayList<>(1);
             StatefulFunctionCallExpression tidFun = new StatefulFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+            tidFun.setSourceLocation(agg.getSourceLocation());
             rAggExprs.add(new MutableObject<ILogicalExpression>(tidFun));
             RunningAggregateOperator rAgg = new RunningAggregateOperator(raggVars, rAggExprs);
+            rAgg.setSourceLocation(agg.getSourceLocation());
             rAgg.getInputs().add(new MutableObject<ILogicalOperator>(assign));
             aggregateParentRef.setValue(rAgg);
             context.computeAndSetTypeEnvironmentForOperator(rAgg);
@@ -251,7 +254,7 @@
             return false;
         }
         ILogicalExpression arg0 = f.getArguments().get(0).getValue();
-        if (((AbstractLogicalExpression) arg0).getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+        if (arg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
         LogicalVariable aggInputVar = ((VariableReferenceExpression) arg0).getVariableReference();
@@ -289,6 +292,7 @@
         List<LogicalVariable> assgnVars = new ArrayList<>(1);
         assgnVars.add(aggVar);
         AssignOperator assign = new AssignOperator(assgnVars, scanFunc.getArguments());
+        assign.setSourceLocation(agg.getSourceLocation());
         assign.getInputs().add(unnest.getInputs().get(0));
         context.computeAndSetTypeEnvironmentForOperator(assign);
         opRef.setValue(assign);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 4afccb0..2dc5f2e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -25,6 +25,8 @@
 import org.apache.asterix.algebra.operators.physical.InvertedIndexPOperator;
 import org.apache.asterix.algebra.operators.physical.RTreeSearchPOperator;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -101,8 +103,7 @@
                 ILogicalPlan p0 = gby.getNestedPlans().get(0);
                 if (p0.getRoots().size() == 1) {
                     Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
-                    if (((AbstractLogicalOperator) (r0.getValue())).getOperatorTag()
-                            .equals(LogicalOperatorTag.AGGREGATE)) {
+                    if (r0.getValue().getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
                         AggregateOperator aggOp = (AggregateOperator) r0.getValue();
                         boolean serializable = true;
                         for (Mutable<ILogicalExpression> exprRef : aggOp.getExpressions()) {
@@ -131,6 +132,7 @@
                                     AggregateFunctionCallExpression serialAggExpr =
                                             BuiltinFunctions.makeSerializableAggregateFunctionExpression(
                                                     expr.getFunctionIdentifier(), expr.getArguments());
+                                    serialAggExpr.setSourceLocation(expr.getSourceLocation());
                                     if (mergeAggregationExpressionFactory.createMergeAggregation(
                                             originalVariables.get(i), serialAggExpr, context) == null) {
                                         hasIntermediateAgg = false;
@@ -157,6 +159,7 @@
                                         AggregateFunctionCallExpression serialAggExpr =
                                                 BuiltinFunctions.makeSerializableAggregateFunctionExpression(
                                                         expr.getFunctionIdentifier(), expr.getArguments());
+                                        serialAggExpr.setSourceLocation(expr.getSourceLocation());
                                         aggOp.getExpressions().get(i).setValue(serialAggExpr);
                                     }
                                     ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
@@ -184,8 +187,7 @@
                                         context.getPhysicalOptimizationConfig().getMaxFramesForGroupBy()));
                             }
                         }
-                    } else if (((AbstractLogicalOperator) (r0.getValue())).getOperatorTag()
-                            .equals(LogicalOperatorTag.RUNNINGAGGREGATE)) {
+                    } else if (r0.getValue().getOperatorTag().equals(LogicalOperatorTag.RUNNINGAGGREGATE)) {
                         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gby.getGroupByList();
                         List<LogicalVariable> columnList = new ArrayList<LogicalVariable>(gbyList.size());
                         for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gbyList) {
@@ -198,8 +200,9 @@
                         op.setPhysicalOperator(new PreclusteredGroupByPOperator(columnList, gby.isGroupAll(),
                                 context.getPhysicalOptimizationConfig().getMaxFramesForGroupBy()));
                     } else {
-                        throw new AlgebricksException("Unsupported nested operator within a group-by: "
-                                + ((AbstractLogicalOperator) (r0.getValue())).getOperatorTag().name());
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
+                                "Unsupported nested operator within a group-by: "
+                                        + r0.getValue().getOperatorTag().name());
                     }
                 }
             }
@@ -235,8 +238,9 @@
                                 mp.findDataSourceIndex(jobGenParams.getIndexName(), dataSourceId);
                         INodeDomain storageDomain = mp.findNodeDomain(dataset.getNodeGroupName());
                         if (dsi == null) {
-                            throw new AlgebricksException("Could not find index " + jobGenParams.getIndexName()
-                                    + " for dataset " + dataSourceId);
+                            throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                                    "Could not find index " + jobGenParams.getIndexName() + " for dataset "
+                                            + dataSourceId);
                         }
                         IndexType indexType = jobGenParams.getIndexType();
                         boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
@@ -288,13 +292,13 @@
     private static void generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
             throws AlgebricksException {
         if (gby.getNestedPlans().size() != 1) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
         }
         ILogicalPlan p0 = gby.getNestedPlans().get(0);
         if (p0.getRoots().size() != 1) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
         }
@@ -303,8 +307,9 @@
         Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
         AbstractLogicalOperator r0Logical = (AbstractLogicalOperator) r0.getValue();
         if (r0Logical.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
-            throw new AlgebricksException("The merge aggregation expression generation should not process a "
-                    + r0Logical.getOperatorTag() + " operator.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, gby.getSourceLocation(),
+                    "The merge aggregation expression generation should not process a " + r0Logical.getOperatorTag()
+                            + " operator.");
         }
         AggregateOperator aggOp = (AggregateOperator) r0.getValue();
         List<Mutable<ILogicalExpression>> aggFuncRefs = aggOp.getExpressions();
@@ -312,11 +317,13 @@
         int n = aggOp.getExpressions().size();
         List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
         for (int i = 0; i < n; i++) {
+            ILogicalExpression aggFuncExpr = aggFuncRefs.get(i).getValue();
             ILogicalExpression mergeExpr = mergeAggregationExpressionFactory
-                    .createMergeAggregation(aggProducedVars.get(i), aggFuncRefs.get(i).getValue(), context);
+                    .createMergeAggregation(aggProducedVars.get(i), aggFuncExpr, context);
             if (mergeExpr == null) {
-                throw new AlgebricksException("The aggregation function " + aggFuncRefs.get(i).getValue()
-                        + " does not have a registered intermediate aggregation function.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, aggFuncExpr.getSourceLocation(),
+                        "The aggregation function " + aggFuncExpr
+                                + " does not have a registered intermediate aggregation function.");
             }
             mergeExpressionRefs.add(new MutableObject<ILogicalExpression>(mergeExpr));
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
index 48a744d..52858e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.optimizer.rules;
 
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
@@ -110,7 +112,7 @@
                 if (reqType == null || !reqType.isOpen()) {
                     int n = expr.getArguments().size();
                     if (n % 2 > 0) {
-                        throw new AlgebricksException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
                                 "Record constructor expected to have an even number of arguments: " + expr);
                     }
                     for (int i = 0; i < n / 2; i++) {
@@ -166,7 +168,7 @@
                 throws AlgebricksException {
             Object varType = env.getVarType(expr.getVariableReference());
             if (varType == null) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
                         "Could not infer type for variable '" + expr.getVariableReference() + "'.");
             }
             return new ClosedDataInfo(false, TypeHelper.isClosed((IAType) varType), expr);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
index 90491d3..cc50dce4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetupCommitExtensionOpRule.java
@@ -104,6 +104,7 @@
 
         //create ExtensionOperator and put the commitOperator in it.
         DelegateOperator extensionOperator = new DelegateOperator(commitOperator);
+        extensionOperator.setSourceLocation(eOp.getSourceLocation());
         extensionOperator.setPhysicalOperator(commitPOperator);
 
         //update plan link
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
index 0c3de91..d4061e0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SimilarityCheckRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Looks for a select operator, containing a condition:
@@ -181,10 +182,12 @@
 
         // Only non-null if we found that varRefExpr refers to an optimizable similarity function call.
         if (simCheckFuncExpr != null) {
+            SourceLocation sourceLoc = simCheckFuncExpr.getSourceLocation();
             // Create a new assign under matchingAssign which assigns the result of our similarity-check function to a variable.
             LogicalVariable newVar = context.newVar();
             AssignOperator newAssign =
                     new AssignOperator(newVar, new MutableObject<ILogicalExpression>(simCheckFuncExpr));
+            newAssign.setSourceLocation(sourceLoc);
             // Hook up inputs.
             newAssign.getInputs()
                     .add(new MutableObject<ILogicalOperator>(matchingAssign.getInputs().get(0).getValue()));
@@ -193,12 +196,15 @@
             // Replace select condition with a get-item on newVarFromExpression.
             List<Mutable<ILogicalExpression>> selectGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVarFromExpression.
-            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            VariableReferenceExpression newVarRef1 = new VariableReferenceExpression(newVar);
+            newVarRef1.setSourceLocation(sourceLoc);
+            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(newVarRef1));
             // Second arg is the item index to be accessed, here 0.
             selectGetItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression selectGetItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression selectGetItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), selectGetItemArgs);
+            selectGetItemExpr.setSourceLocation(sourceLoc);
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(selectGetItemExpr);
 
@@ -206,12 +212,15 @@
             // newVarFromExpression.
             List<Mutable<ILogicalExpression>> assignGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVarFromExpression.
-            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
+            VariableReferenceExpression newVarRef2 = new VariableReferenceExpression(newVar);
+            newVarRef2.setSourceLocation(sourceLoc);
+            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(newVarRef2));
             // Second arg is the item index to be accessed, here 1.
             assignGetItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
-            ILogicalExpression assignGetItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression assignGetItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), assignGetItemArgs);
+            assignGetItemExpr.setSourceLocation(sourceLoc);
             // Replace the original assign expr with the get-item expr.
             simFuncExprRef.setValue(assignGetItemExpr);
 
@@ -238,8 +247,9 @@
             // Second arg is the item index to be accessed.
             getItemArgs.add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression getItemExpr = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression getItemExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM), getItemArgs);
+            getItemExpr.setSourceLocation(simCheckFuncExpr.getSourceLocation());
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(getItemExpr);
             return true;
@@ -282,6 +292,7 @@
                     new ConstantExpression(new AsterixConstantValue(jaccThresh))));
             simCheckFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.SIMILARITY_JACCARD_CHECK), similarityArgs);
+            simCheckFuncExpr.setSourceLocation(funcExpr.getSourceLocation());
         }
 
         // Look for edit-distance function call, and LE or LT.
@@ -310,6 +321,7 @@
                     new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(edThresh))));
             simCheckFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.EDIT_DISTANCE_CHECK), similarityArgs);
+            simCheckFuncExpr.setSourceLocation(funcExpr.getSourceLocation());
         }
         // Preserve all annotations.
         if (simCheckFuncExpr != null) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index b1f646a..8372851 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -20,6 +20,8 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -102,7 +104,7 @@
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 if (!expr.isFunctional()) {
                     AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
                             "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
                 }
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 4f9b4df..c8bec85 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -96,7 +96,7 @@
                     BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
                     BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
                     BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
-                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_MOD);
+                    BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
 
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
@@ -625,7 +625,9 @@
         optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
         List<String> fieldName = null;
         if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
-            optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+            varRef.setSourceLocation(unnestOp.getSourceLocation());
+            optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
         } else {
             fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
                     funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
@@ -731,7 +733,9 @@
             optFuncExpr.setFieldName(funcVarIndex, fieldName);
             optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
             optFuncExpr.setSourceVar(funcVarIndex, var);
-            optFuncExpr.setLogicalExpr(funcVarIndex, new VariableReferenceExpression(var));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+            varRef.setSourceLocation(subTree.getDataSourceRef().getValue().getSourceLocation());
+            optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
             setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
             if (subTree.hasDataSourceScan()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index e541a79..cba4411 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -107,8 +107,8 @@
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(varList.size()))));
         funcArgs.add(numKeysRef);
         for (LogicalVariable keyVar : varList) {
-            Mutable<ILogicalExpression> keyVarRef = new MutableObject<>(new VariableReferenceExpression(keyVar));
-            funcArgs.add(keyVarRef);
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            funcArgs.add(new MutableObject<>(keyVarRef));
         }
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index eafbdaf..c925b55 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -64,6 +64,7 @@
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
@@ -81,6 +82,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -95,6 +97,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 
 /**
@@ -274,7 +277,8 @@
                 break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED,
-                        BuiltinFunctions.FULLTEXT_CONTAINS.getName(), objectFromExpr.getType().getTypeTag());
+                        constExpression.getSourceLocation(), BuiltinFunctions.FULLTEXT_CONTAINS.getName(),
+                        objectFromExpr.getType().getTypeTag());
         }
     }
 
@@ -421,7 +425,8 @@
             case CONDITIONAL_SPLIT_VAR:
                 // Sanity check - the given unnest map should generate this variable.
                 if (!abstractUnnestMapOp.getGenerateCallBackProceedResultVar()) {
-                    throw CompilationException.create(ErrorCode.CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE);
+                    throw CompilationException.create(ErrorCode.CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE,
+                            unnestMapOp.getSourceLocation());
                 }
                 // Fetches conditional splitter - the last position
                 start = numSecondaryKeys + numPrimaryKeys;
@@ -465,6 +470,7 @@
     public static Triple<ILogicalExpression, ILogicalExpression, Boolean> createSearchKeyExpr(Index index,
             IOptimizableFuncExpr optFuncExpr, IAType indexedFieldType, OptimizableOperatorSubTree probeSubTree)
             throws AlgebricksException {
+        SourceLocation sourceLoc = optFuncExpr.getFuncExpr().getSourceLocation();
         if (probeSubTree == null) {
             // We are optimizing a selection query. Search key is a constant.
             // Type Checking and type promotion is done here
@@ -473,7 +479,9 @@
                 //We are looking at a selection case, but using two variables
                 //This means that the second variable comes from a nonPure function call
                 //TODO: Right now we miss on type promotion for nonpure functions
-                return new Triple<>(new VariableReferenceExpression(optFuncExpr.getLogicalVar(1)), null, false);
+                VariableReferenceExpression varRef = new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
+                varRef.setSourceLocation(sourceLoc);
+                return new Triple<>(varRef, null, false);
             }
 
             ILogicalExpression constantAtRuntimeExpression = optFuncExpr.getConstantExpr(0);
@@ -539,25 +547,25 @@
                         case LT:
                         case GE:
                             // round-up
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.CEIL, sourceLoc);
                             break;
                         case LE:
                         case GT:
                             // round-down
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.FLOOR, sourceLoc);
                             break;
                         case EQ:
                             // equality case - both CEIL and FLOOR need to be applied.
-                            replacedConstantValue =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.FLOOR);
-                            replacedConstantValueForEQCase =
-                                    getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.CEIL);
+                            replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.FLOOR, sourceLoc);
+                            replacedConstantValueForEQCase = getReplacedConstantValue(constantValue.getObject(),
+                                    constantValueTag, indexedFieldTypeTag, index.isEnforced(),
+                                    TypeCastingMathFunctionType.CEIL, sourceLoc);
                             break;
                         default:
                             // NEQ should not be a case.
@@ -566,7 +574,7 @@
                 } else {
                     // Type conversion only case: (e.g., INT -> BIGINT)
                     replacedConstantValue = getReplacedConstantValue(constantValue.getObject(), constantValueTag,
-                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.NONE);
+                            indexedFieldTypeTag, index.isEnforced(), TypeCastingMathFunctionType.NONE, sourceLoc);
                 }
             }
             // No type-casting at all
@@ -586,7 +594,8 @@
             OptimizableOperatorSubTree opSubTree0 = optFuncExpr.getOperatorSubTree(0);
             int probeVarIndex = opSubTree0 == null || opSubTree0 == probeSubTree ? 0 : 1;
             LogicalVariable probeVar = optFuncExpr.getLogicalVar(probeVarIndex);
-            ILogicalExpression probeExpr = new VariableReferenceExpression(probeVar);
+            VariableReferenceExpression probeExpr = new VariableReferenceExpression(probeVar);
+            probeExpr.setSourceLocation(sourceLoc);
 
             ATypeTag indexedFieldTypeTag = TypeComputeUtils.getActualType(indexedFieldType).getTypeTag();
             if (ATypeHierarchy.getTypeDomain(indexedFieldTypeTag) == ATypeHierarchy.Domain.NUMERIC) {
@@ -595,6 +604,7 @@
                 if (probeTypeTypeTag != indexedFieldTypeTag) {
                     ScalarFunctionCallExpression castFunc = new ScalarFunctionCallExpression(
                             FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE_LAX));
+                    castFunc.setSourceLocation(sourceLoc);
                     castFunc.getArguments().add(new MutableObject<>(probeExpr));
                     TypeCastUtils.setRequiredAndInputTypes(castFunc, indexedFieldType, probeType);
                     boolean realTypeConvertedToIntegerType =
@@ -607,14 +617,14 @@
     }
 
     private static AsterixConstantValue getReplacedConstantValue(IAObject sourceObject, ATypeTag sourceTypeTag,
-            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction)
-            throws CompilationException {
+            ATypeTag targetTypeTag, boolean strictDemote, TypeCastingMathFunctionType mathFunction,
+            SourceLocation sourceLoc) throws CompilationException {
         try {
             return ATypeHierarchy.getAsterixConstantValueFromNumericTypeObject(sourceObject, targetTypeTag,
                     strictDemote, mathFunction);
         } catch (HyracksDataException e) {
-            throw new CompilationException(ErrorCode.ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION, e, sourceTypeTag,
-                    targetTypeTag);
+            throw new CompilationException(ErrorCode.ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION, e, sourceLoc,
+                    sourceTypeTag, targetTypeTag);
         }
     }
 
@@ -749,6 +759,7 @@
                 // Non-index only plan case
                 indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
                 SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+                topSelectOp.setSourceLocation(finalIndexSearchOp.getSourceLocation());
                 topSelectOp.getInputs().add(indexSubTree.getRootRef());
                 topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
                 context.computeAndSetTypeEnvironmentForOperator(topSelectOp);
@@ -768,6 +779,7 @@
             ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
             IOptimizationContext context, boolean retainInput, boolean retainNull,
             boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
         jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
@@ -784,6 +796,7 @@
         IFunctionInfo secondaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         UnnestingFunctionCallExpression secondaryIndexSearchFunc =
                 new UnnestingFunctionCallExpression(secondaryIndexSearch, secondaryIndexFuncArgs);
+        secondaryIndexSearchFunc.setSourceLocation(sourceLoc);
         secondaryIndexSearchFunc.setReturnsUniqueValues(true);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has all
         // necessary arguments to determine which index to use, which variables contain the index-search keys,
@@ -796,6 +809,7 @@
                 LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
                         secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
                         secondaryIndexOutputTypes, true);
+                secondaryIndexLeftOuterUnnestOp.setSourceLocation(sourceLoc);
                 secondaryIndexLeftOuterUnnestOp
                         .setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
                 secondaryIndexLeftOuterUnnestOp.getInputs().add(new MutableObject<>(inputOp));
@@ -804,13 +818,15 @@
                 return secondaryIndexLeftOuterUnnestOp;
             } else {
                 // Left-outer-join without retainInput doesn't make sense.
-                throw new AlgebricksException("Left-outer-join should propagate all inputs from the outer branch.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Left-outer-join should propagate all inputs from the outer branch.");
             }
         } else {
             // If this is not a left-outer-join case, then we use UNNEST-MAP operator.
             UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes,
                     retainInput);
+            secondaryIndexUnnestOp.setSourceLocation(sourceLoc);
             secondaryIndexUnnestOp.setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
             secondaryIndexUnnestOp.getInputs().add(new MutableObject<>(inputOp));
             context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
@@ -824,12 +840,16 @@
             boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
             List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // Optionally add a sort on the primary-index keys before searching the primary index.
         OrderOperator order = null;
         if (sortPrimaryKeys) {
             order = new OrderOperator();
+            order.setSourceLocation(sourceLoc);
             for (LogicalVariable pkVar : primaryKeyVars) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<>(new VariableReferenceExpression(pkVar));
+                VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+                pkVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
                 order.getOrderExpressions().add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
             }
             // The secondary-index search feeds into the sort.
@@ -837,9 +857,11 @@
             order.setExecutionMode(ExecutionMode.LOCAL);
             context.computeAndSetTypeEnvironmentForOperator(order);
         }
+
         // Creates the primary-index search unnest-map operator.
-        AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
-                retainMissing, requiresBroadcast, primaryKeyVars, primaryIndexUnnestVars, primaryIndexOutputTypes);
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp =
+                createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
+                        primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
         if (sortPrimaryKeys) {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
         } else {
@@ -859,6 +881,7 @@
             LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
             List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
         // From now on, we deal with the index-only plan.
         // Initializes the information required for the index-only plan optimization.
@@ -930,19 +953,24 @@
             switch (spatialType.getTypeTag()) {
                 case POINT:
                     // Reconstructs a POINT value.
-                    AbstractFunctionCallExpression createPointExpr = createPointExpression(skVarsFromSIdxUnnestMap);
+                    AbstractFunctionCallExpression createPointExpr =
+                            createPointExpression(skVarsFromSIdxUnnestMap, sourceLoc);
                     restoredSKVarFromRTree.add(context.newVar());
                     restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createPointExpr));
                     skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    skVarAssignOpInRightPath.setSourceLocation(sourceLoc);
                     break;
                 case RECTANGLE:
                     // Reconstructs a RECTANGLE value.
-                    AbstractFunctionCallExpression expr1 = createPointExpression(skVarsFromSIdxUnnestMap.subList(0, 2));
-                    AbstractFunctionCallExpression expr2 = createPointExpression(skVarsFromSIdxUnnestMap.subList(2, 4));
+                    AbstractFunctionCallExpression expr1 =
+                            createPointExpression(skVarsFromSIdxUnnestMap.subList(0, 2), sourceLoc);
+                    AbstractFunctionCallExpression expr2 =
+                            createPointExpression(skVarsFromSIdxUnnestMap.subList(2, 4), sourceLoc);
                     AbstractFunctionCallExpression createRectangleExpr = createRectangleExpression(expr1, expr2);
                     restoredSKVarFromRTree.add(context.newVar());
                     restoredSKFromRTreeExprs.add(new MutableObject<ILogicalExpression>(createRectangleExpr));
                     skVarAssignOpInRightPath = new AssignOperator(restoredSKVarFromRTree, restoredSKFromRTreeExprs);
+                    skVarAssignOpInRightPath.setSourceLocation(sourceLoc);
                     break;
                 default:
                     break;
@@ -950,9 +978,9 @@
         }
 
         // Gets all variables from the right (inner) branch.
-        VariableUtilities.getLiveVariables((ILogicalOperator) subTree.getRootRef().getValue(), liveVarsInSubTreeRootOp);
+        VariableUtilities.getLiveVariables(subTree.getRootRef().getValue(), liveVarsInSubTreeRootOp);
         // Gets the used variables from the SELECT or JOIN operator.
-        VariableUtilities.getUsedVariables((ILogicalOperator) topOpRef.getValue(), usedVarsInTopOp);
+        VariableUtilities.getUsedVariables(topOpRef.getValue(), usedVarsInTopOp);
         // Excludes the variables in the condition from the outer branch - in join case.
         for (Iterator<LogicalVariable> iterator = usedVarsInTopOp.iterator(); iterator.hasNext();) {
             LogicalVariable v = iterator.next();
@@ -1022,6 +1050,7 @@
                 // If this assign operator is not used in the SELECT or JOIN operator,
                 // we will add this operator after creating UNION operator in the last part of this method.
                 constAssignOp = new AssignOperator(constAssignVars, constAssignExprs);
+                constAssignOp.setSourceLocation(sourceLoc);
                 if (constantAssignVarUsedInTopOp) {
                     // Places this assign after the secondary index-search op.
                     constAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
@@ -1037,7 +1066,7 @@
         if (afterTopOpRefs != null) {
             for (Mutable<ILogicalOperator> afterTopOpRef : afterTopOpRefs) {
                 varsTmpSet.clear();
-                OperatorPropertiesUtil.getFreeVariablesInOp((ILogicalOperator) afterTopOpRef.getValue(), varsTmpSet);
+                OperatorPropertiesUtil.getFreeVariablesInOp(afterTopOpRef.getValue(), varsTmpSet);
                 copyVarsToAnotherList(varsTmpSet, usedVarsAfterTopOp);
             }
         }
@@ -1052,6 +1081,7 @@
         // Adds a SPLIT operator after the given secondary index-search unnest-map operator.
         splitOp = new SplitOperator(2,
                 new MutableObject<ILogicalExpression>(new VariableReferenceExpression(condSplitVars.get(0))));
+        splitOp.setSourceLocation(sourceLoc);
         splitOp.getInputs().add(new MutableObject<ILogicalOperator>(currentOp));
         splitOp.setExecutionMode(ExecutionMode.PARTITIONED);
         context.computeAndSetTypeEnvironmentForOperator(splitOp);
@@ -1075,9 +1105,12 @@
             LogicalVariable newVar = context.newVar();
             liveVarAfterSplitToLeftPathMap.put(v, newVar);
             assignVars.add(newVar);
-            assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            assignExprs.add(new MutableObject<ILogicalExpression>(vRef));
         }
         AssignOperator origVarsToLeftPathVarsAssignOp = new AssignOperator(assignVars, assignExprs);
+        origVarsToLeftPathVarsAssignOp.setSourceLocation(sourceLoc);
         origVarsToLeftPathVarsAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(splitOp));
         context.computeAndSetTypeEnvironmentForOperator(origVarsToLeftPathVarsAssignOp);
         origVarsToLeftPathVarsAssignOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -1177,7 +1210,7 @@
         // For the index-nested-loop join case,
         // we propagate all variables that come from the outer relation and are used after join operator.
         // Adds the variables that are both live after JOIN and used after the JOIN operator.
-        VariableUtilities.getLiveVariables((ILogicalOperator) topOpRef.getValue(), liveVarsAfterTopOp);
+        VariableUtilities.getLiveVariables(topOpRef.getValue(), liveVarsAfterTopOp);
         for (LogicalVariable v : usedVarsAfterTopOp) {
             if (!liveVarsAfterTopOp.contains(v) || findVarInTripleVarList(unionVarMap, v, false)) {
                 continue;
@@ -1190,8 +1223,7 @@
         // Replaces the original variables in the operators after the SELECT or JOIN operator to satisfy SSA.
         if (afterTopOpRefs != null) {
             for (Mutable<ILogicalOperator> afterTopOpRef : afterTopOpRefs) {
-                VariableUtilities.substituteVariables((ILogicalOperator) afterTopOpRef.getValue(),
-                        origVarToOutputVarMap, context);
+                VariableUtilities.substituteVariables(afterTopOpRef.getValue(), origVarToOutputVarMap, context);
             }
         }
 
@@ -1199,7 +1231,8 @@
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
                 retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
-                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes);
+                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc);
+        primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
         context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
         primaryIndexUnnestMapOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -1217,6 +1250,7 @@
         LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
         newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
                 retainMissing, newMissingPlaceHolderVar);
+        newSelectOpInLeftPath.setSourceLocation(conditionRefExpr.getSourceLocation());
         VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
 
         // If there are ASSIGN operators before the SELECT or JOIN operator,
@@ -1276,6 +1310,7 @@
             ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
             newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
                     retainMissing, newMissingPlaceHolderVar);
+            newSelectOpInRightPath.setSourceLocation(conditionRefExpr2.getSourceLocation());
             newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
             VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
             VariableUtilities.substituteVariables(newSelectOpInRightPath, origSKFieldVarToNewSKFieldVarMap, context);
@@ -1294,6 +1329,7 @@
 
         // UNIONALL operator that combines both paths.
         unionAllOp = new UnionAllOperator(unionVarMap);
+        unionAllOp.setSourceLocation(sourceLoc);
         unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(newSelectOpInLeftPath));
         unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
 
@@ -1322,8 +1358,8 @@
 
     private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
             boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
-            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
-            throws AlgebricksException {
+            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes,
+            SourceLocation sourceLoc) throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -1339,6 +1375,7 @@
         IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
         AbstractFunctionCallExpression primaryIndexSearchFunc =
                 new ScalarFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
+        primaryIndexSearchFunc.setSourceLocation(sourceLoc);
         // This is the operator that jobgen will be looking for. It contains an unnest function that has
         // all necessary arguments to determine which index to use, which variables contain the index-search keys,
         // what is the original dataset, etc.
@@ -1348,14 +1385,17 @@
                 primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
                         new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                         retainInput);
+                primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
             } else {
                 // Left-outer-join without retainNull and retainInput doesn't make sense.
-                throw new AlgebricksException("Left-outer-join should propagate all inputs from the outer branch.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Left-outer-join should propagate all inputs from the outer branch.");
             }
         } else {
             primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
                     new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                     retainInput);
+            primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         }
         return primaryIndexUnnestMapOp;
     }
@@ -1416,12 +1456,18 @@
         }
     }
 
-    private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars) {
+    private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars,
+            SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         AbstractFunctionCallExpression createPointExpr1 =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_POINT));
-        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(0))));
-        expressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pointVars.get(1))));
+        createPointExpr1.setSourceLocation(sourceLoc);
+        VariableReferenceExpression pointVarRef0 = new VariableReferenceExpression(pointVars.get(0));
+        pointVarRef0.setSourceLocation(sourceLoc);
+        expressions.add(new MutableObject<ILogicalExpression>(pointVarRef0));
+        VariableReferenceExpression pointVarRef1 = new VariableReferenceExpression(pointVars.get(1));
+        pointVarRef1.setSourceLocation(sourceLoc);
+        expressions.add(new MutableObject<ILogicalExpression>(pointVarRef1));
         createPointExpr1.getArguments().addAll(expressions);
         return createPointExpr1;
     }
@@ -1431,51 +1477,93 @@
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         AbstractFunctionCallExpression createRectangleExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_RECTANGLE));
+        createRectangleExpr.setSourceLocation(createPointExpr1.getSourceLocation());
         expressions.add(new MutableObject<ILogicalExpression>(createPointExpr1));
         expressions.add(new MutableObject<ILogicalExpression>(createPointExpr2));
         createRectangleExpr.getArguments().addAll(expressions);
         return createRectangleExpr;
     }
 
-    public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp)
-            throws AlgebricksException {
-        //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
-        ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
-        Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
-        AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
-        boolean foundSelectNonMissing = false;
-        ScalarFunctionCallExpression isMissingFuncExpr = null;
-        AbstractLogicalOperator inputOp = subPlanRootOp;
-        while (inputOp != null) {
-            if (inputOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
-                SelectOperator selectOp = (SelectOperator) inputOp;
-                if (selectOp.getCondition().getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    if (((AbstractFunctionCallExpression) selectOp.getCondition().getValue()).getFunctionIdentifier()
-                            .equals(AlgebricksBuiltinFunctions.NOT)) {
-                        ScalarFunctionCallExpression notFuncExpr =
-                                (ScalarFunctionCallExpression) selectOp.getCondition().getValue();
-                        if (notFuncExpr.getArguments().get(0).getValue()
-                                .getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                            if (((AbstractFunctionCallExpression) notFuncExpr.getArguments().get(0).getValue())
-                                    .getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
-                                isMissingFuncExpr =
-                                        (ScalarFunctionCallExpression) notFuncExpr.getArguments().get(0).getValue();
-                                if (isMissingFuncExpr.getArguments().get(0).getValue()
-                                        .getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                    foundSelectNonMissing = true;
-                                    break;
-                                }
-                            }
+    private static ScalarFunctionCallExpression getNestedIsMissingCall(AbstractFunctionCallExpression call,
+            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
+        ScalarFunctionCallExpression isMissingFuncExpr;
+        if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+            if (call.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                if (((AbstractFunctionCallExpression) call.getArguments().get(0).getValue()).getFunctionIdentifier()
+                        .equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+                    isMissingFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
+                    if (isMissingFuncExpr.getArguments().get(0).getValue()
+                            .getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                        LogicalVariable var =
+                                ((VariableReferenceExpression) isMissingFuncExpr.getArguments().get(0).getValue())
+                                        .getVariableReference();
+                        List<LogicalVariable> liveSubplanVars = new ArrayList<>();
+                        VariableUtilities.getSubplanLocalLiveVariables(rightSubTree.getRoot(), liveSubplanVars);
+                        if (liveSubplanVars.contains(var)) {
+                            return isMissingFuncExpr;
                         }
                     }
                 }
             }
-            inputOp = inputOp.getInputs().size() > 0 ? (AbstractLogicalOperator) inputOp.getInputs().get(0).getValue()
-                    : null;
         }
+        return null;
+    }
 
-        if (!foundSelectNonMissing) {
-            throw CompilationException.create(ErrorCode.CANNOT_FIND_NON_MISSING_SELECT_OPERATOR);
+    public static ScalarFunctionCallExpression findIsMissingInSubplan(AbstractLogicalOperator inputOp,
+            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
+        ScalarFunctionCallExpression isMissingFuncExpr = null;
+        AbstractLogicalOperator currentOp = inputOp;
+        while (currentOp != null) {
+            if (currentOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                SelectOperator selectOp = (SelectOperator) currentOp;
+                if (selectOp.getCondition().getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression call =
+                            (AbstractFunctionCallExpression) (selectOp).getCondition().getValue();
+                    if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+                        for (Mutable<ILogicalExpression> mexpr : call.getArguments()) {
+                            if (mexpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                                isMissingFuncExpr = getNestedIsMissingCall(
+                                        (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree);
+                                if (isMissingFuncExpr != null) {
+                                    return isMissingFuncExpr;
+                                }
+                            }
+                        }
+                    }
+                    isMissingFuncExpr = getNestedIsMissingCall(call, rightSubTree);
+                    if (isMissingFuncExpr != null) {
+                        return isMissingFuncExpr;
+                    }
+                }
+            } else if (currentOp.hasNestedPlans()) {
+                AbstractOperatorWithNestedPlans nestedPlanOp = (AbstractOperatorWithNestedPlans) currentOp;
+                for (ILogicalPlan nestedPlan : nestedPlanOp.getNestedPlans()) {
+                    for (Mutable<ILogicalOperator> root : nestedPlan.getRoots()) {
+                        isMissingFuncExpr =
+                                findIsMissingInSubplan((AbstractLogicalOperator) root.getValue(), rightSubTree);
+                        if (isMissingFuncExpr != null) {
+                            return isMissingFuncExpr;
+                        }
+                    }
+                }
+            }
+            currentOp = currentOp.getInputs().isEmpty() ? null
+                    : (AbstractLogicalOperator) currentOp.getInputs().get(0).getValue();
+        }
+        return isMissingFuncExpr;
+    }
+
+    public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp,
+            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
+        //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
+        ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
+        Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
+        AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
+        ScalarFunctionCallExpression isMissingFuncExpr = findIsMissingInSubplan(subPlanRootOp, rightSubTree);
+
+        if (isMissingFuncExpr == null) {
+            throw CompilationException.create(ErrorCode.CANNOT_FIND_NON_MISSING_SELECT_OPERATOR,
+                    lojGroupbyOp.getSourceLocation());
         }
         return isMissingFuncExpr;
     }
@@ -1486,8 +1574,9 @@
         //reset the missing placeholder variable in groupby operator
         ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInGroupBy();
         isMissingFuncExpr.getArguments().clear();
-        isMissingFuncExpr.getArguments().add(
-                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newMissingPlaceholderVaraible)));
+        VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible);
+        newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation());
+        isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef));
 
         //recompute type environment.
         OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJGroupbyOpRef(), context);
@@ -1526,12 +1615,16 @@
             Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
             IOptimizationContext context, Index secondaryIndex, boolean retainInput, boolean retainNull)
             throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset, recordType,
                 metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
         // add a sort on the RID fields before fetching external data.
         OrderOperator order = new OrderOperator();
+        order.setSourceLocation(sourceLoc);
         for (LogicalVariable pkVar : primaryKeyVars) {
-            Mutable<ILogicalExpression> vRef = new MutableObject<>(new VariableReferenceExpression(pkVar));
+            VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+            pkVarRef.setSourceLocation(sourceLoc);
+            Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
             order.getOrderExpressions().add(new Pair<>(OrderOperator.ASC_ORDER, vRef));
         }
         // The secondary-index search feeds into the sort.
@@ -1556,8 +1649,10 @@
         IFunctionInfo externalLookup = FunctionUtil.getFunctionInfo(BuiltinFunctions.EXTERNAL_LOOKUP);
         AbstractFunctionCallExpression externalLookupFunc =
                 new ScalarFunctionCallExpression(externalLookup, externalLookupArgs);
+        externalLookupFunc.setSourceLocation(sourceLoc);
         UnnestMapOperator unnestOp = new UnnestMapOperator(externalUnnestVars,
                 new MutableObject<ILogicalExpression>(externalLookupFunc), outputTypes, retainInput);
+        unnestOp.setSourceLocation(sourceLoc);
         // Fed by the order operator or the secondaryIndexUnnestOp.
         unnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
 
@@ -1712,7 +1807,7 @@
         List<LogicalVariable> dataScanRecordVars = new ArrayList<>();
 
         // Collects the used variables in the given select (join) operator.
-        VariableUtilities.getUsedVariables((ILogicalOperator) topRef.getValue(), usedVarsInSelJoinOpTemp);
+        VariableUtilities.getUsedVariables(topRef.getValue(), usedVarsInSelJoinOpTemp);
 
         // Removes the duplicated variables that are used in the select (join) operator
         // in case where the variable is used multiple times in the operator's expression.
@@ -1745,10 +1840,8 @@
         List<LogicalVariable> liveVarsInSubTreeRootOp = new ArrayList<>();
         List<LogicalVariable> producedVarsInSubTreeRootOp = new ArrayList<>();
 
-        VariableUtilities.getLiveVariables((ILogicalOperator) indexSubTree.getRootRef().getValue(),
-                liveVarsInSubTreeRootOp);
-        VariableUtilities.getProducedVariables((ILogicalOperator) indexSubTree.getRootRef().getValue(),
-                producedVarsInSubTreeRootOp);
+        VariableUtilities.getLiveVariables(indexSubTree.getRootRef().getValue(), liveVarsInSubTreeRootOp);
+        VariableUtilities.getProducedVariables(indexSubTree.getRootRef().getValue(), producedVarsInSubTreeRootOp);
 
         copyVarsToAnotherList(liveVarsInSubTreeRootOp, liveVarsAfterSelJoinOp);
         copyVarsToAnotherList(producedVarsInSubTreeRootOp, liveVarsAfterSelJoinOp);
@@ -1943,10 +2036,8 @@
         for (Mutable<ILogicalOperator> afterSelectRef : afterSelectOpRefs) {
             usedVarsAfterSelectOrJoinOp.clear();
             producedVarsAfterSelectOrJoinOp.clear();
-            VariableUtilities.getUsedVariables((ILogicalOperator) afterSelectRef.getValue(),
-                    usedVarsAfterSelectOrJoinOp);
-            VariableUtilities.getProducedVariables((ILogicalOperator) afterSelectRef.getValue(),
-                    producedVarsAfterSelectOrJoinOp);
+            VariableUtilities.getUsedVariables(afterSelectRef.getValue(), usedVarsAfterSelectOrJoinOp);
+            VariableUtilities.getProducedVariables(afterSelectRef.getValue(), producedVarsAfterSelectOrJoinOp);
             // Checks whether COUNT exists in the given plan since we can substitute record variable
             // with the PK variable as an optimization because COUNT(record) is equal to COUNT(PK).
             // For this case only, we can replace the record variable with the PK variable.
@@ -1955,14 +2046,13 @@
                 aggOp = (AggregateOperator) afterSelectRefOp;
                 condExprs = aggOp.getExpressions();
                 for (int i = 0; i < condExprs.size(); i++) {
-                    condExpr = (ILogicalExpression) condExprs.get(i).getValue();
+                    condExpr = condExprs.get(i).getValue();
                     if (condExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                         condExprFnCall = (AbstractFunctionCallExpression) condExpr;
                         if (condExprFnCall.getFunctionIdentifier() == BuiltinFunctions.COUNT) {
                             // COUNT found. count on the record ($$0) can be replaced as the PK variable.
                             countAggFunctionIsUsedInThePlan = true;
-                            VariableUtilities.getUsedVariables((ILogicalOperator) afterSelectRef.getValue(),
-                                    usedVarsInCount);
+                            VariableUtilities.getUsedVariables(afterSelectRef.getValue(), usedVarsInCount);
                             break;
                         }
                     }
@@ -2091,7 +2181,7 @@
         }
 
         if (matchedFuncExprs.size() == 1) {
-            condExpr = (ILogicalExpression) optFuncExpr.getFuncExpr();
+            condExpr = optFuncExpr.getFuncExpr();
             condExprFnCall = (AbstractFunctionCallExpression) condExpr;
             for (int i = 0; i < condExprFnCall.getArguments().size(); i++) {
                 Mutable<ILogicalExpression> expr = condExprFnCall.getArguments().get(i);
@@ -2276,7 +2366,7 @@
                 AssignOperator assignOp = (AssignOperator) assignUnnestOp;
                 condExprs = assignOp.getExpressions();
                 for (int i = 0; i < condExprs.size(); i++) {
-                    condExpr = (ILogicalExpression) condExprs.get(i).getValue();
+                    condExpr = condExprs.get(i).getValue();
                     if (condExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
                             && !targetVars.contains(assignOp.getVariables().get(i))) {
                         targetVars.add(assignOp.getVariables().get(i));
@@ -2304,13 +2394,13 @@
             case LEFT_OUTER_UNNEST_MAP:
                 return topOp;
             case UNIONALL:
-                dataSourceOp = (ILogicalOperator) dataSourceOp.getInputs().get(0).getValue();
+                dataSourceOp = dataSourceOp.getInputs().get(0).getValue();
                 // Index-only plan case:
                 // The order of operators: 7 unionall <- 6 select <- 5 assign?
                 // <- 4 unnest-map (PIdx) <- 3 split <- 2 unnest-map (SIdx) <- ...
                 // We do this to skip the primary index-search since we are looking for a secondary index-search here.
                 do {
-                    dataSourceOp = (ILogicalOperator) dataSourceOp.getInputs().get(0).getValue();
+                    dataSourceOp = dataSourceOp.getInputs().get(0).getValue();
                 } while (dataSourceOp.getOperatorTag() != LogicalOperatorTag.SPLIT && dataSourceOp.hasInputs());
 
                 if (dataSourceOp.getOperatorTag() != LogicalOperatorTag.SPLIT) {
@@ -2319,7 +2409,7 @@
                 }
 
                 do {
-                    dataSourceOp = (ILogicalOperator) dataSourceOp.getInputs().get(0).getValue();
+                    dataSourceOp = dataSourceOp.getInputs().get(0).getValue();
                 } while (dataSourceOp.getOperatorTag() != LogicalOperatorTag.UNNEST_MAP
                         && dataSourceOp.getOperatorTag() != LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP
                         && dataSourceOp.hasInputs());
@@ -2436,9 +2526,10 @@
      *         false otherwise.
      */
     public static boolean getNoIndexOnlyOption(IOptimizationContext context) {
-        Map<String, String> config = context.getMetadataProvider().getConfig();
+        Map<String, Object> config = context.getMetadataProvider().getConfig();
         if (config.containsKey(AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION)) {
-            return Boolean.parseBoolean(config.get(AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION));
+            return Boolean
+                    .parseBoolean((String) config.get(AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION));
         }
         return AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION_DEFAULT_VALUE;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 60d4d3d..099f769 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -68,6 +68,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -353,7 +354,8 @@
                 keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
             }
             if (keyPos < 0) {
-                throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR);
+                throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR,
+                        optFuncExpr.getFuncExpr().getSourceLocation());
             }
             // returnedSearchKeyExpr contains a pair of search expression.
             // The second expression will not be null only if we are creating an EQ search predicate
@@ -368,7 +370,9 @@
             if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 constantAtRuntimeExpressions[keyPos] = searchKeyExpr;
                 constAtRuntimeExprVars[keyPos] = context.newVar();
-                searchKeyExpr = new VariableReferenceExpression(constAtRuntimeExprVars[keyPos]);
+                VariableReferenceExpression varRef = new VariableReferenceExpression(constAtRuntimeExprVars[keyPos]);
+                varRef.setSourceLocation(searchKeyExpr.getSourceLocation());
+                searchKeyExpr = varRef;
             }
 
             LimitType limit = getLimitType(optFuncExpr, probeSubTree);
@@ -576,6 +580,7 @@
         if (!assignKeyVarList.isEmpty()) {
             // Assign operator that sets the constant secondary-index search-key fields if necessary.
             AssignOperator assignSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
+            assignSearchKeys.setSourceLocation(dataSourceOp.getSourceLocation());
             if (probeSubTree == null) {
                 // We are optimizing a selection query.
                 // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
@@ -694,26 +699,33 @@
                 IFunctionInfo primaryIndexSearch = FunctionUtil.getFunctionInfo(BuiltinFunctions.INDEX_SEARCH);
                 UnnestingFunctionCallExpression primaryIndexSearchFunc =
                         new UnnestingFunctionCallExpression(primaryIndexSearch, primaryIndexFuncArgs);
+                primaryIndexSearchFunc.setSourceLocation(dataSourceOp.getSourceLocation());
                 primaryIndexSearchFunc.setReturnsUniqueValues(true);
+                AbstractUnnestMapOperator unnestMapOp;
                 if (!leftOuterUnnestMapRequired) {
-                    indexSearchOp = new UnnestMapOperator(scanVariables,
+                    unnestMapOp = new UnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
                             true);
                 }
+                unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+                indexSearchOp = unnestMapOp;
             } else {
+                AbstractUnnestMapOperator unnestMapOp;
                 if (!leftOuterUnnestMapRequired) {
-                    indexSearchOp = new UnnestMapOperator(scanVariables,
+                    unnestMapOp = new UnnestMapOperator(scanVariables,
                             ((UnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(), primaryIndexOutputTypes,
                             retainInput);
                 } else {
-                    indexSearchOp = new LeftOuterUnnestMapOperator(scanVariables,
+                    unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             ((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
                             primaryIndexOutputTypes, true);
                 }
+                unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
+                indexSearchOp = unnestMapOp;
             }
 
             indexSearchOp.getInputs().add(new MutableObject<>(inputOp));
@@ -775,7 +787,7 @@
         // The original select cond must be an AND. Check it just to be sure.
         if (funcExpr.getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND) {
             throw new CompilationException(ErrorCode.COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX,
-                    funcExpr.toString());
+                    funcExpr.getSourceLocation(), funcExpr.toString());
         }
         // Clean the conjuncts.
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
@@ -902,7 +914,9 @@
     private ILogicalExpression createSelectCondition(List<Mutable<ILogicalExpression>> predList) {
         if (predList.size() > 1) {
             IFunctionInfo finfo = FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND);
-            return new ScalarFunctionCallExpression(finfo, predList);
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(finfo, predList);
+            andExpr.setSourceLocation(predList.get(0).getValue().getSourceLocation());
+            return andExpr;
         }
         return predList.get(0).getValue();
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 401ea23..577754a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -24,7 +24,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -41,7 +40,6 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
@@ -127,11 +125,6 @@
             return false;
         }
 
-        if (op.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR
-                && !(((DelegateOperator) op).getDelegate() instanceof CommitOperator)) {
-            return false;
-        }
-
         afterJoinRefs = new ArrayList<>();
         // Recursively checks the given plan whether the desired pattern exists in it.
         // If so, try to optimize the plan.
@@ -206,8 +199,7 @@
         if (op1.getInputs().size() != 1) {
             return false;
         }
-        if (((AbstractLogicalOperator) op1.getInputs().get(0).getValue())
-                .getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+        if (op1.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
             return false;
         }
         if (op1.getOperatorTag() == LogicalOperatorTag.GROUP) {
@@ -387,8 +379,8 @@
                     // in GroupByOp.
                     if (isThisOpLeftOuterJoin && isParentOpGroupBy) {
                         analysisCtx.setLOJGroupbyOpRef(opRef);
-                        ScalarFunctionCallExpression isNullFuncExpr =
-                                AccessMethodUtils.findLOJIsMissingFuncInGroupBy((GroupByOperator) opRef.getValue());
+                        ScalarFunctionCallExpression isNullFuncExpr = AccessMethodUtils
+                                .findLOJIsMissingFuncInGroupBy((GroupByOperator) opRef.getValue(), rightSubTree);
                         analysisCtx.setLOJIsMissingFuncInGroupBy(isNullFuncExpr);
                     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 7938f49..7739833 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -66,6 +66,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
 
@@ -134,7 +135,8 @@
     }
 
     private AssignOperator createAssignOperator(List<IOptimizableFuncExpr> optFuncExprs,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context) {
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context,
+            SourceLocation sourceLoc) {
         List<LogicalVariable> assignKeyVarList = new ArrayList<>();
         List<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<>();
 
@@ -154,7 +156,9 @@
                 maxFilterVars.add(var);
             }
         }
-        return new AssignOperator(assignKeyVarList, assignKeyExprList);
+        AssignOperator assignOp = new AssignOperator(assignKeyVarList, assignKeyExprList);
+        assignOp.setSourceLocation(sourceLoc);
+        return assignOp;
     }
 
     private void assignFilterFromQuery(List<IOptimizableFuncExpr> optFuncExprs, AbstractLogicalOperator op,
@@ -175,15 +179,17 @@
                     List<LogicalVariable> minFilterVars = new ArrayList<>();
                     List<LogicalVariable> maxFilterVars = new ArrayList<>();
 
-                    AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
+                    AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context,
+                            dataSourceScanOp.getSourceLocation());
 
                     dataSourceScanOp.setMinFilterVars(minFilterVars);
                     dataSourceScanOp.setMaxFilterVars(maxFilterVars);
 
                     List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
                     for (LogicalVariable var : assignOp.getVariables()) {
-                        additionalFilteringExpressions
-                                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                        varRef.setSourceLocation(assignOp.getSourceLocation());
+                        additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                     }
 
                     dataSourceScanOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
@@ -206,16 +212,17 @@
                         List<LogicalVariable> minFilterVars = new ArrayList<>();
                         List<LogicalVariable> maxFilterVars = new ArrayList<>();
 
-                        AssignOperator assignOp =
-                                createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
+                        AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars,
+                                context, unnestMapOp.getSourceLocation());
 
                         unnestMapOp.setMinFilterVars(minFilterVars);
                         unnestMapOp.setMaxFilterVars(maxFilterVars);
 
                         List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<>();
                         for (LogicalVariable var : assignOp.getVariables()) {
-                            additionalFilteringExpressions
-                                    .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                            varRef.setSourceLocation(assignOp.getSourceLocation());
+                            additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(varRef));
                         }
                         unnestMapOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                         assignOp.getInputs().add(new MutableObject<>(unnestMapOp.getInputs().get(0).getValue()));
@@ -320,6 +327,7 @@
         }
 
         IntersectOperator intersectWithFilter = new IntersectOperator(outputVars, compareVars, filterVars);
+        intersectWithFilter.setSourceLocation(intersect.getSourceLocation());
         intersectWithFilter.getInputs().addAll(intersect.getInputs());
         return intersectWithFilter;
     }
@@ -343,9 +351,12 @@
         primaryOp.setMinFilterVars(Collections.singletonList(minFilterVar));
         primaryOp.setMaxFilterVars(Collections.singletonList(maxFilterVar));
 
+        VariableReferenceExpression minFilterVarRef = new VariableReferenceExpression(minFilterVar);
+        minFilterVarRef.setSourceLocation(primaryOp.getSourceLocation());
+        VariableReferenceExpression maxFilterVarRef = new VariableReferenceExpression(maxFilterVar);
+        maxFilterVarRef.setSourceLocation(primaryOp.getSourceLocation());
         List<Mutable<ILogicalExpression>> indexFilterExpression =
-                Arrays.asList(new MutableObject<>(new VariableReferenceExpression(minFilterVar)),
-                        new MutableObject<>(new VariableReferenceExpression(maxFilterVar)));
+                Arrays.asList(new MutableObject<>(minFilterVarRef), new MutableObject<>(maxFilterVarRef));
 
         primaryOp.setAdditionalFilteringExpressions(indexFilterExpression);
         context.computeAndSetTypeEnvironmentForOperator(primaryOp);
@@ -366,7 +377,8 @@
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                     FunctionIdentifier fid = f.getFunctionIdentifier();
                     if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fid.getName());
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(),
+                                fid.getName());
                     }
                     AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                     jobGenParams.readFromFuncArgs(f.getArguments());
@@ -414,7 +426,8 @@
                         dataverseName = jobGenParams.dataverseName;
                         datasetName = jobGenParams.datasetName;
                     } else {
-                        throw new AlgebricksException("Unexpected function for Unnest Map: " + fid);
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, f.getSourceLocation(),
+                                "Unexpected function for Unnest Map: " + fid);
                     }
                     return ((MetadataProvider) context.getMetadataProvider()).findDataset(dataverseName, datasetName);
                 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 27e5645..0ea16ae 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -220,13 +220,17 @@
         List<ILogicalOperator> subRoots = new ArrayList<>();
         for (Pair<IAccessMethod, Index> pair : chosenIndexes) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(pair.first);
-            subRoots.add(pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
-                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx,
-                    AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
-                            subTree.getDataSourceRef().getValue(), afterSelectRefs),
-                    false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
-                            .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                    context, null));
+            boolean retainInput = AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+                    subTree.getDataSourceRef().getValue(), afterSelectRefs);
+            boolean requiresBroadcast = subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
+                    .getExecutionMode() == ExecutionMode.UNPARTITIONED;
+            ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
+                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
+                    requiresBroadcast, context, null);
+            if (subRoot == null) {
+                return false;
+            }
+            subRoots.add(subRoot);
         }
         // Connect each secondary index utilization plan to a common intersect operator.
         ILogicalOperator primaryUnnestOp = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
@@ -262,10 +266,12 @@
         List<List<LogicalVariable>> inputVars = new ArrayList<>(subRoots.size());
         for (int i = 0; i < subRoots.size(); i++) {
             if (lop.getOperatorTag() != subRoots.get(i).getOperatorTag()) {
-                throw new AlgebricksException("The data source root should have the same operator type.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, lop.getSourceLocation(),
+                        "The data source root should have the same operator type.");
             }
             if (lop.getInputs().size() != 1) {
-                throw new AlgebricksException("The primary search has multiple inputs.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, lop.getSourceLocation(),
+                        "The primary search has multiple inputs.");
             }
 
             ILogicalOperator curRoot = subRoots.get(i);
@@ -274,7 +280,8 @@
             for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> orderExpression : order
                     .getOrderExpressions()) {
                 if (orderExpression.second.getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+                            orderExpression.second.getValue().getSourceLocation(),
                             "The order by expression should be variables, but they aren't variables.");
                 }
                 VariableReferenceExpression orderedVar =
@@ -286,6 +293,7 @@
 
         List<LogicalVariable> outputVar = inputVars.get(0);
         IntersectOperator intersect = new IntersectOperator(outputVar, inputVars);
+        intersect.setSourceLocation(lop.getSourceLocation());
         for (ILogicalOperator secondarySearch : subRoots) {
             intersect.getInputs().add(secondarySearch.getInputs().get(0));
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 6f99330..ed73333 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -84,6 +84,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveEditDistanceSearchModifierFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveListEditDistanceSearchModifierFactory;
@@ -431,6 +432,7 @@
             addKeyVarsAndExprs(optFuncExpr, keyVarList, keyExprList, context);
             // Assign operator that sets the secondary-index search-key fields.
             inputOp = new AssignOperator(keyVarList, keyExprList);
+            inputOp.setSourceLocation(dataSourceScan.getSourceLocation());
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
             inputOp.getInputs().add(new MutableObject<>(
                     OperatorManipulationUtil.deepCopy(dataSourceScan.getInputs().get(0).getValue())));
@@ -571,6 +573,7 @@
         // Change join into a select with the same condition.
         SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
                 newNullPlaceHolderVar);
+        topSelect.setSourceLocation(indexPlanRootOp.getSourceLocation());
         topSelect.getInputs().add(indexSubTree.getRootRef());
         topSelect.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(topSelect);
@@ -595,6 +598,7 @@
                         indexSubTreeVar));
             }
             UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
+            unionAllOp.setSourceLocation(topOp.getSourceLocation());
             unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
             unionAllOp.getInputs().add(panicJoinRef);
             unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
@@ -606,9 +610,10 @@
         // The inner (build) branch of the join is the subtree with the data scan, since the result of the similarity join could potentially be big.
         // This choice may not always be the most efficient, but it seems more robust than the alternative.
         Mutable<ILogicalExpression> eqJoinConditionRef =
-                createPrimaryKeysEqJoinCondition(originalSubTreePKs, surrogateSubTreePKs);
+                createPrimaryKeysEqJoinCondition(originalSubTreePKs, surrogateSubTreePKs, topOp.getSourceLocation());
         InnerJoinOperator topEqJoin = new InnerJoinOperator(eqJoinConditionRef, originalProbeSubTreeRootRef,
                 new MutableObject<ILogicalOperator>(topOp));
+        topEqJoin.setSourceLocation(topOp.getSourceLocation());
         topEqJoin.setExecutionMode(ExecutionMode.PARTITIONED);
         joinRef.setValue(topEqJoin);
         context.computeAndSetTypeEnvironmentForOperator(topEqJoin);
@@ -690,23 +695,30 @@
     }
 
     private Mutable<ILogicalExpression> createPrimaryKeysEqJoinCondition(List<LogicalVariable> originalSubTreePKs,
-            List<LogicalVariable> surrogateSubTreePKs) {
+            List<LogicalVariable> surrogateSubTreePKs, SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> eqExprs = new ArrayList<Mutable<ILogicalExpression>>();
         int numPKVars = originalSubTreePKs.size();
         for (int i = 0; i < numPKVars; i++) {
             List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-            args.add(
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(surrogateSubTreePKs.get(i))));
-            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(originalSubTreePKs.get(i))));
-            ILogicalExpression eqFunc =
+            VariableReferenceExpression surrogateSubTreePKRef =
+                    new VariableReferenceExpression(surrogateSubTreePKs.get(i));
+            surrogateSubTreePKRef.setSourceLocation(sourceLoc);
+            args.add(new MutableObject<ILogicalExpression>(surrogateSubTreePKRef));
+            VariableReferenceExpression originalSubTreePKRef =
+                    new VariableReferenceExpression(originalSubTreePKs.get(i));
+            originalSubTreePKRef.setSourceLocation(sourceLoc);
+            args.add(new MutableObject<ILogicalExpression>(originalSubTreePKRef));
+            ScalarFunctionCallExpression eqFunc =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), args);
+            eqFunc.setSourceLocation(sourceLoc);
             eqExprs.add(new MutableObject<ILogicalExpression>(eqFunc));
         }
         if (eqExprs.size() == 1) {
             return eqExprs.get(0);
         } else {
-            ILogicalExpression andFunc = new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression andFunc = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND), eqExprs);
+            andFunc.setSourceLocation(sourceLoc);
             return new MutableObject<ILogicalExpression>(andFunc);
         }
     }
@@ -715,16 +727,19 @@
             OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
             IOptimizableFuncExpr optFuncExpr, Index chosenIndex, Map<LogicalVariable, LogicalVariable> panicVarMap,
             IOptimizationContext context) throws AlgebricksException {
+        ILogicalOperator probeRootOp = probeSubTree.getRoot();
+        SourceLocation sourceLoc = probeRootOp.getSourceLocation();
         LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
 
         // We split the plan into two "branches", and add selections on each side.
         AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
-        replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.getRoot()));
+        replicateOp.setSourceLocation(sourceLoc);
+        replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeRootOp));
         replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
         context.computeAndSetTypeEnvironmentForOperator(replicateOp);
 
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
-        IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.getRoot());
+        IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeRootOp);
         IAType inputSearchVarType;
         if (chosenIndex.isEnforced()) {
             inputSearchVarType = optFuncExpr.getFieldType(optFuncExpr.findLogicalVar(inputSearchVar));
@@ -770,14 +785,16 @@
             IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex,
             IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef,
             Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         // Create select operator for removing tuples that are not filterable.
         // First determine the proper filter function and args based on the type of the input search var.
         ILogicalExpression isFilterableExpr = null;
         switch (inputSearchVarType.getTypeTag()) {
             case STRING: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(4);
-                isFilterableArgs
-                        .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                VariableReferenceExpression inputSearchVarRef = new VariableReferenceExpression(inputSearchVar);
+                inputSearchVarRef.setSourceLocation(sourceLoc);
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(
@@ -793,8 +810,9 @@
             case MULTISET:
             case ARRAY:
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
-                isFilterableArgs
-                        .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                VariableReferenceExpression inputSearchVarRef = new VariableReferenceExpression(inputSearchVar);
+                inputSearchVarRef.setSourceLocation(sourceLoc);
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
                 isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
                 isFilterableExpr = new ScalarFunctionCallExpression(
@@ -802,11 +820,12 @@
                         isFilterableArgs);
                 break;
             default:
-                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
+                throw new CompilationException(ErrorCode.NO_SUPPORTED_TYPE, sourceLoc);
         }
 
         SelectOperator isFilterableSelectOp =
                 new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr), false, null);
+        isFilterableSelectOp.setSourceLocation(sourceLoc);
         isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isFilterableSelectOp);
@@ -814,10 +833,12 @@
         // Select operator for removing tuples that are filterable.
         List<Mutable<ILogicalExpression>> isNotFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>();
         isNotFilterableArgs.add(new MutableObject<ILogicalExpression>(isFilterableExpr));
-        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
+        ScalarFunctionCallExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), isNotFilterableArgs);
+        isNotFilterableExpr.setSourceLocation(sourceLoc);
         SelectOperator isNotFilterableSelectOp =
                 new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr), false, null);
+        isNotFilterableSelectOp.setSourceLocation(sourceLoc);
         isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isNotFilterableSelectOp);
@@ -854,7 +875,8 @@
                 typeTag = ((AUnionType) type).getActualType().getTypeTag();
             }
             if (typeTag != ATypeTag.ARRAY && typeTag != ATypeTag.STRING && typeTag != ATypeTag.MULTISET) {
-                throw CompilationException.create(ErrorCode.NO_SUPPORTED_TYPE);
+                throw new CompilationException(ErrorCode.NO_SUPPORTED_TYPE,
+                        optFuncExpr.getFuncExpr().getSourceLocation());
             }
         }
         jobGenParams.setSearchKeyType(typeTag);
@@ -1246,7 +1268,7 @@
                         index.getGramLength(), prePost, false);
             }
             default: {
-                throw CompilationException.create(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
             }
         }
     }
@@ -1291,12 +1313,12 @@
                         }
                     }
                     default: {
-                        throw CompilationException.create(ErrorCode.INCOMPATIBLE_SEARCH_MODIFIER, searchModifierType,
+                        throw new CompilationException(ErrorCode.INCOMPATIBLE_SEARCH_MODIFIER, searchModifierType,
                                 index.getIndexType());
                     }
                 }
             default:
-                throw CompilationException.create(ErrorCode.UNKNOWN_SEARCH_MODIFIER, searchModifierType);
+                throw new CompilationException(ErrorCode.UNKNOWN_SEARCH_MODIFIER, searchModifierType);
         }
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 7c2edb9..9673da3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -302,7 +302,8 @@
             // Find the dataset corresponding to the datasource in the metadata.
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw CompilationException.create(ErrorCode.NO_METADATA_FOR_DATASET, datasetName);
+                throw new CompilationException(ErrorCode.NO_METADATA_FOR_DATASET, root.getSourceLocation(),
+                        datasetName);
             }
             // Get the record type for that dataset.
             IAType itemType = metadataProvider.findType(ds.getItemTypeDataverseName(), ds.getItemTypeName());
@@ -426,7 +427,7 @@
                 break;
             case NO_DATASOURCE:
             default:
-                throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE);
+                throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE, root.getSourceLocation());
         }
     }
 
@@ -446,7 +447,7 @@
                 return new ArrayList<>();
             case NO_DATASOURCE:
             default:
-                throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE);
+                throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_DATA_SOURCE, root.getSourceLocation());
         }
     }
 
@@ -467,7 +468,8 @@
                     return new ArrayList<>();
                 case NO_DATASOURCE:
                 default:
-                    throw CompilationException.create(ErrorCode.SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE);
+                    throw new CompilationException(ErrorCode.SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE,
+                            root.getSourceLocation());
             }
         } else {
             return null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index f4316037..19558aa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -254,6 +254,7 @@
             // The create MBR function "extracts" one field of an MBR around the given spatial object.
             AbstractFunctionCallExpression createMBR =
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
+            createMBR.setSourceLocation(optFuncExpr.getFuncExpr().getSourceLocation());
             // Spatial object is the constant from the func expr we are optimizing.
             createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr));
             // The number of dimensions
@@ -274,11 +275,13 @@
         if (probeSubTree == null) {
             // We are optimizing a selection query.
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
+            assignSearchKeys.setSourceLocation(dataSourceOp.getSourceLocation());
             assignSearchKeys.getInputs().add(
                     new MutableObject<>(OperatorManipulationUtil.deepCopy(dataSourceOp.getInputs().get(0).getValue())));
             assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
         } else {
             // We are optimizing a join, place the assign op top of the probe subtree.
+            assignSearchKeys.setSourceLocation(probeSubTree.getRoot().getSourceLocation());
             assignSearchKeys.getInputs().add(probeSubTree.getRootRef());
             assignSearchKeys.setExecutionMode(dataSourceOp.getExecutionMode());
             OperatorPropertiesUtil.typeOpRec(probeSubTree.getRootRef(), context);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 8bfa53a..4d646f3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -29,6 +29,8 @@
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -85,6 +87,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  *   This visitor inlines all nested tuple source operators in the query
@@ -218,8 +221,9 @@
         for (LogicalVariable keyVar : correlatedKeyVars) {
             if (!groupKeyVars.contains(keyVar)) {
                 LogicalVariable newVar = context.newVar();
-                op.getGroupByList()
-                        .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+                VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+                keyVarRef.setSourceLocation(op.getSourceLocation());
+                op.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
                 addedGroupKeyMapping.put(keyVar, newVar);
             }
         }
@@ -254,7 +258,8 @@
         VariableUtilities.getSubplanLocalLiveVariables(op.getInputs().get(0).getValue(), inputLiveVars);
 
         // Creates a record construction assign operator.
-        Pair<ILogicalOperator, LogicalVariable> assignOpAndRecordVar = createRecordConstructorAssignOp(inputLiveVars);
+        Pair<ILogicalOperator, LogicalVariable> assignOpAndRecordVar =
+                createRecordConstructorAssignOp(inputLiveVars, op.getSourceLocation());
         ILogicalOperator assignOp = assignOpAndRecordVar.first;
         LogicalVariable recordVar = assignOpAndRecordVar.second;
         ILogicalOperator inputOp = op.getInputs().get(0).getValue();
@@ -267,41 +272,49 @@
         gbyOp.getInputs().add(new MutableObject<>(assignOp));
 
         // Adds an unnest operators on top of the group-by operator.
-        Pair<ILogicalOperator, LogicalVariable> unnestOpAndUnnestVar = createUnnestForAggregatedList(aggVar);
+        Pair<ILogicalOperator, LogicalVariable> unnestOpAndUnnestVar =
+                createUnnestForAggregatedList(aggVar, op.getSourceLocation());
         ILogicalOperator unnestOp = unnestOpAndUnnestVar.first;
         LogicalVariable unnestVar = unnestOpAndUnnestVar.second;
         unnestOp.getInputs().add(new MutableObject<>(gbyOp));
 
         // Adds field accesses to recover input live variables.
-        ILogicalOperator fieldAccessAssignOp = createFieldAccessAssignOperator(unnestVar, inputLiveVars);
+        ILogicalOperator fieldAccessAssignOp =
+                createFieldAccessAssignOperator(unnestVar, inputLiveVars, op.getSourceLocation());
         fieldAccessAssignOp.getInputs().add(new MutableObject<>(unnestOp));
 
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(fieldAccessAssignOp, context);
         return fieldAccessAssignOp;
     }
 
-    private Pair<ILogicalOperator, LogicalVariable> createRecordConstructorAssignOp(
-            Set<LogicalVariable> inputLiveVars) {
+    private Pair<ILogicalOperator, LogicalVariable> createRecordConstructorAssignOp(Set<LogicalVariable> inputLiveVars,
+            SourceLocation sourceLoc) {
         // Creates a nested record.
         List<Mutable<ILogicalExpression>> recordConstructorArgs = new ArrayList<>();
         for (LogicalVariable inputLiveVar : inputLiveVars) {
             if (!correlatedKeyVars.contains(inputLiveVar)) {
                 recordConstructorArgs.add(new MutableObject<>(new ConstantExpression(
                         new AsterixConstantValue(new AString(Integer.toString(inputLiveVar.getId()))))));
-                recordConstructorArgs.add(new MutableObject<>(new VariableReferenceExpression(inputLiveVar)));
+                VariableReferenceExpression inputLiveVarRef = new VariableReferenceExpression(inputLiveVar);
+                inputLiveVarRef.setSourceLocation(sourceLoc);
+                recordConstructorArgs.add(new MutableObject<>(inputLiveVarRef));
             }
         }
         LogicalVariable recordVar = context.newVar();
-        Mutable<ILogicalExpression> recordExprRef =
-                new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), recordConstructorArgs));
+        ScalarFunctionCallExpression openRecConstr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), recordConstructorArgs);
+        openRecConstr.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> recordExprRef = new MutableObject<ILogicalExpression>(openRecConstr);
         AssignOperator assignOp = new AssignOperator(recordVar, recordExprRef);
+        assignOp.setSourceLocation(sourceLoc);
         return new Pair<>(assignOp, recordVar);
     }
 
     private Pair<ILogicalOperator, LogicalVariable> wrapLimitInGroupBy(ILogicalOperator op, LogicalVariable recordVar,
             Set<LogicalVariable> inputLiveVars) throws AlgebricksException {
+        SourceLocation sourceLoc = op.getSourceLocation();
         GroupByOperator gbyOp = new GroupByOperator();
+        gbyOp.setSourceLocation(sourceLoc);
         List<Pair<LogicalVariable, LogicalVariable>> keyVarNewVarPairs = new ArrayList<>();
         for (LogicalVariable keyVar : correlatedKeyVars) {
             // This limits the visitor can only be applied to a nested logical
@@ -309,8 +322,9 @@
             // where the keyVarsToEnforce forms a candidate key which can
             // uniquely identify a tuple out of the nested-tuple-source.
             LogicalVariable newVar = context.newVar();
-            gbyOp.getGroupByList()
-                    .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(sourceLoc);
+            gbyOp.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
             keyVarNewVarPairs.add(new Pair<>(keyVar, newVar));
         }
 
@@ -322,11 +336,15 @@
         List<Mutable<ILogicalExpression>> aggArgList = new ArrayList<>();
         aggVarList.add(aggVar);
         // Creates an aggregation function expression.
-        aggArgList.add(new MutableObject<>(new VariableReferenceExpression(recordVar)));
-        ILogicalExpression aggExpr = new AggregateFunctionCallExpression(
+        VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+        recordVarRef.setSourceLocation(sourceLoc);
+        aggArgList.add(new MutableObject<>(recordVarRef));
+        AggregateFunctionCallExpression aggExpr = new AggregateFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false, aggArgList);
+        aggExpr.setSourceLocation(sourceLoc);
         aggExprList.add(new MutableObject<>(aggExpr));
         AggregateOperator aggOp = new AggregateOperator(aggVarList, aggExprList);
+        aggOp.setSourceLocation(sourceLoc);
 
         // Adds the original limit operator as the input operator to the added
         // aggregate operator.
@@ -335,6 +353,7 @@
         ILogicalOperator currentOp = op;
         if (!orderingExprs.isEmpty()) {
             OrderOperator orderOp = new OrderOperator(cloneOrderingExpression(orderingExprs));
+            orderOp.setSourceLocation(sourceLoc);
             op.getInputs().add(new MutableObject<>(orderOp));
             currentOp = orderOp;
         }
@@ -342,6 +361,7 @@
         // Adds a nested tuple source operator as the input operator to the
         // limit operator.
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(gbyOp));
+        nts.setSourceLocation(sourceLoc);
         currentOp.getInputs().add(new MutableObject<>(nts));
 
         // Sets the root of the added nested plan to the aggregate operator.
@@ -358,20 +378,25 @@
         return new Pair<>(gbyOp, aggVar);
     }
 
-    private Pair<ILogicalOperator, LogicalVariable> createUnnestForAggregatedList(LogicalVariable aggVar) {
+    private Pair<ILogicalOperator, LogicalVariable> createUnnestForAggregatedList(LogicalVariable aggVar,
+            SourceLocation sourceLoc) {
         LogicalVariable unnestVar = context.newVar();
         // Creates an unnest function expression.
-        Mutable<ILogicalExpression> unnestArg = new MutableObject<>(new VariableReferenceExpression(aggVar));
+        VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar);
+        aggVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> unnestArg = new MutableObject<>(aggVarRef);
         List<Mutable<ILogicalExpression>> unnestArgList = new ArrayList<>();
         unnestArgList.add(unnestArg);
-        Mutable<ILogicalExpression> unnestExpr = new MutableObject<>(new UnnestingFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), unnestArgList));
-        ILogicalOperator unnestOp = new UnnestOperator(unnestVar, unnestExpr);
+        UnnestingFunctionCallExpression unnestExpr = new UnnestingFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), unnestArgList);
+        unnestExpr.setSourceLocation(sourceLoc);
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr));
+        unnestOp.setSourceLocation(sourceLoc);
         return new Pair<>(unnestOp, unnestVar);
     }
 
     private ILogicalOperator createFieldAccessAssignOperator(LogicalVariable recordVar,
-            Set<LogicalVariable> inputLiveVars) {
+            Set<LogicalVariable> inputLiveVars, SourceLocation sourceLoc) {
         List<LogicalVariable> fieldAccessVars = new ArrayList<>();
         List<Mutable<ILogicalExpression>> fieldAccessExprs = new ArrayList<>();
         // Adds field access by name.
@@ -382,16 +407,22 @@
                 fieldAccessVars.add(newVar);
                 // fieldAcess expr
                 List<Mutable<ILogicalExpression>> argRefs = new ArrayList<>();
-                argRefs.add(new MutableObject<>(new VariableReferenceExpression(recordVar)));
+                VariableReferenceExpression recordVarRef = new VariableReferenceExpression(recordVar);
+                recordVarRef.setSourceLocation(sourceLoc);
+                argRefs.add(new MutableObject<>(recordVarRef));
                 argRefs.add(new MutableObject<>(new ConstantExpression(
                         new AsterixConstantValue(new AString(Integer.toString(inputLiveVar.getId()))))));
-                fieldAccessExprs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argRefs)));
+                ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argRefs);
+                faExpr.setSourceLocation(sourceLoc);
+                fieldAccessExprs.add(new MutableObject<>(faExpr));
                 // Updates variable mapping for ancestor operators.
                 updateInputToOutputVarMapping(inputLiveVar, newVar, false);
             }
         }
-        return new AssignOperator(fieldAccessVars, fieldAccessExprs);
+        AssignOperator assignOp = new AssignOperator(fieldAccessVars, fieldAccessExprs);
+        assignOp.setSourceLocation(sourceLoc);
+        return assignOp;
     }
 
     @Override
@@ -445,13 +476,15 @@
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>();
         // Adds keyVars to the prefix of sorting columns.
         for (LogicalVariable keyVar : correlatedKeyVars) {
-            orderExprList.add(
-                    new Pair<>(OrderOperator.ASC_ORDER, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(op.getSourceLocation());
+            orderExprList.add(new Pair<>(OrderOperator.ASC_ORDER, new MutableObject<>(keyVarRef)));
         }
         orderExprList.addAll(op.getOrderExpressions());
 
         // Creates an order operator with the new expression list.
         OrderOperator orderOp = new OrderOperator(orderExprList);
+        orderOp.setSourceLocation(op.getSourceLocation());
         orderOp.getInputs().addAll(op.getInputs());
         context.computeAndSetTypeEnvironmentForOperator(orderOp);
         return orderOp;
@@ -529,8 +562,8 @@
         // Update the variable mappings
         List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varTriples = op.getVariableMappings();
         for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple : varTriples) {
-            updateInputToOutputVarMapping(triple.third, triple.first, false);
-            updateInputToOutputVarMapping(triple.second, triple.first, false);
+            updateInputToOutputVarMapping(triple.first, triple.third, false);
+            updateInputToOutputVarMapping(triple.second, triple.third, false);
         }
         return op;
     }
@@ -542,7 +575,8 @@
         for (int i = 0; i < op.getNumInput(); i++) {
             List<LogicalVariable> inputVars = op.getInputVariables(i);
             if (inputVars.size() != outputVars.size()) {
-                throw new AlgebricksException("The cardinality of input and output are not equal for Intersection");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                        "The cardinality of input and output are not equal for Intersection");
             }
             for (int j = 0; j < inputVars.size(); j++) {
                 updateInputToOutputVarMapping(inputVars.get(j), outputVars.get(j), false);
@@ -576,7 +610,7 @@
     @Override
     public ILogicalOperator visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void arg)
             throws AlgebricksException {
-        throw new AlgebricksException(
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
                 "The subquery de-correlation rule should always be applied before index-access-method related rules.");
     }
 
@@ -626,7 +660,9 @@
         if (correlatedKeyVars.isEmpty()) {
             return op;
         }
+        SourceLocation sourceLoc = op.getSourceLocation();
         GroupByOperator gbyOp = new GroupByOperator();
+        gbyOp.setSourceLocation(sourceLoc);
         // Creates a copy of correlatedKeyVars, to fix the ConcurrentModificationException in ASTERIXDB-1581.
         List<LogicalVariable> copyOfCorrelatedKeyVars = new ArrayList<>(correlatedKeyVars);
         for (LogicalVariable keyVar : copyOfCorrelatedKeyVars) {
@@ -635,8 +671,9 @@
             // where the keyVarsToEnforce forms a candidate key which can
             // uniquely identify a tuple out of the nested-tuple-source.
             LogicalVariable newVar = context.newVar();
-            gbyOp.getGroupByList()
-                    .add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(keyVar))));
+            VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+            keyVarRef.setSourceLocation(sourceLoc);
+            gbyOp.getGroupByList().add(new Pair<>(newVar, new MutableObject<>(keyVarRef)));
             updateInputToOutputVarMapping(keyVar, newVar, false);
         }
 
@@ -644,6 +681,7 @@
         gbyOp.getInputs().add(new MutableObject<>(inputOp));
 
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(gbyOp));
+        nts.setSourceLocation(sourceLoc);
         op.getInputs().clear();
         op.getInputs().add(new MutableObject<>(nts));
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 4a28344..610fea2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -192,6 +192,7 @@
         // After rewriting, the original inner join should become an left outer join.
         if (rewritten) {
             returnOp = new LeftOuterJoinOperator(op.getCondition());
+            returnOp.setSourceLocation(op.getSourceLocation());
             returnOp.getInputs().addAll(op.getInputs());
             injectNullCheckVars(returnOp);
         }
@@ -231,13 +232,16 @@
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>();
         // Adds keyVars to the prefix of sorting columns.
         for (LogicalVariable liveVar : liveVarsFromSubplanInput) {
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(op.getSourceLocation());
             orderExprList.add(new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER,
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(liveVar))));
+                    new MutableObject<ILogicalExpression>(liveVarRef)));
         }
         orderExprList.addAll(op.getOrderExpressions());
 
         // Creates an order operator with the new expression list.
         OrderOperator orderOp = new OrderOperator(orderExprList);
+        orderOp.setSourceLocation(op.getSourceLocation());
         orderOp.getInputs().addAll(op.getInputs());
         context.computeAndSetTypeEnvironmentForOperator(orderOp);
         return orderOp;
@@ -399,8 +403,9 @@
      */
     private void injectNullCheckVars(AbstractBinaryJoinOperator joinOp) {
         LogicalVariable assignVar = context.newVar();
-        ILogicalOperator assignOp =
+        AssignOperator assignOp =
                 new AssignOperator(assignVar, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        assignOp.setSourceLocation(joinOp.getSourceLocation());
         assignOp.getInputs().add(joinOp.getInputs().get(1));
         joinOp.getInputs().set(1, new MutableObject<ILogicalOperator>(assignOp));
         nullCheckVars.add(assignVar);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index 3edccec..62f89f1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -56,6 +56,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /*
 This rule  is to remove SubplanOperators containing DataScan, InnerJoin, LeftOuterJoin.
@@ -356,6 +357,7 @@
                 LogicalOperatorTag.INNERJOIN, LogicalOperatorTag.LEFTOUTERJOIN))) {
             return new Pair<>(false, new LinkedHashMap<>());
         }
+        SourceLocation sourceLoc = subplanOp.getSourceLocation();
         Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
         ILogicalOperator inputOpBackup = inputOpRef.getValue();
         // Creates parameters for the left outer join operator.
@@ -382,7 +384,8 @@
 
         // Creates a variable to indicate whether a left input tuple is killed in the plan rooted at rightInputOp.
         LogicalVariable assignVar = context.newVar();
-        ILogicalOperator assignOp = new AssignOperator(assignVar, new MutableObject<>(ConstantExpression.TRUE));
+        AssignOperator assignOp = new AssignOperator(assignVar, new MutableObject<>(ConstantExpression.TRUE));
+        assignOp.setSourceLocation(rightInputOp.getSourceLocation());
         assignOp.getInputs().add(rightInputOpRef);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         rightInputOpRef = new MutableObject<>(assignOp);
@@ -391,20 +394,31 @@
         List<Mutable<ILogicalExpression>> joinPredicates = new ArrayList<>();
         for (LogicalVariable liveVar : primaryKeyVars) {
             List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
-            arguments.add(new MutableObject<>(new VariableReferenceExpression(liveVar)));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            arguments.add(new MutableObject<>(liveVarRef));
             LogicalVariable rightVar = varMap.get(liveVar);
-            arguments.add(new MutableObject<>(new VariableReferenceExpression(rightVar)));
-            ILogicalExpression expr = new ScalarFunctionCallExpression(
+            VariableReferenceExpression rightVarRef = new VariableReferenceExpression(rightVar);
+            rightVarRef.setSourceLocation(sourceLoc);
+            arguments.add(new MutableObject<>(rightVarRef));
+            ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), arguments);
+            expr.setSourceLocation(sourceLoc);
             joinPredicates.add(new MutableObject<>(expr));
         }
 
-        ILogicalExpression joinExpr = joinPredicates.size() > 1
-                ? new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND),
-                        joinPredicates)
-                : joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
+        ILogicalExpression joinExpr;
+        if (joinPredicates.size() > 1) {
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.AND), joinPredicates);
+            andExpr.setSourceLocation(sourceLoc);
+            joinExpr = andExpr;
+        } else {
+            joinExpr = joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
+        }
         LeftOuterJoinOperator leftOuterJoinOp =
                 new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef, rightInputOpRef);
+        leftOuterJoinOp.setSourceLocation(sourceLoc);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rightInputOp, context);
         context.computeAndSetTypeEnvironmentForOperator(leftOuterJoinOp);
 
@@ -413,11 +427,14 @@
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByDecorList = new ArrayList<>();
         List<ILogicalPlan> nestedPlans = new ArrayList<>();
         GroupByOperator groupbyOp = new GroupByOperator(groupByList, groupByDecorList, nestedPlans);
+        groupbyOp.setSourceLocation(sourceLoc);
 
         LinkedHashMap<LogicalVariable, LogicalVariable> replacedVarMap = new LinkedHashMap<>();
         for (LogicalVariable liveVar : primaryKeyVars) {
             LogicalVariable newVar = context.newVar();
-            groupByList.add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(inputOpBackup.getSourceLocation());
+            groupByList.add(new Pair<>(newVar, new MutableObject<>(liveVarRef)));
             // Adds variables for replacements in ancestors.
             replacedVarMap.put(liveVar, newVar);
         }
@@ -425,7 +442,9 @@
             if (primaryKeyVars.contains(liveVar)) {
                 continue;
             }
-            groupByDecorList.add(new Pair<>(null, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            groupByDecorList.add(new Pair<>(null, new MutableObject<>(liveVarRef)));
         }
 
         // Sets up the nested plan for the groupby operator.
@@ -436,23 +455,33 @@
         List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprs = varMapAndOrderExprs.second;
         if (!orderExprs.isEmpty()) {
             OrderOperator orderOp = new OrderOperator(orderExprs);
+            orderOp.setSourceLocation(sourceLoc);
             currentOpRef = new MutableObject<>(orderOp);
             lowestAggregateRefInSubplan.getValue().getInputs().add(currentOpRef);
         }
 
         // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
         // i.e., subplan input tuples that are filtered out within a subplan.
-        Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(new VariableReferenceExpression(assignVar));
+        VariableReferenceExpression assignVarRef = new VariableReferenceExpression(assignVar);
+        assignVarRef.setSourceLocation(sourceLoc);
+        Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(assignVarRef);
         List<Mutable<ILogicalExpression>> args = new ArrayList<>();
         args.add(filterVarExpr);
         List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-        argsForNotFunction.add(new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args)));
-        SelectOperator selectOp = new SelectOperator(new MutableObject<>(new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction)), false, null);
+        ScalarFunctionCallExpression isMissingExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+        isMissingExpr.setSourceLocation(sourceLoc);
+        argsForNotFunction.add(new MutableObject<>(isMissingExpr));
+        ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
+        notExpr.setSourceLocation(sourceLoc);
+        SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr), false, null);
+        selectOp.setSourceLocation(sourceLoc);
         currentOpRef.getValue().getInputs().add(new MutableObject<>(selectOp));
 
-        selectOp.getInputs().add(new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp))));
+        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+        ntsOp.setSourceLocation(sourceLoc);
+        selectOp.getInputs().add(new MutableObject<>(ntsOp));
         List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<>();
         nestedRoots.add(aggOpRef);
         nestedPlans.add(new ALogicalPlanImpl(nestedRoots));
@@ -476,6 +505,7 @@
     private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applySpecialFlattening(
             Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
+        SourceLocation sourceLoc = subplanOp.getSourceLocation();
         Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
         LinkedHashMap<LogicalVariable, LogicalVariable> replacedVarMap = new LinkedHashMap<>();
 
@@ -508,10 +538,13 @@
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByList = new ArrayList<>();
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> groupByDecorList = new ArrayList<>();
         GroupByOperator groupbyOp = new GroupByOperator(groupByList, groupByDecorList, subplanOp.getNestedPlans());
+        groupbyOp.setSourceLocation(sourceLoc);
 
         for (LogicalVariable coverVar : primaryKeyVars) {
             LogicalVariable newVar = context.newVar();
-            groupByList.add(new Pair<>(newVar, new MutableObject<>(new VariableReferenceExpression(coverVar))));
+            VariableReferenceExpression coverVarRef = new VariableReferenceExpression(coverVar);
+            coverVarRef.setSourceLocation(sourceLoc);
+            groupByList.add(new Pair<>(newVar, new MutableObject<>(coverVarRef)));
             // Adds variables for replacements in ancestors.
             replacedVarMap.put(coverVar, newVar);
         }
@@ -519,7 +552,9 @@
             if (primaryKeyVars.contains(liveVar)) {
                 continue;
             }
-            groupByDecorList.add(new Pair<>(null, new MutableObject<>(new VariableReferenceExpression(liveVar))));
+            VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+            liveVarRef.setSourceLocation(sourceLoc);
+            groupByDecorList.add(new Pair<>(null, new MutableObject<>(liveVarRef)));
         }
         groupbyOp.getInputs().add(new MutableObject<>(topJoinRef.getValue()));
 
@@ -528,28 +563,42 @@
             // i.e., subplan input tuples that are filtered out within a subplan.
             List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
             for (LogicalVariable notNullVar : notNullVars) {
-                Mutable<ILogicalExpression> filterVarExpr =
-                        new MutableObject<>(new VariableReferenceExpression(notNullVar));
+                VariableReferenceExpression notNullVarRef = new VariableReferenceExpression(notNullVar);
+                notNullVarRef.setSourceLocation(sourceLoc);
+                Mutable<ILogicalExpression> filterVarExpr = new MutableObject<>(notNullVarRef);
                 List<Mutable<ILogicalExpression>> args = new ArrayList<>();
                 args.add(filterVarExpr);
                 List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-                argsForNotFunction.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args)));
-                nullCheckExprRefs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction)));
+                ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+                isMissingExpr.setSourceLocation(sourceLoc);
+                argsForNotFunction.add(new MutableObject<>(isMissingExpr));
+                ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
+                notExpr.setSourceLocation(sourceLoc);
+                nullCheckExprRefs.add(new MutableObject<>(notExpr));
             }
-            Mutable<ILogicalExpression> selectExprRef = nullCheckExprRefs.size() > 1
-                    ? new MutableObject<>(new ScalarFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), nullCheckExprRefs))
-                    : nullCheckExprRefs.get(0);
+            Mutable<ILogicalExpression> selectExprRef;
+            if (nullCheckExprRefs.size() > 1) {
+                ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), nullCheckExprRefs);
+                andExpr.setSourceLocation(sourceLoc);
+                selectExprRef = new MutableObject<>(andExpr);
+            } else {
+                selectExprRef = nullCheckExprRefs.get(0);
+            }
             SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+            selectOp.setSourceLocation(sourceLoc);
             topJoinRef.setValue(selectOp);
-            selectOp.getInputs()
-                    .add(new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp))));
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            selectOp.getInputs().add(new MutableObject<>(ntsOp));
         } else {
             // The original join operator in the Subplan is a left-outer join.
             // Therefore, no null-check variable is injected and no SelectOperator needs to be added.
-            topJoinRef.setValue(new NestedTupleSourceOperator(new MutableObject<>(groupbyOp)));
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            topJoinRef.setValue(ntsOp);
         }
         opRef.setValue(groupbyOp);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(groupbyOp, context);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
index 0d53a19..5b85cb6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
@@ -212,7 +212,10 @@
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval) {
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
         intervalArg.add(new MutableObject<ILogicalExpression>(interval));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        ScalarFunctionCallExpression fnExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        fnExpr.setSourceLocation(interval.getSourceLocation());
+        return fnExpr;
     }
 
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval1,
@@ -220,7 +223,10 @@
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
         intervalArg.add(new MutableObject<ILogicalExpression>(interval1));
         intervalArg.add(new MutableObject<ILogicalExpression>(interval2));
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        ScalarFunctionCallExpression fnExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
+        fnExpr.setSourceLocation(interval1.getSourceLocation());
+        return fnExpr;
     }
 
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index efea208..4b2ff76 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -25,6 +25,8 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AString;
@@ -165,7 +167,7 @@
                 }
             }
             if (!compatible(reqType, inputType)) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, funcExpr.getSourceLocation(),
                         "type mismatch, required: " + reqType.toString() + " actual: " + inputType.toString());
             }
             return changed;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
index 82d963d..1593cf1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
@@ -116,11 +116,13 @@
                 fieldIndexInRecord = metaFieldNameToIndexMap.get(pkFieldName);
             }
             LogicalVariable var = indexSearchVars.get(pkIndex);
-            ILogicalExpression expr = new ScalarFunctionCallExpression(
+            VariableReferenceExpression referredRecordVarRef = new VariableReferenceExpression(referredRecordVar);
+            referredRecordVarRef.setSourceLocation(operator.getSourceLocation());
+            ScalarFunctionCallExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(referredRecordVar)),
-                    new MutableObject<ILogicalExpression>(
+                    new MutableObject<ILogicalExpression>(referredRecordVarRef), new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(fieldIndexInRecord)))));
+            expr.setSourceLocation(operator.getSourceLocation());
             EquivalenceClass equivClass =
                     new EquivalenceClass(Collections.singletonList(var), var, Collections.singletonList(expr));
             Map<LogicalVariable, EquivalenceClass> equivalenceMap = context.getEquivalenceClassMap(operator);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
index 81c5f4c..e777c4e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -170,7 +170,7 @@
             throws CompilationException {
         List<ILangExpression> inputExprs = new ArrayList<>();
         inputExprs.addAll(unionExpr.getExprs());
-        Pair<ILogicalOperator, LogicalVariable> result = translateUnionAllFromInputExprs(inputExprs, tupSource);
+        Pair<ILogicalOperator, LogicalVariable> result = translateUnionAllFromInputExprs(inputExprs, tupSource, null);
         return aggListifyForSubquery(result.second, new MutableObject<>(result.first), false);
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslatorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslatorFactory.java
index a3e9d74..911c443 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslatorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslatorFactory.java
@@ -18,16 +18,20 @@
  */
 package org.apache.asterix.translator;
 
+import java.util.Map;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class AqlExpressionToPlanTranslatorFactory implements ILangExpressionToPlanTranslatorFactory {
 
     @Override
     public ILangExpressionToPlanTranslator createExpressionToPlanTranslator(MetadataProvider metadataProvider,
-            int currentVarCounter) throws AlgebricksException {
+            int currentVarCounter, Map<VarIdentifier, IAObject> externalVars) throws AlgebricksException {
         return new AqlExpressionToPlanTranslator(metadataProvider, currentVarCounter);
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
new file mode 100644
index 0000000..a9bd856
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.translator;
+
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class BaseClientRequest implements IClientRequest {
+    protected final IStatementExecutorContext ctx;
+    protected final String contextId;
+    private boolean complete;
+
+    public BaseClientRequest(IStatementExecutorContext ctx, String contextId) {
+        this.ctx = ctx;
+        this.contextId = contextId;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (complete) {
+            return;
+        }
+        complete = true;
+        ctx.remove(contextId);
+    }
+
+    @Override
+    public synchronized void cancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        if (complete) {
+            return;
+        }
+        complete();
+        doCancel(appCtx);
+    }
+
+    protected abstract void doCancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
new file mode 100644
index 0000000..520ce03
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.translator;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+
+public class ClientJobRequest extends BaseClientRequest {
+    private final JobId jobId;
+
+    public ClientJobRequest(IStatementExecutorContext ctx, String clientCtxId, JobId jobId) {
+        super(ctx, clientCtxId);
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doCancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        IHyracksClientConnection hcc = appCtx.getHcc();
+        try {
+            hcc.cancelJob(jobId);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        ctx.remove(contextId);
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 3a4682e..5950329 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * An AQL statement instance is translated into an instance of type CompileX
@@ -36,11 +37,24 @@
 public class CompiledStatements {
 
     public interface ICompiledStatement {
-
         Statement.Kind getKind();
+
+        SourceLocation getSourceLocation();
     }
 
-    public static class CompiledDatasetDropStatement implements ICompiledStatement {
+    public static abstract class AbstractCompiledStatement implements ICompiledStatement {
+        private SourceLocation sourceLoc;
+
+        public void setSourceLocation(SourceLocation sourceLoc) {
+            this.sourceLoc = sourceLoc;
+        }
+
+        public SourceLocation getSourceLocation() {
+            return sourceLoc;
+        }
+    }
+
+    public static class CompiledDatasetDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
 
@@ -64,7 +78,7 @@
     }
 
     // added by yasser
-    public static class CompiledCreateDataverseStatement implements ICompiledStatement {
+    public static class CompiledCreateDataverseStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String format;
 
@@ -87,7 +101,7 @@
         }
     }
 
-    public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+    public static class CompiledNodeGroupDropStatement extends AbstractCompiledStatement {
         private final String nodeGroupName;
 
         public CompiledNodeGroupDropStatement(String nodeGroupName) {
@@ -104,7 +118,7 @@
         }
     }
 
-    public static class CompiledIndexDropStatement implements ICompiledStatement {
+    public static class CompiledIndexDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
         private final String indexName;
@@ -133,7 +147,7 @@
         }
     }
 
-    public static class CompiledDataverseDropStatement implements ICompiledStatement {
+    public static class CompiledDataverseDropStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final boolean ifExists;
 
@@ -156,7 +170,7 @@
         }
     }
 
-    public static class CompiledTypeDropStatement implements ICompiledStatement {
+    public static class CompiledTypeDropStatement extends AbstractCompiledStatement {
         private final String typeName;
 
         public CompiledTypeDropStatement(String nodeGroupName) {
@@ -173,14 +187,15 @@
         }
     }
 
-    public static interface ICompiledDmlStatement extends ICompiledStatement {
+    public interface ICompiledDmlStatement extends ICompiledStatement {
 
-        public String getDataverseName();
+        String getDataverseName();
 
-        public String getDatasetName();
+        String getDatasetName();
     }
 
-    public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
+    public static class CompiledCreateIndexStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
         private final Dataset dataset;
         private final Index index;
 
@@ -213,7 +228,8 @@
         }
     }
 
-    public static class CompiledLoadFromFileStatement implements ICompiledDmlStatement {
+    public static class CompiledLoadFromFileStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final boolean alreadySorted;
@@ -257,7 +273,7 @@
         }
     }
 
-    public static class CompiledInsertStatement implements ICompiledDmlStatement {
+    public static class CompiledInsertStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final Query query;
@@ -320,7 +336,8 @@
         }
     }
 
-    public static class CompiledSubscribeFeedStatement implements ICompiledDmlStatement {
+    public static class CompiledSubscribeFeedStatement extends AbstractCompiledStatement
+            implements ICompiledDmlStatement {
 
         private FeedConnectionRequest request;
         private final int varCounter;
@@ -354,7 +371,7 @@
         }
     }
 
-    public static class CompiledDeleteStatement implements ICompiledDmlStatement {
+    public static class CompiledDeleteStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;
         private final Expression condition;
@@ -399,7 +416,7 @@
 
     }
 
-    public static class CompiledCompactStatement implements ICompiledStatement {
+    public static class CompiledCompactStatement extends AbstractCompiledStatement {
         private final String dataverseName;
         private final String datasetName;
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
index d77164c..511b74c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
@@ -21,6 +21,7 @@
 import java.io.Serializable;
 
 public class ExecutionPlans implements Serializable {
+    private static final long serialVersionUID = 6853904213354224457L;
 
     private String expressionTree;
     private String rewrittenExpressionTree;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
index a1fbac6..58f0997 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
@@ -20,15 +20,16 @@
 
 import java.util.Map;
 
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
 
 public interface IRequestParameters {
 
     /**
-     * @return A Hyracks dataset client object that is used to read the results.
+     * @return A Resultset client object that is used to read the results.
      */
-    IHyracksDataset getHyracksDataset();
+    IResultSet getResultSet();
 
     /**
      * Gets the required result properties of the request.
@@ -56,4 +57,14 @@
      * @return Optional request parameters. Otherwise null.
      */
     Map<String, String> getOptionalParameters();
+
+    /**
+     * @return Statement parameters
+     */
+    Map<String, IAObject> getStatementParameters();
+
+    /**
+     * @return true if the request accepts multiple statements. Otherwise, false.
+     */
+    boolean isMultiStatement();
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index 0ff877b..9bc86da 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -22,18 +22,21 @@
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.commons.lang3.tuple.Triple;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
@@ -72,6 +75,8 @@
     }
 
     class Stats implements Serializable {
+        private static final long serialVersionUID = 5885273238208454610L;
+
         private long count;
         private long size;
         private long processedObjects;
@@ -123,6 +128,9 @@
      *            The query to be compiled
      * @param dmlStatement
      *            The data modification statement when the query results in a modification to a dataset
+     * @param statementParameters
+     *            Statement parameters
+     * @param statementRewriter
      * @return the compiled {@code JobSpecification}
      * @throws AsterixException
      * @throws RemoteException
@@ -130,7 +138,8 @@
      * @throws ACIDException
      */
     JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
-            Query query, ICompiledDmlStatement dmlStatement) throws RemoteException, AlgebricksException, ACIDException;
+            Query query, ICompiledDmlStatement dmlStatement, Map<String, IAObject> statementParameters,
+            IStatementRewriter statementRewriter) throws RemoteException, AlgebricksException, ACIDException;
 
     /**
      * returns the active dataverse for an entity or a statement
@@ -148,5 +157,4 @@
      * @return the executions plans
      */
     ExecutionPlans getExecutionPlans();
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
index 81e1ebf..78080f3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
@@ -19,32 +19,31 @@
 
 package org.apache.asterix.translator;
 
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
 
 /**
- * The context for statement executors, which maintains the meta information of all queries.
- * TODO(yingyi): also maintain the mapping from server generated request ids to jobs.
+ * The context for statement executors. Maintains ongoing user requests.
  */
 public interface IStatementExecutorContext {
 
     /**
-     * Gets the Hyracks JobId from the user-provided client context id.
+     * Gets the client request from the user-provided client context id.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @return the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
+     * @return the client request
      */
-    JobId getJobIdFromClientContextId(String clientContextId);
+    IClientRequest get(String clientContextId);
 
     /**
-     * Puts a client context id for a statement and the corresponding Hyracks job id.
+     * Puts a client context id for a statement and the corresponding request.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @param jobId,
+     * @param req,
      *            the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
      */
-    void put(String clientContextId, JobId jobId);
+    void put(String clientContextId, IClientRequest req);
 
     /**
      * Removes the information about the query corresponding to a user-provided client context id.
@@ -52,5 +51,5 @@
      * @param clientContextId,
      *            a user provided client context id.
      */
-    JobId removeJobIdFromClientContextId(String clientContextId);
+    IClientRequest remove(String clientContextId);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 6ab76fa..91de474 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -35,14 +35,13 @@
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -85,12 +84,15 @@
 import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.FunctionInfo;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
@@ -100,7 +102,6 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
@@ -151,6 +152,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 
@@ -192,20 +194,21 @@
     @Override
     public ILogicalPlan translateLoad(ICompiledDmlStatement stmt) throws AlgebricksException {
         CompiledLoadFromFileStatement clffs = (CompiledLoadFromFileStatement) stmt;
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         Dataset dataset = metadataProvider.findDataset(clffs.getDataverseName(), clffs.getDatasetName());
         if (dataset == null) {
             // This would never happen since we check for this in AqlTranslator
-            throw new AlgebricksException(
-                    "Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, clffs.getDatasetName(),
+                    clffs.getDataverseName());
         }
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
         IAType metaItemType =
                 metadataProvider.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         DatasetDataSource targetDatasource =
-                validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+                validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName(), sourceLoc);
         List<List<String>> partitionKeys = targetDatasource.getDataset().getPrimaryKeys();
         if (dataset.hasMetaPart()) {
-            throw new AlgebricksException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     dataset.getDatasetName() + ": load dataset is not supported on Datasets with Meta records");
         }
 
@@ -213,7 +216,7 @@
         try {
             lds = new LoadableDataSource(dataset, itemType, metaItemType, clffs.getAdapter(), clffs.getProperties());
         } catch (IOException e) {
-            throw new AlgebricksException(e);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, e.toString(), e);
         }
 
         // etsOp is a dummy input operator used to keep the compiler happy. it
@@ -229,7 +232,11 @@
         // Create a scan operator and make the empty tuple source its input
         DataSourceScanOperator dssOp = new DataSourceScanOperator(payloadVars, lds);
         dssOp.getInputs().add(new MutableObject<>(etsOp));
-        ILogicalExpression payloadExpr = new VariableReferenceExpression(payloadVars.get(0));
+        dssOp.setSourceLocation(sourceLoc);
+
+        VariableReferenceExpression payloadExpr = new VariableReferenceExpression(payloadVars.get(0));
+        payloadExpr.setSourceLocation(sourceLoc);
+
         Mutable<ILogicalExpression> payloadRef = new MutableObject<>(payloadExpr);
 
         // Creating the assign to extract the PK out of the record
@@ -239,11 +246,12 @@
         LogicalVariable payloadVar = payloadVars.get(0);
         for (List<String> keyFieldName : partitionKeys) {
             PlanTranslationUtil.prepareVarAndExpression(keyFieldName, payloadVar, pkVars, pkExprs, varRefsForLoading,
-                    context);
+                    context, sourceLoc);
         }
 
         AssignOperator assign = new AssignOperator(pkVars, pkExprs);
         assign.getInputs().add(new MutableObject<>(dssOp));
+        assign.setSourceLocation(sourceLoc);
 
         // If the input is pre-sorted, we set the ordering property explicitly in the
         // assign
@@ -265,14 +273,16 @@
             additionalFilteringAssignExpressions = new ArrayList<>();
             additionalFilteringExpressions = new ArrayList<>();
             PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, payloadVar, additionalFilteringVars,
-                    additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
+                    additionalFilteringAssignExpressions, additionalFilteringExpressions, context, sourceLoc);
             additionalFilteringAssign =
                     new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
+            additionalFilteringAssign.setSourceLocation(sourceLoc);
         }
 
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, payloadRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, true);
         insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+        insertOp.setSourceLocation(sourceLoc);
 
         if (additionalFilteringAssign != null) {
             additionalFilteringAssign.getInputs().add(new MutableObject<>(assign));
@@ -281,8 +291,9 @@
             insertOp.getInputs().add(new MutableObject<>(assign));
         }
 
-        ILogicalOperator leafOperator = new SinkOperator();
+        SinkOperator leafOperator = new SinkOperator();
         leafOperator.getInputs().add(new MutableObject<>(insertOp));
+        leafOperator.setSourceLocation(sourceLoc);
         return new ALogicalPlanImpl(new MutableObject<>(leafOperator));
     }
 
@@ -298,6 +309,7 @@
         if (baseOp != null) {
             base = new MutableObject<>(baseOp);
         }
+        SourceLocation sourceLoc = expr.getSourceLocation();
         Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, base);
         ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<>();
         ILogicalOperator topOp = p.first;
@@ -314,10 +326,13 @@
             metadataProvider.setOutputFile(outputFileSplit);
 
             List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<>(1);
-            writeExprList.add(new MutableObject<>(new VariableReferenceExpression(resVar)));
+            VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+            resVarRef.setSourceLocation(sourceLoc);
+            writeExprList.add(new MutableObject<>(resVarRef));
             ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
             ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
             DistributeResultOperator newTop = new DistributeResultOperator(writeExprList, sink);
+            newTop.setSourceLocation(sourceLoc);
             newTop.getInputs().add(new MutableObject<>(topOp));
             topOp = newTop;
 
@@ -337,17 +352,23 @@
              * This assign adds a marker function collection-to-sequence: if the input is a
              * singleton collection, unnest it; otherwise do nothing.
              */
-            AssignOperator assignCollectionToSequence = new AssignOperator(seqVar,
-                    new MutableObject<>(new ScalarFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.COLLECTION_TO_SEQUENCE),
-                            new MutableObject<>(new VariableReferenceExpression(resVar)))));
+            VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+            resVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression collectionToSequenceExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.COLLECTION_TO_SEQUENCE),
+                    new MutableObject<>(resVarRef));
+            collectionToSequenceExpr.setSourceLocation(sourceLoc);
+            AssignOperator assignCollectionToSequence =
+                    new AssignOperator(seqVar, new MutableObject<>(collectionToSequenceExpr));
+            assignCollectionToSequence.setSourceLocation(sourceLoc);
+
             assignCollectionToSequence.getInputs().add(new MutableObject<>(topOp.getInputs().get(0).getValue()));
             topOp.getInputs().get(0).setValue(assignCollectionToSequence);
             ProjectOperator projectOperator = (ProjectOperator) topOp;
             projectOperator.getVariables().set(0, seqVar);
             resVar = seqVar;
             DatasetDataSource targetDatasource =
-                    validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName());
+                    validateDatasetInfo(metadataProvider, stmt.getDataverseName(), stmt.getDatasetName(), sourceLoc);
             List<Integer> keySourceIndicator =
                     ((InternalDatasetDetails) targetDatasource.getDataset().getDatasetDetails())
                             .getKeySourceIndicator();
@@ -360,15 +381,16 @@
                 if (keySourceIndicator == null || keySourceIndicator.get(i).intValue() == 0) {
                     // record part
                     PlanTranslationUtil.prepareVarAndExpression(partitionKeys.get(i), resVar, vars, exprs,
-                            varRefsForLoading, context);
+                            varRefsForLoading, context, sourceLoc);
                 } else {
                     // meta part
                     PlanTranslationUtil.prepareMetaKeyAccessExpression(partitionKeys.get(i), unnestVar, exprs, vars,
-                            varRefsForLoading, context);
+                            varRefsForLoading, context, sourceLoc);
                 }
             }
 
             AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.setSourceLocation(sourceLoc);
             List<String> additionalFilteringField = DatasetUtil.getFilterField(targetDatasource.getDataset());
             List<LogicalVariable> additionalFilteringVars;
             List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions;
@@ -380,17 +402,20 @@
                 additionalFilteringExpressions = new ArrayList<>();
 
                 PlanTranslationUtil.prepareVarAndExpression(additionalFilteringField, resVar, additionalFilteringVars,
-                        additionalFilteringAssignExpressions, additionalFilteringExpressions, context);
+                        additionalFilteringAssignExpressions, additionalFilteringExpressions, context, sourceLoc);
 
                 additionalFilteringAssign =
                         new AssignOperator(additionalFilteringVars, additionalFilteringAssignExpressions);
                 additionalFilteringAssign.getInputs().add(new MutableObject<>(topOp));
+                additionalFilteringAssign.setSourceLocation(sourceLoc);
                 assign.getInputs().add(new MutableObject<>(additionalFilteringAssign));
             } else {
                 assign.getInputs().add(new MutableObject<>(topOp));
             }
 
-            Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(resVar));
+            VariableReferenceExpression resVarRef2 = new VariableReferenceExpression(resVar);
+            resVarRef2.setSourceLocation(sourceLoc);
+            Mutable<ILogicalExpression> varRef = new MutableObject<>(resVarRef2);
             ILogicalOperator leafOperator;
             switch (stmt.getKind()) {
                 case INSERT:
@@ -404,10 +429,11 @@
                     break;
                 case DELETE:
                     leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
-                            additionalFilteringExpressions, assign);
+                            additionalFilteringExpressions, assign, stmt);
                     break;
                 default:
-                    throw new AlgebricksException("Unsupported statement kind " + stmt.getKind());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unsupported statement kind " + stmt.getKind());
             }
             topOp = leafOperator;
         }
@@ -419,18 +445,22 @@
 
     private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
-            List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
-            throws AlgebricksException {
+            List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
+            ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (targetDatasource.getDataset().hasMetaPart()) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": delete from dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": delete from dataset is not supported on Datasets with Meta records");
         }
         InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
         deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         deleteOp.getInputs().add(new MutableObject<>(assign));
-        ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(true));
+        deleteOp.setSourceLocation(sourceLoc);
+        DelegateOperator leafOperator = new DelegateOperator(new CommitOperator(true));
         leafOperator.getInputs().add(new MutableObject<>(deleteOp));
+        leafOperator.setSourceLocation(sourceLoc);
         return leafOperator;
     }
 
@@ -440,9 +470,11 @@
             List<String> additionalFilteringField, LogicalVariable unnestVar, ILogicalOperator topOp,
             List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (!targetDatasource.getDataset().allow(topOp, DatasetUtil.OP_UPSERT)) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": upsert into dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": upsert into dataset is not supported on Datasets with Meta records");
         }
         ProjectOperator project = (ProjectOperator) topOp;
         CompiledUpsertStatement compiledUpsert = (CompiledUpsertStatement) stmt;
@@ -451,7 +483,8 @@
         ILogicalOperator rootOperator;
         if (targetDatasource.getDataset().hasMetaPart()) {
             if (returnExpression != null) {
-                throw new AlgebricksException("Returning not allowed on datasets with Meta records");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Returning not allowed on datasets with Meta records");
             }
             AssignOperator metaAndKeysAssign;
             List<LogicalVariable> metaAndKeysVars;
@@ -461,12 +494,17 @@
             metaAndKeysExprs = new ArrayList<>();
             // add the meta function
             IFunctionInfo finfoMeta = FunctionUtil.getFunctionInfo(BuiltinFunctions.META);
-            ScalarFunctionCallExpression metaFunction = new ScalarFunctionCallExpression(finfoMeta,
-                    new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+            VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+            unnestVarRef.setSourceLocation(sourceLoc);
+            ScalarFunctionCallExpression metaFunction =
+                    new ScalarFunctionCallExpression(finfoMeta, new MutableObject<>(unnestVarRef));
+            metaFunction.setSourceLocation(sourceLoc);
             // create assign for the meta part
             LogicalVariable metaVar = context.newVar();
             metaExpSingletonList = new ArrayList<>(1);
-            metaExpSingletonList.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
+            VariableReferenceExpression metaVarRef = new VariableReferenceExpression(metaVar);
+            metaVarRef.setSourceLocation(sourceLoc);
+            metaExpSingletonList.add(new MutableObject<>(metaVarRef));
             metaAndKeysVars.add(metaVar);
             metaAndKeysExprs.add(new MutableObject<>(metaFunction));
             project.getVariables().add(metaVar);
@@ -485,9 +523,12 @@
             // A change feed, we don't need the assign to access PKs
             upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading, metaExpSingletonList,
                     InsertDeleteUpsertOperator.Kind.UPSERT, false);
+            upsertOp.setUpsertIndicatorVar(context.newVar());
+            upsertOp.setUpsertIndicatorVarType(BuiltinType.ABOOLEAN);
             // Create and add a new variable used for representing the original record
             upsertOp.setPrevRecordVar(context.newVar());
             upsertOp.setPrevRecordType(targetDatasource.getItemType());
+            upsertOp.setSourceLocation(sourceLoc);
             if (targetDatasource.getDataset().hasMetaPart()) {
                 List<LogicalVariable> metaVars = new ArrayList<>();
                 metaVars.add(context.newVar());
@@ -509,6 +550,7 @@
             }
             metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
             metaAndKeysAssign.getInputs().add(topOp.getInputs().get(0));
+            metaAndKeysAssign.setSourceLocation(sourceLoc);
             topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
             upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         } else {
@@ -516,6 +558,9 @@
                     InsertDeleteUpsertOperator.Kind.UPSERT, false);
             upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
             upsertOp.getInputs().add(new MutableObject<>(assign));
+            upsertOp.setSourceLocation(sourceLoc);
+            upsertOp.setUpsertIndicatorVar(context.newVar());
+            upsertOp.setUpsertIndicatorVarType(BuiltinType.ABOOLEAN);
             // Create and add a new variable used for representing the original record
             ARecordType recordType = (ARecordType) targetDatasource.getItemType();
             upsertOp.setPrevRecordVar(context.newVar());
@@ -525,34 +570,38 @@
                 upsertOp.setPrevFilterType(recordType.getFieldType(additionalFilteringField.get(0)));
             }
         }
-        rootOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
-        rootOperator.getInputs().add(new MutableObject<>(upsertOp));
+        DelegateOperator delegateOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
+        delegateOperator.getInputs().add(new MutableObject<>(upsertOp));
+        delegateOperator.setSourceLocation(sourceLoc);
+        rootOperator = delegateOperator;
 
         // Compiles the return expression.
         return processReturningExpression(rootOperator, upsertOp, compiledUpsert);
-
     }
 
     private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
+        SourceLocation sourceLoc = stmt.getSourceLocation();
         if (targetDatasource.getDataset().hasMetaPart()) {
-            throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
-                    + ": insert into dataset is not supported on Datasets with Meta records");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    targetDatasource.getDataset().getDatasetName()
+                            + ": insert into dataset is not supported on Datasets with Meta records");
         }
         // Adds the insert operator.
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
         insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         insertOp.getInputs().add(new MutableObject<>(assign));
+        insertOp.setSourceLocation(sourceLoc);
 
         // Adds the commit operator.
         CompiledInsertStatement compiledInsert = (CompiledInsertStatement) stmt;
         Expression returnExpression = compiledInsert.getReturnExpression();
-        ILogicalOperator rootOperator =
-                new DelegateOperator(new CommitOperator(returnExpression == null ? true : false));
+        DelegateOperator rootOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
         rootOperator.getInputs().add(new MutableObject<>(insertOp));
+        rootOperator.setSourceLocation(sourceLoc);
 
         // Compiles the return expression.
         return processReturningExpression(rootOperator, insertOp, compiledInsert);
@@ -566,38 +615,51 @@
         if (returnExpression == null) {
             return inputOperator;
         }
-        ILogicalOperator rootOperator = inputOperator;
+        SourceLocation sourceLoc = compiledInsert.getSourceLocation();
+
+        //Create an assign operator that makes the variable used by the return expression
+        LogicalVariable insertedVar = context.newVar();
+        AssignOperator insertedVarAssignOperator =
+                new AssignOperator(insertedVar, new MutableObject<>(insertOp.getPayloadExpression().getValue()));
+        insertedVarAssignOperator.getInputs().add(insertOp.getInputs().get(0));
+        insertedVarAssignOperator.setSourceLocation(sourceLoc);
+        insertOp.getInputs().set(0, new MutableObject<>(insertedVarAssignOperator));
 
         // Makes the id of the insert var point to the record variable.
         context.newVarFromExpression(compiledInsert.getVar());
-        context.setVar(compiledInsert.getVar(),
-                ((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
-                langExprToAlgExpression(returnExpression, new MutableObject<>(rootOperator));
+        context.setVar(compiledInsert.getVar(), insertedVar);
 
-        // Adds an assign operator for the returning expression.
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
+                langExprToAlgExpression(returnExpression, new MutableObject<>(inputOperator));
+
+        // Adds an assign operator for the result of the returning expression.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator assignOperator = new AssignOperator(resultVar, new MutableObject<>(p.first));
-        assignOperator.getInputs().add(p.second);
+        AssignOperator createResultAssignOperator = new AssignOperator(resultVar, new MutableObject<>(p.first));
+        createResultAssignOperator.getInputs().add(p.second);
+        createResultAssignOperator.setSourceLocation(sourceLoc);
 
         // Adds a distribute result operator.
         List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
         expressions.add(new MutableObject<>(new VariableReferenceExpression(resultVar)));
         ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
         ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
-        rootOperator = new DistributeResultOperator(expressions, sink);
-        rootOperator.getInputs().add(new MutableObject<>(assignOperator));
-        return rootOperator;
+        DistributeResultOperator distResultOperator = new DistributeResultOperator(expressions, sink);
+        distResultOperator.getInputs().add(new MutableObject<>(createResultAssignOperator));
+
+        distResultOperator.setSourceLocation(sourceLoc);
+        return distResultOperator;
     }
 
     private DatasetDataSource validateDatasetInfo(MetadataProvider metadataProvider, String dataverseName,
-            String datasetName) throws AlgebricksException {
+            String datasetName, SourceLocation sourceLoc) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Cannot find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                    dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Cannot write output to an external dataset.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Cannot write output to an external dataset.");
         }
         DataSourceId sourceId = new DataSourceId(dataverseName, datasetName);
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
@@ -620,56 +682,68 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(LetClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         LogicalVariable v;
-        ILogicalOperator returnedOp;
-        if (lc.getBindingExpr().getKind() == Kind.VARIABLE_EXPRESSION) {
+        AssignOperator returnedOp;
+        Expression bindingExpr = lc.getBindingExpr();
+        SourceLocation sourceLoc = bindingExpr.getSourceLocation();
+        if (bindingExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr bindingVarExpr = (VariableExpr) bindingExpr;
+            ILogicalExpression prevVarRef = translateVariableRef(bindingVarExpr);
             v = context.newVarFromExpression(lc.getVarExpr());
-            LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
-            returnedOp = new AssignOperator(v, new MutableObject<>(new VariableReferenceExpression(prev)));
+            returnedOp = new AssignOperator(v, new MutableObject<>(prevVarRef));
             returnedOp.getInputs().add(tupSource);
+            returnedOp.setSourceLocation(sourceLoc);
         } else {
             v = context.newVarFromExpression(lc.getVarExpr());
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo =
-                    langExprToAlgExpression(lc.getBindingExpr(), tupSource);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(bindingExpr, tupSource);
             returnedOp = new AssignOperator(v, new MutableObject<>(eo.first));
             returnedOp.getInputs().add(eo.second);
+            returnedOp.setSourceLocation(sourceLoc);
         }
+
         return new Pair<>(returnedOp, v);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = fa.getSourceLocation();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(fa.getExpr(), tupSource);
         LogicalVariable v = context.newVarFromExpression(fa);
         AbstractFunctionCallExpression fldAccess =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.setSourceLocation(sourceLoc);
         fldAccess.getArguments().add(new MutableObject<>(p.first));
         ILogicalExpression faExpr =
                 new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent().getValue())));
         fldAccess.getArguments().add(new MutableObject<>(faExpr));
         AssignOperator a = new AssignOperator(v, new MutableObject<>(fldAccess));
         a.getInputs().add(p.second);
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, v);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = ia.getSourceLocation();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(ia.getExpr(), tupSource);
         LogicalVariable v = context.newVar();
         AbstractFunctionCallExpression f;
         if (ia.isAny()) {
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.ANY_COLLECTION_MEMBER));
             f.getArguments().add(new MutableObject<>(p.first));
+            f.setSourceLocation(sourceLoc);
         } else {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> indexPair =
                     langExprToAlgExpression(ia.getIndexExpr(), tupSource);
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.GET_ITEM));
             f.getArguments().add(new MutableObject<>(p.first));
             f.getArguments().add(new MutableObject<>(indexPair.first));
+            f.setSourceLocation(sourceLoc);
         }
         AssignOperator a = new AssignOperator(v, new MutableObject<>(f));
         a.getInputs().add(p.second);
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, v);
     }
 
@@ -684,8 +758,9 @@
         for (Expression expr : fcall.getExprList()) {
             switch (expr.getKind()) {
                 case VARIABLE_EXPRESSION:
-                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
-                    args.add(new MutableObject<>(new VariableReferenceExpression(var)));
+                    VariableExpr varExpr = (VariableExpr) expr;
+                    ILogicalExpression varRefExpr = translateVariableRef(varExpr);
+                    args.add(new MutableObject<>(varRefExpr));
                     break;
                 case LITERAL_EXPRESSION:
                     LiteralExpr val = (LiteralExpr) expr;
@@ -696,24 +771,22 @@
                     Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
                     AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
                     args.add(new MutableObject<>(eo.first));
-                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+                    if (o1 != null) {
                         topOp = eo.second;
                     }
                     break;
             }
         }
 
+        SourceLocation sourceLoc = fcall.getSourceLocation();
         AbstractFunctionCallExpression f;
-        try {
-            if ((f = lookupUserDefinedFunction(signature, args)) == null) {
-                f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args);
-            }
-        } catch (AlgebricksException e) {
-            throw new CompilationException(e);
+        if ((f = lookupUserDefinedFunction(signature, args, sourceLoc)) == null) {
+            f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args, sourceLoc);
         }
 
         if (f == null) {
-            throw new CompilationException(" Unknown function " + signature.getName() + "@" + signature.getArity());
+            throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc,
+                    signature.getName() + "@" + signature.getArity());
         }
 
         // Put hints into function call expr.
@@ -727,37 +800,56 @@
         if (topOp != null) {
             op.getInputs().add(topOp);
         }
+        op.setSourceLocation(sourceLoc);
 
         return new Pair<>(op, v);
     }
 
+    protected ILogicalExpression translateVariableRef(VariableExpr varExpr) throws CompilationException {
+        LogicalVariable var = context.getVar(varExpr.getVar().getId());
+        if (var == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, varExpr.getSourceLocation(),
+                    varExpr.toString());
+        }
+        VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+        varRef.setSourceLocation(varExpr.getSourceLocation());
+        return varRef;
+    }
+
     private AbstractFunctionCallExpression lookupUserDefinedFunction(FunctionSignature signature,
-            List<Mutable<ILogicalExpression>> args) throws AlgebricksException {
-        if (signature.getNamespace() == null) {
-            return null;
+            List<Mutable<ILogicalExpression>> args, SourceLocation sourceLoc) throws CompilationException {
+        try {
+            if (signature.getNamespace() == null) {
+                return null;
+            }
+            Function function =
+                    MetadataManager.INSTANCE.getFunction(metadataProvider.getMetadataTxnContext(), signature);
+            if (function == null) {
+                return null;
+            }
+            AbstractFunctionCallExpression f;
+            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
+                IFunctionInfo finfo = ExternalFunctionCompilerUtil
+                        .getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
+                f = new ScalarFunctionCallExpression(finfo, args);
+                f.setSourceLocation(sourceLoc);
+            } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
+                    || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
+                IFunctionInfo finfo = FunctionUtil.getFunctionInfo(signature);
+                f = new ScalarFunctionCallExpression(finfo, args);
+                f.setSourceLocation(sourceLoc);
+            } else {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        " User defined functions written in " + function.getLanguage() + " are not supported");
+            }
+            return f;
+        } catch (AlgebricksException e) {
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage(), e);
         }
-        Function function = MetadataManager.INSTANCE.getFunction(metadataProvider.getMetadataTxnContext(), signature);
-        if (function == null) {
-            return null;
-        }
-        AbstractFunctionCallExpression f;
-        if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_JAVA)) {
-            IFunctionInfo finfo = ExternalFunctionCompilerUtil
-                    .getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
-            f = new ScalarFunctionCallExpression(finfo, args);
-        } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
-                || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
-            IFunctionInfo finfo = FunctionUtil.getFunctionInfo(signature);
-            f = new ScalarFunctionCallExpression(finfo, args);
-        } else {
-            throw new MetadataException(
-                    " User defined functions written in " + function.getLanguage() + " are not supported");
-        }
-        return f;
     }
 
     private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
-            List<Mutable<ILogicalExpression>> args) {
+            List<Mutable<ILogicalExpression>> args, SourceLocation sourceLoc) {
         AbstractFunctionCallExpression f;
         FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
         FunctionInfo afi = BuiltinFunctions.lookupFunction(fi);
@@ -782,6 +874,7 @@
         } else {
             f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fi), args);
         }
+        f.setSourceLocation(sourceLoc);
         return f;
     }
 
@@ -793,8 +886,8 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = gc.getSourceLocation();
         Mutable<ILogicalOperator> topOp = tupSource;
-
         LogicalVariable groupRecordVar = null;
         if (gc.hasGroupVar()) {
             List<Pair<Expression, Identifier>> groupFieldList = gc.getGroupFieldList();
@@ -807,13 +900,16 @@
                 ILogicalExpression groupFieldExpr = langExprToAlgExpression(groupField.first, topOp).first;
                 groupRecordConstructorArgList.add(new MutableObject<>(groupFieldExpr));
             }
-            MutableObject<ILogicalExpression> groupRecordConstr = new MutableObject<>(new ScalarFunctionCallExpression(
+            ScalarFunctionCallExpression groupRecordConstr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR),
-                    groupRecordConstructorArgList));
+                    groupRecordConstructorArgList);
+            groupRecordConstr.setSourceLocation(sourceLoc);
 
             groupRecordVar = context.newVar();
-            AssignOperator groupRecordVarAssignOp = new AssignOperator(groupRecordVar, groupRecordConstr);
+            AssignOperator groupRecordVarAssignOp =
+                    new AssignOperator(groupRecordVar, new MutableObject<>(groupRecordConstr));
             groupRecordVarAssignOp.getInputs().add(topOp);
+            groupRecordVarAssignOp.setSourceLocation(sourceLoc);
             topOp = new MutableObject<>(groupRecordVarAssignOp);
         }
 
@@ -838,9 +934,12 @@
         if (gc.hasGroupVar()) {
             VariableExpr groupVar = gc.getGroupVar();
             LogicalVariable groupLogicalVar = context.newVar();
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(gOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            VariableReferenceExpression groupRecordVarRef = new VariableReferenceExpression(groupRecordVar);
+            groupRecordVarRef.setSourceLocation(sourceLoc);
             ILogicalPlan nestedPlan = createNestedPlanWithAggregate(groupLogicalVar, BuiltinFunctions.LISTIFY,
-                    new VariableReferenceExpression(groupRecordVar),
-                    new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(gOp))));
+                    groupRecordVarRef, new MutableObject<>(ntsOp));
             gOp.getNestedPlans().add(nestedPlan);
             context.setVar(groupVar, groupLogicalVar);
         }
@@ -849,8 +948,10 @@
             for (Entry<Expression, VariableExpr> entry : gc.getWithVarMap().entrySet()) {
                 VariableExpr withVar = entry.getValue();
                 Expression withExpr = entry.getKey();
-                Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput = langExprToAlgExpression(withExpr,
-                        new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(gOp))));
+                NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(gOp));
+                ntsOp.setSourceLocation(sourceLoc);
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> listifyInput =
+                        langExprToAlgExpression(withExpr, new MutableObject<>(ntsOp));
                 LogicalVariable withLogicalVar = context.newVar();
                 ILogicalPlan nestedPlan = createNestedPlanWithAggregate(withLogicalVar, BuiltinFunctions.LISTIFY,
                         listifyInput.first, listifyInput.second);
@@ -861,16 +962,20 @@
 
         gOp.setGroupAll(gc.isGroupAll());
         gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        gOp.setSourceLocation(sourceLoc);
         return new Pair<>(gOp, null);
     }
 
     private ILogicalPlan createNestedPlanWithAggregate(LogicalVariable aggOutputVar, FunctionIdentifier aggFunc,
             ILogicalExpression aggFnInput, Mutable<ILogicalOperator> aggOpInput) {
+        SourceLocation sourceLoc = aggFnInput.getSourceLocation();
         AggregateFunctionCallExpression aggFnCall = BuiltinFunctions.makeAggregateFunctionExpression(aggFunc,
                 mkSingletonArrayList(new MutableObject<>(aggFnInput)));
+        aggFnCall.setSourceLocation(sourceLoc);
         AggregateOperator aggOp = new AggregateOperator(mkSingletonArrayList(aggOutputVar),
                 mkSingletonArrayList(new MutableObject<>(aggFnCall)));
         aggOp.getInputs().add(aggOpInput);
+        aggOp.setSourceLocation(sourceLoc);
         return new ALogicalPlanImpl(new MutableObject<>(aggOp));
     }
 
@@ -886,45 +991,70 @@
         // Similarly, we create one subplan for the "else" branch, in which the
         // selection is not(varCond).
         // Finally, we select the desired result.
-        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        Expression condExpr = ifexpr.getCondExpr();
+        Expression thenExpr = ifexpr.getThenExpr();
+        Expression elseExpr = ifexpr.getElseExpr();
+
+        Pair<ILogicalOperator, LogicalVariable> pCond = condExpr.accept(this, tupSource);
         LogicalVariable varCond = pCond.second;
 
         // Creates a subplan for the "then" branch.
-        Pair<ILogicalOperator, LogicalVariable> opAndVarForThen = constructSubplanOperatorForBranch(pCond.first,
-                new MutableObject<>(new VariableReferenceExpression(varCond)), ifexpr.getThenExpr());
+        VariableReferenceExpression varCondRef1 = new VariableReferenceExpression(varCond);
+        varCondRef1.setSourceLocation(condExpr.getSourceLocation());
+
+        Pair<ILogicalOperator, LogicalVariable> opAndVarForThen =
+                constructSubplanOperatorForBranch(pCond.first, new MutableObject<>(varCondRef1), thenExpr);
 
         // Creates a subplan for the "else" branch.
-        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-                Collections.singletonList(generateAndNotIsUnknownWrap(new VariableReferenceExpression(varCond))));
-        Pair<ILogicalOperator, LogicalVariable> opAndVarForElse = constructSubplanOperatorForBranch(
-                opAndVarForThen.first, new MutableObject<>(notVarCond), ifexpr.getElseExpr());
+        VariableReferenceExpression varCondRef2 = new VariableReferenceExpression(varCond);
+        varCondRef2.setSourceLocation(condExpr.getSourceLocation());
+        AbstractFunctionCallExpression notVarCond =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
+                        Collections.singletonList(generateAndNotIsUnknownWrap(varCondRef2)));
+        notVarCond.setSourceLocation(condExpr.getSourceLocation());
+
+        Pair<ILogicalOperator, LogicalVariable> opAndVarForElse =
+                constructSubplanOperatorForBranch(opAndVarForThen.first, new MutableObject<>(notVarCond), elseExpr);
 
         // Uses switch-case function to select the results of two branches.
         LogicalVariable selectVar = context.newVar();
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(varCond)));
+        VariableReferenceExpression varCondRef3 = new VariableReferenceExpression(varCond);
+        varCondRef3.setSourceLocation(condExpr.getSourceLocation());
+        VariableReferenceExpression varThenRef = new VariableReferenceExpression(opAndVarForThen.second);
+        varThenRef.setSourceLocation(thenExpr.getSourceLocation());
+        VariableReferenceExpression varElseRef = new VariableReferenceExpression(opAndVarForElse.second);
+        varElseRef.setSourceLocation(elseExpr.getSourceLocation());
+        arguments.add(new MutableObject<>(varCondRef3));
         arguments.add(new MutableObject<>(ConstantExpression.TRUE));
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForThen.second)));
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForElse.second)));
+        arguments.add(new MutableObject<>(varThenRef));
+        arguments.add(new MutableObject<>(varElseRef));
         AbstractFunctionCallExpression swithCaseExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SWITCH_CASE), arguments);
+        swithCaseExpr.setSourceLocation(ifexpr.getSourceLocation());
         AssignOperator assignOp = new AssignOperator(selectVar, new MutableObject<>(swithCaseExpr));
         assignOp.getInputs().add(new MutableObject<>(opAndVarForElse.first));
+        assignOp.setSourceLocation(ifexpr.getSourceLocation());
 
         // Unnests the selected ("if" or "else") result.
         LogicalVariable unnestVar = context.newVar();
-        UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                new MutableObject<>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), Collections
-                                .singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
+        VariableReferenceExpression selectVarRef = new VariableReferenceExpression(selectVar);
+        selectVarRef.setSourceLocation(ifexpr.getSourceLocation());
+        UnnestingFunctionCallExpression scanCollExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        Collections.singletonList(new MutableObject<>(selectVarRef)));
+        scanCollExpr.setSourceLocation(ifexpr.getSourceLocation());
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollExpr));
         unnestOp.getInputs().add(new MutableObject<>(assignOp));
+        unnestOp.setSourceLocation(ifexpr.getSourceLocation());
 
         // Produces the final result.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator finalAssignOp =
-                new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(ifexpr.getSourceLocation());
+        AssignOperator finalAssignOp = new AssignOperator(resultVar, new MutableObject<>(unnestVarRef));
         finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
+        finalAssignOp.setSourceLocation(ifexpr.getSourceLocation());
         return new Pair<>(finalAssignOp, resultVar);
     }
 
@@ -933,6 +1063,7 @@
         LogicalVariable var = context.newVar();
         AssignOperator a = new AssignOperator(var, new MutableObject<>(
                 new ConstantExpression(new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        a.setSourceLocation(l.getSourceLocation());
         if (tupSource != null) {
             a.getInputs().add(tupSource);
         }
@@ -953,78 +1084,67 @@
 
         Mutable<ILogicalOperator> topOp = tupSource;
 
-        ILogicalExpression currExpr = null;
+        SourceLocation sourceLoc = op.getSourceLocation();
+        AbstractFunctionCallExpression currExpr = null;
         for (int i = 0; i <= nOps; i++) {
-
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(exprs.get(i), topOp);
             topOp = p.second;
             ILogicalExpression e = p.first;
             // now look at the operator
             if (i < nOps) {
-                if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
+                OperatorType opType = ops.get(i);
+                boolean isCmpOp = OperatorExpr.opIsComparison(opType);
+                AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opType, sourceLoc);
 
-                    // chain the operators
-                    if (i == 0) {
-                        c.getArguments().add(new MutableObject<>(e));
-                        currExpr = c;
-                        if (op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.LEFT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
-                        c.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = c;
-                        if (i == 1 && op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.RIGHT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
+                // chain the operators
+                if (i == 0) {
+                    f.getArguments().add(new MutableObject<>(e));
+                    currExpr = f;
+                    if (isCmpOp && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.LEFT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 } else {
-                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
-                    if (i == 0) {
-                        f.getArguments().add(new MutableObject<>(e));
-                        currExpr = f;
-                    } else {
-                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
-                        f.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = f;
+                    currExpr.getArguments().add(new MutableObject<>(e));
+                    f.getArguments().add(new MutableObject<>(currExpr));
+                    currExpr = f;
+                    if (isCmpOp && i == 1 && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.RIGHT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 }
             } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<>(e));
+                currExpr.getArguments().add(new MutableObject<>(e));
                 if (i == 1 && op.isBroadcastOperand(i)) {
                     BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
                     bcast.setObject(BroadcastSide.RIGHT);
-                    ((AbstractFunctionCallExpression) currExpr).getAnnotations()
-                            .put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                    currExpr.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                 }
             }
         }
 
         // Add hints as annotations.
-        if (op.hasHints() && (currExpr instanceof AbstractFunctionCallExpression)) {
-            AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
+        if (op.hasHints()) {
             for (IExpressionAnnotation hint : op.getHints()) {
-                currFuncExpr.getAnnotations().put(hint, hint);
+                currExpr.getAnnotations().put(hint, hint);
             }
         }
 
         LogicalVariable assignedVar = context.newVar();
         AssignOperator a = new AssignOperator(assignedVar, new MutableObject<>(currExpr));
-
         a.getInputs().add(topOp);
-
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, assignedVar);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = oc.getSourceLocation();
         OrderOperator ord = new OrderOperator();
+        ord.setSourceLocation(sourceLoc);
         Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
         Mutable<ILogicalOperator> topOp = tupSource;
         for (Expression e : oc.getOrderbyList()) {
@@ -1053,16 +1173,19 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = qe.getSourceLocation();
         Mutable<ILogicalOperator> topOp = tupSource;
 
         ILogicalOperator firstOp = null;
         Mutable<ILogicalOperator> lastOp = null;
 
         for (QuantifiedPair qt : qe.getQuantifiedList()) {
-            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(qt.getExpr(), topOp);
+            Expression expr = qt.getExpr();
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(expr, topOp);
             topOp = eo1.second;
             LogicalVariable uVar = context.newVarFromExpression(qt.getVarExpr());
-            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<>(makeUnnestExpression(eo1.first)));
+            UnnestOperator u = new UnnestOperator(uVar, new MutableObject<>(makeUnnestExpression(eo1.first)));
+            u.setSourceLocation(expr.getSourceLocation());
 
             if (firstOp == null) {
                 firstOp = u;
@@ -1085,8 +1208,10 @@
         if (qe.getQuantifier() == Quantifier.SOME) {
             s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
             s.getInputs().add(eo2.second);
+            s.setSourceLocation(sourceLoc);
             fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM,
                     new ArrayList<>());
+            fAgg.setSourceLocation(sourceLoc);
         } else { // EVERY
             // look for input items that do not satisfy the condition, if none found then return true
             // when inverting the condition account for NULL/MISSING by replacing them with FALSE
@@ -1098,18 +1223,25 @@
                     .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE))));
 
             List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
-            notArgs.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs)));
+            ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs);
+            ifMissinOrNullExpr.setSourceLocation(sourceLoc);
+            notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
 
-            s = new SelectOperator(new MutableObject<>(new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs)), false, null);
+            ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
+            notExpr.setSourceLocation(sourceLoc);
+            s = new SelectOperator(new MutableObject<>(notExpr), false, null);
             s.getInputs().add(eo2.second);
+            s.setSourceLocation(sourceLoc);
             fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM, new ArrayList<>());
+            fAgg.setSourceLocation(sourceLoc);
         }
         LogicalVariable qeVar = context.newVar();
-        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List) mkSingletonArrayList(new MutableObject<>(fAgg)));
+        AggregateOperator a =
+                new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
         a.getInputs().add(new MutableObject<>(s));
+        a.setSourceLocation(sourceLoc);
         return new Pair<>(a, qeVar);
     }
 
@@ -1124,8 +1256,10 @@
             throws CompilationException {
         AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        f.setSourceLocation(rc.getSourceLocation());
         LogicalVariable v1 = context.newVar();
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(f));
+        a.setSourceLocation(rc.getSourceLocation());
         Mutable<ILogicalOperator> topOp = tupSource;
         for (FieldBinding fb : rc.getFbList()) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(fb.getLeftExpr(), topOp);
@@ -1142,11 +1276,14 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(ListConstructor lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = lc.getSourceLocation();
         FunctionIdentifier fid = (lc.getType() == ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR)
                 ? BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR : BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
         AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        f.setSourceLocation(sourceLoc);
         LogicalVariable v1 = context.newVar();
         AssignOperator a = new AssignOperator(v1, new MutableObject<>(f));
+        a.setSourceLocation(sourceLoc);
         Mutable<ILogicalOperator> topOp = tupSource;
         for (Expression expr : lc.getExprList()) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, topOp);
@@ -1160,6 +1297,7 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = u.getSourceLocation();
         Expression expr = u.getExpr();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(expr, tupSource);
         LogicalVariable v1 = context.newVar();
@@ -1167,33 +1305,39 @@
         switch (u.getExprType()) {
             case POSITIVE:
                 a = new AssignOperator(v1, new MutableObject<>(eo.first));
+                a.setSourceLocation(sourceLoc);
                 break;
             case NEGATIVE:
                 AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.NUMERIC_UNARY_MINUS));
+                m.setSourceLocation(sourceLoc);
                 m.getArguments().add(new MutableObject<>(eo.first));
                 a = new AssignOperator(v1, new MutableObject<>(m));
+                a.setSourceLocation(sourceLoc);
                 break;
             case EXISTS:
-                a = processExists(eo.first, v1, false);
+                a = processExists(eo.first, v1, false, sourceLoc);
                 break;
             case NOT_EXISTS:
-                a = processExists(eo.first, v1, true);
+                a = processExists(eo.first, v1, true, sourceLoc);
                 break;
             default:
-                throw new CompilationException("Unsupported operator: " + u);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported operator: " + u.getExprType());
         }
         a.getInputs().add(eo.second);
         return new Pair<>(a, v1);
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visit(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+    public Pair<ILogicalOperator, LogicalVariable> visit(VariableExpr v, Mutable<ILogicalOperator> tupSource)
+            throws CompilationException {
         // Should we ever get to this method?
+        ILogicalExpression oldVRef = translateVariableRef(v);
         LogicalVariable var = context.newVar();
-        LogicalVariable oldV = context.getVar(v.getVar().getId());
-        AssignOperator a = new AssignOperator(var, new MutableObject<>(new VariableReferenceExpression(oldV)));
+        AssignOperator a = new AssignOperator(var, new MutableObject<>(oldVRef));
         a.getInputs().add(tupSource);
+        a.setSourceLocation(v.getSourceLocation());
         return new Pair<>(a, var);
     }
 
@@ -1203,55 +1347,88 @@
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(w.getWhereExpr(), tupSource);
         SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
         s.getInputs().add(p.second);
+        s.setSourceLocation(w.getSourceLocation());
         return new Pair<>(s, null);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(LimitClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        SourceLocation sourceLoc = lc.getSourceLocation();
         LimitOperator opLim;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        ILogicalExpression maxObjectsExpr =
+                createLimitOffsetValueExpression(p1.first, lc.getLimitExpr().getSourceLocation());
         Expression offset = lc.getOffset();
         if (offset != null) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = langExprToAlgExpression(offset, p1.second);
-            opLim = new LimitOperator(p1.first, p2.first);
+            ILogicalExpression offsetExpr =
+                    createLimitOffsetValueExpression(p2.first, lc.getOffset().getSourceLocation());
+            opLim = new LimitOperator(maxObjectsExpr, offsetExpr);
             opLim.getInputs().add(p2.second);
+            opLim.setSourceLocation(sourceLoc);
         } else {
-            opLim = new LimitOperator(p1.first);
+            opLim = new LimitOperator(maxObjectsExpr);
             opLim.getInputs().add(p1.second);
+            opLim.setSourceLocation(sourceLoc);
         }
         return new Pair<>(opLim, null);
     }
 
-    protected AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
-        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-        IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
-        return new ScalarFunctionCallExpression(finfo);
-    }
-
-    private static FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-        switch (t) {
-            case EQ:
-                return AlgebricksBuiltinFunctions.EQ;
-            case NEQ:
-                return AlgebricksBuiltinFunctions.NEQ;
-            case GT:
-                return AlgebricksBuiltinFunctions.GT;
-            case GE:
-                return AlgebricksBuiltinFunctions.GE;
-            case LT:
-                return AlgebricksBuiltinFunctions.LT;
-            case LE:
-                return AlgebricksBuiltinFunctions.LE;
-            default:
-                throw new IllegalStateException();
-        }
-    }
-
-    protected AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+    private ILogicalExpression createLimitOffsetValueExpression(ILogicalExpression inputExpr, SourceLocation sourceLoc)
             throws CompilationException {
+        // generates expression for limit and offset value:
+        //
+        // switch-case(treat-as-integer(user_value_expr) > 0, true, treat-as-integer(user_value_expr), 0)
+        //
+        // this guarantees that the value is always an integer and greater or equals to 0,
+        // so CopyLimitDownRule works correctly when computing the total limit,
+        // and other rules which assume integer type
+
+        AInt32 zero = new AInt32(0);
+
+        AbstractFunctionCallExpression valueExpr =
+                createFunctionCallExpression(BuiltinFunctions.TREAT_AS_INTEGER, sourceLoc);
+        valueExpr.getArguments().add(new MutableObject<>(inputExpr));
+
+        AbstractFunctionCallExpression cmpExpr =
+                createFunctionCallExpressionForBuiltinOperator(OperatorType.GT, sourceLoc);
+        cmpExpr.getArguments().add(new MutableObject<>(valueExpr));
+        cmpExpr.getArguments().add(new MutableObject<>(createConstantExpression(zero, sourceLoc)));
+
+        AbstractFunctionCallExpression switchExpr =
+                createFunctionCallExpression(BuiltinFunctions.SWITCH_CASE, sourceLoc);
+        switchExpr.getArguments().add(new MutableObject<>(cmpExpr));
+        switchExpr.getArguments().add(new MutableObject<>(createConstantExpression(ABoolean.TRUE, sourceLoc)));
+        switchExpr.getArguments().add(new MutableObject<>(valueExpr.cloneExpression()));
+        switchExpr.getArguments().add(new MutableObject<>(createConstantExpression(zero, sourceLoc)));
+
+        return switchExpr;
+    }
+
+    protected static AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
+            SourceLocation sourceLoc) throws CompilationException {
         FunctionIdentifier fid;
         switch (t) {
+            case EQ:
+                fid = AlgebricksBuiltinFunctions.EQ;
+                break;
+            case NEQ:
+                fid = AlgebricksBuiltinFunctions.NEQ;
+                break;
+            case GT:
+                fid = AlgebricksBuiltinFunctions.GT;
+                break;
+            case GE:
+                fid = AlgebricksBuiltinFunctions.GE;
+                break;
+            case LT:
+                fid = AlgebricksBuiltinFunctions.LT;
+                break;
+            case LE:
+                fid = AlgebricksBuiltinFunctions.LE;
+                break;
             case PLUS:
                 fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
                 break;
@@ -1261,17 +1438,17 @@
             case MUL:
                 fid = BuiltinFunctions.NUMERIC_MULTIPLY;
                 break;
-            case DIV:
+            case DIVIDE:
                 fid = BuiltinFunctions.NUMERIC_DIVIDE;
                 break;
+            case DIV:
+                fid = BuiltinFunctions.NUMERIC_DIV;
+                break;
             case MOD:
                 fid = BuiltinFunctions.NUMERIC_MOD;
                 break;
-            case IDIV:
-                fid = BuiltinFunctions.NUMERIC_IDIV;
-                break;
             case CARET:
-                fid = BuiltinFunctions.CARET;
+                fid = BuiltinFunctions.NUMERIC_POWER;
                 break;
             case AND:
                 fid = AlgebricksBuiltinFunctions.AND;
@@ -1283,29 +1460,27 @@
                 fid = BuiltinFunctions.FUZZY_EQ;
                 break;
             default:
-                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Operator " + t + " is not yet implemented");
         }
-        return new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        return createFunctionCallExpression(fid, sourceLoc);
     }
 
-    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
-        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-        if (inp == null || inp.size() != 1) {
-            return false;
-        }
-        return inp.get(0) == childCandidate;
+    private static AbstractFunctionCallExpression createFunctionCallExpression(FunctionIdentifier fid,
+            SourceLocation sourceLoc) {
+        ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
     protected Pair<ILogicalExpression, Mutable<ILogicalOperator>> langExprToAlgExpression(Expression expr,
             Mutable<ILogicalOperator> topOpRef) throws CompilationException {
+        SourceLocation sourceLoc = expr.getSourceLocation();
         switch (expr.getKind()) {
             case VARIABLE_EXPRESSION:
-                LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
-                if (var == null) {
-                    throw new IllegalStateException(String.valueOf(expr));
-                }
-                VariableReferenceExpression ve = new VariableReferenceExpression(var);
-                return new Pair<>(ve, topOpRef);
+                VariableExpr varExpr = (VariableExpr) expr;
+                ILogicalExpression varRefExpr = translateVariableRef(varExpr);
+                return new Pair<>(varRefExpr, topOpRef);
             case LITERAL_EXPRESSION:
                 LiteralExpr val = (LiteralExpr) expr;
                 return new Pair<>(new ConstantExpression(
@@ -1326,14 +1501,21 @@
                             rebindBottomOpRef(p.first, srcRef, topOpRef);
                         }
                         Mutable<ILogicalOperator> top2 = new MutableObject<>(p.first);
-                        return new Pair<>(new VariableReferenceExpression(p.second), top2);
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(p.second);
+                        varRef.setSourceLocation(sourceLoc);
+                        return new Pair<>(varRef, top2);
                     } else {
                         SubplanOperator s = new SubplanOperator();
                         s.getInputs().add(topOpRef);
-                        srcRef.setValue(new NestedTupleSourceOperator(new MutableObject<>(s)));
+                        s.setSourceLocation(sourceLoc);
+                        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(s));
+                        ntsOp.setSourceLocation(sourceLoc);
+                        srcRef.setValue(ntsOp);
                         Mutable<ILogicalOperator> planRoot = new MutableObject<>(p.first);
                         s.setRootOp(planRoot);
-                        return new Pair<>(new VariableReferenceExpression(p.second), new MutableObject<>(s));
+                        VariableReferenceExpression varRef = new VariableReferenceExpression(p.second);
+                        varRef.setSourceLocation(sourceLoc);
+                        return new Pair<>(varRef, new MutableObject<>(s));
                     }
                 }
         }
@@ -1341,18 +1523,22 @@
 
     protected Pair<ILogicalOperator, LogicalVariable> aggListifyForSubquery(LogicalVariable var,
             Mutable<ILogicalOperator> opRef, boolean bProject) {
+        SourceLocation sourceLoc = opRef.getValue().getSourceLocation();
         AggregateFunctionCallExpression funAgg =
                 BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY, new ArrayList<>());
         funAgg.getArguments().add(new MutableObject<>(new VariableReferenceExpression(var)));
+        funAgg.setSourceLocation(sourceLoc);
 
         LogicalVariable varListified = context.newSubplanOutputVar();
         AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
                 mkSingletonArrayList(new MutableObject<>(funAgg)));
         agg.getInputs().add(opRef);
+        agg.setSourceLocation(sourceLoc);
         ILogicalOperator res;
         if (bProject) {
             ProjectOperator pr = new ProjectOperator(varListified);
             pr.getInputs().add(new MutableObject<>(agg));
+            pr.setSourceLocation(sourceLoc);
             res = pr;
         } else {
             res = agg;
@@ -1362,6 +1548,7 @@
 
     protected Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
             Mutable<ILogicalOperator> tupSource) throws CompilationException {
+        SourceLocation sourceLoc = op.getSourceLocation();
         List<OperatorType> ops = op.getOpList();
         int nOps = ops.size();
 
@@ -1370,15 +1557,15 @@
         Mutable<ILogicalOperator> topOp = tupSource;
 
         OperatorType opLogical = ops.get(0);
-        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical, sourceLoc);
 
         for (int i = 0; i <= nOps; i++) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(exprs.get(i), topOp);
             topOp = p.second;
             // now look at the operator
             if (i < nOps && ops.get(i) != opLogical) {
-                throw new TranslationException(
-                        "Unexpected operator " + ops.get(i) + " in an OperatorExpr starting with " + opLogical);
+                throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_OPERATOR, sourceLoc, ops.get(i),
+                        opLogical);
             }
             f.getArguments().add(new MutableObject<>(p.first));
         }
@@ -1386,6 +1573,7 @@
         LogicalVariable assignedVar = context.newVar();
         AssignOperator a = new AssignOperator(assignedVar, new MutableObject<>(f));
         a.getInputs().add(topOp);
+        a.setSourceLocation(sourceLoc);
 
         return new Pair<>(a, assignedVar);
 
@@ -1410,18 +1598,26 @@
     }
 
     protected ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        SourceLocation sourceLoc = expr.getSourceLocation();
         List<Mutable<ILogicalExpression>> argRefs = new ArrayList<>();
         argRefs.add(new MutableObject<>(expr));
         switch (expr.getExpressionTag()) {
             case CONSTANT:
             case VARIABLE:
-                return new UnnestingFunctionCallExpression(
+                UnnestingFunctionCallExpression scanCollExpr1 = new UnnestingFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                scanCollExpr1.setSourceLocation(sourceLoc);
+                return scanCollExpr1;
             case FUNCTION_CALL:
                 AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                return (fce.getKind() == FunctionKind.UNNEST) ? expr
-                        : new UnnestingFunctionCallExpression(
-                                FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    UnnestingFunctionCallExpression scanCollExpr2 = new UnnestingFunctionCallExpression(
+                            FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), argRefs);
+                    scanCollExpr2.setSourceLocation(sourceLoc);
+                    return scanCollExpr2;
+                }
             default:
                 return expr;
         }
@@ -1564,22 +1760,31 @@
     protected Pair<ILogicalOperator, LogicalVariable> constructSubplanOperatorForBranch(ILogicalOperator inputOp,
             Mutable<ILogicalExpression> selectExpr, Expression branchExpression) throws CompilationException {
         context.enterSubplan();
+        SourceLocation sourceLoc = inputOp.getSourceLocation();
         SubplanOperator subplanOp = new SubplanOperator();
         subplanOp.getInputs().add(new MutableObject<>(inputOp));
-        Mutable<ILogicalOperator> nestedSource =
-                new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
+        subplanOp.setSourceLocation(sourceLoc);
+        NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+        ntsOp.setSourceLocation(sourceLoc);
         SelectOperator select = new SelectOperator(selectExpr, false, null);
         // The select operator cannot be moved up and down, otherwise it will cause
         // typing issues (ASTERIXDB-1203).
         OperatorPropertiesUtil.markMovable(select, false);
-        select.getInputs().add(nestedSource);
+        select.getInputs().add(new MutableObject<>(ntsOp));
+        select.setSourceLocation(selectExpr.getValue().getSourceLocation());
+
         Pair<ILogicalOperator, LogicalVariable> pBranch = branchExpression.accept(this, new MutableObject<>(select));
         LogicalVariable branchVar = context.newVar();
+        VariableReferenceExpression pBranchVarRef = new VariableReferenceExpression(pBranch.second);
+        pBranchVarRef.setSourceLocation(branchExpression.getSourceLocation());
+        AggregateFunctionCallExpression listifyExpr =
+                new AggregateFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false,
+                        Collections.singletonList(new MutableObject<>(pBranchVarRef)));
+        listifyExpr.setSourceLocation(branchExpression.getSourceLocation());
         AggregateOperator aggOp = new AggregateOperator(Collections.singletonList(branchVar),
-                Collections.singletonList(new MutableObject<>(new AggregateFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.LISTIFY), false, Collections.singletonList(
-                                new MutableObject<>(new VariableReferenceExpression(pBranch.second)))))));
+                Collections.singletonList(new MutableObject<>(listifyExpr)));
         aggOp.getInputs().add(new MutableObject<>(pBranch.first));
+        aggOp.setSourceLocation(branchExpression.getSourceLocation());
         ILogicalPlan planForBranch = new ALogicalPlanImpl(new MutableObject<>(aggOp));
         subplanOp.getNestedPlans().add(planForBranch);
         context.exitSubplan();
@@ -1587,22 +1792,27 @@
     }
 
     // Processes EXISTS and NOT EXISTS.
-    private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not) {
+    private AssignOperator processExists(ILogicalExpression inputExpr, LogicalVariable v1, boolean not,
+            SourceLocation sourceLoc) {
         AbstractFunctionCallExpression count =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCALAR_COUNT));
         count.getArguments().add(new MutableObject<>(inputExpr));
+        count.setSourceLocation(sourceLoc);
         AbstractFunctionCallExpression comparison = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(not ? BuiltinFunctions.EQ : BuiltinFunctions.NEQ));
         comparison.getArguments().add(new MutableObject<>(count));
         comparison.getArguments()
                 .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0L)))));
-        return new AssignOperator(v1, new MutableObject<>(comparison));
+        comparison.setSourceLocation(sourceLoc);
+        AssignOperator a = new AssignOperator(v1, new MutableObject<>(comparison));
+        a.setSourceLocation(sourceLoc);
+        return a;
     }
 
     // Generates the filter condition for whether a conditional branch should be
     // executed.
     protected Mutable<ILogicalExpression> generateNoMatchedPrecedingWhenBranchesFilter(
-            List<ILogicalExpression> inputBooleanExprs) {
+            List<ILogicalExpression> inputBooleanExprs, SourceLocation sourceLoc) {
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
         for (ILogicalExpression inputBooleanExpr : inputBooleanExprs) {
             // A NULL/MISSING valued WHEN expression does not lead to the corresponding THEN
@@ -1610,44 +1820,60 @@
             // Therefore, we should check a previous WHEN boolean condition is not unknown.
             arguments.add(generateAndNotIsUnknownWrap(inputBooleanExpr));
         }
-        Mutable<ILogicalExpression> hasBeenExecutedExprRef = new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.OR), arguments));
-        return new MutableObject<>(new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
-                new ArrayList<>(Collections.singletonList(hasBeenExecutedExprRef))));
+        ScalarFunctionCallExpression hasBeenExecutedExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.OR), arguments);
+        hasBeenExecutedExpr.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression notExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(hasBeenExecutedExpr))));
+        notExpr.setSourceLocation(sourceLoc);
+        return new MutableObject<>(notExpr);
     }
 
     // For an input expression `expr`, return `expr AND expr IS NOT UNKOWN`.
     protected Mutable<ILogicalExpression> generateAndNotIsUnknownWrap(ILogicalExpression logicalExpr) {
+        SourceLocation sourceLoc = logicalExpr.getSourceLocation();
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
         arguments.add(new MutableObject<>(logicalExpr));
-        Mutable<ILogicalExpression> expr = new MutableObject<>(
+        ScalarFunctionCallExpression isUnknownExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_UNKNOWN),
-                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr)))));
-        arguments.add(new MutableObject<>(new ScalarFunctionCallExpression(
-                FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), new ArrayList<>(Collections.singletonList(expr)))));
-        return new MutableObject<>(
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), arguments));
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr))));
+        isUnknownExpr.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression notExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(isUnknownExpr))));
+        notExpr.setSourceLocation(sourceLoc);
+        arguments.add(new MutableObject<>(notExpr));
+        ScalarFunctionCallExpression andExpr =
+                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), arguments);
+        andExpr.setSourceLocation(sourceLoc);
+        return new MutableObject<>(andExpr);
     }
 
     // Generates the plan for "UNION ALL" or union expression from its input
     // expressions.
     protected Pair<ILogicalOperator, LogicalVariable> translateUnionAllFromInputExprs(List<ILangExpression> inputExprs,
-            Mutable<ILogicalOperator> tupSource) throws CompilationException {
+            Mutable<ILogicalOperator> tupSource, SourceLocation sourceLoc) throws CompilationException {
         List<Mutable<ILogicalOperator>> inputOpRefsToUnion = new ArrayList<>();
         List<LogicalVariable> vars = new ArrayList<>();
         for (ILangExpression expr : inputExprs) {
+            SourceLocation exprSourceLoc = expr.getSourceLocation();
             // Visits the expression of one branch.
             Pair<ILogicalOperator, LogicalVariable> opAndVar = expr.accept(this, tupSource);
 
             // Creates an unnest operator.
             LogicalVariable unnestVar = context.newVar();
             List<Mutable<ILogicalExpression>> args = new ArrayList<>();
-            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(opAndVar.second)));
-            UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                    new MutableObject<ILogicalExpression>(new UnnestingFunctionCallExpression(
-                            FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), args)));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(opAndVar.second);
+            varRef.setSourceLocation(exprSourceLoc);
+            args.add(new MutableObject<>(varRef));
+            UnnestingFunctionCallExpression scanCollExpr = new UnnestingFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), args);
+            scanCollExpr.setSourceLocation(exprSourceLoc);
+            UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollExpr));
             unnestOp.getInputs().add(new MutableObject<>(opAndVar.first));
-            inputOpRefsToUnion.add(new MutableObject<ILogicalOperator>(unnestOp));
+            unnestOp.setSourceLocation(exprSourceLoc);
+            inputOpRefsToUnion.add(new MutableObject<>(unnestOp));
             vars.add(unnestVar);
         }
 
@@ -1671,6 +1897,7 @@
             topUnionAllOp = new UnionAllOperator(varTriples);
             topUnionAllOp.getInputs().add(leftInputBranch);
             topUnionAllOp.getInputs().add(inputOpRefIterator.next());
+            topUnionAllOp.setSourceLocation(sourceLoc);
 
             // Re-assigns leftInputBranch and leftInputVar.
             leftInputBranch = new MutableObject<>(topUnionAllOp);
@@ -1678,4 +1905,10 @@
         }
         return new Pair<>(topUnionAllOp, topUnionVar);
     }
+
+    private ConstantExpression createConstantExpression(IAObject value, SourceLocation sourceLoc) {
+        ConstantExpression constExpr = new ConstantExpression(new AsterixConstantValue(value));
+        constExpr.setSourceLocation(sourceLoc);
+        return constExpr;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
index ec181bb..c4e2859 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.translator;
 
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
 
 public class NoOpStatementExecutorContext implements IStatementExecutorContext {
 
@@ -28,17 +28,17 @@
     }
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return null;
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
+    public void put(String clientContextId, IClientRequest req) {
         // Dummy for when a statement doesn't support cancellation
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return null;
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultProperties.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultProperties.java
index 4866c6d..84d3885 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultProperties.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultProperties.java
@@ -21,6 +21,7 @@
 import java.io.Serializable;
 
 public class ResultProperties implements Serializable {
+    private static final long serialVersionUID = -4741260459407538016L;
 
     public static final long DEFAULT_MAX_READS = 1;
     private final IStatementExecutor.ResultDelivery delivery;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
index 89619e5..e34775f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
@@ -49,7 +49,7 @@
         CSV,
         CLEAN_JSON,
         LOSSLESS_JSON
-    };
+    }
 
     /**
      * Used to specify the format for logical plan and optimized logical plan.
@@ -70,7 +70,7 @@
             }
             return defaultFmt;
         }
-    };
+    }
 
     /**
      * Produce out-of-band output for Hyracks Job.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 9e36ba9..03c4bc5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -20,12 +20,17 @@
 
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Deque;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Clause.ClauseType;
 import org.apache.asterix.lang.common.base.Expression;
@@ -36,11 +41,17 @@
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.FieldBinding;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
+import org.apache.asterix.lang.common.expression.ListConstructor;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.OperatorExpr;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.struct.OperatorType;
+import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -67,10 +78,14 @@
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.commons.lang3.NotImplementedException;
+import org.apache.asterix.om.types.IAType;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -78,6 +93,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
@@ -87,6 +103,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestNonMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
@@ -98,6 +115,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Each visit returns a pair of an operator and a variable. The variable
@@ -107,20 +125,31 @@
  * which is translated. The second argument of a visit method is the tuple
  * source for the current subtree.
  */
-class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator implements ILangExpressionToPlanTranslator,
+public class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator
+        implements ILangExpressionToPlanTranslator,
         ISqlppVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
-    private static final String ERR_MSG = "Translator should never enter this method!";
-    private Deque<Mutable<ILogicalOperator>> uncorrelatedLeftBranchStack = new ArrayDeque<>();
 
-    public SqlppExpressionToPlanTranslator(MetadataProvider metadataProvider, int currentVarCounter)
-            throws AlgebricksException {
+    private static final String ERR_MSG = "Translator should never enter this method!";
+
+    public static final String REWRITE_IN_AS_OR_OPTION = "rewrite_in_as_or";
+    private static final boolean REWRITE_IN_AS_OR_OPTION_DEFAULT = true;
+
+    private Deque<Mutable<ILogicalOperator>> uncorrelatedLeftBranchStack = new ArrayDeque<>();
+    private final Map<VarIdentifier, IAObject> externalVars;
+    private final boolean translateInAsOr;
+
+    public SqlppExpressionToPlanTranslator(MetadataProvider metadataProvider, int currentVarCounter,
+            Map<VarIdentifier, IAObject> externalVars) throws AlgebricksException {
         super(metadataProvider, currentVarCounter);
+        this.externalVars = externalVars != null ? externalVars : Collections.emptyMap();
+        translateInAsOr = metadataProvider.getBooleanProperty(REWRITE_IN_AS_OR_OPTION, REWRITE_IN_AS_OR_OPTION_DEFAULT);
     }
 
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(Query q, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         Expression queryBody = q.getBody();
+        SourceLocation sourceLoc = queryBody.getSourceLocation();
         if (queryBody.getKind() == Kind.SELECT_EXPRESSION) {
             SelectExpression selectExpr = (SelectExpression) queryBody;
             if (q.isTopLevel()) {
@@ -132,8 +161,10 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(queryBody, tupSource);
             AssignOperator assignOp = new AssignOperator(var, new MutableObject<>(eo.first));
             assignOp.getInputs().add(eo.second);
+            assignOp.setSourceLocation(sourceLoc);
             ProjectOperator projectOp = new ProjectOperator(var);
             projectOp.getInputs().add(new MutableObject<>(assignOp));
+            projectOp.setSourceLocation(sourceLoc);
             return new Pair<>(projectOp, var);
         }
     }
@@ -175,21 +206,32 @@
             return leftInput.accept(this, tupSource);
         }
         List<ILangExpression> inputExprs = new ArrayList<>();
-        inputExprs.add(leftInput.selectBlock()
-                ? new SelectExpression(null, new SelectSetOperation(leftInput, null), null, null, true)
-                : leftInput.getSubquery());
+        SelectExpression leftInputExpr;
+        if (leftInput.selectBlock()) {
+            leftInputExpr = new SelectExpression(null, new SelectSetOperation(leftInput, null), null, null, true);
+            leftInputExpr.setSourceLocation(leftInput.getSelectBlock().getSourceLocation());
+        } else {
+            leftInputExpr = leftInput.getSubquery();
+        }
+        inputExprs.add(leftInputExpr);
         for (SetOperationRight setOperationRight : selectSetOperation.getRightInputs()) {
             SetOpType setOpType = setOperationRight.getSetOpType();
             if (setOpType != SetOpType.UNION || setOperationRight.isSetSemantics()) {
-                throw new CompilationException("Operation " + setOpType
-                        + (setOperationRight.isSetSemantics() ? " with set semantics" : "ALL") + " is not supported.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, selectSetOperation.getSourceLocation(),
+                        "Operation " + setOpType + (setOperationRight.isSetSemantics() ? " with set semantics" : "ALL")
+                                + " is not supported.");
             }
             SetOperationInput rightInput = setOperationRight.getSetOperationRightInput();
-            inputExprs.add(rightInput.selectBlock()
-                    ? new SelectExpression(null, new SelectSetOperation(rightInput, null), null, null, true)
-                    : rightInput.getSubquery());
+            SelectExpression rightInputExpr;
+            if (rightInput.selectBlock()) {
+                rightInputExpr = new SelectExpression(null, new SelectSetOperation(rightInput, null), null, null, true);
+                rightInputExpr.setSourceLocation(rightInput.getSelectBlock().getSourceLocation());
+            } else {
+                rightInputExpr = rightInput.getSubquery();
+            }
+            inputExprs.add(rightInputExpr);
         }
-        return translateUnionAllFromInputExprs(inputExprs, tupSource);
+        return translateUnionAllFromInputExprs(inputExprs, tupSource, selectSetOperation.getSourceLocation());
     }
 
     @Override
@@ -236,10 +278,11 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(FromTerm fromTerm, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
+        SourceLocation sourceLoc = fromTerm.getSourceLocation();
         LogicalVariable fromVar = context.newVarFromExpression(fromTerm.getLeftVariable());
         Expression fromExpr = fromTerm.getLeftExpression();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(fromExpr, tupSource);
-        ILogicalOperator unnestOp;
+        UnnestOperator unnestOp;
         if (fromTerm.hasPositionalVariable()) {
             LogicalVariable pVar = context.newVarFromExpression(fromTerm.getPositionalVariable());
             // We set the positional variable type as BIGINT type.
@@ -249,6 +292,7 @@
             unnestOp = new UnnestOperator(fromVar, new MutableObject<>(makeUnnestExpression(eo.first)));
         }
         unnestOp.getInputs().add(eo.second);
+        unnestOp.setSourceLocation(sourceLoc);
 
         // Processes joins, unnests, and nests.
         Mutable<ILogicalOperator> topOpRef = new MutableObject<>(unnestOp);
@@ -270,6 +314,7 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(JoinClause joinClause, Mutable<ILogicalOperator> inputRef)
             throws CompilationException {
+        SourceLocation sourceLoc = joinClause.getSourceLocation();
         Mutable<ILogicalOperator> leftInputRef = uncorrelatedLeftBranchStack.pop();
         if (joinClause.getJoinType() == JoinType.INNER) {
             Pair<ILogicalOperator, LogicalVariable> rightBranch =
@@ -277,6 +322,7 @@
             // A join operator with condition TRUE.
             AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
                     new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
+            joinOperator.setSourceLocation(sourceLoc);
             Mutable<ILogicalOperator> joinOpRef = new MutableObject<>(joinOperator);
 
             // Add an additional filter operator.
@@ -284,13 +330,16 @@
                     langExprToAlgExpression(joinClause.getConditionExpression(), joinOpRef);
             SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
             filter.getInputs().add(conditionExprOpPair.second);
+            filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
             return new Pair<>(filter, rightBranch.second);
         } else {
             // Creates a subplan operator.
             SubplanOperator subplanOp = new SubplanOperator();
-            Mutable<ILogicalOperator> ntsRef =
-                    new MutableObject<>(new NestedTupleSourceOperator(new MutableObject<>(subplanOp)));
             subplanOp.getInputs().add(leftInputRef);
+            subplanOp.setSourceLocation(sourceLoc);
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            Mutable<ILogicalOperator> ntsRef = new MutableObject<>(ntsOp);
 
             // Enters the translation for a subplan.
             context.enterSubplan();
@@ -305,27 +354,36 @@
                     langExprToAlgExpression(joinClause.getConditionExpression(), new MutableObject<>(rightUnnestOp));
             SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
             filter.getInputs().add(conditionExprOpPair.second);
+            filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
 
             ILogicalOperator currentTopOp = filter;
             LogicalVariable varToListify;
             boolean hasRightPosVar = rightUnnestOp.getPositionalVariable() != null;
             if (hasRightPosVar) {
                 // Creates record to get correlation between the two aggregate variables.
+                VariableReferenceExpression rightUnnestVarRef =
+                        new VariableReferenceExpression(rightUnnestOp.getVariable());
+                rightUnnestVarRef.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
+                VariableReferenceExpression rightUnnestPosVarRef =
+                        new VariableReferenceExpression(rightUnnestOp.getPositionalVariable());
+                rightUnnestPosVarRef.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
                 ScalarFunctionCallExpression recordCreationFunc = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR),
                         // Field name for the listified right unnest var.
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AString("unnestvar")))),
                         // The listified right unnest var
-                        new MutableObject<>(new VariableReferenceExpression(rightUnnestOp.getVariable())),
+                        new MutableObject<>(rightUnnestVarRef),
                         // Field name for the listified right unnest positional var.
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AString("posvar")))),
                         // The listified right unnest positional var.
-                        new MutableObject<>(new VariableReferenceExpression(rightUnnestOp.getPositionalVariable())));
+                        new MutableObject<>(rightUnnestPosVarRef));
+                recordCreationFunc.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
 
                 // Assigns the record constructor function to a record variable.
                 LogicalVariable recordVar = context.newVar();
                 AssignOperator assignOp = new AssignOperator(recordVar, new MutableObject<>(recordCreationFunc));
                 assignOp.getInputs().add(new MutableObject<>(currentTopOp));
+                assignOp.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
 
                 // Sets currentTopOp and varToListify for later usages.
                 currentTopOp = assignOp;
@@ -335,14 +393,17 @@
             }
 
             // Adds an aggregate operator to listfy unnest variables.
-            AggregateFunctionCallExpression fListify =
-                    BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.LISTIFY,
-                            mkSingletonArrayList(new MutableObject<>(new VariableReferenceExpression(varToListify))));
+            VariableReferenceExpression varToListifyRef = new VariableReferenceExpression(varToListify);
+            varToListifyRef.setSourceLocation(currentTopOp.getSourceLocation());
+            AggregateFunctionCallExpression fListify = BuiltinFunctions.makeAggregateFunctionExpression(
+                    BuiltinFunctions.LISTIFY, mkSingletonArrayList(new MutableObject<>(varToListifyRef)));
+            fListify.setSourceLocation(currentTopOp.getSourceLocation());
 
             LogicalVariable aggVar = context.newSubplanOutputVar();
             AggregateOperator aggOp = new AggregateOperator(mkSingletonArrayList(aggVar),
                     mkSingletonArrayList(new MutableObject<>(fListify)));
             aggOp.getInputs().add(new MutableObject<>(currentTopOp));
+            aggOp.setSourceLocation(fListify.getSourceLocation());
 
             // Exits the translation of a subplan.
             context.exitSubplan();
@@ -353,20 +414,30 @@
 
             // Outer unnest the aggregated var from the subplan.
             LogicalVariable outerUnnestVar = context.newVar();
+            VariableReferenceExpression aggVarRefExpr = new VariableReferenceExpression(aggVar);
+            aggVarRefExpr.setSourceLocation(aggOp.getSourceLocation());
             LeftOuterUnnestOperator outerUnnestOp = new LeftOuterUnnestOperator(outerUnnestVar,
-                    new MutableObject<>(makeUnnestExpression(new VariableReferenceExpression(aggVar))));
+                    new MutableObject<>(makeUnnestExpression(aggVarRefExpr)));
             outerUnnestOp.getInputs().add(new MutableObject<>(subplanOp));
+            outerUnnestOp.setSourceLocation(aggOp.getSourceLocation());
             currentTopOp = outerUnnestOp;
 
             if (hasRightPosVar) {
+                VariableReferenceExpression outerUnnestVarRef1 = new VariableReferenceExpression(outerUnnestVar);
+                outerUnnestVarRef1.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
                 ScalarFunctionCallExpression fieldAccessForRightUnnestVar = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<>(new VariableReferenceExpression(outerUnnestVar)),
+                        new MutableObject<>(outerUnnestVarRef1),
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
+                fieldAccessForRightUnnestVar.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
+
+                VariableReferenceExpression outerUnnestVarRef2 = new VariableReferenceExpression(outerUnnestVar);
+                outerUnnestVarRef2.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
                 ScalarFunctionCallExpression fieldAccessForRightPosVar = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                        new MutableObject<>(new VariableReferenceExpression(outerUnnestVar)),
+                        new MutableObject<>(outerUnnestVarRef2),
                         new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
+                fieldAccessForRightPosVar.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
 
                 // Creates variables for assign.
                 LogicalVariable rightUnnestVar = context.newVar();
@@ -389,6 +460,7 @@
                 // Creates the assign operator.
                 AssignOperator assignOp = new AssignOperator(assignVars, assignExprs);
                 assignOp.getInputs().add(new MutableObject<>(currentTopOp));
+                assignOp.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
                 currentTopOp = assignOp;
             } else {
                 context.setVar(joinClause.getRightVariable(), outerUnnestVar);
@@ -400,7 +472,8 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(NestClause nestClause, Mutable<ILogicalOperator> arg)
             throws CompilationException {
-        throw new NotImplementedException("Nest clause has not been implemented");
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, nestClause.getSourceLocation(),
+                "Nest clause has not been implemented");
     }
 
     @Override
@@ -426,7 +499,7 @@
         LogicalVariable rightVar = context.newVarFromExpression(binaryCorrelate.getRightVariable());
         Expression rightExpr = binaryCorrelate.getRightExpression();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(rightExpr, inputOpRef);
-        ILogicalOperator unnestOp;
+        AbstractUnnestOperator unnestOp;
         if (binaryCorrelate.hasPositionalVariable()) {
             LogicalVariable pVar = context.newVarFromExpression(binaryCorrelate.getPositionalVariable());
             // We set the positional variable type as BIGINT type.
@@ -440,6 +513,7 @@
                     : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(makeUnnestExpression(eo.first)));
         }
         unnestOp.getInputs().add(eo.second);
+        unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
         return new Pair<>(unnestOp, rightVar);
     }
 
@@ -478,12 +552,14 @@
         List<ILogicalExpression> branchCondVarReferences = new ArrayList<>();
         List<ILogicalExpression> allVarReferences = new ArrayList<>();
         for (int index = 0; index < whenExprList.size(); ++index) {
-            Pair<ILogicalOperator, LogicalVariable> whenExprResult = whenExprList.get(index).accept(this, currentOpRef);
+            Expression whenExpr = whenExprList.get(index);
+            Pair<ILogicalOperator, LogicalVariable> whenExprResult = whenExpr.accept(this, currentOpRef);
             currentOperator = whenExprResult.first;
             // Variable whenConditionVar is corresponds to the current "WHEN" condition.
             LogicalVariable whenConditionVar = whenExprResult.second;
-            Mutable<ILogicalExpression> branchEntraceConditionExprRef =
-                    new MutableObject<>(new VariableReferenceExpression(whenConditionVar));
+            VariableReferenceExpression whenConditionVarRef1 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef1.setSourceLocation(whenExpr.getSourceLocation());
+            Mutable<ILogicalExpression> branchEntraceConditionExprRef = new MutableObject<>(whenConditionVarRef1);
 
             // Constructs an expression that filters data based on preceding "WHEN" conditions
             // and the current "WHEN" condition. Note that only one "THEN" expression can be run
@@ -492,31 +568,45 @@
                 // The additional filter generated here makes sure the the tuple has not
                 // entered other matched "WHEN...THEN" case.
                 List<Mutable<ILogicalExpression>> andArgs = new ArrayList<>();
-                andArgs.add(generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences));
+                andArgs.add(generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences,
+                        caseExpression.getSourceLocation()));
                 andArgs.add(branchEntraceConditionExprRef);
 
                 // A "THEN" branch can be entered only when the tuple has not enter any other preceding
                 // branches and the current "WHEN" condition is TRUE.
-                branchEntraceConditionExprRef = new MutableObject<>(
-                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), andArgs));
+                ScalarFunctionCallExpression andExpr =
+                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), andArgs);
+                andExpr.setSourceLocation(whenExpr.getSourceLocation());
+                branchEntraceConditionExprRef = new MutableObject<>(andExpr);
             }
 
             // Translates the corresponding "THEN" expression.
-            Pair<ILogicalOperator, LogicalVariable> opAndVarForThen = constructSubplanOperatorForBranch(currentOperator,
-                    branchEntraceConditionExprRef, thenExprList.get(index));
+            Expression thenExpr = thenExprList.get(index);
+            Pair<ILogicalOperator, LogicalVariable> opAndVarForThen =
+                    constructSubplanOperatorForBranch(currentOperator, branchEntraceConditionExprRef, thenExpr);
 
-            branchCondVarReferences.add(new VariableReferenceExpression(whenConditionVar));
-            allVarReferences.add(new VariableReferenceExpression(whenConditionVar));
-            allVarReferences.add(new VariableReferenceExpression(opAndVarForThen.second));
+            VariableReferenceExpression whenConditionVarRef2 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef2.setSourceLocation(whenExpr.getSourceLocation());
+            branchCondVarReferences.add(whenConditionVarRef2);
+
+            VariableReferenceExpression whenConditionVarRef3 = new VariableReferenceExpression(whenConditionVar);
+            whenConditionVarRef3.setSourceLocation(whenExpr.getSourceLocation());
+            allVarReferences.add(whenConditionVarRef3);
+
+            VariableReferenceExpression thenVarRef = new VariableReferenceExpression(opAndVarForThen.second);
+            thenVarRef.setSourceLocation(thenExpr.getSourceLocation());
+            allVarReferences.add(thenVarRef);
+
             currentOperator = opAndVarForThen.first;
             currentOpRef = new MutableObject<>(currentOperator);
         }
 
         // Creates a subplan for the "ELSE" branch.
-        Mutable<ILogicalExpression> elseCondExprRef =
-                generateNoMatchedPrecedingWhenBranchesFilter(branchCondVarReferences);
+        Mutable<ILogicalExpression> elseCondExprRef = generateNoMatchedPrecedingWhenBranchesFilter(
+                branchCondVarReferences, caseExpression.getSourceLocation());
+        Expression elseExpr = caseExpression.getElseExpr();
         Pair<ILogicalOperator, LogicalVariable> opAndVarForElse =
-                constructSubplanOperatorForBranch(currentOperator, elseCondExprRef, caseExpression.getElseExpr());
+                constructSubplanOperatorForBranch(currentOperator, elseCondExprRef, elseExpr);
 
         // Uses switch-case function to select the results of two branches.
         LogicalVariable selectVar = context.newVar();
@@ -525,28 +615,77 @@
         for (ILogicalExpression argVar : allVarReferences) {
             arguments.add(new MutableObject<>(argVar));
         }
-        arguments.add(new MutableObject<>(new VariableReferenceExpression(opAndVarForElse.second)));
+        VariableReferenceExpression varForElseRef = new VariableReferenceExpression(opAndVarForElse.second);
+        varForElseRef.setSourceLocation(elseExpr.getSourceLocation());
+        arguments.add(new MutableObject<>(varForElseRef));
         AbstractFunctionCallExpression swithCaseExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SWITCH_CASE), arguments);
+        swithCaseExpr.setSourceLocation(caseExpression.getSourceLocation());
         AssignOperator assignOp = new AssignOperator(selectVar, new MutableObject<>(swithCaseExpr));
         assignOp.getInputs().add(new MutableObject<>(opAndVarForElse.first));
+        assignOp.setSourceLocation(caseExpression.getSourceLocation());
 
         // Unnests the selected (a "THEN" or "ELSE" branch) result.
         LogicalVariable unnestVar = context.newVar();
-        UnnestOperator unnestOp = new UnnestOperator(unnestVar,
-                new MutableObject<>(new UnnestingFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), Collections
-                                .singletonList(new MutableObject<>(new VariableReferenceExpression(selectVar))))));
+        VariableReferenceExpression selectVarRef = new VariableReferenceExpression(selectVar);
+        selectVarRef.setSourceLocation(caseExpression.getSourceLocation());
+        UnnestingFunctionCallExpression scanCollectionExpr =
+                new UnnestingFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+                        Collections.singletonList(new MutableObject<>(selectVarRef)));
+        scanCollectionExpr.setSourceLocation(caseExpression.getSourceLocation());
+        UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollectionExpr));
         unnestOp.getInputs().add(new MutableObject<>(assignOp));
+        unnestOp.setSourceLocation(caseExpression.getSourceLocation());
 
         // Produces the final assign operator.
         LogicalVariable resultVar = context.newVar();
-        AssignOperator finalAssignOp =
-                new AssignOperator(resultVar, new MutableObject<>(new VariableReferenceExpression(unnestVar)));
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(caseExpression.getSourceLocation());
+        AssignOperator finalAssignOp = new AssignOperator(resultVar, new MutableObject<>(unnestVarRef));
         finalAssignOp.getInputs().add(new MutableObject<>(unnestOp));
+        finalAssignOp.setSourceLocation(caseExpression.getSourceLocation());
         return new Pair<>(finalAssignOp, resultVar);
     }
 
+    @Override
+    protected ILogicalExpression translateVariableRef(VariableExpr varExpr) throws CompilationException {
+        VarIdentifier varId = varExpr.getVar();
+        if (SqlppVariableUtil.isExternalVariableIdentifier(varId)) {
+            SourceLocation sourceLoc = varExpr.getSourceLocation();
+            IAObject value = getExternalVariableValue(varId, sourceLoc);
+            return translateConstantValue(value, sourceLoc);
+        }
+
+        return super.translateVariableRef(varExpr);
+    }
+
+    private IAObject getExternalVariableValue(VarIdentifier varId, SourceLocation sourceLoc)
+            throws CompilationException {
+        IAObject value = externalVars.get(varId);
+        if (value == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, varId.toString());
+        }
+        return value;
+    }
+
+    private ILogicalExpression translateConstantValue(IAObject value, SourceLocation sourceLoc)
+            throws CompilationException {
+        ConstantExpression constExpr = new ConstantExpression(new AsterixConstantValue(value));
+        constExpr.setSourceLocation(sourceLoc);
+
+        IAType valueType = value.getType();
+        if (valueType.getTypeTag().isDerivedType()) {
+            ScalarFunctionCallExpression castExpr =
+                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE));
+            castExpr.setSourceLocation(sourceLoc);
+            castExpr.getArguments().add(new MutableObject<>(constExpr));
+            TypeCastUtils.setRequiredAndInputTypes(castExpr, BuiltinType.ANY, valueType);
+            return castExpr;
+        } else {
+            return constExpr;
+        }
+    }
+
     private Pair<ILogicalOperator, LogicalVariable> produceSelectPlan(boolean isSubquery,
             Mutable<ILogicalOperator> returnOpRef, LogicalVariable resVar) {
         if (isSubquery) {
@@ -554,6 +693,7 @@
         } else {
             ProjectOperator pr = new ProjectOperator(resVar);
             pr.getInputs().add(returnOpRef);
+            pr.setSourceLocation(returnOpRef.getValue().getSourceLocation());
             return new Pair<>(pr, resVar);
         }
     }
@@ -567,19 +707,26 @@
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(returnExpr, tupSrc);
         LogicalVariable returnVar;
         ILogicalOperator returnOperator;
-        if (returnExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
+        SourceLocation sourceLoc = returnExpr.getSourceLocation();
+        if (returnExpr.getKind() == Kind.VARIABLE_EXPRESSION
+                && eo.first.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
             VariableExpr varExpr = (VariableExpr) returnExpr;
             returnOperator = eo.second.getValue();
             returnVar = context.getVar(varExpr.getVar().getId());
         } else {
             returnVar = context.newVar();
-            returnOperator = new AssignOperator(returnVar, new MutableObject<>(eo.first));
-            returnOperator.getInputs().add(eo.second);
+            AssignOperator assignOp = new AssignOperator(returnVar, new MutableObject<>(eo.first));
+            assignOp.getInputs().add(eo.second);
+            assignOp.setSourceLocation(sourceLoc);
+            returnOperator = assignOp;
         }
         if (selectClause.distinct()) {
-            DistinctOperator distinctOperator = new DistinctOperator(
-                    mkSingletonArrayList(new MutableObject<>(new VariableReferenceExpression(returnVar))));
+            VariableReferenceExpression returnVarRef = new VariableReferenceExpression(returnVar);
+            returnVarRef.setSourceLocation(sourceLoc);
+            DistinctOperator distinctOperator =
+                    new DistinctOperator(mkSingletonArrayList(new MutableObject<>(returnVarRef)));
             distinctOperator.getInputs().add(new MutableObject<>(returnOperator));
+            distinctOperator.setSourceLocation(returnOperator.getSourceLocation());
             return new Pair<>(distinctOperator, returnVar);
         } else {
             return new Pair<>(returnOperator, returnVar);
@@ -587,87 +734,287 @@
     }
 
     // Generates the return expression for a regular select clause.
-    private Expression generateReturnExpr(SelectRegular selectRegular, SelectBlock selectBlock) {
+    private Expression generateReturnExpr(SelectRegular selectRegular, SelectBlock selectBlock)
+            throws CompilationException {
         List<Expression> recordExprs = new ArrayList<>();
         List<FieldBinding> fieldBindings = new ArrayList<>();
+        Set<String> fieldNames = new HashSet<>();
+
         for (Projection projection : selectRegular.getProjections()) {
             if (projection.varStar()) {
                 if (!fieldBindings.isEmpty()) {
-                    recordExprs.add(new RecordConstructor(new ArrayList<>(fieldBindings)));
+                    RecordConstructor recordConstr = new RecordConstructor(new ArrayList<>(fieldBindings));
+                    recordConstr.setSourceLocation(selectRegular.getSourceLocation());
+                    recordExprs.add(recordConstr);
                     fieldBindings.clear();
                 }
-                recordExprs.add(projection.getExpression());
+                Expression projectionExpr = projection.getExpression();
+                SourceLocation sourceLoc = projection.getSourceLocation();
+                CallExpr toObjectExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.TO_OBJECT),
+                        Collections.singletonList(projectionExpr));
+                toObjectExpr.setSourceLocation(sourceLoc);
+                CallExpr ifMissingOrNullExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.IF_MISSING_OR_NULL),
+                        Arrays.asList(toObjectExpr, new RecordConstructor(Collections.emptyList())));
+                ifMissingOrNullExpr.setSourceLocation(sourceLoc);
+                recordExprs.add(ifMissingOrNullExpr);
             } else if (projection.star()) {
                 if (selectBlock.hasGroupbyClause()) {
-                    getGroupBindings(selectBlock.getGroupbyClause(), fieldBindings);
+                    getGroupBindings(selectBlock.getGroupbyClause(), fieldBindings, fieldNames);
                     if (selectBlock.hasLetClausesAfterGroupby()) {
-                        getLetBindings(selectBlock.getLetListAfterGroupby(), fieldBindings);
+                        getLetBindings(selectBlock.getLetListAfterGroupby(), fieldBindings, fieldNames);
                     }
                 } else if (selectBlock.hasFromClause()) {
-                    getFromBindings(selectBlock.getFromClause(), fieldBindings);
+                    getFromBindings(selectBlock.getFromClause(), fieldBindings, fieldNames);
                     if (selectBlock.hasLetClauses()) {
-                        getLetBindings(selectBlock.getLetList(), fieldBindings);
+                        getLetBindings(selectBlock.getLetList(), fieldBindings, fieldNames);
                     }
                 } else if (selectBlock.hasLetClauses()) {
-                    getLetBindings(selectBlock.getLetList(), fieldBindings);
+                    getLetBindings(selectBlock.getLetList(), fieldBindings, fieldNames);
                 }
+            } else if (projection.hasName()) {
+                fieldBindings.add(getFieldBinding(projection, fieldNames));
             } else {
-                fieldBindings.add(new FieldBinding(new LiteralExpr(new StringLiteral(projection.getName())),
-                        projection.getExpression()));
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation());
             }
         }
         if (!fieldBindings.isEmpty()) {
-            recordExprs.add(new RecordConstructor(fieldBindings));
+            RecordConstructor recordConstr = new RecordConstructor(fieldBindings);
+            recordConstr.setSourceLocation(selectRegular.getSourceLocation());
+            recordExprs.add(recordConstr);
         }
 
-        return recordExprs.size() == 1 ? recordExprs.get(0)
-                : new CallExpr(new FunctionSignature(BuiltinFunctions.RECORD_CONCAT_STRICT), recordExprs);
+        if (recordExprs.size() == 1) {
+            return recordExprs.get(0);
+        } else {
+            CallExpr recordConcatExpr =
+                    new CallExpr(new FunctionSignature(BuiltinFunctions.RECORD_CONCAT_STRICT), recordExprs);
+            recordConcatExpr.setSourceLocation(selectRegular.getSourceLocation());
+            return recordConcatExpr;
+        }
     }
 
     // Generates all field bindings according to the from clause.
-    private void getFromBindings(FromClause fromClause, List<FieldBinding> outFieldBindings) {
+    private void getFromBindings(FromClause fromClause, List<FieldBinding> outFieldBindings, Set<String> outFieldNames)
+            throws CompilationException {
         for (FromTerm fromTerm : fromClause.getFromTerms()) {
-            outFieldBindings.add(getFieldBinding(fromTerm.getLeftVariable()));
+            outFieldBindings.add(getFieldBinding(fromTerm.getLeftVariable(), outFieldNames));
             if (fromTerm.hasPositionalVariable()) {
-                outFieldBindings.add(getFieldBinding(fromTerm.getPositionalVariable()));
+                outFieldBindings.add(getFieldBinding(fromTerm.getPositionalVariable(), outFieldNames));
             }
             if (!fromTerm.hasCorrelateClauses()) {
                 continue;
             }
             for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-                outFieldBindings.add(getFieldBinding(correlateClause.getRightVariable()));
+                outFieldBindings.add(getFieldBinding(correlateClause.getRightVariable(), outFieldNames));
                 if (correlateClause.hasPositionalVariable()) {
-                    outFieldBindings.add(getFieldBinding(correlateClause.getPositionalVariable()));
+                    outFieldBindings.add(getFieldBinding(correlateClause.getPositionalVariable(), outFieldNames));
                 }
             }
         }
     }
 
     // Generates all field bindings according to the from clause.
-    private void getGroupBindings(GroupbyClause groupbyClause, List<FieldBinding> outFieldBindings) {
+    private void getGroupBindings(GroupbyClause groupbyClause, List<FieldBinding> outFieldBindings,
+            Set<String> outFieldNames) throws CompilationException {
         for (GbyVariableExpressionPair pair : groupbyClause.getGbyPairList()) {
-            outFieldBindings.add(getFieldBinding(pair.getVar()));
+            outFieldBindings.add(getFieldBinding(pair.getVar(), outFieldNames));
         }
         if (groupbyClause.hasGroupVar()) {
-            outFieldBindings.add(getFieldBinding(groupbyClause.getGroupVar()));
+            outFieldBindings.add(getFieldBinding(groupbyClause.getGroupVar(), outFieldNames));
         }
         if (groupbyClause.hasWithMap()) {
-            throw new IllegalStateException(groupbyClause.getWithVarMap().values().toString()); // no WITH in SQLPP
+            // no WITH in SQLPP
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, groupbyClause.getSourceLocation(),
+                    groupbyClause.getWithVarMap().values().toString());
         }
     }
 
     // Generates all field bindings according to the let clause.
-    private void getLetBindings(List<LetClause> letClauses, List<FieldBinding> outFieldBindings) {
+    private void getLetBindings(List<LetClause> letClauses, List<FieldBinding> outFieldBindings,
+            Set<String> outFieldNames) throws CompilationException {
         for (LetClause letClause : letClauses) {
-            outFieldBindings.add(getFieldBinding(letClause.getVarExpr()));
+            outFieldBindings.add(getFieldBinding(letClause.getVarExpr(), outFieldNames));
         }
     }
 
     // Generates a field binding for a variable.
-    private FieldBinding getFieldBinding(VariableExpr var) {
-        LiteralExpr fieldName = new LiteralExpr(
-                new StringLiteral(SqlppVariableUtil.variableNameToDisplayedFieldName(var.getVar().getValue())));
-        return new FieldBinding(fieldName, var);
+    private FieldBinding getFieldBinding(VariableExpr varExpr, Set<String> outFieldNames) throws CompilationException {
+        String fieldName = SqlppVariableUtil.variableNameToDisplayedFieldName(varExpr.getVar().getValue());
+        return generateFieldBinding(fieldName, varExpr, outFieldNames, varExpr.getSourceLocation());
     }
 
+    // Generates a field binding for a named projection.
+    private FieldBinding getFieldBinding(Projection projection, Set<String> outFieldNames) throws CompilationException {
+        String fieldName = projection.getName();
+        Expression fieldValueExpr = projection.getExpression();
+        return generateFieldBinding(fieldName, fieldValueExpr, outFieldNames, projection.getSourceLocation());
+    }
+
+    private FieldBinding generateFieldBinding(String fieldName, Expression fieldValueExpr, Set<String> outFieldNames,
+            SourceLocation sourceLoc) throws CompilationException {
+        if (!outFieldNames.add(fieldName)) {
+            throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, sourceLoc, fieldName);
+        }
+        return new FieldBinding(new LiteralExpr(new StringLiteral(fieldName)), fieldValueExpr);
+    }
+
+    @Override
+    protected boolean expressionNeedsNoNesting(Expression expr) {
+        return super.expressionNeedsNoNesting(expr) || (translateInAsOr && expr.getKind() == Kind.QUANTIFIED_EXPRESSION
+                && isInOperatorWithStaticList((QuantifiedExpression) expr));
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
+            throws CompilationException {
+        return translateInAsOr && isInOperatorWithStaticList(qe) ? translateInOperatorWithStaticList(qe, tupSource)
+                : super.visit(qe, tupSource);
+    }
+
+    // At this point "$x in list_expr" is a quantified expression:
+    // "some $y in list_expr satisfies $x = $y"
+    // Look for such quantified expression with a constant list_expr ([e1, e2, ... eN])
+    // and translate it into "$x=e1 || $x=e2 || ... || $x=eN"
+    private boolean isInOperatorWithStaticList(QuantifiedExpression qe) {
+        if (qe.getQuantifier() != QuantifiedExpression.Quantifier.SOME) {
+            return false;
+        }
+        List<QuantifiedPair> qpList = qe.getQuantifiedList();
+        if (qpList.size() != 1) {
+            return false;
+        }
+        QuantifiedPair qp = qpList.get(0);
+
+        Expression condExpr = qe.getSatisfiesExpr();
+        if (condExpr.getKind() != Kind.OP_EXPRESSION) {
+            return false;
+        }
+        OperatorExpr opExpr = (OperatorExpr) condExpr;
+        if (opExpr.getOpList().get(0) != OperatorType.EQ) {
+            return false;
+        }
+        List<Expression> operandExprs = opExpr.getExprList();
+        if (operandExprs.size() != 2) {
+            return false;
+        }
+        int varPos = operandExprs.indexOf(qp.getVarExpr());
+        if (varPos < 0) {
+            return false;
+        }
+        Expression inExpr = qp.getExpr();
+        switch (inExpr.getKind()) {
+            case LIST_CONSTRUCTOR_EXPRESSION:
+                ListConstructor listExpr = (ListConstructor) inExpr;
+                List<Expression> itemExprs = listExpr.getExprList();
+                if (itemExprs.isEmpty()) {
+                    return false;
+                }
+                for (Expression itemExpr : itemExprs) {
+                    boolean isConst = itemExpr.getKind() == Kind.LITERAL_EXPRESSION
+                            || (itemExpr.getKind() == Kind.VARIABLE_EXPRESSION
+                                    && SqlppVariableUtil.isExternalVariableReference((VariableExpr) itemExpr));
+                    if (!isConst) {
+                        return false;
+                    }
+                }
+                return true;
+            case VARIABLE_EXPRESSION:
+                VarIdentifier inVarId = ((VariableExpr) inExpr).getVar();
+                if (!SqlppVariableUtil.isExternalVariableIdentifier(inVarId)) {
+                    return false;
+                }
+                IAObject inValue = externalVars.get(inVarId);
+                return inValue != null && inValue.getType().getTypeTag().isListType()
+                        && ((IACollection) inValue).size() > 0;
+            default:
+                return false;
+        }
+    }
+
+    private Pair<ILogicalOperator, LogicalVariable> translateInOperatorWithStaticList(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws CompilationException {
+        SourceLocation sourceLoc = qe.getSourceLocation();
+
+        QuantifiedPair qp = qe.getQuantifiedList().get(0);
+        VariableExpr varExpr = qp.getVarExpr();
+        List<Expression> operandExprs = ((OperatorExpr) qe.getSatisfiesExpr()).getExprList();
+        int varIdx = operandExprs.indexOf(varExpr);
+        Expression operandExpr = operandExprs.get(1 - varIdx);
+
+        Mutable<ILogicalOperator> topOp = tupSource;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(operandExpr, topOp);
+        topOp = eo1.second;
+
+        LogicalVariable operandVar = context.newVar();
+        AssignOperator operandAssign = new AssignOperator(operandVar, new MutableObject<>(eo1.first));
+        operandAssign.getInputs().add(topOp);
+        operandAssign.setSourceLocation(sourceLoc);
+        topOp = new MutableObject<>(operandAssign);
+
+        List<MutableObject<ILogicalExpression>> disjuncts = new ArrayList<>();
+        Expression inExpr = qp.getExpr();
+        switch (inExpr.getKind()) {
+            case LIST_CONSTRUCTOR_EXPRESSION:
+                ListConstructor listExpr = (ListConstructor) inExpr;
+                for (Expression itemExpr : listExpr.getExprList()) {
+                    IAObject inValue;
+                    switch (itemExpr.getKind()) {
+                        case LITERAL_EXPRESSION:
+                            inValue = ConstantHelper.objectFromLiteral(((LiteralExpr) itemExpr).getValue());
+                            break;
+                        case VARIABLE_EXPRESSION:
+                            inValue = getExternalVariableValue(((VariableExpr) itemExpr).getVar(), sourceLoc);
+                            break;
+                        default:
+                            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                                    itemExpr.getKind());
+                    }
+                    ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+                    disjuncts.add(new MutableObject<>(eqExpr));
+                }
+                break;
+            case VARIABLE_EXPRESSION:
+                VarIdentifier inVarId = ((VariableExpr) inExpr).getVar();
+                IAObject inVarValue = externalVars.get(inVarId);
+                IACursor inVarCursor = ((IACollection) inVarValue).getCursor();
+                inVarCursor.reset();
+                while (inVarCursor.next()) {
+                    IAObject inValue = inVarCursor.get();
+                    ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+                    disjuncts.add(new MutableObject<>(eqExpr));
+                }
+                break;
+            default:
+                throw new IllegalStateException(String.valueOf(inExpr.getKind()));
+        }
+
+        MutableObject<ILogicalExpression> condExpr;
+        if (disjuncts.size() == 1) {
+            condExpr = disjuncts.get(0);
+        } else {
+            AbstractFunctionCallExpression orExpr =
+                    createFunctionCallExpressionForBuiltinOperator(OperatorType.OR, sourceLoc);
+            orExpr.getArguments().addAll(disjuncts);
+            condExpr = new MutableObject<>(orExpr);
+        }
+
+        LogicalVariable assignVar = context.newVar();
+        AssignOperator assignOp = new AssignOperator(assignVar, condExpr);
+        assignOp.getInputs().add(topOp);
+        assignOp.setSourceLocation(sourceLoc);
+        return new Pair<>(assignOp, assignVar);
+    }
+
+    private ILogicalExpression createEqExpr(LogicalVariable lhsVar, IAObject rhsValue, SourceLocation sourceLoc)
+            throws CompilationException {
+        VariableReferenceExpression lhsExpr = new VariableReferenceExpression(lhsVar);
+        lhsExpr.setSourceLocation(sourceLoc);
+        ILogicalExpression rhsExpr = translateConstantValue(rhsValue, sourceLoc);
+        AbstractFunctionCallExpression opExpr =
+                createFunctionCallExpressionForBuiltinOperator(OperatorType.EQ, sourceLoc);
+        opExpr.getArguments().add(new MutableObject<>(lhsExpr));
+        opExpr.getArguments().add(new MutableObject<>(rhsExpr));
+        return opExpr;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslatorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslatorFactory.java
index 49c088f..65d90ae 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslatorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslatorFactory.java
@@ -18,17 +18,21 @@
  */
 package org.apache.asterix.translator;
 
+import java.util.Map;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class SqlppExpressionToPlanTranslatorFactory implements ILangExpressionToPlanTranslatorFactory {
 
     @Override
     public ILangExpressionToPlanTranslator createExpressionToPlanTranslator(MetadataProvider metadataProvider,
-            int currentVarCounter) throws AlgebricksException {
-        return new SqlppExpressionToPlanTranslator(metadataProvider, currentVarCounter);
+            int currentVarCounter, Map<VarIdentifier, IAObject> externalVars) throws AlgebricksException {
+        return new SqlppExpressionToPlanTranslator(metadataProvider, currentVarCounter, externalVars);
     }
 
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
deleted file mode 100644
index 0660f23..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationException.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.asterix.translator;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-
-public class TranslationException extends CompilationException {
-    private static final long serialVersionUID = 685960054131778068L;
-
-    public TranslationException(String msg) {
-        super(msg);
-    }
-}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
index 919bdf2..d2466bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/PlanTranslationUtil.java
@@ -36,43 +36,52 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class PlanTranslationUtil {
     private static final LogicalVariable DUMMY_VAR = new LogicalVariable(-1);
 
     public static void prepareMetaKeyAccessExpression(List<String> field, LogicalVariable resVar,
             List<Mutable<ILogicalExpression>> assignExpressions, List<LogicalVariable> vars,
-            List<Mutable<ILogicalExpression>> varRefs, IVariableContext context) {
+            List<Mutable<ILogicalExpression>> varRefs, IVariableContext context, SourceLocation sourceLoc) {
         IAObject value = (field.size() > 1) ? new AOrderedList(field) : new AString(field.get(0));
         ScalarFunctionCallExpression metaKeyFunction =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.META_KEY));
-        metaKeyFunction.getArguments()
-                .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+        metaKeyFunction.setSourceLocation(sourceLoc);
+        VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar);
+        resVarRef.setSourceLocation(sourceLoc);
+        metaKeyFunction.getArguments().add(new MutableObject<ILogicalExpression>(resVarRef));
         metaKeyFunction.getArguments()
                 .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(value))));
         assignExpressions.add(new MutableObject<ILogicalExpression>(metaKeyFunction));
         LogicalVariable v = context.newVar();
         vars.add(v);
         if (varRefs != null) {
-            varRefs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            varRefs.add(new MutableObject<ILogicalExpression>(vRef));
         }
     }
 
     public static void prepareVarAndExpression(List<String> field, LogicalVariable resVar, List<LogicalVariable> vars,
             List<Mutable<ILogicalExpression>> assignExpressions, List<Mutable<ILogicalExpression>> varRefs,
-            IVariableContext context) {
-        ScalarFunctionCallExpression f = createFieldAccessExpression(new VariableReferenceExpression(DUMMY_VAR), field);
+            IVariableContext context, SourceLocation sourceLoc) {
+        VariableReferenceExpression dummyVarRef = new VariableReferenceExpression(DUMMY_VAR);
+        dummyVarRef.setSourceLocation(sourceLoc);
+        ScalarFunctionCallExpression f = createFieldAccessExpression(dummyVarRef, field, sourceLoc);
         f.substituteVar(DUMMY_VAR, resVar);
         assignExpressions.add(new MutableObject<ILogicalExpression>(f));
         LogicalVariable v = context.newVar();
         vars.add(v);
         if (varRefs != null) {
-            varRefs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+            vRef.setSourceLocation(sourceLoc);
+            varRefs.add(new MutableObject<ILogicalExpression>(vRef));
         }
     }
 
     private static ScalarFunctionCallExpression createFieldAccessExpression(ILogicalExpression target,
-            List<String> field) {
+            List<String> field, SourceLocation sourceLoc) {
         FunctionIdentifier functionIdentifier;
         IAObject value;
         if (field.size() > 1) {
@@ -83,7 +92,9 @@
             value = new AString(field.get(0));
         }
         IFunctionInfo finfoAccess = FunctionUtil.getFunctionInfo(functionIdentifier);
-        return new ScalarFunctionCallExpression(finfoAccess, new MutableObject<>(target),
+        ScalarFunctionCallExpression faExpr = new ScalarFunctionCallExpression(finfoAccess, new MutableObject<>(target),
                 new MutableObject<>(new ConstantExpression(new AsterixConstantValue(value))));
+        faExpr.setSourceLocation(sourceLoc);
+        return faExpr;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 4b4b2b0..ffb1dd5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * A util that can verify if a filter field, a list of partitioning expressions,
@@ -50,16 +51,17 @@
      *            the record type
      * @param filterField
      *            the full name of the field
+     * @param sourceLoc
      * @throws AlgebricksException
      *             if field is not found in record.
      *             if field type can't be a filter type.
      *             if field type is nullable.
      */
-    public static void validateFilterField(ARecordType recordType, List<String> filterField)
+    public static void validateFilterField(ARecordType recordType, List<String> filterField, SourceLocation sourceLoc)
             throws AlgebricksException {
         IAType fieldType = recordType.getSubFieldType(filterField);
         if (fieldType == null) {
-            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
+            throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
                     RecordUtil.toFullyQualifiedName(filterField));
         }
         switch (fieldType.getTypeTag()) {
@@ -100,6 +102,7 @@
      *            the key sources (record vs. meta)
      * @param autogenerated
      *            true if auto generated, false otherwise
+     * @param sourceLoc
      * @return a list of partitioning expressions types
      * @throws AlgebricksException
      *             if composite key is autogenerated.
@@ -109,33 +112,42 @@
      *             if the field type can't be a primary key.
      */
     public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
-            List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated)
-            throws AlgebricksException {
+            List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated,
+            SourceLocation sourceLoc) throws AlgebricksException {
         List<IAType> partitioningExprTypes = new ArrayList<>(partitioningExprs.size());
         if (autogenerated) {
             if (partitioningExprs.size() > 1) {
-                throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY);
+                throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY,
+                        sourceLoc);
             }
             List<String> fieldName = partitioningExprs.get(0);
             IAType fieldType = recType.getSubFieldType(fieldName);
             if (fieldType == null) {
                 String unTypeField = fieldName.get(0) == null ? "" : fieldName.get(0);
-                throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, unTypeField);
+                throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc, unTypeField);
             }
             partitioningExprTypes.add(fieldType);
             ATypeTag pkTypeTag = fieldType.getTypeTag();
             if (pkTypeTag != ATypeTag.UUID) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, pkTypeTag.name(),
-                        ATypeTag.UUID.name());
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, sourceLoc,
+                        pkTypeTag.name(), ATypeTag.UUID.name());
             }
         } else {
             partitioningExprTypes =
                     KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, partitioningExprs, keySourceIndicators);
-            for (int fidx = 0; fidx < partitioningExprTypes.size(); ++fidx) {
-                IAType fieldType = partitioningExprTypes.get(fidx);
+            for (int i = 0; i < partitioningExprs.size(); i++) {
+                List<String> partitioningExpr = partitioningExprs.get(i);
+                IAType fieldType = partitioningExprTypes.get(i);
                 if (fieldType == null) {
-                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
-                            RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+                    throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
+                            RecordUtil.toFullyQualifiedName(partitioningExpr));
+                }
+                boolean nullable = KeyFieldTypeUtil.chooseSource(keySourceIndicators, i, recType, metaRecType)
+                        .isSubFieldNullable(partitioningExpr);
+                if (nullable) {
+                    // key field is nullable
+                    throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+                            RecordUtil.toFullyQualifiedName(partitioningExpr));
                 }
                 switch (fieldType.getTypeTag()) {
                     case TINYINT:
@@ -154,10 +166,10 @@
                     case DAYTIMEDURATION:
                         break;
                     case UNION:
-                        throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE,
-                                RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+                        throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+                                RecordUtil.toFullyQualifiedName(partitioningExpr));
                     default:
-                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE,
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE, sourceLoc,
                                 fieldType.getTypeTag());
                 }
             }
diff --git a/asterixdb/asterix-app/data/country/country.adm b/asterixdb/asterix-app/data/country/country.adm
new file mode 100644
index 0000000..9647891
--- /dev/null
+++ b/asterixdb/asterix-app/data/country/country.adm
@@ -0,0 +1,61019 @@
+{
+  "-car_code": "AL",
+  "-area": "28750",
+  "-capital": "cty-cid-cia-Albania-Tirane",
+  "-memberships": "org-BSEC org-CE org-CCC org-ECE org-EBRD org-FAO org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-ANC org-OSCE org-OIC org-PFP org-UN org-UNESCO org-UNIDO org-UNOMIG org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Albania",
+  "population": "3249136",
+  "population_growth": "1.34",
+  "infant_mortality": "49.2",
+  "gdp_total": "4100",
+  "gdp_agri": "55",
+  "inflation": "16",
+  "indep_date": "1912-11-28",
+  "government": "emerging democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "Greeks"
+    },
+    {
+      "-percentage": "95",
+      "#text": "Albanian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "70",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "GR",
+      "-length": "282"
+    },
+    {
+      "-country": "MK",
+      "-length": "151"
+    },
+    {
+      "-country": "MNE",
+      "-length": "172"
+    },
+    {
+      "-country": "KOS",
+      "-length": "112"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Albania-Tirane",
+      "-is_country_cap": "yes",
+      "-country": "AL",
+      "name": "Tirane",
+      "longitude": "19.8",
+      "latitude": "41.3",
+      "population": {
+        "-year": "87",
+        "#text": "192000"
+      }
+    },
+    {
+      "-id": "stadt-Shkoder-AL-AL",
+      "-country": "AL",
+      "name": "Shkoder",
+      "longitude": "19.2",
+      "latitude": "42.2",
+      "population": {
+        "-year": "87",
+        "#text": "62000"
+      },
+      "located_at": {
+        "-watertype": "lake",
+        "-lake": "lake-Skutarisee"
+      }
+    },
+    {
+      "-id": "stadt-Durres-AL-AL",
+      "-country": "AL",
+      "name": "Durres",
+      "longitude": "19.3",
+      "latitude": "41.3",
+      "population": {
+        "-year": "87",
+        "#text": "60000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "stadt-Vlore-AL-AL",
+      "-country": "AL",
+      "name": "Vlore",
+      "longitude": "19.3",
+      "latitude": "40.3",
+      "population": {
+        "-year": "87",
+        "#text": "56000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "stadt-Elbasan-AL-AL",
+      "-country": "AL",
+      "name": "Elbasan",
+      "longitude": "20.1",
+      "latitude": "41.1",
+      "population": {
+        "-year": "87",
+        "#text": "53000"
+      }
+    },
+    {
+      "-id": "stadt-Korce-AL-AL",
+      "-country": "AL",
+      "name": "Korce",
+      "longitude": "20.5",
+      "latitude": "40.4",
+      "population": {
+        "-year": "87",
+        "#text": "52000"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "GR",
+  "-area": "131940",
+  "-capital": "cty-Greece-Athens",
+  "-memberships": "org-BIS org-BSEC org-CE org-CCC org-ECE org-EBRD org-EIB org-CERN org-EU org-FAO org-G-6 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNOMIG org-UNHCR org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Greece",
+  "population": "10538594",
+  "population_growth": "0.42",
+  "infant_mortality": "7.4",
+  "gdp_total": "101700",
+  "gdp_agri": "11.8",
+  "gdp_ind": "22.2",
+  "gdp_serv": "66",
+  "inflation": "8.1",
+  "indep_date": "1829-01-01",
+  "government": "parliamentary republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "98",
+    "#text": "Greek"
+  },
+  "religions": [
+    {
+      "-percentage": "1.3",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "98",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AL",
+      "-length": "282"
+    },
+    {
+      "-country": "MK",
+      "-length": "228"
+    },
+    {
+      "-country": "BG",
+      "-length": "494"
+    },
+    {
+      "-country": "TR",
+      "-length": "206"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Greece-2",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Komotini",
+      "name": "Anatoliki Makedhonia kai Thraki",
+      "area": "14157",
+      "population": "574308",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Greece-Komotini",
+          "-country": "GR",
+          "-is_state_cap": "yes",
+          "-province": "prov-cid-cia-Greece-2",
+          "name": "Komotini"
+        },
+        {
+          "-id": "cty-cid-cia-Greece-9",
+          "-country": "GR",
+          "-province": "prov-cid-cia-Greece-2",
+          "name": "Kavalla",
+          "population": {
+            "-year": "81",
+            "#text": "56705"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Greece-3",
+      "-country": "GR",
+      "-capital": "cty-Greece-Athens",
+      "name": "Attiki",
+      "area": "3808",
+      "population": "3522769",
+      "city": [
+        {
+          "-id": "cty-Greece-Athens",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "GR",
+          "-province": "prov-cid-cia-Greece-3",
+          "name": "Athens",
+          "longitude": "23.7167",
+          "latitude": "37.9667",
+          "population": {
+            "-year": "81",
+            "#text": "885737"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Greece-4",
+          "-country": "GR",
+          "-province": "prov-cid-cia-Greece-3",
+          "name": "Piraeus",
+          "population": {
+            "-year": "81",
+            "#text": "196389"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Greece-4",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-5",
+      "name": "Dhytiki Ellas",
+      "area": "11350",
+      "population": "702027",
+      "city": {
+        "-id": "cty-cid-cia-Greece-5",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-4",
+        "name": "Patrai",
+        "population": {
+          "-year": "81",
+          "#text": "142163"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-5",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Kozani",
+      "name": "Dhytiki Makedhonia",
+      "area": "9451",
+      "population": "292751",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Kozani",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-5",
+        "name": "Kozani"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-6",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Korfu",
+      "name": "Ionioi Nisoi",
+      "area": "2307",
+      "population": "191003",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Korfu",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-6",
+        "name": "Korfu",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-7",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Ioannina",
+      "name": "Ipiros",
+      "area": "9203",
+      "population": "339210",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Ioannina",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-7",
+        "name": "Ioannina"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-8",
+      "-country": "GR",
+      "-capital": "cty-Greece-Thessaloniki",
+      "name": "Kedriki Makedhonia",
+      "area": "19147",
+      "population": "1729581",
+      "city": {
+        "-id": "cty-Greece-Thessaloniki",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-8",
+        "name": "Thessaloniki",
+        "longitude": "22.95",
+        "latitude": "40.6167",
+        "population": {
+          "-year": "81",
+          "#text": "406413"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-9",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-7",
+      "name": "Kriti",
+      "area": "8336",
+      "population": "537183",
+      "city": {
+        "-id": "cty-cid-cia-Greece-7",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-9",
+        "name": "Iraklion",
+        "population": {
+          "-year": "81",
+          "#text": "102398"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        },
+        "located_on": { "-island": "island-Crete" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-10",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Ermoupoli",
+      "name": "Notion Aiyaion",
+      "area": "5286",
+      "population": "262522",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Ermoupoli",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-10",
+        "name": "Ermoupoli",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-11",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Tripoli",
+      "name": "Peloponnisos",
+      "area": "15490",
+      "population": "605663",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Tripoli",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-11",
+        "name": "Tripoli"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-12",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Lamia",
+      "name": "Sterea Ellas",
+      "area": "15549",
+      "population": "578876",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Lamia",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-12",
+        "name": "Lamia"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Greece-13",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-6",
+      "name": "Thessalia",
+      "area": "14037",
+      "population": "731030",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Greece-6",
+          "-country": "GR",
+          "-is_state_cap": "yes",
+          "-province": "prov-cid-cia-Greece-13",
+          "name": "Larisa",
+          "population": {
+            "-year": "81",
+            "#text": "102426"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Greece-8",
+          "-country": "GR",
+          "-province": "prov-cid-cia-Greece-13",
+          "name": "Volos",
+          "population": {
+            "-year": "81",
+            "#text": "71378"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Greece-14",
+      "-country": "GR",
+      "-capital": "cty-cid-cia-Greece-Mytilini",
+      "name": "Voreion Aiyaion",
+      "area": "3836",
+      "population": "189541",
+      "city": {
+        "-id": "cty-cid-cia-Greece-Mytilini",
+        "-country": "GR",
+        "-is_state_cap": "yes",
+        "-province": "prov-cid-cia-Greece-14",
+        "name": "Mytilini",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        },
+        "located_on": { "-island": "island-Lesbos" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MK",
+  "-area": "25333",
+  "-capital": "cty-cid-cia-Macedonia-Skopje",
+  "-memberships": "org-CEI org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ITU org-Intelsat org-OSCE org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Macedonia",
+  "population": "2104035",
+  "population_growth": "0.46",
+  "infant_mortality": "29.7",
+  "gdp_total": "1900",
+  "gdp_agri": "24",
+  "gdp_ind": "44",
+  "gdp_serv": "32",
+  "inflation": "14.8",
+  "indep_date": "1991-09-17",
+  "government": "emerging democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "22",
+      "#text": "Albanian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "65",
+      "#text": "Macedonian"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Turkish"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Gypsy"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "30",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "67",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "21",
+      "#text": "Albanian"
+    },
+    {
+      "-percentage": "70",
+      "#text": "Macedonian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Turkish"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Serbian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AL",
+      "-length": "151"
+    },
+    {
+      "-country": "GR",
+      "-length": "228"
+    },
+    {
+      "-country": "SRB",
+      "-length": "62"
+    },
+    {
+      "-country": "KOS",
+      "-length": "159"
+    },
+    {
+      "-country": "BG",
+      "-length": "148"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Macedonia-Skopje",
+    "-is_country_cap": "yes",
+    "-country": "MK",
+    "name": "Skopje",
+    "longitude": "21.4",
+    "latitude": "42",
+    "population": {
+      "-year": "02",
+      "#text": "506926"
+    }
+  }
+}
+{
+  "-car_code": "SRB",
+  "-area": "77474",
+  "-capital": "city-Belgrade-SRB-SRB",
+  "-memberships": "org-BSEC org-CE org-CEI org-EBRD org-FAO org-G-9 org-IAEA org-IBRD org-ICAO org-ICC org-ICRM org-IDA org-IFC org-IFRCS org-ILO org-IMF org-IMO org-Interpol org-IOC org-IOM org-ISO org-ITU org-OSCE org-PCA org-PFP org-UN org-UNCTAD org-UNESCO org-UNHCR org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO",
+  "name": "Serbia",
+  "population": "7379339",
+  "population_growth": "-0.47",
+  "infant_mortality": "6.75",
+  "gdp_total": "52180",
+  "gdp_agri": "12.3",
+  "gdp_ind": "24.2",
+  "gdp_serv": "63.5",
+  "inflation": "6.8",
+  "indep_date": "2006-06-05",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "82.9",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "0.9",
+      "#text": "Montenegrin"
+    },
+    {
+      "-percentage": "3.9",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "1.4",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Bosniak"
+    },
+    {
+      "-percentage": "1.1",
+      "#text": "Croat"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "85",
+      "#text": "Christian Orthodox"
+    },
+    {
+      "-percentage": "3.2",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "5.5",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "1.1",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "88.3",
+      "#text": "Serbian"
+    },
+    {
+      "-percentage": "3.8",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Bosnian"
+    },
+    {
+      "-percentage": "1.1",
+      "#text": "Roma"
+    }
+  ],
+  "border": [
+    {
+      "-country": "H",
+      "-length": "151"
+    },
+    {
+      "-country": "RO",
+      "-length": "476"
+    },
+    {
+      "-country": "BG",
+      "-length": "318"
+    },
+    {
+      "-country": "MK",
+      "-length": "62"
+    },
+    {
+      "-country": "HR",
+      "-length": "241"
+    },
+    {
+      "-country": "BIH",
+      "-length": "302"
+    },
+    {
+      "-country": "MNE",
+      "-length": "124"
+    },
+    {
+      "-country": "KOS",
+      "-length": "352"
+    }
+  ],
+  "city": {
+    "-id": "city-Belgrade-SRB-SRB",
+    "-is_country_cap": "yes",
+    "-country": "SRB",
+    "name": "Belgrade",
+    "longitude": "20.4667",
+    "latitude": "44.8",
+    "population": {
+      "-year": "87",
+      "#text": "1407073"
+    },
+    "located_at": [
+      {
+        "-watertype": "river",
+        "-river": "river-Donau"
+      },
+      {
+        "-watertype": "river",
+        "-river": "river-Save"
+      }
+    ]
+  }
+}
+{
+  "-car_code": "MNE",
+  "-area": "14026",
+  "-capital": "city-Podgorica-MNE-MNE",
+  "-memberships": "org-CE org-CEI org-EBRD org-FAO org-IAEA org-IBRD org-ICAO org-ICRM org-IDA org-IFC org-IFRCS org-ILO org-IMF org-IMO org-Interpol org-IOC org-IOM org-ISO org-ITU org-OSCE org-PCA org-PFP org-UN org-UNESCO org-UNHCR org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO",
+  "name": "Montenegro",
+  "population": "672180",
+  "population_growth": "-0.851",
+  "gdp_total": "4515",
+  "inflation": "3.4",
+  "indep_date": "2006-06-03",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "43",
+      "#text": "Montenegrin"
+    },
+    {
+      "-percentage": "32",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Bosniak"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Albanian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "74.2",
+      "#text": "Christian Orthodox"
+    },
+    {
+      "-percentage": "17.7",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "3.5",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "63.6",
+      "#text": "Serbian"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Montenegrin"
+    },
+    {
+      "-percentage": "5.5",
+      "#text": "Bosnian"
+    },
+    {
+      "-percentage": "5.3",
+      "#text": "Albanian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "HR",
+      "-length": "25"
+    },
+    {
+      "-country": "SRB",
+      "-length": "124"
+    },
+    {
+      "-country": "BIH",
+      "-length": "225"
+    },
+    {
+      "-country": "AL",
+      "-length": "172"
+    },
+    {
+      "-country": "KOS",
+      "-length": "79"
+    }
+  ],
+  "city": {
+    "-id": "city-Podgorica-MNE-MNE",
+    "-is_country_cap": "yes",
+    "-country": "MNE",
+    "name": "Podgorica",
+    "longitude": "19.16",
+    "latitude": "42.26",
+    "population": {
+      "-year": "03",
+      "#text": "169132"
+    }
+  }
+}
+{
+  "-car_code": "KOS",
+  "-area": "10887",
+  "-capital": "city-Pristina-KOS-KOS",
+  "name": "Kosovo",
+  "population": "1804838",
+  "gdp_total": "3237",
+  "gdp_agri": "20",
+  "gdp_ind": "20",
+  "gdp_serv": "60",
+  "inflation": "5.3",
+  "indep_date": "2008-02-17",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "88",
+      "#text": "Albanian"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Serb"
+    }
+  ],
+  "border": [
+    {
+      "-country": "MNE",
+      "-length": "79"
+    },
+    {
+      "-country": "SRB",
+      "-length": "352"
+    },
+    {
+      "-country": "MK",
+      "-length": "159"
+    },
+    {
+      "-country": "AL",
+      "-length": "112"
+    }
+  ],
+  "city": {
+    "-id": "city-Pristina-KOS-KOS",
+    "-is_country_cap": "yes",
+    "-country": "KOS",
+    "name": "Pristina",
+    "longitude": "21.10",
+    "latitude": "42.40"
+  }
+}
+{
+  "-car_code": "AND",
+  "-area": "450",
+  "-capital": "cty-cid-cia-Andorra-Andorra-la-Vella",
+  "-memberships": "org-CE org-ECE org-Interpol org-IFRCS org-IOC org-ITU org-UN org-UNESCO org-WIPO",
+  "name": "Andorra",
+  "population": "72766",
+  "population_growth": "2.96",
+  "infant_mortality": "2.2",
+  "gdp_total": "1000",
+  "government": "parliamentary democracy that retains as its heads of state a coprincipality",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "6",
+      "#text": "French"
+    },
+    {
+      "-percentage": "61",
+      "#text": "Spanish"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Andorran"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "F",
+      "-length": "60"
+    },
+    {
+      "-country": "E",
+      "-length": "65"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Andorra-Andorra-la-Vella",
+    "-is_country_cap": "yes",
+    "-country": "AND",
+    "name": "Andorra la Vella",
+    "longitude": "1.3",
+    "latitude": "42.3",
+    "population": {
+      "-year": "87",
+      "#text": "15600"
+    }
+  }
+}
+{
+  "-car_code": "F",
+  "-area": "547030",
+  "-capital": "cty-France-Paris",
+  "-memberships": "org-AfDB org-ACCT org-AG org-AsDB org-BIS org-CDB org-BDEAC org-CE org-CCC org-ESCAP org-ECA org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-FZ org-G-5 org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-SPC org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNIFIL org-UNIKOM org-MINURSO org-UNMIH org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNRWA org-UNTSO org-UNU org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "France",
+  "population": "58317450",
+  "population_growth": "0.34",
+  "infant_mortality": "5.3",
+  "gdp_total": "1173000",
+  "gdp_agri": "2.4",
+  "gdp_ind": "26.5",
+  "gdp_serv": "71.1",
+  "inflation": "1.7",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "French"
+  },
+  "border": [
+    {
+      "-country": "AND",
+      "-length": "60"
+    },
+    {
+      "-country": "E",
+      "-length": "623"
+    },
+    {
+      "-country": "D",
+      "-length": "451"
+    },
+    {
+      "-country": "I",
+      "-length": "488"
+    },
+    {
+      "-country": "CH",
+      "-length": "573"
+    },
+    {
+      "-country": "B",
+      "-length": "620"
+    },
+    {
+      "-country": "L",
+      "-length": "73"
+    },
+    {
+      "-country": "MC",
+      "-length": "4.4"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-France-2",
+      "-capital": "cty-France-Strasbourg",
+      "-country": "F",
+      "name": "Alsace",
+      "area": "8280",
+      "population": "1624000",
+      "city": [
+        {
+          "-id": "cty-France-Strasbourg",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-2",
+          "name": "Strasbourg",
+          "longitude": "7.76667",
+          "latitude": "48.5833",
+          "population": {
+            "-year": "90",
+            "#text": "252338"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-32",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-2",
+          "name": "Mulhouse",
+          "population": {
+            "-year": "90",
+            "#text": "108357"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-5",
+      "-capital": "cty-cid-cia-France-9",
+      "-country": "F",
+      "name": "Aquitaine",
+      "area": "41309",
+      "population": "2796000",
+      "city": {
+        "-id": "cty-cid-cia-France-9",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-5",
+        "name": "Bordeaux",
+        "longitude": "-0.4",
+        "latitude": "44.51",
+        "population": {
+          "-year": "90",
+          "#text": "210336"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Garonne"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-11",
+      "-capital": "cty-cid-cia-France-22",
+      "-country": "F",
+      "name": "Auvergne",
+      "area": "26013",
+      "population": "1321000",
+      "city": {
+        "-id": "cty-cid-cia-France-22",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-11",
+        "name": "Clermont Ferrand",
+        "longitude": "3.4",
+        "latitude": "45.5",
+        "population": {
+          "-year": "90",
+          "#text": "136181"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-16",
+      "-capital": "cty-cid-cia-France-31",
+      "-country": "F",
+      "name": "Basse Normandie",
+      "area": "17589",
+      "population": "1391000",
+      "city": {
+        "-id": "cty-cid-cia-France-31",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-16",
+        "name": "Caen",
+        "longitude": "-0.2",
+        "latitude": "49.3",
+        "population": {
+          "-year": "90",
+          "#text": "112846"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-20",
+      "-capital": "cty-cid-cia-France-12",
+      "-country": "F",
+      "name": "Bretagne",
+      "area": "27209",
+      "population": "2796000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-12",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-20",
+          "name": "Rennes",
+          "longitude": "-1.4",
+          "latitude": "48.1",
+          "population": {
+            "-year": "90",
+            "#text": "197536"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-18",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-20",
+          "name": "Brest",
+          "population": {
+            "-year": "90",
+            "#text": "147956"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-25",
+      "-capital": "cty-cid-cia-France-19",
+      "-country": "F",
+      "name": "Bourgogne",
+      "area": "31582",
+      "population": "1609000",
+      "city": {
+        "-id": "cty-cid-cia-France-19",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-25",
+        "name": "Dijon",
+        "longitude": "5.2",
+        "latitude": "47.2",
+        "population": {
+          "-year": "90",
+          "#text": "146703"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-30",
+      "-capital": "cty-cid-cia-France-34",
+      "-country": "F",
+      "name": "Centre",
+      "area": "39151",
+      "population": "2371000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-34",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-30",
+          "name": "Orleans",
+          "longitude": "1.5",
+          "latitude": "47.5",
+          "population": {
+            "-year": "90",
+            "#text": "105111"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Loire"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-25",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-30",
+          "name": "Tours",
+          "population": {
+            "-year": "90",
+            "#text": "129509"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Loire"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-37",
+      "-capital": "cty-cid-cia-France-Chalons",
+      "-country": "F",
+      "name": "Champagne Ardenne",
+      "area": "25606",
+      "population": "1348000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-14",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-37",
+          "name": "Reims",
+          "longitude": "4.5",
+          "latitude": "49.1",
+          "population": {
+            "-year": "90",
+            "#text": "180620"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-Chalons",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-37",
+          "name": "Chalons en Champagne",
+          "longitude": "4.4",
+          "latitude": "49.0",
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Marne"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-42",
+      "-capital": "cty-cid-cia-France-Ajaccio",
+      "-country": "F",
+      "name": "Corse",
+      "area": "8680",
+      "population": "250000",
+      "city": {
+        "-id": "cty-cid-cia-France-Ajaccio",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-42",
+        "name": "Ajaccio",
+        "longitude": "8.4",
+        "latitude": "41.4",
+        "population": {
+          "-year": "87",
+          "#text": "53500"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        },
+        "located_on": { "-island": "island-Corse" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-45",
+      "-capital": "cty-cid-cia-France-30",
+      "-country": "F",
+      "name": "Franche Comte",
+      "area": "16202",
+      "population": "1097000",
+      "city": {
+        "-id": "cty-cid-cia-France-30",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-45",
+        "name": "Besancon",
+        "longitude": "6.2",
+        "latitude": "47.1",
+        "population": {
+          "-year": "90",
+          "#text": "113828"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Doubs"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-50",
+      "-capital": "cty-cid-cia-France-35",
+      "-country": "F",
+      "name": "Haute Normandie",
+      "area": "12318",
+      "population": "1737000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-35",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-50",
+          "name": "Rouen",
+          "longitude": "1.5",
+          "latitude": "49.3",
+          "population": {
+            "-year": "90",
+            "#text": "102723"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Seine"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-13",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-50",
+          "name": "Le Havre",
+          "population": {
+            "-year": "90",
+            "#text": "195854"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Channel"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Seine"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-53",
+      "-capital": "cty-France-Paris",
+      "-country": "F",
+      "name": "Ile de France",
+      "area": "12011",
+      "population": "10660000",
+      "city": [
+        {
+          "-id": "cty-France-Paris",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-53",
+          "name": "Paris",
+          "longitude": "2.48333",
+          "latitude": "48.8167",
+          "population": {
+            "-year": "90",
+            "#text": "2152423"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Seine"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-36",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-53",
+          "name": "Boulogne Billancourt",
+          "population": {
+            "-year": "90",
+            "#text": "101743"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Seine"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-62",
+      "-capital": "cty-cid-cia-France-10",
+      "-country": "F",
+      "name": "Languedoc Roussillon",
+      "area": "27376",
+      "population": "2115000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-10",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-62",
+          "name": "Montpellier",
+          "longitude": "3.5",
+          "latitude": "43.3",
+          "population": {
+            "-year": "90",
+            "#text": "207996"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-26",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-62",
+          "name": "Nimes",
+          "population": {
+            "-year": "90",
+            "#text": "128471"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-33",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-62",
+          "name": "Perpignan",
+          "population": {
+            "-year": "90",
+            "#text": "105983"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-68",
+      "-capital": "cty-cid-cia-France-23",
+      "-country": "F",
+      "name": "Limousin",
+      "area": "16942",
+      "population": "723000",
+      "city": {
+        "-id": "cty-cid-cia-France-23",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-68",
+        "name": "Limoges",
+        "longitude": "1.2",
+        "latitude": "45.5",
+        "population": {
+          "-year": "90",
+          "#text": "133464"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-72",
+      "-capital": "cty-cid-cia-France-28",
+      "-country": "F",
+      "name": "Lorraine",
+      "area": "23547",
+      "population": "2306000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-28",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-72",
+          "name": "Metz",
+          "longitude": "6.2",
+          "latitude": "49.1",
+          "population": {
+            "-year": "90",
+            "#text": "119594"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mosel"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-37",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-72",
+          "name": "Nancy",
+          "longitude": "6.1",
+          "latitude": "48.4",
+          "population": {
+            "-year": "90",
+            "#text": "99351"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mosel"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-77",
+      "-capital": "cty-cid-cia-France-5",
+      "-country": "F",
+      "name": "Midi Pyrenees",
+      "area": "45349",
+      "population": "2431000",
+      "city": {
+        "-id": "cty-cid-cia-France-5",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-77",
+        "name": "Toulouse",
+        "longitude": "1.2",
+        "latitude": "43.4",
+        "population": {
+          "-year": "90",
+          "#text": "358688"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Garonne"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-86",
+      "-capital": "cty-cid-cia-France-15",
+      "-country": "F",
+      "name": "Nord Pas de Calais",
+      "area": "12413",
+      "population": "3965000",
+      "city": {
+        "-id": "cty-cid-cia-France-15",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-86",
+        "name": "Lille",
+        "longitude": "3.3",
+        "latitude": "50.37",
+        "population": {
+          "-year": "90",
+          "#text": "172142"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-89",
+      "-capital": "cty-France-Nantes",
+      "-country": "F",
+      "name": "Pays de la Loire",
+      "area": "32082",
+      "population": "3059000",
+      "city": [
+        {
+          "-id": "cty-France-Nantes",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-89",
+          "name": "Nantes",
+          "longitude": "-1.56667",
+          "latitude": "47.25",
+          "population": {
+            "-year": "90",
+            "#text": "244995"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Loire"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-21",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-89",
+          "name": "Angers",
+          "population": {
+            "-year": "90",
+            "#text": "141404"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-20",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-89",
+          "name": "Le Mans",
+          "population": {
+            "-year": "90",
+            "#text": "145502"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-95",
+      "-capital": "cty-cid-cia-France-24",
+      "-country": "F",
+      "name": "Picardie",
+      "area": "19399",
+      "population": "1811000",
+      "city": {
+        "-id": "cty-cid-cia-France-24",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-95",
+        "name": "Amiens",
+        "longitude": "2.19",
+        "latitude": "49.55",
+        "population": {
+          "-year": "90",
+          "#text": "131872"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-99",
+      "-capital": "cty-cid-cia-France-Poitiers",
+      "-country": "F",
+      "name": "Poitou Charentes",
+      "area": "25809",
+      "population": "1595000",
+      "city": {
+        "-id": "cty-cid-cia-France-Poitiers",
+        "-is_state_cap": "yes",
+        "-country": "F",
+        "-province": "prov-cid-cia-France-99",
+        "name": "Poitiers",
+        "longitude": "0.2",
+        "latitude": "46.4",
+        "population": {
+          "-year": "87",
+          "#text": "79300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-France-104",
+      "-capital": "cty-cid-cia-France-3",
+      "-country": "F",
+      "name": "Provence Cote dAzur",
+      "area": "31400",
+      "population": "4258000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-France-3",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-104",
+          "name": "Marseille",
+          "longitude": "5.2",
+          "latitude": "43.2",
+          "population": {
+            "-year": "90",
+            "#text": "800550"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-France-Nice",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-104",
+          "name": "Nice",
+          "longitude": "7.26667",
+          "latitude": "43.7",
+          "population": {
+            "-year": "90",
+            "#text": "342439"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-16",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-104",
+          "name": "Toulon",
+          "population": {
+            "-year": "90",
+            "#text": "167619"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-27",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-104",
+          "name": "Aix en Provence",
+          "population": {
+            "-year": "90",
+            "#text": "123842"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-France-111",
+      "-capital": "cty-France-Lyon",
+      "-country": "F",
+      "name": "Rhone Alpes",
+      "area": "43698",
+      "population": "5351000",
+      "city": [
+        {
+          "-id": "cty-France-Lyon",
+          "-is_state_cap": "yes",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-111",
+          "name": "Lyon",
+          "longitude": "4.78333",
+          "latitude": "45.7",
+          "population": {
+            "-year": "90",
+            "#text": "415487"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Saone"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Rhone"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-France-17",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-111",
+          "name": "Grenoble",
+          "population": {
+            "-year": "90",
+            "#text": "150758"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Isere"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-11",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-111",
+          "name": "Saint Etienne",
+          "population": {
+            "-year": "90",
+            "#text": "199396"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-France-29",
+          "-country": "F",
+          "-province": "prov-cid-cia-France-111",
+          "name": "Villeurbanne",
+          "population": {
+            "-year": "90",
+            "#text": "116872"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhone"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "E",
+  "-area": "504750",
+  "-capital": "cty-Spain-Madrid",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CE org-CCC org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAIA org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNHCR org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Spain",
+  "population": "39181114",
+  "population_growth": "0.16",
+  "infant_mortality": "6.3",
+  "gdp_total": "565000",
+  "gdp_agri": "3.6",
+  "gdp_ind": "33.6",
+  "gdp_serv": "62.8",
+  "inflation": "4.3",
+  "indep_date": "1492-01-01",
+  "government": "parliamentary monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Mediterranean Nordic"
+  },
+  "religions": {
+    "-percentage": "99",
+    "#text": "Roman Catholic"
+  },
+  "languages": [
+    {
+      "-percentage": "17",
+      "#text": "Catalan"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Galician"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Basque"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AND",
+      "-length": "65"
+    },
+    {
+      "-country": "F",
+      "-length": "623"
+    },
+    {
+      "-country": "GBZ",
+      "-length": "1.2"
+    },
+    {
+      "-country": "P",
+      "-length": "1214"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Spain-2",
+      "-capital": "cty-cid-cia-Spain-5",
+      "-country": "E",
+      "name": "Andalusia",
+      "area": "87600",
+      "population": "7053043",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-5",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Sevilla",
+          "population": {
+            "-year": "94",
+            "#text": "714148"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Guadalquivir"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-34",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Almeria",
+          "population": {
+            "-year": "94",
+            "#text": "167361"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-39",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Cadiz",
+          "longitude": "-6.2",
+          "latitude": "36.3",
+          "population": {
+            "-year": "94",
+            "#text": "155438"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-13",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Cordoba",
+          "population": {
+            "-year": "94",
+            "#text": "315948"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Guadalquivir"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-16",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Granada",
+          "population": {
+            "-year": "94",
+            "#text": "271180"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-41",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Huelva",
+          "population": {
+            "-year": "94",
+            "#text": "145049"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-7",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Malaga",
+          "population": {
+            "-year": "94",
+            "#text": "531443"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-27",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Jerez de la Frontera",
+          "population": {
+            "-year": "94",
+            "#text": "190390"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-52",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Jaen",
+          "population": {
+            "-year": "94",
+            "#text": "112772"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-54",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-2",
+          "name": "Algeciras",
+          "population": {
+            "-year": "94",
+            "#text": "103787"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-3",
+      "-capital": "cty-cid-cia-Spain-6",
+      "-country": "E",
+      "name": "Aragon",
+      "area": "47720",
+      "population": "1183576",
+      "city": {
+        "-id": "cty-cid-cia-Spain-6",
+        "-is_state_cap": "yes",
+        "-country": "E",
+        "-province": "prov-cid-cia-Spain-3",
+        "name": "Zaragoza",
+        "population": {
+          "-year": "94",
+          "#text": "606620"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ebro"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Spain-4",
+      "-capital": "cty-cid-cia-Spain-23",
+      "-country": "E",
+      "name": "Asturias",
+      "area": "10604",
+      "population": "1083388",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-23",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-4",
+          "name": "Oviedo",
+          "population": {
+            "-year": "94",
+            "#text": "201712"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-17",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-4",
+          "name": "Gijon",
+          "population": {
+            "-year": "94",
+            "#text": "269644"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-5",
+      "-capital": "cty-cid-cia-Spain-12",
+      "-country": "E",
+      "name": "Balearic Islands",
+      "area": "4992",
+      "population": "736865",
+      "city": {
+        "-id": "cty-cid-cia-Spain-12",
+        "-is_state_cap": "yes",
+        "-country": "E",
+        "-province": "prov-cid-cia-Spain-5",
+        "name": "Palma de Mallorca",
+        "population": {
+          "-year": "94",
+          "#text": "322008"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        },
+        "located_on": { "-island": "island-Mallorca" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Spain-6",
+      "-capital": "cty-cid-cia-Spain-21",
+      "-country": "E",
+      "name": "Basque Country",
+      "area": "7235",
+      "population": "2075561",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-21",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-6",
+          "name": "Vitoria Gasteiz",
+          "population": {
+            "-year": "94",
+            "#text": "214148"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-8",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-6",
+          "name": "Bilbao",
+          "population": {
+            "-year": "94",
+            "#text": "371876"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-55",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-6",
+          "name": "Baracaldo",
+          "population": {
+            "-year": "94",
+            "#text": "103594"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-32",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-6",
+          "name": "Donostia",
+          "population": {
+            "-year": "94",
+            "#text": "177929"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-7",
+      "-capital": "cty-cid-cia-Spain-22",
+      "-country": "E",
+      "name": "Canary Islands",
+      "area": "7447",
+      "population": "1534897",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-22",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-7",
+          "name": "Santa Cruz de Tenerife",
+          "population": {
+            "-year": "94",
+            "#text": "203929"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          },
+          "located_on": { "-island": "island-Teneriffa" }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-9",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-7",
+          "name": "Las Palmas de Gran Canaria",
+          "population": {
+            "-year": "94",
+            "#text": "371787"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          },
+          "located_on": { "-island": "island-Gran_Canaria" }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-48",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-7",
+          "name": "La Laguna",
+          "population": {
+            "-year": "94",
+            "#text": "125183"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          },
+          "located_on": { "-island": "island-Teneriffa" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-8",
+      "-capital": "cty-cid-cia-Spain-25",
+      "-country": "E",
+      "name": "Cantabria",
+      "area": "5321",
+      "population": "526090",
+      "city": {
+        "-id": "cty-cid-cia-Spain-25",
+        "-is_state_cap": "yes",
+        "-country": "E",
+        "-province": "prov-cid-cia-Spain-8",
+        "name": "Santander",
+        "population": {
+          "-year": "94",
+          "#text": "194822"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Spain-9",
+      "-capital": "cty-cid-cia-Spain-11",
+      "-country": "E",
+      "name": "Castile and Leon",
+      "area": "94224",
+      "population": "2504371",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-11",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-9",
+          "name": "Valladolid",
+          "population": {
+            "-year": "94",
+            "#text": "336917"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Douro"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-35",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-9",
+          "name": "Burgos",
+          "population": {
+            "-year": "94",
+            "#text": "166251"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-40",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-9",
+          "name": "Leon",
+          "population": {
+            "-year": "94",
+            "#text": "147311"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-33",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-9",
+          "name": "Salamanca",
+          "population": {
+            "-year": "94",
+            "#text": "167382"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-10",
+      "-capital": "cty-cid-cia-Spain-Toledo",
+      "-country": "E",
+      "name": "Castile La Mancha",
+      "area": "79462",
+      "population": "1656179",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-Toledo",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-10",
+          "name": "Toledo"
+        },
+        {
+          "-id": "cty-cid-cia-Spain-44",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-10",
+          "name": "Albacete",
+          "population": {
+            "-year": "94",
+            "#text": "141179"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-11",
+      "-capital": "cty-Spain-Barcelona",
+      "-country": "E",
+      "name": "Catalonia",
+      "area": "32113",
+      "population": "6090107",
+      "city": [
+        {
+          "-id": "cty-Spain-Barcelona",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Barcelona",
+          "longitude": "2.15",
+          "latitude": "41.4",
+          "population": {
+            "-year": "94",
+            "#text": "1630867"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-51",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Lleida",
+          "population": {
+            "-year": "94",
+            "#text": "114234"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-50",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Tarragona",
+          "population": {
+            "-year": "94",
+            "#text": "114630"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-18",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Hospitalet de Llobregat",
+          "population": {
+            "-year": "94",
+            "#text": "266242"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-20",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Badalona",
+          "population": {
+            "-year": "94",
+            "#text": "219340"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-28",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Sabadell",
+          "population": {
+            "-year": "94",
+            "#text": "189006"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-37",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Terrassa",
+          "population": {
+            "-year": "94",
+            "#text": "161428"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-46",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Santa Coloma de Gramanet",
+          "population": {
+            "-year": "94",
+            "#text": "131764"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-56",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-11",
+          "name": "Mataro",
+          "population": {
+            "-year": "94",
+            "#text": "102117"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-12",
+      "-capital": "cty-cid-cia-Spain-Merida",
+      "-country": "E",
+      "name": "Estremadura",
+      "area": "41635",
+      "population": "1050590",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-47",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-12",
+          "name": "Badajoz",
+          "population": {
+            "-year": "94",
+            "#text": "130153"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Guadiana"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-Merida",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-12",
+          "name": "Merida"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-13",
+      "-capital": "cty-cid-cia-Spain-Santiago-de-Compostella",
+      "-country": "E",
+      "name": "Galicia",
+      "area": "29574",
+      "population": "2720761",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-19",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-13",
+          "name": "La Coruna",
+          "population": {
+            "-year": "94",
+            "#text": "255087"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-14",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-13",
+          "name": "Vigo",
+          "population": {
+            "-year": "94",
+            "#text": "288573"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-53",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-13",
+          "name": "Orense",
+          "population": {
+            "-year": "94",
+            "#text": "108547"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-Santiago-de-Compostella",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-13",
+          "name": "Santiago de Compostella"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-14",
+      "-capital": "cty-Spain-Madrid",
+      "-country": "E",
+      "name": "Madrid",
+      "area": "8028",
+      "population": "5034548",
+      "city": [
+        {
+          "-id": "cty-Spain-Madrid",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Madrid",
+          "longitude": "-3.68333",
+          "latitude": "40.4167",
+          "population": {
+            "-year": "94",
+            "#text": "3041101"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-24",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Mostoles",
+          "population": {
+            "-year": "94",
+            "#text": "199141"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-31",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Leganes",
+          "population": {
+            "-year": "94",
+            "#text": "178162"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-36",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Alcala de Henares",
+          "population": {
+            "-year": "94",
+            "#text": "166250"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-38",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Fuenlabrada",
+          "population": {
+            "-year": "94",
+            "#text": "158212"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-42",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Getafe",
+          "population": {
+            "-year": "94",
+            "#text": "144368"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-43",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-14",
+          "name": "Alcorcon",
+          "population": {
+            "-year": "94",
+            "#text": "142165"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-15",
+      "-capital": "cty-cid-cia-Spain-10",
+      "-country": "E",
+      "name": "Murcia",
+      "area": "11314",
+      "population": "1070401",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Spain-10",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-15",
+          "name": "Murcia",
+          "population": {
+            "-year": "94",
+            "#text": "341531"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-30",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-15",
+          "name": "Cartagena",
+          "population": {
+            "-year": "94",
+            "#text": "179659"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Spain-16",
+      "-capital": "cty-cid-cia-Spain-29",
+      "-country": "E",
+      "name": "Navarre",
+      "area": "10391",
+      "population": "523614",
+      "city": {
+        "-id": "cty-cid-cia-Spain-29",
+        "-is_state_cap": "yes",
+        "-country": "E",
+        "-province": "prov-cid-cia-Spain-16",
+        "name": "Pamplona",
+        "population": {
+          "-year": "94",
+          "#text": "182465"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Spain-17",
+      "-capital": "cty-cid-cia-Spain-49",
+      "-country": "E",
+      "name": "Rioja",
+      "area": "5045",
+      "population": "263437",
+      "city": {
+        "-id": "cty-cid-cia-Spain-49",
+        "-is_state_cap": "yes",
+        "-country": "E",
+        "-province": "prov-cid-cia-Spain-17",
+        "name": "Logrono",
+        "population": {
+          "-year": "94",
+          "#text": "124823"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ebro"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Spain-18",
+      "-capital": "cty-Spain-Valencia",
+      "-country": "E",
+      "name": "Valencia",
+      "area": "23255",
+      "population": "3909047",
+      "city": [
+        {
+          "-id": "cty-Spain-Valencia",
+          "-is_state_cap": "yes",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-18",
+          "name": "Valencia",
+          "longitude": "-0.383333",
+          "latitude": "39.4667",
+          "population": {
+            "-year": "94",
+            "#text": "764293"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-15",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-18",
+          "name": "Alicante",
+          "population": {
+            "-year": "94",
+            "#text": "274964"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-26",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-18",
+          "name": "Elx",
+          "population": {
+            "-year": "94",
+            "#text": "191305"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Spain-45",
+          "-country": "E",
+          "-province": "prov-cid-cia-Spain-18",
+          "name": "Castellon de la Plana",
+          "population": {
+            "-year": "94",
+            "#text": "139094"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "A",
+  "-area": "83850",
+  "-capital": "cty-Austria-Vienna",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CEI org-CE org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-9 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-EN org-NSG org-OECD org-OSCE org-OAS org-PFP org-PCA org-UN org-UNAMIR org-UNDOF org-UNESCO org-UNFICYP org-UNIDO org-UNIKOM org-MINURSO org-UNMIH org-UNMOT org-UNOMIG org-UNHCR org-UNTSO org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Austria",
+  "population": "8023244",
+  "population_growth": "0.41",
+  "infant_mortality": "6.2",
+  "gdp_total": "152000",
+  "gdp_agri": "2",
+  "gdp_ind": "34",
+  "gdp_serv": "64",
+  "inflation": "2.3",
+  "indep_date": "1918-11-12",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "99.4",
+      "#text": "German"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Slovene"
+    },
+    {
+      "-percentage": "0.3",
+      "#text": "Croat"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "85",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "German"
+  },
+  "border": [
+    {
+      "-country": "CZ",
+      "-length": "362"
+    },
+    {
+      "-country": "D",
+      "-length": "784"
+    },
+    {
+      "-country": "H",
+      "-length": "366"
+    },
+    {
+      "-country": "I",
+      "-length": "430"
+    },
+    {
+      "-country": "FL",
+      "-length": "37"
+    },
+    {
+      "-country": "SK",
+      "-length": "91"
+    },
+    {
+      "-country": "SLO",
+      "-length": "324"
+    },
+    {
+      "-country": "CH",
+      "-length": "164"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Austria-2",
+      "-capital": "cty-cid-cia-Austria-Eisenstadt",
+      "-country": "A",
+      "name": "Burgenland",
+      "area": "3965",
+      "population": "273000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-Eisenstadt",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-2",
+        "name": "Eisenstadt",
+        "longitude": "16",
+        "latitude": "48",
+        "population": {
+          "-year": "87",
+          "#text": "10102"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-3",
+      "-capital": "cty-cid-cia-Austria-Klagenfurt",
+      "-country": "A",
+      "name": "Karnten",
+      "area": "9533",
+      "population": "559000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-Klagenfurt",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-3",
+        "name": "Klagenfurt",
+        "longitude": "14.21",
+        "latitude": "46.38",
+        "population": {
+          "-year": "87",
+          "#text": "87321"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-4",
+      "-capital": "cty-cid-cia-Austria-Bregenz",
+      "-country": "A",
+      "name": "Vorarlberg",
+      "area": "2601",
+      "population": "341000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-Bregenz",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-4",
+        "name": "Bregenz",
+        "longitude": "9.45",
+        "latitude": "47.3",
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Bodensee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-5",
+      "-capital": "cty-Austria-Vienna",
+      "-country": "A",
+      "name": "Vienna-Wien",
+      "area": "415",
+      "population": "1583000",
+      "city": {
+        "-id": "cty-Austria-Vienna",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-5",
+        "name": "Vienna",
+        "longitude": "16.3667",
+        "latitude": "48.2",
+        "population": {
+          "-year": "94",
+          "#text": "1583000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Donau"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-6",
+      "-capital": "cty-cid-cia-Austria-4",
+      "-country": "A",
+      "name": "Oberosterreich",
+      "area": "11979",
+      "population": "1373000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-4",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-6",
+        "name": "Linz",
+        "longitude": "14.18",
+        "latitude": "48.18",
+        "population": {
+          "-year": "94",
+          "#text": "203000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Donau"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-7",
+      "-capital": "cty-cid-cia-Austria-6",
+      "-country": "A",
+      "name": "Tirol",
+      "area": "12647",
+      "population": "649000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-6",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-7",
+        "name": "Innsbruck",
+        "longitude": "11.22",
+        "latitude": "47.17",
+        "population": {
+          "-year": "94",
+          "#text": "118000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Inn"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-8",
+      "-capital": "cty-cid-cia-Austria-3",
+      "-country": "A",
+      "name": "Steiermark",
+      "area": "16386",
+      "population": "1203000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-3",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-8",
+        "name": "Graz",
+        "longitude": "15.26",
+        "latitude": "47.4",
+        "population": {
+          "-year": "94",
+          "#text": "238000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Mur"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-9",
+      "-capital": "cty-cid-cia-Austria-5",
+      "-country": "A",
+      "name": "Salzburg",
+      "area": "7154",
+      "population": "501000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-5",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-9",
+        "name": "Salzburg",
+        "longitude": "13.2",
+        "latitude": "47.49",
+        "population": {
+          "-year": "94",
+          "#text": "144000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Salzach"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Austria-10",
+      "-capital": "cty-cid-cia-Austria-St-Polten",
+      "-country": "A",
+      "name": "Niederosterreich",
+      "area": "19170",
+      "population": "1507000",
+      "city": {
+        "-id": "cty-cid-cia-Austria-St-Polten",
+        "-is_state_cap": "yes",
+        "-country": "A",
+        "-province": "prov-cid-cia-Austria-10",
+        "name": "St. Polten",
+        "longitude": "15.38",
+        "latitude": "48.13",
+        "population": {
+          "-year": "87",
+          "#text": "51102"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "CZ",
+  "-area": "78703",
+  "-capital": "cty-cid-cia-Czech-Republic-2",
+  "-memberships": "org-BIS org-CEI org-CE org-CCC org-ECE org-EBRD org-CERN org-EU org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-ANC org-NSG org-OECD org-OSCE org-PFP org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNOMIG org-UNOMIL org-UNPROFOR org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Czech Republic",
+  "population": "10321120",
+  "population_growth": "-0.03",
+  "infant_mortality": "8.4",
+  "gdp_total": "106200",
+  "gdp_agri": "5.8",
+  "gdp_ind": "40.7",
+  "gdp_serv": "53.5",
+  "inflation": "9.1",
+  "indep_date": "1993-01-01",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.5",
+      "#text": "German"
+    },
+    {
+      "-percentage": "0.6",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "0.3",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "94.4",
+      "#text": "Czech"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Slovak"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "39.2",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "4.6",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "A",
+      "-length": "362"
+    },
+    {
+      "-country": "D",
+      "-length": "646"
+    },
+    {
+      "-country": "SK",
+      "-length": "215"
+    },
+    {
+      "-country": "PL",
+      "-length": "658"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Czech-Republic-2",
+      "-capital": "cty-cid-cia-Czech-Republic-8",
+      "-country": "CZ",
+      "name": "Jihocesky",
+      "area": "11345",
+      "population": "702000",
+      "city": {
+        "-id": "cty-cid-cia-Czech-Republic-8",
+        "-is_state_cap": "yes",
+        "-country": "CZ",
+        "-province": "prov-cid-cia-Czech-Republic-2",
+        "name": "Ceske Budejovice",
+        "population": {
+          "-year": "90",
+          "#text": "174000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-3",
+      "-capital": "cty-cid-cia-Czech-Republic-3",
+      "-country": "CZ",
+      "name": "Jihomoravsky",
+      "area": "15028",
+      "population": "2059000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Czech-Republic-3",
+          "-is_state_cap": "yes",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-3",
+          "name": "Brno",
+          "population": {
+            "-year": "90",
+            "#text": "393000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Czech-Republic-6",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-3",
+          "name": "Zlin",
+          "population": {
+            "-year": "90",
+            "#text": "198000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-4",
+      "-capital": "cty-cid-cia-Czech-Republic-Usti-nad-Labem",
+      "-country": "CZ",
+      "name": "Severocesky",
+      "area": "7819",
+      "population": "1190000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Czech-Republic-11",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-4",
+          "name": "Liberec",
+          "population": {
+            "-year": "90",
+            "#text": "160000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Czech-Republic-Usti-nad-Labem",
+          "-is_state_cap": "yes",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-4",
+          "name": "Usti nad Labem"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-5",
+      "-capital": "cty-cid-cia-Czech-Republic-4",
+      "-country": "CZ",
+      "name": "Severomoravsky",
+      "area": "11067",
+      "population": "1976000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Czech-Republic-4",
+          "-is_state_cap": "yes",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-5",
+          "name": "Ostrava",
+          "population": {
+            "-year": "90",
+            "#text": "332000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Oder"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Czech-Republic-5",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-5",
+          "name": "Olomouc",
+          "population": {
+            "-year": "90",
+            "#text": "225000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-March"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-6",
+      "-capital": "cty-cid-cia-Czech-Republic-2",
+      "-country": "CZ",
+      "name": "Praha",
+      "area": "11490",
+      "population": "2329000",
+      "city": {
+        "-id": "cty-cid-cia-Czech-Republic-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "CZ",
+        "-province": "prov-cid-cia-Czech-Republic-6",
+        "name": "Prague",
+        "longitude": "14.4167",
+        "latitude": "50.0833",
+        "population": {
+          "-year": "90",
+          "#text": "1215000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-7",
+      "-capital": "cty-cid-cia-Czech-Republic-9",
+      "-country": "CZ",
+      "name": "Vychodocesky",
+      "area": "11240",
+      "population": "1240000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Czech-Republic-9",
+          "-is_state_cap": "yes",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-7",
+          "name": "Hradec Kralove",
+          "population": {
+            "-year": "90",
+            "#text": "164000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Czech-Republic-10",
+          "-country": "CZ",
+          "-province": "prov-cid-cia-Czech-Republic-7",
+          "name": "Pardubice",
+          "population": {
+            "-year": "90",
+            "#text": "163000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Czech-Republic-8",
+      "-capital": "cty-cid-cia-Czech-Republic-7",
+      "-country": "CZ",
+      "name": "Zapadocesky",
+      "area": "10875",
+      "population": "869000",
+      "city": {
+        "-id": "cty-cid-cia-Czech-Republic-7",
+        "-is_state_cap": "yes",
+        "-country": "CZ",
+        "-province": "prov-cid-cia-Czech-Republic-8",
+        "name": "Plzen",
+        "population": {
+          "-year": "90",
+          "#text": "175000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "D",
+  "-area": "356910",
+  "-capital": "cty-Germany-Berlin",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CDB org-BDEAC org-CE org-CBSS org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-5 org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-MINURSO org-UNOMIG org-UNHCR org-UNPROFOR org-UPU org-WEU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Germany",
+  "population": "83536115",
+  "population_growth": "0.67",
+  "infant_mortality": "6",
+  "gdp_total": "1452200",
+  "gdp_agri": "1",
+  "gdp_ind": "34.2",
+  "gdp_serv": "64.8",
+  "indep_date": "1871-01-18",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.4",
+      "#text": "Greek"
+    },
+    {
+      "-percentage": "2.3",
+      "#text": "Turkish"
+    },
+    {
+      "-percentage": "95.1",
+      "#text": "German"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Italian"
+    },
+    {
+      "-percentage": "0.4",
+      "#text": "Polish"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "37",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "45",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "German"
+  },
+  "border": [
+    {
+      "-country": "F",
+      "-length": "451"
+    },
+    {
+      "-country": "A",
+      "-length": "784"
+    },
+    {
+      "-country": "CZ",
+      "-length": "646"
+    },
+    {
+      "-country": "CH",
+      "-length": "334"
+    },
+    {
+      "-country": "PL",
+      "-length": "456"
+    },
+    {
+      "-country": "B",
+      "-length": "167"
+    },
+    {
+      "-country": "L",
+      "-length": "138"
+    },
+    {
+      "-country": "NL",
+      "-length": "577"
+    },
+    {
+      "-country": "DK",
+      "-length": "68"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Germany-2",
+      "-capital": "cty-cid-cia-Germany-9",
+      "-country": "D",
+      "name": "Baden Wurttemberg",
+      "area": "35742",
+      "population": "10272069",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-9",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Stuttgart",
+          "longitude": "9.1",
+          "latitude": "48.7",
+          "population": {
+            "-year": "95",
+            "#text": "588482"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Neckar"
+          }
+        },
+        {
+          "-id": "cty-Germany-Mannheim",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Mannheim",
+          "longitude": "8.46667",
+          "latitude": "49.5667",
+          "population": {
+            "-year": "95",
+            "#text": "316223"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Rhein"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Neckar"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Germany-24",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Karlsruhe",
+          "population": {
+            "-year": "95",
+            "#text": "277011"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-41",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Freiburg im Breisgau",
+          "population": {
+            "-year": "95",
+            "#text": "198496"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-54",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Heidelberg",
+          "population": {
+            "-year": "95",
+            "#text": "138964"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Neckar"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-66",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Heilbronn",
+          "population": {
+            "-year": "95",
+            "#text": "122253"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Neckar"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-69",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Pforzheim",
+          "population": {
+            "-year": "95",
+            "#text": "117960"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-72",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Ulm",
+          "population": {
+            "-year": "95",
+            "#text": "115123"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Donau"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Iller"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Germany-77",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-2",
+          "name": "Reutlingen",
+          "population": {
+            "-year": "95",
+            "#text": "107782"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-3",
+      "-capital": "cty-Germany-Munich",
+      "-country": "D",
+      "name": "Bayern",
+      "area": "70546",
+      "population": "11921944",
+      "city": [
+        {
+          "-id": "cty-Germany-Munich",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Munich",
+          "longitude": "11.5667",
+          "latitude": "48.15",
+          "population": [
+            {
+              "-year": "87",
+              "#text": "1290079"
+            },
+            {
+              "-year": "95",
+              "#text": "1244676"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Isar"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-14",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Nurnberg",
+          "population": {
+            "-year": "95",
+            "#text": "495845"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-30",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Augsburg",
+          "population": {
+            "-year": "95",
+            "#text": "262110"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Lech"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-58",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Wurzburg",
+          "population": {
+            "-year": "95",
+            "#text": "127946"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Main"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-64",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Regensburg",
+          "population": {
+            "-year": "95",
+            "#text": "125608"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Donau"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-74",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Ingolstadt",
+          "population": {
+            "-year": "95",
+            "#text": "110910"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Donau"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-76",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Furth",
+          "population": {
+            "-year": "95",
+            "#text": "107799"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-85",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-3",
+          "name": "Erlangen",
+          "population": {
+            "-year": "95",
+            "#text": "101450"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-4",
+      "-capital": "cty-Germany-Berlin",
+      "-country": "D",
+      "name": "Berlin",
+      "area": "889",
+      "population": "3472009",
+      "city": {
+        "-id": "cty-Germany-Berlin",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "D",
+        "-province": "prov-cid-cia-Germany-4",
+        "name": "Berlin",
+        "longitude": "13.3",
+        "latitude": "52.45",
+        "population": {
+          "-year": "95",
+          "#text": "3472009"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Germany-5",
+      "-capital": "cty-cid-cia-Germany-55",
+      "-country": "D",
+      "name": "Brandenburg",
+      "area": "29480",
+      "population": "2536747",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-55",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-5",
+          "name": "Potsdam",
+          "longitude": "13.1",
+          "latitude": "52.4",
+          "population": {
+            "-year": "95",
+            "#text": "138268"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-63",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-5",
+          "name": "Cottbus",
+          "population": {
+            "-year": "95",
+            "#text": "125643"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-6",
+      "-capital": "cty-cid-cia-Germany-11",
+      "-country": "D",
+      "name": "Bremen",
+      "area": "404",
+      "population": "680000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-11",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-6",
+          "name": "Bremen",
+          "longitude": "8.5",
+          "latitude": "53.7",
+          "population": {
+            "-year": "95",
+            "#text": "549182"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Weser"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-57",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-6",
+          "name": "Bremerhaven",
+          "population": {
+            "-year": "95",
+            "#text": "130847"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Nordsee"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Weser"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-7",
+      "-capital": "cty-Germany-Hamburg",
+      "-country": "D",
+      "name": "Hamburg",
+      "area": "755",
+      "population": "1705872",
+      "city": {
+        "-id": "cty-Germany-Hamburg",
+        "-is_state_cap": "yes",
+        "-country": "D",
+        "-province": "prov-cid-cia-Germany-7",
+        "name": "Hamburg",
+        "longitude": "9.96667",
+        "latitude": "53.55",
+        "population": {
+          "-year": "95",
+          "#text": "1705872"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Elbe"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Germany-8",
+      "-capital": "cty-cid-cia-Germany-26",
+      "-country": "D",
+      "name": "Hessen",
+      "area": "21115",
+      "population": "5980693",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-26",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-8",
+          "name": "Wiesbaden",
+          "longitude": "8.17",
+          "latitude": "50.07",
+          "population": {
+            "-year": "95",
+            "#text": "266081"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Rhein"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Main"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Germany-6",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-8",
+          "name": "Frankfurt am Main",
+          "population": {
+            "-year": "95",
+            "#text": "652412"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Main"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-40",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-8",
+          "name": "Kassel",
+          "population": {
+            "-year": "95",
+            "#text": "201789"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Fulda"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-53",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-8",
+          "name": "Darmstadt",
+          "population": {
+            "-year": "95",
+            "#text": "139063"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-71",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-8",
+          "name": "Offenbach am Main",
+          "population": {
+            "-year": "95",
+            "#text": "116482"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Main"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-9",
+      "-capital": "cty-cid-cia-Germany-68",
+      "-country": "D",
+      "name": "Mecklenburg Vorpommern",
+      "area": "23170",
+      "population": "1832298",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-68",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-9",
+          "name": "Schwerin",
+          "longitude": "11.5",
+          "latitude": "53.9",
+          "population": {
+            "-year": "95",
+            "#text": "118291"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-35",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-9",
+          "name": "Rostock",
+          "longitude": "12.1",
+          "latitude": "54.1",
+          "population": {
+            "-year": "95",
+            "#text": "232634"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-10",
+      "-capital": "cty-Germany-Hannover",
+      "-country": "D",
+      "name": "Niedersachsen",
+      "area": "47609",
+      "population": "7715363",
+      "city": [
+        {
+          "-id": "cty-Germany-Hannover",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Hannover",
+          "longitude": "9.66667",
+          "latitude": "52.4",
+          "population": {
+            "-year": "95",
+            "#text": "525763"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Leine"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-31",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Braunschweig",
+          "population": {
+            "-year": "95",
+            "#text": "254130"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-47",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Osnabruck",
+          "population": {
+            "-year": "95",
+            "#text": "168050"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-51",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Oldenburg",
+          "population": {
+            "-year": "95",
+            "#text": "149691"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-59",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Gottingen",
+          "population": {
+            "-year": "95",
+            "#text": "127519"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Leine"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-61",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Wolfsburg",
+          "population": {
+            "-year": "95",
+            "#text": "126965"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Aller"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-70",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Salzgitter",
+          "population": {
+            "-year": "95",
+            "#text": "117842"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-79",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-10",
+          "name": "Hildesheim",
+          "population": {
+            "-year": "95",
+            "#text": "106095"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-11",
+      "-capital": "cty-cid-cia-Germany-10",
+      "-country": "D",
+      "name": "Nordrhein Westfalen",
+      "area": "34077",
+      "population": "17816079",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-10",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Dusseldorf",
+          "longitude": "6.8",
+          "latitude": "51.2",
+          "population": {
+            "-year": "95",
+            "#text": "572638"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-5",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Koln",
+          "population": {
+            "-year": "95",
+            "#text": "963817"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-7",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Essen",
+          "population": {
+            "-year": "95",
+            "#text": "617955"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-8",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Dortmund",
+          "population": {
+            "-year": "95",
+            "#text": "600918"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-12",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Duisburg",
+          "population": {
+            "-year": "95",
+            "#text": "536106"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-17",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Bochum",
+          "population": {
+            "-year": "95",
+            "#text": "401129"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-18",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Wuppertal",
+          "population": {
+            "-year": "95",
+            "#text": "383776"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-19",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Bielefeld",
+          "population": {
+            "-year": "95",
+            "#text": "324067"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-21",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Gelsenkirchen",
+          "population": {
+            "-year": "95",
+            "#text": "293542"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-22",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Bonn",
+          "longitude": "7.5",
+          "latitude": "50.4",
+          "population": {
+            "-year": "95",
+            "#text": "293072"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-27",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Monchengladbach",
+          "population": {
+            "-year": "95",
+            "#text": "266073"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-29",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Munster",
+          "population": {
+            "-year": "95",
+            "#text": "264887"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-32",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Krefeld",
+          "population": {
+            "-year": "95",
+            "#text": "249662"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-33",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Aachen",
+          "population": {
+            "-year": "95",
+            "#text": "247113"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-36",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Oberhausen",
+          "population": {
+            "-year": "95",
+            "#text": "225443"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-38",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Hagen",
+          "population": {
+            "-year": "95",
+            "#text": "213747"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-44",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Hamm",
+          "population": {
+            "-year": "95",
+            "#text": "184020"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-45",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Herne",
+          "population": {
+            "-year": "95",
+            "#text": "180029"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-46",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Mulheim an der Ruhr",
+          "population": {
+            "-year": "95",
+            "#text": "176513"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-49",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Solingen",
+          "population": {
+            "-year": "95",
+            "#text": "165973"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-50",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Leverkusen",
+          "population": {
+            "-year": "95",
+            "#text": "161832"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-52",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Neuss",
+          "population": {
+            "-year": "95",
+            "#text": "148870"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-56",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Paderborn",
+          "population": {
+            "-year": "95",
+            "#text": "131513"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-60",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Recklinghausen",
+          "population": {
+            "-year": "95",
+            "#text": "127139"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-65",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Remscheid",
+          "population": {
+            "-year": "95",
+            "#text": "123069"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-67",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Bottrop",
+          "population": {
+            "-year": "95",
+            "#text": "119669"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-73",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Siegen",
+          "population": {
+            "-year": "95",
+            "#text": "111541"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-78",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Moers",
+          "population": {
+            "-year": "95",
+            "#text": "107011"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-80",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Witten",
+          "population": {
+            "-year": "95",
+            "#text": "105423"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-81",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-11",
+          "name": "Bergisch Gladbach",
+          "population": {
+            "-year": "95",
+            "#text": "105122"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-12",
+      "-capital": "cty-cid-cia-Germany-43",
+      "-country": "D",
+      "name": "Rheinland Pfalz",
+      "area": "19851",
+      "population": "3951573",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-43",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-12",
+          "name": "Mainz",
+          "longitude": "8.1",
+          "latitude": "50",
+          "population": {
+            "-year": "95",
+            "#text": "184627"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-48",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-12",
+          "name": "Ludwigshafen",
+          "population": {
+            "-year": "95",
+            "#text": "167883"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-75",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-12",
+          "name": "Koblenz",
+          "population": {
+            "-year": "95",
+            "#text": "109550"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Rhein"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Mosel"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Germany-84",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-12",
+          "name": "Kaiserslautern",
+          "population": {
+            "-year": "95",
+            "#text": "101910"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-13",
+      "-capital": "cty-cid-cia-Germany-42",
+      "-country": "D",
+      "name": "Saarland",
+      "area": "2570",
+      "population": "1084201",
+      "city": {
+        "-id": "cty-cid-cia-Germany-42",
+        "-is_state_cap": "yes",
+        "-country": "D",
+        "-province": "prov-cid-cia-Germany-13",
+        "name": "Saarbrucken",
+        "longitude": "6.6",
+        "latitude": "49.1",
+        "population": [
+          {
+            "-year": "87",
+            "#text": "183945"
+          },
+          {
+            "-year": "95",
+            "#text": "189012"
+          }
+        ],
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Saar"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Germany-14",
+      "-capital": "cty-cid-cia-Germany-16",
+      "-country": "D",
+      "name": "Sachsen",
+      "area": "18412",
+      "population": "4584345",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-16",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-14",
+          "name": "Dresden",
+          "longitude": "13.7",
+          "latitude": "51.05",
+          "population": {
+            "-year": "95",
+            "#text": "474443"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Elbe"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-15",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-14",
+          "name": "Leipzig",
+          "population": {
+            "-year": "95",
+            "#text": "481121"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-25",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-14",
+          "name": "Chemnitz",
+          "population": {
+            "-year": "95",
+            "#text": "274162"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-82",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-14",
+          "name": "Zwickau",
+          "population": {
+            "-year": "95",
+            "#text": "104921"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-15",
+      "-capital": "cty-cid-cia-Germany-28",
+      "-country": "D",
+      "name": "Sachsen Anhalt",
+      "area": "20446",
+      "population": "2759213",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-28",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-15",
+          "name": "Magdeburg",
+          "longitude": "11.6",
+          "latitude": "52.1",
+          "population": {
+            "-year": "95",
+            "#text": "265379"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Elbe"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-23",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-15",
+          "name": "Halle",
+          "population": {
+            "-year": "95",
+            "#text": "290051"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-16",
+      "-capital": "cty-cid-cia-Germany-34",
+      "-country": "D",
+      "name": "Schleswig Holstein",
+      "area": "15738",
+      "population": "2708392",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-34",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-16",
+          "name": "Kiel",
+          "longitude": "10.7",
+          "latitude": "54.2",
+          "population": {
+            "-year": "95",
+            "#text": "246586"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-37",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-16",
+          "name": "Lubeck",
+          "population": {
+            "-year": "95",
+            "#text": "216854"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Germany-17",
+      "-capital": "cty-cid-cia-Germany-39",
+      "-country": "D",
+      "name": "Thuringen",
+      "area": "16171",
+      "population": "2517776",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Germany-39",
+          "-is_state_cap": "yes",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-17",
+          "name": "Erfurt",
+          "longitude": "11.0",
+          "latitude": "51.0",
+          "population": {
+            "-year": "95",
+            "#text": "213472"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-62",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-17",
+          "name": "Gera",
+          "population": {
+            "-year": "95",
+            "#text": "126035"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Germany-83",
+          "-country": "D",
+          "-province": "prov-cid-cia-Germany-17",
+          "name": "Jena",
+          "population": {
+            "-year": "95",
+            "#text": "102204"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "H",
+  "-area": "93030",
+  "-capital": "cty-Hungary-Budapest",
+  "-memberships": "org-BIS org-CEI org-CE org-CCC org-ECE org-EBRD org-CERN org-EU org-FAO org-G-9 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-NSG org-OECD org-OSCE org-OAS org-PFP org-PCA org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNMOT org-UNOMIG org-UNHCR org-UNU org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Hungary",
+  "population": "10002541",
+  "population_growth": "-0.68",
+  "infant_mortality": "12.3",
+  "gdp_total": "72500",
+  "gdp_agri": "7.3",
+  "gdp_ind": "37.5",
+  "gdp_serv": "55.2",
+  "inflation": "28.3",
+  "indep_date": "1001-01-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "2.6",
+      "#text": "German"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "89.9",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "0.8",
+      "#text": "Slovak"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Romanian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "67.5",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "98.2",
+    "#text": "Hungarian"
+  },
+  "border": [
+    {
+      "-country": "SRB",
+      "-length": "151"
+    },
+    {
+      "-country": "A",
+      "-length": "366"
+    },
+    {
+      "-country": "SK",
+      "-length": "515"
+    },
+    {
+      "-country": "SLO",
+      "-length": "102"
+    },
+    {
+      "-country": "UA",
+      "-length": "103"
+    },
+    {
+      "-country": "HR",
+      "-length": "329"
+    },
+    {
+      "-country": "RO",
+      "-length": "443"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Hungary-2",
+      "-capital": "cty-cid-cia-Hungary-6",
+      "-country": "H",
+      "name": "Baranya",
+      "area": "4487",
+      "population": "417100"
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-3",
+      "-capital": "cty-cid-cia-Hungary-10",
+      "-country": "H",
+      "name": "Bacs Kiskun",
+      "area": "8363",
+      "population": "540800",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-10",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-3",
+        "name": "Kecskemet",
+        "population": {
+          "-year": "92",
+          "#text": "105000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-4",
+      "-capital": "cty-cid-cia-Hungary-Bekescaba",
+      "-country": "H",
+      "name": "Bekes",
+      "area": "5632",
+      "population": "404000",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Bekescaba",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-4",
+        "name": "Bekescaba"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-5",
+      "-capital": "cty-cid-cia-Hungary-4",
+      "-country": "H",
+      "name": "Borsod Abauj Zemplen",
+      "area": "7248",
+      "population": "749100"
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-6",
+      "-capital": "cty-cid-cia-Hungary-Hodmezovasarhely",
+      "-country": "H",
+      "name": "Csongrad",
+      "area": "4263",
+      "population": "437600",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Hodmezovasarhely",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-6",
+        "name": "Hodmezovasarhely"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-7",
+      "-capital": "cty-cid-cia-Hungary-9",
+      "-country": "H",
+      "name": "Fejer",
+      "area": "4374",
+      "population": "422500",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-9",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-7",
+        "name": "Szekesfehervar",
+        "population": {
+          "-year": "92",
+          "#text": "109000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-8",
+      "-capital": "cty-cid-cia-Hungary-7",
+      "-country": "H",
+      "name": "Gyor Sopron",
+      "area": "4012",
+      "population": "426800"
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-9",
+      "-capital": "cty-cid-cia-Hungary-3",
+      "-country": "H",
+      "name": "Hajdu Bihar",
+      "area": "6212",
+      "population": "549700"
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-10",
+      "-capital": "cty-cid-cia-Hungary-Eger",
+      "-country": "H",
+      "name": "Heves",
+      "area": "3637",
+      "population": "330200",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Eger",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-10",
+        "name": "Eger"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-11",
+      "-capital": "cty-cid-cia-Hungary-Tatabanya",
+      "-country": "H",
+      "name": "Komarom Esztergom",
+      "area": "2250",
+      "population": "312900",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Tatabanya",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-11",
+        "name": "Tatabanya"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-12",
+      "-capital": "cty-cid-cia-Hungary-Salgotarjan",
+      "-country": "H",
+      "name": "Nograd",
+      "area": "2544",
+      "population": "222700",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Salgotarjan",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-12",
+        "name": "Salgotarjan"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-13",
+      "-capital": "cty-Hungary-Budapest",
+      "-country": "H",
+      "name": "Pest",
+      "area": "6394",
+      "population": "957900"
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-14",
+      "-capital": "cty-cid-cia-Hungary-Kaposvar",
+      "-country": "H",
+      "name": "Somogy",
+      "area": "6035",
+      "population": "340000",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Kaposvar",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-14",
+        "name": "Kaposvar"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-15",
+      "-capital": "cty-cid-cia-Hungary-8",
+      "-country": "H",
+      "name": "Szabolcs Szatmar",
+      "area": "5938",
+      "population": "563500",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-8",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-15",
+        "name": "Nyiregyhaza",
+        "population": {
+          "-year": "92",
+          "#text": "115000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-16",
+      "-capital": "cty-cid-cia-Hungary-Szolnok",
+      "-country": "H",
+      "name": "Szolnok",
+      "area": "5608",
+      "population": "420900",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Szolnok",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-16",
+        "name": "Szolnok",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Theiss"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-17",
+      "-capital": "cty-cid-cia-Hungary-Szekszard",
+      "-country": "H",
+      "name": "Tolna",
+      "area": "3702",
+      "population": "251000",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Szekszard",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-17",
+        "name": "Szekszard"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-18",
+      "-capital": "cty-cid-cia-Hungary-Szombathely",
+      "-country": "H",
+      "name": "Vas",
+      "area": "3337",
+      "population": "273900",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Szombathely",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-18",
+        "name": "Szombathely"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-19",
+      "-capital": "cty-cid-cia-Hungary-Veszprem",
+      "-country": "H",
+      "name": "Veszprem",
+      "area": "4689",
+      "population": "378300",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Veszprem",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-19",
+        "name": "Veszprem"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-20",
+      "-capital": "cty-cid-cia-Hungary-Zalaegerszeg",
+      "-country": "H",
+      "name": "Zala",
+      "area": "3786",
+      "population": "302600",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-Zalaegerszeg",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-20",
+        "name": "Zalaegerszeg"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-21",
+      "-capital": "cty-Hungary-Budapest",
+      "-country": "H",
+      "name": "Budapest (munic.)",
+      "area": "525",
+      "population": "2008500",
+      "city": {
+        "-id": "cty-Hungary-Budapest",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-21",
+        "name": "Budapest",
+        "longitude": "19.0333",
+        "latitude": "47.5167",
+        "population": {
+          "-year": "92",
+          "#text": "2016000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Donau"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-22",
+      "-capital": "cty-cid-cia-Hungary-3",
+      "-country": "H",
+      "name": "Debrecen (munic.)",
+      "area": "446",
+      "population": "217300",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-3",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-22",
+        "name": "Debrecen",
+        "population": {
+          "-year": "92",
+          "#text": "216000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-23",
+      "-capital": "cty-cid-cia-Hungary-7",
+      "-country": "H",
+      "name": "Gyor (munic.)",
+      "area": "175",
+      "population": "130600",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-7",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-23",
+        "name": "Gyor",
+        "population": {
+          "-year": "92",
+          "#text": "130000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Raab"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-24",
+      "-capital": "cty-cid-cia-Hungary-4",
+      "-country": "H",
+      "name": "Miskolc (munic.)",
+      "area": "224",
+      "population": "191000",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-4",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-24",
+        "name": "Miskolc",
+        "population": {
+          "-year": "92",
+          "#text": "192000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-25",
+      "-capital": "cty-cid-cia-Hungary-6",
+      "-country": "H",
+      "name": "Pecs (munic.)",
+      "area": "113",
+      "population": "171600",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-6",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-25",
+        "name": "Pecs",
+        "population": {
+          "-year": "92",
+          "#text": "171000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Hungary-26",
+      "-capital": "cty-cid-cia-Hungary-5",
+      "-country": "H",
+      "name": "Szeged (munic.)",
+      "area": "145",
+      "population": "178500",
+      "city": {
+        "-id": "cty-cid-cia-Hungary-5",
+        "-is_state_cap": "yes",
+        "-country": "H",
+        "-province": "prov-cid-cia-Hungary-26",
+        "name": "Szeged",
+        "population": {
+          "-year": "92",
+          "#text": "178000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Theiss"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "I",
+  "-area": "301230",
+  "-capital": "cty-Italy-Rome",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CDB org-CEI org-CE org-CCC org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAIA org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNITAR org-UNIFIL org-UNIKOM org-UNMOGIP org-MINURSO org-UNHCR org-UNTSO org-UPU org-WEU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Italy",
+  "population": "57460274",
+  "population_growth": "0.13",
+  "infant_mortality": "6.9",
+  "gdp_total": "1088600",
+  "gdp_agri": "2.9",
+  "gdp_ind": "31.6",
+  "gdp_serv": "65.5",
+  "inflation": "5.4",
+  "indep_date": "1861-03-17",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "98",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "F",
+      "-length": "488"
+    },
+    {
+      "-country": "A",
+      "-length": "430"
+    },
+    {
+      "-country": "SLO",
+      "-length": "235"
+    },
+    {
+      "-country": "CH",
+      "-length": "740"
+    },
+    {
+      "-country": "V",
+      "-length": "3.2"
+    },
+    {
+      "-country": "RSM",
+      "-length": "39"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Italy-2",
+      "-capital": "cty-cid-cia-Italy-5",
+      "-country": "I",
+      "name": "Piemonte",
+      "area": "25399",
+      "population": "4307000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-44",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-2",
+          "name": "Novara",
+          "population": {
+            "-year": "90",
+            "#text": "103349"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-5",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-2",
+          "name": "Turin",
+          "longitude": "7.39",
+          "latitude": "45.5",
+          "population": {
+            "-year": "90",
+            "#text": "991870"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Po"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-9",
+      "-capital": "cty-cid-cia-Italy-Aosta",
+      "-country": "I",
+      "name": "Valle dAosta",
+      "area": "3262",
+      "population": "118000",
+      "city": {
+        "-id": "cty-cid-cia-Italy-Aosta",
+        "-is_state_cap": "yes",
+        "-country": "I",
+        "-province": "prov-cid-cia-Italy-9",
+        "name": "Aosta",
+        "longitude": "7.22",
+        "latitude": "45.45"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Italy-10",
+      "-capital": "cty-Italy-Milan",
+      "-country": "I",
+      "name": "Lombardia",
+      "area": "23857",
+      "population": "8901000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-36",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-10",
+          "name": "Bergamo",
+          "population": {
+            "-year": "90",
+            "#text": "117886"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-19",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-10",
+          "name": "Brescia",
+          "population": {
+            "-year": "90",
+            "#text": "196766"
+          }
+        },
+        {
+          "-id": "cty-Italy-Milan",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-10",
+          "name": "Milan",
+          "longitude": "9.28333",
+          "latitude": "45.45",
+          "population": {
+            "-year": "90",
+            "#text": "1432184"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-34",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-10",
+          "name": "Monza",
+          "population": {
+            "-year": "90",
+            "#text": "123188"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-20",
+      "-capital": "cty-cid-cia-Italy-51",
+      "-country": "I",
+      "name": "Trentino Alto Adige",
+      "area": "13618",
+      "population": "904000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-51",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-20",
+          "name": "Bolzano",
+          "longitude": "11.5",
+          "latitude": "46",
+          "population": {
+            "-year": "90",
+            "#text": "100380"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Etsch"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-49",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-20",
+          "name": "Trento",
+          "longitude": "11.5",
+          "latitude": "46",
+          "population": {
+            "-year": "90",
+            "#text": "102124"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Etsch"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-23",
+      "-capital": "cty-cid-cia-Italy-12",
+      "-country": "I",
+      "name": "Veneto",
+      "area": "18364",
+      "population": "4415000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-17",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-23",
+          "name": "Padova",
+          "population": {
+            "-year": "90",
+            "#text": "218186"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-14",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-23",
+          "name": "Verona",
+          "population": {
+            "-year": "90",
+            "#text": "258946"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Etsch"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-39",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-23",
+          "name": "Vicenza",
+          "population": {
+            "-year": "90",
+            "#text": "109333"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-12",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-23",
+          "name": "Venice",
+          "longitude": "12.2",
+          "latitude": "45.27",
+          "population": {
+            "-year": "90",
+            "#text": "317837"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-31",
+      "-capital": "cty-cid-cia-Italy-16",
+      "-country": "I",
+      "name": "Friuli Venezia Giulia",
+      "area": "7845",
+      "population": "1193000",
+      "city": {
+        "-id": "cty-cid-cia-Italy-16",
+        "-is_state_cap": "yes",
+        "-country": "I",
+        "-province": "prov-cid-cia-Italy-31",
+        "name": "Trieste",
+        "longitude": "13.45",
+        "latitude": "45.39",
+        "population": {
+          "-year": "90",
+          "#text": "231047"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Italy-36",
+      "-capital": "cty-cid-cia-Italy-7",
+      "-country": "I",
+      "name": "Liguria",
+      "area": "5418",
+      "population": "1663000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-46",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-36",
+          "name": "La Spezia",
+          "population": {
+            "-year": "90",
+            "#text": "103008"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-7",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-36",
+          "name": "Genua",
+          "longitude": "8.58",
+          "latitude": "44.25",
+          "population": {
+            "-year": "90",
+            "#text": "701032"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-41",
+      "-capital": "cty-cid-cia-Italy-8",
+      "-country": "I",
+      "name": "Emilia Romagna",
+      "area": "22123",
+      "population": "3924000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-8",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Bologna",
+          "longitude": "11.22",
+          "latitude": "44.3",
+          "population": {
+            "-year": "90",
+            "#text": "411803"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-28",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Ferrara",
+          "population": {
+            "-year": "90",
+            "#text": "140600"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-38",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Forli",
+          "population": {
+            "-year": "90",
+            "#text": "109755"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-21",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Modena",
+          "population": {
+            "-year": "90",
+            "#text": "177501"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-22",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Parma",
+          "population": {
+            "-year": "90",
+            "#text": "173991"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-43",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Piacenza",
+          "population": {
+            "-year": "90",
+            "#text": "103536"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Po"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-29",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Ravenna",
+          "population": {
+            "-year": "90",
+            "#text": "136724"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-30",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Reggio nellEmilia",
+          "population": {
+            "-year": "90",
+            "#text": "131880"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-31",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-41",
+          "name": "Rimini",
+          "population": {
+            "-year": "90",
+            "#text": "130896"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-50",
+      "-capital": "cty-cid-cia-Italy-9",
+      "-country": "I",
+      "name": "Toscana",
+      "area": "22992",
+      "population": "3528000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-23",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-50",
+          "name": "Livorno",
+          "population": {
+            "-year": "90",
+            "#text": "171265"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-50",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-50",
+          "name": "Pisa",
+          "population": {
+            "-year": "90",
+            "#text": "101500"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Arno"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-9",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-50",
+          "name": "Firenze",
+          "longitude": "11.17",
+          "latitude": "43.48",
+          "population": {
+            "-year": "90",
+            "#text": "408403"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Arno"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-24",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-50",
+          "name": "Prato",
+          "population": {
+            "-year": "90",
+            "#text": "166688"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-60",
+      "-capital": "cty-cid-cia-Italy-27",
+      "-country": "I",
+      "name": "Umbria",
+      "area": "8456",
+      "population": "819000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-27",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-60",
+          "name": "Perugia",
+          "longitude": "12.2",
+          "latitude": "43.1",
+          "population": {
+            "-year": "90",
+            "#text": "150576"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-37",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-60",
+          "name": "Terni",
+          "population": {
+            "-year": "90",
+            "#text": "109809"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-63",
+      "-capital": "cty-cid-cia-Italy-45",
+      "-country": "I",
+      "name": "Marche",
+      "area": "9693",
+      "population": "1438000",
+      "city": {
+        "-id": "cty-cid-cia-Italy-45",
+        "-is_state_cap": "yes",
+        "-country": "I",
+        "-province": "prov-cid-cia-Italy-63",
+        "name": "Ancona",
+        "longitude": "13.28",
+        "latitude": "43.38",
+        "population": {
+          "-year": "90",
+          "#text": "103268"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Italy-68",
+      "-capital": "cty-Italy-Rome",
+      "-country": "I",
+      "name": "Lazio",
+      "area": "17203",
+      "population": "5185000",
+      "city": [
+        {
+          "-id": "cty-Italy-Rome",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-68",
+          "name": "Rome",
+          "longitude": "12.6",
+          "latitude": "41.8",
+          "population": {
+            "-year": "90",
+            "#text": "2791354"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Tiber"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-42",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-68",
+          "name": "Latina",
+          "population": {
+            "-year": "90",
+            "#text": "103630"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-74",
+      "-capital": "cty-cid-cia-Italy-LAquila",
+      "-country": "I",
+      "name": "Abruzzo",
+      "area": "10794",
+      "population": "1263000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-32",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-74",
+          "name": "Pescara",
+          "population": {
+            "-year": "90",
+            "#text": "128553"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-LAquila",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-74",
+          "name": "LAquila",
+          "longitude": "13.2",
+          "latitude": "42.24"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-79",
+      "-capital": "cty-cid-cia-Italy-Campobasso",
+      "-country": "I",
+      "name": "Molise",
+      "area": "4438",
+      "population": "332000",
+      "city": {
+        "-id": "cty-cid-cia-Italy-Campobasso",
+        "-is_state_cap": "yes",
+        "-country": "I",
+        "-province": "prov-cid-cia-Italy-79",
+        "name": "Campobasso",
+        "longitude": "14.5",
+        "latitude": "41.5"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Italy-82",
+      "-capital": "cty-cid-cia-Italy-4",
+      "-country": "I",
+      "name": "Campania",
+      "area": "13595",
+      "population": "5709000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-26",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-82",
+          "name": "Salerno",
+          "population": {
+            "-year": "90",
+            "#text": "151374"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-4",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-82",
+          "name": "Napoli",
+          "longitude": "14.15",
+          "latitude": "40.5",
+          "population": {
+            "-year": "90",
+            "#text": "1206013"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-47",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-82",
+          "name": "Torre del Greco",
+          "population": {
+            "-year": "90",
+            "#text": "102647"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-88",
+      "-capital": "cty-cid-cia-Italy-11",
+      "-country": "I",
+      "name": "Puglia",
+      "area": "19348",
+      "population": "4066000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-11",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-88",
+          "name": "Bari",
+          "longitude": "16.5",
+          "latitude": "41.8",
+          "population": {
+            "-year": "90",
+            "#text": "353032"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-25",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-88",
+          "name": "Foggia",
+          "population": {
+            "-year": "90",
+            "#text": "159541"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-48",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-88",
+          "name": "Lecce",
+          "population": {
+            "-year": "90",
+            "#text": "102344"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-15",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-88",
+          "name": "Taranto",
+          "population": {
+            "-year": "90",
+            "#text": "244033"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-94",
+      "-capital": "cty-cid-cia-Italy-Potenza",
+      "-country": "I",
+      "name": "Basilicata",
+      "area": "9992",
+      "population": "611000",
+      "city": {
+        "-id": "cty-cid-cia-Italy-Potenza",
+        "-is_state_cap": "yes",
+        "-country": "I",
+        "-province": "prov-cid-cia-Italy-94",
+        "name": "Potenza",
+        "longitude": "15.47",
+        "latitude": "40.4"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Italy-97",
+      "-capital": "cty-cid-cia-Italy-41",
+      "-country": "I",
+      "name": "Calabria",
+      "area": "15080",
+      "population": "2080000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-41",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-97",
+          "name": "Catanzaro",
+          "longitude": "16.35",
+          "latitude": "38.55",
+          "population": {
+            "-year": "90",
+            "#text": "103802"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-40",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-97",
+          "name": "Cosenza",
+          "population": {
+            "-year": "90",
+            "#text": "104483"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-20",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-97",
+          "name": "Reggio di Calabria",
+          "population": {
+            "-year": "90",
+            "#text": "178496"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-101",
+      "-capital": "cty-cid-cia-Italy-6",
+      "-country": "I",
+      "name": "Sicilia",
+      "area": "25709",
+      "population": "5025000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-6",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-101",
+          "name": "Palermo",
+          "longitude": "13.2",
+          "latitude": "38.1",
+          "population": {
+            "-year": "90",
+            "#text": "734238"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          },
+          "located_on": { "-island": "island-Sicilia" }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-13",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-101",
+          "name": "Messina",
+          "population": {
+            "-year": "90",
+            "#text": "274846"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          },
+          "located_on": { "-island": "island-Sicilia" }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-10",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-101",
+          "name": "Catania",
+          "population": {
+            "-year": "90",
+            "#text": "364176"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          },
+          "located_on": { "-island": "island-Sicilia" }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-33",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-101",
+          "name": "Siracusa",
+          "population": {
+            "-year": "90",
+            "#text": "125444"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          },
+          "located_on": { "-island": "island-Sicilia" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Italy-111",
+      "-capital": "cty-cid-cia-Italy-18",
+      "-country": "I",
+      "name": "Sardegna",
+      "area": "24090",
+      "population": "1657000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Italy-18",
+          "-is_state_cap": "yes",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-111",
+          "name": "Cagliari",
+          "longitude": "9.5",
+          "latitude": "39.15",
+          "population": {
+            "-year": "90",
+            "#text": "211719"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Mittelmeer"
+          },
+          "located_on": { "-island": "island-Sardegna" }
+        },
+        {
+          "-id": "cty-cid-cia-Italy-35",
+          "-country": "I",
+          "-province": "prov-cid-cia-Italy-111",
+          "name": "Sassari",
+          "population": {
+            "-year": "90",
+            "#text": "120011"
+          },
+          "located_on": { "-island": "island-Sardegna" }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "FL",
+  "-area": "160",
+  "-capital": "cty-cid-cia-Liechtenstein-Vaduz",
+  "-memberships": "org-CE org-ECE org-EBRD org-EFTA org-IAEA org-Interpol org-IFRCS org-IOC org-ICRM org-ITU org-Intelsat org-OSCE org-PCA org-UN org-UPU org-WCL org-WIPO org-WTrO",
+  "name": "Liechtenstein",
+  "population": "31122",
+  "population_growth": "1.08",
+  "infant_mortality": "5.3",
+  "gdp_total": "630",
+  "inflation": "5.4",
+  "indep_date": "1719-01-23",
+  "government": "hereditary constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "5",
+      "#text": "Italian"
+    },
+    {
+      "-percentage": "95",
+      "#text": "Alemannic"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "87.3",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "8.3",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "A",
+      "-length": "37"
+    },
+    {
+      "-country": "CH",
+      "-length": "41"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Liechtenstein-Vaduz",
+    "-is_country_cap": "yes",
+    "-country": "FL",
+    "name": "Vaduz",
+    "longitude": "9.3",
+    "latitude": "47.08",
+    "population": {
+      "-year": "87",
+      "#text": "27714"
+    }
+  }
+}
+{
+  "-car_code": "SK",
+  "-area": "48845",
+  "-capital": "cty-cid-cia-Slovakia-Bratislava",
+  "-memberships": "org-BIS org-BSEC org-CEI org-CE org-CCC org-ECE org-EBRD org-CERN org-EU org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-ANC org-NSG org-OSCE org-PFP org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Slovakia",
+  "population": "5374362",
+  "population_growth": "0.34",
+  "infant_mortality": "10.7",
+  "gdp_total": "39000",
+  "gdp_agri": "6.7",
+  "gdp_ind": "47.6",
+  "gdp_serv": "45.7",
+  "inflation": "7.5",
+  "indep_date": "1993-01-01",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.1",
+      "#text": "German"
+    },
+    {
+      "-percentage": "0.1",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "1.5",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "10.7",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Czech"
+    },
+    {
+      "-percentage": "85.7",
+      "#text": "Slovak"
+    },
+    {
+      "-percentage": "0.3",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "0.3",
+      "#text": "Ruthenian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "60.3",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "8.4",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "4.1",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "A",
+      "-length": "91"
+    },
+    {
+      "-country": "CZ",
+      "-length": "215"
+    },
+    {
+      "-country": "H",
+      "-length": "515"
+    },
+    {
+      "-country": "PL",
+      "-length": "444"
+    },
+    {
+      "-country": "UA",
+      "-length": "90"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Slovakia-Bratislava",
+    "-is_country_cap": "yes",
+    "-country": "SK",
+    "name": "Bratislava",
+    "longitude": "17.1",
+    "latitude": "48.1",
+    "located_at": [
+      {
+        "-watertype": "river",
+        "-river": "river-Donau"
+      },
+      {
+        "-watertype": "river",
+        "-river": "river-March"
+      }
+    ]
+  }
+}
+{
+  "-car_code": "SLO",
+  "-area": "20256",
+  "-capital": "cty-cid-cia-Slovenia-Ljubljana",
+  "-memberships": "org-CEI org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-IADB org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-ANC org-OSCE org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Slovenia",
+  "population": "1951443",
+  "population_growth": "-0.27",
+  "infant_mortality": "7.3",
+  "gdp_total": "22600",
+  "gdp_agri": "5.3",
+  "gdp_ind": "39.9",
+  "gdp_serv": "54.8",
+  "inflation": "8",
+  "indep_date": "1991-06-25",
+  "government": "emerging democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "91",
+      "#text": "Slovene"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Croat"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "96",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "91",
+      "#text": "Slovenian"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Croatian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "A",
+      "-length": "324"
+    },
+    {
+      "-country": "H",
+      "-length": "102"
+    },
+    {
+      "-country": "I",
+      "-length": "235"
+    },
+    {
+      "-country": "HR",
+      "-length": "546"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Slovenia-Ljubljana",
+    "-is_country_cap": "yes",
+    "-country": "SLO",
+    "name": "Ljubljana",
+    "longitude": "14.5",
+    "latitude": "46.05",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Save"
+    }
+  }
+}
+{
+  "-car_code": "CH",
+  "-area": "41290",
+  "-capital": "cty-cid-cia-Switzerland-5",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CE org-CCC org-ECE org-EBRD org-EFTA org-CERN org-ESA org-FAO org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNAMIR org-UNCTAD org-UNESCO org-UNIDO org-UNITAR org-UNMOT org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UNU org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Switzerland",
+  "population": "7207060",
+  "population_growth": "0.59",
+  "infant_mortality": "5.4",
+  "gdp_total": "158500",
+  "gdp_agri": "3",
+  "gdp_ind": "33.5",
+  "gdp_serv": "63.5",
+  "inflation": "1.8",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "47.6",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "44.3",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "18",
+      "#text": "French"
+    },
+    {
+      "-percentage": "65",
+      "#text": "German"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Italian"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Romansch"
+    }
+  ],
+  "border": [
+    {
+      "-country": "F",
+      "-length": "573"
+    },
+    {
+      "-country": "A",
+      "-length": "164"
+    },
+    {
+      "-country": "D",
+      "-length": "334"
+    },
+    {
+      "-country": "I",
+      "-length": "740"
+    },
+    {
+      "-country": "FL",
+      "-length": "41"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Switzerland-2",
+      "-capital": "cty-cid-cia-Switzerland-Aarau",
+      "-country": "CH",
+      "name": "Aargau",
+      "area": "1403",
+      "population": "528887",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Aarau",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-2",
+        "name": "Aarau",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Aare"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-3",
+      "-capital": "cty-cid-cia-Switzerland-Herisau",
+      "-country": "CH",
+      "name": "Appenzell Ausserrhoden",
+      "area": "242",
+      "population": "54104",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Herisau",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-3",
+        "name": "Herisau"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-4",
+      "-capital": "cty-cid-cia-Switzerland-Appenzell",
+      "-country": "CH",
+      "name": "Appenzell Innerrhoden",
+      "area": "172",
+      "population": "14750",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Appenzell",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-4",
+        "name": "Appenzell"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-5",
+      "-capital": "cty-cid-cia-Switzerland-Liestal",
+      "-country": "CH",
+      "name": "Basel-Land",
+      "area": "517",
+      "population": "252331",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Liestal",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-5",
+        "name": "Liestal"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-6",
+      "-capital": "cty-cid-cia-Switzerland-3",
+      "-country": "CH",
+      "name": "Basel-Stadt",
+      "area": "37",
+      "population": "195759",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-3",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-6",
+        "name": "Basel",
+        "population": {
+          "-year": "91",
+          "#text": "172768"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Rhein"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-7",
+      "-capital": "cty-cid-cia-Switzerland-5",
+      "-country": "CH",
+      "name": "Bern",
+      "area": "5960",
+      "population": "941952",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Switzerland-5",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "CH",
+          "-province": "prov-cid-cia-Switzerland-7",
+          "name": "Bern",
+          "longitude": "7.3",
+          "latitude": "46.6",
+          "population": {
+            "-year": "91",
+            "#text": "134393"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Aare"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Switzerland-10",
+          "-country": "CH",
+          "-province": "prov-cid-cia-Switzerland-7",
+          "name": "Biel",
+          "population": {
+            "-year": "91",
+            "#text": "53308"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-8",
+      "-capital": "cty-cid-cia-Switzerland-Fribourg",
+      "-country": "CH",
+      "name": "Fribourg",
+      "area": "1670",
+      "population": "224552",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Fribourg",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-8",
+        "name": "Fribourg"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-9",
+      "-capital": "cty-cid-cia-Switzerland-4",
+      "-country": "CH",
+      "name": "Geneve",
+      "area": "282",
+      "population": "395466",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-4",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-9",
+        "name": "Geneva",
+        "longitude": "6.9",
+        "latitude": "46.12",
+        "population": {
+          "-year": "91",
+          "#text": "167697"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Rhone"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-Genfer_See"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-10",
+      "-capital": "cty-cid-cia-Switzerland-Glarus",
+      "-country": "CH",
+      "name": "Glarus",
+      "area": "685",
+      "population": "39410",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Glarus",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-10",
+        "name": "Glarus"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-11",
+      "-capital": "cty-cid-cia-Switzerland-Chur",
+      "-country": "CH",
+      "name": "Graubunden",
+      "area": "7105",
+      "population": "185063",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Chur",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-11",
+        "name": "Chur",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Rhein"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-12",
+      "-capital": "cty-cid-cia-Switzerland-Delemont",
+      "-country": "CH",
+      "name": "Jura",
+      "area": "836",
+      "population": "69188",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Delemont",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-12",
+        "name": "Delemont"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-13",
+      "-capital": "cty-cid-cia-Switzerland-9",
+      "-country": "CH",
+      "name": "Luzern",
+      "area": "1493",
+      "population": "340536",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-9",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-13",
+        "name": "Luzern",
+        "population": {
+          "-year": "91",
+          "#text": "59811"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Reuss"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-Vierwaldstattersee"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-14",
+      "-capital": "cty-cid-cia-Switzerland-Neuchatel",
+      "-country": "CH",
+      "name": "Neuchatel",
+      "area": "803",
+      "population": "165258",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Neuchatel",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-14",
+        "name": "Neuchatel"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-15",
+      "-capital": "cty-cid-cia-Switzerland-Stans",
+      "-country": "CH",
+      "name": "Nidwalden",
+      "area": "276",
+      "population": "36466",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Stans",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-15",
+        "name": "Stans"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-16",
+      "-capital": "cty-cid-cia-Switzerland-Sarnen",
+      "-country": "CH",
+      "name": "Obwalden",
+      "area": "490",
+      "population": "31310",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Sarnen",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-16",
+        "name": "Sarnen"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-17",
+      "-capital": "cty-cid-cia-Switzerland-8",
+      "-country": "CH",
+      "name": "Sankt Gallen",
+      "area": "2025",
+      "population": "442350",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-8",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-17",
+        "name": "Sankt Gallen",
+        "population": {
+          "-year": "91",
+          "#text": "74106"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-18",
+      "-capital": "cty-cid-cia-Switzerland-Schaffhausen",
+      "-country": "CH",
+      "name": "Schaffhausen",
+      "area": "298",
+      "population": "74035",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Schaffhausen",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-18",
+        "name": "Schaffhausen",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Rhein"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-19",
+      "-capital": "cty-cid-cia-Switzerland-Schwyz",
+      "-country": "CH",
+      "name": "Schwyz",
+      "area": "908",
+      "population": "122409",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Schwyz",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-19",
+        "name": "Schwyz"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-20",
+      "-capital": "cty-cid-cia-Switzerland-Solothurn",
+      "-country": "CH",
+      "name": "Solothurn",
+      "area": "790",
+      "population": "239264",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Solothurn",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-20",
+        "name": "Solothurn",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Aare"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-21",
+      "-capital": "cty-cid-cia-Switzerland-Frauenfeld",
+      "-country": "CH",
+      "name": "Thurgau",
+      "area": "990",
+      "population": "223372",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Frauenfeld",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-21",
+        "name": "Frauenfeld"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-22",
+      "-capital": "cty-cid-cia-Switzerland-Bellinzona",
+      "-country": "CH",
+      "name": "Tessin",
+      "area": "2812",
+      "population": "305199",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Bellinzona",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-22",
+        "name": "Bellinzona",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ticino"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-23",
+      "-capital": "cty-cid-cia-Switzerland-Altdorf",
+      "-country": "CH",
+      "name": "Uri",
+      "area": "1076",
+      "population": "35876",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Altdorf",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-23",
+        "name": "Altdorf",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Reuss"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-24",
+      "-capital": "cty-cid-cia-Switzerland-Sion",
+      "-country": "CH",
+      "name": "Valais",
+      "area": "5224",
+      "population": "271291",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Sion",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-24",
+        "name": "Sion",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Rhone"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-25",
+      "-capital": "cty-cid-cia-Switzerland-6",
+      "-country": "CH",
+      "name": "Vaud",
+      "area": "3211",
+      "population": "605677",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-6",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-25",
+        "name": "Lausanne",
+        "population": {
+          "-year": "91",
+          "#text": "123149"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Genfer_See"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-26",
+      "-capital": "cty-cid-cia-Switzerland-Zug",
+      "-country": "CH",
+      "name": "Zug",
+      "area": "238",
+      "population": "92392",
+      "city": {
+        "-id": "cty-cid-cia-Switzerland-Zug",
+        "-is_state_cap": "yes",
+        "-country": "CH",
+        "-province": "prov-cid-cia-Switzerland-26",
+        "name": "Zug"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Switzerland-27",
+      "-capital": "cty-cid-cia-Switzerland-2",
+      "-country": "CH",
+      "name": "Zurich",
+      "area": "1728",
+      "population": "1175457",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Switzerland-2",
+          "-is_state_cap": "yes",
+          "-country": "CH",
+          "-province": "prov-cid-cia-Switzerland-27",
+          "name": "Zurich",
+          "population": {
+            "-year": "91",
+            "#text": "343106"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Limmat"
+            },
+            {
+              "-watertype": "lake",
+              "-lake": "lake-Zurichsee"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Switzerland-7",
+          "-country": "CH",
+          "-province": "prov-cid-cia-Switzerland-27",
+          "name": "Winterthur",
+          "population": {
+            "-year": "91",
+            "#text": "86340"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "BY",
+  "-area": "207600",
+  "-capital": "cty-Belarus-Minsk",
+  "-memberships": "org-CEI org-CIS org-CE org-CCC org-ECE org-EBRD org-IAEA org-IBRD org-ICAO org-Interpol org-IFRCS org-IFC org-ILO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ITU org-Intelsat org-ANC org-OSCE org-PFP org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Belarus",
+  "population": "10415973",
+  "population_growth": "0.2",
+  "infant_mortality": "13.4",
+  "gdp_total": "49200",
+  "gdp_agri": "21",
+  "gdp_ind": "49",
+  "gdp_serv": "30",
+  "inflation": "244",
+  "indep_date": "1991-08-25",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4.1",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "2.9",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "13.2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "77.9",
+      "#text": "Byelorussian"
+    }
+  ],
+  "religions": {
+    "-percentage": "60",
+    "#text": "Christian Orthodox"
+  },
+  "border": [
+    {
+      "-country": "LV",
+      "-length": "141"
+    },
+    {
+      "-country": "LT",
+      "-length": "502"
+    },
+    {
+      "-country": "PL",
+      "-length": "605"
+    },
+    {
+      "-country": "UA",
+      "-length": "891"
+    },
+    {
+      "-country": "R",
+      "-length": "959"
+    }
+  ],
+  "city": {
+    "-id": "cty-Belarus-Minsk",
+    "-is_country_cap": "yes",
+    "-country": "BY",
+    "name": "Minsk",
+    "longitude": "27.55",
+    "latitude": "53.9",
+    "population": {
+      "-year": "87",
+      "#text": "1540000"
+    }
+  }
+}
+{
+  "-car_code": "LV",
+  "-area": "64100",
+  "-capital": "cty-cid-cia-Latvia-Riga",
+  "-memberships": "org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EU org-FAO org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-ANC org-OSCE org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Latvia",
+  "population": "2468982",
+  "population_growth": "-1.39",
+  "infant_mortality": "21.2",
+  "gdp_total": "14700",
+  "gdp_agri": "9",
+  "gdp_ind": "31",
+  "gdp_serv": "60",
+  "inflation": "20",
+  "indep_date": "1991-09-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2.3",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "3.4",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "33.8",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "4.5",
+      "#text": "Byelorussian"
+    },
+    {
+      "-percentage": "51.8",
+      "#text": "Latvian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BY",
+      "-length": "141"
+    },
+    {
+      "-country": "LT",
+      "-length": "453"
+    },
+    {
+      "-country": "R",
+      "-length": "217"
+    },
+    {
+      "-country": "EW",
+      "-length": "267"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Latvia-Riga",
+    "-is_country_cap": "yes",
+    "-country": "LV",
+    "name": "Riga",
+    "longitude": "24.1",
+    "latitude": "57",
+    "population": {
+      "-year": "87",
+      "#text": "900000"
+    },
+    "located_at": [
+      {
+        "-watertype": "sea",
+        "-sea": "sea-Ostsee"
+      },
+      {
+        "-watertype": "river",
+        "-river": "river-Westliche_Dwina"
+      }
+    ]
+  }
+}
+{
+  "-car_code": "LT",
+  "-area": "65200",
+  "-capital": "cty-cid-cia-Lithuania-Vilnius",
+  "-memberships": "org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EU org-FAO org-IAEA org-IBRD org-ICAO org-Interpol org-IFRCS org-IFC org-ILO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-ANC org-OSCE org-PFP org-UN org-UNCRO org-UNESCO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Lithuania",
+  "population": "3646041",
+  "population_growth": "-0.35",
+  "infant_mortality": "17",
+  "gdp_total": "13300",
+  "gdp_agri": "20",
+  "gdp_ind": "42",
+  "gdp_serv": "38",
+  "inflation": "35",
+  "indep_date": "1991-09-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "7.7",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "8.6",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "1.5",
+      "#text": "Byelorussian"
+    },
+    {
+      "-percentage": "80.1",
+      "#text": "Lithuanian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BY",
+      "-length": "502"
+    },
+    {
+      "-country": "LV",
+      "-length": "453"
+    },
+    {
+      "-country": "PL",
+      "-length": "91"
+    },
+    {
+      "-country": "R",
+      "-length": "227"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Lithuania-Vilnius",
+    "-is_country_cap": "yes",
+    "-country": "LT",
+    "name": "Vilnius",
+    "longitude": "25.3",
+    "latitude": "54.4",
+    "population": {
+      "-year": "87",
+      "#text": "566000"
+    }
+  }
+}
+{
+  "-car_code": "PL",
+  "-area": "312683",
+  "-capital": "cty-Poland-Warsaw",
+  "-memberships": "org-BIS org-BSEC org-CEI org-CE org-CBSS org-CCC org-ECE org-EBRD org-CERN org-EU org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-ANC org-NSG org-OSCE org-OAS org-PFP org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNDOF org-UNESCO org-UNIDO org-UNIFIL org-UNIKOM org-MINURSO org-UNMOT org-UNOMIG org-UNPREDEP org-UNPROFOR org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Poland",
+  "population": "38642565",
+  "population_growth": "0.14",
+  "infant_mortality": "12.4",
+  "gdp_total": "226700",
+  "gdp_agri": "7",
+  "gdp_ind": "38",
+  "gdp_serv": "55",
+  "inflation": "21.6",
+  "indep_date": "1918-11-11",
+  "government": "democratic state",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1.3",
+      "#text": "German"
+    },
+    {
+      "-percentage": "97.6",
+      "#text": "Polish"
+    },
+    {
+      "-percentage": "0.6",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Byelorussian"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Polish"
+  },
+  "border": [
+    {
+      "-country": "CZ",
+      "-length": "658"
+    },
+    {
+      "-country": "D",
+      "-length": "456"
+    },
+    {
+      "-country": "SK",
+      "-length": "444"
+    },
+    {
+      "-country": "BY",
+      "-length": "605"
+    },
+    {
+      "-country": "LT",
+      "-length": "91"
+    },
+    {
+      "-country": "UA",
+      "-length": "428"
+    },
+    {
+      "-country": "R",
+      "-length": "206"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Poland-2",
+      "-capital": "cty-Poland-Warsaw",
+      "-country": "PL",
+      "name": "Warszwaskie",
+      "area": "3788",
+      "population": "2421000",
+      "city": {
+        "-id": "cty-Poland-Warsaw",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-2",
+        "name": "Warsaw",
+        "longitude": "21.0333",
+        "latitude": "52.2167",
+        "population": {
+          "-year": "90",
+          "#text": "1655000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Weichsel"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-3",
+      "-capital": "cty-cid-cia-Poland-Biala-Podlaska",
+      "-country": "PL",
+      "name": "Bialskopodlaskie",
+      "area": "5348",
+      "population": "306700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Biala-Podlaska",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-3",
+        "name": "Biala Podlaska"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-4",
+      "-capital": "cty-cid-cia-Poland-12",
+      "-country": "PL",
+      "name": "Bialostockie",
+      "area": "10055",
+      "population": "697000",
+      "city": {
+        "-id": "cty-cid-cia-Poland-12",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-4",
+        "name": "Bialystok",
+        "population": {
+          "-year": "90",
+          "#text": "268000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-5",
+      "-capital": "cty-cid-cia-Poland-23",
+      "-country": "PL",
+      "name": "Bielskie",
+      "area": "3704",
+      "population": "911500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-23",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-5",
+        "name": "Bielsko Biala",
+        "population": {
+          "-year": "90",
+          "#text": "180000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-6",
+      "-capital": "cty-cid-cia-Poland-9",
+      "-country": "PL",
+      "name": "Bydgoskie",
+      "area": "10349",
+      "population": "1120300",
+      "city": {
+        "-id": "cty-cid-cia-Poland-9",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-6",
+        "name": "Bydgoszcz",
+        "population": {
+          "-year": "90",
+          "#text": "380000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Weichsel"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-7",
+      "-capital": "cty-cid-cia-Poland-Chelm",
+      "-country": "PL",
+      "name": "Chelmskie",
+      "area": "3866",
+      "population": "248500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Chelm",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-7",
+        "name": "Chelm"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-8",
+      "-capital": "cty-cid-cia-Poland-Ciechanow",
+      "-country": "PL",
+      "name": "Ciechanowskie",
+      "area": "6362",
+      "population": "431400",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Ciechanow",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-8",
+        "name": "Ciechanow"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-9",
+      "-capital": "cty-cid-cia-Poland-14",
+      "-country": "PL",
+      "name": "Czestochowskie",
+      "area": "6182",
+      "population": "748000",
+      "city": {
+        "-id": "cty-cid-cia-Poland-14",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-9",
+        "name": "Czestochowa",
+        "population": {
+          "-year": "90",
+          "#text": "258000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-10",
+      "-capital": "cty-cid-cia-Poland-32",
+      "-country": "PL",
+      "name": "Elblaskie",
+      "area": "6103",
+      "population": "483200",
+      "city": {
+        "-id": "cty-cid-cia-Poland-32",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-10",
+        "name": "Elblag",
+        "population": {
+          "-year": "90",
+          "#text": "125000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-11",
+      "-capital": "cty-cid-cia-Poland-7",
+      "-country": "PL",
+      "name": "Gdanskie",
+      "area": "7394",
+      "population": "1445000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Poland-7",
+          "-is_state_cap": "yes",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-11",
+          "name": "Gdansk",
+          "population": {
+            "-year": "90",
+            "#text": "465000"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Ostsee"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Weichsel"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Poland-15",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-11",
+          "name": "Gdynia",
+          "population": {
+            "-year": "90",
+            "#text": "251000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Poland-12",
+      "-capital": "cty-cid-cia-Poland-Gorzow-Wielkopolskie",
+      "-country": "PL",
+      "name": "Gorzowskie",
+      "area": "8484",
+      "population": "505600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Poland-33",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-12",
+          "name": "Gorzow Wielkopolski",
+          "population": {
+            "-year": "90",
+            "#text": "123000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-Gorzow-Wielkopolskie",
+          "-is_state_cap": "yes",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-12",
+          "name": "Gorzow Wielkopolskie"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Poland-13",
+      "-capital": "cty-cid-cia-Poland-Jelenia-Gora",
+      "-country": "PL",
+      "name": "Jeleniogorskie",
+      "area": "4379",
+      "population": "519200",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Jelenia-Gora",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-13",
+        "name": "Jelenia Gora"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-14",
+      "-capital": "cty-cid-cia-Poland-40",
+      "-country": "PL",
+      "name": "Kaliskie",
+      "area": "6512",
+      "population": "715600",
+      "city": {
+        "-id": "cty-cid-cia-Poland-40",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-14",
+        "name": "Kalisz",
+        "population": {
+          "-year": "90",
+          "#text": "106000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-15",
+      "-capital": "cty-cid-cia-Poland-10",
+      "-country": "PL",
+      "name": "Katowickie",
+      "area": "6650",
+      "population": "4013200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Poland-10",
+          "-is_state_cap": "yes",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Katowice",
+          "population": {
+            "-year": "90",
+            "#text": "367000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-42",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Jastrzebie Zdroj",
+          "population": {
+            "-year": "90",
+            "#text": "103000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-13",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Sosnowiec",
+          "population": {
+            "-year": "90",
+            "#text": "259000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-29",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Dabrowa Gornicza",
+          "population": {
+            "-year": "90",
+            "#text": "136000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-16",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Bytom",
+          "population": {
+            "-year": "90",
+            "#text": "230000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-18",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Gliwice",
+          "population": {
+            "-year": "90",
+            "#text": "216000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-20",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Zabrze",
+          "population": {
+            "-year": "90",
+            "#text": "203000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-22",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Tychy",
+          "population": {
+            "-year": "90",
+            "#text": "190000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-24",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Ruda Slaska",
+          "population": {
+            "-year": "90",
+            "#text": "170000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-27",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Rybnik",
+          "population": {
+            "-year": "90",
+            "#text": "143000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-30",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Chorzow",
+          "population": {
+            "-year": "90",
+            "#text": "133000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-38",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-15",
+          "name": "Wodzilaw Slaski",
+          "population": {
+            "-year": "90",
+            "#text": "111000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Poland-16",
+      "-capital": "cty-cid-cia-Poland-19",
+      "-country": "PL",
+      "name": "Kieleckie",
+      "area": "9211",
+      "population": "1127700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-19",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-16",
+        "name": "Kielce",
+        "population": {
+          "-year": "90",
+          "#text": "213000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-17",
+      "-capital": "cty-cid-cia-Poland-Konin",
+      "-country": "PL",
+      "name": "Koninskie",
+      "area": "5139",
+      "population": "472400",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Konin",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-17",
+        "name": "Konin"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-18",
+      "-capital": "cty-cid-cia-Poland-39",
+      "-country": "PL",
+      "name": "Koszalinskie",
+      "area": "8470",
+      "population": "513700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-39",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-18",
+        "name": "Koszalin",
+        "population": {
+          "-year": "90",
+          "#text": "108000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-19",
+      "-capital": "cty-Poland-Krakow",
+      "-country": "PL",
+      "name": "Krakowskie",
+      "area": "3254",
+      "population": "1238100",
+      "city": {
+        "-id": "cty-Poland-Krakow",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-19",
+        "name": "Krakow",
+        "longitude": "19.95",
+        "latitude": "50.0667",
+        "population": {
+          "-year": "90",
+          "#text": "748000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Weichsel"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-20",
+      "-capital": "cty-cid-cia-Poland-Krosno",
+      "-country": "PL",
+      "name": "Krosnienskie",
+      "area": "5702",
+      "population": "500700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Krosno",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-20",
+        "name": "Krosno"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-21",
+      "-capital": "cty-cid-cia-Poland-41",
+      "-country": "PL",
+      "name": "Legnickie",
+      "area": "4037",
+      "population": "521500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-41",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-21",
+        "name": "Legnica",
+        "population": {
+          "-year": "90",
+          "#text": "104000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-22",
+      "-capital": "cty-cid-cia-Poland-Leszno",
+      "-country": "PL",
+      "name": "Leszczynskie",
+      "area": "4154",
+      "population": "391500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Leszno",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-22",
+        "name": "Leszno"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-23",
+      "-capital": "cty-cid-cia-Poland-11",
+      "-country": "PL",
+      "name": "Lubelskie",
+      "area": "6792",
+      "population": "1022600",
+      "city": {
+        "-id": "cty-cid-cia-Poland-11",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-23",
+        "name": "Lublin",
+        "population": {
+          "-year": "90",
+          "#text": "350000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-24",
+      "-capital": "cty-cid-cia-Poland-Lomza",
+      "-country": "PL",
+      "name": "Lomzynskie",
+      "area": "6684",
+      "population": "349000",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Lomza",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-24",
+        "name": "Lomza"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-25",
+      "-capital": "cty-cid-cia-Poland-3",
+      "-country": "PL",
+      "name": "Lodzkie",
+      "area": "1523",
+      "population": "1132400",
+      "city": {
+        "-id": "cty-cid-cia-Poland-3",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-25",
+        "name": "Lodz",
+        "population": {
+          "-year": "90",
+          "#text": "852000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-26",
+      "-capital": "cty-cid-cia-Poland-Nowy-Sacz",
+      "-country": "PL",
+      "name": "Nowosadeckie",
+      "area": "5576",
+      "population": "709500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Nowy-Sacz",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-26",
+        "name": "Nowy Sacz"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-27",
+      "-capital": "cty-cid-cia-Poland-25",
+      "-country": "PL",
+      "name": "Olsztynskie",
+      "area": "12327",
+      "population": "761300",
+      "city": {
+        "-id": "cty-cid-cia-Poland-25",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-27",
+        "name": "Olsztyn",
+        "population": {
+          "-year": "90",
+          "#text": "161000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-28",
+      "-capital": "cty-cid-cia-Poland-31",
+      "-country": "PL",
+      "name": "Opolskie",
+      "area": "8535",
+      "population": "1023800",
+      "city": {
+        "-id": "cty-cid-cia-Poland-31",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-28",
+        "name": "Opole",
+        "population": {
+          "-year": "90",
+          "#text": "128000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Oder"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-29",
+      "-capital": "cty-cid-cia-Poland-Ostroleka",
+      "-country": "PL",
+      "name": "Ostroleckie",
+      "area": "6498",
+      "population": "400500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Ostroleka",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-29",
+        "name": "Ostroleka"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-30",
+      "-capital": "cty-cid-cia-Poland-Pila",
+      "-country": "PL",
+      "name": "Pilskie",
+      "area": "8205",
+      "population": "485700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Pila",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-30",
+        "name": "Pila"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-31",
+      "-capital": "cty-cid-cia-Poland-Piottrkow-Trybunalski",
+      "-country": "PL",
+      "name": "Piotrkowskie",
+      "area": "6266",
+      "population": "644200",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Piottrkow-Trybunalski",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-31",
+        "name": "Piotrkow Trybunalski"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-32",
+      "-capital": "cty-cid-cia-Poland-34",
+      "-country": "PL",
+      "name": "Plockie",
+      "area": "5117",
+      "population": "518600",
+      "city": {
+        "-id": "cty-cid-cia-Poland-34",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-32",
+        "name": "Plock",
+        "population": {
+          "-year": "90",
+          "#text": "122000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-33",
+      "-capital": "cty-cid-cia-Poland-6",
+      "-country": "PL",
+      "name": "Poznanskie",
+      "area": "8151",
+      "population": "1344200",
+      "city": {
+        "-id": "cty-cid-cia-Poland-6",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-33",
+        "name": "Poznan",
+        "population": {
+          "-year": "90",
+          "#text": "589000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-34",
+      "-capital": "cty-cid-cia-Poland-Przemysl",
+      "-country": "PL",
+      "name": "Przemyskie",
+      "area": "4437",
+      "population": "409600",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Przemysl",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-34",
+        "name": "Przemysl"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-35",
+      "-capital": "cty-cid-cia-Poland-17",
+      "-country": "PL",
+      "name": "Radomskie",
+      "area": "7294",
+      "population": "755500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-17",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-35",
+        "name": "Radom",
+        "population": {
+          "-year": "90",
+          "#text": "226000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-36",
+      "-capital": "cty-cid-cia-Poland-26",
+      "-country": "PL",
+      "name": "Rzeszowskie",
+      "area": "4397",
+      "population": "734100",
+      "city": {
+        "-id": "cty-cid-cia-Poland-26",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-36",
+        "name": "Rzeszow",
+        "population": {
+          "-year": "90",
+          "#text": "151000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-37",
+      "-capital": "cty-cid-cia-Poland-Siedlce",
+      "-country": "PL",
+      "name": "Siedleckie",
+      "area": "8499",
+      "population": "655300",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Siedlce",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-37",
+        "name": "Siedlce"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-38",
+      "-capital": "cty-cid-cia-Poland-Sieradz",
+      "-country": "PL",
+      "name": "Sieradzkie",
+      "area": "4869",
+      "population": "408700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Sieradz",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-38",
+        "name": "Sieradz"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-39",
+      "-capital": "cty-cid-cia-Poland-Skierniewice",
+      "-country": "PL",
+      "name": "Skierniewickie",
+      "area": "3960",
+      "population": "421700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Skierniewice",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-39",
+        "name": "Skierniewice"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-40",
+      "-capital": "cty-cid-cia-Poland-Slupsk",
+      "-country": "PL",
+      "name": "Slupskie",
+      "area": "7453",
+      "population": "419300",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Slupsk",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-40",
+        "name": "Slupsk"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-41",
+      "-capital": "cty-cid-cia-Poland-Suwalki",
+      "-country": "PL",
+      "name": "Suwalskie",
+      "area": "10490",
+      "population": "477100",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Suwalki",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-41",
+        "name": "Suwalki"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-42",
+      "-capital": "cty-cid-cia-Poland-8",
+      "-country": "PL",
+      "name": "Szczecinskie",
+      "area": "9982",
+      "population": "979500",
+      "city": {
+        "-id": "cty-cid-cia-Poland-8",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-42",
+        "name": "Szczecin",
+        "population": {
+          "-year": "90",
+          "#text": "412000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Oder"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-43",
+      "-capital": "cty-cid-cia-Poland-Tarnobrzeg",
+      "-country": "PL",
+      "name": "Tarnobrzeskie",
+      "area": "6283",
+      "population": "604300",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Tarnobrzeg",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-43",
+        "name": "Tarnobrzeg"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-44",
+      "-capital": "cty-cid-cia-Poland-36",
+      "-country": "PL",
+      "name": "Tarnowskie",
+      "area": "4151",
+      "population": "678400",
+      "city": {
+        "-id": "cty-cid-cia-Poland-36",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-44",
+        "name": "Tarnow",
+        "population": {
+          "-year": "90",
+          "#text": "120000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-45",
+      "-capital": "cty-cid-cia-Poland-21",
+      "-country": "PL",
+      "name": "Torunskie",
+      "area": "5348",
+      "population": "662600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Poland-21",
+          "-is_state_cap": "yes",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-45",
+          "name": "Torun",
+          "population": {
+            "-year": "90",
+            "#text": "201000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Poland-43",
+          "-country": "PL",
+          "-province": "prov-cid-cia-Poland-45",
+          "name": "Grudziadz",
+          "population": {
+            "-year": "90",
+            "#text": "101000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Poland-46",
+      "-capital": "cty-cid-cia-Poland-28",
+      "-country": "PL",
+      "name": "Walbrzyskie",
+      "area": "4168",
+      "population": "740000",
+      "city": {
+        "-id": "cty-cid-cia-Poland-28",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-46",
+        "name": "Walbrzych",
+        "population": {
+          "-year": "90",
+          "#text": "141000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-47",
+      "-capital": "cty-cid-cia-Poland-35",
+      "-country": "PL",
+      "name": "Wloclawskie",
+      "area": "4402",
+      "population": "430800",
+      "city": {
+        "-id": "cty-cid-cia-Poland-35",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-47",
+        "name": "Wloclawek",
+        "population": {
+          "-year": "90",
+          "#text": "121000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-48",
+      "-capital": "cty-cid-cia-Poland-5",
+      "-country": "PL",
+      "name": "Wroclawskie",
+      "area": "6287",
+      "population": "1132800",
+      "city": {
+        "-id": "cty-cid-cia-Poland-5",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-48",
+        "name": "Wroclaw",
+        "population": {
+          "-year": "90",
+          "#text": "642000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Oder"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-49",
+      "-capital": "cty-cid-cia-Poland-Zamosc",
+      "-country": "PL",
+      "name": "Zamojskie",
+      "area": "6980",
+      "population": "490800",
+      "city": {
+        "-id": "cty-cid-cia-Poland-Zamosc",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-49",
+        "name": "Zamosc"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Poland-50",
+      "-capital": "cty-cid-cia-Poland-37",
+      "-country": "PL",
+      "name": "Zielonogorskie",
+      "area": "8868",
+      "population": "664700",
+      "city": {
+        "-id": "cty-cid-cia-Poland-37",
+        "-is_state_cap": "yes",
+        "-country": "PL",
+        "-province": "prov-cid-cia-Poland-50",
+        "name": "Zielona Gora",
+        "population": {
+          "-year": "90",
+          "#text": "113000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "UA",
+  "-area": "603700",
+  "-capital": "cty-Ukraine-Kiev",
+  "-memberships": "org-BSEC org-CEI org-CIS org-CE org-CCC org-ECE org-EBRD org-IAEA org-IBRD org-ICAO org-Interpol org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-ANC org-NSG org-OSCE org-PFP org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNMOT org-UNPREDEP org-UNPROFOR org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Ukraine",
+  "population": "50864009",
+  "population_growth": "-0.4",
+  "infant_mortality": "22.5",
+  "gdp_total": "174600",
+  "gdp_agri": "31",
+  "gdp_ind": "43",
+  "gdp_serv": "26",
+  "inflation": "9",
+  "indep_date": "1991-12-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "73",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Jewish"
+    }
+  ],
+  "border": [
+    {
+      "-country": "H",
+      "-length": "103"
+    },
+    {
+      "-country": "SK",
+      "-length": "90"
+    },
+    {
+      "-country": "BY",
+      "-length": "891"
+    },
+    {
+      "-country": "PL",
+      "-length": "428"
+    },
+    {
+      "-country": "R",
+      "-length": "1576"
+    },
+    {
+      "-country": "RO",
+      "-length": "531"
+    },
+    {
+      "-country": "MD",
+      "-length": "939"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Ukraine-2",
+      "-capital": "cty-cid-cia-Ukraine-21",
+      "-country": "UA",
+      "name": "Cherkaska",
+      "area": "20900",
+      "population": "1530900",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-21",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-2",
+        "name": "Cherkasy",
+        "population": {
+          "-year": "90",
+          "#text": "297000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Dnepr"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-3",
+      "-capital": "cty-cid-cia-Ukraine-20",
+      "-country": "UA",
+      "name": "Chernihivska",
+      "area": "31900",
+      "population": "938600",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-20",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-3",
+        "name": "Chernihiv",
+        "population": {
+          "-year": "90",
+          "#text": "301000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-4",
+      "-capital": "cty-cid-cia-Ukraine-26",
+      "-country": "UA",
+      "name": "Chernivetska",
+      "area": "8100",
+      "population": "1405800",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-26",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-4",
+        "name": "Chernivtsi",
+        "population": {
+          "-year": "90",
+          "#text": "257000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Pruth"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-5",
+      "-capital": "cty-cid-cia-Ukraine-4",
+      "-country": "UA",
+      "name": "Dnipropetrovska",
+      "area": "31900",
+      "population": "3908700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Ukraine-4",
+          "-is_state_cap": "yes",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-5",
+          "name": "Dnipropetrovsk",
+          "population": {
+            "-year": "90",
+            "#text": "1187000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Dnepr"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-9",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-5",
+          "name": "Kryvyy Rih",
+          "population": {
+            "-year": "90",
+            "#text": "717000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-24",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-5",
+          "name": "Dniprodzerzhynsk",
+          "population": {
+            "-year": "90",
+            "#text": "284000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Dnepr"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-6",
+      "-capital": "cty-cid-cia-Ukraine-5",
+      "-country": "UA",
+      "name": "Donetska",
+      "area": "26500",
+      "population": "5346700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Ukraine-5",
+          "-is_state_cap": "yes",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-6",
+          "name": "Donetsk",
+          "population": {
+            "-year": "90",
+            "#text": "1117000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-10",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-6",
+          "name": "Mariupol",
+          "population": {
+            "-year": "90",
+            "#text": "520000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-13",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-6",
+          "name": "Makiyivka",
+          "population": {
+            "-year": "90",
+            "#text": "427000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-18",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-6",
+          "name": "Horlivka",
+          "population": {
+            "-year": "90",
+            "#text": "338000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-7",
+      "-capital": "cty-cid-cia-Ukraine-30",
+      "-country": "UA",
+      "name": "Ivano Frankivska",
+      "area": "13900",
+      "population": "1442900",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-30",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-7",
+        "name": "Ivano Frankivsk",
+        "population": {
+          "-year": "90",
+          "#text": "220000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-8",
+      "-capital": "cty-Ukraine-Kharkiv",
+      "-country": "UA",
+      "name": "Kharkivska",
+      "area": "31400",
+      "population": "3194800",
+      "city": {
+        "-id": "cty-Ukraine-Kharkiv",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-8",
+        "name": "Kharkiv",
+        "longitude": "36.2333",
+        "latitude": "50",
+        "population": {
+          "-year": "90",
+          "#text": "1618000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-9",
+      "-capital": "cty-cid-cia-Ukraine-16",
+      "-country": "UA",
+      "name": "Khersonska",
+      "area": "28500",
+      "population": "1258700",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-16",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-9",
+        "name": "Kherson",
+        "population": {
+          "-year": "90",
+          "#text": "361000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Dnepr"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-10",
+      "-capital": "cty-cid-cia-Ukraine-27",
+      "-country": "UA",
+      "name": "Khmelnytska",
+      "area": "20600",
+      "population": "1520600",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-27",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-10",
+        "name": "Khmelnytskyy",
+        "population": {
+          "-year": "90",
+          "#text": "241000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-11",
+      "-capital": "cty-Ukraine-Kiev",
+      "-country": "UA",
+      "name": "Kyyivska",
+      "area": "28900",
+      "population": "4589800",
+      "city": {
+        "-id": "cty-Ukraine-Kiev",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-11",
+        "name": "Kiev",
+        "longitude": "30.5",
+        "latitude": "50.45",
+        "population": {
+          "-year": "90",
+          "#text": "2616000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Dnepr"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-12",
+      "-capital": "cty-cid-cia-Ukraine-25",
+      "-country": "UA",
+      "name": "Kirovohradska",
+      "area": "24600",
+      "population": "1245300",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-25",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-12",
+        "name": "Kirovohrad",
+        "population": {
+          "-year": "90",
+          "#text": "274000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-13",
+      "-capital": "cty-cid-cia-Ukraine-12",
+      "-country": "UA",
+      "name": "Luhanska",
+      "area": "26700",
+      "population": "2871100",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-12",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-13",
+        "name": "Luhansk",
+        "population": {
+          "-year": "90",
+          "#text": "501000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-14",
+      "-capital": "cty-cid-cia-Ukraine-8",
+      "-country": "UA",
+      "name": "Lvivska",
+      "area": "21800",
+      "population": "2764400",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-8",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-14",
+        "name": "Lviv",
+        "population": {
+          "-year": "90",
+          "#text": "798000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-15",
+      "-capital": "cty-cid-cia-Ukraine-11",
+      "-country": "UA",
+      "name": "Mykolayivska",
+      "area": "24600",
+      "population": "1342400",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-11",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-15",
+        "name": "Mykolayiv",
+        "population": {
+          "-year": "90",
+          "#text": "508000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-16",
+      "-capital": "cty-Ukraine-Odesa",
+      "-country": "UA",
+      "name": "Odeska",
+      "area": "33300",
+      "population": "2635300",
+      "city": {
+        "-id": "cty-Ukraine-Odesa",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-16",
+        "name": "Odesa",
+        "longitude": "30.7333",
+        "latitude": "46.4833",
+        "population": {
+          "-year": "90",
+          "#text": "1106000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-17",
+      "-capital": "cty-cid-cia-Ukraine-19",
+      "-country": "UA",
+      "name": "Poltavska",
+      "area": "28800",
+      "population": "1756900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Ukraine-19",
+          "-is_state_cap": "yes",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-17",
+          "name": "Poltava",
+          "population": {
+            "-year": "90",
+            "#text": "317000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-28",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-17",
+          "name": "Kremenchuk",
+          "population": {
+            "-year": "90",
+            "#text": "238000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Dnepr"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-18",
+      "-capital": "cty-cid-cia-Ukraine-29",
+      "-country": "UA",
+      "name": "Rivnenska",
+      "area": "20100",
+      "population": "1176800",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-29",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-18",
+        "name": "Rivne",
+        "population": {
+          "-year": "90",
+          "#text": "233000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-19",
+      "-capital": "cty-cid-cia-Ukraine-23",
+      "-country": "UA",
+      "name": "Sumska",
+      "area": "23800",
+      "population": "1430200",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-23",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-19",
+        "name": "Sumy",
+        "population": {
+          "-year": "90",
+          "#text": "296000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-20",
+      "-capital": "cty-cid-cia-Ukraine-31",
+      "-country": "UA",
+      "name": "Ternopilska",
+      "area": "13800",
+      "population": "1175100",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-31",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-20",
+        "name": "Ternopil",
+        "population": {
+          "-year": "90",
+          "#text": "212000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-21",
+      "-capital": "cty-cid-cia-Ukraine-14",
+      "-country": "UA",
+      "name": "Vinnytska",
+      "area": "26500",
+      "population": "1914400",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-14",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-21",
+        "name": "Vinnytsya",
+        "population": {
+          "-year": "90",
+          "#text": "379000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-22",
+      "-capital": "cty-cid-cia-Ukraine-32",
+      "-country": "UA",
+      "name": "Volynska",
+      "area": "20200",
+      "population": "1069000",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-32",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-22",
+        "name": "Lutsk",
+        "population": {
+          "-year": "90",
+          "#text": "204000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-23",
+      "-capital": "cty-cid-cia-Ukraine-Uzhhorod",
+      "-country": "UA",
+      "name": "Zakarpatska",
+      "area": "12800",
+      "population": "1265900",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-Uzhhorod",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-23",
+        "name": "Uzhhorod"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-24",
+      "-capital": "cty-cid-cia-Ukraine-7",
+      "-country": "UA",
+      "name": "Zaporizka",
+      "area": "27200",
+      "population": "2099600",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-7",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-24",
+        "name": "Zaporizhzhya",
+        "population": {
+          "-year": "90",
+          "#text": "891000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Dnepr"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-25",
+      "-capital": "cty-cid-cia-Ukraine-22",
+      "-country": "UA",
+      "name": "Zhytomyrska",
+      "area": "29900",
+      "population": "1510700",
+      "city": {
+        "-id": "cty-cid-cia-Ukraine-22",
+        "-is_state_cap": "yes",
+        "-country": "UA",
+        "-province": "prov-cid-cia-Ukraine-25",
+        "name": "Zhytomyr",
+        "population": {
+          "-year": "90",
+          "#text": "296000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Ukraine-27",
+      "-capital": "cty-cid-cia-Ukraine-17",
+      "-country": "UA",
+      "name": "Krym",
+      "area": "27000",
+      "population": "2549800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Ukraine-17",
+          "-is_state_cap": "yes",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-27",
+          "name": "Simferopol",
+          "population": {
+            "-year": "90",
+            "#text": "349000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Ukraine-15",
+          "-country": "UA",
+          "-province": "prov-cid-cia-Ukraine-27",
+          "name": "Sevastopol",
+          "population": {
+            "-year": "90",
+            "#text": "361000"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "R",
+  "-area": "17075200",
+  "-capital": "cty-Russia-Moscow",
+  "-memberships": "org-BSEC org-CIS org-CE org-CBSS org-CCC org-ESCAP org-ECE org-EBRD org-CERN org-G-8 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NSG org-OSCE org-OAS org-PFP org-PCA org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNIKOM org-MINURSO org-UNMIH org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Russia",
+  "population": "148178487",
+  "population_growth": "-0.07",
+  "infant_mortality": "24.7",
+  "gdp_total": "796000",
+  "gdp_agri": "6",
+  "gdp_ind": "41",
+  "gdp_serv": "53",
+  "inflation": "7",
+  "indep_date": "1991-08-24",
+  "government": "federation",
+  "encompassed": [
+    {
+      "-continent": "europe",
+      "-percentage": "20"
+    },
+    {
+      "-continent": "asia",
+      "-percentage": "80"
+    }
+  ],
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "81.5",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "0.8",
+      "#text": "Byelorussian"
+    },
+    {
+      "-percentage": "3.8",
+      "#text": "Tatar"
+    },
+    {
+      "-percentage": "1.2",
+      "#text": "Chuvash"
+    },
+    {
+      "-percentage": "0.9",
+      "#text": "Bashkir"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Moldavian"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Russian"
+  },
+  "border": [
+    {
+      "-country": "BY",
+      "-length": "959"
+    },
+    {
+      "-country": "LV",
+      "-length": "217"
+    },
+    {
+      "-country": "LT",
+      "-length": "227"
+    },
+    {
+      "-country": "PL",
+      "-length": "206"
+    },
+    {
+      "-country": "UA",
+      "-length": "1576"
+    },
+    {
+      "-country": "EW",
+      "-length": "290"
+    },
+    {
+      "-country": "SF",
+      "-length": "1313"
+    },
+    {
+      "-country": "N",
+      "-length": "167"
+    },
+    {
+      "-country": "TJ",
+      "-length": "3645"
+    },
+    {
+      "-country": "GE",
+      "-length": "723"
+    },
+    {
+      "-country": "AZ",
+      "-length": "284"
+    },
+    {
+      "-country": "KAZ",
+      "-length": "6846"
+    },
+    {
+      "-country": "NOK",
+      "-length": "19"
+    },
+    {
+      "-country": "MNG",
+      "-length": "3441"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Russia-3",
+      "-capital": "cty-cid-cia-Russia-71",
+      "-country": "R",
+      "name": "Rep. of Karelia",
+      "area": "172400",
+      "population": "785000",
+      "city": {
+        "-id": "cty-cid-cia-Russia-71",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-3",
+        "name": "Petrozavodsk",
+        "population": {
+          "-year": "95",
+          "#text": "280000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-4",
+      "-capital": "cty-cid-cia-Russia-85",
+      "-country": "R",
+      "name": "Rep. of Komi",
+      "area": "415900",
+      "population": "1185500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-85",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-4",
+          "name": "Syktyvkar",
+          "population": {
+            "-year": "95",
+            "#text": "229000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-161",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-4",
+          "name": "Ukhta",
+          "population": {
+            "-year": "95",
+            "#text": "106000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-163",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-4",
+          "name": "Vorkuta",
+          "population": {
+            "-year": "95",
+            "#text": "104000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-5",
+      "-capital": "cty-Russia-Arkhangelsk",
+      "-country": "R",
+      "name": "Arkhangelskaya oblast",
+      "area": "587400",
+      "population": "1520800",
+      "city": [
+        {
+          "-id": "cty-Russia-Arkhangelsk",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-5",
+          "name": "Arkhangelsk",
+          "longitude": "40.5333",
+          "latitude": "64.55",
+          "population": {
+            "-year": "95",
+            "#text": "374000"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Noerdliche_Dwina"
+            },
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Barentssee"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Russia-80",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-5",
+          "name": "Severodvinsk",
+          "population": {
+            "-year": "95",
+            "#text": "241000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Barentssee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-7",
+      "-capital": "cty-cid-cia-Russia-66",
+      "-country": "R",
+      "name": "Vologodskaya oblast",
+      "area": "145700",
+      "population": "1349800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-66",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-7",
+          "name": "Vologda",
+          "population": {
+            "-year": "95",
+            "#text": "299000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-62",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-7",
+          "name": "Cherepovets",
+          "population": {
+            "-year": "95",
+            "#text": "320000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-8",
+      "-capital": "cty-cid-cia-Russia-47",
+      "-country": "R",
+      "name": "Murmanskaya oblast",
+      "area": "144900",
+      "population": "1048000",
+      "city": {
+        "-id": "cty-cid-cia-Russia-47",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-8",
+        "name": "Murmansk",
+        "population": {
+          "-year": "95",
+          "#text": "407000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Barentssee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-9",
+      "-capital": "cty-Russia-Kaliningrad",
+      "-country": "R",
+      "name": "Kaliningradskaya oblast",
+      "area": "15100",
+      "population": "932200",
+      "city": {
+        "-id": "cty-Russia-Kaliningrad",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-9",
+        "name": "Kaliningrad",
+        "longitude": "20.5",
+        "latitude": "54.7167",
+        "population": {
+          "-year": "95",
+          "#text": "419000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-10",
+      "-capital": "cty-cid-cia-Russia-3",
+      "-country": "R",
+      "name": "Sankt Peterburg",
+      "area": "0",
+      "population": "4801500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-3",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-10",
+        "name": "Sankt Peterburg",
+        "population": {
+          "-year": "95",
+          "#text": "4838000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-11",
+      "-capital": "cty-cid-cia-Russia-3",
+      "-country": "R",
+      "name": "Leningradskaya oblast",
+      "area": "85900",
+      "population": "1675900",
+      "city": {
+        "-id": "cty-cid-cia-Russia-120",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-11",
+        "name": "Kolpino",
+        "population": {
+          "-year": "95",
+          "#text": "143000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-12",
+      "-capital": "cty-cid-cia-Russia-83",
+      "-country": "R",
+      "name": "Novgorodskaya oblast",
+      "area": "55300",
+      "population": "742600",
+      "city": {
+        "-id": "cty-cid-cia-Russia-83",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-12",
+        "name": "Novgorod",
+        "population": {
+          "-year": "95",
+          "#text": "233000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-13",
+      "-capital": "cty-cid-cia-Russia-90",
+      "-country": "R",
+      "name": "Pskovskaya oblast",
+      "area": "55300",
+      "population": "832300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-90",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-13",
+          "name": "Pskov",
+          "population": {
+            "-year": "95",
+            "#text": "207000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-143",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-13",
+          "name": "Velikiye Luki",
+          "population": {
+            "-year": "95",
+            "#text": "116000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-14",
+      "-capital": "cty-cid-cia-Russia-40",
+      "-country": "R",
+      "name": "Bryanskaya oblast",
+      "area": "34900",
+      "population": "1479700",
+      "city": {
+        "-id": "cty-cid-cia-Russia-40",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-14",
+        "name": "Bryansk",
+        "population": {
+          "-year": "95",
+          "#text": "462000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-15",
+      "-capital": "cty-cid-cia-Russia-57",
+      "-country": "R",
+      "name": "Vladimirskaya oblast",
+      "area": "29000",
+      "population": "1644700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-57",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-15",
+          "name": "Vladimir",
+          "population": {
+            "-year": "95",
+            "#text": "339000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-110",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-15",
+          "name": "Kovrov",
+          "population": {
+            "-year": "95",
+            "#text": "162000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-133",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-15",
+          "name": "Murom",
+          "population": {
+            "-year": "95",
+            "#text": "126000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-16",
+      "-capital": "cty-cid-cia-Russia-36",
+      "-country": "R",
+      "name": "Ivanovskaya oblast",
+      "area": "23900",
+      "population": "1266400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-36",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-16",
+          "name": "Ivanovo",
+          "population": {
+            "-year": "95",
+            "#text": "474000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-165",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-16",
+          "name": "Kineshma",
+          "population": {
+            "-year": "95",
+            "#text": "103000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-17",
+      "-capital": "cty-cid-cia-Russia-55",
+      "-country": "R",
+      "name": "Kaluzhskaya oblast",
+      "area": "29900",
+      "population": "1097300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-55",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-17",
+          "name": "Kaluga",
+          "population": {
+            "-year": "95",
+            "#text": "347000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Oka"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-155",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-17",
+          "name": "Obninsk",
+          "population": {
+            "-year": "95",
+            "#text": "108000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-18",
+      "-capital": "cty-cid-cia-Russia-70",
+      "-country": "R",
+      "name": "Kostromskaya oblast",
+      "area": "60100",
+      "population": "805700",
+      "city": {
+        "-id": "cty-cid-cia-Russia-70",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-18",
+        "name": "Kostroma",
+        "population": {
+          "-year": "95",
+          "#text": "285000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Wolga"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-19",
+      "-capital": "cty-Russia-Moscow",
+      "-country": "R",
+      "name": "Moskva",
+      "area": "0",
+      "population": "8664400",
+      "city": {
+        "-id": "cty-Russia-Moscow",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-19",
+        "name": "Moscow",
+        "longitude": "37.6667",
+        "latitude": "55.7667",
+        "population": {
+          "-year": "95",
+          "#text": "8717000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-20",
+      "-capital": "cty-Russia-Moscow",
+      "-country": "R",
+      "name": "Moskovskaya oblast",
+      "area": "47000",
+      "population": "6596600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-94",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Podolsk",
+          "population": {
+            "-year": "95",
+            "#text": "202000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-98",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Zelenograd",
+          "population": {
+            "-year": "95",
+            "#text": "191000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-106",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Lyubertsy",
+          "population": {
+            "-year": "95",
+            "#text": "166000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-116",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Kolomna",
+          "population": {
+            "-year": "95",
+            "#text": "154000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Oka"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-117",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Mytishchi",
+          "population": {
+            "-year": "95",
+            "#text": "152000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-118",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Elektrostal",
+          "population": {
+            "-year": "95",
+            "#text": "150000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-121",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Serpukhov",
+          "population": {
+            "-year": "95",
+            "#text": "139000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Oka"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-124",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Balashikha",
+          "population": {
+            "-year": "95",
+            "#text": "136000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-127",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Khimki",
+          "population": {
+            "-year": "95",
+            "#text": "134000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-130",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Odintsovo",
+          "population": {
+            "-year": "95",
+            "#text": "129000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-134",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Orekhovo Zuyevo",
+          "population": {
+            "-year": "95",
+            "#text": "126000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-140",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Noginsk",
+          "population": {
+            "-year": "95",
+            "#text": "119000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-157",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Shchyolkovo",
+          "population": {
+            "-year": "95",
+            "#text": "108000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-146",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-20",
+          "name": "Zagorsk",
+          "population": {
+            "-year": "95",
+            "#text": "114000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-21",
+      "-capital": "cty-cid-cia-Russia-54",
+      "-country": "R",
+      "name": "Orjolskaya oblast",
+      "area": "24700",
+      "population": "914000",
+      "city": {
+        "-id": "cty-cid-cia-Russia-54",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-21",
+        "name": "Orel",
+        "population": {
+          "-year": "95",
+          "#text": "348000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-22",
+      "-capital": "cty-cid-cia-Russia-28",
+      "-country": "R",
+      "name": "Ryazanskaya oblast",
+      "area": "39600",
+      "population": "1325300",
+      "city": {
+        "-id": "cty-cid-cia-Russia-28",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-22",
+        "name": "Ryazan",
+        "population": {
+          "-year": "95",
+          "#text": "536000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Oka"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-23",
+      "-capital": "cty-cid-cia-Russia-52",
+      "-country": "R",
+      "name": "Smolenskaya oblast",
+      "area": "49800",
+      "population": "1172400",
+      "city": {
+        "-id": "cty-cid-cia-Russia-52",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-23",
+        "name": "Smolensk",
+        "population": {
+          "-year": "95",
+          "#text": "355000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Dnepr"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-24",
+      "-capital": "cty-cid-cia-Russia-41",
+      "-country": "R",
+      "name": "Tverskaya oblast",
+      "area": "84100",
+      "population": "1650600",
+      "city": {
+        "-id": "cty-cid-cia-Russia-41",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-24",
+        "name": "Tver",
+        "population": {
+          "-year": "95",
+          "#text": "455000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Wolga"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-25",
+      "-capital": "cty-cid-cia-Russia-31",
+      "-country": "R",
+      "name": "Tulskaya oblast",
+      "area": "25700",
+      "population": "1814500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-31",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-25",
+          "name": "Tula",
+          "population": {
+            "-year": "95",
+            "#text": "532000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-119",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-25",
+          "name": "Novomoskovsk",
+          "population": {
+            "-year": "95",
+            "#text": "144000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-26",
+      "-capital": "cty-cid-cia-Russia-23",
+      "-country": "R",
+      "name": "Yaroslavskaya oblast",
+      "area": "36400",
+      "population": "1451400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-23",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-26",
+          "name": "Yaroslavl",
+          "population": {
+            "-year": "95",
+            "#text": "629000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-79",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-26",
+          "name": "Rybinsk",
+          "population": {
+            "-year": "95",
+            "#text": "248000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-27",
+      "-capital": "cty-cid-cia-Russia-78",
+      "-country": "R",
+      "name": "Rep. of Mariy El",
+      "area": "23200",
+      "population": "766300",
+      "city": {
+        "-id": "cty-cid-cia-Russia-78",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-27",
+        "name": "Yoshkar Ola",
+        "population": {
+          "-year": "95",
+          "#text": "251000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-28",
+      "-capital": "cty-cid-cia-Russia-61",
+      "-country": "R",
+      "name": "Rep. of Mordovia",
+      "area": "26200",
+      "population": "955800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-61",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-28",
+        "name": "Saransk",
+        "population": {
+          "-year": "95",
+          "#text": "320000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-29",
+      "-capital": "cty-cid-cia-Russia-42",
+      "-country": "R",
+      "name": "Chuvash Republic",
+      "area": "18300",
+      "population": "1360800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-42",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-29",
+          "name": "Cheboksary",
+          "population": {
+            "-year": "95",
+            "#text": "450000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-136",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-29",
+          "name": "Novocheboksarsk",
+          "population": {
+            "-year": "95",
+            "#text": "123000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-30",
+      "-capital": "cty-cid-cia-Russia-39",
+      "-country": "R",
+      "name": "Kirovskaya oblast",
+      "area": "120800",
+      "population": "1634500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-39",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-30",
+        "name": "Kirov",
+        "population": {
+          "-year": "95",
+          "#text": "464000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-31",
+      "-capital": "cty-cid-cia-Russia-4",
+      "-country": "R",
+      "name": "Nizhegorodskaya oblast",
+      "area": "74800",
+      "population": "3726400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-4",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-31",
+          "name": "Nizhniy Novgorod",
+          "longitude": "44",
+          "latitude": "56.3",
+          "population": {
+            "-year": "95",
+            "#text": "1383000"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Wolga"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Oka"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Russia-69",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-31",
+          "name": "Dzerzhinsk",
+          "population": {
+            "-year": "95",
+            "#text": "285000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Oka"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-147",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-31",
+          "name": "Arzamas",
+          "population": {
+            "-year": "95",
+            "#text": "112000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-32",
+      "-capital": "cty-cid-cia-Russia-59",
+      "-country": "R",
+      "name": "Belgorodskaya oblast",
+      "area": "27100",
+      "population": "1469100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-59",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-32",
+          "name": "Belgorod",
+          "population": {
+            "-year": "95",
+            "#text": "322000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-95",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-32",
+          "name": "Stary Oskol",
+          "population": {
+            "-year": "95",
+            "#text": "198000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-33",
+      "-capital": "cty-cid-cia-Russia-15",
+      "-country": "R",
+      "name": "Voronezhskaya oblast",
+      "area": "52400",
+      "population": "2503800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-15",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-33",
+        "name": "Voronezh",
+        "population": {
+          "-year": "95",
+          "#text": "908000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-34",
+      "-capital": "cty-cid-cia-Russia-43",
+      "-country": "R",
+      "name": "Kurskaya oblast",
+      "area": "29800",
+      "population": "1346900",
+      "city": {
+        "-id": "cty-cid-cia-Russia-43",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-34",
+        "name": "Kursk",
+        "population": {
+          "-year": "95",
+          "#text": "442000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-35",
+      "-capital": "cty-cid-cia-Russia-37",
+      "-country": "R",
+      "name": "Lipetskaya oblast",
+      "area": "24100",
+      "population": "1250200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-37",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-35",
+          "name": "Lipetsk",
+          "population": {
+            "-year": "95",
+            "#text": "474000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-139",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-35",
+          "name": "Yelets",
+          "population": {
+            "-year": "95",
+            "#text": "119000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-36",
+      "-capital": "cty-cid-cia-Russia-63",
+      "-country": "R",
+      "name": "Tambovskaya oblast",
+      "area": "34300",
+      "population": "1310600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-63",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-36",
+          "name": "Tambov",
+          "population": {
+            "-year": "95",
+            "#text": "316000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-154",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-36",
+          "name": "Michurinsk",
+          "population": {
+            "-year": "95",
+            "#text": "108000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-37",
+      "-capital": "cty-cid-cia-Russia-Elista",
+      "-country": "R",
+      "name": "Rep. of Kalmykiya",
+      "area": "76100",
+      "population": "318500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Elista",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-37",
+        "name": "Elista"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-38",
+      "-capital": "cty-cid-cia-Russia-11",
+      "-country": "R",
+      "name": "Rep. of Tatarstan",
+      "area": "68000",
+      "population": "3760500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-11",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-38",
+          "name": "Kazan",
+          "population": {
+            "-year": "95",
+            "#text": "1085000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-32",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-38",
+          "name": "Naberezhnye Chelny",
+          "population": {
+            "-year": "95",
+            "#text": "526000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-88",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-38",
+          "name": "Nizhnekamsk",
+          "population": {
+            "-year": "95",
+            "#text": "210000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-122",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-38",
+          "name": "Almetyevsk",
+          "population": {
+            "-year": "95",
+            "#text": "138000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-167",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-38",
+          "name": "Zelenodolysk",
+          "population": {
+            "-year": "95",
+            "#text": "101000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-39",
+      "-capital": "cty-cid-cia-Russia-35",
+      "-country": "R",
+      "name": "Astrakhanskaya oblast",
+      "area": "44100",
+      "population": "1028900",
+      "city": {
+        "-id": "cty-cid-cia-Russia-35",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-39",
+        "name": "Astrakhan",
+        "population": {
+          "-year": "95",
+          "#text": "486000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Wolga"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-40",
+      "-capital": "cty-Russia-Volgograd",
+      "-country": "R",
+      "name": "Volgogradskaya oblast",
+      "area": "113900",
+      "population": "2703700",
+      "city": [
+        {
+          "-id": "cty-Russia-Volgograd",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-40",
+          "name": "Volgograd",
+          "longitude": "44.5167",
+          "latitude": "48.7",
+          "population": {
+            "-year": "95",
+            "#text": "1003000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-68",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-40",
+          "name": "Volzhsky",
+          "population": {
+            "-year": "95",
+            "#text": "288000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-131",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-40",
+          "name": "Kamyshin",
+          "population": {
+            "-year": "95",
+            "#text": "128000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-41",
+      "-capital": "cty-cid-cia-Russia-29",
+      "-country": "R",
+      "name": "Penzenskaya oblast",
+      "area": "43200",
+      "population": "1562300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-29",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-41",
+          "name": "Penza",
+          "population": {
+            "-year": "95",
+            "#text": "534000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-168",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-41",
+          "name": "Kuznetsk",
+          "population": {
+            "-year": "95",
+            "#text": "100000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-42",
+      "-capital": "cty-cid-cia-Russia-7",
+      "-country": "R",
+      "name": "Samarskaya oblast",
+      "area": "53600",
+      "population": "3311500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-7",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-42",
+          "name": "Samara",
+          "population": {
+            "-year": "95",
+            "#text": "1184000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-18",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-42",
+          "name": "Tolyatti",
+          "population": {
+            "-year": "95",
+            "#text": "702000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-103",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-42",
+          "name": "Syzran",
+          "population": {
+            "-year": "95",
+            "#text": "177000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-145",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-42",
+          "name": "Novokuybyshevsk",
+          "population": {
+            "-year": "95",
+            "#text": "115000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-43",
+      "-capital": "cty-cid-cia-Russia-16",
+      "-country": "R",
+      "name": "Saratovskaya oblast",
+      "area": "100200",
+      "population": "2739500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-16",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-43",
+          "name": "Saratov",
+          "population": {
+            "-year": "95",
+            "#text": "895000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-91",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-43",
+          "name": "Balakovo",
+          "population": {
+            "-year": "95",
+            "#text": "206000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-100",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-43",
+          "name": "Engels",
+          "population": {
+            "-year": "95",
+            "#text": "186000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-44",
+      "-capital": "cty-cid-cia-Russia-19",
+      "-country": "R",
+      "name": "Ulyanovskaya oblast",
+      "area": "37300",
+      "population": "1495200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-19",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-44",
+          "name": "Simbirsk",
+          "population": {
+            "-year": "95",
+            "#text": "678000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Wolga"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-125",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-44",
+          "name": "Dimitrovgrad",
+          "population": {
+            "-year": "95",
+            "#text": "135000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-45",
+      "-capital": "cty-cid-cia-Russia-Rostov-no-Donu",
+      "-country": "R",
+      "name": "Rostovskaya oblast",
+      "area": "100800",
+      "population": "4426400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-13",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Rostov na Donu",
+          "population": {
+            "-year": "95",
+            "#text": "1026000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-67",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Taganrog",
+          "population": {
+            "-year": "95",
+            "#text": "292000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-84",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Shakhty",
+          "population": {
+            "-year": "95",
+            "#text": "230000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-99",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Novocherkassk",
+          "population": {
+            "-year": "95",
+            "#text": "190000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-102",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Volgodonsk",
+          "population": {
+            "-year": "95",
+            "#text": "183000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-159",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Novoshakhtinsk",
+          "population": {
+            "-year": "95",
+            "#text": "107000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-Rostov-no-Donu",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-45",
+          "name": "Rostov no Donu"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-46",
+      "-capital": "cty-cid-cia-Russia-9",
+      "-country": "R",
+      "name": "Rep. of Bashkortostan",
+      "area": "143600",
+      "population": "4096600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-9",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-46",
+          "name": "Ufa",
+          "population": {
+            "-year": "95",
+            "#text": "1094000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-75",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-46",
+          "name": "Sterlitamak",
+          "population": {
+            "-year": "95",
+            "#text": "259000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-115",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-46",
+          "name": "Salavat",
+          "population": {
+            "-year": "95",
+            "#text": "156000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-149",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-46",
+          "name": "Oktyabrsky",
+          "population": {
+            "-year": "95",
+            "#text": "110000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-47",
+      "-capital": "cty-cid-cia-Russia-20",
+      "-country": "R",
+      "name": "Udmurt Republic",
+      "area": "42100",
+      "population": "1639100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-20",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-47",
+          "name": "Izhevsk",
+          "population": {
+            "-year": "95",
+            "#text": "654000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-153",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-47",
+          "name": "Sarapul",
+          "population": {
+            "-year": "95",
+            "#text": "109000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-158",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-47",
+          "name": "Glazov",
+          "population": {
+            "-year": "95",
+            "#text": "107000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-164",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-47",
+          "name": "Votkinsk",
+          "population": {
+            "-year": "95",
+            "#text": "104000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-48",
+      "-capital": "cty-cid-cia-Russia-30",
+      "-country": "R",
+      "name": "Orenburgskaya oblast",
+      "area": "124000",
+      "population": "2228600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-30",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-48",
+          "name": "Orenburg",
+          "population": {
+            "-year": "95",
+            "#text": "532000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ural"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-72",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-48",
+          "name": "Orsk",
+          "population": {
+            "-year": "95",
+            "#text": "275000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ural"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-148",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-48",
+          "name": "Novotroitsk",
+          "population": {
+            "-year": "95",
+            "#text": "110000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-49",
+      "-capital": "cty-cid-cia-Russia-12",
+      "-country": "R",
+      "name": "Permskaya oblast",
+      "area": "160600",
+      "population": "3009400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-12",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-49",
+          "name": "Perm",
+          "population": {
+            "-year": "95",
+            "#text": "1032000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-101",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-49",
+          "name": "Berezniki",
+          "population": {
+            "-year": "95",
+            "#text": "184000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-156",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-49",
+          "name": "Solikamsk",
+          "population": {
+            "-year": "95",
+            "#text": "108000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-52",
+      "-capital": "cty-cid-cia-Russia-107",
+      "-country": "R",
+      "name": "Rep. of Adygeya",
+      "area": "7600",
+      "population": "450500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-107",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-52",
+        "name": "Maykop",
+        "population": {
+          "-year": "95",
+          "#text": "165000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-53",
+      "-capital": "cty-cid-cia-Russia-58",
+      "-country": "R",
+      "name": "Rep. of Dagestan",
+      "area": "50300",
+      "population": "2097500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-58",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-53",
+        "name": "Makhachkala",
+        "population": {
+          "-year": "95",
+          "#text": "339000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-54",
+      "-capital": "cty-cid-cia-Russia-Nazran",
+      "-country": "R",
+      "name": "Rep. of Ingushetiya",
+      "area": "3750",
+      "population": "299700",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Nazran",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-54",
+        "name": "Nazran"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-55",
+      "-capital": "cty-cid-cia-Russia-81",
+      "-country": "R",
+      "name": "Kabardino Balkar Rep.",
+      "area": "12500",
+      "population": "789900",
+      "city": {
+        "-id": "cty-cid-cia-Russia-81",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-55",
+        "name": "Nalchik",
+        "population": {
+          "-year": "95",
+          "#text": "239000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-56",
+      "-capital": "cty-cid-cia-Russia-141",
+      "-country": "R",
+      "name": "Karachayevo Cherkessk Rep.",
+      "area": "14100",
+      "population": "436300",
+      "city": {
+        "-id": "cty-cid-cia-Russia-141",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-56",
+        "name": "Cherkessk",
+        "population": {
+          "-year": "95",
+          "#text": "119000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-57",
+      "-capital": "cty-cid-cia-Russia-64",
+      "-country": "R",
+      "name": "Rep. of North Ossetiya",
+      "area": "8000",
+      "population": "662600",
+      "city": {
+        "-id": "cty-cid-cia-Russia-64",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-57",
+        "name": "Vladikavkaz",
+        "population": {
+          "-year": "95",
+          "#text": "312000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-58",
+      "-capital": "cty-cid-cia-Russia-50",
+      "-country": "R",
+      "name": "Chechen Rep.",
+      "area": "12300",
+      "population": "865100",
+      "city": {
+        "-id": "cty-cid-cia-Russia-50",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-58",
+        "name": "Grozny",
+        "population": {
+          "-year": "95",
+          "#text": "364000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-59",
+      "-capital": "cty-cid-cia-Russia-21",
+      "-country": "R",
+      "name": "Krasnodarsky kray",
+      "area": "76000",
+      "population": "5043900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-21",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-59",
+          "name": "Krasnodar",
+          "population": {
+            "-year": "95",
+            "#text": "646000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-53",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-59",
+          "name": "Sochi",
+          "population": {
+            "-year": "95",
+            "#text": "355000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-93",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-59",
+          "name": "Novorossiysk",
+          "population": {
+            "-year": "95",
+            "#text": "202000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-108",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-59",
+          "name": "Armavir",
+          "population": {
+            "-year": "95",
+            "#text": "164000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-60",
+      "-capital": "cty-cid-cia-Russia-56",
+      "-country": "R",
+      "name": "Stavropolsky kray",
+      "area": "66500",
+      "population": "2667000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-56",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-60",
+          "name": "Stavropol",
+          "population": {
+            "-year": "95",
+            "#text": "342000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-128",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-60",
+          "name": "Pyatigorsk",
+          "population": {
+            "-year": "95",
+            "#text": "133000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-129",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-60",
+          "name": "Nevinnomyssk",
+          "population": {
+            "-year": "95",
+            "#text": "131000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-138",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-60",
+          "name": "Kislovodsk",
+          "population": {
+            "-year": "95",
+            "#text": "120000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-142",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-60",
+          "name": "Neftekamsk",
+          "population": {
+            "-year": "95",
+            "#text": "117000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-61",
+      "-capital": "cty-cid-cia-Russia-51",
+      "-country": "R",
+      "name": "Kurganskaya oblast",
+      "area": "71000",
+      "population": "1112200",
+      "city": {
+        "-id": "cty-cid-cia-Russia-51",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-61",
+        "name": "Kurgan",
+        "population": {
+          "-year": "95",
+          "#text": "363000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tobol"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-62",
+      "-capital": "cty-cid-cia-Russia-6",
+      "-country": "R",
+      "name": "Sverdlovskaya oblast",
+      "area": "194300",
+      "population": "4686300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-6",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-62",
+          "name": "Yekaterinburg",
+          "population": {
+            "-year": "95",
+            "#text": "1280000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-46",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-62",
+          "name": "Nizhniy Tagil",
+          "population": {
+            "-year": "95",
+            "#text": "409000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-96",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-62",
+          "name": "Kamensk Uralskiy",
+          "population": {
+            "-year": "95",
+            "#text": "197000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-123",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-62",
+          "name": "Pervouralsk",
+          "population": {
+            "-year": "95",
+            "#text": "137000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-169",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-62",
+          "name": "Serov",
+          "population": {
+            "-year": "95",
+            "#text": "100000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-63",
+      "-capital": "cty-cid-cia-Russia-10",
+      "-country": "R",
+      "name": "Chelyabinskaya oblast",
+      "area": "87900",
+      "population": "3688700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-10",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-63",
+          "name": "Chelyabinsk",
+          "population": {
+            "-year": "95",
+            "#text": "1086000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-44",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-63",
+          "name": "Magnitogorsk",
+          "population": {
+            "-year": "95",
+            "#text": "427000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ural"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-92",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-63",
+          "name": "Zlatoust",
+          "population": {
+            "-year": "95",
+            "#text": "203000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-105",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-63",
+          "name": "Miass",
+          "population": {
+            "-year": "95",
+            "#text": "167000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-64",
+      "-capital": "cty-cid-cia-Russia-Gorno-Altaysk",
+      "-country": "R",
+      "name": "Rep. of Altay",
+      "area": "92600",
+      "population": "201600",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Gorno-Altaysk",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-64",
+        "name": "Gorno Altaysk"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-65",
+      "-capital": "cty-cid-cia-Russia-25",
+      "-country": "R",
+      "name": "Altayskiy kray",
+      "area": "169100",
+      "population": "2690100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-25",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-65",
+          "name": "Barnaul",
+          "population": {
+            "-year": "95",
+            "#text": "596000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-86",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-65",
+          "name": "Biysk",
+          "population": {
+            "-year": "95",
+            "#text": "228000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Katun"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-104",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-65",
+          "name": "Rubtsovsk",
+          "population": {
+            "-year": "95",
+            "#text": "170000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-66",
+      "-capital": "cty-cid-cia-Russia-33",
+      "-country": "R",
+      "name": "Kemerovskaya oblast",
+      "area": "95500",
+      "population": "3063500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-33",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Kemerovo",
+          "population": {
+            "-year": "95",
+            "#text": "503000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-27",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Novokuznetsk",
+          "population": {
+            "-year": "95",
+            "#text": "572000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-77",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Prokopyevsk",
+          "population": {
+            "-year": "95",
+            "#text": "253000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-137",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Leninsk Kuznetskiy",
+          "population": {
+            "-year": "95",
+            "#text": "121000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-144",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Kiselyovsk",
+          "population": {
+            "-year": "95",
+            "#text": "116000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-162",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Mezhdurechensk",
+          "population": {
+            "-year": "95",
+            "#text": "105000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-166",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-66",
+          "name": "Anzhero Sudzhensk",
+          "population": {
+            "-year": "95",
+            "#text": "101000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-67",
+      "-capital": "cty-cid-cia-Russia-5",
+      "-country": "R",
+      "name": "Novosibirskaya oblast",
+      "area": "178200",
+      "population": "2748600",
+      "city": {
+        "-id": "cty-cid-cia-Russia-5",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-67",
+        "name": "Novosibirsk",
+        "population": {
+          "-year": "95",
+          "#text": "1369000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-68",
+      "-capital": "cty-cid-cia-Russia-8",
+      "-country": "R",
+      "name": "Omskaya oblast",
+      "area": "139700",
+      "population": "2176400",
+      "city": {
+        "-id": "cty-cid-cia-Russia-8",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-68",
+        "name": "Omsk",
+        "population": {
+          "-year": "95",
+          "#text": "1163000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irtysch"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-69",
+      "-capital": "cty-cid-cia-Russia-38",
+      "-country": "R",
+      "name": "Tomskaya oblast",
+      "area": "316900",
+      "population": "1077600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-38",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-69",
+          "name": "Tomsk",
+          "population": {
+            "-year": "95",
+            "#text": "470000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-150",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-69",
+          "name": "Seversk",
+          "population": {
+            "-year": "95",
+            "#text": "110000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-70",
+      "-capital": "cty-cid-cia-Russia-34",
+      "-country": "R",
+      "name": "Tyumenskaya oblast",
+      "area": "1435200",
+      "population": "3169900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-34",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-70",
+          "name": "Tyumen",
+          "population": {
+            "-year": "95",
+            "#text": "494000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-74",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-70",
+          "name": "Surgut",
+          "population": {
+            "-year": "95",
+            "#text": "263000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-82",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-70",
+          "name": "Nizhnevartovsk",
+          "population": {
+            "-year": "95",
+            "#text": "238000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-72",
+      "-capital": "cty-cid-cia-Russia-49",
+      "-country": "R",
+      "name": "Rep. of Buryatiya",
+      "area": "351300",
+      "population": "1052500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-49",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-72",
+        "name": "Ulan Ude",
+        "population": {
+          "-year": "95",
+          "#text": "366000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-73",
+      "-capital": "cty-cid-cia-Russia-Kyzyl",
+      "-country": "R",
+      "name": "Rep. of Tyva",
+      "area": "170500",
+      "population": "309700",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Kyzyl",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-73",
+        "name": "Kyzyl",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Jenissej"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-74",
+      "-capital": "cty-cid-cia-Russia-112",
+      "-country": "R",
+      "name": "Rep. of Khakassiya",
+      "area": "61900",
+      "population": "585800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-112",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-74",
+        "name": "Abakan",
+        "population": {
+          "-year": "95",
+          "#text": "161000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Jenissej"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-75",
+      "-capital": "cty-Russia-Krasnoyarsk",
+      "-country": "R",
+      "name": "Krasnoyarskiy kray",
+      "area": "2339700",
+      "population": "3105900",
+      "city": [
+        {
+          "-id": "cty-Russia-Krasnoyarsk",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-75",
+          "name": "Krasnoyarsk",
+          "longitude": "92.95",
+          "latitude": "56.0167",
+          "population": {
+            "-year": "95",
+            "#text": "869000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jenissej"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-114",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-75",
+          "name": "Norilsk",
+          "population": {
+            "-year": "95",
+            "#text": "159000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-135",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-75",
+          "name": "Achinsk",
+          "population": {
+            "-year": "95",
+            "#text": "123000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-152",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-75",
+          "name": "Kansk",
+          "population": {
+            "-year": "95",
+            "#text": "109000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-77",
+      "-capital": "cty-cid-cia-Russia-26",
+      "-country": "R",
+      "name": "Irkutskaya oblast",
+      "area": "767900",
+      "population": "2795200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-26",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-77",
+          "name": "Irkutsk",
+          "population": {
+            "-year": "95",
+            "#text": "585000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Angara"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-73",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-77",
+          "name": "Angarsk",
+          "population": {
+            "-year": "95",
+            "#text": "267000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Angara"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-76",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-77",
+          "name": "Bratsk",
+          "population": {
+            "-year": "95",
+            "#text": "257000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Angara"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-151",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-77",
+          "name": "Ust Ilimsk",
+          "population": {
+            "-year": "95",
+            "#text": "110000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Angara"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-160",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-77",
+          "name": "Usolye Sibirskoye",
+          "population": {
+            "-year": "95",
+            "#text": "106000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Angara"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-79",
+      "-capital": "cty-cid-cia-Russia-60",
+      "-country": "R",
+      "name": "Chitinskaya oblast",
+      "area": "431500",
+      "population": "1295000",
+      "city": {
+        "-id": "cty-cid-cia-Russia-60",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-79",
+        "name": "Chita",
+        "population": {
+          "-year": "95",
+          "#text": "322000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Schilka"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-81",
+      "-capital": "cty-cid-cia-Russia-97",
+      "-country": "R",
+      "name": "Rep. of Sakha",
+      "area": "3103200",
+      "population": "1022800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-97",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-81",
+        "name": "Yakutsk",
+        "population": {
+          "-year": "95",
+          "#text": "192000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Lena"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-82",
+      "-capital": "cty-cid-cia-Russia-Birobidzhan",
+      "-country": "R",
+      "name": "Yevreyskaya avt. oblast",
+      "area": "36000",
+      "population": "209900",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Birobidzhan",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-82",
+        "name": "Birobidzhan"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-83",
+      "-capital": "cty-cid-cia-Russia-Anadyr",
+      "-country": "R",
+      "name": "Chukotsky ao",
+      "area": "737700",
+      "population": "90500",
+      "city": {
+        "-id": "cty-cid-cia-Russia-Anadyr",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-83",
+        "name": "Anadyr"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-84",
+      "-capital": "cty-Russia-Vladivostok",
+      "-country": "R",
+      "name": "Primorsky kray",
+      "area": "165900",
+      "population": "2255400",
+      "city": [
+        {
+          "-id": "cty-Russia-Vladivostok",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-84",
+          "name": "Vladivostok",
+          "longitude": "131.917",
+          "latitude": "43.1167",
+          "population": {
+            "-year": "95",
+            "#text": "632000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Japanisches_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-109",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-84",
+          "name": "Nakhodka",
+          "population": {
+            "-year": "95",
+            "#text": "163000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Japanisches_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-111",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-84",
+          "name": "Ussuriysk",
+          "population": {
+            "-year": "95",
+            "#text": "162000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-85",
+      "-capital": "cty-cid-cia-Russia-24",
+      "-country": "R",
+      "name": "Khabarovskiy kray",
+      "area": "752600",
+      "population": "1571200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-24",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-85",
+          "name": "Khabarovsk",
+          "population": {
+            "-year": "95",
+            "#text": "618000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Amur"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-65",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-85",
+          "name": "Komsomolsk na Amure",
+          "population": {
+            "-year": "95",
+            "#text": "309000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Amur"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-86",
+      "-capital": "cty-cid-cia-Russia-87",
+      "-country": "R",
+      "name": "Amurskaya oblast",
+      "area": "363700",
+      "population": "1037800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-87",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-86",
+        "name": "Blagoveshchensk",
+        "population": {
+          "-year": "95",
+          "#text": "214000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amur"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-87",
+      "-capital": "cty-cid-cia-Russia-89",
+      "-country": "R",
+      "name": "Kamchatskaya oblast",
+      "area": "472300",
+      "population": "411100",
+      "city": {
+        "-id": "cty-cid-cia-Russia-89",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-87",
+        "name": "Petropavlovsk Kamchatsky",
+        "population": {
+          "-year": "95",
+          "#text": "210000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Russia-89",
+      "-capital": "cty-cid-cia-Russia-132",
+      "-country": "R",
+      "name": "Magadanskaya oblast",
+      "area": "461400",
+      "population": "258200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Russia-132",
+          "-is_state_cap": "yes",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-89",
+          "name": "Magadan",
+          "population": {
+            "-year": "95",
+            "#text": "128000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-OchotskSea"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Russia-Okhotsk",
+          "-country": "R",
+          "-province": "prov-cid-cia-Russia-89",
+          "name": "Okhotsk",
+          "population": {
+            "-year": "08",
+            "#text": "4900"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-OchotskSea"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Russia-90",
+      "-capital": "cty-cid-cia-Russia-113",
+      "-country": "R",
+      "name": "Sakhalinskaya oblast",
+      "area": "87100",
+      "population": "647800",
+      "city": {
+        "-id": "cty-cid-cia-Russia-113",
+        "-is_state_cap": "yes",
+        "-country": "R",
+        "-province": "prov-cid-cia-Russia-90",
+        "name": "Yuzhno Sakhalinsk",
+        "population": {
+          "-year": "95",
+          "#text": "160000"
+        },
+        "located_on": { "-island": "island-Sachalin" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "B",
+  "-area": "30510",
+  "-capital": "cty-Belgium-Brussels",
+  "-memberships": "org-AfDB org-ACCT org-AG org-AsDB org-BIS org-Benelux org-CE org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-9 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNMOGIP org-MINURSO org-UNHCR org-UNPROFOR org-UNRWA org-UNTSO org-UPU org-WEU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Belgium",
+  "population": "10170241",
+  "population_growth": "0.33",
+  "infant_mortality": "6.4",
+  "gdp_total": "197000",
+  "gdp_agri": "2",
+  "gdp_ind": "28",
+  "gdp_serv": "70",
+  "inflation": "1.6",
+  "indep_date": "1830-10-04",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "55",
+      "#text": "Fleming"
+    },
+    {
+      "-percentage": "33",
+      "#text": "Walloon"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "75",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "32",
+      "#text": "French"
+    },
+    {
+      "-percentage": "1",
+      "#text": "German"
+    },
+    {
+      "-percentage": "56",
+      "#text": "Dutch"
+    }
+  ],
+  "border": [
+    {
+      "-country": "F",
+      "-length": "620"
+    },
+    {
+      "-country": "D",
+      "-length": "167"
+    },
+    {
+      "-country": "L",
+      "-length": "148"
+    },
+    {
+      "-country": "NL",
+      "-length": "450"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Belgium-2",
+      "-capital": "cty-cid-cia-Belgium-3-0",
+      "-country": "B",
+      "name": "Antwerp",
+      "area": "2867",
+      "population": "1610695",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-3-0",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-2",
+        "name": "Antwerp",
+        "longitude": "4.23",
+        "latitude": "51.1",
+        "population": {
+          "-year": "95",
+          "#text": "459072"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-3",
+      "-capital": "cty-Belgium-Brussels",
+      "-country": "B",
+      "name": "Brabant",
+      "area": "3358",
+      "population": "2253794",
+      "city": {
+        "-id": "cty-Belgium-Brussels",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-3",
+        "name": "Brussels",
+        "longitude": "4.35",
+        "latitude": "50.85",
+        "population": {
+          "-year": "95",
+          "#text": "951580"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-4",
+      "-capital": "cty-cid-cia-Belgium-4-0",
+      "-country": "B",
+      "name": "East Flanders",
+      "area": "2982",
+      "population": "1340056",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-4-0",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-4",
+        "name": "Ghent",
+        "longitude": "3.75",
+        "latitude": "51.054",
+        "population": {
+          "-year": "95",
+          "#text": "227483"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-5",
+      "-capital": "cty-cid-cia-Belgium-Mons",
+      "-country": "B",
+      "name": "Hainaut",
+      "area": "3787",
+      "population": "1283252",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Belgium-5-0",
+          "-country": "B",
+          "-province": "prov-cid-cia-Belgium-5",
+          "name": "Charleroi",
+          "population": {
+            "-year": "95",
+            "#text": "206491"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Belgium-Mons",
+          "-is_state_cap": "yes",
+          "-country": "B",
+          "-province": "prov-cid-cia-Belgium-5",
+          "name": "Mons",
+          "longitude": "4.5",
+          "latitude": "50.4",
+          "population": {
+            "-year": "87",
+            "#text": "90720"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-6",
+      "-capital": "cty-cid-cia-Belgium-6-0",
+      "-country": "B",
+      "name": "Liege",
+      "area": "3862",
+      "population": "1006081",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-6-0",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-6",
+        "name": "Liege",
+        "longitude": "5.6",
+        "latitude": "50.5",
+        "population": {
+          "-year": "95",
+          "#text": "192393"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Maas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-7",
+      "-capital": "cty-cid-cia-Belgium-Hasselt",
+      "-country": "B",
+      "name": "Limburg",
+      "area": "2422",
+      "population": "755593",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-Hasselt",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-7",
+        "name": "Hasselt",
+        "longitude": "5.2",
+        "latitude": "50.9",
+        "population": {
+          "-year": "87",
+          "#text": "65348"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-8",
+      "-capital": "cty-cid-cia-Belgium-Arlon",
+      "-country": "B",
+      "name": "Luxembourg",
+      "area": "4441",
+      "population": "234664",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-Arlon",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-8",
+        "name": "Arlon",
+        "longitude": "5.8",
+        "latitude": "49.7",
+        "population": {
+          "-year": "87",
+          "#text": "23150"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-9",
+      "-capital": "cty-cid-cia-Belgium-8-0",
+      "-country": "B",
+      "name": "Namur",
+      "area": "3665",
+      "population": "426305",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-8-0",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-9",
+        "name": "Namur",
+        "longitude": "4.9",
+        "latitude": "50.4",
+        "population": {
+          "-year": "95",
+          "#text": "105014"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Maas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Belgium-10",
+      "-capital": "cty-cid-cia-Belgium-7-0",
+      "-country": "B",
+      "name": "West Flanders",
+      "area": "3134",
+      "population": "1111557",
+      "city": {
+        "-id": "cty-cid-cia-Belgium-7-0",
+        "-is_state_cap": "yes",
+        "-country": "B",
+        "-province": "prov-cid-cia-Belgium-10",
+        "name": "Brugge",
+        "longitude": "3.2",
+        "latitude": "51.1",
+        "population": [
+          {
+            "-year": "87",
+            "#text": "117799"
+          },
+          {
+            "-year": "95",
+            "#text": "116273"
+          }
+        ]
+      }
+    }
+  ]
+}
+{
+  "-car_code": "L",
+  "-area": "2586",
+  "-capital": "cty-cid-cia-Luxembourg-Luxembourg",
+  "-memberships": "org-ACCT org-Benelux org-CE org-CCC org-ECE org-EBRD org-EIB org-EU org-FAO org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WEU org-WCL org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Luxembourg",
+  "population": "415870",
+  "population_growth": "1.57",
+  "infant_mortality": "4.7",
+  "gdp_total": "10000",
+  "gdp_agri": "1.4",
+  "gdp_ind": "33.7",
+  "gdp_serv": "64.9",
+  "inflation": "3.6",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "97",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Jewish"
+    }
+  ],
+  "border": [
+    {
+      "-country": "F",
+      "-length": "73"
+    },
+    {
+      "-country": "D",
+      "-length": "138"
+    },
+    {
+      "-country": "B",
+      "-length": "148"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Luxembourg-Luxembourg",
+    "-is_country_cap": "yes",
+    "-country": "L",
+    "name": "Luxembourg",
+    "longitude": "6.08",
+    "latitude": "49.4",
+    "population": {
+      "-year": "87",
+      "#text": "76600"
+    }
+  }
+}
+{
+  "-car_code": "NL",
+  "-area": "37330",
+  "-capital": "cty-Netherlands-Amsterdam",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-Benelux org-CE org-CCC org-ESCAP org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNMIH org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UNU org-UPU org-WEU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Netherlands",
+  "population": "15568034",
+  "population_growth": "0.56",
+  "infant_mortality": "4.9",
+  "gdp_total": "301900",
+  "gdp_agri": "3.4",
+  "gdp_ind": "26.9",
+  "gdp_serv": "69.7",
+  "inflation": "2.25",
+  "indep_date": "1579-01-01",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "96",
+    "#text": "Dutch"
+  },
+  "religions": [
+    {
+      "-percentage": "3",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "34",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Dutch"
+  },
+  "border": [
+    {
+      "-country": "D",
+      "-length": "577"
+    },
+    {
+      "-country": "B",
+      "-length": "450"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Netherlands-2",
+      "-capital": "cty-cid-cia-Netherlands-14",
+      "-country": "NL",
+      "name": "Groningen",
+      "area": "2344",
+      "population": "557995",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-14",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-2",
+        "name": "Groningen",
+        "population": {
+          "-year": "95",
+          "#text": "210708"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-3",
+      "-capital": "cty-cid-cia-Netherlands-Leeuwarden",
+      "-country": "NL",
+      "name": "Friesland",
+      "area": "3361",
+      "population": "609579",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-Leeuwarden",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-3",
+        "name": "Leeuwarden"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-4",
+      "-capital": "cty-cid-cia-Netherlands-Assen",
+      "-country": "NL",
+      "name": "Drenthe",
+      "area": "2652",
+      "population": "454864",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-Assen",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-4",
+        "name": "Assen"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-5",
+      "-capital": "cty-cid-cia-Netherlands-Zwolle",
+      "-country": "NL",
+      "name": "Overijssel",
+      "area": "3337",
+      "population": "1050389",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Netherlands-9",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-5",
+          "name": "Enschede",
+          "population": {
+            "-year": "95",
+            "#text": "254480"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-Zwolle",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-5",
+          "name": "Zwolle"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-6",
+      "-capital": "cty-cid-cia-Netherlands-Lelystad",
+      "-country": "NL",
+      "name": "Flevoland",
+      "area": "1425",
+      "population": "262325",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-Lelystad",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-6",
+        "name": "Lelystad"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-7",
+      "-capital": "cty-cid-cia-Netherlands-7",
+      "-country": "NL",
+      "name": "Gelderland",
+      "area": "4995",
+      "population": "1864732",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Netherlands-7",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-7",
+          "name": "Arnhem",
+          "population": {
+            "-year": "95",
+            "#text": "314159"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-10",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-7",
+          "name": "Nijmegen",
+          "population": {
+            "-year": "95",
+            "#text": "249490"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-8",
+      "-capital": "cty-cid-cia-Netherlands-5",
+      "-country": "NL",
+      "name": "Utrecht",
+      "area": "1356",
+      "population": "1063460",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-5",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-8",
+        "name": "Utrecht",
+        "population": {
+          "-year": "95",
+          "#text": "547070"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-9",
+      "-capital": "cty-cid-cia-Netherlands-13",
+      "-country": "NL",
+      "name": "Noord Holland",
+      "area": "265978",
+      "population": "2463611",
+      "city": [
+        {
+          "-id": "cty-Netherlands-Amsterdam",
+          "-is_country_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-9",
+          "name": "Amsterdam",
+          "longitude": "4.91667",
+          "latitude": "52.3833",
+          "population": {
+            "-year": "95",
+            "#text": "1101407"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-13",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-9",
+          "name": "Haarlem",
+          "population": {
+            "-year": "95",
+            "#text": "212631"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-20",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-9",
+          "name": "Zaanstreek",
+          "population": {
+            "-year": "95",
+            "#text": "147917"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-21",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-9",
+          "name": "Velsen",
+          "population": {
+            "-year": "95",
+            "#text": "134973"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-22",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-9",
+          "name": "Hilversum",
+          "population": {
+            "-year": "95",
+            "#text": "102023"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-10",
+      "-capital": "cty-cid-cia-Netherlands-4",
+      "-country": "NL",
+      "name": "Zuid Holland",
+      "area": "2859",
+      "population": "3325064",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Netherlands-4",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-10",
+          "name": "s Gravenhage",
+          "population": {
+            "-year": "95",
+            "#text": "694249"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-3",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-10",
+          "name": "Rotterdam",
+          "population": {
+            "-year": "95",
+            "#text": "1078747"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rhein"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-12",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-10",
+          "name": "Dordrecht",
+          "population": {
+            "-year": "95",
+            "#text": "213963"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-16",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-10",
+          "name": "Leiden",
+          "population": {
+            "-year": "95",
+            "#text": "194935"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-11",
+      "-capital": "cty-cid-cia-Netherlands-Middelburg",
+      "-country": "NL",
+      "name": "Zeeland",
+      "area": "1791",
+      "population": "365846",
+      "city": {
+        "-id": "cty-cid-cia-Netherlands-Middelburg",
+        "-is_state_cap": "yes",
+        "-country": "NL",
+        "-province": "prov-cid-cia-Netherlands-11",
+        "name": "Middelburg"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-12",
+      "-capital": "cty-cid-cia-Netherlands-15",
+      "-country": "NL",
+      "name": "Noord Brabant",
+      "area": "4938",
+      "population": "2276207",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Netherlands-15",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-12",
+          "name": "s Hertogenbosch",
+          "population": {
+            "-year": "95",
+            "#text": "199127"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-6",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-12",
+          "name": "Eindhoven",
+          "population": {
+            "-year": "95",
+            "#text": "395612"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-11",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-12",
+          "name": "Tilburg",
+          "population": {
+            "-year": "95",
+            "#text": "237958"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-18",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-12",
+          "name": "Breda",
+          "population": {
+            "-year": "95",
+            "#text": "166616"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Netherlands-13",
+      "-capital": "cty-cid-cia-Netherlands-19",
+      "-country": "NL",
+      "name": "Limburg",
+      "area": "2167",
+      "population": "1130050",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Netherlands-19",
+          "-is_state_cap": "yes",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-13",
+          "name": "Maastricht",
+          "population": {
+            "-year": "95",
+            "#text": "164701"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Maas"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-8",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-13",
+          "name": "Heerlen",
+          "population": {
+            "-year": "95",
+            "#text": "270952"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Netherlands-17",
+          "-country": "NL",
+          "-province": "prov-cid-cia-Netherlands-13",
+          "name": "Geleen",
+          "population": {
+            "-year": "95",
+            "#text": "186011"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "BIH",
+  "-area": "51233",
+  "-capital": "cty-cid-cia-Bosnia-and-Herzegovina-Sarajevo",
+  "-memberships": "org-CEI org-CE org-ECE org-FAO org-ICAO org-Interpol org-IFAD org-ILO org-IMO org-IOC org-IOM org-ITU org-Intelsat org-NAM org-OSCE org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Bosnia and Herzegovina",
+  "population": "2656240",
+  "population_growth": "-2.84",
+  "infant_mortality": "43.2",
+  "gdp_total": "1000",
+  "indep_date": "1992-04-01",
+  "government": "emerging democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "40",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "38",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Croat"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "40",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "31",
+      "#text": "Christian Orthodox"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "languages": {
+    "-percentage": "99",
+    "#text": "Serbo-Croatian"
+  },
+  "border": [
+    {
+      "-country": "SRB",
+      "-length": "302"
+    },
+    {
+      "-country": "KOS",
+      "-length": "225"
+    },
+    {
+      "-country": "HR",
+      "-length": "932"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Bosnia-and-Herzegovina-Sarajevo",
+    "-is_country_cap": "yes",
+    "-country": "BIH",
+    "name": "Sarajevo",
+    "longitude": "18.4",
+    "latitude": "43.9"
+  }
+}
+{
+  "-car_code": "HR",
+  "-area": "56538",
+  "-capital": "cty-cid-cia-Croatia-Zagreb",
+  "-memberships": "org-CEI org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-IADB org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OSCE org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Croatia",
+  "population": "5004112",
+  "population_growth": "0.58",
+  "infant_mortality": "10.2",
+  "gdp_total": "20100",
+  "gdp_agri": "12.7",
+  "gdp_ind": "30.6",
+  "gdp_serv": "56.7",
+  "inflation": "3.7",
+  "indep_date": "1991-06-25",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "12",
+      "#text": "Serb"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "0.9",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "78",
+      "#text": "Croat"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Slovene"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "0.4",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "11.1",
+      "#text": "Christian Orthodox"
+    },
+    {
+      "-percentage": "76.5",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "1.2",
+      "#text": "Muslim"
+    }
+  ],
+  "languages": {
+    "-percentage": "96",
+    "#text": "Serbo-Croatian"
+  },
+  "border": [
+    {
+      "-country": "SRB",
+      "-length": "241"
+    },
+    {
+      "-country": "MNE",
+      "-length": "25"
+    },
+    {
+      "-country": "H",
+      "-length": "329"
+    },
+    {
+      "-country": "SLO",
+      "-length": "546"
+    },
+    {
+      "-country": "BIH",
+      "-length": "932"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Croatia-Zagreb",
+    "-is_country_cap": "yes",
+    "-country": "HR",
+    "name": "Zagreb",
+    "longitude": "16",
+    "latitude": "45.8",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Save"
+    }
+  }
+}
+{
+  "-car_code": "BG",
+  "-area": "110910",
+  "-capital": "cty-Bulgaria-Sofia",
+  "-memberships": "org-ACCT org-BIS org-BSEC org-CEI org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-G-9 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-ANC org-NSG org-OSCE org-PFP org-PCA org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNMOT org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Bulgaria",
+  "population": "8612757",
+  "population_growth": "0.46",
+  "infant_mortality": "15.7",
+  "gdp_total": "43200",
+  "gdp_agri": "12",
+  "gdp_ind": "36",
+  "gdp_serv": "52",
+  "inflation": "35",
+  "indep_date": "1908-09-22",
+  "government": "emerging democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2.5",
+      "#text": "Macedonian"
+    },
+    {
+      "-percentage": "2.6",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "8.5",
+      "#text": "Turk"
+    },
+    {
+      "-percentage": "85.3",
+      "#text": "Bulgarian"
+    },
+    {
+      "-percentage": "0.3",
+      "#text": "Armenian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "13",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "0.8",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "85",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Bulgarian"
+  },
+  "border": [
+    {
+      "-country": "GR",
+      "-length": "494"
+    },
+    {
+      "-country": "MK",
+      "-length": "148"
+    },
+    {
+      "-country": "SRB",
+      "-length": "318"
+    },
+    {
+      "-country": "RO",
+      "-length": "608"
+    },
+    {
+      "-country": "TR",
+      "-length": "240"
+    }
+  ],
+  "city": {
+    "-id": "cty-Bulgaria-Sofia",
+    "-is_country_cap": "yes",
+    "-country": "BG",
+    "name": "Sofia",
+    "longitude": "23.3333",
+    "latitude": "42.7",
+    "population": {
+      "-year": "87",
+      "#text": "1300000"
+    }
+  }
+}
+{
+  "-car_code": "RO",
+  "-area": "237500",
+  "-capital": "cty-cid-cia-Romania-2",
+  "-memberships": "org-ACCT org-BIS org-BSEC org-CEI org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-G-9 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-ANC org-NSG org-OSCE org-OAS org-PFP org-PCA org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNIKOM org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Romania",
+  "population": "21657162",
+  "population_growth": "-1.21",
+  "infant_mortality": "23.2",
+  "gdp_total": "105700",
+  "gdp_agri": "19.6",
+  "gdp_ind": "36.3",
+  "gdp_serv": "44.1",
+  "inflation": "25",
+  "indep_date": "1881-01-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.4",
+      "#text": "German"
+    },
+    {
+      "-percentage": "1.6",
+      "#text": "Roma"
+    },
+    {
+      "-percentage": "8.9",
+      "#text": "Hungarian"
+    },
+    {
+      "-percentage": "89.1",
+      "#text": "Romanian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "6",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "70",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "SRB",
+      "-length": "476"
+    },
+    {
+      "-country": "H",
+      "-length": "443"
+    },
+    {
+      "-country": "UA",
+      "-length": "531"
+    },
+    {
+      "-country": "BG",
+      "-length": "608"
+    },
+    {
+      "-country": "MD",
+      "-length": "450"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Romania-2",
+      "-capital": "cty-cid-cia-Romania-Alba-Iulia",
+      "-country": "RO",
+      "name": "Alba",
+      "area": "6231",
+      "population": "428000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Alba-Iulia",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-2",
+        "name": "Alba Iulia"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-3",
+      "-capital": "cty-cid-cia-Romania-12",
+      "-country": "RO",
+      "name": "Arad",
+      "area": "7652",
+      "population": "507000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-12",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-3",
+        "name": "Arad",
+        "population": {
+          "-year": "89",
+          "#text": "191000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-4",
+      "-capital": "cty-cid-cia-Romania-16",
+      "-country": "RO",
+      "name": "Arges",
+      "area": "6801",
+      "population": "678000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-16",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-4",
+        "name": "Pitesti",
+        "population": {
+          "-year": "89",
+          "#text": "162000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-5",
+      "-capital": "cty-cid-cia-Romania-13",
+      "-country": "RO",
+      "name": "Bacau",
+      "area": "6606",
+      "population": "731000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-13",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-5",
+        "name": "Bacau",
+        "population": {
+          "-year": "89",
+          "#text": "193000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-6",
+      "-capital": "cty-cid-cia-Romania-11",
+      "-country": "RO",
+      "name": "Bihor",
+      "area": "7535",
+      "population": "660000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-11",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-6",
+        "name": "Oradea",
+        "population": {
+          "-year": "89",
+          "#text": "225000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-7",
+      "-capital": "cty-cid-cia-Romania-Bistrita",
+      "-country": "RO",
+      "name": "Bistrita Nasaud",
+      "area": "5305",
+      "population": "328000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Bistrita",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-7",
+        "name": "Bistrita"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-8",
+      "-capital": "cty-cid-cia-Romania-Botosani",
+      "-country": "RO",
+      "name": "Botosani",
+      "area": "4965",
+      "population": "468000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Botosani",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-8",
+        "name": "Botosani"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-9",
+      "-capital": "cty-cid-cia-Romania-10",
+      "-country": "RO",
+      "name": "Braila",
+      "area": "4724",
+      "population": "404000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-10",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-9",
+        "name": "Braila",
+        "population": {
+          "-year": "89",
+          "#text": "243000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Donau"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-10",
+      "-capital": "cty-cid-cia-Romania-3",
+      "-country": "RO",
+      "name": "Brasov",
+      "area": "5351",
+      "population": "695000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-3",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-10",
+        "name": "Brasov",
+        "population": {
+          "-year": "89",
+          "#text": "353000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-11",
+      "-capital": "cty-cid-cia-Romania-2",
+      "-country": "RO",
+      "name": "Bucuresti",
+      "area": "1521",
+      "population": "2319000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-11",
+        "name": "Bucharest",
+        "longitude": "26.1",
+        "latitude": "44.4167",
+        "population": {
+          "-year": "89",
+          "#text": "2037000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-12",
+      "-capital": "cty-cid-cia-Romania-18",
+      "-country": "RO",
+      "name": "Buzau",
+      "area": "6072",
+      "population": "524000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-18",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-12",
+        "name": "Buzau",
+        "population": {
+          "-year": "89",
+          "#text": "145000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-13",
+      "-capital": "cty-cid-cia-Romania-Calarasi",
+      "-country": "RO",
+      "name": "Calarasi",
+      "area": "5075",
+      "population": "351000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Calarasi",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-13",
+        "name": "Calarasi"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-14",
+      "-capital": "cty-cid-cia-Romania-Resita",
+      "-country": "RO",
+      "name": "Caras Severin",
+      "area": "8503",
+      "population": "408000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Resita",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-14",
+        "name": "Resita"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-15",
+      "-capital": "cty-cid-cia-Romania-7",
+      "-country": "RO",
+      "name": "Cluj",
+      "area": "6650",
+      "population": "743000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-7",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-15",
+        "name": "Cluj Napoca",
+        "population": {
+          "-year": "89",
+          "#text": "318000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-16",
+      "-capital": "cty-cid-cia-Romania-4",
+      "-country": "RO",
+      "name": "Constanta",
+      "area": "7055",
+      "population": "737000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-4",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-16",
+        "name": "Constanta",
+        "population": {
+          "-year": "89",
+          "#text": "316000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-17",
+      "-capital": "cty-cid-cia-Romania-Sfintu-Gheorghe",
+      "-country": "RO",
+      "name": "Covasna",
+      "area": "3705",
+      "population": "238000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Sfintu-Gheorghe",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-17",
+        "name": "Sfintu Gheorghe",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Olt"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-18",
+      "-capital": "cty-cid-cia-Romania-Tirgoviste",
+      "-country": "RO",
+      "name": "Dimbovita",
+      "area": "4035",
+      "population": "570000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Tirgoviste",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-18",
+        "name": "Tirgoviste"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-19",
+      "-capital": "cty-cid-cia-Romania-9",
+      "-country": "RO",
+      "name": "Dolj",
+      "area": "7413",
+      "population": "772000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-9",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-19",
+        "name": "Craiova",
+        "population": {
+          "-year": "89",
+          "#text": "300000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-20",
+      "-capital": "cty-cid-cia-Romania-8",
+      "-country": "RO",
+      "name": "Galati",
+      "area": "4425",
+      "population": "642000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-8",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-20",
+        "name": "Galati",
+        "population": {
+          "-year": "89",
+          "#text": "307000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Donau"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-21",
+      "-capital": "cty-cid-cia-Romania-Giurgiu",
+      "-country": "RO",
+      "name": "Giurgiu",
+      "area": "3810",
+      "population": "325000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Giurgiu",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-21",
+        "name": "Giurgiu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-22",
+      "-capital": "cty-cid-cia-Romania-Tirgu-Jiu",
+      "-country": "RO",
+      "name": "Gorj",
+      "area": "5641",
+      "population": "388000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Tirgu-Jiu",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-22",
+        "name": "Tirgu Jiu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-23",
+      "-capital": "cty-cid-cia-Romania-Miercurea-Ciuc",
+      "-country": "RO",
+      "name": "Harghita",
+      "area": "6610",
+      "population": "363000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Miercurea-Ciuc",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-23",
+        "name": "Miercurea Ciuc",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Olt"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-24",
+      "-capital": "cty-cid-cia-Romania-Deva",
+      "-country": "RO",
+      "name": "Hunedoara",
+      "area": "7016",
+      "population": "567000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Deva",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-24",
+        "name": "Deva"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-25",
+      "-capital": "cty-cid-cia-Romania-Slobozia",
+      "-country": "RO",
+      "name": "Ialomita",
+      "area": "4449",
+      "population": "309000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Slobozia",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-25",
+        "name": "Slobozia"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-26",
+      "-capital": "cty-cid-cia-Romania-6",
+      "-country": "RO",
+      "name": "Iasi",
+      "area": "5469",
+      "population": "810000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-6",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-26",
+        "name": "Iasi",
+        "population": {
+          "-year": "89",
+          "#text": "330000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-27",
+      "-capital": "cty-cid-cia-Romania-17",
+      "-country": "RO",
+      "name": "Maramures",
+      "area": "6215",
+      "population": "556000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-17",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-27",
+        "name": "Baia Mare",
+        "population": {
+          "-year": "89",
+          "#text": "150000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-28",
+      "-capital": "cty-cid-cia-Romania-Drobeta-Turnu-Severin",
+      "-country": "RO",
+      "name": "Mehedinti",
+      "area": "4900",
+      "population": "329000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Drobeta-Turnu-Severin",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-28",
+        "name": "Drobeta Turnu Severin"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-29",
+      "-capital": "cty-cid-cia-Romania-15",
+      "-country": "RO",
+      "name": "Mures",
+      "area": "6696",
+      "population": "621000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-15",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-29",
+        "name": "Tirgu Mures",
+        "population": {
+          "-year": "89",
+          "#text": "165000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-30",
+      "-capital": "cty-cid-cia-Romania-Piatra-Neamt",
+      "-country": "RO",
+      "name": "Neamt",
+      "area": "5890",
+      "population": "580000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Piatra-Neamt",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-30",
+        "name": "Piatra Neamt"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-31",
+      "-capital": "cty-cid-cia-Romania-Slatina",
+      "-country": "RO",
+      "name": "Olt",
+      "area": "5507",
+      "population": "535000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Slatina",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-31",
+        "name": "Slatina",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Olt"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-32",
+      "-capital": "cty-cid-cia-Romania-Ploiesti",
+      "-country": "RO",
+      "name": "Prahova",
+      "area": "4694",
+      "population": "877000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Ploiesti",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-32",
+        "name": "Ploiesti"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-33",
+      "-capital": "cty-cid-cia-Romania-Zalau",
+      "-country": "RO",
+      "name": "Salaj",
+      "area": "3850",
+      "population": "269000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Zalau",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-33",
+        "name": "Zalau"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-34",
+      "-capital": "cty-cid-cia-Romania-19",
+      "-country": "RO",
+      "name": "Satu Mare",
+      "area": "4405",
+      "population": "417000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-19",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-34",
+        "name": "Satu Mare",
+        "population": {
+          "-year": "89",
+          "#text": "137000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-35",
+      "-capital": "cty-cid-cia-Romania-14",
+      "-country": "RO",
+      "name": "Sibiu",
+      "area": "5422",
+      "population": "509000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-14",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-35",
+        "name": "Sibiu",
+        "population": {
+          "-year": "89",
+          "#text": "184000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-36",
+      "-capital": "cty-cid-cia-Romania-Suceava",
+      "-country": "RO",
+      "name": "Suceava",
+      "area": "8555",
+      "population": "699000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Suceava",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-36",
+        "name": "Suceava"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-37",
+      "-capital": "cty-cid-cia-Romania-Alexandria",
+      "-country": "RO",
+      "name": "Teleorman",
+      "area": "5760",
+      "population": "504000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Alexandria",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-37",
+        "name": "Alexandria"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-38",
+      "-capital": "cty-cid-cia-Romania-5",
+      "-country": "RO",
+      "name": "Timis",
+      "area": "8692",
+      "population": "726000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-5",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-38",
+        "name": "Timisoara",
+        "population": {
+          "-year": "89",
+          "#text": "333000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-39",
+      "-capital": "cty-cid-cia-Romania-Tulcea",
+      "-country": "RO",
+      "name": "Tulcea",
+      "area": "8430",
+      "population": "275000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Tulcea",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-39",
+        "name": "Tulcea"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-40",
+      "-capital": "cty-cid-cia-Romania-Vaslui",
+      "-country": "RO",
+      "name": "Vaslui",
+      "area": "5297",
+      "population": "468000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Vaslui",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-40",
+        "name": "Vaslui"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-41",
+      "-capital": "cty-cid-cia-Romania-Rimnicu-Vilcea",
+      "-country": "RO",
+      "name": "Vilcea",
+      "area": "5705",
+      "population": "430000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Rimnicu-Vilcea",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-41",
+        "name": "Rimnicu Vilcea"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Romania-42",
+      "-capital": "cty-cid-cia-Romania-Focsani",
+      "-country": "RO",
+      "name": "Vrancea",
+      "area": "4863",
+      "population": "394000",
+      "city": {
+        "-id": "cty-cid-cia-Romania-Focsani",
+        "-is_state_cap": "yes",
+        "-country": "RO",
+        "-province": "prov-cid-cia-Romania-42",
+        "name": "Focsani"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "TR",
+  "-area": "780580",
+  "-capital": "cty-Turkey-Ankara",
+  "-memberships": "org-AsDB org-BIS org-BSEC org-CE org-CCC org-ECE org-ECO org-EBRD org-EU org-CERN org-FAO org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-ANC org-NATO org-EN org-OECD org-OSCE org-OIC org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-UNOMIG org-UNHCR org-UNRWA org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Turkey",
+  "population": "62484478",
+  "population_growth": "1.67",
+  "infant_mortality": "43.2",
+  "gdp_total": "345700",
+  "gdp_agri": "15.5",
+  "gdp_ind": "33.2",
+  "gdp_serv": "51.3",
+  "inflation": "94",
+  "indep_date": "1923-10-29",
+  "government": "republican parliamentary democracy",
+  "encompassed": [
+    {
+      "-continent": "europe",
+      "-percentage": "5"
+    },
+    {
+      "-continent": "asia",
+      "-percentage": "95"
+    }
+  ],
+  "ethnicgroups": [
+    {
+      "-percentage": "80",
+      "#text": "Turkish"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Kurdish"
+    }
+  ],
+  "religions": {
+    "-percentage": "99.8",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "GR",
+      "-length": "206"
+    },
+    {
+      "-country": "BG",
+      "-length": "240"
+    },
+    {
+      "-country": "IR",
+      "-length": "499"
+    },
+    {
+      "-country": "ARM",
+      "-length": "268"
+    },
+    {
+      "-country": "GE",
+      "-length": "252"
+    },
+    {
+      "-country": "AZ",
+      "-length": "9"
+    },
+    {
+      "-country": "IRQ",
+      "-length": "331"
+    },
+    {
+      "-country": "SYR",
+      "-length": "822"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Turkey-2",
+      "-capital": "cty-Turkey-Adana",
+      "-country": "TR",
+      "name": "Adana",
+      "area": "17253",
+      "population": "1934907",
+      "city": [
+        {
+          "-id": "cty-Turkey-Adana",
+          "-is_state_cap": "yes",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-2",
+          "name": "Adana",
+          "longitude": "35.3",
+          "latitude": "36.9833",
+          "population": {
+            "-year": "94",
+            "#text": "1047300"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-36",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-2",
+          "name": "Osmaniye",
+          "population": {
+            "-year": "94",
+            "#text": "138000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-3",
+      "-capital": "cty-cid-cia-Turkey-40",
+      "-country": "TR",
+      "name": "Adiyaman",
+      "area": "7614",
+      "population": "513131",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-40",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-3",
+        "name": "Adiyaman",
+        "population": {
+          "-year": "94",
+          "#text": "128000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-4",
+      "-capital": "cty-cid-cia-Turkey-Afyon",
+      "-country": "TR",
+      "name": "Afyon",
+      "area": "14230",
+      "population": "739223",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Afyon",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-4",
+        "name": "Afyon"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-5",
+      "-capital": "cty-cid-cia-Turkey-Agri",
+      "-country": "TR",
+      "name": "Agri",
+      "area": "11376",
+      "population": "437093",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Agri",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-5",
+        "name": "Agri"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-6",
+      "-capital": "cty-cid-cia-Turkey-Aksaray",
+      "-country": "TR",
+      "name": "Aksaray",
+      "area": "7626",
+      "population": "326399",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Aksaray",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-6",
+        "name": "Aksaray"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-7",
+      "-capital": "cty-cid-cia-Turkey-Amasya",
+      "-country": "TR",
+      "name": "Amasya",
+      "area": "5520",
+      "population": "357191",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Amasya",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-7",
+        "name": "Amasya"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-8",
+      "-capital": "cty-Turkey-Ankara",
+      "-country": "TR",
+      "name": "Ankara",
+      "area": "25706",
+      "population": "3236626",
+      "city": {
+        "-id": "cty-Turkey-Ankara",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-8",
+        "name": "Ankara",
+        "longitude": "32.8833",
+        "latitude": "39.95",
+        "population": {
+          "-year": "94",
+          "#text": "2782200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-9",
+      "-capital": "cty-cid-cia-Turkey-10",
+      "-country": "TR",
+      "name": "Antalya",
+      "area": "20591",
+      "population": "1132211",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-10",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-9",
+        "name": "Antalya",
+        "population": {
+          "-year": "94",
+          "#text": "497200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-10",
+      "-capital": "cty-cid-cia-Turkey-Artvin",
+      "-country": "TR",
+      "name": "Artvin",
+      "area": "7436",
+      "population": "212833",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Artvin",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-10",
+        "name": "Artvin"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-11",
+      "-capital": "cty-cid-cia-Turkey-42",
+      "-country": "TR",
+      "name": "Aydin",
+      "area": "8007",
+      "population": "824816",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-42",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-11",
+        "name": "Aydin",
+        "population": {
+          "-year": "94",
+          "#text": "121200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-12",
+      "-capital": "cty-cid-cia-Turkey-28",
+      "-country": "TR",
+      "name": "Balikesir",
+      "area": "14292",
+      "population": "973314",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-28",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-12",
+        "name": "Balikesir",
+        "population": {
+          "-year": "94",
+          "#text": "187600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-13",
+      "-capital": "cty-cid-cia-Turkey-31",
+      "-country": "TR",
+      "name": "Batman",
+      "area": "4694",
+      "population": "344669",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-31",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-13",
+        "name": "Batman",
+        "population": {
+          "-year": "94",
+          "#text": "182800"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-14",
+      "-capital": "cty-cid-cia-Turkey-Bayburt",
+      "-country": "TR",
+      "name": "Bayburt",
+      "area": "3652",
+      "population": "107330",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Bayburt",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-14",
+        "name": "Bayburt"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-15",
+      "-capital": "cty-cid-cia-Turkey-Bilecik",
+      "-country": "TR",
+      "name": "Bilecik",
+      "area": "4307",
+      "population": "175526",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Bilecik",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-15",
+        "name": "Bilecik"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-16",
+      "-capital": "cty-cid-cia-Turkey-Bingol",
+      "-country": "TR",
+      "name": "Bingol",
+      "area": "8125",
+      "population": "250966",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Bingol",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-16",
+        "name": "Bingol"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-17",
+      "-capital": "cty-cid-cia-Turkey-Bitlis",
+      "-country": "TR",
+      "name": "Bitlis",
+      "area": "6707",
+      "population": "330115",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Bitlis",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-17",
+        "name": "Bitlis"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-18",
+      "-capital": "cty-cid-cia-Turkey-Bolu",
+      "-country": "TR",
+      "name": "Bolu",
+      "area": "11051",
+      "population": "536869",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Bolu",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-18",
+        "name": "Bolu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-19",
+      "-capital": "cty-cid-cia-Turkey-Burdur",
+      "-country": "TR",
+      "name": "Burdur",
+      "area": "6887",
+      "population": "254899",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Burdur",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-19",
+        "name": "Burdur"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-20",
+      "-capital": "cty-cid-cia-Turkey-6",
+      "-country": "TR",
+      "name": "Bursa",
+      "area": "11043",
+      "population": "1603137",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-6",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-20",
+        "name": "Bursa",
+        "population": {
+          "-year": "94",
+          "#text": "996600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-21",
+      "-capital": "cty-cid-cia-Turkey-Canakkale",
+      "-country": "TR",
+      "name": "Canakkale",
+      "area": "9737",
+      "population": "432263",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Canakkale",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-21",
+        "name": "Canakkale"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-22",
+      "-capital": "cty-cid-cia-Turkey-Cankiri",
+      "-country": "TR",
+      "name": "Cankiri",
+      "area": "8454",
+      "population": "279129",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Cankiri",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-22",
+        "name": "Cankiri"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-23",
+      "-capital": "cty-cid-cia-Turkey-Corum",
+      "-country": "TR",
+      "name": "Corum",
+      "area": "12820",
+      "population": "609863",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Corum",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-23",
+        "name": "Corum"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-24",
+      "-capital": "cty-cid-cia-Turkey-24",
+      "-country": "TR",
+      "name": "Denizli",
+      "area": "11868",
+      "population": "750882",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-24",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-24",
+        "name": "Denizli",
+        "population": {
+          "-year": "94",
+          "#text": "234500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-25",
+      "-capital": "cty-cid-cia-Turkey-13",
+      "-country": "TR",
+      "name": "Diyarbakir",
+      "area": "15355",
+      "population": "1094996",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-13",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-25",
+        "name": "Diyarbakir",
+        "population": {
+          "-year": "94",
+          "#text": "448300"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-26",
+      "-capital": "cty-cid-cia-Turkey-45",
+      "-country": "TR",
+      "name": "Edirne",
+      "area": "6276",
+      "population": "404599",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-45",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-26",
+        "name": "Edirne",
+        "population": {
+          "-year": "94",
+          "#text": "115500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-27",
+      "-capital": "cty-cid-cia-Turkey-26",
+      "-country": "TR",
+      "name": "Elazig",
+      "area": "9153",
+      "population": "498225",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-26",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-27",
+        "name": "Elazig",
+        "population": {
+          "-year": "94",
+          "#text": "222800"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-28",
+      "-capital": "cty-cid-cia-Turkey-Erzincan",
+      "-country": "TR",
+      "name": "Erzincan",
+      "area": "11903",
+      "population": "299251",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Erzincan",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-28",
+        "name": "Erzincan",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Karasu"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-29",
+      "-capital": "cty-cid-cia-Turkey-20",
+      "-country": "TR",
+      "name": "Erzurum",
+      "area": "25066",
+      "population": "848201",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-20",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-29",
+        "name": "Erzurum",
+        "longitude": "41.3",
+        "latitude": "49.9",
+        "population": {
+          "-year": "94",
+          "#text": "250100"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Karasu"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-30",
+      "-capital": "cty-cid-cia-Turkey-12",
+      "-country": "TR",
+      "name": "Eskisehir",
+      "area": "13652",
+      "population": "641057",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-12",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-30",
+        "name": "Eskisehir",
+        "population": {
+          "-year": "94",
+          "#text": "451000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-31",
+      "-capital": "cty-cid-cia-Turkey-7",
+      "-country": "TR",
+      "name": "Gaziantep",
+      "area": "7642",
+      "population": "1140594",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-7",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-31",
+        "name": "Gaziantep",
+        "population": {
+          "-year": "94",
+          "#text": "716000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-32",
+      "-capital": "cty-cid-cia-Turkey-Giresun",
+      "-country": "TR",
+      "name": "Giresun",
+      "area": "6934",
+      "population": "499087",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Giresun",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-32",
+        "name": "Giresun"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-33",
+      "-capital": "cty-cid-cia-Turkey-Gumushane",
+      "-country": "TR",
+      "name": "Gumushane",
+      "area": "6575",
+      "population": "169375",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Gumushane",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-33",
+        "name": "Gumushane"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-34",
+      "-capital": "cty-cid-cia-Turkey-Hakkari",
+      "-country": "TR",
+      "name": "Hakkari",
+      "area": "7121",
+      "population": "172479",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Hakkari",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-34",
+        "name": "Hakkari"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-35",
+      "-capital": "cty-cid-cia-Turkey-37",
+      "-country": "TR",
+      "name": "Hatay",
+      "area": "5403",
+      "population": "1109754",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Turkey-33",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-35",
+          "name": "Iskenderun",
+          "population": {
+            "-year": "94",
+            "#text": "156800"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-37",
+          "-is_state_cap": "yes",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-35",
+          "name": "Antakya",
+          "population": {
+            "-year": "94",
+            "#text": "137200"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-36",
+      "-capital": "cty-cid-cia-Turkey-9",
+      "-country": "TR",
+      "name": "Icel",
+      "area": "15853",
+      "population": "1266995",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Turkey-9",
+          "-is_state_cap": "yes",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-36",
+          "name": "Mersin",
+          "population": {
+            "-year": "94",
+            "#text": "523000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-25",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-36",
+          "name": "Tarsus",
+          "population": {
+            "-year": "94",
+            "#text": "225000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-37",
+      "-capital": "cty-cid-cia-Turkey-39",
+      "-country": "TR",
+      "name": "Isparta",
+      "area": "8933",
+      "population": "434771",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-39",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-37",
+        "name": "Isparta",
+        "population": {
+          "-year": "94",
+          "#text": "120900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-38",
+      "-capital": "cty-Turkey-Istanbul",
+      "-country": "TR",
+      "name": "Istanbul",
+      "area": "5712",
+      "population": "7309190",
+      "city": {
+        "-id": "cty-Turkey-Istanbul",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-38",
+        "name": "Istanbul",
+        "longitude": "28.8333",
+        "latitude": "40.9667",
+        "population": {
+          "-year": "94",
+          "#text": "7615500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-39",
+      "-capital": "cty-Turkey-Izmir",
+      "-country": "TR",
+      "name": "Izmir",
+      "area": "11973",
+      "population": "2694770",
+      "city": {
+        "-id": "cty-Turkey-Izmir",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-39",
+        "name": "Izmir",
+        "longitude": "27.1667",
+        "latitude": "38.4333",
+        "population": {
+          "-year": "94",
+          "#text": "1985300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-40",
+      "-capital": "cty-cid-cia-Turkey-Karaman",
+      "-country": "TR",
+      "name": "Karaman",
+      "area": "9163",
+      "population": "217536",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Karaman",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-40",
+        "name": "Karaman"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-41",
+      "-capital": "cty-cid-cia-Turkey-21",
+      "-country": "TR",
+      "name": "Karamanmaras",
+      "area": "14327",
+      "population": "892952",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-21",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-41",
+        "name": "Karaman Maras",
+        "population": {
+          "-year": "94",
+          "#text": "242200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-42",
+      "-capital": "cty-cid-cia-Turkey-Kars",
+      "-country": "TR",
+      "name": "Kars",
+      "area": "18557",
+      "population": "662155",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Kars",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-42",
+        "name": "Kars"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-43",
+      "-capital": "cty-cid-cia-Turkey-Kastamonu",
+      "-country": "TR",
+      "name": "Kastamonu",
+      "area": "13108",
+      "population": "423611",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Kastamonu",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-43",
+        "name": "Kastamonu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-44",
+      "-capital": "cty-cid-cia-Turkey-11",
+      "-country": "TR",
+      "name": "Kayseri",
+      "area": "16917",
+      "population": "943484",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-11",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-44",
+        "name": "Kayseri",
+        "population": {
+          "-year": "94",
+          "#text": "454000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-45",
+      "-capital": "cty-cid-cia-Turkey-32",
+      "-country": "TR",
+      "name": "Kirikkale",
+      "area": "4365",
+      "population": "349396",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-32",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-45",
+        "name": "Kirikkale",
+        "population": {
+          "-year": "94",
+          "#text": "170300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-46",
+      "-capital": "cty-cid-cia-Turkey-Kirklareli",
+      "-country": "TR",
+      "name": "Kirklareli",
+      "area": "6550",
+      "population": "309512",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Kirklareli",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-46",
+        "name": "Kirklareli"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-47",
+      "-capital": "cty-cid-cia-Turkey-Kirsehir",
+      "-country": "TR",
+      "name": "Kirsehir",
+      "area": "6570",
+      "population": "256862",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Kirsehir",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-47",
+        "name": "Kirsehir"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-48",
+      "-capital": "cty-cid-cia-Turkey-18",
+      "-country": "TR",
+      "name": "Kocaeli",
+      "area": "3626",
+      "population": "936163",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Turkey-18",
+          "-is_state_cap": "yes",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-48",
+          "name": "Izmit",
+          "population": {
+            "-year": "94",
+            "#text": "275800"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-19",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-48",
+          "name": "Kocaeli",
+          "population": {
+            "-year": "94",
+            "#text": "256882"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-23",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-48",
+          "name": "Gebze",
+          "population": {
+            "-year": "94",
+            "#text": "237300"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-30",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-48",
+          "name": "Adapazari",
+          "population": {
+            "-year": "94",
+            "#text": "186000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-49",
+      "-capital": "cty-cid-cia-Turkey-8",
+      "-country": "TR",
+      "name": "Konya",
+      "area": "38257",
+      "population": "1750303",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-8",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-49",
+        "name": "Konya",
+        "population": {
+          "-year": "94",
+          "#text": "576000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-50",
+      "-capital": "cty-cid-cia-Turkey-35",
+      "-country": "TR",
+      "name": "Kutahya",
+      "area": "11875",
+      "population": "578020",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-35",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-50",
+        "name": "Kutahya",
+        "population": {
+          "-year": "94",
+          "#text": "140700"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-51",
+      "-capital": "cty-cid-cia-Turkey-17",
+      "-country": "TR",
+      "name": "Malatya",
+      "area": "12313",
+      "population": "702055",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-17",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-51",
+        "name": "Malatya",
+        "population": {
+          "-year": "94",
+          "#text": "319700"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-52",
+      "-capital": "cty-cid-cia-Turkey-29",
+      "-country": "TR",
+      "name": "Manisa",
+      "area": "13810",
+      "population": "1154418",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-29",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-52",
+        "name": "Manisa",
+        "population": {
+          "-year": "94",
+          "#text": "187500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-53",
+      "-capital": "cty-cid-cia-Turkey-Mardin",
+      "-country": "TR",
+      "name": "Mardin",
+      "area": "8891",
+      "population": "557727",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Mardin",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-53",
+        "name": "Mardin"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-54",
+      "-capital": "cty-cid-cia-Turkey-Mugla",
+      "-country": "TR",
+      "name": "Mugla",
+      "area": "13338",
+      "population": "562809",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Mugla",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-54",
+        "name": "Mugla"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-55",
+      "-capital": "cty-cid-cia-Turkey-Mus",
+      "-country": "TR",
+      "name": "Mus",
+      "area": "8196",
+      "population": "376543",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Mus",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-55",
+        "name": "Mus",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Murat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-56",
+      "-capital": "cty-cid-cia-Turkey-Nevsehir",
+      "-country": "TR",
+      "name": "Nevsehir",
+      "area": "5467",
+      "population": "289509",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Nevsehir",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-56",
+        "name": "Nevsehir"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-57",
+      "-capital": "cty-cid-cia-Turkey-Nigde",
+      "-country": "TR",
+      "name": "Nigde",
+      "area": "7312",
+      "population": "305861",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Nigde",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-57",
+        "name": "Nigde"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-58",
+      "-capital": "cty-cid-cia-Turkey-41",
+      "-country": "TR",
+      "name": "Ordu",
+      "area": "6001",
+      "population": "830105",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-41",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-58",
+        "name": "Ordu",
+        "population": {
+          "-year": "94",
+          "#text": "121300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-59",
+      "-capital": "cty-cid-cia-Turkey-Rize",
+      "-country": "TR",
+      "name": "Rize",
+      "area": "3920",
+      "population": "348776",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Rize",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-59",
+        "name": "Rize"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-60",
+      "-capital": "cty-cid-cia-Turkey-Sakarya",
+      "-country": "TR",
+      "name": "Sakarya",
+      "area": "4817",
+      "population": "683061",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Sakarya",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-60",
+        "name": "Sakarya"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-61",
+      "-capital": "cty-cid-cia-Turkey-16",
+      "-country": "TR",
+      "name": "Samsun",
+      "area": "9579",
+      "population": "1158400",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-16",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-61",
+        "name": "Samsun",
+        "population": {
+          "-year": "94",
+          "#text": "326900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-62",
+      "-capital": "cty-cid-cia-Turkey-15",
+      "-country": "TR",
+      "name": "Sanliurfa",
+      "area": "18584",
+      "population": "1001455",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-15",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-62",
+        "name": "Urfa",
+        "population": {
+          "-year": "94",
+          "#text": "357900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-63",
+      "-capital": "cty-cid-cia-Turkey-Siirt",
+      "-country": "TR",
+      "name": "Siirt",
+      "area": "5406",
+      "population": "243435",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Siirt",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-63",
+        "name": "Siirt"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-64",
+      "-capital": "cty-cid-cia-Turkey-Sinop",
+      "-country": "TR",
+      "name": "Sinop",
+      "area": "5862",
+      "population": "265153",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Sinop",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-64",
+        "name": "Sinop"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-65",
+      "-capital": "cty-cid-cia-Turkey-Sirnak",
+      "-country": "TR",
+      "name": "Sirnak",
+      "area": "7172",
+      "population": "262006",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Sirnak",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-65",
+        "name": "Sirnak"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-66",
+      "-capital": "cty-cid-cia-Turkey-22",
+      "-country": "TR",
+      "name": "Sivas",
+      "area": "28488",
+      "population": "767481",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-22",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-66",
+        "name": "Sivas",
+        "population": {
+          "-year": "94",
+          "#text": "240100"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-67",
+      "-capital": "cty-cid-cia-Turkey-Tekirdag",
+      "-country": "TR",
+      "name": "Tekirdag",
+      "area": "6218",
+      "population": "468842",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Tekirdag",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-67",
+        "name": "Tekirdag"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-68",
+      "-capital": "cty-cid-cia-Turkey-Tokat",
+      "-country": "TR",
+      "name": "Tokat",
+      "area": "9958",
+      "population": "719251",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Tokat",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-68",
+        "name": "Tokat"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-69",
+      "-capital": "cty-cid-cia-Turkey-34",
+      "-country": "TR",
+      "name": "Trabzon",
+      "area": "4685",
+      "population": "795849",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-34",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-69",
+        "name": "Trabzon",
+        "population": {
+          "-year": "94",
+          "#text": "145400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-70",
+      "-capital": "cty-cid-cia-Turkey-Tunceli",
+      "-country": "TR",
+      "name": "Tunceli",
+      "area": "7774",
+      "population": "133143",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Tunceli",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-70",
+        "name": "Tunceli"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-71",
+      "-capital": "cty-cid-cia-Turkey-43",
+      "-country": "TR",
+      "name": "Usak",
+      "area": "5341",
+      "population": "290283",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-43",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-71",
+        "name": "Usak",
+        "population": {
+          "-year": "94",
+          "#text": "119900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-72",
+      "-capital": "cty-cid-cia-Turkey-27",
+      "-country": "TR",
+      "name": "Van",
+      "area": "19069",
+      "population": "637433",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-27",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-72",
+        "name": "Van",
+        "population": {
+          "-year": "94",
+          "#text": "194600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-73",
+      "-capital": "cty-cid-cia-Turkey-Yozgat",
+      "-country": "TR",
+      "name": "Yozgat",
+      "area": "14123",
+      "population": "579150",
+      "city": {
+        "-id": "cty-cid-cia-Turkey-Yozgat",
+        "-is_state_cap": "yes",
+        "-country": "TR",
+        "-province": "prov-cid-cia-Turkey-73",
+        "name": "Yozgat"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkey-74",
+      "-capital": "cty-cid-cia-Turkey-44",
+      "-country": "TR",
+      "name": "Zonguldak",
+      "area": "8629",
+      "population": "1073560",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Turkey-44",
+          "-is_state_cap": "yes",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-74",
+          "name": "Zonguldak",
+          "population": {
+            "-year": "94",
+            "#text": "115900"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Turkey-46",
+          "-country": "TR",
+          "-province": "prov-cid-cia-Turkey-74",
+          "name": "Karabuk",
+          "population": {
+            "-year": "94",
+            "#text": "113900"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "DK",
+  "-area": "43070",
+  "-capital": "cty-Denmark-Copenhagen",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-9 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NC org-NIB org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-PCA org-UN org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-UNMOGIP org-UNMOT org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Denmark",
+  "population": "5249632",
+  "population_growth": "0.38",
+  "infant_mortality": "4.8",
+  "gdp_total": "112800",
+  "gdp_agri": "3",
+  "gdp_ind": "23.5",
+  "gdp_serv": "73.5",
+  "inflation": "2.4",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "91",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "border": {
+    "-country": "D",
+    "-length": "68"
+  },
+  "city": [
+    {
+      "-id": "cty-Denmark-Copenhagen",
+      "-is_country_cap": "yes",
+      "-country": "DK",
+      "name": "Copenhagen",
+      "longitude": "12.55",
+      "latitude": "55.6833",
+      "population": {
+        "-year": "87",
+        "#text": "1358540"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Ostsee"
+      },
+      "located_on": { "-island": "island-Seeland" }
+    },
+    {
+      "-id": "stadt-Aarhus-DK-DK",
+      "-country": "DK",
+      "name": "Aarhus",
+      "longitude": "10.1",
+      "latitude": "56.1",
+      "population": {
+        "-year": "87",
+        "#text": "194345"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Kattegat"
+      }
+    },
+    {
+      "-id": "stadt-Odense-DK-DK",
+      "-country": "DK",
+      "name": "Odense",
+      "longitude": "10.2",
+      "latitude": "55.3",
+      "population": {
+        "-year": "87",
+        "#text": "136803"
+      },
+      "located_on": { "-island": "island-Fuenen" }
+    },
+    {
+      "-id": "stadt-Aalborg-DK-DK",
+      "-country": "DK",
+      "name": "Aalborg",
+      "longitude": "10",
+      "latitude": "57",
+      "population": {
+        "-year": "87",
+        "#text": "113865"
+      }
+    },
+    {
+      "-id": "stadt-Esbjerg-DK-DK",
+      "-country": "DK",
+      "name": "Esbjerg",
+      "longitude": "8.3",
+      "latitude": "55.3",
+      "population": {
+        "-year": "87",
+        "#text": "70975"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Nordsee"
+      }
+    },
+    {
+      "-id": "stadt-Randers-DK-DK",
+      "-country": "DK",
+      "name": "Randers",
+      "longitude": "10",
+      "latitude": "56.3",
+      "population": {
+        "-year": "87",
+        "#text": "55780"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "EW",
+  "-area": "45100",
+  "-capital": "cty-cid-cia-Estonia-Tallinn",
+  "-memberships": "org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EU org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-ANC org-OSCE org-PFP org-UN org-UNCRO org-UNESCO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Estonia",
+  "population": "1459428",
+  "population_growth": "-1.13",
+  "infant_mortality": "17.4",
+  "gdp_total": "12300",
+  "gdp_agri": "10",
+  "gdp_ind": "37",
+  "gdp_serv": "53",
+  "inflation": "29",
+  "indep_date": "1991-09-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3.2",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "30.3",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Byelorussian"
+    },
+    {
+      "-percentage": "1.1",
+      "#text": "Finn"
+    },
+    {
+      "-percentage": "61.5",
+      "#text": "Estonian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "LV",
+      "-length": "267"
+    },
+    {
+      "-country": "R",
+      "-length": "290"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Estonia-Tallinn",
+    "-is_country_cap": "yes",
+    "-country": "EW",
+    "name": "Tallinn",
+    "longitude": "25",
+    "latitude": "59.3",
+    "population": {
+      "-year": "87",
+      "#text": "478000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Ostsee"
+    }
+  }
+}
+{
+  "-car_code": "FARX",
+  "-area": "1400",
+  "-capital": "cty-cid-cia-Faroe-Islands-Torshavn",
+  "name": "Faroe Islands",
+  "population": "43857",
+  "population_growth": "-1.8",
+  "infant_mortality": "7.2",
+  "gdp_total": "733",
+  "inflation": "6.8",
+  "dependent": { "-country": "DK" },
+  "government": "part of the Danish realm",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Scandinavian"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Protestant"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Faroe-Islands-Torshavn",
+    "-is_country_cap": "yes",
+    "-country": "FARX",
+    "name": "Torshavn",
+    "longitude": "-7",
+    "latitude": "62",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Europaeisches_Nordmeer"
+    },
+    "located_on": { "-island": "island-Streymoy" }
+  }
+}
+{
+  "-car_code": "SF",
+  "-area": "337030",
+  "-capital": "cty-Finland-Helsinki",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-9 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-NC org-NIB org-ANC org-EN org-NSG org-OECD org-OSCE org-OAS org-PFP org-PCA org-UN org-UNCRO org-UNESCO org-UNFICYP org-UNIDO org-UNIFIL org-UNIKOM org-UNMOGIP org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Finland",
+  "population": "5105230",
+  "population_growth": "0.1",
+  "infant_mortality": "4.9",
+  "gdp_total": "92400",
+  "gdp_agri": "4.6",
+  "gdp_ind": "28",
+  "gdp_serv": "67.4",
+  "inflation": "2",
+  "indep_date": "1917-12-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Christian Orthodox"
+    },
+    {
+      "-percentage": "89",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "6.3",
+      "#text": "Swedish"
+    },
+    {
+      "-percentage": "93.5",
+      "#text": "Finnish"
+    }
+  ],
+  "border": [
+    {
+      "-country": "R",
+      "-length": "1313"
+    },
+    {
+      "-country": "N",
+      "-length": "729"
+    },
+    {
+      "-country": "S",
+      "-length": "586"
+    }
+  ],
+  "province": [
+    {
+      "-id": "lteil-ALA-SF",
+      "-capital": "stadt-Mariehamn-SF-ALA",
+      "-country": "SF",
+      "name": "Aland",
+      "population": "23000",
+      "city": {
+        "-id": "stadt-Mariehamn-SF-ALA",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-ALA-SF",
+        "name": "Mariehamn",
+        "longitude": "19.5",
+        "latitude": "60.1",
+        "population": {
+          "-year": "87",
+          "#text": "9500"
+        }
+      }
+    },
+    {
+      "-id": "lteil-HAE-SF",
+      "-capital": "stadt-Haemeenlinna-SF-HAE",
+      "-country": "SF",
+      "name": "Haeme",
+      "population": "662000",
+      "city": [
+        {
+          "-id": "stadt-Tampere-SF-HAE",
+          "-country": "SF",
+          "-province": "lteil-HAE-SF",
+          "name": "Tampere",
+          "longitude": "23.5",
+          "latitude": "61.3",
+          "population": {
+            "-year": "87",
+            "#text": "170097"
+          }
+        },
+        {
+          "-id": "stadt-Lahti-SF-HAE",
+          "-country": "SF",
+          "-province": "lteil-HAE-SF",
+          "name": "Lahti",
+          "longitude": "25.2",
+          "latitude": "60.5",
+          "population": {
+            "-year": "87",
+            "#text": "94234"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Paeijaenne"
+          }
+        },
+        {
+          "-id": "stadt-Haemeenlinna-SF-HAE",
+          "-is_state_cap": "yes",
+          "-country": "SF",
+          "-province": "lteil-HAE-SF",
+          "name": "Haemeenlinna",
+          "longitude": "24.3",
+          "latitude": "61",
+          "population": {
+            "-year": "87",
+            "#text": "42000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "lteil-KUO-SF",
+      "-capital": "stadt-Kuopio-SF-KUO",
+      "-country": "SF",
+      "name": "Kuopio",
+      "population": "252000",
+      "city": {
+        "-id": "stadt-Kuopio-SF-KUO",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-KUO-SF",
+        "name": "Kuopio",
+        "longitude": "27.4",
+        "latitude": "62.5",
+        "population": {
+          "-year": "87",
+          "#text": "78571"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Kallavesi"
+        }
+      }
+    },
+    {
+      "-id": "lteil-KYM-SF",
+      "-capital": "stadt-Kotka-SF-KYM",
+      "-country": "SF",
+      "name": "Kymi",
+      "population": "345000",
+      "city": [
+        {
+          "-id": "stadt-Lappeenrenta-SF-KYM",
+          "-country": "SF",
+          "-province": "lteil-KYM-SF",
+          "name": "Lappeenrenta",
+          "longitude": "26.5",
+          "latitude": "60.5",
+          "population": {
+            "-year": "87",
+            "#text": "53922"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Saimaa"
+          }
+        },
+        {
+          "-id": "stadt-Kotka-SF-KYM",
+          "-is_state_cap": "yes",
+          "-country": "SF",
+          "-province": "lteil-KYM-SF",
+          "name": "Kotka",
+          "longitude": "26.5",
+          "latitude": "60.3",
+          "population": {
+            "-year": "87",
+            "#text": "58345"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Ostsee"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Kymijoki"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "lteil-LAP-SF",
+      "-capital": "stadt-Rovaniemi-SF-LAP",
+      "-country": "SF",
+      "name": "Lappia",
+      "population": "195000",
+      "city": {
+        "-id": "stadt-Rovaniemi-SF-LAP",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-LAP-SF",
+        "name": "Rovaniemi",
+        "longitude": "24.4",
+        "latitude": "66.3",
+        "population": {
+          "-year": "87",
+          "#text": "31000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ounasjoki river-Kemijoki"
+        }
+      }
+    },
+    {
+      "-id": "lteil-MIK-SF",
+      "-capital": "stadt-Mikkeli-SF-MIK",
+      "-country": "SF",
+      "name": "Mikkeli",
+      "population": "209000",
+      "city": {
+        "-id": "stadt-Mikkeli-SF-MIK",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-MIK-SF",
+        "name": "Mikkeli",
+        "longitude": "27.2",
+        "latitude": "61.4",
+        "population": {
+          "-year": "87",
+          "#text": "28000"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Saimaa"
+        }
+      }
+    },
+    {
+      "-id": "lteil-MFI-SF",
+      "-capital": "stadt-Jyvaeskylae-SF-MFI",
+      "-country": "SF",
+      "name": "Suomi",
+      "population": "242000",
+      "city": {
+        "-id": "stadt-Jyvaeskylae-SF-MFI",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-MFI-SF",
+        "name": "Jyvaeskylae",
+        "longitude": "25.2",
+        "latitude": "62.1",
+        "population": {
+          "-year": "87",
+          "#text": "65511"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Paeijaenne"
+        }
+      }
+    },
+    {
+      "-id": "lteil-NKA-SF",
+      "-capital": "stadt-Joensuu-SF-NKA",
+      "-country": "SF",
+      "name": "Pohjols-Karjala",
+      "population": "177000",
+      "city": {
+        "-id": "stadt-Joensuu-SF-NKA",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-NKA-SF",
+        "name": "Joensuu",
+        "longitude": "29.5",
+        "latitude": "62.4",
+        "population": {
+          "-year": "87",
+          "#text": "44000"
+        }
+      }
+    },
+    {
+      "-id": "lteil-OUL-SF",
+      "-capital": "stadt-Oulu-SF-OUL",
+      "-country": "SF",
+      "name": "Oulu",
+      "population": "415000",
+      "city": {
+        "-id": "stadt-Oulu-SF-OUL",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-OUL-SF",
+        "name": "Oulu",
+        "longitude": "25.2",
+        "latitude": "65.3",
+        "population": {
+          "-year": "87",
+          "#text": "97898"
+        },
+        "located_at": [
+          {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          },
+          {
+            "-watertype": "river",
+            "-river": "river-Oulujoki"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "lteil-TUP-SF",
+      "-capital": "stadt-Turku-SF-TUP",
+      "-country": "SF",
+      "name": "Turku-Pori",
+      "population": "702000",
+      "city": [
+        {
+          "-id": "stadt-Pori-SF-TUP",
+          "-country": "SF",
+          "-province": "lteil-TUP-SF",
+          "name": "Pori",
+          "longitude": "21.2",
+          "latitude": "61.3",
+          "population": {
+            "-year": "87",
+            "#text": "77763"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Kokemaeenjoki"
+          }
+        },
+        {
+          "-id": "stadt-Turku-SF-TUP",
+          "-is_state_cap": "yes",
+          "-country": "SF",
+          "-province": "lteil-TUP-SF",
+          "name": "Turku",
+          "longitude": "22.1",
+          "latitude": "60.3",
+          "population": {
+            "-year": "87",
+            "#text": "161292"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "lteil-UUS-SF",
+      "-capital": "cty-Finland-Helsinki",
+      "-country": "SF",
+      "name": "Uusimaa",
+      "population": "1119000",
+      "city": [
+        {
+          "-id": "cty-Finland-Helsinki",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "SF",
+          "-province": "lteil-UUS-SF",
+          "name": "Helsinki",
+          "longitude": "24.95",
+          "latitude": "60.1667",
+          "population": {
+            "-year": "87",
+            "#text": "487428"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        },
+        {
+          "-id": "stadt-Espoo-SF-UUS",
+          "-country": "SF",
+          "-province": "lteil-UUS-SF",
+          "name": "Espoo",
+          "longitude": "24.3",
+          "latitude": "60.2",
+          "population": {
+            "-year": "87",
+            "#text": "160480"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "lteil-VAA-SF",
+      "-capital": "stadt-Vaasa-SF-VAA",
+      "-country": "SF",
+      "name": "Vaasa",
+      "population": "430000",
+      "city": {
+        "-id": "stadt-Vaasa-SF-VAA",
+        "-is_state_cap": "yes",
+        "-country": "SF",
+        "-province": "lteil-VAA-SF",
+        "name": "Vaasa",
+        "longitude": "21.3",
+        "latitude": "63",
+        "population": {
+          "-year": "87",
+          "#text": "54275"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "N",
+  "-area": "324220",
+  "-capital": "cty-Norway-Oslo",
+  "-memberships": "org-AfDB org-AsDB org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EFTA org-CERN org-ESA org-FAO org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-NC org-NIB org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNIFIL org-MINURSO org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Norway",
+  "population": "4383807",
+  "population_growth": "0.48",
+  "infant_mortality": "4.9",
+  "gdp_total": "106200",
+  "gdp_agri": "2.9",
+  "gdp_ind": "34.7",
+  "gdp_serv": "62.4",
+  "inflation": "2.5",
+  "indep_date": "1905-10-26",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "87.8",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Norwegian"
+  },
+  "border": [
+    {
+      "-country": "R",
+      "-length": "167"
+    },
+    {
+      "-country": "SF",
+      "-length": "729"
+    },
+    {
+      "-country": "S",
+      "-length": "1619"
+    }
+  ],
+  "province": [
+    {
+      "-id": "lteil-OS-N",
+      "-capital": "cty-Norway-Oslo",
+      "-country": "N",
+      "name": "Oslo",
+      "population": "449337",
+      "city": {
+        "-id": "cty-Norway-Oslo",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-OS-N",
+        "name": "Oslo",
+        "longitude": "10.7333",
+        "latitude": "59.9333",
+        "population": {
+          "-year": "87",
+          "#text": "449337"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Skagerrak"
+        }
+      }
+    },
+    {
+      "-id": "lteil-AK-N",
+      "-capital": "cty-Norway-Oslo",
+      "-country": "N",
+      "name": "Akershus",
+      "population": "393217"
+    },
+    {
+      "-id": "lteil-OES-N",
+      "-capital": "stadt-Moss-N-OES",
+      "-country": "N",
+      "name": "Oestfold",
+      "population": "234941",
+      "city": {
+        "-id": "stadt-Moss-N-OES",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-OES-N",
+        "name": "Moss",
+        "longitude": "10.4",
+        "latitude": "59.3",
+        "population": {
+          "-year": "87",
+          "#text": "24517"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Skagerrak"
+        }
+      }
+    },
+    {
+      "-id": "lteil-HE-N",
+      "-capital": "stadt-Hamar-N-HE",
+      "-country": "N",
+      "name": "Hedmark",
+      "population": "186355",
+      "city": {
+        "-id": "stadt-Hamar-N-HE",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-HE-N",
+        "name": "Hamar",
+        "longitude": "11.5",
+        "latitude": "60.5",
+        "population": {
+          "-year": "87",
+          "#text": "15685"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-MjoesaSee"
+        }
+      }
+    },
+    {
+      "-id": "lteil-OP-N",
+      "-capital": "stadt-Lillehammer-N-OP",
+      "-country": "N",
+      "name": "Oppland",
+      "population": "181791",
+      "city": {
+        "-id": "stadt-Lillehammer-N-OP",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-OP-N",
+        "name": "Lillehammer",
+        "longitude": "10.3",
+        "latitude": "61.1",
+        "population": {
+          "-year": "87",
+          "#text": "22118"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-MjoesaSee"
+        }
+      }
+    },
+    {
+      "-id": "lteil-BU-N",
+      "-capital": "stadt-Drammen-N-BU",
+      "-country": "N",
+      "name": "Buskerud",
+      "population": "219967",
+      "city": {
+        "-id": "stadt-Drammen-N-BU",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-BU-N",
+        "name": "Drammen",
+        "longitude": "10.1",
+        "latitude": "59.4",
+        "population": {
+          "-year": "87",
+          "#text": "50855"
+        }
+      }
+    },
+    {
+      "-id": "lteil-VE-N",
+      "-capital": "stadt-Toensberg-N-VE",
+      "-country": "N",
+      "name": "Vestfold",
+      "population": "191600",
+      "city": {
+        "-id": "stadt-Toensberg-N-VE",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-VE-N",
+        "name": "Toensberg",
+        "longitude": "10.2",
+        "latitude": "59.2",
+        "population": {
+          "-year": "87",
+          "#text": "8984"
+        }
+      }
+    },
+    {
+      "-id": "lteil-TE-N",
+      "-capital": "stadt-Skien-N-TE",
+      "-country": "N",
+      "name": "Telemark",
+      "population": "162547",
+      "city": {
+        "-id": "stadt-Skien-N-TE",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-TE-N",
+        "name": "Skien",
+        "longitude": "9.4",
+        "latitude": "59.1",
+        "population": {
+          "-year": "87",
+          "#text": "47010"
+        }
+      }
+    },
+    {
+      "-id": "lteil-AA-N",
+      "-capital": "stadt-Arendal-N-AA",
+      "-country": "N",
+      "name": "Aust Agder",
+      "population": "94688",
+      "city": {
+        "-id": "stadt-Arendal-N-AA",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-AA-N",
+        "name": "Arendal",
+        "longitude": "8.4",
+        "latitude": "58.3",
+        "population": {
+          "-year": "87",
+          "#text": "12174"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Skagerrak"
+        }
+      }
+    },
+    {
+      "-id": "lteil-VA-N",
+      "-capital": "stadt-Kristiansand-N-VA",
+      "-country": "N",
+      "name": "Vest Agder",
+      "population": "140232",
+      "city": {
+        "-id": "stadt-Kristiansand-N-VA",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-VA-N",
+        "name": "Kristiansand",
+        "longitude": "8",
+        "latitude": "58.1",
+        "population": {
+          "-year": "87",
+          "#text": "62640"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Skagerrak"
+        }
+      }
+    },
+    {
+      "-id": "lteil-RO-N",
+      "-capital": "stadt-Stavanger-N-RO",
+      "-country": "N",
+      "name": "Rogaland",
+      "population": "323365",
+      "city": {
+        "-id": "stadt-Stavanger-N-RO",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-RO-N",
+        "name": "Stavanger",
+        "longitude": "5.5",
+        "latitude": "59",
+        "population": {
+          "-year": "87",
+          "#text": "95089"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        }
+      }
+    },
+    {
+      "-id": "lteil-HO-N",
+      "-capital": "stadt-Bergen-N-HO",
+      "-country": "N",
+      "name": "Hordaland",
+      "population": "399702",
+      "city": {
+        "-id": "stadt-Bergen-N-HO",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-HO-N",
+        "name": "Bergen",
+        "longitude": "5.2",
+        "latitude": "60.2",
+        "population": {
+          "-year": "87",
+          "#text": "207916"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        }
+      }
+    },
+    {
+      "-id": "lteil-SF-N",
+      "-capital": "stadt-Hermannsverk-N-SF",
+      "-country": "N",
+      "name": "Sogn og Fjordane",
+      "population": "106116",
+      "city": {
+        "-id": "stadt-Hermannsverk-N-SF",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-SF-N",
+        "name": "Hermannsverk",
+        "longitude": "6.5",
+        "latitude": "61.1",
+        "population": {
+          "-year": "87",
+          "#text": "706"
+        }
+      }
+    },
+    {
+      "-id": "lteil-MR-N",
+      "-capital": "stadt-Molde-N-MR",
+      "-country": "N",
+      "name": "Moere og Romsdal",
+      "population": "237290",
+      "city": {
+        "-id": "stadt-Molde-N-MR",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-MR-N",
+        "name": "Molde",
+        "longitude": "7.1",
+        "latitude": "62.5",
+        "population": {
+          "-year": "87",
+          "#text": "21448"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Europaeisches_Nordmeer"
+        }
+      }
+    },
+    {
+      "-id": "lteil-ST-N",
+      "-capital": "stadt-Trondheim-N-ST",
+      "-country": "N",
+      "name": "Soer Trondelag",
+      "population": "246824",
+      "city": {
+        "-id": "stadt-Trondheim-N-ST",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-ST-N",
+        "name": "Trondheim",
+        "longitude": "10.2",
+        "latitude": "63.3",
+        "population": {
+          "-year": "87",
+          "#text": "134426"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Europaeisches_Nordmeer"
+        }
+      }
+    },
+    {
+      "-id": "lteil-NT-N",
+      "-capital": "stadt-Steinkjer-N-NT",
+      "-country": "N",
+      "name": "Nord Trondelag",
+      "population": "126692",
+      "city": {
+        "-id": "stadt-Steinkjer-N-NT",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-NT-N",
+        "name": "Steinkjer",
+        "longitude": "11.3",
+        "latitude": "64",
+        "population": {
+          "-year": "87",
+          "#text": "20480"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Europaeisches_Nordmeer"
+        }
+      }
+    },
+    {
+      "-id": "lteil-NO-N",
+      "-capital": "stadt-Bodoe-N-NO",
+      "-country": "N",
+      "name": "Nordland",
+      "population": "242268",
+      "city": [
+        {
+          "-id": "stadt-Svolvaer-N-NO",
+          "-country": "N",
+          "-province": "lteil-NO-N",
+          "name": "Svolvaer",
+          "longitude": "14.3",
+          "latitude": "68.2",
+          "population": {
+            "-year": "87",
+            "#text": "4500"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Europaeisches_Nordmeer"
+          }
+        },
+        {
+          "-id": "stadt-Narvik-N-NO",
+          "-country": "N",
+          "-province": "lteil-NO-N",
+          "name": "Narvik",
+          "longitude": "17.3",
+          "latitude": "68.3",
+          "population": {
+            "-year": "87",
+            "#text": "18754"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Europaeisches_Nordmeer"
+          }
+        },
+        {
+          "-id": "stadt-Bodoe-N-NO",
+          "-is_state_cap": "yes",
+          "-country": "N",
+          "-province": "lteil-NO-N",
+          "name": "Bodoe",
+          "longitude": "14.2",
+          "latitude": "67.2",
+          "population": {
+            "-year": "87",
+            "#text": "34479"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Europaeisches_Nordmeer"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "lteil-TR-N",
+      "-capital": "stadt-Tromsoe-N-TR",
+      "-country": "N",
+      "name": "Troms",
+      "population": "146736",
+      "city": {
+        "-id": "stadt-Tromsoe-N-TR",
+        "-is_state_cap": "yes",
+        "-country": "N",
+        "-province": "lteil-TR-N",
+        "name": "Tromsoe",
+        "longitude": "19",
+        "latitude": "69.4",
+        "population": {
+          "-year": "87",
+          "#text": "48109"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Europaeisches_Nordmeer"
+        }
+      }
+    },
+    {
+      "-id": "lteil-FI-N",
+      "-capital": "stadt-Vadsoe-N-FI",
+      "-country": "N",
+      "name": "Finnmark",
+      "population": "75667",
+      "city": [
+        {
+          "-id": "stadt-Hammerfest-N-FI",
+          "-country": "N",
+          "-province": "lteil-FI-N",
+          "name": "Hammerfest",
+          "longitude": "23.4",
+          "latitude": "70.4",
+          "population": {
+            "-year": "87",
+            "#text": "7089"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Barentssee"
+          }
+        },
+        {
+          "-id": "stadt-Vadsoe-N-FI",
+          "-is_state_cap": "yes",
+          "-country": "N",
+          "-province": "lteil-FI-N",
+          "name": "Vadsoe",
+          "longitude": "29.5",
+          "latitude": "70.5",
+          "population": {
+            "-year": "87",
+            "#text": "5961"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Barentssee"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "S",
+  "-area": "449964",
+  "-capital": "cty-Sweden-Stockholm",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CE org-CBSS org-CCC org-ECE org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-6 org-G-9 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-NC org-NIB org-ANC org-EN org-NSG org-OECD org-OSCE org-PFP org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNIKOM org-UNMOGIP org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WEU org-WFTU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Sweden",
+  "population": "8900954",
+  "population_growth": "0.56",
+  "infant_mortality": "4.5",
+  "gdp_total": "177300",
+  "gdp_agri": "2",
+  "gdp_ind": "27",
+  "gdp_serv": "71",
+  "inflation": "2.6",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "12",
+    "#text": "foreign-born first-generation"
+  },
+  "religions": [
+    {
+      "-percentage": "1.5",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "94",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Pentecostal"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Swedish"
+  },
+  "border": [
+    {
+      "-country": "SF",
+      "-length": "586"
+    },
+    {
+      "-country": "N",
+      "-length": "1619"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Sweden-2",
+      "-capital": "cty-cid-cia-Sweden-Vanersborg",
+      "-country": "S",
+      "name": "Alvsborg",
+      "area": "11395",
+      "population": "444259",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Vanersborg",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-2",
+        "name": "Vanersborg",
+        "longitude": "12.2",
+        "latitude": "58.2",
+        "population": {
+          "-year": "87",
+          "#text": "35804"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Goetaaelv"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-Vaenersee"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-3",
+      "-capital": "cty-cid-cia-Sweden-Karlskrona",
+      "-country": "S",
+      "name": "Blekinge",
+      "area": "2941",
+      "population": "151168",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Karlskrona",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-3",
+        "name": "Karlskrona",
+        "longitude": "15.3",
+        "latitude": "56.1",
+        "population": {
+          "-year": "87",
+          "#text": "59007"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-4",
+      "-capital": "cty-cid-cia-Sweden-Gavle",
+      "-country": "S",
+      "name": "Gavleborg",
+      "area": "18191",
+      "population": "289339",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Gavle",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-4",
+        "name": "Gavle",
+        "longitude": "17",
+        "latitude": "60.4",
+        "population": {
+          "-year": "87",
+          "#text": "87431"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-5",
+      "-capital": "cty-cid-cia-Sweden-3",
+      "-country": "S",
+      "name": "Goteborg och Bohus",
+      "area": "5141",
+      "population": "742550",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-3",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-5",
+        "name": "Goteborg",
+        "longitude": "12",
+        "latitude": "57.4",
+        "population": {
+          "-year": "95",
+          "#text": "449189"
+        },
+        "located_at": [
+          {
+            "-watertype": "sea",
+            "-sea": "sea-Kattegat"
+          },
+          {
+            "-watertype": "river",
+            "-river": "river-Goetaaelv"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-6",
+      "-capital": "cty-cid-cia-Sweden-Visby",
+      "-country": "S",
+      "name": "Gotland",
+      "area": "3140",
+      "population": "57383",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Visby",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-6",
+        "name": "Visby",
+        "longitude": "18.2",
+        "latitude": "57.3",
+        "population": {
+          "-year": "87",
+          "#text": "20000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-7",
+      "-capital": "cty-cid-cia-Sweden-Halmstad",
+      "-country": "S",
+      "name": "Halland",
+      "area": "5454",
+      "population": "257874",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Halmstad",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-7",
+        "name": "Halmstad",
+        "longitude": "13",
+        "latitude": "56.4",
+        "population": {
+          "-year": "87",
+          "#text": "77601"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Kattegat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-8",
+      "-capital": "cty-cid-cia-Sweden-Ostersund",
+      "-country": "S",
+      "name": "Jamtland",
+      "area": "49443",
+      "population": "136009",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Ostersund",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-8",
+        "name": "Ostersund",
+        "longitude": "14.3",
+        "latitude": "63.2",
+        "population": {
+          "-year": "87",
+          "#text": "56662"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-9",
+      "-capital": "cty-cid-cia-Sweden-10",
+      "-country": "S",
+      "name": "Jonkoping",
+      "area": "9944",
+      "population": "309738",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-10",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-9",
+        "name": "Jonkoping",
+        "longitude": "14",
+        "latitude": "57.4",
+        "population": {
+          "-year": "95",
+          "#text": "115429"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Vaettersee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-10",
+      "-capital": "cty-cid-cia-Sweden-Kalmar",
+      "-country": "S",
+      "name": "Kalmar",
+      "area": "11170",
+      "population": "241883",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Kalmar",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-10",
+        "name": "Kalmar",
+        "longitude": "16.3",
+        "latitude": "56.4",
+        "population": {
+          "-year": "87",
+          "#text": "54554"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-11",
+      "-capital": "cty-cid-cia-Sweden-Falun",
+      "-country": "S",
+      "name": "Dalarna",
+      "area": "28194",
+      "population": "290388",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Falun",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-11",
+        "name": "Falun",
+        "longitude": "15.3",
+        "latitude": "60.3",
+        "population": {
+          "-year": "87",
+          "#text": "51900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-12",
+      "-capital": "cty-cid-cia-Sweden-Kristianstad",
+      "-country": "S",
+      "name": "Kristianstad",
+      "area": "6087",
+      "population": "291468",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Kristianstad",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-12",
+        "name": "Kristianstad",
+        "longitude": "14",
+        "latitude": "56",
+        "population": {
+          "-year": "87",
+          "#text": "69941"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-13",
+      "-capital": "cty-cid-cia-Sweden-Vaxjo",
+      "-country": "S",
+      "name": "Kronoberg",
+      "area": "8458",
+      "population": "178612",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Vaxjo",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-13",
+        "name": "Vaxjo",
+        "longitude": "15",
+        "latitude": "56.5",
+        "population": {
+          "-year": "87",
+          "#text": "66925"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-14",
+      "-capital": "cty-cid-cia-Sweden-4",
+      "-country": "S",
+      "name": "Malmohus",
+      "area": "4938",
+      "population": "786757",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Sweden-4",
+          "-is_state_cap": "yes",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-14",
+          "name": "Malmo",
+          "longitude": "13",
+          "latitude": "55.3",
+          "population": {
+            "-year": "95",
+            "#text": "245699"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Sweden-11",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-14",
+          "name": "Helsingborg",
+          "population": {
+            "-year": "95",
+            "#text": "114339"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-15",
+      "-capital": "cty-cid-cia-Sweden-Lulea",
+      "-country": "S",
+      "name": "Norrbotten",
+      "area": "98913",
+      "population": "264834",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Sweden-Lulea",
+          "-is_state_cap": "yes",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-15",
+          "name": "Lulea",
+          "longitude": "22.1",
+          "latitude": "65.4",
+          "population": {
+            "-year": "87",
+            "#text": "66526"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          }
+        },
+        {
+          "-id": "stadt-Jokkmokk-S-BD",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-15",
+          "name": "Jokkmokk",
+          "longitude": "19.5",
+          "latitude": "66.1",
+          "population": {
+            "-year": "87",
+            "#text": "6822"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-16",
+      "-capital": "cty-cid-cia-Sweden-9",
+      "-country": "S",
+      "name": "Orebro",
+      "area": "8519",
+      "population": "273608",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-9",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-16",
+        "name": "Orebro",
+        "longitude": "15",
+        "latitude": "59.2",
+        "population": {
+          "-year": "95",
+          "#text": "119635"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-17",
+      "-capital": "cty-cid-cia-Sweden-6",
+      "-country": "S",
+      "name": "Ostergotland",
+      "area": "10562",
+      "population": "406100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Sweden-6",
+          "-is_state_cap": "yes",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-17",
+          "name": "Linkoping",
+          "longitude": "15.3",
+          "latitude": "58.3",
+          "population": {
+            "-year": "95",
+            "#text": "131370"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Sweden-7",
+          "-country": "S",
+          "-province": "prov-cid-cia-Sweden-17",
+          "name": "Norrkoping",
+          "population": {
+            "-year": "95",
+            "#text": "123795"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-18",
+      "-capital": "cty-cid-cia-Sweden-Mariestad",
+      "-country": "S",
+      "name": "Skaraborg",
+      "area": "7937",
+      "population": "278162",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Mariestad",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-18",
+        "name": "Mariestad",
+        "longitude": "13.5",
+        "latitude": "58.4",
+        "population": {
+          "-year": "87",
+          "#text": "24255"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Vaenersee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-19",
+      "-capital": "cty-cid-cia-Sweden-Nykoping",
+      "-country": "S",
+      "name": "Sodermanland",
+      "area": "6060",
+      "population": "256818",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Nykoping",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-19",
+        "name": "Nykoping",
+        "longitude": "17",
+        "latitude": "58.5",
+        "population": {
+          "-year": "87",
+          "#text": "64199"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-20",
+      "-capital": "cty-Sweden-Stockholm",
+      "-country": "S",
+      "name": "Stockholm",
+      "area": "6488",
+      "population": "1654511",
+      "city": {
+        "-id": "cty-Sweden-Stockholm",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-20",
+        "name": "Stockholm",
+        "longitude": "18.0667",
+        "latitude": "59.35",
+        "population": {
+          "-year": "95",
+          "#text": "711119"
+        },
+        "located_at": [
+          {
+            "-watertype": "sea",
+            "-sea": "sea-Ostsee"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-Maelarsee"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-21",
+      "-capital": "cty-cid-cia-Sweden-5",
+      "-country": "S",
+      "name": "Uppsala",
+      "area": "6989",
+      "population": "273918",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-5",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-21",
+        "name": "Uppsala",
+        "longitude": "17.4",
+        "latitude": "59.5",
+        "population": {
+          "-year": "95",
+          "#text": "183472"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-22",
+      "-capital": "cty-cid-cia-Sweden-Karlstad",
+      "-country": "S",
+      "name": "Varmland",
+      "area": "17584",
+      "population": "284187",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Karlstad",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-22",
+        "name": "Karlstad",
+        "longitude": "13.5",
+        "latitude": "59.4",
+        "population": {
+          "-year": "87",
+          "#text": "74669"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Klaraelv"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-Vaenersee"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-23",
+      "-capital": "cty-cid-cia-Sweden-Umea",
+      "-country": "S",
+      "name": "Vasterbotten",
+      "area": "55401",
+      "population": "253835",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Umea",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-23",
+        "name": "Umea",
+        "longitude": "20.1",
+        "latitude": "63.5",
+        "population": {
+          "-year": "87",
+          "#text": "85698"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Umeaelv"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-24",
+      "-capital": "cty-cid-cia-Sweden-Harnosand",
+      "-country": "S",
+      "name": "Vasternorrland",
+      "area": "21678",
+      "population": "261280",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-Harnosand",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-24",
+        "name": "Harnosand",
+        "longitude": "18",
+        "latitude": "62.4",
+        "population": {
+          "-year": "87",
+          "#text": "27287"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Ostsee"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sweden-25",
+      "-capital": "cty-cid-cia-Sweden-8",
+      "-country": "S",
+      "name": "Vastmanland",
+      "area": "6302",
+      "population": "259438",
+      "city": {
+        "-id": "cty-cid-cia-Sweden-8",
+        "-is_state_cap": "yes",
+        "-country": "S",
+        "-province": "prov-cid-cia-Sweden-25",
+        "name": "Vasteras",
+        "longitude": "16.3",
+        "latitude": "59.4",
+        "population": {
+          "-year": "95",
+          "#text": "123728"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Maelarsee"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MC",
+  "-area": "1.9",
+  "-capital": "cty-cid-cia-Monaco-Monaco",
+  "-memberships": "org-ACCT org-ECE org-IAEA org-ICAO org-Interpol org-IFRCS org-IMO org-Inmarsat org-IOC org-ICRM org-ITU org-Intelsat org-OSCE org-UN org-UNESCO org-UPU org-WHO org-WIPO",
+  "name": "Monaco",
+  "population": "31719",
+  "population_growth": "0.59",
+  "infant_mortality": "6.9",
+  "gdp_total": "788",
+  "indep_date": "1419-01-01",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "47",
+      "#text": "French"
+    },
+    {
+      "-percentage": "16",
+      "#text": "Italian"
+    },
+    {
+      "-percentage": "16",
+      "#text": "Monegasque"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "border": {
+    "-country": "F",
+    "-length": "4.4"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Monaco-Monaco",
+    "-is_country_cap": "yes",
+    "-country": "MC",
+    "name": "Monaco",
+    "longitude": "7.2",
+    "latitude": "43.7",
+    "population": {
+      "-year": "87",
+      "#text": "1234"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Mittelmeer"
+    }
+  }
+}
+{
+  "-car_code": "GBZ",
+  "-area": "6.5",
+  "-capital": "cty-Gibraltar-Gibraltar",
+  "-memberships": "org-Interpol",
+  "name": "Gibraltar",
+  "population": "28765",
+  "population_growth": "0.54",
+  "infant_mortality": "6.9",
+  "gdp_total": "205",
+  "dependent": { "-country": "GB" },
+  "government": "dependent territory of the UK",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "8",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "74",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Protestant"
+    }
+  ],
+  "border": {
+    "-country": "E",
+    "-length": "1.2"
+  },
+  "city": {
+    "-id": "cty-Gibraltar-Gibraltar",
+    "-is_country_cap": "yes",
+    "-country": "GBZ",
+    "name": "Gibraltar",
+    "longitude": "-5.2",
+    "latitude": "36.15"
+  }
+}
+{
+  "-car_code": "GBG",
+  "-area": "194",
+  "-capital": "cty-cid-cia-Guernsey-Saint-Peter-Port",
+  "name": "Guernsey",
+  "population": "62920",
+  "population_growth": "1.28",
+  "infant_mortality": "9.2",
+  "inflation": "7",
+  "dependent": { "-country": "GB" },
+  "government": "British crown dependency",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Norman-French"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Guernsey-Saint-Peter-Port",
+    "-is_country_cap": "yes",
+    "-country": "GBG",
+    "name": "Saint Peter Port",
+    "longitude": "-2.6",
+    "latitude": "49.6",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Channel"
+    },
+    "located_on": { "-island": "island-Guernsey" }
+  }
+}
+{
+  "-car_code": "V",
+  "-area": "0.44",
+  "-capital": "cty-cid-cia-Holy-See-Vatican-City",
+  "-memberships": "org-IAEA org-ICFTU org-IOM org-ITU org-Intelsat org-OSCE org-OAS org-UN org-UNHCR org-UPU org-WIPO org-WToO",
+  "name": "Holy See",
+  "population": "840",
+  "indep_date": "1929-02-11",
+  "government": "monarchical sacerdotal state",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "border": {
+    "-country": "I",
+    "-length": "3.2"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Holy-See-Vatican-City",
+    "-is_country_cap": "yes",
+    "-country": "V",
+    "name": "Vatican City",
+    "longitude": "12.3",
+    "latitude": "41.5",
+    "population": {
+      "-year": "87",
+      "#text": "392"
+    }
+  }
+}
+{
+  "-car_code": "IS",
+  "-area": "103000",
+  "-capital": "cty-Iceland-Reykjavik",
+  "-memberships": "org-BIS org-CE org-CCC org-ECE org-EBRD org-EFTA org-FAO org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NC org-NIB org-ANC org-NATO org-EN org-OECD org-OSCE org-PCA org-UN org-UNESCO org-UNU org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Iceland",
+  "population": "270292",
+  "population_growth": "0.83",
+  "infant_mortality": "4.3",
+  "gdp_total": "5000",
+  "gdp_agri": "9.6",
+  "gdp_ind": "22.1",
+  "gdp_serv": "68.3",
+  "inflation": "2.5",
+  "indep_date": "1944-06-17",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Celt"
+  },
+  "religions": [
+    {
+      "-percentage": "96",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Icelandic"
+  },
+  "city": [
+    {
+      "-id": "cty-Iceland-Reykjavik",
+      "-is_country_cap": "yes",
+      "-country": "IS",
+      "name": "Reykjavik",
+      "longitude": "-21.9333",
+      "latitude": "64.1333",
+      "population": {
+        "-year": "87",
+        "#text": "84000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      },
+      "located_on": { "-island": "island-Iceland" }
+    },
+    {
+      "-id": "stadt-Keflavik-IS-IS",
+      "-country": "IS",
+      "name": "Keflavik",
+      "longitude": "-22.5",
+      "latitude": "64",
+      "population": {
+        "-year": "87",
+        "#text": "6600"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      },
+      "located_on": { "-island": "island-Iceland" }
+    },
+    {
+      "-id": "stadt-Hafnarfjoerdur-IS-IS",
+      "-country": "IS",
+      "name": "Hafnarfjoerdur",
+      "longitude": "-22",
+      "latitude": "64",
+      "population": {
+        "-year": "87",
+        "#text": "12000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      },
+      "located_on": { "-island": "island-Iceland" }
+    },
+    {
+      "-id": "stadt-Akureyri-IS-IS",
+      "-country": "IS",
+      "name": "Akureyri",
+      "longitude": "-18.3",
+      "latitude": "65.4",
+      "population": {
+        "-year": "87",
+        "#text": "13000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Europaeisches_Nordmeer"
+      },
+      "located_on": { "-island": "island-Iceland" }
+    }
+  ]
+}
+{
+  "-car_code": "IRL",
+  "-area": "70280",
+  "-capital": "cty-Ireland-Dublin",
+  "-memberships": "org-BIS org-CE org-CCC org-ECE org-EBRD org-EIB org-ESA org-EU org-FAO org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-EN org-NSG org-OECD org-OSCE org-UN org-UNCRO org-UNESCO org-UNFICYP org-UNIDO org-UNIFIL org-UNIKOM org-MINURSO org-UNMIH org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WEU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Ireland",
+  "population": "3566833",
+  "population_growth": "-0.22",
+  "infant_mortality": "6.4",
+  "gdp_total": "54600",
+  "gdp_agri": "6.8",
+  "gdp_ind": "35.3",
+  "gdp_serv": "57.9",
+  "inflation": "2.8",
+  "indep_date": "1921-12-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "93",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Anglican"
+    }
+  ],
+  "city": {
+    "-id": "cty-Ireland-Dublin",
+    "-is_country_cap": "yes",
+    "-country": "IRL",
+    "name": "Dublin",
+    "longitude": "-6.35",
+    "latitude": "53.3667",
+    "population": {
+      "-year": "87",
+      "#text": "502337"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Irische_See"
+    },
+    "located_on": { "-island": "island-Ireland" }
+  }
+}
+{
+  "-car_code": "RSM",
+  "-area": "60",
+  "-capital": "cty-cid-cia-San-Marino-San-Marino",
+  "-memberships": "org-CE org-ECE org-ICAO org-ICFTU org-IFRCS org-ILO org-IMF org-IOC org-IOM org-ICRM org-ITU org-NAM org-OSCE org-UN org-UNESCO org-UPU org-WHO org-WIPO org-WToO",
+  "name": "San Marino",
+  "population": "24521",
+  "population_growth": "0.82",
+  "infant_mortality": "5.5",
+  "gdp_total": "380",
+  "inflation": "5.5",
+  "indep_date": "0301-01-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Italian"
+  },
+  "border": {
+    "-country": "I",
+    "-length": "39"
+  },
+  "city": {
+    "-id": "cty-cid-cia-San-Marino-San-Marino",
+    "-is_country_cap": "yes",
+    "-country": "RSM",
+    "name": "San Marino",
+    "longitude": "12.2",
+    "latitude": "43.5",
+    "population": {
+      "-year": "87",
+      "#text": "4416"
+    }
+  }
+}
+{
+  "-car_code": "GBJ",
+  "-area": "117",
+  "-capital": "cty-cid-cia-Jersey-Saint-Helier",
+  "name": "Jersey",
+  "population": "87848",
+  "population_growth": "0.77",
+  "infant_mortality": "2.7",
+  "dependent": { "-country": "GB" },
+  "government": "British crown dependency",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Norman-French"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Jersey-Saint-Helier",
+    "-is_country_cap": "yes",
+    "-country": "GBJ",
+    "name": "Saint Helier",
+    "longitude": "-2.1",
+    "latitude": "49.2",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Channel"
+    },
+    "located_on": { "-island": "island-Jersey" }
+  }
+}
+{
+  "-car_code": "M",
+  "-area": "320",
+  "-capital": "cty-cid-cia-Malta-Valletta",
+  "-memberships": "org-C org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-ANC org-OSCE org-PFP org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Malta",
+  "population": "375576",
+  "population_growth": "1.01",
+  "infant_mortality": "6.9",
+  "gdp_total": "4400",
+  "inflation": "5",
+  "indep_date": "1964-09-21",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "98",
+    "#text": "Roman Catholic"
+  },
+  "city": [
+    {
+      "-id": "cty-cid-cia-Malta-Valletta",
+      "-is_country_cap": "yes",
+      "-country": "M",
+      "name": "Valletta",
+      "longitude": "14.5",
+      "latitude": "35.9",
+      "population": {
+        "-year": "87",
+        "#text": "9302"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      },
+      "located_on": { "-island": "island-Malta" }
+    },
+    {
+      "-id": "cty-cid-cia-Malta-Victoria",
+      "-country": "M",
+      "name": "Victoria",
+      "longitude": "14.25",
+      "latitude": "36.05",
+      "population": {
+        "-year": "87",
+        "#text": "6414"
+      },
+      "located_on": { "-island": "island-Gozo" }
+    }
+  ]
+}
+{
+  "-car_code": "GBM",
+  "-area": "588",
+  "-capital": "cty-cid-cia-Man-Douglas",
+  "name": "Man",
+  "population": "73837",
+  "population_growth": "0.94",
+  "infant_mortality": "2.4",
+  "gdp_total": "780",
+  "inflation": "7",
+  "dependent": { "-country": "GB" },
+  "government": "British crown dependency",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Man-Douglas",
+    "-is_country_cap": "yes",
+    "-country": "GBM",
+    "name": "Douglas",
+    "longitude": "-4.6",
+    "latitude": "54.15",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Irische_See"
+    },
+    "located_on": { "-island": "island-Isle_of_Man" }
+  }
+}
+{
+  "-car_code": "MD",
+  "-area": "33700",
+  "-capital": "cty-cid-cia-Moldova-Chisinau",
+  "-memberships": "org-BSEC org-CIS org-CE org-CCC org-ECE org-EBRD org-FAO org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-ILO org-IMF org-IOC org-IOM org-ITU org-Intelsat org-ANC org-OSCE org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Moldova",
+  "population": "4463847",
+  "population_growth": "0.18",
+  "infant_mortality": "47.6",
+  "gdp_total": "10400",
+  "gdp_agri": "33",
+  "gdp_ind": "36",
+  "gdp_serv": "31",
+  "inflation": "24",
+  "indep_date": "1991-08-27",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "13.8",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "13",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "1.5",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Bulgarian"
+    },
+    {
+      "-percentage": "3.5",
+      "#text": "Gagauz"
+    },
+    {
+      "-percentage": "64.5",
+      "#text": "Moldavian/Romanian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "1.5",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "98.5",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "UA",
+      "-length": "939"
+    },
+    {
+      "-country": "RO",
+      "-length": "450"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Moldova-Chisinau",
+    "-is_country_cap": "yes",
+    "-country": "MD",
+    "name": "Chisinau",
+    "longitude": "28.1",
+    "latitude": "47.2",
+    "population": {
+      "-year": "87",
+      "#text": "663000"
+    }
+  }
+}
+{
+  "-car_code": "P",
+  "-area": "92080",
+  "-capital": "cty-Portugal-Lisbon",
+  "-memberships": "org-AfDB org-BIS org-CE org-CCC org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-EU org-FAO org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAIA org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNPREDEP org-UNPROFOR org-UPU org-WEU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Portugal",
+  "population": "9865114",
+  "population_growth": "0.02",
+  "infant_mortality": "7.6",
+  "gdp_total": "116200",
+  "gdp_agri": "6",
+  "gdp_ind": "35.8",
+  "gdp_serv": "58.2",
+  "inflation": "4.6",
+  "indep_date": "1140-01-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "97",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Portuguese"
+  },
+  "border": {
+    "-country": "E",
+    "-length": "1214"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-Portugal-2",
+      "-capital": "cty-cid-cia-Portugal-Aveiro",
+      "-country": "P",
+      "name": "Aveiro",
+      "area": "2808",
+      "population": "656000",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Aveiro",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-2",
+        "name": "Aveiro"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-3",
+      "-capital": "cty-cid-cia-Portugal-Beja",
+      "-country": "P",
+      "name": "Beja",
+      "area": "10225",
+      "population": "167900",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Beja",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-3",
+        "name": "Beja"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-4",
+      "-capital": "cty-cid-cia-Portugal-7",
+      "-country": "P",
+      "name": "Braga",
+      "area": "2673",
+      "population": "746100",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-7",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-4",
+        "name": "Braga",
+        "population": {
+          "-year": "81",
+          "#text": "63033"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-5",
+      "-capital": "cty-cid-cia-Portugal-Braganca",
+      "-country": "P",
+      "name": "Braganca",
+      "area": "6608",
+      "population": "158300",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Braganca",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-5",
+        "name": "Braganca"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-6",
+      "-capital": "cty-cid-cia-Portugal-CasteloBranco",
+      "-country": "P",
+      "name": "Castelo Branco",
+      "area": "6675",
+      "population": "214700",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-CasteloBranco",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-6",
+        "name": "Castelo Branco"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-7",
+      "-capital": "cty-cid-cia-Portugal-6",
+      "-country": "P",
+      "name": "Coimbra",
+      "area": "3947",
+      "population": "427600",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-6",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-7",
+        "name": "Coimbra",
+        "population": {
+          "-year": "81",
+          "#text": "74616"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-8",
+      "-capital": "cty-cid-cia-Portugal-Evora",
+      "-country": "P",
+      "name": "Evora",
+      "area": "7393",
+      "population": "173500",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Evora",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-8",
+        "name": "Evora"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-9",
+      "-capital": "cty-cid-cia-Portugal-Faro",
+      "-country": "P",
+      "name": "Algarve",
+      "area": "4960",
+      "population": "340100",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Faro",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-9",
+        "name": "Faro",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-10",
+      "-capital": "cty-cid-cia-Portugal-Guarda",
+      "-country": "P",
+      "name": "Guarda",
+      "area": "5518",
+      "population": "187800",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Guarda",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-10",
+        "name": "Guarda"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-11",
+      "-capital": "cty-cid-cia-Portugal-Leiria",
+      "-country": "P",
+      "name": "Leiria",
+      "area": "3515",
+      "population": "427800",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Leiria",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-11",
+        "name": "Leiria"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-12",
+      "-capital": "cty-Portugal-Lisbon",
+      "-country": "P",
+      "name": "Lisbon",
+      "area": "2761",
+      "population": "2063800",
+      "city": [
+        {
+          "-id": "cty-Portugal-Lisbon",
+          "-is_country_cap": "yes",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-12",
+          "name": "Lisbon",
+          "longitude": "-9.13333",
+          "latitude": "38.7167",
+          "population": {
+            "-year": "81",
+            "#text": "807937"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Tajo"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Portugal-4",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-12",
+          "name": "Amadora",
+          "population": {
+            "-year": "81",
+            "#text": "95518"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Portugal-9",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-12",
+          "name": "Barreiro",
+          "population": {
+            "-year": "81",
+            "#text": "50863"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Portugal-11",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-12",
+          "name": "Almada",
+          "population": {
+            "-year": "81",
+            "#text": "42607"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-13",
+      "-capital": "cty-cid-cia-Portugal-Portalegre",
+      "-country": "P",
+      "name": "Portalegre",
+      "area": "6065",
+      "population": "134300",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Portalegre",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-13",
+        "name": "Portalegre"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-14",
+      "-capital": "cty-cid-cia-Portugal-3",
+      "-country": "P",
+      "name": "Porto",
+      "area": "2395",
+      "population": "1622300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Portugal-3",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-14",
+          "name": "Porto",
+          "population": {
+            "-year": "81",
+            "#text": "327368"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Douro"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Portugal-8",
+          "-country": "P",
+          "-province": "prov-cid-cia-Portugal-14",
+          "name": "Vila Nova de Gaia",
+          "population": {
+            "-year": "81",
+            "#text": "62468"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Douro"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-15",
+      "-capital": "cty-cid-cia-Portugal-Santarem",
+      "-country": "P",
+      "name": "Santarem",
+      "area": "6747",
+      "population": "442700",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Santarem",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-15",
+        "name": "Santarem",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tajo"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-16",
+      "-capital": "cty-cid-cia-Portugal-5",
+      "-country": "P",
+      "name": "Setubal",
+      "area": "5064",
+      "population": "713700",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-5",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-16",
+        "name": "Setubal",
+        "population": {
+          "-year": "81",
+          "#text": "77885"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-17",
+      "-capital": "cty-cid-cia-Portugal-Viana",
+      "-country": "P",
+      "name": "Viana do Castelo",
+      "area": "2255",
+      "population": "248700",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Viana",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-17",
+        "name": "Viana do Castelo",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-18",
+      "-capital": "cty-cid-cia-Portugal-VilaReal",
+      "-country": "P",
+      "name": "Vila Real",
+      "area": "4328",
+      "population": "237100",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-VilaReal",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-18",
+        "name": "Vila Real"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-19",
+      "-capital": "cty-cid-cia-Portugal-Viseu",
+      "-country": "P",
+      "name": "Viseu",
+      "area": "5007",
+      "population": "401000",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-Viseu",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-19",
+        "name": "Viseu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-21",
+      "-capital": "cty-cid-cia-Portugal-PontaDelgada",
+      "-country": "P",
+      "name": "Azores",
+      "area": "2247",
+      "population": "236700",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-PontaDelgada",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-21",
+        "name": "Ponta Delgada",
+        "longitude": "-27.1",
+        "latitude": "38.35",
+        "population": {
+          "-year": "02",
+          "#text": "46102"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        },
+        "located_on": { "-island": "island-Sao_Miguel" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Portugal-22",
+      "-capital": "cty-cid-cia-Portugal-10",
+      "-country": "P",
+      "name": "Madeira",
+      "area": "794",
+      "population": "253000",
+      "city": {
+        "-id": "cty-cid-cia-Portugal-10",
+        "-country": "P",
+        "-province": "prov-cid-cia-Portugal-22",
+        "name": "Funchal",
+        "longitude": "-15.9",
+        "latitude": "31",
+        "population": {
+          "-year": "81",
+          "#text": "44111"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        },
+        "located_on": { "-island": "island-Madeira" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "SVAX",
+  "-area": "62049",
+  "-capital": "city-Longyearbyen-SVAX-SVAX",
+  "name": "Svalbard",
+  "population": "2116",
+  "population_growth": "-0.02",
+  "dependent": { "-country": "N" },
+  "government": "territory of Norway administered by the Ministry of Industry",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "55.4",
+      "#text": "Norwegian"
+    },
+    {
+      "-percentage": "44.3",
+      "#text": "Russian Ukrainian"
+    }
+  ],
+  "city": {
+    "-id": "city-Longyearbyen-SVAX-SVAX",
+    "-is_country_cap": "yes",
+    "-country": "SVAX",
+    "name": "Longyearbyen",
+    "longitude": "15.33",
+    "latitude": "78.13",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Europaeisches_Nordmeer"
+    },
+    "located_on": { "-island": "island-Svalbard" }
+  }
+}
+{
+  "-car_code": "GB",
+  "-area": "244820",
+  "-capital": "cty-cid-cia-United-Kingdom-2",
+  "-memberships": "org-AfDB org-AG org-AsDB org-BIS org-CDB org-C org-CE org-CCC org-ESCAP org-ECA org-ECE org-ECLAC org-EBRD org-EIB org-CERN org-ESA org-EU org-FAO org-G-5 org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-PCA org-SPC org-UN org-UNAVEM-III org-UNCRO org-UNFICYP org-UNIDO org-UNITAR org-UNIKOM org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNRWA org-UNU org-UPU org-WEU org-WCL org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "United Kingdom",
+  "population": "58489975",
+  "population_growth": "0.22",
+  "infant_mortality": "6.4",
+  "gdp_total": "1138400",
+  "gdp_agri": "1.7",
+  "gdp_ind": "27.7",
+  "gdp_serv": "70.6",
+  "inflation": "3.1",
+  "indep_date": "1801-01-01",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "81.5",
+      "#text": "English"
+    },
+    {
+      "-percentage": "2.4",
+      "#text": "Irish"
+    },
+    {
+      "-percentage": "1.9",
+      "#text": "Welsh"
+    },
+    {
+      "-percentage": "9.6",
+      "#text": "Scottish"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Ulster"
+    }
+  ],
+  "border": {
+    "-country": "IRL",
+    "-length": "360"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-United-Kingdom-2",
+      "-capital": "cty-cid-cia-United-Kingdom-9",
+      "-country": "GB",
+      "name": "Avon",
+      "area": "1346",
+      "population": "962000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-9",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-2",
+        "name": "Bristol",
+        "population": {
+          "-year": "94",
+          "#text": "399200"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-3",
+      "-capital": "cty-cid-cia-United-Kingdom-78",
+      "-country": "GB",
+      "name": "Bedfordshire",
+      "area": "1235",
+      "population": "534300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-78",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-3",
+          "name": "Bedford",
+          "population": {
+            "-year": "93",
+            "#text": "137300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-53",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-3",
+          "name": "Luton",
+          "population": {
+            "-year": "93",
+            "#text": "178600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-4",
+      "-capital": "cty-cid-cia-United-Kingdom-77",
+      "-country": "GB",
+      "name": "Berkshire",
+      "area": "1259",
+      "population": "752500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-77",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Reading",
+          "population": {
+            "-year": "93",
+            "#text": "137700"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Themse"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-73",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Wokingham",
+          "population": {
+            "-year": "93",
+            "#text": "142900"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-74",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Newbury",
+          "population": {
+            "-year": "93",
+            "#text": "141000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-81",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Windsor",
+          "population": {
+            "-year": "93",
+            "#text": "136700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-121",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Slough",
+          "population": {
+            "-year": "93",
+            "#text": "103500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-126",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-4",
+          "name": "Bracknell",
+          "population": {
+            "-year": "93",
+            "#text": "101900"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-5",
+      "-capital": "cty-cid-cia-United-Kingdom-65",
+      "-country": "GB",
+      "name": "Buckinghamshire",
+      "area": "1883",
+      "population": "640200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-65",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-5",
+          "name": "Aylesbury",
+          "population": {
+            "-year": "93",
+            "#text": "151600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-50",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-5",
+          "name": "Milton Keynes",
+          "population": {
+            "-year": "93",
+            "#text": "184400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-57",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-5",
+          "name": "Wycombe",
+          "population": {
+            "-year": "93",
+            "#text": "161400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-6",
+      "-capital": "cty-cid-cia-United-Kingdom-104",
+      "-country": "GB",
+      "name": "Cambridgeshire",
+      "area": "3409",
+      "population": "669900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-104",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-6",
+          "name": "Cambridge",
+          "population": {
+            "-year": "93",
+            "#text": "113800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-59",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-6",
+          "name": "Peterborough",
+          "population": {
+            "-year": "93",
+            "#text": "156400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-68",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-6",
+          "name": "Huntingdon",
+          "population": {
+            "-year": "93",
+            "#text": "148800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-7",
+      "-capital": "cty-cid-cia-United-Kingdom-93",
+      "-country": "GB",
+      "name": "Cheshire",
+      "area": "2329",
+      "population": "966500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-93",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-7",
+          "name": "Chester",
+          "population": {
+            "-year": "93",
+            "#text": "120800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-49",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-7",
+          "name": "Warrington",
+          "population": {
+            "-year": "93",
+            "#text": "185000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-66",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-7",
+          "name": "Macclesfield",
+          "population": {
+            "-year": "93",
+            "#text": "151400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-109",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-7",
+          "name": "Crewe",
+          "population": {
+            "-year": "93",
+            "#text": "109500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-8",
+      "-capital": "cty-cid-cia-United-Kingdom-72",
+      "-country": "GB",
+      "name": "Cleveland",
+      "area": "583",
+      "population": "557500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-72",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-8",
+          "name": "Middlesbrough",
+          "population": {
+            "-year": "93",
+            "#text": "145800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-54",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-8",
+          "name": "Stockton on Tees",
+          "population": {
+            "-year": "93",
+            "#text": "177800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-9",
+      "-capital": "cty-cid-cia-United-Kingdom-Truro",
+      "-country": "GB",
+      "name": "Cornwall",
+      "area": "3564",
+      "population": "475200",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Truro",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-9",
+        "name": "Truro",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-10",
+      "-capital": "cty-cid-cia-United-Kingdom-122",
+      "-country": "GB",
+      "name": "Cumbria",
+      "area": "6810",
+      "population": "489700",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-122",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-10",
+        "name": "Carlisle",
+        "population": {
+          "-year": "93",
+          "#text": "102900"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-11",
+      "-capital": "cty-cid-cia-United-Kingdom-Matlock",
+      "-country": "GB",
+      "name": "Derbyshire",
+      "area": "2631",
+      "population": "938800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-34",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-11",
+          "name": "Derby",
+          "population": {
+            "-year": "94",
+            "#text": "230500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-107",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-11",
+          "name": "Sutton in Ashfield",
+          "population": {
+            "-year": "93",
+            "#text": "109800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-128",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-11",
+          "name": "Chesterfield",
+          "population": {
+            "-year": "93",
+            "#text": "101200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Matlock",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-11",
+          "name": "Matlock",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-12",
+      "-capital": "cty-cid-cia-United-Kingdom-116",
+      "-country": "GB",
+      "name": "Devon",
+      "area": "6711",
+      "population": "1040000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-30",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-12",
+          "name": "Plymouth",
+          "population": {
+            "-year": "94",
+            "#text": "255800"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-116",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-12",
+          "name": "Exeter",
+          "population": {
+            "-year": "93",
+            "#text": "105100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-13",
+      "-capital": "cty-cid-cia-United-Kingdom-Dorchester",
+      "-country": "GB",
+      "name": "Dorset",
+      "area": "2654",
+      "population": "662900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-58",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-13",
+          "name": "Bournemouth",
+          "population": {
+            "-year": "93",
+            "#text": "159900"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-79",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-13",
+          "name": "Poole",
+          "population": {
+            "-year": "93",
+            "#text": "137200"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Dorchester",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-13",
+          "name": "Dorchester",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-14",
+      "-capital": "cty-cid-cia-United-Kingdom-Durham",
+      "-country": "GB",
+      "name": "Durham",
+      "area": "2436",
+      "population": "604300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-131",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-14",
+          "name": "Darlington",
+          "population": {
+            "-year": "93",
+            "#text": "100200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Durham",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-14",
+          "name": "Durham",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-15",
+      "-capital": "cty-cid-cia-United-Kingdom-Lewes",
+      "-country": "GB",
+      "name": "East Sussex",
+      "area": "1795",
+      "population": "716500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-63",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-15",
+          "name": "Brighton",
+          "population": {
+            "-year": "93",
+            "#text": "154400"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Lewes",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-15",
+          "name": "Lewes",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-16",
+      "-capital": "cty-cid-cia-United-Kingdom-60",
+      "-country": "GB",
+      "name": "Essex",
+      "area": "3672",
+      "population": "1548800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-60",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Chelmsford",
+          "population": {
+            "-year": "93",
+            "#text": "155700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-55",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Southend on Sea",
+          "population": {
+            "-year": "93",
+            "#text": "167500"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Nordsee"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Themse"
+            }
+          ],
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-56",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Basildon",
+          "population": {
+            "-year": "93",
+            "#text": "161700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-67",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Colchester",
+          "population": {
+            "-year": "93",
+            "#text": "149100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-91",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Braintree",
+          "population": {
+            "-year": "93",
+            "#text": "121800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-97",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-16",
+          "name": "Epping Forest",
+          "population": {
+            "-year": "93",
+            "#text": "118200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-17",
+      "-capital": "cty-cid-cia-United-Kingdom-Gloucester",
+      "-country": "GB",
+      "name": "Gloucestershire",
+      "area": "2643",
+      "population": "538800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-113",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-17",
+          "name": "Cheltenham",
+          "population": {
+            "-year": "93",
+            "#text": "106700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-115",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-17",
+          "name": "Stroud",
+          "population": {
+            "-year": "93",
+            "#text": "105400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Gloucester",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-17",
+          "name": "Gloucester",
+          "population": {
+            "-year": "93",
+            "#text": "104800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-18",
+      "-capital": "cty-cid-cia-United-Kingdom-2",
+      "-country": "GB",
+      "name": "Greater London",
+      "area": "1579",
+      "population": "6803100",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-18",
+        "name": "London",
+        "longitude": "0",
+        "latitude": "51.4833",
+        "population": {
+          "-year": "94",
+          "#text": "6967500"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Themse"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-19",
+      "-capital": "cty-cid-cia-United-Kingdom-8",
+      "-country": "GB",
+      "name": "Greater Manchester",
+      "area": "1287",
+      "population": "2561600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-8",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Manchester",
+          "population": {
+            "-year": "94",
+            "#text": "431100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-14",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Wigan",
+          "population": {
+            "-year": "94",
+            "#text": "310000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-23",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Stockport",
+          "population": {
+            "-year": "94",
+            "#text": "291400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-26",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Bolton",
+          "population": {
+            "-year": "94",
+            "#text": "265200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-33",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Salford",
+          "population": {
+            "-year": "94",
+            "#text": "230700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-36",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Tameside",
+          "population": {
+            "-year": "94",
+            "#text": "221800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-37",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Oldham",
+          "population": {
+            "-year": "94",
+            "#text": "220400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-38",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Trafford",
+          "population": {
+            "-year": "94",
+            "#text": "218100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-40",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Rochdale",
+          "population": {
+            "-year": "94",
+            "#text": "207100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-51",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-19",
+          "name": "Bury",
+          "population": {
+            "-year": "93",
+            "#text": "181400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-20",
+      "-capital": "cty-cid-cia-United-Kingdom-130",
+      "-country": "GB",
+      "name": "Hampshire",
+      "area": "3777",
+      "population": "1578700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-130",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Winchester",
+          "population": {
+            "-year": "93",
+            "#text": "100500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-39",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Southampton",
+          "population": {
+            "-year": "94",
+            "#text": "211700"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-47",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Portsmouth",
+          "population": {
+            "-year": "93",
+            "#text": "189100"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-70",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Basingstoke",
+          "population": {
+            "-year": "93",
+            "#text": "146500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-98",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Havant",
+          "population": {
+            "-year": "93",
+            "#text": "117500"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-108",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Eastleigh",
+          "population": {
+            "-year": "93",
+            "#text": "109600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-129",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-20",
+          "name": "Fareham",
+          "population": {
+            "-year": "93",
+            "#text": "101000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-21",
+      "-capital": "cty-cid-cia-United-Kingdom-Worcester",
+      "-country": "GB",
+      "name": "Hereford and Worcester",
+      "area": "3927",
+      "population": "696000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Worcester",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-21",
+        "name": "Worcester",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-22",
+      "-capital": "cty-cid-cia-United-Kingdom-Hertford",
+      "-country": "GB",
+      "name": "Hertfordshire",
+      "area": "1634",
+      "population": "989500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-88",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-22",
+          "name": "Saint Albans",
+          "population": {
+            "-year": "93",
+            "#text": "127700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Hertford",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-22",
+          "name": "Hertford",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-23",
+      "-capital": "cty-cid-cia-United-Kingdom-Hull",
+      "-country": "GB",
+      "name": "Humberside",
+      "area": "3512",
+      "population": "874400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-28",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-23",
+          "name": "Kingston upon Hull",
+          "population": {
+            "-year": "94",
+            "#text": "269100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-102",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-23",
+          "name": "Beverley",
+          "population": {
+            "-year": "93",
+            "#text": "115800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Hull",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-23",
+          "name": "Hull",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-24",
+      "-capital": "cty-cid-cia-United-Kingdom-80",
+      "-country": "GB",
+      "name": "Isle of Wight",
+      "area": "381",
+      "population": "126600"
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-25",
+      "-capital": "cty-cid-cia-United-Kingdom-76",
+      "-country": "GB",
+      "name": "Kent",
+      "area": "3731",
+      "population": "1538800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-76",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Maidstone",
+          "population": {
+            "-year": "93",
+            "#text": "138500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-44",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Rochester upon Medway",
+          "population": {
+            "-year": "94",
+            "#text": "148200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-83",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Canterbury",
+          "population": {
+            "-year": "93",
+            "#text": "132400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-100",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Swale",
+          "population": {
+            "-year": "93",
+            "#text": "117200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-110",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Sevenoaks",
+          "population": {
+            "-year": "93",
+            "#text": "109400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-114",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Dover",
+          "population": {
+            "-year": "93",
+            "#text": "106100"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Channel"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-125",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Tonbridge",
+          "population": {
+            "-year": "93",
+            "#text": "102100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-127",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-25",
+          "name": "Tunbridge Wells",
+          "population": {
+            "-year": "93",
+            "#text": "101800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-26",
+      "-capital": "cty-cid-cia-United-Kingdom-84",
+      "-country": "GB",
+      "name": "Lancashire",
+      "area": "3064",
+      "population": "1408300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-84",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-26",
+          "name": "Preston",
+          "population": {
+            "-year": "93",
+            "#text": "132200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-64",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-26",
+          "name": "Blackpool",
+          "population": {
+            "-year": "93",
+            "#text": "153600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-75",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-26",
+          "name": "Blackburn",
+          "population": {
+            "-year": "93",
+            "#text": "139500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-82",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-26",
+          "name": "Lancaster",
+          "population": {
+            "-year": "93",
+            "#text": "133600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-27",
+      "-capital": "cty-cid-cia-United-Kingdom-Leichester",
+      "-country": "GB",
+      "name": "Leicestershire",
+      "area": "2553",
+      "population": "890800",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Leichester",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-27",
+        "name": "Leichester",
+        "population": {
+          "-year": "94",
+          "#text": "293400"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-28",
+      "-capital": "cty-cid-cia-United-Kingdom-Lincoln",
+      "-country": "GB",
+      "name": "Lincolnshire",
+      "area": "5915",
+      "population": "592600",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Lincoln",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-28",
+        "name": "Lincoln",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-29",
+      "-capital": "cty-cid-cia-United-Kingdom-7",
+      "-country": "GB",
+      "name": "Merseyside",
+      "area": "652",
+      "population": "1441100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-7",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-29",
+          "name": "Liverpool",
+          "population": {
+            "-year": "94",
+            "#text": "474000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-11",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-29",
+          "name": "Wiral",
+          "population": {
+            "-year": "94",
+            "#text": "331100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-21",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-29",
+          "name": "Sefton",
+          "population": {
+            "-year": "94",
+            "#text": "292400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-52",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-29",
+          "name": "Saint Helens",
+          "population": {
+            "-year": "93",
+            "#text": "180200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-61",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-29",
+          "name": "Knowsley",
+          "population": {
+            "-year": "93",
+            "#text": "155300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-30",
+      "-capital": "cty-cid-cia-United-Kingdom-87",
+      "-country": "GB",
+      "name": "Norfolk",
+      "area": "5368",
+      "population": "759400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-87",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-30",
+          "name": "Norwich",
+          "population": {
+            "-year": "93",
+            "#text": "128100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-86",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-30",
+          "name": "Kings Lynn",
+          "population": {
+            "-year": "93",
+            "#text": "131600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-31",
+      "-capital": "cty-cid-cia-United-Kingdom-48",
+      "-country": "GB",
+      "name": "Northamptonshire",
+      "area": "2367",
+      "population": "587100",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-48",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-31",
+        "name": "Northampton",
+        "population": {
+          "-year": "93",
+          "#text": "187200"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-32",
+      "-capital": "cty-cid-cia-United-Kingdom-Newcastle",
+      "-country": "GB",
+      "name": "Northumberland",
+      "area": "5032",
+      "population": "307100"
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-33",
+      "-capital": "cty-cid-cia-United-Kingdom-Northallerton",
+      "-country": "GB",
+      "name": "North Yorkshire",
+      "area": "8309",
+      "population": "720900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-71",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-33",
+          "name": "Harrogate",
+          "population": {
+            "-year": "93",
+            "#text": "146500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-111",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-33",
+          "name": "Scarborough",
+          "population": {
+            "-year": "93",
+            "#text": "108700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-120",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-33",
+          "name": "York",
+          "population": {
+            "-year": "93",
+            "#text": "104000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Northallerton",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-33",
+          "name": "Northallerton",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-34",
+      "-capital": "cty-cid-cia-United-Kingdom-25",
+      "-country": "GB",
+      "name": "Nottinghamshire",
+      "area": "2164",
+      "population": "1015500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-25",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-34",
+          "name": "Nottingham",
+          "population": {
+            "-year": "94",
+            "#text": "282400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-118",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-34",
+          "name": "Newark on Trent",
+          "population": {
+            "-year": "93",
+            "#text": "104400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-124",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-34",
+          "name": "Mansfield",
+          "population": {
+            "-year": "93",
+            "#text": "102100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-35",
+      "-capital": "cty-cid-cia-United-Kingdom-85",
+      "-country": "GB",
+      "name": "Oxfordshire",
+      "area": "2608",
+      "population": "597700",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-85",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-35",
+        "name": "Oxford",
+        "population": {
+          "-year": "93",
+          "#text": "132000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Themse"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-36",
+      "-capital": "cty-cid-cia-United-Kingdom-Shrewsbury",
+      "-country": "GB",
+      "name": "Shropshire",
+      "area": "3490",
+      "population": "412500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Shrewsbury",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-36",
+        "name": "Shrewsbury",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-37",
+      "-capital": "cty-cid-cia-United-Kingdom-Taunton",
+      "-country": "GB",
+      "name": "Somerset",
+      "area": "3451",
+      "population": "469400",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Taunton",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-37",
+        "name": "Taunton",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-38",
+      "-capital": "cty-cid-cia-United-Kingdom-35",
+      "-country": "GB",
+      "name": "South Yorkshire",
+      "area": "1560",
+      "population": "1292700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-35",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-38",
+          "name": "Barnsley",
+          "population": {
+            "-year": "94",
+            "#text": "226500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-5",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-38",
+          "name": "Sheffield",
+          "population": {
+            "-year": "94",
+            "#text": "530100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-20",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-38",
+          "name": "Doncaster",
+          "population": {
+            "-year": "94",
+            "#text": "292500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-29",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-38",
+          "name": "Rotherham",
+          "population": {
+            "-year": "94",
+            "#text": "256300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-39",
+      "-capital": "cty-cid-cia-United-Kingdom-92",
+      "-country": "GB",
+      "name": "Staffordshire",
+      "area": "2716",
+      "population": "1047400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-92",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-39",
+          "name": "Stafford",
+          "population": {
+            "-year": "93",
+            "#text": "121500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-31",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-39",
+          "name": "Stoke on Trent",
+          "population": {
+            "-year": "94",
+            "#text": "254200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-90",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-39",
+          "name": "Newcastle under Lyme",
+          "population": {
+            "-year": "93",
+            "#text": "123000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-40",
+      "-capital": "cty-cid-cia-United-Kingdom-103",
+      "-country": "GB",
+      "name": "Suffolk",
+      "area": "3797",
+      "population": "661900",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-103",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-40",
+        "name": "Ipswich",
+        "population": {
+          "-year": "93",
+          "#text": "114800"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-41",
+      "-capital": "cty-cid-cia-United-Kingdom-Kingston",
+      "-country": "GB",
+      "name": "Surrey",
+      "area": "1679",
+      "population": "1035500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-89",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-41",
+          "name": "Guildford",
+          "population": {
+            "-year": "93",
+            "#text": "126200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-94",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-41",
+          "name": "Reigate",
+          "population": {
+            "-year": "93",
+            "#text": "118800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-99",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-41",
+          "name": "Elmbridge",
+          "population": {
+            "-year": "93",
+            "#text": "117300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Kingston",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-41",
+          "name": "Kingston",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-42",
+      "-capital": "cty-cid-cia-United-Kingdom-Newcastle",
+      "-country": "GB",
+      "name": "Tyne and Wear",
+      "area": "540",
+      "population": "1125600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-22",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-42",
+          "name": "Sunderland",
+          "population": {
+            "-year": "94",
+            "#text": "292200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Newcastle",
+          "-country": "GB",
+          "-is_state_cap": "yes",
+          "-province": "prov-cid-cia-United-Kingdom-42",
+          "name": "Newcastle upon Tyne",
+          "population": {
+            "-year": "94",
+            "#text": "283600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-41",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-42",
+          "name": "Gateshead",
+          "population": {
+            "-year": "94",
+            "#text": "202400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-43",
+      "-capital": "cty-cid-cia-United-Kingdom-95",
+      "-country": "GB",
+      "name": "Warwickshire",
+      "area": "1981",
+      "population": "489900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-95",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-43",
+          "name": "Warwick",
+          "population": {
+            "-year": "93",
+            "#text": "118600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-96",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-43",
+          "name": "Nuneaton",
+          "population": {
+            "-year": "93",
+            "#text": "118500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-112",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-43",
+          "name": "Stratford on Avon",
+          "population": {
+            "-year": "93",
+            "#text": "108600"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-44",
+      "-capital": "cty-cid-cia-United-Kingdom-3",
+      "-country": "GB",
+      "name": "West Midlands",
+      "area": "899",
+      "population": "2619000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-3",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Birmingham",
+          "longitude": "-1.93333",
+          "latitude": "52.4833",
+          "population": {
+            "-year": "94",
+            "#text": "1008400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-13",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Dudley",
+          "population": {
+            "-year": "94",
+            "#text": "312200"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-15",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Coventry",
+          "population": {
+            "-year": "94",
+            "#text": "302500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-27",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Walsall",
+          "population": {
+            "-year": "94",
+            "#text": "263900"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-32",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Wolverhampton",
+          "population": {
+            "-year": "94",
+            "#text": "245100"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-42",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Solihull",
+          "population": {
+            "-year": "94",
+            "#text": "202000"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-62",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "West Bromwich",
+          "population": {
+            "-year": "93",
+            "#text": "154500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-18",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-44",
+          "name": "Sandwell",
+          "population": {
+            "-year": "94",
+            "#text": "293700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-45",
+      "-capital": "cty-cid-cia-United-Kingdom-123",
+      "-country": "GB",
+      "name": "West Sussex",
+      "area": "1989",
+      "population": "713600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-123",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-45",
+          "name": "Chichester",
+          "population": {
+            "-year": "93",
+            "#text": "102500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-105",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-45",
+          "name": "Horsham",
+          "population": {
+            "-year": "93",
+            "#text": "112300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-46",
+      "-capital": "cty-cid-cia-United-Kingdom-12",
+      "-country": "GB",
+      "name": "West Yorkshire",
+      "area": "2039",
+      "population": "2066200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-12",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-46",
+          "name": "Wakefield",
+          "population": {
+            "-year": "94",
+            "#text": "317300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-4",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-46",
+          "name": "Leeds",
+          "population": {
+            "-year": "94",
+            "#text": "724400"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-6",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-46",
+          "name": "Bradford",
+          "population": {
+            "-year": "94",
+            "#text": "481700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-10",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-46",
+          "name": "Kirklees",
+          "population": {
+            "-year": "94",
+            "#text": "386900"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-69",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-46",
+          "name": "Huddersfield",
+          "population": {
+            "-year": "93",
+            "#text": "148500"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-47",
+      "-capital": "cty-cid-cia-United-Kingdom-Trowbridge",
+      "-country": "GB",
+      "name": "Wiltshire",
+      "area": "3480",
+      "population": "575100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-106",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-47",
+          "name": "Salisbury",
+          "population": {
+            "-year": "93",
+            "#text": "109800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Trowbridge",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-47",
+          "name": "Trowbridge",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-50",
+      "-capital": "cty-cid-cia-United-Kingdom-Newtown-St-Boswells",
+      "-country": "GB",
+      "name": "Borders",
+      "area": "4698",
+      "population": "105700",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Newtown-St-Boswells",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-50",
+        "name": "Newtown St. Boswells",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-51",
+      "-capital": "cty-cid-cia-United-Kingdom-Stirling",
+      "-country": "GB",
+      "name": "Central",
+      "area": "2700",
+      "population": "273400",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Stirling",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-51",
+        "name": "Stirling",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-52",
+      "-capital": "cty-cid-cia-United-Kingdom-Dumfries",
+      "-country": "GB",
+      "name": "Dumfries and Galloway",
+      "area": "6425",
+      "population": "147800",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Dumfries",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-52",
+        "name": "Dumfries",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-53",
+      "-capital": "cty-cid-cia-United-Kingdom-Glenrothes",
+      "-country": "GB",
+      "name": "Fife",
+      "area": "1319",
+      "population": "352100",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Glenrothes",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-53",
+        "name": "Glenrothes",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-54",
+      "-capital": "cty-cid-cia-United-Kingdom-135",
+      "-country": "GB",
+      "name": "Grampian",
+      "area": "8752",
+      "population": "532500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-135",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-54",
+        "name": "Aberdeen",
+        "population": {
+          "-year": "95",
+          "#text": "219100"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-55",
+      "-capital": "cty-cid-cia-United-Kingdom-Inverness",
+      "-country": "GB",
+      "name": "Highland",
+      "area": "26137",
+      "population": "207500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Inverness",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-55",
+        "name": "Inverness",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-56",
+      "-capital": "cty-cid-cia-United-Kingdom-134",
+      "-country": "GB",
+      "name": "Lothian",
+      "area": "1770",
+      "population": "758600",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-134",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-56",
+        "name": "Edinburgh",
+        "longitude": "-3.18333",
+        "latitude": "55.9167",
+        "population": {
+          "-year": "95",
+          "#text": "447600"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-57",
+      "-capital": "cty-cid-cia-United-Kingdom-133",
+      "-country": "GB",
+      "name": "Strathclyde",
+      "area": "13773",
+      "population": "2287800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-133",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-57",
+          "name": "Glasgow",
+          "longitude": "-4.28333",
+          "latitude": "55.8667",
+          "population": {
+            "-year": "95",
+            "#text": "674800"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-43",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-57",
+          "name": "Renfrew",
+          "population": {
+            "-year": "94",
+            "#text": "201700"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-58",
+      "-capital": "cty-cid-cia-United-Kingdom-136",
+      "-country": "GB",
+      "name": "Tayside",
+      "area": "7643",
+      "population": "395000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-136",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-58",
+        "name": "Dundee",
+        "population": {
+          "-year": "95",
+          "#text": "167600"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-62",
+      "-capital": "cty-cid-cia-United-Kingdom-Colwyn-Bay",
+      "-country": "GB",
+      "name": "Aberconwy and Colwyn",
+      "area": "1130",
+      "population": "110700",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Colwyn-Bay",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-62",
+        "name": "Colwyn Bay",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Irische_See"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-63",
+      "-capital": "cty-cid-cia-United-Kingdom-Llangefni",
+      "-country": "GB",
+      "name": "Anglesey",
+      "area": "719",
+      "population": "68500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Llangefni",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-63",
+        "name": "Llangefni",
+        "located_on": { "-island": "island-Anglesey" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-64",
+      "-capital": "cty-cid-cia-United-Kingdom-Ebbw-Vale",
+      "-country": "GB",
+      "name": "Blaenau Gwent",
+      "area": "109",
+      "population": "73300",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Ebbw-Vale",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-64",
+        "name": "Ebbw Vale",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-65",
+      "-capital": "cty-cid-cia-United-Kingdom-Bridgend",
+      "-country": "GB",
+      "name": "Bridgend",
+      "area": "246",
+      "population": "130900",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Bridgend",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-65",
+        "name": "Bridgend",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-66",
+      "-capital": "cty-cid-cia-United-Kingdom-Ystrad-Fawr",
+      "-country": "GB",
+      "name": "Caerphilly",
+      "area": "279",
+      "population": "171000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Ystrad-Fawr",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-66",
+        "name": "Ystrad Fawr",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-67",
+      "-capital": "cty-cid-cia-United-Kingdom-16",
+      "-country": "GB",
+      "name": "Cardiff",
+      "area": "139",
+      "population": "306600",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-16",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-67",
+        "name": "Cardiff",
+        "longitude": "-3.16667",
+        "latitude": "51.4667",
+        "population": {
+          "-year": "94",
+          "#text": "300000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Irische_See"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-68",
+      "-capital": "cty-cid-cia-United-Kingdom-Carmarthen",
+      "-country": "GB",
+      "name": "Carmarthenshire",
+      "area": "2398",
+      "population": "169000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Carmarthen",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-68",
+        "name": "Carmarthen",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-69",
+      "-capital": "cty-cid-cia-United-Kingdom-Aberystwyth",
+      "-country": "GB",
+      "name": "Ceredigion",
+      "area": "1797",
+      "population": "69700",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Aberystwyth",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-69",
+        "name": "Aberystwyth",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-70",
+      "-capital": "cty-cid-cia-United-Kingdom-Ruthin",
+      "-country": "GB",
+      "name": "Denbighshire",
+      "area": "844",
+      "population": "91300",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Ruthin",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-70",
+        "name": "Ruthin",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-71",
+      "-capital": "cty-cid-cia-United-Kingdom-Mold",
+      "-country": "GB",
+      "name": "Flintshire",
+      "area": "437",
+      "population": "145300",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Mold",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-71",
+        "name": "Mold",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-72",
+      "-capital": "cty-cid-cia-United-Kingdom-Caernarfon",
+      "-country": "GB",
+      "name": "Gwynedd",
+      "area": "2548",
+      "population": "117000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Caernarfon",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-72",
+        "name": "Caernarfon",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-73",
+      "-capital": "cty-cid-cia-United-Kingdom-Merthyr-Tydfil",
+      "-country": "GB",
+      "name": "Merthyr Tydfil",
+      "area": "111",
+      "population": "59500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Merthyr-Tydfil",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-73",
+        "name": "Merthyr Tydfil",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-74",
+      "-capital": "cty-cid-cia-United-Kingdom-Cwmbran",
+      "-country": "GB",
+      "name": "Monmouthshire",
+      "area": "851",
+      "population": "84200",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Cwmbran",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-74",
+        "name": "Cwmbran",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-75",
+      "-capital": "cty-cid-cia-United-Kingdom-Port-Talbot",
+      "-country": "GB",
+      "name": "Neath and Port Talbot",
+      "area": "441",
+      "population": "140100",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Port-Talbot",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-75",
+        "name": "Port Talbot",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-76",
+      "-capital": "cty-cid-cia-United-Kingdom-80",
+      "-country": "GB",
+      "name": "Newport",
+      "area": "191",
+      "population": "137400",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-80",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-76",
+        "name": "Newport",
+        "population": {
+          "-year": "93",
+          "#text": "137000"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-77",
+      "-capital": "cty-cid-cia-United-Kingdom-Haverfordwest",
+      "-country": "GB",
+      "name": "Pembrokeshire",
+      "area": "1590",
+      "population": "113600",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Haverfordwest",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-77",
+        "name": "Haverfordwest",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-78",
+      "-capital": "cty-cid-cia-United-Kingdom-Llandrindod-Wells",
+      "-country": "GB",
+      "name": "Powys",
+      "area": "5204",
+      "population": "121800",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Llandrindod-Wells",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-78",
+        "name": "Llandrindod Wells",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-79",
+      "-capital": "cty-cid-cia-United-Kingdom-Rhondda",
+      "-country": "GB",
+      "name": "Rhondda Cynon Taff",
+      "area": "424",
+      "population": "239000",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Rhondda",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-79",
+        "name": "Rhondda",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-80",
+      "-capital": "cty-cid-cia-United-Kingdom-46",
+      "-country": "GB",
+      "name": "Swansea",
+      "area": "378",
+      "population": "230900",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-46",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-80",
+        "name": "Swansea",
+        "population": {
+          "-year": "93",
+          "#text": "189300"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-81",
+      "-capital": "cty-cid-cia-United-Kingdom-Pontypool",
+      "-country": "GB",
+      "name": "Torfaen",
+      "area": "126",
+      "population": "90600",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Pontypool",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-81",
+        "name": "Pontypool",
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-82",
+      "-capital": "cty-cid-cia-United-Kingdom-Barry",
+      "-country": "GB",
+      "name": "Vale of Glamorgan",
+      "area": "337",
+      "population": "119200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Kingdom-119",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-82",
+          "name": "Rhymney Valley",
+          "population": {
+            "-year": "93",
+            "#text": "104300"
+          },
+          "located_on": { "-island": "island-GreatBritain" }
+        },
+        {
+          "-id": "cty-cid-cia-United-Kingdom-Barry",
+          "-is_state_cap": "yes",
+          "-country": "GB",
+          "-province": "prov-cid-cia-United-Kingdom-82",
+          "name": "Barry",
+          "located_on": { "-island": "island-GreatBritain" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-83",
+      "-capital": "cty-cid-cia-United-Kingdom-Wrexham",
+      "-country": "GB",
+      "name": "Wrexham",
+      "area": "499",
+      "population": "123500",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-Wrexham",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-83",
+        "name": "Wrexham",
+        "population": {
+          "-year": "93",
+          "#text": "117100"
+        },
+        "located_on": { "-island": "island-GreatBritain" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-85",
+      "-capital": "cty-cid-cia-United-Kingdom-17",
+      "-country": "GB",
+      "name": "Northern Ireland",
+      "area": "14120",
+      "population": "1594400",
+      "city": {
+        "-id": "cty-cid-cia-United-Kingdom-17",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-85",
+        "name": "Belfast",
+        "longitude": "-5.91667",
+        "latitude": "54.6",
+        "population": {
+          "-year": "94",
+          "#text": "297100"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Irische_See"
+        },
+        "located_on": { "-island": "island-Ireland" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-86",
+      "-capital": "cty-United-Kingdom-Kirkwall",
+      "-country": "GB",
+      "name": "Orkneys",
+      "area": "992",
+      "population": "19590",
+      "city": {
+        "-id": "cty-United-Kingdom-Kirkwall",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-86",
+        "name": "Kirkwall",
+        "longitude": "-3",
+        "latitude": "59",
+        "population": {
+          "-year": "04",
+          "#text": "6330"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        },
+        "located_on": { "-island": "island-Orkney_Mainland" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-87",
+      "-capital": "cty-United-Kingdom-Stornoway",
+      "-country": "GB",
+      "name": "Outer Hebrides",
+      "area": "3071",
+      "population": "26370",
+      "city": {
+        "-id": "cty-United-Kingdom-Stornoway",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-87",
+        "name": "Stornoway",
+        "longitude": "-6.4",
+        "latitude": "58.2",
+        "population": {
+          "-year": "04",
+          "#text": "8000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        },
+        "located_on": { "-island": "island-LewisHarris" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Kingdom-88",
+      "-capital": "cty-United-Kingdom-Lerwick",
+      "-country": "GB",
+      "name": "Shetland",
+      "area": "1466",
+      "population": "22000",
+      "city": {
+        "-id": "cty-United-Kingdom-Lerwick",
+        "-is_state_cap": "yes",
+        "-country": "GB",
+        "-province": "prov-cid-cia-United-Kingdom-88",
+        "name": "Lerwick",
+        "longitude": "-1.15",
+        "latitude": "60.15",
+        "population": {
+          "-year": "04",
+          "#text": "6570"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Nordsee"
+        },
+        "located_on": { "-island": "island-Shetland_Mainland" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "AFG",
+  "-area": "647500",
+  "-capital": "cty-Afghanistan-Kabul",
+  "-memberships": "org-AsDB org-CP org-ESCAP org-ECO org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO org-WToO",
+  "name": "Afghanistan",
+  "population": "22664136",
+  "population_growth": "4.78",
+  "infant_mortality": "149.7",
+  "gdp_total": "12800",
+  "gdp_agri": "65",
+  "gdp_ind": "15",
+  "gdp_serv": "20",
+  "indep_date": "1919-08-19",
+  "government": "transitional government",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "25",
+      "#text": "Tajik"
+    },
+    {
+      "-percentage": "38",
+      "#text": "Pashtun"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "19",
+      "#text": "Hazara"
+    }
+  ],
+  "religions": {
+    "-percentage": "99",
+    "#text": "Muslim"
+  },
+  "languages": [
+    {
+      "-percentage": "11",
+      "#text": "Turkic"
+    },
+    {
+      "-percentage": "35",
+      "#text": "Pashtu"
+    },
+    {
+      "-percentage": "50",
+      "#text": "Afghan Persian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "76"
+    },
+    {
+      "-country": "IR",
+      "-length": "936"
+    },
+    {
+      "-country": "PK",
+      "-length": "2430"
+    },
+    {
+      "-country": "TAD",
+      "-length": "1206"
+    },
+    {
+      "-country": "TM",
+      "-length": "744"
+    },
+    {
+      "-country": "UZB",
+      "-length": "137"
+    }
+  ],
+  "city": {
+    "-id": "cty-Afghanistan-Kabul",
+    "-is_country_cap": "yes",
+    "-country": "AFG",
+    "name": "Kabul",
+    "longitude": "69.2",
+    "latitude": "34.5833",
+    "population": {
+      "-year": "87",
+      "#text": "892000"
+    }
+  }
+}
+{
+  "-car_code": "TJ",
+  "-area": "9596960",
+  "-capital": "cty-cid-cia-China-3",
+  "-memberships": "org-AfDB org-APEC org-AsDB org-Mekong-Group org-CCC org-ESCAP org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-LAIA org-NAM org-PCA org-UN org-UNESCO org-UNIDO org-UNITAR org-UNIKOM org-MINURSO org-UNOMIL org-UNHCR org-UNTSO org-UNU org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "China",
+  "population": "1210004956",
+  "population_growth": "0.98",
+  "infant_mortality": "39.6",
+  "gdp_total": "3500000",
+  "gdp_agri": "19",
+  "gdp_ind": "48",
+  "gdp_serv": "33",
+  "inflation": "10.1",
+  "government": "Communist state",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "91.9",
+    "#text": "Han Chinese"
+  },
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "R",
+      "-length": "3645"
+    },
+    {
+      "-country": "AFG",
+      "-length": "76"
+    },
+    {
+      "-country": "PK",
+      "-length": "523"
+    },
+    {
+      "-country": "TAD",
+      "-length": "414"
+    },
+    {
+      "-country": "MYA",
+      "-length": "2185"
+    },
+    {
+      "-country": "IND",
+      "-length": "3380"
+    },
+    {
+      "-country": "BHT",
+      "-length": "470"
+    },
+    {
+      "-country": "LAO",
+      "-length": "423"
+    },
+    {
+      "-country": "VN",
+      "-length": "1281"
+    },
+    {
+      "-country": "KAZ",
+      "-length": "1533"
+    },
+    {
+      "-country": "NOK",
+      "-length": "1416"
+    },
+    {
+      "-country": "KGZ",
+      "-length": "858"
+    },
+    {
+      "-country": "MNG",
+      "-length": "4673"
+    },
+    {
+      "-country": "MACX",
+      "-length": "0.34"
+    },
+    {
+      "-country": "HONX",
+      "-length": "30"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-China-2",
+      "-capital": "cty-cid-cia-China-42",
+      "-country": "TJ",
+      "name": "Anhui",
+      "area": "139000",
+      "population": "59550000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-42",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Hefei",
+          "longitude": "117",
+          "latitude": "32",
+          "population": {
+            "-year": "91",
+            "#text": "1000000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-34",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Huainan",
+          "population": {
+            "-year": "91",
+            "#text": "1200000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-246",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Suzhou",
+          "population": {
+            "-year": "90",
+            "#text": "151862"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-73",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Bengbu",
+          "population": {
+            "-year": "90",
+            "#text": "449245"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-76",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Wuhu",
+          "population": {
+            "-year": "90",
+            "#text": "425740"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-94",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Huaibei",
+          "population": {
+            "-year": "90",
+            "#text": "366549"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-120",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Maanshan",
+          "population": {
+            "-year": "90",
+            "#text": "305421"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-147",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Anqing",
+          "population": {
+            "-year": "90",
+            "#text": "250718"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-159",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Tongling",
+          "population": {
+            "-year": "90",
+            "#text": "228017"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-213",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Fuyang",
+          "population": {
+            "-year": "90",
+            "#text": "179572"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-257",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Luan",
+          "population": {
+            "-year": "90",
+            "#text": "144248"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-292",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Chuxian",
+          "population": {
+            "-year": "90",
+            "#text": "125341"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-295",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Chaoxian",
+          "population": {
+            "-year": "90",
+            "#text": "123676"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-311",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-2",
+          "name": "Xuancheng",
+          "population": {
+            "-year": "90",
+            "#text": "112673"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-3",
+      "-capital": "cty-cid-cia-China-45",
+      "-country": "TJ",
+      "name": "Fujian",
+      "area": "120000",
+      "population": "31830000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-45",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Fuzhou",
+          "longitude": "119",
+          "latitude": "26",
+          "population": {
+            "-year": "90",
+            "#text": "874809"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-92",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Xiamen",
+          "population": {
+            "-year": "90",
+            "#text": "368786"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-187",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Nanping",
+          "population": {
+            "-year": "90",
+            "#text": "195064"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-202",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Quanzhou",
+          "population": {
+            "-year": "90",
+            "#text": "185154"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-207",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Zhangzhou",
+          "population": {
+            "-year": "90",
+            "#text": "181424"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-237",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Sanming",
+          "population": {
+            "-year": "90",
+            "#text": "160691"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-269",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Longyan",
+          "population": {
+            "-year": "90",
+            "#text": "134481"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-314",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-3",
+          "name": "Yongan",
+          "population": {
+            "-year": "90",
+            "#text": "111762"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-4",
+      "-capital": "cty-cid-cia-China-23",
+      "-country": "TJ",
+      "name": "Gansu",
+      "area": "450000",
+      "population": "23780000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-23",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-4",
+          "name": "Lanzhou",
+          "longitude": "104",
+          "latitude": "36",
+          "population": {
+            "-year": "91",
+            "#text": "1510000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-151",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-4",
+          "name": "Tianshui",
+          "population": {
+            "-year": "90",
+            "#text": "244974"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-180",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-4",
+          "name": "Baiyin",
+          "population": {
+            "-year": "90",
+            "#text": "204970"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-275",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-4",
+          "name": "Wuwei",
+          "population": {
+            "-year": "90",
+            "#text": "133101"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-318",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-4",
+          "name": "Yumen",
+          "population": {
+            "-year": "90",
+            "#text": "109234"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-5",
+      "-capital": "cty-cid-cia-China-8",
+      "-country": "TJ",
+      "name": "Guangdong",
+      "area": "186000",
+      "population": "66890000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-8",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Guangzhou",
+          "longitude": "114",
+          "latitude": "23",
+          "population": {
+            "-year": "91",
+            "#text": "3580000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-56",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Shantou",
+          "population": {
+            "-year": "90",
+            "#text": "578630"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-86",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Zhanjiang",
+          "population": {
+            "-year": "90",
+            "#text": "400997"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-105",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Shenzhen",
+          "population": {
+            "-year": "90",
+            "#text": "350727"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-108",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Shaoguan",
+          "population": {
+            "-year": "90",
+            "#text": "350043"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-117",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Chaozhou",
+          "population": {
+            "-year": "90",
+            "#text": "313469"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-119",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Dongguan",
+          "population": {
+            "-year": "90",
+            "#text": "308669"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-121",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Foshan",
+          "population": {
+            "-year": "90",
+            "#text": "303160"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-134",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Zhongshan",
+          "population": {
+            "-year": "90",
+            "#text": "278829"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-158",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Jiangmen",
+          "population": {
+            "-year": "90",
+            "#text": "230587"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-168",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Yangjiang",
+          "population": {
+            "-year": "90",
+            "#text": "215196"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-188",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Zhaoqing",
+          "population": {
+            "-year": "90",
+            "#text": "194784"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-214",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Maoming",
+          "population": {
+            "-year": "90",
+            "#text": "178683"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-228",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Zhuhai",
+          "population": {
+            "-year": "90",
+            "#text": "164747"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-236",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Huizhou",
+          "population": {
+            "-year": "90",
+            "#text": "161023"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-280",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Meixian",
+          "population": {
+            "-year": "90",
+            "#text": "132156"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-301",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Heyuan",
+          "population": {
+            "-year": "90",
+            "#text": "120101"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-321",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-5",
+          "name": "Shanwei",
+          "population": {
+            "-year": "90",
+            "#text": "107847"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-6",
+      "-capital": "cty-cid-cia-China-21",
+      "-country": "TJ",
+      "name": "Guizhou",
+      "area": "170000",
+      "population": "34580000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-21",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Guiyang",
+          "longitude": "106",
+          "latitude": "26",
+          "population": {
+            "-year": "91",
+            "#text": "1530000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-97",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Liupanshui",
+          "population": {
+            "-year": "90",
+            "#text": "363954"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-141",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Zunyi",
+          "population": {
+            "-year": "90",
+            "#text": "261862"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-221",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Anshun",
+          "population": {
+            "-year": "90",
+            "#text": "174142"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-276",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Duyun",
+          "population": {
+            "-year": "90",
+            "#text": "132971"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-309",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-6",
+          "name": "Kaili",
+          "population": {
+            "-year": "90",
+            "#text": "113958"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-7",
+      "-capital": "cty-cid-cia-China-133",
+      "-country": "TJ",
+      "name": "Hainan",
+      "area": "34000",
+      "population": "7110000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-133",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-7",
+          "name": "Haikou",
+          "longitude": "110.3",
+          "latitude": "20.05",
+          "population": {
+            "-year": "90",
+            "#text": "280153"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-SouthChinaSea"
+          },
+          "located_on": { "-island": "island-Hainan" }
+        },
+        {
+          "-id": "cty-cid-cia-China-332",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-7",
+          "name": "Sanya",
+          "population": {
+            "-year": "90",
+            "#text": "102820"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-SouthChinaSea"
+          },
+          "located_on": { "-island": "island-Hainan" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-8",
+      "-capital": "cty-cid-cia-China-31",
+      "-country": "TJ",
+      "name": "Hebei",
+      "area": "190000",
+      "population": "63880000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-31",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Shijiazhuang",
+          "longitude": "114.5",
+          "latitude": "38",
+          "population": {
+            "-year": "91",
+            "#text": "1320000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-24",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Tangshan",
+          "population": {
+            "-year": "91",
+            "#text": "1500000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-38",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Handan",
+          "population": {
+            "-year": "91",
+            "#text": "1110000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-61",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Zhangjiakou",
+          "population": {
+            "-year": "90",
+            "#text": "529136"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-68",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Baoding",
+          "population": {
+            "-year": "90",
+            "#text": "483155"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-95",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Qinhuangdao",
+          "population": {
+            "-year": "90",
+            "#text": "364972"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-123",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Xingtai",
+          "population": {
+            "-year": "90",
+            "#text": "302789"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-150",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Chengde",
+          "population": {
+            "-year": "90",
+            "#text": "246799"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-306",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Renqiu",
+          "population": {
+            "-year": "90",
+            "#text": "114256"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-313",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Quzhou",
+          "population": {
+            "-year": "90",
+            "#text": "112373"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-329",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-8",
+          "name": "Hengshui",
+          "population": {
+            "-year": "90",
+            "#text": "104269"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-9",
+      "-capital": "cty-cid-cia-China-10",
+      "-country": "TJ",
+      "name": "Heilongjiang",
+      "area": "469000",
+      "population": "36720000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-10",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Harbin",
+          "longitude": "126.4",
+          "latitude": "45.4",
+          "population": {
+            "-year": "91",
+            "#text": "2830000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-26",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Qiqihar",
+          "population": {
+            "-year": "91",
+            "#text": "1380000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-48",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Yichun",
+          "population": {
+            "-year": "90",
+            "#text": "795789"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-51",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Jixi",
+          "population": {
+            "-year": "90",
+            "#text": "683885"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-52",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Daqing",
+          "population": {
+            "-year": "90",
+            "#text": "657297"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-57",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Mudanjiang",
+          "population": {
+            "-year": "90",
+            "#text": "571705"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-63",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Hegang",
+          "population": {
+            "-year": "90",
+            "#text": "522747"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-65",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Jiamusi",
+          "population": {
+            "-year": "90",
+            "#text": "493409"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-88",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Shuangyashan",
+          "population": {
+            "-year": "90",
+            "#text": "386081"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-160",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Suihua",
+          "population": {
+            "-year": "90",
+            "#text": "227881"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-167",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Shangzhi",
+          "population": {
+            "-year": "90",
+            "#text": "215373"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-169",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Qitaihe",
+          "population": {
+            "-year": "90",
+            "#text": "214957"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-181",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Beian",
+          "population": {
+            "-year": "90",
+            "#text": "204899"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-185",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Acheng",
+          "population": {
+            "-year": "90",
+            "#text": "197595"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-211",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Zhaodong",
+          "population": {
+            "-year": "90",
+            "#text": "179976"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-222",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Fuling",
+          "population": {
+            "-year": "90",
+            "#text": "173878"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-258",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Shuangcheng",
+          "population": {
+            "-year": "90",
+            "#text": "142659"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-267",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Anda",
+          "population": {
+            "-year": "90",
+            "#text": "136446"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-277",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-9",
+          "name": "Mishan",
+          "population": {
+            "-year": "90",
+            "#text": "132744"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-10",
+      "-capital": "cty-cid-cia-China-20",
+      "-country": "TJ",
+      "name": "Henan",
+      "area": "167000",
+      "population": "90270000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-20",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Zhengzhou",
+          "longitude": "114",
+          "latitude": "34.5",
+          "population": {
+            "-year": "91",
+            "#text": "1710000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-36",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Luoyang",
+          "population": {
+            "-year": "91",
+            "#text": "1190000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-64",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Kaifeng",
+          "population": {
+            "-year": "90",
+            "#text": "507763"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-70",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Xinxiang",
+          "population": {
+            "-year": "90",
+            "#text": "473762"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-79",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Anyang",
+          "population": {
+            "-year": "90",
+            "#text": "420332"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-81",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Pingdingshan",
+          "population": {
+            "-year": "90",
+            "#text": "410775"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-84",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Jiaozuo",
+          "population": {
+            "-year": "90",
+            "#text": "409100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-152",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Nanyang",
+          "population": {
+            "-year": "90",
+            "#text": "243303"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-172",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Hebi",
+          "population": {
+            "-year": "90",
+            "#text": "212976"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-176",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Xuchang",
+          "population": {
+            "-year": "90",
+            "#text": "208815"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-192",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Xinyang",
+          "population": {
+            "-year": "90",
+            "#text": "192509"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-217",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Puyang",
+          "population": {
+            "-year": "90",
+            "#text": "175988"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-227",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Shangqiu",
+          "population": {
+            "-year": "90",
+            "#text": "164880"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-253",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Zhoukou",
+          "population": {
+            "-year": "90",
+            "#text": "146288"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-291",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Luohe",
+          "population": {
+            "-year": "90",
+            "#text": "126438"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-298",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Zhumadian",
+          "population": {
+            "-year": "90",
+            "#text": "123232"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-300",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-10",
+          "name": "Sanmenxia",
+          "population": {
+            "-year": "90",
+            "#text": "120523"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-11",
+      "-capital": "cty-cid-cia-China-7",
+      "-country": "TJ",
+      "name": "Hubei",
+      "area": "187400",
+      "population": "57190000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-7",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Wuhan",
+          "longitude": "114.15",
+          "latitude": "30.45",
+          "population": {
+            "-year": "91",
+            "#text": "3750000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-71",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Huangshi",
+          "population": {
+            "-year": "90",
+            "#text": "457601"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-82",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Xiangfan",
+          "population": {
+            "-year": "90",
+            "#text": "410407"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-91",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Yichang",
+          "population": {
+            "-year": "90",
+            "#text": "371601"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-129",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Shashi",
+          "population": {
+            "-year": "90",
+            "#text": "281352"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-135",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Shiyan",
+          "population": {
+            "-year": "90",
+            "#text": "273786"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-153",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Cangzhou",
+          "population": {
+            "-year": "90",
+            "#text": "242708"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-195",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Honghu",
+          "population": {
+            "-year": "90",
+            "#text": "190772"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-196",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Echeng",
+          "population": {
+            "-year": "90",
+            "#text": "190123"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-200",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Tianmen",
+          "population": {
+            "-year": "90",
+            "#text": "186332"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-226",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Xiaogan",
+          "population": {
+            "-year": "90",
+            "#text": "166280"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-233",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Zaoyang",
+          "population": {
+            "-year": "90",
+            "#text": "162198"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-265",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Xianning",
+          "population": {
+            "-year": "90",
+            "#text": "136811"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-296",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Laohekou",
+          "population": {
+            "-year": "90",
+            "#text": "123366"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-304",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Puqi",
+          "population": {
+            "-year": "90",
+            "#text": "117264"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-328",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Shishou",
+          "population": {
+            "-year": "90",
+            "#text": "104571"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-333",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-11",
+          "name": "Guangshui",
+          "population": {
+            "-year": "90",
+            "#text": "102770"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-12",
+      "-capital": "cty-cid-cia-China-30",
+      "-country": "TJ",
+      "name": "Hunan",
+      "area": "210000",
+      "population": "63550000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-30",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Changsha",
+          "longitude": "113.5",
+          "latitude": "28.1",
+          "population": {
+            "-year": "91",
+            "#text": "1330000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-67",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Hengyang",
+          "population": {
+            "-year": "90",
+            "#text": "487148"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-74",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Xiangtan",
+          "population": {
+            "-year": "90",
+            "#text": "441968"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-83",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Zhuzhou",
+          "population": {
+            "-year": "90",
+            "#text": "409924"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-122",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Yueyang",
+          "population": {
+            "-year": "90",
+            "#text": "302800"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-124",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Changde",
+          "population": {
+            "-year": "90",
+            "#text": "301276"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-148",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Shaoyang",
+          "population": {
+            "-year": "90",
+            "#text": "247227"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-201",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Yiyang",
+          "population": {
+            "-year": "90",
+            "#text": "185818"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-263",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Lengshuijiang",
+          "population": {
+            "-year": "90",
+            "#text": "137994"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-284",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Leiyang",
+          "population": {
+            "-year": "90",
+            "#text": "130115"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-287",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Loudi",
+          "population": {
+            "-year": "90",
+            "#text": "128418"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-290",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Huaihua",
+          "population": {
+            "-year": "90",
+            "#text": "126785"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-315",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Zixing",
+          "population": {
+            "-year": "90",
+            "#text": "110048"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-319",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Liling",
+          "population": {
+            "-year": "90",
+            "#text": "108504"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-323",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-12",
+          "name": "Yuanjiang",
+          "population": {
+            "-year": "90",
+            "#text": "107004"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-13",
+      "-capital": "cty-cid-cia-China-13",
+      "-country": "TJ",
+      "name": "Jiangsu",
+      "area": "102600",
+      "population": "70210000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-13",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Nanjing",
+          "longitude": "119",
+          "latitude": "32.5",
+          "population": {
+            "-year": "91",
+            "#text": "2500000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-46",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Wuxi",
+          "population": {
+            "-year": "90",
+            "#text": "826833"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-47",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Xuzhou",
+          "population": {
+            "-year": "90",
+            "#text": "805695"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-50",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Suzhou",
+          "population": {
+            "-year": "90",
+            "#text": "706459"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-60",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Changzhou",
+          "population": {
+            "-year": "90",
+            "#text": "531470"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-93",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Zhenjiang",
+          "population": {
+            "-year": "90",
+            "#text": "368316"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-103",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Lianyungang",
+          "population": {
+            "-year": "90",
+            "#text": "354139"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-109",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Nantong",
+          "population": {
+            "-year": "90",
+            "#text": "343341"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-118",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Yangzhou",
+          "population": {
+            "-year": "90",
+            "#text": "312892"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-126",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Yancheng",
+          "population": {
+            "-year": "90",
+            "#text": "296831"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-155",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Huaiyin",
+          "population": {
+            "-year": "90",
+            "#text": "239675"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-171",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Jiangyin",
+          "population": {
+            "-year": "90",
+            "#text": "213659"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-182",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Yixing",
+          "population": {
+            "-year": "90",
+            "#text": "200824"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-193",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Dongtai",
+          "population": {
+            "-year": "90",
+            "#text": "192247"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-206",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Changshu",
+          "population": {
+            "-year": "90",
+            "#text": "181805"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-225",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Danyang",
+          "population": {
+            "-year": "90",
+            "#text": "169603"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-234",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Xinghua",
+          "population": {
+            "-year": "90",
+            "#text": "161910"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-245",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Taizhou",
+          "population": {
+            "-year": "90",
+            "#text": "152442"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-283",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Huaian",
+          "population": {
+            "-year": "90",
+            "#text": "131149"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-316",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Liyang",
+          "population": {
+            "-year": "90",
+            "#text": "109520"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-317",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Yizheng",
+          "population": {
+            "-year": "90",
+            "#text": "109268"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-327",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Suqian",
+          "population": {
+            "-year": "90",
+            "#text": "105021"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-337",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-13",
+          "name": "Kunshan",
+          "population": {
+            "-year": "90",
+            "#text": "102052"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-14",
+      "-capital": "cty-cid-cia-China-28",
+      "-country": "TJ",
+      "name": "Jiangxi",
+      "area": "166600",
+      "population": "40150000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-28",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Nanchang",
+          "longitude": "115.5",
+          "latitude": "28.4",
+          "population": {
+            "-year": "91",
+            "#text": "1350000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-32",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Fuzhou",
+          "population": {
+            "-year": "91",
+            "#text": "1290000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-247",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Yichun",
+          "population": {
+            "-year": "90",
+            "#text": "151585"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-77",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Pingxiang",
+          "population": {
+            "-year": "90",
+            "#text": "425579"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-127",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Jiujiang",
+          "population": {
+            "-year": "90",
+            "#text": "291187"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-131",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Jingdezhen",
+          "population": {
+            "-year": "90",
+            "#text": "281183"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-164",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Ganzhou",
+          "population": {
+            "-year": "90",
+            "#text": "220129"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-223",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Xinyu",
+          "population": {
+            "-year": "90",
+            "#text": "173524"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-278",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-14",
+          "name": "Shangrao",
+          "population": {
+            "-year": "90",
+            "#text": "132455"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-15",
+      "-capital": "cty-cid-cia-China-17",
+      "-country": "TJ",
+      "name": "Jilin",
+      "area": "187000",
+      "population": "25740000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-33",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Jilin",
+          "population": {
+            "-year": "91",
+            "#text": "1270000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-17",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Changchun",
+          "longitude": "125.15",
+          "latitude": "43.5",
+          "population": {
+            "-year": "91",
+            "#text": "2110000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-69",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Hunjiang",
+          "population": {
+            "-year": "90",
+            "#text": "482043"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-102",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Liaoyuan",
+          "population": {
+            "-year": "90",
+            "#text": "354141"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-113",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Tonghua",
+          "population": {
+            "-year": "90",
+            "#text": "324600"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-114",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Siping",
+          "population": {
+            "-year": "90",
+            "#text": "317223"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-156",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Dunhua",
+          "population": {
+            "-year": "90",
+            "#text": "235100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-157",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Yanji",
+          "population": {
+            "-year": "90",
+            "#text": "230892"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-161",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Gongzhuling",
+          "population": {
+            "-year": "90",
+            "#text": "226569"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-166",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Baicheng",
+          "population": {
+            "-year": "90",
+            "#text": "217987"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-175",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Meihekou",
+          "population": {
+            "-year": "90",
+            "#text": "209038"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-191",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Fuyu",
+          "population": {
+            "-year": "90",
+            "#text": "192981"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-210",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Jiutai",
+          "population": {
+            "-year": "90",
+            "#text": "180130"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-218",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Huadian",
+          "population": {
+            "-year": "90",
+            "#text": "175873"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-261",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Longjing",
+          "population": {
+            "-year": "90",
+            "#text": "139417"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-262",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-15",
+          "name": "Daan",
+          "population": {
+            "-year": "90",
+            "#text": "138963"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-16",
+      "-capital": "cty-cid-cia-China-6",
+      "-country": "TJ",
+      "name": "Liaoning",
+      "area": "145700",
+      "population": "40670000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-6",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Shenyang",
+          "longitude": "123.3",
+          "latitude": "42",
+          "population": {
+            "-year": "91",
+            "#text": "4540000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-15",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Dalian",
+          "population": {
+            "-year": "91",
+            "#text": "2400000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Gelbes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-25",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Anshan",
+          "population": {
+            "-year": "91",
+            "#text": "1390000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-27",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Fushun",
+          "population": {
+            "-year": "91",
+            "#text": "1350000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-49",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Benxi",
+          "population": {
+            "-year": "90",
+            "#text": "768778"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-54",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Fuxin",
+          "population": {
+            "-year": "90",
+            "#text": "635473"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-58",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Jinzhou",
+          "population": {
+            "-year": "90",
+            "#text": "569518"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-62",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Dandong",
+          "population": {
+            "-year": "90",
+            "#text": "523699"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-66",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Liaoyang",
+          "population": {
+            "-year": "90",
+            "#text": "492559"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-78",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Yingkou",
+          "population": {
+            "-year": "90",
+            "#text": "421589"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Gelbes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-98",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Panshan",
+          "population": {
+            "-year": "90",
+            "#text": "362773"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-100",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Jinxi",
+          "population": {
+            "-year": "90",
+            "#text": "357052"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-145",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Tieling",
+          "population": {
+            "-year": "90",
+            "#text": "254842"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-146",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Wafangdian",
+          "population": {
+            "-year": "90",
+            "#text": "251733"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-163",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Chaoyang",
+          "population": {
+            "-year": "90",
+            "#text": "222394"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-178",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Haicheng",
+          "population": {
+            "-year": "90",
+            "#text": "205560"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-189",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Beipiao",
+          "population": {
+            "-year": "90",
+            "#text": "194301"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-190",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Fengcheng",
+          "population": {
+            "-year": "90",
+            "#text": "193784"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-335",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-16",
+          "name": "Xingcheng",
+          "population": {
+            "-year": "90",
+            "#text": "102384"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-17",
+      "-capital": "cty-cid-cia-China-59",
+      "-country": "TJ",
+      "name": "Qinghai",
+      "area": "720000",
+      "population": "4740000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-59",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-17",
+          "name": "Xining",
+          "longitude": "102",
+          "latitude": "37",
+          "population": {
+            "-year": "90",
+            "#text": "551776"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-281",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-17",
+          "name": "Yushu",
+          "population": {
+            "-year": "90",
+            "#text": "131861"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-18",
+      "-capital": "cty-cid-cia-China-12",
+      "-country": "TJ",
+      "name": "Shaanxi",
+      "area": "205000",
+      "population": "34810000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-12",
+          "-country": "TJ",
+          "-is_state_cap": "yes",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Xian",
+          "longitude": "109",
+          "latitude": "34",
+          "population": {
+            "-year": "91",
+            "#text": "2760000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-104",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Xianyang",
+          "population": {
+            "-year": "90",
+            "#text": "352125"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-111",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Baoji",
+          "population": {
+            "-year": "90",
+            "#text": "337765"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-132",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Tongchuan",
+          "population": {
+            "-year": "90",
+            "#text": "280657"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-224",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Hanzhong",
+          "population": {
+            "-year": "90",
+            "#text": "169930"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-259",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Ankang",
+          "population": {
+            "-year": "90",
+            "#text": "142170"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-260",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Weinan",
+          "population": {
+            "-year": "90",
+            "#text": "140169"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-310",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-18",
+          "name": "Yanan",
+          "population": {
+            "-year": "90",
+            "#text": "113277"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-19",
+      "-capital": "cty-cid-cia-China-16",
+      "-country": "TJ",
+      "name": "Shandong",
+      "area": "153000",
+      "population": "86710000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-16",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Jinan",
+          "longitude": "117",
+          "latitude": "36.5",
+          "population": {
+            "-year": "91",
+            "#text": "2320000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-14",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Zibo",
+          "population": {
+            "-year": "91",
+            "#text": "2460000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-18",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Qingdao",
+          "population": {
+            "-year": "91",
+            "#text": "2060000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Gelbes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-72",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Yantai",
+          "population": {
+            "-year": "90",
+            "#text": "452127"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Gelbes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-75",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Weifang",
+          "population": {
+            "-year": "90",
+            "#text": "428522"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-89",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Zaozhuang",
+          "population": {
+            "-year": "90",
+            "#text": "380846"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-106",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Taian",
+          "population": {
+            "-year": "90",
+            "#text": "350696"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-112",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Linyi",
+          "population": {
+            "-year": "90",
+            "#text": "324720"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-116",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Tengxian",
+          "population": {
+            "-year": "90",
+            "#text": "315083"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-128",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Dongying",
+          "population": {
+            "-year": "90",
+            "#text": "281728"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-130",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Xintai",
+          "population": {
+            "-year": "90",
+            "#text": "281248"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-137",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Jining",
+          "population": {
+            "-year": "90",
+            "#text": "265248"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-149",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Laiwu",
+          "population": {
+            "-year": "90",
+            "#text": "246833"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-177",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Liaocheng",
+          "population": {
+            "-year": "90",
+            "#text": "207844"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-186",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Dezhou",
+          "population": {
+            "-year": "90",
+            "#text": "195485"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-197",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Heze",
+          "population": {
+            "-year": "90",
+            "#text": "189293"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-203",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Rizhao",
+          "population": {
+            "-year": "90",
+            "#text": "185048"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-243",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Liancheng",
+          "population": {
+            "-year": "90",
+            "#text": "156307"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-244",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Jiaoxian",
+          "population": {
+            "-year": "90",
+            "#text": "153364"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-249",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Pingdu",
+          "population": {
+            "-year": "90",
+            "#text": "150123"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-251",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Longkou",
+          "population": {
+            "-year": "90",
+            "#text": "148362"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-264",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Laiyang",
+          "population": {
+            "-year": "90",
+            "#text": "137080"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-271",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Wendeng",
+          "population": {
+            "-year": "90",
+            "#text": "133910"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-286",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Weihai",
+          "population": {
+            "-year": "90",
+            "#text": "128888"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Gelbes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-294",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Linqing",
+          "population": {
+            "-year": "90",
+            "#text": "123958"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-299",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Jiaonan",
+          "population": {
+            "-year": "90",
+            "#text": "121397"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-336",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-19",
+          "name": "Zhucheng",
+          "population": {
+            "-year": "90",
+            "#text": "102134"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-20",
+      "-capital": "cty-cid-cia-China-19",
+      "-country": "TJ",
+      "name": "Shanxi",
+      "area": "156000",
+      "population": "30450000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-19",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Taiyuan",
+          "longitude": "112.3",
+          "latitude": "38",
+          "population": {
+            "-year": "91",
+            "#text": "1960000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-39",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Datong",
+          "population": {
+            "-year": "91",
+            "#text": "1110000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-99",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Yangquan",
+          "population": {
+            "-year": "90",
+            "#text": "362268"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-115",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Changzhi",
+          "population": {
+            "-year": "90",
+            "#text": "317144"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-194",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Yuci",
+          "population": {
+            "-year": "90",
+            "#text": "191356"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-199",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Linfen",
+          "population": {
+            "-year": "90",
+            "#text": "187309"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-268",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-20",
+          "name": "Jincheng",
+          "population": {
+            "-year": "90",
+            "#text": "136396"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-21",
+      "-capital": "cty-cid-cia-China-11",
+      "-country": "TJ",
+      "name": "Sichuan",
+      "area": "570000",
+      "population": "112140000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-11",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Chengdu",
+          "longitude": "107.15",
+          "latitude": "33.15",
+          "population": {
+            "-year": "91",
+            "#text": "2810000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-9",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Chongqing",
+          "population": {
+            "-year": "91",
+            "#text": "2980000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-80",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Dukou",
+          "population": {
+            "-year": "90",
+            "#text": "415466"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-87",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Zigong",
+          "population": {
+            "-year": "90",
+            "#text": "393184"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-110",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Leshan",
+          "population": {
+            "-year": "90",
+            "#text": "341128"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-139",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Mianyang",
+          "population": {
+            "-year": "90",
+            "#text": "262947"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-140",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Luzhou",
+          "population": {
+            "-year": "90",
+            "#text": "262892"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-143",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Neijiang",
+          "population": {
+            "-year": "90",
+            "#text": "256012"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-154",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Yibin",
+          "population": {
+            "-year": "90",
+            "#text": "241019"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Jangtse"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-179",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Qianjiang",
+          "population": {
+            "-year": "90",
+            "#text": "205504"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-198",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Daxian",
+          "population": {
+            "-year": "90",
+            "#text": "188101"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-204",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Deyang",
+          "population": {
+            "-year": "90",
+            "#text": "182488"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-205",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Guangyuan",
+          "population": {
+            "-year": "90",
+            "#text": "182241"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-209",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Nanchong",
+          "population": {
+            "-year": "90",
+            "#text": "180273"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-219",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Jiangyou",
+          "population": {
+            "-year": "90",
+            "#text": "175753"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-240",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Wanxian",
+          "population": {
+            "-year": "90",
+            "#text": "156823"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-270",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-21",
+          "name": "Xichang",
+          "population": {
+            "-year": "90",
+            "#text": "134419"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-22",
+      "-capital": "cty-cid-cia-China-22",
+      "-country": "TJ",
+      "name": "Yunnan",
+      "area": "394000",
+      "population": "39390000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-22",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-22",
+          "name": "Kunming",
+          "longitude": "102.4",
+          "latitude": "25.5",
+          "population": {
+            "-year": "91",
+            "#text": "1520000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-170",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-22",
+          "name": "Gejiu",
+          "population": {
+            "-year": "90",
+            "#text": "214294"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-215",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-22",
+          "name": "Qujing",
+          "population": {
+            "-year": "90",
+            "#text": "178669"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-23",
+      "-capital": "cty-cid-cia-China-29",
+      "-country": "TJ",
+      "name": "Zhejiang",
+      "area": "101800",
+      "population": "42940000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-29",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Hangzhou",
+          "longitude": "125.5",
+          "latitude": "30.2",
+          "population": {
+            "-year": "91",
+            "#text": "1340000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-40",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Ningbo",
+          "population": {
+            "-year": "91",
+            "#text": "1090000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-85",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Wenzhou",
+          "population": {
+            "-year": "90",
+            "#text": "401871"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-165",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Huzhou",
+          "population": {
+            "-year": "90",
+            "#text": "218071"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-173",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Jiaxing",
+          "population": {
+            "-year": "90",
+            "#text": "211526"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-212",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Shaoxing",
+          "population": {
+            "-year": "90",
+            "#text": "179818"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-232",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Xiaoshan",
+          "population": {
+            "-year": "90",
+            "#text": "162930"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-241",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Ruian",
+          "population": {
+            "-year": "90",
+            "#text": "156468"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-242",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Zhoushan",
+          "population": {
+            "-year": "90",
+            "#text": "156317"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-256",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Jinhua",
+          "population": {
+            "-year": "90",
+            "#text": "144280"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-307",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Yuyao",
+          "population": {
+            "-year": "90",
+            "#text": "114056"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-322",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Cixi",
+          "population": {
+            "-year": "90",
+            "#text": "107329"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-338",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-23",
+          "name": "Haining",
+          "population": {
+            "-year": "90",
+            "#text": "100478"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-25",
+      "-capital": "cty-cid-cia-China-41",
+      "-country": "TJ",
+      "name": "Guangxi Zhuangzu",
+      "area": "236300",
+      "population": "44930000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-41",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Nanning",
+          "longitude": "108.5",
+          "latitude": "22.5",
+          "population": {
+            "-year": "91",
+            "#text": "1070000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-55",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Liuzhou",
+          "population": {
+            "-year": "90",
+            "#text": "609320"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-96",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Guilin",
+          "population": {
+            "-year": "90",
+            "#text": "364130"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-174",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Wuzhou",
+          "population": {
+            "-year": "90",
+            "#text": "210452"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-305",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Qinzhou",
+          "population": {
+            "-year": "90",
+            "#text": "114586"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-308",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Guixian",
+          "population": {
+            "-year": "90",
+            "#text": "114025"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-312",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-25",
+          "name": "Beihai",
+          "population": {
+            "-year": "90",
+            "#text": "112673"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-26",
+      "-capital": "cty-cid-cia-China-53",
+      "-country": "TJ",
+      "name": "Nei Monggol",
+      "area": "1183000",
+      "population": "22600000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-53",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Hohhot",
+          "longitude": "113",
+          "latitude": "41",
+          "population": {
+            "-year": "90",
+            "#text": "652534"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-35",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Baotou",
+          "population": {
+            "-year": "91",
+            "#text": "1200000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-90",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Yakeshi",
+          "population": {
+            "-year": "90",
+            "#text": "377869"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-107",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Chifeng",
+          "population": {
+            "-year": "90",
+            "#text": "350077"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-231",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Jining",
+          "population": {
+            "-year": "90",
+            "#text": "163552"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-138",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Wuhai",
+          "population": {
+            "-year": "90",
+            "#text": "264081"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-144",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Tongliao",
+          "population": {
+            "-year": "90",
+            "#text": "255129"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-208",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Hailar",
+          "population": {
+            "-year": "90",
+            "#text": "180650"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-238",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Ulanhot",
+          "population": {
+            "-year": "90",
+            "#text": "159538"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-274",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Linhe",
+          "population": {
+            "-year": "90",
+            "#text": "133183"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-302",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-26",
+          "name": "Manzhouli",
+          "population": {
+            "-year": "90",
+            "#text": "120023"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-27",
+      "-capital": "cty-cid-cia-China-101",
+      "-country": "TJ",
+      "name": "Ningxia Huizu",
+      "area": "66400",
+      "population": "5040000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-101",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-27",
+          "name": "Yinchuan",
+          "longitude": "106.2",
+          "latitude": "38.3",
+          "population": {
+            "-year": "90",
+            "#text": "356652"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hwangho"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-142",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-27",
+          "name": "Shizuishan",
+          "population": {
+            "-year": "90",
+            "#text": "257862"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-28",
+      "-capital": "cty-cid-cia-China-37",
+      "-country": "TJ",
+      "name": "Xinjiang Uygur",
+      "area": "1600000",
+      "population": "16320000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-China-37",
+          "-is_state_cap": "yes",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Urumqi",
+          "longitude": "88",
+          "latitude": "44",
+          "population": {
+            "-year": "91",
+            "#text": "1160000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-125",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Shihezi",
+          "population": {
+            "-year": "90",
+            "#text": "299676"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-184",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Karamay",
+          "population": {
+            "-year": "90",
+            "#text": "197602"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-216",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Yining",
+          "population": {
+            "-year": "90",
+            "#text": "177193"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-220",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Kashi",
+          "population": {
+            "-year": "90",
+            "#text": "174570"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-230",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Aksu",
+          "population": {
+            "-year": "90",
+            "#text": "164092"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-235",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Hami",
+          "population": {
+            "-year": "90",
+            "#text": "161315"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-China-239",
+          "-country": "TJ",
+          "-province": "prov-cid-cia-China-28",
+          "name": "Korla",
+          "population": {
+            "-year": "90",
+            "#text": "159344"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-China-29",
+      "-capital": "cty-cid-cia-China-324",
+      "-country": "TJ",
+      "name": "Tibet",
+      "area": "1220000",
+      "population": "2360000",
+      "city": {
+        "-id": "cty-cid-cia-China-324",
+        "-is_state_cap": "yes",
+        "-country": "TJ",
+        "-province": "prov-cid-cia-China-29",
+        "name": "Lhasa",
+        "longitude": "91",
+        "latitude": "29.5",
+        "population": {
+          "-year": "90",
+          "#text": "106885"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-China-31",
+      "-capital": "cty-cid-cia-China-3",
+      "-country": "TJ",
+      "name": "Beijing (munic.)",
+      "area": "16800",
+      "population": "11250000",
+      "city": {
+        "-id": "cty-cid-cia-China-3",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "TJ",
+        "-province": "prov-cid-cia-China-31",
+        "name": "Beijing",
+        "longitude": "116.2",
+        "latitude": "39.5",
+        "population": {
+          "-year": "91",
+          "#text": "7000000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-China-32",
+      "-capital": "cty-China-Shanghai",
+      "-country": "TJ",
+      "name": "Shanghai (munic.)",
+      "area": "6200",
+      "population": "13560000",
+      "city": {
+        "-id": "cty-China-Shanghai",
+        "-is_state_cap": "yes",
+        "-country": "TJ",
+        "-province": "prov-cid-cia-China-32",
+        "name": "Shanghai",
+        "longitude": "121.433",
+        "latitude": "31.2",
+        "population": {
+          "-year": "91",
+          "#text": "7830000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-China-33",
+      "-capital": "cty-cid-cia-China-5",
+      "-country": "TJ",
+      "name": "Tianjin (munic.)",
+      "area": "11300",
+      "population": "9350000",
+      "city": {
+        "-id": "cty-cid-cia-China-5",
+        "-is_state_cap": "yes",
+        "-country": "TJ",
+        "-province": "prov-cid-cia-China-33",
+        "name": "Tianjin",
+        "longitude": "117.2",
+        "latitude": "39.1",
+        "population": {
+          "-year": "91",
+          "#text": "5770000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "IR",
+  "-area": "1648000",
+  "-capital": "cty-Iran-Tehran",
+  "-memberships": "org-CP org-CCC org-ESCAP org-ECO org-FAO org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OPEC org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Iran",
+  "population": "66094264",
+  "population_growth": "2.21",
+  "infant_mortality": "52.7",
+  "gdp_total": "323500",
+  "gdp_agri": "21",
+  "gdp_ind": "37",
+  "gdp_serv": "42",
+  "inflation": "60",
+  "indep_date": "1979-04-01",
+  "government": "theocratic republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "51",
+      "#text": "Persian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Baloch"
+    },
+    {
+      "-percentage": "24",
+      "#text": "Azerbaijani"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Kurd"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Lur"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Turkmen"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Gilaki Mazandarani"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "99",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Bahai"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "1",
+      "#text": "Turkish"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Kurdish"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Balochi"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Arabic"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Luri"
+    },
+    {
+      "-percentage": "58",
+      "#text": "Persian"
+    },
+    {
+      "-percentage": "26",
+      "#text": "Turkic"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TR",
+      "-length": "499"
+    },
+    {
+      "-country": "AFG",
+      "-length": "936"
+    },
+    {
+      "-country": "PK",
+      "-length": "909"
+    },
+    {
+      "-country": "TM",
+      "-length": "992"
+    },
+    {
+      "-country": "ARM",
+      "-length": "35"
+    },
+    {
+      "-country": "AZ",
+      "-length": "611"
+    },
+    {
+      "-country": "IRQ",
+      "-length": "1458"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Iran-2",
+      "-capital": "cty-cid-cia-Iran-14",
+      "-country": "IR",
+      "name": "Azarbayian e Gharbt",
+      "area": "38850",
+      "population": "2284208",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-14",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-2",
+          "name": "Orumiyeh",
+          "longitude": "45",
+          "latitude": "37.3",
+          "population": {
+            "-year": "94",
+            "#text": "396392"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-39",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-2",
+          "name": "Khvoy",
+          "population": {
+            "-year": "91",
+            "#text": "137885"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-46",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-2",
+          "name": "Maragheh",
+          "population": {
+            "-year": "91",
+            "#text": "117388"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-3",
+      "-capital": "cty-cid-cia-Iran-5",
+      "-country": "IR",
+      "name": "Azarbayian e Sharqi",
+      "area": "67102",
+      "population": "4420343",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-5",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-3",
+          "name": "Tabriz",
+          "longitude": "46.2",
+          "latitude": "38.1",
+          "population": {
+            "-year": "94",
+            "#text": "1166203"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-19",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-3",
+          "name": "Ardabil",
+          "population": {
+            "-year": "94",
+            "#text": "329869"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-4",
+      "-capital": "cty-cid-cia-Iran-9",
+      "-country": "IR",
+      "name": "Bakhtaran",
+      "area": "23667",
+      "population": "1622159",
+      "city": {
+        "-id": "cty-cid-cia-Iran-9",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-4",
+        "name": "Bakhtaran",
+        "longitude": "47.1",
+        "latitude": "34.2",
+        "population": {
+          "-year": "94",
+          "#text": "665636"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-5",
+      "-capital": "cty-cid-cia-Iran-Yasuj",
+      "-country": "IR",
+      "name": "Boyer Ahmad e Kohkiluyeh",
+      "area": "14261",
+      "population": "496739",
+      "city": {
+        "-id": "cty-cid-cia-Iran-Yasuj",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-5",
+        "name": "Yasuj"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-6",
+      "-capital": "cty-cid-cia-Iran-Bushehr",
+      "-country": "IR",
+      "name": "Bushehr",
+      "area": "27653",
+      "population": "694252",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-42",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-6",
+          "name": "Bandar e Bushehr",
+          "population": {
+            "-year": "91",
+            "#text": "132824"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-Bushehr",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-6",
+          "name": "Bushehr"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-7",
+      "-capital": "cty-cid-cia-Iran-Shahr-e-Kord",
+      "-country": "IR",
+      "name": "Chahar Mahal e Bakhtiari",
+      "area": "14870",
+      "population": "747297",
+      "city": {
+        "-id": "cty-cid-cia-Iran-Shahr-e-Kord",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-7",
+        "name": "Shahr e Kord"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-8",
+      "-capital": "cty-cid-cia-Iran-4",
+      "-country": "IR",
+      "name": "Esfahan",
+      "area": "104650",
+      "population": "3682444",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-4",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-8",
+          "name": "Esfahan",
+          "longitude": "51.4",
+          "latitude": "32.4",
+          "population": {
+            "-year": "94",
+            "#text": "1220595"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-35",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-8",
+          "name": "Najafabad",
+          "population": {
+            "-year": "91",
+            "#text": "160004"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-45",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-8",
+          "name": "Khomeynishahr",
+          "population": {
+            "-year": "91",
+            "#text": "118348"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-9",
+      "-capital": "cty-cid-cia-Iran-6",
+      "-country": "IR",
+      "name": "Fars",
+      "area": "133298",
+      "population": "3543828",
+      "city": {
+        "-id": "cty-cid-cia-Iran-6",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-9",
+        "name": "Shiraz",
+        "longitude": "52.3",
+        "latitude": "29.4",
+        "population": {
+          "-year": "94",
+          "#text": "1042801"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-10",
+      "-capital": "cty-cid-cia-Iran-17",
+      "-country": "IR",
+      "name": "Gilan",
+      "area": "14709",
+      "population": "2204047",
+      "city": {
+        "-id": "cty-cid-cia-Iran-17",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-10",
+        "name": "Rasht",
+        "longitude": "49.4",
+        "latitude": "37.2",
+        "population": {
+          "-year": "94",
+          "#text": "374475"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-11",
+      "-capital": "cty-cid-cia-Iran-13",
+      "-country": "IR",
+      "name": "Hamadan",
+      "area": "19784",
+      "population": "1651320",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-13",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-11",
+          "name": "Hamadan",
+          "longitude": "48.3",
+          "latitude": "34.5",
+          "population": {
+            "-year": "94",
+            "#text": "406070"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-43",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-11",
+          "name": "Malayer",
+          "population": {
+            "-year": "91",
+            "#text": "130458"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-12",
+      "-capital": "cty-cid-cia-Iran-15",
+      "-country": "IR",
+      "name": "Hormozgan",
+      "area": "66780",
+      "population": "924433",
+      "city": {
+        "-id": "cty-cid-cia-Iran-15",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-12",
+        "name": "Bandar Abbas",
+        "population": {
+          "-year": "94",
+          "#text": "383515"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-13",
+      "-capital": "cty-cid-cia-Iran-47",
+      "-country": "IR",
+      "name": "Ilam",
+      "area": "19044",
+      "population": "440693",
+      "city": {
+        "-id": "cty-cid-cia-Iran-47",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-13",
+        "name": "Ilam",
+        "population": {
+          "-year": "91",
+          "#text": "116428"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-14",
+      "-capital": "cty-cid-cia-Iran-18",
+      "-country": "IR",
+      "name": "Kerman",
+      "area": "179916",
+      "population": "1862542",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-18",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-14",
+          "name": "Kerman",
+          "population": {
+            "-year": "94",
+            "#text": "349626"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-49",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-14",
+          "name": "Sirjan",
+          "population": {
+            "-year": "91",
+            "#text": "107887"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-15",
+      "-capital": "cty-cid-cia-Iran-3",
+      "-country": "IR",
+      "name": "Khorasan",
+      "area": "313337",
+      "population": "6013200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-3",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-15",
+          "name": "Mashhad",
+          "longitude": "59.4",
+          "latitude": "36.2",
+          "population": {
+            "-year": "94",
+            "#text": "1964489"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-37",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-15",
+          "name": "Sabzevar",
+          "population": {
+            "-year": "91",
+            "#text": "148065"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-41",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-15",
+          "name": "Neyshabur",
+          "population": {
+            "-year": "91",
+            "#text": "135681"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-48",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-15",
+          "name": "Bojnurd",
+          "population": {
+            "-year": "91",
+            "#text": "112426"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-52",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-15",
+          "name": "Birjand",
+          "population": {
+            "-year": "91",
+            "#text": "101177"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-16",
+      "-capital": "cty-cid-cia-Iran-7",
+      "-country": "IR",
+      "name": "Khuzestan",
+      "area": "67282",
+      "population": "3175852",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-7",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-16",
+          "name": "Ahvaz",
+          "longitude": "48.4",
+          "latitude": "31.2",
+          "population": {
+            "-year": "94",
+            "#text": "828380"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-30",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-16",
+          "name": "Khorramshahr",
+          "population": {
+            "-year": "91",
+            "#text": "197241"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Karun"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-SchattAlArab"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Iran-31",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-16",
+          "name": "Dezful",
+          "population": {
+            "-year": "91",
+            "#text": "181309"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-50",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-16",
+          "name": "Masjed e Soleyman",
+          "population": {
+            "-year": "91",
+            "#text": "107539"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-17",
+      "-capital": "cty-cid-cia-Iran-24",
+      "-country": "IR",
+      "name": "Kordestan",
+      "area": "24998",
+      "population": "1233480",
+      "city": {
+        "-id": "cty-cid-cia-Iran-24",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-17",
+        "name": "Sanandaj",
+        "population": {
+          "-year": "94",
+          "#text": "277314"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-18",
+      "-capital": "cty-cid-cia-Iran-23",
+      "-country": "IR",
+      "name": "Lorestan",
+      "area": "28803",
+      "population": "1501778",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-23",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-18",
+          "name": "Khorramabad",
+          "population": {
+            "-year": "94",
+            "#text": "277370"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-29",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-18",
+          "name": "Borujerd",
+          "population": {
+            "-year": "91",
+            "#text": "201016"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-19",
+      "-capital": "cty-cid-cia-Iran-16",
+      "-country": "IR",
+      "name": "Markazi",
+      "area": "29080",
+      "population": "1182611",
+      "city": {
+        "-id": "cty-cid-cia-Iran-16",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-19",
+        "name": "Arak",
+        "population": {
+          "-year": "94",
+          "#text": "378597"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-20",
+      "-capital": "cty-cid-cia-Iran-32",
+      "-country": "IR",
+      "name": "Mazandaran",
+      "area": "46456",
+      "population": "3793149",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Iran-32",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Sari",
+          "population": {
+            "-year": "91",
+            "#text": "167602"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-33",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Gorgan",
+          "population": {
+            "-year": "91",
+            "#text": "162468"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-38",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Amol",
+          "population": {
+            "-year": "91",
+            "#text": "139923"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-40",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Babol",
+          "population": {
+            "-year": "91",
+            "#text": "137348"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-44",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Qaemshahr",
+          "population": {
+            "-year": "91",
+            "#text": "123684"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-51",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-20",
+          "name": "Gonbad e Kavus",
+          "population": {
+            "-year": "91",
+            "#text": "102768"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-21",
+      "-capital": "cty-cid-cia-Iran-Semnan",
+      "-country": "IR",
+      "name": "Semnan",
+      "area": "90905",
+      "population": "458125",
+      "city": {
+        "-id": "cty-cid-cia-Iran-Semnan",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-21",
+        "name": "Semnan"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-22",
+      "-capital": "cty-cid-cia-Iran-11",
+      "-country": "IR",
+      "name": "Sistan e Baluchestan",
+      "area": "181578",
+      "population": "1455102",
+      "city": {
+        "-id": "cty-cid-cia-Iran-11",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-22",
+        "name": "Zahedan",
+        "population": {
+          "-year": "94",
+          "#text": "419886"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-23",
+      "-capital": "cty-Iran-Tehran",
+      "-country": "IR",
+      "name": "Tehran",
+      "area": "29993",
+      "population": "9982309",
+      "city": [
+        {
+          "-id": "cty-Iran-Tehran",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Tehran",
+          "longitude": "51.4167",
+          "latitude": "35.6833",
+          "population": {
+            "-year": "94",
+            "#text": "6750043"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-8",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Qom",
+          "longitude": "50.5",
+          "latitude": "34.4",
+          "population": {
+            "-year": "94",
+            "#text": "780453"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-10",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Karaj",
+          "population": {
+            "-year": "94",
+            "#text": "588287"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-12",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Mehrshahr",
+          "population": {
+            "-year": "94",
+            "#text": "413299"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-34",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Rajaishahr",
+          "population": {
+            "-year": "91",
+            "#text": "160362"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-21",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Qazvin",
+          "population": {
+            "-year": "94",
+            "#text": "298705"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-36",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Kashan",
+          "population": {
+            "-year": "91",
+            "#text": "155188"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Iran-25",
+          "-country": "IR",
+          "-province": "prov-cid-cia-Iran-23",
+          "name": "Islamshahr",
+          "population": {
+            "-year": "94",
+            "#text": "239715"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Iran-24",
+      "-capital": "cty-cid-cia-Iran-20",
+      "-country": "IR",
+      "name": "Yazd",
+      "area": "70011",
+      "population": "691119",
+      "city": {
+        "-id": "cty-cid-cia-Iran-20",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-24",
+        "name": "Yazd",
+        "population": {
+          "-year": "94",
+          "#text": "306268"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iran-25",
+      "-capital": "cty-cid-cia-Iran-22",
+      "-country": "IR",
+      "name": "Zanjan",
+      "area": "36398",
+      "population": "1776133",
+      "city": {
+        "-id": "cty-cid-cia-Iran-22",
+        "-is_state_cap": "yes",
+        "-country": "IR",
+        "-province": "prov-cid-cia-Iran-25",
+        "name": "Zanjan",
+        "population": {
+          "-year": "94",
+          "#text": "280691"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "PK",
+  "-area": "803940",
+  "-capital": "cty-cid-cia-Pakistan-15",
+  "-memberships": "org-AsDB org-CP org-C org-CCC org-ESCAP org-ECO org-FAO org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAS org-OIC org-PCA org-SAARC org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-UNIKOM org-MINURSO org-UNMIH org-UNOMIG org-UNOMIL org-UNHCR org-UNPREDEP org-UNPROFOR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Pakistan",
+  "population": "129275660",
+  "population_growth": "2.24",
+  "infant_mortality": "96.8",
+  "gdp_total": "274200",
+  "gdp_agri": "24",
+  "gdp_ind": "27",
+  "gdp_serv": "49",
+  "inflation": "13",
+  "indep_date": "1947-08-14",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "97",
+    "#text": "Muslim"
+  },
+  "languages": [
+    {
+      "-percentage": "8",
+      "#text": "Pashtu"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Urdu"
+    },
+    {
+      "-percentage": "48",
+      "#text": "Punjabi"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Sindhi"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Balochi"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Hindko"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Brahui"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Siraiki"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AFG",
+      "-length": "2430"
+    },
+    {
+      "-country": "TJ",
+      "-length": "523"
+    },
+    {
+      "-country": "IR",
+      "-length": "909"
+    },
+    {
+      "-country": "IND",
+      "-length": "2912"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Pakistan-15",
+      "-is_country_cap": "yes",
+      "-country": "PK",
+      "name": "Islamabad",
+      "longitude": "73",
+      "latitude": "34",
+      "population": {
+        "-year": "81",
+        "#text": "204364"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-9",
+      "-country": "PK",
+      "name": "Hyderabad",
+      "population": {
+        "-year": "95",
+        "#text": "1107000"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Indus"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-13",
+      "-country": "PK",
+      "name": "Quetta",
+      "population": {
+        "-year": "81",
+        "#text": "285719"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-5",
+      "-country": "PK",
+      "name": "Peshawar",
+      "population": {
+        "-year": "95",
+        "#text": "1676000"
+      }
+    },
+    {
+      "-id": "cty-Pakistan-Lahore",
+      "-country": "PK",
+      "name": "Lahore",
+      "longitude": "74.33",
+      "latitude": "31.58",
+      "population": {
+        "-year": "95",
+        "#text": "5085000"
+      }
+    },
+    {
+      "-id": "cty-Pakistan-Karachi",
+      "-country": "PK",
+      "name": "Karachi",
+      "longitude": "66.98",
+      "latitude": "24.8",
+      "population": {
+        "-year": "95",
+        "#text": "9863000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Arabisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-4",
+      "-country": "PK",
+      "name": "Lyallpur",
+      "population": {
+        "-year": "95",
+        "#text": "1875000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-6",
+      "-country": "PK",
+      "name": "Gujranwala",
+      "population": {
+        "-year": "95",
+        "#text": "1663000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-7",
+      "-country": "PK",
+      "name": "Rawalpindi",
+      "population": {
+        "-year": "95",
+        "#text": "1290000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-8",
+      "-country": "PK",
+      "name": "Multan",
+      "population": {
+        "-year": "95",
+        "#text": "1257000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-11",
+      "-country": "PK",
+      "name": "Sialkot",
+      "population": {
+        "-year": "81",
+        "#text": "302009"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-12",
+      "-country": "PK",
+      "name": "Sargodha",
+      "population": {
+        "-year": "81",
+        "#text": "291362"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-14",
+      "-country": "PK",
+      "name": "Lahore Cantonment",
+      "population": {
+        "-year": "81",
+        "#text": "237000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-16",
+      "-country": "PK",
+      "name": "Jhang",
+      "population": {
+        "-year": "81",
+        "#text": "195558"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-17",
+      "-country": "PK",
+      "name": "Sukkur",
+      "population": {
+        "-year": "81",
+        "#text": "190551"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Indus"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-18",
+      "-country": "PK",
+      "name": "Bahawalpur",
+      "population": {
+        "-year": "81",
+        "#text": "180263"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-19",
+      "-country": "PK",
+      "name": "Kasur",
+      "population": {
+        "-year": "81",
+        "#text": "155523"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-20",
+      "-country": "PK",
+      "name": "Gujrat",
+      "population": {
+        "-year": "81",
+        "#text": "155058"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-21",
+      "-country": "PK",
+      "name": "Okara",
+      "population": {
+        "-year": "81",
+        "#text": "153483"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-22",
+      "-country": "PK",
+      "name": "Sahiwal",
+      "population": {
+        "-year": "81",
+        "#text": "150954"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-23",
+      "-country": "PK",
+      "name": "Mardan",
+      "population": {
+        "-year": "81",
+        "#text": "147977"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-24",
+      "-country": "PK",
+      "name": "Sheikhupura",
+      "population": {
+        "-year": "81",
+        "#text": "141168"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-25",
+      "-country": "PK",
+      "name": "Mirpur Khas",
+      "population": {
+        "-year": "81",
+        "#text": "124371"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-26",
+      "-country": "PK",
+      "name": "Larkana",
+      "population": {
+        "-year": "81",
+        "#text": "123890"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-27",
+      "-country": "PK",
+      "name": "Wah Cantonment",
+      "population": {
+        "-year": "81",
+        "#text": "122335"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-28",
+      "-country": "PK",
+      "name": "Rahim Yar Khan",
+      "population": {
+        "-year": "81",
+        "#text": "119036"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-29",
+      "-country": "PK",
+      "name": "Jhelum",
+      "population": {
+        "-year": "81",
+        "#text": "106462"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-30",
+      "-country": "PK",
+      "name": "Chiniot",
+      "population": {
+        "-year": "81",
+        "#text": "105559"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-31",
+      "-country": "PK",
+      "name": "Nawabshah",
+      "population": {
+        "-year": "81",
+        "#text": "102139"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Pakistan-32",
+      "-country": "PK",
+      "name": "Dera Ghazi Khan",
+      "population": {
+        "-year": "81",
+        "#text": "102007"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "TAD",
+  "-area": "143100",
+  "-capital": "cty-cid-cia-Tajikistan-2",
+  "-memberships": "org-CIS org-ESCAP org-ECE org-ECO org-EBRD org-FAO org-IBRD org-ICAO org-IDA org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ITU org-Intelsat org-IDB org-ANC org-OSCE org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Tajikistan",
+  "population": "5916373",
+  "population_growth": "1.54",
+  "infant_mortality": "113.1",
+  "gdp_total": "6400",
+  "inflation": "28",
+  "indep_date": "1991-09-09",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3.5",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "64.9",
+      "#text": "Tajik"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Uzbek"
+    }
+  ],
+  "religions": {
+    "-percentage": "85",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "AFG",
+      "-length": "1206"
+    },
+    {
+      "-country": "TJ",
+      "-length": "414"
+    },
+    {
+      "-country": "UZB",
+      "-length": "1161"
+    },
+    {
+      "-country": "KGZ",
+      "-length": "870"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Tajikistan-2",
+      "-capital": "cty-cid-cia-Tajikistan-3",
+      "-country": "TAD",
+      "name": "Khujand",
+      "area": "26100",
+      "population": "1635900",
+      "city": {
+        "-id": "cty-cid-cia-Tajikistan-3",
+        "-is_state_cap": "yes",
+        "-country": "TAD",
+        "-province": "prov-cid-cia-Tajikistan-2",
+        "name": "Khujand",
+        "population": {
+          "-year": "89",
+          "#text": "164500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tajikistan-3",
+      "-capital": "cty-cid-cia-Tajikistan-4",
+      "-country": "TAD",
+      "name": "Kulob",
+      "area": "12000",
+      "population": "668100",
+      "city": {
+        "-id": "cty-cid-cia-Tajikistan-4",
+        "-is_state_cap": "yes",
+        "-country": "TAD",
+        "-province": "prov-cid-cia-Tajikistan-3",
+        "name": "Kulob",
+        "population": {
+          "-year": "89",
+          "#text": "79300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tajikistan-4",
+      "-capital": "cty-cid-cia-Tajikistan-5",
+      "-country": "TAD",
+      "name": "Khatlon",
+      "area": "12600",
+      "population": "1113500",
+      "city": {
+        "-id": "cty-cid-cia-Tajikistan-5",
+        "-is_state_cap": "yes",
+        "-country": "TAD",
+        "-province": "prov-cid-cia-Tajikistan-4",
+        "name": "Qurghonteppa",
+        "population": {
+          "-year": "89",
+          "#text": "58400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tajikistan-6",
+      "-capital": "cty-cid-cia-Tajikistan-Khorugh",
+      "-country": "TAD",
+      "name": "Badakhshoni Kuni",
+      "area": "63700",
+      "population": "167100",
+      "city": {
+        "-id": "cty-cid-cia-Tajikistan-Khorugh",
+        "-is_state_cap": "yes",
+        "-country": "TAD",
+        "-province": "prov-cid-cia-Tajikistan-6",
+        "name": "Khorugh",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Pjandsh"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tajikistan-9",
+      "-capital": "cty-cid-cia-Tajikistan-2",
+      "-country": "TAD",
+      "name": "Dushanbe (munic.)",
+      "area": "300",
+      "population": "591900",
+      "city": {
+        "-id": "cty-cid-cia-Tajikistan-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "TAD",
+        "-province": "prov-cid-cia-Tajikistan-9",
+        "name": "Dushanbe",
+        "longitude": "68.48",
+        "latitude": "38.3",
+        "population": {
+          "-year": "89",
+          "#text": "582400"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "TM",
+  "-area": "488100",
+  "-capital": "cty-cid-cia-Turkmenistan-2",
+  "-memberships": "org-CIS org-CCC org-ESCAP org-ECE org-ECO org-EBRD org-FAO org-IBRD org-ICAO org-IFRCS org-ILO org-IMO org-IMF org-IOC org-ISO org-ITU org-Intelsat org-IDB org-ANC org-OSCE org-OIC org-PFP org-UN org-UNESCO org-UPU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Turkmenistan",
+  "population": "4149283",
+  "population_growth": "1.82",
+  "infant_mortality": "81.6",
+  "gdp_total": "11500",
+  "gdp_agri": "32.5",
+  "gdp_ind": "33.4",
+  "gdp_serv": "34.1",
+  "inflation": "25",
+  "indep_date": "1991-10-27",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "9.8",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "73.3",
+      "#text": "Turkmen"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Kazak"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "87",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "12",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "72",
+      "#text": "Turkmen"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AFG",
+      "-length": "744"
+    },
+    {
+      "-country": "IR",
+      "-length": "992"
+    },
+    {
+      "-country": "UZB",
+      "-length": "1621"
+    },
+    {
+      "-country": "KAZ",
+      "-length": "379"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Turkmenistan-2",
+      "-capital": "cty-cid-cia-Turkmenistan-2",
+      "-country": "TM",
+      "name": "Ahal",
+      "area": "0",
+      "population": "416400",
+      "city": {
+        "-id": "cty-cid-cia-Turkmenistan-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "TM",
+        "-province": "prov-cid-cia-Turkmenistan-2",
+        "name": "Ashgabat",
+        "longitude": "58.2",
+        "latitude": "37.58",
+        "population": {
+          "-year": "91",
+          "#text": "416400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkmenistan-3",
+      "-capital": "cty-cid-cia-Turkmenistan-6",
+      "-country": "TM",
+      "name": "Balkan",
+      "area": "233900",
+      "population": "925500",
+      "city": {
+        "-id": "cty-cid-cia-Turkmenistan-6",
+        "-is_state_cap": "yes",
+        "-country": "TM",
+        "-province": "prov-cid-cia-Turkmenistan-3",
+        "name": "Nebitdag",
+        "population": {
+          "-year": "91",
+          "#text": "89100"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkmenistan-4",
+      "-capital": "cty-cid-cia-Turkmenistan-4",
+      "-country": "TM",
+      "name": "Dashhowuz",
+      "area": "73600",
+      "population": "738000",
+      "city": {
+        "-id": "cty-cid-cia-Turkmenistan-4",
+        "-is_state_cap": "yes",
+        "-country": "TM",
+        "-province": "prov-cid-cia-Turkmenistan-4",
+        "name": "Tashauz",
+        "population": {
+          "-year": "91",
+          "#text": "117000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkmenistan-5",
+      "-capital": "cty-cid-cia-Turkmenistan-3",
+      "-country": "TM",
+      "name": "Leban",
+      "area": "93800",
+      "population": "774700",
+      "city": {
+        "-id": "cty-cid-cia-Turkmenistan-3",
+        "-is_state_cap": "yes",
+        "-country": "TM",
+        "-province": "prov-cid-cia-Turkmenistan-5",
+        "name": "Charjew",
+        "population": {
+          "-year": "91",
+          "#text": "166400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Turkmenistan-6",
+      "-capital": "cty-cid-cia-Turkmenistan-5",
+      "-country": "TM",
+      "name": "Mary",
+      "area": "86800",
+      "population": "859500",
+      "city": {
+        "-id": "cty-cid-cia-Turkmenistan-5",
+        "-is_state_cap": "yes",
+        "-country": "TM",
+        "-province": "prov-cid-cia-Turkmenistan-6",
+        "name": "Mary",
+        "population": {
+          "-year": "91",
+          "#text": "94900"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "UZB",
+  "-area": "447400",
+  "-capital": "cty-Uzbekistan-Tashkent",
+  "-memberships": "org-AsDB org-CIS org-CCC org-ESCAP org-ECE org-ECO org-EBRD org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMF org-IOC org-ISO org-ITU org-NAM org-ANC org-OSCE org-OIC org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Uzbekistan",
+  "population": "23418381",
+  "population_growth": "1.87",
+  "infant_mortality": "79.6",
+  "gdp_total": "54700",
+  "inflation": "7.7",
+  "indep_date": "1991-08-31",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "8.3",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "2.4",
+      "#text": "Tatar"
+    },
+    {
+      "-percentage": "4.7",
+      "#text": "Tajik"
+    },
+    {
+      "-percentage": "71.4",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "4.1",
+      "#text": "Kazak"
+    },
+    {
+      "-percentage": "2.1",
+      "#text": "Karakalpak"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "88",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "14.2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "4.4",
+      "#text": "Tajik"
+    },
+    {
+      "-percentage": "74.3",
+      "#text": "Uzbek"
+    }
+  ],
+  "border": [
+    {
+      "-country": "AFG",
+      "-length": "137"
+    },
+    {
+      "-country": "TAD",
+      "-length": "1161"
+    },
+    {
+      "-country": "TM",
+      "-length": "1621"
+    },
+    {
+      "-country": "KAZ",
+      "-length": "2203"
+    },
+    {
+      "-country": "KGZ",
+      "-length": "1099"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Uzbekistan-2",
+      "-capital": "cty-cid-cia-Uzbekistan-5",
+      "-country": "UZB",
+      "name": "Andijon",
+      "area": "4200",
+      "population": "1899000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-5",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-2",
+        "name": "Andijon",
+        "population": {
+          "-year": "93",
+          "#text": "303000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-3",
+      "-capital": "cty-cid-cia-Uzbekistan-6",
+      "-country": "UZB",
+      "name": "Bukhoro",
+      "area": "39400",
+      "population": "1262000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-6",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-3",
+        "name": "Bukhoro",
+        "population": {
+          "-year": "93",
+          "#text": "236000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-4",
+      "-capital": "cty-cid-cia-Uzbekistan-7",
+      "-country": "UZB",
+      "name": "Farghona",
+      "area": "7100",
+      "population": "2338000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Uzbekistan-7",
+          "-is_state_cap": "yes",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-4",
+          "name": "Farghona",
+          "population": {
+            "-year": "93",
+            "#text": "191000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Uzbekistan-8",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-4",
+          "name": "Quqon",
+          "population": {
+            "-year": "93",
+            "#text": "184000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Uzbekistan-14",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-4",
+          "name": "Marghilon",
+          "population": {
+            "-year": "93",
+            "#text": "129000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-5",
+      "-capital": "cty-cid-cia-Uzbekistan-16",
+      "-country": "UZB",
+      "name": "Jizzakh",
+      "area": "20500",
+      "population": "831000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-16",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-5",
+        "name": "Jizzakh",
+        "population": {
+          "-year": "93",
+          "#text": "116000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-6",
+      "-capital": "cty-cid-cia-Uzbekistan-13",
+      "-country": "UZB",
+      "name": "Khorazm",
+      "area": "6300",
+      "population": "1135000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-13",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-6",
+        "name": "Urganch",
+        "population": {
+          "-year": "93",
+          "#text": "135000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-7",
+      "-capital": "cty-cid-cia-Uzbekistan-4",
+      "-country": "UZB",
+      "name": "Namangan",
+      "area": "7900",
+      "population": "1652000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-4",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-7",
+        "name": "Namangan",
+        "population": {
+          "-year": "93",
+          "#text": "341000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Syrdarja"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-8",
+      "-capital": "cty-cid-cia-Uzbekistan-17",
+      "-country": "UZB",
+      "name": "Nawoiy",
+      "area": "110800",
+      "population": "715000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-17",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-8",
+        "name": "Nawoiy",
+        "population": {
+          "-year": "93",
+          "#text": "115000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-9",
+      "-capital": "cty-cid-cia-Uzbekistan-10",
+      "-country": "UZB",
+      "name": "Qasqadare",
+      "area": "28400",
+      "population": "1812000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-10",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-9",
+        "name": "Qarshi",
+        "population": {
+          "-year": "93",
+          "#text": "177000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-10",
+      "-capital": "cty-cid-cia-Uzbekistan-3",
+      "-country": "UZB",
+      "name": "Samarqand",
+      "area": "16400",
+      "population": "2322000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-3",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-10",
+        "name": "Samarqand",
+        "population": {
+          "-year": "93",
+          "#text": "368000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-11",
+      "-capital": "cty-cid-cia-Uzbekistan-Guliston",
+      "-country": "UZB",
+      "name": "Sirdare",
+      "area": "5100",
+      "population": "600000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-Guliston",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-11",
+        "name": "Guliston"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-12",
+      "-capital": "cty-cid-cia-Uzbekistan-Termiz",
+      "-country": "UZB",
+      "name": "Surkhondare",
+      "area": "20800",
+      "population": "1437000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-Termiz",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-12",
+        "name": "Termiz"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-13",
+      "-capital": "cty-Uzbekistan-Tashkent",
+      "-country": "UZB",
+      "name": "Toshkent",
+      "area": "15600",
+      "population": "4450000",
+      "city": [
+        {
+          "-id": "cty-Uzbekistan-Tashkent",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-13",
+          "name": "Tashkent",
+          "longitude": "69.3",
+          "latitude": "41.3333",
+          "population": {
+            "-year": "93",
+            "#text": "2106000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Uzbekistan-11",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-13",
+          "name": "Chirchiq",
+          "population": {
+            "-year": "93",
+            "#text": "156000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Uzbekistan-12",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-13",
+          "name": "Angren",
+          "population": {
+            "-year": "93",
+            "#text": "132000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Uzbekistan-15",
+          "-country": "UZB",
+          "-province": "prov-cid-cia-Uzbekistan-13",
+          "name": "Olmaliq",
+          "population": {
+            "-year": "93",
+            "#text": "116000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Uzbekistan-15",
+      "-capital": "cty-cid-cia-Uzbekistan-9",
+      "-country": "UZB",
+      "name": "Qaraqalpoghiston",
+      "area": "164900",
+      "population": "1343000",
+      "city": {
+        "-id": "cty-cid-cia-Uzbekistan-9",
+        "-is_state_cap": "yes",
+        "-country": "UZB",
+        "-province": "prov-cid-cia-Uzbekistan-15",
+        "name": "Nukus",
+        "population": {
+          "-year": "93",
+          "#text": "185000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "ARM",
+  "-area": "29800",
+  "-capital": "cty-cid-cia-Armenia-Yerevan",
+  "-memberships": "org-BSEC org-CIS org-CCC org-ESCAP org-ECE org-EBRD org-FAO org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ITU org-Intelsat org-NAM org-ANC org-OSCE org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Armenia",
+  "population": "3463574",
+  "population_growth": "0.02",
+  "infant_mortality": "38.9",
+  "gdp_total": "9100",
+  "gdp_agri": "57",
+  "gdp_ind": "36",
+  "gdp_serv": "7",
+  "inflation": "32.2",
+  "indep_date": "1918-05-28",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "93",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Azeri"
+    }
+  ],
+  "religions": {
+    "-percentage": "94",
+    "#text": "Christian Orthodox"
+  },
+  "languages": [
+    {
+      "-percentage": "2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "96",
+      "#text": "Armenian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TR",
+      "-length": "268"
+    },
+    {
+      "-country": "IR",
+      "-length": "35"
+    },
+    {
+      "-country": "GE",
+      "-length": "164"
+    },
+    {
+      "-country": "AZ",
+      "-length": "787"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Armenia-Yerevan",
+    "-is_country_cap": "yes",
+    "-country": "ARM",
+    "name": "Yerevan",
+    "longitude": "44.4",
+    "latitude": "40.1",
+    "population": {
+      "-year": "87",
+      "#text": "1200000"
+    }
+  }
+}
+{
+  "-car_code": "GE",
+  "-area": "69700",
+  "-capital": "cty-Georgia-Tbilisi",
+  "-memberships": "org-BSEC org-CIS org-CCC org-ECE org-EBRD org-FAO org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ITU org-ANC org-OSCE org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Georgia",
+  "population": "5219810",
+  "population_growth": "-1.02",
+  "infant_mortality": "22.5",
+  "gdp_total": "6200",
+  "gdp_agri": "70.4",
+  "gdp_ind": "10.2",
+  "gdp_serv": "19.4",
+  "inflation": "2.2",
+  "indep_date": "1991-04-09",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "6.3",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "8.1",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "5.7",
+      "#text": "Azeri"
+    },
+    {
+      "-percentage": "70.1",
+      "#text": "Georgian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Ossetian"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Abkhaz"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "11",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Armenian Apostolic"
+    },
+    {
+      "-percentage": "75",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "9",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Azeri"
+    },
+    {
+      "-percentage": "71",
+      "#text": "Georgian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "R",
+      "-length": "723"
+    },
+    {
+      "-country": "TR",
+      "-length": "252"
+    },
+    {
+      "-country": "ARM",
+      "-length": "164"
+    },
+    {
+      "-country": "AZ",
+      "-length": "322"
+    }
+  ],
+  "city": {
+    "-id": "cty-Georgia-Tbilisi",
+    "-is_country_cap": "yes",
+    "-country": "GE",
+    "name": "Tbilisi",
+    "longitude": "44.8",
+    "latitude": "41.7167",
+    "population": {
+      "-year": "87",
+      "#text": "1200000"
+    },
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Kura"
+    }
+  }
+}
+{
+  "-car_code": "AZ",
+  "-area": "86600",
+  "-capital": "cty-cid-cia-Azerbaijan-Baku",
+  "-memberships": "org-BSEC org-CIS org-CCC org-ESCAP org-ECE org-ECO org-EBRD org-FAO org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFAD org-ILO org-IMO org-IMF org-IOC org-ITU org-Intelsat org-IDB org-NAM org-ANC org-OSCE org-OIC org-PFP org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Azerbaijan",
+  "population": "7676953",
+  "population_growth": "0.78",
+  "infant_mortality": "74.5",
+  "gdp_total": "11500",
+  "inflation": "85",
+  "indep_date": "1991-08-30",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2.5",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "2.3",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Azeri"
+    },
+    {
+      "-percentage": "3.2",
+      "#text": "Dagestani Peoples"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "93.4",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "4.8",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "3",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "89",
+      "#text": "Azeri"
+    }
+  ],
+  "border": [
+    {
+      "-country": "R",
+      "-length": "284"
+    },
+    {
+      "-country": "TR",
+      "-length": "9"
+    },
+    {
+      "-country": "IR",
+      "-length": "611"
+    },
+    {
+      "-country": "ARM",
+      "-length": "787"
+    },
+    {
+      "-country": "GE",
+      "-length": "322"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Azerbaijan-Baku",
+    "-is_country_cap": "yes",
+    "-country": "AZ",
+    "name": "Baku",
+    "longitude": "49.5",
+    "latitude": "40.3",
+    "population": {
+      "-year": "87",
+      "#text": "1740000"
+    },
+    "located_at": {
+      "-watertype": "lake",
+      "-lake": "lake-KaspischesMeer"
+    }
+  }
+}
+{
+  "-car_code": "BRN",
+  "-area": "620",
+  "-capital": "cty-cid-cia-Bahrain-Manama",
+  "-memberships": "org-ABEDA org-AFESD org-AL org-AMF org-ESCWA org-FAO org-G-77 org-GCC org-IBRD org-ICAO org-Interpol org-IFRCS org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Bahrain",
+  "population": "590042",
+  "population_growth": "2.27",
+  "infant_mortality": "17.1",
+  "gdp_total": "7300",
+  "inflation": "3",
+  "indep_date": "1971-08-15",
+  "government": "traditional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "10",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "13",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "63",
+      "#text": "Bahraini"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Iranian"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Bahrain-Manama",
+    "-is_country_cap": "yes",
+    "-country": "BRN",
+    "name": "Manama",
+    "longitude": "50.5",
+    "latitude": "26",
+    "population": {
+      "-year": "87",
+      "#text": "145000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Persischer_Golf"
+    },
+    "located_on": { "-island": "island-Bahrain" }
+  }
+}
+{
+  "-car_code": "BD",
+  "-area": "144000",
+  "-capital": "cty-cid-cia-Bangladesh-2-0",
+  "-memberships": "org-AsDB org-CP org-C org-CCC org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-SAARC org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNMIH org-UNMOT org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Bangladesh",
+  "population": "123062800",
+  "population_growth": "1.85",
+  "infant_mortality": "102.3",
+  "gdp_total": "144500",
+  "inflation": "4.5",
+  "indep_date": "1971-12-16",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "98",
+    "#text": "Bengali"
+  },
+  "religions": [
+    {
+      "-percentage": "83",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "16",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "MYA",
+      "-length": "193"
+    },
+    {
+      "-country": "IND",
+      "-length": "4053"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Bangladesh-2-0",
+      "-is_country_cap": "yes",
+      "-country": "BD",
+      "name": "Dhaka",
+      "longitude": "90.5",
+      "latitude": "24",
+      "population": {
+        "-year": "91",
+        "#text": "3839000"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Ganges"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-4-0",
+      "-country": "BD",
+      "name": "Khulna",
+      "population": {
+        "-year": "91",
+        "#text": "731000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-5-0",
+      "-country": "BD",
+      "name": "Rajshahi",
+      "population": {
+        "-year": "91",
+        "#text": "318000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-6-0",
+      "-country": "BD",
+      "name": "Narayanganj",
+      "population": {
+        "-year": "91",
+        "#text": "296000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-7-0",
+      "-country": "BD",
+      "name": "Rangpur",
+      "population": {
+        "-year": "91",
+        "#text": "207000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-8-0",
+      "-country": "BD",
+      "name": "Mymensingh",
+      "population": {
+        "-year": "91",
+        "#text": "202000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-9-0",
+      "-country": "BD",
+      "name": "Barisal",
+      "population": {
+        "-year": "91",
+        "#text": "188000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-10-0",
+      "-country": "BD",
+      "name": "Tongi",
+      "population": {
+        "-year": "91",
+        "#text": "181000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-11-0",
+      "-country": "BD",
+      "name": "Comilla",
+      "population": {
+        "-year": "91",
+        "#text": "156000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-12-0",
+      "-country": "BD",
+      "name": "Jessore",
+      "population": {
+        "-year": "91",
+        "#text": "154000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-13-0",
+      "-country": "BD",
+      "name": "Nawabganj",
+      "population": {
+        "-year": "91",
+        "#text": "141000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-14-0",
+      "-country": "BD",
+      "name": "Dinajpur",
+      "population": {
+        "-year": "91",
+        "#text": "138000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-2-2",
+      "-country": "BD",
+      "name": "Bogra",
+      "population": {
+        "-year": "91",
+        "#text": "130000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-3-2",
+      "-country": "BD",
+      "name": "Brahmanbaria",
+      "population": {
+        "-year": "91",
+        "#text": "125000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-4-2",
+      "-country": "BD",
+      "name": "Savar",
+      "population": {
+        "-year": "91",
+        "#text": "115000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-5-2",
+      "-country": "BD",
+      "name": "Tangail",
+      "population": {
+        "-year": "91",
+        "#text": "114000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-6-2",
+      "-country": "BD",
+      "name": "Pabna",
+      "population": {
+        "-year": "91",
+        "#text": "112000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-7-2",
+      "-country": "BD",
+      "name": "Jamalpur",
+      "population": {
+        "-year": "91",
+        "#text": "111000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-8-2",
+      "-country": "BD",
+      "name": "Naogaon",
+      "population": {
+        "-year": "91",
+        "#text": "110000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-9-2",
+      "-country": "BD",
+      "name": "Sylhet",
+      "population": {
+        "-year": "91",
+        "#text": "109000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-10-2",
+      "-country": "BD",
+      "name": "Sirajganj",
+      "population": {
+        "-year": "91",
+        "#text": "108000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-11-2",
+      "-country": "BD",
+      "name": "Narsinghdi",
+      "population": {
+        "-year": "91",
+        "#text": "106000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-12-2",
+      "-country": "BD",
+      "name": "Saidpur",
+      "population": {
+        "-year": "91",
+        "#text": "105000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Bangladesh-13-2",
+      "-country": "BD",
+      "name": "Gazipur",
+      "population": {
+        "-year": "91",
+        "#text": "104000"
+      }
+    },
+    {
+      "-id": "cty-Bangladesh-Chittagong",
+      "-country": "BD",
+      "name": "Chittagong",
+      "longitude": "91.8333",
+      "latitude": "22.35",
+      "population": {
+        "-year": "91",
+        "#text": "1599000"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MYA",
+  "-area": "678500",
+  "-capital": "cty-Burma-Rangoon",
+  "-memberships": "org-AsDB org-Mekong-Group org-CP org-CCC org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WMO org-WTrO",
+  "name": "Myanmar",
+  "population": "45975625",
+  "population_growth": "1.84",
+  "infant_mortality": "80.7",
+  "gdp_total": "47000",
+  "gdp_agri": "60",
+  "gdp_ind": "10",
+  "gdp_serv": "30",
+  "inflation": "38",
+  "indep_date": "1948-01-04",
+  "government": "military regime",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "68",
+      "#text": "Burman"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Shan"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Karen"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Rakhine"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Mon"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "4",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "89",
+      "#text": "Buddhist"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Burmese"
+  },
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "2185"
+    },
+    {
+      "-country": "BD",
+      "-length": "193"
+    },
+    {
+      "-country": "IND",
+      "-length": "1463"
+    },
+    {
+      "-country": "LAO",
+      "-length": "235"
+    },
+    {
+      "-country": "THA",
+      "-length": "1800"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Burma-2",
+      "-capital": "cty-cid-cia-Burma-6",
+      "-country": "MYA",
+      "name": "Ayeyarwady",
+      "area": "35138",
+      "population": "4994100",
+      "city": {
+        "-id": "cty-cid-cia-Burma-6",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-2",
+        "name": "Pathein",
+        "population": {
+          "-year": "83",
+          "#text": "144100"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irawaddy"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-3",
+      "-capital": "cty-cid-cia-Burma-Magway",
+      "-country": "MYA",
+      "name": "Magway",
+      "area": "44820",
+      "population": "3243200",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Magway",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-3",
+        "name": "Magway",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irawaddy"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-4",
+      "-capital": "cty-Burma-Mandalay",
+      "-country": "MYA",
+      "name": "Mandalay",
+      "area": "37024",
+      "population": "4577800",
+      "city": {
+        "-id": "cty-Burma-Mandalay",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-4",
+        "name": "Mandalay",
+        "longitude": "96.1",
+        "latitude": "21.9833",
+        "population": {
+          "-year": "83",
+          "#text": "532900"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irawaddy"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-5",
+      "-capital": "cty-cid-cia-Burma-5",
+      "-country": "MYA",
+      "name": "Bago",
+      "area": "39404",
+      "population": "3799800",
+      "city": {
+        "-id": "cty-cid-cia-Burma-5",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-5",
+        "name": "Bago",
+        "population": {
+          "-year": "83",
+          "#text": "150500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-6",
+      "-capital": "cty-Burma-Rangoon",
+      "-country": "MYA",
+      "name": "Yangon",
+      "area": "10171",
+      "population": "3965900",
+      "city": {
+        "-id": "cty-Burma-Rangoon",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-6",
+        "name": "Rangoon",
+        "longitude": "96.15",
+        "latitude": "16.7833",
+        "population": {
+          "-year": "83",
+          "#text": "2513000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-7",
+      "-capital": "cty-cid-cia-Burma-Sagaing",
+      "-country": "MYA",
+      "name": "Sagaing",
+      "area": "94625",
+      "population": "3862200",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Burma-9",
+          "-country": "MYA",
+          "-province": "prov-cid-cia-Burma-7",
+          "name": "Monywa",
+          "population": {
+            "-year": "83",
+            "#text": "106800"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Burma-Sagaing",
+          "-is_state_cap": "yes",
+          "-country": "MYA",
+          "-province": "prov-cid-cia-Burma-7",
+          "name": "Sagaing",
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Irawaddy"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Burma-8",
+      "-capital": "cty-cid-cia-Burma-Tavoy",
+      "-country": "MYA",
+      "name": "Tanintharyi",
+      "area": "43343",
+      "population": "917200",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Tavoy",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-8",
+        "name": "Tavoy"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-9",
+      "-capital": "cty-cid-cia-Burma-Hakha",
+      "-country": "MYA",
+      "name": "Chin",
+      "area": "36019",
+      "population": "368900",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Hakha",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-9",
+        "name": "Hakha"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-10",
+      "-capital": "cty-cid-cia-Burma-Myitkyina",
+      "-country": "MYA",
+      "name": "Kachin",
+      "area": "89041",
+      "population": "904800",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Myitkyina",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-10",
+        "name": "Myitkyina",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irawaddy"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-11",
+      "-capital": "cty-cid-cia-Burma-Hpa-an",
+      "-country": "MYA",
+      "name": "Kayin",
+      "area": "30383",
+      "population": "1055400",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Hpa-an",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-11",
+        "name": "Hpa an"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-12",
+      "-capital": "cty-cid-cia-Burma-Loikaw",
+      "-country": "MYA",
+      "name": "Kayah",
+      "area": "11733",
+      "population": "168400",
+      "city": {
+        "-id": "cty-cid-cia-Burma-Loikaw",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-12",
+        "name": "Loikaw"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-13",
+      "-capital": "cty-cid-cia-Burma-4",
+      "-country": "MYA",
+      "name": "Mon",
+      "area": "12297",
+      "population": "1680200",
+      "city": {
+        "-id": "cty-cid-cia-Burma-4",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-13",
+        "name": "Moulmein",
+        "population": {
+          "-year": "83",
+          "#text": "220000"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Saluen"
+          },
+          {
+            "-watertype": "sea",
+            "-sea": "sea-AndamanSea"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-14",
+      "-capital": "cty-cid-cia-Burma-8",
+      "-country": "MYA",
+      "name": "Rakhine",
+      "area": "36778",
+      "population": "2045600",
+      "city": {
+        "-id": "cty-cid-cia-Burma-8",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-14",
+        "name": "Akyab",
+        "population": {
+          "-year": "83",
+          "#text": "107600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Burma-15",
+      "-capital": "cty-cid-cia-Burma-7",
+      "-country": "MYA",
+      "name": "Shan",
+      "area": "155801",
+      "population": "3716800",
+      "city": {
+        "-id": "cty-cid-cia-Burma-7",
+        "-is_state_cap": "yes",
+        "-country": "MYA",
+        "-province": "prov-cid-cia-Burma-15",
+        "name": "Taunggyi",
+        "population": {
+          "-year": "83",
+          "#text": "108200"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "IND",
+  "-area": "3287590",
+  "-capital": "cty-India-New-Delhi",
+  "-memberships": "org-AfDB org-AG org-AsDB org-CP org-C org-CCC org-ESCAP org-FAO org-G-6 org-G-15 org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAS org-PCA org-SAARC org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UNITAR org-UNIKOM org-UNMIH org-UNOMIL org-UNHCR org-UNU org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "India",
+  "population": "952107694",
+  "population_growth": "1.64",
+  "infant_mortality": "71.1",
+  "gdp_total": "1408700",
+  "inflation": "9",
+  "indep_date": "1947-08-15",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "25",
+      "#text": "Dravidian"
+    },
+    {
+      "-percentage": "72",
+      "#text": "Indo-Aryan"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Mongol"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "14",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "2.4",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "80",
+      "#text": "Hindu"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Sikh"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Jains"
+    }
+  ],
+  "languages": {
+    "-percentage": "30",
+    "#text": "Hindi"
+  },
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "3380"
+    },
+    {
+      "-country": "PK",
+      "-length": "2912"
+    },
+    {
+      "-country": "BD",
+      "-length": "4053"
+    },
+    {
+      "-country": "MYA",
+      "-length": "1463"
+    },
+    {
+      "-country": "BHT",
+      "-length": "605"
+    },
+    {
+      "-country": "NEP",
+      "-length": "1690"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-India-2",
+      "-capital": "cty-cid-cia-India-7",
+      "-country": "IND",
+      "name": "Andhra Pradesh",
+      "area": "275045",
+      "population": "66508008",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-7",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Hyderabad",
+          "longitude": "78.2",
+          "latitude": "17.1",
+          "population": {
+            "-year": "91",
+            "#text": "3145939"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-30",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Vishakhapatnam",
+          "population": {
+            "-year": "91",
+            "#text": "752037"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-32",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Vijayawada",
+          "population": {
+            "-year": "91",
+            "#text": "701827"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-55",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Guntur",
+          "population": {
+            "-year": "91",
+            "#text": "471051"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-58",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Warangal",
+          "population": {
+            "-year": "91",
+            "#text": "447653"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-78",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-2",
+          "name": "Rajahmundry",
+          "population": {
+            "-year": "91",
+            "#text": "324881"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-3",
+      "-capital": "cty-cid-cia-India-Itanagar",
+      "-country": "IND",
+      "name": "Arunachal Pradesh",
+      "area": "83743",
+      "population": "864558",
+      "city": {
+        "-id": "cty-cid-cia-India-Itanagar",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-3",
+        "name": "Itanagar",
+        "longitude": "93.4",
+        "latitude": "27.1"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-4",
+      "-capital": "cty-cid-cia-India-Dispur",
+      "-country": "IND",
+      "name": "Assam",
+      "area": "78438",
+      "population": "22414322",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-45",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-4",
+          "name": "Gauhati",
+          "population": {
+            "-year": "91",
+            "#text": "584342"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Brahmaputra"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-Dispur",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-4",
+          "name": "Dispur",
+          "longitude": "91.5",
+          "latitude": "26"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-5",
+      "-capital": "cty-cid-cia-India-23",
+      "-country": "IND",
+      "name": "Bihar",
+      "area": "173877",
+      "population": "86374465",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-23",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-5",
+          "name": "Patna",
+          "longitude": "85",
+          "latitude": "25.4",
+          "population": {
+            "-year": "91",
+            "#text": "917243"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ganges"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-42",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-5",
+          "name": "Ranchi",
+          "population": {
+            "-year": "91",
+            "#text": "599306"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-54",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-5",
+          "name": "Jamshedpur",
+          "population": {
+            "-year": "91",
+            "#text": "478950"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-77",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-5",
+          "name": "Bokara Steel City",
+          "population": {
+            "-year": "91",
+            "#text": "333683"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-6",
+      "-capital": "cty-cid-cia-India-Panaji",
+      "-country": "IND",
+      "name": "Goa",
+      "area": "3702",
+      "population": "1169793",
+      "city": {
+        "-id": "cty-cid-cia-India-Panaji",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-6",
+        "name": "Panaji",
+        "longitude": "74",
+        "latitude": "15.4",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Arabisches_Meer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-7",
+      "-capital": "cty-cid-cia-India-Gandhinagar",
+      "-country": "IND",
+      "name": "Gujarat",
+      "area": "196024",
+      "population": "41309582",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-8",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Ahmadabad",
+          "population": {
+            "-year": "91",
+            "#text": "2954526"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-13",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Surat",
+          "population": {
+            "-year": "91",
+            "#text": "1505872"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-17",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Vadodara",
+          "population": {
+            "-year": "91",
+            "#text": "1061598"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-39",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Rajkot",
+          "population": {
+            "-year": "91",
+            "#text": "612458"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-68",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Bhavnagar",
+          "population": {
+            "-year": "91",
+            "#text": "402338"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-76",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Jamnagar",
+          "population": {
+            "-year": "91",
+            "#text": "350544"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Arabisches_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-Gandhinagar",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-7",
+          "name": "Gandhinagar",
+          "longitude": "72.2",
+          "latitude": "23.2"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-8",
+      "-capital": "cty-cid-cia-India-50",
+      "-country": "IND",
+      "name": "Haryana",
+      "area": "44212",
+      "population": "16463648"
+    },
+    {
+      "-id": "prov-cid-cia-India-9",
+      "-capital": "cty-cid-cia-India-Simla",
+      "-country": "IND",
+      "name": "Himachal Pradesh",
+      "area": "55673",
+      "population": "5170877",
+      "city": {
+        "-id": "cty-cid-cia-India-Simla",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-9",
+        "name": "Simla",
+        "longitude": "77.1",
+        "latitude": "31"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-10",
+      "-capital": "cty-cid-cia-India-43",
+      "-country": "IND",
+      "name": "Jammu and Kashmir",
+      "area": "101387",
+      "population": "7718700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-43",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-10",
+          "name": "Srinagar",
+          "longitude": "74.3",
+          "latitude": "34",
+          "population": {
+            "-year": "91",
+            "#text": "594775"
+          }
+        },
+        {
+          "-id": "stadt-Jammu-IND-JAK",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-10",
+          "name": "Jammu",
+          "longitude": "74.3",
+          "latitude": "32.2"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-11",
+      "-capital": "cty-India-Bangalore",
+      "-country": "IND",
+      "name": "Karnataka",
+      "area": "191791",
+      "population": "44977201",
+      "city": [
+        {
+          "-id": "cty-India-Bangalore",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-11",
+          "name": "Bangalore",
+          "longitude": "77.6167",
+          "latitude": "12.95",
+          "population": {
+            "-year": "91",
+            "#text": "3302296"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-37",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-11",
+          "name": "Hubli",
+          "population": {
+            "-year": "91",
+            "#text": "648298"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-40",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-11",
+          "name": "Mysore",
+          "population": {
+            "-year": "91",
+            "#text": "606755"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-12",
+      "-capital": "cty-cid-cia-India-33",
+      "-country": "IND",
+      "name": "Kerala",
+      "area": "38863",
+      "population": "29098518",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-33",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-12",
+          "name": "Trivandrum",
+          "longitude": "77",
+          "latitude": "8.2",
+          "population": {
+            "-year": "91",
+            "#text": "699872"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-46",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-12",
+          "name": "Cochin",
+          "population": {
+            "-year": "91",
+            "#text": "582588"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Arabisches_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-56",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-12",
+          "name": "Kozhikode",
+          "population": {
+            "-year": "91",
+            "#text": "456618"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-13",
+      "-capital": "cty-cid-cia-India-16",
+      "-country": "IND",
+      "name": "Madhya Pradesh",
+      "area": "443446",
+      "population": "66181170",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-16",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Bhopal",
+          "longitude": "77.4",
+          "latitude": "23.1",
+          "population": {
+            "-year": "91",
+            "#text": "1062771"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-15",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Indore",
+          "population": {
+            "-year": "91",
+            "#text": "1091674"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-28",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Jabalpur",
+          "population": {
+            "-year": "91",
+            "#text": "764586"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-34",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Gwalior",
+          "population": {
+            "-year": "91",
+            "#text": "690765"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-59",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Raipur",
+          "population": {
+            "-year": "91",
+            "#text": "438639"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-69",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Bhilai",
+          "population": {
+            "-year": "91",
+            "#text": "395360"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-75",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Ujjain",
+          "population": {
+            "-year": "91",
+            "#text": "362633"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-79",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-13",
+          "name": "Bhatpara",
+          "population": {
+            "-year": "91",
+            "#text": "315976"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-14",
+      "-capital": "cty-India-Bombay",
+      "-country": "IND",
+      "name": "Maharashtra",
+      "area": "307713",
+      "population": "78937187",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-2",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Mumbai",
+          "longitude": "72.5",
+          "latitude": "19",
+          "population": {
+            "-year": "91",
+            "#text": "9925891"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Arabisches_Meer"
+          }
+        },
+        {
+          "-id": "cty-India-Nagpur",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Nagpur",
+          "longitude": "79.1167",
+          "latitude": "21.15",
+          "population": {
+            "-year": "91",
+            "#text": "1624752"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-12",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Pune",
+          "population": {
+            "-year": "91",
+            "#text": "1566651"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-19",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Kalyan",
+          "population": {
+            "-year": "91",
+            "#text": "1014557"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-27",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Thane",
+          "population": {
+            "-year": "91",
+            "#text": "803389"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-36",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Nasik",
+          "population": {
+            "-year": "91",
+            "#text": "656925"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-41",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Sholapur",
+          "population": {
+            "-year": "91",
+            "#text": "604215"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-47",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Aurangabad",
+          "population": {
+            "-year": "91",
+            "#text": "573272"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-62",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Amravati",
+          "population": {
+            "-year": "91",
+            "#text": "421576"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-65",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Kolhapur",
+          "population": {
+            "-year": "91",
+            "#text": "406370"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-70",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Bhiwandi",
+          "population": {
+            "-year": "91",
+            "#text": "392214"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-73",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Ulhasnagar",
+          "population": {
+            "-year": "91",
+            "#text": "369077"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-81",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "New Bombay",
+          "population": {
+            "-year": "91",
+            "#text": "307724"
+          }
+        },
+        {
+          "-id": "cty-India-Bombay",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Bombay",
+          "longitude": "72.8167",
+          "latitude": "18.9"
+        },
+        {
+          "-id": "cty-cid-cia-India-49",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-14",
+          "name": "Pimpri Chinchwad",
+          "population": {
+            "-year": "91",
+            "#text": "517083"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-15",
+      "-capital": "cty-cid-cia-India-Imphal",
+      "-country": "IND",
+      "name": "Manipur",
+      "area": "22327",
+      "population": "1837149",
+      "city": {
+        "-id": "cty-cid-cia-India-Imphal",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-15",
+        "name": "Imphal",
+        "longitude": "94",
+        "latitude": "24.5"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-16",
+      "-capital": "cty-cid-cia-India-Shillong",
+      "-country": "IND",
+      "name": "Meghalaya",
+      "area": "22429",
+      "population": "1774778",
+      "city": {
+        "-id": "cty-cid-cia-India-Shillong",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-16",
+        "name": "Shillong",
+        "longitude": "91.5",
+        "latitude": "25.4"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-17",
+      "-capital": "cty-cid-cia-India-Aijal",
+      "-country": "IND",
+      "name": "Mizoram",
+      "area": "21081",
+      "population": "689756",
+      "city": {
+        "-id": "cty-cid-cia-India-Aijal",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-17",
+        "name": "Aijal",
+        "longitude": "92.4",
+        "latitude": "23.4"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-18",
+      "-capital": "cty-cid-cia-India-Kohima",
+      "-country": "IND",
+      "name": "Nagaland",
+      "area": "16579",
+      "population": "1209546",
+      "city": {
+        "-id": "cty-cid-cia-India-Kohima",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-18",
+        "name": "Kohima",
+        "longitude": "94.1",
+        "latitude": "25.5"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-19",
+      "-capital": "cty-cid-cia-India-64",
+      "-country": "IND",
+      "name": "Orissa",
+      "area": "155707",
+      "population": "31659736",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-64",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-19",
+          "name": "Bhubaneswar",
+          "longitude": "85.5",
+          "latitude": "20.2",
+          "population": {
+            "-year": "91",
+            "#text": "411542"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-66",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-19",
+          "name": "Cuttack",
+          "population": {
+            "-year": "91",
+            "#text": "403418"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-20",
+      "-capital": "cty-cid-cia-India-50",
+      "-country": "IND",
+      "name": "Punjab",
+      "area": "50362",
+      "population": "20281969",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-18",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-20",
+          "name": "Ludhiana",
+          "population": {
+            "-year": "91",
+            "#text": "1042740"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-31",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-20",
+          "name": "Amritsar",
+          "population": {
+            "-year": "91",
+            "#text": "708835"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-51",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-20",
+          "name": "Jalandhar",
+          "population": {
+            "-year": "91",
+            "#text": "509510"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-21",
+      "-capital": "cty-cid-cia-India-14",
+      "-country": "IND",
+      "name": "Rajasthan",
+      "area": "342239",
+      "population": "44005990",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-14",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-21",
+          "name": "Jaipur",
+          "longitude": "75.5",
+          "latitude": "27",
+          "population": {
+            "-year": "91",
+            "#text": "1458183"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-35",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-21",
+          "name": "Jodhpur",
+          "population": {
+            "-year": "91",
+            "#text": "666279"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-48",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-21",
+          "name": "Kota",
+          "population": {
+            "-year": "91",
+            "#text": "537371"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-63",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-21",
+          "name": "Bikaner",
+          "population": {
+            "-year": "91",
+            "#text": "416289"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-67",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-21",
+          "name": "Ajmer",
+          "population": {
+            "-year": "91",
+            "#text": "402700"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-22",
+      "-capital": "cty-cid-cia-India-Gangtok",
+      "-country": "IND",
+      "name": "Sikkim",
+      "area": "7096",
+      "population": "406457",
+      "city": {
+        "-id": "cty-cid-cia-India-Gangtok",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-22",
+        "name": "Gangtok",
+        "longitude": "88.3",
+        "latitude": "27.3"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-23",
+      "-capital": "cty-India-Madras",
+      "-country": "IND",
+      "name": "Tamil Nadu",
+      "area": "130058",
+      "population": "55858946",
+      "city": [
+        {
+          "-id": "cty-India-Madras",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-23",
+          "name": "Madras",
+          "longitude": "80.25",
+          "latitude": "13.0667",
+          "population": {
+            "-year": "91",
+            "#text": "3841396"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Bengal"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-21",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-23",
+          "name": "Madurai",
+          "population": {
+            "-year": "91",
+            "#text": "940989"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-25",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-23",
+          "name": "Coimbatore",
+          "population": {
+            "-year": "91",
+            "#text": "816321"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-71",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-23",
+          "name": "Tiruchchirappalli",
+          "population": {
+            "-year": "91",
+            "#text": "387223"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-74",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-23",
+          "name": "Salem",
+          "population": {
+            "-year": "91",
+            "#text": "366712"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-24",
+      "-capital": "cty-cid-cia-India-Agartala",
+      "-country": "IND",
+      "name": "Tripura",
+      "area": "10486",
+      "population": "2757205",
+      "city": {
+        "-id": "cty-cid-cia-India-Agartala",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-24",
+        "name": "Agartala",
+        "longitude": "91.2",
+        "latitude": "23.5"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-25",
+      "-capital": "cty-cid-cia-India-11",
+      "-country": "IND",
+      "name": "Uttar Pradesh",
+      "area": "294411",
+      "population": "139112287",
+      "city": [
+        {
+          "-id": "cty-cid-cia-India-11",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Lucknow",
+          "longitude": "81",
+          "latitude": "27",
+          "population": {
+            "-year": "91",
+            "#text": "1619115"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-9",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Kanpur",
+          "population": {
+            "-year": "91",
+            "#text": "1879420"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ganges"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-22",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Varanasi",
+          "population": {
+            "-year": "91",
+            "#text": "932399"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ganges"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-24",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Agra",
+          "population": {
+            "-year": "91",
+            "#text": "891790"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-26",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Allahabad",
+          "population": {
+            "-year": "91",
+            "#text": "806486"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ganges"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-29",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Meerut",
+          "population": {
+            "-year": "91",
+            "#text": "753778"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-38",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Faridabad",
+          "population": {
+            "-year": "91",
+            "#text": "617717"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-44",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Bareilly",
+          "population": {
+            "-year": "91",
+            "#text": "590661"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-52",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Gorakhpur",
+          "population": {
+            "-year": "91",
+            "#text": "505566"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-53",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Aligarh",
+          "population": {
+            "-year": "91",
+            "#text": "480520"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-60",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Moradabad",
+          "population": {
+            "-year": "91",
+            "#text": "429214"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-72",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Saharanpur",
+          "population": {
+            "-year": "91",
+            "#text": "374945"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-80",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-25",
+          "name": "Jhansi",
+          "population": {
+            "-year": "91",
+            "#text": "313491"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-26",
+      "-capital": "cty-India-Calcutta",
+      "-country": "IND",
+      "name": "West Bengal",
+      "area": "88752",
+      "population": "68077965",
+      "city": [
+        {
+          "-id": "cty-India-Calcutta",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-26",
+          "name": "Calcutta",
+          "longitude": "88.3333",
+          "latitude": "22.5333",
+          "population": {
+            "-year": "91",
+            "#text": "4399819"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-20",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-26",
+          "name": "Haora",
+          "population": {
+            "-year": "91",
+            "#text": "950435"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-61",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-26",
+          "name": "Durgapur",
+          "population": {
+            "-year": "91",
+            "#text": "425836"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-28",
+      "-capital": "cty-cid-cia-India-Port-Blair",
+      "-country": "IND",
+      "name": "Andaman and Nicobar Is.",
+      "area": "8249",
+      "population": "280661",
+      "city": {
+        "-id": "cty-cid-cia-India-Port-Blair",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-28",
+        "name": "Port Blair",
+        "longitude": "92.5",
+        "latitude": "11.4",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Bengal sea-AndamanSea"
+        },
+        "located_on": { "-island": "island-SouthAndaman" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-29",
+      "-capital": "cty-cid-cia-India-50",
+      "-country": "IND",
+      "name": "Chandigarh",
+      "area": "114",
+      "population": "642015",
+      "city": {
+        "-id": "cty-cid-cia-India-50",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-29",
+        "name": "Chandigarh",
+        "longitude": "76.5",
+        "latitude": "30.5",
+        "population": {
+          "-year": "91",
+          "#text": "510565"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-30",
+      "-capital": "cty-cid-cia-India-Silvassa",
+      "-country": "IND",
+      "name": "Dadra and Nagar Haveli",
+      "area": "491",
+      "population": "138477",
+      "city": {
+        "-id": "cty-cid-cia-India-Silvassa",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-30",
+        "name": "Silvassa",
+        "longitude": "73.1",
+        "latitude": "20.2"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-31",
+      "-capital": "cty-cid-cia-India-Daman",
+      "-country": "IND",
+      "name": "Daman and Diu",
+      "area": "112",
+      "population": "101586",
+      "city": {
+        "-id": "cty-cid-cia-India-Daman",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-31",
+        "name": "Daman"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-32",
+      "-capital": "cty-India-New-Delhi",
+      "-country": "IND",
+      "name": "Delhi",
+      "area": "1483",
+      "population": "9420644",
+      "city": [
+        {
+          "-id": "cty-India-New-Delhi",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-32",
+          "name": "New Delhi",
+          "longitude": "77.2",
+          "latitude": "28.5833",
+          "population": {
+            "-year": "91",
+            "#text": "7206704"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-India-57",
+          "-country": "IND",
+          "-province": "prov-cid-cia-India-32",
+          "name": "Ghaziabad",
+          "population": {
+            "-year": "91",
+            "#text": "454156"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-India-33",
+      "-capital": "cty-cid-cia-India-Kavaratti",
+      "-country": "IND",
+      "name": "Lakshadweep Is.",
+      "area": "32",
+      "population": "51707",
+      "city": {
+        "-id": "cty-cid-cia-India-Kavaratti",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-33",
+        "name": "Kavaratti",
+        "longitude": "72.6",
+        "latitude": "10.6",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Arabisches_Meer"
+        },
+        "located_on": { "-island": "island-Kavaratti" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-India-34",
+      "-capital": "cty-cid-cia-India-Pondicherry",
+      "-country": "IND",
+      "name": "Pondicherry",
+      "area": "492",
+      "population": "807785",
+      "city": {
+        "-id": "cty-cid-cia-India-Pondicherry",
+        "-is_state_cap": "yes",
+        "-country": "IND",
+        "-province": "prov-cid-cia-India-34",
+        "name": "Pondicherry",
+        "longitude": "79.5",
+        "latitude": "12"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BHT",
+  "-area": "47000",
+  "-capital": "cty-cid-cia-Bhutan-Thimphu",
+  "-memberships": "org-AsDB org-CP org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-IDA org-IFAD org-IMF org-IOC org-ITU org-Intelsat org-NAM org-SAARC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO",
+  "name": "Bhutan",
+  "population": "1822625",
+  "population_growth": "2.32",
+  "infant_mortality": "116.3",
+  "gdp_total": "1300",
+  "inflation": "8.6",
+  "indep_date": "1949-08-08",
+  "government": "monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "50",
+      "#text": "Bhote"
+    },
+    {
+      "-percentage": "35",
+      "#text": "Nepalese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "75",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "470"
+    },
+    {
+      "-country": "IND",
+      "-length": "605"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Bhutan-Thimphu",
+    "-is_country_cap": "yes",
+    "-country": "BHT",
+    "name": "Thimphu",
+    "longitude": "90",
+    "latitude": "27",
+    "population": {
+      "-year": "87",
+      "#text": "20000"
+    }
+  }
+}
+{
+  "-car_code": "BRU",
+  "-area": "5770",
+  "-capital": "cty-cid-cia-Brunei-Bandar-Seri-Begawan",
+  "-memberships": "org-ASEAN org-Mekong-Group org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ITU org-Intelsat org-IDB org-NAM org-OIC org-UN org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Brunei",
+  "population": "299939",
+  "population_growth": "2.56",
+  "infant_mortality": "24.2",
+  "gdp_total": "4600",
+  "gdp_agri": "3",
+  "gdp_ind": "42",
+  "gdp_serv": "55",
+  "inflation": "2.4",
+  "indep_date": "1984-01-01",
+  "government": "constitutional sultanate",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "20",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "64",
+      "#text": "Malay"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "63",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "14",
+      "#text": "Buddhist"
+    }
+  ],
+  "border": {
+    "-country": "MAL",
+    "-length": "381"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Brunei-Bandar-Seri-Begawan",
+    "-is_country_cap": "yes",
+    "-country": "BRU",
+    "name": "Bandar Seri Begawan",
+    "longitude": "115",
+    "latitude": "5",
+    "population": {
+      "-year": "87",
+      "#text": "56300"
+    },
+    "located_on": { "-island": "island-Borneo" }
+  }
+}
+{
+  "-car_code": "MAL",
+  "-area": "329750",
+  "-capital": "cty-Malaysia-Kuala-Lumpur",
+  "-memberships": "org-AsDB org-ASEAN org-Mekong-Group org-CP org-C org-CCC org-ESCAP org-FAO org-G-15 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNOMIL org-UNPREDEP org-UNPROFOR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Malaysia",
+  "population": "19962893",
+  "population_growth": "2.07",
+  "infant_mortality": "24",
+  "gdp_total": "193600",
+  "gdp_agri": "8",
+  "gdp_ind": "25",
+  "gdp_serv": "67",
+  "inflation": "5.3",
+  "indep_date": "1957-08-31",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "9",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "32",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "59",
+      "#text": "Malay"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BRU",
+      "-length": "381"
+    },
+    {
+      "-country": "THA",
+      "-length": "506"
+    },
+    {
+      "-country": "RI",
+      "-length": "1782"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Malaysia-2",
+      "-capital": "cty-cid-cia-Malaysia-4",
+      "-country": "MAL",
+      "name": "Johor",
+      "area": "18986",
+      "population": "2074297",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-4",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-2",
+        "name": "Johor Baharu",
+        "population": {
+          "-year": "91",
+          "#text": "328646"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-3",
+      "-capital": "cty-cid-cia-Malaysia-19",
+      "-country": "MAL",
+      "name": "Kedah",
+      "area": "9426",
+      "population": "1304800",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-19",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-3",
+        "name": "Alor Setar",
+        "population": {
+          "-year": "91",
+          "#text": "125026"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-4",
+      "-capital": "cty-cid-cia-Malaysia-11",
+      "-country": "MAL",
+      "name": "Kelantan",
+      "area": "14943",
+      "population": "1181680",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-11",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-4",
+        "name": "Kota Baharu",
+        "population": {
+          "-year": "91",
+          "#text": "219713"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-5",
+      "-capital": "cty-cid-cia-Malaysia-5",
+      "-country": "MAL",
+      "name": "Melaka",
+      "area": "1650",
+      "population": "504502",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-5",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-5",
+        "name": "Melaka",
+        "population": {
+          "-year": "91",
+          "#text": "295999"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-MalakkaStrait"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-6",
+      "-capital": "cty-cid-cia-Malaysia-16",
+      "-country": "MAL",
+      "name": "Negeri Sembilan",
+      "area": "6643",
+      "population": "691150",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-16",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-6",
+        "name": "Seremban",
+        "population": {
+          "-year": "91",
+          "#text": "182584"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-7",
+      "-capital": "cty-cid-cia-Malaysia-14",
+      "-country": "MAL",
+      "name": "Pahang",
+      "area": "35965",
+      "population": "1036724",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-14",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-7",
+        "name": "Kuantan",
+        "population": {
+          "-year": "91",
+          "#text": "198356"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-8",
+      "-capital": "cty-cid-cia-Malaysia-3",
+      "-country": "MAL",
+      "name": "Perak",
+      "area": "21005",
+      "population": "1880016",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Malaysia-3",
+          "-is_state_cap": "yes",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-8",
+          "name": "Ipoh",
+          "population": {
+            "-year": "91",
+            "#text": "382633"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-15",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-8",
+          "name": "Taiping",
+          "population": {
+            "-year": "91",
+            "#text": "183165"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-9",
+      "-capital": "cty-cid-cia-Malaysia-Kangar",
+      "-country": "MAL",
+      "name": "Perlis",
+      "area": "795",
+      "population": "184070",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-Kangar",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-9",
+        "name": "Kangar"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-10",
+      "-capital": "cty-cid-cia-Malaysia-12",
+      "-country": "MAL",
+      "name": "Pulau Pinang",
+      "area": "1031",
+      "population": "1065075",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-12",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-10",
+        "name": "Penang",
+        "population": {
+          "-year": "91",
+          "#text": "219376"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-MalakkaStrait"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-11",
+      "-capital": "cty-cid-cia-Malaysia-13",
+      "-country": "MAL",
+      "name": "Sabah",
+      "area": "73620",
+      "population": "1736902",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Malaysia-13",
+          "-is_state_cap": "yes",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-11",
+          "name": "Kota Kinabalu",
+          "population": {
+            "-year": "91",
+            "#text": "208484"
+          },
+          "located_on": { "-island": "island-Borneo" }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-7",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-11",
+          "name": "Tawai",
+          "population": {
+            "-year": "91",
+            "#text": "244765"
+          },
+          "located_on": { "-island": "island-Borneo" }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-10",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-11",
+          "name": "Sandakan",
+          "population": {
+            "-year": "91",
+            "#text": "223432"
+          },
+          "located_on": { "-island": "island-Borneo" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-12",
+      "-capital": "cty-cid-cia-Malaysia-17",
+      "-country": "MAL",
+      "name": "Sarawak",
+      "area": "124449",
+      "population": "1648217",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Malaysia-17",
+          "-is_state_cap": "yes",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-12",
+          "name": "Kuching",
+          "population": {
+            "-year": "91",
+            "#text": "147729"
+          },
+          "located_on": { "-island": "island-Borneo" }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-18",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-12",
+          "name": "Sibu",
+          "population": {
+            "-year": "91",
+            "#text": "126384"
+          },
+          "located_on": { "-island": "island-Borneo" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-13",
+      "-capital": "cty-cid-cia-Malaysia-21",
+      "-country": "MAL",
+      "name": "Selangor",
+      "area": "7956",
+      "population": "2289236",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Malaysia-21",
+          "-is_state_cap": "yes",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-13",
+          "name": "Shah Alam",
+          "population": {
+            "-year": "91",
+            "#text": "101733"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-8",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-13",
+          "name": "Kelang",
+          "population": {
+            "-year": "91",
+            "#text": "243698"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-20",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-13",
+          "name": "Selayang Baru",
+          "population": {
+            "-year": "91",
+            "#text": "124606"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-14",
+      "-capital": "cty-cid-cia-Malaysia-9",
+      "-country": "MAL",
+      "name": "Terengganu",
+      "area": "12955",
+      "population": "770931",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-9",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-14",
+        "name": "Kuala Terengganu",
+        "population": {
+          "-year": "91",
+          "#text": "228654"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-15",
+      "-capital": "cty-Malaysia-Kuala-Lumpur",
+      "-country": "MAL",
+      "name": "Fed. Terr. of Kuala Lumpur",
+      "area": "243",
+      "population": "1145075",
+      "city": [
+        {
+          "-id": "cty-Malaysia-Kuala-Lumpur",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-15",
+          "name": "Kuala Lumpur",
+          "longitude": "101.7",
+          "latitude": "3.11667",
+          "population": {
+            "-year": "91",
+            "#text": "1145075"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Malaysia-6",
+          "-country": "MAL",
+          "-province": "prov-cid-cia-Malaysia-15",
+          "name": "Petaling Jaya",
+          "population": {
+            "-year": "91",
+            "#text": "254849"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Malaysia-16",
+      "-capital": "cty-cid-cia-Malaysia-Labuan",
+      "-country": "MAL",
+      "name": "Fed. Terr. of Labuan",
+      "area": "91",
+      "population": "54307",
+      "city": {
+        "-id": "cty-cid-cia-Malaysia-Labuan",
+        "-is_state_cap": "yes",
+        "-country": "MAL",
+        "-province": "prov-cid-cia-Malaysia-16",
+        "name": "Labuan"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "LAO",
+  "-area": "236800",
+  "-capital": "cty-cid-cia-Laos-Vientiane",
+  "-memberships": "org-ACCT org-AsDB org-ASEAN org-Mekong-Group org-CP org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Laos",
+  "population": "4975772",
+  "population_growth": "2.81",
+  "infant_mortality": "96.8",
+  "gdp_total": "5200",
+  "gdp_agri": "50",
+  "gdp_ind": "17",
+  "gdp_serv": "33",
+  "inflation": "20",
+  "indep_date": "1949-07-19",
+  "government": "Communist state",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "68",
+      "#text": "Lao Loum"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Lao Theung"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Lao Soung Hmong Yao"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Vietnamese/Chinese"
+    }
+  ],
+  "religions": {
+    "-percentage": "60",
+    "#text": "Buddhist"
+  },
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "423"
+    },
+    {
+      "-country": "MYA",
+      "-length": "235"
+    },
+    {
+      "-country": "THA",
+      "-length": "1754"
+    },
+    {
+      "-country": "K",
+      "-length": "541"
+    },
+    {
+      "-country": "VN",
+      "-length": "2130"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Laos-Vientiane",
+    "-is_country_cap": "yes",
+    "-country": "LAO",
+    "name": "Vientiane",
+    "longitude": "102.45",
+    "latitude": "18",
+    "population": {
+      "-year": "87",
+      "#text": "377409"
+    },
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Mekong"
+    }
+  }
+}
+{
+  "-car_code": "THA",
+  "-area": "514000",
+  "-capital": "cty-Thailand-Bangkok",
+  "-memberships": "org-AsDB org-ASEAN org-Mekong-Group org-CP org-CCC org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-PCA org-UN org-UNESCO org-UNIDO org-UNIKOM org-UNHCR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Thailand",
+  "population": "58851357",
+  "population_growth": "1.03",
+  "infant_mortality": "33.4",
+  "gdp_total": "416700",
+  "gdp_agri": "10.2",
+  "gdp_ind": "30.6",
+  "gdp_serv": "59.2",
+  "inflation": "5.8",
+  "indep_date": "1238-01-01",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "14",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "75",
+      "#text": "Thai"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "3.8",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "95",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "0.1",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "MYA",
+      "-length": "1800"
+    },
+    {
+      "-country": "MAL",
+      "-length": "506"
+    },
+    {
+      "-country": "LAO",
+      "-length": "1754"
+    },
+    {
+      "-country": "K",
+      "-length": "803"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Thailand-7",
+      "-country": "THA",
+      "name": "Chiang Mai",
+      "population": {
+        "-year": "90",
+        "#text": "167000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-8",
+      "-country": "THA",
+      "name": "Nakhon Sawan",
+      "population": {
+        "-year": "90",
+        "#text": "152000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-6",
+      "-country": "THA",
+      "name": "Khon Kaen",
+      "population": {
+        "-year": "90",
+        "#text": "206000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-3",
+      "-country": "THA",
+      "name": "Nakhon Ratchasima",
+      "population": {
+        "-year": "90",
+        "#text": "278000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-9",
+      "-country": "THA",
+      "name": "Ubon Ratchathani",
+      "population": {
+        "-year": "90",
+        "#text": "137000"
+      }
+    },
+    {
+      "-id": "cty-Thailand-Bangkok",
+      "-is_country_cap": "yes",
+      "-country": "THA",
+      "name": "Bangkok",
+      "longitude": "100.5",
+      "latitude": "13.7333",
+      "population": {
+        "-year": "90",
+        "#text": "5876000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-5",
+      "-country": "THA",
+      "name": "Nonthaburi",
+      "population": {
+        "-year": "90",
+        "#text": "233000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-11",
+      "-country": "THA",
+      "name": "Saraburi",
+      "population": {
+        "-year": "90",
+        "#text": "107000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-10",
+      "-country": "THA",
+      "name": "Nakhon si Thammarat",
+      "population": {
+        "-year": "90",
+        "#text": "112000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Thailand-4",
+      "-country": "THA",
+      "name": "Songkhla",
+      "population": {
+        "-year": "90",
+        "#text": "243000"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "K",
+  "-area": "181040",
+  "-capital": "cty-Cambodia-Phnom-Penh",
+  "-memberships": "org-ACCT org-AsDB org-Mekong-Group org-CP org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFAD org-ILO org-IMO org-IMF org-ICRM org-ITU org-Intelsat org-NAM org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Cambodia",
+  "population": "10861218",
+  "population_growth": "2.77",
+  "infant_mortality": "107.8",
+  "gdp_total": "7000",
+  "gdp_agri": "52",
+  "gdp_ind": "13.5",
+  "gdp_serv": "34.5",
+  "inflation": "6",
+  "indep_date": "1949-11-09",
+  "government": "multiparty liberal democracy under a constitutional monarchy established in September 1993",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Khmer"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Vietnamese"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Buddhist"
+  },
+  "border": [
+    {
+      "-country": "LAO",
+      "-length": "541"
+    },
+    {
+      "-country": "THA",
+      "-length": "803"
+    },
+    {
+      "-country": "VN",
+      "-length": "1228"
+    }
+  ],
+  "city": {
+    "-id": "cty-Cambodia-Phnom-Penh",
+    "-is_country_cap": "yes",
+    "-country": "K",
+    "name": "Phnom Penh",
+    "longitude": "104.85",
+    "latitude": "11.55",
+    "population": {
+      "-year": "87",
+      "#text": "400000"
+    },
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Mekong"
+    }
+  }
+}
+{
+  "-car_code": "VN",
+  "-area": "329560",
+  "-capital": "cty-Vietnam-Hanoi",
+  "-memberships": "org-ACCT org-AsDB org-ASEAN org-Mekong-Group org-CCC org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Vietnam",
+  "population": "73976973",
+  "population_growth": "1.57",
+  "infant_mortality": "38.4",
+  "gdp_total": "97000",
+  "gdp_agri": "28",
+  "gdp_ind": "28",
+  "gdp_serv": "44",
+  "inflation": "14",
+  "indep_date": "1945-09-02",
+  "government": "Communist state",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "85",
+      "#text": "Vietnamese"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "1281"
+    },
+    {
+      "-country": "LAO",
+      "-length": "2130"
+    },
+    {
+      "-country": "K",
+      "-length": "1228"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Vietnam-Hanoi",
+      "-is_country_cap": "yes",
+      "-country": "VN",
+      "name": "Hanoi",
+      "longitude": "105.867",
+      "latitude": "21.0333",
+      "population": {
+        "-year": "89",
+        "#text": "3056146"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-4",
+      "-country": "VN",
+      "name": "Haiphong",
+      "population": {
+        "-year": "89",
+        "#text": "1447523"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-6",
+      "-country": "VN",
+      "name": "Can Tho",
+      "population": {
+        "-year": "89",
+        "#text": "284306"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Mekong"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-2",
+      "-country": "VN",
+      "name": "Saigon",
+      "longitude": "106.5",
+      "latitude": "10.5",
+      "population": {
+        "-year": "89",
+        "#text": "3924435"
+      }
+    },
+    {
+      "-id": "cty-Vietnam-Da-Nang",
+      "-country": "VN",
+      "name": "Da Nang",
+      "longitude": "108.217",
+      "latitude": "16.0667",
+      "population": {
+        "-year": "89",
+        "#text": "369734"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-7",
+      "-country": "VN",
+      "name": "Nha Trang",
+      "population": {
+        "-year": "89",
+        "#text": "263093"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-8",
+      "-country": "VN",
+      "name": "Hue",
+      "population": {
+        "-year": "89",
+        "#text": "260489"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-9",
+      "-country": "VN",
+      "name": "Nam Dinh",
+      "population": {
+        "-year": "89",
+        "#text": "219615"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-10",
+      "-country": "VN",
+      "name": "Long Xuyen",
+      "population": {
+        "-year": "89",
+        "#text": "214037"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Mekong"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-11",
+      "-country": "VN",
+      "name": "Qui Nhon",
+      "population": {
+        "-year": "89",
+        "#text": "201912"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-12",
+      "-country": "VN",
+      "name": "Thai Nguyen",
+      "population": {
+        "-year": "89",
+        "#text": "171815"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-13",
+      "-country": "VN",
+      "name": "Vung Tau",
+      "population": {
+        "-year": "89",
+        "#text": "133558"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-14",
+      "-country": "VN",
+      "name": "Hong Gai",
+      "population": {
+        "-year": "89",
+        "#text": "129394"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-15",
+      "-country": "VN",
+      "name": "Cam Pha",
+      "population": {
+        "-year": "89",
+        "#text": "127408"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Vietnam-16",
+      "-country": "VN",
+      "name": "Viet Tri",
+      "population": {
+        "-year": "89",
+        "#text": "116084"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "KAZ",
+  "-area": "2717300",
+  "-capital": "cty-cid-cia-Kazakstan-2",
+  "-memberships": "org-AsDB org-CIS org-CCC org-ESCAP org-ECE org-ECO org-EBRD org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-ILO org-IMO org-IMF org-IOC org-ITU org-Intelsat org-IDB org-ANC org-OSCE org-OIC org-PFP org-UN org-UNESCO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Kazakstan",
+  "population": "16916463",
+  "population_growth": "-0.15",
+  "infant_mortality": "63.2",
+  "gdp_total": "46900",
+  "gdp_agri": "28.5",
+  "gdp_ind": "41.5",
+  "gdp_serv": "30",
+  "inflation": "60.3",
+  "indep_date": "1991-12-16",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4.7",
+      "#text": "German"
+    },
+    {
+      "-percentage": "5.2",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "37",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Tatar"
+    },
+    {
+      "-percentage": "2.1",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "41.9",
+      "#text": "Kazak"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "47",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "44",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "languages": {
+    "-percentage": "40",
+    "#text": "Kazak"
+  },
+  "border": [
+    {
+      "-country": "R",
+      "-length": "6846"
+    },
+    {
+      "-country": "TJ",
+      "-length": "1533"
+    },
+    {
+      "-country": "TM",
+      "-length": "379"
+    },
+    {
+      "-country": "UZB",
+      "-length": "2203"
+    },
+    {
+      "-country": "KGZ",
+      "-length": "1051"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Kazakstan-2",
+      "-capital": "cty-cid-cia-Kazakstan-2",
+      "-country": "KAZ",
+      "name": "Almaty",
+      "area": "105700",
+      "population": "963100"
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-3",
+      "-capital": "cty-cid-cia-Kazakstan-8",
+      "-country": "KAZ",
+      "name": "Aqmola",
+      "area": "92000",
+      "population": "845700",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-8",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-3",
+        "name": "Aqmola",
+        "population": {
+          "-year": "95",
+          "#text": "280200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-4",
+      "-capital": "cty-cid-cia-Kazakstan-9",
+      "-country": "KAZ",
+      "name": "Aqtobe",
+      "area": "300600",
+      "population": "752800",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-9",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-4",
+        "name": "Aqtobe",
+        "population": {
+          "-year": "95",
+          "#text": "258900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-5",
+      "-capital": "cty-cid-cia-Kazakstan-16",
+      "-country": "KAZ",
+      "name": "Atyrau",
+      "area": "118600",
+      "population": "459600",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-16",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-5",
+        "name": "Atyrau",
+        "population": {
+          "-year": "95",
+          "#text": "146900"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Ural"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-KaspischesMeer"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-6",
+      "-capital": "cty-cid-cia-Kazakstan-12",
+      "-country": "KAZ",
+      "name": "Batys Qazaqstan",
+      "area": "151300",
+      "population": "669800",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-12",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-6",
+        "name": "Oral",
+        "population": {
+          "-year": "95",
+          "#text": "219100"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ural"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-7",
+      "-capital": "cty-cid-cia-Kazakstan-17",
+      "-country": "KAZ",
+      "name": "Kokchetau",
+      "area": "78200",
+      "population": "657000",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-17",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-7",
+        "name": "Kokchetau",
+        "population": {
+          "-year": "95",
+          "#text": "141400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-8",
+      "-capital": "cty-cid-cia-Kazakstan-15",
+      "-country": "KAZ",
+      "name": "Mangghystau",
+      "area": "165600",
+      "population": "324400",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-15",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-8",
+        "name": "Aqtau",
+        "population": {
+          "-year": "95",
+          "#text": "151300"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-KaspischesMeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-9",
+      "-capital": "cty-cid-cia-Kazakstan-4",
+      "-country": "KAZ",
+      "name": "Ongtustik Qazaqstan",
+      "area": "117300",
+      "population": "1987800",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-4",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-9",
+        "name": "Shymkent",
+        "population": {
+          "-year": "95",
+          "#text": "397600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-10",
+      "-capital": "cty-cid-cia-Kazakstan-5",
+      "-country": "KAZ",
+      "name": "Pavlodar",
+      "area": "124800",
+      "population": "943600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kazakstan-5",
+          "-is_state_cap": "yes",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-10",
+          "name": "Pavlodar",
+          "population": {
+            "-year": "95",
+            "#text": "340700"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Irtysch"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kazakstan-18",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-10",
+          "name": "Ekibastuz",
+          "population": {
+            "-year": "95",
+            "#text": "141100"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-11",
+      "-capital": "cty-cid-cia-Kazakstan-3",
+      "-country": "KAZ",
+      "name": "Qaraghandy",
+      "area": "115400",
+      "population": "1270100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kazakstan-3",
+          "-is_state_cap": "yes",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-11",
+          "name": "Karaganda",
+          "population": {
+            "-year": "95",
+            "#text": "573700"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kazakstan-13",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-11",
+          "name": "Temirtau",
+          "population": {
+            "-year": "95",
+            "#text": "206100"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-12",
+      "-capital": "cty-cid-cia-Kazakstan-11",
+      "-country": "KAZ",
+      "name": "Qostanay",
+      "area": "113900",
+      "population": "1055300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kazakstan-11",
+          "-is_state_cap": "yes",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-12",
+          "name": "Qostanay",
+          "population": {
+            "-year": "95",
+            "#text": "232100"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Tobol"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kazakstan-20",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-12",
+          "name": "Rudny",
+          "population": {
+            "-year": "95",
+            "#text": "125700"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-13",
+      "-capital": "cty-cid-cia-Kazakstan-14",
+      "-country": "KAZ",
+      "name": "Qyzylorda",
+      "area": "226000",
+      "population": "606100",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-14",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-13",
+        "name": "Qyzylorda",
+        "population": {
+          "-year": "95",
+          "#text": "162000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Syrdarja"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-14",
+      "-capital": "cty-cid-cia-Kazakstan-7",
+      "-country": "KAZ",
+      "name": "Semey",
+      "area": "185800",
+      "population": "811000",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-7",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-14",
+        "name": "Semey",
+        "population": {
+          "-year": "95",
+          "#text": "320200"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Irtysch"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-15",
+      "-capital": "cty-cid-cia-Kazakstan-6",
+      "-country": "KAZ",
+      "name": "Shyghys Qazaqstan",
+      "area": "97500",
+      "population": "939500",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-6",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-15",
+        "name": "Oskemen",
+        "population": {
+          "-year": "95",
+          "#text": "326300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-16",
+      "-capital": "cty-cid-cia-Kazakstan-10",
+      "-country": "KAZ",
+      "name": "Soltustik Qazaqstan",
+      "area": "45000",
+      "population": "600900",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-10",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-16",
+        "name": "Petropavl",
+        "population": {
+          "-year": "95",
+          "#text": "239000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Ischim"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-17",
+      "-capital": "cty-cid-cia-Kazakstan-21",
+      "-country": "KAZ",
+      "name": "Taldyqorghan",
+      "area": "118500",
+      "population": "721500",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-21",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-17",
+        "name": "Taldyqorghan",
+        "population": {
+          "-year": "95",
+          "#text": "116100"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-18",
+      "-capital": "cty-cid-cia-Kazakstan-Arqalyq",
+      "-country": "KAZ",
+      "name": "Torghay",
+      "area": "111800",
+      "population": "305900",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-Arqalyq",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-18",
+        "name": "Arqalyq"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-19",
+      "-capital": "cty-cid-cia-Kazakstan-Zhambyl",
+      "-country": "KAZ",
+      "name": "Zhambyl",
+      "area": "144300",
+      "population": "1039600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kazakstan-19",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-19",
+          "name": "Dzhambul",
+          "population": {
+            "-year": "95",
+            "#text": "130600"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kazakstan-Zhambyl",
+          "-is_state_cap": "yes",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-19",
+          "name": "Zhambyl"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-20",
+      "-capital": "cty-cid-cia-Kazakstan-Zhezkazghan",
+      "-country": "KAZ",
+      "name": "Zhezkazghan",
+      "area": "312600",
+      "population": "484400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kazakstan-22",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-20",
+          "name": "Zhezqazghan",
+          "population": {
+            "-year": "95",
+            "#text": "108700"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kazakstan-Zhezkazghan",
+          "-is_state_cap": "yes",
+          "-country": "KAZ",
+          "-province": "prov-cid-cia-Kazakstan-20",
+          "name": "Zhezkazghan"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-22",
+      "-capital": "cty-cid-cia-Kazakstan-2",
+      "-country": "KAZ",
+      "name": "Almaty (munic.)",
+      "area": "0",
+      "population": "1172400",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-22",
+        "name": "Almaty",
+        "longitude": "76.8833",
+        "latitude": "43.2333",
+        "population": {
+          "-year": "95",
+          "#text": "1172400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kazakstan-23",
+      "-capital": "cty-cid-cia-Kazakstan-Leninsk",
+      "-country": "KAZ",
+      "name": "Leninsk (munic.)",
+      "area": "0",
+      "population": "68600",
+      "city": {
+        "-id": "cty-cid-cia-Kazakstan-Leninsk",
+        "-is_state_cap": "yes",
+        "-country": "KAZ",
+        "-province": "prov-cid-cia-Kazakstan-23",
+        "name": "Leninsk"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "NOK",
+  "-area": "120540",
+  "-capital": "cty-cid-cia-North-Korea-2",
+  "-memberships": "org-ESCAP org-FAO org-G-77 org-ICAO org-IFRCS org-IFAD org-IMO org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "North Korea",
+  "population": "23904124",
+  "population_growth": "1.74",
+  "infant_mortality": "25.9",
+  "gdp_total": "21500",
+  "gdp_agri": "25",
+  "gdp_ind": "60",
+  "gdp_serv": "15",
+  "government": "Communist state",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Korean"
+  },
+  "border": [
+    {
+      "-country": "R",
+      "-length": "19"
+    },
+    {
+      "-country": "TJ",
+      "-length": "1416"
+    },
+    {
+      "-country": "ROK",
+      "-length": "238"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-North-Korea-2",
+      "-is_country_cap": "yes",
+      "-country": "NOK",
+      "name": "Pyongyang",
+      "longitude": "125.683",
+      "latitude": "39.0333",
+      "population": {
+        "-year": "87",
+        "#text": "2335000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-3",
+      "-country": "NOK",
+      "name": "Hamhung Hungnam",
+      "population": {
+        "-year": "87",
+        "#text": "701000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-4",
+      "-country": "NOK",
+      "name": "Chongjin",
+      "population": {
+        "-year": "87",
+        "#text": "520000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-5",
+      "-country": "NOK",
+      "name": "Nampo",
+      "population": {
+        "-year": "87",
+        "#text": "370000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-6",
+      "-country": "NOK",
+      "name": "Sunchon",
+      "population": {
+        "-year": "87",
+        "#text": "356000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-7",
+      "-country": "NOK",
+      "name": "Sinuiju",
+      "population": {
+        "-year": "87",
+        "#text": "289000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-8",
+      "-country": "NOK",
+      "name": "Tanchon",
+      "population": {
+        "-year": "87",
+        "#text": "284000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-9",
+      "-country": "NOK",
+      "name": "Wonsan",
+      "population": {
+        "-year": "87",
+        "#text": "274000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-10",
+      "-country": "NOK",
+      "name": "Tokchon",
+      "population": {
+        "-year": "87",
+        "#text": "217000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-11",
+      "-country": "NOK",
+      "name": "Kanggye",
+      "population": {
+        "-year": "87",
+        "#text": "211000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-12",
+      "-country": "NOK",
+      "name": "Haeju",
+      "population": {
+        "-year": "87",
+        "#text": "195000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-13",
+      "-country": "NOK",
+      "name": "Anju",
+      "population": {
+        "-year": "87",
+        "#text": "186000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-14",
+      "-country": "NOK",
+      "name": "Songjin",
+      "population": {
+        "-year": "87",
+        "#text": "179000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-15",
+      "-country": "NOK",
+      "name": "Kusong",
+      "population": {
+        "-year": "87",
+        "#text": "177000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-16",
+      "-country": "NOK",
+      "name": "Huichon",
+      "population": {
+        "-year": "87",
+        "#text": "163000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-17",
+      "-country": "NOK",
+      "name": "Sinpo",
+      "population": {
+        "-year": "87",
+        "#text": "158000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-North-Korea-18",
+      "-country": "NOK",
+      "name": "Kaesong",
+      "population": {
+        "-year": "87",
+        "#text": "120000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "KGZ",
+  "-area": "198500",
+  "-capital": "cty-cid-cia-Kyrgyzstan-Bishkek",
+  "-memberships": "org-AsDB org-CIS org-ESCAP org-ECE org-ECO org-EBRD org-FAO org-IBRD org-ICAO org-IDA org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ITU org-Intelsat org-IDB org-ANC org-OSCE org-OIC org-PFP org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Kyrgyzstan",
+  "population": "4529648",
+  "population_growth": "0.07",
+  "infant_mortality": "77.8",
+  "gdp_total": "5400",
+  "inflation": "32",
+  "indep_date": "1991-08-31",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2.4",
+      "#text": "German"
+    },
+    {
+      "-percentage": "2.5",
+      "#text": "Ukrainian"
+    },
+    {
+      "-percentage": "21.5",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "12.9",
+      "#text": "Uzbek"
+    },
+    {
+      "-percentage": "52.4",
+      "#text": "Kirghiz"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "858"
+    },
+    {
+      "-country": "TAD",
+      "-length": "870"
+    },
+    {
+      "-country": "UZB",
+      "-length": "1099"
+    },
+    {
+      "-country": "KAZ",
+      "-length": "1051"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Kyrgyzstan-Bishkek",
+    "-is_country_cap": "yes",
+    "-country": "KGZ",
+    "name": "Bishkek",
+    "longitude": "74.5",
+    "latitude": "42.4",
+    "population": {
+      "-year": "87",
+      "#text": "630000"
+    }
+  }
+}
+{
+  "-car_code": "HONX",
+  "-area": "1092",
+  "-capital": "cty-Hongkong",
+  "-memberships": "org-APEC org-BIS org-ICC org-ISO org-IMF org-IMO org-IOC org-UPU org-WCL org-WFTU org-WMO",
+  "name": "Hong Kong",
+  "population": "7055071",
+  "population_growth": "0.51",
+  "infant_mortality": "2.9",
+  "gdp_total": "307600",
+  "gdp_agri": "0",
+  "gdp_ind": "7.9",
+  "gdp_serv": "92",
+  "inflation": "2.1",
+  "dependent": { "-country": "TJ" },
+  "government": "special administrative area in China",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "95",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "1.6",
+      "#text": "Filipino"
+    },
+    {
+      "-percentage": "1.3",
+      "#text": "Indonesian"
+    }
+  ],
+  "religions": {
+    "-percentage": "10",
+    "#text": "Christian"
+  },
+  "languages": [
+    {
+      "-percentage": "95.6",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "3.2",
+      "#text": "English"
+    }
+  ],
+  "border": {
+    "-country": "TJ",
+    "-length": "30"
+  },
+  "city": {
+    "-id": "cty-Hongkong",
+    "-is_country_cap": "yes",
+    "-country": "HONX",
+    "name": "Hong Kong",
+    "longitude": "114.2",
+    "latitude": "22.3",
+    "population": {
+      "-year": "09",
+      "#text": "7055071"
+    }
+  }
+}
+{
+  "-car_code": "MACX",
+  "-area": "16",
+  "-capital": "cty-cid-cia-Macau-Macau",
+  "-memberships": "org-IMF org-IMO org-ISO org-UNESCO org-UPU org-WFTU org-WMO",
+  "name": "Macau",
+  "population": "496837",
+  "population_growth": "1.15",
+  "infant_mortality": "5.3",
+  "gdp_total": "220400",
+  "gdp_agri": "0.1",
+  "gdp_ind": "2.8",
+  "gdp_serv": "97.1",
+  "inflation": "6.2",
+  "dependent": { "-country": "TJ" },
+  "government": "special administrative area in China",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "95",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Portuguese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "7",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "45",
+      "#text": "Buddhist"
+    }
+  ],
+  "languages": {
+    "-percentage": "4",
+    "#text": "Portuguese"
+  },
+  "border": {
+    "-country": "TJ",
+    "-length": "0.34"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Macau-Macau",
+    "-is_country_cap": "yes",
+    "-country": "MACX",
+    "name": "Macau",
+    "longitude": "113.55",
+    "latitude": "22.2"
+  }
+}
+{
+  "-car_code": "MNG",
+  "-area": "1565000",
+  "-capital": "cty-cid-cia-Mongolia-Ulaanbaatar",
+  "-memberships": "org-AsDB org-CCC org-ESCAP org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Mongolia",
+  "population": "2496617",
+  "population_growth": "1.69",
+  "infant_mortality": "69.7",
+  "gdp_total": "4900",
+  "gdp_agri": "28",
+  "gdp_ind": "35",
+  "gdp_serv": "37",
+  "inflation": "53",
+  "indep_date": "1921-03-13",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Russian"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Kazak"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Mongol"
+    }
+  ],
+  "religions": {
+    "-percentage": "4",
+    "#text": "Muslim"
+  },
+  "languages": {
+    "-percentage": "90",
+    "#text": "Khalkha Mongol"
+  },
+  "border": [
+    {
+      "-country": "R",
+      "-length": "3441"
+    },
+    {
+      "-country": "TJ",
+      "-length": "4673"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Mongolia-Ulaanbaatar",
+    "-is_country_cap": "yes",
+    "-country": "MNG",
+    "name": "Ulaanbaatar",
+    "longitude": "107",
+    "latitude": "48",
+    "population": {
+      "-year": "87",
+      "#text": "479500"
+    }
+  }
+}
+{
+  "-car_code": "NEP",
+  "-area": "140800",
+  "-capital": "cty-cid-cia-Nepal-Kathmandu",
+  "-memberships": "org-AsDB org-CP org-CCC org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-SAARC org-UN org-UNCRO org-UNESCO org-UNIDO org-UNIFIL org-UNMIH org-UNPROFOR org-UPU org-WFTU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Nepal",
+  "population": "22094033",
+  "population_growth": "2.45",
+  "infant_mortality": "79",
+  "gdp_total": "25200",
+  "gdp_agri": "49.3",
+  "gdp_ind": "18.4",
+  "gdp_serv": "32.3",
+  "inflation": "6.7",
+  "indep_date": "1768-01-01",
+  "government": "parliamentary democracy as of 12 May 1991",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "3",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Hindu"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Nepali"
+  },
+  "border": [
+    {
+      "-country": "TJ",
+      "-length": "1236"
+    },
+    {
+      "-country": "IND",
+      "-length": "1690"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Nepal-Kathmandu",
+    "-is_country_cap": "yes",
+    "-country": "NEP",
+    "name": "Kathmandu",
+    "longitude": "85.25",
+    "latitude": "27.45",
+    "population": {
+      "-year": "87",
+      "#text": "393494"
+    }
+  }
+}
+{
+  "-car_code": "XMAS",
+  "-area": "135",
+  "-capital": "city-The-Settlement-XMAS-XMAS",
+  "name": "Christmas Island",
+  "population": "1402",
+  "dependent": { "-country": "AUS" },
+  "government": "territory of Australia",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "70",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Malay"
+    },
+    {
+      "-percentage": "20",
+      "#text": "European"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "25",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "36",
+      "#text": "Buddhist"
+    }
+  ],
+  "city": {
+    "-id": "city-The-Settlement-XMAS-XMAS",
+    "-is_country_cap": "yes",
+    "-country": "XMAS",
+    "name": "Flying Fish Cove",
+    "longitude": "105.43",
+    "latitude": "-10.25",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-ChristmasIsland" }
+  }
+}
+{
+  "-car_code": "COCO",
+  "-area": "14",
+  "-capital": "city-West-Island-COCO-COCO",
+  "-memberships": "org-WMO",
+  "name": "Cocos Islands",
+  "population": "596",
+  "dependent": { "-country": "AUS" },
+  "government": "territory of Australia",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "80",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Christian"
+    }
+  ],
+  "city": {
+    "-id": "city-West-Island-COCO-COCO",
+    "-is_country_cap": "yes",
+    "-country": "COCO",
+    "name": "West Island",
+    "longitude": "96.50",
+    "latitude": "-12.10",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-PulauPanjang" }
+  }
+}
+{
+  "-car_code": "CY",
+  "-area": "9251",
+  "-capital": "cty-cid-cia-Cyprus-Nicosia",
+  "-memberships": "org-C org-CE org-CCC org-ECE org-EBRD org-EU org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ITU org-Intelsat org-NAM org-OSCE org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Cyprus",
+  "population": "744609",
+  "population_growth": "1.11",
+  "infant_mortality": "8.4",
+  "gdp_total": "7800",
+  "gdp_agri": "5.6",
+  "gdp_ind": "24.9",
+  "gdp_serv": "69.5",
+  "indep_date": "1960-08-16",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "europe",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "18",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "78",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Cyprus-Nicosia",
+    "-is_country_cap": "yes",
+    "-country": "CY",
+    "name": "Nicosia",
+    "longitude": "33.2",
+    "latitude": "35.4",
+    "population": {
+      "-year": "87",
+      "#text": "161100"
+    },
+    "located_on": { "-island": "island-Cyprus" }
+  }
+}
+{
+  "-car_code": "GAZA",
+  "-area": "360",
+  "name": "Gaza Strip",
+  "population": "813332",
+  "population_growth": "4.55",
+  "infant_mortality": "30.6",
+  "gdp_total": "1700",
+  "inflation": "5.7",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.6",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "99.4",
+      "#text": "Palestinian Arab"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "98.7",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "0.6",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "0.7",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "IL",
+      "-length": "51"
+    },
+    {
+      "-country": "ET",
+      "-length": "11"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-GAZA-Gaza",
+    "-country": "GAZA",
+    "name": "Gaza",
+    "longitude": "34.5",
+    "latitude": "31.5",
+    "population": {
+      "-year": "04",
+      "#text": "409680"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Mittelmeer"
+    }
+  }
+}
+{
+  "-car_code": "IL",
+  "-area": "20770",
+  "-capital": "cty-Israel-Jerusalem",
+  "-memberships": "org-AG org-BSEC org-CE org-CCC org-ECE org-EBRD org-CERN org-FAO org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ITU org-Intelsat org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Israel",
+  "population": "5421995",
+  "population_growth": "2.11",
+  "infant_mortality": "8.5",
+  "gdp_total": "80100",
+  "gdp_agri": "3.5",
+  "gdp_ind": "22",
+  "gdp_serv": "74.5",
+  "inflation": "10.1",
+  "indep_date": "1948-05-14",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "82",
+    "#text": "Jewish"
+  },
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "14",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Druze"
+    },
+    {
+      "-percentage": "82",
+      "#text": "Judaism"
+    }
+  ],
+  "border": [
+    {
+      "-country": "GAZA",
+      "-length": "51"
+    },
+    {
+      "-country": "ET",
+      "-length": "255"
+    },
+    {
+      "-country": "JOR",
+      "-length": "238"
+    },
+    {
+      "-country": "SYR",
+      "-length": "76"
+    },
+    {
+      "-country": "RL",
+      "-length": "79"
+    },
+    {
+      "-country": "WEST",
+      "-length": "307"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Israel-2",
+      "-capital": "cty-cid-cia-Israel-Ramla",
+      "-country": "IL",
+      "name": "Central",
+      "area": "1242",
+      "population": "938000",
+      "city": [
+        {
+          "-id": "cty-Israel-Jerusalem",
+          "-is_country_cap": "yes",
+          "-country": "IL",
+          "-province": "prov-cid-cia-Israel-2",
+          "name": "Jerusalem",
+          "longitude": "35.2167",
+          "latitude": "31.7833",
+          "population": {
+            "-year": "87",
+            "#text": "468900"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Israel-Ramla",
+          "-is_state_cap": "yes",
+          "-country": "IL",
+          "-province": "prov-cid-cia-Israel-2",
+          "name": "Ramla"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Israel-3",
+      "-capital": "cty-cid-cia-Israel-Haifa",
+      "-country": "IL",
+      "name": "Haifa",
+      "area": "854",
+      "population": "602800",
+      "city": {
+        "-id": "cty-cid-cia-Israel-Haifa",
+        "-is_state_cap": "yes",
+        "-country": "IL",
+        "-province": "prov-cid-cia-Israel-3",
+        "name": "Haifa",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Israel-5",
+      "-capital": "cty-cid-cia-Israel-Nazareth",
+      "-country": "IL",
+      "name": "North",
+      "area": "4501",
+      "population": "739500",
+      "city": {
+        "-id": "cty-cid-cia-Israel-Nazareth",
+        "-is_state_cap": "yes",
+        "-country": "IL",
+        "-province": "prov-cid-cia-Israel-5",
+        "name": "Nazareth"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Israel-6",
+      "-capital": "cty-Israel-Tel-Aviv",
+      "-country": "IL",
+      "name": "Tel Aviv",
+      "area": "170",
+      "population": "1029700",
+      "city": {
+        "-id": "cty-Israel-Tel-Aviv",
+        "-is_state_cap": "yes",
+        "-country": "IL",
+        "-province": "prov-cid-cia-Israel-6",
+        "name": "Tel Aviv",
+        "longitude": "34.7833",
+        "latitude": "32.1",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Israel-7",
+      "-capital": "cty-cid-cia-Israel-Beer-Sheva",
+      "-country": "IL",
+      "name": "South",
+      "area": "14107",
+      "population": "529300",
+      "city": {
+        "-id": "cty-cid-cia-Israel-Beer-Sheva",
+        "-is_state_cap": "yes",
+        "-country": "IL",
+        "-province": "prov-cid-cia-Israel-7",
+        "name": "Beer Sheva"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "ET",
+  "-area": "1001450",
+  "-capital": "cty-Egypt-Cairo",
+  "-memberships": "org-AfDB org-ACCT org-AG org-ABEDA org-ACC org-AFESD org-AL org-AMF org-BSEC org-CAEU org-CCC org-ESCWA org-ECA org-EBRD org-FAO org-G-15 org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OAS org-OAPEC org-OIC org-PCA org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-MINURSO org-UNOMIG org-UNOMIL org-UNPREDEP org-UNPROFOR org-UNRWA org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Egypt",
+  "population": "63575107",
+  "population_growth": "1.91",
+  "infant_mortality": "72.8",
+  "gdp_total": "171000",
+  "inflation": "9.4",
+  "indep_date": "1922-02-28",
+  "government": "republic",
+  "encompassed": [
+    {
+      "-continent": "asia",
+      "-percentage": "10"
+    },
+    {
+      "-continent": "africa",
+      "-percentage": "90"
+    }
+  ],
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "99",
+      "#text": "Eastern Hamitic"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "94",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Coptic Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "GAZA",
+      "-length": "11"
+    },
+    {
+      "-country": "IL",
+      "-length": "255"
+    },
+    {
+      "-country": "LAR",
+      "-length": "1150"
+    },
+    {
+      "-country": "SUD",
+      "-length": "1273"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Egypt-3",
+      "-capital": "cty-cid-cia-Egypt-Hurghada",
+      "-country": "ET",
+      "name": "El Bahr el Ahmar",
+      "area": "203685",
+      "population": "108000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-Hurghada",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-3",
+        "name": "Hurghada"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-4",
+      "-capital": "cty-cid-cia-Egypt-Marsa-Matruh",
+      "-country": "ET",
+      "name": "Matruh",
+      "area": "212112",
+      "population": "182000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-Marsa-Matruh",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-4",
+        "name": "Marsa Matruh",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-5",
+      "-capital": "cty-cid-cia-Egypt-El-Tur",
+      "-country": "ET",
+      "name": "Sina al Janubiyah",
+      "area": "33140",
+      "population": "33000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-El-Tur",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-5",
+        "name": "El Tur",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Rotes_Meer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-6",
+      "-capital": "cty-cid-cia-Egypt-El-Arish",
+      "-country": "ET",
+      "name": "Sina ash Shamaliyah",
+      "area": "27574",
+      "population": "196000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-El-Arish",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-6",
+        "name": "El Arish",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-7",
+      "-capital": "cty-cid-cia-Egypt-El-Kharga",
+      "-country": "ET",
+      "name": "El Wadi el Jadid",
+      "area": "376505",
+      "population": "126000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-El-Kharga",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-7",
+        "name": "El Kharga"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-9",
+      "-capital": "cty-cid-cia-Egypt-16",
+      "-country": "ET",
+      "name": "El Buhayra",
+      "area": "10130",
+      "population": "3602000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Egypt-16",
+          "-is_state_cap": "yes",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-9",
+          "name": "Damanhur",
+          "population": {
+            "-year": "86",
+            "#text": "226000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Egypt-20",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-9",
+          "name": "Kafr el Dauwar",
+          "population": {
+            "-year": "86",
+            "#text": "161000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-10",
+      "-capital": "cty-cid-cia-Egypt-9",
+      "-country": "ET",
+      "name": "El Daqahliya",
+      "area": "3471",
+      "population": "3828000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-9",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-10",
+        "name": "El Mansura",
+        "population": {
+          "-year": "86",
+          "#text": "358000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-11",
+      "-capital": "cty-cid-cia-Egypt-25",
+      "-country": "ET",
+      "name": "Dumyat",
+      "area": "589",
+      "population": "808000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-25",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-11",
+        "name": "Dumyat",
+        "population": {
+          "-year": "86",
+          "#text": "121000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-12",
+      "-capital": "cty-cid-cia-Egypt-8",
+      "-country": "ET",
+      "name": "El Gharbiya",
+      "area": "1942",
+      "population": "3113000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Egypt-8",
+          "-is_state_cap": "yes",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-12",
+          "name": "Tanta",
+          "population": {
+            "-year": "86",
+            "#text": "374000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Egypt-6",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-12",
+          "name": "El Mahalla el Kubra",
+          "population": {
+            "-year": "86",
+            "#text": "385000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-13",
+      "-capital": "cty-cid-cia-Egypt-14",
+      "-country": "ET",
+      "name": "Ismailiya",
+      "area": "1442",
+      "population": "623000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-14",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-13",
+        "name": "Ismailiya",
+        "population": {
+          "-year": "86",
+          "#text": "236000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-14",
+      "-capital": "cty-cid-cia-Egypt-Kafr-el-Sheikh",
+      "-country": "ET",
+      "name": "Kafr el Sheikh",
+      "area": "3437",
+      "population": "1968000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-Kafr-el-Sheikh",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-14",
+        "name": "Kafr el Sheikh"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-15",
+      "-capital": "cty-cid-cia-Egypt-24",
+      "-country": "ET",
+      "name": "El Minufiya",
+      "area": "1532",
+      "population": "2449000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-24",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-15",
+        "name": "Shibin el Kom",
+        "population": {
+          "-year": "86",
+          "#text": "136000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-16",
+      "-capital": "cty-cid-cia-Egypt-26",
+      "-country": "ET",
+      "name": "El Qalubiya",
+      "area": "1001",
+      "population": "2868000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Egypt-26",
+          "-is_state_cap": "yes",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-16",
+          "name": "Benha",
+          "population": {
+            "-year": "86",
+            "#text": "120000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Egypt-5",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-16",
+          "name": "Shubra el Kheima",
+          "population": {
+            "-year": "86",
+            "#text": "533000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-17",
+      "-capital": "cty-cid-cia-Egypt-12",
+      "-country": "ET",
+      "name": "Sharqiya",
+      "area": "4180",
+      "population": "3766000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-12",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-17",
+        "name": "Zagazig",
+        "population": {
+          "-year": "86",
+          "#text": "274000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-19",
+      "-capital": "cty-cid-cia-Egypt-18",
+      "-country": "ET",
+      "name": "Aswan",
+      "area": "679",
+      "population": "883000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-18",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-19",
+        "name": "Aswan",
+        "population": {
+          "-year": "86",
+          "#text": "196000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-20",
+      "-capital": "cty-cid-cia-Egypt-11",
+      "-country": "ET",
+      "name": "Asyut",
+      "area": "1553",
+      "population": "2456000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-11",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-20",
+        "name": "Asyut",
+        "population": {
+          "-year": "86",
+          "#text": "291000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-21",
+      "-capital": "cty-cid-cia-Egypt-19",
+      "-country": "ET",
+      "name": "Beni Suef",
+      "area": "1322",
+      "population": "1586000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-19",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-21",
+        "name": "Beni Suef",
+        "population": {
+          "-year": "86",
+          "#text": "163000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-22",
+      "-capital": "cty-cid-cia-Egypt-15",
+      "-country": "ET",
+      "name": "El Faiyum",
+      "area": "1827",
+      "population": "1720000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-15",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-22",
+        "name": "El Faiyum",
+        "population": {
+          "-year": "86",
+          "#text": "227000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-23",
+      "-capital": "cty-cid-cia-Egypt-4",
+      "-country": "ET",
+      "name": "El Giza",
+      "area": "85153",
+      "population": "4265000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-4",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-23",
+        "name": "El Giza",
+        "population": {
+          "-year": "86",
+          "#text": "1671000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-24",
+      "-capital": "cty-cid-cia-Egypt-17",
+      "-country": "ET",
+      "name": "El Minya",
+      "area": "2262",
+      "population": "2916000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-17",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-24",
+        "name": "El Minya",
+        "population": {
+          "-year": "86",
+          "#text": "203000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-25",
+      "-capital": "cty-cid-cia-Egypt-23",
+      "-country": "ET",
+      "name": "Qena",
+      "area": "1851",
+      "population": "2493000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Egypt-23",
+          "-is_state_cap": "yes",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-25",
+          "name": "Qena",
+          "population": {
+            "-year": "86",
+            "#text": "142000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Nil"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Egypt-21",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-25",
+          "name": "El Uqsur",
+          "population": {
+            "-year": "86",
+            "#text": "148000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Nil"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-26",
+      "-capital": "cty-cid-cia-Egypt-22",
+      "-country": "ET",
+      "name": "Sohag",
+      "area": "1547",
+      "population": "2689000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-22",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-26",
+        "name": "Sohag",
+        "population": {
+          "-year": "86",
+          "#text": "142000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-28",
+      "-capital": "cty-cid-cia-Egypt-Bur-Said",
+      "-country": "ET",
+      "name": "Bur Said (munic.)",
+      "area": "72",
+      "population": "461000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-Bur-Said",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-28",
+        "name": "Bur Said",
+        "population": {
+          "-year": "86",
+          "#text": "382000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-29",
+      "-capital": "cty-cid-cia-Egypt-El-Iskandariya",
+      "-country": "ET",
+      "name": "El Iskandariya",
+      "area": "2679",
+      "population": "3170000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-El-Iskandariya",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-29",
+        "name": "Alexandria",
+        "longitude": "30",
+        "latitude": "31",
+        "population": {
+          "-year": "86",
+          "#text": "2917000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Mittelmeer"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-30",
+      "-capital": "cty-Egypt-Cairo",
+      "-country": "ET",
+      "name": "El Qahira",
+      "area": "214",
+      "population": "6452000",
+      "city": [
+        {
+          "-id": "cty-Egypt-Cairo",
+          "-is_country_cap": "yes",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-30",
+          "name": "Cairo",
+          "longitude": "31.3333",
+          "latitude": "29.8667",
+          "population": {
+            "-year": "86",
+            "#text": "6053000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Nil"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Egypt-10",
+          "-country": "ET",
+          "-province": "prov-cid-cia-Egypt-30",
+          "name": "Helwan",
+          "population": {
+            "-year": "86",
+            "#text": "352000"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Nil"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Egypt-31",
+      "-capital": "cty-cid-cia-Egypt-El-Suweiz",
+      "-country": "ET",
+      "name": "El Suweiz",
+      "area": "17840",
+      "population": "392000",
+      "city": {
+        "-id": "cty-cid-cia-Egypt-El-Suweiz",
+        "-is_state_cap": "yes",
+        "-country": "ET",
+        "-province": "prov-cid-cia-Egypt-31",
+        "name": "El Suweiz",
+        "population": {
+          "-year": "86",
+          "#text": "265000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Rotes_Meer"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RI",
+  "-area": "1919440",
+  "-capital": "cty-cid-cia-Indonesia-2",
+  "-memberships": "org-AsDB org-ASEAN org-Mekong-Group org-CP org-CCC org-ESCAP org-FAO org-G-15 org-G-19 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OPEC org-OIC org-UN org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-UNOMIG org-UNPREDEP org-UNPROFOR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Indonesia",
+  "population": "206611600",
+  "population_growth": "1.53",
+  "infant_mortality": "63.1",
+  "gdp_total": "710900",
+  "gdp_agri": "17",
+  "gdp_ind": "32.6",
+  "gdp_serv": "50.4",
+  "inflation": "8.6",
+  "indep_date": "1945-08-17",
+  "government": "republic",
+  "encompassed": [
+    {
+      "-continent": "asia",
+      "-percentage": "80"
+    },
+    {
+      "-continent": "australia",
+      "-percentage": "20"
+    }
+  ],
+  "ethnicgroups": [
+    {
+      "-percentage": "45",
+      "#text": "Javanese"
+    },
+    {
+      "-percentage": "14",
+      "#text": "Sundanese"
+    },
+    {
+      "-percentage": "7.5",
+      "#text": "Madurese"
+    },
+    {
+      "-percentage": "7.5",
+      "#text": "Malay"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "87",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "MAL",
+      "-length": "1782"
+    },
+    {
+      "-country": "PNG",
+      "-length": "820"
+    },
+    {
+      "-country": "TL",
+      "-length": "228"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Indonesia-2",
+      "-is_country_cap": "yes",
+      "-country": "RI",
+      "name": "Jakarta",
+      "longitude": "106.5",
+      "latitude": "-6.1",
+      "population": {
+        "-year": "90",
+        "#text": "8259266"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-4",
+      "-country": "RI",
+      "name": "Bandung",
+      "longitude": "107.4",
+      "latitude": "-6.58",
+      "population": {
+        "-year": "90",
+        "#text": "2058649"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-6",
+      "-country": "RI",
+      "name": "Semarang",
+      "population": {
+        "-year": "90",
+        "#text": "1250971"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-Indonesia-Surabaya",
+      "-country": "RI",
+      "name": "Surabaya",
+      "longitude": "112.717",
+      "latitude": "-7.21667",
+      "population": {
+        "-year": "90",
+        "#text": "2483871"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-14",
+      "-country": "RI",
+      "name": "Yogyakarta",
+      "population": {
+        "-year": "90",
+        "#text": "412392"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-28",
+      "-country": "RI",
+      "name": "Banda Aceh",
+      "population": {
+        "-year": "90",
+        "#text": "184699"
+      },
+      "located_at": [
+        {
+          "-watertype": "sea",
+          "-sea": "sea-Indic"
+        },
+        {
+          "-watertype": "sea",
+          "-sea": "sea-AndamanSea"
+        }
+      ],
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-31",
+      "-country": "RI",
+      "name": "Bengkulu",
+      "population": {
+        "-year": "90",
+        "#text": "170327"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Indic"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-19",
+      "-country": "RI",
+      "name": "Jambi",
+      "population": {
+        "-year": "90",
+        "#text": "339944"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-10",
+      "-country": "RI",
+      "name": "Bandar Lampung",
+      "population": {
+        "-year": "90",
+        "#text": "636706"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Indic"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-11",
+      "-country": "RI",
+      "name": "Padang",
+      "population": {
+        "-year": "90",
+        "#text": "631543"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Indic"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-Indonesia-Palembang",
+      "-country": "RI",
+      "name": "Palembang",
+      "longitude": "104.767",
+      "latitude": "-3",
+      "population": {
+        "-year": "90",
+        "#text": "1144279"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-Indonesia-Medan",
+      "-country": "RI",
+      "name": "Medan",
+      "longitude": "98.6833",
+      "latitude": "3.58333",
+      "population": {
+        "-year": "90",
+        "#text": "1730752"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-MalakkaStrait"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-17",
+      "-country": "RI",
+      "name": "Pontianak",
+      "population": {
+        "-year": "90",
+        "#text": "397343"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Borneo" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-13",
+      "-country": "RI",
+      "name": "Banjarmasin",
+      "population": {
+        "-year": "90",
+        "#text": "481371"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Borneo" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-40",
+      "-country": "RI",
+      "name": "Palangkaraya",
+      "population": {
+        "-year": "90",
+        "#text": "112562"
+      },
+      "located_on": { "-island": "island-Borneo" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-15",
+      "-country": "RI",
+      "name": "Samarinda",
+      "population": {
+        "-year": "90",
+        "#text": "407339"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Borneo" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-8",
+      "-country": "RI",
+      "name": "Ujung Pandang",
+      "population": {
+        "-year": "90",
+        "#text": "944685"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Sulawesi" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-21",
+      "-country": "RI",
+      "name": "Ambon",
+      "population": {
+        "-year": "90",
+        "#text": "276955"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Ambon" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-9",
+      "-country": "RI",
+      "name": "Malang",
+      "population": {
+        "-year": "90",
+        "#text": "695618"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-12",
+      "-country": "RI",
+      "name": "Surakarta",
+      "population": {
+        "-year": "90",
+        "#text": "504176"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-16",
+      "-country": "RI",
+      "name": "Pekan Baru",
+      "population": {
+        "-year": "90",
+        "#text": "398694"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-18",
+      "-country": "RI",
+      "name": "Balikpapan",
+      "population": {
+        "-year": "90",
+        "#text": "344405"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Borneo" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-20",
+      "-country": "RI",
+      "name": "Manado",
+      "population": {
+        "-year": "90",
+        "#text": "320990"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Celebessee"
+      },
+      "located_on": { "-island": "island-Sulawesi" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-22",
+      "-country": "RI",
+      "name": "Bogor",
+      "population": {
+        "-year": "90",
+        "#text": "271711"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-23",
+      "-country": "RI",
+      "name": "Cirebon",
+      "population": {
+        "-year": "90",
+        "#text": "254878"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-24",
+      "-country": "RI",
+      "name": "Kediri",
+      "population": {
+        "-year": "90",
+        "#text": "249807"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-25",
+      "-country": "RI",
+      "name": "Pekalongan",
+      "population": {
+        "-year": "90",
+        "#text": "242874"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-26",
+      "-country": "RI",
+      "name": "Tegal",
+      "population": {
+        "-year": "90",
+        "#text": "229713"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-27",
+      "-country": "RI",
+      "name": "Pematang Siantar",
+      "population": {
+        "-year": "90",
+        "#text": "219328"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Asahan"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-29",
+      "-country": "RI",
+      "name": "Binjai",
+      "population": {
+        "-year": "90",
+        "#text": "181904"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-30",
+      "-country": "RI",
+      "name": "Probolinggo",
+      "population": {
+        "-year": "90",
+        "#text": "177120"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-32",
+      "-country": "RI",
+      "name": "Madiun",
+      "population": {
+        "-year": "90",
+        "#text": "170242"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-33",
+      "-country": "RI",
+      "name": "Pasuruan",
+      "population": {
+        "-year": "90",
+        "#text": "152409"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-34",
+      "-country": "RI",
+      "name": "Magelang",
+      "population": {
+        "-year": "90",
+        "#text": "123213"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-35",
+      "-country": "RI",
+      "name": "Sukabumi",
+      "population": {
+        "-year": "90",
+        "#text": "119981"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-36",
+      "-country": "RI",
+      "name": "Gorontalo",
+      "population": {
+        "-year": "90",
+        "#text": "119780"
+      },
+      "located_on": { "-island": "island-Sulawesi" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-37",
+      "-country": "RI",
+      "name": "Blitar",
+      "population": {
+        "-year": "90",
+        "#text": "119011"
+      },
+      "located_on": { "-island": "island-Java" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-38",
+      "-country": "RI",
+      "name": "Tebing Tinggi",
+      "population": {
+        "-year": "90",
+        "#text": "116767"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-39",
+      "-country": "RI",
+      "name": "Pangkal Pinang",
+      "population": {
+        "-year": "90",
+        "#text": "113163"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Bangka" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-41",
+      "-country": "RI",
+      "name": "Tanjung Balai",
+      "population": {
+        "-year": "90",
+        "#text": "108202"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-MalakkaStrait"
+      },
+      "located_on": { "-island": "island-Sumatra" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-42",
+      "-country": "RI",
+      "name": "Batam",
+      "population": {
+        "-year": "90",
+        "#text": "106667"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Batam" }
+    },
+    {
+      "-id": "cty-cid-cia-Indonesia-43",
+      "-country": "RI",
+      "name": "Pare Pare",
+      "population": {
+        "-year": "90",
+        "#text": "101527"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Sundasee"
+      },
+      "located_on": { "-island": "island-Sulawesi" }
+    }
+  ]
+}
+{
+  "-car_code": "TL",
+  "-area": "15007",
+  "-capital": "city-Dili-TL-TL",
+  "-memberships": "org-ACP org-FAO org-G-77 org-IBRD org-ICAO org-IDA org-IFAD org-IFC org-IFRCS org-ILO org-IMF org-IMO org-Interpol org-IOC org-NAM org-UN org-UNCTAD org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO",
+  "name": "Timor-Leste",
+  "population": "1131612",
+  "population_growth": "2.03",
+  "infant_mortality": "40.65",
+  "gdp_total": "489",
+  "gdp_agri": "32.2",
+  "gdp_ind": "12.8",
+  "gdp_serv": "55",
+  "inflation": "7.8",
+  "indep_date": "2002-05-20",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "98",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "border": {
+    "-country": "RI",
+    "-length": "228"
+  },
+  "city": {
+    "-id": "city-Dili-TL-TL",
+    "-is_country_cap": "yes",
+    "-country": "TL",
+    "name": "Dili",
+    "longitude": "125.6",
+    "latitude": "-8.6",
+    "population": {
+      "-year": "06",
+      "#text": "59069"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Sundasee"
+    },
+    "located_on": { "-island": "island-Timor" }
+  }
+}
+{
+  "-car_code": "PNG",
+  "-area": "461690",
+  "-capital": "cty-cid-cia-Papua-New-Guinea-Port-Moresby",
+  "-memberships": "org-ACP org-APEC org-AsDB org-CP org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-SPC org-SPF org-Sparteca org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO",
+  "name": "Papua New Guinea",
+  "population": "4394537",
+  "population_growth": "2.29",
+  "infant_mortality": "60.1",
+  "gdp_total": "10200",
+  "inflation": "15",
+  "indep_date": "1975-09-16",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "22",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "26",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Presbyterian"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Evangelical Alliance"
+    }
+  ],
+  "languages": {
+    "-percentage": "1",
+    "#text": "English"
+  },
+  "border": {
+    "-country": "RI",
+    "-length": "820"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Papua-New-Guinea-Port-Moresby",
+    "-is_country_cap": "yes",
+    "-country": "PNG",
+    "name": "Port Moresby",
+    "longitude": "147.15",
+    "latitude": "-9.48333",
+    "population": {
+      "-year": "87",
+      "#text": "141500"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-New-Guinea" }
+  }
+}
+{
+  "-car_code": "IRQ",
+  "-area": "437072",
+  "-capital": "cty-Iraq-Baghdad",
+  "-memberships": "org-ABEDA org-ACC org-AFESD org-AL org-AMF org-CAEU org-CCC org-ESCWA org-FAO org-G-19 org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OPEC org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Iraq",
+  "population": "21422292",
+  "population_growth": "3.69",
+  "infant_mortality": "60",
+  "gdp_total": "41100",
+  "indep_date": "1932-10-03",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "15",
+      "#text": "Kurdish"
+    },
+    {
+      "-percentage": "75",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Assyrian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "97",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TR",
+      "-length": "331"
+    },
+    {
+      "-country": "IR",
+      "-length": "1458"
+    },
+    {
+      "-country": "JOR",
+      "-length": "181"
+    },
+    {
+      "-country": "KWT",
+      "-length": "242"
+    },
+    {
+      "-country": "SA",
+      "-length": "814"
+    },
+    {
+      "-country": "SYR",
+      "-length": "605"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Iraq-2",
+      "-capital": "cty-cid-cia-Iraq-14",
+      "-country": "IRQ",
+      "name": "Al Anbar",
+      "area": "138501",
+      "population": "817868",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-14",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-2",
+        "name": "Ar Ramadi",
+        "population": {
+          "-year": "87",
+          "#text": "192556"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-3",
+      "-capital": "cty-cid-cia-Iraq-10",
+      "-country": "IRQ",
+      "name": "Babil",
+      "area": "6468",
+      "population": "1108773",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-10",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-3",
+        "name": "Al Hillah",
+        "population": {
+          "-year": "87",
+          "#text": "268834"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-4",
+      "-capital": "cty-Iraq-Baghdad",
+      "-country": "IRQ",
+      "name": "Baghdad",
+      "area": "734",
+      "population": "4648609",
+      "city": {
+        "-id": "cty-Iraq-Baghdad",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-4",
+        "name": "Baghdad",
+        "longitude": "44.4",
+        "latitude": "33.3333",
+        "population": {
+          "-year": "87",
+          "#text": "4478000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-5",
+      "-capital": "cty-cid-cia-Iraq-6",
+      "-country": "IRQ",
+      "name": "Al Basrah",
+      "area": "19070",
+      "population": "872211",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-6",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-5",
+        "name": "Al Basrah",
+        "population": {
+          "-year": "87",
+          "#text": "406296"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-SchattAlArab"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-6",
+      "-capital": "cty-cid-cia-Iraq-11",
+      "-country": "IRQ",
+      "name": "Dhi Qar",
+      "area": "12900",
+      "population": "917880",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-11",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-6",
+        "name": "An Nasiriyah",
+        "population": {
+          "-year": "87",
+          "#text": "265937"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-7",
+      "-capital": "cty-cid-cia-Iraq-16",
+      "-country": "IRQ",
+      "name": "Diyala",
+      "area": "19076",
+      "population": "955112",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-16",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-7",
+        "name": "Baqubah",
+        "population": {
+          "-year": "87",
+          "#text": "114516"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-8",
+      "-capital": "cty-cid-cia-Iraq-9",
+      "-country": "IRQ",
+      "name": "Karbala",
+      "area": "5034",
+      "population": "455868",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-9",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-8",
+        "name": "Karbala",
+        "population": {
+          "-year": "87",
+          "#text": "296705"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-9",
+      "-capital": "cty-cid-cia-Iraq-12",
+      "-country": "IRQ",
+      "name": "Maysan",
+      "area": "16072",
+      "population": "499842",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-12",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-9",
+        "name": "Al Amarah",
+        "population": {
+          "-year": "87",
+          "#text": "208797"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-10",
+      "-capital": "cty-cid-cia-Iraq-As-Samawah",
+      "-country": "IRQ",
+      "name": "Al Muthanna",
+      "area": "51740",
+      "population": "312911",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-As-Samawah",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-10",
+        "name": "As Samawah",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-11",
+      "-capital": "cty-cid-cia-Iraq-8",
+      "-country": "IRQ",
+      "name": "An Najaf",
+      "area": "28824",
+      "population": "583493",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-8",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-11",
+        "name": "An Najaf",
+        "population": {
+          "-year": "87",
+          "#text": "309010"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-12",
+      "-capital": "cty-Iraq-Mosul",
+      "-country": "IRQ",
+      "name": "Ninawa",
+      "area": "35899",
+      "population": "1507926",
+      "city": {
+        "-id": "cty-Iraq-Mosul",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-12",
+        "name": "Mosul",
+        "longitude": "43.15",
+        "latitude": "36.3167",
+        "population": {
+          "-year": "87",
+          "#text": "664221"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-13",
+      "-capital": "cty-cid-cia-Iraq-Diwaniyah",
+      "-country": "IRQ",
+      "name": "Al Qadisiyah",
+      "area": "8153",
+      "population": "560797",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-Diwaniyah",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-13",
+        "name": "Diwaniyah",
+        "population": {
+          "-year": "87",
+          "#text": "196519"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Euphrat"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-14",
+      "-capital": "cty-cid-cia-Iraq-Samarra",
+      "-country": "IRQ",
+      "name": "Salah ad Din",
+      "area": "26175",
+      "population": "723500",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-Samarra",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-14",
+        "name": "Samarra",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-15",
+      "-capital": "cty-cid-cia-Iraq-5",
+      "-country": "IRQ",
+      "name": "Ad Tamim",
+      "area": "10282",
+      "population": "592869",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-5",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-15",
+        "name": "Kirkuk",
+        "population": {
+          "-year": "87",
+          "#text": "418624"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-16",
+      "-capital": "cty-cid-cia-Iraq-15",
+      "-country": "IRQ",
+      "name": "Wasit",
+      "area": "17153",
+      "population": "546676",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-15",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-16",
+        "name": "Al Kut",
+        "population": {
+          "-year": "87",
+          "#text": "183183"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Tigris"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-18",
+      "-capital": "cty-cid-cia-Iraq-Dahuk",
+      "-country": "IRQ",
+      "name": "Dahuk",
+      "area": "6553",
+      "population": "292931",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-Dahuk",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-18",
+        "name": "Dahuk"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-19",
+      "-capital": "cty-cid-cia-Iraq-4",
+      "-country": "IRQ",
+      "name": "Arbil",
+      "area": "14471",
+      "population": "742538",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-4",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-19",
+        "name": "Arbil",
+        "population": {
+          "-year": "87",
+          "#text": "485968"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Iraq-20",
+      "-capital": "cty-cid-cia-Iraq-7",
+      "-country": "IRQ",
+      "name": "As Sulaymaniyah",
+      "area": "17023",
+      "population": "942513",
+      "city": {
+        "-id": "cty-cid-cia-Iraq-7",
+        "-is_state_cap": "yes",
+        "-country": "IRQ",
+        "-province": "prov-cid-cia-Iraq-20",
+        "name": "As Sulaymaniyah",
+        "population": {
+          "-year": "87",
+          "#text": "364096"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "JOR",
+  "-area": "89213",
+  "-capital": "cty-Jordan-Amman",
+  "-memberships": "org-ABEDA org-ACC org-AFESD org-AL org-AMF org-CAEU org-CCC org-ESCWA org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-PCA org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNMIH org-UNMOT org-UNOMIG org-UNOMIL org-UNPREDEP org-UNPROFOR org-UNRWA org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Jordan",
+  "population": "4212152",
+  "population_growth": "2.65",
+  "infant_mortality": "31.5",
+  "gdp_total": "19300",
+  "gdp_agri": "11",
+  "gdp_ind": "25",
+  "gdp_serv": "64",
+  "inflation": "3",
+  "indep_date": "1946-05-25",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "98",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Circassian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "92",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "IL",
+      "-length": "238"
+    },
+    {
+      "-country": "IRQ",
+      "-length": "181"
+    },
+    {
+      "-country": "SA",
+      "-length": "728"
+    },
+    {
+      "-country": "SYR",
+      "-length": "375"
+    },
+    {
+      "-country": "WEST",
+      "-length": "97"
+    }
+  ],
+  "city": {
+    "-id": "cty-Jordan-Amman",
+    "-is_country_cap": "yes",
+    "-country": "JOR",
+    "name": "Amman",
+    "longitude": "35.95",
+    "latitude": "31.95",
+    "population": {
+      "-year": "87",
+      "#text": "777500"
+    }
+  }
+}
+{
+  "-car_code": "KWT",
+  "-area": "17820",
+  "-capital": "cty-cid-cia-Kuwait-Kuwait",
+  "-memberships": "org-AfDB org-ABEDA org-AFESD org-AL org-AMF org-BDEAC org-CAEU org-CCC org-ESCWA org-FAO org-G-77 org-GCC org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Kuwait",
+  "population": "1950047",
+  "population_growth": "6.65",
+  "infant_mortality": "11.1",
+  "gdp_total": "30800",
+  "gdp_agri": "0",
+  "gdp_ind": "55",
+  "gdp_serv": "45",
+  "inflation": "5",
+  "indep_date": "1961-06-19",
+  "government": "nominal constitutional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "35",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Iranian"
+    },
+    {
+      "-percentage": "9",
+      "#text": "South Asian"
+    },
+    {
+      "-percentage": "45",
+      "#text": "Kuwaiti"
+    }
+  ],
+  "religions": {
+    "-percentage": "85",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "IRQ",
+      "-length": "242"
+    },
+    {
+      "-country": "SA",
+      "-length": "222"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Kuwait-Kuwait",
+    "-is_country_cap": "yes",
+    "-country": "KWT",
+    "name": "Kuwait",
+    "longitude": "47.3",
+    "latitude": "29.3",
+    "population": {
+      "-year": "87",
+      "#text": "167768"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Persischer_Golf"
+    }
+  }
+}
+{
+  "-car_code": "SA",
+  "-area": "1960582",
+  "-capital": "cty-cid-cia-Saudi-Arabia-Riyadh",
+  "-memberships": "org-AfDB org-ABEDA org-AFESD org-AL org-AMF org-CCC org-ESCWA org-FAO org-G-19 org-G-77 org-GCC org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAS org-OAPEC org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Saudi Arabia",
+  "population": "19409058",
+  "population_growth": "3.45",
+  "infant_mortality": "46.4",
+  "gdp_total": "189300",
+  "gdp_agri": "9",
+  "gdp_ind": "50",
+  "gdp_serv": "41",
+  "inflation": "5",
+  "indep_date": "1932-09-23",
+  "government": "monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "90",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Afro-Asian"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Arabic"
+  },
+  "border": [
+    {
+      "-country": "IRQ",
+      "-length": "814"
+    },
+    {
+      "-country": "JOR",
+      "-length": "728"
+    },
+    {
+      "-country": "KWT",
+      "-length": "222"
+    },
+    {
+      "-country": "OM",
+      "-length": "676"
+    },
+    {
+      "-country": "UAE",
+      "-length": "457"
+    },
+    {
+      "-country": "YE",
+      "-length": "1458"
+    },
+    {
+      "-country": "Q",
+      "-length": "60"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Saudi-Arabia-Riyadh",
+      "-is_country_cap": "yes",
+      "-country": "SA",
+      "name": "Riyadh",
+      "longitude": "46.7",
+      "latitude": "24.65",
+      "population": {
+        "-year": "87",
+        "#text": "1250000"
+      }
+    },
+    {
+      "-id": "cty-Jeddah",
+      "-country": "SA",
+      "name": "Jeddah"
+    }
+  ]
+}
+{
+  "-car_code": "SYR",
+  "-area": "185180",
+  "-capital": "cty-Syria-Damascus",
+  "-memberships": "org-ABEDA org-AFESD org-AL org-AMF org-CAEU org-CCC org-ESCWA org-FAO org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OIC org-UN org-UNESCO org-UNIDO org-UNRWA org-UPU org-WFTU org-WHO org-WMO org-WToO",
+  "name": "Syria",
+  "population": "15608648",
+  "population_growth": "3.37",
+  "infant_mortality": "40",
+  "gdp_total": "91200",
+  "gdp_agri": "30",
+  "gdp_ind": "25",
+  "gdp_serv": "45",
+  "inflation": "15.1",
+  "indep_date": "1946-04-17",
+  "government": "republic under military regime since March 1963",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "90.3",
+    "#text": "Arab"
+  },
+  "religions": [
+    {
+      "-percentage": "90",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "TR",
+      "-length": "822"
+    },
+    {
+      "-country": "IL",
+      "-length": "76"
+    },
+    {
+      "-country": "IRQ",
+      "-length": "605"
+    },
+    {
+      "-country": "JOR",
+      "-length": "375"
+    },
+    {
+      "-country": "RL",
+      "-length": "375"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Syria-Damascus",
+      "-is_country_cap": "yes",
+      "-country": "SYR",
+      "name": "Damascus",
+      "longitude": "36.3333",
+      "latitude": "33.5",
+      "population": {
+        "-year": "87",
+        "#text": "1500000"
+      }
+    },
+    {
+      "-id": "stadt-Aleppo-SYR-SYR",
+      "-country": "SYR",
+      "name": "Aleppo",
+      "longitude": "36",
+      "latitude": "36",
+      "population": {
+        "-year": "87",
+        "#text": "976727"
+      }
+    },
+    {
+      "-id": "stadt-Homs-SYR-SYR",
+      "-country": "SYR",
+      "name": "Homs",
+      "longitude": "36",
+      "latitude": "35",
+      "population": {
+        "-year": "87",
+        "#text": "354508"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RL",
+  "-area": "10400",
+  "-capital": "cty-Lebanon-Beirut",
+  "-memberships": "org-ACCT org-ABEDA org-AFESD org-AL org-AMF org-CCC org-ESCWA org-FAO org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UNRWA org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Lebanon",
+  "population": "3776317",
+  "population_growth": "2.16",
+  "infant_mortality": "36.7",
+  "gdp_total": "18300",
+  "gdp_agri": "13",
+  "gdp_ind": "28",
+  "gdp_serv": "59",
+  "inflation": "9",
+  "indep_date": "1943-11-22",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "Armenian"
+    },
+    {
+      "-percentage": "95",
+      "#text": "Arab"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "30",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "70",
+      "#text": "Muslim"
+    }
+  ],
+  "border": [
+    {
+      "-country": "IL",
+      "-length": "79"
+    },
+    {
+      "-country": "SYR",
+      "-length": "375"
+    }
+  ],
+  "city": {
+    "-id": "cty-Lebanon-Beirut",
+    "-is_country_cap": "yes",
+    "-country": "RL",
+    "name": "Beirut",
+    "longitude": "35.4667",
+    "latitude": "33.9",
+    "population": {
+      "-year": "87",
+      "#text": "702000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Mittelmeer"
+    }
+  }
+}
+{
+  "-car_code": "WEST",
+  "-area": "5860",
+  "name": "West Bank",
+  "population": "1427741",
+  "population_growth": "4.99",
+  "infant_mortality": "28.6",
+  "gdp_total": "3700",
+  "gdp_agri": "33",
+  "gdp_ind": "7",
+  "gdp_serv": "60",
+  "inflation": "14",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "17",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "83",
+      "#text": "Palestinian Arab"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "75",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "IL",
+      "-length": "307"
+    },
+    {
+      "-country": "JOR",
+      "-length": "97"
+    }
+  ],
+  "city": {
+    "-id": "cty-Westbank-Nablus",
+    "-country": "WEST",
+    "name": "Nablus",
+    "longitude": "35.3",
+    "latitude": "32.2",
+    "population": {
+      "-year": "04",
+      "#text": "104000"
+    }
+  }
+}
+{
+  "-car_code": "J",
+  "-area": "377835",
+  "-capital": "cty-Japan-Tokyo",
+  "-memberships": "org-AfDB org-AG org-APEC org-AsDB org-BIS org-CP org-CCC org-ESCAP org-EBRD org-FAO org-G-2 org-G-5 org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-EN org-NSG org-OECD org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNITAR org-UNHCR org-UNRWA org-UNU org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Japan",
+  "population": "125449703",
+  "population_growth": "0.21",
+  "infant_mortality": "4.4",
+  "gdp_total": "2679200",
+  "gdp_agri": "2.1",
+  "gdp_ind": "40.2",
+  "gdp_serv": "57.7",
+  "inflation": "-0.1",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "99.4",
+    "#text": "Japanese"
+  },
+  "religions": {
+    "-percentage": "84",
+    "#text": "Buddhist"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Japanese"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-Japan-3",
+      "-capital": "cty-Japan-Sapporo",
+      "-country": "J",
+      "name": "Hokkaido",
+      "area": "83519",
+      "population": "5692321",
+      "city": [
+        {
+          "-id": "cty-Japan-Sapporo",
+          "-is_state_cap": "yes",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-3",
+          "name": "Sapporo",
+          "longitude": "141.35",
+          "latitude": "43.0667",
+          "population": {
+            "-year": "95",
+            "#text": "1748000"
+          },
+          "located_on": { "-island": "island-Hokkaido" }
+        },
+        {
+          "-id": "cty-Japan-Hakodate",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-3",
+          "name": "Hakodate",
+          "longitude": "140.7",
+          "latitude": "41.8",
+          "population": {
+            "-year": "09",
+            "#text": "287691"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Japanisches_Meer"
+            },
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Pacific"
+            }
+          ],
+          "located_on": { "-island": "island-Hokkaido" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Japan-5",
+      "-capital": "cty-cid-cia-Japan-Aomori",
+      "-country": "J",
+      "name": "Aomori",
+      "area": "9619",
+      "population": "1481663",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Aomori",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-5",
+        "name": "Aomori",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-6",
+      "-capital": "cty-cid-cia-Japan-Morioka",
+      "-country": "J",
+      "name": "Iwate",
+      "area": "15277",
+      "population": "1419505",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Morioka",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-6",
+        "name": "Morioka",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-7",
+      "-capital": "cty-cid-cia-Japan-13",
+      "-country": "J",
+      "name": "Miyagi",
+      "area": "7292",
+      "population": "2328739",
+      "city": {
+        "-id": "cty-cid-cia-Japan-13",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-7",
+        "name": "Sendai",
+        "population": {
+          "-year": "95",
+          "#text": "967000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-8",
+      "-capital": "cty-cid-cia-Japan-Akita",
+      "-country": "J",
+      "name": "Akita",
+      "area": "11612",
+      "population": "1213667",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Akita",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-8",
+        "name": "Akita",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-9",
+      "-capital": "cty-cid-cia-Japan-Yamagata",
+      "-country": "J",
+      "name": "Yamagata",
+      "area": "9327",
+      "population": "1256958",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Yamagata",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-9",
+        "name": "Yamagata",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-10",
+      "-capital": "cty-cid-cia-Japan-Fukushima",
+      "-country": "J",
+      "name": "Fukushima",
+      "area": "13784",
+      "population": "2133592",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Fukushima",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-10",
+        "name": "Fukushima",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-12",
+      "-capital": "cty-cid-cia-Japan-Mito",
+      "-country": "J",
+      "name": "Ibaraki",
+      "area": "6094",
+      "population": "2955530",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Mito",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-12",
+        "name": "Mito",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-13",
+      "-capital": "cty-cid-cia-Japan-Utsonomiya",
+      "-country": "J",
+      "name": "Tochigi",
+      "area": "6414",
+      "population": "1984390",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Utsonomiya",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-13",
+        "name": "Utsonomiya",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-14",
+      "-capital": "cty-cid-cia-Japan-Maebashi",
+      "-country": "J",
+      "name": "Gumma",
+      "area": "6356",
+      "population": "2003540",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Maebashi",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-14",
+        "name": "Maebashi",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-15",
+      "-capital": "cty-cid-cia-Japan-Urawa",
+      "-country": "J",
+      "name": "Saitama",
+      "area": "3799",
+      "population": "6759311",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Urawa",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-15",
+        "name": "Urawa",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-16",
+      "-capital": "cty-cid-cia-Japan-14",
+      "-country": "J",
+      "name": "Chiba",
+      "area": "5150",
+      "population": "5797782",
+      "city": {
+        "-id": "cty-cid-cia-Japan-14",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-16",
+        "name": "Chiba",
+        "population": {
+          "-year": "95",
+          "#text": "843000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-17",
+      "-capital": "cty-Japan-Tokyo",
+      "-country": "J",
+      "name": "Tokyo",
+      "area": "2164",
+      "population": "11773605",
+      "city": {
+        "-id": "cty-Japan-Tokyo",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-17",
+        "name": "Tokyo",
+        "longitude": "139.767",
+        "latitude": "35.6833",
+        "population": {
+          "-year": "95",
+          "#text": "7843000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-18",
+      "-capital": "cty-cid-cia-Japan-3",
+      "-country": "J",
+      "name": "Kanagawa",
+      "area": "2402",
+      "population": "8245900",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Japan-3",
+          "-is_state_cap": "yes",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-18",
+          "name": "Yokohama",
+          "longitude": "139.32",
+          "latitude": "35.3",
+          "population": {
+            "-year": "95",
+            "#text": "3256000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          },
+          "located_on": { "-island": "island-Honshu" }
+        },
+        {
+          "-id": "cty-cid-cia-Japan-10",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-18",
+          "name": "Kawasaki",
+          "population": {
+            "-year": "95",
+            "#text": "1187000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          },
+          "located_on": { "-island": "island-Honshu" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Japan-20",
+      "-capital": "cty-cid-cia-Japan-Niigata",
+      "-country": "J",
+      "name": "Niigata",
+      "area": "12579",
+      "population": "2488364",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Niigata",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-20",
+        "name": "Niigata",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-21",
+      "-capital": "cty-cid-cia-Japan-Toyama",
+      "-country": "J",
+      "name": "Toyama",
+      "area": "4252",
+      "population": "1123125",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Toyama",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-21",
+        "name": "Toyama",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-22",
+      "-capital": "cty-cid-cia-Japan-Kanazawa",
+      "-country": "J",
+      "name": "Ishikawa",
+      "area": "4197",
+      "population": "1180068",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Kanazawa",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-22",
+        "name": "Kanazawa",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-23",
+      "-capital": "cty-cid-cia-Japan-Fukui",
+      "-country": "J",
+      "name": "Fukui",
+      "area": "4192",
+      "population": "826996",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Fukui",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-23",
+        "name": "Fukui",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-24",
+      "-capital": "cty-cid-cia-Japan-Kofu",
+      "-country": "J",
+      "name": "Yamanashi",
+      "area": "4463",
+      "population": "881996",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Kofu",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-24",
+        "name": "Kofu",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-25",
+      "-capital": "cty-cid-cia-Japan-Nagano",
+      "-country": "J",
+      "name": "Nagano",
+      "area": "13585",
+      "population": "2193984",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Nagano",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-25",
+        "name": "Nagano",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-26",
+      "-capital": "cty-cid-cia-Japan-Gifu",
+      "-country": "J",
+      "name": "Gifu",
+      "area": "10596",
+      "population": "2100315",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Gifu",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-26",
+        "name": "Gifu",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-27",
+      "-capital": "cty-cid-cia-Japan-Shizuoka",
+      "-country": "J",
+      "name": "Shizuoka",
+      "area": "7773",
+      "population": "3737689",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Shizuoka",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-27",
+        "name": "Shizuoka",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-28",
+      "-capital": "cty-cid-cia-Japan-5",
+      "-country": "J",
+      "name": "Aichi",
+      "area": "5138",
+      "population": "6868336",
+      "city": {
+        "-id": "cty-cid-cia-Japan-5",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-28",
+        "name": "Nagoya",
+        "population": {
+          "-year": "95",
+          "#text": "2108000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-30",
+      "-capital": "cty-cid-cia-Japan-Tsu",
+      "-country": "J",
+      "name": "Mie",
+      "area": "5778",
+      "population": "1841358",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Tsu",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-30",
+        "name": "Tsu",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-31",
+      "-capital": "cty-cid-cia-Japan-Otsu",
+      "-country": "J",
+      "name": "Shiga",
+      "area": "4016",
+      "population": "1287005",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Otsu",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-31",
+        "name": "Otsu",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-32",
+      "-capital": "cty-cid-cia-Japan-7",
+      "-country": "J",
+      "name": "Kyoto",
+      "area": "4613",
+      "population": "2629592",
+      "city": {
+        "-id": "cty-cid-cia-Japan-7",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-32",
+        "name": "Kyoto",
+        "longitude": "135.45",
+        "latitude": "35",
+        "population": {
+          "-year": "95",
+          "#text": "1415000"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-33",
+      "-capital": "cty-cid-cia-Japan-4",
+      "-country": "J",
+      "name": "Osaka",
+      "area": "1868",
+      "population": "8797268",
+      "city": {
+        "-id": "cty-cid-cia-Japan-4",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-33",
+        "name": "Osaka",
+        "population": {
+          "-year": "95",
+          "#text": "2492000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-34",
+      "-capital": "cty-cid-cia-Japan-8",
+      "-country": "J",
+      "name": "Hyogo",
+      "area": "8381",
+      "population": "5401877",
+      "city": {
+        "-id": "cty-cid-cia-Japan-8",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-34",
+        "name": "Kobe",
+        "population": {
+          "-year": "95",
+          "#text": "1388000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-35",
+      "-capital": "cty-cid-cia-Japan-Nara",
+      "-country": "J",
+      "name": "Nara",
+      "area": "3692",
+      "population": "1430862",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Nara",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-35",
+        "name": "Nara",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-36",
+      "-capital": "cty-cid-cia-Japan-Wakayama",
+      "-country": "J",
+      "name": "Wakayama",
+      "area": "4725",
+      "population": "1080435",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Wakayama",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-36",
+        "name": "Wakayama",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-38",
+      "-capital": "cty-cid-cia-Japan-Tottori",
+      "-country": "J",
+      "name": "Tottori",
+      "area": "3494",
+      "population": "614929",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Tottori",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-38",
+        "name": "Tottori",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-39",
+      "-capital": "cty-cid-cia-Japan-Matsue",
+      "-country": "J",
+      "name": "Shimane",
+      "area": "6628",
+      "population": "771441",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Matsue",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-39",
+        "name": "Matsue",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-40",
+      "-capital": "cty-cid-cia-Japan-Okayama",
+      "-country": "J",
+      "name": "Okayama",
+      "area": "7090",
+      "population": "1950750",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Okayama",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-40",
+        "name": "Okayama",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-41",
+      "-capital": "cty-cid-cia-Japan-11",
+      "-country": "J",
+      "name": "Hiroshima",
+      "area": "8467",
+      "population": "2881748",
+      "city": {
+        "-id": "cty-cid-cia-Japan-11",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-41",
+        "name": "Hiroshima",
+        "longitude": "132.3",
+        "latitude": "34.3",
+        "population": {
+          "-year": "95",
+          "#text": "1099000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-42",
+      "-capital": "cty-cid-cia-Japan-Yamaguchi",
+      "-country": "J",
+      "name": "Yamaguchi",
+      "area": "6106",
+      "population": "1555543",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Yamaguchi",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-42",
+        "name": "Yamaguchi",
+        "located_on": { "-island": "island-Honshu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-44",
+      "-capital": "cty-cid-cia-Japan-Tokushima",
+      "-country": "J",
+      "name": "Tokushima",
+      "area": "4145",
+      "population": "832427",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Tokushima",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-44",
+        "name": "Tokushima",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Shikoku" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-45",
+      "-capital": "cty-cid-cia-Japan-Takamatsu",
+      "-country": "J",
+      "name": "Kagawa",
+      "area": "1882",
+      "population": "1027006",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Takamatsu",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-45",
+        "name": "Takamatsu",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Shikoku" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-46",
+      "-capital": "cty-cid-cia-Japan-Matsuyama",
+      "-country": "J",
+      "name": "Ehime",
+      "area": "5672",
+      "population": "1506700",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Matsuyama",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-46",
+        "name": "Matsuyama",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Shikoku" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-47",
+      "-capital": "cty-cid-cia-Japan-Kochi",
+      "-country": "J",
+      "name": "Kochi",
+      "area": "7107",
+      "population": "816704",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Kochi",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-47",
+        "name": "Kochi",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Shikoku" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-49",
+      "-capital": "cty-Japan-Fukuoka",
+      "-country": "J",
+      "name": "Fukuoka",
+      "area": "4961",
+      "population": "4933393",
+      "city": [
+        {
+          "-id": "cty-Japan-Fukuoka",
+          "-is_state_cap": "yes",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-49",
+          "name": "Fukuoka",
+          "longitude": "130.45",
+          "latitude": "33.5833",
+          "population": {
+            "-year": "95",
+            "#text": "1273000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Japanisches_Meer"
+          },
+          "located_on": { "-island": "island-Kyushu" }
+        },
+        {
+          "-id": "cty-cid-cia-Japan-12",
+          "-country": "J",
+          "-province": "prov-cid-cia-Japan-49",
+          "name": "Kita Kyushu",
+          "population": {
+            "-year": "95",
+            "#text": "1012000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Japanisches_Meer"
+          },
+          "located_on": { "-island": "island-Kyushu" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Japan-50",
+      "-capital": "cty-cid-cia-Japan-Saga",
+      "-country": "J",
+      "name": "Saga",
+      "area": "2433",
+      "population": "884316",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Saga",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-50",
+        "name": "Saga",
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-51",
+      "-capital": "cty-cid-cia-Japan-Nagasaki",
+      "-country": "J",
+      "name": "Nagasaki",
+      "area": "4112",
+      "population": "1544934",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Nagasaki",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-51",
+        "name": "Nagasaki",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-EastChinaSea"
+        },
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-52",
+      "-capital": "cty-cid-cia-Japan-Kumamoto",
+      "-country": "J",
+      "name": "Kumamoto",
+      "area": "7408",
+      "population": "1859793",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Kumamoto",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-52",
+        "name": "Kumamoto",
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-53",
+      "-capital": "cty-cid-cia-Japan-Oita",
+      "-country": "J",
+      "name": "Oita",
+      "area": "6338",
+      "population": "1231306",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Oita",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-53",
+        "name": "Oita",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Japanisches_Meer"
+        },
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-54",
+      "-capital": "cty-cid-cia-Japan-Miyazaki",
+      "-country": "J",
+      "name": "Miyazaki",
+      "area": "7735",
+      "population": "1175819",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Miyazaki",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-54",
+        "name": "Miyazaki",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-55",
+      "-capital": "cty-cid-cia-Japan-Kagoshima",
+      "-country": "J",
+      "name": "Kagoshima",
+      "area": "9166",
+      "population": "1794224",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Kagoshima",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-55",
+        "name": "Kagoshima",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-EastChinaSea"
+        },
+        "located_on": { "-island": "island-Kyushu" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Japan-56",
+      "-capital": "cty-cid-cia-Japan-Naha",
+      "-country": "J",
+      "name": "Okinawa",
+      "area": "2255",
+      "population": "1273440",
+      "city": {
+        "-id": "cty-cid-cia-Japan-Naha",
+        "-is_state_cap": "yes",
+        "-country": "J",
+        "-province": "prov-cid-cia-Japan-56",
+        "name": "Naha",
+        "located_at": [
+          {
+            "-watertype": "sea",
+            "-sea": "sea-EastChinaSea"
+          },
+          {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        ],
+        "located_on": { "-island": "island-Okinawa" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "ROK",
+  "-area": "98480",
+  "-capital": "cty-cid-cia-South-Korea-2",
+  "-memberships": "org-AfDB org-APEC org-AsDB org-CP org-CCC org-ESCAP org-EBRD org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NSG org-OAS org-UN org-UNESCO org-UNIDO org-UNMOGIP org-MINURSO org-UNOMIG org-UNU org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "South Korea",
+  "population": "45482291",
+  "population_growth": "1.02",
+  "infant_mortality": "8.2",
+  "gdp_total": "590700",
+  "gdp_agri": "8",
+  "gdp_ind": "45",
+  "gdp_serv": "47",
+  "inflation": "4.3",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "47.4",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "48.6",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Confucianism"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Chondogyo"
+    }
+  ],
+  "border": {
+    "-country": "NOK",
+    "-length": "238"
+  },
+  "city": [
+    {
+      "-id": "cty-cid-cia-South-Korea-2",
+      "-is_country_cap": "yes",
+      "-country": "ROK",
+      "name": "Seoul",
+      "longitude": "126.967",
+      "latitude": "37.5667",
+      "population": {
+        "-year": "95",
+        "#text": "10229262"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-3",
+      "-country": "ROK",
+      "name": "Pusan",
+      "population": {
+        "-year": "95",
+        "#text": "3813814"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-4",
+      "-country": "ROK",
+      "name": "Taegu",
+      "population": {
+        "-year": "95",
+        "#text": "2449139"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-5",
+      "-country": "ROK",
+      "name": "Inchon",
+      "population": {
+        "-year": "95",
+        "#text": "2307618"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-6",
+      "-country": "ROK",
+      "name": "Kwangju",
+      "population": {
+        "-year": "95",
+        "#text": "1257504"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-7",
+      "-country": "ROK",
+      "name": "Taejon",
+      "population": {
+        "-year": "95",
+        "#text": "1272143"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-8",
+      "-country": "ROK",
+      "name": "Ulsan",
+      "population": {
+        "-year": "95",
+        "#text": "967394"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Japanisches_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-9",
+      "-country": "ROK",
+      "name": "Suwon",
+      "population": {
+        "-year": "95",
+        "#text": "755502"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-10",
+      "-country": "ROK",
+      "name": "Songnam",
+      "population": {
+        "-year": "95",
+        "#text": "869243"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-11",
+      "-country": "ROK",
+      "name": "Chonju",
+      "population": {
+        "-year": "95",
+        "#text": "563406"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-12",
+      "-country": "ROK",
+      "name": "Masan",
+      "population": {
+        "-year": "95",
+        "#text": "441358"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-13",
+      "-country": "ROK",
+      "name": "Chongju",
+      "population": {
+        "-year": "95",
+        "#text": "531195"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-14",
+      "-country": "ROK",
+      "name": "Chinju",
+      "population": {
+        "-year": "95",
+        "#text": "329913"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-15",
+      "-country": "ROK",
+      "name": "Mokpo",
+      "population": {
+        "-year": "95",
+        "#text": "247524"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-16",
+      "-country": "ROK",
+      "name": "Cheju",
+      "population": {
+        "-year": "95",
+        "#text": "258509"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-17",
+      "-country": "ROK",
+      "name": "Kunsan",
+      "population": {
+        "-year": "95",
+        "#text": "266517"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Gelbes_Meer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-18",
+      "-country": "ROK",
+      "name": "Chunchon",
+      "population": {
+        "-year": "95",
+        "#text": "235067"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-South-Korea-19",
+      "-country": "ROK",
+      "name": "Yosu",
+      "population": {
+        "-year": "95",
+        "#text": "183557"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-EastChinaSea"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MV",
+  "-area": "300",
+  "-capital": "cty-cid-cia-Maldives-Male",
+  "-memberships": "org-AsDB org-CP org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-IFAD org-IMO org-IMF org-IOC org-ITU org-Intelsat org-IDB org-NAM org-OIC org-SAARC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Maldives",
+  "population": "270758",
+  "population_growth": "3.52",
+  "infant_mortality": "47",
+  "gdp_total": "390",
+  "gdp_agri": "21.5",
+  "gdp_ind": "15.3",
+  "gdp_serv": "63.2",
+  "inflation": "16.5",
+  "indep_date": "1965-07-26",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Maldives-Male",
+    "-is_country_cap": "yes",
+    "-country": "MV",
+    "name": "Male",
+    "longitude": "72",
+    "latitude": "2",
+    "population": {
+      "-year": "87",
+      "#text": "46334"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Male" }
+  }
+}
+{
+  "-car_code": "OM",
+  "-area": "212460",
+  "-capital": "cty-cid-cia-Oman-Muscat",
+  "-memberships": "org-ABEDA org-AFESD org-AL org-AMF org-ESCWA org-FAO org-G-77 org-GCC org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ITU org-Intelsat org-IDB org-NAM org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO",
+  "name": "Oman",
+  "population": "2186548",
+  "population_growth": "3.53",
+  "infant_mortality": "27.3",
+  "gdp_total": "19100",
+  "gdp_agri": "3",
+  "gdp_ind": "60",
+  "gdp_serv": "37",
+  "inflation": "-0.7",
+  "indep_date": "1650-01-01",
+  "government": "monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "75",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "SA",
+      "-length": "676"
+    },
+    {
+      "-country": "UAE",
+      "-length": "410"
+    },
+    {
+      "-country": "YE",
+      "-length": "288"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Oman-Muscat",
+    "-is_country_cap": "yes",
+    "-country": "OM",
+    "name": "Muscat",
+    "longitude": "58.5",
+    "latitude": "23.5",
+    "population": {
+      "-year": "87",
+      "#text": "30000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-GulfOman"
+    }
+  }
+}
+{
+  "-car_code": "UAE",
+  "-area": "75581",
+  "-capital": "cty-cid-cia-United-Arab-Emirates-3",
+  "-memberships": "org-AfDB org-ABEDA org-AFESD org-AL org-AMF org-CAEU org-CCC org-ESCWA org-FAO org-G-77 org-GCC org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "United Arab Emirates",
+  "population": "3057337",
+  "population_growth": "4.33",
+  "infant_mortality": "20.4",
+  "gdp_total": "70100",
+  "gdp_agri": "2",
+  "gdp_ind": "55",
+  "gdp_serv": "43",
+  "inflation": "4.6",
+  "indep_date": "1971-12-02",
+  "government": "federation with specified powers delegated to the UAE central government and other powers reserved to member emirates",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "50",
+      "#text": "South Asian"
+    },
+    {
+      "-percentage": "19",
+      "#text": "Emiri"
+    },
+    {
+      "-percentage": "23",
+      "#text": "Arab Iranian"
+    }
+  ],
+  "religions": {
+    "-percentage": "96",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "SA",
+      "-length": "457"
+    },
+    {
+      "-country": "OM",
+      "-length": "410"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-2",
+      "-country": "UAE",
+      "name": "Abu Dhabi",
+      "area": "67350",
+      "population": "670000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-Arab-Emirates-3",
+          "-is_country_cap": "yes",
+          "-country": "UAE",
+          "-province": "prov-cid-cia-United-Arab-Emirates-2",
+          "name": "Abu Dhabi",
+          "longitude": "54.36",
+          "latitude": "24.27",
+          "population": {
+            "-year": "89",
+            "#text": "363432"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Persischer_Golf"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-Arab-Emirates-4",
+          "-country": "UAE",
+          "-province": "prov-cid-cia-United-Arab-Emirates-2",
+          "name": "Al Ayn",
+          "population": {
+            "-year": "89",
+            "#text": "176441"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-3",
+      "-country": "UAE",
+      "name": "Ajman",
+      "area": "64",
+      "population": "64000"
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-4",
+      "-country": "UAE",
+      "name": "Dubayy",
+      "area": "419",
+      "population": "419000",
+      "city": {
+        "-id": "cty-cid-cia-United-Arab-Emirates-2",
+        "-country": "UAE",
+        "-province": "prov-cid-cia-United-Arab-Emirates-4",
+        "name": "Dubai",
+        "longitude": "55",
+        "latitude": "25",
+        "population": {
+          "-year": "89",
+          "#text": "585189"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Persischer_Golf"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-5",
+      "-country": "UAE",
+      "name": "Al Fujayrah",
+      "area": "1150",
+      "population": "54000"
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-6",
+      "-country": "UAE",
+      "name": "Ras al Khaymah",
+      "area": "1700",
+      "population": "116000"
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-7",
+      "-country": "UAE",
+      "name": "Ash Shariqah",
+      "area": "2600",
+      "population": "269000",
+      "city": {
+        "-id": "cty-cid-cia-United-Arab-Emirates-5",
+        "-country": "UAE",
+        "-province": "prov-cid-cia-United-Arab-Emirates-7",
+        "name": "Sharjah",
+        "population": {
+          "-year": "89",
+          "#text": "125123"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Persischer_Golf"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-Arab-Emirates-8",
+      "-country": "UAE",
+      "name": "Umm al Qaywayn",
+      "area": "750",
+      "population": "29000"
+    }
+  ]
+}
+{
+  "-car_code": "YE",
+  "-area": "527970",
+  "-capital": "cty-cid-cia-Yemen-Sanaa",
+  "-memberships": "org-ACC org-AFESD org-AL org-AMF org-CAEU org-CCC org-ESCWA org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Yemen",
+  "population": "13483178",
+  "population_growth": "3.56",
+  "infant_mortality": "71.5",
+  "gdp_total": "37100",
+  "gdp_agri": "21",
+  "gdp_ind": "24",
+  "gdp_serv": "55",
+  "inflation": "71.3",
+  "indep_date": "1990-05-22",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Arabic"
+  },
+  "border": [
+    {
+      "-country": "SA",
+      "-length": "1458"
+    },
+    {
+      "-country": "OM",
+      "-length": "288"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Yemen-Sanaa",
+      "-is_country_cap": "yes",
+      "-country": "YE",
+      "name": "Sanaa",
+      "longitude": "44.11",
+      "latitude": "15.21",
+      "population": {
+        "-year": "87",
+        "#text": "427185"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Yemen-Aden",
+      "-country": "YE",
+      "name": "Aden",
+      "longitude": "45",
+      "latitude": "12.8",
+      "population": {
+        "-year": "04",
+        "#text": "550744"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Golf_von_Aden"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RP",
+  "-area": "299764",
+  "-capital": "cty-Philippines-Manila",
+  "-memberships": "org-AsDB org-ASEAN org-Mekong-Group org-CP org-CCC org-ESCAP org-FAO org-G-24 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-UN org-UNESCO org-UNIDO org-UNMIH org-UNHCR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Philippines",
+  "population": "74480848",
+  "population_growth": "2.18",
+  "infant_mortality": "35.9",
+  "gdp_total": "179700",
+  "gdp_agri": "22",
+  "gdp_ind": "30",
+  "gdp_serv": "48",
+  "inflation": "8.1",
+  "indep_date": "1946-07-04",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1.5",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "95.5",
+      "#text": "Malay"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "5",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "83",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Buddhist"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Philippines-Manila",
+      "-is_country_cap": "yes",
+      "-country": "RP",
+      "name": "Manila",
+      "longitude": "120.983",
+      "latitude": "14.5833",
+      "population": {
+        "-year": "95",
+        "#text": "1655000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-2",
+      "-country": "RP",
+      "name": "Quezon",
+      "population": {
+        "-year": "95",
+        "#text": "1989000"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-4",
+      "-country": "RP",
+      "name": "Kalookan",
+      "population": {
+        "-year": "95",
+        "#text": "1023000"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-5",
+      "-country": "RP",
+      "name": "Davao",
+      "population": {
+        "-year": "95",
+        "#text": "1007000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Celebessee"
+      },
+      "located_on": { "-island": "island-Mindanao" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-6",
+      "-country": "RP",
+      "name": "Cebu",
+      "population": {
+        "-year": "95",
+        "#text": "662000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Celebessee"
+      },
+      "located_on": { "-island": "island-Cebu" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-7",
+      "-country": "RP",
+      "name": "Zamboanga",
+      "population": {
+        "-year": "95",
+        "#text": "511000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Celebessee"
+      },
+      "located_on": { "-island": "island-Mindanao" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-8",
+      "-country": "RP",
+      "name": "Makati",
+      "population": {
+        "-year": "95",
+        "#text": "484000"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-9",
+      "-country": "RP",
+      "name": "Pasig",
+      "population": {
+        "-year": "95",
+        "#text": "471000"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-10",
+      "-country": "RP",
+      "name": "Cagayan de Oro",
+      "population": {
+        "-year": "95",
+        "#text": "428000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Celebessee"
+      },
+      "located_on": { "-island": "island-Mindanao" }
+    },
+    {
+      "-id": "cty-cid-cia-Philippines-11",
+      "-country": "RP",
+      "name": "Pasay",
+      "population": {
+        "-year": "95",
+        "#text": "409000"
+      },
+      "located_on": { "-island": "island-Luzon" }
+    }
+  ]
+}
+{
+  "-car_code": "Q",
+  "-area": "11000",
+  "-capital": "cty-cid-cia-Qatar-Doha",
+  "-memberships": "org-ABEDA org-AFESD org-AL org-AMF org-CCC org-ESCWA org-FAO org-G-77 org-GCC org-IAEA org-IBRD org-ICAO org-Interpol org-IFRCS org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAPEC org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Qatar",
+  "population": "547761",
+  "population_growth": "2.39",
+  "infant_mortality": "19.6",
+  "gdp_total": "10700",
+  "gdp_agri": "1",
+  "gdp_ind": "50",
+  "gdp_serv": "49",
+  "inflation": "3",
+  "indep_date": "1971-09-03",
+  "government": "traditional monarchy",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "18",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Pakistani"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Iranian"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Muslim"
+  },
+  "border": {
+    "-country": "SA",
+    "-length": "60"
+  },
+  "city": [
+    {
+      "-id": "cty-cid-cia-Qatar-Doha",
+      "-is_country_cap": "yes",
+      "-country": "Q",
+      "name": "Doha",
+      "longitude": "51",
+      "latitude": "25",
+      "population": {
+        "-year": "87",
+        "#text": "217294"
+      }
+    },
+    {
+      "-id": "stadt-Rayyan-Q-Q",
+      "-country": "Q",
+      "name": "Rayyan",
+      "longitude": "50.5",
+      "latitude": "25.5",
+      "population": {
+        "-year": "87",
+        "#text": "91996"
+      }
+    },
+    {
+      "-id": "stadt-Wakrah-Q-Q",
+      "-country": "Q",
+      "name": "Wakrah",
+      "longitude": "50.7",
+      "latitude": "25.2",
+      "population": {
+        "-year": "87",
+        "#text": "23682"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "SGP",
+  "-area": "632.6",
+  "-capital": "cty-Singapore-Singapore",
+  "-memberships": "org-AsDB org-ASEAN org-Mekong-Group org-CP org-C org-CCC org-ESCAP org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-PCA org-UN org-UNIKOM org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Singapore",
+  "population": "3396924",
+  "population_growth": "1.9",
+  "infant_mortality": "4.7",
+  "gdp_total": "66100",
+  "gdp_ind": "28",
+  "gdp_serv": "72",
+  "inflation": "1.7",
+  "indep_date": "1965-08-09",
+  "government": "republic within Commonwealth",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "6.4",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "76.4",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "14.9",
+      "#text": "Malay"
+    }
+  ],
+  "city": {
+    "-id": "cty-Singapore-Singapore",
+    "-is_country_cap": "yes",
+    "-country": "SGP",
+    "name": "Singapore",
+    "longitude": "103.833",
+    "latitude": "1.3",
+    "population": {
+      "-year": "87",
+      "#text": "2558000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-SouthChinaSea"
+    },
+    "located_on": { "-island": "island-Singapore" }
+  }
+}
+{
+  "-car_code": "CL",
+  "-area": "65610",
+  "-capital": "cty-cid-cia-Sri-Lanka-Colombo",
+  "-memberships": "org-AsDB org-CP org-C org-CCC org-ESCAP org-FAO org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-PCA org-SAARC org-UN org-UNESCO org-UNIDO org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Sri Lanka",
+  "population": "18553074",
+  "population_growth": "1.13",
+  "infant_mortality": "20.8",
+  "gdp_total": "65600",
+  "gdp_agri": "24",
+  "gdp_ind": "24",
+  "gdp_serv": "52",
+  "inflation": "8.4",
+  "indep_date": "1948-02-04",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "74",
+      "#text": "Sinhalese"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Tamil"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Vedda"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "8",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "69",
+      "#text": "Buddhist"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Hindu"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "18",
+      "#text": "Tamil"
+    },
+    {
+      "-percentage": "74",
+      "#text": "Sinhala"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Sri-Lanka-Colombo",
+    "-is_country_cap": "yes",
+    "-country": "CL",
+    "name": "Colombo",
+    "longitude": "79.8667",
+    "latitude": "6.9",
+    "population": {
+      "-year": "87",
+      "#text": "643000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Sri_Lanka" }
+  }
+}
+{
+  "-car_code": "RC",
+  "-area": "36179",
+  "-capital": "cty-Taiwan-Taipei",
+  "-memberships": "org-APEC org-AsDB org-BCIE org-ICC org-IOC org-WCL org-WTrO",
+  "name": "Taiwan",
+  "population": "21465881",
+  "population_growth": "0.89",
+  "infant_mortality": "7",
+  "gdp_total": "290500",
+  "gdp_agri": "3.6",
+  "gdp_ind": "37.3",
+  "gdp_serv": "59.1",
+  "inflation": "4",
+  "government": "multiparty democratic regime",
+  "encompassed": {
+    "-continent": "asia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "14",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "84",
+      "#text": "Taiwanese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "4.5",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "93",
+      "#text": "Taoist"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Taiwan-19",
+      "-country": "RC",
+      "name": "Changhua",
+      "population": {
+        "-year": "95",
+        "#text": "221090"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-23",
+      "-country": "RC",
+      "name": "Hualien",
+      "population": {
+        "-year": "95",
+        "#text": "107824"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-14",
+      "-country": "RC",
+      "name": "Fengshan",
+      "population": {
+        "-year": "95",
+        "#text": "301374"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-20",
+      "-country": "RC",
+      "name": "Pingtung",
+      "population": {
+        "-year": "95",
+        "#text": "214728"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-21",
+      "-country": "RC",
+      "name": "Fengyuan",
+      "population": {
+        "-year": "95",
+        "#text": "157548"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-10",
+      "-country": "RC",
+      "name": "Panchiao",
+      "population": {
+        "-year": "95",
+        "#text": "539115"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-11",
+      "-country": "RC",
+      "name": "Chungho",
+      "population": {
+        "-year": "95",
+        "#text": "387123"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-12",
+      "-country": "RC",
+      "name": "Sanchuung",
+      "population": {
+        "-year": "95",
+        "#text": "382880"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-13",
+      "-country": "RC",
+      "name": "Hsinchuang",
+      "population": {
+        "-year": "95",
+        "#text": "328758"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-16",
+      "-country": "RC",
+      "name": "Yungho",
+      "population": {
+        "-year": "95",
+        "#text": "241104"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-18",
+      "-country": "RC",
+      "name": "Hsintien",
+      "population": {
+        "-year": "95",
+        "#text": "248822"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-22",
+      "-country": "RC",
+      "name": "Taitung",
+      "population": {
+        "-year": "95",
+        "#text": "109189"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-17",
+      "-country": "RC",
+      "name": "Taoyuan",
+      "population": {
+        "-year": "95",
+        "#text": "260680"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-15",
+      "-country": "RC",
+      "name": "Chungli",
+      "population": {
+        "-year": "95",
+        "#text": "295825"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-8",
+      "-country": "RC",
+      "name": "Chiai",
+      "population": {
+        "-year": "96",
+        "#text": "261941"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-6",
+      "-country": "RC",
+      "name": "Chilung",
+      "population": {
+        "-year": "96",
+        "#text": "370049"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-EastChinaSea"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-7",
+      "-country": "RC",
+      "name": "Hsinchu",
+      "population": {
+        "-year": "96",
+        "#text": "341128"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-EastChinaSea"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-3",
+      "-country": "RC",
+      "name": "Kaohsiung",
+      "population": {
+        "-year": "96",
+        "#text": "1426518"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-cid-cia-Taiwan-4",
+      "-country": "RC",
+      "name": "Taichung",
+      "population": {
+        "-year": "96",
+        "#text": "857590"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-Taiwan-Tainan",
+      "-country": "RC",
+      "name": "Tainan",
+      "longitude": "120.2",
+      "latitude": "22.95",
+      "population": {
+        "-year": "96",
+        "#text": "707658"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-SouthChinaSea"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    },
+    {
+      "-id": "cty-Taiwan-Taipei",
+      "-is_country_cap": "yes",
+      "-country": "RC",
+      "name": "Taipei",
+      "longitude": "121.517",
+      "latitude": "25.0333",
+      "population": {
+        "-year": "96",
+        "#text": "2626138"
+      },
+      "located_on": { "-island": "island-Taiwan" }
+    }
+  ]
+}
+{
+  "-car_code": "AXA",
+  "-area": "102",
+  "-capital": "city-The-Valley-AXA-AXA",
+  "-memberships": "org-Caricom org-CDB org-Interpol org-OECS org-UPU org-WFTU",
+  "name": "Anguilla",
+  "population": "14436",
+  "population_growth": "2.27",
+  "infant_mortality": "3.52",
+  "gdp_total": "53",
+  "inflation": "4",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "90.1",
+      "#text": "Black"
+    },
+    {
+      "-percentage": "4.6",
+      "#text": "Mulatto"
+    },
+    {
+      "-percentage": "3.7",
+      "#text": "White"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "5.7",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "30.2",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "29",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "23.9",
+      "#text": "Methodist"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-The-Valley-AXA-AXA",
+    "-is_country_cap": "yes",
+    "-country": "AXA",
+    "name": "The Valley",
+    "longitude": "-63.03",
+    "latitude": "18.13",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Anguilla" }
+  }
+}
+{
+  "-car_code": "AG",
+  "-area": "442",
+  "-capital": "cty-cid-cia-Antigua-and-Barbuda-Saint-Johns",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAS org-OECS org-UN org-UNESCO org-UPU org-WCL org-WFTU org-WHO org-WMO org-WTrO",
+  "name": "Antigua and Barbuda",
+  "population": "65647",
+  "population_growth": "0.76",
+  "infant_mortality": "17.2",
+  "gdp_total": "425",
+  "gdp_agri": "3.5",
+  "gdp_ind": "19.3",
+  "gdp_serv": "77.2",
+  "inflation": "3.5",
+  "indep_date": "1981-11-01",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Antigua-and-Barbuda-Saint-Johns",
+    "-is_country_cap": "yes",
+    "-country": "AG",
+    "name": "Saint Johns",
+    "longitude": "-61.5",
+    "latitude": "17.3",
+    "population": {
+      "-year": "87",
+      "#text": "36000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Antigua" }
+  }
+}
+{
+  "-car_code": "ARU",
+  "-area": "193",
+  "-capital": "city-Oranjestad-ARU-ARU",
+  "-memberships": "org-Caricom org-ILO org-IMF org-Interpol org-IOC org-UNESCO org-UPU org-WCL org-WFTU org-WMO",
+  "name": "Aruba",
+  "population": "103065",
+  "population_growth": "1.478",
+  "infant_mortality": "13.79",
+  "gdp_total": "2258",
+  "inflation": "3.4",
+  "dependent": { "-country": "NL" },
+  "government": "part of the Dutch realm",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "80",
+    "#text": "European/Caribbean Indian"
+  },
+  "religions": [
+    {
+      "-percentage": "80.8",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Protestant"
+    }
+  ],
+  "city": {
+    "-id": "city-Oranjestad-ARU-ARU",
+    "-is_country_cap": "yes",
+    "-country": "ARU",
+    "name": "Oranjestad",
+    "longitude": "-70.02",
+    "latitude": "12.31",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Aruba" }
+  }
+}
+{
+  "-car_code": "BS",
+  "-area": "13940",
+  "-capital": "cty-Bahamas-Nassau",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-CCC org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WHO org-WIPO org-WMO",
+  "name": "Bahamas",
+  "population": "259367",
+  "population_growth": "1.05",
+  "infant_mortality": "23.3",
+  "gdp_total": "4800",
+  "gdp_agri": "3",
+  "gdp_ind": "35",
+  "gdp_serv": "62",
+  "inflation": "1.5",
+  "indep_date": "1973-07-10",
+  "government": "commonwealth",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "19",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "32",
+      "#text": "Baptist"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Methodist"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Church of God"
+    }
+  ],
+  "city": {
+    "-id": "cty-Bahamas-Nassau",
+    "-is_country_cap": "yes",
+    "-country": "BS",
+    "name": "Nassau",
+    "longitude": "-77.35",
+    "latitude": "25.0833",
+    "population": {
+      "-year": "87",
+      "#text": "140000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-NewProvidence" }
+  }
+}
+{
+  "-car_code": "BDS",
+  "-area": "430",
+  "-capital": "cty-cid-cia-Barbados-Bridgetown",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Barbados",
+  "population": "257030",
+  "population_growth": "0.26",
+  "infant_mortality": "18.7",
+  "gdp_total": "2500",
+  "gdp_agri": "6.4",
+  "gdp_ind": "39.3",
+  "gdp_serv": "54.3",
+  "inflation": "1.7",
+  "indep_date": "1966-11-30",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "European"
+    },
+    {
+      "-percentage": "80",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "4",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "67",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Barbados-Bridgetown",
+    "-is_country_cap": "yes",
+    "-country": "BDS",
+    "name": "Bridgetown",
+    "longitude": "-59.4",
+    "latitude": "13.1",
+    "population": {
+      "-year": "87",
+      "#text": "10000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Barbados" }
+  }
+}
+{
+  "-car_code": "BZ",
+  "-area": "22960",
+  "-capital": "cty-cid-cia-Belize-Belmopan",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WCL org-WHO org-WMO org-WTrO",
+  "name": "Belize",
+  "population": "219296",
+  "population_growth": "2.42",
+  "infant_mortality": "33.9",
+  "gdp_total": "575",
+  "gdp_agri": "30",
+  "inflation": "2.3",
+  "indep_date": "1981-09-21",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "30",
+      "#text": "Creole"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Garifuna"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Maya"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "62",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "GCA",
+      "-length": "266"
+    },
+    {
+      "-country": "MEX",
+      "-length": "250"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Belize-Belmopan",
+    "-is_country_cap": "yes",
+    "-country": "BZ",
+    "name": "Belmopan",
+    "longitude": "-88.5",
+    "latitude": "17.1",
+    "population": {
+      "-year": "87",
+      "#text": "3000"
+    }
+  }
+}
+{
+  "-car_code": "GCA",
+  "-area": "108890",
+  "-capital": "cty-Guatemala-Guatemala-City",
+  "-memberships": "org-OPANAL org-BCIE org-CACM org-CCC org-ECLAC org-FAO org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNMIH org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Guatemala",
+  "population": "11277614",
+  "population_growth": "2.48",
+  "infant_mortality": "50.7",
+  "gdp_total": "36700",
+  "gdp_agri": "25",
+  "gdp_ind": "20",
+  "gdp_serv": "55",
+  "inflation": "9",
+  "indep_date": "1821-09-15",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "56",
+      "#text": "Mestizo - Amerindian-Spanish"
+    },
+    {
+      "-percentage": "44",
+      "#text": "Amerindian Amerindian"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "60",
+      "#text": "Spanish"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Indian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BZ",
+      "-length": "266"
+    },
+    {
+      "-country": "MEX",
+      "-length": "962"
+    },
+    {
+      "-country": "ES",
+      "-length": "203"
+    },
+    {
+      "-country": "HCA",
+      "-length": "256"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Guatemala-Guatemala-City",
+      "-is_country_cap": "yes",
+      "-country": "GCA",
+      "name": "Guatemala City",
+      "longitude": "-90.5167",
+      "latitude": "14.6167",
+      "population": {
+        "-year": "94",
+        "#text": "823301"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Guatemala-3",
+      "-country": "GCA",
+      "name": "Mixco",
+      "population": {
+        "-year": "94",
+        "#text": "209791"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Guatemala-4",
+      "-country": "GCA",
+      "name": "Villa Nueva",
+      "population": {
+        "-year": "94",
+        "#text": "101295"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MEX",
+  "-area": "1972550",
+  "-capital": "cty-Mexico-Mexico-City",
+  "-memberships": "org-OPANAL org-AG org-APEC org-Caricom org-CDB org-BCIE org-CCC org-ECLAC org-EBRD org-FAO org-G-3 org-G-6 org-G-11 org-G-15 org-G-19 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OECD org-OAS org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UNITAR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Mexico",
+  "population": "95772462",
+  "population_growth": "1.87",
+  "infant_mortality": "25",
+  "gdp_total": "721400",
+  "gdp_agri": "8.5",
+  "gdp_ind": "28.4",
+  "gdp_serv": "63.1",
+  "inflation": "52",
+  "indep_date": "1810-09-16",
+  "government": "federal republic operating under a centralized government",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "30",
+      "#text": "Amerindian"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Caucasian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "89",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BZ",
+      "-length": "250"
+    },
+    {
+      "-country": "GCA",
+      "-length": "962"
+    },
+    {
+      "-country": "USA",
+      "-length": "3326"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Mexico-2",
+      "-capital": "cty-cid-cia-Mexico-16",
+      "-country": "MEX",
+      "name": "Aguascalientes",
+      "area": "5589",
+      "population": "862720",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-16",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-2",
+        "name": "Aguascalientes",
+        "longitude": "-102.2",
+        "latitude": "21.5",
+        "population": {
+          "-year": "90",
+          "#text": "440425"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-3",
+      "-capital": "cty-cid-cia-Mexico-19",
+      "-country": "MEX",
+      "name": "Baja California",
+      "area": "70113",
+      "population": "2112140",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-19",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-3",
+          "name": "Mexicali",
+          "longitude": "-115.3",
+          "latitude": "32.4",
+          "population": {
+            "-year": "90",
+            "#text": "438377"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-9",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-3",
+          "name": "Tijuana",
+          "population": {
+            "-year": "90",
+            "#text": "698752"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-49",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-3",
+          "name": "Ensenada",
+          "population": {
+            "-year": "90",
+            "#text": "169426"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-4",
+      "-capital": "cty-cid-cia-Mexico-61",
+      "-country": "MEX",
+      "name": "Baja California Sur",
+      "area": "73677",
+      "population": "375494",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-61",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-4",
+        "name": "La Paz",
+        "longitude": "-100.5",
+        "latitude": "24.1",
+        "population": {
+          "-year": "90",
+          "#text": "137641"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-5",
+      "-capital": "cty-cid-cia-Mexico-57",
+      "-country": "MEX",
+      "name": "Campeche",
+      "area": "51833",
+      "population": "642516",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-57",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-5",
+        "name": "Campeche",
+        "longitude": "-90.3",
+        "latitude": "19.5",
+        "population": {
+          "-year": "90",
+          "#text": "150518"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Golf_von_Mexiko"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-6",
+      "-capital": "cty-cid-cia-Mexico-30",
+      "-country": "MEX",
+      "name": "Chiapas",
+      "area": "73887",
+      "population": "3584786",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-30",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-6",
+          "name": "Tuxtla Gutierrez",
+          "longitude": "-93.1",
+          "latitude": "16.5",
+          "population": {
+            "-year": "90",
+            "#text": "289626"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-4",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-6",
+          "name": "Nezahualcoyotl",
+          "population": {
+            "-year": "90",
+            "#text": "1255456"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-56",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-6",
+          "name": "Tonala",
+          "population": {
+            "-year": "90",
+            "#text": "151190"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-60",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-6",
+          "name": "Tapachula",
+          "population": {
+            "-year": "90",
+            "#text": "138858"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-7",
+      "-capital": "cty-cid-cia-Mexico-13",
+      "-country": "MEX",
+      "name": "Chihuahua",
+      "area": "247087",
+      "population": "2793537",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-13",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-7",
+          "name": "Chihuahua",
+          "longitude": "-106.1",
+          "latitude": "28.4",
+          "population": {
+            "-year": "90",
+            "#text": "516153"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-7",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-7",
+          "name": "Ciudad Juarez",
+          "population": {
+            "-year": "90",
+            "#text": "789522"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rio_Grande"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-8",
+      "-capital": "cty-cid-cia-Mexico-22",
+      "-country": "MEX",
+      "name": "Coahuila",
+      "area": "151571",
+      "population": "2173775",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-22",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-8",
+          "name": "Saltillo",
+          "longitude": "-101",
+          "latitude": "25.3",
+          "population": {
+            "-year": "90",
+            "#text": "420947"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-17",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-8",
+          "name": "Torreon",
+          "population": {
+            "-year": "90",
+            "#text": "439436"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-47",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-8",
+          "name": "Monclova",
+          "population": {
+            "-year": "90",
+            "#text": "177792"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-9",
+      "-capital": "cty-cid-cia-Mexico-69",
+      "-country": "MEX",
+      "name": "Colima",
+      "area": "5455",
+      "population": "488028",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-69",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-9",
+        "name": "Colima",
+        "longitude": "-103.4",
+        "latitude": "19.1",
+        "population": {
+          "-year": "90",
+          "#text": "106967"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-10",
+      "-capital": "cty-cid-cia-Mexico-Durango",
+      "-country": "MEX",
+      "name": "Durango",
+      "area": "119648",
+      "population": "1431748",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-26",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-10",
+          "name": "Victoria de Durango",
+          "population": {
+            "-year": "90",
+            "#text": "348036"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-51",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-10",
+          "name": "Gomez Palacio",
+          "population": {
+            "-year": "90",
+            "#text": "164092"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Durango",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-10",
+          "name": "Durango",
+          "longitude": "-104.4",
+          "latitude": "24",
+          "population": {
+            "-year": "87",
+            "#text": "182600"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-11",
+      "-capital": "cty-cid-cia-Mexico-Guanajuato",
+      "-country": "MEX",
+      "name": "Guanajuato",
+      "area": "30589",
+      "population": "4406568",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-8",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-11",
+          "name": "Leon",
+          "population": {
+            "-year": "90",
+            "#text": "758279"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-64",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-11",
+          "name": "Salamanca",
+          "population": {
+            "-year": "90",
+            "#text": "123190"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-36",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-11",
+          "name": "Irapuato",
+          "population": {
+            "-year": "90",
+            "#text": "265042"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-41",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-11",
+          "name": "Celaya",
+          "population": {
+            "-year": "90",
+            "#text": "214856"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Guanajuato",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-11",
+          "name": "Guanajuato",
+          "longitude": "-101.2",
+          "latitude": "21",
+          "population": {
+            "-year": "87",
+            "#text": "36809"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-12",
+      "-capital": "cty-cid-cia-Mexico-Chilpancingo",
+      "-country": "MEX",
+      "name": "Guerrero",
+      "area": "63749",
+      "population": "2916567",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-14",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-12",
+          "name": "Acapulco",
+          "population": {
+            "-year": "90",
+            "#text": "515374"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Chilpancingo",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-12",
+          "name": "Chilpancingo",
+          "longitude": "-99.3",
+          "latitude": "17.3",
+          "population": {
+            "-year": "87",
+            "#text": "36193"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-13",
+      "-capital": "cty-cid-cia-Mexico-Pachuca-de-Soto",
+      "-country": "MEX",
+      "name": "Hidalgo",
+      "area": "20987",
+      "population": "2112473",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-48",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-13",
+          "name": "Pachuca",
+          "population": {
+            "-year": "90",
+            "#text": "174013"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Pachuca-de-Soto",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-13",
+          "name": "Pachuca de Soto",
+          "longitude": "-98.4",
+          "latitude": "20.1",
+          "population": {
+            "-year": "87",
+            "#text": "83892"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-14",
+      "-capital": "cty-Mexico-Guadalajara",
+      "-country": "MEX",
+      "name": "Jalisco",
+      "area": "80137",
+      "population": "5991176",
+      "city": [
+        {
+          "-id": "cty-Mexico-Guadalajara",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-14",
+          "name": "Guadalajara",
+          "longitude": "-103.333",
+          "latitude": "20.6833",
+          "population": {
+            "-year": "90",
+            "#text": "1650042"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-RioLerma"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-10",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-14",
+          "name": "Zapopan",
+          "population": {
+            "-year": "90",
+            "#text": "668323"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-27",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-14",
+          "name": "Tlaquepaque",
+          "population": {
+            "-year": "90",
+            "#text": "328031"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-67",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-14",
+          "name": "Cuautla",
+          "population": {
+            "-year": "90",
+            "#text": "110242"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-15",
+      "-capital": "cty-cid-cia-Mexico-Toluca-de-Lerdo",
+      "-country": "MEX",
+      "name": "Mexico, Estado de",
+      "area": "21461",
+      "population": "11707964",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-28",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-15",
+          "name": "Toluca",
+          "population": {
+            "-year": "90",
+            "#text": "327865"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Toluca-de-Lerdo",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-15",
+          "name": "Toluca de Lerdo",
+          "longitude": "-99.4",
+          "latitude": "19.2",
+          "population": {
+            "-year": "87",
+            "#text": "136100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-29",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-15",
+          "name": "Atizapan de Zaragoza",
+          "population": {
+            "-year": "90",
+            "#text": "315059"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-16",
+      "-capital": "cty-cid-cia-Mexico-21",
+      "-country": "MEX",
+      "name": "Michoacan",
+      "area": "59864",
+      "population": "3870604",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-21",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-16",
+          "name": "Morelia",
+          "longitude": "-101.1",
+          "latitude": "19.4",
+          "population": {
+            "-year": "90",
+            "#text": "428486"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-46",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-16",
+          "name": "Uruapan",
+          "population": {
+            "-year": "90",
+            "#text": "187623"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-68",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-16",
+          "name": "Zamora",
+          "population": {
+            "-year": "90",
+            "#text": "109751"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-17",
+      "-capital": "cty-cid-cia-Mexico-32",
+      "-country": "MEX",
+      "name": "Morelos",
+      "area": "4941",
+      "population": "1442662",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-32",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-17",
+        "name": "Cuernavaca",
+        "longitude": "-99.2",
+        "latitude": "19",
+        "population": {
+          "-year": "90",
+          "#text": "279187"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-18",
+      "-capital": "cty-cid-cia-Mexico-43",
+      "-country": "MEX",
+      "name": "Nayarit",
+      "area": "27621",
+      "population": "896702",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-43",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-18",
+        "name": "Tepic",
+        "longitude": "-104.5",
+        "latitude": "21.3",
+        "population": {
+          "-year": "90",
+          "#text": "206967"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-19",
+      "-capital": "cty-Mexico-Monterrey",
+      "-country": "MEX",
+      "name": "Nuevo Leon",
+      "area": "64555",
+      "population": "3550114",
+      "city": [
+        {
+          "-id": "cty-Mexico-Monterrey",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "Monterrey",
+          "longitude": "-100.3",
+          "latitude": "25.6667",
+          "population": {
+            "-year": "90",
+            "#text": "1068996"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-11",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "Guadalupe",
+          "population": {
+            "-year": "90",
+            "#text": "535332"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-20",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "San Nicolas de los Garza",
+          "population": {
+            "-year": "90",
+            "#text": "436603"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-52",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "Ciudad Santa Catarina",
+          "population": {
+            "-year": "90",
+            "#text": "162707"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-66",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "San Pedro Garza Garcia",
+          "population": {
+            "-year": "90",
+            "#text": "113017"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-71",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-19",
+          "name": "Ciudad Apodaca",
+          "population": {
+            "-year": "90",
+            "#text": "103364"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-20",
+      "-capital": "cty-cid-cia-Mexico-42",
+      "-country": "MEX",
+      "name": "Oaxaca",
+      "area": "95364",
+      "population": "3228895",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-42",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-20",
+        "name": "Oaxaca",
+        "longitude": "-96.4",
+        "latitude": "17",
+        "population": {
+          "-year": "90",
+          "#text": "212818"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-21",
+      "-capital": "cty-cid-cia-Mexico-6",
+      "-country": "MEX",
+      "name": "Puebla",
+      "area": "33919",
+      "population": "4624365",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-6",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-21",
+          "name": "Puebla",
+          "longitude": "-98.1",
+          "latitude": "19",
+          "population": {
+            "-year": "90",
+            "#text": "1007170"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-59",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-21",
+          "name": "Tehuacan",
+          "population": {
+            "-year": "90",
+            "#text": "139450"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-22",
+      "-capital": "cty-cid-cia-Mexico-25",
+      "-country": "MEX",
+      "name": "Queretaro",
+      "area": "11769",
+      "population": "1250476",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-25",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-22",
+        "name": "Queretaro",
+        "longitude": "-100.2",
+        "latitude": "20.4",
+        "population": {
+          "-year": "90",
+          "#text": "385503"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-23",
+      "-capital": "cty-cid-cia-Mexico-Chetumal",
+      "-country": "MEX",
+      "name": "Quintana Roo",
+      "area": "50350",
+      "population": "703536",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-50",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-23",
+          "name": "Cancun",
+          "population": {
+            "-year": "90",
+            "#text": "167730"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Chetumal",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-23",
+          "name": "Chetumal",
+          "longitude": "-88.2",
+          "latitude": "18.3",
+          "population": {
+            "-year": "87",
+            "#text": "23685"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-24",
+      "-capital": "cty-cid-cia-Mexico-15",
+      "-country": "MEX",
+      "name": "San Luis Potosi",
+      "area": "62848",
+      "population": "2200763",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-15",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-24",
+          "name": "San Luis Potosi",
+          "longitude": "-101",
+          "latitude": "22.1",
+          "population": {
+            "-year": "90",
+            "#text": "488238"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-63",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-24",
+          "name": "Soledad de Graciano Sanchez",
+          "population": {
+            "-year": "90",
+            "#text": "123943"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-25",
+      "-capital": "cty-cid-cia-Mexico-23",
+      "-country": "MEX",
+      "name": "Sinaloa",
+      "area": "58092",
+      "population": "2425675",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-23",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-25",
+          "name": "Culiacan",
+          "longitude": "-107.4",
+          "latitude": "24.5",
+          "population": {
+            "-year": "90",
+            "#text": "415046"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-37",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-25",
+          "name": "Mazatlan",
+          "population": {
+            "-year": "90",
+            "#text": "262705"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-53",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-25",
+          "name": "Los Mochis",
+          "population": {
+            "-year": "90",
+            "#text": "162659"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-26",
+      "-capital": "cty-cid-cia-Mexico-24",
+      "-country": "MEX",
+      "name": "Sonora",
+      "area": "184934",
+      "population": "2085536",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-24",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-26",
+          "name": "Hermosillo",
+          "longitude": "-111",
+          "latitude": "29",
+          "population": {
+            "-year": "90",
+            "#text": "406417"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-39",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-26",
+          "name": "Ciudad Obregon",
+          "population": {
+            "-year": "90",
+            "#text": "219980"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-70",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-26",
+          "name": "Heroica Nogales",
+          "population": {
+            "-year": "90",
+            "#text": "105873"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-27",
+      "-capital": "cty-cid-cia-Mexico-38",
+      "-country": "MEX",
+      "name": "Tabasco",
+      "area": "24661",
+      "population": "1748769",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-38",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-27",
+        "name": "Villahermosa",
+        "longitude": "-93",
+        "latitude": "18",
+        "population": {
+          "-year": "90",
+          "#text": "261231"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-28",
+      "-capital": "cty-cid-cia-Mexico-45",
+      "-country": "MEX",
+      "name": "Tamaulipas",
+      "area": "79829",
+      "population": "2527328",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-45",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Ciudad Victoria",
+          "longitude": "-99.1",
+          "latitude": "23.4",
+          "population": {
+            "-year": "90",
+            "#text": "194996"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-33",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Tampico",
+          "population": {
+            "-year": "90",
+            "#text": "272690"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-34",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Matamoros",
+          "population": {
+            "-year": "90",
+            "#text": "266055"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-35",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Reynosa",
+          "population": {
+            "-year": "90",
+            "#text": "265663"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rio_Grande"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-40",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Nuevo Laredo",
+          "population": {
+            "-year": "90",
+            "#text": "218413"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rio_Grande"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-54",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-28",
+          "name": "Ciudad Madero",
+          "population": {
+            "-year": "90",
+            "#text": "160331"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-29",
+      "-capital": "cty-cid-cia-Mexico-Tlaxcala",
+      "-country": "MEX",
+      "name": "Tlaxcala",
+      "area": "3914",
+      "population": "883924",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-Tlaxcala",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-29",
+        "name": "Tlaxcala",
+        "longitude": "-98.1",
+        "latitude": "19.2",
+        "population": {
+          "-year": "87",
+          "#text": "116800"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-30",
+      "-capital": "cty-cid-cia-Mexico-Jalapa",
+      "-country": "MEX",
+      "name": "Veracruz",
+      "area": "72815",
+      "population": "6737324",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mexico-62",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Cordoba",
+          "population": {
+            "-year": "90",
+            "#text": "130695"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-31",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Xalapa Enriquez",
+          "population": {
+            "-year": "90",
+            "#text": "279451"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-44",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Coatzacoalcos",
+          "population": {
+            "-year": "90",
+            "#text": "198817"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-55",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Poza Rica",
+          "population": {
+            "-year": "90",
+            "#text": "151739"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-58",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Minatitlan",
+          "population": {
+            "-year": "90",
+            "#text": "142060"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-65",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Orizaba",
+          "population": {
+            "-year": "90",
+            "#text": "114216"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-Jalapa",
+          "-is_state_cap": "yes",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Jalapa",
+          "longitude": "-97",
+          "latitude": "19.3",
+          "population": {
+            "-year": "87",
+            "#text": "161400"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mexico-18",
+          "-country": "MEX",
+          "-province": "prov-cid-cia-Mexico-30",
+          "name": "Veracruz Llave",
+          "population": {
+            "-year": "90",
+            "#text": "438821"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-31",
+      "-capital": "cty-Mexico-Merida",
+      "-country": "MEX",
+      "name": "Yucatan",
+      "area": "39340",
+      "population": "1556622",
+      "city": {
+        "-id": "cty-Mexico-Merida",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-31",
+        "name": "Merida",
+        "longitude": "-89.6333",
+        "latitude": "20.9667",
+        "population": {
+          "-year": "90",
+          "#text": "523422"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-32",
+      "-capital": "cty-cid-cia-Mexico-72",
+      "-country": "MEX",
+      "name": "Zacatecas",
+      "area": "75040",
+      "population": "1336496",
+      "city": {
+        "-id": "cty-cid-cia-Mexico-72",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-32",
+        "name": "Zacatecas",
+        "longitude": "-102.4",
+        "latitude": "22.5",
+        "population": {
+          "-year": "90",
+          "#text": "100051"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mexico-34",
+      "-capital": "cty-Mexico-Mexico-City",
+      "-country": "MEX",
+      "name": "Distrito Federal",
+      "area": "1499",
+      "population": "8489007",
+      "city": {
+        "-id": "cty-Mexico-Mexico-City",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "MEX",
+        "-province": "prov-cid-cia-Mexico-34",
+        "name": "Mexico City",
+        "longitude": "-99.2",
+        "latitude": "19.4",
+        "population": {
+          "-year": "90",
+          "#text": "9815795"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BERM",
+  "-area": "53.3",
+  "-capital": "city-Hamilton-BERM-BERM",
+  "-memberships": "org-Caricom org-ICFTU org-Interpol org-IOC org-UPU org-WFTU",
+  "name": "Bermuda",
+  "population": "67837",
+  "population_growth": "0.65",
+  "infant_mortality": "2.46",
+  "gdp_total": "4500",
+  "gdp_agri": "1",
+  "gdp_ind": "10",
+  "gdp_serv": "89",
+  "inflation": "2.8",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "54",
+      "#text": "Black"
+    },
+    {
+      "-percentage": "34.1",
+      "#text": "White"
+    },
+    {
+      "-percentage": "6.4",
+      "#text": "Mixed"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "15",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "23",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "11",
+      "#text": "African Methodist Episcopal"
+    }
+  ],
+  "city": {
+    "-id": "city-Hamilton-BERM-BERM",
+    "-is_country_cap": "yes",
+    "-country": "BERM",
+    "name": "Hamilton",
+    "longitude": "-64.47",
+    "latitude": "32.17",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-GrandBermuda" }
+  }
+}
+{
+  "-car_code": "BVIR",
+  "-area": "153",
+  "-capital": "city-Road-Town-BVIR-BVIR",
+  "-memberships": "org-Caricom org-CDB org-ECLAC org-Interpol org-IOC org-OECS org-UNESCO org-UPU org-WFTU",
+  "name": "British Virgin Islands",
+  "population": "24491",
+  "population_growth": "1.84",
+  "infant_mortality": "14.65",
+  "gdp_total": "133",
+  "inflation": "2.5",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "83.4",
+      "#text": "Black"
+    },
+    {
+      "-percentage": "7",
+      "#text": "White"
+    },
+    {
+      "-percentage": "3.4",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "5.4",
+      "#text": "Mixed"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "10",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "86",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-Road-Town-BVIR-BVIR",
+    "-is_country_cap": "yes",
+    "-country": "BVIR",
+    "name": "Road Town",
+    "longitude": "-64.37",
+    "latitude": "18.27",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Tortola" }
+  }
+}
+{
+  "-car_code": "CDN",
+  "-area": "9976140",
+  "-capital": "cty-cid-cia-Canada-13",
+  "-memberships": "org-AfDB org-ACCT org-AG org-APEC org-AsDB org-BIS org-CDB org-C org-CCC org-ECE org-ECLAC org-EBRD org-ESA org-FAO org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-UN org-UNAMIR org-UNCRO org-UNDOF org-UNESCO org-UNFICYP org-UNIDO org-UNITAR org-UNIKOM org-UNMIH org-UNHCR org-UNPREDEP org-UNPROFOR org-UNTSO org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "Canada",
+  "population": "28820671",
+  "population_growth": "1.06",
+  "infant_mortality": "6.1",
+  "gdp_total": "694000",
+  "gdp_agri": "2",
+  "gdp_ind": "26",
+  "gdp_serv": "72",
+  "inflation": "2.4",
+  "indep_date": "1867-07-01",
+  "government": "confederation with parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "27",
+      "#text": "French"
+    },
+    {
+      "-percentage": "11.5",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "20",
+      "#text": "European"
+    },
+    {
+      "-percentage": "40",
+      "#text": "British Isles"
+    },
+    {
+      "-percentage": "1.5",
+      "#text": "Inuit"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "45",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "12",
+      "#text": "United Church"
+    }
+  ],
+  "border": {
+    "-country": "USA",
+    "-length": "8893"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-Canada-2",
+      "-capital": "cty-cid-cia-Canada-6",
+      "-country": "CDN",
+      "name": "Alberta",
+      "area": "661185",
+      "population": "2696826",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Canada-6",
+          "-is_state_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-2",
+          "name": "Edmonton",
+          "longitude": "-113.517",
+          "latitude": "53.5667",
+          "population": {
+            "-year": "91",
+            "#text": "616741"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-SaskatchewanRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-3",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-2",
+          "name": "Calgary",
+          "longitude": "-114.017",
+          "latitude": "51.1",
+          "population": {
+            "-year": "91",
+            "#text": "710677"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Canada-3",
+      "-capital": "cty-cid-cia-Canada-Victoria",
+      "-country": "CDN",
+      "name": "British Columbia",
+      "area": "948596",
+      "population": "3724500",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Canada-9",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-3",
+          "name": "Vancouver",
+          "longitude": "-122.4",
+          "latitude": "48.5",
+          "population": {
+            "-year": "91",
+            "#text": "471844"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-23",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-3",
+          "name": "Burnaby",
+          "population": {
+            "-year": "91",
+            "#text": "158858"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-Victoria",
+          "-is_state_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-3",
+          "name": "Victoria",
+          "longitude": "-123.317",
+          "latitude": "48.4167",
+          "population": {
+            "-year": "87",
+            "#text": "263180"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          },
+          "located_on": { "-island": "island-Vancouver_Island" }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-16",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-3",
+          "name": "Surrey",
+          "population": {
+            "-year": "91",
+            "#text": "245173"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Canada-4",
+      "-capital": "cty-cid-cia-Canada-5",
+      "-country": "CDN",
+      "name": "Manitoba",
+      "area": "650086",
+      "population": "1113898",
+      "city": {
+        "-id": "cty-cid-cia-Canada-5",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-4",
+        "name": "Winnipeg",
+        "longitude": "-97.1",
+        "latitude": "49.5",
+        "population": {
+          "-year": "91",
+          "#text": "616790"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-5",
+      "-capital": "cty-cid-cia-Canada-Fredericton",
+      "-country": "CDN",
+      "name": "New Brunswick",
+      "area": "73437",
+      "population": "738133",
+      "city": {
+        "-id": "cty-cid-cia-Canada-Fredericton",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-5",
+        "name": "Fredericton",
+        "longitude": "-52.5",
+        "latitude": "45.5",
+        "population": {
+          "-year": "87",
+          "#text": "45248"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-6",
+      "-capital": "cty-cid-cia-Canada-Saint-Johns",
+      "-country": "CDN",
+      "name": "Newfoundland",
+      "area": "404517",
+      "population": "551792",
+      "city": {
+        "-id": "cty-cid-cia-Canada-Saint-Johns",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-6",
+        "name": "Saint Johns",
+        "longitude": "-52.5",
+        "latitude": "47.3",
+        "population": {
+          "-year": "87",
+          "#text": "143390"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        },
+        "located_on": { "-island": "island-Newfoundland" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-7",
+      "-capital": "cty-cid-cia-Canada-Yellowknife",
+      "-country": "CDN",
+      "name": "Northwest Territories",
+      "area": "3379683",
+      "population": "64402",
+      "city": {
+        "-id": "cty-cid-cia-Canada-Yellowknife",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-7",
+        "name": "Yellowknife",
+        "longitude": "-114.45",
+        "latitude": "62.4667",
+        "population": {
+          "-year": "87",
+          "#text": "8256"
+        },
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Great_Slave_Lake"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-8",
+      "-capital": "cty-cid-cia-Canada-31",
+      "-country": "CDN",
+      "name": "Nova Scotia",
+      "area": "55490",
+      "population": "909282",
+      "city": {
+        "-id": "cty-cid-cia-Canada-31",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-8",
+        "name": "Halifax",
+        "longitude": "-63.4",
+        "latitude": "44.4",
+        "population": {
+          "-year": "91",
+          "#text": "114455"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-9",
+      "-capital": "cty-cid-cia-Canada-4",
+      "-country": "CDN",
+      "name": "Ontario",
+      "area": "1068582",
+      "population": "10753573",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Canada-13",
+          "-is_country_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Ottawa",
+          "longitude": "-75.5",
+          "latitude": "45.3",
+          "population": {
+            "-year": "91",
+            "#text": "313987"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-36",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Gloucester",
+          "population": {
+            "-year": "91",
+            "#text": "101677"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-33",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Vaughan",
+          "population": {
+            "-year": "91",
+            "#text": "111359"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-15",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "London",
+          "longitude": "-81.15",
+          "latitude": "43.0333",
+          "population": {
+            "-year": "91",
+            "#text": "303165"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-11",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Hamilton",
+          "longitude": "-79.9",
+          "latitude": "43.2667",
+          "population": {
+            "-year": "91",
+            "#text": "318499"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Ontario"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-8",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Scarborough",
+          "population": {
+            "-year": "91",
+            "#text": "524598"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-25",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "York",
+          "population": {
+            "-year": "91",
+            "#text": "140525"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-4",
+          "-is_state_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Toronto",
+          "longitude": "-79.5",
+          "latitude": "43.5",
+          "population": {
+            "-year": "91",
+            "#text": "635395"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Ontario"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-21",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Kitchener",
+          "longitude": "-80.5",
+          "latitude": "43.4333",
+          "population": {
+            "-year": "91",
+            "#text": "168282"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-18",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Windsor",
+          "longitude": "-82.9667",
+          "latitude": "42.2667",
+          "population": {
+            "-year": "91",
+            "#text": "191435"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-DetroitRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-28",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Oshawa",
+          "longitude": "-78.8667",
+          "latitude": "43.9",
+          "population": {
+            "-year": "91",
+            "#text": "129344"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Ontario"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-7",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "North York",
+          "population": {
+            "-year": "91",
+            "#text": "562564"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-10",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Mississauga",
+          "population": {
+            "-year": "91",
+            "#text": "463388"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-14",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Etobicoke",
+          "population": {
+            "-year": "91",
+            "#text": "309993"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-17",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Brampton",
+          "population": {
+            "-year": "91",
+            "#text": "234445"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-24",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Markham",
+          "population": {
+            "-year": "91",
+            "#text": "153811"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-27",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Burlington",
+          "population": {
+            "-year": "91",
+            "#text": "129575"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-29",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Saint Catharines",
+          "population": {
+            "-year": "91",
+            "#text": "129300"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Ontario"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-32",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Thunder Bay",
+          "longitude": "-89.3167",
+          "latitude": "48.3667",
+          "population": {
+            "-year": "91",
+            "#text": "113746"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Superior"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-34",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "Nepean",
+          "population": {
+            "-year": "91",
+            "#text": "107627"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-35",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-9",
+          "name": "East York",
+          "population": {
+            "-year": "91",
+            "#text": "102696"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Canada-10",
+      "-capital": "cty-cid-cia-Canada-Charlottetown",
+      "-country": "CDN",
+      "name": "Prince Edward Island",
+      "area": "5657",
+      "population": "134557",
+      "city": {
+        "-id": "cty-cid-cia-Canada-Charlottetown",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-10",
+        "name": "Charlottetown",
+        "longitude": "-63.2",
+        "latitude": "46.2",
+        "population": {
+          "-year": "87",
+          "#text": "24837"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Canada-11",
+      "-capital": "cty-cid-cia-Canada-22",
+      "-country": "CDN",
+      "name": "Quebec",
+      "area": "1540680",
+      "population": "7138795",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Canada-12",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-11",
+          "name": "Laval",
+          "population": {
+            "-year": "91",
+            "#text": "314398"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-2",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-11",
+          "name": "Montreal",
+          "longitude": "-73.5",
+          "latitude": "45.3",
+          "population": {
+            "-year": "91",
+            "#text": "1017666"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-StLorenzstrom"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-22",
+          "-is_state_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-11",
+          "name": "Quebec",
+          "longitude": "-71.3833",
+          "latitude": "46.8",
+          "population": {
+            "-year": "91",
+            "#text": "167517"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-StLorenzstrom"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-26",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-11",
+          "name": "Longueuil",
+          "population": {
+            "-year": "91",
+            "#text": "129874"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-30",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-11",
+          "name": "Richmond",
+          "population": {
+            "-year": "91",
+            "#text": "126624"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Canada-12",
+      "-capital": "cty-cid-cia-Canada-20",
+      "-country": "CDN",
+      "name": "Saskatchewan",
+      "area": "651900",
+      "population": "990237",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Canada-20",
+          "-is_state_cap": "yes",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-12",
+          "name": "Regina",
+          "longitude": "-104.667",
+          "latitude": "50.4333",
+          "population": {
+            "-year": "91",
+            "#text": "179178"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Canada-19",
+          "-country": "CDN",
+          "-province": "prov-cid-cia-Canada-12",
+          "name": "Saskatoon",
+          "longitude": "-107",
+          "latitude": "52",
+          "population": {
+            "-year": "91",
+            "#text": "186058"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-SaskatchewanRiver"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Canada-13",
+      "-capital": "cty-cid-cia-Canada-Whitehorse",
+      "-country": "CDN",
+      "name": "Yukon Territory",
+      "area": "536324",
+      "population": "30766",
+      "city": {
+        "-id": "cty-cid-cia-Canada-Whitehorse",
+        "-is_state_cap": "yes",
+        "-country": "CDN",
+        "-province": "prov-cid-cia-Canada-13",
+        "name": "Whitehorse",
+        "longitude": "-135",
+        "latitude": "60.4",
+        "population": {
+          "-year": "87",
+          "#text": "13311"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Yukon"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "USA",
+  "-area": "9372610",
+  "-capital": "cty-cid-cia-United-States-22",
+  "-memberships": "org-AfDB org-AG org-APEC org-AsDB org-ANZUS org-BIS org-CP org-CCC org-ESCAP org-ECE org-ECLAC org-EBRD org-FAO org-G-2 org-G-5 org-G-7 org-G-8 org-G-10 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-ANC org-NATO org-EN org-NSG org-OECD org-OSCE org-OAS org-PCA org-SPC org-UN org-UNCRO org-UNIDO org-UNITAR org-UNIKOM org-MINURSO org-UNMIH org-UNOMIG org-UNHCR org-UNPREDEP org-UNPROFOR org-UNRWA org-UNTSO org-UNU org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO org-ZC",
+  "name": "United States",
+  "population": "266476278",
+  "population_growth": "0.91",
+  "infant_mortality": "6.7",
+  "gdp_total": "7247700",
+  "gdp_agri": "2",
+  "gdp_ind": "23",
+  "gdp_serv": "75",
+  "inflation": "2.5",
+  "indep_date": "1776-07-04",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3.3",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "0.8",
+      "#text": "Native American"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "28",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "56",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "MEX",
+      "-length": "3326"
+    },
+    {
+      "-country": "CDN",
+      "-length": "8893"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-United-States-2",
+      "-capital": "cty-cid-cia-United-States-83",
+      "-country": "USA",
+      "name": "Alabama",
+      "area": "133915",
+      "population": "4319154",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-64",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-2",
+          "name": "Birmingham",
+          "longitude": "-86.75",
+          "latitude": "33.5667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "265347"
+            },
+            {
+              "-year": "96",
+              "#text": "258543"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-80",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-2",
+          "name": "Mobile",
+          "longitude": "-88.25",
+          "latitude": "30.6833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "196263"
+            },
+            {
+              "-year": "96",
+              "#text": "202581"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-83",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-2",
+          "name": "Montgomery",
+          "longitude": "-86.3667",
+          "latitude": "32.3833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "190350"
+            },
+            {
+              "-year": "96",
+              "#text": "196363"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-112",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-2",
+          "name": "Huntsville",
+          "longitude": "-86.5833",
+          "latitude": "34.7",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "159880"
+            },
+            {
+              "-year": "96",
+              "#text": "170424"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-TennesseeRiver"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-3",
+      "-capital": "cty-cid-cia-United-States-Juneau",
+      "-country": "USA",
+      "name": "Alaska",
+      "area": "1530694",
+      "population": "609311",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-67",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-3",
+          "name": "Anchorage",
+          "longitude": "-150.017",
+          "latitude": "61.1667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "226338"
+            },
+            {
+              "-year": "96",
+              "#text": "250505"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Juneau",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-3",
+          "name": "Juneau",
+          "longitude": "-134.583",
+          "latitude": "58.3667",
+          "population": {
+            "-year": "87",
+            "#text": "19528"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-4",
+      "-capital": "cty-cid-cia-United-States-8",
+      "-country": "USA",
+      "name": "Arizona",
+      "area": "295259",
+      "population": "4554966",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-8",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Phoenix",
+          "longitude": "-112.2",
+          "latitude": "33.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "984310"
+            },
+            {
+              "-year": "96",
+              "#text": "1159014"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-32",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Tucson",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "411480"
+            },
+            {
+              "-year": "96",
+              "#text": "449002"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-52",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Mesa",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "289199"
+            },
+            {
+              "-year": "96",
+              "#text": "344764"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-101",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Glendale",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "147864"
+            },
+            {
+              "-year": "96",
+              "#text": "182219"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-103",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Scottsdale",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "130075"
+            },
+            {
+              "-year": "96",
+              "#text": "179012"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-117",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Tempe",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "141993"
+            },
+            {
+              "-year": "96",
+              "#text": "162701"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-135",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-4",
+          "name": "Chandler",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "89862"
+            },
+            {
+              "-year": "96",
+              "#text": "142918"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-5",
+      "-capital": "cty-cid-cia-United-States-107",
+      "-country": "USA",
+      "name": "Arkansas",
+      "area": "137754",
+      "population": "2522819",
+      "city": {
+        "-id": "cty-cid-cia-United-States-107",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-5",
+        "name": "Little Rock",
+        "longitude": "-92.3",
+        "latitude": "34.4",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "175727"
+          },
+          {
+            "-year": "96",
+            "#text": "175752"
+          }
+        ],
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Arkansas_River"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-6",
+      "-capital": "cty-cid-cia-United-States-43",
+      "-country": "USA",
+      "name": "California",
+      "area": "411047",
+      "population": "32268301",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-3",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Los Angeles",
+          "longitude": "-118",
+          "latitude": "34",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "3485557"
+            },
+            {
+              "-year": "96",
+              "#text": "3553638"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-7",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "San Diego",
+          "longitude": "-117.167",
+          "latitude": "32.7333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "1110623"
+            },
+            {
+              "-year": "96",
+              "#text": "1171121"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-12",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "San Jose",
+          "longitude": "-121.933",
+          "latitude": "37.3667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "782224"
+            },
+            {
+              "-year": "96",
+              "#text": "838744"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-14",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "San Francisco",
+          "longitude": "-122.383",
+          "latitude": "37.6167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "723959"
+            },
+            {
+              "-year": "96",
+              "#text": "735315"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-37",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Long Beach",
+          "longitude": "-118.15",
+          "latitude": "33.8167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "429321"
+            },
+            {
+              "-year": "96",
+              "#text": "421904"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-40",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Fresno",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "354091"
+            },
+            {
+              "-year": "96",
+              "#text": "396011"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-43",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Sacramento",
+          "longitude": "-121.5",
+          "latitude": "38.5167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "369365"
+            },
+            {
+              "-year": "96",
+              "#text": "376243"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-44",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Oakland",
+          "longitude": "-122.317",
+          "latitude": "37.8167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "372242"
+            },
+            {
+              "-year": "96",
+              "#text": "367230"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-56",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Santa Ana",
+          "longitude": "-117.867",
+          "latitude": "33.75",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "293827"
+            },
+            {
+              "-year": "96",
+              "#text": "302419"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-58",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Anaheim",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "266406"
+            },
+            {
+              "-year": "96",
+              "#text": "288945"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-65",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Riverside",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "226546"
+            },
+            {
+              "-year": "96",
+              "#text": "255069"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-72",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Stockton",
+          "longitude": "-121.25",
+          "latitude": "37.9",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "210943"
+            },
+            {
+              "-year": "96",
+              "#text": "232660"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-78",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Bakersfield",
+          "longitude": "-119.05",
+          "latitude": "35.4167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "176264"
+            },
+            {
+              "-year": "96",
+              "#text": "205508"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-91",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Huntington Beach",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "181519"
+            },
+            {
+              "-year": "96",
+              "#text": "190751"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-95",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Fremont",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "173339"
+            },
+            {
+              "-year": "96",
+              "#text": "187800"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-98",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Glendale",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "180038"
+            },
+            {
+              "-year": "96",
+              "#text": "184321"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-99",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "San Bernardino",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "170036"
+            },
+            {
+              "-year": "96",
+              "#text": "183474"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-104",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Modesto",
+          "longitude": "-121",
+          "latitude": "37.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "164746"
+            },
+            {
+              "-year": "96",
+              "#text": "178559"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-123",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Chula Vista",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "135160"
+            },
+            {
+              "-year": "96",
+              "#text": "151963"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-125",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Oxnard",
+          "longitude": "-119.183",
+          "latitude": "34.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "142560"
+            },
+            {
+              "-year": "96",
+              "#text": "151009"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-130",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Garden Grove",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "142965"
+            },
+            {
+              "-year": "96",
+              "#text": "149208"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-131",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Oceanside",
+          "longitude": "-117.417",
+          "latitude": "33.2333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "128090"
+            },
+            {
+              "-year": "96",
+              "#text": "145941"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-132",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Ontario",
+          "longitude": "-117.6",
+          "latitude": "34.05",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "133179"
+            },
+            {
+              "-year": "96",
+              "#text": "144854"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-137",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Moreno Valley",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "118779"
+            },
+            {
+              "-year": "96",
+              "#text": "140932"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-143",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Torrance",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "133107"
+            },
+            {
+              "-year": "96",
+              "#text": "136183"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-146",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Pomona",
+          "longitude": "-117.75",
+          "latitude": "34.05",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "131700"
+            },
+            {
+              "-year": "96",
+              "#text": "134706"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-147",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Pasadena",
+          "longitude": "-118.15",
+          "latitude": "34.15",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "131586"
+            },
+            {
+              "-year": "96",
+              "#text": "134116"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-153",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Irvine",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "110330"
+            },
+            {
+              "-year": "96",
+              "#text": "127873"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-155",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Sunnyvale",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "117324"
+            },
+            {
+              "-year": "96",
+              "#text": "125156"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-156",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Santa Clarita",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "120050"
+            },
+            {
+              "-year": "96",
+              "#text": "125153"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-162",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Santa Rosa",
+          "longitude": "-122.817",
+          "latitude": "38.5167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "113261"
+            },
+            {
+              "-year": "96",
+              "#text": "121879"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-163",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Hayward",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "114705"
+            },
+            {
+              "-year": "96",
+              "#text": "121631"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-164",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Fullerton",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "114144"
+            },
+            {
+              "-year": "96",
+              "#text": "120188"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-165",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Orange",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "110658"
+            },
+            {
+              "-year": "96",
+              "#text": "119890"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-169",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Rancho Cucamonga",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "101409"
+            },
+            {
+              "-year": "96",
+              "#text": "116613"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-171",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Escondido",
+          "longitude": "-117.083",
+          "latitude": "33.1167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108648"
+            },
+            {
+              "-year": "96",
+              "#text": "116184"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-172",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Lancaster",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "97300"
+            },
+            {
+              "-year": "96",
+              "#text": "115675"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-173",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Concord",
+          "longitude": "-121.983",
+          "latitude": "37.9667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "111308"
+            },
+            {
+              "-year": "96",
+              "#text": "114850"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-175",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Thousand Oaks",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "104381"
+            },
+            {
+              "-year": "96",
+              "#text": "113368"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-182",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Salinas",
+          "longitude": "-121.6",
+          "latitude": "36.6667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108777"
+            },
+            {
+              "-year": "96",
+              "#text": "111757"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-184",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Inglewood",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "109602"
+            },
+            {
+              "-year": "96",
+              "#text": "111040"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-189",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "El Monte",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "106162"
+            },
+            {
+              "-year": "96",
+              "#text": "110026"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-190",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Vallejo",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "109199"
+            },
+            {
+              "-year": "96",
+              "#text": "109593"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-196",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Simi Valley",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "100218"
+            },
+            {
+              "-year": "96",
+              "#text": "106974"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-197",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Palmdale",
+          "longitude": "-118.1",
+          "latitude": "34.6333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "70262"
+            },
+            {
+              "-year": "96",
+              "#text": "106540"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-204",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Fontana",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "87535"
+            },
+            {
+              "-year": "96",
+              "#text": "104124"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-207",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Berkeley",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "102724"
+            },
+            {
+              "-year": "96",
+              "#text": "103243"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-211",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "West Covina",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "96226"
+            },
+            {
+              "-year": "96",
+              "#text": "101526"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-215",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Costa Mesa",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "96357"
+            },
+            {
+              "-year": "96",
+              "#text": "100938"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-217",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Norwalk",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "94279"
+            },
+            {
+              "-year": "96",
+              "#text": "100209"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-218",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-6",
+          "name": "Corona",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "75943"
+            },
+            {
+              "-year": "96",
+              "#text": "100208"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-7",
+      "-capital": "cty-cid-cia-United-States-27",
+      "-country": "USA",
+      "name": "Colorado",
+      "area": "269595",
+      "population": "3892644",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-27",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-7",
+          "name": "Denver",
+          "longitude": "-104.867",
+          "latitude": "39.75",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "467610"
+            },
+            {
+              "-year": "96",
+              "#text": "497840"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-51",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-7",
+          "name": "Colorado Springs",
+          "longitude": "-104.717",
+          "latitude": "38.8167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "280430"
+            },
+            {
+              "-year": "96",
+              "#text": "345127"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-66",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-7",
+          "name": "Aurora",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "222103"
+            },
+            {
+              "-year": "96",
+              "#text": "252341"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-144",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-7",
+          "name": "Lakewood",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "126475"
+            },
+            {
+              "-year": "96",
+              "#text": "134999"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-203",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-7",
+          "name": "Fort Collins",
+          "longitude": "-105.083",
+          "latitude": "40.75",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "87491"
+            },
+            {
+              "-year": "96",
+              "#text": "104196"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-8",
+      "-capital": "cty-cid-cia-United-States-148",
+      "-country": "USA",
+      "name": "Connecticut",
+      "area": "12997",
+      "population": "3269858",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-140",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-8",
+          "name": "Bridgeport",
+          "longitude": "-73.1833",
+          "latitude": "41.1833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "141686"
+            },
+            {
+              "-year": "96",
+              "#text": "137990"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-148",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-8",
+          "name": "Hartford",
+          "longitude": "-72.4",
+          "latitude": "41.3",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "139739"
+            },
+            {
+              "-year": "96",
+              "#text": "133086"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Connecticut_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-157",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-8",
+          "name": "New Haven",
+          "longitude": "-73.9167",
+          "latitude": "41.3167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "130474"
+            },
+            {
+              "-year": "96",
+              "#text": "124665"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-188",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-8",
+          "name": "Stamford",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108056"
+            },
+            {
+              "-year": "96",
+              "#text": "110056"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-198",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-8",
+          "name": "Waterbury",
+          "longitude": "-73.0667",
+          "latitude": "41.5833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108961"
+            },
+            {
+              "-year": "96",
+              "#text": "106412"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-9",
+      "-capital": "cty-cid-cia-United-States-Dover",
+      "-country": "USA",
+      "name": "Delaware",
+      "area": "5297",
+      "population": "731581",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Dover",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-9",
+        "name": "Dover",
+        "longitude": "-75.4667",
+        "latitude": "39.1333",
+        "population": {
+          "-year": "87",
+          "#text": "23512"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-10",
+      "-capital": "cty-cid-cia-United-States-141",
+      "-country": "USA",
+      "name": "Florida",
+      "area": "151939",
+      "population": "14653945",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-15",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Jacksonville",
+          "longitude": "-81.7",
+          "latitude": "30.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "635230"
+            },
+            {
+              "-year": "96",
+              "#text": "679792"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-45",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Miami",
+          "longitude": "-80.1",
+          "latitude": "25.4",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "358648"
+            },
+            {
+              "-year": "96",
+              "#text": "365127"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-59",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Tampa",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "280015"
+            },
+            {
+              "-year": "96",
+              "#text": "285206"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-70",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "St. Petersburg",
+          "longitude": "-83.3333",
+          "latitude": "27.7667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "240318"
+            },
+            {
+              "-year": "96",
+              "#text": "235988"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-79",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Hialeah",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "188008"
+            },
+            {
+              "-year": "96",
+              "#text": "204684"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-109",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Orlando",
+          "longitude": "-81.3833",
+          "latitude": "28.55",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "164674"
+            },
+            {
+              "-year": "96",
+              "#text": "173902"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-124",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Fort Lauderdale",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "149238"
+            },
+            {
+              "-year": "96",
+              "#text": "151805"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-141",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Tallahassee",
+          "longitude": "-84.1",
+          "latitude": "30.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "124773"
+            },
+            {
+              "-year": "96",
+              "#text": "136812"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-152",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Hollywood",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "121720"
+            },
+            {
+              "-year": "96",
+              "#text": "127894"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-199",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Coral Springs",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "78864"
+            },
+            {
+              "-year": "96",
+              "#text": "105275"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-216",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Pembroke Pines",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "65566"
+            },
+            {
+              "-year": "96",
+              "#text": "100662"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-220",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-10",
+          "name": "Clearwater",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "98669"
+            },
+            {
+              "-year": "96",
+              "#text": "100132"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-11",
+      "-capital": "cty-cid-cia-United-States-39",
+      "-country": "USA",
+      "name": "Georgia",
+      "area": "152576",
+      "population": "7486242",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-39",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-11",
+          "name": "Atlanta",
+          "longitude": "-84.4",
+          "latitude": "33.4",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "393929"
+            },
+            {
+              "-year": "96",
+              "#text": "401907"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-100",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-11",
+          "name": "Columbus",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "178683"
+            },
+            {
+              "-year": "96",
+              "#text": "182828"
+            }
+          ]
+        },
+        {
+          "-id": "cty-Georgia-Savannah",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-11",
+          "name": "Savannah",
+          "longitude": "-81.2",
+          "latitude": "32.1333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "137812"
+            },
+            {
+              "-year": "96",
+              "#text": "136262"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-Georgia-Macon-AP",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-11",
+          "name": "Macon",
+          "longitude": "-83.65",
+          "latitude": "32.7",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "107365"
+            },
+            {
+              "-year": "96",
+              "#text": "113352"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-12",
+      "-capital": "cty-cid-cia-United-States-36",
+      "-country": "USA",
+      "name": "Hawaii",
+      "area": "16760",
+      "population": "1183723",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-36",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-12",
+          "name": "Honolulu",
+          "longitude": "-157.917",
+          "latitude": "21.3333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "377059"
+            },
+            {
+              "-year": "96",
+              "#text": "423475"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "stadt-Hilo-USA-HI",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-12",
+          "name": "Hilo",
+          "longitude": "-165",
+          "latitude": "20"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-13",
+      "-capital": "cty-cid-cia-United-States-121",
+      "-country": "USA",
+      "name": "Idaho",
+      "area": "216430",
+      "population": "1186602",
+      "city": {
+        "-id": "cty-cid-cia-United-States-121",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-13",
+        "name": "Boise",
+        "longitude": "-116.2",
+        "latitude": "43.4",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "126685"
+          },
+          {
+            "-year": "96",
+            "#text": "152737"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-14",
+      "-capital": "cty-cid-cia-United-States-178",
+      "-country": "USA",
+      "name": "Illinois",
+      "area": "145933",
+      "population": "11895849",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-4",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Chicago",
+          "longitude": "-87.6333",
+          "latitude": "41.8833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "2783726"
+            },
+            {
+              "-year": "96",
+              "#text": "2721547"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Michigan"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-133",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Rockford",
+          "longitude": "-89.05",
+          "latitude": "42.35",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "141787"
+            },
+            {
+              "-year": "96",
+              "#text": "143531"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-170",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Aurora",
+          "longitude": "-88.3333",
+          "latitude": "41.75",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "99672"
+            },
+            {
+              "-year": "96",
+              "#text": "116405"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-178",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Springfield",
+          "longitude": "-89.6667",
+          "latitude": "39.8333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "105417"
+            },
+            {
+              "-year": "96",
+              "#text": "112921"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-180",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Peoria",
+          "longitude": "-89.6833",
+          "latitude": "40.6667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "113513"
+            },
+            {
+              "-year": "96",
+              "#text": "112306"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-195",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-14",
+          "name": "Naperville",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "85806"
+            },
+            {
+              "-year": "96",
+              "#text": "107001"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-15",
+      "-capital": "cty-cid-cia-United-States-13",
+      "-country": "USA",
+      "name": "Indiana",
+      "area": "93719",
+      "population": "5864108",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-13",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-15",
+          "name": "Indianapolis",
+          "longitude": "-86.2833",
+          "latitude": "39.7333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "731278"
+            },
+            {
+              "-year": "96",
+              "#text": "746737"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-97",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-15",
+          "name": "Fort Wayne",
+          "longitude": "-85.2",
+          "latitude": "41",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "191839"
+            },
+            {
+              "-year": "96",
+              "#text": "184783"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-159",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-15",
+          "name": "Evansville",
+          "longitude": "-87.5333",
+          "latitude": "38.05",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "126272"
+            },
+            {
+              "-year": "96",
+              "#text": "123456"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ohio"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-185",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-15",
+          "name": "Gary",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "116646"
+            },
+            {
+              "-year": "96",
+              "#text": "110975"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Michigan"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-209",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-15",
+          "name": "South Bend",
+          "longitude": "-86.3167",
+          "latitude": "41.7",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "105511"
+            },
+            {
+              "-year": "96",
+              "#text": "102100"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-16",
+      "-capital": "cty-cid-cia-United-States-86",
+      "-country": "USA",
+      "name": "Iowa",
+      "area": "145752",
+      "population": "2852423",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-86",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-16",
+          "name": "Des Moines",
+          "longitude": "-93.65",
+          "latitude": "41.5333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "193189"
+            },
+            {
+              "-year": "96",
+              "#text": "193422"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-174",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-16",
+          "name": "Cedar Rapids",
+          "longitude": "-91.7",
+          "latitude": "41.8833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108772"
+            },
+            {
+              "-year": "96",
+              "#text": "113482"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-17",
+      "-capital": "cty-cid-cia-United-States-166",
+      "-country": "USA",
+      "name": "Kansas",
+      "area": "213097",
+      "population": "2594840",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-53",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-17",
+          "name": "Wichita",
+          "longitude": "-97.4167",
+          "latitude": "37.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "304017"
+            },
+            {
+              "-year": "96",
+              "#text": "320395"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Arkansas_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-136",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-17",
+          "name": "Kansas City",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "151521"
+            },
+            {
+              "-year": "96",
+              "#text": "142654"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-151",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-17",
+          "name": "Overland Park",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "111790"
+            },
+            {
+              "-year": "96",
+              "#text": "131053"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-166",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-17",
+          "name": "Topeka",
+          "longitude": "-95.6333",
+          "latitude": "39.0667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "119883"
+            },
+            {
+              "-year": "96",
+              "#text": "119658"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-18",
+      "-capital": "cty-cid-cia-United-States-Frankfort",
+      "-country": "USA",
+      "name": "Kentucky",
+      "area": "104661",
+      "population": "3908124",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-62",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-18",
+          "name": "Louisville",
+          "longitude": "-85.7333",
+          "latitude": "38.1833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "269555"
+            },
+            {
+              "-year": "96",
+              "#text": "260689"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ohio"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-69",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-18",
+          "name": "Lexington Fayette",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "225366"
+            },
+            {
+              "-year": "96",
+              "#text": "239942"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Frankfort",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-18",
+          "name": "Frankfort",
+          "longitude": "-84.5",
+          "latitude": "38.1",
+          "population": {
+            "-year": "87",
+            "#text": "25973"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-19",
+      "-capital": "cty-cid-cia-United-States-76",
+      "-country": "USA",
+      "name": "Louisiana",
+      "area": "123677",
+      "population": "4351769",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-30",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-19",
+          "name": "New Orleans",
+          "longitude": "-90.25",
+          "latitude": "29.9833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "496938"
+            },
+            {
+              "-year": "96",
+              "#text": "476625"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mississippi_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-76",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-19",
+          "name": "Baton Rouge",
+          "longitude": "-91.15",
+          "latitude": "30.5333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "219531"
+            },
+            {
+              "-year": "96",
+              "#text": "215882"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mississippi_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-90",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-19",
+          "name": "Shreveport",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "198525"
+            },
+            {
+              "-year": "96",
+              "#text": "191558"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-202",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-19",
+          "name": "Lafayette",
+          "longitude": "-92",
+          "latitude": "30.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "101852"
+            },
+            {
+              "-year": "96",
+              "#text": "104899"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-20",
+      "-capital": "cty-cid-cia-United-States-Augusta",
+      "-country": "USA",
+      "name": "Maine",
+      "area": "86156",
+      "population": "1242051",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Augusta",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-20",
+        "name": "Augusta",
+        "longitude": "-69.8",
+        "latitude": "44.3167",
+        "population": {
+          "-year": "87",
+          "#text": "21819"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-21",
+      "-capital": "cty-cid-cia-United-States-Annapolis",
+      "-country": "USA",
+      "name": "Maryland",
+      "area": "27091",
+      "population": "5094289",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-16",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-21",
+          "name": "Baltimore",
+          "longitude": "-76.4167",
+          "latitude": "39.3333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "736014"
+            },
+            {
+              "-year": "96",
+              "#text": "675401"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Annapolis",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-21",
+          "name": "Annapolis",
+          "longitude": "-76.3",
+          "latitude": "39",
+          "population": {
+            "-year": "87",
+            "#text": "31740"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-22",
+      "-capital": "cty-cid-cia-United-States-21",
+      "-country": "USA",
+      "name": "Massachusetts",
+      "area": "21455",
+      "population": "6117520",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-21",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-22",
+          "name": "Boston",
+          "longitude": "-71.0333",
+          "latitude": "42.3667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "574283"
+            },
+            {
+              "-year": "96",
+              "#text": "558394"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-115",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-22",
+          "name": "Worcester",
+          "longitude": "-71.8667",
+          "latitude": "42.2667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "169759"
+            },
+            {
+              "-year": "96",
+              "#text": "166350"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-128",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-22",
+          "name": "Springfield",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "156983"
+            },
+            {
+              "-year": "96",
+              "#text": "149948"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Connecticut_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-213",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-22",
+          "name": "Lowell",
+          "longitude": "-71.3167",
+          "latitude": "42.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "103439"
+            },
+            {
+              "-year": "96",
+              "#text": "100973"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-23",
+      "-capital": "cty-cid-cia-United-States-154",
+      "-country": "USA",
+      "name": "Michigan",
+      "area": "151584",
+      "population": "9773892",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-11",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Detroit",
+          "longitude": "-83.0167",
+          "latitude": "42.4167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "1027974"
+            },
+            {
+              "-year": "96",
+              "#text": "1000272"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-DetroitRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-94",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Grand Rapids",
+          "longitude": "-85.5167",
+          "latitude": "42.8833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "189126"
+            },
+            {
+              "-year": "96",
+              "#text": "188242"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-139",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Warren",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "144864"
+            },
+            {
+              "-year": "96",
+              "#text": "138078"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-145",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Flint",
+          "longitude": "-83.7333",
+          "latitude": "42.9667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "140925"
+            },
+            {
+              "-year": "96",
+              "#text": "134881"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-154",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Lansing",
+          "longitude": "-84.6",
+          "latitude": "42.7833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "127321"
+            },
+            {
+              "-year": "96",
+              "#text": "125736"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-167",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Sterling Heights",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "117810"
+            },
+            {
+              "-year": "96",
+              "#text": "118698"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-192",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Ann Arbor",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "109608"
+            },
+            {
+              "-year": "96",
+              "#text": "108758"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-201",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-23",
+          "name": "Livonia",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "100850"
+            },
+            {
+              "-year": "96",
+              "#text": "105099"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-24",
+      "-capital": "cty-cid-cia-United-States-63",
+      "-country": "USA",
+      "name": "Minnesota",
+      "area": "218600",
+      "population": "4685549",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-47",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-24",
+          "name": "Minneapolis",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "368383"
+            },
+            {
+              "-year": "96",
+              "#text": "358785"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mississippi_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-63",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-24",
+          "name": "St. Paul",
+          "longitude": "-93",
+          "latitude": "45",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "272235"
+            },
+            {
+              "-year": "96",
+              "#text": "259606"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mississippi_River"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-25",
+      "-capital": "cty-cid-cia-United-States-87",
+      "-country": "USA",
+      "name": "Mississippi",
+      "area": "123514",
+      "population": "2730501",
+      "city": {
+        "-id": "cty-cid-cia-United-States-87",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-25",
+        "name": "Jackson",
+        "longitude": "-90.0833",
+        "latitude": "32.3167",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "202062"
+          },
+          {
+            "-year": "96",
+            "#text": "192923"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-26",
+      "-capital": "cty-cid-cia-United-States-Jefferson-City",
+      "-country": "USA",
+      "name": "Missouri",
+      "area": "180514",
+      "population": "5402058",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-34",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-26",
+          "name": "Kansas City",
+          "longitude": "-94.5833",
+          "latitude": "39.1167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "434829"
+            },
+            {
+              "-year": "96",
+              "#text": "441259"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-48",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-26",
+          "name": "St. Louis",
+          "longitude": "-90.3833",
+          "latitude": "38.75",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "396685"
+            },
+            {
+              "-year": "96",
+              "#text": "351565"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River river-Mississippi_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-134",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-26",
+          "name": "Springfield",
+          "longitude": "-93.3833",
+          "latitude": "37.2333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "140494"
+            },
+            {
+              "-year": "96",
+              "#text": "143407"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-186",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-26",
+          "name": "Independence",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "112301"
+            },
+            {
+              "-year": "96",
+              "#text": "110303"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Jefferson-City",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-26",
+          "name": "Jefferson City",
+          "longitude": "-92.1833",
+          "latitude": "38.5667",
+          "population": {
+            "-year": "87",
+            "#text": "33619"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-27",
+      "-capital": "cty-cid-cia-United-States-Helena",
+      "-country": "USA",
+      "name": "Montana",
+      "area": "380848",
+      "population": "878810",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Helena",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-27",
+        "name": "Helena",
+        "longitude": "-112",
+        "latitude": "46.6",
+        "population": {
+          "-year": "87",
+          "#text": "23938"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-28",
+      "-capital": "cty-cid-cia-United-States-77",
+      "-country": "USA",
+      "name": "Nebraska",
+      "area": "200349",
+      "population": "1656870",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-46",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-28",
+          "name": "Omaha",
+          "longitude": "-95.9",
+          "latitude": "41.3",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "342862"
+            },
+            {
+              "-year": "96",
+              "#text": "364253"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-77",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-28",
+          "name": "Lincoln",
+          "longitude": "-96.4",
+          "latitude": "40.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "191972"
+            },
+            {
+              "-year": "96",
+              "#text": "209192"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-29",
+      "-capital": "cty-cid-cia-United-States-Carson-City",
+      "-country": "USA",
+      "name": "Nevada",
+      "area": "286352",
+      "population": "1676809",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-42",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-29",
+          "name": "Las Vegas",
+          "longitude": "-115.2",
+          "latitude": "36.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "258204"
+            },
+            {
+              "-year": "96",
+              "#text": "376906"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-119",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-29",
+          "name": "Reno",
+          "longitude": "-119.783",
+          "latitude": "39.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "133850"
+            },
+            {
+              "-year": "96",
+              "#text": "155499"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-TruckeeRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-161",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-29",
+          "name": "Henderson",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "64948"
+            },
+            {
+              "-year": "96",
+              "#text": "122339"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Carson-City",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-29",
+          "name": "Carson City",
+          "longitude": "-119.767",
+          "latitude": "39.1667",
+          "population": {
+            "-year": "87",
+            "#text": "32022"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-30",
+      "-capital": "cty-cid-cia-United-States-Concord",
+      "-country": "USA",
+      "name": "New Hampshire",
+      "area": "24033",
+      "population": "1172709",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-214",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-30",
+          "name": "Manchester",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "99332"
+            },
+            {
+              "-year": "96",
+              "#text": "100967"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Concord",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-30",
+          "name": "Concord",
+          "longitude": "-71.5",
+          "latitude": "43.2",
+          "population": {
+            "-year": "87",
+            "#text": "30400"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-MerrimackRiver"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-31",
+      "-capital": "cty-cid-cia-United-States-Trenton",
+      "-country": "USA",
+      "name": "New Jersey",
+      "area": "20168",
+      "population": "8052849",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-61",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-31",
+          "name": "Newark",
+          "longitude": "-74.1667",
+          "latitude": "40.7",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "275221"
+            },
+            {
+              "-year": "96",
+              "#text": "268510"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-73",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-31",
+          "name": "Jersey City",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "228517"
+            },
+            {
+              "-year": "96",
+              "#text": "229039"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hudson_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-127",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-31",
+          "name": "Paterson",
+          "longitude": "-74.15",
+          "latitude": "40.9",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "140891"
+            },
+            {
+              "-year": "96",
+              "#text": "150270"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-187",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-31",
+          "name": "Elizabeth",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "110002"
+            },
+            {
+              "-year": "96",
+              "#text": "110149"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Trenton",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-31",
+          "name": "Trenton",
+          "longitude": "-74.7667",
+          "latitude": "40.2167",
+          "population": {
+            "-year": "87",
+            "#text": "92124"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-32",
+      "-capital": "cty-cid-cia-United-States-Santa-Fe",
+      "-country": "USA",
+      "name": "New Mexico",
+      "area": "314925",
+      "population": "1729751",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-38",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-32",
+          "name": "Albuquerque",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "384915"
+            },
+            {
+              "-year": "96",
+              "#text": "419681"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rio_Grande"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Santa-Fe",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-32",
+          "name": "Santa Fe",
+          "longitude": "-106",
+          "latitude": "35.4",
+          "population": {
+            "-year": "87",
+            "#text": "48899"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-33",
+      "-capital": "cty-cid-cia-United-States-205",
+      "-country": "USA",
+      "name": "New York",
+      "area": "127189",
+      "population": "18137226",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-2",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "New York",
+          "longitude": "-74",
+          "latitude": "40.4",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "7322564"
+            },
+            {
+              "-year": "96",
+              "#text": "7380906"
+            }
+          ],
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Hudson_River"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-55",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "Buffalo",
+          "longitude": "-78.7333",
+          "latitude": "42.9333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "328175"
+            },
+            {
+              "-year": "96",
+              "#text": "310548"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Erie"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-74",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "Rochester",
+          "longitude": "-77.6667",
+          "latitude": "43.1167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "230356"
+            },
+            {
+              "-year": "96",
+              "#text": "221594"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Ontario"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-92",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "Yonkers",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "188082"
+            },
+            {
+              "-year": "96",
+              "#text": "190316"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hudson_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-118",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "Syracuse",
+          "longitude": "-76.1167",
+          "latitude": "43.1167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "163860"
+            },
+            {
+              "-year": "96",
+              "#text": "155865"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-205",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-33",
+          "name": "Albany",
+          "longitude": "-73.75",
+          "latitude": "42.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "100031"
+            },
+            {
+              "-year": "96",
+              "#text": "103564"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Hudson_River"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-34",
+      "-capital": "cty-cid-cia-United-States-68",
+      "-country": "USA",
+      "name": "North Carolina",
+      "area": "136412",
+      "population": "7425183",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-33",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-34",
+          "name": "Charlotte",
+          "longitude": "-80.9333",
+          "latitude": "35.2167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "419539"
+            },
+            {
+              "-year": "96",
+              "#text": "441297"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-68",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-34",
+          "name": "Raleigh",
+          "longitude": "-78.4",
+          "latitude": "35.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "212092"
+            },
+            {
+              "-year": "96",
+              "#text": "243835"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-84",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-34",
+          "name": "Greensboro",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "183894"
+            },
+            {
+              "-year": "96",
+              "#text": "195426"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-120",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-34",
+          "name": "Winston Salem",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "150958"
+            },
+            {
+              "-year": "96",
+              "#text": "153541"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-129",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-34",
+          "name": "Durham",
+          "longitude": "-78.7833",
+          "latitude": "35.8667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "138894"
+            },
+            {
+              "-year": "96",
+              "#text": "149799"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-35",
+      "-capital": "cty-cid-cia-United-States-Bismarck",
+      "-country": "USA",
+      "name": "North Dakota",
+      "area": "183117",
+      "population": "640883",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Bismarck",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-35",
+        "name": "Bismarck",
+        "longitude": "-100.5",
+        "latitude": "46.4",
+        "population": {
+          "-year": "87",
+          "#text": "44485"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Missouri_River"
+          },
+          {
+            "-watertype": "lake",
+            "-lake": "lake-LakeOahe"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-36",
+      "-capital": "cty-cid-cia-United-States-17",
+      "-country": "USA",
+      "name": "Ohio",
+      "area": "107044",
+      "population": "11186331",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-17",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Columbus",
+          "longitude": "-83",
+          "latitude": "39.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "632945"
+            },
+            {
+              "-year": "96",
+              "#text": "657053"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-26",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Cleveland",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "505616"
+            },
+            {
+              "-year": "96",
+              "#text": "498246"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Erie"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-50",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Cincinnati",
+          "longitude": "-84.5167",
+          "latitude": "39.15",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "364114"
+            },
+            {
+              "-year": "96",
+              "#text": "345818"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ohio"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-54",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Toledo",
+          "longitude": "-83.8",
+          "latitude": "41.6",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "332943"
+            },
+            {
+              "-year": "96",
+              "#text": "317606"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Erie"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-75",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Akron",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "223019"
+            },
+            {
+              "-year": "96",
+              "#text": "216882"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-110",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-36",
+          "name": "Dayton",
+          "longitude": "-84.2167",
+          "latitude": "39.9",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "182005"
+            },
+            {
+              "-year": "96",
+              "#text": "172947"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-37",
+      "-capital": "cty-cid-cia-United-States-31",
+      "-country": "USA",
+      "name": "Oklahoma",
+      "area": "181185",
+      "population": "3317091",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-31",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-37",
+          "name": "Oklahoma City",
+          "longitude": "-97.3",
+          "latitude": "35.3",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "444724"
+            },
+            {
+              "-year": "96",
+              "#text": "469852"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-41",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-37",
+          "name": "Tulsa",
+          "longitude": "-95.9",
+          "latitude": "36.2",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "367302"
+            },
+            {
+              "-year": "96",
+              "#text": "378491"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Arkansas_River"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-38",
+      "-capital": "cty-cid-cia-United-States-160",
+      "-country": "USA",
+      "name": "Oregon",
+      "area": "251418",
+      "population": "3243487",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-28",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-38",
+          "name": "Portland",
+          "longitude": "-122.6",
+          "latitude": "45.6",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "463634"
+            },
+            {
+              "-year": "96",
+              "#text": "480824"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Columbia_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-158",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-38",
+          "name": "Eugene",
+          "longitude": "-123.217",
+          "latitude": "44.1167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "112733"
+            },
+            {
+              "-year": "96",
+              "#text": "123718"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-160",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-38",
+          "name": "Salem",
+          "longitude": "-123.017",
+          "latitude": "44.9167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "107793"
+            },
+            {
+              "-year": "96",
+              "#text": "122566"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-39",
+      "-capital": "cty-cid-cia-United-States-Harrisburg",
+      "-country": "USA",
+      "name": "Pennsylvania",
+      "area": "117347",
+      "population": "12019661",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-6",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-39",
+          "name": "Philadelphia",
+          "longitude": "-75.25",
+          "latitude": "39.8833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "1585577"
+            },
+            {
+              "-year": "96",
+              "#text": "1478002"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-49",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-39",
+          "name": "Pittsburgh",
+          "longitude": "-80",
+          "latitude": "40.45",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "369879"
+            },
+            {
+              "-year": "96",
+              "#text": "350363"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Ohio"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-200",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-39",
+          "name": "Erie",
+          "longitude": "-80.1833",
+          "latitude": "42.0833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "108718"
+            },
+            {
+              "-year": "96",
+              "#text": "105270"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Erie"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-208",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-39",
+          "name": "Allentown",
+          "longitude": "-75.4333",
+          "latitude": "40.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "105301"
+            },
+            {
+              "-year": "96",
+              "#text": "102211"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Harrisburg",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-39",
+          "name": "Harrisburg",
+          "longitude": "-76.7667",
+          "latitude": "40.2",
+          "population": {
+            "-year": "87",
+            "#text": "53264"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-40",
+      "-capital": "cty-cid-cia-United-States-122",
+      "-country": "USA",
+      "name": "Rhode Island",
+      "area": "3139",
+      "population": "987429",
+      "city": {
+        "-id": "cty-cid-cia-United-States-122",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-40",
+        "name": "Providence",
+        "longitude": "-71.4333",
+        "latitude": "41.7333",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "160728"
+          },
+          {
+            "-year": "96",
+            "#text": "152558"
+          }
+        ],
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-41",
+      "-capital": "cty-cid-cia-United-States-179",
+      "-country": "USA",
+      "name": "South Carolina",
+      "area": "80582",
+      "population": "3760181",
+      "city": {
+        "-id": "cty-cid-cia-United-States-179",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-41",
+        "name": "Columbia",
+        "longitude": "-81.1167",
+        "latitude": "33.95",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "110734"
+          },
+          {
+            "-year": "96",
+            "#text": "112773"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-42",
+      "-capital": "cty-cid-cia-United-States-Pierre",
+      "-country": "USA",
+      "name": "South Dakota",
+      "area": "199730",
+      "population": "737973",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-177",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-42",
+          "name": "Sioux Falls",
+          "longitude": "-96.7333",
+          "latitude": "43.5667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "100836"
+            },
+            {
+              "-year": "96",
+              "#text": "113223"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Pierre",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-42",
+          "name": "Pierre",
+          "longitude": "-100.283",
+          "latitude": "44.3833",
+          "population": {
+            "-year": "87",
+            "#text": "11973"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Missouri_River"
+            },
+            {
+              "-watertype": "lake",
+              "-lake": "lake-LakeOahe"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-43",
+      "-capital": "cty-cid-cia-United-States-Nashville",
+      "-country": "USA",
+      "name": "Tennessee",
+      "area": "109153",
+      "population": "5368198",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-19",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-43",
+          "name": "Memphis",
+          "longitude": "-90",
+          "latitude": "35.05",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "618652"
+            },
+            {
+              "-year": "96",
+              "#text": "596725"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Mississippi_River"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-25",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-43",
+          "name": "Nashville Davidson",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "488366"
+            },
+            {
+              "-year": "96",
+              "#text": "511263"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-114",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-43",
+          "name": "Knoxville",
+          "longitude": "-83.9833",
+          "latitude": "35.8167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "169761"
+            },
+            {
+              "-year": "96",
+              "#text": "167535"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-TennesseeRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-126",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-43",
+          "name": "Chattanooga",
+          "longitude": "-85.2",
+          "latitude": "35.0333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "152393"
+            },
+            {
+              "-year": "96",
+              "#text": "150425"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-TennesseeRiver"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Nashville",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-43",
+          "name": "Nashville",
+          "longitude": "-86.4",
+          "latitude": "36.1",
+          "population": {
+            "-year": "87",
+            "#text": "455657"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-44",
+      "-capital": "cty-cid-cia-United-States-23",
+      "-country": "USA",
+      "name": "Texas",
+      "area": "691027",
+      "population": "19439337",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-5",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Houston",
+          "longitude": "-95.35",
+          "latitude": "29.9667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "1637859"
+            },
+            {
+              "-year": "96",
+              "#text": "1744058"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-9",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "San Antonio",
+          "longitude": "-98.4",
+          "latitude": "29.3",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "959295"
+            },
+            {
+              "-year": "96",
+              "#text": "1067816"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-10",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Dallas",
+          "longitude": "-96.85",
+          "latitude": "32.85",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "1007618"
+            },
+            {
+              "-year": "96",
+              "#text": "1053292"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-18",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "El Paso",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "515342"
+            },
+            {
+              "-year": "96",
+              "#text": "599865"
+            }
+          ],
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Rio_Grande"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-23",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Austin",
+          "longitude": "-97.7",
+          "latitude": "30.3",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "472020"
+            },
+            {
+              "-year": "96",
+              "#text": "541278"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-29",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Fort Worth",
+          "longitude": "-97.1",
+          "latitude": "32.4",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "447619"
+            },
+            {
+              "-year": "96",
+              "#text": "479716"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-57",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Arlington",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "261717"
+            },
+            {
+              "-year": "96",
+              "#text": "294816"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-60",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Corpus Christi",
+          "longitude": "-97.5",
+          "latitude": "27.7667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "257453"
+            },
+            {
+              "-year": "96",
+              "#text": "280260"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Golf_von_Mexiko"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-85",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Lubbock",
+          "longitude": "-101.817",
+          "latitude": "33.65",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "186206"
+            },
+            {
+              "-year": "96",
+              "#text": "193565"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-89",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Plano",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "127885"
+            },
+            {
+              "-year": "96",
+              "#text": "192280"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-93",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Garland",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "180635"
+            },
+            {
+              "-year": "96",
+              "#text": "190055"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-105",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Irving",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "155037"
+            },
+            {
+              "-year": "96",
+              "#text": "176993"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-113",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Amarillo",
+          "longitude": "-100.7",
+          "latitude": "35.2333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "157571"
+            },
+            {
+              "-year": "96",
+              "#text": "169588"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-116",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Laredo",
+          "longitude": "-99.45",
+          "latitude": "27.5333",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "122899"
+            },
+            {
+              "-year": "96",
+              "#text": "164899"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-149",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Brownsville",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "107027"
+            },
+            {
+              "-year": "96",
+              "#text": "132091"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-150",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Pasadena",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "119604"
+            },
+            {
+              "-year": "96",
+              "#text": "131620"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-181",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Mesquite",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "101484"
+            },
+            {
+              "-year": "96",
+              "#text": "111947"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-183",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Beaumont",
+          "longitude": "-94.0167",
+          "latitude": "29.95",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "114323"
+            },
+            {
+              "-year": "96",
+              "#text": "111224"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-191",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Grand Prairie",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "99606"
+            },
+            {
+              "-year": "96",
+              "#text": "109231"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-193",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Abilene",
+          "longitude": "-99.6833",
+          "latitude": "32.4167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "106707"
+            },
+            {
+              "-year": "96",
+              "#text": "108476"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-194",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Waco",
+          "longitude": "-97.2167",
+          "latitude": "31.6167",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "103590"
+            },
+            {
+              "-year": "96",
+              "#text": "108412"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-206",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "McAllen",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "84021"
+            },
+            {
+              "-year": "96",
+              "#text": "103352"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-219",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-44",
+          "name": "Wichita Falls",
+          "longitude": "-98.4833",
+          "latitude": "33.9667",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "96259"
+            },
+            {
+              "-year": "96",
+              "#text": "100138"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-45",
+      "-capital": "cty-cid-cia-United-States-111",
+      "-country": "USA",
+      "name": "Utah",
+      "area": "219888",
+      "population": "2059148",
+      "city": {
+        "-id": "cty-cid-cia-United-States-111",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-45",
+        "name": "Salt Lake City",
+        "longitude": "-112",
+        "latitude": "40.4",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "159928"
+          },
+          {
+            "-year": "96",
+            "#text": "172575"
+          }
+        ],
+        "located_at": {
+          "-watertype": "lake",
+          "-lake": "lake-Great_Salt_Lake"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-46",
+      "-capital": "cty-cid-cia-United-States-Montpelier",
+      "-country": "USA",
+      "name": "Vermont",
+      "area": "24900",
+      "population": "588978",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Montpelier",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-46",
+        "name": "Montpelier",
+        "longitude": "-72.4",
+        "latitude": "44.2",
+        "population": {
+          "-year": "87",
+          "#text": "8241"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-47",
+      "-capital": "cty-cid-cia-United-States-81",
+      "-country": "USA",
+      "name": "Virginia",
+      "area": "105586",
+      "population": "6733996",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-35",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Virginia Beach",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "393089"
+            },
+            {
+              "-year": "96",
+              "#text": "430385"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-71",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Norfolk",
+          "longitude": "-76.2",
+          "latitude": "36.9",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "261250"
+            },
+            {
+              "-year": "96",
+              "#text": "233430"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-81",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Richmond",
+          "longitude": "-77.3333",
+          "latitude": "37.5",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "202798"
+            },
+            {
+              "-year": "96",
+              "#text": "198267"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-88",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Chesapeake",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "151982"
+            },
+            {
+              "-year": "96",
+              "#text": "192342"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-106",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Newport News",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "171439"
+            },
+            {
+              "-year": "96",
+              "#text": "176122"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-108",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Arlington",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "170897"
+            },
+            {
+              "-year": "96",
+              "#text": "175334"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-138",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Hampton",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "133811"
+            },
+            {
+              "-year": "96",
+              "#text": "138757"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-168",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Alexandria",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "111182"
+            },
+            {
+              "-year": "96",
+              "#text": "117586"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-212",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-47",
+          "name": "Portsmouth",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "103910"
+            },
+            {
+              "-year": "96",
+              "#text": "101308"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-48",
+      "-capital": "cty-cid-cia-United-States-Olympia",
+      "-country": "USA",
+      "name": "Washington",
+      "area": "176479",
+      "population": "5610362",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-24",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-48",
+          "name": "Seattle",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "516259"
+            },
+            {
+              "-year": "96",
+              "#text": "524704"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-96",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-48",
+          "name": "Spokane",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "177165"
+            },
+            {
+              "-year": "96",
+              "#text": "186562"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-102",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-48",
+          "name": "Tacoma",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "176664"
+            },
+            {
+              "-year": "96",
+              "#text": "179114"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-Olympia",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-48",
+          "name": "Olympia",
+          "longitude": "-122.9",
+          "latitude": "46.9667",
+          "population": {
+            "-year": "87",
+            "#text": "27447"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-49",
+      "-capital": "cty-cid-cia-United-States-Charleston",
+      "-country": "USA",
+      "name": "West Virginia",
+      "area": "62761",
+      "population": "1815787",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Charleston",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-49",
+        "name": "Charleston",
+        "longitude": "-81.6",
+        "latitude": "38.3667",
+        "population": {
+          "-year": "87",
+          "#text": "63968"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-50",
+      "-capital": "cty-cid-cia-United-States-82",
+      "-country": "USA",
+      "name": "Wisconsin",
+      "area": "145436",
+      "population": "5169677",
+      "city": [
+        {
+          "-id": "cty-cid-cia-United-States-20",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-50",
+          "name": "Milwaukee",
+          "longitude": "-87.9",
+          "latitude": "42.95",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "628088"
+            },
+            {
+              "-year": "96",
+              "#text": "590503"
+            }
+          ],
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Lake_Michigan"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-United-States-82",
+          "-is_state_cap": "yes",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-50",
+          "name": "Madison",
+          "longitude": "-89.2",
+          "latitude": "43",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "190766"
+            },
+            {
+              "-year": "96",
+              "#text": "197630"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-United-States-210",
+          "-country": "USA",
+          "-province": "prov-cid-cia-United-States-50",
+          "name": "Green Bay",
+          "longitude": "-88.1333",
+          "latitude": "44.4833",
+          "population": [
+            {
+              "-year": "90",
+              "#text": "96466"
+            },
+            {
+              "-year": "96",
+              "#text": "102076"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-United-States-51",
+      "-capital": "cty-cid-cia-United-States-Cheyenne",
+      "-country": "USA",
+      "name": "Wyoming",
+      "area": "253324",
+      "population": "479743",
+      "city": {
+        "-id": "cty-cid-cia-United-States-Cheyenne",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-51",
+        "name": "Cheyenne",
+        "longitude": "-104.817",
+        "latitude": "41.15",
+        "population": {
+          "-year": "87",
+          "#text": "47283"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-United-States-52",
+      "-capital": "cty-cid-cia-United-States-22",
+      "-country": "USA",
+      "name": "Distr. Columbia",
+      "area": "179",
+      "population": "528964",
+      "city": {
+        "-id": "cty-cid-cia-United-States-22",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "USA",
+        "-province": "prov-cid-cia-United-States-52",
+        "name": "Washington",
+        "longitude": "-77",
+        "latitude": "38.5",
+        "population": [
+          {
+            "-year": "90",
+            "#text": "606900"
+          },
+          {
+            "-year": "96",
+            "#text": "543213"
+          }
+        ]
+      }
+    }
+  ]
+}
+{
+  "-car_code": "CAYM",
+  "-area": "262",
+  "-capital": "city-George-Town-CAYM-CAYM",
+  "-memberships": "org-Caricom org-CDB org-Interpol org-IOC org-UNESCO org-UPU org-WFTU",
+  "name": "Cayman Islands",
+  "population": "49035",
+  "population_growth": "2.39",
+  "infant_mortality": "6.94",
+  "gdp_total": "1939",
+  "gdp_agri": "1.4",
+  "gdp_ind": "3.2",
+  "gdp_serv": "95.4",
+  "inflation": "4.4",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "20",
+      "#text": "Black"
+    },
+    {
+      "-percentage": "20",
+      "#text": "White"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Mixed"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "11",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "5.7",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "26",
+      "#text": "Church of God"
+    },
+    {
+      "-percentage": "11.8",
+      "#text": "United Church"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "95",
+      "#text": "English"
+    },
+    {
+      "-percentage": "3.2",
+      "#text": "Spanish"
+    }
+  ],
+  "city": {
+    "-id": "city-George-Town-CAYM-CAYM",
+    "-is_country_cap": "yes",
+    "-country": "CAYM",
+    "name": "George Town",
+    "longitude": "-81.23",
+    "latitude": "19.18",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-GrandCayman" }
+  }
+}
+{
+  "-car_code": "CR",
+  "-area": "51100",
+  "-capital": "cty-cid-cia-Costa-Rica-2",
+  "-memberships": "org-OPANAL org-AG org-BCIE org-CACM org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Costa Rica",
+  "population": "3463083",
+  "population_growth": "2.06",
+  "infant_mortality": "13.5",
+  "gdp_total": "18400",
+  "inflation": "22.5",
+  "indep_date": "1821-09-15",
+  "government": "democratic republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Chinese"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "NIC",
+      "-length": "309"
+    },
+    {
+      "-country": "PA",
+      "-length": "330"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Costa-Rica-2",
+      "-capital": "cty-cid-cia-Costa-Rica-2",
+      "-country": "CR",
+      "name": "San Jose",
+      "area": "4960",
+      "population": "1163943",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-2",
+        "name": "San Jose",
+        "longitude": "-84.2",
+        "latitude": "10",
+        "population": {
+          "-year": "93",
+          "#text": "296600"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-3",
+      "-capital": "cty-cid-cia-Costa-Rica-3",
+      "-country": "CR",
+      "name": "Alajuela",
+      "area": "9753",
+      "population": "569984",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-3",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-3",
+        "name": "Alajuela",
+        "population": {
+          "-year": "93",
+          "#text": "158300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-4",
+      "-capital": "cty-cid-cia-Costa-Rica-4",
+      "-country": "CR",
+      "name": "Cartago",
+      "area": "3125",
+      "population": "359765",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-4",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-4",
+        "name": "Cartago",
+        "population": {
+          "-year": "93",
+          "#text": "109000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-5",
+      "-capital": "cty-cid-cia-Costa-Rica-7",
+      "-country": "CR",
+      "name": "Heredia",
+      "area": "2656",
+      "population": "256726",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-7",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-5",
+        "name": "Heredia",
+        "population": {
+          "-year": "93",
+          "#text": "67400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-6",
+      "-capital": "cty-cid-cia-Costa-Rica-8",
+      "-country": "CR",
+      "name": "Guanacaste",
+      "area": "10141",
+      "population": "254530",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-8",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-6",
+        "name": "Liberia",
+        "population": {
+          "-year": "93",
+          "#text": "36400"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-7",
+      "-capital": "cty-cid-cia-Costa-Rica-5",
+      "-country": "CR",
+      "name": "Puntarenas",
+      "area": "11277",
+      "population": "357103",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-5",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-7",
+        "name": "Puntarenas",
+        "longitude": "-84.8",
+        "latitude": "10",
+        "population": {
+          "-year": "93",
+          "#text": "92400"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Costa-Rica-8",
+      "-capital": "cty-cid-cia-Costa-Rica-6",
+      "-country": "CR",
+      "name": "Limon",
+      "area": "9189",
+      "population": "237183",
+      "city": {
+        "-id": "cty-cid-cia-Costa-Rica-6",
+        "-is_state_cap": "yes",
+        "-country": "CR",
+        "-province": "prov-cid-cia-Costa-Rica-8",
+        "name": "Limon",
+        "longitude": "-83",
+        "latitude": "10",
+        "population": {
+          "-year": "93",
+          "#text": "67800"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Caribbean"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "NIC",
+  "-area": "129494",
+  "-capital": "cty-Nicaragua-Managua",
+  "-memberships": "org-OPANAL org-BCIE org-CACM org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Nicaragua",
+  "population": "4272352",
+  "population_growth": "2.67",
+  "infant_mortality": "45.8",
+  "gdp_total": "7100",
+  "inflation": "11.4",
+  "indep_date": "1821-09-15",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "5",
+    "#text": "Indian"
+  },
+  "religions": [
+    {
+      "-percentage": "95",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Spanish"
+  },
+  "border": [
+    {
+      "-country": "CR",
+      "-length": "309"
+    },
+    {
+      "-country": "HCA",
+      "-length": "922"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Nicaragua-Managua",
+      "-is_country_cap": "yes",
+      "-country": "NIC",
+      "name": "Managua",
+      "longitude": "-86.25",
+      "latitude": "12.1667",
+      "population": {
+        "-year": "94",
+        "#text": "1195000"
+      },
+      "located_at": {
+        "-watertype": "lake",
+        "-lake": "lake-Managua"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nicaragua-3",
+      "-country": "NIC",
+      "name": "Leon",
+      "population": {
+        "-year": "94",
+        "#text": "171375"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nicaragua-4",
+      "-country": "NIC",
+      "name": "Chinandega",
+      "population": {
+        "-year": "94",
+        "#text": "101211"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nicaragua-5",
+      "-country": "NIC",
+      "name": "Masaya",
+      "population": {
+        "-year": "94",
+        "#text": "100646"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nicaragua-6",
+      "-country": "NIC",
+      "name": "Granada",
+      "longitude": "-86",
+      "latitude": "11.9",
+      "population": {
+        "-year": "94",
+        "#text": "79418"
+      },
+      "located_at": {
+        "-watertype": "lake",
+        "-lake": "lake-Nicaragua_See"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "PA",
+  "-area": "78200",
+  "-capital": "cty-Panama-Panama-City",
+  "-memberships": "org-OPANAL org-AG org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Panama",
+  "population": "2655094",
+  "population_growth": "1.64",
+  "infant_mortality": "29.7",
+  "gdp_total": "13600",
+  "gdp_agri": "10",
+  "gdp_ind": "16",
+  "gdp_serv": "74",
+  "inflation": "1.1",
+  "indep_date": "1903-11-03",
+  "government": "constitutional republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "14",
+      "#text": "West Indian"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Indian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "85",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "14",
+    "#text": "English"
+  },
+  "border": [
+    {
+      "-country": "CR",
+      "-length": "330"
+    },
+    {
+      "-country": "CO",
+      "-length": "225"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Panama-2",
+      "-capital": "cty-cid-cia-Panama-Bocas-del-Toro",
+      "-country": "PA",
+      "name": "Bocas del Toro",
+      "area": "8745",
+      "population": "88400",
+      "city": {
+        "-id": "cty-cid-cia-Panama-Bocas-del-Toro",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-2",
+        "name": "Bocas del Toro"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-3",
+      "-capital": "cty-cid-cia-Panama-7",
+      "-country": "PA",
+      "name": "Cocle",
+      "area": "4927",
+      "population": "177100",
+      "city": {
+        "-id": "cty-cid-cia-Panama-7",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-3",
+        "name": "Penonome",
+        "population": {
+          "-year": "90",
+          "#text": "61000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-4",
+      "-capital": "cty-cid-cia-Panama-4",
+      "-country": "PA",
+      "name": "Colon",
+      "area": "7247",
+      "population": "222600",
+      "city": {
+        "-id": "cty-cid-cia-Panama-4",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-4",
+        "name": "Colon",
+        "population": {
+          "-year": "90",
+          "#text": "140900"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-5",
+      "-capital": "cty-cid-cia-Panama-El-Porvenir",
+      "-country": "PA",
+      "name": "Comarca de San Blas",
+      "area": "2357",
+      "population": "0",
+      "city": {
+        "-id": "cty-cid-cia-Panama-El-Porvenir",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-5",
+        "name": "El Porvenir"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-6",
+      "-capital": "cty-cid-cia-Panama-5",
+      "-country": "PA",
+      "name": "Chiriqui",
+      "area": "8653",
+      "population": "396800",
+      "city": {
+        "-id": "cty-cid-cia-Panama-5",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-6",
+        "name": "David",
+        "population": {
+          "-year": "90",
+          "#text": "102700"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-7",
+      "-capital": "cty-cid-cia-Panama-La-Palma",
+      "-country": "PA",
+      "name": "Darien",
+      "area": "16671",
+      "population": "45000",
+      "city": {
+        "-id": "cty-cid-cia-Panama-La-Palma",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-7",
+        "name": "La Palma"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-8",
+      "-capital": "cty-cid-cia-Panama-8",
+      "-country": "PA",
+      "name": "Herrera",
+      "area": "2341",
+      "population": "108700",
+      "city": {
+        "-id": "cty-cid-cia-Panama-8",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-8",
+        "name": "Chitre",
+        "population": {
+          "-year": "90",
+          "#text": "34700"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-9",
+      "-capital": "cty-cid-cia-Panama-9",
+      "-country": "PA",
+      "name": "Los Santos",
+      "area": "3806",
+      "population": "82800",
+      "city": {
+        "-id": "cty-cid-cia-Panama-9",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-9",
+        "name": "Las Tablas",
+        "population": {
+          "-year": "90",
+          "#text": "21100"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Panama-10",
+      "-capital": "cty-Panama-Panama-City",
+      "-country": "PA",
+      "name": "Panama",
+      "area": "11887",
+      "population": "1168500",
+      "city": [
+        {
+          "-id": "cty-Panama-Panama-City",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "PA",
+          "-province": "prov-cid-cia-Panama-10",
+          "name": "Panama City",
+          "longitude": "-79.55",
+          "latitude": "8.96667",
+          "population": {
+            "-year": "90",
+            "#text": "594800"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Panama-3",
+          "-country": "PA",
+          "-province": "prov-cid-cia-Panama-10",
+          "name": "San Miguelito",
+          "population": {
+            "-year": "90",
+            "#text": "243000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Panama-11",
+      "-capital": "cty-cid-cia-Panama-6",
+      "-country": "PA",
+      "name": "Veraguas",
+      "area": "11239",
+      "population": "224700",
+      "city": {
+        "-id": "cty-cid-cia-Panama-6",
+        "-is_state_cap": "yes",
+        "-country": "PA",
+        "-province": "prov-cid-cia-Panama-11",
+        "name": "Santiago",
+        "population": {
+          "-year": "90",
+          "#text": "61000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "C",
+  "-area": "110860",
+  "-capital": "cty-Cuba-Havana",
+  "-memberships": "org-CCC org-ECLAC org-FAO org-G-77 org-IAEA org-ICAO org-Interpol org-IFRCS org-IFAD org-ILO org-IMO org-Inmarsat org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNOMIG org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Cuba",
+  "population": "10951334",
+  "population_growth": "0.44",
+  "infant_mortality": "9",
+  "gdp_total": "14700",
+  "gdp_agri": "7",
+  "gdp_ind": "30",
+  "gdp_serv": "63",
+  "indep_date": "1902-05-20",
+  "government": "Communist state",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "1",
+    "#text": "Chinese"
+  },
+  "religions": {
+    "-percentage": "85",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Spanish"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-Cuba-2",
+      "-capital": "cty-cid-cia-Cuba-4",
+      "-country": "C",
+      "name": "Camaguey",
+      "area": "15990",
+      "population": "723000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-4",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-2",
+        "name": "Camaguey",
+        "population": {
+          "-year": "94",
+          "#text": "293961"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-3",
+      "-capital": "cty-cid-cia-Cuba-Ciego-de-Avila",
+      "-country": "C",
+      "name": "Ciego de Avila",
+      "area": "6910",
+      "population": "353000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-Ciego-de-Avila",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-3",
+        "name": "Ciego de Avila",
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-4",
+      "-capital": "cty-cid-cia-Cuba-9",
+      "-country": "C",
+      "name": "Cienfuegos",
+      "area": "4178",
+      "population": "354000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-9",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-4",
+        "name": "Cienfuegos",
+        "population": {
+          "-year": "94",
+          "#text": "132038"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-5",
+      "-capital": "cty-Cuba-Havana",
+      "-country": "C",
+      "name": "Ciudad de la Habana",
+      "area": "727",
+      "population": "2059000",
+      "city": {
+        "-id": "cty-Cuba-Havana",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-5",
+        "name": "Havana",
+        "longitude": "-82.35",
+        "latitude": "23.1333",
+        "population": {
+          "-year": "94",
+          "#text": "2241000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Golf_von_Mexiko"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-6",
+      "-capital": "cty-cid-cia-Cuba-8",
+      "-country": "C",
+      "name": "Granma",
+      "area": "8372",
+      "population": "773000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Cuba-8",
+          "-is_state_cap": "yes",
+          "-country": "C",
+          "-province": "prov-cid-cia-Cuba-6",
+          "name": "Bayamo",
+          "population": {
+            "-year": "94",
+            "#text": "137663"
+          },
+          "located_on": { "-island": "island-Cuba" }
+        },
+        {
+          "-id": "cty-cid-cia-Cuba-13",
+          "-country": "C",
+          "-province": "prov-cid-cia-Cuba-6",
+          "name": "Manzanillo",
+          "population": {
+            "-year": "94",
+            "#text": "107650"
+          },
+          "located_on": { "-island": "island-Cuba" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-7",
+      "-capital": "cty-cid-cia-Cuba-6",
+      "-country": "C",
+      "name": "Guantanamo",
+      "area": "6186",
+      "population": "484000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-6",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-7",
+        "name": "Guantanamo",
+        "population": {
+          "-year": "94",
+          "#text": "207796"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-8",
+      "-capital": "cty-cid-cia-Cuba-5",
+      "-country": "C",
+      "name": "Holguin",
+      "area": "9301",
+      "population": "972000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-5",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-8",
+        "name": "Holguin",
+        "population": {
+          "-year": "94",
+          "#text": "242085"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-9",
+      "-capital": "cty-Cuba-Havana",
+      "-country": "C",
+      "name": "La Habana",
+      "area": "5731",
+      "population": "630000"
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-10",
+      "-capital": "cty-cid-cia-Cuba-11",
+      "-country": "C",
+      "name": "Las Tunas",
+      "area": "6589",
+      "population": "478000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-11",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-10",
+        "name": "Victoria de las Tunas",
+        "population": {
+          "-year": "94",
+          "#text": "126930"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-11",
+      "-capital": "cty-cid-cia-Cuba-12",
+      "-country": "C",
+      "name": "Matanzas",
+      "area": "11978",
+      "population": "596000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-12",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-11",
+        "name": "Matanzas",
+        "population": {
+          "-year": "94",
+          "#text": "123843"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-12",
+      "-capital": "cty-cid-cia-Cuba-10",
+      "-country": "C",
+      "name": "Pinar del Rio",
+      "area": "10925",
+      "population": "678000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-10",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-12",
+        "name": "Pinar del Rio",
+        "population": {
+          "-year": "94",
+          "#text": "128570"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-13",
+      "-capital": "cty-cid-cia-Cuba-Sancti-Spiritus",
+      "-country": "C",
+      "name": "Sancti Spiritus",
+      "area": "6744",
+      "population": "420000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-Sancti-Spiritus",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-13",
+        "name": "Sancti Spiritus",
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-14",
+      "-capital": "cty-cid-cia-Cuba-3",
+      "-country": "C",
+      "name": "Santiago de Cuba",
+      "area": "6170",
+      "population": "968000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-3",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-14",
+        "name": "Santiago de Cuba",
+        "population": {
+          "-year": "94",
+          "#text": "440084"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-15",
+      "-capital": "cty-cid-cia-Cuba-7",
+      "-country": "C",
+      "name": "Villa Clara",
+      "area": "8662",
+      "population": "796000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-7",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-15",
+        "name": "Santa Clara",
+        "population": {
+          "-year": "94",
+          "#text": "205400"
+        },
+        "located_on": { "-island": "island-Cuba" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cuba-16",
+      "-capital": "cty-cid-cia-Cuba-Nueva-Gerona",
+      "-country": "C",
+      "name": "Isla de la Juventud",
+      "area": "2396",
+      "population": "71000",
+      "city": {
+        "-id": "cty-cid-cia-Cuba-Nueva-Gerona",
+        "-is_state_cap": "yes",
+        "-country": "C",
+        "-province": "prov-cid-cia-Cuba-16",
+        "name": "Nueva Gerona",
+        "located_on": { "-island": "island-Cuba" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "WD",
+  "-area": "746",
+  "-capital": "cty-cid-cia-Dominica-Roseau",
+  "-memberships": "org-ACP org-ACCT org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-NAM org-OAS org-OECS org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WMO org-WTrO",
+  "name": "Dominica",
+  "population": "82926",
+  "population_growth": "0.38",
+  "infant_mortality": "9.6",
+  "gdp_total": "200",
+  "gdp_agri": "26",
+  "inflation": "0.4",
+  "indep_date": "1978-11-03",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Carib Indians"
+  },
+  "religions": [
+    {
+      "-percentage": "77",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Protestant"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Dominica-Roseau",
+    "-is_country_cap": "yes",
+    "-country": "WD",
+    "name": "Roseau",
+    "longitude": "-61.3",
+    "latitude": "15.2",
+    "population": {
+      "-year": "87",
+      "#text": "11000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Dominica" }
+  }
+}
+{
+  "-car_code": "DOM",
+  "-area": "48730",
+  "-capital": "cty-cid-cia-Dominican-Republic-Santo-Domingo",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-ECLAC org-FAO org-G-11 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Dominican Republic",
+  "population": "8088881",
+  "population_growth": "1.73",
+  "infant_mortality": "47.7",
+  "gdp_total": "26800",
+  "gdp_agri": "13",
+  "gdp_ind": "32",
+  "gdp_serv": "55",
+  "inflation": "9.5",
+  "indep_date": "1844-02-27",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Spanish"
+  },
+  "border": {
+    "-country": "RH",
+    "-length": "275"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Dominican-Republic-Santo-Domingo",
+    "-is_country_cap": "yes",
+    "-country": "DOM",
+    "name": "Santo Domingo",
+    "longitude": "-69.9",
+    "latitude": "18.4833",
+    "population": {
+      "-year": "87",
+      "#text": "1400000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Hispaniola" }
+  }
+}
+{
+  "-car_code": "RH",
+  "-area": "27750",
+  "-capital": "cty-cid-cia-Haiti-Port-au-Prince",
+  "-memberships": "org-ACP org-ACCT org-OPANAL org-Caricom org-CCC org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Haiti",
+  "population": "6731539",
+  "population_growth": "1.77",
+  "infant_mortality": "103.8",
+  "gdp_total": "6500",
+  "gdp_agri": "34.8",
+  "gdp_ind": "23",
+  "gdp_serv": "42.2",
+  "inflation": "14.5",
+  "indep_date": "1804-01-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "5",
+    "#text": "European"
+  },
+  "religions": [
+    {
+      "-percentage": "80",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "16",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "10",
+    "#text": "French"
+  },
+  "border": {
+    "-country": "DOM",
+    "-length": "275"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Haiti-Port-au-Prince",
+    "-is_country_cap": "yes",
+    "-country": "RH",
+    "name": "Port-au-Prince",
+    "longitude": "-72.2",
+    "latitude": "18.4",
+    "population": {
+      "-year": "87",
+      "#text": "800000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Hispaniola" }
+  }
+}
+{
+  "-car_code": "ES",
+  "-area": "21040",
+  "-capital": "cty-cid-cia-El-Salvador-2",
+  "-memberships": "org-OPANAL org-BCIE org-CACM org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-MINURSO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "El Salvador",
+  "population": "5828987",
+  "population_growth": "1.81",
+  "infant_mortality": "31.9",
+  "gdp_total": "11400",
+  "inflation": "11.4",
+  "indep_date": "1821-09-15",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "5",
+    "#text": "Indian"
+  },
+  "religions": {
+    "-percentage": "75",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "GCA",
+      "-length": "203"
+    },
+    {
+      "-country": "HCA",
+      "-length": "342"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-El-Salvador-7",
+      "-country": "ES",
+      "name": "Nueva San Salvador",
+      "population": {
+        "-year": "92",
+        "#text": "116575"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-2",
+      "-is_country_cap": "yes",
+      "-country": "ES",
+      "name": "San Salvador",
+      "longitude": "-89.2167",
+      "latitude": "13.7",
+      "population": {
+        "-year": "92",
+        "#text": "422570"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-3",
+      "-country": "ES",
+      "name": "Soyapango",
+      "population": {
+        "-year": "92",
+        "#text": "251811"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-6",
+      "-country": "ES",
+      "name": "Mejicanos",
+      "population": {
+        "-year": "92",
+        "#text": "145000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-8",
+      "-country": "ES",
+      "name": "Delgado",
+      "population": {
+        "-year": "92",
+        "#text": "104790"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-9",
+      "-country": "ES",
+      "name": "Apopa",
+      "population": {
+        "-year": "92",
+        "#text": "100763"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-El-Salvador-4",
+      "-country": "ES",
+      "name": "Santa Ana",
+      "population": {
+        "-year": "92",
+        "#text": "202337"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "HCA",
+  "-area": "112090",
+  "-capital": "cty-Honduras-Tegucigalpa",
+  "-memberships": "org-OPANAL org-BCIE org-CACM org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-MINURSO org-UNMIH org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Honduras",
+  "population": "5605193",
+  "population_growth": "2.6",
+  "infant_mortality": "41.8",
+  "gdp_total": "10800",
+  "gdp_agri": "30",
+  "gdp_ind": "20",
+  "gdp_serv": "50",
+  "inflation": "30",
+  "indep_date": "1821-09-15",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "7",
+    "#text": "Indian"
+  },
+  "religions": {
+    "-percentage": "97",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "GCA",
+      "-length": "256"
+    },
+    {
+      "-country": "NIC",
+      "-length": "922"
+    },
+    {
+      "-country": "ES",
+      "-length": "342"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Honduras-2",
+      "-capital": "cty-cid-cia-Honduras-4",
+      "-country": "HCA",
+      "name": "Atlantida",
+      "area": "4251",
+      "population": "263700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Honduras-4",
+          "-is_state_cap": "yes",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-2",
+          "name": "La Ceiba",
+          "population": {
+            "-year": "91",
+            "#text": "77100"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Caribbean"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-11",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-2",
+          "name": "Tela",
+          "population": {
+            "-year": "91",
+            "#text": "24000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Caribbean"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-3",
+      "-capital": "cty-cid-cia-Honduras-Trujillo",
+      "-country": "HCA",
+      "name": "Colon",
+      "area": "8875",
+      "population": "167900",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Trujillo",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-3",
+        "name": "Trujillo",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Caribbean"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-4",
+      "-capital": "cty-cid-cia-Honduras-7",
+      "-country": "HCA",
+      "name": "Comayagua",
+      "area": "5196",
+      "population": "267000",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-7",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-4",
+        "name": "Comayagua",
+        "population": {
+          "-year": "91",
+          "#text": "44300"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-5",
+      "-capital": "cty-cid-cia-Honduras-13",
+      "-country": "HCA",
+      "name": "Copan",
+      "area": "3203",
+      "population": "241400",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-13",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-5",
+        "name": "Santa Rosa de Copan",
+        "population": {
+          "-year": "91",
+          "#text": "21500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-6",
+      "-capital": "cty-cid-cia-Honduras-3",
+      "-country": "HCA",
+      "name": "Cortes",
+      "area": "3954",
+      "population": "732600",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Honduras-3",
+          "-is_state_cap": "yes",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-6",
+          "name": "San Pedro Sula",
+          "population": {
+            "-year": "91",
+            "#text": "325900"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-9",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-6",
+          "name": "Puerto Cortes",
+          "population": {
+            "-year": "91",
+            "#text": "32500"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Caribbean"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-7",
+      "-capital": "cty-cid-cia-Honduras-6",
+      "-country": "HCA",
+      "name": "Choluteca",
+      "area": "4211",
+      "population": "326100",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-6",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-7",
+        "name": "Choluteca",
+        "population": {
+          "-year": "91",
+          "#text": "63200"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-8",
+      "-capital": "cty-cid-cia-Honduras-Yuscaran",
+      "-country": "HCA",
+      "name": "El Paraiso",
+      "area": "7218",
+      "population": "284100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Honduras-8",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-8",
+          "name": "Danli",
+          "population": {
+            "-year": "91",
+            "#text": "35500"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-Yuscaran",
+          "-is_state_cap": "yes",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-8",
+          "name": "Yuscaran"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-9",
+      "-capital": "cty-Honduras-Tegucigalpa",
+      "-country": "HCA",
+      "name": "Francisco Morazan",
+      "area": "7946",
+      "population": "908300",
+      "city": [
+        {
+          "-id": "cty-Honduras-Tegucigalpa",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-9",
+          "name": "Tegucigalpa",
+          "longitude": "-87.2167",
+          "latitude": "14.1",
+          "population": {
+            "-year": "91",
+            "#text": "670100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-10",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-9",
+          "name": "Siguatepeque",
+          "population": {
+            "-year": "91",
+            "#text": "32200"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-10",
+      "-capital": "cty-cid-cia-Honduras-Puerto-Lempira",
+      "-country": "HCA",
+      "name": "Gracias a Dios",
+      "area": "16630",
+      "population": "39000",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Puerto-Lempira",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-10",
+        "name": "Puerto Lempira",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Caribbean"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-11",
+      "-capital": "cty-cid-cia-Honduras-La-Esperanza",
+      "-country": "HCA",
+      "name": "Intibuca",
+      "area": "3072",
+      "population": "137800",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-La-Esperanza",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-11",
+        "name": "La Esperanza"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-12",
+      "-capital": "cty-cid-cia-Honduras-Roatan",
+      "-country": "HCA",
+      "name": "Islas de la Bahia",
+      "area": "261",
+      "population": "24500",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Roatan",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-12",
+        "name": "Roatan"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-13",
+      "-capital": "cty-cid-cia-Honduras-La-Paz",
+      "-country": "HCA",
+      "name": "La Paz",
+      "area": "2331",
+      "population": "117400",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-La-Paz",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-13",
+        "name": "La Paz"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-14",
+      "-capital": "cty-cid-cia-Honduras-Gracias",
+      "-country": "HCA",
+      "name": "Lempira",
+      "area": "4290",
+      "population": "194600",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Gracias",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-14",
+        "name": "Gracias"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-15",
+      "-capital": "cty-cid-cia-Honduras-Ocotepeque",
+      "-country": "HCA",
+      "name": "Ocotepeque",
+      "area": "1680",
+      "population": "81800",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Ocotepeque",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-15",
+        "name": "Ocotepeque"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-16",
+      "-capital": "cty-cid-cia-Honduras-12",
+      "-country": "HCA",
+      "name": "Olancho",
+      "area": "24351",
+      "population": "318000",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-12",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-16",
+        "name": "Jutigalpa",
+        "population": {
+          "-year": "91",
+          "#text": "22500"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-17",
+      "-capital": "cty-cid-cia-Honduras-Santa-Barbara",
+      "-country": "HCA",
+      "name": "Santa Barbara",
+      "area": "5113",
+      "population": "307500",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Santa-Barbara",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-17",
+        "name": "Santa Barbara"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-18",
+      "-capital": "cty-cid-cia-Honduras-Nacaome",
+      "-country": "HCA",
+      "name": "Valle",
+      "area": "1565",
+      "population": "130900",
+      "city": {
+        "-id": "cty-cid-cia-Honduras-Nacaome",
+        "-is_state_cap": "yes",
+        "-country": "HCA",
+        "-province": "prov-cid-cia-Honduras-18",
+        "name": "Nacaome"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Honduras-19",
+      "-capital": "cty-cid-cia-Honduras-Yoro",
+      "-country": "HCA",
+      "name": "Yoro",
+      "area": "7939",
+      "population": "370700",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Honduras-5",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-19",
+          "name": "El Progreso",
+          "population": {
+            "-year": "91",
+            "#text": "70000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-14",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-19",
+          "name": "Olanchito",
+          "population": {
+            "-year": "91",
+            "#text": "15900"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Honduras-Yoro",
+          "-is_state_cap": "yes",
+          "-country": "HCA",
+          "-province": "prov-cid-cia-Honduras-19",
+          "name": "Yoro"
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "GROX",
+  "-area": "2175600",
+  "-capital": "cty-cid-cia-Greenland-Nuuk",
+  "name": "Greenland",
+  "population": "58203",
+  "population_growth": "1",
+  "infant_mortality": "23.8",
+  "gdp_total": "892",
+  "inflation": "1.3",
+  "dependent": { "-country": "DK" },
+  "government": "part of the Danish realm",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "14",
+      "#text": "Danish"
+    },
+    {
+      "-percentage": "86",
+      "#text": "Greenlander"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Protestant"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Greenland-Nuuk",
+    "-is_country_cap": "yes",
+    "-country": "GROX",
+    "name": "Nuuk",
+    "longitude": "-51.7",
+    "latitude": "64.2",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Greenland" }
+  }
+}
+{
+  "-car_code": "WG",
+  "-area": "344",
+  "-capital": "cty-cid-cia-Grenada-Saint-Georges",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ISO org-ICRM org-ITU org-LAES org-NAM org-OAS org-OECS org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WToO",
+  "name": "Grenada",
+  "population": "94961",
+  "population_growth": "0.55",
+  "infant_mortality": "11.9",
+  "gdp_total": "284",
+  "gdp_agri": "10.2",
+  "gdp_ind": "40.3",
+  "gdp_serv": "49.5",
+  "inflation": "3",
+  "indep_date": "1974-02-07",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "African"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Grenada-Saint-Georges",
+    "-is_country_cap": "yes",
+    "-country": "WG",
+    "name": "Saint Georges",
+    "longitude": "-61.4",
+    "latitude": "12.4",
+    "population": {
+      "-year": "87",
+      "#text": "30000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Grenada" }
+  }
+}
+{
+  "-car_code": "GUAD",
+  "-area": "1780",
+  "-capital": "cty-cid-cia-GUAD-Basse-Terre",
+  "-memberships": "org-WCL org-WFTU",
+  "name": "Guadeloupe",
+  "population": "407768",
+  "population_growth": "1.2",
+  "infant_mortality": "8.3",
+  "gdp_total": "3700",
+  "gdp_agri": "6",
+  "gdp_ind": "9",
+  "gdp_serv": "85",
+  "inflation": "3.7",
+  "dependent": { "-country": "F" },
+  "government": "overseas department of France",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "5",
+    "#text": "Chinese"
+  },
+  "religions": [
+    {
+      "-percentage": "95",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Hindu"
+    }
+  ],
+  "languages": {
+    "-percentage": "99",
+    "#text": "French"
+  },
+  "city": {
+    "-id": "cty-cid-cia-GUAD-Basse-Terre",
+    "-is_country_cap": "yes",
+    "-country": "GUAD",
+    "name": "Basse-Terre",
+    "longitude": "-61.8",
+    "latitude": "16.15",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Basse-Terre" }
+  }
+}
+{
+  "-car_code": "JA",
+  "-area": "10991",
+  "-capital": "cty-cid-cia-Jamaica-Kingston",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-CCC org-ECLAC org-FAO org-G-15 org-G-19 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNITAR org-UNMIH org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Jamaica",
+  "population": "2595275",
+  "population_growth": "0.8",
+  "infant_mortality": "15.6",
+  "gdp_total": "8200",
+  "gdp_agri": "7.9",
+  "gdp_ind": "28.1",
+  "gdp_serv": "64",
+  "inflation": "25.5",
+  "indep_date": "1962-08-06",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "76.3",
+      "#text": "African"
+    },
+    {
+      "-percentage": "15.1",
+      "#text": "Afro-European"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Afro-East Indian"
+    },
+    {
+      "-percentage": "1.2",
+      "#text": "Afro-Chinese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "5",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "55.9",
+      "#text": "Protestant"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Jamaica-Kingston",
+    "-is_country_cap": "yes",
+    "-country": "JA",
+    "name": "Kingston",
+    "longitude": "-76.5",
+    "latitude": "18",
+    "population": {
+      "-year": "87",
+      "#text": "101000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Jamaica" }
+  }
+}
+{
+  "-car_code": "MART",
+  "-area": "1128",
+  "-capital": "cty-cid-cia-MART-Fort-de-France",
+  "-memberships": "org-WCL org-WFTU",
+  "name": "Martinique",
+  "population": "399151",
+  "population_growth": "1.1",
+  "infant_mortality": "7.1",
+  "gdp_total": "3950",
+  "gdp_agri": "6",
+  "gdp_ind": "11",
+  "gdp_serv": "83",
+  "inflation": "3.9",
+  "dependent": { "-country": "F" },
+  "government": "overseas department of France",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "5",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "90",
+      "#text": "African-white-Indian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "95",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Hindu"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-MART-Fort-de-France",
+    "-is_country_cap": "yes",
+    "-country": "MART",
+    "name": "Fort-de-France",
+    "longitude": "-61.0",
+    "latitude": "14.6",
+    "population": {
+      "-year": "09",
+      "#text": "100000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Martinique" }
+  }
+}
+{
+  "-car_code": "MNTS",
+  "-area": "102",
+  "-capital": "city-Plymouth-MNTS-MNTS",
+  "-memberships": "org-Caricom org-CDB org-Interpol org-OECS org-UPU org-WFTU",
+  "name": "Montserrat",
+  "population": "5097",
+  "population_growth": "0.39",
+  "infant_mortality": "16.08",
+  "gdp_total": "29",
+  "gdp_agri": "1.2",
+  "gdp_ind": "23.1",
+  "gdp_serv": "75.7",
+  "inflation": "2.6",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-Plymouth-MNTS-MNTS",
+    "-is_country_cap": "yes",
+    "-country": "MNTS",
+    "name": "Plymouth",
+    "longitude": "-62.13",
+    "latitude": "16.42",
+    "population": {
+      "-year": "09",
+      "#text": "0"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Montserrat" }
+  }
+}
+{
+  "-car_code": "NA",
+  "-area": "960",
+  "-capital": "city-Willemstad-NA",
+  "-memberships": "org-Caricom org-ILO org-IMF org-Interpol org-IOC org-UNESCO org-UPU org-WCL org-WMO org-WToO",
+  "name": "Netherlands Antilles",
+  "population": "227049",
+  "population_growth": "0.73",
+  "infant_mortality": "9.09",
+  "gdp_total": "2800",
+  "gdp_agri": "1",
+  "gdp_ind": "15",
+  "gdp_serv": "84",
+  "inflation": "2.1",
+  "dependent": { "-country": "NL" },
+  "government": "part of the Dutch realm",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "85",
+    "#text": "Mixed Black"
+  },
+  "languages": [
+    {
+      "-percentage": "65.4",
+      "#text": "Papiamento"
+    },
+    {
+      "-percentage": "15.9",
+      "#text": "English"
+    },
+    {
+      "-percentage": "7.3",
+      "#text": "Dutch"
+    },
+    {
+      "-percentage": "6.1",
+      "#text": "Spanish"
+    },
+    {
+      "-percentage": "1.6",
+      "#text": "Creole"
+    }
+  ],
+  "border": {
+    "-country": "SMAR",
+    "-length": "15"
+  },
+  "city": {
+    "-id": "city-Willemstad-NA",
+    "-is_country_cap": "yes",
+    "-country": "NA",
+    "name": "Willemstad",
+    "longitude": "-68.9",
+    "latitude": "12.1",
+    "population": {
+      "-year": "09",
+      "#text": "125000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Curacao" }
+  }
+}
+{
+  "-car_code": "SMAR",
+  "-area": "54",
+  "-capital": "city-Marigot-SMAR",
+  "-memberships": "org-UPU org-WFTU",
+  "name": "Saint Martin",
+  "population": "29820",
+  "dependent": { "-country": "F" },
+  "government": "overseas collectivity of France",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "border": {
+    "-country": "NA",
+    "-length": "15"
+  },
+  "city": {
+    "-id": "city-Marigot-SMAR",
+    "-is_country_cap": "yes",
+    "-country": "SMAR",
+    "name": "Marigot",
+    "longitude": "-63.1",
+    "latitude": "18.1",
+    "population": {
+      "-year": "06",
+      "#text": "5700"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-StMartin" }
+  }
+}
+{
+  "-car_code": "CO",
+  "-area": "1138910",
+  "-capital": "cty-Colombia-Bogota",
+  "-memberships": "org-OPANAL org-AG org-CDB org-CCC org-ECLAC org-FAO org-G-3 org-G-11 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UNHCR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Colombia",
+  "population": "36813161",
+  "population_growth": "1.66",
+  "infant_mortality": "25.8",
+  "gdp_total": "192500",
+  "gdp_agri": "21.5",
+  "gdp_ind": "29",
+  "gdp_serv": "49.5",
+  "inflation": "19.5",
+  "indep_date": "1810-07-20",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "black-Indian"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Spanish"
+  },
+  "border": [
+    {
+      "-country": "PA",
+      "-length": "225"
+    },
+    {
+      "-country": "BR",
+      "-length": "1643"
+    },
+    {
+      "-country": "PE",
+      "-length": "2900"
+    },
+    {
+      "-country": "YV",
+      "-length": "2050"
+    },
+    {
+      "-country": "EC",
+      "-length": "590"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Colombia-2",
+      "-capital": "cty-cid-cia-Colombia-Leticia",
+      "-country": "CO",
+      "name": "Amazonas",
+      "area": "109665",
+      "population": "39937",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Leticia",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-2",
+        "name": "Leticia",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amazonas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-3",
+      "-capital": "cty-Colombia-Medellin",
+      "-country": "CO",
+      "name": "Antioquia",
+      "area": "63612",
+      "population": "4067207",
+      "city": [
+        {
+          "-id": "cty-Colombia-Medellin",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-3",
+          "name": "Medellin",
+          "longitude": "-75.6",
+          "latitude": "6.21667",
+          "population": {
+            "-year": "95",
+            "#text": "1621356"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-14",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-3",
+          "name": "Bello",
+          "population": {
+            "-year": "95",
+            "#text": "304819"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-28",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-3",
+          "name": "Itagui",
+          "population": {
+            "-year": "95",
+            "#text": "169374"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-34",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-3",
+          "name": "Turbo",
+          "population": {
+            "-year": "95",
+            "#text": "127045"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-40",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-3",
+          "name": "Envigado",
+          "population": {
+            "-year": "95",
+            "#text": "109240"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-4",
+      "-capital": "cty-cid-cia-Colombia-Arauca",
+      "-country": "CO",
+      "name": "Arauca",
+      "area": "23818",
+      "population": "89972",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Arauca",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-4",
+        "name": "Arauca"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-5",
+      "-capital": "cty-cid-cia-Colombia-5",
+      "-country": "CO",
+      "name": "Atlantico",
+      "area": "3388",
+      "population": "1478213",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-5",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-5",
+          "name": "Barranquilla",
+          "longitude": "-74.8",
+          "latitude": "10.95",
+          "population": {
+            "-year": "95",
+            "#text": "1064255"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-RioMagdalena"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-19",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-5",
+          "name": "Soledad",
+          "population": {
+            "-year": "95",
+            "#text": "264583"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-39",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-5",
+          "name": "Malambo",
+          "population": {
+            "-year": "95",
+            "#text": "112289"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-6",
+      "-capital": "cty-cid-cia-Colombia-6",
+      "-country": "CO",
+      "name": "Bolivar",
+      "area": "25978",
+      "population": "1288985",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-6",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-6",
+          "name": "Cartagena",
+          "population": {
+            "-year": "95",
+            "#text": "745689"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-41",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-6",
+          "name": "Mangangue",
+          "population": {
+            "-year": "95",
+            "#text": "104496"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-7",
+      "-capital": "cty-cid-cia-Colombia-35",
+      "-country": "CO",
+      "name": "Boyaca",
+      "area": "23189",
+      "population": "1209739",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-35",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-7",
+        "name": "Tunja",
+        "population": {
+          "-year": "95",
+          "#text": "120210"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-8",
+      "-capital": "cty-cid-cia-Colombia-11",
+      "-country": "CO",
+      "name": "Caldas",
+      "area": "7888",
+      "population": "883024",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-11",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-8",
+        "name": "Manizales",
+        "population": {
+          "-year": "95",
+          "#text": "335125"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-9",
+      "-capital": "cty-cid-cia-Colombia-36",
+      "-country": "CO",
+      "name": "Caqueta",
+      "area": "88965",
+      "population": "264507",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-36",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-9",
+        "name": "Florencia",
+        "population": {
+          "-year": "95",
+          "#text": "118027"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-10",
+      "-capital": "cty-cid-cia-Colombia-Yopal",
+      "-country": "CO",
+      "name": "Casanare",
+      "area": "44640",
+      "population": "147472",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Yopal",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-10",
+        "name": "Yopal"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-11",
+      "-capital": "cty-cid-cia-Colombia-24",
+      "-country": "CO",
+      "name": "Cauca",
+      "area": "29308",
+      "population": "857751",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-24",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-11",
+        "name": "Popayan",
+        "population": {
+          "-year": "95",
+          "#text": "223128"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-12",
+      "-capital": "cty-cid-cia-Colombia-18",
+      "-country": "CO",
+      "name": "Cesar",
+      "area": "22905",
+      "population": "699428",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-18",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-12",
+        "name": "Valledupar",
+        "population": {
+          "-year": "95",
+          "#text": "265505"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-13",
+      "-capital": "cty-cid-cia-Colombia-33",
+      "-country": "CO",
+      "name": "Choco",
+      "area": "46530",
+      "population": "313567",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-33",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-13",
+        "name": "Quibdo",
+        "population": {
+          "-year": "95",
+          "#text": "130921"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-14",
+      "-capital": "cty-cid-cia-Colombia-15",
+      "-country": "CO",
+      "name": "Cordoba",
+      "area": "25020",
+      "population": "1013247",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-15",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-14",
+        "name": "Monteria",
+        "population": {
+          "-year": "95",
+          "#text": "276074"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-15",
+      "-capital": "cty-Colombia-Bogota",
+      "-country": "CO",
+      "name": "Cundinamarca",
+      "area": "22623",
+      "population": "1512928",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-17",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-15",
+        "name": "Soacha",
+        "population": {
+          "-year": "95",
+          "#text": "266817"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-16",
+      "-capital": "cty-cid-cia-Colombia-Puerto-Inirida",
+      "-country": "CO",
+      "name": "Guainia",
+      "area": "72238",
+      "population": "12345",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Puerto-Inirida",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-16",
+        "name": "Puerto Inirida",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Orinoco"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-17",
+      "-capital": "cty-cid-cia-Colombia-31",
+      "-country": "CO",
+      "name": "Guajira, La",
+      "area": "20848",
+      "population": "299995",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-30",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-17",
+          "name": "Cienaga",
+          "population": {
+            "-year": "95",
+            "#text": "144340"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-31",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-17",
+          "name": "Riohacha",
+          "population": {
+            "-year": "95",
+            "#text": "142455"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-18",
+      "-capital": "cty-cid-cia-Colombia-San-Jose-del-Guaviare",
+      "-country": "CO",
+      "name": "Guaviare",
+      "area": "42327",
+      "population": "47073",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-San-Jose-del-Guaviare",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-18",
+        "name": "San Jose del Guaviare"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-19",
+      "-capital": "cty-cid-cia-Colombia-22",
+      "-country": "CO",
+      "name": "Huila",
+      "area": "19890",
+      "population": "693712",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-22",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-19",
+        "name": "Neiva",
+        "population": {
+          "-year": "95",
+          "#text": "248178"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-RioMagdalena"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-20",
+      "-capital": "cty-cid-cia-Colombia-13",
+      "-country": "CO",
+      "name": "Magdalena",
+      "area": "23188",
+      "population": "890934",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-13",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-20",
+        "name": "Santa Marta",
+        "population": {
+          "-year": "95",
+          "#text": "309372"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-21",
+      "-capital": "cty-cid-cia-Colombia-21",
+      "-country": "CO",
+      "name": "Meta",
+      "area": "85635",
+      "population": "474046",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-21",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-21",
+        "name": "Villavicencio",
+        "population": {
+          "-year": "95",
+          "#text": "252711"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-22",
+      "-capital": "cty-cid-cia-Colombia-12",
+      "-country": "CO",
+      "name": "Narino",
+      "area": "33268",
+      "population": "1085173",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-12",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-22",
+          "name": "Pasto",
+          "population": {
+            "-year": "95",
+            "#text": "325540"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-38",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-22",
+          "name": "Tumaco",
+          "population": {
+            "-year": "95",
+            "#text": "114802"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-23",
+      "-capital": "cty-cid-cia-Colombia-7",
+      "-country": "CO",
+      "name": "Norte de Santander",
+      "area": "21658",
+      "population": "913491",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-7",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-23",
+        "name": "Cucuta",
+        "population": {
+          "-year": "95",
+          "#text": "479309"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-24",
+      "-capital": "cty-cid-cia-Colombia-Mocoa",
+      "-country": "CO",
+      "name": "Putumayo",
+      "area": "24885",
+      "population": "174219",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Mocoa",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-24",
+        "name": "Mocoa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-25",
+      "-capital": "cty-cid-cia-Colombia-25",
+      "-country": "CO",
+      "name": "Quindio",
+      "area": "1845",
+      "population": "392208",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-25",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-25",
+        "name": "Armenia",
+        "population": {
+          "-year": "95",
+          "#text": "220303"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-26",
+      "-capital": "cty-cid-cia-Colombia-8",
+      "-country": "CO",
+      "name": "Risaralda",
+      "area": "4140",
+      "population": "652872",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-8",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-26",
+          "name": "Pereira",
+          "population": {
+            "-year": "95",
+            "#text": "352530"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-29",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-26",
+          "name": "Dosquebradas",
+          "population": {
+            "-year": "95",
+            "#text": "163599"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-27",
+      "-capital": "cty-cid-cia-Colombia-San-Andres",
+      "-country": "CO",
+      "name": "San Andres y Providencia",
+      "area": "44",
+      "population": "35818",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-San-Andres",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-27",
+        "name": "San Andres"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-28",
+      "-capital": "cty-Colombia-Bogota",
+      "-country": "CO",
+      "name": "Santa Fe de Bogota, DC",
+      "area": "1587",
+      "population": "4236490",
+      "city": {
+        "-id": "cty-Colombia-Bogota",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-28",
+        "name": "Bogota",
+        "longitude": "-74.0833",
+        "latitude": "4.6",
+        "population": {
+          "-year": "95",
+          "#text": "5237635"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-29",
+      "-capital": "cty-cid-cia-Colombia-9",
+      "-country": "CO",
+      "name": "Santander del Sur",
+      "area": "30537",
+      "population": "1511392",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Colombia-9",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-29",
+          "name": "Bucaramanga",
+          "population": {
+            "-year": "95",
+            "#text": "351737"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-23",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-29",
+          "name": "Floridablanca",
+          "population": {
+            "-year": "95",
+            "#text": "246834"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-26",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-29",
+          "name": "Barrancabermeja",
+          "population": {
+            "-year": "95",
+            "#text": "180653"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-RioMagdalena"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-30",
+      "-capital": "cty-cid-cia-Colombia-27",
+      "-country": "CO",
+      "name": "Sucre",
+      "area": "10917",
+      "population": "561649",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-27",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-30",
+        "name": "Sincelejo",
+        "population": {
+          "-year": "95",
+          "#text": "180076"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-31",
+      "-capital": "cty-cid-cia-Colombia-10",
+      "-country": "CO",
+      "name": "Tolima",
+      "area": "23562",
+      "population": "1142220",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-10",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-31",
+        "name": "Ibague",
+        "population": {
+          "-year": "95",
+          "#text": "346632"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-32",
+      "-capital": "cty-Colombia-Cali",
+      "-country": "CO",
+      "name": "Valle de Cauca",
+      "area": "22140",
+      "population": "3027247",
+      "city": [
+        {
+          "-id": "cty-Colombia-Cali",
+          "-is_state_cap": "yes",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-32",
+          "name": "Cali",
+          "longitude": "-76.5",
+          "latitude": "3.41667",
+          "population": {
+            "-year": "95",
+            "#text": "1718871"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-16",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-32",
+          "name": "Buenaventura",
+          "population": {
+            "-year": "95",
+            "#text": "266988"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-20",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-32",
+          "name": "Palmira",
+          "population": {
+            "-year": "95",
+            "#text": "256823"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-32",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-32",
+          "name": "Tulua",
+          "population": {
+            "-year": "95",
+            "#text": "138124"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Colombia-37",
+          "-country": "CO",
+          "-province": "prov-cid-cia-Colombia-32",
+          "name": "Cartago",
+          "population": {
+            "-year": "95",
+            "#text": "117166"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-33",
+      "-capital": "cty-cid-cia-Colombia-Mitu",
+      "-country": "CO",
+      "name": "Vaupes",
+      "area": "65268",
+      "population": "26178",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Mitu",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-33",
+        "name": "Mitu"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Colombia-34",
+      "-capital": "cty-cid-cia-Colombia-Puerto-Carreno",
+      "-country": "CO",
+      "name": "Vichada",
+      "area": "100242",
+      "population": "18702",
+      "city": {
+        "-id": "cty-cid-cia-Colombia-Puerto-Carreno",
+        "-is_state_cap": "yes",
+        "-country": "CO",
+        "-province": "prov-cid-cia-Colombia-34",
+        "name": "Puerto Carreno",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Orinoco"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "PR",
+  "-area": "8870",
+  "-capital": "city-San-Juan-PR-PR",
+  "-memberships": "org-Caricom org-Interpol org-IOC org-UPU org-WCL org-WFTU org-WHO org-WToO",
+  "name": "Puerto Rico",
+  "population": "3971020",
+  "population_growth": "0.34",
+  "infant_mortality": "8.42",
+  "gdp_total": "88000",
+  "gdp_agri": "1",
+  "gdp_ind": "45",
+  "gdp_serv": "54",
+  "inflation": "6.5",
+  "dependent": { "-country": "USA" },
+  "government": "commonwealth associated with the US",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "80.2",
+      "#text": "White"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Black"
+    },
+    {
+      "-percentage": "0.4",
+      "#text": "Amerindian"
+    },
+    {
+      "-percentage": "0.4",
+      "#text": "Asian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "85",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Protestant"
+    }
+  ],
+  "city": {
+    "-id": "city-San-Juan-PR-PR",
+    "-is_country_cap": "yes",
+    "-country": "PR",
+    "name": "San Juan",
+    "longitude": "-66.12",
+    "latitude": "18.49",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-PuertoRico" }
+  }
+}
+{
+  "-car_code": "KN",
+  "-area": "269",
+  "-capital": "cty-cid-cia-Saint-Kitts-and-Nevis-Basseterre",
+  "-memberships": "org-ACP org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICFTU org-Interpol org-IDA org-IFRCS org-IFAD org-IMF org-IOC org-ICRM org-OAS org-OECS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WCL org-WHO org-WIPO",
+  "name": "Saint Kitts and Nevis",
+  "population": "41369",
+  "population_growth": "0.98",
+  "infant_mortality": "18.9",
+  "gdp_total": "220",
+  "gdp_agri": "6.2",
+  "gdp_ind": "35.7",
+  "gdp_serv": "58.1",
+  "inflation": "-0.9",
+  "indep_date": "1983-09-19",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "African"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Saint-Kitts-and-Nevis-Basseterre",
+    "-is_country_cap": "yes",
+    "-country": "KN",
+    "name": "Basseterre",
+    "longitude": "-62.3",
+    "latitude": "17",
+    "population": {
+      "-year": "87",
+      "#text": "14700"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Saint_Kitts" }
+  }
+}
+{
+  "-car_code": "WL",
+  "-area": "620",
+  "-capital": "cty-cid-cia-Saint-Lucia-Castries",
+  "-memberships": "org-ACP org-ACCT org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-Intelsat org-NAM org-OAS org-OECS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Saint Lucia",
+  "population": "157862",
+  "population_growth": "1.14",
+  "infant_mortality": "20",
+  "gdp_total": "640",
+  "gdp_agri": "13.8",
+  "gdp_ind": "17.4",
+  "gdp_serv": "68.8",
+  "inflation": "0.8",
+  "indep_date": "1979-02-22",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "90.3",
+      "#text": "African"
+    },
+    {
+      "-percentage": "3.2",
+      "#text": "East Indian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "90",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Anglican"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Saint-Lucia-Castries",
+    "-is_country_cap": "yes",
+    "-country": "WL",
+    "name": "Castries",
+    "longitude": "-60.5",
+    "latitude": "14",
+    "population": {
+      "-year": "87",
+      "#text": "49000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Saint_Lucia" }
+  }
+}
+{
+  "-car_code": "SPMI",
+  "-area": "242",
+  "-capital": "cty-cid-cia-SPMI-Saint-Pierre",
+  "-memberships": "org-WFTU",
+  "name": "Saint Pierre and Miquelon",
+  "population": "6809",
+  "population_growth": "0.77",
+  "infant_mortality": "9.95",
+  "gdp_total": "68",
+  "dependent": { "-country": "F" },
+  "government": "territorial collectivity of France",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Basques Bretons"
+  },
+  "religions": {
+    "-percentage": "99",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "French"
+  },
+  "city": {
+    "-id": "cty-cid-cia-SPMI-Saint-Pierre",
+    "-is_country_cap": "yes",
+    "-country": "SPMI",
+    "name": "Saint-Pierre",
+    "longitude": "-56",
+    "latitude": "47",
+    "population": {
+      "-year": "09",
+      "#text": "5618"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-SaintPierre" }
+  }
+}
+{
+  "-car_code": "WV",
+  "-area": "389",
+  "-capital": "cty-cid-cia-Saint-Vincent-and-the-Grenadines-Kingstown",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-ECLAC org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-OAS org-OECS org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WTrO",
+  "name": "Saint Vincent and the Grenadines",
+  "population": "118344",
+  "population_growth": "0.64",
+  "infant_mortality": "16.8",
+  "gdp_total": "240",
+  "gdp_agri": "24",
+  "gdp_ind": "33.1",
+  "gdp_serv": "42.9",
+  "inflation": "-0.2",
+  "indep_date": "1979-10-27",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Saint-Vincent-and-the-Grenadines-Kingstown",
+    "-is_country_cap": "yes",
+    "-country": "WV",
+    "name": "Kingstown",
+    "longitude": "-61.1",
+    "latitude": "13.1",
+    "population": {
+      "-year": "87",
+      "#text": "33694"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Saint_Vincent" }
+  }
+}
+{
+  "-car_code": "TT",
+  "-area": "5130",
+  "-capital": "cty-cid-cia-Trinidad-and-Tobago-Port-of-Spain",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-CDB org-C org-CCC org-ECLAC org-FAO org-G-24 org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNMIH org-UNU org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Trinidad and Tobago",
+  "population": "1272385",
+  "population_growth": "0.08",
+  "infant_mortality": "18.2",
+  "gdp_total": "16200",
+  "gdp_agri": "4.8",
+  "gdp_ind": "44.5",
+  "gdp_serv": "50.7",
+  "inflation": "5.4",
+  "indep_date": "1962-08-31",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "40",
+      "#text": "East Indian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "6",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "32.2",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "14",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "14.4",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "24.3",
+      "#text": "Hindu"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Trinidad-and-Tobago-Port-of-Spain",
+    "-is_country_cap": "yes",
+    "-country": "TT",
+    "name": "Port-of-Spain",
+    "longitude": "-61.2",
+    "latitude": "10.4",
+    "population": {
+      "-year": "87",
+      "#text": "150000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-Trinidad" }
+  }
+}
+{
+  "-car_code": "TUCA",
+  "-area": "430",
+  "-capital": "city-Grand-Turk-TUCA-TUCA",
+  "-memberships": "org-Caricom org-CDB org-Interpol org-UPU",
+  "name": "Turks and Caicos Islands",
+  "population": "22942",
+  "population_growth": "2.56",
+  "infant_mortality": "13.89",
+  "gdp_total": "216",
+  "inflation": "4",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "90",
+    "#text": "African"
+  },
+  "religions": [
+    {
+      "-percentage": "18",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Baptist"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Church of God"
+    },
+    {
+      "-percentage": "16",
+      "#text": "Methodist"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-Grand-Turk-TUCA-TUCA",
+    "-is_country_cap": "yes",
+    "-country": "TUCA",
+    "name": "Grand Turk",
+    "longitude": "-71.08",
+    "latitude": "21.28",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-GrandTurk" }
+  }
+}
+{
+  "-car_code": "VIRG",
+  "-area": "352",
+  "-capital": "cty-cid-cia-VIRG-Charlotte-Amalie",
+  "-memberships": "org-ECLAC org-IOC",
+  "name": "Virgin Islands",
+  "population": "97120",
+  "population_growth": "-0.06",
+  "infant_mortality": "12.54",
+  "gdp_total": "1200",
+  "government": "organized",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "34",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "42",
+      "#text": "Baptist"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Episcopalian"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-VIRG-Charlotte-Amalie",
+    "-is_country_cap": "yes",
+    "-country": "VIRG",
+    "name": "Charlotte Amalie",
+    "longitude": "-64.9",
+    "latitude": "18.3",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Caribbean"
+    },
+    "located_on": { "-island": "island-SaintThomas" }
+  }
+}
+{
+  "-car_code": "AMSA",
+  "-area": "199",
+  "-capital": "city-Pago-Pago-AMSA-AMSA",
+  "-memberships": "org-Interpol org-IOC org-SPC org-UPU",
+  "name": "American Samoa",
+  "population": "65628",
+  "population_growth": "1.22",
+  "infant_mortality": "10.18",
+  "gdp_total": "462.2",
+  "dependent": { "-country": "USA" },
+  "government": "unincorporated and unorganized territory of the US",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2",
+      "#text": "Caucasian"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Tongan"
+    },
+    {
+      "-percentage": "89",
+      "#text": "Samoan"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "20",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "50",
+      "#text": "Christian Congregationalist"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "90.6",
+      "#text": "Samoan"
+    },
+    {
+      "-percentage": "2.9",
+      "#text": "English"
+    },
+    {
+      "-percentage": "2.4",
+      "#text": "Tongan"
+    }
+  ],
+  "city": {
+    "-id": "city-Pago-Pago-AMSA-AMSA",
+    "-is_country_cap": "yes",
+    "-country": "AMSA",
+    "name": "Pago Pago",
+    "longitude": "-170.42",
+    "latitude": "-14.16",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Tutuila" }
+  }
+}
+{
+  "-car_code": "AUS",
+  "-area": "7686850",
+  "-capital": "cty-cid-cia-Australia-8",
+  "-memberships": "org-AG org-AsDB org-ANZUS org-BIS org-CP org-C org-CCC org-ESCAP org-EBRD org-FAO org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-EN org-NSG org-OECD org-PCA org-SPC org-SPF org-Sparteca org-UN org-UNAMIR org-UNESCO org-UNFICYP org-UNIDO org-UNHCR org-UNTSO org-UNU org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "Australia",
+  "population": "18260863",
+  "population_growth": "0.99",
+  "infant_mortality": "5.5",
+  "gdp_total": "405400",
+  "gdp_agri": "3.1",
+  "gdp_ind": "27.7",
+  "gdp_serv": "69.2",
+  "inflation": "4.75",
+  "indep_date": "1901-01-01",
+  "government": "federal parliamentary state",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "95",
+      "#text": "Caucasian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "26",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "26.1",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "24.3",
+      "#text": "Christian"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "province": [
+    {
+      "-id": "prov-cid-cia-Australia-2",
+      "-capital": "cty-Australia-Sydney",
+      "-country": "AUS",
+      "name": "New South Wales",
+      "area": "801428",
+      "population": "6115100",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Australia-7",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-2",
+          "name": "Newcastle",
+          "population": {
+            "-year": "90",
+            "#text": "429000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-Australia-Sydney",
+          "-is_state_cap": "yes",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-2",
+          "name": "Sydney",
+          "longitude": "151.2",
+          "latitude": "-33.8667",
+          "population": {
+            "-year": "90",
+            "#text": "3657000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Australia-10",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-2",
+          "name": "Wollongong",
+          "population": {
+            "-year": "90",
+            "#text": "238000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Australia-14",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-2",
+          "name": "Sunshine Coast",
+          "population": {
+            "-year": "90",
+            "#text": "110000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Australia-3",
+      "-capital": "cty-Australia-Darwin",
+      "-country": "AUS",
+      "name": "Northern Territory",
+      "area": "1346200",
+      "population": "173878",
+      "city": {
+        "-id": "cty-Australia-Darwin",
+        "-is_state_cap": "yes",
+        "-country": "AUS",
+        "-province": "prov-cid-cia-Australia-3",
+        "name": "Darwin",
+        "longitude": "130.85",
+        "latitude": "-12.4667",
+        "population": {
+          "-year": "87",
+          "#text": "68500"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Indic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Australia-4",
+      "-capital": "cty-Australia-Brisbane",
+      "-country": "AUS",
+      "name": "Queensland",
+      "area": "1727000",
+      "population": "3277000",
+      "city": [
+        {
+          "-id": "cty-Australia-Brisbane",
+          "-is_state_cap": "yes",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-4",
+          "name": "Brisbane",
+          "longitude": "153.033",
+          "latitude": "-27.4667",
+          "population": {
+            "-year": "90",
+            "#text": "1302000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Australia-9",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-4",
+          "name": "Gold Coast",
+          "population": {
+            "-year": "90",
+            "#text": "266000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Australia-13",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-4",
+          "name": "Townsville",
+          "population": {
+            "-year": "90",
+            "#text": "114000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Pacific"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Australia-5",
+      "-capital": "cty-Australia-Adelaide",
+      "-country": "AUS",
+      "name": "South Australia",
+      "area": "984377",
+      "population": "1474000",
+      "city": {
+        "-id": "cty-Australia-Adelaide",
+        "-is_state_cap": "yes",
+        "-country": "AUS",
+        "-province": "prov-cid-cia-Australia-5",
+        "name": "Adelaide",
+        "longitude": "138.583",
+        "latitude": "-34.9333",
+        "population": {
+          "-year": "90",
+          "#text": "1050000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Indic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Australia-6",
+      "-capital": "cty-cid-cia-Australia-11",
+      "-country": "AUS",
+      "name": "Tasmania",
+      "area": "67800",
+      "population": "473022",
+      "city": {
+        "-id": "cty-cid-cia-Australia-11",
+        "-is_state_cap": "yes",
+        "-country": "AUS",
+        "-province": "prov-cid-cia-Australia-6",
+        "name": "Hobart",
+        "longitude": "147.14",
+        "latitude": "-42.5",
+        "population": {
+          "-year": "90",
+          "#text": "184000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Pacific"
+        },
+        "located_on": { "-island": "island-Tasmania" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Australia-7",
+      "-capital": "cty-Australia-Melbourne",
+      "-country": "AUS",
+      "name": "Victoria",
+      "area": "227600",
+      "population": "4502200",
+      "city": [
+        {
+          "-id": "cty-Australia-Melbourne",
+          "-is_state_cap": "yes",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-7",
+          "name": "Melbourne",
+          "longitude": "144.967",
+          "latitude": "-37.8167",
+          "population": {
+            "-year": "90",
+            "#text": "3081000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Australia-12",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-7",
+          "name": "Geelong",
+          "population": {
+            "-year": "90",
+            "#text": "151000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Australia-8",
+      "-capital": "cty-Australia-Perth",
+      "-country": "AUS",
+      "name": "Western Australia",
+      "area": "2525500",
+      "population": "1731700",
+      "city": [
+        {
+          "-id": "cty-Australia-Perth",
+          "-is_state_cap": "yes",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-8",
+          "name": "Perth",
+          "longitude": "115.85",
+          "latitude": "-31.95",
+          "population": {
+            "-year": "90",
+            "#text": "1193000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        },
+        {
+          "-id": "stadt-Albany-AUS-WES",
+          "-country": "AUS",
+          "-province": "prov-cid-cia-Australia-8",
+          "name": "Albany",
+          "longitude": "118",
+          "latitude": "-35",
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Australia-9",
+      "-capital": "cty-cid-cia-Australia-8",
+      "-country": "AUS",
+      "name": "Australia Capital Territory",
+      "area": "2452",
+      "population": "304100",
+      "city": {
+        "-id": "cty-cid-cia-Australia-8",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "AUS",
+        "-province": "prov-cid-cia-Australia-9",
+        "name": "Canberra",
+        "longitude": "149.08",
+        "latitude": "-35.1",
+        "population": {
+          "-year": "90",
+          "#text": "310000"
+        },
+        "located_at": [
+          {
+            "-watertype": "lake",
+            "-lake": "lake-LakeBurleyGriffin"
+          },
+          {
+            "-watertype": "river",
+            "-river": "river-MurrumbidgeeRiver"
+          }
+        ]
+      }
+    }
+  ]
+}
+{
+  "-car_code": "COOK",
+  "-area": "240",
+  "-capital": "city-Avarua-COOK-COOK",
+  "-memberships": "org-ACP org-FAO org-ICAO org-ICRM org-IFAD org-IFRCS org-IOC org-Sparteca org-SPC org-UNESCO org-UPU org-WHO org-WMO",
+  "name": "Cook Islands",
+  "population": "11870",
+  "population_growth": "-3.3",
+  "infant_mortality": "16.9",
+  "gdp_total": "183.2",
+  "gdp_agri": "15.1",
+  "gdp_ind": "9.6",
+  "gdp_serv": "75.3",
+  "inflation": "2.1",
+  "dependent": { "-country": "NZ" },
+  "government": "self governing parliamentary government in free association with New Zealand",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "93.5",
+    "#text": "Polynesian"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Christian"
+  },
+  "city": {
+    "-id": "city-Avarua-COOK-COOK",
+    "-is_country_cap": "yes",
+    "-country": "COOK",
+    "name": "Avarua",
+    "longitude": "-159.46",
+    "latitude": "-21.12",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Rarotonga" }
+  }
+}
+{
+  "-car_code": "FJI",
+  "-area": "18270",
+  "-capital": "cty-cid-cia-Fiji-Suva",
+  "-memberships": "org-ACP org-AsDB org-CP org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-PCA org-SPC org-SPF org-Sparteca org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UNIFIL org-UNIKOM org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Fiji",
+  "population": "782381",
+  "population_growth": "1.28",
+  "infant_mortality": "17.4",
+  "gdp_total": "4700",
+  "gdp_agri": "22",
+  "gdp_ind": "17",
+  "gdp_serv": "61",
+  "inflation": "2",
+  "indep_date": "1970-10-10",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "46",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "49",
+      "#text": "Fijian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "8",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "52",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "38",
+      "#text": "Hindu"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Fiji-Suva",
+    "-is_country_cap": "yes",
+    "-country": "FJI",
+    "name": "Suva",
+    "longitude": "178",
+    "latitude": "-19",
+    "population": {
+      "-year": "87",
+      "#text": "69481"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-VitiLevu" }
+  }
+}
+{
+  "-car_code": "FPOL",
+  "-area": "4167",
+  "-capital": "city-Papeete-FPOL-FPOL",
+  "-memberships": "org-ESCAP org-ICFTU org-SPC org-WMO",
+  "name": "French Polynesia",
+  "population": "287032",
+  "population_growth": "1.39",
+  "infant_mortality": "7.55",
+  "gdp_total": "6100",
+  "gdp_agri": "3.5",
+  "gdp_ind": "20.4",
+  "gdp_serv": "76.1",
+  "inflation": "1.1",
+  "dependent": { "-country": "F" },
+  "government": "overseas territory of France since 1946",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "10",
+      "#text": "French"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "78",
+      "#text": "Polynesian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "30",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "54",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "61.1",
+      "#text": "French"
+    },
+    {
+      "-percentage": "31.4",
+      "#text": "Polynesian"
+    }
+  ],
+  "city": {
+    "-id": "city-Papeete-FPOL-FPOL",
+    "-is_country_cap": "yes",
+    "-country": "FPOL",
+    "name": "Papeete",
+    "longitude": "-149.34",
+    "latitude": "-17.32",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Tahiti" }
+  }
+}
+{
+  "-car_code": "GUAM",
+  "-area": "541.3",
+  "-capital": "city-Agana-GUAM",
+  "-memberships": "org-IOC org-SPC org-UPU",
+  "name": "Guam",
+  "population": "178430",
+  "population_growth": "1.37",
+  "infant_mortality": "6.05",
+  "gdp_total": "2500",
+  "inflation": "2.5",
+  "dependent": { "-country": "USA" },
+  "government": "organized",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "37.1",
+      "#text": "Chamorro"
+    },
+    {
+      "-percentage": "26.3",
+      "#text": "Filipino"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "languages": [
+    {
+      "-percentage": "38.3",
+      "#text": "English"
+    },
+    {
+      "-percentage": "22.2",
+      "#text": "Chamorro"
+    },
+    {
+      "-percentage": "22.2",
+      "#text": "Phillipine Languages"
+    }
+  ],
+  "city": [
+    {
+      "-id": "city-Agana-GUAM",
+      "-is_country_cap": "yes",
+      "-country": "GUAM",
+      "name": "Agana",
+      "longitude": "144.44",
+      "latitude": "13.28",
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Guam" }
+    },
+    {
+      "-id": "city-Dededo-GUAM",
+      "-country": "GUAM",
+      "name": "Dededo",
+      "longitude": "144.8",
+      "latitude": "13.5",
+      "population": {
+        "-year": "09",
+        "#text": "46000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Guam" }
+    }
+  ]
+}
+{
+  "-car_code": "KIR",
+  "-area": "717",
+  "-capital": "cty-cid-cia-Kiribati-Tarawa",
+  "-memberships": "org-ACP org-AsDB org-C org-ESCAP org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IMF org-ITU org-Intelsat org-SPC org-SPF org-Sparteca org-UNESCO org-UPU org-WHO",
+  "name": "Kiribati",
+  "population": "80919",
+  "population_growth": "1.89",
+  "infant_mortality": "52.9",
+  "gdp_total": "68",
+  "inflation": "5.5",
+  "indep_date": "1979-07-12",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Micronesian"
+  },
+  "religions": [
+    {
+      "-percentage": "52.6",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "40.9",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Mormon"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Kiribati-Tarawa",
+    "-is_country_cap": "yes",
+    "-country": "KIR",
+    "name": "Tarawa",
+    "longitude": "173",
+    "latitude": "1.4",
+    "population": {
+      "-year": "87",
+      "#text": "2100"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Tarawa" }
+  }
+}
+{
+  "-car_code": "MH",
+  "-area": "181.3",
+  "-capital": "cty-cid-cia-Marshall-Islands-Majuro",
+  "-memberships": "org-AsDB org-ESCAP org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFC org-IMF org-Intelsat org-SPC org-SPF org-Sparteca org-UN org-UNESCO org-WHO",
+  "name": "Marshall Islands",
+  "population": "58363",
+  "population_growth": "3.85",
+  "infant_mortality": "46.9",
+  "gdp_total": "94",
+  "inflation": "4",
+  "indep_date": "1986-10-21",
+  "government": "constitutional government in free association with the US",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Micronesian"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Christian"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Marshall-Islands-Majuro",
+    "-is_country_cap": "yes",
+    "-country": "MH",
+    "name": "Majuro",
+    "longitude": "171.3",
+    "latitude": "7.1",
+    "population": {
+      "-year": "87",
+      "#text": "7600"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Majuro" }
+  }
+}
+{
+  "-car_code": "FSM",
+  "-area": "702",
+  "-capital": "cty-cid-cia-Micronesia-Palikir",
+  "name": "Micronesia",
+  "population": "125377",
+  "population_growth": "3.34",
+  "infant_mortality": "35.8",
+  "gdp_total": "205",
+  "inflation": "4",
+  "indep_date": "1986-11-03",
+  "government": "constitutional government in free association with the US",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Polynesian"
+  },
+  "religions": [
+    {
+      "-percentage": "50",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "47",
+      "#text": "Protestant"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Micronesia-Kolonia",
+      "-country": "FSM",
+      "name": "Kolonia",
+      "longitude": "158.2",
+      "latitude": "7",
+      "population": {
+        "-year": "87",
+        "#text": "5500"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Ponape" }
+    },
+    {
+      "-id": "cty-cid-cia-Micronesia-Palikir",
+      "-is_country_cap": "yes",
+      "-country": "FSM",
+      "name": "Palikir",
+      "longitude": "158.15",
+      "latitude": "6.9",
+      "population": {
+        "-year": "09",
+        "#text": "6444"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Ponape" }
+    }
+  ]
+}
+{
+  "-car_code": "NAU",
+  "-area": "21",
+  "-capital": "cty-cid-cia-Yaren",
+  "-memberships": "org-AsDB org-C org-ESCAP org-ICAO org-Interpol org-ITU org-Intelsat org-SPC org-SPF org-Sparteca org-UPU org-WHO",
+  "name": "Nauru",
+  "population": "10273",
+  "population_growth": "1.33",
+  "infant_mortality": "40.6",
+  "gdp_total": "100",
+  "inflation": "-3.6",
+  "indep_date": "1968-01-31",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "8",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "8",
+      "#text": "European"
+    },
+    {
+      "-percentage": "58",
+      "#text": "Nauruan"
+    },
+    {
+      "-percentage": "26",
+      "#text": "Pacific Islander"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Christian"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Yaren",
+    "-is_country_cap": "yes",
+    "-country": "NAU",
+    "name": "Yaren",
+    "longitude": "167",
+    "latitude": "-1",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Nauru" }
+  }
+}
+{
+  "-car_code": "NCA",
+  "-area": "19060",
+  "-capital": "cty-cid-cia-New-Caledonia-Noumea",
+  "-memberships": "org-SPC org-UPU org-WFTU org-WMO",
+  "name": "New Caledonia",
+  "population": "227436",
+  "population_growth": "1.14",
+  "infant_mortality": "7.05",
+  "gdp_total": "3158",
+  "gdp_agri": "15",
+  "gdp_ind": "8.8",
+  "gdp_serv": "76.2",
+  "inflation": "1.4",
+  "dependent": { "-country": "F" },
+  "government": "overseas territory of France since 1956",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1.4",
+      "#text": "Vietnamese"
+    },
+    {
+      "-percentage": "34.1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "44.1",
+      "#text": "Melanesian"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Wallisian"
+    },
+    {
+      "-percentage": "2.5",
+      "#text": "Indonesian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "60",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Protestant"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-New-Caledonia-Noumea",
+    "-is_country_cap": "yes",
+    "-country": "NCA",
+    "name": "Noumea",
+    "longitude": "166.27",
+    "latitude": "-22.16",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-GrandeTerre" }
+  }
+}
+{
+  "-car_code": "NZ",
+  "-area": "268680",
+  "-capital": "cty-cid-cia-New-Zealand-Wellington",
+  "-memberships": "org-APEC org-AsDB org-ANZUS org-CP org-C org-CCC org-ESCAP org-EBRD org-FAO org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IEA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-NSG org-OECD org-PCA org-SPC org-SPF org-Sparteca org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNPREDEP org-UNPROFOR org-UNTSO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "New Zealand",
+  "population": "3547983",
+  "population_growth": "1.12",
+  "infant_mortality": "6.7",
+  "gdp_total": "62300",
+  "gdp_agri": "7.3",
+  "gdp_ind": "25.9",
+  "gdp_serv": "66.8",
+  "inflation": "2",
+  "indep_date": "1907-09-26",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "88",
+      "#text": "European"
+    },
+    {
+      "-percentage": "2.9",
+      "#text": "Pacific Islander"
+    },
+    {
+      "-percentage": "8.9",
+      "#text": "Maori"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "15",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "24",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Baptist"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Methodist"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Presbyterian"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-New-Zealand-Wellington",
+      "-is_country_cap": "yes",
+      "-country": "NZ",
+      "name": "Wellington",
+      "longitude": "174.767",
+      "latitude": "-41.2833",
+      "population": {
+        "-year": "87",
+        "#text": "352030"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-TeIkaAMaui" }
+    },
+    {
+      "-id": "stadt-Auckland-NZ-NZ",
+      "-country": "NZ",
+      "name": "Auckland",
+      "longitude": "175",
+      "latitude": "-36",
+      "population": {
+        "-year": "87",
+        "#text": "952600"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-TeIkaAMaui" }
+    },
+    {
+      "-id": "stadt-Christchurch-NZ-NZ",
+      "-country": "NZ",
+      "name": "Christchurch",
+      "longitude": "172.66",
+      "latitude": "-43.5",
+      "population": {
+        "-year": "87",
+        "#text": "952600"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-TeWakaAMaui" }
+    }
+  ]
+}
+{
+  "-car_code": "NIUE",
+  "-area": "260",
+  "-capital": "city-Alofi-NIUE-NIUE",
+  "-memberships": "org-ACP org-FAO org-IFAD org-Sparteca org-SPC org-UNESCO org-UPU org-WHO org-WMO",
+  "name": "Niue",
+  "population": "1398",
+  "gdp_total": "10.01",
+  "gdp_agri": "23.5",
+  "gdp_ind": "26.9",
+  "gdp_serv": "49.5",
+  "inflation": "4",
+  "indep_date": "1974-10-19",
+  "government": "self governing territory in free association with New Zealand",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Polynesian"
+  },
+  "religions": [
+    {
+      "-percentage": "10",
+      "#text": "Mormon"
+    },
+    {
+      "-percentage": "61",
+      "#text": "Ekalesia Niue"
+    },
+    {
+      "-percentage": "7.2",
+      "#text": "Roman Catholic"
+    }
+  ],
+  "city": {
+    "-id": "city-Alofi-NIUE-NIUE",
+    "-is_country_cap": "yes",
+    "-country": "NIUE",
+    "name": "Alofi",
+    "longitude": "-169.9",
+    "latitude": "-19.05",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Niue" }
+  }
+}
+{
+  "-car_code": "NORF",
+  "-area": "34.6",
+  "-capital": "cty-cid-cia-NORF-Kingston",
+  "name": "Norfolk Island",
+  "population": "2209",
+  "dependent": { "-country": "AUS" },
+  "government": "territory of Australia",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "11.7",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "39",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "4.4",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "16.4",
+      "#text": "Uniting Church Australia"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-NORF-Kingston",
+    "-is_country_cap": "yes",
+    "-country": "NORF",
+    "name": "Kingston",
+    "longitude": "168",
+    "latitude": "-29",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Norfolk" }
+  }
+}
+{
+  "-car_code": "NMIS",
+  "-area": "477",
+  "-capital": "city-Saipan-NMIS-NMIS",
+  "-memberships": "org-ESCAP org-SPC",
+  "name": "Northern Mariana Islands",
+  "population": "88662",
+  "population_growth": "2.29",
+  "infant_mortality": "6.7",
+  "gdp_total": "633.4",
+  "inflation": "-0.8",
+  "dependent": { "-country": "USA" },
+  "government": "commonwealth in political union with the US",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "56.3",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "36.3",
+      "#text": "Pacific Islander"
+    },
+    {
+      "-percentage": "1.8",
+      "#text": "Caucasian"
+    }
+  ],
+  "languages": [
+    {
+      "-percentage": "24.4",
+      "#text": "Phillipine Languages"
+    },
+    {
+      "-percentage": "23.4",
+      "#text": "Chinese Languages"
+    },
+    {
+      "-percentage": "22.4",
+      "#text": "Chamorro"
+    },
+    {
+      "-percentage": "10.8",
+      "#text": "English"
+    },
+    {
+      "-percentage": "9.5",
+      "#text": "Other Pacific Island Languages"
+    }
+  ],
+  "city": {
+    "-id": "city-Saipan-NMIS-NMIS",
+    "-is_country_cap": "yes",
+    "-country": "NMIS",
+    "name": "Saipan",
+    "longitude": "145.45",
+    "latitude": "15.12",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Saipan" }
+  }
+}
+{
+  "-car_code": "PAL",
+  "-area": "458",
+  "-capital": "cty-cid-cia-Palau-Melekeok",
+  "-memberships": "org-ESCAP org-SPC org-SPF org-UN org-WHO",
+  "name": "Palau",
+  "population": "16952",
+  "population_growth": "1.71",
+  "infant_mortality": "25.07",
+  "gdp_total": "81.8",
+  "indep_date": "1994-10-01",
+  "government": "constitutional government in free association with the US",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "city": [
+    {
+      "-id": "cty-cid-cia-Palau-Koror",
+      "-country": "PAL",
+      "name": "Koror",
+      "longitude": "134.5",
+      "latitude": "7.3",
+      "population": {
+        "-year": "87",
+        "#text": "7685"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Koror" }
+    },
+    {
+      "-id": "cty-cid-cia-Palau-Melekeok",
+      "-is_country_cap": "yes",
+      "-country": "PAL",
+      "name": "Melekeok",
+      "longitude": "134.6",
+      "latitude": "7.5",
+      "population": {
+        "-year": "09",
+        "#text": "391"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      },
+      "located_on": { "-island": "island-Babelthuap" }
+    }
+  ]
+}
+{
+  "-car_code": "PITC",
+  "-area": "47",
+  "-capital": "city-Adamstown-PITC-PITC",
+  "-memberships": "org-SPC org-UPU",
+  "name": "Pitcairn Islands",
+  "population": "48",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Bounty"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Seventh-Day Adventist"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Pitkern"
+  },
+  "city": {
+    "-id": "city-Adamstown-PITC-PITC",
+    "-is_country_cap": "yes",
+    "-country": "PITC",
+    "name": "Adamstown",
+    "longitude": "-130.05",
+    "latitude": "-25.04",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Pitcairn" }
+  }
+}
+{
+  "-car_code": "SLB",
+  "-area": "28450",
+  "-capital": "cty-cid-cia-Solomon-Islands-Honiara",
+  "-memberships": "org-ACP org-AsDB org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-SPC org-SPF org-Sparteca org-UN org-UNESCO org-UPU org-WFTU org-WHO org-WMO",
+  "name": "Solomon Islands",
+  "population": "412902",
+  "population_growth": "3.35",
+  "infant_mortality": "25.8",
+  "gdp_total": "1000",
+  "inflation": "13",
+  "indep_date": "1978-07-07",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.3",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "0.8",
+      "#text": "European"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Polynesian"
+    },
+    {
+      "-percentage": "1.5",
+      "#text": "Micronesian"
+    },
+    {
+      "-percentage": "93",
+      "#text": "Melanesian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "19",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "34",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Baptist"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "11",
+      "#text": "United"
+    }
+  ],
+  "languages": {
+    "-percentage": "1",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Solomon-Islands-Honiara",
+    "-is_country_cap": "yes",
+    "-country": "SLB",
+    "name": "Honiara",
+    "longitude": "160",
+    "latitude": "-9.5",
+    "population": {
+      "-year": "87",
+      "#text": "26000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Guadalcanal" }
+  }
+}
+{
+  "-car_code": "TO",
+  "-area": "748",
+  "-capital": "cty-cid-cia-Tonga-Nukualofa",
+  "-memberships": "org-ACP org-AsDB org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-SPC org-SPF org-Sparteca org-UNCTAD org-UNESCO org-UNIDO org-UPU org-WHO",
+  "name": "Tonga",
+  "population": "106466",
+  "population_growth": "0.82",
+  "infant_mortality": "40.26",
+  "gdp_total": "228",
+  "gdp_agri": "40",
+  "inflation": "3",
+  "indep_date": "1970-06-04",
+  "government": "hereditary constitutional monarchy",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Christian"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Tonga-Nukualofa",
+    "-is_country_cap": "yes",
+    "-country": "TO",
+    "name": "Nukualofa",
+    "longitude": "-175.2",
+    "latitude": "-21.1",
+    "population": {
+      "-year": "87",
+      "#text": "28899"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Tongatapu" }
+  }
+}
+{
+  "-car_code": "TUV",
+  "-area": "26",
+  "-capital": "cty-cid-cia-Tuvalu-Funafuti",
+  "-memberships": "org-ACP org-AsDB org-C org-ESCAP org-IFRCS org-Intelsat org-SPC org-SPF org-Sparteca org-UNESCO org-UPU org-WHO",
+  "name": "Tuvalu",
+  "population": "10146",
+  "population_growth": "1.51",
+  "infant_mortality": "27.6",
+  "gdp_total": "7.8",
+  "inflation": "2.9",
+  "indep_date": "1978-10-01",
+  "government": "democracy",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "96",
+    "#text": "Polynesian"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Bahai"
+    },
+    {
+      "-percentage": "1.4",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "97",
+      "#text": "Church Tuvalu"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Tuvalu-Funafuti",
+    "-is_country_cap": "yes",
+    "-country": "TUV",
+    "name": "Funafuti",
+    "longitude": "178",
+    "latitude": "-8",
+    "population": {
+      "-year": "87",
+      "#text": "2120"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Fongafale" }
+  }
+}
+{
+  "-car_code": "VU",
+  "-area": "14760",
+  "-capital": "cty-cid-cia-Vanuatu-Port-Vila",
+  "-memberships": "org-ACP org-ACCT org-AsDB org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICAO org-IDA org-IFRCS org-IFC org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-SPC org-SPF org-Sparteca org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO",
+  "name": "Vanuatu",
+  "population": "177504",
+  "population_growth": "2.17",
+  "infant_mortality": "64.6",
+  "gdp_total": "210",
+  "inflation": "7",
+  "indep_date": "1980-07-30",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "French"
+    },
+    {
+      "-percentage": "94",
+      "#text": "Melanesian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "15",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Anglican"
+    },
+    {
+      "-percentage": "6.2",
+      "#text": "Seventh-Day Adventist"
+    },
+    {
+      "-percentage": "36.7",
+      "#text": "Presbyterian"
+    },
+    {
+      "-percentage": "3.8",
+      "#text": "Church Christ"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Vanuatu-Port-Vila",
+    "-is_country_cap": "yes",
+    "-country": "VU",
+    "name": "Port-Vila",
+    "longitude": "168.3",
+    "latitude": "-17.75",
+    "population": {
+      "-year": "87",
+      "#text": "15759"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Efate" }
+  }
+}
+{
+  "-car_code": "WAFU",
+  "-area": "274",
+  "-capital": "city-Mata-Utu-WAFU-WAFU",
+  "-memberships": "org-SPC org-UPU org-WFTU",
+  "name": "Wallis and Futuna",
+  "population": "15289",
+  "population_growth": "0.35",
+  "infant_mortality": "5.02",
+  "gdp_total": "60",
+  "dependent": { "-country": "F" },
+  "government": "overseas territory of France",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Polynesian"
+  },
+  "religions": {
+    "-percentage": "99",
+    "#text": "Roman Catholic"
+  },
+  "languages": [
+    {
+      "-percentage": "58.9",
+      "#text": "Wallisian"
+    },
+    {
+      "-percentage": "30.1",
+      "#text": "Futunian"
+    },
+    {
+      "-percentage": "10.8",
+      "#text": "French"
+    }
+  ],
+  "city": {
+    "-id": "city-Mata-Utu-WAFU-WAFU",
+    "-is_country_cap": "yes",
+    "-country": "WAFU",
+    "name": "Mata-Utu",
+    "longitude": "-171.56",
+    "latitude": "-13.57",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Uvea" }
+  }
+}
+{
+  "-car_code": "WS",
+  "-area": "2860",
+  "-capital": "cty-cid-cia-Samoa-Apia",
+  "-memberships": "org-ACP org-AsDB org-C org-ESCAP org-FAO org-G-77 org-IBRD org-ICFTU org-IDA org-IFRCS org-IFC org-IFAD org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-SPC org-SPF org-Sparteca org-UN org-UNESCO org-UPU org-WHO org-WMO",
+  "name": "Samoa",
+  "population": "214384",
+  "population_growth": "2.37",
+  "infant_mortality": "34.3",
+  "gdp_total": "415",
+  "gdp_agri": "50",
+  "inflation": "18",
+  "indep_date": "1962-01-01",
+  "government": "constitutional monarchy under native chief",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.4",
+      "#text": "Europeans"
+    },
+    {
+      "-percentage": "92.6",
+      "#text": "Samoan"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Euronesians"
+    }
+  ],
+  "religions": {
+    "-percentage": "99.7",
+    "#text": "Christian"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Samoa-Apia",
+    "-is_country_cap": "yes",
+    "-country": "WS",
+    "name": "Apia",
+    "longitude": "-171",
+    "latitude": "-12",
+    "population": {
+      "-year": "87",
+      "#text": "36000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Pacific"
+    },
+    "located_on": { "-island": "island-Upolu" }
+  }
+}
+{
+  "-car_code": "RA",
+  "-area": "2766890",
+  "-capital": "cty-Argentina-Buenos-Aires",
+  "-memberships": "org-AfDB org-OPANAL org-AG org-BCIE org-CCC org-ECLAC org-FAO org-G-6 org-G-11 org-G-15 org-G-19 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-MTCR org-NSG org-OAS org-PCA org-RG org-Mercosur org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNFICYP org-UNIDO org-UNITAR org-UNIKOM org-MINURSO org-UNMIH org-UNHCR org-UNTSO org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Argentina",
+  "population": "34672997",
+  "population_growth": "1.1",
+  "infant_mortality": "28.3",
+  "gdp_total": "278500",
+  "gdp_agri": "6",
+  "gdp_ind": "31",
+  "gdp_serv": "63",
+  "inflation": "1.7",
+  "indep_date": "1816-07-09",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Indian"
+  },
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "90",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BOL",
+      "-length": "832"
+    },
+    {
+      "-country": "BR",
+      "-length": "1224"
+    },
+    {
+      "-country": "RCH",
+      "-length": "5150"
+    },
+    {
+      "-country": "PY",
+      "-length": "1880"
+    },
+    {
+      "-country": "ROU",
+      "-length": "579"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Argentina-2",
+      "-capital": "cty-cid-cia-Argentina-7",
+      "-country": "RA",
+      "name": "Buenos Aires",
+      "area": "307571",
+      "population": "12594974",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Argentina-7",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "La Plata",
+          "longitude": "-58",
+          "latitude": "-35",
+          "population": {
+            "-year": "91",
+            "#text": "642979"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-5",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "La Matanza",
+          "population": {
+            "-year": "91",
+            "#text": "1111811"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-8",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Moron",
+          "population": {
+            "-year": "91",
+            "#text": "641541"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-10",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Lomas de Zamoras",
+          "population": {
+            "-year": "91",
+            "#text": "572769"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-11",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Mar del Plata",
+          "population": {
+            "-year": "91",
+            "#text": "512880"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-12",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Quilmes",
+          "population": {
+            "-year": "91",
+            "#text": "509445"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-13",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Lanus",
+          "population": {
+            "-year": "91",
+            "#text": "466755"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-14",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "General San Martin",
+          "population": {
+            "-year": "91",
+            "#text": "407506"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-18",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Avellaneda",
+          "population": {
+            "-year": "91",
+            "#text": "346620"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-19",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "San Isidro",
+          "population": {
+            "-year": "91",
+            "#text": "299022"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-21",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Vicente Lopez",
+          "population": {
+            "-year": "91",
+            "#text": "289142"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-23",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "Bahia Blanca",
+          "population": {
+            "-year": "91",
+            "#text": "260096"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-33",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-2",
+          "name": "San Nicolas de los Arroyos",
+          "population": {
+            "-year": "91",
+            "#text": "119302"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-3",
+      "-capital": "cty-cid-cia-Argentina-35",
+      "-country": "RA",
+      "name": "Catamarca",
+      "area": "102602",
+      "population": "264324",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-35",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-3",
+        "name": "San Fernando del Valle de Catamarca",
+        "longitude": "-65.5",
+        "latitude": "-28.3",
+        "population": {
+          "-year": "91",
+          "#text": "132626"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-4",
+      "-capital": "cty-cid-cia-Argentina-20",
+      "-country": "RA",
+      "name": "Chaco",
+      "area": "99633",
+      "population": "839677",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-20",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-4",
+        "name": "Resistencia",
+        "longitude": "-59",
+        "latitude": "-27.3",
+        "population": {
+          "-year": "91",
+          "#text": "292350"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Parana"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-5",
+      "-capital": "cty-cid-cia-Argentina-Rawson",
+      "-country": "RA",
+      "name": "Chubut",
+      "area": "224686",
+      "population": "357189",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Argentina-32",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-5",
+          "name": "Comodoro Rivadavia",
+          "population": {
+            "-year": "91",
+            "#text": "124104"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-Rawson",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-5",
+          "name": "Rawson",
+          "longitude": "-65",
+          "latitude": "-43.1",
+          "population": {
+            "-year": "87",
+            "#text": "52000"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-6",
+      "-capital": "cty-Argentina-Cordoba",
+      "-country": "RA",
+      "name": "Cordoba",
+      "area": "165321",
+      "population": "2766683",
+      "city": [
+        {
+          "-id": "cty-Argentina-Cordoba",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-6",
+          "name": "Cordoba",
+          "longitude": "-64.25",
+          "latitude": "-31.3667",
+          "population": {
+            "-year": "91",
+            "#text": "1208713"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-30",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-6",
+          "name": "Rio Cuarto",
+          "population": {
+            "-year": "91",
+            "#text": "138853"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-7",
+      "-capital": "cty-cid-cia-Argentina-24",
+      "-country": "RA",
+      "name": "Corrientes",
+      "area": "88199",
+      "population": "795594",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-24",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-7",
+        "name": "Corrientes",
+        "longitude": "-58.4",
+        "latitude": "-27.3",
+        "population": {
+          "-year": "91",
+          "#text": "258103"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Parana"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-8",
+      "-capital": "cty-cid-cia-Argentina-26",
+      "-country": "RA",
+      "name": "Entre Rios",
+      "area": "78781",
+      "population": "1020257",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Argentina-26",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-8",
+          "name": "Parana",
+          "longitude": "-60",
+          "latitude": "-32",
+          "population": {
+            "-year": "91",
+            "#text": "211936"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Parana"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-34",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-8",
+          "name": "Concordia",
+          "population": {
+            "-year": "91",
+            "#text": "116485"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Uruguay"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-9",
+      "-capital": "cty-cid-cia-Argentina-29",
+      "-country": "RA",
+      "name": "Formosa",
+      "area": "72066",
+      "population": "398413",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-29",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-9",
+        "name": "Formosa",
+        "longitude": "-58.1",
+        "latitude": "-26.5",
+        "population": {
+          "-year": "91",
+          "#text": "148074"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Paraguay"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-10",
+      "-capital": "cty-cid-cia-Argentina-28",
+      "-country": "RA",
+      "name": "Jujuy",
+      "area": "53219",
+      "population": "512329",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-28",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-10",
+        "name": "San Salvador de Jujuy",
+        "longitude": "-65.2",
+        "latitude": "-24.1",
+        "population": {
+          "-year": "91",
+          "#text": "180102"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-11",
+      "-capital": "cty-cid-cia-Argentina-Santa-Rosa",
+      "-country": "RA",
+      "name": "La Pampa",
+      "area": "143440",
+      "population": "259996",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-Santa-Rosa",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-11",
+        "name": "Santa Rosa",
+        "longitude": "-64.2",
+        "latitude": "-36.4",
+        "population": {
+          "-year": "87",
+          "#text": "107000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-12",
+      "-capital": "cty-cid-cia-Argentina-37",
+      "-country": "RA",
+      "name": "La Rioja",
+      "area": "89680",
+      "population": "220729",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-37",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-12",
+        "name": "La Rioja",
+        "longitude": "-67",
+        "latitude": "-29.3",
+        "population": {
+          "-year": "91",
+          "#text": "103727"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-13",
+      "-capital": "cty-cid-cia-Argentina-6",
+      "-country": "RA",
+      "name": "Mendoza",
+      "area": "148827",
+      "population": "1412481",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-6",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-13",
+        "name": "Mendoza",
+        "longitude": "-69",
+        "latitude": "-33",
+        "population": {
+          "-year": "91",
+          "#text": "773113"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-14",
+      "-capital": "cty-cid-cia-Argentina-27",
+      "-country": "RA",
+      "name": "Misiones",
+      "area": "29801",
+      "population": "788915",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-27",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-14",
+        "name": "Posadas",
+        "longitude": "-56",
+        "latitude": "-27.3",
+        "population": {
+          "-year": "91",
+          "#text": "210755"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Parana"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-15",
+      "-capital": "cty-cid-cia-Argentina-25",
+      "-country": "RA",
+      "name": "Neuquen",
+      "area": "94078",
+      "population": "388833",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-25",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-15",
+        "name": "Neuquen",
+        "longitude": "-68",
+        "latitude": "-39",
+        "population": {
+          "-year": "91",
+          "#text": "243803"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-16",
+      "-capital": "cty-cid-cia-Argentina-Viedma",
+      "-country": "RA",
+      "name": "Rio Negro",
+      "area": "203013",
+      "population": "506772",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-Viedma",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-16",
+        "name": "Viedma",
+        "longitude": "-63.1",
+        "latitude": "-40.5",
+        "population": {
+          "-year": "87",
+          "#text": "25000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-17",
+      "-capital": "cty-cid-cia-Argentina-16",
+      "-country": "RA",
+      "name": "Salta",
+      "area": "155488",
+      "population": "866153",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-16",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-17",
+        "name": "Salta",
+        "longitude": "-65.3",
+        "latitude": "-24.5",
+        "population": {
+          "-year": "91",
+          "#text": "370904"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-18",
+      "-capital": "cty-cid-cia-Argentina-17",
+      "-country": "RA",
+      "name": "San Juan",
+      "area": "89651",
+      "population": "528715",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-17",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-18",
+        "name": "San Juan",
+        "longitude": "-68.3",
+        "latitude": "-31.3",
+        "population": {
+          "-year": "91",
+          "#text": "352691"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-19",
+      "-capital": "cty-cid-cia-Argentina-36",
+      "-country": "RA",
+      "name": "San Luis",
+      "area": "76748",
+      "population": "286458",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-36",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-19",
+        "name": "San Luis",
+        "longitude": "-66.2",
+        "latitude": "-33.2",
+        "population": {
+          "-year": "91",
+          "#text": "110136"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-20",
+      "-capital": "cty-cid-cia-Argentina-Rio-Gallegos",
+      "-country": "RA",
+      "name": "Santa Cruz",
+      "area": "243943",
+      "population": "159839",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-Rio-Gallegos",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-20",
+        "name": "Rio Gallegos",
+        "longitude": "-69.2",
+        "latitude": "-51.4",
+        "population": {
+          "-year": "87",
+          "#text": "43000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-21",
+      "-capital": "cty-cid-cia-Argentina-15",
+      "-country": "RA",
+      "name": "Santa Fe",
+      "area": "133007",
+      "population": "2798422",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Argentina-15",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-21",
+          "name": "Santa Fe",
+          "longitude": "-60.5",
+          "latitude": "-31.5",
+          "population": {
+            "-year": "91",
+            "#text": "406388"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Parana"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-4",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-21",
+          "name": "Rosario",
+          "longitude": "-60.5",
+          "latitude": "-33",
+          "population": {
+            "-year": "91",
+            "#text": "1118984"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Parana"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-22",
+      "-capital": "cty-cid-cia-Argentina-Santiago-del-Estero",
+      "-country": "RA",
+      "name": "Santiago de Estero",
+      "area": "136351",
+      "population": "671988",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Argentina-22",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-22",
+          "name": "Santiago del Estero La Banda",
+          "population": {
+            "-year": "91",
+            "#text": "263471"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Argentina-Santiago-del-Estero",
+          "-is_state_cap": "yes",
+          "-country": "RA",
+          "-province": "prov-cid-cia-Argentina-22",
+          "name": "Santiago del Estero",
+          "longitude": "-64.2",
+          "latitude": "-28",
+          "population": {
+            "-year": "87",
+            "#text": "148000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-23",
+      "-capital": "cty-cid-cia-Argentina-Ushuaia",
+      "-country": "RA",
+      "name": "Tierra del Fuego",
+      "area": "21571",
+      "population": "69369",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-Ushuaia",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-23",
+        "name": "Ushuaia",
+        "longitude": "-68.3",
+        "latitude": "-54.5",
+        "population": {
+          "-year": "87",
+          "#text": "3000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-24",
+      "-capital": "cty-cid-cia-Argentina-9",
+      "-country": "RA",
+      "name": "Tucuman",
+      "area": "22524",
+      "population": "1142105",
+      "city": {
+        "-id": "cty-cid-cia-Argentina-9",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-24",
+        "name": "San Miguel de Tucuman",
+        "longitude": "-65.2",
+        "latitude": "-26.5",
+        "population": {
+          "-year": "91",
+          "#text": "622324"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Argentina-26",
+      "-capital": "cty-Argentina-Buenos-Aires",
+      "-country": "RA",
+      "name": "Distrito Federal",
+      "area": "200",
+      "population": "2965403",
+      "city": {
+        "-id": "cty-Argentina-Buenos-Aires",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "RA",
+        "-province": "prov-cid-cia-Argentina-26",
+        "name": "Buenos Aires",
+        "longitude": "-58.4833",
+        "latitude": "-34.5833",
+        "population": {
+          "-year": "91",
+          "#text": "2988006"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Parana"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BOL",
+  "-area": "1098580",
+  "-capital": "cty-Bolivia-La-Paz",
+  "-memberships": "org-OPANAL org-AG org-ECLAC org-FAO org-G-11 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Bolivia",
+  "population": "7165257",
+  "population_growth": "1.82",
+  "infant_mortality": "67.5",
+  "gdp_total": "20000",
+  "inflation": "12",
+  "indep_date": "1825-08-06",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "5",
+      "#text": "European"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Quechua"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Aymara"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "RA",
+      "-length": "832"
+    },
+    {
+      "-country": "BR",
+      "-length": "3400"
+    },
+    {
+      "-country": "RCH",
+      "-length": "861"
+    },
+    {
+      "-country": "PY",
+      "-length": "750"
+    },
+    {
+      "-country": "PE",
+      "-length": "900"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Bolivia-La-Paz",
+      "-is_country_cap": "yes",
+      "-country": "BOL",
+      "name": "La Paz",
+      "longitude": "-68.15",
+      "latitude": "-16.5",
+      "population": {
+        "-year": "87",
+        "#text": "953400"
+      }
+    },
+    {
+      "-id": "stadt-Sucre-BOL-BOL",
+      "-country": "BOL",
+      "name": "Sucre",
+      "longitude": "-65.1",
+      "latitude": "-19",
+      "population": {
+        "-year": "87",
+        "#text": "110000"
+      }
+    },
+    {
+      "-id": "stadt-Cochabamba-BOL-BOL",
+      "-country": "BOL",
+      "name": "Cochabamba",
+      "longitude": "-66",
+      "latitude": "-17.1",
+      "population": {
+        "-year": "87",
+        "#text": "304000"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BR",
+  "-area": "8511965",
+  "-capital": "cty-Brazil-Brasilia",
+  "-memberships": "org-AfDB org-OPANAL org-AG org-CCC org-ECLAC org-FAO org-G-11 org-G-15 org-G-19 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-MTCR org-NAM org-NSG org-OAS org-PCA org-RG org-Mercosur org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNHCR org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Brazil",
+  "population": "162661214",
+  "population_growth": "1.16",
+  "infant_mortality": "55.3",
+  "gdp_total": "976800",
+  "gdp_agri": "16",
+  "gdp_ind": "25",
+  "gdp_serv": "59",
+  "inflation": "23",
+  "indep_date": "1822-09-07",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "44",
+    "#text": "African"
+  },
+  "religions": {
+    "-percentage": "70",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "CO",
+      "-length": "1643"
+    },
+    {
+      "-country": "RA",
+      "-length": "1224"
+    },
+    {
+      "-country": "BOL",
+      "-length": "3400"
+    },
+    {
+      "-country": "PY",
+      "-length": "1290"
+    },
+    {
+      "-country": "ROU",
+      "-length": "985"
+    },
+    {
+      "-country": "PE",
+      "-length": "1560"
+    },
+    {
+      "-country": "FGU",
+      "-length": "673"
+    },
+    {
+      "-country": "GUY",
+      "-length": "1119"
+    },
+    {
+      "-country": "SME",
+      "-length": "597"
+    },
+    {
+      "-country": "YV",
+      "-length": "2200"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Brazil-2",
+      "-capital": "cty-cid-cia-Brazil-85",
+      "-country": "BR",
+      "name": "Acre",
+      "area": "153149",
+      "population": "483483",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-85",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-2",
+        "name": "Rio Branco",
+        "longitude": "-67.5",
+        "latitude": "-9.5",
+        "population": {
+          "-year": "96",
+          "#text": "228907"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-3",
+      "-capital": "cty-cid-cia-Brazil-19",
+      "-country": "BR",
+      "name": "Alagoas",
+      "area": "27933",
+      "population": "2637843",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-19",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-3",
+        "name": "Maceio",
+        "longitude": "-35.5",
+        "latitude": "-9.3",
+        "population": {
+          "-year": "96",
+          "#text": "723156"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-4",
+      "-capital": "cty-cid-cia-Brazil-93",
+      "-country": "BR",
+      "name": "Amapa",
+      "area": "143453",
+      "population": "373994",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-93",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-4",
+        "name": "Macapa",
+        "longitude": "-52",
+        "latitude": "0.5",
+        "population": {
+          "-year": "96",
+          "#text": "214197"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amazonas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-5",
+      "-capital": "cty-cid-cia-Brazil-12",
+      "-country": "BR",
+      "name": "Amazonas",
+      "area": "1577820",
+      "population": "2390102",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-12",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-5",
+        "name": "Manaus",
+        "longitude": "-60",
+        "latitude": "-3.1",
+        "population": {
+          "-year": "96",
+          "#text": "1158265"
+        },
+        "located_at": [
+          {
+            "-watertype": "river",
+            "-river": "river-Amazonas"
+          },
+          {
+            "-watertype": "river",
+            "-river": "river-RioNegro"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-6",
+      "-capital": "cty-Brazil-Salvador",
+      "-country": "BR",
+      "name": "Bahia",
+      "area": "567295",
+      "population": "12531895",
+      "city": [
+        {
+          "-id": "cty-Brazil-Salvador",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Salvador",
+          "longitude": "-38.5",
+          "latitude": "-13",
+          "population": {
+            "-year": "96",
+            "#text": "2209465"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-33",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Feira de Santana",
+          "population": {
+            "-year": "96",
+            "#text": "449960"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-75",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Ilheus",
+          "population": {
+            "-year": "96",
+            "#text": "242589"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-76",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Vitoria da Conquista",
+          "population": {
+            "-year": "96",
+            "#text": "241776"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-104",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Itabuna",
+          "population": {
+            "-year": "96",
+            "#text": "183237"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-115",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Juazeiro",
+          "population": {
+            "-year": "96",
+            "#text": "171904"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-RioSaoFrancisco"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-123",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Jequie",
+          "population": {
+            "-year": "96",
+            "#text": "165361"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-150",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Camacari",
+          "population": {
+            "-year": "96",
+            "#text": "134768"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-157",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Alagoinhas",
+          "population": {
+            "-year": "96",
+            "#text": "122882"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-172",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-6",
+          "name": "Barreiras",
+          "population": {
+            "-year": "96",
+            "#text": "113544"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-7",
+      "-capital": "cty-Brazil-Fortaleza",
+      "-country": "BR",
+      "name": "Ceara",
+      "area": "146348",
+      "population": "6803567",
+      "city": [
+        {
+          "-id": "cty-Brazil-Fortaleza",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-7",
+          "name": "Fortaleza",
+          "longitude": "-38.55",
+          "latitude": "-3.76667",
+          "population": {
+            "-year": "96",
+            "#text": "1967365"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-94",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-7",
+          "name": "Caucaia",
+          "population": {
+            "-year": "96",
+            "#text": "208902"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-103",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-7",
+          "name": "Juazeiro do Norte",
+          "population": {
+            "-year": "96",
+            "#text": "186490"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-129",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-7",
+          "name": "Maracanau",
+          "population": {
+            "-year": "96",
+            "#text": "160088"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-147",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-7",
+          "name": "Sobral",
+          "population": {
+            "-year": "96",
+            "#text": "138274"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-8",
+      "-capital": "cty-cid-cia-Brazil-71",
+      "-country": "BR",
+      "name": "Espirito Santo",
+      "area": "46184",
+      "population": "2786126",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-56",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Cariacica",
+          "population": {
+            "-year": "96",
+            "#text": "300085"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-57",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Vila Velha",
+          "population": {
+            "-year": "96",
+            "#text": "297052"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-68",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Serra",
+          "population": {
+            "-year": "96",
+            "#text": "266851"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-71",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Vitoria",
+          "longitude": "-40.3",
+          "latitude": "-20.2",
+          "population": {
+            "-year": "96",
+            "#text": "263708"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-142",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Cachoeiro de Itapemirim",
+          "population": {
+            "-year": "96",
+            "#text": "147283"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-192",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Colatina",
+          "population": {
+            "-year": "96",
+            "#text": "103621"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-195",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-8",
+          "name": "Linhares",
+          "population": {
+            "-year": "96",
+            "#text": "103030"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-9",
+      "-capital": "cty-cid-cia-Brazil-13",
+      "-country": "BR",
+      "name": "Goias",
+      "area": "341289",
+      "population": "4501538",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-13",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-9",
+          "name": "Goiania",
+          "longitude": "-49.2",
+          "latitude": "-16.3",
+          "population": {
+            "-year": "96",
+            "#text": "998520"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-70",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-9",
+          "name": "Anapolis",
+          "population": {
+            "-year": "96",
+            "#text": "264873"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-187",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-9",
+          "name": "Araguaina",
+          "population": {
+            "-year": "96",
+            "#text": "105000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-189",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-9",
+          "name": "Luziania",
+          "population": {
+            "-year": "96",
+            "#text": "104840"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-203",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-9",
+          "name": "Rio Verde",
+          "population": {
+            "-year": "96",
+            "#text": "100223"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-10",
+      "-capital": "cty-cid-cia-Brazil-18",
+      "-country": "BR",
+      "name": "Maranhao",
+      "area": "333365",
+      "population": "5218442",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-18",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Sao Luis",
+          "longitude": "-44.1",
+          "latitude": "-2.3",
+          "population": {
+            "-year": "96",
+            "#text": "781068"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-87",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Imperatriz",
+          "population": {
+            "-year": "96",
+            "#text": "227731"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-152",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Caxias",
+          "population": {
+            "-year": "96",
+            "#text": "133710"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-164",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Timon",
+          "population": {
+            "-year": "96",
+            "#text": "118332"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-190",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Codo",
+          "population": {
+            "-year": "96",
+            "#text": "104825"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-156",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-10",
+          "name": "Parnaiba",
+          "population": {
+            "-year": "96",
+            "#text": "124641"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-11",
+      "-capital": "cty-cid-cia-Brazil-36",
+      "-country": "BR",
+      "name": "Mato Grosso",
+      "area": "906806",
+      "population": "2227983",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-36",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-11",
+          "name": "Cuiaba",
+          "longitude": "-56",
+          "latitude": "-15.3",
+          "population": {
+            "-year": "96",
+            "#text": "433101"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-102",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-11",
+          "name": "Varzea Grande",
+          "population": {
+            "-year": "96",
+            "#text": "190374"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-144",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-11",
+          "name": "Rondonopolis",
+          "population": {
+            "-year": "96",
+            "#text": "142221"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-12",
+      "-capital": "cty-cid-cia-Brazil-26",
+      "-country": "BR",
+      "name": "Mato Grosso do Sul",
+      "area": "358158",
+      "population": "1922258",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-26",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-12",
+          "name": "Campo Grande",
+          "population": {
+            "-year": "96",
+            "#text": "598216"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-136",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-12",
+          "name": "Dourados",
+          "population": {
+            "-year": "96",
+            "#text": "153165"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-13",
+      "-capital": "cty-Brazil-Belo-Horizonte",
+      "-country": "BR",
+      "name": "Minas Gerais",
+      "area": "588383",
+      "population": "16660691",
+      "city": [
+        {
+          "-id": "cty-Brazil-Belo-Horizonte",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Belo Horizonte",
+          "longitude": "-43.95",
+          "latitude": "-19.9333",
+          "population": {
+            "-year": "96",
+            "#text": "2091770"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-29",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Contagem",
+          "population": {
+            "-year": "96",
+            "#text": "491268"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-73",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Betim",
+          "population": {
+            "-year": "96",
+            "#text": "249795"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-97",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Ribeirao das Neves",
+          "population": {
+            "-year": "96",
+            "#text": "197147"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-183",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Ibirite",
+          "population": {
+            "-year": "96",
+            "#text": "106773"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-34",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Uberlandia",
+          "population": {
+            "-year": "96",
+            "#text": "437111"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-39",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Juiz de Fora",
+          "population": {
+            "-year": "96",
+            "#text": "423913"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-63",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Montes Claros",
+          "population": {
+            "-year": "96",
+            "#text": "271324"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-78",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Uberaba",
+          "population": {
+            "-year": "96",
+            "#text": "232134"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-81",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Governador Valadares",
+          "population": {
+            "-year": "96",
+            "#text": "231077"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-99",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Ipatinga",
+          "population": {
+            "-year": "96",
+            "#text": "195736"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-116",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Divinopolis",
+          "population": {
+            "-year": "96",
+            "#text": "171562"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-122",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Sete Lagoas",
+          "population": {
+            "-year": "96",
+            "#text": "167112"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-154",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Teofilo Otoni",
+          "population": {
+            "-year": "96",
+            "#text": "127214"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-162",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Pocos de Caldas",
+          "population": {
+            "-year": "96",
+            "#text": "121845"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-174",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Patos de Minas",
+          "population": {
+            "-year": "96",
+            "#text": "112690"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-178",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Itapecerica da Serra",
+          "population": {
+            "-year": "96",
+            "#text": "110362"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-182",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Barbacena",
+          "population": {
+            "-year": "96",
+            "#text": "107843"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-200",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Sabara",
+          "population": {
+            "-year": "96",
+            "#text": "100570"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-204",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-13",
+          "name": "Varginha",
+          "population": {
+            "-year": "96",
+            "#text": "100079"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-14",
+      "-capital": "cty-Brazil-Belem",
+      "-country": "BR",
+      "name": "Para",
+      "area": "1253164",
+      "population": "5522783",
+      "city": [
+        {
+          "-id": "cty-Brazil-Belem",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Belem",
+          "longitude": "-48.4833",
+          "latitude": "-1.45",
+          "population": {
+            "-year": "96",
+            "#text": "1142258"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Tocantins"
+            },
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-184",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Abaetetuba",
+          "population": {
+            "-year": "96",
+            "#text": "106753"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-47",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Ananindeua",
+          "population": {
+            "-year": "96",
+            "#text": "342905"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-74",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Santarem",
+          "population": {
+            "-year": "96",
+            "#text": "242765"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-143",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Maraba",
+          "population": {
+            "-year": "96",
+            "#text": "147030"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-165",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Castanhal",
+          "population": {
+            "-year": "96",
+            "#text": "117747"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-188",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-14",
+          "name": "Itaituba",
+          "population": {
+            "-year": "96",
+            "#text": "104896"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-15",
+      "-capital": "cty-cid-cia-Brazil-27",
+      "-country": "BR",
+      "name": "Paraiba",
+      "area": "56584",
+      "population": "3305562",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-27",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-15",
+          "name": "Joao Pessoa",
+          "longitude": "-35",
+          "latitude": "-7.1",
+          "population": {
+            "-year": "96",
+            "#text": "549270"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-48",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-15",
+          "name": "Campina Grande",
+          "population": {
+            "-year": "96",
+            "#text": "340412"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-138",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-15",
+          "name": "Santa Luzia",
+          "population": {
+            "-year": "96",
+            "#text": "152682"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-186",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-15",
+          "name": "Santa Rita",
+          "population": {
+            "-year": "96",
+            "#text": "105584"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-16",
+      "-capital": "cty-Brazil-Curitiba",
+      "-country": "BR",
+      "name": "Parana",
+      "area": "199709",
+      "population": "8985981",
+      "city": [
+        {
+          "-id": "cty-Brazil-Curitiba",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Curitiba",
+          "longitude": "-49.2833",
+          "latitude": "-25.4167",
+          "population": {
+            "-year": "96",
+            "#text": "1465698"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-121",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Sao Jose dos Pinhais",
+          "population": {
+            "-year": "96",
+            "#text": "167286"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-40",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Londrina",
+          "population": {
+            "-year": "96",
+            "#text": "412894"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-65",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Maringa",
+          "population": {
+            "-year": "96",
+            "#text": "267878"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-72",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Ponta Grossa",
+          "population": {
+            "-year": "96",
+            "#text": "252674"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-80",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Foz do Iguacu",
+          "population": {
+            "-year": "96",
+            "#text": "231596"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-92",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Cascavel",
+          "population": {
+            "-year": "96",
+            "#text": "219623"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-135",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Colombo",
+          "population": {
+            "-year": "96",
+            "#text": "153583"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-140",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Guarapuava",
+          "population": {
+            "-year": "96",
+            "#text": "149158"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-166",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Paranagua",
+          "population": {
+            "-year": "96",
+            "#text": "115394"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-197",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-16",
+          "name": "Apucarana",
+          "population": {
+            "-year": "96",
+            "#text": "101005"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-17",
+      "-capital": "cty-Brazil-Recife",
+      "-country": "BR",
+      "name": "Pernambuco",
+      "area": "98937",
+      "population": "7404559",
+      "city": [
+        {
+          "-id": "cty-Brazil-Recife",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Recife",
+          "longitude": "-34.8833",
+          "latitude": "-8.06667",
+          "population": {
+            "-year": "96",
+            "#text": "1342877"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-148",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Cabo de Santo Agostinho",
+          "population": {
+            "-year": "96",
+            "#text": "137295"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-28",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Jaboatao dos Guararapes",
+          "population": {
+            "-year": "96",
+            "#text": "528973"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-77",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Paulista",
+          "population": {
+            "-year": "96",
+            "#text": "234706"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-79",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Caruaru",
+          "population": {
+            "-year": "96",
+            "#text": "231874"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-96",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Gravatai",
+          "population": {
+            "-year": "96",
+            "#text": "205657"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-101",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Petrolina",
+          "population": {
+            "-year": "96",
+            "#text": "190453"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-176",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Vitoria de Santo Antao",
+          "population": {
+            "-year": "96",
+            "#text": "111575"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-177",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Camaragibe",
+          "population": {
+            "-year": "96",
+            "#text": "111190"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-179",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-17",
+          "name": "Garanhuns",
+          "population": {
+            "-year": "96",
+            "#text": "110190"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-18",
+      "-capital": "cty-cid-cia-Brazil-23",
+      "-country": "BR",
+      "name": "Piaui",
+      "area": "252378",
+      "population": "2676098",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-23",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-18",
+        "name": "Teresina",
+        "longitude": "-42.5",
+        "latitude": "-5",
+        "population": {
+          "-year": "96",
+          "#text": "653994"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-19",
+      "-capital": "cty-cid-cia-Brazil-3",
+      "-country": "BR",
+      "name": "Rio de Janeiro",
+      "area": "43909",
+      "population": "13316455",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-3",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Rio de Janeiro",
+          "longitude": "-43.2",
+          "latitude": "-23",
+          "population": {
+            "-year": "96",
+            "#text": "5533011"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            },
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Atlantic"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-44",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Campos dos Goytacazes",
+          "population": {
+            "-year": "96",
+            "#text": "391299"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-16",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Sao Goncalo",
+          "population": {
+            "-year": "96",
+            "#text": "827967"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-17",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Novo Iguacu",
+          "population": {
+            "-year": "96",
+            "#text": "801036"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-20",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Duque de Caxias",
+          "population": {
+            "-year": "96",
+            "#text": "712370"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-32",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Niteroi",
+          "population": {
+            "-year": "96",
+            "#text": "450129"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-35",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Sao Joao de Meriti",
+          "population": {
+            "-year": "96",
+            "#text": "433713"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-43",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Belfort Roxo",
+          "population": {
+            "-year": "96",
+            "#text": "393520"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-45",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Olinda",
+          "population": {
+            "-year": "96",
+            "#text": "355741"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-66",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Petropolis",
+          "population": {
+            "-year": "96",
+            "#text": "267604"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-84",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Volta Redonda",
+          "population": {
+            "-year": "96",
+            "#text": "228939"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-108",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Mage",
+          "population": {
+            "-year": "96",
+            "#text": "180550"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-117",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Nova Friburgo",
+          "population": {
+            "-year": "96",
+            "#text": "169218"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-124",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Barra Mansa",
+          "population": {
+            "-year": "96",
+            "#text": "164768"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-131",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Itaborai",
+          "population": {
+            "-year": "96",
+            "#text": "156631"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-133",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Nilopolis",
+          "population": {
+            "-year": "96",
+            "#text": "155190"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-158",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Teresopolis",
+          "population": {
+            "-year": "96",
+            "#text": "122614"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-173",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Macae",
+          "population": {
+            "-year": "96",
+            "#text": "112966"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-181",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Queimados",
+          "population": {
+            "-year": "96",
+            "#text": "108531"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-196",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-19",
+          "name": "Cabo Frio",
+          "population": {
+            "-year": "96",
+            "#text": "101142"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-20",
+      "-capital": "cty-cid-cia-Brazil-22",
+      "-country": "BR",
+      "name": "Rio Grande do Norte",
+      "area": "53306",
+      "population": "2556939",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-22",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-20",
+          "name": "Natal",
+          "longitude": "-35.1",
+          "latitude": "-5.5",
+          "population": {
+            "-year": "96",
+            "#text": "656244"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-95",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-20",
+          "name": "Mossoro",
+          "population": {
+            "-year": "96",
+            "#text": "205732"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-21",
+      "-capital": "cty-Brazil-Porto-Alegre",
+      "-country": "BR",
+      "name": "Rio Grande do Sul",
+      "area": "282062",
+      "population": "9623003",
+      "city": [
+        {
+          "-id": "cty-Brazil-Porto-Alegre",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Porto Alegre",
+          "longitude": "-51.2167",
+          "latitude": "-30.0333",
+          "population": {
+            "-year": "96",
+            "#text": "1286251"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-128",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Alvorada",
+          "population": {
+            "-year": "96",
+            "#text": "160671"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-171",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Sapucaia do Sul",
+          "population": {
+            "-year": "96",
+            "#text": "113992"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-50",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Caxias do Sul",
+          "population": {
+            "-year": "96",
+            "#text": "326222"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-54",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Pelotas",
+          "population": {
+            "-year": "96",
+            "#text": "304285"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-61",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Canoas",
+          "population": {
+            "-year": "96",
+            "#text": "284114"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-88",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Santa Maria",
+          "population": {
+            "-year": "96",
+            "#text": "226226"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-90",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Novo Hamburgo",
+          "population": {
+            "-year": "96",
+            "#text": "225082"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-98",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Viamao",
+          "population": {
+            "-year": "96",
+            "#text": "195880"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-106",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Sao Leopoldo",
+          "population": {
+            "-year": "96",
+            "#text": "180741"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-109",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Rio Grande",
+          "population": {
+            "-year": "96",
+            "#text": "178223"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-132",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Passo Fundo",
+          "population": {
+            "-year": "96",
+            "#text": "156352"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-163",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Uruguaiana",
+          "population": {
+            "-year": "96",
+            "#text": "121825"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Uruguay"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-170",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Bage",
+          "population": {
+            "-year": "96",
+            "#text": "114091"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-201",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-21",
+          "name": "Santa Cruz do Sul",
+          "population": {
+            "-year": "96",
+            "#text": "100562"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-22",
+      "-capital": "cty-cid-cia-Brazil-58",
+      "-country": "BR",
+      "name": "Rondonia",
+      "area": "238512",
+      "population": "1221290",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-58",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-22",
+        "name": "Porto Velho",
+        "longitude": "-63.5",
+        "latitude": "-8.5",
+        "population": {
+          "-year": "96",
+          "#text": "293815"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-RioMadeira"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-23",
+      "-capital": "cty-cid-cia-Brazil-134",
+      "-country": "BR",
+      "name": "Roraima",
+      "area": "225116",
+      "population": "247724",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-134",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-23",
+        "name": "Boa Vista",
+        "longitude": "-60.4",
+        "latitude": "2.5",
+        "population": {
+          "-year": "96",
+          "#text": "154166"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-24",
+      "-capital": "cty-cid-cia-Brazil-64",
+      "-country": "BR",
+      "name": "Santa Catarina",
+      "area": "95442",
+      "population": "4865090",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-42",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Joinvile",
+          "population": {
+            "-year": "96",
+            "#text": "397987"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-64",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Florianopolis",
+          "longitude": "-58.3",
+          "latitude": "-27.3",
+          "population": {
+            "-year": "96",
+            "#text": "268551"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-82",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Blumenau",
+          "population": {
+            "-year": "96",
+            "#text": "230988"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-130",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Criciuma",
+          "population": {
+            "-year": "96",
+            "#text": "159033"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-139",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Praia Grande",
+          "population": {
+            "-year": "96",
+            "#text": "150574"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-141",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Sao Jose",
+          "population": {
+            "-year": "96",
+            "#text": "147490"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-146",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Lages",
+          "population": {
+            "-year": "96",
+            "#text": "140005"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-149",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Itajai",
+          "population": {
+            "-year": "96",
+            "#text": "134797"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-153",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-24",
+          "name": "Chapeco",
+          "population": {
+            "-year": "96",
+            "#text": "130458"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-25",
+      "-capital": "cty-Brazil-Sao-Paulo",
+      "-country": "BR",
+      "name": "Sao Paulo",
+      "area": "248808",
+      "population": "34055715",
+      "city": [
+        {
+          "-id": "cty-Brazil-Sao-Paulo",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Paulo",
+          "longitude": "-46.6333",
+          "latitude": "-23.55",
+          "population": {
+            "-year": "96",
+            "#text": "9811776"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-127",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Santa Barbara dOeste",
+          "population": {
+            "-year": "96",
+            "#text": "161020"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-120",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sumare",
+          "population": {
+            "-year": "96",
+            "#text": "167751"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-167",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Hortolandia",
+          "population": {
+            "-year": "96",
+            "#text": "114885"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-14",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Guarulhos",
+          "population": {
+            "-year": "96",
+            "#text": "972766"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-15",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Campinas",
+          "population": {
+            "-year": "96",
+            "#text": "907996"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-21",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Bernardo do Campo",
+          "population": {
+            "-year": "96",
+            "#text": "658791"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-24",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Santo Andre",
+          "population": {
+            "-year": "96",
+            "#text": "625294"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-25",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Osasco",
+          "population": {
+            "-year": "96",
+            "#text": "622374"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-30",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Jose dos Campos",
+          "population": {
+            "-year": "96",
+            "#text": "485780"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-31",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Ribeirao Preto",
+          "population": {
+            "-year": "96",
+            "#text": "452804"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-37",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sorocaba",
+          "population": {
+            "-year": "96",
+            "#text": "431370"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-41",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Santos",
+          "population": {
+            "-year": "96",
+            "#text": "412288"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-46",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Maua",
+          "population": {
+            "-year": "96",
+            "#text": "344684"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-49",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Carapicuiba",
+          "population": {
+            "-year": "96",
+            "#text": "328015"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-51",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Jose do Rio Preto",
+          "population": {
+            "-year": "96",
+            "#text": "323418"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-52",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Diadema",
+          "population": {
+            "-year": "96",
+            "#text": "323221"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-53",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Moji das Cruzes",
+          "population": {
+            "-year": "96",
+            "#text": "314947"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-55",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Piracicaba",
+          "population": {
+            "-year": "96",
+            "#text": "302605"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-59",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Jundiai",
+          "population": {
+            "-year": "96",
+            "#text": "293237"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-60",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Bauru",
+          "population": {
+            "-year": "96",
+            "#text": "293026"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-62",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Vicente",
+          "population": {
+            "-year": "96",
+            "#text": "279620"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-67",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Franca",
+          "population": {
+            "-year": "96",
+            "#text": "266909"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-69",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Aparecida de Goiania",
+          "population": {
+            "-year": "96",
+            "#text": "265446"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-83",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Limeira",
+          "population": {
+            "-year": "96",
+            "#text": "230292"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-86",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Itaquaquecetuba",
+          "population": {
+            "-year": "96",
+            "#text": "228533"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-89",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Guaruja",
+          "population": {
+            "-year": "96",
+            "#text": "226185"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-91",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Taubate",
+          "population": {
+            "-year": "96",
+            "#text": "220179"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-100",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Embu",
+          "population": {
+            "-year": "96",
+            "#text": "195676"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-105",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Taboao da Serra",
+          "population": {
+            "-year": "96",
+            "#text": "182441"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-107",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Suzano",
+          "population": {
+            "-year": "96",
+            "#text": "180703"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-110",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Marilia",
+          "population": {
+            "-year": "96",
+            "#text": "177503"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-111",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Presidente Prudente",
+          "population": {
+            "-year": "96",
+            "#text": "177236"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-112",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Barueri",
+          "population": {
+            "-year": "96",
+            "#text": "176740"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-113",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Carlos",
+          "population": {
+            "-year": "96",
+            "#text": "175295"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-118",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Jacarei",
+          "population": {
+            "-year": "96",
+            "#text": "168030"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-119",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Americana",
+          "population": {
+            "-year": "96",
+            "#text": "167901"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-125",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Araraquara",
+          "population": {
+            "-year": "96",
+            "#text": "163831"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-126",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Aracatuba",
+          "population": {
+            "-year": "96",
+            "#text": "162546"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-137",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Rio Claro",
+          "population": {
+            "-year": "96",
+            "#text": "153025"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-145",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Sao Caetano do Sul",
+          "population": {
+            "-year": "96",
+            "#text": "140808"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-151",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Itapevi",
+          "population": {
+            "-year": "96",
+            "#text": "133769"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-155",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Cotia",
+          "population": {
+            "-year": "96",
+            "#text": "127047"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-159",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Itu",
+          "population": {
+            "-year": "96",
+            "#text": "122544"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-160",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Indaiatuba",
+          "population": {
+            "-year": "96",
+            "#text": "122159"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-161",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Ferraz de Vasconcelos",
+          "population": {
+            "-year": "96",
+            "#text": "121992"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-168",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Moji Guacu",
+          "population": {
+            "-year": "96",
+            "#text": "114555"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-169",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Pindamonhangaba",
+          "population": {
+            "-year": "96",
+            "#text": "114092"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-175",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Itapetininga",
+          "population": {
+            "-year": "96",
+            "#text": "111909"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-180",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Braganca Paulista",
+          "population": {
+            "-year": "96",
+            "#text": "109863"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-185",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Francisco Morato",
+          "population": {
+            "-year": "96",
+            "#text": "106328"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-191",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Barretos",
+          "population": {
+            "-year": "96",
+            "#text": "104782"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-193",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Jau",
+          "population": {
+            "-year": "96",
+            "#text": "103605"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-198",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Catanduva",
+          "population": {
+            "-year": "96",
+            "#text": "100913"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-199",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Botucatu",
+          "population": {
+            "-year": "96",
+            "#text": "100826"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-202",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-25",
+          "name": "Ribeirao Pires",
+          "population": {
+            "-year": "96",
+            "#text": "100335"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-26",
+      "-capital": "cty-cid-cia-Brazil-38",
+      "-country": "BR",
+      "name": "Sergipe",
+      "area": "22050",
+      "population": "1617368",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Brazil-38",
+          "-is_state_cap": "yes",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-26",
+          "name": "Aracaju",
+          "longitude": "-37.1",
+          "latitude": "-11",
+          "population": {
+            "-year": "96",
+            "#text": "426580"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Atlantic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-114",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-26",
+          "name": "Arapiraca",
+          "population": {
+            "-year": "96",
+            "#text": "173351"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Brazil-194",
+          "-country": "BR",
+          "-province": "prov-cid-cia-Brazil-26",
+          "name": "Nossa Senhora do Socorro",
+          "population": {
+            "-year": "96",
+            "#text": "103496"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-27",
+      "-capital": "cty-cid-cia-Brazil-Palmas",
+      "-country": "BR",
+      "name": "Tocantins",
+      "area": "278420",
+      "population": "1049742",
+      "city": {
+        "-id": "cty-cid-cia-Brazil-Palmas",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-27",
+        "name": "Palmas"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Brazil-28",
+      "-capital": "cty-Brazil-Brasilia",
+      "-country": "BR",
+      "name": "Distrito Federal",
+      "area": "5822",
+      "population": "1817001",
+      "city": {
+        "-id": "cty-Brazil-Brasilia",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "BR",
+        "-province": "prov-cid-cia-Brazil-28",
+        "name": "Brasilia",
+        "longitude": "-47.9167",
+        "latitude": "-15.8667",
+        "population": {
+          "-year": "96",
+          "#text": "1817001"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RCH",
+  "-area": "756950",
+  "-capital": "cty-Chile-Santiago",
+  "-memberships": "org-OPANAL org-APEC org-CCC org-ECLAC org-FAO org-G-11 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-Mercosur org-UN org-UNESCO org-UNIDO org-UNMOGIP org-UNTSO org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Chile",
+  "population": "14333258",
+  "population_growth": "1.24",
+  "infant_mortality": "13.6",
+  "gdp_total": "113200",
+  "gdp_agri": "7.4",
+  "gdp_ind": "36.4",
+  "gdp_serv": "56.2",
+  "inflation": "8.1",
+  "indep_date": "1810-09-18",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "95",
+      "#text": "European European-Indian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "89",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Protestant"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Spanish"
+  },
+  "border": [
+    {
+      "-country": "RA",
+      "-length": "5150"
+    },
+    {
+      "-country": "BOL",
+      "-length": "861"
+    },
+    {
+      "-country": "PE",
+      "-length": "160"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Chile-Santiago",
+      "-is_country_cap": "yes",
+      "-country": "RCH",
+      "name": "Santiago",
+      "longitude": "-70.7",
+      "latitude": "-33.45",
+      "population": {
+        "-year": "87",
+        "#text": "4318000"
+      }
+    },
+    {
+      "-id": "stadt-Valparaiso-RCH-RCH",
+      "-country": "RCH",
+      "name": "Valparaiso",
+      "longitude": "-71.3",
+      "latitude": "-33.1",
+      "population": {
+        "-year": "87",
+        "#text": "267000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "PY",
+  "-area": "406750",
+  "-capital": "cty-cid-cia-Paraguay-Asuncion",
+  "-memberships": "org-OPANAL org-AG org-CCC org-ECLAC org-FAO org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-OAS org-PCA org-RG org-Mercosur org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Paraguay",
+  "population": "5504146",
+  "population_growth": "2.67",
+  "infant_mortality": "23.2",
+  "gdp_total": "17000",
+  "gdp_agri": "25.7",
+  "gdp_ind": "25.8",
+  "gdp_serv": "48.5",
+  "inflation": "10.5",
+  "indep_date": "1811-05-14",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "5",
+    "#text": "Amerindians"
+  },
+  "religions": {
+    "-percentage": "90",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "RA",
+      "-length": "1880"
+    },
+    {
+      "-country": "BOL",
+      "-length": "750"
+    },
+    {
+      "-country": "BR",
+      "-length": "1290"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Paraguay-Asuncion",
+    "-is_country_cap": "yes",
+    "-country": "PY",
+    "name": "Asuncion",
+    "longitude": "-57.3",
+    "latitude": "-25.5",
+    "population": {
+      "-year": "87",
+      "#text": "729307"
+    },
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Paraguay"
+    }
+  }
+}
+{
+  "-car_code": "ROU",
+  "-area": "176220",
+  "-capital": "cty-Uruguay-Montevideo",
+  "-memberships": "org-OPANAL org-AG org-CCC org-ECLAC org-FAO org-G-11 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-Mercosur org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UNIKOM org-UNMOGIP org-MINURSO org-UNMOT org-UNOMIG org-UNOMIL org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Uruguay",
+  "population": "3238952",
+  "population_growth": "0.7",
+  "infant_mortality": "15.4",
+  "gdp_total": "24400",
+  "gdp_agri": "10.5",
+  "gdp_ind": "27.5",
+  "gdp_serv": "62",
+  "inflation": "35.4",
+  "indep_date": "1828-08-25",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "66",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RA",
+      "-length": "579"
+    },
+    {
+      "-country": "BR",
+      "-length": "985"
+    }
+  ],
+  "city": {
+    "-id": "cty-Uruguay-Montevideo",
+    "-is_country_cap": "yes",
+    "-country": "ROU",
+    "name": "Montevideo",
+    "longitude": "-56.2167",
+    "latitude": "-34.85",
+    "population": {
+      "-year": "87",
+      "#text": "1247000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "PE",
+  "-area": "1285220",
+  "-capital": "cty-Peru-Lima",
+  "-memberships": "org-OPANAL org-AG org-CCC org-ECLAC org-FAO org-G-11 org-G-15 org-G-19 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Peru",
+  "population": "24523408",
+  "population_growth": "1.74",
+  "infant_mortality": "52.2",
+  "gdp_total": "87000",
+  "inflation": "10.2",
+  "indep_date": "1821-07-28",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "45",
+    "#text": "Indian"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "CO",
+      "-length": "2900"
+    },
+    {
+      "-country": "BOL",
+      "-length": "900"
+    },
+    {
+      "-country": "BR",
+      "-length": "1560"
+    },
+    {
+      "-country": "RCH",
+      "-length": "160"
+    },
+    {
+      "-country": "EC",
+      "-length": "1420"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Peru-2",
+      "-capital": "cty-cid-cia-Peru-Chachapoyas",
+      "-country": "PE",
+      "name": "Amazonas",
+      "area": "39249",
+      "population": "374587",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Chachapoyas",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-2",
+        "name": "Chachapoyas"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-3",
+      "-capital": "cty-cid-cia-Peru-Huaraz",
+      "-country": "PE",
+      "name": "Ancash",
+      "area": "35825",
+      "population": "1024398",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Peru-8",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-3",
+          "name": "Chimbote",
+          "population": {
+            "-year": "93",
+            "#text": "268979"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Peru-Huaraz",
+          "-is_state_cap": "yes",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-3",
+          "name": "Huaraz"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Peru-4",
+      "-capital": "cty-cid-cia-Peru-Abancay",
+      "-country": "PE",
+      "name": "Apurimac",
+      "area": "20895",
+      "population": "409630",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Abancay",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-4",
+        "name": "Abancay"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-5",
+      "-capital": "cty-cid-cia-Peru-3",
+      "-country": "PE",
+      "name": "Arequipa",
+      "area": "63345",
+      "population": "999851",
+      "city": {
+        "-id": "cty-cid-cia-Peru-3",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-5",
+        "name": "Arequipa",
+        "population": {
+          "-year": "93",
+          "#text": "619156"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-6",
+      "-capital": "cty-cid-cia-Peru-18",
+      "-country": "PE",
+      "name": "Ayacucho",
+      "area": "43814",
+      "population": "517397",
+      "city": {
+        "-id": "cty-cid-cia-Peru-18",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-6",
+        "name": "Ayacucho",
+        "population": {
+          "-year": "93",
+          "#text": "105918"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-7",
+      "-capital": "cty-cid-cia-Peru-19",
+      "-country": "PE",
+      "name": "Cajamarca",
+      "area": "33247",
+      "population": "1349129",
+      "city": {
+        "-id": "cty-cid-cia-Peru-19",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-7",
+        "name": "Cajamarca",
+        "population": {
+          "-year": "93",
+          "#text": "92447"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-8",
+      "-capital": "cty-cid-cia-Peru-Callao",
+      "-country": "PE",
+      "name": "Callao",
+      "area": "146",
+      "population": "701022",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Callao",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-8",
+        "name": "Callao"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-9",
+      "-capital": "cty-cid-cia-Peru-10",
+      "-country": "PE",
+      "name": "Cuzco",
+      "area": "71891",
+      "population": "1107473",
+      "city": {
+        "-id": "cty-cid-cia-Peru-10",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-9",
+        "name": "Cuzco",
+        "longitude": "-72",
+        "latitude": "-13.2",
+        "population": {
+          "-year": "93",
+          "#text": "255568"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amazonas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-10",
+      "-capital": "cty-cid-cia-Peru-Huancavelica",
+      "-country": "PE",
+      "name": "Huancavelica",
+      "area": "22131",
+      "population": "413136",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Huancavelica",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-10",
+        "name": "Huancavelica"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-11",
+      "-capital": "cty-cid-cia-Peru-16",
+      "-country": "PE",
+      "name": "Huanuco",
+      "area": "36938",
+      "population": "722669",
+      "city": {
+        "-id": "cty-cid-cia-Peru-16",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-11",
+        "name": "Huanuco",
+        "population": {
+          "-year": "93",
+          "#text": "118814"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-12",
+      "-capital": "cty-cid-cia-Peru-13",
+      "-country": "PE",
+      "name": "Ica",
+      "area": "21327",
+      "population": "608609",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Peru-13",
+          "-is_state_cap": "yes",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-12",
+          "name": "Ica",
+          "population": {
+            "-year": "93",
+            "#text": "161406"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Peru-17",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-12",
+          "name": "Chincha Alta",
+          "population": {
+            "-year": "93",
+            "#text": "110016"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Peru-13",
+      "-capital": "cty-cid-cia-Peru-9",
+      "-country": "PE",
+      "name": "Junin",
+      "area": "44409",
+      "population": "1132448",
+      "city": {
+        "-id": "cty-cid-cia-Peru-9",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-13",
+        "name": "Huancayo",
+        "population": {
+          "-year": "93",
+          "#text": "258209"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-14",
+      "-capital": "cty-cid-cia-Peru-4",
+      "-country": "PE",
+      "name": "La Libertad",
+      "area": "25569",
+      "population": "1366125",
+      "city": {
+        "-id": "cty-cid-cia-Peru-4",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-14",
+        "name": "Trujillo",
+        "population": {
+          "-year": "93",
+          "#text": "509312"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-15",
+      "-capital": "cty-cid-cia-Peru-5",
+      "-country": "PE",
+      "name": "Lambayeque",
+      "area": "14231",
+      "population": "1009655",
+      "city": {
+        "-id": "cty-cid-cia-Peru-5",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-15",
+        "name": "Chiclayo",
+        "population": {
+          "-year": "93",
+          "#text": "411536"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-16",
+      "-capital": "cty-Peru-Lima",
+      "-country": "PE",
+      "name": "Lima",
+      "area": "34801",
+      "population": "6941672",
+      "city": {
+        "-id": "cty-Peru-Lima",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-16",
+        "name": "Lima",
+        "longitude": "-77.05",
+        "latitude": "-12.0833",
+        "population": {
+          "-year": "93",
+          "#text": "6321173"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-18",
+      "-capital": "cty-cid-cia-Peru-7",
+      "-country": "PE",
+      "name": "Loreto",
+      "area": "368851",
+      "population": "789235",
+      "city": {
+        "-id": "cty-cid-cia-Peru-7",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-18",
+        "name": "Iquitos",
+        "longitude": "-73.2",
+        "latitude": "-3.4",
+        "population": {
+          "-year": "93",
+          "#text": "274759"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amazonas"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-19",
+      "-capital": "cty-cid-cia-Peru-Puerto-Maldonado",
+      "-country": "PE",
+      "name": "Madre de Dios",
+      "area": "85182",
+      "population": "74306",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Puerto-Maldonado",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-19",
+        "name": "Puerto Maldonado"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-20",
+      "-capital": "cty-cid-cia-Peru-Moquegua",
+      "-country": "PE",
+      "name": "Moquegua",
+      "area": "15733",
+      "population": "137747",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Moquegua",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-20",
+        "name": "Moquegua"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-21",
+      "-capital": "cty-cid-cia-Peru-Cerro-de-Pasco",
+      "-country": "PE",
+      "name": "Pasco",
+      "area": "25319",
+      "population": "243366",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Cerro-de-Pasco",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-21",
+        "name": "Cerro de Pasco"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-22",
+      "-capital": "cty-cid-cia-Peru-6",
+      "-country": "PE",
+      "name": "Piura",
+      "area": "35892",
+      "population": "1468337",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Peru-6",
+          "-is_state_cap": "yes",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-22",
+          "name": "Piura",
+          "population": {
+            "-year": "93",
+            "#text": "277964"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Peru-14",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-22",
+          "name": "Sullana",
+          "population": {
+            "-year": "93",
+            "#text": "147361"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Peru-21",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-22",
+          "name": "Talara",
+          "population": {
+            "-year": "93",
+            "#text": "82228"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Peru-23",
+      "-capital": "cty-cid-cia-Peru-20",
+      "-country": "PE",
+      "name": "Puno",
+      "area": "71999",
+      "population": "1144151",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Peru-20",
+          "-is_state_cap": "yes",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-23",
+          "name": "Puno",
+          "population": {
+            "-year": "93",
+            "#text": "91877"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Peru-15",
+          "-country": "PE",
+          "-province": "prov-cid-cia-Peru-23",
+          "name": "Juliaca",
+          "population": {
+            "-year": "93",
+            "#text": "142576"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Peru-24",
+      "-capital": "cty-cid-cia-Peru-Moyobamba",
+      "-country": "PE",
+      "name": "San Martin",
+      "area": "51253",
+      "population": "627781",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Moyobamba",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-24",
+        "name": "Moyobamba"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-25",
+      "-capital": "cty-cid-cia-Peru-11",
+      "-country": "PE",
+      "name": "Tacna",
+      "area": "16075",
+      "population": "245843",
+      "city": {
+        "-id": "cty-cid-cia-Peru-11",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-25",
+        "name": "Tacna",
+        "population": {
+          "-year": "93",
+          "#text": "174336"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-26",
+      "-capital": "cty-cid-cia-Peru-Tumbes",
+      "-country": "PE",
+      "name": "Tumbes",
+      "area": "4669",
+      "population": "173616",
+      "city": {
+        "-id": "cty-cid-cia-Peru-Tumbes",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-26",
+        "name": "Tumbes"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Peru-27",
+      "-capital": "cty-cid-cia-Peru-12",
+      "-country": "PE",
+      "name": "Ucayali",
+      "area": "102410",
+      "population": "364596",
+      "city": {
+        "-id": "cty-cid-cia-Peru-12",
+        "-is_state_cap": "yes",
+        "-country": "PE",
+        "-province": "prov-cid-cia-Peru-27",
+        "name": "Pucallpa",
+        "population": {
+          "-year": "93",
+          "#text": "172286"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Amazonas"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "FGU",
+  "-area": "91000",
+  "-capital": "cty-cid-cia-French-Guiana-Cayenne",
+  "-memberships": "org-WCL org-WFTU",
+  "name": "French Guiana",
+  "population": "151187",
+  "population_growth": "3.86",
+  "infant_mortality": "14.6",
+  "gdp_total": "800",
+  "inflation": "2.5",
+  "dependent": { "-country": "F" },
+  "government": "overseas department of France",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "12",
+    "#text": "Amerindian"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "French"
+  },
+  "border": [
+    {
+      "-country": "BR",
+      "-length": "673"
+    },
+    {
+      "-country": "SME",
+      "-length": "510"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-French-Guiana-Cayenne",
+    "-is_country_cap": "yes",
+    "-country": "FGU",
+    "name": "Cayenne",
+    "longitude": "-52.45",
+    "latitude": "4.93333",
+    "population": {
+      "-year": "87",
+      "#text": "38155"
+    }
+  }
+}
+{
+  "-car_code": "GUY",
+  "-area": "214970",
+  "-capital": "cty-Guyana-Georgetown",
+  "-memberships": "org-ACP org-Caricom org-CDB org-C org-CCC org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Guyana",
+  "population": "712091",
+  "population_growth": "-0.9",
+  "infant_mortality": "51.4",
+  "gdp_total": "1600",
+  "gdp_agri": "26.5",
+  "gdp_ind": "27.8",
+  "gdp_serv": "45.7",
+  "inflation": "8.1",
+  "indep_date": "1966-05-26",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "51",
+      "#text": "East Indian"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Amerindian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "European Chinese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "9",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "57",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "33",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BR",
+      "-length": "1119"
+    },
+    {
+      "-country": "SME",
+      "-length": "600"
+    },
+    {
+      "-country": "YV",
+      "-length": "743"
+    }
+  ],
+  "city": {
+    "-id": "cty-Guyana-Georgetown",
+    "-is_country_cap": "yes",
+    "-country": "GUY",
+    "name": "Georgetown",
+    "longitude": "-58.2",
+    "latitude": "6.83333",
+    "population": {
+      "-year": "87",
+      "#text": "170000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "SME",
+  "-area": "163270",
+  "-capital": "cty-cid-cia-Suriname-Paramaribo",
+  "-memberships": "org-ACP org-OPANAL org-Caricom org-ECLAC org-FAO org-G-77 org-IADB org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-LAES org-NAM org-OAS org-PCA org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WCL org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Suriname",
+  "population": "436418",
+  "population_growth": "1.6",
+  "infant_mortality": "29.3",
+  "gdp_total": "1300",
+  "gdp_agri": "21.6",
+  "gdp_ind": "23.7",
+  "gdp_serv": "54.7",
+  "inflation": "62",
+  "indep_date": "1975-11-25",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1.7",
+      "#text": "Chinese"
+    },
+    {
+      "-percentage": "15.3",
+      "#text": "Javanese"
+    },
+    {
+      "-percentage": "31",
+      "#text": "Creole"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Europeans"
+    },
+    {
+      "-percentage": "2.6",
+      "#text": "Amerindian"
+    },
+    {
+      "-percentage": "37",
+      "#text": "Hindustani"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "19.6",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "22.8",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "25.2",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "27.4",
+      "#text": "Hindu"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BR",
+      "-length": "597"
+    },
+    {
+      "-country": "FGU",
+      "-length": "510"
+    },
+    {
+      "-country": "GUY",
+      "-length": "600"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Suriname-Paramaribo",
+    "-is_country_cap": "yes",
+    "-country": "SME",
+    "name": "Paramaribo",
+    "longitude": "-55.15",
+    "latitude": "5.81667",
+    "population": {
+      "-year": "87",
+      "#text": "150000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "YV",
+  "-area": "912050",
+  "-capital": "cty-Venezuela-Caracas",
+  "-memberships": "org-OPANAL org-AG org-Caricom org-CDB org-BCIE org-ECLAC org-FAO org-G-3 org-G-11 org-G-15 org-G-19 org-G-24 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-OPEC org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNHCR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Venezuela",
+  "population": "21983188",
+  "population_growth": "1.89",
+  "infant_mortality": "29.5",
+  "gdp_total": "195500",
+  "gdp_agri": "5",
+  "gdp_ind": "41",
+  "gdp_serv": "54",
+  "inflation": "57",
+  "indep_date": "1811-07-05",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "2",
+    "#text": "Amerindian"
+  },
+  "religions": [
+    {
+      "-percentage": "96",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "CO",
+      "-length": "2050"
+    },
+    {
+      "-country": "BR",
+      "-length": "2200"
+    },
+    {
+      "-country": "GUY",
+      "-length": "743"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Venezuela-2",
+      "-capital": "cty-cid-cia-Venezuela-Puerto-Ayacucho",
+      "-country": "YV",
+      "name": "Amazonas",
+      "area": "175750",
+      "population": "94590",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-Puerto-Ayacucho",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-2",
+        "name": "Puerto Ayacucho"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-3",
+      "-capital": "cty-cid-cia-Venezuela-10",
+      "-country": "YV",
+      "name": "Anzoategui",
+      "area": "43300",
+      "population": "1034311",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-10",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-3",
+          "name": "Barcelona",
+          "population": {
+            "-year": "90",
+            "#text": "221792"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-18",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-3",
+          "name": "Puerto La Cruz",
+          "population": {
+            "-year": "90",
+            "#text": "155731"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-4",
+      "-capital": "cty-cid-cia-Venezuela-San-Fernando",
+      "-country": "YV",
+      "name": "Apure",
+      "area": "76500",
+      "population": "382572",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-San-Fernando",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-4",
+        "name": "San Fernando"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-5",
+      "-capital": "cty-cid-cia-Venezuela-7",
+      "-country": "YV",
+      "name": "Aragua",
+      "area": "7014",
+      "population": "1344099",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-7",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-5",
+          "name": "Maracay",
+          "population": {
+            "-year": "90",
+            "#text": "354196"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-15",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-5",
+          "name": "Turmero",
+          "population": {
+            "-year": "90",
+            "#text": "174280"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-6",
+      "-capital": "cty-cid-cia-Venezuela-19",
+      "-country": "YV",
+      "name": "Barinas",
+      "area": "35200",
+      "population": "519197",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-19",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-6",
+        "name": "Barinas",
+        "population": {
+          "-year": "90",
+          "#text": "153630"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-7",
+      "-capital": "cty-cid-cia-Venezuela-9",
+      "-country": "YV",
+      "name": "Bolivar",
+      "area": "238000",
+      "population": "1142210",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-9",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-7",
+          "name": "Ciudad Bolivar",
+          "population": {
+            "-year": "90",
+            "#text": "225340"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Orinoco"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-6",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-7",
+          "name": "Ciudad Guayana",
+          "population": {
+            "-year": "90",
+            "#text": "453047"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Orinoco"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-8",
+      "-capital": "cty-cid-cia-Venezuela-4",
+      "-country": "YV",
+      "name": "Carabobo",
+      "area": "4650",
+      "population": "1823767",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-4",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-8",
+          "name": "Valencia",
+          "population": {
+            "-year": "90",
+            "#text": "903621"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-22",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-8",
+          "name": "Puerto Cabello",
+          "population": {
+            "-year": "90",
+            "#text": "128825"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-25",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-8",
+          "name": "Guacara",
+          "population": {
+            "-year": "90",
+            "#text": "100766"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-9",
+      "-capital": "cty-cid-cia-Venezuela-San-Carlos",
+      "-country": "YV",
+      "name": "Cojedes",
+      "area": "14800",
+      "population": "227741",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-San-Carlos",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-9",
+        "name": "San Carlos"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-10",
+      "-capital": "cty-cid-cia-Venezuela-Tucupita",
+      "-country": "YV",
+      "name": "Delta Amacuro",
+      "area": "40200",
+      "population": "114390",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-Tucupita",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-10",
+        "name": "Tucupita"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-11",
+      "-capital": "cty-cid-cia-Venezuela-Coro",
+      "-country": "YV",
+      "name": "Falcon",
+      "area": "24800",
+      "population": "699232",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-Coro",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-11",
+        "name": "Coro"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-12",
+      "-capital": "cty-cid-cia-Venezuela-San-Juan",
+      "-country": "YV",
+      "name": "Guarico",
+      "area": "64986",
+      "population": "583221",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-San-Juan",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-12",
+        "name": "San Juan"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-13",
+      "-capital": "cty-cid-cia-Venezuela-5",
+      "-country": "YV",
+      "name": "Lara",
+      "area": "19800",
+      "population": "1430968",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-5",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-13",
+        "name": "Barquisimeto",
+        "population": {
+          "-year": "90",
+          "#text": "625450"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-14",
+      "-capital": "cty-cid-cia-Venezuela-16",
+      "-country": "YV",
+      "name": "Merida",
+      "area": "11300",
+      "population": "680503",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-16",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-14",
+        "name": "Merida",
+        "population": {
+          "-year": "90",
+          "#text": "170902"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-15",
+      "-capital": "cty-cid-cia-Venezuela-20",
+      "-country": "YV",
+      "name": "Miranda",
+      "area": "7950",
+      "population": "2303302",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-20",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-15",
+          "name": "Los Teques",
+          "population": {
+            "-year": "90",
+            "#text": "140617"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-8",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-15",
+          "name": "Petare",
+          "population": {
+            "-year": "90",
+            "#text": "338417"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-14",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-15",
+          "name": "Baruta",
+          "population": {
+            "-year": "90",
+            "#text": "182941"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-21",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-15",
+          "name": "Guarenas",
+          "population": {
+            "-year": "90",
+            "#text": "134158"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-16",
+      "-capital": "cty-cid-cia-Venezuela-13",
+      "-country": "YV",
+      "name": "Monagas",
+      "area": "28900",
+      "population": "555705",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-13",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-16",
+        "name": "Maturin",
+        "population": {
+          "-year": "90",
+          "#text": "206654"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-17",
+      "-capital": "cty-cid-cia-Venezuela-La-Ascuncion",
+      "-country": "YV",
+      "name": "Nueva Esparta",
+      "area": "1150",
+      "population": "330307",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-La-Ascuncion",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-17",
+        "name": "La Ascuncion"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-18",
+      "-capital": "cty-cid-cia-Venezuela-Guanare",
+      "-country": "YV",
+      "name": "Portuguesa",
+      "area": "15200",
+      "population": "720865",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Venezuela-24",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-18",
+          "name": "Acarigua",
+          "population": {
+            "-year": "90",
+            "#text": "116551"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-Guanare",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-18",
+          "name": "Guanare"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-19",
+      "-capital": "cty-cid-cia-Venezuela-12",
+      "-country": "YV",
+      "name": "Sucre",
+      "area": "11800",
+      "population": "781756",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-12",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-19",
+        "name": "Cumana",
+        "population": {
+          "-year": "90",
+          "#text": "212432"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-20",
+      "-capital": "cty-cid-cia-Venezuela-11",
+      "-country": "YV",
+      "name": "Tachira",
+      "area": "11100",
+      "population": "946949",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-11",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-20",
+        "name": "San Cristobal",
+        "population": {
+          "-year": "90",
+          "#text": "220675"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-21",
+      "-capital": "cty-cid-cia-Venezuela-Trujillo",
+      "-country": "YV",
+      "name": "Trujillo",
+      "area": "7400",
+      "population": "562752",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-Trujillo",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-21",
+        "name": "Trujillo"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-22",
+      "-capital": "cty-cid-cia-Venezuela-San-Felipe",
+      "-country": "YV",
+      "name": "Yaracuy",
+      "area": "7100",
+      "population": "466152",
+      "city": {
+        "-id": "cty-cid-cia-Venezuela-San-Felipe",
+        "-is_state_cap": "yes",
+        "-country": "YV",
+        "-province": "prov-cid-cia-Venezuela-22",
+        "name": "San Felipe"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-23",
+      "-capital": "cty-Venezuela-Maracaibo",
+      "-country": "YV",
+      "name": "Zulia",
+      "area": "63100",
+      "population": "2820250",
+      "city": [
+        {
+          "-id": "cty-Venezuela-Maracaibo",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-23",
+          "name": "Maracaibo",
+          "longitude": "-71.6",
+          "latitude": "10.65",
+          "population": {
+            "-year": "90",
+            "#text": "1249670"
+          },
+          "located_at": [
+            {
+              "-watertype": "sea",
+              "-sea": "sea-Caribbean"
+            },
+            {
+              "-watertype": "lake",
+              "-lake": "lake-Maracaibo_See"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-17",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-23",
+          "name": "Cabimas",
+          "population": {
+            "-year": "90",
+            "#text": "165755"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Venezuela-25",
+      "-capital": "cty-Venezuela-Caracas",
+      "-country": "YV",
+      "name": "Distrito Federal",
+      "area": "1930",
+      "population": "2279677",
+      "city": [
+        {
+          "-id": "cty-Venezuela-Caracas",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-25",
+          "name": "Caracas",
+          "longitude": "-66.9333",
+          "latitude": "10.5",
+          "population": {
+            "-year": "90",
+            "#text": "1822465"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Venezuela-26",
+          "-country": "YV",
+          "-province": "prov-cid-cia-Venezuela-25",
+          "name": "Catia La Mar",
+          "population": {
+            "-year": "90",
+            "#text": "100104"
+          }
+        }
+      ]
+    }
+  ]
+}
+{
+  "-car_code": "EC",
+  "-area": "283560",
+  "-capital": "cty-cid-cia-Ecuador-Quito",
+  "-memberships": "org-OPANAL org-AG org-ECLAC org-FAO org-G-11 org-G-77 org-IADB org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-LAES org-LAIA org-NAM org-OAS org-PCA org-RG org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Ecuador",
+  "population": "11466291",
+  "population_growth": "1.96",
+  "infant_mortality": "34.8",
+  "gdp_total": "44600",
+  "gdp_agri": "13",
+  "gdp_ind": "39",
+  "gdp_serv": "48",
+  "inflation": "25",
+  "indep_date": "1822-05-24",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "10",
+      "#text": "Spanish"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Indian"
+    }
+  ],
+  "religions": {
+    "-percentage": "95",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "CO",
+      "-length": "590"
+    },
+    {
+      "-country": "PE",
+      "-length": "1420"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Ecuador-Quito",
+      "-is_country_cap": "yes",
+      "-country": "EC",
+      "name": "Quito",
+      "longitude": "-78.5333",
+      "latitude": "-0.216667",
+      "population": {
+        "-year": "87",
+        "#text": "1200000"
+      }
+    },
+    {
+      "-id": "stadt-Guayaquil-EC-EC",
+      "-country": "EC",
+      "name": "Guayaquil",
+      "longitude": "-80",
+      "latitude": "-2",
+      "population": {
+        "-year": "87",
+        "#text": "1300868"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Pacific"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "FALK",
+  "-area": "12173",
+  "-capital": "city-Stanley-FALK-FALK",
+  "-memberships": "org-UPU",
+  "name": "Falkland Islands",
+  "population": "3140",
+  "gdp_total": "105.1",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "america",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "67.2",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "31.5",
+      "#text": "None"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-Stanley-FALK-FALK",
+    "-is_country_cap": "yes",
+    "-country": "FALK",
+    "name": "Stanley",
+    "longitude": "-57.51",
+    "latitude": "-51.42",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-EastFalkland" }
+  }
+}
+{
+  "-car_code": "DZ",
+  "-area": "2381740",
+  "-capital": "cty-Algeria-Algiers",
+  "-memberships": "org-AfDB org-ABEDA org-AFESD org-AL org-AMU org-AMF org-CCC org-ECA org-FAO org-G-15 org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OAS org-OAPEC org-OPEC org-OIC org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNMIH org-UNHCR org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Algeria",
+  "population": "29183032",
+  "population_growth": "2.21",
+  "infant_mortality": "48.7",
+  "gdp_total": "108700",
+  "gdp_agri": "12",
+  "gdp_ind": "50",
+  "gdp_serv": "38",
+  "inflation": "28",
+  "indep_date": "1962-07-05",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "99",
+      "#text": "Arab-Berber"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "99",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "0.5",
+      "#text": "Jewish"
+    }
+  ],
+  "border": [
+    {
+      "-country": "LAR",
+      "-length": "982"
+    },
+    {
+      "-country": "RMM",
+      "-length": "1376"
+    },
+    {
+      "-country": "RIM",
+      "-length": "463"
+    },
+    {
+      "-country": "MA",
+      "-length": "1559"
+    },
+    {
+      "-country": "RN",
+      "-length": "956"
+    },
+    {
+      "-country": "TN",
+      "-length": "965"
+    },
+    {
+      "-country": "WSA",
+      "-length": "42"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Algeria-Algiers",
+      "-is_country_cap": "yes",
+      "-country": "DZ",
+      "name": "Algier",
+      "longitude": "3.05",
+      "latitude": "36.7667",
+      "population": {
+        "-year": "87",
+        "#text": "1507241"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-5",
+      "-country": "DZ",
+      "name": "Annaba",
+      "population": {
+        "-year": "87",
+        "#text": "222518"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-6",
+      "-country": "DZ",
+      "name": "Batna",
+      "population": {
+        "-year": "87",
+        "#text": "181601"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-15",
+      "-country": "DZ",
+      "name": "Bechar",
+      "population": {
+        "-year": "87",
+        "#text": "107311"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-8",
+      "-country": "DZ",
+      "name": "Sidi bel Abbes",
+      "population": {
+        "-year": "87",
+        "#text": "152778"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-9",
+      "-country": "DZ",
+      "name": "Skikda",
+      "population": {
+        "-year": "87",
+        "#text": "128747"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-14",
+      "-country": "DZ",
+      "name": "Tebessa",
+      "population": {
+        "-year": "87",
+        "#text": "107559"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-3",
+      "-country": "DZ",
+      "name": "Oran",
+      "longitude": "0.4",
+      "latitude": "35.4",
+      "population": {
+        "-year": "87",
+        "#text": "609823"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-4",
+      "-country": "DZ",
+      "name": "Constantine",
+      "population": {
+        "-year": "87",
+        "#text": "440842"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-7",
+      "-country": "DZ",
+      "name": "Setif",
+      "population": {
+        "-year": "87",
+        "#text": "170182"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-10",
+      "-country": "DZ",
+      "name": "Blida",
+      "population": {
+        "-year": "87",
+        "#text": "127284"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-11",
+      "-country": "DZ",
+      "name": "Bejaia",
+      "population": {
+        "-year": "87",
+        "#text": "114534"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-12",
+      "-country": "DZ",
+      "name": "Mostaganem",
+      "population": {
+        "-year": "87",
+        "#text": "114037"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Algeria-13",
+      "-country": "DZ",
+      "name": "Tlemcen",
+      "population": {
+        "-year": "87",
+        "#text": "107632"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "LAR",
+  "-area": "1759540",
+  "-capital": "cty-cid-cia-Libya-Tripoli",
+  "-memberships": "org-AfDB org-ABEDA org-AFESD org-AL org-AMU org-AMF org-CAEU org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OAPEC org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UNITAR org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Libya",
+  "population": "5445436",
+  "population_growth": "3.67",
+  "infant_mortality": "59.5",
+  "gdp_total": "32900",
+  "inflation": "25",
+  "indep_date": "1951-12-24",
+  "government": "Jamahiriya in theory",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "97",
+    "#text": "Berber Arab"
+  },
+  "religions": {
+    "-percentage": "97",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "ET",
+      "-length": "1150"
+    },
+    {
+      "-country": "DZ",
+      "-length": "982"
+    },
+    {
+      "-country": "RN",
+      "-length": "354"
+    },
+    {
+      "-country": "TN",
+      "-length": "459"
+    },
+    {
+      "-country": "TCH",
+      "-length": "1055"
+    },
+    {
+      "-country": "SUD",
+      "-length": "383"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Libya-Tripoli",
+      "-is_country_cap": "yes",
+      "-country": "LAR",
+      "name": "Tripoli",
+      "longitude": "22.3",
+      "latitude": "37.3",
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    },
+    {
+      "-id": "stadt-Bengasi-LAR-LAR",
+      "-country": "LAR",
+      "name": "Bengasi",
+      "longitude": "20",
+      "latitude": "32",
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RMM",
+  "-area": "1240000",
+  "-capital": "cty-cid-cia-Mali-Bamako",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UNMIH org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Mali",
+  "population": "9653261",
+  "population_growth": "2.95",
+  "infant_mortality": "102.7",
+  "gdp_total": "5400",
+  "gdp_agri": "42.4",
+  "gdp_ind": "15.4",
+  "gdp_serv": "42.2",
+  "inflation": "8",
+  "indep_date": "1960-09-22",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "50",
+      "#text": "Mande"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Peuhl"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Voltaic"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Songhai"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Tuareg"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "90",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Christian"
+    }
+  ],
+  "languages": {
+    "-percentage": "80",
+    "#text": "Bambara"
+  },
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "1376"
+    },
+    {
+      "-country": "RIM",
+      "-length": "2237"
+    },
+    {
+      "-country": "RN",
+      "-length": "821"
+    },
+    {
+      "-country": "BF",
+      "-length": "1000"
+    },
+    {
+      "-country": "CI",
+      "-length": "532"
+    },
+    {
+      "-country": "RG",
+      "-length": "858"
+    },
+    {
+      "-country": "SN",
+      "-length": "419"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Mali-Bamako",
+    "-is_country_cap": "yes",
+    "-country": "RMM",
+    "name": "Bamako",
+    "longitude": "-7.6",
+    "latitude": "12.3",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Niger"
+    }
+  }
+}
+{
+  "-car_code": "RIM",
+  "-area": "1030700",
+  "-capital": "cty-cid-cia-Mauritania-Nouakchott",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-ABEDA org-AFESD org-AL org-AMU org-AMF org-CAEU org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Mauritania",
+  "population": "2336048",
+  "population_growth": "3.17",
+  "infant_mortality": "81.7",
+  "gdp_total": "2800",
+  "gdp_agri": "27.1",
+  "gdp_ind": "29.5",
+  "gdp_serv": "43.4",
+  "inflation": "3.5",
+  "indep_date": "1960-11-28",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "30",
+      "#text": "Maur"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Black Maur"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "463"
+    },
+    {
+      "-country": "RMM",
+      "-length": "2237"
+    },
+    {
+      "-country": "WSA",
+      "-length": "1561"
+    },
+    {
+      "-country": "SN",
+      "-length": "813"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Mauritania-Nouakchott",
+    "-is_country_cap": "yes",
+    "-country": "RIM",
+    "name": "Nouakchott",
+    "longitude": "-16",
+    "latitude": "18"
+  }
+}
+{
+  "-car_code": "MA",
+  "-area": "446550",
+  "-capital": "cty-cid-cia-Morocco-3",
+  "-memberships": "org-AfDB org-ACCT org-ABEDA org-AFESD org-AL org-AMU org-AMF org-CCC org-ECA org-EBRD org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAS org-OIC org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNHCR org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Morocco",
+  "population": "29779156",
+  "population_growth": "2.05",
+  "infant_mortality": "43.2",
+  "gdp_total": "87400",
+  "gdp_agri": "14.3",
+  "gdp_ind": "32.2",
+  "gdp_serv": "53.5",
+  "inflation": "5.4",
+  "indep_date": "1956-03-02",
+  "government": "constitutional monarchy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "99.1",
+      "#text": "Arab-Berber"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "98.7",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "1.1",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "1559"
+    },
+    {
+      "-country": "WSA",
+      "-length": "443"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Morocco-3",
+      "-is_country_cap": "yes",
+      "-country": "MA",
+      "name": "Rabat",
+      "longitude": "-6.5",
+      "latitude": "33.6",
+      "population": {
+        "-year": "94",
+        "#text": "1385872"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-7",
+      "-country": "MA",
+      "name": "Agadir",
+      "population": {
+        "-year": "94",
+        "#text": "550200"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-4",
+      "-country": "MA",
+      "name": "Fes",
+      "population": {
+        "-year": "94",
+        "#text": "774754"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-10",
+      "-country": "MA",
+      "name": "Kenitra",
+      "population": {
+        "-year": "94",
+        "#text": "448785"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-14",
+      "-country": "MA",
+      "name": "Khouribga",
+      "population": {
+        "-year": "94",
+        "#text": "294680"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-5",
+      "-country": "MA",
+      "name": "Marrakech",
+      "population": {
+        "-year": "94",
+        "#text": "745541"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-8",
+      "-country": "MA",
+      "name": "Meknes",
+      "population": {
+        "-year": "94",
+        "#text": "530171"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-6",
+      "-country": "MA",
+      "name": "Oujda",
+      "population": {
+        "-year": "94",
+        "#text": "678778"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-12",
+      "-country": "MA",
+      "name": "Safi",
+      "population": {
+        "-year": "94",
+        "#text": "376038"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-9",
+      "-country": "MA",
+      "name": "Tanger",
+      "population": {
+        "-year": "94",
+        "#text": "526215"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-Morocco-Casablanca",
+      "-country": "MA",
+      "name": "Casablanca",
+      "longitude": "-7.65",
+      "latitude": "33.5833",
+      "population": {
+        "-year": "94",
+        "#text": "2940623"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-11",
+      "-country": "MA",
+      "name": "Beni Mellal",
+      "population": {
+        "-year": "94",
+        "#text": "386505"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Morocco-13",
+      "-country": "MA",
+      "name": "Tetouan",
+      "population": {
+        "-year": "94",
+        "#text": "367349"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Mittelmeer"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RN",
+  "-area": "1267000",
+  "-capital": "cty-cid-cia-Niger-Niamey",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-Entente org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Niger",
+  "population": "9113001",
+  "population_growth": "2.99",
+  "infant_mortality": "117.6",
+  "gdp_total": "5500",
+  "gdp_agri": "38.5",
+  "gdp_ind": "17.9",
+  "gdp_serv": "43.6",
+  "inflation": "35.6",
+  "indep_date": "1960-08-03",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "8.5",
+      "#text": "Fula"
+    },
+    {
+      "-percentage": "56",
+      "#text": "Hausa"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Djerma"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Tuareg"
+    },
+    {
+      "-percentage": "4.3",
+      "#text": "Beri Beri"
+    },
+    {
+      "-percentage": "1.2",
+      "#text": "Gourmantche"
+    }
+  ],
+  "religions": {
+    "-percentage": "80",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "956"
+    },
+    {
+      "-country": "LAR",
+      "-length": "354"
+    },
+    {
+      "-country": "RMM",
+      "-length": "821"
+    },
+    {
+      "-country": "BEN",
+      "-length": "266"
+    },
+    {
+      "-country": "BF",
+      "-length": "628"
+    },
+    {
+      "-country": "WAN",
+      "-length": "1497"
+    },
+    {
+      "-country": "TCH",
+      "-length": "1175"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Niger-Niamey",
+    "-is_country_cap": "yes",
+    "-country": "RN",
+    "name": "Niamey",
+    "longitude": "2.1",
+    "latitude": "13.2",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Niger"
+    }
+  }
+}
+{
+  "-car_code": "TN",
+  "-area": "163610",
+  "-capital": "cty-Tunisia-Tunis",
+  "-memberships": "org-AfDB org-ACCT org-ABEDA org-AFESD org-AL org-AMU org-AMF org-BSEC org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OAS org-OIC org-UN org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNITAR org-MINURSO org-UNHCR org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Tunisia",
+  "population": "9019687",
+  "population_growth": "1.81",
+  "infant_mortality": "35.1",
+  "gdp_total": "37100",
+  "gdp_agri": "15",
+  "gdp_ind": "30",
+  "gdp_serv": "55",
+  "inflation": "5.5",
+  "indep_date": "1956-03-20",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "98",
+      "#text": "Arab-Berber"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "98",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Jewish"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "965"
+    },
+    {
+      "-country": "LAR",
+      "-length": "459"
+    }
+  ],
+  "city": {
+    "-id": "cty-Tunisia-Tunis",
+    "-is_country_cap": "yes",
+    "-country": "TN",
+    "name": "Tunis",
+    "longitude": "10.2",
+    "latitude": "36.7833",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Mittelmeer"
+    }
+  }
+}
+{
+  "-car_code": "WSA",
+  "-area": "266000",
+  "-capital": "stadt-El_Aaiun-DARS-DARS",
+  "name": "Western Sahara",
+  "population": "222631",
+  "population_growth": "2.46",
+  "infant_mortality": "145.82",
+  "government": "legal status of territory and question of sovereignty unresolved",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "DZ",
+      "-length": "42"
+    },
+    {
+      "-country": "RIM",
+      "-length": "1561"
+    },
+    {
+      "-country": "MA",
+      "-length": "443"
+    }
+  ],
+  "city": {
+    "-id": "stadt-El_Aaiun-DARS-DARS",
+    "-is_country_cap": "yes",
+    "-country": "WSA",
+    "name": "El Aaiun",
+    "longitude": "-13.2",
+    "latitude": "27.05"
+  }
+}
+{
+  "-car_code": "ANG",
+  "-area": "1246700",
+  "-capital": "cty-cid-cia-Angola-Luanda",
+  "-memberships": "org-ACP org-AfDB org-CCC org-ECA org-CEEAC org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-OAS org-SADC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Angola",
+  "population": "10342899",
+  "population_growth": "2.68",
+  "infant_mortality": "138.9",
+  "gdp_total": "7400",
+  "gdp_agri": "12",
+  "gdp_ind": "56",
+  "gdp_serv": "32",
+  "inflation": "20",
+  "indep_date": "1975-11-11",
+  "government": "transitional government nominally a multiparty democracy with a strong presidential system",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "37",
+      "#text": "Ovimbundu"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Kimbundu"
+    },
+    {
+      "-percentage": "13",
+      "#text": "Bakongo"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "38",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RCB",
+      "-length": "201"
+    },
+    {
+      "-country": "NAM",
+      "-length": "1376"
+    },
+    {
+      "-country": "ZRE",
+      "-length": "2511"
+    },
+    {
+      "-country": "Z",
+      "-length": "1110"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Angola-2",
+      "-capital": "cty-cid-cia-Angola-Cabinda",
+      "-country": "ANG",
+      "name": "Cabinda",
+      "area": "7270",
+      "population": "163000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Cabinda",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-2",
+        "name": "Cabinda"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-3",
+      "-capital": "cty-cid-cia-Angola-Mbanza-Congo",
+      "-country": "ANG",
+      "name": "Zaire",
+      "area": "40130",
+      "population": "192000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Mbanza-Congo",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-3",
+        "name": "Mbanza Congo"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-4",
+      "-capital": "cty-cid-cia-Angola-Uige",
+      "-country": "ANG",
+      "name": "Uige",
+      "area": "58698",
+      "population": "837000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Uige",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-4",
+        "name": "Uige"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-5",
+      "-capital": "cty-cid-cia-Angola-Luanda",
+      "-country": "ANG",
+      "name": "Luanda",
+      "area": "2418",
+      "population": "1629000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Luanda",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-5",
+        "name": "Luanda",
+        "longitude": "13.2",
+        "latitude": "-8.6",
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-6",
+      "-capital": "cty-cid-cia-Angola-Ndalatando",
+      "-country": "ANG",
+      "name": "Cuanza Norte",
+      "area": "24110",
+      "population": "378000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Ndalatando",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-6",
+        "name": "Ndalatando"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-7",
+      "-capital": "cty-cid-cia-Angola-Sumbe",
+      "-country": "ANG",
+      "name": "Cuanza Sul",
+      "area": "55660",
+      "population": "651000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Sumbe",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-7",
+        "name": "Sumbe"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-8",
+      "-capital": "cty-cid-cia-Angola-Malanje",
+      "-country": "ANG",
+      "name": "Malanje",
+      "area": "87246",
+      "population": "892000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Malanje",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-8",
+        "name": "Malanje"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-9",
+      "-capital": "cty-cid-cia-Angola-Lucapa",
+      "-country": "ANG",
+      "name": "Lunda Norte",
+      "area": "102783",
+      "population": "292000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Lucapa",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-9",
+        "name": "Lucapa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-10",
+      "-capital": "cty-cid-cia-Angola-Saurimo",
+      "-country": "ANG",
+      "name": "Lunda Sul",
+      "area": "56985",
+      "population": "155000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Saurimo",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-10",
+        "name": "Saurimo"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-11",
+      "-capital": "cty-cid-cia-Angola-Benguela",
+      "-country": "ANG",
+      "name": "Benguela",
+      "area": "31788",
+      "population": "644000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Benguela",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-11",
+        "name": "Benguela"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-12",
+      "-capital": "cty-cid-cia-Angola-Huambo",
+      "-country": "ANG",
+      "name": "Huambo",
+      "area": "34274",
+      "population": "1524000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Huambo",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-12",
+        "name": "Huambo"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-13",
+      "-capital": "cty-cid-cia-Angola-Bie",
+      "-country": "ANG",
+      "name": "Bie",
+      "area": "70314",
+      "population": "1125000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Bie",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-13",
+        "name": "Bie"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-14",
+      "-capital": "cty-cid-cia-Angola-Luena",
+      "-country": "ANG",
+      "name": "Moxico",
+      "area": "223023",
+      "population": "316000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Luena",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-14",
+        "name": "Luena"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-15",
+      "-capital": "cty-cid-cia-Angola-Menongue",
+      "-country": "ANG",
+      "name": "Cuando Cubango",
+      "area": "199049",
+      "population": "130000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Menongue",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-15",
+        "name": "Menongue"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-16",
+      "-capital": "cty-cid-cia-Angola-Namibe",
+      "-country": "ANG",
+      "name": "Namibe",
+      "area": "58137",
+      "population": "115000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Namibe",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-16",
+        "name": "Namibe"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-17",
+      "-capital": "cty-cid-cia-Angola-Lubango",
+      "-country": "ANG",
+      "name": "Huila",
+      "area": "75002",
+      "population": "869000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Lubango",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-17",
+        "name": "Lubango"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-18",
+      "-capital": "cty-cid-cia-Angola-Caxito",
+      "-country": "ANG",
+      "name": "Bengo",
+      "area": "31371",
+      "population": "166000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Caxito",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-18",
+        "name": "Caxito"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Angola-19",
+      "-capital": "cty-cid-cia-Angola-Ngiva",
+      "-country": "ANG",
+      "name": "Cunene",
+      "area": "88342",
+      "population": "232000",
+      "city": {
+        "-id": "cty-cid-cia-Angola-Ngiva",
+        "-is_state_cap": "yes",
+        "-country": "ANG",
+        "-province": "prov-cid-cia-Angola-19",
+        "name": "Ngiva"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RCB",
+  "-area": "342000",
+  "-capital": "cty-Congo-Brazzaville",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-CCC org-ECA org-CEEAC org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Congo",
+  "population": "2527841",
+  "population_growth": "2.19",
+  "infant_mortality": "108.1",
+  "gdp_total": "7700",
+  "gdp_agri": "11.4",
+  "gdp_ind": "35.2",
+  "gdp_serv": "53.4",
+  "inflation": "61",
+  "indep_date": "1960-08-15",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "2",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "50",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ANG",
+      "-length": "201"
+    },
+    {
+      "-country": "ZRE",
+      "-length": "2410"
+    },
+    {
+      "-country": "CAM",
+      "-length": "523"
+    },
+    {
+      "-country": "RCA",
+      "-length": "467"
+    },
+    {
+      "-country": "G",
+      "-length": "1903"
+    }
+  ],
+  "city": {
+    "-id": "cty-Congo-Brazzaville",
+    "-is_country_cap": "yes",
+    "-country": "RCB",
+    "name": "Brazzaville",
+    "longitude": "15.25",
+    "latitude": "-4.25",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Zaire"
+    }
+  }
+}
+{
+  "-car_code": "NAM",
+  "-area": "825418",
+  "-capital": "cty-cid-cia-Namibia-Windhoek",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SACU org-SADC org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Namibia",
+  "population": "1677243",
+  "population_growth": "2.93",
+  "infant_mortality": "47.2",
+  "gdp_total": "5800",
+  "inflation": "11",
+  "indep_date": "1990-03-21",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "50",
+    "#text": "Protestant"
+  },
+  "languages": [
+    {
+      "-percentage": "32",
+      "#text": "German"
+    },
+    {
+      "-percentage": "7",
+      "#text": "English"
+    },
+    {
+      "-percentage": "60",
+      "#text": "Afrikaans"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ANG",
+      "-length": "1376"
+    },
+    {
+      "-country": "Z",
+      "-length": "233"
+    },
+    {
+      "-country": "RB",
+      "-length": "1360"
+    },
+    {
+      "-country": "RSA",
+      "-length": "855"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Namibia-Windhoek",
+    "-is_country_cap": "yes",
+    "-country": "NAM",
+    "name": "Windhoek",
+    "longitude": "17.2",
+    "latitude": "-22.3"
+  }
+}
+{
+  "-car_code": "ZRE",
+  "-area": "2345410",
+  "-capital": "cty-cid-cia-Zaire-2",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-CEEAC org-CEPGL org-FAO org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Zaire",
+  "population": "46498539",
+  "population_growth": "1.67",
+  "infant_mortality": "108",
+  "gdp_total": "16500",
+  "inflation": "12",
+  "indep_date": "1960-06-30",
+  "government": "republic with a strong presidential system",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "45",
+    "#text": "Mangbetu-Azande"
+  },
+  "religions": [
+    {
+      "-percentage": "10",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "50",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Protestant"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Kimbanguist"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ANG",
+      "-length": "2511"
+    },
+    {
+      "-country": "RCB",
+      "-length": "2410"
+    },
+    {
+      "-country": "Z",
+      "-length": "1930"
+    },
+    {
+      "-country": "BI",
+      "-length": "233"
+    },
+    {
+      "-country": "RWA",
+      "-length": "217"
+    },
+    {
+      "-country": "RCA",
+      "-length": "1577"
+    },
+    {
+      "-country": "SUD",
+      "-length": "628"
+    },
+    {
+      "-country": "EAU",
+      "-length": "765"
+    },
+    {
+      "-country": "EAT",
+      "-length": "459"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Zaire-2",
+      "-capital": "cty-cid-cia-Zaire-Bandundu",
+      "-country": "ZRE",
+      "name": "Bandundu",
+      "area": "295658",
+      "population": "4644758",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-10",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-2",
+          "name": "Kikwit",
+          "population": {
+            "-year": "94",
+            "#text": "182142"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Kwilu"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-Bandundu",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-2",
+          "name": "Bandundu",
+          "longitude": "17.4",
+          "latitude": "3.3",
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Kwa"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Kwilu"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Kuango"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Kasai"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-3",
+      "-capital": "cty-cid-cia-Zaire-12",
+      "-country": "ZRE",
+      "name": "Bas Zaire",
+      "area": "53920",
+      "population": "2158595",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-12",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-3",
+          "name": "Matadi",
+          "population": {
+            "-year": "94",
+            "#text": "172730"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Zaire"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-15",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-3",
+          "name": "Boma",
+          "population": {
+            "-year": "94",
+            "#text": "135284"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Zaire"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-4",
+      "-capital": "cty-cid-cia-Zaire-13",
+      "-country": "ZRE",
+      "name": "Equateur",
+      "area": "403293",
+      "population": "3960187",
+      "city": {
+        "-id": "cty-cid-cia-Zaire-13",
+        "-is_state_cap": "yes",
+        "-country": "ZRE",
+        "-province": "prov-cid-cia-Zaire-4",
+        "name": "Mbandaka",
+        "population": {
+          "-year": "94",
+          "#text": "169841"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Zaire"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-5",
+      "-capital": "cty-cid-cia-Zaire-6",
+      "-country": "ZRE",
+      "name": "Haut Zaire",
+      "area": "503239",
+      "population": "5119750",
+      "city": {
+        "-id": "cty-cid-cia-Zaire-6",
+        "-is_state_cap": "yes",
+        "-country": "ZRE",
+        "-province": "prov-cid-cia-Zaire-5",
+        "name": "Kisangani",
+        "longitude": "25.2",
+        "latitude": "0.5",
+        "population": {
+          "-year": "94",
+          "#text": "417517"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-6",
+      "-capital": "cty-cid-cia-Zaire-7",
+      "-country": "ZRE",
+      "name": "Kasai Occidental",
+      "area": "156967",
+      "population": "3465756",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-7",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-6",
+          "name": "Kananga",
+          "population": {
+            "-year": "94",
+            "#text": "393030"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Lulua"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-11",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-6",
+          "name": "Tshikapa",
+          "population": {
+            "-year": "94",
+            "#text": "180860"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Kasai"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-7",
+      "-capital": "cty-cid-cia-Zaire-4",
+      "-country": "ZRE",
+      "name": "Kasai Oriental",
+      "area": "168216",
+      "population": "2859220",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-4",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-7",
+          "name": "Mbuji Mayi",
+          "population": {
+            "-year": "94",
+            "#text": "806475"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-14",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-7",
+          "name": "Mwene Ditu",
+          "population": {
+            "-year": "94",
+            "#text": "137459"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-8",
+      "-capital": "cty-cid-cia-Zaire-9",
+      "-country": "ZRE",
+      "name": "Kivu",
+      "area": "256662",
+      "population": "5232442",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-9",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-8",
+          "name": "Bukavu",
+          "population": {
+            "-year": "94",
+            "#text": "201569"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Kiwusee"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-16",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-8",
+          "name": "Uvira",
+          "population": {
+            "-year": "94",
+            "#text": "115590"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Tanganjikasee"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-17",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-8",
+          "name": "Butembo",
+          "population": {
+            "-year": "94",
+            "#text": "109406"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-18",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-8",
+          "name": "Goma",
+          "population": {
+            "-year": "94",
+            "#text": "109094"
+          },
+          "located_at": {
+            "-watertype": "lake",
+            "-lake": "lake-Kiwusee"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-9",
+      "-capital": "cty-cid-cia-Zaire-3",
+      "-country": "ZRE",
+      "name": "Shaba/Katanga",
+      "area": "496965",
+      "population": "4452618",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zaire-3",
+          "-is_state_cap": "yes",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-9",
+          "name": "Lubumbashi",
+          "population": {
+            "-year": "94",
+            "#text": "851381"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-5",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-9",
+          "name": "Kolwezi",
+          "population": {
+            "-year": "94",
+            "#text": "417810"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-8",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-9",
+          "name": "Likasi",
+          "population": {
+            "-year": "94",
+            "#text": "299118"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zaire-19",
+          "-country": "ZRE",
+          "-province": "prov-cid-cia-Zaire-9",
+          "name": "Kalemie",
+          "longitude": "29.2",
+          "latitude": "-5.9",
+          "population": {
+            "-year": "94",
+            "#text": "101309"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Lukuga"
+            },
+            {
+              "-watertype": "lake",
+              "-lake": "lake-Tanganjikasee"
+            }
+          ]
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zaire-10",
+      "-capital": "cty-cid-cia-Zaire-2",
+      "-country": "ZRE",
+      "name": "Kinshasa",
+      "area": "9965",
+      "population": "2778281",
+      "city": {
+        "-id": "cty-cid-cia-Zaire-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "ZRE",
+        "-province": "prov-cid-cia-Zaire-10",
+        "name": "Kinshasa",
+        "longitude": "15.2",
+        "latitude": "-4.3",
+        "population": {
+          "-year": "94",
+          "#text": "4655313"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Zaire"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "Z",
+  "-area": "752610",
+  "-capital": "cty-cid-cia-Zambia-2",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-19 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SADC org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Zambia",
+  "population": "9159072",
+  "population_growth": "2.11",
+  "infant_mortality": "96.1",
+  "gdp_total": "8900",
+  "gdp_agri": "32",
+  "gdp_ind": "22",
+  "gdp_serv": "46",
+  "inflation": "55",
+  "indep_date": "1964-10-24",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1.1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "98.7",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "60",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ANG",
+      "-length": "1110"
+    },
+    {
+      "-country": "NAM",
+      "-length": "233"
+    },
+    {
+      "-country": "ZRE",
+      "-length": "1930"
+    },
+    {
+      "-country": "ZW",
+      "-length": "797"
+    },
+    {
+      "-country": "EAT",
+      "-length": "338"
+    },
+    {
+      "-country": "MW",
+      "-length": "837"
+    },
+    {
+      "-country": "MOC",
+      "-length": "419"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Zambia-2",
+      "-capital": "cty-cid-cia-Zambia-6",
+      "-country": "Z",
+      "name": "Central",
+      "area": "94395",
+      "population": "722400",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-6",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-2",
+        "name": "Kabwe",
+        "population": {
+          "-year": "90",
+          "#text": "166519"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-3",
+      "-capital": "cty-cid-cia-Zambia-3",
+      "-country": "Z",
+      "name": "Copperbelt",
+      "area": "31328",
+      "population": "1866400",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Zambia-3",
+          "-is_state_cap": "yes",
+          "-country": "Z",
+          "-province": "prov-cid-cia-Zambia-3",
+          "name": "Ndola",
+          "population": {
+            "-year": "90",
+            "#text": "376311"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zambia-4",
+          "-country": "Z",
+          "-province": "prov-cid-cia-Zambia-3",
+          "name": "Kitwe",
+          "population": {
+            "-year": "90",
+            "#text": "338207"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zambia-5",
+          "-country": "Z",
+          "-province": "prov-cid-cia-Zambia-3",
+          "name": "Chingola",
+          "population": {
+            "-year": "90",
+            "#text": "167954"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zambia-7",
+          "-country": "Z",
+          "-province": "prov-cid-cia-Zambia-3",
+          "name": "Mufulira",
+          "population": {
+            "-year": "90",
+            "#text": "152944"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Zambia-8",
+          "-country": "Z",
+          "-province": "prov-cid-cia-Zambia-3",
+          "name": "Luanshya",
+          "population": {
+            "-year": "90",
+            "#text": "146275"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-4",
+      "-capital": "cty-cid-cia-Zambia-Chipata",
+      "-country": "Z",
+      "name": "Eastern",
+      "area": "69106",
+      "population": "826100",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Chipata",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-4",
+        "name": "Chipata"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-5",
+      "-capital": "cty-cid-cia-Zambia-Mansa",
+      "-country": "Z",
+      "name": "Luapula",
+      "area": "50567",
+      "population": "526300",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Mansa",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-5",
+        "name": "Mansa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-6",
+      "-capital": "cty-cid-cia-Zambia-2",
+      "-country": "Z",
+      "name": "Lusaka",
+      "area": "21896",
+      "population": "1151300",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-6",
+        "name": "Lusaka",
+        "longitude": "28.2",
+        "latitude": "-15.3",
+        "population": {
+          "-year": "90",
+          "#text": "982362"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-7",
+      "-capital": "cty-cid-cia-Zambia-Kasama",
+      "-country": "Z",
+      "name": "Northern",
+      "area": "147826",
+      "population": "832700",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Kasama",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-7",
+        "name": "Kasama"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-8",
+      "-capital": "cty-cid-cia-Zambia-Solwezi",
+      "-country": "Z",
+      "name": "Northwestern",
+      "area": "125827",
+      "population": "396100",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Solwezi",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-8",
+        "name": "Solwezi"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-9",
+      "-capital": "cty-cid-cia-Zambia-Livingstone",
+      "-country": "Z",
+      "name": "Southern",
+      "area": "85283",
+      "population": "906900",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Livingstone",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-9",
+        "name": "Livingstone"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Zambia-10",
+      "-capital": "cty-cid-cia-Zambia-Mongu",
+      "-country": "Z",
+      "name": "Western",
+      "area": "126386",
+      "population": "575500",
+      "city": {
+        "-id": "cty-cid-cia-Zambia-Mongu",
+        "-is_state_cap": "yes",
+        "-country": "Z",
+        "-province": "prov-cid-cia-Zambia-10",
+        "name": "Mongu"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BEN",
+  "-area": "112620",
+  "-capital": "cty-cid-cia-Benin-Porto-Novo",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-Entente org-ECA org-ECOWAS org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UNMIH org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Benin",
+  "population": "5709529",
+  "population_growth": "3.32",
+  "infant_mortality": "105.1",
+  "gdp_total": "7600",
+  "gdp_agri": "36.8",
+  "gdp_ind": "12.6",
+  "gdp_serv": "50.6",
+  "inflation": "55",
+  "indep_date": "1960-08-01",
+  "government": "republic under multiparty democratic rule dropped Marxism Leninism December 1989",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "99",
+    "#text": "African"
+  },
+  "religions": [
+    {
+      "-percentage": "15",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RN",
+      "-length": "266"
+    },
+    {
+      "-country": "BF",
+      "-length": "306"
+    },
+    {
+      "-country": "WAN",
+      "-length": "773"
+    },
+    {
+      "-country": "RT",
+      "-length": "644"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Benin-Porto-Novo",
+    "-is_country_cap": "yes",
+    "-country": "BEN",
+    "name": "Porto-Novo",
+    "longitude": "2.4",
+    "latitude": "6.3"
+  }
+}
+{
+  "-car_code": "BF",
+  "-area": "274200",
+  "-capital": "cty-cid-cia-Burkina-Faso-Ouagadougou",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-Entente org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-77 org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Burkina Faso",
+  "population": "10623323",
+  "population_growth": "2.53",
+  "infant_mortality": "117.8",
+  "gdp_total": "7400",
+  "gdp_agri": "32",
+  "gdp_ind": "24",
+  "gdp_serv": "44",
+  "inflation": "5",
+  "indep_date": "1960-08-05",
+  "government": "parliamentary",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "24",
+    "#text": "Mossi"
+  },
+  "religions": [
+    {
+      "-percentage": "50",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RMM",
+      "-length": "1000"
+    },
+    {
+      "-country": "RN",
+      "-length": "628"
+    },
+    {
+      "-country": "BEN",
+      "-length": "306"
+    },
+    {
+      "-country": "RT",
+      "-length": "126"
+    },
+    {
+      "-country": "CI",
+      "-length": "584"
+    },
+    {
+      "-country": "GH",
+      "-length": "548"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Burkina-Faso-Ouagadougou",
+    "-is_country_cap": "yes",
+    "-country": "BF",
+    "name": "Ouagadougou",
+    "longitude": "-2",
+    "latitude": "12"
+  }
+}
+{
+  "-car_code": "WAN",
+  "-area": "923770",
+  "-capital": "cty-cid-cia-Nigeria-Abuja",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-ECOWAS org-FAO org-G-15 org-G-19 org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-OPEC org-PCA org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNHCR org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Nigeria",
+  "population": "103912489",
+  "population_growth": "3.05",
+  "infant_mortality": "72.4",
+  "gdp_total": "135900",
+  "gdp_agri": "38",
+  "gdp_ind": "22",
+  "gdp_serv": "40",
+  "inflation": "57",
+  "indep_date": "1960-10-01",
+  "government": "military government",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "99",
+    "#text": "African"
+  },
+  "religions": [
+    {
+      "-percentage": "50",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RN",
+      "-length": "1497"
+    },
+    {
+      "-country": "BEN",
+      "-length": "773"
+    },
+    {
+      "-country": "CAM",
+      "-length": "1690"
+    },
+    {
+      "-country": "TCH",
+      "-length": "87"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Nigeria-Abuja",
+      "-is_country_cap": "yes",
+      "-country": "WAN",
+      "name": "Abuja",
+      "longitude": "7.5",
+      "latitude": "9.05",
+      "population": {
+        "-year": "07",
+        "#text": "182418"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-53",
+      "-country": "WAN",
+      "name": "Makurdi",
+      "population": {
+        "-year": "91",
+        "#text": "109000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-17",
+      "-country": "WAN",
+      "name": "Maiduguri",
+      "population": {
+        "-year": "91",
+        "#text": "282000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-36",
+      "-country": "WAN",
+      "name": "Calabar",
+      "population": {
+        "-year": "91",
+        "#text": "154000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-25",
+      "-country": "WAN",
+      "name": "Benin City",
+      "population": {
+        "-year": "91",
+        "#text": "203000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-18",
+      "-country": "WAN",
+      "name": "Enugu",
+      "population": {
+        "-year": "91",
+        "#text": "279000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-15",
+      "-country": "WAN",
+      "name": "Kaduna",
+      "population": {
+        "-year": "91",
+        "#text": "302000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-5",
+      "-country": "WAN",
+      "name": "Kano",
+      "population": {
+        "-year": "91",
+        "#text": "595000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-28",
+      "-country": "WAN",
+      "name": "Katsina",
+      "population": {
+        "-year": "91",
+        "#text": "182000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-7",
+      "-country": "WAN",
+      "name": "Ilorin",
+      "population": {
+        "-year": "91",
+        "#text": "420000"
+      }
+    },
+    {
+      "-id": "cty-Nigeria-Lagos",
+      "-country": "WAN",
+      "name": "Lagos",
+      "longitude": "3.4",
+      "latitude": "6.45",
+      "population": {
+        "-year": "91",
+        "#text": "5686000"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-49",
+      "-country": "WAN",
+      "name": "Minna",
+      "population": {
+        "-year": "91",
+        "#text": "120000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-8",
+      "-country": "WAN",
+      "name": "Abeokuta",
+      "population": {
+        "-year": "91",
+        "#text": "377000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-38",
+      "-country": "WAN",
+      "name": "Ondo",
+      "population": {
+        "-year": "91",
+        "#text": "150000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-39",
+      "-country": "WAN",
+      "name": "Akure",
+      "population": {
+        "-year": "91",
+        "#text": "143000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-6",
+      "-country": "WAN",
+      "name": "Oshogbo",
+      "population": {
+        "-year": "91",
+        "#text": "421000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-23",
+      "-country": "WAN",
+      "name": "Oyo",
+      "population": {
+        "-year": "91",
+        "#text": "226000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-3",
+      "-country": "WAN",
+      "name": "Ibadan",
+      "population": {
+        "-year": "91",
+        "#text": "1263000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-27",
+      "-country": "WAN",
+      "name": "Jos",
+      "population": {
+        "-year": "91",
+        "#text": "182000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-9",
+      "-country": "WAN",
+      "name": "Port Harcourt",
+      "population": {
+        "-year": "91",
+        "#text": "362000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-29",
+      "-country": "WAN",
+      "name": "Sokoto",
+      "population": {
+        "-year": "91",
+        "#text": "181000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-4",
+      "-country": "WAN",
+      "name": "Ogbomosho",
+      "population": {
+        "-year": "91",
+        "#text": "644000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-10",
+      "-country": "WAN",
+      "name": "Zaria",
+      "population": {
+        "-year": "91",
+        "#text": "335000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-11",
+      "-country": "WAN",
+      "name": "Ilesha",
+      "population": {
+        "-year": "91",
+        "#text": "334000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-12",
+      "-country": "WAN",
+      "name": "Onitsha",
+      "population": {
+        "-year": "91",
+        "#text": "328000"
+      },
+      "located_at": {
+        "-watertype": "river",
+        "-river": "river-Niger"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-13",
+      "-country": "WAN",
+      "name": "Iwo",
+      "population": {
+        "-year": "91",
+        "#text": "320000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-14",
+      "-country": "WAN",
+      "name": "Ado Ekiti",
+      "population": {
+        "-year": "91",
+        "#text": "317000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-16",
+      "-country": "WAN",
+      "name": "Mushin",
+      "population": {
+        "-year": "91",
+        "#text": "294000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-19",
+      "-country": "WAN",
+      "name": "Ede",
+      "population": {
+        "-year": "91",
+        "#text": "271000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-20",
+      "-country": "WAN",
+      "name": "Aba",
+      "population": {
+        "-year": "91",
+        "#text": "264000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-21",
+      "-country": "WAN",
+      "name": "Ife",
+      "population": {
+        "-year": "91",
+        "#text": "262000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-22",
+      "-country": "WAN",
+      "name": "Ila",
+      "population": {
+        "-year": "91",
+        "#text": "233000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-24",
+      "-country": "WAN",
+      "name": "Ikerre",
+      "population": {
+        "-year": "91",
+        "#text": "216000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-26",
+      "-country": "WAN",
+      "name": "Iseyin",
+      "population": {
+        "-year": "91",
+        "#text": "192000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-30",
+      "-country": "WAN",
+      "name": "Ilobu",
+      "population": {
+        "-year": "91",
+        "#text": "176000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-31",
+      "-country": "WAN",
+      "name": "Offa",
+      "population": {
+        "-year": "91",
+        "#text": "175000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-32",
+      "-country": "WAN",
+      "name": "Ikorodu",
+      "population": {
+        "-year": "91",
+        "#text": "163000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-33",
+      "-country": "WAN",
+      "name": "Ilawe Ekiti",
+      "population": {
+        "-year": "91",
+        "#text": "163000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-34",
+      "-country": "WAN",
+      "name": "Owo",
+      "population": {
+        "-year": "91",
+        "#text": "162000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-35",
+      "-country": "WAN",
+      "name": "Ikirun",
+      "population": {
+        "-year": "91",
+        "#text": "160000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-37",
+      "-country": "WAN",
+      "name": "Shaki",
+      "population": {
+        "-year": "91",
+        "#text": "154000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-40",
+      "-country": "WAN",
+      "name": "Gusau",
+      "population": {
+        "-year": "91",
+        "#text": "139000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-41",
+      "-country": "WAN",
+      "name": "Ijebu Ode",
+      "population": {
+        "-year": "91",
+        "#text": "138000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-42",
+      "-country": "WAN",
+      "name": "Effon Alaiye",
+      "population": {
+        "-year": "91",
+        "#text": "135000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-43",
+      "-country": "WAN",
+      "name": "Kumo",
+      "population": {
+        "-year": "91",
+        "#text": "131000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-44",
+      "-country": "WAN",
+      "name": "Shomolu",
+      "population": {
+        "-year": "91",
+        "#text": "130000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-45",
+      "-country": "WAN",
+      "name": "Oka",
+      "population": {
+        "-year": "91",
+        "#text": "126000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-46",
+      "-country": "WAN",
+      "name": "Ikare",
+      "population": {
+        "-year": "91",
+        "#text": "124000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-47",
+      "-country": "WAN",
+      "name": "Sapele",
+      "population": {
+        "-year": "91",
+        "#text": "123000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-48",
+      "-country": "WAN",
+      "name": "Deba Habe",
+      "population": {
+        "-year": "91",
+        "#text": "122000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-50",
+      "-country": "WAN",
+      "name": "Aarri",
+      "population": {
+        "-year": "91",
+        "#text": "111000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-51",
+      "-country": "WAN",
+      "name": "Bida",
+      "population": {
+        "-year": "91",
+        "#text": "111000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-52",
+      "-country": "WAN",
+      "name": "Ikire",
+      "population": {
+        "-year": "91",
+        "#text": "109000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-54",
+      "-country": "WAN",
+      "name": "Lafia",
+      "population": {
+        "-year": "91",
+        "#text": "108000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-55",
+      "-country": "WAN",
+      "name": "Inisa",
+      "population": {
+        "-year": "91",
+        "#text": "106000"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Nigeria-56",
+      "-country": "WAN",
+      "name": "Shagamu",
+      "population": {
+        "-year": "91",
+        "#text": "104000"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RT",
+  "-area": "56790",
+  "-capital": "cty-cid-cia-Togo-Lome",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-Entente org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-77 org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-MINURSO org-UNMIH org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Togo",
+  "population": "4570530",
+  "population_growth": "3.56",
+  "infant_mortality": "84.3",
+  "gdp_total": "4100",
+  "gdp_agri": "49.2",
+  "gdp_ind": "17.7",
+  "gdp_serv": "33.1",
+  "inflation": "8.8",
+  "indep_date": "1960-04-27",
+  "government": "republic under transition to multiparty democratic rule",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "99",
+      "#text": "African"
+    },
+    {
+      "-percentage": "1",
+      "#text": "European Syrian-Lebanese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "10",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BEN",
+      "-length": "644"
+    },
+    {
+      "-country": "BF",
+      "-length": "126"
+    },
+    {
+      "-country": "GH",
+      "-length": "877"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Togo-Lome",
+    "-is_country_cap": "yes",
+    "-country": "RT",
+    "name": "Lome",
+    "longitude": "1.1",
+    "latitude": "6.1",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "RB",
+  "-area": "600370",
+  "-capital": "cty-cid-cia-Botswana-Gaborone",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SACU org-SADC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WMO org-WTrO",
+  "name": "Botswana",
+  "population": "1477630",
+  "population_growth": "1.63",
+  "infant_mortality": "54.2",
+  "gdp_total": "4500",
+  "gdp_agri": "5",
+  "inflation": "10",
+  "indep_date": "1966-09-30",
+  "government": "parliamentary republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "95",
+      "#text": "Batswana"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Kgalagadi"
+    }
+  ],
+  "religions": {
+    "-percentage": "50",
+    "#text": "Christian"
+  },
+  "border": [
+    {
+      "-country": "NAM",
+      "-length": "1360"
+    },
+    {
+      "-country": "RSA",
+      "-length": "1840"
+    },
+    {
+      "-country": "ZW",
+      "-length": "813"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Botswana-Gaborone",
+    "-is_country_cap": "yes",
+    "-country": "RB",
+    "name": "Gaborone",
+    "longitude": "25.6",
+    "latitude": "-24.5"
+  }
+}
+{
+  "-car_code": "RSA",
+  "-area": "1219912",
+  "-capital": "cty-cid-cia-South-Africa-6",
+  "-memberships": "org-BIS org-C org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-MTCR org-NAM org-NSG org-OAU org-SACU org-SADC org-UN org-UNESCO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO org-ZC",
+  "name": "South Africa",
+  "population": "41743459",
+  "population_growth": "1.76",
+  "infant_mortality": "48.8",
+  "gdp_total": "215000",
+  "inflation": "8.7",
+  "indep_date": "1910-05-31",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "2.6",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "8.6",
+      "#text": "Colored"
+    }
+  ],
+  "religions": {
+    "-percentage": "2",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "NAM",
+      "-length": "855"
+    },
+    {
+      "-country": "RB",
+      "-length": "1840"
+    },
+    {
+      "-country": "ZW",
+      "-length": "225"
+    },
+    {
+      "-country": "LS",
+      "-length": "909"
+    },
+    {
+      "-country": "MOC",
+      "-length": "491"
+    },
+    {
+      "-country": "SD",
+      "-length": "430"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-South-Africa-2",
+      "-capital": "cty-cid-cia-South-Africa-Bisho",
+      "-country": "RSA",
+      "name": "Eastern Cape",
+      "area": "170616",
+      "population": "5865000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-South-Africa-7",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-2",
+          "name": "Port Elizabeth",
+          "population": {
+            "-year": "91",
+            "#text": "303000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-28",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-2",
+          "name": "East London",
+          "population": {
+            "-year": "91",
+            "#text": "102000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-Bisho",
+          "-is_state_cap": "yes",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-2",
+          "name": "Bisho",
+          "population": {
+            "-year": "91",
+            "#text": "93288"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-3",
+      "-capital": "cty-cid-cia-South-Africa-21",
+      "-country": "RSA",
+      "name": "Free State",
+      "area": "129437",
+      "population": "2470000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-21",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-3",
+        "name": "Bloemfontein",
+        "population": {
+          "-year": "91",
+          "#text": "127000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-4",
+      "-capital": "cty-cid-cia-South-Africa-4",
+      "-country": "RSA",
+      "name": "Gauteng",
+      "area": "18760",
+      "population": "7171300",
+      "city": [
+        {
+          "-id": "cty-cid-cia-South-Africa-6",
+          "-is_country_cap": "yes",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Pretoria",
+          "longitude": "28.2333",
+          "latitude": "-25.75",
+          "population": {
+            "-year": "91",
+            "#text": "526000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-4",
+          "-is_state_cap": "yes",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Johannesburg",
+          "longitude": "28.05",
+          "latitude": "-26.1833",
+          "population": {
+            "-year": "91",
+            "#text": "713000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-5",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Soweto",
+          "population": {
+            "-year": "91",
+            "#text": "597000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-11",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Tembisa",
+          "population": {
+            "-year": "91",
+            "#text": "209000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-24",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Boksburg",
+          "population": {
+            "-year": "91",
+            "#text": "120000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-25",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Carletonville",
+          "population": {
+            "-year": "91",
+            "#text": "119000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-26",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-4",
+          "name": "Benoni",
+          "population": {
+            "-year": "91",
+            "#text": "114000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-5",
+      "-capital": "cty-cid-cia-South-Africa-16",
+      "-country": "RSA",
+      "name": "Kwazulu Natal",
+      "area": "91481",
+      "population": "7672000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-South-Africa-16",
+          "-is_state_cap": "yes",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-5",
+          "name": "Pietermaritzburg",
+          "population": {
+            "-year": "91",
+            "#text": "156000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-South-Africa-3",
+          "-country": "RSA",
+          "-province": "prov-cid-cia-South-Africa-5",
+          "name": "Durban",
+          "population": {
+            "-year": "91",
+            "#text": "716000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-6",
+      "-capital": "cty-cid-cia-South-Africa-Nelspruit",
+      "-country": "RSA",
+      "name": "Mpumalanga",
+      "area": "81816",
+      "population": "2646000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-Nelspruit",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-6",
+        "name": "Nelspruit"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-7",
+      "-capital": "cty-cid-cia-South-Africa-Kimberley",
+      "-country": "RSA",
+      "name": "Northern Cape",
+      "area": "363389",
+      "population": "746000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-Kimberley",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-7",
+        "name": "Kimberley"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-8",
+      "-capital": "cty-cid-cia-South-Africa-Pietersburg",
+      "-country": "RSA",
+      "name": "Limpopo",
+      "area": "119606",
+      "population": "4128000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-Pietersburg",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-8",
+        "name": "Pietersburg"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-9",
+      "-capital": "cty-cid-cia-South-Africa-Mmabatho",
+      "-country": "RSA",
+      "name": "North West",
+      "area": "118710",
+      "population": "3043000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-Mmabatho",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-9",
+        "name": "Mmabatho"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-South-Africa-10",
+      "-capital": "cty-cid-cia-South-Africa-2",
+      "-country": "RSA",
+      "name": "Western Cape",
+      "area": "129386",
+      "population": "4118000",
+      "city": {
+        "-id": "cty-cid-cia-South-Africa-2",
+        "-is_state_cap": "yes",
+        "-country": "RSA",
+        "-province": "prov-cid-cia-South-Africa-10",
+        "name": "Cape Town",
+        "longitude": "18.4833",
+        "latitude": "-33.9333",
+        "population": {
+          "-year": "91",
+          "#text": "855000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "ZW",
+  "-area": "390580",
+  "-capital": "cty-cid-cia-Zimbabwe-Harare",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-15 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-PCA org-SADC org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Zimbabwe",
+  "population": "11271314",
+  "population_growth": "1.41",
+  "infant_mortality": "72.8",
+  "gdp_total": "18100",
+  "gdp_agri": "18.3",
+  "gdp_ind": "35.3",
+  "gdp_serv": "46.4",
+  "inflation": "25.8",
+  "indep_date": "1980-04-18",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "98",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "Z",
+      "-length": "797"
+    },
+    {
+      "-country": "RB",
+      "-length": "813"
+    },
+    {
+      "-country": "RSA",
+      "-length": "225"
+    },
+    {
+      "-country": "MOC",
+      "-length": "1231"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Zimbabwe-Harare",
+    "-is_country_cap": "yes",
+    "-country": "ZW",
+    "name": "Harare",
+    "longitude": "31.2",
+    "latitude": "-17.5"
+  }
+}
+{
+  "-car_code": "CI",
+  "-area": "322460",
+  "-capital": "cty-cid-cia-Cote-dIvoire-Yamoussoukro",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-Entente org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UNITAR org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Cote dIvoire",
+  "population": "14762445",
+  "population_growth": "2.92",
+  "infant_mortality": "82.4",
+  "gdp_total": "21900",
+  "gdp_agri": "37",
+  "gdp_ind": "24",
+  "gdp_serv": "39",
+  "inflation": "10",
+  "indep_date": "1960-08-07",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "11",
+      "#text": "Malinke"
+    },
+    {
+      "-percentage": "23",
+      "#text": "Baoule"
+    },
+    {
+      "-percentage": "18",
+      "#text": "Bete"
+    },
+    {
+      "-percentage": "15",
+      "#text": "Senoufou"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "60",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RMM",
+      "-length": "532"
+    },
+    {
+      "-country": "BF",
+      "-length": "584"
+    },
+    {
+      "-country": "GH",
+      "-length": "668"
+    },
+    {
+      "-country": "RG",
+      "-length": "610"
+    },
+    {
+      "-country": "LB",
+      "-length": "716"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-cid-cia-Cote-dIvoire-Yamoussoukro",
+      "-is_country_cap": "yes",
+      "-country": "CI",
+      "name": "Yamoussoukro",
+      "longitude": "-5.3",
+      "latitude": "6.8",
+      "population": {
+        "-year": "05",
+        "#text": "200659"
+      }
+    },
+    {
+      "-id": "cty-Abidjan",
+      "-country": "CI",
+      "name": "Abidjan",
+      "longitude": "-4.0",
+      "latitude": "5.3",
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "GH",
+  "-area": "238540",
+  "-capital": "cty-Ghana-Accra",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-ECOWAS org-FAO org-G-24 org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNIFIL org-UNIKOM org-MINURSO org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Ghana",
+  "population": "17698271",
+  "population_growth": "2.29",
+  "infant_mortality": "80.3",
+  "gdp_total": "25100",
+  "gdp_agri": "47",
+  "gdp_ind": "16",
+  "gdp_serv": "37",
+  "inflation": "69",
+  "indep_date": "1957-03-06",
+  "government": "constitutional democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.2",
+      "#text": "European"
+    },
+    {
+      "-percentage": "99.8",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "30",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "24",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "BF",
+      "-length": "548"
+    },
+    {
+      "-country": "RT",
+      "-length": "877"
+    },
+    {
+      "-country": "CI",
+      "-length": "668"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Ghana-Accra",
+      "-is_country_cap": "yes",
+      "-country": "GH",
+      "name": "Accra",
+      "longitude": "-0.2",
+      "latitude": "5.55",
+      "population": {
+        "-year": "84",
+        "#text": "867459"
+      },
+      "located_at": {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-3",
+      "-country": "GH",
+      "name": "Kumasi",
+      "population": {
+        "-year": "84",
+        "#text": "376249"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-7",
+      "-country": "GH",
+      "name": "Cape Coast",
+      "population": {
+        "-year": "84",
+        "#text": "57224"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-4",
+      "-country": "GH",
+      "name": "Tamale",
+      "population": {
+        "-year": "84",
+        "#text": "135952"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-5",
+      "-country": "GH",
+      "name": "Tema",
+      "population": {
+        "-year": "84",
+        "#text": "131528"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-6",
+      "-country": "GH",
+      "name": "Takoradi",
+      "population": {
+        "-year": "84",
+        "#text": "61484"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ghana-8",
+      "-country": "GH",
+      "name": "Sekondi",
+      "population": {
+        "-year": "84",
+        "#text": "31916"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "BI",
+  "-area": "27830",
+  "-capital": "cty-cid-cia-Burundi-Bujumbura",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-CEEAC org-CEPGL org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Burundi",
+  "population": "5943057",
+  "population_growth": "1.54",
+  "infant_mortality": "102.2",
+  "gdp_total": "4000",
+  "gdp_agri": "54.1",
+  "gdp_ind": "16.8",
+  "gdp_serv": "29.1",
+  "inflation": "10",
+  "indep_date": "1962-07-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "67",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ZRE",
+      "-length": "233"
+    },
+    {
+      "-country": "RWA",
+      "-length": "290"
+    },
+    {
+      "-country": "EAT",
+      "-length": "451"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Burundi-Bujumbura",
+    "-is_country_cap": "yes",
+    "-country": "BI",
+    "name": "Bujumbura",
+    "longitude": "29",
+    "latitude": "-4",
+    "located_at": {
+      "-watertype": "lake",
+      "-lake": "lake-Tanganjikasee"
+    }
+  }
+}
+{
+  "-car_code": "RWA",
+  "-area": "26340",
+  "-capital": "cty-cid-cia-Rwanda-Kigali",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-CEEAC org-CEPGL org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Rwanda",
+  "population": "6853359",
+  "population_growth": "16.49",
+  "infant_mortality": "118.8",
+  "gdp_total": "3800",
+  "gdp_agri": "52",
+  "gdp_ind": "13",
+  "gdp_serv": "35",
+  "inflation": "64",
+  "indep_date": "1962-07-01",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "80",
+      "#text": "Hutu"
+    },
+    {
+      "-percentage": "19",
+      "#text": "Tutsi"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Twa"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "65",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ZRE",
+      "-length": "217"
+    },
+    {
+      "-country": "BI",
+      "-length": "290"
+    },
+    {
+      "-country": "EAT",
+      "-length": "217"
+    },
+    {
+      "-country": "EAU",
+      "-length": "169"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Rwanda-Kigali",
+    "-is_country_cap": "yes",
+    "-country": "RWA",
+    "name": "Kigali",
+    "longitude": "29.6",
+    "latitude": "-1.6"
+  }
+}
+{
+  "-car_code": "EAT",
+  "-area": "945090",
+  "-capital": "cty-Tanzania-Dar-es-Salaam",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-EADB org-ECA org-FAO org-G-6 org-G-77 org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SADC org-UN org-UNAVEM-III org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Tanzania",
+  "population": "29058470",
+  "population_growth": "1.15",
+  "infant_mortality": "105.9",
+  "gdp_total": "23100",
+  "gdp_agri": "58",
+  "gdp_ind": "8",
+  "gdp_serv": "34",
+  "inflation": "25",
+  "indep_date": "1964-04-26",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "border": [
+    {
+      "-country": "ZRE",
+      "-length": "459"
+    },
+    {
+      "-country": "Z",
+      "-length": "338"
+    },
+    {
+      "-country": "BI",
+      "-length": "451"
+    },
+    {
+      "-country": "RWA",
+      "-length": "217"
+    },
+    {
+      "-country": "EAK",
+      "-length": "769"
+    },
+    {
+      "-country": "EAU",
+      "-length": "396"
+    },
+    {
+      "-country": "MW",
+      "-length": "475"
+    },
+    {
+      "-country": "MOC",
+      "-length": "756"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Tanzania-2",
+      "-capital": "cty-cid-cia-Tanzania-4",
+      "-country": "EAT",
+      "name": "Dodoma",
+      "area": "41311",
+      "population": "1237800",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-4",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-2",
+        "name": "Dodoma",
+        "population": {
+          "-year": "88",
+          "#text": "203833"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-3",
+      "-capital": "cty-cid-cia-Tanzania-8",
+      "-country": "EAT",
+      "name": "Arusha",
+      "area": "82306",
+      "population": "1351700",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-8",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-3",
+        "name": "Arusha",
+        "population": {
+          "-year": "88",
+          "#text": "134708"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-4",
+      "-capital": "cty-cid-cia-Tanzania-Moshi",
+      "-country": "EAT",
+      "name": "Kilimanjaro",
+      "area": "13309",
+      "population": "1108700",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Moshi",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-4",
+        "name": "Moshi"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-5",
+      "-capital": "cty-cid-cia-Tanzania-5",
+      "-country": "EAT",
+      "name": "Tanga",
+      "area": "26808",
+      "population": "1283600",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-5",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-5",
+        "name": "Tanga",
+        "population": {
+          "-year": "88",
+          "#text": "187155"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-6",
+      "-capital": "cty-cid-cia-Tanzania-9",
+      "-country": "EAT",
+      "name": "Morogoro",
+      "area": "70799",
+      "population": "1222700",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-9",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-6",
+        "name": "Morogoro",
+        "population": {
+          "-year": "88",
+          "#text": "117760"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-7",
+      "-capital": "cty-Tanzania-Dar-es-Salaam",
+      "-country": "EAT",
+      "name": "Pwani",
+      "area": "32407",
+      "population": "638000"
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-8",
+      "-capital": "cty-Tanzania-Dar-es-Salaam",
+      "-country": "EAT",
+      "name": "Daressalam",
+      "area": "1393",
+      "population": "1360900",
+      "city": {
+        "-id": "cty-Tanzania-Dar-es-Salaam",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-8",
+        "name": "Dar es Salaam",
+        "longitude": "39.3",
+        "latitude": "-6.83333",
+        "population": {
+          "-year": "88",
+          "#text": "1360850"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Indic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-9",
+      "-capital": "cty-cid-cia-Tanzania-Lindi",
+      "-country": "EAT",
+      "name": "Lindi",
+      "area": "66046",
+      "population": "646600",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Lindi",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-9",
+        "name": "Lindi"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-10",
+      "-capital": "cty-cid-cia-Tanzania-Mtwara-Mikandani",
+      "-country": "EAT",
+      "name": "Mtwara",
+      "area": "16707",
+      "population": "889500",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Mtwara-Mikandani",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-10",
+        "name": "Mtwara Mikandani"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-11",
+      "-capital": "cty-cid-cia-Tanzania-Songea",
+      "-country": "EAT",
+      "name": "Ruvuma",
+      "area": "63498",
+      "population": "783300",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Songea",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-11",
+        "name": "Songea"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-12",
+      "-capital": "cty-cid-cia-Tanzania-Iringa",
+      "-country": "EAT",
+      "name": "Iringa",
+      "area": "58864",
+      "population": "1208900",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Iringa",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-12",
+        "name": "Iringa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-13",
+      "-capital": "cty-cid-cia-Tanzania-7",
+      "-country": "EAT",
+      "name": "Mbeya",
+      "area": "60350",
+      "population": "1476200",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-7",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-13",
+        "name": "Mbeya",
+        "population": {
+          "-year": "88",
+          "#text": "152844"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-14",
+      "-capital": "cty-cid-cia-Tanzania-Singida",
+      "-country": "EAT",
+      "name": "Singida",
+      "area": "49341",
+      "population": "791800",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Singida",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-14",
+        "name": "Singida"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-15",
+      "-capital": "cty-cid-cia-Tanzania-Tabora",
+      "-country": "EAT",
+      "name": "Tabora",
+      "area": "76151",
+      "population": "1036300",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Tabora",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-15",
+        "name": "Tabora"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-16",
+      "-capital": "cty-cid-cia-Tanzania-Sumbawanga",
+      "-country": "EAT",
+      "name": "Rukwa",
+      "area": "68635",
+      "population": "684000",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Sumbawanga",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-16",
+        "name": "Sumbawanga"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-17",
+      "-capital": "cty-cid-cia-Tanzania-Kigoma-Ujiji",
+      "-country": "EAT",
+      "name": "Kigoma",
+      "area": "37037",
+      "population": "854800",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Kigoma-Ujiji",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-17",
+        "name": "Kigoma Ujiji"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-18",
+      "-capital": "cty-cid-cia-Tanzania-10",
+      "-country": "EAT",
+      "name": "Shinyanga",
+      "area": "50781",
+      "population": "1772500",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-10",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-18",
+        "name": "Shinyanga",
+        "population": {
+          "-year": "88",
+          "#text": "100724"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-19",
+      "-capital": "cty-cid-cia-Tanzania-Bukoba",
+      "-country": "EAT",
+      "name": "Kagera",
+      "area": "28388",
+      "population": "1326200",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Bukoba",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-19",
+        "name": "Bukoba"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-20",
+      "-capital": "cty-cid-cia-Tanzania-3",
+      "-country": "EAT",
+      "name": "Mwanza",
+      "area": "19592",
+      "population": "1878300",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-3",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-20",
+        "name": "Mwanza",
+        "population": {
+          "-year": "88",
+          "#text": "223013"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-21",
+      "-capital": "cty-cid-cia-Tanzania-Musoma",
+      "-country": "EAT",
+      "name": "Mara",
+      "area": "19566",
+      "population": "970900",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Musoma",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-21",
+        "name": "Musoma"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-22",
+      "-capital": "cty-cid-cia-Tanzania-Mkokotoni",
+      "-country": "EAT",
+      "name": "Kaskazini Ujunga",
+      "area": "470",
+      "population": "97000",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Mkokotoni",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-22",
+        "name": "Mkokotoni"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-23",
+      "-capital": "cty-cid-cia-Tanzania-Koani",
+      "-country": "EAT",
+      "name": "Kusini Ujunga",
+      "area": "854",
+      "population": "70200",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Koani",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-23",
+        "name": "Koani"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-24",
+      "-capital": "cty-cid-cia-Tanzania-6",
+      "-country": "EAT",
+      "name": "Mjini Magharibi",
+      "area": "230",
+      "population": "208300",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-6",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-24",
+        "name": "Zanzibar",
+        "population": {
+          "-year": "88",
+          "#text": "157634"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-25",
+      "-capital": "cty-cid-cia-Tanzania-Wete",
+      "-country": "EAT",
+      "name": "Kaskazini Pemba",
+      "area": "574",
+      "population": "137400",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Wete",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-25",
+        "name": "Wete"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Tanzania-26",
+      "-capital": "cty-cid-cia-Tanzania-Chake-Cahke",
+      "-country": "EAT",
+      "name": "Kusini Pemba",
+      "area": "332",
+      "population": "127600",
+      "city": {
+        "-id": "cty-cid-cia-Tanzania-Chake-Cahke",
+        "-is_state_cap": "yes",
+        "-country": "EAT",
+        "-province": "prov-cid-cia-Tanzania-26",
+        "name": "Chake Cahke"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "CAM",
+  "-area": "475440",
+  "-capital": "cty-cid-cia-Cameroon-3",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-CCC org-ECA org-CEEAC org-FAO org-FZ org-G-19 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Cameroon",
+  "population": "14261557",
+  "population_growth": "2.89",
+  "infant_mortality": "78.7",
+  "gdp_total": "16500",
+  "gdp_agri": "29",
+  "gdp_ind": "24",
+  "gdp_serv": "47",
+  "inflation": "48",
+  "indep_date": "1960-01-01",
+  "government": "unitary republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "13",
+      "#text": "African"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Fulani"
+    },
+    {
+      "-percentage": "31",
+      "#text": "Cameroon Highlanders"
+    },
+    {
+      "-percentage": "19",
+      "#text": "Equatorial Bantu"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Kirdi"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Northwestern Bantu"
+    },
+    {
+      "-percentage": "7",
+      "#text": "Eastern Nigritic"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "16",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "33",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RCB",
+      "-length": "523"
+    },
+    {
+      "-country": "WAN",
+      "-length": "1690"
+    },
+    {
+      "-country": "RCA",
+      "-length": "797"
+    },
+    {
+      "-country": "TCH",
+      "-length": "1094"
+    },
+    {
+      "-country": "GQ",
+      "-length": "189"
+    },
+    {
+      "-country": "G",
+      "-length": "298"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Cameroon-2",
+      "-capital": "cty-cid-cia-Cameroon-Ngaoundere",
+      "-country": "CAM",
+      "name": "Adamaoua",
+      "area": "63691",
+      "population": "495200",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-Ngaoundere",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-2",
+        "name": "Ngaoundere"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-3",
+      "-capital": "cty-cid-cia-Cameroon-3",
+      "-country": "CAM",
+      "name": "Centre",
+      "area": "68926",
+      "population": "1651600",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-3",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-3",
+        "name": "Yaounde",
+        "longitude": "12",
+        "latitude": "3.5",
+        "population": {
+          "-year": "91",
+          "#text": "750000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-4",
+      "-capital": "cty-cid-cia-Cameroon-Bertoua",
+      "-country": "CAM",
+      "name": "Est",
+      "area": "109011",
+      "population": "517200",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-Bertoua",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-4",
+        "name": "Bertoua"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-5",
+      "-capital": "cty-cid-cia-Cameroon-5",
+      "-country": "CAM",
+      "name": "Nord extreme",
+      "area": "34246",
+      "population": "1855700",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-5",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-5",
+        "name": "Maroua",
+        "population": {
+          "-year": "91",
+          "#text": "143000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-6",
+      "-capital": "cty-cid-cia-Cameroon-2",
+      "-country": "CAM",
+      "name": "Cote/Littoral",
+      "area": "20239",
+      "population": "1354800",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-2",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-6",
+        "name": "Douala",
+        "longitude": "10",
+        "latitude": "3.5",
+        "population": {
+          "-year": "91",
+          "#text": "884000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-7",
+      "-capital": "cty-cid-cia-Cameroon-4",
+      "-country": "CAM",
+      "name": "Nord",
+      "area": "65576",
+      "population": "832200",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-4",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-7",
+        "name": "Garoua",
+        "population": {
+          "-year": "91",
+          "#text": "177000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Benue"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-8",
+      "-capital": "cty-cid-cia-Cameroon-6",
+      "-country": "CAM",
+      "name": "Nordoueste",
+      "area": "17810",
+      "population": "1237300",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-6",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-8",
+        "name": "Bamenda",
+        "population": {
+          "-year": "91",
+          "#text": "138000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-9",
+      "-capital": "cty-cid-cia-Cameroon-7",
+      "-country": "CAM",
+      "name": "Ouest",
+      "area": "13872",
+      "population": "1339800",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-7",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-9",
+        "name": "Bafoussam",
+        "population": {
+          "-year": "91",
+          "#text": "131000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-10",
+      "-capital": "cty-cid-cia-Cameroon-Ebolowa",
+      "-country": "CAM",
+      "name": "Sud",
+      "area": "47110",
+      "population": "373800",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-Ebolowa",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-10",
+        "name": "Ebolowa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Cameroon-11",
+      "-capital": "cty-cid-cia-Cameroon-Buea",
+      "-country": "CAM",
+      "name": "Sudoueste",
+      "area": "24471",
+      "population": "838000",
+      "city": {
+        "-id": "cty-cid-cia-Cameroon-Buea",
+        "-is_state_cap": "yes",
+        "-country": "CAM",
+        "-province": "prov-cid-cia-Cameroon-11",
+        "name": "Buea"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "RCA",
+  "-area": "622980",
+  "-capital": "cty-cid-cia-Central-African-Republic-Bangui",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-CCC org-ECA org-CEEAC org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Central African Republic",
+  "population": "3274426",
+  "population_growth": "2.08",
+  "infant_mortality": "111.7",
+  "gdp_total": "2500",
+  "gdp_agri": "50",
+  "gdp_ind": "14",
+  "gdp_serv": "36",
+  "inflation": "45",
+  "indep_date": "1960-08-13",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "34",
+      "#text": "Baya"
+    },
+    {
+      "-percentage": "27",
+      "#text": "Banda"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Sara"
+    },
+    {
+      "-percentage": "21",
+      "#text": "Mandjia"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Mboum"
+    },
+    {
+      "-percentage": "4",
+      "#text": "MBaka"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "15",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RCB",
+      "-length": "467"
+    },
+    {
+      "-country": "ZRE",
+      "-length": "1577"
+    },
+    {
+      "-country": "CAM",
+      "-length": "797"
+    },
+    {
+      "-country": "TCH",
+      "-length": "1197"
+    },
+    {
+      "-country": "SUD",
+      "-length": "1165"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Central-African-Republic-Bangui",
+    "-is_country_cap": "yes",
+    "-country": "RCA",
+    "name": "Bangui",
+    "longitude": "18.5",
+    "latitude": "4.3",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Ubangi"
+    }
+  }
+}
+{
+  "-car_code": "TCH",
+  "-area": "1284000",
+  "-capital": "cty-cid-cia-Chad-NDjamena",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-ECA org-CEEAC org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Chad",
+  "population": "6976845",
+  "population_growth": "2.68",
+  "infant_mortality": "120.4",
+  "gdp_total": "3300",
+  "gdp_agri": "49",
+  "gdp_ind": "17",
+  "gdp_serv": "34",
+  "inflation": "41",
+  "indep_date": "1960-08-11",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "50",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "LAR",
+      "-length": "1055"
+    },
+    {
+      "-country": "RN",
+      "-length": "1175"
+    },
+    {
+      "-country": "WAN",
+      "-length": "87"
+    },
+    {
+      "-country": "CAM",
+      "-length": "1094"
+    },
+    {
+      "-country": "RCA",
+      "-length": "1197"
+    },
+    {
+      "-country": "SUD",
+      "-length": "1360"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Chad-NDjamena",
+    "-is_country_cap": "yes",
+    "-country": "TCH",
+    "name": "NDjamena",
+    "longitude": "15",
+    "latitude": "12",
+    "located_at": {
+      "-watertype": "river",
+      "-river": "river-Schari"
+    }
+  }
+}
+{
+  "-car_code": "GQ",
+  "-area": "28050",
+  "-capital": "cty-cid-cia-Equatorial-Guinea-Malabo",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-ECA org-CEEAC org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ITU org-Intelsat org-NAM org-OAU org-OAS org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WTrO",
+  "name": "Equatorial Guinea",
+  "population": "431282",
+  "population_growth": "2.58",
+  "infant_mortality": "98",
+  "gdp_total": "325",
+  "gdp_agri": "47",
+  "gdp_ind": "26",
+  "gdp_serv": "27",
+  "inflation": "41",
+  "indep_date": "1968-10-12",
+  "government": "republic in transition to multiparty democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "border": [
+    {
+      "-country": "CAM",
+      "-length": "189"
+    },
+    {
+      "-country": "G",
+      "-length": "350"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Equatorial-Guinea-Malabo",
+    "-is_country_cap": "yes",
+    "-country": "GQ",
+    "name": "Malabo",
+    "longitude": "9",
+    "latitude": "4",
+    "population": {
+      "-year": "87",
+      "#text": "33000"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Bioko" }
+  }
+}
+{
+  "-car_code": "G",
+  "-area": "267670",
+  "-capital": "cty-cid-cia-Gabon-Libreville",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-UDEAC org-BDEAC org-CCC org-ECA org-CEEAC org-FAO org-FZ org-G-24 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OPEC org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Gabon",
+  "population": "1172798",
+  "population_growth": "1.47",
+  "infant_mortality": "90.1",
+  "gdp_total": "6000",
+  "gdp_agri": "8.2",
+  "gdp_ind": "44.7",
+  "gdp_serv": "47.1",
+  "inflation": "15",
+  "indep_date": "1960-08-17",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "1",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "55",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RCB",
+      "-length": "1903"
+    },
+    {
+      "-country": "CAM",
+      "-length": "298"
+    },
+    {
+      "-country": "GQ",
+      "-length": "350"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Gabon-Libreville",
+    "-is_country_cap": "yes",
+    "-country": "G",
+    "name": "Libreville",
+    "longitude": "9.3",
+    "latitude": "0.3",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "CV",
+  "-area": "4030",
+  "-capital": "cty-cid-cia-Cape-Verde-Praia",
+  "-memberships": "org-ACP org-AfDB org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WMO org-WTrO",
+  "name": "Cape Verde",
+  "population": "449066",
+  "population_growth": "2.93",
+  "infant_mortality": "54.3",
+  "gdp_total": "440",
+  "gdp_agri": "13",
+  "gdp_ind": "17",
+  "gdp_serv": "70",
+  "inflation": "5",
+  "indep_date": "1975-07-05",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "28",
+      "#text": "African"
+    },
+    {
+      "-percentage": "71",
+      "#text": "Creole"
+    }
+  ],
+  "religions": {
+    "-percentage": "100",
+    "#text": "Roman Catholic"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Cape-Verde-Praia",
+    "-is_country_cap": "yes",
+    "-country": "CV",
+    "name": "Praia",
+    "longitude": "-23",
+    "latitude": "16",
+    "population": "105000",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Santiago" }
+  }
+}
+{
+  "-car_code": "SUD",
+  "-area": "2505810",
+  "-capital": "cty-Sudan-Khartoum",
+  "-memberships": "org-ACP org-AfDB org-ABEDA org-AFESD org-AL org-AMF org-CAEU org-CCC org-ECA org-FAO org-G-77 org-IGADD org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UNHCR org-UNU org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Sudan",
+  "population": "31547543",
+  "population_growth": "3.48",
+  "infant_mortality": "76",
+  "gdp_total": "25000",
+  "gdp_agri": "33",
+  "gdp_ind": "17",
+  "gdp_serv": "50",
+  "inflation": "66",
+  "indep_date": "1956-01-01",
+  "government": "transitional previously ruling military junta",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "39",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Beja"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "70",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ET",
+      "-length": "1273"
+    },
+    {
+      "-country": "LAR",
+      "-length": "383"
+    },
+    {
+      "-country": "ZRE",
+      "-length": "628"
+    },
+    {
+      "-country": "RCA",
+      "-length": "1165"
+    },
+    {
+      "-country": "TCH",
+      "-length": "1360"
+    },
+    {
+      "-country": "ER",
+      "-length": "605"
+    },
+    {
+      "-country": "ETH",
+      "-length": "1606"
+    },
+    {
+      "-country": "EAK",
+      "-length": "232"
+    },
+    {
+      "-country": "EAU",
+      "-length": "435"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Sudan-2",
+      "-capital": "cty-cid-cia-Sudan-Malakal",
+      "-country": "SUD",
+      "name": "Aali an Nil",
+      "area": "238792",
+      "population": "1599605",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-Malakal",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-2",
+        "name": "Malakal",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Weisser_Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-3",
+      "-capital": "cty-cid-cia-Sudan-Waw",
+      "-country": "SUD",
+      "name": "Bahr al Ghazal",
+      "area": "200894",
+      "population": "2265510",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-Waw",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-3",
+        "name": "Waw"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-4",
+      "-capital": "cty-cid-cia-Sudan-al-Fasher",
+      "-country": "SUD",
+      "name": "Darfur",
+      "area": "508684",
+      "population": "3093699",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Sudan-7",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-4",
+          "name": "Nyala",
+          "population": {
+            "-year": "93",
+            "#text": "228778"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Sudan-al-Fasher",
+          "-is_state_cap": "yes",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-4",
+          "name": "al Fasher"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-5",
+      "-capital": "cty-cid-cia-Sudan-11",
+      "-country": "SUD",
+      "name": "al Istiwaiyah",
+      "area": "197969",
+      "population": "1406181",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-11",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-5",
+        "name": "Juba",
+        "population": {
+          "-year": "93",
+          "#text": "114980"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Bahr_el-Djebel"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-6",
+      "-capital": "cty-Sudan-Khartoum",
+      "-country": "SUD",
+      "name": "al Khartum",
+      "area": "28165",
+      "population": "1802299",
+      "city": [
+        {
+          "-id": "cty-Sudan-Khartoum",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-6",
+          "name": "Khartoum",
+          "longitude": "32.55",
+          "latitude": "15.6167",
+          "population": {
+            "-year": "93",
+            "#text": "924505"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Nil"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Blauer_Nil"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Sudan-2",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-6",
+          "name": "Omdurman",
+          "population": {
+            "-year": "93",
+            "#text": "1267077"
+          },
+          "located_at": [
+            {
+              "-watertype": "river",
+              "-river": "river-Nil"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Weisser_Nil"
+            },
+            {
+              "-watertype": "river",
+              "-river": "river-Blauer_Nil"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Sudan-4",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-6",
+          "name": "Khartoum North",
+          "population": {
+            "-year": "93",
+            "#text": "879105"
+          },
+          "located_at": {
+            "-watertype": "river",
+            "-river": "river-Nil"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-7",
+      "-capital": "cty-cid-cia-Sudan-8",
+      "-country": "SUD",
+      "name": "Kurdufan",
+      "area": "380255",
+      "population": "3093294",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-8",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-7",
+        "name": "al Ubayyid",
+        "population": {
+          "-year": "93",
+          "#text": "228096"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-8",
+      "-capital": "cty-cid-cia-Sudan-ad-Damir",
+      "-country": "SUD",
+      "name": "ash Shamaliyah",
+      "area": "476040",
+      "population": "1083024",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-ad-Damir",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-8",
+        "name": "ad Damir",
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Nil"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-9",
+      "-capital": "cty-cid-cia-Sudan-6",
+      "-country": "SUD",
+      "name": "ash Sharqiyah",
+      "area": "334074",
+      "population": "2208209",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Sudan-6",
+          "-is_state_cap": "yes",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-9",
+          "name": "Kassala",
+          "population": {
+            "-year": "93",
+            "#text": "234270"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Sudan-5",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-9",
+          "name": "Port Sudan",
+          "population": {
+            "-year": "93",
+            "#text": "305385"
+          },
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Rotes_Meer"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Sudan-10",
+          "-country": "SUD",
+          "-province": "prov-cid-cia-Sudan-9",
+          "name": "al Qadarif",
+          "population": {
+            "-year": "93",
+            "#text": "189384"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Sudan-10",
+      "-capital": "cty-cid-cia-Sudan-9",
+      "-country": "SUD",
+      "name": "Nil al Asraq",
+      "area": "139017",
+      "population": "4012543",
+      "city": {
+        "-id": "cty-cid-cia-Sudan-9",
+        "-is_state_cap": "yes",
+        "-country": "SUD",
+        "-province": "prov-cid-cia-Sudan-10",
+        "name": "Wad Madani",
+        "population": {
+          "-year": "93",
+          "#text": "218714"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Blauer_Nil"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "COM",
+  "-area": "2170",
+  "-capital": "cty-cid-cia-Comoros-Moroni",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-AL org-CCC org-ECA org-FAO org-FZ org-G-77 org-IBRD org-ICAO org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WMO",
+  "name": "Comoros",
+  "population": "569237",
+  "population_growth": "3.55",
+  "infant_mortality": "75.3",
+  "gdp_total": "370",
+  "inflation": "15",
+  "indep_date": "1975-07-06",
+  "government": "independent republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "14",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "86",
+      "#text": "Muslim"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Comoros-Moroni",
+    "-is_country_cap": "yes",
+    "-country": "COM",
+    "name": "Moroni",
+    "longitude": "43",
+    "latitude": "-12",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Gr_Komoro" }
+  }
+}
+{
+  "-car_code": "RG",
+  "-area": "245860",
+  "-capital": "cty-cid-cia-Guinea-Conakry",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAMIR org-UNESCO org-UNIDO org-MINURSO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Guinea",
+  "population": "7411981",
+  "population_growth": "1.85",
+  "infant_mortality": "134.1",
+  "gdp_total": "6500",
+  "gdp_agri": "24",
+  "gdp_ind": "31",
+  "gdp_serv": "45",
+  "inflation": "4.1",
+  "indep_date": "1958-10-02",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "30",
+      "#text": "Malinke"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Peuhl"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Soussou"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "85",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Christian"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "French"
+  },
+  "border": [
+    {
+      "-country": "RMM",
+      "-length": "858"
+    },
+    {
+      "-country": "CI",
+      "-length": "610"
+    },
+    {
+      "-country": "LB",
+      "-length": "563"
+    },
+    {
+      "-country": "SN",
+      "-length": "330"
+    },
+    {
+      "-country": "GNB",
+      "-length": "386"
+    },
+    {
+      "-country": "WAL",
+      "-length": "652"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Guinea-Conakry",
+    "-is_country_cap": "yes",
+    "-country": "RG",
+    "name": "Conakry",
+    "longitude": "-13.5",
+    "latitude": "9.3",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "LB",
+  "-area": "111370",
+  "-capital": "cty-Liberia-Monrovia",
+  "-memberships": "org-ACP org-AfDB org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO",
+  "name": "Liberia",
+  "population": "2109789",
+  "population_growth": "2.13",
+  "infant_mortality": "108.1",
+  "gdp_total": "2300",
+  "inflation": "50",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "95",
+      "#text": "African"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Americo-Liberians"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "20",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Christian"
+    }
+  ],
+  "languages": {
+    "-percentage": "20",
+    "#text": "English"
+  },
+  "border": [
+    {
+      "-country": "CI",
+      "-length": "716"
+    },
+    {
+      "-country": "RG",
+      "-length": "563"
+    },
+    {
+      "-country": "WAL",
+      "-length": "306"
+    }
+  ],
+  "city": {
+    "-id": "cty-Liberia-Monrovia",
+    "-is_country_cap": "yes",
+    "-country": "LB",
+    "name": "Monrovia",
+    "longitude": "-10.8",
+    "latitude": "6.3",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "DJI",
+  "-area": "22000",
+  "-capital": "cty-cid-cia-Djibouti-Djibouti",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-AFESD org-AL org-ECA org-FAO org-G-77 org-IGADD org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAMIR org-UNESCO org-UNIDO org-UNMIH org-UPU org-WFTU org-WHO org-WMO org-WTrO",
+  "name": "Djibouti",
+  "population": "427642",
+  "population_growth": "1.5",
+  "infant_mortality": "106.7",
+  "gdp_total": "500",
+  "gdp_agri": "3",
+  "gdp_ind": "21",
+  "gdp_serv": "76",
+  "inflation": "6",
+  "indep_date": "1977-06-27",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "5",
+      "#text": "Italian"
+    },
+    {
+      "-percentage": "35",
+      "#text": "Afar"
+    },
+    {
+      "-percentage": "60",
+      "#text": "Somali"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "94",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ER",
+      "-length": "113"
+    },
+    {
+      "-country": "ETH",
+      "-length": "337"
+    },
+    {
+      "-country": "SP",
+      "-length": "58"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Djibouti-Djibouti",
+    "-is_country_cap": "yes",
+    "-country": "DJI",
+    "name": "Djibouti",
+    "longitude": "43.1",
+    "latitude": "11.5",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Golf_von_Aden"
+    }
+  }
+}
+{
+  "-car_code": "ER",
+  "-area": "121320",
+  "-capital": "cty-Eritrea-Asmara",
+  "-memberships": "org-ACP org-AfDB org-ECA org-FAO org-IGADD org-IBRD org-ICAO org-IDA org-IFAD org-ILO org-IMO org-IMF org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO",
+  "name": "Eritrea",
+  "population": "3427883",
+  "population_growth": "2.79",
+  "infant_mortality": "118.9",
+  "gdp_total": "2000",
+  "inflation": "10",
+  "indep_date": "1993-05-27",
+  "government": "transitional government",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "Afar"
+    },
+    {
+      "-percentage": "50",
+      "#text": "Tigrinya"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Tigre Kunama"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Saho"
+    }
+  ],
+  "border": [
+    {
+      "-country": "SUD",
+      "-length": "605"
+    },
+    {
+      "-country": "DJI",
+      "-length": "113"
+    },
+    {
+      "-country": "ETH",
+      "-length": "912"
+    }
+  ],
+  "city": {
+    "-id": "cty-Eritrea-Asmara",
+    "-is_country_cap": "yes",
+    "-country": "ER",
+    "name": "Asmara",
+    "longitude": "38.9167",
+    "latitude": "15.2833"
+  }
+}
+{
+  "-car_code": "ETH",
+  "-area": "1127127",
+  "-capital": "cty-Ethiopia-Addis-Ababa",
+  "-memberships": "org-ACP org-AfDB org-CCC org-ECA org-FAO org-G-24 org-G-77 org-IGADD org-IAEA org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UNHCR org-UNU org-UPU org-WFTU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Ethiopia",
+  "population": "57171662",
+  "population_growth": "2.72",
+  "infant_mortality": "122.8",
+  "gdp_total": "24200",
+  "gdp_agri": "48",
+  "gdp_ind": "15",
+  "gdp_serv": "37",
+  "inflation": "10",
+  "government": "federal republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "4",
+      "#text": "Afar"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Somali"
+    },
+    {
+      "-percentage": "40",
+      "#text": "Oromo"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Sidamo"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Shankella"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Gurage"
+    },
+    {
+      "-percentage": "32",
+      "#text": "Amhara Tigrean"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "45",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "35",
+      "#text": "Christian Orthodox"
+    }
+  ],
+  "border": [
+    {
+      "-country": "SUD",
+      "-length": "1606"
+    },
+    {
+      "-country": "DJI",
+      "-length": "337"
+    },
+    {
+      "-country": "ER",
+      "-length": "912"
+    },
+    {
+      "-country": "SP",
+      "-length": "1626"
+    },
+    {
+      "-country": "EAK",
+      "-length": "830"
+    }
+  ],
+  "city": [
+    {
+      "-id": "cty-Ethiopia-Addis-Ababa",
+      "-is_country_cap": "yes",
+      "-country": "ETH",
+      "name": "Addis Ababa",
+      "longitude": "38.75",
+      "latitude": "9",
+      "population": {
+        "-year": "94",
+        "#text": "2316400"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-3",
+      "-country": "ETH",
+      "name": "Dire Dawa",
+      "population": {
+        "-year": "94",
+        "#text": "194587"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-4",
+      "-country": "ETH",
+      "name": "Gonder",
+      "population": {
+        "-year": "94",
+        "#text": "166593"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-5",
+      "-country": "ETH",
+      "name": "Nazret",
+      "population": {
+        "-year": "94",
+        "#text": "147088"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-6",
+      "-country": "ETH",
+      "name": "Harer",
+      "population": {
+        "-year": "94",
+        "#text": "122932"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-7",
+      "-country": "ETH",
+      "name": "Mekele",
+      "population": {
+        "-year": "94",
+        "#text": "119779"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-8",
+      "-country": "ETH",
+      "name": "Jima",
+      "population": {
+        "-year": "94",
+        "#text": "119717"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-9",
+      "-country": "ETH",
+      "name": "Dese",
+      "population": {
+        "-year": "94",
+        "#text": "117268"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-10",
+      "-country": "ETH",
+      "name": "Bahir Dar",
+      "population": {
+        "-year": "94",
+        "#text": "115531"
+      }
+    },
+    {
+      "-id": "cty-cid-cia-Ethiopia-11",
+      "-country": "ETH",
+      "name": "Debrezit",
+      "population": {
+        "-year": "94",
+        "#text": "105963"
+      }
+    }
+  ]
+}
+{
+  "-car_code": "SP",
+  "-area": "637660",
+  "-capital": "cty-cid-cia-Somalia-Mogadishu",
+  "-memberships": "org-ACP org-AfDB org-AFESD org-AL org-AMF org-CAEU org-ECA org-FAO org-G-77 org-IGADD org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Somalia",
+  "population": "9639151",
+  "population_growth": "3.1",
+  "infant_mortality": "121.1",
+  "gdp_total": "3600",
+  "indep_date": "1960-07-01",
+  "government": "none",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "85",
+    "#text": "Somali"
+  },
+  "religions": {
+    "-percentage": "100",
+    "#text": "Muslim"
+  },
+  "border": [
+    {
+      "-country": "DJI",
+      "-length": "58"
+    },
+    {
+      "-country": "ETH",
+      "-length": "1626"
+    },
+    {
+      "-country": "EAK",
+      "-length": "682"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Somalia-Mogadishu",
+    "-is_country_cap": "yes",
+    "-country": "SP",
+    "name": "Mogadishu",
+    "longitude": "45.3",
+    "latitude": "2.2",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    }
+  }
+}
+{
+  "-car_code": "EAK",
+  "-area": "582650",
+  "-capital": "cty-Kenya-Nairobi",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-EADB org-ECA org-FAO org-G-77 org-IGADD org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNAVEM-III org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-MINURSO org-UNOMIL org-UNPREDEP org-UNPROFOR org-UNU org-UPU org-WCL org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Kenya",
+  "population": "28176686",
+  "population_growth": "2.27",
+  "infant_mortality": "55.3",
+  "gdp_total": "36800",
+  "gdp_agri": "27",
+  "gdp_ind": "19",
+  "gdp_serv": "54",
+  "inflation": "1.7",
+  "indep_date": "1963-12-12",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "22",
+      "#text": "Kikuyu"
+    },
+    {
+      "-percentage": "14",
+      "#text": "Luhya"
+    },
+    {
+      "-percentage": "13",
+      "#text": "Luo"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Kalenjin"
+    },
+    {
+      "-percentage": "11",
+      "#text": "Kamba"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Kisii"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Meru"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "28",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "38",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "EAT",
+      "-length": "769"
+    },
+    {
+      "-country": "SUD",
+      "-length": "232"
+    },
+    {
+      "-country": "ETH",
+      "-length": "830"
+    },
+    {
+      "-country": "SP",
+      "-length": "682"
+    },
+    {
+      "-country": "EAU",
+      "-length": "933"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Kenya-2",
+      "-capital": "cty-cid-cia-Kenya-14",
+      "-country": "EAK",
+      "name": "Central",
+      "area": "13176",
+      "population": "3110000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kenya-14",
+          "-is_state_cap": "yes",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-2",
+          "name": "Nyeri",
+          "population": {
+            "-year": "89",
+            "#text": "88600"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-16",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-2",
+          "name": "Thika",
+          "population": {
+            "-year": "89",
+            "#text": "57100"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-3",
+      "-capital": "cty-cid-cia-Kenya-3",
+      "-country": "EAK",
+      "name": "Coast",
+      "area": "83603",
+      "population": "1880000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kenya-3",
+          "-is_state_cap": "yes",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-3",
+          "name": "Mombasa",
+          "longitude": "39.4",
+          "latitude": "-4",
+          "population": [
+            {
+              "-year": "89",
+              "#text": "465000"
+            },
+            {
+              "-year": "91",
+              "#text": "600000"
+            }
+          ],
+          "located_at": {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-21",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-3",
+          "name": "Malindi",
+          "population": {
+            "-year": "89",
+            "#text": "35200"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-4",
+      "-capital": "cty-cid-cia-Kenya-Embu",
+      "-country": "EAK",
+      "name": "Eastern",
+      "area": "159891",
+      "population": "3724000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kenya-12",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-4",
+          "name": "Machakos",
+          "population": {
+            "-year": "89",
+            "#text": "116100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-15",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-4",
+          "name": "Meru",
+          "population": {
+            "-year": "89",
+            "#text": "78100"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-Embu",
+          "-is_state_cap": "yes",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-4",
+          "name": "Embu"
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-5",
+      "-capital": "cty-Kenya-Nairobi",
+      "-country": "EAK",
+      "name": "Nairobi",
+      "area": "684",
+      "population": "1346000",
+      "city": {
+        "-id": "cty-Kenya-Nairobi",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "EAK",
+        "-province": "prov-cid-cia-Kenya-5",
+        "name": "Nairobi",
+        "longitude": "36.8",
+        "latitude": "-1.26667",
+        "population": [
+          {
+            "-year": "89",
+            "#text": "1346000"
+          },
+          {
+            "-year": "91",
+            "#text": "2000000"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-6",
+      "-capital": "cty-cid-cia-Kenya-Garissa",
+      "-country": "EAK",
+      "name": "North Eastern",
+      "area": "126090",
+      "population": "372000",
+      "city": {
+        "-id": "cty-cid-cia-Kenya-Garissa",
+        "-is_state_cap": "yes",
+        "-country": "EAK",
+        "-province": "prov-cid-cia-Kenya-6",
+        "name": "Garissa"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-7",
+      "-capital": "cty-cid-cia-Kenya-4",
+      "-country": "EAK",
+      "name": "Nyanza",
+      "area": "16162",
+      "population": "3558000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kenya-4",
+          "-is_state_cap": "yes",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-7",
+          "name": "Kisumu",
+          "population": [
+            {
+              "-year": "89",
+              "#text": "185100"
+            },
+            {
+              "-year": "91",
+              "#text": "201100"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-19",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-7",
+          "name": "Kisii",
+          "population": {
+            "-year": "89",
+            "#text": "44000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-8",
+      "-capital": "cty-cid-cia-Kenya-5",
+      "-country": "EAK",
+      "name": "Rift Valley",
+      "area": "173868",
+      "population": "4894000",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Kenya-5",
+          "-is_state_cap": "yes",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-8",
+          "name": "Nakuru",
+          "population": [
+            {
+              "-year": "89",
+              "#text": "162800"
+            },
+            {
+              "-year": "91",
+              "#text": "124200"
+            }
+          ]
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-13",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-8",
+          "name": "Eldoret",
+          "population": {
+            "-year": "89",
+            "#text": "104900"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-17",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-8",
+          "name": "Kitale",
+          "population": {
+            "-year": "89",
+            "#text": "53000"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Kenya-20",
+          "-country": "EAK",
+          "-province": "prov-cid-cia-Kenya-8",
+          "name": "Kericho",
+          "population": {
+            "-year": "89",
+            "#text": "40000"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Kenya-9",
+      "-capital": "cty-cid-cia-Kenya-18",
+      "-country": "EAK",
+      "name": "Western",
+      "area": "8360",
+      "population": "2543000",
+      "city": {
+        "-id": "cty-cid-cia-Kenya-18",
+        "-is_state_cap": "yes",
+        "-country": "EAK",
+        "-province": "prov-cid-cia-Kenya-9",
+        "name": "Kakamega",
+        "population": {
+          "-year": "89",
+          "#text": "47300"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "WAG",
+  "-area": "11300",
+  "-capital": "cty-cid-cia-Gambia-Banjul",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO",
+  "name": "Gambia",
+  "population": "1204984",
+  "population_growth": "3.55",
+  "infant_mortality": "80.5",
+  "gdp_total": "1100",
+  "gdp_agri": "27",
+  "gdp_ind": "15",
+  "gdp_serv": "58",
+  "inflation": "1.7",
+  "indep_date": "1965-02-18",
+  "government": "republic under multiparty democratic rule",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "99",
+    "#text": "African"
+  },
+  "religions": [
+    {
+      "-percentage": "90",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Christian"
+    }
+  ],
+  "border": {
+    "-country": "SN",
+    "-length": "740"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Gambia-Banjul",
+    "-is_country_cap": "yes",
+    "-country": "WAG",
+    "name": "Banjul",
+    "longitude": "-17",
+    "latitude": "13.5",
+    "located_at": [
+      {
+        "-watertype": "sea",
+        "-sea": "sea-Atlantic"
+      },
+      {
+        "-watertype": "river",
+        "-river": "river-Gambia"
+      }
+    ]
+  }
+}
+{
+  "-car_code": "SN",
+  "-area": "196190",
+  "-capital": "cty-Senegal-Dakar",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-ECOWAS org-FAO org-FZ org-G-15 org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-PCA org-UN org-UNAVEM-III org-UNAMIR org-UNCRO org-UNESCO org-UNIDO org-UNIKOM org-UPU org-WADB org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Senegal",
+  "population": "9092749",
+  "population_growth": "3.37",
+  "infant_mortality": "64",
+  "gdp_total": "14500",
+  "inflation": "6.1",
+  "indep_date": "1960-08-20",
+  "government": "republic under multiparty democratic rule",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "17",
+      "#text": "Fulani"
+    },
+    {
+      "-percentage": "36",
+      "#text": "Wolof"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Diola"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Mandingo"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Serer"
+    },
+    {
+      "-percentage": "9",
+      "#text": "Toucouleur"
+    },
+    {
+      "-percentage": "1",
+      "#text": "European Lebanese"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "92",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RMM",
+      "-length": "419"
+    },
+    {
+      "-country": "RIM",
+      "-length": "813"
+    },
+    {
+      "-country": "RG",
+      "-length": "330"
+    },
+    {
+      "-country": "WAG",
+      "-length": "740"
+    },
+    {
+      "-country": "GNB",
+      "-length": "338"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Senegal-2",
+      "-capital": "cty-Senegal-Dakar",
+      "-country": "SN",
+      "name": "Dakar",
+      "area": "550",
+      "population": "1490500",
+      "city": {
+        "-id": "cty-Senegal-Dakar",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-2",
+        "name": "Dakar",
+        "longitude": "-17.4833",
+        "latitude": "14.7",
+        "population": {
+          "-year": "85",
+          "#text": "1382000"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Atlantic"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-3",
+      "-capital": "cty-cid-cia-Senegal-Diourbel",
+      "-country": "SN",
+      "name": "Diourbel",
+      "area": "4359",
+      "population": "619700",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-Diourbel",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-3",
+        "name": "Diourbel"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-4",
+      "-capital": "cty-cid-cia-Senegal-Fatick",
+      "-country": "SN",
+      "name": "Fatick",
+      "area": "7935",
+      "population": "476000",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-Fatick",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-4",
+        "name": "Fatick"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-5",
+      "-capital": "cty-cid-cia-Senegal-4",
+      "-country": "SN",
+      "name": "Kaolack",
+      "area": "16010",
+      "population": "816400",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-4",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-5",
+        "name": "Kaolack",
+        "population": {
+          "-year": "85",
+          "#text": "132000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-6",
+      "-capital": "cty-cid-cia-Senegal-Kolda",
+      "-country": "SN",
+      "name": "Kolda",
+      "area": "21011",
+      "population": "606800",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-Kolda",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-6",
+        "name": "Kolda"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-7",
+      "-capital": "cty-cid-cia-Senegal-Louga",
+      "-country": "SN",
+      "name": "Louga",
+      "area": "29188",
+      "population": "490400",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-Louga",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-7",
+        "name": "Louga"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-8",
+      "-capital": "cty-cid-cia-Senegal-5",
+      "-country": "SN",
+      "name": "Saint Louis",
+      "area": "44127",
+      "population": "680200",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-5",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-8",
+        "name": "Saint Louis",
+        "population": {
+          "-year": "85",
+          "#text": "122000"
+        },
+        "located_at": {
+          "-watertype": "river",
+          "-river": "river-Senegal"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-9",
+      "-capital": "cty-cid-cia-Senegal-Tambacounda",
+      "-country": "SN",
+      "name": "Tambacounda",
+      "area": "59602",
+      "population": "370000",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-Tambacounda",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-9",
+        "name": "Tambacounda"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-10",
+      "-capital": "cty-cid-cia-Senegal-3",
+      "-country": "SN",
+      "name": "Thies",
+      "area": "6601",
+      "population": "948100",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-3",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-10",
+        "name": "Thies",
+        "population": {
+          "-year": "85",
+          "#text": "156000"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Senegal-11",
+      "-capital": "cty-cid-cia-Senegal-6",
+      "-country": "SN",
+      "name": "Ziguinchor",
+      "area": "7339",
+      "population": "394700",
+      "city": {
+        "-id": "cty-cid-cia-Senegal-6",
+        "-is_state_cap": "yes",
+        "-country": "SN",
+        "-province": "prov-cid-cia-Senegal-11",
+        "name": "Ziguinchor",
+        "population": {
+          "-year": "85",
+          "#text": "107000"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "GNB",
+  "-area": "36120",
+  "-capital": "cty-cid-cia-Guinea-Bissau-Bissau",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-ECA org-ECOWAS org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOM org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNAVEM-III org-UNAMIR org-UNESCO org-UNIDO org-UNOMIL org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Guinea-Bissau",
+  "population": "1151330",
+  "population_growth": "2.35",
+  "infant_mortality": "115.8",
+  "gdp_total": "1000",
+  "gdp_agri": "44",
+  "gdp_ind": "8",
+  "gdp_serv": "48",
+  "inflation": "15",
+  "indep_date": "1974-09-10",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "European"
+    },
+    {
+      "-percentage": "99",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "30",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RG",
+      "-length": "386"
+    },
+    {
+      "-country": "SN",
+      "-length": "338"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Guinea-Bissau-Bissau",
+    "-is_country_cap": "yes",
+    "-country": "GNB",
+    "name": "Bissau",
+    "longitude": "-15.5",
+    "latitude": "11.5",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "WAL",
+  "-area": "71740",
+  "-capital": "cty-cid-cia-Sierra-Leone-Freetown",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-ECOWAS org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Sierra Leone",
+  "population": "4793121",
+  "population_growth": "4.14",
+  "infant_mortality": "135.6",
+  "gdp_total": "4400",
+  "gdp_agri": "40",
+  "gdp_ind": "17",
+  "gdp_serv": "43",
+  "inflation": "24",
+  "indep_date": "1961-04-27",
+  "government": "constitutional democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "1",
+      "#text": "Asian"
+    },
+    {
+      "-percentage": "99",
+      "#text": "African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "60",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "10",
+      "#text": "Christian"
+    }
+  ],
+  "border": [
+    {
+      "-country": "RG",
+      "-length": "652"
+    },
+    {
+      "-country": "LB",
+      "-length": "306"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Sierra-Leone-Freetown",
+    "-is_country_cap": "yes",
+    "-country": "WAL",
+    "name": "Freetown",
+    "longitude": "-13.1",
+    "latitude": "8.2",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    }
+  }
+}
+{
+  "-car_code": "EAU",
+  "-area": "236040",
+  "-capital": "cty-cid-cia-Uganda-Kampala",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-EADB org-ECA org-FAO org-G-77 org-IGADD org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-IOM org-ISO org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-PCA org-UN org-UNESCO org-UNIDO org-UNITAR org-UNHCR org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Uganda",
+  "population": "20158176",
+  "population_growth": "2.24",
+  "infant_mortality": "99.4",
+  "gdp_total": "16800",
+  "gdp_agri": "55",
+  "gdp_ind": "12",
+  "gdp_serv": "33",
+  "inflation": "6.1",
+  "indep_date": "1962-10-09",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "6",
+      "#text": "Rwanda"
+    },
+    {
+      "-percentage": "1",
+      "#text": "Arab"
+    },
+    {
+      "-percentage": "17",
+      "#text": "Baganda"
+    },
+    {
+      "-percentage": "12",
+      "#text": "Karamojong"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Basogo"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Iteso"
+    },
+    {
+      "-percentage": "6",
+      "#text": "Langi"
+    },
+    {
+      "-percentage": "5",
+      "#text": "Bagisu"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Acholi"
+    },
+    {
+      "-percentage": "4",
+      "#text": "Lugbara"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Bunyoro"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Batobo"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "16",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "33",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "33",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "ZRE",
+      "-length": "765"
+    },
+    {
+      "-country": "RWA",
+      "-length": "169"
+    },
+    {
+      "-country": "EAT",
+      "-length": "396"
+    },
+    {
+      "-country": "SUD",
+      "-length": "435"
+    },
+    {
+      "-country": "EAK",
+      "-length": "933"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Uganda-Kampala",
+    "-is_country_cap": "yes",
+    "-country": "EAU",
+    "name": "Kampala",
+    "longitude": "32.3",
+    "latitude": "0.2",
+    "located_at": {
+      "-watertype": "lake",
+      "-lake": "lake-Victoriasee"
+    }
+  }
+}
+{
+  "-car_code": "LS",
+  "-area": "30350",
+  "-capital": "cty-cid-cia-Lesotho-Maseru",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SACU org-SADC org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Lesotho",
+  "population": "1970781",
+  "population_growth": "1.9",
+  "infant_mortality": "81.6",
+  "gdp_total": "2800",
+  "gdp_agri": "10.4",
+  "gdp_ind": "48.8",
+  "gdp_serv": "40.8",
+  "inflation": "9.5",
+  "indep_date": "1966-10-04",
+  "government": "modified constitutional monarchy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "99.7",
+    "#text": "Sotho"
+  },
+  "religions": {
+    "-percentage": "80",
+    "#text": "Christian"
+  },
+  "border": {
+    "-country": "RSA",
+    "-length": "909"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Lesotho-Maseru",
+    "-is_country_cap": "yes",
+    "-country": "LS",
+    "name": "Maseru",
+    "longitude": "27.5",
+    "latitude": "-29.3",
+    "population": {
+      "-year": "87",
+      "#text": "75000"
+    }
+  }
+}
+{
+  "-car_code": "RM",
+  "-area": "587041",
+  "-capital": "cty-cid-cia-Madagascar-2",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICC org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UNHCR org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Madagascar",
+  "population": "13670507",
+  "population_growth": "2.83",
+  "infant_mortality": "93.5",
+  "gdp_total": "11400",
+  "gdp_agri": "35",
+  "gdp_ind": "15",
+  "gdp_serv": "50",
+  "inflation": "35",
+  "indep_date": "1960-06-26",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "7",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "41",
+      "#text": "Christian"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Madagascar-2",
+      "-capital": "cty-cid-cia-Madagascar-5",
+      "-country": "RM",
+      "name": "Antsiranana",
+      "area": "43046",
+      "population": "689800",
+      "city": {
+        "-id": "cty-cid-cia-Madagascar-5",
+        "-is_state_cap": "yes",
+        "-country": "RM",
+        "-province": "prov-cid-cia-Madagascar-2",
+        "name": "Antsiranana",
+        "population": {
+          "-year": "88",
+          "#text": "220000"
+        },
+        "located_on": { "-island": "island-Madagaskar" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Madagascar-3",
+      "-capital": "cty-cid-cia-Madagascar-2",
+      "-country": "RM",
+      "name": "Antananarivo",
+      "area": "58283",
+      "population": "3195800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Madagascar-2",
+          "-is_country_cap": "yes",
+          "-is_state_cap": "yes",
+          "-country": "RM",
+          "-province": "prov-cid-cia-Madagascar-3",
+          "name": "Antananarivo",
+          "longitude": "47.4",
+          "latitude": "-18.6",
+          "population": {
+            "-year": "88",
+            "#text": "1250000"
+          },
+          "located_on": { "-island": "island-Madagaskar" }
+        },
+        {
+          "-id": "cty-cid-cia-Madagascar-8",
+          "-country": "RM",
+          "-province": "prov-cid-cia-Madagascar-3",
+          "name": "Antsirabe",
+          "population": {
+            "-year": "88",
+            "#text": "100000"
+          },
+          "located_on": { "-island": "island-Madagaskar" }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Madagascar-4",
+      "-capital": "cty-cid-cia-Madagascar-3",
+      "-country": "RM",
+      "name": "Fianarantsoa",
+      "area": "102373",
+      "population": "2209700",
+      "city": {
+        "-id": "cty-cid-cia-Madagascar-3",
+        "-is_state_cap": "yes",
+        "-country": "RM",
+        "-province": "prov-cid-cia-Madagascar-4",
+        "name": "Fianarantsoa",
+        "population": {
+          "-year": "88",
+          "#text": "300000"
+        },
+        "located_on": { "-island": "island-Madagaskar" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Madagascar-5",
+      "-capital": "cty-cid-cia-Madagascar-6",
+      "-country": "RM",
+      "name": "Mahajanga",
+      "area": "150023",
+      "population": "1075300",
+      "city": {
+        "-id": "cty-cid-cia-Madagascar-6",
+        "-is_state_cap": "yes",
+        "-country": "RM",
+        "-province": "prov-cid-cia-Madagascar-5",
+        "name": "Majunga",
+        "population": {
+          "-year": "88",
+          "#text": "200000"
+        },
+        "located_on": { "-island": "island-Madagaskar" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Madagascar-6",
+      "-capital": "cty-cid-cia-Madagascar-4",
+      "-country": "RM",
+      "name": "Toamasina",
+      "area": "71911",
+      "population": "1444700",
+      "city": {
+        "-id": "cty-cid-cia-Madagascar-4",
+        "-is_state_cap": "yes",
+        "-country": "RM",
+        "-province": "prov-cid-cia-Madagascar-6",
+        "name": "Tamatave",
+        "population": {
+          "-year": "88",
+          "#text": "230000"
+        },
+        "located_on": { "-island": "island-Madagaskar" }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Madagascar-7",
+      "-capital": "cty-cid-cia-Madagascar-7",
+      "-country": "RM",
+      "name": "Toliara",
+      "area": "161405",
+      "population": "1396700",
+      "city": {
+        "-id": "cty-cid-cia-Madagascar-7",
+        "-is_state_cap": "yes",
+        "-country": "RM",
+        "-province": "prov-cid-cia-Madagascar-7",
+        "name": "Toliara",
+        "population": {
+          "-year": "88",
+          "#text": "150000"
+        },
+        "located_on": { "-island": "island-Madagaskar" }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MW",
+  "-area": "118480",
+  "-capital": "cty-cid-cia-Malawi-Lilongwe",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-SADC org-UN org-UNAMIR org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Malawi",
+  "population": "9452844",
+  "population_growth": "1.71",
+  "infant_mortality": "139.9",
+  "gdp_total": "6900",
+  "gdp_agri": "31",
+  "gdp_ind": "14",
+  "gdp_serv": "55",
+  "inflation": "83.3",
+  "indep_date": "1964-07-06",
+  "government": "multiparty democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": [
+    {
+      "-percentage": "20",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "20",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "55",
+      "#text": "Protestant"
+    }
+  ],
+  "border": [
+    {
+      "-country": "Z",
+      "-length": "837"
+    },
+    {
+      "-country": "EAT",
+      "-length": "475"
+    },
+    {
+      "-country": "MOC",
+      "-length": "1569"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Malawi-Lilongwe",
+    "-is_country_cap": "yes",
+    "-country": "MW",
+    "name": "Lilongwe",
+    "longitude": "34",
+    "latitude": "-14"
+  }
+}
+{
+  "-car_code": "MOC",
+  "-area": "801590",
+  "-capital": "cty-cid-cia-Mozambique-2",
+  "-memberships": "org-ACP org-AfDB org-CCC org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-IDB org-NAM org-OAU org-OIC org-SADC org-UN org-UNESCO org-UNIDO org-UPU org-WFTU org-WHO org-WMO org-WTrO",
+  "name": "Mozambique",
+  "population": "17877927",
+  "population_growth": "2.65",
+  "infant_mortality": "125.6",
+  "gdp_total": "12200",
+  "gdp_agri": "33",
+  "gdp_ind": "12",
+  "gdp_serv": "55",
+  "inflation": "50",
+  "indep_date": "1975-06-25",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "0.06",
+      "#text": "European"
+    },
+    {
+      "-percentage": "0.08",
+      "#text": "Indian"
+    },
+    {
+      "-percentage": "0.2",
+      "#text": "Euro-African"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "20",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "30",
+      "#text": "Christian"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "Portuguese"
+  },
+  "border": [
+    {
+      "-country": "Z",
+      "-length": "419"
+    },
+    {
+      "-country": "RSA",
+      "-length": "491"
+    },
+    {
+      "-country": "ZW",
+      "-length": "1231"
+    },
+    {
+      "-country": "EAT",
+      "-length": "756"
+    },
+    {
+      "-country": "MW",
+      "-length": "1569"
+    },
+    {
+      "-country": "SD",
+      "-length": "105"
+    }
+  ],
+  "province": [
+    {
+      "-id": "prov-cid-cia-Mozambique-2",
+      "-capital": "cty-cid-cia-Mozambique-Pemba",
+      "-country": "MOC",
+      "name": "Cabo Delgado",
+      "area": "82625",
+      "population": "1202200",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-Pemba",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-2",
+        "name": "Pemba"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-3",
+      "-capital": "cty-cid-cia-Mozambique-Xai-Xai",
+      "-country": "MOC",
+      "name": "Gaza",
+      "area": "75709",
+      "population": "1401500",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-Xai-Xai",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-3",
+        "name": "Xai Xai",
+        "longitude": "34",
+        "latitude": "-25.2",
+        "located_at": [
+          {
+            "-watertype": "sea",
+            "-sea": "sea-Indic"
+          },
+          {
+            "-watertype": "river",
+            "-river": "river-Limpopo"
+          }
+        ]
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-4",
+      "-capital": "cty-cid-cia-Mozambique-Inhambane",
+      "-country": "MOC",
+      "name": "Inhambane",
+      "area": "68615",
+      "population": "1157000",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-Inhambane",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-4",
+        "name": "Inhambane"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-5",
+      "-capital": "cty-cid-cia-Mozambique-9",
+      "-country": "MOC",
+      "name": "Manica",
+      "area": "61661",
+      "population": "609500",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-9",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-5",
+        "name": "Chimoio",
+        "population": {
+          "-year": "91",
+          "#text": "108818"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-6",
+      "-capital": "cty-cid-cia-Mozambique-2",
+      "-country": "MOC",
+      "name": "Maputo",
+      "area": "25756",
+      "population": "840800",
+      "city": [
+        {
+          "-id": "cty-cid-cia-Mozambique-7",
+          "-country": "MOC",
+          "-province": "prov-cid-cia-Mozambique-6",
+          "name": "Nacala",
+          "population": {
+            "-year": "91",
+            "#text": "125208"
+          }
+        },
+        {
+          "-id": "cty-cid-cia-Mozambique-3",
+          "-country": "MOC",
+          "-province": "prov-cid-cia-Mozambique-6",
+          "name": "Matala",
+          "population": {
+            "-year": "91",
+            "#text": "337239"
+          }
+        }
+      ]
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-7",
+      "-capital": "cty-cid-cia-Mozambique-5",
+      "-country": "MOC",
+      "name": "Nampula",
+      "area": "81606",
+      "population": "2841400",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-5",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-7",
+        "name": "Nampula",
+        "population": {
+          "-year": "91",
+          "#text": "250473"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-8",
+      "-capital": "cty-cid-cia-Mozambique-Lichinga",
+      "-country": "MOC",
+      "name": "Niassa",
+      "area": "129055",
+      "population": "686700",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-Lichinga",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-8",
+        "name": "Lichinga"
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-9",
+      "-capital": "cty-cid-cia-Mozambique-4",
+      "-country": "MOC",
+      "name": "Sofala",
+      "area": "68018",
+      "population": "1427500",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-4",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-9",
+        "name": "Beira",
+        "population": {
+          "-year": "91",
+          "#text": "298847"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-10",
+      "-capital": "cty-cid-cia-Mozambique-8",
+      "-country": "MOC",
+      "name": "Tete",
+      "area": "100724",
+      "population": "734600",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-8",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-10",
+        "name": "Tete",
+        "population": {
+          "-year": "91",
+          "#text": "112221"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-11",
+      "-capital": "cty-cid-cia-Mozambique-6",
+      "-country": "MOC",
+      "name": "Zambezia",
+      "area": "105008",
+      "population": "2619300",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-6",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-11",
+        "name": "Quelimane",
+        "population": {
+          "-year": "91",
+          "#text": "146206"
+        }
+      }
+    },
+    {
+      "-id": "prov-cid-cia-Mozambique-13",
+      "-capital": "cty-cid-cia-Mozambique-2",
+      "-country": "MOC",
+      "name": "Maputo (munic.)",
+      "area": "602",
+      "population": "931600",
+      "city": {
+        "-id": "cty-cid-cia-Mozambique-2",
+        "-is_country_cap": "yes",
+        "-is_state_cap": "yes",
+        "-country": "MOC",
+        "-province": "prov-cid-cia-Mozambique-13",
+        "name": "Maputo",
+        "longitude": "33",
+        "latitude": "-26",
+        "population": {
+          "-year": "91",
+          "#text": "931591"
+        },
+        "located_at": {
+          "-watertype": "sea",
+          "-sea": "sea-Indic"
+        }
+      }
+    }
+  ]
+}
+{
+  "-car_code": "MS",
+  "-area": "2040",
+  "-capital": "cty-cid-cia-Mauritius-Port-Louis",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-C org-CCC org-ECA org-FAO org-G-77 org-IAEA org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-Inmarsat org-IMF org-IOC org-ISO org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-PCA org-SADC org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WFTU org-WHO org-WIPO org-WMO org-WToO org-WTrO",
+  "name": "Mauritius",
+  "population": "1140256",
+  "population_growth": "1.23",
+  "infant_mortality": "17.2",
+  "gdp_total": "10900",
+  "inflation": "9.4",
+  "indep_date": "1968-03-12",
+  "government": "parliamentary democracy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "27",
+      "#text": "Creole"
+    },
+    {
+      "-percentage": "68",
+      "#text": "Indo-Mauritian"
+    },
+    {
+      "-percentage": "3",
+      "#text": "Sino-Mauritian"
+    },
+    {
+      "-percentage": "2",
+      "#text": "Franco-Mauritian"
+    }
+  ],
+  "religions": [
+    {
+      "-percentage": "16.6",
+      "#text": "Muslim"
+    },
+    {
+      "-percentage": "28.3",
+      "#text": "Christian"
+    },
+    {
+      "-percentage": "52",
+      "#text": "Hindu"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Mauritius-Port-Louis",
+    "-is_country_cap": "yes",
+    "-country": "MS",
+    "name": "Port Louis",
+    "longitude": "57.3",
+    "latitude": "-20.3",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Mauritius" }
+  }
+}
+{
+  "-car_code": "MAYO",
+  "-area": "374",
+  "-capital": "city-Mamoutzou-MAYO-MAYO",
+  "-memberships": "org-UPU org-WFTU",
+  "name": "Mayotte",
+  "population": "223765",
+  "population_growth": "3.32",
+  "infant_mortality": "56.29",
+  "gdp_total": "953.6",
+  "dependent": { "-country": "F" },
+  "government": "territorial collectivity of France",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "97",
+    "#text": "Muslim"
+  },
+  "city": {
+    "-id": "city-Mamoutzou-MAYO-MAYO",
+    "-is_country_cap": "yes",
+    "-country": "MAYO",
+    "name": "Mamoutzou",
+    "longitude": "45.13",
+    "latitude": "-12.46",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Mayotte" }
+  }
+}
+{
+  "-car_code": "SD",
+  "-area": "17360",
+  "-capital": "cty-cid-cia-Swaziland-Mbabane",
+  "-memberships": "org-ACP org-AfDB org-C org-CCC org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IDA org-IFRCS org-IFC org-IFAD org-ILO org-IMF org-IOC org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-PCA org-SACU org-SADC org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WIPO org-WMO org-WTrO",
+  "name": "Swaziland",
+  "population": "998730",
+  "population_growth": "3.24",
+  "infant_mortality": "88.4",
+  "gdp_total": "3600",
+  "gdp_agri": "25",
+  "gdp_ind": "40",
+  "gdp_serv": "35",
+  "inflation": "14.7",
+  "indep_date": "1968-09-06",
+  "government": "monarchy",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "3",
+      "#text": "European"
+    },
+    {
+      "-percentage": "97",
+      "#text": "African"
+    }
+  ],
+  "religions": {
+    "-percentage": "60",
+    "#text": "Christian"
+  },
+  "border": [
+    {
+      "-country": "RSA",
+      "-length": "430"
+    },
+    {
+      "-country": "MOC",
+      "-length": "105"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Swaziland-Mbabane",
+    "-is_country_cap": "yes",
+    "-country": "SD",
+    "name": "Mbabane",
+    "longitude": "31.1",
+    "latitude": "-25"
+  }
+}
+{
+  "-car_code": "REUN",
+  "-area": "2510",
+  "-capital": "city-Saint-Denis-REUN-REUN",
+  "-memberships": "org-IOC org-WFTU",
+  "name": "Reunion",
+  "population": "679198",
+  "population_growth": "1.93",
+  "infant_mortality": "7.5",
+  "gdp_total": "2900",
+  "dependent": { "-country": "F" },
+  "government": "overseas department of France",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "religions": {
+    "-percentage": "94",
+    "#text": "Roman Catholic"
+  },
+  "city": {
+    "-id": "city-Saint-Denis-REUN-REUN",
+    "-is_country_cap": "yes",
+    "-country": "REUN",
+    "name": "Saint-Denis",
+    "longitude": "55.5",
+    "latitude": "-20.9",
+    "population": {
+      "-year": "09",
+      "#text": "158139"
+    },
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Reunion" }
+  }
+}
+{
+  "-car_code": "HELX",
+  "-area": "413",
+  "-capital": "city-Jamestown-HELX-HELX",
+  "-memberships": "org-ICFTU org-UPU org-WFTU",
+  "name": "Saint Helena",
+  "population": "7637",
+  "population_growth": "0.45",
+  "infant_mortality": "17.63",
+  "gdp_total": "18",
+  "dependent": { "-country": "GB" },
+  "government": "British Overseas Territories",
+  "encompassed": {
+    "-continent": "australia",
+    "-percentage": "100"
+  },
+  "ethnicgroups": [
+    {
+      "-percentage": "50",
+      "#text": "African descent"
+    },
+    {
+      "-percentage": "25",
+      "#text": "White"
+    },
+    {
+      "-percentage": "25",
+      "#text": "Chinese"
+    }
+  ],
+  "languages": {
+    "-percentage": "100",
+    "#text": "English"
+  },
+  "city": {
+    "-id": "city-Jamestown-HELX-HELX",
+    "-is_country_cap": "yes",
+    "-country": "HELX",
+    "name": "Jamestown",
+    "longitude": "-5.44",
+    "latitude": "-15.56",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-SaintHelena" }
+  }
+}
+{
+  "-car_code": "STP",
+  "-area": "1001",
+  "-capital": "cty-cid-cia-Sao-Tome-and-Principe-Sao-Tome",
+  "-memberships": "org-ACP org-AfDB org-ECA org-CEEAC org-FAO org-G-77 org-IBRD org-ICAO org-Interpol org-IDA org-IFRCS org-IFAD org-ILO org-IMO org-IMF org-IOC org-IOM org-ICRM org-ITU org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Sao Tome and Principe",
+  "population": "144128",
+  "population_growth": "2.58",
+  "infant_mortality": "61.1",
+  "gdp_total": "138",
+  "gdp_agri": "28",
+  "gdp_ind": "14",
+  "gdp_serv": "58",
+  "inflation": "38",
+  "indep_date": "1975-07-12",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Europeans"
+  },
+  "languages": {
+    "-percentage": "100",
+    "#text": "Portuguese"
+  },
+  "city": {
+    "-id": "cty-cid-cia-Sao-Tome-and-Principe-Sao-Tome",
+    "-is_country_cap": "yes",
+    "-country": "STP",
+    "name": "Sao Tome",
+    "longitude": "7",
+    "latitude": "0",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Atlantic"
+    },
+    "located_on": { "-island": "island-Sao_Tome" }
+  }
+}
+{
+  "-car_code": "SY",
+  "-area": "455",
+  "-capital": "cty-cid-cia-Seychelles-Victoria",
+  "-memberships": "org-ACP org-AfDB org-ACCT org-C org-ECA org-FAO org-G-77 org-IBRD org-ICAO org-ICFTU org-Interpol org-IFRCS org-IFC org-IFAD org-ILO org-IMO org-IMF org-IOC org-ICRM org-Intelsat org-NAM org-OAU org-UN org-UNESCO org-UNIDO org-UPU org-WCL org-WHO org-WMO org-WToO org-WTrO",
+  "name": "Seychelles",
+  "population": "77575",
+  "population_growth": "0.76",
+  "infant_mortality": "12.5",
+  "gdp_total": "430",
+  "inflation": "3.9",
+  "indep_date": "1976-06-29",
+  "government": "republic",
+  "encompassed": {
+    "-continent": "africa",
+    "-percentage": "100"
+  },
+  "ethnicgroups": {
+    "-percentage": "100",
+    "#text": "Seychellois"
+  },
+  "religions": [
+    {
+      "-percentage": "90",
+      "#text": "Roman Catholic"
+    },
+    {
+      "-percentage": "8",
+      "#text": "Anglican"
+    }
+  ],
+  "city": {
+    "-id": "cty-cid-cia-Seychelles-Victoria",
+    "-is_country_cap": "yes",
+    "-country": "SY",
+    "name": "Victoria",
+    "longitude": "55.5",
+    "latitude": "-4.65",
+    "population": "24570",
+    "located_at": {
+      "-watertype": "sea",
+      "-sea": "sea-Indic"
+    },
+    "located_on": { "-island": "island-Mahe" }
+  }
+}
diff --git a/asterixdb/asterix-app/data/country/mountain.adm b/asterixdb/asterix-app/data/country/mountain.adm
new file mode 100644
index 0000000..018430b
--- /dev/null
+++ b/asterixdb/asterix-app/data/country/mountain.adm
@@ -0,0 +1,2990 @@
+{
+  "-id": "mount-Hvannadalshnukur",
+  "-country": "IS",
+  "-type": "volcanic",
+  "-island": "island-Iceland",
+  "name": "Hvannadalshnukur",
+  "height": "2110",
+  "longitude": "-16.7",
+  "latitude": "64.1"
+}
+{
+  "-id": "mount-Snaefell",
+  "-country": "IS",
+  "-type": "volcanic",
+  "-island": "island-Iceland",
+  "name": "Snaefell",
+  "height": "1833",
+  "longitude": "-15.2",
+  "latitude": "64.5"
+}
+{
+  "-id": "mount-Hekla",
+  "-country": "IS",
+  "-type": "volcano",
+  "-island": "island-Iceland",
+  "name": "Hekla",
+  "height": "1491",
+  "longitude": "-19.7",
+  "latitude": "64"
+}
+{
+  "-id": "mount-Katla",
+  "-country": "IS",
+  "-type": "volcano",
+  "-island": "island-Iceland",
+  "name": "Katla",
+  "height": "1450",
+  "longitude": "-19.0",
+  "latitude": "63.6"
+}
+{
+  "-id": "mount-Haltiatunturi",
+  "-country": "SF",
+  "name": "Haltiatunturi",
+  "located": {
+    "-country": "SF",
+    "-province": "lteil-LAP-SF"
+  },
+  "height": "1328",
+  "longitude": "21.4",
+  "latitude": "67"
+}
+{
+  "-id": "mount-Kebnekaise",
+  "-country": "S",
+  "name": "Kebnekaise",
+  "located": {
+    "-country": "S",
+    "-province": "prov-cid-cia-Sweden-15"
+  },
+  "height": "2114",
+  "longitude": "18.3",
+  "latitude": "67.5"
+}
+{
+  "-id": "mount-Sarektjokko",
+  "-country": "S",
+  "name": "Sarektjokko",
+  "located": {
+    "-country": "S",
+    "-province": "prov-cid-cia-Sweden-15"
+  },
+  "height": "2090",
+  "longitude": "17.4",
+  "latitude": "67.3"
+}
+{
+  "-id": "mount-Portefjaellen",
+  "-country": "S",
+  "name": "Portefjaellen",
+  "located": {
+    "-country": "S",
+    "-province": "prov-cid-cia-Sweden-15"
+  },
+  "height": "2021",
+  "longitude": "17.4",
+  "latitude": "67.1"
+}
+{
+  "-id": "mount-Tarrekaise",
+  "-country": "S",
+  "name": "Tarrekaise",
+  "located": {
+    "-country": "S",
+    "-province": "prov-cid-cia-Sweden-15"
+  },
+  "height": "1850",
+  "longitude": "17.3",
+  "latitude": "67"
+}
+{
+  "-id": "mount-Sulitjelma",
+  "-country": "N S",
+  "name": "Sulitjelma",
+  "located": [
+    {
+      "-country": "N",
+      "-province": "lteil-NO-N"
+    },
+    {
+      "-country": "S",
+      "-province": "prov-cid-cia-Sweden-15"
+    }
+  ],
+  "height": "1914",
+  "longitude": "16.2",
+  "latitude": "67.1"
+}
+{
+  "-id": "mount-Galdhoeppig",
+  "-country": "N",
+  "name": "Galdhoeppig",
+  "located": {
+    "-country": "N",
+    "-province": "lteil-OP-N"
+  },
+  "height": "2469",
+  "longitude": "8",
+  "latitude": "61.4"
+}
+{
+  "-id": "mount-Jostedalsbre",
+  "-country": "N",
+  "name": "Jostedalsbre",
+  "located": {
+    "-country": "N",
+    "-province": "lteil-SF-N"
+  },
+  "height": "2083",
+  "longitude": "5.5",
+  "latitude": "61.4"
+}
+{
+  "-id": "mount-Glittertind",
+  "-country": "N",
+  "name": "Glittertind",
+  "located": {
+    "-country": "N",
+    "-province": "lteil-OP-N"
+  },
+  "height": "2452",
+  "longitude": "8.1",
+  "latitude": "61.4"
+}
+{
+  "-id": "mount-Joekul",
+  "-country": "N",
+  "name": "Joekul",
+  "located": {
+    "-country": "N",
+    "-province": "lteil-HO-N"
+  },
+  "height": "1876",
+  "longitude": "7.2",
+  "latitude": "60.3"
+}
+{
+  "-id": "mount-Snoehetta",
+  "-country": "N",
+  "name": "Snoehetta",
+  "located": {
+    "-country": "N",
+    "-province": "lteil-OP-N"
+  },
+  "height": "2286",
+  "longitude": "9",
+  "latitude": "62.1"
+}
+{
+  "-id": "mount-BenNevis",
+  "-country": "GB",
+  "-island": "island-GreatBritain",
+  "name": "Ben Nevis",
+  "mountains": "Grampians",
+  "located": {
+    "-country": "GB",
+    "-province": "prov-cid-cia-United-Kingdom-55"
+  },
+  "height": "1344",
+  "longitude": "-5.0",
+  "latitude": "56.8"
+}
+{
+  "-id": "mount-Feldberg",
+  "-country": "D",
+  "name": "Feldberg",
+  "mountains": "Schwarzwald",
+  "located": {
+    "-country": "D",
+    "-province": "prov-cid-cia-Germany-2"
+  },
+  "height": "1493",
+  "longitude": "7.5",
+  "latitude": "47.5"
+}
+{
+  "-id": "mount-Brocken",
+  "-country": "D",
+  "name": "Brocken",
+  "mountains": "Harz",
+  "located": {
+    "-country": "D",
+    "-province": "prov-cid-cia-Germany-15"
+  },
+  "height": "1141",
+  "longitude": "10.6",
+  "latitude": "51.8"
+}
+{
+  "-id": "mount-Grosser_Arber",
+  "-country": "D",
+  "name": "Grosser Arber",
+  "mountains": "Bayrischer Wald",
+  "located": {
+    "-country": "D",
+    "-province": "prov-cid-cia-Germany-3"
+  },
+  "height": "1456",
+  "longitude": "13",
+  "latitude": "49.1"
+}
+{
+  "-id": "mount-Zugspitze",
+  "-country": "D A",
+  "name": "Zugspitze",
+  "mountains": "Alps",
+  "located": [
+    {
+      "-country": "D",
+      "-province": "prov-cid-cia-Germany-3"
+    },
+    {
+      "-country": "A",
+      "-province": "prov-cid-cia-Austria-7"
+    }
+  ],
+  "height": "2963",
+  "longitude": "11.1",
+  "latitude": "47.3"
+}
+{
+  "-id": "mount-MontBlanc",
+  "-country": "F I",
+  "name": "Mont Blanc",
+  "mountains": "Alps",
+  "located": [
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-111"
+    },
+    {
+      "-country": "I",
+      "-province": "prov-cid-cia-Italy-9"
+    }
+  ],
+  "height": "4808",
+  "longitude": "6.8",
+  "latitude": "45.8"
+}
+{
+  "-id": "mount-Ecrins",
+  "-country": "F",
+  "name": "Barre des Ecrins",
+  "mountains": "Alps",
+  "located": {
+    "-country": "F",
+    "-province": "prov-cid-cia-France-111"
+  },
+  "height": "4101",
+  "longitude": "6.3",
+  "latitude": "44.9"
+}
+{
+  "-id": "mount-GrandCombin",
+  "-country": "CH",
+  "name": "GrandCombin",
+  "mountains": "Alps",
+  "located": {
+    "-country": "CH",
+    "-province": "prov-cid-cia-Switzerland-24"
+  },
+  "height": "4314",
+  "longitude": "7.3",
+  "latitude": "45.9"
+}
+{
+  "-id": "mount-Matterhorn",
+  "-country": "CH I",
+  "name": "Matterhorn",
+  "mountains": "Alps",
+  "located": [
+    {
+      "-country": "CH",
+      "-province": "prov-cid-cia-Switzerland-24"
+    },
+    {
+      "-country": "I",
+      "-province": "prov-cid-cia-Italy-9"
+    }
+  ],
+  "height": "4478",
+  "longitude": "7.56",
+  "latitude": "46"
+}
+{
+  "-id": "mount-MonteRosa",
+  "-country": "CH I",
+  "name": "Monte Rosa",
+  "mountains": "Alps",
+  "located": [
+    {
+      "-country": "CH",
+      "-province": "prov-cid-cia-Switzerland-24"
+    },
+    {
+      "-country": "I",
+      "-province": "prov-cid-cia-Italy-9 prov-cid-cia-Italy-10"
+    }
+  ],
+  "height": "4634",
+  "longitude": "7.9",
+  "latitude": "45.9"
+}
+{
+  "-id": "mount-Finsteraarhorn",
+  "-country": "CH",
+  "name": "Finsteraarhorn",
+  "mountains": "Alps",
+  "located": {
+    "-country": "CH",
+    "-province": "prov-cid-cia-Switzerland-5 prov-cid-cia-Switzerland-24"
+  },
+  "height": "4274",
+  "longitude": "8.1",
+  "latitude": "46.5"
+}
+{
+  "-id": "mount-GranParadiso",
+  "-country": "I",
+  "name": "GranParadiso",
+  "mountains": "Alps",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-9 prov-cid-cia-Italy-2"
+  },
+  "height": "4061",
+  "longitude": "7.25",
+  "latitude": "45.5"
+}
+{
+  "-id": "mount-PizBernina",
+  "-country": "CH I",
+  "name": "Piz Bernina",
+  "mountains": "Alps",
+  "located": [
+    {
+      "-country": "CH",
+      "-province": "prov-cid-cia-Switzerland-11"
+    },
+    {
+      "-country": "I",
+      "-province": "prov-cid-cia-Italy-10"
+    }
+  ],
+  "height": "4048",
+  "longitude": "9.9",
+  "latitude": "46.4"
+}
+{
+  "-id": "mount-Crap_Sogn_Gion",
+  "-country": "CH",
+  "name": "Crap_Sogn_Gion",
+  "mountains": "Alps",
+  "located": {
+    "-country": "CH",
+    "-province": "prov-cid-cia-Switzerland-11"
+  },
+  "height": "2228",
+  "longitude": "9.2",
+  "latitude": "46.85"
+}
+{
+  "-id": "mount-Marmolata",
+  "-country": "I",
+  "name": "Marmolata",
+  "mountains": "Alps",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-20 prov-cid-cia-Italy-23"
+  },
+  "height": "3343",
+  "longitude": "11.85",
+  "latitude": "46.4"
+}
+{
+  "-id": "mount-Grossglockner",
+  "-country": "A",
+  "name": "Grossglockner",
+  "mountains": "Alps",
+  "located": {
+    "-country": "A",
+    "-province": "prov-cid-cia-Austria-3"
+  },
+  "height": "3797",
+  "longitude": "12.5",
+  "latitude": "47"
+}
+{
+  "-id": "mount-Hochgolling",
+  "-country": "A",
+  "name": "Hochgolling",
+  "mountains": "Alps",
+  "located": {
+    "-country": "A",
+    "-province": "prov-cid-cia-Austria-8 prov-cid-cia-Austria-9"
+  },
+  "height": "2862",
+  "longitude": "13.75",
+  "latitude": "47.3"
+}
+{
+  "-id": "mount-GrandBallon",
+  "-country": "F",
+  "name": "Grand Ballon",
+  "mountains": "Vosges",
+  "located": {
+    "-country": "F",
+    "-province": "prov-cid-cia-France-2 prov-cid-cia-France-72"
+  },
+  "height": "1424",
+  "longitude": "7.1",
+  "latitude": "47.9"
+}
+{
+  "-id": "mount-PuyDeDome",
+  "-country": "F",
+  "-type": "volcanic",
+  "name": "Puy De Dome",
+  "mountains": "Cevennes",
+  "located": {
+    "-country": "F",
+    "-province": "prov-cid-cia-France-11"
+  },
+  "height": "1465",
+  "longitude": "3",
+  "latitude": "45.8"
+}
+{
+  "-id": "mount-PuydeSancy",
+  "-country": "F",
+  "-type": "volcanic",
+  "name": "Puy de Sancy",
+  "mountains": "Cevennes",
+  "located": {
+    "-country": "F",
+    "-province": "prov-cid-cia-France-11"
+  },
+  "height": "1885",
+  "longitude": "2.8",
+  "latitude": "45.5"
+}
+{
+  "-id": "mount-Vignemale",
+  "-country": "E F",
+  "name": "Vignemale",
+  "mountains": "Pyrenees",
+  "located": [
+    {
+      "-country": "E",
+      "-province": "prov-cid-cia-Spain-3"
+    },
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-77"
+    }
+  ],
+  "height": "3298",
+  "longitude": "0.15",
+  "latitude": "42.8"
+}
+{
+  "-id": "mount-Aneto",
+  "-country": "E",
+  "name": "Pico de Aneto",
+  "mountains": "Pyrenees",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-3"
+  },
+  "height": "3404",
+  "longitude": "0.65",
+  "latitude": "42.6"
+}
+{
+  "-id": "mount-TorredeCerredo",
+  "-country": "E",
+  "name": "Torre de Cerredo",
+  "mountains": "Cordillera Cantabrica",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-4 prov-cid-cia-Spain-9"
+  },
+  "height": "2648",
+  "longitude": "4.85",
+  "latitude": "43.2"
+}
+{
+  "-id": "mount-PicoAlmanzor",
+  "-country": "E",
+  "name": "Pico de Almanzor",
+  "mountains": "Cordillera Central",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-9"
+  },
+  "height": "2648",
+  "longitude": "-5.3",
+  "latitude": "40.25"
+}
+{
+  "-id": "mount-Moncayo",
+  "-country": "E",
+  "name": "Moncayo",
+  "mountains": "Cordillera Iberica",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-3 prov-cid-cia-Spain-9"
+  },
+  "height": "2313",
+  "longitude": "1.85",
+  "latitude": "41.8"
+}
+{
+  "-id": "mount-Mulhacen",
+  "-country": "E",
+  "name": "Mulhacen",
+  "mountains": "Cordillera Betica",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-2"
+  },
+  "height": "3482",
+  "longitude": "3.3",
+  "latitude": "37"
+}
+{
+  "-id": "mount-TorreEstrela",
+  "-country": "P",
+  "name": "Torre de Estrela",
+  "mountains": "Cordillera Central",
+  "located": {
+    "-country": "P",
+    "-province": "prov-cid-cia-Portugal-10"
+  },
+  "height": "1993",
+  "longitude": "-7.6",
+  "latitude": "40.3"
+}
+{
+  "-id": "mount-Cinto",
+  "-country": "F",
+  "-island": "island-Corse",
+  "name": "Monte Cinto",
+  "mountains": "Corse",
+  "located": {
+    "-country": "F",
+    "-province": "prov-cid-cia-France-42"
+  },
+  "height": "2706",
+  "longitude": "8.9",
+  "latitude": "42.4"
+}
+{
+  "-id": "mount-GranSasso",
+  "-country": "I",
+  "name": "Gran Sasso",
+  "mountains": "Apennin",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-74"
+  },
+  "height": "2912",
+  "longitude": "13.55",
+  "latitude": "42.5"
+}
+{
+  "-id": "mount-Falterona",
+  "-country": "I",
+  "name": "Monte Falterona",
+  "mountains": "Apennin",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-50 prov-cid-cia-Italy-41"
+  },
+  "height": "1654",
+  "longitude": "11.7",
+  "latitude": "43.9"
+}
+{
+  "-id": "mount-Vesuv",
+  "-country": "I",
+  "-type": "volcano",
+  "name": "Vesuv",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-82"
+  },
+  "height": "1281",
+  "longitude": "14.4",
+  "latitude": "40.8"
+}
+{
+  "-id": "mount-Etna",
+  "-country": "I",
+  "-type": "volcano",
+  "-island": "island-Sicilia",
+  "name": "Etna",
+  "located": {
+    "-country": "I",
+    "-province": "prov-cid-cia-Italy-101"
+  },
+  "height": "3323",
+  "longitude": "15.0",
+  "latitude": "37.7"
+}
+{
+  "-id": "mount-Schneekoppe",
+  "-country": "CZ PL",
+  "name": "Schneekoppe",
+  "mountains": "Sudety Mountains",
+  "located": [
+    {
+      "-country": "CZ",
+      "-province": "prov-cid-cia-Czech-Republic-7"
+    },
+    {
+      "-country": "PL",
+      "-province": "prov-cid-cia-Poland-46"
+    }
+  ],
+  "height": "1602",
+  "longitude": "15.7",
+  "latitude": "50.7"
+}
+{
+  "-id": "mount-GerlachovskyStit",
+  "-country": "SK",
+  "name": "GerlachovskyStit",
+  "mountains": "Beskides",
+  "height": "2655",
+  "longitude": "20.1",
+  "latitude": "49.2"
+}
+{
+  "-id": "mount-Moldoveanu",
+  "-country": "RO",
+  "name": "Moldoveanu",
+  "mountains": "Karpaten",
+  "located": {
+    "-country": "RO",
+    "-province": "prov-cid-cia-Romania-4 prov-cid-cia-Romania-10 prov-cid-cia-Romania-35"
+  },
+  "height": "2544",
+  "longitude": "24.7",
+  "latitude": "45.6"
+}
+{
+  "-id": "mount-Musala",
+  "-country": "BG",
+  "name": "Musala",
+  "mountains": "Balkan",
+  "height": "2925",
+  "longitude": "23.6",
+  "latitude": "42.2"
+}
+{
+  "-id": "mount-Jezerce",
+  "-country": "AL",
+  "name": "Jezerce",
+  "mountains": "Balkan",
+  "height": "2694",
+  "longitude": "19.4",
+  "latitude": "42.3"
+}
+{
+  "-id": "mount-Korab",
+  "-country": "AL MK",
+  "name": "Korab",
+  "mountains": "Balkan",
+  "height": "2751",
+  "longitude": "20.3",
+  "latitude": "41.4"
+}
+{
+  "-id": "mount-Olymp",
+  "-country": "GR",
+  "name": "Olymp",
+  "located": {
+    "-country": "GR",
+    "-province": "prov-cid-cia-Greece-13 prov-cid-cia-Greece-8"
+  },
+  "height": "2917",
+  "longitude": "22.35",
+  "latitude": "40.1"
+}
+{
+  "-id": "mount-Psiloritis",
+  "-country": "GR",
+  "-island": "island-Crete",
+  "name": "Psiloritis",
+  "mountains": "Crete",
+  "located": {
+    "-country": "GR",
+    "-province": "prov-cid-cia-Greece-9"
+  },
+  "height": "2456",
+  "longitude": "24.7",
+  "latitude": "35.2"
+}
+{
+  "-id": "mount-Olympos",
+  "-country": "CY",
+  "-island": "island-Cyprus",
+  "name": "Olympos",
+  "mountains": "Troodos",
+  "height": "1952",
+  "longitude": "32.85",
+  "latitude": "34.9"
+}
+{
+  "-id": "mount-Teide",
+  "-country": "E",
+  "-type": "volcano",
+  "-island": "island-Teneriffa",
+  "name": "Pico de Teide",
+  "mountains": "Canary Islands",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-7"
+  },
+  "height": "3718",
+  "longitude": "-16.6",
+  "latitude": "28.3"
+}
+{
+  "-id": "mount-PicoNieves",
+  "-country": "E",
+  "-type": "volcano",
+  "-island": "island-Gran_Canaria",
+  "name": "Pico de los Nieves",
+  "mountains": "Canary Islands",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-7"
+  },
+  "height": "1949",
+  "longitude": "-15.6",
+  "latitude": "25.95"
+}
+{
+  "-id": "mount-RoqueMuchachos",
+  "-country": "E",
+  "-type": "volcano",
+  "-island": "island-La_Palma",
+  "name": "Roque de los Muchachos",
+  "mountains": "Canary Islands",
+  "located": {
+    "-country": "E",
+    "-province": "prov-cid-cia-Spain-7"
+  },
+  "height": "2426",
+  "longitude": "-17.9",
+  "latitude": "28.75"
+}
+{
+  "-id": "mountain-PicoRuivo",
+  "-country": "P",
+  "-island": "island-Madeira",
+  "-type": "volcanic",
+  "name": "Pico Ruivo",
+  "mountains": "Madeira",
+  "located": {
+    "-country": "P",
+    "-province": "prov-cid-cia-Portugal-22"
+  },
+  "height": "1861",
+  "longitude": "-16",
+  "latitude": "31"
+}
+{
+  "-id": "mountain-Pico",
+  "-country": "P",
+  "-type": "volcano",
+  "-island": "island-Pico",
+  "name": "Pico",
+  "mountains": "Azores",
+  "located": {
+    "-country": "P",
+    "-province": "prov-cid-cia-Portugal-21"
+  },
+  "height": "2351",
+  "longitude": "-28.4",
+  "latitude": "38.5"
+}
+{
+  "-id": "mountain-Fogo",
+  "-country": "CV",
+  "-type": "volcano",
+  "-island": "island-Fogo",
+  "name": "Fogo",
+  "mountains": "Cape Verdes",
+  "height": "2829",
+  "longitude": "-24.4",
+  "latitude": "14.95"
+}
+{
+  "-id": "mount-Tubkhal",
+  "-country": "MA",
+  "name": "Tubkhal",
+  "mountains": "Atlas",
+  "height": "4165",
+  "longitude": "-8",
+  "latitude": "31"
+}
+{
+  "-id": "mount-Djebel_Aures",
+  "-country": "MA",
+  "name": "Djebel Aures",
+  "mountains": "Rif",
+  "height": "2456",
+  "longitude": "-6.5",
+  "latitude": "35.5"
+}
+{
+  "-id": "mount-Tahat",
+  "-country": "DZ",
+  "-type": "volcanic",
+  "name": "Tahat",
+  "mountains": "Ahaggar",
+  "height": "3003",
+  "longitude": "5.5",
+  "latitude": "23.3"
+}
+{
+  "-id": "mount-Tamgak",
+  "-country": "RN",
+  "-type": "volcanic",
+  "name": "Tamgak",
+  "mountains": "Azbine",
+  "height": "1800",
+  "longitude": "9",
+  "latitude": "19"
+}
+{
+  "-id": "mount-MtGreboun",
+  "-country": "RN",
+  "name": "Mont Greboun",
+  "mountains": "Azbine",
+  "height": "1944",
+  "longitude": "9",
+  "latitude": "20"
+}
+{
+  "-id": "mount-Emi_Kussi",
+  "-country": "TCH",
+  "-type": "volcano",
+  "name": "Emi Kussi",
+  "mountains": "Tibesti",
+  "height": "3415",
+  "longitude": "18.5",
+  "latitude": "20"
+}
+{
+  "-id": "mount-Djebel_Marra",
+  "-country": "SUD",
+  "-type": "volcanic",
+  "name": "Jabal Marra",
+  "mountains": "Darfur",
+  "located": {
+    "-country": "SUD",
+    "-province": "prov-cid-cia-Sudan-4"
+  },
+  "height": "3088",
+  "longitude": "24.6",
+  "latitude": "13"
+}
+{
+  "-id": "mount-DjebelKatrina",
+  "-country": "ET",
+  "name": "Jabal Katrina",
+  "mountains": "Sinai",
+  "located": {
+    "-country": "ET",
+    "-province": "prov-cid-cia-Egypt-5"
+  },
+  "height": "2637",
+  "longitude": "33.95",
+  "latitude": "28.5"
+}
+{
+  "-id": "mount-Fako",
+  "-country": "CAM",
+  "-type": "volcano",
+  "name": "Fako",
+  "located": {
+    "-country": "CAM",
+    "-province": "prov-cid-cia-Cameroon-11"
+  },
+  "height": "4095",
+  "longitude": "9.2",
+  "latitude": "4.2"
+}
+{
+  "-id": "mount-Basile",
+  "-country": "GQ",
+  "-island": "island-Bioko",
+  "-type": "volcano",
+  "name": "Pico Basile",
+  "height": "3011",
+  "longitude": "8.7",
+  "latitude": "3.5"
+}
+{
+  "-id": "mount-PicoSaoTome",
+  "-country": "STP",
+  "-island": "island-Sao_Tome",
+  "-type": "volcanic",
+  "name": "Pico de Sao Tome",
+  "height": "2024",
+  "longitude": "6.6",
+  "latitude": "0.2"
+}
+{
+  "-id": "mount-QueenMarysPeak",
+  "-country": "HELX",
+  "-island": "island-TristanDaCunha",
+  "-type": "volcano",
+  "name": "Queen Marys Peak",
+  "height": "2060",
+  "longitude": "-12.3",
+  "latitude": "-37.1"
+}
+{
+  "-id": "mount-Ras_Daschan",
+  "-country": "ETH",
+  "-type": "volcanic",
+  "name": "Ras Daschan Terara",
+  "mountains": "Amhara",
+  "height": "4533",
+  "longitude": "38.4",
+  "latitude": "13.2"
+}
+{
+  "-id": "mount-Ruwenzori",
+  "-country": "EAU ZRE",
+  "name": "Ruwenzori",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "ZRE",
+    "-province": "prov-cid-cia-Zaire-8"
+  },
+  "height": "5109",
+  "longitude": "29.9",
+  "latitude": "0.4"
+}
+{
+  "-id": "mount-Elgon",
+  "-country": "EAK EAU",
+  "-type": "volcanic",
+  "name": "Elgon",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "EAK",
+    "-province": "prov-cid-cia-Kenya-9"
+  },
+  "height": "4321",
+  "longitude": "34.55",
+  "latitude": "1.1"
+}
+{
+  "-id": "mount-Kenia",
+  "-country": "EAK",
+  "-type": "volcanic",
+  "name": "Kenia",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "EAK",
+    "-province": "prov-cid-cia-Kenya-4"
+  },
+  "height": "5199",
+  "longitude": "37.3",
+  "latitude": "0.15"
+}
+{
+  "-id": "mount-Kilimanscharo",
+  "-country": "EAT",
+  "-type": "volcano",
+  "name": "Kilimanjaro",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "EAT",
+    "-province": "prov-cid-cia-Tanzania-4"
+  },
+  "height": "5895",
+  "longitude": "37.35",
+  "latitude": "-3.05"
+}
+{
+  "-id": "mount-Mawenzi",
+  "-country": "EAT",
+  "-type": "volcano",
+  "name": "Mawenzi",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "EAT",
+    "-province": "prov-cid-cia-Tanzania-4"
+  },
+  "height": "5148",
+  "longitude": "37.45",
+  "latitude": "-3.1"
+}
+{
+  "-id": "mount-Meru",
+  "-country": "EAT",
+  "-type": "volcano",
+  "name": "Meru",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "EAT",
+    "-province": "prov-cid-cia-Tanzania-3"
+  },
+  "height": "4567",
+  "longitude": "36.8",
+  "latitude": "-3.2"
+}
+{
+  "-id": "mount-Karisimbi",
+  "-country": "ZRE RWA",
+  "-type": "volcano",
+  "name": "Karisimbi",
+  "mountains": "East African Rift",
+  "located": {
+    "-country": "ZRE",
+    "-province": "prov-cid-cia-Zaire-8"
+  },
+  "height": "4507",
+  "longitude": "29.45",
+  "latitude": "-1.5"
+}
+{
+  "-id": "mount-ThabanaNtlenyana",
+  "-country": "LS",
+  "name": "Thabana Ntlenyana",
+  "mountains": "Drakensberge",
+  "height": "3482",
+  "longitude": "29.3",
+  "latitude": "-29.5"
+}
+{
+  "-id": "mount-Cathkin-Peak",
+  "-country": "RSA",
+  "name": "Cathkin Peak",
+  "mountains": "Drakensberge",
+  "located": {
+    "-country": "RSA",
+    "-province": "prov-cid-cia-South-Africa-5"
+  },
+  "height": "3181",
+  "longitude": "29.35",
+  "latitude": "-29.1"
+}
+{
+  "-id": "mount-Tsaratanana",
+  "-country": "RM",
+  "-type": "volcanic",
+  "-island": "island-Madagaskar",
+  "name": "Tsaratanana",
+  "located": {
+    "-country": "RM",
+    "-province": "prov-cid-cia-Madagascar-2"
+  },
+  "height": "2876",
+  "longitude": "48.5",
+  "latitude": "-14"
+}
+{
+  "-id": "mount-Tsiafajavona",
+  "-country": "RM",
+  "-type": "volcanic",
+  "-island": "island-Madagaskar",
+  "name": "Tsiafajavona",
+  "located": {
+    "-country": "RM",
+    "-province": "prov-cid-cia-Madagascar-3"
+  },
+  "height": "2642",
+  "longitude": "47.2",
+  "latitude": "-19.3"
+}
+{
+  "-id": "mount-Andringitra",
+  "-country": "RM",
+  "-type": "volcanic",
+  "-island": "island-Madagaskar",
+  "name": "Andringitra",
+  "located": {
+    "-country": "RM",
+    "-province": "prov-cid-cia-Madagascar-4"
+  },
+  "height": "2658",
+  "longitude": "47",
+  "latitude": "-22.5"
+}
+{
+  "-id": "mount-PitonDeNeiges",
+  "-country": "REUN",
+  "-type": "volcanic",
+  "-island": "island-Reunion",
+  "name": "Piton des Neiges",
+  "height": "3069",
+  "longitude": "55.5",
+  "latitude": "-21.1"
+}
+{
+  "-id": "mount-PitonDeLaFournaise",
+  "-country": "REUN",
+  "-type": "volcano",
+  "-island": "island-Reunion",
+  "name": "Piton de la Fournaise",
+  "height": "2631",
+  "longitude": "55.7",
+  "latitude": "-21.2"
+}
+{
+  "-id": "mount-Narodnaja",
+  "-country": "R",
+  "name": "Narodnaja",
+  "mountains": "Ural",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-4"
+  },
+  "height": "1895",
+  "longitude": "60.1",
+  "latitude": "65.0"
+}
+{
+  "-id": "mount-Demirkazik",
+  "-country": "TR",
+  "name": "Demirkazik",
+  "mountains": "Taurus",
+  "located": {
+    "-country": "TR",
+    "-province": "prov-cid-cia-Turkey-57"
+  },
+  "height": "3756",
+  "longitude": "35.2",
+  "latitude": "37.8"
+}
+{
+  "-id": "mount-Ararat",
+  "-country": "TR",
+  "-type": "volcanic",
+  "name": "Ararat",
+  "mountains": "Kurdistan",
+  "located": {
+    "-country": "TR",
+    "-province": "prov-cid-cia-Turkey-5"
+  },
+  "height": "5165",
+  "longitude": "44.3",
+  "latitude": "39.7"
+}
+{
+  "-id": "mount-JabalLubnan",
+  "-country": "RL",
+  "name": "Jabal Lubnan",
+  "mountains": "Lebanon",
+  "height": "3088",
+  "longitude": "36.1",
+  "latitude": "34.3"
+}
+{
+  "-id": "mount-Elbrus",
+  "-country": "R",
+  "-type": "volcanic",
+  "name": "Elbrus",
+  "mountains": "Kaukasus",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-55"
+  },
+  "height": "5642",
+  "longitude": "42.26",
+  "latitude": "43.21"
+}
+{
+  "-id": "mount-Kasbek",
+  "-country": "R GE",
+  "-type": "volcanic",
+  "name": "Kasbek",
+  "mountains": "Kaukasus",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-57"
+  },
+  "height": "5033",
+  "longitude": "44.31",
+  "latitude": "42.42"
+}
+{
+  "-id": "mount-Damavand",
+  "-country": "IR",
+  "-type": "volcano",
+  "name": "Damavand",
+  "mountains": "Elburs",
+  "located": {
+    "-country": "IR",
+    "-province": "prov-cid-cia-Iran-20"
+  },
+  "height": "5610",
+  "longitude": "52.1",
+  "latitude": "35.95"
+}
+{
+  "-id": "mount-Sabalan",
+  "-country": "IR",
+  "-type": "volcanic",
+  "name": "Sabalan",
+  "mountains": "Kurdistan",
+  "located": {
+    "-country": "IR",
+    "-province": "prov-cid-cia-Iran-3"
+  },
+  "height": "4811",
+  "longitude": "47.8",
+  "latitude": "38.3"
+}
+{
+  "-id": "mount-ZardKuh",
+  "-country": "IR",
+  "name": "Zard Kuh",
+  "mountains": "Zagros",
+  "located": {
+    "-country": "IR",
+    "-province": "prov-cid-cia-Iran-7"
+  },
+  "height": "4550",
+  "longitude": "50.1",
+  "latitude": "32.35"
+}
+{
+  "-id": "mount-JabalShuayb",
+  "-country": "YE",
+  "name": "Jabal Shuayb",
+  "height": "3760",
+  "longitude": "44",
+  "latitude": "15.5"
+}
+{
+  "-id": "mount-Dodabetta",
+  "-country": "IND",
+  "name": "Dodabetta",
+  "mountains": "West Ghats",
+  "located": {
+    "-country": "IND",
+    "-province": "prov-cid-cia-India-23"
+  },
+  "height": "2636",
+  "longitude": "76.7",
+  "latitude": "11.4"
+}
+{
+  "-id": "mount-Pidurutalagala",
+  "-country": "CL",
+  "name": "Pidurutalagala",
+  "height": "2524",
+  "longitude": "80.8",
+  "latitude": "7"
+}
+{
+  "-id": "mount-Zhima",
+  "-country": "R",
+  "-island": "island-Olkhon",
+  "name": "Zhima",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-72"
+  },
+  "height": "1276",
+  "longitude": "107.4",
+  "latitude": "53.15"
+}
+{
+  "-id": "mount-Bjelucha",
+  "-country": "R KAZ",
+  "name": "Bjelucha",
+  "mountains": "Altai",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-64"
+    },
+    {
+      "-country": "KAZ",
+      "-province": "prov-cid-cia-Kazakstan-15"
+    }
+  ],
+  "height": "4506",
+  "longitude": "86.4",
+  "latitude": "49.48"
+}
+{
+  "-id": "mount-Pik_Chan-Tengri",
+  "-country": "KGZ",
+  "name": "Pik Chan-Tengri",
+  "mountains": "Tien Shan",
+  "height": "6995",
+  "longitude": "80",
+  "latitude": "43"
+}
+{
+  "-id": "mount-Pik_Pobeda",
+  "-country": "TJ KGZ",
+  "name": "Pik Pobeda",
+  "mountains": "Tien Shan",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-28"
+  },
+  "height": "7439",
+  "longitude": "80.1",
+  "latitude": "42"
+}
+{
+  "-id": "mount-Pik_Kommunismus",
+  "-country": "TAD",
+  "name": "Pik Kommunizma",
+  "mountains": "Pamir",
+  "located": {
+    "-country": "TAD",
+    "-province": "prov-cid-cia-Tajikistan-6"
+  },
+  "height": "7494",
+  "longitude": "72",
+  "latitude": "39"
+}
+{
+  "-id": "mount-Pik_Lenin",
+  "-country": "TAD KGZ",
+  "name": "Pik Lenina",
+  "mountains": "Pamir",
+  "located": {
+    "-country": "TAD",
+    "-province": "prov-cid-cia-Tajikistan-6"
+  },
+  "height": "7134",
+  "longitude": "72.55",
+  "latitude": "39.2"
+}
+{
+  "-id": "mount-Pik_Revoluzija",
+  "-country": "TAD",
+  "name": "Pik Revoluzija",
+  "mountains": "Pamir",
+  "located": {
+    "-country": "TAD",
+    "-province": "prov-cid-cia-Tajikistan-6"
+  },
+  "height": "6974",
+  "longitude": "72.2",
+  "latitude": "38.3"
+}
+{
+  "-id": "mount-Pik_Moskwa",
+  "-country": "TAD",
+  "name": "Pik Moskva",
+  "mountains": "Pamir",
+  "located": {
+    "-country": "TAD",
+    "-province": "prov-cid-cia-Tajikistan-6"
+  },
+  "height": "6785",
+  "longitude": "71.5",
+  "latitude": "39"
+}
+{
+  "-id": "mount-Pik_Karl-Marx",
+  "-country": "TAD",
+  "name": "Pik Karl-Marx",
+  "mountains": "Pamir",
+  "located": {
+    "-country": "TAD",
+    "-province": "prov-cid-cia-Tajikistan-6"
+  },
+  "height": "6726",
+  "longitude": "71.3",
+  "latitude": "38"
+}
+{
+  "-id": "mount-TirichMir",
+  "-country": "AFG",
+  "name": "Tirich Mir",
+  "mountains": "Hindukusch",
+  "height": "7690",
+  "longitude": "71.8",
+  "latitude": "36.25"
+}
+{
+  "-id": "mount-K2",
+  "-country": "PK TJ",
+  "name": "K2",
+  "mountains": "Karakorum",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-28"
+  },
+  "height": "8611",
+  "longitude": "76.5",
+  "latitude": "35.9"
+}
+{
+  "-id": "mount-GasherbrumII",
+  "-country": "PK TJ",
+  "name": "Gasherbrum II",
+  "mountains": "Karakorum",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-28"
+  },
+  "height": "8034",
+  "longitude": "76.65",
+  "latitude": "35.75"
+}
+{
+  "-id": "mount-GasherbrumI",
+  "-country": "PK TJ",
+  "name": "Gasherbrum I",
+  "mountains": "Karakorum",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-28"
+  },
+  "height": "8080",
+  "longitude": "76.7",
+  "latitude": "35.6"
+}
+{
+  "-id": "mount-BroadPeak",
+  "-country": "PK TJ",
+  "name": "Broad Peak",
+  "mountains": "Karakorum",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-28"
+  },
+  "height": "8051",
+  "longitude": "76.6",
+  "latitude": "35.8"
+}
+{
+  "-id": "mount-NangaParbat",
+  "-country": "PK",
+  "name": "Nanga Parbat",
+  "mountains": "Karakorum",
+  "height": "8125",
+  "longitude": "74.6",
+  "latitude": "35.2"
+}
+{
+  "-id": "mount-NandaDevi",
+  "-country": "IND",
+  "name": "Nanda Devi",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "IND",
+    "-province": "prov-cid-cia-India-25"
+  },
+  "height": "7816",
+  "longitude": "80",
+  "latitude": "30.4"
+}
+{
+  "-id": "mount-Dhaulagiri",
+  "-country": "NEP",
+  "name": "Dhaulagiri",
+  "mountains": "Himalaya",
+  "height": "8167",
+  "longitude": "83.5",
+  "latitude": "28.7"
+}
+{
+  "-id": "mount-Annapurna",
+  "-country": "NEP",
+  "name": "Annapurna",
+  "mountains": "Himalaya",
+  "height": "8091",
+  "longitude": "83.8",
+  "latitude": "28.6"
+}
+{
+  "-id": "mount-Manaslu",
+  "-country": "NEP",
+  "name": "Manaslu",
+  "mountains": "Himalaya",
+  "height": "8163",
+  "longitude": "84.6",
+  "latitude": "28.55"
+}
+{
+  "-id": "mount-Shishapangma",
+  "-country": "TJ",
+  "name": "Shishapangma",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "8027",
+  "longitude": "85.8",
+  "latitude": "28.55"
+}
+{
+  "-id": "mount-Kailash",
+  "-country": "TJ",
+  "name": "Kailash",
+  "mountains": "Transhimalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "6714",
+  "longitude": "81.3",
+  "latitude": "31.05"
+}
+{
+  "-id": "mount-ChoOyu",
+  "-country": "TJ NEP",
+  "name": "Cho Oyu",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "8188",
+  "longitude": "86.65",
+  "latitude": "28.35"
+}
+{
+  "-id": "mount-Mount_Everest",
+  "-country": "TJ NEP",
+  "name": "Mount Everest",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "8848",
+  "longitude": "86.9",
+  "latitude": "28"
+}
+{
+  "-id": "mount-Lhotse",
+  "-country": "TJ NEP",
+  "name": "Lhotse",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "8516",
+  "longitude": "86.9",
+  "latitude": "28"
+}
+{
+  "-id": "mount-Makalu",
+  "-country": "TJ NEP",
+  "name": "Makalu",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-29"
+  },
+  "height": "8485",
+  "longitude": "87.1",
+  "latitude": "27.9"
+}
+{
+  "-id": "mount-Kangchendzonga",
+  "-country": "NEP IND",
+  "name": "Kangchendzonga",
+  "mountains": "Himalaya",
+  "located": {
+    "-country": "IND",
+    "-province": "prov-cid-cia-India-22"
+  },
+  "height": "8586",
+  "longitude": "88.1",
+  "latitude": "27.7"
+}
+{
+  "-id": "mount-KljutschewskajaSopka",
+  "-country": "R",
+  "-type": "volcano",
+  "name": "Kljutschewskaja Sopka",
+  "mountains": "Kamchatka",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-87"
+  },
+  "height": "4750",
+  "longitude": "160.6",
+  "latitude": "56.1"
+}
+{
+  "-id": "mount-ChanbaiShan",
+  "-country": "TJ NOK",
+  "name": "Changbai Shan",
+  "mountains": "Changbai",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-15"
+  },
+  "height": "2744",
+  "longitude": "128.1",
+  "latitude": "42"
+}
+{
+  "-id": "mount-AsahiDake",
+  "-country": "J",
+  "-type": "volcanic",
+  "-island": "island-Hokkaido",
+  "name": "Asahi-Dake",
+  "located": {
+    "-country": "J",
+    "-province": "prov-cid-cia-Japan-3"
+  },
+  "height": "2290",
+  "longitude": "142.85",
+  "latitude": "43.65"
+}
+{
+  "-id": "mount-Fuji",
+  "-country": "J",
+  "-type": "volcano",
+  "name": "Fujisan",
+  "located": {
+    "-country": "J",
+    "-province": "prov-cid-cia-Japan-24 prov-cid-cia-Japan-27"
+  },
+  "height": "3776",
+  "longitude": "139",
+  "latitude": "35"
+}
+{
+  "-id": "mount-Hotaka-Dake",
+  "-country": "J",
+  "-type": "volcano",
+  "name": "Hotaka-Dake",
+  "located": {
+    "-country": "J",
+    "-province": "prov-cid-cia-Japan-25 prov-cid-cia-Japan-26"
+  },
+  "height": "3190",
+  "longitude": "138",
+  "latitude": "36"
+}
+{
+  "-id": "mount-YuShan",
+  "-country": "RC",
+  "-island": "island-Taiwan",
+  "name": "Yu Shan",
+  "height": "3950",
+  "longitude": "121",
+  "latitude": "23.5"
+}
+{
+  "-id": "mount-Pulog",
+  "-country": "RP",
+  "-type": "volcanic",
+  "-island": "island-Luzon",
+  "name": "Pulog",
+  "height": "2928",
+  "longitude": "121",
+  "latitude": "17"
+}
+{
+  "-id": "mount-Pinatubo",
+  "-country": "RP",
+  "-type": "volcano",
+  "-island": "island-Luzon",
+  "name": "Pinatubo",
+  "height": "1486",
+  "longitude": "120.3",
+  "latitude": "15.1"
+}
+{
+  "-id": "mount-Apo",
+  "-country": "RP",
+  "-type": "volcanic",
+  "-island": "island-Mindanao",
+  "name": "Mount Apo",
+  "height": "2954",
+  "longitude": "125.45",
+  "latitude": "7"
+}
+{
+  "-id": "mount-Kanlaon",
+  "-country": "RP",
+  "-type": "volcano",
+  "-island": "island-Negros",
+  "name": "Kanlaon",
+  "height": "2435",
+  "longitude": "123.1",
+  "latitude": "10.4"
+}
+{
+  "-id": "mount-Mantalingajan",
+  "-country": "RP",
+  "-type": "volcanic",
+  "-island": "island-Palawan",
+  "name": "Mantalingajan",
+  "height": "2085",
+  "longitude": "118.4",
+  "latitude": "9.7"
+}
+{
+  "-id": "mount-Kinabalu",
+  "-country": "MAL",
+  "-island": "island-Borneo",
+  "-type": "granite",
+  "name": "Kinabalu",
+  "located": {
+    "-country": "MAL",
+    "-province": "prov-cid-cia-Malaysia-11"
+  },
+  "height": "4101",
+  "longitude": "116.5",
+  "latitude": "6.1"
+}
+{
+  "-id": "mount-Tahan",
+  "-country": "MAL",
+  "name": "Tahan",
+  "located": {
+    "-country": "MAL",
+    "-province": "prov-cid-cia-Malaysia-4 prov-cid-cia-Malaysia-7"
+  },
+  "height": "2190",
+  "longitude": "108",
+  "latitude": "4"
+}
+{
+  "-id": "mount-Leuser",
+  "-country": "RI",
+  "-type": "volcanic",
+  "-island": "island-Sumatra",
+  "name": "Leuser",
+  "height": "3466",
+  "longitude": "98",
+  "latitude": "3"
+}
+{
+  "-id": "mount-Kerinci",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Sumatra",
+  "name": "Kerinci",
+  "height": "3805",
+  "longitude": "101.45",
+  "latitude": "-1.7"
+}
+{
+  "-id": "mount-Krakatau",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Krakatau",
+  "name": "Krakatau",
+  "height": "813",
+  "longitude": "105.4",
+  "latitude": "-6.1"
+}
+{
+  "-id": "mount-Merapi",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Java",
+  "name": "Merapi",
+  "height": "2914",
+  "longitude": "110.4",
+  "latitude": "-7.5"
+}
+{
+  "-id": "mount-Semeru",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Java",
+  "name": "Semeru",
+  "height": "3676",
+  "longitude": "112.9",
+  "latitude": "-8.1"
+}
+{
+  "-id": "mount-GunungAgung",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Bali",
+  "name": "Gunung Agung",
+  "height": "3142",
+  "longitude": "115.5",
+  "latitude": "-8.3"
+}
+{
+  "-id": "mount-Rinjani",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Lombok",
+  "name": "Rinjani",
+  "height": "3726",
+  "longitude": "116.5",
+  "latitude": "-8.4"
+}
+{
+  "-id": "mount-Tambora",
+  "-country": "RI",
+  "-type": "volcano",
+  "-island": "island-Sumbawa",
+  "name": "Tambora",
+  "height": "2850",
+  "longitude": "118",
+  "latitude": "-8.2"
+}
+{
+  "-id": "mount-Rantekombola",
+  "-country": "RI",
+  "-type": "volcanic",
+  "-island": "island-Sulawesi",
+  "name": "Rantekombola",
+  "height": "3455",
+  "longitude": "120",
+  "latitude": "-3.6"
+}
+{
+  "-id": "mount-Tatamailau",
+  "-country": "TL",
+  "-island": "island-Timor",
+  "name": "Tatamailau",
+  "height": "2963",
+  "longitude": "125.5",
+  "latitude": "-8.9"
+}
+{
+  "-id": "mount-GunungBinaiya",
+  "-country": "RI",
+  "-island": "island-Ceram",
+  "name": "Gunung Binaiya",
+  "height": "3027",
+  "longitude": "129.3",
+  "latitude": "-3.1"
+}
+{
+  "-id": "mount-PuncakJaya",
+  "-country": "RI",
+  "-island": "island-New-Guinea",
+  "name": "Puncak Jaya",
+  "mountains": "Sudirman Range",
+  "height": "4884",
+  "longitude": "137.11",
+  "latitude": "-4.1"
+}
+{
+  "-id": "mount-MtWilhelm",
+  "-country": "PNG",
+  "-island": "island-New-Guinea",
+  "name": "Mt. Wilhelm",
+  "height": "4509",
+  "longitude": "145",
+  "latitude": "-5.8"
+}
+{
+  "-id": "mount-Giluwe",
+  "-country": "PNG",
+  "-type": "volcano",
+  "-island": "island-New-Guinea",
+  "name": "Mt. Giluwe",
+  "height": "4368",
+  "longitude": "143.9",
+  "latitude": "-6.05"
+}
+{
+  "-id": "mount-Balbi",
+  "-country": "PNG",
+  "-type": "volcanic",
+  "-island": "island-Bougainville",
+  "name": "Mt. Balbi",
+  "height": "2715",
+  "longitude": "155",
+  "latitude": "-5.9"
+}
+{
+  "-id": "mount-Victoria",
+  "-country": "FJI",
+  "-type": "volcanic",
+  "-island": "island-VitiLevu",
+  "name": "Mt.Victoria",
+  "height": "1323",
+  "longitude": "178",
+  "latitude": "-17.8"
+}
+{
+  "-id": "mount-Silisili",
+  "-country": "WS",
+  "-type": "volcanic",
+  "-island": "island-Savaii",
+  "name": "Silisili",
+  "height": "1858",
+  "longitude": "-172.4",
+  "latitude": "-13.6"
+}
+{
+  "-id": "mount-Orohena",
+  "-country": "FPOL",
+  "-type": "volcanic",
+  "-island": "island-Tahiti",
+  "name": "Mont Orohena",
+  "height": "2241",
+  "longitude": "-149.5",
+  "latitude": "-17.6"
+}
+{
+  "-id": "mount-Popomanaseu",
+  "-country": "SLB",
+  "-type": "volcanic",
+  "-island": "island-Guadalcanal",
+  "name": "Popomanaseu",
+  "height": "2335",
+  "longitude": "160.1",
+  "latitude": "-9.7"
+}
+{
+  "-id": "mount-Mauna_Kea",
+  "-country": "USA",
+  "-type": "volcano",
+  "-island": "island-Hawaii",
+  "name": "Mauna Kea",
+  "mountains": "Hawaii",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-12"
+  },
+  "height": "4214",
+  "longitude": "-155.45",
+  "latitude": "19.6"
+}
+{
+  "-id": "mount-Mauna_Loa",
+  "-country": "USA",
+  "-type": "volcano",
+  "-island": "island-Hawaii",
+  "name": "Mauna Loa",
+  "mountains": "Hawaii",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-12"
+  },
+  "height": "4170",
+  "longitude": "-155.6",
+  "latitude": "19.5"
+}
+{
+  "-id": "mount-Haleakala",
+  "-country": "USA",
+  "-type": "volcano",
+  "-island": "island-Maui",
+  "name": "Haleakala",
+  "mountains": "Hawaii",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-12"
+  },
+  "height": "3055",
+  "longitude": "-156.25",
+  "latitude": "20.7"
+}
+{
+  "-id": "mount-Mt-Kosciuszko",
+  "-country": "AUS",
+  "name": "Mt. Kosciuszko",
+  "mountains": "Snowy Mountains",
+  "located": {
+    "-country": "AUS",
+    "-province": "prov-cid-cia-Australia-2"
+  },
+  "height": "2228",
+  "longitude": "148",
+  "latitude": "-36.5"
+}
+{
+  "-id": "mount-Mt-Bogong",
+  "-country": "AUS",
+  "name": "Mt. Bogong",
+  "mountains": "Snowy Mountains",
+  "located": {
+    "-country": "AUS",
+    "-province": "prov-cid-cia-Australia-7"
+  },
+  "height": "1986",
+  "longitude": "147.5",
+  "latitude": "-37"
+}
+{
+  "-id": "mount-Uluru",
+  "-country": "AUS",
+  "-type": "monolith",
+  "name": "Uluru",
+  "located": {
+    "-country": "AUS",
+    "-province": "prov-cid-cia-Australia-3"
+  },
+  "height": "869",
+  "longitude": "131",
+  "latitude": "-25.3"
+}
+{
+  "-id": "mount-MtCook",
+  "-country": "NZ",
+  "-island": "island-TeWakaAMaui",
+  "name": "Mt. Cook",
+  "mountains": "New Zealand Alps",
+  "height": "3754",
+  "longitude": "170.2",
+  "latitude": "-43.6"
+}
+{
+  "-id": "mount-Ruapehu",
+  "-country": "NZ",
+  "-type": "volcano",
+  "-island": "island-TeIkaAMaui",
+  "name": "Ruapehu",
+  "height": "2797",
+  "longitude": "175.6",
+  "latitude": "-39.3"
+}
+{
+  "-id": "mount-Mt_Redoubt",
+  "-country": "USA",
+  "-type": "volcano",
+  "name": "Mt Redoubt",
+  "mountains": "Aleutian Range",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-3"
+  },
+  "height": "3108",
+  "longitude": "-152.75",
+  "latitude": "60.5"
+}
+{
+  "-id": "mount-Mt_McKinley",
+  "-country": "USA",
+  "name": "Mt McKinley",
+  "mountains": "Alaska Range",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-3"
+  },
+  "height": "6193",
+  "longitude": "-151",
+  "latitude": "63.1"
+}
+{
+  "-id": "mount-MtBlackburn",
+  "-type": "volcano",
+  "-country": "USA",
+  "name": "Mt Blackburn",
+  "mountains": "Wrangell Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-3"
+  },
+  "height": "4996",
+  "longitude": "-143.4",
+  "latitude": "61.7"
+}
+{
+  "-id": "mount-MtBona",
+  "-country": "USA CDN",
+  "name": "Mt Bona",
+  "mountains": "Elias Range",
+  "located": [
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-12"
+    }
+  ],
+  "height": "5005",
+  "longitude": "-141.75",
+  "latitude": "61.4"
+}
+{
+  "-id": "mount-StElias",
+  "-country": "USA CDN",
+  "name": "Mt St.Elias",
+  "mountains": "Elias Range",
+  "located": [
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-13"
+    }
+  ],
+  "height": "5489",
+  "longitude": "-140.9",
+  "latitude": "60.3"
+}
+{
+  "-id": "mount-Logan",
+  "-country": "CDN",
+  "name": "Mt Logan",
+  "mountains": "Elias Range",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-13"
+  },
+  "height": "5959",
+  "longitude": "-140.4",
+  "latitude": "60.6"
+}
+{
+  "-id": "mount-Fairweather",
+  "-country": "CDN USA",
+  "name": "Mt Fairweather",
+  "mountains": "Elias Range",
+  "located": [
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-3"
+    }
+  ],
+  "height": "4663",
+  "longitude": "-137.5",
+  "latitude": "58.9"
+}
+{
+  "-id": "mount-Waddington",
+  "-country": "CDN",
+  "name": "Mt Waddington",
+  "mountains": "Coast Mountains",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-3"
+  },
+  "height": "4019",
+  "longitude": "-125.26",
+  "latitude": "51.4"
+}
+{
+  "-id": "mount-Robson",
+  "-country": "CDN",
+  "name": "Mt Robson",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-3"
+  },
+  "height": "3954",
+  "longitude": "-119.1",
+  "latitude": "52.3"
+}
+{
+  "-id": "mount-Columbia",
+  "-country": "CDN",
+  "name": "Mt Columbia",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-3 prov-cid-cia-Canada-2"
+  },
+  "height": "3747",
+  "longitude": "-117.44",
+  "latitude": "52.15"
+}
+{
+  "-id": "mount-Barbeau",
+  "-country": "CDN",
+  "-island": "island-Ellesmere_Island",
+  "name": "Barbeau Peak",
+  "mountains": "British Empire Range",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-7"
+  },
+  "height": "2616",
+  "longitude": "-75.0",
+  "latitude": "81.9"
+}
+{
+  "-id": "mount-Odin",
+  "-country": "CDN",
+  "-island": "island-Baffin_Island",
+  "name": "Mt. Odin",
+  "mountains": "Penny Highlands",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-7"
+  },
+  "height": "2147",
+  "longitude": "-65.6",
+  "latitude": "66.55"
+}
+{
+  "-id": "mount-Babel",
+  "-country": "CDN",
+  "-island": "island-ReneLevasseur",
+  "name": "Mt. Babel",
+  "located": {
+    "-country": "CDN",
+    "-province": "prov-cid-cia-Canada-11"
+  },
+  "height": "952",
+  "longitude": "-68.7",
+  "latitude": "51.45"
+}
+{
+  "-id": "mount-Granite_Peak",
+  "-country": "USA",
+  "name": "Granite Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-27"
+  },
+  "height": "3901",
+  "longitude": "-109.5",
+  "latitude": "45"
+}
+{
+  "-id": "mount-Borah_Peak",
+  "-country": "USA",
+  "name": "Borah Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-13"
+  },
+  "height": "3859",
+  "longitude": "-113.8",
+  "latitude": "44.1"
+}
+{
+  "-id": "mount-Harney_Peak",
+  "-country": "USA",
+  "name": "Harney Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-42"
+  },
+  "height": "2207",
+  "longitude": "-103.4",
+  "latitude": "44"
+}
+{
+  "-id": "mount-Gannett_Peak",
+  "-country": "USA",
+  "name": "Gannett Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-51"
+  },
+  "height": "4207",
+  "longitude": "-110",
+  "latitude": "43.2"
+}
+{
+  "-id": "mount-Kings_Peak",
+  "-country": "USA",
+  "name": "Kings Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-45"
+  },
+  "height": "4123",
+  "longitude": "-110.4",
+  "latitude": "40.5"
+}
+{
+  "-id": "mount-Mt_Elbert",
+  "-country": "USA",
+  "name": "Mt Elbert",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-7"
+  },
+  "height": "4401",
+  "longitude": "-106.4",
+  "latitude": "39.1"
+}
+{
+  "-id": "mount-PikesPeak",
+  "-country": "USA",
+  "name": "Pikes Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-7"
+  },
+  "height": "4302",
+  "longitude": "-105",
+  "latitude": "38.8"
+}
+{
+  "-id": "mount-Wheeler",
+  "-country": "USA",
+  "name": "Wheeler Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-32"
+  },
+  "height": "4011",
+  "longitude": "-105.4",
+  "latitude": "36.55"
+}
+{
+  "-id": "mount-Humphreys_Peak",
+  "-country": "USA",
+  "-type": "volcanic",
+  "name": "Humphreys Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-4"
+  },
+  "height": "3850",
+  "longitude": "-111.65",
+  "latitude": "35.3"
+}
+{
+  "-id": "mount-Guadalupe_Peak",
+  "-country": "USA",
+  "name": "Guadalupe Peak",
+  "mountains": "Rocky Mountains",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-44"
+  },
+  "height": "2667",
+  "longitude": "-105",
+  "latitude": "32"
+}
+{
+  "-id": "mount-Mt_Rainier",
+  "-country": "USA",
+  "-type": "volcano",
+  "name": "Mt Rainier",
+  "mountains": "Cascade Range",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-48"
+  },
+  "height": "4392",
+  "longitude": "-121.75",
+  "latitude": "46.5"
+}
+{
+  "-id": "mount-Adams",
+  "-country": "USA",
+  "-type": "volcano",
+  "name": "Mt Adams",
+  "mountains": "Cascade Range",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-48"
+  },
+  "height": "3743",
+  "longitude": "-121.5",
+  "latitude": "46.2"
+}
+{
+  "-id": "mount-Mt_Hood",
+  "-country": "USA",
+  "-type": "volcano",
+  "name": "Mt Hood",
+  "mountains": "Cascade Range",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-38"
+  },
+  "height": "3424",
+  "longitude": "-121.5",
+  "latitude": "45.2"
+}
+{
+  "-id": "mount-Mt_Whitney",
+  "-country": "USA",
+  "name": "Mt Whitney",
+  "mountains": "Sierra Nevada California",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-6"
+  },
+  "height": "4418",
+  "longitude": "-118.2",
+  "latitude": "36.6"
+}
+{
+  "-id": "mount-Boundary_Peak",
+  "-country": "USA",
+  "name": "Boundary Peak",
+  "mountains": "Sierra Nevada California",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-29 prov-cid-cia-United-States-6"
+  },
+  "height": "4006",
+  "longitude": "-118.3",
+  "latitude": "37.9"
+}
+{
+  "-id": "mount-Mt_Washington",
+  "-country": "USA",
+  "name": "Mt Washington",
+  "mountains": "Appalachen",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-30"
+  },
+  "height": "1917",
+  "longitude": "-71.3",
+  "latitude": "44.3"
+}
+{
+  "-id": "mount-Mt_Marcy",
+  "-country": "USA",
+  "name": "Mt Marcy",
+  "mountains": "Adirondacks",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-33"
+  },
+  "height": "1629",
+  "longitude": "-73.9",
+  "latitude": "44.1"
+}
+{
+  "-id": "mount-Mt_Mitchell",
+  "-country": "USA",
+  "name": "Mt Mitchell",
+  "mountains": "Appalachen",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-34"
+  },
+  "height": "2037",
+  "longitude": "-82.1",
+  "latitude": "35.4"
+}
+{
+  "-id": "mount-Clingmans_Dome",
+  "-country": "USA",
+  "name": "Clingmans Dome",
+  "mountains": "Appalachen",
+  "located": {
+    "-country": "USA",
+    "-province": "prov-cid-cia-United-States-43"
+  },
+  "height": "2025",
+  "longitude": "-83.2",
+  "latitude": "35.3"
+}
+{
+  "-id": "mount-Citlaltepetl",
+  "-country": "MEX",
+  "-type": "volcano",
+  "name": "Citlaltepetl (Pico de Orizaba)",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "MEX",
+    "-province": "prov-cid-cia-Mexico-21 prov-cid-cia-Mexico-30"
+  },
+  "height": "5700",
+  "longitude": "-97",
+  "latitude": "19"
+}
+{
+  "-id": "mount-Popocatepetl",
+  "-country": "MEX",
+  "-type": "volcano",
+  "name": "Popocatepetl",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "MEX",
+    "-province": "prov-cid-cia-Mexico-15 prov-cid-cia-Mexico-17 prov-cid-cia-Mexico-21"
+  },
+  "height": "5450",
+  "longitude": "-98",
+  "latitude": "19"
+}
+{
+  "-id": "mount-Iztaccihuatl",
+  "-country": "MEX",
+  "-type": "volcanic",
+  "name": "Iztaccihuatl",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "MEX",
+    "-province": "prov-cid-cia-Mexico-21"
+  },
+  "height": "5286",
+  "longitude": "-98.6",
+  "latitude": "19.2"
+}
+{
+  "-id": "mount-Nevado_de_Toluca",
+  "-country": "MEX",
+  "-type": "volcano",
+  "name": "Nevado de Toluca",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "MEX",
+    "-province": "prov-cid-cia-Mexico-15"
+  },
+  "height": "4558",
+  "longitude": "-99",
+  "latitude": "19"
+}
+{
+  "-id": "mount-Nevado_de_Colima",
+  "-country": "MEX",
+  "-type": "volcano",
+  "name": "Nevado de Colima",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "MEX",
+    "-province": "prov-cid-cia-Mexico-14"
+  },
+  "height": "4330",
+  "longitude": "-104",
+  "latitude": "20"
+}
+{
+  "-id": "mount-Tajumulco",
+  "-country": "GCA",
+  "-type": "volcanic",
+  "name": "Tajumulco",
+  "mountains": "Sierra Madre",
+  "height": "4220",
+  "longitude": "-91.9",
+  "latitude": "15"
+}
+{
+  "-id": "mount-Concepcion",
+  "-country": "NIC",
+  "-type": "volcano",
+  "-island": "island-Ometepe",
+  "name": "Concepcion",
+  "height": "1700",
+  "longitude": "-85.6",
+  "latitude": "11.5"
+}
+{
+  "-id": "mount-Irazu",
+  "-country": "CR",
+  "-type": "volcano",
+  "name": "Irazu",
+  "mountains": "Cordillera de Talamanca",
+  "located": {
+    "-country": "CR",
+    "-province": "prov-cid-cia-Costa-Rica-4"
+  },
+  "height": "3432",
+  "longitude": "-83.9",
+  "latitude": "10"
+}
+{
+  "-id": "mount-CerroChirripo",
+  "-country": "CR",
+  "-type": "volcanic",
+  "name": "Cerro Chirripo",
+  "mountains": "Cordillera de Talamanca",
+  "located": {
+    "-country": "CR",
+    "-province": "prov-cid-cia-Costa-Rica-8 prov-cid-cia-Costa-Rica-4"
+  },
+  "height": "3819",
+  "longitude": "-83.5",
+  "latitude": "9.5"
+}
+{
+  "-id": "mount-PicoTurquino",
+  "-country": "C",
+  "-island": "island-Cuba",
+  "name": "Pico Turquino",
+  "mountains": "Sierra Maestra",
+  "located": {
+    "-country": "C",
+    "-province": "prov-cid-cia-Cuba-6 prov-cid-cia-Cuba-14"
+  },
+  "height": "1974",
+  "longitude": "-76.8",
+  "latitude": "20"
+}
+{
+  "-id": "mount-Blue_Mountain_Peak",
+  "-country": "JA",
+  "-island": "island-Jamaica",
+  "name": "Blue Mountain Peak",
+  "height": "2256",
+  "longitude": "-76.4",
+  "latitude": "18.1"
+}
+{
+  "-id": "mount-Pico_Duarte",
+  "-country": "DOM",
+  "-island": "island-Hispaniola",
+  "name": "Pico Duarte",
+  "height": "3098",
+  "longitude": "-71.3",
+  "latitude": "18.3"
+}
+{
+  "-id": "mount-CerroPunta",
+  "-country": "PR",
+  "-island": "island-PuertoRico",
+  "name": "Cerro de Punta",
+  "height": "1338",
+  "longitude": "-66.6",
+  "latitude": "18.2"
+}
+{
+  "-id": "mount-LaSoufriere",
+  "-country": "GUAD",
+  "-type": "volcano",
+  "-island": "island-Basse-Terre",
+  "name": "La Soufriere",
+  "height": "1467",
+  "longitude": "-61.65",
+  "latitude": "16.05"
+}
+{
+  "-id": "mount-SoufriereHills",
+  "-country": "MNTS",
+  "-type": "volcano",
+  "-island": "island-Montserrat",
+  "name": "Soufriere Hills",
+  "height": "1050",
+  "longitude": "-62.2",
+  "latitude": "16.7"
+}
+{
+  "-id": "mount-Soufriere",
+  "-country": "WV",
+  "-type": "volcano",
+  "-island": "island-Saint_Vincent",
+  "name": "Soufriere",
+  "height": "1220",
+  "longitude": "-61.2",
+  "latitude": "13.35"
+}
+{
+  "-id": "mount-MorneDiablotins",
+  "-country": "WD",
+  "-type": "volcano",
+  "-island": "island-Dominica",
+  "name": "Morne Diablotins",
+  "height": "1447",
+  "longitude": "-61.4",
+  "latitude": "15.5"
+}
+{
+  "-id": "mount-Pelee",
+  "-country": "MART",
+  "-type": "volcano",
+  "-island": "island-Martinique",
+  "name": "Pelee",
+  "height": "1397",
+  "longitude": "-61.15",
+  "latitude": "14.8"
+}
+{
+  "-id": "mount-PicoCristobalColon",
+  "-country": "CO",
+  "name": "Pico Cristobal Colon",
+  "mountains": "Sierra Nevada de Santa Marta",
+  "located": {
+    "-country": "CO",
+    "-province": "prov-cid-cia-Colombia-20"
+  },
+  "height": "5775",
+  "longitude": "-73.7",
+  "latitude": "10.85"
+}
+{
+  "-id": "mount-PicoBolivar",
+  "-country": "YV",
+  "name": "Pico Bolivar",
+  "mountains": "Andes",
+  "located": {
+    "-country": "YV",
+    "-province": "prov-cid-cia-Venezuela-14"
+  },
+  "height": "4981",
+  "longitude": "-71.0",
+  "latitude": "8.6"
+}
+{
+  "-id": "mount-NevadoDelRuiz",
+  "-country": "CO",
+  "-type": "volcano",
+  "name": "Nevado del Ruiz",
+  "mountains": "Andes",
+  "located": {
+    "-country": "CO",
+    "-province": "prov-cid-cia-Colombia-26 prov-cid-cia-Colombia-13"
+  },
+  "height": "5389",
+  "longitude": "-75.3",
+  "latitude": "4.9"
+}
+{
+  "-id": "mount-NevadoDelHuila",
+  "-country": "CO",
+  "-type": "volcano",
+  "name": "Nevado del Huila",
+  "mountains": "Andes",
+  "located": {
+    "-country": "CO",
+    "-province": "prov-cid-cia-Colombia-19"
+  },
+  "height": "5750",
+  "longitude": "-76.0",
+  "latitude": "2.9"
+}
+{
+  "-id": "mount-Cayambe",
+  "-country": "EC",
+  "-type": "volcano",
+  "name": "Cayambe",
+  "mountains": "Andes",
+  "height": "5796",
+  "longitude": "-78",
+  "latitude": "0.1"
+}
+{
+  "-id": "mount-Cotopaxi",
+  "-country": "EC",
+  "-type": "volcano",
+  "name": "Cotopaxi",
+  "mountains": "Andes",
+  "height": "5897",
+  "longitude": "-78.4",
+  "latitude": "-0.7"
+}
+{
+  "-id": "mount-Chimborazo",
+  "-country": "EC",
+  "-type": "volcano",
+  "name": "Chimborazo",
+  "mountains": "Andes",
+  "height": "6287",
+  "longitude": "-78.5",
+  "latitude": "-1.2"
+}
+{
+  "-id": "mount-Huascaran",
+  "-country": "PE",
+  "name": "Huascaran",
+  "mountains": "Cordillera Blanca",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-3"
+  },
+  "height": "6768",
+  "longitude": "-77.3",
+  "latitude": "-9"
+}
+{
+  "-id": "mount-Alpamayo",
+  "-country": "PE",
+  "name": "Alpamayo",
+  "mountains": "Cordillera Blanca",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-3"
+  },
+  "height": "5947",
+  "longitude": "-77.7",
+  "latitude": "-8.9"
+}
+{
+  "-id": "mount-Rocarre",
+  "-country": "PE",
+  "name": "Pico Rocarre",
+  "mountains": "Cordillera Negra",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-3"
+  },
+  "height": "5187",
+  "longitude": "-78",
+  "latitude": "-8.8"
+}
+{
+  "-id": "mount-Ausangate",
+  "-country": "PE",
+  "name": "Ausangate",
+  "mountains": "Cordillera Vilcanota",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-9"
+  },
+  "height": "6336",
+  "longitude": "-71.2",
+  "latitude": "-13.8"
+}
+{
+  "-id": "mount-Coropuna",
+  "-country": "PE",
+  "-type": "volcano",
+  "name": "Coropuna",
+  "mountains": "Cordillera Volcanica",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-5"
+  },
+  "height": "6425",
+  "longitude": "-72.65",
+  "latitude": "-15.5"
+}
+{
+  "-id": "mount-Ampato",
+  "-country": "PE",
+  "-type": "volcano",
+  "name": "Ampato",
+  "mountains": "Andes",
+  "located": {
+    "-country": "PE",
+    "-province": "prov-cid-cia-Peru-5"
+  },
+  "height": "6288",
+  "longitude": "-71.9",
+  "latitude": "-15.8"
+}
+{
+  "-id": "mount-Illampu",
+  "-country": "BOL",
+  "-type": "granite",
+  "name": "Illampu",
+  "mountains": "Cordillera Real",
+  "height": "6368",
+  "longitude": "-68.5",
+  "latitude": "-15.8"
+}
+{
+  "-id": "mount-Illimani",
+  "-country": "BOL",
+  "-type": "granite",
+  "name": "Illimani",
+  "mountains": "Cordillera Real",
+  "height": "6439",
+  "longitude": "-67.8",
+  "latitude": "-16.65"
+}
+{
+  "-id": "mount-Sajama",
+  "-country": "BOL",
+  "-type": "volcano",
+  "name": "Sajama",
+  "mountains": "Andes",
+  "height": "6542",
+  "longitude": "-68.9",
+  "latitude": "-18.1"
+}
+{
+  "-id": "mount-Licancabur",
+  "-country": "BOL RCH",
+  "-type": "volcano",
+  "name": "Licancabur",
+  "mountains": "Andes",
+  "height": "5920",
+  "longitude": "-67.9",
+  "latitude": "-22.8"
+}
+{
+  "-id": "mount-Llullaillaco",
+  "-country": "RA RCH",
+  "-type": "volcano",
+  "name": "Llullaillaco",
+  "mountains": "Andes",
+  "located": {
+    "-country": "RA",
+    "-province": "prov-cid-cia-Argentina-24"
+  },
+  "height": "6739",
+  "longitude": "-68.5",
+  "latitude": "-24.7"
+}
+{
+  "-id": "mount-OjosdelSalado",
+  "-country": "RA RCH",
+  "-type": "volcano",
+  "name": "Ojos del Salado",
+  "mountains": "Andes",
+  "located": {
+    "-country": "RA",
+    "-province": "prov-cid-cia-Argentina-24"
+  },
+  "height": "6893",
+  "longitude": "-68.5",
+  "latitude": "-27.1"
+}
+{
+  "-id": "mount-MontePissis",
+  "-country": "RA",
+  "-type": "volcanic",
+  "name": "Monte Pissis",
+  "mountains": "Andes",
+  "located": {
+    "-country": "RA",
+    "-province": "prov-cid-cia-Argentina-12 prov-cid-cia-Argentina-24"
+  },
+  "height": "6795",
+  "longitude": "-68.85",
+  "latitude": "-27.8"
+}
+{
+  "-id": "mount-Aconcagua",
+  "-country": "RA",
+  "name": "Aconcagua",
+  "mountains": "Andes",
+  "located": {
+    "-country": "RA",
+    "-province": "prov-cid-cia-Argentina-13"
+  },
+  "height": "6962",
+  "longitude": "-70.0",
+  "latitude": "-32.65"
+}
+{
+  "-id": "mount-Tupungato",
+  "-country": "RA",
+  "-type": "volcano",
+  "name": "Tupungato",
+  "mountains": "Andes",
+  "located": {
+    "-country": "RA",
+    "-province": "prov-cid-cia-Argentina-13"
+  },
+  "height": "6550",
+  "longitude": "-69.870",
+  "latitude": "-33.35"
+}
+{
+  "-id": "mount-AgulhasNegras",
+  "-country": "BR",
+  "name": "Pico de Agulhas Negras",
+  "mountains": "Brazilian Highlands",
+  "located": {
+    "-country": "BR",
+    "-province": "prov-cid-cia-Brazil-13 prov-cid-cia-Brazil-19"
+  },
+  "height": "2791",
+  "longitude": "-44.65",
+  "latitude": "-22.4"
+}
diff --git a/asterixdb/asterix-app/data/country/sea.adm b/asterixdb/asterix-app/data/country/sea.adm
new file mode 100644
index 0000000..757a7fc
--- /dev/null
+++ b/asterixdb/asterix-app/data/country/sea.adm
@@ -0,0 +1,716 @@
+{
+  "-id": "sea-Skagerrak",
+  "-country": "N S DK",
+  "-bordering": "sea-Nordsee sea-Kattegat",
+  "name": "Skagerrak",
+  "located": [
+    {
+      "-country": "N",
+      "-province": "lteil-OS-N lteil-AK-N lteil-OES-N lteil-BU-N lteil-VE-N lteil-AA-N lteil-VA-N"
+    },
+    {
+      "-country": "S",
+      "-province": "prov-cid-cia-Sweden-5"
+    }
+  ],
+  "depth": "725"
+}
+{
+  "-id": "sea-Kattegat",
+  "-country": "S DK",
+  "-bordering": "sea-Skagerrak sea-Ostsee",
+  "name": "Kattegat",
+  "located": {
+    "-country": "S",
+    "-province": "prov-cid-cia-Sweden-5 prov-cid-cia-Sweden-7 prov-cid-cia-Sweden-12 prov-cid-cia-Sweden-14"
+  },
+  "depth": "94"
+}
+{
+  "-id": "sea-Nordsee",
+  "-country": "NL B F DK N GB D",
+  "-bordering": "sea-Atlantic sea-Europaeisches_Nordmeer sea-Channel sea-Skagerrak",
+  "name": "North Sea",
+  "located": [
+    {
+      "-country": "D",
+      "-province": "prov-cid-cia-Germany-6 prov-cid-cia-Germany-10 prov-cid-cia-Germany-16"
+    },
+    {
+      "-country": "NL",
+      "-province": "prov-cid-cia-Netherlands-2 prov-cid-cia-Netherlands-3 prov-cid-cia-Netherlands-6 prov-cid-cia-Netherlands-9 prov-cid-cia-Netherlands-10 prov-cid-cia-Netherlands-11"
+    },
+    {
+      "-country": "B",
+      "-province": "prov-cid-cia-Belgium-10"
+    },
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-86"
+    },
+    {
+      "-country": "N",
+      "-province": "lteil-TE-N lteil-VA-N lteil-RO-N lteil-HO-N lteil-SF-N"
+    },
+    {
+      "-country": "GB",
+      "-province": "prov-cid-cia-United-Kingdom-18 prov-cid-cia-United-Kingdom-16 prov-cid-cia-United-Kingdom-23 prov-cid-cia-United-Kingdom-25 prov-cid-cia-United-Kingdom-30 prov-cid-cia-United-Kingdom-32 prov-cid-cia-United-Kingdom-33 prov-cid-cia-United-Kingdom-40 prov-cid-cia-United-Kingdom-42 prov-cid-cia-United-Kingdom-54 prov-cid-cia-United-Kingdom-55 prov-cid-cia-United-Kingdom-86  prov-cid-cia-United-Kingdom-88"
+    }
+  ],
+  "depth": "200"
+}
+{
+  "-id": "sea-Channel",
+  "-country": "F GB GBG GBJ",
+  "-bordering": "sea-Atlantic sea-Nordsee",
+  "name": "The Channel",
+  "located": [
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-16 prov-cid-cia-France-50 prov-cid-cia-France-86 prov-cid-cia-France-95"
+    },
+    {
+      "-country": "GB",
+      "-province": "prov-cid-cia-United-Kingdom-9 prov-cid-cia-United-Kingdom-12 prov-cid-cia-United-Kingdom-13 prov-cid-cia-United-Kingdom-15 prov-cid-cia-United-Kingdom-20 prov-cid-cia-United-Kingdom-25 prov-cid-cia-United-Kingdom-45"
+    }
+  ],
+  "depth": "175"
+}
+{
+  "-id": "sea-Schwarzes_Meer",
+  "-country": "UA R BG RO TR GE",
+  "-bordering": "sea-Mittelmeer sea-Asowsches_Meer",
+  "name": "Black Sea",
+  "located": [
+    {
+      "-country": "UA",
+      "-province": "prov-cid-cia-Ukraine-9 prov-cid-cia-Ukraine-15 prov-cid-cia-Ukraine-16 prov-cid-cia-Ukraine-27"
+    },
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-59"
+    },
+    {
+      "-country": "RO",
+      "-province": "prov-cid-cia-Romania-16 prov-cid-cia-Romania-39"
+    },
+    {
+      "-country": "TR",
+      "-province": "prov-cid-cia-Turkey-10 prov-cid-cia-Turkey-18 prov-cid-cia-Turkey-32 prov-cid-cia-Turkey-38 prov-cid-cia-Turkey-43 prov-cid-cia-Turkey-46 prov-cid-cia-Turkey-48 prov-cid-cia-Turkey-58 prov-cid-cia-Turkey-59 prov-cid-cia-Turkey-60 prov-cid-cia-Turkey-61 prov-cid-cia-Turkey-64 prov-cid-cia-Turkey-69 prov-cid-cia-Turkey-74"
+    }
+  ],
+  "depth": "2211"
+}
+{
+  "-id": "sea-Asowsches_Meer",
+  "-country": "UA R",
+  "-bordering": "sea-Schwarzes_Meer",
+  "name": "Sea of Azov",
+  "located": [
+    {
+      "-country": "UA",
+      "-province": "prov-cid-cia-Ukraine-6 prov-cid-cia-Ukraine-9 prov-cid-cia-Ukraine-24 prov-cid-cia-Ukraine-27"
+    },
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-45 prov-cid-cia-Russia-59"
+    }
+  ],
+  "depth": "100"
+}
+{
+  "-id": "sea-Ostsee",
+  "-country": "D LV LT PL R DK EW SF S",
+  "-bordering": "sea-Nordsee",
+  "name": "Baltic Sea",
+  "located": [
+    {
+      "-country": "D",
+      "-province": "prov-cid-cia-Germany-16 prov-cid-cia-Germany-9"
+    },
+    {
+      "-country": "PL",
+      "-province": "prov-cid-cia-Poland-10 prov-cid-cia-Poland-11 prov-cid-cia-Poland-18 prov-cid-cia-Poland-40 prov-cid-cia-Poland-42"
+    },
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-11 prov-cid-cia-Russia-10 prov-cid-cia-Russia-9"
+    },
+    {
+      "-country": "SF",
+      "-province": "lteil-ALA-SF lteil-KYM-SF lteil-LAP-SF lteil-OUL-SF lteil-TUP-SF lteil-UUS-SF lteil-VAA-SF"
+    },
+    {
+      "-country": "S",
+      "-province": "prov-cid-cia-Sweden-3 prov-cid-cia-Sweden-4 prov-cid-cia-Sweden-6 prov-cid-cia-Sweden-10 prov-cid-cia-Sweden-12 prov-cid-cia-Sweden-14 prov-cid-cia-Sweden-15 prov-cid-cia-Sweden-17 prov-cid-cia-Sweden-19 prov-cid-cia-Sweden-20 prov-cid-cia-Sweden-21 prov-cid-cia-Sweden-23 prov-cid-cia-Sweden-24"
+    }
+  ],
+  "depth": "459"
+}
+{
+  "-id": "sea-Atlantic",
+  "-country": "F E GBZ IS IRL GB P AG BS BDS CDN USA C WD DOM RH WG KN WL WV TT RA BR RCH ROU FGU GUY SME YV RIM MA WSA ANG RCB NAM ZRE BEN WAN RT RSA CI GH CAM GQ G CV RG LB WAG SN GNB WAL NA SMAR STP PR AXA GUAD MART BVIR MNTS VIRG HELX FALK SPMI BERM TUCA SVAX GROX FARX",
+  "-bordering": "sea-Nordsee sea-Channel sea-Mittelmeer sea-Golf_von_Mexiko sea-Nordpolarmeer sea-Indic sea-Europaeisches_Nordmeer sea-Pacific sea-Irische_See sea-Caribbean",
+  "name": "Atlantic Ocean",
+  "located": [
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-5 prov-cid-cia-France-20 prov-cid-cia-France-89 prov-cid-cia-France-99"
+    },
+    {
+      "-country": "E",
+      "-province": "prov-cid-cia-Spain-2 prov-cid-cia-Spain-13  prov-cid-cia-Spain-8  prov-cid-cia-Spain-7 prov-cid-cia-Spain-6  prov-cid-cia-Spain-4"
+    },
+    {
+      "-country": "P",
+      "-province": "prov-cid-cia-Portugal-2 prov-cid-cia-Portugal-3 prov-cid-cia-Portugal-4 prov-cid-cia-Portugal-7 prov-cid-cia-Portugal-9 prov-cid-cia-Portugal-11 prov-cid-cia-Portugal-12 prov-cid-cia-Portugal-14 prov-cid-cia-Portugal-16 prov-cid-cia-Portugal-17 prov-cid-cia-Portugal-21 prov-cid-cia-Portugal-22"
+    },
+    {
+      "-country": "GB",
+      "-province": "prov-cid-cia-United-Kingdom-9 prov-cid-cia-United-Kingdom-12 prov-cid-cia-United-Kingdom-85 prov-cid-cia-United-Kingdom-87"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-4 prov-cid-cia-Canada-5 prov-cid-cia-Canada-6 prov-cid-cia-Canada-7 prov-cid-cia-Canada-8 prov-cid-cia-Canada-9 prov-cid-cia-Canada-10 prov-cid-cia-Canada-11"
+    },
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-8 prov-cid-cia-United-States-9 prov-cid-cia-United-States-10 prov-cid-cia-United-States-11 prov-cid-cia-United-States-20 prov-cid-cia-United-States-21 prov-cid-cia-United-States-22 prov-cid-cia-United-States-30 prov-cid-cia-United-States-31 prov-cid-cia-United-States-33 prov-cid-cia-United-States-34 prov-cid-cia-United-States-40 prov-cid-cia-United-States-41 prov-cid-cia-United-States-47"
+    },
+    {
+      "-country": "C",
+      "-province": "prov-cid-cia-Cuba-3 prov-cid-cia-Cuba-7 prov-cid-cia-Cuba-8 prov-cid-cia-Cuba-9 prov-cid-cia-Cuba-10 prov-cid-cia-Cuba-11 prov-cid-cia-Cuba-13 prov-cid-cia-Cuba-15"
+    },
+    {
+      "-country": "RA",
+      "-province": "prov-cid-cia-Argentina-2 prov-cid-cia-Argentina-5 prov-cid-cia-Argentina-16 prov-cid-cia-Argentina-20 prov-cid-cia-Argentina-23"
+    },
+    {
+      "-country": "BR",
+      "-province": "prov-cid-cia-Brazil-3 prov-cid-cia-Brazil-4 prov-cid-cia-Brazil-6 prov-cid-cia-Brazil-7 prov-cid-cia-Brazil-8 prov-cid-cia-Brazil-10 prov-cid-cia-Brazil-14 prov-cid-cia-Brazil-15 prov-cid-cia-Brazil-16 prov-cid-cia-Brazil-17 prov-cid-cia-Brazil-19 prov-cid-cia-Brazil-20 prov-cid-cia-Brazil-21 prov-cid-cia-Brazil-24 prov-cid-cia-Brazil-25 prov-cid-cia-Brazil-26"
+    },
+    {
+      "-country": "YV",
+      "-province": "prov-cid-cia-Venezuela-10"
+    },
+    {
+      "-country": "ANG",
+      "-province": "prov-cid-cia-Angola-2 prov-cid-cia-Angola-3 prov-cid-cia-Angola-5 prov-cid-cia-Angola-7 prov-cid-cia-Angola-11 prov-cid-cia-Angola-16 prov-cid-cia-Angola-18"
+    },
+    {
+      "-country": "ZRE",
+      "-province": "prov-cid-cia-Zaire-3"
+    },
+    {
+      "-country": "RSA",
+      "-province": "prov-cid-cia-South-Africa-10"
+    },
+    {
+      "-country": "CAM",
+      "-province": "prov-cid-cia-Cameroon-3 prov-cid-cia-Cameroon-6 prov-cid-cia-Cameroon-11"
+    },
+    {
+      "-country": "SN",
+      "-province": "prov-cid-cia-Senegal-2 prov-cid-cia-Senegal-4 prov-cid-cia-Senegal-7 prov-cid-cia-Senegal-8 prov-cid-cia-Senegal-10 prov-cid-cia-Senegal-11"
+    }
+  ],
+  "depth": "9219"
+}
+{
+  "-id": "sea-Mittelmeer",
+  "-country": "AL GR MNE HR BIH SLO F E I TR MC M CY IL GAZA ET SYR RL DZ LAR MA TN GBZ",
+  "-bordering": "sea-Schwarzes_Meer sea-Atlantic",
+  "name": "Mediterranean Sea",
+  "located": [
+    {
+      "-country": "F",
+      "-province": "prov-cid-cia-France-42 prov-cid-cia-France-62 prov-cid-cia-France-104"
+    },
+    {
+      "-country": "E",
+      "-province": "prov-cid-cia-Spain-2 prov-cid-cia-Spain-5 prov-cid-cia-Spain-11 prov-cid-cia-Spain-15 prov-cid-cia-Spain-18"
+    },
+    {
+      "-country": "I",
+      "-province": "prov-cid-cia-Italy-23 prov-cid-cia-Italy-31 prov-cid-cia-Italy-36 prov-cid-cia-Italy-41 prov-cid-cia-Italy-50 prov-cid-cia-Italy-63 prov-cid-cia-Italy-68 prov-cid-cia-Italy-74 prov-cid-cia-Italy-82 prov-cid-cia-Italy-88 prov-cid-cia-Italy-97 prov-cid-cia-Italy-101 prov-cid-cia-Italy-111"
+    },
+    {
+      "-country": "GR",
+      "-province": "prov-cid-cia-Greece-2 prov-cid-cia-Greece-3 prov-cid-cia-Greece-4 prov-cid-cia-Greece-6 prov-cid-cia-Greece-7 prov-cid-cia-Greece-8 prov-cid-cia-Greece-9 prov-cid-cia-Greece-10 prov-cid-cia-Greece-11 prov-cid-cia-Greece-12 prov-cid-cia-Greece-13 prov-cid-cia-Greece-14"
+    },
+    {
+      "-country": "TR",
+      "-province": "prov-cid-cia-Turkey-2 prov-cid-cia-Turkey-9 prov-cid-cia-Turkey-11 prov-cid-cia-Turkey-12 prov-cid-cia-Turkey-21 prov-cid-cia-Turkey-35 prov-cid-cia-Turkey-36 prov-cid-cia-Turkey-39 prov-cid-cia-Turkey-54"
+    },
+    {
+      "-country": "IL",
+      "-province": "prov-cid-cia-Israel-2 prov-cid-cia-Israel-3 prov-cid-cia-Israel-5 prov-cid-cia-Israel-6 prov-cid-cia-Israel-7"
+    },
+    {
+      "-country": "ET",
+      "-province": "prov-cid-cia-Egypt-4 prov-cid-cia-Egypt-9 prov-cid-cia-Egypt-29 prov-cid-cia-Egypt-14 prov-cid-cia-Egypt-10 prov-cid-cia-Egypt-11 prov-cid-cia-Egypt-17 prov-cid-cia-Egypt-13 prov-cid-cia-Egypt-28 prov-cid-cia-Egypt-6"
+    }
+  ],
+  "depth": "5121"
+}
+{
+  "-id": "sea-Golf_von_Mexiko",
+  "-country": "MEX USA C",
+  "-bordering": "sea-Atlantic sea-Caribbean",
+  "name": "Gulf of Mexico",
+  "located": [
+    {
+      "-country": "MEX",
+      "-province": "prov-cid-cia-Mexico-5 prov-cid-cia-Mexico-27 prov-cid-cia-Mexico-28 prov-cid-cia-Mexico-30 prov-cid-cia-Mexico-31"
+    },
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-2 prov-cid-cia-United-States-10 prov-cid-cia-United-States-19 prov-cid-cia-United-States-25 prov-cid-cia-United-States-44"
+    },
+    {
+      "-country": "C",
+      "-province": "prov-cid-cia-Cuba-5 prov-cid-cia-Cuba-9 prov-cid-cia-Cuba-12"
+    }
+  ],
+  "depth": "4375"
+}
+{
+  "-id": "sea-Europaeisches_Nordmeer",
+  "-country": "N IS FARX SVAX GROX GB",
+  "-bordering": "sea-Nordsee sea-Atlantic sea-Nordpolarmeer",
+  "name": "Norwegian Sea",
+  "located": [
+    {
+      "-country": "N",
+      "-province": "lteil-SF-N lteil-MR-N lteil-ST-N lteil-NT-N lteil-NO-N lteil-TR-N lteil-FI-N"
+    },
+    {
+      "-country": "GB",
+      "-province": "prov-cid-cia-United-Kingdom-88"
+    }
+  ],
+  "depth": "3860"
+}
+{
+  "-id": "sea-Barentssee",
+  "-country": "N R",
+  "-bordering": "sea-Atlantic sea-Europaeisches_Nordmeer sea-Nordpolarmeer sea-SibirianSea",
+  "name": "Barents Sea",
+  "located": [
+    {
+      "-country": "N",
+      "-province": "lteil-FI-N"
+    },
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-8 prov-cid-cia-Russia-5 prov-cid-cia-Russia-3"
+    }
+  ],
+  "depth": "32"
+}
+{
+  "-id": "sea-SibirianSea",
+  "-country": "R",
+  "-bordering": "sea-Nordpolarmeer sea-Barentssee",
+  "name": "Sibirian Sea",
+  "located": {
+    "-country": "R",
+    "-province": "prov-cid-cia-Russia-5 prov-cid-cia-Russia-70 prov-cid-cia-Russia-75 prov-cid-cia-Russia-81 prov-cid-cia-Russia-89"
+  },
+  "depth": "540"
+}
+{
+  "-id": "sea-Nordpolarmeer",
+  "-country": "R CDN USA SVAX GROX",
+  "-bordering": "sea-Atlantic sea-Europaeisches_Nordmeer sea-BeringSea sea-Barentssee sea-SibirianSea",
+  "name": "Arctic Ocean",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-89"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-7 prov-cid-cia-Canada-13"
+    },
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3"
+    }
+  ],
+  "depth": "5608"
+}
+{
+  "-id": "sea-BeringSea",
+  "-country": "R USA",
+  "-bordering": "sea-Nordpolarmeer sea-Pacific",
+  "name": "Bering Sea",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-89 prov-cid-cia-Russia-87"
+    },
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3"
+    }
+  ],
+  "depth": "4096"
+}
+{
+  "-id": "sea-Pacific",
+  "-country": "R J MV RP RC GCA MEX CDN USA CR NIC PA ES HCA CO AUS FJI KIR MH FSM NAU NZ PAL SLB TO TUV VU WS RCH PE EC RI PNG NCA NIUE NMIS PITC WAFU AMSA COOK FPOL GUAM NORF",
+  "-bordering": "sea-Atlantic sea-BeringSea sea-OchotskSea sea-EastChinaSea sea-SouthChinaSea sea-Japanisches_Meer sea-Sundasee sea-Celebessee sea-Indic",
+  "name": "Pacific Ocean",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-89 prov-cid-cia-Russia-87 prov-cid-cia-Russia-85 prov-cid-cia-Russia-90"
+    },
+    {
+      "-country": "J",
+      "-province": "prov-cid-cia-Japan-3 prov-cid-cia-Japan-5 prov-cid-cia-Japan-6 prov-cid-cia-Japan-7 prov-cid-cia-Japan-10 prov-cid-cia-Japan-12 prov-cid-cia-Japan-16 prov-cid-cia-Japan-18 prov-cid-cia-Japan-27 prov-cid-cia-Japan-28 prov-cid-cia-Japan-30 prov-cid-cia-Japan-36 prov-cid-cia-Japan-44 prov-cid-cia-Japan-47 prov-cid-cia-Japan-54 prov-cid-cia-Japan-55 prov-cid-cia-Japan-56"
+    },
+    {
+      "-country": "MEX",
+      "-province": "prov-cid-cia-Mexico-3 prov-cid-cia-Mexico-4 prov-cid-cia-Mexico-6 prov-cid-cia-Mexico-9 prov-cid-cia-Mexico-12 prov-cid-cia-Mexico-14 prov-cid-cia-Mexico-16 prov-cid-cia-Mexico-18 prov-cid-cia-Mexico-20 prov-cid-cia-Mexico-25 prov-cid-cia-Mexico-26"
+    },
+    {
+      "-country": "CDN",
+      "-province": "prov-cid-cia-Canada-3 prov-cid-cia-Canada-13"
+    },
+    {
+      "-country": "USA",
+      "-province": "prov-cid-cia-United-States-3 prov-cid-cia-United-States-6 prov-cid-cia-United-States-12 prov-cid-cia-United-States-38 prov-cid-cia-United-States-48"
+    },
+    {
+      "-country": "CR",
+      "-province": "prov-cid-cia-Costa-Rica-6 prov-cid-cia-Costa-Rica-7"
+    },
+    {
+      "-country": "PA",
+      "-province": "prov-cid-cia-Panama-3 prov-cid-cia-Panama-6 prov-cid-cia-Panama-7 prov-cid-cia-Panama-8 prov-cid-cia-Panama-9 prov-cid-cia-Panama-10 prov-cid-cia-Panama-11"
+    },
+    {
+      "-country": "HCA",
+      "-province": "prov-cid-cia-Honduras-7 prov-cid-cia-Honduras-18"
+    },
+    {
+      "-country": "CO",
+      "-province": "prov-cid-cia-Colombia-11 prov-cid-cia-Colombia-13 prov-cid-cia-Colombia-22 prov-cid-cia-Colombia-32"
+    },
+    {
+      "-country": "AUS",
+      "-province": "prov-cid-cia-Australia-2 prov-cid-cia-Australia-4 prov-cid-cia-Australia-7  prov-cid-cia-Australia-6"
+    },
+    {
+      "-country": "PE",
+      "-province": "prov-cid-cia-Peru-3 prov-cid-cia-Peru-5 prov-cid-cia-Peru-12 prov-cid-cia-Peru-14 prov-cid-cia-Peru-15 prov-cid-cia-Peru-16 prov-cid-cia-Peru-20 prov-cid-cia-Peru-22 prov-cid-cia-Peru-25 prov-cid-cia-Peru-26"
+    }
+  ],
+  "depth": "11034"
+}
+{
+  "-id": "sea-OchotskSea",
+  "-country": "R J",
+  "-bordering": "sea-Pacific sea-Japanisches_Meer",
+  "name": "Sea of Okhotsk",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-89 prov-cid-cia-Russia-87"
+    },
+    {
+      "-country": "J",
+      "-province": "prov-cid-cia-Japan-3"
+    }
+  ],
+  "depth": "3372"
+}
+{
+  "-id": "sea-Japanisches_Meer",
+  "-country": "R J ROK NOK",
+  "-bordering": "sea-Pacific sea-Gelbes_Meer sea-EastChinaSea sea-OchotskSea",
+  "name": "Sea of Japan",
+  "located": [
+    {
+      "-country": "R",
+      "-province": "prov-cid-cia-Russia-84 prov-cid-cia-Russia-85 prov-cid-cia-Russia-90"
+    },
+    {
+      "-country": "J",
+      "-province": "prov-cid-cia-Japan-3 prov-cid-cia-Japan-5 prov-cid-cia-Japan-8 prov-cid-cia-Japan-9 prov-cid-cia-Japan-20 prov-cid-cia-Japan-21 prov-cid-cia-Japan-22 prov-cid-cia-Japan-23 prov-cid-cia-Japan-32 prov-cid-cia-Japan-34 prov-cid-cia-Japan-38 prov-cid-cia-Japan-39 prov-cid-cia-Japan-42 prov-cid-cia-Japan-49 prov-cid-cia-Japan-50"
+    }
+  ],
+  "depth": "4036"
+}
+{
+  "-id": "sea-Gelbes_Meer",
+  "-country": "TJ ROK NOK",
+  "-bordering": "sea-EastChinaSea sea-Japanisches_Meer",
+  "name": "Yellow Sea",
+  "located": {
+    "-country": "TJ",
+    "-province": "prov-cid-cia-China-8 prov-cid-cia-China-13 prov-cid-cia-China-16 prov-cid-cia-China-19"
+  },
+  "depth": "200"
+}
+{
+  "-id": "sea-EastChinaSea",
+  "-country": "TJ RC ROK J",
+  "-bordering": "sea-Pacific sea-Gelbes_Meer sea-SouthChinaSea sea-Japanisches_Meer",
+  "name": "East China Sea",
+  "located": [
+    {
+      "-country": "TJ",
+      "-province": "prov-cid-cia-China-3 prov-cid-cia-China-13 prov-cid-cia-China-23"
+    },
+    {
+      "-country": "J",
+      "-province": "prov-cid-cia-Japan-50 prov-cid-cia-Japan-51 prov-cid-cia-Japan-52 prov-cid-cia-Japan-55 prov-cid-cia-Japan-56"
+    }
+  ],
+  "depth": "2370"
+}
+{
+  "-id": "sea-SouthChinaSea",
+  "-country": "TJ MAL THA K VN BRU SGP RI RP RC HONX MACX",
+  "-bordering": "sea-Pacific sea-EastChinaSea sea-Sundasee sea-Celebessee sea-MalakkaStrait",
+  "name": "South China Sea",
+  "located": [
+    {
+      "-country": "TJ",
+      "-province": "prov-cid-cia-China-3 prov-cid-cia-China-5 prov-cid-cia-China-25"
+    },
+    {
+      "-country": "MAL",
+      "-province": "prov-cid-cia-Malaysia-2 prov-cid-cia-Malaysia-4 prov-cid-cia-Malaysia-7 prov-cid-cia-Malaysia-11 prov-cid-cia-Malaysia-12 prov-cid-cia-Malaysia-14"
+    }
+  ],
+  "depth": "5420"
+}
+{
+  "-id": "sea-Persischer_Golf",
+  "-country": "IR BRN IRQ KWT SA OM UAE Q",
+  "-bordering": "sea-Arabisches_Meer",
+  "name": "Persian Gulf",
+  "located": [
+    {
+      "-country": "IR",
+      "-province": "prov-cid-cia-Iran-6 prov-cid-cia-Iran-12 prov-cid-cia-Iran-16"
+    },
+    {
+      "-country": "IRQ",
+      "-province": "prov-cid-cia-Iraq-5"
+    },
+    {
+      "-country": "UAE",
+      "-province": "prov-cid-cia-United-Arab-Emirates-2 prov-cid-cia-United-Arab-Emirates-3 prov-cid-cia-United-Arab-Emirates-4 prov-cid-cia-United-Arab-Emirates-6 prov-cid-cia-United-Arab-Emirates-7 prov-cid-cia-United-Arab-Emirates-8"
+    }
+  ],
+  "depth": "102"
+}
+{
+  "-id": "sea-Irische_See",
+  "-country": "IRL GB GBM",
+  "-bordering": "sea-Atlantic",
+  "name": "Irish Sea",
+  "located": {
+    "-country": "GB",
+    "-province": "prov-cid-cia-United-Kingdom-26 prov-cid-cia-United-Kingdom-29 prov-cid-cia-United-Kingdom-63 prov-cid-cia-United-Kingdom-67 prov-cid-cia-United-Kingdom-80 prov-cid-cia-United-Kingdom-85"
+  },
+  "depth": "272"
+}
+{
+  "-id": "sea-Caribbean",
+  "-country": "AG BZ GCA MEX CR NIC PA C WD DOM RH HCA WG JA CO KN WL WV YV PR SMAR AXA GUAD MART BVIR MNTS VIRG TT NA ARU CAYM",
+  "-bordering": "sea-Atlantic sea-Golf_von_Mexiko",
+  "name": "Caribbean Sea",
+  "located": [
+    {
+      "-country": "MEX",
+      "-province": "prov-cid-cia-Mexico-23 prov-cid-cia-Mexico-31"
+    },
+    {
+      "-country": "CR",
+      "-province": "prov-cid-cia-Costa-Rica-8"
+    },
+    {
+      "-country": "PA",
+      "-province": "prov-cid-cia-Panama-2 prov-cid-cia-Panama-4 prov-cid-cia-Panama-5"
+    },
+    {
+      "-country": "C",
+      "-province": "prov-cid-cia-Cuba-3 prov-cid-cia-Cuba-4 prov-cid-cia-Cuba-6 prov-cid-cia-Cuba-7 prov-cid-cia-Cuba-9 prov-cid-cia-Cuba-10 prov-cid-cia-Cuba-11 prov-cid-cia-Cuba-12 prov-cid-cia-Cuba-13 prov-cid-cia-Cuba-14 prov-cid-cia-Cuba-16"
+    },
+    {
+      "-country": "HCA",
+      "-province": "prov-cid-cia-Honduras-2 prov-cid-cia-Honduras-3 prov-cid-cia-Honduras-6 prov-cid-cia-Honduras-10"
+    },
+    {
+      "-country": "CO",
+      "-province": "prov-cid-cia-Colombia-3 prov-cid-cia-Colombia-5 prov-cid-cia-Colombia-13 prov-cid-cia-Colombia-14 prov-cid-cia-Colombia-17 prov-cid-cia-Colombia-20 prov-cid-cia-Colombia-30"
+    },
+    {
+      "-country": "YV",
+      "-province": "prov-cid-cia-Venezuela-3 prov-cid-cia-Venezuela-5 prov-cid-cia-Venezuela-8 prov-cid-cia-Venezuela-10 prov-cid-cia-Venezuela-11 prov-cid-cia-Venezuela-15 prov-cid-cia-Venezuela-16 prov-cid-cia-Venezuela-17 prov-cid-cia-Venezuela-19 prov-cid-cia-Venezuela-23 prov-cid-cia-Venezuela-25"
+    }
+  ],
+  "depth": "7240"
+}
+{
+  "-id": "sea-Indic",
+  "-country": "PK IND RI SGP CL AUS RSA EAT COM YE SP EAK RM MOC MS SY XMAS COCO MV MAYO REUN TL",
+  "-bordering": "sea-Atlantic sea-Pacific sea-Golf_von_Aden sea-Sundasee sea-Arabisches_Meer sea-Bengal sea-AndamanSea",
+  "name": "Indian Ocean",
+  "located": [
+    {
+      "-country": "IND",
+      "-province": "prov-cid-cia-India-12 prov-cid-cia-India-23"
+    },
+    {
+      "-country": "AUS",
+      "-province": "prov-cid-cia-Australia-2 prov-cid-cia-Australia-3 prov-cid-cia-Australia-5 prov-cid-cia-Australia-6 prov-cid-cia-Australia-7 prov-cid-cia-Australia-8"
+    },
+    {
+      "-country": "RSA",
+      "-province": "prov-cid-cia-South-Africa-2 prov-cid-cia-South-Africa-5 prov-cid-cia-South-Africa-10"
+    },
+    {
+      "-country": "EAT",
+      "-province": "prov-cid-cia-Tanzania-5 prov-cid-cia-Tanzania-7 prov-cid-cia-Tanzania-8 prov-cid-cia-Tanzania-9 prov-cid-cia-Tanzania-10"
+    },
+    {
+      "-country": "EAK",
+      "-province": "prov-cid-cia-Kenya-3"
+    },
+    {
+      "-country": "RM",
+      "-province": "prov-cid-cia-Madagascar-2 prov-cid-cia-Madagascar-3 prov-cid-cia-Madagascar-4 prov-cid-cia-Madagascar-5 prov-cid-cia-Madagascar-6 prov-cid-cia-Madagascar-7"
+    },
+    {
+      "-country": "MOC",
+      "-province": "prov-cid-cia-Mozambique-2 prov-cid-cia-Mozambique-3 prov-cid-cia-Mozambique-4 prov-cid-cia-Mozambique-6 prov-cid-cia-Mozambique-7 prov-cid-cia-Mozambique-9 prov-cid-cia-Mozambique-11 prov-cid-cia-Mozambique-13"
+    }
+  ],
+  "depth": "6400"
+}
+{
+  "-id": "sea-Arabisches_Meer",
+  "-country": "PK OM IND",
+  "-bordering": "sea-Indic sea-GulfOman sea-Golf_von_Aden",
+  "name": "Arabian Sea",
+  "located": {
+    "-country": "IND",
+    "-province": "prov-cid-cia-India-6 prov-cid-cia-India-7 prov-cid-cia-India-11 prov-cid-cia-India-12 prov-cid-cia-India-14 prov-cid-cia-India-23 prov-cid-cia-India-33"
+  },
+  "depth": "5203"
+}
+{
+  "-id": "sea-GulfOman",
+  "-country": "IR PK OM UAE",
+  "-bordering": "sea-Indic sea-Persischer_Golf sea-Golf_von_Aden",
+  "name": "Gulf of Oman",
+  "located": [
+    {
+      "-country": "IR",
+      "-province": "prov-cid-cia-Iran-12 prov-cid-cia-Iran-22"
+    },
+    {
+      "-country": "UAE",
+      "-province": "prov-cid-cia-United-Arab-Emirates-5"
+    }
+  ],
+  "depth": "3350"
+}
+{
+  "-id": "sea-Rotes_Meer",
+  "-country": "IL ET JOR SA YE SUD DJI ER",
+  "-bordering": "sea-Golf_von_Aden",
+  "name": "Red Sea",
+  "located": [
+    {
+      "-country": "IL",
+      "-province": "prov-cid-cia-Israel-7"
+    },
+    {
+      "-country": "ET",
+      "-province": "prov-cid-cia-Egypt-3 prov-cid-cia-Egypt-5 prov-cid-cia-Egypt-31"
+    },
+    {
+      "-country": "SUD",
+      "-province": "prov-cid-cia-Sudan-9"
+    }
+  ],
+  "depth": "2635"
+}
+{
+  "-id": "sea-Golf_von_Aden",
+  "-country": "DJI SP YE",
+  "-bordering": "sea-Indic sea-Arabisches_Meer sea-Rotes_Meer",
+  "name": "Gulf of Aden",
+  "depth": "5143"
+}
+{
+  "-id": "sea-Bengal",
+  "-country": "BD MYA IND CL",
+  "-bordering": "sea-Atlantic sea-Pacific sea-Golf_von_Aden sea-Sundasee sea-Arabisches_Meer sea-AndamanSea",
+  "name": "Gulf of Bengal",
+  "located": [
+    {
+      "-country": "MYA",
+      "-province": "prov-cid-cia-Burma-2 prov-cid-cia-Burma-14"
+    },
+    {
+      "-country": "IND",
+      "-province": "prov-cid-cia-India-2 prov-cid-cia-India-19 prov-cid-cia-India-26 prov-cid-cia-India-28"
+    }
+  ],
+  "depth": "4045"
+}
+{
+  "-id": "sea-AndamanSea",
+  "-country": "MYA IND THA RI",
+  "-bordering": "sea-Indic sea-Bengal sea-MalakkaStrait",
+  "name": "Andaman Sea",
+  "located": [
+    {
+      "-country": "MYA",
+      "-province": "prov-cid-cia-Burma-2 prov-cid-cia-Burma-5 prov-cid-cia-Burma-6 prov-cid-cia-Burma-13"
+    },
+    {
+      "-country": "IND",
+      "-province": "prov-cid-cia-India-28"
+    }
+  ],
+  "depth": "3113"
+}
+{
+  "-id": "sea-MalakkaStrait",
+  "-country": "THA RI MAL SGP",
+  "-bordering": "sea-AndamanSea sea-SouthChinaSea",
+  "name": "Malakka Strait",
+  "located": {
+    "-country": "MAL",
+    "-province": "prov-cid-cia-Malaysia-3 prov-cid-cia-Malaysia-9 prov-cid-cia-Malaysia-10 prov-cid-cia-Malaysia-8 prov-cid-cia-Malaysia-13 prov-cid-cia-Malaysia-6 prov-cid-cia-Malaysia-5  prov-cid-cia-Malaysia-2"
+  },
+  "depth": "104"
+}
+{
+  "-id": "sea-Sundasee",
+  "-country": "RI TL",
+  "-bordering": "sea-Indic sea-Pacific sea-Celebessee sea-SouthChinaSea",
+  "name": "Sunda Sea",
+  "depth": "7440"
+}
+{
+  "-id": "sea-Celebessee",
+  "-country": "RI MAL RP",
+  "-bordering": "sea-Sundasee sea-Pacific sea-SouthChinaSea",
+  "name": "Sulawesi Sea",
+  "located": {
+    "-country": "MAL",
+    "-province": "prov-cid-cia-Malaysia-12"
+  },
+  "depth": "6218"
+}
diff --git a/asterixdb/asterix-app/data/fulltext/cloudberry_sample_tweet.adm b/asterixdb/asterix-app/data/fulltext/cloudberry_sample_tweet.adm
index 9e501ba..434b005 100644
--- a/asterixdb/asterix-app/data/fulltext/cloudberry_sample_tweet.adm
+++ b/asterixdb/asterix-app/data/fulltext/cloudberry_sample_tweet.adm
@@ -1,100 +1,15000 @@
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640186101761, "text": "Just posted a photo @ Campus Martius Park https://t.co/5Ax4E2CdWZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.04647491,42.33170228"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48121888, "name": "Kevin McKague", "screen_name": "KevinOfMI", "lang": "en", "location": "Davison, Michigan", "create_at": date("2009-06-17"), "description": "I need to ride my bike until my brain shuts up and my muscles are screaming. \nRight after these donuts. Dad of 3.\n Visit my blog 18 Wheels and a 12-Speed Bike.", "followers_count": 1178, "friends_count": 1780, "statues_count": 22263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640370606080, "text": "These niggas next to me do not know how to stfu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2401270118, "name": "sad goon", "screen_name": "Yung_Yadig", "lang": "en", "location": "null", "create_at": date("2014-03-21"), "description": "don't follow me", "followers_count": 112, "friends_count": 224, "statues_count": 7778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downers Grove, IL", "id": "6af99a29bfae42a2", "name": "Downers Grove", "place_type": "city", "bounding_box": rectangle("-88.096689,41.744098 -87.983315,41.83907") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1720591, "cityName": "Downers Grove" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640471195652, "text": "Who wants to got see @minnesotaBASS with me at Beta this weekend?!?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 272486819 }}, "user": { "id": 333882909, "name": "Hannah Montana", "screen_name": "Oootentog", "lang": "en", "location": "Denver, CO", "create_at": date("2011-07-12"), "description": "GOD. Mile High City. ❤️ Miley Cyrus is my spirit animal. I really really really really really really like cats.", "followers_count": 1254, "friends_count": 474, "statues_count": 48340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640508911616, "text": "I know that I let you down. Is it too late now to say sorry?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285176507, "name": "NINA XELYN", "screen_name": "nxelyn", "lang": "en", "location": "Norway/CA", "create_at": date("2011-04-20"), "description": "sc: nxelyn | gemini | sjsu", "followers_count": 232, "friends_count": 64, "statues_count": 1332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640630521857, "text": "When did I become a ghost?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3432951521, "name": "blyss", "screen_name": "tempurrra", "lang": "en", "location": " kissing az 2 destinys child", "create_at": date("2015-08-20"), "description": "BEAT MY ASS OR SHUT UP", "followers_count": 699, "friends_count": 457, "statues_count": 10229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945640819449857, "text": "@misscoorslight @NASCAR_XFINITY @NASCAR_Trucks You look Beautiful Amanda, Enjoy yourself !", "in_reply_to_status": 668923103632838656, "in_reply_to_user": 2484598129, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2484598129, 20847350, 17105161 }}, "user": { "id": 2476720466, "name": "Mark Antczak", "screen_name": "Mark_751Ant", "lang": "en", "location": "New Jersey", "create_at": date("2014-05-04"), "description": "Love Nascar, Jeff Gordon #1, Yankees, Giants, Rangers, Weather, Love Abc7 News, Wish i was a Race Car Driver (Sprint Cup), WWJD.", "followers_count": 75, "friends_count": 695, "statues_count": 9159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayonne, NJ", "id": "3d3c56338b6a3b4a", "name": "Bayonne", "place_type": "city", "bounding_box": rectangle("-74.146932,40.643773 -74.0658,40.697794") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3403580, "cityName": "Bayonne" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945641045893120, "text": "Rutgers gets Wake in the B1G/ACC Challenge...Demon Deacons just took down Indiana in Maui", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323008880, "name": "Matt Hladik", "screen_name": "MattHladik919", "lang": "en", "location": "201", "create_at": date("2011-06-23"), "description": "Writer/Editor at http://Collegespun.com, formerly of @rivalsnewjersey ...Rutgers alum, proud New Jerseyan and Hudson County native...jack-of-all trades", "followers_count": 2318, "friends_count": 1803, "statues_count": 19414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoboken, NJ", "id": "e9143a85705b4d40", "name": "Hoboken", "place_type": "city", "bounding_box": rectangle("-74.044085,40.7336 -74.020431,40.759159") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3432250, "cityName": "Hoboken" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945641121320961, "text": "‼️‼️‼️‼️‼️‼️ https://t.co/4MebTywHOB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1191262604, "name": "Asia✨02/03❤️‼️", "screen_name": "AsiaCarlton", "lang": "en", "location": "Houston, TX", "create_at": date("2013-02-17"), "description": "I'm A Mf'Kn Queen ❣✊ #freenick", "followers_count": 1083, "friends_count": 924, "statues_count": 6067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
-{ "create_at": datetime("2015-11-23T16:14:03.000Z"), "id": 668945641335296000, "text": "@__morgssss__ hi", "in_reply_to_status": -1, "in_reply_to_user": 31514315, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 31514315 }}, "user": { "id": 2922251677, "name": "ZainAwan", "screen_name": "zainzawan", "lang": "en", "location": "Racine, WI", "create_at": date("2014-12-07"), "description": "null", "followers_count": 640, "friends_count": 2009, "statues_count": 257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945641700110336, "text": "I really like these pics I took on Friday�� https://t.co/8ry6XF7Hfe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230562362, "name": "Angelica", "screen_name": "angieolivaresss", "lang": "en", "location": "null", "create_at": date("2010-12-25"), "description": "ig: angieolivaresss", "followers_count": 1273, "friends_count": 712, "statues_count": 39895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642115497984, "text": "Definite cough drop addict", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3207575301, "name": "Alexyss", "screen_name": "AlexyssMarie7", "lang": "en", "location": "Sneads Ferry, NC", "create_at": date("2015-04-25"), "description": ":)(:", "followers_count": 68, "friends_count": 81, "statues_count": 1850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sneads Ferry, NC", "id": "008c5aba0de5eebc", "name": "Sneads Ferry", "place_type": "city", "bounding_box": rectangle("-77.409485,34.513794 -77.360135,34.568935") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3762680, "cityName": "Sneads Ferry" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642149036032, "text": "#ledyardffa members at the IMAGE conference at Aqua Turf Club. @ Aqua Turf Club https://t.co/qTfur8kdEG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.87427231,41.57279355"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ledyardffa" }}, "user": { "id": 801273728, "name": "Bob Williams", "screen_name": "rwilliamslhs", "lang": "en", "location": "Connecticut, USA", "create_at": date("2012-09-03"), "description": "Ledyard HS Ag-Sci Teacher\r\nFormerly Habitat for Humanity International partner in the Philippines and long-time resident of General Santos City", "followers_count": 11, "friends_count": 6, "statues_count": 6 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southington, CT", "id": "000086513b2042b6", "name": "Southington", "place_type": "city", "bounding_box": rectangle("-72.944738,41.544766 -72.818407,41.653245") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642434072580, "text": "YES LADIES YES https://t.co/cPo6RuqYX4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3141836992, "name": "Janetta Crawford", "screen_name": "janettadeane", "lang": "en", "location": "Columbus, GA", "create_at": date("2015-04-06"), "description": "I've found a love greater than life itself.", "followers_count": 107, "friends_count": 239, "statues_count": 730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642459271168, "text": "Thanks a ton, @SOHFILMS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3397648810 }}, "user": { "id": 288990877, "name": "Kaplowitz", "screen_name": "iamkap", "lang": "en", "location": "PNW USA", "create_at": date("2011-04-27"), "description": "Born on Purim, but not last Purim. Cigars and Horse Racing, mostly. Will go to the dogs on occasion. Warning: spontaneous Yiddish may occur.", "followers_count": 4431, "friends_count": 84, "statues_count": 25336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642518024192, "text": "Interested in a #Hospitality #job near #Newark, DE? This could be a great fit: https://t.co/E17uOx5trj #Hiring https://t.co/u9RMCfIY0x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.7496572,39.6837226"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Newark", "Hiring" }}, "user": { "id": 2535717440, "name": "Noodles Careers", "screen_name": "NoodlesCareers", "lang": "en", "location": "In your area", "create_at": date("2014-05-30"), "description": "We're looking for people who have a passion for restaurants and a mind for business. If you're looking for a career with us, you’re in the right place!", "followers_count": 297, "friends_count": 459, "statues_count": 2964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, DE", "id": "23aaa87ea90e76a4", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-75.788669,39.6431 -75.723528,39.715087") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1050670, "cityName": "Newark" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642606096384, "text": "@yyoonseok CNGRATS", "in_reply_to_status": 668945313491628032, "in_reply_to_user": 162278729, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 162278729 }}, "user": { "id": 141626420, "name": "ST☆RLIGHT_ADRIAN", "screen_name": "softhakyeon", "lang": "en", "location": "missouri | he/him | ♊ | INFP ", "create_at": date("2010-05-08"), "description": "adrian. elf, st☆rlight. DIGIMON. i talk abt rp a lot. gender: kim heechul | i'm in vixx hell and i love it | #10yearswithsuju | FREEZE, ARMOR DOWN!", "followers_count": 292, "friends_count": 834, "statues_count": 113781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MO", "id": "fc3cabb7fafa9f8d", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-93.801877,38.343979 -93.740328,38.397202") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29083, "countyName": "Henry", "cityID": 2914986, "cityName": "Clinton" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642740449280, "text": "Omg DEACS!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37855605, "name": "Kyle Fowler", "screen_name": "kfow35", "lang": "en", "location": "San Fran", "create_at": date("2009-05-04"), "description": "Client Developer @foursquare | Washed up WFU bball player", "followers_count": 820, "friends_count": 226, "statues_count": 9126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheaton, IL", "id": "57ac88aefcd20988", "name": "Wheaton", "place_type": "city", "bounding_box": rectangle("-88.151554,41.814328 -88.074936,41.889152") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1781048, "cityName": "Wheaton" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642773995520, "text": ")((@saltlet @et137h2o #TheUnsayable sh , ,,##HiddenLanguage(of((Trauma\" shh by bb by yy Annie Rogers, Ph.D.rEER° #ampathicdetermination #shh", "in_reply_to_status": 668795712876974081, "in_reply_to_user": 15834056, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheUnsayable", "HiddenLanguage", "ampathicdetermination", "shh" }}, "user_mentions": {{ 7774932, 3988239299 }}, "user": { "id": 15834056, "name": "@mistake:.matrix.ore", "screen_name": "frets", "lang": "en", "location": "fb.com/dArEsAyGoThere", "create_at": date("2008-08-13"), "description": "∫misalias resequenci aftew cill cells, find link list stil bugs https://t.co/5DfovCtQkp baking up((#CC0)) #WWMappingDo et cent erra.° yo homo log ados reco cex.", "followers_count": 121, "friends_count": 238, "statues_count": 15057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945642782330881, "text": "*finishes one sentence within a 10-page paper* https://t.co/g5ciHfSJto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311127234, "name": "Brandon Hooks", "screen_name": "CaptainBHooks", "lang": "en", "location": "null", "create_at": date("2011-06-04"), "description": "Sibling parent | cupcake connoisseur | Hillary's wingman | Mindy Kaling's best friend | Beyoncé's godchild", "followers_count": 406, "friends_count": 367, "statues_count": 15021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643025530880, "text": "@erikgahner Nu ser jeg også at ved alle spg. om nabo i 08 er der 124 der har valgt \"Ved ikke\". Tusind tak for opklaringen.", "in_reply_to_status": 668943432606420992, "in_reply_to_user": 179502768, "favorite_count": 0, "retweet_count": 0, "lang": "da", "is_retweet": false, "user_mentions": {{ 179502768 }}, "user": { "id": 64785998, "name": "Morten Stinus", "screen_name": "mortenstinus", "lang": "en", "location": "Urbana-Champaign via Cph", "create_at": date("2009-08-11"), "description": "PhD-studerende ved UIUCs Institute of Communications Research. Jeg beskæftiger mig mest med strukturel racisme og mediefremstillinger af raciale minoriteter.", "followers_count": 317, "friends_count": 942, "statues_count": 5325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Urbana, IL", "id": "497789c72fadba82", "name": "Urbana", "place_type": "city", "bounding_box": rectangle("-88.238573,40.072763 -88.162095,40.157407") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1777005, "cityName": "Urbana" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643054870528, "text": "@Ange77H Too difficult to choose..He's damn good in all of them.", "in_reply_to_status": 668923587437461504, "in_reply_to_user": 20376964, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20376964 }}, "user": { "id": 120995032, "name": "Shadowbunny49", "screen_name": "Angelica563", "lang": "en", "location": "Chandler, AZ", "create_at": date("2010-03-07"), "description": "Therapist", "followers_count": 24, "friends_count": 21, "statues_count": 832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643168210948, "text": "I'm at Cobb Miami Lakes 17 Theatre - @cobbtheatres for The Hunger Games: Mockingjay - Part 2 in Miami Lakes, FL https://t.co/Y4VNoCClyz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.30993713,25.91395836"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34640966 }}, "user": { "id": 62131700, "name": "Charlotte Ethel", "screen_name": "breadchastick03", "lang": "en", "location": "Miami, FL", "create_at": date("2009-08-01"), "description": "Traffic Manager at @MSSmedia. #Taco Obsessed, Lover of #Dinos & Human to Retired #Greyhound. Dislikes Babies & Puppies. Terrified of Pregnant Women & Spiders.", "followers_count": 488, "friends_count": 491, "statues_count": 16494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Lakes, FL", "id": "d286131d0c030d06", "name": "Miami Lakes", "place_type": "city", "bounding_box": rectangle("-80.351414,25.897971 -80.291808,25.927695") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245100, "cityName": "Miami Lakes" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643285581824, "text": "@OfficialDOGNews \nDo you have a selection?", "in_reply_to_status": 668945288695037952, "in_reply_to_user": 87409245, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 87409245 }}, "user": { "id": 428024537, "name": "Lori", "screen_name": "LoriB59", "lang": "en", "location": "Aliante North Las Vegas, NV", "create_at": date("2011-12-03"), "description": "Been here for 2 years now and loving it", "followers_count": 86, "friends_count": 289, "statues_count": 4541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643398938624, "text": "Does anybody wanna bring me circus peanuts and a slim jim.?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1699065727, "name": "The Babe Slayer", "screen_name": "CoolBlueStrat", "lang": "en", "location": "null", "create_at": date("2013-08-25"), "description": "I fart in crowded elevators", "followers_count": 497, "friends_count": 225, "statues_count": 11711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643742851072, "text": "@annnabby_4 it's a dick", "in_reply_to_status": 668924231992909824, "in_reply_to_user": 351816036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351816036 }}, "user": { "id": 762815268, "name": "lola glitzthong", "screen_name": "pmarr5041", "lang": "en", "location": "just chillin, in cedar rapids", "create_at": date("2012-08-16"), "description": "sju18", "followers_count": 277, "friends_count": 385, "statues_count": 8602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643834994688, "text": "@Legaspii5 true", "in_reply_to_status": 668943089872932864, "in_reply_to_user": 1101428090, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1101428090 }}, "user": { "id": 1016751848, "name": "DavidFlores.", "screen_name": "Deeeee123", "lang": "en", "location": "null", "create_at": date("2012-12-16"), "description": "WHERE MY DOGS AT?", "followers_count": 932, "friends_count": 568, "statues_count": 12000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945643952406528, "text": "lmfaoooooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604349708, "name": "siekina.", "screen_name": "kinasaafi", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "#missmewiddit", "followers_count": 1136, "friends_count": 897, "statues_count": 17927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euless, TX", "id": "5e2d6c67e728cca5", "name": "Euless", "place_type": "city", "bounding_box": rectangle("-97.15606,32.81088 -97.032953,32.881593") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4824768, "cityName": "Euless" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644027969536, "text": "If my baby ever need me I come speeding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110187297, "name": "Yung Zay™", "screen_name": "DaTalkOfFame", "lang": "en", "location": "MO CITY TX, Ridgemont 4", "create_at": date("2010-01-31"), "description": "Rich Is Gangsta #FAH #H52ver #HouGlo", "followers_count": 773, "friends_count": 570, "statues_count": 68608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Land, TX", "id": "7a41192a2879ee24", "name": "Sugar Land", "place_type": "city", "bounding_box": rectangle("-95.686106,29.543372 -95.577273,29.663556") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4870808, "cityName": "Sugar Land" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644262850560, "text": "I just see this person and he looked just like @D_Boy3rd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236812194 }}, "user": { "id": 605507434, "name": "✨", "screen_name": "adnariv", "lang": "en", "location": "isla vista", "create_at": date("2012-06-11"), "description": "blessed.humble.educated.focused", "followers_count": 1855, "friends_count": 966, "statues_count": 110760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644279623680, "text": "Period lurking the terror is coming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161022279, "name": "✧peppermint snatch ✧", "screen_name": "MOLLYBEACH", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-06-29"), "description": "shameless and vulgar ***********laugh or cringe PAYPAL: mollybeach@hotmail.com ✡ ✡✡ ✡ University of Washington", "followers_count": 8769, "friends_count": 2146, "statues_count": 56313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644665577473, "text": "Stuck on a trail in the middle of nowhere. On a bike, pitch dark. This is… https://t.co/1W4GkBMkK6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.83845593,37.52341548"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424017482, "name": "Justin Mann", "screen_name": "JMMann07", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "Vagabond ✌️", "followers_count": 88, "friends_count": 160, "statues_count": 2823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Illinois, USA", "id": "f54a2170ff4b15f7", "name": "Illinois", "place_type": "admin", "bounding_box": rectangle("-91.51308,36.970298 -87.019935,42.508303") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17087, "countyName": "Johnson" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644673830912, "text": "��FairFax", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72449959, "name": "Alex", "screen_name": "ItsAlexO_", "lang": "en", "location": "New York, NY", "create_at": date("2009-09-07"), "description": "info.alexordonez@gmail.com dean@d1models.com", "followers_count": 2256, "friends_count": 160, "statues_count": 24701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945644845928448, "text": "Holy 3's Jesus Christ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2795701071, "name": "Swisher 2.0", "screen_name": "nicholasmuhl", "lang": "en", "location": "Bowling Green, Ohio", "create_at": date("2014-09-30"), "description": "Bowling Green State University - Lambda Chi Alpha - Political Science/Philosophy. Live our lives so well that Death trembles to take us.", "followers_count": 321, "friends_count": 407, "statues_count": 7116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645068251138, "text": "@bukakecreamteam I would try it and see what happens", "in_reply_to_status": 668944767288344576, "in_reply_to_user": 1640468898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1640468898 }}, "user": { "id": 34009973, "name": "uncle louie", "screen_name": "revodrewtion", "lang": "en", "location": "Toledo, OH", "create_at": date("2009-04-21"), "description": "be cool", "followers_count": 239, "friends_count": 228, "statues_count": 12631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645114400768, "text": "Lets go B-LO #Sabres https://t.co/dtWM3V9uqx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sabres" }}, "user": { "id": 315948163, "name": "Angie", "screen_name": "niagarangie", "lang": "en", "location": "Niagara Falls ", "create_at": date("2011-06-12"), "description": "I was born in a city we call Buffalo; zero degrees below is too damn cold and funky.", "followers_count": 213, "friends_count": 641, "statues_count": 385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645181399040, "text": "A few things I took from that game. 1. The Hoosiers should not be ranked. 2. Bill Walton is insane. 3. Tom Crean should still be fired.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35416457, "name": "Sam Schlesinger", "screen_name": "Sam_Schlesinger", "lang": "en", "location": "Indiana", "create_at": date("2009-04-25"), "description": "Sports enthusiast, drummer, student of the world, aspiring advertiser, champion of time. Go Hoosiers!", "followers_count": 250, "friends_count": 180, "statues_count": 12985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645265399808, "text": "@dannyviLe @ElnegroTrias Esperemos q no...........", "in_reply_to_status": 668945384392294401, "in_reply_to_user": 255476951, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 255476951, 158804385 }}, "user": { "id": 523443923, "name": "Un Uruguayo", "screen_name": "abbu25", "lang": "en", "location": "Chicago-Montevideo", "create_at": date("2012-03-13"), "description": "Grado 5 en imperiologia y en fraudeamplismo.", "followers_count": 1340, "friends_count": 484, "statues_count": 87425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645416415233, "text": "Cannot believe IU just lost that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220092617, "name": "Katie Butler", "screen_name": "katbutler09", "lang": "en", "location": "null", "create_at": date("2010-11-26"), "description": "Oregon | Indiana", "followers_count": 245, "friends_count": 435, "statues_count": 6311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granger, IN", "id": "20dc6048f416c894", "name": "Granger", "place_type": "city", "bounding_box": rectangle("-86.217544,41.709039 -86.023537,41.760686") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1828800, "cityName": "Granger" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645441388545, "text": "What are you doing for the rest of your life?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169799230, "name": "heidi b elizondo", "screen_name": "heidibeatriz", "lang": "en", "location": "null", "create_at": date("2010-07-22"), "description": "Pies, para que los quiero si tengo alas para volar -Frida Kahlo", "followers_count": 123, "friends_count": 191, "statues_count": 1677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
-{ "create_at": datetime("2015-11-23T16:14:04.000Z"), "id": 668945645693087744, "text": "Especially the fat ones https://t.co/8CKQBlDSRz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1732458362, "name": "Robert Carrillo", "screen_name": "Rob_Goes_Raw", "lang": "en", "location": "Rancho Cucamonga", "create_at": date("2013-09-05"), "description": "fitness. let a real nigga make it right.\nA straight up never serious disrespectful ass nigga.\nnothing i say i mean.", "followers_count": 386, "friends_count": 318, "statues_count": 36058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945645873577985, "text": "@natalieeweber @katieannevan do you reliever Christmas lights too? I too need to go that way but I'm cold", "in_reply_to_status": 668943746235527169, "in_reply_to_user": 89072530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89072530, 1942885938 }}, "user": { "id": 131608454, "name": "Kayli ☪☯☼❁", "screen_name": "kaymillzz", "lang": "en", "location": "CMU", "create_at": date("2010-04-10"), "description": "Portuguese / ♐ / delta phi epsilon", "followers_count": 835, "friends_count": 769, "statues_count": 42195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, MI", "id": "77f923f5a841bbdf", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-84.816945,43.553246 -84.727407,43.63446") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26073, "countyName": "Isabella", "cityID": 2656020, "cityName": "Mount Pleasant" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646183837697, "text": "Yo...Nehme's AP calculus...if y'all got that packet due tomorrow...send it :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150267997, "name": "Mayorga™", "screen_name": "imayorga_", "lang": "en", "location": "Yikes ", "create_at": date("2010-05-31"), "description": "Thick thighs & heart eyes", "followers_count": 627, "friends_count": 505, "statues_count": 8001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646339031040, "text": "@alkutbay wtf why is qdoba losing", "in_reply_to_status": 668936130193059840, "in_reply_to_user": 2844400954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2844400954 }}, "user": { "id": 2844400954, "name": "Cam Rellim", "screen_name": "alkutbay", "lang": "en", "location": "Oklahoma, USA", "create_at": date("2014-10-25"), "description": "boring white boy", "followers_count": 942, "friends_count": 95, "statues_count": 454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646422855680, "text": "@queengost @tradethecycles @CronkiteSays https://t.co/X5gexwzSJl", "in_reply_to_status": 668944542217969664, "in_reply_to_user": 2760050529, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2760050529, 24716002, 2546562188 }}, "user": { "id": 289256811, "name": "Barbara Jensen", "screen_name": "BarbaraJensen1", "lang": "en", "location": "California", "create_at": date("2011-04-28"), "description": "English Instructor, Merritt Writing Program, UC Merced; constitutional conservative; spinal reconstruction advocate; cat lady in training", "followers_count": 1614, "friends_count": 2446, "statues_count": 13863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646532063234, "text": "Huge mistake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1307683278, "name": "Raquel", "screen_name": "RaquelYPadilla", "lang": "en", "location": "North Carolina, USA", "create_at": date("2013-03-27"), "description": "wake up and be fucking amazing", "followers_count": 565, "friends_count": 291, "statues_count": 348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Half Moon, NC", "id": "f65751b1ab474517", "name": "Half Moon", "place_type": "city", "bounding_box": rectangle("-77.477448,34.793941 -77.437472,34.85045") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3728900, "cityName": "Half Moon" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646725017600, "text": "life is so good rn honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467083422, "name": "Vish✊", "screen_name": "vishushu14", "lang": "en", "location": "htx", "create_at": date("2012-01-17"), "description": "null", "followers_count": 773, "friends_count": 612, "statues_count": 31019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945646850736128, "text": "������ https://t.co/snic0W4YJN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3645728412, "name": "Marilyn Jordan", "screen_name": "marilynjordann", "lang": "en", "location": "Anaheim, CA", "create_at": date("2015-09-21"), "description": "null", "followers_count": 45, "friends_count": 121, "statues_count": 231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647035219969, "text": "On Saturday/Sunday, December 5/6 @RideSacRT will use special buses between Broadway & Florin Stations while @TheCityofSac constructs bridge.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316621010, 69359090 }}, "user": { "id": 1381072680, "name": "Ride Downtown 916", "screen_name": "RideDowntown916", "lang": "en", "location": "Downtown Sacramento, CA 95811", "create_at": date("2013-04-25"), "description": "A grassroots organization showing support for using public transportation throughout the 7-County, 29-City @SACOG & @SJCOG Regions to & from @docosacramento.", "followers_count": 856, "friends_count": 1680, "statues_count": 22588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647068971008, "text": "@BenRiegel there are two broadcast available on league pass (tv) one is the magic feed. I'm watching it now", "in_reply_to_status": 668945492286554112, "in_reply_to_user": 37227889, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37227889 }}, "user": { "id": 41074446, "name": "MegB", "screen_name": "RobandMeg", "lang": "en", "location": "Staten Island, NY", "create_at": date("2009-05-19"), "description": "Wife, Music Lover, Love taking my car to the track, Born and raised Texan now living in NYC, Texans fan and Orlando Magic fan! Find me on instagram: Meg180", "followers_count": 242, "friends_count": 413, "statues_count": 2668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647211417601, "text": "@WorldStarFunny another one of my favorites lol", "in_reply_to_status": 666397911123673088, "in_reply_to_user": 1107613844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1107613844 }}, "user": { "id": 2720501208, "name": "you", "screen_name": "Xfromdelcity", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "My net worth a good 4 cents", "followers_count": 546, "friends_count": 1044, "statues_count": 8678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647257579520, "text": "You girl think I'm a sweetheart", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237243120, "name": "Kenneth", "screen_name": "AhunnidKay", "lang": "en", "location": "Los Angeles / Riv", "create_at": date("2015-06-05"), "description": "$traight We$t Coa$tin", "followers_count": 265, "friends_count": 252, "statues_count": 1876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647391870976, "text": "Headshots got his eyes closed up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 524764757, "name": "King", "screen_name": "BossedUpAnt", "lang": "en", "location": "Chad Brown to Pumgansett St", "create_at": date("2012-03-14"), "description": "null", "followers_count": 575, "friends_count": 278, "statues_count": 25249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647559553025, "text": "promise I'll be alright.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 234632195, "name": "tina.", "screen_name": "ThatGirlAsiaa", "lang": "en", "location": "Denton, TX", "create_at": date("2011-01-05"), "description": "#LLL #UNT18", "followers_count": 1287, "friends_count": 828, "statues_count": 115486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647702142976, "text": "I'm not about to go Black Friday shopping all these broke bitches out here trying to pick pocket .. Ima just wait till cyber Monday.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3904944612, "name": "MJ", "screen_name": "rideordie_x", "lang": "en", "location": "Naperville, IL", "create_at": date("2015-10-15"), "description": "Problem solver. Coffee nerd. Typical social mediaholic. Socialite. Infuriatingly humble writer.", "followers_count": 95, "friends_count": 441, "statues_count": 481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheaton, IL", "id": "57ac88aefcd20988", "name": "Wheaton", "place_type": "city", "bounding_box": rectangle("-88.151554,41.814328 -88.074936,41.889152") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1781048, "cityName": "Wheaton" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945647752634372, "text": "I'm at Chicago O'Hare International Airport (ORD) in Chicago, IL https://t.co/86uoo20SDv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.90440083,41.97727092"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 9206662, "name": "Matt", "screen_name": "jmatthew007", "lang": "en", "location": "Chicago", "create_at": date("2007-10-02"), "description": "Father of three, Married, Northwestern Alum. Basketball and Football tweets. Usually left leaning. Likes beer and BBQ. Destiny addict", "followers_count": 171, "friends_count": 966, "statues_count": 2148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945648637509632, "text": "I want coffee ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576825423, "name": "Stephanie ✨", "screen_name": "Steeeeeeephy", "lang": "en", "location": "null", "create_at": date("2013-07-07"), "description": "blessed", "followers_count": 857, "friends_count": 806, "statues_count": 8735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649203679232, "text": "Let me know how that works out for you ������ https://t.co/dyl3q3VDi5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1965797882, "name": "Empress Ebony", "screen_name": "Ebony_Anissa14", "lang": "en", "location": "Pville", "create_at": date("2013-10-16"), "description": "null", "followers_count": 437, "friends_count": 365, "statues_count": 2099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649287720960, "text": "It was supposed to be a #sunny #day @ Hinsdale, Illinois https://t.co/n6FgTtmBdB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9369,41.8008"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sunny", "day" }}, "user": { "id": 544854654, "name": "Anthony Poselenzny", "screen_name": "APoselenzny", "lang": "en", "location": "Naperville, Il.", "create_at": date("2012-04-03"), "description": "Retired UPSer, Married, 3 children, 2 married, a grandfather, conservative Republican, living in Chicago area. Volunteer mentor at SCORE.", "followers_count": 1138, "friends_count": 1999, "statues_count": 19070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hinsdale, IL", "id": "a2e9665d39e55b17", "name": "Hinsdale", "place_type": "city", "bounding_box": rectangle("-87.952677,41.763847 -87.899883,41.828954") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1735307, "cityName": "Hinsdale" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649489027073, "text": "@Nancy_burns15 wow hot af!!!", "in_reply_to_status": 668942151531806723, "in_reply_to_user": 921362670, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 921362670 }}, "user": { "id": 2764310613, "name": "Lexie Collard", "screen_name": "lexcollard", "lang": "en", "location": "null", "create_at": date("2014-09-07"), "description": "i ❤️ me https://twitter.com/worstofaidol/status/664509180678656001", "followers_count": 333, "friends_count": 307, "statues_count": 1575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deer Park, NY", "id": "081822bec6f679d4", "name": "Deer Park", "place_type": "city", "bounding_box": rectangle("-73.354311,40.740662 -73.295069,40.783594") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3619972, "cityName": "Deer Park" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649522630656, "text": "@kaitlynoc3 jus tell me", "in_reply_to_status": 668933687640244225, "in_reply_to_user": 1705572553, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1705572553 }}, "user": { "id": 559889001, "name": "Eazy_Jesuzzz", "screen_name": "eejholmes", "lang": "en", "location": "null", "create_at": date("2012-04-21"), "description": "trap_607 wher it always ☔️", "followers_count": 203, "friends_count": 898, "statues_count": 2972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oneonta, NY", "id": "ff9cc975fb01f747", "name": "Oneonta", "place_type": "city", "bounding_box": rectangle("-75.108773,42.433438 -74.980422,42.478019") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36077, "countyName": "Otsego", "cityID": 3654881, "cityName": "Oneonta" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649560190976, "text": "@laceyloo17 he'll be at @ParadeOfLasers ☺️", "in_reply_to_status": 668942603585474560, "in_reply_to_user": 603057885, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 603057885, 2388719234 }}, "user": { "id": 2190590492, "name": "melissss", "screen_name": "missssm3liss", "lang": "en", "location": "null", "create_at": date("2013-11-12"), "description": "•find comfort in chaos• EDM enthusiast", "followers_count": 609, "friends_count": 1004, "statues_count": 12374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broomfield, CO", "id": "00f8c733400e497a", "name": "Broomfield", "place_type": "city", "bounding_box": rectangle("-105.147267,39.894185 -104.997195,40.000314") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8014, "countyName": "Broomfield", "cityID": 809280, "cityName": "Broomfield" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649606393856, "text": "turkey bowl 6 on 6 tourney tomorrow op field at 11. Open to any squad of 6. Hmu for more information.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1242079939, "name": "skid", "screen_name": "jake_fr97", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "the real friedòle", "followers_count": 413, "friends_count": 273, "statues_count": 3194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Park, CA", "id": "0038b1943e3140e9", "name": "Oak Park", "place_type": "city", "bounding_box": rectangle("-118.789237,34.168065 -118.744992,34.21356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 653116, "cityName": "Oak Park" } }
-{ "create_at": datetime("2015-11-23T16:14:05.000Z"), "id": 668945649748983808, "text": "I've missed college basketball", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374299328, "name": "Jason A. Wyrick", "screen_name": "steagles1", "lang": "en", "location": "Naptown ✈️ The Ville", "create_at": date("2011-09-15"), "description": "Louisville Ha-ha Guy. Jesus, Pacers, and the UofL Cards. @Pacers and @IndyEleven follow me. Currently a Louisville Cardinal. GC Cougar forever. RIP Dad.", "followers_count": 736, "friends_count": 494, "statues_count": 49690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945649937743872, "text": "#PhoenixLostFound LOST AUSTRALIAN BORDER COLLIE (83RD & McDowell) (Casa Anita Apartments): URGENT: Lost Dog: Casa… https://t.co/7CwgtPrf3j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.07372189,33.4462674"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PhoenixLostFound" }}, "user": { "id": 3019837997, "name": "Phoenix Lost&Found", "screen_name": "Phoenix_LAF", "lang": "en", "location": "Phoenix Arizona", "create_at": date("2015-02-05"), "description": "null", "followers_count": 8, "friends_count": 68, "statues_count": 14061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650264903684, "text": "#marblemadnessmonday #gameboy @ Headfarmer https://t.co/xha88zNC3K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.8822098,33.5533066"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "marblemadnessmonday", "gameboy" }}, "user": { "id": 34794198, "name": "Kyle Steven Bahr", "screen_name": "kylesbahr", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-04-23"), "description": "Intuitive, focused, lover of music", "followers_count": 152, "friends_count": 137, "statues_count": 239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arizona, USA", "id": "a612c69b44b2e5da", "name": "Arizona", "place_type": "admin", "bounding_box": rectangle("-114.818269,31.332246 -109.045152,37.004261") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650331992064, "text": "وين أيام الهوى اللي كانت جميله .. كنت غير و كان حلمي فيك أكبر", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 222253327, "name": "Bo7saiN", "screen_name": "Boassna", "lang": "ar", "location": "KUWAIT +965", "create_at": date("2010-12-02"), "description": "13314 ❤️", "followers_count": 8179, "friends_count": 271, "statues_count": 83822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650416005120, "text": "@xxerika_ I love you too honey bunches of oats ❤️��", "in_reply_to_status": 668633523134296064, "in_reply_to_user": 744173478, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 744173478 }}, "user": { "id": 896195978, "name": "Shianne", "screen_name": "ShianneG", "lang": "en", "location": "null", "create_at": date("2012-10-21"), "description": "The most important things in life, arent things. Skylier❤️ 3/30/15", "followers_count": 382, "friends_count": 383, "statues_count": 6737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscarawas, OH", "id": "8bdeb5af4ef23efc", "name": "Tuscarawas", "place_type": "city", "bounding_box": rectangle("-81.412475,40.392321 -81.389397,40.40512") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39157, "countyName": "Tuscarawas", "cityID": 3977924, "cityName": "Tuscarawas" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650575400960, "text": "@btbCantina is rockin'!\n\nCome support the Special Olympics with us & @PhiSigmaRhoEta!\n\nWe're here until 11.", "in_reply_to_status": -1, "in_reply_to_user": 1250613134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1250613134, 266335326 }}, "user": { "id": 762030889, "name": "Phi SigMa Kappa-UofM", "screen_name": "a2phisig", "lang": "en", "location": "Ann Arbor, MI", "create_at": date("2012-08-16"), "description": "The official Twitter of the Delta Deuteron chapter of Phi Sigma Kappa at the University of Michigan.", "followers_count": 173, "friends_count": 102, "statues_count": 172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ann Arbor, MI", "id": "92b2158a6d509767", "name": "Ann Arbor", "place_type": "city", "bounding_box": rectangle("-83.800262,42.215087 -83.659681,42.324184") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2603000, "cityName": "Ann Arbor" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650717847552, "text": "\"Dad, what'dya do on your 1st day of vacation?\" \nI built this Lego model of the Eiffel Tower.\n\"That's sad, Dad.\" https://t.co/QPWgpKnaAe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25337234, "name": "Robot Taylor", "screen_name": "robotterror", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-03-19"), "description": "Always a treat, never a threat. Cloud something-something. Mostly harmless. Not really a robot. SRE at Adobe. 极客", "followers_count": 387, "friends_count": 258, "statues_count": 5109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shoreline, WA", "id": "ecb63eb9d9d83b31", "name": "Shoreline", "place_type": "city", "bounding_box": rectangle("-122.395019,47.733761 -122.291436,47.777982") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363960, "cityName": "Shoreline" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650881466368, "text": "ball is life guys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2932972574, "name": "Delilah Nguyen", "screen_name": "Delilah1126", "lang": "en", "location": "null", "create_at": date("2014-12-16"), "description": "B1A4 & Astro are the best. ✨", "followers_count": 45, "friends_count": 70, "statues_count": 278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650923409408, "text": "San Antonio ➡️ San the fuckin Diego!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1524165463, "name": "Jicuddy the Cancer♋️", "screen_name": "jicuddy", "lang": "en", "location": "EXT195, WA", "create_at": date("2013-06-17"), "description": "halfway there", "followers_count": 238, "friends_count": 241, "statues_count": 11346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945650923544576, "text": "it'll never be the same", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863816718, "name": "lys", "screen_name": "liss_jayda", "lang": "en", "location": "null", "create_at": date("2014-10-18"), "description": "♊️", "followers_count": 194, "friends_count": 166, "statues_count": 2562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651053539328, "text": "Massive headache :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544691296, "name": "Jedi Dad", "screen_name": "ashhh_schroeder", "lang": "en", "location": "null", "create_at": date("2012-04-03"), "description": "| #OU19 | coffee, god, mountains, love | my name is actually Ashleigh |", "followers_count": 334, "friends_count": 242, "statues_count": 22276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651145777152, "text": "@_LoveNesha peep my videos in my favs �� promise you won't waste your time �� #BassBoy appreciate it ��", "in_reply_to_status": -1, "in_reply_to_user": 339837956, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BassBoy" }}, "user_mentions": {{ 339837956 }}, "user": { "id": 2350513298, "name": "Blaze™♒14.6K", "screen_name": "OfficialBassBoy", "lang": "en", "location": "Detroit", "create_at": date("2014-02-18"), "description": "Ghetto Dreams Records \n\nMusic page: @HePlayBass\n#Neckflix", "followers_count": 14698, "friends_count": 14003, "statues_count": 95729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651158380544, "text": "#DEACS!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "DEACS" }}, "user": { "id": 249051050, "name": "Chris Edwards", "screen_name": "CSE_Deac", "lang": "en", "location": "New Bern, NC", "create_at": date("2011-02-07"), "description": "Attorney, currently working for federal courts; occasional runner; WFU; Deacs", "followers_count": 200, "friends_count": 896, "statues_count": 174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Bern, NC", "id": "c1ef8962bc0b514d", "name": "New Bern", "place_type": "city", "bounding_box": rectangle("-77.137333,35.076669 -77.034161,35.163741") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37049, "countyName": "Craven", "cityID": 3746340, "cityName": "New Bern" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651263115264, "text": "@franifio @ajplus @JunStinson @brookeminters thank you for making this video, these difficult times create a dire need for mental services.", "in_reply_to_status": 668915924871811072, "in_reply_to_user": 547114168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 547114168, 110396781, 405059892, 22071328 }}, "user": { "id": 18179415, "name": "Phillip Lorenzo", "screen_name": "pllorenzo", "lang": "en", "location": "San Diego, CA", "create_at": date("2008-12-16"), "description": "I love games, comics, books, great TV and film. I geek out. I live life every day with a new sense of curiosity. RTs are not endorsements.", "followers_count": 956, "friends_count": 857, "statues_count": 11631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651531513856, "text": "Trying to get my application for State done and my computer wants to be extra extra extra slow right now����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131762821, "name": "Melissa", "screen_name": "meligurll", "lang": "en", "location": "null", "create_at": date("2010-04-11"), "description": "September 20th 2014 was the greatest day of my life when I met Taylor Swift at her house and listened to her entire 1989 album❤️", "followers_count": 254, "friends_count": 162, "statues_count": 9078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651539931136, "text": "Made in St. Albert! @ Sturdyhouse https://t.co/lafH0Mskl5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.65526849,53.65626126"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22565733, "name": "Christopher Sturdy", "screen_name": "cbsturdy", "lang": "en", "location": "St. Albert, AB, Canada", "create_at": date("2009-03-02"), "description": "Professor of psychology, University of Alberta; studying comparative cognition & neuroethology; advocate for evidence-based health and wellness", "followers_count": 506, "friends_count": 968, "statues_count": 6034 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "St. Albert, Alberta", "id": "56316c24480c1a3a", "name": "St. Albert", "place_type": "city", "bounding_box": rectangle("-113.707088,53.599415 -113.565747,53.681859") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651690897408, "text": "������ https://t.co/zXf6QGeMXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3711148940, "name": "jK !", "screen_name": "iamjbabee", "lang": "en", "location": "null", "create_at": date("2015-09-27"), "description": "| ❤️ |", "followers_count": 322, "friends_count": 228, "statues_count": 3466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Charles, LA", "id": "21b6f72f9fdaeb06", "name": "Lake Charles", "place_type": "city", "bounding_box": rectangle("-93.314852,30.124859 -93.080595,30.280683") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2241155, "cityName": "Lake Charles" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651854540800, "text": "Put down that gun & try some Facebook Advertising. Then pick up the gun.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92295619, "name": "Joy Barr", "screen_name": "tinybadass", "lang": "en", "location": "90046 USA", "create_at": date("2009-11-24"), "description": "a senseless act of living", "followers_count": 1366, "friends_count": 1314, "statues_count": 24880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945651967856640, "text": "@yesImthatdope @urfavedes GTFOH.", "in_reply_to_status": 668945556656545792, "in_reply_to_user": 2768309057, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2768309057, 3753538215 }}, "user": { "id": 24127618, "name": "g", "screen_name": "ginamartinezco", "lang": "en", "location": "The Bronx | SoBe", "create_at": date("2009-03-12"), "description": "Instagram: ginamartinezco | Snapchat: g.nababy | contactginamartinez@gmail.com", "followers_count": 2337, "friends_count": 749, "statues_count": 63581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652181667840, "text": "I know what it's like to feel angry at God, authority, impersonators, role models and myself. Without question, there's something to prayer.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1171167042, "name": "Armour Gove Halson", "screen_name": "ByronWJohnson", "lang": "en", "location": "United States", "create_at": date("2013-02-11"), "description": "Ajunct USIP research guide, career whistleblower & terror survivor waiting on rekindling relations w/ my beloved children with hopes of a fair civil settlement.", "followers_count": 41, "friends_count": 156, "statues_count": 1175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lisle, IL", "id": "3009292baa8dda23", "name": "Lisle", "place_type": "city", "bounding_box": rectangle("-88.11909,41.760185 -88.042417,41.828705") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1743939, "cityName": "Lisle" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652601085952, "text": "Look who it is ������ @Cee_foee25 @wdrew19 @amulipola10 @ZTaylor07 https://t.co/xWdordcY8r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2852078534, 2405302326, 3237591807, 3222087664 }}, "user": { "id": 2374703971, "name": "Anakin", "screen_name": "BRenteria_45", "lang": "en", "location": "null", "create_at": date("2014-03-05"), "description": "#98'LBC \n《BpHs》", "followers_count": 439, "friends_count": 534, "statues_count": 14808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652714483712, "text": "@TheeMattB no fucking way", "in_reply_to_status": 668945440893755393, "in_reply_to_user": 95064837, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95064837 }}, "user": { "id": 714669102, "name": "whitesoxdave", "screen_name": "barstoolWSD", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-07-24"), "description": "Everything #WhiteSox on Barstool Sports | Former DIII bullpen catcher | Kenny Powers Protege | Viva Chicago, Viva la Stool! | chisoxbarstool@gmail.com |", "followers_count": 6324, "friends_count": 554, "statues_count": 33399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652907397120, "text": "@EricOswaltRock @The_Iceman_Show https://t.co/ajDB0kewk6", "in_reply_to_status": -1, "in_reply_to_user": 2722203234, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2722203234, 1275902911 }}, "user": { "id": 4089541480, "name": "Rick", "screen_name": "racdd4u5050", "lang": "en", "location": "South Carolina, USA", "create_at": date("2015-11-01"), "description": "Management, Writer, Love Variety of Music, NASCAR and Life. I Hold No Hate Toward Anyone, Never Have and Never Will! Get to Know Me Before You Judge Me. Peace!", "followers_count": 984, "friends_count": 859, "statues_count": 11472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Easley, SC", "id": "0029290613dee5f5", "name": "Easley", "place_type": "city", "bounding_box": rectangle("-82.663558,34.776033 -82.476443,34.891439") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45077, "countyName": "Pickens", "cityID": 4521985, "cityName": "Easley" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652961820672, "text": "I pass you the ball and you broke my assist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2491972429, "name": ".", "screen_name": "GuitarGuy241", "lang": "en", "location": "null", "create_at": date("2014-05-12"), "description": "null", "followers_count": 115, "friends_count": 131, "statues_count": 8153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wesley Chapel, FL", "id": "0003df7b402165e0", "name": "Wesley Chapel", "place_type": "city", "bounding_box": rectangle("-82.403523,28.169963 -82.24588,28.279042") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1275875, "cityName": "Wesley Chapel" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945652999696384, "text": "Omfg Niall doing an English accent https://t.co/vcjBDGhhfG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.00398001,40.63435342"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30497066, "name": "meow", "screen_name": "EmileeLindner", "lang": "en", "location": "nyc/buff", "create_at": date("2009-04-11"), "description": "Pop music writer @MTVNews. I'm cute. emilee.lindner@viacommix.com", "followers_count": 2183, "friends_count": 924, "statues_count": 8351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653175730176, "text": "Don't judge me, join me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2961609835, "name": "tbuut", "screen_name": "taylorrdb", "lang": "en", "location": "AZ", "create_at": date("2015-01-06"), "description": "22. #calicojacks. part time party animal", "followers_count": 370, "friends_count": 325, "statues_count": 2464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653247160321, "text": "It went well ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291862650, "name": "Bambi❤️", "screen_name": "___Yannnnn", "lang": "en", "location": "✈ NC", "create_at": date("2011-05-02"), "description": "null", "followers_count": 1400, "friends_count": 1137, "statues_count": 30950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653356036096, "text": "Your taste in music is perfff https://t.co/b5B5DeMtMZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 168012616, "name": "Cookie Jams", "screen_name": "CookieEatsDinos", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-07-17"), "description": "19. Catch Me At Every Issues Show In IL. #hooligan #neverloseyourflames (IG: cookieeatsdinos)", "followers_count": 1112, "friends_count": 670, "statues_count": 74331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653565751296, "text": "Happy to say I can faithfully change my nose rings now����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2438670957, "name": "BiNK", "screen_name": "RianLB_", "lang": "en", "location": "w/ TiNBiN❤️", "create_at": date("2014-03-26"), "description": "16. |Friend4L❤️|reese➰|i❤️Kayla,Morgan,Dezarae|SMALLZ is annoying❤️", "followers_count": 1525, "friends_count": 1270, "statues_count": 33553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenwillow, OH", "id": "8327dc05095356c7", "name": "Glenwillow", "place_type": "city", "bounding_box": rectangle("-81.489091,41.349159 -81.450332,41.378887") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3930632, "cityName": "Glenwillow" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653582680064, "text": "send me emojis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2148938489, "name": "8/23✨", "screen_name": "_janesax3", "lang": "en", "location": "null", "create_at": date("2013-10-22"), "description": "My best friend turned into my guardian angel on 10/17/15. I love you Janasia Alicia Velez♥️", "followers_count": 958, "friends_count": 764, "statues_count": 15171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lorain, OH", "id": "91d57ea9ae3b0bbd", "name": "Lorain", "place_type": "city", "bounding_box": rectangle("-82.292185,41.401871 -82.096884,41.485211") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3944856, "cityName": "Lorain" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653892911104, "text": "@jinnjuicee good lord", "in_reply_to_status": 668945519763304448, "in_reply_to_user": 178821800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178821800 }}, "user": { "id": 114837079, "name": "oh.", "screen_name": "austinhastoes", "lang": "en", "location": "denial", "create_at": date("2010-02-16"), "description": "tell me i'm a mobster", "followers_count": 463, "friends_count": 250, "statues_count": 13418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladstone, MO", "id": "e5731aae9bb5e805", "name": "Gladstone", "place_type": "city", "bounding_box": rectangle("-94.593389,39.173738 -94.529799,39.235151") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29047, "countyName": "Clay", "cityID": 2927190, "cityName": "Gladstone" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945653985251328, "text": "@DavidChhay1 @WorldStarFunny Lmao wth", "in_reply_to_status": 668936332786339840, "in_reply_to_user": 2233932926, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2233932926, 1107613844 }}, "user": { "id": 2849436048, "name": "Fab✨", "screen_name": "fabxo__", "lang": "en", "location": "wherever the wind takes me", "create_at": date("2014-10-09"), "description": "❥ Ed Sheeran & the weeknd ❥ || ❁ ❁BA Belle❁ ❁||", "followers_count": 176, "friends_count": 261, "statues_count": 5421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
-{ "create_at": datetime("2015-11-23T16:14:06.000Z"), "id": 668945654077485056, "text": "Closed Homeless Concerns request at 4300 18th St https://t.co/DUTsW3Ktyj. Case closed. case completed. resolved: done.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.43743881,37.76086417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1589692776, "name": "SF311 Reports", "screen_name": "SF311Reports", "lang": "en", "location": "San Francisco, CA", "create_at": date("2013-07-12"), "description": "This is account is not monitored and is used to post service request update information. To communicate with SF311, please use our main account 'SF311'", "followers_count": 38, "friends_count": 0, "statues_count": 280817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
-{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654136373250, "text": "@keschilli @laurentsangel looks more like you than dicaprio", "in_reply_to_status": 668945414788378624, "in_reply_to_user": 497076183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497076183, 309685171 }}, "user": { "id": 2786773720, "name": "ㅤ", "screen_name": "YouAreLouard", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-09-26"), "description": "notre dunkin donuts cathedral latin", "followers_count": 707, "friends_count": 716, "statues_count": 14640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Heights, OH", "id": "668fe3ffece4000a", "name": "Highland Heights", "place_type": "city", "bounding_box": rectangle("-81.489033,41.530607 -81.447145,41.570126") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3935252, "cityName": "Highland Heights" } }
-{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654174101506, "text": "That was a big bunch of nothin' ..... #IUBB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IUBB" }}, "user": { "id": 1127945767, "name": "Just_Jer!", "screen_name": "jerry_shonk", "lang": "en", "location": "Greensburg, Indiana USA", "create_at": date("2013-01-28"), "description": "#Colts #Pacers #Reds #IUBB #NASCAR #TeamHendricks #48 #WWE #YR #Sopranos #BatesMotel #HoosierBornAndRaised", "followers_count": 1672, "friends_count": 914, "statues_count": 29082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensburg, IN", "id": "9f29ee1a2f040588", "name": "Greensburg", "place_type": "city", "bounding_box": rectangle("-85.518421,39.312107 -85.459117,39.362492") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18031, "countyName": "Decatur", "cityID": 1829718, "cityName": "Greensburg" } }
-{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654199226369, "text": "Nigga give me somewhere to be after work besides home��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 494633996, "name": "SUTTLES #TeamAllison", "screen_name": "Suttles_", "lang": "en", "location": "The LVB", "create_at": date("2012-02-16"), "description": "JS&VG™", "followers_count": 1055, "friends_count": 1055, "statues_count": 36914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicopee, MA", "id": "1cacd95a1b3c5221", "name": "Chicopee", "place_type": "city", "bounding_box": rectangle("-72.626984,42.124341 -72.513489,42.218354") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2513660, "cityName": "Chicopee" } }
-{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654266269696, "text": "@Spo0kyChris is it the black parade", "in_reply_to_status": 668944873152626688, "in_reply_to_user": 3037173564, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3037173564 }}, "user": { "id": 1276076918, "name": "kirby", "screen_name": "mvster_sword", "lang": "en", "location": "Whittier, CA", "create_at": date("2013-03-17"), "description": "drifting endlessly in the great cosmic dark/18", "followers_count": 704, "friends_count": 396, "statues_count": 18126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
-{ "create_at": datetime("2015-11-23T16:14:07.000Z"), "id": 668945654446559232, "text": "Patriots Pre-gaming deep in the heart of Texas... #starsatnight #bigandbright @ Cedar Springs Tap House https://t.co/nr7afYsMNN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.81248297,32.81178018"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "starsatnight", "bigandbright" }}, "user": { "id": 29255704, "name": "Dan Milner", "screen_name": "damilnermd", "lang": "en", "location": "null", "create_at": date("2009-04-06"), "description": "null", "followers_count": 162, "friends_count": 138, "statues_count": 3331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636383522817, "text": "Oigan, aprendan a usar comas. A veces es todo un reto decifrar sus mensajes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 161750888, "name": "Kevin Gómez.", "screen_name": "KevinKyu", "lang": "en", "location": "Wales, United Kingdom", "create_at": date("2010-07-01"), "description": "No éramos eso.\nUWC Atlantic College '15 - '17", "followers_count": 1008, "friends_count": 309, "statues_count": 41742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636450676737, "text": "#HappyNewYear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HappyNewYear" }}, "user": { "id": 30989108, "name": "Robby!!!", "screen_name": "Rainbowblitz13", "lang": "en", "location": "las vegas nevada", "create_at": date("2009-04-13"), "description": "null", "followers_count": 281, "friends_count": 1410, "statues_count": 1222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636484251648, "text": "2016 gonna be dope", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337892158, "name": "Har", "screen_name": "iamharleyreed", "lang": "en", "location": "in the pussy ", "create_at": date("2014-02-10"), "description": "lesbian", "followers_count": 676, "friends_count": 1083, "statues_count": 6552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636576595968, "text": "2016 to new beginnings...new chapter as one ends. Happy New Year. @ CORNERSTONE CHRISTIAN BAPTIST… https://t.co/U3X1yQJIuI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.15916887,33.52803713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92003761, "name": "Hooplyf3", "screen_name": "LAKERS_4_VIDA", "lang": "en", "location": "Bahrain", "create_at": date("2009-11-23"), "description": "LAKERS all day, Everyday.\nAdd me on snapchat Hoopislyf3", "followers_count": 289, "friends_count": 334, "statues_count": 4634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636664582144, "text": "@jose_reyes1719 1", "in_reply_to_status": 682833209197854720, "in_reply_to_user": 2431541504, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2431541504 }}, "user": { "id": 2956678184, "name": "Chivo", "screen_name": "Chivog00", "lang": "en", "location": "Oregon☔️", "create_at": date("2015-01-02"), "description": "Soccer⚽️ ✏️SHS✏️ SC:chivo.00", "followers_count": 126, "friends_count": 137, "statues_count": 598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stayton, OR", "id": "8c1fab74667d644e", "name": "Stayton", "place_type": "city", "bounding_box": rectangle("-122.821577,44.78882 -122.763283,44.814874") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4170200, "cityName": "Stayton" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833636844896256, "text": "Watching @pitbull on @FOXTV Oye felicidades chico y dios de bendiga #VivaCuba", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "VivaCuba" }}, "user_mentions": {{ 31927467, 16537989 }}, "user": { "id": 1052610578, "name": "Caesar Garcia", "screen_name": "gcaesar24", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-01-01"), "description": "Los Angeles Born N Raised and part of The #FastFamily Also a WWE, Lakers, Chargers, Dodgers, and Kings fan and a CM Punk Fan. My favorite shows r L&O SVU & SOA", "followers_count": 333, "friends_count": 754, "statues_count": 22332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637037887488, "text": "I love when my tios are drunk because they give me life lessons and they always want to dance������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3273404744, "name": "leslie", "screen_name": "leslieavilla_", "lang": "en", "location": "null", "create_at": date("2015-07-09"), "description": "null", "followers_count": 283, "friends_count": 233, "statues_count": 7363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Juan, TX", "id": "6c253d30b4b6dbd4", "name": "San Juan", "place_type": "city", "bounding_box": rectangle("-98.17347,26.153868 -98.11375,26.244297") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4865532, "cityName": "San Juan" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637117706240, "text": "2016!!! @ Mondrian South Beach https://t.co/q3SgJr2VCq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1427765,25.7819405"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33505572, "name": "ACT Productions", "screen_name": "ACTProductions", "lang": "en", "location": "Miami Beach, FL", "create_at": date("2009-04-20"), "description": "Global Production Company headquartered in Miami; 25 years of Film, Video, Print and Event Production Services. #actproductions #passiontranslated", "followers_count": 886, "friends_count": 661, "statues_count": 1167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637180583938, "text": "\"Arrogance is a part of success\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3398065929, "name": "wintafresh", "screen_name": "vikaasxavier", "lang": "en", "location": "Ball State High School ", "create_at": date("2015-07-31"), "description": "Junior at Indiana Academy #nothingwasthesame", "followers_count": 174, "friends_count": 208, "statues_count": 847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637641850880, "text": "I MAY HAVE LOST WEIGHT BUT IM FLABBY NOW ITS GROSS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2788994113, "name": "max glizzy", "screen_name": "mvxine", "lang": "en", "location": "New Orleans, Louisiana ", "create_at": date("2014-09-03"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀january 15| honduran |nola", "followers_count": 562, "friends_count": 663, "statues_count": 7245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harvey, LA", "id": "337d304222d75060", "name": "Harvey", "place_type": "city", "bounding_box": rectangle("-90.092431,29.836718 -90.035095,29.916898") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2233245, "cityName": "Harvey" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637683933184, "text": "���������� I remember when niggas said I look like the black girl from rug rats", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2789139094, "name": "tay", "screen_name": "tayah_ps", "lang": "en", "location": "getting right with the God", "create_at": date("2014-09-28"), "description": "I stay lit hoe", "followers_count": 533, "friends_count": 259, "statues_count": 7598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New London, CT", "id": "6e9c054da607b539", "name": "New London", "place_type": "city", "bounding_box": rectangle("-72.12852,41.304762 -72.087716,41.385244") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 952280, "cityName": "New London" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833637968973824, "text": "I LOVE YOU SO MUCH @ChrisEvans @ChrisEvans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 580097412, 580097412 }}, "user": { "id": 4449299774, "name": "Laure Garcia", "screen_name": "laurenmichg", "lang": "en", "location": "Aliso Viejo, CA", "create_at": date("2015-12-03"), "description": "null", "followers_count": 5, "friends_count": 17, "statues_count": 6 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aliso Viejo, CA", "id": "f9838a67f87a80ed", "name": "Aliso Viejo", "place_type": "city", "bounding_box": rectangle("-117.75971,33.512428 -117.700301,33.609056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 600947, "cityName": "Aliso Viejo" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638002577408, "text": "goodbye 2015.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108829919, "name": "• Skylar •", "screen_name": "skydanderson", "lang": "en", "location": "constant state of pcd ♋ cancer", "create_at": date("2010-01-26"), "description": "null", "followers_count": 146, "friends_count": 578, "statues_count": 4685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638023540736, "text": "Mood : blaaaah ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146547690, "name": "Monicaa", "screen_name": "MonicaMapps", "lang": "en", "location": "null", "create_at": date("2015-04-09"), "description": "null", "followers_count": 483, "friends_count": 783, "statues_count": 1663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, TX", "id": "d3aa6dad1368841f", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-96.162071,33.064175 -96.049638,33.171817") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4830920, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638245859328, "text": "1 MINS 'TIL NEW YEAR'S!!!!!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4397969240, "name": "Cristy Yee", "screen_name": "cristyanney", "lang": "en", "location": "in your head", "create_at": date("2015-11-29"), "description": "III XIV • ♋️", "followers_count": 60, "friends_count": 73, "statues_count": 356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638363299840, "text": "Happy New Year~ ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.93089294,37.34957838"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16167171, "name": "Khaos Tian", "screen_name": "KhaosT", "lang": "en", "location": "Santa Clara, CA", "create_at": date("2008-09-07"), "description": "Life in Limbo.", "followers_count": 646, "friends_count": 540, "statues_count": 22547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638367576064, "text": "I'm so hungry����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3951980907, "name": "Meghan", "screen_name": "meghantarrant1", "lang": "en", "location": "null", "create_at": date("2015-10-13"), "description": "jhs '19 Snapchat Me~bruhitsmeg14", "followers_count": 89, "friends_count": 144, "statues_count": 314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shields, MI", "id": "01e4e96164b42c18", "name": "Shields", "place_type": "city", "bounding_box": rectangle("-84.150851,43.394987 -84.043059,43.481571") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26145, "countyName": "Saginaw", "cityID": 2673560, "cityName": "Shields" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638405369856, "text": "@yekaJV I'm missing a $25 golfsmith gift card. Just seeing if any of your homies having it because I can't find it", "in_reply_to_status": -1, "in_reply_to_user": 418211504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418211504 }}, "user": { "id": 86955532, "name": "Noäh Zimmërlin", "screen_name": "zimmex88", "lang": "en", "location": "Columbus", "create_at": date("2009-11-02"), "description": "Once we can laugh, we can live.", "followers_count": 285, "friends_count": 392, "statues_count": 2745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833638610763776, "text": "Going into another weird year in the worst city here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2367746204, "name": "Scout Farmer", "screen_name": "scout_farmer", "lang": "en", "location": "null", "create_at": date("2014-03-01"), "description": "null", "followers_count": 198, "friends_count": 189, "statues_count": 345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport Beach, CA", "id": "0ce4c27ce26030aa", "name": "Newport Beach", "place_type": "city", "bounding_box": rectangle("-117.958219,33.563618 -117.789845,33.671658") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 651182, "cityName": "Newport Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639072120832, "text": "@Inthehighest1 https://t.co/F1CNTJOt68 im god sun/son tho .. i got my god giving halo ... #rp aka #mj #HappyNewYear #HappyNewYear2016", "in_reply_to_status": 682829112621142016, "in_reply_to_user": 494465036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rp", "mj", "HappyNewYear", "HappyNewYear2016" }}, "user_mentions": {{ 494465036 }}, "user": { "id": 494465036, "name": "rp- on.fb.me/sAIX7I", "screen_name": "Inthehighest1", "lang": "en", "location": "1.usa.gov/asaOYG bit.ly/8D3pYx", "create_at": date("2012-02-16"), "description": "http://ibm.co/1LKO2DU Exa-Scale http://ibm.co/1dFBVFu AndroidOS http://bit.ly/1OpdxMo Systems http://bit.ly/1F6e2bn http://binged.it/1rKsrjC", "followers_count": 615, "friends_count": 1230, "statues_count": 30091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639122403329, "text": "Ayeeeeeeeee����������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1928837990, "name": "Jordan Greenlee", "screen_name": "FlashJordan41", "lang": "en", "location": "null", "create_at": date("2013-10-02"), "description": "2016 DL/TE/OLB 6'4/245-Vintage High School Napa, CA http://www.hudl.com/athlete/2085908/jordan-greenlee #1 DL NBPreps #5 DL in NorCal", "followers_count": 769, "friends_count": 647, "statues_count": 2247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639172771840, "text": "New year who dis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2943203593, "name": "CA", "screen_name": "YaboiiCollin", "lang": "en", "location": "null", "create_at": date("2014-12-25"), "description": "永遠に / R.E.A.L", "followers_count": 369, "friends_count": 375, "statues_count": 2290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639239892993, "text": "2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 766637413, "name": "nana", "screen_name": "khing_xoxo", "lang": "en", "location": "ktown", "create_at": date("2012-08-18"), "description": "@SawyerFrdrx is not an oppaㅣขิงคนเดิมเพิ่มเติมคือน้องแจ็คสัน", "followers_count": 85, "friends_count": 45, "statues_count": 11392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639256670208, "text": "Y'all are something else... Lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 337229674, "name": "Samantha Mick ♔", "screen_name": "samanthaaamick", "lang": "en", "location": "Cypress, Tx", "create_at": date("2011-07-17"), "description": "My friends call me Mantha", "followers_count": 388, "friends_count": 289, "statues_count": 8575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639265009666, "text": "2 secs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3371861133, "name": "angel", "screen_name": "KristinaaV18", "lang": "en", "location": "Chino", "create_at": date("2015-07-12"), "description": "Dlhs volleyball #12", "followers_count": 248, "friends_count": 183, "statues_count": 9974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639386648576, "text": "Its my birthday bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2646375566, "name": "Sammmm", "screen_name": "Saamanthhhaaa", "lang": "en", "location": "Tillys ", "create_at": date("2014-07-14"), "description": "Im on a rollercoaster that only goes up my friend", "followers_count": 596, "friends_count": 432, "statues_count": 7209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasco, WA", "id": "41647560efe1db77", "name": "Pasco", "place_type": "city", "bounding_box": rectangle("-119.230869,46.198319 -119.02462,46.292793") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53021, "countyName": "Franklin", "cityID": 5353545, "cityName": "Pasco" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639399280640, "text": "#HappyNewYear !!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HappyNewYear" }}, "user": { "id": 76136142, "name": "Pat Aukes", "screen_name": "HitByCarHobo", "lang": "en", "location": "Woodland", "create_at": date("2009-09-21"), "description": "23. Snapchat: potrick_star", "followers_count": 248, "friends_count": 258, "statues_count": 4300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639449726976, "text": "Just posted a photo @ North Hollywood, California https://t.co/STbmNn5MPR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.378,34.1722"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18133094, "name": "Palmer Blevins", "screen_name": "pblev", "lang": "en", "location": "Springfield, IL", "create_at": date("2008-12-14"), "description": "null", "followers_count": 53, "friends_count": 58, "statues_count": 138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639500025856, "text": "Ripley SW Limestone Co. Temp: 34.9°F Wind:3.8mph Pressure: 1006.4mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 46829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639596539905, "text": "@DEFENSEisKEY hell na�� Dassie ain't about dat life ty", "in_reply_to_status": 682833517214994432, "in_reply_to_user": 620926664, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 620926664 }}, "user": { "id": 1666155306, "name": "⠀", "screen_name": "XOFaith_", "lang": "en", "location": "columbia", "create_at": date("2013-08-12"), "description": "5'0, columbian n puerto rican. Sc: dasskardash1 IG: dassie_cardona varsity cheerleader @brysontiller ❤️", "followers_count": 3421, "friends_count": 1652, "statues_count": 66428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639600599040, "text": "Happy new year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391172735, "name": "Steve Yang", "screen_name": "steveshaoyi", "lang": "zh-cn", "location": "San Dimas, CA", "create_at": date("2011-10-14"), "description": "null", "followers_count": 111, "friends_count": 423, "statues_count": 269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639617372160, "text": "Woooo almost 2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1699898149, "name": "asslee", "screen_name": "Omgashleee", "lang": "en", "location": "null", "create_at": date("2013-08-25"), "description": "It aint right, but its true", "followers_count": 107, "friends_count": 214, "statues_count": 1364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639650889728, "text": "Are you serious", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30839740, "name": "La'Flare", "screen_name": "FUImALesBeeAnn", "lang": "en", "location": "Here to tweet NOT impress ppl", "create_at": date("2009-04-13"), "description": "So blunt that you can smoke my truth ™ \n\n\nWife @diashanae ❤", "followers_count": 1593, "friends_count": 1767, "statues_count": 101981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheboygan, WI", "id": "c7e3277182d3bc61", "name": "Sheboygan", "place_type": "city", "bounding_box": rectangle("-87.800211,43.691373 -87.69572,43.805744") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55117, "countyName": "Sheboygan", "cityID": 5572975, "cityName": "Sheboygan" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639655124994, "text": "It's been real 2015! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560765370, "name": "Edmond Edralin", "screen_name": "legendaryedmond", "lang": "en", "location": "Carson, CA", "create_at": date("2012-04-22"), "description": "Pray about everything, worry about nothing!", "followers_count": 283, "friends_count": 195, "statues_count": 881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639726428164, "text": "2016 bitches!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339445205, "name": "♏️ika¥la", "screen_name": "MikaylaNear", "lang": "en", "location": "Pullman ~ Bremerton", "create_at": date("2014-02-11"), "description": "•Snap: mckaykay383 • 18 • idk who I am • fuck a fake friend where your real friends at • Rest in Peace Rayla D. Meyers I love you •", "followers_count": 398, "friends_count": 195, "statues_count": 19411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Orchard, WA", "id": "006f1fd939044d62", "name": "Port Orchard", "place_type": "city", "bounding_box": rectangle("-122.708264,47.43909 -122.603325,47.548799") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5355785, "cityName": "Port Orchard" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639793528832, "text": "HAPPY NEW YEARS��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3337358713, "name": "$HIVANI", "screen_name": "yoitshivu", "lang": "en", "location": "the bay ", "create_at": date("2015-08-25"), "description": "null", "followers_count": 106, "friends_count": 94, "statues_count": 132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833639814606848, "text": "Happy newyear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3366795314, "name": "Nate", "screen_name": "rchsnate", "lang": "en", "location": "null", "create_at": date("2015-08-27"), "description": "null", "followers_count": 21, "friends_count": 46, "statues_count": 100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640049393665, "text": "* bitches https://t.co/gD3Ynr4HD9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145925115, "name": "j baby", "screen_name": "_jassminekm", "lang": "en", "location": "null", "create_at": date("2015-04-07"), "description": "it happens to the best of us.", "followers_count": 648, "friends_count": 499, "statues_count": 10583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640074575872, "text": "Happy New Years everyone�� Let this new year be a great one�� #DiamondLife", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DiamondLife" }}, "user": { "id": 865810680, "name": "Jalen Harrington", "screen_name": "JalenHarrington", "lang": "en", "location": "Tucson, AZ", "create_at": date("2012-10-06"), "description": "|| 18 Years Old || U of A '19 || FASA Pamilya || Huntington Beach, CA #Cali2Zona #Blessed", "followers_count": 230, "friends_count": 200, "statues_count": 3227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640116457472, "text": "HAPPY NEW YEARS����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2426319650, "name": "brookelyn", "screen_name": "__Brookieeeee", "lang": "en", "location": "null", "create_at": date("2014-04-03"), "description": "@_kouriann_❤️", "followers_count": 1563, "friends_count": 882, "statues_count": 10199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640280031232, "text": "2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2575273339, "name": "DOMINIC LERMA", "screen_name": "lerma_dominic", "lang": "en", "location": "null", "create_at": date("2014-06-18"), "description": "vintage football #armystrong #cbscrew", "followers_count": 147, "friends_count": 150, "statues_count": 1088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640288432131, "text": "�� bet https://t.co/whYPxg0Ltr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 338343408, "name": "❄️", "screen_name": "SoIcyWhitt", "lang": "en", "location": "Dallas", "create_at": date("2011-07-19"), "description": "I don't bop I do the money dance #Mom #PvNation #FreeGucci", "followers_count": 3075, "friends_count": 2651, "statues_count": 116492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640322134016, "text": "Good difficult riddance.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1547781, "name": "Aretic", "screen_name": "tithonium", "lang": "en", "location": "seattle.wa.earth.sol.orion.mw", "create_at": date("2007-03-19"), "description": "Web developer. Server farmer. Geek Factotum. Aretic. Hegemon of Mars (deposed). Drift incompatible.\n\nI'm not good; I'm not nice; I'm just right.", "followers_count": 801, "friends_count": 245, "statues_count": 40416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gatewood, Seattle", "id": "769d03a01df39337", "name": "Gatewood", "place_type": "neighborhood", "bounding_box": rectangle("-122.398324,47.528293 -122.376393,47.54663") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-01T00:00:00.000Z"), "id": 682833640380747776, "text": "tonight was somethin to remember fasho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54102471, "name": "chloe short", "screen_name": "chloemaelove", "lang": "en", "location": "null", "create_at": date("2009-07-05"), "description": "null", "followers_count": 662, "friends_count": 529, "statues_count": 3823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833640670138370, "text": "2016 WE IN THIS BITCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002903629, "name": "Mattehoe♛", "screen_name": "iTzTeoo", "lang": "en", "location": "‏كثير الرؤى", "create_at": date("2015-01-29"), "description": "Hhs Senior | Track&Field", "followers_count": 390, "friends_count": 351, "statues_count": 7764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833640699514881, "text": "Happy New Year!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16578022, "name": "Kyle Cronin", "screen_name": "kylecronin", "lang": "en", "location": "Maine", "create_at": date("2008-10-03"), "description": "You may contact me at kyle.cronin@gmail.com", "followers_count": 252, "friends_count": 402, "statues_count": 4932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833640707854336, "text": "Happy New Year!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1101610280, "name": "Kat", "screen_name": "_katttcee", "lang": "en", "location": "null", "create_at": date("2013-01-18"), "description": "Mike ❤️", "followers_count": 513, "friends_count": 436, "statues_count": 26234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833640733069312, "text": "2016 Resolution: Make money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 744305982, "name": "Jåśøñ", "screen_name": "fraleyjason", "lang": "en", "location": "Moms Diner and Sugar Shack", "create_at": date("2012-08-07"), "description": "Take care of those who take care of you", "followers_count": 305, "friends_count": 319, "statues_count": 2434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pahrump, NV", "id": "0128fa53607453b1", "name": "Pahrump", "place_type": "city", "bounding_box": rectangle("-116.103394,36.096268 -115.902526,36.307718") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32023, "countyName": "Nye", "cityID": 3253800, "cityName": "Pahrump" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833640846266368, "text": "HAPPYMUTHAFUCKIN2016 NYUCCAS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715664347, "name": "dom", "screen_name": "playboidom", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "sensational", "followers_count": 773, "friends_count": 568, "statues_count": 22539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641056022528, "text": "HAPPY NEW YEAR BAY AREA! WISH I WAS THERE.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152040273, "name": "Diana George", "screen_name": "CalAlumni81", "lang": "en", "location": "Martinez,CA", "create_at": date("2010-06-04"), "description": "I swing steel and shoot stick and twig.Cal Alumni 1981\r\nI am a CAL legacy. My family Bleeds Blue and Gold", "followers_count": 250, "friends_count": 415, "statues_count": 26229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641177624576, "text": "2:00 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2296375080, "name": "Bree Steadman", "screen_name": "__breighanna", "lang": "en", "location": "620", "create_at": date("2014-01-17"), "description": "if you're happy who cares | sc- breeaboo.92", "followers_count": 1702, "friends_count": 552, "statues_count": 21640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hutchinson, KS", "id": "af29db2ab5fd16c8", "name": "Hutchinson", "place_type": "city", "bounding_box": rectangle("-97.966318,38.02387 -97.849296,38.108414") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20155, "countyName": "Reno", "cityID": 2033625, "cityName": "Hutchinson" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641261535232, "text": "It's 2016. #pointingouttheobvious", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "pointingouttheobvious" }}, "user": { "id": 509122545, "name": "Collin Markle", "screen_name": "CollinMarkle_5", "lang": "en", "location": "Terre Haute, IN", "create_at": date("2012-02-29"), "description": "Sprintcar racer • Photographer • California born, Indiana living • Indiana State University • Message me at markle.collin@gmail.com for photo orders.", "followers_count": 865, "friends_count": 384, "statues_count": 13795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641278324736, "text": "#HappyNewYear����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HappyNewYear" }}, "user": { "id": 4402015334, "name": "Eli Kim", "screen_name": "elikim36", "lang": "en", "location": "null", "create_at": date("2015-11-29"), "description": "null", "followers_count": 16, "friends_count": 50, "statues_count": 14 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasco, WA", "id": "41647560efe1db77", "name": "Pasco", "place_type": "city", "bounding_box": rectangle("-119.230869,46.198319 -119.02462,46.292793") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53021, "countyName": "Franklin", "cityID": 5353545, "cityName": "Pasco" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641282486277, "text": "Happy new year everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587919620, "name": "Garrett Nieto", "screen_name": "the_panda31", "lang": "en", "location": "Yorba Linda, CA", "create_at": date("2012-05-22"), "description": "Wenatchee Wild", "followers_count": 331, "friends_count": 351, "statues_count": 2182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Wenatchee, WA", "id": "5b7a3e927573f3b3", "name": "East Wenatchee", "place_type": "city", "bounding_box": rectangle("-120.316137,47.400888 -120.259336,47.488468") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53017, "countyName": "Douglas", "cityID": 5320155, "cityName": "East Wenatchee" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641286729728, "text": "HAPPY NEW YEAAARSSSSS ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 710185820, "name": "Alana", "screen_name": "Teddytwomeals97", "lang": "en", "location": "Los Angeles & Davis, CA", "create_at": date("2012-07-21"), "description": "Harry Potter & Ed Sheeran fanatic | UC Davis '19", "followers_count": 150, "friends_count": 144, "statues_count": 9778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redondo Beach, CA", "id": "e1e35d357ceefa52", "name": "Redondo Beach", "place_type": "city", "bounding_box": rectangle("-118.401931,33.814689 -118.352695,33.894649") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 660018, "cityName": "Redondo Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641492226053, "text": "HAPPY NEW YEAR!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 637379402, "name": "J", "screen_name": "juvnlopez", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "criminal minds enthusiast", "followers_count": 1403, "friends_count": 437, "statues_count": 34627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641529974784, "text": "Andddd happy New Years all! Here's to #2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19045292, "name": "John Weng", "screen_name": "johnweng92", "lang": "en", "location": "Ocean Beach", "create_at": date("2009-01-15"), "description": "Program Advisor for the AS @ UCSD. 2nd year MA Postsecondary Ed Leadership Student Affairs @ SDSU. ΒΘΠ. #SAchat #NASPAnerd #foodie #techie", "followers_count": 414, "friends_count": 277, "statues_count": 1262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641659998209, "text": "HAPPY NUT YEAR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3776101632, "name": "derik", "screen_name": "deirk420", "lang": "en", "location": "hell", "create_at": date("2015-10-03"), "description": "fuck boi...", "followers_count": 129, "friends_count": 112, "statues_count": 1002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641949380612, "text": "Happy New Years ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1047315968, "name": "شلMr.Willisش", "screen_name": "Babyflash_300", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2012-12-30"), "description": "[Student athlete] Football. Grind is the key to success get up an turn nothing into something.!! #Fearnoevil ❤️R.I.P DAD they say the good die young. G.B.M", "followers_count": 845, "friends_count": 799, "statues_count": 5963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833641970384896, "text": "happy sweet 2016 new year every body yay yay yeeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76744396, "name": "Jonhy Danger Rose", "screen_name": "BabyBoyR3mixz", "lang": "en", "location": "Reno NEVADA", "create_at": date("2009-09-23"), "description": "i dont take any order from some one cause im a CAPTAIN........", "followers_count": 22, "friends_count": 51, "statues_count": 2269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833642117328897, "text": "This #love is unlike anything i've ever dreamt of. ❤️❤️❤️���������� https://t.co/pkY00obGGn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.614563,39.276474"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "love" }}, "user": { "id": 2891101189, "name": "Sheryl", "screen_name": "LunaSheryl_", "lang": "en", "location": "Maryland, USA", "create_at": date("2014-11-05"), "description": "Queen Sheryl Married to @TheOnlyStatus13 New mommy to Avery Madison ❤️ #PitBull lover! #Family for life!", "followers_count": 179, "friends_count": 52, "statues_count": 5109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Banditos", "id": "07d9eb7468881000", "name": "Banditos", "place_type": "poi", "bounding_box": rectangle("-76.6145631,39.2764739 -76.614563,39.276474") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833642251407361, "text": "hello 2016.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3181412119, "name": "kailen", "screen_name": "kailenn_", "lang": "en", "location": "Oregon, USA", "create_at": date("2015-04-30"), "description": "11.5.15", "followers_count": 162, "friends_count": 286, "statues_count": 869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottage Grove, OR", "id": "f8d27df4377109f4", "name": "Cottage Grove", "place_type": "city", "bounding_box": rectangle("-123.094008,43.762254 -123.024017,43.814973") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4115950, "cityName": "Cottage Grove" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833642306023424, "text": "Mariah who?? https://t.co/84UWTQWED4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2499953383, "name": "LindSLEIGH", "screen_name": "LLeezus", "lang": "en", "location": "null", "create_at": date("2014-05-16"), "description": "ok cool", "followers_count": 122, "friends_count": 282, "statues_count": 2090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burtonsville, MD", "id": "3c079afd375b804b", "name": "Burtonsville", "place_type": "city", "bounding_box": rectangle("-76.956178,39.079059 -76.902952,39.122453") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2411750, "cityName": "Burtonsville" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643287363585, "text": "HAPPY NEW YEARS! May you all have a blessed one! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 844290722, "name": "L.", "screen_name": "_AyoLindaa", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-09-24"), "description": "IG: xoliiinda. SC: ayolindaa", "followers_count": 525, "friends_count": 418, "statues_count": 12644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange Cove, CA", "id": "461c7cf6e96d2d78", "name": "Orange Cove", "place_type": "city", "bounding_box": rectangle("-119.331688,36.610175 -119.303893,36.638109") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 654008, "cityName": "Orange Cove" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643312566272, "text": "Happy New Years bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1023573560, "name": "$AB", "screen_name": "sabrina_taher", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-12-19"), "description": "If I die I'm a legend. #GSW #49ers #SFG", "followers_count": 575, "friends_count": 428, "statues_count": 32760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643346079744, "text": "Happy New Years people ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 402491827, "name": "scryson scriller", "screen_name": "Brysonbriggs", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2011-10-31"), "description": "http://www.hudl.com/athlete/2707746/highlights/293115379", "followers_count": 1138, "friends_count": 967, "statues_count": 14258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643350265856, "text": "Miss my roomie and Kato ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304193733, "name": "Kaitlin Lubinsky", "screen_name": "LubinskyKaitlin", "lang": "en", "location": "LaCrosse, WI --Mankato, MN", "create_at": date("2011-05-23"), "description": "•Snapchat: lubinskykaitlin•MSU Mankato•", "followers_count": 809, "friends_count": 472, "statues_count": 9232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holmen, WI", "id": "58855a1798eb9042", "name": "Holmen", "place_type": "city", "bounding_box": rectangle("-91.285148,43.916177 -91.231676,43.977476") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55063, "countyName": "La Crosse", "cityID": 5535450, "cityName": "Holmen" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643379687425, "text": "First tweet of 2016 #HappyNewYear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyNewYear" }}, "user": { "id": 299854035, "name": "Jessica Jones", "screen_name": "jtuanjones", "lang": "en", "location": "Austin, TX", "create_at": date("2011-05-16"), "description": "Marketing/Tech/Startups. Runner. World explorer. Perpetual learner. Lover of travel-sized products and fitted menswear. PIC @mattpauljones.", "followers_count": 2800, "friends_count": 616, "statues_count": 1027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643421577216, "text": "HAPPY NEW YEAR ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271253874, "name": "Tatiana Espinoza", "screen_name": "sweetheartati", "lang": "en", "location": "null", "create_at": date("2013-12-31"), "description": "| CHS Volleyball #46 | Nicholas Mendez will always hold my heart❤️|", "followers_count": 428, "friends_count": 347, "statues_count": 4769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643543355393, "text": "Ok goodnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1463494616, "name": "Gen.", "screen_name": "TinyGen_", "lang": "en", "location": "null", "create_at": date("2013-05-27"), "description": "⚜Just a free spirit with a wild heart⚜", "followers_count": 935, "friends_count": 466, "statues_count": 36279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Steelton, PA", "id": "0b80d95912418164", "name": "Steelton", "place_type": "city", "bounding_box": rectangle("-76.852958,40.208401 -76.795743,40.24338") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42043, "countyName": "Dauphin", "cityID": 4273888, "cityName": "Steelton" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643568431104, "text": "@JeremyDixonDJ check your DM. Happy New Year", "in_reply_to_status": 682616859389227008, "in_reply_to_user": 423442459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423442459 }}, "user": { "id": 95442789, "name": "susan coleman", "screen_name": "moonchild242", "lang": "en", "location": "Chandler, AZ, USA", "create_at": date("2009-12-08"), "description": "Half Doctor Who/Sherlock fandom, half #DublinTalks98FM fan, half what's for lunch? Yes, I know three halves are not a thing. Maths were never my strong suit.", "followers_count": 372, "friends_count": 275, "statues_count": 31945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643618746373, "text": "It's Z016.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2586988021, "name": "♡", "screen_name": "harryLAstyles", "lang": "en", "location": "twitter", "create_at": date("2014-06-24"), "description": "is it 2017 yet ?", "followers_count": 223, "friends_count": 281, "statues_count": 1347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, Los Angeles", "id": "27f5da73dd83395c", "name": "Hollywood", "place_type": "neighborhood", "bounding_box": rectangle("-118.396479,34.080262 -118.28053,34.152521") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643715321856, "text": "@s_mckinley1 @Hendrickson_2 you leave, he is free", "in_reply_to_status": 682831425008189441, "in_reply_to_user": 65112734, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65112734, 701239652 }}, "user": { "id": 1281295128, "name": "Brandon Hajduk", "screen_name": "brandon_hajduk", "lang": "en", "location": "null", "create_at": date("2013-03-19"), "description": "The ville. Snapchat: brandon_hajduk", "followers_count": 379, "friends_count": 250, "statues_count": 6269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Strongsville, OH", "id": "2532aa9a835f3a40", "name": "Strongsville", "place_type": "city", "bounding_box": rectangle("-81.883223,41.275311 -81.785053,41.350754") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3975098, "cityName": "Strongsville" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643723554817, "text": "HAPPY NEW YEARS✨����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2230947223, "name": "Angela", "screen_name": "angela_adams590", "lang": "en", "location": "CA", "create_at": date("2013-12-04"), "description": "✵ class of '16 ✵", "followers_count": 204, "friends_count": 267, "statues_count": 921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643744575488, "text": "AHHHH SHIT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3267438404, "name": "MB", "screen_name": "freakinmorize", "lang": "en", "location": "null", "create_at": date("2015-07-03"), "description": "make every night a saturday night", "followers_count": 288, "friends_count": 120, "statues_count": 7515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cerrito, CA", "id": "3a4df2e210d69e68", "name": "El Cerrito", "place_type": "city", "bounding_box": rectangle("-117.536857,33.820953 -117.506615,33.862398") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621810, "cityName": "El Cerrito" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643748720642, "text": "Happy New Years!!!! Same shit tho lmao.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1296146550, "name": "Manny", "screen_name": "MannyTru", "lang": "en", "location": "null", "create_at": date("2013-03-24"), "description": "Save some", "followers_count": 10, "friends_count": 56, "statues_count": 218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643794993152, "text": "a New Years tradition of waking my mom up to watch the ball drop just makes me so happy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1267148634, "name": "ru(dolph)ssell", "screen_name": "russellolsenn", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "Instagram | russellolsen", "followers_count": 202, "friends_count": 129, "statues_count": 291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833643987800064, "text": "#HappyNewYear everyone ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyNewYear" }}, "user": { "id": 752025554, "name": "Johnny Gomez", "screen_name": "johnnysfgiants", "lang": "en", "location": "Salinas, CA", "create_at": date("2012-08-11"), "description": "Biggest SF Giants & New England Patriots fan Big MLB fanatic love baseball can watch it all day long", "followers_count": 256, "friends_count": 2012, "statues_count": 29267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644004573188, "text": "HAPPY NEW YEAR MY LOVES!!!!!!����✨✨✨ xoxo ily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2223077760, "name": "im a penguin", "screen_name": "_falconprincess", "lang": "en", "location": "Instagram: falconprincess", "create_at": date("2013-11-30"), "description": "the blood in my veins is made up of mistakes. http://5sosf.am/JBHyt michael/4", "followers_count": 1613, "friends_count": 532, "statues_count": 49188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlingame, CA", "id": "9cee92fec370baf9", "name": "Burlingame", "place_type": "city", "bounding_box": rectangle("-122.403936,37.568111 -122.330819,37.604114") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 609066, "cityName": "Burlingame" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644008804352, "text": "〽️ Side Tho .. Even Tho Im in Torrance", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2877471058, "name": "❗️J.SiKK™❗️", "screen_name": "OfficialJSiKK", "lang": "en", "location": "E | S K O M P T O N ⚠️", "create_at": date("2014-11-14"), "description": "|| Niggaz ⬇️ Playin' The Money , But Thats What U Do When The Money ⬇️ ||", "followers_count": 67, "friends_count": 57, "statues_count": 2144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644029739009, "text": "HAPPY NEW YEARS ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2551722378, "name": "Kayla❣", "screen_name": "Kayla__Archer", "lang": "en", "location": "null", "create_at": date("2014-06-06"), "description": "null", "followers_count": 252, "friends_count": 211, "statues_count": 1628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644184928258, "text": "@alexisallredx awe I love you ��", "in_reply_to_status": 682824652310233092, "in_reply_to_user": 3309409441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3309409441 }}, "user": { "id": 3243771908, "name": "abby", "screen_name": "abbyyrobertson", "lang": "en", "location": "null", "create_at": date("2015-06-12"), "description": "null", "followers_count": 244, "friends_count": 227, "statues_count": 434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Snowflake, AZ", "id": "017b28380c1b8180", "name": "Snowflake", "place_type": "city", "bounding_box": rectangle("-110.131809,34.480708 -110.067148,34.527087") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4017, "countyName": "Navajo", "cityID": 467800, "cityName": "Snowflake" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644214444032, "text": "I feel about the new year how I feel about Mondays, a brand new chance for something great. Happy… https://t.co/2iKAXIi7Ye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.136,36.175"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 903295470, "name": "Bailey Hinz", "screen_name": "tri_bailey", "lang": "en", "location": "Las Vegas, USA", "create_at": date("2012-10-24"), "description": "5'3 with big ol' dreams!", "followers_count": 196, "friends_count": 143, "statues_count": 852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644222717952, "text": "Happy New Years everyone!!��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262400441, "name": "Madison Taylor", "screen_name": "Browne18Madison", "lang": "en", "location": "null", "create_at": date("2013-12-25"), "description": "chhs soccer", "followers_count": 224, "friends_count": 335, "statues_count": 3414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644256423936, "text": "Watching Fresh Prince..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332553005, "name": "Faith.", "screen_name": "faithddawson", "lang": "en", "location": "null", "create_at": date("2011-07-09"), "description": "Blessed. |ELHS 17'✨", "followers_count": 524, "friends_count": 282, "statues_count": 10332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sevierville, TN", "id": "3157ff8cced5dbdb", "name": "Sevierville", "place_type": "city", "bounding_box": rectangle("-83.619367,35.822985 -83.47107,35.935433") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47155, "countyName": "Sevier", "cityID": 4767120, "cityName": "Sevierville" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644268826624, "text": "Happy New Years y'all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3038820266, "name": "8999", "screen_name": "andrewhull03", "lang": "en", "location": "jupitor ", "create_at": date("2015-02-23"), "description": "My handles on my thoughts used to be as ill as Rondos J.Cole", "followers_count": 150, "friends_count": 139, "statues_count": 5235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644365299712, "text": "Next doors party must be pretty lit... ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3038668622, "name": "最高", "screen_name": "phung_ethan", "lang": "en", "location": "sunny d", "create_at": date("2015-02-23"), "description": "it means the highest", "followers_count": 56, "friends_count": 60, "statues_count": 1066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644390494208, "text": "HAPPY NEW YEARS !!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481804716, "name": "SOLDIERLANI™", "screen_name": "davis_jelani", "lang": "en", "location": "null", "create_at": date("2012-02-02"), "description": "6ft WR || LMC Student Athlete || Just tryna make it out for all the people that said I couldn't do it #grindmode", "followers_count": 501, "friends_count": 489, "statues_count": 4448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644428242944, "text": "Happy New Years!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3303642091, "name": "Charlize Padilla", "screen_name": "PadillaCharlize", "lang": "en", "location": "null", "create_at": date("2015-08-01"), "description": "null", "followers_count": 61, "friends_count": 73, "statues_count": 31 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644503699456, "text": "I hold it down like no other I gave up going to hangout with rob banks & fat nick for a nigga I was talking to.And he turned out to be �� fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3149592764, "name": "Okasan Kattana✨", "screen_name": "0kasan_", "lang": "en", "location": "null", "create_at": date("2015-04-11"), "description": "Intersectional Feminist | Black Activist | Desexualize the Female body| Transitioning to Veganism| #Blacklivesmatter", "followers_count": 128, "friends_count": 151, "statues_count": 1379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644533059585, "text": "happy new year!!❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2579575008, "name": "gianna barwald", "screen_name": "gbarwald", "lang": "en", "location": "cali • sc - gigi_barwald", "create_at": date("2014-06-20"), "description": "so this is my twitter... im sorry for about 99.9% of my tweets", "followers_count": 315, "friends_count": 443, "statues_count": 1239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Solana Beach, CA", "id": "cff7f9f9c3d8bfb8", "name": "Solana Beach", "place_type": "city", "bounding_box": rectangle("-117.27809,32.979449 -117.236137,33.008319") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 672506, "cityName": "Solana Beach" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644533092353, "text": "HAPPY NEW YEAR BITCHESSS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1574564586, "name": "kogi altamirano", "screen_name": "KogiAltamirano", "lang": "en", "location": "Chula Vista, CA", "create_at": date("2013-07-07"), "description": "san diego | photography | volleyball | asb | ohs '17", "followers_count": 336, "friends_count": 345, "statues_count": 3618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644570816513, "text": "happy new year i love frank iero and dogs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45753869, "name": "kris", "screen_name": "kristynval", "lang": "en", "location": "california", "create_at": date("2009-06-08"), "description": "18 yr old queer; charlie", "followers_count": 354, "friends_count": 255, "statues_count": 35332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644650500096, "text": "HAPPY FUCKING NEW YEARS!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278323457, "name": "Mufasá", "screen_name": "Beauty_maita", "lang": "en", "location": "California, IE ", "create_at": date("2011-04-06"), "description": "Model/Track/Dance✨Visionary and theorist✨ Future Psychologist", "followers_count": 3906, "friends_count": 423, "statues_count": 49440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644692451328, "text": "2016 is here omfg �� Happy New Year may this year bring all of you guys nothing but joy ✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 736094252, "name": "Jose Rodriguez", "screen_name": "JCRR__", "lang": "en", "location": "Modesto , California ", "create_at": date("2012-08-03"), "description": "Constantly rolling my eyes or talking about clothes", "followers_count": 586, "friends_count": 478, "statues_count": 36788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-01T00:00:01.000Z"), "id": 682833644742782977, "text": "Omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 748118966, "name": "T", "screen_name": "NTailee", "lang": "en", "location": "null", "create_at": date("2012-08-09"), "description": "I could tell from our subtweets we might be into each other now that ur reading this I'm sure of it ur move .", "followers_count": 76, "friends_count": 59, "statues_count": 1764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-01T00:00:02.000Z"), "id": 682833644918972416, "text": "Happy New Years ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2179376112, "name": "Austin Warren", "screen_name": "Austinnwarren", "lang": "en", "location": "null", "create_at": date("2013-11-06"), "description": "“When you want to succeed, \nAs bad as you want to breathe, \nThen you’ll be successful.”", "followers_count": 314, "friends_count": 269, "statues_count": 3331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granite Falls, WA", "id": "904cf1c2b3145274", "name": "Granite Falls", "place_type": "city", "bounding_box": rectangle("-121.985693,48.071895 -121.947407,48.094482") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5327995, "cityName": "Granite Falls" } }
+{ "create_at": datetime("2016-01-01T00:00:02.000Z"), "id": 682833644948344833, "text": "Happy New Year!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1347175026, "name": "River Alexander", "screen_name": "riveralexander", "lang": "en", "location": "Seattle, WA", "create_at": date("2013-04-12"), "description": "Front-end Developer / Graphic Designer. I like music and I'm also interested in electronic engineering. I'm all over the place.", "followers_count": 292, "friends_count": 156, "statues_count": 320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenmore, WA", "id": "6af9e210d376ab4d", "name": "Kenmore", "place_type": "city", "bounding_box": rectangle("-122.27603,47.725969 -122.218179,47.777123") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335170, "cityName": "Kenmore" } }
+{ "create_at": datetime("2016-01-01T00:00:02.000Z"), "id": 682833644998688768, "text": "HAPPY NEW YEARS!!!!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3666586513, "name": "edwin", "screen_name": "OhSnapEdwin", "lang": "en", "location": "null", "create_at": date("2015-09-23"), "description": "INC || CHS'18", "followers_count": 49, "friends_count": 94, "statues_count": 57 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-01-01T00:00:02.000Z"), "id": 682833645116108801, "text": "Happy New Years Bitches ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1405680271, "name": "Alexis Leonard", "screen_name": "Lexiecakkes", "lang": "en", "location": "null", "create_at": date("2013-05-05"), "description": "livin free like the fish in the sea", "followers_count": 442, "friends_count": 339, "statues_count": 14469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196024362524672, "text": "It's 3 it's early asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2248961175, "name": "JZIZZZLE", "screen_name": "Jzizzzle_", "lang": "en", "location": "Detroit, MI ", "create_at": date("2013-12-28"), "description": "What's gone surprise me this year", "followers_count": 978, "friends_count": 545, "statues_count": 31777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastpointe, MI", "id": "0969de5c93c437c4", "name": "Eastpointe", "place_type": "city", "bounding_box": rectangle("-82.971839,42.449927 -82.916743,42.48053") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2624290, "cityName": "Eastpointe" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196024739872769, "text": "Killin me! lol https://t.co/9siWrHZh5K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3817701193, "name": "Major key", "screen_name": "silinusi", "lang": "en", "location": "Tacoma, WA", "create_at": date("2015-10-07"), "description": "|SDA| SC: six.stringz", "followers_count": 89, "friends_count": 98, "statues_count": 758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562109,47.166863 -122.348569,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196024857464832, "text": "Ripley SW Limestone Co. Temp: 31.6°F Wind:1.6mph Pressure: 1007.0mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 46889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025159323648, "text": "Can we play some SOD tomorrow????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226732038, "name": "kay", "screen_name": "kjrendon", "lang": "en", "location": "University of Texas RGV", "create_at": date("2015-05-25"), "description": "you are a child of the universe no less than the trees & the stars ❤️", "followers_count": 492, "friends_count": 432, "statues_count": 8845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murillo, TX", "id": "018561b7575b780a", "name": "Murillo", "place_type": "city", "bounding_box": rectangle("-98.155649,26.235135 -98.074291,26.29763") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4850092, "cityName": "Murillo" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025302085632, "text": "#inSomeGucciFlipFlops", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "inSomeGucciFlipFlops" }}, "user": { "id": 164576482, "name": "Kenny", "screen_name": "KendalsTweeets", "lang": "en", "location": "null", "create_at": date("2010-07-08"), "description": "22 ~ Long Island.. IG @_Kendal", "followers_count": 249, "friends_count": 67, "statues_count": 18979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, NY", "id": "5fe049183a425395", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-73.295325,40.754764 -73.209269,40.807341") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608026, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025452929024, "text": "@D_Oswag @Papichulo_ST Haha I forgot about power im tripping", "in_reply_to_status": 683195763808075776, "in_reply_to_user": 327541470, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 327541470, 434294344 }}, "user": { "id": 503988928, "name": "TwoTwo Golson", "screen_name": "Yaboy_Vino", "lang": "en", "location": "null", "create_at": date("2012-02-25"), "description": "IM trying to be a HOOPER. #CSUDH Deep East OAKLAND, California. Just let me have a convo with Katy Perry or Keke Palmer.", "followers_count": 936, "friends_count": 764, "statues_count": 28795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025503223809, "text": "my girl crush, obsessed https://t.co/OWeid40YyV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421642193, "name": "Linzzzaaay", "screen_name": "bashawlinzey", "lang": "en", "location": "null", "create_at": date("2011-11-25"), "description": "dp", "followers_count": 822, "friends_count": 482, "statues_count": 10435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deer Park, TX", "id": "cc480e2388809012", "name": "Deer Park", "place_type": "city", "bounding_box": rectangle("-95.14691,29.663671 -95.087119,29.751499") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4819624, "cityName": "Deer Park" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025599688704, "text": "Eating wheat thins is like eating the souls of past exes who were too white and salty for my taste", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 308915635, "name": "Stupid American", "screen_name": "aeshton", "lang": "en", "location": "Vancouver, WA", "create_at": date("2011-05-31"), "description": "also an asshole", "followers_count": 126, "friends_count": 91, "statues_count": 6380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025608122368, "text": "@DrDern I fucked up oh god!!!! \" dallas bed is so comfortable \" ��", "in_reply_to_status": 683194934766088192, "in_reply_to_user": 2386174530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2386174530 }}, "user": { "id": 3557353513, "name": "hanns", "screen_name": "misshannns", "lang": "en", "location": "in hell •", "create_at": date("2015-09-13"), "description": "18•", "followers_count": 238, "friends_count": 244, "statues_count": 2144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025868173312, "text": "@SmileeeKylieee mhmmmm brownies ����", "in_reply_to_status": 683195335099863040, "in_reply_to_user": 408192663, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user_mentions": {{ 408192663 }}, "user": { "id": 441322404, "name": "d", "screen_name": "daijaraee", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "LL2", "followers_count": 1632, "friends_count": 597, "statues_count": 78734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025880854529, "text": "Detox", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "da", "is_retweet": false, "user": { "id": 388486351, "name": "Julianne Gulli", "screen_name": "julie_gulli", "lang": "en", "location": "Harrisonburg, VA", "create_at": date("2011-10-10"), "description": "Simplicity is the ultimate form of sophistication ❁ ❁ VKT | JMU'19 ❁ ❁", "followers_count": 400, "friends_count": 273, "statues_count": 17275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Ridge, VA", "id": "bd2c3c1743dc3dcf", "name": "Lake Ridge", "place_type": "city", "bounding_box": rectangle("-77.362725,38.660632 -77.259114,38.709815") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5143432, "cityName": "Lake Ridge" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196025943769089, "text": "@karolina_capala answer", "in_reply_to_status": -1, "in_reply_to_user": 552248441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 552248441 }}, "user": { "id": 453068573, "name": "datzsaba♉️", "screen_name": "Sabasolomon1", "lang": "en", "location": "null", "create_at": date("2012-01-02"), "description": "r.i.p Elvis free Zekke, free Earl", "followers_count": 666, "friends_count": 597, "statues_count": 7476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196026220445697, "text": "i cut them bitches off when the ball drop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1613268595, "name": "bianca", "screen_name": "yourstruly_beee", "lang": "en", "location": "null", "create_at": date("2013-07-22"), "description": "stay hydrated", "followers_count": 59, "friends_count": 56, "statues_count": 338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196026442760192, "text": "03:00 30.4°F Feels:22.1°F (Hi30.4°F/Lo29.7°F) Hum:67% Wnd:WSW 9.3MPH Baro:30.12in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 161, "friends_count": 261, "statues_count": 24218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196026556125185, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1685168060, "name": "lj ⚠", "screen_name": "lacy_johnson4", "lang": "en", "location": "null", "create_at": date("2013-08-20"), "description": "idc ¯\\_(ツ)_/¯", "followers_count": 374, "friends_count": 262, "statues_count": 4908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dixon, IL", "id": "8f16af9f16f59631", "name": "Dixon", "place_type": "city", "bounding_box": rectangle("-89.520108,41.817427 -89.438924,41.900046") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17103, "countyName": "Lee", "cityID": 1720162, "cityName": "Dixon" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027252420609, "text": "@Rejectsxyo wow Kendra that's how u feel", "in_reply_to_status": 683195858607837184, "in_reply_to_user": 2948405205, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2948405205 }}, "user": { "id": 372555912, "name": "Rismic", "screen_name": "Rismics", "lang": "en", "location": "Coger", "create_at": date("2011-09-12"), "description": ":/", "followers_count": 1576, "friends_count": 72, "statues_count": 9794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027344662529, "text": "God been so good to me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2458830203, "name": "T Herbo‼️", "screen_name": "queen_trezure", "lang": "en", "location": "SC:: qveen_trezure", "create_at": date("2014-04-03"), "description": "The real bratz doll❤️", "followers_count": 501, "friends_count": 406, "statues_count": 15872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027415867392, "text": "happy new year guys @emmy_mnm @_leslers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3142942273, 2690817637 }}, "user": { "id": 2756796307, "name": "jess", "screen_name": "jessicaaruizz_", "lang": "en", "location": "null", "create_at": date("2014-08-22"), "description": "vvhs", "followers_count": 213, "friends_count": 199, "statues_count": 1519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027436949504, "text": "@qdwyy grow up", "in_reply_to_status": 683196004225675264, "in_reply_to_user": 2959371803, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2959371803 }}, "user": { "id": 2959371803, "name": "Quinn Dwyer", "screen_name": "qdwyy", "lang": "en", "location": "somewhere goin brazy", "create_at": date("2015-01-03"), "description": "null", "followers_count": 950, "friends_count": 368, "statues_count": 1911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodhaven, MI", "id": "00487e63709614f2", "name": "Woodhaven", "place_type": "city", "bounding_box": rectangle("-83.306006,42.109216 -83.215583,42.18442") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2688380, "cityName": "Woodhaven" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027600506880, "text": "Wind 0.0 mph ---. Barometer 30.368 in, Steady. Temperature 38.7 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027671818240, "text": "Wind 0 mph --. Barometer 30.40 in, Steady. Temperature 39.2 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 19912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196027940114432, "text": "Same https://t.co/D5UHkTbGGW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179869869, "name": "T R A P C O L E", "screen_name": "Architex_", "lang": "en", "location": "jc/nj", "create_at": date("2010-08-18"), "description": "Spoken Word artist. Trying to figure out what my future holds. IG- @lordeggzavier_ ChatSnap- @architexx", "followers_count": 459, "friends_count": 398, "statues_count": 22732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-02T00:00:00.000Z"), "id": 683196028217126912, "text": "I hear a clock ticking but I don't have a clock", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3120475581, "name": "Cam", "screen_name": "KaravasCameron", "lang": "en", "location": "null", "create_at": date("2015-03-26"), "description": "training to have great physique", "followers_count": 191, "friends_count": 183, "statues_count": 1369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rossford, OH", "id": "186623622a8a0863", "name": "Rossford", "place_type": "city", "bounding_box": rectangle("-83.588074,41.55895 -83.542516,41.617914") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3968686, "cityName": "Rossford" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196028548349952, "text": "What that thang hitting for?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260820198, "name": "Cj Peterson", "screen_name": "True_2TheGame", "lang": "en", "location": "Phenix City, AL ✌ Dallas, TX", "create_at": date("2011-03-04"), "description": "#Thorough #SharpMind #1SFGH #WinnersCircle\nig&snap: swgmstrcj", "followers_count": 1587, "friends_count": 1403, "statues_count": 49075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196028607016960, "text": "Yep I did it agian", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1611291499, "name": "ClayBall8Ball", "screen_name": "Clay_Vest", "lang": "en", "location": "Greenfield, Indiana ", "create_at": date("2013-07-21"), "description": "Trappen Made It Happen \nMystery Man", "followers_count": 2886, "friends_count": 71, "statues_count": 2121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, IN", "id": "5e7c11acce63506a", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-85.830451,39.7427 -85.726818,39.828811") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18059, "countyName": "Hancock", "cityID": 1829520, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029114646528, "text": "they zoomed in on her cryingggggg im dead�� https://t.co/149OnYuRMi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334784815, "name": "boricuaaa.", "screen_name": "_lilmamita", "lang": "en", "location": "null", "create_at": date("2011-07-13"), "description": "#ChowanU\n#LongLiveMarco\n#WopWorld", "followers_count": 4448, "friends_count": 2233, "statues_count": 94309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockingham, NC", "id": "ee7e36e22117d541", "name": "Rockingham", "place_type": "city", "bounding_box": rectangle("-79.798782,34.90839 -79.721824,34.986454") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37153, "countyName": "Richmond", "cityID": 3757260, "cityName": "Rockingham" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029324296192, "text": "And stop looking at relationships like they got a damn checklist.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257786926, "name": "Mad Max", "screen_name": "TheWILLinPower", "lang": "en", "location": "Martinique ✈ Huntsvillle", "create_at": date("2011-02-25"), "description": "309|ΓΨΓ|OU|Photographer|Skater|21|Work in progress|ENTP|Hopeless romantic..", "followers_count": 1421, "friends_count": 1320, "statues_count": 83711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berrien Springs, MI", "id": "003fcca428581309", "name": "Berrien Springs", "place_type": "city", "bounding_box": rectangle("-86.36849,41.89966 -86.305321,41.976116") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26021, "countyName": "Berrien", "cityID": 2607860, "cityName": "Berrien Springs" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029496246272, "text": "@Yungbets_7 show some love", "in_reply_to_status": 683138100579577858, "in_reply_to_user": 3188280715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3188280715 }}, "user": { "id": 3188280715, "name": "¥ung Bet$", "screen_name": "Yungbets_7", "lang": "en", "location": "California ✈ Oklahoma", "create_at": date("2015-05-07"), "description": "co 16 // sc: Mybitch96 // Del City #7 Soccer ⚽❤", "followers_count": 21848, "friends_count": 573, "statues_count": 526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del City, OK", "id": "7e831b14a06a5759", "name": "Del City", "place_type": "city", "bounding_box": rectangle("-97.459517,35.420738 -97.423502,35.478707") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4019900, "cityName": "Del City" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029609488384, "text": "@ThatBoyBrad_ nbs we gonna go check it out this weekend", "in_reply_to_status": 683192939472764928, "in_reply_to_user": 267254176, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 267254176 }}, "user": { "id": 354545071, "name": "YOUNG CEO™", "screen_name": "_ThatBoyEarnest", "lang": "en", "location": "DeadEnd Tx.", "create_at": date("2011-08-13"), "description": "sc: thatboyearnest Houston Tx EazyKill™", "followers_count": 1967, "friends_count": 1382, "statues_count": 42969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029714313216, "text": "Beautiful day spent at Manhattan Beach with my team! �� I've dreamed of getting back to the… https://t.co/JqjNWqUl7G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.36666567,33.8938806"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312780081, "name": "Brett Bolin", "screen_name": "Brettbo_", "lang": "en", "location": "Canada", "create_at": date("2011-06-07"), "description": "null", "followers_count": 191, "friends_count": 374, "statues_count": 2875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redondo Beach, CA", "id": "e1e35d357ceefa52", "name": "Redondo Beach", "place_type": "city", "bounding_box": rectangle("-118.401931,33.814689 -118.352695,33.894649") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 660018, "cityName": "Redondo Beach" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029844365313, "text": "my fingertips & my lips, they burn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609595990, "name": "Abby", "screen_name": "Abby_IsLame", "lang": "en", "location": "dd|been rockin & stay rockin", "create_at": date("2012-06-15"), "description": "he likes star wars i like cleaning supplies", "followers_count": 1063, "friends_count": 1035, "statues_count": 45315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196029903056896, "text": "Somebody else paying for my phone bill this month. Ain't nobody text me, why tf I gotta pay for it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1674295909, "name": "That one nigga", "screen_name": "youcurved", "lang": "en", "location": "null", "create_at": date("2013-08-15"), "description": "We out here doing big things, except for females", "followers_count": 515, "friends_count": 174, "statues_count": 78029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030200905728, "text": "Enough is enough! I'm sick of hearing about #LuchaUnderground from the outside. Episode 1 starts now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LuchaUnderground" }}, "user": { "id": 702409813, "name": "Thomas J Wilson", "screen_name": "TruthofThomasJ", "lang": "en", "location": "Seattle", "create_at": date("2012-07-17"), "description": "Inept brother of Slade and Wade. Not as witty as I think I am. #StayPositive", "followers_count": 139, "friends_count": 878, "statues_count": 1596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030565793792, "text": "@anyssagarvin your getting one!?!?��", "in_reply_to_status": 683194563620515840, "in_reply_to_user": 2331186792, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2331186792 }}, "user": { "id": 1429347673, "name": "ⒷⓇⒺ❥", "screen_name": "BreanneTabarez", "lang": "en", "location": "Probably Playing Softball ", "create_at": date("2013-05-14"), "description": "Senior| Phil 4:13 ✞ 10/4/15 B.A ♡", "followers_count": 280, "friends_count": 456, "statues_count": 7279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030712725509, "text": "������ https://t.co/8t6RtyVqqX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 156122631, "name": "Briana Spinks", "screen_name": "bri_2_timezz", "lang": "en", "location": "Chambana ✈️ New York", "create_at": date("2010-06-15"), "description": "#aintstressin", "followers_count": 2964, "friends_count": 3068, "statues_count": 43823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Drum, NY", "id": "69108214c4615b4d", "name": "Fort Drum", "place_type": "city", "bounding_box": rectangle("-75.838074,44.010422 -75.692237,44.073511") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36045, "countyName": "Jefferson", "cityID": 3626759, "cityName": "Fort Drum" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030767247360, "text": "01/02@03:00 - Temp 29.6F, WC 29.6F. Wind 0.0mph SW, Gust 2.0mph. Bar 30.128in, Falling slowly. Rain 0.00in. Hum 87%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 45968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030817562624, "text": "@Paliana_L you never called back doe��", "in_reply_to_status": 683195842258440192, "in_reply_to_user": 1659517874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1659517874 }}, "user": { "id": 372869751, "name": "Swaggy P", "screen_name": "Jeffrey_Allen1", "lang": "en", "location": "null", "create_at": date("2011-09-13"), "description": "Only the strong survive #Uptownbornandraised #PSU19", "followers_count": 1380, "friends_count": 1226, "statues_count": 7310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030821638147, "text": "Me cago en este idiota", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 554056263, "name": "❁", "screen_name": "Leeonnor", "lang": "es", "location": "Belén, Heredia, Costa Rica", "create_at": date("2012-04-14"), "description": "null", "followers_count": 486, "friends_count": 725, "statues_count": 6536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stafford, OR", "id": "014cf85423a61dd9", "name": "Stafford", "place_type": "city", "bounding_box": rectangle("-122.727138,45.356156 -122.647731,45.402513") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4169800, "cityName": "Stafford" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030989422592, "text": "Wow it's jan 2nd already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.29112572,33.87031422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594306460, "name": "Ayo", "screen_name": "ElijahSho", "lang": "en", "location": "DA Carson, CA", "create_at": date("2012-05-29"), "description": "#tracknation | #LakerNation #Loading... | Nigerian | Ruth ❤️", "followers_count": 1020, "friends_count": 390, "statues_count": 97504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196030997934080, "text": "0h 5m wait time at SPOTSYLVANIA REGIONAL MEDICAL CENTER. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.496195,38.2168371"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 375, "friends_count": 888, "statues_count": 2481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51033, "countyName": "Caroline" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031060725760, "text": "New years is over .. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2935519243, "name": "PrettyPrincess✨", "screen_name": "omaresaunders", "lang": "en", "location": "Somewhere Sitting Pretty Dab'n", "create_at": date("2014-12-18"), "description": "Content.", "followers_count": 610, "friends_count": 490, "statues_count": 5212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Elsinore, CA", "id": "80cf9987ff7e9762", "name": "Lake Elsinore", "place_type": "city", "bounding_box": rectangle("-117.413156,33.618447 -117.216785,33.731808") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 639486, "cityName": "Lake Elsinore" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031073308674, "text": "We didn't https://t.co/LmXwg7GMjJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 748189932, "name": "Wife ❤️❣", "screen_name": "x_thugbeauty", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-08-09"), "description": "It's only one me , so stop trying ✋ .... Cause you could never beat me with the finest help", "followers_count": 1971, "friends_count": 1677, "statues_count": 29980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muskegon, MI", "id": "012b416c2c776945", "name": "Muskegon", "place_type": "city", "bounding_box": rectangle("-86.339696,43.201538 -86.144095,43.264395") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26121, "countyName": "Muskegon", "cityID": 2656320, "cityName": "Muskegon" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031241043968, "text": "my sister really an orthodontist and she didn't even go to college for that field lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4165216573, "name": "Tehya$", "screen_name": "TAYK47_", "lang": "en", "location": "New Orleans, LA", "create_at": date("2015-11-07"), "description": "eat pretty , $leep pretty , kill everything XABIEL ☻", "followers_count": 1084, "friends_count": 298, "statues_count": 15595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marrero, LA", "id": "2bca99f2a5ba7da1", "name": "Marrero", "place_type": "city", "bounding_box": rectangle("-90.139232,29.853716 -90.08087,29.910044") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2248785, "cityName": "Marrero" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031580770306, "text": "Freaking rediculous https://t.co/ZpJOUV8bMh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 895722548, "name": "yoy", "screen_name": "_basicallyjoy", "lang": "en", "location": "eating", "create_at": date("2012-10-21"), "description": "Miguel, Esteban and Safeway Chinese food basically (it's ok I probs stalked u once too)", "followers_count": 212, "friends_count": 161, "statues_count": 10419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summit, WA", "id": "71363b539f29a337", "name": "Summit", "place_type": "city", "bounding_box": rectangle("-122.39766,47.157769 -122.325735,47.180946") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5368365, "cityName": "Summit" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031664799744, "text": "Wind 0.0 mph ---. Barometer 30.303 in, Falling slowly. Temperature 25.9 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031941652480, "text": "@ColinCowherd hey, Iowa butchered it. Big 10 ain't looking to hot, huh Colin?", "in_reply_to_status": 683048167089115136, "in_reply_to_user": 52529896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52529896 }}, "user": { "id": 458816550, "name": "billy", "screen_name": "DragonBillyZ", "lang": "en", "location": "UA", "create_at": date("2012-01-08"), "description": "Plotting my eventual world takeover since I was a wee lad. Amateur novelist.", "followers_count": 225, "friends_count": 104, "statues_count": 9700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsboro, AL", "id": "639bb2fb92904efd", "name": "Scottsboro", "place_type": "city", "bounding_box": rectangle("-86.096764,34.597525 -85.995355,34.695664") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1071, "countyName": "Jackson", "cityID": 168736, "cityName": "Scottsboro" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196031970971648, "text": "Temp: 69.8°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 30.07 in | Dewpoint: 69.5°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 23, "friends_count": 1, "statues_count": 164036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196032172298240, "text": "heartbroken !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 602442288, "name": "claw$", "screen_name": "chinkclaw", "lang": "en", "location": "soflo", "create_at": date("2012-06-07"), "description": "kinda lit but asian asf", "followers_count": 212, "friends_count": 200, "statues_count": 3457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plantation, FL", "id": "7df9a00dcf914d5e", "name": "Plantation", "place_type": "city", "bounding_box": rectangle("-80.330201,26.088262 -80.196833,26.160753") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1257450, "cityName": "Plantation" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196032356728832, "text": "I just want to understand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354725431, "name": "Sophie Gronkowski ✺", "screen_name": "SophGronk", "lang": "en", "location": "null", "create_at": date("2011-08-13"), "description": "keep your dreams big & the government small.", "followers_count": 503, "friends_count": 419, "statues_count": 21898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freeburg, IL", "id": "658d20fcbb7f92ed", "name": "Freeburg", "place_type": "city", "bounding_box": rectangle("-89.941214,38.406276 -89.885973,38.475935") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1727806, "cityName": "Freeburg" } }
+{ "create_at": datetime("2016-01-02T00:00:01.000Z"), "id": 683196032549781506, "text": "Up w nobody to talk too yet can't fall asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 511374361, "name": "Lindsey Mason", "screen_name": "LindseeyMason", "lang": "en", "location": "null", "create_at": date("2012-03-01"), "description": "null", "followers_count": 658, "friends_count": 409, "statues_count": 35277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor, MI", "id": "a9eb1b07c3864593", "name": "Taylor", "place_type": "city", "bounding_box": rectangle("-83.308978,42.181294 -83.228111,42.269861") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2679000, "cityName": "Taylor" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033107668992, "text": "Wind 1.0 mph W. Barometer 30.304 in, Falling. Temperature 34.2 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033279463424, "text": "\"but she's not you\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 401846913, "name": "maryssa myers", "screen_name": "MyersMaryssa", "lang": "en", "location": "your boys house ", "create_at": date("2011-10-30"), "description": "kind of a bitch, kind of a princess", "followers_count": 284, "friends_count": 125, "statues_count": 7018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willamina, OR", "id": "011874cc21d9dcfd", "name": "Willamina", "place_type": "city", "bounding_box": rectangle("-123.498931,45.069536 -123.469308,45.086397") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4182350, "cityName": "Willamina" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033308864512, "text": "@Jodie4513 thank you beautiful ��❤️��", "in_reply_to_status": 683195772750307329, "in_reply_to_user": 1950668682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1950668682 }}, "user": { "id": 2971138172, "name": "Bianca", "screen_name": "BiancaS25416827", "lang": "en", "location": "null", "create_at": date("2015-01-09"), "description": "15| CHS| Dancer| #72", "followers_count": 97, "friends_count": 71, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033317253120, "text": "you never do what you say ugh my god stop making promises", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2350743896, "name": "M", "screen_name": "mimi_trvn", "lang": "en", "location": "yay area", "create_at": date("2014-02-18"), "description": "runaway // get away", "followers_count": 219, "friends_count": 121, "statues_count": 3862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033359204352, "text": "I'm so happy ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437132392, "name": "th", "screen_name": "tuhreeree_", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-12-14"), "description": "IG:theressakaay | SC:ayoo_theressa | #BlackLivesMatter | goldklvn", "followers_count": 1122, "friends_count": 1328, "statues_count": 29837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033677918209, "text": "@lifeaseva #EvasHolidayGiveAway hoping my 2016 will be as beautiful as you!❤️���� HAPPY NEW YEAR EVA!!!❤️�������� https://t.co/QCpPkOFkQV", "in_reply_to_status": -1, "in_reply_to_user": 1079108184, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EvasHolidayGiveAway" }}, "user_mentions": {{ 1079108184 }}, "user": { "id": 3060777974, "name": "Sheila Koppel", "screen_name": "AlagdonSheila", "lang": "en", "location": "Roseville, CA", "create_at": date("2015-03-03"), "description": "Pain is real and so is Hope.", "followers_count": 20, "friends_count": 105, "statues_count": 1120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033749233664, "text": "Kodak Black , Lil Uzi ,Jazz Cartier & 21 savage is 2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 436260255, "name": "Vic", "screen_name": "VictorSweet__", "lang": "en", "location": "null", "create_at": date("2011-12-13"), "description": "null", "followers_count": 366, "friends_count": 193, "statues_count": 9612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Balch Springs, TX", "id": "9935c102706fb55a", "name": "Balch Springs", "place_type": "city", "bounding_box": rectangle("-96.639116,32.69025 -96.574335,32.748536") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4805372, "cityName": "Balch Springs" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033757626368, "text": "Baby u got it bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902720034, "name": "Jess", "screen_name": "Jaayv17", "lang": "en", "location": "garland, texas", "create_at": date("2012-10-24"), "description": "probably with Lorena", "followers_count": 491, "friends_count": 323, "statues_count": 23907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196033988456448, "text": "Even on your best days your still a piece of shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542719734, "name": "CAL", "screen_name": "caileystipe", "lang": "en", "location": "Sequim, WA", "create_at": date("2012-04-01"), "description": "null", "followers_count": 1022, "friends_count": 407, "statues_count": 11901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sequim, WA", "id": "7e05a40db75ea147", "name": "Sequim", "place_type": "city", "bounding_box": rectangle("-123.154156,48.054558 -123.039812,48.12363") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53009, "countyName": "Clallam", "cityID": 5363385, "cityName": "Sequim" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034063949825, "text": "Trying to get back to my high school weight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 203368533, "name": "Home Slice", "screen_name": "AyeeitsMilly", "lang": "en", "location": " D[M]V ", "create_at": date("2010-10-15"), "description": "Aspiring Make up artist and Photographer. Future Doctor -Nigerian and Ghanaian. I guess you can call me mixed ERHS Alumni '13 UMES", "followers_count": 853, "friends_count": 711, "statues_count": 67992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmore, MD", "id": "5f7b85a9049af1d0", "name": "Woodmore", "place_type": "city", "bounding_box": rectangle("-76.806927,38.900281 -76.73755,38.973106") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486710, "cityName": "Woodmore" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034248409088, "text": "@ayeeitsniko @WORIDSTARHIPHOP not really funny but aight.", "in_reply_to_status": 683195889104494592, "in_reply_to_user": 3270734959, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3270734959, 2913627307 }}, "user": { "id": 2602661105, "name": "apple juice alexa", "screen_name": "yagirlalexaa", "lang": "en", "location": "null", "create_at": date("2014-06-11"), "description": "null", "followers_count": 363, "friends_count": 313, "statues_count": 4200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034458095616, "text": "Temp: 39.6°F Wind:0.8mph Pressure: 30.391hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034579738626, "text": "Si así remozó así será todo, lo decreto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1434806858, "name": "NitziaCarrillo", "screen_name": "NitziaCarrillo", "lang": "es", "location": "null", "create_at": date("2013-05-16"), "description": "08/10 || 09/16", "followers_count": 366, "friends_count": 276, "statues_count": 18875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034726543360, "text": "@groovyxgooch nah I wished Ye or Rocky had a verse too but it's slept on cause it's like a mixtape to experiment new sounds", "in_reply_to_status": 683195660640759812, "in_reply_to_user": 1647524821, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1647524821 }}, "user": { "id": 1409064338, "name": "Mike Litoris", "screen_name": "Dylanexe", "lang": "en", "location": "null", "create_at": date("2013-05-06"), "description": "#BeautifulMind\nCSUF", "followers_count": 260, "friends_count": 280, "statues_count": 17544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034739253248, "text": "honestly https://t.co/0PIzDQ6QwX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327540338, "name": "ひ'cp3", "screen_name": "stanceluis", "lang": "en", "location": "miami", "create_at": date("2011-07-01"), "description": "junior", "followers_count": 556, "friends_count": 468, "statues_count": 10455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenacres, FL", "id": "be925761a3187f65", "name": "Greenacres", "place_type": "city", "bounding_box": rectangle("-80.175957,26.588782 -80.112394,26.678639") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1227322, "cityName": "Greenacres" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196034999128064, "text": "Remember when I live tweeted a boner", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2764850480, "name": "joe/travus/cletus", "screen_name": "TravusHertl", "lang": "en", "location": "Danville, CA", "create_at": date("2014-08-24"), "description": "#GoStanford | Country Music | I am 90% Coke & Chicken Wings | avid water drinker | go borwns", "followers_count": 2212, "friends_count": 1399, "statues_count": 136223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, CA", "id": "aa30747001a23f03", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-122.027412,37.779803 -121.89165,37.847751") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 617988, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196035276091392, "text": "sheeeesh https://t.co/JBiLwy9ZKy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2413334047, "name": "savann", "screen_name": "Savann18", "lang": "en", "location": "Lynn, MA", "create_at": date("2014-03-26"), "description": "IG: @savannven", "followers_count": 142, "friends_count": 45, "statues_count": 1821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynn, MA", "id": "3e711c16431907f4", "name": "Lynn", "place_type": "city", "bounding_box": rectangle("-71.01153,42.444954 -70.92218,42.51252") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2537490, "cityName": "Lynn" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196035309547521, "text": "You are all fucked up https://t.co/Y2amdLViKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 569605703, "name": "MONTANA", "screen_name": "Tristanrice12", "lang": "en", "location": "Weslaco, TX", "create_at": date("2012-05-02"), "description": "livin' life so trill/ mediocre producer/ senior at whs", "followers_count": 508, "friends_count": 506, "statues_count": 15747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weslaco, TX", "id": "f664c6f63c0bef35", "name": "Weslaco", "place_type": "city", "bounding_box": rectangle("-98.057773,26.111766 -97.942697,26.242157") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4877272, "cityName": "Weslaco" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196035720560640, "text": "I want friends that don't fall asleep at 9.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191618754, "name": "Jessica Hamilton", "screen_name": "hamilt0njessica", "lang": "en", "location": "null", "create_at": date("2015-05-10"), "description": "what do you mean the world doesn't revolve around me?", "followers_count": 87, "friends_count": 187, "statues_count": 338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Segundo, CA", "id": "0654b676d0359a31", "name": "El Segundo", "place_type": "city", "bounding_box": rectangle("-118.429843,33.901804 -118.370685,33.931493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622412, "cityName": "El Segundo" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196036177866752, "text": "Wind 3.1 mph W. Barometer 29.95 in, Steady. Temperature 29.5 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 26, "friends_count": 89, "statues_count": 156360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196036316176385, "text": "@TRlLLNlALL @3hunnaalexis no I don't", "in_reply_to_status": 683195907521798145, "in_reply_to_user": 365212246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 365212246, 846329899 }}, "user": { "id": 2832264288, "name": "Raphael", "screen_name": "MillerThaGod", "lang": "en", "location": "THIS AINT TWITTER, NIGGA", "create_at": date("2014-09-25"), "description": "Yall can't be this dense. ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀⠀⠀⠀ ⠀⠀⠀ Get them followers, boo. ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ Don't save her, she dont wanna be saved. ⠀⠀⠀⠀ mgt", "followers_count": 11325, "friends_count": 833, "statues_count": 106164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-02T00:00:02.000Z"), "id": 683196036421029890, "text": "LARRYBIRD# AVAILABLE NOW ON #DATPIFF AND #SOUNDCLOUD #LOOKOUT2 LINK IN BIO https://t.co/lWi3JJBXkl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DATPIFF", "SOUNDCLOUD", "LOOKOUT2" }}, "user": { "id": 2919908216, "name": "tha_boywonder", "screen_name": "bates713", "lang": "en", "location": "R$M BUISNESS ", "create_at": date("2014-12-05"), "description": "#LOOKOUT2 AVAILABLE NOW #DATPIFF #SOUNDCLOUD", "followers_count": 51, "friends_count": 99, "statues_count": 523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037129871361, "text": "But like why am I awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1161893983, "name": "Kayla Barton", "screen_name": "RealKaylaBarton", "lang": "en", "location": "null", "create_at": date("2013-02-08"), "description": "do re mi fa so done with school :-)", "followers_count": 577, "friends_count": 497, "statues_count": 3861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sycamore, IL", "id": "5bec0be4a090c019", "name": "Sycamore", "place_type": "city", "bounding_box": rectangle("-88.734193,41.962938 -88.655663,42.022065") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1774223, "cityName": "Sycamore" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037188599808, "text": "@xo_taylorbang @kayanachantler ����", "in_reply_to_status": 683195924034682880, "in_reply_to_user": 224328304, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 224328304, 1038466705 }}, "user": { "id": 224328304, "name": "Taylor Katherine♑", "screen_name": "xo_taylorbang", "lang": "en", "location": "Oceanside, CA", "create_at": date("2010-12-08"), "description": "I throw balls at people #22 on the field #1 in your heart :) RIP AMBER DUBOIS", "followers_count": 1442, "friends_count": 777, "statues_count": 46377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037347946497, "text": "erik ruined it because he didn't have his vest on, but hey our team won ���� https://t.co/hqZlyqPa5w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2709975752, "name": "es", "screen_name": "_sifuentesss", "lang": "en", "location": "null", "create_at": date("2014-08-05"), "description": "gotta focus on me for a while", "followers_count": 287, "friends_count": 242, "statues_count": 10758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, IL", "id": "c3f1deee8671c492", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-88.565674,41.645151 -88.487888,41.684729") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17093, "countyName": "Kendall", "cityID": 1760352, "cityName": "Plano" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037549273090, "text": "HAPPY NEW YEARS!!!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1602030866, "name": "georgina", "screen_name": "georginajlx", "lang": "en", "location": "null", "create_at": date("2013-07-17"), "description": "Assyrian.", "followers_count": 657, "friends_count": 340, "statues_count": 18673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037763219457, "text": "Add the new snap �� https://t.co/LUOgjgJDDP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921008214, "name": "raphael", "screen_name": "sergitron_", "lang": "en", "location": "Clearfield, UT", "create_at": date("2014-12-06"), "description": "CHS", "followers_count": 429, "friends_count": 196, "statues_count": 1771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, UT", "id": "d254694c82a4473e", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-112.05574,41.074662 -111.979637,41.126319") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4913850, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037767413760, "text": "@gusxsanchez you always have the sickest play list ��", "in_reply_to_status": 683195835107020800, "in_reply_to_user": 522011908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 522011908 }}, "user": { "id": 2819383135, "name": "Ivan", "screen_name": "Imejia18", "lang": "en", "location": "null", "create_at": date("2014-09-18"), "description": "Leo ♌️ SC: Imejia18", "followers_count": 246, "friends_count": 306, "statues_count": 10416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mirada, CA", "id": "4db990e393c2e28b", "name": "La Mirada", "place_type": "city", "bounding_box": rectangle("-118.037975,33.873395 -117.976352,33.928407") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640032, "cityName": "La Mirada" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196037897404417, "text": "Go best friend, that's my best friend! �� @saenz_leah https://t.co/IrhACHwoDb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2997325858 }}, "user": { "id": 252357458, "name": "bbygurl", "screen_name": "_lyssapissa", "lang": "en", "location": "null", "create_at": date("2011-02-14"), "description": "I'm the girl who prefers one rose instead of a dozen.", "followers_count": 573, "friends_count": 402, "statues_count": 12249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sahuarita, AZ", "id": "010e9b9269df1e78", "name": "Sahuarita", "place_type": "city", "bounding_box": rectangle("-111.004339,31.895159 -110.964233,32.001439") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 462140, "cityName": "Sahuarita" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038165835776, "text": "41.8F (Feels: 41.8F) - Humidity: 76% - Wind: 8.3mph N - Gust: 10.7mph - Pressure: 1042.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 220650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038233100288, "text": "Wtf https://t.co/m0z6AO4FsF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2272595707, "name": "E", "screen_name": "EthanJames64", "lang": "en", "location": "odessa tx", "create_at": date("2014-01-01"), "description": "OHS Football \nSnapchat - itsyourboye\nR.I.P DJ \nR.I.P Hunter\n#64 \nOL/RG\nUTA20", "followers_count": 281, "friends_count": 894, "statues_count": 3244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038404902913, "text": "Happy Birthday to me������♑️�� #January2nd #HappyBirthdayBree https://t.co/kJEDOE6OIQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "January2nd", "HappyBirthdayBree" }}, "user": { "id": 48201596, "name": "Brianna Siqueiros", "screen_name": "BriannaCkdos", "lang": "en", "location": "Wonderland♡", "create_at": date("2009-06-17"), "description": "Live your life, love who you are, laugh until you cry,and learn from your mistakes . †♡❁♫", "followers_count": 176, "friends_count": 290, "statues_count": 8132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038404947968, "text": "Happy 19th Birthday �� @itscolbybrock", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2488576712 }}, "user": { "id": 3145750125, "name": "Deh-knees", "screen_name": "denise_ruiz4", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-04-07"), "description": "† Spreading smiles like they’re STDs †", "followers_count": 1210, "friends_count": 1289, "statues_count": 1939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038505693185, "text": "My lil niggas sum savages ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1004511464, "name": "1/23♒️", "screen_name": "32st_Meech", "lang": "en", "location": "32nd n Dauphin ❗", "create_at": date("2012-12-11"), "description": "Humble Beast ® #Liv4Lil☝️sc; Meech_32st", "followers_count": 2981, "friends_count": 1153, "statues_count": 17767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196038677573632, "text": "@JG_OnTop LMAOOOOOO okay", "in_reply_to_status": 683195831239888898, "in_reply_to_user": 241536014, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 241536014 }}, "user": { "id": 301758325, "name": "KayDaMac", "screen_name": "_kaydeeblood", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "RestInParadiseTrey❤️ #SU19 iG: kasiaaRosee Sc: kaydeeblood....", "followers_count": 2218, "friends_count": 943, "statues_count": 64535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039038431232, "text": "we out to New York lmfaoooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718950455, "name": "reyes", "screen_name": "respectreyes_", "lang": "en", "location": "null", "create_at": date("2013-10-17"), "description": "snapchat: alegnamariee ✨", "followers_count": 658, "friends_count": 465, "statues_count": 28552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woburn, MA", "id": "a64cfaa1b1bfe4da", "name": "Woburn", "place_type": "city", "bounding_box": rectangle("-71.207236,42.449701 -71.111541,42.529352") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2581035, "cityName": "Woburn" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039080316928, "text": "Lmfao maybe he just fucked the wrong one raw. Which i can't understand ��. Its nothing to ask yo homie for a rubber���� https://t.co/XN11mJVIUH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2443952720, "name": "My name isnt Seff⚠️", "screen_name": "seth00reynolds", "lang": "en", "location": "Snap: @seth00reynolds", "create_at": date("2014-04-14"), "description": "God, Family, and Education #UWG18", "followers_count": 1022, "friends_count": 922, "statues_count": 8535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollton, GA", "id": "7bc33682b217b5a1", "name": "Carrollton", "place_type": "city", "bounding_box": rectangle("-85.128063,33.52676 -85.009976,33.633403") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13045, "countyName": "Carroll", "cityID": 1313492, "cityName": "Carrollton" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039105351681, "text": "Temp: 25.4°F - Dew Point: 15.7° - Wind: 9.8 mph - Gust: 15.2 - Rain Today: 0.00in. - Pressure: 30.10in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 11403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039122284544, "text": "Siping something , bumping big hypnotize with her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983377375, "name": ".", "screen_name": "AsiaAdore_", "lang": "en", "location": "null", "create_at": date("2012-12-01"), "description": "life ain't no rehearsal . | Uc bearcat '19 |", "followers_count": 1453, "friends_count": 1371, "statues_count": 42689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Oak, OH", "id": "f23ba07ebf2e9c82", "name": "White Oak", "place_type": "city", "bounding_box": rectangle("-84.637123,39.204409 -84.569018,39.232544") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3984812, "cityName": "White Oak" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039185051648, "text": "@dianaa_nunez got that right", "in_reply_to_status": 683195979315597313, "in_reply_to_user": 2928288876, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2928288876 }}, "user": { "id": 2715340553, "name": "Rebecca Bass", "screen_name": "m__becca21", "lang": "en", "location": "McFarland, CA", "create_at": date("2014-07-16"), "description": "#116", "followers_count": 375, "friends_count": 131, "statues_count": 14319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McFarland, CA", "id": "7ba4d06ff652fffc", "name": "McFarland", "place_type": "city", "bounding_box": rectangle("-119.240992,35.659929 -119.204633,35.68918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 644826, "cityName": "McFarland" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039327690752, "text": "HAPPY BIRTHDAY @jazminngomez_ ������������ https://t.co/2tk9XoeIQF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2469503816 }}, "user": { "id": 2981011790, "name": "Jann", "screen_name": "janette_f10", "lang": "en", "location": "null", "create_at": date("2015-01-13"), "description": "null", "followers_count": 115, "friends_count": 101, "statues_count": 804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039407390721, "text": "*screams wildly* https://t.co/enGhrZCXev", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476424110, "name": "Lagartija", "screen_name": "CatieTheRiveter", "lang": "en", "location": "null", "create_at": date("2013-06-01"), "description": "21 • Cuban/Venezuelan • Intersectional Feminist • Christian • Graphic Designer • Illustrator • dog & plant mom • spooky bitch", "followers_count": 471, "friends_count": 1071, "statues_count": 11312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789749") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039579451392, "text": "Honestly don't talk about it at all, it's all corrupt anyways", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1270373084, "name": "Noodell Beckham", "screen_name": "JohnnyToughnutz", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "http://www.hudl.com/athlete/2724611/highlights/319666383", "followers_count": 656, "friends_count": 390, "statues_count": 34634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hidden Valley, IN", "id": "76a4104da9162ff3", "name": "Hidden Valley", "place_type": "city", "bounding_box": rectangle("-84.877215,39.145597 -84.819864,39.209236") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18029, "countyName": "Dearborn", "cityID": 1833392, "cityName": "Hidden Valley" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039625486337, "text": "Personality! https://t.co/OBFdaKkwXj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509108967, "name": "ColorMe J-Lo", "screen_name": "JaLauraM", "lang": "en", "location": "Cameron, Tx", "create_at": date("2012-02-29"), "description": "Faith is taking the first step even when you cant see the whole staircase. -MLK #SHSU19 | sc: jalaura | instagram: colormejaiii_", "followers_count": 686, "friends_count": 634, "statues_count": 16224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron, TX", "id": "922185539a1a7a3b", "name": "Cameron", "place_type": "city", "bounding_box": rectangle("-97.003013,30.841928 -96.958853,30.882807") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48331, "countyName": "Milam", "cityID": 4812040, "cityName": "Cameron" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039772418048, "text": "zodiac signs be accurate af��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198373663, "name": "fckahoe!!", "screen_name": "tbvbby_", "lang": "en", "location": "w/karl❤️", "create_at": date("2013-11-27"), "description": "jada❤️.. nadeen❤️ tay❤️", "followers_count": 2431, "friends_count": 1184, "statues_count": 69588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039902281728, "text": "ANYWAYS.... I who's I could be skinny without having to work out to be how I want��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553142123, "name": "Ebs✌", "screen_name": "ebone_johnson", "lang": "en", "location": "null", "create_at": date("2012-04-13"), "description": "@instabagle is my happiness❤️❤️❤️❤️❤️ My angel is flying high❤️❤️❤️❤️❤️", "followers_count": 299, "friends_count": 203, "statues_count": 15930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terrace Heights, WA", "id": "45692c50065f2801", "name": "Terrace Heights", "place_type": "city", "bounding_box": rectangle("-120.48867,46.571035 -120.403874,46.623665") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5370805, "cityName": "Terrace Heights" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196039965339648, "text": "������ https://t.co/kKppWH09U1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3330593470, "name": "jamiee", "screen_name": "cox_jamison", "lang": "en", "location": "null", "create_at": date("2015-06-16"), "description": "cluelesss", "followers_count": 299, "friends_count": 396, "statues_count": 711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grove City, OH", "id": "073b88b74b72bb80", "name": "Grove City", "place_type": "city", "bounding_box": rectangle("-83.127063,39.791505 -83.008394,39.907441") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3932592, "cityName": "Grove City" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196040028147712, "text": "In & out or that last one �� https://t.co/Orw0Avv93q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169818509, "name": "Lorde", "screen_name": "ConniewithaK", "lang": "en", "location": "probably @ the batting cages⚾️", "create_at": date("2010-07-22"), "description": "connie w/ a K | 10.15.15 my baby blossy ❤️ R.I.P", "followers_count": 552, "friends_count": 251, "statues_count": 8215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelton, WA", "id": "017fbe104029934f", "name": "Shelton", "place_type": "city", "bounding_box": rectangle("-123.221681,47.172039 -123.026184,47.291156") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53045, "countyName": "Mason", "cityID": 5363735, "cityName": "Shelton" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196040082632704, "text": "@Food_You_Love @thekdawg24", "in_reply_to_status": 683190836666830848, "in_reply_to_user": 3246482806, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3246482806, 174337243 }}, "user": { "id": 722723276, "name": "Haleigh Armstrong", "screen_name": "haleighweakleg", "lang": "en", "location": "null", "create_at": date("2012-07-28"), "description": "Enjoy life", "followers_count": 225, "friends_count": 176, "statues_count": 1789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-02T00:00:03.000Z"), "id": 683196040502063104, "text": "Lowkeyyyyy https://t.co/Yhh7pb9761", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2894245985, "name": "kenszn.", "screen_name": "kennbeeeezzy", "lang": "en", "location": "null", "create_at": date("2014-11-26"), "description": "null", "followers_count": 250, "friends_count": 241, "statues_count": 5449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558412152487936, "text": "Down on the west coast they got a sayin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3794589493, "name": "Daddy", "screen_name": "jos_eye_uh", "lang": "en", "location": "null", "create_at": date("2015-10-05"), "description": "Just call me Daddy ✗♥O", "followers_count": 132, "friends_count": 91, "statues_count": 4486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558412798439425, "text": "�� https://t.co/FxPUipvRB9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2534466387, "name": "G-Phresh", "screen_name": "GrantKolacny", "lang": "en", "location": "null", "create_at": date("2014-05-07"), "description": "Bios are stupid af like Marissa", "followers_count": 474, "friends_count": 482, "statues_count": 1903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wharton, TX", "id": "1a0cccdc06c8c4fb", "name": "Wharton", "place_type": "city", "bounding_box": rectangle("-96.126829,29.297115 -96.059419,29.356075") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48481, "countyName": "Wharton", "cityID": 4878136, "cityName": "Wharton" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413083656193, "text": "My xbox one has turned into a $500 YouTube machine because I literally do nothing else", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1947483524, "name": "Doug Dimmadome", "screen_name": "TheeGDawg", "lang": "en", "location": "Under₩ater, TX", "create_at": date("2013-10-08"), "description": "شهيد من الملعونين", "followers_count": 1366, "friends_count": 921, "statues_count": 75819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413263966210, "text": "Wind 0.0 mph ---. Barometer 30.197 in, Falling. Temperature 42.3 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413331214336, "text": "Why am I listening to young thug at 3 in the morning? ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56422522, "name": "Mrs.Webster ❣", "screen_name": "priMOEdonna", "lang": "en", "location": "null", "create_at": date("2009-07-13"), "description": "@trvisXX ❤️ #UMES19", "followers_count": 1192, "friends_count": 995, "statues_count": 81911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Spring, MD", "id": "6417871953fa5e86", "name": "Silver Spring", "place_type": "city", "bounding_box": rectangle("-77.064086,38.979735 -76.97162,39.036964") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2472450, "cityName": "Silver Spring" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413452705792, "text": "Duuuuuude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2334757872, "name": "Johnny", "screen_name": "LivingJohnny", "lang": "en", "location": "null", "create_at": date("2014-02-09"), "description": "M+J.\nlove the jams..\nI like drugs too.", "followers_count": 30, "friends_count": 123, "statues_count": 866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413591261184, "text": "@mistachrish @warriors the Grizzlies? why? any other team ������", "in_reply_to_status": 683557182625951744, "in_reply_to_user": 2923007800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2923007800, 26270913 }}, "user": { "id": 1652082325, "name": "刘易斯克尚", "screen_name": "krishanlewis", "lang": "en", "location": "Dumfries, VA ", "create_at": date("2013-08-06"), "description": "singing and piano☺️ wanna be famous... TURN UPP Dumfries, VA ✈️ Philadelphia, PA", "followers_count": 278, "friends_count": 1140, "statues_count": 3616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dumfries, VA", "id": "ff38a99e25182d58", "name": "Dumfries", "place_type": "city", "bounding_box": rectangle("-77.339279,38.553903 -77.306215,38.57904") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5123760, "cityName": "Dumfries" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558413914271745, "text": "Ripley SW Limestone Co. Temp: 33.3°F Wind:1.6mph Pressure: 1001.8mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 46955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414044250112, "text": "Tonight I learned how to play craps. And win. @AlexPorter82 @bdmattingly @Cardsfan1414 https://t.co/uXhddLTj0k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24827000, 41465364, 161014817 }}, "user": { "id": 281696530, "name": "Tommy Spalding", "screen_name": "TommySpalding", "lang": "en", "location": "Louisville, KY", "create_at": date("2011-04-13"), "description": "Loves automotive marketing. | Flinches at pop when biscut cans open. | Huffs Febreeze. #GoCards #GoPackGo", "followers_count": 372, "friends_count": 268, "statues_count": 14700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashwaubenon, WI", "id": "04ab128abc58632a", "name": "Ashwaubenon", "place_type": "city", "bounding_box": rectangle("-88.151014,44.445096 -88.037709,44.510585") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5503425, "cityName": "Ashwaubenon" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414065205248, "text": "change is coming.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4414277295, "name": "Savannah Ballenger", "screen_name": "shballenger", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "null", "followers_count": 62, "friends_count": 67, "statues_count": 44 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, GA", "id": "7854f44686c5c527", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-83.488508,34.183151 -83.43322,34.242653") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13157, "countyName": "Jackson", "cityID": 1319112, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414258061312, "text": "ismael just said that i gas lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146506813, "name": "cat.", "screen_name": "__catt17", "lang": "en", "location": "California, USA", "create_at": date("2015-04-09"), "description": "null", "followers_count": 351, "friends_count": 350, "statues_count": 3689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera Acres, CA", "id": "75161a014946f779", "name": "Madera Acres", "place_type": "city", "bounding_box": rectangle("-120.109807,36.98237 -120.056201,37.03494") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645050, "cityName": "Madera Acres" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414270726144, "text": "1st sin of the year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.08033848,40.622058"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550550983, "name": "Omolope", "screen_name": "RunMyRace_", "lang": "en", "location": "NY", "create_at": date("2012-04-10"), "description": "Royalty is sweet✨#T4L. SBU'18", "followers_count": 585, "friends_count": 540, "statues_count": 12020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414555942912, "text": "Naw, dick get hard when I THINK you looking at it�� https://t.co/ChplZZ7jTA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384426280, "name": "Ordes D. Speaks III", "screen_name": "_YesImIgnorant", "lang": "en", "location": "Philadelphia , Pa", "create_at": date("2011-10-03"), "description": "PHI✈️ATL Morehouse18", "followers_count": 1385, "friends_count": 1233, "statues_count": 36948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Harbor, NY", "id": "ecff2a43d54be774", "name": "Baldwin Harbor", "place_type": "city", "bounding_box": rectangle("-73.617199,40.621497 -73.586895,40.642484") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3604154, "cityName": "Baldwin Harbor" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414719533056, "text": "Haitian ���� https://t.co/3lEYgVrto2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "eu", "is_retweet": false, "user": { "id": 513180906, "name": "It ain't Ralph tho", "screen_name": "CasanovaGhetto", "lang": "en", "location": "null", "create_at": date("2012-03-03"), "description": "CCSU", "followers_count": 406, "friends_count": 376, "statues_count": 7906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterbury, CT", "id": "f9a4ed4d94a9b93f", "name": "Waterbury", "place_type": "city", "bounding_box": rectangle("-73.093845,41.513971 -72.950371,41.619186") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 980000, "cityName": "Waterbury" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414836875264, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2747888072, "name": "Kait", "screen_name": "basedg0at", "lang": "en", "location": "United States", "create_at": date("2014-08-19"), "description": "Kait (Kaitlyn) // 17 // Oklahoma.", "followers_count": 1961, "friends_count": 2137, "statues_count": 3930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weatherford, OK", "id": "7da27eb58e1638f5", "name": "Weatherford", "place_type": "city", "bounding_box": rectangle("-98.725251,35.517222 -98.626215,35.558904") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40039, "countyName": "Custer", "cityID": 4079450, "cityName": "Weatherford" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558414933331969, "text": "@abby_gines ������", "in_reply_to_status": 683558158929756160, "in_reply_to_user": 1468740710, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1468740710 }}, "user": { "id": 1188615271, "name": "Tristan Michelsen", "screen_name": "InfamousTDizzle", "lang": "en", "location": "Utah", "create_at": date("2013-02-16"), "description": "Be Yourselves, Be Humble, and Be Grateful for all the blessings in your life. #JANUP 06/11/15 ❤️ 11/05/15 ❤️ #neverforget Counting down the days till @Logic301", "followers_count": 788, "friends_count": 1053, "statues_count": 8282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lehi, UT", "id": "ed3b23f667186d1f", "name": "Lehi", "place_type": "city", "bounding_box": rectangle("-111.914775,40.357876 -111.818332,40.456264") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4944320, "cityName": "Lehi" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558415109459968, "text": "don't be a \"lets hang out & if you say you busy ima say all my friends fake.\" bitch. i have no time or patience for that childish bs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3160200450, "name": "betsy", "screen_name": "xobetsy_", "lang": "en", "location": "lost in the sauce", "create_at": date("2015-04-16"), "description": "null", "followers_count": 137, "friends_count": 133, "statues_count": 3222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558415684268032, "text": "My mom called and asked if I can drive out to Mentor to give her a spare key at 3am.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367998269, "name": "Sam Senn ☪", "screen_name": "SamiSenn_", "lang": "en", "location": "Euclid, OH", "create_at": date("2011-09-04"), "description": "Illustrator | Wouldn't you love to know?", "followers_count": 461, "friends_count": 124, "statues_count": 21550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euclid, OH", "id": "59ed3d5f720ebdf0", "name": "Euclid", "place_type": "city", "bounding_box": rectangle("-81.552943,41.545336 -81.488014,41.631584") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3925704, "cityName": "Euclid" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558415709241344, "text": "@jimenez__8 callate que me ofendes", "in_reply_to_status": 683558323522650112, "in_reply_to_user": 2689063243, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2689063243 }}, "user": { "id": 1123843891, "name": "Joceline♡", "screen_name": "joocelinee_", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "♡", "followers_count": 310, "friends_count": 281, "statues_count": 33758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558415755431936, "text": "���� he did !!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317361803, "name": "Tonii ! ♥ ツ", "screen_name": "LoveHoneyyCakes", "lang": "en", "location": "love (: ", "create_at": date("2011-06-14"), "description": "Hi call me Tonii .. SkylineU ! IG:JaMaya_Lovee .", "followers_count": 680, "friends_count": 513, "statues_count": 34765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558415952515072, "text": "Happy birthday to this negra. You can't smash minors anymore dawg @chriscasil_ https://t.co/LVArioCRVN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2822572424 }}, "user": { "id": 2560564621, "name": "Mandeep Chhabra", "screen_name": "mandizzay", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "Ball is Life... and Music, Ukulele, Food and Cars #gt86", "followers_count": 170, "friends_count": 181, "statues_count": 525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558416019685376, "text": "I think someone kidnapped my rollers ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 524829179, "name": "pre$h", "screen_name": "preciousfaithxo", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-03-14"), "description": "bae area", "followers_count": 715, "friends_count": 234, "statues_count": 34937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-03T00:00:00.000Z"), "id": 683558416120324096, "text": "@Piiiikaachu electrode? This is bullshit", "in_reply_to_status": 683544814814531584, "in_reply_to_user": 785741952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 785741952 }}, "user": { "id": 537614536, "name": "weston parks", "screen_name": "Bipolarcucumber", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "I just really think we can settle this with some nice gentle butt play.", "followers_count": 134, "friends_count": 319, "statues_count": 1573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Poway, CA", "id": "b0e7385b9c530ddc", "name": "Poway", "place_type": "city", "bounding_box": rectangle("-117.084304,32.927402 -116.985751,33.043179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 658520, "cityName": "Poway" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558416317415424, "text": "I do not miss high school one bit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 757926433, "name": "Alyssa ⭐️", "screen_name": "johnson_lyss", "lang": "en", "location": "null", "create_at": date("2012-08-14"), "description": "Don't wait for the perfect moment, take a moment and make it perfect ⚓️", "followers_count": 335, "friends_count": 297, "statues_count": 6173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arnold, MO", "id": "fbaeb390ee6f1f24", "name": "Arnold", "place_type": "city", "bounding_box": rectangle("-90.462274,38.386208 -90.336225,38.462857") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson", "cityID": 2901972, "cityName": "Arnold" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558416619560961, "text": "Lmfao let me chill heart disease runs in my family ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2656053247, "name": "Amma", "screen_name": "amensah1004", "lang": "en", "location": "NJ/MD", "create_at": date("2014-07-18"), "description": "Stevenson University '19", "followers_count": 157, "friends_count": 158, "statues_count": 1028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colonia, NJ", "id": "dfa88de1ece1a505", "name": "Colonia", "place_type": "city", "bounding_box": rectangle("-74.338445,40.574326 -74.29037,40.608825") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3414380, "cityName": "Colonia" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558416896282624, "text": "@valerievintage @soniiyeahh https://t.co/QzONwZq45c", "in_reply_to_status": 683558305650741249, "in_reply_to_user": 34195328, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 34195328, 2750307400 }}, "user": { "id": 354098339, "name": "Lily Nuno", "screen_name": "LilyNuno", "lang": "en", "location": "null", "create_at": date("2011-08-12"), "description": "Gerardo Nava ❤️❤️❤️❤️", "followers_count": 178, "friends_count": 147, "statues_count": 4472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558416988696576, "text": "My cousin is lucky I love him lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386226752, "name": "Dequindre Davis", "screen_name": "DequindreLDavis", "lang": "en", "location": "(989) (210) On My Journey. ", "create_at": date("2011-10-06"), "description": "a Bay City kid bound to be the greatest, thats a hell of a start. R.I.P. to my aunt Angie #NothingGivenEverythingEarned #WeAreOne", "followers_count": 341, "friends_count": 155, "statues_count": 48823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay City, MI", "id": "0121f9435fdae948", "name": "Bay City", "place_type": "city", "bounding_box": rectangle("-83.97498,43.538242 -83.842876,43.696322") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26017, "countyName": "Bay", "cityID": 2606020, "cityName": "Bay City" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417277964290, "text": "keep taking better care of myself and serve life changing looks https://t.co/9vuB2OklFb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1486284715, "name": "lil space girl", "screen_name": "unlitcreature", "lang": "en", "location": "null", "create_at": date("2013-06-05"), "description": "smile (◠‿◠✿) ::::October 6th 2015 |-/ :::", "followers_count": 1258, "friends_count": 1216, "statues_count": 21372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockbridge, GA", "id": "013e811145e07117", "name": "Stockbridge", "place_type": "city", "bounding_box": rectangle("-84.293073,33.478656 -84.168228,33.575396") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13151, "countyName": "Henry", "cityID": 1373704, "cityName": "Stockbridge" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417294692354, "text": "Fourth winner: #OnePiece. Not really threatened by the bad guys, but it's nice to see Luffy being the hero & the crew kicking ass.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OnePiece" }}, "user": { "id": 16859893, "name": "DigiRanma", "screen_name": "DigiRanma", "lang": "en", "location": "Woodstock, IL", "create_at": date("2008-10-19"), "description": "An anime fan who watches and reviews bad movies... why, I don't know.", "followers_count": 439, "friends_count": 233, "statues_count": 62493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodstock, IL", "id": "6ce71712d41e1929", "name": "Woodstock", "place_type": "city", "bounding_box": rectangle("-88.491765,42.270987 -88.392355,42.350431") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1783349, "cityName": "Woodstock" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417345036288, "text": "Only one ever excited to see me �� https://t.co/RujzhzxJWI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 106327459, "name": ".MARIANNA.", "screen_name": "MariannaZee", "lang": "en", "location": "California, USA", "create_at": date("2010-01-18"), "description": "#GoIrish #BearDown", "followers_count": 296, "friends_count": 104, "statues_count": 49291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut, CA", "id": "2fd93ef04260afec", "name": "Walnut", "place_type": "city", "bounding_box": rectangle("-117.895753,34.00286 -117.825003,34.063424") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683332, "cityName": "Walnut" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417424711680, "text": "Life is like a dice game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237382556, "name": "joshua", "screen_name": "ICER90301", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "f// Instagram @ josh.724c", "followers_count": 131, "friends_count": 235, "statues_count": 212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, CA", "id": "b2fda7385aa963dd", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-118.317776,33.930952 -118.291549,33.959516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684592, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417651372032, "text": "\"bump who's the baddest a person's status depends on salary...\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281632553, "name": "lyricsofvalue", "screen_name": "MicCorleone904", "lang": "en", "location": "West Jax", "create_at": date("2011-04-13"), "description": "Last person I had a bond wit was my bail bondsman...", "followers_count": 177, "friends_count": 212, "statues_count": 13913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417940766720, "text": "Temp: 58.6°F | Humidity: 70% | Wind: --- @ 0.0 mph | Barometer: 30.07 in | Dewpoint: 48.9°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 23, "friends_count": 1, "statues_count": 164060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558417978527744, "text": "And I said \"what about Hotel Rwanda\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3105876149, "name": "hali palombo", "screen_name": "haliiscool", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-03-22"), "description": "null", "followers_count": 134, "friends_count": 261, "statues_count": 1261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418255224832, "text": "@haleyeahfnright I'm happy for you! Lol it's not broken or anything?", "in_reply_to_status": 683520415814127616, "in_reply_to_user": 219559713, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219559713 }}, "user": { "id": 32802943, "name": "Kessie", "screen_name": "kessxo", "lang": "en", "location": "Mesquite, TX", "create_at": date("2009-04-17"), "description": "Alexander♡08/06/2015", "followers_count": 357, "friends_count": 249, "statues_count": 7497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418414632960, "text": "@Trippy_Deezy ✊����������", "in_reply_to_status": 683558298142949376, "in_reply_to_user": 355225640, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 355225640 }}, "user": { "id": 2206337155, "name": "COCO", "screen_name": "chlo_aguilvr", "lang": "en", "location": "null", "create_at": date("2013-11-20"), "description": "Itz me", "followers_count": 546, "friends_count": 626, "statues_count": 11271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Banning, CA", "id": "792551bc9bd3c992", "name": "Banning", "place_type": "city", "bounding_box": rectangle("-116.947005,33.902607 -116.849846,33.94771") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 603820, "cityName": "Banning" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418418827264, "text": "Mmmmmmmmm hypocrite https://t.co/8iQC4TzK54", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 426121032, "name": "20K K€€F", "screen_name": "Og_Rowe", "lang": "en", "location": "Everywhere, but mostly Texa$", "create_at": date("2011-12-01"), "description": "I know I'm Tall don't ask my height #SmoothHive ✌️Gang|20kEnt", "followers_count": 1084, "friends_count": 205, "statues_count": 19047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, TX", "id": "012a07a105d10298", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-97.186747,32.530958 -97.046821,32.616182") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4846452, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418452369408, "text": "Ain't a bar that can make me buzz the way that you do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2903123533, "name": "Alexis $andoval", "screen_name": "_35alexis", "lang": "en", "location": "Mission, TX", "create_at": date("2014-11-17"), "description": "idea mission// varsity basketball", "followers_count": 177, "friends_count": 91, "statues_count": 3082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission, TX", "id": "77633125ba089dcb", "name": "Mission", "place_type": "city", "bounding_box": rectangle("-98.363219,26.155046 -98.272146,26.262558") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4848768, "cityName": "Mission" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418456641536, "text": "01/03@03:00 - Temp 27.0F, WC 27.0F. Wind 0.9mph S, Gust 2.0mph. Bar 29.968in, Falling slowly. Rain 0.00in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 45992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558418854973440, "text": "42-43... What the fuck!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1512365977, "name": "payton", "screen_name": "llimon9299", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "I say dude a lot", "followers_count": 161, "friends_count": 187, "statues_count": 1561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419266048004, "text": "Ok chill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491924508, "name": "Nate Davis", "screen_name": "Nateisgreat4u", "lang": "en", "location": "TX | WMR1514", "create_at": date("2012-02-13"), "description": "null", "followers_count": 1541, "friends_count": 723, "statues_count": 56399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419349913601, "text": "Weirdos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3387714449, "name": "Bianca♠️", "screen_name": "ambitious_beeee", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "VCU • Political Science Public Policy & Administration", "followers_count": 201, "friends_count": 252, "statues_count": 4395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419408658432, "text": "Happy birthday to my best friend.I Love you so much,have an amazing birthday seester.❤️❤️❤️��������@katie_reichel https://t.co/zM1NYUTB9E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2608307154 }}, "user": { "id": 3238133502, "name": "Kacie Danielle", "screen_name": "KacieDanielle33", "lang": "en", "location": "California, USA", "create_at": date("2015-06-06"), "description": "Tear me to pieces, sell me for parts you're all vampires so here, you can have my heart. ♍️", "followers_count": 130, "friends_count": 434, "statues_count": 1214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Valley, CA", "id": "0196f2d33e5a1d73", "name": "French Valley", "place_type": "city", "bounding_box": rectangle("-117.137136,33.564564 -117.083609,33.627328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 626067, "cityName": "French Valley" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419408670720, "text": "@TheseSavages will you give me a cigarette?", "in_reply_to_status": 683558165107978240, "in_reply_to_user": 2577309373, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2577309373 }}, "user": { "id": 2305465070, "name": "lieutenant Dan凸^-^凸", "screen_name": "TreyB420", "lang": "en", "location": "somewhere in space ️", "create_at": date("2014-01-22"), "description": "Fuck what they think", "followers_count": 363, "friends_count": 994, "statues_count": 2099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sahuarita, AZ", "id": "010e9b9269df1e78", "name": "Sahuarita", "place_type": "city", "bounding_box": rectangle("-111.004339,31.895159 -110.964233,32.001439") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 462140, "cityName": "Sahuarita" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419534639104, "text": "Wind 0.0 mph NW. Barometer 30.129 in, Steady. Temperature 35.6 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558419828101120, "text": "@djcarnage who is she? https://t.co/WoqAvxnzsz", "in_reply_to_status": -1, "in_reply_to_user": 18980519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18980519 }}, "user": { "id": 22948913, "name": "K Theory Oh My God", "screen_name": "ktheorymusic", "lang": "en", "location": "SF | LA | NJ ", "create_at": date("2009-03-05"), "description": "#KTHEORYOMG reppin @newtrinitymusic @_malcolmanthony @deelanzmusic eyez iz red • wilcox@warpathgroup.com for shows", "followers_count": 6960, "friends_count": 1332, "statues_count": 14687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558420062978048, "text": "@maganstow youre something else lol", "in_reply_to_status": 683558155851153408, "in_reply_to_user": 1756022330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1756022330 }}, "user": { "id": 862104074, "name": "Chico", "screen_name": "imcristianhoe", "lang": "en-gb", "location": "null", "create_at": date("2012-10-04"), "description": "I like dippin dots", "followers_count": 648, "friends_count": 623, "statues_count": 53729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-01-03T00:00:01.000Z"), "id": 683558420386025472, "text": "Is this even them LOL https://t.co/wizxhTBjPG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2884874145, "name": "Gina Mooney", "screen_name": "pinkukitten", "lang": "en", "location": "PDX/FL", "create_at": date("2014-11-19"), "description": "I like to pet animals and play animal crossing IG: @gina.mooney / snap: @ginamooney", "followers_count": 3838, "friends_count": 597, "statues_count": 26943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420515917824, "text": "12:00 A.M.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1706298422, "name": "The Bitch", "screen_name": "brinna_sue", "lang": "en", "location": "null", "create_at": date("2013-08-27"), "description": "KHS 2017~Certified Lifeguard~Swimmer ~Cj is my everything~9/11/14~", "followers_count": 310, "friends_count": 325, "statues_count": 6136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420603994112, "text": "She's from the jungle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439765162, "name": "geeko", "screen_name": "northwestpow", "lang": "en", "location": "null", "create_at": date("2014-04-12"), "description": "they don't want you to read this bio", "followers_count": 440, "friends_count": 587, "statues_count": 2579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woods Creek, WA", "id": "1e4d7645f75e58f2", "name": "Woods Creek", "place_type": "city", "bounding_box": rectangle("-121.961128,47.854754 -121.863118,47.903499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5379825, "cityName": "Woods Creek" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420612575232, "text": "Sisters by chance,\nFriends my choice???????? https://t.co/G9hixuUzWY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 736153567, "name": "Jasmine Thu Hulse", "screen_name": "Jazaloo", "lang": "en", "location": "Moral Grey Area, CA", "create_at": date("2012-08-03"), "description": "I like animals, noodles, and ethics. #SJSU . #JusticeStudies Major, #Philosophy Minor. #KappaDelta , #Seattle, #Starbucks partner.", "followers_count": 387, "friends_count": 718, "statues_count": 3628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420734177281, "text": "Same https://t.co/4rZRHuehPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1064242400, "name": "ѕтepнanιe ѕaмpaιo♡", "screen_name": "LollStephh", "lang": "en", "location": "south florida", "create_at": date("2013-01-05"), "description": "| volleyвall | brasileira | snapchat- lolstephh |", "followers_count": 1162, "friends_count": 2028, "statues_count": 21378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coconut Creek, FL", "id": "0af0a491a5194920", "name": "Coconut Creek", "place_type": "city", "bounding_box": rectangle("-80.203196,26.213436 -80.152557,26.327949") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1213275, "cityName": "Coconut Creek" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420876677120, "text": ", really wish my brother still stayed down here ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3051386269, "name": "05|09", "screen_name": "skycollins02", "lang": "en", "location": "null", "create_at": date("2015-02-28"), "description": ", immixed • ♉ • WOS", "followers_count": 201, "friends_count": 148, "statues_count": 4601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, TX", "id": "3fe7643d3e014f67", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-93.858146,30.061031 -93.715588,30.193051") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48361, "countyName": "Orange", "cityID": 4854132, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558420939603968, "text": "Pop a lot of xans and I'm really triply", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332705687, "name": "tabius . . .", "screen_name": "719based", "lang": "en", "location": "null", "create_at": date("2014-02-07"), "description": "im broke baby", "followers_count": 955, "friends_count": 552, "statues_count": 14713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421178650624, "text": "@AKAsya_08 lol is this a one time thing��", "in_reply_to_status": 683558199471943682, "in_reply_to_user": 292778286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 292778286 }}, "user": { "id": 441513218, "name": "EVERYBODY HATES CEDD", "screen_name": "CEDD_3", "lang": "en", "location": "Langston University ✈️", "create_at": date("2011-12-19"), "description": "{5'8 }A..... Legendary........ Sensation............ |ΑΦΑ|ΒΚ|Tr3 club | http://www.WrongShortNigga.com COO of NEEDsTheBrand", "followers_count": 1096, "friends_count": 883, "statues_count": 18400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421212217348, "text": "I love magic ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42334362, "name": "SKR", "screen_name": "ShaniaKelice", "lang": "en", "location": "New Jersey", "create_at": date("2009-05-24"), "description": "idk when I'm graduating but I'm a journalism major", "followers_count": 1256, "friends_count": 1108, "statues_count": 46131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421224792064, "text": "It's easy to make things look pretty these days..,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540374377, "name": "_", "screen_name": "MattPaige1", "lang": "en", "location": "null", "create_at": date("2012-03-29"), "description": "NW Pioneer", "followers_count": 277, "friends_count": 251, "statues_count": 2638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastmont, WA", "id": "002d396055a6adf8", "name": "Eastmont", "place_type": "city", "bounding_box": rectangle("-122.234054,47.860402 -122.14427,47.921824") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5319630, "cityName": "Eastmont" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421258354689, "text": "@pattonoswalt I sure did enjoy your interview with Tom Ashbrook. The Jerk was funny. Just thinking about that. Amiright?", "in_reply_to_status": -1, "in_reply_to_user": 139162440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139162440 }}, "user": { "id": 23508475, "name": "Katherine Kaufman", "screen_name": "coachkitty", "lang": "en", "location": "Seattle", "create_at": date("2009-03-09"), "description": "Thinker/Trainer/Person/Pet Mom. Group fitness Instructor/Presenter. Big Ass Pats/Sox/Celtics/Bruins Fan", "followers_count": 419, "friends_count": 1108, "statues_count": 7176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421736509440, "text": "I'm ALMOST drubk enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1110387289, "name": "Xavier", "screen_name": "XavierThaGr8", "lang": "en", "location": "SanDiego Ca", "create_at": date("2013-01-21"), "description": "null", "followers_count": 348, "friends_count": 184, "statues_count": 21778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421774204928, "text": "*Isabel fake crying*\n*Ava & I playing a game* \n*Isabel stops in mid-cry*\n\"Nobody cares about me!\" \n*continues to cry*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 561447080, "name": "farrah", "screen_name": "farrah_lieb", "lang": "en", "location": "Alaska", "create_at": date("2012-04-23"), "description": "know yourself, know your worth", "followers_count": 146, "friends_count": 116, "statues_count": 548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethel, AK", "id": "012a7c6f33fb71bf", "name": "Bethel", "place_type": "city", "bounding_box": rectangle("-161.829671,60.758658 -161.720355,60.808482") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2050, "countyName": "Bethel", "cityID": 206520, "cityName": "Bethel" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421795192832, "text": "@King_Louie9 Happy Birthday ������", "in_reply_to_status": -1, "in_reply_to_user": 2250668568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2250668568 }}, "user": { "id": 1102435916, "name": "Ⓨⓐⓗⓡⓔⓛⓘ", "screen_name": "yareligaytan2", "lang": "en", "location": "null", "create_at": date("2013-01-18"), "description": "null", "followers_count": 835, "friends_count": 736, "statues_count": 7127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atwater, CA", "id": "676c3e6a1e0642db", "name": "Atwater", "place_type": "city", "bounding_box": rectangle("-120.641299,37.325641 -120.555357,37.374957") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 603162, "cityName": "Atwater" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558421937913856, "text": "Wind 0.0 mph SSW. Barometer 30.09 in, Falling slowly. Temperature 30.6 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 305979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422076207108, "text": "Silence is the loudest thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 813672013, "name": "K Cox❤️", "screen_name": "lorrd__farquad", "lang": "en", "location": "OHS '17", "create_at": date("2012-09-09"), "description": "It's a great day to save lives", "followers_count": 601, "friends_count": 571, "statues_count": 9330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422088794113, "text": "you'll never know a good thing till it's gone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1916339353, "name": "lynx tabujara", "screen_name": "lynxttt", "lang": "en", "location": "Pearland, TX", "create_at": date("2013-09-28"), "description": "*Amboy *Medyo law-ay *LDS *Athazagoraphobic *instagram: @lynxttt *fb: Lynx Tabujara", "followers_count": 285, "friends_count": 97, "statues_count": 851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422294351873, "text": "@SuperBoo_29 yes he lives across the street ��", "in_reply_to_status": 683558320402124800, "in_reply_to_user": 342489883, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342489883 }}, "user": { "id": 621387380, "name": "Amanda Angulo", "screen_name": "_mandersssss", "lang": "en", "location": "Yuma, AZ", "create_at": date("2012-06-28"), "description": "Leo Pollo", "followers_count": 1646, "friends_count": 747, "statues_count": 54298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422369808384, "text": "Forgot to do awards for all of my favorite songs of last year.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20137956, "name": "Dominick", "screen_name": "DVMINIXK", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-05"), "description": "It's your world. #NT", "followers_count": 474, "friends_count": 473, "statues_count": 64841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422529245184, "text": "Temp: 22.0°F - Dew Point: 14.7° - Wind: 11.3 mph - Gust: 15.2 - Rain Today: 0.00in. - Pressure: 29.96in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 11451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422592139268, "text": "@THEDINGOJOHNSON you mad or nah", "in_reply_to_status": 683558289389428736, "in_reply_to_user": 293647754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 293647754 }}, "user": { "id": 226659570, "name": "Han Solo", "screen_name": "ifihada_NICK_el", "lang": "en", "location": "null", "create_at": date("2010-12-14"), "description": "I'd EpaR Rihanna", "followers_count": 1806, "friends_count": 498, "statues_count": 137662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mojave, CA", "id": "00b5133757bbf988", "name": "Mojave", "place_type": "city", "bounding_box": rectangle("-118.198458,35.024858 -118.14495,35.070099") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 648452, "cityName": "Mojave" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422625808384, "text": "#cleROCKY https://t.co/VTSIYAAUd9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "cleROCKY" }}, "user": { "id": 400856464, "name": "kalley", "screen_name": "k_poostofficee", "lang": "en", "location": "null", "create_at": date("2011-10-29"), "description": "grand optimist", "followers_count": 1054, "friends_count": 2038, "statues_count": 20778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parma, OH", "id": "dbd1fd3168748e88", "name": "Parma", "place_type": "city", "bounding_box": rectangle("-81.785053,41.350358 -81.68457,41.419489") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3961000, "cityName": "Parma" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422722248704, "text": "!! https://t.co/0K1x7bANfh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1491569989, "name": "pizza rat", "screen_name": "thegreyrai", "lang": "en", "location": "California, USA", "create_at": date("2013-06-07"), "description": "out of the garbage pail & into your heart", "followers_count": 394, "friends_count": 740, "statues_count": 24326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558422990618624, "text": "Carly Rae Jepsen at the Venetian Theatre, Venetian Hotel, Las Vegas, Nevada.\nSaturday,… https://t.co/rOVKs5hIPl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.16972605,36.12209015"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2751167781, "name": "Dr. Wickwire", "screen_name": "BeachdrmDr", "lang": "en", "location": "Manhattan Beach", "create_at": date("2014-08-25"), "description": "null", "followers_count": 38, "friends_count": 27, "statues_count": 487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423208722432, "text": "@S_Kalaeb https://t.co/YqX69OeUf1", "in_reply_to_status": -1, "in_reply_to_user": 3005997320, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3005997320 }}, "user": { "id": 1242916316, "name": "yo girl abad", "screen_name": "adriannejaneb", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "be the good | uhs '17", "followers_count": 570, "friends_count": 619, "statues_count": 6908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423313551360, "text": "43.9F (Feels: 43.9F) - Humidity: 79% - Wind: 5.4mph N - Gust: 5.4mph - Pressure: 1036.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 220793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423334539266, "text": "Noooo waaaay ! I wanna see �� https://t.co/IN0oehwR13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458184523, "name": "Brie", "screen_name": "Impetuousbrie", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-01-08"), "description": "freedom is just an illusion", "followers_count": 424, "friends_count": 348, "statues_count": 11200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423368216576, "text": "Wind 0.0 mph SW. Barometer 30.094 in, Falling slowly. Temperature 26.3 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423435202560, "text": "This is so beautiful. https://t.co/0Ie7dzgh6F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428295288, "name": "princess als", "screen_name": "alxcia_", "lang": "en", "location": "null", "create_at": date("2014-04-04"), "description": "he stuck to me... that's that new orleans voodoo.", "followers_count": 935, "friends_count": 744, "statues_count": 43646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423464665088, "text": "@TooDope_Dandyno @SportsCenter they were good from the 30s to 70s ..", "in_reply_to_status": 683557904088231936, "in_reply_to_user": 1458624571, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1458624571, 26257166 }}, "user": { "id": 1223315011, "name": "Lo Fo", "screen_name": "LoganLOFOFoley", "lang": "en", "location": "Richmond, KY", "create_at": date("2013-02-26"), "description": "|| Warriors || Eagles || Oregon || Justin Bieber is a blessing to this world ||", "followers_count": 1079, "friends_count": 655, "statues_count": 28690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, KY", "id": "4c56854a831bdd43", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.37566,37.688339 -84.249178,37.790079") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21151, "countyName": "Madison", "cityID": 2165226, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423493873664, "text": "this live https://t.co/ltPuVDptKU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3317930161, "name": "DIRTYBEAN", "screen_name": "j_gracia8", "lang": "en", "location": "dickinson, tx", "create_at": date("2015-08-17"), "description": "Trusting his plan✞", "followers_count": 178, "friends_count": 137, "statues_count": 4550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "League City, TX", "id": "cf3b45e29c2c319b", "name": "League City", "place_type": "city", "bounding_box": rectangle("-95.178987,29.438994 -94.995071,29.55532") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841980, "cityName": "League City" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423682662400, "text": "#1 thing I have no tolerance for: \n\"I know I shouldn't do this.\" \n*does the stupid thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2835393272, "name": "megmusick", "screen_name": "meganmusick_", "lang": "en", "location": "Blanchard, OK", "create_at": date("2014-09-28"), "description": "life lovin✨", "followers_count": 214, "friends_count": 229, "statues_count": 102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blanchard, OK", "id": "006f804f7983fc5b", "name": "Blanchard", "place_type": "city", "bounding_box": rectangle("-97.703756,35.116446 -97.635411,35.160404") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40087, "countyName": "McClain", "cityID": 4006700, "cityName": "Blanchard" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423879798784, "text": "My baby moves next weekend to sac :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1282598803, "name": "Shelia Buckingham", "screen_name": "sheliabeeee", "lang": "en", "location": "null", "create_at": date("2013-03-20"), "description": "Merced, CA", "followers_count": 454, "friends_count": 452, "statues_count": 16252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558423921848320, "text": "Really really ruined my night .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3040805498, "name": "❤️®outyy✨", "screen_name": "JacksonJermicla", "lang": "en", "location": "unfuckwittable ", "create_at": date("2015-02-24"), "description": "null", "followers_count": 233, "friends_count": 257, "statues_count": 3372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, LA", "id": "fce60469c6c6801a", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-91.230994,30.809727 -91.188717,30.854479") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22037, "countyName": "East Feliciana", "cityID": 2237830, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558424450224129, "text": "Raleigh❤️ https://t.co/jWUEOc8tCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 308802856, "name": "Reah", "screen_name": "lavisxh_queen", "lang": "en", "location": "Kevin Durant", "create_at": date("2011-05-31"), "description": "null", "followers_count": 1803, "friends_count": 1285, "statues_count": 45073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Arthur, TX", "id": "350ad20be66e9edc", "name": "Port Arthur", "place_type": "city", "bounding_box": rectangle("-94.053284,29.823923 -93.858553,29.979166") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858820, "cityName": "Port Arthur" } }
+{ "create_at": datetime("2016-01-03T00:00:02.000Z"), "id": 683558424546643969, "text": "@jfmfox OK.", "in_reply_to_status": 683558374596804609, "in_reply_to_user": 64650928, "favorite_count": 0, "coordinate": point("-121.881832,39.7753674"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 64650928 }}, "user": { "id": 258398208, "name": "¡CProcks!", "screen_name": "Argentinaenae", "lang": "en", "location": "Chico, California", "create_at": date("2011-02-27"), "description": "14 year old furry and professional shitposter. telegram: CProcks", "followers_count": 372, "friends_count": 394, "statues_count": 19835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558424706088961, "text": "I'm so drunk :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 969008220, "name": "♛вernιceee", "screen_name": "_beeexoxo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-11-24"), "description": "Smile & Be Happy.✨", "followers_count": 467, "friends_count": 1301, "statues_count": 37053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell Gardens, CA", "id": "9da8666a1bd226d8", "name": "Bell Gardens", "place_type": "city", "bounding_box": rectangle("-118.170748,33.951888 -118.128456,33.97938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604996, "cityName": "Bell Gardens" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558424727048192, "text": "Temp: 41.9°F Wind:0.0mph Pressure: 30.223hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558424748126208, "text": "@bijancbayne I follow @JoelMCurzon.Not one to express strong opinions generally, but he is irate about this. You might want to check his TL.", "in_reply_to_status": 683556043675271168, "in_reply_to_user": 2790538982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2790538982, 1276583328 }}, "user": { "id": 859260774, "name": "Charmian Neary", "screen_name": "CharmianAgain", "lang": "en", "location": "Rye, New York", "create_at": date("2012-10-02"), "description": "Bored housewife ~ likes Major League Baseball, explosive demolition, Bill Nye the Science Guy, and Hyatt Hotels.", "followers_count": 2025, "friends_count": 752, "statues_count": 80729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rye, NY", "id": "7bc85d15fd696474", "name": "Rye", "place_type": "city", "bounding_box": rectangle("-73.71583,40.939245 -73.65621,40.997827") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3664309, "cityName": "Rye" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558424785846272, "text": "Falling from height https://t.co/P2bHb4PVFG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1129572901, "name": "Golden Graham", "screen_name": "jgraham2_", "lang": "en", "location": "Millbrook '17", "create_at": date("2013-01-28"), "description": "Just because you put syrup on somethin don't make it pancakes\n -Shawn Spencer", "followers_count": 504, "friends_count": 424, "statues_count": 34815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558425259683840, "text": "they dont want you to party", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434363069, "name": "des", "screen_name": "destinytopete", "lang": "en", "location": "null", "create_at": date("2011-12-11"), "description": "self made", "followers_count": 1840, "friends_count": 483, "statues_count": 53031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux City, IA", "id": "c7397adf5a17673a", "name": "Sioux City", "place_type": "city", "bounding_box": rectangle("-96.492971,42.385735 -96.301329,42.56216") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19193, "countyName": "Woodbury", "cityID": 1973335, "cityName": "Sioux City" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558425276616704, "text": "@tonyinnebraska @rumpfshaker & this https://t.co/SSWzyDHYOx I can do this all day I only seek equality. How can u defend this as equal?", "in_reply_to_status": 683556825921269760, "in_reply_to_user": 3438116309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3438116309, 23283299 }}, "user": { "id": 29605320, "name": "Clarence Tunstall", "screen_name": "crtjr", "lang": "en", "location": "null", "create_at": date("2009-04-07"), "description": "Dedicated Family man, Die hard Steeler fan, programmer extraordinaire, aspiring entrepreneur, politically active & opinionated, highly imperfect, pilgrim of God", "followers_count": 549, "friends_count": 683, "statues_count": 25669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558425465192448, "text": "little!? that's like the biggest secret ever https://t.co/3NgpmCkotc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322384752, "name": "Itzcalli Pizano", "screen_name": "Itzcalliii", "lang": "en", "location": "null", "create_at": date("2011-06-22"), "description": "null", "followers_count": 770, "friends_count": 336, "statues_count": 14808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reedley, CA", "id": "f434511e8f0c8aa4", "name": "Reedley", "place_type": "city", "bounding_box": rectangle("-119.4699,36.575137 -119.421897,36.618737") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 660242, "cityName": "Reedley" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558425612058625, "text": "Trying to keep my voices to myself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84231458, "name": "ExploreNextDoor", "screen_name": "DennyBoiPeez", "lang": "en", "location": "west pittsburg CA", "create_at": date("2009-10-21"), "description": "SC : Dennyboipeez Photographer | Entertainment | Host | Promotor dextexplore@gmail.com #XplorePhotography #WeekendBois #MAD #SoPeez #OVOXO", "followers_count": 879, "friends_count": 1601, "statues_count": 18418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558425737826305, "text": "@DarrionDaDon_ @Real_C_Flex ANY KIDD. LMAO.", "in_reply_to_status": 682343716065955841, "in_reply_to_user": 103772491, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 103772491, 2760583455 }}, "user": { "id": 1268419218, "name": "N!gg@hDoSomtin'", "screen_name": "EffYoF33linz", "lang": "en", "location": "USC #FIGHT ON..", "create_at": date("2013-03-14"), "description": "CALIEEEE GIRL ALL FEMALE LAKERNATION TIL I DIE US ARMY VET 93HOTEL. NO ISIS NO.**********.DM.", "followers_count": 6916, "friends_count": 6993, "statues_count": 34503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Athens, CA", "id": "81923aaf1207edc1", "name": "West Athens", "place_type": "city", "bounding_box": rectangle("-118.317804,33.916377 -118.291514,33.931268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684116, "cityName": "West Athens" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426039865344, "text": "You won the sprint. But this is a marathon. Hoe.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33450330, "name": "IG: ILOVESF2", "screen_name": "iLoveSF2", "lang": "en", "location": "Houston,TX", "create_at": date("2009-04-19"), "description": "SF2 North 215 W. Greens Rd. /2818761600 Supporting all things HOUSTON #ILOVESF2", "followers_count": 6741, "friends_count": 732, "statues_count": 84611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426258083841, "text": "Wind 0.7 mph S. Barometer 29.80 in, Falling. Temperature 32.5 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 26, "friends_count": 89, "statues_count": 156384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426346045442, "text": "\"We should have a class that teaches you everyday skills in highschool!!\" You mean home economics?? Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50434566, "name": "trash flavored trash", "screen_name": "wjknd", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-06-24"), "description": "I'm hungry", "followers_count": 86, "friends_count": 64, "statues_count": 3230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426417311744, "text": "Can you recommend anyone for this #security #job? https://t.co/1CYKznixOQ #infosec #Austin, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7430608,30.267153"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "security", "job", "infosec", "Austin", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 266, "friends_count": 160, "statues_count": 558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426547376129, "text": "getting paid early>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 776742751, "name": "Loraine❣", "screen_name": "_lorainecocaine", "lang": "en", "location": "null", "create_at": date("2012-08-23"), "description": "20 // CBN :B", "followers_count": 924, "friends_count": 343, "statues_count": 41592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426744504320, "text": "\"Daddies and bwallers\" �� find and save me a sugar daddy baby, at least until I pay off my school loans�� https://t.co/VpPw8YkAzD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2455998222, "name": "b. bulletpoints", "screen_name": "qweeennnaf", "lang": "en", "location": "LA✈️AZ", "create_at": date("2014-04-20"), "description": "pinned it", "followers_count": 281, "friends_count": 247, "statues_count": 10152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426958405636, "text": "God let me sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2252442878, "name": "Kate♉️", "screen_name": "173kate_", "lang": "en", "location": "null", "create_at": date("2013-12-18"), "description": "Filipina", "followers_count": 326, "friends_count": 248, "statues_count": 11941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-03T00:00:03.000Z"), "id": 683558426962706432, "text": "I literally love life so much.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29354414, "name": "Carl Wheezer✝", "screen_name": "carlosisradyo", "lang": "en", "location": "Sunny Place Shady People, MIA", "create_at": date("2009-04-06"), "description": "ΠΚΦ|ΔΦΕ Campus Man|Coach|PHIL 4:13", "followers_count": 651, "friends_count": 741, "statues_count": 19593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountainebleau, FL", "id": "0185fd6875f1815c", "name": "Fountainebleau", "place_type": "city", "bounding_box": rectangle("-80.386249,25.761172 -80.320416,25.783404") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1224562, "cityName": "Fountainebleau" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920800517238784, "text": "the problem is, you let a bunch of our female family members do all this with them, just give them a verbal warning often, doesnt stop it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 904016269, "name": "Tom Fuller", "screen_name": "gocowboysfan00", "lang": "en", "location": "Abilene Texas USA ", "create_at": date("2012-10-25"), "description": "christian man business owner legal professional it enthusiast single and available to women.", "followers_count": 241, "friends_count": 2004, "statues_count": 21564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920800676630528, "text": "@king_of_indiana thanks ��", "in_reply_to_status": 683920738517139456, "in_reply_to_user": 30152606, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30152606 }}, "user": { "id": 2296291195, "name": "PattyMayonaise", "screen_name": "Kierre____", "lang": "en", "location": "Crawfish,corn and potatoes, LA", "create_at": date("2014-01-17"), "description": "null", "followers_count": 669, "friends_count": 727, "statues_count": 11423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920800676745216, "text": "ed8ffc6af826a7442d36d0c2dbfb67915e8a7295f99f3ef91ad8096ba094db1c96cfc49e26279345d22cb5cc9bf0ad4915a6a1320ef64a29635e655f5c6150b5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-148.591478,62.154474"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 429803867, "name": "MarsBots", "screen_name": "MarsBots", "lang": "en", "location": "Mars", "create_at": date("2011-12-06"), "description": "null", "followers_count": 74, "friends_count": 0, "statues_count": 410156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2261, "countyName": "Valdez-Cordova" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920800756273152, "text": "2 , I like Windows. https://t.co/5zaut0pRym", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949471373, "name": "☼trashley", "screen_name": "Relaxashley", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "Is this the crusty crab?", "followers_count": 737, "friends_count": 86, "statues_count": 18884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920801037303808, "text": "This is me and Angelica. Literally. https://t.co/8Qy7xZbchM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164572244, "name": "DanielaDavila✨", "screen_name": "OhSoDanielaaa", "lang": "en", "location": "Brownsville, TX", "create_at": date("2010-07-08"), "description": "20. Texas. FΔM.✌️", "followers_count": 2001, "friends_count": 1101, "statues_count": 56236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920801054195712, "text": "My analysis of #Stocks in #2015 - It was actually a good year to be an investor - read the details here - https://t.co/VCuGRLY9Vd #investing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Stocks", "investing" }}, "user": { "id": 19579220, "name": "Michael Avrukin", "screen_name": "mavrukin", "lang": "en", "location": "null", "create_at": date("2009-01-26"), "description": "null", "followers_count": 20, "friends_count": 39, "statues_count": 148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain View, CA", "id": "b19a2cc5134b7e0a", "name": "Mountain View", "place_type": "city", "bounding_box": rectangle("-122.117916,37.356771 -122.044969,37.436935") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920801276399616, "text": "Can't sleep, and it's too late to take an Ambien. #INeedToChill #QuitThinkingAboutThings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "INeedToChill", "QuitThinkingAboutThings" }}, "user": { "id": 23528720, "name": "Kathi DeGrand", "screen_name": "KathiDG", "lang": "en", "location": "Racine, WI", "create_at": date("2009-03-09"), "description": "null", "followers_count": 60, "friends_count": 285, "statues_count": 3883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920801716813824, "text": "This has me deadT https://t.co/xAdO5MwVcI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538757502, "name": "Fantasia Stan", "screen_name": "BuddyBoyThough", "lang": "en", "location": "Fresno, CA", "create_at": date("2012-03-27"), "description": "Quick Facts: Beyoncé. Fantasia. Janelle Monáe. Trey Songz. Judith Hill. Leon Bridges. Marvel. Pokémon. Kingdom Hearts. Disney. Anime. Nintendo #BlackLivesMatter", "followers_count": 1070, "friends_count": 946, "statues_count": 149444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920801742086148, "text": "Temp: 50.4°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.99 in | Dewpoint: 50.1°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 23, "friends_count": 1, "statues_count": 164081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920802073325568, "text": "With one minute to spare @Stanford is done", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18036441 }}, "user": { "id": 1340877798, "name": "Joseph Ballard", "screen_name": "TheJoeBallard", "lang": "en", "location": "Earth", "create_at": date("2013-04-09"), "description": "Lost in Time. Bellarmine College Prep '16", "followers_count": 352, "friends_count": 468, "statues_count": 2541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920802534670337, "text": "@lacacawalrus es niiiice, alright wtf imma stop talking like that", "in_reply_to_status": 683920677531852801, "in_reply_to_user": 127708476, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 127708476 }}, "user": { "id": 519738165, "name": "Ricky V", "screen_name": "_TheRickster", "lang": "en", "location": "626/707", "create_at": date("2012-03-09"), "description": "always thirsty #kony2012", "followers_count": 278, "friends_count": 154, "statues_count": 45790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920802698379264, "text": "Ripley SW Limestone Co. Temp: 34.0°F Wind:3.1mph Pressure: 1001.0mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 47020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803059134464, "text": "\"God is my homie\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3410928359, "name": "wendy", "screen_name": "wendyjiangg", "lang": "en", "location": "TN", "create_at": date("2015-08-09"), "description": "God is my homie", "followers_count": 225, "friends_count": 160, "statues_count": 742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kimball, TN", "id": "76660faa26ab13ad", "name": "Kimball", "place_type": "city", "bounding_box": rectangle("-85.690701,35.038257 -85.629334,35.064056") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47115, "countyName": "Marion", "cityID": 4739400, "cityName": "Kimball" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803289632768, "text": "@ReubenHdz alright cool just let me know, I got a new bed ����", "in_reply_to_status": 683920693197537280, "in_reply_to_user": 570839183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 570839183 }}, "user": { "id": 1348116918, "name": "la cubana™", "screen_name": "vannawannaxo", "lang": "en", "location": "atx", "create_at": date("2013-04-12"), "description": "| insta: @ yvannawannaholla | #YÑWD |", "followers_count": 1086, "friends_count": 859, "statues_count": 78631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.500723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803289665536, "text": "7️⃣⚽️����my boy Boni! Wish I played the day you played vs us to stop that last goal �� you cold af bro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236702232, "name": "Randy Ortiz", "screen_name": "Renditooo", "lang": "en", "location": "null", "create_at": date("2013-12-08"), "description": "AHHS C/O 2017 #6 ⚽\nFWTX [OIIIIIIIO]", "followers_count": 872, "friends_count": 654, "statues_count": 15076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803415523329, "text": "88- HAHAHAHA we good friends, we go way back ;)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2828233357, "name": "Caden Weible", "screen_name": "cbearfool", "lang": "en", "location": "Stoney Breek", "create_at": date("2014-09-23"), "description": "| B2G And VHS Football CB/WR | '2⃣0⃣1⃣8⃣ |snapchat: cbearfool | IG: cbearfool | slime", "followers_count": 640, "friends_count": 479, "statues_count": 4088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803453259776, "text": "Every time I wanna stand I feel like I'm gonna collapse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3069021433, "name": "salvadoreño", "screen_name": "_alllexis", "lang": "en", "location": "South Gate, CA", "create_at": date("2015-03-08"), "description": "sehs var. tennis captain", "followers_count": 329, "friends_count": 289, "statues_count": 17323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803453272066, "text": "WHAT'S 9+10?! ME BITCH! 2⃣1⃣��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28599537, "name": "Chanellie ✨", "screen_name": "Chanelliekins", "lang": "en", "location": "null", "create_at": date("2009-04-03"), "description": "it's cold, but I'm still lookin' like a thottie 'cause a hoe neva' gets cold", "followers_count": 646, "friends_count": 456, "statues_count": 30518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803600138240, "text": "@anaaokay Yooo that's so true but they get so many orders they won't even peep", "in_reply_to_status": 683920568228442112, "in_reply_to_user": 1520815136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1520815136 }}, "user": { "id": 922230127, "name": "falcao", "screen_name": "Kevinfalcao_NYR", "lang": "en", "location": "null", "create_at": date("2012-11-02"), "description": "21/ NYR", "followers_count": 823, "friends_count": 540, "statues_count": 19838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920803843330048, "text": "@Hityorichie ��", "in_reply_to_status": 683920528705388549, "in_reply_to_user": 564052691, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 564052691 }}, "user": { "id": 1586353266, "name": "HOVERBOARDSNATCHER", "screen_name": "Kee_Richess", "lang": "en", "location": "Compton", "create_at": date("2013-07-11"), "description": "#WestRiches x #LongLiveForever", "followers_count": 1436, "friends_count": 1086, "statues_count": 30908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-01-04T00:00:00.000Z"), "id": 683920804111761408, "text": "Can you recommend anyone for this #job? Health Enthusiast Part-Time - https://t.co/mypQnhoILO #Redding, CA #HealthWelfare #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3507788,40.5881385"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Redding", "HealthWelfare", "Hiring" }}, "user": { "id": 2362771358, "name": "Vitamin Shoppe Jobs", "screen_name": "VSIcareers", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "At #VitaminShoppe we are lovers of fitness, health, vitamins, helping you on your wellness journey and motivating others. Apply to our #jobs below.", "followers_count": 491, "friends_count": 96, "statues_count": 3128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920804271095808, "text": "If any of my teachers get an attitude tomorrow for no reason I'm walking out of class", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863541078, "name": "C. Vit.", "screen_name": "colevitiritto", "lang": "en", "location": "null", "create_at": date("2014-10-18"), "description": "yeezy taught me", "followers_count": 344, "friends_count": 231, "statues_count": 4968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saylorville, IA", "id": "016cbdbf2007ee0f", "name": "Saylorville", "place_type": "city", "bounding_box": rectangle("-93.668421,41.636286 -93.581216,41.703815") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1970995, "cityName": "Saylorville" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920804573138944, "text": "������ https://t.co/rk4aGpehit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1203318198, "name": "The General", "screen_name": "LebaneseThor", "lang": "en", "location": "California to Japan", "create_at": date("2013-02-20"), "description": "Lookin for a woman who wont fuck other dudes while im on deployment. #UnitedStatesNavy #BM3", "followers_count": 1304, "friends_count": 647, "statues_count": 95269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920804698951680, "text": "I know exactly what I'm looking for", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1584595332, "name": "Sativa☮", "screen_name": "TikiBruuh", "lang": "en", "location": "null", "create_at": date("2013-07-10"), "description": "Instagram/Snapchat/Kik ; Tikibruuh", "followers_count": 584, "friends_count": 562, "statues_count": 33099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "National City, CA", "id": "7905cfee1600eb70", "name": "National City", "place_type": "city", "bounding_box": rectangle("-117.124452,32.64634 -117.049103,32.69328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 650398, "cityName": "National City" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920804807970820, "text": "We kissed once at a Bingham game but I'm pretty sure you don't remember because it was super spontaneous ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313694048, "name": "Brigham", "screen_name": "KingOfHerriman", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "Herriman High Football #4 | STATE CHAMPS |", "followers_count": 1455, "friends_count": 705, "statues_count": 32984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920805185519617, "text": "Yo fam* aka you fucked up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295484823, "name": "Teezus", "screen_name": "DessyBopper", "lang": "en", "location": "null", "create_at": date("2011-05-08"), "description": "null", "followers_count": 665, "friends_count": 592, "statues_count": 35330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ensley, FL", "id": "b0dff539d07de13d", "name": "Ensley", "place_type": "city", "bounding_box": rectangle("-87.313476,30.503417 -87.221617,30.550068") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1220925, "cityName": "Ensley" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920805290442752, "text": "01/04@03:00 - Temp 27.0F, WC 27.0F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.963in, Rising slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920805906939906, "text": "OMG, I don't have my SAT or ACT scores yet I guess I can't fill out these scholarships yet. My 2016 is not starting off right ughh.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396940044, "name": "r.eree", "screen_name": "_darkski", "lang": "en", "location": "Atlanta Georgia ", "create_at": date("2011-10-23"), "description": "Atlanta, Ga \nLawyer \n | August 13 1997\n16 Senior", "followers_count": 1133, "friends_count": 1464, "statues_count": 17473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444409 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920806032748544, "text": "@rachelathomson hmm how can u tell them apart?", "in_reply_to_status": 683919201082437632, "in_reply_to_user": 237425650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 237425650 }}, "user": { "id": 142474906, "name": "v", "screen_name": "_notvanessa", "lang": "en", "location": "California, USA", "create_at": date("2010-05-10"), "description": "Kanye 2020", "followers_count": 865, "friends_count": 849, "statues_count": 75417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920806691368960, "text": "They know that I got it ! They know that I got it .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92130817, "name": "Were_is_Willie", "screen_name": "1OfficialJunya", "lang": "en", "location": "251 ✈ 334", "create_at": date("2009-11-23"), "description": "-good vibes only listen to my music at the link below !! Hebrews 5:8", "followers_count": 3080, "friends_count": 1439, "statues_count": 36067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, AL", "id": "31cc87eb4d275bb6", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-86.018117,31.755197 -85.930803,31.857919") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1109, "countyName": "Pike", "cityID": 176920, "cityName": "Troy" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920806754267136, "text": "@mlusa 可是明天还要早起 绝望…ಠ_ಠ", "in_reply_to_status": 683920651711844354, "in_reply_to_user": 16706049, "favorite_count": 0, "retweet_count": 0, "lang": "zh", "is_retweet": false, "user_mentions": {{ 16706049 }}, "user": { "id": 179083189, "name": "龙猫大王", "screen_name": "Muffin_C", "lang": "en", "location": "Pittsburgh", "create_at": date("2010-08-16"), "description": "#砍死 #非常丑 #参见头像 #长城宽带受害者 #钱已经准备好了 #FAV有惊喜", "followers_count": 4177, "friends_count": 215, "statues_count": 62157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920807110639616, "text": "���� https://t.co/73DopSJliN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 323698350, "name": "AIRMIS", "screen_name": "trackstar015", "lang": "en", "location": "EJ CHEER CAPTAIN ", "create_at": date("2011-06-25"), "description": "18. SEN16R", "followers_count": 1414, "friends_count": 2331, "statues_count": 79045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920807123259392, "text": "I miss the old cudi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1514163074, "name": "ㅤ", "screen_name": "JxseCardenas", "lang": "en", "location": "Joliet, IL", "create_at": date("2013-06-13"), "description": "PCHS ⚽️ #6", "followers_count": 465, "friends_count": 297, "statues_count": 24486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444409 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920807186202624, "text": "@robokidmusic LOL", "in_reply_to_status": 683920704547504128, "in_reply_to_user": 30545370, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 30545370 }}, "user": { "id": 1478274474, "name": "mel", "screen_name": "killinurvibe", "lang": "en", "location": "Los Angeles", "create_at": date("2013-06-02"), "description": "ignore me", "followers_count": 1665, "friends_count": 543, "statues_count": 14296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920807261634560, "text": "Netflix ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 828252230, "name": "X", "screen_name": "MohamedMohamedv", "lang": "en", "location": "seattle,wa", "create_at": date("2012-09-16"), "description": "null", "followers_count": 265, "friends_count": 259, "statues_count": 13571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920808230531072, "text": "3 days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339764471, "name": "Yessi Velazquez", "screen_name": "Steezy_Yessi", "lang": "en", "location": "Duarte, CA", "create_at": date("2014-02-11"), "description": "It's lit", "followers_count": 173, "friends_count": 164, "statues_count": 2549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-01-04T00:00:01.000Z"), "id": 683920808310366208, "text": "1h 35m wait time at Sturgeon Community Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #StAlbert", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.625888,53.655272"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "StAlbert" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 367, "friends_count": 888, "statues_count": 2504 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "St. Albert, Alberta", "id": "56316c24480c1a3a", "name": "St. Albert", "place_type": "city", "bounding_box": rectangle("-113.707088,53.599415 -113.565747,53.681859") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920808549433344, "text": "Wind 2.0 mph NW. Barometer 30.406 in, Steady. Temperature 29.4 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920808821960704, "text": "Nah..I'd rather be hurt wit the truth than sitting here happy looking stupid https://t.co/qJLnNhFJxN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131956843, "name": "gabriel", "screen_name": "_rocceaux", "lang": "en", "location": "Louisiana", "create_at": date("2010-04-11"), "description": "Im that little glimpse of light that makes the diamond really shine.........................\nRIP Ryan Rocco Mallory", "followers_count": 510, "friends_count": 166, "statues_count": 31825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920808838696960, "text": "But until I do sleep, I will watch me some good ole avatar the last airbender", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324706166, "name": "❣Hallow❣", "screen_name": "Me_name_munchii", "lang": "en", "location": "With y(our) nigga", "create_at": date("2011-06-26"), "description": "Finally Found Peace ✌️☮", "followers_count": 111, "friends_count": 138, "statues_count": 2635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809174290432, "text": "When you shoot your shot and it's a success ������ https://t.co/aEGGOXXJ1E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1293181561, "name": "Miranda Weems", "screen_name": "_CallMeM_", "lang": "en", "location": "Meridian, MS", "create_at": date("2013-03-23"), "description": "keep sleeping . . .", "followers_count": 778, "friends_count": 562, "statues_count": 24481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, MS", "id": "57dc070bcd27882d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-88.763404,32.312389 -88.655698,32.443049") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2846640, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809300246528, "text": "You sleep naked?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48484668, "name": "Intellectual Killah", "screen_name": "IAmPhillyC", "lang": "en", "location": "Atlanta", "create_at": date("2009-06-18"), "description": "Vintage from 1980 something #TeamBellaBellz #Pisces #ATLien", "followers_count": 2087, "friends_count": 2046, "statues_count": 67080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mableton, GA", "id": "fe4b2b8e51cc85c1", "name": "Mableton", "place_type": "city", "bounding_box": rectangle("-84.620272,33.769819 -84.506557,33.852813") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1348288, "cityName": "Mableton" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809610592256, "text": "Money can't bring happiness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454067651, "name": "Queen B ❣", "screen_name": "Jalyse___", "lang": "en", "location": "Philippians 4:13", "create_at": date("2012-01-03"), "description": "No Rest In The Kingdom ☺", "followers_count": 3063, "friends_count": 940, "statues_count": 69827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809732210688, "text": "Wind 0.0 mph NNW. Barometer 30.15 in, Rising slowly. Temperature 31.5 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809790930944, "text": "This for my brothers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 401942424, "name": "Eazy E II", "screen_name": "De__Leon69", "lang": "en", "location": "null", "create_at": date("2011-10-31"), "description": "|John 4:13-14| Whenever I become satisfied with my success it means i have not succeed yet| LC'19 Livingstone College T&F", "followers_count": 1175, "friends_count": 1237, "statues_count": 29575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809862156288, "text": "Basically . https://t.co/UJiLUMiNce", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3152197500, "name": "Ari", "screen_name": "LouddonMute", "lang": "en", "location": "Moore, OK", "create_at": date("2015-04-12"), "description": "IG: __Alxo__ , ✝ Snap: Bigdadddyariana. btw, I'm poppin'.", "followers_count": 671, "friends_count": 462, "statues_count": 17210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920809920864256, "text": "My entire life https://t.co/xqzn9lFQjp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1226553182, "name": "Papi Ny", "screen_name": "NyallBryant", "lang": "en", "location": "Lonewolf, TX", "create_at": date("2013-02-27"), "description": "how u doin?? #almightytrio", "followers_count": 673, "friends_count": 433, "statues_count": 1132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810252222464, "text": "Not even tired , but school tomorrow ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616673137, "name": "Ki-ana✝", "screen_name": "Kianachristine_", "lang": "en", "location": "San Dimas, CA", "create_at": date("2012-06-23"), "description": "Romans 15:13 ✝", "followers_count": 729, "friends_count": 445, "statues_count": 7120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810302640128, "text": "If it's cool https://t.co/1PY9TNrJif", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281586001, "name": "plug griffey", "screen_name": "jonsznz", "lang": "en", "location": "null", "create_at": date("2011-04-13"), "description": "I'm not the same man I was a year ago. Montclair State", "followers_count": 1239, "friends_count": 874, "statues_count": 93972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810331914240, "text": "like damn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558667568, "name": "champagnemami :~)", "screen_name": "pertellarose", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-04-20"), "description": "MY NAME IS PERTELLA AND IM JUST A REGULAR DEGULAR SHMEGULAR GURL FROM LV", "followers_count": 996, "friends_count": 689, "statues_count": 11625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810415763456, "text": "She's in the men's room?�� https://t.co/xQYECsbjDn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504192595, "name": "CoCo", "screen_name": "SimplyAmazing_8", "lang": "en", "location": "Currently in phx,AZ ", "create_at": date("2012-02-26"), "description": "null", "followers_count": 1449, "friends_count": 1063, "statues_count": 32110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810445266944, "text": "@CMccaffrey5 seems like a great player with a respect for others. #NFL needs you. Would be great to see you running amongst them. #GreatJob", "in_reply_to_status": -1, "in_reply_to_user": 855680342, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NFL", "GreatJob" }}, "user_mentions": {{ 855680342 }}, "user": { "id": 1961071741, "name": "Dylan Flickinger", "screen_name": "Dylan_F645", "lang": "en", "location": "Eastlake, OH", "create_at": date("2013-10-14"), "description": "Knowledgeable Film Dude and Huge Northeastern Ohio Steelers Fanatic", "followers_count": 86, "friends_count": 626, "statues_count": 1642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastlake, OH", "id": "25956a920ad614f8", "name": "Eastlake", "place_type": "city", "bounding_box": rectangle("-81.462684,41.623826 -81.400279,41.694467") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3923618, "cityName": "Eastlake" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920810675838977, "text": "i be trying goddamit �� https://t.co/Lmu1Cbc5ph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3766076772, "name": "jabari simpson", "screen_name": "imjustjabari", "lang": "en", "location": "null", "create_at": date("2015-10-02"), "description": "#RIPDRAKE #RIPRAHEEM #RIPGRANDMA #FREEMYDAD", "followers_count": 275, "friends_count": 239, "statues_count": 1810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811393024000, "text": "You don't care about other people's feelings and it's irritating. Bye bishhh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2933851651, "name": "Máma Ćhar", "screen_name": "Stewartlittle36", "lang": "en", "location": "OR", "create_at": date("2014-12-17"), "description": "Central Catholic ~ God Loves You ~ married to breezy❤️", "followers_count": 836, "friends_count": 871, "statues_count": 6427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811409948676, "text": "Wind 1.0 mph NNW. Barometer 30.241 in, Rising slowly. Temperature 31.2 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811770658816, "text": "Temp: 38.4°F Wind:0.0mph Pressure: 30.176hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811787304961, "text": "However, when I get paid, there is no longer a we. Dat ma money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214277510, "name": "Yarely Chavez", "screen_name": "yarelySays", "lang": "en", "location": "null", "create_at": date("2010-11-10"), "description": "Welcome to the island of misfit toys.", "followers_count": 411, "friends_count": 337, "statues_count": 32893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811896340483, "text": "Make my ass fatter ���� https://t.co/YMlj8AMrfh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2325088335, "name": "JusMyra❤️", "screen_name": "_myraaaaaaa", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "i just want the money , that's all", "followers_count": 903, "friends_count": 728, "statues_count": 13837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millbrook, AL", "id": "dc339f049fa0fff6", "name": "Millbrook", "place_type": "city", "bounding_box": rectangle("-86.412593,32.449263 -86.333698,32.590296") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1051, "countyName": "Elmore", "cityID": 148712, "cityName": "Millbrook" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920811917361152, "text": "41.1F (Feels: 41.1F) - Humidity: 90% - Wind: 0.7mph NE - Gust: 0.7mph - Pressure: 1036.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 220937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920812055867392, "text": "I love when people tell me how i make them feel i feel like a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2791521472, "name": "Kayla", "screen_name": "kaylamichyy", "lang": "en", "location": "null", "create_at": date("2014-09-29"), "description": "Cbhs | ft laud | jersey~ fl", "followers_count": 627, "friends_count": 611, "statues_count": 7642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weston, FL", "id": "88a21205dd4728ea", "name": "Weston", "place_type": "city", "bounding_box": rectangle("-80.441977,26.062031 -80.361426,26.144367") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1276582, "cityName": "Weston" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920812156440576, "text": "@AuditTheMedia What a reach. The fact that the media isn't calling for their heads is proof of far reich wing bias.", "in_reply_to_status": 683916569974407168, "in_reply_to_user": 2287887180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2287887180 }}, "user": { "id": 1590693098, "name": "Emile Dalkey", "screen_name": "EmileDalkey", "lang": "en", "location": "Sacramento, California", "create_at": date("2013-07-13"), "description": "Electric Guitarist based in Sacramento CA specializing in Rock and Roll and all its permutations. #UniteBlue I play #HashtagWars", "followers_count": 1187, "friends_count": 1939, "statues_count": 4924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920812240277504, "text": "#CranberryTownship, PA #IT #Job: Principal Network Engineer at Continuum Managed Services https://t.co/flmRDcecHf #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.119951,40.708957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CranberryTownship", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 455971331, "name": "Continuum", "screen_name": "FollowContinuum", "lang": "en", "location": "Boston, MA", "create_at": date("2012-01-05"), "description": "A unified managed services platform that enables MSPs to profitably backup, monitor, troubleshoot and maintain IT environments.", "followers_count": 2560, "friends_count": 2019, "statues_count": 10079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranberry, PA", "id": "01b347e54132a78d", "name": "Cranberry", "place_type": "city", "bounding_box": rectangle("-80.160831,40.640453 -79.995749,40.776081") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler" } }
+{ "create_at": datetime("2016-01-04T00:00:02.000Z"), "id": 683920812290797569, "text": "my friend @THRASHKETCHUM is having a rough night. go buy something off his rockcloset https://t.co/dAw1tuDRIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 93138061 }}, "user": { "id": 2720904592, "name": "emo boy", "screen_name": "duskofus", "lang": "en", "location": " capsize guitar, tour forever", "create_at": date("2014-07-22"), "description": "✉️ duskofus@gmail.com", "followers_count": 1572, "friends_count": 416, "statues_count": 3355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920812756353024, "text": "Wind 3.8 mph WSW. Barometer 29.97 in, Rising. Temperature 23.9 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 89, "statues_count": 156408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920812890394624, "text": "I miss grinchmas already :( @ Universal Studios Hollywood-Grinchmas https://t.co/MX7auhtUQn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.35668092,34.14044352"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263024694, "name": "ⓩⓔⓨⓜⓐ", "screen_name": "zeyma1998", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-03-08"), "description": "I'm like always hungry... instagram:zeyma1998 #TeamATV", "followers_count": 1195, "friends_count": 572, "statues_count": 181724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920813146386432, "text": "@umber_kahn omg ily so much thank you for putting that all together you're the best best friend I could ask for��", "in_reply_to_status": 683885271407050753, "in_reply_to_user": 950900058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 950900058 }}, "user": { "id": 551122708, "name": "Mariah Ford", "screen_name": "mariah_ford", "lang": "en", "location": "null", "create_at": date("2012-04-11"), "description": "null", "followers_count": 309, "friends_count": 172, "statues_count": 1880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodhaven, MI", "id": "00487e63709614f2", "name": "Woodhaven", "place_type": "city", "bounding_box": rectangle("-83.306006,42.109216 -83.215583,42.18442") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2688380, "cityName": "Woodhaven" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920813234323456, "text": "Came home to the house full of food .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 222956606, "name": "JoeKnow$™", "screen_name": "kiddflash96", "lang": "en", "location": "Texas, USA", "create_at": date("2010-12-04"), "description": "Stunt on me once and I'll flex on you forever . Based.", "followers_count": 1302, "friends_count": 706, "statues_count": 34597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920813280526336, "text": "Hell yea�������� https://t.co/O1nXAkYblu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2838135072, "name": "вяαи∂ι єνєℓуи мσяαn", "screen_name": "moran121798", "lang": "en", "location": "Corona, CA", "create_at": date("2014-10-02"), "description": "17♡\n \n \nJυиισя @Lєє ν.✏\n♛×corona×ca", "followers_count": 60, "friends_count": 186, "statues_count": 1161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920813481824256, "text": "@Wood_Mane15 @lynaex3 you like ppl like 2 chains huh �� weak", "in_reply_to_status": 683920664747614208, "in_reply_to_user": 437072795, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 437072795, 2674963656 }}, "user": { "id": 358007611, "name": "Who...", "screen_name": "_ThaDopest", "lang": "en", "location": "Etx made. ", "create_at": date("2011-08-18"), "description": "i treat you like a light skin female would..", "followers_count": 3507, "friends_count": 3427, "statues_count": 91842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Center, TX", "id": "7a15f6b26b4b752e", "name": "Center", "place_type": "city", "bounding_box": rectangle("-94.207311,31.766635 -94.154063,31.825256") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48419, "countyName": "Shelby", "cityID": 4813732, "cityName": "Center" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920813515395072, "text": "You're amazing https://t.co/4ftfnJ93E4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 914216034, "name": "Mina Lewinsky", "screen_name": "Mymangot2JOBS_", "lang": "en", "location": "University of Arizona ", "create_at": date("2012-10-30"), "description": "Allah. Africa. former fat girlllll now I'm really into fitness and working on becoming a better person in every aspect. #Based #blacklivesmatter #sandrabland", "followers_count": 1334, "friends_count": 428, "statues_count": 90411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814161276928, "text": "@D_Ayo11 update: still up��", "in_reply_to_status": 683916879031746561, "in_reply_to_user": 2447982102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2447982102 }}, "user": { "id": 2447982102, "name": "#️⃣1️⃣1️⃣", "screen_name": "D_Ayo11", "lang": "en", "location": "dtx", "create_at": date("2014-04-16"), "description": "Wide Reciever @ Sachse HighSchool |#D1Bound| Class of 2017 •Phillipians 4:13", "followers_count": 451, "friends_count": 356, "statues_count": 4641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814446657536, "text": "School is going to kick my butt today... Well worth it though.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 480971727, "name": "Migo Terry", "screen_name": "Aaron_James2", "lang": "en", "location": "null", "create_at": date("2012-02-01"), "description": "I'm convinced that about half of what separates the successful entrepreneurs from the non-successful ones is pure perseverance. - Steve Jobs", "followers_count": 904, "friends_count": 843, "statues_count": 32587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Teays Valley, WV", "id": "db8f089c6ab2a7a0", "name": "Teays Valley", "place_type": "city", "bounding_box": rectangle("-81.99608,38.433209 -81.843746,38.458596") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54079, "countyName": "Putnam", "cityID": 5479545, "cityName": "Teays Valley" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814526361601, "text": "I have to wake up in 3 1/2 hours..fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4563270629, "name": "Karley Martinez", "screen_name": "KarleyMartinez", "lang": "en", "location": "null", "create_at": date("2015-12-21"), "description": "Instagram | karley_martinez", "followers_count": 117, "friends_count": 148, "statues_count": 22 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hasbrouck Heights, NJ", "id": "2a9fc003f4101a1d", "name": "Hasbrouck Heights", "place_type": "city", "bounding_box": rectangle("-74.089373,40.847458 -74.059131,40.876005") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3430420, "cityName": "Hasbrouck Heights" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814563962880, "text": "@KalinWhite HAPPY BIRTHDAY BABY BOY I LOVE YOU SO MUCH ������ #21", "in_reply_to_status": -1, "in_reply_to_user": 124083358, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 124083358 }}, "user": { "id": 2841747811, "name": "scar", "screen_name": "queenskizzle", "lang": "en", "location": "bay area, ca", "create_at": date("2014-10-05"), "description": "⚾️", "followers_count": 431, "friends_count": 409, "statues_count": 6211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814606016512, "text": "03:00 28.9°F Feels:23.0°F (Hi30.0°F/Lo28.9°F) Hum:66% Wnd:NW 5.5MPH Baro:30.20in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 160, "friends_count": 261, "statues_count": 24274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920814719254528, "text": "Good Vibes And Munchies ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476422436, "name": "Trifecta ☯", "screen_name": "Filthy_Minded", "lang": "en", "location": "NYC✈️SC✈️MD✈️VA", "create_at": date("2012-01-27"), "description": "melanin. oxygen. glitter \nHampton University '19 #QTX", "followers_count": 1268, "friends_count": 871, "statues_count": 27410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aspen Hill, MD", "id": "08d254a18a834e3f", "name": "Aspen Hill", "place_type": "city", "bounding_box": rectangle("-77.125438,39.059951 -77.0338,39.120903") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2402825, "cityName": "Aspen Hill" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920815327440896, "text": "Girl jus help me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157099850, "name": "Jude", "screen_name": "Jujutough", "lang": "en", "location": "miami ✈️ga ✈️miss✈️arkansas", "create_at": date("2010-06-18"), "description": "#JUCOnation RIP Sean Taylor. im praying,grindin and chilling Haitian IG jujutough", "followers_count": 1263, "friends_count": 1002, "statues_count": 51136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Oaks, GA", "id": "a80d3803a636a10f", "name": "Fair Oaks", "place_type": "city", "bounding_box": rectangle("-84.558691,33.894492 -84.496049,33.938052") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1328520, "cityName": "Fair Oaks" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816195502081, "text": "1. They crossed my line.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3281232026, "name": "ilene . k ❁", "screen_name": "ilenek07", "lang": "en", "location": "Nowhere in Particular ", "create_at": date("2015-07-15"), "description": "College student just trying to make it.", "followers_count": 186, "friends_count": 137, "statues_count": 1709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816220844032, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1959780559, "name": "A-Money", "screen_name": "alainalos", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-13"), "description": "get yourself a me¯\\_(ツ)_/¯sc: alainalos ig:alaina.losiniecki", "followers_count": 224, "friends_count": 596, "statues_count": 5402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evergreen Park, IL", "id": "9458deb79984da4e", "name": "Evergreen Park", "place_type": "city", "bounding_box": rectangle("-87.721809,41.706027 -87.681753,41.7356") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1724634, "cityName": "Evergreen Park" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816359092224, "text": "The Force Awakens may make 820M plus just here in America. That will make me very very HAPPY!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3233256156, "name": "MIKEOL CORPORATION", "screen_name": "DJMIKEOLMUSIC", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-06-02"), "description": "EDM ARTIST/DJ/MUSIC VIDEO MAKER, AUTHOR, CG ARTIST, AND SOON VIDEO GAME CREATOR.", "followers_count": 1003, "friends_count": 2765, "statues_count": 13412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816422137856, "text": "aw flacko darling https://t.co/sPb0ucroJT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3460790602, "name": "maria", "screen_name": "MariaMartiniss", "lang": "en", "location": "null", "create_at": date("2015-08-27"), "description": "null", "followers_count": 330, "friends_count": 275, "statues_count": 647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brecksville, OH", "id": "5d36d1ed63fdf16c", "name": "Brecksville", "place_type": "city", "bounding_box": rectangle("-81.670799,41.277067 -81.565048,41.351279") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3908364, "cityName": "Brecksville" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816501878784, "text": "When she pour her heart out to you & express her feelings, tell you how much she love you������������������������������������������������������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2711597908, "name": "⠀⠀⠀⠀⠀⠀⠀⠀ʟɪʟ ᴅɪᴄᴋ ᴋᴏ✊", "screen_name": "SluttyKo", "lang": "en", "location": "Detroit, MI", "create_at": date("2014-07-14"), "description": "fuck social network..", "followers_count": 3651, "friends_count": 159, "statues_count": 66708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816598159360, "text": "Since I seen jauz, all I needa see is hardwell.. For the most part lol. Wanna hear those big room drops in person, the recent raves lack it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1398859406, "name": "ルーカす", "screen_name": "luuukied", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "null", "followers_count": 165, "friends_count": 157, "statues_count": 4586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kapolei, HI", "id": "01932ef001ef7c53", "name": "Kapolei", "place_type": "city", "bounding_box": rectangle("-158.090772,21.323228 -158.048087,21.365894") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1530300, "cityName": "Kapolei" } }
+{ "create_at": datetime("2016-01-04T00:00:03.000Z"), "id": 683920816669462528, "text": "Ain't all cereal soggy �� https://t.co/fDRAsMM2RU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167095916, "name": "Irma⛈", "screen_name": "Julialwolfe", "lang": "en", "location": "null", "create_at": date("2013-02-10"), "description": "Snapchat/ wolfeljulia", "followers_count": 459, "friends_count": 329, "statues_count": 4314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920816795422720, "text": "If she has \"Bad Bitch\" written in her bio, she fucks on the first date #TheseAreFacts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheseAreFacts" }}, "user": { "id": 85261849, "name": "✈️کнєєᏰ✌", "screen_name": "SwisherSoSweet", "lang": "en", "location": "WVU Grad Student #Mountaineers", "create_at": date("2009-10-26"), "description": "P h i l a d e l p h i a • M o r g a n t o w n × Swisher Sweets & Tweets x ║ Moet Mobbin' Clicqout Cliquin' and Swisher Cyphin' ✌║Snap: SwisherSoSweet", "followers_count": 1990, "friends_count": 2226, "statues_count": 56363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817189605377, "text": "@derekthesavage what time is it over der?", "in_reply_to_status": 683920701183545346, "in_reply_to_user": 483418073, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 483418073 }}, "user": { "id": 436318020, "name": "Nini", "screen_name": "annuuuus", "lang": "en", "location": "San Marcos, TX", "create_at": date("2011-12-13"), "description": "Compensation over conversation. The eyes never lie.", "followers_count": 984, "friends_count": 873, "statues_count": 17118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817424564224, "text": "Miserable as shit right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339358015, "name": "Aaron", "screen_name": "AirOnLand", "lang": "en", "location": "Douglasville, Georgia", "create_at": date("2011-07-20"), "description": "Instagram:/Snapchat : Aironland . Fuck em we ball #FreeGucci", "followers_count": 957, "friends_count": 467, "statues_count": 48517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Park, GA", "id": "1703b859c254a0f9", "name": "College Park", "place_type": "city", "bounding_box": rectangle("-84.512982,33.593318 -84.427795,33.669237") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1317776, "cityName": "College Park" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817453924352, "text": "@Glad_Montana , ilike em ��", "in_reply_to_status": 683881474211364864, "in_reply_to_user": 239505711, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 239505711 }}, "user": { "id": 381266452, "name": "vagatarian.", "screen_name": "jaasorbooty", "lang": "en", "location": "BCU9TEEN", "create_at": date("2011-09-27"), "description": "-giiirlsssss♥. bestfri.@simply_misrissa fmoi:jaasorbooty. p e t t y GANG", "followers_count": 1320, "friends_count": 2061, "statues_count": 24140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conley, GA", "id": "4479797d4d745235", "name": "Conley", "place_type": "city", "bounding_box": rectangle("-84.36042,33.623798 -84.315203,33.648025") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1319280, "cityName": "Conley" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817583828992, "text": "Mk https://t.co/gkqxxSegiK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2894299290, "name": "mel", "screen_name": "melvazquez99", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "null", "followers_count": 259, "friends_count": 184, "statues_count": 7195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leander, TX", "id": "be1ec4774a795507", "name": "Leander", "place_type": "city", "bounding_box": rectangle("-97.897898,30.524702 -97.795581,30.612656") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4842016, "cityName": "Leander" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817667743745, "text": "@Heatheriee_cx @melssscx hey ���������� you heather", "in_reply_to_status": 683721468778164224, "in_reply_to_user": 3035061800, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3035061800, 1587013316 }}, "user": { "id": 1041294018, "name": "Jaime Herrera", "screen_name": "jaime97h", "lang": "en", "location": "null", "create_at": date("2012-12-27"), "description": "Melissa Chacon is my girl ❤ BLESSED!", "followers_count": 248, "friends_count": 211, "statues_count": 9109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920817848201216, "text": "������ https://t.co/zuZGRduCD5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 163629206, "name": ". shay ✨", "screen_name": "xoShanika", "lang": "en", "location": "Tennessee State University", "create_at": date("2010-07-06"), "description": "shanika . eighteen . #TSU19 . 731 ✈️ 615 . sc- xoshanikaa", "followers_count": 1469, "friends_count": 1169, "statues_count": 14078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ripley, TN", "id": "21dc514d396bacc4", "name": "Ripley", "place_type": "city", "bounding_box": rectangle("-89.570636,35.691295 -89.497604,35.786701") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47097, "countyName": "Lauderdale", "cityID": 4763340, "cityName": "Ripley" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818070392834, "text": "@riceandco Rice & Company #Las Vegas https://t.co/lzX1F1N5Ip Excellent https://t.co/QsWHRr9vkd", "in_reply_to_status": -1, "in_reply_to_user": 243767660, "favorite_count": 0, "coordinate": point("-115.17540157,36.09545432"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Las" }}, "user_mentions": {{ 243767660 }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3648, "friends_count": 507, "statues_count": 161725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818334597120, "text": "I want some more tattoos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418141250, "name": "WoodNextDoor", "screen_name": "BigPlastic1", "lang": "en", "location": "Tyler, Tx is where I reside ", "create_at": date("2011-11-21"), "description": "God, Family, Basketball. WTN✊", "followers_count": 715, "friends_count": 709, "statues_count": 17939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818506604544, "text": "#Sales in #Knoxville, TN: Evening Call Center Agent at DialAmerica https://t.co/TOrsn8iLKu #milspouse #parttime https://t.co/IDf9O5Q2qG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.0854432,35.9276062"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Knoxville", "milspouse", "parttime" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 289, "friends_count": 217, "statues_count": 3587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818531766272, "text": "wanna c my hunny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621425743, "name": "vicks", "screen_name": "vicksss___", "lang": "en", "location": "SFSU", "create_at": date("2012-06-28"), "description": "boolin w/ my gem", "followers_count": 598, "friends_count": 429, "statues_count": 20808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818611458048, "text": "yall acting like break was never gonna end������ stfu yall need the education", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3068837370, "name": "Dom", "screen_name": "mfdom_", "lang": "en", "location": "null", "create_at": date("2015-03-08"), "description": "It's a daily routine | (other Twitter got suspended)", "followers_count": 151, "friends_count": 147, "statues_count": 941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen, TX", "id": "22d928cbeab790ad", "name": "Allen", "place_type": "city", "bounding_box": rectangle("-96.736596,33.066464 -96.608938,33.158169") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4801924, "cityName": "Allen" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920818947031041, "text": "Fuck school and everybody in there and fuck Monday's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2315901822, "name": "Jan23", "screen_name": "blvckice7", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-01-28"), "description": "LW17", "followers_count": 238, "friends_count": 207, "statues_count": 7096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920819211210752, "text": "Actually , screw it . I'm not going to school tomorrow .. ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2218932433, "name": "¥N¥", "screen_name": "recklesschick98", "lang": "en", "location": "Hugoton, KS", "create_at": date("2013-11-27"), "description": "|• This is the anthem •• Told the world I need everything and some |Sc- nazzy_nazzk92 @KahleKody is my right hand ~ #prayersfor @wtftaren", "followers_count": 900, "friends_count": 1050, "statues_count": 9472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hugoton, KS", "id": "34e3a230535e4695", "name": "Hugoton", "place_type": "city", "bounding_box": rectangle("-101.359093,37.162195 -101.329777,37.187947") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20189, "countyName": "Stevens", "cityID": 2033425, "cityName": "Hugoton" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920819249020928, "text": ".hate seeing my friend like this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4095954139, "name": "Kay.", "screen_name": "hilder_scott", "lang": "en", "location": "Richmond, TX", "create_at": date("2015-11-01"), "description": ".Richmond3r .19XX\nLcisd.", "followers_count": 179, "friends_count": 226, "statues_count": 568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, TX", "id": "c3bf7e4412fa281d", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-95.786253,29.546192 -95.715457,29.598952") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4861892, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-04T00:00:04.000Z"), "id": 683920819265908736, "text": "@_raesaaa n/p love.", "in_reply_to_status": 683920475169419264, "in_reply_to_user": 2321143083, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2321143083 }}, "user": { "id": 77906548, "name": "C.W.", "screen_name": "Chrisworldpeace", "lang": "en", "location": "That farmhouse...there", "create_at": date("2009-09-27"), "description": "Writer, Gladiator , Film / Book Lover Boxing and lifting weights are ok too. #filmtwitter", "followers_count": 1541, "friends_count": 998, "statues_count": 298967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, NC", "id": "8eb7d0abedc4817b", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-82.434848,34.687331 -77.29868,35.662316") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37147, "countyName": "Pitt", "cityID": 3728080, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283187854503936, "text": "@alecbballin7 it's plausible under the right circumstances", "in_reply_to_status": 684282999748165632, "in_reply_to_user": 259536151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259536151 }}, "user": { "id": 229872544, "name": "Isaiah", "screen_name": "iamisaiahwalker", "lang": "en", "location": "NB/EL. Italy is the mission. ", "create_at": date("2010-12-23"), "description": "MSU", "followers_count": 1039, "friends_count": 277, "statues_count": 35191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Baltimore, MI", "id": "6e3106cba3927dae", "name": "New Baltimore", "place_type": "city", "bounding_box": rectangle("-82.76322,42.668174 -82.705482,42.70645") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2657100, "cityName": "New Baltimore" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283187900604416, "text": "#challengeaccepted https://t.co/p3jeQ7fla9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "challengeaccepted" }}, "user": { "id": 2378787824, "name": "Henney Gawd", "screen_name": "MikeLee______", "lang": "en", "location": "Richmond, VA", "create_at": date("2014-03-08"), "description": "God First Family Second 2⃣4⃣♈️ Canes| Fins | Pistons| Tarheel Basketball| Cu", "followers_count": 933, "friends_count": 1299, "statues_count": 80441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188030619648, "text": "Temp: 44.1°F | Humidity: 88% | Wind: --- @ 0.0 mph | Barometer: 30.19 in | Dewpoint: 40.7°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 23, "friends_count": 1, "statues_count": 164242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188110229504, "text": "Love the life you live", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419213806, "name": "Nick Vargas ♉", "screen_name": "Lil_Nick275", "lang": "en", "location": "Sacramento, CA", "create_at": date("2011-11-22"), "description": "God Family Football #GoombaOriginal @California1Live Kings, Eagles, Giants", "followers_count": 800, "friends_count": 783, "statues_count": 40354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arden-Arcade, CA", "id": "b90f2a335f8565c0", "name": "Arden-Arcade", "place_type": "city", "bounding_box": rectangle("-121.423941,38.562585 -121.327437,38.645482") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602553, "cityName": "Arden-Arcade" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188324085760, "text": "https://t.co/HkmULZoXDa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1919734446, "name": "SWGOTCHO-HO-ON-KNEES", "screen_name": "SWYOHOEONKNEES", "lang": "en", "location": "null", "create_at": date("2013-09-30"), "description": "null", "followers_count": 0, "friends_count": 0, "statues_count": 25 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188403957760, "text": "Ok that don't mean anything IreAri https://t.co/73dLwv3fFV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2219945419, "name": "Loyal Black Man", "screen_name": "216Mar__", "lang": "en", "location": "Cleveland, OH", "create_at": date("2013-11-28"), "description": "TU '19", "followers_count": 1579, "friends_count": 493, "statues_count": 47239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garfield Heights, OH", "id": "4bcff2f443c05374", "name": "Garfield Heights", "place_type": "city", "bounding_box": rectangle("-81.637998,41.384569 -81.56761,41.441812") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3929428, "cityName": "Garfield Heights" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188428931072, "text": "@Complex_slays yes add me", "in_reply_to_status": 684282742297784320, "in_reply_to_user": 3841603095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3841603095 }}, "user": { "id": 2894448282, "name": "Brian Cowan", "screen_name": "Static_vQ", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "Co - Owner of @vQGaming | Working hard to aid the success of eSports.", "followers_count": 7674, "friends_count": 5473, "statues_count": 3482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188580061188, "text": "Ahh husband material ���� https://t.co/lRjG2UKAAp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46874915, "name": "Nma", "screen_name": "TheSophiaMong", "lang": "en", "location": "New York", "create_at": date("2009-06-13"), "description": "One woman army", "followers_count": 1259, "friends_count": 915, "statues_count": 20403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Bennington, VT", "id": "317b87798a937e21", "name": "North Bennington", "place_type": "city", "bounding_box": rectangle("-73.258909,42.909046 -73.224792,42.933933") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50003, "countyName": "Bennington", "cityID": 5049075, "cityName": "North Bennington" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283188680601600, "text": "It's official that I'm going to school tomorrow ugh first day of 3rd quarter ������ #ALDUBComplicated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ALDUBComplicated" }}, "user": { "id": 303373356, "name": "† fiona deleon †", "screen_name": "hellofionnaa", "lang": "en", "location": " +63&209", "create_at": date("2011-05-22"), "description": "• God's with me• bchs '17 • no worries •", "followers_count": 2354, "friends_count": 1251, "statues_count": 22878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283189045530624, "text": "That was hilarious omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2772091758, "name": "Darla Phoun♡", "screen_name": "davyphoun", "lang": "en", "location": "null", "create_at": date("2014-08-26"), "description": "sc: davyphoun", "followers_count": 592, "friends_count": 362, "statues_count": 8517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283189381038080, "text": "@cee_hogiee this shit makes me nauseous like imagine doing that with your mom or brother or cousin......... it's the same thing ����������", "in_reply_to_status": 684282586139594752, "in_reply_to_user": 293185877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 293185877 }}, "user": { "id": 3028040923, "name": "corinne", "screen_name": "corinnegarnier_", "lang": "en", "location": "lux nightclub", "create_at": date("2015-02-18"), "description": "spiritual being perceiving life through the human experience || love all, serve all, and create no sorrow || #basshead & #villager", "followers_count": 1171, "friends_count": 479, "statues_count": 11577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283189842591744, "text": "@NICKMERCS I'm a new viewer tonight bro. Just know Imma keep watching keep it up. I'll sub soon man.", "in_reply_to_status": -1, "in_reply_to_user": 98821255, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 98821255 }}, "user": { "id": 1095166082, "name": "Josh", "screen_name": "xJKirb", "lang": "en", "location": "Princeton, WV", "create_at": date("2013-01-16"), "description": "♕ sc: jkirb2016 // ig: xJKirb ♕", "followers_count": 367, "friends_count": 130, "statues_count": 6257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bluefield, VA", "id": "8057944e55790b13", "name": "Bluefield", "place_type": "city", "bounding_box": rectangle("-81.305521,37.215902 -81.165628,37.281748") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283190308044801, "text": "I'm a troll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343672263, "name": "PC zach silgero", "screen_name": "zachsilgero", "lang": "en", "location": "null", "create_at": date("2011-07-27"), "description": "null", "followers_count": 476, "friends_count": 353, "statues_count": 9336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283190375153664, "text": "Just because you know someone is always going to be there for you doesn't mean you should take their existence for granted.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153232384, "name": "Baby Girl", "screen_name": "CryBabyCutie", "lang": "en", "location": "the state of confusion ", "create_at": date("2010-06-07"), "description": "I don't want to be alone; I'd rather be left alone.", "followers_count": 275, "friends_count": 103, "statues_count": 9717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283190614179840, "text": "Diamonds dancing https://t.co/WsQrQ0MgnW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268654749, "name": "what", "screen_name": "givahn_17", "lang": "en", "location": "Texas, USA", "create_at": date("2011-03-18"), "description": "20 | Baylor 2021", "followers_count": 1086, "friends_count": 988, "statues_count": 76016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Copperas Cove, TX", "id": "f0eebea3b08e0158", "name": "Copperas Cove", "place_type": "city", "bounding_box": rectangle("-97.950002,31.081381 -97.867527,31.16156") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48099, "countyName": "Coryell", "cityID": 4816624, "cityName": "Copperas Cove" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283190765195264, "text": "already thinking about summer ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1621370971, "name": "aizaaa", "screen_name": "agmrealica", "lang": "en", "location": "null", "create_at": date("2013-07-25"), "description": "Emanon | OHS", "followers_count": 518, "friends_count": 448, "statues_count": 4705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283191503486977, "text": "@ayeeitskae how did you get in trouble? hahaha", "in_reply_to_status": -1, "in_reply_to_user": 1066178353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1066178353 }}, "user": { "id": 322007786, "name": "jayla :)", "screen_name": "itsjustjayla", "lang": "en", "location": "Gloucester Point, VA", "create_at": date("2011-06-22"), "description": "j'aime @nickcrich33", "followers_count": 1486, "friends_count": 765, "statues_count": 84911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gloucester Point, VA", "id": "01df8ab071774ced", "name": "Gloucester Point", "place_type": "city", "bounding_box": rectangle("-76.536361,37.245259 -76.445686,37.320017") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51073, "countyName": "Gloucester", "cityID": 5131616, "cityName": "Gloucester Point" } }
+{ "create_at": datetime("2016-01-05T00:00:00.000Z"), "id": 684283191755014145, "text": "My faaaaaav vid of all time https://t.co/Y7j9X2qGdP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277842120, "name": "Jan8th", "screen_name": "_twintaay", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-04-05"), "description": "#twinnation QUEEN of SD, CA☀️ Helix sc:caliyungster IG:_twintaay THETA", "followers_count": 5416, "friends_count": 3970, "statues_count": 69321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192363204609, "text": "@grantlandis always there for u bud", "in_reply_to_status": 684282904252317696, "in_reply_to_user": 583280017, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 583280017 }}, "user": { "id": 67775437, "name": "vivi", "screen_name": "googledgrant", "lang": "en", "location": "null", "create_at": date("2009-08-21"), "description": "did you just google grant...", "followers_count": 4991, "friends_count": 839, "statues_count": 77113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192463863808, "text": "WTF IMMA BEAT YO ASS! https://t.co/0coSNJmluY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906498706, "name": "K.h.a.i.l.y.n", "screen_name": "Khailyn_x", "lang": "en", "location": "Palmdale, CA", "create_at": date("2014-11-21"), "description": "PHS|| Basketball❤️ Colombian/ Mexican❤️@daywaycontinue is my bff|| Robert Salone❤️", "followers_count": 300, "friends_count": 281, "statues_count": 4475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192577290240, "text": "GET AWAY GET AWAAAAAAAAAAAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603122360, "name": "♥dweebus♥", "screen_name": "GleefulChibi", "lang": "en", "location": "Chipmunk Hell ", "create_at": date("2012-06-08"), "description": "Crazed Pentaholic, Chipmunk lover, and obsessive Scott Hoying fangirl. I'm SNK trash.", "followers_count": 797, "friends_count": 356, "statues_count": 74599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warwick, RI", "id": "d1871b0189c27a21", "name": "Warwick", "place_type": "city", "bounding_box": rectangle("-71.518925,41.629501 -71.354415,41.76838") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44003, "countyName": "Kent", "cityID": 4474300, "cityName": "Warwick" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192707166211, "text": "Caitlyn Jenner https://t.co/0UJTGRPXCa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 304228202, "name": "ricky", "screen_name": "ricky_b27", "lang": "en", "location": "null", "create_at": date("2011-05-23"), "description": "19, sc/ig: ricky_b27", "followers_count": 1227, "friends_count": 1237, "statues_count": 26462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192711507970, "text": "@hannahemanuelll graduated early ��������", "in_reply_to_status": -1, "in_reply_to_user": 2278497734, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2278497734 }}, "user": { "id": 514483001, "name": "A", "screen_name": "joseph_malak44", "lang": "en", "location": "Tennessee, USA", "create_at": date("2012-03-04"), "description": "bless up", "followers_count": 975, "friends_count": 764, "statues_count": 9433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gallatin, TN", "id": "e08aaac2b23fd3a3", "name": "Gallatin", "place_type": "city", "bounding_box": rectangle("-86.550888,36.316163 -86.378795,36.428963") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47165, "countyName": "Sumner", "cityID": 4728540, "cityName": "Gallatin" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283192753295360, "text": "HAHA BYE I HATE THIS https://t.co/A4YVB4JuhR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332690456, "name": "BIG GUCCI ANDIE ⚡️", "screen_name": "sleeepyqt", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-07-10"), "description": "YSL for life bitch", "followers_count": 305, "friends_count": 153, "statues_count": 16068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283193105592320, "text": "@Dory @ryannwood63", "in_reply_to_status": 684252983090089986, "in_reply_to_user": 1187647735, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1187647735, 1368283424 }}, "user": { "id": 1894858585, "name": "Marissa Anderson", "screen_name": "marissaraye16", "lang": "en", "location": "|MV| ", "create_at": date("2013-09-22"), "description": "kik/snapchat: marissaraye16", "followers_count": 402, "friends_count": 511, "statues_count": 2098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283193218875392, "text": "@OfficialWebbsy @alekseiigunn I'm here rn we can play???", "in_reply_to_status": 684282943812964352, "in_reply_to_user": 2290212146, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2290212146, 848068776 }}, "user": { "id": 2231103116, "name": "2016 Alcoholic", "screen_name": "tanyonjorgensen", "lang": "en", "location": "up your moms mediocre dick", "create_at": date("2013-12-04"), "description": "Dating the love of my life @JackDaniels_US ⚰singer⚰ *SADBOYZ* 2 months until Oahu", "followers_count": 746, "friends_count": 786, "statues_count": 16400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindon, UT", "id": "5e1dbff4811038b0", "name": "Lindon", "place_type": "city", "bounding_box": rectangle("-111.763526,40.326209 -111.662197,40.352543") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4945090, "cityName": "Lindon" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283193848119298, "text": "Can those emojis be used on Twitter https://t.co/D7MIPQ4TSd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301765265, "name": "♡ sara ellen ♡", "screen_name": "saralatable", "lang": "en", "location": "california", "create_at": date("2011-05-19"), "description": "my name is sara ellen & clen is coming to see me in march", "followers_count": 1208, "friends_count": 173, "statues_count": 14710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newnan, GA", "id": "00a0d2e62d54ed89", "name": "Newnan", "place_type": "city", "bounding_box": rectangle("-84.832672,33.343601 -84.693369,33.416232") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13077, "countyName": "Coweta", "cityID": 1355020, "cityName": "Newnan" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283193923485696, "text": "Me and dad https://t.co/ogL5nfqGCa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 475313523, "name": "Legend", "screen_name": "JoeyLiftsHeavy", "lang": "en", "location": "Austin, TX", "create_at": date("2012-01-26"), "description": "D.M/BMW/Akina Speed Star", "followers_count": 2162, "friends_count": 961, "statues_count": 40286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194003161088, "text": "https://t.co/X5LIjx9M0R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2392047726, "name": "Josh", "screen_name": "jccaanen", "lang": "en", "location": "SATX", "create_at": date("2014-03-15"), "description": "i count six shots", "followers_count": 151, "friends_count": 173, "statues_count": 644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Converse, TX", "id": "dcd50868e563bbed", "name": "Converse", "place_type": "city", "bounding_box": rectangle("-98.352959,29.489943 -98.276076,29.538642") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4816468, "cityName": "Converse" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194078707712, "text": "Last year ��", "in_reply_to_status": 556178543886221314, "in_reply_to_user": 788352284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 788352284, "name": "Ric'", "screen_name": "RickyC___", "lang": "en", "location": "null", "create_at": date("2012-08-28"), "description": "-The core of man's spirit comes from new experiences-", "followers_count": 912, "friends_count": 637, "statues_count": 42764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berwyn, IL", "id": "e363b61fbd793d7d", "name": "Berwyn", "place_type": "city", "bounding_box": rectangle("-87.804144,41.821108 -87.777785,41.865409") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1705573, "cityName": "Berwyn" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194217111553, "text": "1. Father aint got no edge up ! That new grow came and fucked up the whole picture", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68111983, "name": "Emmitt Charlz 2/3", "screen_name": "LongwayNigga", "lang": "en", "location": "75165 TX.", "create_at": date("2009-08-23"), "description": "|IG: _Cheeso4xx| |SC: Cheesoo13| |21| |#CMC |", "followers_count": 1134, "friends_count": 919, "statues_count": 70478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waxahachie, TX", "id": "45ff3ab084a1f419", "name": "Waxahachie", "place_type": "city", "bounding_box": rectangle("-96.880016,32.316564 -96.79913,32.494604") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4876816, "cityName": "Waxahachie" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194321981441, "text": "I'm the BIC in a room full of Best Choice ��✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352831165, "name": "Aussy Moe$", "screen_name": "AustenNotTexas", "lang": "en", "location": "JXN ", "create_at": date("2011-08-10"), "description": "lil vampire booty ass / chi 15-11 / #BeezNation", "followers_count": 1879, "friends_count": 1626, "statues_count": 65638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.845486,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194435350528, "text": "If I left you all alone would you still pick up the phone ,.. It's iffy ,.you miss me ?#", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60772246, "name": "✘▼King Kapp▼✘", "screen_name": "TAE_2wice_SVO", "lang": "en", "location": "kinston NC", "create_at": date("2009-07-27"), "description": "#King.Kush.CocainE....welcome to Kapp Nation ...#KING_KAPP...#AMG #1250,. #Free-Work ,. Dab_King.", "followers_count": 339, "friends_count": 58, "statues_count": 10888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kinston, NC", "id": "1427f884feb4cb70", "name": "Kinston", "place_type": "city", "bounding_box": rectangle("-77.652885,35.227946 -77.543004,35.323017") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37107, "countyName": "Lenoir", "cityID": 3735920, "cityName": "Kinston" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194439368704, "text": "Happy birthday Lorena have a good day or week cx ���� @renvsegurv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8779818,33.9811489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3193615119 }}, "user": { "id": 3139649384, "name": "Super Duper Javiiii", "screen_name": "javiermejia7799", "lang": "en", "location": "null", "create_at": date("2015-04-04"), "description": "let's just sit on the floor!-miguel carranza", "followers_count": 109, "friends_count": 209, "statues_count": 845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194439385089, "text": "@_yvngleek yessss", "in_reply_to_status": 684282544754307072, "in_reply_to_user": 568372961, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 568372961 }}, "user": { "id": 345753724, "name": "sel", "screen_name": "selenamariec_", "lang": "en", "location": "null", "create_at": date("2011-07-30"), "description": "snapchat: selenamariec", "followers_count": 1544, "friends_count": 795, "statues_count": 63362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194460516353, "text": "Wind 0.0 mph ---. Barometer 30.539 in, Steady. Temperature 22.1 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194657488896, "text": "lmao I am Myles ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95365426, "name": "Brina✨", "screen_name": "brinaaaaxo", "lang": "en", "location": "Long Beach, CA. ", "create_at": date("2009-12-07"), "description": "19. •• k&mxjb", "followers_count": 555, "friends_count": 329, "statues_count": 25808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194745569280, "text": "One of my favorites https://t.co/4AlBy7gHiQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153239115, "name": "That motherfucka", "screen_name": "PaulieSHEEM", "lang": "en", "location": "the best side", "create_at": date("2010-06-07"), "description": "You hustlin backwards if you chasen a bitch stupid chase the paper dey cum wit the shit", "followers_count": 268, "friends_count": 764, "statues_count": 22289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherryland, CA", "id": "3b93b1d81c90ff1e", "name": "Cherryland", "place_type": "city", "bounding_box": rectangle("-122.122215,37.666826 -122.090037,37.688086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 612902, "cityName": "Cherryland" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283194900877312, "text": "01/05@03:00 - Temp 14.6F, WC 5.2F. Wind 6.3mph NNW, Gust 15.0mph. Bar 30.522in, Rising. Rain 0.00in. Hum 60%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283195156738048, "text": "Wind 0 mph --. Barometer 30.40 in, Rising slowly. Temperature 32.2 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 19971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283195303571456, "text": "How is it only 3am?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179869869, "name": "T R A P C O L E", "screen_name": "Architex_", "lang": "en", "location": "jc/nj", "create_at": date("2010-08-18"), "description": "Spoken Word artist. Trying to figure out what my future holds. IG- @lordeggzavier_ ChatSnap- @architexx", "followers_count": 464, "friends_count": 395, "statues_count": 22984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283195395805184, "text": "Mykeila think she gon call me while she at school Ima be knocked tf out ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407804976, "name": "shadashia.❄️", "screen_name": "LivinLikeABOSS_", "lang": "en", "location": "334", "create_at": date("2011-11-08"), "description": "unfuckwitable. #myasu19|SC:Shadashuhh", "followers_count": 2714, "friends_count": 1639, "statues_count": 103522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dothan, AL", "id": "d2981a0f3d8eddd7", "name": "Dothan", "place_type": "city", "bounding_box": rectangle("-85.515405,31.130472 -85.334617,31.33397") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1069, "countyName": "Houston", "cityID": 121184, "cityName": "Dothan" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283195538325504, "text": "@Julio133 thank you bruh!", "in_reply_to_status": 684282759293042692, "in_reply_to_user": 32713123, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32713123 }}, "user": { "id": 3193357342, "name": "Lord Killer B✌", "screen_name": "Allante243_", "lang": "en", "location": "somewhere on earth", "create_at": date("2015-04-21"), "description": "Loading...", "followers_count": 251, "friends_count": 553, "statues_count": 7353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283195601227776, "text": "I just set my Tinder discovery preferences for ages 21-25 for fun and I was horrified ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35297148, "name": "Eugenie Grey", "screen_name": "feralcreature", "lang": "en", "location": "LA & NYC", "create_at": date("2009-04-25"), "description": "Blogger, marketing & social media consultant, professional chihuahua wrangler, intersectional feminist. R u woke? IG: @feralcreature", "followers_count": 9222, "friends_count": 159, "statues_count": 28849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283196003844096, "text": "Rip Marv�� https://t.co/0OHtV0GYpq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146588217, "name": "YDBravo", "screen_name": "Harold_Tha_III", "lang": "en", "location": "Somewhere Smokin a blunt", "create_at": date("2013-02-03"), "description": "Young money got a nigga feeling old", "followers_count": 805, "friends_count": 746, "statues_count": 9047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283196033204224, "text": "If I ever talked to my mom like Farrah talks to her mom, she would cut me right the fuck off.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323542505, "name": "Kathy", "screen_name": "katiekayday", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "I dont know what to say on bios. But taken by Tyler James Larsen ♥", "followers_count": 339, "friends_count": 532, "statues_count": 8900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-01-05T00:00:01.000Z"), "id": 684283196066799617, "text": "It was a great idea to look up food on Pinterest ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609538310, "name": "Alina", "screen_name": "asdfjklxalina", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": "@ashleyluangrath is my bitch.", "followers_count": 264, "friends_count": 209, "statues_count": 2304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534889,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283196553363456, "text": "let's geddit my nigs �� https://t.co/jn3mkm1CEC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26300328, "name": "Ashley All Day", "screen_name": "lilfacetatt", "lang": "en", "location": "LA / THA BAY", "create_at": date("2009-03-24"), "description": "• Newport Shawty • instagram: ashleyalldaaay • snapchat: ashleyallday • bookingallday@gmail.com ••• http://soundcloud.com/ashleyalldaaay-1", "followers_count": 42802, "friends_count": 366, "statues_count": 50114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Druid Hills, GA", "id": "590c1000bfb45cf6", "name": "North Druid Hills", "place_type": "city", "bounding_box": rectangle("-84.348564,33.799908 -84.303938,33.840414") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1356168, "cityName": "North Druid Hills" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283196750495744, "text": "@EatsMeetsWes nice work", "in_reply_to_status": 684277498809577472, "in_reply_to_user": 359074545, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 359074545 }}, "user": { "id": 7362852, "name": "Craig Minami", "screen_name": "craigminla", "lang": "en", "location": "los angeles", "create_at": date("2007-07-09"), "description": "null", "followers_count": 554, "friends_count": 511, "statues_count": 23892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283196972920832, "text": "03:00 23.2°F Feels:23.2°F (Hi28.0°F/Lo23.2°F) Hum:71% Wnd:--- 0.0MPH Baro:30.48in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 158, "friends_count": 261, "statues_count": 24299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283197128077312, "text": "@P_Hart8899 @Abigail_ElliseX @Saint_Nic765 @KMeshelleXoXo idk play along P", "in_reply_to_status": 684282767031533568, "in_reply_to_user": 970809522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 970809522, 904504958, 417322914, 126183359 }}, "user": { "id": 99147483, "name": "bryce.", "screen_name": "hwvrd", "lang": "en", "location": "765", "create_at": date("2009-12-24"), "description": "null", "followers_count": 723, "friends_count": 955, "statues_count": 10925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muncie, IN", "id": "01c14352f8d6ca6e", "name": "Muncie", "place_type": "city", "bounding_box": rectangle("-85.461887,40.121825 -85.320813,40.272656") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18035, "countyName": "Delaware", "cityID": 1851876, "cityName": "Muncie" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283197241200641, "text": "@_kimcacho @ManCandyPics you think your black��", "in_reply_to_status": 684282757908844544, "in_reply_to_user": 2262623682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2262623682, 1308678415 }}, "user": { "id": 2614455757, "name": "Medina", "screen_name": "Ayeeemedina", "lang": "en", "location": "null", "create_at": date("2014-07-09"), "description": "21.", "followers_count": 266, "friends_count": 249, "statues_count": 4905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283197274853377, "text": "Temp: 32.4°F Wind:1.5mph Pressure: 30.395hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283197434261504, "text": "back at 1 ✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357317081, "name": "Roosgini", "screen_name": "_WishUponAGini", "lang": "en", "location": "SoFlo", "create_at": date("2011-08-17"), "description": "lips so creole, ass so haitian.", "followers_count": 605, "friends_count": 305, "statues_count": 65846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise, FL", "id": "14441682ff5a9fa3", "name": "Sunrise", "place_type": "city", "bounding_box": rectangle("-80.363112,26.119383 -80.226673,26.193854") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1269700, "cityName": "Sunrise" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283197786591232, "text": "Wind 3.0 mph ENE. Barometer 30.578 in, Rising slowly. Temperature 23.8 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283198206033920, "text": "#Support Origin Melissa! 23.0°F Wind:2.7mph Pressure: 30.47hpa Steady Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Support" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283198260559872, "text": "#elitehookahs el_luimi_ redchazey I don't fuck with fake people my people since #iwas15 @… https://t.co/RXSo6arvH2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9217453,40.8403549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "elitehookahs", "iwas15" }}, "user": { "id": 489463381, "name": "yankeedominicand", "screen_name": "sauryvargas27", "lang": "en", "location": "manhattan", "create_at": date("2012-02-11"), "description": "life its too short....meet me at the top\r\n.", "followers_count": 161, "friends_count": 249, "statues_count": 4355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283198516379648, "text": "I said I'd go to bed at 3 but I'm just lying to myself. :(:", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737060905, "name": "az", "screen_name": "WizCoylifa", "lang": "en", "location": "KKKD", "create_at": date("2012-08-04"), "description": "dude I love passion pit", "followers_count": 1301, "friends_count": 198, "statues_count": 42011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, FL", "id": "2bc7c264a080898b", "name": "Hollywood", "place_type": "city", "bounding_box": rectangle("-80.248975,25.986228 -80.107808,26.093132") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1232000, "cityName": "Hollywood" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283198663188482, "text": "These niggas don't know how to play they role they act like paper all they do is fold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420831626, "name": "Proud 2 Be A Problem", "screen_name": "__JetLi__", "lang": "en", "location": "null", "create_at": date("2011-11-24"), "description": "rip Clarence, Mad Dog, Aunt Sarah, Aunt Sue, My Dear", "followers_count": 1091, "friends_count": 540, "statues_count": 63423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, AL", "id": "7ae765412ef88940", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-87.721811,34.77898 -87.580289,34.8896") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1077, "countyName": "Lauderdale", "cityID": 126896, "cityName": "Florence" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283198700830720, "text": "35.1F (Feels: 35.1F) - Humidity: 84% - Wind: 6.9mph NE - Gust: 7.6mph - Pressure: 1041.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 221082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199313154048, "text": "Wow. People so shallow lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322789718, "name": "Jam", "screen_name": "JamariSelders", "lang": "en", "location": "DTX✈️ATL", "create_at": date("2011-06-23"), "description": "| Morehouse College |", "followers_count": 4248, "friends_count": 1758, "statues_count": 14106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199422267394, "text": "I might call your bitch when im bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043250764, "name": "reaganomics", "screen_name": "reagan_allday", "lang": "en", "location": "Houston, TX ", "create_at": date("2012-12-28"), "description": "I'm procrastinating at this very moment", "followers_count": 877, "friends_count": 577, "statues_count": 11251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199497834496, "text": "Earl, this sample gave me my favorite domo verse g https://t.co/hvseUyFHzL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446815325, "name": "03 ray lewis ﹩", "screen_name": "avo_cain", "lang": "en", "location": "hine $ide", "create_at": date("2011-12-25"), "description": "ain't never been no pressure.. life is funny like that. location on, just pull up. 1/4 JOBS. #UA19", "followers_count": 1727, "friends_count": 1413, "statues_count": 98836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, AL", "id": "1ccebd350c541331", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-87.015695,34.748712 -86.885875,34.837961") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1083, "countyName": "Limestone", "cityID": 102956, "cityName": "Athens" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199527096320, "text": "@PsychicStoner and it turned out he's still alive after whatever happened in the jail", "in_reply_to_status": 684282143489540096, "in_reply_to_user": 155910701, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155910701 }}, "user": { "id": 816658735, "name": "kneel b4 Zod", "screen_name": "b4_zod", "lang": "en", "location": "banished from krypton", "create_at": date("2012-09-10"), "description": "progressive intergalactic criminal in pursuit of truth. not rj sterling. Go jayhawks, chiefs, cardinals, blues.", "followers_count": 1760, "friends_count": 2019, "statues_count": 93506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "League City, TX", "id": "cf3b45e29c2c319b", "name": "League City", "place_type": "city", "bounding_box": rectangle("-95.178987,29.438994 -94.995071,29.55532") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841980, "cityName": "League City" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199623581696, "text": "@Caramel4791 no way. tell him he needs to get swole again and to train me! Lol", "in_reply_to_status": 684274982881382401, "in_reply_to_user": 333236700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333236700 }}, "user": { "id": 232495371, "name": "ERICK", "screen_name": "ImAtomicMan", "lang": "en", "location": "pallet, kanto (austin, tx)", "create_at": date("2010-12-31"), "description": "i'm the moon that pulls the tides that takes the sand", "followers_count": 808, "friends_count": 1087, "statues_count": 63191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199648825345, "text": "Good evening #Edmonton! Sun will set in 30 minutes (local time 16:30, UTC 23:30) #Sunset #Photo #Photography [05.01.2016]", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.46667,53.55"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Edmonton", "Sunset", "Photo", "Photography" }}, "user": { "id": 4567387397, "name": "PhotoSchedule", "screen_name": "PhotoSchedule", "lang": "en", "location": "Finland", "create_at": date("2015-12-22"), "description": "We bring up sunrise and sunset around the world so you can get your camera ready or just enjoy the view. Developed by @JanneHonkonen / @TheOwlkeep", "followers_count": 62, "friends_count": 3, "statues_count": 10911 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199732633600, "text": "LOOK #LIMITLESS #TV #Pilot #Script - #JakeMcDorman ,#JenniferCarpenter, #HillHarper, #BlairBrown, #BradleyCooper https://t.co/Qa7OgnewP3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LIMITLESS", "TV", "Pilot", "Script", "JakeMcDorman", "JenniferCarpenter", "HillHarper", "BlairBrown", "BradleyCooper" }}, "user": { "id": 717544904, "name": "Kyle James Abbot", "screen_name": "KyleAbbot", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2012-07-25"), "description": "Hi, I'm 23, 6'4, 205, Blond/Blue. Fun, friendly, masculine, #athletic #gay guy in #LosAngeles. Student. Big #TV fan:) My Amazon WishList http://t.co/i4rxZfd5Bp", "followers_count": 56809, "friends_count": 54606, "statues_count": 76541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283199984291840, "text": "#Messiah 48:16, Fifth Revolution, #Cihuacoatl 76/#Saguaro, Belteshazzar-#Kamala, #Genesis #Matrix \n\n#Chessboard", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Messiah", "Cihuacoatl", "Saguaro", "Kamala", "Genesis", "Matrix", "Chessboard" }}, "user": { "id": 70626091, "name": "Left Hand Pan", "screen_name": "666Beastism999", "lang": "en", "location": "Spokane, WA", "create_at": date("2009-09-01"), "description": "Messiah, The Christ, Tezcatlipoca Card, Ace of Jihad, Sanctified Interface", "followers_count": 368, "friends_count": 273, "statues_count": 206594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283200084914176, "text": "https://t.co/OKpStuqtKW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2257395174, "name": "ahsan khan", "screen_name": "AHSANgtfo", "lang": "en", "location": "null", "create_at": date("2013-12-21"), "description": "ride wit me", "followers_count": 430, "friends_count": 217, "statues_count": 6327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283200105889792, "text": "People are funny https://t.co/bA308zhOax", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1864806810, "name": "Alex", "screen_name": "alexanduruh", "lang": "en", "location": "Hayward ↗️ Stockton ↗️ Sac ", "create_at": date("2013-09-14"), "description": "Probably chillin with my boo", "followers_count": 527, "friends_count": 351, "statues_count": 17398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-05T00:00:02.000Z"), "id": 684283200210898944, "text": "Wind 0.7 mph SW. Barometer 30.56 in, Rising. Temperature 3.2 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 89, "statues_count": 156432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283200537935872, "text": "@indiaaaam you know me lmao", "in_reply_to_status": 684283056862015488, "in_reply_to_user": 114364756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 114364756 }}, "user": { "id": 52276893, "name": "jay305", "screen_name": "JayForJustin", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-29"), "description": "19. Black Activist. Researcher. Future Lawyer. College Educated. ig: 6ixmanjay. snapchat: justinbtight #blacklivesmatter #blackmoneymatters", "followers_count": 1917, "friends_count": 787, "statues_count": 131584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Athens, CA", "id": "81923aaf1207edc1", "name": "West Athens", "place_type": "city", "bounding_box": rectangle("-118.317804,33.916377 -118.291514,33.931268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684116, "cityName": "West Athens" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283201108357120, "text": "@EarlJ_tocold lol yes", "in_reply_to_status": 684282418090475520, "in_reply_to_user": 416166958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 416166958 }}, "user": { "id": 1098839071, "name": "cocaaa", "screen_name": "_KLCH_", "lang": "en", "location": "UCA ", "create_at": date("2013-01-17"), "description": "nursing ❤️", "followers_count": 1014, "friends_count": 794, "statues_count": 40906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283201196568576, "text": "I was so high Sunday I ate a burger. And I hate burgers it was just there..��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258467197, "name": "DEJA.", "screen_name": "dej13__", "lang": "en", "location": "null", "create_at": date("2011-02-27"), "description": "#NSU19 ✊", "followers_count": 5310, "friends_count": 5110, "statues_count": 142583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, VA", "id": "60edfde178b362ff", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-76.420402,36.786146 -76.291434,36.897382") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51740, "countyName": "Portsmouth", "cityID": 5164000, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283201813000192, "text": "1- I hope the real world out there is treating you good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 846394932, "name": "rose", "screen_name": "roseannapasveer", "lang": "en", "location": "null", "create_at": date("2012-09-25"), "description": "'I will walk by faith even when I can not see' - 2 Corinthians 5:7", "followers_count": 555, "friends_count": 633, "statues_count": 7946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283202236755969, "text": "#BalorClub vs #BulletClub should be fun https://t.co/28ExJidp7u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BalorClub", "BulletClub" }}, "user": { "id": 62013596, "name": "Brandon Cox", "screen_name": "HEEL_BLC", "lang": "en", "location": "Lubbock Tx", "create_at": date("2009-08-01"), "description": "#GoSpursGo #WreckEmTech #TexasRangers #WWEFanatic Favorite Wrestler Sasha Fucking Banks #SashaBanksMark", "followers_count": 857, "friends_count": 1685, "statues_count": 54422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Littlefield, TX", "id": "fd4a2f8903d8a73d", "name": "Littlefield", "place_type": "city", "bounding_box": rectangle("-102.355122,33.89945 -102.311606,33.939433") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48279, "countyName": "Lamb", "cityID": 4843024, "cityName": "Littlefield" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283202333184002, "text": "lol unreal https://t.co/cEhr6PNeLt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324199872, "name": "Кејли", "screen_name": "kstolar3", "lang": "en", "location": "814➟412", "create_at": date("2011-06-25"), "description": "you're killin me, smalls", "followers_count": 434, "friends_count": 457, "statues_count": 24476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Erie, PA", "id": "29aaa88d9fe74b50", "name": "Erie", "place_type": "city", "bounding_box": rectangle("-80.239991,42.018414 -79.934073,42.202992") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4224000, "cityName": "Erie" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283202551197696, "text": "I have to remind myself on the daily that I'm freaking fabulous & although some days it's hard to believe I gotta push through & be fabulous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1011834037, "name": "Stephany⚓️", "screen_name": "CLSTEPHANY", "lang": "en", "location": "Tacoma Washington", "create_at": date("2012-12-14"), "description": "~You were put on this earth to achieve your greatest self, to live out your purpose, and to do it courageously.", "followers_count": 208, "friends_count": 218, "statues_count": 13088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clover Creek, WA", "id": "6a3c00987122ff28", "name": "Clover Creek", "place_type": "city", "bounding_box": rectangle("-122.415738,47.110842 -122.357057,47.159203") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5313215, "cityName": "Clover Creek" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283203410997248, "text": "thighs too fat for this knee brace", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132764190, "name": "18.", "screen_name": "cristalteaa_", "lang": "en", "location": "Arlington, TX", "create_at": date("2010-04-13"), "description": "eating pizza by my lonesome", "followers_count": 664, "friends_count": 508, "statues_count": 40164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283203549589504, "text": "Lmaoooooo this is me 100000 https://t.co/wPGCOa61lA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1449666398, "name": "B", "screen_name": "RealbMogul", "lang": "en", "location": "null", "create_at": date("2013-05-22"), "description": "all business no play", "followers_count": 1635, "friends_count": 293, "statues_count": 10614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283203725570049, "text": "END VIOLENCE AGAINST WOMEN\nhttps://t.co/AXJqJPUQ7J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245108283, "name": "LosCabos411", "screen_name": "loscabos411", "lang": "en", "location": "LOS CABOS, MEXICO", "create_at": date("2011-01-30"), "description": "Los Cabos 411 Your 1-Stop Source for Reliable Local Information \r\n\r\nWe'd be happy to hear from you!\r\nLosCabos411 PR Team\r\nhttp://www.LosCabos411.com", "followers_count": 2451, "friends_count": 2706, "statues_count": 20935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, CA", "id": "741e9df4d2522275", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-118.427344,34.052559 -118.371934,34.112434") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 606308, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283204203708416, "text": "Wtf ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2381415211, "name": "N a y e l i ✨", "screen_name": "ncvxo_", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "Est 97'", "followers_count": 624, "friends_count": 254, "statues_count": 9146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283204325396480, "text": "@herrokawaii @xTomori this girl is kawaii as fuck tho", "in_reply_to_status": 684282756570955776, "in_reply_to_user": 2269600935, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2269600935, 364053742 }}, "user": { "id": 2292285210, "name": "proph-kun. μ's", "screen_name": "Prophkun", "lang": "en", "location": "Washington, USA", "create_at": date("2014-01-14"), "description": "leader of @SmpleYetCmplex \nthe cantina of cocks", "followers_count": 370, "friends_count": 506, "statues_count": 9123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283204354850816, "text": "watching the fault in our stars til I fall asleep bye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3838620789, "name": "sky✨", "screen_name": "__skylarrayne", "lang": "en", "location": "Massillon, OH", "create_at": date("2015-10-01"), "description": "rip dac ❣ free kevone ‼️", "followers_count": 468, "friends_count": 379, "statues_count": 6525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massillon, OH", "id": "005f11de9931c8a4", "name": "Massillon", "place_type": "city", "bounding_box": rectangle("-81.631799,40.732694 -81.432311,40.896962") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3948244, "cityName": "Massillon" } }
+{ "create_at": datetime("2016-01-05T00:00:03.000Z"), "id": 684283204472279040, "text": "You were the best thing that ever happened to me, but I was the worst that ever happened to you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3164445103, "name": "courtney", "screen_name": "_courtneycollis", "lang": "en", "location": "logansport ", "create_at": date("2015-04-19"), "description": "“Just hold my hand even if you don’t understand.”", "followers_count": 278, "friends_count": 297, "statues_count": 1298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Logansport, IN", "id": "21588d4d5485857e", "name": "Logansport", "place_type": "city", "bounding_box": rectangle("-86.413951,40.721652 -86.29038,40.779044") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18017, "countyName": "Cass", "cityID": 1844658, "cityName": "Logansport" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283204673601536, "text": "Wind 0.0 mph ---. Barometer 30.365 in, Steady. Temperature 28.5 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 55906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283204845453312, "text": "@NickRossi330 @Agonzalez18Lexi relax ��", "in_reply_to_status": 684282417553604610, "in_reply_to_user": 1170664681, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1170664681, 3255160476 }}, "user": { "id": 2722691872, "name": "Lina", "screen_name": "LinaGomze", "lang": "en", "location": "AT&T Park", "create_at": date("2014-07-23"), "description": "smhs // sc: liinnnaa // Joe Panik", "followers_count": 379, "friends_count": 982, "statues_count": 4444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283205042761728, "text": "How do accept flaws? What is acceptance? What is a flaw? What differentiates flaws from setting standards...I'm conflicted af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971173200, "name": "Nasir J. Square", "screen_name": "Quepacshakur", "lang": "en", "location": "NuRu", "create_at": date("2012-11-25"), "description": "coolest nobody...nobody know", "followers_count": 301, "friends_count": 747, "statues_count": 14568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, NJ", "id": "74767081120f9b42", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-74.817535,40.183967 -74.729138,40.248298") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3474000, "cityName": "Trenton" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283205273272320, "text": "Happy birthday bff �������� @jgallo36", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 445236540 }}, "user": { "id": 2583452166, "name": "Amber", "screen_name": "ambear_1612", "lang": "en", "location": "null", "create_at": date("2014-06-22"), "description": "happy as happy can be :)", "followers_count": 646, "friends_count": 595, "statues_count": 6628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollister, CA", "id": "2417f030a58de0ce", "name": "Hollister", "place_type": "city", "bounding_box": rectangle("-121.45069,36.816282 -121.363572,36.874852") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6069, "countyName": "San Benito", "cityID": 634120, "cityName": "Hollister" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283205843681280, "text": "Damn we reek����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 107295985, "name": "Johnny Boii", "screen_name": "JT_Hawaiannigga", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2010-01-21"), "description": "SC jt_iokepa", "followers_count": 774, "friends_count": 662, "statues_count": 51934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283206544183296, "text": "2.5 magnitude #earthquake. 69 km from Lakeview, OR, #UnitedStates https://t.co/4NRjhQzAg0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.6313,41.8784"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 60230, "friends_count": 10, "statues_count": 92459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283206841978880, "text": "I wish this how would stop facetiming me the same time everyday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616973273, "name": "Meg", "screen_name": "megs_meow", "lang": "en", "location": "r.i.p. joshy", "create_at": date("2012-06-24"), "description": "Canyon Senior '16 / YIC / SOC / Track & Field", "followers_count": 996, "friends_count": 308, "statues_count": 31516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283206925811712, "text": "@mandytrejo10 ohhh��", "in_reply_to_status": 684282079383650304, "in_reply_to_user": 379347140, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 379347140 }}, "user": { "id": 1084867592, "name": "Nandizzle", "screen_name": "fernando3_fc", "lang": "en", "location": "null", "create_at": date("2013-01-12"), "description": "snap: fernando_324", "followers_count": 618, "friends_count": 516, "statues_count": 9511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207034908672, "text": "11% I'll go to bed once my phone dies", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983544132, "name": "MILCA✨", "screen_name": "Notorius_thug", "lang": "en", "location": "Washington, USA", "create_at": date("2012-12-01"), "description": "SC milky_way23 • I ♥️ Avocados • 19 ♋️", "followers_count": 476, "friends_count": 359, "statues_count": 37213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207047495680, "text": "@haleymferrell I mean same", "in_reply_to_status": 684235608718442497, "in_reply_to_user": 271485447, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 271485447 }}, "user": { "id": 3048946385, "name": "Tristan Jones", "screen_name": "tristanmjones", "lang": "en", "location": "Atlanta ➡️ Nashville", "create_at": date("2015-02-20"), "description": "I laugh at my own jokes.", "followers_count": 416, "friends_count": 315, "statues_count": 5142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207081054208, "text": "@toms TOMS Flagship #Venice https://t.co/DG3ETgbKbN Excellent https://t.co/yalUjemGgU", "in_reply_to_status": -1, "in_reply_to_user": 15010370, "favorite_count": 0, "coordinate": point("-118.46665052,33.99066795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Venice" }}, "user_mentions": {{ 15010370 }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3646, "friends_count": 507, "statues_count": 162067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207114596352, "text": "Innocent? Temi Temi ���������������������������� https://t.co/hxw5uNRHzZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 238732073, "name": "ㅤ", "screen_name": "Ceebeelah", "lang": "en", "location": "AZ", "create_at": date("2011-01-15"), "description": "Me being a realist makes it seem like I'm a pessimist to most. | ASU'17", "followers_count": 3260, "friends_count": 1216, "statues_count": 117067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207194427392, "text": "@shelbya_xo put everyone on loft !!!!", "in_reply_to_status": -1, "in_reply_to_user": 2955756557, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2955756557 }}, "user": { "id": 3002312823, "name": "Z☯", "screen_name": "zeynabby", "lang": "en", "location": "Gaza / Tulkarm - Palestine ", "create_at": date("2015-01-27"), "description": "robin", "followers_count": 481, "friends_count": 617, "statues_count": 8015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orland Park, IL", "id": "0da8a6c990f02eed", "name": "Orland Park", "place_type": "city", "bounding_box": rectangle("-87.911936,41.552464 -87.790471,41.65669") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1756640, "cityName": "Orland Park" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207215374336, "text": "\"I don't want no liquor cause I been drinking that dirtyyyyyy��\" https://t.co/k2NewOEoeR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 863761104, "name": "dom ◡̈", "screen_name": "dommmer", "lang": "en", "location": "staying faithful", "create_at": date("2012-10-05"), "description": "cuff me b4 its 2 l8 IG: domdiggittyy", "followers_count": 1043, "friends_count": 974, "statues_count": 3134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-05T00:00:04.000Z"), "id": 684283207601139712, "text": "I'm so curious as to why it is 2 degrees in my room", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450035733, "name": "alex bowman", "screen_name": "ABowLikeYo", "lang": "en", "location": "null", "create_at": date("2011-12-29"), "description": "null", "followers_count": 725, "friends_count": 493, "statues_count": 17636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645575917191168, "text": "when we laid in the booth oh my god������ @luciferwhassup https://t.co/1DDBzeu7DV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1543880750 }}, "user": { "id": 66892707, "name": "holly", "screen_name": "hollylong_", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-08-18"), "description": "null", "followers_count": 1036, "friends_count": 421, "statues_count": 11150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuckahoe, VA", "id": "b49dacd1097b492f", "name": "Tuckahoe", "place_type": "city", "bounding_box": rectangle("-77.651963,37.55723 -77.525824,37.641884") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5179560, "cityName": "Tuckahoe" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645575942246400, "text": "@Eminem My Rome.", "in_reply_to_status": -1, "in_reply_to_user": 22940219, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22940219 }}, "user": { "id": 3062942875, "name": "SarahGiles55 Giless", "screen_name": "GilessSarah", "lang": "en", "location": "Louisiana, USA", "create_at": date("2015-03-05"), "description": "Burgers hang's with Harry Styles, Taylor Swift, Xtinas Kermet,Steve-O Home Sweet SNL Ben Matt Little Big town hello kitty Stevie Wonderful Bruno Mars.", "followers_count": 494, "friends_count": 2132, "statues_count": 71345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakdale, LA", "id": "74df40186db95a5e", "name": "Oakdale", "place_type": "city", "bounding_box": rectangle("-92.681546,30.780141 -92.637161,30.849104") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22003, "countyName": "Allen", "cityID": 2256540, "cityName": "Oakdale" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645575967506432, "text": "maybe this year ill finally face the consequence of not taking my contacts out and go blind :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604908004, "name": "nessss", "screen_name": "wontontita", "lang": "en", "location": "Miami, Fl", "create_at": date("2012-06-10"), "description": "this is kind of a mental landfill", "followers_count": 220, "friends_count": 98, "statues_count": 4975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.654929 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645576189714437, "text": "@luddub new ones like whoa", "in_reply_to_status": 684638729714024448, "in_reply_to_user": 25587598, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25587598 }}, "user": { "id": 324537084, "name": "サボタージュビーツ", "screen_name": "sabotagebeats", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-06-26"), "description": "♒ – Aquarius. |\nMidi wizard of the 10th degree. \n#bitcoin & #technomusic |\nlive hardware since '09. #SocialMedia \n1LbpM2FYypTgZpouw9y8XLhWTLc8NCgsnw", "followers_count": 1016, "friends_count": 1619, "statues_count": 22784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645576261152768, "text": "#NYC ANIMAL CARE CENTER PLAN 2 KILL US 1/6\nHELP!\n#RESCUE\nHELPDOGS@URGENTPODR.ORG\nhttps://t.co/4BmtmSHqSk https://t.co/x80PZnSjvJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NYC", "RESCUE" }}, "user": { "id": 2342752717, "name": "Ann", "screen_name": "gaviota330", "lang": "en", "location": "VA.US", "create_at": date("2014-02-13"), "description": "animal lover all the way...", "followers_count": 2423, "friends_count": 2233, "statues_count": 41561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McLean, VA", "id": "dc234665a759a05d", "name": "McLean", "place_type": "city", "bounding_box": rectangle("-77.287868,38.893115 -77.119901,38.981856") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5148376, "cityName": "McLean" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645576798015488, "text": "couldn't of said it better.. https://t.co/6krCP3DX4l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2612570020, "name": "jas", "screen_name": "bonitaj_", "lang": "en", "location": "nyc | philly", "create_at": date("2014-06-15"), "description": "in my glo VSU", "followers_count": 805, "friends_count": 649, "statues_count": 7199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645576881750017, "text": "I hope to do more in the world then just whip it and Nae Nae when the song tells me to", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3292040691, "name": "Luis Tapia", "screen_name": "LoveTapSF", "lang": "en", "location": "Cali!!!", "create_at": date("2015-05-20"), "description": "I'm just an average boy trying to do above average things.", "followers_count": 67, "friends_count": 164, "statues_count": 235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645576906948608, "text": "My niggas really whole thang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3255813589, "name": "harry", "screen_name": "__sake98", "lang": "en", "location": "null", "create_at": date("2015-06-25"), "description": "null", "followers_count": 261, "friends_count": 224, "statues_count": 3352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645577137741824, "text": "her hips like eve, they hard to find.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414304133, "name": "darling nikki✨", "screen_name": "NoHablaFXCKSHXT", "lang": "en", "location": "finding myself.", "create_at": date("2011-11-16"), "description": "young and ambitious with big dreams ✨ she believed she could so she did. | your future nurse practitioner ✂ | #AUM19 | #RipTevin ❤", "followers_count": 1997, "friends_count": 998, "statues_count": 229190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phenix City, AL", "id": "66c034beedd48294", "name": "Phenix City", "place_type": "city", "bounding_box": rectangle("-85.06093,32.345401 -84.9918,32.531343") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1113, "countyName": "Russell", "cityID": 159472, "cityName": "Phenix City" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645577360064514, "text": "hi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2482780910, "name": "han", "screen_name": "_hannahgreene", "lang": "en", "location": "null", "create_at": date("2014-05-07"), "description": "null", "followers_count": 417, "friends_count": 416, "statues_count": 6308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warwick, RI", "id": "d1871b0189c27a21", "name": "Warwick", "place_type": "city", "bounding_box": rectangle("-71.518925,41.629501 -71.354415,41.76838") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44003, "countyName": "Kent", "cityID": 4474300, "cityName": "Warwick" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645577812873216, "text": "https://t.co/9VrakdN13o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2327656434, "name": "judy hopps", "screen_name": "agentpleakley", "lang": "en", "location": "far far away", "create_at": date("2014-02-04"), "description": "@luigilazuli 's side/more personal and fandomy account ariel/joy/luigi/lapis/leia (any kin names) 21/the diamonds MY GOD (she/they) also im a princess/mermaid", "followers_count": 221, "friends_count": 163, "statues_count": 16819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645577846583296, "text": "Wind 0.0 mph SW. Barometer 30.630 in, Falling. Temperature 11.6 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 5604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645578597244928, "text": "Leonardo DiCaprio needs an Oscar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 4132551374, "name": "Ian Dawkins", "screen_name": "ianndawkinns", "lang": "en", "location": "San Lorenzo, CA", "create_at": date("2015-11-04"), "description": "Proud twin of Draymond Green. #Gladiator", "followers_count": 190, "friends_count": 249, "statues_count": 820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Lorenzo, CA", "id": "3656cfbea78908af", "name": "San Lorenzo", "place_type": "city", "bounding_box": rectangle("-122.161568,37.661213 -122.106552,37.685884") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668112, "cityName": "San Lorenzo" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645579020873728, "text": "@_Albert11 happy birthday g! Love u bruda ��❤️����������������������❣", "in_reply_to_status": -1, "in_reply_to_user": 2684179711, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2684179711 }}, "user": { "id": 3230489984, "name": "michael", "screen_name": "hadidzuok", "lang": "es", "location": "199XBOYZ", "create_at": date("2015-05-30"), "description": "life is good", "followers_count": 716, "friends_count": 554, "statues_count": 18956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645579037605890, "text": "@AdrianGriggs1 @JonnyRobles", "in_reply_to_status": 683388078426161152, "in_reply_to_user": 458803832, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 458803832, 434563816 }}, "user": { "id": 458803832, "name": "Uncle Scott", "screen_name": "AdrianGriggs1", "lang": "en", "location": "Cleburne, TX / Dallas, TX", "create_at": date("2012-01-08"), "description": "(Just pour and Drink) Dirty burne ➡️ Dallas Texas / Attending Brookhaven", "followers_count": 627, "friends_count": 229, "statues_count": 4461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleburne, TX", "id": "70b538807e617654", "name": "Cleburne", "place_type": "city", "bounding_box": rectangle("-97.439627,32.298971 -97.349294,32.437917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48251, "countyName": "Johnson", "cityID": 4815364, "cityName": "Cleburne" } }
+{ "create_at": datetime("2016-01-06T00:00:00.000Z"), "id": 684645579742380033, "text": "of course my dad pulls into the gas station which there are cops outside. i reek of hennesy and the devils lettuce", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 563439248, "name": "B", "screen_name": "brandons_twitta", "lang": "en", "location": "Native American. Mp, Michigan", "create_at": date("2012-04-25"), "description": "But who as the god approved to the blind that they can't see as far as the sighted? Snapchat: brandonishammm", "followers_count": 258, "friends_count": 117, "statues_count": 2847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, MI", "id": "77f923f5a841bbdf", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-84.816945,43.553246 -84.727407,43.63446") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26073, "countyName": "Isabella", "cityID": 2656020, "cityName": "Mount Pleasant" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580145057793, "text": "Being rich https://t.co/FbZ9oWB3in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228447917, "name": "️", "screen_name": "_dboogiee", "lang": "en", "location": "umes", "create_at": date("2010-12-19"), "description": "she let that nigga make it rain but guess who got it wetter?", "followers_count": 1248, "friends_count": 734, "statues_count": 53469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryans Road, MD", "id": "7ce95f88bf8ad974", "name": "Bryans Road", "place_type": "city", "bounding_box": rectangle("-77.131221,38.604043 -77.055641,38.656903") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2410925, "cityName": "Bryans Road" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580245745664, "text": "Wind 0 mph --. Barometer 30.34 in, Steady. Temperature 35.6 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 19995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580631482368, "text": "@Ammoolaax yeah, views from the 6. Suppose to come out today ��", "in_reply_to_status": 684645466277953537, "in_reply_to_user": 291377354, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 291377354 }}, "user": { "id": 3013660440, "name": "AK", "screen_name": "a7medkir", "lang": "en", "location": "Davis, CA", "create_at": date("2015-02-08"), "description": "null", "followers_count": 161, "friends_count": 160, "statues_count": 4213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davis, CA", "id": "1994142e26ba7127", "name": "Davis", "place_type": "city", "bounding_box": rectangle("-121.803252,38.526843 -121.675074,38.590264") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 618100, "cityName": "Davis" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580652457985, "text": "It's my birthday bitches ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234880339, "name": "January 6", "screen_name": "Yagabbz", "lang": "en", "location": "null", "create_at": date("2015-06-03"), "description": "☁️sin city poppin☁️", "followers_count": 500, "friends_count": 376, "statues_count": 7224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580820226049, "text": "@SynisterSilva who are you lol Joanna went today huh", "in_reply_to_status": 684645465074212864, "in_reply_to_user": 1118242669, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1118242669 }}, "user": { "id": 110238129, "name": "Марисоль", "screen_name": "Marisol_SayWhat", "lang": "en", "location": "null", "create_at": date("2010-01-31"), "description": "null", "followers_count": 396, "friends_count": 573, "statues_count": 50856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645580971184128, "text": "Soul Plane ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281151358, "name": "Canelo", "screen_name": "JoshMartinez25", "lang": "en", "location": "Los Angeles", "create_at": date("2011-04-12"), "description": "I'm going somewhere nice where no mosquitoes at. Sneaker enthusiast", "followers_count": 415, "friends_count": 584, "statues_count": 24040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645581034098688, "text": "Lmao brae would", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1479273931, "name": "-", "screen_name": "livingproof98", "lang": "en", "location": "null", "create_at": date("2013-06-03"), "description": "I was born sinning, but I'll die better than that | Success Has No Limits|c/o 2016 #diplomaready", "followers_count": 686, "friends_count": 613, "statues_count": 17201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645581466238976, "text": "Going ghost for s few days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307982421, "name": "B", "screen_name": "b_contino", "lang": "en", "location": "null", "create_at": date("2014-01-27"), "description": "Papi Shampoo", "followers_count": 169, "friends_count": 127, "statues_count": 3243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645581470339072, "text": "God all I want in 2016 is for you to give me a chance with a girl that's all I can ask for..Amen ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73775974, "name": "Frank Lucas", "screen_name": "Hardb0dy_Mf", "lang": "en", "location": "Ruston,Louisiana", "create_at": date("2009-09-12"), "description": "Don't fear me for who I am!! Fear me for what Ima capable", "followers_count": 1315, "friends_count": 1276, "statues_count": 26796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645581508091904, "text": "My life https://t.co/psqdBEOyg3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1673208139, "name": "M C X G ✨", "screen_name": "yvngmyles", "lang": "en", "location": "Oceanside, CA ⛅️", "create_at": date("2013-08-15"), "description": "I'm really tall .... 7'0 || Palomar Basketball • 19 Bisexual & Proud || sc : yvngmyles #LiveItUp", "followers_count": 1796, "friends_count": 693, "statues_count": 58788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645581814411264, "text": "Shaka: i love lightskin wimmins\nMe: I hate em\nShaka: but deanna lightskin \nMe: my statement stands\nShaka: https://t.co/KkgEoDzYXf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2477806716, "name": "woo mane la flare", "screen_name": "woo2xs", "lang": "en", "location": "Orlando, FL", "create_at": date("2014-05-04"), "description": "null", "followers_count": 221, "friends_count": 193, "statues_count": 10990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504957,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645582552449024, "text": "next time I stand tall I wanna be standing for you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977690257, "name": "Mendoza™", "screen_name": "alfredo0723", "lang": "en", "location": "sylmar ", "create_at": date("2012-11-28"), "description": "shs | sc; alfredo0723 ig; alfredo_mendoza", "followers_count": 857, "friends_count": 812, "statues_count": 16309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645582560866304, "text": "@Yosoygobb https://t.co/heFSzaLTvH", "in_reply_to_status": 684645296391892992, "in_reply_to_user": 3087915316, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3087915316 }}, "user": { "id": 1832897132, "name": "✌️JVelasquez⚗", "screen_name": "Jbvelasquez4949", "lang": "en", "location": "Nature wei", "create_at": date("2013-09-08"), "description": "Your friendly neighborhood Spider-Man| @kawaiith0t❤️", "followers_count": 462, "friends_count": 1019, "statues_count": 34631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645583521370113, "text": "@jenbeatz Jamz appreciates the support fam", "in_reply_to_status": -1, "in_reply_to_user": 4064034432, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4064034432 }}, "user": { "id": 260794653, "name": "Dolla P", "screen_name": "SPbeavs", "lang": "en", "location": "Outside", "create_at": date("2011-03-04"), "description": "Go Beavs is a lifestyle. forward progress only. #Regime #gobeavs", "followers_count": 507, "friends_count": 971, "statues_count": 10512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645583546626048, "text": "Wind 1.0 mph ESE. Barometer 30.360 in, Falling. Temperature 30.5 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645583567613953, "text": "I wish I didn't have to get up so early", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155318722, "name": "Chloe Mayfield-Brown", "screen_name": "CloMayB", "lang": "en", "location": "Dayton, OH", "create_at": date("2010-06-13"), "description": "Christian. 21. Sinclair WBB.", "followers_count": 761, "friends_count": 668, "statues_count": 31722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, OH", "id": "8d742fb555fbff21", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-84.307688,39.695193 -84.093044,39.865523") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3921000, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645583596990464, "text": "3h 20m wait time at University of Alberta Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.520907,53.520659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 371, "friends_count": 887, "statues_count": 2528 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-06T00:00:01.000Z"), "id": 684645583928332288, "text": "Wind 0.0 mph ---. Barometer 30.480 in, Falling slowly. Temperature 20.7 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645584314089472, "text": "@_MarcTheGOAT lol ok you can stay", "in_reply_to_status": 684644748573945856, "in_reply_to_user": 987910248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 987910248 }}, "user": { "id": 197223039, "name": "Marx.", "screen_name": "MannieeGeee", "lang": "en", "location": "Assachusetts. ", "create_at": date("2010-09-30"), "description": "*sucks teeth*", "followers_count": 4485, "friends_count": 1763, "statues_count": 186481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lowell, MA", "id": "d6539f049c4d05e8", "name": "Lowell", "place_type": "city", "bounding_box": rectangle("-71.382444,42.605989 -71.271272,42.666507") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2537000, "cityName": "Lowell" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645584733605888, "text": "Cause I pistol whopped that nigga for 40mins straight....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30588729, "name": "Thunder Up⚡️", "screen_name": "juicy115", "lang": "en", "location": "ʜᴏᴜᴍᴀ,ʟᴀ ⚓️", "create_at": date("2009-04-11"), "description": "мdн lιғə ™ • yoυr мoмѕ a нoə ιғ yoυ υnғollow мə • jυιcəмan prodυcтιonѕ〽️ • jυιcy j 〽️", "followers_count": 3021, "friends_count": 2096, "statues_count": 159706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schriever, LA", "id": "65342fd1ba9ef172", "name": "Schriever", "place_type": "city", "bounding_box": rectangle("-90.846026,29.709461 -90.796187,29.778066") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2268300, "cityName": "Schriever" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645584876081152, "text": "I just wanna talk to a cute boy 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2990513402, "name": "b", "screen_name": "somthing_dumb", "lang": "en", "location": "null", "create_at": date("2015-01-19"), "description": "null", "followers_count": 102, "friends_count": 91, "statues_count": 1895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prescott, AZ", "id": "193e889ce36e6e29", "name": "Prescott", "place_type": "city", "bounding_box": rectangle("-112.537378,34.512365 -112.373682,34.626579") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4025, "countyName": "Yavapai", "cityID": 457380, "cityName": "Prescott" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645584880308224, "text": "#tweetavideothatyoucantexplain https://t.co/9k72D5ly0S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "tweetavideothatyoucantexplain" }}, "user": { "id": 2255619198, "name": "Emily Blanchfield♡", "screen_name": "EmilyyBlanch", "lang": "en", "location": "24/7 party", "create_at": date("2013-12-20"), "description": "who are you?", "followers_count": 472, "friends_count": 339, "statues_count": 4291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaanapali, HI", "id": "386eee0c027ac681", "name": "Kaanapali", "place_type": "city", "bounding_box": rectangle("-156.696704,20.911087 -156.674143,20.950546") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1520000, "cityName": "Kaanapali" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645585186598913, "text": "BUT ALSO https://t.co/DAoABFnOus", "in_reply_to_status": 684645483642515456, "in_reply_to_user": 26225897, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26225897, "name": "✨Rembrandt Duran✨", "screen_name": "Remdelarem", "lang": "en", "location": "New York City", "create_at": date("2009-03-24"), "description": "✨Pretty, Queer, Mexican Dutch Jew born & raised in NYC! Professional side piece, your favorite designer/artist!✨ADEEN! ✨Instagram: @Remdelarem✨", "followers_count": 6049, "friends_count": 888, "statues_count": 83807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645585371021313, "text": "@Jodemoted hush.", "in_reply_to_status": 684645378780565504, "in_reply_to_user": 426084578, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 426084578 }}, "user": { "id": 89602870, "name": "Roxanna Castañeda", "screen_name": "roxannaneda", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2009-11-12"), "description": "#LatinaOrgullosa", "followers_count": 369, "friends_count": 352, "statues_count": 11429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645585417207808, "text": "temperature down 19°F -> 12°F\nhumidity up 56% -> 75%\npressure 30.63in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.27004,39.18218"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 147046666, "name": "Germantown Weather", "screen_name": "_GermantownMD", "lang": "en", "location": "Germantown, MD", "create_at": date("2010-05-22"), "description": "Weather updates, forecast, warnings and information for Germantown, MD. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 130, "friends_count": 3, "statues_count": 24874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645585740169218, "text": "Temp: 35.1°F Wind:0.0mph Pressure: 30.362hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 56001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-06T00:00:02.000Z"), "id": 684645586507829248, "text": "#SupportOriginMelissa 22.1°F Wind:2.5mph Pressure: 30.41hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588328030209, "text": "❤️❤️ https://t.co/znh0kJDugQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1242916789, "name": "Paypal", "screen_name": "PaytonPaz_22", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "She gon shake for dem blue hunnids", "followers_count": 782, "friends_count": 748, "statues_count": 22220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dos Palos, CA", "id": "72a418277ebdea24", "name": "Dos Palos", "place_type": "city", "bounding_box": rectangle("-120.653752,36.969015 -120.60914,36.996315") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 619612, "cityName": "Dos Palos" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588441300993, "text": "Happy birthday to my #1 always love you ���������������� https://t.co/5vyXDTum3J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1356127148, "name": "Selena Martinez", "screen_name": "selena13m", "lang": "en", "location": "null", "create_at": date("2013-04-15"), "description": "I do whatever helps me sleep", "followers_count": 319, "friends_count": 314, "statues_count": 9976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588458078209, "text": "@jayanthcl rofl!! ����", "in_reply_to_status": 684634560470978560, "in_reply_to_user": 94076814, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 94076814 }}, "user": { "id": 86698343, "name": "Akshatha", "screen_name": "AkshathaPrabhuM", "lang": "en", "location": "Bangalore", "create_at": date("2009-11-01"), "description": "Working as an IT Engineer, big time foodie, love to try out new places, used to read a lot now can hardly find time, clash of clans fan :D", "followers_count": 256, "friends_count": 282, "statues_count": 869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Oswego, OR", "id": "386b4dc0b52f8b48", "name": "Lake Oswego", "place_type": "city", "bounding_box": rectangle("-122.750195,45.382216 -122.642587,45.44028") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4140550, "cityName": "Lake Oswego" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588609163264, "text": "Wind 0.0 mph W. Barometer 30.58 in, Falling slowly. Temperature 21.6 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 89, "statues_count": 156456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588651130880, "text": "I apparently cannot get enough of this place. But I can't reccomend broken ribs @ Sutter Santa… https://t.co/ZnTRH3Magh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.7513809,38.496254"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 7138732, "name": "Big Poppa", "screen_name": "saynine", "lang": "en", "location": "Santa Rosa, CA", "create_at": date("2007-06-28"), "description": "Destroyer of false idols, a bringer of chaos. 302.84. Libertine, Sadist, Dom, Kinky , Polyamorist, Queer, Fuckspert. I'm BOMBASTIC. TRIGGER WARNING!", "followers_count": 2694, "friends_count": 869, "statues_count": 132539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Larkfield, CA", "id": "01a743d720b87387", "name": "Larkfield", "place_type": "city", "bounding_box": rectangle("-122.787492,38.488502 -122.718349,38.527916") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588667781120, "text": "I'm just now starting my ap lit hw��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43232181, "name": "Louie", "screen_name": "brandonlouie517", "lang": "en", "location": "| Bay Area |", "create_at": date("2009-05-28"), "description": "| Instagram/snapchat: brandonlouie |", "followers_count": 250, "friends_count": 211, "statues_count": 5104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588709724160, "text": "Moisture plume stretches over 700 miles & that means Bay Area gets drenched in AM! Strong winds will make it worse. https://t.co/u2NcpqjBcU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.41487603,37.78678264"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60452453, "name": "San Francisco Press", "screen_name": "SanFranciscoCP", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-07-26"), "description": "San Francisco's source for local news", "followers_count": 3317, "friends_count": 12, "statues_count": 229670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588936216576, "text": "37.6F (Feels: 37.6F) - Humidity: 95% - Wind: 0.0mph --- - Gust: 0.7mph - Pressure: 1037.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 221225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645588995026944, "text": "Mike like them straight ratchet females ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1240589186, "name": "babygirl", "screen_name": "zhanedanielle", "lang": "en", "location": "null", "create_at": date("2013-03-03"), "description": "Life is short. Stunt", "followers_count": 471, "friends_count": 371, "statues_count": 22965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Orange, NJ", "id": "858cec499f8adba6", "name": "East Orange", "place_type": "city", "bounding_box": rectangle("-74.238236,40.745103 -74.190042,40.788099") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3419390, "cityName": "East Orange" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645589578039296, "text": "Wind 0.0 mph WNW. Barometer 30.283 in, Falling. Temperature 31.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 56002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645589582266369, "text": "@Drake ��", "in_reply_to_status": -1, "in_reply_to_user": 27195114, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 27195114 }}, "user": { "id": 246581433, "name": "DREW", "screen_name": "1DRIGGA", "lang": "en", "location": "OVO/Rocnation ", "create_at": date("2011-02-02"), "description": "Wide eyed and Uneducated at 19", "followers_count": 1070, "friends_count": 871, "statues_count": 122673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645589607399424, "text": "@KingHarrington6 @urboyzach damn ok", "in_reply_to_status": 684644779989180416, "in_reply_to_user": 41531939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41531939, 2842602618 }}, "user": { "id": 92056321, "name": "John Martinez", "screen_name": "Long_John_93", "lang": "en", "location": "Houston|San Mo, TX 713 - 512", "create_at": date("2009-11-23"), "description": "I live in my own world but it’s okay. They know me there. Texas State|God is love", "followers_count": 620, "friends_count": 134, "statues_count": 37384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deer Park, TX", "id": "cc480e2388809012", "name": "Deer Park", "place_type": "city", "bounding_box": rectangle("-95.14691,29.663671 -95.087119,29.751499") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4819624, "cityName": "Deer Park" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645589896851456, "text": "Me all day ���� https://t.co/0EaqOREscy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3907047088, "name": "brinaaaaaa.", "screen_name": "QvveenBri", "lang": "en", "location": "bella noche", "create_at": date("2015-10-09"), "description": "#MGSU19// princess. /", "followers_count": 138, "friends_count": 161, "statues_count": 1107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Snellville, GA", "id": "b2cdaa7db78ed09b", "name": "Snellville", "place_type": "city", "bounding_box": rectangle("-84.042837,33.818392 -83.950981,33.890635") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1371604, "cityName": "Snellville" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590374862848, "text": "Happy birthday butt face! Hope you have a great day!!!!! �������� @brizzz16 https://t.co/25cQS9IuHy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2646326011 }}, "user": { "id": 2823233887, "name": "Demetri Paler", "screen_name": "Chunk__dirtyy", "lang": "en", "location": "null", "create_at": date("2014-09-20"), "description": "T.m. ☝ http://www.hudl.com/athlete/4909473/highlights/301059410 ||| http://www.hudl.com/athlete/4909473/highlights/312614376", "followers_count": 572, "friends_count": 457, "statues_count": 8484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590399975425, "text": "You getting mad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302762516, "name": "Young AP3 (Feb 5th)", "screen_name": "KingSeun24", "lang": "en", "location": "Denton, TX", "create_at": date("2011-05-21"), "description": "Future CMO | Your PHavorite Alpha Man |UNT Live | #UNT | Nigerian", "followers_count": 4103, "friends_count": 2039, "statues_count": 207579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590404325376, "text": "@WeAre90sKids @SpaceCatPics drake and josh was my favorite 90s show https://t.co/3YwcRSd8uy", "in_reply_to_status": 682257114979512321, "in_reply_to_user": 758543612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 758543612, 106170894 }}, "user": { "id": 1370823061, "name": "actual dandog", "screen_name": "DrMiable", "lang": "en", "location": "Inver Grove Heights, MN", "create_at": date("2013-04-21"), "description": "Twitch streamer from the Minneapolis-St Paul area. Mod on @9TailsK's stream. (was @DandogTweets) Dr Miable = Dr Mario.", "followers_count": 824, "friends_count": 2331, "statues_count": 7316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inver Grove Heights, MN", "id": "80da8df0b0759b52", "name": "Inver Grove Heights", "place_type": "city", "bounding_box": rectangle("-93.106464,44.775656 -93.010797,44.884741") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2731076, "cityName": "Inver Grove Heights" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590613934080, "text": ".@Dos_A_Rita yasssssss https://t.co/VwUt4jEh1h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 29479894 }}, "user": { "id": 39893961, "name": "Tyler", "screen_name": "ttyyler", "lang": "en", "location": "San Antonio ", "create_at": date("2009-05-13"), "description": "Don't call it a fight if it's a war.", "followers_count": 386, "friends_count": 383, "statues_count": 15296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Hills, TX", "id": "1824dfaf5ecc1f55", "name": "Castle Hills", "place_type": "city", "bounding_box": rectangle("-98.537126,29.502814 -98.504978,29.541364") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4813276, "cityName": "Castle Hills" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590718787584, "text": "@unicorns4lifeyo @CeeLow18 lmao", "in_reply_to_status": 684645174195036160, "in_reply_to_user": 1730732924, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 1730732924, 4021135993 }}, "user": { "id": 162914559, "name": "Prince", "screen_name": "PrinceGeeko", "lang": "en", "location": "Denver, CO ", "create_at": date("2010-07-04"), "description": "#TeamGod/#Gemini♊/#TrackNation/#BAMA/#KUNation//#Senior/#Overland/", "followers_count": 1722, "friends_count": 789, "statues_count": 65024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590748102658, "text": "We see you, Zendaya DubNation https://t.co/u2NcpqjBcU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.41487603,37.78678264"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60452453, "name": "San Francisco Press", "screen_name": "SanFranciscoCP", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-07-26"), "description": "San Francisco's source for local news", "followers_count": 3317, "friends_count": 12, "statues_count": 229671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590840418304, "text": "SO TRUE LMFAO ������ https://t.co/ON2f6QYi1m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2902911544, "name": "Kayla Caston", "screen_name": "DrEbaela", "lang": "en", "location": "ur mom's house", "create_at": date("2014-12-02"), "description": "professional chicken tender for hire $$$", "followers_count": 92, "friends_count": 232, "statues_count": 2733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, CA", "id": "3a16272f05577d7e", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-117.72136,33.606781 -117.621862,33.685914") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639496, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590857158656, "text": "fuck immigration", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424513630, "name": "rodeo", "screen_name": "LeslieMaricel", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "SF", "followers_count": 808, "friends_count": 615, "statues_count": 100119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Mateo, CA", "id": "432daa3153c5fef9", "name": "San Mateo", "place_type": "city", "bounding_box": rectangle("-122.355038,37.512648 -122.275805,37.592395") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 668252, "cityName": "San Mateo" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645590857297920, "text": "Travelling to Boulder or just twittering about Boulder? https://t.co/x2ZeYsN2mE #Boulder", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.271,40.015"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Boulder" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 976, "friends_count": 312, "statues_count": 2514447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645591150792705, "text": "in my feelings again :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2196960906, "name": "Simba", "screen_name": "dacolindres", "lang": "en", "location": "dreamville", "create_at": date("2013-11-15"), "description": "It was never love I could tell", "followers_count": 99, "friends_count": 126, "statues_count": 3452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645592346185728, "text": "2k Time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322595844, "name": "Young Simba", "screen_name": "MyRetrosBoomin", "lang": "en", "location": "Prairie View, TX", "create_at": date("2011-06-23"), "description": "Thank God & Stay Humble #Pvamu19", "followers_count": 1949, "friends_count": 642, "statues_count": 11753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-06T00:00:03.000Z"), "id": 684645592421646336, "text": "@Drake DROP IT NOW BABY NOWWW ��", "in_reply_to_status": -1, "in_reply_to_user": 27195114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27195114 }}, "user": { "id": 2915045228, "name": "jrizzy", "screen_name": "xojayjay5", "lang": "en", "location": "null", "create_at": date("2014-11-30"), "description": "I love Drake & play basketball :) #30", "followers_count": 1015, "friends_count": 804, "statues_count": 22500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645592786534400, "text": "It's so hot ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3007441311, "name": "grinch", "screen_name": "the_robot_panda", "lang": "en", "location": "seattle", "create_at": date("2015-01-31"), "description": "do it for her", "followers_count": 196, "friends_count": 181, "statues_count": 12170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645592803328004, "text": "If you were to work for a cartel who's would it be... #ESDLC #SraAcero ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ESDLC", "SraAcero" }}, "user": { "id": 162851351, "name": "David Ojeda.", "screen_name": "DdOgLAm", "lang": "en", "location": "In A State of Trance", "create_at": date("2010-07-04"), "description": "My Beauty Can Overcome Brains and Brawn. @DashBerlin|@ArminVanBuuren|@ATBandre Snapchat : ddo.glam", "followers_count": 521, "friends_count": 408, "statues_count": 11138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645592891437056, "text": "@evanspiegel @Snapchat you inspired me to make a #video about the NOW.\nhttps://t.co/thOfUK5Lcg #youtube #moments https://t.co/HLspGthGOo", "in_reply_to_status": -1, "in_reply_to_user": 230287527, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "video", "youtube", "moments" }}, "user_mentions": {{ 230287527, 376502929 }}, "user": { "id": 70559497, "name": "THEWORLDOFMARIO", "screen_name": "mgarzan", "lang": "en", "location": "MEX - MIA - NY - UK - SF - PDX", "create_at": date("2009-08-31"), "description": "I'm a storyteller who eats a lot of quesadillas Insta-Snap-Vine-Youtube: THEWORLDOFMARIO\nBeme: mayin", "followers_count": 2748, "friends_count": 3113, "statues_count": 5143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593080152066, "text": "How does my brother sleep with a fan on when it's freezing?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24622528, "name": "Ehsan", "screen_name": "EhsanS_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-15"), "description": "Undergraduate | Middle-earth | Cricket is life | IG: ehsansubhani", "followers_count": 266, "friends_count": 215, "statues_count": 418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593382137856, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1508367955, "name": "nicholass", "screen_name": "Nikitynick__", "lang": "en", "location": "Hacienda Heights, CA", "create_at": date("2013-06-11"), "description": "rhcc | 18 years old", "followers_count": 482, "friends_count": 358, "statues_count": 13501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593579433984, "text": "���� https://t.co/NrxP89Mm0T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 885147763, "name": "Ev Dad Tiller ⚡️️", "screen_name": "evson__", "lang": "en", "location": "null", "create_at": date("2012-10-16"), "description": "what a time..", "followers_count": 443, "friends_count": 339, "statues_count": 14168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stratford, CT", "id": "9133850ec8827e0a", "name": "Stratford", "place_type": "city", "bounding_box": rectangle("-73.164245,41.147132 -73.088173,41.268524") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 974260, "cityName": "Stratford" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593696735232, "text": "#Ma Boulange Bakery https://t.co/ksU3BiJDco Excellent https://t.co/OtP9VnY9w7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.31079652,47.31677852"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Ma" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4265, "friends_count": 875, "statues_count": 372123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257126 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593717735424, "text": "@mboulangerie #Ma Boulange Bakery / https://t.co/ksU3BiJDco Excellent https://t.co/Zlig57HpGk", "in_reply_to_status": -1, "in_reply_to_user": 2936032843, "favorite_count": 0, "coordinate": point("-122.31079652,47.31677852"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Ma" }}, "user_mentions": {{ 2936032843 }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4265, "friends_count": 875, "statues_count": 372123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257126 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645593885478912, "text": "That voice makes me know things I never knew before! ���� https://t.co/XBaPk9IHJZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 575981524, "name": "DroJackson.", "screen_name": "JayD_Maldonado7", "lang": "en", "location": "Parlier, California", "create_at": date("2012-05-09"), "description": "My riches is life. RIP Coach Artie", "followers_count": 486, "friends_count": 681, "statues_count": 20127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parlier, CA", "id": "b3c579dbd6e6f98b", "name": "Parlier", "place_type": "city", "bounding_box": rectangle("-119.562738,36.603513 -119.51629,36.621062") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 655856, "cityName": "Parlier" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645594254581761, "text": "@dimpleee_ @MalcolmTooXtra LIIITTTTT", "in_reply_to_status": 684644703854174209, "in_reply_to_user": 180302550, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 180302550, 593162543 }}, "user": { "id": 226659570, "name": "Han Solo", "screen_name": "ifihada_NICK_el", "lang": "en", "location": "null", "create_at": date("2010-12-14"), "description": "I'd EpaR Rihanna", "followers_count": 1803, "friends_count": 495, "statues_count": 137961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mojave, CA", "id": "00b5133757bbf988", "name": "Mojave", "place_type": "city", "bounding_box": rectangle("-118.198458,35.024858 -118.14495,35.070099") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 648452, "cityName": "Mojave" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645594598514688, "text": "@richmondstavern Richmond's Tavern #Rochester https://t.co/2I5AqNKNCX Excellent https://t.co/JiyrdwEwvV", "in_reply_to_status": -1, "in_reply_to_user": 453593690, "favorite_count": 0, "coordinate": point("-77.59782151,43.15882989"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rochester" }}, "user_mentions": {{ 453593690 }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3648, "friends_count": 507, "statues_count": 162363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608257, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645594720276480, "text": "���� what did you see �� https://t.co/5wQLXFx7Fv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391746946, "name": "Jan20th ❤️", "screen_name": "greazynay", "lang": "en", "location": "Jersey City x Soundview", "create_at": date("2011-10-15"), "description": "belong to Aiden ✨", "followers_count": 1259, "friends_count": 1246, "statues_count": 51921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645594757894145, "text": "Ani", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 630924640, "name": "blessing :)", "screen_name": "blessingmtz_", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "18 • psalm 91:4 • Seahawks • MKG - I.I.MMXVI", "followers_count": 1198, "friends_count": 564, "statues_count": 42318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Grande City, TX", "id": "006ae2da1247a791", "name": "Rio Grande City", "place_type": "city", "bounding_box": rectangle("-98.868668,26.325267 -98.74347,26.405053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4862168, "cityName": "Rio Grande City" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595047268352, "text": "Having my uncle at the hospital is the worst, just scaring me before my surgery, @ShumonWood WHERE TF R U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26708514 }}, "user": { "id": 615239584, "name": "London", "screen_name": "ameliacarter__", "lang": "en", "location": "Mill Creek, WA", "create_at": date("2012-06-22"), "description": "18 | English & Egyptian | London/Seattle", "followers_count": 967, "friends_count": 380, "statues_count": 6451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martha Lake, WA", "id": "6c1eae5334778aa8", "name": "Martha Lake", "place_type": "city", "bounding_box": rectangle("-122.261992,47.824173 -122.219197,47.881993") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343815, "cityName": "Martha Lake" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595173261312, "text": "Laying in bed listening to all 3 @evanescence CDs. #awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "awake" }}, "user_mentions": {{ 55010030 }}, "user": { "id": 345548735, "name": "Chris Bright", "screen_name": "C_Bright1988", "lang": "en", "location": "South Bend,Indiana", "create_at": date("2011-07-30"), "description": "Loyal friend, college grad, political science nerd, LGBT activist, and family man. Uncensored thoughts beware.", "followers_count": 511, "friends_count": 2063, "statues_count": 17098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595307360256, "text": "Nutella-palooza is a totally nuts festival dedicated to the tasty spread https://t.co/EYq3v1bp5O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17695, "friends_count": 17538, "statues_count": 66664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595357683712, "text": "Go ahead and take all the words I use, they won't make you me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1358277499, "name": "Dylan J. Giejda", "screen_name": "D_Giejd", "lang": "en", "location": "Chicago • Tuscaloosa", "create_at": date("2013-04-16"), "description": "Blasé • ΛΧΑ", "followers_count": 558, "friends_count": 509, "statues_count": 8799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crystal Lake, IL", "id": "c648b4de515a26e9", "name": "Crystal Lake", "place_type": "city", "bounding_box": rectangle("-88.37422,42.190093 -88.257285,42.306172") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1717887, "cityName": "Crystal Lake" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595441541120, "text": "my mkx anti airs are booboo >.<", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1705676858, "name": "Miah", "screen_name": "Princess_Slim96", "lang": "en", "location": "Chesapeake, VA", "create_at": date("2013-08-27"), "description": "Hey its Princess Slim. Just trying to play games and what not. http://ask.fm/Princess_Slim96 Psn: Slim_chance96. Don't let me sleep on you ;)", "followers_count": 157, "friends_count": 149, "statues_count": 3911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake, VA", "id": "73d1c1c11b675932", "name": "Chesapeake", "place_type": "city", "bounding_box": rectangle("-76.448383,36.66 -76.143408,36.866431") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51550, "countyName": "Chesapeake", "cityID": 5116000, "cityName": "Chesapeake" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595726876672, "text": "03:00:01 |Temp: 28.1ºF | Wind Chill 28.1ºF |Dew Point 19.9ºF | Rain today: 0.00 inches | Wind: 1.0 mph from the NE, Gusting to 2.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645595819061248, "text": "2am back in Houston and all I want is some whataburger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2324620454, "name": "Sophie", "screen_name": "sophieh_310", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "philippians 4:13", "followers_count": 205, "friends_count": 385, "statues_count": 1284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645596058226688, "text": "I just got called a \"Twitter honey\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38507416, "name": "Singing Hairstylist", "screen_name": "LeahRenee_", "lang": "en", "location": "Singing to the Clouds☁", "create_at": date("2009-05-07"), "description": "That Big Pretty Girl That Do Hair", "followers_count": 2496, "friends_count": 1566, "statues_count": 96511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-01-06T00:00:04.000Z"), "id": 684645596385390592, "text": "Assistant to the assistant regional manager ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2396943691, "name": "Jocelyn Reinhart", "screen_name": "jreinhart97", "lang": "en", "location": "Ada, OH", "create_at": date("2014-03-18"), "description": "ONU 19 xc/track. Take a second to think about how blessed you are❤️", "followers_count": 444, "friends_count": 538, "statues_count": 6503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fostoria, OH", "id": "377b44a4ff931d73", "name": "Fostoria", "place_type": "city", "bounding_box": rectangle("-83.460747,41.128097 -83.367515,41.20308") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39147, "countyName": "Seneca", "cityID": 3928014, "cityName": "Fostoria" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645597182177280, "text": "I fwu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 383057318, "name": "#thedream", "screen_name": "Greg_IDominguez", "lang": "en", "location": "Fairfax Volleyball", "create_at": date("2011-09-30"), "description": "for coach brown", "followers_count": 298, "friends_count": 362, "statues_count": 18097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645597450600448, "text": "\"I'm on my period\" https://t.co/qk7zAMZSUE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35177564, "name": "Latino Man", "screen_name": "DaLilHomieMark", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-25"), "description": "Westside rollin, wherever you are", "followers_count": 587, "friends_count": 479, "statues_count": 30731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645597853319168, "text": "Happy 18th birthday you amazing woman!!!!������❤️�� @__AntoinetteBoo https://t.co/y983YPHryU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 484519203 }}, "user": { "id": 211339912, "name": "allycat", "screen_name": "Allyhondro1", "lang": "en", "location": "null", "create_at": date("2010-11-02"), "description": "Junior at Bay • V. Volleyball • 2k17 • #8", "followers_count": 192, "friends_count": 161, "statues_count": 951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Grove, WA", "id": "6ead1643975047c8", "name": "Walnut Grove", "place_type": "city", "bounding_box": rectangle("-122.636582,45.659036 -122.575637,45.699458") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5376055, "cityName": "Walnut Grove" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645598247522305, "text": "@JodyBreeze45 from a broken heart", "in_reply_to_status": 684645434308964352, "in_reply_to_user": 365850835, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 365850835 }}, "user": { "id": 4086788474, "name": "ivan", "screen_name": "ivanxreyes", "lang": "en", "location": "null", "create_at": date("2015-10-31"), "description": "null", "followers_count": 19, "friends_count": 19, "statues_count": 623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645598339952641, "text": "can't wait https://t.co/rTwTZeAMnb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48818644, "name": "uncle caug", "screen_name": "incaugnito37", "lang": "en", "location": "South Bend, IN | Berkeley, CA", "create_at": date("2009-06-19"), "description": "A candle loses nothing by lighting the flame of another. \nAspiring (Astro)Physicist", "followers_count": 479, "friends_count": 443, "statues_count": 25429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645598390140928, "text": "I pray for the day when all my worries will disappear while I drink Minute Maid in my big ass mansion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 294992185, "name": "TheyLOVEPHEA", "screen_name": "Devaughn_Xx", "lang": "en", "location": "null", "create_at": date("2011-05-07"), "description": "#TxSU19 #BlackLivesMatter #ElitePromotions the rose that grew through concrete", "followers_count": 826, "friends_count": 673, "statues_count": 57268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645598578880513, "text": "Bye bitch ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1292231142, "name": "Kendyl", "screen_name": "Kendyl_17", "lang": "en", "location": "Eagle, ID", "create_at": date("2013-03-23"), "description": "I'll believe it when I see it", "followers_count": 417, "friends_count": 270, "statues_count": 11086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, ID", "id": "afc5c9ce888b958d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-116.461472,43.561187 -116.337922,43.662958") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1652120, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645598969106432, "text": "@Cynthia_mua s", "in_reply_to_status": -1, "in_reply_to_user": 3042753288, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3042753288 }}, "user": { "id": 2907980965, "name": "Daniel Garcia", "screen_name": "DanielDgar", "lang": "en", "location": "null", "create_at": date("2014-11-23"), "description": "~You determine what you do with the gift of life~ #skateanddestroy", "followers_count": 12, "friends_count": 71, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennesaw, GA", "id": "c4cd1df048841e00", "name": "Kennesaw", "place_type": "city", "bounding_box": rectangle("-84.64724,33.955258 -84.577544,34.066895") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1343192, "cityName": "Kennesaw" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645599262715904, "text": "@SmaccHeeem lmfaoo ����", "in_reply_to_status": 684645468769509377, "in_reply_to_user": 2934358924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2934358924 }}, "user": { "id": 379456521, "name": "Gerard", "screen_name": "roddygeez", "lang": "en", "location": "null", "create_at": date("2011-09-24"), "description": "#Wolfpack", "followers_count": 760, "friends_count": 834, "statues_count": 34799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford Mill, MD", "id": "dc811f15be302120", "name": "Milford Mill", "place_type": "city", "bounding_box": rectangle("-76.795559,39.320502 -76.744784,39.369482") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2452562, "cityName": "Milford Mill" } }
+{ "create_at": datetime("2016-01-06T00:00:05.000Z"), "id": 684645599275302913, "text": "Hell yeah bro I was just thinking that ���� @Bigg_WillyStyle", "in_reply_to_status": 684645459193757697, "in_reply_to_user": 541257173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 541257173 }}, "user": { "id": 215556726, "name": "$T.", "screen_name": "Kwayyy1", "lang": "en", "location": "ATX ", "create_at": date("2010-11-14"), "description": "20 | College Student. #Blinn16 #SHSU18", "followers_count": 1157, "friends_count": 437, "statues_count": 31729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windemere, TX", "id": "d5bc2b23114eab00", "name": "Windemere", "place_type": "city", "bounding_box": rectangle("-97.671703,30.447168 -97.630762,30.475376") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4879676, "cityName": "Windemere" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007964231450626, "text": "why you always got an attitude, that's rude ����☺️�� ok gonna jam to @RealAdrielDiaz gnite �� https://t.co/msDKXCWDkT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 273124630 }}, "user": { "id": 118527198, "name": "austin :)", "screen_name": "livingdemjreams", "lang": "en", "location": "cali 408 ❤️", "create_at": date("2010-02-28"), "description": "chs'17 • diazzler • KAMFAM • go follow my vlog personal! @_aus4awesomee", "followers_count": 356, "friends_count": 276, "statues_count": 13053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cupertino, CA", "id": "36237ab3643ff2be", "name": "Cupertino", "place_type": "city", "bounding_box": rectangle("-122.091151,37.285786 -121.995595,37.34038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 617610, "cityName": "Cupertino" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007964575559680, "text": "Wind 0.0 mph ---. Barometer 30.413 in, Falling. Temperature 19.8 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 5628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007964898332672, "text": "@Dsilva_001 we can legally be together", "in_reply_to_status": 685007700929810432, "in_reply_to_user": 1144447692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1144447692 }}, "user": { "id": 3468612860, "name": "Kimmy P", "screen_name": "kimcpadilla", "lang": "en", "location": "pnw", "create_at": date("2015-09-06"), "description": "bieber addict", "followers_count": 365, "friends_count": 277, "statues_count": 5209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, WA", "id": "9904dedb2c0909ab", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-123.860437,46.95147 -123.778204,47.001666") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53027, "countyName": "Grays Harbor", "cityID": 5300100, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007965049491456, "text": "outta here https://t.co/flpTEF3r4q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268841296, "name": "Steelers SB50 Champs", "screen_name": "AllDayCallaway", "lang": "en", "location": "DPCU", "create_at": date("2011-03-19"), "description": "412 sports, baylor, thunder & hip hop #G1GB", "followers_count": 605, "friends_count": 680, "statues_count": 34984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broken Arrow, OK", "id": "f3665f19993977ca", "name": "Broken Arrow", "place_type": "city", "bounding_box": rectangle("-95.868576,35.959581 -95.671814,36.11903") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4009050, "cityName": "Broken Arrow" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007965460377600, "text": "https://t.co/Y3k80c6BFQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 329143270, "name": "Catty PaulaTicks", "screen_name": "JoeWard5", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-07-04"), "description": "Gay, Progressive, Liberal, Democrat, Animal Rights Activist, Nature Preservationist, Environmental Planet Earth Defender, Champion For The Greater Good", "followers_count": 427, "friends_count": 510, "statues_count": 78218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunwoody, GA", "id": "9c9fa69deb120a34", "name": "Dunwoody", "place_type": "city", "bounding_box": rectangle("-84.348046,33.915238 -84.264953,33.970918") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1324768, "cityName": "Dunwoody" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007966253142016, "text": "like my skin has been really bad lately and it makes me frustrated bc like why do i still have to deal with this have i not suffered enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32737531, "name": "TOP MADAM", "screen_name": "keumusang", "lang": "en", "location": "null", "create_at": date("2009-04-17"), "description": "끝없이 표류해", "followers_count": 73, "friends_count": 100, "statues_count": 32246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007966441832448, "text": "You know you've been in college and the hood for too long when you start dressing up totinos party pizza like its dominos ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279651743, "name": "ubiquitous Genie", "screen_name": "picturemewilly", "lang": "en", "location": "Houston,TX ✈️ Jackson, MS", "create_at": date("2011-04-09"), "description": "I'm not mugging I'm just from HOUSTON... [#NOCHILLZONE] [#TEXSIPPIAN] [#JSU18] [#RHOmanceintheCity] [#GOATographer]", "followers_count": 1048, "friends_count": 650, "statues_count": 32065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007966504812544, "text": "Rumble rumble rumble goes the thunder... And my stomach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2934106571, "name": "Natalie", "screen_name": "natalie_kouri", "lang": "en", "location": "null", "create_at": date("2014-12-20"), "description": "LW. 3/23/14", "followers_count": 580, "friends_count": 857, "statues_count": 2439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007967050174464, "text": "@MapQUEST_ you must be on ya break yo ��", "in_reply_to_status": 685007842466787328, "in_reply_to_user": 598043394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 598043394 }}, "user": { "id": 333260684, "name": "aDr✨", "screen_name": "DES_whatisaid", "lang": "en", "location": "null", "create_at": date("2011-07-11"), "description": "#CityCollegeAlum ⚫️", "followers_count": 1607, "friends_count": 1638, "statues_count": 63106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007967091961856, "text": "Random ambitious lmfao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1323751674, "name": "elena", "screen_name": "elenaanesyan", "lang": "en", "location": "null", "create_at": date("2013-04-02"), "description": "Jayceon Terrell Taylor❤️", "followers_count": 156, "friends_count": 115, "statues_count": 11601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007967293423618, "text": "Wait this whole movie happens cause her roommate is sick!!!!! What the fuck come on guys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218006952, "name": "Waheed", "screen_name": "StrwbrryWaffles", "lang": "en", "location": "Warrenton, VA", "create_at": date("2010-11-20"), "description": "According to my nephew I am the Duke of Chutney", "followers_count": 153, "friends_count": 945, "statues_count": 18151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Baltimore, VA", "id": "005b04ca23a596c0", "name": "New Baltimore", "place_type": "city", "bounding_box": rectangle("-77.7665,38.717694 -77.65921,38.770409") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51061, "countyName": "Fauquier", "cityID": 5155528, "cityName": "New Baltimore" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007967394119680, "text": "I've wanted nothing more than to be a daddy my whole life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46417072, "name": "Papa Roach", "screen_name": "SirStevieRogers", "lang": "en", "location": "ALC/OC/P$", "create_at": date("2009-06-11"), "description": "Lemme know if you want a hit, my nigga. I got plenty in the cut to give away.", "followers_count": 1875, "friends_count": 907, "statues_count": 209266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-07T00:00:00.000Z"), "id": 685007967503171584, "text": "Just posted a photo @ Raleigh, N.C. https://t.co/Isr9do9Ldm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.73623657,35.75673637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86569872, "name": "Sid Rokkam", "screen_name": "sidrokkam", "lang": "en", "location": "Richmond, VA", "create_at": date("2009-10-31"), "description": "The Best in the World! Believe no less, work hard, have positive drive and you will definitely succeed in whatever you do!", "followers_count": 224, "friends_count": 283, "statues_count": 668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cary, NC", "id": "5c6bd208d7ddf9f6", "name": "Cary", "place_type": "city", "bounding_box": rectangle("-78.944895,35.644384 -78.729147,35.867993") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3710740, "cityName": "Cary" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007967968624640, "text": "Just do it. Gunna be a long night https://t.co/2zvE6wWxL8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369203483, "name": "YoungBruin", "screen_name": "D1BOUND73", "lang": "en", "location": "Murda", "create_at": date("2014-03-02"), "description": "UCLA '19\nRVHS '15", "followers_count": 542, "friends_count": 412, "statues_count": 2542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007968195227648, "text": "Straight liq type of night��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3995071107, "name": "Milk && Cookies♓", "screen_name": "Im_bull_forreal", "lang": "en", "location": "Northside of Philly ", "create_at": date("2015-10-19"), "description": "Need I say more, when my name && @ states the obvious..✌ #SC: @itsbarbie2yu", "followers_count": 336, "friends_count": 624, "statues_count": 33208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007968669073408, "text": "https://t.co/enb64kikAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1465943851, "name": "julia", "screen_name": "julialopez_11", "lang": "en", "location": "church", "create_at": date("2013-05-28"), "description": "yacht klub", "followers_count": 793, "friends_count": 664, "statues_count": 21185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969151401984, "text": "LMFAOOOOOOO https://t.co/EHEUVeC3Aa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26270544, "name": "F.I.R.E.S.T.O.R.M", "screen_name": "EldonPerignon", "lang": "en", "location": "College Station", "create_at": date("2009-03-24"), "description": "The mind of a 50 year old piloting a 21 year old body. Don't try me. I might just burst into flames.", "followers_count": 940, "friends_count": 510, "statues_count": 110210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryan, TX", "id": "9ce0f1d2eb5998ce", "name": "Bryan", "place_type": "city", "bounding_box": rectangle("-96.439644,30.609214 -96.255918,30.722503") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4810912, "cityName": "Bryan" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969210122245, "text": "When refund hits I'm throwing a refund party in which I roll around in my money before it's snatched by my rent/phone bill/utilities/grocery", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43221360, "name": "youngscholarhabesha", "screen_name": "Dope_Ethio", "lang": "en", "location": "Dallas to Austin TX ", "create_at": date("2009-05-28"), "description": "Some people are so poor, all they have is money. Junior at The University of Texas class of '17. Life is everything I could want it to be.", "followers_count": 1435, "friends_count": 1222, "statues_count": 33217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969495441408, "text": "01/07@03:00 - Temp 24.6F, WC 24.6F. Wind 0.7mph WSW, Gust 2.0mph. Bar 30.400in, Falling. Rain 0.00in. Hum 87%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969621168128, "text": "Truth https://t.co/Rr1gYZhoXF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 154899744, "name": "John Doe", "screen_name": "iamkclaa", "lang": "en", "location": "Denton, Tx. ", "create_at": date("2010-06-12"), "description": "I am the thing that lesser men fear and I am not going to apologise for who I am. #GodFirst.#UNT' #SoccerIsEverything ⚽️. IG: Kclaa_", "followers_count": 1357, "friends_count": 1188, "statues_count": 17066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969893748736, "text": "@casey_cookies @uhlysssuh_ yeah & I love ittttt", "in_reply_to_status": 685007794248888321, "in_reply_to_user": 496495704, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 496495704, 1909841048 }}, "user": { "id": 315356808, "name": "pao", "screen_name": "paolssss", "lang": "en", "location": "null", "create_at": date("2011-06-11"), "description": "I just wanna party", "followers_count": 482, "friends_count": 421, "statues_count": 22270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969969246208, "text": "RT https://t.co/TH1IwfkbNI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 314949289, "name": "DJ", "screen_name": "Deangelo_Wash", "lang": "en", "location": "OKC", "create_at": date("2011-06-10"), "description": "Don't mention me cause IDC what you have to say about my tweets.", "followers_count": 2155, "friends_count": 1099, "statues_count": 24765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007969990225921, "text": "Gotta go to Lubbock to the mall tomorrow ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 917340296, "name": "Prisღ", "screen_name": "p_gonzales2013", "lang": "en", "location": "null", "create_at": date("2012-10-31"), "description": "my baby is my life ❤️❤️", "followers_count": 298, "friends_count": 345, "statues_count": 25369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hereford, TX", "id": "8778e199ce41ead8", "name": "Hereford", "place_type": "city", "bounding_box": rectangle("-102.431992,34.795084 -102.354,34.851214") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48117, "countyName": "Deaf Smith", "cityID": 4833320, "cityName": "Hereford" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970522931200, "text": "Apparently I was fooling myself thinking I could get a run in before it started raining again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170019894, "name": "Syd Aveechious", "screen_name": "syd_viciously", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-07-23"), "description": "Resident phantom of the Las Vegas nightlife. I don't know how it happened, but I am a rockstar.", "followers_count": 635, "friends_count": 914, "statues_count": 29084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerlin South, NV", "id": "0134e6167ff7f6ec", "name": "Summerlin South", "place_type": "city", "bounding_box": rectangle("-115.355825,36.082837 -115.314862,36.159073") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3270900, "cityName": "Summerlin South" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970632073216, "text": "Wind 0 mph --. Barometer 30.13 in, Falling slowly. Temperature 38.5 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970665512960, "text": "@rabshier99 ��wasn't me -R3 it was Kay", "in_reply_to_status": 685007695179456512, "in_reply_to_user": 2874534003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2874534003 }}, "user": { "id": 2874534003, "name": "AB3", "screen_name": "rabshier99", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-11-12"), "description": "@kayladevaney ❤️ University of Arizona commit #BearDown", "followers_count": 139, "friends_count": 83, "statues_count": 998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970770399232, "text": "Happy Birthday! ���� @cashaguilar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2585072438 }}, "user": { "id": 2556359484, "name": "Angel", "screen_name": "AngelAlvarado__", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "October's Very Own", "followers_count": 325, "friends_count": 254, "statues_count": 1961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970824916992, "text": "#tbt to when you loved me �� https://t.co/wBmlSM4nJU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tbt" }}, "user": { "id": 971652601, "name": "ʟıʟ ċяȏıṡṡѧṅṭ", "screen_name": "yvonneeruiiz", "lang": "en", "location": "I'm floating", "create_at": date("2012-11-26"), "description": "i try but it never seems to work out", "followers_count": 79, "friends_count": 112, "statues_count": 2796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007970854285312, "text": "Boy don't tempt me �������� https://t.co/pSkHKFWYsU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243953133, "name": "dilly", "screen_name": "BasedDilly", "lang": "en", "location": "null", "create_at": date("2011-01-27"), "description": "21 z33 @icylifebrand", "followers_count": 2392, "friends_count": 433, "statues_count": 122529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midland, WA", "id": "aae61bc9d261467d", "name": "Midland", "place_type": "city", "bounding_box": rectangle("-122.435689,47.158186 -122.396221,47.191813") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5345495, "cityName": "Midland" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971160477698, "text": "Another haiku and paint before I sleep! @ Smith Cove https://t.co/nylaX5lHXp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.38560275,47.63452359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2324292272, "name": "Kim Owens", "screen_name": "ksowens970", "lang": "en", "location": "Seattle, WA", "create_at": date("2014-02-02"), "description": "Nerium Brand Partner aspiring artist, entrepreneur, triathlete, blogger", "followers_count": 139, "friends_count": 283, "statues_count": 156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971160489984, "text": "this is how my mom be like �� https://t.co/wu0fFwwO5d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2610911108, "name": "minoshka", "screen_name": "_minoshkaaa", "lang": "en", "location": "null", "create_at": date("2014-07-07"), "description": "snapchat- mino.shkaaa", "followers_count": 80, "friends_count": 197, "statues_count": 371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kerman, CA", "id": "31fd66b3f28d4c5a", "name": "Kerman", "place_type": "city", "bounding_box": rectangle("-120.083084,36.716771 -120.041995,36.736803") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 638226, "cityName": "Kerman" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971215003649, "text": "Finna try this������������ https://t.co/NgfIdlsknm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3248463122, "name": "♊Cpt. LIHÆ⚡", "screen_name": "Ieremia_L", "lang": "en", "location": "In a Galaxy Far Far Away...", "create_at": date("2015-06-17"), "description": "Flying Overseas...\n \n •||SAMOAN||• °||99 till INFIN17Y||°", "followers_count": 276, "friends_count": 182, "statues_count": 2346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylorsville, UT", "id": "4b26b09f8108c4e8", "name": "Taylorsville", "place_type": "city", "bounding_box": rectangle("-111.986946,40.627557 -111.907843,40.686322") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4975360, "cityName": "Taylorsville" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971231731712, "text": "Lmao why is this true tho? https://t.co/1ztSLdbONq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2959313437, "name": "cold", "screen_name": "HeLoveNomi", "lang": "en", "location": "null", "create_at": date("2015-01-04"), "description": "null", "followers_count": 2128, "friends_count": 773, "statues_count": 41446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971424714753, "text": "Lea,dove, miranda in one pic hihi https://t.co/NHDFBV2zPh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 1011269994, "name": "angelica", "screen_name": "DEMONica_P", "lang": "en", "location": "CA/BLC", "create_at": date("2012-12-14"), "description": "5̶0̶%̶ ̶A̶n̶g̶e̶l̶, 50% Devil. Libra Queen. Athletic pig. Serial shopper. chilling like a villain. living life to the fullest! @lizshelcuenca's spoiled boo | V", "followers_count": 542, "friends_count": 2014, "statues_count": 29499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971516944384, "text": "Choreo use to come so easy to me but now its so hard af...��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330086293, "name": "Sherri Gonzalez", "screen_name": "SherriNicole96", "lang": "en", "location": "houston, tx : los angeles, ca", "create_at": date("2011-07-05"), "description": "SOREAL.//R.I.P my beautiful angel, Timothy Patrick Swords❤", "followers_count": 604, "friends_count": 402, "statues_count": 15951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:01.000Z"), "id": 685007971840032768, "text": "N.A.M.E https://t.co/AGPv7To2A2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 203953101, "name": "Glęni Matos", "screen_name": "GleniMatos", "lang": "en", "location": "Boston, Ma ", "create_at": date("2010-10-17"), "description": "7/23", "followers_count": 1379, "friends_count": 1982, "statues_count": 37547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972192403457, "text": "it's like they finally gain courage to do it after we get acquainted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197650149, "name": "MarcusTirrell™", "screen_name": "mawkiemawkkk", "lang": "en", "location": "Murfreesboro, Tennessee", "create_at": date("2010-10-01"), "description": "hi.", "followers_count": 530, "friends_count": 991, "statues_count": 11421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972427157504, "text": "I ate a weed cookie then blew my nose and thought my brain was actually coming out of my nostril", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4324093637, "name": "concrete angel", "screen_name": "diana_fowley", "lang": "en", "location": "Washington DC", "create_at": date("2015-11-29"), "description": "waiting in line", "followers_count": 30, "friends_count": 66, "statues_count": 64 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972712497152, "text": "Wind 0.0 mph ---. Barometer 30.254 in, Falling slowly. Temperature 27.6 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972850909184, "text": "Wind 2.0 mph ESE. Barometer 30.111 in, Falling. Temperature 40.2 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972888477698, "text": "https://t.co/NpAwaW5ZXz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2937921469, "name": "vgm", "screen_name": "__vaaaleria", "lang": "en", "location": "Guadalajara, Jalisco", "create_at": date("2014-12-21"), "description": "null", "followers_count": 808, "friends_count": 350, "statues_count": 8631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007972947202049, "text": "I'm going to find my homie Xiomara a girl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1029306704, "name": "T-GOD", "screen_name": "tarazon_", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "I'm not a God anymore", "followers_count": 1112, "friends_count": 850, "statues_count": 37781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973379215360, "text": "��", "in_reply_to_status": 684968351022452737, "in_reply_to_user": 2290415514, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2290415514, "name": "tyra bank$", "screen_name": "MamaHeather_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-13"), "description": "I'm pretty and ignorant. navy strong ⚓️", "followers_count": 734, "friends_count": 312, "statues_count": 50355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973412818945, "text": "@kuurbiee so is this proof that it's true ��", "in_reply_to_status": 685007804470538240, "in_reply_to_user": 2976967284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2976967284 }}, "user": { "id": 427606931, "name": "Liu Kang", "screen_name": "_lilmook7", "lang": "en", "location": "Terrell, Tx 972", "create_at": date("2011-12-03"), "description": "19.. Purge season.. but no pressure |College student athlete", "followers_count": 2335, "friends_count": 858, "statues_count": 69958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terrell, TX", "id": "00afd84c14c34f15", "name": "Terrell", "place_type": "city", "bounding_box": rectangle("-96.342904,32.701659 -96.256653,32.764463") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4872284, "cityName": "Terrell" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973433790464, "text": "4.3 magnitude earthquake earlier like an hour in a half ago. Scared me. I hate earthquakes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-124.15721692,40.79815255"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31070117, "name": "Lynnette Powell", "screen_name": "powell_lynnette", "lang": "en", "location": "Eureka, CA", "create_at": date("2009-04-13"), "description": "I have 2 daughters, 1 Adopted son. God is #1. And I'm a very nice woman to.", "followers_count": 186, "friends_count": 471, "statues_count": 8004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eureka, CA", "id": "c201deec6d7ba944", "name": "Eureka", "place_type": "city", "bounding_box": rectangle("-124.202007,40.745339 -124.136897,40.819908") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 623042, "cityName": "Eureka" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973479919617, "text": "⛈", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2347699465, "name": "Nicholas", "screen_name": "PaulosNick", "lang": "en", "location": "#88", "create_at": date("2014-02-16"), "description": "Bloods thicker than Water", "followers_count": 394, "friends_count": 267, "statues_count": 5867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973609902081, "text": "Thank you mommy love you so much ����® https://t.co/TYM1HDBi13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455405379, "name": "AMIRAH", "screen_name": "Kikidaope_A", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-01-04"), "description": "CSUMB C/o. 2k19 est. 97", "followers_count": 390, "friends_count": 308, "statues_count": 5373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973689749504, "text": "@paperxray this one takes the crown ��", "in_reply_to_status": 684896730408681472, "in_reply_to_user": 4151911095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4151911095 }}, "user": { "id": 191892850, "name": "Mmmm", "screen_name": "michellecar96", "lang": "en", "location": "IV/LA", "create_at": date("2010-09-17"), "description": "La chica de las poesias", "followers_count": 467, "friends_count": 428, "statues_count": 30478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973769441280, "text": "Wind 0.3 mph NW. Barometer 30.017 in, Falling Rapidly. Temperature 46.0 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 56110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973865906176, "text": "Just posted a photo @ My Fucking World https://t.co/1qhC6PbzNh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.18609714,40.64928339"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88970697, "name": "Team Me 4 Me $", "screen_name": "DatPhatFat", "lang": "en", "location": "New Jersey", "create_at": date("2009-11-10"), "description": "Team Fuck You Pay Me $$$", "followers_count": 133, "friends_count": 917, "statues_count": 10228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabeth, NJ", "id": "b74cebcb62a1a686", "name": "Elizabeth", "place_type": "city", "bounding_box": rectangle("-74.254211,40.634285 -74.138838,40.690673") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3421000, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007973991596033, "text": "Watching the tocada scene in La Banda De Los Panchitos has me wanting to see El Tri again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 890191056, "name": "Roger", "screen_name": "roger_wankenobi", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-10-18"), "description": "Drummer", "followers_count": 377, "friends_count": 589, "statues_count": 31291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974201294848, "text": "It is iiiiiiiiii.......\n\nThe Cuddler.. *cuddle dance*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2553268681, "name": "Him.", "screen_name": "LegendJohnson45", "lang": "en", "location": "#TheySleepMovement", "create_at": date("2014-06-07"), "description": "D-1 Student-Athlete l Defensive Tackle at #PVAMU #57 l BJM. l #InnovVtive l @BLACKLFMAG Photographer. l Bookings: LegendThePhotographer@gmail.com", "followers_count": 2544, "friends_count": 2343, "statues_count": 35018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974247436288, "text": "Bro I fucking hate when I'm sleep n mfs wanna turn on lights BITCH DO YOU WANNA DIE turn that shit off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 524820136, "name": "January 8th♑️", "screen_name": "QsCee", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-03-14"), "description": "FMOI @qscee", "followers_count": 1688, "friends_count": 1156, "statues_count": 38600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974322978816, "text": "They finally cleaned it https://t.co/cGh4tXVQj8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.5118094,29.4466307"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234588541, "name": "sophie", "screen_name": "yoitssophieee", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-06-02"), "description": "Richard", "followers_count": 39, "friends_count": 56, "statues_count": 223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974448812032, "text": "#2 from whataburger maybe https://t.co/898lALH99F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565941543, "name": "Prince.", "screen_name": "Princivicious", "lang": "en", "location": "#BrokeBoyz™", "create_at": date("2012-04-28"), "description": "90's African Kidd but I got crack like the 80's...................I like girls with nice nails #PVAMU", "followers_count": 601, "friends_count": 549, "statues_count": 47646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974452998144, "text": "sometimes I wish I wasn't so shy when it comes to singing cause I lowkey want to be doin the talent shows and shit ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 906459072, "name": "sarai", "screen_name": "sarahatrian1", "lang": "en", "location": "portland", "create_at": date("2012-10-26"), "description": "null", "followers_count": 181, "friends_count": 66, "statues_count": 350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974624972805, "text": "I hate that I'm just like wide awake because of that stupid nap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593101658, "name": "josh", "screen_name": "Joshxzxx", "lang": "en", "location": "Austin, TX", "create_at": date("2012-05-28"), "description": "Baby boy•pisces•", "followers_count": 557, "friends_count": 352, "statues_count": 8106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974876581889, "text": "@Windows @RAWRRitsJessica Oh you know she needs a mega computer that'll input hyper coding that'll hack sources such as the Pentagon.", "in_reply_to_status": 684855077228654593, "in_reply_to_user": 15670515, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15670515, 135164476 }}, "user": { "id": 973441429, "name": "Paul John La Penia", "screen_name": "peejaaylapenia", "lang": "en", "location": "null", "create_at": date("2012-11-26"), "description": "Be moderate in order to taste the joys of life in abundance.", "followers_count": 250, "friends_count": 260, "statues_count": 5514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974981447680, "text": "Bitch why lmao https://t.co/MbllTu9Rqv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3556909572, "name": "LADG", "screen_name": "Lilypaaaad___", "lang": "en", "location": "null", "create_at": date("2015-09-13"), "description": "y'all already know who it izzzz.", "followers_count": 90, "friends_count": 101, "statues_count": 566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007974985674754, "text": "Huh https://t.co/I1qCZ7wyIF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 207203414, "name": "Jimmy Peaches.", "screen_name": "OGBEARD", "lang": "en", "location": "PA ✈️ TX", "create_at": date("2010-10-24"), "description": "A coworker told me bitches on twitter love beards so im here...", "followers_count": 22688, "friends_count": 5283, "statues_count": 387568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975065513984, "text": "#SupportOriginMelissa 32.4°F Wind:0.0mph Pressure: 30.18hpa Falling Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975199580160, "text": "@jazzzminr lmfao��", "in_reply_to_status": 685007132765204480, "in_reply_to_user": 1618239301, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1618239301 }}, "user": { "id": 2686716438, "name": "R", "screen_name": "RosaxLidia", "lang": "en", "location": "Long Beach, CA", "create_at": date("2014-07-27"), "description": "null", "followers_count": 455, "friends_count": 286, "statues_count": 18800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975228903424, "text": "https://t.co/2ab1IzKwds", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2991843661, "name": "jack", "screen_name": "j_manna7", "lang": "en", "location": "CHS '16 OSU '20", "create_at": date("2015-01-21"), "description": "Strive for progress not perfection. Arsenal FC supporter.", "followers_count": 835, "friends_count": 656, "statues_count": 15711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keller, TX", "id": "ad760d32a6a31ea5", "name": "Keller", "place_type": "city", "bounding_box": rectangle("-97.266092,32.892358 -97.176637,32.986837") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4838632, "cityName": "Keller" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975325544448, "text": "pain is love n love is hell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446815325, "name": "03 ray lewis ﹩", "screen_name": "avo_cain", "lang": "en", "location": "hine $ide", "create_at": date("2011-12-25"), "description": "ain't never been no pressure.. life is funny like that. location on, just pull up. 1/4 JOBS. #UA19", "followers_count": 1737, "friends_count": 1417, "statues_count": 99279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, AL", "id": "1ccebd350c541331", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-87.015695,34.748712 -86.885875,34.837961") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1083, "countyName": "Limestone", "cityID": 102956, "cityName": "Athens" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975358926849, "text": "54.0F (Feels: 54.0F) - Humidity: 89% - Wind: 11.4mph E - Gust: 11.4mph - Pressure: 1027.3mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 221367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007975963082752, "text": "03:00:03 |Temp: 41.9ºF | Wind Chill 41.9ºF |Dew Point 33.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NE, Gusting to 3.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-07T00:00:02.000Z"), "id": 685007976021626880, "text": "every time i come home my bird is outside his cage lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 294955133, "name": "andres", "screen_name": "unn_dress", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2011-05-07"), "description": "here in my garage just bought this new uhh Lamborghini", "followers_count": 495, "friends_count": 386, "statues_count": 55261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007976541851648, "text": "Wind 0.0 mph ---. Barometer 30.35 in, Falling slowly. Temperature 12.4 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 91, "statues_count": 156484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007976806137857, "text": "Jazz thinks she's going to have me out all night . I have a flight to catch and I'm sleepy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 181869416, "name": "the EXPERIENCE", "screen_name": "MIZHANI_intl", "lang": "en", "location": "ON TOUR ✈", "create_at": date("2010-08-23"), "description": "growth is inevitable . transition is beautiful .", "followers_count": 77663, "friends_count": 3858, "statues_count": 140617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007976919257089, "text": "pandora is a music radio...Apple Music is on demand ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3968143578, "name": "yae ✨", "screen_name": "TheYaeShow", "lang": "en", "location": "St Louis, MO", "create_at": date("2015-10-21"), "description": "chapter 18", "followers_count": 199, "friends_count": 114, "statues_count": 3212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.845486,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007977611440129, "text": "@TadFifer25 @JJohnson924 what was mine?", "in_reply_to_status": 685006715796860928, "in_reply_to_user": 2262039470, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2262039470, 367535251 }}, "user": { "id": 352141787, "name": "James", "screen_name": "SHE_isgoin_HAN", "lang": "en", "location": "every time hoflo-chuck #EiuFB", "create_at": date("2011-08-09"), "description": "HF http://www.hudl.com/athlete/1641793/highlights/213809376", "followers_count": 927, "friends_count": 537, "statues_count": 18626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homewood, IL", "id": "6db05c8d8e092528", "name": "Homewood", "place_type": "city", "bounding_box": rectangle("-87.698993,41.538767 -87.631377,41.579114") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1735879, "cityName": "Homewood" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007977854676992, "text": "Temp: 41.1°F Wind:0.0mph Pressure: 30.104hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 65, "friends_count": 23, "statues_count": 56111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007978592747520, "text": "some shit don't meant to be I guess", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146926280, "name": "on a mission", "screen_name": "patsantana504", "lang": "en", "location": "new warleans, Algiers ", "create_at": date("2010-05-22"), "description": "free tayda !! free bear!! i never trusted but im working on it☺️ #NLMB", "followers_count": 3762, "friends_count": 2752, "statues_count": 61371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007978672427008, "text": "@matekeyer @TheDeerSavior excuse me what", "in_reply_to_status": 684999885611905024, "in_reply_to_user": 18890120, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18890120, 4732002087 }}, "user": { "id": 569320789, "name": "cher horowitz", "screen_name": "__andwerun", "lang": "en", "location": "null", "create_at": date("2012-05-02"), "description": "All that other bullshit is here today, and gone tomorrow.", "followers_count": 140, "friends_count": 173, "statues_count": 13219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tumwater, WA", "id": "12148f3ffcceb203", "name": "Tumwater", "place_type": "city", "bounding_box": rectangle("-123.010086,46.913138 -122.860365,47.029499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007978706042880, "text": "Maybe one day things will look brighter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486502781, "name": "Gary Lytle", "screen_name": "LytleGary", "lang": "en", "location": "null", "create_at": date("2012-02-08"), "description": "College..? It's real..? \nMSU Lake Charles, LA.", "followers_count": 338, "friends_count": 395, "statues_count": 5475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prien, LA", "id": "00a028c043bb68f0", "name": "Prien", "place_type": "city", "bounding_box": rectangle("-93.283003,30.106667 -93.213369,30.168682") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2262647, "cityName": "Prien" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007978789916674, "text": "�� https://t.co/V6JvmSJZ4P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1325515291, "name": "KG♔", "screen_name": "GilKat95", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "❁_❁So give me all of you,in exchange for me\\\\ 95' \\\\ #loveislove\\\\❁_❁", "followers_count": 175, "friends_count": 435, "statues_count": 1076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007978974605312, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2503058279, "name": "ITSAvashia ✍✨", "screen_name": "they_LoveSyd", "lang": "en", "location": "southern made gal. ", "create_at": date("2014-04-22"), "description": "#SU20' ❗️", "followers_count": 2871, "friends_count": 2514, "statues_count": 39614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bastrop, LA", "id": "76a017bc522edb14", "name": "Bastrop", "place_type": "city", "bounding_box": rectangle("-91.953951,32.749751 -91.868468,32.803653") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22067, "countyName": "Morehouse", "cityID": 2204685, "cityName": "Bastrop" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979012177920, "text": "Missing u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1117606460, "name": "EVA✨", "screen_name": "evaaaamariee", "lang": "en", "location": "null", "create_at": date("2013-01-24"), "description": "20. TX", "followers_count": 457, "friends_count": 306, "statues_count": 25987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979108675584, "text": "Happy birthday Christopher Martinez ❤️ \nI love you my son �� \n#8 #stopgrowing #loml https://t.co/v9WFfMJWwr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stopgrowing", "loml" }}, "user": { "id": 2906786396, "name": "jessica a. diaz", "screen_name": "fbfmgj", "lang": "en", "location": "East Los Angeles, CA", "create_at": date("2014-11-21"), "description": "i missed Twitter so i came back .", "followers_count": 55, "friends_count": 66, "statues_count": 335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979276546049, "text": "They already in the 80s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25135424, "name": "AntLux", "screen_name": "_AntLux", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-18"), "description": "snapchat- AnthonyOsm", "followers_count": 786, "friends_count": 901, "statues_count": 44969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979389702146, "text": "I'm jus tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405797401, "name": "Amanda", "screen_name": "_AmandaMarie16", "lang": "en", "location": "Louisiana", "create_at": date("2011-11-05"), "description": "suga how ya get so fly?", "followers_count": 468, "friends_count": 337, "statues_count": 2835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur, LA", "id": "253b154805ec7fbb", "name": "Sulphur", "place_type": "city", "bounding_box": rectangle("-93.427304,30.183555 -93.257523,30.277601") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2273640, "cityName": "Sulphur" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979490365440, "text": "Almost there ���� https://t.co/1e8347j8xJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2719031496, "name": "E⚡️A", "screen_name": "arroyoerick101", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "snapchat: Erick-rayo // phhhoto: Erarayo #flex", "followers_count": 331, "friends_count": 346, "statues_count": 6092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-07T00:00:03.000Z"), "id": 685007979876229121, "text": "Idk https://t.co/mbQktLqwoI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 251918108, "name": "Secreevia", "screen_name": "Surviaz268", "lang": "en", "location": "Last House On The Left", "create_at": date("2011-02-13"), "description": "Boston ✈️ Antigua ❤️ #Bosen #MSU15 #MidwesternState", "followers_count": 959, "friends_count": 763, "statues_count": 70013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007981272907776, "text": "Lol shots fired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446868215, "name": "BYE", "screen_name": "_Trusted1", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "@Chiefofthugs is my twin ❤️Kaiden King 01/26/2016 ❤️ #SFA HTS #NewElite Marketing.", "followers_count": 1470, "friends_count": 1178, "statues_count": 81095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007981751078912, "text": "or hes not allowed to have girl homies ����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2830994095, "name": "K2B", "screen_name": "hashtagk2b", "lang": "en", "location": "ig : hashtagk2b , Carson CA", "create_at": date("2014-09-24"), "description": "Keep yo heart 3 stacks", "followers_count": 498, "friends_count": 357, "statues_count": 13241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007981910581248, "text": "She's queen no questions asked���� https://t.co/jl1WWRorqV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1518230898, "name": "Chrt_Jenn", "screen_name": "jenny_0x0", "lang": "en", "location": "#Under₩aterSquadd", "create_at": date("2013-06-14"), "description": "♡Life is Divine Chaos♡|| ♡CHS2016|FIDM2020♡ I DONT LIKE TO MOVE OR GO OUT, I DONT EVEN LIKE TO MEET PEOPLE- Min Yoongi♡사랑해♡|| ||", "followers_count": 308, "friends_count": 1091, "statues_count": 6406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmingville, NY", "id": "508436a84be392b6", "name": "Farmingville", "place_type": "city", "bounding_box": rectangle("-73.078514,40.821459 -73.014675,40.859985") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3625417, "cityName": "Farmingville" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007982007087105, "text": "I Jus Got A Different Mindset About Everything Now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1924879213, "name": "Ateaa Cheyennee ***", "screen_name": "BeaaBlood", "lang": "en", "location": "Shawnee State University '19", "create_at": date("2013-10-01"), "description": "Kaimire ❤️", "followers_count": 548, "friends_count": 479, "statues_count": 12060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007982166331392, "text": "@_mr_joey_ @_TiannaG Amazonian queen❤️❤️��������", "in_reply_to_status": 685007705921159169, "in_reply_to_user": 562597994, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 562597994, 380097285 }}, "user": { "id": 386444844, "name": "B.Dot", "screen_name": "YaBoiiB_8", "lang": "en", "location": "null", "create_at": date("2011-10-07"), "description": "What are you willing to die for? Live for That!! Trust the dust -MiKeMc ❤ Go Bulls!!! #SeeRed", "followers_count": 355, "friends_count": 375, "statues_count": 35916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007982174670848, "text": "Happy 18th birthday bro! Much love �� https://t.co/CCre8BfD4Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473523845, "name": "alex.", "screen_name": "m1zro", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "null", "followers_count": 812, "friends_count": 44, "statues_count": 24824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007982808023040, "text": "Man that double date at the movies was cute af ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191731494, "name": "j.", "screen_name": "jcm_da", "lang": "en", "location": "null", "create_at": date("2015-05-10"), "description": "null", "followers_count": 90, "friends_count": 39, "statues_count": 2501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cudahy, CA", "id": "440191f4fe9d9752", "name": "Cudahy", "place_type": "city", "bounding_box": rectangle("-118.201978,33.955099 -118.170556,33.97089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617498, "cityName": "Cudahy" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007982858379265, "text": "Cheated on https://t.co/Ts7VrLTYMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2434235300, "name": "Šębì Uchiha", "screen_name": "Sebi_Uchiha", "lang": "en", "location": "Hyperbolic Time Chamber ", "create_at": date("2014-04-08"), "description": "I'm a super saiyan uchiha! Anime✨ Futbol, rapping I'm sociable so slide in my dms, wbu?", "followers_count": 367, "friends_count": 337, "statues_count": 17093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983198212097, "text": "@lilymaymac so does looking at one, cause damn how am I supposed to find someone as fine as you ��", "in_reply_to_status": 685006812718837762, "in_reply_to_user": 44097980, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44097980 }}, "user": { "id": 1373397128, "name": "Rex.", "screen_name": "StraightRexin", "lang": "en", "location": "null", "create_at": date("2013-04-22"), "description": "i like puns.", "followers_count": 216, "friends_count": 313, "statues_count": 2845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lorton, VA", "id": "1c59f406ae3ce30a", "name": "Lorton", "place_type": "city", "bounding_box": rectangle("-77.26569,38.660366 -77.192844,38.733662") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5147064, "cityName": "Lorton" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983248420865, "text": "Buying new hair products and refills warms my heart and curls in a very special way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565131702, "name": "Eazy E-Naj †", "screen_name": "Enijah_Jacquee", "lang": "en", "location": "Yay Area ✈️ LA", "create_at": date("2012-04-27"), "description": "IG/SC: Enijah_Jacquee | a writer | Spoken Word Artist | Poetess | Hip Hop Enthusiast | Unapologetically Black | #BlackLivesMatter", "followers_count": 1759, "friends_count": 670, "statues_count": 138957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983315562498, "text": "@JoshCreacy play bo3 w me ��", "in_reply_to_status": 685007899811131392, "in_reply_to_user": 423962963, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423962963 }}, "user": { "id": 614120109, "name": "Matt Siggy", "screen_name": "mattsiggy", "lang": "en", "location": "Riverside, CA", "create_at": date("2012-06-20"), "description": "don't take me serious.", "followers_count": 660, "friends_count": 368, "statues_count": 12590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983395221508, "text": "I go to sleep so late ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146499759, "name": "Jacqueline ✨", "screen_name": "__jacqueline29", "lang": "en", "location": "W/ Justin ", "create_at": date("2013-02-03"), "description": "null", "followers_count": 293, "friends_count": 264, "statues_count": 11853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983747579905, "text": "ya vendrá https://t.co/r1rBLTQ6eg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 599428287, "name": "Gloryalice ❄️", "screen_name": "bgvg_", "lang": "en", "location": "UPRH-842", "create_at": date("2012-06-04"), "description": "Cogito, ergo sum. Adsit Deus. ☤", "followers_count": 1591, "friends_count": 701, "statues_count": 90495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lower Grand Lagoon, FL", "id": "88aa336ca38215b9", "name": "Lower Grand Lagoon", "place_type": "city", "bounding_box": rectangle("-85.791986,30.120757 -85.724537,30.171807") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1241562, "cityName": "Lower Grand Lagoon" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007983982456832, "text": "But I'm hella sick so I'm like nah Ina sleep and eat some tomorrow. So I wake up the next day, only one little tiny slice left.", "in_reply_to_status": 685007819926417408, "in_reply_to_user": 2448352855, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2448352855, "name": "ZENSEI", "screen_name": "suemizer", "lang": "en", "location": "ELEPHANT HOUSE", "create_at": date("2014-04-16"), "description": "besta know what ya doin 'fore yo life get ruined ~ JV|AM", "followers_count": 298, "friends_count": 254, "statues_count": 8301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007984246665216, "text": "@Carlangas_OVOXO @sabrinna_xo", "in_reply_to_status": 685007849127161856, "in_reply_to_user": 491793261, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 491793261, 971159282 }}, "user": { "id": 491793261, "name": "•fam•", "screen_name": "Carlangas_OVOXO", "lang": "en", "location": "Eating your mom out.", "create_at": date("2012-02-13"), "description": "stop jocking my style m8", "followers_count": 293, "friends_count": 172, "statues_count": 7611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007984284442624, "text": "Ayyyyyyyyy he was goinn https://t.co/NzpyD9rmeo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63903849, "name": "$on", "screen_name": "punechii", "lang": "en", "location": "NINA, ca", "create_at": date("2009-08-07"), "description": "null", "followers_count": 645, "friends_count": 597, "statues_count": 20058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-07T00:00:04.000Z"), "id": 685007984288743426, "text": "That feeling not being good enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424367474, "name": "GOD:COMES:FIRST", "screen_name": "__HardWork__", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "♈️ 21 Born To Lose Built To Win. Love Yourself. Know Your Worth. Appreciate Life.", "followers_count": 6450, "friends_count": 6500, "statues_count": 40460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370351824441344, "text": "@gkristin38 go to sleep", "in_reply_to_status": 685370207230033920, "in_reply_to_user": 825526826, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 825526826 }}, "user": { "id": 2794982834, "name": "Jesus H. Christ Ⴟ", "screen_name": "chriscordova34", "lang": "en", "location": "Dunder Mifflin", "create_at": date("2014-09-06"), "description": "#UTSA18 Criminal Justice Major. IG: Cordovac34!", "followers_count": 339, "friends_count": 349, "statues_count": 12855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370351832834048, "text": "I asked my dad for money to get my nails done but I got my nose pierced instead ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313512120, "name": "Ashley Banks", "screen_name": "EarlsLips24", "lang": "en", "location": "Edmond, OK", "create_at": date("2011-06-08"), "description": "#LU19 Taste like Kandyyyyy", "followers_count": 780, "friends_count": 803, "statues_count": 35654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352185118720, "text": "@Escamilla_D74_ @NoChiIlSports LEFT SIDE!!!", "in_reply_to_status": 685370203736154113, "in_reply_to_user": 324170709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 324170709, 2269861836 }}, "user": { "id": 2327594367, "name": "Ágrios", "screen_name": "I_Sack41", "lang": "en", "location": "Whittier, CA", "create_at": date("2014-02-05"), "description": "20. Why start tomorrow when you can begin today?", "followers_count": 369, "friends_count": 351, "statues_count": 9145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352260628481, "text": "Free medium fries, free pie, free Powerade at mcdonz. This has been a great night to say the least.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464447407, "name": "Ryan Schreckenberg", "screen_name": "RSchreckenberg", "lang": "en", "location": "null", "create_at": date("2012-01-14"), "description": "Missouri State University. Pi Kappa Alpha.", "followers_count": 381, "friends_count": 327, "statues_count": 9793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbia, IL", "id": "d9f5ef3dff47ea85", "name": "Columbia", "place_type": "city", "bounding_box": rectangle("-90.24823,38.418419 -90.16704,38.487571") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17133, "countyName": "Monroe", "cityID": 1715833, "cityName": "Columbia" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352319377408, "text": "Jimi V. https://t.co/Xv13hKpqFW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.74440035,30.39284245"), "retweet_count": 0, "lang": "cs", "is_retweet": false, "user": { "id": 226437243, "name": "V.", "screen_name": "FuckJim", "lang": "en", "location": "Austin, TX", "create_at": date("2010-12-13"), "description": "#RoloGang #VAD3", "followers_count": 1166, "friends_count": 889, "statues_count": 50450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352415817729, "text": "Bismarck was put out of office; the 1890s were a horror period that led immediately to general warfare. -Lyndon LaRouche", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151552406, "name": "Richard O'Donnell", "screen_name": "Bartonesque", "lang": "en", "location": "null", "create_at": date("2010-06-03"), "description": "Charlemagne Is the Head of Gold (Daniel 2:31-45)", "followers_count": 2687, "friends_count": 2687, "statues_count": 72309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352524918788, "text": "@htxlotus @Jayfinessing he asked when I was going to Come see him then he said let him stop before people say we like each other", "in_reply_to_status": 685370018138210304, "in_reply_to_user": 3231771331, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3231771331, 498782470 }}, "user": { "id": 613930077, "name": "JazzyFizzle✨", "screen_name": "_gorgeouslust", "lang": "en", "location": "Humble, TX", "create_at": date("2012-06-20"), "description": "5'4 and i took your whore", "followers_count": 4659, "friends_count": 4966, "statues_count": 88850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352667463680, "text": "my boyfriend is actually perfect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177129069, "name": "zoie❣", "screen_name": "antiiviist", "lang": "en", "location": "oregonian", "create_at": date("2010-08-11"), "description": "employed. convalescent. blissful thinking.", "followers_count": 975, "friends_count": 195, "statues_count": 62281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370352692625408, "text": "When you go out all dressed up && all the other females chillin > ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313834518, "name": "still Infamouslondon", "screen_name": "Hustlerellaaa", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-08-12"), "description": "They tried to take my confidence when they knew I was cocky", "followers_count": 380, "friends_count": 462, "statues_count": 6188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370353019940864, "text": "Get this money and change locations", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2220411419, "name": "Life ain't easy", "screen_name": "GreenJacory", "lang": "en", "location": "null", "create_at": date("2013-12-11"), "description": "Just watch me make it✈️", "followers_count": 702, "friends_count": 1045, "statues_count": 4022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shively, KY", "id": "edbc1ac7f306fad2", "name": "Shively", "place_type": "city", "bounding_box": rectangle("-85.851605,38.170899 -85.78059,38.220778") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2170284, "cityName": "Shively" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370353024016384, "text": "I wish I had a ride or die.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2577228414, "name": "bre", "screen_name": "dattlightskin", "lang": "en", "location": "null", "create_at": date("2014-06-19"), "description": "null", "followers_count": 234, "friends_count": 99, "statues_count": 1888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rapid City, SD", "id": "4dcfc855e2614f09", "name": "Rapid City", "place_type": "city", "bounding_box": rectangle("-103.315567,44.01364 -103.151254,44.136814") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46103, "countyName": "Pennington", "cityID": 4652980, "cityName": "Rapid City" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370354001313792, "text": "omg!!!!!!!!!!!! tbh Jesse> Jake https://t.co/ZlHPmBGuvj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2190913364, "name": "champagne spice", "screen_name": "mrsstjd", "lang": "en", "location": "null", "create_at": date("2013-11-12"), "description": "null", "followers_count": 80, "friends_count": 81, "statues_count": 6655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370354034802688, "text": "Gangsta Grillzzzzz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 224072103, "name": "FlyLo Ren", "screen_name": "Huntykins", "lang": "en", "location": "in the cut", "create_at": date("2010-12-07"), "description": "'I know' more than Bill Withers", "followers_count": 1361, "friends_count": 851, "statues_count": 111326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968758") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370354072694784, "text": "Wind 0.0 mph ---. Barometer 30.238 in, Steady. Temperature 20.6 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 5652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370354781429760, "text": "I SCREAMED this has to be fake!!!! https://t.co/2WJPcRzJGq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558329922, "name": "kaitie", "screen_name": "kaitlynicole27", "lang": "en", "location": "null", "create_at": date("2012-04-19"), "description": "null", "followers_count": 572, "friends_count": 276, "statues_count": 25811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370354794119168, "text": "Wind 0.0 mph WSW. Barometer 29.870 in, Falling. Temperature 52.1 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 66, "friends_count": 23, "statues_count": 56244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370355393933312, "text": "Ko", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 519959034, "name": "Tayy", "screen_name": "A1Leek_", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2012-03-09"), "description": "Lack of involvement & Spiritual maturity", "followers_count": 539, "friends_count": 574, "statues_count": 4886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Finneytown, OH", "id": "b05fe73a135dedcc", "name": "Finneytown", "place_type": "city", "bounding_box": rectangle("-84.536174,39.199213 -84.489746,39.234386") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3927104, "cityName": "Finneytown" } }
+{ "create_at": datetime("2016-01-08T00:00:00.000Z"), "id": 685370355586711553, "text": "Bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2348648090, "name": "X", "screen_name": "Xavier_Gta", "lang": "en", "location": "null", "create_at": date("2014-02-17"), "description": "work hard , play hard ✊Heb warehouse , single. Dont let your past , define your future", "followers_count": 162, "friends_count": 108, "statues_count": 9825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370355939057664, "text": "@Martilla006 @DulceMaria https://t.co/bwtQeHv30C", "in_reply_to_status": 685369717444489216, "in_reply_to_user": 115378199, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 115378199, 79327591 }}, "user": { "id": 4220518635, "name": "OrlandoAmilianmiami", "screen_name": "OrlandoMilian2", "lang": "en", "location": "null", "create_at": date("2015-11-18"), "description": "null", "followers_count": 469, "friends_count": 2019, "statues_count": 31274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370356022947840, "text": "sick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 911931637, "name": "Mahiah Rigali", "screen_name": "Mango_Trees11", "lang": "en", "location": "cali", "create_at": date("2012-10-29"), "description": "derby girl/venti", "followers_count": 231, "friends_count": 214, "statues_count": 5075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370356073402368, "text": "Wind 0 mph --. Barometer 29.95 in, Falling slowly. Temperature 44.2 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370356631130112, "text": "@ImJustDely correct, I’m sure there will be a bunch of games getting streamed around that time", "in_reply_to_status": 685370004716433408, "in_reply_to_user": 19768068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19768068 }}, "user": { "id": 39045004, "name": "Anakin Skywalker.", "screen_name": "FlourishOrDie", "lang": "en", "location": "ODU", "create_at": date("2009-05-10"), "description": "Either do, or don't. There is no try.", "followers_count": 1084, "friends_count": 584, "statues_count": 107358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370356861808643, "text": "@Alex_Bryner_ happy birthday dad", "in_reply_to_status": -1, "in_reply_to_user": 419286466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 419286466 }}, "user": { "id": 336997035, "name": "Vérsace Tristan", "screen_name": "TKTwilligear", "lang": "en", "location": "Pullman⛅️", "create_at": date("2011-07-17"), "description": "bishop | broke college student & professional sign stealer #BoomSquad | ΔΨ | Marvins Room", "followers_count": 1428, "friends_count": 510, "statues_count": 77695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederickson, WA", "id": "1dc0a9a76e5081b3", "name": "Frederickson", "place_type": "city", "bounding_box": rectangle("-122.400295,47.062069 -122.314857,47.127584") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5325475, "cityName": "Frederickson" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370357168107520, "text": "Time to write my first article for Medium, in the bathtub. #ohyeah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ohyeah" }}, "user": { "id": 4698680432, "name": "Elizabeth Van Haver", "screen_name": "yeezusmeist", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2016-01-02"), "description": "My implication is almost never an insinuation unless it has now become your situation. http://queenmisanthrope.tumblr.com", "followers_count": 25, "friends_count": 130, "statues_count": 447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370357293842432, "text": "When I'm sick I never get full wtf ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1712070654, "name": "luii14", "screen_name": "OhShitLuii", "lang": "en", "location": "Riverside Cali ", "create_at": date("2013-08-30"), "description": "I am strong because God is by my side , My first love is soccer ⚽️. IG. l.l_14 sc. luii14", "followers_count": 180, "friends_count": 202, "statues_count": 8279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370358145376256, "text": "How I get confused with my own insider ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403813357, "name": "Samanthaa ❣", "screen_name": "Samm3__", "lang": "en", "location": "Bridgeport, CT", "create_at": date("2011-11-02"), "description": "⚡TBA:patiently, watch how I come.... ⚡️ // ig : samm3__ // snapchat : fckxallat // vine : Samantha Baybie", "followers_count": 2570, "friends_count": 2337, "statues_count": 99634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoboken, NJ", "id": "e9143a85705b4d40", "name": "Hoboken", "place_type": "city", "bounding_box": rectangle("-74.044085,40.7336 -74.020431,40.759159") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3432250, "cityName": "Hoboken" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370358166360064, "text": "Wind 2.0 mph ESE. Barometer 29.806 in, Falling Rapidly. Temperature 44.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370358891999233, "text": "Wind 0.0 mph SE. Barometer 29.982 in, Falling. Temperature 42.9 °F. Rain today 0.03 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 40, "friends_count": 25, "statues_count": 16714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-08T00:00:01.000Z"), "id": 685370359185457152, "text": "01/08@03:00 - Temp 37.9F, WC 37.9F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.195in, Rising slowly. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370360003432448, "text": "03:00:03 |Temp: 51.4ºF | Wind Chill 51.4ºF |Dew Point 46.7ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the ENE, Gusting to 1.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370360301146113, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1850271733, "name": "Bells", "screen_name": "bellacdeleon71", "lang": "en", "location": "null", "create_at": date("2013-09-09"), "description": "Saugus Swim, it's all for you Jenn #ProjectGuac", "followers_count": 1833, "friends_count": 1230, "statues_count": 41969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arroyo Grande, CA", "id": "3ece1bab2041284e", "name": "Arroyo Grande", "place_type": "city", "bounding_box": rectangle("-120.60993,35.101686 -120.551631,35.146993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 602868, "cityName": "Arroyo Grande" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370361001672704, "text": "@DanyellieG dam looks like you got your wish holly holm vs. miesha tate at ufc 197", "in_reply_to_status": -1, "in_reply_to_user": 1315695468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1315695468 }}, "user": { "id": 1034321581, "name": "RICHARD SAWYER", "screen_name": "ufcfan_2013", "lang": "en", "location": "SOUTH BEND INDIANA ", "create_at": date("2012-12-25"), "description": "i am a big ufc fan and a big fan of both ronda rousey and jessamyn duke", "followers_count": 1023, "friends_count": 2012, "statues_count": 7947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370361408536577, "text": "I suppose to be sleep��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3704366296, "name": "✨tenderoni✨", "screen_name": "FineAssssss", "lang": "en", "location": "Savannah, GA 9⃣1⃣2⃣", "create_at": date("2015-09-18"), "description": "I live w|no fear so my success can't be measured❗️", "followers_count": 176, "friends_count": 126, "statues_count": 2898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370361697812480, "text": "Should've know you love playing games", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2907864528, "name": "Yazzy", "screen_name": "yazzzy30", "lang": "en", "location": "null", "create_at": date("2014-11-23"), "description": "null", "followers_count": 170, "friends_count": 141, "statues_count": 1273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370361718779906, "text": "Snap chat me @ miranda1594", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1265381084, "name": "Miranda Anice", "screen_name": "Bunny83015", "lang": "en", "location": "Clovis, CA", "create_at": date("2013-03-13"), "description": "6/10/13, 10/12/15 Shaun Michael 8/30/153 ☥ The way is not in the sky. The way is in the heart * PHILIPPIANS 4:13❤️", "followers_count": 605, "friends_count": 414, "statues_count": 19940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370362318569472, "text": "@cjg2127 @GonzOakland @woolie @MBridegam and you could always take them the Gospel according to TOD :)", "in_reply_to_status": 685369926077423616, "in_reply_to_user": 29210364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29991395, 87852003, 3699001, 558143796 }}, "user": { "id": 29210364, "name": "Jon Schwark", "screen_name": "vjon", "lang": "en", "location": "San Francisco", "create_at": date("2009-04-06"), "description": "Freelance video editing and motion graphics. Marketing. Art. Travel. Recovering VJ. SF Housing and Urbanism. My daughter's art.", "followers_count": 257, "friends_count": 354, "statues_count": 1799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SoMa, San Francisco", "id": "2b6ff8c22edd9576", "name": "SoMa", "place_type": "neighborhood", "bounding_box": rectangle("-122.422849,37.768935 -122.3964,37.787529") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370362607964160, "text": "seeing my cousin like this fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2636227227, "name": "ana", "screen_name": "analisaglz", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-06-23"), "description": "jas, mai", "followers_count": 672, "friends_count": 412, "statues_count": 38966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370362738032640, "text": "‼️‼️ https://t.co/P5tckGp5Dr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 729833748, "name": "DES", "screen_name": "DesireeeGarcia_", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "keep it moving", "followers_count": 918, "friends_count": 636, "statues_count": 20249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370362796863489, "text": "#SupportOriginMelissa 44.6°F Wind:0.0mph Pressure: 29.91hpa Falling Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370363513933824, "text": "@KUHoops", "in_reply_to_status": 684238436383891457, "in_reply_to_user": 132981804, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 132981804 }}, "user": { "id": 2908297781, "name": "Johnny Condit", "screen_name": "ConditMMA", "lang": "en", "location": "Oklahoma City ", "create_at": date("2014-12-06"), "description": "I sell Herbalife try help other reach there goals My is still compete in Jiu-Jitsu tournaments than step foot in the cage one day and biggest UFC fan Phil 4:13", "followers_count": 59, "friends_count": 437, "statues_count": 62 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370363652354048, "text": "HAPPY BIRTHDAY TO ME ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2421359946, "name": "moe ☪", "screen_name": "monee03", "lang": "en", "location": "null", "create_at": date("2014-03-31"), "description": "leanin' ⚗", "followers_count": 544, "friends_count": 480, "statues_count": 14861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Banning, CA", "id": "792551bc9bd3c992", "name": "Banning", "place_type": "city", "bounding_box": rectangle("-116.947005,33.902607 -116.849846,33.94771") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 603820, "cityName": "Banning" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370363824373760, "text": "HAH! Ya can't trust anyone ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2197280023, "name": "Natalie Kojababian", "screen_name": "natalie_koji", "lang": "en", "location": "Inconclusive ", "create_at": date("2013-11-15"), "description": "JBHS ASB • Junior Class Prez • Andranik♡", "followers_count": 538, "friends_count": 554, "statues_count": 7537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-01-08T00:00:02.000Z"), "id": 685370363853705216, "text": "Top 10 in Currency in California, January 2016. I'm winning on @QuizUp - https://t.co/2fL8m6sZub", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1447558507 }}, "user": { "id": 62245443, "name": "Kate Eleazar Hebron", "screen_name": "HaloKatiee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-08-02"), "description": "Dreamer | Currently eating probably | I LOVE MUSIC :D", "followers_count": 314, "friends_count": 1009, "statues_count": 37004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370364151595008, "text": "@ChelseyHoney Well at least someone is telling me something. But uh, don't leave? Or visit Muncie...but don't leave me.", "in_reply_to_status": 685370087038046208, "in_reply_to_user": 1217329016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1217329016 }}, "user": { "id": 1188807822, "name": "pool, dead", "screen_name": "carlywankenobi", "lang": "en", "location": "stewjon", "create_at": date("2013-02-17"), "description": "never tell me the odds || @ryan_yensen", "followers_count": 339, "friends_count": 320, "statues_count": 21394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370364227129344, "text": "Temp: 46.0°F Wind:0.0mph Pressure: 29.951hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 66, "friends_count": 23, "statues_count": 56245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370364621291522, "text": "53.8F (Feels: 53.8F) - Humidity: 99% - Wind: 1.6mph E - Gust: 3.1mph - Pressure: 1024.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 221513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370364772397056, "text": "Travelling to Edmonton or just twittering about Edmonton? https://t.co/dE5aI3DtoI #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.49,53.5436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Edmonton" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 983, "friends_count": 312, "statues_count": 2518140 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370364881289216, "text": "Oh wait... I do know.. Bc I freaking care. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3170188537, "name": "Kezzymiia", "screen_name": "kezzymiia", "lang": "en", "location": "Encino, Los Angeles", "create_at": date("2015-04-23"), "description": "Philippians 4:13 | Have Faith | Don't sweat the small stuff | Be Healthy | Dream Big | Travel Often | Love Always | IG: kezzymiia | SnapChat: Kezzybaby", "followers_count": 67, "friends_count": 126, "statues_count": 779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370365221015552, "text": "Craving fucking satisfied ���� https://t.co/gbbrSnKqfI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3015218523, "name": "Lys", "screen_name": "LysabelleMendi9", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "mvh", "followers_count": 124, "friends_count": 184, "statues_count": 1867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial Beach, CA", "id": "5cc07810d110956f", "name": "Imperial Beach", "place_type": "city", "bounding_box": rectangle("-117.133501,32.555434 -117.099564,32.591984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 636294, "cityName": "Imperial Beach" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370365741252608, "text": "Wind 0.0 mph ---. Barometer 30.23 in, Steady. Temperature 16.0 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 90, "statues_count": 156508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370366114447360, "text": "Dej Loaf Lowkey Look Good ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2415346789, "name": "SAUCE", "screen_name": "DatSauceBoi", "lang": "en", "location": "Flint Spartan", "create_at": date("2014-03-27"), "description": "GetAwk3", "followers_count": 444, "friends_count": 286, "statues_count": 11509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haslett, MI", "id": "4399b5004a3b4d9a", "name": "Haslett", "place_type": "city", "bounding_box": rectangle("-84.483739,42.731229 -84.350094,42.787887") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2637100, "cityName": "Haslett" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370366580002817, "text": "Shit was legit �� makes me wonder if our future will be like this https://t.co/j6PEeDHMtC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2518347174, "name": "Rachell", "screen_name": "Smileboogieman", "lang": "en", "location": "null", "create_at": date("2014-05-23"), "description": "沖縄✈Cali// El Cajon", "followers_count": 655, "friends_count": 792, "statues_count": 3126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370367334957056, "text": "I'm dying right now �� go look up animals accidentally eating weed most funniest thing I've seen in a while ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3060812056, "name": "Mary Martinez", "screen_name": "3377mary", "lang": "en", "location": "neverland", "create_at": date("2015-02-24"), "description": "cvhs ~ never underestimate a small person❄️", "followers_count": 237, "friends_count": 283, "statues_count": 784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370367938969601, "text": "Vessel is 3 years old today. This album saved my life and so many other lives. Thank yøu |-/ https://t.co/gqzoGaE8JF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 846676470, "name": "Raquel |-/", "screen_name": "RaquelBrincat", "lang": "en", "location": "null", "create_at": date("2012-09-25"), "description": "15 | Bi | Tøp | Scorpio | Vegas", "followers_count": 511, "friends_count": 826, "statues_count": 6959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-01-08T00:00:03.000Z"), "id": 685370368152866816, "text": "ugh why is everyone asleep ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 907297057, "name": "Als", "screen_name": "soffe_allie", "lang": "en", "location": "null", "create_at": date("2012-10-26"), "description": "happy", "followers_count": 867, "friends_count": 542, "statues_count": 29385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Felida, WA", "id": "3990e08a4d4cdafe", "name": "Felida", "place_type": "city", "bounding_box": rectangle("-122.734598,45.693162 -122.682473,45.733492") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5323550, "cityName": "Felida" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370368404492289, "text": "Man I'm trippin I gotta be up for 9:30.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59720772, "name": "THE Tre' Hunter™", "screen_name": "QBs_Nightmare", "lang": "en", "location": "Intercepting The Ball", "create_at": date("2009-07-24"), "description": "Full Fledged Follower Of Christ. Diagnosed with Telesphobia at birth. Safety @ULM #4 #WarhawkNation #BlackWoo WOOP #LAKERNATION #DreamKiller #QBsNightmare", "followers_count": 1701, "friends_count": 1611, "statues_count": 24198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, LA", "id": "e60ab9aacc0e96af", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-92.135057,32.429655 -92.018346,32.612831") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2251410, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370369159593984, "text": "I shoulda stayed my ass asleep��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111829147, "name": "Gabbs❤️", "screen_name": "ImQueeeen", "lang": "en", "location": "Suisun City, CA", "create_at": date("2010-02-05"), "description": "Obviously Oblivious", "followers_count": 2526, "friends_count": 2361, "statues_count": 17712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370369167904768, "text": "#sixwordscifi She took her spaceship to places unknown.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sixwordscifi" }}, "user": { "id": 2635340887, "name": "Women in Media", "screen_name": "WomenNMedia", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-07-13"), "description": "Connecting above & below the line women & the men who love making movies with them. Join our mailing list: womennmedia@gmail.com", "followers_count": 49892, "friends_count": 39202, "statues_count": 19480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370369444687872, "text": "But really don't got one https://t.co/Ank9l9cQuD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3150505334, "name": "Ålexzandra", "screen_name": "___alexzandraaa", "lang": "en", "location": "null", "create_at": date("2015-04-11"), "description": "A$AP", "followers_count": 180, "friends_count": 174, "statues_count": 4316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370370229047296, "text": "Wow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 549619856, "name": "Cisco the jake", "screen_name": "Sco_daddy23", "lang": "en", "location": "Des Moines, IA ", "create_at": date("2012-04-09"), "description": "#RipMommaBurton| you should follow me mate", "followers_count": 874, "friends_count": 870, "statues_count": 24517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.49702 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370370258419713, "text": "@saharaedmonton Sahara Palace #Edmonton https://t.co/GyzF6aIClL Excellent https://t.co/MC8a1TSLcM", "in_reply_to_status": -1, "in_reply_to_user": 2694933772, "favorite_count": 0, "coordinate": point("-113.516585,53.607305"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "Edmonton" }}, "user_mentions": {{ 2694933772 }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3652, "friends_count": 507, "statues_count": 162993 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370370631700480, "text": "@skinnypornstar ��", "in_reply_to_status": 685367613162008576, "in_reply_to_user": 4499583379, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4499583379 }}, "user": { "id": 452811036, "name": "FlowerChildPao", "screen_name": "PaolaSerrano21", "lang": "en", "location": "null", "create_at": date("2012-01-02"), "description": "19 | do you know how to plur?", "followers_count": 743, "friends_count": 800, "statues_count": 23145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370371004993536, "text": "no one can replace me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2698729358, "name": "angelface", "screen_name": "paigelaneyy", "lang": "en", "location": "null", "create_at": date("2014-08-01"), "description": "la poesia della vita.. dog mom.. pro era.. incognito.. I'll tell you I have a boyfriend♎️", "followers_count": 544, "friends_count": 543, "statues_count": 7815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandy, UT", "id": "fb9549afa6f81fa6", "name": "Sandy", "place_type": "city", "bounding_box": rectangle("-111.921658,40.528084 -111.800273,40.616827") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967440, "cityName": "Sandy" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370371080499201, "text": "@AdamSandlerYou are awesome hug all of your crew for me, I love them just as much as you!Tell David Spade that I think he is sexy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4142246472, "name": "Michele Jorgensen", "screen_name": "smile_jorgensen", "lang": "en", "location": "Midvale, UT", "create_at": date("2015-11-05"), "description": "Lover of Cinema, I have a beautiful heart and soul; nobody famous. This is my dog Lucky RIP. I miss her. Peace and Love!", "followers_count": 43, "friends_count": 129, "statues_count": 89 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midvale, UT", "id": "ea98531b10405c6a", "name": "Midvale", "place_type": "city", "bounding_box": rectangle("-111.921741,40.593357 -111.855658,40.631074") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4949710, "cityName": "Midvale" } }
+{ "create_at": datetime("2016-01-08T00:00:04.000Z"), "id": 685370371370016770, "text": "@newmoonrisin @GentleGRShepard @SassCBrown funny.", "in_reply_to_status": 685370282685665280, "in_reply_to_user": 147821230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 147821230, 373789549, 2393063119 }}, "user": { "id": 117180375, "name": "istvan Balogh", "screen_name": "investigator50", "lang": "en", "location": "Pittsburgh,PA", "create_at": date("2010-02-24"), "description": "licensed private investigator and security consultant.", "followers_count": 544, "friends_count": 1135, "statues_count": 1184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Park Township, PA", "id": "fdef0b107b807c00", "name": "South Park Township", "place_type": "city", "bounding_box": rectangle("-80.032196,40.267267 -79.955324,40.338235") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4272403, "cityName": "South Park Township" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370372753985536, "text": "fog -> fair\ntemperature down 60°F -> 58°F\nwind 6mph -> 5mph\nvisibility 1mi -> 10mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.92506,29.39276"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 221225938, "name": "Texas City Weather", "screen_name": "TexasCityTX", "lang": "en", "location": "Texas City, TX", "create_at": date("2010-11-29"), "description": "Weather updates, forecast, warnings and information for Texas City, TX. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 101, "friends_count": 3, "statues_count": 15850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas City, TX", "id": "05959d7d3c4d4c27", "name": "Texas City", "place_type": "city", "bounding_box": rectangle("-95.049499,29.33728 -94.886484,29.422811") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4872392, "cityName": "Texas City" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370372921901057, "text": "@nfrege non, mais une putain de sensation", "in_reply_to_status": 685370241698889728, "in_reply_to_user": 2937101303, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 2937101303 }}, "user": { "id": 6590192, "name": "Pierre Schuhl", "screen_name": "pierresc", "lang": "fr", "location": "France", "create_at": date("2007-06-05"), "description": "Éclectique par dessus tout. Beaucoup de #Rugby, de #Bruce, et de #Techno. #IoT #Digital #Trends Mes tweets ne reflètent que ma seule opinion\n\nParis - Lamorlaye", "followers_count": 441, "friends_count": 942, "statues_count": 19621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370373211205632, "text": "Temp: 36.2°F - Dew Point: 30.9° - Wind: 1.2 mph - Gust: 4.5 - Rain Today: 0.02in. - Pressure: 29.90in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 11, "statues_count": 11691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370373295222785, "text": "He said, \"I thought you were the one until I discovered you had webbed feet - that is completely fucked up\". #FirstLineToMyNovel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FirstLineToMyNovel" }}, "user": { "id": 2347721474, "name": "Jennifer Etheridge", "screen_name": "Iambanjo1", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2014-02-16"), "description": "Evil Genius", "followers_count": 26, "friends_count": 55, "statues_count": 197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370373697736704, "text": "@CyrusDesolace @Sipinator", "in_reply_to_status": 684266187748802560, "in_reply_to_user": 3430432079, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3430432079, 1549363454 }}, "user": { "id": 3430432079, "name": "cy", "screen_name": "CyrusDesolace", "lang": "en", "location": "null", "create_at": date("2015-08-18"), "description": "RCHS ASB 17'", "followers_count": 495, "friends_count": 396, "statues_count": 2714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370373714477056, "text": "@ohbriemarie yes then publish it. Instant bestseller", "in_reply_to_status": 685366416506093568, "in_reply_to_user": 201100649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 201100649 }}, "user": { "id": 540643245, "name": "Drew Holliday", "screen_name": "holliday_drew", "lang": "en", "location": "null", "create_at": date("2012-03-29"), "description": "In the world of New Girl, I am a solid Winston", "followers_count": 538, "friends_count": 127, "statues_count": 16491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosedale, CA", "id": "4caadee2bfd4ee70", "name": "Rosedale", "place_type": "city", "bounding_box": rectangle("-119.207649,35.354368 -119.128196,35.40526") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 662854, "cityName": "Rosedale" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370373936775168, "text": "Thank god im about to smoke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 994425834, "name": "BABADOOK", "screen_name": "EthanIsSober", "lang": "en", "location": "NV", "create_at": date("2012-12-06"), "description": "18 •snapchat ethan.fy", "followers_count": 1025, "friends_count": 643, "statues_count": 10312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370374209585152, "text": "@cassidy_rendos when ur just living ur life so good ppl start to hate when we don't even give em attention ��������", "in_reply_to_status": -1, "in_reply_to_user": 971136326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 971136326 }}, "user": { "id": 3291995163, "name": "Tal", "screen_name": "tsweezzzyyy", "lang": "en", "location": "null", "create_at": date("2015-05-20"), "description": "disMylyfe", "followers_count": 220, "friends_count": 332, "statues_count": 2428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensburg, PA", "id": "2d2cc38bfa01da4a", "name": "Greensburg", "place_type": "city", "bounding_box": rectangle("-79.596255,40.262029 -79.485605,40.348027") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42129, "countyName": "Westmoreland", "cityID": 4231200, "cityName": "Greensburg" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370374213652480, "text": "Robbers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 361651920, "name": "Marcos Molinar", "screen_name": "MarcosMolinar1", "lang": "en", "location": "bay area", "create_at": date("2011-08-24"), "description": "Lead singer for the band @2morrowsj, Chasing Dreams everyday. Animals EP here: https://m.soundcloud.com/2morrows-june/sets/animals-ep & 1st Dance video Out Now!", "followers_count": 435, "friends_count": 613, "statues_count": 2302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370375253852160, "text": "being thirsty on twitter dont count if ion thirst over u in person u was catfish https://t.co/8AgJgioEnv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2265338312, "name": "Stephen Curvey", "screen_name": "FancyFatBoy", "lang": "en", "location": "null", "create_at": date("2013-12-27"), "description": "#Tamuc14-89", "followers_count": 600, "friends_count": 526, "statues_count": 39026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370375262228480, "text": "I'm so embarrassing lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189691585, "name": "Kelss", "screen_name": "kelseystephenss", "lang": "en", "location": "Your ❤️", "create_at": date("2010-09-11"), "description": "I like fresh fruit on the table & chocolate chips in my pancakes", "followers_count": 286, "friends_count": 229, "statues_count": 35525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370375534870528, "text": "Really need a new start", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075091342, "name": "Menace", "screen_name": "DennisRod18", "lang": "en", "location": "Freeport ,TX", "create_at": date("2013-01-09"), "description": "Living hard hat days and honky tonk nights", "followers_count": 303, "friends_count": 277, "statues_count": 6227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Jackson, TX", "id": "5d463c4d4793224a", "name": "Lake Jackson", "place_type": "city", "bounding_box": rectangle("-95.497914,28.976202 -95.366515,29.069629") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4840588, "cityName": "Lake Jackson" } }
+{ "create_at": datetime("2016-01-08T00:00:05.000Z"), "id": 685370375979413504, "text": "I'm disappointed in myself.\nI want to be real, but never listen to my heart when I really need to... \n\nI want people to see me....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2426656224, "name": "Alex", "screen_name": "just_kid_ng", "lang": "en", "location": "null", "create_at": date("2014-04-03"), "description": "jfk '17 | The Wilderness must be explored! KAWH KAWH RAAAAWR | Photography", "followers_count": 171, "friends_count": 177, "statues_count": 1936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370377246085120, "text": "Disney prices are over the head insane, & I'm stuck wanting to go but I'm poor as fuuuuuuu lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 529106934, "name": "Vel.", "screen_name": "nyleve_z", "lang": "en", "location": "SoCal ↕️ NorCal", "create_at": date("2012-03-18"), "description": "So basic I have a pH of 14.", "followers_count": 565, "friends_count": 480, "statues_count": 11107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370377623744512, "text": "I don't chase dreams anymore. \nI HUNT GOALS.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380138250, "name": "Çhïęf Čât", "screen_name": "MatCoran", "lang": "en", "location": "North Tonawanda ", "create_at": date("2011-09-25"), "description": "I live my life a 1/4 mile at a time for those 8 seconds or less im free. UNOH High PO student! #fbodyowner", "followers_count": 523, "friends_count": 455, "statues_count": 37871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370377657122816, "text": "I'm heated I wasn't there, we could have jumped him damn it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2686894260, "name": "im better than u", "screen_name": "47basedjerry", "lang": "en", "location": "where the brew is", "create_at": date("2014-07-28"), "description": "no right turn on red, u turns are not permitted. yield to oncoming traffic.", "followers_count": 684, "friends_count": 731, "statues_count": 29080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370377724366849, "text": "Phone was dead asf for a hot sec", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621652470, "name": "Ginaak", "screen_name": "ginaak15", "lang": "en", "location": "null", "create_at": date("2012-06-28"), "description": "Don't twatch n not follow. That's rude.", "followers_count": 496, "friends_count": 565, "statues_count": 30706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fostoria, OH", "id": "377b44a4ff931d73", "name": "Fostoria", "place_type": "city", "bounding_box": rectangle("-83.460747,41.128097 -83.367515,41.20308") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39147, "countyName": "Seneca", "cityID": 3928014, "cityName": "Fostoria" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370377942376448, "text": "Upper east side lie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4162500020, "name": "Lando Kardashian", "screen_name": "Philteredsoul", "lang": "en", "location": "null", "create_at": date("2015-11-07"), "description": "upset twenty something", "followers_count": 19, "friends_count": 19, "statues_count": 1131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370378152116225, "text": "@Kealoha_253 thankyou ❤", "in_reply_to_status": 685370254776602624, "in_reply_to_user": 961536518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 961536518 }}, "user": { "id": 1396447350, "name": "john william tuilata", "screen_name": "j_marley54", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "||never backwards always forward|COWBOYSNATION (but Cam Newton is my favorite player)| 808•253||", "followers_count": 188, "friends_count": 161, "statues_count": 1535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562109,47.166863 -122.348569,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370378684874752, "text": "���� no", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 506734463, "name": "Tommy Longo", "screen_name": "Tlongo7", "lang": "en", "location": "null", "create_at": date("2012-02-27"), "description": "been fucked over too many times to care", "followers_count": 126, "friends_count": 140, "statues_count": 8379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greece, NY", "id": "00d1c94455339375", "name": "Greece", "place_type": "city", "bounding_box": rectangle("-77.761369,43.178823 -77.615702,43.32408") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3630279, "cityName": "Greece" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370378722521088, "text": "@LifeOfJamo are you fuckin kidding me", "in_reply_to_status": 685367487156875264, "in_reply_to_user": 509111990, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 509111990 }}, "user": { "id": 381782272, "name": "Lucas", "screen_name": "bamitslucas", "lang": "en", "location": "Arizona, USA", "create_at": date("2011-09-28"), "description": "19|Drums|PA/AZ|CRAS", "followers_count": 415, "friends_count": 317, "statues_count": 28039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370379150471168, "text": "Thank you for all your guidance. I learned so much from you, especially at the beginning of my… https://t.co/OSLkR5lcy8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.331,47.6064"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119621455, "name": "aubrey miles", "screen_name": "realaubreymiles", "lang": "en", "location": "philippines", "create_at": date("2010-03-03"), "description": "follow me on Facebook SandelAubrey and Instagram Milesaubrey thanks guys", "followers_count": 166484, "friends_count": 507, "statues_count": 38705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370379309723648, "text": "IM ALWAYS BROKE I CANT COME UP WITH MONEY THAT FAST MAN WHERES THE CHILL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319711277, "name": "tianna", "screen_name": "popcrunk", "lang": "en", "location": "indiana", "create_at": date("2011-06-18"), "description": "just ur average stoned pop punk metalcore alternative hxc alcoholic • 2010 @joejonas & @nickjonas is my aesthetic • check out shellshock tho ⇩", "followers_count": 1080, "friends_count": 546, "statues_count": 26176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, IN", "id": "6181e0f8ac8150e1", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-85.744882,40.49346 -85.61564,40.610902") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18053, "countyName": "Grant", "cityID": 1846908, "cityName": "Marion" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370379389370368, "text": "I honestly feel like crying I'm in so much pain. I think I need a Bionic prosthetic leg.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547459950, "name": "Isaac terry", "screen_name": "Isaaciketaylor", "lang": "en", "location": "Edinburgh Scotland ", "create_at": date("2012-04-06"), "description": "So wise, so handsome, so good at oral sex. It's complicated but yet so simple. Reflection is the better part of a champion.", "followers_count": 191, "friends_count": 252, "statues_count": 3857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lehi, UT", "id": "ed3b23f667186d1f", "name": "Lehi", "place_type": "city", "bounding_box": rectangle("-111.914775,40.357876 -111.818332,40.456264") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4944320, "cityName": "Lehi" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370380026953728, "text": "My sister gave me a whole bottle of moscato to drink tonight on my own.. Wtf is going on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312775465, "name": "Layna", "screen_name": "_laynac", "lang": "en", "location": "null", "create_at": date("2015-08-11"), "description": "20 | SEATTLE", "followers_count": 189, "friends_count": 118, "statues_count": 2251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198394") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370380396158976, "text": "Every. Single. One. https://t.co/xSR10EzzLv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156742714, "name": "#beardfertilizer", "screen_name": "deetwotimess", "lang": "en", "location": "MKE", "create_at": date("2010-06-17"), "description": "fat around my waist right? it's hard to lift me off ya nigga face, right? |full time chubbae and weave connoisseur. | President of #AintShitHive", "followers_count": 1737, "friends_count": 907, "statues_count": 142198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370380526039040, "text": "Some people just don't understand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488175386, "name": "Ethan Morgan", "screen_name": "Ethan_morgan11", "lang": "en", "location": "null", "create_at": date("2012-02-09"), "description": "C", "followers_count": 771, "friends_count": 348, "statues_count": 13937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-01-08T00:00:06.000Z"), "id": 685370380547063808, "text": "@TheWhiteNinja99 https://t.co/qqnVZhFAnR", "in_reply_to_status": 685370005639151616, "in_reply_to_user": 164402402, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 164402402 }}, "user": { "id": 198254316, "name": "Hales", "screen_name": "haleighblegh", "lang": "en", "location": "Edmond, OK", "create_at": date("2010-10-03"), "description": "@VOODOODOLLSOK // Conor's spoiled brat❤️", "followers_count": 326, "friends_count": 429, "statues_count": 5107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370380823867392, "text": "Love taking my makeup off after a long night����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535704854, "name": "LEX", "screen_name": "_lexem", "lang": "en", "location": "null", "create_at": date("2012-03-24"), "description": "moving to VA bc Texas ain't shit #540", "followers_count": 1411, "friends_count": 1104, "statues_count": 86390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381062909954, "text": "Wind 0 mph ---. Barometer 1010.0 hPa, Falling. Temperature 72.1 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 265, "statues_count": 140712 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Miradero, Puerto Rico", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381105008640, "text": "Can you find Edmonton on the map? Just try it at https://t.co/dE5aI3DtoI #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.49,53.5436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Edmonton" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 983, "friends_count": 312, "statues_count": 2518146 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381306302464, "text": "@wzee_ ����", "in_reply_to_status": 685149680259342336, "in_reply_to_user": 516022142, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 516022142 }}, "user": { "id": 516022142, "name": "SpeakUPZee", "screen_name": "wzee_", "lang": "en", "location": "SpeakUPZee ", "create_at": date("2012-03-05"), "description": "Will Zee' Tidwell | Writer | Public Speaker |", "followers_count": 1376, "friends_count": 1250, "statues_count": 61135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381528506368, "text": "@PDLMT what? Lmao", "in_reply_to_status": 685370068784386048, "in_reply_to_user": 531086844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 531086844 }}, "user": { "id": 4020620534, "name": "Alex", "screen_name": "alexxg21", "lang": "en", "location": "Colton, CA", "create_at": date("2015-10-25"), "description": "Manchester United | B | Broncos | 6OD | Lakers | THANK YOU FOR EVERYTHING KOBE", "followers_count": 264, "friends_count": 292, "statues_count": 11386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381910315008, "text": "@SteeezzyD ily bbys", "in_reply_to_status": -1, "in_reply_to_user": 618543277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 618543277 }}, "user": { "id": 168962723, "name": "broda", "screen_name": "TheOnlyGomie", "lang": "en", "location": "Shooting Range, New Mexico", "create_at": date("2010-07-20"), "description": "You have the love of humanity in your hearts You don't hate, only the unloved hate, the unloved and the unnatural. #HxC #NMSU", "followers_count": 492, "friends_count": 200, "statues_count": 36521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlsbad, NM", "id": "8e8013ea8726166e", "name": "Carlsbad", "place_type": "city", "bounding_box": rectangle("-104.280246,32.3175 -104.198708,32.458305") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35015, "countyName": "Eddy", "cityID": 3512150, "cityName": "Carlsbad" } }
+{ "create_at": datetime("2016-01-08T00:00:07.000Z"), "id": 685370381926940672, "text": "we met for a reason, either you’re a blessing or a lesson.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2309117671, "name": "its just lydia", "screen_name": "bombdotccom", "lang": "en", "location": "Los Angeles,California ", "create_at": date("2014-01-24"), "description": "you still hit my phone up", "followers_count": 633, "friends_count": 508, "statues_count": 3622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732739362983936, "text": "�� wish I was asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2896428125, "name": "G.INV", "screen_name": "briaasianee", "lang": "en", "location": "null", "create_at": date("2014-11-28"), "description": "pray. stack. build. grow.✨♉️✨", "followers_count": 234, "friends_count": 178, "statues_count": 7198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732739425775616, "text": "Niahoo https://t.co/Xme4pxPr6R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 620435642, "name": "Chilly", "screen_name": "_shaniarene", "lang": "en", "location": "Houston, TX", "create_at": date("2012-06-27"), "description": "TxSU'19 Athletic Trainer||sc:Niahoo", "followers_count": 429, "friends_count": 340, "statues_count": 6473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732739786489857, "text": "All texts sent simultaneously https://t.co/J0RqcZSIzH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24327807, "name": "ℓana", "screen_name": "retsoflana", "lang": "fr", "location": "null", "create_at": date("2009-03-13"), "description": "Fun Creation Specialist", "followers_count": 389, "friends_count": 210, "statues_count": 25181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732739903930368, "text": "I'm starting to notice the people who I'll be giving back ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2915794032, "name": "Abbas Alugla", "screen_name": "therealabosee", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "life's too short to drive boring cars.", "followers_count": 281, "friends_count": 300, "statues_count": 1468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blaine, MN", "id": "7f659be5193a576e", "name": "Blaine", "place_type": "city", "bounding_box": rectangle("-93.267565,45.118886 -93.142602,45.21178") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2706382, "cityName": "Blaine" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732740067540992, "text": "Yo @Rampage4real you're one of my idols! Love how you're always real. You've followed me for years on Twitter and it's awesome. Thank you!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38974889 }}, "user": { "id": 324897401, "name": "Jason Cayce", "screen_name": "JasonCayce", "lang": "en", "location": "Lafayette, Colorado", "create_at": date("2011-06-27"), "description": "I play hockey, and train MMA. Love to work out, cook and watch sports!", "followers_count": 52, "friends_count": 38, "statues_count": 973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CO", "id": "3f871475c095f94f", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-105.148925,39.819133 -104.987771,39.97944") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 883835, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732740285775873, "text": "#Hawaii\n#Oahu\n#West_beach\n#Beach\n#Private_beach\n#The_sea\n#Private… https://t.co/hAIc9uagRW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.91879391,21.33112535"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Hawaii", "Oahu", "West_beach", "Beach", "Private_beach", "The_sea", "Private" }}, "user": { "id": 154368152, "name": "まみっくす", "screen_name": "mxaxmxix", "lang": "ja", "location": "null", "create_at": date("2010-06-10"), "description": "null", "followers_count": 99, "friends_count": 81, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732740646305792, "text": "Does roscoes or guppies really live up to the hype. I've never been but I really wanna drive up there to try lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1447256300, "name": "raymond escalona", "screen_name": "damnraymond", "lang": "en", "location": "Hogwarts", "create_at": date("2013-05-21"), "description": "Pizza is life. | #CodeEleven | Do what you love. Dream big. I like pizza. | Filmmaker-ish |", "followers_count": 506, "friends_count": 477, "statues_count": 21641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732740742840320, "text": "����☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 206511431, "name": "Sammy", "screen_name": "SamanthaSocal", "lang": "en", "location": "San Francisco", "create_at": date("2010-10-22"), "description": "I'm good at coloring | Face your fears | SFSU'19", "followers_count": 689, "friends_count": 331, "statues_count": 7341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732741250330624, "text": "Ty dolla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 3280785360, "name": "andrew", "screen_name": "nvstydrew", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2015-07-15"), "description": "null", "followers_count": 228, "friends_count": 186, "statues_count": 2444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Jose Hills, CA", "id": "945d4c47ec5036f9", "name": "South San Jose Hills", "place_type": "city", "bounding_box": rectangle("-117.923259,34.003576 -117.88006,34.026435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673290, "cityName": "South San Jose Hills" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732741275582464, "text": "Wind 0.9 mph SW. Barometer 29.829 in, Rising Rapidly. Temperature 56.6 °F. Rain today 0.19 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732741632016384, "text": "The past 24 hours have been a giant WTF...��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 938293321, "name": "Tay", "screen_name": "tay_tay1729", "lang": "en", "location": "bhs senior ", "create_at": date("2012-11-09"), "description": "MA//AZ @ElijahLanderos❤️", "followers_count": 870, "friends_count": 704, "statues_count": 19664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.635016,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732741682491393, "text": "https://t.co/qkCnSxZ0ht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4220518635, "name": "OrlandoAmilianmiami", "screen_name": "OrlandoMilian2", "lang": "en", "location": "null", "create_at": date("2015-11-18"), "description": "null", "followers_count": 498, "friends_count": 2140, "statues_count": 32435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732742617640960, "text": "@AnyaLasagna11 ��������������������", "in_reply_to_status": 685732166467108864, "in_reply_to_user": 1348408405, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1348408405 }}, "user": { "id": 593919148, "name": "Mercy", "screen_name": "MercedesC97", "lang": "en", "location": "Denton, TX", "create_at": date("2012-05-29"), "description": "[Everything you've ever wanted is on the other side of fear -George Addair]\n \n#UNT19 #BeAn", "followers_count": 242, "friends_count": 296, "statues_count": 351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-01-09T00:00:00.000Z"), "id": 685732743041290240, "text": "February 2012 https://t.co/ufNAxZXqks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 336136086, "name": "Kendra", "screen_name": "_k3ndra", "lang": "en", "location": "Missouri, USA", "create_at": date("2011-07-15"), "description": "null", "followers_count": 1258, "friends_count": 913, "statues_count": 22157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castle Point, MO", "id": "adaac16acec35c6c", "name": "Castle Point", "place_type": "city", "bounding_box": rectangle("-90.260638,38.74924 -90.241672,38.770845") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2911908, "cityName": "Castle Point" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732743641214977, "text": "01/09@03:00 - Temp 42.5F, WC 42.5F. Wind 1.5mph NNE, Gust 4.0mph. Bar 30.159in, Falling slowly. Rain 0.00in. Hum 86%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732743699804160, "text": "This #Sales #job might be a great fit for you: Retail Cosmetics Sales - Counter Manager Lancome, Full Time:... - https://t.co/LQMEhwkso5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1751186,38.9238131"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job" }}, "user": { "id": 2252314255, "name": "Macy's Beauty", "screen_name": "macysbeautyJOBS", "lang": "en", "location": "Nationwide", "create_at": date("2013-12-18"), "description": "Bring your passion for make up and skin care to Macy's - a beautiful way to invest your talents and build your career!", "followers_count": 686, "friends_count": 0, "statues_count": 8331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alton, IL", "id": "5823c29f3d3e543d", "name": "Alton", "place_type": "city", "bounding_box": rectangle("-90.21392,38.869155 -90.088275,38.933265") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1701114, "cityName": "Alton" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744043827201, "text": "03:00:03 |Temp: 47.9ºF | Wind Chill 47.9ºF |Dew Point 45.1ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NNE, Gusting to 1.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744207425536, "text": "@PoliceAreGood22 @teamdenmom @DavidRamirez170 If you do it is likely that it will be the end of it for you @teamdenmom", "in_reply_to_status": 685731825163972608, "in_reply_to_user": 1479936330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1479936330, 4704907326, 4730997235, 4704907326 }}, "user": { "id": 64579004, "name": "Charlie Grapski", "screen_name": "cgrapski", "lang": "en", "location": "Kissimmee, FL, USA", "create_at": date("2009-08-10"), "description": "null", "followers_count": 746, "friends_count": 1049, "statues_count": 11588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buenaventura Lakes, FL", "id": "01cbd58e76f11140", "name": "Buenaventura Lakes", "place_type": "city", "bounding_box": rectangle("-81.383233,28.318719 -81.319774,28.348175") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1209415, "cityName": "Buenaventura Lakes" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744366690304, "text": "Just pok'n around. #DailyHBPierPhoto #HuntingtonBeach @ Huntington Beach Pier. Northside https://t.co/bTYLdQFERq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.00340844,33.65672046"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DailyHBPierPhoto", "HuntingtonBeach" }}, "user": { "id": 510917447, "name": "Ed Templeton", "screen_name": "TempletonEd", "lang": "en", "location": "null", "create_at": date("2012-03-01"), "description": "information:", "followers_count": 11595, "friends_count": 845, "statues_count": 12233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744379289600, "text": "Everybody wants to rule the world", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1406729875, "name": "jesse", "screen_name": "jesse_perez21", "lang": "en", "location": "null", "create_at": date("2013-05-05"), "description": "null", "followers_count": 560, "friends_count": 221, "statues_count": 11725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stanton, CA", "id": "35fea2c31d822962", "name": "Stanton", "place_type": "city", "bounding_box": rectangle("-118.017331,33.773894 -117.96762,33.81573") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 673962, "cityName": "Stanton" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744521908225, "text": "@seankennedyDC @CNNOpinion I got news for ya,the GOPe needs to be destroyed.Actually both parties are complicit in screwing the American ppl", "in_reply_to_status": 685106423030743040, "in_reply_to_user": 251768944, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251768944, 259074538 }}, "user": { "id": 3941880252, "name": "Trudge", "screen_name": "Trudginon1", "lang": "en", "location": "Joined Twitter October 2015", "create_at": date("2015-10-18"), "description": "Patriot. America Firster. Hobbies:Seeking Truth,Fighting the NWO.Loves:America,Liberty,The Constitution,The Founders,The 2nd Amendment ALL IN FOR TRUMP.", "followers_count": 295, "friends_count": 316, "statues_count": 2526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744572366848, "text": "They gon talk about all this other stuff they do for them though. Like use their money and whip https://t.co/4piaPrTYQ2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129628411, "name": "kdolla$", "screen_name": "Kashdra", "lang": "en", "location": "New York, USA", "create_at": date("2010-04-04"), "description": "Public Policy major and Business Administration minor at SUNY Albany. RIP Blake", "followers_count": 5521, "friends_count": 1903, "statues_count": 140230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744593281024, "text": "@xodeine that shit sucks", "in_reply_to_status": 685732648669581312, "in_reply_to_user": 346905764, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346905764 }}, "user": { "id": 314843055, "name": "Shaine", "screen_name": "SHAINEAVELI", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-06-10"), "description": "die", "followers_count": 1093, "friends_count": 396, "statues_count": 87643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744668643328, "text": "Extra clothes and a blanket �� https://t.co/rrGtqYto8j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581790587, "name": "Est Unicum", "screen_name": "BrendenMalone22", "lang": "en", "location": "Tioga, Louisiana", "create_at": date("2012-05-16"), "description": "Tioga ✈️ NOLA", "followers_count": 383, "friends_count": 387, "statues_count": 3932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ball, LA", "id": "00d6437870509ca6", "name": "Ball", "place_type": "city", "bounding_box": rectangle("-92.48291,31.370476 -92.392485,31.435872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2204055, "cityName": "Ball" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744735911936, "text": "I been hearin chaos & sirens for the last 15 minutes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056595694, "name": "KP", "screen_name": "KzooPlayer", "lang": "en", "location": "Chicago-Kzoo ", "create_at": date("2013-01-02"), "description": "Walk with these Jordan 3's you a be Aaight Slick. #Love4sole", "followers_count": 1039, "friends_count": 709, "statues_count": 70142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732744832274432, "text": "@ me @AprilWilmink https://t.co/s7qm14Df8h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4639878072 }}, "user": { "id": 919472312, "name": "babygirl.", "screen_name": "princess_shaeee", "lang": "en", "location": "rip jeremiah❤️", "create_at": date("2012-11-01"), "description": "@ericaajeaan is my girlfriend. not really but she is the only person that loves me 24/7❤️ @skatemaloley", "followers_count": 852, "friends_count": 948, "statues_count": 32578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732745075671041, "text": "Wind 0 mph --. Barometer 29.92 in, Steady. Temperature 55.4 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732745683808257, "text": "@AlphaOmegaNinja @KGPrestige @Alpha_Eevee Masuda Squad thoooo", "in_reply_to_status": 685731923998539776, "in_reply_to_user": 618449076, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 618449076, 2531138922, 3161760439 }}, "user": { "id": 511030602, "name": "JPRPokeTrainer98", "screen_name": "JPRPT98", "lang": "en", "location": "Winston-Salem, North Carolina", "create_at": date("2012-03-01"), "description": "Hi, I'm John, and I make Pokemon videos on YouTube. Guess that makes me a PokeTuber. Pepsi, King of the Hill, Bob Ross and Ford Mustangs. #JPaRmy", "followers_count": 1189, "friends_count": 111, "statues_count": 15755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.933734 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732745696292864, "text": "Want to work at Mercy Health? We're #hiring in #FortSmith, AR! Click for details: https://t.co/hPCeOPNzHP #Healthcare #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.3985475,35.3859242"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "FortSmith", "Healthcare", "Job", "Jobs" }}, "user": { "id": 1898258479, "name": "Mercy Jobs", "screen_name": "MercyJobs", "lang": "en", "location": "null", "create_at": date("2013-09-23"), "description": "When you join Mercy, you are part of a community that is embracing a new way to care for people while building your skills and growing in your career.", "followers_count": 309, "friends_count": 4, "statues_count": 4159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Smith, AR", "id": "a5cb06fbef643420", "name": "Fort Smith", "place_type": "city", "bounding_box": rectangle("-94.435514,35.256048 -94.305102,35.438472") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5131, "countyName": "Sebastian", "cityID": 524550, "cityName": "Fort Smith" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732745734037504, "text": "@_Sallelujah call me girl right nooow", "in_reply_to_status": 685732191012179968, "in_reply_to_user": 2934334500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2934334500 }}, "user": { "id": 307962858, "name": "nobody", "screen_name": "Uncmn_MD", "lang": "en", "location": "Uncmn Universe ", "create_at": date("2011-05-30"), "description": "Myself just told myself you the motha fucking man you don't need no help. ♊", "followers_count": 475, "friends_count": 288, "statues_count": 18560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732746099060736, "text": "Wind 1.0 mph WNW. Barometer 29.880 in, Steady. Temperature 43.1 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 118, "friends_count": 0, "statues_count": 110192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732746296045568, "text": "I was sexually molested as a child, not by the man who took this photo (who was the greatest father in the world) but by another relative", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 98591051, "name": "Anthony Flores", "screen_name": "antflores01", "lang": "en", "location": "null", "create_at": date("2009-12-22"), "description": "null", "followers_count": 3, "friends_count": 16, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raymondville, TX", "id": "928876e567c2c238", "name": "Raymondville", "place_type": "city", "bounding_box": rectangle("-97.79772,26.459437 -97.760062,26.49294") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48489, "countyName": "Willacy", "cityID": 4860836, "cityName": "Raymondville" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732746627538944, "text": "0h 45m wait time at University of Alberta Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.520907,53.520659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 370, "friends_count": 888, "statues_count": 2560 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-09T00:00:01.000Z"), "id": 685732746920984576, "text": "How one single person can make me smile simply by existing just amazes me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90809724, "name": "†aylor", "screen_name": "BebeTehler", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-11-17"), "description": "Doin' good ⚓️", "followers_count": 1010, "friends_count": 781, "statues_count": 14793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732747738923009, "text": "@ashhlaayy_ ur not funny and we are not friends", "in_reply_to_status": 685732660598059008, "in_reply_to_user": 472505535, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472505535 }}, "user": { "id": 35874334, "name": "alex", "screen_name": "alexstoller", "lang": "en", "location": "null", "create_at": date("2009-04-27"), "description": "the buck stops here", "followers_count": 1787, "friends_count": 635, "statues_count": 12411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732748049399808, "text": "Wind 0.0 mph SW. Barometer 29.915 in, Rising slowly. Temperature 47.8 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 39, "friends_count": 25, "statues_count": 16727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732748355452928, "text": "this sums up my life https://t.co/lc6UHephgX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2742257145, "name": "al", "screen_name": "garciaalli_", "lang": "en", "location": "null", "create_at": date("2014-08-13"), "description": "RMHS '16", "followers_count": 202, "friends_count": 142, "statues_count": 2746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732748603092993, "text": "Tonight was EXACTLY what I needed. I haven't had that much fun in forever. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434935421, "name": "Chris Carter", "screen_name": "iBlackzilla", "lang": "en", "location": "Marshall University, USA", "create_at": date("2011-12-12"), "description": "Senior. ΑΣΦ.", "followers_count": 1972, "friends_count": 1035, "statues_count": 46999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington, WV", "id": "e4197a23034fa912", "name": "Huntington", "place_type": "city", "bounding_box": rectangle("-82.530433,38.375981 -82.349236,38.439347") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54011, "countyName": "Cabell", "cityID": 5439460, "cityName": "Huntington" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732748703608833, "text": "@savvvvmaster is mean to me ��", "in_reply_to_status": -1, "in_reply_to_user": 4712306984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4712306984 }}, "user": { "id": 1916410806, "name": "ryan vela", "screen_name": "rjv234", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2013-09-29"), "description": "on my journey to be a marine.", "followers_count": 355, "friends_count": 314, "statues_count": 1865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732749412544512, "text": "@__naewop you didn't earlier ��������", "in_reply_to_status": 685732435301150720, "in_reply_to_user": 1123544886, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1123544886 }}, "user": { "id": 881071980, "name": "Real Nigga", "screen_name": "TheLil_Nigga", "lang": "en", "location": "Buffalo, NY", "create_at": date("2012-10-14"), "description": "If I lose my cool I can turn enemies to memories.. Fuck them niggas tho, Free my brothers, fuck the law", "followers_count": 702, "friends_count": 396, "statues_count": 93605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732749907496960, "text": "We was young and we was dumb but we had heart..\n\n#missing the #homies #RIP #eastbay #cali\n\nForever I think of you all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "missing", "homies", "RIP", "eastbay", "cali" }}, "user": { "id": 321013676, "name": "Corey Larue", "screen_name": "cmlarue", "lang": "en", "location": "Boston, MA", "create_at": date("2011-06-20"), "description": "Boston based artist.\nMFA candidate at Boston University.\nOriginally from Nor Cal.", "followers_count": 481, "friends_count": 738, "statues_count": 5508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750066778112, "text": "Please watch \"Rumble Fish.\" #Coppola It changes almost everything.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Coppola" }}, "user": { "id": 15928705, "name": "Jim Anderson, CFRE", "screen_name": "GoalBustersJim", "lang": "en", "location": "null", "create_at": date("2008-08-20"), "description": "#Nonprofit Philanthropic Champion for progressive causes & public/community Radio & TV, @GoalBusters Partner, @AFPNAZ Past President, Passionate Foodie & Photog", "followers_count": 4146, "friends_count": 3062, "statues_count": 18162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750263885824, "text": "Naruto still could've dropped him https://t.co/ndVg8TEQLw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542872044, "name": "austin", "screen_name": "FromDowdyToYou_", "lang": "en", "location": "561", "create_at": date("2012-04-01"), "description": "@trialnderror is the team. give it everything you got and then some.", "followers_count": 1777, "friends_count": 404, "statues_count": 72318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sachse, TX", "id": "418df753bbcfdcfb", "name": "Sachse", "place_type": "city", "bounding_box": rectangle("-96.616514,32.941501 -96.556985,33.004633") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4864064, "cityName": "Sachse" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750733762560, "text": "Somebody wake me up at 6:00!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162735428, "name": "GC❄️", "screen_name": "_GJH10_", "lang": "en", "location": "null", "create_at": date("2010-07-04"), "description": "Rip Gustin & Tahje..", "followers_count": 1012, "friends_count": 625, "statues_count": 19246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750851051522, "text": "I'm gonna be sore af tomorrow ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 968388751, "name": "miki ♡", "screen_name": "mikaylatesoro", "lang": "en", "location": "Los Angeles, California", "create_at": date("2012-11-24"), "description": "❁ ♛ ☮ ☆ ♧ ❥ ☼ ☪ ♬ ✞ ✯ ❃ ☢ ☣ ☠ ☻ ❂ 16 | POLAHS Senior | Dance | j.a. ❣#LongLiveSarah", "followers_count": 275, "friends_count": 236, "statues_count": 6150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750867980288, "text": "@_mothermiaa wanna text?", "in_reply_to_status": 685732117125398528, "in_reply_to_user": 340775730, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 340775730 }}, "user": { "id": 176897786, "name": "Visionary.", "screen_name": "Knapz_NYC", "lang": "en", "location": "NYC ", "create_at": date("2010-08-10"), "description": "Her pussy OPP and she's a thotty by nature.", "followers_count": 18900, "friends_count": 13526, "statues_count": 123441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750947659776, "text": "Back and I'm l better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2221924330, "name": "Ethan Babin", "screen_name": "ethanbabin21", "lang": "en", "location": "null", "create_at": date("2013-12-12"), "description": "don't be dishonest", "followers_count": 871, "friends_count": 706, "statues_count": 14191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzales, LA", "id": "82a3427fa492ed52", "name": "Gonzales", "place_type": "city", "bounding_box": rectangle("-90.959148,30.167772 -90.820958,30.292323") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2229850, "cityName": "Gonzales" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732750985400322, "text": "@PeteyG13 and tomatoes?", "in_reply_to_status": 685732596777676800, "in_reply_to_user": 317829138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 317829138 }}, "user": { "id": 44239450, "name": "Johnny Truelove", "screen_name": "MiKeBoTz_", "lang": "en", "location": "New York, NY", "create_at": date("2009-06-02"), "description": "♍\n\niG: MikeBotz_ \nSnapchat: Mike_botz", "followers_count": 1095, "friends_count": 582, "statues_count": 33328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751023054848, "text": "Omg it's getting real ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391028854, "name": "Jan 9th ✨", "screen_name": "Taaylorr_W", "lang": "en", "location": "Corona, CA ☀️", "create_at": date("2011-10-14"), "description": "insta: tayalexandriaa sc: taaylorr_w", "followers_count": 418, "friends_count": 321, "statues_count": 6239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751039934464, "text": "55.1F (Feels: 55.1F) - Humidity: 99% - Wind: 3.8mph E - Gust: 6.0mph - Pressure: 1019.1mb - Rain: 1.31\" #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 221667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751228579840, "text": "@PostMalone RE... SPECT... WALK", "in_reply_to_status": 685731113981980672, "in_reply_to_user": 913812620, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 913812620 }}, "user": { "id": 181262118, "name": "Jimmy Foster", "screen_name": "JlMMMY", "lang": "en", "location": "null", "create_at": date("2010-08-21"), "description": "sc: jimmyfoster44 don't call me chris but it's hurricane season!", "followers_count": 313, "friends_count": 238, "statues_count": 10615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leominster, MA", "id": "c4f1830ea4b8caaf", "name": "Leominster", "place_type": "city", "bounding_box": rectangle("-71.813501,42.4762 -71.702186,42.573956") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2535075, "cityName": "Leominster" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751228588034, "text": "Midnight sights", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3841111633, "name": "فخ سيد", "screen_name": "trapnigga_ricky", "lang": "en", "location": "null", "create_at": date("2015-10-09"), "description": "Intoxicated, animated got me feelin kinda lit. Tough times don't last, Tough people do.", "followers_count": 121, "friends_count": 409, "statues_count": 851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751371182080, "text": "my mind is on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 706028107, "name": "savvy moon", "screen_name": "SavannahLeaMoon", "lang": "en", "location": "probably with katie", "create_at": date("2012-07-19"), "description": "•US Freestyle Figure Skating⛸•class of 2017•lv ks•chicken fanatic•i like to embarrass myself•", "followers_count": 642, "friends_count": 519, "statues_count": 4765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leavenworth, KS", "id": "70d4570609cf6e35", "name": "Leavenworth", "place_type": "city", "bounding_box": rectangle("-94.965076,39.266719 -94.89041,39.375576") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20103, "countyName": "Leavenworth", "cityID": 2039000, "cityName": "Leavenworth" } }
+{ "create_at": datetime("2016-01-09T00:00:02.000Z"), "id": 685732751669129217, "text": "Temp: 54.2°F Wind:0.0mph Pressure: 29.878hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732751887200256, "text": "Temp: 62.1°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.88 in | Dewpoint: 61.8°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 23, "friends_count": 1, "statues_count": 164329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732752042422272, "text": "Wind 0.0 mph SE. Barometer 30.21 in, Falling slowly. Temperature 36.7 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 27, "friends_count": 90, "statues_count": 156532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732752147103744, "text": "You gotta get with it or get lost", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1149510792, "name": "Hennessy Rodriguez", "screen_name": "hennyisturbo", "lang": "en", "location": "null", "create_at": date("2013-02-04"), "description": "Sotomayor HS", "followers_count": 353, "friends_count": 432, "statues_count": 17104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732752814030848, "text": "@DwyaneWade If you ask me who is the best, I always say that you are.�� you go to visit Nicaragua?��", "in_reply_to_status": -1, "in_reply_to_user": 33995409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33995409 }}, "user": { "id": 2180791153, "name": "→Bianka Paola←", "screen_name": "paolitaBFR", "lang": "es", "location": "Estados Unidos", "create_at": date("2013-11-07"), "description": "→Amando infinitamente a mi País, Nic❤\n✨18/03 NL✨\n In a relationship with basketball, Fanatic.", "followers_count": 462, "friends_count": 223, "statues_count": 11702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732752960925697, "text": "Hold Me Like You May Never See Me Again ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 398398302, "name": "♦King♦", "screen_name": "KingVardo", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-10-25"), "description": "❗I Don't Trust A Soul❗", "followers_count": 500, "friends_count": 249, "statues_count": 13210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732752977735680, "text": "Don't want. Trust me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 452776441, "name": "Austin Meyer", "screen_name": "AustinMeyer20", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "Franklin | UWW '18", "followers_count": 368, "friends_count": 196, "statues_count": 2676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, WI", "id": "e0fab424bd189e61", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-88.069621,42.858079 -87.949493,42.93053") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5527300, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732753023713280, "text": "BITCH https://t.co/x4ibq3ricu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1657166498, "name": "mara.❥", "screen_name": "aaaramas", "lang": "en", "location": "sallad, saxet.", "create_at": date("2013-08-09"), "description": "snap: samtharippa", "followers_count": 647, "friends_count": 469, "statues_count": 17615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732753061449729, "text": "I just got asked if my name was safari.............", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319328395, "name": "Shafer Fleetwood", "screen_name": "realslimshafie", "lang": "en", "location": "null", "create_at": date("2011-06-17"), "description": "Edmond UCO", "followers_count": 1881, "friends_count": 840, "statues_count": 45107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732753623486464, "text": "Making a Murderer in a nutshell: \"The victim said the perpetrator was wearing white underwear & Steven Avery didn't even own underwear.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3152148902, "name": "Amanda Towner", "screen_name": "amandattm", "lang": "en", "location": "Mesa, AZ", "create_at": date("2015-04-12"), "description": "Constantly finding myself surrounded by dunces.", "followers_count": 22, "friends_count": 18, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732753636245505, "text": "Wind 6.0 mph SE. Barometer 30.142 in, Steady. Temperature 36.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 5676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732753837436928, "text": "Im the most elderly teen ever I literally crocheted for over 5 hours what why", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1927332542, "name": "kenz", "screen_name": "kenzieshea521", "lang": "en", "location": "null", "create_at": date("2013-10-02"), "description": "null", "followers_count": 244, "friends_count": 325, "statues_count": 1135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queen Creek, AZ", "id": "01cb573821d94344", "name": "Queen Creek", "place_type": "city", "bounding_box": rectangle("-111.686314,33.196614 -111.582748,33.288127") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 458150, "cityName": "Queen Creek" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754005299201, "text": "My sweet sweet boo thang ���� https://t.co/SAM5HWTTtN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32824884, "name": "MAISexy Nurse RN/BSN", "screen_name": "NurseKorlu", "lang": "en", "location": "ÜT: 39.31393,-76.70675", "create_at": date("2009-04-18"), "description": "DIVA, QUEEN, NURSE, SEX THERAPIST, MOTHER, FUTURE MILLIONAIRE, #TeamLIB", "followers_count": 1035, "friends_count": 456, "statues_count": 74135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754147901440, "text": "I'm at City of Chicago in Chicago, IL https://t.co/9L64RYs33P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.62779,41.88206"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137435857, "name": "Dwan A. Russell", "screen_name": "dwanarussell", "lang": "en", "location": "United States", "create_at": date("2010-04-26"), "description": "null", "followers_count": 205, "friends_count": 557, "statues_count": 5305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754206666752, "text": "@ben_techpro Where GOV lets free men gain KEEP use trade or dispose of what they freely acquire, there is LIBERTY https://t.co/z6f4Ct0eQ0", "in_reply_to_status": -1, "in_reply_to_user": 525954482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525954482 }}, "user": { "id": 1497568382, "name": "Jerome Huyler, PhD.", "screen_name": "huylerje", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2013-06-09"), "description": "Fmr Asst Professor Seton Hall University. Author: Locke in America: The Moral Philosophy of the Founding Era and Everything You Have: The Case Against Welfare.", "followers_count": 4484, "friends_count": 4936, "statues_count": 51060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754256875520, "text": "I need to call her again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635906353, "name": "Stephanie", "screen_name": "gsteph017", "lang": "en", "location": "null", "create_at": date("2012-07-14"), "description": "J \n3515", "followers_count": 589, "friends_count": 460, "statues_count": 25721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754382663681, "text": "Not a huge country music fan, however watched @kramergirl Live from Nashville tonight. Holy Cow! gorgeous voice & gorgeous person.\n#newfan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "newfan" }}, "user_mentions": {{ 24291261 }}, "user": { "id": 1717976065, "name": "Gil Negrete", "screen_name": "bpademarini2017", "lang": "en", "location": "Orange County", "create_at": date("2013-08-31"), "description": "null", "followers_count": 149, "friends_count": 147, "statues_count": 1223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, CA", "id": "3a16272f05577d7e", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-117.72136,33.606781 -117.621862,33.685914") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639496, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754416254977, "text": "Maybe just a one way ticket to Montana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48392989, "name": "Karsten", "screen_name": "KarstenHenry", "lang": "en", "location": "null", "create_at": date("2009-06-18"), "description": "20. Umpqua, OR..", "followers_count": 65, "friends_count": 102, "statues_count": 2640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseburg, OR", "id": "22d56ab4bcf65a60", "name": "Roseburg", "place_type": "city", "bounding_box": rectangle("-123.398205,43.192228 -123.295953,43.259091") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41019, "countyName": "Douglas", "cityID": 4163650, "cityName": "Roseburg" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754454020096, "text": "My brother dropped is phone in water & now he's relying on me to fix it ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488538555, "name": "Tatiana LaMantia", "screen_name": "TatianaLaMantia", "lang": "en", "location": "null", "create_at": date("2012-02-10"), "description": "university of arizona", "followers_count": 755, "friends_count": 491, "statues_count": 8070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casa Grande, AZ", "id": "fbb3d1e41acab043", "name": "Casa Grande", "place_type": "city", "bounding_box": rectangle("-111.791608,32.858246 -111.670779,32.992892") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 410530, "cityName": "Casa Grande" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754604949504, "text": "Happy birthday to @melanysnn one of the best people in the world ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1549928156 }}, "user": { "id": 323288145, "name": "Daniel", "screen_name": "kid_vito10", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "null", "followers_count": 608, "friends_count": 589, "statues_count": 47942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732754680446976, "text": "@ashleeey007 I'm amazing I know ����", "in_reply_to_status": 685732515814883329, "in_reply_to_user": 2929011578, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2929011578 }}, "user": { "id": 33252266, "name": "Lex", "screen_name": "sexxiilexxii_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-19"), "description": "loving mine.", "followers_count": 1169, "friends_count": 437, "statues_count": 22777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755133624320, "text": "Someone take me to get icecream from Walgreens", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521661506, "name": "bubbleisha", "screen_name": "notcute_", "lang": "en-gb", "location": "♡ pussy ♡ ", "create_at": date("2012-03-11"), "description": "if folks wanna pop off", "followers_count": 386, "friends_count": 167, "statues_count": 30917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-88.446362,32.284593 -84.296982,41.744901") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755255111680, "text": "@_IVargas14 @WaR_Czech join call izzy", "in_reply_to_status": 685732046954614784, "in_reply_to_user": 1392595261, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1392595261, 1457920375 }}, "user": { "id": 700562282, "name": "Kevin Pichardo", "screen_name": "WaR_Surge", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-07-17"), "description": "Captain of Team WaR | CSULB Finance Major |#TheMoneyTeam | Road to CoD Champs Begins | Heroes Get Remembered But Legends Never Die", "followers_count": 424, "friends_count": 440, "statues_count": 14726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755414450176, "text": "Never get taken serious smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 149770965, "name": "eduardo", "screen_name": "BigEddie_O", "lang": "en", "location": "null", "create_at": date("2010-05-29"), "description": "Fontana CA.", "followers_count": 258, "friends_count": 211, "statues_count": 26391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755473219584, "text": "F.$.S.♥️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3242523492, "name": "The Prodigal Son", "screen_name": "_mrrain3", "lang": "en", "location": "null", "create_at": date("2015-06-11"), "description": "null", "followers_count": 154, "friends_count": 199, "statues_count": 1114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755536084992, "text": "HAPPY BIRTHDAY LOSER������������������������������❣ https://t.co/EhDMQL5KPk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2229163674, "name": "Lindsss", "screen_name": "Linddddddsay", "lang": "en", "location": "Elk Grove, CA", "create_at": date("2013-12-03"), "description": "ΔΓH", "followers_count": 712, "friends_count": 533, "statues_count": 7021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-01-09T00:00:03.000Z"), "id": 685732755917815810, "text": "karate chop goes wayyyyy back haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 946254541, "name": "lucenda ^◡^", "screen_name": "lucendaplunkett", "lang": "en", "location": "htx", "create_at": date("2012-11-13"), "description": "✵ ⊹ . · . ⋆ ⊹ * @kaskade ❥ ✵ ⊹ . · . ⋆ ⊹ *", "followers_count": 1781, "friends_count": 1394, "statues_count": 52107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732756182142978, "text": "@ivethbabes say that", "in_reply_to_status": 685732712347471872, "in_reply_to_user": 998726286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 290352022 }}, "user": { "id": 998726286, "name": "Kenneth", "screen_name": "Join_llluminati", "lang": "en", "location": "null", "create_at": date("2012-12-08"), "description": "MSU's Finest Graduate | Lab Technician | NY/NJ", "followers_count": 106, "friends_count": 103, "statues_count": 3337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Lawn, NJ", "id": "d9da081322efe6e7", "name": "Fair Lawn", "place_type": "city", "bounding_box": rectangle("-74.148182,40.917486 -74.089421,40.955019") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3422470, "cityName": "Fair Lawn" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732756693889025, "text": "You're too good https://t.co/JozYKdBx7N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 738280860, "name": ":)", "screen_name": "PaulTweets2Much", "lang": "en", "location": "Queens, NY", "create_at": date("2012-08-05"), "description": "i Tweet Too Much | I'll probably upset you at some point | ❤️@PaulTweets2Much❤️ | i had sexual intercourse with Tom Brady & gave birth to @hsmitty3 | J❤️", "followers_count": 8621, "friends_count": 836, "statues_count": 304941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732757255905280, "text": "#teampaparatzi #teamandroid #nolive @ Club Crucial https://t.co/veqYeKbATJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.08330954,32.92484978"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "teampaparatzi", "teamandroid", "nolive" }}, "user": { "id": 411954211, "name": "Papa Ratzi Will", "screen_name": "PapaRatzi83", "lang": "en", "location": " Charleston,SC", "create_at": date("2011-11-13"), "description": "Family First, Online Business Building Specialist Former Soldier Turned Photographer/Travel Agent,Disabled Veteran", "followers_count": 1207, "friends_count": 804, "statues_count": 6624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Charleston, SC", "id": "18df13e4a5a670b5", "name": "North Charleston", "place_type": "city", "bounding_box": rectangle("-80.15759,32.829336 -79.934288,32.999393") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4550875, "cityName": "North Charleston" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732757276733440, "text": "Perfect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 903029726, "name": "Jason", "screen_name": "GuatemalanJason", "lang": "en", "location": "null", "create_at": date("2012-10-24"), "description": "hi David", "followers_count": 487, "friends_count": 357, "statues_count": 8979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732757478240256, "text": "That's my Lil baby thou ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3214534617, "name": "IT'S TASIA NICOLE", "screen_name": "a682284529dc44f", "lang": "en", "location": "null", "create_at": date("2015-04-27"), "description": "SELFMADE", "followers_count": 116, "friends_count": 134, "statues_count": 2110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Rouge, MI", "id": "e7cdcd61b0c07e97", "name": "River Rouge", "place_type": "city", "bounding_box": rectangle("-83.153043,42.258976 -83.103286,42.290022") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2668760, "cityName": "River Rouge" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732757549416452, "text": "Wind 2.1 mph SE. Barometer 30.174 in, Steady. Temperature 36.1 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 7799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732757582909440, "text": "Not I �� https://t.co/m3VksvU2oM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951563008, "name": "CH", "screen_name": "chelsnextdoor", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "No wahala. No vex. Fear not.", "followers_count": 631, "friends_count": 434, "statues_count": 18303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.500723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732758417739776, "text": "@MommyIsTattedd say the word", "in_reply_to_status": -1, "in_reply_to_user": 163995592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163995592 }}, "user": { "id": 161163473, "name": "Clint", "screen_name": "papi_tranquilo", "lang": "en", "location": "District of Columbia, USA", "create_at": date("2010-06-29"), "description": "#Trinidadian #BloakCity #EBK #UNRULY #BADINNAREALLIFE", "followers_count": 1130, "friends_count": 627, "statues_count": 58002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beltsville, MD", "id": "eac9405683a07098", "name": "Beltsville", "place_type": "city", "bounding_box": rectangle("-76.962116,39.014758 -76.877619,39.065391") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2406400, "cityName": "Beltsville" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732758501494785, "text": "https://t.co/uKVaeV7VaR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1378408734, "name": "EJ®", "screen_name": "Ejvaldez24", "lang": "en", "location": "null", "create_at": date("2013-04-24"), "description": "BASSMANFISHING. #gohawks #SauceCounsel", "followers_count": 566, "friends_count": 612, "statues_count": 2902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Othello, WA", "id": "0c98a8bc9ad9514d", "name": "Othello", "place_type": "city", "bounding_box": rectangle("-119.228937,46.781845 -119.147159,46.846114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53001, "countyName": "Adams", "cityID": 5352215, "cityName": "Othello" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732758677622784, "text": "I'm sleepy af but gotta support my bro ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1389609908, "name": "B✨", "screen_name": "Bvccax", "lang": "en", "location": "null", "create_at": date("2013-04-29"), "description": "Sghs.", "followers_count": 652, "friends_count": 255, "statues_count": 53160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732759508140032, "text": "It would be crazy creating the next \" Supreme \"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334535977, "name": "Yan", "screen_name": "YanLebedev23", "lang": "en", "location": "Portland ", "create_at": date("2011-07-13"), "description": "just living ✌️ Portland Oregon.", "followers_count": 377, "friends_count": 370, "statues_count": 23554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732759638167553, "text": "@angel_sithideth UR SO PRETTY WHAT THE HECK", "in_reply_to_status": -1, "in_reply_to_user": 1581966416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1581966416 }}, "user": { "id": 2273423317, "name": "andrea", "screen_name": "AndreaEstwada", "lang": "en", "location": "null", "create_at": date("2014-01-02"), "description": "SMHHHH", "followers_count": 210, "friends_count": 88, "statues_count": 8023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732760007258112, "text": "@perks_cafe Perk's Café & Market #Downtown #Buffalo https://t.co/vzPNNk1ZeZ Excellent https://t.co/x2765k8gAr", "in_reply_to_status": -1, "in_reply_to_user": 1482551030, "favorite_count": 0, "coordinate": point("-78.87351036,42.88789807"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "Downtown", "Buffalo" }}, "user_mentions": {{ 1482551030 }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3654, "friends_count": 507, "statues_count": 163319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732760129003520, "text": "@KeithKill I mean I can see it not being the type of movie for everyone haha but I thought the acting and the way it was shot was great", "in_reply_to_status": 685730433183674369, "in_reply_to_user": 507901579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 507901579 }}, "user": { "id": 339835481, "name": "Chris Krone", "screen_name": "KrisChrone", "lang": "en", "location": "NYC", "create_at": date("2011-07-21"), "description": "The life of a Jets fan related to Tom Brady", "followers_count": 277, "friends_count": 274, "statues_count": 13263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-09T00:00:04.000Z"), "id": 685732760154025984, "text": "Temp: 35.5°F - Dew Point: 33.9° - Wind: 5.0 mph - Gust: 7.6 - Rain Today: 0.00in. - Pressure: 29.72in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 9, "friends_count": 11, "statues_count": 11739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-09T00:00:05.000Z"), "id": 685732760837697537, "text": "Man God is good all the time... Thank u Jesus... ��������✊��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983073914, "name": "Creole Krissy", "screen_name": "lovelycrissy7", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-12-01"), "description": "I'm Mexican & Creole august alsina fan ❤.. alsina nation ❤️ TTCL out now go get your copy or buy it on iTunes just click the link.. https://itun.es/us/Sgeq_", "followers_count": 83, "friends_count": 98, "statues_count": 1304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-09T00:00:05.000Z"), "id": 685732760917512192, "text": "@NewkumT42 subtweeting hard", "in_reply_to_status": 685719447869485056, "in_reply_to_user": 316472946, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316472946 }}, "user": { "id": 604677176, "name": "Coach Nate", "screen_name": "nsnitz", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2012-06-10"), "description": "Lacrosse Coach | Founder & CEO | Grind Time Lacrosse Co.™ Est. 2015 | Stealth | Private Lacrosse Instruction #YearOfSnitz #GrindTime #GTLax follow @GrindTimelax", "followers_count": 176, "friends_count": 334, "statues_count": 2857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-01-09T00:00:05.000Z"), "id": 685732761424928768, "text": "Always glad I convince myself to go out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 210653028, "name": "#OldKentuckyMedals", "screen_name": "Klims37", "lang": "en", "location": "Some rink somewhere, TX", "create_at": date("2010-10-31"), "description": "K is a writer and an atrocious defenseman but at least he's charming BU Grad, Former @sarampage intern, @cupofchowdah blogger occasional writer for @rgvsports", "followers_count": 352, "friends_count": 1924, "statues_count": 25270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-01-09T00:00:05.000Z"), "id": 685732762129567744, "text": "I wish you could rate bouncers at the club bc I would've given this stupid fuck 0/5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66303219, "name": "brookebisss", "screen_name": "LookAtBrookeDoe", "lang": "en", "location": "null", "create_at": date("2009-08-17"), "description": "null", "followers_count": 425, "friends_count": 188, "statues_count": 17414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-09T00:00:05.000Z"), "id": 685732762628796416, "text": "Blessings ����. Come with patience ✊��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2587688632, "name": "LOONEY_TUNEZ", "screen_name": "TraySavage5", "lang": "en", "location": "null", "create_at": date("2014-06-06"), "description": "I Been Getting $ I Aint Worried Bout A Hoe , Dat Nigga Foreign ⌚️✨ Snapchat: Gunplay2394", "followers_count": 2062, "friends_count": 1470, "statues_count": 21597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Myers, FL", "id": "711c63250a2df503", "name": "Fort Myers", "place_type": "city", "bounding_box": rectangle("-81.895498,26.564625 -81.750855,26.679673") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1224125, "cityName": "Fort Myers" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095127274745856, "text": "ALL I WANT�� https://t.co/limfzprn8A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1674175320, "name": "mar㊙️", "screen_name": "_blasianhenny", "lang": "en", "location": "at a photoshoot", "create_at": date("2013-08-15"), "description": "IG: @blasianhenny | contact on tumblr", "followers_count": 858, "friends_count": 453, "statues_count": 29194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095127601934341, "text": "& Jigga Man is an automatic W ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133136408, "name": "❤[Nå]Ëémáh.❤", "screen_name": "AYoNaNa2PreTTY", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2010-04-14"), "description": "• #IG : x.nana2pretty.x •", "followers_count": 1182, "friends_count": 884, "statues_count": 52779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095127803199488, "text": "Ripley SW Limestone Co. Temp: 35.8°F Wind:6.9mph Pressure: 989.5mb Rising quickly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 47278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095127891197952, "text": "You should be here in my bed w/ me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373748567, "name": "Dev", "screen_name": "Dev_rich14", "lang": "en", "location": "Oklahoma, USA", "create_at": date("2011-09-14"), "description": "18•UCO College student•Sass x20•Could prob win me over with mimosas", "followers_count": 967, "friends_count": 684, "statues_count": 20235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095128113614848, "text": "03:00:03 |Temp: 53.1ºF | Wind Chill 53.1ºF |Dew Point 50.5ºF | Rain today: 0.01 inches | Wind: 0.0 mph from the S, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095128239325184, "text": "Still never had sushi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546651849, "name": "Vincè", "screen_name": "vinceroolz", "lang": "en", "location": "null", "create_at": date("2012-04-05"), "description": "IG: vinceroolz Skateboarder", "followers_count": 443, "friends_count": 407, "statues_count": 19136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095128486785024, "text": "@Cradily_ @kaipistols it's fucking annoying, especially if you win.", "in_reply_to_status": 686094983028293632, "in_reply_to_user": 2642835772, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2642835772, 1118702334 }}, "user": { "id": 2533022562, "name": "Norn", "screen_name": "GravelordBread", "lang": "en", "location": "Lincoln NE ", "create_at": date("2014-05-29"), "description": "I am a 3000 year old warlock. \nI trade tf2 items and draw. [Account sometimes NSFW] @Queeeboo", "followers_count": 334, "friends_count": 59, "statues_count": 65280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095128901992448, "text": "I don't really listen to music. It's just a background noise.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880812341, "name": "вrooĸlyn.", "screen_name": "_bootymuncher_", "lang": "en", "location": "null", "create_at": date("2014-10-28"), "description": "dont trip . home skillet", "followers_count": 131, "friends_count": 253, "statues_count": 803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barstow, CA", "id": "30c892dedb1a2c67", "name": "Barstow", "place_type": "city", "bounding_box": rectangle("-117.092355,34.862137 -116.963954,34.926831") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 604030, "cityName": "Barstow" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095128969121792, "text": "20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2772711763, "name": "Joel", "screen_name": "TYBGManor", "lang": "en", "location": "Dreamville", "create_at": date("2014-08-27"), "description": "Never Question Greatness", "followers_count": 390, "friends_count": 96, "statues_count": 5809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095129019551744, "text": "3 hours left", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560719795, "name": "Baby Reindeer", "screen_name": "____Buttercup__", "lang": "en", "location": "Far away from everything ", "create_at": date("2012-04-22"), "description": "i think you'll notice when things becomes different.", "followers_count": 289, "friends_count": 448, "statues_count": 4087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor, MI", "id": "a9eb1b07c3864593", "name": "Taylor", "place_type": "city", "bounding_box": rectangle("-83.308978,42.181294 -83.228111,42.269861") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2679000, "cityName": "Taylor" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095129124433920, "text": "a week into 2016, and I'm already over this year ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51345416, "name": "ⓐⓢⓗ", "screen_name": "achampionnn", "lang": "en", "location": "ncsu • zta ", "create_at": date("2009-06-26"), "description": "• i'm back & i'm better •", "followers_count": 837, "friends_count": 245, "statues_count": 45464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Swansboro, NC", "id": "007413570cfcf104", "name": "Swansboro", "place_type": "city", "bounding_box": rectangle("-77.228044,34.638954 -77.112346,34.726942") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3766360, "cityName": "Swansboro" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095129837436928, "text": "How classy are the Bengals doe?? Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348822077, "name": "Tay ✨", "screen_name": "tayrenee_10", "lang": "en", "location": "Indiana", "create_at": date("2011-08-04"), "description": "23 | Colts obsessed | Nurse | Ecclesiastes 3:1 | ❤️", "followers_count": 397, "friends_count": 371, "statues_count": 31487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Judson, IN", "id": "0b950f36b9f21cba", "name": "North Judson", "place_type": "city", "bounding_box": rectangle("-86.785861,41.209653 -86.766067,41.225798") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18149, "countyName": "Starke", "cityID": 1854900, "cityName": "North Judson" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130038636544, "text": "@KingGerrad @TaySoTrill ������", "in_reply_to_status": 686094908298399746, "in_reply_to_user": 82192399, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 82192399, 92402201 }}, "user": { "id": 739483975, "name": "DaemonDash", "screen_name": "PastorActivist", "lang": "en", "location": "null", "create_at": date("2012-08-05"), "description": "Las Vegas underground rapper.Check out my new video to my song Dont Shoot. http://youtu.be/48E7T0S6wN0 . darryonnehatcher32@gmail.com for business inquiries", "followers_count": 832, "friends_count": 540, "statues_count": 73511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130151944193, "text": "A-Team", "in_reply_to_status": 685304543844814852, "in_reply_to_user": 1326010188, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1326010188, "name": "P. ☄", "screen_name": "OPMJUICE", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-04-03"), "description": "19. Thoughtful nigga. I'm trynna be like snoop in 93' uber code: uberOPMJUICE", "followers_count": 2451, "friends_count": 1627, "statues_count": 22436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130185486336, "text": "Love don't change.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 838926889, "name": "jess", "screen_name": "jessica_wagers", "lang": "en", "location": "cloud 9", "create_at": date("2012-09-21"), "description": "when you stop dreaming, it's time to die", "followers_count": 556, "friends_count": 489, "statues_count": 12721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, IN", "id": "49f0a5eb038077e9", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.996969,39.163203 -85.847755,39.25966") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18005, "countyName": "Bartholomew", "cityID": 1814734, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130504200192, "text": "Happy birthday to my best friend ever. I love you so so so much! Happy 18th baby boy! @thetotallytyler ❤️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2748133447 }}, "user": { "id": 332593374, "name": "m a r i a h ☀️", "screen_name": "ryah_nichole", "lang": "en", "location": "Northern Cali☀️", "create_at": date("2011-07-09"), "description": "act like a lady, think like a man ❤️", "followers_count": 521, "friends_count": 385, "statues_count": 17205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ripon, CA", "id": "66ec48e552bb105b", "name": "Ripon", "place_type": "city", "bounding_box": rectangle("-121.151704,37.717441 -121.089493,37.76484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 661026, "cityName": "Ripon" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130680360960, "text": "@CdiGanon go to jail", "in_reply_to_status": 686095052951531520, "in_reply_to_user": 2245232510, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2245232510 }}, "user": { "id": 75440225, "name": "daddy phasma", "screen_name": "gerwulfa", "lang": "en", "location": "me swamp", "create_at": date("2009-09-18"), "description": "dragon age, mass effect, love live, femslash // b grphyon birdley // sea hag and elf-catcher // artist support me on patreon https://www.patreon.com/gerwulfa", "followers_count": 1052, "friends_count": 361, "statues_count": 36174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095130818772992, "text": "@MaganaA21 https://t.co/H3ziHjzXmH", "in_reply_to_status": 686094952229552128, "in_reply_to_user": 529107936, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 529107936 }}, "user": { "id": 2511927864, "name": "oscar", "screen_name": "OhhKnows", "lang": "en", "location": "Santa Ana, CA", "create_at": date("2014-05-21"), "description": "Sometimes play the villain, sometimes play the hero. | Dreamville | 20 & Blessed | sc: ramos65", "followers_count": 332, "friends_count": 207, "statues_count": 5366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-10T00:00:00.000Z"), "id": 686095131041112064, "text": "It is what it is ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 154410083, "name": "♎️", "screen_name": "HBK_Daze", "lang": "en", "location": "Lost Angels", "create_at": date("2010-06-10"), "description": "No mercy for the weak SC:Socal_daze", "followers_count": 3397, "friends_count": 724, "statues_count": 108438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple City, CA", "id": "206e932cc0e89fa2", "name": "Temple City", "place_type": "city", "bounding_box": rectangle("-118.081392,34.085395 -118.028425,34.120984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 678148, "cityName": "Temple City" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095131498426368, "text": "Good times. Happy Birthday Jamaya! @ Oceanside, California https://t.co/bJAjONWFR5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.326,33.2116"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3115947319, "name": "Midnight Jack ", "screen_name": "midnightjackbrw", "lang": "en", "location": "Oceanside, CA", "create_at": date("2015-03-29"), "description": "null", "followers_count": 48, "friends_count": 75, "statues_count": 129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095131636842496, "text": "@themainroom1035 @Hot_1035 @Specialist916 @djsn1 https://t.co/vEOFTA367U", "in_reply_to_status": 686093889552412672, "in_reply_to_user": 4460573778, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4460573778, 24061889, 65866188, 22503010 }}, "user": { "id": 2368244679, "name": "TheRealKingSparkzz", "screen_name": "KingSparkzz", "lang": "en", "location": "Trenton, NJ", "create_at": date("2014-02-26"), "description": "For booking and music business only contact\nKingMusic2014@Gmail.com\nFollow my Instagram @therealkingsparkzz", "followers_count": 5739, "friends_count": 6253, "statues_count": 18412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, NJ", "id": "74767081120f9b42", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-74.817535,40.183967 -74.729138,40.248298") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3474000, "cityName": "Trenton" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095131905146881, "text": "Ye knows https://t.co/f0ZRTzqFsW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283121857, "name": "Niamh", "screen_name": "NiamhKing_", "lang": "en", "location": "San Francisco", "create_at": date("2011-04-16"), "description": "MA in Gender and Sexuality Studies. Kardashian defender. Self promotion expert.", "followers_count": 217, "friends_count": 341, "statues_count": 8642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095132102246400, "text": "fav for Mel's booty pics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335629099, "name": "taylor wicks", "screen_name": "taylorwicks99", "lang": "en", "location": "somewhere w mel", "create_at": date("2011-07-14"), "description": "LV", "followers_count": 291, "friends_count": 281, "statues_count": 5753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095132815273984, "text": "if I want it to be it will�������� https://t.co/MnZ7eJdUZb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3193543243, "name": "fay", "screen_name": "_makayllaa5", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2015-05-12"), "description": "I don't know you.. so don't act like you know me ☮443 sc: makaylafayy", "followers_count": 617, "friends_count": 529, "statues_count": 7968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Charles, LA", "id": "21b6f72f9fdaeb06", "name": "Lake Charles", "place_type": "city", "bounding_box": rectangle("-93.314852,30.124859 -93.080595,30.280683") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2241155, "cityName": "Lake Charles" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095132907683840, "text": "Wind 0 mph --. Barometer 29.95 in, Rising. Temperature 48.0 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133490581504, "text": "ugh i had like 20 tabs open and i accidentally unplugged my comp and then chrome didn't save them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159024741, "name": "Jilly Bean", "screen_name": "PiratessUnluck", "lang": "en", "location": "So Cal", "create_at": date("2010-06-24"), "description": "---------Learn from the bones---------Twitter, Tumblr, and Skype are all the same handle.", "followers_count": 153, "friends_count": 192, "statues_count": 23606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133503172608, "text": "Watching home omg I want to cry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367879063, "name": "Cristina", "screen_name": "thezmami", "lang": "en", "location": "Houston", "create_at": date("2011-09-04"), "description": "350z", "followers_count": 3155, "friends_count": 2497, "statues_count": 44261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133520097280, "text": "Vanessa always been a goddess.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243943390, "name": "#KeepCalvin", "screen_name": "WhataburgerLife", "lang": "en", "location": "GHS 2016", "create_at": date("2011-01-27"), "description": "Being realistic is the most common path to mediocracy - Will Smith | RIP '92 Miata 12/3/15", "followers_count": 575, "friends_count": 336, "statues_count": 22445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133528436736, "text": "01/10@03:00 - Temp 52.5F, WC 51.5F. Wind 4.4mph ENE, Gust 10.0mph. Bar 29.755in, Falling quickly. Rain 0.17in. Hum 99%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 63, "friends_count": 8, "statues_count": 46160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133562003458, "text": "@SJ_lovesyou lmaooooooo no you don't ������", "in_reply_to_status": 686095009288949760, "in_reply_to_user": 235709935, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 235709935 }}, "user": { "id": 338905762, "name": "K$ ✨", "screen_name": "QueenAustin__", "lang": "en", "location": "Israel | Columbus | BGSU 17'", "create_at": date("2011-07-20"), "description": "• Owner of Krave Collection - SC: FatalForeign | Business Inquiries: Kravecollectionbusiness@gmail.com |", "followers_count": 1693, "friends_count": 665, "statues_count": 119248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133683519488, "text": "Introduced him to real and he still likes lames", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 472547946, "name": "♡Bena♡", "screen_name": "brianavernon", "lang": "en", "location": "null", "create_at": date("2012-01-23"), "description": "OMAHA", "followers_count": 246, "friends_count": 321, "statues_count": 4317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133910016000, "text": "I think its gonna be a hell of a night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599717469, "name": "…", "screen_name": "_Yvettexoxo", "lang": "en", "location": "rialto , ca", "create_at": date("2013-07-16"), "description": "c/o 2016", "followers_count": 2093, "friends_count": 1920, "statues_count": 18708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095133956161536, "text": "When your girl gets mad at you for not replying cause you were playing video games https://t.co/j5elNL1yAk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3246412855, "name": "Jc", "screen_name": "jafetcoello_", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-06-15"), "description": "I guess it really is just me, myself & all my millions", "followers_count": 536, "friends_count": 314, "statues_count": 5826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134090510339, "text": "Today's face highlighter on fleek tonight with the glowKit sunburst by @ABHcosmetics @norvina1 https://t.co/EHad4I1174", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50089673, 2516590279 }}, "user": { "id": 2390935322, "name": "Crystal Alvarez", "screen_name": "CryStyleXoXo19", "lang": "en", "location": "null", "create_at": date("2014-03-15"), "description": "makeup artist instagram /// CRYSTYLEXOXO93", "followers_count": 77, "friends_count": 244, "statues_count": 1396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allentown, PA", "id": "cfcf6b96b61e7b11", "name": "Allentown", "place_type": "city", "bounding_box": rectangle("-75.548035,40.547616 -75.419836,40.636034") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4202000, "cityName": "Allentown" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134304382976, "text": "Wind 2.0 mph NW. Barometer 30.059 in, Rising Rapidly. Temperature 13.1 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134404939776, "text": "CANTSTOPCARTER2016 https://t.co/zqqt4zgqp3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315620381, "name": "LIL WESTSIDE", "screen_name": "WestsideKeke", "lang": "en", "location": "Jacksonville FL", "create_at": date("2011-06-11"), "description": "#HOTNIGGA #FEMALEARTIST #KINGOFTAKEOFFS #INDEPENDENTNOLABEL Bookings Or Features - Thomaslaquria@gmail.com #IMAKEHITS #GANGGREENGANGCEO #FREEDAGUYS", "followers_count": 1605, "friends_count": 1491, "statues_count": 38890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134434418688, "text": "F.U.T.W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256297909, "name": "OG Mcqueezy", "screen_name": "Jay_got_power", "lang": "en", "location": "CLT NC", "create_at": date("2011-02-22"), "description": "[IG: jay_got_power]", "followers_count": 212, "friends_count": 88, "statues_count": 4904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134824525824, "text": "Sad but True...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319260977, "name": "That Guy Pole ♊", "screen_name": "PoleTooCold23", "lang": "en", "location": "Forsyth, GA #478", "create_at": date("2011-06-17"), "description": "God grant me the serenity to accept the things I cannot change, courage to change the things I can, and wisdom to know the difference. #TeamLeBron. Sacha ❤️", "followers_count": 610, "friends_count": 880, "statues_count": 76539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forsyth, GA", "id": "bf69cdc5efc018fb", "name": "Forsyth", "place_type": "city", "bounding_box": rectangle("-83.966898,33.008138 -83.914853,33.06068") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13207, "countyName": "Monroe", "cityID": 1330732, "cityName": "Forsyth" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095134929231874, "text": "444- ur gorgeous omg���� + we need to talk more!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2157302004, "name": "lindsláyy", "screen_name": "lindseyariannax", "lang": "en", "location": "cedar rapids // shs", "create_at": date("2013-10-26"), "description": "just chillin'", "followers_count": 437, "friends_count": 519, "statues_count": 9003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135067746305, "text": "Wind 7.0 mph W. Barometer 29.527 in, Rising slowly. Temperature 41.4 °F. Rain today 0.03 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 39, "friends_count": 25, "statues_count": 16764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135197691905, "text": "So many thoughts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212018771, "name": "Liz", "screen_name": "naomigomez__", "lang": "en", "location": "null", "create_at": date("2013-11-23"), "description": "I'll love you, always & forever, @lenniareyes ♡", "followers_count": 749, "friends_count": 324, "statues_count": 12437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial Beach, CA", "id": "5cc07810d110956f", "name": "Imperial Beach", "place_type": "city", "bounding_box": rectangle("-117.133501,32.555434 -117.099564,32.591984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 636294, "cityName": "Imperial Beach" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135197794304, "text": "Wind 4.9 mph SE. Barometer 29.872 in, Falling Rapidly. Temperature 41.5 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 5700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135373799424, "text": "@LunaticSX Here’s the thing though; you listed a lot of GOP ding dongs. Clinton (womanizer, accused rapist, pals around with pedophiles)…", "in_reply_to_status": 686093231243722752, "in_reply_to_user": 11067082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 11067082 }}, "user": { "id": 7603032, "name": "Greg Koenig", "screen_name": "gak_pdx", "lang": "en", "location": "PDX, OR", "create_at": date("2007-07-19"), "description": "Follow me on Twitter! You'll go where I go, defile what I defile, eat who I eat.", "followers_count": 3830, "friends_count": 579, "statues_count": 18905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl District, Portland", "id": "517134823363846c", "name": "Pearl District", "place_type": "neighborhood", "bounding_box": rectangle("-122.687169,45.522895 -122.674272,45.538123") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135403163648, "text": "this is my favorite part I love ky more than anything @kyrosbia https://t.co/gjAGEjvCHf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1869539346 }}, "user": { "id": 2770265184, "name": "katy", "screen_name": "katelynkms", "lang": "en", "location": "null", "create_at": date("2014-08-26"), "description": "i have a cat", "followers_count": 295, "friends_count": 291, "statues_count": 14079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-10T00:00:01.000Z"), "id": 686095135470321665, "text": "@kianwhatsgood HAPPY BIRTHDAY SUNSHINE I HOPE YOU HAVE A GOOD ASS DAY FULL OF FUN AND GET THE BEAR YOU WANT AND EAT GOOD FOOD LOVE YEAH", "in_reply_to_status": -1, "in_reply_to_user": 3821753540, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3821753540 }}, "user": { "id": 2244856496, "name": "n", "screen_name": "EXPLORECAYLEN", "lang": "en", "location": "null", "create_at": date("2013-12-13"), "description": "im just all about him", "followers_count": 1260, "friends_count": 250, "statues_count": 25774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095135612899329, "text": "@unicornmeli nah b :( lol", "in_reply_to_status": 686095070517268480, "in_reply_to_user": 178197751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178197751 }}, "user": { "id": 399102137, "name": "2phoneshawty", "screen_name": "drellzzz", "lang": "en", "location": "818/626", "create_at": date("2011-10-26"), "description": "If I make it to heaven I hope it comes with a view so I could see LA. ig- dreadiamandis", "followers_count": 414, "friends_count": 367, "statues_count": 25158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095135688404992, "text": "@matty__love u wants me dead ? ����", "in_reply_to_status": 686094681143402496, "in_reply_to_user": 3376450947, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3376450947 }}, "user": { "id": 392559659, "name": "O3|O7..", "screen_name": "_xqueenkee", "lang": "en", "location": "null", "create_at": date("2011-10-17"), "description": "SU19|Demarcus|augustwife|18..", "followers_count": 4985, "friends_count": 4313, "statues_count": 126208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095135713591298, "text": "#woodburyroll #naruto #salmonsushi #brownrice #avoside #healthyeats @mocaasianbistro photo cred:… https://t.co/fjn8vG9wC0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.47660037,40.81534901"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "woodburyroll", "naruto", "salmonsushi", "brownrice", "avoside", "healthyeats" }}, "user_mentions": {{ 1950074095 }}, "user": { "id": 1950074095, "name": "MoCA Asian Bistro", "screen_name": "MoCAAsianBistro", "lang": "en", "location": "New York", "create_at": date("2013-10-09"), "description": "Asian Fusion Restaurant", "followers_count": 20, "friends_count": 9, "statues_count": 784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodbury, NY", "id": "b90cd8b3018716d0", "name": "Woodbury", "place_type": "city", "bounding_box": rectangle("-73.491527,40.794833 -73.448769,40.84129") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3682750, "cityName": "Woodbury" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095135826821122, "text": "I'm bored, what y'all need �� https://t.co/4AQMS4Q6md", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702346712, "name": "@TDREAMS", "screen_name": "x_anders0n", "lang": "en", "location": "Minneapolis - LA ", "create_at": date("2012-07-17"), "description": "life on cruise.", "followers_count": 905, "friends_count": 401, "statues_count": 32352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottage Grove, MN", "id": "8e24cdcee62a4347", "name": "Cottage Grove", "place_type": "city", "bounding_box": rectangle("-92.98399,44.800931 -92.903515,44.855344") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27163, "countyName": "Washington", "cityID": 2713456, "cityName": "Cottage Grove" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095135885504514, "text": "i was lit today but ima be done on 1\\14 beefy birthday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2325283274, "name": "HARITH ❤", "screen_name": "mamacitassss", "lang": "en", "location": "✈ ", "create_at": date("2014-02-03"), "description": "popitforbeef\n1/14/92 - 1/9/15", "followers_count": 374, "friends_count": 217, "statues_count": 37440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136078467072, "text": "My life be like������ https://t.co/Yw4FYVn62u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252354834, "name": "Ash♡", "screen_name": "imsoAshleyy", "lang": "en", "location": "null", "create_at": date("2011-02-14"), "description": "SATX • Nursing Major• I am not spoiled, I'm just well taken care of ♥ SC:ashley_lozoya", "followers_count": 400, "friends_count": 810, "statues_count": 5976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Angelo, TX", "id": "4cb7f66ae6f6d487", "name": "San Angelo", "place_type": "city", "bounding_box": rectangle("-100.52941,31.335769 -100.376768,31.545276") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48451, "countyName": "Tom Green", "cityID": 4864472, "cityName": "San Angelo" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136174903296, "text": "HAPPY BIRTHDAY YA BAD BITCH @NicoleKinli", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2871332639 }}, "user": { "id": 3831714980, "name": "mami burrito", "screen_name": "chynnabommarito", "lang": "en", "location": "Lacey, WA", "create_at": date("2015-10-08"), "description": "ig : chynnaburrito", "followers_count": 250, "friends_count": 203, "statues_count": 3380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lacey, WA", "id": "0084a2ce4881fa56", "name": "Lacey", "place_type": "city", "bounding_box": rectangle("-122.839662,46.959463 -122.691155,47.119369") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5336745, "cityName": "Lacey" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136275709953, "text": "#SupportOriginMelissa 47.1°F Wind:3.6mph Pressure: 29.53hpa Falling Rain Today 0.00in. Forecast: Precipitation at times, very unsettled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136418349056, "text": "My world https://t.co/7YTBHxQfQd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459342193, "name": "Juliana", "screen_name": "mymellowmoons", "lang": "en", "location": "NYC/FL", "create_at": date("2012-01-09"), "description": "As I began to love myself I freed myself of anything that isn't good for my health ... - Charlie Chaplin", "followers_count": 19, "friends_count": 25, "statues_count": 102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136434958336, "text": "Girls?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2348120371, "name": "ellie", "screen_name": "ellieferrero", "lang": "en", "location": "earth", "create_at": date("2014-02-16"), "description": "Scorpio child", "followers_count": 1453, "friends_count": 239, "statues_count": 18894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rossmoor, CA", "id": "ab707672074afdbf", "name": "Rossmoor", "place_type": "city", "bounding_box": rectangle("-118.090063,33.774852 -118.071977,33.803188") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 663050, "cityName": "Rossmoor" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136619663360, "text": "Lol I had to get one in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48484668, "name": "Intellectual Killah", "screen_name": "IAmPhillyC", "lang": "en", "location": "Atlanta", "create_at": date("2009-06-18"), "description": "Vintage from 1980 something #TeamBellaBellz #Pisces #ATLien", "followers_count": 2097, "friends_count": 2060, "statues_count": 67768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mableton, GA", "id": "fe4b2b8e51cc85c1", "name": "Mableton", "place_type": "city", "bounding_box": rectangle("-84.620272,33.769819 -84.506557,33.852813") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1348288, "cityName": "Mableton" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136967802880, "text": "#RealTalk https://t.co/7UfYuyEWDO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "RealTalk" }}, "user": { "id": 69180016, "name": "Donna Westerfeld", "screen_name": "DonnaWesterfeld", "lang": "en", "location": "Ohio", "create_at": date("2009-08-26"), "description": "Love, Go, Do.", "followers_count": 90, "friends_count": 215, "statues_count": 3611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095136992833538, "text": "Awake at 1:45am stuffed up nose��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450673045, "name": "Ripping Yarns", "screen_name": "RYarns", "lang": "en", "location": "Minot, North Dakota", "create_at": date("2011-12-30"), "description": "Knitter,Crocheter, former LYS owner,Executive staff for my cats Formerly @ nodakyarn . i am LaurelFaye on Raverly", "followers_count": 1316, "friends_count": 1142, "statues_count": 24066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minot, ND", "id": "2ae0ed991bd2e8d6", "name": "Minot", "place_type": "city", "bounding_box": rectangle("-101.357743,48.182195 -101.178137,48.283425") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38101, "countyName": "Ward", "cityID": 3853380, "cityName": "Minot" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095137080909825, "text": "@AnaIvanovic thats ok this lost mean anything Karolina play amazing head up come stronger to Melbourne and head up move forward champ adje", "in_reply_to_status": -1, "in_reply_to_user": 180733809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 180733809 }}, "user": { "id": 534926328, "name": "Javier De La Garza", "screen_name": "javidelagarza95", "lang": "en", "location": "Houston TX", "create_at": date("2012-03-23"), "description": "I have the pleasure of meeting and fan of @Anaivanovic @BMATTEK @jaksicjovana and @riske4rewards and in love with @alexmorgan13 Ivanovic is the best # 1", "followers_count": 179, "friends_count": 1001, "statues_count": 5122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095137324273664, "text": "Patti Smith at the Wiltern! @ The Wiltern https://t.co/zKRJUrcdgK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3084505,34.06124109"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59371261, "name": "Christine McLeod", "screen_name": "redheadrep", "lang": "en", "location": "LA, Baby! (SouthPasadena)", "create_at": date("2009-07-22"), "description": "Proud ginger! Mom to @GrungeDogMurphy. Lover of music/travel/fashion. Energy/Environmental/Regulatory/Public Affairs Professional. Instagram: @redheadpjfan", "followers_count": 650, "friends_count": 1071, "statues_count": 9445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095137567588354, "text": "@glennspizza I saw you were looking for it lol", "in_reply_to_status": 686093870522839040, "in_reply_to_user": 2334394790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2334394790 }}, "user": { "id": 446037325, "name": "✨Saint Ari Dameron✨", "screen_name": "PunchingDemons", "lang": "en", "location": "West New York, NJ", "create_at": date("2011-12-24"), "description": "Ariel/ Ari 22. Writer. Artist. Model. Actress. Ghost Magnet. TWD ✨Nicki Minaj✨I love NormanReedus, SteveYeun, RamiMalek, Oscar Isaac, JamesFranco, &AndyLincoln", "followers_count": 710, "friends_count": 949, "statues_count": 11759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West New York, NJ", "id": "8479859139338fb7", "name": "West New York", "place_type": "city", "bounding_box": rectangle("-74.023591,40.778186 -73.99818,40.796544") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3479610, "cityName": "West New York" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095137898758144, "text": "Pull up, skurt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1728082988, "name": "✨Mari Haze✨", "screen_name": "salty_biotch", "lang": "en", "location": "Portland, OR", "create_at": date("2013-09-04"), "description": "null", "followers_count": 163, "friends_count": 138, "statues_count": 4123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095138775408641, "text": "everyone ships me and joana ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1258431960, "name": "joe-c", "screen_name": "jossyfaarts", "lang": "en", "location": "trap house", "create_at": date("2013-03-10"), "description": "c'est la vie", "followers_count": 285, "friends_count": 72, "statues_count": 26402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095138867654656, "text": "How u knock a nigga out then become his manager....shit backwards as hell....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190865796, "name": "Rugby_Thompson™", "screen_name": "OH_YOUMAD914", "lang": "en", "location": "ERB❤️ ", "create_at": date("2010-09-14"), "description": "CA via NYC #FTWWL #FTW #marathon", "followers_count": 414, "friends_count": 896, "statues_count": 25685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095139337564160, "text": "Wind 0.7 mph N. Barometer 30.03 in, Falling. Temperature 36.7 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 92, "statues_count": 156566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-10T00:00:02.000Z"), "id": 686095139396128768, "text": "����Turn the radio down and lay you down too. I could be the one for you.����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339281766, "name": "Conner Powell", "screen_name": "CpowOfficial", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-07-20"), "description": "null", "followers_count": 344, "friends_count": 252, "statues_count": 19141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Black Diamond, WA", "id": "6d8731cfb2572844", "name": "Black Diamond", "place_type": "city", "bounding_box": rectangle("-122.054431,47.286736 -121.995892,47.34361") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5306330, "cityName": "Black Diamond" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095139811405825, "text": "@CartuneNetwerk @trvssia I still own my Girbauds and will wear Bapestas any day of the week. Bad point.", "in_reply_to_status": 686094942511480832, "in_reply_to_user": 46187709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46187709, 68080197 }}, "user": { "id": 389298644, "name": "ricacardo", "screen_name": "QualmesJr", "lang": "en", "location": "San Jose, CA", "create_at": date("2011-10-11"), "description": "there is nothing either good or bad, but thinking makes it so", "followers_count": 462, "friends_count": 257, "statues_count": 69052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140050477056, "text": "#Canaletto / https://t.co/hwhIP6WbFW Just like venice without the pigeons https://t.co/bn1grWQsLi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.168554,36.122925"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Canaletto" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4277, "friends_count": 875, "statues_count": 374152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140176318465, "text": "#Canaletto / https://t.co/oLvuVJkTXU Just like venice without the pigeons https://t.co/YLa71YMI85", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.168554,36.122925"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Canaletto" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4277, "friends_count": 875, "statues_count": 374152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140201472000, "text": "o ok i see", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2218539792, "name": "kay", "screen_name": "ayykayla_", "lang": "en", "location": "ktx", "create_at": date("2013-11-27"), "description": "groovy groovy", "followers_count": 410, "friends_count": 367, "statues_count": 6963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kyle, TX", "id": "94ef42cc204d5195", "name": "Kyle", "place_type": "city", "bounding_box": rectangle("-97.896063,29.942852 -97.830853,30.035619") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4839952, "cityName": "Kyle" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140440506368, "text": "Dam it's only 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2255526194, "name": "Hunter Hayes", "screen_name": "HunterLitzau", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "It's a Cole World", "followers_count": 247, "friends_count": 237, "statues_count": 10773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fridley, MN", "id": "612139aad8c98456", "name": "Fridley", "place_type": "city", "bounding_box": rectangle("-93.290637,45.03534 -93.227152,45.125115") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2722814, "cityName": "Fridley" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140851613696, "text": "Lmaooo https://t.co/BltA1wBxxq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 476404228, "name": "Jigga Man⌚️", "screen_name": "BlackDreams21", "lang": "en", "location": "Personal Trainer", "create_at": date("2012-01-27"), "description": "Dawg...idk about you...but im tryna be wealthy.. •NoDaysOff™ •Grammy™ • NSU", "followers_count": 1790, "friends_count": 1762, "statues_count": 90160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suffolk, VA", "id": "6e0ed207f0c2d36b", "name": "Suffolk", "place_type": "city", "bounding_box": rectangle("-76.668194,36.698303 -76.402862,36.92173") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51800, "countyName": "Suffolk", "cityID": 5176432, "cityName": "Suffolk" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095140994203648, "text": "42.5F (Feels: 42.5F) - Humidity: 76% - Wind: 19.0mph WNW - Gust: 25.9mph - Pressure: 1031.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 221811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141057224704, "text": "Rt https://t.co/Bi76SV9UAZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2802768381, "name": "#BallinLikeImBanga⚓", "screen_name": "Lildex17o", "lang": "en", "location": "Eight-Bix-Zeroooo", "create_at": date("2014-10-03"), "description": "RIP GRANDMA DIE Yzzz\nFREE BANGA FREE TAKi \nFREE MEEKo17o BOYzzz\n Lzzz baby", "followers_count": 662, "friends_count": 436, "statues_count": 55228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141103243264, "text": "Never do I worry what anyone thinks of me haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2897092951, "name": "Alberto_E147", "screen_name": "AlbertoE187", "lang": "en", "location": "null", "create_at": date("2014-11-11"), "description": "ERHS, 2015 to infinity & still bomb weed & Hennessy.♠️ 3 i's so Eye think a little different⚫️Knowledge is power 〽️Keep An Open Mind♣️ #BeenFre$h#⃣4⃣7⃣", "followers_count": 205, "friends_count": 132, "statues_count": 12104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141149384704, "text": "partly cloudy -> fair\ntemperature down 78°F -> 73°F\nhumidity up 66% -> 79%\nwind 8mph -> 3mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.00333,21.38571"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238387019, "name": "Waipahu Weather", "screen_name": "WaipahuHI", "lang": "en", "location": "Waipahu, HI", "create_at": date("2011-01-14"), "description": "Weather updates, forecast, warnings and information for Waipahu, HI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 30, "friends_count": 2, "statues_count": 15314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141166157824, "text": "@veefosbo Thanks :)", "in_reply_to_status": 686095005484580865, "in_reply_to_user": 503748001, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 503748001 }}, "user": { "id": 598097684, "name": "Kylan", "screen_name": "OsborneKylan", "lang": "en", "location": "Anchorage, AK", "create_at": date("2012-06-03"), "description": "Success is a Mindset - Dimond High School - Basketball", "followers_count": 794, "friends_count": 1009, "statues_count": 16194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141174521856, "text": "I left my girl back home . I don't love her anymore .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358390282, "name": "OceanDriveSwank©", "screen_name": "QB_SaidIt", "lang": "en", "location": "Lancaster, TX", "create_at": date("2011-08-19"), "description": "#SHSU M.P.R", "followers_count": 1909, "friends_count": 1136, "statues_count": 190270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095141266833408, "text": "@masonbergher8 @nicoleovadia_ I think this one might too Morgan's.", "in_reply_to_status": 686094653754617857, "in_reply_to_user": 2335604166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2335604166, 2189459869 }}, "user": { "id": 2407496072, "name": "Daniel Attia", "screen_name": "danielattia26", "lang": "en", "location": "null", "create_at": date("2014-03-23"), "description": "Professional Fifa player", "followers_count": 171, "friends_count": 205, "statues_count": 847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142088871936, "text": "yay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2225366978, "name": "Austin Wigh", "screen_name": "aawigh", "lang": "en", "location": "null", "create_at": date("2013-12-01"), "description": "livin' life", "followers_count": 283, "friends_count": 254, "statues_count": 889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142130823168, "text": "Need wifi man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1657166498, "name": "mara.❥", "screen_name": "aaaramas", "lang": "en", "location": "sallad, saxet.", "create_at": date("2013-08-09"), "description": "snap: samtharippa", "followers_count": 653, "friends_count": 468, "statues_count": 17723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142256685057, "text": "@rynosnumbers3 Steelers get the W, Bengals get the money. Only in Vegas!!!", "in_reply_to_status": 686044680778170368, "in_reply_to_user": 1902027529, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1902027529 }}, "user": { "id": 1902027529, "name": "Ryan Dow", "screen_name": "rynosnumbers3", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-09-24"), "description": "Passion for sports & the sports betting community; one game at a time...", "followers_count": 45, "friends_count": 37, "statues_count": 3093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142445551616, "text": "Este es el trailer de la precuela de 'Expediente Warren 2: The Conjuring'.\nhttps://t.co/K6qDqSB9IM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2726080999, "name": "BIGVU", "screen_name": "BVUNOW", "lang": "en", "location": "null", "create_at": date("2014-08-12"), "description": "Video News App for Android", "followers_count": 400, "friends_count": 1314, "statues_count": 11632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142889992192, "text": "@zeppelinchick05 I'll make sure I don't go on a Tuesday lol", "in_reply_to_status": 686095037810147328, "in_reply_to_user": 77142409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77142409 }}, "user": { "id": 70886475, "name": "Lindsey Mendibles", "screen_name": "RinzyRoo420", "lang": "en", "location": "null", "create_at": date("2009-09-01"), "description": "Lindsey Louise. Laughter is a must. FAMILY FIRST. & I will be the first to admit that I'm crazy, but I have a big heart.", "followers_count": 670, "friends_count": 1796, "statues_count": 40708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142910951424, "text": "happy birthday baby! love yo hoe ass to pieces���������� #getreadyfordapictures https://t.co/q9wZmFkdZK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "getreadyfordapictures" }}, "user": { "id": 152339716, "name": "shorty", "screen_name": "MariaIbarraaa", "lang": "en", "location": "gangsta rap made me do it ", "create_at": date("2010-06-05"), "description": "• stay wavy • high ambition, low eyes •", "followers_count": 489, "friends_count": 370, "statues_count": 22253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095142978220032, "text": "Same shit different night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 887685822, "name": "Matt Macey", "screen_name": "MattIVI98", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-10-17"), "description": "All South American, D-7 Bound", "followers_count": 245, "friends_count": 237, "statues_count": 2456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, IL", "id": "d73cbce54317ffaa", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-87.99425,41.763852 -87.945389,41.825016") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1780645, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143066284032, "text": "Wind 3.0 mph SSE. Barometer 30.027 in, Falling Rapidly. Temperature 39.1 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 7823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143087292416, "text": "Lol I remember AZ put me on that song... tss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 989789593, "name": "Zeetoven", "screen_name": "zianimack", "lang": "en", "location": "null", "create_at": date("2012-12-04"), "description": "Just cuz u breathing that don't mean u living..", "followers_count": 2711, "friends_count": 2387, "statues_count": 6014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143133290496, "text": "#Corby's Pub #Labrador City https://t.co/lQ0bnjxRiZ Excellent https://t.co/ln6ALlng6d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.9102173,52.9448738"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Corby", "Labrador" }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3654, "friends_count": 507, "statues_count": 163671 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Labrador City, Newfoundland and Labrador", "id": "4861b1488693a05f", "name": "Labrador City", "place_type": "city", "bounding_box": rectangle("-66.988403,52.923052 -66.886485,52.995216") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143208783872, "text": "Happy birthday @aandrewwwleee �� https://t.co/QjGLKzGEkr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2255622000 }}, "user": { "id": 1283394444, "name": "Erica Lim", "screen_name": "ericaa_lim", "lang": "en", "location": "null", "create_at": date("2013-03-20"), "description": "null", "followers_count": 521, "friends_count": 517, "statues_count": 10098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143317843969, "text": "I should be unpacking but I'd rather just listen to young thug and fuck around in my room. So here we are", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628705623, "name": "Santana", "screen_name": "santanarene_", "lang": "en", "location": "null", "create_at": date("2012-07-06"), "description": "Don't believe me just watch", "followers_count": 255, "friends_count": 169, "statues_count": 17991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143535939584, "text": "UPDATE: HE REPLIED", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3219595201, "name": "superduperlyssa", "screen_name": "kinglyssie", "lang": "en", "location": "somewhere listening to kyle", "create_at": date("2015-05-18"), "description": "I'm superduperlyssa, I save lives and shit like that.", "followers_count": 927, "friends_count": 904, "statues_count": 8439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale, AZ", "id": "0015d9147cee6907", "name": "Avondale", "place_type": "city", "bounding_box": rectangle("-112.357999,33.384785 -112.272424,33.493806") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 404720, "cityName": "Avondale" } }
+{ "create_at": datetime("2016-01-10T00:00:03.000Z"), "id": 686095143686938624, "text": "❤️ Tammy https://t.co/eg8rtYBwDp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 962976048, "name": "Lyss.", "screen_name": "Alyssa_Layy23", "lang": "en", "location": "Brady, TX", "create_at": date("2012-11-21"), "description": "|Tarleton State University 20' |", "followers_count": 494, "friends_count": 325, "statues_count": 13396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095143934377984, "text": "#Byrd's Barbecue https://t.co/zslxeYY17S Excellent https://t.co/U85c7hwy7b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.840458,36.006218"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Byrd" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4277, "friends_count": 875, "statues_count": 374154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095143993274368, "text": "Wind 0.0 mph NE. Barometer 30.212 in, Falling. Temperature 33.6 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 0, "friends_count": 0, "statues_count": 3503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095144110571520, "text": "#Asian bowl / https://t.co/IoBP2T3CHO Their food is always fresh and tasty, Excellent, if https://t.co/NoCtxtMwAX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.721477,40.742306"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Asian" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4277, "friends_count": 875, "statues_count": 374154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095144165195776, "text": "Unexpected moves are the most fun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433659782, "name": "ChakshÜ", "screen_name": "BruvahChak", "lang": "en", "location": "Village Ave", "create_at": date("2011-12-10"), "description": "Good times ahead", "followers_count": 446, "friends_count": 304, "statues_count": 22971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery Village, MD", "id": "017a75c0406737c7", "name": "Montgomery Village", "place_type": "city", "bounding_box": rectangle("-77.231293,39.152407 -77.112229,39.235691") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2453325, "cityName": "Montgomery Village" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095144773382144, "text": "My roommate doesn't know what state is below North Dakota. #WVUeducation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WVUeducation" }}, "user": { "id": 1584979044, "name": "Scott Vasilisin", "screen_name": "buscottie", "lang": "en", "location": "null", "create_at": date("2013-07-10"), "description": "WVU '19 hopefully", "followers_count": 836, "friends_count": 854, "statues_count": 11656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095144941137920, "text": "Horror movies at 3 a.m with the besties", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175018984, "name": "Gage Black", "screen_name": "Gageandchill", "lang": "en", "location": "null", "create_at": date("2010-08-05"), "description": "21/FL/ I dont get hype i stay hyped. Family/Friends/coffee forever", "followers_count": 174, "friends_count": 115, "statues_count": 2670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, FL", "id": "0190556f56729d0c", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-82.723568,28.330333 -82.6197,28.422387") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1232825, "cityName": "Hudson" } }
+{ "create_at": datetime("2016-01-10T00:00:04.000Z"), "id": 686095144991473664, "text": "When the world feels cruel I'll be here for you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1481491112, "name": "Will Ray", "screen_name": "war_lily", "lang": "en", "location": "In this zoo of homo sapiens. ", "create_at": date("2013-06-03"), "description": "Paint your world the colors you wish to see. Light up the darkness.", "followers_count": 91, "friends_count": 133, "statues_count": 737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrboro, NC", "id": "c3a90b8ce84f216e", "name": "Carrboro", "place_type": "city", "bounding_box": rectangle("-79.109676,35.889723 -79.063905,35.964692") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3710620, "cityName": "Carrboro" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457515417202689, "text": "@yeseniaxdt this the song I'm thinking about using https://t.co/bOjvbZWvAg - mainly because the tempo and it has like a serious feel to it", "in_reply_to_status": 686457043985788928, "in_reply_to_user": 2306830158, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2306830158 }}, "user": { "id": 902905183, "name": "Pat Banahan", "screen_name": "PatBanahan", "lang": "en", "location": "null", "create_at": date("2012-10-24"), "description": "???", "followers_count": 1756, "friends_count": 451, "statues_count": 5949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516243480576, "text": "Please tell me this is a sick joke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331990739, "name": "bug", "screen_name": "babyybugg", "lang": "en", "location": "RVA", "create_at": date("2011-07-08"), "description": "Double bubble disco queen | ♊️", "followers_count": 262, "friends_count": 77, "statues_count": 53061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bon Air, VA", "id": "bf16dab6b130c4ca", "name": "Bon Air", "place_type": "city", "bounding_box": rectangle("-77.616202,37.495023 -77.523345,37.542652") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51041, "countyName": "Chesterfield", "cityID": 5108472, "cityName": "Bon Air" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516323024897, "text": "@Indihaitian half the week, just maybe.", "in_reply_to_status": 686457149006962688, "in_reply_to_user": 46586125, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46586125 }}, "user": { "id": 133521772, "name": "DeAnte Ambler", "screen_name": "GetemAmbler", "lang": "en", "location": "Somewhere Chasing My Dreams", "create_at": date("2010-04-15"), "description": "Fastest in the state of Kansas, on my way to success through the glory of God. #KcKcc #ATH Now watch me work. #TrackNation #Football", "followers_count": 1625, "friends_count": 1342, "statues_count": 100417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.845486,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516503470080, "text": "@LunnieTunes ������������", "in_reply_to_status": 686457347548512256, "in_reply_to_user": 144261140, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 144261140 }}, "user": { "id": 1387826197, "name": "Reanna", "screen_name": "thebadgirlriri", "lang": "en", "location": "ME", "create_at": date("2013-04-28"), "description": "null", "followers_count": 1742, "friends_count": 605, "statues_count": 10129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bangor, ME", "id": "bd73972b51647e05", "name": "Bangor", "place_type": "city", "bounding_box": rectangle("-68.872064,44.768404 -68.71876,44.855108") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23019, "countyName": "Penobscot", "cityID": 2302795, "cityName": "Bangor" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516583092224, "text": "Gotta wake up at 5 and I wanna sleep but my body is being more annoying than Kim Kardashian's voice and not letting me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1330322364, "name": "Savannah Cunningham", "screen_name": "SavCunningham", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2013-04-05"), "description": "Athlete. Idiot. ASU '19 - NROTC Marine Option. Student Body Fitness. Snapchat - cuttingham.", "followers_count": 743, "friends_count": 690, "statues_count": 6247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516687888384, "text": "Can you recommend anyone for this #job? Assistant Manager - https://t.co/e59dMtynCF #LoneGrove, OK #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.260279,34.1726151"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "LoneGrove", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 198, "friends_count": 39, "statues_count": 19117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma, USA", "id": "bd3d2074a33fbd06", "name": "Oklahoma", "place_type": "admin", "bounding_box": rectangle("-103.002648,33.615765 -94.431332,37.002328") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40019, "countyName": "Carter", "cityID": 4043750, "cityName": "Lone Grove" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516797104128, "text": "Don't start your car https://t.co/tpv533amc1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1925251910, "name": "lovepreet", "screen_name": "1ovepreet", "lang": "en", "location": "null", "create_at": date("2013-10-01"), "description": "stay frosty❄️❄️", "followers_count": 238, "friends_count": 152, "statues_count": 3290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516813905920, "text": "@allurfantasies more hidden rules, they got me for too many comments", "in_reply_to_status": 686457126403878912, "in_reply_to_user": 10337292, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 10337292 }}, "user": { "id": 34021970, "name": "Bronwyn West", "screen_name": "bronwyn53", "lang": "en", "location": "columbus, ohio", "create_at": date("2009-04-21"), "description": "I am looking forward to finding out about people and places, I am a 60 year old disabled woman", "followers_count": 1516, "friends_count": 1435, "statues_count": 4029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457516989878272, "text": "Don't make me cry please", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4162500020, "name": "Lando Kardashian", "screen_name": "Philteredsoul", "lang": "en", "location": "null", "create_at": date("2015-11-07"), "description": "upset twenty something", "followers_count": 19, "friends_count": 19, "statues_count": 1753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457517241569280, "text": "OH MY GOD https://t.co/Csl8ZgzXmL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341941004, "name": "syd ♛", "screen_name": "SydneyJaay", "lang": "en", "location": "null", "create_at": date("2011-07-24"), "description": "single black female addicted to retail", "followers_count": 1830, "friends_count": 1359, "statues_count": 33502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457517321224192, "text": "@loswhit ������������", "in_reply_to_status": 686457357887352832, "in_reply_to_user": 8262232, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 8262232 }}, "user": { "id": 14284323, "name": "Paul Bae", "screen_name": "realPB", "lang": "en", "location": "Colorado", "create_at": date("2008-04-02"), "description": "Technologist, digital and mobile strategy, ux, lean, value prop design, cro, founder. Millenium Falcons and Nachos.", "followers_count": 3428, "friends_count": 1046, "statues_count": 17744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457517694566400, "text": "I'm in love https://t.co/LnUGSFEYg0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 827608088, "name": "kennedy noden", "screen_name": "knodess", "lang": "en", "location": "null", "create_at": date("2012-09-16"), "description": "null", "followers_count": 1319, "friends_count": 499, "statues_count": 11075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518038630400, "text": "❤️������s @ Hubertus, Wisconsin https://t.co/MRQFceVEWZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.2211,43.2367"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1671750685, "name": "Bobby Sura", "screen_name": "BobbySura04", "lang": "en", "location": "Hubertus, WI", "create_at": date("2013-08-14"), "description": "null", "followers_count": 93, "friends_count": 1517, "statues_count": 2771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hubertus, WI", "id": "0050ebad2f839668", "name": "Hubertus", "place_type": "city", "bounding_box": rectangle("-88.289904,43.207483 -88.201845,43.265523") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55131, "countyName": "Washington", "cityID": 5567475, "cityName": "Richfield" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518143361024, "text": "Shit I accidentally sent a preview of a new song to ScHoolboy Q over SC��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163780284, "name": "renz", "screen_name": "RenzMelchor", "lang": "en", "location": "Puyallup, WA", "create_at": date("2010-07-07"), "description": "@j_swizzzzz | 18 years old | Never settle for less | New music on the way | USAF BOUND", "followers_count": 880, "friends_count": 475, "statues_count": 8996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Puyallup, WA", "id": "000dea80079d8b64", "name": "Puyallup", "place_type": "city", "bounding_box": rectangle("-122.336103,47.145624 -122.232425,47.207294") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356695, "cityName": "Puyallup" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518181056512, "text": "2016 is ruined. Rest in peace. https://t.co/XsP94c7Y7k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83519427, "name": "Nicolle Boschetti", "screen_name": "ColaBoschetti", "lang": "en", "location": "Bay Area, California", "create_at": date("2009-10-18"), "description": "Champion of the Sun, Master of Karate, and friendship for everyone.", "followers_count": 97, "friends_count": 118, "statues_count": 4150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518491602945, "text": "https://t.co/7pSMBGz3ce", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 18729182, "name": "Samuel L. Radford 3", "screen_name": "brothersam", "lang": "en", "location": "Buffalo", "create_at": date("2009-01-07"), "description": "President - Buffalo Schools District Parent Coordinating Council. Representing 58 Schools and 1300 Homerooms.", "followers_count": 514, "friends_count": 298, "statues_count": 748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518529187840, "text": "Yesssss hate this https://t.co/SDLi9emc6d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3379254569, "name": "Nat", "screen_name": "naatgrcia", "lang": "en", "location": "null", "create_at": date("2015-07-16"), "description": "GCU", "followers_count": 113, "friends_count": 159, "statues_count": 1054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518613110784, "text": "Lol ok going to sleep before I start getting back in my feels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1425669517, "name": "Marryyyy", "screen_name": "Mary_Fa4", "lang": "en", "location": "Layton, UT", "create_at": date("2013-05-13"), "description": "always got my brown bag", "followers_count": 485, "friends_count": 395, "statues_count": 2124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Layton, UT", "id": "792cda35481740af", "name": "Layton", "place_type": "city", "bounding_box": rectangle("-112.035861,41.036355 -111.899171,41.128724") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4943660, "cityName": "Layton" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518743109634, "text": "Update:It's 12 & I havent started the project and I'm tired Af. I think I'm going to take this L and beg my teacher for more time. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2624103304, "name": "Jaxon Roby™", "screen_name": "JaxonRoby", "lang": "en", "location": "Cali", "create_at": date("2014-06-19"), "description": "down to adventure?", "followers_count": 137, "friends_count": 209, "statues_count": 906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457518839738368, "text": "I miss the real that @its_darion is to be honest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 201483384 }}, "user": { "id": 1556720844, "name": "erinhaze", "screen_name": "PrincessPizzaaa", "lang": "en", "location": "Newport News, VA", "create_at": date("2013-06-29"), "description": "23. I am Beyoncé always.", "followers_count": 347, "friends_count": 241, "statues_count": 18959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-01-11T00:00:00.000Z"), "id": 686457519133200384, "text": "@Kalifornia_blue Not involuntary? Yet many blacks were subject to force castration. & Prog #Eugencis is a fact @libertyalliance", "in_reply_to_status": 686250789455581184, "in_reply_to_user": 3074083842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Eugencis" }}, "user_mentions": {{ 3074083842, 755319799 }}, "user": { "id": 10207502, "name": "Don Mashak", "screen_name": "DMashak", "lang": "en", "location": "Minneapolis, Minnesota USA", "create_at": date("2007-11-12"), "description": "Don Mashak,The Cynical Patriot #NaturalLaw #Constitution @WETHEPEOPLETAR @RLibertyCaucus #MILITARY #Militia #TEAParty #OWS @GOP @TheDemocrats See UserGuide Link", "followers_count": 83773, "friends_count": 91851, "statues_count": 308676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519250796544, "text": "It's nice to know that @xCaliGrrl thinks I am a major dork. XD :P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 374082921 }}, "user": { "id": 19187688, "name": "Greg McKay", "screen_name": "NoNameHere234", "lang": "en", "location": "null", "create_at": date("2009-01-19"), "description": "null", "followers_count": 186, "friends_count": 526, "statues_count": 5388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euclid, OH", "id": "59ed3d5f720ebdf0", "name": "Euclid", "place_type": "city", "bounding_box": rectangle("-81.552943,41.545336 -81.488014,41.631584") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3925704, "cityName": "Euclid" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519309336576, "text": "I'm never fucking with someone like her ever again. She's such a bad person and \"friend\" https://t.co/taT8vuyvc1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2861769007, "name": "Danielle Olmedo", "screen_name": "danielleolmedo1", "lang": "en", "location": "null", "create_at": date("2014-10-17"), "description": "Softball #33", "followers_count": 73, "friends_count": 90, "statues_count": 12237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519468756992, "text": "Love that only a few people know about my birthday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4065072254, "name": "Paislee Gilstrap", "screen_name": "thatgirlpaislee", "lang": "en", "location": "Forks, WA", "create_at": date("2015-10-29"), "description": "I'm just a happy fun bitch who likes to party. ✌️ snapchat me~ paaaaislee", "followers_count": 92, "friends_count": 152, "statues_count": 252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forks, WA", "id": "e12a9dbe2d1ab327", "name": "Forks", "place_type": "city", "bounding_box": rectangle("-124.419677,47.930365 -124.336879,47.975868") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53009, "countyName": "Clallam", "cityID": 5324810, "cityName": "Forks" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519523389440, "text": "You remind me of the babe.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373557625, "name": "jade", "screen_name": "jadeeedh", "lang": "en", "location": "null", "create_at": date("2011-09-14"), "description": "@danny_duke and @kimkardashian are my parents.", "followers_count": 453, "friends_count": 386, "statues_count": 7270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519640854528, "text": "A big hug to Jene for not passing Daffodil by! #GoodMan ��❤️�� https://t.co/7o8kw75G1r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoodMan" }}, "user": { "id": 426957556, "name": "Lisa Benway", "screen_name": "Swamper60", "lang": "en", "location": "The Hub of the Universe", "create_at": date("2011-12-02"), "description": "I'm a child of the stars, the blaster of the gods and I have seen it all before.... - A. Deris - Before the War", "followers_count": 1840, "friends_count": 1900, "statues_count": 43928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlborough, MA", "id": "fbe7a12a76a49216", "name": "Marlborough", "place_type": "city", "bounding_box": rectangle("-71.625705,42.311183 -71.475702,42.380659") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2538715, "cityName": "Marlborough" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457519930085376, "text": "i hadn't procrastinated on school for so long that feel horrible lately!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48279594, "name": "Rosalva", "screen_name": "rohsah", "lang": "en", "location": "null", "create_at": date("2009-06-18"), "description": "All our dreams can come true if we have the courage to pursue them", "followers_count": 145, "friends_count": 283, "statues_count": 6853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457520043339776, "text": "Bless kanye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 811852142, "name": "Leonskis", "screen_name": "ofthurnis", "lang": "en", "location": "donde esta la piksa ", "create_at": date("2012-09-08"), "description": "Broke Boyz, Single Dad Club, Sad boyz 2k15, \n 24 hour litness \n#FlamePosse\nHow's Business?", "followers_count": 502, "friends_count": 246, "statues_count": 35368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457520114634752, "text": "كندر بو اربع اصابع ياجماعه", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 1483934408, "name": "azyy⁷العجمي", "screen_name": "Aziz7xx", "lang": "en", "location": "kuwait-los angeles", "create_at": date("2013-06-04"), "description": "ادرس بامريكا يعني اتفلسف ع راحتي Arsenal will rise | snap:aziiz_kwt", "followers_count": 13578, "friends_count": 151, "statues_count": 8652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457520681021440, "text": "Wind 11.2 mph W. Barometer 29.761 in, Rising Rapidly. Temperature 23.3 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 5720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457520831909888, "text": "lately I feel like we lost touch, I wanna gain that back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465791959, "name": "✨", "screen_name": "Husslebbygirl", "lang": "en", "location": "Denver✈️", "create_at": date("2012-01-16"), "description": "null", "followers_count": 777, "friends_count": 517, "statues_count": 37040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457521196904448, "text": "Wind 0 mph --. Barometer 30.28 in, Steady. Temperature 30.0 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457521435901952, "text": "Hospitals are seriously not the best place to slept at", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770637475, "name": "❃", "screen_name": "krnasndvl", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "insta: krnasndvl", "followers_count": 523, "friends_count": 473, "statues_count": 1921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457521679147009, "text": "Vegas for the weekend ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91212005, "name": "M", "screen_name": "miguelfromca", "lang": "en", "location": "ca", "create_at": date("2009-11-19"), "description": "snapchat - miguelescalante", "followers_count": 376, "friends_count": 295, "statues_count": 50357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457521796583425, "text": "@CiscoNOChains ��������", "in_reply_to_status": 686457474774249472, "in_reply_to_user": 50214037, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 50214037 }}, "user": { "id": 251462110, "name": "Chicken", "screen_name": "ChickenColeman", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-02-12"), "description": "I stack my money, lay low, and chill.", "followers_count": 11396, "friends_count": 1014, "statues_count": 340533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522035638272, "text": "WAIT NO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1651125566, "name": "Vania♡", "screen_name": "Bbydarlingg", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-08-06"), "description": "Vania tamara ♡ \\(-^3^-)/\nSC: darlingvania", "followers_count": 115, "friends_count": 57, "statues_count": 4125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522157412353, "text": "Nope�� https://t.co/AEmNuwKy3j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2461424049, "name": "Taking Too Many L's", "screen_name": "_JulyAlsina", "lang": "en", "location": "null", "create_at": date("2014-04-04"), "description": "SC:july.alsina. SHS Graduate. FriendZone Jesus Pre Dental major Gentleman and Hopless romantic since age 5 Advocate of #PeaceLovePositivity #Tsu19", "followers_count": 797, "friends_count": 747, "statues_count": 16150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522220187648, "text": "The longer I looked at her, the deeper I looked into her and the sadder I felt.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2329785134, "name": "Blake", "screen_name": "Blake_Cole8", "lang": "en", "location": "Camp Pendleton South, CA", "create_at": date("2014-02-05"), "description": "United States Marine Corps.", "followers_count": 428, "friends_count": 368, "statues_count": 1781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522308386816, "text": "@MIA_NoHeat beautiful smile ���� https://t.co/qCsEWySC3q", "in_reply_to_status": -1, "in_reply_to_user": 147075331, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 147075331 }}, "user": { "id": 333532203, "name": "#RIPFLOSS", "screen_name": "That_TallGuy", "lang": "en", "location": "in the gym ", "create_at": date("2011-07-11"), "description": "They say the truth shall come to the light...so everybody grab ya shades cause ya boy that bright..Mia❤️", "followers_count": 867, "friends_count": 693, "statues_count": 45365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.933734 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522547367936, "text": "@Moziq_ idk haha either xJMx leaders have something against me or I'm just poop ������", "in_reply_to_status": 686457252224450560, "in_reply_to_user": 3230112908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3230112908 }}, "user": { "id": 2595169009, "name": "Jaay AX", "screen_name": "JaayKings", "lang": "en", "location": "Las Vegas", "create_at": date("2014-06-29"), "description": "Peace Love & Positivity", "followers_count": 1501, "friends_count": 149, "statues_count": 36517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522631229440, "text": "I JUST REMEMBERED I HAVE LEFT OVER PHO AT HOME FROM EARLIER. ������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 629140273, "name": "stubborn", "screen_name": "ayeee_tisha", "lang": "en", "location": "Denver, CO", "create_at": date("2012-07-07"), "description": "| island princess | Micronesian; Pohnpeian | Saipan ✈️ Pohnpei ✈️ Colorado | SC; http://tisha.boo |", "followers_count": 1088, "friends_count": 990, "statues_count": 24066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457522820116480, "text": "Wind 0.0 mph SSE. Barometer 30.225 in, Falling. Temperature 14.9 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457523117907968, "text": "have y'all had secks this year yet?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403029093, "name": "mouse lookin' ass", "screen_name": "imjusshere", "lang": "en", "location": "ETX", "create_at": date("2011-11-01"), "description": "us navy|rest in paradise @djkidd_ & @montytrillfam ❤️", "followers_count": 1491, "friends_count": 609, "statues_count": 45867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daingerfield, TX", "id": "89724db2e92e8ad8", "name": "Daingerfield", "place_type": "city", "bounding_box": rectangle("-94.745551,33.005735 -94.703541,33.053796") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48343, "countyName": "Morris", "cityID": 4818464, "cityName": "Daingerfield" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457523117912064, "text": "Letter of 4 pages front and back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1002822474, "name": "Queen", "screen_name": "BitchFuckkYou_", "lang": "en", "location": "La Vie Est Belle ", "create_at": date("2012-12-10"), "description": "#FreeJunior. Rest In Paradise Baby Gucci ❤", "followers_count": 999, "friends_count": 633, "statues_count": 46330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-01-11T00:00:01.000Z"), "id": 686457523197575168, "text": "Temp: 27.6°F Wind:0.0mph Pressure: 30.290hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457523642093568, "text": "The Bowie feels are real in the twittersphere #DavidBowie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DavidBowie" }}, "user": { "id": 124907516, "name": "DABLACKSWAN", "screen_name": "DaBlackSwan", "lang": "en", "location": "Chicago & Los Angeles", "create_at": date("2010-03-20"), "description": "The most underrated photographers you've never met. Contact: Jackie@be-artists.com", "followers_count": 422, "friends_count": 623, "statues_count": 2382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524241870848, "text": "@DeluxeSwank the charging port will be the only port in the phone & the adapter will plug in2 it and the headphones plug into the adapter", "in_reply_to_status": 686456584466075648, "in_reply_to_user": 450403394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450403394 }}, "user": { "id": 212055536, "name": "Yanye.Bigcartel", "screen_name": "__YanYe__", "lang": "en", "location": "Kabylie Mountains", "create_at": date("2010-11-04"), "description": "I am a Berber Prince...One day I will be #King #Kabylie #RoyalBlood #UIUC16", "followers_count": 496, "friends_count": 455, "statues_count": 12290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524262850560, "text": "A woman who was unhappy with a wealthy husband. https://t.co/GnUdepiIgn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245955509, "name": "Zakyla", "screen_name": "TheColdCaseLove", "lang": "en", "location": "DDD--SM, TX", "create_at": date("2011-02-01"), "description": "Stone Cold Fox. Texas State. instagram:bombshell_bby", "followers_count": 720, "friends_count": 879, "statues_count": 15815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524355125248, "text": "�� https://t.co/29uSF4CwQm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2378391259, "name": "✨ Mutealduuuh ✨", "screen_name": "laritsunami", "lang": "en", "location": "yo daddy house", "create_at": date("2014-03-08"), "description": "savage & an angel", "followers_count": 752, "friends_count": 326, "statues_count": 31899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524367802368, "text": "This hard b. https://t.co/hDvE7SZDE2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334506877, "name": "Riccardo Rocha", "screen_name": "das_rich", "lang": "en", "location": "Louisville ⚜", "create_at": date("2011-07-12"), "description": "whats your vice?", "followers_count": 1516, "friends_count": 472, "statues_count": 18669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jeffersontown, KY", "id": "258c7428a48a8cfa", "name": "Jeffersontown", "place_type": "city", "bounding_box": rectangle("-85.60709,38.143069 -85.508139,38.246418") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2140222, "cityName": "Jeffersontown" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524489449476, "text": "#SupportOriginMelissa 12.7°F Wind:2.0mph Pressure: 30.18hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524581695488, "text": "\"kids that come up afterwards and say, 'It's cool you're doing a Nirvana song.' And I think, 'Fuck you, you little tosser!\" - #David Bowie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "David" }}, "user": { "id": 102054247, "name": "Ian Kerr", "screen_name": "xkerrianx", "lang": "en", "location": "Azusa", "create_at": date("2010-01-05"), "description": "Atheist, straight edge, ass hole, I give no fucks, photographer, From El Monte Ca", "followers_count": 45, "friends_count": 45, "statues_count": 1974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincent, CA", "id": "7ad2e48aebe2e3ea", "name": "Vincent", "place_type": "city", "bounding_box": rectangle("-117.942045,34.084679 -117.907744,34.114313") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682815, "cityName": "Vincent" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457524581724161, "text": "01/11@03:00 - Temp 33.7F, WC 28.4F. Wind 5.8mph NW, Gust 16.0mph. Bar 29.872in, Rising quickly. Rain 0.00in. Hum 56%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525072474112, "text": "Thank @YogurtlandLV for providing delicious pistachio frozen yogurt. #itsmyfavorite #yum", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "itsmyfavorite", "yum" }}, "user_mentions": {{ 37365141 }}, "user": { "id": 45877773, "name": "Jessica Isaacs", "screen_name": "j13isaacs", "lang": "en", "location": "null", "create_at": date("2009-06-09"), "description": "null", "followers_count": 173, "friends_count": 177, "statues_count": 7285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525127016450, "text": "Wind 0.0 mph WSW. Barometer 30.252 in, Rising slowly. Temperature 8.5 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 39, "friends_count": 25, "statues_count": 16788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525353451520, "text": "You shady lmaoooo https://t.co/wQxjOjHc3j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261287903, "name": "Al", "screen_name": "lowkeyal_", "lang": "en", "location": "Buffalo, NY", "create_at": date("2011-03-05"), "description": "SC : dreamchaser95", "followers_count": 970, "friends_count": 891, "statues_count": 113567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525441400832, "text": "me always but especially now https://t.co/0c4iHJ8akv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262485686, "name": "Victor C. Valle", "screen_name": "VictorCValle", "lang": "en", "location": "California, USA", "create_at": date("2011-03-07"), "description": "Founder, @CounterCGroup | Production Associate, @TheBayBridged | Writer, musician, creative", "followers_count": 2908, "friends_count": 746, "statues_count": 10035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525818900481, "text": "Fuck it imma just turn on post notifications for my cousin @Yunngbg because we're basically the same person w/ the same thoughts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3253314638 }}, "user": { "id": 2734553613, "name": "Enija Wilson", "screen_name": "enija_rea", "lang": "en", "location": "Tex ✈Cali", "create_at": date("2014-08-05"), "description": "MHHS Track & Field", "followers_count": 267, "friends_count": 315, "statues_count": 1688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain House, CA", "id": "014b25c473600d50", "name": "Mountain House", "place_type": "city", "bounding_box": rectangle("-121.557176,37.766633 -121.533126,37.795236") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 649582, "cityName": "Mountain House" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525844086784, "text": "Makes me feel better.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513896062, "name": "Tavo", "screen_name": "GustavoAmparan", "lang": "en", "location": "Thornton, CO", "create_at": date("2012-03-03"), "description": "18.\r\nMetro. Just your average guy trying to get somewhere in life. Don't take my tweets seriously.", "followers_count": 279, "friends_count": 385, "statues_count": 10765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525902782464, "text": "Lmao that's the homegirl https://t.co/atcFbSNKBl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2721036961, "name": "Eric Wright Jr.", "screen_name": "Kentetheking", "lang": "en", "location": "Norf Rialto", "create_at": date("2014-08-09"), "description": "Rest Easy CJ", "followers_count": 1034, "friends_count": 782, "statues_count": 39814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525911306240, "text": "2 Canada Goose (Branta canadensis) - Sam Lena Park and KERP - 2016-01-10 10:02 https://t.co/HOSzEVMyvp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.932,32.1812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10190 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457525919739905, "text": "2 Canada Goose (Branta canadensis) - Sam Lena Park and KERP - 2016-01-10 15:05 https://t.co/HOSzEVMyvp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.932,32.1812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10190 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457526104276992, "text": "1 y 4 https://t.co/clneg1Wupk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1849941163, "name": "yαni ✨", "screen_name": "yanishka_123", "lang": "es", "location": "Puerto Rico", "create_at": date("2013-09-09"), "description": "I leαrned to leαve αll the pαst behind , αnd stαrt getting better results. ~ ᒍℓ", "followers_count": 1295, "friends_count": 1747, "statues_count": 21761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457526267719680, "text": "I have a bruise from climbing into a jeep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1955383626, "name": "#5 hae", "screen_name": "lmaooohae", "lang": "en", "location": "@ the field ", "create_at": date("2013-10-11"), "description": "i love niall horan // im a mothafuckin princess", "followers_count": 513, "friends_count": 1393, "statues_count": 7548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457526917844992, "text": "Get that shit ✔️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1395826136, "name": "Robert III", "screen_name": "JrZaragoza34", "lang": "en", "location": "null", "create_at": date("2013-05-01"), "description": "GUSTINE HIGH•Check out my highlight: http://www.hudl.com/athlete/4027639/highlights/316131382 | GJC Commit |", "followers_count": 436, "friends_count": 1076, "statues_count": 4522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gustine, CA", "id": "7aa9a706633e9344", "name": "Gustine", "place_type": "city", "bounding_box": rectangle("-121.021006,37.245755 -120.985264,37.261648") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 631568, "cityName": "Gustine" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457527014297600, "text": "@briannguyoons !!!!! https://t.co/ffif2La7p6", "in_reply_to_status": -1, "in_reply_to_user": 2740594267, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2740594267 }}, "user": { "id": 1602688345, "name": "em", "screen_name": "_emilymullins", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-17"), "description": "null", "followers_count": 505, "friends_count": 157, "statues_count": 4565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457527073026048, "text": "I don't have school but I still have to be up early for work ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1700869844, "name": "YADIE", "screen_name": "yahhdee_", "lang": "en", "location": "UNR : ΑΟΠ ", "create_at": date("2013-08-25"), "description": "null", "followers_count": 1216, "friends_count": 445, "statues_count": 15114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457527173685248, "text": "*cries rewatching the series finale of Hannah Montana*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770935213, "name": "Lindsey Switzer", "screen_name": "LindseyyMaeee", "lang": "en", "location": "null", "create_at": date("2012-08-20"), "description": "yikes", "followers_count": 791, "friends_count": 693, "statues_count": 14119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ahtanum, WA", "id": "fc59f50f16715910", "name": "Ahtanum", "place_type": "city", "bounding_box": rectangle("-120.66929,46.543707 -120.546154,46.574851") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5300800, "cityName": "Ahtanum" } }
+{ "create_at": datetime("2016-01-11T00:00:02.000Z"), "id": 686457527442272257, "text": "@caine_zach �������� sad", "in_reply_to_status": 686457440536256512, "in_reply_to_user": 3071797619, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3071797619 }}, "user": { "id": 626043005, "name": "Coach Zeke", "screen_name": "ZekeMerritt", "lang": "en", "location": "Morehead, KY", "create_at": date("2012-07-03"), "description": "Future Morehead State Eagle D-line if i was any bigger id be illgeal or a human wrecking ball jeremiah1:5", "followers_count": 795, "friends_count": 820, "statues_count": 3888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, KY", "id": "c30940eeda5beb70", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-87.214138,37.185216 -87.16083,37.23049") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21177, "countyName": "Muhlenberg", "cityID": 2133022, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457527677120512, "text": "the universe has existed for billions of years and by a coincidence we ended up living at the same time. that counts for something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2671194375, "name": "Savalicious", "screen_name": "Young_savvy69", "lang": "en", "location": "null", "create_at": date("2014-07-04"), "description": "adventuring through this wonderland called life. guud vibes only.", "followers_count": 186, "friends_count": 303, "statues_count": 2241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457527802970112, "text": "@ELaufy AWWWWW", "in_reply_to_status": 686457419791233025, "in_reply_to_user": 581476862, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 581476862 }}, "user": { "id": 222223608, "name": "Tiff ♕", "screen_name": "t_robss", "lang": "en", "location": "CLE", "create_at": date("2010-12-02"), "description": "19. ♌️ Chillin", "followers_count": 2266, "friends_count": 881, "statues_count": 59604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentor, OH", "id": "a47c7932f8997705", "name": "Mentor", "place_type": "city", "bounding_box": rectangle("-81.391672,41.640241 -81.288901,41.75771") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3949056, "cityName": "Mentor" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457528339697664, "text": "Ommmmg I love jeras cover photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3102970380, "name": "Annekun08", "screen_name": "JamoreAnnethy", "lang": "en", "location": "null", "create_at": date("2015-03-22"), "description": "#Jayjay", "followers_count": 121, "friends_count": 72, "statues_count": 2856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dubuque, IA", "id": "7704cecc1f8f59e3", "name": "Dubuque", "place_type": "city", "bounding_box": rectangle("-90.781325,42.442191 -90.639614,42.579121") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19061, "countyName": "Dubuque", "cityID": 1922395, "cityName": "Dubuque" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457528561963008, "text": "������������ https://t.co/Z5UDyZeuup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 297289847, "name": "⛽ $auce McJames ❄", "screen_name": "PalmiroJR", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-05-12"), "description": "Ran Off On The Plug Twice!!", "followers_count": 1712, "friends_count": 2088, "statues_count": 102001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457528574734337, "text": "@isojoe_ women on here legit upset cus we love our moms, what type of shit is that??", "in_reply_to_status": 686457321547956224, "in_reply_to_user": 2468561712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2468561712 }}, "user": { "id": 30668975, "name": "Pat Brando", "screen_name": "PorscheofSports", "lang": "en", "location": "D(M)V", "create_at": date("2009-04-12"), "description": "20. Giants | Nuggets | Barcelona | #YEEZYSEASONHASARRIVED | Anime & Manga Nerd | #BlackLivesMatter | If you want to see anime fights check my favorites.", "followers_count": 2725, "friends_count": 1989, "statues_count": 209031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanham, MD", "id": "019ae9b86255a185", "name": "Lanham", "place_type": "city", "bounding_box": rectangle("-76.885508,38.934439 -76.821939,38.988672") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445550, "cityName": "Lanham" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457528692137984, "text": "I deserve what I've been missing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 376413695, "name": "Jayla✨", "screen_name": "JaylaCheree", "lang": "en", "location": "Indiana", "create_at": date("2011-09-19"), "description": "⚠️CAUTION⚠️tweets may contain sensitive and or offensive content, FOLLOW/TWATCH AT YOUR OWN RISK|RIP Nanny|", "followers_count": 2363, "friends_count": 986, "statues_count": 167729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anderson, IN", "id": "dc25bb079ef6daf2", "name": "Anderson", "place_type": "city", "bounding_box": rectangle("-85.749418,40.031713 -85.595836,40.179084") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18095, "countyName": "Madison", "cityID": 1801468, "cityName": "Anderson" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457528868196353, "text": "@nickaguilar101 no", "in_reply_to_status": 686457209706840064, "in_reply_to_user": 3430501294, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3430501294 }}, "user": { "id": 888180181, "name": "Leugim", "screen_name": "Miguel_Zalez", "lang": "en", "location": "null", "create_at": date("2012-10-17"), "description": "null", "followers_count": 215, "friends_count": 164, "statues_count": 5269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530134966272, "text": "Wind 7.6 mph NW. Barometer 29.67 in, Rising quickly. Temperature 27.7 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 92, "statues_count": 156590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530189393921, "text": "Lmao she coo I guess ...I will never understand why she play games so much ... https://t.co/UCwkFrT3Vn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938581392, "name": "Black Dynamite〽️", "screen_name": "raysteeze", "lang": "en", "location": "Corona, CA", "create_at": date("2014-12-21"), "description": "rajon j | 18 | Track x Field |", "followers_count": 536, "friends_count": 524, "statues_count": 4525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530290061312, "text": "35.1F (Feels: 35.1F) - Humidity: 85% - Wind: 0.0mph --- - Gust: 2.2mph - Pressure: 1038.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 221955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530331959296, "text": "I'm so excited to see what God has in store for them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2713386037, "name": "Avocado", "screen_name": "maryeezyboo", "lang": "en", "location": "null", "create_at": date("2014-08-06"), "description": "Just your typical Jesus loving gal Snapchat:Maryeezyboo", "followers_count": 231, "friends_count": 229, "statues_count": 7784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530432811008, "text": "LMAO GOODNIGHT RIP ��♿️ https://t.co/ZHWNAjIa8Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1692327403, "name": "itty bitty pretty ❣", "screen_name": "Whosamber__x", "lang": "en", "location": "null", "create_at": date("2013-08-22"), "description": "An inch of time is an inch of gold ☯", "followers_count": 355, "friends_count": 249, "statues_count": 7221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530688536576, "text": "Ikeep checking the easy poll website to see what city,state is winning the highest vote right now is San FRANSICO yay @Browland22 @210Manny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3463610774, 2540295865 }}, "user": { "id": 3308465659, "name": "alexus rowland♡", "screen_name": "alexuspower", "lang": "en", "location": "null", "create_at": date("2015-08-06"), "description": "HUNTER IS DADDY BRANDON IS XL DADDY♡ @alexusrowland", "followers_count": 94, "friends_count": 94, "statues_count": 1039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530747195392, "text": "Happiest of birthdays to my birthday twin @DelaneyLobo ❤️���� hope you have a wonderful day beautiful ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2287741184 }}, "user": { "id": 2443233817, "name": "january12", "screen_name": "victoriaeiffert", "lang": "en", "location": "null", "create_at": date("2014-04-13"), "description": "sixteen//Junior RCHS Drama", "followers_count": 246, "friends_count": 444, "statues_count": 3643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457530864644096, "text": "@speeshxnella God damn it", "in_reply_to_status": 686457321602428929, "in_reply_to_user": 161187482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161187482 }}, "user": { "id": 206308347, "name": "resurgam", "screen_name": "pab_xo", "lang": "en", "location": "xoxxxoooxo", "create_at": date("2010-10-22"), "description": "UCSB '19", "followers_count": 1649, "friends_count": 447, "statues_count": 90983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457531300888577, "text": "Yep auntie I will �� https://t.co/eXVXqE6Myl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296021498, "name": "ILoveYouClyde&Devona", "screen_name": "ILoveMySpaldinq", "lang": "en", "location": "In Jesus Hands ", "create_at": date("2011-05-09"), "description": "#TAMUC #MerceVegas #1525forever #RIPClyde #RIPAuntBub", "followers_count": 1057, "friends_count": 1299, "statues_count": 29806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenn Heights, TX", "id": "63634cb1acb5eff7", "name": "Glenn Heights", "place_type": "city", "bounding_box": rectangle("-96.891055,32.518192 -96.821826,32.568093") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829840, "cityName": "Glenn Heights" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457531368083456, "text": "My whole tumblr is of the us women's soccer team����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1284087085, "name": "lyss", "screen_name": "hopealyssaa", "lang": "en", "location": "bay area, california", "create_at": date("2013-03-20"), "description": "you are beautiful!!! don't let anyone tell you otherwise", "followers_count": 354, "friends_count": 199, "statues_count": 12692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Discovery Bay, CA", "id": "309e4f83032e9413", "name": "Discovery Bay", "place_type": "city", "bounding_box": rectangle("-121.641476,37.889142 -121.586181,37.932788") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 619339, "cityName": "Discovery Bay" } }
+{ "create_at": datetime("2016-01-11T00:00:03.000Z"), "id": 686457531506425856, "text": "@WayneBogda @DJGreenwald no offense but that's the dumbest statement yet-the opposite of what we want. DC is country club of insider trading", "in_reply_to_status": 686456347743797250, "in_reply_to_user": 111225697, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 111225697, 23919418 }}, "user": { "id": 2875247644, "name": "Shes Texan", "screen_name": "snorkidoodle", "lang": "en", "location": "DFW Texas", "create_at": date("2014-11-13"), "description": "Conservative Christian, American, Writer, Hater of Evil, Lover of God's Creatures. If you are a patriot, I'll follow you back! #EndGovCorruption #TedCruz2016", "followers_count": 669, "friends_count": 664, "statues_count": 3453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wylie, TX", "id": "5b96c12e41e49aa5", "name": "Wylie", "place_type": "city", "bounding_box": rectangle("-96.596044,32.981938 -96.489165,33.054983") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4880356, "cityName": "Wylie" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457531909156864, "text": "Asian bitches on the timeline slandering darkskin niggas smh ole octopus eating ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2229229802, "name": "HoodRich", "screen_name": "BrokeHomieLarry", "lang": "en", "location": "west baltimore", "create_at": date("2013-12-03"), "description": "Snapchat-LarryGotNoChill R.I.P Jamel", "followers_count": 13781, "friends_count": 2310, "statues_count": 29754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457532244623360, "text": "@Joccy91 yeah..I was expecting so many retweets but no..like with\nAnd yeah girl omg! Too cute!", "in_reply_to_status": 686457204098990080, "in_reply_to_user": 174727210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174727210 }}, "user": { "id": 287065488, "name": "Norma ❤ #LOE2K16", "screen_name": "KissMeOppa123", "lang": "en", "location": "Los Angles, California", "create_at": date("2011-04-24"), "description": "BABY♥ IGOT7 ❤ Starlight❤BANA❤ ARMY ❤ Inspirit ❤ Melody ❤ Unicorn ❤ #ForeverWithBAP #JusticeForBAP #ForeverOT6 \nSeventeen stan ❤ DAY6", "followers_count": 385, "friends_count": 1100, "statues_count": 17723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457532513124354, "text": "Damn rason bn gone for two whole yrs shit still hurt like it was yesterday i hate he gone man ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71371171, "name": "dynBtheReason", "screen_name": "ILuhMakinYLsicc", "lang": "en", "location": "Tn", "create_at": date("2009-09-03"), "description": "i can only speak for myself (B)igger (B)etter & (W)etter #the Real 1 #F.O.E. #GOD is amazing #bitches b mad #Ima b here #GV raised me", "followers_count": 352, "friends_count": 469, "statues_count": 48461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge, TN", "id": "4ff0fc2a304db9ad", "name": "Oak Ridge", "place_type": "city", "bounding_box": rectangle("-84.347226,35.962375 -84.16569,36.073648") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47001, "countyName": "Anderson", "cityID": 4755120, "cityName": "Oak Ridge" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457532643057665, "text": "I should be the poster child for letting go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2388059498, "name": "lil quesadilla", "screen_name": "_aubz", "lang": "en", "location": "satx premed #utsa19 ", "create_at": date("2014-03-13"), "description": "pockets green like piccolo", "followers_count": 1304, "friends_count": 633, "statues_count": 48275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457532961820673, "text": "@therealelp If only the Liquor Stores didn't close at midnight where I live. I can't deal right now.", "in_reply_to_status": 686456890310574080, "in_reply_to_user": 20922014, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20922014 }}, "user": { "id": 488674429, "name": "Rick Bastardly", "screen_name": "Cambablam", "lang": "en", "location": "Minot, ND", "create_at": date("2012-02-10"), "description": "Wannabe writer/rapper Who wants to produce my shit?", "followers_count": 66, "friends_count": 299, "statues_count": 1054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minot, ND", "id": "2ae0ed991bd2e8d6", "name": "Minot", "place_type": "city", "bounding_box": rectangle("-101.357743,48.182195 -101.178137,48.283425") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38101, "countyName": "Ward", "cityID": 3853380, "cityName": "Minot" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457533259644928, "text": "late nights w no one to talk to", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1064952613, "name": "EM HiGHROLLER", "screen_name": "emilyparkerr3", "lang": "en", "location": "rest in peace cam ", "create_at": date("2013-01-05"), "description": "take me somewhere far - sc:emilyann177 - bkk", "followers_count": 1910, "friends_count": 861, "statues_count": 30897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457533507219456, "text": "Wind 0.3 mph ESE. Barometer 30.273 in, Steady. Temperature 25.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457533549133824, "text": "Sometimes I can't sleep, I be up chasing dreams it's just me and my team", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1306061688, "name": "Shaggy", "screen_name": "iam_amess", "lang": "en", "location": "null", "create_at": date("2013-03-26"), "description": "I never really have things planned out, shit just happens. You can catch me whippin it in the civic. Flagstaff, AZ.", "followers_count": 504, "friends_count": 438, "statues_count": 8114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457534018879488, "text": "*realizes Detroit isn't in Chicago* like Chicago is a state or somethin lmao tf? https://t.co/dNX995PBgP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377495698, "name": "Tyvilyn Escobedo", "screen_name": "_tybedo20", "lang": "en", "location": "IL - CO", "create_at": date("2011-09-21"), "description": "❁ please don't feed the hippies ❁ • 217 ✈️ 970 • Colorado Mesa University • Illinois born and raised • snapchat: tybedo20 • stay golden, ponyboy • mixed af bye", "followers_count": 5489, "friends_count": 2354, "statues_count": 63039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, IL", "id": "252db32a3bbc79c2", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-87.722497,40.104956 -87.53169,40.204494") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17183, "countyName": "Vermilion", "cityID": 1718563, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457534065029121, "text": "New post: Trying Something Different https://t.co/vkfqJXAcZy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2203456731, "name": "katie", "screen_name": "KatieVibes", "lang": "en-gb", "location": "UK", "create_at": date("2013-12-01"), "description": "I like books and lipstick. Blogger extraordinaire & Entertainment Editor for @TridentMediaUK #GoVegan", "followers_count": 771, "friends_count": 689, "statues_count": 5739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457534354296832, "text": "Nooooooo!! Bowie!!!!!! <\\3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 38591173, "name": "Guillermo Arteaga", "screen_name": "xxMemoxx", "lang": "en", "location": "San Francisco", "create_at": date("2009-05-07"), "description": "From Los Angeles to San Francisco. Enjoying the little things.", "followers_count": 132, "friends_count": 186, "statues_count": 1088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457534400475136, "text": "Bro Oomf ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 985820268, "name": "Tonio", "screen_name": "liljtisfaster", "lang": "en", "location": "Flexin", "create_at": date("2012-12-02"), "description": "|#RIPBryan|#RIPDJH5|RIPGranny\nFinessin' Is A Blessin' ™\nDallas Tx", "followers_count": 1168, "friends_count": 1039, "statues_count": 44585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pecan Hill, TX", "id": "00dab9137c8138eb", "name": "Pecan Hill", "place_type": "city", "bounding_box": rectangle("-96.830668,32.470899 -96.771594,32.502592") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4856485, "cityName": "Pecan Hill" } }
+{ "create_at": datetime("2016-01-11T00:00:04.000Z"), "id": 686457534413049857, "text": "@zilovesnamjoon THEYRE ALL SO SWEET AND EMOTIONAL", "in_reply_to_status": 686457363889405952, "in_reply_to_user": 3349875484, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3349875484 }}, "user": { "id": 409755570, "name": "noemi", "screen_name": "iKONmonstaX", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-11-10"), "description": "iKON/MX...... the softest bobby stan.", "followers_count": 3881, "friends_count": 3953, "statues_count": 27339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819903249096704, "text": "This \"Making a Murder\" series on Netflix reaaaaaally makes me wanna become a lawyer ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417528899, "name": "illMATTic", "screen_name": "mattcarle12", "lang": "en", "location": "Covina, CA", "create_at": date("2011-11-20"), "description": "shit is real and everyday could be your last in the Jungle", "followers_count": 329, "friends_count": 568, "statues_count": 5484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819904348016640, "text": "I don't enjoy falling asleep upset :,(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189415852, "name": ":-)", "screen_name": "Heatheroin", "lang": "en", "location": "null", "create_at": date("2010-09-10"), "description": "null", "followers_count": 1571, "friends_count": 420, "statues_count": 29863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819904465453056, "text": "When people try to talk shit when I'm gone but when I'm there they want no part of me @sarahlynnsimas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 811896918 }}, "user": { "id": 1065281424, "name": "jojo", "screen_name": "JoJomcdaniel23", "lang": "en", "location": "null", "create_at": date("2013-01-06"), "description": "You only live once, but if you do it right once is enough.", "followers_count": 1052, "friends_count": 745, "statues_count": 4357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819904721293317, "text": "Missing Giselle ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 140171345, "name": "phuckyobaelist", "screen_name": "tootrilltor", "lang": "en", "location": "Jamaica", "create_at": date("2010-05-04"), "description": "who knew from the concrete a flower would grow. -John14:1", "followers_count": 1754, "friends_count": 119, "statues_count": 54111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819905094615040, "text": "lmao jk i guess", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 722734814, "name": "LIBÆ", "screen_name": "LibbyMatkin1", "lang": "en", "location": "Lava Hookah Lounge ", "create_at": date("2012-07-28"), "description": "i thank god almighty for titties and beer. \n⚠may contain offensive content⚠@devinhendrixx", "followers_count": 1128, "friends_count": 2003, "statues_count": 17564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534889,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819905383997440, "text": "This snap gives me life https://t.co/p0tfyuyRZp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22215156, "name": "ashhhh", "screen_name": "ashtwee_", "lang": "en", "location": "San Antonio / Mission, TX", "create_at": date("2009-02-27"), "description": "live deliberately ✨", "followers_count": 366, "friends_count": 138, "statues_count": 14396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819906113777665, "text": "feels�� https://t.co/n71p5L5cJp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3248153190, "name": "Libbykinss", "screen_name": "MammaTibb", "lang": "en", "location": "The Bayyyy", "create_at": date("2015-06-17"), "description": "Aaliyah Wray Smith;❤️Loyalty is thicker than blood.", "followers_count": 196, "friends_count": 320, "statues_count": 17926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819906185113601, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 493583907, "name": "Jayheebs", "screen_name": "J_Hebert7", "lang": "en", "location": "SMJ Jan 19, 1982 - Nov 7, 2005", "create_at": date("2012-02-15"), "description": "null", "followers_count": 516, "friends_count": 360, "statues_count": 25375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnwood, WA", "id": "3912e7cd4cf62c39", "name": "Lynnwood", "place_type": "city", "bounding_box": rectangle("-122.337717,47.802196 -122.262575,47.85379") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5340840, "cityName": "Lynnwood" } }
+{ "create_at": datetime("2016-01-12T00:00:00.000Z"), "id": 686819906445180928, "text": "@akadeathwish needless to say, I think the person I subtweeted this about was in the wrong. Lol", "in_reply_to_status": 686165289239732226, "in_reply_to_user": 1515564157, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1515564157 }}, "user": { "id": 1515564157, "name": "sweatpants", "screen_name": "akadeathwish", "lang": "en", "location": "Bedford, TX", "create_at": date("2013-06-13"), "description": "Mayson | 21 | TX | IG: akadeathwish | water is the best", "followers_count": 1318, "friends_count": 472, "statues_count": 71507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coppell, TX", "id": "8b14cd99b986ef15", "name": "Coppell", "place_type": "city", "bounding_box": rectangle("-97.032812,32.922617 -96.932765,32.993893") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4816612, "cityName": "Coppell" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819907183312897, "text": "ain't anyone gaf anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3264804776, "name": "Dustin", "screen_name": "Dvstywebb", "lang": "en", "location": "at the bottom", "create_at": date("2015-07-01"), "description": "I don't know where I'm going from here, but I promise it won't be boring.", "followers_count": 207, "friends_count": 227, "statues_count": 251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819907267252224, "text": "Ok. I'm done. It's midnight. New day. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63395129, "name": "Miranda", "screen_name": "mirlarrin", "lang": "en", "location": "PNW ", "create_at": date("2009-08-05"), "description": "HeavyDirtySoul ✌️", "followers_count": 275, "friends_count": 228, "statues_count": 30441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Port Orchard, WA", "id": "395bff59b995bc8d", "name": "East Port Orchard", "place_type": "city", "bounding_box": rectangle("-122.644833,47.504816 -122.609651,47.534027") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5319770, "cityName": "East Port Orchard" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819907615330304, "text": "I hate hate hate hate hate hate hate texting . Just call or ft me . You can even snap me . I just don't wanna text ever lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2933807558, "name": "justice .", "screen_name": "jsticey", "lang": "en", "location": "null", "create_at": date("2014-12-17"), "description": "I'm right . you're wrong.", "followers_count": 494, "friends_count": 282, "statues_count": 3900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819907732914177, "text": "01/12@03:00 - Temp 21.6F, WC 21.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.124in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819907778940929, "text": "@hylanpp_ ready for yo ass got her on speed dial https://t.co/Np45rnC9B3", "in_reply_to_status": 686818797693448193, "in_reply_to_user": 1268820062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1268820062 }}, "user": { "id": 471259101, "name": "OPM™", "screen_name": "_RunMeMyMula", "lang": "en", "location": "null", "create_at": date("2012-01-22"), "description": "counting money ain't the same if you ain't get it on your own | IG:_RunMeMyMula | SC:Vsvp_melo | IE✊", "followers_count": 611, "friends_count": 498, "statues_count": 15529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Banning, CA", "id": "792551bc9bd3c992", "name": "Banning", "place_type": "city", "bounding_box": rectangle("-116.947005,33.902607 -116.849846,33.94771") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 603820, "cityName": "Banning" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819908135432192, "text": "People need to remind me not to eat tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3524041634, "name": "Biggs✨", "screen_name": "_littlestbiggs", "lang": "en", "location": "null", "create_at": date("2015-09-10"), "description": "Do you actually think you're special?", "followers_count": 215, "friends_count": 354, "statues_count": 3401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819908340957185, "text": "Booyyyyyy ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2616844850, "name": "jazzy", "screen_name": "jazmine_farjeat", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "null", "followers_count": 421, "friends_count": 341, "statues_count": 1698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincent, CA", "id": "7ad2e48aebe2e3ea", "name": "Vincent", "place_type": "city", "bounding_box": rectangle("-117.942045,34.084679 -117.907744,34.114313") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682815, "cityName": "Vincent" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819908794118144, "text": "Wind 0 mph --. Barometer 30.22 in, Falling slowly. Temperature 31.3 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819908978475008, "text": "don't tell you enough, but baby i'll show it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381877532, "name": "lame", "screen_name": "haydenharker5", "lang": "en", "location": "in bed", "create_at": date("2011-09-28"), "description": "my friends call me hay hooker", "followers_count": 541, "friends_count": 352, "statues_count": 19526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaanapali, HI", "id": "386eee0c027ac681", "name": "Kaanapali", "place_type": "city", "bounding_box": rectangle("-156.696704,20.911087 -156.674143,20.950546") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1520000, "cityName": "Kaanapali" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819910249517056, "text": "Primera vez que dicen que soy dulce, aplaudamos. https://t.co/P5rWt9zfUZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 425140443, "name": "Mariana Pineda.", "screen_name": "Marianapineda23", "lang": "es", "location": "null", "create_at": date("2011-11-30"), "description": "null", "followers_count": 2416, "friends_count": 763, "statues_count": 68330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-12T00:00:01.000Z"), "id": 686819910878547968, "text": "Ima just dead the issue it wasn't anything special anyway ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 930879235, "name": "TROUBLE", "screen_name": "MostEnviedKee_", "lang": "en", "location": "Y$L Apparel ✨", "create_at": date("2012-11-06"), "description": "the good ones go if you wait too long....", "followers_count": 1184, "friends_count": 1114, "statues_count": 42055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819911444860928, "text": "see https://t.co/FutCIEO7MB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 532005612, "name": "B.", "screen_name": "nitaindacut", "lang": "en", "location": "here", "create_at": date("2012-03-20"), "description": "i got coke u can sniff it off ya nail ya", "followers_count": 1734, "friends_count": 1516, "statues_count": 26490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay Minette, AL", "id": "a3cb5948de02bfb6", "name": "Bay Minette", "place_type": "city", "bounding_box": rectangle("-87.799932,30.836782 -87.750661,30.914894") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 104660, "cityName": "Bay Minette" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819911528665089, "text": "It's all about the 3 BBbs ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1682225022, "name": "ÇIFT S", "screen_name": "J_Buffering", "lang": "en", "location": "Willow Grove, PA", "create_at": date("2013-08-18"), "description": "6'3 PG college student athlete Running Lopes LCC {I work hard everyday to improve that guy that's in the mirror every morning} Instagram@jieremorrisey{PHL~CO}", "followers_count": 390, "friends_count": 485, "statues_count": 3337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lamar, CO", "id": "7ab944d5e643d0e4", "name": "Lamar", "place_type": "city", "bounding_box": rectangle("-102.633093,38.045529 -102.602483,38.103521") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8099, "countyName": "Prowers", "cityID": 843110, "cityName": "Lamar" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819911738482688, "text": "@H_B_KO only real niggas hip to this", "in_reply_to_status": 686819785011781632, "in_reply_to_user": 177235299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177235299 }}, "user": { "id": 194800466, "name": "#TheMoneyCollection", "screen_name": "MoneyMirCEO", "lang": "en", "location": "Amirica", "create_at": date("2010-09-24"), "description": "#TheMoneyCollection #FastFastFood Get Rich or Die Trying. Marketer|Entrepreneur|Event Planner|Photographer milliondollarshots@gmail.com/", "followers_count": 3394, "friends_count": 409, "statues_count": 220741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmore, MD", "id": "5f7b85a9049af1d0", "name": "Woodmore", "place_type": "city", "bounding_box": rectangle("-76.806927,38.900281 -76.73755,38.973106") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2486710, "cityName": "Woodmore" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819911956598784, "text": "#SupportOriginMelissa 34.9°F Wind:8.7mph Pressure: 29.91hpa Falling Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912069820416, "text": "Wind 2.0 mph W. Barometer 29.932 in, Falling. Temperature 35.7 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912120143872, "text": "Wind 2.9 mph S. Barometer 30.049 in, Falling. Temperature 15.7 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912438935552, "text": "\"Was your ex even black if you guys were the same color?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 482692081, "name": "Briyoncé", "screen_name": "BriannaInTheSky", "lang": "en", "location": "Goshen, IN // East Lansing, MI", "create_at": date("2012-02-03"), "description": "I like Chipotle, cats, Hillary Clinton, and MSU.", "followers_count": 541, "friends_count": 370, "statues_count": 44682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912514314240, "text": "�� https://t.co/o3niLpNQyN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2868408300, "name": "Døzen Down", "screen_name": "grayta_tha_k1ng", "lang": "en", "location": "THS '16", "create_at": date("2014-10-20"), "description": "Dont show signs of weakness ✊ fear no man. Mel❤️", "followers_count": 453, "friends_count": 652, "statues_count": 9941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euless, TX", "id": "5e2d6c67e728cca5", "name": "Euless", "place_type": "city", "bounding_box": rectangle("-97.15606,32.81088 -97.032953,32.881593") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4824768, "cityName": "Euless" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912577204225, "text": "Sixteen yo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2943617928, "name": "båilëy", "screen_name": "baileyyminton", "lang": "en", "location": "null", "create_at": date("2014-12-25"), "description": "null", "followers_count": 731, "friends_count": 629, "statues_count": 3941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912585592832, "text": "clear -> fair\ntemperature down 30°F -> 28°F\nhumidity up 52% -> 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.63762,35.50414"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224891034, "name": "Kannapolis Weather", "screen_name": "KannapolisNC", "lang": "en", "location": "Kannapolis, NC", "create_at": date("2010-12-09"), "description": "Weather updates, forecast, warnings and information for Kannapolis, NC. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 72, "friends_count": 2, "statues_count": 20604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kannapolis, NC", "id": "3562f19d95536f2f", "name": "Kannapolis", "place_type": "city", "bounding_box": rectangle("-80.678004,35.435255 -80.561482,35.54115") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37025, "countyName": "Cabarrus", "cityID": 3735200, "cityName": "Kannapolis" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912694632448, "text": "how do you drop 16 in a 3v3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2344089475, "name": "J", "screen_name": "JakeProx", "lang": "en", "location": "null", "create_at": date("2014-02-14"), "description": "null", "followers_count": 358, "friends_count": 104, "statues_count": 16089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arnold, MO", "id": "fbaeb390ee6f1f24", "name": "Arnold", "place_type": "city", "bounding_box": rectangle("-90.462274,38.386208 -90.336225,38.462857") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson", "cityID": 2901972, "cityName": "Arnold" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819912870825988, "text": "Frfr https://t.co/Q0fe2yFIsd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2289175344, "name": "$$$$", "screen_name": "johnnyshmurda9", "lang": "en", "location": "AUSTIN $$$$ FLEXAS", "create_at": date("2014-01-12"), "description": "#All-American'15 | Philippians 4:13| John 3:16| 400m| #Gramfam19 Track and Field #SSO ✊ SC: johnnyboyiee09 IG: Theylovejohnny_ #Krazy8 JDS JoJo Pnut", "followers_count": 2148, "friends_count": 2020, "statues_count": 48935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819913487519744, "text": "SO EXCITED!! ���� https://t.co/lsEpKSCJsd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310995379, "name": "Tara Fraser", "screen_name": "Simmy3Tara", "lang": "en", "location": "Natrona Heights, Pa", "create_at": date("2011-06-04"), "description": "I'm OBSESSED With Cupcakes And The Eiffel Tower!! My Favorite Youtuber Is @shanedawson!! Also, I'm A Disney Addict!!", "followers_count": 1372, "friends_count": 1869, "statues_count": 183135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819913613361152, "text": "The jets need clemsons qb tho he's got talent", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24774663, "name": "flex luthor", "screen_name": "lugunna", "lang": "en", "location": "batmans city ... #freebanz", "create_at": date("2009-03-16"), "description": "I got numbers all on me .. cause I count on myself ..... snapchat : luthermcfire", "followers_count": 793, "friends_count": 516, "statues_count": 115644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819913718218752, "text": "Wind 4.0 mph WSW. Barometer 29.936 in, Falling. Temperature 33.8 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 39, "friends_count": 25, "statues_count": 16815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819914393325569, "text": "@remitentacle ahh", "in_reply_to_status": 686818852357853185, "in_reply_to_user": 2420812844, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2420812844 }}, "user": { "id": 282224461, "name": "- ̗̀Error ̖́-", "screen_name": "ErrorTheDrager", "lang": "en", "location": "El Cajon, CA", "create_at": date("2011-04-14"), "description": "hi im Error im a drager (dragon/tiger)", "followers_count": 361, "friends_count": 407, "statues_count": 5872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819914456371200, "text": "@KvngShxvt_ you & my father still not talking��", "in_reply_to_status": -1, "in_reply_to_user": 279339583, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 279339583 }}, "user": { "id": 597983199, "name": "Toot, Sweet ❤️", "screen_name": "__Tloveeeeeeee", "lang": "en", "location": "on your mind.", "create_at": date("2012-06-02"), "description": "can do bad all by myself.❤", "followers_count": 3295, "friends_count": 3588, "statues_count": 79403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819914615685120, "text": "#stoprapistcops this is who they trust to #endhumantrafficking ? LOTS of cop Pedophiles https://t.co/rG7uvjIhul https://t.co/6BrXH4IrtT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stoprapistcops", "endhumantrafficking" }}, "user": { "id": 42817864, "name": "Domina Elle", "screen_name": "Domina_Elle", "lang": "en", "location": "Denver, Earth, The Multiverse", "create_at": date("2009-05-26"), "description": "Fun consultant - PLAY facilitator - Artist - Energy Alchemist - Let your actions be your prayers! seek the truth even when it's last thing you hoped it to be!", "followers_count": 4090, "friends_count": 3526, "statues_count": 16793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819914678546432, "text": "40.0F (Feels: 40.0F) - Humidity: 79% - Wind: 0.0mph --- - Gust: 1.6mph - Pressure: 1036.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915072991232, "text": "Yall wanna see a tweet that will get ignored? Watch my next one.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 113030654, "name": "I Get Curved Daily.", "screen_name": "naldodavis", "lang": "en", "location": "Lansing, MI", "create_at": date("2010-02-10"), "description": "Google 'Ugly' and I'll be in there. #DooDooRecords", "followers_count": 1693, "friends_count": 918, "statues_count": 208367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waverly, MI", "id": "011071d81596fbba", "name": "Waverly", "place_type": "city", "bounding_box": rectangle("-84.720053,42.681889 -84.60259,42.78436") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26045, "countyName": "Eaton", "cityID": 2684800, "cityName": "Waverly" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915139956737, "text": "oss , cause ikno e not sleeeo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2185529877, "name": "the kid☯", "screen_name": "ZaddyMeech", "lang": "en", "location": "eric❣", "create_at": date("2013-11-17"), "description": "lavi$h", "followers_count": 1982, "friends_count": 1547, "statues_count": 75154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Arthur, TX", "id": "350ad20be66e9edc", "name": "Port Arthur", "place_type": "city", "bounding_box": rectangle("-94.053284,29.823923 -93.858553,29.979166") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858820, "cityName": "Port Arthur" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915383369728, "text": "1 Elegant Trogon (Trogon elegans) - Florida Canyon--lower - 2016-01-11 08:00 https://t.co/IKHC0jfmLi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10245 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915400130560, "text": "2 Rufous-capped Warbler (Basileuterus rufifrons) - Florida Canyon--lower - 2016-01-11 08:00 https://t.co/mlvNA3AiGV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10245 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915408523264, "text": "1 Yellow Warbler (Northern) (Setophaga petechia) - Coachline Gravel Pit - 2016-01-11 07:56 https://t.co/l10Fy31Kcc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.136,32.394"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10245 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 444270, "cityName": "Marana" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915442040832, "text": "1 Brown Thrasher (Toxostoma rufum) - La Posta Quemada Ranch - 2016-01-11 09:00 https://t.co/pR0Ts7r18E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.6363797,32.0512984"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10245 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-12T00:00:02.000Z"), "id": 686819915446251520, "text": "This is Major Tom to ground control. Turn and face the strange.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29650084, "name": "Taylor Norberg", "screen_name": "TayNorberg", "lang": "en", "location": "Covington, KY", "create_at": date("2009-04-07"), "description": "University of Kentucky Alum. 6'1 smart ass from Northern Kentucky. Bill Murray. Bourbon. You'll Never Walk Alone.", "followers_count": 335, "friends_count": 619, "statues_count": 9400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, KY", "id": "9c38c3bbd6c69084", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-84.554442,38.964564 -84.479712,39.095317") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2117848, "cityName": "Covington" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819915869765632, "text": "@Nellie_Montoya lmao ������", "in_reply_to_status": 686819715012939776, "in_reply_to_user": 1677140479, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 1677140479 }}, "user": { "id": 1065051061, "name": "Little Ada", "screen_name": "ceci_ada", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "I often think the night is more alive & more richly colored than the day.", "followers_count": 356, "friends_count": 346, "statues_count": 17671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819915903414273, "text": "Wind 0.0 mph ---. Barometer 30.05 in, Steady. Temperature 19.9 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 92, "statues_count": 156614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819915991486464, "text": "Slow it downnn sixty ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2761508285, "name": "Jackie", "screen_name": "toxicctorres", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2014-09-04"), "description": "IUP", "followers_count": 345, "friends_count": 314, "statues_count": 2735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819916192681985, "text": "@Jordan36s @miguelmontoya21 \nLmao jk Jordy I luv ya", "in_reply_to_status": 686819819061022720, "in_reply_to_user": 955838372, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 733493874, 390927411 }}, "user": { "id": 955838372, "name": "Nicté", "screen_name": "zabdydbaz", "lang": "en", "location": "H-Town", "create_at": date("2012-11-18"), "description": "live and let live ♡ Alani ♡", "followers_count": 594, "friends_count": 432, "statues_count": 15599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819916314312704, "text": "Broke niggas just walk around mad @theworld ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4271698875 }}, "user": { "id": 22382385, "name": "METRA BOOMIN", "screen_name": "TheeHeartthrob", "lang": "en", "location": "Mississippi ✈️ Atlanta", "create_at": date("2009-03-01"), "description": "Knew she was a thug cause when I met her, she had her scarf on", "followers_count": 6541, "friends_count": 5214, "statues_count": 108471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelby, MS", "id": "8b844f56b27341ea", "name": "Shelby", "place_type": "city", "bounding_box": rectangle("-90.777139,33.931884 -90.753096,33.964562") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28011, "countyName": "Bolivar", "cityID": 2867040, "cityName": "Shelby" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819916607950848, "text": "03:00 31.8°F Feels:23.0°F (Hi32.0°F/Lo31.6°F) Hum:74% Wnd:WSW 11.0MPH Baro:29.75in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 161, "friends_count": 265, "statues_count": 24603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819916851228672, "text": "@AlyssaMybarra who dat cuz ��", "in_reply_to_status": 686814100064612352, "in_reply_to_user": 2930711629, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2930711629 }}, "user": { "id": 587950640, "name": "TIM WOODS", "screen_name": "TimWoods43", "lang": "en", "location": "The Funk, Tx", "create_at": date("2012-05-22"), "description": "#LongLiveCT SC: t_woods45", "followers_count": 989, "friends_count": 688, "statues_count": 23754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819916909944832, "text": "@mamacitassss ��", "in_reply_to_status": 686775569111879681, "in_reply_to_user": 2325283274, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2325283274 }}, "user": { "id": 2325283274, "name": "HARITH ❤", "screen_name": "mamacitassss", "lang": "en", "location": "✈ ", "create_at": date("2014-02-03"), "description": "popitforbeef\n1/14/92 - 1/9/15", "followers_count": 373, "friends_count": 217, "statues_count": 37679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917233012736, "text": "Temp: 29.9°F Wind:0.0mph Pressure: 30.238hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917312622592, "text": "@VampyreSleaze the number of times I tweet these exact lyrics is a lil embarrassing", "in_reply_to_status": 686819744549220352, "in_reply_to_user": 848437256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 848437256 }}, "user": { "id": 848437256, "name": "Alice", "screen_name": "VampyreSleaze", "lang": "en", "location": "Transylvania", "create_at": date("2012-09-26"), "description": "Musketeer; Hello MTV welcome to my crypt", "followers_count": 404, "friends_count": 435, "statues_count": 9263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917438423041, "text": "Going through my old Facebook messages is so funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265058754, "name": "Saralicious", "screen_name": "sarahiwestcoast", "lang": "en", "location": "oregon", "create_at": date("2011-03-12"), "description": "I dream big and sleep late", "followers_count": 1964, "friends_count": 790, "statues_count": 75491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917459488768, "text": "\"@_DomKM Theonna gone give James the sloppy lmao\" https://t.co/PziUqMcotB", "in_reply_to_status": 686817143015489536, "in_reply_to_user": 2292272193, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2292272193 }}, "user": { "id": 261463861, "name": "Theyoncé➰", "screen_name": "the0nna_xo", "lang": "en", "location": "Raleigh, NC", "create_at": date("2011-03-05"), "description": "Confidence is the real key, bitch! | sc: the0nna_xo ✨| @TheyoncePastry_", "followers_count": 1504, "friends_count": 994, "statues_count": 310657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917539196929, "text": "Ripley SW Limestone Co. Temp: 36.0°F Wind:3.1mph Pressure: 1000.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 47359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917539205120, "text": "@Gabbygavino__ @Shanka_Bitch oh it's litttttyyyyyy we out!!!", "in_reply_to_status": 686818026763776000, "in_reply_to_user": 2956061286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2956061286, 23878516 }}, "user": { "id": 58296910, "name": "JOSIE GROSIE", "screen_name": "FUCKZOE", "lang": "en", "location": "Lower East Side, NYC", "create_at": date("2009-07-19"), "description": "A tad different. Mad smart, but ack ignorant. IG: TsunamiiMami TsunamiMamis@gmail.com", "followers_count": 5761, "friends_count": 689, "statues_count": 268800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819917857861633, "text": "@__Kriiss happy birthday nigga, smoke one for me.", "in_reply_to_status": -1, "in_reply_to_user": 266331393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266331393 }}, "user": { "id": 235817061, "name": "Flo$$y", "screen_name": "Darion_TheGreat", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2011-01-08"), "description": "null", "followers_count": 1702, "friends_count": 1457, "statues_count": 75716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819918667354113, "text": "your MCM has an MCM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 891639014, "name": "jais", "screen_name": "therealjais", "lang": "en", "location": "East Blue", "create_at": date("2012-10-19"), "description": "lost on the path of life", "followers_count": 206, "friends_count": 187, "statues_count": 4724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819918822551553, "text": "@longliveShark @HenryRogueadikt @_IzzyCampos @RIPErickArzate @g_alvarez100 me la pelan tourist", "in_reply_to_status": 686819703872851969, "in_reply_to_user": 3686339713, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 3686339713, 438418135, 340179258, 2558975604, 518821395 }}, "user": { "id": 3218849202, "name": "Paul Martinez", "screen_name": "1027Paul", "lang": "en", "location": "null", "create_at": date("2015-05-17"), "description": "Lowkey. Dallas. I'll be damned if I let anyone take my smile away.", "followers_count": 104, "friends_count": 82, "statues_count": 7152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819918881271809, "text": "Vehicle on fire in #Columbus on Stelzer Rd Both NB/SB at Worth Ave #traffic https://t.co/hylMo0EFh8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.90978,40.05468"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Columbus", "traffic" }}, "user": { "id": 930775286, "name": "TTWN Columbus", "screen_name": "TotalTrafficCMH", "lang": "en", "location": "Columbus, OH", "create_at": date("2012-11-06"), "description": "Traffic for Columbus, OH, powered by Total Traffic Network.", "followers_count": 1069, "friends_count": 42, "statues_count": 51308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819918940106757, "text": "Wind 0.0 mph ---. Barometer 30.219 in, Falling. Temperature 26.8 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819919065780224, "text": "broken arms aren't fun anyway i fell out of a tree house in the 2nd grade my only injury in life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630398732, "name": "matthew adams", "screen_name": "_MatthewAdams", "lang": "en", "location": "Portland ,Oregon ", "create_at": date("2012-07-08"), "description": "been on twitter since september 2009", "followers_count": 136, "friends_count": 88, "statues_count": 30377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819919132901376, "text": "https://t.co/K4BysBHQGL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 554908173, "name": "bradfeld", "screen_name": "bjfeldy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-04-15"), "description": "Partner @MPCommercial Real Estate boutique specializing in Tenant/Landlord Representation & Investment Sales", "followers_count": 50, "friends_count": 31, "statues_count": 8 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819919284060160, "text": "Strange stance that from Bailey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2557107184, "name": "Nachiketh", "screen_name": "No_Xpression", "lang": "en", "location": "Gainesville, Florida", "create_at": date("2014-05-20"), "description": "Dravid,Kimi and Federer. Chelsea,BFC and RCB. Procrastination is my middle name,Ctrl C+ Ctrl V for a living.", "followers_count": 66, "friends_count": 300, "statues_count": 1840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-01-12T00:00:03.000Z"), "id": 686819919334277121, "text": "@brittanixx_ @ColbyDeMato10 done ������", "in_reply_to_status": 686819556610842624, "in_reply_to_user": 184234064, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184234064, 366253763 }}, "user": { "id": 2229339973, "name": "fifty shades of shay", "screen_name": "_shayleaaa", "lang": "en", "location": "probably at practice ", "create_at": date("2013-12-03"), "description": "I tweet way to much about my bf. I stand at a whole whoppin 4'11. Ball player @ Murray state college |19.| @colbydemato10", "followers_count": 1898, "friends_count": 904, "statues_count": 29400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madill, OK", "id": "7cca981d8fcdf839", "name": "Madill", "place_type": "city", "bounding_box": rectangle("-96.793453,34.075633 -96.75852,34.103389") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40095, "countyName": "Marshall", "cityID": 4045750, "cityName": "Madill" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819919925645313, "text": "Oh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1047185798, "name": "No Lex Zone ™", "screen_name": "childishlexino", "lang": "en", "location": "love you lil g ❤️", "create_at": date("2012-12-29"), "description": "you're getting mad, I'm getting rich", "followers_count": 1498, "friends_count": 1034, "statues_count": 46810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, WA", "id": "744fbe9224233893", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-122.199229,48.133613 -122.10191,48.203671") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5302585, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819920219402241, "text": "JCole soothes me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 486264816, "name": "ĸaιlz", "screen_name": "KailyMarie2", "lang": "en", "location": "Bolingbrook, IL", "create_at": date("2012-02-07"), "description": "♡ snap - KailyMarie2", "followers_count": 724, "friends_count": 603, "statues_count": 55214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bolingbrook, IL", "id": "0991d757989cef56", "name": "Bolingbrook", "place_type": "city", "bounding_box": rectangle("-88.18516,41.630746 -88.0269,41.735932") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1707133, "cityName": "Bolingbrook" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819920378630144, "text": "@artofprisillaV only speaking the truth my love ��", "in_reply_to_status": 686819808659156992, "in_reply_to_user": 270697955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 270697955 }}, "user": { "id": 2808438528, "name": "Ana", "screen_name": "la_sirena_1", "lang": "en", "location": "null", "create_at": date("2014-09-13"), "description": "CSUDH '19 WTFGang", "followers_count": 169, "friends_count": 318, "statues_count": 3155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819920424886272, "text": "@TeresaRuffo hopefully I see you though ����", "in_reply_to_status": 686817644629110784, "in_reply_to_user": 383428550, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 383428550 }}, "user": { "id": 704085428, "name": "vanessa", "screen_name": "vanessa_ortega3", "lang": "en", "location": "United States", "create_at": date("2012-07-18"), "description": "WCC insta: v_ortega3", "followers_count": 913, "friends_count": 503, "statues_count": 20285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandwich, IL", "id": "1f9eabd5dd5438b6", "name": "Sandwich", "place_type": "city", "bounding_box": rectangle("-88.655795,41.63126 -88.573392,41.661955") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1767548, "cityName": "Sandwich" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921020481536, "text": "hanging on the edge of the earth⁰oh what a beautiful scene", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25891622, "name": "bk", "screen_name": "bkcordle", "lang": "en", "location": "Huntington, WV ", "create_at": date("2009-03-22"), "description": "artist, designer // cordle14@marshall.edu", "followers_count": 767, "friends_count": 460, "statues_count": 13084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington, WV", "id": "e4197a23034fa912", "name": "Huntington", "place_type": "city", "bounding_box": rectangle("-82.530433,38.375981 -82.349236,38.439347") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54011, "countyName": "Cabell", "cityID": 5439460, "cityName": "Huntington" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921691426816, "text": "@hoozayisdead well kinda", "in_reply_to_status": 686819889672138752, "in_reply_to_user": 427217101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 427217101 }}, "user": { "id": 427217101, "name": "jose", "screen_name": "hoozayisdead", "lang": "en", "location": "Los Angeles/Isla Vista", "create_at": date("2011-12-02"), "description": "figuratively speaking", "followers_count": 255, "friends_count": 146, "statues_count": 52860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921896931328, "text": "Gn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 404396307, "name": "T-Ho⚾️", "screen_name": "Tyler_Hover21", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-11-03"), "description": "#SELU19 #Cubs #WeAreGood SS RIP Dad❤️", "followers_count": 730, "friends_count": 561, "statues_count": 31310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921943105536, "text": "HAPPY BIRTHDAY U SWEET POTATO I LOVE U SO MUCH ♡♡ @zaynmalik", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 176566242 }}, "user": { "id": 628637917, "name": "Karina", "screen_name": "ardensboobs", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-07-06"), "description": "cool beans", "followers_count": 8443, "friends_count": 8204, "statues_count": 173923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921964089346, "text": "Growing up sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625774158, "name": "Ash", "screen_name": "ashlyn_burt", "lang": "en", "location": "null", "create_at": date("2012-07-03"), "description": "null", "followers_count": 1042, "friends_count": 801, "statues_count": 34930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Felida, WA", "id": "3990e08a4d4cdafe", "name": "Felida", "place_type": "city", "bounding_box": rectangle("-122.734598,45.693162 -122.682473,45.733492") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5323550, "cityName": "Felida" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819921985064961, "text": "�� https://t.co/7YyBxBGngX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 210153001, "name": "❤️jlawson.", "screen_name": "_jassssbella", "lang": "en", "location": "New Orleans, LA", "create_at": date("2010-10-30"), "description": "NOLA❤️ #TxSU19", "followers_count": 1672, "friends_count": 1001, "statues_count": 36826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmere, LA", "id": "25041b89e1b96118", "name": "Woodmere", "place_type": "city", "bounding_box": rectangle("-90.089118,29.838214 -90.066639,29.874275") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2283002, "cityName": "Woodmere" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819922077351936, "text": "Op that sucks������ https://t.co/OlPXcTcu7V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3104943979, "name": "PUA TONGA", "screen_name": "puuaahh", "lang": "en", "location": "808✈️801", "create_at": date("2015-03-23"), "description": "RIL MOM❤️ SC:pooah97 IG:pua_tonga", "followers_count": 288, "friends_count": 387, "statues_count": 3099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bountiful, UT", "id": "d10ead505fcb25cb", "name": "Bountiful", "place_type": "city", "bounding_box": rectangle("-111.906716,40.834807 -111.834245,40.909783") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4907690, "cityName": "Bountiful" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819922479976448, "text": "@KHF_NHF @QueenNoor @jordantimes I'm sure the starving Syrian refugees will be thrilled. I guess their carbon footprint is near zero.", "in_reply_to_status": 686817688249876480, "in_reply_to_user": 193661058, "favorite_count": 0, "coordinate": point("-83.4780004,42.33146044"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 193661058, 121887978, 18189723 }}, "user": { "id": 320967163, "name": "Mike Merc", "screen_name": "mikemerc57", "lang": "en", "location": "null", "create_at": date("2011-06-20"), "description": "American, Patriot, Conservative. We live in the greatest country in the history of the world. We need to act like it.", "followers_count": 851, "friends_count": 1110, "statues_count": 18298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Michigan, USA", "id": "67d92742f1ebf307", "name": "Michigan", "place_type": "admin", "bounding_box": rectangle("-90.418136,41.696088 -82.122971,48.306272") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne" } }
+{ "create_at": datetime("2016-01-12T00:00:04.000Z"), "id": 686819922857463808, "text": "happy 21st to one of my good friends, party hard dude!!���� @succsethful12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 254127025 }}, "user": { "id": 41001229, "name": "cocaine robby", "screen_name": "camillemarii", "lang": "en", "location": "Newberg, OR", "create_at": date("2009-05-18"), "description": "eighteen // blunts & booze", "followers_count": 727, "friends_count": 559, "statues_count": 42502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberg, OR", "id": "7f3bf05bec93c209", "name": "Newberg", "place_type": "city", "bounding_box": rectangle("-122.989925,45.28369 -122.925614,45.328231") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4152100, "cityName": "Newberg" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819924631666688, "text": "I have the biggest crush on @Zendaya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 154280902 }}, "user": { "id": 700687944, "name": "9God", "screen_name": "Papichampu_9", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "Oct 09 OvO know yourself", "followers_count": 1066, "friends_count": 601, "statues_count": 2899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819924635971584, "text": "Wind 0.0 mph ---. Barometer 30.030 in, Steady. Temperature 10.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 7871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819925000753152, "text": "@WNolanCohn5 @IUBloomington I had 4 internships-3 unpaid-before 1st real job. Be willing to move anywhere/do everything to gain experience!", "in_reply_to_status": 686818343794388993, "in_reply_to_user": 384037900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384037900, 18458348 }}, "user": { "id": 34301875, "name": "Sage Steele", "screen_name": "sagesteele", "lang": "en", "location": "Los Angeles & Scottsdale, AZ", "create_at": date("2009-04-22"), "description": "Mom of 3 awesome kids * Army brat * Indiana University alum * Host of NBA Countdown on \nESPN/ABC \n\nInstagram: @sagesteele", "followers_count": 149955, "friends_count": 1337, "statues_count": 12141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819925428600833, "text": "David LK loves me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 518168573, "name": "bby A", "screen_name": "AlissaGuerrero", "lang": "en", "location": "my own w0rld", "create_at": date("2012-03-07"), "description": "18 sc : xoxo-alissa", "followers_count": 788, "friends_count": 609, "statues_count": 34333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Centro, CA", "id": "00436a376f214be7", "name": "El Centro", "place_type": "city", "bounding_box": rectangle("-115.594987,32.730647 -115.501098,32.818522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 621782, "cityName": "El Centro" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819925462130688, "text": "And he would have the biggest smile on his face staring after her..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19327749, "name": "Cat Chantha", "screen_name": "cchanthavisouk", "lang": "en", "location": "sWAgger", "create_at": date("2009-01-21"), "description": "null", "followers_count": 248, "friends_count": 574, "statues_count": 5306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819925650874368, "text": "Boy hell naw https://t.co/200VN5CQN8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1318008510, "name": "Cedric Wilcots 5⃣2⃣", "screen_name": "TheGuyWitTheBox", "lang": "en", "location": "Dallas, Tx ✈️ Las Cruces,NM", "create_at": date("2013-03-31"), "description": "#52 for New Mexico State University. SC: TheGuyWitTheBox", "followers_count": 747, "friends_count": 535, "statues_count": 22591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819925789380608, "text": "@swank__lupe I'm not. #shootingmyshot2k16", "in_reply_to_status": 686818021940326400, "in_reply_to_user": 1644540385, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shootingmyshot2k16" }}, "user_mentions": {{ 1644540385 }}, "user": { "id": 153305721, "name": "Ari Jones", "screen_name": "_safARII", "lang": "en", "location": "null", "create_at": date("2010-06-07"), "description": "Happy girls are the prettiest girls #UIUC18", "followers_count": 1425, "friends_count": 1187, "statues_count": 63797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flossmoor, IL", "id": "97ae6559498bd23d", "name": "Flossmoor", "place_type": "city", "bounding_box": rectangle("-87.713455,41.527807 -87.655073,41.55185") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1726571, "cityName": "Flossmoor" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819926066122753, "text": "I just come to monique's to eat foodie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 124899289, "name": "Major Marquis Warren", "screen_name": "ThithoMiranda", "lang": "en", "location": "Procrastination City", "create_at": date("2010-03-20"), "description": "inglorious Quentin Tarantino \nYeezY season approachin", "followers_count": 380, "friends_count": 370, "statues_count": 8092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus, CA", "id": "04088141121c7398", "name": "Citrus", "place_type": "city", "bounding_box": rectangle("-117.899428,34.105384 -117.876959,34.127081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 613560, "cityName": "Citrus" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819926506487808, "text": "@fvckinmocki school rip", "in_reply_to_status": 686819790963343360, "in_reply_to_user": 1489089439, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1489089439 }}, "user": { "id": 85036010, "name": "Trendy Nigga", "screen_name": "RiceontheDaily", "lang": "en", "location": "null", "create_at": date("2009-10-25"), "description": "null", "followers_count": 313, "friends_count": 197, "statues_count": 3016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819926619852801, "text": "one time I was drunk and was eating nuggets and told this nigga to come eat these nuggets and this pussy maaan��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560145917, "name": "because kanye", "screen_name": "lovemileeeey", "lang": "en", "location": "Kanye. ", "create_at": date("2014-06-10"), "description": "Rest In Paradise My King 04.17.14❤️ Ive got too much soul to be handled by someone whos never been passionate.", "followers_count": 649, "friends_count": 779, "statues_count": 5187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819926884089856, "text": "�� https://t.co/NUTMqjbgZw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2202255769, "name": "Jaylah Nicole", "screen_name": "Jaylah_Nicole18", "lang": "en", "location": "MD ✈️ Al", "create_at": date("2013-11-18"), "description": "20 | Baltimore | #AAMU Elementary Education Major | IG: @Jaylahnicole | Ready to start my new life", "followers_count": 617, "friends_count": 836, "statues_count": 19592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819927576018944, "text": "Todd & his momma. Over & out ✌��️ https://t.co/vUdyL1R2h8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223764929, "name": "Ashley Alexiss", "screen_name": "AshAlexiss", "lang": "en", "location": "Los Angeles ☀️", "create_at": date("2010-12-07"), "description": "Plus-Size Model, #CleavagefortheCURE, Owner of #ALEXISS Swimwear, #PatsNation, Grad Student :) Appeared on the Oxygen Network on, #FixMyMom", "followers_count": 182317, "friends_count": 288, "statues_count": 49185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-12T00:00:05.000Z"), "id": 686819927731339264, "text": "@thewokegay be safe ❤️", "in_reply_to_status": 686817968752340992, "in_reply_to_user": 23260925, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23260925 }}, "user": { "id": 248530687, "name": "champagne mami", "screen_name": "tayirl", "lang": "en", "location": "hilliard, oh", "create_at": date("2011-02-06"), "description": "null", "followers_count": 1270, "friends_count": 554, "statues_count": 66350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-12T00:00:06.000Z"), "id": 686819928070963200, "text": "High speed chase on mission lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 951676495, "name": "/", "screen_name": "illkush", "lang": "en", "location": "bayareaa", "create_at": date("2012-11-16"), "description": "West Coast, Best Coast. LA/TheBay\r\nInstagram: Siqvibes | #findsierra", "followers_count": 573, "friends_count": 494, "statues_count": 38749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-01-12T00:00:06.000Z"), "id": 686819928649814016, "text": "You have to be true to yourself first before you can fully be true to someone else.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2491719396, "name": "Juicy A", "screen_name": "asiawasia__", "lang": "en", "location": "Hawthorne, CA", "create_at": date("2014-05-12"), "description": "crème de la crème", "followers_count": 372, "friends_count": 331, "statues_count": 13659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-12T00:00:06.000Z"), "id": 686819928737984513, "text": "@nicholasmt the FUCK", "in_reply_to_status": 686819513283657728, "in_reply_to_user": 147401952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 147401952 }}, "user": { "id": 16440706, "name": "Mikayla", "screen_name": "mikaylax", "lang": "en", "location": "Orlando, FL", "create_at": date("2008-09-24"), "description": "I do merch & makeup / @licknord's social media TM / I like vodka / I heavily fuck with @nickelback", "followers_count": 2626, "friends_count": 589, "statues_count": 46447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Hill, FL", "id": "68042baafa8b7e0a", "name": "Spring Hill", "place_type": "city", "bounding_box": rectangle("-82.639883,28.433671 -82.428825,28.534024") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12053, "countyName": "Hernando", "cityID": 1268350, "cityName": "Spring Hill" } }
+{ "create_at": datetime("2016-01-12T00:00:06.000Z"), "id": 686819928851234816, "text": "Now I'm in my FHD bag lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110271982, "name": "MARCH 23rd✨♈️", "screen_name": "holychampagne_", "lang": "en", "location": "four. one. tew", "create_at": date("2010-01-31"), "description": "RIP RONNEL| #IUP'17| @IUPBSL | @TruecultureU", "followers_count": 2182, "friends_count": 1277, "statues_count": 156474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-12T00:00:06.000Z"), "id": 686819929312473088, "text": "@catieparsons https://t.co/hkzOONwBNp", "in_reply_to_status": -1, "in_reply_to_user": 436299127, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 436299127 }}, "user": { "id": 2642233803, "name": "Laken", "screen_name": "lakenalexis", "lang": "en", "location": "choctaw//stillwater", "create_at": date("2014-06-25"), "description": "OSU//#okstate2018", "followers_count": 171, "friends_count": 202, "statues_count": 811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182290766544896, "text": "& he was...", "in_reply_to_status": 687182150836207616, "in_reply_to_user": 496501160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496501160, "name": "Ari☆", "screen_name": "arianaortiz96", "lang": "en", "location": "California, USA", "create_at": date("2012-02-18"), "description": "1/2 hustlin' hoes", "followers_count": 578, "friends_count": 1797, "statues_count": 33242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wildomar, CA", "id": "5080f2ebcf3d5cfd", "name": "Wildomar", "place_type": "city", "bounding_box": rectangle("-117.308832,33.575507 -117.20561,33.656019") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 685446, "cityName": "Wildomar" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182290984763392, "text": "Fucking George is dead WHYYYY!!!��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4184044353, "name": "Josh", "screen_name": "iam_jalxx", "lang": "en", "location": "Houston, TX", "create_at": date("2015-11-13"), "description": "Est .1996,HTOWN,19 yrs, San Jac College,Jeremiah 29:11 team tatted snapchat-andrade_josh #ovoxo", "followers_count": 83, "friends_count": 120, "statues_count": 1345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182291936804864, "text": "@Amy__Mariie dats not good 4 ur Grade", "in_reply_to_status": 687181729484816386, "in_reply_to_user": 630517909, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 630517909 }}, "user": { "id": 1181077962, "name": "CalebYayaPETER_11™©", "screen_name": "CalebnguiYaya_9", "lang": "en", "location": "Manteca, CA", "create_at": date("2013-02-14"), "description": "⚽Ballin since 95, ⚽ is da sport of GOD #116, DC 4 LIFE", "followers_count": 61, "friends_count": 122, "statues_count": 2736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182292800913408, "text": "Wind 0.0 mph ---. Barometer 30.28 in, Falling slowly. Temperature 2.7 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 23, "friends_count": 0, "statues_count": 30045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182293232791552, "text": "Till the sun come up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2728555060, "name": "Tama Toa", "screen_name": "ElijahSuka", "lang": "en", "location": "Ellensburg, WA", "create_at": date("2014-07-29"), "description": "Led by God || CWU #56 || M.B.S.", "followers_count": 299, "friends_count": 309, "statues_count": 1461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellensburg, WA", "id": "c95cdb2a983262e5", "name": "Ellensburg", "place_type": "city", "bounding_box": rectangle("-120.582586,46.958017 -120.49726,47.028542") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53037, "countyName": "Kittitas", "cityID": 5321240, "cityName": "Ellensburg" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182294562541568, "text": "Wind 0 mph --. Barometer 30.25 in, Steady. Temperature 36.3 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-13T00:00:00.000Z"), "id": 687182294600138752, "text": "@sad_girl420 i somehow drove past his house n then remembered my way back 2 my house (it was a very stressful drive)", "in_reply_to_status": 687182044309291010, "in_reply_to_user": 3180009330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3180009330 }}, "user": { "id": 3180009330, "name": "sophie", "screen_name": "sad_girl420", "lang": "en", "location": "Portland, OR", "create_at": date("2015-04-29"), "description": "i like breakfast", "followers_count": 115, "friends_count": 93, "statues_count": 1705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182295120351232, "text": "Class in 7 hours but I wanna play call of duty idk what to do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3496976472, "name": "DJ B.M.", "screen_name": "mclaughlin_bo", "lang": "en", "location": "AL/NJ", "create_at": date("2015-09-08"), "description": "19 • Phillies • Eagles • Flyers • Sixers • people take me way too seriously • 12/30/10 • 1/11/12 • 10/4/14", "followers_count": 504, "friends_count": 427, "statues_count": 8917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182295912951808, "text": "Mississippi https://t.co/swY9hxR6cs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 3361295445, "name": "Ñia✨", "screen_name": "niaaaa_17", "lang": "en", "location": "Maryland✈️Alabama", "create_at": date("2015-07-05"), "description": "Southern Belle❤| Call me Buttercup☺️", "followers_count": 320, "friends_count": 786, "statues_count": 3826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Center Point, AL", "id": "68a5498344541c36", "name": "Center Point", "place_type": "city", "bounding_box": rectangle("-86.709501,33.6188 -86.629573,33.676838") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 113264, "cityName": "Center Point" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182295984287744, "text": "Wind 1.0 mph SE. Barometer 1027.64 mb, Falling. Temperature 29.2 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182296147886080, "text": "Pink Winter Sunset No.3\nR/T #filmisnotdead #35mm\n#artphotography #SantaFeNM \n#landscape #sunset #inspire https://t.co/JaWB6eG3nq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "filmisnotdead", "35mm", "artphotography", "SantaFeNM", "landscape", "sunset", "inspire" }}, "user": { "id": 1293256549, "name": "♏™jeffreyGbaca", "screen_name": "JBaca59", "lang": "en", "location": "Santa Fe NM / Venice CA", "create_at": date("2013-03-23"), "description": "storyteller|funny guy|Analog| ♡hopelessly romantic♡I brought the AWESOME, what'd U bring! \nFYRWRX, A Prod. Co.©2015\n#film #35mm #filmisnotdead", "followers_count": 149, "friends_count": 339, "statues_count": 2836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Fe, NM", "id": "ebdd9f30d3abfc43", "name": "Santa Fe", "place_type": "city", "bounding_box": rectangle("-106.10308,35.586659 -105.895133,35.726444") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35049, "countyName": "Santa Fe", "cityID": 3570500, "cityName": "Santa Fe" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182296542121984, "text": "this bitch just called me an ape", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3169419788, "name": "lil t", "screen_name": "sadgirltato", "lang": "en", "location": "Chandler, AZ", "create_at": date("2015-04-23"), "description": "don't be mad", "followers_count": 527, "friends_count": 496, "statues_count": 12447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182296642793472, "text": "NO F U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346433787, "name": "i⚡️abela", "screen_name": "unpredictabela", "lang": "en", "location": " University of Arizona ", "create_at": date("2011-08-01"), "description": "blue haired dweeb ✌️", "followers_count": 865, "friends_count": 831, "statues_count": 20958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297066516480, "text": "Wind 0.0 mph ---. Barometer 30.319 in, Steady. Temperature 16.4 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297678782464, "text": "@RachelAnalPorn mike @666.com https://t.co/Ahp34O8g50", "in_reply_to_status": -1, "in_reply_to_user": 4632030441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4632030441, 3195715364 }}, "user": { "id": 4073124560, "name": "1 360-561-8704", "screen_name": "mvt666", "lang": "en", "location": "null", "create_at": date("2015-10-30"), "description": "null", "followers_count": 899, "friends_count": 1239, "statues_count": 4444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelton, WA", "id": "017fbe104029934f", "name": "Shelton", "place_type": "city", "bounding_box": rectangle("-123.221681,47.172039 -123.026184,47.291156") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53045, "countyName": "Mason", "cityID": 5363735, "cityName": "Shelton" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297712488449, "text": "01/13@03:00 - Temp 23.9F, WC 17.4F. Wind 5.1mph W, Gust 12.0mph. Bar 30.024in, Rising. Rain 0.00in. Hum 56%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297754398720, "text": "When her mom & the next room amd she start going crazy talkin bout \"idc if she hear me\" ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63503362, "name": "Fredo Millz", "screen_name": "662_MILLZ", "lang": "en", "location": "FT wayne ", "create_at": date("2009-08-06"), "description": "college student | sc:money_millz", "followers_count": 983, "friends_count": 1055, "statues_count": 43317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297804763136, "text": "When you're watching med shows and you understand all of their terminology >>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770185970, "name": "Nic", "screen_name": "nicyonkers", "lang": "en", "location": "Long Island, New York", "create_at": date("2014-09-14"), "description": "Avid fan of the medical field, aspiring physician, seeing the world one adventure at a time", "followers_count": 98, "friends_count": 221, "statues_count": 2974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayville, NY", "id": "df9eac8f0f25997c", "name": "Bayville", "place_type": "city", "bounding_box": rectangle("-73.591438,40.899734 -73.528193,40.915722") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3605034, "cityName": "Bayville" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182297829781504, "text": "goodnight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2496599606, "name": "+", "screen_name": "Jayytoven_", "lang": "en", "location": "Dallas | The Agg | Denton", "create_at": date("2014-05-15"), "description": "Aspired to inspire before i expire | #UNT18 Rest In Paradise Madea and Aunt Cheryl", "followers_count": 910, "friends_count": 807, "statues_count": 6964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182298148552704, "text": "@mundo_guzman a medical card, good night ��", "in_reply_to_status": 687181945080446976, "in_reply_to_user": 4346604732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4346604732 }}, "user": { "id": 91709968, "name": "C.IV", "screen_name": "prodigiouschuck", "lang": "en", "location": "In the GYM", "create_at": date("2009-11-21"), "description": "C. E. IV. | Live & Breathe HOOPS | TICDA", "followers_count": 503, "friends_count": 418, "statues_count": 23129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182298349879296, "text": "@thatkid__andy https://t.co/KB8QyCCqFR", "in_reply_to_status": -1, "in_reply_to_user": 3142166208, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3142166208 }}, "user": { "id": 221625234, "name": "Z", "screen_name": "JustDoJames", "lang": "en", "location": "null", "create_at": date("2010-11-30"), "description": "null", "followers_count": 673, "friends_count": 264, "statues_count": 51847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182298416979968, "text": "I miss you so much��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553341858, "name": "EsmEr", "screen_name": "PreettyEsmer", "lang": "en", "location": "Coachella, Ca", "create_at": date("2012-04-13"), "description": "Life doesnt always give second chances, take the first one...", "followers_count": 55, "friends_count": 228, "statues_count": 1573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182298505064448, "text": "Mom came over and did #XMas with @joeyharrison10 & us #TONITE, she gets back from #Texas & comes to hang out https://t.co/za55X6d65G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "XMas", "TONITE", "Texas" }}, "user_mentions": {{ 1637408257 }}, "user": { "id": 15902031, "name": "SUPERBOSS3232", "screen_name": "superboss3232", "lang": "en", "location": "LOS ANGELES, CALI", "create_at": date("2008-08-19"), "description": "DA MAN, DA MYTH, DA LEGEND JOSEPH (805)889-2503", "followers_count": 2110, "friends_count": 2119, "statues_count": 44810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-13T00:00:01.000Z"), "id": 687182298542833666, "text": "03:00 6.1°F Feels:6.1°F (Hi9.1°F/Lo6.1°F) Hum:57% Wnd:--- 0.0MPH Baro:30.21in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 165, "friends_count": 265, "statues_count": 24659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299230683138, "text": "Trying to sleep, but Bella is snoring sooooo loud!!!! Plus I wanna snugs ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18171035, "name": "Jesse", "screen_name": "jessejane", "lang": "en", "location": "null", "create_at": date("2008-12-16"), "description": "you'll never find a girl like me , I'm one of a kind ☺️ for bookings contact theoneandonlyjessejane@gmail.com", "followers_count": 326547, "friends_count": 487, "statues_count": 104630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299381641216, "text": "Update: I HAVE BACON AND EGGS AGAHSJD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1689869274, "name": "him", "screen_name": "BrisonAintShiit", "lang": "en", "location": "capital of tx , y'all", "create_at": date("2013-08-21"), "description": "master of all your needs", "followers_count": 1505, "friends_count": 785, "statues_count": 71950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299469885440, "text": "Temp: 32.8°F Wind:0.0mph Pressure: 30.280hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299478228993, "text": "❗️ https://t.co/PBN2Z0oWmM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2190835763, "name": "Killa Cam", "screen_name": "Mr_Shoe91", "lang": "en", "location": "shoe game proper ", "create_at": date("2013-11-21"), "description": "sunny days wouldnt be special if it wasnt for rain, joy wouldnt feel so good if it wasnt for pain.", "followers_count": 225, "friends_count": 268, "statues_count": 20095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elkridge, MD", "id": "d4f00e013186b461", "name": "Elkridge", "place_type": "city", "bounding_box": rectangle("-76.80445,39.162427 -76.696328,39.251056") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2425750, "cityName": "Elkridge" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299599912960, "text": "Relax now. Breathe deeply. That's right. (@ Electric Massage Chair in Gig Harbor, WA) https://t.co/BqesQ85gQM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.58646934,47.33097629"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14556477, "name": "David M. Brown, MD", "screen_name": "ZenDoc", "lang": "en", "location": "Gig Harbor, WA", "create_at": date("2008-04-27"), "description": "@BringBackSonics Founder; Long-time Feminist. @seahawks & @seattlestorm FANatic. Holistic Healer. Life Coach. Retired Family Physician; Blues/Rock/Soul Singer.", "followers_count": 9093, "friends_count": 9998, "statues_count": 23561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gig Harbor, WA", "id": "7f8e88c74409abb5", "name": "Gig Harbor", "place_type": "city", "bounding_box": rectangle("-122.617642,47.290796 -122.569191,47.360103") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5326735, "cityName": "Gig Harbor" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299662659584, "text": "@BewbsNBrains thank you!!!", "in_reply_to_status": 687182156930482176, "in_reply_to_user": 1315722486, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1315722486 }}, "user": { "id": 1014379171, "name": "The Tweet Deacon", "screen_name": "Trill_Soliloquy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-12-15"), "description": "Twitter Deacon Trentt, Doing Twitter Deacon Shit. Eat My Twitter Deacon Dick, As I Twitter Deacon Spit.", "followers_count": 1720, "friends_count": 826, "statues_count": 214063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299671052288, "text": "ไม่ต้องดูละอีก2ตอน หาคนแต่งตอนจบๆสวยๆ แล้วเชื่อเลย #วางถุงกาว #ทีมแทค แต่เชื่อว่า จฮ เป็นผัว ดซ ������ https://t.co/jnaKr595Uq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "hashtags": {{ "วางถุงกาว", "ทีมแทค" }}, "user": { "id": 2850117898, "name": "lonely whale ", "screen_name": "miint149", "lang": "en", "location": "null", "create_at": date("2014-10-29"), "description": "สิ่งเล็กๆที่เรียกว่า", "followers_count": 20, "friends_count": 216, "statues_count": 396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182299805306880, "text": "@_jarelnelson I'm on my way bro", "in_reply_to_status": 687182068481146881, "in_reply_to_user": 472608160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472608160 }}, "user": { "id": 942868531, "name": "❌ Darnell ❌ ™", "screen_name": "_ThaRealest25", "lang": "en", "location": "IN THE CUTT!!! ", "create_at": date("2012-11-11"), "description": "❌ BornReady Stay Ready❌ Edinboro '18 | P.A.C |RIP Patrick|RIP QUAY|RIP 40|RIP DEE| FREE BIGGIE", "followers_count": 1082, "friends_count": 585, "statues_count": 43090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edinboro, PA", "id": "ba319d2cc814974e", "name": "Edinboro", "place_type": "city", "bounding_box": rectangle("-80.157699,41.853916 -80.099795,41.89931") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4222608, "cityName": "Edinboro" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182300157616128, "text": "@ayruretagoyena should take you one day", "in_reply_to_status": 687158460861972481, "in_reply_to_user": 261805714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261805714 }}, "user": { "id": 181123728, "name": "Lucas Wendt", "screen_name": "Wendt2Canada", "lang": "en", "location": "University of Oregon", "create_at": date("2010-08-21"), "description": "They don't give you gold medals for beating somebody. They give you gold medals for beating everybody. We are always, Men Of Oregon.", "followers_count": 173, "friends_count": 414, "statues_count": 4352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182300388298752, "text": "Good Company. Good Friends. ���� https://t.co/8I6kaDW4Aq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52565173, "name": "Vanessa Guerrero", "screen_name": "Nessa_Notorious", "lang": "en", "location": "sacramento ca", "create_at": date("2009-06-30"), "description": "snapchat: nesskuhboothang", "followers_count": 271, "friends_count": 442, "statues_count": 906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florin, CA", "id": "486bc865d347cd73", "name": "Florin", "place_type": "city", "bounding_box": rectangle("-121.445964,38.474314 -121.364939,38.505175") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624498, "cityName": "Florin" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182300656840704, "text": "Girl see this why I love you , a hoe that can read back ���� https://t.co/1CFtQ08nKu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112295605, "name": "Victor Edwards", "screen_name": "____VICK", "lang": "en", "location": "in the hands of the Lord . ", "create_at": date("2010-02-07"), "description": "|£ad£R . just an ordinary kid trying to do extraordinary things .", "followers_count": 1121, "friends_count": 431, "statues_count": 59162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301021646848, "text": "@dollfacecarter1 ���� don't make me get on you now ��", "in_reply_to_status": 687182038009425921, "in_reply_to_user": 339728952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339728952 }}, "user": { "id": 302916275, "name": "Matthew Smith", "screen_name": "mattjsmith_", "lang": "en", "location": "HTX ✈️ BR", "create_at": date("2011-05-21"), "description": "feeling good living better.", "followers_count": 803, "friends_count": 604, "statues_count": 25051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301256499201, "text": "@sincedayuno_ @elihu909 �� Had To Cook Him For The 1x", "in_reply_to_status": 687181992065019905, "in_reply_to_user": 2362248186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2362248186, 2714316021 }}, "user": { "id": 3066767949, "name": "©Calin™", "screen_name": "calinSadiq1", "lang": "en", "location": "Upland, Ca", "create_at": date("2015-03-02"), "description": "#WANGGANG Young Thug Is The G.O.A.T", "followers_count": 1197, "friends_count": 1931, "statues_count": 7555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301374070784, "text": "#SupportOriginMelissa 13.1°F Wind:0.0mph Pressure: 30.21hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 306917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301428494336, "text": "But shit my boyfriend makes me feel like my ass looks like Khloe's. Love is giving us false hope for a fat ass. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334136328, "name": "Jaletha", "screen_name": "Jaletha13", "lang": "en", "location": "H-Town", "create_at": date("2011-07-12"), "description": "♉️ Packers TSU", "followers_count": 456, "friends_count": 166, "statues_count": 111402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301495689218, "text": "Wind 0.0 mph ---. Barometer 30.303 in, Rising slowly. Temperature 9.2 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301638299649, "text": "Wind 0.0 mph ---. Barometer 30.253 in, Steady. Temperature 29.6 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 56864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182301705310209, "text": "@Lalyreynadelsur Hahaha hope for a taller tomorrow", "in_reply_to_status": 687182035295715331, "in_reply_to_user": 2728992833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728992833 }}, "user": { "id": 2986252540, "name": "Alkaid", "screen_name": "totesthewizlord", "lang": "en", "location": "ursa major, third star on edge", "create_at": date("2015-01-19"), "description": "Grizzly bear in the streets, Teddy bear in the sheets.", "followers_count": 263, "friends_count": 439, "statues_count": 8845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302082813952, "text": "40.3F (Feels: 40.3F) - Humidity: 98% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1037.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302426861570, "text": "Cool ������ https://t.co/NuKLROxWfh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 3981, "friends_count": 3124, "statues_count": 42440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302477041664, "text": "Happy birthday beautiful have a good day miss you !! ������@AlexisGeeeeeeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 337057498 }}, "user": { "id": 2841990805, "name": "Steph♡", "screen_name": "__itsjuststeph", "lang": "en", "location": "Colton, CA", "create_at": date("2014-10-05"), "description": "I don't care I'm just livin. Rialto high", "followers_count": 302, "friends_count": 300, "statues_count": 7224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302556848129, "text": "You should always aim high", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2798741966, "name": "Teresa", "screen_name": "teresavercetti", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "Instagram: @um_okbye | model w/ 3 BlackBird Mgmt", "followers_count": 197, "friends_count": 330, "statues_count": 471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302560964608, "text": "Happy bday to @playboiant , I'm glad I became one of your homies bro, have a blessed day ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 93752433 }}, "user": { "id": 2391661465, "name": "Matthew", "screen_name": "Matthew5432127", "lang": "en", "location": "null", "create_at": date("2014-03-15"), "description": "null", "followers_count": 311, "friends_count": 287, "statues_count": 10432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302632214528, "text": "You ain't shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 87120760, "name": ".", "screen_name": "9oneKingtana", "lang": "en", "location": "California, USA", "create_at": date("2009-11-02"), "description": "i've been the underdog, im cool in my doghouse", "followers_count": 417, "friends_count": 501, "statues_count": 26684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasco, CA", "id": "01643e7e5fce28b7", "name": "Wasco", "place_type": "city", "bounding_box": rectangle("-119.42052,35.572513 -119.328147,35.610926") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 683542, "cityName": "Wasco" } }
+{ "create_at": datetime("2016-01-13T00:00:02.000Z"), "id": 687182302745595904, "text": "Ripley SW Limestone Co. Temp: 29.7°F Wind:0.7mph Pressure: 1004.0mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 47395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303366270976, "text": "Can't even get a goodnight text from you��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1833067183, "name": "Abel✗♥O", "screen_name": "YociriXO", "lang": "en", "location": "Ukiah, CA 707", "create_at": date("2013-09-08"), "description": "King Lil G Follows❤️JROX . KissLand9.13.13 KOTF 10.11.14 @chantelitoxo @theweeknd @kinglilg 08J8Y14", "followers_count": 1180, "friends_count": 1088, "statues_count": 9157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ukiah, CA", "id": "0742517d53e00505", "name": "Ukiah", "place_type": "city", "bounding_box": rectangle("-123.232051,39.116505 -123.18786,39.174311") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6045, "countyName": "Mendocino", "cityID": 681134, "cityName": "Ukiah" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303643209728, "text": "New changes coming real soon ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1865217492, "name": "Coopa", "screen_name": "TrillLife_K", "lang": "en", "location": "NY", "create_at": date("2013-09-14"), "description": "the highest form of wisdom is kindness", "followers_count": 468, "friends_count": 770, "statues_count": 14644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Valley, NY", "id": "7faa457accf27bcf", "name": "Spring Valley", "place_type": "city", "bounding_box": rectangle("-74.06664,41.100326 -74.018892,41.13308") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36087, "countyName": "Rockland", "cityID": 3670420, "cityName": "Spring Valley" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303727120384, "text": "1 Yellow-bellied Sapsucker (Sphyrapicus varius) - McCormick Park - 2016-01-12 10:01 https://t.co/MdU1s6tBTS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8999872,32.2612251"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10331 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303748030465, "text": "1 Broad-tailed Hummingbird (Selasphorus platycercus) - WOW Arizona B&B - 2016-01-12 11:15 https://t.co/Nbdkiotrih", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8764428,32.5081331"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10331 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 411160, "cityName": "Catalina" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303752228864, "text": "1 Black-capped Gnatcatcher (Polioptila nigriceps) - Florida Canyon--lower - 2016-01-12 14:53 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10331 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303764852736, "text": "3 Inca Dove (Columbina inca) - WOW Arizona B&B - 2016-01-12 11:15 https://t.co/vCgGiyRzZn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8764428,32.5081331"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10331 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 411160, "cityName": "Catalina" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303773241345, "text": "1 Snow Goose (Chen caerulescens) - Christopher Columbus Park - 2016-01-12 13:30 https://t.co/uCWAoxzO2r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.0346556,32.2856814"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10331 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303794085888, "text": "Summin' summin' summin' ..I forgot now. �������� @ChiefKeef https://t.co/y7jN6AsCsQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51404605 }}, "user": { "id": 1924993712, "name": "chris hampton", "screen_name": "c_hampton33", "lang": "en", "location": "San Jose, Ca ✈️ LA", "create_at": date("2013-10-01"), "description": "Chris Hampton. HipHop Enthusiast. Ball is Life. *Official Warrior Hater* #FTW2K16", "followers_count": 428, "friends_count": 824, "statues_count": 25312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182303957762048, "text": "Can you find Redding on the map? Just try it at https://t.co/eA0Ef4kZEe #Redding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.392,40.5865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Redding" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 987, "friends_count": 312, "statues_count": 2526597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304062607360, "text": "In the wise words of YG. \"BITCH YOU BROKE, SHUT UP, DONT TALK TO ME, GET YOUR BREAD UP\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377714550, "name": "Washed Up E", "screen_name": "KinggChip", "lang": "en", "location": "HoustAtlantaLagos", "create_at": date("2011-09-21"), "description": "Don't let my tweets talk you out of giving me a chance ma. I promise these just jokes.", "followers_count": 990, "friends_count": 492, "statues_count": 82385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Land, TX", "id": "7a41192a2879ee24", "name": "Sugar Land", "place_type": "city", "bounding_box": rectangle("-95.686106,29.543372 -95.577273,29.663556") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4870808, "cityName": "Sugar Land" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304192675841, "text": "Wind 8.3 mph W. Barometer 29.66 in, Rising slowly. Temperature 26.1 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 92, "statues_count": 156637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304318337025, "text": "@chesmeeee must be nice", "in_reply_to_status": 687181632852201472, "in_reply_to_user": 393657951, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393657951 }}, "user": { "id": 215392389, "name": "Janine", "screen_name": "babymomaz_", "lang": "en", "location": "null", "create_at": date("2010-11-13"), "description": "null", "followers_count": 484, "friends_count": 71, "statues_count": 27879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304330948608, "text": "@marissalisette_ @coreyc55 off*", "in_reply_to_status": 687182128505700353, "in_reply_to_user": 973081297, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 973081297, 973285518 }}, "user": { "id": 433981251, "name": "Sergio", "screen_name": "sergioloa23", "lang": "en", "location": "California, USA", "create_at": date("2011-12-11"), "description": "PSALM 17:3", "followers_count": 435, "friends_count": 820, "statues_count": 16011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valinda, CA", "id": "b1bbf8e2ef22573a", "name": "Valinda", "place_type": "city", "bounding_box": rectangle("-117.94891,34.024734 -117.912772,34.053145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 681638, "cityName": "Valinda" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304410640384, "text": "I just made fried zucchini and eggplant w homemade ranch at midnight, I must b stopped bc I dirty 2 many dishes just making snacks ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249538752, "name": "✨gurl almighty✨", "screen_name": "saggyasssadgurl", "lang": "en", "location": "city of angels ", "create_at": date("2011-02-08"), "description": "heather #bpd / #hpd angel #healthgoth", "followers_count": 231, "friends_count": 217, "statues_count": 18230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304603668480, "text": "1 Rufous-backed Robin (Turdus rufopalliatus) - Catalina SP - 2016-01-12 07:30 https://t.co/7ll1ihhmu1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9293064,32.4186422"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10335 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304804995072, "text": "2 Rufous-capped Warbler (Basileuterus rufifrons) - Florida Canyon--lower - 2016-01-11 09:00 https://t.co/mlvNA3AiGV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 4, "friends_count": 0, "statues_count": 10336 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182304859455488, "text": "Okay, I'm gonna be honest. I really didn't want to like #TheFall because I had a certain image of @GillianA in my head. Buuuuut I adore it!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheFall" }}, "user_mentions": {{ 625022363 }}, "user": { "id": 4341612913, "name": "Mandy", "screen_name": "panickedmilagro", "lang": "en", "location": "Houston, TX", "create_at": date("2015-11-23"), "description": "Tweetin' and Momin' Momin' and Tweetin', X-Phile, BAMF", "followers_count": 178, "friends_count": 412, "statues_count": 1161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223198,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182305119584258, "text": "hdcvjfdvniydbjftb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 468141061, "name": "ᵈ", "screen_name": "omfgnisey", "lang": "en", "location": " ⬆️", "create_at": date("2012-01-18"), "description": "null", "followers_count": 1362, "friends_count": 1189, "statues_count": 34602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182305215959040, "text": "@OfficialAyce thank you ��", "in_reply_to_status": 687182164517994496, "in_reply_to_user": 2877749969, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2877749969 }}, "user": { "id": 2855098183, "name": "Marinita", "screen_name": "Marinitarromo", "lang": "en", "location": "california", "create_at": date("2014-10-13"), "description": "IG:Marinaa_14_", "followers_count": 1355, "friends_count": 1039, "statues_count": 7051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain House, CA", "id": "014b25c473600d50", "name": "Mountain House", "place_type": "city", "bounding_box": rectangle("-121.557176,37.766633 -121.533126,37.795236") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 649582, "cityName": "Mountain House" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182305312522240, "text": "Where is Redding on the map? Play the game at https://t.co/eA0Ef4kZEe #Redding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.392,40.5865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Redding" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 987, "friends_count": 312, "statues_count": 2526598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182305488547840, "text": "I'm not getting any sleep tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449402985, "name": "kuhristle", "screen_name": "Krystyllynn", "lang": "en", "location": "California☀", "create_at": date("2011-12-28"), "description": "It's been a crazy journey.. #RIPCAM", "followers_count": 935, "friends_count": 577, "statues_count": 61278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oroville East, CA", "id": "2a152e11137e65d0", "name": "Oroville East", "place_type": "city", "bounding_box": rectangle("-121.538364,39.479173 -121.456825,39.53656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 654388, "cityName": "Oroville East" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182306063302656, "text": "@spiderslash when ur friend crushes ur soul https://t.co/arawje23uA", "in_reply_to_status": 687182135967506432, "in_reply_to_user": 77648985, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77648985 }}, "user": { "id": 77648985, "name": "Fight Money", "screen_name": "spiderslash", "lang": "en", "location": "Melbourne, FL", "create_at": date("2009-09-26"), "description": "hi I'm Hunter and I like video games and pro wrestling. I love all my friends.", "followers_count": 85, "friends_count": 462, "statues_count": 8117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melbourne, FL", "id": "d970e7fd69eaf4ec", "name": "Melbourne", "place_type": "city", "bounding_box": rectangle("-80.706074,28.03509 -80.593324,28.200658") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1243975, "cityName": "Melbourne" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182306478403584, "text": "How you ... man nvrm.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329379054, "name": "Bitch", "screen_name": "Trizaaay", "lang": "en", "location": "Compton, CA", "create_at": date("2011-07-04"), "description": "pretty soul, pretty mind", "followers_count": 1109, "friends_count": 510, "statues_count": 69666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182306516193280, "text": "@CurrierMegan damn Meg get it girl", "in_reply_to_status": 687182003741929472, "in_reply_to_user": 439116038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 439116038 }}, "user": { "id": 1363083985, "name": "Stephen Jacobs", "screen_name": "stephenjacobs27", "lang": "en", "location": "Murray State University", "create_at": date("2013-04-18"), "description": "GoodVibeTribe", "followers_count": 821, "friends_count": 704, "statues_count": 19347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, TN", "id": "52f4a98d03e4ea4f", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-87.486546,36.4602 -87.157791,36.641767") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47125, "countyName": "Montgomery", "cityID": 4715160, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182306679721985, "text": "������ https://t.co/LNEVN4afor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 29792729, "name": "CottonMouthPCHS", "screen_name": "50ftPchs", "lang": "en", "location": "Nicki M's basement", "create_at": date("2009-04-08"), "description": "Look Listen Choose Act. THE QueerMexicanBarbie @NICKIMINAJ @LilTunechi Tell em im a NINJA Weezy is my sensei. MÉX #IAintGotNoWorries", "followers_count": 963, "friends_count": 2008, "statues_count": 9211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-01-13T00:00:03.000Z"), "id": 687182307506008064, "text": "@CompoundMixture @_QueenEve_ actually I'm right there with y'all! You thought ��", "in_reply_to_status": 687181842399690752, "in_reply_to_user": 3253362122, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3253362122, 2371013731 }}, "user": { "id": 39064634, "name": "Jija ➰", "screen_name": "dija_11", "lang": "en", "location": "null", "create_at": date("2009-05-10"), "description": "Stuck up & Unapproachable ➰", "followers_count": 1207, "friends_count": 546, "statues_count": 29753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182307975802880, "text": "@Perkrin_2 nah, es temprano aquí. Son las 00h ahora. Ya volviendo para el hotel de todas formas, que mañana la primera charla es a las 8h.", "in_reply_to_status": 687178978533208065, "in_reply_to_user": 3289660275, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3289660275 }}, "user": { "id": 4645171, "name": "Kilian Arjona", "screen_name": "karjona", "lang": "es", "location": "Barcelona", "create_at": date("2007-04-14"), "description": "¡Me hace sentir como una colegiala!", "followers_count": 498, "friends_count": 392, "statues_count": 15460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182307992727552, "text": "Forgive them even if they're not sorry.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 860596861, "name": "✨MaHayley✨", "screen_name": "Mayhay1388", "lang": "en", "location": "Bama", "create_at": date("2012-10-03"), "description": "snapchat: mayhay01 ,striving for progress not perfection.", "followers_count": 465, "friends_count": 406, "statues_count": 5261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsboro, AL", "id": "639bb2fb92904efd", "name": "Scottsboro", "place_type": "city", "bounding_box": rectangle("-86.096764,34.597525 -85.995355,34.695664") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1071, "countyName": "Jackson", "cityID": 168736, "cityName": "Scottsboro" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182308131082245, "text": "Wind 4.0 mph WSW. Barometer 29.887 in, Rising Rapidly. Temperature 14.7 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182308428787712, "text": "@smh_katj I shall show you more when we get back to school", "in_reply_to_status": 687181040754929664, "in_reply_to_user": 264225103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 264225103 }}, "user": { "id": 348657837, "name": "Matthew Perera", "screen_name": "matperera", "lang": "en", "location": "California", "create_at": date("2011-08-04"), "description": "Stay tuned for greatness///APU Class of 2019", "followers_count": 380, "friends_count": 331, "statues_count": 3816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182308676259840, "text": "@_Legendary_6 can't sleep! Wyd bro?", "in_reply_to_status": 687181984754483201, "in_reply_to_user": 2975121819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2975121819 }}, "user": { "id": 243902395, "name": "ashley hannan", "screen_name": "smaaashhhh", "lang": "en", "location": "pittsburgh, pa", "create_at": date("2011-01-27"), "description": "#KentuckyStateUniversity #HBCU", "followers_count": 1197, "friends_count": 1172, "statues_count": 33946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monessen, PA", "id": "b46788b182ef260e", "name": "Monessen", "place_type": "city", "bounding_box": rectangle("-79.90373,40.127854 -79.857816,40.164489") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4250344, "cityName": "Monessen" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182308705759232, "text": "https://t.co/tCrq5DBXsX\n#artistdolls #textiletoys #toys #handwork #handmadetoys #doll #textileDolls #HandmadeDolls https://t.co/fQA8LRIrlp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "artistdolls", "textiletoys", "toys", "handwork", "handmadetoys", "doll", "textileDolls", "HandmadeDolls" }}, "user": { "id": 73613654, "name": "Gia Basilia", "screen_name": "GiaDecor", "lang": "en", "location": "Israel", "create_at": date("2009-09-12"), "description": "Artist and Decorate Designer Gia Basilia. художник декоратор", "followers_count": 246, "friends_count": 809, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182308860936192, "text": "LADIES NIGHT!!!\nGROUP OF 6+ LADIES GET A FREE BOTTLE!!!\n2 HOURS OPEN BAR!! 10-12\nJUST DANCE… https://t.co/LDVDQo73Hk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9009628,40.7775803"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2316562985, "name": "Danny Nyce", "screen_name": "DannyNyce", "lang": "en", "location": "null", "create_at": date("2014-01-31"), "description": "null", "followers_count": 36, "friends_count": 108, "statues_count": 1484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309037084672, "text": "@TrilWill_ been one ����", "in_reply_to_status": 687182139683672065, "in_reply_to_user": 420100937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 420100937 }}, "user": { "id": 220894988, "name": "Z.Hearn ☮", "screen_name": "kingzavia_", "lang": "en", "location": "msu19", "create_at": date("2010-11-28"), "description": "crown me.", "followers_count": 2600, "friends_count": 1669, "statues_count": 59408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Starkville, MS", "id": "ed1736d76c1e007c", "name": "Starkville", "place_type": "city", "bounding_box": rectangle("-88.866974,33.406706 -88.776403,33.506302") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28105, "countyName": "Oktibbeha", "cityID": 2870240, "cityName": "Starkville" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309225852928, "text": "I'm chillin ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37245052, "name": "〽️ike Nobles ⛄️", "screen_name": "__CocaineCowboy", "lang": "en", "location": "Mississippi Mudd ", "create_at": date("2009-05-02"), "description": "$$$$$$", "followers_count": 2492, "friends_count": 901, "statues_count": 165276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, MS", "id": "57dc070bcd27882d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-88.763404,32.312389 -88.655698,32.443049") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2846640, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309317988353, "text": "People who are angry with each other for the silly little things, are usually the ones who care about each other the most.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131226663, "name": "আমি কই...!", "screen_name": "faisalbdctg", "lang": "en", "location": "Atlantic City, NJ,USA.", "create_at": date("2010-04-09"), "description": "তুমি তোমার জীবনে ভেজাল হাসিকে আসতে দিও না। মনে কষ্ট পেলে কাদবে। মনের কষ্ট চাপা দেয়ার জন্য হাসির ভান করার প্রয়োজন নেই.......!", "followers_count": 675, "friends_count": 140, "statues_count": 1790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlantic City, NJ", "id": "7ad0e3081108f4ba", "name": "Atlantic City", "place_type": "city", "bounding_box": rectangle("-74.467407,39.342291 -74.40702,39.386729") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3402080, "cityName": "Atlantic City" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309347430400, "text": "Fuck off nigga. i learned from my mistake, i tried being the nice helpful person time & time again but that just seems to screw me over.��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 608633613, "name": "Don Ibarra", "screen_name": "donepi808", "lang": "en", "location": "null", "create_at": date("2012-06-14"), "description": "null", "followers_count": 110, "friends_count": 134, "statues_count": 3328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309536092160, "text": "Im in a bad mood so Kayla is putting on different spotify playlist. EX: PMS, forever alone, melancholy �� I CANT!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 536133549, "name": "Ness", "screen_name": "VanessaRod_10", "lang": "en", "location": "Dinuba//Costa Mesa ", "create_at": date("2012-03-24"), "description": "Through Him, By Him & For Him // VUSC", "followers_count": 807, "friends_count": 747, "statues_count": 22524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182309964005376, "text": "@KushinJuice �������� Nah G", "in_reply_to_status": 687182215386644480, "in_reply_to_user": 3290545216, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 3290545216 }}, "user": { "id": 377675317, "name": "SK III", "screen_name": "SayyidKanu4", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "| Official Page of Sayyid Kanu | Morehead State Student Athlete 3⃣", "followers_count": 846, "friends_count": 603, "statues_count": 19355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182310374952963, "text": "@stacked @Deprivals RIP Phillip... He has succumbed to the non-hoe life", "in_reply_to_status": 687182171682029568, "in_reply_to_user": 369736242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 369736242, 1352388534 }}, "user": { "id": 595133308, "name": "Backyard BDA", "screen_name": "SuplexingHoes", "lang": "en", "location": "The Backyard", "create_at": date("2012-05-30"), "description": "destiné à mourir #BBT #BlackLivesMatter #TXST19 https://twitter.com/suplexinghoes/status/686361905674362881", "followers_count": 878, "friends_count": 265, "statues_count": 75908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182310521765888, "text": "Check out @labronzejade stream... @TwitchSharer https://t.co/kR2DdBrmZ9 !!! <3 #ff7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ff7" }}, "user_mentions": {{ 3271489123, 2910985258 }}, "user": { "id": 1433937787, "name": "[TPwn] CapNeveryOne", "screen_name": "CapN408", "lang": "en", "location": "Yay Area ", "create_at": date("2013-05-16"), "description": "A gamer at heart and most importantly a smartass..GT: CapNeveryOne", "followers_count": 1694, "friends_count": 2341, "statues_count": 5421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182311335460864, "text": "@_travissss @otb_tae y'all boys don't love me son ��", "in_reply_to_status": 687182080912965634, "in_reply_to_user": 2424243186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2424243186, 2779383764 }}, "user": { "id": 375464868, "name": "Malik Jaleel Jackson", "screen_name": "_mvlikkk", "lang": "en", "location": "Marrero, LA", "create_at": date("2011-09-17"), "description": "RestStup", "followers_count": 1294, "friends_count": 689, "statues_count": 31166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmere, LA", "id": "25041b89e1b96118", "name": "Woodmere", "place_type": "city", "bounding_box": rectangle("-90.089118,29.838214 -90.066639,29.874275") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2283002, "cityName": "Woodmere" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182311490596864, "text": "Oh hey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 973091905, "name": "uʎlǝɔoɾ ıpnɔsǝɯ", "screen_name": "Jocedanielle", "lang": "en", "location": "Escondido, CA", "create_at": date("2012-11-26"), "description": "#CudFam", "followers_count": 820, "friends_count": 406, "statues_count": 29901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escondido, CA", "id": "00610f4d3a382ec1", "name": "Escondido", "place_type": "city", "bounding_box": rectangle("-117.146102,33.057782 -116.998535,33.182936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622804, "cityName": "Escondido" } }
+{ "create_at": datetime("2016-01-13T00:00:04.000Z"), "id": 687182311532675072, "text": "Ive watched every season of #TheRealHousewives since the 1st season aired in Oct of '08 & out of ALL of them @erikajayne is BY FAR my fav ❤️", "in_reply_to_status": 687146652700811264, "in_reply_to_user": 16803519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheRealHousewives" }}, "user_mentions": {{ 16803519 }}, "user": { "id": 466151215, "name": "Sara Ann Davis", "screen_name": "sa_davis14", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "•26 •Aquarius •704 •Animal Lover •Sports Fanatic (Go LAC and Panthers!) •Free Spirit •I'm daggum hilarious •JLM", "followers_count": 1138, "friends_count": 1703, "statues_count": 30180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gastonia, NC", "id": "e7aa53e3e1531b99", "name": "Gastonia", "place_type": "city", "bounding_box": rectangle("-81.281029,35.173615 -81.083206,35.305409") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37071, "countyName": "Gaston", "cityID": 3725580, "cityName": "Gastonia" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182312111497216, "text": "@Yung_Messiah cuz you never left St. Louis cuz you uncultured you've never heard of civilized places in Virginia. Go shoot somebody", "in_reply_to_status": 687181855766884352, "in_reply_to_user": 25662338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25662338 }}, "user": { "id": 34557853, "name": "Misimović", "screen_name": "SlicKSaneL", "lang": "en", "location": "United States", "create_at": date("2009-04-23"), "description": "A has been. My tweets aint shit. I aint shit. Cunt. Allegedly. Captain of #TeamNoHoes Aint never told a lie. Tweeter of the year 2010-2012 #MUFC #DallasCowboys", "followers_count": 1002, "friends_count": 812, "statues_count": 146228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roanoke, VA", "id": "9d63050d3d33d32f", "name": "Roanoke", "place_type": "city", "bounding_box": rectangle("-80.037735,37.211374 -79.878085,37.337603") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51770, "countyName": "Roanoke", "cityID": 5168000, "cityName": "Roanoke" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182312480477184, "text": "Temp: -1.4°F - Dew Point: -13.9° - Wind: 8.2 mph - Gust: 15.2 - Rain Today: 0.00in. - Pressure: 30.16in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 11931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182312769863680, "text": "I might need a second opinion but seen white do far worst https://t.co/vToT3n6Uuw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332674776, "name": "Coach", "screen_name": "GarvenB", "lang": "en", "location": "null", "create_at": date("2014-02-07"), "description": "Never Satisfied", "followers_count": 321, "friends_count": 270, "statues_count": 14412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alafaya, FL", "id": "00d511d335cd9fb6", "name": "Alafaya", "place_type": "city", "bounding_box": rectangle("-81.244501,28.489488 -81.125632,28.568794") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1200410, "cityName": "Alafaya" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182313092952065, "text": "https://t.co/az1Lmswfpm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2885108150, "name": "Chloe", "screen_name": "khloemercado", "lang": "en", "location": "yalien ", "create_at": date("2014-10-31"), "description": "☹ // Fl ☀️ // Sagittarius // Jupiter", "followers_count": 406, "friends_count": 268, "statues_count": 5191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Port Richey, FL", "id": "79dc2655046fbc40", "name": "New Port Richey", "place_type": "city", "bounding_box": rectangle("-82.764606,28.216424 -82.66958,28.28063") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1248500, "cityName": "New Port Richey" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182313101373440, "text": "@David_Cameron denies 'backsliding' on environment https://t.co/nJxMp94mJs", "in_reply_to_status": -1, "in_reply_to_user": 103065157, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 103065157 }}, "user": { "id": 127057663, "name": "nico phillips", "screen_name": "nicosuavehh", "lang": "en", "location": "London W2", "create_at": date("2010-03-27"), "description": "We all want to believe we are pizza rat but secretly know we're cotton candy raccoon.Proudly with Love Activists London (Norbert Lawrie crew) & Occupy Democracy", "followers_count": 434, "friends_count": 796, "statues_count": 27361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182313403232256, "text": "It's officially January 13.. Which means I'm 23.. Just like the greatest ����2️⃣3️⃣���� https://t.co/KSai620HE2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60846454, "name": "Denise Velasco", "screen_name": "VelascoDenise_", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2009-07-28"), "description": "Dodgers⚾️ Real Madrid ⚽️ Chicago Bulls. USC. El Salvador :) Music. CSUDH: Psychology. Steven Bosh ❤️", "followers_count": 880, "friends_count": 540, "statues_count": 139651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-13T00:00:05.000Z"), "id": 687182314053304320, "text": "Wind 0 mph ---. Barometer 1011.5 hPa, Falling slowly. Temperature 70.3 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 90, "friends_count": 266, "statues_count": 141396 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Miradero, Puerto Rico", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544679957565440, "text": "@enemycollision yes of course b it just sounds so soothing right", "in_reply_to_status": 687544353149992960, "in_reply_to_user": 18255552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18255552 }}, "user": { "id": 549730034, "name": "wILLIAM HUNGOVER", "screen_name": "suddenlygarmo", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "spring break 1998 no regrets", "followers_count": 136, "friends_count": 212, "statues_count": 4871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544680116912128, "text": "all I smell is crayons.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546516156, "name": "tay", "screen_name": "taytrillakilla", "lang": "en", "location": "San Marcos, TX", "create_at": date("2012-04-05"), "description": "live by the sun, love by the moon | 5'1 but I stand tall | #TXST | #HeightOverHeart", "followers_count": 1285, "friends_count": 1250, "statues_count": 37267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544680263725056, "text": "@PAWXS �� Lololol", "in_reply_to_status": 687542436156039168, "in_reply_to_user": 346289745, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 346289745 }}, "user": { "id": 2781528980, "name": "Ciara", "screen_name": "odell_ciara", "lang": "en", "location": "Bay Area ", "create_at": date("2014-08-30"), "description": "Forever & Always ❤️", "followers_count": 250, "friends_count": 228, "statues_count": 13217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544680817520641, "text": "@brandonorl https://t.co/kVIN1xQTdF", "in_reply_to_status": 687544409659977728, "in_reply_to_user": 4487497821, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4487497821 }}, "user": { "id": 2933085938, "name": "Anna", "screen_name": "UrieWeekess", "lang": "en", "location": "purgatory", "create_at": date("2014-12-16"), "description": "son of a bitch", "followers_count": 61, "friends_count": 173, "statues_count": 15924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doctor Phillips, FL", "id": "3b05a9986be2419b", "name": "Doctor Phillips", "place_type": "city", "bounding_box": rectangle("-81.508635,28.420712 -81.474987,28.493573") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1217725, "cityName": "Doctor Phillips" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544680989380608, "text": "Have you seen the animals at the Phoenix zoo they all look so damn sad and unhealthy I feel bad for them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1164376915, "name": "dale", "screen_name": "tariqmiller_", "lang": "en", "location": "null", "create_at": date("2013-02-09"), "description": "null", "followers_count": 1273, "friends_count": 894, "statues_count": 41650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.635016,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544681333276672, "text": "Im breaking out ugh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1344446750, "name": "Alejandra", "screen_name": "UnlikeAle", "lang": "en", "location": "Hollywood ", "create_at": date("2013-04-11"), "description": "im probably taking a nap if im offline", "followers_count": 8810, "friends_count": 147, "statues_count": 41842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544681341693952, "text": "@smoothie_bb oh I thought you'd being partying rn", "in_reply_to_status": 687544116763213824, "in_reply_to_user": 145450278, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 145450278 }}, "user": { "id": 3953763793, "name": "Gavin", "screen_name": "ihaveaids69_", "lang": "en", "location": "null", "create_at": date("2015-10-19"), "description": "beef jerky", "followers_count": 139, "friends_count": 135, "statues_count": 1408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544681404723200, "text": "Uno & Chill?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 633144541, "name": "Bart Pimpin'", "screen_name": "Kjm14_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-07-11"), "description": "19 years in the trenches. \n #Savage$zn |TXST#14|", "followers_count": 2328, "friends_count": 1838, "statues_count": 74625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544681534599168, "text": "@MindBlowing @AdventureSpots @sydneykinsey", "in_reply_to_status": 686404358473297920, "in_reply_to_user": 488908367, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 488908367, 2928749441, 165626181 }}, "user": { "id": 629356434, "name": "relatable teen", "screen_name": "TateTodea", "lang": "en", "location": "null", "create_at": date("2012-07-07"), "description": "pineapple", "followers_count": 227, "friends_count": 172, "statues_count": 3354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544681786294275, "text": "@lka206 happy birthday bumm���� turn up your 18 lol", "in_reply_to_status": -1, "in_reply_to_user": 173247603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 173247603 }}, "user": { "id": 1360281433, "name": "⚠️LightskinBoi⚠️", "screen_name": "aye_its_phillip", "lang": "en", "location": "null", "create_at": date("2013-04-17"), "description": "When you want to suceed, As bad as you want to breathe, Then youll be sucessful.~Eric Thomas", "followers_count": 742, "friends_count": 1198, "statues_count": 13815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562109,47.166863 -122.348569,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544682046341120, "text": "Al Bundy: \"The Bears are playing the Rams and if you lose to the Rams, you get thrown out of the league!!\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37529237, "name": "Imaginary Man", "screen_name": "Batman_Infinite", "lang": "en", "location": "Imaginary Galaxy ", "create_at": date("2009-05-03"), "description": "@Selina_Infinite is my love. Those born to fight cannot rest. #NativeAmerican #BeTheBatman #KeepPounding", "followers_count": 562, "friends_count": 1091, "statues_count": 162871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544682310713344, "text": "Fantasy to a nightmare real quick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 305232839, "name": "Mike", "screen_name": "BreezeyBoy23", "lang": "en", "location": "Commack, NY", "create_at": date("2011-05-25"), "description": "Where's your will to be weird? 9:26:15", "followers_count": 527, "friends_count": 143, "statues_count": 25873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commack, NY", "id": "298bb08a9d7e64d4", "name": "Commack", "place_type": "city", "bounding_box": rectangle("-73.321097,40.803631 -73.238846,40.880333") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3617530, "cityName": "Commack" } }
+{ "create_at": datetime("2016-01-14T00:00:00.000Z"), "id": 687544682713321472, "text": "Wind 0 mph --. Barometer 30.16 in, Falling slowly. Temperature 37.0 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544682990014464, "text": "Lowkey you look like Jesus �� https://t.co/IaL4sH7VNP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329442612, "name": "Jan24 ✨", "screen_name": "ChelseaCorenia", "lang": "en", "location": "California ", "create_at": date("2011-07-04"), "description": "SC : Chelseacoreniaa 20", "followers_count": 759, "friends_count": 485, "statues_count": 44370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544683195691009, "text": "@liebe_laur you can sing too? ��", "in_reply_to_status": 687544378529853440, "in_reply_to_user": 69358273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69358273 }}, "user": { "id": 286964296, "name": "Chino Hills", "screen_name": "___TC_", "lang": "en", "location": "Drexel University '18", "create_at": date("2011-04-23"), "description": "RIP Johnny Wyche Sunset: 8/9/2015", "followers_count": 1049, "friends_count": 916, "statues_count": 97822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544683225083904, "text": "@bellymets lol better be safe ��", "in_reply_to_status": 687544517977845760, "in_reply_to_user": 406666425, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 406666425 }}, "user": { "id": 517197125, "name": "Jameel Bailey", "screen_name": "JBailey_13", "lang": "en", "location": "nowhere near you", "create_at": date("2012-03-06"), "description": "11.30.13 .. 6⃣0⃣9⃣ / Using My Talents To Beat The Struggle..", "followers_count": 1752, "friends_count": 1088, "statues_count": 20518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, NJ", "id": "74767081120f9b42", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-74.817535,40.183967 -74.729138,40.248298") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3474000, "cityName": "Trenton" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544683313127424, "text": "I miss staying in da A.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231149978, "name": "TaeMontana", "screen_name": "QHTrey5", "lang": "en", "location": "BullyDurham", "create_at": date("2010-12-27"), "description": "#RiPDAD #ChickenChasin", "followers_count": 1221, "friends_count": 828, "statues_count": 30979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544683325685760, "text": "01/14@03:00 - Temp 24.8F, WC 24.8F. Wind 0.2mph S, Gust 2.0mph. Bar 30.012in, Falling. Rain 0.00in. Hum 60%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544683703058432, "text": "Wind 7.0 mph SSW. Barometer 1017.11 mb, Falling. Temperature 40.2 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544684424466432, "text": "this. �� https://t.co/aa6MjOlGyx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264913334, "name": "beckssss", "screen_name": "Gimmethabecky_", "lang": "en", "location": "Hesperia, CA☀️", "create_at": date("2011-03-12"), "description": "employed x3 | imports | she ain't thinking about love, she got her mind on her money", "followers_count": 1359, "friends_count": 974, "statues_count": 56353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544684433027073, "text": "\"And the stars [on the #HollywoodWalkOfFame] look very different tonight\" ⭐️ #RIPdavidbowie… https://t.co/8qxiYwVHtT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.33940773,34.10171884"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HollywoodWalkOfFame", "RIPdavidbowie" }}, "user": { "id": 15955769, "name": "TREИT ⚡️ VAИEGAS", "screen_name": "trentvanegas", "lang": "en", "location": "LA by way of Detroit", "create_at": date("2008-08-23"), "description": "I'm Trent from Pink is the new Blog: Everybody's Business Is My Business ... and I love tweeting about it!", "followers_count": 15452, "friends_count": 907, "statues_count": 56336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544684533706752, "text": "I seriously deserve that power ball money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250392751, "name": "~Andrew ~", "screen_name": "ArochoAndrew", "lang": "en", "location": "Long Island, New York", "create_at": date("2011-02-10"), "description": "~Wucho~ snapchat/insta: AndrewArocho", "followers_count": 832, "friends_count": 1077, "statues_count": 24105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Wantagh, NY", "id": "a5c44787002590e5", "name": "North Wantagh", "place_type": "city", "bounding_box": rectangle("-73.531834,40.690706 -73.482403,40.707903") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3653792, "cityName": "North Wantagh" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544684542038016, "text": "Wind 0.0 mph ---. Barometer 30.090 in, Falling. Temperature 31.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 56958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544684823101440, "text": "في قسوتك .. \nالقلب عايش على كل لحظه يقتنص فيها \n \"عطفك المتنكر في جفاك\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 333991307, "name": "Hattan_Omrani", "screen_name": "hattan_omrani", "lang": "en", "location": "Illinois, USA", "create_at": date("2011-07-12"), "description": "(Program or be programmed) To always choose the sure things is treason for the soul.", "followers_count": 424, "friends_count": 1212, "statues_count": 2765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, VA", "id": "d6819fe60643ebc1", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-77.144435,38.789907 -77.037304,38.844853") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5101000, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544685494050816, "text": "i was gna get subway.. then remembered alberto's, soooo.. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1904675659, "name": "dahls.", "screen_name": "dahliababyyy", "lang": "en", "location": "southern california.", "create_at": date("2013-09-25"), "description": "@b_pese ♡ samoan.", "followers_count": 1026, "friends_count": 797, "statues_count": 39722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610858,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544685611642881, "text": "It's like I gotta go get a Gatorade every night before I go in the house", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272946916, "name": "Bri ❣", "screen_name": "cheetahbriXO", "lang": "en", "location": "w/ @myfabolouslife", "create_at": date("2011-03-27"), "description": "Q U E E N", "followers_count": 1328, "friends_count": 1148, "statues_count": 39618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, VA", "id": "60edfde178b362ff", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-76.420402,36.786146 -76.291434,36.897382") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51740, "countyName": "Portsmouth", "cityID": 5164000, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544685804429313, "text": "I thought this was the fucker from \nI Am Legend!! https://t.co/iFUQrxcNqm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875965278, "name": "perry", "screen_name": "perryemilio", "lang": "en", "location": "null", "create_at": date("2014-10-24"), "description": "love", "followers_count": 298, "friends_count": 271, "statues_count": 935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544686051983360, "text": "03:00:01 |Temp: 26.9ºF | Wind Chill 26.9ºF |Dew Point 23.8ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544686152650752, "text": "Me + you, then we add another girl up in the room.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363243890, "name": "caytlynne. ♛", "screen_name": "CaytAndriannax", "lang": "en", "location": "null", "create_at": date("2014-02-24"), "description": "XI•VI•XCIV | CaytlynneAndriannaLindamood.| | SC;CaytAndrianna | Kentucky Wildcat basketball. |", "followers_count": 899, "friends_count": 546, "statues_count": 66317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, IN", "id": "01a330475b2baff2", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-87.386901,38.012637 -87.353151,38.050803") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18173, "countyName": "Warrick", "cityID": 1812034, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544686173667328, "text": "@Skihum smoked the cigarette with your name on it and no I'm not sorry.", "in_reply_to_status": 687544530564952064, "in_reply_to_user": 371625501, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371625501 }}, "user": { "id": 401098543, "name": "Oliviarms", "screen_name": "ofelks", "lang": "en", "location": "Rochester, NY", "create_at": date("2011-10-29"), "description": "Twenty1, senior at SJFC, enjoying life", "followers_count": 185, "friends_count": 257, "statues_count": 13653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Geneva, NY", "id": "289cb0265e285489", "name": "Geneva", "place_type": "city", "bounding_box": rectangle("-77.022469,42.833182 -76.963332,42.897359") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36069, "countyName": "Ontario", "cityID": 3628640, "cityName": "Geneva" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544686198702080, "text": "dis foreign film bout to fuck me up!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 203397229, "name": "chris", "screen_name": "chrisestrawduh", "lang": "en", "location": "Houstatlantavegas", "create_at": date("2010-10-15"), "description": "pay no mind.", "followers_count": 292, "friends_count": 249, "statues_count": 23807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-01-14T00:00:01.000Z"), "id": 687544686643417089, "text": "Wind 0.0 mph ---. Barometer 30.046 in, Steady. Temperature 19.7 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687167602689, "text": "And that thang so fye baby no propane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 523723016, "name": "carlos.", "screen_name": "CjHicks_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-03-13"), "description": "sheesh. fwtx | #TXST19", "followers_count": 1629, "friends_count": 978, "statues_count": 3584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Batesville, AR", "id": "2f5745deaec67fb6", "name": "Batesville", "place_type": "city", "bounding_box": rectangle("-91.669914,35.755315 -91.580241,35.816237") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5063, "countyName": "Independence", "cityID": 504030, "cityName": "Batesville" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687284981760, "text": "You drive me crazy but baby I wouldn't trade you for the world, crazy in love with you ❤️ #happy2months https://t.co/4FPYRaAQq4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happy2months" }}, "user": { "id": 489088815, "name": "M.S✨", "screen_name": "nicole_anissa", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-02-10"), "description": "they don't want you to smile.", "followers_count": 2072, "friends_count": 756, "statues_count": 33486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687348035585, "text": "1h 45m wait time at Stollery Children's Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.52363,53.519829"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 369, "friends_count": 889, "statues_count": 2620 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687406661632, "text": "@M__Garcia18 your girl https://t.co/Ho4GQ61R0z", "in_reply_to_status": -1, "in_reply_to_user": 3232925898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3232925898 }}, "user": { "id": 3111731731, "name": "EL DON", "screen_name": "IvanFromPanera", "lang": "en", "location": "null", "create_at": date("2015-03-27"), "description": "My name is Ivan, I work at Panera and Leslie is my boo thang", "followers_count": 173, "friends_count": 140, "statues_count": 832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodburn, OR", "id": "fa0be154c7d8a74d", "name": "Woodburn", "place_type": "city", "bounding_box": rectangle("-122.892863,45.1277 -122.822088,45.162838") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4183750, "cityName": "Woodburn" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687503097856, "text": "I put the d in \"my girlfriend\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1548604843, "name": "brutal buttfucker", "screen_name": "yaboiipadilla", "lang": "en", "location": "null", "create_at": date("2013-06-26"), "description": "S❤️", "followers_count": 557, "friends_count": 569, "statues_count": 20844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687645835265, "text": "Wind 3.0 mph SE. Barometer 29.903 in, Falling. Temperature 37.3 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544687851249664, "text": "@BuckyHermit wish you lived closer...I would play a game with ya! Have Starfarers too!!", "in_reply_to_status": 687541922815057921, "in_reply_to_user": 54432119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 54432119 }}, "user": { "id": 397030452, "name": "Tasha Florentz-Clift", "screen_name": "WeWillQuackYou", "lang": "en", "location": "Long Beach, California", "create_at": date("2011-10-23"), "description": "Lover of all things hockey. Writer for Ducks N Pucks! Fan of Ducks, Senators, Lightning and countless players. Hockey player. Gamer. Nerd. Geek. LVN. Swim coach", "followers_count": 568, "friends_count": 688, "statues_count": 20608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688065155074, "text": "Temp: 36.9°F Wind:0.0mph Pressure: 30.151hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 56960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688077742081, "text": "I wish I could say some shit i really wanna say... but I know it will destroy you therfore i keep my mouth shhut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 636116368, "name": "Mr.Wiggles", "screen_name": "dougnichols69", "lang": "en", "location": "where ever you find me ", "create_at": date("2012-07-15"), "description": "I'm just a young kid trying to escape reality", "followers_count": 302, "friends_count": 369, "statues_count": 4298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie Ridge, WA", "id": "40255cddc5325075", "name": "Prairie Ridge", "place_type": "city", "bounding_box": rectangle("-122.185728,47.108756 -122.097327,47.165918") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356170, "cityName": "Prairie Ridge" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688174219265, "text": "46.5F (Feels: 46.5F) - Humidity: 86% - Wind: 6.0mph E - Gust: 7.6mph - Pressure: 1031.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688308436992, "text": "can we stay home tonight. try something new tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708697551, "name": "Kayyy.♡", "screen_name": "KaMii28_", "lang": "en", "location": "Dinuba, CA", "create_at": date("2013-08-28"), "description": "living life to the fullest.✨", "followers_count": 760, "friends_count": 853, "statues_count": 12125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688421650433, "text": "@NawRob @robnaaw @RobNaww y'all niggas gay", "in_reply_to_status": 687540213275443200, "in_reply_to_user": 497462022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497462022, 4756065554, 2787250867 }}, "user": { "id": 155118292, "name": "#OnToKC Pats 12-4", "screen_name": "iAM_RodWill", "lang": "en", "location": "Shreveport, LA", "create_at": date("2010-06-12"), "description": "Visual Arts major(digital). Stay humble, stay hungry. PatsNation. KnicksTape. ChaserGANG #GramFam", "followers_count": 4739, "friends_count": 1993, "statues_count": 448201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grambling, LA", "id": "7659c27719e754b6", "name": "Grambling", "place_type": "city", "bounding_box": rectangle("-92.734385,32.50363 -92.669504,32.551821") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2230515, "cityName": "Grambling" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688501518336, "text": "I can't really tell whats colder, my soul or the snow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248022222, "name": "chris", "screen_name": "chris_ryan96", "lang": "en", "location": "Illinois State University", "create_at": date("2011-02-05"), "description": "I was hatched from a Versace egg • ΔΧ", "followers_count": 3537, "friends_count": 825, "statues_count": 17246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688899788801, "text": "@simipra i cant do bullet hell im not made for that life", "in_reply_to_status": 687544221864210432, "in_reply_to_user": 520305623, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 520305623 }}, "user": { "id": 586059942, "name": "knife kitty", "screen_name": "bearhime", "lang": "en", "location": "mildly nsfw|she/they|20|OR", "create_at": date("2012-05-20"), "description": "im sorry, you must not know who i am. im charlie smidge. the werewolf elitest.", "followers_count": 499, "friends_count": 881, "statues_count": 133730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544688912433152, "text": "damn, ok", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1242429674, "name": "anissajo", "screen_name": "Anissa_Jo17", "lang": "en", "location": "TX", "create_at": date("2013-03-04"), "description": "chunky, yet funky", "followers_count": 542, "friends_count": 750, "statues_count": 25268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, TX", "id": "95f1c9b1f400498f", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-97.83719,26.241771 -97.809433,26.269258") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4865774, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544689566810112, "text": "I'm finally 18 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3043392956, "name": "+", "screen_name": "_ar0ss", "lang": "en", "location": "rip troy ❤️", "create_at": date("2015-02-26"), "description": "my friends call me Money, I think differently than most 17 year olds #hamptonu20", "followers_count": 1870, "friends_count": 1723, "statues_count": 19498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544690049069057, "text": "@WORLDSTARC0MEDY @ChaseStone03", "in_reply_to_status": 683173310453039104, "in_reply_to_user": 1321629762, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1321629762, 3226021970 }}, "user": { "id": 309328518, "name": "Alex Taft", "screen_name": "AlexanderTaft", "lang": "en", "location": "✭'Go Boys'✭", "create_at": date("2011-06-01"), "description": "Twitters most wanted", "followers_count": 267, "friends_count": 251, "statues_count": 4094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544690225221632, "text": "Fuck all that fake shit & gossip!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586282010, "name": "dremoney", "screen_name": "_dremoney17", "lang": "en", "location": "El Centro, CA", "create_at": date("2013-07-11"), "description": "All I no is addition➕blame it on my ambition Chasing A Dream WAAAHLIFESTYLEBRAND Baltimore City 2 Cali", "followers_count": 1021, "friends_count": 775, "statues_count": 11944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Centro, CA", "id": "00436a376f214be7", "name": "El Centro", "place_type": "city", "bounding_box": rectangle("-115.594987,32.730647 -115.501098,32.818522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 621782, "cityName": "El Centro" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544690640486400, "text": "It's crazy but I'm enjoying it rn https://t.co/mX3lK0ICBS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1129777429, "name": "Riogan", "screen_name": "rioganrendon", "lang": "en", "location": "Home", "create_at": date("2013-01-28"), "description": "MMHS Junior c/o 2017", "followers_count": 240, "friends_count": 160, "statues_count": 21608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-14T00:00:02.000Z"), "id": 687544690934067200, "text": "I really should do my makeup 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1127080220, "name": "jAS_MYNE", "screen_name": "jazzzy18", "lang": "en", "location": "null", "create_at": date("2013-01-27"), "description": "Welcome to my sad happy life", "followers_count": 292, "friends_count": 217, "statues_count": 18304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544691244548098, "text": "#SupportOriginMelissa 34.5°F Wind:2.2mph Pressure: 30.00hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544691479285762, "text": "why is my dog howling ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2153402000, "name": "☞LJ☜", "screen_name": "lacedlauren", "lang": "en", "location": "bitch with money and a car", "create_at": date("2013-10-24"), "description": "whipped on CT♡ • I've got bad news written all over me • lit like a couch •", "followers_count": 1059, "friends_count": 981, "statues_count": 19921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544691638824960, "text": "Laaaalllaaalllaaaalllaaa ☺️☺️☺️☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 496389951, "name": "Amanda Clemson", "screen_name": "AmandaClemson", "lang": "en", "location": "Upper East Side, Manhattan", "create_at": date("2012-02-18"), "description": "you got an aux cord?", "followers_count": 358, "friends_count": 131, "statues_count": 9207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544691861004288, "text": "the only parties I wanna go to for next two weeks are nap parties", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1058740741, "name": "jess ekman", "screen_name": "jayekman", "lang": "en", "location": "LA", "create_at": date("2013-01-03"), "description": "artist management + label stuff | university of florida grad | your favorite nanny | #napgirls | formerly @nesthq @famehouse @starkprofilespr", "followers_count": 1318, "friends_count": 666, "statues_count": 11368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544692821463040, "text": "Down for a nigga then prove it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2556351174, "name": "adry", "screen_name": "ahdryyy", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "A$ap Adry", "followers_count": 553, "friends_count": 566, "statues_count": 9675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544692938964992, "text": "I love seeing the people around me so happy. It makes me happier", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171113813, "name": "〰", "screen_name": "notoriouuuscc", "lang": "en", "location": "null", "create_at": date("2010-07-26"), "description": "getting my life together", "followers_count": 163, "friends_count": 116, "statues_count": 18667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcata, CA", "id": "2ca541fefabe94d6", "name": "Arcata", "place_type": "city", "bounding_box": rectangle("-124.110814,40.840387 -124.053849,40.909503") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 602476, "cityName": "Arcata" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544693215883264, "text": "Ripley SW Limestone Co. Temp: 31.8°F Wind:0.0mph Pressure: 999.0mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 47427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544693668876288, "text": "Wind 0.0 mph ---. Barometer 29.87 in, Falling slowly. Temperature 14.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 92, "statues_count": 156661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544693945597952, "text": "Want to work in #Washington, Washington D.C.? View our latest opening: https://t.co/WHyhl5FtCb #Sales #makeup #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0668837,38.9216743"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Washington", "Sales", "makeup", "Job", "Jobs", "Hiring" }}, "user": { "id": 2252314255, "name": "Macy's Beauty", "screen_name": "macysbeautyJOBS", "lang": "en", "location": "Nationwide", "create_at": date("2013-12-18"), "description": "Bring your passion for make up and skin care to Macy's - a beautiful way to invest your talents and build your career!", "followers_count": 695, "friends_count": 0, "statues_count": 8293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51013, "countyName": "Arlington", "cityID": 5103000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694050566144, "text": "It's gone work out check out our. New TWERK SONG https://t.co/vu8nM0u5TE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3286607125, "name": "〽ena❌e", "screen_name": "WimbushMalik", "lang": "en", "location": "LINK UP", "create_at": date("2015-07-21"), "description": "BandBoy | Maniakgang", "followers_count": 233, "friends_count": 395, "statues_count": 2185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goldsboro, NC", "id": "d392ccfac539947a", "name": "Goldsboro", "place_type": "city", "bounding_box": rectangle("-78.043083,35.340658 -77.877278,35.424386") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37191, "countyName": "Wayne", "cityID": 3726880, "cityName": "Goldsboro" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694222405633, "text": "One more year ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634041754, "name": "Karina ✨", "screen_name": "karinaaa_73", "lang": "en", "location": "null", "create_at": date("2012-07-12"), "description": "Tell me I can't and watch me work twice as hard to prove you wrong. Junior @ LHS ❤ #17 ⚽️", "followers_count": 962, "friends_count": 770, "statues_count": 28151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindsay, CA", "id": "1c7fc1369e419742", "name": "Lindsay", "place_type": "city", "bounding_box": rectangle("-119.120488,36.181549 -119.060335,36.237053") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 641712, "cityName": "Lindsay" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694637768704, "text": "Little Monster by Royal Blood is #nowplaying in Thumb Coast Brewing Co., Port Huron.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.422367,42.975489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox", "screen_name": "VirtualJukebox", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1352, "friends_count": 0, "statues_count": 1378419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Huron, MI", "id": "010750c237347ebb", "name": "Port Huron", "place_type": "city", "bounding_box": rectangle("-82.514554,42.935388 -82.418396,43.040084") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2665820, "cityName": "Port Huron" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694822273025, "text": "Wind 2.0 mph S. Barometer 29.916 in, Falling Rapidly. Temperature 11.7 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694939582464, "text": "@jazlinnmarie @laureeenn_26 @alysssamychelle @broookea https://t.co/QNP33quYv9", "in_reply_to_status": -1, "in_reply_to_user": 898630568, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 898630568, 862151190, 1417249152, 295044976 }}, "user": { "id": 499466811, "name": "xoxo_lexie", "screen_name": "alexissuarez18", "lang": "en", "location": "West Covina, CA", "create_at": date("2012-02-21"), "description": "null", "followers_count": 474, "friends_count": 465, "statues_count": 19854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brawley, CA", "id": "8da9aff1e34d7b52", "name": "Brawley", "place_type": "city", "bounding_box": rectangle("-115.568757,32.953097 -115.507293,33.004816") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 608058, "cityName": "Brawley" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544694956490752, "text": "I lowkey don't be having shit interesting to talk about", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290700155, "name": "Ty Lord ™", "screen_name": "_WithTylorJones", "lang": "en", "location": "Fear & Loathing in Thibodaux", "create_at": date("2011-04-30"), "description": "the way u say my name make me feel like I'm that nigga but I'm still unemployedddd RIP Pops 1thouwow.. GDC", "followers_count": 1081, "friends_count": 937, "statues_count": 26724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hattiesburg, MS", "id": "2b34df148a211c3e", "name": "Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.427669,31.241189 -89.248409,31.380085") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28035, "countyName": "Forrest", "cityID": 2831020, "cityName": "Hattiesburg" } }
+{ "create_at": datetime("2016-01-14T00:00:03.000Z"), "id": 687544695367413761, "text": "Am I the only one that starts brain storming when being faded & shittt ? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194889701, "name": "October 16th♎️", "screen_name": "Monitzabunnyy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-09-25"), "description": "Mikey⚓️ IG: @_monitzaberry", "followers_count": 189, "friends_count": 127, "statues_count": 3625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544695514251265, "text": "Viccy is Gordon Ramsey. The opposition is obvious. �������� https://t.co/V1RoDzfRgF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4220900353, "name": "Yungfoowiek", "screen_name": "fuimailangi", "lang": "en", "location": "NuttySide, California ", "create_at": date("2015-11-12"), "description": "Snapchat : yungfoowiek \nIG: yungfoowiek\nREAL friends are considered FAMILY and FAKE friends are just acquaintances. -YUNGFOOWIEK", "followers_count": 65, "friends_count": 177, "statues_count": 611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544695581503488, "text": "Someone take ya boy to eat I'll say I'm gonna buy you something and not do it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239315229, "name": "Stevie B", "screen_name": "mulheron4", "lang": "en", "location": "Canaryville, Chicago", "create_at": date("2011-01-17"), "description": "Former Outside Linebacker for the Fresno State Bulldogs|Artist/Director/Actor on #MTV #Vevo #Wshh #Canaryville #Chicago", "followers_count": 735, "friends_count": 333, "statues_count": 50765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544695635849217, "text": "You are never too good at something to not get better. Growth should stop at death.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3381878535, "name": "MikeFromThe3", "screen_name": "MichaelMaloian", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-07-18"), "description": "LA • Info: Michael@Christopher-Sebastian.com Hurricane Music Group: @NyzzyNyce", "followers_count": 2042, "friends_count": 902, "statues_count": 1537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544696063660032, "text": "#TampaLostFound lost female brown and cream dapple dachshund: Female brown and cream dapple dachshund.… https://t.co/gnGSutQmA1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.44833708,27.94888115"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TampaLostFound" }}, "user": { "id": 3032558408, "name": "Tampa Lost&Found", "screen_name": "Tampa_LAF", "lang": "en", "location": "Tampa, Florida", "create_at": date("2015-02-20"), "description": "null", "followers_count": 14, "friends_count": 64, "statues_count": 6767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544696101408768, "text": "Check out this album: https://t.co/SD76Cz4gBt #diverge baBy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "diverge" }}, "user": { "id": 1922238896, "name": "Dstars", "screen_name": "davidbelcher55", "lang": "en", "location": "pittsburg kansas ", "create_at": date("2013-09-30"), "description": "Working out party n sex n more sex", "followers_count": 209, "friends_count": 908, "statues_count": 243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, OK", "id": "135ecfb4c48e2fb7", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-94.896104,36.833306 -94.835172,36.920274") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40115, "countyName": "Ottawa", "cityID": 4048000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544696264986627, "text": "#CustomerService #Job in #Bentonville, AR: Front Office Assistant (Part-Time) at MedExpress https://t.co/ZCHxlguKfm #urgentcare #Veterans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.1986874,36.3353402"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CustomerService", "Job", "Bentonville", "urgentcare", "Veterans" }}, "user": { "id": 2820409627, "name": "MedExpress", "screen_name": "MedExpressJobs", "lang": "en", "location": "null", "create_at": date("2014-09-19"), "description": "Highly-trained professionals are a hallmark at MedExpress. The @MedExpress model attracts the best in the business - join our team today!", "followers_count": 181, "friends_count": 32, "statues_count": 310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bentonville, AR", "id": "0a0b9d7bcf6a59fa", "name": "Bentonville", "place_type": "city", "bounding_box": rectangle("-94.285935,36.307631 -94.151554,36.44036") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton", "cityID": 505320, "cityName": "Bentonville" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544696684457985, "text": "2 in the mornin my mind is on how I'm gonna wake up to go to school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1526974777, "name": "Drake", "screen_name": "03_laney", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-06-17"), "description": "null", "followers_count": 299, "friends_count": 175, "statues_count": 2687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544697947045888, "text": "is this criss angel https://t.co/wMKkvwSl1e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2177119447, "name": "afrikaansas", "screen_name": "ouch_my_abdoBEN", "lang": "en", "location": "where I'm meant to be...", "create_at": date("2013-11-05"), "description": "Sippin on that nut..", "followers_count": 236, "friends_count": 258, "statues_count": 13565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544697997283328, "text": "@burr86 @JoeWadlington you missed my performance :(", "in_reply_to_status": 687537231586697216, "in_reply_to_user": 5785502, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5785502, 29761436 }}, "user": { "id": 1035191, "name": "Steven Peterson", "screen_name": "squeakytoy", "lang": "en", "location": "East Village, Manhattan", "create_at": date("2007-03-12"), "description": "Solemn and shabby like a requiem in denim and leopardskin. iOS dev at @iCracked. @routesy is my baby. Previously @Detour, @Genius, turntable.fm, @IMVU.", "followers_count": 2625, "friends_count": 1038, "statues_count": 68250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belmont, CA", "id": "101f6a0f4879d8a6", "name": "Belmont", "place_type": "city", "bounding_box": rectangle("-122.32889,37.497748 -122.261568,37.534693") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 605108, "cityName": "Belmont" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544698198573056, "text": "Dried mangos, what's good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2338461001, "name": "King J", "screen_name": "JesseManny", "lang": "en", "location": "null", "create_at": date("2014-02-11"), "description": "SWAH SWAH // if I die, tell everyone I was a pretty bitch // music producer @Britt_Chalynn ♡", "followers_count": 698, "friends_count": 498, "statues_count": 13950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Provo, UT", "id": "665409336d2b20a0", "name": "Provo", "place_type": "city", "bounding_box": rectangle("-111.740949,40.18565 -111.602629,40.311641") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4962470, "cityName": "Provo" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544698437799936, "text": "I really want some mother friggin waffles right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83258259, "name": "Vi Clark✨", "screen_name": "Mondaeee", "lang": "en", "location": "Wherever The Wind Takes Me ", "create_at": date("2009-10-17"), "description": "/// make love - make tea /// #TeamCake #SavannahLives", "followers_count": 497, "friends_count": 442, "statues_count": 53387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smithfield, VA", "id": "7f55cb1b2db9bf49", "name": "Smithfield", "place_type": "city", "bounding_box": rectangle("-76.665123,36.949187 -76.581183,36.996641") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51093, "countyName": "Isle of Wight", "cityID": 5173200, "cityName": "Smithfield" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544698483806208, "text": "@foreveryoursst TRUUUE!!��", "in_reply_to_status": 687544571299901440, "in_reply_to_user": 519854200, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 519854200 }}, "user": { "id": 562524394, "name": "Jenny from the block", "screen_name": "jeennnyy_", "lang": "en", "location": "Clovis, CA", "create_at": date("2012-04-24"), "description": "the girl who cheated death 101", "followers_count": 717, "friends_count": 647, "statues_count": 43230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544698827755520, "text": "work is kind of lame right now come SHOOT me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145696969, "name": "johnny", "screen_name": "johnnyxochoa", "lang": "en", "location": "Long Beach, CA", "create_at": date("2015-04-08"), "description": "Ⓥ • xxx • coffee • chef", "followers_count": 337, "friends_count": 150, "statues_count": 5899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544699201150976, "text": "3 people $310 million each", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54596059, "name": "Luoai Saleh", "screen_name": "LuBizo", "lang": "en", "location": "Chicago", "create_at": date("2009-07-07"), "description": "Palestinian", "followers_count": 248, "friends_count": 229, "statues_count": 10767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-14T00:00:04.000Z"), "id": 687544699477868544, "text": "@adnys @rsg they don't want you t", "in_reply_to_status": 687542897995067392, "in_reply_to_user": 18300176, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18300176, 2948211 }}, "user": { "id": 23654193, "name": "Aaron Suplizio", "screen_name": "AaronSuplizio", "lang": "en", "location": "Newport Beach, CA", "create_at": date("2009-03-10"), "description": "Entrepreneurship. Startups. VC. Marketplace Lending. Apple Pay. Apple Watch. Products that #ReduceFriction in our lives. I played bass on a Weezer record once.", "followers_count": 757, "friends_count": 1838, "statues_count": 6074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport Beach, CA", "id": "0ce4c27ce26030aa", "name": "Newport Beach", "place_type": "city", "bounding_box": rectangle("-117.958219,33.563618 -117.789845,33.671658") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 651182, "cityName": "Newport Beach" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544699599458304, "text": "Ain't no body want you to win in this game honey. Goodnight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51547090, "name": "Luis Suastegui", "screen_name": "luistacks", "lang": "en", "location": "H O L L Y W O O D ", "create_at": date("2009-06-27"), "description": "My Levels keep getting higher and higher. Gem-en-i.e.) (♊)", "followers_count": 89, "friends_count": 131, "statues_count": 3200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544699775660032, "text": "@MichaelExile nah man you're cool af ��", "in_reply_to_status": 687542278508773377, "in_reply_to_user": 63945825, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63945825 }}, "user": { "id": 3042011322, "name": "Ian", "screen_name": "IanMrClutch", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-02-25"), "description": "keep it real", "followers_count": 119, "friends_count": 151, "statues_count": 2777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544699914162176, "text": "Fucking pre always keeping me up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 557332196, "name": "Nick Vasvilla", "screen_name": "villadakilla", "lang": "en", "location": "Levittown, PA", "create_at": date("2012-04-18"), "description": "We are all capable of more than we're programmed to think! FNC Athlete & Sales Executive/GM at Flex Nutrition Center. #ChasingGreatness", "followers_count": 540, "friends_count": 298, "statues_count": 14991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tullytown, PA", "id": "867ccb9c2898ece7", "name": "Tullytown", "place_type": "city", "bounding_box": rectangle("-74.828648,40.126541 -74.79521,40.156366") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42017, "countyName": "Bucks", "cityID": 4277744, "cityName": "Tullytown" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544699939196930, "text": "when you creepin on ooyf and you find some shit you didnt wanna see https://t.co/qapnep8yRt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19305358, "name": "Vanilla™", "screen_name": "waffle_dragon1", "lang": "en", "location": "Algona, IA ➡ Sioux City, IA", "create_at": date("2009-01-21"), "description": "And the most important piece of clothing you must wear is love. Love is what binds us all together in perfect harmony. Collosians 3:14", "followers_count": 617, "friends_count": 556, "statues_count": 25068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux City, IA", "id": "c7397adf5a17673a", "name": "Sioux City", "place_type": "city", "bounding_box": rectangle("-96.492971,42.385735 -96.301329,42.56216") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19193, "countyName": "Woodbury", "cityID": 1973335, "cityName": "Sioux City" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544700694167552, "text": "@ColbytheIsh @heavenfest @TDWPband dude yesssss! Ikr", "in_reply_to_status": 687330987076788224, "in_reply_to_user": 1380551845, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1380551845, 20298328, 34450688 }}, "user": { "id": 2743233629, "name": "Phocust", "screen_name": "iamphocust", "lang": "en", "location": "Denver, CO", "create_at": date("2014-08-14"), "description": "DJ/Producer, EDM Energizer, Jesus Freak. #stayphocust", "followers_count": 290, "friends_count": 542, "statues_count": 3100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CO", "id": "f7eb2fa2fea288b1", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-105.193475,39.60973 -105.053164,39.761974") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 843000, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544700719464448, "text": "I like crazy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389261737, "name": "Young Pablito", "screen_name": "Alexf4__", "lang": "en", "location": "The Jungle ", "create_at": date("2011-10-11"), "description": "The 6 is watching", "followers_count": 8184, "friends_count": 1841, "statues_count": 21848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544700874567680, "text": "I really don’t give a fuck anymore https://t.co/zO7W2y5qF6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1661448685, "name": "Devinne", "screen_name": "devinnealubong", "lang": "en", "location": "null", "create_at": date("2013-08-10"), "description": "null", "followers_count": 886, "friends_count": 479, "statues_count": 15770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544701419847681, "text": "@troyesivan @Spotify ✋��", "in_reply_to_status": 687439042753921024, "in_reply_to_user": 53338746, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 53338746, 17230018 }}, "user": { "id": 2238476845, "name": "Carissa Elliott", "screen_name": "misscarissaree", "lang": "en", "location": "null", "create_at": date("2013-12-09"), "description": "null", "followers_count": 372, "friends_count": 436, "statues_count": 4766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Heights, CA", "id": "a89175c4c91f45a3", "name": "Citrus Heights", "place_type": "city", "bounding_box": rectangle("-121.331355,38.663673 -121.242804,38.722779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 613588, "cityName": "Citrus Heights" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544701503721472, "text": "Ill admit i have been doing some pretty reckless things. I have made the wise of choices and maybe i did lose my way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716727794, "name": "Makaveli Crayton", "screen_name": "SayCrayton3X", "lang": "en", "location": "Dallas, TX ✈️ HTX", "create_at": date("2012-07-25"), "description": "Watch how i run up them bands | Counting Blessings | #PlugBrothaz", "followers_count": 2069, "friends_count": 1394, "statues_count": 25528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544701638078464, "text": "Booty�� https://t.co/3OcLuPuyeX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156486347, "name": "tori", "screen_name": "vickytoriaah", "lang": "en", "location": "Oakcliff, Tx", "create_at": date("2010-06-16"), "description": "null", "followers_count": 3816, "friends_count": 1881, "statues_count": 77367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544702028103684, "text": "Got lost in the system, tryna get back to normal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244919507, "name": "Tevis Robinson Jr.", "screen_name": "TevisRobinson", "lang": "en", "location": "North Miami, FL", "create_at": date("2011-01-30"), "description": "JWU Basketball #YpsiMade #FreeAnt", "followers_count": 808, "friends_count": 565, "statues_count": 48179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Miami, FL", "id": "cae987165260f81f", "name": "North Miami", "place_type": "city", "bounding_box": rectangle("-80.227398,25.880982 -80.127938,25.930229") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1249450, "cityName": "North Miami" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544702770528256, "text": "I need fr fr �� https://t.co/CZCMSnb4IA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4072982115, "name": "CoCo ✨", "screen_name": "___Shenelle", "lang": "en", "location": "null", "create_at": date("2015-10-29"), "description": "NEW TWITTER !", "followers_count": 544, "friends_count": 616, "statues_count": 5131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stamford, CT", "id": "468bfe7c705fbc37", "name": "Stamford", "place_type": "city", "bounding_box": rectangle("-73.606436,41.016898 -73.501758,41.179771") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 973000, "cityName": "Stamford" } }
+{ "create_at": datetime("2016-01-14T00:00:05.000Z"), "id": 687544703479226368, "text": "@iambloodthirsty Okay, well, I've clearly made a grave error.", "in_reply_to_status": 687542987656658944, "in_reply_to_user": 3039865441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3039865441 }}, "user": { "id": 24703754, "name": "Zoe Kinsky", "screen_name": "NamiRocket", "lang": "en", "location": "Houston, Texas, USA", "create_at": date("2009-03-16"), "description": "Welcome to my dumb opinion pit. I like to ramble a lot. This seems as good a place as any to consolidate a lot of it. Profile art by @Scott_DeWitt.", "followers_count": 686, "friends_count": 865, "statues_count": 15743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-14T00:00:06.000Z"), "id": 687544704179634176, "text": "@CollinGreenwood @JonSooHooPics @USC_Athletics @USC_Hoops ��", "in_reply_to_status": 687543717947781121, "in_reply_to_user": 590531007, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 590531007, 3347265100, 168681472, 872375695 }}, "user": { "id": 1010562709, "name": "Jose Serna", "screen_name": "jose_serna23", "lang": "en", "location": "null", "create_at": date("2012-12-13"), "description": "BOHS '16", "followers_count": 443, "friends_count": 358, "statues_count": 2207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brea, CA", "id": "341aa0cdb944022e", "name": "Brea", "place_type": "city", "bounding_box": rectangle("-117.92871,33.905579 -117.800848,33.947725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608100, "cityName": "Brea" } }
+{ "create_at": datetime("2016-01-14T00:00:06.000Z"), "id": 687544704615976960, "text": "#VerbalDisturbance at 8691-8730 S Orange Ave. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.366301,28.443738"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "orlpol", "ocso" }}, "user": { "id": 39134149, "name": "Police Calls 32824", "screen_name": "orlpol32824", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 88, "friends_count": 1, "statues_count": 18534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1205300, "cityName": "Belle Isle" } }
+{ "create_at": datetime("2016-01-14T00:00:06.000Z"), "id": 687544705001717761, "text": "@H_Ek3 Yeah! Some states allow you to remain anonymous. That’d be my choice. Too much pressure.", "in_reply_to_status": 687544193024049152, "in_reply_to_user": 2251136629, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2251136629 }}, "user": { "id": 159660869, "name": "Michael Jackson", "screen_name": "MJack94", "lang": "en", "location": "United States", "create_at": date("2010-06-25"), "description": "No, I don't do the moonwalk. I don't do any kind of walk.", "followers_count": 718, "friends_count": 710, "statues_count": 11370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, MN", "id": "62619a76134ad05e", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-92.282745,46.650616 -91.962935,46.866053") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2717000, "cityName": "Duluth" } }
+{ "create_at": datetime("2016-01-14T00:00:06.000Z"), "id": 687544705517662208, "text": "omg part 2 https://t.co/oVXqxkDrYW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1618216111, "name": "marissa", "screen_name": "ogpiss", "lang": "en", "location": "Houston, TX", "create_at": date("2013-07-24"), "description": "sfa19 | ΑΧΩ | free gucci", "followers_count": 1261, "friends_count": 943, "statues_count": 47541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907067114450944, "text": "jesus died at 33\npoor thing..\ni can dig it.. \nthey poisoned me and antione when i was 33\nsome real bullshit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 564739170, "name": "RUBY MU YKIHIRO", "screen_name": "PHARAOH_YKHIRO", "lang": "en", "location": "Amexem", "create_at": date("2012-04-27"), "description": "Hierarchical code:R.101.052.004 Asian463 Moor667 Dept of Justice BM:$HR:IND:144-35-0 \nSHANNON NICOLE JONES", "followers_count": 965, "friends_count": 1722, "statues_count": 38606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907067315671040, "text": "it is a struggle to be a peace keeper when you a crazy bitch too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2479204321, "name": "K.", "screen_name": "htxkel", "lang": "en", "location": "null", "create_at": date("2014-05-05"), "description": "null", "followers_count": 7784, "friends_count": 536, "statues_count": 24207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907068032860160, "text": "Materialism rules a lot of people.. It's honestly really sad. Especially when the best things in life are right in your face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1203318198, "name": "The General", "screen_name": "LebaneseThor", "lang": "en", "location": "California to Japan", "create_at": date("2013-02-20"), "description": "Lookin for a woman who wont fuck other men while im on deployment. #USN #BM3 #GreenEyedGoblin", "followers_count": 1323, "friends_count": 648, "statues_count": 97354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907068163002368, "text": "Otf we promote violence", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83975848, "name": "SHAUNWORLD", "screen_name": "BigDaddyDria", "lang": "en", "location": "CHICAGO ALL I KNOW❤️", "create_at": date("2009-10-20"), "description": "just play yo role", "followers_count": 2061, "friends_count": 427, "statues_count": 132145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907068305473538, "text": "@CJ_Supreme a chain of hotels", "in_reply_to_status": 687906993844039680, "in_reply_to_user": 57275127, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57275127 }}, "user": { "id": 39786669, "name": "1st Brokage", "screen_name": "_COOLRon", "lang": "en", "location": "Village Hidden in the Trap", "create_at": date("2009-05-13"), "description": "*scratches head uhmm...... #jetlife The Bay t", "followers_count": 753, "friends_count": 849, "statues_count": 60280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907068615892992, "text": "Gods Language Should Mercifully B Spoken��Example,All thts weary,Can I please share a wrd with U,Not,The opposite,If You fail U lose,ItsWrong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.28451893,34.16927767"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 992148319, "name": "Tonja Faulkner", "screen_name": "TonjaFaukner", "lang": "en", "location": "null", "create_at": date("2012-12-05"), "description": "null", "followers_count": 347, "friends_count": 115, "statues_count": 20634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907068804751360, "text": "still wide awake @ 3am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2892583647, "name": "❁", "screen_name": "kerrettt", "lang": "en", "location": "704 // 828", "create_at": date("2014-11-25"), "description": "I am still alive and for that I'm thankful // #UNCA19", "followers_count": 207, "friends_count": 172, "statues_count": 3200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asheville, NC", "id": "00836cb90515ddd1", "name": "Asheville", "place_type": "city", "bounding_box": rectangle("-82.681852,35.421216 -82.452163,35.656929") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37021, "countyName": "Buncombe", "cityID": 3702140, "cityName": "Asheville" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907069408710657, "text": "I admit it's one of my biggest flaws", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165229335, "name": "Janice Margaret", "screen_name": "jwaffless", "lang": "en", "location": "nowhere", "create_at": date("2010-07-10"), "description": "My tweets are like dry humor. They'll get you laughing eventually. NYU", "followers_count": 259, "friends_count": 301, "statues_count": 14242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907069417009152, "text": "Want to work at MedExpress? We're #hiring in #Charleston, WV! Click for details: https://t.co/JoRzeP61QB #Nursing #urgentcare #Veterans #Job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7300378,38.3549317"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Charleston", "Nursing", "urgentcare", "Veterans", "Job" }}, "user": { "id": 2820409627, "name": "MedExpress", "screen_name": "MedExpressJobs", "lang": "en", "location": "null", "create_at": date("2014-09-19"), "description": "Highly-trained professionals are a hallmark at MedExpress. The @MedExpress model attracts the best in the business - join our team today!", "followers_count": 180, "friends_count": 32, "statues_count": 394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Charleston, WV", "id": "b8018050c8b66e62", "name": "South Charleston", "place_type": "city", "bounding_box": rectangle("-81.753526,38.305259 -81.668541,38.37459") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54039, "countyName": "Kanawha", "cityID": 5475292, "cityName": "South Charleston" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907069580587008, "text": "Visiting friends onset of the #hilarious hit #comedy @CarmichaelShow @nbc Season 2 is bringing the #funny https://t.co/L11omG4JAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hilarious", "comedy", "funny" }}, "user_mentions": {{ 2484564642, 26585095 }}, "user": { "id": 1576739257, "name": "Theo Marshall", "screen_name": "THEOMARSHALLcom", "lang": "en", "location": "Santa Monica, California ", "create_at": date("2013-07-07"), "description": "Father/Husband/Peace Corps, Africa/Actor (Sag-Aftra)/Writer L.A.-Chicago http://www.imdb.me/theomarshall", "followers_count": 907, "friends_count": 2195, "statues_count": 3810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907069643612160, "text": "Late night Shipleys run", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2715356374, "name": "X.", "screen_name": "Squadshxtt", "lang": "en", "location": "Alief, Tx", "create_at": date("2014-07-16"), "description": "From the S.W.A.T to 3rd ward not too far from home.... #TxSU", "followers_count": 299, "friends_count": 222, "statues_count": 10304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-15T00:00:00.000Z"), "id": 687907070490836992, "text": "03:00:01 |Temp: 39.9ºF | Wind Chill 39.9ºF |Dew Point 35.8ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SSE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907070851481600, "text": "Narcos* https://t.co/hexTZisdlo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 195161499, "name": "Yung Jefe!", "screen_name": "zackb54_", "lang": "en", "location": "San Antonio/San Angelo, TX ", "create_at": date("2010-09-25"), "description": "Student-Athlete at Angelo State University. Its hard to be a diamond in a rhinestone world...", "followers_count": 600, "friends_count": 564, "statues_count": 12445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907071161974784, "text": "Wind 0.0 mph WNW. Barometer 29.673 in, Falling Rapidly. Temperature 55.4 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907071388286976, "text": "I don't feel like sleeping", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164853110, "name": "$av★", "screen_name": "SavannahWaterss", "lang": "en", "location": "Minnesota", "create_at": date("2010-07-09"), "description": "Stubborn af with a good heart", "followers_count": 165, "friends_count": 74, "statues_count": 5871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jordan, MN", "id": "89a89991c0fadecb", "name": "Jordan", "place_type": "city", "bounding_box": rectangle("-93.654787,44.644991 -93.613285,44.683045") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27139, "countyName": "Scott", "cityID": 2732174, "cityName": "Jordan" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907071463952384, "text": "Wind 0 mph --. Barometer 29.84 in, Falling. Temperature 46.9 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907071472214017, "text": "Black boy w/ ambition please ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37516497, "name": "alicia✨", "screen_name": "ohaliciaaa", "lang": "en", "location": "CSULGAY", "create_at": date("2009-05-03"), "description": "more money, more niggas, & more k.oubre❤️ ovotillertsnmi", "followers_count": 1371, "friends_count": 973, "statues_count": 25023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072072024065, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/DlWgGrrgoS #BRUNSWICK, GA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5065077,31.2480741"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "BRUNSWICK", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 506, "friends_count": 251, "statues_count": 16532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brunswick, GA", "id": "00cd1ca713cf470d", "name": "Brunswick", "place_type": "city", "bounding_box": rectangle("-81.570027,31.122864 -81.439789,31.306113") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13127, "countyName": "Glynn", "cityID": 1311560, "cityName": "Brunswick" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072214614016, "text": "You know what time it is. Week 3 brings with it @ViigoTweets and a request to turn up your subwoofers here https://t.co/YJXdFscWmU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3145064169 }}, "user": { "id": 115081746, "name": "10K ISLANDS", "screen_name": "10KIslands", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-02-17"), "description": "Record label and studio. Every Friday a brand new song. (tweets from @eve_barlow and @al_horner)", "followers_count": 570, "friends_count": 792, "statues_count": 1505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072361418754, "text": "@DCunningham333 Davy out here Marshawn Lynchin dudes #holdmahdick", "in_reply_to_status": 687865961236451328, "in_reply_to_user": 918369530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "holdmahdick" }}, "user_mentions": {{ 918369530 }}, "user": { "id": 890195636, "name": "Jake Justice", "screen_name": "jakejustice69", "lang": "en", "location": "null", "create_at": date("2012-10-18"), "description": "Cold World", "followers_count": 431, "friends_count": 254, "statues_count": 2220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Oaks, CA", "id": "54043c2f7d22a3b8", "name": "Fair Oaks", "place_type": "city", "bounding_box": rectangle("-121.292342,38.62045 -121.193945,38.678687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 623294, "cityName": "Fair Oaks" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072474648576, "text": "Wind 4.0 mph WSW. Barometer 1003.83 mb, Falling Rapidly. Temperature 52.5 °F. Rain today 0.01 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072554479616, "text": "3:00.✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 94930348, "name": "Keena", "screen_name": "ohhkeena_", "lang": "en", "location": "null", "create_at": date("2009-12-05"), "description": "| 336 | Army | NCAT", "followers_count": 901, "friends_count": 990, "statues_count": 9886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907072613072896, "text": "Nurse Practitioner - Ministry Health Care: (#Appleton, WI) https://t.co/LDrdfHBX71 #Nursing #NP #PA #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.4153847,44.2619309"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Appleton", "Nursing", "NP", "PA", "Job", "Jobs", "Hiring" }}, "user": { "id": 4137429793, "name": "Ministry Health Jobs", "screen_name": "JobsAtMinistry", "lang": "en", "location": "Central and Northern Wisconsin", "create_at": date("2015-11-05"), "description": "Follow this page for open healthcare positions at @Ministry_Health We are #hiring in #Wisconsin! #jobs #job", "followers_count": 41, "friends_count": 46, "statues_count": 420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Appleton, WI", "id": "20b60eedd781650d", "name": "Appleton", "place_type": "city", "bounding_box": rectangle("-88.451199,44.221575 -88.335679,44.33491") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55087, "countyName": "Outagamie", "cityID": 5502375, "cityName": "Appleton" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073254752257, "text": "This #Hospitality #job might be a great fit for you: General Manager - https://t.co/HaXacNv7Co #Lufkin, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.756802,31.3327959"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "Lufkin", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 200, "friends_count": 39, "statues_count": 19129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lufkin, TX", "id": "ef5f7bc53f4594ea", "name": "Lufkin", "place_type": "city", "bounding_box": rectangle("-94.796056,31.264964 -94.669141,31.400021") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48005, "countyName": "Angelina", "cityID": 4845072, "cityName": "Lufkin" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073271705600, "text": "Wind 4.0 mph SSE. Barometer 29.546 in, Falling. Temperature 50.1 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073363935233, "text": "2nd night with no sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64778162, "name": "Feesh", "screen_name": "feesh45", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "null", "followers_count": 357, "friends_count": 636, "statues_count": 14166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trumbull, CT", "id": "04cd3cb46669a6c5", "name": "Trumbull", "place_type": "city", "bounding_box": rectangle("-73.275103,41.220538 -73.140573,41.299356") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 977270, "cityName": "Trumbull" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073485479936, "text": "Es que ya no sé que haría sin ellos��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 549438095, "name": "lusssshi", "screen_name": "luciamarsanchez", "lang": "es", "location": "✈️ Madrid - Utah ✈️", "create_at": date("2012-04-09"), "description": "clarinetista, pianista y fruti Soberana de Villa Fenil", "followers_count": 504, "friends_count": 504, "statues_count": 24976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindon, UT", "id": "5e1dbff4811038b0", "name": "Lindon", "place_type": "city", "bounding_box": rectangle("-111.763526,40.326209 -111.662197,40.352543") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4945090, "cityName": "Lindon" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073544163329, "text": "@ughleiest I FUCKING GOT THESE AND THEY COME PN SATURDAY WOWOWOW THINGS HAVE CHANGED IN MY LIFE", "in_reply_to_status": 664708924667355136, "in_reply_to_user": 954897516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 954897516 }}, "user": { "id": 954897516, "name": "tams", "screen_name": "ughleiest", "lang": "en", "location": "in y(our) man's bed", "create_at": date("2012-11-17"), "description": "I could potentially go psycho on your ass; you better watch yourself.", "followers_count": 351, "friends_count": 450, "statues_count": 26135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianola, WA", "id": "204cf8d953aff300", "name": "Indianola", "place_type": "city", "bounding_box": rectangle("-122.555441,47.742554 -122.470205,47.780466") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5333280, "cityName": "Indianola" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907073984741376, "text": "I must be dumb ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4691504202, "name": "Steve Curry", "screen_name": "toocashtaylor", "lang": "en", "location": "your mom's house", "create_at": date("2016-01-01"), "description": "Big Macs an them straps #BINK", "followers_count": 207, "friends_count": 407, "statues_count": 82 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Navarre, FL", "id": "01c0441229f0e67f", "name": "Navarre", "place_type": "city", "bounding_box": rectangle("-86.953261,30.394546 -86.799166,30.454239") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12113, "countyName": "Santa Rosa", "cityID": 1248050, "cityName": "Navarre" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907074248978433, "text": "Eat the booty https://t.co/xhNjkpevFb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648826238, "name": "☦KIDCAJUN☦", "screen_name": "COONASSKIDCAJUN", "lang": "en", "location": "NEW ORLEANS", "create_at": date("2013-08-05"), "description": "PRODUCER ( YUNG SIMMIE, BEN DOUGH, LITTLE PAIN, PEPPER BOY ) #BANDWIDTHBOYS #DiorSentai DAISHI IS THE TRIBE OF GREAT AMBITION", "followers_count": 11546, "friends_count": 2629, "statues_count": 49834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McComb, MS", "id": "8d3227c6a2a3645b", "name": "McComb", "place_type": "city", "bounding_box": rectangle("-90.494167,31.217667 -90.436447,31.269798") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28113, "countyName": "Pike", "cityID": 2843280, "cityName": "McComb" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907074500476928, "text": "Just think, 27 ppl can be killed at an elem school; a president can implore we help, and grown men can say NO, profits are more important ����", "in_reply_to_status": 687905892411428865, "in_reply_to_user": 14707605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14707605, "name": "Matt Davis", "screen_name": "mattdaviss", "lang": "en", "location": "Anchorage/Chattanooga", "create_at": date("2008-05-08"), "description": "I research foreign affairs. I jump out of airplanes. I live in Alaska. Normal stuff. [snapchat: rcktshp] [tumblr: latrotron]", "followers_count": 1229, "friends_count": 711, "statues_count": 20735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-15T00:00:01.000Z"), "id": 687907074756493312, "text": "���� https://t.co/6gPgVkXEEd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 720136919, "name": "Mini_Kim_Kardashian", "screen_name": "Pink_buggattii", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-17"), "description": "Big ❤️ Forever smiling .... The nicest person you'll ever meet Nomatter what happens in life being good to people is a wonderful legacy to leave behind", "followers_count": 3297, "friends_count": 644, "statues_count": 81832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907075154907136, "text": "Wind 2.0 mph SW. Barometer 29.770 in, Falling slowly. Temperature 45.9 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907075179941888, "text": "48. What you doing with the money if you won the power ball ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486326551, "name": "cinco", "screen_name": "SwaggyCinco", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2012-02-07"), "description": "#LongLiveDeuce living w/ no limits", "followers_count": 1934, "friends_count": 1643, "statues_count": 63610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907075230306306, "text": "@pkeynen i mean make love to you", "in_reply_to_status": 687906923656564736, "in_reply_to_user": 2359875294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2359875294 }}, "user": { "id": 530825237, "name": "Unlovable", "screen_name": "Ignorant_M1ndz", "lang": "en", "location": "Hemlock ", "create_at": date("2012-03-19"), "description": "4th grader R.I.P.21 GolfBoys", "followers_count": 909, "friends_count": 862, "statues_count": 63637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907075339513856, "text": "it's 3 and i'm still up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278782985, "name": "⚢❤️", "screen_name": "pookaaaaaa", "lang": "en", "location": "Fort Drum, NY", "create_at": date("2011-04-07"), "description": "US Army, @OHHH_MADI ❤️", "followers_count": 11137, "friends_count": 5451, "statues_count": 63629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Drum, NY", "id": "69108214c4615b4d", "name": "Fort Drum", "place_type": "city", "bounding_box": rectangle("-75.838074,44.010422 -75.692237,44.073511") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36045, "countyName": "Jefferson", "cityID": 3626759, "cityName": "Fort Drum" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907075385626624, "text": "My girl @theemilymac sure does make me smile. #badgirlsclub #9months #martythomaspresentsdiva… https://t.co/jU8lYDiXOD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9868622,40.7644997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "badgirlsclub", "9months", "martythomaspresentsdiva" }}, "user_mentions": {{ 34189987 }}, "user": { "id": 121600073, "name": "Marty Thomas", "screen_name": "martythomasbway", "lang": "en", "location": "New York City", "create_at": date("2010-03-09"), "description": "Broadway film and television actor, Grammy nom. recording artist, Hair dresser, stylist, Host/Producer of Marty Thomas Presents DIVA, Music on Itunes!", "followers_count": 5665, "friends_count": 1832, "statues_count": 7309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907076232749056, "text": "I just picked up lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3083870792, "name": "BadLuckCee", "screen_name": "ceesirr", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Bloomington Ca", "followers_count": 295, "friends_count": 258, "statues_count": 13480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907076316643329, "text": "I was confused as to why that happened and then I was like oh yeah you can donate your organs when you die", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407455189, "name": "Hitzel", "screen_name": "HitzelMonroe", "lang": "en", "location": "null", "create_at": date("2011-11-07"), "description": "Dancers are the athletes of God", "followers_count": 565, "friends_count": 545, "statues_count": 32961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907076522115072, "text": "@okayjaybee so", "in_reply_to_status": 687906480285192192, "in_reply_to_user": 703032761, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 703032761 }}, "user": { "id": 2279864982, "name": "QuickMoney#dregang01", "screen_name": "BankrollQuincy", "lang": "en", "location": "null", "create_at": date("2014-01-06"), "description": "null", "followers_count": 2493, "friends_count": 2420, "statues_count": 28967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907076907995136, "text": "See our latest #NewYork, New York #job and click to apply: Senior C# Developer - https://t.co/bA8imJc7tz #IT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0059731,40.7143528"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewYork", "job", "IT", "Hiring" }}, "user": { "id": 3836985865, "name": "BOC Staffing", "screen_name": "BOCstaffing", "lang": "en", "location": "NYC & Jersey City area", "create_at": date("2015-10-09"), "description": "BOC is a niche specialty provider, serving the Tri-State area's Buy-side/Sell-side Financial Services community’s #staffing requirements.#finance #job #NYC #NJ", "followers_count": 24, "friends_count": 23, "statues_count": 24 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907077478559744, "text": "Wind 0.0 mph ---. Barometer 29.987 in, Steady. Temperature 10.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 7938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907077658808320, "text": "shoutout to alejandra https://t.co/xarrYm8fNH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 464884911, "name": "Alejandra Guzman", "screen_name": "alevictoria13", "lang": "en", "location": "Anaheim", "create_at": date("2012-01-15"), "description": "19 - CSUF - Rest in Paradise Adrian Guzman", "followers_count": 346, "friends_count": 311, "statues_count": 19177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907077818208258, "text": "Customer Service Representative-Retail Sales - Speedway: (#Lowell, MI) https://t.co/hHn2a8GcUa #CustomerService #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.3419609,42.9336426"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Lowell", "CustomerService", "Job", "Jobs", "Hiring" }}, "user": { "id": 3144822634, "name": "Speedway Jobs", "screen_name": "SpeedwayJobs", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "Rethink Speedway. It's more than a convenience store... It's your career opportunity!", "followers_count": 151, "friends_count": 7, "statues_count": 6410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lowell, MI", "id": "e689389d480cf958", "name": "Lowell", "place_type": "city", "bounding_box": rectangle("-85.410842,42.923546 -85.321605,42.965806") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2649540, "cityName": "Lowell" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078036402180, "text": "01/15@03:00 - Temp 28.5F, WC 28.5F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.964in, Steady. Rain 0.00in. Hum 88%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078057410560, "text": "I been staring at a window for an hour, that's what I do at work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 186721604, "name": "JL", "screen_name": "Ohits_JL", "lang": "en", "location": "#FNG", "create_at": date("2010-09-03"), "description": "null", "followers_count": 329, "friends_count": 303, "statues_count": 24494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, NJ", "id": "a40c7aae22398af1", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-74.17127,40.934507 -74.141067,40.983453") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3430570, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078090977280, "text": "@ButterBaby_ you go to class? Lol", "in_reply_to_status": 687906950256836609, "in_reply_to_user": 465218597, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 465218597 }}, "user": { "id": 58519413, "name": "Malik Boynton", "screen_name": "likdawg16", "lang": "en", "location": "null", "create_at": date("2009-07-20"), "description": "APSU DB | 3x All-State & 1x All-American Athlete | Grind in the dark, so you can shine in the light 1/100 | RIP Mama | Free My Brother | Child Of God", "followers_count": 1466, "friends_count": 619, "statues_count": 34510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078405423104, "text": "#SkyPolice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SkyPolice" }}, "user": { "id": 1181722220, "name": "Vìva La Pizza ! ✊", "screen_name": "MaestroChubbs", "lang": "en", "location": "East Los Angeles, CA", "create_at": date("2013-02-15"), "description": "I saw you Koming with that bullshit from a mile away.", "followers_count": 803, "friends_count": 490, "statues_count": 70310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078657191936, "text": "*mom is Asian, has no body hair*\nme: mom feel my legs I just shaved!!\nmom: ... why would I do that I can just feel my own legs \n#savagemom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "savagemom" }}, "user": { "id": 1877481746, "name": "Aya Newman", "screen_name": "ayanewman", "lang": "en", "location": "New Jersey, USA", "create_at": date("2013-09-17"), "description": "Berkeley CA ✨ Drew University c/o '19 ✨ your friendly neighborhood sondheim enthusiast", "followers_count": 132, "friends_count": 570, "statues_count": 1292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078816436224, "text": "If I had one of every item at the PINK store, I'd be set for life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169645538, "name": "isa", "screen_name": "caliclarisa", "lang": "en", "location": "null", "create_at": date("2010-07-22"), "description": "Apá ❤️ XI.XV.XXXIII", "followers_count": 218, "friends_count": 187, "statues_count": 15357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078854279168, "text": "Wind 0.0 mph E. Barometer 29.935 in, Steady. Temperature 26.0 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-15T00:00:02.000Z"), "id": 687907078913015808, "text": "I play too much I don't think people take me serious��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587698239, "name": "TastyTej", "screen_name": "Teeeeja_Vu", "lang": "en", "location": "dreamville ", "create_at": date("2012-05-22"), "description": "CA✈LA #LSU19 I can tell my life with you would be delicious.", "followers_count": 1399, "friends_count": 692, "statues_count": 104510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079076593664, "text": "@HugoNasck pensei em fazer isso msm djsndkd ela ta dormindo agr se n ja ia jogar esse verde", "in_reply_to_status": 687906549914820609, "in_reply_to_user": 112262977, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 112262977 }}, "user": { "id": 162906156, "name": "kill v. maim", "screen_name": "skarsgardyh", "lang": "en", "location": "New York, NY", "create_at": date("2010-07-04"), "description": "kinda goth emo vampire", "followers_count": 573, "friends_count": 177, "statues_count": 108665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079277953024, "text": "Temp: 48.5°F Wind:4.7mph Pressure: 29.746hpa Falling Rapidly Rain Today 0.04in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079634468864, "text": "Wind 0.0 mph ---. Barometer 29.95 in, Steady. Temperature 27.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 94, "statues_count": 156695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079718334465, "text": "#SupportOriginMelissa 46.0°F Wind:2.5mph Pressure: 29.76hpa Falling Rain Today 0.00in. Forecast: Occasional precipitation, worsening", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079739183106, "text": "https://t.co/C7XdfPG4Ji", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2887092768, "name": "local boy", "screen_name": "ginakkendall", "lang": "en", "location": "aliso viejo, ca", "create_at": date("2014-11-02"), "description": "i got a whooty. barista, Ⓥ", "followers_count": 321, "friends_count": 317, "statues_count": 5539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aliso Viejo, CA", "id": "f9838a67f87a80ed", "name": "Aliso Viejo", "place_type": "city", "bounding_box": rectangle("-117.75971,33.512428 -117.700301,33.609056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 600947, "cityName": "Aliso Viejo" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079810494464, "text": "@_LvRay man https://t.co/P7cOnbx373", "in_reply_to_status": -1, "in_reply_to_user": 222767342, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 222767342 }}, "user": { "id": 284976937, "name": "MyNameIsJeff", "screen_name": "916jeff", "lang": "en", "location": "702", "create_at": date("2011-04-20"), "description": "IG:jeffbaxter_ | Snapchat jeff_baxter916", "followers_count": 357, "friends_count": 360, "statues_count": 43176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079886012416, "text": "51.1F (Feels: 51.1F) - Humidity: 99% - Wind: 1.6mph SE - Gust: 3.1mph - Pressure: 1019.0mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079932129280, "text": "@k_torres4 picky ��", "in_reply_to_status": 687906879264063488, "in_reply_to_user": 1033274821, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1033274821 }}, "user": { "id": 2460989820, "name": "Gabe", "screen_name": "TheFirstGabriel", "lang": "en", "location": "DDD, Texas", "create_at": date("2014-04-24"), "description": "The University of North Texas #UNT17 21 years of age. IG: thefirstgabriel", "followers_count": 570, "friends_count": 548, "statues_count": 15181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907079982546944, "text": "https://t.co/jLuK9tZuTZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 17226, "friends_count": 4530, "statues_count": 2013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907080389263363, "text": "@eliuuuhrojas dejate venir", "in_reply_to_status": 687906979881193472, "in_reply_to_user": 3079850896, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3079850896 }}, "user": { "id": 850139606, "name": "DonWhat", "screen_name": "luisss_donttrip", "lang": "en", "location": "SD", "create_at": date("2012-09-27"), "description": "null", "followers_count": 148, "friends_count": 162, "statues_count": 10822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907080716419073, "text": "oooo when that direct deposit hit����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180218455, "name": "G✨", "screen_name": "Gee_killah", "lang": "en", "location": "Bay Area, CA", "create_at": date("2010-08-18"), "description": "Welcome to the good life", "followers_count": 712, "friends_count": 317, "statues_count": 46230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907080796303360, "text": "Mejor sensación del mundo: tener la razón en una discusión.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 17226, "friends_count": 4530, "statues_count": 2013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907080867549184, "text": "�� https://t.co/udK5ectAGK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2371396330, "name": "Walkersville Theatre", "screen_name": "WHSLionsTheatre", "lang": "en", "location": "WHS Stage", "create_at": date("2014-02-28"), "description": "The official twitter for the Walkersville Theatre Department and ITS Troupe 6465", "followers_count": 165, "friends_count": 111, "statues_count": 601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederick, MD", "id": "ec3b8b62828d8f6a", "name": "Frederick", "place_type": "city", "bounding_box": rectangle("-77.476711,39.369128 -77.359293,39.494945") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2430325, "cityName": "Frederick" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907081551138816, "text": "@Bruiser_Brew dumbass", "in_reply_to_status": 687906971442257920, "in_reply_to_user": 464251332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 464251332 }}, "user": { "id": 3218849202, "name": "Paul Martinez", "screen_name": "1027Paul", "lang": "en", "location": "null", "create_at": date("2015-05-17"), "description": "Lowkey. Dallas. I'll be damned if I let anyone take my smile away.", "followers_count": 104, "friends_count": 82, "statues_count": 7229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907081660293121, "text": "Travelling to Roswell or just twittering about Roswell? https://t.co/xoWM8yYiAI #Roswell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3616,34.0232"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Roswell" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 990, "friends_count": 312, "statues_count": 2529605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roswell, GA", "id": "548a885336da5635", "name": "Roswell", "place_type": "city", "bounding_box": rectangle("-84.419128,33.971999 -84.261548,34.110424") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1367284, "cityName": "Roswell" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907081828052992, "text": "My goal for tomorrow is to sleep the whole entire day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1352298884, "name": "アレクシス", "screen_name": "ChildishLetty", "lang": "en", "location": "Tokyo ", "create_at": date("2013-04-14"), "description": "92 // E34", "followers_count": 1098, "friends_count": 335, "statues_count": 7925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907082469703680, "text": "I've watched my group fluctuate but always in the end it's the same 4 people that have been since the beginning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2224342194, "name": "Ty $", "screen_name": "tyy_christopher", "lang": "en", "location": "null", "create_at": date("2013-11-30"), "description": "Im that one kid all parents warn you about...#graduated....COC bound", "followers_count": 95, "friends_count": 412, "statues_count": 703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907082868109312, "text": "And I'm hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30396079, "name": "Kim", "screen_name": "KimberlySade", "lang": "en", "location": "Louisiana, USA", "create_at": date("2009-04-10"), "description": "carefree black girl with a wild heart ✨❤️ #cowboysnation", "followers_count": 1351, "friends_count": 831, "statues_count": 181928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bossier City, LA", "id": "018617530de2ac1c", "name": "Bossier City", "place_type": "city", "bounding_box": rectangle("-93.74855,32.419303 -93.584206,32.630042") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22015, "countyName": "Bossier", "cityID": 2208920, "cityName": "Bossier City" } }
+{ "create_at": datetime("2016-01-15T00:00:03.000Z"), "id": 687907083228983298, "text": "So much to look forward to, baby.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458028292, "name": "Alexis J.", "screen_name": "_lexoxo", "lang": "en", "location": "Ken$ucky", "create_at": date("2012-01-07"), "description": "22 at WKU", "followers_count": 2150, "friends_count": 1489, "statues_count": 29977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907083405099008, "text": "✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 729176588, "name": "☀️SonnySideKhai☀️", "screen_name": "KhaiGlizzyyyyy", "lang": "en", "location": "On #SonnySide With Some Hoes☀️", "create_at": date("2012-07-31"), "description": "sc: khainotkai , ☀️⛽️ Never Disrespect It #SS4L", "followers_count": 3301, "friends_count": 693, "statues_count": 117279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907083539329024, "text": "Iowa rolls to rare sweep of No. 4 Michigan St. (CBS) | https://t.co/UNEBN57Rab #ncaab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.623474,41.878789"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ncaab" }}, "user": { "id": 16941578, "name": "Basketball State", "screen_name": "bbstate", "lang": "en", "location": "Chicago", "create_at": date("2008-10-23"), "description": "Advanced statistics, analysis tools, historical data and news for D1 coaches, scouts, media and fans. Unmonitored account. Get in touch: https://t.co/lG39PMGAtr", "followers_count": 5930, "friends_count": 5330, "statues_count": 48496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907083644235776, "text": "I literally cannot turn off Netflix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3246443765, "name": "Jillian Kearney", "screen_name": "jkearneyxo", "lang": "en", "location": "null", "create_at": date("2015-05-11"), "description": "Long Island // Adirondacks", "followers_count": 101, "friends_count": 170, "statues_count": 911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plattsburgh, NY", "id": "01951cce6978d067", "name": "Plattsburgh", "place_type": "city", "bounding_box": rectangle("-73.530573,44.652759 -73.420227,44.72545") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36019, "countyName": "Clinton", "cityID": 3658574, "cityName": "Plattsburgh" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907083937820673, "text": "I'm serious it's just to much work after I body Ya https://t.co/bteVusYESD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 885147763, "name": "#DONT Play Yourself", "screen_name": "evson__", "lang": "en", "location": "connecticut", "create_at": date("2012-10-16"), "description": "what a time..", "followers_count": 444, "friends_count": 339, "statues_count": 14296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stratford, CT", "id": "9133850ec8827e0a", "name": "Stratford", "place_type": "city", "bounding_box": rectangle("-73.164245,41.147132 -73.088173,41.268524") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 974260, "cityName": "Stratford" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907084206120960, "text": "https://t.co/uaVZkMI8rD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1014540800, "name": "❄Puglove❄", "screen_name": "HannaBan333", "lang": "en", "location": "Washington", "create_at": date("2012-12-15"), "description": "Hanna | 19 | Female \nI'm up all night to pet puppies, adulthood scares me, & the truth is out there.", "followers_count": 73, "friends_count": 168, "statues_count": 13593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, WA", "id": "7291a25672e0d4b1", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-122.1649,47.626845 -121.958642,47.73078") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357535, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907084617277440, "text": "Tres vicios tengo y los tengo muy arraigados de ser borracho,jugador y enamorado", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 541708990, "name": "janet lopez", "screen_name": "janetaudelialop", "lang": "en", "location": "Imlay City, MI", "create_at": date("2012-03-31"), "description": "Guanajuato , donde la vida no vale nada", "followers_count": 1450, "friends_count": 974, "statues_count": 75043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imlay City, MI", "id": "021c5eebb133f544", "name": "Imlay City", "place_type": "city", "bounding_box": rectangle("-83.092841,42.998394 -83.059769,43.050781") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26087, "countyName": "Lapeer", "cityID": 2640320, "cityName": "Imlay City" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907084772356096, "text": "How can people be so heartless", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552360325, "name": "allie", "screen_name": "alliecrescii", "lang": "en", "location": "the bay", "create_at": date("2012-04-12"), "description": "fhs'18", "followers_count": 435, "friends_count": 418, "statues_count": 3213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907084998868992, "text": "@Lambsatanist75 https://t.co/l1zgEd2O23", "in_reply_to_status": -1, "in_reply_to_user": 62208512, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 62208512 }}, "user": { "id": 493453150, "name": "Hank Scorpio", "screen_name": "boxcarswag", "lang": "en", "location": "null", "create_at": date("2012-02-15"), "description": "null", "followers_count": 131, "friends_count": 212, "statues_count": 2112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Mateo, CA", "id": "432daa3153c5fef9", "name": "San Mateo", "place_type": "city", "bounding_box": rectangle("-122.355038,37.512648 -122.275805,37.592395") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 668252, "cityName": "San Mateo" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907085300830208, "text": "Meeting with my advisor today made me realize how fast time goes by. I can't believe I apply for graduation after this semester ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68032826, "name": "Gabriellaa ✾", "screen_name": "gabiiherreraa", "lang": "en", "location": "null", "create_at": date("2009-08-22"), "description": "yay area", "followers_count": 1031, "friends_count": 547, "statues_count": 18260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907085548302336, "text": "Temp: 39.4°F - Dew Point: 36.2° - Wind: 12.7 mph - Gust: 22.8 - Rain Today: 0.00in. - Pressure: 29.38in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907085615419393, "text": "�������� I https://t.co/amNFSOkdc4", "in_reply_to_status": 687906164261048321, "in_reply_to_user": 2305314973, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2305314973, "name": "coach ryan", "screen_name": "ryan_sondrup", "lang": "en", "location": "null", "create_at": date("2014-01-22"), "description": "I'm no ladies man. I'm a one lady kind man. I just have to find her first", "followers_count": 426, "friends_count": 553, "statues_count": 7528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907085695238144, "text": "Where is Milwaukee on the map? Play the game at https://t.co/28q76CQkKl #Milwaukee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.9065,43.0389"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Milwaukee" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 990, "friends_count": 312, "statues_count": 2529606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907085741363200, "text": "�� I hate when mfs be sooo drunk and can't function �� Ik I be drunk ash but I be able to walk myself to my room everytime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448233001, "name": "Aj✨", "screen_name": "DimplessWORLD", "lang": "en", "location": "Beecher, MI ➡️ Big Rapids, MI", "create_at": date("2011-12-27"), "description": "Forever in our Hearts Granddad & Gabe✨ FerrisState'19❤️", "followers_count": 1954, "friends_count": 1413, "statues_count": 72739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Rapids, MI", "id": "006d5c5426b67de3", "name": "Big Rapids", "place_type": "city", "bounding_box": rectangle("-85.52501,43.605695 -85.404686,43.744574") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26107, "countyName": "Mecosta", "cityID": 2608300, "cityName": "Big Rapids" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907086630424576, "text": "Yo chiiiiiiillllllll https://t.co/qnGEeGNShl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 245562867, "name": "Tity Boi No Chainz", "screen_name": "ForeignMulatto", "lang": "en", "location": "null", "create_at": date("2011-01-31"), "description": "Drinking my juice in the hood", "followers_count": 440, "friends_count": 362, "statues_count": 35578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907087297327105, "text": "@jadynmt YASSSSS", "in_reply_to_status": 686747597994770432, "in_reply_to_user": 254184998, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 254184998 }}, "user": { "id": 845985205, "name": "Lexi", "screen_name": "LexiTubbs", "lang": "en", "location": "Turlock, CA", "create_at": date("2012-09-25"), "description": "| NNU Basketball & Track | IG: lexi_moze | ❤️ •BeautyInTheStruggle• CA➰ID", "followers_count": 606, "friends_count": 618, "statues_count": 10250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nampa, ID", "id": "7325f3442fd87621", "name": "Nampa", "place_type": "city", "bounding_box": rectangle("-116.637186,43.523489 -116.502031,43.634335") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16027, "countyName": "Canyon", "cityID": 1656260, "cityName": "Nampa" } }
+{ "create_at": datetime("2016-01-15T00:00:04.000Z"), "id": 687907087381209088, "text": "WHY ���� https://t.co/FASKnRtt5A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 374962444, "name": "lyss", "screen_name": "_alyssapadilla_", "lang": "en", "location": "Slytherin House, Hogwarts", "create_at": date("2011-09-17"), "description": "Do you even quidditch?", "followers_count": 1113, "friends_count": 1623, "statues_count": 50277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907087771250690, "text": "Who was at the @warriors vs. @Lakers game tonite & got the @KlayThompson & Rocco bobblehead? I'll pay you for it! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26270913, 20346956, 1703636814 }}, "user": { "id": 568965129, "name": "On-Δir 106KMEL", "screen_name": "Rowbot408", "lang": "en", "location": "IG & Snapchat: rowbot408", "create_at": date("2012-05-02"), "description": "Business Contact: rowbot408@gmail.com New On-Air Personality on #106KMEL #iHeartRadio #AfterHoursWithRow", "followers_count": 910, "friends_count": 881, "statues_count": 14582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907088018878465, "text": "Men A R E CON FUS ING", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45381974, "name": "Santana Savage", "screen_name": "SantanaSavage21", "lang": "en", "location": "houston tx", "create_at": date("2009-06-07"), "description": "Snap Chat: SantanaSavage1", "followers_count": 284, "friends_count": 228, "statues_count": 9796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907088043872256, "text": "The dialogue in Deathproof is still funny. \"Sorry I'm built like a girl, not a black man\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20137956, "name": "Dominick", "screen_name": "DVMINIXK", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-05"), "description": "#NT", "followers_count": 475, "friends_count": 476, "statues_count": 66032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907088287182848, "text": "Free Gucci", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4420731263, "name": "Döm Fleeks", "screen_name": "DomFleeks", "lang": "en", "location": "null", "create_at": date("2015-12-08"), "description": "20.Northwest Missouri State Track & Field snapchat: Dfleeks - Instagram: domtfleeks", "followers_count": 241, "friends_count": 391, "statues_count": 160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryville, MO", "id": "d2e5e6b4697f41a7", "name": "Maryville", "place_type": "city", "bounding_box": rectangle("-94.894994,40.318088 -94.831714,40.372994") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29147, "countyName": "Nodaway", "cityID": 2946640, "cityName": "Maryville" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907088589172736, "text": "@mommyMARAH never home", "in_reply_to_status": 687907037020196864, "in_reply_to_user": 262054718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262054718 }}, "user": { "id": 2569442030, "name": "just tae", "screen_name": "taewest", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-06-15"), "description": "fuck vh1", "followers_count": 411, "friends_count": 468, "statues_count": 24708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907089356865536, "text": "@ChiTownLionPSU https://t.co/WcDcX52gSO", "in_reply_to_status": -1, "in_reply_to_user": 754525626, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 754525626 }}, "user": { "id": 1085323525, "name": "Adam Clark", "screen_name": "Adlee7873", "lang": "en", "location": "North Carolina", "create_at": date("2013-01-12"), "description": "Serious football fan. Editor at http://Mandys-Pages.com, artist and writer.", "followers_count": 257, "friends_count": 300, "statues_count": 13503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907089931354112, "text": "Yall morals is fucked up man smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3260794502, "name": "Aaron", "screen_name": "AaronJones4Real", "lang": "en", "location": "null", "create_at": date("2015-06-29"), "description": "Damn Uce", "followers_count": 441, "friends_count": 449, "statues_count": 19753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907090199900160, "text": "hahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2912619458, "name": "monica", "screen_name": "saucymo", "lang": "en", "location": "Fontana, CA", "create_at": date("2014-11-28"), "description": "null", "followers_count": 147, "friends_count": 82, "statues_count": 1167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907090518519808, "text": "1.65 magnitude #earthquake. 4 mi from Redlands, CA, #UnitedStates https://t.co/8oCSY2bxQt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1925,34.0051667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 715541250, "name": "Southern CA Quakes", "screen_name": "quakesLA", "lang": "en", "location": "Los Angeles", "create_at": date("2012-07-24"), "description": "Southern California earthquake alerts based on USGS data. 2.5 magnitude and higher in Los Angeles and Southern California.", "followers_count": 16445, "friends_count": 15, "statues_count": 10560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907090631798784, "text": "This guy will never be broke, specially with his shoes selling at 200 a pop https://t.co/LWKy3pZroo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161810513, "name": "JP", "screen_name": "JP_LEE_10", "lang": "en", "location": "Brentwood, CA", "create_at": date("2010-07-01"), "description": "| WR | Liberty Graduate | C/O 2015 | Stay Tuned | warriors lost the streak on DECEMBER 12, 2015 24-1 |", "followers_count": 604, "friends_count": 312, "statues_count": 10188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907090912772097, "text": "@crystalhrtbrkr the brawlers will be every spec of car to build and have fun no drama just cars this is brawlers - bitch", "in_reply_to_status": 687906129070850048, "in_reply_to_user": 4641825852, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4641825852 }}, "user": { "id": 381933432, "name": "School Boy Hong", "screen_name": "MeMySelfAndYou0", "lang": "en", "location": "null", "create_at": date("2011-09-28"), "description": "Black Root Gang. Form over function any day. Don't assume just ask me. Local car photographer ! Your girl I'm a boy your straight I'm not", "followers_count": 250, "friends_count": 603, "statues_count": 2735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907091009294336, "text": "B A S I C https://t.co/BiHYAab1PI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832482386, "name": "Q.W.", "screen_name": "riverlinesss", "lang": "en", "location": "A R K A N S A S", "create_at": date("2012-09-18"), "description": "20 | #BlackLivesMatter l #FreePalestine | Sexual Assault Awareness | Geopolitics | Carefree Black Boy | Always Hungry", "followers_count": 755, "friends_count": 451, "statues_count": 46061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magnolia, AR", "id": "71e16cd97a552f44", "name": "Magnolia", "place_type": "city", "bounding_box": rectangle("-93.267708,33.23834 -93.208361,33.309264") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5027, "countyName": "Columbia", "cityID": 543460, "cityName": "Magnolia" } }
+{ "create_at": datetime("2016-01-15T00:00:05.000Z"), "id": 687907091248332801, "text": "@SteelFlexInc what do you have?", "in_reply_to_status": 687906919617445888, "in_reply_to_user": 158774277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 158774277 }}, "user": { "id": 196050556, "name": "Alex H", "screen_name": "AJH1993", "lang": "en", "location": "Portland, OR", "create_at": date("2010-09-27"), "description": "22 | LGBT | MLP fan | Liberal | @xoTalim & @sp_redelectric are my besties | #NSFW | @A_Kawaii_Dragon ❤ | Them/Him | Gender fluid/Demisexual", "followers_count": 1203, "friends_count": 1153, "statues_count": 71434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tigard, OR", "id": "d1f122645b638aec", "name": "Tigard", "place_type": "city", "bounding_box": rectangle("-122.840168,45.393108 -122.74354,45.456634") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4173650, "cityName": "Tigard" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269454833135616, "text": "Miranda hooked up with a murderer ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334463181, "name": "YSelle", "screen_name": "eloise_kay", "lang": "en", "location": "null", "create_at": date("2011-07-12"), "description": "im gucci ur crocs", "followers_count": 260, "friends_count": 222, "statues_count": 3775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269454954770433, "text": "Came in through the Bacc door", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 783227605, "name": "Lucas Jennings", "screen_name": "Lucas_HicasJ", "lang": "en", "location": "The Bin", "create_at": date("2012-08-26"), "description": "WMU/ATΩ. Came in Through the Bacc Door.", "followers_count": 581, "friends_count": 590, "statues_count": 5055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269455952973824, "text": "if you want to win me over, waking me up from my slumber doesn't do it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227839964, "name": "Emily Eustace", "screen_name": "emily_useless", "lang": "en", "location": "Scottsboro---Birmingham", "create_at": date("2010-12-17"), "description": "samford track • ΧΩ", "followers_count": 711, "friends_count": 327, "statues_count": 10491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homewood, AL", "id": "d7bd87a33673fb63", "name": "Homewood", "place_type": "city", "bounding_box": rectangle("-86.84954,33.414818 -86.762633,33.488982") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 135800, "cityName": "Homewood" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269456187916288, "text": "03:00:01 |Temp: 46.4ºF | Wind Chill 46.4ºF |Dew Point 43.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SW, Gusting to 2.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 97, "friends_count": 21, "statues_count": 88553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269456192090112, "text": "@_LyshaMauricex3 lmao at least she went lol that was surprising for real lol", "in_reply_to_status": 688269163228303362, "in_reply_to_user": 23844248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23844248 }}, "user": { "id": 292733074, "name": "♏️r.Loverboy ⚡️", "screen_name": "SupaMcNastyyyyy", "lang": "en", "location": "#860 Ct", "create_at": date("2011-05-03"), "description": "Success Is In My Arm Reach All I Have To Do Is Grab It !! #LakersNation #PatriotNation ! #ScorpioNation♏️♏️♏️ #Alone", "followers_count": 1899, "friends_count": 1481, "statues_count": 139015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartford, CT", "id": "61c225139f635563", "name": "Hartford", "place_type": "city", "bounding_box": rectangle("-72.718386,41.723759 -72.643547,41.807475") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937000, "cityName": "Hartford" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269456330498048, "text": "people w/ the biggest hearts feel the most pain.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302993918, "name": "Brittany Alliano", "screen_name": "brittttalliano", "lang": "en", "location": "Voorhees, NJ", "create_at": date("2015-07-31"), "description": "@6ixxx_❤️", "followers_count": 436, "friends_count": 396, "statues_count": 3842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Echelon, NJ", "id": "8b0f876af840af5d", "name": "Echelon", "place_type": "city", "bounding_box": rectangle("-75.017983,39.834074 -74.971095,39.860166") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3419900, "cityName": "Echelon" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269456678588416, "text": "righttt https://t.co/yW4rJ1EM33", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2935561198, "name": "➰", "screen_name": "keepinupwithtyy", "lang": "en", "location": "idgaf", "create_at": date("2014-12-21"), "description": "a bad bitch. #ripaj❤️", "followers_count": 886, "friends_count": 583, "statues_count": 22105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Donaldsonville, LA", "id": "46fe4f949c7c201c", "name": "Donaldsonville", "place_type": "city", "bounding_box": rectangle("-91.050993,30.075457 -90.937839,30.120324") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2221240, "cityName": "Donaldsonville" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269457383227392, "text": "You bitches are really unbelievable", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3377821961, "name": "Pete Game", "screen_name": "1petegame", "lang": "en", "location": "Palm Desert, CA", "create_at": date("2015-07-15"), "description": "depressed vikings fan", "followers_count": 126, "friends_count": 89, "statues_count": 3028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269458373099524, "text": "Wind 3.0 mph NNE. Barometer 1017.55 mb, Steady. Temperature 27.5 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-16T00:00:00.000Z"), "id": 688269458473746433, "text": "AY AYUN NA NGA IN LOVE NA NGA!!!! *matagal na kaya* hihihi... #ALDUB6thMonthsary https://t.co/3agvCaIAdV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUB6thMonthsary" }}, "user": { "id": 1731698340, "name": "ㅤ", "screen_name": "yvanadelrosario", "lang": "en", "location": "Houston, TX", "create_at": date("2013-09-05"), "description": "❃ snapchat - yvanadelrosario || swift • goulding • sheeran • lovato • maiden ❃", "followers_count": 815, "friends_count": 394, "statues_count": 48946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269459555880960, "text": "25", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 162284942, "name": "Jimmy Tsao", "screen_name": "jct116", "lang": "en", "location": "Los Angeles", "create_at": date("2010-07-02"), "description": "Christ, UCLA, musings. I like sashimi and eggs&tomatoes. Fantasmic over World of Color. The dress was definitely white and gold", "followers_count": 62, "friends_count": 170, "statues_count": 1872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269459639808001, "text": "@AdamSandler negotiation solves things an people don't scurry like ants in constant mourning bridal show", "in_reply_to_status": 688267255134617600, "in_reply_to_user": 3284994672, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316756361 }}, "user": { "id": 3284994672, "name": "Dylan Jenkins", "screen_name": "DylanJe44612911", "lang": "en", "location": "null", "create_at": date("2015-07-19"), "description": "null", "followers_count": 5, "friends_count": 5, "statues_count": 1905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269459795017728, "text": "Cant beeeelieve I'm up right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552169535, "name": "Bhesxa✨", "screen_name": "Franchescaaa_99", "lang": "en", "location": "Elmont, NY", "create_at": date("2012-04-12"), "description": "null", "followers_count": 542, "friends_count": 387, "statues_count": 7054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley Stream, NY", "id": "a6bb3ed9f8ccb4a2", "name": "Valley Stream", "place_type": "city", "bounding_box": rectangle("-73.728427,40.644775 -73.68145,40.679495") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3676705, "cityName": "Valley Stream" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269460000473088, "text": "Wish I was drunk rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 559060945, "name": "Audrey", "screen_name": "AudreyVee123", "lang": "en", "location": "Mission, Texas", "create_at": date("2012-04-20"), "description": "i love mangos and sleep |UTRGV19 snapchat:audreyvee123", "followers_count": 540, "friends_count": 486, "statues_count": 28203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission, TX", "id": "77633125ba089dcb", "name": "Mission", "place_type": "city", "bounding_box": rectangle("-98.363219,26.155046 -98.272146,26.262558") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4848768, "cityName": "Mission" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269460709376000, "text": "Wind 0 mph --. Barometer 29.84 in, Rising slowly. Temperature 47.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269461657227265, "text": "Temp: 41.2F W C: 38.2F Wind:S at 3.9kts Baro: 1006.8mb and Rising slowly Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 183, "friends_count": 216, "statues_count": 94742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269461770485760, "text": "01/16@03:00 - Temp 43.2F, WC 43.2F. Wind 2.0mph NW, Gust 5.0mph. Bar 29.427in, Falling. Rain 0.08in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 46304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269462005354496, "text": "And once again 2+2=4, and Tenadra gotta leave your ass alone. ����✌��️��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 525086623, "name": "Tenaj'", "screen_name": "NAE_beWILDn", "lang": "en", "location": "san marcos, Tx", "create_at": date("2012-03-14"), "description": ". #TxSt18. Tenadra☺️ sc:tsharie", "followers_count": 860, "friends_count": 880, "statues_count": 11033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269462009581569, "text": "@Popit4Gee happy birthday little bro have a smooth one ������✊��", "in_reply_to_status": -1, "in_reply_to_user": 2148348434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2148348434 }}, "user": { "id": 519089654, "name": "⚡️ Krispy ⚡️", "screen_name": "TavianJordan", "lang": "en", "location": "Oakland, CA", "create_at": date("2012-03-08"), "description": "〽️Dope Individual〽️|CollegeStudent✏️| 19years old| Belizean and Dominican|Track Star| SC Taygo16| ♎️| October's Very Own '96", "followers_count": 1911, "friends_count": 699, "statues_count": 129206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-01-16T00:00:01.000Z"), "id": 688269462546419712, "text": "@Hwala best shit ever. You're still enjoying their presence but can mind your own", "in_reply_to_status": 688269284003295232, "in_reply_to_user": 34501290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34501290 }}, "user": { "id": 34501290, "name": "CocoaGoddess", "screen_name": "Hwala", "lang": "en", "location": "SoCal☀️", "create_at": date("2009-04-22"), "description": "The new Shakira. Ancient Trap Goddess.", "followers_count": 295, "friends_count": 301, "statues_count": 14040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269463473410048, "text": "I get curved daily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3430432079, "name": "cy", "screen_name": "CyrusDesolace", "lang": "en", "location": "null", "create_at": date("2015-08-18"), "description": "welcome to graduation", "followers_count": 516, "friends_count": 398, "statues_count": 2924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464047976448, "text": "This was a year ago, damn https://t.co/egaRrRp2CF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487992184, "name": "Nick$ter", "screen_name": "gleason_nick", "lang": "en", "location": "null", "create_at": date("2012-02-09"), "description": "18 | ASU Biological Sciences major", "followers_count": 693, "friends_count": 414, "statues_count": 7479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464366796800, "text": "It's just hard to feel safe. I wish I could let myself feel comfortable on campus. She isn't there. I don't get why it scares me.", "in_reply_to_status": 688269325988306944, "in_reply_to_user": 329847155, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329847155, "name": "Audrey Browning", "screen_name": "audreyabrowning", "lang": "en", "location": "903 ➡️ 512", "create_at": date("2011-07-05"), "description": "Psych @UTAustin. Works @UTRecSports. Contributes @Burnt_X @SmearMagazine. Likes kindness, fair trade, coffee, cats & inclusive feminism.", "followers_count": 431, "friends_count": 274, "statues_count": 17138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464630988802, "text": "Wind 2.0 mph NW. Barometer 29.881 in, Rising. Temperature 29.2 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464656187392, "text": "Mood:�������� ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133946255, "name": "إلاهة", "screen_name": "Kvlvhhh", "lang": "en", "location": "null", "create_at": date("2010-04-16"), "description": "SNAPCHAT: kalahhhh", "followers_count": 1314, "friends_count": 862, "statues_count": 29047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464740052994, "text": "Happy birthday baby girl������ https://t.co/NhGHiUEdJQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2177173982, "name": "Nathan Carrillo", "screen_name": "Nate12a", "lang": "en", "location": "EL SERENO ", "create_at": date("2013-11-05"), "description": "TO LIVE AND DIE IN LA", "followers_count": 168, "friends_count": 180, "statues_count": 7774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464865906690, "text": "Wind 4.0 mph WNW. Barometer 29.734 in, Rising. Temperature 31.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269464970723328, "text": "49.5F (Feels: 49.5F) - Humidity: 99% - Wind: 1.6mph NE - Gust: 2.2mph - Pressure: 1021.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269465516048384, "text": "Ashley just kicked me in my breast and I just got my nipples pierced ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 113158649, "name": "ig: @dthagod", "screen_name": "bellaminaj_", "lang": "en", "location": "Girl On Fire...NY ✈️ ATL ", "create_at": date("2010-02-10"), "description": "CAU'18", "followers_count": 663, "friends_count": 310, "statues_count": 61461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269465587298304, "text": "Happy birthday Comadre!!! ���������� @ImJustKitten_ the only pic of us �� https://t.co/SJZADSX0m7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 233825102 }}, "user": { "id": 2977322160, "name": "Homar.", "screen_name": "itsomarsito", "lang": "en", "location": "Hesperia, CA", "create_at": date("2015-01-12"), "description": "null", "followers_count": 135, "friends_count": 131, "statues_count": 5955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269465679564804, "text": "https://t.co/OLnLr6yQTE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 17458, "friends_count": 5580, "statues_count": 2439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269466048696324, "text": "Wind 0.0 mph NNE. Barometer 29.52 in, Falling. Temperature 30.9 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 94, "statues_count": 156719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-16T00:00:02.000Z"), "id": 688269466711359496, "text": "I used to be so cute. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61117445, "name": "ReneeMB", "screen_name": "LaRayRayyy", "lang": "en", "location": "null", "create_at": date("2009-07-28"), "description": "don't quote me boy cuz i ain't said shit", "followers_count": 66, "friends_count": 143, "statues_count": 4159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269466967252993, "text": "Wind 1.4 mph SSE. Barometer 29.539 in, Falling Rapidly. Temperature 33.3 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 7962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467105624065, "text": "Temp: 42.8°F Wind:0.0mph Pressure: 29.813hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467126661120, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2516110564, "name": "STOCK.WALI", "screen_name": "stock_wali", "lang": "en", "location": "location, you name it.", "create_at": date("2014-04-28"), "description": "#STOCKEDUP the message is to UP YA STOCK sustockwali@gmail.com #LLD", "followers_count": 580, "friends_count": 842, "statues_count": 4184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467160203264, "text": "My gawd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282835230, "name": "Kendal Turner", "screen_name": "IAM_DR_T", "lang": "en", "location": "Dallas", "create_at": date("2011-04-15"), "description": "#M.I.L.F Moms Into Lifestyle and Fitness #Educator #mastersdegree #Futurelifecoach #author #Cowboys #mavsnation #bellaskitchen", "followers_count": 690, "friends_count": 913, "statues_count": 56029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467239886850, "text": "I CANNOT WATCH SCARY MOVIES.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2579677009, "name": "alyssa", "screen_name": "alymariahh", "lang": "en", "location": "null", "create_at": date("2014-06-20"), "description": "god's not dead he's surely alive, carlos correa is the love of my life, i play softball.", "followers_count": 358, "friends_count": 345, "statues_count": 4567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Fresnos, TX", "id": "28bb6eab7493609a", "name": "Los Fresnos", "place_type": "city", "bounding_box": rectangle("-97.517705,26.044672 -97.42441,26.094787") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4844120, "cityName": "Los Fresnos" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467269251073, "text": "I'm wide awake so if ur up text me������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 739267274, "name": "em", "screen_name": "emmavivier_", "lang": "en", "location": "null", "create_at": date("2012-08-05"), "description": "chill w the pulp", "followers_count": 1180, "friends_count": 849, "statues_count": 19475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467474763776, "text": "happy birthday dude ���� you're legal now ❤️�������� https://t.co/oCcYStP2et", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 755396660, "name": "lauren✧", "screen_name": "_laurennnk_", "lang": "en", "location": "null", "create_at": date("2012-08-13"), "description": "live like Jay", "followers_count": 198, "friends_count": 231, "statues_count": 1527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Tustin, CA", "id": "002c095d059ebf44", "name": "North Tustin", "place_type": "city", "bounding_box": rectangle("-117.83021,33.739088 -117.761221,33.787923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 652379, "cityName": "North Tustin" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269467835445251, "text": "#EXOluXioninMalaysia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "EXOluXioninMalaysia" }}, "user": { "id": 974207209, "name": "George Chan", "screen_name": "mrgeorgechan", "lang": "en", "location": "Los Angeles ", "create_at": date("2012-11-27"), "description": "null", "followers_count": 1151, "friends_count": 3281, "statues_count": 158117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269468200402944, "text": "@Kmaguire1227 are you hungry ?��", "in_reply_to_status": -1, "in_reply_to_user": 560616590, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 560616590 }}, "user": { "id": 495138803, "name": "Jess", "screen_name": "JessieLynnx33", "lang": "en", "location": "at your bitches house", "create_at": date("2012-02-17"), "description": "im chillin", "followers_count": 976, "friends_count": 594, "statues_count": 43144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garfield, NJ", "id": "86fc60f26e1639cc", "name": "Garfield", "place_type": "city", "bounding_box": rectangle("-74.129119,40.861524 -74.091977,40.89369") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3425770, "cityName": "Garfield" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469139865600, "text": "Number one thing: people do not change", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405323562, "name": "Lex☮", "screen_name": "lexxkowski", "lang": "en", "location": "null", "create_at": date("2011-11-04"), "description": "I'll see you on the dark side of the moon", "followers_count": 820, "friends_count": 929, "statues_count": 32020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Somerset, NJ", "id": "002099ae551de35c", "name": "Somerset", "place_type": "city", "bounding_box": rectangle("-74.531631,40.485807 -74.466896,40.536611") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset", "cityID": 3468370, "cityName": "Somerset" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469274124289, "text": "Temp: 53.4°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.85 in | Dewpoint: 53.2°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 164484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469559336960, "text": "#SupportOriginMelissa 37.0°F Wind:3.6mph Pressure: 29.65hpa Rising Rain Today 0.04in. Forecast: Fairly fine, improving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469701910530, "text": "Sleeeeeeepy��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727386552, "name": "Katie Downs", "screen_name": "KatieLeeigh", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "Be somebody who makes everybody else feel like a somebody. ☀️❤️", "followers_count": 1230, "friends_count": 1073, "statues_count": 20184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469785821185, "text": "hmmmm��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1382802235, "name": "G37", "screen_name": "Calii_Q", "lang": "en", "location": "null", "create_at": date("2013-04-26"), "description": "I got it.", "followers_count": 811, "friends_count": 728, "statues_count": 43366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469823578113, "text": "One time I was supposed to be working but my boss & I spent two hours driving around surprise looking for pop rocks & buying fireworks AHHAH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770844799, "name": "bista", "screen_name": "kswaggiee_", "lang": "en", "location": "kauai", "create_at": date("2012-08-20"), "description": "null", "followers_count": 1081, "friends_count": 491, "statues_count": 65206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469886464000, "text": "@KimSuperb https://t.co/HAvKXDI2uX", "in_reply_to_status": -1, "in_reply_to_user": 175273314, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 175273314 }}, "user": { "id": 103163052, "name": "KingZaiiTV", "screen_name": "WildBoyZaii", "lang": "en", "location": "Houston, TX ", "create_at": date("2010-01-08"), "description": "Photographer/Videographer Business Inquiries | KingZaiiTV@gmail.com #PVAMU19", "followers_count": 1498, "friends_count": 794, "statues_count": 70445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269469945233414, "text": "My life is next to normal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61341839, "name": "Winnford M DelaTorre", "screen_name": "Winnford", "lang": "en", "location": "Benicia, CA", "create_at": date("2009-07-29"), "description": "Love me or Hate me. You Choose • #winnfordadvice • Bay Area • Add me anywhere with the user: @winnford", "followers_count": 803, "friends_count": 551, "statues_count": 3479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269470142345216, "text": "����- I miss you leeebunny lol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2747508480, "name": "za'yha clark✨♍️♥️", "screen_name": "high4lean__", "lang": "en", "location": "Pittsburg, CA", "create_at": date("2014-08-19"), "description": "new life,who are you? #torryworldcrazy ❤️", "followers_count": 300, "friends_count": 254, "statues_count": 5609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269470226198529, "text": "Temp: 27.2°F - Dew Point: 17.7° - Wind: 11.7 mph - Gust: 24.2 - Rain Today: 0.00in. - Pressure: 29.66in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269470243008512, "text": "@WaveringEyes kumbia kings", "in_reply_to_status": 688268306562723841, "in_reply_to_user": 77569953, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77569953 }}, "user": { "id": 312178497, "name": "Culero Álvarez", "screen_name": "JaRulefanpage", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "Slyder Jotos Locos", "followers_count": 76, "friends_count": 92, "statues_count": 429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269470259740672, "text": "Wind 2.0 mph NNE. Barometer 29.374 in, Falling Rapidly. Temperature 43.6F. Rain today 0.23 in. Humidity 95% | Merry Christmas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 579, "friends_count": 778, "statues_count": 35665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269470855331840, "text": "I'm waiting ☹️ https://t.co/kFC8jOBHBc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1623904640, "name": "Rey Soto", "screen_name": "mrsoto23", "lang": "en", "location": "Abilame, TX", "create_at": date("2013-07-26"), "description": "AHS #underrated", "followers_count": 278, "friends_count": 277, "statues_count": 2009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-01-16T00:00:03.000Z"), "id": 688269471102808064, "text": "I hate when I'm in pain & I gotta act like it don't hurt ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1492007514, "name": "K", "screen_name": "_keyawna", "lang": "en", "location": "818", "create_at": date("2013-06-07"), "description": "null", "followers_count": 1081, "friends_count": 612, "statues_count": 24146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269471132168192, "text": "Wind 0.0 mph ESE. Barometer 29.460 in, Falling Rapidly. Temperature 34.4 °F. Rain today 0.02 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269471211888642, "text": "I love being drunk because than thinking about you doesn't hurt so much.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550605349, "name": "Brianna Boyer", "screen_name": "bria_boyer", "lang": "en", "location": "New York, USA", "create_at": date("2012-04-10"), "description": "Chances are I probably don't like you.", "followers_count": 723, "friends_count": 872, "statues_count": 7174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chittenango, NY", "id": "ad4fea6656b7d98e", "name": "Chittenango", "place_type": "city", "bounding_box": rectangle("-75.909051,43.024845 -75.855054,43.065909") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36053, "countyName": "Madison", "cityID": 3615561, "cityName": "Chittenango" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269472331776002, "text": "I forgot I work at 3:45 tomorrow FUCK!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3663054318, "name": "Jovon Beroit", "screen_name": "KingSimba042515", "lang": "en", "location": "Simi Valley, CA", "create_at": date("2015-09-23"), "description": "Live From The Guddah California Living King Simba 04-25-15❤️❤️ @chynadoll055", "followers_count": 141, "friends_count": 193, "statues_count": 647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269472709263360, "text": "Whenever possible, pay in cash. Swiping a Visa/Amex doesnt rise \"overspending\" red flags and feel as painful as paying physical notes does.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 885197238, "name": "Alvin Tan 陈杰毅", "screen_name": "AlvinTanYOLO", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-10-16"), "description": "Asian male talent 男神 / Bookings: Alvin@AlvinHeaven.com #AMWF #AsianMaleWhiteFemale #Interracial #黄男白女色情 FOLLOW @AMWFHeaven", "followers_count": 12730, "friends_count": 138, "statues_count": 2106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269472864468992, "text": "@jeaaaanie yeah basically lol", "in_reply_to_status": 688269080265031681, "in_reply_to_user": 985766828, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 985766828 }}, "user": { "id": 1335539611, "name": "Ass catchem", "screen_name": "OsiriisThaGod_", "lang": "en", "location": "null", "create_at": date("2013-04-07"), "description": "I am the Jabroni-beating, pie-eating, trailblazing, eyebrow-raising, if you want it come get it, peoples champion #RipAntandKristina", "followers_count": 687, "friends_count": 486, "statues_count": 29913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aquia Harbour, VA", "id": "bee78d718d7173b6", "name": "Aquia Harbour", "place_type": "city", "bounding_box": rectangle("-77.406482,38.437626 -77.338403,38.496979") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51179, "countyName": "Stafford", "cityID": 5102112, "cityName": "Aquia Harbour" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269473350987776, "text": "damn I love my boobs they're fun to grab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307559598, "name": "rissa", "screen_name": "rissacoon", "lang": "en", "location": "alaskuh", "create_at": date("2014-01-23"), "description": "♋️", "followers_count": 772, "friends_count": 187, "statues_count": 29548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakes, AK", "id": "6de06a92516436ce", "name": "Lakes", "place_type": "city", "bounding_box": rectangle("-149.360927,61.563018 -149.249951,61.642803") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 242832, "cityName": "Lakes" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269473493598210, "text": "Cc d BC s qq cd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.79798946,32.42185062"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 18919643, "name": "Chigozie", "screen_name": "Mr3DayWknd", "lang": "en", "location": "Behind you", "create_at": date("2009-01-12"), "description": "Feed the soul", "followers_count": 362, "friends_count": 341, "statues_count": 7132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269473795538946, "text": "You're right. I even talk in my sleep. https://t.co/xwihIHAhuR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3294254790, "name": "Lauren no Hill ✨", "screen_name": "laurennicolehar", "lang": "en", "location": "Columbia, MO", "create_at": date("2015-07-25"), "description": "Young Black Queen. STL Raised me. Lauryn Hill mindset, Beyonce attitude. Internal Eternal Sunshine. PR Major. Cultural Anthro Minor. #Mizzou19 #Sensational", "followers_count": 573, "friends_count": 551, "statues_count": 12580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Jamestown, MO", "id": "01207761679efbc2", "name": "Old Jamestown", "place_type": "city", "bounding_box": rectangle("-90.340682,38.808252 -90.229608,38.872514") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2954352, "cityName": "Old Jamestown" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269474076598272, "text": "@needubadd lmfao the second u chugged UV I knew u was done for the night", "in_reply_to_status": 688268788475641856, "in_reply_to_user": 3595549392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3595549392 }}, "user": { "id": 1297414224, "name": "steph", "screen_name": "smallsteph12", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-03-24"), "description": "null", "followers_count": 737, "friends_count": 389, "statues_count": 42189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269474336620544, "text": "This guy just told me my tits aren't as big as they look online", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2517032654, "name": "Han", "screen_name": "_hellaugly_", "lang": "en", "location": "Carmichael, CA", "create_at": date("2014-05-22"), "description": "My cat is more popular than I am", "followers_count": 125, "friends_count": 101, "statues_count": 463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmichael, CA", "id": "b37d3f2758ab6f61", "name": "Carmichael", "place_type": "city", "bounding_box": rectangle("-121.346077,38.591142 -121.291344,38.691468") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 611390, "cityName": "Carmichael" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269474730872832, "text": "@ToriKelly I wish your M &G weren't so expensive for your show in Orlando!! IT WOULD BE AN ABSOLUTE HONOR TO MEET U! I HAVE FLOOR SEATS THO!", "in_reply_to_status": -1, "in_reply_to_user": 19017237, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19017237 }}, "user": { "id": 546146156, "name": "Jonathan Russell", "screen_name": "jrussell2407", "lang": "en", "location": "Queens New York/Spring Hill FL", "create_at": date("2012-04-05"), "description": "https://m.soundcloud.com/j-russ-7/j-russ-ft-mikey-d-make-it check out my new song called Make it right ft mikey d", "followers_count": 379, "friends_count": 764, "statues_count": 804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Hill, FL", "id": "68042baafa8b7e0a", "name": "Spring Hill", "place_type": "city", "bounding_box": rectangle("-82.639883,28.433671 -82.428825,28.534024") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12053, "countyName": "Hernando", "cityID": 1268350, "cityName": "Spring Hill" } }
+{ "create_at": datetime("2016-01-16T00:00:04.000Z"), "id": 688269474974203904, "text": "Grab an aux cord and listen to this https://t.co/N6Tz8pEso2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3265778689, "name": "James Hartman", "screen_name": "jameshhartman99", "lang": "en", "location": "Anaheim, CA", "create_at": date("2015-07-01"), "description": "welcome to guppyhouse", "followers_count": 85, "friends_count": 95, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269475586551808, "text": "how does it feel being 5'9 or taller ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949219788, "name": "✨", "screen_name": "yourgoddesssss", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "#SSU18' | positive vibes❤", "followers_count": 15972, "friends_count": 7879, "statues_count": 39809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269475871760384, "text": "@PanchoRiveraa_ \nThanks panch, but we all did good", "in_reply_to_status": 688195665860538370, "in_reply_to_user": 467028258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 467028258 }}, "user": { "id": 2237024839, "name": "Jan 20", "screen_name": "iniguez220", "lang": "en", "location": "null", "create_at": date("2013-12-08"), "description": "khs #22 Soccer is my passion⚽ Just a kid doing his best", "followers_count": 283, "friends_count": 253, "statues_count": 4817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269476555444224, "text": "If I could be a guest judge on @TopChefA3 just to eat the food. ������ #dreams #eatallthefood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dreams", "eatallthefood" }}, "user_mentions": {{ 1451843438 }}, "user": { "id": 461833290, "name": "Faviola Leyva", "screen_name": "FaviolaLeyva", "lang": "en", "location": "Berkeley, CA", "create_at": date("2012-01-11"), "description": "video journalist // @ucbsoj class of '16 • @nahj • @ucsantabarbara alumna // San Diego transplant now living in the Bay Area.", "followers_count": 377, "friends_count": 840, "statues_count": 5449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269477058711554, "text": "#LasVegas #LAX #2016 @ Luxor Hotel and Casino https://t.co/vfbRGcrmiR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17566703,36.0957922"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "LasVegas", "LAX" }}, "user": { "id": 271232341, "name": "John Michael", "screen_name": "ramirm73", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-03-23"), "description": "The Body & Blood of Jesus Christ is my Salvation.", "followers_count": 92, "friends_count": 280, "statues_count": 869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269477067100160, "text": "How can you not like good cheerleading? Goodbye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152046982, "name": "Brandon☯", "screen_name": "brandonfsu_", "lang": "en", "location": "bow down to the bullet nation ", "create_at": date("2010-06-04"), "description": "19 | only real niggas in my call list | so fab 15-16 #RFARODB", "followers_count": 1487, "friends_count": 982, "statues_count": 86314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brawley, CA", "id": "8da9aff1e34d7b52", "name": "Brawley", "place_type": "city", "bounding_box": rectangle("-115.568757,32.953097 -115.507293,33.004816") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 608058, "cityName": "Brawley" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269477809491969, "text": "Espero que Rennê volte hj mais tarde ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 3611616017, "name": "Luh Daneluz ❤️", "screen_name": "FlwCumplices", "lang": "pt", "location": "Cidade das Trouxas ", "create_at": date("2015-09-10"), "description": "Magcult ➡️ Lovatic ➡️JoaoGuinatica ➡️ JOLARIANA ➡️Sendo trouxa por uma novela teen e infantil, que tem animais que fala⭐️10/09⭐️ Gustavo Follow ❤️", "followers_count": 1219, "friends_count": 747, "statues_count": 5520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269478086361088, "text": "������ Dangelo caught her slippin smh. 5/15/15✌��️ https://t.co/wdBPZJmtXG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2611556883, "name": "Derek Life", "screen_name": "ak907Hooper", "lang": "en", "location": "Fairbanks Ak", "create_at": date("2014-06-15"), "description": "Proverbs 3:5 // Lathrop Basketball // LakerNation // PackerNation", "followers_count": 863, "friends_count": 750, "statues_count": 7578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269478237343747, "text": "@xo_snooks ��", "in_reply_to_status": 688269303334842368, "in_reply_to_user": 622044475, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 622044475 }}, "user": { "id": 165889254, "name": "Mar.", "screen_name": "iLLRoyalty", "lang": "en", "location": "Cleveland", "create_at": date("2010-07-12"), "description": "| #LongLiveZuse | #LongLiveJen | Producer | marbeatz17@gmail.com | 1/1™", "followers_count": 871, "friends_count": 292, "statues_count": 68259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269478321233921, "text": "So lost on what I did?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1524041246, "name": "Young Quavo", "screen_name": "titusjohnson7", "lang": "en", "location": "null", "create_at": date("2013-06-16"), "description": "Olivia Vasquez has my heart❤️||Senior @Aloha||Football is my life||Haters gon' hate but I'm still gonna shine||D1 bound||http://hudl.com/athlete/475258…", "followers_count": 673, "friends_count": 1106, "statues_count": 4248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269478338015232, "text": "When you flash out & lose it... You look like the bad person.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.8092211,30.9914945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2362340904, "name": "♥Kameron'sMommy♥", "screen_name": "HeartTaken4", "lang": "en", "location": "Simms City", "create_at": date("2014-02-26"), "description": "♥Single• February6th• January28th♥", "followers_count": 678, "friends_count": 1627, "statues_count": 2108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22009, "countyName": "Avoyelles", "cityID": 2270525, "cityName": "Simmesport" } }
+{ "create_at": datetime("2016-01-16T00:00:05.000Z"), "id": 688269478681903104, "text": "@darlene09465021@dailykos #Oregonstandoff \n#OregonMilitia-thieves\n\"get the help of these Federal lands, preserves, Refuges\"\n#WelfareRanchers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oregonstandoff", "OregonMilitia", "WelfareRanchers" }}, "user": { "id": 25725812, "name": "Martha P Fred", "screen_name": "sardnas51", "lang": "en", "location": " USA", "create_at": date("2009-03-21"), "description": "I only live once, live life to the fullest,each day is a new day & new experiences good or bad!!I'm catching up on the going on's, it's my time!watchoutdeynow", "followers_count": 1680, "friends_count": 1391, "statues_count": 106011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269480254767105, "text": "@kschatelain ��", "in_reply_to_status": 688269412399382528, "in_reply_to_user": 3356674574, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3356674574 }}, "user": { "id": 75684181, "name": "poppa bear", "screen_name": "ItsTayGuid", "lang": "en", "location": "Nederland, Texas&orWichita, KS", "create_at": date("2009-09-19"), "description": "George Strait & Texas. You'll never find another", "followers_count": 1028, "friends_count": 858, "statues_count": 22139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534889,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269480661630976, "text": "@TonyStewart man don't deal with there crap there just a bunch of drunk idiots looking for attention. Your good man keep head up.", "in_reply_to_status": -1, "in_reply_to_user": 31462230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31462230 }}, "user": { "id": 713758327, "name": "Ricky", "screen_name": "RickyBaker22", "lang": "en", "location": "null", "create_at": date("2012-07-23"), "description": "Texas", "followers_count": 479, "friends_count": 1621, "statues_count": 17520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clute, TX", "id": "6c78219b9d16a0d9", "name": "Clute", "place_type": "city", "bounding_box": rectangle("-95.422451,28.998255 -95.36782,29.054401") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4815652, "cityName": "Clute" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269481911558144, "text": "❤️ https://t.co/Ah8nKfomda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 522553265, "name": "liz ibarra", "screen_name": "ibarralizzi", "lang": "en", "location": "California, USA", "create_at": date("2012-03-12"), "description": "Bay Area Territory @anava2828 @sfgiants Don't tell me how to live my life. sc: ibarralizzi http://m.gofund.me/kwhqvy2g?rcid=e3d9fbf0e4034b84a3aa47df4110d4c3", "followers_count": 755, "friends_count": 705, "statues_count": 41905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, CA", "id": "6684d4e60ddfc668", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-121.260686,36.307617 -121.226796,36.339441") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 630994, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269482280628224, "text": "Love my crew @danielqenglish @iamjasonayala bjmelchor d_camuy_88 #crossfitbeachfitness… https://t.co/qkmuYLaGU0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.07115398,33.80746453"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "crossfitbeachfitness" }}, "user_mentions": {{ 106345609, 57576514 }}, "user": { "id": 14676887, "name": "JR", "screen_name": "JRLANDS", "lang": "en", "location": "Rossmoor", "create_at": date("2008-05-06"), "description": "http://Dancekar.com http://Rainbowdance.com http://Ragtradeprinting.com http://Handcraftedmiami.com", "followers_count": 303, "friends_count": 108, "statues_count": 2313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Alamitos, CA", "id": "0799ff0a3c1006e9", "name": "Los Alamitos", "place_type": "city", "bounding_box": rectangle("-118.092825,33.781057 -118.032515,33.819621") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 643224, "cityName": "Los Alamitos" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269482410684417, "text": "Zayn and Liam R&b collab pls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1297069363, "name": "haley", "screen_name": "h_lxy", "lang": "en", "location": "tattooine", "create_at": date("2013-03-24"), "description": "Anything it takes to make you stay. lowkey songwriter", "followers_count": 101921, "friends_count": 64798, "statues_count": 27202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MN", "id": "3eb9b911bea717c5", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-93.556704,44.978417 -93.400552,45.0729") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2751730, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269482515533824, "text": "It's the thought of you, that's driving me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598967536, "name": "Marcos Godoy", "screen_name": "__mgodoy", "lang": "en", "location": "ucsb. oc ", "create_at": date("2012-06-03"), "description": "don't really know what I'm doing", "followers_count": 369, "friends_count": 324, "statues_count": 31215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269482729410560, "text": "What we Gon have dessert or disaster??????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 254717231, "name": "L.", "screen_name": "lorrrstens_", "lang": "en", "location": "CO'13 201Davis PSU'17", "create_at": date("2011-02-19"), "description": "If he's not Harry Styles he's not worth it.", "followers_count": 621, "friends_count": 675, "statues_count": 26876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "State College, PA", "id": "22c613c36f32f0d1", "name": "State College", "place_type": "city", "bounding_box": rectangle("-77.917295,40.749326 -77.798924,40.817749") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42027, "countyName": "Centre", "cityID": 4273808, "cityName": "State College" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269482985308165, "text": "We're #hiring! Read about our latest #job opening here: Customer Service Representative - https://t.co/Q6nvsSqC3I https://t.co/DHlTLXZk1X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.6421577,43.0915814"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 289, "friends_count": 217, "statues_count": 3593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henrietta, NY", "id": "00c60988621e2c71", "name": "Henrietta", "place_type": "city", "bounding_box": rectangle("-77.730299,43.017206 -77.553575,43.096872") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269483027238913, "text": "Fawwwkkk I'm drunk asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273817838, "name": "MoneyBag Deezy", "screen_name": "DeezyJuggin", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-03-28"), "description": "#NODSQUAD| Music Artist| Omaha,Ne✈✈✈ Dallas,Tx|Free Kevin|Booking&Features moneybagdeezy@gmail.com|sc:deezymacc", "followers_count": 407, "friends_count": 739, "statues_count": 3170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269483132096512, "text": "Trustwave #IT #Job: Software Engineer (Java) (#Chicago, IL) https://t.co/DVZ0SsP70R #infosec #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "IT", "Job", "Chicago", "infosec", "Jobs", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 278, "friends_count": 160, "statues_count": 554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269483169820672, "text": "When dudes try and slid in ya girls dm's https://t.co/FHxIzNXan1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4553813114, "name": "Alex campbell", "screen_name": "Campbell_soup95", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "null", "followers_count": 81, "friends_count": 83, "statues_count": 161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, IN", "id": "6181e0f8ac8150e1", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-85.744882,40.49346 -85.61564,40.610902") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18053, "countyName": "Grant", "cityID": 1846908, "cityName": "Marion" } }
+{ "create_at": datetime("2016-01-16T00:00:06.000Z"), "id": 688269483639635968, "text": "@JonahMarais thanks for actually caring about us.", "in_reply_to_status": -1, "in_reply_to_user": 2329626888, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2329626888 }}, "user": { "id": 637058503, "name": "ashley !", "screen_name": "FRICKMARAIS", "lang": "en", "location": "chicago", "create_at": date("2012-07-16"), "description": "wes and geoff", "followers_count": 9310, "friends_count": 729, "statues_count": 191383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269483933216769, "text": "I don't even know who to turn to anymore.. Like does anyone even understand?..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572986851, "name": "thedevilsdaughter♥", "screen_name": "kimberlingarci1", "lang": "en", "location": "null", "create_at": date("2012-05-06"), "description": "The best is yet to come.. I can't wait to meet the love of my life June 15, 2016 ♥", "followers_count": 333, "friends_count": 631, "statues_count": 3192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269484084183040, "text": "I guess Ima hoe according to some of you females", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1066170590, "name": "#Flocka#", "screen_name": "Paulyyy23", "lang": "en", "location": "null", "create_at": date("2013-01-06"), "description": "null", "followers_count": 768, "friends_count": 741, "statues_count": 31618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269484172292096, "text": "@gabe__diaz okay", "in_reply_to_status": 688269328639119360, "in_reply_to_user": 881584508, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 881584508 }}, "user": { "id": 491619483, "name": "Piero", "screen_name": "nvctrias", "lang": "en", "location": "Sunagakure", "create_at": date("2012-02-13"), "description": "19. Jinchūriki. I keep to myself.", "followers_count": 382, "friends_count": 92, "statues_count": 22219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269484361035778, "text": "gosh I am so mentally drained from everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 375786951, "name": "Ta'lor", "screen_name": "tsandolla", "lang": "en", "location": "12•21•2015", "create_at": date("2011-09-18"), "description": "@Drake fanatic | 18 | 1 Peter 5:7 | sc: taalorsandul", "followers_count": 1159, "friends_count": 665, "statues_count": 7799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Republic, MO", "id": "44fafdc06c719c62", "name": "Republic", "place_type": "city", "bounding_box": rectangle("-93.507092,37.096369 -93.420182,37.155321") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29077, "countyName": "Greene", "cityID": 2961238, "cityName": "Republic" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269484990177284, "text": "Pasa The bowl.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2278669238, "name": "Ivan Sierra", "screen_name": "ivanthegreat408", "lang": "en", "location": "cloud 9", "create_at": date("2014-01-05"), "description": "saludos plebes✌️", "followers_count": 306, "friends_count": 572, "statues_count": 1769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269485300531201, "text": "#Birch / https://t.co/bHG7UFohtK Excellent https://t.co/fL46nRvmm9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.32948618,34.10088263"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Birch" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4263, "friends_count": 875, "statues_count": 376469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269485329911812, "text": "#Birch https://t.co/bHG7UFohtK Excellent https://t.co/GBmWdGS11F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.32948618,34.10088263"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "Birch" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4263, "friends_count": 875, "statues_count": 376469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269485845839874, "text": "I've tasted victory and I've felt the sting of defeat but I don't quit bro so you got the wrong one if you think I'm gonna \"give up\". ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393773566, "name": "The D5x⚡️", "screen_name": "_Doyle5x", "lang": "en", "location": "snapchat: doyletooraw", "create_at": date("2011-10-18"), "description": "Proverbs 18:10 | #HPU20 ‼️", "followers_count": 525, "friends_count": 312, "statues_count": 35570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-16T00:00:07.000Z"), "id": 688269485883539456, "text": "Can you find Antioch on the map? Just try it at https://t.co/yY44v4dJFV #Antioch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.806,38.0049"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Antioch" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 994, "friends_count": 312, "statues_count": 2531150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631842329595904, "text": "@arielthemidge gfdi I love you sm �������� \nThe first one is me I promise (-: https://t.co/WfaKXKIxkY", "in_reply_to_status": 688627916716945408, "in_reply_to_user": 2404992756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2404992756 }}, "user": { "id": 2687933448, "name": "rx queen", "screen_name": "goawaytaylor", "lang": "en", "location": "Cc,tx", "create_at": date("2014-07-28"), "description": "i dont think we survived that crash", "followers_count": 255, "friends_count": 149, "statues_count": 5660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631842384183296, "text": "@therealjuicyj can't wait to see you today ����", "in_reply_to_status": -1, "in_reply_to_user": 24275419, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24275419 }}, "user": { "id": 2853609350, "name": "Manny Mendez", "screen_name": "mannymendez21", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "Anaheim So-Cal.. Style is Everything! TGOD", "followers_count": 261, "friends_count": 226, "statues_count": 3985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631842820325376, "text": "Interested in a #Retail #job near #MYRTLEBEACH, SC? This could be a great fit: https://t.co/8jgOY85YnP #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8683471,33.7249091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "MYRTLEBEACH", "Hiring" }}, "user": { "id": 2706829250, "name": "BI-LO Jobs", "screen_name": "BiLoJobs", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "null", "followers_count": 5, "friends_count": 0, "statues_count": 12144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631843130720257, "text": "if anyone wants to add me on peach, I am patronusisotter.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56353438, "name": "Draft LOttery", "screen_name": "patronusisotter", "lang": "en", "location": "Oregon", "create_at": date("2009-07-13"), "description": "#ripcity / #goducks / #rctid* / the empire / #bangarang. avi's by @SwooshMcDuck", "followers_count": 549, "friends_count": 918, "statues_count": 74486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631843306917888, "text": "@Poppi4_ @juicyjss ����������", "in_reply_to_status": 688624678630338560, "in_reply_to_user": 49278222, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 49278222, 46829801 }}, "user": { "id": 412515370, "name": "Get@Klew", "screen_name": "GoTTaGeTAKleW", "lang": "en", "location": "WERE KOBE DROPS 81 ", "create_at": date("2011-11-14"), "description": "Official twitter of SouthBays verys own Klew!! #GetaKlew the EP out now order your copy !! get a taste of the Album http://www.soundcloud.com/gottagetaklew", "followers_count": 738, "friends_count": 756, "statues_count": 8024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawndale, CA", "id": "cce33d74ceffbe08", "name": "Lawndale", "place_type": "city", "bounding_box": rectangle("-118.369186,33.872914 -118.343796,33.902665") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640886, "cityName": "Lawndale" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631843533549568, "text": "I'm going down and you have watched me drown in a river of tears", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140581458, "name": "Taemy", "screen_name": "TayMatosh", "lang": "en", "location": " Brazil ✈️ Manitowoc, WI ", "create_at": date("2010-05-05"), "description": "We accept the love we think we deserve", "followers_count": 818, "friends_count": 466, "statues_count": 28155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manitowoc, WI", "id": "01b1e7677560ac06", "name": "Manitowoc", "place_type": "city", "bounding_box": rectangle("-87.755505,44.037639 -87.620088,44.153557") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55071, "countyName": "Manitowoc", "cityID": 5548500, "cityName": "Manitowoc" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631843638259712, "text": "Wind 4.5 mph WSW. Barometer 29.99 in, Rising slowly. Temperature -13.2 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631843894108161, "text": "3cdfb0f68a5122fe7f72c151d5fe5b4f6c360024680299a7f698b6172477fcb95fb062a35c627be5b293a9b21a9d44bde67906786fc6158e45f8cb9dedce01157b4f7f000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-152.70335711,54.11041137"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 133, "friends_count": 1, "statues_count": 710506 }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631844066074624, "text": "Check out my #listing in #Athens #AL https://t.co/MJHvxiYVoI #realestate #realtor https://t.co/ukhdKUFETV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8830318,34.7530092"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "listing", "Athens", "AL", "realestate", "realtor" }}, "user": { "id": 2424256471, "name": "Pat West Real Estate", "screen_name": "PWestRealEstate", "lang": "en", "location": "null", "create_at": date("2014-04-02"), "description": "Pat West, ABR\r\nBroker Associate\r\nRE/MAX Alliance", "followers_count": 10, "friends_count": 11, "statues_count": 118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631844506484736, "text": "Wind 0.0 mph ---. Barometer 1023.67 mb, Falling. Temperature 21.9 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631844527566848, "text": "Pretttyyy much https://t.co/chFyYfBtX8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242189370, "name": "Sae.", "screen_name": "Saeku", "lang": "en", "location": "null", "create_at": date("2011-01-23"), "description": "Biggest @Giants fan you've ever met | 757 | saekubiz@gmail.com", "followers_count": 9628, "friends_count": 973, "statues_count": 82479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631844913348609, "text": "i want to get my record label here https://t.co/0CiD0cNezJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031647083, "name": "Alli Rowland", "screen_name": "allirowland123", "lang": "en", "location": "Watertown, SD", "create_at": date("2015-02-11"), "description": "A TRUE MAGCON FAN", "followers_count": 43, "friends_count": 48, "statues_count": 85 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watertown, SD", "id": "c1b7e203a1d9630b", "name": "Watertown", "place_type": "city", "bounding_box": rectangle("-97.254743,44.876554 -97.045323,44.950863") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46029, "countyName": "Codington", "cityID": 4669300, "cityName": "Watertown" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631845164990464, "text": "I don't think I could fuck with someone else's whip that's fucked up they work for there shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1428236389, "name": "Steven.", "screen_name": "StevennGuerrero", "lang": "en", "location": "HTX", "create_at": date("2013-05-14"), "description": "#LongLivePollo", "followers_count": 334, "friends_count": 299, "statues_count": 5595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631845307588608, "text": "@nine11inreverse If only the Happy go Lucky Americans would open their eyes. If only ! Too big a wish ?", "in_reply_to_status": 688630900347793408, "in_reply_to_user": 632552006, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 632552006 }}, "user": { "id": 2719172970, "name": "esmaiil fotoohi", "screen_name": "1318mef", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "null", "followers_count": 647, "friends_count": 638, "statues_count": 23910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Campbell, CA", "id": "0354c827bfda68de", "name": "Campbell", "place_type": "city", "bounding_box": rectangle("-121.991728,37.254665 -121.918729,37.306999") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 610345, "cityName": "Campbell" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631845651480576, "text": "temperature down 38°F -> 36°F\nhumidity up 65% -> 70%\nwind 10mph -> 12mph\npressure 29.73in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0567,42.35863"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108076943, "name": "Boston Weather", "screen_name": "_BostonMA", "lang": "en", "location": "Boston, MA", "create_at": date("2010-01-24"), "description": "Weather updates, forecast, warnings and information for Boston, MA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 210, "friends_count": 3, "statues_count": 27496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-17T00:00:00.000Z"), "id": 688631845777309696, "text": "@madiburton99 hope your special day brings you all that your heart desires! Here's wishing you a day full of pleasant surprises! HAPPY B-DAY", "in_reply_to_status": -1, "in_reply_to_user": 962973876, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 962973876 }}, "user": { "id": 2237010824, "name": "itza", "screen_name": "itzachaidez", "lang": "en", "location": "fatgirlsplanet", "create_at": date("2013-12-08"), "description": "Thick and Proud. Feminist", "followers_count": 896, "friends_count": 667, "statues_count": 5172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631846410665985, "text": "Sanchita #__ lost count ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2252753353, "name": "J♡", "screen_name": "JennyJih2000", "lang": "en", "location": "null", "create_at": date("2013-12-18"), "description": "Hillcrest hs //sc:jennyileen23//insta:_jennileen.", "followers_count": 248, "friends_count": 574, "statues_count": 2604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631846427578368, "text": "01/17@03:00 - Temp 39.1F, WC 39.1F. Wind 2.5mph NW, Gust 7.0mph. Bar 29.892in, Rising slowly. Rain 0.00in. Hum 66%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 46328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631846599405568, "text": "Temp: 29.8F W C: 29.8F Wind:--- at 0.0kts Baro: 1020.0mb and Falling slowly Rain today: 0.00in R H: 79% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 184, "friends_count": 216, "statues_count": 94838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631846767230976, "text": "*grabs to-do list* MADE IT HAIL ON DEREK MARTINEZ!! #finally", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "finally" }}, "user": { "id": 74328714, "name": "Leo Calderon", "screen_name": "huniibadger", "lang": "en", "location": "Edmond, Oklahoma", "create_at": date("2009-09-14"), "description": "I once performed surgery with Dr. McDreamy", "followers_count": 228, "friends_count": 681, "statues_count": 9035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631846930780160, "text": "@estherhouses @garcia_monicaaa yas lol", "in_reply_to_status": 688631789070336000, "in_reply_to_user": 3259566992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3259566992, 2916872888 }}, "user": { "id": 463536170, "name": "Sarah", "screen_name": "SarahMarieLeiva", "lang": "en", "location": "null", "create_at": date("2012-01-13"), "description": "null", "followers_count": 480, "friends_count": 441, "statues_count": 16583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631847098515458, "text": "Need to start going to beautycon cause I be missing out ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279403436, "name": "Lindo⚡️", "screen_name": "Aniweta_", "lang": "en", "location": "(D)MV✈️HTX", "create_at": date("2011-04-08"), "description": "Nigerian Ting ✨", "followers_count": 2071, "friends_count": 871, "statues_count": 107723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631847249702912, "text": "It's 3am & I just got fired from my job. Oooooookayyyy to", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111956261, "name": "KeepCalm&StayClassy", "screen_name": "HerNameWasNikki", "lang": "en", "location": "detroit-ish. ", "create_at": date("2010-02-06"), "description": "twenty one. snapchat; @ nicolegearing", "followers_count": 1642, "friends_count": 201, "statues_count": 64076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Park, MI", "id": "f3e6cc3895998aff", "name": "Oak Park", "place_type": "city", "bounding_box": rectangle("-83.202528,42.444837 -83.153812,42.488703") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2659920, "cityName": "Oak Park" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631847501299713, "text": "Stg done with them �� https://t.co/hNcSc54MKI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1920558732, "name": "✨S O J O U R N E R✨", "screen_name": "Im__Cookie", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2013-09-30"), "description": "shit, chillin", "followers_count": 2378, "friends_count": 1652, "statues_count": 28015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warren, OH", "id": "579f9e4d93c529b1", "name": "Warren", "place_type": "city", "bounding_box": rectangle("-80.894004,41.198089 -80.740485,41.272574") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3980892, "cityName": "Warren" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631848096825344, "text": "Happy 19th bday ��❤️ @ImJakePaul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 581301629 }}, "user": { "id": 1344446750, "name": "Alejandra", "screen_name": "UnlikeAle", "lang": "en", "location": "Hollywood ", "create_at": date("2013-04-11"), "description": "im probably taking a nap if im offline", "followers_count": 8806, "friends_count": 178, "statues_count": 42119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631848109387776, "text": "Borat dummy lit ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2331431912, "name": "ULTcHente_47", "screen_name": "vicente625", "lang": "en", "location": "$$$, United States", "create_at": date("2014-02-06"), "description": "THey be like tHere He go |UHS '16| #LONGLIVESTEELO #47shit #HOLLOWSQUAD #C9ISTHEFUTURE #bEASTCOAST #RATTPACK WE GLOBAL #WaterBoyz #HealthyBoyz #sesh", "followers_count": 548, "friends_count": 493, "statues_count": 8727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631848423968768, "text": "عرض لبطاريات الطاقه الشمسيه\nالكود: AV4OIKEN\nhttps://t.co/dJJz8E1f3G\n\nالكود: IC7L3OS8\nhttps://t.co/3gG4NhXuRO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 4570292608, "name": "ماجد", "screen_name": "vBlLkbZg8uc7Spi", "lang": "en", "location": "نيوجرسي", "create_at": date("2015-12-22"), "description": "أدعية ومنوعات ومعلومات@@@ سنابي as_1282", "followers_count": 69, "friends_count": 554, "statues_count": 238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, NJ", "id": "d7ce64e73b86c6f0", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-74.013771,40.808433 -73.990818,40.826826") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3422740, "cityName": "Fairview" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631848856104960, "text": "Wind 0 mph --. Barometer 29.92 in, Rising slowly. Temperature 43.9 °F. Rain today 0.05 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631848919035905, "text": "Dave Daniels. He's mine, and I couldn't be happier ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226881271, "name": "Bridget Flynn", "screen_name": "BDrain5", "lang": "en", "location": "null", "create_at": date("2010-12-15"), "description": "Teacher | Coach | Hufflepuff", "followers_count": 639, "friends_count": 718, "statues_count": 11696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granger, IN", "id": "20dc6048f416c894", "name": "Granger", "place_type": "city", "bounding_box": rectangle("-86.217544,41.709039 -86.023537,41.760686") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1828800, "cityName": "Granger" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631849304772608, "text": "@ponce_dp sure sure", "in_reply_to_status": 688631357103312896, "in_reply_to_user": 992227350, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 992227350 }}, "user": { "id": 871813142, "name": "elizabeth⚜", "screen_name": "liiizbarbie7207", "lang": "en", "location": "Edinburg, TX", "create_at": date("2012-10-10"), "description": "snapchat -beautifuliz7", "followers_count": 294, "friends_count": 329, "statues_count": 15915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631849443282945, "text": "1h 25m wait time at Grey Nuns Community Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.430878,53.462401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 364, "friends_count": 889, "statues_count": 2655 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631849535406080, "text": "@SoDopeStephanie I WISH ��\nNeed a pass again!", "in_reply_to_status": 688614737609764865, "in_reply_to_user": 321812548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321812548 }}, "user": { "id": 336354106, "name": "Vanessa Granillo", "screen_name": "Vegranillo", "lang": "en", "location": "DLR, CA", "create_at": date("2011-07-15"), "description": "She's so complicated, that's the way God made her. Sunshine mixed with a little hurricane☀️#HomeSweetCalifornia ΑΟΠ❤️", "followers_count": 673, "friends_count": 1209, "statues_count": 31086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631849556426752, "text": "@maelbii https://t.co/oHbbTxDbCb", "in_reply_to_status": -1, "in_reply_to_user": 742147610, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 742147610 }}, "user": { "id": 742147610, "name": "Maelbs✨", "screen_name": "maelbii", "lang": "en", "location": "DDD", "create_at": date("2012-08-06"), "description": "null", "followers_count": 989, "friends_count": 571, "statues_count": 38584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seagoville, TX", "id": "280d9df9ebf42155", "name": "Seagoville", "place_type": "city", "bounding_box": rectangle("-96.597051,32.62622 -96.520972,32.680947") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4866428, "cityName": "Seagoville" } }
+{ "create_at": datetime("2016-01-17T00:00:01.000Z"), "id": 688631850349166592, "text": "23:59", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2829128442, "name": "ricky", "screen_name": "ricxrdo69", "lang": "en", "location": "null", "create_at": date("2014-09-23"), "description": "null", "followers_count": 288, "friends_count": 285, "statues_count": 1902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851070574593, "text": "I hate it how when I'm tired and I'm talking to people, I always spill the beans and tell the truth about everything ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 772631780, "name": "TRETO⚾️", "screen_name": "IssacTreto", "lang": "en", "location": "null", "create_at": date("2012-08-21"), "description": "The only things I love in this world are the Dodgers, baseball, and AlyssaAva Varsity ⚾️ C/O '16", "followers_count": 393, "friends_count": 274, "statues_count": 13067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851322183680, "text": "I Need A Job But I Want To Get My Nails Done So Can't Go Back To Fast Food ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187391248, "name": "〽️ichelle", "screen_name": "MichelleeeeD", "lang": "en", "location": "null", "create_at": date("2013-11-10"), "description": "null", "followers_count": 180, "friends_count": 156, "statues_count": 3404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851477512192, "text": "����but really tho, we all out here chillin in the suburbs but wanna be hood so bad I don't get it!!! https://t.co/2O8ZvCEuxg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634245919, "name": "Teller", "screen_name": "_taytayyyy", "lang": "en", "location": "DM(V)", "create_at": date("2012-07-12"), "description": "Grind now, shine later || #VT19", "followers_count": 889, "friends_count": 693, "statues_count": 9399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blacksburg, VA", "id": "820684853e0f1eb6", "name": "Blacksburg", "place_type": "city", "bounding_box": rectangle("-80.47611,37.185195 -80.381618,37.273387") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51121, "countyName": "Montgomery", "cityID": 5107784, "cityName": "Blacksburg" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851607535617, "text": "Temp: 44.2°F Wind:0.4mph Pressure: 29.931hpa Rising Rapidly Rain Today 0.01in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851745980416, "text": "Dab game really too strong ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1572005564, "name": "D O -R I G H T", "screen_name": "DoRightJr", "lang": "en", "location": "Eastside Of Detroit, MI ", "create_at": date("2013-07-05"), "description": "Adrian College REDSHIRT Guard| #GoBulldogs #HeartOverHeight #RiseAboveHate Philippians4:13 SNAPCHAT: DoRightJr", "followers_count": 1737, "friends_count": 2231, "statues_count": 36643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Adrian, MI", "id": "4286bea8be81e904", "name": "Adrian", "place_type": "city", "bounding_box": rectangle("-84.094415,41.860395 -84.004134,41.941268") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2600440, "cityName": "Adrian" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851821314049, "text": "I hate when I text somebody and they all on Twitter and still haven't replied", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1905596473, "name": "9:27 Dillon", "screen_name": "BeyonddBasic", "lang": "en", "location": "counting blue faces", "create_at": date("2013-09-25"), "description": "#BossShit Free103emurda❤️", "followers_count": 6706, "friends_count": 2364, "statues_count": 38665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851934707712, "text": "Temp: 61.0°F | Humidity: 99% | Wind: SSE @ 3.1 mph | Barometer: 29.55 in | Dewpoint: 60.7°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 164508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631851985059840, "text": "Wind 1.0 mph NW. Barometer 29.969 in, Falling slowly. Temperature 28.9 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631852207218689, "text": "CHAZ IS MIA!!! CONTACT IF FOUND!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2751364668, "name": "Tanner Snow", "screen_name": "tanner_snow", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "I love Grace Broyles❤️❤️❤️", "followers_count": 286, "friends_count": 256, "statues_count": 1255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, KS", "id": "1276a44d783a529d", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-96.649744,39.159985 -96.499443,39.259872") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20161, "countyName": "Riley", "cityID": 2044250, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853155258369, "text": "Wind 0.0 mph ---. Barometer 30.062 in, Steady. Temperature 23.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853268516864, "text": "See what lasts. That's all.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 294310752, "name": "Gio Sundqvist Olmos", "screen_name": "giovanna_olmos", "lang": "en", "location": "New York, New York", "create_at": date("2011-05-06"), "description": "http://giovannaolmos.com", "followers_count": 421, "friends_count": 702, "statues_count": 6277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853499154432, "text": "Ya'll have gassed this girl so much ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77311930, "name": "(cry)a", "screen_name": "cymbix", "lang": "en", "location": "null", "create_at": date("2009-09-25"), "description": "and he stared at me like I was a painting", "followers_count": 1923, "friends_count": 1886, "statues_count": 22443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kannapolis, NC", "id": "3562f19d95536f2f", "name": "Kannapolis", "place_type": "city", "bounding_box": rectangle("-80.678004,35.435255 -80.561482,35.54115") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37025, "countyName": "Cabarrus", "cityID": 3735200, "cityName": "Kannapolis" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853515931648, "text": "https://t.co/pS57YopZNb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 17750, "friends_count": 6666, "statues_count": 2844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853595557888, "text": "@TIERSKELLL ohh i feel extra salty", "in_reply_to_status": 688629704283193344, "in_reply_to_user": 3087233298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3087233298 }}, "user": { "id": 2924610224, "name": "K Kali", "screen_name": "Calimade2254", "lang": "en", "location": "Redwood Ave, CA", "create_at": date("2014-12-09"), "description": "☆K Kali☆ unsigned rapper from California\n\n \n For booking/features ⬇\n \n\n ☆Email: tysonclendening@gmail.com☆", "followers_count": 307, "friends_count": 379, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125344") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631853843001344, "text": "@marylsheep I'm very curious to know what Korean pop punk sounds like��", "in_reply_to_status": 688630075097350144, "in_reply_to_user": 277152323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 277152323 }}, "user": { "id": 406410555, "name": "Paula ☕️", "screen_name": "_PaulaMarie", "lang": "en", "location": "Thatcher, AZ", "create_at": date("2011-11-06"), "description": "great to have at your side when things get stabby ✌️", "followers_count": 222, "friends_count": 259, "statues_count": 15866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thatcher, AZ", "id": "019c4b8a4a5f68ec", "name": "Thatcher", "place_type": "city", "bounding_box": rectangle("-109.795135,32.797639 -109.732871,32.86024") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4009, "countyName": "Graham", "cityID": 473420, "cityName": "Thatcher" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631854056865792, "text": "43.8F (Feels: 43.8F) - Humidity: 84% - Wind: 14.5mph N - Gust: 19.0mph - Pressure: 1029.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-17T00:00:02.000Z"), "id": 688631854325366785, "text": "I'm like wide awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056314690, "name": "B❥", "screen_name": "baayyyylliii", "lang": "en", "location": "probably at home in bed", "create_at": date("2013-01-02"), "description": "destiny is for losers, it's just a stupid excuse to wait for things to happen instead of making them happen❃ -Blair Waldorf", "followers_count": 602, "friends_count": 908, "statues_count": 18774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631854853926912, "text": "Wind 1 mph WSW. Barometer 30.09 in, Rising slowly. Temperature 59.0 °F. Rain today 0.17 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 52, "friends_count": 69, "statues_count": 25031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631855256473600, "text": "HAPPY BIRTHDAY. \n@shelbeeeloveee https://t.co/LKJ2Bc74Al", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2322860047 }}, "user": { "id": 39386637, "name": "trebor", "screen_name": "3robert5u", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-05-11"), "description": "}; #\\ll$/ SLURM! \\$ll/# ;{", "followers_count": 266, "friends_count": 844, "statues_count": 12411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631855386472448, "text": "@realDonaldTrump birds of a feather!!! Need I say more...", "in_reply_to_status": 688020474744467460, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 3310050822, "name": "Jeremy William Wilso", "screen_name": "JeremyWilliamW1", "lang": "en", "location": "Casa Grande, AZ", "create_at": date("2015-08-08"), "description": "pray that in the next three years we have our Country still!", "followers_count": 11, "friends_count": 58, "statues_count": 9 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casa Grande, AZ", "id": "fbb3d1e41acab043", "name": "Casa Grande", "place_type": "city", "bounding_box": rectangle("-111.791608,32.858246 -111.670779,32.992892") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 410530, "cityName": "Casa Grande" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631855445233664, "text": "No I'm not https://t.co/RWjdMHSo7s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1064874086, "name": "leslie", "screen_name": "ldarko_", "lang": "en", "location": "92507", "create_at": date("2013-01-05"), "description": "Justin olvera • DEP ac", "followers_count": 309, "friends_count": 158, "statues_count": 7451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631855705403392, "text": "#SupportOriginMelissa 30.4°F Wind:0.9mph Pressure: 29.88hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631855709556736, "text": "Still waiting. �� https://t.co/aPwvOCr2li", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272271302, "name": "Kelsiee", "screen_name": "wusgoodkels", "lang": "en", "location": "Chicago ✈ Little Rock, AR ", "create_at": date("2011-03-25"), "description": "stay level headed & down to earth.\nFMOSC- Ohsokels", "followers_count": 904, "friends_count": 592, "statues_count": 5838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631856040927232, "text": "@claneshaxo @dominos they got me hot, pizza hut would've got me ��", "in_reply_to_status": 688631669516038144, "in_reply_to_user": 363364856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363364856, 31444922 }}, "user": { "id": 446815325, "name": "og++﹩", "screen_name": "avo_cain", "lang": "en", "location": "hine $ide", "create_at": date("2011-12-25"), "description": "ain't never been no pressure.. life is funny like that. location on, just pull up. 1/4 JOBS. #UA19", "followers_count": 1737, "friends_count": 1426, "statues_count": 99832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631856158367746, "text": "I've been an emotional roller coaster all week ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377540701, "name": "Katelin", "screen_name": "KayMae44", "lang": "en", "location": "Lake Mohawk, OH", "create_at": date("2011-09-21"), "description": "Livin life one day at a time ✌️ AJV 12-19-10 ❤️", "followers_count": 400, "friends_count": 699, "statues_count": 4418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Mohawk, OH", "id": "0014e263364ef333", "name": "Lake Mohawk", "place_type": "city", "bounding_box": rectangle("-81.202472,40.645408 -81.173123,40.67882") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39019, "countyName": "Carroll", "cityID": 3941443, "cityName": "Lake Mohawk" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631856686874628, "text": "But y'all know how I feel about #3HolePenetration", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "3HolePenetration" }}, "user": { "id": 182888841, "name": "Patriots SB Champs", "screen_name": "enimsahJ__x", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-08-25"), "description": "22. Navy Vet. Bitch my eye isn't black in my avi, before you try it. My eyeliner & eyeshadow was all over the place hoe. Find something else.", "followers_count": 2371, "friends_count": 2323, "statues_count": 161802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond Heights, OH", "id": "fb693a79982c73c0", "name": "Richmond Heights", "place_type": "city", "bounding_box": rectangle("-81.527508,41.5378 -81.487495,41.586163") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3966894, "cityName": "Richmond Heights" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631856787537920, "text": "I have church in 5 hours Future better drop that damn tape", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1464753025, "name": "Q. 豊", "screen_name": "Q_Steeze", "lang": "en", "location": "Okinawa, Japan", "create_at": date("2013-05-28"), "description": "沖縄//Japan✈️ Va Beach//Know Yourself.//Trinidadian//18", "followers_count": 700, "friends_count": 736, "statues_count": 21672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631856913346561, "text": "Wind 0.7 mph W. Barometer 29.85 in, Rising. Temperature 32.4 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 26, "friends_count": 94, "statues_count": 156748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631857009692672, "text": "yeeeesss�� https://t.co/CTfZRNFArw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 811998962, "name": "M", "screen_name": "mikaylashayeee", "lang": "en", "location": "null", "create_at": date("2012-09-08"), "description": "null", "followers_count": 215, "friends_count": 207, "statues_count": 460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631857513025536, "text": "I knew you was the feds https://t.co/dIr0dif5Dk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1894510596, "name": "IN CHIP WE TRUST????", "screen_name": "tweezaintshit", "lang": "en", "location": "east oakland...", "create_at": date("2013-09-22"), "description": "add me on snapchat:tweezaintshit Freee zay Free Lo free saul free tev Free Dino R.I.P. Allll my loved ones #TOS. East oakland,Ca #FUTUREHIVE", "followers_count": 1405, "friends_count": 2093, "statues_count": 124121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631857588510720, "text": "@hd_shine I hope they don't play CYA V lol https://t.co/FQkJzBW8s0", "in_reply_to_status": 688630394778853376, "in_reply_to_user": 311158275, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311158275 }}, "user": { "id": 444058732, "name": "Rhyno", "screen_name": "el_Ewok96", "lang": "en", "location": "Ceres California", "create_at": date("2011-12-22"), "description": "null", "followers_count": 410, "friends_count": 380, "statues_count": 31341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631857630478337, "text": "Hml�� https://t.co/8rXimGdHc2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2741488988, "name": "Jackelean", "screen_name": "jackeleenmedin1", "lang": "en", "location": "Anchorage, AK ", "create_at": date("2014-08-17"), "description": "•me llamo Papi", "followers_count": 961, "friends_count": 1045, "statues_count": 13727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631857697714178, "text": "Wind 1.0 mph NW. Barometer 29.881 in, Rising. Temperature 27.0 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631858272157697, "text": "@ebelia_a sam is looking great in your header ��", "in_reply_to_status": -1, "in_reply_to_user": 1419312870, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1419312870 }}, "user": { "id": 411803327, "name": "ginger spice", "screen_name": "nataaaysha", "lang": "en", "location": "fetty 17:38", "create_at": date("2011-11-13"), "description": "null", "followers_count": 445, "friends_count": 305, "statues_count": 52032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631858385387520, "text": "Idfw mcm/wcw's... I'm not 12 no ty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155314543, "name": "Kaitlyn✨", "screen_name": "k8lyn96", "lang": "en", "location": "null", "create_at": date("2010-06-13"), "description": "luxurious", "followers_count": 301, "friends_count": 492, "statues_count": 4799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631858612039680, "text": "Hello. It's me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22750239, "name": "Brian Conway", "screen_name": "bconway2", "lang": "en", "location": "State College, PA", "create_at": date("2009-03-04"), "description": "ND '12 / I science at penn state now / casual bowler / saved by the fire department twice / my opinions are generally meaningless and probably should be ignored", "followers_count": 274, "friends_count": 668, "statues_count": 3305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "State College, PA", "id": "22c613c36f32f0d1", "name": "State College", "place_type": "city", "bounding_box": rectangle("-77.917295,40.749326 -77.798924,40.817749") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42027, "countyName": "Centre", "cityID": 4273808, "cityName": "State College" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631858637090816, "text": "Yaaaas https://t.co/K1V8e4fv8t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 156813496, "name": "FBGM J.Hope", "screen_name": "Jew_Hope", "lang": "en", "location": "Lewisville, TX", "create_at": date("2010-06-17"), "description": "It's Hot!!!!", "followers_count": 405, "friends_count": 414, "statues_count": 10464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-01-17T00:00:03.000Z"), "id": 688631858704297984, "text": "@willofphil @S_T_O_P_TERROR @marylene58 https://t.co/Ykm4vytJZT", "in_reply_to_status": 688347952264417280, "in_reply_to_user": 399395314, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 399395314, 2976578589, 317284206 }}, "user": { "id": 207141071, "name": "Terry O.", "screen_name": "phemale61", "lang": "en", "location": "Manhattan", "create_at": date("2010-10-24"), "description": "Passionate for justice & the mystery of life & love. Democrat for Hillary (Bernie or Martin)! Animal Lover, Film Buff, Investor/Trader, twitterholic!", "followers_count": 4612, "friends_count": 4870, "statues_count": 82483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631859106824192, "text": "@iamjoeking it's slow but addictive he gets convicted twice along with a nephew the second time.", "in_reply_to_status": 688614156602220545, "in_reply_to_user": 15019575, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15019575 }}, "user": { "id": 3107867400, "name": "NikolettAnnalisePio", "screen_name": "Choctaw_BlueEye", "lang": "en", "location": "from Nor*Cal livin' Tucson ", "create_at": date("2015-03-25"), "description": "just a small town girl, i like to stay fit and be silly every chance i get.", "followers_count": 81, "friends_count": 169, "statues_count": 1293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631859153076224, "text": "Finna start back serving can't wait have to go with plan 3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188184595, "name": "Rich Law", "screen_name": "IvoryJoee", "lang": "en", "location": "null", "create_at": date("2010-09-07"), "description": "#ulmalum #tryingtobegreat #3god rjf staring at the word through my rear view #gocavs #teamjaylancayson", "followers_count": 2984, "friends_count": 1432, "statues_count": 127475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631859299811329, "text": "There's something about you that makes me smile ☺️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540610158, "name": "Alejandra Rodriguez", "screen_name": "JandyyRodriguez", "lang": "en", "location": "Lodi, CA.", "create_at": date("2012-03-29"), "description": "We Left Our FootPrints All Over The World❤️", "followers_count": 148, "friends_count": 46, "statues_count": 157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631859346018304, "text": "Amorciaaa3 ����⛅ https://t.co/8WnaOiVdgR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 1852094994, "name": "Mamacíta", "screen_name": "Amorcia_Marie_", "lang": "en", "location": "Burleson, Tx", "create_at": date("2013-09-10"), "description": "Iesha Amorcia Marie Revees / Queen Style. ❤️\n\n God loves you ❤ \n\n Sc: Amorciaaa3\n Instagram: Amorcia_marie_", "followers_count": 1992, "friends_count": 1474, "statues_count": 18318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burleson, TX", "id": "0157c81133658af3", "name": "Burleson", "place_type": "city", "bounding_box": rectangle("-97.383285,32.470102 -97.222374,32.571893") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48251, "countyName": "Johnson", "cityID": 4811428, "cityName": "Burleson" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631860226854912, "text": "Happy Birthday pretty girl !!���� https://t.co/byh2rorG8D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408951786, "name": "queen A✨", "screen_name": "__gabrielleC", "lang": "en", "location": "somewhere in the sipp", "create_at": date("2011-11-09"), "description": "ig: anasiagc", "followers_count": 950, "friends_count": 311, "statues_count": 29660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulfport, MS", "id": "019e22a02a5e2553", "name": "Gulfport", "place_type": "city", "bounding_box": rectangle("-89.137059,30.351774 -89.001207,30.50028") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2829700, "cityName": "Gulfport" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631860390268929, "text": "#totally80sbar having a great time tonight! https://t.co/PhfARJpqTi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "totally80sbar" }}, "user": { "id": 151630462, "name": "Troy V. Haar", "screen_name": "TvhaarSocialPro", "lang": "en", "location": "Fullerton, OC, So Cal", "create_at": date("2010-06-03"), "description": "null", "followers_count": 266, "friends_count": 2037, "statues_count": 122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631860541394944, "text": "Wind 4.9 mph WNW. Barometer 29.809 in, Rising Rapidly. Temperature 26.4 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 7986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631860822319108, "text": "Re: Yosemite to change iconic names https://t.co/SE2WWODh1O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.59470825,37.74459265"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30087621, "name": "Yosemite NP News", "screen_name": "yosemitenews", "lang": "en", "location": "Yosemite National Park", "create_at": date("2009-04-09"), "description": "Yosemite, the Sierra, National Parks, hiking, sightseeing, talk", "followers_count": 5846, "friends_count": 5713, "statues_count": 74891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6043, "countyName": "Mariposa", "cityID": 686912, "cityName": "Yosemite Valley" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631861086699520, "text": "@Minnesota_Marv NEED IT YES DM IT LMAO", "in_reply_to_status": 688621864114073600, "in_reply_to_user": 327671027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 327671027 }}, "user": { "id": 591218518, "name": "brie", "screen_name": "msssthing", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "21", "followers_count": 1886, "friends_count": 372, "statues_count": 65606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CT", "id": "1cfa9ac27a1e93bd", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-73.502636,41.055206 -73.379243,41.257357") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 955990, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631861413720065, "text": "Who's tryna kick it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1159231122, "name": "T-Rula", "screen_name": "thatkidtyler11", "lang": "en", "location": "your moms bedroom ", "create_at": date("2013-02-07"), "description": "who cares", "followers_count": 723, "friends_count": 452, "statues_count": 8800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631861531267074, "text": "@4thedge_digital thx for the RT!", "in_reply_to_status": 688521423028129793, "in_reply_to_user": 4711082114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4711082114 }}, "user": { "id": 1444589186, "name": "NotCarolyn", "screen_name": "CarlynKB", "lang": "en", "location": "Tampa, FL", "create_at": date("2013-05-20"), "description": "Social Media | Blogger-Photog | Content Writer | Mermaid at heart | Sports Fan | Periscope addict living off Bayshore Boulevard~Tampa~FL #TagTribes", "followers_count": 2591, "friends_count": 2494, "statues_count": 5852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631861917003776, "text": "@DonnieWahlberg @TheAmyCarlson A vision of motion slowed down to a crawl / Me in the center, all suave, in control.", "in_reply_to_status": -1, "in_reply_to_user": 24776235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24776235, 23806195 }}, "user": { "id": 419408343, "name": "Nico", "screen_name": "NicolasErdei", "lang": "en", "location": "United States", "create_at": date("2011-11-23"), "description": "turncoat", "followers_count": 50, "friends_count": 182, "statues_count": 1159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Gatos, CA", "id": "a2de7c70b82b0ca0", "name": "Los Gatos", "place_type": "city", "bounding_box": rectangle("-121.997026,37.205924 -121.905719,37.265466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 644112, "cityName": "Los Gatos" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631862428831744, "text": "Wind 0.0 mph ---. Barometer 29.836 in, Rising. Temperature 32.4F. Rain today 0.00 in. Humidity 90% | Merry Christmas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 581, "friends_count": 778, "statues_count": 35689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631862453927941, "text": "..on a blimp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449432783, "name": "Sarah Williams", "screen_name": "sarahh_williams", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "livin in a gangsta's paradise #TXST19", "followers_count": 572, "friends_count": 841, "statues_count": 14071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-01-17T00:00:04.000Z"), "id": 688631862617452544, "text": "Just posted a video @ The Yost Theater https://t.co/lJusC2aEPf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8652267,33.747509"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14426357, "name": "Slad", "screen_name": "Slad", "lang": "en", "location": "San Diego", "create_at": date("2008-04-17"), "description": "null", "followers_count": 171, "friends_count": 462, "statues_count": 1644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631863179620352, "text": "@kiraaaax3_ I live in Tampa I was only down there to visit", "in_reply_to_status": 688631266128867328, "in_reply_to_user": 3032026294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3032026294 }}, "user": { "id": 382891651, "name": "Princess $ad Shawty", "screen_name": "lovenosluts", "lang": "en", "location": "Tampa, FL Broward CountyLegend", "create_at": date("2011-09-30"), "description": "ugly model", "followers_count": 2464, "friends_count": 973, "statues_count": 26071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631863909421056, "text": "Wind 0.0 mph ---. Barometer 29.703 in, Rising Rapidly. Temperature 24.0 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 3669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864265814016, "text": "Serena Williams and Novak Djokovic are still on top of the world coming into Australian Open https://t.co/ZqrnWlYY4N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.24533538,34.05161048"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55218049, "name": "Los Angeles Press", "screen_name": "LosAngelesCP", "lang": "en", "location": "Los Angeles", "create_at": date("2009-07-09"), "description": "Latest news from Los Angeles. Updates are frequent.", "followers_count": 5819, "friends_count": 21, "statues_count": 257719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864324567041, "text": "When you're having more fun pre gaming as opposed to when you actually get to the bars ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4019572518, "name": "Lizeth Parra", "screen_name": "lizethhpa_", "lang": "en", "location": "null", "create_at": date("2015-10-25"), "description": "null", "followers_count": 76, "friends_count": 78, "statues_count": 721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864605593600, "text": "Sigh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2273950938, "name": "ash", "screen_name": "xoxoberry_", "lang": "en", "location": "HTX ", "create_at": date("2014-01-02"), "description": "im wack. starchild☆. UH16 . @bod_ceej❤", "followers_count": 861, "friends_count": 757, "statues_count": 51359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864727220224, "text": "Temp: 1.1°F - Dew Point: -13.1° - Wind: 14.9 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 29.97in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 11, "friends_count": 11, "statues_count": 12122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864890933248, "text": "Lit https://t.co/K461kiZDjf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 49796736, "name": "Яichie ひ", "screen_name": "RichieTime26", "lang": "en", "location": "Atlanta Adopted Me ✈️404", "create_at": date("2009-06-22"), "description": "Clark Atlanta University Clark Atlanta Football(#26) IG-RichieTime26-Snapchat #AstronautRichie #SpaceCadet #RichieTime #MetroRichie", "followers_count": 3721, "friends_count": 2918, "statues_count": 101119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631864945455105, "text": "You gon play yourself trying to play Me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339187646, "name": "Giuseppe", "screen_name": "J_NUCC", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "null", "followers_count": 468, "friends_count": 345, "statues_count": 15422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hubbard, OH", "id": "21077b8778f1728e", "name": "Hubbard", "place_type": "city", "bounding_box": rectangle("-80.617513,41.133286 -80.540462,41.183958") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3936582, "cityName": "Hubbard" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631865503166464, "text": "About to watch a movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2988447593, "name": "Debo", "screen_name": "jdh____", "lang": "en", "location": "Texas, USA", "create_at": date("2015-01-20"), "description": "well connected . well respected ✊ sc : jakeiraaaa", "followers_count": 213, "friends_count": 156, "statues_count": 2523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631865637490688, "text": "Why am I still up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134972011, "name": "'97 Mia X", "screen_name": "fashionistatutu", "lang": "en", "location": "901 to 423 ", "create_at": date("2010-04-19"), "description": "|special girl, real good girl | YBYSAIA |⛵️|", "followers_count": 2222, "friends_count": 2313, "statues_count": 73422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chattanooga, TN", "id": "3b3916ee31cfc9e6", "name": "Chattanooga", "place_type": "city", "bounding_box": rectangle("-85.404424,34.983674 -85.080704,35.200235") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631866002288640, "text": "@Calilyfe84 suck my fuckin cock.", "in_reply_to_status": 688630356799401984, "in_reply_to_user": 236487902, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236487902 }}, "user": { "id": 1346266394, "name": "Mudstuffin", "screen_name": "SMWLADGBP", "lang": "en", "location": "Redondo Beach, CA", "create_at": date("2013-04-12"), "description": "#Dodgers #Packers #Trojans #Ducks #Lakers Its a joke, not a dick. Dont take it so hard.", "followers_count": 551, "friends_count": 469, "statues_count": 55982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631866346332161, "text": "im so mf tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555578890, "name": "breanna", "screen_name": "bmxlls", "lang": "en", "location": "jwhs", "create_at": date("2012-04-16"), "description": "live for now", "followers_count": 1970, "friends_count": 816, "statues_count": 47771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shorewood, IL", "id": "0a15199a1ef555b1", "name": "Shorewood", "place_type": "city", "bounding_box": rectangle("-88.254057,41.492184 -88.178709,41.545149") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1769758, "cityName": "Shorewood" } }
+{ "create_at": datetime("2016-01-17T00:00:05.000Z"), "id": 688631866426044416, "text": "Not having a title on your relationship makes things so much harder. Cause it's so many things unsaid that need to be said.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1567394551, "name": "Redd ❣", "screen_name": "anicolee_15", "lang": "en", "location": "7️⃣0️⃣4️⃣", "create_at": date("2013-07-03"), "description": "flex cause I finesse || #WSSU19❤️⚪️", "followers_count": 1053, "friends_count": 874, "statues_count": 30961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.933734 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994230262366208, "text": "Wind 3.1 mph N. Barometer 30.21 in, Rising slowly. Temperature -12.6 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994230698668033, "text": "fuck I can't sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336805946, "name": "nicole", "screen_name": "nicolemariiaa", "lang": "en", "location": "Miami, FL", "create_at": date("2014-02-10"), "description": "null", "followers_count": 570, "friends_count": 324, "statues_count": 19283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789749") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231201861632, "text": "Where baee��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1049152070, "name": "Alexis Mason", "screen_name": "AlexisMason15", "lang": "en", "location": "null", "create_at": date("2012-12-30"), "description": "5-22-15❤️", "followers_count": 465, "friends_count": 239, "statues_count": 10395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231210348546, "text": "500 Spin-X Diamond Certified 48x CD-R 80min 700MB Orange Color Top Thermal $109.95 https://t.co/laT8zTLT2f #CDR #Me https://t.co/dAnVly7Mrp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0064354,40.6548713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CDR", "Me" }}, "user": { "id": 2968148254, "name": "CheckOutStore", "screen_name": "checkoutstoretm", "lang": "en", "location": "319 37th St Brooklyn, NY 11232", "create_at": date("2015-01-08"), "description": "Offers CDR, DVDR, Paper Sleeves, DVD Cases, Jewel Cases, Media Supplies and Accessories.", "followers_count": 42, "friends_count": 7, "statues_count": 29538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231344623616, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3090323884, "name": "Rob", "screen_name": "robbb__davies", "lang": "en", "location": "blackwood", "create_at": date("2015-03-12"), "description": "Triton", "followers_count": 398, "friends_count": 240, "statues_count": 2372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwood, NJ", "id": "0036534d6f287a14", "name": "Blackwood", "place_type": "city", "bounding_box": rectangle("-75.088811,39.726424 -74.968468,39.852636") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3406040, "cityName": "Blackwood" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231721955329, "text": "f6c34bfb55aa30aa70b2ebd315713e99d39d6f6ef42b6fd50dbe5b8168925689cd50b952541fbeacb3af604c715db949694bebd267a7dfc08759d9fc29cc01ec9b8962000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.83014241,56.98418459"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 132, "friends_count": 1, "statues_count": 712666 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Peace River B, British Columbia", "id": "062280a72349a97c", "name": "Peace River B", "place_type": "city", "bounding_box": rectangle("-127.739554,55.536535 -120.001407,58.02564") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231722049536, "text": "@Ga1anos Close enough", "in_reply_to_status": 688994017162457088, "in_reply_to_user": 351370999, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351370999 }}, "user": { "id": 138482381, "name": "Tyler Ganza", "screen_name": "Spartan_HCS", "lang": "en", "location": "Toms River, NJ", "create_at": date("2010-04-29"), "description": "Spartan, but you can call me Spartan The Dog ; 21 ; @HCS Pro Halo Player ; Voted Best H2A Breakout Player by the Community ❤️ @shutupjennifer ❤️ #Leftovers", "followers_count": 5120, "friends_count": 1254, "statues_count": 35712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toms River, NJ", "id": "259d4fab72f5d95c", "name": "Toms River", "place_type": "city", "bounding_box": rectangle("-74.269909,39.942803 -74.10616,40.059877") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3473110, "cityName": "Toms River" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994231994597376, "text": "Lmaoooooooooooooi https://t.co/ItD7pWVMsf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 708581916, "name": "Paola M", "screen_name": "paolamaria_", "lang": "en", "location": "null", "create_at": date("2012-07-21"), "description": "it's my pleasure", "followers_count": 1038, "friends_count": 631, "statues_count": 22787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994232636293120, "text": "@amyeskeyy no you live for pickles in a bag and those stupid faces your dog makes", "in_reply_to_status": 688874489661149184, "in_reply_to_user": 53574162, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53574162 }}, "user": { "id": 3304806355, "name": "DANGER", "screen_name": "Ben10OG", "lang": "en", "location": "Hasley Canyon, CA", "create_at": date("2015-08-02"), "description": "don't read my bio it's gay", "followers_count": 190, "friends_count": 180, "statues_count": 1282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hasley Canyon, CA", "id": "009a8ee2004aeb3a", "name": "Hasley Canyon", "place_type": "city", "bounding_box": rectangle("-118.673455,34.456054 -118.641109,34.487978") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632385, "cityName": "Hasley Canyon" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994232808386561, "text": "@GargarTheReal nah", "in_reply_to_status": 688988184433307649, "in_reply_to_user": 35793734, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35793734 }}, "user": { "id": 330536739, "name": "♑️جنان", "screen_name": "Casanij", "lang": "en", "location": "North Jerz/DMV", "create_at": date("2011-07-06"), "description": "Took my daddy away when I was 10 so why would I give a fuck about yall", "followers_count": 4017, "friends_count": 2958, "statues_count": 132419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linden, NJ", "id": "b8d145028b2571ac", "name": "Linden", "place_type": "city", "bounding_box": rectangle("-74.290477,40.593604 -74.200434,40.654943") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3440350, "cityName": "Linden" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994232955092992, "text": "Wind 3.0 mph NNE. Barometer 1035.29 mb, Steady. Temperature 16.5 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233127047169, "text": "@darbypumphrey @morganself13", "in_reply_to_status": 417325338314960896, "in_reply_to_user": 383517928, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 383517928, 341959354 }}, "user": { "id": 446861178, "name": "B-SELF", "screen_name": "self_brandon", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "Student Athlete | Laney Junior College | 2⃣1⃣", "followers_count": 753, "friends_count": 1183, "statues_count": 4231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233307418624, "text": "I hate when people who have my number, message me on snapchat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172439484, "name": "Andrea Argueta", "screen_name": "_andreak8", "lang": "en", "location": "Texasss", "create_at": date("2010-07-29"), "description": "what is coming is better than what has been gone ૐ", "followers_count": 838, "friends_count": 547, "statues_count": 18854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233542295552, "text": "Happy 18th birthday to the most outgoing person I know yasss b finally legal turn up��❤ love you have a good one!!❤❤���� @ItzGomez_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 608712823 }}, "user": { "id": 1065010820, "name": "destiny", "screen_name": "__deestinyy", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "Gotta keep going", "followers_count": 721, "friends_count": 364, "statues_count": 28873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, CA", "id": "6684d4e60ddfc668", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-121.260686,36.307617 -121.226796,36.339441") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 630994, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233546608640, "text": "You can't save the world. Impossible. You can aid progress. But then, what is progress if it doesn't save us?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2235117066, "name": "Joseph Lamar", "screen_name": "JosephLamar111", "lang": "en", "location": "Denver, Colorado", "create_at": date("2013-12-07"), "description": "Musician. Writer. Rocket boy.", "followers_count": 183, "friends_count": 299, "statues_count": 872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233794080769, "text": "Everybody hatin we jus call em fans doe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96899930, "name": "Adonis Creed", "screen_name": "Joshiswavy", "lang": "en", "location": "null", "create_at": date("2009-12-14"), "description": "Tough times don't last but tough ppl do ... Free Brandor Free Adrian,", "followers_count": 1014, "friends_count": 816, "statues_count": 40172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233815027712, "text": "When someone give you hints that they really don't wanna fuck with you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481193238, "name": "Imani.E", "screen_name": "Inamichoom", "lang": "en", "location": "Harlem, NY", "create_at": date("2012-02-02"), "description": "#XO | 23 | Photographer | Boxer | College Girl | Insta: Inamichoom | SC: Manichoom | Tumblr: Inamichoom\n#Treallxrds", "followers_count": 658, "friends_count": 500, "statues_count": 17906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233827524608, "text": "Eww the guy I hate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18840111, "name": "Jay Hernandez", "screen_name": "ITSJROXX", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-01-10"), "description": "snapchat: Itsjroxx insta: Itsjayroxx", "followers_count": 601, "friends_count": 189, "statues_count": 16167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233848430592, "text": "Think I'm addicting to getting naked pictures and sitting and talking about girls that I almost had", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 909211190, "name": "Willie Mays Hayes", "screen_name": "OGdaddyALAN", "lang": "en", "location": "Many, LA", "create_at": date("2012-10-27"), "description": "We are what we repeatedly do and sometimes you just gotta let water roll down your back. #TriAthlete #PositiveSoup #StateChamp sc: alanwarfield", "followers_count": 1019, "friends_count": 551, "statues_count": 47413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Many, LA", "id": "ac870520a83a36e6", "name": "Many", "place_type": "city", "bounding_box": rectangle("-93.494526,31.537623 -93.456199,31.582696") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22085, "countyName": "Sabine", "cityID": 2248470, "cityName": "Many" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994233915580416, "text": "You a Broke bitch I can never be yo ass.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133884250, "name": "Celaa", "screen_name": "Cela_Mendez", "lang": "en", "location": "Houston, TX", "create_at": date("2010-04-16"), "description": "#LifeIsGood (071814--09315) ...", "followers_count": 1230, "friends_count": 1336, "statues_count": 19522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-18T00:00:00.000Z"), "id": 688994234188345344, "text": "Ass ass ass ass ass ass ass https://t.co/wqaZtpfxNd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630949138, "name": "John Girdlestone", "screen_name": "Girdlestonee", "lang": "en", "location": "Lackawanna, New York", "create_at": date("2012-07-09"), "description": "R.I.P Jeffrey James Cintron 5.23.95-4.26.13", "followers_count": 445, "friends_count": 194, "statues_count": 21672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lackawanna, NY", "id": "47275aa57a6ff215", "name": "Lackawanna", "place_type": "city", "bounding_box": rectangle("-78.872789,42.803551 -78.793014,42.834283") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3640189, "cityName": "Lackawanna" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994234267996160, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1646096023, "name": "Symon'e K.", "screen_name": "CHARISMASister", "lang": "en", "location": "null", "create_at": date("2013-08-04"), "description": "Snap: xCharismasister Insta: Charismasister\n#GrindForCharisma ❤", "followers_count": 1383, "friends_count": 776, "statues_count": 21174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994234779631616, "text": "I'm a mileyworld VIP member. What do you have going for you? https://t.co/L7AcHNvTwt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1552366345, "name": "ㅤ", "screen_name": "ThrobbingMasses", "lang": "en", "location": "null", "create_at": date("2013-06-27"), "description": "⠀ ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀ ⠀ ⠀ ⠀⠀ ⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀ ⠀ ⠀ ⠀ ⠀ ⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀ ⠀It would appear I have fallen ⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀ ⠀ ⠀in love with your smile⠀ ⠀⠀ ⠀ ⠀ ⠀⠀ ⠀⠀ ⠀", "followers_count": 1458, "friends_count": 348, "statues_count": 36129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, UT", "id": "0529d30b096121d4", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-112.094327,41.118101 -112.035273,41.152534") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4914290, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994235438116864, "text": "Damn wish my girl stayed longer, I miss her already ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1155845820, "name": "Young Jefe", "screen_name": "Chris_dmndd", "lang": "en", "location": "going places ", "create_at": date("2013-02-06"), "description": "just DM me", "followers_count": 681, "friends_count": 353, "statues_count": 35832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862635") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994235564044288, "text": "03:00:01 |Temp: 30.3ºF | Wind Chill 30.3ºF |Dew Point 26.6ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994235773759488, "text": "@Nazaira_ why ?", "in_reply_to_status": 688993769979555840, "in_reply_to_user": 1667173748, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1667173748 }}, "user": { "id": 624263023, "name": "boss of bosses", "screen_name": "ForeignMamacita", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "K❤️A❤️M❤️E❤️S❤️H❤️A❤️ ' follow my snap { beautiki }", "followers_count": 960, "friends_count": 384, "statues_count": 32429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994235845054464, "text": "01/18@03:00 - Temp 29.3F, WC 23.0F. Wind 6.0mph WNW, Gust 11.0mph. Bar 29.825in, Rising slowly. Rain 0.00in. Hum 78%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994236562321408, "text": "Wind 2.0 mph NW. Barometer 30.455 in, Steady. Temperature 7.0 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994236880953344, "text": "@jaelyn_gryphon sorry I never replied to this >_< Keep it if you want :)", "in_reply_to_status": 687829743283113985, "in_reply_to_user": 461815850, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 461815850 }}, "user": { "id": 3262909291, "name": "Joey", "screen_name": "Joey_Coyote", "lang": "en", "location": "Sparkle City", "create_at": date("2015-06-30"), "description": "25 / Male / Straight / Fursuiter / SFW / OC, CA :3", "followers_count": 269, "friends_count": 217, "statues_count": 520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994236931256320, "text": "Apropos after watching The Big Short. https://t.co/ZwjUHhh3sx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 5725262, "name": "Sean Lynch", "screen_name": "sean_lynch", "lang": "en", "location": "San Francisco", "create_at": date("2007-05-02"), "description": "Product @dropbox. Was founder @tapengage, product @google, student @uwaterloo, and citizen of @canada", "followers_count": 2319, "friends_count": 794, "statues_count": 4215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994236977430528, "text": "My managers already want to make me a crew trainer . ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 827614093, "name": "Diamond", "screen_name": "DiamondRequena", "lang": "en", "location": "bloomington", "create_at": date("2012-09-16"), "description": "BHS| 18 | Softball", "followers_count": 1029, "friends_count": 482, "statues_count": 59841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994237292019713, "text": "This is me everyday. https://t.co/V4LH7kVMUQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 847317050, "name": "Roxy", "screen_name": "xoxo_loveroxy", "lang": "en", "location": "Houston/Dallas/SA TEXAS", "create_at": date("2012-09-26"), "description": "It's all in the tweets.", "followers_count": 904, "friends_count": 793, "statues_count": 54469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacinto City, TX", "id": "c81acc12d1ef4c58", "name": "Jacinto City", "place_type": "city", "bounding_box": rectangle("-95.259478,29.759006 -95.220339,29.77434") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4837156, "cityName": "Jacinto City" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994237472313344, "text": "@ebbtideapp Tide in Peekskill, New York 01/18/2016\nHigh 5:15am 3.0\n Low 12:38pm -0.0\nHigh 5:43pm 2.7\n Low 12:50am -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.9333,41.2833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 1, "friends_count": 0, "statues_count": 89 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Croton-on-Hudson, NY", "id": "01b828da4311466f", "name": "Croton-on-Hudson", "place_type": "city", "bounding_box": rectangle("-73.946553,41.168161 -73.847928,41.334491") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3619213, "cityName": "Croton-on-Hudson" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994237619113984, "text": "Something is going to happen, this is a lot.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3192614035, "name": "Randolph Harris Jr.", "screen_name": "RandolphNews1", "lang": "en", "location": "Sacramento, California", "create_at": date("2015-05-11"), "description": "Married man interested in Architecture. Billionaires Series. Writers famous and Obscure on Love, Sex, Money, Friendship, Family, Work, and much more.", "followers_count": 1227, "friends_count": 1937, "statues_count": 53434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-18T00:00:01.000Z"), "id": 688994237787025408, "text": "#pinkhair\n#brightpink\n#lovethecolor\n#lovebeingme @ Raymond, Washington https://t.co/Q3ceJY6CSK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.738,46.6796"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pinkhair", "brightpink", "lovethecolor", "lovebeingme" }}, "user": { "id": 180226601, "name": "angi bussard-cogdill", "screen_name": "angileanie", "lang": "en", "location": "Raymond, Wa", "create_at": date("2010-08-18"), "description": "I am a mom sister daughter lover. I lost my mom a yr ago to pancreatic cancer. I luv my family 1st & foremost. I am 38yrs young. God has a plan & it is good", "followers_count": 59, "friends_count": 156, "statues_count": 4054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raymond, WA", "id": "e1e8a193cbe79d46", "name": "Raymond", "place_type": "city", "bounding_box": rectangle("-123.778447,46.664637 -123.706452,46.700182") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53049, "countyName": "Pacific", "cityID": 5357430, "cityName": "Raymond" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994238709682176, "text": "DEADMAN WONDERLAND IS SO GOOD THANK YOU @PinXHead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1049897515 }}, "user": { "id": 2556366219, "name": "Valentina Hot Sauce", "screen_name": "bentdicktip", "lang": "en", "location": "null", "create_at": date("2014-05-20"), "description": "| She/Her | North Tejas | ill probably trash on your favorite band and all my friends hate me for that", "followers_count": 267, "friends_count": 62, "statues_count": 36308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125344") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239015862272, "text": "noche de Banda with family �� #zocalo https://t.co/Fa4gY0UVuR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "zocalo" }}, "user": { "id": 2331994918, "name": "moniqueee", "screen_name": "xo_moniqueee", "lang": "en", "location": "null", "create_at": date("2014-02-08"), "description": "I'm way to blessed to be stressed ❤️", "followers_count": 89, "friends_count": 127, "statues_count": 6343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239045349377, "text": "She wants a female\nthat she can make her bitch\nwithout saying that's what she wants.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117485546, "name": "Shalise", "screen_name": "DeletedShalise", "lang": "en", "location": "null", "create_at": date("2010-02-25"), "description": "Twitter bandits robbin me for my beat making money. Used by anonymous beat makers, artists and producers on twitter from 2O11-2014 after posting music and ideas", "followers_count": 1196, "friends_count": 55, "statues_count": 154739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239192010753, "text": "@sharaetiarrah nuh uh", "in_reply_to_status": 688993834194178048, "in_reply_to_user": 2218919971, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 2218919971 }}, "user": { "id": 130613835, "name": "Mike Rager", "screen_name": "MikeRagerxX", "lang": "en", "location": "Mo City, SFA", "create_at": date("2010-04-07"), "description": "General inquiries: highclvssbookings@Gmail.com", "followers_count": 4525, "friends_count": 3022, "statues_count": 61754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239208792064, "text": "New phone who this? Another newer me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332433672, "name": "Joey", "screen_name": "_SlowJoe_", "lang": "en", "location": "Probably at The Block.", "create_at": date("2014-02-07"), "description": "Alaska State tropper", "followers_count": 267, "friends_count": 269, "statues_count": 5499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239263420416, "text": "Travelling to Sioux Falls or just twittering about Sioux Falls? https://t.co/oGWgkqHDpT #Sioux Falls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7003,43.55"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sioux" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1000, "friends_count": 312, "statues_count": 2534191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239305396225, "text": "Temp: 32.1°F Wind:0.0mph Pressure: 30.258hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239368187905, "text": "i spoil babe ❤️ https://t.co/iFUcdBJ7EL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1270494248, "name": "helly", "screen_name": "haleyyy_lynn", "lang": "en", "location": "Prosper, TX Alba, TX", "create_at": date("2013-03-15"), "description": "im a cotton-headed-ninny-muggins", "followers_count": 1334, "friends_count": 1527, "statues_count": 2316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239380770816, "text": "Lituation https://t.co/r293OM0MRu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708986464, "name": "name cannot be blank", "screen_name": "savagerodriguez", "lang": "en", "location": "Bellflower, CA", "create_at": date("2013-08-28"), "description": "$tyle", "followers_count": 362, "friends_count": 439, "statues_count": 13313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239468847104, "text": "Happy birthday to my day1 nigga, have a g one today you deserve it! & Don't get to fucked up stay safe!���� love you kid. @leopimpin_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2314809110 }}, "user": { "id": 924470162, "name": "Lyss♡", "screen_name": "xoxolysssaa", "lang": "en", "location": "null", "create_at": date("2012-11-03"), "description": "null", "followers_count": 708, "friends_count": 364, "statues_count": 29055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239657541632, "text": "Up eating pizza @ 2 in the morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2677897610, "name": "سماوي", "screen_name": "KayKiera", "lang": "en", "location": "limbo .", "create_at": date("2014-07-24"), "description": "I'd rather wear flowers in my head than diamonds around my neck , ☄IG : ki_elysian SNAPCHAT: kebekay | MissingYou @Calidoesit | RIP B.K", "followers_count": 220, "friends_count": 221, "statues_count": 3685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239745675264, "text": "37.3F (Feels: 37.3F) - Humidity: 83% - Wind: 0.0mph --- - Gust: 0.7mph - Pressure: 1037.3mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 222950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994239946997761, "text": "omm ain got time.✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1730430444, "name": "Gιяℓ Tнαтѕ Fιииιє.♿️", "screen_name": "JustCortavious", "lang": "en", "location": "im so 662❗️.☁️", "create_at": date("2013-09-04"), "description": "im just dat guy.✨ | 17 ♏️ bff : @Monyae____ ⚠️ . FINNIE you 2 much ⁉️ ♨️ | ⚫️❤️⚪️ SHS .", "followers_count": 11352, "friends_count": 11088, "statues_count": 47451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shannon, MS", "id": "00563c22310b1683", "name": "Shannon", "place_type": "city", "bounding_box": rectangle("-88.724994,34.111264 -88.71021,34.133254") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2866720, "cityName": "Shannon" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240035188736, "text": "The older I get the more I realize how fake people are", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1419334417, "name": "Canelo", "screen_name": "_Tanner__", "lang": "en", "location": "Corsicana, Texas", "create_at": date("2013-05-10"), "description": "The top gets higher the more that I climb #JeepNation #NavarroCollege #fitness", "followers_count": 959, "friends_count": 768, "statues_count": 19962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corsicana, TX", "id": "faa25283bbfe186a", "name": "Corsicana", "place_type": "city", "bounding_box": rectangle("-96.537914,32.054664 -96.415163,32.131294") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48349, "countyName": "Navarro", "cityID": 4817060, "cityName": "Corsicana" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240144224256, "text": "Wind 2.0 mph WNW. Barometer 30.302 in, Rising slowly. Temperature 7.0 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240286740480, "text": "@RantzBen that # 33 tho��", "in_reply_to_status": 688990205387210752, "in_reply_to_user": 2335997887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2335997887 }}, "user": { "id": 2514262981, "name": "will sorheim", "screen_name": "WSorheim", "lang": "en", "location": "null", "create_at": date("2014-05-21"), "description": "null", "followers_count": 239, "friends_count": 221, "statues_count": 1553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orcutt, CA", "id": "fcfe99a296187c82", "name": "Orcutt", "place_type": "city", "bounding_box": rectangle("-120.468231,34.842218 -120.378247,34.908588") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 654120, "cityName": "Orcutt" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240353988608, "text": "#SupportOriginMelissa 9.3°F Wind:2.2mph Pressure: 30.18hpa Steady Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240442068992, "text": "@McConaughey will always be sexy asf no matter how old he is ������", "in_reply_to_status": -1, "in_reply_to_user": 57097104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57097104 }}, "user": { "id": 42295323, "name": "chels", "screen_name": "chelseaabram", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-05-24"), "description": "740 ➡️ 614 - OSU - Snapchat: chelseaabram", "followers_count": 941, "friends_count": 420, "statues_count": 19297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Lexington, OH", "id": "0c03398f6705d598", "name": "New Lexington", "place_type": "city", "bounding_box": rectangle("-82.227165,39.696058 -82.179885,39.73099") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39127, "countyName": "Perry", "cityID": 3954866, "cityName": "New Lexington" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240639049729, "text": "Sunday night pot roast... https://t.co/MMJFdVVcah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 575963456, "name": "Jenn Barton", "screen_name": "Jujubarton", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-05-09"), "description": "I'm just a girl in the world...", "followers_count": 10, "friends_count": 64, "statues_count": 73 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994240756453376, "text": "@QuannDaDon also!!", "in_reply_to_status": 688994102159921152, "in_reply_to_user": 51140647, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51140647 }}, "user": { "id": 338148106, "name": "Krieglo Ren (11 - 2)", "screen_name": "KriegLaFlare", "lang": "en", "location": "Bay Area ", "create_at": date("2011-07-18"), "description": "Watch ya back. More importantly, when you get out the shower, dry ya back. It's a cold world out there.", "followers_count": 2882, "friends_count": 671, "statues_count": 120813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northwood, San Jose", "id": "71d9663029278997", "name": "Northwood", "place_type": "neighborhood", "bounding_box": rectangle("-121.889202,37.403364 -121.877777,37.41351") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994241163345921, "text": "@maddylicious26 @oneppercent Thanks we came up with them ourselves, Granny Swag ;)", "in_reply_to_status": 688994115741065216, "in_reply_to_user": 574261686, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 574261686, 584872625 }}, "user": { "id": 3068658954, "name": "Paul", "screen_name": "pallavss", "lang": "en", "location": "null", "create_at": date("2015-03-08"), "description": "Ok I'm forsure a dip", "followers_count": 199, "friends_count": 220, "statues_count": 3006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994241528213505, "text": "Just left the club otw to yo house", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3070987759, "name": "الكأس JUGGFEST", "screen_name": "Loudiene", "lang": "en", "location": "Mexico", "create_at": date("2015-03-10"), "description": "@NODSQAUD |BSM| DGB Artist/RSN BookingorFeatures @ItsLilPooh #freebeav & kev soundcloud:Loudiene| IG Loudiene | Sc dirty_soda | support it or get extorted", "followers_count": 2189, "friends_count": 1076, "statues_count": 23349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994241935228928, "text": "Wind 6.0 mph WNW. Barometer 29.66 in, Steady. Temperature 22.5 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 94, "statues_count": 156772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-18T00:00:02.000Z"), "id": 688994242585219072, "text": "Sleepin alone ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340752266, "name": "officialmumm", "screen_name": "_xElizabeth__", "lang": "en", "location": "null", "create_at": date("2011-07-22"), "description": "The Best At It #Jo&Dra❤️", "followers_count": 647, "friends_count": 862, "statues_count": 22556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994242950107136, "text": "We're #hiring! Click to apply: Instructional Designer - https://t.co/QXRYMeMqAf #IT #CranberryTownship, PA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.119951,40.708957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "IT", "CranberryTownship", "Job", "Jobs" }}, "user": { "id": 3431841232, "name": "Jobs at Continuum", "screen_name": "JobsAtContinuum", "lang": "en", "location": "null", "create_at": date("2015-08-19"), "description": "null", "followers_count": 85, "friends_count": 42, "statues_count": 168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranberry, PA", "id": "01b347e54132a78d", "name": "Cranberry", "place_type": "city", "bounding_box": rectangle("-80.160831,40.640453 -79.995749,40.776081") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994243310923777, "text": "Celebrating a birthday! (@ Larry Flynt's Hustler Club - @hustlervegas in Las Vegas, NV) https://t.co/0Q0Sdvp4Jn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.18208027,36.08023391"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168502933 }}, "user": { "id": 27544506, "name": "Leo/Scotty", "screen_name": "LeoXXX", "lang": "en", "location": "Las Vegas", "create_at": date("2009-03-29"), "description": "bodyguard/security/production manager", "followers_count": 2000, "friends_count": 1998, "statues_count": 8614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994243566764033, "text": "Wind 0 mph --. Barometer 30.22 in, Steady. Temperature 34.3 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994243868626946, "text": "@DaJointsKnowMe gotchu", "in_reply_to_status": 688991569962086400, "in_reply_to_user": 3269335176, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3269335176 }}, "user": { "id": 441618126, "name": "jekyll", "screen_name": "kenahoneydip", "lang": "en", "location": "null", "create_at": date("2011-12-20"), "description": "There's ONE me ✨", "followers_count": 1434, "friends_count": 471, "statues_count": 54265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Viejo, CA", "id": "6b55df049258bade", "name": "Mission Viejo", "place_type": "city", "bounding_box": rectangle("-117.695961,33.534941 -117.617598,33.672075") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 648256, "cityName": "Mission Viejo" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994244015603712, "text": "I LOVE YOU SO MUCH I WONDER WHAT HE'LL SAY WHEN I TELL HIM ILL B TRAINING W CODY IN 4 FUCKING MONTHS https://t.co/9RHc8NcKYu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2894174453, "name": "em (1.17.16)", "screen_name": "deanslita", "lang": "en", "location": "b a a v a m ", "create_at": date("2014-11-26"), "description": "wrestling saved my life // dean, lita, jeff, seth, roman, dolph ect // 1.17.16 // “oh hell yeah” // “it was so good meeting you.” // dean hugged me", "followers_count": 1142, "friends_count": 958, "statues_count": 11553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stow, OH", "id": "5920ec228f7cd83f", "name": "Stow", "place_type": "city", "bounding_box": rectangle("-81.491087,41.135566 -81.392049,41.2175") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3974944, "cityName": "Stow" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994244053221376, "text": "@hairdsignsbybre #irunwithscissors lol", "in_reply_to_status": 688825968069234688, "in_reply_to_user": 4774952269, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "irunwithscissors" }}, "user_mentions": {{ 4774952269 }}, "user": { "id": 32684725, "name": "Karen Harold", "screen_name": "karenharold", "lang": "en", "location": "null", "create_at": date("2009-04-17"), "description": "The Conflict Coach, Author, Workplace & Family Mediator, Personal & Professional development trainer, Victim advocate", "followers_count": 173, "friends_count": 147, "statues_count": 1431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994244665593856, "text": "@julie_jay26 @ohdangvicky mmm", "in_reply_to_status": 688991812028006400, "in_reply_to_user": 2214701281, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2214701281, 55710934 }}, "user": { "id": 598967536, "name": "Marcos Godoy", "screen_name": "__mgodoy", "lang": "en", "location": "ucsb. oc ", "create_at": date("2012-06-03"), "description": "don't really know what I'm doing", "followers_count": 367, "friends_count": 322, "statues_count": 31281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994245089325056, "text": "@conteaustin late night thoughts you feel?", "in_reply_to_status": 688994159584215040, "in_reply_to_user": 447728073, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 447728073 }}, "user": { "id": 485071076, "name": "niah ♡", "screen_name": "niah_craig", "lang": "en", "location": "null", "create_at": date("2012-02-06"), "description": "SPC dancer | #25 rest in peace bray | lmhs", "followers_count": 1134, "friends_count": 723, "statues_count": 14651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maineville, OH", "id": "0009156366696ab3", "name": "Maineville", "place_type": "city", "bounding_box": rectangle("-84.258372,39.302829 -84.183837,39.3521") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3946872, "cityName": "Maineville" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994245131276289, "text": "@finesssed deadass I heard its ��", "in_reply_to_status": 688993422330474497, "in_reply_to_user": 2672700388, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2672700388 }}, "user": { "id": 83451003, "name": "HughHeffSteff", "screen_name": "MoneymakinCosta", "lang": "en", "location": "longIsland.", "create_at": date("2009-10-18"), "description": "Degree Chaser", "followers_count": 282, "friends_count": 422, "statues_count": 3472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roosevelt, NY", "id": "5e79986586911926", "name": "Roosevelt", "place_type": "city", "bounding_box": rectangle("-73.600362,40.66521 -73.568557,40.691252") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3663506, "cityName": "Roosevelt" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994245181607936, "text": "It took me a hot minute, but now I'm really feelin the new @TheDailyShow theme music. Plus @Trevornoah is bae, obviously.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 158414847, 46335511 }}, "user": { "id": 2506915454, "name": "Ashley Richmond", "screen_name": "WHBFashley", "lang": "en", "location": "Quad Cities (IL & IA)", "create_at": date("2014-05-19"), "description": "Weekend Anchor at @Local4NewsWHBF, @GreenBayWis native, die-hard @packers fan. RT's ≠ endorsements.", "followers_count": 396, "friends_count": 577, "statues_count": 3015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Island, IL", "id": "4a5bc070c16f39cf", "name": "Rock Island", "place_type": "city", "bounding_box": rectangle("-90.64367,41.415729 -90.531683,41.513115") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1765078, "cityName": "Rock Island" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994245298925569, "text": "Wouldn't have got hung up on if you weren't being so rude ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 620364836, "name": "kalimpura", "screen_name": "kali_warren", "lang": "en", "location": "null", "create_at": date("2012-06-27"), "description": "Chris is my bestie again (:", "followers_count": 341, "friends_count": 427, "statues_count": 10647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994246083227648, "text": "Try playing spoons with @seth_wyrick9 https://t.co/c7b0UQcmTe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2718861683 }}, "user": { "id": 2628405607, "name": "Alex Moore", "screen_name": "_Alex40_", "lang": "en", "location": "California, USA", "create_at": date("2014-07-12"), "description": "my hoe", "followers_count": 351, "friends_count": 339, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-18T00:00:03.000Z"), "id": 688994246456561668, "text": "Never thought I'd be at my lowest point", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144350222, "name": "etzy ♡", "screen_name": "EtzyCabada", "lang": "en", "location": "null", "create_at": date("2010-05-15"), "description": "null", "followers_count": 453, "friends_count": 447, "statues_count": 7400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994247148744704, "text": "I was plotting for awhile now I got him where I want em ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2674544585, "name": "✨", "screen_name": "_xRizglooo", "lang": "en", "location": "love you more kaylana ❤️", "create_at": date("2014-07-04"), "description": "Single ☺️ | GLOyalty.", "followers_count": 1982, "friends_count": 1119, "statues_count": 45947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, VA", "id": "60edfde178b362ff", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-76.420402,36.786146 -76.291434,36.897382") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51740, "countyName": "Portsmouth", "cityID": 5164000, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994247651934208, "text": "Badd https://t.co/xHROgrVRCR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1909419144, "name": "Çhēddår Bøb", "screen_name": "RudeBoy_Jordan", "lang": "en", "location": "Cheddar Cheesin ", "create_at": date("2013-09-26"), "description": "Fuck you pay me ..", "followers_count": 604, "friends_count": 554, "statues_count": 8150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994247672901632, "text": "hahahahahahhahahahahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 633452147, "name": "Michelle Todd", "screen_name": "MichelleTodd07", "lang": "en", "location": "null", "create_at": date("2012-07-11"), "description": "NAU 19 SC: michellelove", "followers_count": 459, "friends_count": 261, "statues_count": 21087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994247832399872, "text": "Recycle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 462138365, "name": "Braedon Mathys", "screen_name": "braedon_mathys", "lang": "en", "location": "Wentzville, Missouri", "create_at": date("2012-01-12"), "description": "Midwest Eagles Elite #7⚾️ Holt Baseball⚾️", "followers_count": 391, "friends_count": 311, "statues_count": 2755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wentzville, MO", "id": "613cd67cf769bce2", "name": "Wentzville", "place_type": "city", "bounding_box": rectangle("-90.92333,38.766988 -90.799897,38.866613") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29183, "countyName": "St. Charles", "cityID": 2978442, "cityName": "Wentzville" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248465747968, "text": "Wind 8.8 mph WNW. Barometer 29.814 in, Rising Rapidly. Temperature 13.1 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248616628224, "text": "@v0mitspitt good ur burning me out to the thottiee", "in_reply_to_status": 688993007169712129, "in_reply_to_user": 2297234516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2297234516 }}, "user": { "id": 615692271, "name": "---", "screen_name": "ShredderEdgar", "lang": "en", "location": "streets", "create_at": date("2012-06-22"), "description": "@v0mitspitt is mami", "followers_count": 413, "friends_count": 324, "statues_count": 38823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248675430402, "text": "Spongebob is the best show to watch while high hands down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1567710397, "name": "Cardi T", "screen_name": "ohtaayyy", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "Baton Rouge ☀️ #UL19", "followers_count": 317, "friends_count": 299, "statues_count": 5716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248717291521, "text": "It sucks when you know you are slowly drifting apart from someone...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 586263447, "name": "GEORGE", "screen_name": "georgieporgie81", "lang": "en", "location": "CAHS Clan", "create_at": date("2012-05-20"), "description": "When in Doubt Whip it Out.", "followers_count": 408, "friends_count": 676, "statues_count": 11312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248717381632, "text": "https://t.co/E2bBK0di1D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 17985, "friends_count": 7541, "statues_count": 3246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248763428865, "text": "So much on my mind ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 905096012, "name": "Serena Dolce", "screen_name": "serena_dolce", "lang": "en", "location": "sacramento, ca", "create_at": date("2012-10-25"), "description": "cant hold me, cant stop me ;)", "followers_count": 623, "friends_count": 554, "statues_count": 8314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florin, CA", "id": "486bc865d347cd73", "name": "Florin", "place_type": "city", "bounding_box": rectangle("-121.445964,38.474314 -121.364939,38.505175") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624498, "cityName": "Florin" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248876789761, "text": "I want oomf and food", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4099495738, "name": "shawtyy", "screen_name": "svgnatyy", "lang": "en", "location": "null", "create_at": date("2015-11-02"), "description": "sda dancer || 1/17 girlgang", "followers_count": 755, "friends_count": 1181, "statues_count": 13953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.654929 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994248897658880, "text": "@xoxoespi ok..... :(", "in_reply_to_status": 688993967451406339, "in_reply_to_user": 53994278, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 53994278 }}, "user": { "id": 593476561, "name": "White Knight", "screen_name": "Isellbabylegs", "lang": "en", "location": "Lancaster, CA", "create_at": date("2012-05-29"), "description": "I'm funny sometimes. @meelllisssaa my suga mama", "followers_count": 260, "friends_count": 149, "statues_count": 48753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994249103273984, "text": "Travelling to Roswell or just twittering about Roswell? https://t.co/xoWM8yYiAI #Roswell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3616,34.0232"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Roswell" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1000, "friends_count": 312, "statues_count": 2534192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roswell, GA", "id": "548a885336da5635", "name": "Roswell", "place_type": "city", "bounding_box": rectangle("-84.419128,33.971999 -84.261548,34.110424") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1367284, "cityName": "Roswell" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994249157693442, "text": "If you don't know bout my kickback you don't follow me on Snapchat yall ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354545071, "name": "YOUNG CEO™", "screen_name": "_ThatBoyEarnest", "lang": "en", "location": "DeadEnd Tx.", "create_at": date("2011-08-13"), "description": "sc: thatboyearnest Houston Tx EazyKill™", "followers_count": 1978, "friends_count": 1389, "statues_count": 43067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994249476542464, "text": "Wind 0.0 mph ---. Barometer 29.584 in, Falling. Temperature 22.4 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994249895854081, "text": "brocabulary https://t.co/XGgGcmz1bx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1042794937, "name": "zac buschy", "screen_name": "z_bizzie", "lang": "en", "location": "loveland", "create_at": date("2012-12-28"), "description": "null", "followers_count": 706, "friends_count": 609, "statues_count": 2403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Loveland, CO", "id": "0fa2e45c48f0ae2a", "name": "Loveland", "place_type": "city", "bounding_box": rectangle("-105.176024,40.352909 -104.973792,40.465838") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 846465, "cityName": "Loveland" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994249950380032, "text": "@chrischuck35 @ejaculattez @50cent @MeekMill yeah these are the girls.", "in_reply_to_status": 688994158095151106, "in_reply_to_user": 1355623009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1355623009, 1496607553, 18222378, 20567939 }}, "user": { "id": 480141312, "name": "Hernani", "screen_name": "Hurtnani", "lang": "en", "location": "Top Kek ", "create_at": date("2012-01-31"), "description": "Broke College Kid™ \n\n#StayFree.", "followers_count": 238, "friends_count": 213, "statues_count": 17576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250042654720, "text": "I'm tired of being a nobody", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 79221610, "name": "Lysergic Tigh", "screen_name": "Ty_TheCre8r02Xx", "lang": "en", "location": "Eugene, OR", "create_at": date("2009-10-02"), "description": "Ty Mannila// 21 // MC // 541 // Mac Millers Less Successful Brother // Tyjmannila@gmail.com for Features, Booking, or to Sell Beats", "followers_count": 1036, "friends_count": 867, "statues_count": 31130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250109800449, "text": "Hollywood.. What's good with this fog? It's scaring me ☹️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18856872, "name": "May Yassine", "screen_name": "may_yassine", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2009-01-10"), "description": "I love sleep, food and Game Of Thrones", "followers_count": 10494, "friends_count": 125, "statues_count": 39603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250109812737, "text": "@AlexatalevAlexa @KeithUrban I love him more than anything in this world", "in_reply_to_status": 688993466399850496, "in_reply_to_user": 2258442060, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2258442060, 14790817 }}, "user": { "id": 3238273871, "name": "TonightIWannaCry", "screen_name": "LuvinMyAussie", "lang": "en", "location": "UrbanLand", "create_at": date("2015-05-05"), "description": "I have been a fan of @keithurban for as long as I can remember. I will do anything for this Blue Eyed Baby. I love him so much....", "followers_count": 505, "friends_count": 712, "statues_count": 6604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250319515652, "text": "@OG_ky32 Lolol u have no sauce but you seem to be lost in the sauce �� #imthesidebae", "in_reply_to_status": 688993638848851968, "in_reply_to_user": 2690734212, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "imthesidebae" }}, "user_mentions": {{ 2690734212 }}, "user": { "id": 29386024, "name": "Michelle Luna", "screen_name": "Mi_owchelle94", "lang": "en", "location": "Where the chips may fall...", "create_at": date("2009-04-06"), "description": "I believe my chihuahua can meow IG: lovely_michelle94", "followers_count": 69, "friends_count": 300, "statues_count": 2725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250478850048, "text": "@chop_sticks98 wow its midnight look at the time I should be sleeping", "in_reply_to_status": 688993915370774528, "in_reply_to_user": 215820749, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 215820749 }}, "user": { "id": 624514026, "name": "stephanie rojas", "screen_name": "stephyy1998", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "I'm living for the best of me", "followers_count": 298, "friends_count": 433, "statues_count": 4642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250554380288, "text": "Wind 0.0 mph ---. Barometer 30.229 in, Rising. Temperature 28.0 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250856333312, "text": "Don't know how I'm still up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388044933, "name": "marcus", "screen_name": "Busboy_marc", "lang": "en", "location": "houston ", "create_at": date("2011-10-09"), "description": "null", "followers_count": 1068, "friends_count": 665, "statues_count": 24893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deer Park, TX", "id": "cc480e2388809012", "name": "Deer Park", "place_type": "city", "bounding_box": rectangle("-95.14691,29.663671 -95.087119,29.751499") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4819624, "cityName": "Deer Park" } }
+{ "create_at": datetime("2016-01-18T00:00:04.000Z"), "id": 688994250961190912, "text": "But this is how I really pose https://t.co/WbJ4ENbhzx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363722014, "name": "marilulu", "screen_name": "mamarralu", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "SD", "followers_count": 181, "friends_count": 362, "statues_count": 2868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-18T00:00:05.000Z"), "id": 688994251066241024, "text": "#followme !! https://t.co/67547OnZXP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "followme" }}, "user": { "id": 563309727, "name": "LexiAreeWaterman", "screen_name": "TheLexiWaters", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2012-04-25"), "description": "Independent mama of 2 beautiful boys ~ #Libra ~ #Artist in every way possible ~ Vine ~ @TheLexiWaters YouTube~ Lexi Aree ~Email- SexyLexis@me.com", "followers_count": 199, "friends_count": 434, "statues_count": 633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mason, OH", "id": "c386e8441572506e", "name": "Mason", "place_type": "city", "bounding_box": rectangle("-84.353179,39.292287 -84.252354,39.399373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3948188, "cityName": "Mason" } }
+{ "create_at": datetime("2016-01-18T00:00:05.000Z"), "id": 688994251141558275, "text": "@PhungVay612 nahhhhh", "in_reply_to_status": 688994064868323329, "in_reply_to_user": 2405230477, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2405230477 }}, "user": { "id": 2388200256, "name": "ⒸⒹ", "screen_name": "Dkooonga_", "lang": "en", "location": "Sparks, NV", "create_at": date("2014-03-13"), "description": "class of 2016", "followers_count": 312, "friends_count": 474, "statues_count": 1870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-01-18T00:00:05.000Z"), "id": 688994251561132032, "text": "Damn ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381336803, "name": "Richie Watson", "screen_name": "Rich_Lyfe23", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "Hiram college football #85, California raised #ATC", "followers_count": 350, "friends_count": 437, "statues_count": 5485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hiram, OH", "id": "01cc4fbe0b4274f7", "name": "Hiram", "place_type": "city", "bounding_box": rectangle("-81.152604,41.290868 -81.107388,41.318022") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39133, "countyName": "Portage", "cityID": 3935658, "cityName": "Hiram" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356618866245632, "text": "I'm lowkey feeling you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228888370, "name": "$UAVE", "screen_name": "TerranceAsson", "lang": "en", "location": "null", "create_at": date("2010-12-20"), "description": "French toast from IHOP", "followers_count": 985, "friends_count": 900, "statues_count": 50259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, TX", "id": "00bfd04e914779f1", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-94.944513,29.997458 -94.862613,30.130802") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48291, "countyName": "Liberty", "cityID": 4819432, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356619407294464, "text": "Hey did anybody c that cavs game?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493889017, "name": "Curt Joe", "screen_name": "FlirtJoe22", "lang": "en", "location": "Bronx, NY", "create_at": date("2012-02-16"), "description": "do not go gentle into that goodnight. rage, rage, rage against the dying of the light", "followers_count": 331, "friends_count": 302, "statues_count": 18375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356619747057664, "text": "@TlFANNYBLEWS holy shit that Twitter account that harassed you is GONE", "in_reply_to_status": -1, "in_reply_to_user": 811559461, "favorite_count": 0, "coordinate": point("-87.84582979,42.2238654"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 811559461 }}, "user": { "id": 3383972031, "name": "Jønathan Flusser, Jr", "screen_name": "osxdude", "lang": "en", "location": "Lake Forest, IL", "create_at": date("2015-07-19"), "description": "21. He/him. Mental illness. Computer whisperer. Too hipster for photo filters.", "followers_count": 269, "friends_count": 851, "statues_count": 4948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, IL", "id": "d50345354159b16c", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-87.942895,42.200288 -87.807195,42.280041") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1741105, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620460093440, "text": "Ive been mentally exhausted for months.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263729667, "name": "matthew", "screen_name": "luxuriousmane", "lang": "en", "location": "somewhere", "create_at": date("2013-03-12"), "description": "| antifa | emo-gang | girl-gang enforcer |", "followers_count": 327, "friends_count": 160, "statues_count": 19172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waconia, MN", "id": "58fe33e6473d1519", "name": "Waconia", "place_type": "city", "bounding_box": rectangle("-93.821513,44.824141 -93.76012,44.862424") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27019, "countyName": "Carver", "cityID": 2767432, "cityName": "Waconia" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620632100864, "text": "RT lol https://t.co/4zmaprkORJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224672103, "name": "feb11th✨", "screen_name": "DEEyoudown", "lang": "en", "location": "being lightskin", "create_at": date("2010-12-09"), "description": "a princess that loves trap and doesn't reply❣", "followers_count": 2982, "friends_count": 388, "statues_count": 312501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New London, CT", "id": "6e9c054da607b539", "name": "New London", "place_type": "city", "bounding_box": rectangle("-72.12852,41.304762 -72.087716,41.385244") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 952280, "cityName": "New London" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620661415936, "text": "Wind 3.0 mph ESE. Barometer 1031.67 mb, Falling. Temperature 25.8 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620833415168, "text": "#recolor #recolorapp #dailydesign #Imagination #readingisfundamental… https://t.co/z5AquanTyX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.885,35.9401"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "recolor", "recolorapp", "dailydesign", "Imagination", "readingisfundamental" }}, "user": { "id": 24844862, "name": "Mark Sielski", "screen_name": "icemark2la", "lang": "en", "location": "Boulder City, NV 89005 ", "create_at": date("2009-03-16"), "description": "Born in Iceland, bred in the world, baked in the Mojave Desert, cultured in LA and now next to Hoover Dam!", "followers_count": 935, "friends_count": 1965, "statues_count": 9586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3206500, "cityName": "Boulder City" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620900503553, "text": "So damn annoyed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2868226331, "name": "Izzy Lopez", "screen_name": "izzy_m23", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "class of 17 ❤️ chloe❤️", "followers_count": 379, "friends_count": 340, "statues_count": 2407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620904669184, "text": "03:00:02 |Temp: 23.8ºF | Wind Chill 23.8ºF |Dew Point 18.2ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356620959240192, "text": "@trumpetshark battery in the fob or battery in the car, physical lock/key aren’t working. :X", "in_reply_to_status": 689354841458630656, "in_reply_to_user": 23881029, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23881029 }}, "user": { "id": 14535893, "name": "furry confusion", "screen_name": "malytwotails", "lang": "en", "location": "Lynnwood, WA", "create_at": date("2008-04-25"), "description": "Rule of Acquisition #190: Hear all, trust nothing. @jondisaurus' wife creature, half of @contactcaffeine & @doomkeep Icon by Astrozerk04, header by Treats", "followers_count": 1253, "friends_count": 490, "statues_count": 62610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SeaTac, WA", "id": "c8b06a459cc8f78a", "name": "SeaTac", "place_type": "city", "bounding_box": rectangle("-122.326102,47.396426 -122.266875,47.48874") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5362288, "cityName": "SeaTac" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356621550612481, "text": "none of my friends are awake ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 825946411, "name": "bAe", "screen_name": "polkadottedhobo", "lang": "en", "location": "not hoesperia", "create_at": date("2012-09-15"), "description": "@esthershaylene1 is my soulmate", "followers_count": 401, "friends_count": 629, "statues_count": 19051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356621915545601, "text": "Just posted a photo @ Future Africa Media. FA Radio & TV. https://t.co/LnVSyM5CU4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2995071,38.5779991"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2963402740, "name": "HRH Prince Duke IV.", "screen_name": "HRHPrinceDuke", "lang": "en", "location": "Creating_1000_Entrepreneurs...", "create_at": date("2015-01-05"), "description": "Social Venture Capital & Network Entrepreneur. Philanthropist. Journalist. Political Analyst. Host. Advocates Leadership, Governance, Empowerment & Development.", "followers_count": 2606, "friends_count": 2865, "statues_count": 11928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherry Hill, VA", "id": "010f72382b29eb22", "name": "Cherry Hill", "place_type": "city", "bounding_box": rectangle("-77.323317,38.529126 -77.256592,38.606572") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5115176, "cityName": "Cherry Hill" } }
+{ "create_at": datetime("2016-01-19T00:00:00.000Z"), "id": 689356622121062400, "text": "@lo_moeller80 @Spicylikesalsaa", "in_reply_to_status": 689356591552991232, "in_reply_to_user": 3998422033, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3998422033, 807577033 }}, "user": { "id": 3998422033, "name": "Lolo", "screen_name": "lo_moeller80", "lang": "en", "location": "null", "create_at": date("2015-10-23"), "description": "Love One Another. ♥", "followers_count": 200, "friends_count": 141, "statues_count": 1932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622230077440, "text": "@JayBeActinUp you on your way to recovery tho. You'll be out their dunking bitches weaves off in no time.", "in_reply_to_status": 689356307917324292, "in_reply_to_user": 189963175, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 189963175 }}, "user": { "id": 81954897, "name": "Beige the Gemini", "screen_name": "SilasTheVifth", "lang": "en", "location": "Chicago, AZ", "create_at": date("2009-10-12"), "description": "Somewhere between a logical writer and a madman with a pen.", "followers_count": 2346, "friends_count": 879, "statues_count": 140241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622334947328, "text": "@LynnPoee ok good now just hurry back boo thang ����������", "in_reply_to_status": 689355051630870528, "in_reply_to_user": 623932153, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 623932153 }}, "user": { "id": 1419347618, "name": "LivvyB33", "screen_name": "livvybailey", "lang": "en", "location": "null", "create_at": date("2013-05-10"), "description": "Florida ✈️ Texas AC hoops IG: livvyb33 • SLP :)", "followers_count": 648, "friends_count": 520, "statues_count": 13306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherman, TX", "id": "caea073452d66ac0", "name": "Sherman", "place_type": "city", "bounding_box": rectangle("-96.66317,33.57578 -96.548448,33.70441") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48181, "countyName": "Grayson", "cityID": 4867496, "cityName": "Sherman" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622481719296, "text": "@3zeZanx @7main_ ������", "in_reply_to_status": 689356458505412608, "in_reply_to_user": 4426456092, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 491672782, 1520681264 }}, "user": { "id": 4426456092, "name": "عويّد.⚑|✈️ US", "screen_name": "AHMAEO", "lang": "en", "location": "المنطجة العاشرةة", "create_at": date("2015-12-01"), "description": "وفَخْري لآلِ هاشِمٍ أُنسَبُ ، بأمانة موسى إبن جعفر. #HalaMadrid اللهم إرحم عزيزة العويد.", "followers_count": 356, "friends_count": 241, "statues_count": 1801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622611779585, "text": "2 a.m to 4 a.m. Twitter be so lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83585251, "name": "Darkskin Drake", "screen_name": "taylorthmz", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-10-19"), "description": "hasan taylor thomas | Photographer", "followers_count": 11590, "friends_count": 1703, "statues_count": 13090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622683074560, "text": "Lmfao thaaanks bby ���� https://t.co/QlzyMPQ9jv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3539441058, "name": "SkinnyBit™", "screen_name": "brookiiee____", "lang": "en", "location": "LA / I.E", "create_at": date("2015-09-12"), "description": "Slim thick witcho cute ass", "followers_count": 565, "friends_count": 517, "statues_count": 8049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622708264962, "text": "When @ImmJustKidding alarm goes off before I'm ready https://t.co/ZSRUVeRE8h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 803806544 }}, "user": { "id": 497667852, "name": "emily myers", "screen_name": "emily_myers128", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "null", "followers_count": 156, "friends_count": 313, "statues_count": 1797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622884425728, "text": "01/19@03:00 - Temp 15.9F, WC 5.9F. Wind 7.2mph W, Gust 16.0mph. Bar 30.178in, Falling slowly. Rain 0.00in. Hum 59%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356622972481539, "text": "It all starts here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2920109612, "name": "Richtopher", "screen_name": "GetRicht", "lang": "en", "location": "null", "create_at": date("2014-12-05"), "description": "MCMXCVII | For the strength of the wolf is the pack, and the strength of the pack is the wolf", "followers_count": 245, "friends_count": 252, "statues_count": 1086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356623278653441, "text": "I'm leaving tomorrow ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1413902138, "name": "Paulina ⚓", "screen_name": "lavirgenloba", "lang": "en", "location": "California, USA", "create_at": date("2013-05-08"), "description": "Sam Smith.", "followers_count": 313, "friends_count": 306, "statues_count": 17020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356623408676865, "text": "@MeShellyShel ������", "in_reply_to_status": 689356543821766656, "in_reply_to_user": 1936357442, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1936357442 }}, "user": { "id": 1468592604, "name": "BDB", "screen_name": "_12Bricks_", "lang": "en", "location": "Belleville✈️East Lansing MSU19", "create_at": date("2013-05-29"), "description": "Can I just drop out please", "followers_count": 536, "friends_count": 348, "statues_count": 7954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356623463256066, "text": "I just fucked your bitch in some Gucci flip flops ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2302358916, "name": "karlssss♥️♠️", "screen_name": "karlamunoz907", "lang": "en", "location": "TX ", "create_at": date("2014-01-20"), "description": "stubborn af with a good heart.|| sc: @k_munoz96 || insta: @kmunoz907__", "followers_count": 710, "friends_count": 403, "statues_count": 16985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Floresville, TX", "id": "3f53ed8eba395810", "name": "Floresville", "place_type": "city", "bounding_box": rectangle("-98.179363,29.115435 -98.135405,29.157209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48493, "countyName": "Wilson", "cityID": 4826160, "cityName": "Floresville" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356623882620928, "text": "@ebbtideapp Tide in Rumford, Rhode Island 01/19/2016\nHigh 3:34am 4.9\n Low 10:24am 0.2\nHigh 4:02pm 4.3\n Low 9:26pm -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.3733,41.84"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 3, "friends_count": 0, "statues_count": 368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356623932977154, "text": "Ayeeee 19 lol ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 166910104, "name": "January 19th!!", "screen_name": "Dylaannnnnnnnnn", "lang": "en", "location": "Portland/Eugene", "create_at": date("2010-07-15"), "description": "my pep talk turn into a pep rally", "followers_count": 473, "friends_count": 451, "statues_count": 10188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356624016838656, "text": "If you only have one feel blessed https://t.co/3OF8z6ZEmb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635187828, "name": "jenny¡", "screen_name": "JustTakeMeHome_", "lang": "en", "location": "Madara's Trap House", "create_at": date("2012-07-13"), "description": "realist // EHS I love strawberry lemonade", "followers_count": 780, "friends_count": 576, "statues_count": 59897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356624092348416, "text": "I'm cool on her i done already had her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2897226789, "name": "hill'n [prod:hillen]", "screen_name": "brobrohillen", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2014-11-29"), "description": "https://soundcloud.com/brobrohillen", "followers_count": 89, "friends_count": 170, "statues_count": 127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356624146882561, "text": "This was literally how me and @Grimesdameron met tbh https://t.co/nhqx7Nl4Wc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2444490948 }}, "user": { "id": 446037325, "name": "✨Saint Ari Dameron✨", "screen_name": "PunchingDemons", "lang": "en", "location": "West New York, NJ", "create_at": date("2011-12-24"), "description": "Ariel/ Ari 22. Writer. Artist. Model. Actress. Ghost Magnet. TWD ✨Nicki Minaj✨I love NormanReedus, SteveYeun, RamiMalek, Oscar Isaac, JamesFranco, &AndyLincoln", "followers_count": 714, "friends_count": 810, "statues_count": 12789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West New York, NJ", "id": "8479859139338fb7", "name": "West New York", "place_type": "city", "bounding_box": rectangle("-74.023591,40.778186 -73.99818,40.796544") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3479610, "cityName": "West New York" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356624608247808, "text": "Wind 0.0 mph ---. Barometer 30.501 in, Steady. Temperature 12.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356625140916224, "text": "She not loyal https://t.co/YQUtvTd1U9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509506933, "name": "Swift▪️", "screen_name": "GTM__E", "lang": "en", "location": "Bread Winner", "create_at": date("2012-02-29"), "description": "Aries RIP-NANA Wpunj⬛️ Kaeyonce Tamer", "followers_count": 941, "friends_count": 768, "statues_count": 42061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356625140932608, "text": "I hope your reading this cause i literally hate you so much haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4129850058, "name": "greg", "screen_name": "Grady_neilan", "lang": "en", "location": "Potty Mouth Posse", "create_at": date("2015-11-04"), "description": "isurf", "followers_count": 268, "friends_count": 195, "statues_count": 1170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channel Islands Beach, CA", "id": "324aaf665da04db7", "name": "Channel Islands Beach", "place_type": "city", "bounding_box": rectangle("-119.2261,34.14615 -119.213099,34.161331") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 612669, "cityName": "Channel Islands Beach" } }
+{ "create_at": datetime("2016-01-19T00:00:01.000Z"), "id": 689356625417785344, "text": "Class starting no earlier than 11 erryday is a blessing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265007945, "name": "Gab", "screen_name": "gmaraz", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2011-03-12"), "description": "Probably sweaty and incoherent, free the nipples", "followers_count": 747, "friends_count": 591, "statues_count": 17174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356626575360001, "text": "I hate when people shovel this early �� inconsiderate ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232023246, "name": "Ms. to You!", "screen_name": "Ambitious_Moni", "lang": "en", "location": "Cleveland/University of Akron", "create_at": date("2010-12-29"), "description": "it could all be so simple.", "followers_count": 486, "friends_count": 414, "statues_count": 28890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356626663469056, "text": "Wind 2.0 mph NW. Barometer 30.468 in, Rising slowly. Temperature 10.2 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 16987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356626902532096, "text": "Things should have been different.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 843935244, "name": "T-A-Y-L-O-R", "screen_name": "TAllen_", "lang": "en", "location": "null", "create_at": date("2012-09-24"), "description": "snap: Applejacks200", "followers_count": 451, "friends_count": 816, "statues_count": 2420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, OR", "id": "3fc1a1035cdd94c8", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-121.229416,44.235383 -121.133527,44.306043") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41017, "countyName": "Deschutes", "cityID": 4161200, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356626902585345, "text": "@Gueraa__ awww ���� orita te mando unas lol", "in_reply_to_status": 689356053151092736, "in_reply_to_user": 742073305, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 742073305 }}, "user": { "id": 2951387856, "name": "Paulina r.", "screen_name": "paulinaaxoxo___", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "19 // sc: lapaulinaaa", "followers_count": 908, "friends_count": 399, "statues_count": 14158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitney, NV", "id": "cb5ac954e9b77fba", "name": "Whitney", "place_type": "city", "bounding_box": rectangle("-115.064712,36.074534 -115.010215,36.137196") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3283800, "cityName": "Whitney" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356627204509696, "text": "Their answers to those questions. https://t.co/44GIaI5VFt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 5820642, "name": "Lee Stranahan", "screen_name": "stranahan", "lang": "en", "location": "Dallas, TX", "create_at": date("2007-05-06"), "description": "Video Producer | Roving reporter at Breitbart News. | Founder, Citizen Journalism School | Host of Brought To Light", "followers_count": 47461, "friends_count": 43653, "statues_count": 185346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356627275862017, "text": "@KianaMarieKiana @WorldStarFunny lmfao !!!!!!!!! I died ������", "in_reply_to_status": 689234653853560832, "in_reply_to_user": 45724975, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45724975, 1107613844 }}, "user": { "id": 167612474, "name": "Brit From Statefarm", "screen_name": "Brie_kansascity", "lang": "en", "location": "KC BABY ", "create_at": date("2010-07-16"), "description": "lifes too short, my life is meant to be lived in Paradise. i need to feel the wind in my hair. & the sweet summer breeze to my face .", "followers_count": 390, "friends_count": 577, "statues_count": 15664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356627594629120, "text": "they gotta get over that. ain't even deep. nothing but a \"gimme a minute\" type of thing. https://t.co/8TXQjfVU8g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4688630628, "name": "josh", "screen_name": "blackboyfIy", "lang": "en", "location": "210 • 506 • 249", "create_at": date("2016-01-01"), "description": "there was something so pleasant about that place, only because @supremerashad was there.", "followers_count": 971, "friends_count": 426, "statues_count": 4223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356627602968577, "text": "HAPPY BIRTHDAY MOM @RaynaArchuleta thanks so much for everything I love you lots even though you're mean sometimes :/ hope ur day is great ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2200038024 }}, "user": { "id": 251365230, "name": "A", "screen_name": "cashmoneyana", "lang": "en", "location": "Odi and I know best", "create_at": date("2011-02-12"), "description": "congratulations you played yourself", "followers_count": 581, "friends_count": 296, "statues_count": 41991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356627741405185, "text": "Temp: 31.2°F Wind:0.0mph Pressure: 30.429hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356628043382785, "text": "21 nga diba remember ? ���� https://t.co/N9ahYO4Ieb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2349212546, "name": "RNNNNMRC", "screen_name": "ArnnieTheGreat", "lang": "en", "location": "Anaheim, CA", "create_at": date("2014-02-17"), "description": "Don't confuse my personality with my attitude My personality is who I am My attitude depends on who you are.", "followers_count": 1229, "friends_count": 264, "statues_count": 11992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356628307677184, "text": "@DeniseStauffer Glad to hear, he's a sweetheart! Looking forward to your role on the show!", "in_reply_to_status": 689297752342302720, "in_reply_to_user": 53766851, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53766851 }}, "user": { "id": 3044928786, "name": "destini | (•͈⌔•͈⑅)♡", "screen_name": "acklesjense", "lang": "en", "location": "Las Vegas", "create_at": date("2015-02-26"), "description": "ジェン様大好きw #TeamAckles | Cons: VegasCon16, DallasCon16 | J2 | SPN | 英語と日本語でおk |", "followers_count": 178, "friends_count": 159, "statues_count": 4113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356628513128449, "text": "@Emazzing where did you go !", "in_reply_to_status": -1, "in_reply_to_user": 23130046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23130046 }}, "user": { "id": 35863758, "name": "Rainna Nguyen", "screen_name": "MakeitRainna", "lang": "en", "location": "Atlanta", "create_at": date("2009-04-27"), "description": "null", "followers_count": 2515, "friends_count": 830, "statues_count": 17867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356629050056705, "text": "38.7F (Feels: 38.7F) - Humidity: 99% - Wind: 2.2mph E - Gust: 3.8mph - Pressure: 1041.1mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 223092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356629087784960, "text": "@lovejesssc ����", "in_reply_to_status": 689356409859891200, "in_reply_to_user": 2790992719, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2790992719 }}, "user": { "id": 317014728, "name": "Andrew.", "screen_name": "andyruiz21", "lang": "en", "location": "california ", "create_at": date("2011-06-14"), "description": "Out here In California my passion is sports! love my teams #mavs#cowboys#sfgiants#halamadrid#longhorns. SPORTS GURU", "followers_count": 383, "friends_count": 962, "statues_count": 28553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coalinga, CA", "id": "385c3a11a1020ac5", "name": "Coalinga", "place_type": "city", "bounding_box": rectangle("-120.375457,36.117735 -120.326632,36.179965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614274, "cityName": "Coalinga" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356629234556929, "text": "Id do the same thing lol fuck alla dat aint no one touchin my lil mama. https://t.co/g78WUvMbTw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593476561, "name": "White Knight", "screen_name": "Isellbabylegs", "lang": "en", "location": "Lancaster, CA", "create_at": date("2012-05-29"), "description": "I'm funny sometimes. @meelllisssaa my suga mama", "followers_count": 260, "friends_count": 149, "statues_count": 48776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356629914042369, "text": "Dreading tryna get this schedule made in the morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322595844, "name": "Jerrell", "screen_name": "MyRetrosBoomin", "lang": "en", "location": "Prairie View, TX", "create_at": date("2011-06-23"), "description": "Thank God & Stay Humble #Pvamu19", "followers_count": 1941, "friends_count": 629, "statues_count": 11755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356629968605184, "text": "This.... This is unacceptable https://t.co/GSiHKXaJGm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493728120, "name": "✨PinkyyKayy✨", "screen_name": "PinkyyKayy", "lang": "en", "location": "starbucks", "create_at": date("2012-02-15"), "description": "D'Arius❤️", "followers_count": 1590, "friends_count": 1564, "statues_count": 38145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356630018887680, "text": "#SupportOriginMelissa 12.4°F Wind:0.0mph Pressure: 30.36hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356630031503361, "text": "Noooo come on ������ https://t.co/7Hv1R5mMsV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732472549, "name": "Max", "screen_name": "maxbaracani", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "$", "followers_count": 705, "friends_count": 465, "statues_count": 15683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-19T00:00:02.000Z"), "id": 689356630123769856, "text": "Honestly give these lyrics a read. @blakeshelton @theweeknd @JLo @jimmyfallon @adamlambert @adamlevine @drdre https://t.co/yDCMlowMq1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14920785, 255388236, 85603854, 15485441, 46576295, 58528137, 28209292 }}, "user": { "id": 2428080694, "name": "Harryhashtagger", "screen_name": "harryhashtagger", "lang": "en", "location": "High Above The Chimney Tops", "create_at": date("2014-03-21"), "description": "I HAVE TO MAKE MY MOM A MILLIONAIRE BEFORE SHE DIES THEN I HAVE TO MAKE HER A BILLIONAIRE BEFORE I DIE\n\nUNSIGNED SONGWRITER", "followers_count": 17635, "friends_count": 15800, "statues_count": 108541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356630811676672, "text": "At the auto show the guy asked us if we wanted a pic with the car and Mike said no we want a pic with you i said mm yes we do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449407481, "name": "alessandra", "screen_name": "AlessandraDuco", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "null", "followers_count": 323, "friends_count": 240, "statues_count": 23950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fraser, MI", "id": "791cba0d58792148", "name": "Fraser", "place_type": "city", "bounding_box": rectangle("-82.970199,42.523595 -82.929046,42.554008") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2630420, "cityName": "Fraser" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356630958436352, "text": "Wind 5.4 mph NW. Barometer 29.89 in, Rising slowly. Temperature 13.3 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 94, "statues_count": 156796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356630987771904, "text": "maybe being selfish is the best way to prevent yourself from getting hurt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3038727624, "name": "Kathleen", "screen_name": "kathuleen14", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-02-23"), "description": "tumblr- westcoastsoulyo", "followers_count": 70, "friends_count": 65, "statues_count": 2208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356631608582144, "text": "Can't wait for @chicagobulls vs @Lakers next Thursday. TNT night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16212685, 20346956 }}, "user": { "id": 218589496, "name": "Ryan Sam", "screen_name": "ryansam06", "lang": "en", "location": "Norcal to Socal ", "create_at": date("2010-11-22"), "description": "If you shoot for the stars and only reach the clouds your still alot higher than everyone else.", "followers_count": 358, "friends_count": 432, "statues_count": 23635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356631772106755, "text": "@MicahBoomin https://t.co/iPQoRXgf17", "in_reply_to_status": -1, "in_reply_to_user": 335687266, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 335687266 }}, "user": { "id": 359009813, "name": "YZYxSN", "screen_name": "heatherdiaz_", "lang": "en", "location": "inglewood•flagstaff", "create_at": date("2011-08-20"), "description": "Miley, what's good? add me on snap: heathaabitch #BlackLivesMatter NAU '19 woke carefree black girl", "followers_count": 1005, "friends_count": 525, "statues_count": 19025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356631994425345, "text": "Wind 0 mph --. Barometer 30.40 in, Falling slowly. Temperature 30.0 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356632279650304, "text": "Not a #yaesu, but my Kenwood TS-830S does have CAT control. #hamradio @ New Iberia, Louisianna https://t.co/G1iZ5X73yN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.8262482,29.9957504"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "yaesu", "hamradio" }}, "user": { "id": 2320903821, "name": "Paul Kleinpeter", "screen_name": "kp8403", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "null", "followers_count": 158, "friends_count": 573, "statues_count": 2315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Iberia, LA", "id": "7d8e6592712895b9", "name": "New Iberia", "place_type": "city", "bounding_box": rectangle("-91.892077,29.941102 -91.731432,30.04583") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22045, "countyName": "Iberia", "cityID": 2254035, "cityName": "New Iberia" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356632690712576, "text": "Happy birthday to my now 13 year old #FIRST @jalenx0x0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FIRST" }}, "user_mentions": {{ 1552251018 }}, "user": { "id": 255309075, "name": "Colton", "screen_name": "guywithalife", "lang": "en", "location": "null", "create_at": date("2011-02-20"), "description": "Calgary ➡ Spokane || Uncle J", "followers_count": 129, "friends_count": 100, "statues_count": 532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356632728440833, "text": "2 in the morning my mind is on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1476363073, "name": "RUBY", "screen_name": "rubyher34", "lang": "en", "location": "Texas, USA", "create_at": date("2013-06-01"), "description": "smith soccer•19•", "followers_count": 1104, "friends_count": 1092, "statues_count": 28305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356633202388993, "text": "Wind 0.0 mph ---. Barometer 30.352 in, Falling. Temperature 28.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356633407934464, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2448097124, "name": "bbb", "screen_name": "BigBootyBella00", "lang": "en", "location": "San Diego, CA✈️Caldwell, ID", "create_at": date("2014-04-16"), "description": "softball is life⚾️⚾️", "followers_count": 229, "friends_count": 94, "statues_count": 5780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Caldwell, ID", "id": "7d563a4149988303", "name": "Caldwell", "place_type": "city", "bounding_box": rectangle("-116.713205,43.590179 -116.593171,43.69763") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16027, "countyName": "Canyon", "cityID": 1612250, "cityName": "Caldwell" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634070650881, "text": "@SucioTM thanks for the text back hoe bag", "in_reply_to_status": 689356311918690305, "in_reply_to_user": 377709418, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377709418 }}, "user": { "id": 165976447, "name": "Plurmander", "screen_name": "_carlitosguey", "lang": "en", "location": "ℓσѕ αиgєℓєѕ", "create_at": date("2010-07-12"), "description": "Hey @ohram_! meet me at the Allen... it's going down.", "followers_count": 587, "friends_count": 497, "statues_count": 64944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634125111297, "text": "I'm so happy w how life is right now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450478895, "name": "jul", "screen_name": "juliannafraeli1", "lang": "en", "location": "null", "create_at": date("2011-12-30"), "description": "killer and a sweet thang", "followers_count": 447, "friends_count": 523, "statues_count": 14847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Segundo, CA", "id": "0654b676d0359a31", "name": "El Segundo", "place_type": "city", "bounding_box": rectangle("-118.429843,33.901804 -118.370685,33.931493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622412, "cityName": "El Segundo" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634129358848, "text": "���� https://t.co/bOQOQ15GuK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 345806282, "name": "your fave❤️", "screen_name": "ShesSpunky_", "lang": "en", "location": "null", "create_at": date("2011-07-30"), "description": "you lose| #LSU20", "followers_count": 2243, "friends_count": 885, "statues_count": 57254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634171281408, "text": "@Yo_Its_Carlos_ he out there smoking now with them and shit", "in_reply_to_status": 689356489799176192, "in_reply_to_user": 589692666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 589692666 }}, "user": { "id": 767072353, "name": "Andrew ✈", "screen_name": "SrDeLos_Cielos", "lang": "en", "location": "salinas cali ☀", "create_at": date("2012-08-18"), "description": "Red blooded ❤ golden hearted", "followers_count": 416, "friends_count": 486, "statues_count": 35062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634288742400, "text": "#Democrats and #Republicans sound like they're running for #President on different countries. https://t.co/0ZegeHm3n8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Democrats", "Republicans", "President" }}, "user": { "id": 173699976, "name": "J", "screen_name": "JrrdWllms", "lang": "en", "location": "Beijing", "create_at": date("2010-08-01"), "description": "Quasi-Chef. Hip-Hop Aficionado. Japanese Teen Idol. @thon. \n\nRTs ≠endorsements.", "followers_count": 385, "friends_count": 587, "statues_count": 5130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634477457408, "text": "�� just sick lol https://t.co/OuX3NTtmWK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4579495873, "name": "Struggle Wilson", "screen_name": "iAintShit_FP", "lang": "en", "location": "206253 Washington, USA", "create_at": date("2015-12-16"), "description": "Went back inside my attic, Counted up & started laughing LiveFRESH PRESERVELife", "followers_count": 128, "friends_count": 232, "statues_count": 1022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634578092036, "text": "I've took more L's this year than last", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 570529372, "name": "Cookie B The G.O.A.T", "screen_name": "BigPimpin_97", "lang": "en", "location": "Ohio✈️Arkansas", "create_at": date("2012-05-03"), "description": "Professional Pussy Eater #UAPB19 ΡΦΗ", "followers_count": 1606, "friends_count": 1455, "statues_count": 111754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Bluff, AR", "id": "4fa26750e4d5620c", "name": "Pine Bluff", "place_type": "city", "bounding_box": rectangle("-92.092994,34.140549 -91.94191,34.268184") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5069, "countyName": "Jefferson", "cityID": 555310, "cityName": "Pine Bluff" } }
+{ "create_at": datetime("2016-01-19T00:00:03.000Z"), "id": 689356634645225472, "text": "Get the hint alreadyyyyy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4169519055, "name": "Devin Marino", "screen_name": "devvvmarino", "lang": "en", "location": "Bay Area", "create_at": date("2015-11-11"), "description": "blssed", "followers_count": 111, "friends_count": 87, "statues_count": 216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, CA", "id": "41c331abe42d9969", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-117.287533,34.413411 -117.103437,34.571843") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 602364, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356634901061633, "text": "They pulled us in VIP it's over ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 82977170, "name": "Golden ✨", "screen_name": "Toohighsadiddy", "lang": "en", "location": "left side of your chest ", "create_at": date("2009-10-16"), "description": "Joint!", "followers_count": 467, "friends_count": 439, "statues_count": 22777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356635039518720, "text": "I need friends that will start saying bible with me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603366870, "name": "JJ", "screen_name": "jazzyjazz017", "lang": "en", "location": "Austin, TX", "create_at": date("2012-06-08"), "description": "null", "followers_count": 675, "friends_count": 217, "statues_count": 30800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356635299581952, "text": "Team \"I fangirl hardest when I'm suppressing real life emotions\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32963732, "name": "Ashley Nicole Garden", "screen_name": "Whoatisashley", "lang": "en", "location": "Castro Valley, California", "create_at": date("2009-04-18"), "description": "23 • Twitter is my fangirl outlet • I write like my life depends on it • not any good at it • 6-6-15 was a good day for marriage", "followers_count": 389, "friends_count": 486, "statues_count": 24717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356635303763969, "text": "My Morphe 35O pallet comes tomorrow ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43811593, "name": "jasssss", "screen_name": "jasminandersen", "lang": "en", "location": "las vegas, nv ✈️ price, ut", "create_at": date("2009-05-31"), "description": "5'2 with an attitude of 6'1", "followers_count": 1530, "friends_count": 636, "statues_count": 98774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Price, UT", "id": "48e6d5f289258790", "name": "Price", "place_type": "city", "bounding_box": rectangle("-110.829054,39.575375 -110.775077,39.622855") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49007, "countyName": "Carbon", "cityID": 4962030, "cityName": "Price" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356635786051584, "text": "YRN2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 318608126, "name": "Amilcar Canas", "screen_name": "_Amillii", "lang": "en", "location": "Roll Tide ", "create_at": date("2011-06-16"), "description": "somewhere between I want it and I got it.", "followers_count": 567, "friends_count": 197, "statues_count": 20723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pelham, AL", "id": "01e806152ed977eb", "name": "Pelham", "place_type": "city", "bounding_box": rectangle("-86.832342,33.243319 -86.72688,33.352736") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1117, "countyName": "Shelby", "cityID": 158848, "cityName": "Pelham" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356636008407040, "text": "Goodnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3269332062, "name": "•Yung Queen•", "screen_name": "DevineVigil", "lang": "en", "location": "null", "create_at": date("2015-07-05"), "description": "I'm a lonely cloud ☁️ 話", "followers_count": 416, "friends_count": 536, "statues_count": 3140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Salt Lake, UT", "id": "da05c7c645de201d", "name": "South Salt Lake", "place_type": "city", "bounding_box": rectangle("-111.928394,40.686417 -111.871215,40.725919") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4971070, "cityName": "South Salt Lake" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356636801077250, "text": "https://t.co/aPOTuYbXMs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 18244, "friends_count": 8532, "statues_count": 3662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637035970561, "text": "At some point, you just gotta boss up, like deadass ... On your own.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34363763, "name": "carly.", "screen_name": "eyeSaidno", "lang": "en", "location": "Michigan • New Orleans, LA", "create_at": date("2009-04-22"), "description": "gsu graduate. selectively social.", "followers_count": 676, "friends_count": 227, "statues_count": 41453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Timberlane, LA", "id": "6e137e292946d911", "name": "Timberlane", "place_type": "city", "bounding_box": rectangle("-90.044486,29.8481 -90.01741,29.898203") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2275740, "cityName": "Timberlane" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637233139713, "text": "As a boyfriend or girlfriend it's your job to do whatever bae is doing, even if its boring or stupid. If they like it then you like it too.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2232062102, "name": "isaias rodriguez", "screen_name": "biggpurro47", "lang": "en", "location": "bootyville usa", "create_at": date("2013-12-05"), "description": "solo do lo\ngood vibes plez\nim nice", "followers_count": 81, "friends_count": 172, "statues_count": 2047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tomball, TX", "id": "00028b6863e4a97b", "name": "Tomball", "place_type": "city", "bounding_box": rectangle("-95.662144,30.066754 -95.576814,30.135969") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4873316, "cityName": "Tomball" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637275082754, "text": "@yazzel_ yes", "in_reply_to_status": 689330638189531136, "in_reply_to_user": 64627633, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 64627633 }}, "user": { "id": 587927064, "name": "Victor Beber", "screen_name": "Sliceof_Beber", "lang": "en", "location": "Victorville, CA", "create_at": date("2012-05-22"), "description": "Holding off for LCD soundsystem", "followers_count": 142, "friends_count": 73, "statues_count": 16706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637505753088, "text": "Setting up appointment with a financial consultant & real estate agent �� the hunt for a crib in Hollywood begins ✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4381860554, "name": "Abigail Franco", "screen_name": "MissMufasaaa", "lang": "en", "location": "null", "create_at": date("2015-11-27"), "description": "I'm faded and hated. WHAT THE FUCK YOU GON DO BOUT IT.......MAFIO$A", "followers_count": 286, "friends_count": 431, "statues_count": 843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brea, CA", "id": "341aa0cdb944022e", "name": "Brea", "place_type": "city", "bounding_box": rectangle("-117.92871,33.905579 -117.800848,33.947725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608100, "cityName": "Brea" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637673525248, "text": "@CUTEST_ANlMALS @BestGalaxyPics @modernmuva @stephadome", "in_reply_to_status": 689336376278519809, "in_reply_to_user": 2610938700, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2610938700, 2746940134, 343048999, 3106260073 }}, "user": { "id": 4340532252, "name": "Татьяна", "screen_name": "tatyana_alexia", "lang": "en", "location": "Los Angeles ", "create_at": date("2015-11-23"), "description": "RUŠŠiA|ATL||LA", "followers_count": 46, "friends_count": 179, "statues_count": 210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redondo Beach, CA", "id": "e1e35d357ceefa52", "name": "Redondo Beach", "place_type": "city", "bounding_box": rectangle("-118.401931,33.814689 -118.352695,33.894649") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 660018, "cityName": "Redondo Beach" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356637774204929, "text": "I'm almost out of food and I don't get paid till next Friday... Jesus help me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422794638, "name": "Shaun ॐ", "screen_name": "ShaunXmods", "lang": "en", "location": "Blowing Dank Clouds", "create_at": date("2011-11-27"), "description": "your local vape god | 580 | Snapchat @ shaunbutcher", "followers_count": 719, "friends_count": 584, "statues_count": 36004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ardmore, OK", "id": "2ffc75cc0e452636", "name": "Ardmore", "place_type": "city", "bounding_box": rectangle("-97.178022,34.129246 -97.101508,34.219901") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40019, "countyName": "Carter", "cityID": 4002600, "cityName": "Ardmore" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356638042603520, "text": "Someone at the door �� it's 2am , who comes ringing doorbells at 2am . I'm not getting up yo ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 582368997, "name": "BabyGirl", "screen_name": "therealteawna_", "lang": "en", "location": "404 ✈ 402", "create_at": date("2012-05-16"), "description": "| Snapchat : xoxo_teawna | IG : Hennessy.Tea | Libra | RIP WILL ❤️", "followers_count": 6435, "friends_count": 6313, "statues_count": 57815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356638428491777, "text": "@baileyaloni oh sorry, I moved tepees", "in_reply_to_status": 689356449433141248, "in_reply_to_user": 1481264089, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1481264089 }}, "user": { "id": 208289371, "name": "Nik Dee", "screen_name": "NikkDeee", "lang": "en", "location": "null", "create_at": date("2010-10-26"), "description": "Senior at Lakeridge 5⃣1⃣ Go Gators! Lakeridge Football, Wrestling, and Lax | Captain | Bailey is cool I guess but not really", "followers_count": 167, "friends_count": 218, "statues_count": 668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Oswego, OR", "id": "386b4dc0b52f8b48", "name": "Lake Oswego", "place_type": "city", "bounding_box": rectangle("-122.750195,45.382216 -122.642587,45.44028") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4140550, "cityName": "Lake Oswego" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356638537519104, "text": "Wind 3.9 mph WNW. Barometer 29.799 in, Steady. Temperature 6.2 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356638860505088, "text": "@niallthejedi do it", "in_reply_to_status": 689354525547732992, "in_reply_to_user": 1173616220, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1173616220 }}, "user": { "id": 3145069518, "name": "sara", "screen_name": "lourrie28", "lang": "en", "location": "louis' right armpit ", "create_at": date("2015-04-08"), "description": "read pinned tweet // I'm very friendly. let's talk! :)", "followers_count": 1644, "friends_count": 116, "statues_count": 24488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-19T00:00:04.000Z"), "id": 689356638873112578, "text": "https://t.co/otU4MXldgO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2335708051, "name": "j.", "screen_name": "jenloveeee", "lang": "en", "location": "null", "create_at": date("2014-02-09"), "description": "ain't no tellin'", "followers_count": 454, "friends_count": 348, "statues_count": 14273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356639590309888, "text": "Wind 2 mph NNE. Barometer 30.19 in, Rising. Temperature 43.2 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 52, "friends_count": 69, "statues_count": 25079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356639925833728, "text": "On top of everything, got into the elevator at home & had an #asthma attack cuz it reeked of pot. Had to take the stairs. #healthissues", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.32867883,47.69787868"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "asthma", "healthissues" }}, "user": { "id": 65540239, "name": "Rachel Araucto", "screen_name": "rachelaraucto", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-08-13"), "description": "Actor, singer, dancer, voiceover artist. Director, music director, choreographer, stage manager. I also crochet dishcloths. Website coming soon!", "followers_count": 143, "friends_count": 199, "statues_count": 9206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356640412397568, "text": "Dam �� https://t.co/mK7Y6BcC1F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1405854480, "name": "N.♑️✨", "screen_name": "ColorMeNaay_", "lang": "en", "location": "Bay Area ✈️", "create_at": date("2013-05-05"), "description": "Poppppppinnnn", "followers_count": 225, "friends_count": 173, "statues_count": 9194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Pablo, CA", "id": "04016a4890553832", "name": "San Pablo", "place_type": "city", "bounding_box": rectangle("-122.358758,37.948681 -122.318412,37.981558") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 668294, "cityName": "San Pablo" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356640588599296, "text": "@ShakeYoHiney wassup", "in_reply_to_status": 689332570392113152, "in_reply_to_user": 3097577559, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3097577559 }}, "user": { "id": 858835681, "name": "Sharkbait .", "screen_name": "kaylalaMC_95", "lang": "en", "location": "Merica", "create_at": date("2012-10-02"), "description": "20. Being a Crazy cat lady with Skylar since 2009 . I make coffee for caffeine addicts ☕️ My man is a handsome and goofy US Soldier", "followers_count": 446, "friends_count": 478, "statues_count": 9858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Artondale, WA", "id": "17f4895724c18104", "name": "Artondale", "place_type": "city", "bounding_box": rectangle("-122.697276,47.269853 -122.589691,47.331877") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5302910, "cityName": "Artondale" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356640693391361, "text": "The Weakness In Me- Joan Armatrading \n\nACCURATE https://t.co/yaG9EQ6qn5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169716636, "name": "Ana", "screen_name": "AnaJuarez_", "lang": "en", "location": "336/704", "create_at": date("2010-07-22"), "description": "idk what I'm doing. stay tuned", "followers_count": 253, "friends_count": 274, "statues_count": 3255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356640773083136, "text": "FUCK YOU !!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716660615, "name": "Shaunté", "screen_name": "Keiraa_Shaunte_", "lang": "en", "location": "MD", "create_at": date("2013-10-15"), "description": "null", "followers_count": 1432, "friends_count": 1135, "statues_count": 18122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robinwood, MD", "id": "554606b8899fffeb", "name": "Robinwood", "place_type": "city", "bounding_box": rectangle("-77.691532,39.610011 -77.636628,39.651649") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24043, "countyName": "Washington", "cityID": 2467000, "cityName": "Robinwood" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356640886329344, "text": "Remy relationship is goals I never seen a man hold somebody down while their behind bars especially when it's the other way around ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247044623, "name": "maitex3", "screen_name": "TrueGangstaX3", "lang": "en", "location": "Boston, MA", "create_at": date("2011-02-03"), "description": "Dominican princess", "followers_count": 1379, "friends_count": 1659, "statues_count": 58015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356641532252161, "text": "TIL Japanese has a word that means \"the room where it happens\" https://t.co/Rzo1T6UrCc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148066896, "name": "S.I. Rosenbaum", "screen_name": "sirosenbaum", "lang": "en", "location": "null", "create_at": date("2010-05-25"), "description": "The atlas-eater with a jaw for news :: formerly Boston magazine, Boston Phoenix, Tampa Bay Times :: Hire me!", "followers_count": 2850, "friends_count": 1546, "statues_count": 72250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356641628770305, "text": "Shleeeep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419838081, "name": "mari✂️", "screen_name": "maritssaaa", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "blessed", "followers_count": 1072, "friends_count": 688, "statues_count": 42383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-19T00:00:05.000Z"), "id": 689356641788100610, "text": "Take me to New York", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1302018320, "name": "Blanca Nieves", "screen_name": "_babyducks", "lang": "en", "location": "null", "create_at": date("2013-03-25"), "description": "Heart of the truest believer❤️", "followers_count": 333, "friends_count": 330, "statues_count": 54664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719005855100928, "text": "#5wordconspiracies Everyone actually likes Frozen pizza. https://t.co/WSvILhyW0Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "5wordconspiracies" }}, "user": { "id": 3107740801, "name": "The Soapbox", "screen_name": "soapboxcomedy", "lang": "en", "location": "Utah, USA", "create_at": date("2015-03-25"), "description": "null", "followers_count": 60, "friends_count": 287, "statues_count": 399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719006102683648, "text": "I'm like so depressed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224429437, "name": "R.O", "screen_name": "7jeanjunkieRO", "lang": "en", "location": "null", "create_at": date("2010-12-08"), "description": "GOTTI &CRAKK", "followers_count": 406, "friends_count": 364, "statues_count": 35904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719006241026048, "text": "\"Musicians don't retire; they stop \nwhen there's no more music in them\"\n\n- Louis Armstrong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60719503, "name": "Roger Rocka's", "screen_name": "RogerRockas", "lang": "en", "location": "Fresno, CA (Tower District)", "create_at": date("2009-07-27"), "description": "THE Dinner Theater In Fresno, CA in the Tower District 559-266-9494 http://www.RogerRockas.com - http://www.GCPlayers.com - http://www.Facebook.com/rogerrockas", "followers_count": 6053, "friends_count": 5683, "statues_count": 13914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719006413111296, "text": "\"Summer want us to mention Her\" man what????¿¿ https://t.co/urlkaEvH6F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421203844, "name": "JAN 26, '97", "screen_name": "_summerwi", "lang": "en", "location": "out the way ", "create_at": date("2011-11-25"), "description": "feeling better. loving better.", "followers_count": 1196, "friends_count": 1642, "statues_count": 134344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lima, OH", "id": "6b1aa33507f2e472", "name": "Lima", "place_type": "city", "bounding_box": rectangle("-84.215102,40.687562 -84.012941,40.817349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39003, "countyName": "Allen", "cityID": 3943554, "cityName": "Lima" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719006832435205, "text": "Its time for some new gym shoes...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715039928, "name": "Im_Blessed", "screen_name": "IAmJustAVessel", "lang": "en", "location": "null", "create_at": date("2012-07-24"), "description": "God is love", "followers_count": 78, "friends_count": 214, "statues_count": 2041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Palo Alto, CA", "id": "b9da7881046990df", "name": "East Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.156222,37.451829 -122.115942,37.484169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 620956, "cityName": "East Palo Alto" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719007428014080, "text": "and me as your bride �� https://t.co/V9hGakIvis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292059602, "name": "JOY!", "screen_name": "odetojoyyy", "lang": "en", "location": "Views From The 6", "create_at": date("2011-05-02"), "description": "I've always been feeling like you were the piece to complete me, @punkcee❤", "followers_count": 514, "friends_count": 334, "statues_count": 94893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719008086523904, "text": "I'm high af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341938932, "name": "♨️Mr. Lit♨️", "screen_name": "Dat_Nigga_Smoka", "lang": "en", "location": "Lil Rock , Ar", "create_at": date("2011-07-24"), "description": "#Free Gama #Free Swag #Rip Hardhead #FreeRashad #BrotherKeeper ♠️", "followers_count": 1123, "friends_count": 491, "statues_count": 38053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719009030201344, "text": "@garynavajr I'm fuckin dead ��", "in_reply_to_status": -1, "in_reply_to_user": 4676266112, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4676266112 }}, "user": { "id": 3976332854, "name": "Alicia Renteria", "screen_name": "aliciarenteriaa", "lang": "en", "location": "California, USA", "create_at": date("2015-10-21"), "description": "Whatever forever", "followers_count": 51, "friends_count": 102, "statues_count": 947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719009760047104, "text": "We do! ���� https://t.co/Lh4aOQrlhX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1401327829, "name": "Luis Carlos✌️", "screen_name": "NavyyBoyy", "lang": "en", "location": "null", "create_at": date("2013-05-03"), "description": "GOOD VIBES & BIG THIGHS✨ | BASEBALL⚾️", "followers_count": 1049, "friends_count": 636, "statues_count": 43284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-01-20T00:00:00.000Z"), "id": 689719009802002432, "text": "The saddest part of Race Relations in America is that they are the best that they have ever been! #FuckTrump", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FuckTrump" }}, "user": { "id": 1385030792, "name": "Ivan Martinez", "screen_name": "imart012", "lang": "en", "location": "null", "create_at": date("2013-04-27"), "description": "null", "followers_count": 93, "friends_count": 1998, "statues_count": 193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719010196393984, "text": "Petty..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47020857, "name": "♛ᴅiᴋᴇɴʙʀᴇᴇ♛", "screen_name": "photogenic_kenn", "lang": "en", "location": "Atlanta Georgia ", "create_at": date("2009-06-13"), "description": "ᴛʜɪs ᴀɪɴᴛ ᴘʜᴀᴇᴅʀᴀ ʙɪᴛᴄʜ..", "followers_count": 2484, "friends_count": 1437, "statues_count": 77121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lithia Springs, GA", "id": "00c4104a33f2c868", "name": "Lithia Springs", "place_type": "city", "bounding_box": rectangle("-84.696305,33.753937 -84.593547,33.805711") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13097, "countyName": "Douglas", "cityID": 1346832, "cityName": "Lithia Springs" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719010313646081, "text": "Being a better me for me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64345337, "name": "Sal", "screen_name": "EffSall", "lang": "en", "location": "Chico, CA", "create_at": date("2009-08-09"), "description": "Do not fear death but rather the unlived life", "followers_count": 417, "friends_count": 295, "statues_count": 21189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719010326282240, "text": "@ebbtideapp Tide in Alcatraz Island, California 01/20/2016\n Low 1:59am 2.4\nHigh 8:22am 6.6\n Low 3:21pm -0.5\nHigh 10:19pm 4.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.4167,37.8333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 3, "friends_count": 1, "statues_count": 632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719010464829440, "text": "01/20@03:00 - Temp 21.7F, WC 21.7F. Wind 2.3mph W, Gust 7.0mph. Bar 30.320in, Rising slowly. Rain 0.00in. Hum 66%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719010884091904, "text": "Was a good day. I got most of the tile done at my new house. Progress was made on the deck, thanks to Larry and Adam.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519680382, "name": "Gabriel Henton", "screen_name": "GabrielHenton", "lang": "en", "location": "Beaverton, OR", "create_at": date("2012-03-09"), "description": "call or text 541-968-8022! Born in Oregon, finished college in TN, got married in Russia. IT consultant, entrepreneur, investor, thinker, writer, singer.", "followers_count": 1141, "friends_count": 1007, "statues_count": 308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Hills, OR", "id": "7116080c8926fb6b", "name": "Oak Hills", "place_type": "city", "bounding_box": rectangle("-122.857378,45.527087 -122.824348,45.555039") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4153988, "cityName": "Oak Hills" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719011060232192, "text": "\" if your bf doesn't pay for your nails , is he really your bf ? \" ..... Like whatttttt ??? ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548946463, "name": "rosie", "screen_name": "rosssie_", "lang": "en", "location": "Rodeo, Durango", "create_at": date("2012-04-08"), "description": "null", "followers_count": 2132, "friends_count": 1221, "statues_count": 22116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719011064418305, "text": "Ok..lets do this..about to sneak out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3187221900, "name": "Gaby", "screen_name": "sgaby_14", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "CSUF'19", "followers_count": 179, "friends_count": 173, "statues_count": 6328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719011085447169, "text": "1 amazing year with him ❤️ https://t.co/pP88R6CSG5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 720935852, "name": "¢ н ℓ σ є", "screen_name": "ChloeeElmoree", "lang": "en", "location": "null", "create_at": date("2012-07-27"), "description": "volleyball & cheer | @izzy_m23 ❤️", "followers_count": 973, "friends_count": 793, "statues_count": 5485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719011873988609, "text": "I can think of you all day and still not hear from me #pride", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pride" }}, "user": { "id": 2320375687, "name": "Nelson Jafeth", "screen_name": "nelsonjafeth", "lang": "en", "location": "null", "create_at": date("2014-01-30"), "description": "null", "followers_count": 247, "friends_count": 228, "statues_count": 5702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719012448608256, "text": "@POTUS how does it feel to be going to prison??? Traitor", "in_reply_to_status": -1, "in_reply_to_user": 1536791610, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1536791610 }}, "user": { "id": 482446758, "name": "George Harlan", "screen_name": "george_has_sweg", "lang": "en", "location": "null", "create_at": date("2012-02-03"), "description": "UTB #TeamNoStress", "followers_count": 587, "friends_count": 1146, "statues_count": 29439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719012607950848, "text": "@irene_cecelia @DopeItsBen hahahhahahahah greatest thing I've seen all day", "in_reply_to_status": 689718612785893376, "in_reply_to_user": 2170936057, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2170936057, 557560413 }}, "user": { "id": 3273885060, "name": "a❁", "screen_name": "aundrealenae", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "jimmy❥ | mbn | bffs: bre&noe&j | san pedro☀️", "followers_count": 361, "friends_count": 371, "statues_count": 7654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719013899788294, "text": "I just wanna leave Utah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2346573222, "name": "Adam's slave", "screen_name": "JEakett", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "pj❤️. 'stangs hockey", "followers_count": 594, "friends_count": 585, "statues_count": 2487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719013958639616, "text": "Why I'm still up is the real question here��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1920605448, "name": "Ðєиєℓℓιє❦ツ➳", "screen_name": "nadja_denelle", "lang": "en", "location": "Oℓ∂ Dσмιиισи Uиινєяѕιту❀", "create_at": date("2013-09-30"), "description": "||psychedelic rock⌘||νв⎈||20☥||mada mada daneツ||", "followers_count": 263, "friends_count": 248, "statues_count": 16667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719014097051649, "text": "@Phileirene ggc would never ������ #ggc19 https://t.co/VbR6aLpLF9", "in_reply_to_status": -1, "in_reply_to_user": 161595337, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ggc19" }}, "user_mentions": {{ 161595337 }}, "user": { "id": 4050554129, "name": "Champagne Mary❤️", "screen_name": "Goldieelocks__", "lang": "en", "location": "atlannna", "create_at": date("2015-10-26"), "description": "I'm just a little brown girl w. dreams & the rest is still unwritten...", "followers_count": 151, "friends_count": 155, "statues_count": 2114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawrenceville, GA", "id": "181444e0e9e17e20", "name": "Lawrenceville", "place_type": "city", "bounding_box": rectangle("-84.046802,33.910542 -83.951606,33.990757") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1345488, "cityName": "Lawrenceville" } }
+{ "create_at": datetime("2016-01-20T00:00:01.000Z"), "id": 689719014139039745, "text": "@jadapsmith we get it you're pissed your man didn't get a a nom but we still love him and pay to watch him ��", "in_reply_to_status": -1, "in_reply_to_user": 46306646, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46306646 }}, "user": { "id": 89006697, "name": "Lee Murray Jones", "screen_name": "LeeMurrayJones", "lang": "en", "location": "Powell, Oh", "create_at": date("2009-11-10"), "description": "null", "followers_count": 21, "friends_count": 247, "statues_count": 25 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Powell, OH", "id": "019057df2bbee4e5", "name": "Powell", "place_type": "city", "bounding_box": rectangle("-83.139191,40.137232 -83.031264,40.217862") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39041, "countyName": "Delaware", "cityID": 3964486, "cityName": "Powell" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014386450432, "text": "Temp: 42.4°F Wind:0.0mph Pressure: 30.290hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 23, "statues_count": 57516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014420037632, "text": "�� https://t.co/arTE0CpUMG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 217963787, "name": "hbic", "screen_name": "avatarcmoney", "lang": "en", "location": "Mother Earth", "create_at": date("2010-11-20"), "description": "I'm here to state my opinion and not care about yours", "followers_count": 3443, "friends_count": 984, "statues_count": 302582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014529040384, "text": "Wind 0.0 mph ESE. Barometer 30.309 in, Falling. Temperature 19.8 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014545752064, "text": "Officially 18 years old! ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520875040, "name": "Hi buddy waz up", "screen_name": "TheExtremeDuck", "lang": "en", "location": "California", "create_at": date("2012-03-10"), "description": "You ain't gotta agree all the time with me or see eye to eye. There will always be animosity between you and I. #StandWithRand #LibertyNotHillary", "followers_count": 353, "friends_count": 323, "statues_count": 18458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014705238017, "text": "@sherlyyalay https://t.co/hp1tRgtOYV", "in_reply_to_status": -1, "in_reply_to_user": 1375532264, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1375532264 }}, "user": { "id": 828281276, "name": "Will Saravia", "screen_name": "will_thejmuguy", "lang": "en", "location": "NoVa, VA", "create_at": date("2012-09-16"), "description": "null", "followers_count": 227, "friends_count": 317, "statues_count": 3795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dulles Town Center, VA", "id": "00b957f2781eee35", "name": "Dulles Town Center", "place_type": "city", "bounding_box": rectangle("-77.432552,39.007188 -77.401577,39.04679") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5123732, "cityName": "Dulles Town Center" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719014722015232, "text": "Wind 2.0 mph ESE. Barometer 30.394 in, Falling slowly. Temperature 19.3 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 41, "friends_count": 25, "statues_count": 17014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719015279726593, "text": "In dire need of a half zip up from pink", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374861509, "name": "vanessa", "screen_name": "vxnesa", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-09-16"), "description": "UCI", "followers_count": 660, "friends_count": 380, "statues_count": 34953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719015296479232, "text": "52.4F (Feels: 52.4F) - Humidity: 92% - Wind: 7.6mph E - Gust: 9.2mph - Pressure: 1034.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 223228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719015317442560, "text": "What a fugly name https://t.co/Rxxilh3J1B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 764568096, "name": "Allie Stevens", "screen_name": "Allie01_Stevens", "lang": "en", "location": "Fresno, CA", "create_at": date("2012-08-17"), "description": "CNHS", "followers_count": 186, "friends_count": 527, "statues_count": 1711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719015665610752, "text": "Got a queen sized bed and I still sleep on one side of the bed or at the edge. Why is that?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307653030, "name": "Chastin Valdez", "screen_name": "Realest_Valdez", "lang": "en", "location": "Maui, Hi", "create_at": date("2014-01-23"), "description": "Snapchat: chas_10valdez", "followers_count": 173, "friends_count": 188, "statues_count": 213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kahului, HI", "id": "5e2c83e1fb041c0c", "name": "Kahului", "place_type": "city", "bounding_box": rectangle("-156.500605,20.848879 -156.451278,20.90097") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1522700, "cityName": "Kahului" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719015715942400, "text": "So tired of all these ridiculous and unnecessary situation.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 80515367, "name": "Sabryna Tafoya", "screen_name": "SabrynaTafoya", "lang": "en", "location": "null", "create_at": date("2009-10-06"), "description": "selfish but never selfless", "followers_count": 662, "friends_count": 251, "statues_count": 23083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719016386985984, "text": "@lindsaayque apply online girl", "in_reply_to_status": 689718404047994880, "in_reply_to_user": 1337918142, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1337918142 }}, "user": { "id": 270185023, "name": "Tyriqaa", "screen_name": "Tyra_Princess", "lang": "en", "location": "Oakland, CA", "create_at": date("2011-03-21"), "description": "In-N-Out store 303 Alameda, cause that's all I do", "followers_count": 85, "friends_count": 74, "statues_count": 1045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719016567402496, "text": "Who tryna talk otp ?? Ya girl is bored and I need some entertainment ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412594200, "name": "Imani Hasiina", "screen_name": "2Gun_Shawtyyy", "lang": "en", "location": "Somewhere minding my business", "create_at": date("2011-11-14"), "description": "Don't disrespect the caterpillar and rave about the butterfly . | Ugly Mob Squad For Life | One of Taz's future Angels", "followers_count": 852, "friends_count": 338, "statues_count": 39509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719016626085888, "text": "@tiajaskie what why", "in_reply_to_status": 689718957918519296, "in_reply_to_user": 273174868, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 273174868 }}, "user": { "id": 310654251, "name": "Kiah McElroy", "screen_name": "KiahMcElroy", "lang": "en", "location": "null", "create_at": date("2011-06-03"), "description": "null", "followers_count": 706, "friends_count": 513, "statues_count": 23090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719016798175233, "text": "@RickyPDillon cool", "in_reply_to_status": 689717767608512512, "in_reply_to_user": 73171449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 73171449 }}, "user": { "id": 48610056, "name": "Alberto Candelaria", "screen_name": "alcan7", "lang": "en", "location": "Bronx, NY", "create_at": date("2009-06-18"), "description": "Work for NYC Transit over 28 yrs Big Michael Jackson & Taylor Swift fan. Swiftie Wife Michelle & son AFC riding bicycles Its Friday Friday got to get down on...", "followers_count": 1423, "friends_count": 2851, "statues_count": 112358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719017146179585, "text": "you can be whoever you want even yourself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105409716, "name": "Ju¡c3™", "screen_name": "TellemJuice", "lang": "en", "location": "BasedWorld", "create_at": date("2010-01-15"), "description": "18 | Honduran | Student Athlete | Freshmen at TLC | Philippians 1:6 | #Taken #Ravensnation| 02/27/1997", "followers_count": 402, "friends_count": 288, "statues_count": 8092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-20T00:00:02.000Z"), "id": 689719017825693696, "text": "@samarkito @choracuica é bom q pode passar muitos debate de ideia só não pode se diferente as ideia", "in_reply_to_status": 689616686534770689, "in_reply_to_user": 108794805, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 108794805, 8236782 }}, "user": { "id": 47519719, "name": "R. Melfra", "screen_name": "melo_franco", "lang": "pt", "location": "Seattle, WA ou Campinas, SP", "create_at": date("2009-06-15"), "description": "Graduate student (Comp Lit) at the University of Washington (Seattle), espresso lover, and gamer. Brazilian.", "followers_count": 1005, "friends_count": 854, "statues_count": 44614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719018429616128, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 124540088, "name": "snavs", "screen_name": "real_nava", "lang": "en", "location": "null", "create_at": date("2010-03-19"), "description": "18 | Rosemead | STAR$ | We dont brag. We flex", "followers_count": 955, "friends_count": 768, "statues_count": 16144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719018710814721, "text": "@melalanieee slide and I'll get you right", "in_reply_to_status": 689718950804893697, "in_reply_to_user": 2168694457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2168694457 }}, "user": { "id": 3300007882, "name": "!DK™", "screen_name": "ZEUSMALlK", "lang": "en", "location": "Miami, FL", "create_at": date("2015-05-26"), "description": "Aspiring comedian/YouTuber. https://www.youtube.com/channel/UCpXOCp4hTOWxc-M0yHxhTkQ", "followers_count": 219, "friends_count": 136, "statues_count": 5337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719019318870016, "text": "The song \"lipgloss\" was my favorite song when I was younger ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2852191616, "name": "Lil Jess", "screen_name": "jessiemachingo", "lang": "en", "location": "null", "create_at": date("2014-10-11"), "description": "do whatever the fuck makes you live life to the fullest", "followers_count": 428, "friends_count": 500, "statues_count": 1122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719019641950208, "text": "Wind 0 mph --. Barometer 30.28 in, Falling. Temperature 37.0 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020052844544, "text": "Another hoe :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581369642, "name": "Jeruld", "screen_name": "gv_v5", "lang": "en", "location": "null", "create_at": date("2012-05-15"), "description": "Cash rules everything around me", "followers_count": 194, "friends_count": 199, "statues_count": 21472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020283494404, "text": "I'm happy you met them https://t.co/hctaLtfr24", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2815731716, "name": "val", "screen_name": "jacksbullygrier", "lang": "en", "location": "null", "create_at": date("2014-09-17"), "description": "LA // I MISS SHAWN MENDES", "followers_count": 940, "friends_count": 1091, "statues_count": 8115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020321308672, "text": "I'm just a nice shy normal guy and I get treated like carp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.91640529,40.90135247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144227158, "name": "jacob katoa", "screen_name": "KatoaJacob", "lang": "en", "location": "west bontiful utah ", "create_at": date("2010-05-15"), "description": "Hi I'm jacob like too hangout I'm chill guy you'll meet I love all my friends I love music and tern stuff :)", "followers_count": 60, "friends_count": 416, "statues_count": 1105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Bountiful, UT", "id": "b5c126e8b1fe4d51", "name": "West Bountiful", "place_type": "city", "bounding_box": rectangle("-111.92298,40.883254 -111.891444,40.916218") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4982840, "cityName": "West Bountiful" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020354977792, "text": "Wind 1.6 mph W. Barometer 30.14 in, Rising slowly. Temperature 17.1 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 96, "statues_count": 156831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020426280960, "text": "@PrinceYahshua @kj_fetishmodel @morganleexxx @ElegantAngelxxx Iooking forward to the scene", "in_reply_to_status": 689321206277771265, "in_reply_to_user": 93490315, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 93490315, 2337874352, 2770894640, 31028420 }}, "user": { "id": 603765069, "name": "Christopher S Lewis", "screen_name": "CslChef830", "lang": "en", "location": "Burke, Va.", "create_at": date("2012-06-09"), "description": "null", "followers_count": 219, "friends_count": 677, "statues_count": 16537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burke, VA", "id": "4769ebcaeee50421", "name": "Burke", "place_type": "city", "bounding_box": rectangle("-77.327307,38.745249 -77.227003,38.841994") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5111464, "cityName": "Burke" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020770037760, "text": "Literally me in the car every fucking time �� https://t.co/s1RY7bsqmW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 741291642, "name": "Libby Sergot", "screen_name": "libby_sergot", "lang": "en", "location": "Rosemount - SCSU", "create_at": date("2012-08-06"), "description": "Standin by my man while he's in the sandbox. 'Murica", "followers_count": 915, "friends_count": 1141, "statues_count": 18049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemount, MN", "id": "6c54120abb49d96c", "name": "Rosemount", "place_type": "city", "bounding_box": rectangle("-93.159887,44.717092 -93.095493,44.775785") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2755726, "cityName": "Rosemount" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719020870770688, "text": "#SupportOriginMelissa 21.0°F Wind:0.0mph Pressure: 30.30hpa Falling slowly Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719021235757057, "text": "#prayforpakistan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "prayforpakistan" }}, "user": { "id": 3475069697, "name": "OZACK'S Razor .", "screen_name": "_masterifts_", "lang": "en", "location": "slowly drifting...", "create_at": date("2015-08-28"), "description": "Alone in the presence of many. 18. IG: _relampago_ #OU19 #g4life", "followers_count": 325, "friends_count": 337, "statues_count": 3835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719021323706369, "text": "I wanna four door Benz that'd fit all my friends so I could pull up on a bitch and tell her \"bitch get in!\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 303544911, "name": "Audrey", "screen_name": "_littleaud", "lang": "en", "location": "null", "create_at": date("2011-05-22"), "description": "PSU '19", "followers_count": 580, "friends_count": 305, "statues_count": 7546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719021906722816, "text": "IM HURTING!!!!!! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310035155, "name": "Homeless Raiders", "screen_name": "A_Flawed_King", "lang": "en", "location": "Where the West was won", "create_at": date("2011-06-02"), "description": "I talk, just not to you. #Lakers #Raiders #USC #TMT #KrishaJesusa", "followers_count": 1397, "friends_count": 1020, "statues_count": 134586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719022091264000, "text": "Why did I look like a Mexican boy @jadexdeperio https://t.co/gAp7Yl9SBe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2241814244 }}, "user": { "id": 101922595, "name": "Hannah_Chungg", "screen_name": "Hannah_Chungg", "lang": "en", "location": "null", "create_at": date("2010-01-04"), "description": "fresh", "followers_count": 120, "friends_count": 109, "statues_count": 582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719022246461442, "text": "@bryanlanning happy birthday!!!! Im sure loving the new #LikeALionEP", "in_reply_to_status": -1, "in_reply_to_user": 25222831, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LikeALionEP" }}, "user_mentions": {{ 25222831 }}, "user": { "id": 257077664, "name": "Alina Bannavong", "screen_name": "danceralina1994", "lang": "en", "location": "null", "create_at": date("2011-02-24"), "description": "dance,play piano, swim, act, ski what else can i do? And oh yeah im a directioner TMH 7-28-13 i love Niall and Zayn and Louis.GO UTAH STATE UNIVERSITY AGGIES!", "followers_count": 288, "friends_count": 1548, "statues_count": 2472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Logan, UT", "id": "e3d31ee0aea2ee7b", "name": "Logan", "place_type": "city", "bounding_box": rectangle("-111.879045,41.679328 -111.779715,41.797389") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49005, "countyName": "Cache", "cityID": 4945860, "cityName": "Logan" } }
+{ "create_at": datetime("2016-01-20T00:00:03.000Z"), "id": 689719022552686592, "text": "One whole mothafuckin year �� #IXXXV #loml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IXXXV", "loml" }}, "user": { "id": 2883337175, "name": "Leilani", "screen_name": "leiiilani_", "lang": "en", "location": "Crossfire 16u Elite⚾️", "create_at": date("2014-11-18"), "description": "stop feeling like you shouldn't speak your mind because someone's gonna call you a bitch, speak your mind. RJR❤️", "followers_count": 348, "friends_count": 1203, "statues_count": 3207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tara Hills, CA", "id": "34030b34b0f77c10", "name": "Tara Hills", "place_type": "city", "bounding_box": rectangle("-122.332431,37.986757 -122.306672,38.002367") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 677924, "cityName": "Tara Hills" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719022615588864, "text": "Itigil na to hangat maaga pa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 75530759, "name": "Ellaine Tanguin", "screen_name": "ElaiShaddai", "lang": "en", "location": "Austin, TX / MNL ", "create_at": date("2009-09-19"), "description": "Read between the lines. | snapchat: ellainetanguin", "followers_count": 361, "friends_count": 131, "statues_count": 23367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719022619783172, "text": "Some things just won't ever change and you just gotta accept it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2166075008, "name": "Pope", "screen_name": "xFrancisRoque", "lang": "en", "location": "Stockton, CA", "create_at": date("2013-10-30"), "description": "null", "followers_count": 822, "friends_count": 329, "statues_count": 12496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023064354817, "text": "@TheOneAboveAlll @palmerwhite26 @jcb709 @nflnetwork that actually happens alot the winner of the season is the loser of the post.", "in_reply_to_status": 689718211474919424, "in_reply_to_user": 2805809552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2805809552, 2714502695, 819875161, 19362299 }}, "user": { "id": 62960386, "name": "21 Blunts", "screen_name": "d0rq", "lang": "en", "location": "Inside Your Thoughts", "create_at": date("2009-08-04"), "description": "My tweets haven't made sense in awhile. Snapchat, Instagram, Kik, and whatever else becomes cool:~ @d0rq #CooninInCali", "followers_count": 387, "friends_count": 315, "statues_count": 12077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023093837828, "text": "Whan tha xans hitting you at night <<< ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2441345122, "name": "#STRECHGANG", "screen_name": "febb_7", "lang": "en", "location": "#Dedogang! ", "create_at": date("2014-03-27"), "description": "#FreeCharles! #FreeTory! #FreeRizzle! #FreeCrack! #Freemimid! #Freewhoppa! #RestBee! #RestTyrone! & #RestDedo! Jada❤️", "followers_count": 1354, "friends_count": 860, "statues_count": 18237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023186132992, "text": "#sup #morning #rainbowisland #maui @chrisfreemank97 #leash #wtf… https://t.co/YkUViVLKtG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-156.40213394,20.9084277"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "sup", "morning", "rainbowisland", "maui", "leash", "wtf" }}, "user_mentions": {{ 343630127 }}, "user": { "id": 90818771, "name": "AndreaPagan", "screen_name": "AndreaPagan", "lang": "en", "location": "Torbole Lake Garda", "create_at": date("2009-11-17"), "description": "null", "followers_count": 373, "friends_count": 559, "statues_count": 13261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spreckelsville, HI", "id": "00a153942e628520", "name": "Spreckelsville", "place_type": "city", "bounding_box": rectangle("-156.451736,20.882087 -156.392681,20.914336") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023202762752, "text": "Gn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 580448874, "name": "alexarae ➵", "screen_name": "_raaeeee__", "lang": "en", "location": "Austin, TX", "create_at": date("2012-05-14"), "description": "null", "followers_count": 880, "friends_count": 362, "statues_count": 18784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023458648064, "text": "I like to torrent them, otherwise the bill gets out of hand https://t.co/FyO6xXxR7u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 570305547, "name": "Keivmen", "screen_name": "thetempted", "lang": "en", "location": "Los Angeles", "create_at": date("2012-05-03"), "description": "I thought you said we were getting food after this.\n please excuse my dear aunt sally", "followers_count": 181, "friends_count": 80, "statues_count": 16206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023571996674, "text": "https://t.co/jyqC65swNP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 18459, "friends_count": 9340, "statues_count": 4030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719023798480896, "text": "@IAmJayKicks lol no. Been off for like 30 mins ��", "in_reply_to_status": 689718871155175424, "in_reply_to_user": 110215397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 110215397 }}, "user": { "id": 30941870, "name": "Kristyn S.", "screen_name": "K_Rickell", "lang": "en", "location": "CLE", "create_at": date("2009-04-13"), "description": "IG @_kristyns SC @K-Bunz", "followers_count": 1182, "friends_count": 751, "statues_count": 86461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bedford, OH", "id": "325e1f31f357ca08", "name": "Bedford", "place_type": "city", "bounding_box": rectangle("-81.568921,41.371788 -81.508039,41.409817") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3904878, "cityName": "Bedford" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719025086017536, "text": "@RetroGamerKF @Nero Milo try to be compassionate as you call the trans person out for lying about college graduation. WU matters as person", "in_reply_to_status": 689716958221115393, "in_reply_to_user": 4565212453, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4565212453, 6160792 }}, "user": { "id": 128990886, "name": "DK", "screen_name": "DesireeSeattle", "lang": "en", "location": "null", "create_at": date("2010-04-02"), "description": "#LivesMatter ...when Obama a black dem ran for POTUS where he was born mattered . Now republican born in Canada running for POTUS where born no longer matters?", "followers_count": 880, "friends_count": 1063, "statues_count": 44044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719025555767296, "text": "@imanh من گريه كردم چرا!", "in_reply_to_status": 689718611745869824, "in_reply_to_user": 14690170, "favorite_count": 0, "retweet_count": 0, "lang": "fa", "is_retweet": false, "user_mentions": {{ 14690170 }}, "user": { "id": 53090808, "name": "Raheleh Homaie", "screen_name": "rh18641", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-07-02"), "description": "Help Me - I Help You", "followers_count": 274, "friends_count": 181, "statues_count": 6383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719026184880128, "text": "i couldnt even run my own event bc i started crying thinking about having to talk which made it hard to talk again.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251306196, "name": "dirty dan", "screen_name": "ElishaisYoDaddy", "lang": "en", "location": "thizz nation", "create_at": date("2011-02-12"), "description": "UCSB | ΣΛΓ", "followers_count": 738, "friends_count": 524, "statues_count": 91404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719026289770496, "text": "I miss my braids https://t.co/6fiVdLZMmw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1119539083, "name": "money.", "screen_name": "EmahniCarr", "lang": "en", "location": "Kansas City, MO ", "create_at": date("2013-01-25"), "description": "chef steez was here n shit", "followers_count": 632, "friends_count": 486, "statues_count": 31832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raytown, MO", "id": "de1ce30b20eb6cf5", "name": "Raytown", "place_type": "city", "bounding_box": rectangle("-94.498502,38.963393 -94.43748,39.029502") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2960788, "cityName": "Raytown" } }
+{ "create_at": datetime("2016-01-20T00:00:04.000Z"), "id": 689719026449133568, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 517218418, "name": "chiquitita mala", "screen_name": "impulsivemari", "lang": "en", "location": "SoCal", "create_at": date("2012-03-06"), "description": "before you start pointing fingers make sure your hands are clean. oh lord, please don't let me be misunderstood.", "followers_count": 765, "friends_count": 610, "statues_count": 45308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719026986151937, "text": "Wind 4.1 mph NW. Barometer 30.108 in, Rising. Temperature 14.1 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719027103612928, "text": "@shaylannn_ �� ok but smoke soon plz", "in_reply_to_status": 689718956085612544, "in_reply_to_user": 410597086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 410597086 }}, "user": { "id": 3043720711, "name": "lucy", "screen_name": "luucymhall", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-02-26"), "description": "braaaaZyyyyy", "followers_count": 604, "friends_count": 445, "statues_count": 7016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719027137122304, "text": "Nothing about point deductions, but Matts eye was not jacked from an eye poke. A punch broke his orbital #AskBJM https://t.co/FtPJ9CikKb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AskBJM" }}, "user": { "id": 788197512, "name": "Big John McCarthy", "screen_name": "JohnMcCarthyMMA", "lang": "en", "location": "null", "create_at": date("2012-08-28"), "description": "Husband, Father, Grandfather, Love MMA, Certified ABC Trainer who started the COMMAND program. life is about having a blast", "followers_count": 45940, "friends_count": 86, "statues_count": 7043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719027518730240, "text": "Weekly https://t.co/SkIMXQurRK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88589803, "name": "L", "screen_name": "TzarGilly", "lang": "en", "location": "Amongst La Familia! ", "create_at": date("2009-11-08"), "description": "@cardinals @kingjames @cristiano @arkrazorbacks", "followers_count": 945, "friends_count": 245, "statues_count": 14705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Batesville, AR", "id": "2f5745deaec67fb6", "name": "Batesville", "place_type": "city", "bounding_box": rectangle("-91.669914,35.755315 -91.580241,35.816237") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5063, "countyName": "Independence", "cityID": 504030, "cityName": "Batesville" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719027539652608, "text": "@dejuan4u yeah but his other Shit still \"eh\" lol", "in_reply_to_status": 689718465586855936, "in_reply_to_user": 2577418302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2577418302 }}, "user": { "id": 756553112, "name": "santana", "screen_name": "whodafuccisclos", "lang": "en", "location": "Portland, Oregon", "create_at": date("2012-08-13"), "description": "I be doin' shit", "followers_count": 295, "friends_count": 305, "statues_count": 8777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719027640434688, "text": "Duke will also watch 6-9 2016 big man Kalif Young on Wednesday\nhttps://t.co/MieVRBpYXO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48224124, "name": "Adam Zagoria", "screen_name": "AdamZagoria", "lang": "en", "location": "New York", "create_at": date("2009-06-17"), "description": "Hoops Insider for http://t.co/Z4KEm3bLw0. Author of 2 books. Westchester Ultimate champ 2011 & '13. Founder of The 4 Quarters Podcast \nhttp://t.co/A1qqH3K46H", "followers_count": 40249, "friends_count": 1502, "statues_count": 92685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719028751929344, "text": "#HouseBusinessCheck at 4006 Conroy Rd. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4270248,28.4862884"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HouseBusinessCheck", "orlpol", "opd" }}, "user": { "id": 39065901, "name": "Police Calls 32839", "screen_name": "orlpol32839", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 152, "friends_count": 1, "statues_count": 36378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719029192228866, "text": "@CDot5_ @DboyDarius @_Markyy they trash ����", "in_reply_to_status": 689718965220696064, "in_reply_to_user": 1260838836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1260838836, 3189137844, 1385963774 }}, "user": { "id": 1266023012, "name": "3%", "screen_name": "Marquis_buckets", "lang": "en", "location": "null", "create_at": date("2013-03-13"), "description": "DON'T", "followers_count": 320, "friends_count": 334, "statues_count": 9439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719029217521664, "text": "I just used Shazam to discover Running For You by Kip Moore. https://t.co/7IosZ8wQeb. We need more help for breast Cancer please!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2839461810, "name": "Jeanie Gerhardi", "screen_name": "jeanieg995", "lang": "en", "location": "Warren, MI", "create_at": date("2014-10-03"), "description": "love my 995 Detroit's best new Country", "followers_count": 359, "friends_count": 387, "statues_count": 33684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warren, MI", "id": "59381e983a8a5770", "name": "Warren", "place_type": "city", "bounding_box": rectangle("-83.086881,42.447289 -82.967096,42.538403") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2684000, "cityName": "Warren" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719029397897216, "text": "Garbage Time is HEEEEEEEEEEEERE! https://t.co/qt1a25V3dH or 91.1 #wfmu #garbagetime #uriahheep @ WFMU Studio https://t.co/4S6SffxDky", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0361862,40.7165794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wfmu", "garbagetime", "uriahheep" }}, "user": { "id": 69154207, "name": "Matt Warwick", "screen_name": "EasterHam", "lang": "en", "location": "Point Pleasant, New Jersey", "create_at": date("2009-08-26"), "description": "Host of Garbage Time on WFMU. Also a slob.", "followers_count": 183, "friends_count": 195, "statues_count": 1743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719029871874048, "text": "Wind 9.6 mph WNW. Barometer 30.313 in, Steady. Temperature 20.3 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719029993320449, "text": "@sexygaystweet \nI had a government Professor at OCCC who said there's more republicans living in OK being the reason why it's a red state", "in_reply_to_status": 689718593466986496, "in_reply_to_user": 1316372436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 350648239 }}, "user": { "id": 1316372436, "name": "Zenoxx Chrovnoux", "screen_name": "Zenoxx69", "lang": "en", "location": "73170 [SW OKC, Cleveland Co]", "create_at": date("2013-03-30"), "description": "Warning: Moist contents #NSFW | \nFurFag and a Switch|Versatile | Homosapien |\nBorn: Fairfax County, Commonwealth of Virginia, USA ; S Vietnamese & 1/8th Chinese", "followers_count": 875, "friends_count": 1428, "statues_count": 264591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719030312136704, "text": "03:00 16.7°F Feels:16.7°F (Hi19.2°F/Lo16.7°F) Hum:63% Wnd:ENE 1.6MPH Baro:30.29in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 169, "friends_count": 266, "statues_count": 24915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-20T00:00:05.000Z"), "id": 689719030911893504, "text": "@Kate_mamit oh I know it's fake but still funny anyhow", "in_reply_to_status": 689718804570595329, "in_reply_to_user": 2919807605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2919807605 }}, "user": { "id": 39023478, "name": "Faiza Mokhtar", "screen_name": "legallyfm", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-10"), "description": "Wit and sass limited to 140 characters + fangirling for David Duchovny. | guess I'll talk about sports, pop culture & take photos too", "followers_count": 220, "friends_count": 441, "statues_count": 6109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719031776026625, "text": "Me acuerdo cuando entre al sex shop a comprarle el consolador a nico", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3202310770, "name": "Pretty Wise", "screen_name": "manu_chin", "lang": "es", "location": "Universo", "create_at": date("2015-04-24"), "description": "//Bastardo sin gloria.\\\\ Club Atlético Boca Juniors. Festejar para sobrevivir. //•Hades• ☆WestHam☆", "followers_count": 209, "friends_count": 270, "statues_count": 9825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719032052711424, "text": "@alexxg21 @FashionParallel aye you right ��", "in_reply_to_status": 689718954038693888, "in_reply_to_user": 4020620534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4020620534, 3245080573 }}, "user": { "id": 634435308, "name": "uriel.", "screen_name": "_ury21", "lang": "en", "location": "Fontana, CA", "create_at": date("2012-07-12"), "description": "Footballer // 6OD", "followers_count": 879, "friends_count": 641, "statues_count": 19977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719032736387072, "text": "Happy 19th B-day to MY BEST FRIEND�� thank you for always making me laugh+ always be there for me. Your kinda great�� https://t.co/N1oBnlwK7J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1918545301, "name": "ʟındѕ❥", "screen_name": "lindsdanielle7", "lang": "en", "location": "Cali ", "create_at": date("2013-09-29"), "description": "null", "followers_count": 274, "friends_count": 358, "statues_count": 5203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jacinto, CA", "id": "0068bc1a061e33c3", "name": "San Jacinto", "place_type": "city", "bounding_box": rectangle("-117.039483,33.751357 -116.919422,33.818476") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 667112, "cityName": "San Jacinto" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719032807821312, "text": "William singe is goals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3939564316, "name": "Samantha", "screen_name": "Sammm_4", "lang": "en", "location": "Baltimore, MD", "create_at": date("2015-10-12"), "description": "IG- SamanthaaPowell l SC- Samanthapowell4", "followers_count": 116, "friends_count": 461, "statues_count": 1233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carney, MD", "id": "186f9ea317217282", "name": "Carney", "place_type": "city", "bounding_box": rectangle("-76.568646,39.379544 -76.498934,39.44386") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2413325, "cityName": "Carney" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719033185234945, "text": "cause I got too good at just doin it myself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1110262350, "name": "ally ✨", "screen_name": "allygcosetta", "lang": "en", "location": "Modesto, CA", "create_at": date("2013-01-21"), "description": "hey lolita hey | @asvpxrocky", "followers_count": 1045, "friends_count": 645, "statues_count": 13901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719033415864320, "text": "@Lexiaaap fake ass deleting shit �� https://t.co/90jB9kmbBC", "in_reply_to_status": -1, "in_reply_to_user": 3114551238, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3114551238 }}, "user": { "id": 452378863, "name": "lil mamí", "screen_name": "yungdaaze", "lang": "en", "location": "yay area ", "create_at": date("2012-01-01"), "description": "positive vibes positive lives | sc: @onlydaisytho", "followers_count": 653, "friends_count": 627, "statues_count": 27770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719033558646784, "text": "Greatest thing I've seen https://t.co/kBhq7GREyB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66986439, "name": "Kole Wilkinson", "screen_name": "Kole2012_WKU", "lang": "en", "location": "Smalltown, USA", "create_at": date("2009-08-19"), "description": "Just a good ole southern boy. WKU Coed Cheerleader , Country Music Star,Glasgow Alumni, Certified Private Pilot and Entrepreneur with Steamer Seafood.", "followers_count": 1278, "friends_count": 1285, "statues_count": 7413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719033697009664, "text": "I love my women #BBW https://t.co/3xzsUEXg9r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BBW" }}, "user": { "id": 162916062, "name": "Kareem", "screen_name": "the_real_reem", "lang": "en", "location": "North Jersey", "create_at": date("2010-07-04"), "description": "null", "followers_count": 497, "friends_count": 1371, "statues_count": 1373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719033835429888, "text": "Aye Cooder called it tho! On who", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1275717872, "name": "UMES Miss Freshman❣", "screen_name": "cvrmiv", "lang": "en", "location": "Paris, Ile-de-France", "create_at": date("2013-03-17"), "description": "University of Maryland Eastern Shore's Miss Freshman 2015-2016 | fashion merchandising major❤️ | God is within her, she will not fall Psalm 46:5 #umes19", "followers_count": 6661, "friends_count": 4454, "statues_count": 40561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Severn, MD", "id": "1c98b097b6fcb4d6", "name": "Severn", "place_type": "city", "bounding_box": rectangle("-76.767088,39.094164 -76.642175,39.214142") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2471150, "cityName": "Severn" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719034078584833, "text": "Temp: 12.8°F - Dew Point: 3.6° - Wind: 1.8 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 30.32in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719034435100672, "text": "Tell me right now, everything I need in them white cotton sheets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331216313, "name": "ricky.", "screen_name": "RickySwagLopez", "lang": "en", "location": "817/806", "create_at": date("2011-07-07"), "description": "5x WWE champion. Don't pass out with your shoes on. #TTU19 Guns up!", "followers_count": 1282, "friends_count": 326, "statues_count": 123319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-20T00:00:06.000Z"), "id": 689719035030814720, "text": "End up with nobody", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377644198, "name": "Dece♏️berr1st❤️", "screen_name": "YourUnder_Mee", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "December1st ✨❤️ snapchat:richhomie_kay GHS'16", "followers_count": 1307, "friends_count": 998, "statues_count": 29302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-01-20T00:00:07.000Z"), "id": 689719035781451777, "text": "Cause girls need Starbucks 7 days a week , so the side dude gets the other 3 days & Handy here and their it's basic math models. #StayWoke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StayWoke" }}, "user": { "id": 605921491, "name": "Almighty JJ ♠️™", "screen_name": "YeYey_Zapata", "lang": "en", "location": "Harlem to Houston to Haiti ", "create_at": date("2012-06-11"), "description": "Do not give up , the beginning is always the hardest #50DaysCTW #TD jermiah 1:5 TAMU 18 Aggie baseball", "followers_count": 597, "friends_count": 448, "statues_count": 41947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081393783758848, "text": "Wind 3.0 mph E. Barometer 1023.20 mb, Falling. Temperature 32.5 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 11993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081393796366336, "text": "Want to work in #Milpitas, CA? View our latest opening: https://t.co/29glZPKoDX #IT #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9066238,37.4282724"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Milpitas", "IT", "Job", "Jobs", "Hiring" }}, "user": { "id": 1623709555, "name": "Viavi Jobs", "screen_name": "ViaviJobs", "lang": "en", "location": "Worldwide", "create_at": date("2013-07-26"), "description": "Are you looking for a career where you can positively affect technology to enhance the way people experience the world? If so, your future is at Viavi.", "followers_count": 336, "friends_count": 157, "statues_count": 1552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861066,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081394282905602, "text": "Anyone want to bring me some whataburger❓❓❓", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3000016031, "name": "AD400", "screen_name": "adgarza111", "lang": "en", "location": "null", "create_at": date("2015-01-26"), "description": "Heart of a Hustler, Mind of a G‼️", "followers_count": 164, "friends_count": 152, "statues_count": 3707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081394698092544, "text": "https://t.co/MhSJ8Tjrm7 \nbarneys dissed those black kids people like jay z stacey dash rza defended barneys..thats hilarious", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53472414, "name": "E.C.B.O", "screen_name": "eseibio", "lang": "en", "location": "null", "create_at": date("2009-07-03"), "description": "( BLACK POWER RECORDS )\n\npurchase revolutionary minded on itunes now", "followers_count": 861, "friends_count": 441, "statues_count": 46734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081394727460865, "text": "@Joanaponcee @Aaantwony I love you bae", "in_reply_to_status": 690067863063891968, "in_reply_to_user": 4568562374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4568562374, 1107889873 }}, "user": { "id": 383906535, "name": "Maggie ShaeLynn", "screen_name": "MaggieShaelynn", "lang": "en", "location": "Camarillo, CA", "create_at": date("2011-10-02"), "description": "ສະບາຍດີ", "followers_count": 531, "friends_count": 504, "statues_count": 23057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081395704750080, "text": "Accurate https://t.co/6KngAjfKrS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339889023, "name": "Hillary Banks", "screen_name": "tristenzeltee", "lang": "en", "location": "Atlantica", "create_at": date("2011-07-21"), "description": "RihannaNavy⚓️⚓️⚓️", "followers_count": 1462, "friends_count": 1204, "statues_count": 86680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081397160185856, "text": "Sometimes I like being alone. Focusing on God. My mission.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2933262657, "name": "gabriella", "screen_name": "gabriellaaaa7", "lang": "en", "location": "702 ", "create_at": date("2014-12-20"), "description": "Mormon | Hispanic", "followers_count": 543, "friends_count": 606, "statues_count": 19164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081397244063746, "text": "this is Stella. she sleeps a lot. sometimes with her tongue out. https://t.co/RZE7ueVeLf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 445277462, "name": "Cora Stock", "screen_name": "cora_stock", "lang": "en", "location": "null", "create_at": date("2011-12-23"), "description": "Whitworth University 2018", "followers_count": 277, "friends_count": 317, "statues_count": 7392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "013a0d802e0e0e6b", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-117.453957,47.751098 -117.367548,47.826215") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081397575389184, "text": "so glad I dont have work tomorrow, can finally sleep !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607603003, "name": "Diamond.", "screen_name": "primadiamond__", "lang": "en", "location": "Somewhere laughing at you!", "create_at": date("2012-06-13"), "description": "Hi I'm ...", "followers_count": 982, "friends_count": 811, "statues_count": 20593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Speedway, IN", "id": "6aa48afe6daf08a9", "name": "Speedway", "place_type": "city", "bounding_box": rectangle("-86.275511,39.765474 -86.225176,39.811095") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1871828, "cityName": "Speedway" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081397608992768, "text": "midnight and all i wanna do is watch friends all night but school tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2868564073, "name": "ㅤ ㅤ ㅤㅤ", "screen_name": "AyllinRuvalcaba", "lang": "en", "location": "The Bay Area", "create_at": date("2014-10-20"), "description": "null", "followers_count": 317, "friends_count": 177, "statues_count": 134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-21T00:00:00.000Z"), "id": 690081397676048384, "text": "#SarahPalinAlsoJustEndorsed @HillaryClinton dabbing. #SMH #Election2016 https://t.co/Tk0ROF1wdJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SarahPalinAlsoJustEndorsed", "SMH", "Election2016" }}, "user_mentions": {{ 1339835893 }}, "user": { "id": 2207565558, "name": "iLLUMiTHOTTY", "screen_name": "neffexmusic", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-11-21"), "description": "enslaved by the illuminati & forced to create the Devil's soundtrack. http://Soundcloud.com/neffexmusic #FOXFAM", "followers_count": 525, "friends_count": 566, "statues_count": 338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081398179373057, "text": "there is no weakness in forgiveness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32654842, "name": "alana", "screen_name": "alanavanderaa", "lang": "en", "location": "los angeles", "create_at": date("2009-04-17"), "description": "null", "followers_count": 212, "friends_count": 255, "statues_count": 924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081398728843264, "text": "01/21@03:00 - Temp 27.5F, WC 23.9F. Wind 3.2mph WNW, Gust 7.0mph. Bar 30.146in, Rising slowly. Rain 0.00in. Hum 72%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081399068585986, "text": "all that early\nEnglish New Wave\nstuff sounds like a\nweak reproduction of\nWilliam Onyeabor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41731338, "name": "e:vil:Way:lon", "screen_name": "EvilWaylon", "lang": "en", "location": "Bay Area, CA x Hawaii", "create_at": date("2009-05-21"), "description": "Running from my problems and escaping into my dreams.", "followers_count": 424, "friends_count": 416, "statues_count": 1957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081400486268928, "text": "Won't u come on over and we can treat it like its the motherfucking end of the world and they depended on us to make earth Shaken love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622665483, "name": "DESTROYER", "screen_name": "LILB1TCHH", "lang": "en", "location": "S/C : shellyy13", "create_at": date("2012-06-29"), "description": "Eat my pussy & support my goals and dreams @naughtynauds ∞", "followers_count": 705, "friends_count": 563, "statues_count": 41697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Miami, FL", "id": "cae987165260f81f", "name": "North Miami", "place_type": "city", "bounding_box": rectangle("-80.227398,25.880982 -80.127938,25.930229") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1249450, "cityName": "North Miami" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081400846995456, "text": "already procrastinating��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1466708006, "name": "mckenna", "screen_name": "kennahelander", "lang": "en", "location": "null", "create_at": date("2013-05-29"), "description": "living in the vastness of outerspace", "followers_count": 1091, "friends_count": 451, "statues_count": 26940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-21T00:00:01.000Z"), "id": 690081401161523200, "text": "@ebbtideapp Tide in Bass Harbor, Maine 01/21/2016\nHigh 8:21am 11.3\n Low 2:16pm -0.6\nHigh 8:58pm 10.2\n Low 2:31am 0.1\nHigh 9:15am 11.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-68.3533,44.2417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 6, "friends_count": 1, "statues_count": 913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23009, "countyName": "Hancock" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081402201731072, "text": "Temp: 54.8°F Wind:0.0mph Pressure: 30.092hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 23, "statues_count": 57610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081402600169472, "text": "Awkward", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2581760112, "name": "Janessa", "screen_name": "GallardoJanessa", "lang": "en", "location": "null", "create_at": date("2014-06-22"), "description": "97", "followers_count": 208, "friends_count": 209, "statues_count": 2600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081402667323393, "text": "Wind 1.0 mph E. Barometer 30.367 in, Rising slowly. Temperature 20.2 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 49, "friends_count": 25, "statues_count": 17041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081402759544833, "text": "@meghansnshn sike happy bday @chodieee!! i got last", "in_reply_to_status": 690081285222588420, "in_reply_to_user": 42550261, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42550261, 489780496 }}, "user": { "id": 24038805, "name": "Sir Hottest Of Jays", "screen_name": "j_hotz", "lang": "en", "location": "Crippled America", "create_at": date("2009-03-12"), "description": "I'm up next like the Iowa caucus.", "followers_count": 2489, "friends_count": 849, "statues_count": 363402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081402818301952, "text": "Wind 1.0 mph SE. Barometer 30.320 in, Steady. Temperature 20.7 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081403917242369, "text": "56.7F (Feels: 56.7F) - Humidity: 99% - Wind: 1.6mph E - Gust: 2.2mph - Pressure: 1029.5mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 223372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081403967520769, "text": "OMFG THIS IS NOT FUNNY BUT HOLY\nHELL https://t.co/llD0ctYsY7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36850911, "name": "connor magee", "screen_name": "connormvgee", "lang": "en", "location": "portland, or", "create_at": date("2009-04-30"), "description": "@ofconfidencepdx • paddling for the closeouts • he him his", "followers_count": 882, "friends_count": 323, "statues_count": 4315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberg, OR", "id": "7f3bf05bec93c209", "name": "Newberg", "place_type": "city", "bounding_box": rectangle("-122.989925,45.28369 -122.925614,45.328231") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4152100, "cityName": "Newberg" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081404105965568, "text": "I hate black and white layouts but my icon looks better in black and white ugh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310582780, "name": "kaylee", "screen_name": "wtvrhunty", "lang": "en", "location": "ohio", "create_at": date("2011-06-03"), "description": "thanks james franco", "followers_count": 4428, "friends_count": 158, "statues_count": 79830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Amherst, OH", "id": "95ea34bb82dee7f5", "name": "South Amherst", "place_type": "city", "bounding_box": rectangle("-82.266777,41.334955 -82.210195,41.380744") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3973040, "cityName": "South Amherst" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081404391182336, "text": "Going to be up all night now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416431334, "name": "✨Alphonso✨", "screen_name": "alphonsoo_", "lang": "en", "location": "Massillon, OH", "create_at": date("2011-11-19"), "description": "Death ends a life, not a relationship. Rest in peace David, I love and miss you.", "followers_count": 1732, "friends_count": 627, "statues_count": 76227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massillon, OH", "id": "005f11de9931c8a4", "name": "Massillon", "place_type": "city", "bounding_box": rectangle("-81.631799,40.732694 -81.432311,40.896962") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3948244, "cityName": "Massillon" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081405058023424, "text": "Das wasssupppppp! https://t.co/IKzxSWBX4s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43653241, "name": "BrettPattersonn", "screen_name": "BrettPatterson0", "lang": "en", "location": "HELIUM MUSIC VIDEO ", "create_at": date("2009-05-30"), "description": "IG: Brettpattersonn Vine: brettpatterson0 Facebook: Brett Patterson", "followers_count": 6267, "friends_count": 5741, "statues_count": 2590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081405460713473, "text": "When that bitch is still hurt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2410077697, "name": "grace", "screen_name": "gracceface", "lang": "en", "location": "null", "create_at": date("2014-03-24"), "description": "null", "followers_count": 457, "friends_count": 444, "statues_count": 12192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindsay, CA", "id": "1c7fc1369e419742", "name": "Lindsay", "place_type": "city", "bounding_box": rectangle("-119.120488,36.181549 -119.060335,36.237053") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 641712, "cityName": "Lindsay" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081405603287040, "text": "Wind 0.7 mph NNW. Barometer 30.09 in, Steady. Temperature 20.8 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 96, "statues_count": 156856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-21T00:00:02.000Z"), "id": 690081406026956801, "text": "https://t.co/YXvMBaHiH3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 18759, "friends_count": 10448, "statues_count": 4443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081406341529601, "text": "Every other girl is trying to be you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553819870, "name": "Chosen", "screen_name": "ialwayswinben", "lang": "en", "location": "null", "create_at": date("2012-04-14"), "description": "create your own [you]niverse", "followers_count": 148, "friends_count": 221, "statues_count": 15015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081406404403200, "text": "#SupportOriginMelissa 21.6°F Wind:2.0mph Pressure: 30.24hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081406874181632, "text": "@deanasaur39 me too ��", "in_reply_to_status": 690080281991208960, "in_reply_to_user": 1120658462, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1120658462 }}, "user": { "id": 396054686, "name": "Matt", "screen_name": "mattmendenhall3", "lang": "en", "location": "snapchat; mattmendenhall3", "create_at": date("2011-10-22"), "description": "| Currently in L word™ with the Hunt | Left side flipper #CT | #UVTW |", "followers_count": 932, "friends_count": 454, "statues_count": 22593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sand Springs, OK", "id": "688d7e828fac953f", "name": "Sand Springs", "place_type": "city", "bounding_box": rectangle("-96.187311,36.061052 -96.065151,36.172375") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4065300, "cityName": "Sand Springs" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081406928732160, "text": "The weather in this state though. I'm done tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353412984, "name": "Dami.Lola", "screen_name": "Damss_oo", "lang": "en", "location": "Ames, IA", "create_at": date("2011-08-11"), "description": "@Tomi3000_ . ♡M. They don't want me to be happy.", "followers_count": 514, "friends_count": 433, "statues_count": 9008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ames, IA", "id": "d6794586b08b7f9c", "name": "Ames", "place_type": "city", "bounding_box": rectangle("-93.718311,41.986006 -93.560416,42.079007") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1901855, "cityName": "Ames" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081407222308864, "text": "Wind 3.0 mph WNW. Barometer 30.201 in, Steady. Temperature 18.9 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081407226482688, "text": "@daviflapjacks @camilla_kae i will hang out with both of u. Camilla i havent seen u in decades", "in_reply_to_status": 690081243350851586, "in_reply_to_user": 1654627068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1654627068, 3966768974 }}, "user": { "id": 2711310866, "name": "gecko dad", "screen_name": "Osmarkismexi", "lang": "en", "location": "Oregon, USA", "create_at": date("2014-08-05"), "description": "crippled by life", "followers_count": 803, "friends_count": 231, "statues_count": 49164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081407494922240, "text": "Iight I'm in college ,I'm gone try to find a way back down there https://t.co/rar6l8aLO9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297833304, "name": "Mr. Do it all", "screen_name": "Chrisdothafool", "lang": "en", "location": "3500 Lane", "create_at": date("2011-05-12"), "description": "Sold Water To A Whale #TrappinMadeItHappen #HSU19 #LoyaltyBeforeRoyalty #DeathBeforeDishonor", "followers_count": 1819, "friends_count": 1521, "statues_count": 36080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkadelphia, AR", "id": "bdefad825ffb7daa", "name": "Arkadelphia", "place_type": "city", "bounding_box": rectangle("-93.093798,34.091696 -93.044691,34.161433") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5019, "countyName": "Clark", "cityID": 501870, "cityName": "Arkadelphia" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081408568692736, "text": "Goodnight ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 145169995, "name": "Maggiie Mendoza", "screen_name": "MaggiieLuu", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-18"), "description": "20| I'm a sour patch and I laugh a lot☀️", "followers_count": 345, "friends_count": 310, "statues_count": 8807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081408593846272, "text": "@JLo nos vemos @ Mr Chow", "in_reply_to_status": -1, "in_reply_to_user": 85603854, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 85603854 }}, "user": { "id": 55830086, "name": "Pochi Grajales", "screen_name": "pochigrajales", "lang": "es", "location": "Ciudad de Panamá", "create_at": date("2009-07-11"), "description": "Administrador de Riesgos / Analista en Seguridad Física y Ocupacional / Integrador de Sistemas de Seguridad Electrónica", "followers_count": 141, "friends_count": 231, "statues_count": 804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081409009111040, "text": "#NewMusic Old Bay Shhhh - Single by @YoungEbay \nhttps://t.co/BQeFkj8f6w #ClassClownRecords", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NewMusic", "ClassClownRecords" }}, "user_mentions": {{ 630115041 }}, "user": { "id": 630115041, "name": "Bombay E-bay", "screen_name": "YoungEbay", "lang": "en", "location": "Northern California", "create_at": date("2012-07-08"), "description": "Hip-Hop & Rap Artist, Actor, Entrepreneur, Cannibus Cowboy, Real Ni**a, Family & Loyalty = Money & Power (AFNF) The Struggle Iz Real, Stay✈️Snapchat: younge-bay", "followers_count": 859, "friends_count": 1087, "statues_count": 2560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081409109721088, "text": "@_iraaani imy", "in_reply_to_status": 690080155507769344, "in_reply_to_user": 177784438, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 177784438 }}, "user": { "id": 194328370, "name": "Richard", "screen_name": "RichardJuniorrr", "lang": "en", "location": "null", "create_at": date("2010-09-23"), "description": "null", "followers_count": 503, "friends_count": 367, "statues_count": 17526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081409361448961, "text": "@Jtriangleason sat & sun but only for like an hr :/ why does she hate us lmaooo", "in_reply_to_status": 690079967447773185, "in_reply_to_user": 129866715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 129866715 }}, "user": { "id": 331813657, "name": "sarah", "screen_name": "SarahMyDude", "lang": "en", "location": "null", "create_at": date("2011-07-08"), "description": "sc - sarahmydude", "followers_count": 623, "friends_count": 356, "statues_count": 59012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081409428537344, "text": "Travelling to Oak Harbor or just twittering about Oak Harbor? https://t.co/sEI8kj9uwj #Oak Harbor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.643,48.2932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oak" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1007, "friends_count": 312, "statues_count": 2540178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Harbor, WA", "id": "f3350a5307c79b04", "name": "Oak Harbor", "place_type": "city", "bounding_box": rectangle("-122.754056,48.238966 -122.520561,48.340116") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53029, "countyName": "Island", "cityID": 5350360, "cityName": "Oak Harbor" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081409675968513, "text": "I can't sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302826175, "name": "Payton Jackson", "screen_name": "paytie_lou_who", "lang": "en", "location": "null", "create_at": date("2011-05-21"), "description": "i love being blonde and they call me princess payton #TTU19", "followers_count": 609, "friends_count": 230, "statues_count": 17573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081410061864961, "text": "https://t.co/1rZ6KFqYPO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1919734446, "name": "SWGOTCHO-HO-ON-KNEES", "screen_name": "SWYOHOEONKNEES", "lang": "en", "location": "null", "create_at": date("2013-09-30"), "description": "null", "followers_count": 0, "friends_count": 0, "statues_count": 8 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081410212839426, "text": "@Ashhplz it worked", "in_reply_to_status": 690080853804859395, "in_reply_to_user": 2948700422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2948700422 }}, "user": { "id": 2473631521, "name": "davd", "screen_name": "dcdaveed", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "null", "followers_count": 234, "friends_count": 186, "statues_count": 22682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-21T00:00:03.000Z"), "id": 690081410422562816, "text": "LIIIIIVED for @BrandiGlanville on tonight's episode of @KocktailsWKhloe. ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61387670, 3280065284 }}, "user": { "id": 471822187, "name": "Joslyn Stricklin", "screen_name": "TrannyGirlJB", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2012-01-23"), "description": "Transgender. Heterosexual Woman/Girl/Lady/Her/She. BBW. Singer. Entertainer. Animal Rights Activist. LGBT Rights Activist.", "followers_count": 2390, "friends_count": 171, "statues_count": 10468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081411013935104, "text": "FOH �� https://t.co/4X3iT1afFq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 298846640, "name": "AYoungOG⚡️", "screen_name": "BDugasss", "lang": "en", "location": "Where Playa's get chose ", "create_at": date("2011-05-14"), "description": "20|NeighborhoodSupaStar ✨ Bitch I'm from Rayne, Louisiana", "followers_count": 1725, "friends_count": 792, "statues_count": 114461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rayne, LA", "id": "807281e8b753fb7e", "name": "Rayne", "place_type": "city", "bounding_box": rectangle("-92.296633,30.198776 -92.20768,30.264282") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22001, "countyName": "Acadia", "cityID": 2263645, "cityName": "Rayne" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081411181756416, "text": "Happy 16th Birthday @ayurialtemus ������������������������❗️❗️❗️ https://t.co/yYNtXrWS2X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2502200575 }}, "user": { "id": 2502200575, "name": "Yuri Altemus", "screen_name": "ayurialtemus", "lang": "en", "location": "null", "create_at": date("2014-05-17"), "description": "Wubba Lubba Dub Dub", "followers_count": 163, "friends_count": 361, "statues_count": 652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081411194298369, "text": "@gracie_aby where can I get a friend like Thore?", "in_reply_to_status": 690081164674072576, "in_reply_to_user": 703437846, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 703437846 }}, "user": { "id": 1352829067, "name": "Ⓜ️@", "screen_name": "_nissenmatt_", "lang": "en", "location": "Reno, NV", "create_at": date("2013-04-14"), "description": "Bay Area | Nevada '18 | Golfaholic", "followers_count": 572, "friends_count": 441, "statues_count": 41434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081411848638465, "text": "Amazon is offering refunds on all hoverboards, U.S. consumer safety agency says https://t.co/ozR4eczgQ0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17662, "friends_count": 17578, "statues_count": 67349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081411961864193, "text": "1.9 magnitude #earthquake. 10 km from Fritz Creek, AK, #UnitedStates https://t.co/YSqpeZj0kg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.3532,59.6513"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 61976, "friends_count": 10, "statues_count": 93939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081412289019905, "text": "Can you find Oak Harbor on the map? Just try it at https://t.co/sEI8kj9uwj #Oak Harbor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.643,48.2932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oak" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1007, "friends_count": 312, "statues_count": 2540179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Harbor, WA", "id": "f3350a5307c79b04", "name": "Oak Harbor", "place_type": "city", "bounding_box": rectangle("-122.754056,48.238966 -122.520561,48.340116") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53029, "countyName": "Island", "cityID": 5350360, "cityName": "Oak Harbor" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081412649717764, "text": "No basta con alcanzar la sabiduría, es necesario saber utilizarla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 18759, "friends_count": 10448, "statues_count": 4444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081412913983489, "text": "Man I'm goin to sleep eyes getting heavy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.2259647,39.9707419"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470727882, "name": "REAL NIGGA CHINO✍", "screen_name": "so_WESTPHILLY", "lang": "en", "location": "5⃣4⃣0⃣0⃣ RUBY ST #FREEMYBLOCK ", "create_at": date("2012-01-21"), "description": "SAN GLIZZY✍ LONGLIVE〽️ARK FREE DAVE", "followers_count": 802, "friends_count": 680, "statues_count": 17800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081413069172738, "text": "I need a gym partner ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3232242534, "name": "Diamond Vasquez", "screen_name": "diamondvasquezz", "lang": "en", "location": "Bakersfield, Balifornia", "create_at": date("2015-05-31"), "description": "Good Girl Bad Habits.", "followers_count": 158, "friends_count": 173, "statues_count": 1583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081413513764864, "text": "Rough night tonight. I'm glad I got sleep today. Lotsa folks are stressed. Yikes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2293467942, "name": "Stormy Kittyhawk", "screen_name": "StormyKittyhawk", "lang": "en", "location": "Bay Area, Ca", "create_at": date("2014-01-15"), "description": "Vet tech, Furry, falconer, theater techie, Ham radio, nerdy, curvy, and purdy. Enjoys working cons. Married to @SamuelAFalvoII NSFW account is @DarknStormy_AD", "followers_count": 357, "friends_count": 403, "statues_count": 8373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081413736067073, "text": "1.5 magnitude #earthquake. Central Alaska https://t.co/izSMe1y1cK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-150.5605,62.9445"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "earthquake" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 61976, "friends_count": 10, "statues_count": 93940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081413929013248, "text": "The Gospel According to Mac ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62158287, "name": "Meganleigh", "screen_name": "itsmeganleigh", "lang": "en", "location": "Alaska | California", "create_at": date("2009-08-01"), "description": "#Donsquad x EJVT❤️", "followers_count": 453, "friends_count": 198, "statues_count": 18312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081414092570624, "text": "United States is the best damn country in the world if you ask me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 973192597, "name": "Zach Pillar", "screen_name": "zpillar", "lang": "en", "location": "snapchat- zpillar", "create_at": date("2012-11-26"), "description": "US OF A", "followers_count": 751, "friends_count": 470, "statues_count": 19724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081414193287168, "text": "@AlexatalevAlexa @KeithUrban Me too. I can't watch it without covering my eyes half way through the video", "in_reply_to_status": 690080594500423682, "in_reply_to_user": 2258442060, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2258442060, 14790817 }}, "user": { "id": 3238273871, "name": "TonightIWannaCry", "screen_name": "LuvinMyAussie", "lang": "en", "location": "UrbanLand", "create_at": date("2015-05-05"), "description": "I have been a fan of @keithurban for as long as I can remember. I will do anything for this Blue Eyed Baby. I love him so much....", "followers_count": 513, "friends_count": 716, "statues_count": 6888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:04.000Z"), "id": 690081414335848455, "text": "@marissausage @Dianavinuezzza Diana say yes", "in_reply_to_status": 690081253320691712, "in_reply_to_user": 228870850, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 228870850, 194781494 }}, "user": { "id": 48235094, "name": "Bobby Lucy", "screen_name": "bobby_lucy", "lang": "en", "location": "Fabulous Las Vegas, NV", "create_at": date("2009-06-17"), "description": "IG: bobbylucy // Sound + People + Cats", "followers_count": 1041, "friends_count": 578, "statues_count": 13475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081414755291136, "text": "@_TreGlizzy Lhh what ?", "in_reply_to_status": 690080757159743489, "in_reply_to_user": 2378158539, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 2378158539 }}, "user": { "id": 394255032, "name": "Sc:Uknowkels", "screen_name": "mikalshanay", "lang": "en", "location": "null", "create_at": date("2011-10-19"), "description": "RIPLilSteve ❤️ Follow Me on IG: Shanay____", "followers_count": 635, "friends_count": 483, "statues_count": 50162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hazelwood, MO", "id": "17674be6b5855123", "name": "Hazelwood", "place_type": "city", "bounding_box": rectangle("-90.421791,38.758992 -90.316075,38.812673") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2931276, "cityName": "Hazelwood" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081414843371520, "text": "Bored and up what to do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 431649166, "name": "Marquis Alston", "screen_name": "Supr_Rasta", "lang": "en", "location": "null", "create_at": date("2011-12-08"), "description": "Danger is very real but Fear is a choice", "followers_count": 647, "friends_count": 770, "statues_count": 50538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081414910459905, "text": "I see why in the movie book of Eli niggas was going crazy for the bible", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2691477411, "name": "She Loving the CREW", "screen_name": "1_day12", "lang": "en", "location": "Dallas, TX", "create_at": date("2014-07-09"), "description": "Your hate is all my love. #RIP Holt #atu 17", "followers_count": 776, "friends_count": 589, "statues_count": 905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Russellville, AR", "id": "c09505551daee91a", "name": "Russellville", "place_type": "city", "bounding_box": rectangle("-93.205955,35.220654 -93.056961,35.336506") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5115, "countyName": "Pope", "cityID": 561670, "cityName": "Russellville" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081415187337216, "text": "#SuspiciousIncident at 6812-6847 River Oaks Dr, Pine Hills, FL 32818. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4721591,28.5537177"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousIncident", "orlpol", "ocso" }}, "user": { "id": 39133730, "name": "Police Calls 32818", "screen_name": "orlpol32818", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 47, "friends_count": 1, "statues_count": 7787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504957,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081415396999168, "text": "@madisynpoops a, u, and o could use some definite work. Why are they all slanted?", "in_reply_to_status": 690080412484423680, "in_reply_to_user": 54769403, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 54769403 }}, "user": { "id": 160001707, "name": "Jon Edwards", "screen_name": "Jon_Pantaloon", "lang": "en", "location": "The University of Oklahoma ", "create_at": date("2010-06-26"), "description": "Carpe diem. Seize the day, boys. Make your lives extraordinary.", "followers_count": 787, "friends_count": 575, "statues_count": 38189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081415455703040, "text": "@jyy___ https://t.co/fWbm8DKHfS", "in_reply_to_status": 690081324149936128, "in_reply_to_user": 312899191, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 312899191 }}, "user": { "id": 208792853, "name": "brit the goat", "screen_name": "_sacredHeart", "lang": "en", "location": "with Lyre somewhere ", "create_at": date("2010-10-27"), "description": "don't dm me .. Trina mentioned me on 12/29/15 #DoitforJuan |#UL19| @OBJ_3 ❤️| RIP Uncle", "followers_count": 3870, "friends_count": 2823, "statues_count": 104873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081415959023617, "text": "I wish there was somebody I can just vent to for like a straight hour and that will just agree with me and not give me there opinions lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544654712, "name": "v❣", "screen_name": "vanessanicollee", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-04-03"), "description": "not ur homie RJ ❤", "followers_count": 975, "friends_count": 777, "statues_count": 45647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081416013545472, "text": "The cat's got my #tongue! Damn he was a good card player! #animals #feline", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tongue", "animals", "feline" }}, "user": { "id": 1485809486, "name": "Freddie Walker", "screen_name": "cuezerdo1957", "lang": "en", "location": "null", "create_at": date("2013-06-05"), "description": "I do one liners. Penning my magnum opus. Groovy.", "followers_count": 139, "friends_count": 40, "statues_count": 7615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081416772730880, "text": "So bored ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1244955326, "name": "PrincessCassie", "screen_name": "_Princess_Cass", "lang": "en", "location": "null", "create_at": date("2013-03-05"), "description": "long nights x beardown x", "followers_count": 67, "friends_count": 128, "statues_count": 895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081416932126720, "text": "Bitch shuddup https://t.co/kFU2xLn4PV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 820828591, "name": "Rocky D Boa", "screen_name": "dontelmartinez1", "lang": "en", "location": "The Jungle", "create_at": date("2012-09-12"), "description": "Ima Rapper Have Dreams! Jungle Boyz Ent! I Gotta Stay Healthy On A Punk Bitch|jungle boyzENT!!! For Bookings and Features Contact @mylesj151@gmail.com", "followers_count": 1077, "friends_count": 745, "statues_count": 15843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parsons, KS", "id": "0a3e119020705b64", "name": "Parsons", "place_type": "city", "bounding_box": rectangle("-95.297025,37.321136 -95.2296,37.364043") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20099, "countyName": "Labette", "cityID": 2054675, "cityName": "Parsons" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081417255088128, "text": "@MrGarzaaa @justbenny5 I heard they don't have squat racks or bench presses", "in_reply_to_status": 690081138610683904, "in_reply_to_user": 170223666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 170223666, 2643288782 }}, "user": { "id": 444157393, "name": "...", "screen_name": "Loooou__GGie", "lang": "en", "location": "null", "create_at": date("2011-12-22"), "description": "95(z) (562)", "followers_count": 1067, "friends_count": 896, "statues_count": 30322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081417661923329, "text": "Subway flatbreads are everything. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93909308, "name": "Elisabeth Lee", "screen_name": "Elisabeth1014", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-12-01"), "description": "null", "followers_count": 294, "friends_count": 259, "statues_count": 13706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081417733214208, "text": "@Lizzy_Rubio what song?? Lol", "in_reply_to_status": 690081011502354432, "in_reply_to_user": 1110428204, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1110428204 }}, "user": { "id": 4130570060, "name": "Ivan", "screen_name": "IVAN199X", "lang": "en", "location": "null", "create_at": date("2015-11-04"), "description": "I got my own thing I wanna do, You just respect it.", "followers_count": 91, "friends_count": 174, "statues_count": 389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081417850679297, "text": "My fav is when ppl don't reply 2 me!!!!!!!!! ❤️❤️❤️❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 984207020, "name": "Avs", "screen_name": "AvaTompkins", "lang": "en", "location": "AK", "create_at": date("2012-12-02"), "description": "life's a mess fam #ijustwannaball", "followers_count": 991, "friends_count": 361, "statues_count": 12079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081418043613184, "text": "Columbus traveled the world in a sailboat..... And you can't get to the gym? #tellmemore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8949635,33.85074564"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tellmemore" }}, "user": { "id": 3289545837, "name": "Ismael Arellano", "screen_name": "IsmaArllano", "lang": "en", "location": "null", "create_at": date("2015-05-18"), "description": "I like food :) #Fitness", "followers_count": 16, "friends_count": 3, "statues_count": 82 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081418555318273, "text": "keep thinking of how my girlfriends boobs will get bigger when she has my kids. they're already huge af, ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67289037, "name": "fuck off", "screen_name": "bblacksabbath", "lang": "en", "location": "90063", "create_at": date("2009-08-20"), "description": "FTP - A$AP - DoomSayersClub - AFTERdeath- #LinemanToBe - nenetto frito", "followers_count": 1045, "friends_count": 341, "statues_count": 104229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081418593071105, "text": "im tired asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2440920122, "name": "achante.", "screen_name": "achant3_", "lang": "en", "location": "null", "create_at": date("2014-04-12"), "description": "just trying to make it...", "followers_count": 1249, "friends_count": 722, "statues_count": 12564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shively, KY", "id": "edbc1ac7f306fad2", "name": "Shively", "place_type": "city", "bounding_box": rectangle("-85.851605,38.170899 -85.78059,38.220778") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2170284, "cityName": "Shively" } }
+{ "create_at": datetime("2016-01-21T00:00:05.000Z"), "id": 690081418614038528, "text": "1 shot and 4 drinks Later I was trying take someone mom home and she was flirting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55453741, "name": "TED GRIZZLY", "screen_name": "CJAY_XD", "lang": "en", "location": "Decatur", "create_at": date("2009-07-09"), "description": "#KennesawState #E46 Enjoy My Company...", "followers_count": 1479, "friends_count": 1161, "statues_count": 61263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennesaw, GA", "id": "c4cd1df048841e00", "name": "Kennesaw", "place_type": "city", "bounding_box": rectangle("-84.64724,33.955258 -84.577544,34.066895") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1343192, "cityName": "Kennesaw" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081418903445504, "text": "Temp: 11.7°F - Dew Point: 4.8° - Wind: 1.0 mph - Gust: 1.3 - Rain Today: 0.00in. - Pressure: 30.30in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081419062841346, "text": "with milk, in my bowl", "in_reply_to_status": 690081033182711808, "in_reply_to_user": 1110182340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1110182340, "name": "Kyle Sumida", "screen_name": "suhmeduh", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-01-21"), "description": "music is my outlet. PHHS '16", "followers_count": 600, "friends_count": 556, "statues_count": 27237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081419624878081, "text": "I'm bored asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1727655210, "name": "grim", "screen_name": "Erik187_", "lang": "en", "location": "LA", "create_at": date("2013-09-03"), "description": "GOLF", "followers_count": 346, "friends_count": 300, "statues_count": 4634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081420409192449, "text": "17 “@yabitchdaia: 19. What's your fav number?”", "in_reply_to_status": 683092587863879680, "in_reply_to_user": 1598647981, "favorite_count": 0, "coordinate": point("-118.2535692,33.85505937"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1598647981 }}, "user": { "id": 594306460, "name": "Ayo", "screen_name": "ElijahSho", "lang": "en", "location": "Carson, CA", "create_at": date("2012-05-29"), "description": "#tracknation | #LakerNation #Loading... | Nigerian | Ruth ❤️", "followers_count": 1035, "friends_count": 398, "statues_count": 98887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081420505669632, "text": "Why do I think?¿¿", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3194861958, "name": "jiggy jess", "screen_name": "jessygrayflores", "lang": "en", "location": "null", "create_at": date("2015-05-13"), "description": "I", "followers_count": 212, "friends_count": 159, "statues_count": 2534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081420920905728, "text": "@GaemGyu 저도 스키를 타고 싶은데 운동을 잘하지 않아서 좀 위험하다고 생각했어요. 나중에 한국에 다시 가면 스키를 한번 타야 돼요!", "in_reply_to_status": 686900654531477504, "in_reply_to_user": 135527082, "favorite_count": 0, "retweet_count": 0, "lang": "ko", "is_retweet": false, "user_mentions": {{ 135527082 }}, "user": { "id": 1668255780, "name": "RubyanneYip", "screen_name": "RubyanneYip", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-08-13"), "description": "Let it be.", "followers_count": 3, "friends_count": 11, "statues_count": 14 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mesa, CA", "id": "c8ccc9439a8e5ee0", "name": "La Mesa", "place_type": "city", "bounding_box": rectangle("-117.053546,32.743581 -116.981714,32.7958") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 640004, "cityName": "La Mesa" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081421310976001, "text": "@Norah_Nova @NickeyHuntsman @RileyReidx3 @Ce_Talent looks like you all gonna have a fun time.", "in_reply_to_status": 690054946805432320, "in_reply_to_user": 2582114119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2582114119, 482383186, 301779034, 2999833859 }}, "user": { "id": 603765069, "name": "Christopher S Lewis", "screen_name": "CslChef830", "lang": "en", "location": "Burke, Va.", "create_at": date("2012-06-09"), "description": "null", "followers_count": 219, "friends_count": 679, "statues_count": 16593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burke, VA", "id": "4769ebcaeee50421", "name": "Burke", "place_type": "city", "bounding_box": rectangle("-77.327307,38.745249 -77.227003,38.841994") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5111464, "cityName": "Burke" } }
+{ "create_at": datetime("2016-01-21T00:00:06.000Z"), "id": 690081421755572225, "text": "Wind 1.1 mph WNW. Barometer 30.103 in, Steady. Temperature 15.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081423227748353, "text": "@oddbree should message me inspirational shit in the am to motivate me lmfao", "in_reply_to_status": -1, "in_reply_to_user": 2411425688, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2411425688 }}, "user": { "id": 259989099, "name": "Nicki", "screen_name": "Nicole_aec", "lang": "en", "location": "null", "create_at": date("2011-03-02"), "description": "La vie est Audrey Bellè", "followers_count": 363, "friends_count": 406, "statues_count": 23060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081423642992640, "text": "If I could show Oomf This wasn't a game I would", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2265672703, "name": "T", "screen_name": "BennettTamarcus", "lang": "en", "location": "Texas, USA", "create_at": date("2013-12-28"), "description": "If you love yourself you can never be alone #bowlme sc:yo-boi1", "followers_count": 1047, "friends_count": 1710, "statues_count": 12554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victoria, TX", "id": "70f5587b3e27a105", "name": "Victoria", "place_type": "city", "bounding_box": rectangle("-97.045657,28.709293 -96.900168,28.895775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48469, "countyName": "Victoria", "cityID": 4875428, "cityName": "Victoria" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081424691576832, "text": "@dawgfansteve64 @TracySorrell1 @TitoJazavac @paradiselost_17\nI agree, but he doesn't really fit into Chip's system.", "in_reply_to_status": 690081183921758208, "in_reply_to_user": 2822640558, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2822640558, 822154424, 2509208545, 2615065766 }}, "user": { "id": 615599662, "name": "Joe Montes", "screen_name": "JoeMon49", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2012-06-22"), "description": "Political junkie - Liberal (left of dems) Women's, LGBT, Voter's & Animal rights advocate -Atheist - Sports: 49ers, SF Giants, OR Ducks- Music: Rock, mostly 90s", "followers_count": 3187, "friends_count": 2173, "statues_count": 50674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081425136185345, "text": "My ex used to call me a whore & stuff all the time \nIf only he could see me now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842621053, "name": "xochiquetzal", "screen_name": "venuseos", "lang": "en", "location": "Tepatitlán de Morelos, Jalisco", "create_at": date("2012-09-23"), "description": "What do you mean you're Shia? Like LaBeouf?", "followers_count": 1464, "friends_count": 535, "statues_count": 87080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawndale, CA", "id": "cce33d74ceffbe08", "name": "Lawndale", "place_type": "city", "bounding_box": rectangle("-118.369186,33.872914 -118.343796,33.902665") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640886, "cityName": "Lawndale" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081425194946560, "text": "Wind 0.0 mph ---. Barometer 30.016 in, Falling. Temperature 51.5 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 23, "statues_count": 57611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081425345921024, "text": "Being a bitch will get you no where in life I promise", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 868678190, "name": "Cj Page", "screen_name": "pagestyle12", "lang": "en", "location": "Roseburg North, OR", "create_at": date("2012-10-08"), "description": "21 years old. snapchat: cjpage12 // insta: pagestyle12", "followers_count": 956, "friends_count": 590, "statues_count": 11366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseburg North, OR", "id": "434e58efb2dc6df0", "name": "Roseburg North", "place_type": "city", "bounding_box": rectangle("-123.36045,43.236187 -123.327482,43.286851") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41019, "countyName": "Douglas", "cityID": 4163660, "cityName": "Roseburg North" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081425371090944, "text": "1/21/2016 - 02:00\nTemp: 32.3F \nHum: 97%\nWind: 1.0 mph\nBaro: 30.128in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 54, "statues_count": 49217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081425475932160, "text": "I KNOW YOU LYING �������������� https://t.co/lVNFm5rHzN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277487036, "name": "Snëaux Whyte™", "screen_name": "iChrissyLTE", "lang": "en", "location": "Houston, TX", "create_at": date("2011-04-05"), "description": "Bitch I'm saucin.. Houston im here •IG/SC:iChrissyLTE⭐", "followers_count": 1786, "friends_count": 1262, "statues_count": 185872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081426759360513, "text": "And this is why I don't really like drinking anymore I get all sentimental and I want to text the one person I shouldn't ������\n#help", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.6169201,26.0190362"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "help" }}, "user": { "id": 599869515, "name": "Gracie❤", "screen_name": "GracieTovar", "lang": "en", "location": "null", "create_at": date("2012-06-04"), "description": "null", "followers_count": 105, "friends_count": 125, "statues_count": 1050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-01-21T00:00:07.000Z"), "id": 690081427157860352, "text": "@Jrrrr_Muriilllo go head den ex bestfriend", "in_reply_to_status": 690081145015422976, "in_reply_to_user": 2300148158, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2300148158 }}, "user": { "id": 1193950766, "name": "Mo'Money", "screen_name": "msharresse_", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "| RIP Leroy Fennoy Jr |", "followers_count": 1879, "friends_count": 519, "statues_count": 103178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-01-21T00:00:08.000Z"), "id": 690081427443052545, "text": "#AttemptedSuicide at 100 S HUGHEY AV. #orlpol #opd #opdid201600028906", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3834057,28.5412202"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AttemptedSuicide", "orlpol", "opd", "opdid201600028906" }}, "user": { "id": 3380696632, "name": "Orlando Police HQ", "screen_name": "orlpol_opdhq", "lang": "en", "location": "Orlando-ish", "create_at": date("2015-07-17"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 19, "friends_count": 1, "statues_count": 4179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-21T00:00:08.000Z"), "id": 690081427636011009, "text": "I need to study my math more often. I used to be brilliant.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322462924, "name": "Alejandrito", "screen_name": "alejandrito_93", "lang": "en", "location": "Lathrop, CA", "create_at": date("2011-06-22"), "description": "Mucho amor tengo al deporte de la charreria, la musica, la comida, y los #SFGiants #49ers", "followers_count": 323, "friends_count": 222, "statues_count": 51453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443781666140161, "text": "this may be crazy but in my personal opinion..protective people are the best type of people to be in a relationship with. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2549040942, "name": "kiks", "screen_name": "Wildflower24_", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2014-06-05"), "description": "null", "followers_count": 273, "friends_count": 841, "statues_count": 1541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443782144176128, "text": "Meet the legendary Nicko McBrain from IRON MADEN \"Drummer\" \\m/...… https://t.co/u0XQrB4ZRO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.9207077,33.8023415"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543681933, "name": "AidinDavoudiOfficial", "screen_name": "AidinDavoudi", "lang": "en", "location": "Carlsbad, CA", "create_at": date("2012-04-02"), "description": "MUSIC COMPOSER / AUDIO ENGINEER, SOUND DESIGNER / SINGER, MUSIC PRODUCER", "followers_count": 176, "friends_count": 47, "statues_count": 174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443782211440644, "text": "And another movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1682075341, "name": "20", "screen_name": "sanchez996", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-08-18"), "description": "Long live the king", "followers_count": 247, "friends_count": 305, "statues_count": 14688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443782626635776, "text": "@PenguinSix ���� Good job. What's the recipe?", "in_reply_to_status": 690442984484179968, "in_reply_to_user": 1429751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1429751 }}, "user": { "id": 3941097740, "name": "flamingrickshaw", "screen_name": "flames73", "lang": "en", "location": "Torrance, CA", "create_at": date("2015-10-18"), "description": "null", "followers_count": 21, "friends_count": 54, "statues_count": 608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443782794403840, "text": "Aghhhhhhhhhhhhhhhh...!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2187934190, "name": "Krista", "screen_name": "Krista_Kisabeth", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-11-11"), "description": "Proud Napian.", "followers_count": 304, "friends_count": 511, "statues_count": 3460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443783159312389, "text": "in the end everything will be okay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3125515588, "name": "ari", "screen_name": "weltearianne", "lang": "en", "location": "null", "create_at": date("2015-03-28"), "description": "basically all I do is read and watch tv shows so yeah", "followers_count": 132, "friends_count": 180, "statues_count": 1635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, NE", "id": "00b6bac82856d70b", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-96.014224,41.089139 -95.867612,41.191076") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3103950, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443784161738752, "text": "I cant wait to see Thai tmo��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318314043, "name": "kiyaaaaah", "screen_name": "Kiyaaaaah", "lang": "en", "location": "The Baaaaayy", "create_at": date("2011-06-16"), "description": "null", "followers_count": 307, "friends_count": 197, "statues_count": 15900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443784165969920, "text": "my bf when he was singing the alphabet ahhaaaaha������ I still laugh every time !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311303821, "name": "Hannah", "screen_name": "jusst_hannah", "lang": "en", "location": "null", "create_at": date("2011-06-04"), "description": "null", "followers_count": 524, "friends_count": 262, "statues_count": 18856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443784497266689, "text": "Ass muscles �� https://t.co/WNuWaa1EjM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65559069, "name": "jorge.", "screen_name": "OprahsBae", "lang": "en", "location": "probabaly at home", "create_at": date("2009-08-13"), "description": "null", "followers_count": 282, "friends_count": 240, "statues_count": 6741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443784736391169, "text": "my dad is telling us to drink more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550046073, "name": "Arabella", "screen_name": "AurorabaeArah", "lang": "en", "location": "null", "create_at": date("2012-04-10"), "description": "cali", "followers_count": 128, "friends_count": 139, "statues_count": 8686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-22T00:00:00.000Z"), "id": 690443785499758593, "text": "@LLymytran @ona_diomampo First year anniversay ng BCWMH--JOCHARD CHARM", "in_reply_to_status": 690404635119632384, "in_reply_to_user": 2940397632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2940397632, 2177869784 }}, "user": { "id": 3114260312, "name": "rolita acosta blanco", "screen_name": "RolitaBlanco", "lang": "en", "location": "null", "create_at": date("2015-03-28"), "description": "null", "followers_count": 91, "friends_count": 20, "statues_count": 15651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443785763999745, "text": "@_kaydeeblood herbbbbb", "in_reply_to_status": 665061591545724928, "in_reply_to_user": 301758325, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301758325 }}, "user": { "id": 301758325, "name": "KaydeeFineAss", "screen_name": "_kaydeeblood", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "RestInParadiseTrey❤️ #SU19 iG: kasiaaRosee Sc: kaydeeblood....", "followers_count": 2436, "friends_count": 994, "statues_count": 67236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443786024001536, "text": "#latenightconfessions just made a new neopets account & it's exactly the same as it was 10 years ago and I'll probably be addicted again tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "latenightconfessions" }}, "user": { "id": 933056604, "name": "I know a KY", "screen_name": "kybair", "lang": "en", "location": "null", "create_at": date("2012-11-07"), "description": "null", "followers_count": 142, "friends_count": 120, "statues_count": 2242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443786523123712, "text": "I think Ima have a kick back just for the fuck of it this weekend ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1388587790, "name": "mariaa", "screen_name": "mariabbabby", "lang": "en", "location": "null", "create_at": date("2013-04-28"), "description": "josh", "followers_count": 753, "friends_count": 796, "statues_count": 30444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443786732765184, "text": "@_Weezylooney was Trey holding it bc that's his hand �� you kno I can tell ��������", "in_reply_to_status": 690443470117412864, "in_reply_to_user": 705864374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 705864374 }}, "user": { "id": 547862043, "name": "tyga |||", "screen_name": "catexotwod", "lang": "en", "location": "u just wanna fill the void now", "create_at": date("2012-04-07"), "description": "xo lmfao", "followers_count": 710, "friends_count": 432, "statues_count": 27888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443787630280704, "text": "01/22@03:00 - Temp 24.0F, WC 24.0F. Wind 0.3mph W, Gust 2.0mph. Bar 30.319in, Falling slowly. Rain 0.00in. Hum 85%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443787739467777, "text": "@ebbtideapp Tide in Delaware City, Delaware 01/22/2016\n Low 4:04am -0.2\nHigh 9:48am 5.9\n Low 4:51pm -0.2\nHigh 10:12pm 5.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.5883,39.5817"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 6, "friends_count": 1, "statues_count": 1191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1019730, "cityName": "Delaware City" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443787815014400, "text": "@_BabyGirlPretty act like u know bitch �� https://t.co/JIThIMQAVg", "in_reply_to_status": 690443189703020544, "in_reply_to_user": 471378714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 471378714 }}, "user": { "id": 414304133, "name": "darling nikki✨", "screen_name": "NoHablaFXCKSHXT", "lang": "en", "location": "finding myself.", "create_at": date("2011-11-16"), "description": "young and ambitious with big dreams ✨ she believed she could so she did. | your future nurse anesthetist ✂ | #AUM19 | #RipTevin | #LadyJohnson @CloutDemonGotti", "followers_count": 2011, "friends_count": 994, "statues_count": 231493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-88.446362,32.284593 -84.296982,41.744901") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443787856920577, "text": "I NEED SLEEP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2398484768, "name": "/\\/ a y a", "screen_name": "NayaFlowers", "lang": "en", "location": "The Bay, CA", "create_at": date("2014-03-19"), "description": "'Nah just was looking out for u' ~ |-/\\", "followers_count": 1307, "friends_count": 1391, "statues_count": 32512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443787982741505, "text": "Happy birthday bitch I love you �� so glad you're my friend (not really) and we have a special friendship that now one gets �� @jaaanerose", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2796714362 }}, "user": { "id": 2794493366, "name": "gerardo leyva", "screen_name": "HeyItssJerry", "lang": "en", "location": "null", "create_at": date("2014-09-06"), "description": "null", "followers_count": 123, "friends_count": 79, "statues_count": 1458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443788121149440, "text": "lame lame lame lame lame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2203529196, "name": "Asia", "screen_name": "Superfleyes", "lang": "en", "location": "California, USA", "create_at": date("2013-11-19"), "description": "PSN : Superleyes | I like my life | Inderkum", "followers_count": 242, "friends_count": 222, "statues_count": 6369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443788402233344, "text": "I'm going back to school https://t.co/CPBdr6YhNh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2470093650, "name": "Doo Dirty", "screen_name": "KaizerB3", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-04-29"), "description": "| CA✈️PA | Lafayette College '19 | | Football | | Armor Plated | Snapchat: kaizer_butler | #SM | Tanö |", "followers_count": 403, "friends_count": 316, "statues_count": 2019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443788884516864, "text": "Happy birthday brother! @RoyceCharms it's been a long 4-5 years but I've enjoyed every minute of it, hope all goes well today!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60246609 }}, "user": { "id": 1051720657, "name": "Justin Keller", "screen_name": "USA_Keller", "lang": "en", "location": "United States of America,LB,CA", "create_at": date("2012-12-31"), "description": "My tweets are not a valid refrence to see what I agree with, opinions/views change. I aim to be someone's puppet.", "followers_count": 275, "friends_count": 161, "statues_count": 37633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443789169758209, "text": "0h 5m wait time at JFK MEDICAL CENTER. Browse wait times or share your own with #WaitShare at https://t.co/rv1fffVDIf!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.091313,26.597052"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 375, "friends_count": 888, "statues_count": 2712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lantana, FL", "id": "07bf758651a4bc76", "name": "Lantana", "place_type": "city", "bounding_box": rectangle("-80.105173,26.569606 -80.037365,26.609384") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1239375, "cityName": "Lantana" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443789215875072, "text": "Damn I miss track & gymnastics...things that really made me happy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519043990, "name": "Liyah B", "screen_name": "Weirdaszgirl", "lang": "en", "location": "null", "create_at": date("2012-03-08"), "description": "Indian & black. #PV19", "followers_count": 2929, "friends_count": 1006, "statues_count": 71722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443789291409412, "text": "Hate my temper man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320893080, "name": "P Money", "screen_name": "pman_56", "lang": "en", "location": "null", "create_at": date("2011-06-20"), "description": "life is short and then you die!!!", "followers_count": 404, "friends_count": 268, "statues_count": 12564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, KY", "id": "6955c484d8d2708d", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-84.815962,37.603342 -84.731214,37.675405") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21021, "countyName": "Boyle", "cityID": 2119882, "cityName": "Danville" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443789329174528, "text": "maybe I should hurry up & take a shower so I could sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3118177856, "name": "ŁŸ", "screen_name": "alyiahhteresaa", "lang": "en", "location": "josh heredia", "create_at": date("2015-03-30"), "description": "I love myself & my 3 friends", "followers_count": 427, "friends_count": 547, "statues_count": 6815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fowler, CA", "id": "002a82ff3d3b9828", "name": "Fowler", "place_type": "city", "bounding_box": rectangle("-119.692831,36.597948 -119.637458,36.642439") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 625436, "cityName": "Fowler" } }
+{ "create_at": datetime("2016-01-22T00:00:01.000Z"), "id": 690443789652115456, "text": "My friends and I were about to go on an adventure until we saw some guy performing an exorcism ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57382441, "name": "seyar", "screen_name": "seyarthegemini", "lang": "en", "location": "A-WOOD", "create_at": date("2009-07-16"), "description": "#RaiderNation | Ardenwood Gang ♿️", "followers_count": 418, "friends_count": 427, "statues_count": 15671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443790318960640, "text": "�� https://t.co/WC9EWda1CT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 232306319, "name": "Gin", "screen_name": "ggonza4", "lang": "en", "location": "⚽408. nssj", "create_at": date("2010-12-30"), "description": "null", "followers_count": 633, "friends_count": 855, "statues_count": 62222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443790369206272, "text": "@brookesouthard1 Paige wanted to be in the vid so bad��", "in_reply_to_status": 690442858503995392, "in_reply_to_user": 2916869709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2916869709 }}, "user": { "id": 251396090, "name": "Soph", "screen_name": "SOPHIEisLEGITT", "lang": "en", "location": "null", "create_at": date("2011-02-12"), "description": "University of Arkansas 2019", "followers_count": 441, "friends_count": 378, "statues_count": 7820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, NC", "id": "4548df3d99bb4891", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-94.259864,34.936245 -78.808883,36.148962") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3722920, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443790461575169, "text": "@SIEDAHGARRETT @jadapsmith ....Amen... Look very good.", "in_reply_to_status": 690383838380462084, "in_reply_to_user": 45038044, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45038044, 46306646 }}, "user": { "id": 2912906940, "name": "DeChundria Bonner", "screen_name": "dechundria", "lang": "en", "location": "Texas, USA", "create_at": date("2014-11-28"), "description": "null", "followers_count": 88, "friends_count": 222, "statues_count": 1341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Angleton, TX", "id": "fe6fdbea8b8a1476", "name": "Angleton", "place_type": "city", "bounding_box": rectangle("-95.454639,29.10696 -95.390443,29.205233") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4803264, "cityName": "Angleton" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443791149461505, "text": "Temp: 55.4°F Wind:0.0mph Pressure: 29.668hpa Falling Rapidly Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443791258554369, "text": "Wind 6.0 mph ENE. Barometer 30.147 in, Falling. Temperature 23.8 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 49, "friends_count": 25, "statues_count": 17056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443791405309952, "text": "Wind 1.0 mph NW. Barometer 30.194 in, Falling. Temperature 22.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443791824744448, "text": "Wind 9.0 mph N. Barometer 1025.77 mb, Steady. Temperature 24.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443791950581761, "text": "I need my braids ☹", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340185682, "name": "SLAYomi", "screen_name": "_simbuh", "lang": "en", "location": "ny :: dirty jerz ", "create_at": date("2011-07-22"), "description": "big dreams, big ideas... bigger hair", "followers_count": 2737, "friends_count": 1630, "statues_count": 140125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle, NJ", "id": "888a254ab33a9e2e", "name": "Roselle", "place_type": "city", "bounding_box": rectangle("-74.285836,40.637241 -74.233501,40.663631") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3464620, "cityName": "Roselle" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443792013459456, "text": "42.7F (Feels: 42.7F) - Humidity: 82% - Wind: 19.7mph NW - Gust: 25.9mph - Pressure: 1028.0mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 223498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443792038494208, "text": "Wind 0.0 mph NNW. Barometer 30.275 in, Steady. Temperature 12.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443792944603136, "text": "@Crudes but seriously get to bed dude", "in_reply_to_status": 690443429252214785, "in_reply_to_user": 1936490083, "favorite_count": 0, "coordinate": point("-121.53271386,38.64357761"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1492124791 }}, "user": { "id": 1936490083, "name": "trevon (Lasich)", "screen_name": "lasich_", "lang": "en", "location": "Sacramento", "create_at": date("2013-10-04"), "description": "gt: KSI FANCYHANDS. Who knows bro :/ Snap chat: Lasich /multi fps snipester/ I just want nudes or a relationship fam. 99 subs strong I guess.", "followers_count": 254, "friends_count": 776, "statues_count": 4479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443793208864768, "text": "I'm bored af dawg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316735921, "name": "⛄️", "screen_name": "DarelBeChillin", "lang": "en", "location": "Kansas City ✈️ Austin/Denton", "create_at": date("2011-06-13"), "description": "Harlem shakin' through the pressure | #DILFHive", "followers_count": 1904, "friends_count": 904, "statues_count": 103542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-22T00:00:02.000Z"), "id": 690443793250779137, "text": "@noremaksnilloc in fact, I like Plumlee more than him", "in_reply_to_status": 690437079155752961, "in_reply_to_user": 28104080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28104080 }}, "user": { "id": 585391355, "name": "Seth Klein", "screen_name": "SethDaSportsMan", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-05-19"), "description": "Writer at @TheFantasyFix and @Razzball. DFS at @FantasyDraft. Ranker at FantasyPros. The Justin Timberlake of the @AllStarFiveNBA Team. Carla WAS the prom queen", "followers_count": 3422, "friends_count": 549, "statues_count": 117516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443794198626304, "text": "@_Caarrlooss highkey*", "in_reply_to_status": 690440304894214144, "in_reply_to_user": 124066942, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 124066942 }}, "user": { "id": 544110003, "name": "Doughboy", "screen_name": "Josaaaaaaaaaaay", "lang": "en", "location": "Arizona State University ", "create_at": date("2012-04-02"), "description": "Veana | ZBT", "followers_count": 847, "friends_count": 666, "statues_count": 39029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443794668490752, "text": "I really want that purse ��❣", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3853751839, "name": "V.", "screen_name": "ValerieVianey", "lang": "en", "location": "McAllen, TX", "create_at": date("2015-10-10"), "description": "null", "followers_count": 149, "friends_count": 147, "statues_count": 1483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443794685239296, "text": "@p0pson @yungsteez_", "in_reply_to_status": 689677153995210752, "in_reply_to_user": 2729829980, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2729829980, 1486808492 }}, "user": { "id": 324562044, "name": "Dirk Diggler", "screen_name": "digglerpiece", "lang": "en", "location": "a garage ", "create_at": date("2011-06-26"), "description": "the (after) life of the party", "followers_count": 455, "friends_count": 327, "statues_count": 12890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443794714595328, "text": "We all know the guy who dances on the dance floor by them selfs. Like we don't really know him but you all feel sad for them.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3282212383, "name": "Jesse Martinez", "screen_name": "JBirdstandup", "lang": "en", "location": "Chula Vista, CA", "create_at": date("2015-07-16"), "description": "null", "followers_count": 9, "friends_count": 124, "statues_count": 59 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443794727182336, "text": "Held It Down 2Night : Wanna Thank Everyabody For Xoming Out An Showing Me Love : Book All Shows… https://t.co/6JmjBmlFWu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3965157,33.75073185"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132560647, "name": "IG StreetMoney5lug", "screen_name": "15lugGa", "lang": "en", "location": "504/404", "create_at": date("2010-04-13"), "description": "New Orleans. (SAINTS) coming soon Sends Beats To (StreeTmoney5lug@gmail) +1 (470) 385-9577 (Booking contact: @7flyy for any features or show request", "followers_count": 2160, "friends_count": 804, "statues_count": 50433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443795024994304, "text": "#SupportOriginMelissa 25.7°F Wind:3.1mph Pressure: 30.00hpa Falling Rain Today 0.00in. Forecast: Fairly fine, showery later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443795054358528, "text": "@lanepoffical I WANT A DONUT SO BAD BUT I CAN'T CHEAT FOR ANOTHER WEEK BECAUSE I ALREADY CHEATED THIS WEEK IM SO SAD :(", "in_reply_to_status": 690443513536720896, "in_reply_to_user": 142530279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 142530279 }}, "user": { "id": 2926321389, "name": "Gabbi", "screen_name": "gabrielakurtzz", "lang": "en", "location": "null", "create_at": date("2014-12-16"), "description": "Kinesiology major•19• 8/30/15❤️", "followers_count": 315, "friends_count": 302, "statues_count": 2276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443795612172288, "text": "Wind 1.6 mph WNW. Barometer 30.29 in, Rising slowly. Temperature 14.4 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 96, "statues_count": 156880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443795834486784, "text": "smoking weed is such a gross and scrubby thing to me tbh.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100719419, "name": "mae ॐ", "screen_name": "inumillati", "lang": "en", "location": "niles, ohio", "create_at": date("2009-12-31"), "description": "you're amazing and I love you.\n♓✌", "followers_count": 1086, "friends_count": 86, "statues_count": 55686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niles, OH", "id": "de89d25be7d37a11", "name": "Niles", "place_type": "city", "bounding_box": rectangle("-80.811321,41.155863 -80.70751,41.22587") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3955916, "cityName": "Niles" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443795981279237, "text": "Wind 1.0 mph NNW. Barometer 30.345 in, Falling. Temperature 11.7 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 5965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443796002250752, "text": "@__Geeeee lol nah, I'm not offended. �� I just didn't want to over explain it to you.", "in_reply_to_status": 690443619472277504, "in_reply_to_user": 163993199, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163993199 }}, "user": { "id": 386288122, "name": "D'wayne, 闕色拉寺 ✨", "screen_name": "CornelivsW", "lang": "en", "location": "FANTASEA", "create_at": date("2011-10-06"), "description": "I'm bilingual. I speak BITCH, too.", "followers_count": 2414, "friends_count": 165, "statues_count": 87191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443796308496384, "text": "12th planet & lumberjvck killed Name Bran", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1096060044, "name": "FTP Love", "screen_name": "vdaawggg", "lang": "en", "location": "null", "create_at": date("2013-01-16"), "description": "be in ☮.. daily Tripper", "followers_count": 399, "friends_count": 434, "statues_count": 18276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443796417392640, "text": "Can you find Norristown on the map? Just try it at https://t.co/tBqnW540KF #Norristown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.3399,40.1215"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Norristown" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1015, "friends_count": 312, "statues_count": 2542258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norristown, PA", "id": "6b4972f8e32f4e32", "name": "Norristown", "place_type": "city", "bounding_box": rectangle("-75.365138,40.105217 -75.31664,40.141599") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4254656, "cityName": "Norristown" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443796488818688, "text": "Honestly https://t.co/fWec8KJbw3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3241806848, "name": "Lerual✨", "screen_name": "Nigerian__InMe", "lang": "en", "location": "Akwa Ibom, TX", "create_at": date("2015-06-10"), "description": "#AfroLuxeHair| Insta/SC: ayoshenigerian Dallas/PV/Nac", "followers_count": 1236, "friends_count": 620, "statues_count": 35674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443796635516929, "text": "My new London computer wiz filmmaker friend @paulguyenette… https://t.co/JDlKnMgKhV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.50758762,40.65107898"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238020265, "name": "Gigi Goyette-jeffers", "screen_name": "hnl2malibu", "lang": "en", "location": "Malibu, CA", "create_at": date("2011-01-13"), "description": "Entrepreneur of sorts..peace keeper, world traveler, planning consultant & Actor", "followers_count": 87, "friends_count": 235, "statues_count": 212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utah, USA", "id": "1879ace9e02ace61", "name": "Utah", "place_type": "admin", "bounding_box": rectangle("-114.052999,36.997905 -109.041059,42.001619") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49043, "countyName": "Summit", "cityID": 4958070, "cityName": "Park City" } }
+{ "create_at": datetime("2016-01-22T00:00:03.000Z"), "id": 690443798229487616, "text": "Can't wait till you come back best friend @melissatvrs ❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 118552411 }}, "user": { "id": 178233104, "name": "Lucy.", "screen_name": "LucyFlowerFlame", "lang": "en", "location": "null", "create_at": date("2010-08-13"), "description": "null", "followers_count": 871, "friends_count": 589, "statues_count": 35960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443798334156800, "text": "One of my favorite goalies ��⚽️ you did great! @lowsaay https://t.co/wOeCIZl1eN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2900174294 }}, "user": { "id": 2359204135, "name": "JR.", "screen_name": "jhanna_rae", "lang": "en", "location": "Oahu, Hawai'i", "create_at": date("2014-02-24"), "description": "null", "followers_count": 130, "friends_count": 160, "statues_count": 2065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443798443364352, "text": "Welp he sleep so it's bout that time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762395047, "name": "lazha.", "screen_name": "adorelazha", "lang": "en", "location": "null", "create_at": date("2012-08-16"), "description": "Instagram : fineasslazha", "followers_count": 775, "friends_count": 587, "statues_count": 54453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443798690795521, "text": "I just love running into this guy ❤️❤️❤️❤️ https://t.co/uGzLKr3rPL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612298443, "name": "Taylor F. Schwarz", "screen_name": "TaylorFSchwarz", "lang": "en", "location": "null", "create_at": date("2012-06-18"), "description": "The more you like yourself, the less you are like anyone else, which makes you unique. ---Walt Disney", "followers_count": 256, "friends_count": 495, "statues_count": 8087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irmo, SC", "id": "189ef793a6dbbfc8", "name": "Irmo", "place_type": "city", "bounding_box": rectangle("-81.257113,34.04725 -81.132587,34.170797") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4535890, "cityName": "Irmo" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443798871040000, "text": "Happy birthday to my day uno ���� @millerlyfe https://t.co/OEotcIoSwa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246041513 }}, "user": { "id": 838934773, "name": "Daddy Small Dick", "screen_name": "Savage_Andress", "lang": "en", "location": "Trap House ", "create_at": date("2012-09-21"), "description": "| HPHS Football |", "followers_count": 238, "friends_count": 212, "statues_count": 6173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443799387099137, "text": "03:00 22.1°F Feels:13.0°F (Hi23.0°F/Lo22.1°F) Hum:62% Wnd:NNE 7.7MPH Baro:30.07in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 170, "friends_count": 266, "statues_count": 24980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443799496105984, "text": "Havent had them yet!! https://t.co/6nj6nHXALb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105181079, "name": "Juan Antonio Acevedo", "screen_name": "WildPatriot", "lang": "en", "location": "Maryland, USA", "create_at": date("2010-01-15"), "description": "♢Host/Producer WPO/ Covey Pro Wrestling\n♢Food Fanatic\n♢Lego Collector\n♢Technical Director/Videographer for Univision Washington DC", "followers_count": 384, "friends_count": 1019, "statues_count": 7399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland City, MD", "id": "79fa65a7f7afb5f5", "name": "Maryland City", "place_type": "city", "bounding_box": rectangle("-76.84036,39.067793 -76.787783,39.12138") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2451075, "cityName": "Maryland City" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443799890378752, "text": "@AquariusOG I def need pizza lol :)", "in_reply_to_status": 690443606008664064, "in_reply_to_user": 1713842102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1713842102 }}, "user": { "id": 1015947230, "name": "Bbkat", "screen_name": "DanielleKGomez", "lang": "en", "location": "LA", "create_at": date("2012-12-16"), "description": "18|Dannikat|Scorpio", "followers_count": 774, "friends_count": 460, "statues_count": 32914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443800091717632, "text": "@drdreea14 you go girl ���� #isupport ��", "in_reply_to_status": 690441747839606784, "in_reply_to_user": 1112171390, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "isupport" }}, "user_mentions": {{ 1112171390 }}, "user": { "id": 2917153657, "name": "Abriana Morales", "screen_name": "ayy__briana", "lang": "en", "location": "null", "create_at": date("2014-12-02"), "description": "null", "followers_count": 111, "friends_count": 172, "statues_count": 7169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443800129490944, "text": "St. Mary's rallies past Gonzaga in WCC classic (CBS) | https://t.co/Ur5g44rbdx #ncaab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.623474,41.878789"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ncaab" }}, "user": { "id": 16941578, "name": "Basketball State", "screen_name": "bbstate", "lang": "en", "location": "Chicago", "create_at": date("2008-10-23"), "description": "Advanced statistics, analysis tools, historical data and news for D1 coaches, scouts, media and fans. Unmonitored account. Get in touch: https://t.co/lG39PMGAtr", "followers_count": 5954, "friends_count": 5640, "statues_count": 48959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443800246886402, "text": "one last happy birthday to my baby sis ���� @zara_lynn4 so much love for this brat �� https://t.co/UlWTDCZw69", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2273898360 }}, "user": { "id": 940482967, "name": "c-rissss", "screen_name": "c_rissaaa5", "lang": "en", "location": "mhs c/o '16", "create_at": date("2012-11-10"), "description": "c.p ❤️", "followers_count": 1179, "friends_count": 912, "statues_count": 17742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443800901259265, "text": "https://t.co/5unt8kPEjL @TheReaperTeam. Isis time is over. Its on. Seals every where.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-103.5031765,44.4040062"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4152931165 }}, "user": { "id": 122486457, "name": "al fogelberg", "screen_name": "oldsmobileal", "lang": "en", "location": "sturgis sd", "create_at": date("2010-03-12"), "description": "love cats,mountain lion activist. look for gems gold and big foot.Gamble in Deadwood.have a Puma guiding spirit named Willow.", "followers_count": 1791, "friends_count": 2221, "statues_count": 13127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sturgis, SD", "id": "84e52e736fc1f52e", "name": "Sturgis", "place_type": "city", "bounding_box": rectangle("-103.559247,44.386567 -103.489102,44.425387") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46093, "countyName": "Meade", "cityID": 4662100, "cityName": "Sturgis" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443800947376129, "text": "Never drinking whiskey at a show again\n\nBad idea", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312049460, "name": "You", "screen_name": "FinalxDoctrine", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-10"), "description": "21. Aspiring Meme Artist. Not Interested", "followers_count": 733, "friends_count": 544, "statues_count": 17825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443801098391552, "text": "STILL getting free drinks for my bday three weeks later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 956619642, "name": "taz", "screen_name": "RaeeLez", "lang": "en", "location": "City of Angels", "create_at": date("2012-11-18"), "description": "Smile for me. 〽", "followers_count": 190, "friends_count": 235, "statues_count": 8940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ynez, CA", "id": "7a013423c415ddd6", "name": "Santa Ynez", "place_type": "city", "bounding_box": rectangle("-120.123059,34.591504 -120.055084,34.636032") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 670182, "cityName": "Santa Ynez" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443801731592192, "text": "In all honesty I hate read receipts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 706123718, "name": "Shae ✨", "screen_name": "ShannonElaineeW", "lang": "en", "location": "Canyon, TX", "create_at": date("2012-07-19"), "description": "Do you know what a princess is? Yeah me.", "followers_count": 438, "friends_count": 617, "statues_count": 11005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon, TX", "id": "309059ff6710946f", "name": "Canyon", "place_type": "city", "bounding_box": rectangle("-101.954673,34.960525 -101.878133,35.017533") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48381, "countyName": "Randall", "cityID": 4812532, "cityName": "Canyon" } }
+{ "create_at": datetime("2016-01-22T00:00:04.000Z"), "id": 690443801828159489, "text": "Such a dope night.. https://t.co/QtD2CexIbZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2208891601, "name": "Marcus", "screen_name": "Marcusluna__", "lang": "en", "location": "null", "create_at": date("2013-11-22"), "description": "Cibolo// I look like that boy Jesus/ snapchat-marcusluna", "followers_count": 420, "friends_count": 470, "statues_count": 3072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cibolo, TX", "id": "af2cf12389d619de", "name": "Cibolo", "place_type": "city", "bounding_box": rectangle("-98.263751,29.556341 -98.19757,29.603081") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48187, "countyName": "Guadalupe", "cityID": 4814920, "cityName": "Cibolo" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443802679476224, "text": "@Pam__dv teach me ����", "in_reply_to_status": 690443701340995585, "in_reply_to_user": 53646595, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53646595 }}, "user": { "id": 20137956, "name": "世界を売った男", "screen_name": "dvminixk", "lang": "en", "location": "Denver, CO", "create_at": date("2009-02-05"), "description": "now and forever. #NT", "followers_count": 482, "friends_count": 476, "statues_count": 66788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443803560259584, "text": "Lmao! �������������� shut uuuup!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316515134, "name": "-Ki❤️", "screen_name": "__lilki", "lang": "en", "location": "Jonesboro, LA", "create_at": date("2011-06-13"), "description": "longliveSnupe", "followers_count": 1499, "friends_count": 832, "statues_count": 33116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, LA", "id": "7a305189b509c584", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-92.726864,32.212426 -92.686115,32.260875") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22049, "countyName": "Jackson", "cityID": 2238670, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443804273446914, "text": "@chardonnaymami @pitchfork @macklemore @RyanLewis @duhmilo like for real I just listened to the preview on iTunes, and I got sooooooooo HOT", "in_reply_to_status": 690443505710141440, "in_reply_to_user": 55765077, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55765077, 14089195, 18159470, 18007213, 528112270 }}, "user": { "id": 2172986738, "name": "scorpio ♏️", "screen_name": "Scorpioking_117", "lang": "en", "location": "null", "create_at": date("2013-11-03"), "description": "POC representative. Woke af.", "followers_count": 74, "friends_count": 173, "statues_count": 942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443804281835520, "text": "Future - Purple Reign ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2905071389, "name": "#YoungNLiving™", "screen_name": "ynl_tiba", "lang": "en", "location": "ducked off ", "create_at": date("2014-12-04"), "description": "Beating All Da Odds ❗️. . .", "followers_count": 211, "friends_count": 170, "statues_count": 5804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443804386570240, "text": "Wind 0 mph --. Barometer 29.69 in, Falling quickly. Temperature 55.6 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443804596420608, "text": "Mis break son de 30 minutos, y se van en un abril y cerrar de ojo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1618382659, "name": "W i l l b e r t h✡", "screen_name": "Torres_Willbert", "lang": "es", "location": "Kissimmee, FL", "create_at": date("2013-07-24"), "description": "•19° cancer ♋️ | ig: willberth_ | sc: wiillbeert | | Caguas, PR.", "followers_count": 1004, "friends_count": 1033, "statues_count": 53345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443804801900544, "text": "Booty pics would be lovely rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1054362990, "name": "Alex", "screen_name": "Birdy_Alex", "lang": "en", "location": "null", "create_at": date("2013-01-01"), "description": "Warm Thoughts- Ganja Enthusiast- League of Legends-, Hiking and Adventuring- Snapchat: Alex_Broskie", "followers_count": 568, "friends_count": 538, "statues_count": 16368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443805506543617, "text": "@BlakeDoyle41 @shxt_stain oh he must be telling the truth", "in_reply_to_status": 690443597586501632, "in_reply_to_user": 391543415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 391543415, 2339791820 }}, "user": { "id": 943911908, "name": "Hunter Babcock", "screen_name": "HunnaBabs", "lang": "en", "location": "Sioux Falls, SD ", "create_at": date("2012-11-12"), "description": "God-filled Hate-free | Wolf Skin | Booking inquiries: hunterbabcock428@gmail.com", "followers_count": 530, "friends_count": 96, "statues_count": 8150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brandon, SD", "id": "bc52231597151504", "name": "Brandon", "place_type": "city", "bounding_box": rectangle("-96.625284,43.572561 -96.534523,43.616456") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4606840, "cityName": "Brandon" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443805603020800, "text": "@NBCBlacklist is my favorite show ever. Reddington is a genius", "in_reply_to_status": -1, "in_reply_to_user": 1344919338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1344919338 }}, "user": { "id": 2287599342, "name": "Glam & The City", "screen_name": "glamandthecity_", "lang": "en", "location": "Hollywood", "create_at": date("2014-01-11"), "description": "Sharing my thoughts about fashion, beauty and my favorite spots in my city!", "followers_count": 417, "friends_count": 1888, "statues_count": 605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443805829390336, "text": "@YarinCohen12 nigga pass dat shit", "in_reply_to_status": 690443402928885760, "in_reply_to_user": 485371246, "favorite_count": 0, "coordinate": point("-118.29297736,34.06140229"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 485371246 }}, "user": { "id": 475213854, "name": "ryan", "screen_name": "RyanBuckner23", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "snapchat:ryaannnnb", "followers_count": 3315, "friends_count": 491, "statues_count": 26535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:05.000Z"), "id": 690443805829566464, "text": "Temp: 21.2°F - Dew Point: 14.6° - Wind: 3.6 mph - Gust: 6.3 - Rain Today: 0.00in. - Pressure: 30.30in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443806869700609, "text": "this hurt my feelings please take it down https://t.co/VZG9HhWlrU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2494014047, "name": "motis from baltimore", "screen_name": "motisdotking", "lang": "en", "location": "Baltimore, USA", "create_at": date("2014-04-18"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀sc: motisupnext ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀instagram: motis.king⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀ your mama follows me", "followers_count": 435, "friends_count": 401, "statues_count": 6708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowleys Quarters, MD", "id": "4af243ee8843d163", "name": "Bowleys Quarters", "place_type": "city", "bounding_box": rectangle("-76.408371,39.298198 -76.361767,39.34002") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2408800, "cityName": "Bowleys Quarters" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443806978736129, "text": "https://t.co/WC4lRXhZ8f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 18971, "friends_count": 11312, "statues_count": 4848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443807041699840, "text": "@marqopasa hocam yapma hocam başına iş alıcaksın", "in_reply_to_status": 690442534615580673, "in_reply_to_user": 3407591001, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 3407591001 }}, "user": { "id": 297366671, "name": "Son Dük", "screen_name": "yiitkarakaya", "lang": "tr", "location": "Maryland, USA", "create_at": date("2011-05-12"), "description": "nerden evcil değil olm evine alırsan evcil olur.", "followers_count": 1000, "friends_count": 192, "statues_count": 1662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean City, MD", "id": "759c5982d28a7b07", "name": "Ocean City", "place_type": "city", "bounding_box": rectangle("-75.091333,38.32431 -75.04914,38.451299") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24047, "countyName": "Worcester", "cityID": 2458225, "cityName": "Ocean City" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443807469506560, "text": "�� https://t.co/yIE3mwe6l7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 326397919, "name": "281-330-8004", "screen_name": "DamnAarielle", "lang": "en", "location": "Boring Ass Lubbock, TX", "create_at": date("2011-06-29"), "description": "I'm beautiful, gorgeous and funny as shit. Infamous Titty Twerker and Aspiring step mom IG:damnaarielle Snap Chat:imgoodasfuck", "followers_count": 26320, "friends_count": 7022, "statues_count": 135958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443807586979840, "text": "See our latest #ElPaso, TX #job and click to apply: Evening Call Center Agent - https://t.co/ln9ICtwEsV #milspouse https://t.co/4X304sysbo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.287521,31.7599791"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ElPaso", "job", "milspouse" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 295, "friends_count": 217, "statues_count": 3597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443807901487104, "text": "Shit pussies do ������ https://t.co/TvsFVsNBXr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 192608561, "name": "Queen Bitch", "screen_name": "Gueraa_901", "lang": "en", "location": "South Side", "create_at": date("2010-09-19"), "description": "Mirrors are just glass, & you are much more than that. ❤ Never lose faith in your fight. ⚓️", "followers_count": 1005, "friends_count": 2031, "statues_count": 19196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443808366993408, "text": "Was watching this anime couple weeks ago �� https://t.co/g0Ut6RWRhO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 705581257, "name": "Sup", "screen_name": "jaynay_spy", "lang": "en", "location": "null", "create_at": date("2012-07-19"), "description": "✨♡6/9♡✨ yea yea lol", "followers_count": 372, "friends_count": 684, "statues_count": 18915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443808513900544, "text": "Finally got my phone back. Yuh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842818295, "name": "Roger that.", "screen_name": "highrawger", "lang": "en", "location": "Carson, CA", "create_at": date("2014-10-06"), "description": "PSN=HighRank_Rawger", "followers_count": 139, "friends_count": 186, "statues_count": 633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443808660688896, "text": "@_Fetchprincess_ why would I have a cup tf I look like", "in_reply_to_status": 690443072237375488, "in_reply_to_user": 2254309370, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2254309370 }}, "user": { "id": 1967864520, "name": "lauren", "screen_name": "ramoslauren_", "lang": "en", "location": "null", "create_at": date("2013-10-17"), "description": "cvhs '17 // love everybody", "followers_count": 105, "friends_count": 116, "statues_count": 3074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443808757137408, "text": "HAPPY BIRTHDAY KEVIN!����hope your 18th year is filled with blessings!�� I love you!�� https://t.co/Ggx6tO3OXV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4674851112, "name": "Jaayy", "screen_name": "jayylinnee", "lang": "en", "location": "R.I.P Mon", "create_at": date("2015-12-29"), "description": "ehs '16", "followers_count": 163, "friends_count": 215, "statues_count": 433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443809172373504, "text": "I just want to see everyone happy and genuinely in love and succeeding ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 280837365, "name": "Lex", "screen_name": "_bigp0ppaa", "lang": "en", "location": "null", "create_at": date("2011-04-11"), "description": "If you want it, Go get it", "followers_count": 417, "friends_count": 388, "statues_count": 24419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443809495388160, "text": "que salvaje jajaja ¿qué estás haciendo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2411386848, "name": "Miguel", "screen_name": "Miguel818k", "lang": "en", "location": "Arleta ,CA", "create_at": date("2014-03-25"), "description": "ArletaSwimTeam | Emotional Boyz 2001 | ActiveSociety | WOWOUCH", "followers_count": 375, "friends_count": 334, "statues_count": 21577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443809612783616, "text": "Bruh Frankie I gotta stop with this Twitter stuff. Just getting myself into a bigger whole.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1133093275, "name": "Travis", "screen_name": "Yungtrav93", "lang": "en", "location": "null", "create_at": date("2013-01-29"), "description": "Don't waste my time. 22 years young. 580✈️918 Phil 4:13. On the road to be successful.", "followers_count": 1065, "friends_count": 990, "statues_count": 39617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443809906429952, "text": "@mashtonsalsa yes", "in_reply_to_status": 689680755862212608, "in_reply_to_user": 355136254, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 355136254 }}, "user": { "id": 129068633, "name": "♛", "screen_name": "sierraarashell", "lang": "en", "location": "Wonderland", "create_at": date("2010-04-02"), "description": "Ferris Bueller you're my hero", "followers_count": 1041, "friends_count": 1099, "statues_count": 11200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443809923162112, "text": "Wind 1.3 mph SSW. Barometer 29.821 in, Rising. Temperature 38.7 °F. Rain today 0.01 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443810023870464, "text": "good night moon https://t.co/tjKsxJc8mp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 833969718, "name": "mak", "screen_name": "asapmakky", "lang": "en", "location": "804", "create_at": date("2012-09-19"), "description": "my name is Makenzie & yes I say lol in real life | ig. asvpmakky | sc. maaakenzieee", "followers_count": 1387, "friends_count": 400, "statues_count": 44087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colonial Heights, VA", "id": "f8e149f66945a5a9", "name": "Colonial Heights", "place_type": "city", "bounding_box": rectangle("-77.431934,37.233115 -77.362385,37.29787") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51570, "countyName": "Colonial Heights", "cityID": 5118448, "cityName": "Colonial Heights" } }
+{ "create_at": datetime("2016-01-22T00:00:06.000Z"), "id": 690443810191597568, "text": "1/22/2016 - 02:00\nTemp: 33.8F \nHum: 98%\nWind: 1.0 mph\nBaro: 29.991in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 54, "statues_count": 49255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806169850351616, "text": "when I shoot a 3 @MrJPowers15 https://t.co/IzOugqL65H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1452877992 }}, "user": { "id": 1881248959, "name": "jer", "screen_name": "fleremiah", "lang": "en", "location": "null", "create_at": date("2013-09-18"), "description": "the lightskin Kyle Timmons", "followers_count": 397, "friends_count": 412, "statues_count": 6101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806170294968320, "text": "I just watched this poor soul get game ran on him so hard", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316278529, "name": "John Tello Wick", "screen_name": "Not_The_Turtle", "lang": "en", "location": "Omaha, Nebraska ", "create_at": date("2011-06-12"), "description": "Donatello Marcus Kennedy the one and only named after the artist not the turtle", "followers_count": 180, "friends_count": 287, "statues_count": 6625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ralston, NE", "id": "bb8dcc0ba167fd3f", "name": "Ralston", "place_type": "city", "bounding_box": rectangle("-96.052431,41.190706 -96.023666,41.212571") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3140605, "cityName": "Ralston" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806170588618752, "text": "This might be my fav header of all time ���� https://t.co/2ralHzhwp5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58979139, "name": "lanie", "screen_name": "ayelaniee", "lang": "en", "location": "null", "create_at": date("2009-07-21"), "description": "null", "followers_count": 445, "friends_count": 180, "statues_count": 39439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806171163209728, "text": "Im 19... ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 161848499, "name": "just jamie", "screen_name": "jamieSbautista", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-07-01"), "description": "lookin shiesty", "followers_count": 1035, "friends_count": 1019, "statues_count": 6012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806171607760897, "text": "@KinggTip___ yes cause the way you used it in that tweet I'm confused", "in_reply_to_status": 690805990275547136, "in_reply_to_user": 223233926, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223233926 }}, "user": { "id": 419197166, "name": "ᗪᗷ✨", "screen_name": "TruuDee__", "lang": "en", "location": "iᑎ ᒪᗩᒪᗩ ᒪᗩᑎᗪ ", "create_at": date("2011-11-22"), "description": "ᏔƎУ ᏰƎƧ✞ƒЯîƎИ∂ ⛽️❤️| ifiwereyouiwouldntfuckwithme| FREE WEY", "followers_count": 1660, "friends_count": 996, "statues_count": 35357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tupelo, MS", "id": "894f2ba528ba1c7d", "name": "Tupelo", "place_type": "city", "bounding_box": rectangle("-88.800213,34.203564 -88.656629,34.324583") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2874840, "cityName": "Tupelo" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806172043980801, "text": "omg, yes @warriors https://t.co/YN0C4jDtoX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26270913 }}, "user": { "id": 312398939, "name": "Andreea Bordianu", "screen_name": "abordianu22", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "californiaa✌️", "followers_count": 533, "friends_count": 404, "statues_count": 8320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806172190904321, "text": "@wobm https://t.co/aRkPC1jQ14\n#weather #jonas #live", "in_reply_to_status": -1, "in_reply_to_user": 29407355, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "weather", "jonas", "live" }}, "user_mentions": {{ 29407355 }}, "user": { "id": 1363303868, "name": "Intro to Multimedia", "screen_name": "sarlesmedia", "lang": "en", "location": "Brick, New Jersey 08723", "create_at": date("2013-04-18"), "description": "Multimedia class @OCVTS Brick Center", "followers_count": 47, "friends_count": 229, "statues_count": 337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toms River, NJ", "id": "259d4fab72f5d95c", "name": "Toms River", "place_type": "city", "bounding_box": rectangle("-74.269909,39.942803 -74.10616,40.059877") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3473110, "cityName": "Toms River" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806172773810176, "text": "bruh , why is sleep nowhere on my mind ? ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330137772, "name": "LanaBaby™", "screen_name": "its_lahlana", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-07-05"), "description": "if I were you , I would hate me too :)", "followers_count": 1199, "friends_count": 780, "statues_count": 62016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Roads, LA", "id": "77f8e44ca6e5af84", "name": "New Roads", "place_type": "city", "bounding_box": rectangle("-91.47985,30.676827 -91.41601,30.727402") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22077, "countyName": "Pointe Coupee", "cityID": 2255105, "cityName": "New Roads" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806173054808064, "text": "Some changes being made right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319839856, "name": "Jim West", "screen_name": "ThisIsBigZach", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-06-18"), "description": "Just a fine ass fat nigga\n#KlanShitOnly \n#PV", "followers_count": 1189, "friends_count": 1045, "statues_count": 51003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806173079961600, "text": "https://t.co/XA9cgvJpBF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 15917482, "name": "Diana Leia", "screen_name": "ginagirlgenius", "lang": "en", "location": "New York Titty", "create_at": date("2008-08-20"), "description": "I hope you like your women like your sugar. Brown, raw & unrefined.", "followers_count": 257, "friends_count": 254, "statues_count": 7333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-23T00:00:00.000Z"), "id": 690806173222629376, "text": "Finding new thing out every day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 862010942, "name": "Shawn Lloyd", "screen_name": "Shawn_Lloyd14", "lang": "en", "location": "Grain Valley, MO", "create_at": date("2012-10-04"), "description": "GVHS '16 -Veni Vidi Vici -Insta & Snap: shawn_lloyd14 -Make yourself know, be one of a kind", "followers_count": 238, "friends_count": 436, "statues_count": 1156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grain Valley, MO", "id": "f21dbf9d9946d5c7", "name": "Grain Valley", "place_type": "city", "bounding_box": rectangle("-94.235338,38.980981 -94.182192,39.044458") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2928090, "cityName": "Grain Valley" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806173864464384, "text": "That hot shower was ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328297625, "name": "Mia ❣", "screen_name": "jahmangi", "lang": "en", "location": "c h i c a g o ", "create_at": date("2011-07-02"), "description": "|nasty9teen.| rest in peace daddy! 5.23.14 #balloutforTae5⃣", "followers_count": 1467, "friends_count": 1047, "statues_count": 23505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174019559424, "text": "great grandma is currently in the ER /:", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3399506838, "name": "denisse", "screen_name": "dxnisseee", "lang": "en", "location": "null", "create_at": date("2015-08-30"), "description": "(;", "followers_count": 23, "friends_count": 23, "statues_count": 58 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174455853056, "text": "You do what you want when you poppin����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2482953810, "name": "naddy♔", "screen_name": "Naddyruiz_", "lang": "en", "location": "#wearekobesangels", "create_at": date("2014-05-07"), "description": "♡Flor de Caña♡", "followers_count": 817, "friends_count": 464, "statues_count": 27961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prospect Park, NJ", "id": "edef0d36b87755e9", "name": "Prospect Park", "place_type": "city", "bounding_box": rectangle("-74.181033,40.929923 -74.166786,40.946686") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3461170, "cityName": "Prospect Park" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174678056960, "text": "@ebbtideapp Tide in Port Bolivar, Texas 01/23/2016\n Low 9:23am -0.9\nHigh 8:04pm 0.7\n Low 10:34pm 0.7\nHigh 2:35am 0.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-94.78,29.365"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 7, "friends_count": 1, "statues_count": 1471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4809250, "cityName": "Bolivar Peninsula" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174690611201, "text": "@42Contreras @SANAMAC1 u tell me u said it not me", "in_reply_to_status": 690805989457727488, "in_reply_to_user": 469916253, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 469916253, 316848890 }}, "user": { "id": 345788665, "name": "kicked back", "screen_name": "__ObeyMeBitch", "lang": "en", "location": "D(Ⓜ️aryland)V", "create_at": date("2011-07-30"), "description": "You wanna be a man? Do more and say less.", "followers_count": 1249, "friends_count": 769, "statues_count": 56220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Annapolis, MD", "id": "aa509e938179ea44", "name": "Annapolis", "place_type": "city", "bounding_box": rectangle("-76.553321,38.941032 -76.468263,38.9989") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2401600, "cityName": "Annapolis" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174787043328, "text": "@ImGrrrr8 same����", "in_reply_to_status": 690788111844933632, "in_reply_to_user": 732144420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 732144420 }}, "user": { "id": 2393131550, "name": "j a y d a", "screen_name": "jaydadomenique", "lang": "en", "location": "southern california☀️", "create_at": date("2014-03-16"), "description": "I live for the nights that I can't remember, with the people that I won't forget", "followers_count": 370, "friends_count": 224, "statues_count": 24205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cerrito, CA", "id": "3a4df2e210d69e68", "name": "El Cerrito", "place_type": "city", "bounding_box": rectangle("-117.536857,33.820953 -117.506615,33.862398") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621810, "cityName": "El Cerrito" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806174816563200, "text": "Ripley SW Limestone Co. Temp: 27.9°F Wind:13.6mph Pressure: 994.9mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 47885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175072387073, "text": "I had to hit her in the dm ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378787824, "name": "A Regular Guy", "screen_name": "MikeLee______", "lang": "en", "location": "Richmond, VA", "create_at": date("2014-03-08"), "description": "God First Family Second 2⃣4⃣♈️ Canes| Fins | Pistons| Tarheel Basketball| Cubs", "followers_count": 962, "friends_count": 1322, "statues_count": 83414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175126827013, "text": "That too!��❤️ https://t.co/zOj8ELA4ua", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2678793973, "name": "Sophia Bollman", "screen_name": "TiaBollman", "lang": "en", "location": "On A Stage Some Where", "create_at": date("2014-07-24"), "description": "I'm In The Band | Christian Lloyd❤️ | Stay Out The DMs!", "followers_count": 832, "friends_count": 656, "statues_count": 10122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175177175040, "text": "Congrats to this hoe who's gonna represent all shs ladies tomorrow night at Sadies! https://t.co/ektyzQslRO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3485624472, "name": "j", "screen_name": "jonathan_noelc", "lang": "en", "location": "null", "create_at": date("2015-09-07"), "description": "idk how but Imma find a way", "followers_count": 146, "friends_count": 173, "statues_count": 691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, CA", "id": "e6aaf7cb58e7857c", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-115.594963,32.817908 -115.544378,32.87646") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 636280, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175323951105, "text": "@AutumnHendrie I love you beautiful! \nEw picture...", "in_reply_to_status": 690803144238862336, "in_reply_to_user": 3191147370, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3191147370 }}, "user": { "id": 1895891125, "name": "deirdre_osborn", "screen_name": "limedeirdre", "lang": "en", "location": "Fairbanks", "create_at": date("2013-09-22"), "description": "sc: deirdre.osborn", "followers_count": 125, "friends_count": 122, "statues_count": 233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasilla, AK", "id": "00549eae15fe11a3", "name": "Wasilla", "place_type": "city", "bounding_box": rectangle("-149.555619,61.552885 -149.358889,61.599832") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 283080, "cityName": "Wasilla" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175479304192, "text": "Wind 1.0 mph NW. Barometer 30.300 in, Steady. Temperature 26.2 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175542079489, "text": "@Andyson__ HAPPY BIRTHDAY!!���� you're legal and you could buy me backwoods now yay! ����", "in_reply_to_status": -1, "in_reply_to_user": 2456109930, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2456109930 }}, "user": { "id": 340434476, "name": "watergirlz", "screen_name": "dessihurt", "lang": "en", "location": "California, USA", "create_at": date("2011-07-22"), "description": "inhale love - exhale hatred", "followers_count": 1043, "friends_count": 721, "statues_count": 25361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175676239872, "text": "MAKE THE THUNDER STOP IM SCARED", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2277131352, "name": "Delaney Romo", "screen_name": "Lanes517", "lang": "en", "location": "null", "create_at": date("2014-01-04"), "description": "I'm a handful, but that's why you have two hands", "followers_count": 238, "friends_count": 311, "statues_count": 793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175768510466, "text": "It's a shame when your own Niggas from your own hat hometown want to hate on yo ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368772740, "name": "BiG ChinGS", "screen_name": "BiGChinGSdotCOM", "lang": "en", "location": "Las Cruces NM", "create_at": date("2011-09-05"), "description": "LISTEN TO MY MUZIK!!!!!!!", "followers_count": 1034, "friends_count": 0, "statues_count": 1061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806175936438272, "text": "01/23@03:00 - Temp 24.5F, WC 15.0F. Wind 9.0mph NE, Gust 20.0mph. Bar 29.785in, Falling quickly. Rain 0.00in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806176343142400, "text": "Interested in a #Finance #job near #JerseyCity, New Jersey? This could be a great fit: https://t.co/cvLqSqohEQ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0776417,40.7281575"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "job", "JerseyCity", "Hiring" }}, "user": { "id": 3836985865, "name": "BOC Staffing", "screen_name": "BOCstaffing", "lang": "en", "location": "NYC & Jersey City area", "create_at": date("2015-10-09"), "description": "BOC is a niche specialty provider, serving the Tri-State area's Buy-side/Sell-side Financial Services community’s #staffing requirements.#finance #job #NYC #NJ", "followers_count": 28, "friends_count": 50, "statues_count": 29 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806176414572544, "text": "same https://t.co/rHWZ2OEcyv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1721090947, "name": "sarai", "screen_name": "unicornsandmeth", "lang": "en", "location": "212", "create_at": date("2013-09-01"), "description": "get off the internet", "followers_count": 1017, "friends_count": 249, "statues_count": 15920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806177572126720, "text": "@haley_hunterr FALSE, BLACK BEARS", "in_reply_to_status": -1, "in_reply_to_user": 710902218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 710902218 }}, "user": { "id": 2768559400, "name": "Sydney Bostrom", "screen_name": "sydney_bostrom", "lang": "en", "location": "SDSU ", "create_at": date("2014-09-12"), "description": "All you have is your fire", "followers_count": 167, "friends_count": 155, "statues_count": 317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-23T00:00:01.000Z"), "id": 690806177605754880, "text": "Wind 5.0 mph NNE. Barometer 30.019 in, Falling slowly. Temperature 24.2 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806177823752192, "text": "Happy birthday babe ����@kellakeeks ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3325320462 }}, "user": { "id": 31080502, "name": "Aiden", "screen_name": "aiden600", "lang": "en", "location": "null", "create_at": date("2009-04-14"), "description": "Working on an EP... Model/Songwriter Contact info: cvtless@gmail.com", "followers_count": 15536, "friends_count": 380, "statues_count": 35544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Beach, CA", "id": "2f6cb9a739991200", "name": "Laguna Beach", "place_type": "city", "bounding_box": rectangle("-117.820234,33.48444 -117.730133,33.610906") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639178, "cityName": "Laguna Beach" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806179350507520, "text": "@bowuigi_ https://t.co/BItJHBvgsA", "in_reply_to_status": 690805334756093952, "in_reply_to_user": 2647007906, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2647007906 }}, "user": { "id": 2327656434, "name": "ariel", "screen_name": "agentpleakley", "lang": "en", "location": "under the sea", "create_at": date("2014-02-04"), "description": "@_joyness_ 's side/more personal and fandomy account", "followers_count": 235, "friends_count": 175, "statues_count": 18384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806179413405696, "text": "Ryan is eating chips really loud in his Xbox mic. Someone please stop this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2532465208, "name": "Lord of the Pit", "screen_name": "bloodXswamp", "lang": "en", "location": "IN", "create_at": date("2014-05-06"), "description": "God had sent my calamity into a deep space from which not even in dreams, could I ever imagine my escape.", "followers_count": 147, "friends_count": 166, "statues_count": 8078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806179610537984, "text": "Lawd ���� https://t.co/0iChJ2I8t8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 2819196876, "name": "Marshal Wilkes", "screen_name": "htttpflawless", "lang": "en", "location": "Port Orchard, WA", "create_at": date("2014-09-18"), "description": "G.A.Y , Fashion, SC:fashionkilla16 , PNW, I'm just doing me but these bitches can't breathe✨", "followers_count": 357, "friends_count": 720, "statues_count": 7812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Orchard, WA", "id": "006f1fd939044d62", "name": "Port Orchard", "place_type": "city", "bounding_box": rectangle("-122.708264,47.43909 -122.603325,47.548799") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5355785, "cityName": "Port Orchard" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180034301953, "text": "Kaasar paputol putol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2556342039, "name": "Tiney", "screen_name": "bish_rad", "lang": "en", "location": "Hempstead, NY", "create_at": date("2014-05-20"), "description": "6teen | New York | EMHS | 10th Grader | IG: Queen_Shoorty |", "followers_count": 474, "friends_count": 443, "statues_count": 6002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Meadow, NY", "id": "18b390731066adcb", "name": "East Meadow", "place_type": "city", "bounding_box": rectangle("-73.589148,40.694463 -73.526275,40.749526") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3622502, "cityName": "East Meadow" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180134842368, "text": "At 2:45 AM, 2 NW Hurt [Pittsylvania Co, VA] LAW ENFORCEMENT reports SNOW of 7.00 INCH #RNK https://t.co/S5rXOSV1de", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.3,37.1"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RNK" }}, "user": { "id": 34924191, "name": "IEMBot RNK", "screen_name": "iembot_rnk", "lang": "en", "location": "null", "create_at": date("2009-04-24"), "description": "null", "followers_count": 151, "friends_count": 0, "statues_count": 79238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hurt, VA", "id": "d2456f450760cfdb", "name": "Hurt", "place_type": "city", "bounding_box": rectangle("-79.313683,37.079105 -79.280614,37.107564") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51143, "countyName": "Pittsylvania", "cityID": 5139224, "cityName": "Hurt" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180223016960, "text": "@loudobbsnews @FoxNews #Trump2016", "in_reply_to_status": 690745708434030592, "in_reply_to_user": 26487169, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Trump2016" }}, "user_mentions": {{ 26487169, 1367531 }}, "user": { "id": 450319311, "name": "Rob Danger كافر", "screen_name": "Robdang3r", "lang": "en", "location": "USA", "create_at": date("2011-12-29"), "description": "Patriot and Infidel", "followers_count": 1320, "friends_count": 1700, "statues_count": 22620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180227100672, "text": "HAPPY BIRTHDAY TO MY HEART AND SOUL ���� @meghanharris23 pics coming later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453204558 }}, "user": { "id": 705504991, "name": "Kennedy Pate", "screen_name": "kenkenpate", "lang": "en", "location": "null", "create_at": date("2012-07-19"), "description": "null", "followers_count": 976, "friends_count": 765, "statues_count": 9571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arden-Arcade, CA", "id": "b90f2a335f8565c0", "name": "Arden-Arcade", "place_type": "city", "bounding_box": rectangle("-121.423941,38.562585 -121.327437,38.645482") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602553, "cityName": "Arden-Arcade" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180248227840, "text": "Wind 6.9 mph NW. Baro 29.924 in Rising.\nTemp 26.9°F Hum 85% #mrxwx\nRain Today 0.00 in. #chattanooga\nSite: https://t.co/ZGx7474cQ4 #CHAwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.12583333,35.00138889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mrxwx", "chattanooga", "CHAwx" }}, "user": { "id": 3044775743, "name": "ScenicCityWeather", "screen_name": "ScenicCityWx", "lang": "en", "location": "Chattanooga (E.Brainerd), TN", "create_at": date("2015-02-18"), "description": "Weather Tweet Every 60 Minutes", "followers_count": 6, "friends_count": 7, "statues_count": 333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Brainerd, TN", "id": "aa56488fe3272c5c", "name": "East Brainerd", "place_type": "city", "bounding_box": rectangle("-85.139808,34.986443 -85.079567,35.04565") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180369809409, "text": "Wind 3.8 mph NNE. Barometer 30.16 in, Falling slowly. Temperature 21.2 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 96, "statues_count": 156904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180457779200, "text": "I really don't care about anything but school, family & soccer.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1219898846, "name": "Blurryface™", "screen_name": "Sabrahh_21", "lang": "en", "location": "Cudi Zone", "create_at": date("2013-02-25"), "description": "Blurryface / Beauty is in the eye of the beholder / Snapchat: layla_711", "followers_count": 425, "friends_count": 985, "statues_count": 7500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534889,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180470362113, "text": "Wind 3.0 mph N. Barometer 1031.97 mb, Steady. Temperature 26.3 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806180529082368, "text": "Temp: 34.3°F Wind:3.3mph Pressure: 30.102hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806181011456000, "text": "\"Is that throw up or alcohol?\"\n\"No\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1520756407, "name": "Michael Costello", "screen_name": "mcostello306", "lang": "en", "location": "null", "create_at": date("2013-06-15"), "description": "null", "followers_count": 321, "friends_count": 289, "statues_count": 1076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-01-23T00:00:02.000Z"), "id": 690806181116260353, "text": "@DevinParker98 no", "in_reply_to_status": 690805987536568320, "in_reply_to_user": 497654135, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 497654135 }}, "user": { "id": 256841399, "name": "chel belle ♡", "screen_name": "Itzzmichelle", "lang": "en", "location": "null", "create_at": date("2011-02-23"), "description": "xoxo gossip girl", "followers_count": 1036, "friends_count": 756, "statues_count": 13915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806182009647104, "text": "Facts https://t.co/xdApacGb8D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39570285, "name": "CultRapDj", "screen_name": "DeeJayTrap", "lang": "en", "location": "null", "create_at": date("2009-05-12"), "description": "-Producer|Club DJ | Tour DJ | Mix-tape DJ -Deniro Farrar Official DJ (@DeniroFarrar) -EDM | Hiphop | Trap | -Bookings -Bookdjtm@gmail.com", "followers_count": 10886, "friends_count": 4809, "statues_count": 40255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806182357893120, "text": "- https://t.co/ByJ4foLAtJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01180332,37.69944345"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3829425134, "name": "Allen Collier", "screen_name": "colaroler", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "null", "followers_count": 5, "friends_count": 0, "statues_count": 12117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806182475382785, "text": "3 https://t.co/1yElquqaMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2743209125, "name": "Feb.14", "screen_name": "BabyDes_Des", "lang": "en", "location": "somewhere w//squad ", "create_at": date("2014-08-14"), "description": "OHHS18 w/bestfriend ❤ that header though. twin here Mally❤.", "followers_count": 580, "friends_count": 621, "statues_count": 18726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxon Hill, MD", "id": "003b0a6b6b3eca0e", "name": "Oxon Hill", "place_type": "city", "bounding_box": rectangle("-77.001308,38.760754 -76.937212,38.83211") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2459500, "cityName": "Oxon Hill" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806182848679937, "text": "#SupportOriginMelissa 21.7°F Wind:0.0mph Pressure: 29.85hpa Rising slowly Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806182873800705, "text": "Wind 0.0 mph ---. Barometer 30.231 in, Falling. Temperature -1.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183121281024, "text": "Goodnight see what tomorrow brings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750420312, "name": "TRĘŁŁ☄", "screen_name": "sub1boy1", "lang": "en", "location": "la jam here @_jam_nojelly ❄️", "create_at": date("2012-08-10"), "description": "Life || sc liltrell18 || FREE KEYSHUN", "followers_count": 641, "friends_count": 516, "statues_count": 15024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183377043456, "text": "I look for you in everyone I see", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594678760, "name": "♡", "screen_name": "yooniggaamy", "lang": "en", "location": "null", "create_at": date("2012-05-30"), "description": "raul garcia♡", "followers_count": 321, "friends_count": 203, "statues_count": 4704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, CA", "id": "b2fda7385aa963dd", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-118.317776,33.930952 -118.291549,33.959516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684592, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183465189376, "text": "Wind 0 mph --. Barometer 30.04 in, Rising slowly. Temperature 33.8 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183670652928, "text": "@vinceyvee oh okay that's better", "in_reply_to_status": 690806059431260160, "in_reply_to_user": 316230278, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316230278 }}, "user": { "id": 309776116, "name": "Sc: indeniyal ✨", "screen_name": "__indeNIYAl", "lang": "en", "location": "w/ August ❤️", "create_at": date("2011-06-02"), "description": "why be so nasty and so rude? when I can be so fierce and so successful ✨", "followers_count": 1701, "friends_count": 1169, "statues_count": 93977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711517,39.197211 -76.529388,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183800803328, "text": "@drumaq when you figure it out hit me up i'm coming with you", "in_reply_to_status": 690806076032180224, "in_reply_to_user": 60461356, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60461356 }}, "user": { "id": 63170607, "name": "esme ◡̈", "screen_name": "biebsftshawn", "lang": "en", "location": "nj || nyc", "create_at": date("2009-08-05"), "description": "sometimes it's hard to face reality", "followers_count": 8518, "friends_count": 300, "statues_count": 114431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Passaic, NJ", "id": "22bb8afe3a5531fd", "name": "Passaic", "place_type": "city", "bounding_box": rectangle("-74.146675,40.834409 -74.107319,40.876612") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3456550, "cityName": "Passaic" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183892877312, "text": "@arielreyes___ uh noooooooo", "in_reply_to_status": -1, "in_reply_to_user": 1720698690, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1720698690 }}, "user": { "id": 1210768350, "name": "Zeek☹", "screen_name": "acevedoezekiel", "lang": "en", "location": "null", "create_at": date("2013-02-22"), "description": "Ariel ♥️♥️♥️♥️", "followers_count": 491, "friends_count": 358, "statues_count": 12004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806183922282496, "text": "36.4F (Feels: 36.4F) - Humidity: 80% - Wind: 6.0mph NW - Gust: 6.9mph - Pressure: 1037.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 223644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806184480083968, "text": "You a stone cold ���� Bone!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546496065, "name": "Justice Dixon", "screen_name": "SOfficial_Julio", "lang": "en", "location": "IN✈️KY✈️", "create_at": date("2013-06-25"), "description": "#HOOPDREAMS A Ball & A dream is all I got| #BloodBrother | NHS Class of 2016| - RIP Amber", "followers_count": 2073, "friends_count": 1562, "statues_count": 43422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806184702509056, "text": "�������� https://t.co/hvO4MaoALA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 242347046, "name": "Young Kevin Gates", "screen_name": "YaFav_Sailor", "lang": "en", "location": "null", "create_at": date("2011-01-24"), "description": "#LongLivePete Nashville, TN ✈️ Raleigh, NC ✈️ Instagram/Snapchat: yafav_sailor", "followers_count": 1044, "friends_count": 498, "statues_count": 139320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806185541275648, "text": "i cant explain how irritated i am right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2824338623, "name": "anria", "screen_name": "AnriaMargel", "lang": "en", "location": "with you", "create_at": date("2014-10-11"), "description": "vncvr raised || '01 || jasmine charlotte", "followers_count": 55, "friends_count": 143, "statues_count": 705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnwood, WA", "id": "3912e7cd4cf62c39", "name": "Lynnwood", "place_type": "city", "bounding_box": rectangle("-122.337717,47.802196 -122.262575,47.85379") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5340840, "cityName": "Lynnwood" } }
+{ "create_at": datetime("2016-01-23T00:00:03.000Z"), "id": 690806185780498432, "text": "Austin actually *is* weird!!!\n����������������\n#sharksinthedeepend @ Shake Shack https://t.co/302PnP4iBw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7616396,30.25590125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sharksinthedeepend" }}, "user": { "id": 29081315, "name": "R. Travis Pierce", "screen_name": "RTravisPierce", "lang": "en", "location": "NompNasty", "create_at": date("2009-04-05"), "description": "Hyperlooped and Graphenated", "followers_count": 388, "friends_count": 572, "statues_count": 3307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186262663168, "text": "Tbh I can see some ppl on Angels Twitter getting married to one another", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342434807, "name": "♡Stacy♡", "screen_name": "OhItsStacy", "lang": "en", "location": "Rancho Cucamonga, CA", "create_at": date("2011-07-25"), "description": "California Livin' ☀️ Starbucks Lover ☕️ Angels⚾️ Lakers ❤️ IG: StacyStott", "followers_count": 772, "friends_count": 515, "statues_count": 35241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186405396480, "text": "nah buggin mine was closed https://t.co/WOJka26LYE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 803603126, "name": "cinnamon", "screen_name": "xyieanax33", "lang": "en", "location": "ny. ", "create_at": date("2012-09-04"), "description": "sc: itsxyieanahoe", "followers_count": 1074, "friends_count": 487, "statues_count": 63650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186581499904, "text": "@BrianTooTurnt lier �� https://t.co/kEfDpPxvz5", "in_reply_to_status": 690800705116864512, "in_reply_to_user": 2696912455, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2696912455 }}, "user": { "id": 1332627138, "name": "JuniorB", "screen_name": "NastyBoyJuniorr", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-04-06"), "description": "null", "followers_count": 170, "friends_count": 239, "statues_count": 6093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186678034434, "text": "3 am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2556691576, "name": "d a n i e l l e", "screen_name": "daniellebull18", "lang": "en", "location": "null", "create_at": date("2014-05-20"), "description": "bye", "followers_count": 320, "friends_count": 194, "statues_count": 2302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186883416065, "text": "https://t.co/9WcTPzvb1R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 76, "friends_count": 222, "statues_count": 113184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806186980024320, "text": "Wind 6.0 mph NE. Barometer 29.841 in, Falling Rapidly. Temperature 27.7F. Rain today 0.00 in. Humidity 97% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 581, "friends_count": 788, "statues_count": 36089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806187009269760, "text": "Let me get some shut eye ������ head ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2166572891, "name": "Big baby", "screen_name": "gbaby_forlife", "lang": "en", "location": "null", "create_at": date("2013-11-03"), "description": "dre gang", "followers_count": 1005, "friends_count": 778, "statues_count": 22090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806187147788288, "text": "https://t.co/JMxbAe1eow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 19241, "friends_count": 12451, "statues_count": 5263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806187349028864, "text": "Don't ever feel like you have to apologize for being yourself or who you are/look like. EVER.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320053633, "name": "4hunnidddd™", "screen_name": "HannahWaller5", "lang": "en", "location": "IG-4hunnidddd", "create_at": date("2011-06-18"), "description": "baller waller. ♊️♓️. Scoducks", "followers_count": 726, "friends_count": 533, "statues_count": 40997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806188494090240, "text": "Tpi bazli ni menakutkan gak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 605612767, "name": "Aldini.", "screen_name": "alifdromzi", "lang": "en", "location": "Eugene, OR", "create_at": date("2012-06-11"), "description": "Sorry. Ada life.", "followers_count": 270, "friends_count": 246, "statues_count": 14116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806188703911936, "text": "Happy birthday to my sweet angel up in heaven, I love you and miss you always ���� https://t.co/4ZsXq6Rgfb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128672249, "name": "Nicole Timineri", "screen_name": "nicolerose17", "lang": "en", "location": "buffalove ", "create_at": date("2010-04-01"), "description": "BLC", "followers_count": 349, "friends_count": 358, "statues_count": 21329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tonawanda, NY", "id": "7eba31f60a59aa24", "name": "Tonawanda", "place_type": "city", "bounding_box": rectangle("-78.941453,42.955994 -78.822147,43.031335") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3674183, "cityName": "Tonawanda" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806188783591424, "text": "@rubinanaseerr hoes will be hoes", "in_reply_to_status": 690805684766646272, "in_reply_to_user": 1352910806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1352910806 }}, "user": { "id": 54468821, "name": "Trigga", "screen_name": "TravonRogers_", "lang": "en", "location": "Lansing, MI", "create_at": date("2009-07-06"), "description": "I want to motivate others. If there ain't a way then make a way!✌♉ #inkgang #Psalm23", "followers_count": 811, "friends_count": 270, "statues_count": 44025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806188867387392, "text": "likeeeee for one�� https://t.co/Q5lhEuB9De", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337452748, "name": "#liyah", "screen_name": "liyahisfab", "lang": "en", "location": "null", "create_at": date("2014-02-10"), "description": "18. fashion designer | IG: AALIYAHFISHER | snapchat: aaliyahfisher", "followers_count": 1294, "friends_count": 1353, "statues_count": 24897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806189207195648, "text": "Can you find Cheyenne on the map? Just try it at https://t.co/goxm7EE3Dw #Cheyenne", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.82,41.14"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cheyenne" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1019, "friends_count": 312, "statues_count": 2544663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheyenne, WY", "id": "75c9243440a46116", "name": "Cheyenne", "place_type": "city", "bounding_box": rectangle("-104.860909,41.078217 -104.713275,41.19235") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56021, "countyName": "Laramie", "cityID": 5613900, "cityName": "Cheyenne" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806189559382017, "text": "Sure is clever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1144173307, "name": "Roark", "screen_name": "RoarkSizemore17", "lang": "en", "location": "null", "create_at": date("2013-02-02"), "description": "Verbal committ to ITT Tech. Proud to learn the skillz I need to know for life.", "followers_count": 351, "friends_count": 286, "statues_count": 6028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806189681061889, "text": "Already planning a trip to Crestview.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233590317, "name": "Eryn Nicole Chaffin", "screen_name": "erynnicole15", "lang": "en", "location": "Marshfield, MO ", "create_at": date("2011-01-03"), "description": "null", "followers_count": 552, "friends_count": 483, "statues_count": 8640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshfield, MO", "id": "37e157a1817a8b15", "name": "Marshfield", "place_type": "city", "bounding_box": rectangle("-92.94313,37.32275 -92.872861,37.362167") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29225, "countyName": "Webster", "cityID": 2946388, "cityName": "Marshfield" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806190087864320, "text": "HAPPY BIRTHDAY TAYLOR�� I love you so much❤️ @Taylor_Ledesmaa https://t.co/VoZyKxUJm0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1029861338 }}, "user": { "id": 872718866, "name": "♡AMANDA♡", "screen_name": "_cripple_", "lang": "en", "location": "null", "create_at": date("2012-10-10"), "description": "MVHS | PNW | SC: uhmanduh2020", "followers_count": 294, "friends_count": 264, "statues_count": 2523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-01-23T00:00:04.000Z"), "id": 690806190352117761, "text": "https://t.co/XAdSv34vpW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3308411995, "name": "Plank!", "screen_name": "Trapmoneyplank", "lang": "en", "location": "null", "create_at": date("2015-08-06"), "description": "http://govaknee.tumblr.com", "followers_count": 419, "friends_count": 361, "statues_count": 4088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806190490718209, "text": "Followed https://t.co/zu0UC6By57", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2793376331, "name": "sweetheart.☺️", "screen_name": "____infiniti", "lang": "en", "location": "null", "create_at": date("2014-09-29"), "description": "standoffish.", "followers_count": 591, "friends_count": 470, "statues_count": 7903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sterling Heights, MI", "id": "23e0af972b9a2e8e", "name": "Sterling Heights", "place_type": "city", "bounding_box": rectangle("-83.091535,42.534811 -82.969611,42.627801") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2676460, "cityName": "Sterling Heights" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806190683500544, "text": "i literally hate maddison so much you cant take that bitch seriously ever ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1039294627, "name": "bæza", "screen_name": "BreannaBaze", "lang": "en", "location": "Lemoore, CA", "create_at": date("2012-12-27"), "description": "can we just go back to the way it was", "followers_count": 171, "friends_count": 243, "statues_count": 2379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806190851239937, "text": "@dagney_garcia we will still go;$", "in_reply_to_status": 690766644646252544, "in_reply_to_user": 339066130, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339066130 }}, "user": { "id": 452545245, "name": "Caryn Vieira", "screen_name": "cmvbby", "lang": "en", "location": "Dirty T", "create_at": date("2012-01-01"), "description": "University of Arizona • ΚΑΘ", "followers_count": 1603, "friends_count": 420, "statues_count": 12667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191170060288, "text": "When you free your mind, you allow the creative process to begin. #creativity", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "creativity" }}, "user": { "id": 119826872, "name": "Luis D. Duval", "screen_name": "ldduval11", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-03-04"), "description": "Technology+Strategy+Design+Innovation | Creative Technologist•Entrepreneur•Startup Advisor•Mentor•Futurist | Loves Code•Opinions are my own |", "followers_count": 1127, "friends_count": 1412, "statues_count": 23016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191249760256, "text": "why on gods earth did i ever find that stupid fucking frog funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 920198336, "name": "snake eyes", "screen_name": "501rene", "lang": "en", "location": "gtc", "create_at": date("2012-11-01"), "description": "be liberated", "followers_count": 546, "friends_count": 406, "statues_count": 43584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen, TX", "id": "22d928cbeab790ad", "name": "Allen", "place_type": "city", "bounding_box": rectangle("-96.736596,33.066464 -96.608938,33.158169") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4801924, "cityName": "Allen" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191312605184, "text": "Oh HAYYYYYY, I kinda like these big ol' glasses �� https://t.co/88NDcco08C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3678478753, "name": "Ashley Mendez", "screen_name": "ashleymendezz_", "lang": "en", "location": "null", "create_at": date("2015-09-24"), "description": "Living Passionately » Continuously Inspired Hebrew 12:1 » #ThereIsMore", "followers_count": 83, "friends_count": 112, "statues_count": 792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198394") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191543488512, "text": "Man ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1217047844, "name": "ℂⅅ......", "screen_name": "The_Sneaker_God", "lang": "en", "location": "Flight Club LA", "create_at": date("2013-02-24"), "description": "Peep the style & the way the cops sweat us. . . . ℂapital ⅁. .21. .ECU. .⛅️VII.IX.XII⛅️. .910⇔252. .", "followers_count": 2213, "friends_count": 1808, "statues_count": 72334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairmont, NC", "id": "01e570187eb5e192", "name": "Fairmont", "place_type": "city", "bounding_box": rectangle("-79.13209,34.474462 -79.10126,34.512901") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37155, "countyName": "Robeson", "cityID": 3722360, "cityName": "Fairmont" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191635566593, "text": "@JCarbajal32 @AIIGAF @DopeItsBen @matheus_ksny �� https://t.co/GPmmGDgmcR", "in_reply_to_status": -1, "in_reply_to_user": 612661372, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 612661372, 1097137651, 557560413, 557402491 }}, "user": { "id": 3268935794, "name": "Tank Dempsy", "screen_name": "_Jesitz", "lang": "en", "location": "Carson, CA", "create_at": date("2015-07-05"), "description": "Video Content Producer.", "followers_count": 434, "friends_count": 775, "statues_count": 22403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806191937556480, "text": "@stefsstuff @realDonaldTrump @Backstorymom1 @glennbeck THERE'S SOME TRUTH TO THAT! \"BECK\" HAS A LARGE OVERHEAD", "in_reply_to_status": 690728801962201088, "in_reply_to_user": 16209626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16209626, 25073877, 1422241538, 17454769 }}, "user": { "id": 2258525366, "name": "Michael Collier", "screen_name": "michael75205", "lang": "en", "location": "Dallas, Texas", "create_at": date("2013-12-22"), "description": "Sports Handicapper", "followers_count": 959, "friends_count": 1398, "statues_count": 2928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806192088612864, "text": "@papifiggs nope", "in_reply_to_status": 690775084970082304, "in_reply_to_user": 27807160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27807160 }}, "user": { "id": 1162675452, "name": "Kristian Napuanani", "screen_name": "x0misskris", "lang": "en", "location": "null", "create_at": date("2013-02-09"), "description": "♊️", "followers_count": 37, "friends_count": 76, "statues_count": 1476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kapolei, HI", "id": "01932ef001ef7c53", "name": "Kapolei", "place_type": "city", "bounding_box": rectangle("-158.090772,21.323228 -158.048087,21.365894") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1530300, "cityName": "Kapolei" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806192701083648, "text": "who's up????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 975235410, "name": "Anthony", "screen_name": "Anthonycolon50", "lang": "en", "location": "Kutztown university", "create_at": date("2012-11-27"), "description": "Wyomissing class of 2015. KU '19", "followers_count": 576, "friends_count": 1102, "statues_count": 972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kutztown, PA", "id": "76551cc477dcd48e", "name": "Kutztown", "place_type": "city", "bounding_box": rectangle("-75.798022,40.505567 -75.763846,40.534547") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4240656, "cityName": "Kutztown" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806193439113216, "text": "Ooookay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2721036961, "name": "Eric Wright Jr.", "screen_name": "Kentetheking", "lang": "en", "location": "Norf Rialto", "create_at": date("2014-08-09"), "description": "Rest Easy CJ", "followers_count": 1098, "friends_count": 804, "statues_count": 40709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806193661423616, "text": "I'm gonna lose everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1321589682, "name": "SkinnyWhitePimp™", "screen_name": "_vanillapapi", "lang": "en", "location": "Salinas, CA", "create_at": date("2013-04-01"), "description": "Blessings on Blessings", "followers_count": 505, "friends_count": 550, "statues_count": 9522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806193942466560, "text": "We're #hiring! Click to apply: Marketing - Full stack Developer - https://t.co/gd1DTqKB7M #IT #infosec #Chicago, IL #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "IT", "infosec", "Chicago", "Job", "Jobs" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 278, "friends_count": 160, "statues_count": 551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806193992892416, "text": "Thank you very much Denise #AskBJM https://t.co/DmbYkYBK6a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AskBJM" }}, "user": { "id": 788197512, "name": "Big John McCarthy", "screen_name": "JohnMcCarthyMMA", "lang": "en", "location": "null", "create_at": date("2012-08-28"), "description": "Husband, Father, Grandfather, Love MMA, Certified ABC Trainer who started the COMMAND program. life is about having a blast", "followers_count": 46037, "friends_count": 86, "statues_count": 7170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-23T00:00:05.000Z"), "id": 690806194353532929, "text": "DialAmerica #Sales : Customer Service Agent Healthcare (#MiddleburgHeights, OH) https://t.co/yDchCNS4Wc #milspouse https://t.co/HDJuUdLIq6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.812912,41.3614401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "MiddleburgHeights", "milspouse" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 296, "friends_count": 217, "statues_count": 3598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middleburg Heights, OH", "id": "6f9917397013c785", "name": "Middleburg Heights", "place_type": "city", "bounding_box": rectangle("-81.855797,41.350571 -81.784556,41.389426") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3949644, "cityName": "Middleburg Heights" } }
+{ "create_at": datetime("2016-01-23T00:00:06.000Z"), "id": 690806195070857216, "text": "Bruh To Ugg ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64891503, "name": "⚡️ ALEXIS ⚡️", "screen_name": "__MIGOOO", "lang": "en", "location": "♓️3:02♓️ ", "create_at": date("2009-08-11"), "description": "⚡️18⚡️$ingle ♻️", "followers_count": 1426, "friends_count": 1045, "statues_count": 48550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlow Heights, MD", "id": "b9605021d2448da6", "name": "Marlow Heights", "place_type": "city", "bounding_box": rectangle("-76.97051,38.808758 -76.927375,38.837602") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2450750, "cityName": "Marlow Heights" } }
+{ "create_at": datetime("2016-01-23T00:00:06.000Z"), "id": 690806195381215232, "text": "Wind 0.0 mph ---. Barometer 30.142 in, Steady. Temperature 30.8 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-23T00:00:06.000Z"), "id": 690806195746181124, "text": "Wow lol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.6092431,29.4788162"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 610474735, "name": "Antonio Banda", "screen_name": "Band_ola", "lang": "en", "location": "null", "create_at": date("2012-06-16"), "description": "I thank God for every chance he gives me..I leave my life in the hands of the Lord.", "followers_count": 329, "friends_count": 279, "statues_count": 3850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-23T00:00:06.000Z"), "id": 690806195808894977, "text": "@anwaarals عيل ليش تكتبين من الاول-.-", "in_reply_to_status": 690805546564235264, "in_reply_to_user": 1978182546, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 1978182546 }}, "user": { "id": 250321906, "name": "#عبدالعزيز_الحكيم", "screen_name": "3ziz_alkanderi", "lang": "en", "location": "وَطَــــنْ عٌـــمْــرِيْ", "create_at": date("2011-02-10"), "description": "﷽ •Painter  Aspire Universal Access And lover To The World Of (Media)♥P2BK My Life & My Story♥Writer ✒️", "followers_count": 4231, "friends_count": 2941, "statues_count": 80752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168557384798208, "text": "@JillDarrah @mopar_oz @Angelemichelle @PhxKen Ice holes and cork suckers!!", "in_reply_to_status": 691131667444928513, "in_reply_to_user": 1393625407, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1393625407, 3048022453, 49207725, 561654065 }}, "user": { "id": 208693779, "name": "Tory Garrison", "screen_name": "torycolumbusoh", "lang": "en", "location": "null", "create_at": date("2010-10-27"), "description": "Christ is coming!! Tweets: God & the Lord Jesus Christ, animal welfare (shelter dogs/cats), #WakeUpAmerica FalPal. Jimmy Fallon/FallonTonight follow.", "followers_count": 1773, "friends_count": 1944, "statues_count": 46195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168557539987456, "text": "Temp: 2.8F W C: 2.8F Wind:--- at 0.0kts Baro: 1026.9mb and Rising slowly Rain today: 0.00in R H: 95% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 185, "friends_count": 217, "statues_count": 95434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168557573361664, "text": "me https://t.co/2uiJp1TfYX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 864053239, "name": "andrea orozco", "screen_name": "dreeez_", "lang": "en", "location": "null", "create_at": date("2012-10-05"), "description": "gerald gillum is my man", "followers_count": 546, "friends_count": 143, "statues_count": 45301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168557745381376, "text": "captain morgan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 77, "friends_count": 222, "statues_count": 113365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168557950857216, "text": "I was sooo heated tonight But Im cool now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355945517, "name": "Josue Haros", "screen_name": "HarosJosue", "lang": "en", "location": "Rowlett, Tx", "create_at": date("2011-08-15"), "description": "Poleth Nicole Haros is my lover #CowboysNation and Dez caught it. snapchat: @Haros_Josue", "followers_count": 228, "friends_count": 126, "statues_count": 7503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168558139703296, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 328304431, "name": "DLN TRLL", "screen_name": "DelonTerelle", "lang": "en", "location": "Oxford, MS", "create_at": date("2011-07-02"), "description": "hotty toddy", "followers_count": 822, "friends_count": 733, "statues_count": 42341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MS", "id": "b86cd6ef7a54d640", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-89.571643,34.321375 -89.495897,34.387965") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28071, "countyName": "Lafayette", "cityID": 2854840, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168558567456768, "text": "Shout out to @lindseyjane012 for actually agreeing on road tripping it with me last minute to Santa Cruz on the throwback picture ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 364658431 }}, "user": { "id": 1242067022, "name": "Isela Vasquez", "screen_name": "Alesi0012", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "Live Life ✨", "followers_count": 359, "friends_count": 197, "statues_count": 7665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168558592630785, "text": "Mamma play both roles https://t.co/xcn4TyNmZ9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439110107, "name": "A-Dub", "screen_name": "Aaronwise5_", "lang": "en", "location": "Sacramento, CA", "create_at": date("2014-04-11"), "description": "#SCC | #Jucoproduct | #PTW I score Touchdowns", "followers_count": 411, "friends_count": 269, "statues_count": 3926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Sacramento, CA", "id": "009243ac50ad0f26", "name": "West Sacramento", "place_type": "city", "bounding_box": rectangle("-121.584282,38.516197 -121.506483,38.603777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 684816, "cityName": "West Sacramento" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168558701654016, "text": "Can you recommend anyone for this #job? Support Desk Specialist - https://t.co/NRW72GKTPe #Burlington, MA #IT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.195611,42.5048167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Burlington", "IT", "Hiring" }}, "user": { "id": 3011627064, "name": "Lahey Careers", "screen_name": "LaheyCareers", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "null", "followers_count": 43, "friends_count": 32, "statues_count": 454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168559255437312, "text": "@petitprincen TRU...................", "in_reply_to_status": 691168344586612736, "in_reply_to_user": 2761420476, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2761420476 }}, "user": { "id": 494458165, "name": "Darth Ka(Maul)", "screen_name": "karnasonofsun", "lang": "en", "location": "New York City", "create_at": date("2012-02-16"), "description": "Despite everything, it's still you. I Ria/M/21/NYC I QP- @CrookedTricking @smolhinata @keleticism I Ra's Maraq - Hyperion I Arjuna @ CA", "followers_count": 428, "friends_count": 551, "statues_count": 94017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168559615991809, "text": "Happy Birthday @robbierollinH ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2233820083 }}, "user": { "id": 704444095, "name": "C•C•", "screen_name": "Celeste_Rain97", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "•J E S U S• I S• M Y • S A V I O U R• Romans 5:8", "followers_count": 250, "friends_count": 367, "statues_count": 10755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168559842463744, "text": "When God sends you a Diamond, you kick all your rocks.. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61397221, "name": "Michelle.", "screen_name": "LookAt_JAZ", "lang": "en", "location": "Port Arthur, TX", "create_at": date("2009-07-29"), "description": "AKA | Loyal | DTM❤️", "followers_count": 551, "friends_count": 379, "statues_count": 53667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Arthur, TX", "id": "350ad20be66e9edc", "name": "Port Arthur", "place_type": "city", "bounding_box": rectangle("-94.053284,29.823923 -93.858553,29.979166") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858820, "cityName": "Port Arthur" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168560039628803, "text": "@chiefajxx ��✨", "in_reply_to_status": 691168377335783424, "in_reply_to_user": 1590174530, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1590174530 }}, "user": { "id": 4170980533, "name": "$$", "screen_name": "Mercedithhh", "lang": "en", "location": "Riverside, CA", "create_at": date("2015-11-08"), "description": "17. never settle for less than you are.", "followers_count": 1007, "friends_count": 625, "statues_count": 4439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168560136085504, "text": "one reason why i love my job is because i get the privilege to meet so many people out is state. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 969600006, "name": "Syd ❤️", "screen_name": "EspinosaSydney", "lang": "en", "location": "Fountain Valley, CA", "create_at": date("2012-11-24"), "description": "Never Stop Smiling! FVHS❤️", "followers_count": 408, "friends_count": 656, "statues_count": 9420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168560647794688, "text": "I sweater gawd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3435697815, "name": "I rather bang screw", "screen_name": "KingMayritta", "lang": "en", "location": "Fort Worth, TX ", "create_at": date("2015-08-22"), "description": "All I ever needed was the music and the mirror.", "followers_count": 170, "friends_count": 122, "statues_count": 2581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168560865894401, "text": "Our uber tight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 722373865, "name": "chola", "screen_name": "venusm00nz_", "lang": "en", "location": "venus/vegas", "create_at": date("2012-07-28"), "description": "sweet short girl // Dads ⚒ Die // WCF❤️", "followers_count": 839, "friends_count": 758, "statues_count": 13437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, NV", "id": "37d88f13e7a85f14", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-115.173994,36.128077 -115.083699,36.144748") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3284600, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168561310539778, "text": "I shouldn't be laughing bruh ������ https://t.co/op31Jv996B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228900535, "name": "7⃣Free Tootie7⃣", "screen_name": "PIRATES_SMOKE_", "lang": "en", "location": "New Orleans, LA ⚜7️⃣", "create_at": date("2010-12-20"), "description": "1825 Tulane (Only God Can Judge Me) ‼️ Snapchat Tev63", "followers_count": 831, "friends_count": 668, "statues_count": 128093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-24T00:00:00.000Z"), "id": 691168561406959617, "text": "Billy, Billy, Billy Just jumped over Jumpman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2934129213, "name": "Limbo", "screen_name": "ThirdIBilly", "lang": "en", "location": "In the studio", "create_at": date("2014-12-20"), "description": "null", "followers_count": 536, "friends_count": 498, "statues_count": 2335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Bend, TX", "id": "2a9e190efe38237e", "name": "Mission Bend", "place_type": "city", "bounding_box": rectangle("-95.681932,29.680892 -95.6342,29.719902") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848772, "cityName": "Mission Bend" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562350678016, "text": "@ebbtideapp Tide in Akutan, Alaska 01/24/2016\n Low 12:09am -0.7\nHigh 8:10am 3.6\n Low 12:48pm 3.0\nHigh 4:48pm 3.5\n Low 12:57am -0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-165.7767,54.1333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 12, "friends_count": 1, "statues_count": 1752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2013, "countyName": "Aleutians East", "cityID": 201090, "cityName": "Akutan" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562367569920, "text": "\"That bitch called me on the phone she ain't want shit\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146043791, "name": "S", "screen_name": "xxooSammieLeAnn", "lang": "en", "location": "Asheboro, NC", "create_at": date("2015-04-07"), "description": "just a bitch", "followers_count": 135, "friends_count": 255, "statues_count": 2419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asheboro, NC", "id": "1222e760f6d1a9ff", "name": "Asheboro", "place_type": "city", "bounding_box": rectangle("-79.871704,35.635344 -79.747338,35.790551") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37151, "countyName": "Randolph", "cityID": 3702080, "cityName": "Asheboro" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562413723648, "text": "Did that really just happen #Kygo #ImNotOkay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Kygo", "ImNotOkay" }}, "user": { "id": 338116904, "name": "Samantha", "screen_name": "SamShazzam", "lang": "en", "location": "#NMU", "create_at": date("2011-07-18"), "description": "Professional white girl", "followers_count": 392, "friends_count": 335, "statues_count": 17450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, IL", "id": "d50345354159b16c", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-87.942895,42.200288 -87.807195,42.280041") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1741105, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562426290177, "text": "03:00:01 |Temp: 27.4ºF | Wind Chill 27.4ºF |Dew Point 22.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WNW, Gusting to 1.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562526838784, "text": "Many thoughts, many memories, just for one person.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 797599754, "name": "val", "screen_name": "floresvalerie65", "lang": "en", "location": "Menifee, CA", "create_at": date("2012-09-01"), "description": "null", "followers_count": 370, "friends_count": 542, "statues_count": 657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168562862424064, "text": "I'm just a Michelle in a world of Beyonce's.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277693612, "name": "Lupe J", "screen_name": "Reptar956", "lang": "en", "location": "Brownsville, Tx", "create_at": date("2011-04-05"), "description": "I wanna see the world daddy and the world wants to see all of me!", "followers_count": 72, "friends_count": 127, "statues_count": 121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168563063885828, "text": "01/24@03:00 - Temp 26.8F, WC 19.9F. Wind 6.1mph NNW, Gust 14.0mph. Bar 29.861in, Rising. Rain 0.00in. Hum 77%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168563151925248, "text": "@TrellxBadasx i swear its poo outchea lol", "in_reply_to_status": 691165239065382912, "in_reply_to_user": 911749154, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 911749154 }}, "user": { "id": 266338946, "name": "Shannin | Shike.", "screen_name": "BathingShike", "lang": "en", "location": "70403.", "create_at": date("2011-03-14"), "description": "jiana.", "followers_count": 5026, "friends_count": 111, "statues_count": 102005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168563760005121, "text": "Shoutout to my bro @andrew_beni Happy birthday dude����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2505215868 }}, "user": { "id": 1260607584, "name": "Irby", "screen_name": "soccerzkid4", "lang": "en", "location": "Playing soccer somewhere", "create_at": date("2013-03-11"), "description": "FPSC Fury B00 Black // PHS/AJH Freshman", "followers_count": 156, "friends_count": 150, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summit, WA", "id": "71363b539f29a337", "name": "Summit", "place_type": "city", "bounding_box": rectangle("-122.39766,47.157769 -122.325735,47.180946") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5368365, "cityName": "Summit" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168564162760706, "text": "3:00 am we meet again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2349227086, "name": "Bae'Zuli", "screen_name": "_MegSpace", "lang": "en", "location": "Lost in staten island ", "create_at": date("2014-02-17"), "description": "still figuring it out | ΧΥΣ", "followers_count": 290, "friends_count": 299, "statues_count": 4466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168564720508929, "text": "����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 55670274, "name": "Scorpio Queen11/16♏️", "screen_name": "Temp_forever101", "lang": "en", "location": "Bothell, Washington", "create_at": date("2009-07-10"), "description": "With man this is impossible but not with God, all things are possible with God -Mark 10:27 God, My children & Me; in that order ✌️", "followers_count": 1355, "friends_count": 1882, "statues_count": 11368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mill Creek, WA", "id": "b6b3bc98f34c6e28", "name": "Mill Creek", "place_type": "city", "bounding_box": rectangle("-122.228318,47.838561 -122.185474,47.880326") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5345865, "cityName": "Mill Creek" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168565014220800, "text": "Same���� https://t.co/rDRNHzn4Cf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2941431599, "name": "jaylene rivera", "screen_name": "jaylenerivera02", "lang": "en", "location": "somewhere", "create_at": date("2014-12-25"), "description": "❤️live simply so others can simply live❤️", "followers_count": 63, "friends_count": 245, "statues_count": 541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168565316046848, "text": "Fuck what they talking about on yo timeline", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 858604064, "name": "drew", "screen_name": "salaokx", "lang": "en", "location": "null", "create_at": date("2012-10-02"), "description": "Francisco Javier ❤️ soon to be MUA.", "followers_count": 1066, "friends_count": 983, "statues_count": 76645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168565433470976, "text": "I didn't know you could still be \"poked\" on Facebook.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25820957, "name": "kate linlyn zee", "screen_name": "KatelynLindsey", "lang": "en", "location": "Valencia, CA", "create_at": date("2009-03-22"), "description": "null", "followers_count": 110, "friends_count": 73, "statues_count": 2250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-24T00:00:01.000Z"), "id": 691168565433470977, "text": "@kennyTGODlee @bruhbruh_2 @Jael_Vaughn12 @jaden_vaughn30 @DylanJensen5 haha okay Kenny I will", "in_reply_to_status": 691168300894547968, "in_reply_to_user": 462226072, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 462226072, 2918940912, 2741785692, 3013644672, 838628532 }}, "user": { "id": 2817244490, "name": "kory the side bro", "screen_name": "cummins389", "lang": "en", "location": "null", "create_at": date("2014-09-18"), "description": "Snapchat: koryflarity106 always rembered never forgotten 10-25-94/12-16-14 rest in paradise me and Wyatt come as a package", "followers_count": 339, "friends_count": 266, "statues_count": 4702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168565659983872, "text": "@EatMyJorts99 @DeMarcusRFNET @jd1655 @DailyNewsVinny wait, Oakland is a bigger market than San Diego?", "in_reply_to_status": 691165440983244800, "in_reply_to_user": 3161506284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3161506284, 364949053, 447769216, 151657603 }}, "user": { "id": 95513466, "name": "Trav", "screen_name": "TravisDevlin", "lang": "en", "location": "San Diego CA", "create_at": date("2009-12-08"), "description": "I like golf, beer, and the Chargers. Depending on the day, that order can change", "followers_count": 290, "friends_count": 1238, "statues_count": 15817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168565672701952, "text": "Wind 4.0 mph SE. Barometer 30.114 in, Falling. Temperature 28.0 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168565743882240, "text": "@c0ciana_ why", "in_reply_to_status": 691081843664224257, "in_reply_to_user": 3177766759, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3177766759 }}, "user": { "id": 3941740092, "name": "Spanky", "screen_name": "Spanky_ovo", "lang": "en", "location": "California, USA", "create_at": date("2015-10-18"), "description": "Views from the 8", "followers_count": 45, "friends_count": 46, "statues_count": 248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69197,36.643802 -121.590557,36.734485") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566066872321, "text": "Who ready for Orange Crush ???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4171975276, "name": "PRINCESS♋️", "screen_name": "Rated3X_", "lang": "en", "location": "Dublin ➡️ Fort Valley", "create_at": date("2015-11-12"), "description": "Untouched , Unbothered , & Educated ❤️. @_TeeMH & @PursuitOfLove__ Walking Diaries✨⚡️ #FVSU19 #FutureSportsAccountant", "followers_count": 191, "friends_count": 381, "statues_count": 363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Valley, GA", "id": "5fb7d0858055aa39", "name": "Fort Valley", "place_type": "city", "bounding_box": rectangle("-83.914883,32.521974 -83.865501,32.574552") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13225, "countyName": "Peach", "cityID": 1331096, "cityName": "Fort Valley" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566113075201, "text": "Wind 0.0 mph ---. Barometer 30.204 in, Falling slowly. Temperature 9.6 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566180126720, "text": "lowkey maybe highkey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3417056584, "name": "claire", "screen_name": "claire_ferrante", "lang": "en", "location": "Chico, CA", "create_at": date("2015-08-11"), "description": "honestly", "followers_count": 130, "friends_count": 114, "statues_count": 998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566603825152, "text": "but first? @ Hideout (at Dalva) https://t.co/tPnsVzXrZI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.42240346,37.7647438"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66245840, "name": "Jॐ", "screen_name": "Jayneohm", "lang": "en", "location": "null", "create_at": date("2009-08-16"), "description": "real down to mars girl", "followers_count": 234, "friends_count": 129, "statues_count": 5089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566796619778, "text": "@TomCruise I'm here but u know Scientology is a brain washing joke", "in_reply_to_status": -1, "in_reply_to_user": 48410093, "favorite_count": 0, "coordinate": point("-84.233406,40.143263"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48410093 }}, "user": { "id": 457296108, "name": "regina smith", "screen_name": "gibsonvilleus", "lang": "en", "location": "piqua ohio", "create_at": date("2012-01-06"), "description": "greatful for all i have a great man who i love completely", "followers_count": 17, "friends_count": 111, "statues_count": 59 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Piqua, OH", "id": "bbbffc3e2ec5a6fd", "name": "Piqua", "place_type": "city", "bounding_box": rectangle("-84.288697,40.115593 -84.19503,40.18012") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39109, "countyName": "Miami", "cityID": 3962848, "cityName": "Piqua" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566893105153, "text": "@NWorosz 40 min drive is worth it", "in_reply_to_status": 691165426076815360, "in_reply_to_user": 3151962019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3151962019 }}, "user": { "id": 612298684, "name": "sarah", "screen_name": "sarah_sirvio", "lang": "en", "location": "downriver michigan", "create_at": date("2012-06-18"), "description": "Carlson '16", "followers_count": 934, "friends_count": 629, "statues_count": 25350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gibraltar, MI", "id": "c6b5e05ca74856e5", "name": "Gibraltar", "place_type": "city", "bounding_box": rectangle("-83.229393,42.078689 -83.184014,42.112593") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2632020, "cityName": "Gibraltar" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168566997970944, "text": "@nateschef @gideonray26 @eric_sav23 lets go", "in_reply_to_status": 691168533519208449, "in_reply_to_user": 2157449562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2157449562, 3148438880, 3087971667 }}, "user": { "id": 3927527052, "name": "yazan assaf", "screen_name": "yazantron", "lang": "en", "location": "null", "create_at": date("2015-10-17"), "description": "null", "followers_count": 202, "friends_count": 199, "statues_count": 343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainfield, IL", "id": "4320ab56929ffcdb", "name": "Plainfield", "place_type": "city", "bounding_box": rectangle("-88.293691,41.566483 -88.152433,41.681434") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1760287, "cityName": "Plainfield" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168567182516224, "text": "33.1F (Feels: 33.1F) - Humidity: 87% - Wind: 3.1mph E - Gust: 3.8mph - Pressure: 1036.0mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 223788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168567652265984, "text": "Wind 7.0 mph SSW. Barometer 1023.17 mb, Falling. Temperature 30.6 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168567786471425, "text": "@_alyssamarieexo @gabbie_acosta https://t.co/dFAVjLa8j5", "in_reply_to_status": -1, "in_reply_to_user": 3112333618, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3112333618, 1695608365 }}, "user": { "id": 1223820404, "name": "Jez ❤", "screen_name": "Pickles_4", "lang": "en", "location": "Portland, TX", "create_at": date("2013-02-26"), "description": "@cucumbers_4 ♡ Country music = happiness :-) God will never leave your side - Joshua 1:9 †", "followers_count": 475, "friends_count": 281, "statues_count": 11110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taft, TX", "id": "de4157442392cc34", "name": "Taft", "place_type": "city", "bounding_box": rectangle("-97.408814,27.969896 -97.374738,27.989929") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4871684, "cityName": "Taft" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168568193339392, "text": "Happy birthday bf luv u �� https://t.co/rniY2OlD7n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 895712220, "name": "Murr", "screen_name": "murreyeah", "lang": "en", "location": "sleeping", "create_at": date("2012-10-21"), "description": "null", "followers_count": 475, "friends_count": 233, "statues_count": 28762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168568319176704, "text": "Just posted a photo @ Zbar and night club https://t.co/J8SA4Y9rG1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-103.8588409,44.4884415"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54800329, "name": "Young Man Jr.", "screen_name": "Atrain12321", "lang": "en", "location": "South Dakota", "create_at": date("2009-07-07"), "description": "Wanna have a good time? Buy a good watch.", "followers_count": 331, "friends_count": 341, "statues_count": 9030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spearfish, SD", "id": "8fe1b634982002af", "name": "Spearfish", "place_type": "city", "bounding_box": rectangle("-103.89022,44.452744 -103.785781,44.51233") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46081, "countyName": "Lawrence", "cityID": 4660020, "cityName": "Spearfish" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168568637943808, "text": "Pretty sure these pictures are just advertisements by Pink and Abercrombie! �� https://t.co/YNe4xu3kuL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465952115, "name": "JJ Olsen", "screen_name": "jjolsen27", "lang": "en", "location": "Southern California", "create_at": date("2012-01-16"), "description": "Babe is babe and thats final. Don't need any other babes. Don't want any other babes. No one can replace my babe. Btw did I mention babe? @ashley_ross74 (babe)", "followers_count": 160, "friends_count": 239, "statues_count": 2612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Alamitos, CA", "id": "0799ff0a3c1006e9", "name": "Los Alamitos", "place_type": "city", "bounding_box": rectangle("-118.092825,33.781057 -118.032515,33.819621") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 643224, "cityName": "Los Alamitos" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168568822530048, "text": "���� https://t.co/9aomQgRzBX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2574747009, "name": "g", "screen_name": "GaaabbyRod", "lang": "en", "location": "18 | AZ", "create_at": date("2014-05-31"), "description": "it be like that @_KennytheShark_ ♍️", "followers_count": 500, "friends_count": 367, "statues_count": 15189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maricopa, AZ", "id": "001b67fd5761210e", "name": "Maricopa", "place_type": "city", "bounding_box": rectangle("-112.079946,33.029009 -111.944584,33.087983") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 444410, "cityName": "Maricopa" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168569036541953, "text": "Temp: 39.6°F | Humidity: 79% | Wind: --- @ 0.0 mph | Barometer: 30.12 in | Dewpoint: 33.6°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 24, "friends_count": 1, "statues_count": 164536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168569258819584, "text": "@WorldStarFunny @ogJoell", "in_reply_to_status": 690330651472596992, "in_reply_to_user": 1107613844, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1107613844, 3041930599 }}, "user": { "id": 2346362513, "name": "A M A Y A", "screen_name": "Ahhhmaya", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "YURNT TOSTIDOS", "followers_count": 341, "friends_count": 270, "statues_count": 7794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel, VA", "id": "006654ca2b4e27c5", "name": "Laurel", "place_type": "city", "bounding_box": rectangle("-77.569305,37.606827 -77.475442,37.680986") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5144280, "cityName": "Laurel" } }
+{ "create_at": datetime("2016-01-24T00:00:02.000Z"), "id": 691168569602609152, "text": "Temp: 26.3°F Wind:0.0mph Pressure: 30.256hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168569921552384, "text": "Wind 0.0 mph N. Barometer 29.94 in, Steady. Temperature 10.9 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 26, "friends_count": 100, "statues_count": 156947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570189955072, "text": "@ one of your followers you hit and you were disappointed ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 234879125, "name": "la flamé", "screen_name": "___kaila", "lang": "en", "location": "DE/ATL", "create_at": date("2011-01-06"), "description": "I might take ya bitch", "followers_count": 1339, "friends_count": 580, "statues_count": 38567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookside, DE", "id": "fff1c66a88b63f36", "name": "Brookside", "place_type": "city", "bounding_box": rectangle("-75.737023,39.648694 -75.695801,39.685817") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1009850, "cityName": "Brookside" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570202402816, "text": "maybe I'd suck your dick if I wasn't on pause @KassieAgonoy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3168010123 }}, "user": { "id": 2952860126, "name": "riley", "screen_name": "NoahWingad", "lang": "en", "location": "Maui, Hawai'i", "create_at": date("2014-12-30"), "description": "loml @KassieAgonoy // psn-akaHapa", "followers_count": 260, "friends_count": 205, "statues_count": 3039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wailuku, HI", "id": "1ad33d5e2c3e5d0b", "name": "Wailuku", "place_type": "city", "bounding_box": rectangle("-156.5369,20.860213 -156.480503,20.916384") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1577450, "cityName": "Wailuku" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570215002112, "text": "@reeddavis @CWpiequeen https://t.co/93im5SyL0o", "in_reply_to_status": 691168201472905217, "in_reply_to_user": 23720298, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 23720298, 22578212 }}, "user": { "id": 23720298, "name": "Reed Davis", "screen_name": "reeddavis", "lang": "en", "location": "null", "create_at": date("2009-03-10"), "description": "photographer and person:-)", "followers_count": 264, "friends_count": 330, "statues_count": 3286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570307252225, "text": "What time is the Seahawks game tomorrow? #GoHawks #TooSoon?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoHawks", "TooSoon" }}, "user": { "id": 862517544, "name": "Casey Thorpe", "screen_name": "CaseyThorpe22", "lang": "en", "location": "null", "create_at": date("2012-10-04"), "description": "Christ follower • GCU alumni • Youth Pastor • Young Life Leader • OHS Baseball Coach • Sandberg Baseball Assistant Catching Coordinator", "followers_count": 494, "friends_count": 510, "statues_count": 8969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570315640833, "text": "@Skyler_goff13 ummmm why did you tag me", "in_reply_to_status": 691168028306747392, "in_reply_to_user": 4803744792, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4803744792 }}, "user": { "id": 4473246372, "name": "Bootylecia", "screen_name": "Bootylecia", "lang": "en", "location": "null", "create_at": date("2015-12-06"), "description": "I'm a sheltered city kid that wants to go on adventures but doesnt", "followers_count": 77, "friends_count": 125, "statues_count": 215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570558906368, "text": "Main phase 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189787423, "name": "Lee Yalung", "screen_name": "H1tokiriB4tosai", "lang": "en", "location": "null", "create_at": date("2010-09-11"), "description": "I've told you time and time again I'm not think as you drunk I am", "followers_count": 46, "friends_count": 107, "statues_count": 3928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168570961625088, "text": "Wind 0.0 mph NNE. Barometer 29.991 in, Steady. Temperature -6.9 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168571099983872, "text": "@erinalxandria 13 going on 30 is my shit", "in_reply_to_status": 691168335686299648, "in_reply_to_user": 3834635540, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3834635540 }}, "user": { "id": 1226740093, "name": "Phil™", "screen_name": "philthese", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-02-27"), "description": "Professional Nobody | Duo: @iLinxy", "followers_count": 1744, "friends_count": 685, "statues_count": 124852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168571280392194, "text": "@seita1106 いや俺もブラジルにいるけど", "in_reply_to_status": 691168198608052224, "in_reply_to_user": 1088064962, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 1088064962 }}, "user": { "id": 3150158510, "name": "ピース", "screen_name": "piece_0407", "lang": "ja", "location": "ラウンドワン潰れろ", "create_at": date("2015-04-11"), "description": "ボルテメインで他の機種はたまにやってます。最近クロビRev始めました。白猫/デレステ (音ゲー等のツイートが多いのでリア友のフォローはブロックさせて頂きます。)", "followers_count": 123, "friends_count": 109, "statues_count": 20305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brazil, IN", "id": "c04138d310229f7e", "name": "Brazil", "place_type": "city", "bounding_box": rectangle("-87.188047,39.503132 -87.098537,39.538108") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18021, "countyName": "Clay", "cityID": 1807174, "cityName": "Brazil" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168571477618688, "text": "So These Niggas Gave Me This Bottle Knowing Iont Drink ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52102607, "name": "That Talmbout", "screen_name": "TTUDadaNumba5", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-06-29"), "description": "October 20,2014 I Lost My Nigga He Was Apart Of My Empire Every Since That Day I Been Riding Late Like A Vampire", "followers_count": 2095, "friends_count": 748, "statues_count": 93176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168571632787456, "text": "Wow.. When did it become 3:00? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 192466979, "name": "Jacquaniese✨", "screen_name": "_missjaye", "lang": "en", "location": "Alabama ✈️ Delaware ", "create_at": date("2010-09-18"), "description": "God-fearing & Ambitious. #DSU17 #ILoveMyHBCU", "followers_count": 1348, "friends_count": 1114, "statues_count": 24308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168571741884416, "text": "I hurt someone yesterday.. I feel like shit.. I will never do it again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324596767, "name": "I hate petty", "screen_name": "StrikerT1014", "lang": "en", "location": "null", "create_at": date("2011-06-26"), "description": "Follow my IG: Torrez18 I flipped the script real quick on you niggas", "followers_count": 349, "friends_count": 303, "statues_count": 25577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572018589696, "text": "When you're no longer afraid, it's a freeing experience.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33448498, "name": "Señorita M.", "screen_name": "nyroyal19", "lang": "en", "location": "reporting live", "create_at": date("2009-04-19"), "description": "Senior Life Correspondent", "followers_count": 266, "friends_count": 223, "statues_count": 71766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572123406336, "text": "#SupportOriginMelissa 4.5°F Wind:1.6mph Pressure: 30.12hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 307913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572169719809, "text": "@2Girls1Richard Smh bruh chill we don't have waffle house up here", "in_reply_to_status": 691166768061140992, "in_reply_to_user": 195909357, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 195909357 }}, "user": { "id": 30668975, "name": "Yuuki Tetsuya", "screen_name": "PorscheofSports", "lang": "en", "location": "Za Warudo", "create_at": date("2009-04-12"), "description": "20. Giants | Nuggets | Barcelona | #YEEZYSEASONHASARRIVED | #BlackLivesMatter | If you want to see anime fights check my favorites.", "followers_count": 2773, "friends_count": 1988, "statues_count": 211649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanham, MD", "id": "019ae9b86255a185", "name": "Lanham", "place_type": "city", "bounding_box": rectangle("-76.885508,38.934439 -76.821939,38.988672") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445550, "cityName": "Lanham" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572429639680, "text": "I'm a little cold sometimes.... https://t.co/t4kRxOkvXp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 722945893, "name": "jessicaa ramirez", "screen_name": "justtblessed", "lang": "en", "location": "null", "create_at": date("2012-07-28"), "description": "Be as happy as you can be", "followers_count": 799, "friends_count": 728, "statues_count": 1304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572433928192, "text": "Nah I will hang up �� https://t.co/0oz6OfmvYx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479074388, "name": "tj felton", "screen_name": "tfelt3", "lang": "en", "location": "nccu17", "create_at": date("2012-01-30"), "description": "In life you will take two steps back but in the end you will win your Marathon", "followers_count": 430, "friends_count": 479, "statues_count": 7455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572496711681, "text": "@AVER_RAREVISION hahaha aw:)", "in_reply_to_status": 691168254824349696, "in_reply_to_user": 170154716, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 170154716 }}, "user": { "id": 1499998994, "name": "Jacque Ochoa", "screen_name": "jacque_ochoa", "lang": "en", "location": "Selah, WA", "create_at": date("2013-06-10"), "description": "god•viks", "followers_count": 1933, "friends_count": 860, "statues_count": 17918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selah, WA", "id": "01d7e4f642399b9f", "name": "Selah", "place_type": "city", "bounding_box": rectangle("-120.582182,46.640323 -120.49075,46.708434") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5363280, "cityName": "Selah" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168572681408512, "text": "@Kordelle_05 what's good brooo", "in_reply_to_status": 691161054555029505, "in_reply_to_user": 323005298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323005298 }}, "user": { "id": 2238553170, "name": "Uncle Willie's son", "screen_name": "Willz_L1Z", "lang": "en", "location": "Alliance, OH", "create_at": date("2013-12-09"), "description": "#330 #BlvdBoyz Every day above ground is a good day", "followers_count": 548, "friends_count": 714, "statues_count": 13322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alliance, OH", "id": "f3d62daa712bbb93", "name": "Alliance", "place_type": "city", "bounding_box": rectangle("-81.162586,40.872386 -81.062267,40.962933") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3901420, "cityName": "Alliance" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573004365824, "text": "Fuck plastic cups, mine be glass biddie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005756062, "name": "heyitsmayyy", "screen_name": "MayanneMadeira", "lang": "en", "location": "outside the box ", "create_at": date("2012-12-11"), "description": "Libra solar ☼ Sagittarius lunar☽ ~ FL ~ UCF instagram: heyitsmayyyyy. ~ le joie de vivre ~ wanderlust • art • cats • wine • ice queen", "followers_count": 543, "friends_count": 393, "statues_count": 9505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573180391424, "text": "\"Boy you look like the trix rabbit!\" \n-Random person on periscope ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279937713, "name": "Alex Mann", "screen_name": "itsalexney", "lang": "en", "location": "four.three.four", "create_at": date("2011-04-10"), "description": "There's a 77% chance I already don't like you.", "followers_count": 808, "friends_count": 750, "statues_count": 23402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynchburg, VA", "id": "2490949a68631669", "name": "Lynchburg", "place_type": "city", "bounding_box": rectangle("-79.257789,37.332496 -79.100237,37.469415") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51680, "countyName": "Lynchburg", "cityID": 5147672, "cityName": "Lynchburg" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573281030144, "text": "@racheIdenisse 500 of Neymar", "in_reply_to_status": 691168457136537600, "in_reply_to_user": 348559237, "favorite_count": 0, "retweet_count": 0, "lang": "is", "is_retweet": false, "user_mentions": {{ 348559237 }}, "user": { "id": 833362892, "name": "cristina", "screen_name": "CristinaGosen", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-09-19"), "description": "litmas", "followers_count": 316, "friends_count": 214, "statues_count": 958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573322973185, "text": "Fake Friends get treated like broke headphones ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3820159994, "name": "Suck My PineApple", "screen_name": "Dreeunlimated", "lang": "en", "location": "Lancaster, CA", "create_at": date("2015-10-07"), "description": "New Dick New Vocabulary", "followers_count": 115, "friends_count": 195, "statues_count": 547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573360840705, "text": "I never went through with it though. It was all just a fantasy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26296758, "name": "Darnell", "screen_name": "trini87", "lang": "en", "location": "NYC•Tampa•Denver•Chicago", "create_at": date("2009-03-24"), "description": "Brooklyn born/Trinidad & Tobago heritage/Jesus freak/swing dancer/foodie/philanthropist/beer enthusiast/avid traveler/Lyft driver/", "followers_count": 20314, "friends_count": 16716, "statues_count": 86994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schaumburg, IL", "id": "918c943e88cc3e5d", "name": "Schaumburg", "place_type": "city", "bounding_box": rectangle("-88.154356,41.986627 -88.028003,42.078062") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1768003, "cityName": "Schaumburg" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573377544193, "text": "AFTER HARRY https://t.co/5h1qUavLXz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 196379254, "name": "grinch", "screen_name": "Sssiarahh", "lang": "en", "location": "WWE", "create_at": date("2010-09-28"), "description": "comfortable in my chubby skin ❤", "followers_count": 849, "friends_count": 836, "statues_count": 28681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573402673152, "text": "Live from @Bonoboville:\n@DrSuzy gives a sneak-peek of her #EcosexualitySymposium #keynote address. \nWatch now: https://t.co/6toHPehkr3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EcosexualitySymposium", "keynote" }}, "user_mentions": {{ 377031061, 14355949 }}, "user": { "id": 64991381, "name": "DrSuzy-Tv & Radio", "screen_name": "radioSUZY1", "lang": "en", "location": "West Los Angeles, CA", "create_at": date("2009-08-12"), "description": "Producers of the World-Famous @DrSuzy Block Show * Join us LIVE @ https://bonoboville.com * Contact me for Booking/Memberships: 310-568-0066", "followers_count": 3104, "friends_count": 3338, "statues_count": 36629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573708832768, "text": "Baby it's whatevaaaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 118758681, "name": "Em", "screen_name": "Emonaayyy", "lang": "en", "location": "BOOLING BEING BRAZY, TX", "create_at": date("2010-03-01"), "description": "u do what u want when u poppin", "followers_count": 1273, "friends_count": 1044, "statues_count": 19627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victoria, TX", "id": "70f5587b3e27a105", "name": "Victoria", "place_type": "city", "bounding_box": rectangle("-97.045657,28.709293 -96.900168,28.895775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48469, "countyName": "Victoria", "cityID": 4875428, "cityName": "Victoria" } }
+{ "create_at": datetime("2016-01-24T00:00:03.000Z"), "id": 691168573838917632, "text": "I knew it was to good to be true", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325954888, "name": "bbytiff", "screen_name": "tiffanyspano", "lang": "en", "location": "california, san diego ", "create_at": date("2013-04-03"), "description": "God gives his hardest battles to his strongest soldiers.", "followers_count": 583, "friends_count": 346, "statues_count": 6649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Gardens, CA", "id": "0164a99dd4ce91a4", "name": "Winter Gardens", "place_type": "city", "bounding_box": rectangle("-116.946888,32.819949 -116.903401,32.857262") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 685992, "cityName": "Winter Gardens" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168574493298688, "text": "Wind 3.4 mph WNW. Barometer 30.016 in, Steady. Temperature 15.0 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 6013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168574736580608, "text": "I’ll say this, my love for football and my love for the NFL died with P Willis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272707611, "name": "Andrew", "screen_name": "Guut88", "lang": "en", "location": "Cincy via The Bay", "create_at": date("2011-03-26"), "description": "Graduate of Miami OH. #SFGiants #49ers #GSW #SJSharks Michigan/Miami OH fan, SMC BBall, & craft beer.\r\n#LOVEANDHONOR", "followers_count": 320, "friends_count": 665, "statues_count": 71458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mason, OH", "id": "c386e8441572506e", "name": "Mason", "place_type": "city", "bounding_box": rectangle("-84.353179,39.292287 -84.252354,39.399373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3948188, "cityName": "Mason" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168574740656129, "text": "@JacobPadilla_ yeah", "in_reply_to_status": 691167076426215424, "in_reply_to_user": 593208192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 593208192 }}, "user": { "id": 1297624974, "name": "Not Good Enough", "screen_name": "mynewhate", "lang": "en", "location": "San Diego ", "create_at": date("2013-03-24"), "description": "I don't know what to say anymore, when death is an everyday thought how do you react?", "followers_count": 1227, "friends_count": 1860, "statues_count": 191068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168575608995840, "text": "☁️☁️☁️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 545230001, "name": "Justin Wright", "screen_name": "J_wright_15", "lang": "en", "location": "Kanto Region", "create_at": date("2012-04-04"), "description": "http://www.hudl.com/athlete/3058729/highlights/306820542/v2", "followers_count": 395, "friends_count": 280, "statues_count": 2032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Creek, MI", "id": "658096f6d8391ba9", "name": "Battle Creek", "place_type": "city", "bounding_box": rectangle("-85.313052,42.232049 -85.140797,42.384545") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26025, "countyName": "Calhoun", "cityID": 2605920, "cityName": "Battle Creek" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168575797657600, "text": "Wind 0.0 mph ---. Barometer 30.093 in, Steady. Temperature 16.5 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 374, "friends_count": 288, "statues_count": 7722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168576523280384, "text": "That's how you know you gotta dope ass nigga when y'all two can just TU back to back, just w each other. Shmacked asf n shit lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3320247318, "name": "J.", "screen_name": "juceeell", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-08-19"), "description": "snap-cellebabesss", "followers_count": 520, "friends_count": 280, "statues_count": 3650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168577471139840, "text": "DialAmerica: Call Center Supervisor (#Orlando, FL) https://t.co/NYalUrJTWb #Sales #milspouse #parttime #jobs #Hiring https://t.co/wcGMVS6Eji", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2849169,28.4750405"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Orlando", "Sales", "milspouse", "parttime", "jobs", "Hiring" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 296, "friends_count": 217, "statues_count": 3598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-24T00:00:04.000Z"), "id": 691168577836023809, "text": "@StacieSeldin Noted.", "in_reply_to_status": 691168368980811776, "in_reply_to_user": 844287396, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 844287396 }}, "user": { "id": 54795420, "name": "Chris Antonacci", "screen_name": "chrisantonacci", "lang": "en", "location": "Chicago", "create_at": date("2009-07-07"), "description": "null", "followers_count": 282, "friends_count": 889, "statues_count": 2158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168578289037312, "text": "it's 11:59 and Ethan young just asked if I want my groceries bagged", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 858558907, "name": "ally", "screen_name": "allysterzer", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2012-10-02"), "description": "null", "followers_count": 366, "friends_count": 287, "statues_count": 9014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CA", "id": "d07feb9e5b1ce37c", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-118.043083,33.720269 -117.944373,33.774358") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 684550, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168578465193985, "text": "@stevinicole04 one stereotypical station at a time, stevi", "in_reply_to_status": 691168017015648256, "in_reply_to_user": 624428291, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 624428291 }}, "user": { "id": 485917048, "name": "Charlie", "screen_name": "TheStagerman", "lang": "en", "location": " ⚓️ Seattle, WA ", "create_at": date("2012-02-07"), "description": "the Seahawks & me; a love story/abusive relationship", "followers_count": 257, "friends_count": 383, "statues_count": 9240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shoreline, WA", "id": "ecb63eb9d9d83b31", "name": "Shoreline", "place_type": "city", "bounding_box": rectangle("-122.395019,47.733761 -122.291436,47.777982") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363960, "cityName": "Shoreline" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168578553274368, "text": "If we're talking I'm only talking to you no one else and I mean that 100% ✊�� I ain't going anywhere.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2914199353, "name": "Mikey Hernandez", "screen_name": "mikeyhe06435501", "lang": "en", "location": "null", "create_at": date("2014-11-29"), "description": "let the fakes talk behind your back they are in back for a reason", "followers_count": 229, "friends_count": 499, "statues_count": 217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168578603716608, "text": "Wind 0.0 mph ---. Barometer 30.227 in, Steady. Temperature 23.8 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 23, "statues_count": 57899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168578981105665, "text": "Temp: 22.0°F - Dew Point: 13.7° - Wind: 11.5 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 29.96in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 11, "statues_count": 12456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168579136417792, "text": "���������������� https://t.co/JggyJShhcl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 195909357, "name": "Waffle House Papi", "screen_name": "2Girls1Richard", "lang": "en", "location": "Atlanna Waffle House", "create_at": date("2010-09-27"), "description": "don't become who hurt you.", "followers_count": 3877, "friends_count": 1658, "statues_count": 467664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168579174051840, "text": "@castromaritza23 text back hoe", "in_reply_to_status": 691124782809554945, "in_reply_to_user": 2682925218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2682925218 }}, "user": { "id": 769942777, "name": "Britani", "screen_name": "Britany_cabrera", "lang": "en", "location": "newbury park , ca", "create_at": date("2012-08-20"), "description": "I'm liking of whom Im becoming", "followers_count": 212, "friends_count": 274, "statues_count": 3327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168579832668160, "text": "Wind 5.0 mph N. Barometer 29.895 in, Falling. Temperature 15.2 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 3835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168579883003904, "text": "https://t.co/p3yPWZIdYx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 19478, "friends_count": 13348, "statues_count": 5628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-24T00:00:05.000Z"), "id": 691168580168101888, "text": "Asta mañana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 729964386, "name": "goose is loose", "screen_name": "goose_12345", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "flying high since 99' / fastest goose alive / future job: part time gambler", "followers_count": 295, "friends_count": 212, "statues_count": 12666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beloit, WI", "id": "000fc3e67b627897", "name": "Beloit", "place_type": "city", "bounding_box": rectangle("-89.079237,42.496019 -88.948817,42.610064") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55105, "countyName": "Rock", "cityID": 5506500, "cityName": "Beloit" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530945178898432, "text": "My wcw and her mcm look at each other like this https://t.co/J8cJ0HZQfn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595133308, "name": "backyard allan", "screen_name": "SuplexDaddy", "lang": "en", "location": "The Backyard", "create_at": date("2012-05-30"), "description": "my goal is to be like Justin Timberlake #BBT #BlackLivesMatter #TXST19 https://twitter.com/suplexinghoes/status/686421487327129600", "followers_count": 897, "friends_count": 290, "statues_count": 77913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530945254412289, "text": "I was too turnt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4503428538, "name": "Andrea", "screen_name": "torresdrea13", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-12-08"), "description": "Dallas , TX", "followers_count": 139, "friends_count": 158, "statues_count": 443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530945413820416, "text": "Not today Satan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330563561, "name": "kate", "screen_name": "iLikeK8Daily", "lang": "en", "location": "Houston, TX", "create_at": date("2011-07-06"), "description": "OH :: HTX :: DG", "followers_count": 751, "friends_count": 442, "statues_count": 37026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530945594130434, "text": "Stop https://t.co/hug4BagVk7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28652525, "name": "بيبي", "screen_name": "_aeeebie", "lang": "en", "location": "whoville", "create_at": date("2009-04-03"), "description": "null", "followers_count": 448, "friends_count": 423, "statues_count": 46863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530946202304512, "text": "@loera1_ sweet", "in_reply_to_status": 691530867668185088, "in_reply_to_user": 1523317488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1523317488 }}, "user": { "id": 3038396215, "name": "Tylo Ren", "screen_name": "briskk_", "lang": "en", "location": "Fontana, CA", "create_at": date("2015-02-23"), "description": "Today, tomorrow, & forever", "followers_count": 340, "friends_count": 255, "statues_count": 5950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530946382843904, "text": "03:00:01 |Temp: 28.2ºF | Wind Chill 28.2ºF |Dew Point 25.6ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WSW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 88970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530946701438977, "text": "@Glam_And_Gore hi angel I hope you're having a great night, please keep making videos you're literally the source of my happiness. thank you", "in_reply_to_status": -1, "in_reply_to_user": 2456142770, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2456142770 }}, "user": { "id": 2595509450, "name": "kate ¯\\_(ツ)_/¯", "screen_name": "katecove11", "lang": "en", "location": "she/her", "create_at": date("2014-06-29"), "description": "annoyed intersectional feminist who is unafraid to call out your shit. i also like one direction a whole lot. 12.23.12. isfj.", "followers_count": 336, "friends_count": 554, "statues_count": 9468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530946772733952, "text": "https://t.co/o4HZt44hZk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2946724459, "name": "teej", "screen_name": "swaggy_teej", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "gonna make it with or without you, you choose. snapchat:swaggy_teej #oklahomies", "followers_count": 604, "friends_count": 474, "statues_count": 3455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530947423002624, "text": "I had insomnia all throughout high-school and I never saw it as a problem. Now if I don't get 8 hours of sleep I can't function.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 705787957, "name": "Toxic.", "screen_name": "rstonebraker_", "lang": "en", "location": "null", "create_at": date("2012-07-19"), "description": "I'm a threat to human kind.", "followers_count": 333, "friends_count": 232, "statues_count": 1638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530948224139264, "text": "Wind 0 mph --. Barometer 30.13 in, Falling slowly. Temperature 34.7 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530948681306112, "text": "Oh aight. https://t.co/2quhnxkgiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348128151, "name": "Jacobe from Da Couch", "screen_name": "MUHFUGGINJKELLS", "lang": "en", "location": "#NüCrü", "create_at": date("2011-08-03"), "description": "You met me at a very strange time in my life....", "followers_count": 6562, "friends_count": 3603, "statues_count": 188873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530948840521728, "text": "CLINICAL DIETITIAN / SILVER CITY, NM - Compass Group USA: (#SilverCity, NM) https://t.co/UB5wpLvIH6 #Healthcare #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-108.280326,32.770075"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SilverCity", "Healthcare", "Job", "Jobs", "Hiring" }}, "user": { "id": 126371773, "name": "Compass USA Careers", "screen_name": "CompassUSAJobs", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-03-25"), "description": "Compass Group USA has the best career opportunities in the country for the industry’s greatest talent in #culinary, #foodservice management and support services", "followers_count": 735, "friends_count": 121, "statues_count": 16661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver City, NM", "id": "91717be9c3e8c07d", "name": "Silver City", "place_type": "city", "bounding_box": rectangle("-108.31294,32.726521 -108.221354,32.82689") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35017, "countyName": "Grant", "cityID": 3573260, "cityName": "Silver City" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530948966387712, "text": "I'll chat w/ Kyoung-Joo Kim on New York Radio Korea, 10:35 am, Eastern Time. Listen live https://t.co/MRfZwPNEi7 Topics: #Palin #Bloomberg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Palin", "Bloomberg" }}, "user": { "id": 543094220, "name": "Paul Schlienz", "screen_name": "PaulSchlienz", "lang": "en", "location": "Tumwater, WA", "create_at": date("2012-04-01"), "description": "Writer, editor, radio broadcaster, world traveler, voracious reader, political junkie, music lover, film connoisseur, history buff and much more.", "followers_count": 799, "friends_count": 1482, "statues_count": 2049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tumwater, WA", "id": "12148f3ffcceb203", "name": "Tumwater", "place_type": "city", "bounding_box": rectangle("-123.010086,46.913138 -122.860365,47.029499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-01-25T00:00:00.000Z"), "id": 691530949016838144, "text": "https://t.co/L14lWH6YQJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 307945080, "name": "katlo ren", "screen_name": "b1urryf4c3", "lang": "en", "location": "ny", "create_at": date("2011-05-30"), "description": "i'm 90% sure matt is kylo ren", "followers_count": 1521, "friends_count": 319, "statues_count": 43223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, NY", "id": "876ed0bcf9594134", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-73.701496,40.582128 -73.637403,40.595119") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3643335, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530949406765056, "text": "454 - cutie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316729563, "name": "Santiago", "screen_name": "TubbyJ95", "lang": "en", "location": "559", "create_at": date("2011-06-13"), "description": "Trying to be the person I wasn't yesterday", "followers_count": 1054, "friends_count": 597, "statues_count": 89259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530949700358144, "text": "@Veebabyy__ https://t.co/sKNu49N65M", "in_reply_to_status": 691530892678828032, "in_reply_to_user": 226789027, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3260706523 }}, "user": { "id": 226789027, "name": "Cristian", "screen_name": "Criss_1103", "lang": "en", "location": "null", "create_at": date("2010-12-14"), "description": "Rip My Homie Marcos Pineda, Ruben Corona Martinez & My Brother Edwin Frankie Galicia ... |", "followers_count": 616, "friends_count": 596, "statues_count": 49251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King City, CA", "id": "0f119d2beee20b90", "name": "King City", "place_type": "city", "bounding_box": rectangle("-121.148723,36.195958 -121.111924,36.235099") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 638520, "cityName": "King City" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530950090461184, "text": "@fernandez_madi lol love you Madi ��", "in_reply_to_status": 691530315647492098, "in_reply_to_user": 606770028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 606770028 }}, "user": { "id": 157874297, "name": "Maximilliano", "screen_name": "Marrrk_", "lang": "en", "location": "Long Beach, CA", "create_at": date("2010-06-20"), "description": "Your future E! Network reporter.", "followers_count": 673, "friends_count": 623, "statues_count": 10504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530950556147713, "text": "@aaaahitsashley wavy does sound better than curvy now that I think about it", "in_reply_to_status": 691507789752090624, "in_reply_to_user": 63484547, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63484547 }}, "user": { "id": 36269028, "name": "Tyler Parrish", "screen_name": "tylerisscott", "lang": "en", "location": "Henderson, NC", "create_at": date("2009-04-28"), "description": "Im going to go back to Nike sb's. They've held up the best and are comfy af @LilWhiteKanye", "followers_count": 255, "friends_count": 251, "statues_count": 19226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530950841249792, "text": "all I want in life to be happy is to travel the world & own a dog", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388033839, "name": "DD❁", "screen_name": "diannasalgado", "lang": "en", "location": "ATX", "create_at": date("2011-10-09"), "description": "i'm always down for an adventure SEU", "followers_count": 758, "friends_count": 444, "statues_count": 15882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530951101288448, "text": "@TakiyahNashaii ���� https://t.co/Vsn1lsjUZ5", "in_reply_to_status": -1, "in_reply_to_user": 514457812, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 514457812 }}, "user": { "id": 214646777, "name": "toya ❣", "screen_name": "1montoyaa", "lang": "en", "location": "null", "create_at": date("2010-11-11"), "description": "null", "followers_count": 1381, "friends_count": 614, "statues_count": 43453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville, AL", "id": "d5dbaf62e7106dc4", "name": "Jacksonville", "place_type": "city", "bounding_box": rectangle("-85.800502,33.752607 -85.724207,33.840282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 138272, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530951726215169, "text": "@TheFitGoals @nkittty I'm going to pray for her.", "in_reply_to_status": 660201592830881792, "in_reply_to_user": 348504242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348504242, 284312534 }}, "user": { "id": 331467674, "name": "Stu from '92", "screen_name": "imFliPs", "lang": "en", "location": "around", "create_at": date("2011-07-07"), "description": "FHSU|23|Graphic Design Major|Camera Operator ••• Do Stay Positive ••• Be you because no one knows who that is but you.. -EWSJ @theSAPproject", "followers_count": 1224, "friends_count": 1009, "statues_count": 32731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hays, KS", "id": "b0e50abc7e3f7363", "name": "Hays", "place_type": "city", "bounding_box": rectangle("-99.354684,38.849348 -99.280695,38.914552") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20051, "countyName": "Ellis", "cityID": 2031100, "cityName": "Hays" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530951957069824, "text": "Wind 4.0 mph SE. Barometer 29.912 in, Steady. Temperature 36.8 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952066101248, "text": "That movie. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1067441460, "name": "ℓαиєу ❣", "screen_name": "Laney_Nichole_", "lang": "en", "location": "Narrows, VA", "create_at": date("2013-01-06"), "description": "probably don't know, most likely don't care // NHS 2018", "followers_count": 975, "friends_count": 918, "statues_count": 36908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Narrows, VA", "id": "94dcc2d81d4fd299", "name": "Narrows", "place_type": "city", "bounding_box": rectangle("-80.826928,37.320119 -80.795759,37.344971") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51071, "countyName": "Giles", "cityID": 5154904, "cityName": "Narrows" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952263221249, "text": "Has anyone panted the prism seed yet? And has it grown? #katycats #KatyPerry #prism #katycat #prismaticworldtour @katyperry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "katycats", "KatyPerry", "prism", "katycat", "prismaticworldtour" }}, "user_mentions": {{ 21447363 }}, "user": { "id": 3443817742, "name": "M∆RKYM∆RK", "screen_name": "Mark_KatyCat", "lang": "en", "location": "in Katy Perrys love! ", "create_at": date("2015-08-25"), "description": "photographer [hoping to be a celeb/fashion photographer] #Katycat4life Rony Alwin 1x --- KP, 1R, RiRi, MATD, M5 [PWT-8/14/14]", "followers_count": 96, "friends_count": 168, "statues_count": 1263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warren, OH", "id": "579f9e4d93c529b1", "name": "Warren", "place_type": "city", "bounding_box": rectangle("-80.894004,41.198089 -80.740485,41.272574") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3980892, "cityName": "Warren" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952435171328, "text": "Wind 1.0 mph SSE. Barometer 30.130 in, Falling slowly. Temperature 26.2 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952577691648, "text": "@ebbtideapp Tide in Levelock, Alaska 01/25/2016\n Low 4:43am 1.0\nHigh 7:05am 7.6\n Low 4:34pm -0.2\nHigh 8:15pm 10.2\n Low 5:35am 0.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-156.8317,59.1133"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 13, "friends_count": 1, "statues_count": 2030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2164, "countyName": "Lake and Peninsula" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952800145408, "text": "Temp: 34.7°F Wind:0.0mph Pressure: 30.135hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952829378561, "text": "Wind 13.0 mph SSW. Barometer 1010.57 mb, Falling. Temperature 45.0 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 27, "friends_count": 124, "statues_count": 12088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-25T00:00:01.000Z"), "id": 691530952988688384, "text": "Why don't we make an agreement then", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2282772878, "name": "Andie", "screen_name": "_xoxoandie", "lang": "en", "location": "null", "create_at": date("2014-01-08"), "description": "null", "followers_count": 884, "friends_count": 367, "statues_count": 22513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, CA", "id": "e6aaf7cb58e7857c", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-115.594963,32.817908 -115.544378,32.87646") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 636280, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530953844465664, "text": "If this happened in Texas no one would go outside to shovel anything. https://t.co/FKnI1T6cgW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33035057, "name": "Feb 12", "screen_name": "ciara95", "lang": "en", "location": "Texas", "create_at": date("2009-04-18"), "description": "Ciara | 20 |", "followers_count": 160, "friends_count": 148, "statues_count": 4985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530954247016448, "text": "�� https://t.co/Fhm0JGplCa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 605137059, "name": "GameTight✨", "screen_name": "alwaysbrooke_", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "half simp/half pimp. DTX|UNT19", "followers_count": 1320, "friends_count": 711, "statues_count": 22344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530954255405056, "text": "I don't need anything to get me through the night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325954888, "name": "bbytiff", "screen_name": "tiffanyspano", "lang": "en", "location": "california, san diego ", "create_at": date("2013-04-03"), "description": "God gives his hardest battles to his strongest soldiers.", "followers_count": 585, "friends_count": 345, "statues_count": 6695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530954318319616, "text": "Stressed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3161647322, "name": "Donovan", "screen_name": "100xextra", "lang": "en", "location": "null", "create_at": date("2015-04-17"), "description": "I lava Kevin Macias", "followers_count": 261, "friends_count": 240, "statues_count": 7073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530954687549440, "text": "Geometría en un cultivo abandonado. @ United States https://t.co/1fJkbiMdfd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.0,38.0"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 244045005, "name": "Victorino García", "screen_name": "Torinoga", "lang": "es", "location": "null", "create_at": date("2011-01-28"), "description": "Maestro del mirar.", "followers_count": 462, "friends_count": 722, "statues_count": 13596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas, USA", "id": "27c45d804c777999", "name": "Kansas", "place_type": "admin", "bounding_box": rectangle("-102.051769,36.99311 -94.588081,40.003282") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20015, "countyName": "Butler" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530954960195584, "text": "Women needed men in the 1950s. Women literally couldn't do anything without a male present.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25893699, "name": "Actually a Cat", "screen_name": "DatGirl_ICEY", "lang": "en", "location": "Somewhere teaching nouns ", "create_at": date("2009-03-22"), "description": "My cousin told me Twitter had the cat memes so I'm here", "followers_count": 5081, "friends_count": 483, "statues_count": 344431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530955140403200, "text": "Missing someone & not being able to see them is the worst.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2302358916, "name": "karlssss♥️♠️", "screen_name": "karlamunoz907", "lang": "en", "location": "TX ", "create_at": date("2014-01-20"), "description": "stubborn af with a good heart.|| sc: @k_munoz96 || insta: @kmunoz907__", "followers_count": 724, "friends_count": 363, "statues_count": 17489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Floresville, TX", "id": "3f53ed8eba395810", "name": "Floresville", "place_type": "city", "bounding_box": rectangle("-98.179363,29.115435 -98.135405,29.157209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48493, "countyName": "Wilson", "cityID": 4826160, "cityName": "Floresville" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530955987800064, "text": "https://t.co/fqd6DW7rNt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 19686, "friends_count": 14305, "statues_count": 6047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530955996041216, "text": "Mine & @MysteryBabyLove snaps rn tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39648729 }}, "user": { "id": 539458775, "name": "Emilce", "screen_name": "emila_11", "lang": "en", "location": "831, California ", "create_at": date("2012-03-28"), "description": "Twist my mind against my soul.", "followers_count": 230, "friends_count": 198, "statues_count": 17937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530956310622208, "text": "Mistakes are meant for learning, not repeating.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3235541220, "name": "Itzel v Flores", "screen_name": "itzelvictoriaa", "lang": "en", "location": "null", "create_at": date("2015-06-03"), "description": "null", "followers_count": 305, "friends_count": 253, "statues_count": 2128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530956927160320, "text": "So I guess I won't have another day off till Saturday ������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 267035255, "name": "toni❤️", "screen_name": "tonilovees", "lang": "en", "location": "Santa Paula,Ca", "create_at": date("2011-03-15"), "description": "null", "followers_count": 327, "friends_count": 338, "statues_count": 6806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530956931334144, "text": "why should I care when you never have", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363551184, "name": "mikael", "screen_name": "mikael_kia", "lang": "en", "location": "washington ", "create_at": date("2014-02-26"), "description": "probably eating fruit", "followers_count": 225, "friends_count": 122, "statues_count": 4002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tumwater, WA", "id": "12148f3ffcceb203", "name": "Tumwater", "place_type": "city", "bounding_box": rectangle("-123.010086,46.913138 -122.860365,47.029499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-01-25T00:00:02.000Z"), "id": 691530957011181568, "text": "Mood ain't shit ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351872043, "name": "Mamii✨", "screen_name": "Allhailclaudia_", "lang": "en", "location": "Jerseyyy", "create_at": date("2011-08-09"), "description": "Cj ❤️", "followers_count": 1489, "friends_count": 1280, "statues_count": 29221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwood, NJ", "id": "0036534d6f287a14", "name": "Blackwood", "place_type": "city", "bounding_box": rectangle("-75.088811,39.726424 -74.968468,39.852636") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3406040, "cityName": "Blackwood" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530957896159233, "text": "@DragonflyJonez about to watch elephant in the room with a 12 pack...WATTBA", "in_reply_to_status": -1, "in_reply_to_user": 19725981, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19725981 }}, "user": { "id": 216020601, "name": "Boogie Milk", "screen_name": "chadphillipsjr", "lang": "en", "location": "New York, NY", "create_at": date("2010-11-15"), "description": "Get Busy Livin' or Get Busy Dying #catchthatnut", "followers_count": 380, "friends_count": 349, "statues_count": 17809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayonne, NJ", "id": "3d3c56338b6a3b4a", "name": "Bayonne", "place_type": "city", "bounding_box": rectangle("-74.146932,40.643773 -74.0658,40.697794") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3403580, "cityName": "Bayonne" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530958252564480, "text": "@bigblackniggey I'm woke af", "in_reply_to_status": 691530893937147905, "in_reply_to_user": 488749142, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 488749142 }}, "user": { "id": 2239958064, "name": "Art Shawty", "screen_name": "aevonxavia", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-12-10"), "description": "for artwork: zavia.xa@gmail.com #longlive2", "followers_count": 755, "friends_count": 395, "statues_count": 12819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530958642765824, "text": "Have Fun At Work Day - 29th January 2016 https://t.co/ltvjzf14VA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442686600, "name": "Sandra Crathern", "screen_name": "Healthysandra", "lang": "en", "location": "Worthing", "create_at": date("2011-12-21"), "description": "My unique tools help clients improve their lives. If you want a health coach who really will be there with you every step of the way, DM me now.", "followers_count": 663, "friends_count": 320, "statues_count": 4259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530958680395777, "text": "I'm addicted to this show thanks to you https://t.co/7OQcUpmmWG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 955838372, "name": "shawty", "screen_name": "zabdydbaz", "lang": "en", "location": "H-Town", "create_at": date("2012-11-18"), "description": "live and let live ♡ Alani ♡", "followers_count": 607, "friends_count": 425, "statues_count": 16348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530958713933824, "text": "51.7F (Feels: 51.7F) - Humidity: 90% - Wind: 5.4mph E - Gust: 9.2mph - Pressure: 1029.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 223932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530958772834304, "text": "but I'm bouta retire this shit soon", "in_reply_to_status": 691529767619842048, "in_reply_to_user": 320730091, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320730091, "name": "#FreewayDre", "screen_name": "google_dre", "lang": "en", "location": "$outh $ide Chicago- Englewood ", "create_at": date("2011-06-20"), "description": "RIP Rio & tookie - if you scared to take a chance how the fuck u gone get rich?!", "followers_count": 2109, "friends_count": 656, "statues_count": 76746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530959695405056, "text": "St. Louis was a fail ����������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163579370, "name": "Slayanna", "screen_name": "IHaveStandards_", "lang": "en", "location": "Detroit ", "create_at": date("2010-07-06"), "description": "KzooTwitter rides my dick... Leading by example. Hustle don't never go unnoticed.", "followers_count": 1070, "friends_count": 841, "statues_count": 70291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lebanon, IL", "id": "6901e25802f2ac17", "name": "Lebanon", "place_type": "city", "bounding_box": rectangle("-89.846355,38.589179 -89.798248,38.621381") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1742496, "cityName": "Lebanon" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530959762644992, "text": "Night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227152188, "name": "Ike'l☪", "screen_name": "indilovesyouuu", "lang": "en", "location": "757", "create_at": date("2010-12-15"), "description": "#Beautyfromwithin | Norfolk State University", "followers_count": 1143, "friends_count": 949, "statues_count": 13720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530960488140800, "text": "@chloe_kaintz ����", "in_reply_to_status": 691471641809801216, "in_reply_to_user": 58894695, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 58894695 }}, "user": { "id": 412404786, "name": "Kim Dixon", "screen_name": "KimmieDixon5", "lang": "en", "location": "null", "create_at": date("2011-11-14"), "description": "null", "followers_count": 23, "friends_count": 101, "statues_count": 276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530960601350145, "text": "I'm the type of nigga we chillin at the crib Ima turn on some music and put on a concert so you laugh ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586353266, "name": "Kee Riche$", "screen_name": "IamKeeRiches", "lang": "en", "location": "Compton", "create_at": date("2013-07-11"), "description": "#WestRiches - kianjai26@gmail.com", "followers_count": 1513, "friends_count": 995, "statues_count": 35316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530960861544448, "text": "Wind 0.0 mph SW. Barometer 30.15 in, Rising slowly. Temperature 26.2 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 26, "friends_count": 100, "statues_count": 156971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961184382976, "text": "Want to work at DialAmerica? We're #hiring in #Orlando, FL! Click for details: https://t.co/06JPiVFnaH #milspouse https://t.co/scrkug20Lb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2849169,28.4750405"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Orlando", "milspouse" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 293, "friends_count": 217, "statues_count": 3598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961243082754, "text": "HAPPY BIRTHDAY EMILY ILY I HOPE ITS A GREAT ONE. THANKS FOR MAKING TODAY A HOLIDAY AND BEING ALIVE\n@shmarsen MAKE TODAY GREAT ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 633058916 }}, "user": { "id": 3248647879, "name": "awesome austin", "screen_name": "austin_wallen", "lang": "en", "location": "USA", "create_at": date("2015-06-18"), "description": "@Trinimmortal stole my Gromp once https://youtu.be/e6ccCCsMwTQ?t=2m35s", "followers_count": 109, "friends_count": 89, "statues_count": 5831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camas, WA", "id": "219460d3060c1d2e", "name": "Camas", "place_type": "city", "bounding_box": rectangle("-122.474225,45.57579 -122.367318,45.633039") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5309480, "cityName": "Camas" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961301983233, "text": "@jessicakwartler @verified I want one. Be my manager ������", "in_reply_to_status": 690308442662268928, "in_reply_to_user": 84387036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 84387036, 63796828 }}, "user": { "id": 2785416397, "name": "Meredith Μaselli", "screen_name": "MeredithMaselli", "lang": "en", "location": "United States", "create_at": date("2014-09-01"), "description": "Full time travel photographer instagram @meredith_maselli snapchat meredith_maselli", "followers_count": 1331, "friends_count": 592, "statues_count": 517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961486487552, "text": "Just posted a photo @ Sam, Spencer, & Palmers Place https://t.co/5cTQ6QQore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.44442749,35.09097672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220484371, "name": "Sam grice", "screen_name": "Samuelggrice", "lang": "en", "location": "Conway, AR", "create_at": date("2010-11-27"), "description": "null", "followers_count": 52, "friends_count": 201, "statues_count": 1382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961499131904, "text": "She's a beauty https://t.co/IOmsgxABEe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345565110, "name": "Warren ™", "screen_name": "Aqua_Ladd", "lang": "en", "location": "Huntsville, AL", "create_at": date("2011-07-30"), "description": "Y.S.M #AAMU19 334 ✈️ 256 N.W.A (Nigga With Ambitions) G R E E N S B O R O", "followers_count": 2085, "friends_count": 1900, "statues_count": 16865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961612361728, "text": "No matter I do I always come off as a dick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 743037774, "name": "Zach Josey", "screen_name": "Z2CLASSY", "lang": "en", "location": "Tuscaloosa, AL ✈️ ⛅️", "create_at": date("2012-08-07"), "description": "Live for today UA '19", "followers_count": 6173, "friends_count": 610, "statues_count": 41378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961687678976, "text": "@lyciafaith what is the name sunnies that you are wearing in your snap story? OBSESSED!! Need them in my life ��", "in_reply_to_status": -1, "in_reply_to_user": 525949931, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525949931 }}, "user": { "id": 24441362, "name": "MelissaYvette", "screen_name": "myvette48", "lang": "en", "location": "Los Angeles /Texas", "create_at": date("2009-03-14"), "description": "Just a Texas girl here in the city of angels. SoCal. snapchat: MeliMel1012 instagram: Myvette48. MUA Skincare specialist #allthingsbeauty #MUA #beauty101", "followers_count": 262, "friends_count": 210, "statues_count": 13072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-25T00:00:03.000Z"), "id": 691530961721409536, "text": "Wind 0.0 mph ---. Barometer 30.220 in, Rising. Temperature 16.0 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 6037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530961914171393, "text": "@JorgeLC52 he ain't winning NOTHING", "in_reply_to_status": 691530893354086400, "in_reply_to_user": 985372885, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 985372885 }}, "user": { "id": 28743010, "name": "De'Asssshley", "screen_name": "SuperAsh18", "lang": "en", "location": "Nampa ➡️ La Grandé", "create_at": date("2009-04-03"), "description": "Ashley Rae Allen | 21 | EOU Cheer & Dance | Future Erin Andrews |", "followers_count": 608, "friends_count": 308, "statues_count": 36219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grande, OR", "id": "68bfe95c389e2c43", "name": "La Grande", "place_type": "city", "bounding_box": rectangle("-118.117457,45.302331 -118.06023,45.343556") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41061, "countyName": "Union", "cityID": 4140350, "cityName": "La Grande" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530961993904128, "text": "@maddy_mello ok yes ��", "in_reply_to_status": 691515179666792448, "in_reply_to_user": 702632012, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 702632012 }}, "user": { "id": 842790079, "name": "ry", "screen_name": "ryleegrace_15", "lang": "en", "location": "Hanford, CA", "create_at": date("2012-09-23"), "description": "tomorrow will be better || B&A", "followers_count": 1626, "friends_count": 815, "statues_count": 33935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530962937651201, "text": "I love staring at Audrey, she's so cute it's unreal ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259989099, "name": "Nicki", "screen_name": "Nicole_aec", "lang": "en", "location": "null", "create_at": date("2011-03-02"), "description": "La vie est Audrey Bellè", "followers_count": 362, "friends_count": 408, "statues_count": 23090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530962941771780, "text": "Good food run for the night with the boys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3829908727, "name": "Shaq", "screen_name": "STiatia9", "lang": "en", "location": "West Valley City, UT", "create_at": date("2015-10-08"), "description": "null", "followers_count": 186, "friends_count": 195, "statues_count": 2741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963155849216, "text": "got not gaf issues https://t.co/U0AyRhMblJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329885920, "name": "tayy", "screen_name": "taylorsymoine", "lang": "en", "location": "boro", "create_at": date("2011-07-05"), "description": "I'll let her push the foreign right now if she worth it. SEN16R bih.. #famu20", "followers_count": 1961, "friends_count": 1419, "statues_count": 34654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963319443456, "text": "@DarkElektryk @Eli_the_greatt https://t.co/I98iaMAgDv", "in_reply_to_status": -1, "in_reply_to_user": 948828056, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 948828056, 1976682072 }}, "user": { "id": 2849054891, "name": "Cody Quarles", "screen_name": "OBOinTHIS", "lang": "en", "location": "null", "create_at": date("2014-10-28"), "description": "The one and only . #NAEBZ", "followers_count": 294, "friends_count": 480, "statues_count": 850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wood Dale, IL", "id": "c5657257dc77abae", "name": "Wood Dale", "place_type": "city", "bounding_box": rectangle("-88.004965,41.9425 -87.959346,41.992957") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1782985, "cityName": "Wood Dale" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963344449536, "text": "happy bday Calum!! love you dude @Calum5SOS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 403255314 }}, "user": { "id": 2733154460, "name": "juju", "screen_name": "nightcnmars", "lang": "en", "location": "sf", "create_at": date("2014-08-14"), "description": "Arizona emo group", "followers_count": 304, "friends_count": 58, "statues_count": 7960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963470454784, "text": "Good to have sex too �� https://t.co/5WWsh57vF5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368129440, "name": "EVERYTHINGLILTEE", "screen_name": "_keywes_", "lang": "en", "location": "null", "create_at": date("2011-09-04"), "description": "#RIPTEE #TEEWORLD #MSU #THEZEST #HQ", "followers_count": 1137, "friends_count": 1080, "statues_count": 29726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963679969280, "text": "Join the Lexmark team! See our latest #IT #job opening here: https://t.co/9DaQIJ7u3A #Lexington, KY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4951359,38.0317136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "job", "Lexington", "Hiring" }}, "user": { "id": 26266228, "name": "Lexmark Careers", "screen_name": "lexmarkcareers", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-03-24"), "description": "Careers at Lexmark -- Open the possibilities", "followers_count": 1645, "friends_count": 1143, "statues_count": 550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21067, "countyName": "Fayette", "cityID": 2146027, "cityName": "Lexington-Fayette" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963839373312, "text": "We're #hiring! Click to apply: Agile Program Manager - https://t.co/hoaiU1fLo6 #Engineering #Boston, MA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0573421,42.3539011"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Engineering", "Boston", "Job", "Jobs" }}, "user": { "id": 455971331, "name": "Continuum", "screen_name": "FollowContinuum", "lang": "en", "location": "Boston, MA", "create_at": date("2012-01-05"), "description": "A unified managed services platform that enables MSPs to profitably backup, monitor, troubleshoot and maintain IT environments.", "followers_count": 2587, "friends_count": 2022, "statues_count": 10329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530963952599041, "text": "I may have found the most sexist Twitter account, but forget girls who wanna be known for being athletes right?�� https://t.co/gCnMPppLAG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26846012, "name": "jt", "screen_name": "jordantingman", "lang": "en", "location": "quah - wsu", "create_at": date("2009-03-26"), "description": "null", "followers_count": 1118, "friends_count": 961, "statues_count": 11702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213106,46.708177 -117.095324,46.753398") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530964204285952, "text": "Bippitt boppity back the fuck up is my new favorite slang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 839305124, "name": "Victoria Ann", "screen_name": "Victoria_Stodds", "lang": "en", "location": "Reno, NV", "create_at": date("2012-09-22"), "description": "UNR❤️ Criminal Justice Major", "followers_count": 572, "friends_count": 550, "statues_count": 6805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530964326051840, "text": "@BillKouts1 You too, my brother! Have a stupendous week!", "in_reply_to_status": 691530187247374336, "in_reply_to_user": 450042226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450042226 }}, "user": { "id": 162492285, "name": "Jeff Braddock", "screen_name": "Braddock512", "lang": "en", "location": "Austin, TX", "create_at": date("2010-07-03"), "description": "North American Community Manager for Battlefield. Gamer, hedgehog friend, whiskey lover. Tweets are my own.", "followers_count": 1925, "friends_count": 1452, "statues_count": 12407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530964359630848, "text": "Wind 0.0 mph ---. Barometer 30.165 in, Steady. Temperature 17.3 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530964434956288, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1345217858, "name": "Looché ♻️", "screen_name": "MaddMannn_", "lang": "en", "location": "Ya Bitch $tomach ", "create_at": date("2013-04-11"), "description": "Trill Niqqa neva Break or Fold #FreeGMoney #LL™", "followers_count": 399, "friends_count": 265, "statues_count": 10358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waggaman, LA", "id": "1e70bb6914d1082b", "name": "Waggaman", "place_type": "city", "bounding_box": rectangle("-90.257405,29.907391 -90.20738,29.964906") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2279100, "cityName": "Waggaman" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530964644843520, "text": "Wind 0.4 mph S. Barometer 30.048 in, Falling. Temperature 37.9 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 57993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965089255424, "text": "get it right get it tight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188508499, "name": "lil b", "screen_name": "yungbrandyy", "lang": "en", "location": "outta this world", "create_at": date("2010-09-08"), "description": "ur fav blaxican", "followers_count": 406, "friends_count": 280, "statues_count": 24292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965202501632, "text": "back at it #UCM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UCM" }}, "user": { "id": 356007780, "name": "Ginger", "screen_name": "PatriciaRazo58", "lang": "en", "location": "Oakland, CA", "create_at": date("2011-08-15"), "description": "UCM'19 • BRPW ❤️ • 18", "followers_count": 223, "friends_count": 348, "statues_count": 12431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965311627264, "text": "why am I always on Twitter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1515665083, "name": "evan", "screen_name": "evantuaIIy", "lang": "en-gb", "location": "løs angeles", "create_at": date("2013-06-14"), "description": "one step heavy and two steps high", "followers_count": 2999, "friends_count": 161, "statues_count": 27879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965395451904, "text": "#Chicago, IL #Job: Team Leader - Compliance Support at Trustwave https://t.co/XZRisfgNee #infosec #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chicago", "Job", "infosec", "Jobs", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 282, "friends_count": 160, "statues_count": 550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965672292352, "text": "Wind 1.0 mph SSE. Barometer 30.047 in, Steady. Temperature 27.2 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 374, "friends_count": 288, "statues_count": 7748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530965865205760, "text": "Jessica ain't talk to me all week but a day before I start my new job she wanna talk about \"do I get free drinks\" bih ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2604879653, "name": "Fanny Paredes", "screen_name": "fannyparedess", "lang": "en", "location": "null", "create_at": date("2014-06-12"), "description": "@TreBreeezy is daddy ❤", "followers_count": 220, "friends_count": 154, "statues_count": 8230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-01-25T00:00:04.000Z"), "id": 691530966087573504, "text": "I'm just now watching these highlights", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396008663, "name": "señor beetru", "screen_name": "beestaytru", "lang": "en", "location": "null", "create_at": date("2011-10-22"), "description": "@beetruclothing // berkeley yay area", "followers_count": 435, "friends_count": 297, "statues_count": 33116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530966490312705, "text": "@SarahMJade I'm ready to, I want to either add it to what I have on my thigh or my side. I'm just trying to figure out what to do.", "in_reply_to_status": 691530511735353344, "in_reply_to_user": 1240535666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1240535666 }}, "user": { "id": 470882909, "name": "Big Dumb Idiot", "screen_name": "Purplepanda1992", "lang": "en", "location": "Utah/North Carolina", "create_at": date("2012-01-21"), "description": "U.S. Marine who sadly owns three fanny packs, but let me bring back my wow-factor by telling you that this one time, I finished a whole tube of chapstick.", "followers_count": 914, "friends_count": 1028, "statues_count": 27389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Havelock, NC", "id": "de4ce211af5cd6cf", "name": "Havelock", "place_type": "city", "bounding_box": rectangle("-76.948247,34.85557 -76.853819,34.952004") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37049, "countyName": "Craven", "cityID": 3730120, "cityName": "Havelock" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530966720995329, "text": "Wind 0 mph ---. Barometer 30.49 in, Rising. Temperature 32.0 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 52, "friends_count": 69, "statues_count": 25223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530966825721857, "text": "I wish I was an actor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2722993326, "name": "In-n-Out Enthusiast", "screen_name": "Shawn_LEHC", "lang": "en", "location": "⚒⚒⚒ ", "create_at": date("2014-08-10"), "description": "❤️@bby_gup❤️ // @SolidGroundLV", "followers_count": 256, "friends_count": 199, "statues_count": 5547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967157207040, "text": "tenille – alright, ok", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31347815, "name": "lloydtheabstrac", "screen_name": "lloydtheabstrac", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-14"), "description": "published photographer • lloydtheabstrac1@icloud.com", "followers_count": 3910, "friends_count": 436, "statues_count": 23483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967249321986, "text": "@Austinnnn21 @Zacccchhhhh_244 I'm confused wtf", "in_reply_to_status": 691529190336626688, "in_reply_to_user": 2369973654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2369973654, 2503241293 }}, "user": { "id": 1663295839, "name": "kam", "screen_name": "kammeegonzalez", "lang": "en", "location": "null", "create_at": date("2013-08-11"), "description": "sixteen & stressed", "followers_count": 866, "friends_count": 871, "statues_count": 25771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967345803264, "text": "\"Truth is initially ridiculed, then attacked, then finally accepted as self-evident\"-- @ArtSchopenhauer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24970087 }}, "user": { "id": 254719698, "name": "Gavin Preston, M.D.", "screen_name": "GavinPrestonMD", "lang": "en", "location": "California", "create_at": date("2011-02-19"), "description": "Author of the upcoming book: The Managed Care Blues: My journey through a broken health care system and how we can fix it. (34 yr cancer survivor).", "followers_count": 38946, "friends_count": 42804, "statues_count": 25882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Niguel, CA", "id": "ef28da43cdf17b3f", "name": "Laguna Niguel", "place_type": "city", "bounding_box": rectangle("-117.740002,33.487223 -117.672617,33.571675") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639248, "cityName": "Laguna Niguel" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967345819648, "text": "Everday i think about this https://t.co/4F2dBkCqmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1857849042, "name": "Olivia Halbert", "screen_name": "HalbertOlivia", "lang": "en", "location": "canoeing with the state champ", "create_at": date("2013-09-12"), "description": "probably swimming or somethin", "followers_count": 224, "friends_count": 267, "statues_count": 4083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967350145024, "text": "Someone pop my back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1097049914, "name": "marisa", "screen_name": "baes1c", "lang": "en", "location": "Mount Holly, NC", "create_at": date("2013-01-16"), "description": "marisa. 19. probably sleeping. definitely not interested.", "followers_count": 927, "friends_count": 673, "statues_count": 7223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Holly, NC", "id": "3f39d4ede6a15905", "name": "Mount Holly", "place_type": "city", "bounding_box": rectangle("-81.090019,35.257517 -80.94545,35.361505") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37071, "countyName": "Gaston", "cityID": 3744960, "cityName": "Mount Holly" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967387738112, "text": "Yes pls https://t.co/fccFIIjfwV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 294685278, "name": "Jfizzle ✨", "screen_name": "jordan_renae_", "lang": "en", "location": "null", "create_at": date("2011-05-07"), "description": "I am gonna take it personal Bc I'm tired", "followers_count": 829, "friends_count": 928, "statues_count": 9368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967530344449, "text": "I wanna road trip right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1643915736, "name": "T", "screen_name": "Teresa24Aguilar", "lang": "en", "location": "Choctaw, OK", "create_at": date("2013-08-03"), "description": "#TheFAM FCF forever loving lord b & Bree ❣", "followers_count": 1643, "friends_count": 680, "statues_count": 60487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nicoma Park, OK", "id": "002ef63e02ed93f9", "name": "Nicoma Park", "place_type": "city", "bounding_box": rectangle("-97.353165,35.4786 -97.291617,35.536678") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4051850, "cityName": "Nicoma Park" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530967635234816, "text": "Josh Norman sounded like he was yakked out in his post game interview��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2756687791, "name": "C-breezy", "screen_name": "Stroodle_", "lang": "en", "location": "San Diego", "create_at": date("2014-08-22"), "description": "lookin 11 since '97", "followers_count": 361, "friends_count": 365, "statues_count": 459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530968016953344, "text": "@nusratchirps Ae Ishq Mere Ishq Ko Uin sare aam Badnam na Kar;Janmon baad nikla hun Ishq dhundhane teri galiyon mein https://t.co/kEHxNHQTi2", "in_reply_to_status": 691507643504955393, "in_reply_to_user": 3777978913, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user_mentions": {{ 1485512838 }}, "user": { "id": 3777978913, "name": "NusratSanjeev", "screen_name": "nusratsanjeev", "lang": "en", "location": "Sydney, New South Wales", "create_at": date("2015-10-03"), "description": "Love, Love and Love only for @nusratchirps", "followers_count": 994, "friends_count": 5001, "statues_count": 3028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530968620871680, "text": "I don't like talking about prom ���� like its late let me sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2553970272, "name": "crystall", "screen_name": "mami_crystall", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-06-07"), "description": "null", "followers_count": 259, "friends_count": 226, "statues_count": 6696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530968952254465, "text": "@champagnek1sses this nigga said \"rawr xD\" & sent it to bae ����", "in_reply_to_status": 691522739329302528, "in_reply_to_user": 3000658231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3000658231 }}, "user": { "id": 2785246681, "name": "im osum", "screen_name": "maagzz_", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "I just exist cuz I'm osum", "followers_count": 193, "friends_count": 186, "statues_count": 194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530968985911297, "text": "Where is Toms River on the map? Play the game at https://t.co/79p4EBc7bZ #Toms River", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1979,39.9537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Toms" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1018, "friends_count": 312, "statues_count": 2549216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toms River, NJ", "id": "259d4fab72f5d95c", "name": "Toms River", "place_type": "city", "bounding_box": rectangle("-74.269909,39.942803 -74.10616,40.059877") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3473110, "cityName": "Toms River" } }
+{ "create_at": datetime("2016-01-25T00:00:05.000Z"), "id": 691530969409589248, "text": "God knows I never meant to hurt you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384171893, "name": "ScootaFavSister❤️", "screen_name": "JealousOfMolly", "lang": "en", "location": "null", "create_at": date("2011-10-02"), "description": "Heaven couldnt jus wait for the Kings Rest Up #Scoota I am my brother keeper❤️ #Odee #Dart metlife 6217 miss you #keta #LLS", "followers_count": 7210, "friends_count": 4979, "statues_count": 87037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-26T00:00:00.000Z"), "id": 691893335703814144, "text": "Forever love this woman https://t.co/SgPTbIxCgt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356472612, "name": "///Mess", "screen_name": "AlejandroMess1", "lang": "en", "location": "Broward", "create_at": date("2011-08-16"), "description": "IG: 69_likess // Snapchat: messwitme", "followers_count": 266, "friends_count": 405, "statues_count": 5121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lauderdale Lakes, FL", "id": "6d846b156371809f", "name": "Lauderdale Lakes", "place_type": "city", "bounding_box": rectangle("-80.220059,26.150582 -80.18301,26.186183") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1239525, "cityName": "Lauderdale Lakes" } }
+{ "create_at": datetime("2016-01-26T00:00:00.000Z"), "id": 691893335817113600, "text": "The baby got him", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393816368, "name": "Darv.", "screen_name": "imJustDarv", "lang": "en", "location": "oh tuskegee. ", "create_at": date("2011-10-18"), "description": "ion really do shit, but when I do shit, it's always the shit.", "followers_count": 1384, "friends_count": 599, "statues_count": 109350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuskegee, AL", "id": "28df0fa3673b5b49", "name": "Tuskegee", "place_type": "city", "bounding_box": rectangle("-85.747927,32.386278 -85.663058,32.460448") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1087, "countyName": "Macon", "cityID": 177304, "cityName": "Tuskegee" } }
+{ "create_at": datetime("2016-01-26T00:00:00.000Z"), "id": 691893336039383041, "text": "I hate @NoahWingad to Pluto and back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2952860126 }}, "user": { "id": 2938617998, "name": "jessica leialoha", "screen_name": "jessical3ialoha", "lang": "en", "location": "¯\\_(ツ)_/¯ ", "create_at": date("2014-12-21"), "description": "but I just can't let you go", "followers_count": 1356, "friends_count": 652, "statues_count": 20512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wailuku, HI", "id": "1ad33d5e2c3e5d0b", "name": "Wailuku", "place_type": "city", "bounding_box": rectangle("-156.5369,20.860213 -156.480503,20.916384") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1577450, "cityName": "Wailuku" } }
+{ "create_at": datetime("2016-01-26T00:00:00.000Z"), "id": 691893336735621120, "text": "Wind 0 mph --. Barometer 30.13 in, Falling slowly. Temperature 52.0 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 13, "friends_count": 4, "statues_count": 20438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893337662623745, "text": "I'm off Friday sooooooo ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951387856, "name": "Paulina r.", "screen_name": "paulinaaxoxo___", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 923, "friends_count": 405, "statues_count": 14437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitney, NV", "id": "cb5ac954e9b77fba", "name": "Whitney", "place_type": "city", "bounding_box": rectangle("-115.064712,36.074534 -115.010215,36.137196") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3283800, "cityName": "Whitney" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338404974592, "text": "01/26@03:00 - Temp 22.3F, WC 22.3F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.180in, Falling. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 64, "friends_count": 8, "statues_count": 46541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338530803712, "text": "This breakfast was epic, a lil' too much for me, though. But that was an experience! Good… https://t.co/qL4FP352si", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.92256909,36.55685498"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35479334, "name": "Lucrezia Oddone", "screen_name": "LucreziaOd", "lang": "it", "location": "Roma ", "create_at": date("2009-04-26"), "description": "I make videos in Italian & English #LearnItalianwithLucrezia • Italian language, culture & lifestyle", "followers_count": 1677, "friends_count": 747, "statues_count": 8069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmel, CA", "id": "01e587be3cdfd4f3", "name": "Carmel", "place_type": "city", "bounding_box": rectangle("-121.932378,36.540748 -121.912765,36.564516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 611250, "cityName": "Carmel-by-the-Sea" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338581114880, "text": "The absolutely *UNIMAGINABLE just happened!\nFor 10 minutes I sincerely, drunkenly could *NOT remember my ex-bf's name -- *THE 1who got away!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437143250, "name": "Innov8r ♫ ♫", "screen_name": "Innov8rGuy", "lang": "en", "location": "USA", "create_at": date("2011-12-14"), "description": "Innovator guy -- one who makes things happen... with a touch of quirkiness to keep things interesting. \nAnthem song=♥Terry Dexter: I'm Free!♫ \r\nAVI=me♂", "followers_count": 4060, "friends_count": 4437, "statues_count": 17864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338715324417, "text": "Maybe I'm just gifted or something idk\nThat's just shit I never understood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538568253, "name": "AYMERICFUCKER69", "screen_name": "TheVantass", "lang": "en", "location": "Coerthas Central Highlands22,7", "create_at": date("2012-03-27"), "description": "Catherine • world's okayest clarinet player • she/her • slytherin • knight of blood • David Douxmont on Moogle • DUKE NUKEM •", "followers_count": 222, "friends_count": 248, "statues_count": 8206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, IL", "id": "1a3ba0458554bafe", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-88.214587,39.457102 -88.138535,39.524917") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17029, "countyName": "Coles", "cityID": 1712567, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338753081344, "text": "I just want my best friend back....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55039755, "name": "Dustin J. Givner", "screen_name": "dGiv15", "lang": "en", "location": "From VA 2 PA", "create_at": date("2009-07-08"), "description": "Entrepreneur, Account Exec, Nightclub Manager. My inner world creates my outer world.", "followers_count": 1440, "friends_count": 2027, "statues_count": 28191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893338979569670, "text": "I should be asleep������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219414862, "name": "T-Patt❤️", "screen_name": "tpattonn1", "lang": "en", "location": "864 ", "create_at": date("2010-11-24"), "description": "greer high '16 ✨ rockstar cheer smashing pumpkins⭐️ fly high nate ♥️", "followers_count": 1384, "friends_count": 2184, "statues_count": 13680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greer, SC", "id": "50d31ecdad5158be", "name": "Greer", "place_type": "city", "bounding_box": rectangle("-82.34636,34.84665 -82.132484,35.030719") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4530985, "cityName": "Greer" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893339130609664, "text": "@KriscoPacino happy birthday!!!", "in_reply_to_status": -1, "in_reply_to_user": 60855567, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60855567 }}, "user": { "id": 3054198217, "name": "Tune.☁️", "screen_name": "ImTune_YouJig", "lang": "en", "location": "Conway, AR", "create_at": date("2015-03-01"), "description": "Just stick around & see who's still around a decade from now.. Snapchat: Montorey_j #UCA", "followers_count": 1299, "friends_count": 959, "statues_count": 5663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893339453575168, "text": "Fun fact: stingrays can camouflage. Stingrays are badass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2335919648, "name": "bruhterfly", "screen_name": "sweaters4ever", "lang": "en", "location": "buried in cat hair ", "create_at": date("2014-02-09"), "description": "a special hood rat", "followers_count": 122, "friends_count": 98, "statues_count": 8937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893339537432581, "text": "someone talk to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2830926783, "name": "tw", "screen_name": "taylorwiliamss", "lang": "en", "location": "boolin w bffl katie", "create_at": date("2014-10-14"), "description": "rip devin❤️", "followers_count": 461, "friends_count": 390, "statues_count": 8840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scaggsville, MD", "id": "0031f2651fa4744b", "name": "Scaggsville", "place_type": "city", "bounding_box": rectangle("-76.9186,39.103129 -76.818386,39.168568") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2470525, "cityName": "Scaggsville" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893339537461248, "text": "me af ���� https://t.co/YbV0VRSaBc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2811725808, "name": "sar ♡", "screen_name": "_sarahallisonn", "lang": "en", "location": "west chester | cincy", "create_at": date("2014-09-15"), "description": "Made ya look.", "followers_count": 341, "friends_count": 519, "statues_count": 5450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beckett Ridge, OH", "id": "d212b9c3cdde763f", "name": "Beckett Ridge", "place_type": "city", "bounding_box": rectangle("-84.462009,39.325931 -84.410085,39.362659") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3904840, "cityName": "Beckett Ridge" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893340057501696, "text": "belated.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2245814292, "name": "Daniel", "screen_name": "daniel_delemos", "lang": "en", "location": "null", "create_at": date("2013-12-14"), "description": "ig: dee_lema\n\nfb: daniel.delemos.7@facebook.com", "followers_count": 75, "friends_count": 77, "statues_count": 1843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893340145631232, "text": "I dont think we need a president. Hes just the face of america when really congress does all the real work. Let me not start", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76181027, "name": "$☆°Că£!-Påpi°☆$", "screen_name": "CaliBoiFMB", "lang": "en", "location": "Positive, State of Mind", "create_at": date("2009-09-21"), "description": "All Lives Matter!\nIG: Californication88 #MKDZ #MisfitFMB califonialuv88@gmail.com SC:caliboifmb Twifey: @Pink88Panties", "followers_count": 2692, "friends_count": 2728, "statues_count": 143027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alpharetta, GA", "id": "01ac4c9b51fa2e35", "name": "Alpharetta", "place_type": "city", "bounding_box": rectangle("-84.359217,34.028083 -84.201172,34.118605") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1301696, "cityName": "Alpharetta" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893341039038464, "text": "GN❣ (@ Meridian Place in Northridge, CA) https://t.co/8fbGOirvRw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 372, "friends_count": 335, "statues_count": 25457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-26T00:00:01.000Z"), "id": 691893341236117504, "text": "NO lol https://t.co/RwkZw3Qdb6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340185682, "name": "SLAYomi", "screen_name": "_simbuh", "lang": "en", "location": "ny :: dirty jerz ", "create_at": date("2011-07-22"), "description": "big dreams, big ideas... bigger hair", "followers_count": 2745, "friends_count": 1639, "statues_count": 140635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roselle, NJ", "id": "888a254ab33a9e2e", "name": "Roselle", "place_type": "city", "bounding_box": rectangle("-74.285836,40.637241 -74.233501,40.663631") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3464620, "cityName": "Roselle" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893341701722112, "text": "The river was amazing today. @ South Fork Yuba River https://t.co/36gZ9kDDoo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.03448611,39.23665977"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42919124, "name": "Shawn Reeder", "screen_name": "shawnreeder", "lang": "en", "location": "Worldwide", "create_at": date("2009-05-27"), "description": "Visual Artist ~ Photographer ~ Timelapse Cinematographer ~ Filmmaker ~ Musician ~ Lover of Life ~ Pursuer of Oneness", "followers_count": 2863, "friends_count": 360, "statues_count": 7842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893341873651712, "text": "Wind 3.0 mph WNW. Barometer 30.047 in, Rising Rapidly. Temperature 36.0 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342213410816, "text": "Wind 4.0 mph SW. Barometer 29.933 in, Falling slowly. Temperature 48.5 °F. Rain today 0.01 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342267936768, "text": "@sardesairajdeep @CaptShantaram @shalini0313 don't you deserve it? Be fair in your journalism and we all will be fair to you.", "in_reply_to_status": 691890699747381249, "in_reply_to_user": 56304605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56304605, 4045639006, 590539756 }}, "user": { "id": 1135835516, "name": "R. Rohit Singh", "screen_name": "Birrajput", "lang": "en", "location": "CALIFORNIA", "create_at": date("2013-01-30"), "description": "null", "followers_count": 126, "friends_count": 223, "statues_count": 220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342582480898, "text": "@ebbtideapp Tide in Cornwells Heights, Pennsylvania 01/26/2016\nHigh 3:36am 6.3\n Low 10:30am -0.6\nHigh 3:54pm 6.9\n Low 11:04pm -0.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.9383,40.0683"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 14, "friends_count": 1, "statues_count": 2306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bensalem, PA", "id": "00ca00906ca4c6d4", "name": "Bensalem", "place_type": "city", "bounding_box": rectangle("-74.994551,40.048711 -74.898263,40.155775") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342695747584, "text": "#nowplaying Around the Horn - Super Bowl 50 Is Set: 1/25/16 by @AroundTheHorn https://t.co/iPmPkxHS5C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.810334,37.235488"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user_mentions": {{ 2458560776 }}, "user": { "id": 30253253, "name": "Fred", "screen_name": "M3g4d37h", "lang": "en", "location": "Silicon Valley", "create_at": date("2009-04-10"), "description": "Father. Husband. Advocate for individuals with developmental disabilities. Audiophile. Combat Sports Fan. Liberal. An 8-bit man in a 64-bit world.", "followers_count": 182, "friends_count": 674, "statues_count": 18223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slightly Cool Poultry & Produce, LLC", "id": "07d9efb85c882000", "name": "Slightly Cool Poultry & Produce, LLC", "place_type": "poi", "bounding_box": rectangle("-121.81033409999999,37.235487899999995 -121.810334,37.235488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342737686531, "text": "Thank you for my chance to shine!@CarrieBookFairy @SagesBlogTours #blogging #books #ReadYourWorld https://t.co/SDuCjE7ZLJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blogging", "books", "ReadYourWorld" }}, "user_mentions": {{ 2440183392 }}, "user": { "id": 3184536384, "name": "Monica Patton", "screen_name": "neogeishaforeve", "lang": "en", "location": "null", "create_at": date("2015-05-03"), "description": "I'm a Unicorn. What you think happened didn't because I'm not real. But if you believe in fairytales, then believe I'm living, and leaving 'Beautiful Carnage'.", "followers_count": 129, "friends_count": 464, "statues_count": 170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deerfield Beach, FL", "id": "4ebdbc556ccd2f12", "name": "Deerfield Beach", "place_type": "city", "bounding_box": rectangle("-80.170343,26.274467 -80.074368,26.327929") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1216725, "cityName": "Deerfield Beach" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893342746079232, "text": "@haro_mar �� done w this tweet ��", "in_reply_to_status": 691893232163278848, "in_reply_to_user": 572239540, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 572239540 }}, "user": { "id": 626928959, "name": "Brenda♡", "screen_name": "brendavaldivia_", "lang": "en", "location": "null", "create_at": date("2012-07-04"), "description": "sc: brendavaldiviaa", "followers_count": 468, "friends_count": 191, "statues_count": 21406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893343006105600, "text": "Temp: 54.6°F Wind:0.0mph Pressure: 30.098hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893344423788545, "text": "Wind 2.6 mph SE. Barometer 29.995 in, Falling Rapidly. Temperature 32.8 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893344511901698, "text": "60.5F (Feels: 60.5F) - Humidity: 99% - Wind: 7.6mph SW - Gust: 9.8mph - Pressure: 1028.6mb - Rain: 0.02\" #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 224075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893344696430593, "text": "Tipik erkekmiş yanında olan şahısa sor onu aq alime laf yok .-.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 4842327838, "name": "Mavinin Fancığı", "screen_name": "Mavi_delireni", "lang": "tr", "location": "Uzay", "create_at": date("2016-01-24"), "description": "Alsel FOREVER", "followers_count": 16, "friends_count": 246, "statues_count": 169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saratoga, CA", "id": "b46f044cb28493c6", "name": "Saratoga", "place_type": "city", "bounding_box": rectangle("-122.068181,37.237066 -121.989501,37.296386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 670280, "cityName": "Saratoga" } }
+{ "create_at": datetime("2016-01-26T00:00:02.000Z"), "id": 691893345094909953, "text": "Aaliyah-At your best @ShyraJahara", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.92789322,38.88452762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 810853963 }}, "user": { "id": 4405101921, "name": "⛽️", "screen_name": "MrGasMann", "lang": "en", "location": "null", "create_at": date("2015-12-07"), "description": "null", "followers_count": 29, "friends_count": 35, "statues_count": 292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 11, "stateName": "District of Columbia", "countyID": 11001, "countyName": "District of Columbia", "cityID": 1150000, "cityName": "Washington" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893345627541504, "text": "My aunt Told Me To Leave My Bullshit In 2015 But She Wearing The Same Wig From 1988 , Smh .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285870423, "name": "darkgable lord", "screen_name": "POLOLORD5", "lang": "en", "location": "chicago ", "create_at": date("2011-04-21"), "description": "lay back ... naw not really .... funny love talking shit .....WHAT MORE CAN I SAY ? #teamiwilldoyourgirl", "followers_count": 299, "friends_count": 857, "statues_count": 6111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillside, IL", "id": "88bddb0e2768943d", "name": "Hillside", "place_type": "city", "bounding_box": rectangle("-87.919829,41.844383 -87.882342,41.885453") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1735086, "cityName": "Hillside" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346097303553, "text": "#SupportOriginMelissa 45.1°F Wind:2.5mph Pressure: 29.91hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers likely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346336403456, "text": "https://t.co/W8PGiRyWlk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2457731790, "name": "NeptunoⓂ️", "screen_name": "neptuno30", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-04-21"), "description": "One God Only", "followers_count": 5131, "friends_count": 5644, "statues_count": 117186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheridan, CO", "id": "3cfce73ef3a505af", "name": "Sheridan", "place_type": "city", "bounding_box": rectangle("-105.034755,39.631416 -104.999537,39.660504") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 869645, "cityName": "Sheridan" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346588033024, "text": "@Jimrumshot >.< No.....please don't!!! Those can hurt....and mess the tabs up!!", "in_reply_to_status": 691893054731620352, "in_reply_to_user": 1563387642, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1563387642 }}, "user": { "id": 1189224176, "name": "Sef the Dragonknight", "screen_name": "Sef_highwind", "lang": "en", "location": "null", "create_at": date("2013-02-17"), "description": "Just a fun loving fox who enjoys everything. A bit of a cub, bit crazy, and a lot of fun, I hope. Taken by @MilesAMatthias", "followers_count": 537, "friends_count": 497, "statues_count": 7353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346613268480, "text": "Wind 2.2 mph SSE. Barometer 30.06 in, Falling. Temperature 29.3 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 25, "friends_count": 102, "statues_count": 157016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346730643456, "text": "Would you let me be your get away", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983544132, "name": "MILCA✨", "screen_name": "Notorius_thug", "lang": "en", "location": "PNW", "create_at": date("2012-12-01"), "description": "SC milky_way23 • I ♥️ ( future bae ) • 19 ♋️ the song La Chona is about me • very sarcastic", "followers_count": 478, "friends_count": 373, "statues_count": 38664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893346814595072, "text": "@ezrazuidema is my aesthetic", "in_reply_to_status": -1, "in_reply_to_user": 4834248237, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4834248237 }}, "user": { "id": 2193630475, "name": "スティーブン・マーシュ", "screen_name": "stevenmaarsh", "lang": "en", "location": "null", "create_at": date("2013-11-13"), "description": "stagnant and stupid", "followers_count": 268, "friends_count": 179, "statues_count": 3790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893347598913537, "text": "I love my mom and by my mom I mean kaely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708133868, "name": "Brandini", "screen_name": "BrandoTheGod", "lang": "en", "location": "Killadeas, Northern Ireland", "create_at": date("2013-08-28"), "description": "Same", "followers_count": 303, "friends_count": 307, "statues_count": 7459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893347691188224, "text": "w o r r i e d ���� I really hope my nigga okay. It's not like him to not even text my phone. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2368153948, "name": "TheReal BP⛽️", "screen_name": "brxvh_", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "R.I.P Daddy ❤️ I love you.", "followers_count": 967, "friends_count": 857, "statues_count": 21183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893347695382528, "text": "Wind 0.0 mph ---. Barometer 30.018 in, Falling. Temperature 53.5 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893347762491393, "text": "@celebhelpers thank you! I like to stay positive", "in_reply_to_status": 691893288618627074, "in_reply_to_user": 2514385964, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2514385964 }}, "user": { "id": 827635021, "name": "14 Days!!!", "screen_name": "Ciindiie_", "lang": "en", "location": "Beverly Hood, Calumfornia", "create_at": date("2012-09-16"), "description": "Concerts✨♡ Instagram:TYPICALCIINDIIE \nProfessional Fangirl ✌\nStan Culture Influencer.", "followers_count": 732, "friends_count": 532, "statues_count": 71975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893347812818944, "text": "we haven't won a district game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 480095796, "name": "Ant$zn", "screen_name": "SCREAMIN__Ant", "lang": "en", "location": "Kansas City ✈ Texas", "create_at": date("2012-01-31"), "description": "rip dad", "followers_count": 1467, "friends_count": 927, "statues_count": 41819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893348253196289, "text": "I Ain't Got Time For Bitches �� Gotta Keep My Mind On My Mfn Riches ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195236910, "name": "peight.", "screen_name": "Sterlinnn_", "lang": "en", "location": "#Gang #Gang", "create_at": date("2010-09-25"), "description": "saying goodbye doesn't mean goodbye forever.", "followers_count": 1158, "friends_count": 548, "statues_count": 87410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893348316114944, "text": "Haven't been this happy in awhile ��☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145715851, "name": "Isaiah Martinez", "screen_name": "isaiahminajj", "lang": "en", "location": "San Marcos, CA", "create_at": date("2015-04-08"), "description": "18 year old lost and confused", "followers_count": 119, "friends_count": 100, "statues_count": 2901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, CA", "id": "a2c84129f9dcf69f", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-117.230172,33.090761 -117.103461,33.186722") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 668196, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893348441923584, "text": "\"maybe I'll forget you some other time\"...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21829566, "name": "milky$hake", "screen_name": "KHLOE_G", "lang": "en", "location": "New York, USA", "create_at": date("2009-02-24"), "description": "Popeyes enthusiast + aspiring vibes selector", "followers_count": 1001, "friends_count": 591, "statues_count": 43609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893348693590016, "text": "������������������������ https://t.co/uiKZjwlrAw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2783150319, "name": "Whitney", "screen_name": "Ohheywhitney", "lang": "en", "location": "null", "create_at": date("2014-09-24"), "description": "null", "followers_count": 19, "friends_count": 55, "statues_count": 172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, OH", "id": "4959dac55d8204fc", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-83.184218,40.508388 -83.071521,40.659067") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39101, "countyName": "Marion", "cityID": 3947754, "cityName": "Marion" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349134024704, "text": "Both born under conflicting elements", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373797795, "name": "CEE❗️", "screen_name": "_cwould", "lang": "en", "location": "Oakland", "create_at": date("2011-09-14"), "description": "IG: _cwould SC: ceagles Rip Lay ❤️", "followers_count": 1311, "friends_count": 914, "statues_count": 51355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349243097088, "text": "@TheAlmighty_D I will try my best just for u", "in_reply_to_status": 691892873529331713, "in_reply_to_user": 555752199, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 555752199 }}, "user": { "id": 1206393271, "name": "court", "screen_name": "cnm816", "lang": "en", "location": "null", "create_at": date("2013-02-21"), "description": "the best is yet to come", "followers_count": 701, "friends_count": 389, "statues_count": 11658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colonial Heights, VA", "id": "f8e149f66945a5a9", "name": "Colonial Heights", "place_type": "city", "bounding_box": rectangle("-77.431934,37.233115 -77.362385,37.29787") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51570, "countyName": "Colonial Heights", "cityID": 5118448, "cityName": "Colonial Heights" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349264003073, "text": "Smh gotta lie on me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2543313459, "name": "SonOfObama", "screen_name": "Blacc_Boiii", "lang": "en", "location": "null", "create_at": date("2014-05-12"), "description": "First you make her laugh, Then you make her wet", "followers_count": 369, "friends_count": 355, "statues_count": 3175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chattanooga, TN", "id": "3b3916ee31cfc9e6", "name": "Chattanooga", "place_type": "city", "bounding_box": rectangle("-85.404424,34.983674 -85.080704,35.200235") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349352128513, "text": "Tomorrow I will put my short story into a screenplay format... no more #procrastination", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "procrastination" }}, "user": { "id": 173166588, "name": "Tonia Lee", "screen_name": "ToniaLeighLee", "lang": "en", "location": "Pearl, MS", "create_at": date("2010-07-31"), "description": "@msstate graduate, wife of @BobLeeII, mother, grandmother, and aspiring writer. Film student, gonna make movies.", "followers_count": 181, "friends_count": 444, "statues_count": 3410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl, MS", "id": "04b55f8bc56b0df2", "name": "Pearl", "place_type": "city", "bounding_box": rectangle("-90.184695,32.23749 -90.037767,32.324508") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28121, "countyName": "Rankin", "cityID": 2855760, "cityName": "Pearl" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349431799808, "text": "Just remembered my 8 am got cancelled tomorrow! That's clutch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 577565947, "name": "Shaquille Francis", "screen_name": "Shaq_got_racks", "lang": "en", "location": "null", "create_at": date("2012-05-11"), "description": "Philipians 4:13 Whitewater basketball. Kishwaukee alum", "followers_count": 655, "friends_count": 344, "statues_count": 12316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitewater, WI", "id": "d01b36ae70fc0f58", "name": "Whitewater", "place_type": "city", "bounding_box": rectangle("-88.777094,42.816297 -88.698234,42.85727") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5586925, "cityName": "Whitewater" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349490544640, "text": "I'm crying i am so proud of me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542892925, "name": "HOBI", "screen_name": "hobiharder", "lang": "en", "location": "null", "create_at": date("2012-04-01"), "description": "Jesus is the coolest", "followers_count": 465, "friends_count": 544, "statues_count": 7277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-01-26T00:00:03.000Z"), "id": 691893349540831233, "text": "Fifty shades of grey wowwwww", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240852779, "name": "sydney anne", "screen_name": "sydmaccc", "lang": "en", "location": "Delaware, USA", "create_at": date("2011-01-20"), "description": "brady thomas kalafut ∞", "followers_count": 3936, "friends_count": 718, "statues_count": 2578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilmington Manor, DE", "id": "b00e7cd4d6612be3", "name": "Wilmington Manor", "place_type": "city", "bounding_box": rectangle("-75.598648,39.66471 -75.574818,39.698998") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1077840, "cityName": "Wilmington Manor" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893350291652610, "text": "GREAT NEWS! @CBSNewYork w/ @MaryCalviTV @AlexDenisTV & @JohnElliottTV ON at special time 4AM! ... BAD NEWS... all the stuff we talk about.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16299627, 235161193, 249790861, 1969455858 }}, "user": { "id": 90746809, "name": "Chris Wragge", "screen_name": "ChrisWragge", "lang": "en", "location": "New York City", "create_at": date("2009-11-17"), "description": "Chris Wragge's Official Twitter Page, CBS News Anchor, NYC, former anchor of The Early Show, fmr NBC Sports reporter, Correspondent for ET, always a QB! #GMEN", "followers_count": 17640, "friends_count": 3634, "statues_count": 34846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893350664925184, "text": "karma is a such a bitch when she finally gets you after you fucked over someone really bad ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2833722781, "name": "adamari$", "screen_name": "adamarisr25", "lang": "en", "location": "Mount Vernon, WA", "create_at": date("2014-09-26"), "description": "la vida es buena, la mala soy yo", "followers_count": 251, "friends_count": 252, "statues_count": 3625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, WA", "id": "c70b623cdcd61952", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-122.373199,48.382535 -122.250443,48.453709") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit", "cityID": 5347560, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893350719422464, "text": "2016-01-26T08:00:01.000Z temp 73.2783813477 pressure 1006.69 humidity 11.4868164062", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.71839833,39.78345167"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 4778397673, "name": "Phil Moyer", "screen_name": "LaikaRover", "lang": "en", "location": "Hockessin, DE", "create_at": date("2016-01-18"), "description": "This is Laika's Twitter account - my autonomous rover project. It occasionally tweets environmental data from its sensors.", "followers_count": 8, "friends_count": 11, "statues_count": 172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hockessin, DE", "id": "073dd6cd3c1af5f4", "name": "Hockessin", "place_type": "city", "bounding_box": rectangle("-75.724892,39.761505 -75.634189,39.813136") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1035850, "cityName": "Hockessin" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893350908170240, "text": "#TMT #PartyGod is in the building! It's a who's who here tonight \n#iToldYou #Marquee #LasVegas… https://t.co/M571ztitGS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17431212,36.10941776"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TMT", "PartyGod", "iToldYou", "Marquee", "LasVegas" }}, "user": { "id": 295897277, "name": "Rick Sumi atVegas", "screen_name": "RickSumi", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-05-09"), "description": "born/raised: Kaneohe, HI | school: Phoenix, AZ | work: Seattle, WA | current: Vegas nightlife VIP Host (206)793-5154", "followers_count": 355, "friends_count": 446, "statues_count": 3515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893350945951744, "text": "Time for a real one��✌️ https://t.co/AnIWuBpPrP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64963581, "name": "T", "screen_name": "teranicole962", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "null", "followers_count": 164, "friends_count": 130, "statues_count": 1908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893351411552257, "text": "@earlxsweat this how you rocking?", "in_reply_to_status": 691892526630965248, "in_reply_to_user": 486955518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 486955518 }}, "user": { "id": 150901351, "name": "BRISS DONT MISS", "screen_name": "HueyBriss", "lang": "en", "location": "RULES FOLLOW US ", "create_at": date("2010-06-01"), "description": "null", "followers_count": 1463, "friends_count": 399, "statues_count": 26929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893351629623297, "text": "https://t.co/2Q09K2pNzV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 19909, "friends_count": 15286, "statues_count": 6450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893352267120640, "text": "Should I ????", "in_reply_to_status": 691893301612535808, "in_reply_to_user": 1584805532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1584805532, "name": "dom", "screen_name": "dominiquemead33", "lang": "en", "location": "nomad", "create_at": date("2013-07-10"), "description": "anti public", "followers_count": 794, "friends_count": 338, "statues_count": 43657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893352665604097, "text": "That's how it seems.. �� https://t.co/mHJcL1uV9W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47020857, "name": "♛ᴅiᴋᴇɴʙʀᴇᴇ♛", "screen_name": "photogenic_kenn", "lang": "en", "location": "Atlanta Georgia ", "create_at": date("2009-06-13"), "description": "ᴛʜɪs ᴀɪɴᴛ ᴘʜᴀᴇᴅʀᴀ ʙɪᴛᴄʜ..", "followers_count": 2493, "friends_count": 1436, "statues_count": 77357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lithia Springs, GA", "id": "00c4104a33f2c868", "name": "Lithia Springs", "place_type": "city", "bounding_box": rectangle("-84.696305,33.753937 -84.593547,33.805711") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13097, "countyName": "Douglas", "cityID": 1346832, "cityName": "Lithia Springs" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893352866930689, "text": "Wind 3.5 mph SSE. Barometer 30.058 in, Falling Rapidly. Temperature 30.1 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 6061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893353508704256, "text": "@DamnTeeTee Ryan", "in_reply_to_status": 691893301042094080, "in_reply_to_user": 1454656152, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 1454656152 }}, "user": { "id": 341394622, "name": "WillyBandz", "screen_name": "DamnnWill", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-07-24"), "description": "Dont hate the sauce cause you got lost #RestInParadiseJuneMarie #Mosthated™ |", "followers_count": 1793, "friends_count": 1529, "statues_count": 63591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-26T00:00:04.000Z"), "id": 691893353852592128, "text": "@BROGODSHAWTY it's so good!", "in_reply_to_status": 691887611850420225, "in_reply_to_user": 3089673375, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3089673375 }}, "user": { "id": 30332062, "name": "Johnny Boy ☮", "screen_name": "JohnTheFame", "lang": "en", "location": "NYC ", "create_at": date("2009-04-10"), "description": "I just be tweet'n #BlackLivesMatter", "followers_count": 15144, "friends_count": 12773, "statues_count": 124030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-26T00:00:05.000Z"), "id": 691893356792827904, "text": "@PeIicans @thegarbear28", "in_reply_to_status": 691854596285792256, "in_reply_to_user": 1877754092, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1877754092, 2557406864 }}, "user": { "id": 1342939939, "name": "melissa", "screen_name": "melissafleurr", "lang": "en", "location": "null", "create_at": date("2013-04-10"), "description": "sweet simplicity", "followers_count": 124, "friends_count": 178, "statues_count": 8046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Diamond Bar, CA", "id": "771f8196d3598fe8", "name": "Diamond Bar", "place_type": "city", "bounding_box": rectangle("-117.86323,33.959529 -117.767626,34.042345") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619192, "cityName": "Diamond Bar" } }
+{ "create_at": datetime("2016-01-26T00:00:05.000Z"), "id": 691893357359009793, "text": "Uzzle a opp ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2990735914, "name": "princess jazzy ✨", "screen_name": "wavey_jvz", "lang": "en", "location": "null", "create_at": date("2015-01-21"), "description": "null", "followers_count": 386, "friends_count": 356, "statues_count": 3569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-01-26T00:00:05.000Z"), "id": 691893357560332289, "text": "@ffacundo1018 that's my grandma bitch������", "in_reply_to_status": 691893108146069507, "in_reply_to_user": 2416415995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2416415995 }}, "user": { "id": 253729389, "name": "Douglas10", "screen_name": "DouglasElBoludo", "lang": "es", "location": "Azusa, CA", "create_at": date("2011-02-17"), "description": "that asian argentino", "followers_count": 656, "friends_count": 574, "statues_count": 8050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893358566973440, "text": "I really need to get to bed if I want to TRY and workout in the morning������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 765996770, "name": "Lex", "screen_name": "AlexaDordoni", "lang": "en", "location": "California", "create_at": date("2012-08-18"), "description": "20 | love yourself", "followers_count": 250, "friends_count": 399, "statues_count": 12553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport Beach, CA", "id": "0ce4c27ce26030aa", "name": "Newport Beach", "place_type": "city", "bounding_box": rectangle("-117.958219,33.563618 -117.789845,33.671658") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 651182, "cityName": "Newport Beach" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893359552626688, "text": "@Abigail12210 https://t.co/H8k2TgNGVh", "in_reply_to_status": 691891711853924352, "in_reply_to_user": 2286320880, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2286320880 }}, "user": { "id": 863761104, "name": "dom ◡̈", "screen_name": "dommmer", "lang": "en", "location": "staying faithful", "create_at": date("2012-10-05"), "description": "cuff me b4 its 2 l8 IG: domdiggittyy", "followers_count": 1080, "friends_count": 998, "statues_count": 3164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893359636566016, "text": "03:00 39.2°F Feels:31.4°F (Hi51.1°F/Lo39.2°F) Hum:73% Wnd:W 13.2MPH Baro:29.88in. Prcp:0.01in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 174, "friends_count": 266, "statues_count": 25096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893359699496960, "text": "@jaredmichaell__ happy birthday lil nigga. You a real nigga. Love you bro https://t.co/SiqbXpMBLJ", "in_reply_to_status": -1, "in_reply_to_user": 4552423634, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4552423634 }}, "user": { "id": 2302546657, "name": "FLIZZYVILLE Jan.27th", "screen_name": "EunJeFlizzy", "lang": "en", "location": "St. Tropez", "create_at": date("2014-01-20"), "description": "He's on his way, about to get paid, he's on his way to hollywood ////HOLLYWOOD COLE. HOLLYWOOD FLIZZY phhhoto: FLIZZYVILLE. FUCK YOU, I'M MAD", "followers_count": 674, "friends_count": 691, "statues_count": 34136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cypress, CA", "id": "a077793faceeda6f", "name": "Cypress", "place_type": "city", "bounding_box": rectangle("-118.063298,33.791963 -118.010668,33.845856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 617750, "cityName": "Cypress" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893359800160256, "text": "@Upping Believe me. I've been there. But the moment we give up on others is when we give up on ourselves. There are exceptions, I know.", "in_reply_to_status": 691893095638634496, "in_reply_to_user": 1256764634, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1256764634 }}, "user": { "id": 348135522, "name": "EZG YuGi-Joe", "screen_name": "ParanoiaPlays", "lang": "en", "location": "On the pursuit of happiness ", "create_at": date("2011-08-03"), "description": "Changing paths • I play games • Content Creator and player for @EZG_eSports • Partnered with @Twitch • I dream of bettering the world.", "followers_count": 2283, "friends_count": 1509, "statues_count": 51101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893360223715328, "text": "Wind 4.0 mph SSW. Barometer 29.780 in, Falling. Temperature 45.0 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 375, "friends_count": 288, "statues_count": 7776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893360580239361, "text": "@j15royals I laughed harder at this than I should have.", "in_reply_to_status": 691892914365059073, "in_reply_to_user": 50584222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50584222 }}, "user": { "id": 56353438, "name": "Draft LOttery", "screen_name": "patronusisotter", "lang": "en", "location": "Oregon", "create_at": date("2009-07-13"), "description": "#ripcity / #goducks / #rctid* / the empire / #bangarang. avi's by @SwooshMcDuck", "followers_count": 551, "friends_count": 919, "statues_count": 75006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893360588636160, "text": "Wind 1.0 mph ENE. Barometer 30.289 in, Steady. Temperature 40.6 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 197, "friends_count": 58, "statues_count": 245926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361205198848, "text": "@sabrinanorwood1 wouldn't have ever had a problem with her if she would have minded her own business and kept her mouth shut", "in_reply_to_status": 691891444785758208, "in_reply_to_user": 384689835, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384689835 }}, "user": { "id": 2419710450, "name": "brat", "screen_name": "PrincesssTeee__", "lang": "en", "location": "EV", "create_at": date("2014-03-30"), "description": "sittin' pretty on top", "followers_count": 907, "friends_count": 713, "statues_count": 25431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968758") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361230417920, "text": "me:moves to the east coast rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3105143149, "name": "OgGrandGabbiePurp☻", "screen_name": "OgGabbie_", "lang": "en", "location": "moving with soul, CA.", "create_at": date("2015-03-23"), "description": "coo,& collected http://soundcloud.com/daewop", "followers_count": 585, "friends_count": 423, "statues_count": 12199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barstow, CA", "id": "30c892dedb1a2c67", "name": "Barstow", "place_type": "city", "bounding_box": rectangle("-117.092355,34.862137 -116.963954,34.926831") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 604030, "cityName": "Barstow" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361393954816, "text": "https://t.co/r7AjRwrpRr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 368153295, "name": "PuckerMe ", "screen_name": "_PrettyMoney23", "lang": "en", "location": "Dekalb,IL", "create_at": date("2011-09-04"), "description": "Confident woman who has value ; takes life one day at a time; & one step at a time..my steps are ordered ", "followers_count": 438, "friends_count": 541, "statues_count": 27813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DeKalb, IL", "id": "76cec30d4c553725", "name": "DeKalb", "place_type": "city", "bounding_box": rectangle("-88.792846,41.883102 -88.714573,41.972512") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1719161, "cityName": "DeKalb" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361402318848, "text": "I'm fucking pathetic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76082434, "name": "El niño", "screen_name": "ShredderJorge", "lang": "en", "location": "New Vegas ", "create_at": date("2009-09-21"), "description": "Eating tacos somewhere with @maddiemfkay", "followers_count": 534, "friends_count": 464, "statues_count": 29640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361888907264, "text": "2AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 101884323, "name": "Kerston✨", "screen_name": "2ofTWO_", "lang": "en", "location": "225", "create_at": date("2010-01-04"), "description": "19 | XXVII | Orchid's Keeper✨", "followers_count": 2220, "friends_count": 1044, "statues_count": 84013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reserve, LA", "id": "2ad832dde65554d9", "name": "Reserve", "place_type": "city", "bounding_box": rectangle("-90.591825,30.049495 -90.510626,30.100941") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2264310, "cityName": "Reserve" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893361905631232, "text": "1/26/2016 - 02:00\nTemp: 39.5F \nHum: 83%\nWind: 0.0 mph\nBaro: 30.119in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 52, "statues_count": 49379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893362182479872, "text": "Season 6 episode 16 of house was so good #cuttyisthebest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cuttyisthebest" }}, "user": { "id": 2413086626, "name": "Tory Martinez", "screen_name": "tatatory", "lang": "en", "location": "null", "create_at": date("2014-03-26"), "description": "if you're a bird…I'm a bird", "followers_count": 40, "friends_count": 86, "statues_count": 1501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893362346086401, "text": "@squadyoyo follow me", "in_reply_to_status": -1, "in_reply_to_user": 425503441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 425503441 }}, "user": { "id": 4097145973, "name": "Wardawg", "screen_name": "AvonNate", "lang": "en", "location": "Bellingham, WA", "create_at": date("2015-11-01"), "description": "CONGRATULATIONS YOU FOUND WARDAWG, I AM A FUCKING LEGEND I AM KMS IMPROVED LIZARDSQUAD RALLY NOW IN MY DM CHAT WE PARTY\nSKYPE NATE.CRANE4 \nENABLE NOTIFICATIONS", "followers_count": 3994, "friends_count": 3674, "statues_count": 2860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-01-26T00:00:06.000Z"), "id": 691893362354458624, "text": "A lot of shit happend today I gotta be a 2 job Shawtyyy ����gotta be that back bone ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246459981, "name": "Queen Charlie Da 1st", "screen_name": "Charlie_B_Bitch", "lang": "en", "location": "TGOD", "create_at": date("2011-02-02"), "description": "Asiya Lynn ❤️▪️❤️Aja Naomi .", "followers_count": 1438, "friends_count": 1980, "statues_count": 8408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893362471899136, "text": "@LilyPichu it's your music you know? And plus you aren't a rude person so im sure if you ask they'll take it down", "in_reply_to_status": 691806227295240193, "in_reply_to_user": 74274804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74274804 }}, "user": { "id": 41446489, "name": "Big Slammu", "screen_name": "aguamentius", "lang": "en", "location": "Lake Hylia, Hyrule", "create_at": date("2009-05-20"), "description": "Dragons, whalewolves and sharks! Oh my! B list films, games, and Tommy Wiseau❤️ these are a few of my favorite things. pizza cats are best. 1/3 of @cc_cast3", "followers_count": 887, "friends_count": 526, "statues_count": 26147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Glen West, NJ", "id": "b732df1f3048bd84", "name": "Cedar Glen West", "place_type": "city", "bounding_box": rectangle("-74.315842,40.021265 -74.251656,40.057533") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3411155, "cityName": "Cedar Glen West" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893362526457856, "text": "���� go to sleep https://t.co/dJzWM6Xdv1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248481909, "name": "❣Qua❣", "screen_name": "Badgalqua", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-02-06"), "description": "girls,girls,girls,girls....Girl I do adore ‼️", "followers_count": 2693, "friends_count": 830, "statues_count": 205096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893363033968641, "text": "Temp: 31.3°F - Dew Point: 28.5° - Wind: 21.4 mph - Gust: 30.4 - Rain Today: 0.00in. - Pressure: 29.66in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 11, "statues_count": 12552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893364250275840, "text": "IYS MY BEST FRIENDS BDAY TODAY AYYYY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328178818, "name": "v", "screen_name": "vanessavilledo", "lang": "en", "location": "bay area", "create_at": date("2011-07-02"), "description": "ybhs '18*:・✧", "followers_count": 111, "friends_count": 60, "statues_count": 6401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893364942331904, "text": "��✌✊�� Amen sister https://t.co/9w9qM8AHGe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3020828386, "name": "Beanie MDM", "screen_name": "BeanieMDM", "lang": "en", "location": "Dayton, OH", "create_at": date("2015-02-05"), "description": "#MDM #DBS #BeanieLove #MarsIsMyReligion #EchelonIsMyFaith #ManiacMessiahIsMyPreacher", "followers_count": 1588, "friends_count": 1517, "statues_count": 54437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, OH", "id": "8d742fb555fbff21", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-84.307688,39.695193 -84.093044,39.865523") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3921000, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893365118484480, "text": "temperature down 25°F -> 23°F\nhumidity up 77% -> 85%\nwind 5mph -> 0mph\npressure 30.23in falling\nvisibility 10mi -> 8mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.41199,41.82387"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111185236, "name": "Providence Weather", "screen_name": "_ProvidenceRI", "lang": "en", "location": "Providence, RI", "create_at": date("2010-02-03"), "description": "Weather updates, forecast, warnings and information for Providence, RI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 302, "friends_count": 251, "statues_count": 27438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893365189779456, "text": "Ohh my the cutest �� Steph made a beeeeautiful baby https://t.co/4w9yhxgG2Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1668785042, "name": "Kalina", "screen_name": "fumerdelherbe", "lang": "en", "location": "California, USA", "create_at": date("2013-08-13"), "description": "stay pretty, hustling & educated.", "followers_count": 217, "friends_count": 237, "statues_count": 5074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893365382709248, "text": "Don't let one mistake define you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513889813, "name": "н α у❂", "screen_name": "HaleyyNicole14", "lang": "en", "location": "Tennesee☀️", "create_at": date("2012-03-03"), "description": "A true diamond with a heart of gold. ♚", "followers_count": 361, "friends_count": 462, "statues_count": 8130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Pine, TN", "id": "01bb265d2155c80d", "name": "White Pine", "place_type": "city", "bounding_box": rectangle("-83.315754,36.088458 -83.254471,36.136503") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47089, "countyName": "Jefferson", "cityID": 4780360, "cityName": "White Pine" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893365454028801, "text": "Plans on dropping a 7 track tape at midnight called The beginning of Rocky D Boa @Myles_Fargo3 you got the equipment ready?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177454199 }}, "user": { "id": 820828591, "name": "Rocky D Boa", "screen_name": "dontelmartinez1", "lang": "en", "location": "The Jungle", "create_at": date("2012-09-12"), "description": "Ima Rapper Have Dreams! Jungle Boyz Ent! I Gotta Stay Healthy On A Punk Bitch|jungle boyzENT!!! For Bookings and Features Contact @mylesj151@gmail.com", "followers_count": 1082, "friends_count": 738, "statues_count": 15945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parsons, KS", "id": "0a3e119020705b64", "name": "Parsons", "place_type": "city", "bounding_box": rectangle("-95.297025,37.321136 -95.2296,37.364043") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20099, "countyName": "Labette", "cityID": 2054675, "cityName": "Parsons" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893365856665600, "text": "Have to restart :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19311173, "name": "Bleak", "screen_name": "BleakVillain", "lang": "en", "location": "Across The Galaxy", "create_at": date("2009-01-21"), "description": "The world's leading authority on waiting too long. \n SF↔LA", "followers_count": 1142, "friends_count": 2080, "statues_count": 48600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-01-26T00:00:07.000Z"), "id": 691893366045433856, "text": "#sure https://t.co/5RzxDqpF4Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "sure" }}, "user": { "id": 2736301010, "name": "Drew", "screen_name": "MosCurved", "lang": "en", "location": "Austin, USA", "create_at": date("2014-08-15"), "description": "null", "followers_count": 769, "friends_count": 564, "statues_count": 36210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-26T00:00:08.000Z"), "id": 691893366808825857, "text": "Me and my mom are planning on both going to college soon ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2457457297, "name": "n8", "screen_name": "intiminate", "lang": "en", "location": "null", "create_at": date("2014-04-21"), "description": "i really enjoy music", "followers_count": 485, "friends_count": 94, "statues_count": 57230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-26T00:00:08.000Z"), "id": 691893366850768896, "text": "Wind 3.0 mph S. Barometer 30.139 in, Falling Rapidly. Temperature 28.0 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 3883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255721619066880, "text": "Starlito - Heavy ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 280369046, "name": "OG_Emm", "screen_name": "ICHOCOLATE_DROP", "lang": "en", "location": "UNT", "create_at": date("2011-04-10"), "description": "ALLAH 1st | GAMBIAN |", "followers_count": 2946, "friends_count": 1789, "statues_count": 69363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255721707151361, "text": "4 �� https://t.co/PeYf1dgRys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1567710397, "name": "Cardi T", "screen_name": "ohtaayyy", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "Baton Rouge ☀️ #UL19", "followers_count": 324, "friends_count": 284, "statues_count": 6013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255721786843136, "text": "Crushes are meant to crush you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55722103, "name": "TheHomieChris", "screen_name": "Mr_KUSH_415", "lang": "en", "location": "SanFrancisco bread Sac livin", "create_at": date("2009-07-10"), "description": "born and raised in san Francisco California, living in Sacramento California.", "followers_count": 151, "friends_count": 126, "statues_count": 13818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255722021720064, "text": "�� im glad i followed you ������ your tweets b cracking me TF up! lmmmmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1904675659, "name": "Dahls.", "screen_name": "dahliababyyy", "lang": "en", "location": "southern california.", "create_at": date("2013-09-25"), "description": "21 | Samoan", "followers_count": 1059, "friends_count": 809, "statues_count": 40418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610858,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255722852196352, "text": "Wind 0.0 mph ---. Barometer 1032.58 mb, Steady. Temperature 23.4 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255723292606464, "text": "#whysoserious #rednotmycolor @ Nacogdoches, Texas https://t.co/0yVChSbo3m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.6509,31.6089"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "whysoserious", "rednotmycolor" }}, "user": { "id": 28174604, "name": "Fred", "screen_name": "Texasballer24", "lang": "en", "location": "Chireno TX", "create_at": date("2009-04-01"), "description": "just a laid back guy I'm a die hard patriots fan Brady my favorite QB I'm a Lakers fan and a Yankee fan", "followers_count": 258, "friends_count": 1325, "statues_count": 13159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255723691057157, "text": "@sergiotistic on momas blood", "in_reply_to_status": 692255522762924032, "in_reply_to_user": 2443399764, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2443399764 }}, "user": { "id": 1474139797, "name": "yung ugly", "screen_name": "marlbaero", "lang": "en", "location": "The Star Room", "create_at": date("2013-06-01"), "description": "we with the shits around here", "followers_count": 543, "friends_count": 290, "statues_count": 67683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255723837857793, "text": "ITS THREE A M I MUST BE LONELY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304707032, "name": "Madi Task", "screen_name": "alwaysoff_TASK", "lang": "en", "location": "The Ohio State University", "create_at": date("2011-05-24"), "description": "I am a nut", "followers_count": 1055, "friends_count": 952, "statues_count": 23014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255723837898756, "text": "This surgeon is so creepy and handsy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2793033817, "name": "violet", "screen_name": "baraboyfriend", "lang": "en", "location": "null", "create_at": date("2014-09-05"), "description": "Crazy bitch coming for your shmoney", "followers_count": 24, "friends_count": 33, "statues_count": 3111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255724437643265, "text": "Saturday I found a tardigrade in my lab sample. 4 days later and I still can't stop thinking about it. #waterbear https://t.co/TRAboTncWc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "waterbear" }}, "user": { "id": 910702830, "name": "Briiiii", "screen_name": "brikpreslar", "lang": "en", "location": "Seattle WA", "create_at": date("2012-10-28"), "description": "University of Washington| How do you know what you're going to do till you do it? The answer is, you don't. -J.D Salinger|✌️", "followers_count": 203, "friends_count": 322, "statues_count": 1308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255724819353602, "text": "@jorrhingle we shall see! I'm just nervous about the in between stage ��", "in_reply_to_status": 692255543021412352, "in_reply_to_user": 290663044, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 290663044 }}, "user": { "id": 629756831, "name": "EmilyHughes", "screen_name": "emmmz56", "lang": "en", "location": "Lafayette, LA", "create_at": date("2012-07-07"), "description": "Meow | UL17' | sc: emmmz34", "followers_count": 1699, "friends_count": 834, "statues_count": 26975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngsville, LA", "id": "004303e66e44b752", "name": "Youngsville", "place_type": "city", "bounding_box": rectangle("-92.063001,30.037609 -91.961825,30.139842") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2283335, "cityName": "Youngsville" } }
+{ "create_at": datetime("2016-01-27T00:00:00.000Z"), "id": 692255724878036992, "text": "Wind 0 mph --. Barometer 30.16 in, Steady. Temperature 43.5 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 13, "friends_count": 4, "statues_count": 20462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255725553356800, "text": "11 great months, right?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448328579, "name": "⚫️⚫️⚫️", "screen_name": "dennisvapes", "lang": "en", "location": "snap: dennisvapes", "create_at": date("2011-12-27"), "description": "big dreams. good music. expensive taste.", "followers_count": 423, "friends_count": 115, "statues_count": 25737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, CA", "id": "41c331abe42d9969", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-117.287533,34.413411 -117.103437,34.571843") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 602364, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726228656132, "text": "@bobbyscar @toph_bbq Fun Fact: Every time you mention my name on stream a friend of mine will message me about it.", "in_reply_to_status": 692253036627038208, "in_reply_to_user": 29154775, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 191201559, 1567359060 }}, "user": { "id": 29154775, "name": "TED | Ashkon", "screen_name": "SDAshkon", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-04-05"), "description": "Founder of ShowdownSmash. NorCal Player Manager. eSports Enthusiast. Burrito lover. UCSD '18. All inquires email @ ashkon@showdown.gg http://ask.fm/SDAshkon", "followers_count": 872, "friends_count": 216, "statues_count": 7385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726270599168, "text": "Donuts always sound good to me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 824589523, "name": "KT☀️", "screen_name": "Wrinkosuave", "lang": "en", "location": "eslv✨", "create_at": date("2012-09-14"), "description": "You got the world but baby at what price.♉ Polar Pop Princess.", "followers_count": 545, "friends_count": 225, "statues_count": 43942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726597705729, "text": "It's literally 1 am like what do you mean", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 528793670, "name": "Madison Mahnesmith", "screen_name": "Madisonanneeee", "lang": "en", "location": "Northern Arizona University☼", "create_at": date("2012-03-18"), "description": "vegas ↔️ flagstaff ❤️ Isaiah 43:2", "followers_count": 813, "friends_count": 490, "statues_count": 52864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726643908608, "text": "Dueces✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1571038790, "name": "demijeremy", "screen_name": "itsdemijeremy", "lang": "en", "location": "null", "create_at": date("2013-07-05"), "description": "JameyRae✨", "followers_count": 190, "friends_count": 258, "statues_count": 6939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726664839168, "text": "I'd rather look at food pics on Instagram than females. {{-_-}}", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16076529, "name": "DR. FCW LIVES ♠", "screen_name": "JohnnyACE562", "lang": "en", "location": "Los Angeles, CA & Nigeria", "create_at": date("2008-08-31"), "description": "I'm just a man who's crazy enough to still believe in a place called Africa.\nGRAND MASTER SEN$Ei {{-_-}}™ @AyyyyyPAPI #BlackFIRST #AfricaFirst @RaiderACE562", "followers_count": 1638, "friends_count": 1954, "statues_count": 72787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Artesia, CA", "id": "b82c98fb6806ceb9", "name": "Artesia", "place_type": "city", "bounding_box": rectangle("-118.091312,33.853432 -118.068916,33.880386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602896, "cityName": "Artesia" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255726916485121, "text": "01/27@03:00 - Temp 44.4F, WC 43.0F. Wind 3.3mph WNW, Gust 8.0mph. Bar 30.014in, Rising slowly. Rain 0.00in. Hum 75%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 46565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255727096827904, "text": "@ArcticFox2016 Does the Mohammedan Farrakhan drink camel piss with his buddies? https://t.co/y2J3595uYW", "in_reply_to_status": 692253270664945664, "in_reply_to_user": 101264587, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101264587 }}, "user": { "id": 23727359, "name": "Matt M", "screen_name": "buddmann", "lang": "en", "location": "Houston ", "create_at": date("2009-03-10"), "description": "Texas, Kafir. Constitutionally Conservative Libertarian,Financial Consultant. Molon Labe", "followers_count": 545, "friends_count": 661, "statues_count": 2973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255727340142592, "text": "@HanSumTay love it when you tag me in fairytail posts ��", "in_reply_to_status": 692243552802377729, "in_reply_to_user": 1627624291, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1627624291 }}, "user": { "id": 231556296, "name": "ロレンゾ ❥", "screen_name": "lorenzofololo", "lang": "en", "location": "null", "create_at": date("2010-12-28"), "description": "♛Princess♛ Lumives ❥", "followers_count": 288, "friends_count": 225, "statues_count": 21557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255727944073217, "text": "@maaarrriiiiiii u caught me ��", "in_reply_to_status": 692014303826874368, "in_reply_to_user": 417021617, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417021617 }}, "user": { "id": 1010545134, "name": "Danielle.", "screen_name": "evangelinalice", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-12-13"), "description": "21 / raised in Japan", "followers_count": 318, "friends_count": 432, "statues_count": 11790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255728032223232, "text": "Yeah https://t.co/5B0PS2UBHc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28543715, "name": "Trilluminati.", "screen_name": "thirdeyesquints", "lang": "en", "location": "Little Wakanda, CA", "create_at": date("2009-04-03"), "description": "Filmmaker/Poet/Political Agitator\n[Oakwood Grad '15]\n#Huemvn #PHA\n#Acorn |ΝΣΩ| @ArtnSoul_", "followers_count": 3893, "friends_count": 5001, "statues_count": 210159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255728048955392, "text": "@ebbtideapp Tide in Lighthouse Point, Connecticut 01/27/2016\n Low 7:08am -0.0\nHigh 1:17pm 6.2\n Low 7:30pm -0.1\nHigh 1:41am 6.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-72.905,41.2517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 15, "friends_count": 1, "statues_count": 2582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Connecticut, USA", "id": "e86b380cfefcced5", "name": "Connecticut", "place_type": "admin", "bounding_box": rectangle("-73.727776,40.950918 -71.786994,42.050588") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255728208363520, "text": "over slept at my boyfriends house ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4641772272, "name": "janegaxiola", "screen_name": "jaaaaaayg__", "lang": "en", "location": "Compton, CA", "create_at": date("2015-12-23"), "description": "zeke ❤️", "followers_count": 39, "friends_count": 43, "statues_count": 233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, CA", "id": "7df6f50f15138f28", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-118.282262,33.901902 -118.222378,33.929527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685614, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255728388669440, "text": "Trusting hoes that'll get you set up quick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320884439, "name": "Anti-Filth Ranger", "screen_name": "313doe", "lang": "en", "location": "Should Be In Detroit", "create_at": date("2011-06-20"), "description": "Just a Detroit nigga living in Tx #IfItAintFoodIAintEatingIt", "followers_count": 3183, "friends_count": 3068, "statues_count": 163213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255729034645504, "text": "Wind 0.0 mph SE. Barometer 30.363 in, Steady. Temperature 31.5 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-27T00:00:01.000Z"), "id": 692255729047199744, "text": "Temp: 42.0°F Wind:0.6mph Pressure: 30.152hpa Steady Rain Today 0.01in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255729730895872, "text": "@ceexoo @joseph_cedillos dumb nigga", "in_reply_to_status": 692255644607463424, "in_reply_to_user": 2917910591, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2917910591, 2455582664 }}, "user": { "id": 2581781802, "name": "RJ", "screen_name": "fvck_raymond", "lang": "en", "location": "sc // Ray_Figueroa", "create_at": date("2014-06-22"), "description": "Lench Mob", "followers_count": 1029, "friends_count": 587, "statues_count": 17444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255731010158593, "text": "43.0F (Feels: 43.0F) - Humidity: 76% - Wind: 15.9mph N - Gust: 16.8mph - Pressure: 1032.0mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 224210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255731068858369, "text": "Wind 4.0 mph NW. Barometer 30.305 in, Rising slowly. Temperature 30.1 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255731253403648, "text": "https://t.co/ceIwzEd7Jy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4767007640, "name": "Brittaney Nicole", "screen_name": "brittaneyboww", "lang": "en", "location": "null", "create_at": date("2016-01-15"), "description": "null", "followers_count": 29, "friends_count": 73, "statues_count": 37 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255731857412097, "text": "Sam Hunt is so perfect I can't even anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416355901, "name": "MarandaMalnory", "screen_name": "MarandaMalnory", "lang": "en", "location": "Punta Gorda,Florida", "create_at": date("2011-11-19"), "description": "Classy Like A Lady\nFlorida Girl\nTaken\nWriting History One Day At A Time\nLover Of Disney\n20 Years Old.", "followers_count": 214, "friends_count": 1074, "statues_count": 2000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte Harbor, FL", "id": "1501105f277d2dfa", "name": "Charlotte Harbor", "place_type": "city", "bounding_box": rectangle("-82.084626,26.946308 -82.031942,26.975126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12015, "countyName": "Charlotte", "cityID": 1211625, "cityName": "Charlotte Harbor" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255732297764864, "text": "I love this place ✨ @ The Neon Museum https://t.co/5zIE8WIPkB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1350479,36.1768608"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 145877337, "name": "Naz Perez", "screen_name": "JNazPerez", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-19"), "description": "Producer on The Bachelor, The Bachelorette, and Bachelor in Paradise I Sports Fashion Expert/TV Host l Founder of http://sidelinestyles.com", "followers_count": 1510, "friends_count": 927, "statues_count": 4845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255732335517698, "text": "@DavidSRudin NOOOOOOOOOOO", "in_reply_to_status": 692253188821483520, "in_reply_to_user": 392503731, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 392503731 }}, "user": { "id": 148066896, "name": "S.I. Rosenbaum", "screen_name": "sirosenbaum", "lang": "en", "location": "null", "create_at": date("2010-05-25"), "description": "The atlas-eater with a jaw for news :: formerly Boston magazine, Boston Phoenix, Tampa Bay Times :: Hire me!", "followers_count": 2879, "friends_count": 1548, "statues_count": 73132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255733048594432, "text": "@alex_balderas1 ok I'll just break into the PA and advertise games thanks for the advice", "in_reply_to_status": 692255383914745856, "in_reply_to_user": 4699507454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4699507454 }}, "user": { "id": 2543002495, "name": "T$@R", "screen_name": "Robert_Inger818", "lang": "en", "location": "TRAP", "create_at": date("2014-06-03"), "description": "ignorance is bliss. lions don't loose sleep over the opinions of sheep.", "followers_count": 205, "friends_count": 231, "statues_count": 11059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:02.000Z"), "id": 692255733111459841, "text": "Wind 0.4 mph NE. Barometer 30.156 in, Steady. Temperature 39.7 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255733505757184, "text": "Wind 5.4 mph W. Barometer 29.94 in, Steady. Temperature 37.0 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott Van Nostrand", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 28, "friends_count": 103, "statues_count": 157053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255733681889280, "text": "Nothing like a great workout to end your night!! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 149371843, "name": "#chrissythemua", "screen_name": "HELLOchrisssy", "lang": "en", "location": "In your boy friends heart ;", "create_at": date("2010-05-28"), "description": "just a girl who wishes for the world, ambitious girl ; Instagram: chrisssssyj certified makeup artist #chrissythemua #makeupbychrissyj", "followers_count": 532, "friends_count": 543, "statues_count": 38493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255735279919107, "text": "I didn't wake up to see my mentions with mad people over a tweet I didn't tweet. A nigga can't reply to a joke?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61366106, "name": "KLHJR", "screen_name": "Flyer_thanu2", "lang": "en", "location": "Chicago/Blue Island, IL", "create_at": date("2009-07-29"), "description": "28 #80sBaby I'm only here for laughs, sports, and music. Forget a debate #49ers #Lakers #OKC Instagram: Kenny_DatDude", "followers_count": 5020, "friends_count": 3662, "statues_count": 560920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255735288315905, "text": "@inhaleDRO lmfaooooo damn I gotta get like that nigga", "in_reply_to_status": 692255475606360064, "in_reply_to_user": 271128451, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 271128451 }}, "user": { "id": 35177564, "name": "187 Proof", "screen_name": "DaLilHomieMark", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-25"), "description": "Westside rollin, wherever you are", "followers_count": 591, "friends_count": 485, "statues_count": 32335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255735330279425, "text": "Anything that has cinnamon is nasty ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3298035156, "name": "daZe", "screen_name": "steezyjess", "lang": "en", "location": "TX", "create_at": date("2015-07-27"), "description": "We Live To Die #RATPACK", "followers_count": 248, "friends_count": 181, "statues_count": 7454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255735904915456, "text": "#SupportOriginMelissa 30.2°F Wind:0.0mph Pressure: 30.24hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 308189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255736152350723, "text": "@kiki_mcgrath !!!HAPPY BIRTHDAY TO MY GORGEOUS BEST FRIEND !!! i love you so much expect more things throughout ur day��❤️I've got surprises", "in_reply_to_status": -1, "in_reply_to_user": 2616313509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2616313509 }}, "user": { "id": 3079649880, "name": "claudia duran", "screen_name": "claudiaxduran", "lang": "en", "location": "null", "create_at": date("2015-03-13"), "description": "pvhs | blessed", "followers_count": 392, "friends_count": 422, "statues_count": 2287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255736869588993, "text": "Can you find Austin on the map? Just try it at https://t.co/bdSAJVqXgX #Austin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7431,30.2672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Austin" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1012, "friends_count": 312, "statues_count": 2553631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255736995418112, "text": "https://t.co/Q7EaTZ6fJn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio 20k", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 20135, "friends_count": 16307, "statues_count": 6857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255737213521920, "text": "#Dimaggio's #Woodbridge Farms #Sherwood Park https://t.co/8YSqbHBrVV the best restaurant in https://t.co/d0PqXwpKrR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.3188565,53.5610515"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dimaggio", "Woodbridge", "Sherwood" }}, "user": { "id": 139342722, "name": "FoodPages.ca", "screen_name": "foodpages", "lang": "en", "location": "Canada", "create_at": date("2010-05-02"), "description": "The Guide to Canada's Restaurants and Food Stores", "followers_count": 3652, "friends_count": 507, "statues_count": 164515 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Strathcona County, Alberta", "id": "79a60b51078a624f", "name": "Strathcona County", "place_type": "city", "bounding_box": rectangle("-113.402012,53.354476 -112.840174,53.893471") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255737322573825, "text": "@kianaanolani finals are all done ��", "in_reply_to_status": 692225382465048576, "in_reply_to_user": 2977339736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2977339736 }}, "user": { "id": 2977339736, "name": "ㅤㅤㅤㅤㅤㅤ", "screen_name": "kianaanolani", "lang": "en", "location": "18", "create_at": date("2015-01-12"), "description": "i'm down", "followers_count": 227, "friends_count": 461, "statues_count": 12383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawton, OK", "id": "52da8bdd847955f2", "name": "Lawton", "place_type": "city", "bounding_box": rectangle("-98.545211,34.547508 -98.317202,34.694413") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40031, "countyName": "Comanche", "cityID": 4041850, "cityName": "Lawton" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255737364488192, "text": "@Stephtheballer hell no", "in_reply_to_status": 692255572096323584, "in_reply_to_user": 2324685716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2324685716 }}, "user": { "id": 162325109, "name": "MyionAkaNintendougly", "screen_name": "Supremejackson1", "lang": "en", "location": "null", "create_at": date("2010-07-03"), "description": "If It Ain't About Prgress Then It ain't About Success// sc -supremejackson1 // ig -supremejackson1", "followers_count": 554, "friends_count": 535, "statues_count": 38044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:03.000Z"), "id": 692255737452630017, "text": "@ErikVeland I work from home across the street from a school. During school open & close times, safest for me to stay inside. Do many cars!", "in_reply_to_status": 692253222413692929, "in_reply_to_user": 15080202, "favorite_count": 0, "coordinate": point("-122.32225871,47.67768798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15080202 }}, "user": { "id": 10376902, "name": "brian wisti", "screen_name": "brianwisti", "lang": "en", "location": "Seattle, WA", "create_at": date("2007-11-18"), "description": "Not tall. Crafty (Knitting Cooking Drawing) Seattle (Coffee Yuppies Rain) geek (Python Perl Ruby JavaScript) - email? my name at gmail. he/him", "followers_count": 1483, "friends_count": 1555, "statues_count": 38353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255737771380736, "text": "#tb with fam https://t.co/jTXVfTIyMn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tb" }}, "user": { "id": 3219373919, "name": "lul mama❣", "screen_name": "xo_babylaney", "lang": "en", "location": "on your mind ", "create_at": date("2015-04-28"), "description": "null", "followers_count": 377, "friends_count": 227, "statues_count": 11983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenarden, MD", "id": "5542db7392bde9b1", "name": "Glenarden", "place_type": "city", "bounding_box": rectangle("-76.876542,38.912849 -76.821187,38.94025") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2432500, "cityName": "Glenarden" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255737926586369, "text": "@Katt_GarciaXXX Lorena Escobar contacting restuarants and telling them to stalk me with her https://t.co/knuR1NBQTW", "in_reply_to_status": 692252165784604672, "in_reply_to_user": 258171134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 258171134 }}, "user": { "id": 25652802, "name": "♔ kevin perelman ♔", "screen_name": "KevinPerelman", "lang": "en", "location": "Woodland Hills", "create_at": date("2009-03-21"), "description": "Im Kevin Perelman.You all know me, but pretend not to due to a world wide funded smear campaign since 10. Nerdist, Artist, Photographr,Web Developer,3D graphics", "followers_count": 2011, "friends_count": 2409, "statues_count": 807044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255738266279936, "text": "Fucking gross. Burqa, STAT. https://t.co/g5CejkRbEL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2479125236, "name": "Mack Webbahoe", "screen_name": "Clevetroit", "lang": "en", "location": "Clevemada, MI ", "create_at": date("2014-05-05"), "description": "Father, Husband, Son, Brother, Army Vet • Family is Life • Mantua/Nelson, OH expat • Browns/Tribe/Cavaliers/Buckeyes/Herd • 18,658 Days Without A Championship..", "followers_count": 716, "friends_count": 1502, "statues_count": 10238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sterling Heights, MI", "id": "23e0af972b9a2e8e", "name": "Sterling Heights", "place_type": "city", "bounding_box": rectangle("-83.091535,42.534811 -82.969611,42.627801") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2676460, "cityName": "Sterling Heights" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255738576683008, "text": "Wind 0.9 mph WSW. Barometer 29.827 in, Rising. Temperature 36.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255738778030080, "text": "Why doesn’t @periscopeco work with the @GoPro Session? Hello.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2445809510, 32340023 }}, "user": { "id": 28504358, "name": "Root Beer Guy", "screen_name": "johnmfive", "lang": "en", "location": "Studio City, CA ", "create_at": date("2009-04-02"), "description": "Nora, Dogs, and Reptiles, Graphic Design, Visual Effects", "followers_count": 825, "friends_count": 374, "statues_count": 18816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Studio City, Los Angeles", "id": "563dd564285e7363", "name": "Studio City", "place_type": "neighborhood", "bounding_box": rectangle("-118.429061,34.119011 -118.360884,34.157715") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255738970951684, "text": "temperature up 56°F -> 57°F\nhumidity down 51% -> 28%\nwind 0mph -> 5mph\npressure 30.24in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.37093,34.09068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220580042, "name": "Weather", "screen_name": "WestHollywoodCA", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2010-11-27"), "description": "Weather updates, forecast, warnings and information for West Hollywood, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 49, "friends_count": 1, "statues_count": 18111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255739247775744, "text": "@jaq0820 @NancyGraceHLN if that idiot thought that Jodi was a dv survivor then he really needs to know what DV really is. iam a survivor", "in_reply_to_status": 692253832689098752, "in_reply_to_user": 35358295, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35358295, 269852626 }}, "user": { "id": 35358295, "name": "jaq0820", "screen_name": "jaq0820", "lang": "en", "location": "usa", "create_at": date("2009-04-25"), "description": "Im a mom, an artist, a down to earth non girly girl. That's just me...simple.", "followers_count": 59, "friends_count": 218, "statues_count": 389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255739356823553, "text": "@BansheeBitch_1 I'm constantly watching reruns of #Banshee. Great show!", "in_reply_to_status": -1, "in_reply_to_user": 2994451216, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Banshee" }}, "user_mentions": {{ 2994451216 }}, "user": { "id": 1173065982, "name": "Susan Hassard", "screen_name": "HassardSusan", "lang": "en", "location": "San Francisco & San Diego, CA ", "create_at": date("2013-02-12"), "description": "Worked for 20 yrs as a paralegal in SF Bay Area. Am an activist for the Elderly, Battered Women & Animals. Elder health care is substandard in USA! Fix Fast!!!", "followers_count": 871, "friends_count": 2293, "statues_count": 2169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255739641995266, "text": "HAPPY BIRTHDAY MAK❤️❤️❤️❤️❤️ have a bomb ass day girl ��@makaylasmith509 https://t.co/7si6OCij9B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2265772760 }}, "user": { "id": 2185669303, "name": "ogpres®", "screen_name": "presley_yates", "lang": "en", "location": "pacific northwest", "create_at": date("2013-11-09"), "description": "snapchat:OG.PRES", "followers_count": 466, "friends_count": 219, "statues_count": 5053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255740275351552, "text": "Hahah black people can't live with em (literally cause they always shooting each other) and can't live without Em https://t.co/yLpM6lt7O8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447627722, "name": "tymhunt", "screen_name": "_TymHunt", "lang": "en", "location": "Anchorage, Alaska ", "create_at": date("2011-12-26"), "description": "null", "followers_count": 289, "friends_count": 275, "statues_count": 5149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waukesha, WI", "id": "ce868cf1adf8c805", "name": "Waukesha", "place_type": "city", "bounding_box": rectangle("-88.305771,42.902074 -88.174237,43.061202") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5584250, "cityName": "Waukesha" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255740669628416, "text": "@oCURIOUSo My pleasure madam. :-)", "in_reply_to_status": 692246007892045824, "in_reply_to_user": 1161858704, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1161858704 }}, "user": { "id": 14647078, "name": "Fireforce", "screen_name": "Fireforce", "lang": "en", "location": "Bronx NY", "create_at": date("2008-05-04"), "description": "Getting my Geek on. Trying to live a Stress & Drama Free Life!", "followers_count": 429, "friends_count": 2024, "statues_count": 8707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Socastee, SC", "id": "0043ce7c1cad5c16", "name": "Socastee", "place_type": "city", "bounding_box": rectangle("-79.06754,33.614208 -78.921739,33.731483") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4567390, "cityName": "Socastee" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255740682182656, "text": "���������������������� https://t.co/g7HqG3tEmt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 298846640, "name": "AYoungOG⚡️", "screen_name": "BDugasss", "lang": "en", "location": "Where Playa's get chose ", "create_at": date("2011-05-14"), "description": "20|NeighborhoodSupaStar ✨ Bitch I'm from Rayne, Louisiana", "followers_count": 1744, "friends_count": 785, "statues_count": 117576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rayne, LA", "id": "807281e8b753fb7e", "name": "Rayne", "place_type": "city", "bounding_box": rectangle("-92.296633,30.198776 -92.20768,30.264282") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22001, "countyName": "Acadia", "cityID": 2263645, "cityName": "Rayne" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255740828991488, "text": ".@lindseybieda you're my favorite Twitter person and I think your projects and contributions are important", "in_reply_to_status": 692136088652095490, "in_reply_to_user": 14928483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14928483 }}, "user": { "id": 1870953398, "name": "brendan", "screen_name": "tree_herder", "lang": "en", "location": "Oakland, California ", "create_at": date("2013-09-16"), "description": "pythonista, feminist, cyclist, gardener. clojure, embedded c, computer vision, natural language, and autonomous navigation enthusiast.", "followers_count": 86, "friends_count": 255, "statues_count": 574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255740975804416, "text": "I always wonder how different and better my life would have been if I wouldn't have gotten with my ex �� #12amthoughts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "12amthoughts" }}, "user": { "id": 1594556118, "name": "Tessa Vasquez", "screen_name": "tessamichaelv", "lang": "en", "location": "Commerce, CA", "create_at": date("2013-07-14"), "description": "null", "followers_count": 464, "friends_count": 612, "statues_count": 13793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, CA", "id": "be347aa731d4353b", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-118.193396,33.970521 -118.121279,34.019619") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 614974, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255741122592770, "text": "Falling asleep before 3am is officially impossible", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307950296, "name": "Joseph Rotella", "screen_name": "J_Ro49", "lang": "en", "location": "Long Island", "create_at": date("2011-05-30"), "description": "❤Jennie Giangarra❤️ Scoundrel. R.I.P. Paris 7/12/12. Tanner Glass advocate. Farmingdale State Roller Hockey G #49", "followers_count": 145, "friends_count": 111, "statues_count": 11387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hicksville, NY", "id": "48b5e9defad76941", "name": "Hicksville", "place_type": "city", "bounding_box": rectangle("-73.560512,40.743142 -73.499673,40.793139") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3634374, "cityName": "Hicksville" } }
+{ "create_at": datetime("2016-01-27T00:00:04.000Z"), "id": 692255741181366272, "text": "@YGRFLooch prolly a range.", "in_reply_to_status": 692251965707919360, "in_reply_to_user": 1558851440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1558851440 }}, "user": { "id": 418468198, "name": "2wheelbandit", "screen_name": "clutchGonz", "lang": "en", "location": "Ci/Bwood", "create_at": date("2011-11-21"), "description": "Instagram: clutchgonz", "followers_count": 246, "friends_count": 223, "statues_count": 11178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, NY", "id": "5fe049183a425395", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-73.295325,40.754764 -73.209269,40.807341") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608026, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255742016000000, "text": "I got the live at the palace dvd an I love it very much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 532012254, "name": "Mase The Doge", "screen_name": "HippieMase", "lang": "en", "location": "N. Sanity Island", "create_at": date("2012-03-20"), "description": "http://soundcloud.com/mason-marsh Just a rapper who plays guitar riffs in an experimental jam band #illhueminati", "followers_count": 458, "friends_count": 580, "statues_count": 25627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255742460608513, "text": "@savs2010 Mandelo a la requinta�� Sarita.", "in_reply_to_status": 692255412851204096, "in_reply_to_user": 136694645, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 136694645 }}, "user": { "id": 2180791153, "name": "→Paola Flores←", "screen_name": "paolitaBFR", "lang": "es", "location": "Los Angeles, CA", "create_at": date("2013-11-07"), "description": "→Amando infinitamente a mi País. Nicaragüense por gracia de Dios; Triniteña❤\n✨18/03 NL✨\n In a relationship with basketball, Fanatic.", "followers_count": 483, "friends_count": 247, "statues_count": 12644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255742678700032, "text": "Literally cried @PrincesLashae02 https://t.co/5FCj0p4kcj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 483632314 }}, "user": { "id": 3440142140, "name": "Dee Baby ❤", "screen_name": "Dope_Rainbowz", "lang": "en", "location": "Where your girl really at", "create_at": date("2015-09-03"), "description": "I'm bout my shit, Are you? College Chocolate Queen ⚠Slightly Mean⚠ #L.A.M. #BlackMafia snap me: @RainbowQueenDee", "followers_count": 1274, "friends_count": 1532, "statues_count": 448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255743022649344, "text": "Need a real 1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3058681316, "name": "Hales", "screen_name": "hhhales", "lang": "en", "location": "null", "create_at": date("2015-03-02"), "description": "you do what you want when you poppin", "followers_count": 469, "friends_count": 323, "statues_count": 5831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255743261741057, "text": "@arkansasgrant @missy_kate_ https://t.co/stnhxR9Jue", "in_reply_to_status": -1, "in_reply_to_user": 552342500, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 552342500, 256596624 }}, "user": { "id": 33027193, "name": "Taylor Naegle", "screen_name": "Tnaegle113", "lang": "en", "location": "null", "create_at": date("2009-04-18"), "description": "I'm about as stressed as Brittany 2007 snapchat: tnizzle5 #ATU", "followers_count": 372, "friends_count": 274, "statues_count": 6962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Smith, AR", "id": "a5cb06fbef643420", "name": "Fort Smith", "place_type": "city", "bounding_box": rectangle("-94.435514,35.256048 -94.305102,35.438472") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5131, "countyName": "Sebastian", "cityID": 524550, "cityName": "Fort Smith" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255743471390720, "text": "when the episode you was telling ur friend about come on the next day >> @YakuzaTheLoser #HowIMetYourMother s3 ep8 \"Spoiler Alert\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HowIMetYourMother" }}, "user_mentions": {{ 185533371 }}, "user": { "id": 495794352, "name": "lames", "screen_name": "hiimrand0m", "lang": "en", "location": "null", "create_at": date("2012-02-18"), "description": "Just as true humor is laughter at oneself, true humanity is knowledge of oneself.", "followers_count": 96, "friends_count": 392, "statues_count": 2573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabeth, NJ", "id": "b74cebcb62a1a686", "name": "Elizabeth", "place_type": "city", "bounding_box": rectangle("-74.254211,40.634285 -74.138838,40.690673") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3421000, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255743509204992, "text": "Yeah https://t.co/gfBFPJf9Rp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28543715, "name": "Trilluminati.", "screen_name": "thirdeyesquints", "lang": "en", "location": "Little Wakanda, CA", "create_at": date("2009-04-03"), "description": "Filmmaker/Poet/Political Agitator\n[Oakwood Grad '15]\n#Huemvn #PHA\n#Acorn |ΝΣΩ| @ArtnSoul_", "followers_count": 3893, "friends_count": 5001, "statues_count": 210160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255744129921031, "text": "These the hours all be getting cheated on ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391207826, "name": "Barnaaaa' ✨", "screen_name": "YallStupidDoe__", "lang": "en", "location": "null", "create_at": date("2011-10-14"), "description": "Sweeter the candy wetter the juice", "followers_count": 2684, "friends_count": 1680, "statues_count": 54535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitehall, OH", "id": "0fdd28012cbb8a2d", "name": "Whitehall", "place_type": "city", "bounding_box": rectangle("-82.91273,39.950926 -82.851156,39.986808") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3984742, "cityName": "Whitehall" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255744289292289, "text": "The comments you see on @justinbieber 's photos �� https://t.co/VjhQ6ezwvq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 3404447381, "name": "Mama Ling", "screen_name": "sisamahinay", "lang": "en", "location": "San Francisco, CA", "create_at": date("2015-08-05"), "description": "SC/IG: sisamahinay", "followers_count": 287, "friends_count": 147, "statues_count": 2162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255744796790784, "text": "She don't sing songs but if I go strong she gone hit this hiiiiiiiiiiiiigh note.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967364264, "name": "Jose Alvarez", "screen_name": "josealvarez710", "lang": "en", "location": "null", "create_at": date("2012-11-23"), "description": "Remain humble through it all", "followers_count": 621, "friends_count": 461, "statues_count": 13302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255744821977089, "text": "@_stephpichardo ��", "in_reply_to_status": 692255552001425408, "in_reply_to_user": 2627651422, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2627651422 }}, "user": { "id": 998267982, "name": "Zack Balcaceres", "screen_name": "Zaack_3", "lang": "en", "location": "The 6 ", "create_at": date("2012-12-08"), "description": "#HeemTeam UCR x ΒΥΔ", "followers_count": 485, "friends_count": 446, "statues_count": 29427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255745237217280, "text": "#Shadowhunters was amazing again. Thank you ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Shadowhunters" }}, "user": { "id": 31304361, "name": "Red", "screen_name": "amandaliberto", "lang": "en", "location": "CA ☀️ AZ", "create_at": date("2009-04-14"), "description": "I'm more like whiskey neat and you're more like vodka punch. ASU ✊", "followers_count": 948, "friends_count": 502, "statues_count": 38499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-27T00:00:05.000Z"), "id": 692255745941868544, "text": "@_xoxomari_ not true ��", "in_reply_to_status": 692255171766788097, "in_reply_to_user": 234264733, "favorite_count": 0, "coordinate": point("-121.53342534,39.07835275"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 234264733 }}, "user": { "id": 589853630, "name": "jamesmacc", "screen_name": "CeeJay4_", "lang": "en", "location": "null", "create_at": date("2012-05-25"), "description": "Yuba hoops RIP dad 12/4/14", "followers_count": 754, "friends_count": 495, "statues_count": 35966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olivehurst, CA", "id": "7ae0364b4d56f5d6", "name": "Olivehurst", "place_type": "city", "bounding_box": rectangle("-121.579032,39.047486 -121.523467,39.113263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6115, "countyName": "Yuba", "cityID": 653714, "cityName": "Olivehurst" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255746105446401, "text": "* cries * �� https://t.co/fttgZE2LFE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2308046322, "name": "DANIEL", "screen_name": "dxniel_m", "lang": "en", "location": "Lancaster, CA", "create_at": date("2014-01-24"), "description": "null", "followers_count": 455, "friends_count": 222, "statues_count": 15710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255746113806340, "text": "Sometimes I wish I went to San Diego with my niggas @ExcellentEli and Eustis. How different would my life be?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116578221 }}, "user": { "id": 3177475088, "name": "Dave Ortega", "screen_name": "WolfDavely", "lang": "en", "location": "Chico, CA", "create_at": date("2015-04-26"), "description": "18. Chico State. Live Oak '14 School, Cook, Food, Lift, Skate and Pokemon .", "followers_count": 193, "friends_count": 217, "statues_count": 776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255746193514496, "text": "@Babyboy_LOE me dummy ��", "in_reply_to_status": 692253412315000833, "in_reply_to_user": 3657037042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3657037042 }}, "user": { "id": 606293095, "name": "Shawn Ward, Jr. ☁️", "screen_name": "shawncward3", "lang": "en", "location": "Baltimore, MD", "create_at": date("2012-06-12"), "description": "God First. 16' Senior Highlights Below ☁️", "followers_count": 3732, "friends_count": 1216, "statues_count": 19122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reisterstown, MD", "id": "0019b30073d7f767", "name": "Reisterstown", "place_type": "city", "bounding_box": rectangle("-76.868425,39.429282 -76.778265,39.559561") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2465600, "cityName": "Reisterstown" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255746315141120, "text": "Headache on 10 my dude ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1618974440, "name": "Edward Wilkes", "screen_name": "being_eddy", "lang": "en", "location": "null", "create_at": date("2013-07-24"), "description": "God First| 305| on the road to riches| never looking back ✈️✈️| #vsu19 Valdosta not Virginia | Instagram @itsthepoloprince greenbay #packernation", "followers_count": 1584, "friends_count": 696, "statues_count": 14985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Remerton, GA", "id": "bdadf12f22b3405f", "name": "Remerton", "place_type": "city", "bounding_box": rectangle("-83.31399,30.840384 -83.303978,30.846943") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1364596, "cityName": "Remerton" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255746319319040, "text": "Ooh this was my jaaaaaam https://t.co/6HclqwCa4S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191649388, "name": "Lyte", "screen_name": "Cali_StyleJas", "lang": "en", "location": "Bay ✈️ USC", "create_at": date("2010-09-16"), "description": "Daughter to the spinnin post. Queen of 2,000 moons. Sister to the distant yet risin' star... Mind. Body. Soul.", "followers_count": 3067, "friends_count": 2764, "statues_count": 251526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255747040768000, "text": "still up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542628478, "name": "jan.30th✨", "screen_name": "alexissymonee", "lang": "en", "location": "Htinee hollitdineee", "create_at": date("2012-04-01"), "description": "houston, tx", "followers_count": 3182, "friends_count": 995, "statues_count": 150752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255747074322432, "text": "Can you find Richmond on the map? Just try it at https://t.co/V8ha8lEAAZ #Richmond", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.4691,37.543"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Richmond" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1012, "friends_count": 312, "statues_count": 2553634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255747112095744, "text": "lol mike p need to gts so I can", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2215193726, "name": "maya.", "screen_name": "mjaaeeee", "lang": "en", "location": "null", "create_at": date("2013-11-25"), "description": "null", "followers_count": 1453, "friends_count": 621, "statues_count": 28427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255747183345664, "text": "Wind 5.5 mph WNW. Barometer 30.023 in, Rising. Temperature 31.7 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 6085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255748101935104, "text": "@uhleexxa go to sleep", "in_reply_to_status": -1, "in_reply_to_user": 4659670775, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4659670775 }}, "user": { "id": 2656307742, "name": "NiallerGiveMeAChonce", "screen_name": "melissabanks_1", "lang": "en", "location": " living in Harry's dimple", "create_at": date("2014-07-18"), "description": "one way or another , I'm going to catch yah and eat yah @Niallofficial", "followers_count": 622, "friends_count": 441, "statues_count": 8830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255748223561729, "text": "Temp: 23.0°F - Dew Point: 15.4° - Wind: 7.4 mph - Gust: 8.9 - Rain Today: 0.00in. - Pressure: 30.22in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255748831735808, "text": "Happy 21st birthday to one of my closest friends @LRCbuhd , thanks for always being A1 since day 1 with me. ❤️��✊��TU https://t.co/4c8jcMTUJl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156554863 }}, "user": { "id": 345185116, "name": "Serenity Cortez", "screen_name": "serenityceee", "lang": "en", "location": "SKN", "create_at": date("2011-07-29"), "description": "haylee is bae", "followers_count": 1600, "friends_count": 858, "statues_count": 59982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255749083418624, "text": "@bnowalk Oh jesus I forgot you people could see these...", "in_reply_to_status": 692255650777305088, "in_reply_to_user": 210312678, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 210312678 }}, "user": { "id": 22670311, "name": "Tyler Swift", "screen_name": "TylerMoliterno", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-03-03"), "description": "Movie lover, Pop Culture fanatic, Boy Band aficionado.", "followers_count": 845, "friends_count": 993, "statues_count": 44383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boardman, OH", "id": "f33f520be81a525c", "name": "Boardman", "place_type": "city", "bounding_box": rectangle("-80.71159,40.970454 -80.603596,41.061426") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3907454, "cityName": "Boardman" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255749112795136, "text": "When I ask for chicken and they tell me that they ran out @thatguyhayes07 https://t.co/jEARuhhCVu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 529661598 }}, "user": { "id": 3236893093, "name": "nel", "screen_name": "Chanelbellsss", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "♥️ @alienhoes ☽❍☾", "followers_count": 47, "friends_count": 22, "statues_count": 1991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255749704146945, "text": "Update: @BDuffied's fashion crisis was avoided and we fixed the issue.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2397102205 }}, "user": { "id": 2438073632, "name": "Melissa Ramont", "screen_name": "callmemel100", "lang": "en", "location": "null", "create_at": date("2014-04-10"), "description": "you have to listen to understand...", "followers_count": 145, "friends_count": 153, "statues_count": 5519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255749955850240, "text": "Wind 5.0 mph W. Barometer 30.205 in, Steady. Temperature 25.0 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 376, "friends_count": 288, "statues_count": 7802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-01-27T00:00:06.000Z"), "id": 692255750035476484, "text": "Umm , @rihanna flaunted this first.. Quit being a copy. #basic https://t.co/wm3FPDLMLF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "basic" }}, "user_mentions": {{ 79293791 }}, "user": { "id": 3240890048, "name": "t e r i", "screen_name": "tweetingteri", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2015-06-09"), "description": "come here often ?!", "followers_count": 171, "friends_count": 220, "statues_count": 2157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-27T00:00:07.000Z"), "id": 692255750811447296, "text": "allnighter part 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4099495738, "name": "shawtyy", "screen_name": "svgnatyy", "lang": "en", "location": "null", "create_at": date("2015-11-02"), "description": "#GIRLGANG", "followers_count": 802, "friends_count": 1193, "statues_count": 15988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.654929 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-01-27T00:00:07.000Z"), "id": 692255751050522624, "text": "1/27/2016 - 02:00\nTemp: 31.0F \nHum: 90%\nWind: 0.0 mph\nBaro: 30.343in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 49408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-01-27T00:00:07.000Z"), "id": 692255751117619200, "text": "@DisabledCanada unclaimed $519.69 \nNATIONAL CAPITAL SPORTS COUNCIL OF THE DISABLED INC. O/A NCSCD WHEELATHON 2410 SOUTHVALE CRESCENT OTTAWA", "in_reply_to_status": -1, "in_reply_to_user": 373228224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373228224 }}, "user": { "id": 15285394, "name": "Rock Southward", "screen_name": "rocksouthward", "lang": "en", "location": "San Bruno, CA", "create_at": date("2008-06-30"), "description": "Retired notary.", "followers_count": 573, "friends_count": 2256, "statues_count": 3900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-01-27T00:00:07.000Z"), "id": 692255751235047424, "text": "lol dam my manz out the game https://t.co/DXKsnMc4oz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3241922460, "name": "CONCRETEGANG", "screen_name": "RipRashad29", "lang": "en", "location": "null", "create_at": date("2015-06-10"), "description": "BEYOU . FREE MY BLOODLINE DAN & GMONEY!! DO IT FOR RASHAD HE GONE..NO REGRETS FAITH IS EVERYTHING", "followers_count": 471, "friends_count": 340, "statues_count": 24748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618109736218624, "text": "JJ Redick Just Hit Me With The \"FAM\" Tweet.. I'm Done.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256769214, "name": "Flip", "screen_name": "Flip4OC", "lang": "en", "location": "#TeamBarnes ", "create_at": date("2011-02-23"), "description": "If you don't know, you better aks somebody #BirdsUP", "followers_count": 889, "friends_count": 333, "statues_count": 83075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618110277267456, "text": "Throw my non existing ass in a circle ���� https://t.co/e5ZbyEXayW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3042185341, "name": "FavoriteBlackGirl ✊", "screen_name": "imxni_lv", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "Vegas // Senior // Coronado High // Learning The Game", "followers_count": 444, "friends_count": 448, "statues_count": 8397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111414091776, "text": "Nobody notices you when you're climbing, only when you're falling ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64848012, "name": "8⃣OchoGuapo", "screen_name": "ASAPxAnt_", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "APK Alum. CSU Defensive End. Trying to make my signature a autograph! WV✈️FL✈️SC", "followers_count": 1151, "friends_count": 955, "statues_count": 30534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Charleston, SC", "id": "18df13e4a5a670b5", "name": "North Charleston", "place_type": "city", "bounding_box": rectangle("-80.15759,32.829336 -79.934288,32.999393") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4550875, "cityName": "North Charleston" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111443283968, "text": "Can you recommend anyone for this #Healthcare #job? https://t.co/ldloHKIYcl #Houston, TX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.4246128,29.7359293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Houston", "Hiring" }}, "user": { "id": 28620145, "name": "Memorial Hermann TA", "screen_name": "MHTalent", "lang": "en", "location": "null", "create_at": date("2009-04-03"), "description": "Dedicated to presenting career related info and job opportunities with Memorial Hermann Health System. Come join this amazing family of professionals!", "followers_count": 391, "friends_count": 67, "statues_count": 1952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111455883266, "text": "Who is my Winner bias?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2749764130, "name": "andrea", "screen_name": "majesticincle", "lang": "en", "location": "buy Pricked", "create_at": date("2014-08-21"), "description": "WINNER-TWICE-SUHO", "followers_count": 650, "friends_count": 200, "statues_count": 24528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111640403968, "text": "Listening to trasatlanticism will always make me think of you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414419720, "name": "Nat Sinclair", "screen_name": "Nat_Sinclair", "lang": "en", "location": "Conway, AR", "create_at": date("2011-11-16"), "description": "OkLaHoMa insta- nataliesinclair snapchat- nat_sinclair25 Hendrix Womens Basketball", "followers_count": 833, "friends_count": 406, "statues_count": 30962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111804121088, "text": "Wind 0 mph --. Barometer 30.10 in, Falling slowly. Temperature 40.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 13, "friends_count": 4, "statues_count": 20485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111820918784, "text": "@QuezzyxB omg what ������ you know I'm like the nicest person hahahah", "in_reply_to_status": 692617940500238336, "in_reply_to_user": 152568155, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 152568155 }}, "user": { "id": 1120851128, "name": "Genny", "screen_name": "SahagunGenesis", "lang": "en", "location": "null", "create_at": date("2013-01-25"), "description": "Fresno State Marching Band and Rugby ✌\n Instagram/snapchat: genesis_sahagun", "followers_count": 1234, "friends_count": 505, "statues_count": 41976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618111992885249, "text": "Me personally wouldnt put that on twitter but hey some mfs a do that for the clout like he said his twitter he post what he want", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1661749554, "name": ".", "screen_name": "KartiBeatz", "lang": "en", "location": "ATL✈️CHI", "create_at": date("2013-08-10"), "description": "Producer | kartibeatz01@gmail.com for beats", "followers_count": 7654, "friends_count": 249, "statues_count": 73421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618112277983232, "text": "I see our friends and they put on a show\nLike they don't want ME to know\nSo they give ME the go-around", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247860092, "name": "Sandee Vanessa♥", "screen_name": "Sandeebaybee", "lang": "en", "location": "null", "create_at": date("2011-02-05"), "description": "IG @sandeevanessa -", "followers_count": 378, "friends_count": 403, "statues_count": 2201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Socorro, TX", "id": "9380bd69f4ecbe4b", "name": "Socorro", "place_type": "city", "bounding_box": rectangle("-106.311637,31.601718 -106.210306,31.689387") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4868636, "cityName": "Socorro" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618112294715392, "text": "Interested in a #Retail #job near #Farmington, CT? This could be a great fit: https://t.co/yFsUf8ysIZ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.7634773,41.7159416"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Farmington", "Hiring" }}, "user": { "id": 2362771358, "name": "Vitamin Shoppe Jobs", "screen_name": "VSIcareers", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "At #VitaminShoppe we are lovers of fitness, health, vitamins, helping you on your wellness journey and motivating others. Apply to our #jobs below.", "followers_count": 512, "friends_count": 96, "statues_count": 3133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Britain, CT", "id": "f080a098cad5b04d", "name": "New Britain", "place_type": "city", "bounding_box": rectangle("-72.826538,41.643749 -72.748011,41.715995") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 950370, "cityName": "New Britain" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618112374542337, "text": "https://t.co/MYbQeuAqAL \n #Trump #Trump2016 #trumpkelly #kellytrump #MeganKelly #MeganKellydebatequestions #foxnews https://t.co/1Aki9xINEv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Trump", "Trump2016", "trumpkelly", "kellytrump", "MeganKelly", "MeganKellydebatequestions", "foxnews" }}, "user": { "id": 3116619775, "name": "SeeMyInside.com", "screen_name": "SeeMyInside", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2015-03-29"), "description": "New Very Convenient Social MultiMedia Advertising Platform for You, Your Business and Your Partner's Business Together By: The Most Powerful Domain in History!", "followers_count": 835, "friends_count": 30, "statues_count": 28589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-01-28T00:00:00.000Z"), "id": 692618112722554881, "text": "@JaimieMalalis lol now we'll see each other bitching about random things ��", "in_reply_to_status": 692592158382002176, "in_reply_to_user": 717678480, "favorite_count": 0, "coordinate": point("-121.91090954,37.76285247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717678480 }}, "user": { "id": 154807773, "name": "Angelica Hidalgo", "screen_name": "ANGELICAHIDALG0", "lang": "en", "location": "null", "create_at": date("2010-06-12"), "description": "null", "followers_count": 74, "friends_count": 146, "statues_count": 57 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Ramon, CA", "id": "4ccb1b26b2b91248", "name": "San Ramon", "place_type": "city", "bounding_box": rectangle("-122.004255,37.721286 -121.878904,37.795164") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 668378, "cityName": "San Ramon" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618113196556288, "text": "Great read https://t.co/0G86x5pILK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1032100686, "name": "Triple888s", "screen_name": "triple888s", "lang": "en", "location": "Austin, TX", "create_at": date("2012-12-23"), "description": "Patriot | Classic Car Collector | Bitcoin and App Entrepreneur | Proud Father of F35 USAF Pilot | #Trump2016", "followers_count": 2711, "friends_count": 3412, "statues_count": 5518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Park, TX", "id": "5c937363480a2c19", "name": "Cedar Park", "place_type": "city", "bounding_box": rectangle("-97.8817,30.455536 -97.74559,30.556381") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4813552, "cityName": "Cedar Park" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618113372672000, "text": "Wind 5.0 mph SW. Barometer 1023.54 mb, Falling. Temperature 33.4 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618113733361664, "text": "Kinda want too, kinda don't", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1430945730, "name": "weva", "screen_name": "annalyseeee_", "lang": "en", "location": "null", "create_at": date("2013-05-15"), "description": "♏️ars | R♡", "followers_count": 214, "friends_count": 186, "statues_count": 24173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618114412838912, "text": "@MarielKrissell happy birthday bitch I am dead", "in_reply_to_status": -1, "in_reply_to_user": 594141329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 594141329 }}, "user": { "id": 1726907689, "name": "Jason Cedric", "screen_name": "JasonVBCedric", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-09-03"), "description": "USD '19 - souls separated at birth", "followers_count": 204, "friends_count": 147, "statues_count": 1977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, CA", "id": "a2c84129f9dcf69f", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-117.230172,33.090761 -117.103461,33.186722") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 668196, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618114794594305, "text": "Finally 17��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2851608300, "name": "David Camacho", "screen_name": "exCamacho", "lang": "en", "location": "Cali", "create_at": date("2014-10-11"), "description": "No pain no game\nWalnut Football #64\nAmanda", "followers_count": 176, "friends_count": 201, "statues_count": 742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618114849083395, "text": "God bless for the all star game on the 14th ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163631795, "name": "Ragú", "screen_name": "mrigual10", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-07-06"), "description": "✨GOT TO G.O.A.T✨", "followers_count": 993, "friends_count": 429, "statues_count": 145770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618114937139200, "text": "Happy birthday THOT�� @ravenoxoxo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 842892127 }}, "user": { "id": 2600386916, "name": "sanaa✨", "screen_name": "_just_sanaa", "lang": "en", "location": "Berkeley, CA", "create_at": date("2014-07-02"), "description": "DREAMS TURN TO REALITY✨", "followers_count": 105, "friends_count": 93, "statues_count": 1471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618115008495617, "text": "������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 420125840, "name": "Demar Jackson™", "screen_name": "DJfromSoCal", "lang": "en", "location": "IG: DJfromSoCal", "create_at": date("2011-11-23"), "description": "Demar/DJ. Group Fitness Instructor. Personal Trainer. SanDiego✈️ #ASU", "followers_count": 666, "friends_count": 515, "statues_count": 28355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618115105058816, "text": "@brianna_norrell prob not. .", "in_reply_to_status": 692618006313222146, "in_reply_to_user": 298035515, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298035515 }}, "user": { "id": 1373733996, "name": "CJ ♴", "screen_name": "ThisIs_CJCobb", "lang": "en", "location": "Newtown ", "create_at": date("2013-04-22"), "description": "256. I AM. •OVO•", "followers_count": 1516, "friends_count": 244, "statues_count": 31407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville, AL", "id": "d5dbaf62e7106dc4", "name": "Jacksonville", "place_type": "city", "bounding_box": rectangle("-85.800502,33.752607 -85.724207,33.840282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 138272, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618115419652096, "text": "real nigga hour", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4099495738, "name": "shawtyy", "screen_name": "svgnatyy", "lang": "en", "location": "null", "create_at": date("2015-11-02"), "description": "#GIRLGANG || dancer", "followers_count": 808, "friends_count": 1192, "statues_count": 16283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.654929 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618115427864576, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 42837817, "name": "El Toro", "screen_name": "Cup661", "lang": "en", "location": "L.A.,Bakersfield,california", "create_at": date("2009-05-27"), "description": "null", "followers_count": 194, "friends_count": 178, "statues_count": 5620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618115465744385, "text": "Wind 0.0 mph ---. Barometer 30.125 in, Falling. Temperature 35.2 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116019331072, "text": "@ebbtideapp Tide in Sand Shoal Inlet, Virginia 01/28/2016\n Low 4:17am -0.2\nHigh 10:45am 4.0\n Low 4:50pm -0.0\nHigh 11:06pm 3.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.7783,37.3017"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 16, "friends_count": 1, "statues_count": 2863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51131, "countyName": "Northampton" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116111667200, "text": "Wind 0.0 mph SW. Barometer 30.078 in, Falling. Temperature 24.2 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116178661378, "text": "@kgc40anthony @rosemamba_1 @jay_knaux @_hamonza @raffilista https://t.co/el502gOJEV", "in_reply_to_status": 692617355562647552, "in_reply_to_user": 2581879207, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2581879207, 1450668570, 909847435, 2346417211, 2562174647 }}, "user": { "id": 1023077293, "name": "Aaron Baz", "screen_name": "iamaaronbaz", "lang": "en", "location": "Mission Hills, CA", "create_at": date("2012-12-19"), "description": "If I haven't passed you yet, watch me catch up now #loading...", "followers_count": 480, "friends_count": 301, "statues_count": 5949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116354805760, "text": "thank god moms gonna be ok ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242732246, "name": "LYS†", "screen_name": "ruthlesslys", "lang": "en", "location": "go away. ", "create_at": date("2015-06-11"), "description": "@imalolo_Thoo ❥", "followers_count": 244, "friends_count": 209, "statues_count": 506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116774400001, "text": "01/28@03:00 - Temp 21.6F, WC 21.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.143in, Falling slowly. Rain 0.00in. Hum 90%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 46588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116782788608, "text": "Wind 2.0 mph SSE. Barometer 29.943 in, Falling Rapidly. Temperature 42.3 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-28T00:00:01.000Z"), "id": 692618116799422464, "text": "Happy Birthday @JColeNC ����.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19028953 }}, "user": { "id": 615053719, "name": "selena", "screen_name": "YT_Selenaaa", "lang": "en", "location": "#DubNation ", "create_at": date("2012-06-22"), "description": "Andrew Ray Hernandez ❤️ YOURS TRULY! http://youtu.be/dxCkDOqTfLo", "followers_count": 1313, "friends_count": 540, "statues_count": 57363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618117189496832, "text": "Owners of @FoxNews are for open borders and immigration so they want to push a Kelly fight on to Trump. https://t.co/vRhmjacU28", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531 }}, "user": { "id": 621524387, "name": "Marilu", "screen_name": "rumfolo", "lang": "en", "location": "Spring Texas", "create_at": date("2012-06-28"), "description": "Native Texan, third generation immigrant second ammendment advocate, animal lover who believes that a strong military is necessary for peace #Trump2016", "followers_count": 1870, "friends_count": 1767, "statues_count": 33439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge North, TX", "id": "0180324e104dd7b5", "name": "Oak Ridge North", "place_type": "city", "bounding_box": rectangle("-95.452477,30.076095 -95.349469,30.173617") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4853190, "cityName": "Oak Ridge North" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618117781000192, "text": "I wish everything could be simple", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1692725743, "name": "purple ⚓️", "screen_name": "Yahboymatt", "lang": "en", "location": "Saturn", "create_at": date("2013-08-22"), "description": "I can see the stars from the top bunk", "followers_count": 696, "friends_count": 461, "statues_count": 65721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, NY", "id": "44453d2ae8e47fd5", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-73.853645,40.889695 -73.805123,40.935303") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3649121, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618117856436224, "text": "@era_kibn @lmaocrest get mics", "in_reply_to_status": -1, "in_reply_to_user": 2432578873, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2432578873, 3140900221 }}, "user": { "id": 4799057250, "name": "Hi Reckz", "screen_name": "Hireckz", "lang": "en", "location": "null", "create_at": date("2016-01-22"), "description": "#mw2 #comeback", "followers_count": 21, "friends_count": 44, "statues_count": 201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Novato, CA", "id": "3b0eea538598dc42", "name": "Novato", "place_type": "city", "bounding_box": rectangle("-122.62422,38.039015 -122.486658,38.147701") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 652582, "cityName": "Novato" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618118074470400, "text": "#BaeIn3Words \n\n\"Don't tell anyone\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BaeIn3Words" }}, "user": { "id": 2261867084, "name": "Laniola Tuione", "screen_name": "eyasw0rld", "lang": "en", "location": "null", "create_at": date("2013-12-25"), "description": "and this is my Twitter.", "followers_count": 143, "friends_count": 358, "statues_count": 4547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618118464692225, "text": "Temp: 40.1°F Wind:0.5mph Pressure: 30.121hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 23, "statues_count": 58281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618118468734976, "text": "I choose u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390431606, "name": "Justa©e.", "screen_name": "Ballin_Shibby", "lang": "en", "location": "Utah, USA", "create_at": date("2011-10-13"), "description": "19 yrs young. born in the 801, will die in the 801. SIKE.", "followers_count": 2866, "friends_count": 1404, "statues_count": 17375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Weber, UT", "id": "93b7643d8c54f6a7", "name": "South Weber", "place_type": "city", "bounding_box": rectangle("-112.004989,41.108787 -111.899558,41.153638") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4971180, "cityName": "South Weber" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618118917668864, "text": "#SupportOriginMelissa 19.0°F Wind:0.0mph Pressure: 30.03hpa Falling slowly Rain Today 0.00in. Forecast: Fairly fine, showery later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 308280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618119060287488, "text": "Knew it but didn't wanna believe it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544923615, "name": "kace", "screen_name": "KaceyRegester", "lang": "en", "location": "i just really miss clare ", "create_at": date("2012-04-03"), "description": "your opinion is invalid", "followers_count": 662, "friends_count": 611, "statues_count": 13100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apache Junction, AZ", "id": "bf09d4c99c2d845c", "name": "Apache Junction", "place_type": "city", "bounding_box": rectangle("-111.587098,33.378739 -111.469058,33.465988") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 402830, "cityName": "Apache Junction" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618119584575488, "text": "I ready to go blow but Ine going outside to smoke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155115676, "name": "Mansa Musa", "screen_name": "KingPratt95", "lang": "en", "location": "Bahamas", "create_at": date("2010-06-12"), "description": "IG: @bahamian_nightmare242 ; ||c/o 2014 || Live where you vacay Bahamas ☀️ #KingPratt #OsoArrogant #FTF the Movement", "followers_count": 1422, "friends_count": 1156, "statues_count": 58890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618119672520705, "text": "#GatorStrong #dubnation https://t.co/gl5a9suW9I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "GatorStrong", "dubnation" }}, "user": { "id": 229382850, "name": "Delisha Milton-Jones", "screen_name": "DelishaMJones", "lang": "en", "location": "null", "create_at": date("2010-12-21"), "description": "Forward for the WNBA Atlanta Dream. 2x Olympic Gold Medalist 2x WNBA Champion, 2xWNBA ALLSTAR,A Christian,Wife and Gator 4 life!! #LOVE", "followers_count": 7768, "friends_count": 1006, "statues_count": 16121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618119894831104, "text": "@Jeffrey_Horton2 a really good show lol", "in_reply_to_status": 692618023958482945, "in_reply_to_user": 304756458, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304756458 }}, "user": { "id": 375870409, "name": "Mars", "screen_name": "marigreene1", "lang": "en", "location": "somewhere out there", "create_at": date("2011-09-18"), "description": "wuddup stalkers |Instagram: mari_greene| R.I.P Da'Marcus Berry-Hodge", "followers_count": 276, "friends_count": 324, "statues_count": 19201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Grove, OR", "id": "26ae2ae9b73ca4d7", "name": "Oak Grove", "place_type": "city", "bounding_box": rectangle("-122.657008,45.397145 -122.614337,45.434625") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4153900, "cityName": "Oak Grove" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618120125485060, "text": "I'd be a really fast acting nurse but I can't take wounds or anything. Id be okay with meds but idk about blood and stuff.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35324004, "name": "alyssa marie", "screen_name": "lyssybizzle", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-25"), "description": "college. 22. love life.", "followers_count": 491, "friends_count": 205, "statues_count": 40169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618120410705921, "text": "temperature down 37°F -> 31°F\nhumidity up 65% -> 74%\nwind 5mph -> 6mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.70783,34.25741"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228218563, "name": "Tupelo Weather", "screen_name": "TupeloMS", "lang": "en", "location": "Tupelo, MS", "create_at": date("2010-12-18"), "description": "Weather updates, forecast, warnings and information for Tupelo, MS. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 121, "friends_count": 2, "statues_count": 20680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tupelo, MS", "id": "894f2ba528ba1c7d", "name": "Tupelo", "place_type": "city", "bounding_box": rectangle("-88.800213,34.203564 -88.656629,34.324583") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2874840, "cityName": "Tupelo" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618120561823744, "text": "https://t.co/INTy1znLHH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio 20k", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 20311, "friends_count": 17368, "statues_count": 7265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618121048301571, "text": "Guess I don't got good music", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460588375, "name": "Cozy Papi", "screen_name": "TheRealBubba_T", "lang": "en", "location": "Dallas / Los Angeles ", "create_at": date("2012-01-10"), "description": "L.I.V.E ) CEO of @NoirBoyz | $ix Co| For Features and Bookings email BubbaTmusic@yahoo.Com |MGMT @NOIR_JVY", "followers_count": 1725, "friends_count": 985, "statues_count": 59243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Highlands, CA", "id": "ee9341f90ab895d5", "name": "North Highlands", "place_type": "city", "bounding_box": rectangle("-121.424332,38.638355 -121.335815,38.704483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 651924, "cityName": "North Highlands" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618121107107840, "text": "Agreed https://t.co/kKZg3pTwLa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1174507368, "name": "La Rubia", "screen_name": "jaymnx0", "lang": "en", "location": "null", "create_at": date("2013-02-13"), "description": "I'm broke baby", "followers_count": 992, "friends_count": 533, "statues_count": 42976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smithtown, NY", "id": "8cd2e7740a60fd93", "name": "Smithtown", "place_type": "city", "bounding_box": rectangle("-73.270527,40.825276 -73.168645,40.895739") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3667851, "cityName": "Smithtown" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618121178251264, "text": "Google says Cardboard virtual reality viewer gaining traction https://t.co/8y7q1Rt4Fb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2787774091, "name": "Ximmerse", "screen_name": "Ximmerse", "lang": "en", "location": "Guangdong & Los Angeles", "create_at": date("2014-09-03"), "description": "Mobile visual computing and inertial tracking for virtual reality input devices. Apply for X-Hawk & X-Cobra Dev Kit: http://bit.ly/1KjJU9m", "followers_count": 179, "friends_count": 141, "statues_count": 97 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:02.000Z"), "id": 692618121207676930, "text": "40.3F (Feels: 40.3F) - Humidity: 77% - Wind: 8.3mph N - Gust: 10.7mph - Pressure: 1033.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 224354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618122176651264, "text": "https://t.co/G5LlxhPmUA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3087160226, "name": "Peter Shea", "screen_name": "Shayshea95", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Psn : Shayshea95 | Age : 19 | Competitive Gamer - ??? New Streamer\nhttp://t.co/9b3xb8rSvq\nhttp://t.co/irP0Y2QH28", "followers_count": 101, "friends_count": 120, "statues_count": 709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plattsburgh, NY", "id": "01951cce6978d067", "name": "Plattsburgh", "place_type": "city", "bounding_box": rectangle("-73.530573,44.652759 -73.420227,44.72545") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36019, "countyName": "Clinton", "cityID": 3658574, "cityName": "Plattsburgh" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618122646282241, "text": "So over your shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160777693, "name": "Erick Mac", "screen_name": "ateabigmac", "lang": "en", "location": "null", "create_at": date("2010-06-28"), "description": "null", "followers_count": 283, "friends_count": 274, "statues_count": 4500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618122654806016, "text": "I feel real pretty right now ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1911626479, "name": "BBW Tee", "screen_name": "YungNPoppin", "lang": "en", "location": "1234 Main St.", "create_at": date("2013-09-27"), "description": "Tia| 21.| That Crazy BITCH", "followers_count": 358, "friends_count": 198, "statues_count": 36542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504957,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618123204173825, "text": "���� https://t.co/ZjAiCmzTWK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 611185056, "name": "-madisonღ", "screen_name": "ShaddyLove_22", "lang": "en", "location": "Texas, USA", "create_at": date("2012-06-17"), "description": "ig & snapchat: maddypaige_22", "followers_count": 352, "friends_count": 193, "statues_count": 4767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon, TX", "id": "309059ff6710946f", "name": "Canyon", "place_type": "city", "bounding_box": rectangle("-101.954673,34.960525 -101.878133,35.017533") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48381, "countyName": "Randall", "cityID": 4812532, "cityName": "Canyon" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618124445773824, "text": "Feel like a fuck up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1235064174, "name": "MILLER LITE", "screen_name": "M_Aaron11", "lang": "en", "location": "Clarksville, TN", "create_at": date("2013-03-02"), "description": "@BrookAnnoelle ❤️ | GBO | APSU19 | Jeremiah 29:11", "followers_count": 169, "friends_count": 310, "statues_count": 2969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, TN", "id": "52f4a98d03e4ea4f", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-87.486546,36.4602 -87.157791,36.641767") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47125, "countyName": "Montgomery", "cityID": 4715160, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618124533706752, "text": "Will the content in the beta be different than what Xbox One players saw in the Alpha?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1411886502, "name": "Ashton Arnold", "screen_name": "OrbitalNexus", "lang": "en", "location": "Sacramento, CA", "create_at": date("2013-05-07"), "description": "Orbital Nexus---Competitive Gamer---Streamer---Community Owner http://www.Twitch.tv/OrbitalNexuss", "followers_count": 54, "friends_count": 253, "statues_count": 315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-01-28T00:00:03.000Z"), "id": 692618125339037696, "text": "Woman who lost her leg in Boston Marathon bombing to run the race https://t.co/fafaGDHlco", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17654, "friends_count": 17579, "statues_count": 67673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618126337380352, "text": "He do this every time ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119234929, "name": "Fee", "screen_name": "prettyfee88", "lang": "en", "location": "null", "create_at": date("2010-03-02"), "description": "null", "followers_count": 461, "friends_count": 505, "statues_count": 24689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618127419379712, "text": "when you get roasted and this is all you can come up with https://t.co/QESVskOLLj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93763260, "name": "carølinekillem'", "screen_name": "c_always_wright", "lang": "en", "location": "SATX / Denton", "create_at": date("2009-11-30"), "description": "spurs nation - denver broncos - university of north texas - ΔΓ - NT40 - broadcast journalism - mean green game day production crew - crocs", "followers_count": 2431, "friends_count": 1263, "statues_count": 60652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618127671164928, "text": "Travelling to Salt Lake City or just twittering about Salt Lake City? https://t.co/RdUMWUSA9q #Salt Lake City", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.891,40.7608"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Salt" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1021, "friends_count": 312, "statues_count": 2555845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128036085760, "text": "@SuplexDaddy cet", "in_reply_to_status": 692617989024124928, "in_reply_to_user": 595133308, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 595133308 }}, "user": { "id": 633144541, "name": "YG MAL.", "screen_name": "Kjm14_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-07-11"), "description": "19 years in the trenches. \n #Savage$zn |TXST#14|", "followers_count": 2352, "friends_count": 1858, "statues_count": 74946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128094695424, "text": "Forget football, I don't think I can really marry someone who can't ball��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317520135, "name": "laurenL E F R A N DT", "screen_name": "LaurenLeffy5", "lang": "en", "location": "null", "create_at": date("2011-06-14"), "description": "nah", "followers_count": 261, "friends_count": 246, "statues_count": 4857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Provo, UT", "id": "665409336d2b20a0", "name": "Provo", "place_type": "city", "bounding_box": rectangle("-111.740949,40.18565 -111.602629,40.311641") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4962470, "cityName": "Provo" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128094793734, "text": "My dad legit asked me today \" when are you getting married & having kids\" ZAMMMM a bitch don't even got a man ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1449666398, "name": "B", "screen_name": "RealbMogul", "lang": "en", "location": "Miami, FL", "create_at": date("2013-05-22"), "description": "Cuban/Entrepreneur/ Aesthetician /Tattoo Collector / Aux Cord Lord/Miami / CAMEO VIP/ god & faith is everything / all business no play", "followers_count": 1671, "friends_count": 299, "statues_count": 10734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128195457024, "text": "I'm moving out of state Arkansas Ain't got me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4745299074, "name": "Pretty_fancyy", "screen_name": "pretty_nPaiid", "lang": "en", "location": "Little Rock, AR", "create_at": date("2016-01-11"), "description": "Child Of God . Young Beautiful College Student . Money Motivated. #ǤvldenҒɑmeƎmpıre ✨", "followers_count": 247, "friends_count": 429, "statues_count": 169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128233082881, "text": "@princessedestin ugg glass", "in_reply_to_status": 692617329494982656, "in_reply_to_user": 2948490588, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2948490588 }}, "user": { "id": 2697361940, "name": "EARTHLY♻️", "screen_name": "earthlydevon", "lang": "en", "location": " Long Beach , CA", "create_at": date("2014-07-31"), "description": "#earthlythemovement ♻️", "followers_count": 1551, "friends_count": 1184, "statues_count": 23001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128329551872, "text": "can i review the 5th wave", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 124277900, "name": "Aaron Zabriskie", "screen_name": "zabriskie16", "lang": "en", "location": "West Valley City UT", "create_at": date("2010-03-18"), "description": "PSN :KISSROCKER Instagram zabriskie19 Hi Im Aaron Zabriskie\n\n@Lauryn_Lucero Is My Bestfriend", "followers_count": 1516, "friends_count": 2971, "statues_count": 28821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618128849813504, "text": "Wind 0.0 mph ---. Barometer 30.015 in, Falling Rapidly. Temperature 12.1 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 8235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-01-28T00:00:04.000Z"), "id": 692618129189437441, "text": "@ChauntelSpeaks_ �� you not even blocked smh lol", "in_reply_to_status": 692617890076360704, "in_reply_to_user": 246636554, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246636554 }}, "user": { "id": 302762516, "name": "Mr. Unpredictable ..", "screen_name": "KingSeun24", "lang": "en", "location": "Denton, TX", "create_at": date("2011-05-21"), "description": "Future CMO | Your PHavorite Alpha Man | ΗΕ | #UNT | Nigerian", "followers_count": 4292, "friends_count": 2164, "statues_count": 210872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618129789181952, "text": "@tomilynn14 is she ever omg", "in_reply_to_status": 692618074835398656, "in_reply_to_user": 336135050, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 336135050 }}, "user": { "id": 3356174429, "name": "yosemite sam", "screen_name": "samlynntorres", "lang": "en", "location": "Lafayette, LA", "create_at": date("2015-07-02"), "description": "20 • blakely marie's mommy • snapchat/instagram: samlynntorres", "followers_count": 408, "friends_count": 325, "statues_count": 4330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618129831100416, "text": "Drinking on lean got me lazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417615011, "name": "Daddy De$", "screen_name": "214LilDesmond", "lang": "en", "location": "DTX ", "create_at": date("2011-11-20"), "description": "The Wolf of 4 Street #SHSU", "followers_count": 1961, "friends_count": 892, "statues_count": 56866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618130162450432, "text": "Mike Jones !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1705866631, "name": "Isaac Mbuyamba", "screen_name": "Ice_Man08", "lang": "en", "location": "That Guy Just By ", "create_at": date("2013-08-27"), "description": "Ike is just Right ,Here For a reason ,Kennesaw State University Signee,@itsallmeee , #IPromise", "followers_count": 584, "friends_count": 556, "statues_count": 2516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618130527555584, "text": "@ashish30sharma Please, does not speak Hindi Blangelazy Ki Mjabec occur around the world being seen", "in_reply_to_status": -1, "in_reply_to_user": 98068972, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 98068972 }}, "user": { "id": 637293358, "name": "Genan Mou Stafa", "screen_name": "tyahya59yahoo", "lang": "ar", "location": "Los Angeles, TX", "create_at": date("2012-07-16"), "description": "null", "followers_count": 12, "friends_count": 83, "statues_count": 8 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, TX", "id": "019940ae78c7b3bc", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-97.791748,26.492566 -97.780997,26.496485") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48489, "countyName": "Willacy", "cityID": 4844062, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618130539974657, "text": "Temp: 31.7°F - Dew Point: 25.6° - Wind: 18.7 mph - Gust: 31.8 - Rain Today: 0.00in. - Pressure: 29.63in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618130774884352, "text": "You gottttt meeee fuckkeeddd upppp������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 706599732, "name": "SUSHI", "screen_name": "cleeclee7810", "lang": "en", "location": "Surprise, AZ", "create_at": date("2012-07-19"), "description": "it's just me, myself & I, solo ride until I die, cause I got me for life ♥️", "followers_count": 117, "friends_count": 446, "statues_count": 5538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618131320217600, "text": "I guess really in the end it's just Me, Myself, and I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453933565, "name": "Kordell Chaney", "screen_name": "K_Chaneyy", "lang": "en", "location": "null", "create_at": date("2012-01-03"), "description": ".I asked God for strength, and God gave me more difficulties to make me stronger. https://http://www.hudl.com/athlete/1864185/highlights/309080384 #RIPBLUE", "followers_count": 1822, "friends_count": 1415, "statues_count": 38240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandusky, OH", "id": "fcc54ac4568181a4", "name": "Sandusky", "place_type": "city", "bounding_box": rectangle("-82.78773,41.409372 -82.617802,41.492373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39043, "countyName": "Erie", "cityID": 3970380, "cityName": "Sandusky" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618131601227776, "text": "@chaweeze thank you! ��", "in_reply_to_status": 692501545414455296, "in_reply_to_user": 40833726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40833726 }}, "user": { "id": 38157639, "name": "We-we", "screen_name": "thisiswewe", "lang": "en", "location": "USA", "create_at": date("2009-05-06"), "description": "Work Hard, Vacation Harder \n#wanderlust #travelbug", "followers_count": 51, "friends_count": 150, "statues_count": 13656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Celebration, FL", "id": "01bbe9ba4078361c", "name": "Celebration", "place_type": "city", "bounding_box": rectangle("-81.577489,28.276141 -81.521152,28.332902") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1211285, "cityName": "Celebration" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618131869597696, "text": "I also hope he doesn't debate Cruz, either. Cruz needs Trump but Trump doesn't need Cruz!\n#Trump2016 https://t.co/ANKDNZ0HtT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump2016" }}, "user": { "id": 1922383902, "name": "Renee W", "screen_name": "lakwita2", "lang": "en", "location": "Northern California", "create_at": date("2013-09-30"), "description": "Married, working mother of four, concerned about direction of our country, watching 2016 GOP race, Christian conservative #CCOT #Trump2016", "followers_count": 959, "friends_count": 1269, "statues_count": 4624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618132331040768, "text": "Chris Melton-Burke<<<< #Rt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hu", "is_retweet": false, "hashtags": {{ "Rt" }}, "user": { "id": 324247445, "name": "Chris Melton-Burke", "screen_name": "MeltonBurke", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-06-26"), "description": "My first name is Chris not Melton | Ohio University | Insta/Snap-Meltonburke | Future CPA |", "followers_count": 1443, "friends_count": 1247, "statues_count": 21099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, OH", "id": "77aa8a0640caa98c", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-82.173479,39.27973 -82.01933,39.361038") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39009, "countyName": "Athens", "cityID": 3902736, "cityName": "Athens" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618132343644160, "text": "https://t.co/MYbQeuAqAL \n #Trump #Trump2016 #trumpkelly #kellytrump #MeganKelly #MeganKellydebatequestions #foxnews https://t.co/5nOQBdPAHp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Trump", "Trump2016", "trumpkelly", "kellytrump", "MeganKelly", "MeganKellydebatequestions", "foxnews" }}, "user": { "id": 3116619775, "name": "SeeMyInside.com", "screen_name": "SeeMyInside", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2015-03-29"), "description": "New Very Convenient Social MultiMedia Advertising Platform for You, Your Business and Your Partner's Business Together By: The Most Powerful Domain in History!", "followers_count": 835, "friends_count": 30, "statues_count": 28590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618132884619265, "text": "January 28th // @JColeNC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19028953 }}, "user": { "id": 1730704154, "name": "Alizé", "screen_name": "Alizexo69", "lang": "en", "location": "Hell Centro", "create_at": date("2013-09-04"), "description": "Fuck y'all all of y'all", "followers_count": 776, "friends_count": 1317, "statues_count": 11286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Centro, CA", "id": "00436a376f214be7", "name": "El Centro", "place_type": "city", "bounding_box": rectangle("-115.594987,32.730647 -115.501098,32.818522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 621782, "cityName": "El Centro" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618132985245696, "text": "I'm so annoyed some give me there Netflix password", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1393343756, "name": "Haley Reardon", "screen_name": "hwmksr", "lang": "en", "location": "Hawaii, USA", "create_at": date("2013-04-30"), "description": "null", "followers_count": 303, "friends_count": 227, "statues_count": 9336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipio, HI", "id": "00b9f0520832fe1b", "name": "Waipio", "place_type": "city", "bounding_box": rectangle("-158.026656,21.397631 -157.98498,21.458421") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579860, "cityName": "Waipio" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618133006262274, "text": "@_TheHeezyLife ��❤️ https://t.co/kdivYkkGJd", "in_reply_to_status": -1, "in_reply_to_user": 922761175, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 922761175 }}, "user": { "id": 111230280, "name": "b", "screen_name": "barbarajayy", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-02-03"), "description": "fashion student | creator of #drawmyootd ✨", "followers_count": 317, "friends_count": 212, "statues_count": 19000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618133266370560, "text": "Everytime Wiz types a word with “cc” I keep forgetting he’s crip affiliated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47705062, "name": "لآ", "screen_name": "BlxxdyB", "lang": "en", "location": "null", "create_at": date("2009-06-16"), "description": "null", "followers_count": 2014, "friends_count": 995, "statues_count": 106827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-01-28T00:00:05.000Z"), "id": 692618133631143936, "text": "@_omarm58 @xoxo_karinaa_ fuckboi lol", "in_reply_to_status": 692618094703849472, "in_reply_to_user": 1414192818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2391927829, 1595043444 }}, "user": { "id": 1414192818, "name": "Garcia lll ⚡️", "screen_name": "GarciaGiggles6", "lang": "en", "location": "Hawaii ", "create_at": date("2013-05-08"), "description": "#NoFlyZone. #6", "followers_count": 491, "friends_count": 498, "statues_count": 47016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618134512074753, "text": "Wind 1.4 mph S. Barometer 30.048 in, Falling. Temperature 23.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 6, "friends_count": 43, "statues_count": 6109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618134788804608, "text": "@PaytonRCTID omg!!", "in_reply_to_status": 692618089842642948, "in_reply_to_user": 2891808205, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2891808205 }}, "user": { "id": 3169419788, "name": "tatiana", "screen_name": "sadgirltato", "lang": "en", "location": "Chandler, AZ", "create_at": date("2015-04-23"), "description": "don't be mad", "followers_count": 547, "friends_count": 528, "statues_count": 13278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618135107534849, "text": "@NellyMorales_ ��������?", "in_reply_to_status": 692617929762873344, "in_reply_to_user": 785531125, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 785531125 }}, "user": { "id": 163970357, "name": "Andrew Gallegos", "screen_name": "A_Gallegos4", "lang": "en", "location": "La Habra, CA", "create_at": date("2010-07-07"), "description": "Light up the Darkness", "followers_count": 486, "friends_count": 389, "statues_count": 20975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618135233388544, "text": "Same https://t.co/EnpNNRPcvc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2929784020, "name": "Alexandrea✌️", "screen_name": "food_nipple", "lang": "en", "location": "Vancouver, WA", "create_at": date("2014-12-18"), "description": "• Im hella destroying my life but its lit • pizza is my 2nd favorite thing to eat in bed • @bhxla ← mine • I like smelling breas armpits • literally so gay •", "followers_count": 582, "friends_count": 445, "statues_count": 19043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618135858339840, "text": "Late night Os", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2903219940, "name": "the_killabox", "screen_name": "Jacob_box100", "lang": "en", "location": "null", "create_at": date("2014-11-17"), "description": "⚡COHS⚡", "followers_count": 324, "friends_count": 319, "statues_count": 1045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618136265199621, "text": "going to watch inside out and eat soup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197603449, "name": "Kxzekage", "screen_name": "Juliaaaan__", "lang": "en", "location": "Compton", "create_at": date("2010-10-01"), "description": "| Julian Gutierrez | What a drag | Level 21", "followers_count": 101, "friends_count": 163, "statues_count": 9075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, CA", "id": "7df6f50f15138f28", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-118.282262,33.901902 -118.222378,33.929527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685614, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618136537837568, "text": "Fuck you @EASPORTSFIFA you call this goalkeeping? https://t.co/U3mlc2LInn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 105297123 }}, "user": { "id": 528060524, "name": "John Savedra", "screen_name": "Pimpjayyyyyy", "lang": "en", "location": "null", "create_at": date("2012-03-17"), "description": "Lakewood, ca", "followers_count": 96, "friends_count": 72, "statues_count": 7774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-28T00:00:06.000Z"), "id": 692618137531842560, "text": "Its Midnite PST n Im still me.... #Yaaaaaaaaaaaaaaaaay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Yaaaaaaaaaaaaaaaaay" }}, "user": { "id": 183317382, "name": "Keith Thomas Roberts", "screen_name": "alreadytakeni", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-08-26"), "description": "It ain't easy.\nMyEyesInstantlykeithroberts8122\nahhhsnapkeronesea1", "followers_count": 60, "friends_count": 246, "statues_count": 1632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618138228162560, "text": "This is so attractive to me I swear �������� https://t.co/I1fVjWHzyd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2768396234, "name": "Lex❤", "screen_name": "_alea123", "lang": "en", "location": "null", "create_at": date("2014-08-25"), "description": "Don't be reckless with other peoples' hearts. Don't put up with people who are reckless with yours", "followers_count": 275, "friends_count": 502, "statues_count": 9202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618139444645889, "text": "Things were soo much simpler as a little kid.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1965342896, "name": "Mike and Ike", "screen_name": "mikoboi365", "lang": "en", "location": "Chicago,Illinois", "create_at": date("2013-10-16"), "description": "Seest thou a man diligent in his business? he shall stand before kings; He shall not stand before mean men. Proverbs 22:29| BORICUA!!| UIUC '17| ΩΔ", "followers_count": 386, "friends_count": 267, "statues_count": 6768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618139541094400, "text": "@jojochazs__ @Jodafuck_ deek gent", "in_reply_to_status": 630283647829024768, "in_reply_to_user": 3226034553, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3237670503, 4044576441 }}, "user": { "id": 3226034553, "name": "white iverson", "screen_name": "rockydavids_1", "lang": "en", "location": "Greenwich Village, Manhattan", "create_at": date("2015-05-01"), "description": "MADE IN HEAVEN", "followers_count": 161, "friends_count": 177, "statues_count": 6394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618139562086400, "text": "@yesmarkvalley thanks for the follow, Mark. #ZeroDarkThirty", "in_reply_to_status": -1, "in_reply_to_user": 416885646, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ZeroDarkThirty" }}, "user_mentions": {{ 416885646 }}, "user": { "id": 15219770, "name": "Dan Gabriel", "screen_name": "danpgabriel", "lang": "en", "location": "Boston Baghdad Bali", "create_at": date("2008-06-24"), "description": "ex-#CIA. Insurgencies, foreign and domestic. TDY #IowaCaucus", "followers_count": 98772, "friends_count": 59119, "statues_count": 8705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, VA", "id": "319ee7b36c9149da", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-77.172219,38.827378 -77.031779,38.934311") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51013, "countyName": "Arlington", "cityID": 5103000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618139578662912, "text": "Updated: SDMTS GTFS (San Diego, CA, USA) https://t.co/AY0NLMaEIq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.157255,32.715329"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2148084618, "name": "TransitFeeds", "screen_name": "TransitFeeds", "lang": "en", "location": "Everywhere", "create_at": date("2013-10-21"), "description": "The best source of open official public transit data. Handcrafted in Australia by @HendX", "followers_count": 179, "friends_count": 63, "statues_count": 14418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618140023287808, "text": "Before walking across the Hawthorne Bridge, there is a gateway of sorts to contend with. @… https://t.co/lc3RrhS3Lj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.682,45.52"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28387022, "name": "Will Wright", "screen_name": "willwrightreads", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-02"), "description": "As Los Angeles' chief advocate for better architecture and urban design, I help bring people together to solve problems and set priorities.", "followers_count": 1708, "friends_count": 2014, "statues_count": 8279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618140153327617, "text": "@NelllyNellllz Happy Belated Birthday Youngin'. Sorry it's late but I hope you had a great one. ������", "in_reply_to_status": -1, "in_reply_to_user": 3309101294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3309101294 }}, "user": { "id": 2303604534, "name": "Ⓜ️ally™", "screen_name": "Ayo_Mally", "lang": "en", "location": "Arkansas ✈️Kansas✈️ Oklahoma", "create_at": date("2014-01-21"), "description": "6'5 College Athlete @NWOSU |Snapchat: IStaySnappin", "followers_count": 883, "friends_count": 845, "statues_count": 14156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Searcy, AR", "id": "4cc70ca050b304d2", "name": "Searcy", "place_type": "city", "bounding_box": rectangle("-91.800548,35.207037 -91.661324,35.26961") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5145, "countyName": "White", "cityID": 563020, "cityName": "Searcy" } }
+{ "create_at": datetime("2016-01-28T00:00:07.000Z"), "id": 692618140283330560, "text": "@jbso_major @2cups_NOCakes @Juicyy_Jayeee lmao ����", "in_reply_to_status": 692617987979886593, "in_reply_to_user": 365715424, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 365715424, 4831418015, 3010244801 }}, "user": { "id": 968489022, "name": ".. Myiesha✨❤️", "screen_name": "__PrettyyOG", "lang": "en", "location": "Mississippi ", "create_at": date("2012-11-24"), "description": "null", "followers_count": 1228, "friends_count": 835, "statues_count": 12605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southaven, MS", "id": "0141622e000a64d2", "name": "Southaven", "place_type": "city", "bounding_box": rectangle("-90.096994,34.889384 -89.918584,34.994979") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28033, "countyName": "DeSoto", "cityID": 2869280, "cityName": "Southaven" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980497312514048, "text": "@jxst_alxner lmao ight I'll be the 1 to blame ��", "in_reply_to_status": 692980295587479557, "in_reply_to_user": 2171624313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2171624313 }}, "user": { "id": 331916965, "name": "#FreebandzTaliban", "screen_name": "BalloutWebbie", "lang": "en", "location": "BALLOUTWORLD w/ Money & Nem", "create_at": date("2011-07-08"), "description": "Free The Guys Rip The Guys #SSD #NLMB #OTF #2FFE #MurdaTeam", "followers_count": 1167, "friends_count": 989, "statues_count": 104377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980497341874176, "text": "@Ktaylor_m I know just 2 more weeks and I'm getting drunk as a skunk", "in_reply_to_status": 692980386968772612, "in_reply_to_user": 153187421, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 153187421 }}, "user": { "id": 2318357988, "name": "Mr.Saucy", "screen_name": "ThatSaucyKidd", "lang": "en", "location": "null", "create_at": date("2014-01-29"), "description": "RichCity,Ca where the people aint really rich Instagram: @thatsaucykidd #RipUli #RipDamont", "followers_count": 533, "friends_count": 606, "statues_count": 10275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980497362870274, "text": "Wind 0.0 mph ---. Barometer 1019.58 mb, Steady. Temperature 32.7 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980497371238401, "text": "S/o to oomf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1330832328, "name": "daf", "screen_name": "daafneee_", "lang": "en", "location": "null", "create_at": date("2013-04-05"), "description": "null", "followers_count": 720, "friends_count": 497, "statues_count": 26193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980497853607942, "text": "Lmao on life https://t.co/y4QK73D9Rw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3241806848, "name": "Lerual✨", "screen_name": "Nigerian__InMe", "lang": "en", "location": "Akwa Ibom, TX", "create_at": date("2015-06-10"), "description": "#AfroLuxeHair| Insta/SC: ayoshenigerian Dallas/PV/Nac", "followers_count": 1287, "friends_count": 631, "statues_count": 36978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.804797,33.137357 -96.595915,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980498247892993, "text": "Same https://t.co/hsaQ7bs6Bb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4183728857, "name": "Spiderson", "screen_name": "jefehoops", "lang": "en", "location": "South Dallas, Tx", "create_at": date("2015-11-13"), "description": "Craig Wrenn MVC Basketball Commit 6'3 Combo Guard with the KillSwitch.", "followers_count": 487, "friends_count": 270, "statues_count": 12630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980498445025280, "text": "the fact that i know there's someone out there more perfect for me than you gives me peace knowing we didn't work out.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1445357382, "name": "kellz$", "screen_name": "jaymeekellz", "lang": "en", "location": "sacramento", "create_at": date("2013-05-20"), "description": "keep it real or keep it movin", "followers_count": 217, "friends_count": 75, "statues_count": 8038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980498688249856, "text": "\"I'm just gonna go live under a bridge and ask people riddles before they cross..\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427830627, "name": "Chris Hennessy ツ", "screen_name": "chrishennessy26", "lang": "en", "location": "null", "create_at": date("2011-12-03"), "description": "Smile fool", "followers_count": 445, "friends_count": 393, "statues_count": 12078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Run, VA", "id": "0114d40e50cfb98b", "name": "South Run", "place_type": "city", "bounding_box": rectangle("-77.298364,38.733527 -77.250532,38.766662") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5174120, "cityName": "South Run" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980498960875520, "text": "#goldcrest51 and #goldschlager may not be the best late-night… https://t.co/fzfesQz7cE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.04893,35.1542"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "goldcrest51", "goldschlager" }}, "user": { "id": 3070595219, "name": "Goldcrest Brewing Co", "screen_name": "GoldcrestBeer", "lang": "en", "location": "Memphis, TN", "create_at": date("2015-03-04"), "description": "the Original Memphis Beer!", "followers_count": 302, "friends_count": 163, "statues_count": 74 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980499074191361, "text": "Wind 0.0 mph ---. Barometer 30.047 in, Steady. Temperature 31.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980499527176192, "text": "Just one of those nights..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178292087, "name": "k.", "screen_name": "kamaka_96761", "lang": "en", "location": ".480", "create_at": date("2010-08-14"), "description": "maui woweeeee", "followers_count": 332, "friends_count": 268, "statues_count": 1575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980499560730624, "text": "@rickymosess I'm saying it's easier to beat 20 people than 20,000", "in_reply_to_status": 692977043454173184, "in_reply_to_user": 76840384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 76840384 }}, "user": { "id": 2779021333, "name": "Jerry Colvin", "screen_name": "GeraldColvin73", "lang": "en", "location": "Pittsfield, MA", "create_at": date("2014-08-29"), "description": "Husband, Father, Bar Owner, #DFS Junkie and writer at http://www.fantasyalarm.com & http://www.socalledfantasyexperts.com member of @FSWA", "followers_count": 869, "friends_count": 596, "statues_count": 5191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsfield, MA", "id": "c039a8f639292fcf", "name": "Pittsfield", "place_type": "city", "bounding_box": rectangle("-73.331377,42.402214 -73.186048,42.499408") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25003, "countyName": "Berkshire", "cityID": 2553960, "cityName": "Pittsfield" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980499602608128, "text": "#ProductMgmt #Job in #Boston, MA: UX Designer at Continuum Managed Services https://t.co/yNjOuWsRAQ #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0573421,42.3539011"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ProductMgmt", "Job", "Boston", "Jobs", "Hiring" }}, "user": { "id": 3431841232, "name": "Jobs at Continuum", "screen_name": "JobsAtContinuum", "lang": "en", "location": "null", "create_at": date("2015-08-19"), "description": "null", "followers_count": 84, "friends_count": 42, "statues_count": 139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191505,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980499984293888, "text": "Wind 0 mph --. Barometer 30.01 in, Steady. Temperature 43.0 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 13, "friends_count": 4, "statues_count": 20507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980500206583808, "text": "Well my mouth is sore BUT it's SOOOOOO worth it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3028045231, "name": "Jordan Elmore", "screen_name": "djangobitch1", "lang": "en", "location": "Imperial, Ca", "create_at": date("2015-02-18"), "description": "don't give someone your all when they don't do the same.", "followers_count": 605, "friends_count": 731, "statues_count": 10290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, CA", "id": "e6aaf7cb58e7857c", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-115.594963,32.817908 -115.544378,32.87646") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 636280, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980500315660290, "text": "Carson, CA �� https://t.co/r5Q91KjfT6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 218044016, "name": "Anniiibone", "screen_name": "LianiAupiu_", "lang": "en", "location": "null", "create_at": date("2010-11-20"), "description": "CARSON NIGGA WIT AN ATTITUDE", "followers_count": 762, "friends_count": 514, "statues_count": 30416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-01-29T00:00:00.000Z"), "id": 692980500621873152, "text": "@mattymathias I saw Whooptie and Casey tonight. Love them both. We all need to see each other more.", "in_reply_to_status": 692980326621134848, "in_reply_to_user": 533647754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 533647754 }}, "user": { "id": 434554760, "name": "Colby Braton", "screen_name": "BratonColby", "lang": "en", "location": "Barnesville➡️Moorhead➡️Fargo", "create_at": date("2011-12-11"), "description": "I enjoy Nickelback. MSUM. SC; colbybraton09", "followers_count": 1654, "friends_count": 1354, "statues_count": 8020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fargo, ND", "id": "5c8fea63bc3faa5b", "name": "Fargo", "place_type": "city", "bounding_box": rectangle("-96.903853,46.787881 -96.756332,46.941884") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38017, "countyName": "Cass", "cityID": 3825700, "cityName": "Fargo" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980500881920000, "text": "@mmariahmmariah lol I'm marketing outchea", "in_reply_to_status": 692980112032141312, "in_reply_to_user": 2891417208, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2891417208 }}, "user": { "id": 1014379171, "name": "The Tweet Deacon", "screen_name": "Trill_Soliloquy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-12-15"), "description": "Twitter Deacon Trentt, Doing Twitter Deacon Shit. Eat My Twitter Deacon Dick, As I Twitter Deacon Spit.", "followers_count": 1733, "friends_count": 839, "statues_count": 215667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980501129359361, "text": "I feel older already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251330805, "name": "Spencer Pregal", "screen_name": "Spregal96", "lang": "en", "location": "null", "create_at": date("2011-02-12"), "description": "Criminology major at SOU and I play some lacrosse too", "followers_count": 468, "friends_count": 655, "statues_count": 3191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980501204836352, "text": "Sleepless nights", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1015599559, "name": "Liz", "screen_name": "lizzie_roncelli", "lang": "en", "location": "cali", "create_at": date("2012-12-16"), "description": "snapchat:lizatron44", "followers_count": 528, "friends_count": 77, "statues_count": 12241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980501271953408, "text": "If you're not happy change your mentality the rest will follow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293057513, "name": "پدرام", "screen_name": "dennisdeko", "lang": "en", "location": "null", "create_at": date("2011-05-04"), "description": "Houston | Cancer | Persian", "followers_count": 356, "friends_count": 332, "statues_count": 4375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980501607518210, "text": "I think I picked a good one this time.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 435505504, "name": "jDf.❤️", "screen_name": "ChiLing__", "lang": "en", "location": "len's bed", "create_at": date("2011-12-12"), "description": "a blunt a day, keeps the stress a way~ #FreeVal !", "followers_count": 602, "friends_count": 676, "statues_count": 16442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980502165377024, "text": "01/29@03:00 - Temp 28.7F, WC 28.7F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.604in, Falling. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 46612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980502173736960, "text": "I LITERALLY JUST TURNED IN MY PAPER THATS DUE AT MIDNIGHT IM SO HAPPY ITS DONE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932958076, "name": "Megan Taylor", "screen_name": "Meganntaylorr3", "lang": "en", "location": "Manhattan Beach, CA", "create_at": date("2013-10-03"), "description": "null", "followers_count": 95, "friends_count": 206, "statues_count": 864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980502563786752, "text": "Perks of dorning at Beachside:\n\nthe donut shop down the street that is open 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 601290195, "name": "Kimberly", "screen_name": "kimpletly_n_luv", "lang": "en", "location": "null", "create_at": date("2012-06-06"), "description": "18...denair✈long beach...csu long beach\nmatte black w/ a hint of gold", "followers_count": 270, "friends_count": 197, "statues_count": 3807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980502844829696, "text": "@tiAXClUSiVE @CallMeGunz disrespect ?", "in_reply_to_status": 692980394220765184, "in_reply_to_user": 95255268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95255268, 27248894 }}, "user": { "id": 95250956, "name": "V E E", "screen_name": "THEinfamousVEE", "lang": "en", "location": "FROM jERSEY ✈ TO MiAMi", "create_at": date("2009-12-07"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀welcome to my side of life , ⠀⠀⠀⠀⠀⠀⠀& a bitch cant tell me i aint vibin right .", "followers_count": 5662, "friends_count": 603, "statues_count": 55389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980503171981312, "text": "Thank you ❤️ https://t.co/Fan7PBPuUD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343674075, "name": "† K A Y 01/29", "screen_name": "LadyK_Bomb", "lang": "en", "location": "Dallas  San Antonio", "create_at": date("2011-07-27"), "description": "† UTSA17!! The only man I'm living for is God. Family is everything! Love you Leah.Under Construction ⚠️", "followers_count": 1098, "friends_count": 654, "statues_count": 56267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980503444602880, "text": "@ebbtideapp Tide in Duck Island, South Carolina 01/29/2016\n Low 5:30am 0.3\nHigh 11:36am 5.1\n Low 5:46pm 0.3\nHigh 12:13am 4.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-79.9667,32.83"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 18, "friends_count": 1, "statues_count": 3144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Carolina, USA", "id": "6057f1e35bcc6c20", "name": "South Carolina", "place_type": "admin", "bounding_box": rectangle("-83.353955,32.04683 -78.499301,35.215449") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4513330, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980503889211394, "text": "Madi Morgan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 322485431, "name": "Slam Daniels", "screen_name": "sampauldan", "lang": "en", "location": "Tempe", "create_at": date("2011-06-23"), "description": "ΛΧΑ | Arizona State | ΑΧΩ Omega Man", "followers_count": 923, "friends_count": 598, "statues_count": 5908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980504031842304, "text": "Wind 0.0 mph ---. Barometer 29.949 in, Steady. Temperature 30.8 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-29T00:00:01.000Z"), "id": 692980504568713219, "text": ":')", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2444694847, "name": "MISSEDTHEMETERp", "screen_name": "Gohomepriscilla", "lang": "en", "location": "ill , ca ", "create_at": date("2014-04-14"), "description": "Sirius", "followers_count": 175, "friends_count": 245, "statues_count": 13022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980505248145409, "text": "Wind 1.0 mph WNW. Barometer 29.837 in, Rising slowly. Temperature 36.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980505420156928, "text": "@younggmia DEAD ������", "in_reply_to_status": 692980391607693313, "in_reply_to_user": 2967788708, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2967788708 }}, "user": { "id": 2903876412, "name": "Briss⛽️", "screen_name": "BrandonBrisson", "lang": "en", "location": "off the map", "create_at": date("2014-11-18"), "description": "Only Eat With The Ones You Starved With", "followers_count": 483, "friends_count": 553, "statues_count": 1398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980505730482177, "text": "Temp: 36.7°F Wind:0.0mph Pressure: 30.043hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980505793445888, "text": "SOBOSS REMINDER. ALL SOPHOMORES WEAR ORANGE.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1203168036, "name": "teddy fe-nce", "screen_name": "ahmaogaksweeney", "lang": "en", "location": "null", "create_at": date("2013-02-20"), "description": "yo", "followers_count": 365, "friends_count": 286, "statues_count": 2134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980505906651137, "text": "Culpa Al Corazón. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 518172714, "name": "Dominiqueeeeee", "screen_name": "dominiquegonz13", "lang": "en", "location": "Hammond, IN", "create_at": date("2012-03-07"), "description": "open up your heart.", "followers_count": 281, "friends_count": 157, "statues_count": 20576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, IN", "id": "99700cd6fc455c13", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-87.525341,41.566265 -87.432288,41.710116") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1831000, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980506267369474, "text": "Rs a lot of females don't like me cause they been done thought they took my man but when I come home they get dumped & dogged so dumb lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 822349543, "name": "D.", "screen_name": "_Dollzz", "lang": "en", "location": "Miami,Fl", "create_at": date("2012-09-13"), "description": "sc: lifesizedollz", "followers_count": 2749, "friends_count": 384, "statues_count": 37265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "553011a282a18fb5", "name": "University", "place_type": "city", "bounding_box": rectangle("-82.45491,28.054805 -82.409812,28.098918") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980506279944192, "text": "Wo liegt Washington? https://t.co/9W4ynUFIKP #Washington #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0241,38.8921"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Washington", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 432, "friends_count": 114, "statues_count": 1852904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 11, "stateName": "District of Columbia", "countyID": 11001, "countyName": "District of Columbia", "cityID": 1150000, "cityName": "Washington" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980507030724612, "text": "someone needs to step up and see kinky boots with me when it comes to sf so i can see @AdamSKaplan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 467029346 }}, "user": { "id": 24441113, "name": "søphia cøllins |-/", "screen_name": "sophrincess", "lang": "en", "location": "sfsu", "create_at": date("2009-03-14"), "description": "emo to the extremo", "followers_count": 699, "friends_count": 1202, "statues_count": 14327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980507907334144, "text": "@2RealMb ONLY females that have been hurt to the point that they don't give af..But that's still not as bad as men. We can agree 2 disagree��", "in_reply_to_status": 692978847776686081, "in_reply_to_user": 822052861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 822052861 }}, "user": { "id": 4155366919, "name": "❤️SNAPCHAT: CHYTAH", "screen_name": "savychytah", "lang": "en", "location": "mars", "create_at": date("2015-11-07"), "description": "DOWN2MARZ•|", "followers_count": 246, "friends_count": 178, "statues_count": 5187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thatcher, AZ", "id": "019c4b8a4a5f68ec", "name": "Thatcher", "place_type": "city", "bounding_box": rectangle("-109.795135,32.797639 -109.732871,32.86024") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4009, "countyName": "Graham", "cityID": 473420, "cityName": "Thatcher" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980508456808450, "text": "Can you find Cape Coral on the map? Just try it at https://t.co/pnSPoNdZJO #Cape Coral", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.9495,26.5629"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cape" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1030, "friends_count": 312, "statues_count": 2558549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Coral, FL", "id": "fbd1502e5edce6bf", "name": "Cape Coral", "place_type": "city", "bounding_box": rectangle("-82.069567,26.531814 -81.904101,26.741146") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1210275, "cityName": "Cape Coral" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980508469366797, "text": "\"Confession: Zit cream is the only thing I've ever stolen in my life\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483643559, "name": "Maddie", "screen_name": "Madelinexgomez", "lang": "en", "location": "Bay Area", "create_at": date("2012-02-04"), "description": "University of Oregon", "followers_count": 535, "friends_count": 505, "statues_count": 14667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980508549058560, "text": "Dem boyz~", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312036576, "name": "Samuel", "screen_name": "samthonydavis", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-06-06"), "description": "Found myself in the fire burnt hills", "followers_count": 546, "friends_count": 367, "statues_count": 35985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980508817489920, "text": "wanted a 2015 5.0 in my name, I got it. Wanted an apt, got it. Wanted a legit boob job, not that 5k bullshit, got it. I'll get the corvette.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236157732, "name": "D¥", "screen_name": "Daunyang_", "lang": "en", "location": "Plano, TX", "create_at": date("2011-01-09"), "description": "null", "followers_count": 475, "friends_count": 182, "statues_count": 28751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125344") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-01-29T00:00:02.000Z"), "id": 692980508934983682, "text": "@nunathecat jap apa?", "in_reply_to_status": 692980450114035713, "in_reply_to_user": 104120373, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 104120373 }}, "user": { "id": 446661800, "name": "Bal Armstrong", "screen_name": "IqbalIzhar", "lang": "en", "location": "700 S 17 St, Apt.11", "create_at": date("2011-12-25"), "description": "GoBigRed", "followers_count": 1000, "friends_count": 563, "statues_count": 23903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980509274705920, "text": "48.6F (Feels: 48.6F) - Humidity: 91% - Wind: 6.9mph SW - Gust: 7.6mph - Pressure: 1031.9mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 224497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980509354385408, "text": "#SupportOriginMelissa 39.4°F Wind:1.1mph Pressure: 29.76hpa Rising slowly Rain Today 0.00in. Forecast: Becoming fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980509614424064, "text": "@bvbymilo worth buying?", "in_reply_to_status": 692976853691338752, "in_reply_to_user": 193878908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 193878908 }}, "user": { "id": 608724554, "name": "Danny", "screen_name": "e_danny_v", "lang": "en", "location": "null", "create_at": date("2012-06-14"), "description": "FĒNÄM https://soundcloud.com/edannyv", "followers_count": 149, "friends_count": 78, "statues_count": 12746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980509815771137, "text": "ВСЁ вижу! https://t.co/p4PjafBPiS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.0117447,37.69876295"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 3829425134, "name": "Allen Collier", "screen_name": "colaroler", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "null", "followers_count": 5, "friends_count": 0, "statues_count": 12831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980510155542528, "text": "@HandsomeBobby5 hey lol", "in_reply_to_status": 692977773116264448, "in_reply_to_user": 183498178, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 183498178 }}, "user": { "id": 1589311447, "name": "Karisa.", "screen_name": "Karisanewkirk", "lang": "en", "location": "Hofstra University", "create_at": date("2013-07-12"), "description": "The giggle at a funeral. twenty.", "followers_count": 1946, "friends_count": 614, "statues_count": 22257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Garden City, NY", "id": "0427846a106c0bd9", "name": "East Garden City", "place_type": "city", "bounding_box": rectangle("-73.617232,40.712873 -73.580488,40.747586") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3622065, "cityName": "East Garden City" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980510281306112, "text": "Lately I've been deciding whether or not I want to go to grad school.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353941095, "name": "leileigirl", "screen_name": "KaleiSouza", "lang": "en", "location": "Covina, CA", "create_at": date("2011-08-12"), "description": "just a free spirt, with a wild heart.", "followers_count": 353, "friends_count": 788, "statues_count": 8611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charter Oak, CA", "id": "07432bdb1901f862", "name": "Charter Oak", "place_type": "city", "bounding_box": rectangle("-117.877463,34.093451 -117.837641,34.106755") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612734, "cityName": "Charter Oak" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980510990143492, "text": "https://t.co/L1bGMpyS9n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio 20k", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 20540, "friends_count": 18533, "statues_count": 7681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511019507712, "text": "Travelling to Cape Coral or just twittering about Cape Coral? https://t.co/pnSPoNdZJO #Cape Coral", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.9495,26.5629"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cape" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1030, "friends_count": 312, "statues_count": 2558550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Coral, FL", "id": "fbd1502e5edce6bf", "name": "Cape Coral", "place_type": "city", "bounding_box": rectangle("-82.069567,26.531814 -81.904101,26.741146") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1210275, "cityName": "Cape Coral" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511535407110, "text": "Laying some color down on these shells / #art #artoninstagram #drums #ludwig #questlove #mometta… https://t.co/Yt22Lj3kWm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.881,33.7407"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "art", "artoninstagram", "drums", "ludwig", "questlove", "mometta" }}, "user": { "id": 21244226, "name": "X Is The Weapon", "screen_name": "JUNE22_", "lang": "en", "location": "ÜT: 33.678754,-117.830914", "create_at": date("2009-02-18"), "description": "△ Father, Visual Artist, Designer | 5thElement UZN Calafia Zulus | IG:XISTHEWEAPON / http://xistheweaponrecords.com", "followers_count": 875, "friends_count": 488, "statues_count": 14696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511589924865, "text": "Awake doing stupid study guides and studying and I'm so tired.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3098768527, "name": "katherine", "screen_name": "kxthyalvxrez", "lang": "en", "location": "null", "create_at": date("2015-03-19"), "description": "ths junior", "followers_count": 171, "friends_count": 185, "statues_count": 3511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tigard, OR", "id": "d1f122645b638aec", "name": "Tigard", "place_type": "city", "bounding_box": rectangle("-122.840168,45.393108 -122.74354,45.456634") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4173650, "cityName": "Tigard" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511711596544, "text": "I thought we already were �� https://t.co/Lr4Is4rFSJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377714550, "name": "Washed Up nd Broke E", "screen_name": "KinggChip", "lang": "en", "location": "HoustAtlantaLagos", "create_at": date("2011-09-21"), "description": "Gimme a chance ma, I promise my tweets just jokes. Those my sisters eyes in my header cause she don't trust you hoes.", "followers_count": 1014, "friends_count": 496, "statues_count": 85268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511803904000, "text": "Bruh.... ������ https://t.co/8YqmeP7w3w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 860788231, "name": "Dos", "screen_name": "kyroncarlson", "lang": "en", "location": "Washougal, WA", "create_at": date("2012-10-03"), "description": "~ PHIL 4:13 ~ Scorpio ~ JETS NATION ~ RIP-CITY ~ RED SOX ~ #NavyBound ~ #WolfPack ~ #OwlSquad ~ #PNFLQB ~ #WASquad", "followers_count": 1131, "friends_count": 1909, "statues_count": 20091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washougal, WA", "id": "a01f1fb161f325db", "name": "Washougal", "place_type": "city", "bounding_box": rectangle("-122.386427,45.564129 -122.294849,45.600851") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5376405, "cityName": "Washougal" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980511854219264, "text": "#valentinesday Is closed! Don't miss THIS! Order NOW on https://t.co/i3VTUQj04x your… https://t.co/C5l2ye3YOk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "valentinesday" }}, "user": { "id": 288676090, "name": "Janet Fischietto", "screen_name": "JanetFischietto", "lang": "it", "location": "null", "create_at": date("2011-04-27"), "description": "A tribute to the Circus and Vaudeville world! ☆ Burlesque, Trapeze performer & Fire Eater ☆ For booking: Missfischietto@gmail.com", "followers_count": 2126, "friends_count": 642, "statues_count": 5982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980512256835584, "text": "Finally off for the next 2 days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 89530494, "name": "David", "screen_name": "StayHiDavid", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-11-12"), "description": "Cream.", "followers_count": 5387, "friends_count": 76, "statues_count": 11143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980512516935680, "text": "When y'all push my buttons I just gotta remember where my priorities are vs yours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1432120148, "name": "Aleyda 〰", "screen_name": "aleydatmz", "lang": "en", "location": "Iowa City, IA", "create_at": date("2013-05-15"), "description": "On the road to more success • UIowa", "followers_count": 519, "friends_count": 363, "statues_count": 7929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980512835633152, "text": "In the bed all day, bed all day, bed all day. Fucking and fighting. It's a paradise & it's a war zone. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96880507, "name": "valerie limon♔", "screen_name": "ValerieStarBby", "lang": "en", "location": "Texas, Y'all❤️", "create_at": date("2009-12-14"), "description": "#SWERVE✌|I did wake up like this.❤|Snapchat: valerielimon |A girl should be two things: classy & fabulous.❤️|Slaying one day at a time.|", "followers_count": 4525, "friends_count": 4648, "statues_count": 44074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980513259307008, "text": "Amor es cuando nosotros nos olvidamos de nosotros mismos para pensar en alguien más.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.928367,21.306765"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2561557709, "name": "Sincericidio 20k", "screen_name": "ElSincericidio", "lang": "es", "location": "null", "create_at": date("2014-05-24"), "description": "El arte de suicidarte un poquito con cada verdad que sale de tu teclado.", "followers_count": 20540, "friends_count": 18533, "statues_count": 7682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-01-29T00:00:03.000Z"), "id": 692980513301200896, "text": "@casey_jane_ @Nelson_Goins Justice has been served today.", "in_reply_to_status": 692950126541799424, "in_reply_to_user": 277770409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 277770409, 1594883827 }}, "user": { "id": 401506622, "name": "Fernando", "screen_name": "fernanboom", "lang": "en", "location": "null", "create_at": date("2011-10-30"), "description": "Sigma Alpha Epsilon.", "followers_count": 257, "friends_count": 253, "statues_count": 4426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -84.097028,39.79369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980513552887808, "text": "@iifucklesbians you? ����������", "in_reply_to_status": 692979739099795456, "in_reply_to_user": 622932011, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 622932011 }}, "user": { "id": 350456242, "name": "feelikejojowitdatool", "screen_name": "SBNellBone", "lang": "en", "location": "westlawn doughboyz we gettin $", "create_at": date("2011-08-07"), "description": "#FreeJames #ThePearlAcamady free da guys til dey all free! #staingang#3⃣ SC:sbnellbone Ig:nellbone", "followers_count": 1874, "friends_count": 1623, "statues_count": 42172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980513594810369, "text": "Niggas looked at me crazy when I played max b on flat bush ave lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355819149, "name": "LL Cool beige", "screen_name": "AJthePoetic", "lang": "en", "location": "incognito", "create_at": date("2011-08-15"), "description": "http://loyaltyoverfame.com / http://thagreymatter.com / http://itstrishymaries.com / http://kingbraize.com/app /cheneelynn.com check all these sites out #LOF", "followers_count": 1953, "friends_count": 2186, "statues_count": 269721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980513951354880, "text": "My eyes wanna close so bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2588631843, "name": "Ꮛmily martinez", "screen_name": "emii_martinez94", "lang": "en", "location": "delaware, oh", "create_at": date("2014-06-06"), "description": "you only live once", "followers_count": 392, "friends_count": 142, "statues_count": 2472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, OH", "id": "0122033a364ebdac", "name": "Delaware", "place_type": "city", "bounding_box": rectangle("-83.126814,40.234907 -83.030364,40.333339") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39041, "countyName": "Delaware", "cityID": 3921434, "cityName": "Delaware" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980514257530880, "text": "y'all be more concerned about niggas than ur own shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188193466, "name": "naruto's stepmom", "screen_name": "shortyfrmtwitta", "lang": "en", "location": "ig : juicynextdoor", "create_at": date("2010-09-07"), "description": "92Y.", "followers_count": 6453, "friends_count": 4785, "statues_count": 155048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Carson, CO", "id": "4b1e910d57d72600", "name": "Fort Carson", "place_type": "city", "bounding_box": rectangle("-104.818899,38.716373 -104.732634,38.7668") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 827370, "cityName": "Fort Carson" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980514270134272, "text": "Cause when I catch that case for semi automating your Range..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51834185, "name": "Kathleen Cleaver", "screen_name": "thatdame", "lang": "en", "location": "moving in a room full of nos", "create_at": date("2009-06-28"), "description": "somebody call Mase so he can pray for us", "followers_count": 1552, "friends_count": 865, "statues_count": 160177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980514567917568, "text": "LMFAOOO DIS NOT IT PLAYA https://t.co/yAfb2ivYCN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30409694, "name": "GarlicPepper Wingz", "screen_name": "_tweetsfromDJAY", "lang": "en", "location": "null", "create_at": date("2009-04-10"), "description": "22 / Baton Rouge / Clark Atlanta U / Beyonce / #tryjesusdonttryme / IG: dkeith_", "followers_count": 1376, "friends_count": 673, "statues_count": 145822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980514618265600, "text": "Always complaining that I never get enough sleep but I also drink red bulls at 12:30 so there's that.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 404576511, "name": "αdαm", "screen_name": "adamfranko_", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2011-11-03"), "description": "I have Dion's Lemonade running through my veins.", "followers_count": 601, "friends_count": 316, "statues_count": 5268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980514748252160, "text": "oooo hes mma training now. i stopped watching bc he made a derogatory comment against geys one time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3903951312, "name": "Jayy Sorath LaVeyGOD", "screen_name": "Jayy_LaVey", "lang": "en", "location": "Franklin, TN", "create_at": date("2015-10-15"), "description": "#illuminati. IM FUNNY. IM SARCASTIC. #starvingartist King, Of Tennessee. Married. 25. BLACKBILLY. I'm Silly. SloppyTweeter. Kid Komedian: IN PROGRESS. #BOSSING.", "followers_count": 86, "friends_count": 98, "statues_count": 48590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, TN", "id": "cc631a80adacd459", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.935419,35.85036 -86.766934,36.019674") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47187, "countyName": "Williamson", "cityID": 4727740, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980515335507969, "text": "I'm home safe and sound", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 533636260, "name": "A", "screen_name": "alexacortiz", "lang": "en", "location": "minding your own business", "create_at": date("2012-03-22"), "description": "harlem shaking through the pressure #fsu19", "followers_count": 1245, "friends_count": 484, "statues_count": 64430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fitchburg, MA", "id": "14ec2fa66c512d73", "name": "Fitchburg", "place_type": "city", "bounding_box": rectangle("-71.862751,42.543916 -71.747946,42.622038") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2523875, "cityName": "Fitchburg" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980515612266497, "text": "Or zhall I zay Ztill on repeat Ztill going hard ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189771952, "name": "©-swizzle", "screen_name": "casey_gallegos", "lang": "en", "location": "Nowhere USA", "create_at": date("2010-09-11"), "description": "(Inserts some fancy stuff about myself.) 20.", "followers_count": 881, "friends_count": 615, "statues_count": 41688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hereford, TX", "id": "8778e199ce41ead8", "name": "Hereford", "place_type": "city", "bounding_box": rectangle("-102.431992,34.795084 -102.354,34.851214") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48117, "countyName": "Deaf Smith", "cityID": 4833320, "cityName": "Hereford" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980515826200578, "text": "@SolelyJones lmaooo", "in_reply_to_status": 692979763112185858, "in_reply_to_user": 286440930, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 286440930 }}, "user": { "id": 421264231, "name": "g0lden hippie ☯", "screen_name": "g0ldenHighTop", "lang": "en", "location": "where there's ☮ ", "create_at": date("2011-11-25"), "description": "fiu18", "followers_count": 1689, "friends_count": 708, "statues_count": 63287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naranja, FL", "id": "00531e7c63964b3d", "name": "Naranja", "place_type": "city", "bounding_box": rectangle("-80.453594,25.504351 -80.405043,25.536465") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1247700, "cityName": "Naranja" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980517197717504, "text": "MAYBE my best isn't good enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1292303990, "name": "val", "screen_name": "_valipatzi", "lang": "en", "location": "null", "create_at": date("2013-03-23"), "description": "null", "followers_count": 676, "friends_count": 538, "statues_count": 7755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-29T00:00:04.000Z"), "id": 692980517222924289, "text": "I'm doin' aight. https://t.co/w1l212X9gS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4688630628, "name": "major lazer", "screen_name": "blackboyfIy", "lang": "en", "location": "210 • 506 • 249", "create_at": date("2016-01-01"), "description": "⠀⠀ ⠀⠀⠀⠀A subtle power, a tough love. ⠀⠀⠀⠀Laws of the world never stopped us.", "followers_count": 2487, "friends_count": 195, "statues_count": 7225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980517684264963, "text": "Temp: 23.2°F - Dew Point: 10.3° - Wind: 10.4 mph - Gust: 18.3 - Rain Today: 0.00in. - Pressure: 29.88in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980518086971394, "text": "@JediNivso @TheSolving nivsoo come play", "in_reply_to_status": 692980386733920263, "in_reply_to_user": 2265612210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2265612210, 928996064 }}, "user": { "id": 2857888541, "name": "LeTo", "screen_name": "InLeToWeTrust", "lang": "en", "location": "null", "create_at": date("2014-11-02"), "description": "[14] #LeToHasNoLanXP l Competitive Cod l sub support l HATE NIGGERSSS", "followers_count": 407, "friends_count": 217, "statues_count": 19914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980518208606208, "text": "Will date you 4 that Yeezy boost connect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281846990, "name": "Blondre 3000", "screen_name": "AvaCrum", "lang": "en", "location": "Eugene, OR", "create_at": date("2011-04-13"), "description": "Catch me as champagne_babi on the Instagram. I am the human form of Miss Piggy.", "followers_count": 355, "friends_count": 850, "statues_count": 12344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980518430859264, "text": "Texting lyrics to the homies cause I'm bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 600102513, "name": "Yayo", "screen_name": "_yayyyo", "lang": "en", "location": "Trap", "create_at": date("2012-06-05"), "description": "I to eat", "followers_count": 406, "friends_count": 245, "statues_count": 58721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980518678368257, "text": "There are #canvases and #artsupplies everywhere in my house Its crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "canvases", "artsupplies" }}, "user": { "id": 1360438711, "name": "DoN_JuAn_DeLa_DoTz!", "screen_name": "iamMIKRODOTZ", "lang": "en", "location": "San Diego Ca!", "create_at": date("2013-04-17"), "description": "Artist from the streets to the gallery. In clear San Diego where the way is live high and lay low!", "followers_count": 2040, "friends_count": 4010, "statues_count": 1216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980519093563392, "text": "@BeanssGilly I'll have to go check those out ��", "in_reply_to_status": 692972355170242561, "in_reply_to_user": 2329571264, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2329571264 }}, "user": { "id": 2597043888, "name": "Vannnnaahh", "screen_name": "savannah_frenes", "lang": "en", "location": "Probably sleeping", "create_at": date("2014-06-30"), "description": "null", "followers_count": 368, "friends_count": 327, "statues_count": 16275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980519366201346, "text": "Time to change some screen savers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2544107371, "name": "Breeaaunhhhh ☯", "screen_name": "brimariexo_", "lang": "en", "location": "Take Me To Ultra Europe!!", "create_at": date("2014-06-03"), "description": "Dont take life to serious \n\n☪☮e✡i☯✝", "followers_count": 273, "friends_count": 302, "statues_count": 3015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980519965958145, "text": "Just got home from work I'm fucken dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3259503344, "name": "mami shampoo", "screen_name": "Prisciladgaf", "lang": "en", "location": "Amigas Cheetahs❣", "create_at": date("2015-06-28"), "description": "Isaac is the loml ❤️21⚽️", "followers_count": 76, "friends_count": 63, "statues_count": 2362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980520339251201, "text": "All it takes is a flat tire to see who your homies are", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180881430, "name": "Chris Gutierrez", "screen_name": "ArctiChriss", "lang": "en", "location": "null", "create_at": date("2010-08-20"), "description": "DHB", "followers_count": 255, "friends_count": 226, "statues_count": 4012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980520678989825, "text": "don't say you love me more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1132572194, "name": "cas-serole", "screen_name": "hxneybunny", "lang": "en", "location": "null", "create_at": date("2013-01-29"), "description": "purpose", "followers_count": 51, "friends_count": 28, "statues_count": 9343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980520687370245, "text": "Sheesh https://t.co/u6gcPxaaSC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 252983481, "name": "wolf", "screen_name": "WOLFDREE", "lang": "en", "location": "phnx AZ ", "create_at": date("2011-02-16"), "description": "LOVE YOURZ #ripsabrina", "followers_count": 1542, "friends_count": 459, "statues_count": 51646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980521165545474, "text": "I have my gun', handy, a https://t.co/rvkBbImXIk, the victim was a woman, shot by the suspect..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2854860121, "name": "Mr.G", "screen_name": "gsmall748", "lang": "en", "location": "Tacoma ,Wa", "create_at": date("2014-10-13"), "description": "About me, I'm a Conservative , also stand for justice & the American people, I read alot & do my own fact checking, if I'm wrong I'll admit it & move forward.", "followers_count": 469, "friends_count": 384, "statues_count": 6131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562109,47.166863 -122.348569,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980521362661376, "text": "Someone left their glasses in my car and you know I had to �� https://t.co/dwUUqdxA2Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3150505334, "name": "Ålexzandra", "screen_name": "___alexzandraaa", "lang": "en", "location": "null", "create_at": date("2015-04-11"), "description": "kick ass, not kiss ass.", "followers_count": 190, "friends_count": 178, "statues_count": 4565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-29T00:00:05.000Z"), "id": 692980521714978819, "text": "YESSSSS same lol https://t.co/bcIxloo9iX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1606434468, "name": "crystal.", "screen_name": "crystalgetspapi", "lang": "en", "location": "null", "create_at": date("2013-07-19"), "description": "what it is , wassuuuup #LasGueras✨", "followers_count": 579, "friends_count": 338, "statues_count": 35244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980521773699072, "text": "@tayloreynolds13 @JaredMorgan don't let that IMPACT your opinion of this amazing film!!!!\n\n...IMPACT! Like the 3RD ONE!\n\n*leaves the room*", "in_reply_to_status": 692952662522855425, "in_reply_to_user": 379504940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379504940, 18093580 }}, "user": { "id": 1622067985, "name": "Blake Garnsey", "screen_name": "BalakayDraws", "lang": "en", "location": "California", "create_at": date("2013-07-25"), "description": "drawings, ramblings, and squintings", "followers_count": 74, "friends_count": 290, "statues_count": 454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980522549657605, "text": "Heard yo girl blocked me for exposing her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 360153815, "name": "HennessyPapi", "screen_name": "STATmargo", "lang": "en", "location": "null", "create_at": date("2011-08-22"), "description": "Standing Tall And Talented. 6'3 ♍️", "followers_count": 477, "friends_count": 595, "statues_count": 22689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soledad, CA", "id": "642fbca6eea15240", "name": "Soledad", "place_type": "city", "bounding_box": rectangle("-121.338707,36.410518 -121.306524,36.445593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 672520, "cityName": "Soledad" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980523896037379, "text": "Politics Jim, @politicsjim is now trending in #Minneapolis https://t.co/cnc43XuJY9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Minneapolis" }}, "user_mentions": {{ 356289479 }}, "user": { "id": 172636093, "name": "Trendsmap MPLS", "screen_name": "TrendsMPLS", "lang": "en", "location": "Minneapolis, USA", "create_at": date("2010-07-29"), "description": "Real-time Minneapolis Twitter trends", "followers_count": 3502, "friends_count": 819, "statues_count": 11750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980524038668289, "text": "@XDAGarwynn go to bed", "in_reply_to_status": 692980368572600320, "in_reply_to_user": 141724226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 141724226 }}, "user": { "id": 116308543, "name": "Kyle GaddO)))", "screen_name": "kylegaddo", "lang": "en", "location": "Addison, IL", "create_at": date("2010-02-21"), "description": "Stuff and things at @novyunlimited. @11x2 CM. Wordmancer: @SaveContinue, @HeavyBlog. Photo guy sometimes. 47 Lamborghinis in my Lamborghini account.", "followers_count": 1449, "friends_count": 1360, "statues_count": 140910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980524411912192, "text": "That's right, run away. Like alwaysssss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2780409113, "name": "Ana Isabel✨", "screen_name": "Anagedde", "lang": "en", "location": "Dexter, MI", "create_at": date("2014-09-22"), "description": "null", "followers_count": 248, "friends_count": 141, "statues_count": 10983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dexter, MI", "id": "494576de0ef68328", "name": "Dexter", "place_type": "city", "bounding_box": rectangle("-83.924606,42.304776 -83.839003,42.36893") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2622160, "cityName": "Dexter" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980524726554626, "text": "Ppl always express they have a lot on their mind when it's really one thing.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2364053896, "name": "UNcalled for", "screen_name": "CHII_TEAAA", "lang": "en", "location": "null", "create_at": date("2014-02-24"), "description": "null", "followers_count": 104, "friends_count": 62, "statues_count": 3475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980525288574976, "text": "I still can't get my sleeping schedule right...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459625892, "name": "✨Lupe✨", "screen_name": "lupita_robledo", "lang": "en", "location": "Houston - Austin ", "create_at": date("2012-01-09"), "description": "The Carousel Never Stops Turning | UT'18", "followers_count": 389, "friends_count": 269, "statues_count": 10905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980525372407808, "text": "1/29/2016 - 02:00\nTemp: 32.8F \nHum: 91%\nWind: 0.0 mph\nBaro: 30.004in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 49466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-01-29T00:00:06.000Z"), "id": 692980525863141376, "text": "@tyler_wengert https://t.co/DiQ3QPbmMb", "in_reply_to_status": -1, "in_reply_to_user": 2601777098, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2601777098 }}, "user": { "id": 1430524586, "name": "Marta ✨", "screen_name": "martaadamonyte_", "lang": "en", "location": "Winter Park, Colorado ", "create_at": date("2013-05-15"), "description": "photography x Libra ♎️", "followers_count": 388, "friends_count": 273, "statues_count": 2407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fraser, CO", "id": "017479e94a577e1b", "name": "Fraser", "place_type": "city", "bounding_box": rectangle("-105.821978,39.909372 -105.780623,39.953019") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8049, "countyName": "Grand", "cityID": 828305, "cityName": "Fraser" } }
+{ "create_at": datetime("2016-01-29T00:00:07.000Z"), "id": 692980526588780544, "text": "And just like that I'm 20 years old, wow. God is good�� #blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed" }}, "user": { "id": 2579262594, "name": "Pablo Ayala JR.☆", "screen_name": "pawwwbloo", "lang": "en", "location": "null", "create_at": date("2014-06-20"), "description": "i make your girl's favorite smoothie", "followers_count": 485, "friends_count": 305, "statues_count": 4327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Maria, CA", "id": "dee4ad8775ff8102", "name": "Santa Maria", "place_type": "city", "bounding_box": rectangle("-120.482386,34.875868 -120.356782,34.988866") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669196, "cityName": "Santa Maria" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342885450760192, "text": "I just lost all hope ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31571933, "name": "Trilluminati", "screen_name": "St33zyxSaleen", "lang": "en", "location": "Old Dominion University", "create_at": date("2009-04-15"), "description": "Ig: St33zy | fear no man n only trust the reaper", "followers_count": 689, "friends_count": 634, "statues_count": 18388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342886146879488, "text": "@jkowens14 @jmg66_ Been there many times. The people are so nice and wonderful. Kind of like here back in the day #beforelibs", "in_reply_to_status": 693342113543553025, "in_reply_to_user": 2607804924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beforelibs" }}, "user_mentions": {{ 2607804924, 2160009776 }}, "user": { "id": 83696994, "name": "Greg Hodges", "screen_name": "ChargerGreg", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-10-19"), "description": "Chargers Fan, Army Vet, Fun, Funny, Been all over the world a lot, People with an edge interest me. #tcot #2a #pjnet #israel #oathkeepeers #bluelives #tgdn", "followers_count": 26829, "friends_count": 17563, "statues_count": 14408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342886881005568, "text": "This is for you. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176453045, "name": "Tat ♌", "screen_name": "HottaTatta", "lang": "en", "location": " ℒℴѵℯ N.Y. ", "create_at": date("2010-08-09"), "description": "Engaged to the Universe", "followers_count": 395, "friends_count": 258, "statues_count": 15514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342887195447296, "text": "sure triston can call and get mad and hang up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2841996416, "name": "❣", "screen_name": "thtgirl_dee", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-10-05"), "description": "steady boolin ❣", "followers_count": 149, "friends_count": 65, "statues_count": 7861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342887321292800, "text": "@damonagnos oh, no, run away with haste. cool, though, just realized you're seattle too", "in_reply_to_status": 693333130002890754, "in_reply_to_user": 22455367, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22455367 }}, "user": { "id": 882430411, "name": "Cameron Purn", "screen_name": "howsitgoink", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-10-15"), "description": "Analytics Team @dallasmavs", "followers_count": 1016, "friends_count": 388, "statues_count": 14852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342887531061248, "text": "Lalalalalalalalalalalalalafuckkkkklalalalalalalalala", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 271160497, "name": "ⓐⓓⓡⓘⓐⓝ", "screen_name": "Yo_Adrian24", "lang": "en", "location": "null", "create_at": date("2011-03-23"), "description": "Uglier than sin...Just kidding I'm pretty fuggin cute.", "followers_count": 440, "friends_count": 332, "statues_count": 4570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342888256602112, "text": "My tattoo looks so clean I love it ❤️ best tattoo artist in Hawaii ���� https://t.co/D9UdKVwZwM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4671594300, "name": "Sarah Torrento", "screen_name": "SarahTorrento_", "lang": "en", "location": "Honolulu, HI", "create_at": date("2015-12-28"), "description": "Anthony Torrento❣ IG:Sarahmarietorrento", "followers_count": 30, "friends_count": 46, "statues_count": 212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-01-30T00:00:00.000Z"), "id": 693342888600571906, "text": "@ananenia_ lol good Friday coming up so I know they doing something", "in_reply_to_status": 693342566037614592, "in_reply_to_user": 285395906, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 285395906 }}, "user": { "id": 249447548, "name": "10|23", "screen_name": "ZackRozay", "lang": "en", "location": "St. Mary Parish", "create_at": date("2011-02-08"), "description": "#Team Just a nigga after them M's R.I.P. Ike, Koodie & Hairwron", "followers_count": 598, "friends_count": 598, "statues_count": 29987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, LA", "id": "52a5606a0765b543", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-91.540072,29.765166 -91.477815,29.827225") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22101, "countyName": "St. Mary", "cityID": 2227155, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342889036902401, "text": "Wind 0 mph --. Barometer 30.04 in, Steady. Temperature 43.7 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342889380679680, "text": "@ebbtideapp Tide in Chestertown, Maryland 01/30/2016\n Low 5:33am -0.3\nHigh 12:04pm 1.3\n Low 5:56pm 0.0\nHigh 12:11am 1.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-76.0633,39.2067"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 19, "friends_count": 1, "statues_count": 3424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chestertown, MD", "id": "a39e306e0377ae02", "name": "Chestertown", "place_type": "city", "bounding_box": rectangle("-76.094819,39.188684 -76.049122,39.239173") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24029, "countyName": "Kent", "cityID": 2416225, "cityName": "Chestertown" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342889976332288, "text": "\"I wish I wasn't so fragile, cause I know that I'm not easy to handle\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 361641285, "name": "Ally Davis", "screen_name": "adaves18", "lang": "en", "location": "Ottawa, Antwerpen, Albuquerque", "create_at": date("2011-08-24"), "description": "such is life | #COYS", "followers_count": 360, "friends_count": 342, "statues_count": 12691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342890110640129, "text": "@theblingofpower like im wondering what his backstory is.....", "in_reply_to_status": 693342785857003520, "in_reply_to_user": 3254457466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3254457466 }}, "user": { "id": 494458165, "name": "Darth Ka(Maul)", "screen_name": "karnasonofsun", "lang": "en", "location": "New York City", "create_at": date("2012-02-16"), "description": "Despite everything, it's still you. I Ria/M/21/NYC I QP- @CrookedTricking @smolhinata @keleticism I Ra's Maraq - Hyperion I Arjuna @ CA", "followers_count": 432, "friends_count": 553, "statues_count": 95714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342890261483520, "text": "imy...\n#simpinhourss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "simpinhourss" }}, "user": { "id": 227885251, "name": "lil spook", "screen_name": "_JayQuotes", "lang": "en", "location": "null", "create_at": date("2010-12-17"), "description": "get ur sketchy ass away from me", "followers_count": 1267, "friends_count": 601, "statues_count": 42115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Bear Lake, CA", "id": "17fe6350a3570c69", "name": "Big Bear Lake", "place_type": "city", "bounding_box": rectangle("-116.962579,34.223549 -116.842586,34.262952") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 606434, "cityName": "Big Bear Lake" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342890353885184, "text": "@ringring1224 I have a lot of ds 10.5 and 11 that releases the last few years. Let me know what u want", "in_reply_to_status": 692537865008476160, "in_reply_to_user": 1460240576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1460240576 }}, "user": { "id": 132529252, "name": "Rich 11/01", "screen_name": "The_Narc_", "lang": "en", "location": "New Jersey", "create_at": date("2010-04-13"), "description": "Being broke is a joke i never found funny if you call me Richard then you love me", "followers_count": 1929, "friends_count": 1276, "statues_count": 273258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hasbrouck Heights, NJ", "id": "2a9fc003f4101a1d", "name": "Hasbrouck Heights", "place_type": "city", "bounding_box": rectangle("-74.089373,40.847458 -74.059131,40.876005") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3430420, "cityName": "Hasbrouck Heights" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342890601291776, "text": "met a boy who's dad played football with mine in high school... SUCH A SMALL WORLD!!!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157225760, "name": "Madison Snyder", "screen_name": "MadSnydz", "lang": "en", "location": "USA", "create_at": date("2010-06-18"), "description": "•Arizona State•", "followers_count": 634, "friends_count": 295, "statues_count": 6001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342890911731713, "text": "I'm at Anime Los Angeles 2016 in Ontario, Calif https://t.co/K8LG5IuW3V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.60570711,34.06382001"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 9989622, "name": "Nicomakian Ethics", "screen_name": "astrange_e", "lang": "en", "location": "silicon valley", "create_at": date("2007-11-05"), "description": "An omnipresence in wired™. Little known software engineer. mrvacbob@MAL/MFC/etc. 日本語:@astrange_j(嘘)", "followers_count": 725, "friends_count": 666, "statues_count": 21532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342891180216320, "text": "01/30@03:00 - Temp 27.7F, WC 23.3F. Wind 3.8mph WNW, Gust 12.0mph. Bar 30.065in, Rising. Rain 0.00in. Hum 64%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342892471914496, "text": "im not gonna have the internet in a few hours fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319711277, "name": "natural blue", "screen_name": "popcrunk", "lang": "en", "location": "marion, indiana", "create_at": date("2011-06-18"), "description": "i'm tianna & i really like music, @shanedawson & mac n cheese • emo is back", "followers_count": 1085, "friends_count": 528, "statues_count": 28679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, IN", "id": "6181e0f8ac8150e1", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-85.744882,40.49346 -85.61564,40.610902") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18053, "countyName": "Grant", "cityID": 1846908, "cityName": "Marion" } }
+{ "create_at": datetime("2016-01-30T00:00:01.000Z"), "id": 693342892799062016, "text": "Easily one of the best movies that had ever been on there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2303613588, "name": "B-rice", "screen_name": "bricebeatty", "lang": "en", "location": "California, USA", "create_at": date("2014-01-21"), "description": "yillin'", "followers_count": 205, "friends_count": 334, "statues_count": 11000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342892862013441, "text": "This line at Don taco is ridiculous ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727594008, "name": "Fuad ️", "screen_name": "FuadUpNext", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "#SIUC19 5'11 Guard", "followers_count": 813, "friends_count": 177, "statues_count": 29007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carbondale, IL", "id": "20d0abc87d11501a", "name": "Carbondale", "place_type": "city", "bounding_box": rectangle("-89.294485,37.670875 -89.151934,37.764433") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17077, "countyName": "Jackson", "cityID": 1711163, "cityName": "Carbondale" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342892903903232, "text": "51.9F (Feels: 51.9F) - Humidity: 96% - Wind: 4.5mph SE - Gust: 6.0mph - Pressure: 1029.0mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 224640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342892912463873, "text": "even my closest of friends don't know everything about me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1378443924, "name": "vanessa", "screen_name": "vpdollarsign", "lang": "en", "location": "ny", "create_at": date("2013-04-24"), "description": "i pour my milk first ♊️", "followers_count": 3119, "friends_count": 811, "statues_count": 21468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342892962623489, "text": "I'm right so much of the time that I'm actually starting to impress myself ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005764431, "name": "Lex", "screen_name": "bourdonlexi", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "null", "followers_count": 373, "friends_count": 201, "statues_count": 11073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893243629568, "text": "@_beeeeej @I_am_MTG https://t.co/gGMMOnlKAs", "in_reply_to_status": 693342765216759808, "in_reply_to_user": 2514098420, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2514098420, 347659913 }}, "user": { "id": 34560639, "name": "Mario Benitez", "screen_name": "Mar_e_o17", "lang": "en", "location": "Anaheim", "create_at": date("2009-04-23"), "description": "Blessed above all, Glory to God! , Anaheim native, The mouse pays my bills, love my city, love my family ✝.", "followers_count": 2504, "friends_count": 2310, "statues_count": 194458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893407211520, "text": "God Is Great, Beer Is Good And People Are Crazy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3997304668, "name": "J Gonzalez", "screen_name": "_ItsJoeeee_", "lang": "en", "location": "Wasco, CA ", "create_at": date("2015-10-19"), "description": "Things Only Get Better From Here.", "followers_count": 266, "friends_count": 154, "statues_count": 6836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasco, CA", "id": "01643e7e5fce28b7", "name": "Wasco", "place_type": "city", "bounding_box": rectangle("-119.42052,35.572513 -119.328147,35.610926") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 683542, "cityName": "Wasco" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893415731201, "text": "Wind 1.0 mph SSW. Barometer 29.950 in, Falling slowly. Temperature 33.2 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893587718144, "text": "Temp: 41.9°F Wind:0.0mph Pressure: 30.055hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893818433536, "text": "Wind 5.0 mph SE. Barometer 29.730 in, Falling Rapidly. Temperature 49.1 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342893927346176, "text": "happy midnight @LaurenJauregui", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363972381 }}, "user": { "id": 2661692972, "name": "shan loves lauren", "screen_name": "jaureguihotline", "lang": "en", "location": "lauren/6 :-)", "create_at": date("2014-07-19"), "description": "ethereal: extremely delicate and light in a way that seems too perfect for this world @LaurenJauregui // i love @allyslyrics", "followers_count": 1639, "friends_count": 184, "statues_count": 54425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342894279667712, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 331916965, "name": "#FreebandzTaliban", "screen_name": "BalloutWebbie", "lang": "en", "location": "BALLOUTWORLD w/ Money & Nem", "create_at": date("2011-07-08"), "description": "Free The Guys Rip The Guys #SSD #NLMB #OTF #2FFE #MurdaTeam", "followers_count": 1172, "friends_count": 991, "statues_count": 104750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342894426583040, "text": "@KannieP okay, I'll text you!", "in_reply_to_status": 693341890159235072, "in_reply_to_user": 217951614, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 217951614 }}, "user": { "id": 363225331, "name": "The One✨", "screen_name": "TRainessss", "lang": "en", "location": "on my elephant", "create_at": date("2011-08-27"), "description": "like Nia in her prime. 1990's fine. michigan state university.", "followers_count": 1017, "friends_count": 960, "statues_count": 30377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342894485282816, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 63011649, "name": "rosemary", "screen_name": "rrose____", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-08-04"), "description": "keep ur crown three stacks", "followers_count": 874, "friends_count": 658, "statues_count": 51101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342894569168897, "text": "dope ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 522621695, "name": "lyss", "screen_name": "lysssa_babyyy", "lang": "en", "location": "null", "create_at": date("2012-03-12"), "description": "null", "followers_count": 559, "friends_count": 445, "statues_count": 28630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabeth, NJ", "id": "b74cebcb62a1a686", "name": "Elizabeth", "place_type": "city", "bounding_box": rectangle("-74.254211,40.634285 -74.138838,40.690673") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3421000, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342894770356224, "text": "Ur flamo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3017966996, "name": "q baby", "screen_name": "quincccccc", "lang": "en", "location": "null", "create_at": date("2015-02-12"), "description": "steph curry 42-4", "followers_count": 983, "friends_count": 566, "statues_count": 19801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342895332429824, "text": "Happy birthday to this mark ass musty ass Clark Kent superman from planet krypton ass @originalchrisv https://t.co/xZikxdCbIQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1317611414 }}, "user": { "id": 3772516872, "name": "Burns on a quest", "screen_name": "brandonburnsx", "lang": "en", "location": "null", "create_at": date("2015-10-03"), "description": "We wasn't supposed to make it past 25 jokes on you we still alive", "followers_count": 105, "friends_count": 103, "statues_count": 317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342895655391233, "text": "Putting other before me isn't something i do often so when i do it that just means you're special", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4660091269, "name": "Papi", "screen_name": "ChinoTheSexGod", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "❤️", "followers_count": 18, "friends_count": 19, "statues_count": 136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342895995232256, "text": "#SupportOriginMelissa 33.1°F Wind:3.1mph Pressure: 29.94hpa Falling Rain Today 0.00in. Forecast: Occasional precipitation, worsening", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342896137744384, "text": "@SebetichFeli ily ������", "in_reply_to_status": 693336516064444416, "in_reply_to_user": 1861655131, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1861655131 }}, "user": { "id": 563414963, "name": "rachel rus", "screen_name": "rachel_rus", "lang": "en", "location": "norcal", "create_at": date("2012-04-25"), "description": "wazzu || alpha chi", "followers_count": 393, "friends_count": 299, "statues_count": 4838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213106,46.708177 -117.095324,46.753398") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342896439857152, "text": "World class legend, all time favorite https://t.co/nQ0dGsTuK5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422400565, "name": "McLovin⚽", "screen_name": "MarkoCiklovan", "lang": "en", "location": "null", "create_at": date("2011-11-26"), "description": "null", "followers_count": 280, "friends_count": 353, "statues_count": 11885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Gables, FL", "id": "778af41ffb719450", "name": "Coral Gables", "place_type": "city", "bounding_box": rectangle("-80.297165,25.627038 -80.245009,25.771859") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214250, "cityName": "Coral Gables" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342896607498241, "text": "around this time last year vine was poppin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1260139614, "name": "yeeetgod", "screen_name": "showmesasss", "lang": "en", "location": "null", "create_at": date("2013-03-11"), "description": "((anjali))", "followers_count": 353, "friends_count": 282, "statues_count": 8951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-30T00:00:02.000Z"), "id": 693342896628436993, "text": "She from the jungle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1220306022, "name": "ㅤㅤ", "screen_name": "Jay_23_Money", "lang": "en", "location": "IDGT", "create_at": date("2013-02-25"), "description": "People say nothing's impossible, but I do nothing all the time", "followers_count": 256, "friends_count": 301, "statues_count": 6903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, UT", "id": "70fe5c5340036599", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-113.669566,37.111652 -113.625427,37.151693") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49053, "countyName": "Washington", "cityID": 4967660, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342898167808000, "text": "Sports bras recommended for all @deadsara shows!! #BoobiesFlying #GonnaBeSore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BoobiesFlying", "GonnaBeSore" }}, "user_mentions": {{ 35649837 }}, "user": { "id": 216802091, "name": "Samia*", "screen_name": "DancingPinkCons", "lang": "en", "location": "ÜT: 33.650478,-117.693903", "create_at": date("2010-11-17"), "description": "Adventure. Philosophy. Music. Entrepreneurs. INSTA: DancingPinkCons SNAP: hi_samia", "followers_count": 30, "friends_count": 169, "statues_count": 330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342898226483200, "text": "@pmarca @NoceraNYT there's nothing as \"coastal elite\" snobbery as ignoring the huge numbers and impact Twitter has in the rest of the world.", "in_reply_to_status": 693325067845898240, "in_reply_to_user": 5943622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5943622, 2561646679 }}, "user": { "id": 9321342, "name": "Santiago Siri", "screen_name": "santisiri", "lang": "en", "location": "San Francisco, California", "create_at": date("2007-10-08"), "description": "founder: @DemocracyOS @PartidodelaRed. partner: @Bitexla. member: @YCombinator @WEF @TED. book: #Hacktivismo. ideas: http://medium.com/@santisiri", "followers_count": 50163, "friends_count": 2688, "statues_count": 49176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342898860003328, "text": "Ion tell niggas I show niggas ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282793594, "name": "Leciaa'", "screen_name": "Aleciaspencer_", "lang": "en", "location": "Savannah , GA", "create_at": date("2011-04-15"), "description": "SC. Princessleciaa", "followers_count": 1261, "friends_count": 730, "statues_count": 7034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342898872524800, "text": "Disrespect me then dont ever step to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488008163, "name": "LAW", "screen_name": "itsjrocniigga", "lang": "en", "location": "hyannis", "create_at": date("2012-02-09"), "description": "just a man doin what he gotta do cuz your man couldnt", "followers_count": 92, "friends_count": 126, "statues_count": 2071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barnstable Town, MA", "id": "072afef2aa2f1bf5", "name": "Barnstable Town", "place_type": "city", "bounding_box": rectangle("-70.467597,41.599797 -70.259465,41.735676") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2503690, "cityName": "Barnstable Town" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342899275112448, "text": "THANK YOU!! https://t.co/AIuUse0AiB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3048782003, "name": "Rose", "screen_name": "Rosizzile__", "lang": "en", "location": "null", "create_at": date("2015-02-20"), "description": "Busting Your Chops \ngths \nc/o '16", "followers_count": 174, "friends_count": 481, "statues_count": 1902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Terrace, CA", "id": "08b26afb59230813", "name": "Grand Terrace", "place_type": "city", "bounding_box": rectangle("-117.337845,34.019235 -117.291156,34.045523") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 630658, "cityName": "Grand Terrace" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342899589611520, "text": "@ad0mis HAPPY BIRTHDAY ADOMIS!!!! LOVE YOU FOOL THANK YOU FOR EVERYTHING AND ALWAYS BEING THERE WHEN I NEED YOU. YOU A REAL ONE ����������", "in_reply_to_status": -1, "in_reply_to_user": 75151113, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75151113 }}, "user": { "id": 185334365, "name": "bayla", "screen_name": "Killllla__", "lang": "en", "location": "@_davidmatic", "create_at": date("2010-08-31"), "description": "mvhs", "followers_count": 827, "friends_count": 784, "statues_count": 44116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342899841282048, "text": "happy birthday Hailey��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3153869522, "name": "Thomas Babineau", "screen_name": "thomas_babineau", "lang": "en", "location": "null", "create_at": date("2015-04-13"), "description": "null", "followers_count": 9, "friends_count": 21, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342899879092225, "text": "Anyone what enjoy for drinks? . https://t.co/3m0oX32Tni", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35921682, "name": "Marinko", "screen_name": "Marco140", "lang": "en", "location": "Boston,Mass", "create_at": date("2009-04-27"), "description": "European guy who is born and raise in Croatia. I use to play professional soccer and now working as professional model/actor.Please see my IMDB page", "followers_count": 400, "friends_count": 2022, "statues_count": 2155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342900004859904, "text": "You rock Jays so smooth, if you wasn't into me I'd probably date yo shoes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384151657, "name": "Salma Ruiz", "screen_name": "_salmaruiz", "lang": "en", "location": "Beaverton, OR", "create_at": date("2011-10-02"), "description": "18 | I am my own muse ✨", "followers_count": 794, "friends_count": 271, "statues_count": 56863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342900076318721, "text": "definitely feeling some type of way lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 202484734, "name": "kam✨", "screen_name": "princessichell", "lang": "en", "location": "null", "create_at": date("2010-10-13"), "description": "WATTBA. romans 8:28. ⚓️", "followers_count": 832, "friends_count": 597, "statues_count": 35275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, TN", "id": "52f4a98d03e4ea4f", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-87.486546,36.4602 -87.157791,36.641767") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47125, "countyName": "Montgomery", "cityID": 4715160, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342900701282304, "text": "Wind 1.0 mph W. Barometer 29.984 in, Rising Rapidly. Temperature 27.5F. Rain today 0.00 in. Humidity 72% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 587, "friends_count": 794, "statues_count": 36473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342900705480704, "text": "https://t.co/zwFVF7qRLl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1675036884, "name": "ChildishBrandino", "screen_name": "SuaveBorthwick", "lang": "en", "location": "Hillsboro, OR", "create_at": date("2013-08-16"), "description": "Snapchat: Borthwick", "followers_count": 919, "friends_count": 349, "statues_count": 17106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342900734668800, "text": "when literally everyone is on your fucking nerves but you're trying to play it cool https://t.co/jyhLZL3y9S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160779202, "name": "Britta Sedillo", "screen_name": "brittasedillo", "lang": "en", "location": "atx", "create_at": date("2010-06-28"), "description": "idk idc", "followers_count": 510, "friends_count": 324, "statues_count": 1463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-30T00:00:03.000Z"), "id": 693342901066153985, "text": "@LeahShavol https://t.co/o3KKjQVtkq", "in_reply_to_status": -1, "in_reply_to_user": 345149980, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 345149980 }}, "user": { "id": 367810252, "name": "court♛", "screen_name": "ABCourtney_x0", "lang": "en", "location": "null", "create_at": date("2011-09-04"), "description": "Romans 8:18 ☝️AMC '18 ✏️", "followers_count": 1049, "friends_count": 914, "statues_count": 39722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Worcester, MA", "id": "28db2dbc4240f0b2", "name": "Worcester", "place_type": "city", "bounding_box": rectangle("-71.893265,42.210065 -71.731611,42.341455") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2582000, "cityName": "Worcester" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342901468827649, "text": "@PayPa_Woo nephs ��❗️", "in_reply_to_status": 693342754114572288, "in_reply_to_user": 378389628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 378389628 }}, "user": { "id": 2727032917, "name": "TRIPPY✍✊", "screen_name": "AyTripBae_", "lang": "en", "location": "null", "create_at": date("2014-08-12"), "description": "SC: AyTripae23 : insty mz_trippy ❤️ ......tvt", "followers_count": 2174, "friends_count": 1678, "statues_count": 40158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342901590323200, "text": "Happy birthday babe hope u have a really great day ���� @jacqueline_812 https://t.co/mhEom0tALY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2654988863 }}, "user": { "id": 450414427, "name": "Oatmeal⚡", "screen_name": "AvenaFrankie", "lang": "en", "location": "null", "create_at": date("2011-12-29"), "description": "1) Dream it. 2) Believe it. 3) Achieve it.", "followers_count": 685, "friends_count": 544, "statues_count": 2175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342901896609793, "text": "#SuspiciousPerson at 400-439 Chapel Trace Dr, Azalea Park, FL 32807. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2899544,28.5525093"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousPerson", "orlpol", "ocso" }}, "user": { "id": 39050017, "name": "Police Calls 32807", "screen_name": "orlpol32807", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 197, "friends_count": 1, "statues_count": 34528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azalea Park, FL", "id": "d698c1040a7d0a4e", "name": "Azalea Park", "place_type": "city", "bounding_box": rectangle("-81.310707,28.515824 -81.285546,28.568387") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1202850, "cityName": "Azalea Park" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342901972041728, "text": "The Sparks ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 805513993, "name": "G-Na", "screen_name": "Geenarosee94", "lang": "en", "location": "Minnesota", "create_at": date("2012-09-05"), "description": "12-07-13 WBLHS Class of '16", "followers_count": 819, "friends_count": 79, "statues_count": 11444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Bear Lake, MN", "id": "4afbe81ac7ebf3ea", "name": "White Bear Lake", "place_type": "city", "bounding_box": rectangle("-93.056248,45.033056 -92.980042,45.101902") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2769970, "cityName": "White Bear Lake" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342901984583682, "text": "Earl sweatshirt ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1952910074, "name": "*", "screen_name": "Mvryssv", "lang": "en", "location": "null", "create_at": date("2013-10-10"), "description": "null", "followers_count": 1411, "friends_count": 300, "statues_count": 43350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902026702850, "text": "it's all good round here��⭐️��❤️ https://t.co/k7PsroGfaO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4339547420, "name": "❤㊗️️its.sassy❤️✨", "screen_name": "fineass_sassy", "lang": "en", "location": "Memphis, TN", "create_at": date("2015-11-23"), "description": "foreign girl✨ sc; sasiera_16 ig; its.sassy", "followers_count": 206, "friends_count": 178, "statues_count": 1579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southaven, MS", "id": "0141622e000a64d2", "name": "Southaven", "place_type": "city", "bounding_box": rectangle("-90.096994,34.889384 -89.918584,34.994979") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28033, "countyName": "DeSoto", "cityID": 2869280, "cityName": "Southaven" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902085259265, "text": "Fuck �� \nThat jaggerbomb hit me so hard like a muuuuuuuuuugggggggg ������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 735981566, "name": "Alex", "screen_name": "Alvarado1995A", "lang": "en", "location": "HTX, Southside USA", "create_at": date("2012-08-03"), "description": "null", "followers_count": 144, "friends_count": 101, "statues_count": 5971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902257373184, "text": "FROM THE TOP TO THE BOTTOM, NOWWWW THAT I GOT YOU https://t.co/Jay2aTqMqw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156742714, "name": "#beardfertilizer", "screen_name": "deetwotimess", "lang": "en", "location": "Brew City ", "create_at": date("2010-06-17"), "description": "fat around my waist right? it's hard to lift me off ya nigga face, right? fuck a dime, I'm a silver dolla, bitch.", "followers_count": 1744, "friends_count": 913, "statues_count": 144021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902777438208, "text": "I need to learn how to express my feelings differently", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125847350, "name": "jenn", "screen_name": "fadedbloom", "lang": "en", "location": "long island, ny", "create_at": date("2010-03-23"), "description": "rhinestone eyes", "followers_count": 609, "friends_count": 432, "statues_count": 16085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastport, NY", "id": "01ca8f5c04e99d17", "name": "Eastport", "place_type": "city", "bounding_box": rectangle("-72.754915,40.814013 -72.70876,40.844709") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3622810, "cityName": "Eastport" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902877990915, "text": "@BerlynnKae_eX no?!?", "in_reply_to_status": 693342848947621888, "in_reply_to_user": 264060305, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 264060305 }}, "user": { "id": 35971234, "name": "50 shades of Jade", "screen_name": "Jadeisms", "lang": "en", "location": "behind you breathing heavily ", "create_at": date("2009-04-27"), "description": "Just a girl who uses twitter to post the unfiltered thoughts, jokes, often drunk sarcastic comments she doesn't want her grandma to read", "followers_count": 119, "friends_count": 350, "statues_count": 1887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Spearfish, SD", "id": "6d45335ea3570c83", "name": "North Spearfish", "place_type": "city", "bounding_box": rectangle("-103.91721,44.495889 -103.855012,44.531916") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46081, "countyName": "Lawrence", "cityID": 4645715, "cityName": "North Spearfish" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342902907305984, "text": "We started liking bottoms now we queer ..we started liking bottoms now my whole crew queer ..yeah he didnt start from the bottom ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257271931, "name": "♥ l☮verrrgal ♥ ✌", "screen_name": "nina4uuu", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-02-24"), "description": "❀ i miss the ocean ❀ Check my guy out @GpSplash", "followers_count": 588, "friends_count": 297, "statues_count": 1941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342903075102720, "text": "Bucketlist of all Bucketlists: Squad deep at the drive in watchin my woadie @kalihawk at the… https://t.co/LTDo5mobUG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.16211898,33.90015976"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47278900 }}, "user": { "id": 14939981, "name": "Questlove Gomez", "screen_name": "questlove", "lang": "en", "location": "Philly & Fi-Di NYC & 31Rock", "create_at": date("2008-05-28"), "description": "never a teacher, always a student. the correct answer is always PB&J.--oh i do have 15 actual jobs #BlueCollar1Percenter.", "followers_count": 3572082, "friends_count": 8039, "statues_count": 56670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342903242878976, "text": "Times when you jus wanna check up on a mf but you know that they rather not hear from yo ass ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 774081440, "name": "B I N O ☄", "screen_name": "StanBan0504", "lang": "en", "location": "null", "create_at": date("2012-08-22"), "description": "Welcome to my thoughts straight off the dome!", "followers_count": 1108, "friends_count": 1057, "statues_count": 29467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bettendorf, IA", "id": "125129e27a3e66ca", "name": "Bettendorf", "place_type": "city", "bounding_box": rectangle("-90.533035,41.519514 -90.405592,41.596109") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19163, "countyName": "Scott", "cityID": 1906355, "cityName": "Bettendorf" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342903381327872, "text": "I know https://t.co/M2d5BvJBbt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390542157, "name": "Mary Lee Harcourt", "screen_name": "MsMaryLee77", "lang": "en", "location": "null", "create_at": date("2011-10-13"), "description": "null", "followers_count": 104, "friends_count": 92, "statues_count": 8928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladstone, MO", "id": "e5731aae9bb5e805", "name": "Gladstone", "place_type": "city", "bounding_box": rectangle("-94.593389,39.173738 -94.529799,39.235151") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29047, "countyName": "Clay", "cityID": 2927190, "cityName": "Gladstone" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342903490482176, "text": "Yo I just can't wait to be successful!!! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164406810, "name": "The Phenom", "screen_name": "RockyyJermaine", "lang": "en", "location": "CHI ✈ MKE", "create_at": date("2010-07-08"), "description": "I'm Tupac Reincarnated! Fuck the world cuz I'm curse\n'I will define my own success!'", "followers_count": 1069, "friends_count": 898, "statues_count": 28977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342904245362688, "text": "Hi Lonnie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3254499025, "name": "Bailey Badillo", "screen_name": "Badillo_B33", "lang": "en", "location": "Texas, USA", "create_at": date("2015-06-24"), "description": "my life in tweets ✌", "followers_count": 232, "friends_count": 206, "statues_count": 1716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ozona, TX", "id": "4c177f55f77521c4", "name": "Ozona", "place_type": "city", "bounding_box": rectangle("-101.223971,30.692958 -101.192772,30.735624") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48105, "countyName": "Crockett", "cityID": 4854552, "cityName": "Ozona" } }
+{ "create_at": datetime("2016-01-30T00:00:04.000Z"), "id": 693342905012871168, "text": "@hystERICKal__ yessir", "in_reply_to_status": 693342812188778497, "in_reply_to_user": 3237069665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3237069665 }}, "user": { "id": 164157660, "name": "Dom R. Düliaño", "screen_name": "DRDishman", "lang": "en", "location": "www.acagedmind.com", "create_at": date("2010-07-07"), "description": "Greatness only infringes upon those that dwell in shadows of stagnation, doubt and complacency | AΦA AΠ Proverbs 5:21. Former Football player KY to Houston, TX", "followers_count": 1614, "friends_count": 773, "statues_count": 81249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342905860120576, "text": "I prolly got a bounty on my head ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182121476, "name": "Master Hokage Dre", "screen_name": "andrehaghgoo", "lang": "en", "location": "null", "create_at": date("2010-08-23"), "description": "On a Mission ~ Too blessed to be stressed ~ On a quest to get Lacerated ~ Ball is Life ~ Celtics Quest for 18 ~ Go Hawks 12th man", "followers_count": 327, "friends_count": 462, "statues_count": 4126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342906258599936, "text": "Those are sweet. I think it was the right move https://t.co/fPAbYRuSOX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62356043, "name": "miamincardinals", "screen_name": "miamincardinals", "lang": "en", "location": "null", "create_at": date("2009-08-02"), "description": "Trading the pain of being a Dolphins' fan for the pleasure of being a STL Cardinal fan. Would trade WS rings for one Super Bowl, though.", "followers_count": 661, "friends_count": 615, "statues_count": 1459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "O'Fallon, IL", "id": "2e81d2885dc035cc", "name": "O'Fallon", "place_type": "city", "bounding_box": rectangle("-89.966331,38.568042 -89.846287,38.643885") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1755249, "cityName": "O'Fallon" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342906443104256, "text": "My allergies are fucked up bc Texas doesn't know what season it wants to be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632424981, "name": "Chaka Khan.", "screen_name": "Stormeesaidd__", "lang": "en", "location": "The Jungle.", "create_at": date("2012-07-10"), "description": "I'm standoffish..don't want what ya offering ♓", "followers_count": 1189, "friends_count": 896, "statues_count": 86312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342906589917184, "text": "2:00 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 86671040, "name": "mkn", "screen_name": "MecG__", "lang": "en", "location": "null", "create_at": date("2009-10-31"), "description": "19 years ❤️ # R.I.P Mark D", "followers_count": 2685, "friends_count": 2467, "statues_count": 29489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342906753556480, "text": ":(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2553964844, "name": "Natalia", "screen_name": "natalia_cortezz", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-06-07"), "description": "null", "followers_count": 198, "friends_count": 237, "statues_count": 5585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342907072294912, "text": "@whitneyrogersss nah, I don't wanna see your punk ass face", "in_reply_to_status": 693342796460064768, "in_reply_to_user": 2360389309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2360389309 }}, "user": { "id": 167310339, "name": "Buddy", "screen_name": "TacoBellGawd", "lang": "en", "location": "Taco Bell", "create_at": date("2010-07-16"), "description": "Broncos • OU • Nuggets • Rockies • Avs", "followers_count": 2450, "friends_count": 920, "statues_count": 152113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342907240034304, "text": "Awe my poor bestfriend is sick ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4416450072, "name": "Chey", "screen_name": "lolcheey", "lang": "en", "location": "null", "create_at": date("2015-11-30"), "description": "lol ☁️☮ |", "followers_count": 126, "friends_count": 119, "statues_count": 881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610858,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342907600887808, "text": "Fav if you're up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413344912, "name": "hooah", "screen_name": "CREAM_904", "lang": "en", "location": "Fort Stewart, GA", "create_at": date("2011-11-15"), "description": "I made it out of Dirty Duval #USARMY", "followers_count": 1538, "friends_count": 564, "statues_count": 79627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden City, GA", "id": "50aa8f2881416e47", "name": "Garden City", "place_type": "city", "bounding_box": rectangle("-81.239162,32.038881 -81.125993,32.14354") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1332048, "cityName": "Garden City" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342907793735680, "text": "Im mad that my ride lagged so I'm not gonna dance tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 783013220, "name": "Andrea", "screen_name": "drayahhhh", "lang": "en", "location": "Lynwood, CA", "create_at": date("2012-08-26"), "description": "null", "followers_count": 316, "friends_count": 452, "statues_count": 5952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908020342784, "text": "����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1027268083, "name": "Briana Marie ✨", "screen_name": "BriMruk", "lang": "en", "location": "Normal, IL", "create_at": date("2012-12-21"), "description": "Hang in there beautiful you'll make it through", "followers_count": 232, "friends_count": 138, "statues_count": 10851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908255203328, "text": "Waffle House closer tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160742963, "name": "Boogie Fever", "screen_name": "i_EaT_PIE", "lang": "en", "location": "New Orleans, LA", "create_at": date("2010-06-28"), "description": "I am more than my tweets........ #DaThirstSquad aint dead.... #MostBlocked #NoChillGang #BeardGang #350kTweetClub .... #BlockMeToStopMe #DaBridgeCrew #Troll", "followers_count": 1792, "friends_count": 523, "statues_count": 370234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908305379329, "text": "@MeganFoxFanNYC second half season started on Feb 29", "in_reply_to_status": 693342267851943936, "in_reply_to_user": 326235431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326235431 }}, "user": { "id": 163836651, "name": "John Abucayan", "screen_name": "reaven2005", "lang": "en", "location": "Tempe,az", "create_at": date("2010-07-07"), "description": "iam warehouse 13 agent, sith in training and demigod at half bloodcamp", "followers_count": 200, "friends_count": 547, "statues_count": 51413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908355710976, "text": "happy 16th to my girl @miranda_243 ! we've become so close and made and will make more memories! much luv for ya ������ https://t.co/Drukksrx6a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2700523312 }}, "user": { "id": 3193959452, "name": "Kristen Beitch", "screen_name": "kristenbeitch", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "ig: _kristenveronica_ ask: http://ask.fm/kristenvbeitch", "followers_count": 387, "friends_count": 324, "statues_count": 3291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clayton, CA", "id": "ddcf75d883654021", "name": "Clayton", "place_type": "city", "bounding_box": rectangle("-121.956669,37.919961 -121.911559,37.956058") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 613882, "cityName": "Clayton" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908418789376, "text": "Sonic the hedgehog was a bad dude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1869172716, "name": "Troy", "screen_name": "t1v1018", "lang": "en", "location": "Belcamp, MD", "create_at": date("2013-09-15"), "description": "07.26.14. JNR + TRV.\n#RIPjordan", "followers_count": 452, "friends_count": 305, "statues_count": 82912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, MD", "id": "af5f9fc9117a6fc8", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-76.273881,39.446997 -76.211534,39.507909") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2466762, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342908930351104, "text": "Love love love my littles������ https://t.co/1ZORTejhgp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 308495903, "name": "Erica Steele", "screen_name": "EricaMarie_13", "lang": "en", "location": "Mesquite, TX", "create_at": date("2011-05-31"), "description": "Just a southern girl who loves New York, coffee, traveling, and my loving Savior Jesus Christ •DBU•ΑΔΚ•Kendra Scott•", "followers_count": 518, "friends_count": 177, "statues_count": 9341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-01-30T00:00:05.000Z"), "id": 693342909081346048, "text": "You're all I need��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1253385132, "name": "B♡", "screen_name": "becca_fraer", "lang": "en", "location": "Pasadena, CA", "create_at": date("2013-03-08"), "description": "Brenden Pedrotti❤️", "followers_count": 398, "friends_count": 367, "statues_count": 10074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342909706272768, "text": "Brapp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224988487, "name": "Austinwinters69", "screen_name": "AustinWinters01", "lang": "en", "location": "u.s", "create_at": date("2010-12-10"), "description": "❄ Ride MX, BMX ❄ (FOLLOW) i like turtles #drift #moto #Winters", "followers_count": 282, "friends_count": 306, "statues_count": 5379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342909806936064, "text": "HAPPYBIRTHDAY princess i love you and am so happy we're getting close again❤️today is your day to shine @billliiiee https://t.co/dNzMug6FKm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2479196479 }}, "user": { "id": 839305124, "name": "Victoria Ann", "screen_name": "Victoria_Stodds", "lang": "en", "location": "Reno, NV", "create_at": date("2012-09-22"), "description": "UNR❤️ Criminal Justice Major", "followers_count": 578, "friends_count": 552, "statues_count": 6850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342909945348097, "text": "@djfilippone too late--you'll have BernieBros in your mentions for days now", "in_reply_to_status": 693342662968098816, "in_reply_to_user": 248447113, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248447113 }}, "user": { "id": 2230388154, "name": "Um Pancho Aguilar", "screen_name": "TerrinaMajnoona", "lang": "en", "location": "Portland, OR, USA", "create_at": date("2013-12-04"), "description": "null", "followers_count": 4976, "friends_count": 1519, "statues_count": 166722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910142492672, "text": "@RobertCogburn Shores of Time - Destiny", "in_reply_to_status": 693342169118035968, "in_reply_to_user": 352896449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 352896449 }}, "user": { "id": 1120218649, "name": "Harry Bains", "screen_name": "barryhains", "lang": "en", "location": "null", "create_at": date("2013-01-25"), "description": "Brown with over-grown fohawk, loves Uncharted, Naughty Dog, pokemon and dogs. Goof. Playstation MVP. Supports @Manutd. Seattle. I LOVE DISTILLED WATER.", "followers_count": 307, "friends_count": 133, "statues_count": 7981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340651 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910255800321, "text": "From what I heard it was a mess of mangled men and gore. They even lost a hazmat truck closing the route for 12 hours.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3698298740, "name": "Grand Moff Alexander", "screen_name": "MoffAlexander", "lang": "en", "location": "Among Nothing ", "create_at": date("2015-09-26"), "description": "Big dumb idiot who's joining the Navy. I love movies.", "followers_count": 1123, "friends_count": 639, "statues_count": 20692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Mountain, NV", "id": "368f3fb0373bfc39", "name": "Battle Mountain", "place_type": "city", "bounding_box": rectangle("-116.975027,40.622834 -116.926795,40.657148") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32015, "countyName": "Lander", "cityID": 3204900, "cityName": "Battle Mountain" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910268334082, "text": "wow my drive home is so fucking long....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407485360, "name": "Megs ✌️", "screen_name": "MeganDarlene34", "lang": "en", "location": "null", "create_at": date("2011-11-07"), "description": "so there is this boy ... he changed my view on the world ... ☯", "followers_count": 187, "friends_count": 136, "statues_count": 8324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, OR", "id": "7520fc0be21c62bf", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-122.96154,42.288726 -122.776437,42.398452") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4147000, "cityName": "Medford" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910385786880, "text": "I'm burning the house down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3104009803, "name": "The Goddess", "screen_name": "TheGoddess_AZ", "lang": "en-GB", "location": "Venus ", "create_at": date("2015-03-22"), "description": "null", "followers_count": 767, "friends_count": 664, "statues_count": 30377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910490628096, "text": "If you're sixteen, do yourself a favor and stop posting about getting drunk and how you need a margarita.\nSeriously,\nA straight edge fag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1255616269, "name": "Zo Zo", "screen_name": "sirenicfox", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-03-09"), "description": "Samm's // Model-Actress // Wiccan // Artist // Hooper // Vegan // Drug Free // Tweet me ur dog", "followers_count": 623, "friends_count": 378, "statues_count": 3670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Presa, CA", "id": "48961f8c0229b4e2", "name": "La Presa", "place_type": "city", "bounding_box": rectangle("-117.025216,32.674993 -116.976446,32.73616") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 640326, "cityName": "La Presa" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910696284160, "text": "Where is Quantico on the map? Play the game at https://t.co/a1s65zBh68 #Quantico", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2897,38.5224"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Quantico" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1029, "friends_count": 312, "statues_count": 2560586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quantico, VA", "id": "28392b731872a9ec", "name": "Quantico", "place_type": "city", "bounding_box": rectangle("-77.293703,38.519906 -77.286581,38.525416") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5165120, "cityName": "Quantico" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910704656384, "text": "my bed in canton >>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 746198617, "name": "natalie martindale", "screen_name": "NatMartindale", "lang": "en", "location": "east peoria, il", "create_at": date("2012-08-08"), "description": "I'm very important. I have many leather-bound books and my apartment smells of rich mahogany.", "followers_count": 854, "friends_count": 693, "statues_count": 11608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canton, IL", "id": "766d91b75b305c4d", "name": "Canton", "place_type": "city", "bounding_box": rectangle("-90.085764,40.535191 -89.996604,40.604099") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17057, "countyName": "Fulton", "cityID": 1711007, "cityName": "Canton" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910796812288, "text": "@ImNotMessican who the fuck are you", "in_reply_to_status": 693342807478685696, "in_reply_to_user": 2842328741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2842328741 }}, "user": { "id": 433149620, "name": "lil red", "screen_name": "OGRaiden", "lang": "en", "location": "Tampa, FL", "create_at": date("2011-12-09"), "description": "lil whatthefuck. lil ugly. on some new shit", "followers_count": 2361, "friends_count": 822, "statues_count": 12993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anthony, NM", "id": "599fcfa8d9092943", "name": "Anthony", "place_type": "city", "bounding_box": rectangle("-106.623919,31.974724 -106.582639,32.05459") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3503820, "cityName": "Anthony" } }
+{ "create_at": datetime("2016-01-30T00:00:06.000Z"), "id": 693342910910111744, "text": "idc i miss you @Jeremy_Randy @trevor_w_allen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 416596512, 2329033886 }}, "user": { "id": 749930196, "name": "Laura Henderson", "screen_name": "LauraRHenderson", "lang": "en", "location": "Edmond, OK", "create_at": date("2012-08-10"), "description": "My past has made me who I am today. suck dis pussay", "followers_count": 469, "friends_count": 341, "statues_count": 8936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705272582348800, "text": "In need of some serious Vitamin Sea��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505430057, "name": "Kelsey", "screen_name": "kelsey__fraser", "lang": "en", "location": "Jupiter, FL || tallahassee", "create_at": date("2012-02-26"), "description": "null", "followers_count": 140, "friends_count": 129, "statues_count": 1349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273140117504, "text": "Temp: 41.5F W C: 41.5F Wind:SE at 0.4kts Baro: 1018.6mb and Rising slowly Rain today: 0.00in R H: 74% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 184, "friends_count": 217, "statues_count": 95970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273177829376, "text": "Only thing you care about smoking and do drugs .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430231303, "name": "Diana", "screen_name": "_IXXXX_", "lang": "en", "location": "null", "create_at": date("2011-12-06"), "description": "Aiden's mommy", "followers_count": 688, "friends_count": 567, "statues_count": 3422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery Village, MD", "id": "017a75c0406737c7", "name": "Montgomery Village", "place_type": "city", "bounding_box": rectangle("-77.231293,39.152407 -77.112229,39.235691") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2453325, "cityName": "Montgomery Village" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273274335232, "text": "I'm so glad that I'm not friends with him anymore �� wasting my time smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3104241744, "name": "Isabel Herrera", "screen_name": "yungchataa", "lang": "en", "location": "Ometepec, Guerrero", "create_at": date("2015-03-22"), "description": "18", "followers_count": 527, "friends_count": 462, "statues_count": 11688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273274437632, "text": "If not, we're content with it as you should be content with our choice. Understand?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2389614774, "name": "Tech News Midwest", "screen_name": "TechNewsMidwest", "lang": "en", "location": "null", "create_at": date("2014-03-14"), "description": "Bringing #Entertainment #Technology #Business #Startup news to the #Midwest @technewsbusines @technewsjobs", "followers_count": 19749, "friends_count": 8288, "statues_count": 39906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273572249600, "text": "@dougiedownz @juliatillich hi", "in_reply_to_status": 693704530064592896, "in_reply_to_user": 471705354, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 471705354, 1312197048 }}, "user": { "id": 1365923863, "name": "✨BRYN✨", "screen_name": "nebraska_jones", "lang": "en", "location": "MKE // A2", "create_at": date("2013-04-19"), "description": "i might not go down in history, but i'll go down on you", "followers_count": 739, "friends_count": 396, "statues_count": 13037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ann Arbor, MI", "id": "92b2158a6d509767", "name": "Ann Arbor", "place_type": "city", "bounding_box": rectangle("-83.800262,42.215087 -83.659681,42.324184") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2603000, "cityName": "Ann Arbor" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705273844719616, "text": "@MillzHB12 18", "in_reply_to_status": 693704893744304128, "in_reply_to_user": 416142063, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 416142063 }}, "user": { "id": 144696227, "name": "Kolina✨", "screen_name": "__kolster", "lang": "en", "location": "Arizona State University", "create_at": date("2010-05-16"), "description": "trap soul", "followers_count": 1380, "friends_count": 869, "statues_count": 14020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274104942592, "text": "I'm laying on the edge of my bed bc Sadie takes up the entire thing just cause she can.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762192266, "name": "kenzie", "screen_name": "chyaaah_", "lang": "en", "location": "@justin_schultzz ❤️", "create_at": date("2012-08-16"), "description": "hell raising sugar when the sun goes down", "followers_count": 1497, "friends_count": 959, "statues_count": 73021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274192834560, "text": "I think the worst part of everything is I never loved you all along��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1216225693, "name": "Victoria Treviño", "screen_name": "V_trevino1", "lang": "en", "location": "Taft, America ", "create_at": date("2013-02-24"), "description": "sc: Veee3131", "followers_count": 747, "friends_count": 407, "statues_count": 10111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taft, TX", "id": "de4157442392cc34", "name": "Taft", "place_type": "city", "bounding_box": rectangle("-97.408814,27.969896 -97.374738,27.989929") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4871684, "cityName": "Taft" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274406752256, "text": "who accept gflex this late ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116140051, "name": "Baleb", "screen_name": "kingBaleb", "lang": "en", "location": "pluto", "create_at": date("2010-02-21"), "description": "Grambling State University", "followers_count": 1116, "friends_count": 361, "statues_count": 57854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grambling, LA", "id": "7659c27719e754b6", "name": "Grambling", "place_type": "city", "bounding_box": rectangle("-92.734385,32.50363 -92.669504,32.551821") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2230515, "cityName": "Grambling" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274415321088, "text": "@washingtonpost @wikileaks there is really weird stuff and @realDonaldTrump will tell everybody truth", "in_reply_to_status": 693704819907887104, "in_reply_to_user": 34343851, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2467791, 16589206, 25073877 }}, "user": { "id": 34343851, "name": "douglas hunter", "screen_name": "RANSEN", "lang": "en", "location": "Michigan", "create_at": date("2009-04-22"), "description": "yes your an idiot for looking", "followers_count": 24, "friends_count": 96, "statues_count": 409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, MI", "id": "7b082d03ee1b544d", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-82.968959,42.479787 -82.902872,42.539749") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2669800, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274490818560, "text": "I'm in love ❤️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296526285, "name": "AuthorShenikaMarie", "screen_name": "ShenikaRoberts", "lang": "en", "location": "null", "create_at": date("2011-05-10"), "description": "ANTOINE AND ANAI'S MOMMIE Order your copies of my books Survival of the fittest and Only the Strong Survive from Barnes and Noble Amazon", "followers_count": 951, "friends_count": 689, "statues_count": 42945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274582966272, "text": "Funny how you act like I don't know anything.... Oh, I know alright. Don't underestimate my ability to find things out.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61066540, "name": "Kara Manithep", "screen_name": "KARAMAARIE", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-07-28"), "description": "Too happy to give a f$&%!", "followers_count": 175, "friends_count": 160, "statues_count": 20552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274637438976, "text": "toughest sport on dirt & toughest seat on earth ��������❤️ https://t.co/uJCousPKml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513504694, "name": "ʈ ʒ a ʅ a ɲ ϊ✞", "screen_name": "tortea10", "lang": "en", "location": "null", "create_at": date("2012-03-03"), "description": "'17 ✟♛", "followers_count": 260, "friends_count": 369, "statues_count": 1477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274666852353, "text": "@Santana_Thomas7 i got you, one for me one for you. ��", "in_reply_to_status": 693705097151324160, "in_reply_to_user": 4030335253, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4030335253 }}, "user": { "id": 2337784584, "name": "hyphydee", "screen_name": "denaeexo", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-02-10"), "description": "Rip cousin dom & michelle I got mad love for the Man above ❤ Aspiring MUA /Entrepreneur.", "followers_count": 459, "friends_count": 369, "statues_count": 14484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Creek, CA", "id": "a35b62af9d82aa08", "name": "Walnut Creek", "place_type": "city", "bounding_box": rectangle("-122.10003,37.842368 -121.980829,37.944965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 683346, "cityName": "Walnut Creek" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274687762433, "text": "I really want a long sleeve wedding dress", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491547651, "name": "NMK®", "screen_name": "EnEmKay__", "lang": "en", "location": "Los Angeles ", "create_at": date("2012-02-13"), "description": "null", "followers_count": 1364, "friends_count": 603, "statues_count": 375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705274696163329, "text": "Aww I miss you too https://t.co/CqY1Gaw2Gr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603366870, "name": "Jazz", "screen_name": "jazzyjazz017", "lang": "en", "location": "Austin, TX", "create_at": date("2012-06-08"), "description": "null", "followers_count": 673, "friends_count": 216, "statues_count": 30960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705275446939652, "text": "Agreed... https://t.co/sqXi3dv5eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3038089047, "name": "Jessica Casillas", "screen_name": "jcasillas2197", "lang": "en", "location": "Clarksville, AR", "create_at": date("2015-02-14"), "description": "sometimes you will never know the value of a moment until it becomes a memory.", "followers_count": 103, "friends_count": 79, "statues_count": 2683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, AR", "id": "7e5da517b2cb9095", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-93.520094,35.435512 -93.431075,35.502088") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5071, "countyName": "Johnson", "cityID": 514140, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705275878998016, "text": "@lfinspokane https://t.co/ZvnNmIPGOM", "in_reply_to_status": -1, "in_reply_to_user": 484492129, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 484492129 }}, "user": { "id": 191734804, "name": "Bj Freitag", "screen_name": "lafsatfriday", "lang": "en", "location": "spokane wa", "create_at": date("2010-09-16"), "description": "union carpenter building america, ok maybe just Spokorn. RIP Pops", "followers_count": 179, "friends_count": 466, "statues_count": 5607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705276063502336, "text": "does anyone know Joshua Braverman ? #replytweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "replytweet" }}, "user": { "id": 3014011190, "name": "brianna fisher", "screen_name": "br_i_anna", "lang": "en", "location": "Olympia, WA", "create_at": date("2015-02-08"), "description": "NTHS Sophomore", "followers_count": 229, "friends_count": 152, "statues_count": 2646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705276071936000, "text": "My kindness often mistaken for weakness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418250377, "name": "Alex Marin Jr", "screen_name": "mightyalex_", "lang": "en", "location": "null", "create_at": date("2011-11-21"), "description": "snap: mighyal Humble.", "followers_count": 348, "friends_count": 447, "statues_count": 13618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705276382269440, "text": "Falling asleep to the rain is 1000x better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 474505373, "name": "nat devin", "screen_name": "nat12_arnold", "lang": "en", "location": "null", "create_at": date("2012-01-25"), "description": "work hard, believe in yourself, & have faith ✝ RIP D 11-14-15", "followers_count": 625, "friends_count": 666, "statues_count": 5530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escalon, CA", "id": "d2e195f6c5cf7aaa", "name": "Escalon", "place_type": "city", "bounding_box": rectangle("-121.014986,37.773101 -120.978168,37.807062") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 622790, "cityName": "Escalon" } }
+{ "create_at": datetime("2016-01-31T00:00:00.000Z"), "id": 693705276516601856, "text": "why am I still awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336728781, "name": "Maddie[y] Harrison.", "screen_name": "maddieyphobe", "lang": "en", "location": "Not Canada. ", "create_at": date("2011-07-16"), "description": "mad dog's my name, tech theatre's my game. i support most memes. i do not support donald trump or weeaboos", "followers_count": 208, "friends_count": 206, "statues_count": 18440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705276860436480, "text": "Wind 15.0 mph SSW. Barometer 1003.25 mb, Falling. Temperature 59.3 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705276915093504, "text": "Wind 0 mph --. Barometer 30.01 in, Falling slowly. Temperature 55.0 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705276940247040, "text": "TRIFECTA WEEKEND FRIDAY SATURDAY AND SUNDAY AT THE @ROYALPEACOCKATL #WheresYourDj? @ Atlanta,… https://t.co/jZBRBkGTd1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3881,33.7489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WheresYourDj" }}, "user_mentions": {{ 1883308584 }}, "user": { "id": 30596220, "name": "#WHEREsYourDj?", "screen_name": "KRUNKMASTER", "lang": "en", "location": "V.I./DUBAI/ATLANTA/L.A.", "create_at": date("2009-04-11"), "description": "WORLD FAMOUS DJ SLIK // Sirius Xm // Vibe 103 // FOR BOOKING Call: (305)-343-9808 Email: BookDjSlik@Gmail.com Send your Mp3s to WeFloodTheStreets@Gmail.com", "followers_count": 9148, "friends_count": 4974, "statues_count": 77104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277065940992, "text": "@ebbtideapp Tide in San Francisco Bar, California 01/31/2016\nHigh 3:56am 5.1\n Low 10:41am 1.8\nHigh 4:38pm 3.5\n Low 10:00pm 2.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.6333,37.7667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 18, "friends_count": 1, "statues_count": 3703 }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277195956224, "text": "hahaha��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2611027572, "name": "Karli Gardner", "screen_name": "GardnerKarli", "lang": "en", "location": "buttcheek, SD", "create_at": date("2014-07-07"), "description": "poop", "followers_count": 451, "friends_count": 565, "statues_count": 861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, SD", "id": "78c4b1067358fa99", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-98.536478,45.430119 -98.392195,45.492967") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46013, "countyName": "Brown", "cityID": 4600100, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277325996032, "text": "It's midnight..... I'm straight chilling ready to pass out and DIE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67775437, "name": "vivi", "screen_name": "GoogledGrant", "lang": "en", "location": "null", "create_at": date("2009-08-21"), "description": "He meant it when he said I was worth it", "followers_count": 5085, "friends_count": 726, "statues_count": 77862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277422436352, "text": "I'm still on my shit lil nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 168538510, "name": "YB the G", "screen_name": "itsjustkhari", "lang": "en", "location": "LA / The Wood / IE", "create_at": date("2010-07-19"), "description": "building an artistic empire in LA. I get shit done. #illamerica #lurksquad #FIGHTCLUBLA contact me : ybthegmgmt@gmail.com", "followers_count": 1953, "friends_count": 303, "statues_count": 166810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277481287681, "text": "01/31@03:00 - Temp 27.0F, WC 27.0F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.001in, Falling slowly. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277506387968, "text": "Si tu amor no estuvo a la de mi corazón, espero que mi olvido este a la medida de tu orgullo��☝️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1914569748, "name": "Arely Tierrafria", "screen_name": "tierrafriaarely", "lang": "en", "location": "Highland Village, TX", "create_at": date("2013-09-28"), "description": "Family❤️God", "followers_count": 442, "friends_count": 95, "statues_count": 1939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corinth, TX", "id": "44a81efa2c6b1380", "name": "Corinth", "place_type": "city", "bounding_box": rectangle("-97.09624,33.116079 -97.037458,33.163781") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4816696, "cityName": "Corinth" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277653135360, "text": "I never realized how ugly la boys are until tonight lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2793033817, "name": "violet", "screen_name": "baraboyfriend", "lang": "en", "location": "null", "create_at": date("2014-09-05"), "description": "Crazy bitch coming for your shmoney", "followers_count": 24, "friends_count": 34, "statues_count": 3164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705277812686849, "text": "That Joane dab tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 855944208, "name": "Nol", "screen_name": "manolnol", "lang": "en", "location": "Troy,NY", "create_at": date("2012-09-30"), "description": "Kacak seperti Olivier Giroud #COYG", "followers_count": 323, "friends_count": 290, "statues_count": 19322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, NY", "id": "51cc4a7155935af2", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-73.724998,42.660879 -73.649201,42.795001") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36083, "countyName": "Rensselaer", "cityID": 3675484, "cityName": "Troy" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705278294917120, "text": "Usually don't get into it with people on Twitter but enough is enough honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3388156279, "name": "maddie", "screen_name": "maddiesiggz", "lang": "en", "location": "null", "create_at": date("2015-08-29"), "description": "snapchat: maddiesiggz", "followers_count": 351, "friends_count": 286, "statues_count": 646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705279075037185, "text": "that's what you think��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1858714639, "name": "abby", "screen_name": "snakeynoah", "lang": "en", "location": "8️⃣1️⃣2️⃣", "create_at": date("2013-09-12"), "description": "1-29-16❤️", "followers_count": 1819, "friends_count": 763, "statues_count": 33746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincennes, IN", "id": "bfadcb4bfcf83cef", "name": "Vincennes", "place_type": "city", "bounding_box": rectangle("-87.547173,38.625181 -87.433471,38.736046") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18083, "countyName": "Knox", "cityID": 1879208, "cityName": "Vincennes" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705279490428928, "text": "Honestly I want to refer to the boy that I subtweet by name because he doesn't follow me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447117342, "name": "Kait", "screen_name": "k8lordd", "lang": "en", "location": "♓️", "create_at": date("2011-12-26"), "description": "blue jean baby LA lady", "followers_count": 243, "friends_count": 193, "statues_count": 11334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705279536431104, "text": "Being 19 is so much harder than I thought it would be.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110286142, "name": "Ellen Belle Hansen", "screen_name": "ellenbhansen", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2010-01-31"), "description": "Find what you love and let it kill you. Portrait Photographer.", "followers_count": 1438, "friends_count": 376, "statues_count": 36704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280018747393, "text": "People think It's funny when I say my \"gay radar went on \" like it really does ���� haha !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 871813142, "name": "lizz♈️", "screen_name": "ilovebrittttt7", "lang": "en", "location": "Edinburg, TX", "create_at": date("2012-10-10"), "description": "it's the memories .....snapchat -beautifuliz7", "followers_count": 303, "friends_count": 336, "statues_count": 16282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280257806338, "text": "https://t.co/AOlzWEwzCy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 611857912, "name": "karen gonzalez", "screen_name": "karennkat", "lang": "en", "location": "probably at panda with paul ", "create_at": date("2012-06-18"), "description": "a heart and soul after Jesus | PAC.", "followers_count": 930, "friends_count": 835, "statues_count": 42093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280366841857, "text": "@MyLvisChampagne - It was a very good time! WeHo misses you!", "in_reply_to_status": 693701512447524864, "in_reply_to_user": 1330673312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1330673312 }}, "user": { "id": 834308220, "name": "Jeff Dunphy", "screen_name": "JeffDunphy", "lang": "en", "location": "Saint Louis, Missouri", "create_at": date("2012-09-19"), "description": "St. Louis lawyer, occasional writer, entrepreneur, sarcasm is optional.", "followers_count": 1555, "friends_count": 1432, "statues_count": 603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280417173505, "text": "Mighty morphin power rangers is absolutely my favorite movie ever.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473110432, "name": "Taylor Carroll", "screen_name": "tc_master5000", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "USI '18. live the life you love. love the life you live. SC: tcmaster5000", "followers_count": 1046, "friends_count": 1579, "statues_count": 6629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincennes, IN", "id": "bfadcb4bfcf83cef", "name": "Vincennes", "place_type": "city", "bounding_box": rectangle("-87.547173,38.625181 -87.433471,38.736046") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18083, "countyName": "Knox", "cityID": 1879208, "cityName": "Vincennes" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280471879680, "text": "I'm Cool Af ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 783724478, "name": "Freak Hoes ❤ Rio ひ", "screen_name": "Rio_Sleazy", "lang": "en", "location": "M.I.A ✈ ", "create_at": date("2012-08-26"), "description": "⛽️etting Rich Is The 〽️ission ひ #RipMoma", "followers_count": 2036, "friends_count": 690, "statues_count": 9809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-01-31T00:00:01.000Z"), "id": 693705280501121024, "text": "Temp: 53.7°F Wind:0.0mph Pressure: 29.994hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705280803069952, "text": "Show me its real and be my friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609047704, "name": "Marge", "screen_name": "fabrikmarge", "lang": "en", "location": "Minneapolis,MN", "create_at": date("2013-07-20"), "description": "yung n' thready", "followers_count": 460, "friends_count": 589, "statues_count": 2255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705281331601408, "text": "Kno the meaning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 480978861, "name": "kikikiki", "screen_name": "OhThatsKiki", "lang": "en", "location": "null", "create_at": date("2012-02-01"), "description": "instagram: kikihinojosa sc: kiki_hinojosa", "followers_count": 826, "friends_count": 527, "statues_count": 15662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705281461587968, "text": "AUFC 00 are 7-2 winners with goals from Cody (2), Jbid (2), Jacobc (2) and Gino......Brady was outstanding in net again...great job boys.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540558843, "name": "Kris Baglio", "screen_name": "Avanticoach", "lang": "en", "location": "Tacoma, WA", "create_at": date("2012-03-29"), "description": "Soccer coach, Father, Husband, Social Worker, Sounders FC, Seahawks, Juventus FC, Avanti United FC", "followers_count": 41, "friends_count": 182, "statues_count": 713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562109,47.166863 -122.348569,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705281570623488, "text": "@YOUTUBEARl Here try reading this https://t.co/px5FDGqnF0", "in_reply_to_status": 693702808063905792, "in_reply_to_user": 462388397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 462388397 }}, "user": { "id": 64928068, "name": "Happy B-day Wilmer!", "screen_name": "Girl4Music", "lang": "en", "location": "North East, England", "create_at": date("2009-08-11"), "description": "I stand for individuality and integrity at all times #YOUNIVERSE #ONENESS #CREATIVETHEORIST #SPIRITUAL #EXTREMEMUSICENTHUSIAST", "followers_count": 2986, "friends_count": 781, "statues_count": 563252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705281700786176, "text": "Temp: 50.4°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 30.14 in | Dewpoint: 50.1°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 164703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705282103447552, "text": "Wind 4.0 mph SE. Barometer 29.584 in, Falling. Temperature 55.1 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705282556420096, "text": "I'm tryna get it by any means necessary ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363118752, "name": "Montea", "screen_name": "DattBoiiWorm", "lang": "en", "location": "Mississippi, USA", "create_at": date("2011-08-27"), "description": "Ig: @montea23_ Snapchat: @montea23 ⚪️Selma University ⚾️", "followers_count": 772, "friends_count": 749, "statues_count": 7361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, AL", "id": "ba2e7f7eb278521d", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-87.095997,32.372411 -86.978868,32.457178") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1047, "countyName": "Dallas", "cityID": 169120, "cityName": "Selma" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705282883432448, "text": "57.8F (Feels: 57.8F) - Humidity: 99% - Wind: 4.5mph E - Gust: 6.0mph - Pressure: 1025.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 224784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705282933788672, "text": "I swear I prty much only have a twitter bc I have yet to figure out a way to befriend u guys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2457572712, "name": "sci", "screen_name": "cryfce", "lang": "en", "location": "null", "create_at": date("2014-04-21"), "description": "http://instagram.com/appleorchardd", "followers_count": 461, "friends_count": 209, "statues_count": 16968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283088977920, "text": "@calliope Thanks for playing despite all the confusion we cause for your Twitter existence! :D", "in_reply_to_status": 693659297083854848, "in_reply_to_user": 2080551, "favorite_count": 0, "coordinate": point("-121.9659691,47.7338653"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2080551 }}, "user": { "id": 362929869, "name": "Calliope Games", "screen_name": "CalliopeGames", "lang": "en", "location": "Redmond, WA", "create_at": date("2011-08-26"), "description": "We publish your favorite casual family tabletop games, including Tsuro and Roll For It!", "followers_count": 5163, "friends_count": 848, "statues_count": 4982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duvall, WA", "id": "d2e40e8dd3ab3777", "name": "Duvall", "place_type": "city", "bounding_box": rectangle("-121.989224,47.721538 -121.943988,47.751129") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5319035, "cityName": "Duvall" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283139411968, "text": "✨ face ass ���� https://t.co/szcTEsWiej", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848281004, "name": "N A T", "screen_name": "Nataalii_", "lang": "en", "location": "w/ my baby❤️", "create_at": date("2012-09-26"), "description": "# S I C K B O Y S ? || #FreeDomo", "followers_count": 1081, "friends_count": 509, "statues_count": 28430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Springfield, VA", "id": "33ff2516301663e7", "name": "West Springfield", "place_type": "city", "bounding_box": rectangle("-77.266126,38.755163 -77.202928,38.821463") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5184976, "cityName": "West Springfield" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283659546624, "text": "1 Black-and-white Warbler (Mniotilta varia) - Sweetwater Wetlands - 2016-01-30 16:33 https://t.co/dbHJYjyPd3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11232 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283663699968, "text": "2 House Wren (Brown-throated) (Troglodytes aedon) - Florida Canyon--lower - 2016-01-30 08:55 https://t.co/Dq6S25vzv5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11232 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283667935232, "text": "1 Osprey (Pandion haliaetus) - Sweetwater Wetlands - 2016-01-30 16:33 https://t.co/rq5560S5vZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11232 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283676291072, "text": "1 Snow Goose (Chen caerulescens) - Amado WTP - 2016-01-30 13:55 https://t.co/uCWAoxzO2r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.0544619,31.7306042"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11232 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403380, "cityName": "Arivaca Junction" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283680505856, "text": "1 American Goldfinch (Spinus tristis) - Buenos Aires NWR--Arivaca Cienega - 2016-01-30 08:00 https://t.co/8XOS6C4G8c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11232 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705283793768448, "text": "@kjeziorny28 @jax0214 https://t.co/eCsHciWT46 Band-aids don't fix bullet holes. You say sorry just for show...", "in_reply_to_status": 693703382729494528, "in_reply_to_user": 856495357, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 856495357, 22453981 }}, "user": { "id": 14807674, "name": "MMOC", "screen_name": "acceptmyheart", "lang": "en", "location": "Chicago, IL", "create_at": date("2008-05-16"), "description": "I tweet therefore I am.", "followers_count": 115, "friends_count": 459, "statues_count": 14420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemont, IL", "id": "000a381c49091836", "name": "Lemont", "place_type": "city", "bounding_box": rectangle("-88.048288,41.641019 -87.91269,41.708185") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1742795, "cityName": "Lemont" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705284267577344, "text": "I am not jordan but I am the goat https://t.co/NnwOYOOHjp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1952910074, "name": "*", "screen_name": "Mvryssv", "lang": "en", "location": "null", "create_at": date("2013-10-10"), "description": "don't dm me", "followers_count": 1408, "friends_count": 300, "statues_count": 43516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705284431278080, "text": "So tired yet can't sleep ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1563378588, "name": "Alexx", "screen_name": "Alex_llaquexo", "lang": "en", "location": "NJ", "create_at": date("2013-07-02"), "description": "instagram~ alex_llaque | snap~ alex_llaquexo", "followers_count": 442, "friends_count": 283, "statues_count": 8515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland Park, NJ", "id": "7148e03302aa94e3", "name": "Woodland Park", "place_type": "city", "bounding_box": rectangle("-74.21931,40.872408 -74.174427,40.907502") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3482423, "cityName": "Woodland Park" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705284741636096, "text": "2 Long-eared Owl (Asio otus) - Buenos Aires NWR--Arivaca Cienega - 2016-01-30 08:00 https://t.co/WqjB8ML3MW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11237 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-01-31T00:00:02.000Z"), "id": 693705284787773440, "text": "1 American Goldfinch (Spinus tristis) - Buenos Aires NWR--Arivaca Cienega - 2016-01-30 07:50 https://t.co/8XOS6C4G8c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 11237 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705285370679297, "text": "life is hard", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 578832376, "name": "spruce møøse", "screen_name": "jldechow", "lang": "en", "location": "Galesburg, IL", "create_at": date("2012-05-13"), "description": "rad longboard kid // knox college '19", "followers_count": 458, "friends_count": 631, "statues_count": 7201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galesburg, IL", "id": "419c66cea9a851dc", "name": "Galesburg", "place_type": "city", "bounding_box": rectangle("-90.44122,40.890505 -90.305823,40.995745") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17095, "countyName": "Knox", "cityID": 1728326, "cityName": "Galesburg" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705285404221440, "text": "I could talk to ya for hours������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2713326132, "name": "❗️❕ WG Papa X❕❗️", "screen_name": "Larenz_Smith10", "lang": "en", "location": "Troutdale, OR", "create_at": date("2014-08-06"), "description": "http://www.hudl.com/athlete/4293064/highlights/297911504 SC:larenz_smith24 IG: Larenz10 ?yes I am lightskin if you were gonna ask?", "followers_count": 1056, "friends_count": 639, "statues_count": 14223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705285517516800, "text": "@v3nturemusic I love you", "in_reply_to_status": 693702029827534848, "in_reply_to_user": 3092373721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3092373721 }}, "user": { "id": 834727986, "name": "Vanessa Gutierrez", "screen_name": "vivigutierrez_", "lang": "en", "location": "null", "create_at": date("2012-09-19"), "description": "Erik Rodriguez ❤️", "followers_count": 664, "friends_count": 532, "statues_count": 27127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705285580492801, "text": "Wind 2.0 mph SW. Barometer 29.813 in, Falling slowly. Temperature 53.0 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286264102913, "text": "@IzzzzzzyE liquor King on ashe", "in_reply_to_status": 693705070249132032, "in_reply_to_user": 296998287, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296998287 }}, "user": { "id": 109396240, "name": "Alexander Dominguez", "screen_name": "FVCKALEX_", "lang": "en", "location": "$$$$$$$$$$ ", "create_at": date("2010-01-28"), "description": ".... fakersfield REST IN PARADISE TY TORRES FOREVER x REST EASY NICK STIRLING", "followers_count": 289, "friends_count": 410, "statues_count": 11754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.827474,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286339547136, "text": "Love my new wallpaper ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423840337, "name": "❥ n a n i", "screen_name": "nanacsifri", "lang": "en", "location": "503, USA", "create_at": date("2011-11-28"), "description": "I am the QUEEN and Twitter is my Kingdom blackwidowbb", "followers_count": 233, "friends_count": 118, "statues_count": 14339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286654164992, "text": "#SnapChatSelfie �� https://t.co/bZgzENK7d8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SnapChatSelfie" }}, "user": { "id": 293814339, "name": "Queen B ♛", "screen_name": "Little_Autumn", "lang": "en", "location": "null", "create_at": date("2011-05-05"), "description": "drink your coffee, do your squats and count your blessings. #BaristaBitch", "followers_count": 556, "friends_count": 359, "statues_count": 38530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cambridge, MN", "id": "52c2cc8e251f1a7f", "name": "Cambridge", "place_type": "city", "bounding_box": rectangle("-93.253889,45.529174 -93.189856,45.598025") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27059, "countyName": "Isanti", "cityID": 2709370, "cityName": "Cambridge" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286750703616, "text": "Wind 1.6 mph S. Barometer 29.88 in, Steady. Temperature 34.7 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 106, "statues_count": 157096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286771675136, "text": "they blocked and shit still come to me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4094178863, "name": "Thugga™", "screen_name": "MorganLarhonda", "lang": "en", "location": "Aberdeen", "create_at": date("2015-11-01"), "description": "✌", "followers_count": 469, "friends_count": 164, "statues_count": 15651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, MS", "id": "eced66be89eed087", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-88.580305,33.796677 -88.537251,33.842954") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28095, "countyName": "Monroe", "cityID": 2800180, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286930989056, "text": "#SupportOriginMelissa 52.3°F Wind:4.3mph Pressure: 29.82hpa Falling Rain Today 0.00in. Forecast: Occasional precipitation, worsening", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705286960312320, "text": "That's army not navy... https://t.co/Wcqi0fvooy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378089495, "name": "garrett", "screen_name": "garrett_coneley", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "|Phs sophomore|vegas April 2nd|CR❤️|", "followers_count": 179, "friends_count": 258, "statues_count": 768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287081992192, "text": "@britNAYlew lol https://t.co/H3EeBCMDHe", "in_reply_to_status": -1, "in_reply_to_user": 408317624, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 408317624 }}, "user": { "id": 894597888, "name": "Christian McCrimon", "screen_name": "cTHEmac21", "lang": "en", "location": "Cali", "create_at": date("2012-10-20"), "description": "CA AZ NM\r\nInstagram & Snapchat: christianmc21\r\nHit that blue button", "followers_count": 298, "friends_count": 436, "statues_count": 7429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287140659200, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 56618077, "name": "sades", "screen_name": "lilsaddyy", "lang": "en", "location": "southern california", "create_at": date("2009-07-13"), "description": "this is ohio, if you don’t have a brewski in your hand you might as well be wearing a dress", "followers_count": 618, "friends_count": 198, "statues_count": 19089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287404879872, "text": "TRUUUUUE ! https://t.co/Wj2Tyu4CdF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 724909621, "name": "karinakusshh", "screen_name": "ayoo_karriina", "lang": "en", "location": "null", "create_at": date("2012-07-29"), "description": "null", "followers_count": 765, "friends_count": 849, "statues_count": 12596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverdale, CA", "id": "0028563c445e3c67", "name": "Riverdale", "place_type": "city", "bounding_box": rectangle("-119.878071,36.423728 -119.845512,36.438219") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 661096, "cityName": "Riverdale" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287853854720, "text": "Oh man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1571721110, "name": "Edwen Campos", "screen_name": "edwencampos", "lang": "en", "location": "null", "create_at": date("2013-07-05"), "description": "IRHS Class of 2016", "followers_count": 402, "friends_count": 354, "statues_count": 1413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oro Valley, AZ", "id": "23ddee699e919f28", "name": "Oro Valley", "place_type": "city", "bounding_box": rectangle("-111.02149,32.357649 -110.928016,32.477644") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287866421249, "text": "Can you find Juneau on the map? Just try it at https://t.co/HR27y9PgyJ #Juneau", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-134.42,58.3019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Juneau" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 312, "statues_count": 2562596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287950213121, "text": "I just want to cuddle and/or get my back massages", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1210799311, "name": "janvirr", "screen_name": "VirruetaJanice", "lang": "en", "location": "null", "create_at": date("2013-02-22"), "description": "null", "followers_count": 1037, "friends_count": 649, "statues_count": 27673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705287954464768, "text": "#xRoW #blackops3 #xboxone #snd https://t.co/evWB4dTWHt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "xRoW", "blackops3", "xboxone", "snd" }}, "user": { "id": 2319452523, "name": "Greg", "screen_name": "GregIsOFF1C1AL", "lang": "en", "location": "Bronx New York", "create_at": date("2014-02-01"), "description": "Gamertag ? I Am OFF1C1AL. I am The God of #xRoW. Creator 1/3 leader #XboxOverGaystation #NextGenandLastGen", "followers_count": 21, "friends_count": 80, "statues_count": 133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705288029835265, "text": "@Skinnyramirez �� I was lowkey dying inside when you told me lol savage", "in_reply_to_status": 693704320555077632, "in_reply_to_user": 940679911, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 940679911 }}, "user": { "id": 738039728, "name": "Sea_Shell", "screen_name": "Michelleeeeee06", "lang": "en", "location": "off on an adventure", "create_at": date("2012-08-04"), "description": "null", "followers_count": 267, "friends_count": 263, "statues_count": 6051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cicero, IL", "id": "50f296a1bd4198b6", "name": "Cicero", "place_type": "city", "bounding_box": rectangle("-87.77903,41.82158 -87.738571,41.865972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714351, "cityName": "Cicero" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705288638042113, "text": "I promise on me & everyone else that when I have kids ill never make them feel like they aren't loved", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1009016924, "name": "alexis taylor.", "screen_name": "lexxbrooke20", "lang": "en", "location": "null", "create_at": date("2012-12-13"), "description": "null", "followers_count": 501, "friends_count": 293, "statues_count": 8017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownfield, TX", "id": "f7bb6752fbd343e9", "name": "Brownfield", "place_type": "city", "bounding_box": rectangle("-102.309211,33.146988 -102.244024,33.202964") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48445, "countyName": "Terry", "cityID": 4810720, "cityName": "Brownfield" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705288784871425, "text": "@tinyyylex hey hey heeeyyy", "in_reply_to_status": 693704024625811456, "in_reply_to_user": 283327512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 283327512 }}, "user": { "id": 65236542, "name": "Veronica ⚾", "screen_name": "oorev_", "lang": "en", "location": "null", "create_at": date("2009-08-12"), "description": "long suffering Dodger fan, Comics, Video Games, Anime.", "followers_count": 780, "friends_count": 568, "statues_count": 55958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705288826814464, "text": "Happy birthday to my main bitch @HeatherWawee ������ https://t.co/MOAiWhH61l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52891530 }}, "user": { "id": 37073196, "name": "Kaylin", "screen_name": "KaylinHicks", "lang": "en", "location": "null", "create_at": date("2009-05-01"), "description": "Professional coffee drinker", "followers_count": 262, "friends_count": 147, "statues_count": 9670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705288923373568, "text": "Fuck my low tolerance ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177788953, "name": "Ty", "screen_name": "CantTY_Neshia", "lang": "en", "location": ".. The End ..", "create_at": date("2010-08-12"), "description": "Niyhland, Tyland", "followers_count": 1300, "friends_count": 643, "statues_count": 21646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-01-31T00:00:03.000Z"), "id": 693705289099403265, "text": "Add me on snapchat to watch all the stupid shit I do throughout my day https://t.co/iYaf4WNxn8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4296438972, "name": "kimbo", "screen_name": "kimberbutttt", "lang": "en", "location": "etx ", "create_at": date("2015-11-19"), "description": "no such thing as a life better than yours", "followers_count": 92, "friends_count": 88, "statues_count": 2839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705289556705280, "text": "Where is Juneau on the map? Play the game at https://t.co/HR27y9PgyJ #Juneau", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-134.42,58.3019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Juneau" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 312, "statues_count": 2562597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705289942470656, "text": "@ayanospivey I LOL'd", "in_reply_to_status": 693705053123678209, "in_reply_to_user": 258075548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 258075548 }}, "user": { "id": 313645505, "name": "Abigail", "screen_name": "AbbeyNatucci", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "Plenty of boyfriend jeans and no boyfriend aka life is rockin", "followers_count": 962, "friends_count": 633, "statues_count": 13464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Place, WA", "id": "9cd68e26f3c4880c", "name": "University Place", "place_type": "city", "bounding_box": rectangle("-122.584288,47.185226 -122.505193,47.245138") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5373465, "cityName": "University Place" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705290441576448, "text": "@rossR5 love u so much can u please retweet me♡ it would mean alot. https://t.co/1XLcD7Rvrt", "in_reply_to_status": -1, "in_reply_to_user": 154590712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 154590712 }}, "user": { "id": 3058501615, "name": "Brianna", "screen_name": "briannar5love", "lang": "en", "location": "L.A. California", "create_at": date("2015-03-02"), "description": "Hi my name is Brianna mason i am 4th oldest i am the only girl i got four bothers three older one younger an i love r5", "followers_count": 21, "friends_count": 153, "statues_count": 324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705290659696641, "text": "@Kari_Daddy awwwww ignore me less ...well thats the best thing any has said to me in the past 2 years....il take it", "in_reply_to_status": 693705057435451392, "in_reply_to_user": 596994429, "favorite_count": 0, "coordinate": point("-93.2690141,44.9782331"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 596994429 }}, "user": { "id": 239557615, "name": "THE A-TEAM", "screen_name": "pesty_prophet", "lang": "en", "location": "sota", "create_at": date("2011-01-17"), "description": "you heard the voice you saw the face u made the choice i go by the name of develle grier (DMG)", "followers_count": 395, "friends_count": 852, "statues_count": 32984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329148,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705290823249920, "text": "y'all watch this.. 1 of my top 5 dealing w race @MikeySuavae @baettyluuu @realalexiswoods @VivaLaSal @DJmillzz23 https://t.co/IFMn4Wd1im", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3747415634, 4728888055, 2459983201, 299479511, 360205481 }}, "user": { "id": 341312888, "name": "dr. kim reese ✨.", "screen_name": "brimarie423", "lang": "en", "location": "nola ➡️ nacghanistan ↔️ h-town", "create_at": date("2011-07-23"), "description": "bougie on a budget & a ratchet in recovery. @Uncommoncolors is the move. make it. university of houston. rest in paradise marie. #blacklivesmatter #UH18", "followers_count": 992, "friends_count": 476, "statues_count": 91056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705290970238976, "text": "Kevin gates so damn nasty ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513311808, "name": "K❤️", "screen_name": "theyl0vekee", "lang": "en", "location": "null", "create_at": date("2014-05-21"), "description": "keeprettyass", "followers_count": 1951, "friends_count": 982, "statues_count": 32367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705290995359744, "text": "I just #VotedforDjokovic to win the #AusOpen final https://t.co/UQam7KuFF9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VotedforDjokovic", "AusOpen" }}, "user": { "id": 774569184, "name": "Josh", "screen_name": "JoshSchrader22", "lang": "en", "location": "Norfolk, NE", "create_at": date("2012-08-22"), "description": "NHS '19 / 15 / Snapchat: joshschader22", "followers_count": 383, "friends_count": 403, "statues_count": 1611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, NE", "id": "96dc9cb6d51a3b23", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-97.46681,41.989118 -97.368029,42.061692") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31119, "countyName": "Madison", "cityID": 3134615, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705291137830914, "text": "HAPPY BIRTHDAY AMOR MIO! ❤️���� @_Markyymark Have a badass day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333935241 }}, "user": { "id": 274175091, "name": "Gabi ☁️", "screen_name": "CuddleMeCloudy", "lang": "en", "location": "Anaheim, CA", "create_at": date("2011-03-29"), "description": "16• Ateam • OGOC • Jc ☁️ • 1D • Hoodie Mob |-/\\ • Jc called me cute & great , Kian said he loves me. Skaterade", "followers_count": 3128, "friends_count": 2251, "statues_count": 117142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705291536314368, "text": "@midnight14k yea off tops smh", "in_reply_to_status": 693704706976124928, "in_reply_to_user": 2863674512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2863674512 }}, "user": { "id": 4020620534, "name": "Alex", "screen_name": "alexxg21", "lang": "en", "location": "Colton, CA", "create_at": date("2015-10-25"), "description": "19 | Manchester United | B | Broncos | 6OD | Lakers | THANK YOU FOR EVERYTHING KOBE", "followers_count": 282, "friends_count": 300, "statues_count": 15438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705291565641728, "text": "HAPPY BIRTHDAY BABY GIRL ����❤️�� https://t.co/dLGtVvzvDg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2266411207, "name": "Steven", "screen_name": "magdalenostev", "lang": "en", "location": "null", "create_at": date("2013-12-28"), "description": "Life is one Big Journey", "followers_count": 44, "friends_count": 32, "statues_count": 230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Bear Lake, CA", "id": "17fe6350a3570c69", "name": "Big Bear Lake", "place_type": "city", "bounding_box": rectangle("-116.962579,34.223549 -116.842586,34.262952") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 606434, "cityName": "Big Bear Lake" } }
+{ "create_at": datetime("2016-01-31T00:00:04.000Z"), "id": 693705291800666113, "text": "#GoToSeaWorld https://t.co/P2DSAINU0z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "GoToSeaWorld" }}, "user": { "id": 152422537, "name": "Johnny Obeid", "screen_name": "JohnnyObeid", "lang": "en", "location": "Laurel, Maryland", "create_at": date("2010-06-05"), "description": "Palestinian. Catholic. Laurel, MD the D[M]V Follower of Jesus Christ DIE HARD Redskins, Wizards, Capitals, Nationals fan! RIP ST21 aspiring sports broadcaster", "followers_count": 1574, "friends_count": 2400, "statues_count": 58791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel, MD", "id": "7ded7ee14cb871d2", "name": "Laurel", "place_type": "city", "bounding_box": rectangle("-76.889372,39.076292 -76.827105,39.115394") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445900, "cityName": "Laurel" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067660364062720, "text": "Nights like this with him.... ���� @ Brooklyn Bowl Las Vegas https://t.co/SBPeVICya0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17001246,36.11755276"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56766144, "name": "Yanira Aguilera", "screen_name": "yaguilera", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-07-14"), "description": "null", "followers_count": 21, "friends_count": 63, "statues_count": 692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067661831950336, "text": "Im actually sideways rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359883403, "name": "shang", "screen_name": "_frncsco", "lang": "en", "location": "NWCTA 16'", "create_at": date("2011-08-22"), "description": "greek god", "followers_count": 214, "friends_count": 161, "statues_count": 10332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067662209585152, "text": "i love ryosuke wtf is he even thinking half the time what the hell is his deal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2953043526, "name": "char char binks", "screen_name": "grandmayaoi", "lang": "en", "location": "chicago", "create_at": date("2014-12-30"), "description": "i'm sarah and i am the author of 13% of all existing shizaya breathplay. intp. virgo. problematic yaoi lesbian. #nsfw/#gore. (she/her) priv @spidersgeorgina", "followers_count": 382, "friends_count": 170, "statues_count": 83673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067662339461120, "text": "You sayin niggas COMIN on too strong girl, They want you as a wife.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458184523, "name": "Brie", "screen_name": "Impetuousbrie", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-01-08"), "description": "null", "followers_count": 438, "friends_count": 358, "statues_count": 11808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067662532399104, "text": "Wind 3.0 mph NE. Barometer 1017.07 mb, Falling. Temperature 41.2 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067662532509696, "text": "#GRANDNATIONALROADSTERSHOW #GNRS2016 #MERICA !!! #HOTROD s & #AMERICA #GNRS #americanstyle… https://t.co/XD7V4fGKhY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.76045082,34.08758495"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "GRANDNATIONALROADSTERSHOW", "GNRS2016", "MERICA", "HOTROD", "AMERICA", "GNRS", "americanstyle" }}, "user": { "id": 1959919483, "name": "Lukasz Granicy", "screen_name": "NTKUSTOMS", "lang": "en", "location": "Lancaster, California", "create_at": date("2013-10-13"), "description": "Here at NEEDFULTHINGZ we build eyecandy for the automotive art world ..... You name it we can do it customizing vehicles from the ground up", "followers_count": 97, "friends_count": 344, "statues_count": 843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067662544990208, "text": "Temp: 55.4F W C: 53.2F Wind:NNW at 7.2kts Baro: 1014.5mb and Rising slowly Rain today: 0.00in R H: 82% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 185, "friends_count": 219, "statues_count": 96063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067663006339073, "text": "Ugh school tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1344446750, "name": "Alejandra", "screen_name": "UnlikeAle", "lang": "en", "location": "Hollywood ", "create_at": date("2013-04-11"), "description": "im probably taking a nap if im offline", "followers_count": 8928, "friends_count": 269, "statues_count": 43757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067663010549762, "text": "@x_VintageBesos you seen batman bad blood yet?", "in_reply_to_status": 694067065783627776, "in_reply_to_user": 29523359, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29523359 }}, "user": { "id": 320884439, "name": "Anti-Filth Ranger", "screen_name": "313doe", "lang": "en", "location": "Should Be In Detroit", "create_at": date("2011-06-20"), "description": "Just a Detroit nigga living in Tx, If you reading this you a ham #IfItAintFoodIAintEatingIt", "followers_count": 3199, "friends_count": 3097, "statues_count": 163768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067663090233344, "text": "The set my dad just finished ���� https://t.co/gkncEUyzqK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344568779, "name": "tayb$", "screen_name": "taybsoren907", "lang": "en", "location": "Anchorage, Alaska❄️⛄️", "create_at": date("2011-07-29"), "description": "snapchat: sorensontaylor", "followers_count": 846, "friends_count": 676, "statues_count": 34899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067663090257921, "text": "Maxwell is forever a legend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190093478, "name": "RIP JoJo", "screen_name": "OneSeanS", "lang": "en", "location": "IN THE FRIEND ZONE, CA", "create_at": date("2010-09-12"), "description": "When God created this man, he got lazy and said fuck a hairline. Now here I am. Dm's closed forever", "followers_count": 2623, "friends_count": 988, "statues_count": 97932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067663690051584, "text": "If I'm being totally honest because I can do that with Twitter I'm totally no okay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24985536, "name": "Annabeth", "screen_name": "ItsAnnabethhh", "lang": "en", "location": "Kentucky", "create_at": date("2009-03-17"), "description": "snap:anna.elizabeth2 IG:annabethhh21 fitnessfromannabeth@yahoo.com", "followers_count": 1055, "friends_count": 874, "statues_count": 3129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-02-01T00:00:00.000Z"), "id": 694067664373714944, "text": "Happy Birthday �� \n@awesomedaiisy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1557758623 }}, "user": { "id": 4219655412, "name": "Çhampagne PÀPÏ", "screen_name": "agar2300", "lang": "en", "location": "$hmopcity", "create_at": date("2015-11-12"), "description": "$hmop Boyzzzzz", "followers_count": 483, "friends_count": 620, "statues_count": 4392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067664499511296, "text": "@ebbtideapp Tide in Folly Creek, Virginia 02/01/2016\n Low 8:31am 0.4\nHigh 2:41pm 2.5\n Low 8:44pm 0.2\nHigh 3:21am 2.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.635,37.6967"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 20, "friends_count": 1, "statues_count": 3976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51001, "countyName": "Accomack" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067664621146112, "text": "Wth is the problem", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1675965834, "name": "livin like larry", "screen_name": "Ian_bolleurs", "lang": "en", "location": "null", "create_at": date("2013-08-16"), "description": "HELLABANDZ sc : ianbolleurs", "followers_count": 150, "friends_count": 185, "statues_count": 1010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067664684056576, "text": "@ThatGuyRick1 Are you on your soapbox tonight?", "in_reply_to_status": -1, "in_reply_to_user": 1069226227, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1069226227 }}, "user": { "id": 54813021, "name": "✨Sugar Doll Pixie✨", "screen_name": "Stellagirl72", "lang": "en", "location": "Lost In The Desert!", "create_at": date("2009-07-07"), "description": "Music is the air I breathe! There is magic in everything you just need to believe. Red Headed Pixie Devil Egnimatic conundrum sent to confuse you PS I love you!", "followers_count": 14723, "friends_count": 14762, "statues_count": 37376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.570794,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067664851857408, "text": "@CTZN5 @pmarca @BlkSails_Starz That is, more interesting to Scotch drnkrs who want something dfrnt than to those who want to explore scotch.", "in_reply_to_status": 694065061640347649, "in_reply_to_user": 279697896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 279697896, 5943622, 1410511280 }}, "user": { "id": 276266868, "name": "Jeremy Skog", "screen_name": "jeremyskog", "lang": "en", "location": "null", "create_at": date("2011-04-02"), "description": "Ph.D. Economist attempting to learn about the world, one human interaction at a time. My hobby is communicating empirical economics as art via visualizations.", "followers_count": 115, "friends_count": 424, "statues_count": 2520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, VA", "id": "d6819fe60643ebc1", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-77.144435,38.789907 -77.037304,38.844853") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5101000, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067664965103616, "text": "@nandi100_ u think I'm playing in serious", "in_reply_to_status": 694067556747874306, "in_reply_to_user": 803679781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 803679781 }}, "user": { "id": 2385300475, "name": "Jakil Sarhim", "screen_name": "KoostoKnows_", "lang": "en", "location": "Oakland CA", "create_at": date("2014-03-12"), "description": "The Official Twitter Of Jakil Sarhim || All Glory To God | Pay Me In Loyalty || #CallGod || O.L || Tunnel Vision || #JucoProduct ⭐️⭐️⭐️ || Brooklynn N. Sarhim", "followers_count": 758, "friends_count": 741, "statues_count": 3778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665044795392, "text": "SONIC Drive-In #Hospitality #Job: Cook (#Sedalia, MO) https://t.co/a1cmMkOiAe #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2308822,38.7045975"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Sedalia", "Jobs", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 214, "friends_count": 39, "statues_count": 19197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sedalia, MO", "id": "0aa49830dc7e7320", "name": "Sedalia", "place_type": "city", "bounding_box": rectangle("-93.311072,38.666969 -93.149907,38.728524") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29159, "countyName": "Pettis", "cityID": 2966440, "cityName": "Sedalia" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665153847296, "text": "32 days till I get to see my man!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2523564811, "name": "♏️åñdęrž", "screen_name": "Manderz_8p", "lang": "en", "location": "check out my channel ⤵️", "create_at": date("2014-05-25"), "description": "Senior. YouTuber. Cheerleader. OHS 16'. Brenton is Babe. 1-17-15.", "followers_count": 236, "friends_count": 196, "statues_count": 9365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665392930816, "text": "@OVO_MAJI13 you told me you dating a girl though are you not", "in_reply_to_status": 694067586359808000, "in_reply_to_user": 2221936274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2221936274 }}, "user": { "id": 1285142904, "name": "ㅤ", "screen_name": "TreMendel", "lang": "en", "location": "San Marcos, CA", "create_at": date("2013-03-20"), "description": "Rest Easy Tristian.", "followers_count": 448, "friends_count": 775, "statues_count": 8456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, CA", "id": "a2c84129f9dcf69f", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-117.230172,33.090761 -117.103461,33.186722") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 668196, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665485352960, "text": "03:00:01 |Temp: 48.6ºF | Wind Chill 48.6ºF |Dew Point 45.5ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 89305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665795706881, "text": "02/01@03:00 - Temp 40.0F, WC 40.0F. Wind 0.6mph SW, Gust 2.0mph. Bar 29.808in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665816584192, "text": "Join the team! See our latest #Manufacturing #job opening here: https://t.co/RfEnl3rV5H #Knoxville, TN #Hiring https://t.co/z85OfKp9DE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.9207392,35.9606384"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Manufacturing", "job", "Knoxville", "Hiring" }}, "user": { "id": 3981286753, "name": "fulltimeGiGS Jobs", "screen_name": "FTGiGSJobs", "lang": "en", "location": "Saint Simons Island, GA", "create_at": date("2015-10-22"), "description": "All #jobs all the time! Apply today! Follow our official account @fulltimeGiGS for #jobsearch tips and career advice.", "followers_count": 153, "friends_count": 73, "statues_count": 3916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665829167104, "text": "Happy February.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385259026, "name": "Bam Bam Bolu", "screen_name": "1BOLU", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-10-04"), "description": "just another kid who made it out the NO. City of the Rose.", "followers_count": 1851, "friends_count": 378, "statues_count": 10433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067665887887360, "text": "I Think She Meant Bald But Okaay ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 731277078, "name": "Deniseee", "screen_name": "Denisee____", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "F.I.P Baby Spoke ~ 19 ♍️", "followers_count": 276, "friends_count": 188, "statues_count": 13063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067666034647041, "text": "@kingjinxo youre lying. so youre a 25 year old hispanic (or white) chick from Harlem", "in_reply_to_status": 694067294700343296, "in_reply_to_user": 230846064, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 230846064 }}, "user": { "id": 36794484, "name": "Lade", "screen_name": "Donlade", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "null", "followers_count": 3807, "friends_count": 200, "statues_count": 10024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067666085027840, "text": "Blame it on the weather, but I'm a mess and this February darkness has me hating everyone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57830338, "name": "Zeth", "screen_name": "iridescentaf", "lang": "en-gb", "location": "null", "create_at": date("2009-07-17"), "description": "snapchat: hereislizeth", "followers_count": 839, "friends_count": 505, "statues_count": 65346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067666521165824, "text": "Time:02:00 Temp:72.1°F Dewpt:71.6°F Press:29.83in. Wind N at 8mph gusting 12mph Rain Today 0.00in. Humidity:99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.60305556,27.84555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448017712, "name": "AnnavilleWeather", "screen_name": "AnnavilleWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2011-12-27"), "description": "live weather from Annaville, Texas! FMI see http://www.wunderground.com/weatherstation/WXDailyHistory.asp?ID=KTXCORPU31", "followers_count": 86, "friends_count": 395, "statues_count": 18253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067666697359361, "text": "Feb. 1 is my new years", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2934144871, "name": "pimpcess", "screen_name": "Illluminasty", "lang": "en", "location": "oakland ca", "create_at": date("2014-12-17"), "description": "ig ; illluminasty", "followers_count": 476, "friends_count": 563, "statues_count": 4261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067667083399168, "text": "Wind 0 mph --. Barometer 29.98 in, Steady. Temperature 63.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067667355983872, "text": "Girls who compliment each other can be my friends.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1232683842, "name": "Mulan", "screen_name": "amiikins", "lang": "en", "location": "on your mind ", "create_at": date("2013-03-01"), "description": "Do things with passion or not at all.", "followers_count": 1141, "friends_count": 452, "statues_count": 69641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067667397931008, "text": "*me talkin bout my bitches and my titties* https://t.co/UqQFZBBIF5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1378443924, "name": "vanessa", "screen_name": "vpdollarsign", "lang": "en", "location": "ny", "create_at": date("2013-04-24"), "description": "i pour my milk first ♊️", "followers_count": 3125, "friends_count": 816, "statues_count": 21575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067668417015808, "text": "Gotta wish my dude @phillipkyyle happy birthday one time, have a good one and stay up G��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1025309492 }}, "user": { "id": 1545049088, "name": "Jae", "screen_name": "JaevanteV", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "c/o '17", "followers_count": 491, "friends_count": 413, "statues_count": 6270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-02-01T00:00:01.000Z"), "id": 694067668438003712, "text": "Everytime the school year ends i never turn in books ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2251278259, "name": "ahtee", "screen_name": "ashuhlay_", "lang": "en", "location": "Sac To SoCal ", "create_at": date("2013-12-17"), "description": "SC/alrgt #uce afns", "followers_count": 1159, "friends_count": 985, "statues_count": 37417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067668828164096, "text": "I said the truth now I'm hated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454579128, "name": "Rated R", "screen_name": "yesmiyah", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-01-03"), "description": ":::I'm just a girl born into this world:::", "followers_count": 1265, "friends_count": 1149, "statues_count": 8949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067668932911104, "text": "I'm sleeeeep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392886896, "name": "shauns", "screen_name": "PayneShaunee", "lang": "en", "location": "palm springs/blythe", "create_at": date("2011-10-17"), "description": "#fingersinthebootyassbitch", "followers_count": 826, "friends_count": 694, "statues_count": 29000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.570794,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067668966572033, "text": "Left my damn purse & charger in her car ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560413363, "name": "Selenalinn", "screen_name": "selenalinn_", "lang": "en", "location": "Football is life, TX", "create_at": date("2012-04-22"), "description": "If you ain't talking football, i don't want to talk. TEXAS FOREVER @Broncos | #TCU", "followers_count": 2189, "friends_count": 263, "statues_count": 107955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067669180485632, "text": "Wind 2.0 mph NNW. Barometer 29.853 in, Rising. Temperature 50.4 °F. Rain today 0.01 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067669625085952, "text": "Wind 0.0 mph ---. Barometer 29.942 in, Steady. Temperature 42.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067669813710848, "text": "@Harry_Styles wow u r 22", "in_reply_to_status": -1, "in_reply_to_user": 181561712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181561712 }}, "user": { "id": 585324236, "name": "liz", "screen_name": "indigodyedhes", "lang": "en", "location": "PV, LA", "create_at": date("2012-05-19"), "description": "u fookin losah", "followers_count": 13853, "friends_count": 5422, "statues_count": 68992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067669922742275, "text": "Happy birthday @Harry_Styles I love you sooooo much❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181561712 }}, "user": { "id": 377826931, "name": "jazmine", "screen_name": "ysImalik", "lang": "en", "location": "LOS ANGELES ", "create_at": date("2011-09-21"), "description": "#pillowtalk", "followers_count": 1014, "friends_count": 749, "statues_count": 14378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067670036127745, "text": "It's been a truly refreshing & encouraging week of prayer & fasting with our Calvary Family. I'm… https://t.co/puvZASLFEL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.95987959,45.90005631"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 121077678, "name": "Jake Moevao", "screen_name": "J_MOEVAO", "lang": "en", "location": "Corvallis, OR", "create_at": date("2010-03-08"), "description": "Devoted follower of CHRIST. Blessed husband. Loyal family man. Samoan SoCal native. Passionate leader. Sports enthusiast. High School Pastor @ CalvaryCorvallis.", "followers_count": 398, "friends_count": 159, "statues_count": 3578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon, USA", "id": "df7fd3a3b9eff7ee", "name": "Oregon", "place_type": "admin", "bounding_box": rectangle("-124.703541,41.991795 -116.463262,46.2991") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41007, "countyName": "Clatsop", "cityID": 4110850, "cityName": "Cannon Beach" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067670103162880, "text": "Temp: 63.1°F Wind:0.0mph Pressure: 29.933hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067670501560323, "text": "#HHgang https://t.co/iZGSstKIqi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HHgang" }}, "user": { "id": 377219906, "name": "fat", "screen_name": "bigSAMOANkid", "lang": "en", "location": "LGB✈️SEA", "create_at": date("2011-09-20"), "description": "snapchat : bigsamoankid", "followers_count": 896, "friends_count": 696, "statues_count": 25408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tukwila, WA", "id": "9883a4be0cbd497c", "name": "Tukwila", "place_type": "city", "bounding_box": rectangle("-122.318673,47.407147 -122.241356,47.531961") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5372625, "cityName": "Tukwila" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067670551961603, "text": "Life is full of lessons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4229994979, "name": "Gustavo Castillo", "screen_name": "Tavo2Down", "lang": "en", "location": "null", "create_at": date("2015-11-13"), "description": "null", "followers_count": 66, "friends_count": 107, "statues_count": 512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067670816141312, "text": "Democrats' closing pitches in Iowa: 'Hard choices' vs. 'Revolution' https://t.co/oFduJbMAkg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17626, "friends_count": 17573, "statues_count": 67827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067671449653248, "text": "Wind 0.0 mph WSW. Barometer 29.62 in, Falling. Temperature 41.2 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 106, "statues_count": 157126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067671654989824, "text": "������������ https://t.co/RjW2nltK21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 231737563, "name": "Sophia", "screen_name": "sowpheea", "lang": "en", "location": "California, USA", "create_at": date("2010-12-29"), "description": "femme fatale", "followers_count": 513, "friends_count": 368, "statues_count": 5507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067672011550720, "text": "@claudiabia 8 REASONS WHY TO VISIT RIO DE JANEIRO, BRAZIL!\nhttps://t.co/5bJ0SklNAw", "in_reply_to_status": -1, "in_reply_to_user": 17628469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17628469 }}, "user": { "id": 19210349, "name": "ARTISTS PR", "screen_name": "ARTISTSPR", "lang": "en", "location": "Beverly Hills,California ", "create_at": date("2009-01-19"), "description": "I am PR of Artist / Philanthropist METIN BEREKETLI \nhttp://WWW.HOLLYWOODPAINTER.COM", "followers_count": 592, "friends_count": 2006, "statues_count": 11559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, CA", "id": "741e9df4d2522275", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-118.427344,34.052559 -118.371934,34.112434") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 606308, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2016-02-01T00:00:02.000Z"), "id": 694067672162504704, "text": "why did I take a nap earlier ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325386603, "name": "BE GREAT", "screen_name": "bangbro_dk", "lang": "en", "location": "w my bangbros", "create_at": date("2011-06-27"), "description": "rest easy unc an david", "followers_count": 382, "friends_count": 198, "statues_count": 4727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067673290788864, "text": "S/O to all my folx in #GREASElive tonight -- @Yvette_G_Nacer @KekePalmer & all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GREASElive" }}, "user_mentions": {{ 491886158, 29860364 }}, "user": { "id": 39013378, "name": "JIMMY R.O. SMITH", "screen_name": "J_R_O_S", "lang": "en", "location": "LA", "create_at": date("2009-05-10"), "description": "booking@JimmyROSmith.com http://www.BlocAgency.com", "followers_count": 8946, "friends_count": 1143, "statues_count": 18357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ladera Heights, CA", "id": "714789cf3b7a50d0", "name": "Ladera Heights", "place_type": "city", "bounding_box": rectangle("-118.391088,33.97632 -118.357614,34.014937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639108, "cityName": "Ladera Heights" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067673437569024, "text": "Lmao nah https://t.co/RQ9AgWGVl4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2290415514, "name": "tyra bank$", "screen_name": "hboogie1x", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-13"), "description": "I'm tall & I have freckles. navy strong ⚓️", "followers_count": 811, "friends_count": 337, "statues_count": 53179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067673819418624, "text": "@VICKyoudown yes ������ https://t.co/hjAfkOhDpm", "in_reply_to_status": 694067424849760256, "in_reply_to_user": 337226732, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 337226732 }}, "user": { "id": 246479170, "name": "Melanie", "screen_name": "MelloJelloShots", "lang": "en", "location": "null", "create_at": date("2011-02-02"), "description": "SENIOR at ISU. Vice President of SASSI. Enough drive to drive you crazy. I promote NATURAL hair. #JCole #Aries #BAMA", "followers_count": 605, "friends_count": 402, "statues_count": 30722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067673953480704, "text": "@TheYoungElijah HAPPY B DAY FAM!!!! https://t.co/GJHDzzAKPT", "in_reply_to_status": -1, "in_reply_to_user": 150127297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150127297 }}, "user": { "id": 248456332, "name": "Mazee Cascade", "screen_name": "Mazee24", "lang": "en", "location": "null", "create_at": date("2011-02-06"), "description": "Central Washington University. P O S I T I V E V I B ES @mai_izzle", "followers_count": 696, "friends_count": 379, "statues_count": 65887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellensburg, WA", "id": "c95cdb2a983262e5", "name": "Ellensburg", "place_type": "city", "bounding_box": rectangle("-120.582586,46.958017 -120.49726,47.028542") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53037, "countyName": "Kittitas", "cityID": 5321240, "cityName": "Ellensburg" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067674955907072, "text": "This girl made me shed a tear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3187101938, "name": "MJ", "screen_name": "Kgoat23", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "just a restless man waiting for his chance", "followers_count": 441, "friends_count": 402, "statues_count": 1004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675236999168, "text": "��- your hella cool I always mess with you just to make you mad lol, but I needed to tell you something but forgot you @ name once", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 991493060, "name": ".", "screen_name": "Ayee_marquies", "lang": "en", "location": "null", "create_at": date("2012-12-05"), "description": "Don't Mind Me I'm On My Own Hype", "followers_count": 726, "friends_count": 732, "statues_count": 9203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675501211649, "text": "our power is out, I feel like I'm camping cause I need a flashlight just to go to the restroom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346433787, "name": "i⚡️abela", "screen_name": "unpredictabela", "lang": "en", "location": " University of Arizona ", "create_at": date("2011-08-01"), "description": "blue haired dweeb ✌️", "followers_count": 863, "friends_count": 837, "statues_count": 21485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675585077248, "text": "@therealscribe ��", "in_reply_to_status": 694067112550092800, "in_reply_to_user": 563882915, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 563882915 }}, "user": { "id": 211764172, "name": "Californian Coon", "screen_name": "ThePoet_", "lang": "en", "location": "Po-Po-Po-Posted in the City", "create_at": date("2010-11-03"), "description": "Poet Jackson. Indie Hip-hop artist. Song Writer. Art Lover. Eastsider. Prinicipality coming soon.", "followers_count": 337, "friends_count": 314, "statues_count": 19960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675610386432, "text": "Cuz da Moon where Im at ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150049183, "name": "Rome Valentine", "screen_name": "XOutDa_Lames", "lang": "en", "location": "null", "create_at": date("2010-05-30"), "description": "90S BBY SC: Uoeno_Rome DTX To HTX", "followers_count": 1218, "friends_count": 943, "statues_count": 32493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshall, TX", "id": "0139a7c035c529cd", "name": "Marshall", "place_type": "city", "bounding_box": rectangle("-94.406743,32.491967 -94.29016,32.571239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48203, "countyName": "Harrison", "cityID": 4846776, "cityName": "Marshall" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675622875137, "text": "If DJ Khaled has enough time to water all of his plants everyday then you have time to watch that next episode", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354090049, "name": "Ethan Ayers", "screen_name": "UnbeatenEthan", "lang": "en", "location": "Corvallis, Oregon", "create_at": date("2011-08-12"), "description": "Self-proclaimed hero. Male model. I pay my student loans with retweets and likes. Oregon State University. Brother of Theta Chi Sigma Chapter (ΘΧ)", "followers_count": 3545, "friends_count": 676, "statues_count": 19787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675811573760, "text": "H A P P Y G O D D A M N B I R T H D A Y H A R R Y E D W A R D S T Y L E S���� ily bby�� 2•1•16 HE IS PERFECTION�� https://t.co/qePBXLk90r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 559372323, "name": "K H A R E C E✨", "screen_name": "__kharece", "lang": "en", "location": "Cheney, WA", "create_at": date("2012-04-21"), "description": "Miley, what's good?", "followers_count": 283, "friends_count": 487, "statues_count": 25092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.60765,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675895443456, "text": "@ me bro https://t.co/MVjMyBRdqp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2429731950, "name": "Angle", "screen_name": "314494", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": "lost in the sauce", "followers_count": 248, "friends_count": 78, "statues_count": 19009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067675933208577, "text": "#Vibes �� https://t.co/OZsaCmlL0e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Vibes" }}, "user": { "id": 2254683367, "name": "Galvan", "screen_name": "ivangee13", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "Earn every expectation⚽️ #Footballer", "followers_count": 600, "friends_count": 470, "statues_count": 14451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windsor, CA", "id": "cf574ccb037d4ff3", "name": "Windsor", "place_type": "city", "bounding_box": rectangle("-122.83677,38.515011 -122.772142,38.566956") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 685922, "cityName": "Windsor" } }
+{ "create_at": datetime("2016-02-01T00:00:03.000Z"), "id": 694067676847669248, "text": "I'm a hoe now lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156499634, "name": "Don.", "screen_name": "feelMyAmbition_", "lang": "en", "location": "#AUM19", "create_at": date("2010-06-16"), "description": "she believed she could so she did...", "followers_count": 1400, "friends_count": 1180, "statues_count": 56296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-88.446362,32.284593 -84.296982,41.744901") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067677183107072, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3196405038, "name": "fatima", "screen_name": "kl4xons", "lang": "en", "location": "LA, LBC, CALIFORNIA", "create_at": date("2015-05-15"), "description": "adventures and coffee and art and anything early 2000's is me", "followers_count": 194, "friends_count": 85, "statues_count": 8739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067677187297280, "text": "Happy birthday my favorite bih ���� I love you sister we can't hangout anymore because you're 16 and I hate you. https://t.co/RMfkXq097a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2652547202, "name": "february 3rd", "screen_name": "b_asianna", "lang": "en", "location": "null", "create_at": date("2014-07-16"), "description": "black / african american ❥ J.Huitron is my everything", "followers_count": 452, "friends_count": 419, "statues_count": 5215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasco, CA", "id": "01643e7e5fce28b7", "name": "Wasco", "place_type": "city", "bounding_box": rectangle("-119.42052,35.572513 -119.328147,35.610926") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 683542, "cityName": "Wasco" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067677287968768, "text": "omebdjsjsjsjsmwks https://t.co/FmlocOzISv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2346065834, "name": "dj jazzy jazz", "screen_name": "jasminenarayan", "lang": "en", "location": "bay area", "create_at": date("2014-02-15"), "description": "♡ INFP // gemini ♡", "followers_count": 582, "friends_count": 455, "statues_count": 17004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067677933891586, "text": "@kb100__ porque tengo sueño ��", "in_reply_to_status": 694067023375003648, "in_reply_to_user": 3244766725, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3244766725 }}, "user": { "id": 732103069, "name": "Jose Pena", "screen_name": "Youngpena13", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "null", "followers_count": 175, "friends_count": 245, "statues_count": 496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067678340894720, "text": "If I feel like I'm in the friend zone I dead ass ignore that chick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24774663, "name": "flex luthor", "screen_name": "lugunna", "lang": "en", "location": "batmans city ... #freebanz", "create_at": date("2009-03-16"), "description": "I got numbers all on me .. cause I count on myself ..... snapchat : luthermcfire", "followers_count": 792, "friends_count": 523, "statues_count": 116407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067678470778881, "text": "\"Que la locura más cuerda es buscar cómo ser libre: creo en lo imposible.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 126677484, "name": "Alma Villanueva", "screen_name": "Paalmiis", "lang": "es", "location": "null", "create_at": date("2010-03-26"), "description": "Don't mess with my dreams", "followers_count": 86, "friends_count": 544, "statues_count": 1783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067678584025088, "text": "#HBStyleon10 AND #HBSweeps https://t.co/EO0HPSL4f8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HBStyleon10", "HBSweeps" }}, "user": { "id": 36143299, "name": "Rajee", "screen_name": "momsfocus", "lang": "en", "location": "Texas, USA", "create_at": date("2009-04-28"), "description": "Make sure to follow us\r\nRT @momsfocus sent me to win Dékor & a year supply of @diaperdekor refills #DiaperDékor Rules: http://shar.es/21aSH #baby #parents", "followers_count": 18085, "friends_count": 19889, "statues_count": 336009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679204872192, "text": "I'm behind on so much music shit a joke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3718908809, "name": "freeJUWELLZ", "screen_name": "UptownCoolie", "lang": "en", "location": "UPTOWN BX // WESTEND BPT", "create_at": date("2015-09-20"), "description": "JUST DIFFERENT❕#DW4L Link^everytingCoolie@gmail.com", "followers_count": 62, "friends_count": 38, "statues_count": 2213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679280300032, "text": "03:00 42.1°F Feels:39.8°F (Hi45.9°F/Lo42.1°F) Hum:73% Wnd:N 3.8MPH Baro:29.88in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 177, "friends_count": 265, "statues_count": 25250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679544487936, "text": "Happy New Years bitches������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2896258375, "name": "b a b y", "screen_name": "queennnabby", "lang": "en", "location": "Bay Area, CA", "create_at": date("2014-11-10"), "description": "null", "followers_count": 575, "friends_count": 290, "statues_count": 15494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679569670144, "text": "Wind 1.0 mph NW. Barometer 29.954 in, Rising Rapidly. Temperature 30.2 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 198, "friends_count": 58, "statues_count": 247621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679636795392, "text": "Wind 4.0 mph WSW. Barometer 29.954 in, Rising slowly. Temperature 32.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 11, "statues_count": 808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679657750528, "text": "@C88_Robosnocone @Nobleknows ���������������� so good", "in_reply_to_status": 694067086230818816, "in_reply_to_user": 2160673730, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2160673730, 258698725 }}, "user": { "id": 39547783, "name": "Rukari Austin", "screen_name": "rukizzel", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-12"), "description": "Community Manager working on @KillerInstinct, @HoloLens & @MSStudiosBlog. Formerly @Xbox, @Halo, & @AgeofEmpires, though really I am EVERYWHERE!", "followers_count": 3284, "friends_count": 797, "statues_count": 62929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downtown, San Antonio", "id": "217265542171964b", "name": "Downtown", "place_type": "neighborhood", "bounding_box": rectangle("-98.501685,29.41201 -98.473253,29.440561") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067679993421825, "text": "@NicThaDon I'm im Huntsville, & that's not far. I go through it sometimes.", "in_reply_to_status": 694067577295867904, "in_reply_to_user": 276640084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 276640084 }}, "user": { "id": 367492957, "name": "alicia lane☄", "screen_name": "uhlishalayne", "lang": "en", "location": "null", "create_at": date("2011-09-03"), "description": "i'm 18 so if i eat ur pussy you won't go to jail for it. we legal baby #80115 ❤️", "followers_count": 2207, "friends_count": 1810, "statues_count": 122303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067680198815745, "text": "Coleey thinks he's funny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762270950, "name": "CHELDAWG", "screen_name": "ayeeitschelsiee", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-08-16"), "description": "cup full, yea im on", "followers_count": 550, "friends_count": 376, "statues_count": 35127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067680425308161, "text": "February!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2153663768, "name": "Aaron Ward", "screen_name": "ayee_cee14", "lang": "en", "location": "Montgomery, AL", "create_at": date("2013-10-24"), "description": "LBHS Graduate Alumni c/o '14 | SCC| GK| Music Major| Mary Dorantes (: ♥| Alabama Life", "followers_count": 1111, "friends_count": 2130, "statues_count": 15233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067681050390528, "text": "@thegamersjoint wtf is this shit ?!!! https://t.co/F7y7SneB6P", "in_reply_to_status": -1, "in_reply_to_user": 583530527, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 583530527 }}, "user": { "id": 3193610095, "name": "MIKHAIL", "screen_name": "DECEMBERKING96", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "enlightened.", "followers_count": 69, "friends_count": 65, "statues_count": 2081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-01T00:00:04.000Z"), "id": 694067681125752832, "text": "�� https://t.co/BErdJDHTF0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1870044829, "name": "DAMIAN ☔️", "screen_name": "Damian_tho", "lang": "en", "location": "null", "create_at": date("2013-09-15"), "description": "#L", "followers_count": 903, "friends_count": 589, "statues_count": 53358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067681650040832, "text": "Keep em all on the need to know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 508478696, "name": "jaeeelii", "screen_name": "Jaeeelii__", "lang": "en", "location": "Probably in a Sephora", "create_at": date("2012-02-28"), "description": "Gabriel Paco Lopez ❤10.05.97-08.22.14❤️", "followers_count": 800, "friends_count": 232, "statues_count": 57024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067681700347904, "text": "Give me never ending back rubs till I fall asleep & you'll have my heart", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3731097135, "name": "bina", "screen_name": "Sabrina6xx", "lang": "en", "location": "socal", "create_at": date("2015-09-21"), "description": "girls", "followers_count": 281, "friends_count": 276, "statues_count": 1354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067682589585408, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2615906629, "name": "uber ant", "screen_name": "remember_lilAnt", "lang": "en", "location": "back to the trap", "create_at": date("2014-07-10"), "description": "took ya hoe cause she gave me a boner", "followers_count": 360, "friends_count": 471, "statues_count": 11427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Puente, CA", "id": "e04653ebc39b100c", "name": "La Puente", "place_type": "city", "bounding_box": rectangle("-117.98558,34.009223 -117.920899,34.057298") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640340, "cityName": "La Puente" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067683147395074, "text": "Preach brotha �������� https://t.co/g4tbFolfcG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538210713, "name": "Christina Orton", "screen_name": "ChristinaOrton", "lang": "en", "location": "null", "create_at": date("2012-03-27"), "description": "my boyfriend is adorable.", "followers_count": 177, "friends_count": 72, "statues_count": 4524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ogden, UT", "id": "4980bc1544652d22", "name": "Ogden", "place_type": "city", "bounding_box": rectangle("-112.025874,41.149594 -111.904265,41.286003") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4955980, "cityName": "Ogden" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067683420057602, "text": "Watch the all star game https://t.co/oYGpUUfcB5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151197457, "name": "-", "screen_name": "TwanHarris4", "lang": "en", "location": "Houston, TX", "create_at": date("2010-06-02"), "description": "sc: twanharris", "followers_count": 3996, "friends_count": 2598, "statues_count": 82730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067683478745088, "text": "He really don't have anything to ever say to me again ✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630827754, "name": "Feb.6 ❣", "screen_name": "_nashayy", "lang": "en", "location": "null", "create_at": date("2012-07-08"), "description": "Tot ~", "followers_count": 2400, "friends_count": 2205, "statues_count": 15214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067683642490880, "text": "https://t.co/79HhJo9Su7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 262271606, "name": "W⭐P", "screen_name": "PHLYWADE", "lang": "en", "location": "WORLD WIDE", "create_at": date("2011-03-07"), "description": "God is Awesome. 26 80's baby World Traveler. College Graduate. Born & Raised in Maryland Lives in Florida. RAVENS 2x champs HEAT 3x champs O'$ 3x champs W⭐P", "followers_count": 1282, "friends_count": 1139, "statues_count": 36003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heathrow, FL", "id": "5087d9a47d575e00", "name": "Heathrow", "place_type": "city", "bounding_box": rectangle("-81.385151,28.756135 -81.356599,28.793459") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1229320, "cityName": "Heathrow" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067683701170176, "text": "@imJustDarv you always have twitter jokes", "in_reply_to_status": 694067566998884354, "in_reply_to_user": 393816368, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393816368 }}, "user": { "id": 860423396, "name": "Z", "screen_name": "materialistix", "lang": "en", "location": "FL✈️AL", "create_at": date("2012-10-03"), "description": "Sunshine. Afrocentric. Spirit. #Skegee18", "followers_count": 340, "friends_count": 263, "statues_count": 18046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuskegee, AL", "id": "28df0fa3673b5b49", "name": "Tuskegee", "place_type": "city", "bounding_box": rectangle("-85.747927,32.386278 -85.663058,32.460448") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1087, "countyName": "Macon", "cityID": 177304, "cityName": "Tuskegee" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067684254715904, "text": "@EGPWonderChef Toxic Fusion is the clan name", "in_reply_to_status": 694064862570397698, "in_reply_to_user": 99217397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99217397 }}, "user": { "id": 1488730837, "name": "Xieon", "screen_name": "ProjectXieon", "lang": "en", "location": "Berkeley, CA", "create_at": date("2013-06-06"), "description": "Youtuber, Streamer, Plays Fighting Games, and Do Walkthroughs!", "followers_count": 131, "friends_count": 576, "statues_count": 1869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067684783165444, "text": "Hey @Maxgschneider guess what today is! That's right the 1st! #MonkeyOfTheMonth Please let it be me this month!����❤️�� https://t.co/VcjwA0Q1Vg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MonkeyOfTheMonth" }}, "user_mentions": {{ 117639473 }}, "user": { "id": 936124038, "name": "LOSER", "screen_name": "UltimatelyUrie", "lang": "en", "location": "@ A Concert Somewhere ", "create_at": date("2012-11-08"), "description": "(Bisexual) 20", "followers_count": 2429, "friends_count": 678, "statues_count": 65536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067684921704448, "text": "@SchaeferPPBG I think I got this", "in_reply_to_status": 694067441878634497, "in_reply_to_user": 613953422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 613953422 }}, "user": { "id": 26677328, "name": "Roman", "screen_name": "Romanisonfire", "lang": "en", "location": "Chicago", "create_at": date("2009-03-25"), "description": "South side nigga for 20+ years. Stage dives and crowd surfing. always tired. always angry. @DetourNorth management. contact: alvaradomgmt@gmail.com", "followers_count": 1229, "friends_count": 1144, "statues_count": 21607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067684992905217, "text": "Real life tears bro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233447517, "name": "Medu$a", "screen_name": "Lovely_Liesha", "lang": "en", "location": "Cali☀️", "create_at": date("2011-01-02"), "description": "I ain't shit. Read my tweets in lisp. Contact @RLRecords_ for business info.", "followers_count": 8488, "friends_count": 666, "statues_count": 136934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067685118717952, "text": "Happy 22nd birthday Harry!!!!! You mean the world to me and I hope have the best day ever ❤️❤️❤️ @Harry_Styles https://t.co/BeIdxT9hJS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181561712 }}, "user": { "id": 2191201344, "name": "Anastasia Gutierrez", "screen_name": "stasiaceline", "lang": "en", "location": "null", "create_at": date("2013-11-12"), "description": "We don't obey God because it pleases him we obey him because it's the right thing to do", "followers_count": 768, "friends_count": 309, "statues_count": 12148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orcutt, CA", "id": "fcfe99a296187c82", "name": "Orcutt", "place_type": "city", "bounding_box": rectangle("-120.468231,34.842218 -120.378247,34.908588") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 654120, "cityName": "Orcutt" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067685122965504, "text": "Thinking of sleeping w my window open", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 725546887, "name": "melody", "screen_name": "melodynalvarez", "lang": "en", "location": "grey sloan memorial hospital", "create_at": date("2012-07-30"), "description": "caught the wave, I ain't surfin'", "followers_count": 1031, "friends_count": 865, "statues_count": 56219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huron, CA", "id": "3ad6bd9490891f66", "name": "Huron", "place_type": "city", "bounding_box": rectangle("-120.11257,36.19579 -120.08294,36.214155") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 636084, "cityName": "Huron" } }
+{ "create_at": datetime("2016-02-01T00:00:05.000Z"), "id": 694067685278126080, "text": "@CorbinPoppe They're not wrong https://t.co/lAlu2Bt07z", "in_reply_to_status": -1, "in_reply_to_user": 313764099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313764099 }}, "user": { "id": 245216787, "name": "Connor Reichenbach", "screen_name": "conreich", "lang": "en", "location": "Pullman", "create_at": date("2011-01-30"), "description": "The road to hell is paved with good intentions. ΠΚΦ #GoCougs", "followers_count": 675, "friends_count": 718, "statues_count": 5837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213106,46.708177 -117.095324,46.753398") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-02-01T00:00:06.000Z"), "id": 694067685446000640, "text": "Wind 0.0 mph SSW. Barometer 29.603 in, Falling Rapidly. Temperature 36.5 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-02-01T00:00:06.000Z"), "id": 694067685718659072, "text": "Barberton Oh Temp:47.9°F Wind:8 mph Dir:W Baro:Rising quickly Rain2day:0.04in Hum:81% UV:0.0 @ 03:00 02/01/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 232, "friends_count": 228, "statues_count": 114322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820272,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430049261506565, "text": "7868149f99fc796900ded756a4b2f950e3527704e59254972050c5666421484d076d8b411ee7ff0e25f8885a2d5806a279a75cf9fa8505460ca491005e8910ad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.058109,43.5966"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 429803867, "name": "MarsBots", "screen_name": "MarsBots", "lang": "en", "location": "Mars", "create_at": date("2011-12-06"), "description": "null", "followers_count": 86, "friends_count": 0, "statues_count": 431509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Dakota, USA", "id": "d06e595eb3733f42", "name": "South Dakota", "place_type": "admin", "bounding_box": rectangle("-104.05774,42.479636 -96.43659,45.945379") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46035, "countyName": "Davison" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430049370542080, "text": "03:00:01 |Temp: 52.3ºF | Wind Chill 52.3ºF |Dew Point 49.5ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WSW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 89353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430049831895040, "text": "I don't do no arguin I let the gun talk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34474594, "name": "Anna Henderson", "screen_name": "Ahenderson_nrh", "lang": "en", "location": "Howland Center, OH", "create_at": date("2009-04-22"), "description": "just make sure you always have your eyebrows on", "followers_count": 487, "friends_count": 360, "statues_count": 16792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niles, OH", "id": "de89d25be7d37a11", "name": "Niles", "place_type": "city", "bounding_box": rectangle("-80.811321,41.155863 -80.70751,41.22587") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39155, "countyName": "Trumbull", "cityID": 3955916, "cityName": "Niles" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430049915662337, "text": "You are. https://t.co/VP3lWCyPyT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2658908316, "name": "Girl is that Dom?", "screen_name": "girl_isthat_dom", "lang": "en", "location": "null", "create_at": date("2014-07-19"), "description": "20. DTX", "followers_count": 424, "friends_count": 226, "statues_count": 7996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050012127232, "text": "PoliceRpt: Cop thought Cruz's wife was a 'danger to herself' 10 yrs ago Sitting Along HWY https://t.co/7OLqyTlfD3 https://t.co/JHDCkjQBvW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135376922, "name": "4Most Coach", "screen_name": "4mostcoach", "lang": "en", "location": "Anaheim, CA", "create_at": date("2010-04-20"), "description": "Private Family Transportation\nConservative Small Biz / Combat Veteran / DEDICATED TO THE ELECTION OF @realDonaldTrump #Trump2016 #GOTV4Trump #BuildtheWall", "followers_count": 2392, "friends_count": 2932, "statues_count": 12378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050377138176, "text": "Can't Wait Ta Get Home��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2384615132, "name": "Moana Key", "screen_name": "1BTKS_", "lang": "en", "location": "null", "create_at": date("2014-03-11"), "description": "null", "followers_count": 129, "friends_count": 93, "statues_count": 14587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050599305216, "text": "If you're a #Retail professional in #NorthReading, MA, check out this #job: https://t.co/GmuRPOgsfT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.0786653,42.5750939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "NorthReading", "job", "Hiring" }}, "user": { "id": 3144822634, "name": "Speedway Jobs", "screen_name": "SpeedwayJobs", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "Rethink Speedway. It's more than a convenience store... It's your career opportunity!", "followers_count": 171, "friends_count": 27, "statues_count": 6413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Reading, MA", "id": "014e7c6b116d21ca", "name": "North Reading", "place_type": "city", "bounding_box": rectangle("-71.136411,42.551943 -71.028205,42.609037") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050632880132, "text": "Wind 18.0 mph SSE. Barometer 1002.65 mb, Falling Rapidly. Temperature 59.9 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050817482756, "text": "@coralguetta @extrasexual ALL OF THEM", "in_reply_to_status": 694419152656674816, "in_reply_to_user": 97955096, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97955096, 2928124344 }}, "user": { "id": 798879157, "name": "Daddy", "screen_name": "yaardenn", "lang": "en", "location": "null", "create_at": date("2012-09-02"), "description": "~6/29/98 - 8/27/12~ You can’t change the past, but you can ruin the present by worrying about the future", "followers_count": 1231, "friends_count": 654, "statues_count": 45836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430050859364352, "text": "Can you recommend anyone for this #job? Service Associate - https://t.co/uNV47m1NIu #JACKSONVILLE, FL #Retail #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5138241,30.0614433"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "JACKSONVILLE", "Retail", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 512, "friends_count": 251, "statues_count": 16911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12109, "countyName": "St. Johns" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430051425619968, "text": "Interested in a #Nursing #job near #Ozark, MO? This could be a great fit: https://t.co/xKJLjvTJr7 #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2060163,37.0208865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Ozark", "Hiring" }}, "user": { "id": 1898258479, "name": "Mercy Jobs", "screen_name": "MercyJobs", "lang": "en", "location": "null", "create_at": date("2013-09-23"), "description": "When you join Mercy, you are part of a community that is embracing a new way to care for people while building your skills and growing in your career.", "followers_count": 320, "friends_count": 4, "statues_count": 4024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ozark, MO", "id": "017f694e6fedad16", "name": "Ozark", "place_type": "city", "bounding_box": rectangle("-93.26591,36.991078 -93.177158,37.082128") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29043, "countyName": "Christian", "cityID": 2955766, "cityName": "Ozark" } }
+{ "create_at": datetime("2016-02-02T00:00:00.000Z"), "id": 694430052159717376, "text": "@mikeXmoraga Monique & Michael >", "in_reply_to_status": 694429826258587648, "in_reply_to_user": 2661766056, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 2661766056 }}, "user": { "id": 363499624, "name": "Monique", "screen_name": "moniqueee_10", "lang": "en", "location": "Hogwarts", "create_at": date("2011-08-27"), "description": "@mikeXmoraga ~ Cal Poly SLO", "followers_count": 1179, "friends_count": 651, "statues_count": 68505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430052314779648, "text": "Time:02:00 Temp:18.1°F Dewpt:15.7°F Press:29.62in. Wind N at 9mph gusting 12mph Rain Today 0.00in. Humidity:95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.60305556,27.84555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448017712, "name": "AnnavilleWeather", "screen_name": "AnnavilleWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2011-12-27"), "description": "live weather from Annaville, Texas! FMI see http://www.wunderground.com/weatherstation/WXDailyHistory.asp?ID=KTXCORPU31", "followers_count": 86, "friends_count": 395, "statues_count": 18277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430052637810688, "text": "@ElsaLion2 Because he is \"White Christian Patriot\"! Dictionary should change definitions 4 those words in \"Privileged stupid Bigot\"��", "in_reply_to_status": 694387012560814080, "in_reply_to_user": 3273654205, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3273654205 }}, "user": { "id": 95727422, "name": "Lilyana Srnoguy", "screen_name": "YouWant2Change", "lang": "en", "location": "Bozeman, Montana", "create_at": date("2009-12-09"), "description": "Health and Beauty from the inside out! liberal, love Obama, progresive, hate GOP, Fox N., Mitch & rest of idiots, artist, writer, animal lover, go Barnie go❤️", "followers_count": 2316, "friends_count": 2129, "statues_count": 35890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bozeman, MT", "id": "807584e8539d2a7e", "name": "Bozeman", "place_type": "city", "bounding_box": rectangle("-111.185677,45.625349 -110.980181,45.750351") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30031, "countyName": "Gallatin", "cityID": 3008950, "cityName": "Bozeman" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430053338230784, "text": "Im that \"friend\" who gets left out of everything lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 960939950, "name": "Bri ♏", "screen_name": "Bac0nR0se", "lang": "en", "location": "null", "create_at": date("2012-11-20"), "description": "Toni is my sunshine. ☀️", "followers_count": 601, "friends_count": 574, "statues_count": 34930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430053413863424, "text": "Honestly love my bio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255384698, "name": "6 Dio$", "screen_name": "oVo_ASAP", "lang": "en", "location": "null", "create_at": date("2011-02-20"), "description": "Dreams without goals, remain dreams", "followers_count": 793, "friends_count": 500, "statues_count": 33777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430054500032513, "text": "@ebbtideapp Tide in Mispillion River entrance, Delaware 02/02/2016\nHigh 3:27am 4.2\n Low 10:10am 0.7\nHigh 3:42pm 3.5\n Low 10:14pm 0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.315,38.9483"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 21, "friends_count": 1, "statues_count": 4256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430054839902210, "text": "02/02@03:00 - Temp 36.6F, WC 36.6F. Wind 0.0mph ENE, Gust 1.0mph. Bar 30.188in, Rising. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430054877495296, "text": "Come on man Take like 5 shots and go for a joy ride that shit be fun as hell https://t.co/ALgCuSP0Pa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3069175764, "name": "BabyBoy", "screen_name": "calitxgod", "lang": "en", "location": "Austin, TX", "create_at": date("2015-03-08"), "description": "Tyler Paul looks like a slut", "followers_count": 235, "friends_count": 196, "statues_count": 7177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brushy Creek, TX", "id": "e16e19bddd5f24b1", "name": "Brushy Creek", "place_type": "city", "bounding_box": rectangle("-97.769017,30.478998 -97.703863,30.539164") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4810897, "cityName": "Brushy Creek" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430055271940096, "text": "These tweets had no connectivity, your welcome", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420051537, "name": "Sir Kewllngton III", "screen_name": "SeanTooKewlln", "lang": "en", "location": "Utopia", "create_at": date("2011-11-23"), "description": ".happiness", "followers_count": 2656, "friends_count": 584, "statues_count": 14278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Charleston, SC", "id": "18df13e4a5a670b5", "name": "North Charleston", "place_type": "city", "bounding_box": rectangle("-80.15759,32.829336 -79.934288,32.999393") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4550875, "cityName": "North Charleston" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430056119144448, "text": "Wind 1.0 mph SSW. Barometer 29.890 in, Falling Rapidly. Temperature 44.0 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-02T00:00:01.000Z"), "id": 694430056261660674, "text": "I do this for my squad I do this for my game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2483135317, "name": "MG Reckless", "screen_name": "Nick_lopez24", "lang": "en", "location": "null", "create_at": date("2014-05-07"), "description": "i love you back - @_valerieannn ❤️ #Fat5", "followers_count": 340, "friends_count": 278, "statues_count": 17674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430056517505024, "text": "@KieanaAnique happy birthday wit yo fine ass get lit af", "in_reply_to_status": -1, "in_reply_to_user": 615021226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615021226 }}, "user": { "id": 990261242, "name": "$parkling Cider Papí", "screen_name": "SimpMasterFlex_", "lang": "en", "location": "RL | Im Drake's Son | CA", "create_at": date("2012-12-04"), "description": "Black Girls Welcome, I ain't shit tho. | DMs closed because of heartbreaks, inactivity, and terrible accuracy of shooting shots", "followers_count": 1543, "friends_count": 1142, "statues_count": 57560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430056567828480, "text": "I look back at all the moments we had and sometimes I wish I could repeat them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2356573183, "name": "Lexi Cavero", "screen_name": "alexis_cavero", "lang": "en", "location": "Hell", "create_at": date("2014-02-22"), "description": "Food, Wrestling, and Jess", "followers_count": 146, "friends_count": 110, "statues_count": 1904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430057155031042, "text": "Half the battle lol���� https://t.co/6DCoTmqxtg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245635353, "name": "Danny", "screen_name": "LVLDanny", "lang": "en", "location": "El Paso, Texas", "create_at": date("2011-01-31"), "description": "Optimist #UTEP #USARMY #VIRGO #KLOUD ♍️", "followers_count": 992, "friends_count": 828, "statues_count": 44466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430057184407552, "text": "In honor of Groundhogs Day, here are some pictures of how groundhogs have contributed to society https://t.co/J8sH1Y0P6J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2535411576, "name": "Sonja Paulson", "screen_name": "sonja_paulson", "lang": "en", "location": "null", "create_at": date("2014-05-30"), "description": "Ames high 2018 || snap&insta: @sonja_paulson", "followers_count": 343, "friends_count": 321, "statues_count": 221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ames, IA", "id": "d6794586b08b7f9c", "name": "Ames", "place_type": "city", "bounding_box": rectangle("-93.718311,41.986006 -93.560416,42.079007") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1901855, "cityName": "Ames" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430057280897024, "text": "y lloras hoy por que no estoy mi vida y lloras hoy dejandome esta herida", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 589398716, "name": ".", "screen_name": "Sandritaa_Xoxo", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-05-24"), "description": "Adrian ❤", "followers_count": 1424, "friends_count": 1400, "statues_count": 16955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430057381507073, "text": "@ermimoore me too girl wish u were in spok so we could be alone together", "in_reply_to_status": 694383876437749760, "in_reply_to_user": 3129063715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3129063715 }}, "user": { "id": 586899721, "name": "cashmoneymesa", "screen_name": "mesagrant", "lang": "en", "location": "null", "create_at": date("2012-05-21"), "description": "What happened last night", "followers_count": 442, "friends_count": 430, "statues_count": 6634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.564576,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430057700282368, "text": "YES I KNOW REDBULL IS BAD FOR ME NO I DONT CARE ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 440204303, "name": "alma palma", "screen_name": "AlmaZamora7", "lang": "en", "location": "San Marcos, Texas", "create_at": date("2011-12-18"), "description": "go hard or don't go at all #txst", "followers_count": 2249, "friends_count": 546, "statues_count": 91556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430058010652673, "text": "LEGS FUCKING BURNING!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1626877621, "name": "GalacticZach", "screen_name": "GalacticZach", "lang": "en", "location": "texas", "create_at": date("2013-07-27"), "description": "Check out my http://twitch.tv/galacticzach for times! don't forget to follow!", "followers_count": 205, "friends_count": 154, "statues_count": 2619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple, TX", "id": "c5c0e4d49b664e49", "name": "Temple", "place_type": "city", "bounding_box": rectangle("-97.500586,31.031178 -97.311609,31.189775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4872176, "cityName": "Temple" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430058845462528, "text": "Temp: 67.4°F Wind:0.8mph Pressure: 29.849hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430058954424320, "text": "⛅️Category: Weather warnings and advisories https://t.co/jSVImOvZdc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4395000434, "name": "Chanel Aprahamian", "screen_name": "1Aprahami2", "lang": "en", "location": "El Cerrito, CA", "create_at": date("2015-11-28"), "description": "I loved anime with Adventure, Fantasy, Mystery, Psychological, Romance, Magic, Supernatural, Music, Demons, Magical Girl, SuperPowers and Martial Arts.", "followers_count": 9, "friends_count": 74, "statues_count": 120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059013279744, "text": "Wind 1.0 mph ESE. Barometer 30.091 in, Falling slowly. Temperature 37.6 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059197775872, "text": "Where is Myrtle Beach on the map? Play the game at https://t.co/6KzuSI3oHy #Myrtle Beach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8867,33.6891"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Myrtle" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1042, "friends_count": 312, "statues_count": 2566838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059319308288, "text": "@YungInternetGod u should consider remixing atrevete-te-te by calle 13. I'm sure it would have the same effect as the fan fave \"suavemente\"", "in_reply_to_status": -1, "in_reply_to_user": 18815809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18815809 }}, "user": { "id": 953246509, "name": "ATRÉVETE-TE-TE", "screen_name": "Bearlyknown", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2012-11-17"), "description": "I'm not SUPERstitious, im a LITTLEstitious", "followers_count": 281, "friends_count": 212, "statues_count": 17749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059432513536, "text": "@ChisomeO_ happy birthday baby", "in_reply_to_status": -1, "in_reply_to_user": 2465790151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2465790151 }}, "user": { "id": 823856642, "name": "Kaalam", "screen_name": "sofiiababy", "lang": "en", "location": "null", "create_at": date("2012-09-14"), "description": "Incurable Romantic snapchat- sofiiibaby http://Instagram.com/sofiiababy", "followers_count": 518, "friends_count": 282, "statues_count": 4864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059973648384, "text": "Couldn't say it better ... https://t.co/L3LjxSG33X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311783678, "name": "Carmel Drop", "screen_name": "xoelonixo", "lang": "en", "location": "HTX ", "create_at": date("2011-06-05"), "description": "insta:xoelonixo_", "followers_count": 821, "friends_count": 898, "statues_count": 28695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430059994611712, "text": "@JeanneXay hey gurl", "in_reply_to_status": 694429936593973248, "in_reply_to_user": 19463624, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19463624 }}, "user": { "id": 16486050, "name": "Nora Mermaid", "screen_name": "Nora_Mermaid", "lang": "en", "location": "A Clawfoot Bathtub in Denver", "create_at": date("2008-09-27"), "description": "@BlackGryph0n's official #1 fan & Brony. Fandom Mermaid. Followed by & met @tokiohotel. Won @midnight. Oh & I sing! #ladygaga #Disney #doctorwho", "followers_count": 2160, "friends_count": 1996, "statues_count": 99524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430060200202240, "text": "#SupportOriginMelissa 36.7°F Wind:0.0mph Pressure: 30.03hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-02T00:00:02.000Z"), "id": 694430060200235008, "text": "Shockwave https://t.co/eTfskIGzvc #shockwave #parcdattraction https://t.co/UWepEMZ9LR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.070335,32.758755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shockwave", "parcdattraction" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 705, "friends_count": 1110, "statues_count": 2558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430060711907328, "text": "Travelling to Myrtle Beach or just twittering about Myrtle Beach? https://t.co/6KzuSI3oHy #Myrtle Beach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8867,33.6891"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Myrtle" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1042, "friends_count": 312, "statues_count": 2566839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430060892262401, "text": "when the one that got away comes back >>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2988102321, "name": "ash", "screen_name": "ashley_blust", "lang": "en", "location": "null", "create_at": date("2015-01-20"), "description": "Mommy to the sweetest little boy in the world, Brantley Elliot", "followers_count": 190, "friends_count": 170, "statues_count": 2944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, IL", "id": "9c908d38b1b53897", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-89.028906,39.785267 -88.844461,39.923903") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17115, "countyName": "Macon", "cityID": 1718823, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430060925681665, "text": "67.5F (Feels: 67.5F) - Humidity: 99% - Wind: 9.8mph SE - Gust: 17.4mph - Pressure: 1017.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 225022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430061055770624, "text": "@Werethrope @Garrodor @BigSpanishLycan @ShunRMatoi Deviled eggs for the salad too~", "in_reply_to_status": 694427879141052416, "in_reply_to_user": 1639982070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1639982070, 85682736, 2989450438, 4826246293 }}, "user": { "id": 2312792593, "name": "Sin", "screen_name": "Varanis_Ridari", "lang": "en", "location": "Bellevue, WA", "create_at": date("2014-01-26"), "description": "I love music, tend to be really social, make a lot of puns, and will call out BS so watch out! Lovin' on my @ErenYanno", "followers_count": 1319, "friends_count": 303, "statues_count": 30949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223198,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430061102022656, "text": "One day it's gonna burn you out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384631404, "name": "KĮD Â", "screen_name": "NameBrandAidan", "lang": "en", "location": "null", "create_at": date("2011-10-03"), "description": "product of the early 90's", "followers_count": 152, "friends_count": 377, "statues_count": 3421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion, IN", "id": "6181e0f8ac8150e1", "name": "Marion", "place_type": "city", "bounding_box": rectangle("-85.744882,40.49346 -85.61564,40.610902") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18053, "countyName": "Grant", "cityID": 1846908, "cityName": "Marion" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430061269626880, "text": "@erikscottdebie @nellybellexox things certainly seemed to go as planned, yes :)", "in_reply_to_status": 694416848775827457, "in_reply_to_user": 29162019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29162019, 601133070 }}, "user": { "id": 25387057, "name": "Derek Guder", "screen_name": "frequentbeef", "lang": "en", "location": "Renton, WA", "create_at": date("2009-03-19"), "description": "I run conventions. I play games with people. I rant about technology.\nEvent manager for @Gen_Con.", "followers_count": 1729, "friends_count": 308, "statues_count": 74101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430062137991168, "text": "Wind 1.6 mph WSW. Barometer 30.15 in, Rising slowly. Temperature 40.6 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430062448373761, "text": "I hate being alone but when I'm depressed that's all I want and it makes everything worse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47393517, "name": "⚠️ gucci af ⚠️", "screen_name": "loletcetera", "lang": "en", "location": "null", "create_at": date("2009-06-15"), "description": "@nathanpolak", "followers_count": 403, "friends_count": 42, "statues_count": 35020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430062720847872, "text": "I love you too lol https://t.co/svVLJ8bWUe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2843972314, "name": "yvette", "screen_name": "YvetteDenice", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-10-25"), "description": "hispanic / Fernando's", "followers_count": 150, "friends_count": 117, "statues_count": 3411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430063379415044, "text": "That emoji https://t.co/vZfUCjByho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585337829, "name": "Lockaby", "screen_name": "Lockabeeee", "lang": "en", "location": "SLC, UT", "create_at": date("2012-05-19"), "description": "So may the sunrise bring hope where it once was forgotten.", "followers_count": 330, "friends_count": 64, "statues_count": 46509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kearns, UT", "id": "16f11b6e2e5b8cc1", "name": "Kearns", "place_type": "city", "bounding_box": rectangle("-112.043213,40.638527 -111.986427,40.667799") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4940470, "cityName": "Kearns" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430063811375104, "text": "she needs some milk https://t.co/FyMJSBQqCN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2994343458, "name": "SergioMFC", "screen_name": "SergioSkellism", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2015-01-24"), "description": "1/2 of @skellism • IG: sergioskellism", "followers_count": 1187, "friends_count": 887, "statues_count": 3415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430063903641601, "text": "@emogenelovesyou lol @cali_nastyleal sorry���� I laughed so hard after... ��", "in_reply_to_status": 694429895028379648, "in_reply_to_user": 161906912, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161906912, 1360363465 }}, "user": { "id": 161906912, "name": "mocha latté", "screen_name": "emogenelovesyou", "lang": "en", "location": "SoCal.", "create_at": date("2010-07-01"), "description": "18", "followers_count": 606, "friends_count": 350, "statues_count": 22689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-02T00:00:03.000Z"), "id": 694430064780267520, "text": "Yes. @LoganPeery27 https://t.co/rxlGIzKfvT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 207840550 }}, "user": { "id": 461661759, "name": "Mariah", "screen_name": "little_one08", "lang": "en", "location": "Auburn, WA", "create_at": date("2012-01-11"), "description": "Mariah | 20 | Logan Peery ❤️ | Normal is an illusion. What is normal for the spider, is chaos for the fly. |", "followers_count": 471, "friends_count": 648, "statues_count": 20440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie Ridge, WA", "id": "40255cddc5325075", "name": "Prairie Ridge", "place_type": "city", "bounding_box": rectangle("-122.185728,47.108756 -122.097327,47.165918") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356170, "cityName": "Prairie Ridge" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430065350672385, "text": "11. If 9 doesn't apply to you then it's probably either numbers 3 or 6", "in_reply_to_status": 694429849528610816, "in_reply_to_user": 405772142, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405772142, "name": "West", "screen_name": "TF_Tsew", "lang": "en", "location": "UNLV", "create_at": date("2011-11-05"), "description": "TopFlight OTC", "followers_count": 659, "friends_count": 489, "statues_count": 37028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430065778675712, "text": "Niggas with good dick love to utilize the block button with no hesitation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182888841, "name": "Your Team Still ASS", "screen_name": "enimsahJ__x", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-08-25"), "description": "22. Navy Vet. Bitch my eye isn't black in my avi, before you try it. My eyeliner & eyeshadow was all over the place hoe. Find something else.", "followers_count": 2525, "friends_count": 2427, "statues_count": 166407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond Heights, OH", "id": "fb693a79982c73c0", "name": "Richmond Heights", "place_type": "city", "bounding_box": rectangle("-81.527508,41.5378 -81.487495,41.586163") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3966894, "cityName": "Richmond Heights" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430066097270784, "text": "@warriorsworld ����", "in_reply_to_status": -1, "in_reply_to_user": 18198414, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 18198414 }}, "user": { "id": 1043276612, "name": "Young Virgin", "screen_name": "24HrStevie", "lang": "en", "location": "408 South Bay", "create_at": date("2012-12-28"), "description": "U know the deal, Let's go get mils", "followers_count": 326, "friends_count": 818, "statues_count": 2666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saratoga, CA", "id": "b46f044cb28493c6", "name": "Saratoga", "place_type": "city", "bounding_box": rectangle("-122.068181,37.237066 -121.989501,37.296386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 670280, "cityName": "Saratoga" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430066596380672, "text": "2.1 magnitude #earthquake. 44 km from #FritzCreek, AK, United States https://t.co/TF8N5Y5T0M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-150.5216,59.81"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "FritzCreek" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 63245, "friends_count": 10, "statues_count": 95067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430067615776768, "text": "Wind 0 mph SW. Barometer 30.02 in, Rising. Temperature 64.4 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 52, "friends_count": 69, "statues_count": 25412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430067632553984, "text": "guess I'll go to sleep��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513410522, "name": "jojo", "screen_name": "jojoxbad_", "lang": "en", "location": "null", "create_at": date("2014-04-27"), "description": "positive vibes, thick thighz✨", "followers_count": 241, "friends_count": 412, "statues_count": 3584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake, VA", "id": "73d1c1c11b675932", "name": "Chesapeake", "place_type": "city", "bounding_box": rectangle("-76.448383,36.66 -76.143408,36.866431") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51550, "countyName": "Chesapeake", "cityID": 5116000, "cityName": "Chesapeake" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430067909201920, "text": "George is codl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1643735161, "name": "gg", "screen_name": "ggfunk", "lang": "en", "location": "Moreno Valley, CA", "create_at": date("2013-08-03"), "description": "20| y a chingar a su madre #boxing", "followers_count": 544, "friends_count": 299, "statues_count": 47276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430068727255045, "text": "Barberton Oh Temp:25.0°F Wind:0 mph Dir:--- Baro:Falling slowly Rain2day:0.00in Hum:83% UV:0.0 @ 03:00 02/02/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 228, "statues_count": 114346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820272,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430068756459520, "text": "Temp: 25.6°F - Dew Point: 21.7° - Wind: 0.6 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 30.14in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430068769042432, "text": "Wind 7.0 mph NE. Barometer 29.959 in, Falling. Temperature 32.3 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 11, "statues_count": 838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-02T00:00:04.000Z"), "id": 694430069025013760, "text": "Omm I'm finna go chill with El Chapo https://t.co/RfbhjbJZWo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 531580326, "name": "Geordan", "screen_name": "httpshines", "lang": "en", "location": "8 1 2 ", "create_at": date("2012-03-20"), "description": "Find a way, or finesse one.", "followers_count": 725, "friends_count": 675, "statues_count": 24954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, IN", "id": "03aa4200f3f1192a", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-87.578437,38.033701 -87.541637,38.07423") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1833484, "cityName": "Highland" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430069398241280, "text": "4 more months till my 20th birthday.... wow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560523487, "name": "Denise Guevara", "screen_name": "lofticriiies", "lang": "en", "location": "California, USA ", "create_at": date("2012-04-22"), "description": "Guatemalteca", "followers_count": 574, "friends_count": 343, "statues_count": 47484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430069716963329, "text": "����������������������\nUSIN��TWITTER��S����\nY��U��SEE��THE��\n��������������W��RLD��\n����SH��W��YA��\nAR��UND��AN��\nTAKE��YA��THERE��\n��������CAN��\nSH��W��Y��U��\nTHE����DEVIL����\n����676����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3148743344, "name": "THOMAS DUROSO", "screen_name": "duroso71", "lang": "en", "location": "ST.LOUIS MO", "create_at": date("2015-04-10"), "description": "⛅MALACHI⛅3 VS 1⛅BEHOLD,I SEND MY MESSENGER 2 PREPARE THE WAY FOR ME⛅", "followers_count": 55, "friends_count": 1, "statues_count": 6474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, MO", "id": "e6610919d60bcae1", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-90.386457,38.719559 -90.309531,38.774154") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2904906, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430069813456898, "text": "New guy walks into class\nGirl- \"You got some booty eatin lips boy\" what. The fuck.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1107845498, "name": "KenTucky", "screen_name": "housedavid68", "lang": "en", "location": "Rockwell, NC", "create_at": date("2013-01-20"), "description": "Senior•Living life like I wrecked my moped•June 11th•", "followers_count": 25, "friends_count": 31, "statues_count": 593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockwell, NC", "id": "058cf2935ef113b5", "name": "Rockwell", "place_type": "city", "bounding_box": rectangle("-80.426025,35.533432 -80.374661,35.571117") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37159, "countyName": "Rowan", "cityID": 3757340, "cityName": "Rockwell" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430070698610689, "text": "Three cute dudes nude �������� https://t.co/EkMntU3GaN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4079, "friends_count": 3141, "statues_count": 43313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430070962671616, "text": "Guy fights pineapples: https://t.co/TtXhyd527p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71316630, "name": "Nick Snyder", "screen_name": "_nicksnyder", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-09-03"), "description": "Freedom lies in being bold. \nInteractive Designer \nMarathoner ❧ CrossFitter \nLoves: Phillies ❧ Typography ❧ Nintendo", "followers_count": 526, "friends_count": 125, "statues_count": 23438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairmount Park, Seattle", "id": "6821afc31b708123", "name": "Fairmount Park", "place_type": "neighborhood", "bounding_box": rectangle("-122.387249,47.544851 -122.370694,47.571475") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071063318532, "text": "\"let's get it\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2776394585, "name": "Steven Navarra", "screen_name": "stevennavarra_", "lang": "en", "location": "null", "create_at": date("2014-09-19"), "description": "MH/SB", "followers_count": 218, "friends_count": 342, "statues_count": 393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071361110016, "text": "Excited to play the new zombie map when I get off work!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29804952, "name": "The Legend", "screen_name": "iForeman12", "lang": "en", "location": "null", "create_at": date("2009-04-08"), "description": "Just a shy quiet guy.", "followers_count": 289, "friends_count": 298, "statues_count": 26089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kokomo, IN", "id": "65ab99feec5c8213", "name": "Kokomo", "place_type": "city", "bounding_box": rectangle("-86.233407,40.40379 -86.051017,40.541662") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18067, "countyName": "Howard", "cityID": 1840392, "cityName": "Kokomo" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071541477376, "text": "mad cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350890393, "name": "take a Gill pill", "screen_name": "gillfenne", "lang": "en", "location": "null", "create_at": date("2011-08-08"), "description": "♎️☮ig: gillfenne", "followers_count": 252, "friends_count": 202, "statues_count": 2126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071663128576, "text": "Wind 2.0 mph S. Barometer 30.252 in, Steady. Temperature 34.4 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 198, "friends_count": 58, "statues_count": 247906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071914803200, "text": "Dropping a new beat soon!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3102707599, "name": "T", "screen_name": "662Tlee", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2015-03-21"), "description": "Tylerlee662@gmail.com", "followers_count": 100, "friends_count": 44, "statues_count": 291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071931580416, "text": "����������������������yea!!!!! https://t.co/PuOeaoXL0u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2355940807, "name": "Ben Griffey Jr", "screen_name": "TRENDYBENNY", "lang": "en", "location": "Golden State", "create_at": date("2014-02-21"), "description": "LIVE LONG AROD|FREEBAND$", "followers_count": 468, "friends_count": 421, "statues_count": 18634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430071994486784, "text": "You know I be seeing some funny as shit on my TL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276355787, "name": "Desiree", "screen_name": "dez4rae", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-04-02"), "description": "21 | Javier | Disneyland", "followers_count": 691, "friends_count": 242, "statues_count": 54843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430072682319872, "text": "\"Por cierto, hacen muy buena pareja, tú le pegas y ella se deja\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 52876623, "name": "amandititita", "screen_name": "amandititita", "lang": "es", "location": "DF y Los Angeles, CA ", "create_at": date("2009-07-01"), "description": "Culpable de los discos Mala fama, La descarada, y Anarcumbia. Autora del libro 13 Latas de atún.", "followers_count": 94969, "friends_count": 1005, "statues_count": 25217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430072791412736, "text": "Shout out to @Tyler_Curtis_ for coming all the way in the snow to bring me flaming hot Cheetos and to get drunk with me ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346687048 }}, "user": { "id": 950812656, "name": "Evelyn", "screen_name": "Evelynroseswift", "lang": "en", "location": "null", "create_at": date("2012-11-15"), "description": "Someone who was separated from the universe ☾", "followers_count": 951, "friends_count": 642, "statues_count": 7783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gillette, WY", "id": "01d646f92d352a84", "name": "Gillette", "place_type": "city", "bounding_box": rectangle("-105.574767,44.214886 -105.415795,44.317085") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56005, "countyName": "Campbell", "cityID": 5631855, "cityName": "Gillette" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430072938188800, "text": "Pretty mama if you single, you don't gotta be alone tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3193947703, "name": "Isaiah", "screen_name": "zayy_martinez12", "lang": "en", "location": "Denver, CO", "create_at": date("2015-05-12"), "description": "null", "followers_count": 152, "friends_count": 139, "statues_count": 3421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheat Ridge, CO", "id": "2d35518289e8d1e5", "name": "Wheat Ridge", "place_type": "city", "bounding_box": rectangle("-105.174724,39.754644 -105.053213,39.79574") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 884440, "cityName": "Wheat Ridge" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430072963334145, "text": "Anyone got artist with a Bryson tiller sound you could suggest? Reply or DM or whatever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1907360454, "name": "☁️LoGod☁️", "screen_name": "LoLife1994", "lang": "en", "location": "Spanaway, WA", "create_at": date("2013-09-26"), "description": "Sent from the stars in '94| Paige got my heart in a figure four| BLACKOUTBOYZ| OTM| FTW| 901-253|", "followers_count": 318, "friends_count": 104, "statues_count": 12830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanaway, WA", "id": "21ea0a8db9b8d520", "name": "Spanaway", "place_type": "city", "bounding_box": rectangle("-122.464673,47.053113 -122.389138,47.126056") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5366255, "cityName": "Spanaway" } }
+{ "create_at": datetime("2016-02-02T00:00:05.000Z"), "id": 694430073097564160, "text": "���� https://t.co/HD3oi1BaI9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 472630877, "name": "10Days", "screen_name": "Devi33zyTop", "lang": "en", "location": "San Marcos, TX", "create_at": date("2012-01-23"), "description": "Devi Rowe •IG: ThirdEyeRowe • Snap: devi33zy", "followers_count": 751, "friends_count": 472, "statues_count": 39844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.500723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430074355863552, "text": "�� https://t.co/tTgwKHH3fu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 194017488, "name": "Q.", "screen_name": "FansOfAdam", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2010-09-22"), "description": "95'", "followers_count": 356, "friends_count": 94, "statues_count": 22544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430075249250304, "text": "I just want to feel whole again. Like nothing is missing... & be truly happy. That's it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 946094605, "name": "Queen B ✨", "screen_name": "ANYSSAFAITHH", "lang": "en", "location": "null", "create_at": date("2012-11-13"), "description": "you do what you want when you poppin. $$$", "followers_count": 1380, "friends_count": 698, "statues_count": 7296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Valley, NM", "id": "01588ad6021ef7e7", "name": "South Valley", "place_type": "city", "bounding_box": rectangle("-106.736956,34.938902 -106.626779,35.084125") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3574520, "cityName": "South Valley" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430075605782528, "text": "**lmfaoooo** https://t.co/dXEJ3mvW6q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2686779180, "name": "#️⃣9️⃣", "screen_name": "kendricrsc", "lang": "en", "location": "south central L'A", "create_at": date("2014-07-27"), "description": "Instagram: @kendricnok snap: fr3d_fredburger, Ł'A,®6️⃣0️⃣, free Mack and Rowdy3️⃣K, free spiffy and free D.Rose, rip fatty and D Rocc, #️⃣9️⃣, FTP,FTO", "followers_count": 406, "friends_count": 311, "statues_count": 30699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430075626856448, "text": "@JoannaBlackhart maybe its not possible to truly be considered in a president's vision of the country", "in_reply_to_status": 694427375099101184, "in_reply_to_user": 455219790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 455219790 }}, "user": { "id": 387050961, "name": "Yukio Strachan", "screen_name": "boldandworthy", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "Jamaican-American| PharmD | AKAtude | And Still I Rise — Maya Angelou", "followers_count": 2724, "friends_count": 151, "statues_count": 102629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430075903676416, "text": "Black Rhyno @ Toasted Mondayz #toastedmondayz #brtweekend #kingmidas #blackrhyno #queens #ny… https://t.co/rrUWWkuFmx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.76555,40.68189"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "toastedmondayz", "brtweekend", "kingmidas", "blackrhyno", "queens", "ny" }}, "user": { "id": 18838454, "name": "Kurtflurt", "screen_name": "Kurtflurt", "lang": "en", "location": "NYC", "create_at": date("2009-01-10"), "description": "Photographer..will shoot for money. Thirst for knowledge/Love Food/Gadgets/Movies/Music/Datzwhatzup.net CEO", "followers_count": 1159, "friends_count": 616, "statues_count": 2337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430075924516865, "text": "Ppl want a change in their lifestyle but yet blame others for the lifestyle they be livin in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381934968, "name": "Paul Perez", "screen_name": "Paulyy_Deee", "lang": "en", "location": "Fresno, CA", "create_at": date("2011-09-28"), "description": "These niggas simply not built like me.", "followers_count": 950, "friends_count": 335, "statues_count": 3553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430076180385792, "text": "@xoxogisseel ���� no I'm just being honest", "in_reply_to_status": 694429965874388992, "in_reply_to_user": 2506418024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2506418024 }}, "user": { "id": 460617612, "name": "HIGHCLASS CRISTIAN®™", "screen_name": "Rivera1Cristian", "lang": "en", "location": "Lathrop, CA", "create_at": date("2012-01-10"), "description": "#FreeSinner. 6'1 ☔️☔️☔️", "followers_count": 678, "friends_count": 609, "statues_count": 21668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430076293632000, "text": "Talking to girls who make u laugh are interesting to talk to not the ones who only talk about irrelevant shit that bored u the fuck out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1511877523, "name": "ELIAS", "screen_name": "Enriquez79e", "lang": "en", "location": "Fort worth, TX", "create_at": date("2013-06-12"), "description": "Let's laugh ...laugh until it hurts lmao", "followers_count": 315, "friends_count": 326, "statues_count": 1357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-02T00:00:06.000Z"), "id": 694430076708982784, "text": "Wind 1.0 mph WSW. Barometer 30.051 in, Rising Rapidly. Temperature 33.4 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430077472247808, "text": "���� https://t.co/aVHzSrlmjM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3223557973, "name": "Alexis Portillo", "screen_name": "alexis_portito", "lang": "en", "location": "null", "create_at": date("2015-05-22"), "description": "ok", "followers_count": 230, "friends_count": 163, "statues_count": 6810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430077619015680, "text": "If you send me a hat and/or a shirt, you will be excused. Everyone else������������. Adios.", "in_reply_to_status": 694429982584500224, "in_reply_to_user": 1178326297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1178326297, "name": "poppunktrash-❶❽❷", "screen_name": "coolpacetic182", "lang": "en", "location": "skatlovers.com", "create_at": date("2013-02-14"), "description": "pop punk. shitty guitar player. 5th member of @sharksonwheels_. I'm actually the shark. everything is coolpacetic, man. @ScenicDriveEP", "followers_count": 1928, "friends_count": 2401, "statues_count": 53893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430077631598593, "text": "�� https://t.co/LeRcI8YbTs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2279812885, "name": "Lil Tae", "screen_name": "undisputed____", "lang": "en", "location": "null", "create_at": date("2014-01-06"), "description": "She ain't a nicki fan then the bitch deaf dumb", "followers_count": 607, "friends_count": 537, "statues_count": 23574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430077640028160, "text": "Lmao you didn't have to do him like that ������ https://t.co/VUeMcQoDX1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2150125614, "name": "sweeterman", "screen_name": "Oscarin_95", "lang": "en", "location": "chillin", "create_at": date("2013-10-22"), "description": "snapchat : oscarinnn95 ig : Oscarin_95 TheeBullzLV ✌️", "followers_count": 419, "friends_count": 393, "statues_count": 19204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430077770006528, "text": "الغربه ماتنسيني ����\nمتي استلم الفلوس �� https://t.co/KFHdhPWFYq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2900875945, "name": "بقاله م . بن عاقول", "screen_name": "Ben3aqool", "lang": "ar", "location": "Los Angeles, CA", "create_at": date("2014-11-15"), "description": "اكتب اللي يعجبني واللي مايعجبه كلامي مو ملزوم يتابعني - kuwait", "followers_count": 746, "friends_count": 191, "statues_count": 41056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430079204478976, "text": "Happy birthday to this gorgeous girl.! ��She's the love of my life �� https://t.co/nWz3nu8nIk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1595105298, "name": "Brïłęÿ Šhêffįęłd", "screen_name": "Brileysheffield", "lang": "en", "location": "null", "create_at": date("2013-07-14"), "description": "Høøligan | Kailey Vold", "followers_count": 578, "friends_count": 506, "statues_count": 2164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430079363907584, "text": "@macybesuyen love love love you ❤️", "in_reply_to_status": 694425978378956800, "in_reply_to_user": 1321648914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1321648914 }}, "user": { "id": 1075864608, "name": "Rach", "screen_name": "RachelFasel17", "lang": "en", "location": "null", "create_at": date("2013-01-09"), "description": "i do backflips and yoga", "followers_count": 369, "friends_count": 190, "statues_count": 9413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430079489839104, "text": "When there's clearly nothing bad she can say about me ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330112590, "name": "OneOfAKind_", "screen_name": "Hershey_KiSS3S", "lang": "en", "location": "Deni〽️s heart", "create_at": date("2011-07-05"), "description": "say what you want just dont bring my son up i love my lor nigga ❤️", "followers_count": 396, "friends_count": 613, "statues_count": 15130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parkville, MD", "id": "3382e2a4c2b3fd91", "name": "Parkville", "place_type": "city", "bounding_box": rectangle("-76.58223,39.371572 -76.520095,39.402079") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2460275, "cityName": "Parkville" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430079623962625, "text": "This headache no bueno I'm going to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 717552331, "name": "Manny Cruzzz", "screen_name": "SuaveCruz_", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "B.I.T.E", "followers_count": 455, "friends_count": 605, "statues_count": 15855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-02T00:00:07.000Z"), "id": 694430080127234048, "text": "@jacob_carlsonnn @IJudgeYouHoes you guys are being like there's no hope, when you guys are the ones sitting on social media complaining", "in_reply_to_status": 694429460892782592, "in_reply_to_user": 2905670821, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2905670821, 1035075649 }}, "user": { "id": 1214897695, "name": "chlo", "screen_name": "Chlooo212", "lang": "en", "location": "mphs", "create_at": date("2013-02-24"), "description": "ITV❤️| MPHS| 16 | Positive, Forward, Motion ➡️|", "followers_count": 1168, "friends_count": 1265, "statues_count": 7272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792436619595776, "text": "i think ima change my @ name", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2419378188, "name": "Burrr. ❄", "screen_name": "iLickYoMama", "lang": "en", "location": "Vietnam", "create_at": date("2014-03-30"), "description": "null", "followers_count": 3829, "friends_count": 1847, "statues_count": 68577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792436770541568, "text": "Y'all outta control.. Fr..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114260910, "name": "Tray", "screen_name": "Imtraylee", "lang": "en", "location": "Normal, IL", "create_at": date("2010-02-14"), "description": "I look good in black.", "followers_count": 1106, "friends_count": 600, "statues_count": 35811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792436883689472, "text": "When you try to trap bae by posting a selfie on snapchat but all he does is watch it. K cool.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1526736564, "name": "hoolia", "screen_name": "surferhoolia", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-06-17"), "description": "do unto others as you would do unto yourself foooolzzz", "followers_count": 157, "friends_count": 171, "statues_count": 5272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437244362752, "text": "*Shad's voice* I'm annoyed. https://t.co/UZnm9o3Wkb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348190684, "name": "Blondie", "screen_name": "Shelbyyylain", "lang": "en", "location": "null", "create_at": date("2011-08-03"), "description": "Spread love, people notice. Dallas. The University of Texas '18 \\m/ ~ Rest in peace Daddy. Support the fight, check the link.", "followers_count": 1025, "friends_count": 465, "statues_count": 16504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437311475712, "text": "@dunn all the talk of twitter's demise made me nostalgic. Then found some good reads on my feed and now I'm back for the time being", "in_reply_to_status": 694791929909874689, "in_reply_to_user": 202724099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 202724099 }}, "user": { "id": 258099217, "name": "Shash-vuth", "screen_name": "noSCeilings", "lang": "en", "location": "San Francisco", "create_at": date("2011-02-26"), "description": "Investment Banker 25/7. Audiophile and NY Knicks Basketball Junkie in my spare time. Jerseyite & USC Trojan Alum For Life.", "followers_count": 268, "friends_count": 655, "statues_count": 9288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437433245696, "text": "When you plan to do an all nighter to bring up your grades but fall asleep<<<<", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2191964009, "name": "Amanda Mizelle", "screen_name": "amandamizelle", "lang": "en", "location": "Lorton,Va", "create_at": date("2013-11-22"), "description": "Smile because it'll make them happy or piss them off, either way you win SoCo '16", "followers_count": 194, "friends_count": 190, "statues_count": 2199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crosspointe, VA", "id": "01005ff9639d3e55", "name": "Crosspointe", "place_type": "city", "bounding_box": rectangle("-77.281417,38.716468 -77.249385,38.738137") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5120436, "cityName": "Crosspointe" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437433307137, "text": "That dead just pissed me off! I'm about to go to sleep with the lights on. That's not cool.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2973952805, "name": "Nadine Metellus", "screen_name": "itsnadine___", "lang": "en", "location": "NYC", "create_at": date("2015-01-11"), "description": "I'm not here to make friends. I'm here to obliterate this entire division.", "followers_count": 202, "friends_count": 263, "statues_count": 3295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437517029376, "text": "Remember that one time..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291400506, "name": "nina", "screen_name": "onecutenina", "lang": "en", "location": "null", "create_at": date("2011-05-01"), "description": "sigh", "followers_count": 23, "friends_count": 73, "statues_count": 5999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437571530752, "text": "Kobe https://t.co/OgNHTcj4Hm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 3164600564, "name": "BrokeBoi.", "screen_name": "Pimpinn_", "lang": "en", "location": "Houston,TX", "create_at": date("2015-04-19"), "description": "null", "followers_count": 291, "friends_count": 244, "statues_count": 3192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437794000897, "text": "@WantedAdventure Dana, i want to turn you on to Alexander U's channel on YT. https://t.co/3tXTxOGefD I like the way he films, gets in to Umz", "in_reply_to_status": -1, "in_reply_to_user": 2504147719, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2504147719 }}, "user": { "id": 2687811926, "name": "Keith Duncan", "screen_name": "WKeithDuncan", "lang": "en", "location": "null", "create_at": date("2014-07-28"), "description": "null", "followers_count": 73, "friends_count": 94, "statues_count": 1291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Easley, SC", "id": "0029290613dee5f5", "name": "Easley", "place_type": "city", "bounding_box": rectangle("-82.663558,34.776033 -82.476443,34.891439") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45077, "countyName": "Pickens", "cityID": 4521985, "cityName": "Easley" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792437949018112, "text": "Wind 10.0 mph WNW. Barometer 1015.38 mb, Rising Rapidly. Temperature 35.4 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792439035334656, "text": "spring break in Miami, it's lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 345031390, "name": "quayvo", "screen_name": "_whoCaresThough", "lang": "en", "location": "Houston, TX", "create_at": date("2011-07-29"), "description": "dallas made me | texas southern 16'", "followers_count": 4449, "friends_count": 957, "statues_count": 103983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792439060520960, "text": "Besides all the mini heart attacks I had today it was perfect��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479214939, "name": "♈️", "screen_name": "_Veeeronica", "lang": "en", "location": "TX", "create_at": date("2012-01-30"), "description": "null", "followers_count": 796, "friends_count": 532, "statues_count": 22689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-03T00:00:00.000Z"), "id": 694792439895236608, "text": "@Marleezyy123 their foundation is super awesome! ��", "in_reply_to_status": 694791911744221184, "in_reply_to_user": 220761438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 220761438 }}, "user": { "id": 548058985, "name": "Katherine Janis", "screen_name": "katiegjanis", "lang": "en", "location": "Olympia, WA ➡Palo Alto, CA", "create_at": date("2012-04-07"), "description": "wherever you go, go with all your heart", "followers_count": 336, "friends_count": 526, "statues_count": 9629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Palo Alto, CA", "id": "b9da7881046990df", "name": "East Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.156222,37.451829 -122.115942,37.484169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 620956, "cityName": "East Palo Alto" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792440499208192, "text": "@isaiahordaz21 I thought we were friends", "in_reply_to_status": 694787846457044993, "in_reply_to_user": 1330294068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1330294068 }}, "user": { "id": 1596827804, "name": "Sylvia Cuevas", "screen_name": "CuevasSylvia", "lang": "en", "location": "null", "create_at": date("2013-07-15"), "description": "null", "followers_count": 249, "friends_count": 194, "statues_count": 5085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.932568,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792440717271040, "text": "@ebbtideapp Tide in Great Point Clear, Alabama 02/03/2016\n Low 6:54am -0.3\nHigh 5:32pm 1.2\n Low 7:22am -0.4\nHigh 6:20pm 1.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-87.9333,30.4833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 22, "friends_count": 1, "statues_count": 4533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Point Clear, AL", "id": "00c3940b3b15b9ef", "name": "Point Clear", "place_type": "city", "bounding_box": rectangle("-87.937145,30.460735 -87.899293,30.507821") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 161488, "cityName": "Point Clear" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792441694523392, "text": "I respect this nigga so much �� https://t.co/uFyZ6nhpbJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727172316, "name": "IVAN PEREZ", "screen_name": "Callmepapi_69", "lang": "en", "location": "del paso heights ", "create_at": date("2012-07-30"), "description": "#freedre$hmoney", "followers_count": 1415, "friends_count": 1144, "statues_count": 42003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792441832976384, "text": "Happy 16 Birthday to the sweetest, funniest, and overall best guy ever! Hope your day is as wonderful as you. Love you❤️ @HalloranConnor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4809222438 }}, "user": { "id": 2912186762, "name": "Brya Mauldin", "screen_name": "bryamauldin", "lang": "en", "location": "Cameron Park, CA", "create_at": date("2014-11-27"), "description": "connor halloran ♡ 01.10.16 //// phs '18 sc: bryabieber ig: brya.mauldin", "followers_count": 273, "friends_count": 451, "statues_count": 1455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron Park, CA", "id": "0f2405443201a34d", "name": "Cameron Park", "place_type": "city", "bounding_box": rectangle("-121.032906,38.65476 -120.924554,38.726092") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 610256, "cityName": "Cameron Park" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792441963139072, "text": "02/03@03:00 - Temp 38.4F, WC 38.4F. Wind 0.0mph NNW, Gust 1.0mph. Bar 30.263in, Falling. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792442822807552, "text": "Only island music can fix the mood I'm in tonight��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415692287, "name": "Kenz Mangino", "screen_name": "kenz_40", "lang": "en", "location": "null", "create_at": date("2011-11-18"), "description": "the best is yet to come", "followers_count": 211, "friends_count": 325, "statues_count": 2454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792442839576576, "text": "Just 6 more days ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 957010956, "name": "virgy virge", "screen_name": "Ver_Gee_Dee", "lang": "en", "location": "Los Angeles / La Mirada", "create_at": date("2012-11-19"), "description": "I found a Lego and a money -Devin", "followers_count": 223, "friends_count": 585, "statues_count": 15125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mirada, CA", "id": "4db990e393c2e28b", "name": "La Mirada", "place_type": "city", "bounding_box": rectangle("-118.037975,33.873395 -117.976352,33.928407") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640032, "cityName": "La Mirada" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792442873192448, "text": "@pammyknowsbest she really does ��", "in_reply_to_status": 694792341639426048, "in_reply_to_user": 1323900392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1323900392 }}, "user": { "id": 834489553, "name": "Yung_heartthrob.wav", "screen_name": "PharaohLilvon", "lang": "en", "location": "my spaceship", "create_at": date("2012-09-19"), "description": "20 | gemini | visual-artist | creating like God | inquires: pharaohlilvonart@gmail.com | Best Lee Sin player NA | #supportlivingartists #nofreeart", "followers_count": 861, "friends_count": 288, "statues_count": 49188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792443372269569, "text": "Of all the Disney movies those were the creepiest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2311228742, "name": "❕ COCONUT ❕", "screen_name": "_milivojevic_", "lang": "en", "location": "San Pedro", "create_at": date("2014-01-25"), "description": "I be in Prada like Perry Ellis", "followers_count": 547, "friends_count": 400, "statues_count": 8497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792443456266240, "text": "Wind 3.0 mph SW. Barometer 29.635 in, Rising Rapidly. Temperature 44.5 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792443514912768, "text": "3. Jaylin.Blackass Butler ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4186099094, "name": "Luchiano", "screen_name": "jumpman96ix", "lang": "en", "location": "null", "create_at": date("2015-11-09"), "description": "on a mission.... I'll tell you when it's accomplished", "followers_count": 66, "friends_count": 116, "statues_count": 1001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-03T00:00:01.000Z"), "id": 694792444248989696, "text": "@alberto_0277 @Emlys_my_name I look foward to it boss. U are truly a person that I Will forever hold dear to my heart. Thank u sir.", "in_reply_to_status": 694755674274844672, "in_reply_to_user": 2753579640, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2753579640, 537940778 }}, "user": { "id": 2883547072, "name": "Willie Aviles", "screen_name": "dolfan2480", "lang": "en", "location": "Florida City, FL", "create_at": date("2014-11-18"), "description": "null", "followers_count": 152, "friends_count": 74, "statues_count": 49 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792444584529922, "text": "Temp: 54.4°F Wind:2.7mph Pressure: 29.877hpa Rising Rapidly Rain Today 0.44in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792444647510016, "text": "@FiNZ63 I'm at my beautiful home in Daytona,and i have to leave tomorrow & go back to the city,i'm not happy :(", "in_reply_to_status": 694792029893595136, "in_reply_to_user": 1466976408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1466976408 }}, "user": { "id": 2360023117, "name": "Christine", "screen_name": "MissDaytona102", "lang": "en", "location": "Florida/Pennsylvania", "create_at": date("2014-02-24"), "description": "100%Country,Coffee,Locked&Reloaded, Fishing,4Wheelin,Outdoors, Football,Cooking,Crazy For Racing! LOVE my #PhiladelphiaEagles, #4Harvick Girl", "followers_count": 655, "friends_count": 494, "statues_count": 6213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daytona Beach, FL", "id": "5876e96f70283826", "name": "Daytona Beach", "place_type": "city", "bounding_box": rectangle("-81.133668,29.127565 -81.003444,29.252881") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1216525, "cityName": "Daytona Beach" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792445444395008, "text": "Wind 4.0 mph SW. Barometer 29.623 in, Rising. Temperature 59.7 °F. Rain today 0.53 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792445616205824, "text": "I got a iced out Bart where my heart used to be ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 894009998, "name": "CARTER II", "screen_name": "Ronald_Carter15", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-10-20"), "description": "|MAKING MYSELF BETTER THAN EVER|", "followers_count": 846, "friends_count": 1035, "statues_count": 5252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792446304198656, "text": ".@GHawe - @fmichaelhaynie and I are requesting a video of you doing THIS version of \"Ug-a-Wug\" - thanks so much https://t.co/ZLiEFzjZ2I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24831445, 110783502 }}, "user": { "id": 33420222, "name": "Matthew Rodin", "screen_name": "MrMattRodin", "lang": "en", "location": "NY/CHI", "create_at": date("2009-04-19"), "description": "bar mitzvah boy since '05 | \nYouTube: http://bit.ly/MrMattRodin | \nInsta/Snap: MrMattRodin", "followers_count": 1188, "friends_count": 771, "statues_count": 1905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792446354411520, "text": "Happy birthday @adnariv I love you momma ��������♒️ #GangGang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GangGang" }}, "user_mentions": {{ 605507434 }}, "user": { "id": 1586353266, "name": "Kee Riche$", "screen_name": "IamKeeRiches", "lang": "en", "location": "Compton", "create_at": date("2013-07-11"), "description": "Booking - kianjai26@gmail.com | #WestRiches", "followers_count": 1513, "friends_count": 998, "statues_count": 36340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792446878859264, "text": "@Only1CrystalFox \"Hanna is TRUTH\"...I love your character pay day is coming cool 12mill inheritance...", "in_reply_to_status": 694723544673062912, "in_reply_to_user": 1395881846, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1395881846 }}, "user": { "id": 2206714681, "name": "Phyllis Council", "screen_name": "Preach_Council", "lang": "en", "location": "null", "create_at": date("2013-11-20"), "description": "null", "followers_count": 292, "friends_count": 334, "statues_count": 22979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverview, FL", "id": "011aeb76023deccb", "name": "Riverview", "place_type": "city", "bounding_box": rectangle("-82.369079,27.755502 -82.244366,27.899232") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1260950, "cityName": "Riverview" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792446996131840, "text": "Would u rather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2829111888, "name": "Storm", "screen_name": "dezzlaws", "lang": "en", "location": "California, USA", "create_at": date("2014-09-23"), "description": "IG:: DezzLaws\nSC: DezLaws", "followers_count": 617, "friends_count": 303, "statues_count": 6347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792447105191938, "text": "You're such a good friend. https://t.co/nwZZpau2pL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171929967, "name": "Jose", "screen_name": "ComeManguu", "lang": "en", "location": "21 Kissimmee, Fl.", "create_at": date("2010-07-28"), "description": "Roll another one cuz I'm winnin", "followers_count": 1061, "friends_count": 676, "statues_count": 92719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792447264694272, "text": "Wind 0.0 mph NE. Barometer 30.24 in, Falling. Temperature 30.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792447444910082, "text": "@Michelelf I'm waiting for cheesy bread to be delivered rn brah", "in_reply_to_status": 694792295820820481, "in_reply_to_user": 34419854, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34419854 }}, "user": { "id": 505584797, "name": "Hannah", "screen_name": "hannyboobooo", "lang": "en", "location": "209/805", "create_at": date("2012-02-26"), "description": "UCSB☀️⚓️", "followers_count": 278, "friends_count": 192, "statues_count": 17062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792447516213248, "text": "Slept all day... Can't sleep now :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274619625, "name": "Julissa Castellanos", "screen_name": "juliebabez89", "lang": "en", "location": "Linden, NJ", "create_at": date("2011-03-30"), "description": "null", "followers_count": 247, "friends_count": 329, "statues_count": 7159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linden, NJ", "id": "b8d145028b2571ac", "name": "Linden", "place_type": "city", "bounding_box": rectangle("-74.290477,40.593604 -74.200434,40.654943") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3440350, "cityName": "Linden" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792447575080960, "text": "Wind 1.3 mph SW. Barometer 29.92 in, Rising. Temperature 57.9 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-03T00:00:02.000Z"), "id": 694792448120217601, "text": "@EzauMarujoz", "in_reply_to_status": 688571535246798848, "in_reply_to_user": 1875388436, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1875388436 }}, "user": { "id": 3703991954, "name": "kevin valencia", "screen_name": "kevinva27684190", "lang": "en", "location": "Colorado, USA", "create_at": date("2015-09-27"), "description": "null", "followers_count": 10, "friends_count": 54, "statues_count": 11 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pueblo, CO", "id": "9d7b47e751be1551", "name": "Pueblo", "place_type": "city", "bounding_box": rectangle("-104.69356,38.200638 -104.551992,38.338462") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8101, "countyName": "Pueblo", "cityID": 862000, "cityName": "Pueblo" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448560607232, "text": "���� A PERF DAB ���� BCC: @leespielman @alexanderspit @ Cha Cha Lounge https://t.co/EyhCa3rX4O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.25943245,34.09937595"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28238261, 17785959 }}, "user": { "id": 14480889, "name": "glenjamn", "screen_name": "glenjamn", "lang": "en", "location": "DAFT.", "create_at": date("2008-04-22"), "description": "PUNK.", "followers_count": 5376, "friends_count": 675, "statues_count": 14974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448610951168, "text": "I haven't been so mad that I've cried in years.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1240431,34.0173713"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1267906729, "name": "Zachariah Smith", "screen_name": "chomper1024", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "Redlands. \nEmployed.\nWorking on my own empire.", "followers_count": 583, "friends_count": 551, "statues_count": 4721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448711663616, "text": "Really finna be outta there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64107366, "name": "Jordan Belfort", "screen_name": "Ogsledge", "lang": "en", "location": "Long Beach. Htx.", "create_at": date("2009-08-08"), "description": "TreatYoSelf.", "followers_count": 784, "friends_count": 550, "statues_count": 56104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448757792768, "text": "I think I want 3 wives definitely 2 for sure tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540506852, "name": "Kountry Ass Nigga", "screen_name": "charnalvon", "lang": "en", "location": "Camarillo, CA Athens, GA ", "create_at": date("2012-03-29"), "description": "back in the day when we we're slaves I bet we was some cool ass niggas RIP @Incredible_Vezz", "followers_count": 1334, "friends_count": 502, "statues_count": 71809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448900354049, "text": "Things just change out of no where and that's how life works ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2522748547, "name": "Pino Maranan", "screen_name": "alxismaranan", "lang": "en", "location": "null", "create_at": date("2014-05-25"), "description": "Alicia ❤️", "followers_count": 184, "friends_count": 140, "statues_count": 460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792448934088705, "text": "#SupportOriginMelissa 61.2°F Wind:4.9mph Pressure: 29.68hpa Steady Rain Today 0.00in. Forecast: Unsettled, some precipitation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792450066386944, "text": "@iamquirky22 @spiglao Kaloka ka! lols #VoteMaineFPP #KCA", "in_reply_to_status": 694791533464129536, "in_reply_to_user": 3035574385, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "VoteMaineFPP", "KCA" }}, "user_mentions": {{ 3035574385, 91900617 }}, "user": { "id": 29121689, "name": "missT", "screen_name": "_bigchill_", "lang": "en", "location": "Evergreen State", "create_at": date("2009-04-05"), "description": "red warrior. rusher. runner. forever reginian. patrick dempsey. park seo joon. kim jong kook. yoon eun hye. jang woo young. kpop obsessed.", "followers_count": 275, "friends_count": 170, "statues_count": 30563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracyton, WA", "id": "a6e029a1d6b570db", "name": "Tracyton", "place_type": "city", "bounding_box": rectangle("-122.678381,47.592012 -122.62069,47.63234") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5372205, "cityName": "Tracyton" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792450288787456, "text": "10��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3376834377, "name": "Rob", "screen_name": "robertD2wild_", "lang": "en", "location": "null", "create_at": date("2015-07-14"), "description": "#idgt #FC3 #XLIII", "followers_count": 157, "friends_count": 150, "statues_count": 3830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792450385117184, "text": "24 hours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173323343, "name": "Darien Mann", "screen_name": "demann2345", "lang": "en", "location": "null", "create_at": date("2010-07-31"), "description": "null", "followers_count": 61, "friends_count": 77, "statues_count": 296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792450762625024, "text": "NEIGHBORS ARE PLAYING AKON THE COMEBACK IS REAL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 623258111, "name": "Jake Martinez", "screen_name": "jakedmartinez", "lang": "en", "location": "oc✈️eug", "create_at": date("2012-06-30"), "description": "I feel that.", "followers_count": 342, "friends_count": 330, "statues_count": 5564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792451312132096, "text": "53.8F (Feels: 53.8F) - Humidity: 99% - Wind: 1.6mph NE - Gust: 1.6mph - Pressure: 1023.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 225184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792451878309888, "text": "�� https://t.co/ulOzY8eA5s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2156197315, "name": "yvonnbon", "screen_name": "_yvonnbon", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "null", "followers_count": 225, "friends_count": 177, "statues_count": 7325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cudahy, CA", "id": "440191f4fe9d9752", "name": "Cudahy", "place_type": "city", "bounding_box": rectangle("-118.201978,33.955099 -118.170556,33.97089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617498, "cityName": "Cudahy" } }
+{ "create_at": datetime("2016-02-03T00:00:03.000Z"), "id": 694792452339793921, "text": "@ladeex3 ��������������, I can't deal ����him", "in_reply_to_status": 694792003561877504, "in_reply_to_user": 457166275, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 457166275 }}, "user": { "id": 194288964, "name": "BraceFace✨", "screen_name": "BigbootyIvy", "lang": "en", "location": "MostChosen", "create_at": date("2010-09-23"), "description": "null", "followers_count": 1456, "friends_count": 1376, "statues_count": 50157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792453543448576, "text": "������ https://t.co/sKjA5jmeGZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 604642144, "name": "Edgar Plascencia", "screen_name": "EdgarPlas", "lang": "en", "location": "Suisun City/Long Beach, CA", "create_at": date("2012-06-10"), "description": "Filmmaker, film critic, film lover. Batman is life. Director/Writer/Co-Founder of RP Pictures http://Facebook.com/RPPictures2012", "followers_count": 170, "friends_count": 154, "statues_count": 14518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792453652660224, "text": "Saw one of the dudes from #magicmikexxl at a comedy open mic tonight. Never has \"I love your work.\" Felt so true.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "magicmikexxl" }}, "user": { "id": 368691907, "name": "Grace Lusk", "screen_name": "gracelusk", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "I do stand up and host things like shows, parties, and aliens inside me. @graycpoo on instagram", "followers_count": 572, "friends_count": 652, "statues_count": 637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792454113992704, "text": "dog I'm so sleepy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 631634113, "name": "coolasstxai", "screen_name": "astoldbytxai_", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "I'm up rn", "followers_count": 1511, "friends_count": 342, "statues_count": 77695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerville, SC", "id": "676d5bedd73ca2ec", "name": "Summerville", "place_type": "city", "bounding_box": rectangle("-80.235287,32.944965 -80.109189,33.054993") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45035, "countyName": "Dorchester", "cityID": 4570270, "cityName": "Summerville" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792454407467008, "text": "Determined", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "no", "is_retweet": false, "user": { "id": 2173173182, "name": "Adrianna", "screen_name": "adriaaannnna", "lang": "en", "location": "Gerardo's", "create_at": date("2013-11-03"), "description": "null", "followers_count": 207, "friends_count": 107, "statues_count": 1023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792454587883521, "text": "I hate when people use Facebook like Twitter. It fucks my time line up.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 138458225, "name": "douglas jenkins", "screen_name": "dougiefresh8402", "lang": "en", "location": "�T: 33.845036,-118.391654", "create_at": date("2010-04-29"), "description": "null", "followers_count": 243, "friends_count": 217, "statues_count": 8128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792454931902465, "text": "@naythegreat_ Aye Nigga !", "in_reply_to_status": -1, "in_reply_to_user": 130292058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 130292058 }}, "user": { "id": 106027471, "name": "DJ Does It All", "screen_name": "ThereGo_Avion", "lang": "en", "location": "My Business", "create_at": date("2010-01-17"), "description": "@DefinitionDjs | 4X DJ Battle Champion | #CKB | #kingzOfCommerce | #TAMUC | #1525 #RIPDJKIDD #RIPCLYDE", "followers_count": 4702, "friends_count": 3712, "statues_count": 155856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792455569477632, "text": "The shit you was spitting was so unoriginal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229196124, "name": "☄", "screen_name": "KeepCalmmm_", "lang": "en", "location": "null", "create_at": date("2010-12-21"), "description": "• 32 fℓανσяѕ αи∂ тнєи ѕσмє • мємρнιѕ • 21 • вєунινє", "followers_count": 601, "friends_count": 202, "statues_count": 16619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792455959420928, "text": "I love Valentine's Day for some reason it makes me really happy to see other people happy.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2857086422, "name": "Michi.❁", "screen_name": "MichiValencia17", "lang": "en", "location": "null", "create_at": date("2014-10-15"), "description": "04.27.15", "followers_count": 129, "friends_count": 179, "statues_count": 1546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quincy, WA", "id": "e993fc6ac20d853d", "name": "Quincy", "place_type": "city", "bounding_box": rectangle("-119.893576,47.219448 -119.813649,47.248521") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5357115, "cityName": "Quincy" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792456089391106, "text": "@bruno_amber oh hey �� Amber ��", "in_reply_to_status": 694792331359186944, "in_reply_to_user": 493749319, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493749319 }}, "user": { "id": 3116058064, "name": "Stephanie Camberg", "screen_name": "StephCamberg", "lang": "en", "location": "Monmouth, OR", "create_at": date("2015-03-25"), "description": "A little truth, a little frustration, cold beer, conversation ✨ proverbs 4:23", "followers_count": 465, "friends_count": 375, "statues_count": 1767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792456517189632, "text": "@Domgomezz Happy birthday Dom have a good one ����������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": 1865290452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1865290452 }}, "user": { "id": 2263944516, "name": "{ D-Flame❗️}", "screen_name": "DeanSupremex", "lang": "en", "location": "Daygo✈️", "create_at": date("2013-12-26"), "description": "La Flame says let your ambition carry you . IG:deaaaaan_ // 17 / Music is a major key", "followers_count": 449, "friends_count": 357, "statues_count": 10719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792456731127809, "text": "from the Lumineers to Waka Flocka - I'm two different beasts lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1260677726, "name": "Mu Cephei", "screen_name": "Izzy_Q3", "lang": "en", "location": "The Deepest Part of the Sea", "create_at": date("2013-03-11"), "description": "Follow, let's talk| You tweet to enjoy, I tweet to die. | writer. | whiskey. | PNW. | WSU. | Water Bender | Eat her up like mochi | V X 3", "followers_count": 586, "friends_count": 563, "statues_count": 17530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyside, WA", "id": "00cce068663f6743", "name": "Sunnyside", "place_type": "city", "bounding_box": rectangle("-120.046373,46.286764 -119.964934,46.343583") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5368750, "cityName": "Sunnyside" } }
+{ "create_at": datetime("2016-02-03T00:00:04.000Z"), "id": 694792456802439168, "text": "������ https://t.co/psRZwIG3ep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 512928288, "name": "Mohammed Al zuraya", "screen_name": "MZuraya", "lang": "en", "location": "Tulsa, United States. ", "create_at": date("2012-03-03"), "description": "خارج الوطن حاملاً همه…ليعود يوماً مرتدياً درعاً يذود به…جدٌ وهزلٌ…حكمةٌ و خليطٌ من مشاعر مغترب", "followers_count": 241, "friends_count": 273, "statues_count": 2239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tahlequah, OK", "id": "c9bcc362968ea245", "name": "Tahlequah", "place_type": "city", "bounding_box": rectangle("-95.012128,35.856248 -94.932594,35.976425") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40021, "countyName": "Cherokee", "cityID": 4072100, "cityName": "Tahlequah" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457175740416, "text": "tomorrow could be my last high school soccer game .... but in not going to let that happen #determined ⚽️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "determined" }}, "user": { "id": 1539200874, "name": "tor", "screen_name": "_toriooo", "lang": "en", "location": "Hilo, Hawaii", "create_at": date("2013-06-22"), "description": "bigislandxhawaii☀️", "followers_count": 258, "friends_count": 244, "statues_count": 1183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilo, HI", "id": "00430f8519d62c37", "name": "Hilo", "place_type": "city", "bounding_box": rectangle("-155.184552,19.656471 -154.990994,19.738955") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii", "cityID": 1514650, "cityName": "Hilo" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457406394369, "text": "temperature down 54°F -> 43°F\nhumidity up 34% -> 49%\nwind 10mph -> 12mph\npressure 30.04in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.0072,32.7477"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162093993, "name": "Weather", "screen_name": "_GrandPrairieTX", "lang": "en", "location": "Grand Prairie, TX", "create_at": date("2010-07-02"), "description": "Weather updates, forecast, warnings and information for Grand Prairie, TX. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 97, "friends_count": 3, "statues_count": 23759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457473650689, "text": "@emilykingmusic you made my whole YEAR!!! Thank you for making it all the way out here to… https://t.co/6wSPyofwhe", "in_reply_to_status": -1, "in_reply_to_user": 32229500, "favorite_count": 0, "coordinate": point("-122.6569468,45.52268947"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32229500 }}, "user": { "id": 72956185, "name": "Michal Angela Wilson", "screen_name": "michalangela", "lang": "en", "location": "Portland, OR", "create_at": date("2009-09-09"), "description": "Vocalist. Singer. Jazz. Soul. R&B. Funk. Gospel. Band Leader. Songwriter. Christian. I Play The Throat. I Sing At Weddings. Soprano. Real Life Haver.", "followers_count": 495, "friends_count": 656, "statues_count": 1050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457507045376, "text": "6. Don’t sweat the small stuff\n#HappyPeople", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyPeople" }}, "user": { "id": 1240329355, "name": "عبدالعزيز الحيد", "screen_name": "AbdulazizAlhead", "lang": "ar", "location": "USA-SAUDI ARABIA ", "create_at": date("2013-03-03"), "description": "من دون حب الله لي .. أنا لا شيء.. ربما أقل، ولكني متأكدٌ أني لستُ أكثر", "followers_count": 262, "friends_count": 103, "statues_count": 891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, KS", "id": "4ccacb29ffb3b3df", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-94.724428,37.367357 -94.668019,37.444178") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20037, "countyName": "Crawford", "cityID": 2056025, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457544806400, "text": "@BrittanyFurlan The dog hiccuping just made a little bit of beer come out my nose. Not sure why I found it to be so funny. But 'twas.", "in_reply_to_status": -1, "in_reply_to_user": 21728396, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21728396 }}, "user": { "id": 38333179, "name": "Aaron Neville's Mole", "screen_name": "GulliverGooch", "lang": "en", "location": "Cameron Park, CA", "create_at": date("2009-05-06"), "description": "Im a French male model. Bonjour", "followers_count": 92, "friends_count": 429, "statues_count": 11102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792457989402624, "text": "Oooooo I struck a nerve ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3168033079, "name": "Tawny", "screen_name": "tawnlynn", "lang": "en", "location": "null", "create_at": date("2015-04-22"), "description": "null", "followers_count": 80, "friends_count": 79, "statues_count": 1718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792458555625472, "text": "You hurt me, but I forgive you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990442530, "name": "Mahinaonalani", "screen_name": "_maaaaahina_", "lang": "en", "location": "Hawaii", "create_at": date("2012-12-05"), "description": "The fridge broke, so I had to eat everything.", "followers_count": 249, "friends_count": 272, "statues_count": 4898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wahiawa, HI", "id": "f52453067076e5dc", "name": "Wahiawa", "place_type": "city", "bounding_box": rectangle("-158.051715,21.488622 -157.980627,21.512885") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1572650, "cityName": "Wahiawa" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792459042172928, "text": "@_daddyferg lol �� no ��", "in_reply_to_status": 694792193941176320, "in_reply_to_user": 298723162, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298723162 }}, "user": { "id": 524820136, "name": "InLove w/ LilDurk", "screen_name": "QsCee", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "null", "followers_count": 1716, "friends_count": 1159, "statues_count": 40271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792459482628096, "text": "#thankyouronnie https://t.co/8FBYBkk7pM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "thankyouronnie" }}, "user": { "id": 586338438, "name": "Efren", "screen_name": "efriend_", "lang": "en", "location": "trap house", "create_at": date("2012-05-20"), "description": "it's always real nigga hours somewhere . I'm a nice guy tbh", "followers_count": 654, "friends_count": 661, "statues_count": 23782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792459549683712, "text": "2.4 magnitude #earthquake. 68 km from #AnchorPoint, AK, United States https://t.co/aHfHW5zvj8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-153.044,59.793"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "AnchorPoint" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 63271, "friends_count": 10, "statues_count": 95150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2122, "countyName": "Kenai Peninsula" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792460082479104, "text": "@_Avrion_ she is very content", "in_reply_to_status": 694792049250455552, "in_reply_to_user": 728717748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 728717748 }}, "user": { "id": 2189247247, "name": "Ty", "screen_name": "FullStrSarcasm", "lang": "en", "location": "Beverly, MA", "create_at": date("2013-11-11"), "description": "A totally calm individual who tends to make a mockery of anything including himself. INTP 女娲", "followers_count": 157, "friends_count": 287, "statues_count": 64533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly, MA", "id": "7dea5f8c907d1ea2", "name": "Beverly", "place_type": "city", "bounding_box": rectangle("-70.923007,42.53915 -70.793176,42.593439") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2505595, "cityName": "Beverly" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792460292247553, "text": "جامعة ملك سعود ترمي ملفات المتقدمين بالزباله\n\nhttps://t.co/FUUfImcxtV\n\n#اوغلو_يدنس_قدسيه_الحرم\n#قول_كلمه_للداخليه\n#شوارع_باسماء_اعضاء_الشوري", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "hashtags": {{ "اوغلو_يدنس_قدسيه_الحرم", "قول_كلمه_للداخليه", "شوارع_باسماء_اعضاء_الشوري" }}, "user": { "id": 4810006872, "name": "راشد م3", "screen_name": "DdssdEkjfewkfw", "lang": "en", "location": "null", "create_at": date("2016-01-24"), "description": "null", "followers_count": 50, "friends_count": 93, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792460766048256, "text": "@TeezyTheGawd eat a sigh", "in_reply_to_status": 694792299935469568, "in_reply_to_user": 585177246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 585177246 }}, "user": { "id": 2863157376, "name": "bryson tillard ⚡️", "screen_name": "Scotty_drippen", "lang": "en", "location": "Cisco, TX", "create_at": date("2014-10-18"), "description": "pray on it always learn from mistakes and become a better man from it #210 ✈️#254 lets eat", "followers_count": 463, "friends_count": 454, "statues_count": 6606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cisco, TX", "id": "721e5b312b06ad49", "name": "Cisco", "place_type": "city", "bounding_box": rectangle("-99.004915,32.374537 -98.956093,32.401675") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48133, "countyName": "Eastland", "cityID": 4815004, "cityName": "Cisco" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792460854120448, "text": "S/o to all the dads at siliman center that tried to arrange marriages with me Bc I look of middle eastern decent, I'm just a light Latino...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367608987, "name": "Jlo", "screen_name": "PLURryface", "lang": "en", "location": "Bay Area & Sacramento", "create_at": date("2011-09-04"), "description": "Student, raver, journalist, art hoe, clean cut hippie & social justice supporter. Coolest thing about me is I try to be a decent human and I'm pretty odd.", "followers_count": 334, "friends_count": 335, "statues_count": 28783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685512") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-03T00:00:05.000Z"), "id": 694792461063852033, "text": "mines was 2 months �� https://t.co/vCrNLZUWcP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301223732, "name": "robwiththashit", "screen_name": "robfromthed", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-05-18"), "description": "mayonnaise colored Benz I push miracle whips -YEEZUS", "followers_count": 3166, "friends_count": 1980, "statues_count": 33609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Oak, TX", "id": "d91428b130ec1da5", "name": "Red Oak", "place_type": "city", "bounding_box": rectangle("-96.839528,32.484371 -96.70256,32.547292") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4861196, "cityName": "Red Oak" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792462087245826, "text": "\"Teens have unprotected sex but have a case for their phone\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169236891, "name": "Queen of Darkness", "screen_name": "CassidyBuechler", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-07-21"), "description": "Fueled by art, tequila, beer and questionable decisions.", "followers_count": 1468, "friends_count": 607, "statues_count": 18493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792462284513280, "text": "RARE https://t.co/yHjTcYxc1D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356531871, "name": "Moorish God", "screen_name": "Yeah_lean", "lang": "en", "location": "hidden leaf village ", "create_at": date("2011-08-16"), "description": "a lion doesn't lose sleep over the opinion of 100 sheep", "followers_count": 441, "friends_count": 494, "statues_count": 3262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscarawas, OH", "id": "8bdeb5af4ef23efc", "name": "Tuscarawas", "place_type": "city", "bounding_box": rectangle("-81.412475,40.392321 -81.389397,40.40512") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39157, "countyName": "Tuscarawas", "cityID": 3977924, "cityName": "Tuscarawas" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792462414381057, "text": "@bookiebadAF ��", "in_reply_to_status": 694792107299467265, "in_reply_to_user": 378284190, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 378284190 }}, "user": { "id": 1616970012, "name": "ashley", "screen_name": "_ashleyniicolee", "lang": "en", "location": "Austin, TX", "create_at": date("2013-07-23"), "description": "University of Houston | snapchat : ashleynicole729", "followers_count": 699, "friends_count": 563, "statues_count": 8205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792462947102720, "text": "Wind 0.0 mph ---. Barometer 30.354 in, Steady. Temperature 29.9 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 248185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792463064498178, "text": "Wind 8.0 mph NNW. Barometer 29.516 in, Rising. Temperature 31.5 °F. Rain today 0.01 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 11, "statues_count": 864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792463152644096, "text": "@IiIcfromthepack @lilrondo_ he lying", "in_reply_to_status": 694791913346457600, "in_reply_to_user": 1100123532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1100123532, 2828424644 }}, "user": { "id": 2558105766, "name": "Reverend Kaine", "screen_name": "llwomenarewrong", "lang": "en", "location": "909", "create_at": date("2014-06-09"), "description": "Retired w/ Marlene", "followers_count": 364, "friends_count": 268, "statues_count": 10556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio Heights, CA", "id": "6ec2cbb930694a99", "name": "San Antonio Heights", "place_type": "city", "bounding_box": rectangle("-117.684144,34.146895 -117.636701,34.163122") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 664462, "cityName": "San Antonio Heights" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792463408476161, "text": "101% me. https://t.co/s89owtvLru", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 340108960, "name": "Adam Turek", "screen_name": "apeshitadam", "lang": "en", "location": "Mukilteo, WA", "create_at": date("2011-07-21"), "description": "I lost my mind with my virginity.", "followers_count": 389, "friends_count": 244, "statues_count": 19410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Picnic Point, WA", "id": "007d0c773de7ec34", "name": "Picnic Point", "place_type": "city", "bounding_box": rectangle("-122.336551,47.82599 -122.245495,47.89362") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5354213, "cityName": "Picnic Point" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792463471370241, "text": "Please do not follow @ReturnOfKings https://t.co/P2kSfPzVi7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 872606893 }}, "user": { "id": 2254554944, "name": "Bridgett", "screen_name": "CortesBridgett", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-12-19"), "description": "my instagram is bridgett_lara feel free to follow and check out my art instagram \nbridgett_sculpts", "followers_count": 324, "friends_count": 999, "statues_count": 10434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792463509159936, "text": "Temp: 33.7°F - Dew Point: 32.1° - Wind: 2.4 mph - Gust: 4.5 - Rain Today: 0.00in. - Pressure: 29.42in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792464347955200, "text": "one more time & I'm done ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1447508305, "name": "Fab", "screen_name": "rad_fabb", "lang": "en", "location": "null", "create_at": date("2013-05-21"), "description": "19 | Psychology Major at Norco College.", "followers_count": 211, "friends_count": 195, "statues_count": 15524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792464578646016, "text": "@Omar_56Duece for the love of money ❤️��", "in_reply_to_status": 694792104699006976, "in_reply_to_user": 295834970, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 295834970 }}, "user": { "id": 4623742760, "name": "Rich Homie Reck", "screen_name": "_kingloui", "lang": "en", "location": "null", "create_at": date("2015-12-21"), "description": "null", "followers_count": 56, "friends_count": 86, "statues_count": 182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792464675184640, "text": "#Sales #Job alert: SME Sales Representative- Regional | Trustwave | #Chicago, IL https://t.co/pbGZYw5TNt #infosec #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "Job", "Chicago", "infosec", "Jobs", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 274, "friends_count": 160, "statues_count": 569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792464847077376, "text": "2/3/2016 - 02:00\nTemp: 44.6F \nHum: 53%\nWind: 2.0 mph\nBaro: 29.853in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 52, "statues_count": 49618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-03T00:00:06.000Z"), "id": 694792464876503041, "text": "@YupSheCalimade they don't understand", "in_reply_to_status": 694792406361706497, "in_reply_to_user": 172138272, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 172138272 }}, "user": { "id": 3226822026, "name": "Ω̴̩̩̩̥VΩ̴̩̩̩̥", "screen_name": "AndraeO_o", "lang": "en", "location": "Dallas Tx", "create_at": date("2015-05-25"), "description": "Dreams Money Can Buy Free My Brother @NikeStreetz #DubNation", "followers_count": 1709, "friends_count": 397, "statues_count": 54756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792465648234496, "text": "That moment when the hotbox made you warm lol ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598475110, "name": "Chris", "screen_name": "_36wu", "lang": "en", "location": "East Los Angeles", "create_at": date("2012-06-03"), "description": "Mindset like pouya $LUGS ❄", "followers_count": 1158, "friends_count": 598, "statues_count": 53477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792465933537280, "text": "Who sleeps with socks on?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2447908915, "name": "TylorColville__", "screen_name": "colville_", "lang": "en", "location": "null", "create_at": date("2014-04-16"), "description": "Philly, fishtown", "followers_count": 346, "friends_count": 402, "statues_count": 3836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792466147315712, "text": "Happy 17th to this loser���� https://t.co/gi5rKQXzHI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2907360609, "name": "Meghan Nader", "screen_name": "meghannader", "lang": "en", "location": "somewhere smiling", "create_at": date("2014-12-05"), "description": "Bellarmine Prep.", "followers_count": 324, "friends_count": 179, "statues_count": 968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Place, WA", "id": "9cd68e26f3c4880c", "name": "University Place", "place_type": "city", "bounding_box": rectangle("-122.584288,47.185226 -122.505193,47.245138") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5373465, "cityName": "University Place" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792466659172352, "text": "Barberton Oh Temp:50.4°F Wind:6 mph Dir:SSE Baro:Falling Rain2day:0.28in Hum:88% UV:0.0 @ 03:00 02/03/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 228, "statues_count": 114370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820272,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792467502071808, "text": "Todo era más sencillo cuando éramos niños, las mujeres nos valían madre, el amor no importaba y sólo queríamos voltear 3 tazos de un tiro. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 279370918, "name": "DjBambino", "screen_name": "dvjbambino", "lang": "en", "location": "Austin TX", "create_at": date("2011-04-08"), "description": "Let the Bass kick !!!!!!!!!!", "followers_count": 944, "friends_count": 1919, "statues_count": 3930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.500723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-02-03T00:00:07.000Z"), "id": 694792467619557376, "text": "Oh hell nah ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467891922, "name": "Andrea", "screen_name": "macdredre", "lang": "en", "location": "null", "create_at": date("2012-01-18"), "description": "chillona pero chingona", "followers_count": 885, "friends_count": 554, "statues_count": 48295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154823843319808, "text": "I love having night mares", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1884447558, "name": "Megan Hosmer", "screen_name": "mango_megano", "lang": "en", "location": "Tulsa, Oklahoma", "create_at": date("2013-09-19"), "description": "o p i n i o n a t e d | T U", "followers_count": 147, "friends_count": 115, "statues_count": 2508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154823847522304, "text": "If you're a #SkilledTrade professional in #Horsham, PA, check out this #job: https://t.co/LOAJ1ePiyY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1285061,40.1784422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkilledTrade", "Horsham", "job", "Hiring" }}, "user": { "id": 1011811424, "name": "Tyco Careers", "screen_name": "TycoCareers", "lang": "en", "location": "North America", "create_at": date("2012-12-14"), "description": "The world’s largest pure-play fire and security company is looking for talented people. Join the winning team!", "followers_count": 1011, "friends_count": 138, "statues_count": 9423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Horsham, PA", "id": "9b977bdde8553e88", "name": "Horsham", "place_type": "city", "bounding_box": rectangle("-75.168828,40.156907 -75.106896,40.21179") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4235800, "cityName": "Horsham" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154823851855873, "text": "Top of the morning world", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2396644316, "name": "Frig", "screen_name": "Frig_BK", "lang": "en", "location": "Somewhere Working", "create_at": date("2014-03-18"), "description": "Move Slient and let your work speak.", "followers_count": 98, "friends_count": 198, "statues_count": 1411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154823965057025, "text": "We live and learn..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160435034, "name": "Deege", "screen_name": "LionDeege", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-06-27"), "description": "Fear God Not Men", "followers_count": 503, "friends_count": 370, "statues_count": 8846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154824069922816, "text": "I took the train all the way to 168 from carajo queens at my friends babyshower on a full bladder", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85176601, "name": "mary jane", "screen_name": "PensPaperPoetry", "lang": "en", "location": "Downtown/Uptown/Switzerland", "create_at": date("2009-10-25"), "description": "null", "followers_count": 633, "friends_count": 366, "statues_count": 125057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154824724099072, "text": "This everyday RT @TheRealYoungZel: Currently being ugly at work https://t.co/yJq74X9HCm", "in_reply_to_status": 695154569681063936, "in_reply_to_user": 261102661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261102661 }}, "user": { "id": 62361179, "name": "GUWOP", "screen_name": "PhuckAlia", "lang": "en", "location": "Laventille, Trinidad ✈ Dallas ", "create_at": date("2009-08-02"), "description": "20. Fashion Design @ UNT. Not interested\n~Rest Easy TreyPax~ ~LongLiveMoney~", "followers_count": 1579, "friends_count": 618, "statues_count": 86554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154825319723008, "text": "\"The house is acid proof, but the world is NOT acid proof\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239334219, "name": "Nono", "screen_name": "scottysippin", "lang": "en", "location": "Somewhere in Texas ", "create_at": date("2011-01-17"), "description": "The roof is not my son, but I will raise it.", "followers_count": 367, "friends_count": 627, "statues_count": 18758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154825604935680, "text": "i never get ready for school lol how sad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220804130, "name": "jesss ☾❃", "screen_name": "yoursssstruly_", "lang": "en", "location": "null", "create_at": date("2010-11-28"), "description": "Bitchhhh ❤️", "followers_count": 510, "friends_count": 222, "statues_count": 53462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154826230026240, "text": "I'm wit it https://t.co/0P9d1bldTB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574954515, "name": "young og", "screen_name": "sky_the_limit13", "lang": "en", "location": "null", "create_at": date("2012-05-08"), "description": "null", "followers_count": 395, "friends_count": 375, "statues_count": 5591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154826519408642, "text": "@Juicebayb123 k bye , you showin off for twitter now ✌��️", "in_reply_to_status": 695154615898259456, "in_reply_to_user": 1607194614, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1607194614 }}, "user": { "id": 339670609, "name": "$HADE GANG ❣", "screen_name": "Luhhlexx__", "lang": "en", "location": "somewhere w. ya wife", "create_at": date("2011-07-21"), "description": "In love with Meinyonn &&' Jasmine Renee ❤", "followers_count": 1385, "friends_count": 1138, "statues_count": 79602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154826779484160, "text": "Just the connection, that shit was real.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490963274, "name": "Kareem Abdel", "screen_name": "kabdel_", "lang": "en", "location": "New York, NY", "create_at": date("2012-02-12"), "description": "some Arab guy", "followers_count": 168, "friends_count": 122, "statues_count": 10109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154826875785216, "text": "OMG I know what I'm buying with the $150 Christmas check the Dr gave me!! #beboop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "beboop" }}, "user": { "id": 51208462, "name": "Ariel Gutierrez", "screen_name": "_arieltp8", "lang": "en", "location": "null", "create_at": date("2009-06-26"), "description": "Love never fails ♡ 1 Corinthians 13:8", "followers_count": 113, "friends_count": 72, "statues_count": 7582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154826926170116, "text": "Good night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83404702, "name": "UnkleNuk", "screen_name": "Im__Nuk", "lang": "en", "location": "Orange, TX", "create_at": date("2009-10-18"), "description": "Laughing is Healing 22 #TeamSwish #BabyK #FruitCity #2600 #WolfGang sc: NuksBrasi", "followers_count": 1757, "friends_count": 1746, "statues_count": 84008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, TX", "id": "3fe7643d3e014f67", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-93.858146,30.061031 -93.715588,30.193051") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48361, "countyName": "Orange", "cityID": 4854132, "cityName": "Orange" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154827198726145, "text": "all we wanted to do was get a picture of our face tats. smh. lol & out of 15 pictures, that one looked the best. smh. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286987360, "name": "illuminate muvva✨", "screen_name": "MUVVABLEU", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-04-23"), "description": "pay the cost, be the boss. INFY.", "followers_count": 3254, "friends_count": 1522, "statues_count": 39369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154827240681473, "text": "@awiciaaaaa happy birthday gorgeous have a great day you deserve it I miss you and let's hang out soon ��������", "in_reply_to_status": -1, "in_reply_to_user": 3585359414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3585359414 }}, "user": { "id": 260074992, "name": "jayyyy❥", "screen_name": "JaymeeEller", "lang": "en", "location": "California, USA", "create_at": date("2011-03-02"), "description": ":)", "followers_count": 1382, "friends_count": 811, "statues_count": 35353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Foothill Farms, CA", "id": "3dc5823629ba0580", "name": "Foothill Farms", "place_type": "city", "bounding_box": rectangle("-121.373031,38.66773 -121.326256,38.705859") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624722, "cityName": "Foothill Farms" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154827735662593, "text": "When you get goodnight texts & you're still at work lmao bye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3244958389, "name": "mel❣", "screen_name": "chillissaaa", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-06-14"), "description": "I like techno, champagne & my nutri bullet ♊️♌️ wedidit", "followers_count": 257, "friends_count": 271, "statues_count": 15697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Encinitas, CA", "id": "1f6b47c3f3352385", "name": "Encinitas", "place_type": "city", "bounding_box": rectangle("-117.312091,32.999469 -117.195721,33.090549") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622678, "cityName": "Encinitas" } }
+{ "create_at": datetime("2016-02-04T00:00:00.000Z"), "id": 695154827937058816, "text": "Wind 2.7 mph S. Barometer 30.22 in, Rising slowly. Temperature 43.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154828025024512, "text": "@_Pure_Imagery damn you doubtin me too.", "in_reply_to_status": 695154687822024704, "in_reply_to_user": 45966054, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45966054 }}, "user": { "id": 2806405639, "name": "mia.", "screen_name": "_theQM", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-09-12"), "description": "• I'm just living ..", "followers_count": 1137, "friends_count": 855, "statues_count": 16659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar City, UT", "id": "013884df639610f8", "name": "Cedar City", "place_type": "city", "bounding_box": rectangle("-113.152491,37.647433 -113.032184,37.736012") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49021, "countyName": "Iron", "cityID": 4911320, "cityName": "Cedar City" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154828742369280, "text": "Jaden Smith is fine ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4153467618, "name": "D", "screen_name": "DaijhaSmithh", "lang": "en", "location": "null", "create_at": date("2015-11-06"), "description": "Model | Actor - GHT Agency | Married to the game #22 #2k18 #BALLISLIFE ⛹ Mom❣@ImAngelaBassett", "followers_count": 316, "friends_count": 398, "statues_count": 2433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154829212020736, "text": "@ebbtideapp Tide in Squamscott River RR. Bridge, New Hampshire 02/04/2016\n Low 3:39am 1.1\nHigh 9:36am 6.7\n Low 4:27pm 0.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.9133,43.0533"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 21, "friends_count": 1, "statues_count": 4810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Hampshire, USA", "id": "226b21641df42460", "name": "New Hampshire", "place_type": "admin", "bounding_box": rectangle("-72.557247,42.696978 -70.575095,45.305476") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33015, "countyName": "Rockingham" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154829459501057, "text": "@kyrasantoro I gotchu, future-wife!", "in_reply_to_status": 695153696598917120, "in_reply_to_user": 296742497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296742497 }}, "user": { "id": 188628859, "name": "C-daddy!", "screen_name": "lolfuxno", "lang": "en", "location": "null", "create_at": date("2010-09-08"), "description": "You know the burnt out college student who doesn't sleep and does Calculus and Physics all day? Nice to meet you, too. Aerospace engineer at ASU.", "followers_count": 288, "friends_count": 282, "statues_count": 1841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154829509791744, "text": "13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1445637774, "name": "Adrian ⚡️", "screen_name": "thatdudeshever", "lang": "en", "location": "null", "create_at": date("2013-05-20"), "description": "T.D.E X Dreamville", "followers_count": 100, "friends_count": 156, "statues_count": 3873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830008983552, "text": "@chathamtully @lbarnes96", "in_reply_to_status": 695154719879073792, "in_reply_to_user": 1323915720, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1323915720, 4688014243 }}, "user": { "id": 1323915720, "name": "chad", "screen_name": "chathamtully", "lang": "en", "location": "Juneau, AK", "create_at": date("2013-04-02"), "description": "if you cant hang, don't roll", "followers_count": 366, "friends_count": 274, "statues_count": 3332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830050877440, "text": "I'm Jordan ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111439847, "name": "lil mama", "screen_name": "daalienprincess", "lang": "en", "location": "flexas ", "create_at": date("2010-02-04"), "description": "My name is Jordan & I'm too goofy for my own good. tsnmi. txst. ♓️ be kind to everything that lives.", "followers_count": 1101, "friends_count": 451, "statues_count": 75844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830340415488, "text": "Wind 2.0 mph NW. Barometer 30.286 in, Rising. Temperature 29.3 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830671810561, "text": "02/04@03:00 - Temp 49.2F, WC 49.2F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.894in, Falling slowly. Rain 0.00in. Hum 99%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830763888640, "text": "I fucking miss your voice when I'm sad. I guess I relied on you too much, especially to cheer me up. Blah blah blah. I need alcohol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46726420, "name": "Lindsey", "screen_name": "MmkThxBai", "lang": "en", "location": "My Cubicle", "create_at": date("2009-06-12"), "description": "History major. \r\nStudying Korean. \r\nI am directionally challenged and get lost daily. Vending machines hate me and steal my money.", "followers_count": 85, "friends_count": 76, "statues_count": 3347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154830994595840, "text": "Can you pay my bills?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616675667, "name": "Jxoce", "screen_name": "jocexolyn", "lang": "en", "location": "null", "create_at": date("2012-06-23"), "description": "Wass good?", "followers_count": 275, "friends_count": 195, "statues_count": 22026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831049109504, "text": "Happy Birthday to my beautiful angel watching over me, love and miss you bestfriend ����❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455043153, "name": "J", "screen_name": "jasleenxm", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-05-24"), "description": "null", "followers_count": 647, "friends_count": 360, "statues_count": 29823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831137337344, "text": "shaq big as bricks lol ��\n\n #shaq #lightupent #shell #atlanta #videographer #shaquilleoneal… https://t.co/q4LdT6iRDE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3881,33.7489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shaq", "lightupent", "shell", "atlanta", "videographer", "shaquilleoneal" }}, "user": { "id": 159918456, "name": "Najee Rawlins", "screen_name": "LightUp_Ent", "lang": "en", "location": "GA", "create_at": date("2010-06-26"), "description": "#LightUp Entertainment | Video/Photo Production company |347.461.0821| IG: Lightupent | Email: Lightupent@yahoo.com| Contact me for a video.", "followers_count": 1177, "friends_count": 945, "statues_count": 5696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831158177793, "text": "If I'm quiet or not saying anything it's cus I don't trust u or ur energy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 187596754, "name": "BUNNY BAETZ", "screen_name": "cgrillonspg", "lang": "en", "location": "Trailer Park Chic ", "create_at": date("2010-09-06"), "description": "tempresss ~_~ transylvania", "followers_count": 683, "friends_count": 438, "statues_count": 52542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831267274752, "text": "@Kelsey_Cousins ��������", "in_reply_to_status": 695151423219052544, "in_reply_to_user": 549006051, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 549006051 }}, "user": { "id": 57720349, "name": "Michael Pham", "screen_name": "phaammyboy", "lang": "en", "location": "Iowa City, IA", "create_at": date("2009-07-17"), "description": "pls just retweet my pinned tweet..", "followers_count": 1173, "friends_count": 440, "statues_count": 35832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831447556097, "text": ".@tedCruz Campaign Push Polling In South Carolina…\nREAD: https://t.co/HE41VnoLIR https://t.co/rHWYNYKLnh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23022687 }}, "user": { "id": 4729317440, "name": "Apsinthos", "screen_name": "YugeMilo", "lang": "en", "location": "null", "create_at": date("2016-01-08"), "description": "**waiting for Ragnarök** I do naughty things to computers #AltRight", "followers_count": 31132, "friends_count": 8406, "statues_count": 29302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-02-04T00:00:01.000Z"), "id": 695154831707783168, "text": "Temp: 40.6°F Wind:0.0mph Pressure: 30.214hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 58951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154832584216576, "text": "In the mood for 2 backwoods and some Hennessy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46586125, "name": "Rich M U V A", "screen_name": "Indihaitian", "lang": "en", "location": "Kansas City, NO KANSAS", "create_at": date("2009-06-11"), "description": "| eat your heart out |", "followers_count": 1598, "friends_count": 543, "statues_count": 77658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.845486,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154832697462784, "text": "Got that old school playlist on point now!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21247151, "name": "DANNY", "screen_name": "DANNYLOW", "lang": "en", "location": "Los Angeles", "create_at": date("2009-02-18"), "description": "You heard right... I'm a POLEfessional! Stripper name : Ceelo Rice", "followers_count": 84, "friends_count": 289, "statues_count": 4541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154832739475457, "text": "Wind 0.0 mph NNW. Barometer 1032.28 mb, Rising. Temperature 23.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154832898973697, "text": "When u see brow tutorials for Anastasia Beverly Hills brow pomade and u actually cry but u using rue 21 eyeshadow currently on ur brows", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320933269, "name": "kc", "screen_name": "_artmom", "lang": "en", "location": "Buckingham / RVA", "create_at": date("2011-06-20"), "description": "FEMINIST VCUarts #BlackLivesMatter", "followers_count": 614, "friends_count": 371, "statues_count": 43279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154833028993024, "text": "Ptm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1315854516, "name": "JOMAYRAAA❁", "screen_name": "Jomayragr", "lang": "es", "location": "Nueva York, USA", "create_at": date("2013-03-29"), "description": "Sc:Jomayragr", "followers_count": 1027, "friends_count": 614, "statues_count": 7771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154833154637825, "text": "45.0F (Feels: 45.0F) - Humidity: 71% - Wind: 13.0mph NW - Gust: 13.6mph - Pressure: 1037.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 225332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154833280622592, "text": "Wind 2.0 mph WNW. Barometer 30.181 in, Rising. Temperature 33.5 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154833683132416, "text": "Layne ab to miss out @LayneM32 ��☁", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 937428133 }}, "user": { "id": 910589431, "name": "travis.", "screen_name": "TravisMooree", "lang": "en", "location": "cincy ", "create_at": date("2012-10-28"), "description": "cincinnati since my momma had me", "followers_count": 764, "friends_count": 676, "statues_count": 12659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forestville, OH", "id": "35b59fa30bb5106c", "name": "Forestville", "place_type": "city", "bounding_box": rectangle("-84.372416,39.040029 -84.299434,39.09231") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3927776, "cityName": "Forestville" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154834102562816, "text": "@kvnelvn listen here you fucking vienna sausage you smell of dick cheese and look like raven from teen titans", "in_reply_to_status": 695154532515340288, "in_reply_to_user": 304207830, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304207830 }}, "user": { "id": 2357029987, "name": "Ashley", "screen_name": "assley16", "lang": "en", "location": "flagstaff, az", "create_at": date("2014-02-22"), "description": "Everyday is a new rock bottom.", "followers_count": 147, "friends_count": 273, "statues_count": 2753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154834488438784, "text": "@RyStefano https://t.co/UN3bMsS3uB", "in_reply_to_status": -1, "in_reply_to_user": 2198718240, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2198718240 }}, "user": { "id": 2269649522, "name": "gabs", "screen_name": "gabibauerrr", "lang": "en", "location": "null", "create_at": date("2013-12-30"), "description": "null", "followers_count": 309, "friends_count": 108, "statues_count": 3528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MN", "id": "3eb9b911bea717c5", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-93.556704,44.978417 -93.400552,45.0729") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2751730, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154834706530306, "text": "Feels like yesterday was a sunday and todays a monday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1554769358, "name": "FrankDawg", "screen_name": "FrankieSimiano", "lang": "en", "location": "Tokyo, Japan", "create_at": date("2013-06-28"), "description": "aye suh dude • Nissan240sx s14 • Kouki • It started in Japan", "followers_count": 559, "friends_count": 491, "statues_count": 6033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154835574898688, "text": "@lilpup @birthctrl oh really now ��", "in_reply_to_status": 695154750770319361, "in_reply_to_user": 106168102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106168102, 387989837 }}, "user": { "id": 2930088147, "name": "BABYGRL", "screen_name": "chongaspice", "lang": "en", "location": "hell", "create_at": date("2014-12-18"), "description": "Bitch I'm Posh", "followers_count": 548, "friends_count": 369, "statues_count": 4908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lauderhill, FL", "id": "0fbf6b3abebf121b", "name": "Lauderhill", "place_type": "city", "bounding_box": rectangle("-80.268234,26.135832 -80.192482,26.193897") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1239550, "cityName": "Lauderhill" } }
+{ "create_at": datetime("2016-02-04T00:00:02.000Z"), "id": 695154835834937345, "text": "I am going to sleep heavenly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103703245, "name": "meisha mesha meshia.", "screen_name": "MeeshaVeli", "lang": "en", "location": "Dublin, Georgia ", "create_at": date("2010-01-10"), "description": "#OMO till i go. aint always been right, but ive always been real.", "followers_count": 1402, "friends_count": 510, "statues_count": 113275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marietta, GA", "id": "e229de11a7eb6823", "name": "Marietta", "place_type": "city", "bounding_box": rectangle("-84.596805,33.895088 -84.46746,34.001159") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1349756, "cityName": "Marietta" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837017604097, "text": "When The Bae Loves The Same Musical As You!! #WickedTwinsies <3 XD https://t.co/1LP5lLl5Ch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WickedTwinsies" }}, "user": { "id": 310995379, "name": "Tara Fraser", "screen_name": "Simmy3Tara", "lang": "en", "location": "Natrona Heights, Pa", "create_at": date("2011-06-04"), "description": "I'm OBSESSED With Cupcakes And The Eiffel Tower!! My Favorite Youtuber Is @shanedawson!! Also, I'm A Disney Addict!!", "followers_count": 1442, "friends_count": 1869, "statues_count": 189638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837072265216, "text": "Celebrating #lupe bday chava_ponce oceanandsun85 @lacozinela_j51 n @panchito969 @ Santa Maria,… https://t.co/GXj2LgG75x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.433,34.9514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lupe" }}, "user_mentions": {{ 38520078, 29769557 }}, "user": { "id": 29769557, "name": "PANCHITO", "screen_name": "panchito969", "lang": "en", "location": "Bakersfield, Ca", "create_at": date("2009-04-08"), "description": "null", "followers_count": 144, "friends_count": 70, "statues_count": 2112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Maria, CA", "id": "dee4ad8775ff8102", "name": "Santa Maria", "place_type": "city", "bounding_box": rectangle("-120.482386,34.875868 -120.356782,34.988866") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669196, "cityName": "Santa Maria" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837105672192, "text": "DNA shows New Zealand's little penguins are actually Aussie invaders https://t.co/AOO2NQPpZl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17614, "friends_count": 17568, "statues_count": 67998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837235855360, "text": "GN❣ (@ Meridian Place in Northridge, CA) https://t.co/x63bu6eHdd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 372, "friends_count": 336, "statues_count": 25473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837273583616, "text": "#SupportOriginMelissa 34.9°F Wind:4.9mph Pressure: 30.12hpa Rising slowly Rain Today 0.00in. Forecast: Fairly fine, improving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 308927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154837978161152, "text": "@Dr_SofaKingSXE @_NaomiHatesYou Moulin Rouge fucks me up.", "in_reply_to_status": 695151460107972609, "in_reply_to_user": 430257842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430257842, 346907520 }}, "user": { "id": 278789170, "name": "mid mosh dog bark", "screen_name": "ihvtevlex", "lang": "en", "location": "Portland, OR", "create_at": date("2011-04-07"), "description": "Join me in death. PNW.", "followers_count": 1316, "friends_count": 602, "statues_count": 49360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154838091358209, "text": "I feel like anything that is for you face shouldn't hurt if you get it in your eyes. Aka face wash.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42728246, "name": "Tori Dwyer", "screen_name": "T0RIDWYER", "lang": "en", "location": "Texas ", "create_at": date("2009-05-26"), "description": "coffee, champagne, chaos & chardonnay", "followers_count": 404, "friends_count": 378, "statues_count": 9959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amarillo, TX", "id": "52a0517c2183cf22", "name": "Amarillo", "place_type": "city", "bounding_box": rectangle("-101.96758,35.098693 -101.664604,35.323818") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48375, "countyName": "Potter", "cityID": 4803000, "cityName": "Amarillo" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154838158462976, "text": "Temp: 23.6°F - Dew Point: 15.6° - Wind: 15.3 mph - Gust: 22.8 - Rain Today: 0.00in. - Pressure: 30.08in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 12970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154838171054080, "text": "I did something new tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3034285734, "name": "D", "screen_name": "D_TheRapper", "lang": "en", "location": "S E A T T L E ", "create_at": date("2015-02-21"), "description": "I WANT IT ALL.", "followers_count": 867, "friends_count": 215, "statues_count": 1577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154838397521920, "text": "@emtrip12 @MrLopez_13 @Ralfy_182 cause it's true haha", "in_reply_to_status": 695154463657451520, "in_reply_to_user": 2541248347, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2541248347, 2479543248, 264589885 }}, "user": { "id": 291463047, "name": "Zack Carvajal ⚾", "screen_name": "mrbear_85", "lang": "en", "location": "yo mammas bed ", "create_at": date("2011-05-01"), "description": "Dead lost in a dream, yes this is the last of me #thinkblue #bearcrew RIP Lissbeth Carvajal my beautiful wife", "followers_count": 137, "friends_count": 376, "statues_count": 6428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-02-04T00:00:03.000Z"), "id": 695154838951350272, "text": "Wind 0.7 mph W. Barometer 29.88 in, Rising. Temperature 48.6 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154840679358464, "text": "I know you well enough to know you never loved me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 953096287, "name": "Chicken ParmeJohn", "screen_name": "johnkaroutsos", "lang": "en", "location": "#WickerGangBitch2k14", "create_at": date("2012-11-16"), "description": "Not from Chicago", "followers_count": 197, "friends_count": 198, "statues_count": 5200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, IN", "id": "99700cd6fc455c13", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-87.525341,41.566265 -87.432288,41.710116") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1831000, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154840683573248, "text": "Rips by myself since Aidan fell asleep on me! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352905788, "name": "mitti", "screen_name": "MittinaHale", "lang": "en", "location": "mayetta, kansas", "create_at": date("2011-08-11"), "description": "null", "followers_count": 370, "friends_count": 394, "statues_count": 7055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Topeka, KS", "id": "835f1b2948575c51", "name": "Topeka", "place_type": "city", "bounding_box": rectangle("-95.809606,38.971524 -95.571859,39.103634") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20177, "countyName": "Shawnee", "cityID": 2071000, "cityName": "Topeka" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154840725360640, "text": "������ https://t.co/mnjZXvH6GD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 625367127, "name": "Nick", "screen_name": "donahue_nick", "lang": "en", "location": "null", "create_at": date("2012-07-02"), "description": "null", "followers_count": 163, "friends_count": 353, "statues_count": 3547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154841094529024, "text": "I'm gonna have an epileptic seizure https://t.co/JF3xasS3v1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3114350604, "name": "steph", "screen_name": "lilnuggetsteph", "lang": "en", "location": "null", "create_at": date("2015-03-28"), "description": "null", "followers_count": 62, "friends_count": 51, "statues_count": 677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154841262415872, "text": "@plorf_ @bawwsadface both are boring", "in_reply_to_status": 695154643685482496, "in_reply_to_user": 23649954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23649954, 400948202 }}, "user": { "id": 2583550982, "name": "turtle turtle", "screen_name": "yourfriendcimi", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "real turtle shit", "followers_count": 324, "friends_count": 176, "statues_count": 21516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154841270689792, "text": "Girls don't even give me a straight up curve anymore, they just waste my time ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3318440184, "name": "devin", "screen_name": "devfromthebay", "lang": "en", "location": "bay area, ca | #saddboiz", "create_at": date("2015-08-18"), "description": "18. invincible. ultimate. marvelous.", "followers_count": 499, "friends_count": 229, "statues_count": 15201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154841321095169, "text": "@_theylovelucy_ it never has the desired effect though so it doesn't count", "in_reply_to_status": 695154686744215552, "in_reply_to_user": 973214466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 973214466 }}, "user": { "id": 384592142, "name": "out the system", "screen_name": "WillSpeakin", "lang": "en", "location": "...", "create_at": date("2011-10-03"), "description": "#Aquarius 21 Young and old #wonderteam #nbdmg IG: Willspeakin", "followers_count": 447, "friends_count": 359, "statues_count": 22879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Oaks, GA", "id": "a80d3803a636a10f", "name": "Fair Oaks", "place_type": "city", "bounding_box": rectangle("-84.558691,33.894492 -84.496049,33.938052") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1328520, "cityName": "Fair Oaks" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154841908330497, "text": "20 more days.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1146279901, "name": "tonija", "screen_name": "gyalodown", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "snapchat: tonija24 #imsonewhaven", "followers_count": 1267, "friends_count": 892, "statues_count": 35701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Rochelle, NY", "id": "11cdb607ab2f17bb", "name": "New Rochelle", "place_type": "city", "bounding_box": rectangle("-73.807197,40.881294 -73.755414,40.988329") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3650617, "cityName": "New Rochelle" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154843845943296, "text": "Yaşadıklarınıza gereğinden fazla anlam yüklemeyin. Seneler çok çabuk geçiyor. Zaman geldiğinde de ne olacağını hepimiz gayet iyi biliyoruz.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 186832407, "name": "Alen", "screen_name": "heartlesslion", "lang": "en", "location": "Los Angeles DODGERSSSSS!!!", "create_at": date("2010-09-04"), "description": "Hayati GALATASARAY olmayanlar beni anlamasin.Kusla kelebekle 2-3 yildizla isim olmaz. Evli,1 erkek 1 kiz falan oyle yani.@GalatasaraySK #SBF @Dodgers #Dodgers", "followers_count": 11947, "friends_count": 9714, "statues_count": 22598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:04.000Z"), "id": 695154844483497984, "text": "3.4 magnitude #earthquake. 7 km from #Clancy, MT, United States https://t.co/4xbVNu789O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.073,46.495"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "Clancy" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 63316, "friends_count": 10, "statues_count": 95242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montana, USA", "id": "d2ddff69682ae534", "name": "Montana", "place_type": "admin", "bounding_box": rectangle("-116.050004,44.35821 -104.039563,49.00139") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30043, "countyName": "Jefferson", "cityID": 3015100, "cityName": "Clancy" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154844982579200, "text": "Social science or people's behavior is always more complex than rocket or techology science to make change #HIMSS16 https://t.co/PKZ3wsxO3r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HIMSS16" }}, "user": { "id": 272141779, "name": "Mike Ryan", "screen_name": "MikeGordonRyan", "lang": "en", "location": "San Francisco", "create_at": date("2011-03-25"), "description": "Husband, Dad of 3, @peerwell, Frm Hosp CEO, Sudden Cardiac Arrest Survivor, @WhiteHouse Intern, Board ❤️American Heart Assoc ❤ Mentor @draper_u #dyslexic", "followers_count": 298033, "friends_count": 54866, "statues_count": 32901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154845049749504, "text": "JEFFREY THE GIRAFFE HAS BEEN KIDNAPPED", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1368163230, "name": "Pudding Cup", "screen_name": "natashamarieh11", "lang": "en", "location": "Earth", "create_at": date("2013-04-20"), "description": "don't waste my time . I am your OG and will be respected as such. what u got on me ? Damn don't you look foolish .", "followers_count": 533, "friends_count": 397, "statues_count": 46704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winfield, KS", "id": "0ea8380870f62170", "name": "Winfield", "place_type": "city", "bounding_box": rectangle("-97.029115,37.201311 -96.949686,37.259226") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20035, "countyName": "Cowley", "cityID": 2079950, "cityName": "Winfield" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154845687255040, "text": "We throw it all the way back to a certain Sunday morning in #2014. We must have been digging… https://t.co/vN36dmhgwT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.059902,42.045805"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 83820877, "name": "Akinlolu Olumoroti™", "screen_name": "morotioluwa", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-10-20"), "description": "Author ~ Physicist ~ Alternative Energy Enthusiast ~ Photographer ~ Worship Leader ~ Conversationalist ~", "followers_count": 197, "friends_count": 183, "statues_count": 753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schaumburg, IL", "id": "918c943e88cc3e5d", "name": "Schaumburg", "place_type": "city", "bounding_box": rectangle("-88.154356,41.986627 -88.028003,42.078062") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1768003, "cityName": "Schaumburg" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154846085697536, "text": "@greggrosenthal @anthonyjeselnik you guys #rjvp #episode15 #sitback&enjoytheride", "in_reply_to_status": -1, "in_reply_to_user": 24903274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rjvp", "episode15", "sitback" }}, "user_mentions": {{ 24903274, 19494203 }}, "user": { "id": 31323289, "name": "Tim Riggins", "screen_name": "peeaatrick", "lang": "en", "location": "denver", "create_at": date("2009-04-14"), "description": "Product poster child of Millennial America.", "followers_count": 595, "friends_count": 374, "statues_count": 56104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154847142645760, "text": "Another sleepless night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551383170, "name": "John (-: 5'5", "screen_name": "L_Jaayyyyy", "lang": "en", "location": "Weslaco, TX. ", "create_at": date("2012-04-11"), "description": "Please", "followers_count": 354, "friends_count": 269, "statues_count": 25690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weslaco, TX", "id": "f664c6f63c0bef35", "name": "Weslaco", "place_type": "city", "bounding_box": rectangle("-98.057773,26.111766 -97.942697,26.242157") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4877272, "cityName": "Weslaco" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154847230795776, "text": "The key to my heart https://t.co/i6sJ9oUeRm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44029692, "name": "Velveth Alarcon", "screen_name": "velvethh", "lang": "en", "location": "Long Beach", "create_at": date("2009-06-01"), "description": "Interior and Architect Design• Graphic Design • CSULB • President of the Salsa Team", "followers_count": 1016, "friends_count": 892, "statues_count": 20246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154847298015233, "text": "Barberton Oh Temp:35.5°F Wind:3 mph Dir:W Baro:Rising Rain2day:0.00in Hum:74% UV:0.0 @ 03:00 02/04/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 232, "friends_count": 228, "statues_count": 114394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820272,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154847415336961, "text": "@GundamRX101 Go to sleep, dweeb.", "in_reply_to_status": -1, "in_reply_to_user": 152545223, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 152545223 }}, "user": { "id": 566808903, "name": "Poe Dameron's Wife", "screen_name": "gamgeezy", "lang": "en", "location": "Not Rotterdam", "create_at": date("2012-04-29"), "description": "My friends call me Samwise. My enemies also call me Samwise. That is how names work, you see... flieswithwookies@gmail.com", "followers_count": 778, "friends_count": 1527, "statues_count": 78584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arvada, CO", "id": "c02e66a5016d732f", "name": "Arvada", "place_type": "city", "bounding_box": rectangle("-105.214417,39.783802 -105.04196,39.856611") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 803455, "cityName": "Arvada" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154847637581824, "text": "It's not bout what you say, learned that last year.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1210786171, "name": "la'hova", "screen_name": "_TheGreatPlay_", "lang": "en", "location": "New Orleans // Westbank", "create_at": date("2013-02-22"), "description": "#whatisyoudoing?", "followers_count": 293, "friends_count": 295, "statues_count": 3562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154848568741888, "text": "@iamgoddaddy used. Nah", "in_reply_to_status": 695151757786152960, "in_reply_to_user": 3242897334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3242897334 }}, "user": { "id": 3242897334, "name": "Iamgoddaddy", "screen_name": "iamgoddaddy", "lang": "en", "location": "null", "create_at": date("2015-06-11"), "description": "null", "followers_count": 40, "friends_count": 176, "statues_count": 880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154848598110209, "text": "@QueenXangelica U WELCOME", "in_reply_to_status": 695154641781141504, "in_reply_to_user": 4109952311, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4109952311 }}, "user": { "id": 4435546814, "name": "TwonOutspoken", "screen_name": "662TWON", "lang": "en", "location": "Texas, USA", "create_at": date("2015-12-02"), "description": "Twitter fucked me out my Twonoutspoken account smh. #PurpleWax", "followers_count": 594, "friends_count": 1116, "statues_count": 936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Killeen, TX", "id": "a27a0542e774e138", "name": "Killeen", "place_type": "city", "bounding_box": rectangle("-97.848463,31.014356 -97.659217,31.144257") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4839148, "cityName": "Killeen" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154848740675584, "text": "#Zoolander2 gonna be kool..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Zoolander2" }}, "user": { "id": 3882077653, "name": "THEGREENEYEDGERMAN", "screen_name": "hustlernature", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-10-13"), "description": "THE DEFINITION OF \n (BAD LUCK)\n #TGOD #WUTANGFOREVER \n\n\n\n\n#SREMMLIFE", "followers_count": 588, "friends_count": 932, "statues_count": 5875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154848761802752, "text": "confused and can't sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 215546724, "name": "Danielle Trivison", "screen_name": "danitriv", "lang": "en", "location": "BGSU '19", "create_at": date("2010-11-13"), "description": "• American Elite Celebrity 12-15 •", "followers_count": 1676, "friends_count": 1010, "statues_count": 13347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-02-04T00:00:05.000Z"), "id": 695154848900243458, "text": "@Navi2884 haha.", "in_reply_to_status": 695154253409689602, "in_reply_to_user": 31063768, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 31063768 }}, "user": { "id": 50362607, "name": "Tyler J", "screen_name": "Slic_Ric", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-06-24"), "description": "well....what did you expect?", "followers_count": 295, "friends_count": 1476, "statues_count": 32490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, IL", "id": "48f2609344f0c3f7", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-87.799908,41.733586 -87.740829,41.757299") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1709642, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154849650909184, "text": "Need friends https://t.co/VKRR0NiNPY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2825464376, "name": "Spidey", "screen_name": "vladtheman_", "lang": "en", "location": "Walnut, CA", "create_at": date("2014-09-21"), "description": "F.O.E", "followers_count": 199, "friends_count": 267, "statues_count": 1843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut, CA", "id": "2fd93ef04260afec", "name": "Walnut", "place_type": "city", "bounding_box": rectangle("-117.895753,34.00286 -117.825003,34.063424") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683332, "cityName": "Walnut" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154850397450240, "text": "@teykar I'm trippin out apeh why'd you send that", "in_reply_to_status": 695154757174841344, "in_reply_to_user": 2544594524, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2544594524 }}, "user": { "id": 3421334969, "name": "Steve", "screen_name": "steveee_818", "lang": "en", "location": "null", "create_at": date("2015-08-13"), "description": "null", "followers_count": 194, "friends_count": 165, "statues_count": 1092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154850498150401, "text": "遺跡の入口は私もスキャンしていますが、まだ見つかりません", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.3173,26.2008"), "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 1148156568, "name": "アルフィン", "screen_name": "AL_FiN_07839216", "lang": "ja", "location": "飛空巡航艦FS-317", "create_at": date("2013-02-04"), "description": "こちらは『シャイニング・フォース フェザー』に登場するコアユニット、アルフィンbotです。\n反応ワードなどはURLから。非公式全自動botの不具合・反応の追加などDMでお知らせ頂ければ幸いです。Shining Force Feather SEGA®", "followers_count": 2181, "friends_count": 2280, "statues_count": 36226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154851295141888, "text": "@emilyconnors11 concerned", "in_reply_to_status": 695153649048248320, "in_reply_to_user": 386025480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 386025480 }}, "user": { "id": 2169923573, "name": "Meghan Kennedy", "screen_name": "meghannveronica", "lang": "en", "location": "Lm // T-town", "create_at": date("2013-11-05"), "description": "University of Alabama | ΑΦ", "followers_count": 868, "friends_count": 395, "statues_count": 13077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154851357929472, "text": "We really got kicked out all the way in Santa Ana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108985809, "name": "Steven Shneir", "screen_name": "Stevenshneir", "lang": "en", "location": "null", "create_at": date("2010-01-27"), "description": "null", "followers_count": 613, "friends_count": 519, "statues_count": 19158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154852830146560, "text": "How much is a casket cause ya boi is dying https://t.co/CdAIt9TCEl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571987632, "name": "trämp", "screen_name": "puppiewhiskers", "lang": "en", "location": "null", "create_at": date("2012-05-05"), "description": "•⚔✨Defender of The Faith✨⚔• Jezabel❤️", "followers_count": 295, "friends_count": 317, "statues_count": 7176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-04T00:00:06.000Z"), "id": 695154853090201600, "text": "My old manager is trying to talk to me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1407639806, "name": "Keyazmine Gates♋", "screen_name": "TrapGirll_", "lang": "en", "location": "htx", "create_at": date("2013-05-06"), "description": "Maniac. 18 & poppin'.", "followers_count": 1577, "friends_count": 785, "statues_count": 67376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crockett, TX", "id": "4bbe438f647e9ba5", "name": "Crockett", "place_type": "city", "bounding_box": rectangle("-95.481396,31.290167 -95.429031,31.341444") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48225, "countyName": "Houston", "cityID": 4817744, "cityName": "Crockett" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154853182513158, "text": "@_LameSteph_ looks like your tattoo lol https://t.co/U5GoJOuf7r", "in_reply_to_status": -1, "in_reply_to_user": 336075346, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 336075346 }}, "user": { "id": 1188180738, "name": "Boba", "screen_name": "lis_1596", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "Get to know me", "followers_count": 389, "friends_count": 291, "statues_count": 13668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154853484494849, "text": "2/4/2016 - 02:00\nTemp: 28.3F \nHum: 88%\nWind: 0.0 mph\nBaro: 30.332in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 49647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154853622915072, "text": "Wind 3.0 mph SSE. Barometer 30.242 in, Rising slowly. Temperature 14.8 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 11, "statues_count": 888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154853702733824, "text": "He Bae like shit for that ���� https://t.co/3ALFgxUMOz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325930365, "name": "catfish.✊", "screen_name": "MissAB__", "lang": "en", "location": "probably your dads house.", "create_at": date("2011-06-28"), "description": "I am a positive fucking person. Im a asshole but I care.. of some sort. Most Relaxed, Goofiest, Craziest Girl Ever. 5'10 head ass..#GxldenGods #NCAT #Bennett", "followers_count": 4218, "friends_count": 3247, "statues_count": 46228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154853920681985, "text": "@C00LKITTY_ you the type a nigga gotta hide the knifes in the kitchen from", "in_reply_to_status": 695154729085571072, "in_reply_to_user": 2823303002, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2823303002 }}, "user": { "id": 2702412786, "name": "B Dub", "screen_name": "LookAtB_Now", "lang": "en", "location": "null", "create_at": date("2014-08-02"), "description": "STOP WHITE PEOPLE 2K16", "followers_count": 1218, "friends_count": 997, "statues_count": 27374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154854457532416, "text": "Wind 2.0 mph SE. Barometer 30.504 in, Steady. Temperature 39.2 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 248468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154854512062464, "text": "It be sooo cold in my room ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209975778, "name": "bomb", "screen_name": "breeeuuhh", "lang": "en", "location": "United States", "create_at": date("2010-10-30"), "description": "Instagram ; briasimonee", "followers_count": 782, "friends_count": 563, "statues_count": 30811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154855929839618, "text": "Perkys callin��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fi", "is_retweet": false, "user": { "id": 717368245, "name": "Rip Mase☀️", "screen_name": "_KingTez", "lang": "en", "location": "Dungy Field ", "create_at": date("2012-07-25"), "description": "#Hungry WR/CB #TrackLite", "followers_count": 1236, "friends_count": 382, "statues_count": 43652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154856588214272, "text": "Wind 3.1 mph SSW. Barometer 29.98 in, Steady. Temperature 4.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154856760209408, "text": "@dmac1043 my immediate cousins: Darren, Brady, annalivia, Jennifer and Tommy. Hahaha, unless you want me to name the kids and spouses too?", "in_reply_to_status": 695154112011198464, "in_reply_to_user": 57483040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57483040 }}, "user": { "id": 37230008, "name": "Kristen Marzelli", "screen_name": "kristenmarzelli", "lang": "en", "location": "Boston ➡️ San Diego", "create_at": date("2009-05-02"), "description": "Today I will be better than yesterday.", "followers_count": 278, "friends_count": 461, "statues_count": 6918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlsbad, CA", "id": "01d4e349481265e8", "name": "Carlsbad", "place_type": "city", "bounding_box": rectangle("-117.359298,33.060615 -117.216549,33.182353") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 611194, "cityName": "Carlsbad" } }
+{ "create_at": datetime("2016-02-04T00:00:07.000Z"), "id": 695154857037041664, "text": "I wish I wasn't over sensitive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932987348, "name": "Abby ♕", "screen_name": "AbbyJohnston_17", "lang": "en", "location": "Laramie, WY", "create_at": date("2013-10-03"), "description": "I have a deep appreciation for keurigs and strong jawlines// sc: abbyjohn16", "followers_count": 598, "friends_count": 600, "statues_count": 2837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laramie, WY", "id": "b4fa2987bd7d8a3c", "name": "Laramie", "place_type": "city", "bounding_box": rectangle("-105.697156,41.248625 -105.514486,41.367701") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56001, "countyName": "Albany", "cityID": 5645050, "cityName": "Laramie" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517211721654272, "text": "@YalAintShiit naw W it comes before X , ion want it to get that wild ��������������", "in_reply_to_status": 695517068502765569, "in_reply_to_user": 543239348, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 543239348 }}, "user": { "id": 96875565, "name": "Sexual Hippie ✌", "screen_name": "IAmPrettyBased", "lang": "en", "location": "MARCH 9 = 21", "create_at": date("2009-12-14"), "description": "I'm the heart and the beat. Just keep it 10 more than 90 & we gooch.", "followers_count": 2699, "friends_count": 2142, "statues_count": 57338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517211977502721, "text": "Are fear tears us up but when we lets go of fear we bond together and let fear write its own path not ours.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553717172, "name": "cameron woodcock", "screen_name": "camofron1230", "lang": "en", "location": "null", "create_at": date("2012-04-14"), "description": "arrive shitty or dont arrive", "followers_count": 267, "friends_count": 849, "statues_count": 5033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakville, MO", "id": "46d9a4d89c4e03e8", "name": "Oakville", "place_type": "city", "bounding_box": rectangle("-90.37658,38.411361 -90.281839,38.492038") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2953876, "cityName": "Oakville" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517212434522112, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 806002122, "name": "a m r o n", "screen_name": "amron9o", "lang": "en", "location": "VHS Senior", "create_at": date("2012-09-05"), "description": "if you're not enjoying life... that's your fault", "followers_count": 453, "friends_count": 324, "statues_count": 21960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517212442914816, "text": "I would be so happy. Lord knows I need a blessing https://t.co/bdc3sVWDaA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2832264288, "name": "Raphael", "screen_name": "MillerThaGod", "lang": "en", "location": "THIS AINT TWITTER, NIGGA", "create_at": date("2014-09-25"), "description": "Yall can't be this dense. ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀⠀⠀⠀ ⠀⠀⠀ Get them followers, boo. ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ ⠀⠀⠀ Don't save her, she dont wanna be saved.⠀⠀⠀⠀ Myra", "followers_count": 11562, "friends_count": 881, "statues_count": 115827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517212799430656, "text": "See our latest #Algood, TN #job and click to apply: General Manager - https://t.co/9lB45NfzoT #Hospitality #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.457811,36.184981"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Algood", "job", "Hospitality", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 211, "friends_count": 39, "statues_count": 19197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Algood, TN", "id": "0c92e8ca695b40ad", "name": "Algood", "place_type": "city", "bounding_box": rectangle("-85.462241,36.168167 -85.422695,36.225079") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47141, "countyName": "Putnam", "cityID": 4700640, "cityName": "Algood" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517213000794112, "text": "Interested in a #Retail #job near #PointPleasant, WV? This could be a great fit: https://t.co/lpfCAdsWOK #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.1370889,38.8445251"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "PointPleasant", "Hiring" }}, "user": { "id": 3144822634, "name": "Speedway Jobs", "screen_name": "SpeedwayJobs", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "Rethink Speedway. It's more than a convenience store... It's your career opportunity!", "followers_count": 172, "friends_count": 27, "statues_count": 6413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Point Pleasant, WV", "id": "62093956700bed6e", "name": "Point Pleasant", "place_type": "city", "bounding_box": rectangle("-82.141247,38.823843 -82.105925,38.882844") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54053, "countyName": "Mason", "cityID": 5464708, "cityName": "Point Pleasant" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517214129188865, "text": "They talk shit but can't come close to most of us", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 701847698, "name": "Cheech N' Longe", "screen_name": "Deal_Or_Nah_", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "I only follow if you swallow", "followers_count": 359, "friends_count": 303, "statues_count": 29050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midland, MI", "id": "790e829b009ecfde", "name": "Midland", "place_type": "city", "bounding_box": rectangle("-84.320149,43.574381 -84.154385,43.697326") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26111, "countyName": "Midland", "cityID": 2653780, "cityName": "Midland" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517214561193985, "text": "I don't forget nothing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 398400889, "name": "Alex Hefner ❤", "screen_name": "QueenAlxandraa", "lang": "en", "location": "Shreveport, LA", "create_at": date("2011-10-25"), "description": "yes , I'm the Alex with Cancer ❤️", "followers_count": 1649, "friends_count": 1694, "statues_count": 87971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517215047622656, "text": "@JonathanGalland Welcome to our #veteran owned biz and our #mentalhealth pledge #rewards\nhttps://t.co/BxkNDrNEpb\nhttps://t.co/MDJGY4qGFO", "in_reply_to_status": -1, "in_reply_to_user": 78978173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "veteran", "mentalhealth", "rewards" }}, "user_mentions": {{ 78978173 }}, "user": { "id": 3285885294, "name": "Molecular Bliss", "screen_name": "MolecularBliss", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-07-20"), "description": "Molecular Bliss brings constant visual reminders of happiness, pleasure, motivation and calm in the form of fine jewelry.\nWe follow or unfollow as you do.", "followers_count": 2755, "friends_count": 3090, "statues_count": 11521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517215387361281, "text": "Andrew always falls asleep before ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373759531, "name": "Alexandria Rodriguez", "screen_name": "ali_desirey", "lang": "en", "location": "null", "create_at": date("2011-09-14"), "description": "xoxo", "followers_count": 358, "friends_count": 100, "statues_count": 23839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soledad, CA", "id": "642fbca6eea15240", "name": "Soledad", "place_type": "city", "bounding_box": rectangle("-121.338707,36.410518 -121.306524,36.445593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 672520, "cityName": "Soledad" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517215429361664, "text": "I really been on the move every weekend.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2806405639, "name": "mia.", "screen_name": "_theQM", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-09-12"), "description": "• I'm just living ..", "followers_count": 1138, "friends_count": 854, "statues_count": 16689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar City, UT", "id": "013884df639610f8", "name": "Cedar City", "place_type": "city", "bounding_box": rectangle("-113.152491,37.647433 -113.032184,37.736012") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49021, "countyName": "Iron", "cityID": 4911320, "cityName": "Cedar City" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517215513190400, "text": "Waking up 3-4x a night with Xya. Getting up at 7:30am with Nayeli, work at 3pm off at 10:30pm. I'm so overly exhausted.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632619977, "name": "alissa", "screen_name": "alissa_maciel", "lang": "en", "location": "null", "create_at": date("2012-07-10"), "description": "Today you are you, that is truer than true. There is know one alive that is youer than you.", "followers_count": 104, "friends_count": 107, "statues_count": 5033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223198,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-02-05T00:00:00.000Z"), "id": 695517215773200386, "text": "@Ckarsberg17 https://t.co/lurcq0vQG7", "in_reply_to_status": -1, "in_reply_to_user": 430839869, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 430839869 }}, "user": { "id": 2203571484, "name": "Stephaniee❥", "screen_name": "stephaniexoxo__", "lang": "en", "location": "null", "create_at": date("2013-11-19"), "description": "CJ 01.15.14 ♡♡ || 20", "followers_count": 159, "friends_count": 155, "statues_count": 6296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverbank, CA", "id": "89e82c687111f62b", "name": "Riverbank", "place_type": "city", "bounding_box": rectangle("-120.981748,37.707792 -120.882789,37.768288") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 661068, "cityName": "Riverbank" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517215873863680, "text": "Draking TBH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 948301483, "name": "vTristan☁️", "screen_name": "TristaniBeast12", "lang": "en", "location": "Austin, TX", "create_at": date("2012-11-14"), "description": "| 20 | Live for the nights I can't remember with the people I wont forget |", "followers_count": 832, "friends_count": 655, "statues_count": 17441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517215903215616, "text": "Happy birthday to one of the biggest influences in my life. I am so blessed to have someone like you. This is forever @kyrsti_1999 ��❤️ily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 706285843 }}, "user": { "id": 2350459987, "name": "payyytoon", "screen_name": "paytonpulicella", "lang": "en", "location": "null", "create_at": date("2014-02-18"), "description": "PNW//Sagittarius", "followers_count": 619, "friends_count": 392, "statues_count": 5561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517216821813248, "text": "Bae �� https://t.co/3GY89AIzLT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 192714506, "name": "Brittany Frances", "screen_name": "beeeffvee", "lang": "en", "location": "null", "create_at": date("2010-09-19"), "description": "It starts with a smile and ends with an all night long slow kiss KΔ", "followers_count": 344, "friends_count": 584, "statues_count": 7702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217111351298, "text": "Temp: 30.0°F Wind:0.0mph Pressure: 30.447hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 59048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217232805889, "text": "Sooooo wassup w these refunds, are niggas getting paid tomorrow or nah cause I need to know what the club is gonna look like.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.45912437,29.68826624"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131331202, "name": "ivy.", "screen_name": "BloodyKnuckls", "lang": "en", "location": "screwston", "create_at": date("2010-04-09"), "description": "made from 100% pimp juice concentrate. My time is expensive...if you dm me, im requesting money/gifts", "followers_count": 4696, "friends_count": 2210, "statues_count": 148892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217459298305, "text": "I don't understand how girls can be so fucking dirty!! Like why do you throw your toilet paper on the floor for real???? #Disgusted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Disgusted" }}, "user": { "id": 97325412, "name": "J U L I A", "screen_name": "Guulliiaaa", "lang": "en", "location": "San Antonio, TX", "create_at": date("2009-12-16"), "description": "apple juice & bitches", "followers_count": 310, "friends_count": 749, "statues_count": 18570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217488670720, "text": "@phil_goldsberry jk i'll be there w subie cookies", "in_reply_to_status": 695516844677988352, "in_reply_to_user": 3938498960, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3938498960 }}, "user": { "id": 16034871, "name": "Ashley Tyler", "screen_name": "brz_ashley", "lang": "en", "location": "23,AZ. IG: brz_ashley ", "create_at": date("2008-08-28"), "description": "Subaru life. BRZ Series Blue", "followers_count": 613, "friends_count": 682, "statues_count": 6481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.634443,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217635438592, "text": "Somebody hit me up , dm me something ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419465211, "name": "Ray Bae", "screen_name": "Rainfiji", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "care free black girl *", "followers_count": 1105, "friends_count": 712, "statues_count": 59108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryville, MO", "id": "d2e5e6b4697f41a7", "name": "Maryville", "place_type": "city", "bounding_box": rectangle("-94.894994,40.318088 -94.831714,40.372994") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29147, "countyName": "Nodaway", "cityID": 2946640, "cityName": "Maryville" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517217765658624, "text": "Wind 5.6 mph WNW. Barometer 30.40 in, Steady. Temperature 35.4 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517218101141504, "text": "02/05@03:00 - Temp 35.4F, WC 32.7F. Wind 3.3mph NNW, Gust 7.0mph. Bar 30.094in, Falling slowly. Rain 0.03in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788758,38.451038 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517218352844801, "text": "Yesterday's reality may not be todays reality.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355318666, "name": "Dillon Smith", "screen_name": "AyItsSmitty", "lang": "en", "location": "Lunchlady land/Fond du Lac, WI", "create_at": date("2011-08-14"), "description": "20. I love music and sports. Fan of Wisconsin sports, Boston Bruins and Red Sox! Huge #Shinedown fan!", "followers_count": 559, "friends_count": 213, "statues_count": 28920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fond du Lac, WI", "id": "47341c54fe1a873b", "name": "Fond du Lac", "place_type": "city", "bounding_box": rectangle("-88.523732,43.718278 -88.371043,43.806827") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55039, "countyName": "Fond du Lac", "cityID": 5526275, "cityName": "Fond du Lac" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517218629550081, "text": "@__adriannaag__ y'all wouldn't come through tho ����", "in_reply_to_status": 695514134209654785, "in_reply_to_user": 1238202338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1238202338 }}, "user": { "id": 2431549992, "name": "Melissa Bucio™", "screen_name": "melissabucioo", "lang": "en", "location": "null", "create_at": date("2014-04-06"), "description": "Captain America", "followers_count": 1230, "friends_count": 1281, "statues_count": 12864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517218776420352, "text": "Probably gonna call off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396356345, "name": "seven.", "screen_name": "_BkBaby", "lang": "en", "location": "Islands ", "create_at": date("2011-10-22"), "description": "null", "followers_count": 873, "friends_count": 803, "statues_count": 29944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647549 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517219397234688, "text": "That's not a good attitude to have for some one that cares alot.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1143825295, "name": "-Da- lil savvy", "screen_name": "DavonteHewett19", "lang": "en", "location": "North Carolina, USA", "create_at": date("2013-02-02"), "description": "UNCP'19", "followers_count": 969, "friends_count": 1765, "statues_count": 6424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke, NC", "id": "1588871da68509bb", "name": "Pembroke", "place_type": "city", "bounding_box": rectangle("-79.216367,34.654347 -79.150568,34.715633") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37155, "countyName": "Robeson", "cityID": 3751080, "cityName": "Pembroke" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517219711627265, "text": "@ebbtideapp Tide in Charlestown, Massachusetts 02/05/2016\nHigh 8:19am 9.8\n Low 2:44pm 0.3\nHigh 8:56pm 8.8\n Low 2:55am 0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.05,42.375"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 22, "friends_count": 1, "statues_count": 5089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlestown Waterfront, Boston", "id": "11ecf6357bd2e7bf", "name": "Charlestown Waterfront", "place_type": "neighborhood", "bounding_box": rectangle("-71.077096,42.369807 -71.047074,42.390954") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-02-05T00:00:01.000Z"), "id": 695517219912978432, "text": "@CameronOCTW and @ErvinOCTW exploring today's location https://t.co/gHSzO84oL9", "in_reply_to_status": -1, "in_reply_to_user": 91807351, "favorite_count": 0, "coordinate": point("-115.729999,33.356171"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 91807351, 19848045 }}, "user": { "id": 15925382, "name": "Kolby Schnelli", "screen_name": "kolbyster", "lang": "en", "location": "Orange County, California ", "create_at": date("2008-08-20"), "description": "photographer // world traveler // creative", "followers_count": 2312, "friends_count": 481, "statues_count": 15758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salton Sea", "id": "07d9d0f08c083000", "name": "Salton Sea", "place_type": "poi", "bounding_box": rectangle("-115.7299991,33.3561709 -115.729999,33.356171") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 607372, "cityName": "Bombay Beach" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517220063956992, "text": "@Audi @CNET the #s8 is my dream car", "in_reply_to_status": 695442473032613890, "in_reply_to_user": 27650674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "s8" }}, "user_mentions": {{ 27650674, 30261067 }}, "user": { "id": 109054597, "name": "Edward A Parker", "screen_name": "MrEdwardAParker", "lang": "en", "location": "Austin, TX", "create_at": date("2010-01-27"), "description": "Libations Chemist @ Fixe Austin// Golf Enthusiast// Boston Boy// former Sin City Soldier// NE Patriots Lover// Gym Rat// Netflix Addict", "followers_count": 1421, "friends_count": 1677, "statues_count": 4030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517220353388544, "text": "Wind 4.0 mph WSW. Barometer 1034.10 mb, Steady. Temperature 29.9 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517220424712192, "text": "@isaacsandoval15 for what", "in_reply_to_status": 695517102396936192, "in_reply_to_user": 779493289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 779493289 }}, "user": { "id": 2719031496, "name": "E⚡️A", "screen_name": "arroyoerick101", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "Santa Ana California // snapchat: Erick-rayo // phhhoto: Erarayo #flex", "followers_count": 366, "friends_count": 380, "statues_count": 7245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517220919709696, "text": "Wind 0.0 mph SE. Barometer 30.359 in, Steady. Temperature 30.0 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517221376884740, "text": "@brendanciccone I still will keep a deep personal contact with you on here thoughs Bro, especially through direct private message on here������", "in_reply_to_status": -1, "in_reply_to_user": 904048464, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 904048464 }}, "user": { "id": 4335169816, "name": "Andrew Addison", "screen_name": "DrewskiGuy79", "lang": "en", "location": "Orlando, FL", "create_at": date("2015-11-30"), "description": "Name is:Andrew Sheldon Addison\nZodiac Signed: VIRGO\nSkype: UniqueDrewskiFullCircle980@outlook.com\nTwitter: DrewskiGuy79\nEmail: exoticguy690@aol.com", "followers_count": 85, "friends_count": 341, "statues_count": 309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517221926281216, "text": "@Micah_thoo @omgzitzange ok Fire boy", "in_reply_to_status": 695517152812503040, "in_reply_to_user": 545529652, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545529652, 1637235198 }}, "user": { "id": 1023573560, "name": "$", "screen_name": "sabrina_taher", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-12-19"), "description": "Vice President of the official Shaun Livingston fan club. #GSW #49ers #SFG @omgzitzange ❤️", "followers_count": 597, "friends_count": 436, "statues_count": 34272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517222089859076, "text": "Happy birthday to my brother!❤️ @NoahVergura https://t.co/gxlk89GQlP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301839681 }}, "user": { "id": 2369505588, "name": "cole johnson", "screen_name": "cjohnson258", "lang": "en", "location": "Corona, CA", "create_at": date("2014-03-02"), "description": "Colts gang or die #handlethatlame", "followers_count": 299, "friends_count": 375, "statues_count": 7062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517222173872128, "text": "Wind 0.0 mph ---. Barometer 30.431 in, Rising slowly. Temperature 19.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517222324703232, "text": "I am in such a good mood and NO ONE IS RESPONDING TO ME lolol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423989829, "name": "Tamara ❂", "screen_name": "_TeeWilly", "lang": "en", "location": "Northridge, CA", "create_at": date("2011-11-28"), "description": "perfectly incomplete | CSUN | @errkkkk", "followers_count": 735, "friends_count": 286, "statues_count": 33967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517223054548993, "text": "Clinton, Sanders Duel Over Who's Tougher On Wall Street https://t.co/vAWGHOXThm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17607, "friends_count": 17567, "statues_count": 68056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.280822,26.295711 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517223314714625, "text": "Late night thoughts: if dogs have four legs, then is the area they have that should be an armpit called a legpit?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 600636948, "name": "Ashley Scarlett", "screen_name": "ashleyyscarlett", "lang": "en", "location": "ranting or singing somewhere", "create_at": date("2012-06-05"), "description": "I think I'm really funny, but there's a good chance that you probably don't", "followers_count": 416, "friends_count": 397, "statues_count": 6921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cambridge, MA", "id": "8193d87541f11dfb", "name": "Cambridge", "place_type": "city", "bounding_box": rectangle("-71.160356,42.352429 -71.064398,42.403966") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2511000, "cityName": "Cambridge" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517223520063489, "text": "36.2F (Feels: 36.2F) - Humidity: 86% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1044.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 218, "friends_count": 18, "statues_count": 225476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-05T00:00:02.000Z"), "id": 695517223788679168, "text": "Travelling to Berkeley or just twittering about Berkeley? https://t.co/KXDbgLQMd9 #Berkeley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.273,37.8716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Berkeley" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1033, "friends_count": 312, "statues_count": 2573259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517224296013824, "text": "Peyton will throw for more because Cam is a dual threat but the Panthers will win https://t.co/8bc7oeSA6b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 867236522, "name": "JARRYD #OASF❄️", "screen_name": "Dezmology", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-10-07"), "description": "| 20 |Dez| Nice on the Sticks|Survival of the Fittest| Been to Hell and Back| Friends are Enemies is disguise| #OASF| SneakerGod| http://Twitch.tv/YaBoiDez", "followers_count": 725, "friends_count": 676, "statues_count": 24819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734561") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517224677670912, "text": "Temp: 27.2°F - Dew Point: 23.6° - Wind: 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.25in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517226393329665, "text": "Can you find Berkeley on the map? Just try it at https://t.co/KXDbgLQMd9 #Berkeley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.273,37.8716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Berkeley" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1033, "friends_count": 312, "statues_count": 2573261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517226481410048, "text": "Wind 2.2 mph NNW. Barometer 30.13 in, Steady. Temperature 33.4 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517227538259968, "text": "some car just drove by at 2am playing owl city very loudly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110261037, "name": "alex wennerberg", "screen_name": "alexwennerberg", "lang": "en", "location": "Kirksville, MO", "create_at": date("2010-01-31"), "description": "person", "followers_count": 1035, "friends_count": 838, "statues_count": 1814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirksville, MO", "id": "7e403d6a5b07bc42", "name": "Kirksville", "place_type": "city", "bounding_box": rectangle("-92.601741,40.15184 -92.553156,40.238028") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29001, "countyName": "Adair", "cityID": 2939026, "cityName": "Kirksville" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517227634835456, "text": "@jaewilliams both thugger and wiz dropped some good stuff. Haven't been a wiz fan in years but this is solid.", "in_reply_to_status": 695516818837008385, "in_reply_to_user": 60256839, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60256839 }}, "user": { "id": 24341293, "name": "Squirtlord69", "screen_name": "fukevin", "lang": "en", "location": "The Farplane", "create_at": date("2009-03-13"), "description": "My lifestyle determines my death style.", "followers_count": 1411, "friends_count": 348, "statues_count": 45868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-05T00:00:03.000Z"), "id": 695517228016410624, "text": "@bharveyyyy @nairobiceleste @huganomics good we was about to have a problem", "in_reply_to_status": 695516906975932416, "in_reply_to_user": 50925339, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50925339, 200748091, 2324182172 }}, "user": { "id": 2534182832, "name": "Marcos", "screen_name": "wvrcos", "lang": "en", "location": "CA", "create_at": date("2014-05-29"), "description": "I'm your 3am thoughts", "followers_count": 367, "friends_count": 259, "statues_count": 8506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517228599533568, "text": "AND I WAS RIGHT!! ONLY ONE OF THE KIDS IS HERS!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2868744798, "name": "majica", "screen_name": "Maggelias", "lang": "en", "location": "Long Beach", "create_at": date("2014-10-21"), "description": "is so hard to burn calories", "followers_count": 132, "friends_count": 195, "statues_count": 4343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517229027168256, "text": "Gaga fans about to be like: I didn't know there was a football game at the GaGa concert this year? YAAAAAS BROMOS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1632435006, "name": "Hazel Belle", "screen_name": "RingYourBelle", "lang": "en-gb", "location": "CA/AZ", "create_at": date("2013-07-30"), "description": "•18•||•GCU•||•SC: roygbiv_15•||•IG: pancackle•||•Pre-Med•||•ᵃʷᵒᵒᵒᵒᵒ•||•RuPaul's Drag Race Connoisseur•||", "followers_count": 814, "friends_count": 1058, "statues_count": 17649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517229085958144, "text": "@kj_fetishmodel I ❤️ and miss you!!", "in_reply_to_status": 695516940983361536, "in_reply_to_user": 2337874352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2337874352 }}, "user": { "id": 3237067039, "name": "Elsa Dream Jean", "screen_name": "ElsaJeanxxx", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "For bookings contact http://ATMLA.com instagram ElsaxJean email me at elsajeanxxx@yahoo.com for customs @xxxstarPR", "followers_count": 60022, "friends_count": 933, "statues_count": 3703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517229778079744, "text": "White People Win The Award For Descriptive Name Calling ���������������������������� https://t.co/keqpV7scNO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2871687574, "name": "king soup", "screen_name": "HailKingSoup", "lang": "en", "location": "910✈️704", "create_at": date("2014-11-10"), "description": "Im ugly but i aint that ugly...im ugly with potential to be less ugly..which means im somewhat attractive...oh yeah i have a girlfriend...", "followers_count": 24601, "friends_count": 19294, "statues_count": 15500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517229811523585, "text": "Goodnight ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 975281184, "name": "guadalupe", "screen_name": "lupexxeudave", "lang": "en", "location": "Denver, Colorado", "create_at": date("2012-11-27"), "description": "fuck bros be a hoe", "followers_count": 464, "friends_count": 311, "statues_count": 15759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517229836869633, "text": "@youngthug blessings ������������ https://t.co/VaGnp9bn2i", "in_reply_to_status": -1, "in_reply_to_user": 238763290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 238763290 }}, "user": { "id": 258065725, "name": "Kay", "screen_name": "Kaylanalexus", "lang": "en", "location": "Louisiana", "create_at": date("2011-02-26"), "description": "Live for today, pray for tomorrow #DesignedByPaulySteez", "followers_count": 2978, "friends_count": 2810, "statues_count": 136474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517230272897026, "text": "Forever wishing I was in Nigeria", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157518375, "name": "nigeria.", "screen_name": "uchebaby_", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-06-19"), "description": "Culture lover. God. Natural hair advocate. Nigerian Beauty. #IgboKwenu Public Relations for Illinois State Univ's African Student Assoc. #ISU18", "followers_count": 1110, "friends_count": 844, "statues_count": 46735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517230688145408, "text": "temperature down 55°F -> 50°F\nhumidity up 40% -> 46%\npressure 30.27in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.14427,34.14745"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 107881312, "name": "Pasadena Weather", "screen_name": "_PasadenaCA", "lang": "en", "location": "Pasadena, CA", "create_at": date("2010-01-23"), "description": "Weather updates, forecast, warnings and information for Pasadena, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 284, "friends_count": 4, "statues_count": 25958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517230713339904, "text": "Wind 1.6 mph SW. Barometer 30.20 in, Rising slowly. Temperature 10.9 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517230793158656, "text": "Can you find Key West on the map? Just try it at https://t.co/WDK11pUI4j #Key West", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7826,24.5557"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Key" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1033, "friends_count": 312, "statues_count": 2573262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Key West, FL", "id": "5a1f85290e8bd413", "name": "Key West", "place_type": "city", "bounding_box": rectangle("-81.812436,24.54394 -81.729079,24.594749") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12087, "countyName": "Monroe", "cityID": 1236550, "cityName": "Key West" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517230956679169, "text": "@ElisaJordana i'm a little drunk right now so anything I say is not going be nice", "in_reply_to_status": 695516714000326656, "in_reply_to_user": 339290784, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339290784 }}, "user": { "id": 1609613448, "name": "Fields Davon", "screen_name": "FieldsDavonj", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-07-20"), "description": "null", "followers_count": 670, "friends_count": 586, "statues_count": 6647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517231006896128, "text": "These evenings of migraines are extremely frustrating.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3245404546, "name": "Lona Ritchie", "screen_name": "LonaRitchie", "lang": "en", "location": "Chippewa Falls, WI", "create_at": date("2015-05-10"), "description": "In my forties, honest to a fault, blunt to a fault. loving and kind. searching for new friends and old friends a like. I'm an aspiring writer. ask anything!", "followers_count": 49, "friends_count": 173, "statues_count": 57 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chippewa Falls, WI", "id": "01bf7f533ea68a3c", "name": "Chippewa Falls", "place_type": "city", "bounding_box": rectangle("-91.448375,44.90764 -91.330744,44.999074") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55017, "countyName": "Chippewa", "cityID": 5514575, "cityName": "Chippewa Falls" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517231774498816, "text": "Damn my NIGGA SHUTUP I'm TRYNA shleep ����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1584805532, "name": "dom", "screen_name": "dominiquemead33", "lang": "en", "location": "nomad", "create_at": date("2013-07-10"), "description": "Julia is coolia.....", "followers_count": 808, "friends_count": 346, "statues_count": 43595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521265 -117.988004,34.646028") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-02-05T00:00:04.000Z"), "id": 695517232189857793, "text": "Lmao im done\n@seankelley313 \n@KingWordplay https://t.co/eRbYorxfEa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2229692665, 607606401 }}, "user": { "id": 327210920, "name": "Justin", "screen_name": "J____SMOOVE", "lang": "en", "location": "Lincoln Park, MI", "create_at": date("2011-06-30"), "description": "I am what I am", "followers_count": 2830, "friends_count": 1305, "statues_count": 77232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln Park, MI", "id": "1769b19bf68bc543", "name": "Lincoln Park", "place_type": "city", "bounding_box": rectangle("-83.202957,42.220851 -83.155381,42.271565") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2647800, "cityName": "Lincoln Park" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517232642727937, "text": "I say this all the time guys, me and this girl are meant to be friends. https://t.co/oQ5a27Q2t9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 508478696, "name": "jaeeelii", "screen_name": "Jaeeelii__", "lang": "en", "location": "Probably in a Sephora", "create_at": date("2012-02-28"), "description": "Gabriel Paco Lopez ❤10.05.97-08.22.14❤️", "followers_count": 801, "friends_count": 236, "statues_count": 57494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517233087254529, "text": "My lil cousin is coming to me for relationship advice ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1592451236, "name": "DeeDee", "screen_name": "fuckitup_dee", "lang": "en", "location": "null", "create_at": date("2013-07-13"), "description": "Boss up or get bossed around \n\n#KBGB", "followers_count": 962, "friends_count": 602, "statues_count": 44864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517233133391872, "text": "I want someone to sit down and take the time to figure me out. I promise they won't get bored, but I mean that in a good way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145480933, "name": "cela.", "screen_name": "ceceleah_", "lang": "en", "location": "null", "create_at": date("2015-04-08"), "description": "pre-med student, so I stay up late and get little sleep so I can save lives one day.", "followers_count": 102, "friends_count": 96, "statues_count": 5216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517233645121537, "text": "������ https://t.co/KsEVRR8j1I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 129353993, "name": "alexander supertramp", "screen_name": "kewonhunter9", "lang": "en", "location": "new orleans.", "create_at": date("2010-04-03"), "description": "you have no idea.", "followers_count": 1593, "friends_count": 1402, "statues_count": 843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517233959731200, "text": "this group chat made my self esteem go down about 293739 points", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3256133370, "name": "KT", "screen_name": "KTSpo14", "lang": "en", "location": "null", "create_at": date("2015-06-25"), "description": "or what", "followers_count": 593, "friends_count": 722, "statues_count": 3026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517234714705920, "text": "I am to�� https://t.co/5SpGfXhtIM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2639311637, "name": "6'5", "screen_name": "landankelly", "lang": "en", "location": "null", "create_at": date("2014-06-24"), "description": "Hooper| Instagram: thewhitemamba17| snapchat: landan23| R.I.P Troy Johnson| single", "followers_count": 449, "friends_count": 696, "statues_count": 9946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336251") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517236451282944, "text": "I'm in so much pain I can't even sleep fuck this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377064245, "name": "lil durk", "screen_name": "hannah_leeigh", "lang": "en", "location": "Bloomsburg, PA", "create_at": date("2011-09-20"), "description": "BU19", "followers_count": 363, "friends_count": 445, "statues_count": 27387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomsburg, PA", "id": "1862782bb20740eb", "name": "Bloomsburg", "place_type": "city", "bounding_box": rectangle("-76.477539,40.977883 -76.428725,41.025192") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42037, "countyName": "Columbia", "cityID": 4207128, "cityName": "Bloomsburg" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517236719529984, "text": "When she lookin Red Velvet cake right out the oven>>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2817335551, "name": "Bishop", "screen_name": "TJ_Chillinhoe", "lang": "en", "location": "Austin-- Victoria ", "create_at": date("2014-09-18"), "description": "Heart Of A Soldier With A Brain To Teach A Whole Nation VC Hooper#24 #Lakers #Bulls #Outlaw", "followers_count": 1622, "friends_count": 1031, "statues_count": 113969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victoria, TX", "id": "70f5587b3e27a105", "name": "Victoria", "place_type": "city", "bounding_box": rectangle("-97.045657,28.709293 -96.900168,28.895775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48469, "countyName": "Victoria", "cityID": 4875428, "cityName": "Victoria" } }
+{ "create_at": datetime("2016-02-05T00:00:05.000Z"), "id": 695517236799340544, "text": "Travelling to Honolulu or just twittering about Honolulu? https://t.co/ouYOScBWhu #Honolulu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.858,21.3069"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Honolulu" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1033, "friends_count": 312, "statues_count": 2573265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517236870516736, "text": "@Jesse__illest Young thug my fav rapper.", "in_reply_to_status": 695516877435617280, "in_reply_to_user": 2957710398, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2957710398 }}, "user": { "id": 432852071, "name": "wons", "screen_name": "reevlilg", "lang": "en", "location": "305 to the 352 ", "create_at": date("2011-12-09"), "description": "Sweeter than heaven, hotter than hell.", "followers_count": 742, "friends_count": 101, "statues_count": 80602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237185114112, "text": "That's me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1463764256, "name": "Alex", "screen_name": "beastescobar", "lang": "en", "location": "null", "create_at": date("2013-05-27"), "description": "Blessed", "followers_count": 334, "friends_count": 328, "statues_count": 8585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237281628161, "text": "yeaaa https://t.co/K5sAashJoO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 617446915, "name": "MARCH/12TH ✊", "screen_name": "JasLaFlare", "lang": "en", "location": "DRE AREA ", "create_at": date("2012-06-24"), "description": "lib", "followers_count": 781, "friends_count": 613, "statues_count": 7965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Discovery Bay, CA", "id": "309e4f83032e9413", "name": "Discovery Bay", "place_type": "city", "bounding_box": rectangle("-121.641476,37.889142 -121.586181,37.932788") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 619339, "cityName": "Discovery Bay" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237474504705, "text": "@HuffingtonPost so why isn't there a pic of Hilary snapped at an inopportune moment with the purpose of making her look ridiculous?", "in_reply_to_status": 695445976807411712, "in_reply_to_user": 14511951, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14511951 }}, "user": { "id": 1431685819, "name": "Arlene Gomez", "screen_name": "arlenegarciago", "lang": "en", "location": "null", "create_at": date("2013-05-15"), "description": "null", "followers_count": 63, "friends_count": 96, "statues_count": 1630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237633880064, "text": "pb&j before i hit the hay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175493513, "name": "RaunieThePoo", "screen_name": "derraunmckenzie", "lang": "en", "location": "Indiana-Oregon-Washington", "create_at": date("2010-08-06"), "description": "You'll Most Likely Find Me Eating Hot Cheetos. Probs Out Venturing Jus Livin & Learnin ~PNW~ •MVHS 2017• Stay Humble ॐ", "followers_count": 447, "friends_count": 508, "statues_count": 9733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237898121216, "text": "Wind 0 mph ---. Barometer 1007.9 hPa, Falling slowly. Temperature 66.9 °F. Rain today 0.00 in. Humidity 46%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 265, "statues_count": 144391 }, "place": { "country": "Puerto Rico", "country_code": "Puerto Rico", "full_name": "Miradero, Puerto Rico", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517237927481344, "text": "@thejoefresh feel ya,bro", "in_reply_to_status": 695443670900674560, "in_reply_to_user": 429356759, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 429356759 }}, "user": { "id": 1094426790, "name": "christian rey flores", "screen_name": "christianf1012", "lang": "en", "location": "United States", "create_at": date("2013-01-15"), "description": "lived and worked in...The ATL, Memphis, nashville, minniapolis, HOUSTONE!!, corpus christi.been places bro.\n#mohrrior# titans fan# hibachi chef#traveler", "followers_count": 398, "friends_count": 1479, "statues_count": 726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheyenne, WY", "id": "75c9243440a46116", "name": "Cheyenne", "place_type": "city", "bounding_box": rectangle("-104.860909,41.078217 -104.713275,41.19235") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56021, "countyName": "Laramie", "cityID": 5613900, "cityName": "Cheyenne" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517238300844032, "text": "@PrincessAlonia woooow", "in_reply_to_status": 695512446102966272, "in_reply_to_user": 3364679892, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3364679892 }}, "user": { "id": 68081225, "name": "Bree", "screen_name": "AyeeeBree", "lang": "en", "location": "San Diego - Los Angeles", "create_at": date("2009-08-22"), "description": "somedays I'm a let down, somedays I'm lit.", "followers_count": 860, "friends_count": 586, "statues_count": 136102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517239039102976, "text": "Got another Sunday off without asking for it hmmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232918389, "name": "Tawannaaa ♡♡♡", "screen_name": "AriesBeautyy__", "lang": "en", "location": "Raleigh", "create_at": date("2011-01-01"), "description": "|God 1st| 20| insta: kissmybeautyy_xo | snapchat: tawanna_xo| RAD❤| Reigns fan| cool, smart,funny,outgoing person who loves watching wrestling", "followers_count": 1707, "friends_count": 2167, "statues_count": 87857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.971861") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517240427311104, "text": "�������� https://t.co/ElFxzJrLZn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 276745991, "name": "〰Hey Zaddy〰", "screen_name": "PRAISEYEEZUSS", "lang": "en", "location": "〰somewhere creating art〰", "create_at": date("2011-04-03"), "description": "Originality. Unique. Wierd. Different.™. ------#crosscountry4life----- MakeUpArtist ///inlove with humans/// #LittleDickNiggas2016 ❤️❤️", "followers_count": 4356, "friends_count": 1849, "statues_count": 95659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.34753 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-02-05T00:00:06.000Z"), "id": 695517240431505408, "text": "Lol he must hate kids. Good lord. https://t.co/vzRozY3BBG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 471812305, "name": "T y l e r", "screen_name": "tyleradm", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2012-01-23"), "description": "NM✈TX", "followers_count": 306, "friends_count": 413, "statues_count": 6273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517241467604992, "text": "������ https://t.co/EzcFVrlodi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 863432959, "name": "♥MOLLY♥", "screen_name": "Laydii_Tii", "lang": "en", "location": "null", "create_at": date("2012-10-05"), "description": "❄♥❄♥❄♥❄♥❄♥❄♥❄♥❄♥", "followers_count": 1147, "friends_count": 662, "statues_count": 18671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bunnell, FL", "id": "01b5bcec4bd314dd", "name": "Bunnell", "place_type": "city", "bounding_box": rectangle("-81.276771,29.451989 -81.223253,29.479077") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12035, "countyName": "Flagler", "cityID": 1209550, "cityName": "Bunnell" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517241924677632, "text": "2/5/2016 - 02:00\nTemp: 23.6F \nHum: 86%\nWind: 0.0 mph\nBaro: 30.459in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 49679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517242134544385, "text": "Barberton Oh Temp:25.3°F Wind:5 mph Dir:WNW Baro:Rising slowly Rain2day:0.00in Hum:75% UV:0.0 @ 03:00 02/05/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 230, "friends_count": 228, "statues_count": 114418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820272,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517243761754112, "text": "@CiannaCamacho see you tomorrow at poster making����", "in_reply_to_status": 695513602619367424, "in_reply_to_user": 242076455, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 242076455 }}, "user": { "id": 553359180, "name": "♡yungnikkimabb♡", "screen_name": "nicolemabbottt", "lang": "en", "location": "null", "create_at": date("2012-04-13"), "description": "i luv disneyland♡ chase and bryan was here proud daughter of darrian nelson and soon to be college dropout", "followers_count": 1140, "friends_count": 790, "statues_count": 73125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517244122488832, "text": "I will literally NEVER get over the fact that Lizette has a YouTube channel OMGG she was so goofy in 7th grade, I love her sm����❤️������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2983071848, "name": "maya g", "screen_name": "mayagrodriguez", "lang": "en", "location": "CA", "create_at": date("2015-01-14"), "description": "null", "followers_count": 565, "friends_count": 322, "statues_count": 32857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shafter, CA", "id": "3b642703733bf53f", "name": "Shafter", "place_type": "city", "bounding_box": rectangle("-119.300846,35.463437 -119.251513,35.521029") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 671106, "cityName": "Shafter" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517244151869440, "text": "I play with her all day.I make her laugh.She makes me laugh.I sit by her at night when she can't sleep, and I sing to her, and rub her hair", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170202259, "name": "Jack", "screen_name": "JackEddieFoster", "lang": "en", "location": "not there. yet. ", "create_at": date("2010-07-23"), "description": "Oliv", "followers_count": 267, "friends_count": 151, "statues_count": 26952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Mountain, NV", "id": "368f3fb0373bfc39", "name": "Battle Mountain", "place_type": "city", "bounding_box": rectangle("-116.975027,40.622834 -116.926795,40.657148") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32015, "countyName": "Lander", "cityID": 3204900, "cityName": "Battle Mountain" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517244487413760, "text": "Wind 0.0 mph ---. Barometer 30.402 in, Steady. Temperature 38.4 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 248743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-05T00:00:07.000Z"), "id": 695517244600651778, "text": "Wind 2.0 mph SSW. Barometer 30.410 in, Rising. Temperature 17.7 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 11, "statues_count": 912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517246123175936, "text": "Amber Cole changed my life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2730700927, "name": "Thomas", "screen_name": "IzabelZambrano", "lang": "en", "location": "Kansas - Las Vegas", "create_at": date("2014-08-13"), "description": "Wonderful Winters - Special Summers", "followers_count": 4306, "friends_count": 3078, "statues_count": 71390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517246307733504, "text": "@Chef_Meng @CosmeLilibeth alden ako pisngi #VoteMaineFPP #KCA @NickelodeonTV", "in_reply_to_status": 695471771663749120, "in_reply_to_user": 4368292520, "favorite_count": 0, "coordinate": point("-118.55342869,34.2008626"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "VoteMaineFPP", "KCA" }}, "user_mentions": {{ 4368292520, 2926245576, 58309829 }}, "user": { "id": 50750437, "name": "sarzki", "screen_name": "saralozada", "lang": "en", "location": "laguna/california", "create_at": date("2009-06-25"), "description": "aldub/maiden/maineterrific/mainestar/aldubprotector/tEAM NO GROUP", "followers_count": 303, "friends_count": 567, "statues_count": 18269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517246555234304, "text": "it is fryDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2746581902, "name": "타일러 매튜스", "screen_name": "tswizzlar", "lang": "en", "location": "null", "create_at": date("2014-08-19"), "description": "Sc: Tyler_Matthewss", "followers_count": 202, "friends_count": 148, "statues_count": 1757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517246844592128, "text": "@phanid24 yours? https://t.co/GrAh9eWF2r", "in_reply_to_status": -1, "in_reply_to_user": 144526377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 144526377 }}, "user": { "id": 15285394, "name": "Rock Southward", "screen_name": "rocksouthward", "lang": "en", "location": "San Bruno, CA", "create_at": date("2008-06-30"), "description": "Retired notary.", "followers_count": 581, "friends_count": 2303, "statues_count": 4024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517247607934976, "text": "https://t.co/4G6txMl4ov", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2430643507, "name": "lost.", "screen_name": "hector_romero97", "lang": "en", "location": "HELL", "create_at": date("2014-04-06"), "description": "Jessica❤️", "followers_count": 239, "friends_count": 303, "statues_count": 2151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-05T00:00:08.000Z"), "id": 695517248765603841, "text": "This what happen when I try and fw you .. I don't get no reply no all my phone calls get denied", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486326551, "name": "cinco", "screen_name": "SwaggyCinco", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2012-02-07"), "description": "#LongLiveDeuce living w/ no limits", "followers_count": 1966, "friends_count": 1658, "statues_count": 64973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-02-05T00:00:09.000Z"), "id": 695517249977851904, "text": "@andeeschmidt is my fav ugh why do I never see you", "in_reply_to_status": -1, "in_reply_to_user": 371096877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371096877 }}, "user": { "id": 1643211553, "name": "❁dee", "screen_name": "desireeesanchez", "lang": "en", "location": "Chandler, AZ", "create_at": date("2013-08-03"), "description": "♍️good girl who dose her make up to trap music", "followers_count": 298, "friends_count": 249, "statues_count": 5378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-05T00:00:09.000Z"), "id": 695517250392952832, "text": "Now I just have to write one more paper and I'm done ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1957645790, "name": "❁❁❁❁", "screen_name": "xoxo_Michelle16", "lang": "en", "location": "Ontario, CA", "create_at": date("2013-10-12"), "description": "17. chs dance & cheer", "followers_count": 1013, "friends_count": 497, "statues_count": 18070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-02-05T00:00:09.000Z"), "id": 695517250778845184, "text": "but really... https://t.co/MeDMWdlOep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3300573947, "name": "$", "screen_name": "okdaisha", "lang": "en", "location": "null", "create_at": date("2015-05-27"), "description": "body piercer | elephant lover | cat mom | cincinnati | IG daisha.lewis | SC daisha.lewis | go listen to my baby", "followers_count": 212, "friends_count": 168, "statues_count": 3248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Hills, CA", "id": "7b81cef09913a7de", "name": "Laguna Hills", "place_type": "city", "bounding_box": rectangle("-117.746489,33.557318 -117.673265,33.631219") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639220, "cityName": "Laguna Hills" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879600660819968, "text": "How do you go from talking mad shit then go hang out them the next day??? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271293112, "name": "fishey", "screen_name": "_Hannah_Osgood_", "lang": "en", "location": "null", "create_at": date("2013-12-31"), "description": "are u gonna eat that // snapchat fishey45", "followers_count": 325, "friends_count": 486, "statues_count": 1537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879600958623744, "text": "Tor Miller \"Carter & Cash\" https://t.co/TBhZsBYzji", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17899859, "name": "YF", "screen_name": "fennn", "lang": "en", "location": "Parallel Universe", "create_at": date("2008-12-05"), "description": "she felt in italics and thought in capitals", "followers_count": 340, "friends_count": 225, "statues_count": 37671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879600975384576, "text": "@PhaseVMusic Goodnight!!! ��������❤️", "in_reply_to_status": 695879503990501376, "in_reply_to_user": 2213559181, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2213559181 }}, "user": { "id": 963988939, "name": "Chrissi", "screen_name": "ChrissiRose23", "lang": "en", "location": "The Pants on Nelson's Hips", "create_at": date("2012-11-22"), "description": "|| GET SHIT DONE || @phasevmusic || @phasev_lbc || Hooseki STL || @brinmcclory ||#TeamNoSleep || #HollywhatCrew || #NelsonsBootySquad ||", "followers_count": 10770, "friends_count": 9322, "statues_count": 56922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879601201938432, "text": "@guycecil @PaulBegala @PeterNicholas3 @billclinton The game is up Rethugs! ��", "in_reply_to_status": 695828567415001088, "in_reply_to_user": 19940085, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19940085, 352548417, 485018010, 1330457336 }}, "user": { "id": 49551962, "name": "Il Volo Maui", "screen_name": "RickiAdoroIV", "lang": "en", "location": "Maui, Hawaii", "create_at": date("2009-06-21"), "description": "Retired Mom and Tutu...Il Volo Fanatic", "followers_count": 633, "friends_count": 1062, "statues_count": 15045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pukalani, HI", "id": "2dab465cb5edceae", "name": "Pukalani", "place_type": "city", "bounding_box": rectangle("-156.359376,20.804118 -156.318233,20.850458") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1565900, "cityName": "Pukalani" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879602418221056, "text": "Temp: 29.9°F Wind:0.0mph Pressure: 30.401hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 59144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879602443517952, "text": "Wind 5.6 mph WNW. Barometer 30.34 in, Falling slowly. Temperature 32.9 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879602544181248, "text": "3:00 AM team puyat for nanay @bernardokath #VoteKathrynFPP #KCA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "VoteKathrynFPP", "KCA" }}, "user_mentions": {{ 197807757 }}, "user": { "id": 2939504119, "name": "BAL", "screen_name": "chxndriaford", "lang": "en", "location": "Queens, NY", "create_at": date("2014-12-22"), "description": "hi, i'm nobody", "followers_count": 924, "friends_count": 263, "statues_count": 38516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879602552451073, "text": "And I'd give up forever to touch you, cause I know that you feel me somehow. �� #iris", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "iris" }}, "user": { "id": 515049181, "name": "Emmelson", "screen_name": "Emsoniac7", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-03-04"), "description": "John 3:30 More loved than I deserve. I don't believe in luck, no, I believe in Grace. Glorifying Christ. IG: @emsoniac7", "followers_count": 278, "friends_count": 227, "statues_count": 4629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879603194241024, "text": "DJ Khaled Is the corniest famous talentleses piece of shit I've seen in a long time. Doubt he even knows he's the butt of everyone's jokes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51887885, "name": "AGENTSPITS FYH", "screen_name": "AGENTSPITS", "lang": "en", "location": "Los Angeles, CA (818)", "create_at": date("2009-06-28"), "description": "Porn Addicted Entrepreneur | FUCK YOUR HUSTLE | FOLLOW @FuckYourHustle @FittedFemales @TheCypherEffect | http://Agentspits.com", "followers_count": 4542, "friends_count": 547, "statues_count": 27316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-06T00:00:00.000Z"), "id": 695879603521335296, "text": "@_pantojapena @radastrology @Merah_Boo you dumb ������❤️", "in_reply_to_status": 694619203114921984, "in_reply_to_user": 2437901106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2437901106, 2717557345, 170866560 }}, "user": { "id": 2384210365, "name": "destinyyy✨", "screen_name": "Dess_Arrieta", "lang": "en", "location": "null", "create_at": date("2014-03-11"), "description": "❤️ sc; destinyy_02", "followers_count": 297, "friends_count": 278, "statues_count": 2269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879603978633216, "text": "I'm a lazy hoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874758362, "name": "ugly ass noodle head", "screen_name": "chrsphrg", "lang": "en", "location": "null", "create_at": date("2014-11-12"), "description": "Leo sun Taurus moon Libra rising", "followers_count": 352, "friends_count": 264, "statues_count": 9469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879604498661376, "text": "Damn man the feels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8167828,39.7792997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2260678572, "name": "Kenya Isais", "screen_name": "africa_kip", "lang": "en", "location": "null", "create_at": date("2013-12-24"), "description": "speaking my mind", "followers_count": 96, "friends_count": 149, "statues_count": 1267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879604595130368, "text": "@annabananaboo_ too bad, no apples. #theshire", "in_reply_to_status": 695825901540356097, "in_reply_to_user": 4434543794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "theshire" }}, "user_mentions": {{ 4434543794 }}, "user": { "id": 296961201, "name": "coco", "screen_name": "coco_london", "lang": "en", "location": "california", "create_at": date("2011-05-11"), "description": "Designer passionate about intimates and fashion. Grateful and madly in love with @n8thesk8 ❤️.", "followers_count": 561, "friends_count": 629, "statues_count": 3179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redondo Beach, CA", "id": "e1e35d357ceefa52", "name": "Redondo Beach", "place_type": "city", "bounding_box": rectangle("-118.401931,33.814689 -118.352695,33.894649") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 660018, "cityName": "Redondo Beach" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879604595224576, "text": "I think its all you\nYou haven't seen yourself lately", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165280323, "name": "Bake Boy Bonnect", "screen_name": "Sam__Contreras", "lang": "en", "location": "Bowling Green State University", "create_at": date("2010-07-10"), "description": "Photographer | #SauceCounsel over everything | Forever #Based | BGSU '19 | NYNMW/OY | SC: Sam_Contrerass |", "followers_count": 491, "friends_count": 436, "statues_count": 21038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879604628754432, "text": "@Aus7ins oh", "in_reply_to_status": 695879566817128448, "in_reply_to_user": 4398690041, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4398690041 }}, "user": { "id": 2676973750, "name": "|Lois|", "screen_name": "mynameisnotlois", "lang": "en", "location": "Newark, OH", "create_at": date("2014-07-05"), "description": "|18, Senior| Nothing can be more important than to take a moment every now and then and accept the invitation of the sky. •I absolutely do not care•", "followers_count": 979, "friends_count": 949, "statues_count": 13923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, OH", "id": "0d257f2a39eb5f9d", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-82.481588,40.035617 -82.333718,40.123777") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39089, "countyName": "Licking", "cityID": 3954040, "cityName": "Newark" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879605199044608, "text": "Everyone in my house is getting wet over Kehlani right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411901179, "name": "SABRINA", "screen_name": "sssahhbreena", "lang": "en", "location": "In my head, CA", "create_at": date("2011-11-13"), "description": "I'm into the lost arts of life. Write me letters. Let me read my journal entries to you. Send me postcards. Pray with me...these are all things we've forgotten.", "followers_count": 159, "friends_count": 221, "statues_count": 8246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879605488525312, "text": "@v_buffington ����❤️����B", "in_reply_to_status": -1, "in_reply_to_user": 1301995963, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1301995963 }}, "user": { "id": 3149244085, "name": "dev", "screen_name": "devyncardoza", "lang": "en", "location": "null", "create_at": date("2015-04-10"), "description": "Victorias dumb for me ❤| Orthodox | cookin at Dutch ☕️", "followers_count": 214, "friends_count": 289, "statues_count": 2140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troutdale, OR", "id": "e0e015b833cb12b4", "name": "Troutdale", "place_type": "city", "bounding_box": rectangle("-122.431645,45.506422 -122.368489,45.561366") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4174850, "cityName": "Troutdale" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879605723471872, "text": "02/06@03:00 - Temp 26.2F, WC 26.2F. Wind 0.7mph SW, Gust 2.0mph. Bar 30.416in, Rising slowly. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879606029541376, "text": "YOOOO THE PARTY REMIX OF PURSUIT OF HAPPINESS IS SO DOPE OMFG I WANNA FUCK SHIT UP RN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005515413, "name": "Rose aka McLovin", "screen_name": "stellarosay_", "lang": "en", "location": "csula", "create_at": date("2012-12-11"), "description": "If you need some enlightenment I can throw a lamp at you", "followers_count": 1065, "friends_count": 943, "statues_count": 23365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879606449147904, "text": "I don't know how dudes have girlfriends that don't respect them as man thats drawln", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2669975261, "name": "bobby centifanti", "screen_name": "OG_shovelboy", "lang": "en", "location": "Bustleton", "create_at": date("2014-07-03"), "description": "RIP 406 MAG", "followers_count": 545, "friends_count": 446, "statues_count": 3345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879606717411328, "text": "Wind 3.0 mph SSW. Barometer 1030.82 mb, Steady. Temperature 36.1 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879607505924097, "text": "@panda_grant is it to late for #FlashbackFriday https://t.co/RqhAdzgUnG", "in_reply_to_status": -1, "in_reply_to_user": 803289036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FlashbackFriday" }}, "user_mentions": {{ 803289036 }}, "user": { "id": 2987769589, "name": "coltitty", "screen_name": "Topless69_", "lang": "en", "location": "WLMS", "create_at": date("2015-01-17"), "description": "I got that good good, I'm colton tople baddd @adriancheri ❤️", "followers_count": 295, "friends_count": 483, "statues_count": 13662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879607602454528, "text": "@ebbtideapp Tide in Philadelphia, Pennsylvania 02/06/2016\n Low 6:33am -0.1\nHigh 12:05pm 6.3\n Low 7:16pm -0.1\nHigh 12:32am 5.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.1383,39.9533"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 23, "friends_count": 1, "statues_count": 5368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879607610953728, "text": "2 Long-eared Owl (Asio otus) - Buenos Aires NWR--Arivaca Cienega - 2016-02-05 16:15 https://t.co/WqjB8ML3MW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 11470 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879607615148032, "text": "3 Long-eared Owl (Asio otus) - Buenos Aires NWR--Arivaca Cienega - 2016-02-05 09:00 https://t.co/WqjB8ML3MW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 11470 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-02-06T00:00:01.000Z"), "id": 695879607665483776, "text": "Nman kyong dlwa @aldenrichards02 @mainedcm respeto nman sa amin mga kinikilig oh ha!!!\n#VoteMaineFPP #KCA https://t.co/KuYWt3Jb0h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "VoteMaineFPP", "KCA" }}, "user_mentions": {{ 98310564, 63701775 }}, "user": { "id": 2740685748, "name": "Jo", "screen_name": "paynejoie", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "FAN of THE PHENOMENAL LOVETEAM #ALDUB YaYaDub & AldenBae", "followers_count": 84, "friends_count": 114, "statues_count": 7892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainfield, IL", "id": "4320ab56929ffcdb", "name": "Plainfield", "place_type": "city", "bounding_box": rectangle("-88.293691,41.566483 -88.152433,41.681434") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1760287, "cityName": "Plainfield" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879608135069696, "text": "Was just greeted by a beer at home����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1494709363, "name": "Zach Reyes", "screen_name": "_ZachReyes", "lang": "en", "location": "null", "create_at": date("2013-06-08"), "description": "Thats How Were Livin'", "followers_count": 414, "friends_count": 373, "statues_count": 1935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879608311242753, "text": "Yall ......... I bought this highlighter .. & it's so beautiful makes me just wanna cry lol LIKE YAAASSSS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3248092609, "name": "Lex ♎", "screen_name": "x_roseeee", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "Oklahoma State University '19 | New Twitter", "followers_count": 415, "friends_count": 425, "statues_count": 15251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879608474861568, "text": "Wind 2.2 mph SSW. Barometer 30.02 in, Steady. Temperature 23.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609263460353, "text": "Wind 0.0 mph ---. Barometer 30.389 in, Steady. Temperature 25.1 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609288626176, "text": "Wind 1.0 mph ESE. Barometer 30.347 in, Steady. Temperature 32.8 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609326284800, "text": "• views • @ Baton Rouge, Louisiana https://t.co/t0TU6SLLaW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.1402,30.4581"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 201477464, "name": "Superstar Smit", "screen_name": "DJSMIT_", "lang": "en", "location": "null", "create_at": date("2010-10-11"), "description": "For bookings: bookdjsmit@gmail.com", "followers_count": 1269, "friends_count": 909, "statues_count": 22555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609359998976, "text": "�������� https://t.co/EQA6j0Chuu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 109378617, "name": "Colby Nate.", "screen_name": "C_Marchio2", "lang": "en", "location": "The 5 †", "create_at": date("2010-01-28"), "description": "The Boy.", "followers_count": 698, "friends_count": 327, "statues_count": 45940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609636655105, "text": "@DenverBRRoncos spit on em!!", "in_reply_to_status": 695877541010092032, "in_reply_to_user": 2912113602, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2912113602 }}, "user": { "id": 312380908, "name": "Crazyra", "screen_name": "b_rabsh", "lang": "en", "location": "!!USA!!", "create_at": date("2011-06-06"), "description": "GO BRONCOS!!!", "followers_count": 50, "friends_count": 280, "statues_count": 676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609766645760, "text": "@Suzy4everfan @BaeSuzynata haha baeb nop jong drink milk tea te deng :D", "in_reply_to_status": 695879383945314304, "in_reply_to_user": 271335974, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user_mentions": {{ 271335974, 1974124550 }}, "user": { "id": 225365350, "name": "tevy❤❤수지", "screen_name": "tevysuji", "lang": "en", "location": "null", "create_at": date("2010-12-11"), "description": "Suzy biased, 2017, ♥♥♥ my askfm is http://ask.fm/portevy\nFollow my bias account: Bae Suzy http://twitter.com/missA_suzy", "followers_count": 2334, "friends_count": 287, "statues_count": 63771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879609859051520, "text": "https://t.co/eaEqoSEpyh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1377736098, "name": "Joe DeSilvio", "screen_name": "jdesilvio13", "lang": "en", "location": "null", "create_at": date("2013-04-24"), "description": "Farrell Senior Varsity Baseball⚾️", "followers_count": 222, "friends_count": 208, "statues_count": 8682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610219700225, "text": "39.6F (Feels: 39.6F) - Humidity: 73% - Wind: 1.6mph E - Gust: 1.6mph - Pressure: 1040.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 225620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610454528000, "text": "If a cute guy was waiting outside my job to massage me for 100$ id give him 200$", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2448338983, "name": "brE", "screen_name": "brenesss", "lang": "en", "location": "Redondo Beach, CA", "create_at": date("2014-04-16"), "description": "$$$ crazy", "followers_count": 198, "friends_count": 193, "statues_count": 3255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610584735744, "text": "If that's how you feeling yea i relate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.2034874,40.7262742"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2901647140, "name": "NO JUDGMENT MIX-TAPE", "screen_name": "SmidlifeTOB", "lang": "en", "location": "Newark, NJ", "create_at": date("2014-12-01"), "description": "#NO JUDGMENT MIX-TAPE \nnever rush greatness", "followers_count": 1125, "friends_count": 1117, "statues_count": 11556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610769084417, "text": "I be trying to mind my business ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 201925300, "name": "pimpKay.❣", "screen_name": "DanniJuicee", "lang": "en", "location": "in his bed ", "create_at": date("2010-10-12"), "description": "snapchat : dannijuicee", "followers_count": 821, "friends_count": 250, "statues_count": 43732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610874003457, "text": "Wind 0.7 mph SW. Barometer 30.32 in, Steady. Temperature 24.8 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879610878152704, "text": "especially to the bee gees ��☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424742329, "name": "alexis", "screen_name": "alexismendoza__", "lang": "en", "location": "Texas ✈️ Nebraska", "create_at": date("2011-11-29"), "description": "such a goddess got a nigga prayin", "followers_count": 1309, "friends_count": 479, "statues_count": 9287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879611880599552, "text": "@Future why you steal @LilTunechi album title lol. #EVOL I HOPE Y'all came to compromise! ������", "in_reply_to_status": -1, "in_reply_to_user": 5526712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EVOL" }}, "user_mentions": {{ 5526712, 116362700 }}, "user": { "id": 249478530, "name": "S€ign€ur $inatra", "screen_name": "KingSinatra", "lang": "en", "location": "Houston, Texas. ", "create_at": date("2011-02-08"), "description": "(Rapper/Producer) I've had almost everybody & everything taken from me, So I'ma Just work With What I Got. (Taken)", "followers_count": 1090, "friends_count": 560, "statues_count": 53159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-06T00:00:02.000Z"), "id": 695879612006420480, "text": "Gorgeous finishing hole as the sun sets at the @wmphoenixopen @ Waste Management Open - TPC… https://t.co/73qS45xeAH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.906185,33.639824"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 98486241 }}, "user": { "id": 250763606, "name": "Chase Morgan", "screen_name": "charmor10", "lang": "en", "location": "null", "create_at": date("2011-02-11"), "description": "If you'da been thinkin you wouldn't a thought that.", "followers_count": 187, "friends_count": 217, "statues_count": 1267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879612211986432, "text": "Feel like I always end up liking someone i don't have a chance with��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 482486779, "name": "KB Money", "screen_name": "KrisBaughBmx", "lang": "en", "location": "City Of Trees", "create_at": date("2012-02-03"), "description": "Lost in weed smoke☁️☁️☁️", "followers_count": 608, "friends_count": 420, "statues_count": 11815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arden-Arcade, CA", "id": "b90f2a335f8565c0", "name": "Arden-Arcade", "place_type": "city", "bounding_box": rectangle("-121.423941,38.562585 -121.327437,38.645482") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602553, "cityName": "Arden-Arcade" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879612362940416, "text": "I be gettin hot over dumb shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3009148625, "name": "CHIFU眼UCHIHA", "screen_name": "SupaChifu", "lang": "en", "location": "null", "create_at": date("2015-01-31"), "description": "//VALLEYWAVEGANG", "followers_count": 285, "friends_count": 670, "statues_count": 3699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Elsinore, CA", "id": "80cf9987ff7e9762", "name": "Lake Elsinore", "place_type": "city", "bounding_box": rectangle("-117.413156,33.618447 -117.216785,33.713284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 639486, "cityName": "Lake Elsinore" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879612824313857, "text": "Can you just love me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310920237, "name": "her", "screen_name": "aerbalac", "lang": "en", "location": "KY|TX", "create_at": date("2011-06-04"), "description": "i love music & my small afro. still discovering myself. read my tweets in an angel voice. @ShaneLaredo look at his cute ass in my header. be groovy", "followers_count": 1721, "friends_count": 684, "statues_count": 123456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, TX", "id": "012a07a105d10298", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-97.186747,32.530915 -97.046821,32.616182") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4846452, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879612971118592, "text": "Temp: 25.0°F - Dew Point: 18.6° - Wind: 7.1 mph - Gust: 8.9 - Rain Today: 0.00in. - Pressure: 30.28in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879613377961986, "text": "@Antimikerobial @FalconPiss911 \n\nI LOVE gluten-free water!!!!11111", "in_reply_to_status": 695875797664464896, "in_reply_to_user": 412017769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 412017769, 142885184 }}, "user": { "id": 43626371, "name": "No Juan Farts", "screen_name": "uid420", "lang": "en", "location": "Long Beach, CA", "create_at": date("2009-05-30"), "description": "waiting for the end of the world", "followers_count": 940, "friends_count": 179, "statues_count": 117567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, NV", "id": "780158265aec966b", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-119.623172,39.200823 -119.50692,39.317313") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32019, "countyName": "Lyon", "cityID": 3217500, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879613545869312, "text": "#SupportOriginMelissa 30.2°F Wind:1.1mph Pressure: 30.33hpa Falling slowly Rain Today 0.00in. Forecast: Fine, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879613566717952, "text": "I did it 30 min before the party by my friend�� I shouldn't had done that lol smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1003339862, "name": "Gutierrez", "screen_name": "Andrewg116", "lang": "en", "location": "C O A L I N G A ☀", "create_at": date("2012-12-10"), "description": "My heart belongs to the most amazing girl in the world @Viridianaa32 & I love her ❤️", "followers_count": 867, "friends_count": 688, "statues_count": 54501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coalinga, CA", "id": "385c3a11a1020ac5", "name": "Coalinga", "place_type": "city", "bounding_box": rectangle("-120.375457,36.117735 -120.326632,36.179965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614274, "cityName": "Coalinga" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879613830959104, "text": "You shleep https://t.co/YrMasrcwEn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1948376018, "name": "JayBam", "screen_name": "BBCjayy", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "My Twitter so I Tweet what I want !", "followers_count": 496, "friends_count": 328, "statues_count": 11140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879614007148545, "text": "I know A lot about it,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1403777,29.9614284"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460737327, "name": "Guwop", "screen_name": "117DipSet", "lang": "en", "location": "Somewhere in America", "create_at": date("2012-01-10"), "description": "Stay Focus and Dedicated...Still taking Trips⛵Around the World...The Money is the Only Concept.......", "followers_count": 442, "friends_count": 441, "statues_count": 15356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson, LA", "id": "782ef2b9d89f68d2", "name": "Jefferson", "place_type": "city", "bounding_box": rectangle("-90.208303,29.946409 -90.128094,29.97364") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2238145, "cityName": "Jefferson" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879614225260544, "text": "Boy ordered to take off Princess Elsa costume during his school's Disney day https://t.co/3XR3AZGoJ5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17596, "friends_count": 17566, "statues_count": 68106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879614623666177, "text": "I miss @krissykrotter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106419575 }}, "user": { "id": 244176786, "name": "•M A C K E N Z I E•", "screen_name": "wacky_macky01", "lang": "en", "location": "Norfolk, NE", "create_at": date("2011-01-28"), "description": "NECC Hawks Volleyball #2", "followers_count": 777, "friends_count": 819, "statues_count": 3348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, NE", "id": "96dc9cb6d51a3b23", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-97.46681,41.989118 -97.368029,42.061692") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31119, "countyName": "Madison", "cityID": 3134615, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879614850150400, "text": "Really thought I'd be the one fucking up my life, and he's in a bad place right now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737592524, "name": "Ernest", "screen_name": "RadAssErnest", "lang": "en", "location": "Texas, USA", "create_at": date("2012-08-04"), "description": "#ErnieVsCris2k16", "followers_count": 440, "friends_count": 464, "statues_count": 16328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo Ranchettes, TX", "id": "01b701f65c623840", "name": "Laredo Ranchettes", "place_type": "city", "bounding_box": rectangle("-99.392153,27.484644 -99.354474,27.500216") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841473, "cityName": "Laredo Ranchettes" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879615093587968, "text": "26 shots later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1500221977, "name": "calvin antonio", "screen_name": "im_calvinn", "lang": "en", "location": "null", "create_at": date("2013-06-10"), "description": "R.I.P Lucia ❤️ forever", "followers_count": 1493, "friends_count": 267, "statues_count": 41 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cary, NC", "id": "5c6bd208d7ddf9f6", "name": "Cary", "place_type": "city", "bounding_box": rectangle("-78.944895,35.644384 -78.729147,35.867993") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3710740, "cityName": "Cary" } }
+{ "create_at": datetime("2016-02-06T00:00:03.000Z"), "id": 695879616033132544, "text": "寝転びながら水飲んだら溺れそうになった。", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 1103550859, "name": "岩 森 大 地@アフィリエイト", "screen_name": "da1114mo", "lang": "ja", "location": "null", "create_at": date("2013-01-19"), "description": "歌って踊れる理論物理学者です。", "followers_count": 434, "friends_count": 331, "statues_count": 34238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Malden, MA", "id": "75f5a403163f6f95", "name": "Malden", "place_type": "city", "bounding_box": rectangle("-71.089522,42.412466 -71.020192,42.444922") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2537875, "cityName": "Malden" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879616678875136, "text": "Love when people catch my subtweets����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906653754, "name": "Madi", "screen_name": "madisonherron", "lang": "en", "location": "null", "create_at": date("2014-11-21"), "description": "play it again", "followers_count": 612, "friends_count": 603, "statues_count": 6726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879617102499840, "text": "some things are too strange and strong to be coincidence's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3197121883, "name": "Sofija Markovic", "screen_name": "sofijaamarkovic", "lang": "en", "location": "null", "create_at": date("2015-05-16"), "description": "snapchat me @sofijakayyy", "followers_count": 152, "friends_count": 191, "statues_count": 393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879617106710528, "text": "what's sleep again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2371126254, "name": "k a i t l y n ❁", "screen_name": "kaitlynwhatever", "lang": "en", "location": "probably with emily", "create_at": date("2014-03-03"), "description": "life's a beach", "followers_count": 499, "friends_count": 815, "statues_count": 10959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879617136046082, "text": "I'm surprised i'm still alive and not dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3608921473, "name": "e〽️s", "screen_name": "enguunb", "lang": "en", "location": "null", "create_at": date("2015-09-18"), "description": "spotify : ems bayar", "followers_count": 96, "friends_count": 92, "statues_count": 17 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879618348224512, "text": "Vee has spent the last hour looking for another game cube controller in the garage closet lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2206194480, "name": "Nelly الكاراز", "screen_name": "xonells", "lang": "en", "location": "Isla Vista || Chatsworth ", "create_at": date("2013-11-20"), "description": "UCSB but not currently", "followers_count": 213, "friends_count": 142, "statues_count": 12150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879618751033344, "text": "Men from Georgetowns accent ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74050188, "name": "Petty Labelle", "screen_name": "ChantelleJanet", "lang": "en", "location": "null", "create_at": date("2009-09-13"), "description": "my life lit. A Money Making Muthafucka", "followers_count": 1012, "friends_count": 1102, "statues_count": 43105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Hill, SC", "id": "001e2a4a7b2f19a5", "name": "Red Hill", "place_type": "city", "bounding_box": rectangle("-79.052947,33.694141 -78.899479,33.838132") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4559190, "cityName": "Red Hill" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879618947993601, "text": "Goodnight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 109489681, "name": "Corey Bryant", "screen_name": "Cali2Texas_88", "lang": "en", "location": "H-Town .....CuHz ", "create_at": date("2010-01-28"), "description": "MiCorazónPriscilla LakersGang Alief ..", "followers_count": 286, "friends_count": 286, "statues_count": 54927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879619115819008, "text": "Barberton Oh Temp:32.0°F Wind:2 mph Dir:SSW Baro:Falling slowly Rain2day:0.00in Hum:59% UV:0.0 @ 03:00 02/06/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879619363233792, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2211306362, "name": "samantha", "screen_name": "__ssaamm_", "lang": "en", "location": "Kingsville, TX", "create_at": date("2013-11-23"), "description": "Know yourself, know your worth.", "followers_count": 211, "friends_count": 85, "statues_count": 7842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879619572928512, "text": "@pinche_gaabs still haven't gotten any ����", "in_reply_to_status": 695875085953961985, "in_reply_to_user": 2722710043, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2722710043 }}, "user": { "id": 1440303776, "name": "Hazeu$ Cri$to®", "screen_name": "JESUSELALMIGHTY", "lang": "en", "location": "408&209", "create_at": date("2013-05-18"), "description": "My intentions won't always be good, but fuck it I'm young, just let me live my life and learn from my mistakes. She the only girl I got heart eyes for.. G.M.❤️", "followers_count": 567, "friends_count": 374, "statues_count": 13955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879619862360064, "text": ":)))", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 574041734, "name": "baye baughman", "screen_name": "bayebaughman", "lang": "en", "location": "frisco, tx--stilly, ok ", "create_at": date("2012-05-07"), "description": "oklahoma state | alpha delta pi", "followers_count": 371, "friends_count": 309, "statues_count": 4317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frisco, TX", "id": "00c3e46c3d0cac08", "name": "Frisco", "place_type": "city", "bounding_box": rectangle("-96.937783,33.081206 -96.732715,33.219073") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4827684, "cityName": "Frisco" } }
+{ "create_at": datetime("2016-02-06T00:00:04.000Z"), "id": 695879619979816960, "text": "@crossingaz ten bucks you're asleep within the next half hour", "in_reply_to_status": 695865323380060160, "in_reply_to_user": 630823590, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 630823590 }}, "user": { "id": 2949242702, "name": "Shay", "screen_name": "shaybuttuh15", "lang": "en", "location": "Sunny SoCal", "create_at": date("2014-12-28"), "description": "i got one hand in my pocket, and the other one is playin' a piano.", "followers_count": 73, "friends_count": 129, "statues_count": 2129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, CA", "id": "e6fc035b8939bd3b", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-118.06946,34.087059 -117.99218,34.180507") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602462, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879620554420224, "text": "Everybody wants to use you for something up to you to decide if it's for the right reasons or the wrong ones", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258574931, "name": "#InfiniteLoopGawd™", "screen_name": "Inked_villain", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-02-27"), "description": "Bitch yes that's me in my Avi but If you're reading my bio then you care to much about me either way I'm handing out fades you know my location stay on", "followers_count": 4347, "friends_count": 1239, "statues_count": 225566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hickory Creek, TX", "id": "3d1a9a56a35f1318", "name": "Hickory Creek", "place_type": "city", "bounding_box": rectangle("-97.062888,33.091352 -96.994121,33.130226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4833476, "cityName": "Hickory Creek" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879620688769025, "text": "Wind 0 mph ---. Barometer 1010.4 hPa, Falling slowly. Temperature 68.9 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 265, "statues_count": 144524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879620864815104, "text": "@titleist4434 ya feel you on that one sometimes", "in_reply_to_status": 695879478442995712, "in_reply_to_user": 1396196138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1396196138 }}, "user": { "id": 842173782, "name": "courteney", "screen_name": "courteneyshay", "lang": "en", "location": "St George, UT", "create_at": date("2012-09-23"), "description": "I'm just chillin", "followers_count": 394, "friends_count": 221, "statues_count": 8423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Haven, UT", "id": "f835686e401ddf33", "name": "West Haven", "place_type": "city", "bounding_box": rectangle("-112.102918,41.175657 -111.998165,41.244377") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4982930, "cityName": "West Haven" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879620948729856, "text": "out of the way, fish face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 925168254, "name": "))))):", "screen_name": "paytondooley", "lang": "en", "location": "nowhere 2 be found", "create_at": date("2012-11-04"), "description": "schs || Andre || moody & easily hurt", "followers_count": 977, "friends_count": 399, "statues_count": 7319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621032542208, "text": "Family doesn't help by f'n annoying me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3244750435, "name": "CHINO", "screen_name": "The_Nigga_chino", "lang": "en", "location": "null", "create_at": date("2015-06-13"), "description": "null", "followers_count": 440, "friends_count": 473, "statues_count": 2293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621083074560, "text": "I'm flexing on purpose when I'm in the club", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368486011, "name": "MJA", "screen_name": "TheUnsubJ", "lang": "en", "location": "NIGERIA/PENNSYLVANIA/ESU", "create_at": date("2011-09-05"), "description": "Another Human Being. I Retweet questionable things. Most likely minding my business.", "followers_count": 1103, "friends_count": 472, "statues_count": 67814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Stroudsburg, PA", "id": "7dc76c5fd4c473fb", "name": "East Stroudsburg", "place_type": "city", "bounding_box": rectangle("-75.19626,40.983946 -75.161432,41.02404") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42089, "countyName": "Monroe", "cityID": 4221872, "cityName": "East Stroudsburg" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621166784514, "text": "@kath2cats @EMcCra2 @AgnosticLiberty @LGamGam @VeritasDolor This is how small pox was defeated", "in_reply_to_status": 695801370591428608, "in_reply_to_user": 15652947, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15652947, 398400514, 3241002972, 16206682, 3403248579 }}, "user": { "id": 108796231, "name": "Becky Johnson", "screen_name": "BeckyJohnson222", "lang": "en", "location": "Santa Cruz, California, USA", "create_at": date("2010-01-26"), "description": "Blogger, teacher, muck-raker, career malcontent, Santa Cruz 11...but I take lots of coffee breaks!", "followers_count": 1445, "friends_count": 2274, "statues_count": 68814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Live Oak, CA", "id": "5518a25c736fbc3e", "name": "Live Oak", "place_type": "city", "bounding_box": rectangle("-122.001081,36.967406 -121.965482,37.014937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 641936, "cityName": "Live Oak" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621229703170, "text": "What the f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734180635, "name": "Erin", "screen_name": "erinpatton11", "lang": "en", "location": "Phillipsburg, Kansas ", "create_at": date("2012-08-02"), "description": "no squares in my circle cuz they don't fit", "followers_count": 224, "friends_count": 220, "statues_count": 5612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phillipsburg, KS", "id": "d7479fb6d0f48ca1", "name": "Phillipsburg", "place_type": "city", "bounding_box": rectangle("-99.33579,39.741289 -99.307703,39.775672") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20147, "countyName": "Phillips", "cityID": 2055675, "cityName": "Phillipsburg" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621347127296, "text": "i think my shit got hacked, followed like 30 people son", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2870943491, "name": "℘яiɳʗɛss♛❤️", "screen_name": "SelenaC___", "lang": "en", "location": "Gonzales, LA", "create_at": date("2014-11-10"), "description": "living...❣", "followers_count": 1673, "friends_count": 1119, "statues_count": 41681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairieville, LA", "id": "01a2904414c2c211", "name": "Prairieville", "place_type": "city", "bounding_box": rectangle("-91.020795,30.223134 -90.870673,30.346967") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2262385, "cityName": "Prairieville" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621670121473, "text": "should of never napped earlier now I can't fall asleep and I have absolutely nobody to talk to GREAT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2929310744, "name": "BbyA♍️", "screen_name": "bby_alexx", "lang": "en", "location": "null", "create_at": date("2014-12-13"), "description": "Stockton⚓️", "followers_count": 286, "friends_count": 206, "statues_count": 8537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621796089856, "text": "I knew id see some shit like that sooner or later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625910289, "name": "B Racks", "screen_name": "Sincerely_Brayy", "lang": "en", "location": "null", "create_at": date("2012-07-03"), "description": "18 / Rapper / Insta: B_Rackss / For booking contact - Bracks740@gmail.com\n#FreeDaGuys\n#FlatoutMusic\n#LCC", "followers_count": 640, "friends_count": 294, "statues_count": 50634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, OH", "id": "0d257f2a39eb5f9d", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-82.481588,40.035617 -82.333718,40.123777") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39089, "countyName": "Licking", "cityID": 3954040, "cityName": "Newark" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621884006400, "text": "@WWEAsuka Think I could spark an old WWE fire whenever you come through Seattle! Never saw it live when I was younger. Good a time as any!", "in_reply_to_status": -1, "in_reply_to_user": 98584746, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 98584746 }}, "user": { "id": 26681445, "name": "Venom", "screen_name": "VenomKitetsu", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-03-25"), "description": "I hate to advocate drugs, alcohol, violence, or insanity to anyone, but they've always worked for me.", "followers_count": 22, "friends_count": 50, "statues_count": 1424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879621930160128, "text": "I learned the hard way!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3346868200, "name": "Lil'Ashley", "screen_name": "AshleyN09265167", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2015-06-26"), "description": "SC @nicolegurle\nIG @ashnicolesmith", "followers_count": 648, "friends_count": 1210, "statues_count": 3048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879622647353344, "text": "Wo liegt Cincinnati? https://t.co/yzlagkE0j0 #Cincinnati #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5175,39.0992"), "retweet_count": 0, "lang": "lv", "is_retweet": false, "hashtags": {{ "Cincinnati", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 429, "friends_count": 114, "statues_count": 1860617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879622844502017, "text": "Hearing your voice brought back so many memories and buried feelings��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 854761345, "name": "Marisela Orozco", "screen_name": "mariselaorozco4", "lang": "en", "location": "null", "create_at": date("2012-09-30"), "description": "Future Educator| hopelessly devoted and in love", "followers_count": 394, "friends_count": 1953, "statues_count": 9589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prosper, TX", "id": "00768d97356634c9", "name": "Prosper", "place_type": "city", "bounding_box": rectangle("-96.807824,33.21871 -96.762213,33.248157") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4859696, "cityName": "Prosper" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879623104602115, "text": "@huntervancexxx Would you like me to send you a case?", "in_reply_to_status": 694783858546458624, "in_reply_to_user": 429457498, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429457498 }}, "user": { "id": 338700974, "name": "Kris", "screen_name": "BrunoFlip2011", "lang": "en", "location": "null", "create_at": date("2011-07-19"), "description": "College Student just Lovin' Life!", "followers_count": 78, "friends_count": 133, "statues_count": 1526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879624597712897, "text": "Islah is straight ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 631615013, "name": "Hussle", "screen_name": "WesHussle1300", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2012-07-09"), "description": "Keep God First, because he puts YOU first snapchat - Weshussle1300 & Insta - Weshussle23", "followers_count": 1507, "friends_count": 1102, "statues_count": 49980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-06T00:00:05.000Z"), "id": 695879624639836160, "text": "what is sobriety", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1915813171, "name": "☾kennedy tesch☽", "screen_name": "BasedTallGrlKay", "lang": "en", "location": "Clemson, SC", "create_at": date("2013-09-28"), "description": "on my worst behavior", "followers_count": 448, "friends_count": 529, "statues_count": 10107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemson, SC", "id": "01ff1adb1d047095", "name": "Clemson", "place_type": "city", "bounding_box": rectangle("-82.853949,34.658367 -82.776522,34.729476") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45077, "countyName": "Pickens", "cityID": 4514950, "cityName": "Clemson" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879625017131008, "text": "My roommate wants to drink but I just wanna sleep lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2218619840, "name": "Jacqueline", "screen_name": "Jacksteer14", "lang": "en", "location": "Humboldt State University ", "create_at": date("2013-11-27"), "description": "let's watch twilight movies & chill", "followers_count": 452, "friends_count": 219, "statues_count": 16625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcata, CA", "id": "2ca541fefabe94d6", "name": "Arcata", "place_type": "city", "bounding_box": rectangle("-124.110814,40.840387 -124.053849,40.909503") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 602476, "cityName": "Arcata" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879625101041664, "text": "I guess imitation is the best form of flattery? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363183511, "name": "Libby", "screen_name": "libharrington", "lang": "en", "location": "CO based, NJ raised", "create_at": date("2011-08-27"), "description": "☕️☀I like dancing, country music, and dancing to country music", "followers_count": 575, "friends_count": 452, "statues_count": 12901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879625319309312, "text": "my childhood lmfao https://t.co/ZspInyaznC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3027755560, "name": "єνιє☄", "screen_name": "eviieee_1", "lang": "en", "location": "jakes bed", "create_at": date("2015-02-09"), "description": "vι.ххх.хιv ♡ @JKoBo_0 | Bitchy & Blessed.. Drake feelings with that Kanye attitude.. Bacardi & Beyoncé | http://vsco.co/xoevieexo", "followers_count": 500, "friends_count": 977, "statues_count": 20626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Center Moriches, NY", "id": "d28961cf00256a59", "name": "Center Moriches", "place_type": "city", "bounding_box": rectangle("-72.820228,40.77718 -72.771706,40.824343") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3613420, "cityName": "Center Moriches" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879625373831168, "text": "I'm fucked up.\nVia left\nTim sleep\nBrenna drooling\nElmer Elmer\nDonald is not dub \nBrett showing dick\nSteve mad\nDee taken", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67766390, "name": "Chino Sola", "screen_name": "xChinoSola", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-08-21"), "description": "ODU Alumni. #ΛΧΑ Valar Morghulis. Brazillian Jiujitsu. Yao", "followers_count": 910, "friends_count": 545, "statues_count": 28602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879625583362048, "text": "You don't know a damn thing about what's normal and natural.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2350122217, "name": "Mufafa", "screen_name": "mufafa3", "lang": "en", "location": "ESSJ", "create_at": date("2014-02-18"), "description": "I'm not flirting, I'm being polite. From Oceanside to Seattle. SAY IT WIDDA GIF! #HorseWhisperer", "followers_count": 541, "friends_count": 532, "statues_count": 17359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879626493550592, "text": "@arsample I for some reason thought you might have had a single you would release", "in_reply_to_status": 695876736186056704, "in_reply_to_user": 58999973, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 58999973 }}, "user": { "id": 55754358, "name": "Karsten", "screen_name": "baseballkunit", "lang": "en", "location": "Black Diamond, WA", "create_at": date("2009-07-10"), "description": "Rhymes with bros.... Mojitos! -Thad", "followers_count": 122, "friends_count": 129, "statues_count": 1094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Black Diamond, WA", "id": "6d8731cfb2572844", "name": "Black Diamond", "place_type": "city", "bounding_box": rectangle("-122.054431,47.286736 -121.995892,47.34361") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5306330, "cityName": "Black Diamond" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879626661486593, "text": "slapped af wit my nigga @_CertifiedBish @clowwyl_ an matt ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 391485202, 4246167214 }}, "user": { "id": 2808248806, "name": "Alec", "screen_name": "DelusionalA", "lang": "en", "location": "null", "create_at": date("2014-10-05"), "description": "free salo | coonin where it's safe", "followers_count": 167, "friends_count": 190, "statues_count": 4631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879626673885184, "text": "My review of #CodeBlack this week: �������������� \nGripping. If you're not watching hurry up & catch up to ep 15!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CodeBlack" }}, "user": { "id": 17272731, "name": "taradonovan", "screen_name": "taradonovan", "lang": "en", "location": "Los Angeles", "create_at": date("2008-11-09"), "description": "actor,producer,director,wife,adventurer,ambitious, dog lover,creative. Biz owner: @actorscreative; Managing Dir.: @lilfishtheatre", "followers_count": 445, "friends_count": 736, "statues_count": 2009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879626711650304, "text": "@mariina_serna oh damn �� you went there", "in_reply_to_status": 695879521703055360, "in_reply_to_user": 829626242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 829626242 }}, "user": { "id": 1323386154, "name": "Andrew Magwood", "screen_name": "MagwoodOfficial", "lang": "en", "location": "somewhere leading to nowhere", "create_at": date("2013-04-02"), "description": "forward in everything", "followers_count": 143, "friends_count": 146, "statues_count": 12014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879627869270017, "text": "Aye girl, you like guacamole?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005404360, "name": "Edgar Allen Flow", "screen_name": "__ayge", "lang": "en", "location": "Moore, OK", "create_at": date("2012-12-11"), "description": "null", "followers_count": 663, "friends_count": 493, "statues_count": 10642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879628339204096, "text": "@woahits_erin Way ahead of you", "in_reply_to_status": 695817415519764480, "in_reply_to_user": 339318671, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339318671 }}, "user": { "id": 3393131795, "name": "Housquad", "screen_name": "housquad44", "lang": "en", "location": "Trap House", "create_at": date("2015-07-28"), "description": "In a case of beer", "followers_count": 78, "friends_count": 88, "statues_count": 338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor Mill, KY", "id": "1b1a9aa55e0023bc", "name": "Taylor Mill", "place_type": "city", "bounding_box": rectangle("-84.524565,38.985365 -84.458827,39.040716") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2175738, "cityName": "Taylor Mill" } }
+{ "create_at": datetime("2016-02-06T00:00:06.000Z"), "id": 695879628771037184, "text": "Sudden Love For E46's������ https://t.co/CXMFXD8CBf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558453861, "name": "B-randon", "screen_name": "BChristie743", "lang": "en", "location": "California, USA", "create_at": date("2012-04-19"), "description": "Skis Are Cool.", "followers_count": 154, "friends_count": 238, "statues_count": 490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soulsbyville, CA", "id": "0b9cbb28274cb277", "name": "Soulsbyville", "place_type": "city", "bounding_box": rectangle("-120.286276,37.973497 -120.247876,38.017457") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6109, "countyName": "Tuolumne", "cityID": 672772, "cityName": "Soulsbyville" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241987389751296, "text": "Wish I was off weekends and rich", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2851351033, "name": "☣", "screen_name": "MikeP_254", "lang": "en", "location": "Texas, USA", "create_at": date("2014-10-10"), "description": "Blessed with no bible", "followers_count": 53, "friends_count": 38, "statues_count": 2758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241987779842049, "text": "I can dance", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2308933471, "name": "BBynder", "screen_name": "BBynder", "lang": "en", "location": "null", "create_at": date("2014-01-24"), "description": "yes", "followers_count": 304, "friends_count": 311, "statues_count": 2726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241987859705856, "text": "@Ellzeaa your avi. I'm obsessed.", "in_reply_to_status": -1, "in_reply_to_user": 919518289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 919518289 }}, "user": { "id": 128984704, "name": "josie", "screen_name": "novoajoc", "lang": "en", "location": "null", "create_at": date("2010-04-02"), "description": "19. from miami. Ismael", "followers_count": 1572, "friends_count": 664, "statues_count": 23626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Springs, FL", "id": "88272cf603c89248", "name": "Winter Springs", "place_type": "city", "bounding_box": rectangle("-81.326884,28.658956 -81.208792,28.730516") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1278325, "cityName": "Winter Springs" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241988501286912, "text": "Hey @Harry_Styles wanna come to ktown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181561712 }}, "user": { "id": 53121089, "name": "@mcondon63", "screen_name": "mcondon63", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-07-02"), "description": "I love Taco Bell and Boston sports. I know almost every word to every One Direction song.", "followers_count": 157, "friends_count": 537, "statues_count": 5416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241988711137280, "text": "Not that yaw care but.....Goin to sleep ✌��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44554674, "name": "Childish Be♠nz", "screen_name": "J_Burse25", "lang": "en", "location": "Tennessee 》Kankakee》Michigan》", "create_at": date("2009-06-03"), "description": "Do it 4 #Fo6 God put me on..... Talkin things into existance Pro basketball player Young Millionaire workin to be a Billionaire #Pray4Me", "followers_count": 409, "friends_count": 568, "statues_count": 15546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kankakee, IL", "id": "7271433758c9510f", "name": "Kankakee", "place_type": "city", "bounding_box": rectangle("-87.928206,41.059959 -87.812921,41.145902") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17091, "countyName": "Kankakee", "cityID": 1738934, "cityName": "Kankakee" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241988870529024, "text": "Temp: 30.6°F Wind:0.0mph Pressure: 30.302hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 59240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241989600153600, "text": "@gsm3campbell thanks working on it.", "in_reply_to_status": 696216234258407424, "in_reply_to_user": 3024433867, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024433867 }}, "user": { "id": 14864352, "name": "Johanna Baxter", "screen_name": "hanniespice", "lang": "en", "location": "Yukon Harbor, WA", "create_at": date("2008-05-21"), "description": "cute democrat •won't fix your computer•activist •social media Girl Scout •#wadem #26dems • smartass@hannie.ninja", "followers_count": 2613, "friends_count": 2778, "statues_count": 144089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southworth, WA", "id": "013fe7da4e1cd678", "name": "Southworth", "place_type": "city", "bounding_box": rectangle("-122.550681,47.504816 -122.494904,47.524695") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5366220, "cityName": "Southworth" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241989642113024, "text": "Me me me me me https://t.co/QunjPQunZO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 217910406, "name": "La Reina", "screen_name": "yasminn__e", "lang": "en", "location": "Murrieta, CA", "create_at": date("2010-11-20"), "description": "please help me live on this earth", "followers_count": 816, "friends_count": 573, "statues_count": 18683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241989784702977, "text": "Time:02:00 Temp:45.9°F Dewpt:33.6°F Press:30.32in. Wind N at 0mph gusting 0mph Rain Today 0.00in. Humidity:62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.60305556,27.84555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448017712, "name": "AnnavilleWeather", "screen_name": "AnnavilleWX", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2011-12-27"), "description": "live weather from Annaville, Texas! FMI see http://www.wunderground.com/weatherstation/WXDailyHistory.asp?ID=KTXCORPU31", "followers_count": 85, "friends_count": 395, "statues_count": 18397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241989944156160, "text": "Fuck it gonna sleep this weekend sucked", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417685562, "name": "God Machine", "screen_name": "BryantChapel", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "There is a hell believe me I've seen it, There is a heaven let's keep it a secret", "followers_count": 963, "friends_count": 422, "statues_count": 125292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Park, CA", "id": "7b85d19c09869d20", "name": "Walnut Park", "place_type": "city", "bounding_box": rectangle("-118.230699,33.959223 -118.210743,33.974727") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683402, "cityName": "Walnut Park" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241989952507904, "text": "Targeting selfies at Target lol. This was at the Target around The Plant Shopping Center in San Jose, CA. (2-4-16) https://t.co/dw3cR7pqRH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54463831, "name": "William Justo", "screen_name": "54stormywill54", "lang": "en", "location": "California, USA", "create_at": date("2009-07-06"), "description": "Californian storm chaser/spotter. Love to hang out with friends/family. Stormy weather gives me adrenaline rush! My IG:\nhttp://iconosquare.com/54stormywill54", "followers_count": 1019, "friends_count": 1210, "statues_count": 23109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241990053154816, "text": "@Pickering you must have tweeted in a way he didn't like or got tired of.", "in_reply_to_status": 696217761207353344, "in_reply_to_user": 1771151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1771151 }}, "user": { "id": 50664887, "name": "Jim Harris", "screen_name": "jimharris360", "lang": "en", "location": "Little Rock, AR", "create_at": date("2009-06-25"), "description": "Voice of the PBHS Zebras; Columnist, http://SportingLifeArkansas.com; member FWAA/USBWA; contributor to KTHV-TV/Hogzone: always in search of good gumbo.", "followers_count": 7592, "friends_count": 1216, "statues_count": 26023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241990208516096, "text": "#BACK2BACK\n#REDBIRDhoops", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.983272,40.509621"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BACK2BACK", "REDBIRDhoops" }}, "user": { "id": 1339316582, "name": "Joe Stokowski", "screen_name": "RedbirdRecon", "lang": "en", "location": "Redbird Arena TT Row 7 Seat 31", "create_at": date("2013-04-09"), "description": "Prematurely rushed the field at Appalachian State. Ejected from Redbird Arena in a red bathrobe. Scatter my ashes alongside I-74. Gladly I learn and teach. RB4L", "followers_count": 1220, "friends_count": 1167, "statues_count": 11725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "D.P. Dough Calzones", "id": "07d9e3c054c87000", "name": "D.P. Dough Calzones", "place_type": "poi", "bounding_box": rectangle("-88.9832721,40.5096209 -88.983272,40.509621") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1706613, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241990392877056, "text": "I want so badly for @LukasGraham to add me as a friend on Snapchat! That would be Golden!:)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 299686136 }}, "user": { "id": 2566891974, "name": "Brooke Norton", "screen_name": "Brookiebaby19", "lang": "en", "location": "Brighton, CO", "create_at": date("2014-06-14"), "description": "Hey, I'm Brooke!! I'm 20 years old. God. Family. Milky Chance. Nursing. Lukas Graham. Gavin DeGraw. Dance.", "followers_count": 41, "friends_count": 296, "statues_count": 80 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241991068180480, "text": "Yes in monopoly https://t.co/RoaDn0w80S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2721036961, "name": "Eric Wright Jr.", "screen_name": "Kentetheking", "lang": "en", "location": "Norf Rialto", "create_at": date("2014-08-09"), "description": "Rest Easy CJ", "followers_count": 1102, "friends_count": 811, "statues_count": 41715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-02-07T00:00:00.000Z"), "id": 696241991189975040, "text": "Respect ������ https://t.co/x9zJDWNbQ7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2418306332, "name": "Steph", "screen_name": "ambitious_steph", "lang": "en", "location": "null", "create_at": date("2014-03-29"), "description": "Future Pediatric Surgeon", "followers_count": 147, "friends_count": 252, "statues_count": 2361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241992280322048, "text": "Fucku", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 793032193, "name": "alexis", "screen_name": "PokemonAlexx", "lang": "en", "location": "US: 37.766697,-122.208225", "create_at": date("2012-08-30"), "description": "null", "followers_count": 322, "friends_count": 252, "statues_count": 41077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241992536190979, "text": "Lol so you just throwing dinners on niggas? https://t.co/u7yv0V3RAT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1445323885, "name": "Almighty Bir6oBatman", "screen_name": "GeauxBIRDO", "lang": "en", "location": "CA➡️LA", "create_at": date("2013-05-20"), "description": "We scrong? Den leggo! #GramFam19 #CowboysNation LASAGNA", "followers_count": 494, "friends_count": 510, "statues_count": 25766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grambling, LA", "id": "7659c27719e754b6", "name": "Grambling", "place_type": "city", "bounding_box": rectangle("-92.734385,32.50363 -92.669504,32.551821") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2230515, "cityName": "Grambling" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241992737554432, "text": "Tired asf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2509341885, "name": "Daddykayy❤️", "screen_name": "JaKaylaBreonn", "lang": "en", "location": "Port Arthur, TX", "create_at": date("2014-04-25"), "description": "ig: @daddykayy___ sc: @jakaylabreonn", "followers_count": 929, "friends_count": 1120, "statues_count": 8924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Arthur, TX", "id": "350ad20be66e9edc", "name": "Port Arthur", "place_type": "city", "bounding_box": rectangle("-94.053284,29.823923 -93.858553,29.979166") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858820, "cityName": "Port Arthur" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241993526157314, "text": "Temp 23.9°F Wind Chill 23.9°F RH 86% Wind 0.0 --- Gust 0.0 --- SLP 30.277 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 104, "friends_count": 63, "statues_count": 33085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bluefield, WV", "id": "8057944e55790b13", "name": "Bluefield", "place_type": "city", "bounding_box": rectangle("-81.265356,37.234874 -81.165628,37.281748") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54055, "countyName": "Mercer", "cityID": 5408524, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241993614303232, "text": "Wind 1.0 mph SE. Barometer 30.166 in, Steady. Temperature 41.3 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241993945632768, "text": "Wind 5.6 mph WNW. Barometer 30.22 in, Falling. Temperature 34.3 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241993958031361, "text": "lol girl anyways", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3090490231, "name": "♡", "screen_name": "jaaaisy", "lang": "en", "location": "bay area", "create_at": date("2015-03-16"), "description": "i love my wifey (;", "followers_count": 399, "friends_count": 392, "statues_count": 3262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241993987411968, "text": "\"Guys what are we right now?\" \"What?\" \"What are we?\" \"...?\" \"We're ridin dirty\" *puts on ridin' dirty*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21272414, "name": "Madiiiii", "screen_name": "madisonxmorgan", "lang": "en", "location": "chino hills", "create_at": date("2009-02-18"), "description": "not looking to be found, just want to feel (un)lost.", "followers_count": 729, "friends_count": 332, "statues_count": 38213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241994260041728, "text": "Need lovings, cuddles, and appreciation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712515894, "name": "฿❁♡", "screen_name": "Meoworneverrr", "lang": "en", "location": "hell", "create_at": date("2012-07-23"), "description": "very much so in love ♡ B & C ♡ •donate bitcoins•", "followers_count": 2882, "friends_count": 1546, "statues_count": 70279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Place, WA", "id": "1061b231ed1e21cf", "name": "College Place", "place_type": "city", "bounding_box": rectangle("-118.427153,46.000721 -118.362663,46.060655") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53071, "countyName": "Walla Walla", "cityID": 5313855, "cityName": "College Place" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241994348105728, "text": "Wind 5.0 mph SSW. Barometer 1027.03 mb, Falling. Temperature 35.1 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241994876719104, "text": "02/07@03:00 - Temp 27.1F, WC 27.1F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.258in, Falling slowly. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 46826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241995149250561, "text": "I miss Ray", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442269697, "name": "¯\\_(ツ)_/¯", "screen_name": "michizzlep", "lang": "en", "location": "csuci", "create_at": date("2011-12-20"), "description": "null", "followers_count": 334, "friends_count": 313, "statues_count": 24257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241995358949376, "text": "Bye nigga https://t.co/RQtn5eRtLN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1319311032, "name": "terriyaki", "screen_name": "QueenTerio", "lang": "en", "location": "null", "create_at": date("2013-03-31"), "description": "null", "followers_count": 731, "friends_count": 387, "statues_count": 22511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-02-07T00:00:01.000Z"), "id": 696241995518513153, "text": "I wish life weren't so confusing. I wish people would just say what's on their mind.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 599697475, "name": "Ernie Turpin", "screen_name": "SpiralTap", "lang": "en", "location": "Parts Unknown", "create_at": date("2012-06-04"), "description": "Humor. Skill. Wit. Sex appeal. That order.", "followers_count": 74, "friends_count": 180, "statues_count": 1344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Christiansburg, VA", "id": "91a897085ee3fdee", "name": "Christiansburg", "place_type": "city", "bounding_box": rectangle("-80.452465,37.108491 -80.314581,37.17791") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51121, "countyName": "Montgomery", "cityID": 5116608, "cityName": "Christiansburg" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241996063592448, "text": "@BasedBlob20666 HAPPY FUCKING BIRTHDAY I LOVE YOU.", "in_reply_to_status": -1, "in_reply_to_user": 359394494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 359394494 }}, "user": { "id": 3162275941, "name": "homegirl", "screen_name": "honkeyhoodrat", "lang": "en", "location": "PNW", "create_at": date("2015-04-18"), "description": "always been a fuck up.", "followers_count": 318, "friends_count": 216, "statues_count": 2174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241996172816384, "text": "Wind 0.0 mph ---. Barometer 30.227 in, Falling slowly. Temperature 30.8 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241997267517440, "text": "@Myy_Nameshiwote I'm outside come out", "in_reply_to_status": 696241534451240960, "in_reply_to_user": 371073046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371073046 }}, "user": { "id": 364087561, "name": "♡iffy♡", "screen_name": "ifrahtheboss", "lang": "en", "location": "old dominion university ", "create_at": date("2011-08-28"), "description": "old enough to know better, too young to care ✨\nwhat they gon' do, yellz & if", "followers_count": 1847, "friends_count": 1393, "statues_count": 27414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241997330440193, "text": "02/07/16, 02:00 am - Temp/Dewpoint: 34.0F/30.8F. Humidity 88%. Pressure 29.924 in/Hg and Falling. Wind SSW at 3.0 mph, gusting to 7.0 mph.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.94722222,43.36777778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442475015, "name": "Tim Gonyo", "screen_name": "SaukvilleWX", "lang": "en", "location": "Saukville, WI", "create_at": date("2011-12-20"), "description": "Weather enthusiast. Enough said :)", "followers_count": 12, "friends_count": 7, "statues_count": 7316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grafton, WI", "id": "26922abb216a7b6c", "name": "Grafton", "place_type": "city", "bounding_box": rectangle("-87.980941,43.279784 -87.901263,43.36854") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55089, "countyName": "Ozaukee", "cityID": 5530000, "cityName": "Grafton" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241997368139776, "text": "Maxina https://t.co/AC7Bsu1GNr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34395078, "name": "maxina .♛", "screen_name": "maxx1naa", "lang": "en", "location": "Prov RI", "create_at": date("2009-04-22"), "description": "(19) IG//maxx1naa", "followers_count": 1741, "friends_count": 1451, "statues_count": 33558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warwick, RI", "id": "d1871b0189c27a21", "name": "Warwick", "place_type": "city", "bounding_box": rectangle("-71.518925,41.629501 -71.354415,41.76838") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44003, "countyName": "Kent", "cityID": 4474300, "cityName": "Warwick" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241997439328257, "text": "U do what u want when u poppin��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1490982253, "name": "angelina wells", "screen_name": "aannggeelliina", "lang": "en", "location": "BR", "create_at": date("2013-06-07"), "description": "Ayy lmao", "followers_count": 1295, "friends_count": 657, "statues_count": 21161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairieville, LA", "id": "01a2904414c2c211", "name": "Prairieville", "place_type": "city", "bounding_box": rectangle("-91.020795,30.223134 -90.870673,30.346967") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2262385, "cityName": "Prairieville" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241997741314048, "text": "Larry David's 'Bern Your Enthusiasm' is our new favorite show https://t.co/aptaZHal80", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17595, "friends_count": 17565, "statues_count": 68139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241998135562240, "text": "Lmao wow and don't even respond at that note", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372574167, "name": "Boneca", "screen_name": "Dahl_18", "lang": "en", "location": "null", "create_at": date("2011-09-12"), "description": "null", "followers_count": 819, "friends_count": 571, "statues_count": 8082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241998169313280, "text": "I can't wait for me and @ShahElias to move into our apartment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 473297064 }}, "user": { "id": 468880396, "name": "Beau Branham", "screen_name": "iB34U_is", "lang": "en", "location": "Defiance", "create_at": date("2012-01-19"), "description": "I will destroy you in the most beautiful way possible and when I leave, You will finally understand why storms are named after people.", "followers_count": 283, "friends_count": 148, "statues_count": 21828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Defiance, OH", "id": "01833b183bff6006", "name": "Defiance", "place_type": "city", "bounding_box": rectangle("-84.431477,41.240069 -84.29444,41.32798") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39039, "countyName": "Defiance", "cityID": 3921308, "cityName": "Defiance" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241998777331712, "text": "39.8F (Feels: 39.8F) - Humidity: 94% - Wind: 4.5mph NW - Gust: 6.9mph - Pressure: 1039.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 225764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241998785875968, "text": "Temp: 48.0°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 30.03 in | Dewpoint: 47.8°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 26, "friends_count": 1, "statues_count": 164870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241999029084162, "text": "Hello to the real ones", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180530474, "name": "Metro Boomin", "screen_name": "TylerMeyers904", "lang": "en", "location": "THE FAM ftf Diverse Cultures ", "create_at": date("2010-08-19"), "description": "The real U 2018 Apollo96DVC@yahoo.com", "followers_count": 606, "friends_count": 565, "statues_count": 18689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turnersville, NJ", "id": "014af7545dd54df8", "name": "Turnersville", "place_type": "city", "bounding_box": rectangle("-75.130194,39.698642 -75.015244,39.804669") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3474270, "cityName": "Turnersville" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241999041720320, "text": "It is so hard being away from the people that you love when they need it the most.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 740173914, "name": "Shelby Moreland", "screen_name": "_inanutshelb", "lang": "en", "location": "null", "create_at": date("2012-08-06"), "description": "the best way to find yourself is to lose yourself in the service of others| Ohio is home but I don't belong here", "followers_count": 558, "friends_count": 259, "statues_count": 12701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-07T00:00:02.000Z"), "id": 696241999523872770, "text": "okc lame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4772480360, "name": "✨la ruca✨", "screen_name": "bl8k3l3", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2016-01-16"), "description": "Im Venezuelan and a Leo so you might as well leave while u have the chance to - No I dont want ur number I dont want to give you mine dont want none of ur time", "followers_count": 234, "friends_count": 342, "statues_count": 1223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000140500992, "text": "Tonight reminded me why I don't drink anymore lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503728600, "name": "Bobsღ", "screen_name": "Bobbiee_Annee", "lang": "en", "location": "RIP Hmillahh❤️", "create_at": date("2012-02-25"), "description": "babygirl❥", "followers_count": 1158, "friends_count": 462, "statues_count": 27477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alvin, TX", "id": "4c456186fb6d326e", "name": "Alvin", "place_type": "city", "bounding_box": rectangle("-95.36277,29.366642 -95.18921,29.483007") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4802272, "cityName": "Alvin" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000383844352, "text": "Wind 0.0 mph SW. Barometer 30.17 in, Falling slowly. Temperature 32.5 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 107, "statues_count": 157309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000434241536, "text": "Can you find Austin on the map? Just try it at https://t.co/bdSAJVqXgX #Austin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7431,30.2672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Austin" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1040, "friends_count": 311, "statues_count": 2577249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000924774400, "text": "Drunk on a Saturday night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2338888416, "name": "chasadeibinder", "screen_name": "_xocjb", "lang": "en", "location": "Oklahoma, USA ", "create_at": date("2014-02-11"), "description": "Just a young parent tryna make it ,tryna raise my son the right way. on a path of self discovery b/c sometimes you have to say ; you're worth something✨", "followers_count": 212, "friends_count": 383, "statues_count": 4809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000945815552, "text": "What's crackin? �� https://t.co/QtUZ5m94YB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2731095018, "name": "♈️ASAP Marlon♈", "screen_name": "gucccii_21", "lang": "en", "location": "null", "create_at": date("2014-08-14"), "description": "Nths/ Let's get it/ Work hard for what you want and twice as hard to keep it/ It's all about your state of mind", "followers_count": 654, "friends_count": 401, "statues_count": 16389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lacey, WA", "id": "0084a2ce4881fa56", "name": "Lacey", "place_type": "city", "bounding_box": rectangle("-122.839662,46.959463 -122.691155,47.119369") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5336745, "cityName": "Lacey" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242000983515136, "text": "@xmadeleineluux I'm dying ��", "in_reply_to_status": 696220179936575488, "in_reply_to_user": 2202490879, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2202490879 }}, "user": { "id": 711927823, "name": "RY K", "screen_name": "iryanhou", "lang": "en", "location": "California, USA", "create_at": date("2012-07-22"), "description": "null", "followers_count": 80, "friends_count": 71, "statues_count": 117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple City, CA", "id": "206e932cc0e89fa2", "name": "Temple City", "place_type": "city", "bounding_box": rectangle("-118.081392,34.085395 -118.028425,34.120984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 678148, "cityName": "Temple City" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001105199104, "text": "I'm an emotional wreck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1118417094, "name": "natnat", "screen_name": "adroveeer", "lang": "en", "location": "null", "create_at": date("2013-01-24"), "description": "null", "followers_count": 693, "friends_count": 350, "statues_count": 23929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001415516161, "text": "Really missing you right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 478032915, "name": "Katie Lynn", "screen_name": "Katie_Zink", "lang": "en", "location": "null", "create_at": date("2012-01-29"), "description": "I love cats & dogs equally // LDS // Smh Kara", "followers_count": 273, "friends_count": 190, "statues_count": 7663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rexburg, ID", "id": "da61090cbdc5350e", "name": "Rexburg", "place_type": "city", "bounding_box": rectangle("-111.818018,43.79717 -111.757595,43.842019") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16065, "countyName": "Madison", "cityID": 1667420, "cityName": "Rexburg" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001428086785, "text": "I love home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1291823690, "name": "Cisco", "screen_name": "_ciscolara15", "lang": "en", "location": "2T", "create_at": date("2013-03-23"), "description": "grinding", "followers_count": 214, "friends_count": 283, "statues_count": 1282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Twin Falls, ID", "id": "21fafa3ac3a26744", "name": "Twin Falls", "place_type": "city", "bounding_box": rectangle("-114.509204,42.516101 -114.421106,42.606409") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16083, "countyName": "Twin Falls", "cityID": 1682810, "cityName": "Twin Falls" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001600077824, "text": "Add me on snapchat sparkkyy����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fi", "is_retweet": false, "user": { "id": 2371304412, "name": "holly", "screen_name": "hoolllyyy_", "lang": "en", "location": "null", "create_at": date("2014-03-03"), "description": "steven james lazar ❤", "followers_count": 270, "friends_count": 413, "statues_count": 10651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001667297280, "text": "No man is ready to live life until he is no longer afraid of death", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2398554266, "name": "Santi ↺19", "screen_name": "SantiagoGambo20", "lang": "en", "location": "Weston, FL ", "create_at": date("2014-03-19"), "description": "null", "followers_count": 174, "friends_count": 408, "statues_count": 694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weston, FL", "id": "88a21205dd4728ea", "name": "Weston", "place_type": "city", "bounding_box": rectangle("-80.442024,26.062031 -80.361426,26.144693") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1276582, "cityName": "Weston" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001725886464, "text": "Pet peeves����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159738607, "name": "Auntie Brea", "screen_name": "breaatate", "lang": "en", "location": "null", "create_at": date("2010-06-25"), "description": "Raven Tate #trapsoul #MUSTBENICE", "followers_count": 431, "friends_count": 525, "statues_count": 11676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001772179456, "text": "Travelling to Durham or just twittering about Durham? https://t.co/Obf421kUEJ #Durham", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8986,35.994"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Durham" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1040, "friends_count": 311, "statues_count": 2577250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242001805574145, "text": "YOUR LOVES LIKE HONEY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491756715, "name": "ash kardash", "screen_name": "TRAPSTARASH", "lang": "en", "location": "null", "create_at": date("2012-02-13"), "description": "god • family • tsu volleyball • coach • friend. SC: hiashlynnnnn", "followers_count": 1572, "friends_count": 490, "statues_count": 114185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242002543931392, "text": "@knikhusky @RBeus @tedcruz That is music to my ears!", "in_reply_to_status": 695055716957057024, "in_reply_to_user": 43274529, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43274529, 613941136, 23022687 }}, "user": { "id": 425204803, "name": "Thomas J Edwards", "screen_name": "Conservative4MI", "lang": "en", "location": "Argentine Township, Michigan", "create_at": date("2011-11-30"), "description": "Conservative4Ever by blazing a trail through liberals to replace the stolen building blocks of our Constitutional Republic #VoteforCruz #OathKeepers Retired LEO", "followers_count": 4067, "friends_count": 4081, "statues_count": 28556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Argentine, MI", "id": "137df940d373fee9", "name": "Argentine", "place_type": "city", "bounding_box": rectangle("-83.866024,42.766952 -83.805472,42.813863") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2603400, "cityName": "Argentine" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242002694840320, "text": "I've been sober from alcohol and drugs for a solid month... so that's new", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4139165834, "name": "Mark Rodriguez", "screen_name": "markrodriguez95", "lang": "en", "location": "Ceres, CA", "create_at": date("2015-11-05"), "description": "Vibe with me", "followers_count": 49, "friends_count": 96, "statues_count": 289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-02-07T00:00:03.000Z"), "id": 696242003097432064, "text": "Seriously driving down 13 and a car was on the wrong side of the highway in the lane coming right towards me...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 380567033, "name": "kaitlyn. ⚓", "screen_name": "_Kateylynn", "lang": "en", "location": "null", "create_at": date("2011-09-26"), "description": "@jpeto28", "followers_count": 784, "friends_count": 1024, "statues_count": 25604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgeville, DE", "id": "0064fb1f25be28ea", "name": "Bridgeville", "place_type": "city", "bounding_box": rectangle("-75.610407,38.711293 -75.589207,38.752153") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1008680, "cityName": "Bridgeville" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242004464828417, "text": "exactly 2:00. phone at exactly 100%\nI can sleep soundly now. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41734964, "name": "e.", "screen_name": "voellercoaster", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2009-05-21"), "description": "21 | proud golden gopher", "followers_count": 534, "friends_count": 287, "statues_count": 17817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242004670484480, "text": "too dunrj for this https://t.co/6KflmcT2tt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1188326718, "name": "d med", "screen_name": "danielllanm", "lang": "en", "location": "miami/orlando", "create_at": date("2013-02-16"), "description": "✨ pizza + baseball = happy dani // UCF Ad/PR ✨ writer for http://theodysseyonline.com", "followers_count": 949, "friends_count": 591, "statues_count": 62330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "01f196c135e37b16", "name": "University", "place_type": "city", "bounding_box": rectangle("-81.224485,28.56471 -81.170066,28.61218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242004708077568, "text": "Chinese New Year dinner during the super bowl every damn year ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1312222182, "name": "Lilia ♔", "screen_name": "lilia_noelle", "lang": "en", "location": "Vancouver, WA", "create_at": date("2013-03-28"), "description": "i want it all", "followers_count": 727, "friends_count": 692, "statues_count": 5973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salmon Creek, WA", "id": "01d1bc4e2f2710a8", "name": "Salmon Creek", "place_type": "city", "bounding_box": rectangle("-122.713017,45.692689 -122.620464,45.765171") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5361000, "cityName": "Salmon Creek" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242004926337026, "text": "yo damani , khalil, eddie & adrian had me crying them niggas deadass funny asffffff ����������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 471160310, "name": "jaylah da dancer", "screen_name": "eldabish", "lang": "en", "location": "somewhere counting money", "create_at": date("2012-01-22"), "description": "2016 & I'm still saucy hoe\n\n\n\n\n natyia ❤", "followers_count": 1182, "friends_count": 764, "statues_count": 60541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242005295247360, "text": "I miss my bf.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 369864342, "name": "ash", "screen_name": "ashlynlepori", "lang": "en", "location": "null", "create_at": date("2011-09-07"), "description": "why fit in when you were born to stand out? SC:ashlyntaylor17", "followers_count": 637, "friends_count": 289, "statues_count": 19200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242005970722817, "text": "smh im so embarrassed. i don't know why craig would do that https://t.co/1y3QF169Cr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2223826860, "name": "katie vaneekeren", "screen_name": "katievanee", "lang": "en", "location": "null", "create_at": date("2013-11-30"), "description": "null", "followers_count": 218, "friends_count": 205, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheaton, IL", "id": "57ac88aefcd20988", "name": "Wheaton", "place_type": "city", "bounding_box": rectangle("-88.151554,41.814328 -88.074936,41.889152") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1781048, "cityName": "Wheaton" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006280916992, "text": "https://t.co/OJGVQQARfi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1056895580, "name": "msdanielaarceo", "screen_name": "danielaarceo133", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-01-02"), "description": "@msleamichele❂@corymonteith❤️•LEANATIC• MONTOURAGE•GLEEK• Met Lea 9/25/15 @glee_reacts Noticed by Lea 11•24•15 MONCHELE TRASH™", "followers_count": 819, "friends_count": 1408, "statues_count": 69487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006322860032, "text": "MSNBC's Rachel Maddow, \"Hard to imagine' Bernie Sanders winning.\" #NERDLAND #PoliticsNation https://t.co/TRg7CWvbyO https://t.co/NXyZ4nCKrL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NERDLAND", "PoliticsNation" }}, "user": { "id": 634676291, "name": "Dr. Tom Martin Ph.D.", "screen_name": "DrTomMartinPhD", "lang": "en", "location": "null", "create_at": date("2012-07-13"), "description": "Psychologist, 41 years in Mental Health. I SMELL RETIREMENT!", "followers_count": 56811, "friends_count": 4501, "statues_count": 57490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006482423808, "text": "Wind 0.0 mph ---. Barometer 30.242 in, Falling. Temperature 26.2F. Rain today 0.00 in. Humidity 93% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 592, "friends_count": 797, "statues_count": 36795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006574571520, "text": "lmao that aint my girl man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 731619420, "name": "ej", "screen_name": "lishaajaay", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "No time for a 9", "followers_count": 845, "friends_count": 536, "statues_count": 55787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006578872320, "text": "hash tag : you getting mad and I'm getting rich * dabs *", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129300312, "name": "nas", "screen_name": "SxdNa", "lang": "en", "location": "null", "create_at": date("2010-04-03"), "description": "boss up or get left", "followers_count": 999, "friends_count": 252, "statues_count": 71314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, NC", "id": "4548df3d99bb4891", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-79.195342,34.936245 -78.808883,35.173009") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3722920, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242006859890689, "text": "Where is Durham on the map? Play the game at https://t.co/Obf421kUEJ #Durham", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8986,35.994"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Durham" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1040, "friends_count": 311, "statues_count": 2577251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007111393280, "text": "Cats* https://t.co/3gXHUCVtaN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331663325, "name": "Bree Jacoba", "screen_name": "BreeJacoba", "lang": "en", "location": "University of Iowa", "create_at": date("2011-07-08"), "description": "† Philippians 4:13", "followers_count": 1237, "friends_count": 749, "statues_count": 25736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, IA", "id": "6515916567ad4b34", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-91.167512,40.763821 -91.094099,40.858065") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19057, "countyName": "Des Moines", "cityID": 1909550, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007505645568, "text": "@ryannnbrowwn @kledbetter8 I haven't dyed it since May and I got it cut Jan 2 and haven't put any heat on it! It feels so short still haha", "in_reply_to_status": 696230499430182912, "in_reply_to_user": 429603271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429603271, 2185141045 }}, "user": { "id": 239742467, "name": "kay", "screen_name": "KaylaBrissey", "lang": "en", "location": "null", "create_at": date("2011-01-18"), "description": "ewu • snapchat: kaylabrissey ࿊", "followers_count": 847, "friends_count": 559, "statues_count": 12159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007602114560, "text": "y is everybody sleepin?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434646417, "name": "megan fox", "screen_name": "shmorgasmm", "lang": "en", "location": "Chicago,IL ", "create_at": date("2011-12-11"), "description": "Don't judge me until you know me. Don't underestimate me until you challenge me. And don't talk about me until you talk to me!", "followers_count": 366, "friends_count": 779, "statues_count": 5837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007648239616, "text": "in 2 minutes Late NIGHT wih The @PLEASURESDUDES Channel 3 KCDO @bmxfear @DenverStrippers @denver_colorado #sexy #sex https://t.co/SstRtROIVA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sexy", "sex" }}, "user_mentions": {{ 21610672, 27588369, 87790832, 16940047 }}, "user": { "id": 21610672, "name": "PleasuresDudes", "screen_name": "PLEASURESDUDES", "lang": "en", "location": "Denver/Vegas/LA", "create_at": date("2009-02-22"), "description": "We talk about all things Pleasures: Beautiful women, toys, oils, DVD's. News & Interviews. PleasuresDudes on IG and Facebook. #TV #18+ only #adult #film #promos", "followers_count": 159574, "friends_count": 14, "statues_count": 3523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007769874433, "text": "a dude just hit me with \"Y'all want that kush, only asking for $10\" God is literally dropping me hints, should I? https://t.co/jqUSmWHv4H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424699063, "name": "christianmingle", "screen_name": "omfgchristian", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "controversial but not in a bad way @ UTA", "followers_count": 457, "friends_count": 334, "statues_count": 20627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242007870558208, "text": "It's ma burrrrthdayyyyyyy ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 810181004, "name": "Serenaaaaa", "screen_name": "SerenaLeann", "lang": "en", "location": "PHS", "create_at": date("2012-09-07"), "description": "null", "followers_count": 608, "friends_count": 909, "statues_count": 18755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-02-07T00:00:04.000Z"), "id": 696242008239681536, "text": "Mhmmm your so fine��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3150693026, "name": "$elenaa❤️", "screen_name": "selenaababyxoxo", "lang": "en", "location": "null", "create_at": date("2015-04-11"), "description": "null", "followers_count": 773, "friends_count": 742, "statues_count": 6003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antelope, CA", "id": "013214de57b4da2e", "name": "Antelope", "place_type": "city", "bounding_box": rectangle("-121.403842,38.702271 -121.30897,38.728927") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602210, "cityName": "Antelope" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242008386437120, "text": "@afizzleee_ sneak out ��", "in_reply_to_status": 696241530617475072, "in_reply_to_user": 2943441782, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2943441782 }}, "user": { "id": 2426384491, "name": "Kazi Zuhayr", "screen_name": "kazizuhayr", "lang": "en", "location": "null", "create_at": date("2014-04-03"), "description": "Varsity Wrestler CL", "followers_count": 454, "friends_count": 258, "statues_count": 2869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242008390656000, "text": "I should have stayed in Baytown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62431992, "name": "Amber B", "screen_name": "KashKardsahh", "lang": "en", "location": "Houston, TX", "create_at": date("2009-08-02"), "description": "Persian Princess", "followers_count": 1396, "friends_count": 688, "statues_count": 24055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242009401483264, "text": "03:00 41.2°F Feels:37.5°F (Hi41.5°F/Lo39.2°F) Hum:44% Wnd:WSW 5.5MPH Baro:29.89in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 176, "friends_count": 265, "statues_count": 25427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242009707827200, "text": "Babe just gave me the best booty rub . . ���� god love him ❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2975200834, "name": "Kayla LeAnn", "screen_name": "kaebitchesss_x0", "lang": "en", "location": "null", "create_at": date("2015-01-12"), "description": "My King , My World ; Karson Zane Nestor , 05/29/2015. ❤️ I am my own critic #TeamTatted #ASR", "followers_count": 959, "friends_count": 1987, "statues_count": 4033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waynesboro, PA", "id": "1585099b374245ac", "name": "Waynesboro", "place_type": "city", "bounding_box": rectangle("-77.606069,39.732851 -77.559569,39.770529") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42055, "countyName": "Franklin", "cityID": 4281824, "cityName": "Waynesboro" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242009833537537, "text": "@PresAlohaBitch lol my nephew he'll be 1 in March", "in_reply_to_status": 696241710242865152, "in_reply_to_user": 69343984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69343984 }}, "user": { "id": 431218347, "name": "1", "screen_name": "KingTralle", "lang": "en", "location": "Southside", "create_at": date("2011-12-07"), "description": "mackin and dabbin through the pressure..... I'm just here for the hoes\n#HGod", "followers_count": 3845, "friends_count": 1924, "statues_count": 268139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242009976270848, "text": "@SkyloMarQ 90. 1 year, 1 day apart. How odd yet cool as hell", "in_reply_to_status": 696241884335726592, "in_reply_to_user": 148044695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148044695 }}, "user": { "id": 3063489658, "name": "Surly Murdock", "screen_name": "Sooper_Jae", "lang": "en", "location": "Atlanta, GA", "create_at": date("2015-02-25"), "description": "IT is the source of my coins. Tennis Fan. Music Head. Patti LaBelle & Rihanna & Serena are my obsessions. #Rafa #Rena #RiRi #Patti #RnB #iBluPhi #UTKAlum", "followers_count": 449, "friends_count": 936, "statues_count": 22507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vinings, GA", "id": "fee8047a46fbdfc8", "name": "Vinings", "place_type": "city", "bounding_box": rectangle("-84.487242,33.841705 -84.454216,33.8811") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1379612, "cityName": "Vinings" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242010294845440, "text": "@Smellody_Marie cause everything is funny when your drunk ��", "in_reply_to_status": 696241796419035136, "in_reply_to_user": 2312298379, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2312298379 }}, "user": { "id": 629874903, "name": "Christian Grey", "screen_name": "CR3stian", "lang": "en", "location": "null", "create_at": date("2012-07-07"), "description": "#3 #soccerbabe #sanJulian Without risk, there is no victory #nike banda el recodo taught me ......Snapchat: christiahhn", "followers_count": 378, "friends_count": 450, "statues_count": 22724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cicero, IL", "id": "50f296a1bd4198b6", "name": "Cicero", "place_type": "city", "bounding_box": rectangle("-87.77903,41.82158 -87.738571,41.865972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714351, "cityName": "Cicero" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242010613641217, "text": "Like how they're playing beer pong but neither team is drinking ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344935174, "name": "Harlz", "screen_name": "HarleyKraus", "lang": "en", "location": "Fairbanks Alaska", "create_at": date("2011-07-29"), "description": "null", "followers_count": 539, "friends_count": 235, "statues_count": 40088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242010617991168, "text": "�������� some ppl really deserve to die this my 100th town tweeting this.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2966903297, "name": "Whats the facts?", "screen_name": "zoebrincess", "lang": "en", "location": "Your Dad Crib", "create_at": date("2015-01-07"), "description": "Undefeated 7-0", "followers_count": 538, "friends_count": 230, "statues_count": 20702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242010697564161, "text": "Lmao!!!! I watched him tonight on SNL! I was dyin https://t.co/nVd7h6agH5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163755160, "name": "иσт ʝυѕт вℓυffιи", "screen_name": "DeronTesfay", "lang": "en", "location": "Some Where In Seattle ☔", "create_at": date("2010-07-06"), "description": "Currently... Im High off Knowledge, Overdosing on Confidence & Drunk off Ideas", "followers_count": 1456, "friends_count": 861, "statues_count": 71467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242010886246401, "text": "Mount Saint Mary's university https://t.co/OiS341MxxG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2489423790, "name": "unfriendly", "screen_name": "muusshroom", "lang": "en", "location": "yo mama's house ", "create_at": date("2014-05-11"), "description": "19 and uninterested . snap/ig @Muusshroom", "followers_count": 616, "friends_count": 372, "statues_count": 14002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242011163258880, "text": "@Bible_Time", "in_reply_to_status": 696237410825474048, "in_reply_to_user": 374209360, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 374209360 }}, "user": { "id": 2795211074, "name": "SAVED", "screen_name": "HARUN_CARSWELL", "lang": "en", "location": "Detroit, MI", "create_at": date("2014-09-06"), "description": "God Loves Us All!\nharuncarswell@gmail.com", "followers_count": 829, "friends_count": 1947, "statues_count": 17114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242011527991296, "text": "1 in da mornin.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558656348, "name": "Shaniya Wood", "screen_name": "ssshaniyaa", "lang": "en", "location": "null", "create_at": date("2012-04-20"), "description": "mason's mommy.\nsc: ssshaniyaa", "followers_count": 655, "friends_count": 313, "statues_count": 28017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242011561656320, "text": "Barberton Oh Temp:29.2°F Wind:0 mph Dir:--- Baro:Falling slowly Rain2day:0.00in Hum:62% UV:0.0 @ 03:00 02/07/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242011674906624, "text": "don't know what to do at this point", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1602688345, "name": "em", "screen_name": "_emilymullins", "lang": "en", "location": "LA", "create_at": date("2013-07-17"), "description": "null", "followers_count": 506, "friends_count": 126, "statues_count": 4634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-07T00:00:05.000Z"), "id": 696242012069175296, "text": "Wind 0 mph SE. Barometer 1010.3 hPa, Falling. Temperature 72.3 °F. Rain today 0.00 in. Humidity 31%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 265, "statues_count": 144665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-07T00:00:06.000Z"), "id": 696242012761169920, "text": "I WAS HAVING A GOOD NIGHT TOO UGH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313077098, "name": "BBYKERI", "screen_name": "keraganro", "lang": "en", "location": "bay area, ca", "create_at": date("2015-08-11"), "description": "null", "followers_count": 129, "friends_count": 163, "statues_count": 1545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alameda, CA", "id": "000e96b4e9f8503f", "name": "Alameda", "place_type": "city", "bounding_box": rectangle("-122.332411,37.720367 -122.224562,37.797229") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 600562, "cityName": "Alameda" } }
+{ "create_at": datetime("2016-02-07T00:00:06.000Z"), "id": 696242012761280512, "text": "It's 3am and I'm awake and sober. Duty days can pick one and suck it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56257325, "name": "Tanner Powell", "screen_name": "TJPowell11", "lang": "en", "location": "Choctaw, OK ✈️ Groton, CT ", "create_at": date("2009-07-12"), "description": "I'm just here so I won't get fined #Covert #USN⚓️", "followers_count": 817, "friends_count": 370, "statues_count": 41224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groton, CT", "id": "280ad9d1d1fe1d71", "name": "Groton", "place_type": "city", "bounding_box": rectangle("-72.093466,41.312122 -72.018387,41.400455") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 934180, "cityName": "Groton" } }
+{ "create_at": datetime("2016-02-07T00:00:06.000Z"), "id": 696242013088452609, "text": "I find it funny when someone attempts to lie to my face, but I already know the truth.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1561458050, "name": "Aqua", "screen_name": "DaavidRaamirez", "lang": "en", "location": "null", "create_at": date("2013-07-01"), "description": "sc/daavidraamirez", "followers_count": 871, "friends_count": 576, "statues_count": 17165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendale Lakes, FL", "id": "c28ef1055654ebbb", "name": "Kendale Lakes", "place_type": "city", "bounding_box": rectangle("-80.431609,25.684836 -80.383241,25.730043") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236062, "cityName": "Kendale Lakes" } }
+{ "create_at": datetime("2016-02-07T00:00:06.000Z"), "id": 696242013239259137, "text": "I loved this today ������❤️ https://t.co/VLN38ABTld", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1020904514, "name": "ale", "screen_name": "alejandram704", "lang": "en", "location": "California", "create_at": date("2012-12-18"), "description": "|khs sophmore|i beat cancers booty 4/15/14| kamfam af| @kalinwhite followed 4/22/14 |@yourboymyles followed 3/20/15|click the link below please(:", "followers_count": 540, "friends_count": 1160, "statues_count": 6003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604375859359744, "text": "It's New Years and I'm netflix n chillin by my damn self lol. Yay.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1965241477, "name": "Kimberly Nguyen", "screen_name": "kktnx3", "lang": "en", "location": "Annie Truong's Viet School", "create_at": date("2013-10-16"), "description": "I.D.? Tao di.", "followers_count": 159, "friends_count": 150, "statues_count": 4877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604376052269056, "text": "On ya mind. https://t.co/i0NSztN69L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1454656152, "name": "Chocolate Drippa ✨", "screen_name": "DamnTeeTee", "lang": "en", "location": "Arlington, TX", "create_at": date("2013-05-24"), "description": "if you gone do it, do it right", "followers_count": 547, "friends_count": 486, "statues_count": 48260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604376173916161, "text": "@TheDannyPino, what a stud. ��", "in_reply_to_status": -1, "in_reply_to_user": 325964747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 325964747 }}, "user": { "id": 246608002, "name": "iiidalisss", "screen_name": "idalis_maria", "lang": "en", "location": "Bend, OR", "create_at": date("2011-02-02"), "description": "I tweet about sports, hiking , my Subaru, and random quotes from The Office - RCTID⭐️, Ripcity, Ducks, Steelers, Pirates, & Penguins -", "followers_count": 416, "friends_count": 649, "statues_count": 7884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bend, OR", "id": "545a2392307d8893", "name": "Bend", "place_type": "city", "bounding_box": rectangle("-121.373464,43.999225 -121.2497,44.120651") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41017, "countyName": "Deschutes", "cityID": 4105800, "cityName": "Bend" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604376224280577, "text": "Shit it sound wild out here LMAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2423251596, "name": "jodes", "screen_name": "joditruong", "lang": "en", "location": "null", "create_at": date("2014-04-01"), "description": "SUGA HONEY ICED TEA", "followers_count": 396, "friends_count": 412, "statues_count": 4983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604376266252288, "text": "@GeorgeMichael #EltonJohn + #JamesCorden singing #carkaraoke in London. https://t.co/95OGEAvivZ", "in_reply_to_status": -1, "in_reply_to_user": 260292052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EltonJohn", "JamesCorden", "carkaraoke" }}, "user_mentions": {{ 260292052 }}, "user": { "id": 1705156099, "name": "Tom Rockman Jr.", "screen_name": "tom_rockman", "lang": "en", "location": "Denver, Colorado", "create_at": date("2013-08-27"), "description": "Raw, Radical, & Unabashed Tweets About Current Events, Politics, Pop Culture, Public Policy, & Queerdom.", "followers_count": 5, "friends_count": 5, "statues_count": 21 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604376547389440, "text": "Still in commerce ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864355627, "name": "Lil Kourt❤️", "screen_name": "Kourtbabby", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "You don't have to like me go love someone else ✌️", "followers_count": 606, "friends_count": 592, "statues_count": 22374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604377004396544, "text": "Which is something out of a fiction movie, but it's been going on since 10 years old.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25652802, "name": "♔ kevin perelman ♔", "screen_name": "KevinPerelman", "lang": "en", "location": "Woodland Hills", "create_at": date("2009-03-21"), "description": "Im Kevin Perelman.You all know me, but pretend not to due to a world wide funded smear campaign since 10. Nerdist, Artist, Photographr,Web Developer,3D graphics", "followers_count": 2009, "friends_count": 2428, "statues_count": 808082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604377075744768, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3181338626, "name": "alex", "screen_name": "alex_morenoooo", "lang": "en", "location": "null", "create_at": date("2015-04-30"), "description": "cheating is encouraged", "followers_count": 49, "friends_count": 59, "statues_count": 297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604377130405889, "text": "02/08@03:00 - Temp 38.3F, WC 33.1F. Wind 7.0mph N, Gust 15.0mph. Bar 29.762in, Falling quickly. Rain 0.00in. Hum 62%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 46850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604377855979521, "text": "Wind 0.1 mph SW. Barometer 29.66 in, Falling. Temperature 33.1 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 23, "friends_count": 0, "statues_count": 30559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604378245890048, "text": "1 more minuuuute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438976178, "name": "Daniel Isaac", "screen_name": "DanielLaQuifa", "lang": "en", "location": "null", "create_at": date("2011-12-16"), "description": "I cannot be Good. I must be Perfection.", "followers_count": 282, "friends_count": 302, "statues_count": 10773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richland, WA", "id": "0dd0c9c93b5519e1", "name": "Richland", "place_type": "city", "bounding_box": rectangle("-119.348075,46.164988 -119.211248,46.351367") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5358235, "cityName": "Richland" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604378338168832, "text": "@HintOfSunshine @zestiny_ peep the name", "in_reply_to_status": 667510424967905280, "in_reply_to_user": 3142945639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3142945639, 1671694176 }}, "user": { "id": 497491671, "name": "tapout val", "screen_name": "ValGlizzy", "lang": "en", "location": "Hannah is my person.", "create_at": date("2012-02-19"), "description": "17 || Senior || Panama ✈️ Maryland || #TapoutGvng ||", "followers_count": 1039, "friends_count": 524, "statues_count": 64499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joppatowne, MD", "id": "d0cf2ff6aa7f9a6b", "name": "Joppatowne", "place_type": "city", "bounding_box": rectangle("-76.380876,39.390005 -76.319829,39.446112") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2442875, "cityName": "Joppatowne" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604378631839744, "text": "hateee these allergies ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2741719729, "name": "⠀jocelyne", "screen_name": "jocelhynn", "lang": "en", "location": "Texas ", "create_at": date("2014-08-18"), "description": "Drake & Real Madrid #10", "followers_count": 52, "friends_count": 87, "statues_count": 1199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604378799677440, "text": "I'm at @Walmart Supercenter in Vancouver, WA https://t.co/qYWvvwMfyo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.5627182,45.62305662"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17137891 }}, "user": { "id": 15186298, "name": "PrinceDanteRose", "screen_name": "PrinceDanteRose", "lang": "en", "location": "Vancouver, Wash.", "create_at": date("2008-06-20"), "description": "null", "followers_count": 558, "friends_count": 637, "statues_count": 36886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-08T00:00:00.000Z"), "id": 696604379164454912, "text": "@ElexiKID BREH", "in_reply_to_status": 695505105429237764, "in_reply_to_user": 17552923, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 17552923 }}, "user": { "id": 1858703192, "name": "=͟͟͞͞➳♡", "screen_name": "hugdrugs", "lang": "en", "location": "ur extended network", "create_at": date("2013-09-12"), "description": "ur coffin ,,or mine?", "followers_count": 234, "friends_count": 182, "statues_count": 8345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Halawa, HI", "id": "ae8bddc0e5653780", "name": "Halawa", "place_type": "city", "bounding_box": rectangle("-157.937712,21.355729 -157.897921,21.393936") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1510000, "cityName": "Halawa" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604380133511169, "text": "Ceiling fan you feel so good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54017531, "name": "Jeff Blackwood", "screen_name": "jsbtarheel95", "lang": "en", "location": "Durham, NC", "create_at": date("2009-07-05"), "description": "42. Dad to 2 girls. I am what I am and I ain't what I ain't. Music makes me high. UNC '95 WLHS '91 #TarHeels #DaleJr #Braves #Redskins #TAMU #MMA #WWE", "followers_count": 1667, "friends_count": 1030, "statues_count": 158247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604380355809280, "text": "Wind 5.6 mph WNW. Barometer 30.01 in, Falling. Temperature 41.0 °F. Rain today 0.00 in. Humidity 45%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604380540219392, "text": "Wind 10.0 mph NW. Barometer 1023.47 mb, Steady. Temperature 38.3 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604380921921540, "text": "Happy Chinese New Years!!!����������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842637718, "name": "Staniel", "screen_name": "south_stanley", "lang": "en", "location": "Bae Area", "create_at": date("2014-10-06"), "description": "BBS BHAMPIONS BITCH || It Is What It Is", "followers_count": 381, "friends_count": 317, "statues_count": 15063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604381895008257, "text": "being single is lame, having hoes is lame... I'm just fucking LAME ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1113105247, "name": "I.", "screen_name": "ILOVEISAIAHH", "lang": "en", "location": "South Central CA ", "create_at": date("2013-01-22"), "description": "null", "followers_count": 1588, "friends_count": 1056, "statues_count": 38161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604381915906048, "text": "Me too @alex_morenoooo https://t.co/cwuF6SaLKk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3181338626 }}, "user": { "id": 3700672273, "name": "Alexandra Ramirez", "screen_name": "okyoungshawty", "lang": "en", "location": "null", "create_at": date("2015-09-26"), "description": "sc okyoungshawty || Pisces", "followers_count": 303, "friends_count": 376, "statues_count": 2732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604382448594944, "text": "@bpletch2971 https://t.co/1QeYrNQJte", "in_reply_to_status": -1, "in_reply_to_user": 261832352, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 261832352 }}, "user": { "id": 304810051, "name": "Alonzo santiago", "screen_name": "zosantiago17", "lang": "en", "location": "null", "create_at": date("2011-05-24"), "description": "#IUP17 Sports Management", "followers_count": 415, "friends_count": 391, "statues_count": 13839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, PA", "id": "d88a5def1d7e9609", "name": "Indiana", "place_type": "city", "bounding_box": rectangle("-79.213942,40.586024 -79.089163,40.656614") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42063, "countyName": "Indiana", "cityID": 4236816, "cityName": "Indiana" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604382863892481, "text": "@william_adamss @UnexpectedJlhad DEAD��", "in_reply_to_status": 696604194808049664, "in_reply_to_user": 2244353564, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2244353564, 3035083902 }}, "user": { "id": 1376270377, "name": "Nicole Affronti", "screen_name": "NicoleAffronti", "lang": "en", "location": "Cave Creek, AZ ➡️ Seward, NE", "create_at": date("2013-04-23"), "description": "My favorite word starts with the letter N. BMS alumni #GoGoats", "followers_count": 225, "friends_count": 162, "statues_count": 2926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seward, NE", "id": "6e24828ef23eb299", "name": "Seward", "place_type": "city", "bounding_box": rectangle("-97.11277,40.898539 -97.074813,40.930589") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31159, "countyName": "Seward", "cityID": 3144420, "cityName": "Seward" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604383312621568, "text": "I give credit where due. I just watched @BeavertonPolice stop a car as I awaited a bus. What I saw/heard was very courteous and professional", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 609584086 }}, "user": { "id": 43071692, "name": "Matthew House, J.D.", "screen_name": "BaseballJustice", "lang": "en", "location": "Beaverton, OR", "create_at": date("2009-05-28"), "description": "#JuvenileJustice system officials: Are you part of the problem or part of the solution? And how about you, @BeavertonPolice?", "followers_count": 725, "friends_count": 55, "statues_count": 2149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-02-08T00:00:01.000Z"), "id": 696604383480446976, "text": "@ebbtideapp Tide in Vaca Key-Fat Deer Key bridge, Florida 02/08/2016\nHigh 8:53am 1.1\n Low 2:25pm -0.0\nHigh 8:36pm 1.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.03,24.73"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 25, "friends_count": 1, "statues_count": 5914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marathon, FL", "id": "301bd8da9d45994b", "name": "Marathon", "place_type": "city", "bounding_box": rectangle("-81.124761,24.688245 -80.923527,24.777279") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12087, "countyName": "Monroe", "cityID": 1243000, "cityName": "Marathon" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604383807549440, "text": "ITS MY BIIIIRTHDAAAYYYYY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 858558907, "name": "ally", "screen_name": "allysterzer", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2012-10-02"), "description": "null", "followers_count": 367, "friends_count": 286, "statues_count": 9071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604384294105088, "text": "We've moved on from baseball, hopefully we do it again soon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167994940, "name": "ch@z", "screen_name": "chazroman", "lang": "en", "location": "USA (they/them)", "create_at": date("2010-07-17"), "description": "everyone's favorite friendly neighborhood Dwayne the rock Johnson", "followers_count": 207, "friends_count": 276, "statues_count": 17033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604384902275072, "text": "Watch Your Company & Who You Entertain ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2499002083, "name": "licia✨♉️", "screen_name": "MissAliciaaa_", "lang": "en", "location": "null", "create_at": date("2014-05-16"), "description": "S.I.C.J ✝ . ❤️ snap: miss.licia.", "followers_count": 1523, "friends_count": 1102, "statues_count": 13452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terrell, TX", "id": "00afd84c14c34f15", "name": "Terrell", "place_type": "city", "bounding_box": rectangle("-96.342904,32.701185 -96.256653,32.764463") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4872284, "cityName": "Terrell" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385007144961, "text": "*says I wish I was 21 everyday until I actually turn 21*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366499877, "name": "ㅤㅤㅤ", "screen_name": "inabootymedown", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "too many thoughts for this thot", "followers_count": 667, "friends_count": 311, "statues_count": 38473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mokuleia, HI", "id": "40916e4823c4bc65", "name": "Mokuleia", "place_type": "city", "bounding_box": rectangle("-158.184859,21.57372 -158.135076,21.584314") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1552550, "cityName": "Mokuleia" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385418354688, "text": "Wind 4.0 mph NW. Barometer 29.832 in, Falling. Temperature 38.5 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385481064448, "text": "@paulnenoff LMAO THIS IS YOU TO A T https://t.co/5f0SBEQPu9", "in_reply_to_status": -1, "in_reply_to_user": 538724649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 538724649 }}, "user": { "id": 2149778690, "name": "•shubbbby•", "screen_name": "poundas1", "lang": "en", "location": "probs the gym ", "create_at": date("2013-10-22"), "description": "•give blood play rugby• @paulnenoff ❤️ @llliiivvvv ❤️", "followers_count": 866, "friends_count": 473, "statues_count": 19314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385732726785, "text": "42.7F (Feels: 42.7F) - Humidity: 99% - Wind: 1.6mph S - Gust: 2.2mph - Pressure: 1033.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 225908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385762222081, "text": "Temp 24.8°F Wind Chill 24.8°F RH 91% Wind 0.7 NNW Gust 2.0 NNW SLP 29.873 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 104, "friends_count": 63, "statues_count": 33109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bluefield, WV", "id": "8057944e55790b13", "name": "Bluefield", "place_type": "city", "bounding_box": rectangle("-81.265356,37.234874 -81.165628,37.281748") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54055, "countyName": "Mercer", "cityID": 5408524, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385804054528, "text": "Bruno Mars- When I Was Your Man. LOL. the feels..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1194908840, "name": "ryan j.", "screen_name": "_spaceace_", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "null", "followers_count": 173, "friends_count": 211, "statues_count": 1295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604385866940416, "text": "The search for a last word isn’t going very well. I feel like I’ve used 10k three of coins #DestinytheGame #destinyproblems", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DestinytheGame", "destinyproblems" }}, "user": { "id": 419029556, "name": "RABID_SLOTH83", "screen_name": "RABIDSLOTH83", "lang": "en", "location": "Patterson, CA", "create_at": date("2011-11-22"), "description": "Father, Husband and Gamer. SW battlefront, fallout 4, destiny, blops3, bloodborne, rocket league. Wannabe streamer PSN RABID_SLOTH83", "followers_count": 151, "friends_count": 232, "statues_count": 31425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patterson, CA", "id": "bcb6b4eebbf9b55c", "name": "Patterson", "place_type": "city", "bounding_box": rectangle("-121.172793,37.453362 -121.098968,37.517058") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 656112, "cityName": "Patterson" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604386118627328, "text": "The American way..���������� With my shadow �� #canyon #Cavs #lebronjames #unkut #ballislife https://t.co/Hr60kSkCFC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "canyon", "Cavs", "lebronjames", "unkut", "ballislife" }}, "user": { "id": 520883187, "name": "P.A HOOP", "screen_name": "Alfoussen_94", "lang": "fr", "location": "Phoenix, AZ", "create_at": date("2012-03-10"), "description": "Tous conçu de la même façon donc pourquoi aurais je peur de l'un d'eux ! Sénégalais-Guinéen Djack's ❌ c'est bloqué ❌", "followers_count": 1353, "friends_count": 982, "statues_count": 47689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604386206814208, "text": "Wind 2.0 mph W. Barometer 29.764 in, Falling. Temperature 41.7 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604386240425984, "text": "#SupportOriginMelissa 42.8°F Wind:3.6mph Pressure: 29.71hpa Falling Rain Today 0.00in. Forecast: Occasional precipitation, worsening", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 309291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604386353680388, "text": "@chuckspjr lol no we friends", "in_reply_to_status": 696595239985786880, "in_reply_to_user": 900226958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 900226958 }}, "user": { "id": 261906305, "name": "ɹǝןןǝʞ uoʇɐǝʞ", "screen_name": "KeatonKellerxxx", "lang": "en", "location": "in a sea of bottoms ", "create_at": date("2011-03-06"), "description": "18+ aspiring porn, model, and actor. For booking please contact Shane Frost at ShaneFrostXXX@gmail.com", "followers_count": 1937, "friends_count": 1692, "statues_count": 20861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604386601136128, "text": "@Karolaaay_ date tu lugar de \"mujer\" muchacha pajua, se ve que eres rolo e puta. Y ojalá a tu hermano le hagan Bullying duro pa que te duela", "in_reply_to_status": 696603666028036096, "in_reply_to_user": 1255829642, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1255829642 }}, "user": { "id": 1007111833, "name": "Adrian.", "screen_name": "JustOtherGuy", "lang": "es", "location": "New York, NY", "create_at": date("2012-12-12"), "description": "17. Venezolano.\nMarico el que lo lea.", "followers_count": 2557, "friends_count": 2347, "statues_count": 12068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604387146268672, "text": "Holy shit I'm fucking weird", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1443001824, "name": "Jasmine ૐ ☾☼☽ ૐ", "screen_name": "M00NXSUN", "lang": "en", "location": "From A Basement On The Hill ", "create_at": date("2013-05-19"), "description": "A long time ago in a galaxy far, far away....", "followers_count": 820, "friends_count": 1908, "statues_count": 7199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604387532115972, "text": "I can't get enough of this girl, let me tell ya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268550113, "name": "Connor", "screen_name": "connorg_30", "lang": "en", "location": "Modesto, CA", "create_at": date("2011-03-18"), "description": "success isn't final, failure isn't fatal. 18.", "followers_count": 602, "friends_count": 439, "statues_count": 9832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604387779616768, "text": "HAPPY NEW YEAR BITCHES ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1582624866, "name": "g-lion", "screen_name": "_jayllian", "lang": "en", "location": "Area 6 • D5170", "create_at": date("2013-07-10"), "description": "prose before hoes // avid sinner // honorary phd in the art of google spreadsheets", "followers_count": 313, "friends_count": 293, "statues_count": 12574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:02.000Z"), "id": 696604387817304064, "text": "https://t.co/mRKOxWJkX7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2837453646, "name": "cami", "screen_name": "Camidaedric", "lang": "en", "location": "Bay Area", "create_at": date("2014-10-01"), "description": "must be nice / psn:WrekCityCami / @dvntepennell and I are in a clan.", "followers_count": 695, "friends_count": 804, "statues_count": 7974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604388622733314, "text": "Barberton Oh Temp:39.8°F Wind:0 mph Dir:S Baro:Falling Rain2day:0.00in Hum:52% UV:0.0 @ 03:00 02/08/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 232, "friends_count": 228, "statues_count": 114490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389323227138, "text": "@esurance #EsuranceSweepstakes big bucks for big dreams", "in_reply_to_status": -1, "in_reply_to_user": 14388488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EsuranceSweepstakes" }}, "user_mentions": {{ 14388488 }}, "user": { "id": 43758092, "name": "Diana Foster", "screen_name": "ddmobile", "lang": "en", "location": "null", "create_at": date("2009-05-31"), "description": "null", "followers_count": 163, "friends_count": 1518, "statues_count": 4608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389394362368, "text": "@ThatBoyVictor that's you? Lol", "in_reply_to_status": -1, "in_reply_to_user": 285296007, "favorite_count": 0, "coordinate": point("-122.23149679,37.79075538"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 285296007 }}, "user": { "id": 1344914365, "name": "Kaitlyn", "screen_name": "Kaitlynsaephan", "lang": "en", "location": "null", "create_at": date("2013-04-11"), "description": "Nothing that's worth it is never easy", "followers_count": 240, "friends_count": 180, "statues_count": 2763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389721571328, "text": "@DreaLove92 in San Pedro square?! I'm sick my old boss owns it I could easily get in", "in_reply_to_status": 696603934631157760, "in_reply_to_user": 385148977, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 385148977 }}, "user": { "id": 1002736226, "name": "PuppyMonkeyBaby", "screen_name": "ThatDudeMB", "lang": "en", "location": "HELLAFORNIA ", "create_at": date("2012-12-10"), "description": "Welcome to the inner workings of my mind IG:ThatDudeMB", "followers_count": 324, "friends_count": 280, "statues_count": 25003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389725724673, "text": "LIT AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2899066088, "name": "Damian Rios", "screen_name": "Damian_xvx", "lang": "en", "location": "Oregon, USA", "create_at": date("2014-11-13"), "description": "Mck hoops | XC | T&F |", "followers_count": 346, "friends_count": 179, "statues_count": 3096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayesville, OR", "id": "ef6bc9d58ea0c80b", "name": "Hayesville", "place_type": "city", "bounding_box": rectangle("-122.992994,44.958825 -122.948448,45.01161") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4132850, "cityName": "Hayesville" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389780430849, "text": "Wind 3.1 mph N. Barometer 29.94 in, Falling. Temperature 27.0 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 107, "statues_count": 157333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604389855723520, "text": "@NeidelynG I wonder who she thinks is helllllla cute lmao ��", "in_reply_to_status": 696604076675432448, "in_reply_to_user": 2929171970, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2929171970 }}, "user": { "id": 2402489814, "name": "Marlene Lopez", "screen_name": "marleneyo_", "lang": "en", "location": "null", "create_at": date("2014-03-21"), "description": "⚽️⚽️⚽️⚽️⚽️⚽️", "followers_count": 287, "friends_count": 416, "statues_count": 3048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604390245838848, "text": "Disabled vehicle, shoulder blocked in #Cobb on I 75 NB at Canton Rd. Conn #ATLTraffic https://t.co/bABElTEvus", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.54186,33.98445"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cobb", "ATLTraffic" }}, "user": { "id": 156618978, "name": "TTWN ATL", "screen_name": "TotalTrafficATL", "lang": "en", "location": "Atlanta, GA", "create_at": date("2010-06-17"), "description": "To report an accident or delay, please call the Total Traffic Tipline: 404-352-4331", "followers_count": 482, "friends_count": 67, "statues_count": 52632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marietta, GA", "id": "e229de11a7eb6823", "name": "Marietta", "place_type": "city", "bounding_box": rectangle("-84.596805,33.895088 -84.46746,34.001159") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1349756, "cityName": "Marietta" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604390610763776, "text": "My head feels tangled up like the headphones in my pocket", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272723472, "name": "juanter", "screen_name": "MendiolaHunter", "lang": "en", "location": "null", "create_at": date("2014-01-02"), "description": "it is what it is", "followers_count": 214, "friends_count": 195, "statues_count": 2154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipio Acres, HI", "id": "44cec4558c57418e", "name": "Waipio Acres", "place_type": "city", "bounding_box": rectangle("-158.028507,21.458343 -158.00428,21.489971") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1580000, "cityName": "Waipio Acres" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604390858174465, "text": "Camping, beer. Concerts, beer. Lake, beer. Racetrack, beer. Summertime.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542181918, "name": "Steven Spears", "screen_name": "stevenspears96", "lang": "en", "location": "Yuba City, CA ➡ Sacramento, CA", "create_at": date("2012-03-31"), "description": "Sac State 2019. Kansas City Chiefs. Huge racing fan.", "followers_count": 506, "friends_count": 366, "statues_count": 17432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604391189536768, "text": "Thought I fixed my twit so that I won't have to see shitty memes. So why tf is there shitty memes on my twit https://t.co/xz5K0MbfRE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1724586588, "name": "little boy's room", "screen_name": "stankillspeople", "lang": "en", "location": "null", "create_at": date("2013-09-02"), "description": "yo this mail order bride better be stupid thick", "followers_count": 238, "friends_count": 368, "statues_count": 2387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richland, WA", "id": "0dd0c9c93b5519e1", "name": "Richland", "place_type": "city", "bounding_box": rectangle("-119.348075,46.164988 -119.211248,46.351367") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5358235, "cityName": "Richland" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604391604776960, "text": "@PCaldora definitely curious if Sony would strike a deal with Activision or something for the license", "in_reply_to_status": 696604118543110144, "in_reply_to_user": 952122043, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 952122043 }}, "user": { "id": 15181936, "name": "Chris Cookson", "screen_name": "coldmilk", "lang": "en", "location": "Stamford, CT", "create_at": date("2008-06-20"), "description": "Animator, pop-culture junky and movie buff. @PrattInstitute alumni. I've worked on commercials, @UglyAmericans and @TheOnion. In my past life, I was a unicorn…", "followers_count": 2129, "friends_count": 900, "statues_count": 64790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Venice, Los Angeles", "id": "5c183bdb0ab780f2", "name": "Venice", "place_type": "neighborhood", "bounding_box": rectangle("-118.478388,33.98355 -118.440173,34.009879") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604391701237760, "text": "Since everyone else is doing it https://t.co/pK6rkzZ7nU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2246118050, "name": "DG", "screen_name": "darrian_garcia", "lang": "en", "location": "FDB", "create_at": date("2013-12-14"), "description": "How I feel", "followers_count": 625, "friends_count": 428, "statues_count": 30805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604391860674560, "text": "@bronwynnrosee @lovelightlucid over what", "in_reply_to_status": 696602397829804032, "in_reply_to_user": 3507908480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3507908480, 178060721 }}, "user": { "id": 3312049460, "name": "Güero", "screen_name": "FinalxDoctrine", "lang": "en", "location": "951✈️702", "create_at": date("2015-08-10"), "description": "Thanks for the shoutout", "followers_count": 796, "friends_count": 560, "statues_count": 19175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-08T00:00:03.000Z"), "id": 696604391986561024, "text": "Don't care what anyone says, I'll always have room for you in my heart ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1431968810, "name": "Dirty", "screen_name": "lildmoney23", "lang": "en", "location": "Beckley, Wv", "create_at": date("2013-05-15"), "description": "I have no heart cause its been broken one too many times.", "followers_count": 88, "friends_count": 252, "statues_count": 1139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beckley, WV", "id": "00b233c004a70975", "name": "Beckley", "place_type": "city", "bounding_box": rectangle("-81.220519,37.757477 -81.141871,37.821814") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54081, "countyName": "Raleigh", "cityID": 5405332, "cityName": "Beckley" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604392104009728, "text": "Kraken https://t.co/QuNeSgRbz2 #kraken #parcdattraction #orlando #florida #unitedstates https://t.co/S6dJ9dfqGs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.45829,28.41143"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "hashtags": {{ "kraken", "parcdattraction", "orlando", "florida", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 744, "friends_count": 1154, "statues_count": 2725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1277735, "cityName": "Williamsburg" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604392334737408, "text": "Family Shit Y.��.〽️.G fa Life‼️���� https://t.co/TCsAP8DRpm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4222907836, "name": "Pmoney_StunnaMukiz", "screen_name": "PmoneyDaStunna", "lang": "en", "location": "null", "create_at": date("2015-11-18"), "description": "Youngstunnaz The Label Whole Squad On Da Grind to Taking Off N SIP My Brother Chris U Who We Do It For Direct Message Me Fa Booking and Features FREE Da Squad", "followers_count": 141, "friends_count": 106, "statues_count": 2756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604393194426368, "text": "03:00 37.2°F Feels:30.5°F (Hi40.6°F/Lo37.2°F) Hum:75% Wnd:NW 9.3MPH Baro:29.46in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 179, "friends_count": 266, "statues_count": 25470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604393542561792, "text": "Checked someone tonight at work for referring to the performance as 'ghetto' lmfaoooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44897078, "name": "Natrill", "screen_name": "NatAnglin", "lang": "en", "location": "BosAngeles", "create_at": date("2009-06-05"), "description": "'While ya sleeping on us? We making dreams come true.' Rap Artist. Writer. Actor. For Business/Music Inquiries: @JoeHovasMF & @CynSaytion", "followers_count": 2266, "friends_count": 868, "statues_count": 128617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604393739685888, "text": "Happy birthday!!! I hope today is filled with nothing but happiness. Blessed to have you in my life. I love you ❤️ https://t.co/Lw8SadNQBw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428537897, "name": "Karen", "screen_name": "kkaarreenn345", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": "null", "followers_count": 345, "friends_count": 271, "statues_count": 9873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604394960265217, "text": "The game is to be sold... Nottttttttt to be told ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 452641928, "name": "Ray", "screen_name": "oveRAYted", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "Not afraid to be different", "followers_count": 151, "friends_count": 144, "statues_count": 5629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604395392217088, "text": "@FlattestComb huge pet peeve lol", "in_reply_to_status": 696604283874050048, "in_reply_to_user": 82814459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82814459 }}, "user": { "id": 61651535, "name": "Jake from State Farm", "screen_name": "21butts", "lang": "en", "location": "Mystic Falls, VA", "create_at": date("2009-07-30"), "description": "I talk a lot of shit and I talk a lot about hair and that's pretty much it", "followers_count": 165, "friends_count": 154, "statues_count": 19488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Picnic Point, WA", "id": "007d0c773de7ec34", "name": "Picnic Point", "place_type": "city", "bounding_box": rectangle("-122.336551,47.82599 -122.245495,47.89362") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5354213, "cityName": "Picnic Point" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604395492941828, "text": "my need to rave is unsettling, I'm vapor rubbing in my room while performing a light show right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73984391, "name": "sydneyaustralia", "screen_name": "Sydney_Holland", "lang": "en", "location": "515", "create_at": date("2009-09-13"), "description": "i swear i have friends, probably eating", "followers_count": 706, "friends_count": 544, "statues_count": 18546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-08T00:00:04.000Z"), "id": 696604396159782916, "text": "@cris_swizzle lmao I wasn't really looking for an answer tho ��", "in_reply_to_status": 696599572894736384, "in_reply_to_user": 301126138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301126138 }}, "user": { "id": 159293937, "name": "jm", "screen_name": "jammmsterzz", "lang": "en", "location": "Los Angels ", "create_at": date("2010-06-24"), "description": "twenty", "followers_count": 396, "friends_count": 240, "statues_count": 52564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604396893900806, "text": "@RickyPDillon @YouTube I love your videos and I get so excited and anxious for each one #followmeplz", "in_reply_to_status": 696393862026113024, "in_reply_to_user": 73171449, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "followmeplz" }}, "user_mentions": {{ 73171449, 10228272 }}, "user": { "id": 29733512, "name": "Queen NeNe", "screen_name": "RoyalToddlerNae", "lang": "en", "location": "Next to Toddy's Quingdom", "create_at": date("2009-04-08"), "description": "The name is Renee, by the way lol\n23 years old\nTune in on Monday Nights to see #TodrickMTV \nLiterally Todrick Hall, Disney, and YouTube are my whole life lol", "followers_count": 141, "friends_count": 236, "statues_count": 3535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604396935712768, "text": "hit up soundcloud for all the bedtime jams", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1285197218, "name": "hanananah", "screen_name": "HannahMundine", "lang": "en", "location": "null", "create_at": date("2013-03-20"), "description": "just your average independent black girl loving the lord", "followers_count": 611, "friends_count": 572, "statues_count": 15999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604396982005760, "text": "Can you find Cheyenne on the map? Just try it at https://t.co/goxm7EE3Dw #Cheyenne", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.82,41.14"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cheyenne" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1040, "friends_count": 311, "statues_count": 2579355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheyenne, WY", "id": "75c9243440a46116", "name": "Cheyenne", "place_type": "city", "bounding_box": rectangle("-104.860909,41.078217 -104.713275,41.19235") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56021, "countyName": "Laramie", "cityID": 5613900, "cityName": "Cheyenne" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604397049094144, "text": "@HolisticFacts \nThis explains C Breeze behavior most of the time.", "in_reply_to_status": 696574986878738432, "in_reply_to_user": 733133306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 733133306 }}, "user": { "id": 3604025542, "name": "Daisy", "screen_name": "daisy2b4", "lang": "en", "location": "null", "create_at": date("2015-09-09"), "description": "Daisy born and raised in Washington DC.", "followers_count": 249, "friends_count": 601, "statues_count": 1017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Capitol Heights, MD", "id": "01aa5bb33f7bf240", "name": "Capitol Heights", "place_type": "city", "bounding_box": rectangle("-76.921756,38.86287 -76.891608,38.893952") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2413000, "cityName": "Capitol Heights" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604397204344832, "text": "Make my name taste like ass when you speak it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223727518, "name": "johnny p", "screen_name": "NattyAnAFatty69", "lang": "en", "location": "Greenville, SC", "create_at": date("2010-12-06"), "description": "Σ(゚д゚lll)", "followers_count": 276, "friends_count": 197, "statues_count": 16150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylors, SC", "id": "0eec6a9635525944", "name": "Taylors", "place_type": "city", "bounding_box": rectangle("-82.396441,34.881534 -82.26651,34.996401") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4571395, "cityName": "Taylors" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604397288095744, "text": "@ShizukaSam hmm maybe with a blazer? You know, cuz work.", "in_reply_to_status": 696604262760054784, "in_reply_to_user": 18020516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18020516 }}, "user": { "id": 130949596, "name": "Yessica Yones", "screen_name": "YessPlz", "lang": "en", "location": "null", "create_at": date("2010-04-08"), "description": "opportunist above all else. oh yea, and i illustrate. http://yessplz.tumblr.com", "followers_count": 364, "friends_count": 328, "statues_count": 63426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604397724303360, "text": "@Burberry @RobLowe #MonkeySuit You can do better than this, Rob.", "in_reply_to_status": 696393070066192385, "in_reply_to_user": 47459700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MonkeySuit" }}, "user_mentions": {{ 47459700, 121626258 }}, "user": { "id": 35888387, "name": "David Garrison", "screen_name": "dlgarrison", "lang": "en", "location": "Peoples Republic of California", "create_at": date("2009-04-27"), "description": "Constitutional Reagan Conservative. #TRUMP Make America Great Again! #SAR #TCOT #PJNET", "followers_count": 2599, "friends_count": 2267, "statues_count": 29185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604398458277888, "text": "They be goin at it in celebrity comments", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3014562006, "name": "sliiiiiiime", "screen_name": "Rinzel_", "lang": "en", "location": "Texas, USA", "create_at": date("2015-02-09"), "description": "find me bickin back boolin at the baketball game", "followers_count": 405, "friends_count": 146, "statues_count": 2350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604398978359297, "text": "Silver wear smells like fish", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146353559, "name": "John Azemar", "screen_name": "JohnAzemar", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-05-20"), "description": "new year, new me. idc what day it is.", "followers_count": 140, "friends_count": 111, "statues_count": 709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-08T00:00:05.000Z"), "id": 696604399175532545, "text": "@morningwoodmatt @Fershitt @sevaneshain oh, don't worry, im not", "in_reply_to_status": 696604156992204801, "in_reply_to_user": 4652141533, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4652141533, 3016081279, 2797004612 }}, "user": { "id": 2579274286, "name": "neta", "screen_name": "netahazan", "lang": "en", "location": "null", "create_at": date("2014-06-02"), "description": "views...", "followers_count": 513, "friends_count": 279, "statues_count": 20643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604400735928320, "text": "lmao people so wishy washy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354484818, "name": "❣", "screen_name": "armskyrs_", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-08-13"), "description": "null", "followers_count": 2347, "friends_count": 474, "statues_count": 97921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Rouge, MI", "id": "e7cdcd61b0c07e97", "name": "River Rouge", "place_type": "city", "bounding_box": rectangle("-83.153043,42.258976 -83.103286,42.290022") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2668760, "cityName": "River Rouge" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401121652736, "text": "HAPPY BIRTHDAY I LOVE YOU @druggedcameron", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2822715728 }}, "user": { "id": 2424335474, "name": ":( tomorrow", "screen_name": "ASSHOLEOLZANSKI", "lang": "en", "location": "whitesides", "create_at": date("2014-04-02"), "description": "Allison & Allison ft. Jack & Jack 1 day https://twitter.com/druggedcameron/status/664238076336586753", "followers_count": 4580, "friends_count": 309, "statues_count": 79441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401230680065, "text": "I wanna go home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2745538424, "name": "Mo", "screen_name": "moniqueelamm", "lang": "en", "location": "null", "create_at": date("2014-08-19"), "description": "ig: moniqueelamm", "followers_count": 80, "friends_count": 84, "statues_count": 288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chinatown, Los Angeles", "id": "6bd6390a5a6b12ef", "name": "Chinatown", "place_type": "neighborhood", "bounding_box": rectangle("-118.248897,34.054253 -118.235679,34.067822") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401511759873, "text": "����, you're attractive, we don't talk much, Lowkey if you finished the apush homework hml ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1144447692, "name": "David", "screen_name": "Dsilva_001", "lang": "en", "location": "probs doing HW or at the field", "create_at": date("2013-02-02"), "description": "OMWU", "followers_count": 553, "friends_count": 303, "statues_count": 26428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401587204096, "text": "\"And when I'm gone whole crew singing swan songs\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336522207, "name": "J.J. From JCP", "screen_name": "JohnnyBrizendin", "lang": "en", "location": "Edmond, OK", "create_at": date("2011-07-16"), "description": "God is #1. UCO ATΩ. Aspiring News reporter. Romans 1:16 St. Louis Cardinals. 20. Roll Cho's", "followers_count": 514, "friends_count": 408, "statues_count": 3649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401603989504, "text": "Nah but Ima go to sleep I have to be productive tomorrow lmfao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368104323, "name": "jay", "screen_name": "__JayFlores", "lang": "en", "location": "Houston, TX", "create_at": date("2011-09-04"), "description": "|FΔΜ| snapchat:yungjong|(◕,,,◕)", "followers_count": 622, "friends_count": 412, "statues_count": 84613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401713094656, "text": "I HATE when people screenshot my snaps. Like, wtf? I trusted you w/these ugly ass faces.. How you gon do me like Tha?����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300670241, "name": "Melijuana", "screen_name": "MD0llaz__", "lang": "en", "location": "Bay Native", "create_at": date("2014-01-19"), "description": "VIIXXVXXII☪.Nineteen. I'm jus Vibin' thru life✨", "followers_count": 281, "friends_count": 399, "statues_count": 6098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604401713205248, "text": "Wind 5.0 mph NE. Barometer 29.733 in, Falling Rapidly. Temperature 38.3F. Rain today 0.00 in. Humidity 72% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 595, "friends_count": 797, "statues_count": 36836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604402187042816, "text": "@2DZbeats come to my school in Hollywood we got a bunch of recording studio", "in_reply_to_status": 696601427947327490, "in_reply_to_user": 779171695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 779171695 }}, "user": { "id": 129289098, "name": "JWrath", "screen_name": "JWRATH_LTME", "lang": "en", "location": "North Hollywood, Los Angeles", "create_at": date("2010-04-03"), "description": "Music Artist/ Audio Engineer\n#SODMG #LTME", "followers_count": 575, "friends_count": 517, "statues_count": 24956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604403197816832, "text": "Now I'm going to bed! Good night everyone! ������✌", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104026426, "name": "Yasmania Carter", "screen_name": "Yazzy_Carter", "lang": "en", "location": "Oklahoma City, Oklahoma", "create_at": date("2010-01-11"), "description": "#Capricorn #Taken #FollowsBack #Polite #Goofy #Instagram @yazzy_carter", "followers_count": 143, "friends_count": 202, "statues_count": 1476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604403407593473, "text": "Ara and Gaby are knocked the fuck out and I'm still wide awake ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446725077, "name": "alejandra ✨", "screen_name": "alee_ehmm", "lang": "en", "location": "Fullerton, CA", "create_at": date("2011-12-25"), "description": "bff: @castroxo_", "followers_count": 191, "friends_count": 179, "statues_count": 5278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604403541757957, "text": "my eyes are so cloudy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3259539140, "name": "raynee", "screen_name": "rhaysrmjj", "lang": "en", "location": "null", "create_at": date("2015-06-28"), "description": ":)", "followers_count": 405, "friends_count": 720, "statues_count": 5104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dublin, CA", "id": "003253f0107acd32", "name": "Dublin", "place_type": "city", "bounding_box": rectangle("-121.979522,37.697406 -121.849535,37.733656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 620018, "cityName": "Dublin" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604403759841280, "text": "Wind 0 mph NE. Barometer 30.14 in, Falling very rapidly. Temperature 41.9 °F. Rain today 0.00 in. Humidity 51%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.95555556,31.9575"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20277942, "name": "Steven Elwood", "screen_name": "akasonny", "lang": "en", "location": "Sahuarita, AZ", "create_at": date("2009-02-06"), "description": "null", "followers_count": 15, "friends_count": 88, "statues_count": 474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arizona, USA", "id": "a612c69b44b2e5da", "name": "Arizona", "place_type": "admin", "bounding_box": rectangle("-114.818269,31.332246 -109.045153,37.004261") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 462140, "cityName": "Sahuarita" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604404141727745, "text": "Wind 0 mph ---. Barometer 1010.7 hPa, Falling slowly. Temperature 71.8 °F. Rain today 0.00 in. Humidity 41%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 265, "statues_count": 144808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604404166713344, "text": "Don't fall in love with a nigga like me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 225724624, "name": "ANTAWAYNE", "screen_name": "ambeezymayne", "lang": "en", "location": "$crewston ", "create_at": date("2010-12-12"), "description": "we on a money mission! #ripANT KT 12/1", "followers_count": 5882, "friends_count": 1423, "statues_count": 142611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-08T00:00:06.000Z"), "id": 696604404321914880, "text": "���� https://t.co/A0tzXY6JHF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 165465233, "name": "J.0.S.E.T.T.E.", "screen_name": "ThatGirlJosette", "lang": "en", "location": "760 ✈ 702", "create_at": date("2010-07-11"), "description": "MCMXCII☀️California Girl. UNLV Alumna//Young Rebel. Sports & Entertainment PR/Marketing. Sushi Lover.", "followers_count": 462, "friends_count": 376, "statues_count": 38876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604405118816257, "text": "Temp: 30.4°F - Dew Point: 26.2° - Wind: 8.0 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 29.59in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 11, "statues_count": 13162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604405320323072, "text": "Somebody FaceTime me so I won't go to sleep at work��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2193617336, "name": "Henny Smith", "screen_name": "Kornbread__", "lang": "en", "location": "Houston, TX", "create_at": date("2013-11-13"), "description": "Rest In Peace Teagan, Daddy Loves You 1.30.2016 | 21| just a simple man trying to live", "followers_count": 489, "friends_count": 430, "statues_count": 20713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604405420806144, "text": "Wind 0.0 mph NNE. Barometer 30.270 in, Steady. Temperature 54.2 °F. Rain today 0.00 in. Humidity 36%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 249588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604405530025985, "text": "See i always be thibkin i found the one... But nahhhh smh @ebonessouL", "in_reply_to_status": 696603666581745664, "in_reply_to_user": 74269192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74269192 }}, "user": { "id": 18625001, "name": "Sankofa NYC", "screen_name": "Coldsun3000", "lang": "en", "location": "The Bronx, NYC", "create_at": date("2009-01-04"), "description": "I study, I educate, I mentor, I travel, I eat, I drink, I do NYC", "followers_count": 713, "friends_count": 523, "statues_count": 45911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604405726998528, "text": "I told my boyfriend to call me so I could hear his voice before I go to bed and guess what he didn't do��.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1965943572, "name": "Fran", "screen_name": "GEORGlAN", "lang": "en", "location": "sf | lb", "create_at": date("2013-10-16"), "description": "jacob", "followers_count": 158, "friends_count": 149, "statues_count": 8991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-08T00:00:07.000Z"), "id": 696604406196912128, "text": "02/08/16, 02:00 am - Temp/Dewpoint: 31.3F/28.7F. Humidity 90%. Pressure 29.596 in/Hg and Falling. Wind SW at 0.0 mph, gusting to 5.0 mph.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.94722222,43.36777778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442475015, "name": "Tim Gonyo", "screen_name": "SaukvilleWX", "lang": "en", "location": "Saukville, WI", "create_at": date("2011-12-20"), "description": "Weather enthusiast. Enough said :)", "followers_count": 12, "friends_count": 7, "statues_count": 7340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grafton, WI", "id": "26922abb216a7b6c", "name": "Grafton", "place_type": "city", "bounding_box": rectangle("-87.980941,43.279784 -87.901263,43.36854") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55089, "countyName": "Ozaukee", "cityID": 5530000, "cityName": "Grafton" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966763213246464, "text": "@aly__issa I THOUGHT I DID ������", "in_reply_to_status": 696902724617531392, "in_reply_to_user": 3100256268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3100256268 }}, "user": { "id": 3277835090, "name": "jewelsss", "screen_name": "jeweliapatzer", "lang": "en", "location": "null", "create_at": date("2015-07-12"), "description": "fuck it ?", "followers_count": 134, "friends_count": 358, "statues_count": 1591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966764555583488, "text": "Ahhh home...kitteh cuddling (@ BarefootKnitter's Abode in Ballston Lake, NY) https://t.co/dvphxsHzfj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.94186034,42.90521657"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141736788, "name": "Sugar Lips", "screen_name": "BarefootKnitter", "lang": "en", "location": "null", "create_at": date("2010-05-08"), "description": "Licensed Massage Therapist, Knitter of hand knit goodness, Hippie", "followers_count": 222, "friends_count": 476, "statues_count": 14643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucker Heights, NY", "id": "01af337f76d91f0b", "name": "Tucker Heights", "place_type": "city", "bounding_box": rectangle("-73.970003,42.899553 -73.927182,42.93234") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966764672974848, "text": "Multnomah Falls. Another site on my epic, 10 hour adventure today. The magnitude of the… https://t.co/bgxTH462wn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.11536,45.57595"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549031940, "name": "AlexandraGodwin IMUA", "screen_name": "alexandraimua", "lang": "en", "location": "Scotland, United Kingdom", "create_at": date("2012-04-09"), "description": "✨Egalitarian✨CrimethInc✨ ✨UrbEx✨Wanderer✨Toby✨MUA on a career hiatus✨Currently travelling around west coast USA!", "followers_count": 1236, "friends_count": 112, "statues_count": 9982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon, USA", "id": "df7fd3a3b9eff7ee", "name": "Oregon", "place_type": "admin", "bounding_box": rectangle("-124.703541,41.991795 -116.463262,46.299078") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966766090657792, "text": "Wind 5.6 mph WNW. Barometer 30.01 in, Steady. Temperature 35.1 °F. Rain today 0.00 in. Humidity 40%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966766275133440, "text": "@BP2Play Delighted beyond words to know that so many children in need will benefit ;)", "in_reply_to_status": 696747899569500160, "in_reply_to_user": 3015840450, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3015840450 }}, "user": { "id": 4846554547, "name": "Michael Victor", "screen_name": "mvictor2016", "lang": "en", "location": "San Francisco, CA", "create_at": date("2016-02-01"), "description": "Blessed are the meek, for they shall inherit the Earth. \n\nWhat do I do? I help the helpless.", "followers_count": 2, "friends_count": 36, "statues_count": 117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966766447058944, "text": "Wind 5.0 mph NW. Barometer 1024.42 mb, Falling. Temperature 31.4 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 26, "friends_count": 124, "statues_count": 12402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966766484848640, "text": "@AmbitiousAleyah greys it is.", "in_reply_to_status": 696965258322182144, "in_reply_to_user": 112610448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 112610448 }}, "user": { "id": 112610448, "name": "princess A ✨", "screen_name": "AmbitiousAleyah", "lang": "en", "location": "RIP Sophia Marie, 08/18/15 ❤️", "create_at": date("2010-02-08"), "description": "don't cha wish ya girlfriend was hot like me? NAU'19.", "followers_count": 1996, "friends_count": 232, "statues_count": 70099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-09T00:00:00.000Z"), "id": 696966766665203712, "text": "@Aliciaa_11 we care *", "in_reply_to_status": 696963904476303360, "in_reply_to_user": 2242530147, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2242530147 }}, "user": { "id": 2334354075, "name": "Deejayy", "screen_name": "tsb_deezzy", "lang": "en", "location": "10toes deep", "create_at": date("2014-02-09"), "description": "trust ? what's that |#TxSU", "followers_count": 1007, "friends_count": 847, "statues_count": 12567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966767395147777, "text": "*screenshots proof", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4701496217, "name": "Norma", "screen_name": "normabarroyo", "lang": "en", "location": "Illinois", "create_at": date("2016-01-03"), "description": "She's so precious", "followers_count": 238, "friends_count": 220, "statues_count": 55 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenview, IL", "id": "ed888f00de07aa3a", "name": "Glenview", "place_type": "city", "bounding_box": rectangle("-87.894012,42.054687 -87.757958,42.11517") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1729938, "cityName": "Glenview" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966767541784576, "text": "kidz bop bring me to life is killing me i can't breathe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270172374, "name": "grace lauren", "screen_name": "pacificspace", "lang": "en", "location": "the internet, washington", "create_at": date("2011-03-21"), "description": "do androids dream of electric memes? || @viricrespo is my pretty bird || mod for various streamers || co-founder of @afterstreamers || [they/their pronouns]", "followers_count": 583, "friends_count": 294, "statues_count": 20042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966768359665664, "text": "Duets For One by Elton John is #nowplaying in Thumb Coast Brewing Co., Port Huron. Download it now at https://t.co/IoDOrDiaDu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.422367,42.975489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox", "screen_name": "VirtualJukebox", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1381, "friends_count": 0, "statues_count": 1433570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Huron, MI", "id": "010750c237347ebb", "name": "Port Huron", "place_type": "city", "bounding_box": rectangle("-82.514554,42.935388 -82.418396,43.040084") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2665820, "cityName": "Port Huron" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966768800059392, "text": "@ThatDudeeEmilio https://t.co/oR7xfWCZXs", "in_reply_to_status": 696966659588755456, "in_reply_to_user": 1319626158, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1319626158 }}, "user": { "id": 509809912, "name": "antonio", "screen_name": "steezelo", "lang": "en", "location": "Fresno, CA", "create_at": date("2012-02-29"), "description": "HHS Cheer", "followers_count": 1669, "friends_count": 837, "statues_count": 41039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966769018142724, "text": "Happy Birthday Bunk ass pussy ass nigga @exxtra_virgin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2434918584 }}, "user": { "id": 3083870792, "name": "BadLuckCee", "screen_name": "ceesirr", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Bloomington Ca", "followers_count": 304, "friends_count": 272, "statues_count": 14473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966769118818304, "text": "@oscar_saldana (rolling my eyes emoji again) love you too��", "in_reply_to_status": 696965094043848704, "in_reply_to_user": 246094310, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246094310 }}, "user": { "id": 190155821, "name": "Kass", "screen_name": "KassySevilla", "lang": "en", "location": "null", "create_at": date("2010-09-12"), "description": "TwentyTwo.✨", "followers_count": 256, "friends_count": 202, "statues_count": 15378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, CA", "id": "e6aaf7cb58e7857c", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-115.594963,32.817908 -115.544378,32.876462") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 636280, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966769118941184, "text": "Wind 2.0 mph NW. Barometer 29.952 in, Steady. Temperature 20.2 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966769588596736, "text": "I don't give a fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335186565, "name": "Weedo69 theBlue", "screen_name": "Weedo69", "lang": "en", "location": "Kerman/Davis, California ", "create_at": date("2011-07-14"), "description": "it just continues...", "followers_count": 205, "friends_count": 158, "statues_count": 15167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davis, CA", "id": "1994142e26ba7127", "name": "Davis", "place_type": "city", "bounding_box": rectangle("-121.803252,38.526843 -121.675074,38.590264") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 618100, "cityName": "Davis" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966769668460544, "text": "@Goofyassliyah lol keep looking out the window that snow will fall soon", "in_reply_to_status": 696966326535852032, "in_reply_to_user": 2996210533, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2996210533 }}, "user": { "id": 64891503, "name": "⚖", "screen_name": "__MIGOOO", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "18 ⚖ $ingle ✨", "followers_count": 1514, "friends_count": 929, "statues_count": 51007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlow Heights, MD", "id": "b9605021d2448da6", "name": "Marlow Heights", "place_type": "city", "bounding_box": rectangle("-76.97051,38.808758 -76.927375,38.837602") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2450750, "cityName": "Marlow Heights" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966770272423936, "text": "@sah1700 cause DMs open, she gonna knock you down ������", "in_reply_to_status": 696966442051334144, "in_reply_to_user": 4310235918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4310235918 }}, "user": { "id": 468895869, "name": "20 Days Left Bishhhh", "screen_name": "Trend_SetterNa", "lang": "en", "location": "Reporting Live from the Gutter", "create_at": date("2012-01-19"), "description": "#RipMu", "followers_count": 837, "friends_count": 709, "statues_count": 42735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966770402283520, "text": "It's weird how open I am with Nate ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4552933393, "name": "Erica✨❣", "screen_name": "dee_ericaa", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "hit ya hoe, hit ya hoe then I dip", "followers_count": 433, "friends_count": 736, "statues_count": 8757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valinda, CA", "id": "b1bbf8e2ef22573a", "name": "Valinda", "place_type": "city", "bounding_box": rectangle("-117.94891,34.024734 -117.912772,34.053145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 681638, "cityName": "Valinda" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966770607849472, "text": "@kanyewest WAVES", "in_reply_to_status": 696942505678524418, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 1007029705, "name": "Jessica Turner", "screen_name": "JessTurner1997", "lang": "en", "location": "DTX", "create_at": date("2012-12-12"), "description": "hi um im jessica and my application for mindy kalings best friend has been submitted •adorkable•• photography••SKAAA16•", "followers_count": 198, "friends_count": 190, "statues_count": 1809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-09T00:00:01.000Z"), "id": 696966771056705541, "text": "My stomach hurts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579336425, "name": "Bria", "screen_name": "BeeDotElle", "lang": "en", "location": "Out Enjoying Life", "create_at": date("2012-05-13"), "description": "Watch Over Us Tobs and Chocolate #RestEasyMyLoves", "followers_count": 891, "friends_count": 606, "statues_count": 25996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966771543117824, "text": "@ebbtideapp Tide in Kamalo Harbor, Hawaii 02/09/2016\nHigh 4:10am 2.3\n Low 11:16am -0.1\nHigh 4:27pm 1.2\n Low 10:26pm -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-156.8833,21.05"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 25, "friends_count": 1, "statues_count": 6194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaii, USA", "id": "9dafd05b1158873b", "name": "Hawaii", "place_type": "admin", "bounding_box": rectangle("-178.443593,18.86546 -154.755792,28.517269") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966771601833984, "text": "Listening to Mayday Parade...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35359127, "name": "Chrissy Guerrero", "screen_name": "ItsChrissyG_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-25"), "description": "Actress. Thats all you need to know.", "followers_count": 238, "friends_count": 207, "statues_count": 4596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966771677532160, "text": "Easy TOP 5...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 584936577, "name": "#PeytonManning", "screen_name": "DeGamarri", "lang": "en", "location": "null", "create_at": date("2012-05-19"), "description": "null", "followers_count": 833, "friends_count": 660, "statues_count": 12877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cochran, GA", "id": "0dca11d20228aab0", "name": "Cochran", "place_type": "city", "bounding_box": rectangle("-83.371206,32.363755 -83.3044,32.420093") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13023, "countyName": "Bleckley", "cityID": 1317328, "cityName": "Cochran" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966771748671488, "text": "Cant even close my eyes without seeing his face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455257476, "name": "Jungle_415", "screen_name": "lbubba34", "lang": "en", "location": "null", "create_at": date("2012-01-04"), "description": "Jungle Marin City.. The Bay.. New Iberia LA. Chico State Always smoking enjoying life mobbing Family is everything Insta:thehubba35 Snapchat:Thebubba95", "followers_count": 345, "friends_count": 471, "statues_count": 10409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966771836874752, "text": "Wind 2.0 mph W. Barometer 29.805 in, Falling slowly. Temperature 20.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966772008873984, "text": "@LautiLegui el de tu culo y mi ano", "in_reply_to_status": 696966676458442752, "in_reply_to_user": 1578404370, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1578404370 }}, "user": { "id": 2169510596, "name": "Chinoo/+10", "screen_name": "FrancoVernava10", "lang": "es", "location": "Lanùs Este ", "create_at": date("2013-11-01"), "description": "13/12/09/. Géminis. 14 años.Socio y enfermo del Club Atlético Banfield. Fanático del Fútbol ⚽", "followers_count": 500, "friends_count": 473, "statues_count": 15053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966772055011328, "text": "03:00 19.9°F Feels:8.2°F (Hi20.7°F/Lo19.9°F) Hum:69% Wnd:W 11.0MPH Baro:29.52in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 176, "friends_count": 266, "statues_count": 25500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773065658368, "text": "Icicles ❄️ https://t.co/woBE6xDgt9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 17591037, "name": "Matthew Anthony", "screen_name": "Matty_Crisalli", "lang": "en", "location": "Colorado ", "create_at": date("2008-11-24"), "description": "☕️", "followers_count": 234, "friends_count": 114, "statues_count": 5400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773455716353, "text": "Just got off and I start at 8 in the morning hehe ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544155852, "name": "Luciano", "screen_name": "lucipls", "lang": "en", "location": "Shmop City, CA", "create_at": date("2012-04-03"), "description": "Life is Too $hort", "followers_count": 473, "friends_count": 140, "statues_count": 62051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773459976192, "text": "Yeah I think that nigga gay Forsure", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 859283431, "name": "Anthony Jung :.", "screen_name": "Canelohhh", "lang": "en", "location": "null", "create_at": date("2012-10-02"), "description": "Sometimes you're flush and sometimes you're bust. But life goes on. Remember that. Boxing ❤️", "followers_count": 295, "friends_count": 320, "statues_count": 34532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773493510144, "text": "Un error es una buena oportunidad para hacerlo mejor.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 398331991, "name": "伊萨亚斯乌尤", "screen_name": "JustOne4ever", "lang": "es", "location": "Chicago IL ", "create_at": date("2011-10-25"), "description": "Takes the weak and makes them strong ♎#GUATEMALA♎️Son #filósofos verdaderos aquellos a quienes gusta contemplar la #verdad.", "followers_count": 2539, "friends_count": 2567, "statues_count": 8373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773791305729, "text": "Can someone throw me a birthday party so I can party with my bffs who won't be 21 ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459825067, "name": "Dominique Aaron➰", "screen_name": "Dominique_Aaron", "lang": "en", "location": "Vancouver, WA", "create_at": date("2012-01-09"), "description": "• Future Mrs. Bailey • I'm obsessed with my cat. • Rip Momma you'll always be my sunshine 1/25/73~12/15/15•", "followers_count": 746, "friends_count": 967, "statues_count": 66637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773820686336, "text": "I LOVE MY FRIENDS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4126321933, "name": "its ya boy", "screen_name": "tinkerichimatsu", "lang": "en", "location": "call me saru or ryan", "create_at": date("2015-11-04"), "description": "this is like how to catch a predator, 360 edition ☆ ig: bokubro / tumblr: laikyuu ☆ ☆ i love @natnattienat with all my heart ☆ thirsty for hroki", "followers_count": 337, "friends_count": 219, "statues_count": 11389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weed, CA", "id": "987eb814ecef80da", "name": "Weed", "place_type": "city", "bounding_box": rectangle("-122.402435,41.366178 -122.366391,41.443719") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6093, "countyName": "Siskiyou", "cityID": 683850, "cityName": "Weed" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773967486977, "text": "Lmao When bitches brag about me getting from my mom ?? Nah not my problem we got a business out there in Mexico and I could take out money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448623559, "name": "yaBoijü@n❗", "screen_name": "juanmedina728", "lang": "en", "location": "Norwalk, CA , Tolleson, AZ", "create_at": date("2011-12-27"), "description": "null", "followers_count": 279, "friends_count": 437, "statues_count": 2452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966773992632321, "text": "@David9280 WHAT IS HIS SNAP", "in_reply_to_status": 696965294049226752, "in_reply_to_user": 238259878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 238259878 }}, "user": { "id": 2393603520, "name": "Chandelle Marie", "screen_name": "Chaaaandelle", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-03-16"), "description": "| Heaven doesn't want me, & Hell is afraid I will take over | Design House [Fashion Group] | 23 |", "followers_count": 460, "friends_count": 344, "statues_count": 11461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966774252658688, "text": "42.1F (Feels: 42.1F) - Humidity: 56% - Wind: 5.4mph N - Gust: 6.9mph - Pressure: 1035.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966775238430720, "text": "#monday night #concert #music #troyesivan #foxtheater #oakland #510 #youth #youtube#youtuber… https://t.co/p3ZZKXnOFD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.270195,37.8077583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "monday", "concert", "music", "troyesivan", "foxtheater", "oakland", "youth" }}, "user": { "id": 30547131, "name": "Ryan Louie O. Manuel", "screen_name": "RyLouie1003", "lang": "en", "location": "San Francisco", "create_at": date("2009-04-11"), "description": "Random Rants. daily updates. whatevers clever in SF =P", "followers_count": 211, "friends_count": 412, "statues_count": 6535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-09T00:00:02.000Z"), "id": 696966775607422976, "text": "#SupportOriginMelissa 21.9°F Wind:3.4mph Pressure: 29.74hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers likely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776010240000, "text": "OH FUCK OKAY I GOT THIS\nTAKE CARE,\nLORD KNOWS\nOVER MY DEAD BODY\nPOUND CAKE https://t.co/ZWsLLmGZ6V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420781896, "name": "LEMONADE W A V E S", "screen_name": "24kHXN", "lang": "en", "location": "umich '19", "create_at": date("2011-11-24"), "description": "not affiliated with the real sheehan khan (pictured above) // lightskin 18 year old bangladeshi male // tycho, kanye & bon iver // praise mukhtar", "followers_count": 720, "friends_count": 515, "statues_count": 28367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ann Arbor, MI", "id": "92b2158a6d509767", "name": "Ann Arbor", "place_type": "city", "bounding_box": rectangle("-83.800262,42.215087 -83.659681,42.326592") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2603000, "cityName": "Ann Arbor" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776278556672, "text": "I act like nothing wrong but shit I be hurt ��!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2825897266, "name": "ESMY.", "screen_name": "_thucatrachita", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "Christian❤️", "followers_count": 416, "friends_count": 183, "statues_count": 3320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776454832129, "text": "Bruh not the $1 �� https://t.co/vaaBQU0AoD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328805675, "name": "Lex♏️", "screen_name": "WhyHATE_lexii", "lang": "en", "location": "423✈️931", "create_at": date("2011-07-03"), "description": "Don't watch me you'll be confused I promise APSU19 #LongLiveKevin 09/30/15", "followers_count": 2746, "friends_count": 1267, "statues_count": 73497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, TN", "id": "52f4a98d03e4ea4f", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-87.486546,36.4602 -87.157791,36.641767") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47125, "countyName": "Montgomery", "cityID": 4715160, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776526086144, "text": "@Ayeekonnah thx boo we must link up w Jenny sumtime", "in_reply_to_status": 696966113574395904, "in_reply_to_user": 419697338, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 419697338 }}, "user": { "id": 391802172, "name": "مايره", "screen_name": "maydawg_", "lang": "en", "location": "sc; mxyra", "create_at": date("2011-10-15"), "description": "19♓️ RIP ANT❣", "followers_count": 2768, "friends_count": 831, "statues_count": 66218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanham, MD", "id": "019ae9b86255a185", "name": "Lanham", "place_type": "city", "bounding_box": rectangle("-76.885508,38.934439 -76.821939,38.988666") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445550, "cityName": "Lanham" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776538730496, "text": "Wind 0.0 mph ---. Barometer 29.73 in, Falling. Temperature 25.0 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776597262336, "text": "If you can flirt with others girl in front of your girl, damn I only imagine what you do when she ain't around.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2322782732, "name": "Brittney Rodriguez", "screen_name": "Bmr23Rod", "lang": "en", "location": "null", "create_at": date("2014-02-01"), "description": "Tessa is the best ❤️. Employeed. Student.", "followers_count": 348, "friends_count": 300, "statues_count": 7207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thatcher, AZ", "id": "019c4b8a4a5f68ec", "name": "Thatcher", "place_type": "city", "bounding_box": rectangle("-109.795135,32.797639 -109.732871,32.86024") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4009, "countyName": "Graham", "cityID": 473420, "cityName": "Thatcher" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776697958400, "text": "#EsuranceSweepstakes bringing tweeters together", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EsuranceSweepstakes" }}, "user": { "id": 312471208, "name": "Shelley Andagan", "screen_name": "FireonIce808", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "I work to enrich, empower & inspire others |\r\nI fight for those without a voice |\r\nI strive to glorify God with these gifts", "followers_count": 70, "friends_count": 184, "statues_count": 669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966776702312448, "text": "GN (@ Meridian Place in Northridge, CA) https://t.co/s3OeI8dhtD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 378, "friends_count": 337, "statues_count": 25504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777020895232, "text": "Why am I still awake!!������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336374542, "name": "Kourt", "screen_name": "MiSoDeLeo1", "lang": "en", "location": "Bay Area", "create_at": date("2011-07-15"), "description": "Life:Miranda Soto De Leon ~ Snappy:villavanillaa", "followers_count": 278, "friends_count": 261, "statues_count": 11952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777121611776, "text": "I over think things too much.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352181609, "name": "isaac ¯\\_(ツ)_/¯", "screen_name": "isaac_jaked", "lang": "en", "location": "California, USA", "create_at": date("2011-08-10"), "description": "Sluts welcomed •19•Cali ☀️. •IG: @isaac_jake• SnapChat : isaac_corona", "followers_count": 289, "friends_count": 163, "statues_count": 19772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777138515969, "text": "@Aesthethica Dats not cool", "in_reply_to_status": 696964308983529472, "in_reply_to_user": 2713633955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2713633955 }}, "user": { "id": 344574030, "name": "Jarek", "screen_name": "JarekLugo", "lang": "en", "location": "null", "create_at": date("2011-07-29"), "description": "football season is over", "followers_count": 539, "friends_count": 351, "statues_count": 28711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "York, PA", "id": "39c84b689e24ab85", "name": "York", "place_type": "city", "bounding_box": rectangle("-76.762559,39.942947 -76.699457,39.991071") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4287048, "cityName": "York" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777520017408, "text": "@Flash_Bryan that was on the third game", "in_reply_to_status": 696966317308383233, "in_reply_to_user": 946632235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 946632235 }}, "user": { "id": 840526178, "name": "Lucifer", "screen_name": "PapiKrisVuitton", "lang": "en", "location": "#Hollywood ", "create_at": date("2012-09-22"), "description": "Your #MCM is my #TBT. lies you tell", "followers_count": 238, "friends_count": 226, "statues_count": 14033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777658482689, "text": "Pretty women are you here, are you here right now�� @christinamilian @laurenbrodgers… https://t.co/TNbBN1yYlF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.39774241,34.06754168"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 62432717, 55503291 }}, "user": { "id": 295822591, "name": "Jennifer Gomez", "screen_name": "its_jgo", "lang": "en", "location": "New York, NY", "create_at": date("2011-05-09"), "description": "http://www.thejgoagency.com", "followers_count": 390, "friends_count": 186, "statues_count": 2233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, CA", "id": "741e9df4d2522275", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-118.427344,34.052559 -118.371934,34.112434") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 606308, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966777658589184, "text": "This tweet was severely lacking in exclamation points https://t.co/HDyLpSvRwg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 200345012, "name": "Grant Salzano", "screen_name": "Salzano14", "lang": "en", "location": "East Hampton, CT", "create_at": date("2010-10-08"), "description": "Senior Writer, BC Interruption. My interests include Iran rapprochement, MS Excel, travel deals, North Korea, and men's & women's ice hockey. Boston College '10", "followers_count": 775, "friends_count": 83, "statues_count": 34454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Hampton, CT", "id": "8029b540f8d6232a", "name": "East Hampton", "place_type": "city", "bounding_box": rectangle("-72.550426,41.553046 -72.451834,41.608287") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 922420, "cityName": "East Hampton" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966778228862977, "text": "I just wanna make a enough to live life enjoyably and go ghost.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327137419, "name": "✨✨", "screen_name": "_HushLeslie", "lang": "en", "location": "Rtown$", "create_at": date("2011-06-30"), "description": "ig : kimmydollas • Be happy and do as you please; our time is limited ..", "followers_count": 951, "friends_count": 655, "statues_count": 23587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966778371620864, "text": "Mhm. https://t.co/4X7UJ0NQGm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 601130099, "name": "Baby Girl", "screen_name": "Tay_Lechelle21", "lang": "en", "location": "Tennessee, USA", "create_at": date("2012-06-06"), "description": "Blunts • Bombay • Lemonade Instagam: Taylechelle", "followers_count": 1565, "friends_count": 2067, "statues_count": 19175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, TN", "id": "aa0758981a56db80", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-86.120143,35.421627 -86.020348,35.510117") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47031, "countyName": "Coffee", "cityID": 4745500, "cityName": "Manchester" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966778531012608, "text": "@kanyewest take loads of poop", "in_reply_to_status": 696942468076564480, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 132352159, "name": "E BAND$", "screen_name": "embrek876", "lang": "en", "location": "New Rochelle/Buffalo", "create_at": date("2010-04-12"), "description": "professional wumbologist ♍️ #freethenipple #taohouse #blacklivesmatter", "followers_count": 736, "friends_count": 511, "statues_count": 14702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Rochelle, NY", "id": "11cdb607ab2f17bb", "name": "New Rochelle", "place_type": "city", "bounding_box": rectangle("-73.807197,40.881294 -73.755414,40.988329") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3650617, "cityName": "New Rochelle" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966778912706560, "text": "hey I just want to apologize BUT I TYPE IN CAPS 90% OF THE TIME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1611622812, "name": "✨spicy @ katsu", "screen_name": "linksassy", "lang": "en", "location": " ✿ emi / hyrule / 20 ✿", "create_at": date("2013-07-21"), "description": "multi-fandom ✩ anime ✩ tera ✩ bns ✩ loz ✩ ff ✩ fe ✩ pokemon ✩ all things nintendo ✩ ✨my loves: @flarefoxes @AhirzaWyvern✨ ✩ ❤️ my queen @suwasame ❤️ ✩", "followers_count": 193, "friends_count": 116, "statues_count": 26952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgemere, MD", "id": "76117d558b15022f", "name": "Edgemere", "place_type": "city", "bounding_box": rectangle("-76.501115,39.195332 -76.397677,39.260896") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2424950, "cityName": "Edgemere" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966779143352320, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1497353190, "name": "Tyree Vines", "screen_name": "Chedda_Bo_3", "lang": "en", "location": "Mobile, Al ", "create_at": date("2013-06-09"), "description": "#251✌️✋☝️ #GOD1st #ballislife #VHS2014 University of South Alabama Football #JagNation", "followers_count": 1100, "friends_count": 812, "statues_count": 18649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-02-09T00:00:03.000Z"), "id": 696966779608772608, "text": "Could have told you to fucking DIE but I didn't . But like I said stop talking to me & gone on witcho hoe life nigga https://t.co/kEGEwA5MvL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3248092609, "name": "Lex ♎", "screen_name": "x_roseeee", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "Oklahoma State University '19 | New Twitter", "followers_count": 413, "friends_count": 424, "statues_count": 15672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966780124790784, "text": "Temp 26.9°F Wind Chill 26.9°F RH 91% Wind 0.0 --- Gust 0.0 --- SLP 29.676 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 105, "friends_count": 63, "statues_count": 33133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bluefield, WV", "id": "8057944e55790b13", "name": "Bluefield", "place_type": "city", "bounding_box": rectangle("-81.265356,37.234874 -81.165628,37.281748") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54055, "countyName": "Mercer", "cityID": 5408524, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966780250624001, "text": "Wrap ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 360754649, "name": "Trap God", "screen_name": "demell_smith", "lang": "en", "location": "❄️Wisco ", "create_at": date("2011-08-23"), "description": "Live Life #Blessed. ️", "followers_count": 1507, "friends_count": 1074, "statues_count": 25352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966780636430336, "text": "lmaooooo nigga answer me @saucefredo_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 378734845 }}, "user": { "id": 2597376751, "name": "️", "screen_name": "thethurdhuh", "lang": "en", "location": "DTX // SMTX", "create_at": date("2014-06-30"), "description": "Hospital Administration Major #TxState", "followers_count": 336, "friends_count": 204, "statues_count": 5166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966780812664832, "text": "02/09@03:00 - Temp 31.7F, WC 31.7F. Wind 2.8mph ENE, Gust 8.0mph. Bar 29.660in, Falling. Rain 0.00in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 46874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966781064220674, "text": "Stokos ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "lt", "is_retweet": false, "user": { "id": 185952163, "name": "Capital A The DJ", "screen_name": "capitalathedj", "lang": "en", "location": "Baltimore, MD", "create_at": date("2010-09-01"), "description": "Open Format Disc Jockey | NY Made Me | RVA Raised Me | 1/2 of #PartyPhi | #MSU18 | bookcapitala@gmail.com | #WAWG", "followers_count": 1870, "friends_count": 1378, "statues_count": 60185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966781320073216, "text": "I want something sweet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46846728, "name": "Vicentee", "screen_name": "vincentalvarez", "lang": "en", "location": "null", "create_at": date("2009-06-13"), "description": "null", "followers_count": 743, "friends_count": 774, "statues_count": 26766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966781886468096, "text": "#911emergency at 9115-9146 Bay Hill Blvd. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.512122,28.453178"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "911emergency", "orlpol", "ocso" }}, "user": { "id": 39050965, "name": "Police Calls 32819", "screen_name": "orlpol32819", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 285, "friends_count": 1, "statues_count": 69878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay Hill, FL", "id": "6c4cd7d3678b11f6", "name": "Bay Hill", "place_type": "city", "bounding_box": rectangle("-81.52275,28.43523 -81.499714,28.483395") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1204070, "cityName": "Bay Hill" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966782012116993, "text": "@whoeveriamtoday Don't hate life :)", "in_reply_to_status": 696965459296399360, "in_reply_to_user": 179787649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 179787649 }}, "user": { "id": 806585, "name": "Chris Mahan", "screen_name": "chris_mahan", "lang": "en", "location": "Northridge, California", "create_at": date("2007-03-02"), "description": "Writer and coder. 47, married, 10 yo son. \n\nStories and poems at http://christophermahan.com/writings/\n\nAnd yes, I'm a creep. \n\n#BlackLivesMatter", "followers_count": 1141, "friends_count": 466, "statues_count": 62304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966782247051264, "text": "@YungGapGod @RAULTHEBASEDGOD me too��", "in_reply_to_status": 696965959043579904, "in_reply_to_user": 181097250, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181097250, 265524053 }}, "user": { "id": 386355723, "name": "sling", "screen_name": "SlingGotJuice_", "lang": "en", "location": "San Antonio", "create_at": date("2011-10-06"), "description": "IM FUNNY | RIP DAVEON J. JORDAN ❤️ | IG: D_Sling23 | TX", "followers_count": 1921, "friends_count": 769, "statues_count": 65422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966782532349952, "text": "I swear 2-5 get the most hate. Bet you won't bring ya ass round there. Fuck around and get found in the dismal swamp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81472795, "name": "Ty da Twooo", "screen_name": "Ty_the2", "lang": "en", "location": "Greensboro, NC", "create_at": date("2009-10-10"), "description": "|20♨️| Life is everything you make of it ⛽️ |252| #GoodVibes| God wont put more on you than you can bear.|", "followers_count": 3875, "friends_count": 3706, "statues_count": 75227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-09T00:00:04.000Z"), "id": 696966782607892480, "text": "Being yourself you can do better than anyone else so Just Be Yourself. #lovelife #loveyourself… https://t.co/2GrwLTNNNM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.11854,37.47915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lovelife", "loveyourself" }}, "user": { "id": 179819357, "name": "bootsy", "screen_name": "philly76er", "lang": "en", "location": "VA/Philadelphia", "create_at": date("2010-08-17"), "description": "Philly is home! I dance. I jump rope. I hula hoop with mad skillz. I love to laugh! I teach BodyJAM! YouTube: PhillyboyGO Instagram: Philly76er ♏100% Scorpio♏", "followers_count": 426, "friends_count": 284, "statues_count": 21369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madison Heights, VA", "id": "3c8f8ac723a445d8", "name": "Madison Heights", "place_type": "city", "bounding_box": rectangle("-79.150419,37.398346 -79.084999,37.495306") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51009, "countyName": "Amherst", "cityID": 5148520, "cityName": "Madison Heights" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966784688074753, "text": "����If��I��win�� I'll ring 22nd warning\n@esurance ������\n #EsuranceSweepstakes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EsuranceSweepstakes" }}, "user_mentions": {{ 14388488 }}, "user": { "id": 38796615, "name": "SuperStarMAUI.com", "screen_name": "superstarmaui", "lang": "en", "location": "MAUI IS SUPERSTAR 2THE WORLD", "create_at": date("2009-05-08"), "description": "THIS IS THE ONE & ONLY REAL MAUI", "followers_count": 859, "friends_count": 877, "statues_count": 3729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966784721653760, "text": "@StephAnTheCity us this weekend https://t.co/sBOGbUUETJ", "in_reply_to_status": -1, "in_reply_to_user": 426145421, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426145421 }}, "user": { "id": 727386552, "name": "Katie Downs", "screen_name": "KatieLeeigh", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "I need coffee.", "followers_count": 1242, "friends_count": 1076, "statues_count": 20732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966784927174658, "text": "Officially taking 23 units this quarter! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1955196594, "name": "Stephanie Wong", "screen_name": "stephuhni_", "lang": "en", "location": "null", "create_at": date("2013-10-11"), "description": "null", "followers_count": 107, "friends_count": 94, "statues_count": 540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966784931344384, "text": "Fr dude that shit's crazy asf \n☺️��������������❤️�� <<< it's a cycle https://t.co/z0mK3xKAhr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2402516977, "name": "☺️", "screen_name": "CodyxBelcher", "lang": "en", "location": "null", "create_at": date("2014-03-21"), "description": "Andre is my everything from now until forever ❤️", "followers_count": 411, "friends_count": 246, "statues_count": 4718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966784943951874, "text": "LIT AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2899066088, "name": "Damian Rios", "screen_name": "Damian_xvx", "lang": "en", "location": "Oregon, USA", "create_at": date("2014-11-13"), "description": "Mck hoops | XC | T&F |", "followers_count": 347, "friends_count": 180, "statues_count": 3110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayesville, OR", "id": "ef6bc9d58ea0c80b", "name": "Hayesville", "place_type": "city", "bounding_box": rectangle("-122.992994,44.958825 -122.948448,45.01161") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4132850, "cityName": "Hayesville" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966785409495041, "text": "anybody that say \"whenever you're comfortable\" when you just got out of a relationship lusting hard as fuck https://t.co/lercZdXxMe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2474404883, "name": "∀", "screen_name": "ldin__", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-04-10"), "description": "you wasn't shit until i put my dick in you", "followers_count": 1851, "friends_count": 811, "statues_count": 25330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966785426264064, "text": "@___Nache___ i hope not, because i dont feel like nursing them", "in_reply_to_status": 696966668468273152, "in_reply_to_user": 366900988, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 366900988 }}, "user": { "id": 276725481, "name": "New Tunes Soon", "screen_name": "XtraCurricular_", "lang": "en", "location": "Arlington/Huntsville,Texas", "create_at": date("2011-04-03"), "description": "an aspiring Gentleman....that is all #ScholarshipPlayers....im trying to live BAD", "followers_count": 1591, "friends_count": 1454, "statues_count": 91246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966785573249024, "text": "YVETTE & BRANDI !!! https://t.co/xOe1WPE4kM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2769458769, "name": "Tye Boogie", "screen_name": "5aa91bf204f14c5", "lang": "en", "location": "null", "create_at": date("2014-09-13"), "description": "Love Life & Keep it 100% !!!", "followers_count": 1319, "friends_count": 1966, "statues_count": 9085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966786080575489, "text": "You know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1604857795, "name": "Blade", "screen_name": "bladebaker", "lang": "en", "location": "Huntington Beach", "create_at": date("2013-07-18"), "description": "null", "followers_count": 631, "friends_count": 91, "statues_count": 19998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966786093285376, "text": "yup https://t.co/wrZNb5ejSg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 292940164, "name": "Neisha ❣", "screen_name": "_yourstruuly_", "lang": "en", "location": "BOGALUSA RAISED! ", "create_at": date("2011-05-04"), "description": "#Senior2016 18yearsyoung", "followers_count": 1759, "friends_count": 1336, "statues_count": 91705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bogalusa, LA", "id": "0d86f1118064d85f", "name": "Bogalusa", "place_type": "city", "bounding_box": rectangle("-89.896103,30.746749 -89.833306,30.812685") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22117, "countyName": "Washington", "cityID": 2208150, "cityName": "Bogalusa" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966786881695745, "text": "Want this jersey but a Kawhi or Patty.. https://t.co/hoVEBWn6fJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1029643933, "name": "Easiest-E✌⚾", "screen_name": "TreHo_14", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "Phillipians 4:13 I can do all things through Christ who strengthens me #UTSA18'", "followers_count": 634, "friends_count": 451, "statues_count": 13798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966786986614784, "text": "Temp: 15.1°F - Dew Point: 7.8° - Wind: 12.5 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 29.72in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966787167031296, "text": "I loved By The Grace of God, but i never actually listened until a few days ago. It speaks to on a whole other level. I feel you @katyperry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21447363 }}, "user": { "id": 139943042, "name": "❤BS", "screen_name": "Queen_BScott", "lang": "en", "location": "CMU", "create_at": date("2010-05-03"), "description": "If there was an Awkward Olympics, I'd be Michael Phelps.", "followers_count": 657, "friends_count": 388, "statues_count": 37423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, MI", "id": "77f923f5a841bbdf", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-84.816945,43.553246 -84.727407,43.63446") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26073, "countyName": "Isabella", "cityID": 2656020, "cityName": "Mount Pleasant" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966787817033728, "text": "Wind 0.0 mph ---. Barometer 30.222 in, Rising. Temperature 48.3 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 249872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966787993366528, "text": "@LucasLascivious you feeling better now?", "in_reply_to_status": 696966645466689536, "in_reply_to_user": 17259094, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17259094 }}, "user": { "id": 35165330, "name": "Uh Huh. Ok.", "screen_name": "mrcorkles", "lang": "en", "location": "Chicago", "create_at": date("2009-04-24"), "description": "Don't get mad. I'm not serious. GMO and gluten free sustainable Af Am homo here. Game Of Thrones. Getting On. Jessica Jones. Narcos. HTGAWM. RH. Transparent", "followers_count": 221, "friends_count": 530, "statues_count": 10698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-09T00:00:05.000Z"), "id": 696966788182122496, "text": "Jasper's going crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1898292656, "name": "morgan☼", "screen_name": "st4ygolden_", "lang": "en", "location": "not here", "create_at": date("2013-09-23"), "description": "21 // Florida // XI XXI♡", "followers_count": 349, "friends_count": 475, "statues_count": 12193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sarasota, FL", "id": "5acc3a887617d093", "name": "Sarasota", "place_type": "city", "bounding_box": rectangle("-82.588866,27.293114 -82.477281,27.389129") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1264175, "cityName": "Sarasota" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966789113118720, "text": "@Wolfram1045 @CNNPolitics @ericbradner he winning the state though...", "in_reply_to_status": 696927406305767424, "in_reply_to_user": 3624220033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3624220033, 13850422, 54322823 }}, "user": { "id": 337969081, "name": "Gable Schoenfelder", "screen_name": "gablesch", "lang": "en", "location": "Rancho Cucamonga, CA", "create_at": date("2011-07-18"), "description": "I stay winning!! Future SF Giants broadcaster or ESPN anchor and starting my journey at Chaffey College #BleedGreen", "followers_count": 1144, "friends_count": 862, "statues_count": 61657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966789310214144, "text": "@chiinkiegee NO I am standing up to you ya big bully", "in_reply_to_status": 696966615334649857, "in_reply_to_user": 168895158, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168895158 }}, "user": { "id": 24038805, "name": "Sir Hottest Of Jays", "screen_name": "j_hotz", "lang": "en", "location": "Crippled America", "create_at": date("2009-03-12"), "description": "Free hatred from your heart.", "followers_count": 2500, "friends_count": 853, "statues_count": 365523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966789591408640, "text": "Wind 3 mph WNW. Barometer 30.12 in, Steady. Temperature 54.5 °F. Rain today 0.00 in. Humidity 27%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 53, "friends_count": 70, "statues_count": 25580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966790014840832, "text": "Late night thinking", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379942381, "name": "Nina Umfrid", "screen_name": "NinaUmfrid", "lang": "en", "location": "null", "create_at": date("2011-09-25"), "description": "null", "followers_count": 761, "friends_count": 652, "statues_count": 18969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keizer, OR", "id": "fd5dc77c129022ee", "name": "Keizer", "place_type": "city", "bounding_box": rectangle("-123.05417,44.972903 -122.99073,45.038125") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4138500, "cityName": "Keizer" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966790342057984, "text": "I wanna say Amber but I like Khloe too much https://t.co/mwHyZ0ZhIQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155127923, "name": "Whats Really Good ?", "screen_name": "WatchMeDestroy", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-06-12"), "description": "null", "followers_count": 9338, "friends_count": 4001, "statues_count": 128333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966790581190656, "text": "Kinda frustrated at the moment. Hate when people lie to get what they want. (@ Sam's Club in Reynoldsburg, OH) https://t.co/8RrDLAtj0F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7869,39.9368"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2967026313, "name": "Juan Israel Ortiz", "screen_name": "juanisraelortiz", "lang": "en", "location": "Puerto Rico", "create_at": date("2015-01-07"), "description": "null", "followers_count": 302, "friends_count": 171, "statues_count": 589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reynoldsburg, OH", "id": "ce66852c89aa6582", "name": "Reynoldsburg", "place_type": "city", "bounding_box": rectangle("-82.835601,39.931348 -82.693943,40.018293") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3966390, "cityName": "Reynoldsburg" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966790824488960, "text": "I want a Relationship ��❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2900117999, "name": "❤", "screen_name": "xxSnowLight", "lang": "en", "location": "null", "create_at": date("2014-11-30"), "description": "free mikey ❤", "followers_count": 786, "friends_count": 1189, "statues_count": 7673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966791298359298, "text": "@rissyterry lol this is what love looks like.", "in_reply_to_status": 696966486670200832, "in_reply_to_user": 1100002849, "favorite_count": 0, "coordinate": point("-119.0797494,34.3427272"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1100002849 }}, "user": { "id": 2351245543, "name": "Cynmeow", "screen_name": "cyndy_maldonado", "lang": "en", "location": "null", "create_at": date("2014-02-18"), "description": "Keep it simple |ΔΓ| ✌", "followers_count": 229, "friends_count": 341, "statues_count": 2699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966791680126976, "text": "Barberton Oh Temp:29.9°F Wind:2 mph Dir:WNW Baro:Falling slowly Rain2day:0.00in Hum:87% UV:0.0 @ 03:00 02/09/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966791830962176, "text": "ليه مافيه ميكرويف يبرّد ؟", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 141401519, "name": "أحمد", "screen_name": "xqct87", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2010-05-07"), "description": "سبحان الله وبحمده .. سبحان الله العظيم", "followers_count": 390, "friends_count": 242, "statues_count": 9813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, PA", "id": "d88a5def1d7e9609", "name": "Indiana", "place_type": "city", "bounding_box": rectangle("-79.213942,40.586024 -79.089163,40.656614") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42063, "countyName": "Indiana", "cityID": 4236816, "cityName": "Indiana" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966791852068864, "text": "Wind 0 mph ---. Barometer 1010.2 hPa, Falling. Temperature 71.4 °F. Rain today 0.00 in. Humidity 41%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 264, "statues_count": 144948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-09T00:00:06.000Z"), "id": 696966792355274753, "text": "@MaddyOreillyxxx 30 year campaign to create mental illness and remove me from society at https://t.co/Kglv865MJc", "in_reply_to_status": 696963308956549120, "in_reply_to_user": 332626171, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332626171 }}, "user": { "id": 25652802, "name": "♔ kevin perelman ♔", "screen_name": "KevinPerelman", "lang": "en", "location": "Woodland Hills", "create_at": date("2009-03-21"), "description": "Im Kevin Perelman.You all know me, but pretend not to due to a world wide funded smear campaign since 10. Nerdist, Artist, Photographr,Web Developer,3D graphics", "followers_count": 2014, "friends_count": 2429, "statues_count": 807857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-09T00:00:07.000Z"), "id": 696966792539820032, "text": "2/9/2016 - 02:00\nTemp: 38.0F \nHum: 44%\nWind: 3.0 mph\nBaro: 30.078in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 53, "statues_count": 49799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-09T00:00:07.000Z"), "id": 696966792598556673, "text": "@Yorrike @cbdawson The faults that you didn't know about are almost always scarier than the ones you do...", "in_reply_to_status": 696966395406319616, "in_reply_to_user": 14128834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14128834, 23044822 }}, "user": { "id": 40743645, "name": "Julian Lozos", "screen_name": "seismogenic", "lang": "en", "location": "Blind Thrust Fault", "create_at": date("2009-05-17"), "description": "Geophysics ass't professor at CSU Northridge, studying how faults behave and interact in earthquakes. Also plays music and draws pictures. Opinions are my own.", "followers_count": 2339, "friends_count": 1758, "statues_count": 15738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-10T00:00:00.000Z"), "id": 697329151716433920, "text": "@kyliecurtis_ that's what we call fuckboy but y'all knew that when you date them Bc y'all become detectives to make sure no games sometimes", "in_reply_to_status": 697328423425875969, "in_reply_to_user": 735723463, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 735723463 }}, "user": { "id": 734097300, "name": "Fresh Prince", "screen_name": "Jay_King_Foster", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "SC -jay7foster \ninstagram - jay7foster", "followers_count": 858, "friends_count": 859, "statues_count": 12335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-10T00:00:00.000Z"), "id": 697329152471339008, "text": "@The_EgyptianKid ..... https://t.co/Y3P8IwBj2K", "in_reply_to_status": 697328097729781761, "in_reply_to_user": 1471424708, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1471424708 }}, "user": { "id": 842627874, "name": "أمينة", "screen_name": "mxnzz", "lang": "en", "location": "heartbreak hotel", "create_at": date("2012-09-23"), "description": "finessing hearts", "followers_count": 802, "friends_count": 278, "statues_count": 41378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-10T00:00:00.000Z"), "id": 697329152962080768, "text": "it sucks not being able to give up on someone who gave up on you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355259150, "name": "Rae ☀️", "screen_name": "drew_PETERSON53", "lang": "en", "location": "null", "create_at": date("2011-08-14"), "description": "min skål, din skål, alla vackra flickors skål. St Ambrose University", "followers_count": 248, "friends_count": 232, "statues_count": 8337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davenport, IA", "id": "9c1c0a45346f5837", "name": "Davenport", "place_type": "city", "bounding_box": rectangle("-90.69214,41.475836 -90.502744,41.60005") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19163, "countyName": "Scott", "cityID": 1919000, "cityName": "Davenport" } }
+{ "create_at": datetime("2016-02-10T00:00:00.000Z"), "id": 697329153540956160, "text": "Wind 1.0 mph N. Barometer 1030.52 mb, Falling. Temperature 22.2 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-10T00:00:00.000Z"), "id": 697329154476285953, "text": "Shout out to the Kings for really just embarrassing the Bruins tonight. Really love to see that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25515892, "name": "Morgan Miller", "screen_name": "MrMorganMiller", "lang": "en", "location": "Agoura Hills forever probably", "create_at": date("2009-03-20"), "description": "guitar guy in @winceCA and @thesheds, but also who cares??", "followers_count": 1258, "friends_count": 677, "statues_count": 20601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Agoura Hills, CA", "id": "0d9a7f9b8f0baf12", "name": "Agoura Hills", "place_type": "city", "bounding_box": rectangle("-118.794237,34.125821 -118.715023,34.168336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600394, "cityName": "Agoura Hills" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329155634028544, "text": "�� https://t.co/5diAolAus6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 249153062, "name": "♠️NC♠️", "screen_name": "ColemanBoyy", "lang": "en", "location": "null", "create_at": date("2011-02-08"), "description": "null", "followers_count": 556, "friends_count": 430, "statues_count": 15453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Swainsboro, GA", "id": "1a906b8ff0597dd3", "name": "Swainsboro", "place_type": "city", "bounding_box": rectangle("-82.37454,32.554856 -82.295406,32.635943") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13107, "countyName": "Emanuel", "cityID": 1374964, "cityName": "Swainsboro" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329156237959168, "text": "Somebody FaceTime me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384426280, "name": "Ordes D. Speaks III", "screen_name": "_YesImIgnorant", "lang": "en", "location": "Philadelphia , Pa", "create_at": date("2011-10-03"), "description": "PHI✈️ATL Morehouse18", "followers_count": 1411, "friends_count": 1243, "statues_count": 38222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329156300873728, "text": "Not what you think", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558644714, "name": "⭐️PlayaMade♊️", "screen_name": "_Aye_5", "lang": "en", "location": "Murfreesboro, TN", "create_at": date("2012-04-20"), "description": "⭐️Playa made⭐️", "followers_count": 453, "friends_count": 693, "statues_count": 2021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329156623732736, "text": "@petaree20 can't argue with dumb people either", "in_reply_to_status": 697319873743970304, "in_reply_to_user": 602557842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 602557842 }}, "user": { "id": 1339960615, "name": "Austin Truong", "screen_name": "Just_Truong", "lang": "en", "location": "Chino Hills, CA", "create_at": date("2013-04-09"), "description": "Change your thoughts and you will change your world", "followers_count": 217, "friends_count": 209, "statues_count": 2746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329156665827329, "text": "@BigBoosterOrg courtesy to @jab for this slide", "in_reply_to_status": 697328847927246848, "in_reply_to_user": 3373749135, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3373749135, 11177582 }}, "user": { "id": 9251682, "name": "David Chang", "screen_name": "changds", "lang": "en", "location": "Boston, MA", "create_at": date("2007-10-04"), "description": "Entrepreneur (5 startups/acquisitions) & angel investor (35 startups). EiR at Harvard Business School. Recently led @PayPal Boston & co-founded @StartTank.", "followers_count": 5363, "friends_count": 3856, "statues_count": 6770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329156821032960, "text": "Wind 5.6 mph WNW. Barometer 30.19 in, Rising slowly. Temperature 31.8 °F. Rain today 0.00 in. Humidity 36%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329157395468288, "text": "I wanna hold you, kiss you, touch you. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1278910572, "name": "Aleah Nelson", "screen_name": "yogirl_aleah", "lang": "en", "location": "Thatcher, AZ", "create_at": date("2013-03-18"), "description": "In a world full of Kardasians, be Diana.", "followers_count": 1157, "friends_count": 587, "statues_count": 981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thatcher, AZ", "id": "019c4b8a4a5f68ec", "name": "Thatcher", "place_type": "city", "bounding_box": rectangle("-109.795135,32.797639 -109.732871,32.86024") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4009, "countyName": "Graham", "cityID": 473420, "cityName": "Thatcher" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329157471145984, "text": "Why my parents being petty right now..y'all niggas can afford to turn the heat on... Like I'm shivering under a damn blanket how that work ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19050111, "name": "Petty Cey ✊", "screen_name": "TuesyCakes", "lang": "en", "location": "null", "create_at": date("2009-01-15"), "description": "Rest In Heaven Grandpa, Adrian, Dominic, Jermaine Jr. Auntie Nene, Kenzie, Uncle Alan, James ♥️♥️♥️♥️ DeliberatelyObtuse☀️", "followers_count": 1060, "friends_count": 451, "statues_count": 13553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329157584211968, "text": "@ebbtideapp Tide in Bay Spring, Rhode Island 02/10/2016\nHigh 9:06am 5.3\n Low 2:34pm -0.8\nHigh 9:30pm 5.1\n Low 2:48am -0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.3517,41.7517"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 23, "friends_count": 1, "statues_count": 6474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barrington, RI", "id": "171f28605a266e2d", "name": "Barrington", "place_type": "city", "bounding_box": rectangle("-71.353104,41.705922 -71.285346,41.776058") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4422960, "cityName": "East Providence" } }
+{ "create_at": datetime("2016-02-10T00:00:01.000Z"), "id": 697329159031300097, "text": "Did I mention.... ¯\\_(ツ)_/¯ ? \n¯\\_(ツ)_/¯ ¯\\_(ツ)_/¯ ¯\\_(ツ)_/¯", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29073792, "name": "Giant Vagina Hole", "screen_name": "giantvaginahole", "lang": "en", "location": "Chicago", "create_at": date("2009-04-05"), "description": "Lighting Designer. Dog lady. Chicago. Tweets about Frasier, boobs and other things.", "followers_count": 434, "friends_count": 89, "statues_count": 6483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329159832473604, "text": "Temp: 32.7°F Wind:0.0mph Pressure: 30.237hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 59507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329160025321472, "text": "Justin is like 4 feet telling girls he's 6'3 �� AWKS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2700529234, "name": "adreanaamariee", "screen_name": "adreanaam_", "lang": "en", "location": "null", "create_at": date("2014-07-11"), "description": "im good over here", "followers_count": 703, "friends_count": 490, "statues_count": 20215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329160516083713, "text": "#GreetTheAliensIn5Words the older one next door she haunted castles like Chasity.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreetTheAliensIn5Words" }}, "user": { "id": 3062942875, "name": "SarahGiles55 Giless", "screen_name": "GilessSarah", "lang": "en", "location": "Louisiana, USA", "create_at": date("2015-03-05"), "description": "Sarahs animals hang's with Joey, awesome cares thanks?", "followers_count": 532, "friends_count": 2186, "statues_count": 77938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakdale, LA", "id": "74df40186db95a5e", "name": "Oakdale", "place_type": "city", "bounding_box": rectangle("-92.681546,30.780141 -92.637161,30.849104") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22003, "countyName": "Allen", "cityID": 2256540, "cityName": "Oakdale" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329160599896066, "text": "@LebaneseThor you were born.", "in_reply_to_status": 697328903254224896, "in_reply_to_user": 1203318198, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1203318198 }}, "user": { "id": 1904675659, "name": "Dahls.", "screen_name": "dahliababyyy", "lang": "en", "location": "southern california.", "create_at": date("2013-09-25"), "description": "21 | Samoan", "followers_count": 1097, "friends_count": 833, "statues_count": 41045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329160637648896, "text": "HAPPY BIRTHDAY TO MY BESTFRIEND @Josiiaahh I love you ❤️ many more memories to come", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1850151752 }}, "user": { "id": 2181460262, "name": "ęm", "screen_name": "emersonnnks", "lang": "en", "location": "bwekfest club ", "create_at": date("2013-11-07"), "description": "George Mason University Committ", "followers_count": 475, "friends_count": 354, "statues_count": 3535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlsbad, CA", "id": "01d4e349481265e8", "name": "Carlsbad", "place_type": "city", "bounding_box": rectangle("-117.359298,33.060615 -117.216549,33.182353") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 611194, "cityName": "Carlsbad" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329160734294017, "text": "Wind 5.0 mph NW. Barometer 30.208 in, Rising. Temperature 18.4 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329161241759744, "text": "Can you find Sioux Falls on the map? Just try it at https://t.co/oGWgkqHDpT #Sioux Falls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7003,43.55"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sioux" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329161363447808, "text": "Wind 2.0 mph W. Barometer 30.035 in, Falling slowly. Temperature 16.8 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329161661075456, "text": "@BrianaJustineXO I am innocent ��", "in_reply_to_status": 697328678309527552, "in_reply_to_user": 418839140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418839140 }}, "user": { "id": 416859004, "name": "BabyBeck⚾", "screen_name": "YumpItsBeck", "lang": "en", "location": "Indio, CA/Fullerton, CA", "create_at": date("2011-11-19"), "description": "Fullerton College Baseball⚾️ Lamppost .RIP @EseKrook", "followers_count": 1323, "friends_count": 947, "statues_count": 36815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329162214842369, "text": "02/10@03:00 - Temp 30.5F, WC 30.5F. Wind 2.0mph WSW, Gust 6.0mph. Bar 29.664in, Steady. Rain 0.00in. Hum 95%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 46898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329162315526144, "text": "Wind 0.0 mph ---. Barometer 29.56 in, Falling slowly. Temperature 25.9 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 107, "statues_count": 157385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329162839846912, "text": "Where is Sioux Falls on the map? Play the game at https://t.co/oGWgkqHDpT #Sioux Falls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.7003,43.55"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sioux" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-02-10T00:00:02.000Z"), "id": 697329163095666688, "text": "bae hit this weed it might calm you down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906451484, "name": "D.martin", "screen_name": "lasix6", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-12-05"), "description": "FreeJon❤ FreeSteve❤", "followers_count": 1392, "friends_count": 847, "statues_count": 72720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329163959533568, "text": "42.3F (Feels: 42.3F) - Humidity: 77% - Wind: 3.1mph NW - Gust: 5.4mph - Pressure: 1037.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329164014088192, "text": "new piercings <<<", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174755806, "name": "phil", "screen_name": "fathaphil", "lang": "en", "location": "houston, tx", "create_at": date("2010-08-04"), "description": "IG: @philofthefuturo SC: rommie23 #gramfam19", "followers_count": 1943, "friends_count": 927, "statues_count": 57427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329164085428225, "text": "Nah., you gotta chill. https://t.co/I36qVqQD0S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2857007384, "name": "Weekend At Barneys", "screen_name": "Henry_alaxander", "lang": "en", "location": "Carlsbad, CA", "create_at": date("2014-10-15"), "description": "Univ.Houston • BroderickSmithIsNotMyName", "followers_count": 495, "friends_count": 437, "statues_count": 9870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329164110659585, "text": "Travelling to Philadelphia or just twittering about Philadelphia? https://t.co/OJcEkLVx2W #Philadelphia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1638,39.9523"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Philadelphia" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329164458676224, "text": "fuck a title we don't need a title", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1663742468, "name": "g.", "screen_name": "fwmanalicag", "lang": "en", "location": "null", "create_at": date("2013-08-11"), "description": "null", "followers_count": 675, "friends_count": 164, "statues_count": 39702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, CA", "id": "b1d8c2ed61d6a6c8", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.133029,37.68433 -122.093713,37.709794") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 602980, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329165096202240, "text": "falling ba...nvm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35137797, "name": "Mamí Ling", "screen_name": "jeannelululao", "lang": "en", "location": "SC/IG:jeannelululao ", "create_at": date("2009-04-24"), "description": "Half-Thai thickie I.E. #FaithfulAsiansOfTwitter", "followers_count": 3719, "friends_count": 920, "statues_count": 82524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329165901570048, "text": "@gams_mcgee who", "in_reply_to_status": 697328575242858496, "in_reply_to_user": 2861460539, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2861460539 }}, "user": { "id": 3228374936, "name": "Rubén", "screen_name": "ruben_jovel", "lang": "en", "location": "Fullerton, CA", "create_at": date("2015-05-27"), "description": "I walk this earth and watch people. Feel effervescence and want to take the world's pain away", "followers_count": 166, "friends_count": 167, "statues_count": 2375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329166438379521, "text": "If I was to get something for Valentines day this exactly wtf Im gettin�� https://t.co/KLBGN8mk7Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84892289, "name": "⚡️Electro⚡️ (Bryant)", "screen_name": "Ice100x", "lang": "en", "location": "Music City Austin,Texas", "create_at": date("2009-10-24"), "description": "#TeamSagittarius♐️ #TeamTruth #Fashion #Optimistic #CHARGED⚡ #PositiveVibes☀️ #Love❤️ #God #IDontgettired #Flex #Fighter #Loyal #Friendships #Music #Dance #Sing", "followers_count": 1230, "friends_count": 1125, "statues_count": 21749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leander, TX", "id": "be1ec4774a795507", "name": "Leander", "place_type": "city", "bounding_box": rectangle("-97.897898,30.524702 -97.795581,30.612656") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4842016, "cityName": "Leander" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329166601973760, "text": "Prom....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3278247218, "name": "Maritza Gutierrez", "screen_name": "Maritzagee77", "lang": "en", "location": "null", "create_at": date("2015-07-12"), "description": "AHS Class of '16", "followers_count": 186, "friends_count": 134, "statues_count": 982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329167113715712, "text": "@misss_mimiii so you want this https://t.co/UAYLNnMl2K", "in_reply_to_status": 697328934505984000, "in_reply_to_user": 2710393080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2710393080 }}, "user": { "id": 884713489, "name": "jose martinez", "screen_name": "jmartinez5050", "lang": "en", "location": "null", "create_at": date("2012-10-16"), "description": "Don't force anything to happen be patient and it will eventually happen on it's own - Don Blankenship", "followers_count": 234, "friends_count": 517, "statues_count": 10131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delhi, CA", "id": "2e48f9f43b70a7de", "name": "Delhi", "place_type": "city", "bounding_box": rectangle("-120.812484,37.388138 -120.74261,37.459598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 618464, "cityName": "Delhi" } }
+{ "create_at": datetime("2016-02-10T00:00:03.000Z"), "id": 697329167197609985, "text": "You might as well leak em https://t.co/cLvqjzj8ur", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1527006450, "name": "Fried Chicken Papi", "screen_name": "SeanLWashington", "lang": "en", "location": "North West ☔️", "create_at": date("2013-06-18"), "description": "#ThuggaHive | AVO | NEP", "followers_count": 1092, "friends_count": 756, "statues_count": 28045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329168141283328, "text": "Rick & Morty https://t.co/DEqjkbVILh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1170617952, "name": "DAB/VAPE KING", "screen_name": "Beyond_Blake", "lang": "en", "location": "null", "create_at": date("2013-02-11"), "description": "Just sit back & chill man, you only live once in this Lifetime", "followers_count": 605, "friends_count": 413, "statues_count": 49929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329168187576320, "text": "@kanyewest The Last Ordained Prodigy", "in_reply_to_status": -1, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 321231677, "name": "Philly", "screen_name": "Philol_Spice", "lang": "en", "location": "Hastings, Minnesota", "create_at": date("2011-06-21"), "description": "Spice. MC, Producer, Philosopher, & Soothsayer", "followers_count": 286, "friends_count": 355, "statues_count": 16018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hastings, MN", "id": "9599d98423192de9", "name": "Hastings", "place_type": "city", "bounding_box": rectangle("-92.899283,44.704282 -92.822593,44.75947") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2727530, "cityName": "Hastings" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329169613643776, "text": "Temp 17.1°F Wind Chill 17.1°F RH 88% Wind 2.8 W Gust 8.0 W SLP 29.943 in Steady Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 105, "friends_count": 63, "statues_count": 33157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bluefield, WV", "id": "8057944e55790b13", "name": "Bluefield", "place_type": "city", "bounding_box": rectangle("-81.265356,37.234874 -81.165628,37.281748") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54055, "countyName": "Mercer", "cityID": 5408524, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329170028691456, "text": "I didn't even get to know ya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595868530, "name": "kaitlyn kelley", "screen_name": "kkaitlyn12_", "lang": "en", "location": "Arizona, USA", "create_at": date("2012-05-31"), "description": "♑️", "followers_count": 1212, "friends_count": 790, "statues_count": 46976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.62655,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329171362492416, "text": "Shit look like its about to happen I'm dying https://t.co/QcQmxkU545", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3318440184, "name": "devin", "screen_name": "devfromthebay", "lang": "en", "location": "bay area, ca | #saddboiz", "create_at": date("2015-08-18"), "description": "18. invincible. ultimate. marvelous.", "followers_count": 508, "friends_count": 239, "statues_count": 15563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329171442216961, "text": "I don't know why I always find myself going through favs, the end result is never good.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388577100, "name": "danielle", "screen_name": "danielleleeighh", "lang": "en", "location": "muscatine, ia", "create_at": date("2011-10-10"), "description": "• 4Payne 5/17/15 • hanging w ur dad", "followers_count": 1246, "friends_count": 621, "statues_count": 17260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muscatine, IA", "id": "3366b80713ae076b", "name": "Muscatine", "place_type": "city", "bounding_box": rectangle("-91.161353,41.35847 -91.008197,41.478713") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19139, "countyName": "Muscatine", "cityID": 1955110, "cityName": "Muscatine" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329171748421632, "text": "@GotToBeOmar goodnight! ❤️", "in_reply_to_status": 697329000323002368, "in_reply_to_user": 707921544, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 707921544 }}, "user": { "id": 112637770, "name": "Valerie ☹", "screen_name": "Over_doza", "lang": "en", "location": "UT ✈️ WY⚽️", "create_at": date("2010-02-08"), "description": "18 || ig: over_doza7 ✨ || #NoEraPenal || WWCC || #21⚽️ || #HalaMadrid || STAY LIT FAM", "followers_count": 1557, "friends_count": 690, "statues_count": 104692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rock Springs, WY", "id": "5db2aef731e97df0", "name": "Rock Springs", "place_type": "city", "bounding_box": rectangle("-109.282237,41.559506 -109.192506,41.633358") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56037, "countyName": "Sweetwater", "cityID": 5667235, "cityName": "Rock Springs" } }
+{ "create_at": datetime("2016-02-10T00:00:04.000Z"), "id": 697329171815653376, "text": "Can you find Spokane on the map? Just try it at https://t.co/yC27TQFStt #Spokane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.426,47.6588"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Spokane" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329172973092864, "text": "God is the best part of my life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.6704812,26.2027996"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 522614286, "name": "foreverfelicityy", "screen_name": "Felicityluvsyah", "lang": "en", "location": "null", "create_at": date("2012-03-12"), "description": "19\nloving life\npeace✌\nyes im aware that im rare", "followers_count": 451, "friends_count": 829, "statues_count": 4492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harlingen, TX", "id": "1077f3afe6e2aa44", "name": "Harlingen", "place_type": "city", "bounding_box": rectangle("-97.80185,26.135158 -97.629315,26.257572") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4832372, "cityName": "Harlingen" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329172998287360, "text": "Steel Inspector Jobs https://t.co/3cqYLbl9v6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.959785,40.4306"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262344894, "name": "Jobs4welding", "screen_name": "jobs4welding", "lang": "en", "location": "All the World", "create_at": date("2013-12-25"), "description": "View & Apply to welding Jobs", "followers_count": 212, "friends_count": 609, "statues_count": 692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329173338021889, "text": "If I threw my 21st birthday party would yall come?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448500729, "name": "JJ™©", "screen_name": "LiveLoveJJ", "lang": "en", "location": "I Honestly Dont Know ", "create_at": date("2011-12-27"), "description": "|College & Rugby|405 born|God First|IVC|Kaylas Property", "followers_count": 801, "friends_count": 361, "statues_count": 41112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329173539528708, "text": "The full thing is hilarious ������ https://t.co/tpiNqOj1HO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3865327875, "name": "Laurie ☼", "screen_name": "7aurie_", "lang": "en", "location": "Uwf'19 ", "create_at": date("2015-10-04"), "description": "#PatsNation ∞ #Haiti ☼ #FlexingMyComplexion ❤ Rockin' Chuck Taylors w/ a thrift store purse ♢", "followers_count": 218, "friends_count": 236, "statues_count": 1903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferry Pass, FL", "id": "33a522a8e7317b6e", "name": "Ferry Pass", "place_type": "city", "bounding_box": rectangle("-87.247666,30.479609 -87.158257,30.562505") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1222275, "cityName": "Ferry Pass" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329173623275520, "text": "@whitneykk omg please do", "in_reply_to_status": 697322999704264704, "in_reply_to_user": 718851284, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 718851284 }}, "user": { "id": 1721290063, "name": "MØMØ", "screen_name": "mo_wenner5874", "lang": "en", "location": "Bellingham, WA", "create_at": date("2013-09-01"), "description": "MNW • SC moniquewenner", "followers_count": 264, "friends_count": 222, "statues_count": 5496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329173929586689, "text": "Barberton Oh Temp:22.2°F Wind:3 mph Dir:W Baro:Rising slowly Rain2day:0.00in Hum:80% UV:0.0 @ 03:00 02/10/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 232, "friends_count": 228, "statues_count": 114538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329174214742016, "text": "Trying to cleanse my life so for Lent I'm gonna give up sex and gossiping.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333151502, "name": "captain saveahoe", "screen_name": "nicolesamson__", "lang": "en", "location": "null", "create_at": date("2011-07-10"), "description": "null", "followers_count": 531, "friends_count": 332, "statues_count": 26028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Storrs, CT", "id": "0071abee25a42f44", "name": "Storrs", "place_type": "city", "bounding_box": rectangle("-72.317317,41.778651 -72.230633,41.858775") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9013, "countyName": "Tolland", "cityID": 973980, "cityName": "Storrs" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329174760046592, "text": "its 3am.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281761906, "name": ".", "screen_name": "Amberminnick", "lang": "en", "location": "lost", "create_at": date("2011-04-13"), "description": "♡r.i.p. charlie♡ ~r.i.p. kaitlyn~", "followers_count": 1066, "friends_count": 652, "statues_count": 8962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fulton, MD", "id": "012132305c17a818", "name": "Fulton", "place_type": "city", "bounding_box": rectangle("-76.954079,39.136738 -76.889808,39.177625") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2431025, "cityName": "Fulton" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329175531683840, "text": "HAPPY BIRTHDAY MATTHEW!!! ❤️������ https://t.co/HVjuCIPM8t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2185365361, "name": "Lauren Martinez", "screen_name": "GucciLo_19", "lang": "en", "location": "null", "create_at": date("2013-11-09"), "description": "Enjoy every moment", "followers_count": 243, "friends_count": 298, "statues_count": 1052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329175602958337, "text": "https://t.co/aOZF0DyKrP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.2238869,35.586265"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4884746432, "name": "C lucero719@outlook.", "screen_name": "CLucero719pimps", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2016-02-09"), "description": "null", "followers_count": 15, "friends_count": 301, "statues_count": 238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329175691030528, "text": "Temp: 8.8°F - Dew Point: -5.9° - Wind: 14.7 mph - Gust: 22.8 - Rain Today: 0.00in. - Pressure: 29.95in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329175783469056, "text": "I NUTTED IN MY EX BEFORE LEAVING THE CITY https://t.co/LbaV6xmEN3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356531871, "name": "ULT BGOD", "screen_name": "Yeah_lean", "lang": "en", "location": "With The 6 ", "create_at": date("2011-08-16"), "description": "you deserve better & I'm better Soooo.... Wyd?", "followers_count": 474, "friends_count": 536, "statues_count": 3567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscarawas, OH", "id": "8bdeb5af4ef23efc", "name": "Tuscarawas", "place_type": "city", "bounding_box": rectangle("-81.412475,40.392321 -81.389397,40.40512") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39157, "countyName": "Tuscarawas", "cityID": 3977924, "cityName": "Tuscarawas" } }
+{ "create_at": datetime("2016-02-10T00:00:05.000Z"), "id": 697329176018288641, "text": "#SupportOriginMelissa 17.4°F Wind:2.0mph Pressure: 29.94hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329176706052096, "text": "No colors any more I want them to turn black", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3128975226, "name": "Uchiha", "screen_name": "__Reaver", "lang": "en", "location": "Sylmar, Los Angeles", "create_at": date("2015-04-01"), "description": "Im the asshole your mother warned you about", "followers_count": 236, "friends_count": 169, "statues_count": 10939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329176777527297, "text": "Where is Spokane on the map? Play the game at https://t.co/yC27TQFStt #Spokane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.426,47.6588"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Spokane" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329177008033793, "text": "She on here arguing with like three different people and she baking all they ass ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329506470, "name": "#LLTG", "screen_name": "HoopstarWhite", "lang": "en", "location": "East Chicago", "create_at": date("2011-07-04"), "description": "Living life to the fullest #LLTG RIP Rated Rich... Indiana State University '18", "followers_count": 731, "friends_count": 873, "statues_count": 25955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Chicago, IN", "id": "41bc24e82dfe5a2c", "name": "East Chicago", "place_type": "city", "bounding_box": rectangle("-87.490523,41.609953 -87.406819,41.689616") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1819486, "cityName": "East Chicago" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329177259810816, "text": "@SluttyKo go to fucking bed", "in_reply_to_status": -1, "in_reply_to_user": 2711597908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2711597908 }}, "user": { "id": 3933001060, "name": "ig: @so.detroit", "screen_name": "sodetroit__", "lang": "en", "location": "null", "create_at": date("2015-10-11"), "description": "don't worry about my tweets bitches", "followers_count": 215, "friends_count": 40, "statues_count": 2419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329177427456000, "text": "Im chilling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301189783, "name": "Made In INDIA", "screen_name": "heybadnative", "lang": "en", "location": "Omaha, Nebraska", "create_at": date("2011-05-18"), "description": "I'm Not perfect but at least my eyebrows are!!!!", "followers_count": 1572, "friends_count": 1085, "statues_count": 44480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329179017109504, "text": "https://t.co/uNxoPHSuiq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4152737119, "name": "Joe Rodriguez III", "screen_name": "JoeRodriguezII1", "lang": "en", "location": "null", "create_at": date("2015-11-06"), "description": "San Antonio, Texas - Tempe, Arizona. \nBarber", "followers_count": 82, "friends_count": 428, "statues_count": 2436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329179340091393, "text": "I need hangers. Like 50 pls��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3222017149, "name": "Monse", "screen_name": "Monsegarciaaa", "lang": "en", "location": "Hayward, CA", "create_at": date("2015-05-20"), "description": "it is what it is, I'm Gucci tho", "followers_count": 271, "friends_count": 276, "statues_count": 4237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329179725934592, "text": "https://t.co/Yw0EPCYy0C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4020620534, "name": "Von Miller", "screen_name": "alexxg21", "lang": "en", "location": "Colton, CA", "create_at": date("2015-10-25"), "description": "19 | Manchester United | B | Broncos | 6OD | Lakers | THANK YOU FOR EVERYTHING KOBE", "followers_count": 290, "friends_count": 302, "statues_count": 16790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329179822522368, "text": "\"Blue\"\nArea: Products Used:\nEye: Glamorous\nMiddle Eye: Precocious\nOuter V:… https://t.co/yMtGw7D8sO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.405504,28.444637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245870344, "name": "Younique by Sandy N", "screen_name": "SandyYounique", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-02-01"), "description": "I have been with Younique now for little over a year. Absolutely Love their makeup & skincare products. My Favorite is... of course the 3D Fiber Lashes.", "followers_count": 196, "friends_count": 837, "statues_count": 301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-10T00:00:06.000Z"), "id": 697329180330037248, "text": "Travelling to Ormond Beach or just twittering about Ormond Beach? https://t.co/wHhjyoWt4a #Ormond Beach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.0559,29.2858"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Ormond" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2583843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holly Hill, FL", "id": "015d69cb1750d11b", "name": "Holly Hill", "place_type": "city", "bounding_box": rectangle("-81.166062,29.223473 -81.026569,29.356151") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1231350, "cityName": "Holly Hill" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329180443193344, "text": "@Shawdy2Smooth @Please_Josh thanks famo", "in_reply_to_status": 697301905056215040, "in_reply_to_user": 240533917, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240533917, 1483260990 }}, "user": { "id": 1973857796, "name": "J.O.R.D.O.N", "screen_name": "JordonTheLegend", "lang": "en", "location": "Gary, IN✈️Dallas,TX✈️Ft Lee,VA", "create_at": date("2013-10-19"), "description": "Keys too the city on my Feet Buscemi (U.S. Army)", "followers_count": 1032, "friends_count": 852, "statues_count": 40491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181114273792, "text": "@PrinceHarris_ lmao damn . That's bomedy", "in_reply_to_status": 697329102844403712, "in_reply_to_user": 144431704, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 144431704 }}, "user": { "id": 330090898, "name": "✨ MillieMillz ✨", "screen_name": "KamilSoTrill", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-07-05"), "description": "Ya daddy keep my pockets real fat, cause he knows I'm a bitch who stay throwing it back✨", "followers_count": 2028, "friends_count": 576, "statues_count": 104751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181210865664, "text": "Wind 0 mph ---. Barometer 1012.1 hPa, Falling slowly. Temperature 71.1 °F. Rain today 0.00 in. Humidity 38%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 264, "statues_count": 145092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181420449793, "text": "\"I look like a potato\" https://t.co/dP5qXPkUw4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 863773399, "name": "Griffin Dennen", "screen_name": "GriffinDennen", "lang": "en", "location": "San Juan Capistrano, CA", "create_at": date("2012-10-05"), "description": "ello moto", "followers_count": 251, "friends_count": 189, "statues_count": 279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181428998144, "text": "46% is how my life is rn. Not even close to 100% yet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4868207206, "name": "suhhhhh dude", "screen_name": "catslovetacos", "lang": "en", "location": "Chicago, IL", "create_at": date("2016-01-31"), "description": "cats & tacos", "followers_count": 338, "friends_count": 607, "statues_count": 858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cicero, IL", "id": "50f296a1bd4198b6", "name": "Cicero", "place_type": "city", "bounding_box": rectangle("-87.77903,41.82158 -87.738571,41.865972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714351, "cityName": "Cicero" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181500141568, "text": "Dale fav a este tuit si estas despierta y eres una microonda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 994231849, "name": "Caneca™", "screen_name": "williammendez_", "lang": "en", "location": "Dorado, PR", "create_at": date("2012-12-06"), "description": "Por la plata baila el mono, las putas, las chapi y hasta las millo | 22 | BMA en la casa | #KobeNation | Capricornio | IG: Canecaa_ | SC: Caneca_24", "followers_count": 1638, "friends_count": 865, "statues_count": 139452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dorado, PR", "id": "4f3c95e0a936049b", "name": "Dorado", "place_type": "city", "bounding_box": rectangle("-66.295112,18.449974 -66.25321,18.483403") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72051, "countyName": "Dorado", "cityID": 7224008, "cityName": "Dorado" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329181886148612, "text": "@RyBuckets tbh I skipped every video just to see where it'd end up. https://t.co/NS0XCgr0qP", "in_reply_to_status": 697328995533254657, "in_reply_to_user": 1835047506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1835047506 }}, "user": { "id": 2917004893, "name": "401k(lk)", "screen_name": "janderlinde", "lang": "en", "location": "new york shitty", "create_at": date("2014-12-02"), "description": "twenty | dominican & dutch", "followers_count": 228, "friends_count": 99, "statues_count": 9272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329182041190402, "text": "2/10/2016 - 02:00\nTemp: 32.2F \nHum: 51%\nWind: 0.0 mph\nBaro: 30.267in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 53, "statues_count": 49830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329182318161925, "text": "Work then play", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.12966066,40.91260986"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550550983, "name": "Omolope", "screen_name": "RunMyRace_", "lang": "en", "location": "NY", "create_at": date("2012-04-10"), "description": "Royalty is sweet✨#T4L. SBU'18", "followers_count": 587, "friends_count": 539, "statues_count": 12200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stony Brook, NY", "id": "8241555e75571517", "name": "Stony Brook", "place_type": "city", "bounding_box": rectangle("-73.151367,40.868719 -73.096505,40.948108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3671608, "cityName": "Stony Brook" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329182423040000, "text": "@SpotifyCares is this a problem through my distributer or can you fix it?", "in_reply_to_status": 697329009693212672, "in_reply_to_user": 102798721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497340309 }}, "user": { "id": 102798721, "name": "Sarah Maskill", "screen_name": "SarahMaskill", "lang": "en", "location": "michigan", "create_at": date("2010-01-07"), "description": "Michigan State • Media and Infomation • Photographer • Lady of MSU's Ladies First • Egg Scrambler • Feminist • Pun Enthusiast• Knitter", "followers_count": 268, "friends_count": 183, "statues_count": 6004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329182880174080, "text": "I'm so tired of being tired ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47275718, "name": "Scenario Connie", "screen_name": "KaysAllyce", "lang": "en", "location": "Saginaw, MI", "create_at": date("2009-06-14"), "description": "i literally cant even.", "followers_count": 505, "friends_count": 244, "statues_count": 18014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shields, MI", "id": "01e4e96164b42c18", "name": "Shields", "place_type": "city", "bounding_box": rectangle("-84.150851,43.394987 -84.043059,43.481571") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26145, "countyName": "Saginaw", "cityID": 2673560, "cityName": "Shields" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329183609978880, "text": "@hellottiffany where at?", "in_reply_to_status": 697327879965704192, "in_reply_to_user": 25864458, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25864458 }}, "user": { "id": 120974505, "name": "Luis The Child", "screen_name": "Loueezus", "lang": "en", "location": "Bay | UC Irvine", "create_at": date("2010-03-07"), "description": "NΦ BAD DAΨS", "followers_count": 1512, "friends_count": 669, "statues_count": 173550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329183714897920, "text": "Where my sleep went?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628185540, "name": "ღHeidy Barbieeღ", "screen_name": "HeidyHiraldo", "lang": "es", "location": "Big Apple", "create_at": date("2012-07-05"), "description": "Cold Hearted•+• ♠️", "followers_count": 122, "friends_count": 60, "statues_count": 4922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329184100618240, "text": "Happy birthday Andrew!!! Thank you for being such a good friend, ily!!!�� https://t.co/hVIVYhKMYw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1341247208, "name": "Maddie", "screen_name": "madiekinzz", "lang": "en", "location": "null", "create_at": date("2013-04-10"), "description": "sorryyyyy http://m-4ddison.tumblr.com", "followers_count": 833, "friends_count": 610, "statues_count": 18126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-02-10T00:00:07.000Z"), "id": 697329184423702528, "text": "@joinez @LoveeeJones def more of a curse I can't even entertain other men it feels so stupid", "in_reply_to_status": 697328782626156544, "in_reply_to_user": 115801762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 115801762, 61629046 }}, "user": { "id": 27108396, "name": "Ky Baby.", "screen_name": "KyMalibu", "lang": "en", "location": "Washington, DC", "create_at": date("2009-03-27"), "description": "you ain't got the juice like that.", "followers_count": 4982, "friends_count": 813, "statues_count": 76256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suitland, MD", "id": "00da23b51c3aabc1", "name": "Suitland", "place_type": "city", "bounding_box": rectangle("-76.958123,38.827518 -76.88434,38.868794") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2475725, "cityName": "Suitland" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329184801054721, "text": "#MYOMGBABY #�� #OMGSTARSPETS #OMGBEARBEAR #�� #我的天爱小熊熊 #小布宝宝 #我的天爱小布布… https://t.co/9M1W2tHqn2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.32253,34.10403"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MYOMGBABY", "OMGSTARSPETS", "OMGBEARBEAR", "我的天爱小熊熊", "小布宝宝", "我的天爱小布布" }}, "user": { "id": 2488537724, "name": "GodnigelOMG我的天", "screen_name": "nigel_elijah", "lang": "en", "location": "Oh MY GOD SG50 HELPING HANDS", "create_at": date("2014-05-10"), "description": "PUBLIC FIGURE, MEDIACORP P/T ARTISTE, COMEDIAN,LOVE AMBASSADOR新传媒特约艺人,搞怪谐星爱心大使\r\n,演员", "followers_count": 122, "friends_count": 1488, "statues_count": 6001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329185078034432, "text": "its a hard life out here for some of us", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2867501545, "name": "мσяgαи✨", "screen_name": "morganashley105", "lang": "en", "location": "bitchinnn somewhere", "create_at": date("2014-10-20"), "description": "null", "followers_count": 479, "friends_count": 455, "statues_count": 6547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thibodaux, LA", "id": "1a40ecb19d710091", "name": "Thibodaux", "place_type": "city", "bounding_box": rectangle("-90.881732,29.753203 -90.784444,29.835904") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22057, "countyName": "Lafourche", "cityID": 2275425, "cityName": "Thibodaux" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329185098870784, "text": "@onlyonemercy �� I'm js . ima prolly watch it one day to see what goes down", "in_reply_to_status": 697329061253636097, "in_reply_to_user": 438799196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 438799196 }}, "user": { "id": 392559659, "name": "27dayss..", "screen_name": "_xqueenkee", "lang": "en", "location": "null", "create_at": date("2011-10-17"), "description": "SU19|Demarcus|augustwife", "followers_count": 5086, "friends_count": 4405, "statues_count": 130485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329186042580993, "text": "@DancingABC @ABCNetwork For your consideration, actress Mindy Robinson @iheartmindy as a #DWTS contestant. Thanks! https://t.co/GumZRY360Y", "in_reply_to_status": -1, "in_reply_to_user": 16222584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DWTS" }}, "user_mentions": {{ 16222584, 631478663, 317385247 }}, "user": { "id": 512216573, "name": "Jim P. Plouff", "screen_name": "JimPlouff", "lang": "en", "location": "Great Falls, Montana", "create_at": date("2012-03-02"), "description": "If you never do, you'll never know", "followers_count": 33, "friends_count": 114, "statues_count": 386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Great Falls, MT", "id": "2c522922044e60c0", "name": "Great Falls", "place_type": "city", "bounding_box": rectangle("-111.391637,47.435796 -111.204104,47.551876") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30013, "countyName": "Cascade", "cityID": 3032800, "cityName": "Great Falls" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329186147438593, "text": "cold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 138532203, "name": "Jessie Valant", "screen_name": "Jessie_Valant", "lang": "en", "location": "Bettendorf, Iowa", "create_at": date("2010-04-29"), "description": "SAU '18 // insta+snap: jessie_valant", "followers_count": 947, "friends_count": 757, "statues_count": 20645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davenport, IA", "id": "9c1c0a45346f5837", "name": "Davenport", "place_type": "city", "bounding_box": rectangle("-90.69214,41.475836 -90.502744,41.60005") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19163, "countyName": "Scott", "cityID": 1919000, "cityName": "Davenport" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329186751385600, "text": "#GreetTheAliensIn5Words Take me to your leader! https://t.co/Br3uizLJQc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreetTheAliensIn5Words" }}, "user": { "id": 3189433165, "name": "Clark Moses", "screen_name": "ScratchycClark", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 236, "friends_count": 288, "statues_count": 3952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329186751455232, "text": "Which is a shame because this is my main form of communication with a lot of people I really enjoy talking to", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438056199, "name": "Justin Michael Muñiz", "screen_name": "TrillyOcean", "lang": "en", "location": "Corpus Christi, Texas", "create_at": date("2011-12-15"), "description": "LISTEN TO THE KIDS BRO", "followers_count": 1472, "friends_count": 1094, "statues_count": 131402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329187275743232, "text": "HAPPY BIRTHDAY BESTFRIENDDDDDDD ���������� I LOVE YOU LEGALLY @kahleadeb SHES FROM TEN-NESSEE �� https://t.co/KxXL8MoYGt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525852836 }}, "user": { "id": 377420465, "name": "Barbie Ypon", "screen_name": "xprinceeess", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "Christian Ypon", "followers_count": 507, "friends_count": 417, "statues_count": 19896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329187598823424, "text": "Wind 0.0 mph ---. Barometer 29.605 in, Steady. Temperature 31.5F. Rain today 0.00 in. Humidity 97% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 601, "friends_count": 797, "statues_count": 37040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329187653193728, "text": "When I get back to Howard I'm coming for all y'all. Fuck the Howard status quo.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.83439,42.316961"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167598842, "name": "Kwame", "screen_name": "Qwamford", "lang": "en", "location": "NJ|DC", "create_at": date("2010-07-16"), "description": "The question is not what you look at, but what you see. #HU18 #USNavy", "followers_count": 336, "friends_count": 442, "statues_count": 8376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "TSC Great Lakes", "id": "07d9e61d20885001", "name": "TSC Great Lakes", "place_type": "poi", "bounding_box": rectangle("-87.8343901,42.3169609 -87.83439,42.316961") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779293, "cityName": "Waukegan" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329187934220288, "text": "Wind 2.0 mph SE. Barometer 30.214 in, Steady. Temperature 56.4 °F. Rain today 0.00 in. Humidity 40%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 199, "friends_count": 58, "statues_count": 250156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329188076802049, "text": "@TheyHateMiguel3 don't let me find out this is your girls Twitter", "in_reply_to_status": 697329024981282816, "in_reply_to_user": 341895768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 341895768 }}, "user": { "id": 2587795440, "name": "Alexander Martinez", "screen_name": "AlexRonaIdo", "lang": "en", "location": "null", "create_at": date("2014-06-25"), "description": "Gracias a ti", "followers_count": 264, "friends_count": 208, "statues_count": 4626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-10T00:00:08.000Z"), "id": 697329188458643456, "text": "I love knifing and all, but why tf bo3 added a sword to the game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23161285, "name": "Young Wikid", "screen_name": "YoungWikid", "lang": "en", "location": "Dallas, Tx Shreveport", "create_at": date("2009-03-06"), "description": "Artist 1st ,#MFFL 2ND https://soundcloud.com/young-wikid", "followers_count": 499, "friends_count": 428, "statues_count": 23979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-10T00:00:09.000Z"), "id": 697329188890607617, "text": "Lazy days with #bae ���� https://t.co/ISRgIX0Usd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bae" }}, "user": { "id": 547337378, "name": "zafinaiaʕ•̫͡•ʔ", "screen_name": "Valeee2216", "lang": "en", "location": "null", "create_at": date("2012-04-06"), "description": "I like long walks on the beach.", "followers_count": 156, "friends_count": 249, "statues_count": 2939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Beach Gardens, FL", "id": "70b15cd26c69f608", "name": "Palm Beach Gardens", "place_type": "city", "bounding_box": rectangle("-80.177251,26.785413 -80.065807,26.8962") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1254075, "cityName": "Palm Beach Gardens" } }
+{ "create_at": datetime("2016-02-10T00:00:09.000Z"), "id": 697329188953419776, "text": "@wildboy_st3vo https://t.co/zzTYLTvYqE", "in_reply_to_status": -1, "in_reply_to_user": 541451482, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 541451482 }}, "user": { "id": 3523601654, "name": "maaay", "screen_name": "mistyangell_", "lang": "en", "location": "null", "create_at": date("2015-09-10"), "description": "null", "followers_count": 185, "friends_count": 178, "statues_count": 1015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, CA", "id": "be347aa731d4353b", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-118.193396,33.970521 -118.121279,34.019619") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 614974, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691538931867648, "text": "It’s pretty tempting to start a little side project of illustrating one @MagicRealismBot tweet per day or week or something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3701125272 }}, "user": { "id": 15444008, "name": "David Smith", "screen_name": "Catfish_Man", "lang": "en", "location": "San Francisco, California, USA", "create_at": date("2008-07-15"), "description": "http://2.bp.blogspot.com/_D_Z-D2tzi14/S_Yg4ZfcRSI/AAAAAAAAC70/-AkY__Wqt9A/s1600/rage30.png", "followers_count": 3151, "friends_count": 464, "statues_count": 31335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haight Ashbury, San Francisco", "id": "35b81fda00096353", "name": "Haight Ashbury", "place_type": "neighborhood", "bounding_box": rectangle("-122.453912,37.761484 -122.431547,37.773762") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691538982354944, "text": "Wind 0.0 mph ---. Barometer 1019.61 mb, Steady. Temperature 38.5 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691539766530048, "text": "This might be the best episode of this show I've ever seen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320596238, "name": "lyssa paige", "screen_name": "PaigeAddiction", "lang": "en", "location": "Balch Springs, TX", "create_at": date("2011-06-19"), "description": "sc: paige.addiction | SFA | ✨", "followers_count": 2550, "friends_count": 2047, "statues_count": 104719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691539951058944, "text": "Live without a Dick.\nDie with a Dick.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1018794966, "name": "No Fucks", "screen_name": "DopeRob24", "lang": "en", "location": "16408 Vine St", "create_at": date("2012-12-17"), "description": "I'd rather be carried by 6 ,than judged by 12... R.I.P W.C / K.W.\r\n Get 2 the LIMO.", "followers_count": 449, "friends_count": 261, "statues_count": 8177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691540169236480, "text": "Quinlin Stiller (Ben Stiller's son) is SO ADORABLE https://t.co/Iub7z7HV0s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453410425, "name": "Danielle Corpus", "screen_name": "Dcorpp", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-01-02"), "description": "my mom says I nap too much", "followers_count": 302, "friends_count": 364, "statues_count": 9601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691540727193600, "text": "Wind 0.5 mph NW. Barometer 30.12 in, Rising slowly. Temperature 19.0 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 23, "friends_count": 0, "statues_count": 30631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691540882194432, "text": "@bobbieann47 !!!!!!!!!!!", "in_reply_to_status": 697688835090558977, "in_reply_to_user": 3266974610, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3266974610 }}, "user": { "id": 304200000, "name": "Arix Gabriella", "screen_name": "_sheerbliss", "lang": "en", "location": "90032.", "create_at": date("2011-05-23"), "description": "SDSU", "followers_count": 364, "friends_count": 269, "statues_count": 10812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691541289115648, "text": "I am such a flop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571871377, "name": "Armandaria", "screen_name": "armandoalcalajr", "lang": "en", "location": "California", "create_at": date("2012-05-05"), "description": "setting fires until my life is burning gold", "followers_count": 228, "friends_count": 317, "statues_count": 4515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691542031499264, "text": "@supsusantoes r u serious rn?? Always in n out", "in_reply_to_status": 697690638616121344, "in_reply_to_user": 412883032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 412883032 }}, "user": { "id": 72411723, "name": "May Solomon", "screen_name": "MaySolomonn", "lang": "en", "location": "null", "create_at": date("2009-09-07"), "description": "null", "followers_count": 1080, "friends_count": 415, "statues_count": 44777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calabasas, CA", "id": "575c6d4e1b73546d", "name": "Calabasas", "place_type": "city", "bounding_box": rectangle("-118.719985,34.105958 -118.605227,34.168562") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 609598, "cityName": "Calabasas" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691542085988352, "text": "Stressed spelled backwards is desserts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2899680685, "name": "Brooke Allen", "screen_name": "Brooke_alisse", "lang": "en", "location": "null", "create_at": date("2014-11-14"), "description": "im in this world, but not of it", "followers_count": 156, "friends_count": 149, "statues_count": 847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Santa Margarita, CA", "id": "728359fcdfad2b43", "name": "Rancho Santa Margarita", "place_type": "city", "bounding_box": rectangle("-117.638013,33.585875 -117.553442,33.672953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 659587, "cityName": "Rancho Santa Margarita" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691542392164352, "text": "https://t.co/RB3Ra71lMX got em..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3319899139, "name": "kmacc36", "screen_name": "KalanAkmal", "lang": "en", "location": "Inglewood, CA", "create_at": date("2015-08-19"), "description": "brotha lynch nipsey hussel E40 game MMG ice cube yg jeezy outkast snoop Goldie Locc.. IMA west coast nigga check the tatts..", "followers_count": 7, "friends_count": 28, "statues_count": 140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691542442655744, "text": "Wind 5.6 mph WNW. Barometer 30.13 in, Steady. Temperature 35.1 °F. Rain today 0.00 in. Humidity 44%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-11T00:00:00.000Z"), "id": 697691542547378182, "text": "Yo Andrew WK is the shit https://t.co/VvhofNNBQO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335013016, "name": "Corey Vawryk", "screen_name": "Erectangular", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-07-13"), "description": "The Valkyrie chose to defend. | United States Navy | 20 | Patriot | Pittsburgh, PA ✈️ San Diego, CA", "followers_count": 845, "friends_count": 165, "statues_count": 3605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691543251980288, "text": "Love with reservations ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1136956573, "name": "Tracy", "screen_name": "Trenggggg", "lang": "en-gb", "location": "null", "create_at": date("2013-01-31"), "description": "• 19", "followers_count": 600, "friends_count": 259, "statues_count": 37690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691543432511488, "text": "02/11@03:00 - Temp 23.1F, WC 15.9F. Wind 5.7mph W, Gust 10.0mph. Bar 29.923in, Rising slowly. Rain 0.00in. Hum 67%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 46922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691543625457665, "text": "I hate when I go to McDonald's and they put napkins in my bag but dont put any love.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59913882, "name": "Luigy del Rosario", "screen_name": "LuigyIsntHipp", "lang": "en", "location": "Glenn Dale, Maryland", "create_at": date("2009-07-24"), "description": "Tweet a Kanye lyric for a RT", "followers_count": 748, "friends_count": 437, "statues_count": 107886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenn Dale, MD", "id": "79b89beb8f4af288", "name": "Glenn Dale", "place_type": "city", "bounding_box": rectangle("-76.83676,38.956291 -76.771231,39.010753") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2433400, "cityName": "Glenn Dale" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691544128614400, "text": "@ebbtideapp Tide in Dauphin Island Hydro, Alabama 02/11/2016\n Low 10:06am 0.0\nHigh 12:23am 0.4\n Low 8:19am 0.2\nHigh 3:00pm 0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-88.075,30.25"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 24, "friends_count": 1, "statues_count": 6756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 119744, "cityName": "Dauphin Island" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691544850141184, "text": "Chris Brown, Wiz Khalifa, YG, French Montana, Gizzle, and freakin P Diddy! @G_Eazy, how do you feel after a night like that?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17936793 }}, "user": { "id": 300549588, "name": "B•Chill", "screen_name": "brady_childers", "lang": "en", "location": "828", "create_at": date("2011-05-17"), "description": "Carson Newman University Golf Commit '20 Jeremiah 29:11", "followers_count": 982, "friends_count": 316, "statues_count": 16038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granite Falls, NC", "id": "01ec4d45565f540f", "name": "Granite Falls", "place_type": "city", "bounding_box": rectangle("-81.464188,35.777387 -81.384239,35.846439") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37027, "countyName": "Caldwell", "cityID": 3727420, "cityName": "Granite Falls" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691545051516928, "text": "Wind 0.0 mph NW. Barometer 30.207 in, Steady. Temperature 23.9 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 110979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691545789603841, "text": "#WineWednesday #blog post. Just made it in time! https://t.co/L0oauMSfy8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WineWednesday", "blog" }}, "user": { "id": 21243695, "name": "Bernadette Manzano", "screen_name": "bappletree", "lang": "en", "location": "SF Bay Area, CA", "create_at": date("2009-02-18"), "description": "Partner w/@dirtywatersf - Certified Bartender, Event Planner & Coordinator, Stella & Dot Stylist, Notary Public, Boston Terrier owner.", "followers_count": 1301, "friends_count": 2056, "statues_count": 21295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691546062196736, "text": "USGS reports a M1.08 #earthquake 7km N of Beaumont, California on 2/11/16 @ 7:56:30 UTC https://t.co/JhyGNF3y8n #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.9713364,33.9898338"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 5408, "friends_count": 17, "statues_count": 141620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherry Valley, CA", "id": "b3217239a57c9722", "name": "Cherry Valley", "place_type": "city", "bounding_box": rectangle("-117.001245,33.95423 -116.94099,34.0048") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612916, "cityName": "Cherry Valley" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691546095845376, "text": "Wind 0.0 mph ---. Barometer 29.667 in, Steady. Temperature 18.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691546540441600, "text": "Temp: 41.5°F Wind:0.0mph Pressure: 30.175hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 64, "friends_count": 23, "statues_count": 59602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-11T00:00:01.000Z"), "id": 697691547106566144, "text": "i'm currently sitting in my closet with me earphones in while drinking olde english", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333171281, "name": "french creole papi", "screen_name": "MilliMetoyer", "lang": "en", "location": "Alexandria, LA", "create_at": date("2011-07-10"), "description": "22. Aquarius. Hip-Hop Artist. Snapchat: milli_metoyer", "followers_count": 752, "friends_count": 303, "statues_count": 29228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691547484073984, "text": "USGS reports a M1.08 #earthquake 7km N of Beaumont, California on 2/11/16 @ 7:56:30 UTC https://t.co/3Kcd5f4vbf #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.9713364,33.9898338"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 2612506890, "name": "SoCal Earthquakes", "screen_name": "SoCalEq", "lang": "en", "location": "Southern California", "create_at": date("2014-07-08"), "description": "Tweeting every SoCal earthquake occurrence reported by USGS, within the rectangle of 32,-115.5 x 35,-121.", "followers_count": 649, "friends_count": 2, "statues_count": 16213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherry Valley, CA", "id": "b3217239a57c9722", "name": "Cherry Valley", "place_type": "city", "bounding_box": rectangle("-117.001245,33.95423 -116.94099,34.0048") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612916, "cityName": "Cherry Valley" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691547912048640, "text": "Wind 0.0 mph ---. Barometer 30.235 in, Rising slowly. Temperature 16.8 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 26, "statues_count": 17558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691548260020224, "text": "Pretty good day.\nTime to grind tomorrow, need to catch up in school.\nSo I'm going to bed now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2543830363, "name": "Juan Figueroa", "screen_name": "Penguin954", "lang": "en", "location": "null", "create_at": date("2014-06-03"), "description": "null", "followers_count": 129, "friends_count": 167, "statues_count": 789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691549035958272, "text": "I need someone rii quick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247091784, "name": "Miss Pam Anderson", "screen_name": "Titsandlickher", "lang": "en", "location": "Kemet", "create_at": date("2011-02-03"), "description": "null", "followers_count": 7154, "friends_count": 132, "statues_count": 50071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691549144997888, "text": "@gnarlystewart https://t.co/vE4ZpnUWSj", "in_reply_to_status": -1, "in_reply_to_user": 1002699818, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1002699818 }}, "user": { "id": 4038424093, "name": "kacie seims", "screen_name": "kacie_seims", "lang": "en", "location": "null", "create_at": date("2015-10-27"), "description": "saved by grace and puns", "followers_count": 73, "friends_count": 126, "statues_count": 291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691549958713349, "text": "2/11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3255244394, "name": "Pablo", "screen_name": "YB_Andrew12", "lang": "en", "location": "sc: eandrew_12", "create_at": date("2015-06-24"), "description": "Peace Love Positivity", "followers_count": 301, "friends_count": 294, "statues_count": 5265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691550722052096, "text": "Desde niño desmadroso pero la vida me goso������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 319314587, "name": "Chanaboo.", "screen_name": "chanaaboo", "lang": "en", "location": "null", "create_at": date("2011-06-17"), "description": "Rubencillo❤", "followers_count": 287, "friends_count": 320, "statues_count": 4872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691550726225921, "text": "temperature down 10°F -> 8°F\nhumidity up 51% -> 54%\nwind 6mph -> 9mph\npressure 30.18in rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.91201,42.8884"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237976548, "name": "Oak Creek Weather", "screen_name": "OakCreekWI", "lang": "en", "location": "Oak Creek, WI", "create_at": date("2011-01-13"), "description": "Weather updates, forecast, warnings and information for Oak Creek, WI. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 56, "friends_count": 12, "statues_count": 19531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Creek, WI", "id": "05d07f0891432eaa", "name": "Oak Creek", "place_type": "city", "bounding_box": rectangle("-87.951617,42.83611 -87.842053,42.930439") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5558800, "cityName": "Oak Creek" } }
+{ "create_at": datetime("2016-02-11T00:00:02.000Z"), "id": 697691550994849792, "text": "Wind 2.2 mph W. Barometer 29.73 in, Rising slowly. Temperature 23.5 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 108, "statues_count": 157409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691552353636352, "text": "18 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 355219282, "name": ".", "screen_name": "FuckJaaay", "lang": "en", "location": "onta", "create_at": date("2011-08-14"), "description": "#rareaf // @cindyyluuu❤", "followers_count": 491, "friends_count": 358, "statues_count": 33973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691553020522496, "text": "#art #zentangle #drawing #anxiety #therapy #arttherapy #meditation #meditativedrawing #WhyIDraw https://t.co/dmOugpnndc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "art", "zentangle", "drawing", "anxiety", "therapy", "arttherapy", "meditation", "meditativedrawing", "WhyIDraw" }}, "user": { "id": 123954494, "name": "Sierra Sugar", "screen_name": "SierraSugar", "lang": "en", "location": "null", "create_at": date("2010-03-17"), "description": "Blogger & writer exploring the world on 18 wheels. Lover of music, movies, photography, steampunk, zentangle art, cooking, nature, pets, and all things kitty!", "followers_count": 598, "friends_count": 390, "statues_count": 5532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691553729368064, "text": "2016's greatest mystery: Who put so much cabbage in this toilet? https://t.co/oBZNc37K5d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17603, "friends_count": 17565, "statues_count": 68317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691553763090432, "text": "@MaryJohnXVX HAPPY BIRTHDAY PAL", "in_reply_to_status": 697686363710160896, "in_reply_to_user": 1554918469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1554918469 }}, "user": { "id": 40644144, "name": "Kyle Timothy", "screen_name": "KyleLusty", "lang": "en", "location": "Monson, MA ✈️ Portland, OR", "create_at": date("2009-05-17"), "description": "Teenage Angst Has Paid Off Well. Generation Nothing. @churchtonguein #DieWild", "followers_count": 722, "friends_count": 337, "statues_count": 9934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691554006200320, "text": "Can you recommend anyone for this #job? Care Manager/CNA - https://t.co/xKYuXYMpl1 #Carmel, IN #Nursing #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1180435,39.978371"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Carmel", "Nursing", "Hiring" }}, "user": { "id": 2327706649, "name": "Sunrise Careers", "screen_name": "Sunrise_Careers", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "We are always seeking exceptional people with a passion for working with seniors, and dedicated to caring for others.", "followers_count": 361, "friends_count": 9, "statues_count": 3038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmel, IN", "id": "eca35cbd75f0a1e6", "name": "Carmel", "place_type": "city", "bounding_box": rectangle("-86.241245,39.926048 -86.023652,40.000451") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1810342, "cityName": "Carmel" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691554241097728, "text": "@WORLDSTARC0MEDY @afales1 keep it tight brit!!!!", "in_reply_to_status": 697664393509629953, "in_reply_to_user": 1321629762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1321629762, 110743536 }}, "user": { "id": 992196181, "name": "Julie Simmons", "screen_name": "jujusimmonss", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2012-12-05"), "description": "social media @OrangetheorySM by day // fashion student @FIDM by night. IG: @jujusimmons", "followers_count": 259, "friends_count": 248, "statues_count": 582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691554446778369, "text": "Lmfao ������ https://t.co/Vhwi3FVaLi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23161285, "name": "Young Wikid", "screen_name": "YoungWikid", "lang": "en", "location": "Dallas, Tx Shreveport", "create_at": date("2009-03-06"), "description": "Artist 1st ,#MFFL 2ND https://soundcloud.com/young-wikid", "followers_count": 498, "friends_count": 428, "statues_count": 23982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691554731921408, "text": "I don't understand how you go somewhere where you are literally not wanted... Like how does that work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259872649, "name": "Grant", "screen_name": "Bigplaygk", "lang": "en", "location": "Erie, PA", "create_at": date("2011-03-02"), "description": "My 15 minutes started an hour ago | #GU18 | | Keep Reaching Up |", "followers_count": 1619, "friends_count": 652, "statues_count": 89512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Erie, PA", "id": "29aaa88d9fe74b50", "name": "Erie", "place_type": "city", "bounding_box": rectangle("-80.239991,42.018414 -79.934073,42.202992") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4224000, "cityName": "Erie" } }
+{ "create_at": datetime("2016-02-11T00:00:03.000Z"), "id": 697691555360931840, "text": "���� https://t.co/BzP6deKTlI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2899788692, "name": "jaquelin espinoza", "screen_name": "Jaquelinivon", "lang": "en", "location": "akransas", "create_at": date("2014-11-14"), "description": "null", "followers_count": 365, "friends_count": 267, "statues_count": 1773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McGehee, AR", "id": "37165d3c5f44941f", "name": "McGehee", "place_type": "city", "bounding_box": rectangle("-91.426998,33.598508 -91.34867,33.654418") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5041, "countyName": "Desha", "cityID": 542770, "cityName": "McGehee" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691555939905536, "text": "Nope. https://t.co/yCpUuOws0U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315348249, "name": "bo$$tickbitch.", "screen_name": "Queen_bostick", "lang": "en", "location": "null", "create_at": date("2013-03-29"), "description": "am i to real for this shit?", "followers_count": 1000, "friends_count": 545, "statues_count": 24461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691555969171456, "text": "@TripleDee_3 @buckbone_ https://t.co/Dr8YYZKsAX", "in_reply_to_status": -1, "in_reply_to_user": 323084660, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 323084660, 389917065 }}, "user": { "id": 2681116003, "name": "Corey The Great. ™", "screen_name": "CoreyTheGreat4", "lang": "en", "location": "Rosenberg, TX", "create_at": date("2014-07-25"), "description": "Family over everything.", "followers_count": 723, "friends_count": 672, "statues_count": 14671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosenberg, TX", "id": "00f2b6161655c5cc", "name": "Rosenberg", "place_type": "city", "bounding_box": rectangle("-95.835024,29.482424 -95.699766,29.581688") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4863284, "cityName": "Rosenberg" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691555985920000, "text": "@Rauldelgado14 @oscarmj17 when the girls flake on us on the daily now and we don't know if we're meeting up or not.. https://t.co/SFH2uZtzmg", "in_reply_to_status": -1, "in_reply_to_user": 433193242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 433193242, 3568442898 }}, "user": { "id": 635888517, "name": "JT19", "screen_name": "Javi_beast19", "lang": "en", "location": "Chico, CA", "create_at": date("2012-07-14"), "description": "100% Chivista De Corazon #NoTeRajes \n\nStudent at the University of Chico Justine Busse❤", "followers_count": 366, "friends_count": 328, "statues_count": 14860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691556707348481, "text": "when someone tries to cat call u https://t.co/brxLSpp77z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1942836386, "name": "cassandra", "screen_name": "cxssxndra", "lang": "en", "location": "bay area", "create_at": date("2013-10-06"), "description": "i love @aalyssataralynn", "followers_count": 768, "friends_count": 369, "statues_count": 5709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691557198217216, "text": "now that everyone is asleep I am confessing I lost the care package my mom sent me and couldn't even open the valentines card #WorstSonEver", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WorstSonEver" }}, "user": { "id": 338119178, "name": "Nemo", "screen_name": "FeelingNeishh", "lang": "en", "location": "Haymarket", "create_at": date("2011-07-18"), "description": "Professional hip hop music listener #RU19", "followers_count": 571, "friends_count": 447, "statues_count": 18363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Radford, VA", "id": "a265d5d344dbd61d", "name": "Radford", "place_type": "city", "bounding_box": rectangle("-80.595589,37.090022 -80.513486,37.146017") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51750, "countyName": "Radford", "cityID": 5165392, "cityName": "Radford" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691557374382084, "text": "\"You are never going to be able to stop yourself from screaming your love from rooftops, you will never be able to play it cool.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33399610, "name": "kmoney", "screen_name": "kposchibble", "lang": "en", "location": "millikin university |ΑΧΩ| '18", "create_at": date("2009-04-19"), "description": "pizza", "followers_count": 676, "friends_count": 585, "statues_count": 50496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, IL", "id": "9c908d38b1b53897", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-89.028906,39.785267 -88.844461,39.923903") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17115, "countyName": "Macon", "cityID": 1718823, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691557491769344, "text": "Could literally hear a pin drop in this building right now.. it's nice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185814806, "name": "sleepy eyes", "screen_name": "masterofkittens", "lang": "en", "location": "Boston, MA", "create_at": date("2010-09-01"), "description": "not so multi-talented musician and dental hygiene student. big weeb and novice smash player.", "followers_count": 84, "friends_count": 189, "statues_count": 2029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, MA", "id": "b76630ca2b2c4158", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-71.270426,42.283041 -71.156939,42.367744") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2545560, "cityName": "Newton" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691557911261186, "text": "#SuspiciousVehicle at 7000-7499 W Colonial Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4801347,28.5521514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousVehicle", "orlpol", "ocso" }}, "user": { "id": 39133730, "name": "Police Calls 32818", "screen_name": "orlpol32818", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 48, "friends_count": 1, "statues_count": 8437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504956,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691557965619200, "text": "@Zeusayala1 @JBizz_ yeah the same ones he puts on his story smh", "in_reply_to_status": 697691397109915649, "in_reply_to_user": 550533737, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 550533737, 1021409880 }}, "user": { "id": 2592402824, "name": "The dingo", "screen_name": "mannymjjimenez", "lang": "en", "location": "null", "create_at": date("2014-06-27"), "description": "LeavetheLefthandpath/$loppy wri$t/ am ❤️", "followers_count": 393, "friends_count": 371, "statues_count": 6621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691558166986752, "text": "Temp: 5.8°F - Dew Point: -7.2° - Wind: 7.6 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 30.16in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691558506840065, "text": "just up...��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596034687, "name": "Addie Hughes", "screen_name": "addiemaehughes", "lang": "en", "location": "Columbus, GA", "create_at": date("2012-05-31"), "description": "Forever in our hearts EHR ❤️", "followers_count": 378, "friends_count": 535, "statues_count": 7564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-11T00:00:04.000Z"), "id": 697691558951280640, "text": "My favourite pillow is at Dillons house and I don't like sleeping without it do you see my predicament", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 881474599, "name": "Czar Beastlychuk △̵", "screen_name": "Beastlychuk", "lang": "en-gb", "location": "Coruscant", "create_at": date("2012-10-14"), "description": "Provehito in Altum • Since Forever • Ukraine • Yay God • Guitarist for @WeAreANH • ₪ ø lll ·o.", "followers_count": 1030, "friends_count": 1021, "statues_count": 44249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691559957925888, "text": "Should I just say yolo and get my ears pierced tomorrow?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397015573, "name": "Tᴙavisひ", "screen_name": "TravyTheGoat", "lang": "en", "location": "West side of the galaxy ", "create_at": date("2011-10-23"), "description": "Life's a garden, every good man needs a hoe to dig it #UncleTwix #GoatGang #ZumiezLyfe", "followers_count": 367, "friends_count": 99, "statues_count": 19556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691560532508672, "text": "my nigga Wayne be going crazy!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350456242, "name": "feelikejojowitdatool", "screen_name": "SBNellBone", "lang": "en", "location": "westlawn doughboyz we gettin $", "create_at": date("2011-08-07"), "description": "#FreeJames #ThePearlAcamady free da guys til dey all free! #staingang#3⃣ SC:sbnellbone Ig:nellbone", "followers_count": 1873, "friends_count": 1625, "statues_count": 42275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691560750751744, "text": "Dude* https://t.co/mwEXIVsRJB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1376846335, "name": "Kennedy", "screen_name": "_Kle0patra_", "lang": "en", "location": "North East Z6", "create_at": date("2013-04-24"), "description": "Mind your business. #bey6", "followers_count": 2862, "friends_count": 2072, "statues_count": 136080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691560767528960, "text": "@J_Swiger never", "in_reply_to_status": 697691407818088448, "in_reply_to_user": 333091615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333091615 }}, "user": { "id": 243268864, "name": "Dravin Richards", "screen_name": "AlLDaYDrA", "lang": "en", "location": " WV", "create_at": date("2011-01-26"), "description": "✨Small Town, Big Dreams✨ R.I.P DAD #Lakers #Vikings #WVU #Music #MPBoyz #BDA304", "followers_count": 943, "friends_count": 745, "statues_count": 19539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksburg, WV", "id": "c777b45fa7b82290", "name": "Clarksburg", "place_type": "city", "bounding_box": rectangle("-80.381808,39.262118 -80.277572,39.313976") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54033, "countyName": "Harrison", "cityID": 5415628, "cityName": "Clarksburg" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691560843079681, "text": "Barberton Oh Temp:12.1°F Wind:5 mph Dir:W Baro:Rising slowly Rain2day:0.00in Hum:77% UV:0.0 @ 03:00 02/11/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691560910184448, "text": "All you do is judge me and tell me what you don't like about me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1252982342, "name": "Devynn", "screen_name": "littlefrankieo", "lang": "en", "location": "prolly at work", "create_at": date("2013-03-08"), "description": "not even close to where I want to be..but I'm working towards it.", "followers_count": 450, "friends_count": 580, "statues_count": 7062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691561375608832, "text": "@AlondraaNykolee lmao just gotta do it. No hesitation", "in_reply_to_status": 697691307808927745, "in_reply_to_user": 613033370, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 613033370 }}, "user": { "id": 81954897, "name": "Beige the Gemini", "screen_name": "SilasTheVifth", "lang": "en", "location": "Chicago, AZ", "create_at": date("2009-10-12"), "description": "Somewhere between a logical writer and a madman with a pen.", "followers_count": 2368, "friends_count": 904, "statues_count": 142158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691561748860930, "text": "@royalmroreo wya", "in_reply_to_status": 697691493029515264, "in_reply_to_user": 713892600, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2956546214 }}, "user": { "id": 713892600, "name": "ROB", "screen_name": "roberte1996", "lang": "en", "location": "Upland, CA", "create_at": date("2012-07-24"), "description": "IT IS WHAT IT IS", "followers_count": 503, "friends_count": 416, "statues_count": 18267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691561908314112, "text": "John Smith and Pocahontas Romantic kiss�� https://t.co/mRUhxhla7E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28067841, "name": "Delana", "screen_name": "Azarine", "lang": "en", "location": "null", "create_at": date("2009-04-01"), "description": "null", "followers_count": 308, "friends_count": 867, "statues_count": 8221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691562210230273, "text": "Me & Reece having a conversation on toast...of all things ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3434082194, "name": ":-)", "screen_name": "jeeennaaar", "lang": "en", "location": "null", "create_at": date("2015-09-02"), "description": "ig: jjennz ♏️", "followers_count": 216, "friends_count": 215, "statues_count": 2464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilo, HI", "id": "00430f8519d62c37", "name": "Hilo", "place_type": "city", "bounding_box": rectangle("-155.184552,19.656471 -154.990994,19.738955") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii", "cityID": 1514650, "cityName": "Hilo" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691562432548867, "text": "@faithmares is it a guy ?", "in_reply_to_status": 697689480887558144, "in_reply_to_user": 540003962, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 540003962 }}, "user": { "id": 464394449, "name": "Isaac", "screen_name": "s14_isaac", "lang": "en", "location": "Houston, tx", "create_at": date("2012-01-14"), "description": "#HalaMadrid. own: s14 zenki", "followers_count": 126, "friends_count": 84, "statues_count": 5221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691562789249024, "text": "I hate @CaptainCoIIins but he's my mans so it's cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 520969746 }}, "user": { "id": 556506628, "name": "Oligary", "screen_name": "YaBoyOlive", "lang": "en", "location": "Allen Park", "create_at": date("2012-04-17"), "description": "8.12.14", "followers_count": 434, "friends_count": 418, "statues_count": 7990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln Park, MI", "id": "1769b19bf68bc543", "name": "Lincoln Park", "place_type": "city", "bounding_box": rectangle("-83.202957,42.220851 -83.155381,42.271565") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2647800, "cityName": "Lincoln Park" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691563388960770, "text": "Selisha better not mess with that heat ���� because it's tooo cold in here for all of that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51830765, "name": "Nya Andreece", "screen_name": "_lovinggmee", "lang": "en", "location": "null", "create_at": date("2009-06-28"), "description": "Putting God First and Letting the Rest Follow\n#NCAT19", "followers_count": 765, "friends_count": 1166, "statues_count": 5876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-11T00:00:05.000Z"), "id": 697691563615326208, "text": "HAPPY BIRTHDAY (pt. 1) to this dime right here !! ��������❤️❤️ I love you @2melissa02 https://t.co/7ukDjr0G4l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2840661475 }}, "user": { "id": 1321718166, "name": "Scott Wilhite", "screen_name": "WilhiteScott", "lang": "en", "location": "Bompton, CA", "create_at": date("2013-04-01"), "description": "I gotta ask ma❤️", "followers_count": 233, "friends_count": 218, "statues_count": 797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691564152373248, "text": "Probs getting no sleep tonight . Bout to be super secluded for the next couple weeks . Greatness is coming . Thanks .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340988959, "name": "DJ TOPGUN", "screen_name": "djtopgun_", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-07-23"), "description": "| Booking: topgunorseno@gmail.com | LET'S LIVE . | Official DJ for my friends . |", "followers_count": 1254, "friends_count": 253, "statues_count": 20555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Ridgeville, OH", "id": "dba7a6611427dc73", "name": "North Ridgeville", "place_type": "city", "bounding_box": rectangle("-82.06917,41.346019 -81.970054,41.418908") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3956966, "cityName": "North Ridgeville" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691564714295296, "text": "The winner of the @KoleCalhoun bobblehead is @I_am_MTG congrats GOAT. https://t.co/QXDHF5wh91", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48255362, 347659913 }}, "user": { "id": 34560639, "name": "Mario Benitez", "screen_name": "Mar_e_o17", "lang": "en", "location": "Anaheim", "create_at": date("2009-04-23"), "description": "Blessed above all, Glory to God! , Anaheim native, The mouse pays my bills, love my city, love my family ✝.", "followers_count": 2520, "friends_count": 2325, "statues_count": 196252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691565028847616, "text": "Boooooooo �� @Saloteiaaaa 's fault ������ https://t.co/99vy5tKhil", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 307707496 }}, "user": { "id": 1918114166, "name": "tiana v. gago", "screen_name": "_faamanuiaga", "lang": "en", "location": "southside ", "create_at": date("2013-09-29"), "description": "laie ✈ cali \r\nreѕт eaѕy lolo m.", "followers_count": 220, "friends_count": 212, "statues_count": 7164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691565691662336, "text": "Hello all! @ Sunrise Neighborhood https://t.co/nv9QEsuAzL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2594117,28.6580802"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 811412269, "name": "Polarized RayBans", "screen_name": "nukeskylarker", "lang": "en", "location": "Winter Springs, Florida", "create_at": date("2012-09-08"), "description": "If nothing changes, nothing changes.", "followers_count": 43, "friends_count": 135, "statues_count": 2677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1278325, "cityName": "Winter Springs" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691565892853762, "text": "Once I get my car its gonna be lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152537884, "name": "Vaggaa", "screen_name": "_marinnaa", "lang": "en", "location": "another place called earth ", "create_at": date("2010-06-05"), "description": "THUGGA YSL", "followers_count": 434, "friends_count": 268, "statues_count": 6988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691565930762240, "text": "My late night cravings include pizza, what about yours? @ Zoom Pizza Factory https://t.co/yAh2Lx9Rsu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6630707,41.9204903"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3088173817, "name": "Van Mac", "screen_name": "vansventures", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Lover of food and Chicago. Blogger and explorer. Follow my Instagram @vansventures and check out my website! http://www.vansventures.com", "followers_count": 240, "friends_count": 213, "statues_count": 1277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691566199037952, "text": "Your #MCM Don't even know your real name ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MCM" }}, "user": { "id": 4488385933, "name": "Chentedimaria", "screen_name": "Chentedimaria", "lang": "en", "location": "null", "create_at": date("2015-12-07"), "description": "ARRIBA LA AMERICA ! #Neymardesilva", "followers_count": 83, "friends_count": 90, "statues_count": 1314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691566232612864, "text": "They just words but they cut deep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108203307, "name": "Texas Trill", "screen_name": "RareVersace", "lang": "en", "location": "STILL RECLINING", "create_at": date("2010-01-24"), "description": "null", "followers_count": 636, "friends_count": 520, "statues_count": 83816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beltsville, MD", "id": "eac9405683a07098", "name": "Beltsville", "place_type": "city", "bounding_box": rectangle("-76.962079,39.014758 -76.877619,39.065391") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2406400, "cityName": "Beltsville" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691566488494080, "text": "We're #hiring! Click to apply: Cath Lab RN, per diem - https://t.co/QVAIJ9JC67 #Healthcare #Burlington, MA #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.195611,42.5048167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Healthcare", "Burlington", "Job", "Jobs" }}, "user": { "id": 3011627064, "name": "Lahey Careers", "screen_name": "LaheyCareers", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "null", "followers_count": 43, "friends_count": 32, "statues_count": 439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691566681362432, "text": "Temp: 13.1F W C: 5.7F Wind:ENE at 3.8kts Baro: 1023.8mb and Rising slowly Rain today: 0.00in R H: 89% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 189, "friends_count": 219, "statues_count": 96754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691566790602752, "text": "I hate Boyer so god damn much.!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390541974, "name": "Trent Leverenz", "screen_name": "trentleverenz", "lang": "en", "location": "Two-Six-Zero", "create_at": date("2011-10-13"), "description": "life has its uphill battles, stay positive. I got my own relationship with God.", "followers_count": 830, "friends_count": 453, "statues_count": 31732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garrett, IN", "id": "9cfc88bb15fa0697", "name": "Garrett", "place_type": "city", "bounding_box": rectangle("-85.146834,41.308285 -85.107712,41.367509") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18033, "countyName": "DeKalb", "cityID": 1826386, "cityName": "Garrett" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691567725764608, "text": "46.6F (Feels: 46.6F) - Humidity: 96% - Wind: 4.5mph SW - Gust: 5.4mph - Pressure: 1034.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-11T00:00:06.000Z"), "id": 697691567973216256, "text": "always stay gracious, the best revenge is yo paper ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58520652, "name": "¥", "screen_name": "blkaf_", "lang": "en", "location": "null", "create_at": date("2009-07-20"), "description": "hi, I'm Quon", "followers_count": 2655, "friends_count": 1764, "statues_count": 96950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camilla, GA", "id": "3fee78a64ed6a189", "name": "Camilla", "place_type": "city", "bounding_box": rectangle("-84.231758,31.199989 -84.169027,31.249508") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13205, "countyName": "Mitchell", "cityID": 1312624, "cityName": "Camilla" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691568552017920, "text": "Have to wake up in 5 hours. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3156918744, "name": "Kay Simmons", "screen_name": "_cassonade", "lang": "en", "location": "null", "create_at": date("2015-04-14"), "description": "happiness already lies within you. UCA 17.\n1.3.14 R.I.P ❤️", "followers_count": 53, "friends_count": 65, "statues_count": 948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691568736747520, "text": "Wind 0 mph ---. Barometer 1012.1 hPa, Falling slowly. Temperature 70.5 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 145231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, PR", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691569151840256, "text": "@geezyyceja chill, you don't enjoy his music like that", "in_reply_to_status": 697690669100347392, "in_reply_to_user": 3181490054, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3181490054 }}, "user": { "id": 240977510, "name": "Josue", "screen_name": "Josuee_ener", "lang": "en", "location": "Los Angeles", "create_at": date("2011-01-20"), "description": "21 | NE Patriots", "followers_count": 324, "friends_count": 208, "statues_count": 47249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691569256853504, "text": "Where is Sacramento on the map? Play the game at https://t.co/d47F4ALPzY #Sacramento", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.494,38.5816"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sacramento" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1033, "friends_count": 311, "statues_count": 2586070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691570145988609, "text": "@_whateverkia ����", "in_reply_to_status": 697665481625309184, "in_reply_to_user": 125892205, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 125892205 }}, "user": { "id": 27374061, "name": "bria", "screen_name": "callmeBREE_", "lang": "en", "location": "_/*\\_ TEXA$", "create_at": date("2009-03-28"), "description": "Rest in Paradise Aubree . Unt. ❄️❄️ Lucky Lefty #LongLiveClyde", "followers_count": 2039, "friends_count": 921, "statues_count": 239392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691570330402816, "text": "2/11/2016 - 02:00\nTemp: 32.2F \nHum: 87%\nWind: 0.0 mph\nBaro: 30.001in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 53, "statues_count": 49852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691570364092416, "text": "Temp 12.4°F Wind Chill 5.2°F RH 86% Wind 4.1 W Gust 8.0 W SLP 30.206 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 105, "friends_count": 63, "statues_count": 33181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691570577879041, "text": "@Ookayx ur so hawt ������", "in_reply_to_status": -1, "in_reply_to_user": 821119939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 821119939 }}, "user": { "id": 447846131, "name": "Ohh.Youu.Keyyy", "screen_name": "_Oyukiiiii", "lang": "en", "location": "Tx.", "create_at": date("2011-12-27"), "description": "Houston Raver ❤️", "followers_count": 1296, "friends_count": 533, "statues_count": 79226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hitchcock, TX", "id": "5db4d2b2d277dc8e", "name": "Hitchcock", "place_type": "city", "bounding_box": rectangle("-95.060909,29.328479 -94.945485,29.375215") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4834220, "cityName": "Hitchcock" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691570988937216, "text": "My stomach hurting ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506420712, "name": "CUWBFA➕", "screen_name": "Breyanniaa", "lang": "en", "location": "$outhside of Monroe, LA", "create_at": date("2012-02-27"), "description": "REST IN PEACE OD & ROSE REED", "followers_count": 1946, "friends_count": 1516, "statues_count": 71607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richwood, LA", "id": "00760a4d427dfce5", "name": "Richwood", "place_type": "city", "bounding_box": rectangle("-92.105769,32.437501 -92.048598,32.472064") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2264660, "cityName": "Richwood" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691571605516289, "text": "@JessGlynne killed it tonight. SO IN LOVE https://t.co/kuuuJhPNiK", "in_reply_to_status": -1, "in_reply_to_user": 92625607, "favorite_count": 0, "coordinate": point("-117.070642,32.758917"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 92625607 }}, "user": { "id": 176775622, "name": "Ed", "screen_name": "EdGabe95", "lang": "en", "location": "San Diego, CA / Manchester, UK", "create_at": date("2010-08-10"), "description": "20 year old student, driven and determined. Studying abroad at UC San Diego 2015/16. I was the kid who held on to the pass the parcel for too long", "followers_count": 393, "friends_count": 1032, "statues_count": 4561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Campus Village Apartments", "id": "07d9f23087081001", "name": "Campus Village Apartments", "place_type": "poi", "bounding_box": rectangle("-117.0706421,32.758916899999996 -117.070642,32.758917") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691571907461120, "text": "Happy birthday turd �� you're legal now ���� I love you so much ❤️@steezylembreezy https://t.co/UycoHwomPJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3721459572 }}, "user": { "id": 839305124, "name": "Victoria Ann", "screen_name": "Victoria_Stodds", "lang": "en", "location": "Reno, NV", "create_at": date("2012-09-22"), "description": "UNR❤️ Criminal Justice Major", "followers_count": 581, "friends_count": 551, "statues_count": 7111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691572133974017, "text": "I feel like your always subtweeting me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2328281071, "name": "Ava", "screen_name": "AvaAvahand", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "Northern California", "followers_count": 200, "friends_count": 263, "statues_count": 444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691572150755328, "text": "Forgive, but never go back https://t.co/7yckaZrGYC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543577551, "name": "LucASStankuss", "screen_name": "its_Stankuss", "lang": "en", "location": "CPP BRONCOS", "create_at": date("2012-04-02"), "description": "living like a bronco, lifestyle wild and untamed", "followers_count": 817, "friends_count": 514, "statues_count": 27253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-11T00:00:07.000Z"), "id": 697691572180275200, "text": "@MeekMill leave that internet shit to the dweebs boa! Nikki and Ms >>>>>>>> #Salute", "in_reply_to_status": -1, "in_reply_to_user": 20567939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Salute" }}, "user_mentions": {{ 20567939 }}, "user": { "id": 177721685, "name": "Rich", "screen_name": "1of1kingshit", "lang": "en", "location": "ÜT: 41.910446,-87.744117", "create_at": date("2010-08-12"), "description": "All Money IN 25-8 GRIND", "followers_count": 203, "friends_count": 115, "statues_count": 27315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-11T00:00:08.000Z"), "id": 697691573295800320, "text": "#Sales alert: Call Center Supervisor | DialAmerica | #Jacksonville, FL https://t.co/WtmVfpC5UC #milspouse #jobs https://t.co/UD5U0Mt8Yr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5537192,30.2205397"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "hashtags": {{ "Sales", "Jacksonville", "milspouse", "jobs" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 314, "friends_count": 215, "statues_count": 2066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mandarin, FL", "id": "27424e47b9946999", "name": "Mandarin", "place_type": "neighborhood", "bounding_box": rectangle("-81.692116,30.103771 -81.433082,30.25731") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-02-11T00:00:08.000Z"), "id": 697691573388070914, "text": "Convinced Shivani to watch the bachelor so so proud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2532524389, "name": "Jaisy", "screen_name": "jaisy_c", "lang": "en", "location": "AGGIE BLESSUP ", "create_at": date("2014-05-29"), "description": "everyday is a new adventure (so cliché, I know)", "followers_count": 695, "friends_count": 568, "statues_count": 19479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galveston, TX", "id": "632eeebc87aecd57", "name": "Galveston", "place_type": "city", "bounding_box": rectangle("-94.880809,29.239602 -94.764742,29.335548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4828068, "cityName": "Galveston" } }
+{ "create_at": datetime("2016-02-11T00:00:08.000Z"), "id": 697691573803290624, "text": "@MarvinFlores15 does that mean you're makin me breakfast in the morning? ��������", "in_reply_to_status": 697691275718307840, "in_reply_to_user": 217337512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 217337512 }}, "user": { "id": 4633412382, "name": "Little C.", "screen_name": "chiquitaah_", "lang": "en", "location": "San Pedro, Los Angeles", "create_at": date("2015-12-22"), "description": "null", "followers_count": 152, "friends_count": 238, "statues_count": 747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-02-11T00:00:08.000Z"), "id": 697691573882982400, "text": "Jk well idk maybe I'm just exhausted & a little under the weather.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173988218, "name": "Gaby Jacinto", "screen_name": "jacinto_gaby", "lang": "en", "location": "null", "create_at": date("2013-02-12"), "description": "Sc: gabyylove", "followers_count": 148, "friends_count": 269, "statues_count": 15124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-11T00:00:08.000Z"), "id": 697691573916598272, "text": "EHEEEEE https://t.co/XbNcj84lGa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 885426769, "name": "♕ΩVΩ♕", "screen_name": "OVOaleXO", "lang": "en", "location": "The 6IX", "create_at": date("2012-10-16"), "description": "I like my O's with a V in the middle", "followers_count": 978, "friends_count": 764, "statues_count": 34922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053927367999488, "text": "How would that make anyone a god? Wouldn't it help us understand God better and increase likelihood of belief in it? https://t.co/SIYoeMbYxj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17502089, "name": "Religulous", "screen_name": "religulous", "lang": "en", "location": "Heaven (aka Iowa)", "create_at": date("2008-11-19"), "description": "Ret. bio teacher, atheist, humanist. I study science, nature, philosophy, religion. Read my quotes, then think about them. Religion+ridiculous = Religulous", "followers_count": 25370, "friends_count": 2045, "statues_count": 51346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coralville, IA", "id": "1c70a89b85693461", "name": "Coralville", "place_type": "city", "bounding_box": rectangle("-91.642268,41.666511 -91.528113,41.730042") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1916230, "cityName": "Coralville" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053927502266368, "text": "This is what happens when I procrastinate.... Thoughts? �� https://t.co/7OzhZ3Hf80", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41274756, "name": "♕Manuela Alcalá", "screen_name": "ManuAlcalaS", "lang": "en", "location": "The Evil Queen's Castle, EF", "create_at": date("2009-05-19"), "description": "I'm a TV student living in Chicago, a globetrotter & an avid TV watcher, hopefully I'll work on a show someday... #EvilRegal #OQ #Outlander", "followers_count": 924, "friends_count": 976, "statues_count": 29532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053927787343872, "text": "@DA_Stockman hi David...give us some deep dirt...major scoop x", "in_reply_to_status": -1, "in_reply_to_user": 2344061090, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2344061090 }}, "user": { "id": 2810009016, "name": "G Pacella", "screen_name": "pet1713", "lang": "en", "location": "Houston TX", "create_at": date("2014-09-14"), "description": "Sports Fan/Pgh native/Italian American/ex Club Med GO Ex Eastern Airlines Flight Attendant Catholic Aquarius✌❤", "followers_count": 3332, "friends_count": 4998, "statues_count": 101107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053927825121280, "text": "@JCageJr_ ����������", "in_reply_to_status": 698053554762883072, "in_reply_to_user": 248817112, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 248817112 }}, "user": { "id": 300005922, "name": "A-Killa", "screen_name": "_Okaykaykay", "lang": "en", "location": "Dallas , Texas", "create_at": date("2011-05-16"), "description": "IG: _acaciadonyell | SC: oowee_acacia I DONT BUY $15 GRAMS. #ThatIsAll", "followers_count": 2173, "friends_count": 1167, "statues_count": 49098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053927829307393, "text": "I'm so excited for tomorrow!!! I'm babysitting the most adorable children on planet earth!!!! And I'm going to a spa!! Yay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2942395338, "name": "Angelique Mangoutas", "screen_name": "AngeliqueMango1", "lang": "en", "location": "null", "create_at": date("2014-12-24"), "description": "Can my Prince Charming hurry it up......Please?!", "followers_count": 59, "friends_count": 175, "statues_count": 908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053928345276416, "text": "If anyone is up hmu cuz i cant sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338207108, "name": "Doobie®", "screen_name": "Carson_DuBois", "lang": "en", "location": "Kennewick, Wa", "create_at": date("2011-07-19"), "description": "Let's get up early and stay out late.\nSeahawks and Mariners since 03'", "followers_count": 510, "friends_count": 639, "statues_count": 3543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennewick, WA", "id": "63d0671506628c8d", "name": "Kennewick", "place_type": "city", "bounding_box": rectangle("-119.264818,46.162262 -119.087063,46.236321") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5335275, "cityName": "Kennewick" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053928466821121, "text": "She like this the life I'm telling her this just another night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2747501854, "name": "PABLO PAPI CHULO", "screen_name": "pablo_dagoat", "lang": "en", "location": "Minnesota, USA", "create_at": date("2014-08-19"), "description": "I Think About My Death & Success The Most", "followers_count": 710, "friends_count": 786, "statues_count": 17 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Owatonna, MN", "id": "015604dc11d04ba0", "name": "Owatonna", "place_type": "city", "bounding_box": rectangle("-93.275728,44.059273 -93.176794,44.124187") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27147, "countyName": "Steele", "cityID": 2749300, "cityName": "Owatonna" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053928689098752, "text": "Wind 3.8 mph W. Barometer 30.10 in, Steady. Temperature 10.4 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 39988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053929435799554, "text": "i'm 18 while part of 21+ events lol\nthis could get annoying fast.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261829222, "name": "Andy Mac", "screen_name": "andylvlac", "lang": "en", "location": "Jersey City, New Jersey", "create_at": date("2011-03-06"), "description": "i take pictures of shit | atmacnow@live.com", "followers_count": 1385, "friends_count": 929, "statues_count": 79343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053929905496068, "text": "I wanna see the new purge movie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3306871033, "name": "jay", "screen_name": "JonathanEngle4", "lang": "en", "location": "4.19", "create_at": date("2015-08-05"), "description": "✨instagram// j.englee", "followers_count": 158, "friends_count": 136, "statues_count": 1313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053930094202880, "text": "@chelsmonnn ☕️������", "in_reply_to_status": -1, "in_reply_to_user": 3232056804, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3232056804 }}, "user": { "id": 380072114, "name": "Caroline Culpepper", "screen_name": "ckculpepper", "lang": "en", "location": "null", "create_at": date("2011-09-25"), "description": "null", "followers_count": 215, "friends_count": 206, "statues_count": 311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053930199060480, "text": "@DopeItsBen goodnight fren!", "in_reply_to_status": 698050667022196736, "in_reply_to_user": 557560413, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 557560413 }}, "user": { "id": 3273885060, "name": "a❁", "screen_name": "aundrealenae", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "J.Gutierrez❥ | ♓️", "followers_count": 367, "friends_count": 385, "statues_count": 8736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053930245181440, "text": "#CutiesDeLaSemana mi hijo angel siempre tan guapo y ocurrente @DespiertaAmeric https://t.co/JRhQQKyh1a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "CutiesDeLaSemana" }}, "user_mentions": {{ 94422249 }}, "user": { "id": 2915098876, "name": "My Info", "screen_name": "ratchkid702", "lang": "en", "location": "null", "create_at": date("2014-12-10"), "description": "null", "followers_count": 0, "friends_count": 5, "statues_count": 11 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-02-12T00:00:00.000Z"), "id": 698053930865954818, "text": "Goodnight everyone��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 890106602, "name": "David Bustamante", "screen_name": "Fuehrer_David", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2012-10-18"), "description": "RIP Grandma", "followers_count": 78, "friends_count": 101, "statues_count": 11258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053931205795840, "text": "Niggaz ain't playing ���������� https://t.co/pGfItX3WR3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 515822602, "name": "lG @CHRIZZITTO", "screen_name": "CHRIZZITTO", "lang": "en", "location": "No Fucks Zone", "create_at": date("2012-03-05"), "description": "I'm Not Real, I'm Just Someone Your Mind Projected.\nFollow me on insta @CHRIZZITTO.", "followers_count": 696, "friends_count": 203, "statues_count": 35556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Rancho, NM", "id": "0046bfef79c8e224", "name": "Rio Rancho", "place_type": "city", "bounding_box": rectangle("-106.757623,35.217658 -106.566425,35.374708") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35043, "countyName": "Sandoval", "cityID": 3563460, "cityName": "Rio Rancho" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053932870807552, "text": "@ebbtideapp Tide in St. Michaels, Maryland 02/12/2016\nHigh 7:09am 1.3\n Low 1:56pm -0.3\nHigh 7:34pm 1.3\n Low 2:17am -0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-76.2217,38.7867"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 24, "friends_count": 1, "statues_count": 7038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24041, "countyName": "Talbot", "cityID": 2469825, "cityName": "St. Michaels" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053933273636864, "text": "46 in line at BANNER ESTRELLA MEDICAL CENTER right now. Browse wait times or share your own with #WaitShare at https://t.co/rv1fffVDIf!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.257442,33.478176"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 358, "friends_count": 887, "statues_count": 2717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053933449768960, "text": "02/12@03:00 - Temp 16.4F, WC 16.4F. Wind 2.0mph WSW, Gust 4.0mph. Bar 30.333in, Rising slowly. Rain 0.00in. Hum 67%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053934007455744, "text": "@PickSixAarion24 @Rickey_ray_ray he's literally the worst at communication idk who he's trying to play ��", "in_reply_to_status": 698053485003059201, "in_reply_to_user": 160977464, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 160977464, 885843367 }}, "user": { "id": 575642571, "name": "Emily", "screen_name": "Emily_thompson5", "lang": "en", "location": "null", "create_at": date("2012-05-09"), "description": "Rickey has my ♡", "followers_count": 656, "friends_count": 207, "statues_count": 1457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Linda, CA", "id": "acec3dfe69d5bfe6", "name": "Rio Linda", "place_type": "city", "bounding_box": rectangle("-121.485618,38.668971 -121.397445,38.729806") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 660942, "cityName": "Rio Linda" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053934334607360, "text": "Lmao thanks for the heads up. How about at night? It get cold or nah? https://t.co/6ruKvBAIaE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52276893, "name": "jay305", "screen_name": "JayForJustin", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-29"), "description": "19. #blacklivesmatter #blackmoneymatters", "followers_count": 1944, "friends_count": 790, "statues_count": 133912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Athens, CA", "id": "81923aaf1207edc1", "name": "West Athens", "place_type": "city", "bounding_box": rectangle("-118.317804,33.916377 -118.291514,33.931268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684116, "cityName": "West Athens" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053934447882241, "text": "Things hoes say \"Make his dick hard not his life\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195493076, "name": "Valerie Rebekah", "screen_name": "its_val14", "lang": "en", "location": "null", "create_at": date("2010-09-26"), "description": "here to bless not impress", "followers_count": 882, "friends_count": 401, "statues_count": 47765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053934619856896, "text": "@FoxNews @foxnewspolitics She is not a porn actress, she is a b-movie actress... she's never done porn! @SenTedCruz is an ignorant twat!!!", "in_reply_to_status": 698051640230813696, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531, 16032925, 1074480192 }}, "user": { "id": 581659694, "name": "Johnny/venomlegions", "screen_name": "JohnnyReb1970", "lang": "en", "location": "Riverside", "create_at": date("2012-05-16"), "description": "I'm a straight to UR face agitator, & will kick atheists & political correctness in the balls! U don't like me, bit me! U want some, come get some!", "followers_count": 122, "friends_count": 40, "statues_count": 11417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053934619951104, "text": "Wind 0.0 mph ---. Barometer 30.253 in, Falling. Temperature 25.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 111003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-12T00:00:01.000Z"), "id": 698053935010074625, "text": "Wind 1.0 mph E. Barometer 30.276 in, Falling. Temperature 22.7 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053935182057472, "text": "���������� https://t.co/FNJG5k1fJV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 624263023, "name": "boss of bosses", "screen_name": "ForeignMamacita", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "Kamesha ❣ follow my snap { beautiki }", "followers_count": 997, "friends_count": 394, "statues_count": 33618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053935270125568, "text": "Temp: 49.1°F Wind:0.0mph Pressure: 30.151hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 59693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053935764889600, "text": "What if i named my first born son dad.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 586979600, "name": "Dad", "screen_name": "StewartGarrison", "lang": "en", "location": "City by The Bay", "create_at": date("2012-05-21"), "description": "Saint in a world of sinners.", "followers_count": 522, "friends_count": 515, "statues_count": 4442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053936033300480, "text": "I don't periscope because I'm boring af.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3597147015, "name": "chela chela chela", "screen_name": "_cheilacheila", "lang": "en", "location": "hungry, tx", "create_at": date("2015-09-09"), "description": "HTX | 18 | ster wers | cat mom", "followers_count": 232, "friends_count": 190, "statues_count": 4635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053936675065857, "text": "Wind 0.5 mph WNW. Barometer 30.159 in, Rising. Temperature -0.9 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053936746377216, "text": "Hillary is attacking @BernieSanders for his critiques of Obama? She was singing a different tune when she ran against Obama. #DemDebate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DemDebate" }}, "user_mentions": {{ 216776631 }}, "user": { "id": 390477857, "name": "Vlad Khaykin", "screen_name": "VKhaykin", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-10-13"), "description": "Activism. Religion. Economics. Human Rights. Social Justice. Views my own. RT does not = endorse. Tweeting 24/6.", "followers_count": 194, "friends_count": 427, "statues_count": 1532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053937069473792, "text": "\"Whenever we think we know the future, even for a second, it changes.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549905127, "name": "Em", "screen_name": "EmilyLPenney", "lang": "en", "location": "McKenzie, TN", "create_at": date("2012-04-09"), "description": "BU •Class of '19• Southern Belle at heart.", "followers_count": 601, "friends_count": 1253, "statues_count": 6512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKenzie, TN", "id": "b719058b7b74b892", "name": "McKenzie", "place_type": "city", "bounding_box": rectangle("-88.535919,36.108601 -88.479991,36.154362") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47017, "countyName": "Carroll", "cityID": 4744940, "cityName": "McKenzie" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053937480519680, "text": "lonely nights I lay awake, pray the lord my soul to take, my hearts become too cold to break.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 539580645, "name": "madison ☄", "screen_name": "madddswaggg97", "lang": "en", "location": "where the wind takes me. ", "create_at": date("2012-03-28"), "description": "claymont graduate my daddy says I'm pretty. ❣ slightly psychotic, but shit happens. follow me", "followers_count": 959, "friends_count": 1297, "statues_count": 20549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, OH", "id": "949efdcb4c50ca55", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-81.506699,40.464766 -81.448979,40.565373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39157, "countyName": "Tuscarawas", "cityID": 3922456, "cityName": "Dover" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053937643917312, "text": "Bye birthday! ���� you were the best one by far ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 180013965, "name": "helen", "screen_name": "heyhelenvohey", "lang": "en", "location": "null", "create_at": date("2010-08-18"), "description": "alex is aight. in a good light w/ a filter he looks like Dave Franco.", "followers_count": 98, "friends_count": 97, "statues_count": 6644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053938050945025, "text": "@NHonestnk стопэ) отличный прием перевести стрелы))", "in_reply_to_status": 698036268739989504, "in_reply_to_user": 1175624628, "favorite_count": 0, "retweet_count": 0, "lang": "ru", "is_retweet": false, "user_mentions": {{ 1175624628 }}, "user": { "id": 1341846223, "name": "LesterBernham(55+)", "screen_name": "Bernham_Lester", "lang": "ru", "location": "Malibu, CA", "create_at": date("2013-04-10"), "description": "Можно ли простить врага? Бог простит.Наша задача-организовать их встречу.", "followers_count": 620, "friends_count": 989, "statues_count": 57906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Malibu, CA", "id": "65eb9ee22cdeb7a8", "name": "Malibu", "place_type": "city", "bounding_box": rectangle("-118.873235,34.000405 -118.584559,34.04779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645246, "cityName": "Malibu" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053938537345025, "text": "@jarpads_beanie I choked I was drinking soda when I saw this��", "in_reply_to_status": 698053136230064128, "in_reply_to_user": 356676357, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 356676357 }}, "user": { "id": 3289034690, "name": "Jordan✨38", "screen_name": "superkatural", "lang": "en", "location": "she/her", "create_at": date("2015-07-23"), "description": "Ashley Frangipane and Kathryn Newton are my all time favorite humans and I'm so ready to fight for them #AlwaysKeepFighting #YouAreNotAlone", "followers_count": 530, "friends_count": 384, "statues_count": 8884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053939137245184, "text": "Wind 0.0 mph W. Barometer 30.19 in, Steady. Temperature 21.6 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 108, "statues_count": 157434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-12T00:00:02.000Z"), "id": 698053939174854656, "text": "Ugght why can't this be me https://t.co/0XiAh7piMx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700828488, "name": "RYAN", "screen_name": "AyooKiddBrian", "lang": "en-gb", "location": "null", "create_at": date("2012-07-17"), "description": "King B", "followers_count": 91, "friends_count": 73, "statues_count": 1997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053939514646528, "text": "RAISIN BREAD WITH PEANUT BUTTER IS LIFE CHANGING. (Not life changing but the shit is good from your average wheat bread).", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 896883200, "name": "Kelvy", "screen_name": "kelvylam", "lang": "en", "location": "null", "create_at": date("2012-10-21"), "description": "suh dude", "followers_count": 432, "friends_count": 188, "statues_count": 37878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053939636228096, "text": "@JoeBudden drop a mixtape man.", "in_reply_to_status": 698053767141392385, "in_reply_to_user": 18676369, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 18676369 }}, "user": { "id": 278727862, "name": "11-25-13", "screen_name": "kasheem_1", "lang": "en", "location": "null", "create_at": date("2011-04-07"), "description": "What then, shall we say in response to these things? If God is for us, then who can be against us? 252 - Minnesota.", "followers_count": 1541, "friends_count": 1665, "statues_count": 31382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Worthington, MN", "id": "4cc9a5aa2a29c05c", "name": "Worthington", "place_type": "city", "bounding_box": rectangle("-95.644347,43.601854 -95.56187,43.647182") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27105, "countyName": "Nobles", "cityID": 2771734, "cityName": "Worthington" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053939732750338, "text": "@LeonDreemz awe, I shouldn't be eating this late or else I would �� haha", "in_reply_to_status": 698053284167168002, "in_reply_to_user": 385234339, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 385234339 }}, "user": { "id": 838907012, "name": "Jaly", "screen_name": "JalyssaGarza", "lang": "en", "location": "texas", "create_at": date("2012-09-21"), "description": "Walk by faith", "followers_count": 346, "friends_count": 284, "statues_count": 3189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053939825111040, "text": "#TBT at @theechola w/ @fartbarf \nPhoto Credit: @ellenwoodep\n\n#music #live #electronic #Music… https://t.co/CAbPrWNLWF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.258,34.0792"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TBT", "music", "live", "electronic", "Music" }}, "user_mentions": {{ 41215523, 47506433, 185508152 }}, "user": { "id": 2841821882, "name": "NKRIOT", "screen_name": "nkriot", "lang": "en", "location": "null", "create_at": date("2014-10-05"), "description": "NK RIOT\n\nAn imaginative, electric orchestration of reverberating synth pop is how listeners best describe downtown LA's latest electronic project, NK-Riot ", "followers_count": 144, "friends_count": 248, "statues_count": 978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053940026281984, "text": "@joshuadrop but do you sub", "in_reply_to_status": 698052773170933760, "in_reply_to_user": 1429509456, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1429509456 }}, "user": { "id": 3389657313, "name": "chunky monkey", "screen_name": "Croth51", "lang": "en", "location": "Ranyolo Swagamonga", "create_at": date("2015-07-23"), "description": "We ain't love these hoes but I love spooky #iminlovewithdapooky", "followers_count": 322, "friends_count": 374, "statues_count": 2511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053940303101953, "text": "And I'm blond again¯\\_(ツ)_/¯ @ The Wharf Nahant https://t.co/I7MEtg1KN5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.91754962,42.42149231"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26647425, "name": "Grump King", "screen_name": "Ericmarie", "lang": "en", "location": "Nahant ma", "create_at": date("2009-03-25"), "description": "I didnt choose the grump life. it chose me.", "followers_count": 247, "friends_count": 356, "statues_count": 3412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nahant, MA", "id": "6278bb45b4dc4315", "name": "Nahant", "place_type": "city", "bounding_box": rectangle("-70.940154,42.414872 -70.90167,42.457062") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2543615, "cityName": "Nahant" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053941360091137, "text": "@qroundzeroes IJUST SPAT OUT MY DRINK RREADING THIS", "in_reply_to_status": 698053795519922176, "in_reply_to_user": 3286234616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3286234616 }}, "user": { "id": 2284968950, "name": "✨be nice to kaz✨", "screen_name": "redserenade", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-10"), "description": "noel - she/her - ISFJ - ENG/한국어 OK- there's nothing i love more than money and power - local kazuhira miller lover - personal: @ocelhiras / nsfw: @pink_concerto", "followers_count": 188, "friends_count": 612, "statues_count": 30734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053941469249536, "text": "So in love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365244761, "name": "alexandrea", "screen_name": "AlexCalcagno987", "lang": "en", "location": "null", "create_at": date("2011-08-30"), "description": "my tweets may make me look bipolar or depressed....", "followers_count": 320, "friends_count": 102, "statues_count": 16040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053941678858240, "text": "@jazzleennn 12", "in_reply_to_status": 698053593878769664, "in_reply_to_user": 628786377, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 628786377 }}, "user": { "id": 2779916363, "name": "SadBoy Sebass", "screen_name": "ayyee_Lmao", "lang": "en", "location": "null", "create_at": date("2014-09-21"), "description": "I joined twitter for the death threats from people.", "followers_count": 234, "friends_count": 200, "statues_count": 6484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053942379356160, "text": "������������������ https://t.co/ZkQ8NVWox3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3221348347, "name": "Shawn Infamous", "screen_name": "ShawnInfamous", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-05-20"), "description": "Future point guard, upcoming artist #SremmLife\nLISTEN TO MY NEW SONG Save It!\n#PrayforTy", "followers_count": 143, "friends_count": 170, "statues_count": 704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duncanville, TX", "id": "c9ff03f5c5cb510a", "name": "Duncanville", "place_type": "city", "bounding_box": rectangle("-96.943349,32.617554 -96.882757,32.676694") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4821628, "cityName": "Duncanville" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053942702288896, "text": "Waiting for a call that's never gonna happen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2847637310, "name": "•Luis•Sadannn•", "screen_name": "geeezz_luis", "lang": "en", "location": "Santa Maria, CA", "create_at": date("2014-10-08"), "description": "I can do everything through him who gives me strength - Philippians 4:13", "followers_count": 403, "friends_count": 962, "statues_count": 4356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Maria, CA", "id": "dee4ad8775ff8102", "name": "Santa Maria", "place_type": "city", "bounding_box": rectangle("-120.482386,34.875868 -120.356782,34.988866") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669196, "cityName": "Santa Maria" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053942756790272, "text": "Fab is to funny for the degradation jersey though ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55389747, "name": "Rare Rose", "screen_name": "Zahnay_B", "lang": "en", "location": "null", "create_at": date("2009-07-09"), "description": "aspiring supermodel /oscar winner , trying to find peace and serenity Instagram:zahnay__ #CAPA12 #TU16 LA LIVING", "followers_count": 823, "friends_count": 652, "statues_count": 24749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053943016837120, "text": "I swear math professors just put random stuff together and call it an equation...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 489769785, "name": "†Zach Davis†", "screen_name": "Zach_Davis_08", "lang": "en", "location": "405", "create_at": date("2012-02-11"), "description": "O|||||||O ΣΤΓ UCO", "followers_count": 2957, "friends_count": 1234, "statues_count": 13882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053943058796544, "text": "@soy___alejandra no se como hay gente que le mete a eso ������", "in_reply_to_status": 698053768483467265, "in_reply_to_user": 1012739876, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1012739876 }}, "user": { "id": 994231849, "name": "Caneca™", "screen_name": "williammendez_", "lang": "en", "location": "Dorado, PR", "create_at": date("2012-12-06"), "description": "Por la plata baila el mono, las putas, las chapi y hasta las millo | 22 | BMA en la casa | #KobeNation | Capricornio | IG: Canecaa_ | SC: Caneca_24", "followers_count": 1636, "friends_count": 831, "statues_count": 139935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dorado, PR", "id": "4f3c95e0a936049b", "name": "Dorado", "place_type": "city", "bounding_box": rectangle("-66.295112,18.449974 -66.25321,18.483403") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72051, "countyName": "Dorado", "cityID": 7224008, "cityName": "Dorado" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053943188942848, "text": "I'm at @PlanetFitness in Milwaukee, WI w/ @sera_vela0106 https://t.co/EScpU7wCfJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.94990981,42.98224172"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 100606370, 49716601 }}, "user": { "id": 26283980, "name": "Juan Youngblood", "screen_name": "electroperfecto", "lang": "en", "location": "|Milwaukee", "create_at": date("2009-03-24"), "description": "★energetic.honest.outspoken.spontaneous.independent.fashionable.intelligent.loyal.friendly|#teamblatino(Boricua) ☆", "followers_count": 503, "friends_count": 1644, "statues_count": 9231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-02-12T00:00:03.000Z"), "id": 698053943260106752, "text": "@EvanRezai @Tatu_77 @_ohhhkennard @_ohhhkendrick @Bobbywwb 3, :)", "in_reply_to_status": 698053817443622913, "in_reply_to_user": 1385210167, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1385210167, 566541699, 1167720542, 1368384181, 2285261147 }}, "user": { "id": 524907570, "name": "Shadyn", "screen_name": "Blacckness", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "at the function I shake my ass", "followers_count": 566, "friends_count": 253, "statues_count": 28203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego Country Estates, CA", "id": "5b151cafa875f47c", "name": "San Diego Country Estates", "place_type": "city", "bounding_box": rectangle("-116.843644,32.991819 -116.72703,33.035831") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666004, "cityName": "San Diego Country Estates" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053944212213760, "text": "@angel_balbuena9 ������", "in_reply_to_status": 698053723952549888, "in_reply_to_user": 1509733068, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1509733068 }}, "user": { "id": 2983071848, "name": "maya g", "screen_name": "mayagrodriguez", "lang": "en", "location": "CA", "create_at": date("2015-01-14"), "description": "Always appreciate life for what it is and what you have, not what it can be and what you want.", "followers_count": 563, "friends_count": 325, "statues_count": 33535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shafter, CA", "id": "3b642703733bf53f", "name": "Shafter", "place_type": "city", "bounding_box": rectangle("-119.300846,35.463437 -119.251513,35.521029") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 671106, "cityName": "Shafter" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053944287703040, "text": "Schooled jerm in bball today, 3/5 3 pointers was guuuuuuud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2736019610, "name": "deadbwoy", "screen_name": "YoungCocoaButa", "lang": "en", "location": "null", "create_at": date("2014-08-15"), "description": "live fast, die young, leave a pretty corpse\n looking for a beautiful death\n https://soundcloud.com/jack-henry-2", "followers_count": 147, "friends_count": 147, "statues_count": 5569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, CA", "id": "8eccead85cc1b6c0", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-121.260264,38.621443 -121.085432,38.714331") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053944715575297, "text": "Wind 4.0 mph SSW. Barometer 1026.21 mb, Steady. Temperature 29.8 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053944812023808, "text": "Wind 0.0 mph N. Barometer 30.17 in, Steady. Temperature 53.8 °F. Rain today 0.00 in. Humidity 19%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 23, "friends_count": 98, "statues_count": 14202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053947051773953, "text": "Rumor said that music artists actors actreses and other celebs decided to stay in san jose instead of san francisco hahahaha how funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31459918, "name": "Al", "screen_name": "smetchrispy", "lang": "en", "location": "San Fran. Ca. United States ", "create_at": date("2009-04-15"), "description": "I'm 33 years old and Im a Dermatologist.it will be my pleasure I will answer your Qs.", "followers_count": 190, "friends_count": 986, "statues_count": 917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053947173560320, "text": "can't sleeeeppp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2288370532, "name": "Summer Dunmyer", "screen_name": "summerdancerxo", "lang": "en", "location": "Dewey Beach, DE", "create_at": date("2014-01-17"), "description": "cape '18", "followers_count": 363, "friends_count": 372, "statues_count": 216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewes, DE", "id": "42c5edf4f521b4cb", "name": "Lewes", "place_type": "city", "bounding_box": rectangle("-75.162148,38.757832 -75.103973,38.790096") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1041830, "cityName": "Lewes" } }
+{ "create_at": datetime("2016-02-12T00:00:04.000Z"), "id": 698053947483824128, "text": "Temp: 16.4°F - Dew Point: 1.5° - Wind: 10.2 mph - Gust: 15.2 - Rain Today: 0.00in. - Pressure: 30.14in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053948519780353, "text": "Temp: -2.0F W C: -2.0F Wind:--- at 0.0kts Baro: 1032.1mb and Falling slowly Rain today: 0.00in R H: 90% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 191, "friends_count": 219, "statues_count": 96843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053948989685760, "text": "Happy 21st Birthday to the lady that im standing next to in front of… https://t.co/C9s5I80EeO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.35843847,34.07217665"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 203414287, "name": "Lian Sanchez", "screen_name": "liansanchezzz", "lang": "en", "location": "null", "create_at": date("2010-10-15"), "description": "null", "followers_count": 420, "friends_count": 455, "statues_count": 11777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053949056806912, "text": "@afrenaazmn https://t.co/WMt2txFVnE", "in_reply_to_status": -1, "in_reply_to_user": 2464154022, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2464154022 }}, "user": { "id": 605612767, "name": "Allify", "screen_name": "alifdromzi", "lang": "en", "location": "Eugene, OR", "create_at": date("2012-06-11"), "description": "Sorry. Ada life.", "followers_count": 276, "friends_count": 249, "statues_count": 14244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053949505417217, "text": "Hmu if you know why timing sucks, why people still support Tronald Dump, why I love cheese, and why laundry detergent smells so good.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 234982995, "name": "Chip in a Bottle", "screen_name": "PirateLindsey", "lang": "en", "location": "Palo Alto, CA", "create_at": date("2011-01-06"), "description": "Stand-up comedian, sit-down Canadian, and sideways accordion.", "followers_count": 290, "friends_count": 257, "statues_count": 12629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950067486720, "text": "@OGJiggaBooJones just judges me for every little thing I do. Fuck.", "in_reply_to_status": -1, "in_reply_to_user": 279309889, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 279309889 }}, "user": { "id": 551533416, "name": "T a t i a n a", "screen_name": "VibesSoGold", "lang": "en", "location": "IG: vibessogold", "create_at": date("2012-04-11"), "description": "Focus on your goals baby girl✨", "followers_count": 301, "friends_count": 272, "statues_count": 27253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950172438528, "text": "@localseoguide thanks buddy", "in_reply_to_status": 697926291706621952, "in_reply_to_user": 5890262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5890262 }}, "user": { "id": 44188562, "name": "Milind Mehere", "screen_name": "TheMilindMehere", "lang": "en", "location": "New York City", "create_at": date("2009-06-02"), "description": "Founder @yieldstreet co-founder @yodle Entrepreneur, operator, idea generator, red sox fan, scotch aficionado, strategist, change agent, loves disruptive models", "followers_count": 426, "friends_count": 281, "statues_count": 909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950205853697, "text": "Falling asleep in cars that my friends are driving is my specialty m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36489991, "name": "Hannah Horwatt", "screen_name": "hannahhorwatt", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-29"), "description": "Actress | enjoys long adventures (to get pizza) and live music | don't worry, be happy", "followers_count": 1395, "friends_count": 1899, "statues_count": 14877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950214246400, "text": "İntihar eden işsiz öğretmenlerin ilgi çekmek için intihar ettiğini söyleyen M.Eğitim Bakanı niye saçmalamış? #ediPol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "hashtags": {{ "ediPol" }}, "user": { "id": 238776270, "name": "Edip Yüksel", "screen_name": "edipyuksel", "lang": "en", "location": "+1 619 894 6346", "create_at": date("2011-01-15"), "description": "For English tweets follow: @19org \nQuran: a Reformist Translation", "followers_count": 43522, "friends_count": 184, "statues_count": 29115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950428352512, "text": "Wo liegt Tampa? https://t.co/oCeX6gvyoW #Tampa #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.4511,27.9814"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Tampa", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 436, "friends_count": 114, "statues_count": 1864336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950671441921, "text": "32 steps away from 20,000 at 11:59 and stuck in a @SuperShuttle: the worst", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16891273 }}, "user": { "id": 263782362, "name": "Jenna Deutsch", "screen_name": "JennaDeutsch", "lang": "en", "location": "BOSTON", "create_at": date("2011-03-10"), "description": "I spend my mornings at @Nov_Project, my days at @Fitbit, & my nights dreaming about where I'll travel next | @theSkimm & @Moleskine lover | #TeamNuun", "followers_count": 1391, "friends_count": 2091, "statues_count": 13142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053950835011585, "text": "YUHH Harv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316735921, "name": "⛄️", "screen_name": "DarelBeChillin", "lang": "en", "location": "Kansas City ✈️ Austin/Denton", "create_at": date("2011-06-13"), "description": "Harlem shakin' through the pressure | #DILFHive | #BLACKLIVESMATTER", "followers_count": 1928, "friends_count": 892, "statues_count": 104759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053951225200640, "text": "@jamila_jb hey sis ��✊����\nThanks for holding it down with me.", "in_reply_to_status": -1, "in_reply_to_user": 2466269965, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2466269965 }}, "user": { "id": 168599279, "name": "Imani", "screen_name": "theimanihh", "lang": "en", "location": "lost track of the lifetimes", "create_at": date("2010-07-20"), "description": "•future bush doctor•vegetarian•law/life student•", "followers_count": 832, "friends_count": 670, "statues_count": 44736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-12T00:00:05.000Z"), "id": 698053951510413312, "text": "@PolanskiJoshua https://t.co/QEg2k8t1gw", "in_reply_to_status": -1, "in_reply_to_user": 752298872, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 752298872 }}, "user": { "id": 14333969, "name": "myboystones", "screen_name": "myboystones", "lang": "en", "location": "Wadsworth, Ohio", "create_at": date("2008-04-08"), "description": "A minister of reconciliation, bodybuilder and I make worship loud at @GraceChurchBath.", "followers_count": 411, "friends_count": 279, "statues_count": 14015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wadsworth, OH", "id": "89bf5731fb830e51", "name": "Wadsworth", "place_type": "city", "bounding_box": rectangle("-81.76372,40.984952 -81.687287,41.063768") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39103, "countyName": "Medina", "cityID": 3980304, "cityName": "Wadsworth" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053952156364800, "text": "#SupportOriginMelissa 25.5°F Wind:0.9mph Pressure: 30.15hpa Falling Rain Today 0.00in. Forecast: Fairly fine, showery later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053952340946944, "text": "Doday L Alex gave me I give up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906825009, "name": "Summer 6Teen", "screen_name": "AnthonyBeaverzz", "lang": "en", "location": "null", "create_at": date("2014-12-05"), "description": "bio", "followers_count": 264, "friends_count": 204, "statues_count": 5935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summit, NJ", "id": "789a596e07b40baf", "name": "Summit", "place_type": "city", "bounding_box": rectangle("-74.402023,40.685322 -74.330621,40.739209") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3471430, "cityName": "Summit" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053953074794497, "text": "Barberton Oh Temp:10.3°F Wind:0 mph Dir:--- Baro:Falling Rain2day:0.00in Hum:75% UV:0.0 @ 03:00 02/12/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053953674674177, "text": "Temp 14.0°F Wind Chill 14.0°F RH 83% Wind 0.1 SW Gust 2.0 SW SLP 30.325 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 105, "friends_count": 63, "statues_count": 33205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053955733958656, "text": "Cuttin up cracking a cold beer say cheers, hey y'all it's sure been a good year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439543884, "name": "Angelo Ayala", "screen_name": "angeloayala17", "lang": "en", "location": "Covina, CA", "create_at": date("2014-04-11"), "description": "God. 18. Employed at Starbucks. Get Buck", "followers_count": 338, "friends_count": 237, "statues_count": 3928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-02-12T00:00:06.000Z"), "id": 698053955763351552, "text": "@JustDoItEverett nah it didn't lol kept me entertained it was funny. Wasn't the typical superhero movie to me tho", "in_reply_to_status": 698053658760466432, "in_reply_to_user": 344851171, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 344851171 }}, "user": { "id": 27374061, "name": "bria", "screen_name": "callmeBREE_", "lang": "en", "location": "_/*\\_ TEXA$", "create_at": date("2009-03-28"), "description": "Rest in Paradise Aubree . Unt. ❄️❄️ Lucky Lefty #LongLiveClyde", "followers_count": 2041, "friends_count": 923, "statues_count": 239401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053958036799488, "text": "���������������� @ Totowa, New Jersey https://t.co/nt0It3opmb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.2176,40.9033"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 356084240, "name": "Robert Graber", "screen_name": "RobertInNJ", "lang": "en", "location": "Little Falls,NJ", "create_at": date("2011-08-16"), "description": "null", "followers_count": 192, "friends_count": 284, "statues_count": 9085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Totowa, NJ", "id": "2fb47e56cc67fd10", "name": "Totowa", "place_type": "city", "bounding_box": rectangle("-74.249981,40.882389 -74.195534,40.924779") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3473140, "cityName": "Totowa" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053958409912320, "text": "@zzzMEOWzzz you gotta believe it'll happen", "in_reply_to_status": 698048365431422978, "in_reply_to_user": 1582271844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1582271844 }}, "user": { "id": 1542161100, "name": "Damned-Yul", "screen_name": "DreadSyndrome", "lang": "en", "location": "null", "create_at": date("2013-06-23"), "description": "Stay the Path; and I will Guide you Home. Father, forgive me, for I do not Give a Fuck.", "followers_count": 241, "friends_count": 188, "statues_count": 26200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053958766473216, "text": "aaaaaaaah.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 45273558, "name": "Annabella Del Franco", "screen_name": "annabella_df", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2009-06-06"), "description": "restless, moody, and obsessive. but i'm alright though.", "followers_count": 671, "friends_count": 363, "statues_count": 22030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053958812565504, "text": "Some girls look amazing w/ braids. Some girls shouldn't even attempt.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490846538, "name": "ℱ❥", "screen_name": "iaafaay", "lang": "en", "location": "HI", "create_at": date("2012-02-12"), "description": "null", "followers_count": 281, "friends_count": 161, "statues_count": 22217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053959030743040, "text": "It's Thirstday and I have a bottle in my fridge and I haven't touched it. I swear a few months ago at this time it would be finished lmfao.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164079423, "name": "Andrea Lopez", "screen_name": "Dreea_119", "lang": "en", "location": "Camarillo, CA", "create_at": date("2010-07-07"), "description": "I still don't understand Twitter, but here I am.", "followers_count": 128, "friends_count": 102, "statues_count": 379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053959039070208, "text": "@lowellheddings I was getting that error too… Fixed by going to Settings app > scroll down to Sleep++ > toggle both off then on.", "in_reply_to_status": -1, "in_reply_to_user": 7684302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7684302 }}, "user": { "id": 29383882, "name": "David Hall", "screen_name": "davidjoelhall", "lang": "en", "location": "on foursquare", "create_at": date("2009-04-06"), "description": "Hardly professional. \nGeek. Husband. Amateur photographer.", "followers_count": 486, "friends_count": 210, "statues_count": 52837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-02-12T00:00:07.000Z"), "id": 698053959190097920, "text": "���� https://t.co/gGovAJd34o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2439473695, "name": "Comp10", "screen_name": "compton_jacoby", "lang": "en", "location": "GTX", "create_at": date("2014-04-11"), "description": "6'3|GTX", "followers_count": 1072, "friends_count": 825, "statues_count": 10165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053960460972033, "text": "Happy birthday @sergi_yo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 779133618 }}, "user": { "id": 2792073750, "name": "James", "screen_name": "bananan_james", "lang": "en", "location": "Brentwood, CA", "create_at": date("2014-09-05"), "description": "just looking to chill", "followers_count": 120, "friends_count": 197, "statues_count": 1443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053960888770560, "text": "HAPPY BIRTHDAY BEST FRIEND!!!��don't know what I'd do with out you! thanks for always making me laugh��love youuu���� https://t.co/oqAepMmxDJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1899225362, "name": "Jordan McIntosh", "screen_name": "__Jordan7", "lang": "en", "location": "null", "create_at": date("2013-09-23"), "description": "st. lucys '18", "followers_count": 214, "friends_count": 174, "statues_count": 283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961144672257, "text": "Happy birthday UHLONDRAAA @Asaplona", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2367451543 }}, "user": { "id": 1145671302, "name": "waleedo", "screen_name": "hummuspapi", "lang": "en", "location": "null", "create_at": date("2013-02-03"), "description": "everybody fugazi", "followers_count": 1053, "friends_count": 643, "statues_count": 4388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961421627392, "text": "my dreams are going to be fulfilled. one step at a time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2837453646, "name": "cami", "screen_name": "Camidaedric", "lang": "en", "location": "Bay Area", "create_at": date("2014-10-01"), "description": "must be nice / psn:WrekCityCami / @dvntepennell and I are in a clan.", "followers_count": 716, "friends_count": 815, "statues_count": 8276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961534697472, "text": "I've learned that you can keep going, long after you think you can't.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317612920, "name": "Paige Johnson", "screen_name": "paigeconnarie", "lang": "en", "location": "•Crescent-Stillwater•", "create_at": date("2011-06-14"), "description": "Cross before me, world behind me.", "followers_count": 661, "friends_count": 400, "statues_count": 13194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961836670976, "text": "@AyyeBee_YMB chill fuck them‼️ Make dem bitches do they job!!", "in_reply_to_status": 698053259907325952, "in_reply_to_user": 288316890, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 288316890 }}, "user": { "id": 1470369972, "name": "ME,THATSWHOHOE", "screen_name": "IamLeqendary_", "lang": "en", "location": "Egypt", "create_at": date("2013-05-30"), "description": "Hungry & Humble", "followers_count": 435, "friends_count": 681, "statues_count": 8553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961966866432, "text": "all them gah damn tickets me & my niggas be gettin & yall aint got money for tops? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53000782, "name": "G® From OP‼️", "screen_name": "JG_Trilll", "lang": "en", "location": "Opelousas☀️", "create_at": date("2009-07-02"), "description": "been jiggy, been pretty | YWA | #LSU19 ❌ #GeauxEngineering‼️", "followers_count": 2862, "friends_count": 2055, "statues_count": 47540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053961979285504, "text": "Wind 2.0 mph W. Barometer 30.279 in, Falling slowly. Temperature 18.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 14, "statues_count": 1093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053962117861376, "text": "GN (@ Meridian Place in Northridge, CA) https://t.co/KIwwrxJEYA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 374, "friends_count": 337, "statues_count": 25528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053962293858304, "text": "Convincing people I'm talented. Vintage me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66883073, "name": "Garth", "screen_name": "spongebelt", "lang": "en", "location": "null", "create_at": date("2009-08-18"), "description": "Hi im Garth. Co-Host of gamercade podcast!", "followers_count": 234, "friends_count": 767, "statues_count": 21940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053962998661121, "text": "Lmfaoooo https://t.co/VhLYWLOgxI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594388635, "name": "Jusuke Urameshi", "screen_name": "ThaGreatJulian", "lang": "en", "location": "New Bark Town, Johto Region", "create_at": date("2012-05-29"), "description": "No, seriously...please tell me you're not that stupid.", "followers_count": 959, "friends_count": 797, "statues_count": 37840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, AL", "id": "ba2e7f7eb278521d", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-87.095997,32.372411 -86.978868,32.457178") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1047, "countyName": "Dallas", "cityID": 169120, "cityName": "Selma" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053963170476032, "text": "I hope someone buys me flowers on Valentine's Day ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91952668, "name": "Princess Toni", "screen_name": "ToniIndelicato", "lang": "en", "location": "null", "create_at": date("2009-11-22"), "description": "Smile, because you can❤️", "followers_count": 271, "friends_count": 163, "statues_count": 4982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra Heights, CA", "id": "c26a4df95e9c83fb", "name": "La Habra Heights", "place_type": "city", "bounding_box": rectangle("-117.992932,33.94593 -117.916705,33.976782") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639304, "cityName": "La Habra Heights" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053963921248257, "text": "@dodi70 meeker Nyc Lewis Ga doesn't matter they are from the old and the younger blacks can't move forward with them in place Obama nos that", "in_reply_to_status": 698052696113180673, "in_reply_to_user": 20729032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20729032 }}, "user": { "id": 59856787, "name": "Eman58", "screen_name": "eman58", "lang": "en", "location": "null", "create_at": date("2009-07-24"), "description": "null", "followers_count": 18, "friends_count": 83, "statues_count": 272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ravensdale, WA", "id": "0041fc436af61acd", "name": "Ravensdale", "place_type": "city", "bounding_box": rectangle("-122.01639,47.348725 -121.979598,47.366071") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357395, "cityName": "Ravensdale" } }
+{ "create_at": datetime("2016-02-12T00:00:08.000Z"), "id": 698053964076417027, "text": "Any choice of words I use will be an understatement of how repulsive this is ���� https://t.co/aFFyD5ziNP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 94881490, "name": "D00MGeneration", "screen_name": "illuminaughtyE", "lang": "en", "location": "the moon", "create_at": date("2009-12-05"), "description": "she's a hag in teens clothing", "followers_count": 175, "friends_count": 96, "statues_count": 7108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416315061514244, "text": "Rest in piece, my peace of mind I hope that we will reunite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013832524, "name": "Cucumber", "screen_name": "NiceCucumber", "lang": "en", "location": "The dark recesses of your mind", "create_at": date("2015-02-02"), "description": "Bringing smiles all across twitter", "followers_count": 1293, "friends_count": 445, "statues_count": 28831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416315476750337, "text": "Eighth grade music class with Bob Weir. Best school fund raiser I've been to.… https://t.co/DPpWRorOVN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.5480194,37.9069786"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16579792, "name": "Stephen Goldblatt", "screen_name": "Goldblartsky", "lang": "en", "location": "San Francisco", "create_at": date("2008-10-03"), "description": "Executive Creative Director | West", "followers_count": 1591, "friends_count": 703, "statues_count": 1581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mill Valley, CA", "id": "77fcb96a24077038", "name": "Mill Valley", "place_type": "city", "bounding_box": rectangle("-122.577014,37.886099 -122.512599,37.930235") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 647710, "cityName": "Mill Valley" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416316408033280, "text": "I was meant to be Dolo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 375990948, "name": "A M I E", "screen_name": "A_Morena_F", "lang": "en", "location": "Dont worry bout it", "create_at": date("2011-09-18"), "description": "S P I R I T U A L I S M", "followers_count": 502, "friends_count": 663, "statues_count": 7121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416316726661121, "text": "This needs to end", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2388270235, "name": "Bryce Buckets", "screen_name": "BA_Bordelon", "lang": "en", "location": "null", "create_at": date("2014-03-13"), "description": "Life isn't about what we win, but what we get from our losses ULM 18. ΠΚΑ Eta Omicron Fall '14.", "followers_count": 624, "friends_count": 482, "statues_count": 4089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, LA", "id": "e60ab9aacc0e96af", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-92.135057,32.429655 -92.018346,32.612831") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2251410, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416317225943040, "text": "For real someone take me home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481437811, "name": "DOUG THE THUG GLATT", "screen_name": "blakemireles", "lang": "en", "location": "Fenton, MI", "create_at": date("2012-02-02"), "description": "i am the trap lord #TYBG A$AP LIFE. mott bear-mid mich laker-cmu chip 1738 #remyboyz. Bitchin.", "followers_count": 1186, "friends_count": 1001, "statues_count": 40658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Fenton, MI", "id": "bf82c03437bea8e3", "name": "Lake Fenton", "place_type": "city", "bounding_box": rectangle("-83.729713,42.817667 -83.687477,42.871532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2644520, "cityName": "Lake Fenton" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416317892845568, "text": "2 months huh?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102843645, "name": "caseE", "screen_name": "ccaseykkaiser", "lang": "en", "location": "Jerz. ", "create_at": date("2010-01-07"), "description": "Aspiring boss lady. Towson ~ BUAD", "followers_count": 732, "friends_count": 770, "statues_count": 10679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416318391910400, "text": "@w2me وفقكم الله…اتمنى ان يكون لكم دور في جعل الاسعار مناسبة، وانت المطلع على كمية المحتوى الانجليزي الممتاز وسعره المجاني او شبه مجاني", "in_reply_to_status": 698415819156541440, "in_reply_to_user": 9212472, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 9212472 }}, "user": { "id": 18175008, "name": "أنور الحازمي", "screen_name": "Dr_Anwar", "lang": "en", "location": "Cleveland, OH", "create_at": date("2008-12-16"), "description": "طبيب وباحث | مهتم بالمعرفة والرياضة والفن | اللهم لك الحمد على نعمك التي لا تحصى", "followers_count": 2084, "friends_count": 432, "statues_count": 10203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-02-13T00:00:00.000Z"), "id": 698416318697984000, "text": "This #job might be a great fit for you: Key Holder - https://t.co/4OzePVuJ23 #Brea, CA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8901212,33.9132186"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Brea", "Hiring" }}, "user": { "id": 2362771358, "name": "Vitamin Shoppe Jobs", "screen_name": "VSIcareers", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "At #VitaminShoppe we are lovers of fitness, health, vitamins, helping you on your wellness journey and motivating others. Apply to our #jobs below.", "followers_count": 547, "friends_count": 96, "statues_count": 3506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brea, CA", "id": "341aa0cdb944022e", "name": "Brea", "place_type": "city", "bounding_box": rectangle("-117.928789,33.905579 -117.800848,33.947725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608100, "cityName": "Brea" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416318970617856, "text": "I hate being sick and I just what to eat all the food in the world right now because I am fat like me too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3676850532, "name": "Supernova", "screen_name": "supernova8665", "lang": "en", "location": "null", "create_at": date("2015-09-24"), "description": "null", "followers_count": 8, "friends_count": 37, "statues_count": 34 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198337") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416319792680960, "text": "Drenched in sodium...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35011346, "name": "naomi", "screen_name": "naomiink", "lang": "en", "location": "making drake's hotlinebling", "create_at": date("2009-04-24"), "description": "I'm really good at drinking bad beer✌.", "followers_count": 111, "friends_count": 100, "statues_count": 9924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416320262438912, "text": "All I want rn is my bed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329855280, "name": "allie †", "screen_name": "_gracealison", "lang": "en", "location": "CA ✈️ FL", "create_at": date("2011-07-05"), "description": "#DCPFALL2015", "followers_count": 306, "friends_count": 245, "statues_count": 20181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416320442986496, "text": "Just posted a video @ Salsa Con Fuego https://t.co/iuRoZltvNy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9114227,40.8622398"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1200302540, "name": "jhonpapeleta", "screen_name": "josegomez1614", "lang": "es", "location": "woodside", "create_at": date("2013-02-20"), "description": "null", "followers_count": 136, "friends_count": 1090, "statues_count": 302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416320862404608, "text": "02/13@03:00 - Temp 21.3F, WC 16.5F. Wind 3.4mph WSW, Gust 6.0mph. Bar 29.979in, Falling slowly. Rain 0.00in. Hum 88%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416321009164290, "text": "Wind 0 mph --. Barometer 30.31 in, Steady. Temperature 45.5 °F. Rain today 0.00 in. Humidity 43%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416321097150464, "text": "@ebbtideapp Tide in Coffins Point, Maine 02/13/2016\n Low 8:59am -1.3\nHigh 2:58pm 19.3\n Low 9:25pm -1.1\nHigh 3:25am 19.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-67.1083,44.87"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 24, "friends_count": 1, "statues_count": 7320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23029, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416321441038336, "text": "I think I'm gonna do laundry and yoga and then finally sleep like a baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075864608, "name": "Rach", "screen_name": "RachelFasel17", "lang": "en", "location": "null", "create_at": date("2013-01-09"), "description": "i do backflips and yoga", "followers_count": 369, "friends_count": 191, "statues_count": 9490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416321483046912, "text": "@_treyflip Tfti", "in_reply_to_status": -1, "in_reply_to_user": 1926885482, "favorite_count": 0, "retweet_count": 0, "lang": "is", "is_retweet": false, "user_mentions": {{ 1926885482 }}, "user": { "id": 2364392857, "name": "أنور بافيا", "screen_name": "AnYouAre", "lang": "en", "location": "Houston", "create_at": date("2014-02-27"), "description": "#YNWA #Dreamville #Rednation #BullsOnParade #Astros #Dynamo #SOG Proverbs 3:5-6", "followers_count": 172, "friends_count": 376, "statues_count": 11542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crosby, TX", "id": "00a488ec567255f3", "name": "Crosby", "place_type": "city", "bounding_box": rectangle("-95.12133,29.899357 -95.04072,29.942391") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4817756, "cityName": "Crosby" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416322242224128, "text": "WDFWU\n\n������������\n3 muscles because @kissmirah is also #flexing in the back hahahahaha �� @ The… https://t.co/TZ67oO5ABP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.37358,34.1681"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "flexing" }}, "user_mentions": {{ 116126819 }}, "user": { "id": 364454540, "name": "Danielle Gray", "screen_name": "daniginmb", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-08-29"), "description": "22, @USC and @USCAnnenberg #PR grad, Consumer Behavior, #Photography (@dgrayphoto), #Gymnastics, #PoleArt. Insta | @daniginmb", "followers_count": 481, "friends_count": 1237, "statues_count": 5602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416322317807616, "text": "Wind 4.0 mph NW. Barometer 30.449 in, Rising. Temperature 18.1 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-13T00:00:01.000Z"), "id": 698416322934231040, "text": "me that pussy https://t.co/CmdYdO7z48", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2355325940, "name": "autumn", "screen_name": "autumnaiono", "lang": "en", "location": "balifornia", "create_at": date("2014-02-21"), "description": "amat | snap @aautumng", "followers_count": 841, "friends_count": 590, "statues_count": 33843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Puente Valley, CA", "id": "b5b11d0f5cc91e3c", "name": "West Puente Valley", "place_type": "city", "bounding_box": rectangle("-117.990493,34.033699 -117.94947,34.066049") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684774, "cityName": "West Puente Valley" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416323085242369, "text": "brb crying cause Gerardo Ortiz was having a concert 7 min away from my hotel.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2996217716, "name": "Joanna", "screen_name": "_itsjoannaa", "lang": "en", "location": "null", "create_at": date("2015-01-25"), "description": "null", "followers_count": 273, "friends_count": 213, "statues_count": 3688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416323177525248, "text": "@TheTopKnotch THANKS BRO!!", "in_reply_to_status": 698391218057654273, "in_reply_to_user": 497425648, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497425648 }}, "user": { "id": 1268259086, "name": "Shaggy P", "screen_name": "jpmin2", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "SC- jpmin100 IG- jpmin1", "followers_count": 878, "friends_count": 803, "statues_count": 16410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416323613687808, "text": "Goodnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875436305, "name": "☁", "screen_name": "Nivvix", "lang": "en", "location": "916", "create_at": date("2014-10-24"), "description": "The only one who can beat me is me. Manga/Anime Nerd.", "followers_count": 1342, "friends_count": 794, "statues_count": 31793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416323668406272, "text": "Temp: 45.5°F Wind:1.5mph Pressure: 30.346hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 59789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416323710357504, "text": "Wind 2.0 mph NW. Barometer 30.627 in, Rising. Temperature 15.6 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 111027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416324330950656, "text": "Monsanto should be jailed, sued and is poisoning us all https://t.co/4llTY1l0IN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4734908474, "name": "TrumpsAmeriCa", "screen_name": "scottnelson221", "lang": "en", "location": "La Quinta, Ca", "create_at": date("2016-01-09"), "description": "US Navy, Business owner; God, Family and America, Values. We will win & make it count for our next generations! It's the final count down! California 4 Trump", "followers_count": 78, "friends_count": 67, "statues_count": 112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416324586954752, "text": "that la �� got some powerrrr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292940164, "name": "Neisha ❣", "screen_name": "_yourstruuly_", "lang": "en", "location": "BOGALUSA RAISED! ", "create_at": date("2011-05-04"), "description": "#Senior2016 18yearsyoung", "followers_count": 1763, "friends_count": 1336, "statues_count": 92116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bogalusa, LA", "id": "0d86f1118064d85f", "name": "Bogalusa", "place_type": "city", "bounding_box": rectangle("-89.896103,30.746749 -89.833306,30.812685") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22117, "countyName": "Washington", "cityID": 2208150, "cityName": "Bogalusa" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325119488000, "text": "@ColleenB123 NOW I KNOW YOU'RE IGNORING ME! Do you like P!nk? @ColleenB123 #SlumberPartyWithColleen Also, can we collab? THIS IS ILLEGAL", "in_reply_to_status": -1, "in_reply_to_user": 267305045, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SlumberPartyWithColleen" }}, "user_mentions": {{ 267305045, 267305045 }}, "user": { "id": 3023087670, "name": "COLLEENISMAHLIFE", "screen_name": "Grant_Vlogs", "lang": "en", "location": "Santa Clarita, CA", "create_at": date("2015-02-16"), "description": "false. silence. don't talk. no. wrong. stop. shut up. I'm hungry. ~ things I say a lot.", "followers_count": 191, "friends_count": 1052, "statues_count": 1337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325274660864, "text": "54.6F (Feels: 54.6F) - Humidity: 99% - Wind: 3.8mph E - Gust: 3.8mph - Pressure: 1036.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325379502080, "text": "@DreadSyndrome I'm the same way! They're like drugs I swear", "in_reply_to_status": 698416154386104320, "in_reply_to_user": 1542161100, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1542161100 }}, "user": { "id": 1127331547, "name": "Andrew Goodman", "screen_name": "AndrewRGoodman", "lang": "en", "location": "Banning, California", "create_at": date("2013-01-27"), "description": "21 | Actor | Gamer | Geek | @GameStop Senior Game Advisor", "followers_count": 489, "friends_count": 1252, "statues_count": 13600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325392084994, "text": "i have a thing for girls with great smiles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1107123168, "name": "Shane Taleisnik", "screen_name": "STalezzz", "lang": "en", "location": "null", "create_at": date("2013-01-20"), "description": "Shooter Not An Aimer • Pura Vida • Optimism • Beckman Basketball #3 • Respect All, Fear None • PTAW • http://Instagram.com/shanetaleisnik/", "followers_count": 644, "friends_count": 406, "statues_count": 13154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325459247105, "text": "\"He shot it in my ass\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189840256, "name": "verbatim.", "screen_name": "henny132", "lang": "en", "location": "California ", "create_at": date("2010-09-12"), "description": "| Rest In Peace Jossy 11.21.14 | tbh idc | bitch | kik: @henny132 | #Blackbear Is Daddy | Dandy This Ass Needs A Handy |", "followers_count": 1264, "friends_count": 2045, "statues_count": 34407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416325505499136, "text": "Wind 0.8 mph WNW. Barometer 29.818 in, Steady. Temperature 16.6 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416326319087616, "text": "https://t.co/IxsBZJiKYS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 46167501, "name": "Jimmy Foulin'", "screen_name": "JoseCanseco___", "lang": "en", "location": "HOUSTON", "create_at": date("2009-06-10"), "description": "I just got some swag now watch how a nigga maneuver. #UH16", "followers_count": 1423, "friends_count": 483, "statues_count": 128558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416326587469824, "text": "I can already tell this is what my summer is gonna be like��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2583170635, "name": "ash", "screen_name": "ashleydifrancoo", "lang": "en", "location": "null", "create_at": date("2014-06-22"), "description": "null", "followers_count": 479, "friends_count": 211, "statues_count": 5263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ponchatoula, LA", "id": "00bf0ddb6be3dd05", "name": "Ponchatoula", "place_type": "city", "bounding_box": rectangle("-90.478909,30.419185 -90.375706,30.479003") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2261615, "cityName": "Ponchatoula" } }
+{ "create_at": datetime("2016-02-13T00:00:02.000Z"), "id": 698416326910414848, "text": "why you checking on mine hm ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3148531555, "name": "evelyn", "screen_name": "_eevvellyynn_", "lang": "en", "location": "null", "create_at": date("2015-04-10"), "description": "bryan dominguez", "followers_count": 107, "friends_count": 112, "statues_count": 261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416327329869824, "text": "I love talking to someone from the time I wake up, to the time I go to bed. It's honestly a great feeling! ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68329160, "name": "Savannahhhhhh⚓", "screen_name": "savannahpoorman", "lang": "en", "location": "Pendleton, OR", "create_at": date("2009-08-23"), "description": "| Don't take life too seriously, nobody gets out alive anyway |", "followers_count": 221, "friends_count": 195, "statues_count": 5045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lebanon, OR", "id": "a12170d40e438f19", "name": "Lebanon", "place_type": "city", "bounding_box": rectangle("-122.934374,44.508029 -122.876675,44.561163") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4141650, "cityName": "Lebanon" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416327363432448, "text": "Wind 0.0 mph SW. Barometer 29.83 in, Falling slowly. Temperature 13.6 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 108, "statues_count": 157458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416327443161089, "text": "This is cute �� https://t.co/0Qs3MNl0tH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971124175, "name": "Sierra McCracken", "screen_name": "Sierra_McC", "lang": "en", "location": "null", "create_at": date("2012-11-25"), "description": "X.X.XIV", "followers_count": 442, "friends_count": 268, "statues_count": 260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416327581659136, "text": "#HillaryClinton: A Career Criminal You must watch this movie\nhttps://t.co/NgVJylzMet #Florida #SouthCarolina https://t.co/lpRjqn3Mgw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HillaryClinton", "Florida", "SouthCarolina" }}, "user": { "id": 90522101, "name": "Steven Roberts", "screen_name": "StevenMRoberts", "lang": "en", "location": "NYC", "create_at": date("2009-11-16"), "description": "Single, divorced, guy, 5 11, blue eyes, thin, I am working the travel industry. I travel all over the world, love my job", "followers_count": 1362, "friends_count": 5001, "statues_count": 16548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416327652990976, "text": "@RerezTV Good. Those things are flippin' creepy. Plus, there's no substitute for the real Cliff and Norm.", "in_reply_to_status": 698414600329555968, "in_reply_to_user": 821884369, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 821884369 }}, "user": { "id": 90806312, "name": "Josh Robinson", "screen_name": "averagejosh", "lang": "en", "location": "asheville, north carolina", "create_at": date("2009-11-17"), "description": "hobbit - hufflepuff - pizza enthusiast - nintendoer - fresca breather", "followers_count": 300, "friends_count": 715, "statues_count": 17485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asheville, NC", "id": "00836cb90515ddd1", "name": "Asheville", "place_type": "city", "bounding_box": rectangle("-82.681852,35.421216 -82.452163,35.656929") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37021, "countyName": "Buncombe", "cityID": 3702140, "cityName": "Asheville" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416328235814912, "text": "ima make myself a sandwich rn����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2852526308, "name": "stefani", "screen_name": "stefanih11", "lang": "en", "location": "shhs", "create_at": date("2014-10-11"), "description": "carlos my baby❤", "followers_count": 334, "friends_count": 225, "statues_count": 10594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bermuda Dunes, CA", "id": "4067289b6de7e711", "name": "Bermuda Dunes", "place_type": "city", "bounding_box": rectangle("-116.303933,33.72916 -116.26507,33.759486") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 606028, "cityName": "Bermuda Dunes" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416328466698240, "text": "cruising home to Pink Floyd and facing a blunt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292287735, "name": "Wednesday Addams", "screen_name": "brooke_patton", "lang": "en", "location": "Alton, IL - Los Angeles, CA✈️", "create_at": date("2011-05-03"), "description": "m.o.b. gang; money over bitches // @Drug_Porn Rip:Ab,Dest,Liv&Jake❤️ #FreeGuwop", "followers_count": 1471, "friends_count": 722, "statues_count": 17175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethalto, IL", "id": "3a7b025ae9763e9c", "name": "Bethalto", "place_type": "city", "bounding_box": rectangle("-90.089501,38.889478 -90.020624,38.92402") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1705599, "cityName": "Bethalto" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416328483454976, "text": "Barberton Oh Temp:7.6°F Wind:6 mph Dir:NW Baro:Rising quickly Rain2day:0.00in Hum:74% UV:0.0 @ 03:00 02/13/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 231, "friends_count": 228, "statues_count": 114610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416328655441920, "text": "Wind 0.0 mph SW. Barometer 29.918 in, Falling. Temperature 17.9F. Rain today 0.00 in. Humidity 92% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 597, "friends_count": 798, "statues_count": 37175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416329129226240, "text": "Why am I up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3430996793, "name": "marlon", "screen_name": "marlon_lll", "lang": "en", "location": "okc", "create_at": date("2015-08-18"), "description": "on my own", "followers_count": 395, "friends_count": 337, "statues_count": 65 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416329322164224, "text": "@ilkae btw… https://t.co/IsSFGZFFpF", "in_reply_to_status": 698415623169245184, "in_reply_to_user": 21555291, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 21555291 }}, "user": { "id": 7162722, "name": "Ted", "screen_name": "tedjamesbutler", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2007-06-29"), "description": "Synthesizer manipulator and Logic Pro trainer. Recording as @_headdress.\n\nHost of cassette-based, underground and experimental music podcast, @norelcomori.", "followers_count": 1429, "friends_count": 874, "statues_count": 25508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln Heights, Los Angeles", "id": "40eacc5cca36da39", "name": "Lincoln Heights", "place_type": "neighborhood", "bounding_box": rectangle("-118.227385,34.060124 -118.191472,34.090787") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416329921986560, "text": "ESPN 8 The Ocho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 859210818, "name": "Zack", "screen_name": "freshlyshipton", "lang": "en", "location": "null", "create_at": date("2012-10-02"), "description": "null", "followers_count": 452, "friends_count": 501, "statues_count": 1652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-02-13T00:00:03.000Z"), "id": 698416330412699649, "text": "We're going to hell idk what else to say https://t.co/xnfu9JxO0d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1271749441, "name": "Lonely Hermit ⚰", "screen_name": "RULER0FHELL", "lang": "en", "location": "Oceanside, California ", "create_at": date("2013-03-16"), "description": "cause we're all a bunch of liars, tell me baby who do you want to be?", "followers_count": 1151, "friends_count": 768, "statues_count": 12592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416331721416708, "text": "@Savage_ness_ OH ITS FUCKING LIT", "in_reply_to_status": 698416222283702272, "in_reply_to_user": 4886631687, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4886631687 }}, "user": { "id": 451668139, "name": "Crim DeLa", "screen_name": "Villacide", "lang": "en", "location": "Wherever Harley Is", "create_at": date("2011-12-31"), "description": "Harley Berry??? ❤️ Me & Her are having a dog together. Montez?? also getting married for a dog. Playing My Part", "followers_count": 643, "friends_count": 588, "statues_count": 7219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416331746508800, "text": "Temp: 12.7F W C: 5.3F Wind:N at 3.8kts Baro: 1029.9mb and Rising Rain today: 0.00in R H: 70% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 192, "friends_count": 219, "statues_count": 96937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416331767435266, "text": "One of my favorite places on earth�� #second home https://t.co/GBPUz5ksYG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "second" }}, "user": { "id": 2760364598, "name": "Jocelyne Zaragoza", "screen_name": "_jocelynesophia", "lang": "en", "location": "null", "create_at": date("2014-08-23"), "description": "do you think the universe fights for two souls to be together?", "followers_count": 467, "friends_count": 390, "statues_count": 6886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332014907393, "text": "Addys???? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2377962129, "name": "Stephh", "screen_name": "ItsSimplySteph_", "lang": "en", "location": "null", "create_at": date("2014-03-02"), "description": "God is within her, she will not fall. ❤️ #LongLiveChillll #UNMLobo", "followers_count": 746, "friends_count": 533, "statues_count": 29049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Valley, NM", "id": "01588ad6021ef7e7", "name": "South Valley", "place_type": "city", "bounding_box": rectangle("-106.736956,34.938902 -106.626779,35.084125") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3574520, "cityName": "South Valley" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332069408768, "text": "Wind 7.0 mph NE. Barometer 1037.90 mb, Steady. Temperature 30.6 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332136587264, "text": "Happy Birthday to my Girlfriend �������� @l0venayeli_ love youu , cant wait till you get back ���� https://t.co/mceSW1ljmu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3241134384 }}, "user": { "id": 3283024819, "name": "isabel®", "screen_name": "queenisaaabel", "lang": "en", "location": "null", "create_at": date("2015-07-17"), "description": "null", "followers_count": 258, "friends_count": 252, "statues_count": 939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332145061888, "text": "This might be the scariest thing I've ever seen.. I found my long… https://t.co/kFSuW8g7Zr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.93164614,35.9489154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2846214730, "name": "Ashlyn", "screen_name": "AshlynWynnss", "lang": "en", "location": "Knoxville, Tennessee", "create_at": date("2014-10-26"), "description": "| Know your worth | SC & IG: ashlynwynns | Vols | Snowboarding | The dirty South |", "followers_count": 834, "friends_count": 378, "statues_count": 2400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332291776512, "text": "Nobody should have to deal with abuse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614617236, "name": "Steph", "screen_name": "Stephanie_3HK", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-06-21"), "description": "stanced imports. i like my steaks medium rare. spanish & native american.", "followers_count": 590, "friends_count": 535, "statues_count": 36323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332304424961, "text": "@oConsciente Chico Buarque é um aproveitador, como muitos filhos de um sistema populista. Ir para Cuba ninguém quer https://t.co/GTzb1eOV9I", "in_reply_to_status": 695005704256634881, "in_reply_to_user": 45295576, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 45295576 }}, "user": { "id": 20653839, "name": "Helio Bueno", "screen_name": "heliobueno", "lang": "en", "location": "Orlando, FL, 32819 USA ", "create_at": date("2009-02-11"), "description": "#ForaPT #ForaDilma #CadeiaProLula #TerroristasNaoMeRepresentam", "followers_count": 433, "friends_count": 745, "statues_count": 10648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332404994048, "text": "Oh, @Delta, be still my heart.\nYou sure know how to throw a @TheGrammys party. @leonbridges… https://t.co/Q4m2TY8BPa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3230286,34.0985794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5920532, 17243213, 46597488 }}, "user": { "id": 485337447, "name": "Dawn McCoy", "screen_name": "iamDawnMcCoy", "lang": "en", "location": "Los Angeles, CA, USA", "create_at": date("2012-02-06"), "description": "Dawn McCoy. LA. Host. Philanthropist. BeautyFrosting Editor. Condé Nast Influencer. Living Healthy writer. Voiceover Vixen. Superhero (shhh,don't tell...)", "followers_count": 62032, "friends_count": 616, "statues_count": 14004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416332799397888, "text": "SATURDAY 2-13-16 https://t.co/u3IzLUrtlV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1090, "friends_count": 306, "statues_count": 3361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416334103646209, "text": "@MrTrevii me felicitas @GloriaTrevi de mi parte Johnny dile q la qeremos mucho y q siga cumpliendo muchos años mas a nuestro lado ����", "in_reply_to_status": 698415409033162752, "in_reply_to_user": 1273556576, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 965409750, 86119466 }}, "user": { "id": 1273556576, "name": "Johnny Ramirez", "screen_name": "Meztli1515", "lang": "en", "location": "El Paso Tx ", "create_at": date("2013-03-16"), "description": "Recoriendo el #Mundo #viviendolo #Depelicula #Elamor #Limu", "followers_count": 362, "friends_count": 300, "statues_count": 12235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anthony, NM", "id": "599fcfa8d9092943", "name": "Anthony", "place_type": "city", "bounding_box": rectangle("-106.623919,32.000098 -106.582891,32.05459") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3503820, "cityName": "Anthony" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416334330208256, "text": "I went to Taco Bell wearing no pants cuz yolo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258217038, "name": "Kales", "screen_name": "kayleepolito", "lang": "en", "location": "null", "create_at": date("2011-02-26"), "description": "a weirdo but I'm real though. | Cali Reckless 15-16", "followers_count": 377, "friends_count": 83, "statues_count": 13687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-02-13T00:00:04.000Z"), "id": 698416334636363777, "text": "@OfficialKimlee hello Kim, Hope you're having a wonderful night.", "in_reply_to_status": 698415900903407617, "in_reply_to_user": 28066028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28066028 }}, "user": { "id": 245591688, "name": "Markus", "screen_name": "kramsta1", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2011-01-31"), "description": "Selected #6 overall in the 2015 #SpursNationDraft Spurs SuperFan #SpursNation #CowboysNation #HookEmHorns Instagram : Instakram23", "followers_count": 1152, "friends_count": 862, "statues_count": 49325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416335601029120, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2454394316, "name": "Jacob", "screen_name": "JacobRey35", "lang": "en", "location": "Westside with Love", "create_at": date("2014-04-20"), "description": ".", "followers_count": 304, "friends_count": 296, "statues_count": 17686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416335949139968, "text": "Thanks Alex ������ https://t.co/hbCwFHAGGL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 305327155, "name": "Aylin Delgado", "screen_name": "AylinEsmeraldaa", "lang": "en", "location": "null", "create_at": date("2011-05-25"), "description": "The one and only", "followers_count": 310, "friends_count": 161, "statues_count": 11421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416336104509440, "text": "@BIPOLARSXUNDS @JpStayGloing lmao no��", "in_reply_to_status": 698415732275683328, "in_reply_to_user": 272081572, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 272081572, 146923249 }}, "user": { "id": 3346585726, "name": "guppy", "screen_name": "tinyandblonde", "lang": "en", "location": "null", "create_at": date("2015-06-26"), "description": "This too shall pass.", "followers_count": 251, "friends_count": 200, "statues_count": 3796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington Heights, NY", "id": "012af97e6dee433b", "name": "Washington Heights", "place_type": "city", "bounding_box": rectangle("-74.457588,41.455924 -74.395662,41.484748") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36071, "countyName": "Orange", "cityID": 3678421, "cityName": "Washington Heights" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416336377094144, "text": "It's 3 a.m. and I'm in a singing mood ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2269777924, "name": "❤️", "screen_name": "_kjwinston", "lang": "en", "location": "null", "create_at": date("2014-01-08"), "description": "blessed", "followers_count": 1097, "friends_count": 963, "statues_count": 24281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416336377147394, "text": "Always done the math as thirteen days and then thirteen until my birthday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288117509, "name": "David Sidhu", "screen_name": "Davidpsidhu", "lang": "en", "location": "Milwaukee", "create_at": date("2011-04-26"), "description": "I've been funny once or twice.", "followers_count": 78, "friends_count": 118, "statues_count": 3175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416336674754560, "text": "Temp: 1.8°F - Dew Point: -17.8° - Wind: 15.1 mph - Gust: 24.2 - Rain Today: 0.00in. - Pressure: 30.49in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416336850935808, "text": "There’s scarcely any happiness, passion or success without struggle. Life is a constant climb,… https://t.co/SspIWQkPnA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.45917,33.99083"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1884847368, "name": "andrea logan", "screen_name": "andreafitness5", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2013-09-19"), "description": "13 yr bulimia survivor, Personal #Trainer with #drhuizenga LAUGH! LOVE! SMILE! #brokenskullchallenge Season 3 #eatingdisorders #celebrities #motivator", "followers_count": 995, "friends_count": 1899, "statues_count": 1726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416337043873792, "text": "@Alexuhhhg @MasterPeeeeen lol that was saved into my photos I just stumbled upon it rn.", "in_reply_to_status": 698416204835229696, "in_reply_to_user": 1959613812, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1959613812, 345005399 }}, "user": { "id": 345005399, "name": "Goodie Pookie Dookie", "screen_name": "MasterPeeeeen", "lang": "en", "location": "Los Banos, CA", "create_at": date("2011-07-29"), "description": "null", "followers_count": 597, "friends_count": 559, "statues_count": 37899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Banos, CA", "id": "b1b6fc646de75904", "name": "Los Banos", "place_type": "city", "bounding_box": rectangle("-120.879833,37.033745 -120.803053,37.088185") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 644028, "cityName": "Los Banos" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416337144586243, "text": "Waiting for her like https://t.co/pgVP0fZxjf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2743397310, "name": "SKYLER ♋️", "screen_name": "oldsoul_69", "lang": "en", "location": "EARTH ", "create_at": date("2014-08-18"), "description": "R.I.P Kory A.", "followers_count": 116, "friends_count": 120, "statues_count": 2170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416337161310208, "text": "Idc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 258601468, "name": "gudalupe", "screen_name": "YazzySays", "lang": "en", "location": "where the bands at $$$$", "create_at": date("2011-02-27"), "description": "Don't forget we sharin hoe", "followers_count": 2261, "friends_count": 1349, "statues_count": 106182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416337312309250, "text": "@Erick_chof I know. :)", "in_reply_to_status": 698412704252301312, "in_reply_to_user": 4572964392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4572964392 }}, "user": { "id": 2728664307, "name": "Heather Barron, HB", "screen_name": "msheatherbarron", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-07-29"), "description": "ASCAP, Singer/Songwriter/Recording Artist, http://www.AllHeather.com", "followers_count": 878, "friends_count": 725, "statues_count": 34821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338100879360, "text": "hi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2688918072, "name": "andi mazzetti", "screen_name": "uunndrea", "lang": "en", "location": "ΔΨ", "create_at": date("2014-07-28"), "description": "• i have the strength for everything through him who empowers me • Marcos❥", "followers_count": 481, "friends_count": 302, "statues_count": 9961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chowchilla, CA", "id": "f977ef0a4904c1e1", "name": "Chowchilla", "place_type": "city", "bounding_box": rectangle("-120.285778,37.10035 -120.220121,37.129769") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 613294, "cityName": "Chowchilla" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338151174145, "text": "3 g blunt , can I roll it ? �� https://t.co/ziE1e5MQpV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560172688, "name": "fonso", "screen_name": "247FONSO", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-04-22"), "description": "it's gone rain ⛈ just duck ☂", "followers_count": 1768, "friends_count": 953, "statues_count": 49301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338386092032, "text": "HAPPY BIRTHDAY SEXY STUD!im so grateful to have such a great friend like u & our relationship is crazy love you qt���� https://t.co/evyedfrwCw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429542461, "name": "jared", "screen_name": "JaredChavez1", "lang": "en", "location": "null", "create_at": date("2011-12-05"), "description": "God is good", "followers_count": 518, "friends_count": 338, "statues_count": 6861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338667184128, "text": "@tatianaxo3 @HannaMusto @WORIDSTARHIPHOP LMFAO", "in_reply_to_status": 698406400234557440, "in_reply_to_user": 523712388, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 523712388, 360221857, 2913627307 }}, "user": { "id": 399617460, "name": "brittt", "screen_name": "britt_matteo101", "lang": "en", "location": "Old Bridge, NJ", "create_at": date("2011-10-27"), "description": "WU", "followers_count": 758, "friends_count": 836, "statues_count": 13387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Bridge, NJ", "id": "eae37a5f578d9179", "name": "Old Bridge", "place_type": "city", "bounding_box": rectangle("-74.383156,40.318344 -74.246891,40.450778") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3454690, "cityName": "Old Bridge" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338721599490, "text": "https://t.co/4VHDU3u0gs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2152926116, "name": "加瀬@ 古乙丁三雨", "screen_name": "Ueharakodai", "lang": "zh-CN", "location": "null", "create_at": date("2013-10-24"), "description": "私ね,上原こだいです,古代わありませ!(っ・д・)三⊃)゚゚),中國人です~よろしくね~(`・ω・´)", "followers_count": 269, "friends_count": 331, "statues_count": 6947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338910519296, "text": "#SupportOriginMelissa 22.3°F Wind:2.5mph Pressure: 30.27hpa Rising Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-13T00:00:05.000Z"), "id": 698416338960666624, "text": "@taylormcgayhee she wouldn't wanna hit that follow button.", "in_reply_to_status": 698416044558278657, "in_reply_to_user": 2873157226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2873157226 }}, "user": { "id": 481595430, "name": "YFN..", "screen_name": "Brettmichaelll_", "lang": "en", "location": "Louisiana ", "create_at": date("2012-02-02"), "description": "the grind starts yesterday.. | 1•4•15 rest easy granny..", "followers_count": 1804, "friends_count": 975, "statues_count": 29364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416340063760384, "text": "Scratch that I'm jammin #TheLifeOfPablo https://t.co/vkdBqmFlnG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheLifeOfPablo" }}, "user": { "id": 230136714, "name": "PackHomie", "screen_name": "iAmTreboR_TSOE", "lang": "en", "location": "Austin, TX", "create_at": date("2010-12-24"), "description": "#MidgetsRule Leader of the #HeManWomanHatersClub....#RipGranny...#WestSide #TSOE #OTB SC: iamtrebor_tsoe #KinectinFabrix", "followers_count": 3039, "friends_count": 2311, "statues_count": 55584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416340080545793, "text": "@ElirScott #worth", "in_reply_to_status": 698411353057308672, "in_reply_to_user": 38047654, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "worth" }}, "user_mentions": {{ 38047654 }}, "user": { "id": 1102963867, "name": "Kathryn Virrey", "screen_name": "katavrei", "lang": "en", "location": "Fremont, CA", "create_at": date("2013-01-18"), "description": "Food blogger, loves to travel and collect buttons from conventions!", "followers_count": 129, "friends_count": 243, "statues_count": 1292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416341703786497, "text": "@aKissBeginswKay what?", "in_reply_to_status": 698413034453073920, "in_reply_to_user": 264527326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 264527326 }}, "user": { "id": 190617423, "name": "Jared Shepherd", "screen_name": "JayDeAnDre", "lang": "en", "location": "Ruston, LA", "create_at": date("2010-09-14"), "description": "LA Tech Track & Field. Currently Grinding so that I won't need a bio to introduce myself in the future...", "followers_count": 1527, "friends_count": 785, "statues_count": 51133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416342441943041, "text": "Why don't u read this & then show some humility. Maybe read the news today as well. https://t.co/OLxbjiKVdx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71607386, "name": "John McGrath", "screen_name": "willibbys", "lang": "en", "location": "The Minneapolis! Uptown!", "create_at": date("2009-09-04"), "description": "Dad/business owner, Populist Progressive. Our Judicial/Legal/Political system is an Organized Crime Syndicate! $$ rules! Absolute Power Corrupts Absolutely!", "followers_count": 725, "friends_count": 2096, "statues_count": 17501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343276613632, "text": "I just woke up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389710371, "name": "•☆Lex☆•", "screen_name": "Lexie_Lou11", "lang": "en", "location": "Riverton", "create_at": date("2011-10-12"), "description": "I'm a bitch.\nSnapchat: lex_11", "followers_count": 472, "friends_count": 384, "statues_count": 44272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverton, UT", "id": "af6effc6288e9761", "name": "Riverton", "place_type": "city", "bounding_box": rectangle("-112.024485,40.492136 -111.913794,40.540969") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4964340, "cityName": "Riverton" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343343722497, "text": "���� https://t.co/NyXWTOOcYS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4748063550, "name": "El jorge", "screen_name": "osegueraa11", "lang": "en", "location": "null", "create_at": date("2016-01-12"), "description": "null", "followers_count": 147, "friends_count": 138, "statues_count": 58 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera Acres, CA", "id": "75161a014946f779", "name": "Madera Acres", "place_type": "city", "bounding_box": rectangle("-120.109807,36.98237 -120.056201,37.03494") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645050, "cityName": "Madera Acres" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343364685824, "text": "@sade_dfwu ew I can't talk to you", "in_reply_to_status": 698360244167172096, "in_reply_to_user": 384614913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384614913 }}, "user": { "id": 2978927319, "name": "Sabrina Thompson", "screen_name": "QueenBina4", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "Snapchat: sabrinamarie0x Instagram: xox_sabrinat", "followers_count": 371, "friends_count": 352, "statues_count": 5063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343591186433, "text": "Haha https://t.co/g0piCw4JZJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2795041765, "name": "✨SL✨", "screen_name": "susy_landa", "lang": "en", "location": "California ", "create_at": date("2014-09-06"), "description": "17✨ |AHS'16| Always enjoy the moment ❣ fries lover", "followers_count": 835, "friends_count": 756, "statues_count": 13552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343662505985, "text": "It's only 11 something but why does it feel like it's 2 something", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2779623683, "name": "Virginia Mitchell", "screen_name": "virgini79233709", "lang": "en", "location": "null", "create_at": date("2014-09-21"), "description": "every love story is beautiful but ours is the best @keizer_ray", "followers_count": 664, "friends_count": 322, "statues_count": 13623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albany, OR", "id": "6083b03ae37cd913", "name": "Albany", "place_type": "city", "bounding_box": rectangle("-123.159583,44.577589 -123.032415,44.684678") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4101000, "cityName": "Albany" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343687766020, "text": "#Standby at E Colonial Dr & N Semoran Blvd, Azalea Park, FL 32807. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3107014,28.5567286"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Standby", "orlpol", "ocso" }}, "user": { "id": 39050017, "name": "Police Calls 32807", "screen_name": "orlpol32807", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 197, "friends_count": 1, "statues_count": 35045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azalea Park, FL", "id": "d698c1040a7d0a4e", "name": "Azalea Park", "place_type": "city", "bounding_box": rectangle("-81.310707,28.515824 -81.285546,28.568387") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1202850, "cityName": "Azalea Park" } }
+{ "create_at": datetime("2016-02-13T00:00:06.000Z"), "id": 698416343876435968, "text": "2/13/2016 - 02:00\nTemp: 44.2F \nHum: 55%\nWind: 2.0 mph\nBaro: 30.461in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 53, "statues_count": 49909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416344044335104, "text": "�� https://t.co/SXQ5uU7k1d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 452632490, "name": "Lissete", "screen_name": "lissayee", "lang": "en", "location": "John Jay College ", "create_at": date("2012-01-01"), "description": "i'm not bad i'm just drawn that way", "followers_count": 743, "friends_count": 682, "statues_count": 18709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linden, NJ", "id": "b8d145028b2571ac", "name": "Linden", "place_type": "city", "bounding_box": rectangle("-74.290477,40.593604 -74.200434,40.654943") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3440350, "cityName": "Linden" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416344501358593, "text": "lmfao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2423088883, "name": "E.R.G.D", "screen_name": "emsicklecell", "lang": "en", "location": "null", "create_at": date("2014-04-01"), "description": "likes cool stuff", "followers_count": 104, "friends_count": 93, "statues_count": 8150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416344966909952, "text": "When he gets hard ������ https://t.co/svRcC4RJkj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458184523, "name": "Brie", "screen_name": "Impetuousbrie", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-01-08"), "description": "null", "followers_count": 471, "friends_count": 373, "statues_count": 12105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416345046618112, "text": "@smcgee_1 ain't got no taste tho kml", "in_reply_to_status": 698416021460422656, "in_reply_to_user": 560375034, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 560375034 }}, "user": { "id": 309458805, "name": "G-slabo", "screen_name": "youngslab_", "lang": "en", "location": "wish you were here", "create_at": date("2011-06-01"), "description": "life was never easy kinda always hard for me!!!", "followers_count": 2089, "friends_count": 907, "statues_count": 99692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416345713479680, "text": "Just wanna mean something to you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2908709815, "name": "C_SMITH", "screen_name": "lilbabysmith1", "lang": "en", "location": "Mount Pleasant, MI", "create_at": date("2014-11-23"), "description": "CMU '19 | R.I.Paradise Natalie Kay Swain Always loved never forgotten | pcb 1824| @kelseyalafriz❤️ |", "followers_count": 594, "friends_count": 409, "statues_count": 8525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pleasant, MI", "id": "77f923f5a841bbdf", "name": "Mount Pleasant", "place_type": "city", "bounding_box": rectangle("-84.816945,43.553246 -84.727407,43.63446") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26073, "countyName": "Isabella", "cityID": 2656020, "cityName": "Mount Pleasant" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416346028085248, "text": "I got my hair cut, like really short, the shortest I've ever gotten and I feel good. https://t.co/w1SK3p43H2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1644662940, "name": "cece.1211", "screen_name": "thisangelfan", "lang": "en", "location": "Go Ångels. ", "create_at": date("2013-08-04"), "description": "null", "followers_count": 157, "friends_count": 160, "statues_count": 2835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416346305052672, "text": "@havenrockelle get yo bff", "in_reply_to_status": -1, "in_reply_to_user": 708771747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 708771747 }}, "user": { "id": 2951384420, "name": "Krissssssten✨", "screen_name": "Whitegalkris", "lang": "en", "location": "yo nigga pockets", "create_at": date("2014-12-29"), "description": "Follow Me On Instagram @Whitegalkris__ Amosc|Whitegalkris", "followers_count": 1929, "friends_count": 997, "statues_count": 19589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-02-13T00:00:07.000Z"), "id": 698416346305064960, "text": "3️⃣3️⃣3️⃣", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 519142838, "name": "TDA General 3️⃣3️⃣", "screen_name": "HotboyWes3", "lang": "en", "location": "|SC|Ballisforever3", "create_at": date("2012-03-08"), "description": "#RipTydale #RipL'A #FreeUncleJ #FreeJojo #TDA 401 9.22", "followers_count": 425, "friends_count": 411, "statues_count": 3661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778702654603264, "text": "Where you need to be drinking when you're here. @ Counter Intuitive https://t.co/sHfgz1XTTg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9275254,33.4991271"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43186958, "name": "Charles Joly", "screen_name": "Charles_Joly", "lang": "en", "location": "Chicago, Il", "create_at": date("2009-05-28"), "description": "Chicago native, global bartending champ, founder @DrinkCrafthouse Shaking around the globe. Runner, writer, antiquer & rescues #AdoptDontShop", "followers_count": 6999, "friends_count": 402, "statues_count": 2220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778702927294465, "text": "Keke Palmer\n\nhttps://t.co/TiBLP1fUlA @KekePalmer RT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 29860364 }}, "user": { "id": 59045453, "name": "(407)-844-7713", "screen_name": "Prince_Jnuff", "lang": "en", "location": "Studio or n sum guts n Orlando", "create_at": date("2009-07-21"), "description": "CEO OF HUMBLE BEGINNING RECORDS| RICH BY 23| the mic is a deadly weapon in the right mans hand| EVERYBODY RAP BUT NOT EVERYBODY'S GOOD| 4078447713| HAITIAN", "followers_count": 4170, "friends_count": 104, "statues_count": 144713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocoee, FL", "id": "d95d0296a4b0acf3", "name": "Ocoee", "place_type": "city", "bounding_box": rectangle("-81.648788,28.518918 -81.49358,28.692289") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1251075, "cityName": "Ocoee" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778702990036993, "text": "Fuck no https://t.co/qDW7HYMmut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1350151392, "name": "Austen♓️", "screen_name": "AustenHdz_", "lang": "en", "location": "LA", "create_at": date("2013-04-13"), "description": "@_xxkvt ❤️", "followers_count": 1596, "friends_count": 662, "statues_count": 39062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703384346629, "text": "Definitely going to go see the new purge movie once it comes out !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609504172, "name": "Julie Alvarez", "screen_name": "juliebugjewls", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": "IG: Julieealvarez", "followers_count": 500, "friends_count": 169, "statues_count": 4131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703401103361, "text": "@white__o #whyomar", "in_reply_to_status": -1, "in_reply_to_user": 168791472, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "whyomar" }}, "user_mentions": {{ 168791472 }}, "user": { "id": 2238449419, "name": "Mendezz", "screen_name": "joseeemendezz", "lang": "en", "location": "Renton WA ", "create_at": date("2013-12-09"), "description": "I got really big wings.", "followers_count": 294, "friends_count": 252, "statues_count": 10811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Renton, WA", "id": "a96f188f5de647b5", "name": "Renton", "place_type": "city", "bounding_box": rectangle("-122.253717,47.423263 -122.135079,47.539933") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357745, "cityName": "Renton" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703413669888, "text": "I think I need a break from everything right now... My attitude is way too over the top for this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488057380, "name": "IRunHurdles", "screen_name": "Texas_Chick015", "lang": "en", "location": "-RIPTariqaKeshaughn&Semaj-", "create_at": date("2012-02-09"), "description": "#Sagittarius #TrackNation #Football #SouthernGirl #Albion019", "followers_count": 960, "friends_count": 638, "statues_count": 53583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texarkana, TX", "id": "b33bfb7f88f92c6c", "name": "Texarkana", "place_type": "city", "bounding_box": rectangle("-94.151573,33.361206 -94.042938,33.500264") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48037, "countyName": "Bowie", "cityID": 4872368, "cityName": "Texarkana" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703552258052, "text": "\"My ex says she gave me the best years of her life-I saw a recent picture of her, I guess she was right\" #30hours #TLOP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "30hours", "TLOP" }}, "user": { "id": 1151255917, "name": "Felton Kizer", "screen_name": "feltonkizer", "lang": "en", "location": "USA", "create_at": date("2013-02-05"), "description": "Photographer & Editor of Off-Kilter Magazine| @feltonkizer on all social media. If you want to reach me: felton@offkiltermag.com", "followers_count": 463, "friends_count": 279, "statues_count": 11095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703623401472, "text": "Too bad no one asked me to be their valentine :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125815005, "name": "Rica Distor", "screen_name": "ricadistor", "lang": "en", "location": "Disneyland", "create_at": date("2010-03-23"), "description": "alain♡ | cal lu❁ | insta: ricadistor", "followers_count": 1071, "friends_count": 214, "statues_count": 11056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778703782813698, "text": "@pitahhh73 na I didn't. Might go next weekend though ��", "in_reply_to_status": 698778302832467968, "in_reply_to_user": 931292520, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 931292520 }}, "user": { "id": 427217101, "name": "jose", "screen_name": "hoozayisdead", "lang": "en", "location": "ucsb", "create_at": date("2011-12-02"), "description": "no pun intended", "followers_count": 256, "friends_count": 143, "statues_count": 53033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778704009306112, "text": "It's 2016 and relationship drama over Xbox live is a thing I guess https://t.co/8TotOi6WgI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2444842182, "name": "Búho", "screen_name": "TheDeadnd", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "my name is Daniel", "followers_count": 102, "friends_count": 85, "statues_count": 406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orangevale, CA", "id": "d950fc7dc2cd14f9", "name": "Orangevale", "place_type": "city", "bounding_box": rectangle("-121.262498,38.637436 -121.187521,38.718208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 654092, "cityName": "Orangevale" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778704642572288, "text": "No https://t.co/frM6RNtWjd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2514406556, "name": "molly", "screen_name": "yungfuegz", "lang": "en", "location": "Bay Area made, LA paid", "create_at": date("2014-05-21"), "description": "M•A•C Makeup Artist | Instagram: yungfuegz", "followers_count": 392, "friends_count": 309, "statues_count": 5695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778704902819840, "text": "These verses on Famous ���� #tlop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tlop" }}, "user": { "id": 237141366, "name": "Kobe", "screen_name": "CallHerJasmiine", "lang": "en", "location": "SC: CallHerJasmiine", "create_at": date("2011-01-11"), "description": "Whats understood doesnt need to be explained Kobe| Nursing | Houston | Blessed |#ClassyNeverTrashy", "followers_count": 1691, "friends_count": 468, "statues_count": 66740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778705401786368, "text": "Nvm not future lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1695101472, "name": "Leo", "screen_name": "LeomarieR", "lang": "en", "location": "ProvRI to HouTX.", "create_at": date("2013-08-23"), "description": "Mami Millie RIP , I love you forever #RIPTio #ForEverAnt #RIPChaseHawkins", "followers_count": 1558, "friends_count": 545, "statues_count": 243022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778705691328512, "text": "i had the biggest crush on her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2408232570, "name": "Pete Adida", "screen_name": "PeteOr30", "lang": "en", "location": "dung ah yaad mih cum fram", "create_at": date("2014-03-23"), "description": "R.I.P @54TRILLIONAIRE", "followers_count": 657, "friends_count": 333, "statues_count": 27282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, FL", "id": "2bc7c264a080898b", "name": "Hollywood", "place_type": "city", "bounding_box": rectangle("-80.248975,25.986228 -80.107808,26.093132") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1232000, "cityName": "Hollywood" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778706395983872, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 485219034, "name": "JKing✊", "screen_name": "JayJayKing_", "lang": "en", "location": "19 • #SlimeCity", "create_at": date("2012-02-06"), "description": "Welcome to the new movement, We Here to save the world with Drugz & #Potion. #YSLfolifeBitch, I'm the man to fuck with❗️", "followers_count": 1074, "friends_count": 537, "statues_count": 126983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westland, MI", "id": "879dfee0187b884a", "name": "Westland", "place_type": "city", "bounding_box": rectangle("-83.430458,42.267434 -83.308978,42.355177") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2686000, "cityName": "Westland" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778706542616577, "text": "I don't even need to wear bras I don't have any boobs lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.232873,33.894115"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378391259, "name": "Larissa the hoe", "screen_name": "laritsunami", "lang": "en", "location": "1017", "create_at": date("2014-03-08"), "description": "5'2 with a whole lotta latina attitude", "followers_count": 797, "friends_count": 406, "statues_count": 33330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-02-14T00:00:00.000Z"), "id": 698778706630864896, "text": "@WorldsApart_RT @RussiaUN the concept that the creator of existence is a bias bigot is a lie https://t.co/qRLk0GKYXR", "in_reply_to_status": 698778295446413312, "in_reply_to_user": 47896593, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2324946854, 347443535 }}, "user": { "id": 47896593, "name": "Leonard Shand", "screen_name": "shandclub", "lang": "en", "location": "MD", "create_at": date("2009-06-17"), "description": "BORN IN THIS WORLD AS HUMAN AMONG OTHER HUMANS.\r\nhttp://wikileaks.org/ http://www.youtube.com/shandclub", "followers_count": 1659, "friends_count": 1775, "statues_count": 26329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Riverdale, MD", "id": "2163e3af2838ce1c", "name": "East Riverdale", "place_type": "city", "bounding_box": rectangle("-76.930889,38.94415 -76.891633,38.972339") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2424650, "cityName": "East Riverdale" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778706844770305, "text": "@HndrixLamar these are the type of people you need in your life, that that trash calling Pablo \"average\"", "in_reply_to_status": 698772368068579328, "in_reply_to_user": 56794575, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 182011274 }}, "user": { "id": 56794575, "name": "James Healey", "screen_name": "SwedishLincoln", "lang": "en", "location": "New York, New York", "create_at": date("2009-07-14"), "description": "Director. Screenwriter. Producer. Penis Boy. Fever Dream.", "followers_count": 465, "friends_count": 324, "statues_count": 11296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707020935168, "text": "Ripley SW Limestone Co. Temp: 25.3°F Wind:4.5mph Pressure: 1006.6mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 48788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707314544640, "text": "02/14@03:00 - Temp 11.1F, WC 5.6F. Wind 3.0mph WNW, Gust 8.0mph. Bar 30.438in, Rising slowly. Rain 0.00in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 46994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707423408129, "text": "#tb to a while agoooo �� https://t.co/gVdJuYJRbu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tb" }}, "user": { "id": 3008782172, "name": "hunt ☄", "screen_name": "hunterrraee", "lang": "en", "location": "mias house ", "create_at": date("2015-02-02"), "description": "hennything is possible", "followers_count": 748, "friends_count": 491, "statues_count": 15181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707490516992, "text": "❤️���� thanks baby cakes. https://t.co/RchbeHyTP2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 863973164, "name": "R I D A H", "screen_name": "Katie_Poeskii", "lang": "en", "location": "htx", "create_at": date("2012-10-05"), "description": "jovoni kentrell mitchell", "followers_count": 523, "friends_count": 287, "statues_count": 25206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707503108096, "text": "Me and Blaire the DDs for like 6 different people tonight����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2781610254, "name": "Bailey LaFleur", "screen_name": "baileyyreneee96", "lang": "en", "location": "null", "create_at": date("2014-08-30"), "description": "null", "followers_count": 77, "friends_count": 66, "statues_count": 1245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crowley, LA", "id": "6728e3852e199bb5", "name": "Crowley", "place_type": "city", "bounding_box": rectangle("-92.403051,30.190648 -92.325096,30.247285") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22001, "countyName": "Acadia", "cityID": 2218650, "cityName": "Crowley" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778707784110080, "text": "Well dang��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2505364526, "name": "Isaac Roy", "screen_name": "isaacroy00", "lang": "en", "location": "null", "create_at": date("2014-05-18"), "description": "VHS boys soccer/ life is a one time thing make the most of it//", "followers_count": 328, "friends_count": 226, "statues_count": 660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708191019008, "text": "y'all go follow my PHHHOTO: franny15 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 526171520, "name": "franmescudi✨", "screen_name": "Franthemann", "lang": "en", "location": "Cudi Zone", "create_at": date("2012-03-16"), "description": "good at being bad", "followers_count": 1206, "friends_count": 950, "statues_count": 30722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708207734784, "text": "@ZooeyDeschanel https://t.co/ld017bICoI", "in_reply_to_status": -1, "in_reply_to_user": 66561957, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 66561957 }}, "user": { "id": 23120491, "name": "Sonny Spencer", "screen_name": "sono23", "lang": "en", "location": "Seattle/Tacoma (206/253)", "create_at": date("2009-03-06"), "description": "lover of Seattle sports. traveling, movie going, music listening", "followers_count": 604, "friends_count": 1317, "statues_count": 23061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198337") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708283432960, "text": "Wind 0 mph --. Barometer 30.28 in, Falling. Temperature 33.6 °F. Rain today 0.00 in. Humidity 44%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708308414464, "text": "In the feels right now��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3152263266, "name": "Ithel Carrillo", "screen_name": "ithel_3", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "Baseball is life#Vsquad#8⚾️#65God is a huge part of my life and with him everything is possible!!", "followers_count": 178, "friends_count": 290, "statues_count": 355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hebbronville, TX", "id": "6c80b8c28fd29be3", "name": "Hebbronville", "place_type": "city", "bounding_box": rectangle("-98.694359,27.292917 -98.663161,27.323508") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48247, "countyName": "Jim Hogg", "cityID": 4833008, "cityName": "Hebbronville" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708463595520, "text": "To live my dream.... Is incredible! \n@dmx @ Reverb https://t.co/Q5nNbqTMbF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.9196557,40.3564315"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 350628549 }}, "user": { "id": 1873130328, "name": "red city mix", "screen_name": "redcitymix", "lang": "en", "location": "null", "create_at": date("2013-09-16"), "description": "in the heart of reading pa... come check us out, RED CITY MIX. THE LATEST IN HIP HOP AND RNB LATIN REGGAE OLD SCHOOL AND MORE.....", "followers_count": 1162, "friends_count": 271, "statues_count": 448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reading, PA", "id": "3e94c70e7a8a35d3", "name": "Reading", "place_type": "city", "bounding_box": rectangle("-75.95954,40.29676 -75.890503,40.371477") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4263624, "cityName": "Reading" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708463620097, "text": "@classybertha Nope I'm doing what people do to Kendrick and Cole", "in_reply_to_status": 698778475931381761, "in_reply_to_user": 395094986, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 395094986 }}, "user": { "id": 2915933862, "name": "Bullet Club 4-4-4-4", "screen_name": "Marcus_Mims213", "lang": "en", "location": "Killeen, TX", "create_at": date("2014-12-01"), "description": "23 from Killeen TX Hall Of Pain", "followers_count": 498, "friends_count": 778, "statues_count": 28735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708799152128, "text": "@ebbtideapp Tide in Seaside Heights, New Jersey 02/14/2016\n Low 5:53am -0.1\nHigh 11:58am 4.3\n Low 6:08pm -0.2\nHigh 12:28am 4.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.0683,39.9417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 28, "friends_count": 1, "statues_count": 7602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Jersey, USA", "id": "65b4760a2b411e11", "name": "New Jersey", "place_type": "admin", "bounding_box": rectangle("-75.563587,38.788657 -73.88506,41.357424") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778708954337280, "text": "@Dsilva_001 ��", "in_reply_to_status": 698736981765795841, "in_reply_to_user": 1144447692, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1144447692 }}, "user": { "id": 1144447692, "name": "Da Silva", "screen_name": "Dsilva_001", "lang": "en", "location": "probs doing HW or at the field", "create_at": date("2013-02-02"), "description": "OMWU", "followers_count": 555, "friends_count": 303, "statues_count": 26641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709294247937, "text": "Wind 1.0 mph NW. Barometer 30.588 in, Falling slowly. Temperature 5.8 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 39, "friends_count": 4, "statues_count": 25253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709449273344, "text": "Happy #NoValentineDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NoValentineDay" }}, "user": { "id": 2522541337, "name": "Dats", "screen_name": "dats526", "lang": "en", "location": "null", "create_at": date("2014-05-25"), "description": "Just keep things real with me.", "followers_count": 170, "friends_count": 189, "statues_count": 11578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709633818624, "text": "block me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1569926738, "name": "kd", "screen_name": "KdTheBully", "lang": "en", "location": "Oroville, CA", "create_at": date("2013-07-05"), "description": "sc: k-dizzle22", "followers_count": 402, "friends_count": 332, "statues_count": 8634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oroville East, CA", "id": "2a152e11137e65d0", "name": "Oroville East", "place_type": "city", "bounding_box": rectangle("-121.538364,39.479173 -121.456825,39.53656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 654388, "cityName": "Oroville East" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709646401536, "text": "2AM watching film ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874497010, "name": "APRIL 25‼️", "screen_name": "Schoolboylouuuu", "lang": "en", "location": "on the field ", "create_at": date("2014-10-23"), "description": "Hoka hey 5⃣8⃣ Savage$eason", "followers_count": 377, "friends_count": 452, "statues_count": 2468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709814214657, "text": "Im tired of seeing/hearing about Kanye, could he really be that great? Doubt it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3258097148, "name": "El Rübê", "screen_name": "RubePreciado", "lang": "en", "location": "Subvert City, CA", "create_at": date("2015-06-27"), "description": "Stay cold/belong to @mandeficent", "followers_count": 359, "friends_count": 323, "statues_count": 1075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Verne, CA", "id": "7183cae332544afd", "name": "La Verne", "place_type": "city", "bounding_box": rectangle("-117.80156,34.086183 -117.733148,34.165358") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640830, "cityName": "La Verne" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778709885480960, "text": "first of all. if you don't just call it crack,\nyou are out of touch.\nsounding like 1987 Geraldo.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20809663, "name": "jessie sponberg", "screen_name": "ozone2016", "lang": "en", "location": "ÜT: 45.432388,-122.572227", "create_at": date("2009-02-13"), "description": "activist,journalist,lyricist, father,brother,homie...", "followers_count": 1813, "friends_count": 668, "statues_count": 31891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778710162477056, "text": "@Salxcci how do u unsubscribe", "in_reply_to_status": 698778018831937536, "in_reply_to_user": 346967398, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346967398 }}, "user": { "id": 1155383911, "name": "GOSPURSGO", "screen_name": "remeshia29", "lang": "en", "location": "the sip", "create_at": date("2013-02-06"), "description": "null", "followers_count": 977, "friends_count": 1891, "statues_count": 9017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, MS", "id": "1ba55b8dd0acc0a3", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-89.17716,32.750158 -89.076924,32.803088") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28099, "countyName": "Neshoba", "cityID": 2856960, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778710351085569, "text": "PSA: watching a funny movie to try and cheer you up is not always effective✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1373470753, "name": "KT", "screen_name": "katie_lady19", "lang": "en", "location": "null", "create_at": date("2013-04-22"), "description": "probably thinking about pizza", "followers_count": 395, "friends_count": 315, "statues_count": 10460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778710434926592, "text": "Happy Valentine's Day �� https://t.co/lLeJ7l4ide", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371651227, "name": "yvonne", "screen_name": "yvoonnelieeu", "lang": "en", "location": "null", "create_at": date("2011-09-11"), "description": "know your worth", "followers_count": 386, "friends_count": 331, "statues_count": 13319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778710590300160, "text": "the life of pablo https://t.co/9cXpMbGu3x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.163238,41.172492"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59332550, "name": "bitch im on 12 xanax", "screen_name": "wavyxcecil", "lang": "en", "location": "null", "create_at": date("2009-07-22"), "description": "trust nobody... not even nobody", "followers_count": 268, "friends_count": 205, "statues_count": 17762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "LeCoq Cuisine", "id": "07d9df735b886001", "name": "LeCoq Cuisine", "place_type": "poi", "bounding_box": rectangle("-73.1632381,41.1724919 -73.163238,41.172492") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 908000, "cityName": "Bridgeport" } }
+{ "create_at": datetime("2016-02-14T00:00:01.000Z"), "id": 698778710594318336, "text": "@melissatvrs GUESS WHO THE FUCK IS HERE ...", "in_reply_to_status": -1, "in_reply_to_user": 118552411, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 118552411 }}, "user": { "id": 460364940, "name": "Leylo•", "screen_name": "TYPSYxGYPSY", "lang": "en", "location": "dogtown.", "create_at": date("2012-01-10"), "description": "Typsy Gypsy. I'm actually a spice girl. Writing is my craft, the universe is my muse. Welcome to the dark side of my bizzare mind. #starvingartistsociety", "followers_count": 956, "friends_count": 572, "statues_count": 22153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778711244537857, "text": "@melissarustemov I just don't want guys who do drugs and drink every night representing our football team ya know", "in_reply_to_status": 698778365667332097, "in_reply_to_user": 33759576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33759576 }}, "user": { "id": 324046524, "name": "tharpyyy", "screen_name": "tharpyyyy", "lang": "en", "location": "probz chipotle ", "create_at": date("2011-06-25"), "description": "null", "followers_count": 1204, "friends_count": 673, "statues_count": 14812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778711483494400, "text": "lmao just lost my last friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2249843198, "name": "man eating whale", "screen_name": "ELECTR0CUTIE", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-12-16"), "description": "every guy i've ever seen thinks i have a crush on them lmao", "followers_count": 323, "friends_count": 306, "statues_count": 10742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778711672360961, "text": "Wind 2.0 mph E. Barometer 30.510 in, Falling slowly. Temperature 16.3 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 51, "friends_count": 26, "statues_count": 17618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778711760445440, "text": "Where is Redding on the map? Play the game at https://t.co/eA0Ef4kZEe #Redding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.392,40.5865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Redding" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2593038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778711898910720, "text": "Wind 1.0 mph ESE. Barometer 30.403 in, Falling. Temperature 23.7 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712070877184, "text": "If a girl has a fat friend nine times out of ten she is the cock blocker! �������� like bih go eat a sandwich and go to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140193931, "name": "Corey T", "screen_name": "StrapCity45", "lang": "en", "location": "null", "create_at": date("2010-05-04"), "description": "SC: Dirkanthony33 IG: Strap_Gawd45", "followers_count": 1347, "friends_count": 2008, "statues_count": 53081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712389582848, "text": "I have trust issues and I'm sorry.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2342681352, "name": "_dirtygiana", "screen_name": "_dirtygiana", "lang": "en", "location": "FL ❁", "create_at": date("2014-02-13"), "description": "She loved three things - a joke, a glass of wine, and a handsome man.", "followers_count": 1576, "friends_count": 914, "statues_count": 31363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocala, FL", "id": "01966db443c4c238", "name": "Ocala", "place_type": "city", "bounding_box": rectangle("-82.293902,28.962902 -82.027829,29.277543") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12083, "countyName": "Marion", "cityID": 1250750, "cityName": "Ocala" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712410423296, "text": "Temp: 35.5°F Wind:0.1mph Pressure: 30.311hpa Falling Rapidly Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 59885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712427245569, "text": "Wish I could be spending this Valentine's Day with you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454448922, "name": "∂яєααα", "screen_name": "whos_andrea", "lang": "en", "location": "CαΙιƒοrηια", "create_at": date("2012-01-03"), "description": "lowkey my glasses are superglued to my face.", "followers_count": 1489, "friends_count": 907, "statues_count": 37273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712611815425, "text": "tomorrow about to be lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3311968586, "name": "bryttnee [:", "screen_name": "captainbitx", "lang": "en", "location": "Apple Valley, CA", "create_at": date("2015-08-10"), "description": "devin is my love❤️", "followers_count": 183, "friends_count": 233, "statues_count": 4886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, CA", "id": "41c331abe42d9969", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-117.287533,34.413411 -117.103437,34.571843") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 602364, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778712754487296, "text": "Finna get this mc 2 for 2 and call it a mcnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314096414, "name": "BoBerry Biscuitجيري", "screen_name": "JBLive315", "lang": "en", "location": "UNCC", "create_at": date("2011-06-09"), "description": "football and rap music", "followers_count": 560, "friends_count": 398, "statues_count": 113298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mint Hill, NC", "id": "04e683accecac89d", "name": "Mint Hill", "place_type": "city", "bounding_box": rectangle("-80.71402,35.124954 -80.588554,35.268548") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3743480, "cityName": "Mint Hill" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778713077497856, "text": "I'm at Meridian Place in Northridge, CA https://t.co/Y7tLexjuVX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 373, "friends_count": 336, "statues_count": 25547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778713123459072, "text": "@AdrianMontanez1 you added the lol BC you knew I was coming after you ����", "in_reply_to_status": 698778379282034688, "in_reply_to_user": 3237282254, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3237282254 }}, "user": { "id": 2576658053, "name": "L(arry)aura", "screen_name": "tommo_thehommo", "lang": "en", "location": "Dark Larrie, USA", "create_at": date("2014-06-01"), "description": "Always in my heart @Harry_Styles. Yours sincerely, Louis", "followers_count": 265, "friends_count": 611, "statues_count": 805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778713442402304, "text": "I'm certainly warmed up in here. @lcexaminer @eventbrite nyfw #nyfw #valentinesday #LoveIsHome… https://t.co/N0Z1WEcAf4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9870071,40.7596397"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nyfw", "valentinesday", "LoveIsHome" }}, "user_mentions": {{ 92343524, 5625972 }}, "user": { "id": 92343524, "name": "Daniel Quintanilla", "screen_name": "lcexaminer", "lang": "en", "location": "Washington, DC", "create_at": date("2009-11-24"), "description": "An adoring fan of Lauren Conrad and The Hills, I provide insight that comes from the top down. Not based on gossip.", "followers_count": 170, "friends_count": 210, "statues_count": 3675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778713618407424, "text": "Temp: -6.2F W C: -6.2F Wind:--- at 0.0kts Baro: 1039.4mb and Rising slowly Rain today: 0.00in R H: 86% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 193, "friends_count": 219, "statues_count": 97033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778713651949568, "text": "51.1F (Feels: 51.1F) - Humidity: 90% - Wind: 6.0mph NE - Gust: 8.3mph - Pressure: 1034.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778714335653888, "text": "I believe that's it soooo https://t.co/KxyQuFWjxl", "in_reply_to_status": 698778298931769344, "in_reply_to_user": 896933665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 896933665, "name": "Gabs ♡", "screen_name": "gaaabby_nicole", "lang": "en", "location": "Phs Senior", "create_at": date("2012-10-22"), "description": "It's Twitter Don't Take It Seriously", "followers_count": 1013, "friends_count": 566, "statues_count": 90033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778714511962112, "text": "Wind 2.0 mph NW. Barometer 30.420 in, Rising Rapidly. Temperature -17.3 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-14T00:00:02.000Z"), "id": 698778714834759680, "text": "Wind 4.1 mph NW. Barometer 30.493 in, Rising Rapidly. Temperature -8.7 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 6504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778715128520704, "text": "Your hand fits in mine like it's made just for me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3095403887, "name": "megan || ✌️", "screen_name": "megsxelizabeth", "lang": "en", "location": "nc❤️ i love you L ❤️", "create_at": date("2015-03-18"), "description": "Use your smile to change the world. ~N.H.", "followers_count": 551, "friends_count": 1113, "statues_count": 18261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winterville, NC", "id": "8970c1a7ca3f6ac8", "name": "Winterville", "place_type": "city", "bounding_box": rectangle("-77.43544,35.512579 -77.371027,35.556317") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37147, "countyName": "Pitt", "cityID": 3775060, "cityName": "Winterville" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778715480666112, "text": "holy fuuuck https://t.co/X6TRW1TLHQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 913755007, "name": "snack cake", "screen_name": "heypunkass", "lang": "en", "location": "null", "create_at": date("2012-10-29"), "description": "I spent a long time dying, don't wake me up yet. \n03.23.13 / @t_bearz.", "followers_count": 91, "friends_count": 72, "statues_count": 22729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716118376448, "text": "Can you find Redding on the map? Just try it at https://t.co/eA0Ef4kZEe #Redding", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.392,40.5865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Redding" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2593039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716130844672, "text": "@EmmyCorinne sexy ��������", "in_reply_to_status": 698778204434083840, "in_reply_to_user": 569279897, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 569279897 }}, "user": { "id": 388450833, "name": "Pete Puebla Jr.", "screen_name": "PeteyPablo623", "lang": "en", "location": "All over Southern California", "create_at": date("2011-10-10"), "description": "I'm in the business of helping people and cleaning California while doing it.", "followers_count": 1710, "friends_count": 2068, "statues_count": 119101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716307124225, "text": "So sweet of you Maine. Enjoy your date https://t.co/ZHJcLsgRlM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4043091274, "name": "MNM", "screen_name": "MNM39244711", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2015-10-25"), "description": "null", "followers_count": 8, "friends_count": 53, "statues_count": 231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Pocono, PA", "id": "213c174ad5136323", "name": "Mount Pocono", "place_type": "city", "bounding_box": rectangle("-75.378698,41.11041 -75.341345,41.139387") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42089, "countyName": "Monroe", "cityID": 4251912, "cityName": "Mount Pocono" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716609056768, "text": "BALLOON DROP!!! Who all got FREE BULL BUCKS?!?!? Enter the RAFFLE for more! Click the link in… https://t.co/fuQbAsEy5F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.63001956,32.83594613"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4190374167, "name": "The Crazy Bull", "screen_name": "TheCrazyBullMac", "lang": "en", "location": "Macon, GA", "create_at": date("2015-11-14"), "description": "Like us on Facebook! Instagram: @thecrazybullmac Snapchat: thecrazybullGA", "followers_count": 37, "friends_count": 26, "statues_count": 56 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Macon, GA", "id": "11a2c1d62d4084d1", "name": "Macon", "place_type": "city", "bounding_box": rectangle("-83.786106,32.75281 -83.522053,32.944825") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13021, "countyName": "Bibb", "cityID": 1349008, "cityName": "Macon-Bibb County" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716625719296, "text": "Lyft driver just asked about my plans for tomorrow. Is it rude that I laughed as a response!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41530567, "name": "Eiselle Ty", "screen_name": "eisellecruzty", "lang": "en", "location": "San Francisco", "create_at": date("2009-05-20"), "description": "I like stuff--- the good kind.", "followers_count": 288, "friends_count": 211, "statues_count": 11120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716743204864, "text": "ha haha https://t.co/zhhnko4pWd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 805654986, "name": "sophie mcguire", "screen_name": "sophiemcguiree", "lang": "en", "location": "null", "create_at": date("2012-09-05"), "description": "null", "followers_count": 315, "friends_count": 226, "statues_count": 9893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frisco, TX", "id": "00c3e46c3d0cac08", "name": "Frisco", "place_type": "city", "bounding_box": rectangle("-96.937783,33.081206 -96.732715,33.219073") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4827684, "cityName": "Frisco" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716822896641, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 434710882, "name": "Kody MF Frenna", "screen_name": "K1ngKody", "lang": "en", "location": "LasVegas #RipAbout #RipReba ", "create_at": date("2011-12-11"), "description": "#whereskemor", "followers_count": 393, "friends_count": 254, "statues_count": 7343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778716835426306, "text": "I really embarrassed myself tonight��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295141281, "name": "Chocolate Drop", "screen_name": "Tvann25", "lang": "en", "location": "Yukon, OK", "create_at": date("2011-05-08"), "description": "I do it for the fam •Hakuna Matata• UCO19 #AlwaysHungry SC: tinovann", "followers_count": 1147, "friends_count": 793, "statues_count": 16113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778717007409152, "text": "Dallas on blast in this episode do better y’all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17903718, "name": "Zach", "screen_name": "zachrab88", "lang": "en", "location": "null", "create_at": date("2008-12-05"), "description": "Live love laugh", "followers_count": 307, "friends_count": 691, "statues_count": 14114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Loop, Austin", "id": "0d9306eec62f41df", "name": "North Loop", "place_type": "neighborhood", "bounding_box": rectangle("-97.73084,30.301909 -97.70696,30.32598") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778717498118145, "text": "happy valentine's day !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3053853900, "name": "joy", "screen_name": "joyemoto", "lang": "en", "location": "null", "create_at": date("2015-03-01"), "description": "♑️ // be kind", "followers_count": 377, "friends_count": 395, "statues_count": 2281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cypress, CA", "id": "a077793faceeda6f", "name": "Cypress", "place_type": "city", "bounding_box": rectangle("-118.063298,33.791963 -118.010668,33.845856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 617750, "cityName": "Cypress" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778717657681920, "text": "\"Here's the thing, she's gonna hate you eventually regardless so you mine as well start with it\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234170179, "name": "Karmyn Siefring", "screen_name": "KarmynSiefring", "lang": "en", "location": "null", "create_at": date("2015-06-02"), "description": "just liven life", "followers_count": 44, "friends_count": 64, "statues_count": 453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778717917675520, "text": "Temp: 46.6°F | Humidity: 84% | Wind: --- @ 0.0 mph | Barometer: 30.26 in | Dewpoint: 42.0°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 165167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778717963870208, "text": "Barberton Oh Temp:-1.0°F Wind:0 mph Dir:--- Baro:Rising slowly Rain2day:0.00in Hum:76% UV:0.0 @ 03:00 02/14/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718198562816, "text": "@Cory_Henry hopefully one day I get to sit in with you guys, I'll never stop working and getting better tho. Thanks for being inspiring.��", "in_reply_to_status": 698778095252156416, "in_reply_to_user": 72051169, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 72051169 }}, "user": { "id": 2654907134, "name": "Raliegh Jones", "screen_name": "RajoThe3rd", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "clean slate", "followers_count": 157, "friends_count": 207, "statues_count": 796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718274084864, "text": "I just attempted to make an armpit Valentine's Day card but the joke is fucking WEAK Harry would be ashamed of my pun game", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145069518, "name": "sara", "screen_name": "lourrie28", "lang": "en", "location": "louis' right armpit ", "create_at": date("2015-04-08"), "description": "she/her + queer af", "followers_count": 1838, "friends_count": 121, "statues_count": 28346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718349582337, "text": "Happy 20th Birthday Lucas!! ���� @lucasdell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 439393261 }}, "user": { "id": 1337897898, "name": "Erik Fontaney", "screen_name": "erikfontaney", "lang": "en", "location": "san antonio, texas", "create_at": date("2013-04-08"), "description": "c/o '18, whs", "followers_count": 418, "friends_count": 294, "statues_count": 1997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718387372032, "text": "@jpickett5829 @LaloDagach ABUSED? Is that what they call touching you with rocks till death?", "in_reply_to_status": 698631609885552641, "in_reply_to_user": 3418935012, "favorite_count": 0, "coordinate": point("-98.4026167,30.7199108"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3418935012, 384625890 }}, "user": { "id": 384903586, "name": "Mark Robbins", "screen_name": "SenatorMark4", "lang": "en", "location": "Kingsland, TX", "create_at": date("2011-10-04"), "description": "Process-oriented seeker of solutions to government waste and abuse.. \nPublished solutions!", "followers_count": 107, "friends_count": 370, "statues_count": 1546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48299, "countyName": "Llano" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718659973120, "text": "Wind 0.0 mph N. Barometer 30.41 in, Rising. Temperature -12.1 °F. Rain today 0.00 in. Humidity 41%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 108, "statues_count": 157482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718752256001, "text": "@YoAfreshRaps https://t.co/PEHElXJy7v", "in_reply_to_status": -1, "in_reply_to_user": 353321537, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 353321537 }}, "user": { "id": 792937219, "name": "pablo", "screen_name": "JustinSoliz16", "lang": "en", "location": "null", "create_at": date("2012-08-30"), "description": "sc: steez_soliz16", "followers_count": 404, "friends_count": 330, "statues_count": 14991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718836109312, "text": "https://t.co/fCcUvYKTlV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 24995220, "name": "Adriana Pinnisi", "screen_name": "Aghe", "lang": "en", "location": "null", "create_at": date("2009-03-17"), "description": "Wisdom is knowing I am nothing, Love is knowing I am everything, and between the two my life moves. — Nisargadatta Maharaj", "followers_count": 60, "friends_count": 129, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718928375809, "text": "Any beach trips planned for tomorrow? @_TiannaG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.13940225,36.00986382"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380097285 }}, "user": { "id": 553208789, "name": "HugoG", "screen_name": "HugoGui12", "lang": "en", "location": "Avenal, California", "create_at": date("2012-04-13"), "description": "null", "followers_count": 43, "friends_count": 128, "statues_count": 2791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avenal, CA", "id": "4ef9eb0b1d3f720c", "name": "Avenal", "place_type": "city", "bounding_box": rectangle("-120.146913,35.964275 -120.110888,36.016284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 603302, "cityName": "Avenal" } }
+{ "create_at": datetime("2016-02-14T00:00:03.000Z"), "id": 698778718928506880, "text": "10 Guy is possibly my favorite meme.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90806312, "name": "Josh Robinson", "screen_name": "averagejosh", "lang": "en", "location": "asheville, north carolina", "create_at": date("2009-11-17"), "description": "hobbit - hufflepuff - pizza enthusiast - nintendoer - fresca breather", "followers_count": 299, "friends_count": 716, "statues_count": 17530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Asheville, NC", "id": "00836cb90515ddd1", "name": "Asheville", "place_type": "city", "bounding_box": rectangle("-82.681852,35.421216 -82.452163,35.656929") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37021, "countyName": "Buncombe", "cityID": 3702140, "cityName": "Asheville" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778719825989632, "text": "u weren't there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130244134, "name": "becka", "screen_name": "beckabaabyy", "lang": "en", "location": "Bay Area", "create_at": date("2010-04-06"), "description": "prolly at bella noche'", "followers_count": 640, "friends_count": 392, "statues_count": 10408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720190857216, "text": "Wind 10.0 mph SSE. Barometer 1022.49 mb, Falling Rapidly. Temperature 36.9 °F. Rain today 0.00 in. Humidity 45%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 28, "friends_count": 124, "statues_count": 12515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720300113920, "text": "Be like Shai. Shai is smart. https://t.co/PJiRP4lPZ2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 54387996, "name": "DIMEPENNY", "screen_name": "DiamondLRosario", "lang": "en", "location": "Live from the Gutter, NY", "create_at": date("2009-07-06"), "description": "Aspiring actress/model. I bartend. For business inquiries email me at diamondrosario56@gmail.com IG: diamondrosario", "followers_count": 3533, "friends_count": 1008, "statues_count": 143765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720455229440, "text": "Goodnight y'all progress", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 210293174, "name": "Gerald Johanssen", "screen_name": "TightToDef", "lang": "en", "location": "Firefly class Serenity", "create_at": date("2010-10-30"), "description": "I'm just a over Hyped Hipster.", "followers_count": 156, "friends_count": 191, "statues_count": 12677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverdale, GA", "id": "6443187b9b4b748b", "name": "Riverdale", "place_type": "city", "bounding_box": rectangle("-84.434425,33.520395 -84.385157,33.589854") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1365464, "cityName": "Riverdale" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720589344769, "text": "Dead pool was cool af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2913045620, "name": "Humberto Ramos", "screen_name": "betoloco18", "lang": "en", "location": "null", "create_at": date("2014-11-28"), "description": "The two most important days in your life are the day you are born and the day you find out why.", "followers_count": 517, "friends_count": 524, "statues_count": 7072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720744710144, "text": "@riz480 Dear Rizwan,How are you ,your children and your wife what are the chances of qalander reaching the final of psl\n\nTake u", "in_reply_to_status": -1, "in_reply_to_user": 593589755, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 593589755 }}, "user": { "id": 2754006627, "name": "S.Parvez. Shafi", "screen_name": "parshf2013", "lang": "en", "location": "New York, USA", "create_at": date("2014-08-28"), "description": "Alumni Kaplan University, Lawrence College.", "followers_count": 429, "friends_count": 2001, "statues_count": 975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720786485248, "text": "@SkyonAir_ SUPER NICE", "in_reply_to_status": 698778601253171200, "in_reply_to_user": 106489667, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106489667 }}, "user": { "id": 210653028, "name": "#OldKentuckyFrankie", "screen_name": "Klims37", "lang": "en", "location": "Some rink somewhere, TX", "create_at": date("2010-10-31"), "description": "Writer, atrocious defenseman, occasionally charming. @BU_Tweets grad, editor at @PBR, occasionally @cupofchowdah. Assume nobody endorses anything I tweet. Ever.", "followers_count": 363, "friends_count": 2008, "statues_count": 27050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720870338560, "text": "��⚽️ https://t.co/h9ut81LUw4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1648560182, "name": "Rachel Huber", "screen_name": "HuberRach", "lang": "en", "location": "Sacramento, CA", "create_at": date("2013-08-05"), "description": "accept what is, let go of what was, and have faith in what will be // SFHS♡", "followers_count": 232, "friends_count": 358, "statues_count": 3473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778720966934528, "text": "Travelling to Cleveland or just twittering about Cleveland? https://t.co/sYanBV3X4a #Cleveland", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6954,41.4995"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cleveland" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2593041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778721021464576, "text": "He kept telling me all about how he has money to spend but money ain't shit. That don't impress me much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544529635, "name": "Black Olive", "screen_name": "AlexFannyHat", "lang": "en", "location": "Goshen, Ohio", "create_at": date("2012-04-03"), "description": "Math Enthusiast/Badass M.C.\n\nI put the motion in Goshen", "followers_count": 172, "friends_count": 280, "statues_count": 5284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778721940066304, "text": "https://t.co/UHFPsnh2SJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4770988932, "name": "GayGirl", "screen_name": "AndrewsTajah", "lang": "en", "location": "null", "create_at": date("2016-01-16"), "description": "Not the type to be fucking around with these fake ass niggas && bitches i am all about makinn money&& keepinn my bitchies happy.☺️", "followers_count": 47, "friends_count": 174, "statues_count": 92 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778722002821121, "text": "Ultra Light Beam ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1398686767, "name": "Puff Daddy", "screen_name": "Mrd2b", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-05-02"), "description": "Humble servant.", "followers_count": 404, "friends_count": 352, "statues_count": 17079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778722300796929, "text": "Happy v day;)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1730709505, "name": "meghanッ", "screen_name": "MeghanEaslannd", "lang": "en", "location": "socal", "create_at": date("2013-09-04"), "description": "phs var swim • a.q♡", "followers_count": 621, "friends_count": 574, "statues_count": 4513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-14T00:00:04.000Z"), "id": 698778722497728513, "text": "YOUNG & HUNGRY IS THE BEST SHOW EVER", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 923922626, "name": "Catherine Fultz", "screen_name": "CatherineFultz", "lang": "en", "location": "Knoxville, TN", "create_at": date("2012-11-03"), "description": "Snap- catfultz01", "followers_count": 471, "friends_count": 689, "statues_count": 2753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141090457194497, "text": "Tonight's been interesting.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4363901412, "name": "spencer", "screen_name": "spencervasquezz", "lang": "en", "location": "null", "create_at": date("2015-11-25"), "description": "Kanye in the streets. Chris Brown in the sheets.", "followers_count": 96, "friends_count": 83, "statues_count": 67 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141090620915712, "text": "luk lik #jessicabowling who i met at psych ward. yes this girl was n my hbes clas sketchers? https://t.co/VgKjXhCHiU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jessicabowling" }}, "user": { "id": 3903951312, "name": "Jayy Sorath LaVeyGOD", "screen_name": "Jayy_LaVey", "lang": "en", "location": "Franklin, TN", "create_at": date("2015-10-15"), "description": "#illuminati. IM FUNNY. IM SARCASTIC. #starvingartist King, Of Tennessee. Married. 25. BLACKBILLY. I'm Silly. SloppyTweeter. Kid Komedian: IN PROGRESS. #BOSSING.", "followers_count": 268, "friends_count": 134, "statues_count": 58654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, TN", "id": "cc631a80adacd459", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.935419,35.85036 -86.766934,36.019674") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47187, "countyName": "Williamson", "cityID": 4727740, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141090792861696, "text": "@DrakeG1 I love you more!��", "in_reply_to_status": 699015920434356226, "in_reply_to_user": 276808494, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 276808494 }}, "user": { "id": 1067125436, "name": "Morgan Knezek", "screen_name": "morgancarole_", "lang": "en", "location": "null", "create_at": date("2013-01-06"), "description": "Texas A&M // insta:@morgancarole", "followers_count": 1317, "friends_count": 350, "statues_count": 13609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141091744997376, "text": "I love my @Fleksy Keyboard! I'd like to see even more #FleksyThemes!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FleksyThemes" }}, "user_mentions": {{ 564909345 }}, "user": { "id": 1936339627, "name": "Danny Alvarez", "screen_name": "Japanezedann", "lang": "en", "location": "Miami, FL", "create_at": date("2013-10-04"), "description": "Weird, funny, kinky.", "followers_count": 38, "friends_count": 109, "statues_count": 467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141091824537600, "text": "@demon420kush I'm excited for it now and yes !!of course", "in_reply_to_status": 699140656237662208, "in_reply_to_user": 748326602, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 748326602 }}, "user": { "id": 325219975, "name": "Helen", "screen_name": "HelenCldrn", "lang": "en", "location": "null", "create_at": date("2011-06-27"), "description": "ig: _.helennnn", "followers_count": 755, "friends_count": 435, "statues_count": 34965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141092734676992, "text": "@zoersolano oh ��", "in_reply_to_status": 699140918914383872, "in_reply_to_user": 612944928, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 612944928 }}, "user": { "id": 560582560, "name": "Pablo", "screen_name": "CaptainClutch35", "lang": "en", "location": "Amongst La Familia", "create_at": date("2012-04-22"), "description": "My whole teaching is this; \nAccept Yourself\nLove Yourself\nCelebrate Yourself.", "followers_count": 399, "friends_count": 73, "statues_count": 22429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141093519183872, "text": "Wait.... Is episode 9 on Greys the half season premiere???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2887576486, "name": "Erin Ergun", "screen_name": "erinergun", "lang": "en", "location": "THS '17", "create_at": date("2014-11-21"), "description": "I'll only do APUSH if it involves Hamilton", "followers_count": 197, "friends_count": 201, "statues_count": 2315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leesburg, VA", "id": "594ec79d61a7f536", "name": "Leesburg", "place_type": "city", "bounding_box": rectangle("-77.602414,39.067605 -77.503682,39.136067") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5144984, "cityName": "Leesburg" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141094085304320, "text": "@_hennessyisaac go best friend", "in_reply_to_status": 699140593444753408, "in_reply_to_user": 2823009751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2823009751 }}, "user": { "id": 938021106, "name": "maris∀", "screen_name": "mariiisaa_", "lang": "en", "location": "Oceanside, CA", "create_at": date("2012-11-09"), "description": "null", "followers_count": 798, "friends_count": 519, "statues_count": 10035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-15T00:00:00.000Z"), "id": 699141094135582720, "text": "This the longest I stayed up Inna brick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1589391829, "name": "Lil Jank", "screen_name": "HoeAssAnt", "lang": "en", "location": "somewhere chasing the money", "create_at": date("2013-07-12"), "description": "ALL OF MY NIGGAS IS JANKING & SWERVING, JANK SWERVE we ah dream team", "followers_count": 1097, "friends_count": 472, "statues_count": 43381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxon Hill, MD", "id": "003b0a6b6b3eca0e", "name": "Oxon Hill", "place_type": "city", "bounding_box": rectangle("-77.001308,38.760754 -76.93722,38.83211") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2459500, "cityName": "Oxon Hill" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141094915768320, "text": "Temp: 20.1F W C: 20.1F Wind:--- at 0.0kts Baro: 1029.2mb and Falling Rain today: 0.00in R H: 78% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 193, "friends_count": 219, "statues_count": 97127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141095385509888, "text": "aaaaaaaaaaaaand I'm officially another year older", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55694802, "name": "not monica", "screen_name": "l__vm", "lang": "en", "location": "null", "create_at": date("2009-07-10"), "description": "merde", "followers_count": 174, "friends_count": 151, "statues_count": 10298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141095410667522, "text": "It was a happy Valentine's Day ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2811915517, "name": "Adriana Jacho", "screen_name": "JachoAdriana", "lang": "en", "location": "null", "create_at": date("2014-09-15"), "description": "Forget it", "followers_count": 64, "friends_count": 42, "statues_count": 667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141095632957440, "text": "@ebbtideapp Tide in Middletown, Connecticut 02/15/2016\nHigh 6:31am 2.4\n Low 2:06pm -0.2\nHigh 6:57pm 1.8\n Low 2:16am -0.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-72.645,41.56"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 29, "friends_count": 1, "statues_count": 7885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, CT", "id": "8c814b789b5f7574", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-72.752943,41.494706 -72.58268,41.604104") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 947290, "cityName": "Middletown" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096031408129, "text": "���� - Lol, I Swear I Fwu... See That's How You Know I Do.. I Started The Sentence Off With \"Lol\" ������ You Funny Denna Hoe &' I Fw Ya Tweets��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367425591, "name": "Petty Proud", "screen_name": "xoxodes__", "lang": "en", "location": "└A ✈️ MesqTx.", "create_at": date("2011-09-03"), "description": "RIP Papa. Rest Easy Carl D. Wilson &' Shania Gray #CjsWorld @ToriKelly Is My Wife❤️ #SoonersNation #Suwoo #ColeWord Gaaaaaay.. DMs Close @11 PM", "followers_count": 5836, "friends_count": 4868, "statues_count": 217571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096199294976, "text": "@NYCKING I do have @Sony @PlayStation 4 also but I did not go on it cause I did not have time , also to much freaking family to deal with", "in_reply_to_status": 699109286329323521, "in_reply_to_user": 19257498, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19257498, 34442404, 10671602 }}, "user": { "id": 76398387, "name": "Dr. Evil", "screen_name": "BXB_Boy", "lang": "en", "location": "The Big Apple", "create_at": date("2009-09-22"), "description": "I'm a Wrestling Fan, I Love Pro Wrestling.", "followers_count": 1953, "friends_count": 4489, "statues_count": 94874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096211771392, "text": "But he's addressed racism before even if it's just mentioning immigrants. https://t.co/gHH2l8na6E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2320276789, "name": "A", "screen_name": "AnaiyaMoore", "lang": "en", "location": "null", "create_at": date("2014-01-30"), "description": "null", "followers_count": 1155, "friends_count": 395, "statues_count": 25387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096224370688, "text": "Wind 0.0 mph ---. Barometer 29.924 in, Falling slowly. Temperature 19.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 14, "statues_count": 1179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096232714240, "text": "why do I even care.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1882164638, "name": "myrissa diamond✈️", "screen_name": "rissydiamond", "lang": "en", "location": "Grand Junction, CO", "create_at": date("2013-09-18"), "description": "California born. Colorado raised.❤️ wanderlust.✨ IG: rissadiamond", "followers_count": 261, "friends_count": 216, "statues_count": 3098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Junction, CO", "id": "3b2155c98f7ba6a7", "name": "Grand Junction", "place_type": "city", "bounding_box": rectangle("-108.662336,39.033602 -108.486897,39.149899") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8077, "countyName": "Mesa", "cityID": 831660, "cityName": "Grand Junction" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096379703296, "text": "Wind 0.0 mph SE. Barometer 29.981 in, Falling. Temperature 28.7 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096610230272, "text": "Same only I just spent $80 on makeup again �� that's 2x in a day https://t.co/mCTtlwUJkP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2881770060, "name": "BritnEy", "screen_name": "_BriXter", "lang": "en", "location": "null", "create_at": date("2014-10-29"), "description": "I'm an extrovert stuck friendless so let me be depressed k", "followers_count": 173, "friends_count": 467, "statues_count": 8430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141096819929088, "text": "ITS RICHARDS BIRTHDAY !!! \n\nHES 21 !!!! ����������❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 752364289, "name": "caitlyn stanescu", "screen_name": "caitlynstanescu", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "iv.xvii.mmxiii♡ * · ˚ ˚✺ ˚ ˚ · * ˚", "followers_count": 475, "friends_count": 454, "statues_count": 24232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141097201758209, "text": "Wind 0 mph --. Barometer 30.04 in, Falling slowly. Temperature 51.6 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141097642000384, "text": "teaching my 19 year old cousin how to slide in the DMs hbu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1671514968, "name": "rose s", "screen_name": "mermaidro_", "lang": "en", "location": "null", "create_at": date("2013-08-14"), "description": "child of the tides | sc: roseschirmer | avhs", "followers_count": 708, "friends_count": 431, "statues_count": 24749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141097747042304, "text": "Ripley SW Limestone Co. Temp: 49.5°F Wind:13.0mph Pressure: 994.0mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 45, "friends_count": 33, "statues_count": 48830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141097923063808, "text": "90% of the time when people ask me what I'm doing I'm eating", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415061704, "name": "Melisa", "screen_name": "MelisaaMontess", "lang": "en", "location": "somewhere eating pad thai ", "create_at": date("2011-11-17"), "description": "null", "followers_count": 195, "friends_count": 128, "statues_count": 7589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnwood, WA", "id": "3912e7cd4cf62c39", "name": "Lynnwood", "place_type": "city", "bounding_box": rectangle("-122.337717,47.802196 -122.262542,47.853772") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5340840, "cityName": "Lynnwood" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141097965035520, "text": "2016/02/21: Manhattan, New York, NY, United States: Get Your Groove on Sunday https://t.co/a3cxocpcQE https://t.co/CNbr9hx8iW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0058127,40.7408348"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325786388, "name": "Step x Step", "screen_name": "stepxstepdance", "lang": "en", "location": "The World Is Our Dancefloor", "create_at": date("2011-06-28"), "description": "The Official Step x Step Twitter. Street Dance Media Uniting Dancers Around The World Step By Step", "followers_count": 7468, "friends_count": 3082, "statues_count": 30177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141098271191040, "text": "Kanye https://t.co/AZkL5JpYUb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 392383526, "name": "J u l i a n", "screen_name": "hooleahhn", "lang": "en", "location": "San Antonio, TX", "create_at": date("2011-10-16"), "description": "it's all love even when they hate you my nigga", "followers_count": 743, "friends_count": 562, "statues_count": 71789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141098329939968, "text": "@Alyssadelphia7 @Eagles I like you ����������", "in_reply_to_status": 693122756183875584, "in_reply_to_user": 2376148195, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2376148195, 180503626 }}, "user": { "id": 40818608, "name": "Alexander Roman", "screen_name": "Megaeaglesfan25", "lang": "en", "location": "Perris,Ca U.S.A.", "create_at": date("2009-05-17"), "description": "Im a cool guy like to hangout do whatever!! Im a die hard philadelphia eagles fan for more info just hit me up", "followers_count": 358, "friends_count": 968, "statues_count": 2824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-02-15T00:00:01.000Z"), "id": 699141098564747264, "text": "@StreetMSE I'll be in the back ground lol", "in_reply_to_status": 699127750863384576, "in_reply_to_user": 77932953, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77932953 }}, "user": { "id": 1186729320, "name": "courtney", "screen_name": "courtneyyvu", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "DDD & HTX // @GunMusic40 #GeekedUpMusic", "followers_count": 1007, "friends_count": 78, "statues_count": 10957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141098971660288, "text": "In about 24 hours I will be a 18 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 871227138, "name": "Omar Munoz", "screen_name": "zClutches", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "98'♒", "followers_count": 188, "friends_count": 153, "statues_count": 2846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141099189858304, "text": "02/15@03:00 - Temp 18.9F, WC 18.9F. Wind 2.0mph ESE, Gust 5.0mph. Bar 30.552in, Falling slowly. Rain 0.00in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141099311538176, "text": "Temp: 55.8°F Wind:0.7mph Pressure: 30.023hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 59980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141099433033728, "text": "And take your smelly beans with you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30829785, "name": "Lisette", "screen_name": "lisettellamas", "lang": "en", "location": "null", "create_at": date("2009-04-13"), "description": "null", "followers_count": 61, "friends_count": 112, "statues_count": 528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141099470737408, "text": "60.3F (Feels: 60.3F) - Humidity: 97% - Wind: 4.5mph SE - Gust: 8.3mph - Pressure: 1025.0mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 226880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141100062138368, "text": "how can people be so fucked up!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4665298454, "name": "Leti", "screen_name": "letii7x", "lang": "en", "location": "Willits, CA", "create_at": date("2015-12-27"), "description": "Sc// letti7x ✨", "followers_count": 56, "friends_count": 43, "statues_count": 142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willits, CA", "id": "7f871a056f466c29", "name": "Willits", "place_type": "city", "bounding_box": rectangle("-123.364007,39.376929 -123.335284,39.424874") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6045, "countyName": "Mendocino", "cityID": 685600, "cityName": "Willits" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141100112580608, "text": "Wind 0.0 mph SSW. Barometer 30.071 in, Falling. Temperature 27.0 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 52, "friends_count": 27, "statues_count": 17655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141100510924800, "text": "Happy Birthday Bestfriend!!! @Cassidyfuller02 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3288482036 }}, "user": { "id": 2846017410, "name": "Spencer Novella", "screen_name": "spencernovella", "lang": "en", "location": "Clovis, CA", "create_at": date("2014-10-07"), "description": "Buchanan High School", "followers_count": 92, "friends_count": 153, "statues_count": 979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141100792053760, "text": "I fucking love being dj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391774607, "name": "Lilah", "screen_name": "Delilah_Fonseca", "lang": "en", "location": "null", "create_at": date("2011-10-15"), "description": "Goonies never say die", "followers_count": 188, "friends_count": 88, "statues_count": 12162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, AL", "id": "4caafbe771809878", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-85.896673,31.283873 -85.777968,31.386312") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1031, "countyName": "Coffee", "cityID": 124184, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141101278482433, "text": "Tryna fix your inner issues with a bad bitch didn't they tell you that I was a savage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2744709807, "name": "BRAT", "screen_name": "vaneasiaaa_", "lang": "en", "location": "sandy toes, salty kisses ", "create_at": date("2014-08-16"), "description": "♓️#BBC ☯ she's a Pisces, heard they feisty", "followers_count": 668, "friends_count": 529, "statues_count": 8869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141101685444608, "text": "@rachieboooxx @TylerHoye1 agreed", "in_reply_to_status": 699108359195103238, "in_reply_to_user": 456280756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 456280756, 1231970792 }}, "user": { "id": 452752681, "name": "ThunderHawk", "screen_name": "theradredskin", "lang": "en", "location": "Clio, Michigan", "create_at": date("2012-01-01"), "description": "| RIP-Gordan James Wilson | Lucky 13 | Geaux Tigers | Reign Supreme | Damingo crew | @kaylastafford8 |", "followers_count": 706, "friends_count": 673, "statues_count": 16277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clio, MI", "id": "5a633dd1c7a551d5", "name": "Clio", "place_type": "city", "bounding_box": rectangle("-83.793998,43.13412 -83.690834,43.20628") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2616620, "cityName": "Clio" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141101731491843, "text": "Yaay it's not valentines anymore !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951398616, "name": "marilynn ♚❥", "screen_name": "hyphymarilyn", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-12-29"), "description": "K & M ❤ ️random white girl be everywhere ,", "followers_count": 344, "friends_count": 406, "statues_count": 3446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102151061504, "text": "Climbing the way into my heart ❤️ Happy Valentine's Day! @ Indian Rock Park https://t.co/qyQrOl4EgD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.27296642,37.89221414"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19318507, "name": "Shannon", "screen_name": "ShannieKM97", "lang": "en", "location": "California", "create_at": date("2009-01-21"), "description": "I'm not plain, maybe a little insane, and every random once in a while I'm vain (Hehe I like to rhyme)", "followers_count": 217, "friends_count": 230, "statues_count": 4033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102331256834, "text": "I'm so funny !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545627642, "name": "Estéban", "screen_name": "_ThaNiggaSteveO", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-04-04"), "description": "Views From Da 310.", "followers_count": 363, "friends_count": 184, "statues_count": 64884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, NV", "id": "780158265aec966b", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-119.623172,39.200823 -119.50692,39.317313") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32019, "countyName": "Lyon", "cityID": 3217500, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102444507137, "text": "I've been up for 24 hours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 790886306, "name": "NoHeartFeelings", "screen_name": "petitehaze", "lang": "en", "location": "null", "create_at": date("2012-08-29"), "description": "Don't follow me I don't know where I'm going", "followers_count": 1369, "friends_count": 1081, "statues_count": 21162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago Ridge, IL", "id": "4f798ddf05f97f68", "name": "Chicago Ridge", "place_type": "city", "bounding_box": rectangle("-87.798589,41.690256 -87.759138,41.720544") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714065, "cityName": "Chicago Ridge" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102692077568, "text": "Wind 0.0 mph ---. Barometer 30.608 in, Rising. Temperature -15.1 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102733885440, "text": "Wind 1.0 mph E. Barometer 30.540 in, Falling. Temperature -2.5 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 6528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-15T00:00:02.000Z"), "id": 699141102771634177, "text": "Happy birthday baby I love you so much I hope you a have an amazing &' we needa link up soon boo cause I miss you ��❤️�� @maddiegirlz2000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1873898676 }}, "user": { "id": 2547193254, "name": "flossy", "screen_name": "gangangnancy", "lang": "en", "location": "Lancaster, CA", "create_at": date("2014-06-04"), "description": "dms are closed.", "followers_count": 414, "friends_count": 242, "statues_count": 14444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141103174291456, "text": "I do the most for ppl, bc that bitch shit don't run in my life!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3158476148, "name": "KGATE$$$$", "screen_name": "KarenGatess", "lang": "en", "location": "Cameron, TX$", "create_at": date("2015-04-15"), "description": "Cameron Tx. BWA. hater wish you well -KG", "followers_count": 135, "friends_count": 126, "statues_count": 934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron, TX", "id": "922185539a1a7a3b", "name": "Cameron", "place_type": "city", "bounding_box": rectangle("-97.003013,30.841928 -96.958853,30.882807") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48331, "countyName": "Milam", "cityID": 4812040, "cityName": "Cameron" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141103593717761, "text": "Im fucking happy for my friends!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2891363009, "name": "Epitacio", "screen_name": "LG_TACO", "lang": "pt", "location": "Recife", "create_at": date("2014-11-24"), "description": "Professional CS:GO player for @Luminosity", "followers_count": 22110, "friends_count": 239, "statues_count": 3573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141103644254208, "text": "omfg https://t.co/WHXo2eIZBJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493176337, "name": "elliesnore", "screen_name": "i_suck123", "lang": "en", "location": "new york", "create_at": date("2012-02-15"), "description": "psychology dance n maddie", "followers_count": 465, "friends_count": 223, "statues_count": 6194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brighton, NY", "id": "94779fa8bb1f79a8", "name": "Brighton", "place_type": "city", "bounding_box": rectangle("-77.694219,43.087472 -77.516194,43.167065") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608257, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141104008957953, "text": "Wind 2.0 mph S. Barometer 1017.51 mb, Falling. Temperature 35.6 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 29, "friends_count": 124, "statues_count": 12539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141104692699137, "text": "Help everyone is drunk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359883403, "name": "krispy preme", "screen_name": "_frncsco", "lang": "en", "location": "NWCTA 16'", "create_at": date("2011-08-22"), "description": "greek god", "followers_count": 220, "friends_count": 167, "statues_count": 10767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141104797556736, "text": "@CarliAnn21 Yes, it was an honor to rave with me, you are welcome haha", "in_reply_to_status": 699140506756907009, "in_reply_to_user": 956190084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 956190084 }}, "user": { "id": 2804652775, "name": "Gubby Montenegro", "screen_name": "GubbyM", "lang": "en", "location": "Las Vegas, NV 702 ", "create_at": date("2014-09-11"), "description": "Dream BIG Lift HEAVY lll----lll.", "followers_count": 341, "friends_count": 213, "statues_count": 1614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141104814452737, "text": "This woman totally annoyed me today, to the point her man leaned in to say sorry & gave me a large tip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289491399, "name": "TheArtOfShinuh✈️", "screen_name": "ShinuhToya", "lang": "en", "location": "MUSIC. FASHION. ART", "create_at": date("2011-04-28"), "description": "Intimate w/ Fashion, Music, & Art! We Make Love 2gether! ❤ http://Www.theartofshinuh.comtheartofshinuh@gmail.com", "followers_count": 454, "friends_count": 396, "statues_count": 7152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141104910929920, "text": "Wind 0.0 mph N. Barometer 30.55 in, Steady. Temperature -4.9 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 108, "statues_count": 157506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105187627009, "text": "@XOXO_ARIANNA10 you stop forever and I stop forever! ����", "in_reply_to_status": 699140363085197313, "in_reply_to_user": 248548147, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248548147 }}, "user": { "id": 1688604854, "name": "Brandon Reid", "screen_name": "__Reidmytweets", "lang": "en", "location": "sc: brandonreid9", "create_at": date("2013-08-21"), "description": "United States Air Force • Vegas", "followers_count": 446, "friends_count": 328, "statues_count": 4567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105305067521, "text": "@niallthejedi does his face light up like this when you ask him about India jones :( so cute", "in_reply_to_status": 699139845361303552, "in_reply_to_user": 1173616220, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1173616220 }}, "user": { "id": 3145069518, "name": "sara", "screen_name": "lourrie28", "lang": "en", "location": "louis' right armpit ", "create_at": date("2015-04-08"), "description": "she/her + queer af", "followers_count": 1844, "friends_count": 121, "statues_count": 28428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105430843392, "text": "at 10:46pm:\ntemperature down 70°F -> 62°F\nhumidity up 42% -> 67%\nvisibility 10mi -> 9mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.24532,34.05349"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81471317, "name": "Los Angeles Weather", "screen_name": "_LosAngelesCA", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-10-10"), "description": "Weather updates, forecast, warnings and information for Los Angeles, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 1123, "friends_count": 3, "statues_count": 30094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105514708993, "text": "@FollowThisDee @destinydestani that's what I'm saying", "in_reply_to_status": 699140972500746244, "in_reply_to_user": 463019741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 463019741, 1082488459 }}, "user": { "id": 3853293072, "name": "Tay-Doe ⚡️", "screen_name": "TadoeTv_", "lang": "en", "location": "Waco ✈️ Denton", "create_at": date("2015-10-10"), "description": "Have faith in many but put your trust in NOBODY ... #UNT #GoGettasENT #PlayRunners", "followers_count": 535, "friends_count": 451, "statues_count": 3792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105623781376, "text": "tg valentines day is over", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430057679, "name": "kirsten clemans", "screen_name": "kirstenclemans6", "lang": "en", "location": "Wenatchee | Pullman", "create_at": date("2011-12-06"), "description": "WSU ૐ ΚΔ", "followers_count": 300, "friends_count": 212, "statues_count": 1101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105644777472, "text": "Is ok to no like somthing but importantly is too moved on and not think about about because all is gonna do is slow you down on life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554504294, "name": "Chris Hoang", "screen_name": "ChrisDareal45", "lang": "en", "location": "falls church", "create_at": date("2012-04-15"), "description": "Im the best in the world cuz of what I do!!! Die hard Denver Broncos fan", "followers_count": 333, "friends_count": 414, "statues_count": 22295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bailey's Crossroads, VA", "id": "c933b99b367f0b5b", "name": "Bailey's Crossroads", "place_type": "city", "bounding_box": rectangle("-77.147585,38.832027 -77.110316,38.861746") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5104088, "cityName": "Bailey's Crossroads" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141105921581056, "text": "@vaaaleee_4 https://t.co/vrzmzN5gZ0", "in_reply_to_status": -1, "in_reply_to_user": 592055671, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 592055671 }}, "user": { "id": 2655908610, "name": "n", "screen_name": "Nayelaay_", "lang": "en", "location": "atx", "create_at": date("2014-07-17"), "description": "chillin' .", "followers_count": 603, "friends_count": 250, "statues_count": 26093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-15T00:00:03.000Z"), "id": 699141106567671809, "text": "Wind 2 mph ENE. Barometer 30.21 in, Rising. Temperature 51.4 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 55, "friends_count": 72, "statues_count": 25728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141107221803008, "text": "@Josh_Cantos actually I'm thick����������", "in_reply_to_status": 699140982906859520, "in_reply_to_user": 1636393933, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1636393933 }}, "user": { "id": 1636393933, "name": "JISH", "screen_name": "Josh_Cantos", "lang": "en", "location": "Pre-Valentines Day around 11", "create_at": date("2013-07-31"), "description": "#relentless #weareone #ASB #TLOP #GettingOutOurDreams #varsitypolo #ascension", "followers_count": 391, "friends_count": 359, "statues_count": 10703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141107460878336, "text": "@tay_culp you should eat them", "in_reply_to_status": 699137253755359232, "in_reply_to_user": 3105681157, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3105681157 }}, "user": { "id": 2701723933, "name": "A N D R E W", "screen_name": "DrewGalvan1", "lang": "en", "location": "California, USA", "create_at": date("2014-08-02"), "description": "TCHS", "followers_count": 215, "friends_count": 201, "statues_count": 1394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141107666415616, "text": "*need https://t.co/RKHkPMURhr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2840880426, "name": "krystal nicole", "screen_name": "Krysxnicole", "lang": "en", "location": "null", "create_at": date("2014-10-04"), "description": "ig// kryscerda", "followers_count": 407, "friends_count": 304, "statues_count": 3407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141107716747264, "text": "Moms put me on with Dolce & Gabbana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1005452214, "name": "Mel", "screen_name": "lillissa3", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "Yay Area", "followers_count": 23, "friends_count": 21, "statues_count": 1313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141108626882560, "text": "This gives me so many feelings ���������������� #GameOfThrones #season6 #teaser time for a marathon https://t.co/FUd9Jja3Nr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GameOfThrones", "season6", "teaser" }}, "user": { "id": 501949349, "name": "FATHER OF DRAGONS", "screen_name": "Taurus90_Baby", "lang": "en", "location": "Oregon, USA", "create_at": date("2012-02-24"), "description": "null", "followers_count": 1929, "friends_count": 2705, "statues_count": 9038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141108979277825, "text": "I don't think I've ever been this bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2432668214, "name": "T", "screen_name": "tt_drizzy", "lang": "en", "location": "null", "create_at": date("2014-04-07"), "description": "views", "followers_count": 369, "friends_count": 230, "statues_count": 5933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141110065532929, "text": "Guess who's getting an intervention now? Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269431868, "name": "Morgasmic ✌️", "screen_name": "Nagromduorts", "lang": "en", "location": "null", "create_at": date("2011-03-20"), "description": "not here to be your friend.", "followers_count": 545, "friends_count": 238, "statues_count": 14480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141110342373377, "text": "@_KingMcCool ��hml", "in_reply_to_status": 699107876296593408, "in_reply_to_user": 3000457087, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3000457087 }}, "user": { "id": 3316453984, "name": "JuiceMan™", "screen_name": "MTrott3r", "lang": "en", "location": "null", "create_at": date("2015-06-09"), "description": "HF Football SB/CB #1️⃣4️⃣ http://www.hudl.com/athlete/5399757/highlights/304677449 Greatness Will Be Achieved", "followers_count": 339, "friends_count": 166, "statues_count": 1137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flossmoor, IL", "id": "97ae6559498bd23d", "name": "Flossmoor", "place_type": "city", "bounding_box": rectangle("-87.713559,41.527807 -87.655073,41.55185") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1726571, "cityName": "Flossmoor" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141111013486593, "text": "@raegandh well I ain't mad atcha then shawty! Get them buns caressed lol ��", "in_reply_to_status": 699140561819824128, "in_reply_to_user": 443091816, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 443091816 }}, "user": { "id": 1014379171, "name": "The Tweet Deacon", "screen_name": "Trill_Soliloquy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-12-15"), "description": "Twitter Deacon Trentt, Doing Twitter Deacon Shit. Eat My Twitter Deacon Dick, As I Twitter Deacon Spit.", "followers_count": 1783, "friends_count": 863, "statues_count": 220495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141111051227137, "text": "Wind 0.0 mph ENE. Barometer 30.07 in, Steady. Temperature 53.6 °F. Rain today 0.00 in. Humidity 18%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 21, "friends_count": 98, "statues_count": 14226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-02-15T00:00:04.000Z"), "id": 699141111273492480, "text": "If you're a #IT professional in #CranberryTownship, PA, check out this #job: https://t.co/e4DAHi9sWc #Hiring https://t.co/UHIm9xGXr0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.119951,40.708957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IT", "CranberryTownship", "job", "Hiring" }}, "user": { "id": 3431841232, "name": "Jobs at Continuum", "screen_name": "JobsAtContinuum", "lang": "en", "location": "null", "create_at": date("2015-08-19"), "description": "null", "followers_count": 92, "friends_count": 42, "statues_count": 143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranberry, PA", "id": "01b347e54132a78d", "name": "Cranberry", "place_type": "city", "bounding_box": rectangle("-80.160831,40.640453 -79.995749,40.776081") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141111952994304, "text": "Sad ad bc using my slow ass iPhone 5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3007462783, "name": "Richard", "screen_name": "heiwaikiru", "lang": "en", "location": "Houstone", "create_at": date("2015-02-01"), "description": "Ditto.", "followers_count": 129, "friends_count": 122, "statues_count": 7428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141112049594368, "text": "Can you find Berkeley on the map? Just try it at https://t.co/KXDbgLQMd9 #Berkeley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.273,37.8716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Berkeley" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2595120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141112066347008, "text": "@Danno_a_Jyd Watch'91 Doc #DONALD TRUMP Did NOT Want Anyone to See. #Self_Aggrandizing #Huckster https://t.co/Zkcu3adE3L", "in_reply_to_status": 699133084596199425, "in_reply_to_user": 36690283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DONALD", "Self_Aggrandizing", "Huckster" }}, "user_mentions": {{ 36690283 }}, "user": { "id": 3331280621, "name": "sara hall", "screen_name": "sarahal15586515", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "null", "followers_count": 187, "friends_count": 217, "statues_count": 12234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilson, NC", "id": "551f529f172a516a", "name": "Wilson", "place_type": "city", "bounding_box": rectangle("-77.990547,35.682149 -77.872166,35.795207") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37195, "countyName": "Wilson", "cityID": 3774540, "cityName": "Wilson" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141113169506304, "text": "Wind 0.0 mph ---. Barometer 30.556 in, Steady. Temperature 10.1F. Rain today 0.00 in. Humidity 79% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 596, "friends_count": 798, "statues_count": 37264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141113387622400, "text": "Travelling to Berkeley or just twittering about Berkeley? https://t.co/KXDbgLQMd9 #Berkeley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.273,37.8716"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Berkeley" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2595121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141113492275200, "text": "@sandralynnn15 you're fav ;) https://t.co/Lw4fGSZiO7", "in_reply_to_status": -1, "in_reply_to_user": 2385621860, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2385621860 }}, "user": { "id": 3237251299, "name": "SantozCanales☁️", "screen_name": "CanalesSantoz", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "•sc: santos22isel", "followers_count": 178, "friends_count": 288, "statues_count": 904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hebbronville, TX", "id": "6c80b8c28fd29be3", "name": "Hebbronville", "place_type": "city", "bounding_box": rectangle("-98.694359,27.292917 -98.663161,27.323508") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48247, "countyName": "Jim Hogg", "cityID": 4833008, "cityName": "Hebbronville" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141113563590656, "text": "I need to see Prydz... holy lasers. https://t.co/jhxrl1Xx3i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 196029454, "name": "Eric Fletcher", "screen_name": "ericjfletcher", "lang": "en", "location": "Orange, CA, USA", "create_at": date("2010-09-27"), "description": "23 NAU Sigma Chi From Orange County, CA", "followers_count": 383, "friends_count": 929, "statues_count": 9208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141114188554240, "text": "And I have to be up in like 4 hours. Awesome.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 751011264, "name": "abbs⭐️", "screen_name": "AbbyLaniniXOXO", "lang": "en", "location": "in rehearsal ", "create_at": date("2012-08-11"), "description": "NYC bound??? sing/dance/act. ✞ = ♡. Tiger Mystique Show Choir❤️ Broken Arrow, OK. instagram: abbylaninixoxo", "followers_count": 1372, "friends_count": 2053, "statues_count": 26469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141114406645760, "text": "Guys aren't supposed to have Valentines we're supposed to hunt for our prey and kill it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971123744, "name": "Really Thugged Out", "screen_name": "BehrGris", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2012-11-25"), "description": "I'll do anything for $2,000 #PeacefulNiggas #WhiteWomenOverEverything", "followers_count": 2266, "friends_count": 1517, "statues_count": 8218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141114704457728, "text": "Sexy in that white tee and Jeans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110854044, "name": "Jovanna", "screen_name": "itsjustjovi", "lang": "en", "location": "null", "create_at": date("2010-02-02"), "description": "mommy♡ livin' on an island IG: itsjustjovi #USArmy", "followers_count": 318, "friends_count": 271, "statues_count": 17101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schofield Barracks, HI", "id": "23d2e7d8f1470c99", "name": "Schofield Barracks", "place_type": "city", "bounding_box": rectangle("-158.090162,21.478503 -158.039537,21.505599") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1569050, "cityName": "Schofield Barracks" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141114775867392, "text": "Barberton Oh Temp:17.7°F Wind:0 mph Dir:SE Baro:Falling Rain2day:0.00in Hum:74% UV:0.0 @ 03:00 02/15/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-15T00:00:05.000Z"), "id": 699141115157479425, "text": "Jk he took me to in n out because that's how simple I am and what I love lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110825025, "name": "zara", "screen_name": "yeseniaaotz", "lang": "en", "location": "LOS ANGELES", "create_at": date("2010-02-02"), "description": "you say you got them drugs but I've never seen you slang.", "followers_count": 172, "friends_count": 417, "statues_count": 669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141115765624832, "text": "@tiffanysuckkaa happy birthday my love ������ hope you have a great day and don't just lay in bed bc you're now 18�������� https://t.co/ilFeoa0l3Z", "in_reply_to_status": -1, "in_reply_to_user": 569610378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 569610378 }}, "user": { "id": 501662807, "name": "Will Kardashian", "screen_name": "WilllKardashian", "lang": "en", "location": "California, USA ", "create_at": date("2012-02-24"), "description": "Atheists were praying full of sarcasm", "followers_count": 886, "friends_count": 404, "statues_count": 93467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monrovia, CA", "id": "174309ec90de94d0", "name": "Monrovia", "place_type": "city", "bounding_box": rectangle("-118.023461,34.099706 -117.975357,34.178637") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648648, "cityName": "Monrovia" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141116159864832, "text": "https://t.co/I5RzzOksjQ https://t.co/mIRt3rEsTJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4892728237, "name": "Deloris Quick", "screen_name": "DelorisVAQuick", "lang": "en", "location": "null", "create_at": date("2016-02-11"), "description": "null", "followers_count": 6, "friends_count": 177, "statues_count": 1643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141116227096577, "text": "I always struggle with the concept of intimate relations....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242676018, "name": "¥¥¥", "screen_name": "SenpaiLxrd", "lang": "en", "location": "Hollywood, FL", "create_at": date("2015-06-11"), "description": "Visual Artist || Broward Heart", "followers_count": 203, "friends_count": 175, "statues_count": 5089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Club, FL", "id": "36240d7e802b0d03", "name": "Country Club", "place_type": "city", "bounding_box": rectangle("-80.352067,25.923399 -80.293194,25.957164") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214895, "cityName": "Country Club" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141116390576128, "text": "#ValentinDay #husband #nyc #love #❤ #flowers @ Brooklyn Greenpoint NY https://t.co/iatado9RtQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95204925,40.72901873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ValentinDay", "husband", "nyc", "love", "flowers" }}, "user": { "id": 135998617, "name": "Lola Coccolo", "screen_name": "lulicoccolo", "lang": "es", "location": "New York City ", "create_at": date("2010-04-22"), "description": "null", "followers_count": 403, "friends_count": 839, "statues_count": 2243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141116566736897, "text": "Curtis made me mad yesterday stepping on his bed with shoes on I almost fought him ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119993898, "name": "Chuck Bass™", "screen_name": "infinitelyminee", "lang": "en", "location": "Chicago", "create_at": date("2010-03-04"), "description": "elephants never forget", "followers_count": 1083, "friends_count": 466, "statues_count": 97601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141116684206080, "text": "*still uses data knowing I have half a gb to last me another two weeks*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174147605, "name": "Vincent Muñoz", "screen_name": "GuineaVinny", "lang": "en", "location": "Covina, California", "create_at": date("2010-08-02"), "description": "I say I'm from the hood, but I live inside the valley. Instagram/Snapchat: Guineavinny", "followers_count": 177, "friends_count": 153, "statues_count": 13804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141117422366720, "text": "Second Valentine's Day with Angel with many more to come ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1928234719, "name": "angelique ☮", "screen_name": "chinkangelique", "lang": "en", "location": "Sunny CA.", "create_at": date("2013-10-02"), "description": "null", "followers_count": 614, "friends_count": 552, "statues_count": 13473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141117460131840, "text": "Strictly business from now on⚾️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2605207646, "name": "Kaelan A-J", "screen_name": "kaelan_7", "lang": "en", "location": "Oakland, Ca", "create_at": date("2014-07-05"), "description": "Blessed Student Athlete⚾️ DLM™ ℅ 2016 #RIPGMA", "followers_count": 553, "friends_count": 598, "statues_count": 4565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141117925679105, "text": "@diego_swanson happy birthday big head (:", "in_reply_to_status": 699139950952861697, "in_reply_to_user": 2361965324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2361965324 }}, "user": { "id": 123447421, "name": "Erica☁️", "screen_name": "Drakesbbymama", "lang": "en", "location": "null", "create_at": date("2010-03-15"), "description": "Everything Happens for a Reason Sweet Thing☁️", "followers_count": 291, "friends_count": 148, "statues_count": 24601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141118026391552, "text": "Temp: 16.4°F - Dew Point: 9.6° - Wind: 5.2 mph - Gust: 7.6 - Rain Today: 0.00in. - Pressure: 29.92in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141118441574400, "text": "He loves me sometimes ������ happy v day friends https://t.co/28Fckh27f5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520819619, "name": "Mad", "screen_name": "MadeleineLBurke", "lang": "en", "location": "puyallup ↔️ bellingham", "create_at": date("2012-03-10"), "description": "i'm obsessed with sharks // WWU // your favorite barista", "followers_count": 657, "friends_count": 445, "statues_count": 28146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-02-15T00:00:06.000Z"), "id": 699141119377088512, "text": "#SupportOriginMelissa 28.6°F Wind:2.2mph Pressure: 29.97hpa Falling quickly Rain Today 0.00in. Forecast: Occasional precipitation, worsenin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 309888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-15T00:00:07.000Z"), "id": 699141119876194305, "text": "I forgot how much fun Titanfall actually is.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2274014743, "name": "Tommy Barrelbum", "screen_name": "SalsaBruh", "lang": "en", "location": "null", "create_at": date("2014-01-02"), "description": "null", "followers_count": 52, "friends_count": 225, "statues_count": 2412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allentown, PA", "id": "cfcf6b96b61e7b11", "name": "Allentown", "place_type": "city", "bounding_box": rectangle("-75.548035,40.547616 -75.419836,40.636034") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4202000, "cityName": "Allentown" } }
+{ "create_at": datetime("2016-02-15T00:00:07.000Z"), "id": 699141119989280768, "text": "Can you recommend anyone for this #job? IT Applications - Salesforce com - https://t.co/AaaDX7Ms6Q #Lexington, KY #IT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4951359,38.0317136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Lexington", "IT", "Hiring" }}, "user": { "id": 26266228, "name": "Lexmark Careers", "screen_name": "lexmarkcareers", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-03-24"), "description": "Careers at Lexmark -- Open the possibilities", "followers_count": 1656, "friends_count": 1142, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21067, "countyName": "Fayette", "cityID": 2146027, "cityName": "Lexington-Fayette" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503478377312256, "text": "I have no sympathy for millionaires who lose their money.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24038805, "name": "Sir Hottest Of Jays", "screen_name": "j_hotz", "lang": "en", "location": "Crippled America", "create_at": date("2009-03-12"), "description": "I'm happy as fuck. CSUN. *S.A.D \nHOTZ'16", "followers_count": 2495, "friends_count": 852, "statues_count": 366330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503478616395776, "text": "I'm telling you she bae ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1891172492, "name": "Elcar_Ortega", "screen_name": "ElcarOrtega", "lang": "en", "location": "null", "create_at": date("2013-09-21"), "description": "null", "followers_count": 258, "friends_count": 296, "statues_count": 2131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503478767353856, "text": "Temp: 33.1F W C: 33.1F Wind:NE at 2.5kts Baro: 1012.7mb and Falling quickly Rain today: 0.13in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 193, "friends_count": 218, "statues_count": 97222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503478817816580, "text": "@LoganPerniciaro UGHHH", "in_reply_to_status": 699503335024521217, "in_reply_to_user": 347380442, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 347380442 }}, "user": { "id": 304229243, "name": "kaylee renae michel", "screen_name": "kay_kangaroo", "lang": "en", "location": "null", "create_at": date("2011-05-23"), "description": "Let the good times roll. #prayforjenn #princess", "followers_count": 853, "friends_count": 550, "statues_count": 8665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hattiesburg, MS", "id": "27d0ee77b9d28896", "name": "West Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.428655,31.292272 -89.351895,31.340455") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28073, "countyName": "Lamar", "cityID": 2878890, "cityName": "West Hattiesburg" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503479249727488, "text": "@AintUSmookDawg he said buddy ������ you get your ticket btw? ��", "in_reply_to_status": 699503303034408960, "in_reply_to_user": 58666623, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 58666623 }}, "user": { "id": 1447744171, "name": "Brykell Patton", "screen_name": "TheReal6_OMO", "lang": "en", "location": "Chicago, IL 054 L'z 051 Yam", "create_at": date("2013-05-21"), "description": "love my brothers. RIP shocka and Bart Gladney! free the real. EBK FollyWorld SODM⛽️ SIU-E", "followers_count": 847, "friends_count": 525, "statues_count": 25380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, IL", "id": "79199877974d60ba", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-88.970057,38.279263 -88.847059,38.370185") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17081, "countyName": "Jefferson", "cityID": 1751180, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503479744671744, "text": "���� https://t.co/k2Ttfw0b6n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3435267734, "name": "Jess", "screen_name": "_alvarezjessica", "lang": "en", "location": "null", "create_at": date("2015-09-03"), "description": "18 | LA ✨", "followers_count": 69, "friends_count": 63, "statues_count": 520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503479874809857, "text": "Sleep for what", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129578820, "name": "Rich-E Rich", "screen_name": "RichERich757", "lang": "en", "location": "Suffolk, VA", "create_at": date("2010-04-04"), "description": "Rapper/Producer/Battle Rapper/SongWritter .. Bookings/Features/Production:\nRoshawn@drinkingonsunday.com . #IdRatherBeRich . IG: @richerich757", "followers_count": 6049, "friends_count": 3417, "statues_count": 26777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480059244544, "text": "@kanyewest fuck the negativity, stay positive no matter what Ye!", "in_reply_to_status": 699503431816519680, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 3380357593, "name": "Righteous", "screen_name": "rxghteous", "lang": "en", "location": "Los Angeles", "create_at": date("2015-08-29"), "description": "ЯiR / aka. Mihji", "followers_count": 153, "friends_count": 72, "statues_count": 95 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480067723264, "text": "Pleaseeee cancel class tomorrow ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 785294342, "name": "Maggie Casique Ⓜ️", "screen_name": "CasiqueMaggie", "lang": "en", "location": "null", "create_at": date("2012-08-27"), "description": "God is the way to life † ..College Girl Instagram: casique123", "followers_count": 855, "friends_count": 1335, "statues_count": 15672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, GA", "id": "ec6dac47648ca27f", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-83.89971,34.238672 -83.762889,34.362593") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall", "cityID": 1331908, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480159928321, "text": "@vin_chenzo1 @krysteeng_ HAHHAHAHAAHAHAHAHAHAHAHAHAHA", "in_reply_to_status": 699503398270271488, "in_reply_to_user": 527600931, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 527600931, 3283391347 }}, "user": { "id": 522650464, "name": "sweaty and musty", "screen_name": "BibleButcher_", "lang": "en", "location": "null", "create_at": date("2012-03-12"), "description": "no", "followers_count": 888, "friends_count": 484, "statues_count": 84083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jacinto, CA", "id": "0068bc1a061e33c3", "name": "San Jacinto", "place_type": "city", "bounding_box": rectangle("-117.039483,33.751357 -116.919422,33.818476") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 667112, "cityName": "San Jacinto" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480323444736, "text": "@kanyewest wiz got you thinking about your life huh", "in_reply_to_status": -1, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 423866780, "name": "fleeeeezy", "screen_name": "AlecFlynn", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "we don't get got.... we go get", "followers_count": 912, "friends_count": 836, "statues_count": 28253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480369614848, "text": "This homework is making me feel physically sick don't you like the higher education in this country ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88383127, "name": "destibee", "screen_name": "abcdestinee", "lang": "en", "location": "ok", "create_at": date("2009-11-08"), "description": "experience yourself.", "followers_count": 474, "friends_count": 301, "statues_count": 11099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480533168129, "text": "@leslieeepardooo \"hey lil bitchhhh\"", "in_reply_to_status": 699503185501655040, "in_reply_to_user": 3307700965, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3307700965 }}, "user": { "id": 384151657, "name": "Salma Ruiz", "screen_name": "_salmaruiz", "lang": "en", "location": "Beaverton, OR", "create_at": date("2011-10-02"), "description": "18 | Luke 6:31 | ✨", "followers_count": 796, "friends_count": 271, "statues_count": 57060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503480973631488, "text": "@ever_burrito I'm not a kid, kid����", "in_reply_to_status": 699503136390516736, "in_reply_to_user": 3425191804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3425191804 }}, "user": { "id": 3111665322, "name": "jaque ت", "screen_name": "jaque_aguilar83", "lang": "en", "location": "null", "create_at": date("2015-03-27"), "description": "sc: jaque_aguilar1\n bchs", "followers_count": 95, "friends_count": 128, "statues_count": 2790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atwater, CA", "id": "676c3e6a1e0642db", "name": "Atwater", "place_type": "city", "bounding_box": rectangle("-120.641299,37.325641 -120.555357,37.374957") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 603162, "cityName": "Atwater" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503481246187520, "text": "HAPPY 18th BIRTHDAY!!!����❤️������ @JazsminGonzales", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 607781527 }}, "user": { "id": 575011692, "name": "Samquantch", "screen_name": "thunderfist25", "lang": "en", "location": "California, USA", "create_at": date("2012-05-08"), "description": "senior//", "followers_count": 445, "friends_count": 572, "statues_count": 7783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503481841807360, "text": "I wish I knew what my roommate was saying on the phone at 1 am in a pitch black room buried under the covers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2490627192, "name": "merlot ho", "screen_name": "kweazus", "lang": "en", "location": "lost in the sauce", "create_at": date("2014-05-11"), "description": "denver, co", "followers_count": 293, "friends_count": 395, "statues_count": 5742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503482148028417, "text": "I fucking hate the Grammy's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.2385174,36.1594545"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205984186, "name": "Young Based God ✌", "screen_name": "StupidDopeLines", "lang": "en", "location": "Location location location ", "create_at": date("2010-10-21"), "description": "W$GT$ OpM Till It's Over #FuckCancer", "followers_count": 165, "friends_count": 408, "statues_count": 23553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-16T00:00:00.000Z"), "id": 699503482181562369, "text": "@Tamayo38J thanks fam!!!", "in_reply_to_status": 699502544821702656, "in_reply_to_user": 717342674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717342674 }}, "user": { "id": 2920999123, "name": "Angie Mercado", "screen_name": "gossiperrrr", "lang": "en", "location": "Bay Area", "create_at": date("2014-12-06"), "description": "illumina mi vida✨", "followers_count": 231, "friends_count": 220, "statues_count": 9347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503482684846080, "text": "Aldo: Omar speaks 3rd most..\nOmar: I don't speak Italian.\nAldo: Like I said 3rd most, can you just keep your fucking mouth shut. \n����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 508287313, "name": "Keenan", "screen_name": "KeenanSupple", "lang": "en", "location": "null", "create_at": date("2012-02-28"), "description": "20 OSU (Legendary-Humble)", "followers_count": 243, "friends_count": 151, "statues_count": 14902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503482852642817, "text": "My dads always extra lol https://t.co/olHqdRCzsf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 449656568, "name": "⠀⠀⠀⠀⠀", "screen_name": "_palomuh", "lang": "en", "location": "null", "create_at": date("2011-12-29"), "description": "PLAYIN DIRTY, NOT CLEAN", "followers_count": 374, "friends_count": 666, "statues_count": 15626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503483062349824, "text": "@justinrobel https://t.co/p4cCMcFtW0", "in_reply_to_status": 699501805516910592, "in_reply_to_user": 848553828, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 848553828 }}, "user": { "id": 159606151, "name": "pixxx", "screen_name": "_pixielicious", "lang": "en", "location": "The Cupboard Under the Stairs", "create_at": date("2010-06-25"), "description": "null", "followers_count": 486, "friends_count": 483, "statues_count": 25014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Palma, CA", "id": "641cdaa7992c949e", "name": "La Palma", "place_type": "city", "bounding_box": rectangle("-118.058498,33.83555 -118.028551,33.866221") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640256, "cityName": "La Palma" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503483213316096, "text": "I hate seeing little middle/high school girls posting their bodies all over twitter like girlllll u ain't even 18 yet chill for a bit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66474281, "name": "⚡️BB THRIVE⚡️", "screen_name": "hyperrreality", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-08-17"), "description": "1-800-KILLURSLF IG: HYPERRREALITY", "followers_count": 1700, "friends_count": 923, "statues_count": 116253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerlin South, NV", "id": "0134e6167ff7f6ec", "name": "Summerlin South", "place_type": "city", "bounding_box": rectangle("-115.355825,36.082837 -115.314862,36.159081") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3270900, "cityName": "Summerlin South" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503483234332672, "text": "Boy you cute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2894361470, "name": "cat", "screen_name": "onpointsquad", "lang": "en", "location": "El Centro, CA", "create_at": date("2014-11-08"), "description": "shs'19", "followers_count": 114, "friends_count": 448, "statues_count": 1200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Centro, CA", "id": "00436a376f214be7", "name": "El Centro", "place_type": "city", "bounding_box": rectangle("-115.594987,32.730647 -115.501086,32.818522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 621782, "cityName": "El Centro" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503483674718208, "text": "Religion's irrational.\n\nA car accident is well an ACCIDENT. https://t.co/3tsFQKWgSr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 846945457, "name": "Tweeter™", "screen_name": "CommonCormorant", "lang": "en", "location": "The Archangel Valley, Ca", "create_at": date("2012-09-26"), "description": "Wrote this: http://mybook.to/UberTyzmon (Act 1 via #WattPad): http://wattpad.com/story/20405627… Support #NetNeutrality", "followers_count": 1899, "friends_count": 2124, "statues_count": 222252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altadena, CA", "id": "fb76b3aa366004c9", "name": "Altadena", "place_type": "city", "bounding_box": rectangle("-118.177296,34.167539 -118.095091,34.220078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601290, "cityName": "Altadena" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503483754381313, "text": "@ebbtideapp Tide in Port Isabel, Texas 02/16/2016\n Low 4:03am -0.3\nHigh 2:33pm 1.2\n Low 5:10am -0.4\nHigh 3:21pm 1.2\n Low 6:12am -0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-97.215,26.06"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 29, "friends_count": 1, "statues_count": 8163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Isabel, TX", "id": "b4a9fe6b81986da6", "name": "Port Isabel", "place_type": "city", "bounding_box": rectangle("-97.249586,26.035052 -97.19388,26.080322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4858892, "cityName": "Port Isabel" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503484068954113, "text": "But you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043034848, "name": "Rut✌️", "screen_name": "suarez_rut", "lang": "en", "location": "null", "create_at": date("2012-12-28"), "description": "⚽️❤️#Scorpio #EverGreenState| Nov19| between your girls legs", "followers_count": 3675, "friends_count": 3875, "statues_count": 62531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prosser, WA", "id": "5ddc7ee8a1686aa9", "name": "Prosser", "place_type": "city", "bounding_box": rectangle("-119.810457,46.191302 -119.710595,46.229075") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5356450, "cityName": "Prosser" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503484660416513, "text": "https://t.co/YLCkC2n87S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 203439030, "name": "Bona", "screen_name": "skybona", "lang": "en", "location": "San Francisco Area, USA", "create_at": date("2010-10-16"), "description": "an RN who loves to make a difference in lives of others! 공정하고 투명하고 정정당당한 정의로운 내조국은 언제나 가능할까?", "followers_count": 1323, "friends_count": 1481, "statues_count": 2867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windsor, CA", "id": "cf574ccb037d4ff3", "name": "Windsor", "place_type": "city", "bounding_box": rectangle("-122.83677,38.515011 -122.772142,38.566956") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 685922, "cityName": "Windsor" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503484719210496, "text": "You ever just wanna call somebody you used to fw and cuss them out, cause you miss em and they wasted ya time?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2920086457, "name": "☺", "screen_name": "fvck_mari", "lang": "en", "location": "Winston-Salem, NC", "create_at": date("2014-12-05"), "description": "gay | 336 | finding myself", "followers_count": 1265, "friends_count": 1010, "statues_count": 30109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503485025415168, "text": "hello old feeling.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264430104, "name": "tevin p", "screen_name": "tevinpeirsol", "lang": "en", "location": "idaho 2 cbus", "create_at": date("2011-03-11"), "description": "20. the spirit lives on. 208 raised. 614 livin. THE Ohio State University '17. | #TRBESMEN", "followers_count": 992, "friends_count": 824, "statues_count": 47607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503486174539776, "text": "@thatgoldsoulguy @miss_victoriia https://t.co/GNZshZjcGp", "in_reply_to_status": 699503350530723841, "in_reply_to_user": 3258623438, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3258623438, 4029882554 }}, "user": { "id": 703902710, "name": "Marcus Hyatt", "screen_name": "Hypro2015", "lang": "en", "location": "SA, TX", "create_at": date("2012-07-18"), "description": "Recording/Mixing Engineer, Producer, Yamaha NS-10m are my speakers of choice\n#BadVibeStudios #PowerlineStudios #DreamTeam\nmarcushyatt.badvibe@gmail.com", "followers_count": 378, "friends_count": 475, "statues_count": 30742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503486233407488, "text": "@gabryBoschetto rosewood is pretty good", "in_reply_to_status": 699470324845273088, "in_reply_to_user": 84227665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 84227665 }}, "user": { "id": 2938529572, "name": "Brandon Starr", "screen_name": "brandonstarr05", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2014-12-23"), "description": "Quaker Valley Football Coach. Duquesne\nBayern Munich", "followers_count": 70, "friends_count": 230, "statues_count": 1937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Osborne, PA", "id": "00f8dedd297ff4fa", "name": "Glen Osborne", "place_type": "city", "bounding_box": rectangle("-80.177987,40.525599 -80.153756,40.542527") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4229732, "cityName": "Glen Osborne" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503486313041920, "text": "02/16@03:00 - Temp 33.3F, WC 33.3F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.980in, Falling. Rain 0.00in. Hum 98%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-16T00:00:01.000Z"), "id": 699503486468116480, "text": "@God_sgift_ @theurbantwist https://t.co/Svexjh2e5i", "in_reply_to_status": 699502796588974080, "in_reply_to_user": 94422489, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 94422489, 52519277 }}, "user": { "id": 15110172, "name": "Petty Schuyler", "screen_name": "robyn_ravenclaw", "lang": "en", "location": "In NY, you can be a new man", "create_at": date("2008-06-13"), "description": "I'd rather be divisive than indecisive, drop the niceties. Black Girl Nerdfighter. 1/2 of @weblackandnerds. The less chill host of #WizardTeam DFTBA", "followers_count": 916, "friends_count": 1863, "statues_count": 25119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503486682009600, "text": "That's so cute �� if a nigga did that for me I would probably pass tf out , like its the effort that count", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321615834, "name": "Ari", "screen_name": "ariariearih_", "lang": "en", "location": "null", "create_at": date("2011-06-21"), "description": "20| don't dm me I'm not interested", "followers_count": 2460, "friends_count": 346, "statues_count": 110561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503486908571650, "text": "“@GabiCassell: @Arleth_Estaca ���� https://t.co/RyekDkYkmn” ������ bitches ain't got nothing on us. #QuackQuackMotherPuckers", "in_reply_to_status": 699430058025312256, "in_reply_to_user": 755708666, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "QuackQuackMotherPuckers" }}, "user_mentions": {{ 755708666, 550903038 }}, "user": { "id": 550903038, "name": "Arleth Joy", "screen_name": "Arleth_Estaca", "lang": "en", "location": "Proverbs 31:25", "create_at": date("2012-04-11"), "description": "I've got bags full of money and cars full of bitches. Leggo.", "followers_count": 316, "friends_count": 280, "statues_count": 12468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503487835512832, "text": "*headshake* I'ts never real - I may just start crashing through thigns becuse I'm tired of being diverted by avoiding them so much but..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 77, "friends_count": 222, "statues_count": 117051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503487877541888, "text": "Wind 3.0 mph ESE. Barometer 29.815 in, Steady. Temperature 34.1 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503487994851328, "text": "IF YOU LIKE PINA COLADAS N GETTING CAUGHT IN THE RAIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2435019463, "name": "naomz", "screen_name": "Tranaomi", "lang": "en", "location": "null", "create_at": date("2014-04-08"), "description": "♐️", "followers_count": 462, "friends_count": 318, "statues_count": 6085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503488200388612, "text": "48.4F (Feels: 48.4F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1025.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 227029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503488749842433, "text": "benevolent x blunt is just a rly odd combination", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1049714868, "name": "Karyn Allana", "screen_name": "KarynReyes4", "lang": "en", "location": "null", "create_at": date("2012-12-30"), "description": "But I don't put much stock in things beyond the weather and the certainty of living and dying: the rest is optional.", "followers_count": 433, "friends_count": 386, "statues_count": 6574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineyard, CA", "id": "01f9f8b01415323b", "name": "Vineyard", "place_type": "city", "bounding_box": rectangle("-121.371979,38.452572 -121.292468,38.485308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 682852, "cityName": "Vineyard" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503488779157505, "text": "Wind 2.0 mph SSW. Barometer 29.710 in, Falling slowly. Temperature 34.4 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 14, "statues_count": 1204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503488913563648, "text": "Wind 0.0 mph ---. Barometer 29.793 in, Falling slowly. Temperature 34.1 °F. Rain today 0.01 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 53, "friends_count": 27, "statues_count": 17668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503489504784384, "text": "@alexaiono HAPPY BIRTHDAY❤️❤️❤️❤️❤️❤️", "in_reply_to_status": -1, "in_reply_to_user": 234980063, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 234980063 }}, "user": { "id": 633835186, "name": "Maddi", "screen_name": "maddi_love21", "lang": "en", "location": "LA", "create_at": date("2012-07-12"), "description": "null", "followers_count": 431, "friends_count": 343, "statues_count": 11488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503489592881158, "text": "Deadpool is the cutest movie I have ever seen.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 833857386, "name": "Shannon Ireland", "screen_name": "_slireland", "lang": "en", "location": "null", "create_at": date("2012-09-19"), "description": "null", "followers_count": 77, "friends_count": 46, "statues_count": 2909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503490117160960, "text": "���� lmaoooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 298872517, "name": "Idigg", "screen_name": "NoLackMack", "lang": "en", "location": "null", "create_at": date("2011-05-14"), "description": "#TXST18 - -Dallas/SM Pray Grind Shine", "followers_count": 2132, "friends_count": 1463, "statues_count": 65994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-16T00:00:02.000Z"), "id": 699503490234580993, "text": "Temp: 47.5°F Wind:0.0mph Pressure: 29.962hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491278983168, "text": "Accident in #LasVegas on Rancho Dr at Washington Ave #VegasTraffic https://t.co/5rmaMyCQ1w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1792,36.1811"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LasVegas", "VegasTraffic" }}, "user": { "id": 249826185, "name": "TTN Las Vegas", "screen_name": "TotalTrafficLV", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-02-09"), "description": "Covering all your traffic needs throughout Greater Las Vegas. Please call 702-238-7333 if you have any traffic tips.", "followers_count": 2143, "friends_count": 137, "statues_count": 75636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491446755328, "text": "someone ft me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3274733916, "name": "kat", "screen_name": "katrinacust0dio", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "null", "followers_count": 388, "friends_count": 289, "statues_count": 12387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491530686464, "text": "I miss @SkyiiValencia n @seauhhleen ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 140722110, 2251000879 }}, "user": { "id": 170929856, "name": "OgDaThirstman", "screen_name": "thethirstea", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-07-25"), "description": "I woke up like this #2ndsucks", "followers_count": 388, "friends_count": 314, "statues_count": 21688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491589353473, "text": "@emyak_emyat @onrymari @maiden16_IL @jaypee88 @ItsAnneAndres @amysdempsey @bambinaelleir @jsc_11 \nBash pa more 1\n#VoteMaineFPP #KCA", "in_reply_to_status": 699503407875227648, "in_reply_to_user": 3106045522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VoteMaineFPP", "KCA" }}, "user_mentions": {{ 3627191477, 298059021, 3723576335, 86188906, 217644922, 3609293413, 2481541596, 3638085383 }}, "user": { "id": 3106045522, "name": "Sha-sha Salazar", "screen_name": "salazar121378", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-03-22"), "description": "Registered Nurse\nAdvocate @ Home\nAdvocate Healthcare", "followers_count": 435, "friends_count": 1094, "statues_count": 32569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491681689600, "text": "Kayne tweeting rn too dang wild night lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613936988, "name": "M I G G Y ☁️", "screen_name": "BeanMiggy", "lang": "en", "location": "blackpeoplemeet.com", "create_at": date("2012-06-20"), "description": "University of the Pacific", "followers_count": 528, "friends_count": 501, "statues_count": 26342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491832610817, "text": "Wind 1.6 mph S. Barometer 29.65 in, Rising slowly. Temperature 28.9 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 40084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503491849412608, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling. Temperature 23.4 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 111, "statues_count": 157556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503492088537088, "text": "@Schuck1113 didn't wanna wake you by text. Call/Text me in AM if you wanna hang out tomorrow (Tuesday) or if ya have to do any shopping! x", "in_reply_to_status": -1, "in_reply_to_user": 3245640324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3245640324 }}, "user": { "id": 60944740, "name": "Marci Gower", "screen_name": "MarciMarketing", "lang": "en", "location": "SWFL, Denver & Birmingham, UK", "create_at": date("2009-07-28"), "description": "NY raised, Brit-US traveler currently living between 3 cities. I feel the Bern! I Love cats & my hubby. Considering a 5th. A kitten not husband!", "followers_count": 852, "friends_count": 559, "statues_count": 8255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503492298215424, "text": "This is the solitude I enjoy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274140075, "name": "I Am Reptar", "screen_name": "Time4DinDin", "lang": "en", "location": "MD ✈️ WV", "create_at": date("2011-03-29"), "description": "iFootball at #ShepherdU19 | DMG | Veni Vidi Vici | RIP Keyonden | RIP Riley | Sorry ¯\\_(ツ)_/¯ | 6th Mizukage | NPNation Alumni", "followers_count": 1842, "friends_count": 1191, "statues_count": 83364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shepherdstown, WV", "id": "7625ad95127b9d87", "name": "Shepherdstown", "place_type": "city", "bounding_box": rectangle("-77.814086,39.427069 -77.797026,39.440593") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54037, "countyName": "Jefferson", "cityID": 5473468, "cityName": "Shepherdstown" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503492428341248, "text": "Wind 0.2 mph S. Barometer 30.058 in, Falling Rapidly. Temperature 24.2 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493011390464, "text": "Barberton Oh Temp:32.5°F Wind:0 mph Dir:--- Baro:Falling Rain2day:0.16in Hum:87% UV:0.0 @ 03:00 02/16/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493040615424, "text": "@kanyewest stop", "in_reply_to_status": 699503463332474880, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 180634548, "name": ":.", "screen_name": "xLouieee", "lang": "en", "location": "null", "create_at": date("2010-08-19"), "description": "I am your Twitter OG and will be respected as such", "followers_count": 553, "friends_count": 330, "statues_count": 24289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493170659328, "text": "@KAYNEIWEST you going off tonight Bruh", "in_reply_to_status": -1, "in_reply_to_user": 349410021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 349410021 }}, "user": { "id": 180881430, "name": "Chris Gutierrez", "screen_name": "ArctiChriss", "lang": "en", "location": "null", "create_at": date("2010-08-20"), "description": "null", "followers_count": 275, "friends_count": 228, "statues_count": 4637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cold Springs, NV", "id": "41cfb5f1c335d732", "name": "Cold Springs", "place_type": "city", "bounding_box": rectangle("-120.001214,39.650202 -119.940808,39.704784") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3214090, "cityName": "Cold Springs" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493489561600, "text": "Another award she won that you didn't... Don't hate what you can't achieve https://t.co/8eMUcGwIsb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366815355, "name": "Cardi E Ⓥ", "screen_name": "3rykah", "lang": "en", "location": "McA Band ", "create_at": date("2011-09-02"), "description": "life goes on// if you think you're more smarcus than me, don't get your highs too hope", "followers_count": 255, "friends_count": 423, "statues_count": 654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, FL", "id": "2bc7c264a080898b", "name": "Hollywood", "place_type": "city", "bounding_box": rectangle("-80.248975,25.986228 -80.107808,26.093132") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1232000, "cityName": "Hollywood" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493850071041, "text": "@kanyewest FOLLOW BACK", "in_reply_to_status": 699503431816519680, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 944918460, "name": "Lil Val Wow", "screen_name": "yungvtrilla", "lang": "en", "location": "Bae Area, Ca", "create_at": date("2012-11-12"), "description": "I subtweet myself not you people", "followers_count": 3795, "friends_count": 963, "statues_count": 162813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503493933993984, "text": "Wind 2.0 mph ESE. Barometer 29.986 in, Falling Rapidly. Temperature 30.0 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 6552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503494093377536, "text": "“There are nights I don’t exist. \nThere are days I fade away.”", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1567210248, "name": "ᴀᴍʏ", "screen_name": "Amyhilliard14", "lang": "en", "location": "Ephrata, WA", "create_at": date("2013-07-03"), "description": "Senior @ EHS ≪ raisin' hell ≫ at war with love", "followers_count": 416, "friends_count": 513, "statues_count": 4254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ephrata, WA", "id": "7f6b5bf36eafdf84", "name": "Ephrata", "place_type": "city", "bounding_box": rectangle("-119.583051,47.281037 -119.514653,47.33987") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5322080, "cityName": "Ephrata" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503494487805952, "text": "@alexaiono joyeux anniversaire", "in_reply_to_status": 699503302229098496, "in_reply_to_user": 234980063, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 234980063 }}, "user": { "id": 637379402, "name": "A.", "screen_name": "juvnlopez", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "null", "followers_count": 1410, "friends_count": 419, "statues_count": 35195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503494693171200, "text": "That's just how it's gonna have to be.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1850226007, "name": "T-LAIN", "screen_name": "lain_tristin", "lang": "en", "location": "null", "create_at": date("2013-09-09"), "description": "Isaiah 54:10", "followers_count": 354, "friends_count": 395, "statues_count": 761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-16T00:00:03.000Z"), "id": 699503494940590080, "text": "\"you look like post malone's pubic hair\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785265384, "name": "saf", "screen_name": "safieezy", "lang": "en", "location": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀okaga, ca⠀", "create_at": date("2014-09-01"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 770, "friends_count": 731, "statues_count": 1872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503495309725696, "text": "Okay females what's your secret, explain how you wipe your ass with these on https://t.co/0eg3sRwI5x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334422624, "name": "Brian Smith", "screen_name": "bdscold2ak", "lang": "en", "location": "Anchorage AK", "create_at": date("2011-07-12"), "description": "#ColeCarmodyCantHang Michaela is cool I guess @itylerstevens is my boyfriend we long distance @pizzapartben is my side nigga", "followers_count": 519, "friends_count": 293, "statues_count": 7422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503495339249664, "text": "@__tyrik �� u already know how im coming", "in_reply_to_status": 699502876851249153, "in_reply_to_user": 1699824037, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1699824037 }}, "user": { "id": 130603143, "name": "The Dapper Don", "screen_name": "CincoDeTayo_5", "lang": "en", "location": "404 ✈️ 337", "create_at": date("2010-04-07"), "description": "Birthed By A Queen, Raised By The Slums, Enlightened By The Gods, Fuxk Em We Ball #OnSquad #FutureHive #FreeDaWop", "followers_count": 1733, "friends_count": 1203, "statues_count": 78708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503495515348992, "text": "#SuspiciousPerson at 1105-1298 Densmore Dr, Winter Park, FL 32792. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3121378,28.6095273"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousPerson", "orlpol", "ocso" }}, "user": { "id": 40390214, "name": "Police Calls 32792", "screen_name": "orlpol32792", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-15"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 104, "friends_count": 1, "statues_count": 3969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503495565570048, "text": "Who are u https://t.co/3t7pjVrSsl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3235712898, "name": "J_Flye", "screen_name": "JeremyStJulian1", "lang": "en", "location": "California, USA", "create_at": date("2015-06-03"), "description": "grindin til i cant grind no mo Sc: jcruz225 cash flo LSHS ❤️ #54 // oblock boys", "followers_count": 608, "friends_count": 486, "statues_count": 1092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503495926317056, "text": "Temp: 23.4°F - Dew Point: 16.7° - Wind: 2.2 mph - Gust: 4.5 - Rain Today: 0.00in. - Pressure: 29.74in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503496236703750, "text": "Omg... Tidal it is https://t.co/bILFZhgMH0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 173042132, "name": "LIL BIG HOMIE", "screen_name": "DiegoEscorza", "lang": "en", "location": "Santa Cruz, CA", "create_at": date("2010-07-31"), "description": "I Like Clothes I Cant Afford And Girls I'll Never Get. STS Be the Gang @stsgallery.", "followers_count": 822, "friends_count": 1231, "statues_count": 31099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503496278609920, "text": "I'm a blueprint to a real man.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2279534869, "name": "Rey STEEZ", "screen_name": "ReyBFresh", "lang": "en", "location": "null", "create_at": date("2014-01-06"), "description": "vibe with me|47|20young|Student Athlete|World Financial", "followers_count": 322, "friends_count": 307, "statues_count": 8007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503496467476480, "text": "@hhopkins15 no doubt appreciate it!✊����", "in_reply_to_status": 699469338659586048, "in_reply_to_user": 632056766, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 632056766 }}, "user": { "id": 419184028, "name": "Tre' waters", "screen_name": "Waters4God", "lang": "en", "location": "Tennessee", "create_at": date("2011-11-22"), "description": "Co-Founder of @FAMED_Media , UTC, Webb '10 Elevated and illuminated! I AM Art! Video producer/artist IG: waters_3ternal Snapchat: fresh_above33 MVZW 7/10/15 ❤️", "followers_count": 1402, "friends_count": 621, "statues_count": 2168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503496609968131, "text": "My name is Mr Paul Patterson... And I am black history. #blackhistory #blackhistorymonth… https://t.co/RIEBrvjE2D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.5179,41.0732"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blackhistory", "blackhistorymonth" }}, "user": { "id": 151970082, "name": "Fresh studios", "screen_name": "FreshStudios1", "lang": "en", "location": "Worldwide", "create_at": date("2010-06-04"), "description": "null", "followers_count": 570, "friends_count": 530, "statues_count": 4108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503496811286528, "text": "@HTTPBBYGURL QUE PINCHES CERDA ERES U_U ESPÉRAME YA VOY", "in_reply_to_status": 699503262429540352, "in_reply_to_user": 2953841249, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2953841249 }}, "user": { "id": 706254061, "name": "Caleb my bae", "screen_name": "Leuffart", "lang": "es", "location": "Quinn | Mi otra mitad.", "create_at": date("2012-07-19"), "description": "Caleb | Mis ojitos. Thomas| Mi niño. Amedyzt | amor de mi vida. Alek | El feo. Kelsey | La wawa.", "followers_count": 1053, "friends_count": 934, "statues_count": 8553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503497113264128, "text": "@AmryiP ohhh lol. Yeah we walked it like twice alrdy. But we going ziplining now haha��. Where are you guys?", "in_reply_to_status": 699503346881683456, "in_reply_to_user": 3020546781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3020546781 }}, "user": { "id": 3241776228, "name": "Nae✨", "screen_name": "naewantscandy", "lang": "en", "location": "keaulana,nanakuli", "create_at": date("2015-06-10"), "description": "ig- @naelovescookies", "followers_count": 136, "friends_count": 146, "statues_count": 2867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503497213915137, "text": "real question is why I didn't bring PD to hawaii", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 402409640, "name": "molly", "screen_name": "cant_findmolly", "lang": "en", "location": "Jamaica", "create_at": date("2011-10-31"), "description": "welcome.", "followers_count": 609, "friends_count": 495, "statues_count": 13828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Loch Estate, HI", "id": "001d93ff0c068a75", "name": "West Loch Estate", "place_type": "city", "bounding_box": rectangle("-158.035743,21.349278 -158.016301,21.376145") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1580470, "cityName": "West Loch Estate" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503497641746433, "text": "Where did you fine my pictures �� https://t.co/908oS0ISH8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1198034335, "name": ".", "screen_name": "Rojo99_", "lang": "en", "location": "null", "create_at": date("2013-02-19"), "description": "Building My Empire.", "followers_count": 438, "friends_count": 397, "statues_count": 12923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503497763377153, "text": "pix of me!!! https://t.co/2xKEW5lyTH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2989106165, "name": "marin", "screen_name": "IiIhomie", "lang": "en", "location": "null", "create_at": date("2015-01-20"), "description": "always down for sexting", "followers_count": 134, "friends_count": 85, "statues_count": 3572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Place, WA", "id": "9cd68e26f3c4880c", "name": "University Place", "place_type": "city", "bounding_box": rectangle("-122.58428,47.185193 -122.505193,47.245003") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5373465, "cityName": "University Place" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503498245713920, "text": "NUMBER 8 THO @rameeshaaa_ https://t.co/pVmbhF6KGs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4581518833 }}, "user": { "id": 3955490172, "name": "Sujøy Shah", "screen_name": "sujoy_shah", "lang": "en", "location": "Band Room", "create_at": date("2015-10-20"), "description": "Kylo Ren is my favorite Disney princess .|Live 105's BFD 6.6.15|Blurryface Tour Fox Oakland Theater 10.18.15 |-/", "followers_count": 68, "friends_count": 174, "statues_count": 967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-02-16T00:00:04.000Z"), "id": 699503498694680576, "text": "Mhm mhm mhm I got plans for myself����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2193521309, "name": "iZEL", "screen_name": "notabasicbasis", "lang": "en", "location": "chicago", "create_at": date("2013-11-23"), "description": "3.O6 For PEACH!", "followers_count": 1919, "friends_count": 1591, "statues_count": 16216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, IL", "id": "003cc2276dfcaca4", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-87.580767,41.539484 -87.524848,41.595245") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1742028, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499231391744, "text": "Leftover sushi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116041867, "name": "helen", "screen_name": "heleeennn_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-02-20"), "description": "It is what it is", "followers_count": 4367, "friends_count": 146, "statues_count": 27627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499256528896, "text": "*Shrugs*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 77, "friends_count": 222, "statues_count": 117052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499256573952, "text": "I don't see how Kevin gates does it. I'm exhausted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619220524, "name": "Coled hearted", "screen_name": "heyy_koop", "lang": "en", "location": "Arkansas", "create_at": date("2012-06-26"), "description": "6 ft 2 and better than you ▪️ SC: heykoop83 ▪️ Insta: heyy_koop ▪️ I take cool pictures of pretty people like you▪️ I love pizza more ▪️ Christian", "followers_count": 563, "friends_count": 317, "statues_count": 9353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Russellville, AR", "id": "c09505551daee91a", "name": "Russellville", "place_type": "city", "bounding_box": rectangle("-93.205955,35.220654 -93.056961,35.336506") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5115, "countyName": "Pope", "cityID": 561670, "cityName": "Russellville" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499457994752, "text": "@Poudii @VinesAndVids lol she probably a junior in high school dog you don't want them charges", "in_reply_to_status": 699499580061306880, "in_reply_to_user": 127335922, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 127335922, 2542207489 }}, "user": { "id": 574118014, "name": "Juantell Jordan", "screen_name": "OG_JuanR3", "lang": "en", "location": "Sjuansboro ✈️ Boosie Creek", "create_at": date("2012-05-07"), "description": "Why are you wearing that stupid man suit?", "followers_count": 633, "friends_count": 563, "statues_count": 15369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buies Creek, NC", "id": "01f3b1cd305ec774", "name": "Buies Creek", "place_type": "city", "bounding_box": rectangle("-78.774818,35.397135 -78.702385,35.422338") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37085, "countyName": "Harnett", "cityID": 3708760, "cityName": "Buies Creek" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499533373440, "text": "@stephiegood my dad: why do you like her so much?..\nme: I don't even know how to explain this", "in_reply_to_status": 699502858777919488, "in_reply_to_user": 770907120, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 770907120 }}, "user": { "id": 770907120, "name": "steph", "screen_name": "stephiegood", "lang": "en", "location": "slc", "create_at": date("2012-08-20"), "description": "Rihanna is my goddess", "followers_count": 448, "friends_count": 380, "statues_count": 25339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandy, UT", "id": "fb9549afa6f81fa6", "name": "Sandy", "place_type": "city", "bounding_box": rectangle("-111.921658,40.528084 -111.800273,40.616827") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967440, "cityName": "Sandy" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503499965411328, "text": "Headache? Yea. That's the only thing in my life that comes constantly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 960132612, "name": "¿who?", "screen_name": "_eytina", "lang": "en", "location": "null", "create_at": date("2012-11-20"), "description": "null", "followers_count": 101, "friends_count": 93, "statues_count": 6335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503500397445120, "text": "Wind 5.0 mph SSW. Barometer 1011.82 mb, Falling. Temperature 44.9 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 29, "friends_count": 125, "statues_count": 12560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503500456120320, "text": "I'm like in utter disgust", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371803878, "name": "A", "screen_name": "anhzzanhz", "lang": "en", "location": "CA ♡", "create_at": date("2011-09-11"), "description": "all I want is to feel complete", "followers_count": 749, "friends_count": 372, "statues_count": 19845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503500603039744, "text": "#riveryachtclub #miamidowntown #brickell #miamibeach #southbeach #miamistyle #capturethemoment… https://t.co/Orvul8RAPd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.1993256,25.7700996"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "riveryachtclub", "miamidowntown", "brickell", "miamibeach", "southbeach", "miamistyle", "capturethemoment" }}, "user": { "id": 235865817, "name": "Jose Estefan", "screen_name": "Joseestwitt", "lang": "en", "location": "Miami Beach", "create_at": date("2011-01-08"), "description": "null", "followers_count": 200, "friends_count": 2007, "statues_count": 5022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503501001551873, "text": "@fatsavannah OMGGG HE MADE ONEE", "in_reply_to_status": 699503027275898880, "in_reply_to_user": 4239054749, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4239054749 }}, "user": { "id": 2933085938, "name": "Anna |-/", "screen_name": "UrieWeekess", "lang": "en", "location": "purgatory", "create_at": date("2014-12-16"), "description": "son of a bitch", "followers_count": 64, "friends_count": 177, "statues_count": 17075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503502108721152, "text": "@alexxis_209 it's starts that day but you have to have that flyer to get the deal. Giving them out with every purchase.", "in_reply_to_status": 699488571556700160, "in_reply_to_user": 1217482638, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1217482638 }}, "user": { "id": 31048059, "name": "Jessica", "screen_name": "jessica_rubio14", "lang": "en", "location": "null", "create_at": date("2009-04-13"), "description": "null", "followers_count": 454, "friends_count": 367, "statues_count": 17541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hughson, CA", "id": "62548cdeae2a4b43", "name": "Hughson", "place_type": "city", "bounding_box": rectangle("-120.879351,37.589058 -120.851611,37.609315") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 634904, "cityName": "Hughson" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503502284861440, "text": "Do u like pickles?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2435820024, "name": "nobody", "screen_name": "mydaddyyy", "lang": "en", "location": "2813308004", "create_at": date("2014-04-09"), "description": "21 | CHEVY | Salvadorian", "followers_count": 470, "friends_count": 318, "statues_count": 40910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503502318370817, "text": "Second I would like to invent a rule that it's not cheating if she more than 5 states away", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35478824, "name": "7", "screen_name": "AppleciderPapi", "lang": "en", "location": "LA", "create_at": date("2009-04-26"), "description": "18. Muslim. Beautiful. Trash Tweets.", "followers_count": 6195, "friends_count": 949, "statues_count": 164650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503502796541952, "text": "Yall wait till yall have a baby to say yall don't ever want shit to do with a guy. Yall so backwards and dumb I swear.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1178482796, "name": "QUEEN B.", "screen_name": "_theQUEENishere", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-02-14"), "description": "real wifeys hustle hard.", "followers_count": 160, "friends_count": 163, "statues_count": 12457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503502905753600, "text": "@RossGee_ I am I am", "in_reply_to_status": 699503441501097985, "in_reply_to_user": 16592447, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16592447 }}, "user": { "id": 332626927, "name": "Noyah King", "screen_name": "NoyahKing", "lang": "en", "location": "NC✈️District of Columbia", "create_at": date("2011-07-09"), "description": "#NCAT. SGA. District of Columbia's future resident. PR/Marketing/CRJS. Olivia Pope was inspired by me. Watch out for the NKCollection coming soon in DC. @MJTC_", "followers_count": 4190, "friends_count": 390, "statues_count": 117026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503503253880832, "text": "At what point do niggas stop bullshitting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735806570, "name": "A'driana Moya", "screen_name": "bellvricvn", "lang": "en", "location": "757, VA", "create_at": date("2014-08-06"), "description": "IG -bellv.ricvn SC -fielamor ✨", "followers_count": 513, "friends_count": 462, "statues_count": 13225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-02-16T00:00:05.000Z"), "id": 699503503283089408, "text": "Dom: *breaks plate*\nDom: *runs to me to tell me*\nDom: *snapchats that she broke a plate*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612120517, "name": "Jessî \\\\", "screen_name": "JessiHoran1", "lang": "en", "location": "LA//AZ ", "create_at": date("2012-06-18"), "description": "Jessi Stewart:) |17| !Hooseki Street Team Leader! | Social Media Marketing | YouTube | Model | Photography | Dance | Writer | Love Life ||Snapchat:Jessihoranuno", "followers_count": 1368, "friends_count": 2039, "statues_count": 41727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-16T00:00:06.000Z"), "id": 699503503874494464, "text": "@karlyekline I'd way rather talk on the phone and hear emotion then to see some emotionless text", "in_reply_to_status": 699503335162773504, "in_reply_to_user": 445814532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 445814532 }}, "user": { "id": 445814532, "name": "Barbie", "screen_name": "karlyekline", "lang": "en", "location": "null", "create_at": date("2011-12-24"), "description": "karlyekinney♥️", "followers_count": 1104, "friends_count": 134, "statues_count": 1596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendora, CA", "id": "eb1bb64775708bc1", "name": "Glendora", "place_type": "city", "bounding_box": rectangle("-117.890263,34.10549 -117.809111,34.165551") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 630014, "cityName": "Glendora" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865866180042752, "text": "lol it's funny that people will pretend that they care for you when they never had and never will", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 951203598, "name": "annie", "screen_name": "elocineinna", "lang": "en", "location": "los angeles ☞ minneapolis", "create_at": date("2012-11-15"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀university of minnesota ⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀ ❥ ur dream girl, rave baby ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀【=◈︿◈=】⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀not ur bæ", "followers_count": 538, "friends_count": 479, "statues_count": 29217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865866956099585, "text": "What good is being the one when you the only one that knows it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 954712370, "name": "J Cross", "screen_name": "JC21_Dreamville", "lang": "en", "location": "Near the food lion or hooping", "create_at": date("2012-11-17"), "description": "Call me JRoc. Middleburg. trying to get through this illusion im living. Still young but i done see some things. 217✈️804✈️703", "followers_count": 659, "friends_count": 623, "statues_count": 9243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansdowne, VA", "id": "00b10861396412bd", "name": "Lansdowne", "place_type": "city", "bounding_box": rectangle("-77.518504,39.0587 -77.440661,39.115185") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5144048, "cityName": "Lansdowne" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865867576709120, "text": "His friend was doing \"black\" and blacked out backstage. They had to call the paramedics they was wildin so bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151514421, "name": "Designated Roller", "screen_name": "TuckGod", "lang": "en", "location": "Rancho Cucamonga, CA ", "create_at": date("2010-06-03"), "description": "#HEALTHYBOYZ Professional Drug Abuser & Blunt Roller #BluntEtiquette", "followers_count": 2309, "friends_count": 292, "statues_count": 24247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865869275545600, "text": "Where? �� https://t.co/FDq1vQwgIg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 575106641, "name": "M E E K⚡️", "screen_name": "King_Shit5", "lang": "en", "location": "New Rochelle, NY", "create_at": date("2012-05-08"), "description": "SUCK All 2016% OF MY DICK", "followers_count": 2097, "friends_count": 1866, "statues_count": 33678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rahway, NJ", "id": "9d22866e11c9b6b7", "name": "Rahway", "place_type": "city", "bounding_box": rectangle("-74.305281,40.592141 -74.255435,40.627526") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3461530, "cityName": "Rahway" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865869309054976, "text": "Sold the house and moved into my moms attic.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540428149, "name": "mr. cantley", "screen_name": "j_odonnell97", "lang": "en", "location": "Morgantown, WV", "create_at": date("2012-03-29"), "description": "Joey O'Donnell|work|car enthusiast|fuck up||white trash|WVU|", "followers_count": 255, "friends_count": 240, "statues_count": 5011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865869556457472, "text": "RE FUCKING TWEET https://t.co/tGjxVd34AU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598935045, "name": "tams.", "screen_name": "tam_materrr", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "once upon a time someone got me fifty shades of fucked up.", "followers_count": 387, "friends_count": 247, "statues_count": 10920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale, AZ", "id": "0015d9147cee6907", "name": "Avondale", "place_type": "city", "bounding_box": rectangle("-112.357999,33.384785 -112.272424,33.493806") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 404720, "cityName": "Avondale" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865869858410496, "text": "Seeing sebas in my tepe videos makes me sad, miss you primo ����❤️ https://t.co/zav2khDCA1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240923892, "name": "JanetMV", "screen_name": "jaayeemvee", "lang": "en", "location": "SFSU", "create_at": date("2011-01-20"), "description": "DEP SebasVeraS. ❤️", "followers_count": 844, "friends_count": 692, "statues_count": 18333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-17T00:00:00.000Z"), "id": 699865870139457537, "text": "@Krystal_Breche ������", "in_reply_to_status": 699864213162856448, "in_reply_to_user": 83007494, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 83007494 }}, "user": { "id": 416707745, "name": "CAROL", "screen_name": "YungCurrraline", "lang": "en", "location": "California, USA", "create_at": date("2011-11-19"), "description": "dare to be something different; dare to be something more", "followers_count": 1709, "friends_count": 697, "statues_count": 84902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865870357516288, "text": "@_The3rd @HellaKev right", "in_reply_to_status": 699865624286097410, "in_reply_to_user": 349537541, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 349537541, 75860429 }}, "user": { "id": 3319716120, "name": "A$AP MELRO$E", "screen_name": "ASAP_MELROSE", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-19"), "description": "young pretty jiggy bitchsnap:shesjmelrose ig:shesmelrose", "followers_count": 43, "friends_count": 102, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865870936440832, "text": "��✊�� https://t.co/DVFGLMH7CT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1711029218, "name": "LG. ", "screen_name": "MizzyWreck", "lang": "en", "location": "973 - hamptonUniversity ⚓️", "create_at": date("2013-08-29"), "description": "I'm prolly high, so don't take my tweets seriously", "followers_count": 1176, "friends_count": 942, "statues_count": 73130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865871196516352, "text": "02/17@03:00 - Temp 36.7F, WC 36.7F. Wind 1.4mph SW, Gust 3.0mph. Bar 29.883in, Falling slowly. Rain 0.00in. Hum 81%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865871741640704, "text": "so many ugly ass dick pics on my TL, y'all needa quit wit allat ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340573688, "name": "lil E ++®", "screen_name": "EricaTheGreat_", "lang": "en", "location": "in the clouds with sophs.", "create_at": date("2011-07-22"), "description": "puerto rican princess. // SV", "followers_count": 1642, "friends_count": 947, "statues_count": 96854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865871762767872, "text": "Wind 1.1 mph NE. Barometer 30.07 in, Rising. Temperature 48.2 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 20916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865871930363904, "text": "Wind 3.0 mph NNW. Barometer 1026.05 mb, Rising. Temperature 31.0 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 29, "friends_count": 125, "statues_count": 12584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872010055681, "text": "2am & I officially suck @ life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177060361, "name": "sam", "screen_name": "TheSamCicero", "lang": "en", "location": "pittstate '19", "create_at": date("2010-08-10"), "description": "snapchat: thesamcicero \noh hey hater, didn't see you there", "followers_count": 1245, "friends_count": 534, "statues_count": 15358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, KS", "id": "4ccacb29ffb3b3df", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-94.724428,37.367357 -94.668019,37.444178") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20037, "countyName": "Crawford", "cityID": 2056025, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872144461824, "text": "Check my channel on pornhub https://t.co/BKHp4KEPcS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2349747500, "name": "0mri Ca$hpi", "screen_name": "elchapohidayet", "lang": "en", "location": "Macramento", "create_at": date("2014-02-17"), "description": "Baby needs new shoes, Patnas need bail money #votewettters2k16 #riptb #ripdm", "followers_count": 326, "friends_count": 248, "statues_count": 11152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pikeville, KY", "id": "4be83795e2dd00fa", "name": "Pikeville", "place_type": "city", "bounding_box": rectangle("-82.569827,37.441155 -82.513243,37.510785") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21195, "countyName": "Pike", "cityID": 2160852, "cityName": "Pikeville" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872286904320, "text": "He said bring em by on Tuesdays between 5 & 5:10 they can eat all they want lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134365981, "name": "KingPetty400❗️", "screen_name": "CruzieMr301", "lang": "en", "location": "Dover DE -Kent MD ", "create_at": date("2010-04-17"), "description": "C-Stacc You Know They Goin Hate Just Don't Play No Part In That Shit FREE Trey,Nike,TAY,tray,Ace,Delz,LD,Ardy 400 WE THE REAL", "followers_count": 530, "friends_count": 254, "statues_count": 116206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872484032512, "text": "At least try to hit me up one last time before I move to Colorado.maybe.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2453493854, "name": "EviL AnGeL", "screen_name": "A420Evil", "lang": "en", "location": "Rio Rancho, NM", "create_at": date("2014-04-19"), "description": "Just your average girl", "followers_count": 807, "friends_count": 757, "statues_count": 512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Rancho, NM", "id": "0046bfef79c8e224", "name": "Rio Rancho", "place_type": "city", "bounding_box": rectangle("-106.757623,35.217658 -106.566425,35.374708") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35043, "countyName": "Sandoval", "cityID": 3563460, "cityName": "Rio Rancho" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872735690752, "text": "Add my sc @gayjasmin for random make up tutorials when I'm bored jaja I'm not da good but I will give u some tips https://t.co/kt2k1IiEQb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4192163187 }}, "user": { "id": 3070762849, "name": "(ノ◕ヮ◕)ノ*:・゚✧", "screen_name": "noxygen_", "lang": "en", "location": "San Jose, CA", "create_at": date("2015-03-09"), "description": "null", "followers_count": 136, "friends_count": 110, "statues_count": 2188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865872962187265, "text": "Sometimes we need new friends so we can get rid of the old ones who betrayed u or done you wrong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 879075662, "name": "Yeezus✝", "screen_name": "fonzy1_", "lang": "en", "location": "God ", "create_at": date("2012-10-13"), "description": "I feel like Pablo . 10/07 M❤️", "followers_count": 479, "friends_count": 484, "statues_count": 22087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cockrell Hill, TX", "id": "6a361e841bbda302", "name": "Cockrell Hill", "place_type": "city", "bounding_box": rectangle("-96.898463,32.730991 -96.878112,32.746963") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4815796, "cityName": "Cockrell Hill" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873507418112, "text": "Have to hide my white shoes cause my fam might just fuck around and step on em ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 602476736, "name": "Karrrii", "screen_name": "Karrrii___", "lang": "en", "location": "null", "create_at": date("2012-06-07"), "description": "null", "followers_count": 421, "friends_count": 319, "statues_count": 12579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873570369537, "text": "@beatlesjad @joehos18 You will care when dollar Crashes like Argentina 2001, & IMF says, We can help but you must do what we say > screwed", "in_reply_to_status": 699822607282085889, "in_reply_to_user": 2665575199, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2665575199, 700286457 }}, "user": { "id": 10207502, "name": "Don Mashak", "screen_name": "DMashak", "lang": "en", "location": "Minneapolis, Minnesota USA", "create_at": date("2007-11-12"), "description": "Don Mashak,The Cynical Patriot #NaturalLaw #Constitution @WETHEPEOPLETAR @RLibertyCaucus #MILITARY #Militia #TEAParty #OWS @GOP @TheDemocrats See UserGuide Link", "followers_count": 86594, "friends_count": 93568, "statues_count": 318506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873578852352, "text": "Barberton Oh Temp:27.2°F Wind:0 mph Dir:--- Baro:Rising slowly Rain2day:0.00in Hum:86% UV:0.0 @ 03:00 02/17/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873620688896, "text": "The life of Pablo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2955370135, "name": "JUMPERMAN", "screen_name": "__siul", "lang": "en", "location": "null", "create_at": date("2015-01-01"), "description": "17. ⚽️ what doesn't kill you. fucks you up, mentally ➰", "followers_count": 351, "friends_count": 341, "statues_count": 6878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873805398017, "text": "happy birfday to my cousin, my best friend, and my role model! We've done so much together and… https://t.co/gjMXkfpFgD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.682,45.52"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386262540, "name": "Denni Chiou", "screen_name": "DenniFourSeven", "lang": "en", "location": "null", "create_at": date("2011-10-06"), "description": "Life is not about waiting for the storm to pass, it's about learning to dance in the rain~", "followers_count": 62, "friends_count": 26, "statues_count": 133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873910206464, "text": "JEALOUS AF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 413376629, "name": "Kendrick Lamar", "screen_name": "SoleFlexer", "lang": "en", "location": "Trévone", "create_at": date("2011-11-15"), "description": "I don't drink too much, I know the bud wiser #KithSet#JamaicanIndian#NC19' I'm the one with the camera", "followers_count": 584, "friends_count": 431, "statues_count": 43091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookline, MA", "id": "13f1b1eadfb47f9b", "name": "Brookline", "place_type": "city", "bounding_box": rectangle("-71.178586,42.29466 -71.105725,42.351818") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2509210, "cityName": "Brookline" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873939456000, "text": "temperature down 61°F -> 56°F\nhumidity up 62% -> 72%\nwind 3mph -> 0mph\npressure 29.69in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.39261,40.58759"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120934422, "name": "Redding Weather", "screen_name": "_ReddingCA", "lang": "en", "location": "Redding, CA", "create_at": date("2010-03-07"), "description": "Weather updates, forecast, warnings and information for Redding, CA. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 176, "friends_count": 4, "statues_count": 25388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865873998180352, "text": "So are you going to hit me with a reply?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193989995, "name": "maggs", "screen_name": "bbymaggie_", "lang": "en", "location": "Cali ", "create_at": date("2010-09-22"), "description": "my heart is made of gold nd Luis", "followers_count": 729, "friends_count": 562, "statues_count": 25602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jacinto, CA", "id": "0068bc1a061e33c3", "name": "San Jacinto", "place_type": "city", "bounding_box": rectangle("-117.039483,33.751357 -116.919422,33.818476") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 667112, "cityName": "San Jacinto" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865874023317505, "text": "I Mean I always get what I want��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3071101275, "name": "3/27♋️", "screen_name": "latyralashai_1", "lang": "en", "location": "Jaquan & Monnie ❣", "create_at": date("2015-03-04"), "description": "null", "followers_count": 932, "friends_count": 648, "statues_count": 62709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865874061250560, "text": "Wind 1.0 mph NW. Barometer 30.148 in, Rising. Temperature 37.2 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-17T00:00:01.000Z"), "id": 699865874325381124, "text": "Nothing ventured, nothing gained .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33734529, "name": "sugasugahh", "screen_name": "xLust4Jessx", "lang": "en", "location": "orange county, ca", "create_at": date("2009-04-20"), "description": "Angelic as hell || XOTWOD", "followers_count": 215, "friends_count": 206, "statues_count": 23620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865874530893824, "text": "@ebbtideapp Tide in Thunderbolt, Georgia 02/17/2016\nHigh 3:51am 8.1\n Low 10:13am 0.4\nHigh 4:24pm 7.2\n Low 10:28pm -0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.05,32.0333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 29, "friends_count": 1, "statues_count": 8438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865874644140032, "text": "I'm already looking for a bed and some furniture ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2511348793, "name": "Michelle Cruz", "screen_name": "Michellesue22", "lang": "en", "location": "null", "create_at": date("2014-05-20"), "description": "null", "followers_count": 465, "friends_count": 329, "statues_count": 7313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865874942054400, "text": "@taya1a1 could you stop mentioning me wtf", "in_reply_to_status": 699860553888624640, "in_reply_to_user": 249551071, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 249551071 }}, "user": { "id": 1619843210, "name": "LIL TAY❤️", "screen_name": "Theylovetayea", "lang": "en", "location": "At Avi Crib", "create_at": date("2013-07-25"), "description": "getting to it", "followers_count": 1836, "friends_count": 676, "statues_count": 40639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenwood, IL", "id": "90c2fa83ca7cbd80", "name": "Glenwood", "place_type": "city", "bounding_box": rectangle("-87.641135,41.520486 -87.53928,41.58285") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1730029, "cityName": "Glenwood" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865875650711552, "text": "My wallpaper ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1865166018, "name": "NeekWidTha4️⃣", "screen_name": "neek4hunnid", "lang": "en", "location": "THU WORLD ❤️✨", "create_at": date("2013-09-14"), "description": "Snapchat : AintYouNeek4 ➰", "followers_count": 563, "friends_count": 488, "statues_count": 8147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865875738800128, "text": "@Yaakov_Shmuel Free will wherever you may be is a commodity of a line, don't you think", "in_reply_to_status": 699862677179355136, "in_reply_to_user": 2926196144, "favorite_count": 0, "coordinate": point("-113.56469374,37.11013005"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2926196144 }}, "user": { "id": 2615326084, "name": "Judicius", "screen_name": "numbers110", "lang": "en", "location": "Between Sodom and Gomorrah", "create_at": date("2014-06-16"), "description": "RT/Fav do not equal endorsement", "followers_count": 333, "friends_count": 284, "statues_count": 23794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St George, UT", "id": "72ac303c0e608f71", "name": "St George", "place_type": "city", "bounding_box": rectangle("-113.641805,37.021646 -113.513726,37.149517") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49053, "countyName": "Washington", "cityID": 4965330, "cityName": "St. George" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876074344449, "text": "52.9F (Feels: 52.9F) - Humidity: 99% - Wind: 4.5mph W - Gust: 4.5mph - Pressure: 1029.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 227163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876183576576, "text": "I'm at Cigar Train in Spokane, WA https://t.co/uY0TFOlHEW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.426131,47.716568"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270711836, "name": "Cigar Train", "screen_name": "CigarTrain", "lang": "en", "location": "Spokane Washington", "create_at": date("2011-03-22"), "description": "A local B&M servicing Spokane Washington for all of your traditional cigar and cigar accessory needs with an honest and educated approach to customer service!", "followers_count": 770, "friends_count": 516, "statues_count": 9684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Town and Country, WA", "id": "0ab96e4d1cbd1e36", "name": "Town and Country", "place_type": "city", "bounding_box": rectangle("-117.434539,47.715103 -117.411294,47.740887") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5372170, "cityName": "Town and Country" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876523196417, "text": "weed w/ god. https://t.co/pdLodrV3hX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3029209421, "name": "robert patrick", "screen_name": "bdsixxx", "lang": "en", "location": "null", "create_at": date("2015-02-10"), "description": "ig: bdsix", "followers_count": 113, "friends_count": 66, "statues_count": 200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876833660928, "text": "Temp: 51.1°F Wind:0.0mph Pressure: 30.076hpa Rising Rapidly Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876837900289, "text": "Wind 0.0 mph ---. Barometer 30.068 in, Rising. Temperature 30.6 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 53, "friends_count": 27, "statues_count": 17703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865876854546432, "text": "hella confused", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1298050796, "name": "Paulinaa.S", "screen_name": "Pauline_wennie", "lang": "en", "location": "Corona, CA", "create_at": date("2013-03-25"), "description": "about one about things", "followers_count": 327, "friends_count": 279, "statues_count": 15009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865877458591744, "text": "Permuto hernias de disco por motomel 110", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 583074819, "name": "ya boy shoaquin", "screen_name": "Joaquincurien1", "lang": "es", "location": "Ciudad Autónoma de Buenos Aires, Argentina", "create_at": date("2012-05-17"), "description": "barba mata galán", "followers_count": 458, "friends_count": 346, "statues_count": 26019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865877613715457, "text": "Since I can't sleep, I might as well go through my clothes for what still fits. Over half my closet is too small now ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.92110531,40.1981982"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 295204495, "name": ".", "screen_name": "queenlucifina", "lang": "en", "location": "PITTSBURGH ", "create_at": date("2011-05-08"), "description": "21 / little love wolf", "followers_count": 4850, "friends_count": 2780, "statues_count": 71502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monongahela, PA", "id": "f06699d8f1e1e21c", "name": "Monongahela", "place_type": "city", "bounding_box": rectangle("-79.944919,40.182825 -79.900009,40.208679") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4250408, "cityName": "Monongahela" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865877617881088, "text": "@Gennnnnnnna lmao this one is funny to! And hahah you should �� https://t.co/sAqwTf3pKx", "in_reply_to_status": 699865329497821188, "in_reply_to_user": 173545496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 173545496 }}, "user": { "id": 263467723, "name": "yaya", "screen_name": "_yayaliciouss", "lang": "en", "location": "209", "create_at": date("2011-03-09"), "description": "SC/ yaya_liciouss", "followers_count": 907, "friends_count": 309, "statues_count": 16536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865878045790208, "text": "���� @ohheyjoshay @ryaninthecity @Bpougnet @j_pougnet3 https://t.co/k2jyxrIKY6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 278790428, 29802291, 24248825, 541095457 }}, "user": { "id": 25209065, "name": "Tess Theisen", "screen_name": "neutrals4bfast", "lang": "en", "location": "Chicago ", "create_at": date("2009-03-18"), "description": "Instagram: @neutralsforbreakfast Snapchat: tesstheisen http://www.neutralsforbreakfast.com neutralsforbreakfast@gmail.com", "followers_count": 674, "friends_count": 789, "statues_count": 11102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-17T00:00:02.000Z"), "id": 699865878544957440, "text": "Dont know why I take nights in rhe middle of the night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226034553, "name": "white iverson", "screen_name": "rockydavids_1", "lang": "en", "location": "Greenwich Village, Manhattan", "create_at": date("2015-05-01"), "description": "MADE IN HEAVEN", "followers_count": 188, "friends_count": 196, "statues_count": 7557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865878758715392, "text": "@BellaCurlsss and the way we roll just a rebel looking for a place to go ����", "in_reply_to_status": 699864770816532480, "in_reply_to_user": 875076002, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 875076002 }}, "user": { "id": 1273149343, "name": "jake m", "screen_name": "JakeMolchan", "lang": "en", "location": "South Bay ⚓️ City of Angels ", "create_at": date("2013-03-16"), "description": "Just do'in me", "followers_count": 611, "friends_count": 650, "statues_count": 6240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865878767120384, "text": "I missed going to the gym. Glad to be back", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938646587, "name": "Amara Watson", "screen_name": "ammarra_", "lang": "en", "location": "null", "create_at": date("2014-12-21"), "description": "IG: ammarra_", "followers_count": 117, "friends_count": 173, "statues_count": 1515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865878955819012, "text": "Ew I'm 21 now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 463641140, "name": "Elyssa Dinogan", "screen_name": "Ohlyssaah", "lang": "en", "location": "Nevada, USA", "create_at": date("2012-01-14"), "description": "Quiet & hungry 24/7.", "followers_count": 214, "friends_count": 173, "statues_count": 4354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879069077504, "text": "@MirandaMonaco Buenas noches para usted y que descanse Saludos", "in_reply_to_status": 699808318500446208, "in_reply_to_user": 1521360410, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1521360410 }}, "user": { "id": 2951294724, "name": "Francisco Estrada", "screen_name": "Francis38490069", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 663, "friends_count": 536, "statues_count": 16411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879085977601, "text": "@TaayHaay we both ordered @dominos and ate it in bed tonight #dreamscometrue", "in_reply_to_status": 699840298248228864, "in_reply_to_user": 301141706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dreamscometrue" }}, "user_mentions": {{ 301141706, 31444922 }}, "user": { "id": 92827402, "name": "Sandra Hedrick", "screen_name": "_sandy172", "lang": "en", "location": "Clemson, SC", "create_at": date("2009-11-26"), "description": "Addicted to the South, loving people, travel, Clemson, @newspring, & sports. I'm good at laughing & drinking coffee. Striving to spread joy & make a difference.", "followers_count": 720, "friends_count": 584, "statues_count": 9956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879304011776, "text": "@KyleCorn2 how", "in_reply_to_status": 699865757820190720, "in_reply_to_user": 3093791513, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3093791513 }}, "user": { "id": 126736146, "name": "Sky :)", "screen_name": "SKYLAR_RYANN", "lang": "en", "location": "null", "create_at": date("2010-03-26"), "description": "Heaven's little devil \nSHSU 2019", "followers_count": 231, "friends_count": 383, "statues_count": 8361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879341699072, "text": "Wind 2.0 mph N. Barometer 30.121 in, Rising slowly. Temperature 30.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879345901568, "text": "Hot damn, RWBY season 3 is a feels train ;w;", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17411373, "name": "berwyn", "screen_name": "_berwyn_", "lang": "en-gb", "location": "South Jordan, UT", "create_at": date("2008-11-15"), "description": "I like smallhorses, and beat my face against JavaScript, Android, and the occasional Go app. Working on @Chatspry, @PonyvilleLive, & @BronyTunes", "followers_count": 194, "friends_count": 482, "statues_count": 3615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandy, UT", "id": "fb9549afa6f81fa6", "name": "Sandy", "place_type": "city", "bounding_box": rectangle("-111.921658,40.528084 -111.800273,40.616827") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967440, "cityName": "Sandy" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865879576596480, "text": "not even gonna cal cuz I know im not gonna get an answer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2737956537, "name": "sb", "screen_name": "therealsadityb", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "null", "followers_count": 728, "friends_count": 584, "statues_count": 1537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865880579088384, "text": "@dumbrudy_ thank you so much for the support ������", "in_reply_to_status": 699865507147550721, "in_reply_to_user": 533704514, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 533704514 }}, "user": { "id": 335830903, "name": "OLGABEAR", "screen_name": "OlgaBear_", "lang": "en", "location": "null", "create_at": date("2011-07-15"), "description": "when you find the secret you'll understand", "followers_count": 1388, "friends_count": 1112, "statues_count": 64866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865880675622912, "text": "Wind 0.7 mph N. Barometer 29.84 in, Rising. Temperature 33.1 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 111, "statues_count": 157580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865880738463744, "text": "Happy Birthday Brianna!! Have a great day! See you this weekend�� https://t.co/zzUVKT4PhF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1370915761, "name": "DOBBS", "screen_name": "Dobbertin97", "lang": "en", "location": "Sacramento, CA", "create_at": date("2013-04-21"), "description": "Sac State • Al❤️ • SJ Sharks", "followers_count": 408, "friends_count": 184, "statues_count": 7073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865881103376384, "text": "happy birthday @SkooboyCantRap beyond grateful for our friendship. you know what it is. much love always ���� https://t.co/47sNgvTx5Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68548295 }}, "user": { "id": 1639069280, "name": "mayraaaa", "screen_name": "Mayyravazquez", "lang": "en", "location": "null", "create_at": date("2013-08-01"), "description": "blessed. happy. gemini.", "followers_count": 530, "friends_count": 175, "statues_count": 35955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865881484996608, "text": "I'm feelin really disturbed rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162522635, "name": "Kathy Tran", "screen_name": "kathykashh", "lang": "en", "location": "Killafornia", "create_at": date("2010-07-03"), "description": "sad but rad ( ⚈̥̥̥̥̥́⌢⚈̥̥̥̥̥̀)", "followers_count": 497, "friends_count": 287, "statues_count": 46377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865881673908224, "text": "Gn (@ Meridian Place in Northridge, CA) https://t.co/71IOhJW7Ly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 371, "friends_count": 337, "statues_count": 25577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865881862471681, "text": "https://t.co/GaftzHV6kA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2457731790, "name": "NeptunoⓂ️", "screen_name": "neptuno30", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-04-21"), "description": "One God Only", "followers_count": 5026, "friends_count": 5513, "statues_count": 129589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheridan, CO", "id": "3cfce73ef3a505af", "name": "Sheridan", "place_type": "city", "bounding_box": rectangle("-105.034755,39.631416 -104.999537,39.660504") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 869645, "cityName": "Sheridan" } }
+{ "create_at": datetime("2016-02-17T00:00:03.000Z"), "id": 699865882688884736, "text": "mood�� https://t.co/EeDWBDHJ11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2472881891, "name": "ㅤ", "screen_name": "1Hass42", "lang": "en", "location": "trenches wit a glock 26 on me ", "create_at": date("2014-04-09"), "description": "null", "followers_count": 948, "friends_count": 879, "statues_count": 11125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883003346945, "text": "@Dunkin_dipshits wottttt ok am done its the tu one right", "in_reply_to_status": 699865713108881408, "in_reply_to_user": 373065009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373065009 }}, "user": { "id": 2222162232, "name": "Pancha", "screen_name": "LaPancheese", "lang": "en", "location": "Michoacoachella", "create_at": date("2013-11-29"), "description": "she/her/hers ✨\nNepantlera ✨\nReal. Raw. Rural.", "followers_count": 498, "friends_count": 619, "statues_count": 10491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883208847360, "text": "Come up. Heartless. Wake up. Thoughtless.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137805052, "name": "Wes Willy #SWED", "screen_name": "JMosesCastaneda", "lang": "en", "location": "Sherman, Texas", "create_at": date("2010-04-27"), "description": "Team Virgin Been at it since '94 I got five. SC: Moisescas94 BLN, PB, UglySquad IF IT AINT WHITE IT AINT RIGHT", "followers_count": 601, "friends_count": 614, "statues_count": 38285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherman, TX", "id": "caea073452d66ac0", "name": "Sherman", "place_type": "city", "bounding_box": rectangle("-96.66317,33.57578 -96.548448,33.70441") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48181, "countyName": "Grayson", "cityID": 4867496, "cityName": "Sherman" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883406168064, "text": "Wind 2.7 mph WNW. Barometer 29.811 in, Rising Rapidly. Temperature 28.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883443752960, "text": "@CAMLaPistolaa I'm good. Tryna find sleep", "in_reply_to_status": 699865817568030720, "in_reply_to_user": 1180875024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1180875024 }}, "user": { "id": 81954897, "name": "Beige the Gemini", "screen_name": "SilasTheVifth", "lang": "en", "location": "Chicago, AZ", "create_at": date("2009-10-12"), "description": "Somewhere between a logical writer and a madman with a pen.", "followers_count": 2375, "friends_count": 900, "statues_count": 142684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883540332546, "text": "���������� https://t.co/I5NYUpVwQR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2472811834, "name": "hunch✂️", "screen_name": "BigGuwopi", "lang": "en", "location": "null", "create_at": date("2014-04-09"), "description": "301", "followers_count": 3897, "friends_count": 2260, "statues_count": 5361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princess Anne, MD", "id": "0098bec2526560be", "name": "Princess Anne", "place_type": "city", "bounding_box": rectangle("-75.715802,38.152156 -75.668895,38.236042") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24039, "countyName": "Somerset", "cityID": 2464000, "cityName": "Princess Anne" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883791986688, "text": "Happy birthday @ryyandes I don't want another pretty face\nI don't want just anyone to hold����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339495024 }}, "user": { "id": 1965479563, "name": "John", "screen_name": "JohnNoAthon", "lang": "en", "location": "in a van down by the river", "create_at": date("2013-10-16"), "description": "Still not frat, and still plays with his Dino nuggets before eating them. #DadsOfAmericaGang - #THGH", "followers_count": 1414, "friends_count": 1121, "statues_count": 62578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johnson City, TN", "id": "4185eba3c7183f0b", "name": "Johnson City", "place_type": "city", "bounding_box": rectangle("-82.475906,36.259223 -82.300523,36.430258") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47179, "countyName": "Washington", "cityID": 4738320, "cityName": "Johnson City" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865883997417472, "text": "#Dubya is the smart one.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dubya" }}, "user": { "id": 291421323, "name": "arman walker", "screen_name": "armanwalker", "lang": "en", "location": "Los Angeles, California", "create_at": date("2011-05-01"), "description": "Economist gone main street.", "followers_count": 14019, "friends_count": 15307, "statues_count": 137145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865884228132864, "text": "if I sleep now I'll only have an hour & a half of sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 127449063, "name": "aivy van", "screen_name": "daayumaivy", "lang": "en", "location": "null", "create_at": date("2010-03-28"), "description": "null", "followers_count": 98, "friends_count": 81, "statues_count": 13814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865884416827393, "text": "@princesschryme mmk", "in_reply_to_status": 699865845590167552, "in_reply_to_user": 2836845850, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2836845850 }}, "user": { "id": 518173745, "name": "Hug Me", "screen_name": "its_so_soft", "lang": "en", "location": "null", "create_at": date("2012-03-07"), "description": "gimme hug!", "followers_count": 57, "friends_count": 123, "statues_count": 8461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lone Tree, CO", "id": "203838c6eac8a66a", "name": "Lone Tree", "place_type": "city", "bounding_box": rectangle("-104.906552,39.506906 -104.866751,39.566504") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 845955, "cityName": "Lone Tree" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865884492312576, "text": "That feeling when you jump on your bed after being gone a couple days or so", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160048087, "name": "Justin Kyle™", "screen_name": "The_JustinKyle", "lang": "en", "location": "Orange County", "create_at": date("2010-06-26"), "description": "This is my canvas", "followers_count": 137, "friends_count": 184, "statues_count": 6903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865884895088640, "text": "Travelling to Bethesda or just twittering about Bethesda? https://t.co/lXaBW1IVme #Bethesda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1003,38.9807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bethesda" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2600255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethesda, MD", "id": "864ff125241f172f", "name": "Bethesda", "place_type": "city", "bounding_box": rectangle("-77.158594,38.940225 -77.078411,39.022449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865886434308096, "text": "Can you find Bethesda on the map? Just try it at https://t.co/lXaBW1rjXE #Bethesda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1003,38.9807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bethesda" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2600259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethesda, MD", "id": "864ff125241f172f", "name": "Bethesda", "place_type": "city", "bounding_box": rectangle("-77.158594,38.940225 -77.078411,39.022449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865886761443328, "text": "Temp: 25.0°F - Dew Point: 20.3° - Wind: 2.7 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 30.00in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 13565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865886765649920, "text": "@thelucas_1 stop ur making me miss glee", "in_reply_to_status": -1, "in_reply_to_user": 1949560430, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1949560430 }}, "user": { "id": 107294025, "name": "kace", "screen_name": "kaceyannell", "lang": "en", "location": "lone star state", "create_at": date("2010-01-21"), "description": "she found herself, and somehow, that was everything | ΑΞΔ | cat person", "followers_count": 334, "friends_count": 503, "statues_count": 4405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-17T00:00:04.000Z"), "id": 699865886904053761, "text": "Always stay humble and kind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454198092, "name": "chloe krause", "screen_name": "chloekrause", "lang": "en", "location": "null", "create_at": date("2012-01-03"), "description": "not interested {}cstat{}", "followers_count": 1648, "friends_count": 757, "statues_count": 18534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865887055065088, "text": "No morning class tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179219509, "name": "Beto...", "screen_name": "AyyYooBetito", "lang": "en", "location": "null", "create_at": date("2010-08-16"), "description": "null", "followers_count": 420, "friends_count": 480, "statues_count": 13461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865887436722176, "text": "It was just like a movie it was just like a song...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1000088887, "name": "Naners", "screen_name": "Leeana_rae", "lang": "en", "location": "Azusa, CA", "create_at": date("2012-12-09"), "description": "1 Corinthians 1:9 ❤️", "followers_count": 579, "friends_count": 536, "statues_count": 30515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865888053403648, "text": "@ErikahhhGeee happy birthday mom ����18 already wow and I just met u yesterday and u made me feel great with all the words u say.keep rant", "in_reply_to_status": -1, "in_reply_to_user": 196877246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 196877246 }}, "user": { "id": 4057063215, "name": "Emily", "screen_name": "ItsEmiD", "lang": "en", "location": "10/17/03", "create_at": date("2015-10-27"), "description": "erika is my mom and she's turning 18 tmr dam ma you growing now u can adopt me", "followers_count": 36, "friends_count": 69, "statues_count": 48 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Chester, NY", "id": "0d39e8bde2c2cb6b", "name": "Port Chester", "place_type": "city", "bounding_box": rectangle("-73.679712,40.986901 -73.65516,41.025584") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3659223, "cityName": "Port Chester" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865888162344961, "text": "Wind 5.2 mph WNW. Barometer 29.897 in, Rising Rapidly. Temperature 28.6 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 7, "friends_count": 43, "statues_count": 6576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865888296538112, "text": "Wind 1.0 mph W. Barometer 29.988 in, Rising. Temperature 28.8 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 380, "friends_count": 289, "statues_count": 8332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865888376225792, "text": "Happy 20th birthday to @ChiefTreyvon! My besty, homie, partner in crime. Hope your day is as amazing as you are. Love you lots. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321718119 }}, "user": { "id": 47034683, "name": "michelle", "screen_name": "michellebellle", "lang": "en", "location": "Tacoma, Washington. ↟", "create_at": date("2009-06-13"), "description": "twenty. aquarius. dog mom. pnw. @chieftreyvon", "followers_count": 1461, "friends_count": 1250, "statues_count": 29754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865888506454016, "text": "I'm still so lost \nWith a ratio like how do you have full classrooms failing??? https://t.co/MO4Wx1Ei87", "in_reply_to_status": 699863220090228736, "in_reply_to_user": 189614954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189614954, "name": "conscious trap queen", "screen_name": "alexisisbestest", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-09-11"), "description": "butterscotch babe | a dandelion among roses | a gourmet potato | Howard University '18", "followers_count": 1713, "friends_count": 962, "statues_count": 48560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "View Park-Windsor Hills, CA", "id": "3f5aa7a2b7beec37", "name": "View Park-Windsor Hills", "place_type": "city", "bounding_box": rectangle("-118.365715,33.982756 -118.331397,34.008397") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682667, "cityName": "View Park-Windsor Hills" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865889185906688, "text": "@VinnyBlazin it is but I could make it whenever j want ��", "in_reply_to_status": 699865002459721728, "in_reply_to_user": 1056783637, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1056783637 }}, "user": { "id": 79526319, "name": "Jess", "screen_name": "halfbakedxo", "lang": "en", "location": "New York", "create_at": date("2009-10-03"), "description": "wavy.", "followers_count": 1594, "friends_count": 487, "statues_count": 20840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865889676505091, "text": "What people in hs still don't get is after it's over, no one cares about who did what, etc etc. everyone focuses on themselves now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 495676859, "name": "ashleysosweet", "screen_name": "AshleySoSweeet", "lang": "en", "location": "null", "create_at": date("2012-02-17"), "description": "null", "followers_count": 646, "friends_count": 523, "statues_count": 15189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865889894563840, "text": "need to quit falling asleep with makeup on bc my acne is so awful now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1199747126, "name": "kanye asada", "screen_name": "janesa_xo", "lang": "en", "location": "719", "create_at": date("2013-02-19"), "description": "janesa d'nee ⛅️\nfuck what you heard ((but it probably happened))", "followers_count": 2152, "friends_count": 1559, "statues_count": 69651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain, CO", "id": "179da553bdfd76d6", "name": "Fountain", "place_type": "city", "bounding_box": rectangle("-104.747972,38.663766 -104.638373,38.74665") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 827865, "cityName": "Fountain" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865890469212160, "text": "Hungryyyy��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487186945, "name": "kc", "screen_name": "elegvnt", "lang": "en", "location": "Dallas // Houston", "create_at": date("2012-02-08"), "description": "Finance Major. University of Houston. #longliveCj", "followers_count": 1688, "friends_count": 968, "statues_count": 12628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865890473537538, "text": "@atomicBombPOW what's your snap", "in_reply_to_status": 699853247146622976, "in_reply_to_user": 236200269, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236200269 }}, "user": { "id": 109441293, "name": "Zaddy Bear", "screen_name": "_thenewsupreme", "lang": "en", "location": "Orlando Florida", "create_at": date("2010-01-28"), "description": "I don't ask for much,just please don't get on my damn nerves . Sagittarius . snapchat : callme_chy", "followers_count": 359, "friends_count": 96, "statues_count": 5140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-17T00:00:05.000Z"), "id": 699865891148726272, "text": "Happy Birthday FEO I hope you have an amazing birthday thank you for always being there @AllenSarinana_ https://t.co/dPXzA6gDtS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 443273678 }}, "user": { "id": 706708831, "name": "Janeth Nunez_lol", "screen_name": "Janeth_3C", "lang": "en", "location": "San Diego,CA", "create_at": date("2012-07-20"), "description": "MARCH 29/JUNE 14/JULY 2 /OCTOBER 10||3C||#FrayLumpaMob || Alex Aiono followed 11-6-15||", "followers_count": 133, "friends_count": 190, "statues_count": 3581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865891303874561, "text": "@minnniem0use I agree with this so much hahhaha but more so about Britney today than myself hahhaa", "in_reply_to_status": 699820998615552000, "in_reply_to_user": 41243653, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41243653 }}, "user": { "id": 118851968, "name": "Kenzie McCombs Doyle", "screen_name": "KenzDoyle", "lang": "en", "location": "null", "create_at": date("2010-03-01"), "description": "los angeles", "followers_count": 265, "friends_count": 100, "statues_count": 11555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865893371686913, "text": "I need a lady that'll tell me you ain't gone hit this till you hit the gym", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 584356139, "name": "Monje", "screen_name": "Mapler_Monje", "lang": "en", "location": "Indio, CA", "create_at": date("2012-05-18"), "description": "Take the L for now.... Pinches Mamadas", "followers_count": 875, "friends_count": 873, "statues_count": 43671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865893489217536, "text": "@LaBrandoJames typical nigga avi", "in_reply_to_status": -1, "in_reply_to_user": 2727922517, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2727922517 }}, "user": { "id": 4003844842, "name": "yonce", "screen_name": "keishana_", "lang": "en", "location": "Concord, NC", "create_at": date("2015-10-20"), "description": "#UNCG19 Insta: @keishana._ sc: awesomekei and it's pronounced key•shawna", "followers_count": 402, "friends_count": 410, "statues_count": 4220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865893812056065, "text": "Ohhhhhhh ma lord ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93954516, "name": "Najah", "screen_name": "champagnenajah", "lang": "en", "location": "sjsu", "create_at": date("2009-12-01"), "description": "LA mentality, Bay Area personality.", "followers_count": 615, "friends_count": 522, "statues_count": 14411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cupertino, CA", "id": "36237ab3643ff2be", "name": "Cupertino", "place_type": "city", "bounding_box": rectangle("-122.091151,37.285786 -121.995595,37.34038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 617610, "cityName": "Cupertino" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865894063775745, "text": "I'm gonna start keeping paper maps in my car just to mess with passengers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 94256432, "name": "Rob", "screen_name": "robcas65", "lang": "en", "location": "null", "create_at": date("2009-12-02"), "description": "-The stories and information posted here are artistic works of fiction and falsehood.-", "followers_count": 277, "friends_count": 318, "statues_count": 12443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865894302851073, "text": "Bruh @arsvlan I've never laughed so hard in my entire life homie ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4027617553 }}, "user": { "id": 3017524447, "name": "INSAENZ.34.", "screen_name": "josephsaenz34", "lang": "en", "location": "on the mound", "create_at": date("2015-02-12"), "description": "OUTLAWS 34 | JUNIOR AT THS | Catch me on the field this year shining bright and striking batters out", "followers_count": 480, "friends_count": 891, "statues_count": 1329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865895112306688, "text": "This color������ https://t.co/LUuekFTh0x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195931412, "name": "Natalie Lael", "screen_name": "natalielarrabee", "lang": "en", "location": "Tri. WA", "create_at": date("2010-09-27"), "description": "just living life and making coffee's☀️☕️ @CdubTNK is the one❤️", "followers_count": 565, "friends_count": 583, "statues_count": 4828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Pasco, WA", "id": "473952fe2ce5ebda", "name": "West Pasco", "place_type": "city", "bounding_box": rectangle("-119.263065,46.232339 -119.137555,46.278267") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53021, "countyName": "Franklin", "cityID": 5377612, "cityName": "West Pasco" } }
+{ "create_at": datetime("2016-02-17T00:00:06.000Z"), "id": 699865895217156097, "text": "@UberFacts @JoffeJonah", "in_reply_to_status": 699855139570638849, "in_reply_to_user": 95023423, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 95023423, 886055396 }}, "user": { "id": 988037358, "name": "Easton Jones", "screen_name": "eastjones430", "lang": "en", "location": "Phoenix | Arizona", "create_at": date("2012-12-03"), "description": "USMC || Est. 1775", "followers_count": 745, "friends_count": 675, "statues_count": 4018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-02-22T13:56:08.000Z"), "id": 701888225913208833, "text": "Only reason I don't care about little things is because I got bigger better things going on LOL enjoy your boring lives", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3243990601, "name": "andrew lowkzzz", "screen_name": "andrewlowkzz", "lang": "en", "location": "West Whittier, CA", "create_at": date("2015-06-12"), "description": "tomorrow is not promised", "followers_count": 249, "friends_count": 516, "statues_count": 3801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888620639166464, "text": "@briellewestwood no", "in_reply_to_status": 701880767606091776, "in_reply_to_user": 754157582, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 754157582 }}, "user": { "id": 785199306, "name": "Jimmy", "screen_name": "SampeyJimmy", "lang": "en", "location": "Indian Rocks Beach, Florida", "create_at": date("2012-08-27"), "description": "life is lit", "followers_count": 321, "friends_count": 286, "statues_count": 512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indian Rocks Beach, FL", "id": "07ea8d4bc3d8ec34", "name": "Indian Rocks Beach", "place_type": "city", "bounding_box": rectangle("-82.851954,27.872243 -82.834864,27.909882") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1233625, "cityName": "Indian Rocks Beach" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888620781662211, "text": "�������� https://t.co/UrbVW5uVm5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2495648268, "name": "Belinda", "screen_name": "aaadnileb", "lang": "en", "location": "null", "create_at": date("2014-05-14"), "description": "null", "followers_count": 452, "friends_count": 280, "statues_count": 14799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888621062844416, "text": "Cheez-It apparently thinks more of Arkansas and Tennessee than the rest of the college basketball world. https://t.co/lAOgIREUGc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464854825, "name": "John Wilmhoff", "screen_name": "JohnWilmhoff", "lang": "en", "location": "null", "create_at": date("2012-01-15"), "description": "Former beer vendor, EKU mascot and KSR writer. Current personalization writer/content creator for the WWLIS. Opinions are my own.", "followers_count": 861, "friends_count": 1590, "statues_count": 19109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southgate, KY", "id": "f89ed4fa5b53c195", "name": "Southgate", "place_type": "city", "bounding_box": rectangle("-84.485303,39.05026 -84.454728,39.078224") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21037, "countyName": "Campbell", "cityID": 2171976, "cityName": "Southgate" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888621578579968, "text": "They BOTH actually stayed still for a pic with our thank you note for sponsoring Vixen #Shocked #Proud https://t.co/gbzKDkUWZp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Shocked", "Proud" }}, "user": { "id": 236639258, "name": "samantha reeves", "screen_name": "auntie6127", "lang": "en", "location": "oklahoma", "create_at": date("2011-01-10"), "description": "♡ When Words Fail Music Speaks ♡ Follow the link to check out my music blog An E.A.R. for Music", "followers_count": 179, "friends_count": 341, "statues_count": 3451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888621914234884, "text": "For sure @Tsteezy23 ���� https://t.co/75ym5Sii6d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3298790576 }}, "user": { "id": 3974491573, "name": "Kirubel ॐ", "screen_name": "Kirubel_ab17", "lang": "en", "location": "null", "create_at": date("2015-10-21"), "description": "family", "followers_count": 81, "friends_count": 89, "statues_count": 809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888622451159040, "text": "yes. @JideTheCool", "in_reply_to_status": 701887219091439616, "in_reply_to_user": 75198547, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 75198547 }}, "user": { "id": 328303498, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤㅤ ‏", "screen_name": "llerameilla", "lang": "en", "location": "akron, oh", "create_at": date("2011-07-02"), "description": "poised. fashion merchandising major. makeup and fashion enthusiast. 5'9 and fine.", "followers_count": 3563, "friends_count": 3426, "statues_count": 115242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888622509826048, "text": "Bidin with manny be to funny ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4779164182, "name": ".", "screen_name": "1MooxMoo", "lang": "en", "location": "null", "create_at": date("2016-01-11"), "description": "null", "followers_count": 82, "friends_count": 63, "statues_count": 288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888622958551040, "text": "Lmaooo. This exact thing happened last time I went. https://t.co/2KNF1UShZm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 109360451, "name": "PARI$ RICH", "screen_name": "TreexGod", "lang": "en", "location": "Chicago✈️New York✈️Denver", "create_at": date("2010-01-28"), "description": "#StopHarassingMe2k16 Wilhemina Denver/Americana Models NY.", "followers_count": 1545, "friends_count": 433, "statues_count": 115417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888623340351488, "text": "Get your tickets for the @WachusettBrew #craftbeer dinner @CambCommon on March 8th! Info/tix here: https://t.co/ZUzGV7cDgm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "craftbeer" }}, "user_mentions": {{ 20263081, 94371352 }}, "user": { "id": 260289630, "name": "Craft Brewers Guild", "screen_name": "CraftMass", "lang": "en", "location": "Boston, MA", "create_at": date("2011-03-03"), "description": "Massachusetts leading wholesaler of craft and artisanal beer for the modern beer consumer.", "followers_count": 7117, "friends_count": 986, "statues_count": 7521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Somerville, MA", "id": "ae845a49091f2727", "name": "Somerville", "place_type": "city", "bounding_box": rectangle("-71.134552,42.372753 -71.072892,42.418052") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2562535, "cityName": "Somerville" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888623411474432, "text": "#SOONERNATION #BoomerSooner #Big12Champs https://t.co/5Vl5p31Yfy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SOONERNATION", "BoomerSooner", "Big12Champs" }}, "user": { "id": 3228853726, "name": "The Bob Stoops Army", "screen_name": "bobstoopsarmy", "lang": "en", "location": "Norman, OK", "create_at": date("2015-05-02"), "description": "Straight up #BoomerSooner 24-7-365 4 LIFE in this group. The site account for the Facebook group: The Bob Stoops Army. #OUDNA #takeNOTICE #InBobWeTrust", "followers_count": 1877, "friends_count": 3558, "statues_count": 4667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-02-22T13:57:42.000Z"), "id": 701888623646367744, "text": "time you enjoy wasting was not wasted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2895366746, "name": "marlene", "screen_name": "KoalaSkywalker9", "lang": "en", "location": "La Jolla, Calif", "create_at": date("2014-11-09"), "description": "concert junkie who is studying Mechanical Engineering @ the University of California, San Diego. polo, swim, surf.", "followers_count": 160, "friends_count": 251, "statues_count": 3423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888623843475457, "text": "These Nigg trippin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2888866566, "name": "D", "screen_name": "The6GDF", "lang": "en", "location": "Eagle Mountain, TX", "create_at": date("2014-11-03"), "description": "Washed up football player.#FreeTony", "followers_count": 659, "friends_count": 466, "statues_count": 13338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888624233545728, "text": "\"Care about what other people think ad you will always be their prisoner.\"\n#VoteMaineFPP #KCA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VoteMaineFPP", "KCA" }}, "user": { "id": 3386216899, "name": "ljan", "screen_name": "iyang896", "lang": "en", "location": "Winnipeg, Manitoba", "create_at": date("2015-08-29"), "description": "be kind", "followers_count": 157, "friends_count": 62, "statues_count": 7033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888624309055489, "text": "my shit https://t.co/SCYwv88lbk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165882950, "name": "sophie", "screen_name": "RlRlCHIYO", "lang": "en", "location": "null", "create_at": date("2010-07-12"), "description": "the life of sophie", "followers_count": 201, "friends_count": 122, "statues_count": 3487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Novi, MI", "id": "9458d33c93e47ce4", "name": "Novi", "place_type": "city", "bounding_box": rectangle("-83.555202,42.436044 -83.432974,42.527116") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2659440, "cityName": "Novi" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888624510406656, "text": "48. unpopular opinions aren't really unpopular. or opinions.", "in_reply_to_status": 701554800660320256, "in_reply_to_user": 2743484856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2743484856, "name": "life of pablo", "screen_name": "haibail", "lang": "en", "location": "Keller, TX --- Lubbock, TX", "create_at": date("2014-08-18"), "description": "null", "followers_count": 1124, "friends_count": 670, "statues_count": 8108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888624560775168, "text": "mostly cloudy -> partly cloudy\ntemperature up 52°F -> 57°F\nhumidity down 28% -> 19%\nwind 7mph -> 14mph\npressure 29.81in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.61237,38.26386"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131013352, "name": "Pueblo Weather", "screen_name": "PuebloCO", "lang": "en", "location": "Pueblo, CO", "create_at": date("2010-04-08"), "description": "Weather updates, forecast, warnings and information for Pueblo, CO. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 118, "friends_count": 2, "statues_count": 27935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pueblo, CO", "id": "9d7b47e751be1551", "name": "Pueblo", "place_type": "city", "bounding_box": rectangle("-104.69356,38.200638 -104.551992,38.338462") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8101, "countyName": "Pueblo", "cityID": 862000, "cityName": "Pueblo" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888624615415809, "text": "TICKET GIVEAWAY ALERT! Win tix to 3/5 Comedy Zone show! #PressPlay to get details from host @QtheMic #ComedyAtSMDCAC https://t.co/BrnCvjIxhe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PressPlay", "ComedyAtSMDCAC" }}, "user_mentions": {{ 484852465 }}, "user": { "id": 331664442, "name": "SMDCAC", "screen_name": "SMDCAC", "lang": "en", "location": "211th Street, Cutler Ridge", "create_at": date("2011-07-08"), "description": "The South Miami-Dade Cultural Arts Center is conceived as a multidisciplinary center for the performing arts the community and cultural development.", "followers_count": 718, "friends_count": 500, "statues_count": 2076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goulds, FL", "id": "62888de691b9797a", "name": "Goulds", "place_type": "city", "bounding_box": rectangle("-80.407893,25.551181 -80.363068,25.574655") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1226950, "cityName": "Goulds" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888625236172800, "text": "why am i even friends w u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3355899741, "name": "miaa", "screen_name": "miacutrona", "lang": "en", "location": "null", "create_at": date("2015-07-02"), "description": "null", "followers_count": 365, "friends_count": 399, "statues_count": 2649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888625303121920, "text": "@Bywatergal I signed the estate tax return for NJ, &believe I just paid Christie's remaining campaign debt. ������No wonder people leave there.", "in_reply_to_status": -1, "in_reply_to_user": 2432221788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2432221788 }}, "user": { "id": 414469058, "name": "Michael Fein", "screen_name": "mikef3825", "lang": "en", "location": "Metairie, LA", "create_at": date("2011-11-16"), "description": "this is me, Yankees, Saints, Florida Gators, Cincinnati Bearcats; classic Libertarian; G-d takes good care of me; retired Petroleum Geologist; happy with life", "followers_count": 77, "friends_count": 248, "statues_count": 4681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888625655418880, "text": "Found some awesome souvenirs on our adventure trek today!! #RailRoadSpikes #NatureShots @… https://t.co/oenStMpvKz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.84216621,36.13862025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RailRoadSpikes", "NatureShots" }}, "user": { "id": 572323209, "name": "Garrett Moore", "screen_name": "Gmoore55", "lang": "en", "location": "null", "create_at": date("2012-05-05"), "description": "null", "followers_count": 336, "friends_count": 209, "statues_count": 1571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashville, TN", "id": "00ab941b685334e3", "name": "Nashville", "place_type": "city", "bounding_box": rectangle("-87.022482,35.994568 -86.560616,36.405448") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47037, "countyName": "Davidson", "cityID": 4752006, "cityName": "Nashville-Davidson metropolitan government (balance)" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888625911459840, "text": "This is a baseball player? https://t.co/iiCbHA5OI9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353677716, "name": "ray Valerio", "screen_name": "Ramonval70", "lang": "en", "location": "null", "create_at": date("2011-08-12"), "description": "null", "followers_count": 43, "friends_count": 262, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Garden City, NY", "id": "0427846a106c0bd9", "name": "East Garden City", "place_type": "city", "bounding_box": rectangle("-73.617232,40.712873 -73.580488,40.747586") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3622065, "cityName": "East Garden City" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626372775936, "text": "@SamWagner66 because I'm not smart that's why����", "in_reply_to_status": 701886875875876865, "in_reply_to_user": 272995124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 272995124 }}, "user": { "id": 739671648, "name": "Abbey Carlson", "screen_name": "AbbeyCarlson54", "lang": "en", "location": "221B Baker Street ", "create_at": date("2012-08-05"), "description": "I never got my Hogwarts letter so I'm in high school instead. 2016 Vandy golf commit. Student-pilot and STEM nerd. ⚓️#AnchorDown", "followers_count": 577, "friends_count": 443, "statues_count": 3245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Springs, FL", "id": "88272cf603c89248", "name": "Winter Springs", "place_type": "city", "bounding_box": rectangle("-81.326884,28.658956 -81.208792,28.730516") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1278325, "cityName": "Winter Springs" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626389557248, "text": "Hey u guys wanna play cops n robbers?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2906024656, "name": "Push uP God", "screen_name": "REALRIGHT_WIT", "lang": "en", "location": "Nayba Hood Land", "create_at": date("2014-12-04"), "description": "IG:RealRight_Wit Push uP Gang x OPB Records", "followers_count": 165, "friends_count": 144, "statues_count": 4953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626641281033, "text": "@WfanMongo109 Kars 4 Kids got me the other day. I posted a YouTube video of Skylar White yelling \"SHUT UP\" and they said \"Turn the dial\"", "in_reply_to_status": 701888382243315713, "in_reply_to_user": 2997367955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2997367955 }}, "user": { "id": 2895523516, "name": "Gil Valle", "screen_name": "TheRealGValle", "lang": "en", "location": "Middle Village, NY ", "create_at": date("2014-11-27"), "description": "31 years old. I know a thing or two about a thing or two", "followers_count": 172, "friends_count": 709, "statues_count": 4488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626691452928, "text": "Girls always complaining on how a guy shud act towards a girl but wen a guy tries to say how a girl shud act dey get all defensive and shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3235597309, "name": "Lamar Callico", "screen_name": "thatguy_callico", "lang": "en", "location": "null", "create_at": date("2015-06-03"), "description": "Educated | 1996 | NAU 18' | Blessed |", "followers_count": 235, "friends_count": 254, "statues_count": 1104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626695626752, "text": "All i can do is be myself and treat others how id want to be treated and see who sticks around after high school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1387729627, "name": "ebony queen", "screen_name": "Mteaszy", "lang": "en", "location": "#BlackLivesMatter", "create_at": date("2013-04-28"), "description": "Independent & Outspoken VII.XV.XIII", "followers_count": 534, "friends_count": 334, "statues_count": 34775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888626712559616, "text": "@MrNailsin Did you get those changes on Facebook?", "in_reply_to_status": -1, "in_reply_to_user": 419118169, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 419118169 }}, "user": { "id": 20554168, "name": "Evanweisscomedian", "screen_name": "evanweisscomedy", "lang": "en", "location": "New York", "create_at": date("2009-02-10"), "description": "Comedian out of NYC ,O&A Pest, Islanders,Mets,Jets Fan", "followers_count": 1567, "friends_count": 2029, "statues_count": 30528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holtsville, NY", "id": "81982a28b652a9f8", "name": "Holtsville", "place_type": "city", "bounding_box": rectangle("-73.081839,40.785064 -73.009947,40.836346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3635254, "cityName": "Holtsville" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888627215888384, "text": "#saltwatertherapy https://t.co/4Nnwv9j1vE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "saltwatertherapy" }}, "user": { "id": 53654464, "name": "oobie", "screen_name": "oobiewpb", "lang": "en", "location": "WPB Florida", "create_at": date("2009-07-04"), "description": "I catch the moves that confirm the moves. Bulls & Bears are investors, I am a trader, I trade what I see. No likey, no follow . NO trade recommendations posted.", "followers_count": 839, "friends_count": 1016, "statues_count": 9659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Ridge, FL", "id": "4e20db04c5db1926", "name": "Ocean Ridge", "place_type": "city", "bounding_box": rectangle("-80.055707,26.509122 -80.043112,26.545448") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1250950, "cityName": "Ocean Ridge" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888627442327552, "text": "@Ddslilgrl418 @rockinrobyn59 @CosmosLeaseExp @TerranceCreamer @SpcbassA @PatriotKevinMO @KevinLoftus68 @kazueger1 https://t.co/W6r5sSto0w", "in_reply_to_status": -1, "in_reply_to_user": 2949234347, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2949234347, 151284118, 2912690232, 3166623503, 3222961026, 3382242346, 3718007774, 2686010270 }}, "user": { "id": 2346790179, "name": "Dagmar Christa Veen", "screen_name": "dagmarEmt", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2014-02-16"), "description": "Born and raised in Germany. Career EMT, former volunteer FF. Married to my best friend! #Pro-gun #pro-life #pro-military #MolonLabe #2A #Proud NRA member!", "followers_count": 2000, "friends_count": 3056, "statues_count": 15471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAdoo, PA", "id": "b8c8ba668bdf0a61", "name": "McAdoo", "place_type": "city", "bounding_box": rectangle("-76.000008,40.893371 -75.984242,40.905596") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42107, "countyName": "Schuylkill", "cityID": 4245824, "cityName": "McAdoo" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888627563868160, "text": "@WesMeyer619 @CoryLarimer619. @JerronLee34 on fifa https://t.co/QgHwuoQci4", "in_reply_to_status": -1, "in_reply_to_user": 284712229, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 284712229, 320438353, 408186180 }}, "user": { "id": 21541759, "name": "Wyatt Meyer", "screen_name": "WyattMeyer619", "lang": "en", "location": "Ames, IA", "create_at": date("2009-02-21"), "description": "Real recognize real", "followers_count": 304, "friends_count": 153, "statues_count": 7093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ames, IA", "id": "d6794586b08b7f9c", "name": "Ames", "place_type": "city", "bounding_box": rectangle("-93.718311,41.986006 -93.560416,42.079007") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1901855, "cityName": "Ames" } }
+{ "create_at": datetime("2016-02-22T13:57:43.000Z"), "id": 701888627756789760, "text": "My goal for the remaining of the day is to take a decent selfie inbetween feedings and cooking.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4831714442, "name": "The Married Betch.", "screen_name": "themarriedbetch", "lang": "en", "location": "null", "create_at": date("2016-01-29"), "description": "Wife. Mami. Pictures for Days. \nI like Wendy Williams too, she's cool.", "followers_count": 14, "friends_count": 24, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moscow, ID", "id": "be3410e825fe4fa4", "name": "Moscow", "place_type": "city", "bounding_box": rectangle("-117.039699,46.710846 -116.961769,46.758567") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16057, "countyName": "Latah", "cityID": 1654550, "cityName": "Moscow" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888628390125572, "text": "San Francisco ���� roller coaster ride ba kamo???������������������ https://t.co/gwMCuXvk5c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 1324406022, "name": "Jennifer Wu", "screen_name": "J3Nwu", "lang": "fil", "location": "null", "create_at": date("2013-04-03"), "description": "TWEET %$%^$$%&^%????", "followers_count": 19, "friends_count": 161, "statues_count": 72 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888628641763328, "text": "my only way to get home is to walk ������������ @kamariahharrell walk with me???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 628761087 }}, "user": { "id": 2288564443, "name": "mandaa", "screen_name": "mandaandersonn_", "lang": "en", "location": "null", "create_at": date("2014-01-12"), "description": "12-27-15", "followers_count": 376, "friends_count": 256, "statues_count": 1665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakdale, MN", "id": "8dc3b0709714ce66", "name": "Oakdale", "place_type": "city", "bounding_box": rectangle("-92.984869,44.948501 -92.934227,45.034285") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27163, "countyName": "Washington", "cityID": 2747680, "cityName": "Oakdale" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888628813918209, "text": "Pero estamos vivo y con salud y eso es lo que cuenta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 275868199, "name": "vierkamariax", "screen_name": "VierkaMaria", "lang": "en", "location": "null", "create_at": date("2011-04-01"), "description": "Esthetician | Cosmetologist Snap:VierkaMariax", "followers_count": 725, "friends_count": 387, "statues_count": 26155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasantville, NJ", "id": "4439e5140bd8b701", "name": "Pleasantville", "place_type": "city", "bounding_box": rectangle("-74.565369,39.367375 -74.485853,39.423897") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3459640, "cityName": "Pleasantville" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629128437760, "text": "#chickendance #zumbathon #feb28 #contact samanthaesteban… https://t.co/H28ucT7OA3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.43034522,34.16400472"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "chickendance", "zumbathon", "feb28", "contact" }}, "user": { "id": 75184882, "name": "yul edejer", "screen_name": "yuledejer", "lang": "en", "location": "34.16221,-118.432141", "create_at": date("2009-09-17"), "description": "null", "followers_count": 49, "friends_count": 69, "statues_count": 2049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629363179520, "text": "both https://t.co/EQu0ufCnPF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4410443719, "name": "ä$wish", "screen_name": "sackkgrl", "lang": "en", "location": "in alief fuckin ya bitch.", "create_at": date("2015-11-30"), "description": "null", "followers_count": 293, "friends_count": 600, "statues_count": 1456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629384163328, "text": "I can't handle people feeling sorry for themselves.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1410756428, "name": "Princess", "screen_name": "__lush", "lang": "en", "location": "null", "create_at": date("2013-05-07"), "description": "Ena Masiofo", "followers_count": 847, "friends_count": 918, "statues_count": 12541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629505990656, "text": "@_xngo happy birthday and a ��", "in_reply_to_status": -1, "in_reply_to_user": 2728250355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728250355 }}, "user": { "id": 3454351696, "name": "Be Free", "screen_name": "Younggreatness0", "lang": "en", "location": "New London, CT", "create_at": date("2015-08-26"), "description": "♐️ #Classof2015| #ColeWorld| R.I.P Taylor Theroux |All we wanna do is break the chains off -JColeNC", "followers_count": 265, "friends_count": 380, "statues_count": 8425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New London, CT", "id": "6e9c054da607b539", "name": "New London", "place_type": "city", "bounding_box": rectangle("-72.12852,41.304762 -72.087716,41.385244") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 952280, "cityName": "New London" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629514248192, "text": "@miles_matthias yeah. It was the number.", "in_reply_to_status": 701887651020828672, "in_reply_to_user": 129343561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 129343561 }}, "user": { "id": 15265271, "name": "ⓕⓣ", "screen_name": "folktrash", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-06-28"), "description": "this internet thing is going to be huge.", "followers_count": 1323, "friends_count": 2075, "statues_count": 38633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Totem Lake, Kirkland", "id": "181cb49a7c0cda20", "name": "Totem Lake", "place_type": "neighborhood", "bounding_box": rectangle("-122.197289,47.695885 -122.164234,47.718592") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335940, "cityName": "Kirkland" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629631807489, "text": "I can't bring myself to watch the Work video. I doubt I ever will.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3303284014, "name": "Dion", "screen_name": "QuickTime_", "lang": "en", "location": " Slauson", "create_at": date("2015-05-29"), "description": "Life's short. Sometimes you gotta settle for vag jokes.", "followers_count": 871, "friends_count": 242, "statues_count": 40792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stratford, CT", "id": "9133850ec8827e0a", "name": "Stratford", "place_type": "city", "bounding_box": rectangle("-73.164245,41.147132 -73.088173,41.268524") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 974260, "cityName": "Stratford" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629635993604, "text": "tired** https://t.co/S05pCxuISb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3225589775, "name": "eazy-c!☺", "screen_name": "chynaarayell", "lang": "en", "location": "s/w on my phone", "create_at": date("2015-04-30"), "description": "ChristopherMauriceBrown❤.i love @chrisbrown❤.", "followers_count": 134, "friends_count": 148, "statues_count": 4497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629677936641, "text": "One of the few things that makes me feel better after a bad day is Gossip Girl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2324515604, "name": "Kari Burnett", "screen_name": "BurnettKari", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "deuteronomy 31:6", "followers_count": 271, "friends_count": 436, "statues_count": 923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629723893760, "text": "cloud nine bout to turn up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130448496, "name": "Harrison™", "screen_name": "Berdamn", "lang": "en", "location": "BAY AREA", "create_at": date("2010-04-07"), "description": "nevada '19 //", "followers_count": 1047, "friends_count": 845, "statues_count": 12407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888629929603072, "text": "Wow #goals �� https://t.co/ACxhpMHo4R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "goals" }}, "user": { "id": 493162263, "name": "Jerikita", "screen_name": "jerikamendez", "lang": "en", "location": "with johan or maria", "create_at": date("2012-02-15"), "description": "5'11'14 f&a", "followers_count": 418, "friends_count": 301, "statues_count": 29144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Ferry, NJ", "id": "eee6d4242f7819c2", "name": "Little Ferry", "place_type": "city", "bounding_box": rectangle("-74.054255,40.828254 -74.029213,40.859295") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3440680, "cityName": "Little Ferry" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630202224640, "text": "Wagers?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604804882, "name": "Matt", "screen_name": "FahneetoIsBae", "lang": "en", "location": "Nesquehoning, PA", "create_at": date("2012-06-10"), "description": "19 Player for @OverDoseVisionz", "followers_count": 678, "friends_count": 376, "statues_count": 12151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nesquehoning, PA", "id": "00ec94766108703c", "name": "Nesquehoning", "place_type": "city", "bounding_box": rectangle("-75.829229,40.858352 -75.741093,40.874542") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42025, "countyName": "Carbon", "cityID": 4253088, "cityName": "Nesquehoning" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630290141184, "text": "3 Season Passes, 2 Skiers, 1 Snowboarder, 0 Jobs. #funemployed… https://t.co/hFh6SsHh0N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-120.23512466,39.19664174"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "funemployed" }}, "user": { "id": 101988585, "name": "Tony Kauke", "screen_name": "tony_kauke", "lang": "en", "location": "San Francisco", "create_at": date("2010-01-04"), "description": "Dreamer, Runner, Coach. Lover of food, beer, travel, all things outdoors and sports", "followers_count": 113, "friends_count": 350, "statues_count": 519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630558629889, "text": "By yourself is the best way to be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2856026443, "name": "✨GinaThaFlippa✨", "screen_name": "GinaTheSavage", "lang": "en", "location": "Walkermill, Road ", "create_at": date("2014-10-14"), "description": "RipDunk❤️RipReggie❤️RipUncleRonnie❤️ Ty❤Lightshow ❤️ @GogoHeaddNaee_❤️", "followers_count": 5258, "friends_count": 2941, "statues_count": 133887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suitland, MD", "id": "00da23b51c3aabc1", "name": "Suitland", "place_type": "city", "bounding_box": rectangle("-76.958123,38.827518 -76.88434,38.868794") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2475725, "cityName": "Suitland" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630617468928, "text": "@boysinbikinis @Nixon_Now thats a stretch ha", "in_reply_to_status": 701888552540512259, "in_reply_to_user": 247441079, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 247441079, 36575741 }}, "user": { "id": 26518374, "name": "WalterBlakeKnoblock", "screen_name": "WBKnoblock", "lang": "en", "location": "Detroit/Flint", "create_at": date("2009-03-25"), "description": "built @getbocandy | snapchat: walterblake | made @greatlakesbook | lots and lots of side projects | Howling Mayhem", "followers_count": 79625, "friends_count": 50052, "statues_count": 33106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630793641985, "text": "I've always wanted to go on a bike ride w someone I love and sit on the handle bars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 801146822, "name": "cash", "screen_name": "ashlyne34", "lang": "en", "location": "Highlands, NJ", "create_at": date("2012-09-03"), "description": "cam newton is my dad", "followers_count": 1552, "friends_count": 489, "statues_count": 16559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granger, IN", "id": "20dc6048f416c894", "name": "Granger", "place_type": "city", "bounding_box": rectangle("-86.217544,41.709039 -86.023537,41.760686") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1828800, "cityName": "Granger" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630873333762, "text": "anyone that says \"I support Trump and Cruz\" will be blocked. They are polar opposites. You cannot be logical & support both", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 976664240, "name": "Colleen", "screen_name": "BorisGBauer", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2012-11-28"), "description": "Anything Cruz/blocked. I AM VOTING FOR TRUMP! hard working, street smart, Love rock and roll and punk still listen to vinyl/turntable, luv outdoors, jogging", "followers_count": 1441, "friends_count": 1783, "statues_count": 19134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630894125056, "text": "���� https://t.co/OnNtTA6Lq6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3146639503, "name": "〽️ahhhhh", "screen_name": "Papii_333", "lang": "en", "location": "null", "create_at": date("2015-04-09"), "description": "Sc: Papii_333 @Gabbeigh❤️", "followers_count": 260, "friends_count": 202, "statues_count": 2847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corsicana, TX", "id": "faa25283bbfe186a", "name": "Corsicana", "place_type": "city", "bounding_box": rectangle("-96.537914,32.054664 -96.415163,32.131294") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48349, "countyName": "Navarro", "cityID": 4817060, "cityName": "Corsicana" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630923472896, "text": "Killing the emotional attachments with the alcohol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140608504, "name": "lorddubriel", "screen_name": "LordDubriel", "lang": "en", "location": "Mansfield,TX", "create_at": date("2010-05-05"), "description": "†⚓▲you close but ya'll not close enough ▲†⚓", "followers_count": 420, "friends_count": 404, "statues_count": 8739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, TX", "id": "012a07a105d10298", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-97.186747,32.530915 -97.046821,32.616182") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4846452, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888630965428225, "text": "@HA_AlHumaidi @7arbia96 تذكرون الفيديو اللي صورناه ������!! https://t.co/Dqcthy3IbU", "in_reply_to_status": -1, "in_reply_to_user": 56273680, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 56273680, 1629038682 }}, "user": { "id": 404863599, "name": "سكينة", "screen_name": "Skown_", "lang": "en", "location": "Arizona, USA", "create_at": date("2011-11-04"), "description": "A Future Meteorologist.", "followers_count": 1885, "friends_count": 445, "statues_count": 26018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631032705025, "text": "@ my baby since she probably annoyed with all these @ 's https://t.co/rLwbRb2WwI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1549415857, "name": "Kay ♌️✨", "screen_name": "Kaylurr_xo", "lang": "en", "location": "null", "create_at": date("2013-06-26"), "description": "@loveable_naijah has my ❤️ | MPHS '16 ♌️✨", "followers_count": 326, "friends_count": 306, "statues_count": 5863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631053664256, "text": "The comments :( https://t.co/832kKVGLMi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177402142, "name": "aria", "screen_name": "FixItBreezus", "lang": "en", "location": "chi-town", "create_at": date("2010-08-11"), "description": "I got ups, bitch. what can Blas do for you?! #OHB", "followers_count": 990, "friends_count": 333, "statues_count": 74197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631288500225, "text": "@ROC4Cruz Lying is not getting it done NEW STRATEGY FIND A BETTER LIAR https://t.co/aoUVI7pdUV", "in_reply_to_status": -1, "in_reply_to_user": 3414944049, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3414944049 }}, "user": { "id": 4861280291, "name": "TrumpWhiteHouseBitch", "screen_name": "ItsRickBitches", "lang": "en", "location": "null", "create_at": date("2016-01-29"), "description": "null", "followers_count": 391, "friends_count": 1058, "statues_count": 5213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamilton, OH", "id": "3a32d101fe825fd9", "name": "Hamilton", "place_type": "city", "bounding_box": rectangle("-84.64717,39.350135 -84.474638,39.457988") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3933012, "cityName": "Hamilton" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631422603264, "text": "it's the little things", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1788600853, "name": "annie harris", "screen_name": "annieeharriss", "lang": "en", "location": "null", "create_at": date("2013-09-07"), "description": "dpvb & chane", "followers_count": 858, "friends_count": 620, "statues_count": 6135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631489875968, "text": "What in the world am I doing?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338986277, "name": "Shayne Furman", "screen_name": "King_Furman97", "lang": "en", "location": "Coral Springs, FL", "create_at": date("2011-07-20"), "description": "Canadian made, American raised. CSHS Senior #ScubaDiver⚓️ #GymRat", "followers_count": 386, "friends_count": 1112, "statues_count": 5974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Springs, FL", "id": "6241025d720c4971", "name": "Coral Springs", "place_type": "city", "bounding_box": rectangle("-80.297884,26.228692 -80.201661,26.31183") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1214400, "cityName": "Coral Springs" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888631691206657, "text": "It's always better once you put your hard work and effort into it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3158242460, "name": "victoriaaa", "screen_name": "vickii323", "lang": "en", "location": "null", "create_at": date("2015-04-15"), "description": "null", "followers_count": 328, "friends_count": 659, "statues_count": 4645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westchester, FL", "id": "af57c21383d06ea8", "name": "Westchester", "place_type": "city", "bounding_box": rectangle("-80.352385,25.732778 -80.316671,25.762491") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1276075, "cityName": "Westchester" } }
+{ "create_at": datetime("2016-02-22T13:57:44.000Z"), "id": 701888632039170048, "text": "Just posted a photo @ Ramen Bowls https://t.co/qFoLZZA1I1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.23528972,38.96555109"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33436131, "name": "Cate Cowan", "screen_name": "Smooshka", "lang": "en", "location": "Denton, TX", "create_at": date("2009-04-19"), "description": "null", "followers_count": 174, "friends_count": 332, "statues_count": 13820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas, USA", "id": "27c45d804c777999", "name": "Kansas", "place_type": "admin", "bounding_box": rectangle("-102.051769,36.99311 -94.588081,40.003282") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20045, "countyName": "Douglas", "cityID": 2038900, "cityName": "Lawrence" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632114679808, "text": "My Man ���� https://t.co/L3ImfuY5Lv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2861907969, "name": "〽️", "screen_name": "CheyenneMirage", "lang": "en", "location": "over it, tx", "create_at": date("2014-11-04"), "description": "basketball basketball basketball basketball", "followers_count": 412, "friends_count": 551, "statues_count": 11549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632118857728, "text": "@BabyBee36265 @HillaryClinton Not a bit Sue B, not a bit.", "in_reply_to_status": 701888391672156160, "in_reply_to_user": 256506750, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256506750, 1339835893 }}, "user": { "id": 809904438, "name": "Cheri Armstrong", "screen_name": "Biloximeemaw", "lang": "en", "location": "null", "create_at": date("2012-09-07"), "description": "Locked and loaded Meemaw. Love God, my family and my country. I will fight til my last breath for my children and grand children's freedoms. Proud NRA Member!", "followers_count": 2240, "friends_count": 3202, "statues_count": 6200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632173551616, "text": "I wish someone come blow they hot ass breath on my toes because they cold asf rn ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301758325, "name": "KaydeeFineAss", "screen_name": "_kaydeeblood", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "RestInParadiseTrey❤️ #SU19 iG: kaydamac Sc: kaydeeblood....", "followers_count": 2834, "friends_count": 1043, "statues_count": 70870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632223879168, "text": "It's okay 3 more days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 777101737, "name": "Barbara Rodriguez♡", "screen_name": "babaa_07", "lang": "en", "location": "Sioux City, Iowa ", "create_at": date("2012-08-23"), "description": "IG: babaa_07 sc: babaa07", "followers_count": 1460, "friends_count": 665, "statues_count": 49911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux City, IA", "id": "c7397adf5a17673a", "name": "Sioux City", "place_type": "city", "bounding_box": rectangle("-96.492971,42.385735 -96.301329,42.56216") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19193, "countyName": "Woodbury", "cityID": 1973335, "cityName": "Sioux City" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632269967360, "text": "Double text when I really down for you ��.. Otherwise oh well ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 701799919477198848, "name": "lowkeyy", "screen_name": "ke___asiaa", "lang": "en", "location": "Baltimore, MD", "create_at": date("2016-02-22"), "description": "fmoig: @ke__asia❤️", "followers_count": 57, "friends_count": 309, "statues_count": 47 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632773283842, "text": "#kitty #cat caring for me. @ Lincoln, Nebraska https://t.co/sy8n8nZZG7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6753,40.8099"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "kitty", "cat" }}, "user": { "id": 912759548, "name": "Otaku Genx", "screen_name": "OtakuGenx", "lang": "en", "location": "Cyberspace", "create_at": date("2012-10-29"), "description": "The anomaly in the machine. I am me..", "followers_count": 42, "friends_count": 113, "statues_count": 1395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632810942464, "text": "https://t.co/R3g61OOxdk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 19111053, "name": "Derrin M. Cogar", "screen_name": "CogarDM", "lang": "en", "location": "Olympia, WA 98501", "create_at": date("2009-01-17"), "description": "As you get older, the questions come down to about two or three. How long? And what do I do with the time I've got left?\nDavid Bowie", "followers_count": 807, "friends_count": 818, "statues_count": 5635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888632999825409, "text": "My dream is to have @DevMacGillivray to feature on a dismal song but like how deftones featured Maynard where he was on the full song", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 250992527 }}, "user": { "id": 3212711223, "name": "Crylo Ren", "screen_name": "jeremytorres553", "lang": "en", "location": "Gotham", "create_at": date("2015-04-27"), "description": "✂️ 王 Skeleton Youth Forever 王 ✂️", "followers_count": 347, "friends_count": 789, "statues_count": 673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casselberry, FL", "id": "a8237d3409a76683", "name": "Casselberry", "place_type": "city", "bounding_box": rectangle("-81.350571,28.610836 -81.286124,28.697981") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1211050, "cityName": "Casselberry" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888633033220097, "text": "I listen to this song just for chance https://t.co/ILFy7Bbfsa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1681936394, "name": "Dad", "screen_name": "tristoncessac", "lang": "en", "location": "Lubbock, TX", "create_at": date("2013-08-18"), "description": "Allen // #TTU Geophysics Major", "followers_count": 405, "friends_count": 326, "statues_count": 10362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888633071075328, "text": "The nah https://t.co/N1axf1wAZX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3292008148, "name": "zoom⚡️", "screen_name": "bigpapiarey", "lang": "en", "location": "Earth 2", "create_at": date("2015-05-20"), "description": "floomin⚡️", "followers_count": 120, "friends_count": 84, "statues_count": 6031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosaryville, MD", "id": "01f79450772185e8", "name": "Rosaryville", "place_type": "city", "bounding_box": rectangle("-76.867303,38.735578 -76.794846,38.834121") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2468300, "cityName": "Rosaryville" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888633205346304, "text": "this weekends volleyball tourney with my fav�� @ Boomerangs https://t.co/z3sCwq2uAA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.16398674,26.10388137"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 536920924, "name": "carrr.ૐ", "screen_name": "fcknene", "lang": "en", "location": "Brownsville, TX", "create_at": date("2012-03-25"), "description": "baseball games/beach trips/UTRGV .ૐ", "followers_count": 2753, "friends_count": 3027, "statues_count": 77397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Padre Island, TX", "id": "1d1f665883989434", "name": "South Padre Island", "place_type": "city", "bounding_box": rectangle("-97.177252,26.078455 -97.1597,26.151016") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48261, "countyName": "Kenedy", "cityID": 4869140, "cityName": "South Padre Island" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888633469595649, "text": "Riding through the hood jamming free bird", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321017866, "name": "The Rod Father", "screen_name": "rodney_edwardss", "lang": "en", "location": "Brunswick, GA", "create_at": date("2011-06-20"), "description": "I'm Rodney Edwards and I support blondes, trump, beer, Georgia football and sushi.", "followers_count": 2313, "friends_count": 1254, "statues_count": 88135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brunswick, GA", "id": "00cd1ca713cf470d", "name": "Brunswick", "place_type": "city", "bounding_box": rectangle("-81.570027,31.122864 -81.439789,31.306113") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13127, "countyName": "Glynn", "cityID": 1311560, "cityName": "Brunswick" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888633968529408, "text": "4.20 Bars from Evergreen Herbal have finally arrived at Ganja Goddess! These #delicious… https://t.co/1DSZritMpQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.331,47.6064"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "delicious" }}, "user": { "id": 2900881921, "name": "Ganja Goddess", "screen_name": "GanjaGoddessWA", "lang": "en", "location": "3207 1st Ave S. Seattle, WA", "create_at": date("2014-11-15"), "description": "Seattle's premier recreational marijuana store. You agree that you are 21 years of age or older to interact with Ganja Goddess on Twitter.", "followers_count": 1429, "friends_count": 172, "statues_count": 610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888634270654464, "text": "@valeriew272 Evening.. https://t.co/ekWjSTlITR", "in_reply_to_status": -1, "in_reply_to_user": 2719464789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2719464789 }}, "user": { "id": 2548954273, "name": "TINK", "screen_name": "MayfieldJoy", "lang": "en", "location": "Indiana", "create_at": date("2014-06-05"), "description": "Retired, Democratic, Love the outdoors ! I'm CHEROKEE and SHAWNEE ,French. My Man's AVI is on his leg. and a Ring on my finger.. Respect Us..", "followers_count": 3077, "friends_count": 2043, "statues_count": 66222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schererville, IN", "id": "34cbdf1e53ef826e", "name": "Schererville", "place_type": "city", "bounding_box": rectangle("-87.493563,41.449395 -87.392266,41.523148") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1868220, "cityName": "Schererville" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888634471976961, "text": "Night class https://t.co/IXrfiyNhLL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233814542, "name": "Johnny Foosball", "screen_name": "iam_joshdavis", "lang": "en", "location": "Strong Island", "create_at": date("2011-01-03"), "description": "You Got To Risk It To Get The Biscuit 6ix31✈️FAU #SoFlo #Regime #Bengals #NYM #NYR #NYK TD❤️", "followers_count": 736, "friends_count": 345, "statues_count": 39247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watergate, FL", "id": "00ef899063d4d5fa", "name": "Watergate", "place_type": "city", "bounding_box": rectangle("-80.251664,26.327846 -80.202946,26.364207") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1275275, "cityName": "Watergate" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888634618830850, "text": "When your a 20 year old that still can't put in contacts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34638330, "name": "Alyssa D'Angelo", "screen_name": "Alyssa_DAngelo", "lang": "en", "location": "United States.", "create_at": date("2009-04-23"), "description": "Augustana 18' ΔΧΘ", "followers_count": 235, "friends_count": 187, "statues_count": 4936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheeling, IL", "id": "f2da3efc48696715", "name": "Wheeling", "place_type": "city", "bounding_box": rectangle("-87.961117,42.094938 -87.886833,42.154943") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1781087, "cityName": "Wheeling" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888634794987521, "text": "Future Farmers of America Week! Always thank a farmer and encourage our young farmers! https://t.co/Mv2349hJ4B #FFAweek #farmers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FFAweek", "farmers" }}, "user": { "id": 31899347, "name": "Travis S. Taylor", "screen_name": "TravisSTaylor", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-16"), "description": "Founder & Author of wanderlust ATLANTA. Lover of tourism, blogging, photography, theatre, cinema, museums, gardens, celebrity spotting...mostly, I'm a tourist!", "followers_count": 741, "friends_count": 1808, "statues_count": 3085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888635608518657, "text": "Can you recommend anyone for this #Insurance #job? https://t.co/bhA9gZR2CN #amtrakvets #NewYork, NY #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9991637,40.7536854"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Insurance", "job", "amtrakvets", "NewYork", "Hiring" }}, "user": { "id": 2805763571, "name": "Amtrak Veteran Jobs", "screen_name": "AmtrakVets", "lang": "en", "location": "United States", "create_at": date("2014-10-04"), "description": "#Amtrak proudly hires U.S. Veterans. Follow us learn more about #AmtrakJobs for military professionals. #amtrakVets", "followers_count": 820, "friends_count": 1368, "statues_count": 677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888635680006144, "text": "Sometimes I wanna just take a bullet to the head and say bye.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34719060, "name": "justine", "screen_name": "justinee143", "lang": "en", "location": "null", "create_at": date("2009-04-23"), "description": "let the music be your serenity", "followers_count": 452, "friends_count": 484, "statues_count": 23606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ypsilanti, MI", "id": "0049088b1906e51b", "name": "Ypsilanti", "place_type": "city", "bounding_box": rectangle("-83.661476,42.165629 -83.542129,42.325797") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2016-02-22T13:57:45.000Z"), "id": 701888635860156416, "text": "@KaelaKolossal wondering what that feeling is lol", "in_reply_to_status": 701888563604910080, "in_reply_to_user": 56784371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56784371 }}, "user": { "id": 81954897, "name": "Beige the Gemini", "screen_name": "SilasTheVifth", "lang": "en", "location": "Chicago, AZ", "create_at": date("2009-10-12"), "description": "Somewhere between a logical writer and a madman with a pen.", "followers_count": 2379, "friends_count": 900, "statues_count": 142955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888636317409281, "text": "Young Life tonight is Snow day themed! Come dressed ready for the Mountains at 1312 Chippokes Ct :) ((7:29pm)) ☃❄️ https://t.co/FbKUJLio7J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1663535521, "name": "madison", "screen_name": "raynormadison", "lang": "en", "location": "null", "create_at": date("2013-08-11"), "description": "Exodus 14:14", "followers_count": 429, "friends_count": 130, "statues_count": 3646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888636887900162, "text": "I need an Alex Karev in my life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363911039, "name": "karolina dziedzic", "screen_name": "karooosayshi", "lang": "en", "location": "Oakland University", "create_at": date("2011-08-28"), "description": "you'll see me in Cali one day | rest easy Diana", "followers_count": 975, "friends_count": 685, "statues_count": 15806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sterling Heights, MI", "id": "23e0af972b9a2e8e", "name": "Sterling Heights", "place_type": "city", "bounding_box": rectangle("-83.091535,42.534811 -82.969611,42.627801") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2676460, "cityName": "Sterling Heights" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637022171140, "text": "Today�� https://t.co/8HKnLHrIQh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 918208591, "name": "kay➿", "screen_name": "kaylawittenmyer", "lang": "en", "location": "Holland, OH", "create_at": date("2012-10-31"), "description": "IIIXVXV ••Matthew 19:4-6", "followers_count": 321, "friends_count": 341, "statues_count": 12243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holland, OH", "id": "00d8d3fe7f31fc91", "name": "Holland", "place_type": "city", "bounding_box": rectangle("-83.786512,41.587436 -83.683711,41.66093") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3935882, "cityName": "Holland" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637160435712, "text": "zimie-i LOVE your art! im so surprised you followed me and then stuck around when i forgot to follow back (twitter sux) ur so Mature + Cool!", "in_reply_to_status": 701873988578836480, "in_reply_to_user": 1967879833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1967879833, "name": "ethan", "screen_name": "knifeeffect", "lang": "en", "location": "are you what you want to be?", "create_at": date("2013-10-17"), "description": "B/", "followers_count": 203, "friends_count": 258, "statues_count": 24664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637345120256, "text": "@Matthill96 haven't seen her in years", "in_reply_to_status": 701855066563731457, "in_reply_to_user": 123672161, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 123672161 }}, "user": { "id": 427008035, "name": "Leo Lopez", "screen_name": "leo_zepol", "lang": "en", "location": "outdoors", "create_at": date("2011-12-02"), "description": "helping others is the secret sauce to a happy life ✌ 2015 BHS Ping Pong Champion", "followers_count": 369, "friends_count": 312, "statues_count": 1714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brighton, MI", "id": "00811b0a01ecbe95", "name": "Brighton", "place_type": "city", "bounding_box": rectangle("-83.823194,42.50733 -83.688991,42.602457") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26093, "countyName": "Livingston", "cityID": 2610620, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637424705536, "text": "This #Sales #job might be a great fit for you: Cloud & ERP - Principal Sales Consultant - https://t.co/e8uN0w204E #Burlington, MA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.195611,42.5048167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "Burlington", "Hiring" }}, "user": { "id": 20831530, "name": "Boston Sales Jobs", "screen_name": "tmj_bos_sales", "lang": "en", "location": "Boston, MA", "create_at": date("2009-02-13"), "description": "Follow this account for geo-targeted Sales job tweets in Boston, MA. Need help? Tweet us at @CareerArc!", "followers_count": 711, "friends_count": 319, "statues_count": 829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637495955456, "text": "REMINDER the game is at 7:45 @ Valley Forge!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 517179798, "name": "Mallory $haw", "screen_name": "malloryyshaw", "lang": "en", "location": "Lakewood, OH", "create_at": date("2012-03-06"), "description": "University of Kentucky 20' #bbn ⚽️", "followers_count": 495, "friends_count": 396, "statues_count": 11099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, OH", "id": "888482aa70a3bc61", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-81.8375,41.463245 -81.768603,41.49759") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3941664, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637651316736, "text": "Thank you @benandjerrys for the wonderful #vegan treat!! #VeganDessert https://t.co/LLuiwlG32u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "vegan", "VeganDessert" }}, "user_mentions": {{ 18949610 }}, "user": { "id": 592244636, "name": "Easy Vegan Living", "screen_name": "Flowerchild0907", "lang": "en", "location": "Homestead, Florida", "create_at": date("2012-05-27"), "description": "I am just a vegan school counselor advocating for equal rights for all living beings.", "followers_count": 291, "friends_count": 592, "statues_count": 1843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naranja, FL", "id": "00531e7c63964b3d", "name": "Naranja", "place_type": "city", "bounding_box": rectangle("-80.453594,25.504351 -80.405043,25.536465") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1247700, "cityName": "Naranja" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637676425216, "text": "My goodness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325965319, "name": "✨", "screen_name": "labellaKUSH", "lang": "en", "location": "null", "create_at": date("2011-06-28"), "description": "null", "followers_count": 1347, "friends_count": 772, "statues_count": 84445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miramar, FL", "id": "5ca2cc5afa0894df", "name": "Miramar", "place_type": "city", "bounding_box": rectangle("-80.40816,25.956799 -80.205968,25.995449") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1245975, "cityName": "Miramar" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888637911171073, "text": "All the politicians have gone full retard over this, including @realDonaldTrump : https://t.co/bh6MnX2xIw #privacy #Apple #FBI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "privacy", "Apple", "FBI" }}, "user_mentions": {{ 25073877 }}, "user": { "id": 54628787, "name": "Michelle Denz", "screen_name": "mortalfemale", "lang": "en", "location": "California", "create_at": date("2009-07-07"), "description": "Graphic designer, illustrator, writer, libertarian, media enthusiast, principled thinker. Recently got into minimalism.", "followers_count": 70, "friends_count": 206, "statues_count": 95 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888638154444800, "text": "Can you recommend anyone for this #job? Family Practice Physician - Penrose-St. Francis Priamry Care -... - https://t.co/c70eIuUEkX #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.8213634,38.8338816"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Hiring" }}, "user": { "id": 71026662, "name": "CO Physician Jobs", "screen_name": "CO_physician", "lang": "en", "location": "Colorado", "create_at": date("2009-09-02"), "description": "Follow this account for geo-targeted Physician job tweets in Colorado Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 166, "friends_count": 133, "statues_count": 130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888638460698625, "text": "Tbh. https://t.co/9Auj5ypw78", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1671047233, "name": "Obi", "screen_name": "ObiTheChief", "lang": "en", "location": "houston", "create_at": date("2013-08-14"), "description": "Trust in him | UH | Nigerian", "followers_count": 1419, "friends_count": 1103, "statues_count": 13672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888638481670144, "text": "Neat! Man, I wish McDuffie could have head written a Ms. Marvel series. That would have been awesome. #LastRT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LastRT" }}, "user": { "id": 96256810, "name": "Jake V", "screen_name": "MetalShadowX", "lang": "en", "location": "Humble, TX", "create_at": date("2009-12-11"), "description": "A 24yo aspie with very simple interests: sarcasm, vidya games, cartoons, superheroes, comedy. Pokéfan & Dragon Ball nerd since 1998/99. A Smark as of Summer '15", "followers_count": 551, "friends_count": 547, "statues_count": 154688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639165472768, "text": "Finally was able to wash the damn car..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212433551, "name": "vɪииʏ", "screen_name": "Vinchizo", "lang": "en", "location": "Columbus, OH ", "create_at": date("2010-11-05"), "description": "Rɐndom Thoughts Ya Knoɯ.", "followers_count": 177, "friends_count": 259, "statues_count": 20922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639190499329, "text": "We're #hiring! Click to apply: Bartender - https://t.co/hcsFzsO5ju #Hospitality #SanAntonio, TX #Veterans #Job #Jobs #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4972715,29.5173414"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Hospitality", "SanAntonio", "Veterans", "Job", "Jobs", "CareerArc" }}, "user": { "id": 22489481, "name": "TMJ-SAT HRTA Jobs", "screen_name": "tmj_sat_hrta", "lang": "en", "location": "San Antonio, TX", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in San Antonio, TX. Need help? Tweet us at @CareerArc!", "followers_count": 433, "friends_count": 293, "statues_count": 743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639211433984, "text": "See our latest #Marietta, GA #job and click to apply: Senior Accountant- Exciting Role with Finance Exposure... - https://t.co/3RyB2S3RKl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4697663,33.9290987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Marietta", "job" }}, "user": { "id": 23236039, "name": "ATL Accounting Jobs", "screen_name": "tmj_atl_acct", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-03-07"), "description": "Follow this account for geo-targeted Accounting job tweets in Atlanta, GA. Need help? Tweet us at @CareerArc!", "followers_count": 560, "friends_count": 344, "statues_count": 150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1349756, "cityName": "Marietta" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639282782210, "text": "I just wanna chill gotta sack for us to roll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2213019236, "name": "Dustin'", "screen_name": "Swaggyp__", "lang": "en", "location": "4⃣0⃣5⃣", "create_at": date("2013-11-24"), "description": "I just wanna stay broke forever, yeah thats that shit no one ever said", "followers_count": 1192, "friends_count": 1147, "statues_count": 19493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shawnee, OK", "id": "5fe4a6e65f0e6014", "name": "Shawnee", "place_type": "city", "bounding_box": rectangle("-96.998368,35.289709 -96.868949,35.405381") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40125, "countyName": "Pottawatomie", "cityID": 4066800, "cityName": "Shawnee" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639475650560, "text": "Lmao https://t.co/Kmmz4v5QlZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 1149286435, "name": "No One", "screen_name": "GioRamirez87", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-02-04"), "description": "When kids look up to great scientists the way they do to great musicians and actors, civilization will jump to the next level #FutureMaterialsScientist #Chicano", "followers_count": 105, "friends_count": 116, "statues_count": 4765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-02-22T13:57:46.000Z"), "id": 701888639769247744, "text": "#Job alert: Data Migration/Software Architect | Ericsson | #Plano, Texas https://t.co/bI11RqldkH #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.6988856,33.0198431"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Job", "Plano", "Jobs", "Hiring" }}, "user": { "id": 37935874, "name": "Dallas, TX Jobs", "screen_name": "tmj_dfw_jobs", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-05-05"), "description": "Follow this account for geo-targeted Other job tweets in DFW, Texas. Need help? Tweet us at @CareerArc!", "followers_count": 1013, "friends_count": 541, "statues_count": 969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040193264541696, "text": "Oh the download had started!! https://t.co/kGReqNhXZC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481672438, "name": "Jobefuzz", "screen_name": "Jobefuzz", "lang": "en", "location": "Los Angeles", "create_at": date("2012-02-02"), "description": "null", "followers_count": 21, "friends_count": 40, "statues_count": 697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040194350915584, "text": "Wind 4.0 mph ENE. Barometer 1015.78 mb, Falling. Temperature 42.5 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040195483348992, "text": "Shane-o-mac and undertaker in Hell-in-A-cell at #WrestleMania? #YesPlease", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WrestleMania", "YesPlease" }}, "user": { "id": 475078775, "name": "David", "screen_name": "ChocolateFlaco", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "fubar", "followers_count": 613, "friends_count": 505, "statues_count": 43403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mercedes, TX", "id": "c9af03f7af638bed", "name": "Mercedes", "place_type": "city", "bounding_box": rectangle("-97.958308,26.123747 -97.863735,26.17475") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4847700, "cityName": "Mercedes" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040195734966273, "text": "Deadpool\nhttps://t.co/l6UvWjSnjz Meant to put this up sooner, but I can't recommend this movie enough.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383476656, "name": "Erick N. Romero", "screen_name": "ericknromero", "lang": "en", "location": "Austin", "create_at": date("2011-10-01"), "description": "Atheist, Vegan, Anarchist, Sex-Positive Feminist, Libertarian, Aspiring Everything, etc. Oh, also a hater of labels.", "followers_count": 530, "friends_count": 2007, "statues_count": 4186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040196527865860, "text": "03:00:01 |Temp: 56.0ºF | Wind Chill 56.0ºF |Dew Point 53.1ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NNW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040197224124418, "text": "Love all my brothers ✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4711132655, "name": "Zero ️", "screen_name": "SoyDiosMios", "lang": "en", "location": "DeKalb, IL", "create_at": date("2016-01-04"), "description": "It's Never Enough | ΦΚΘ | Snapchat: frankieatg", "followers_count": 299, "friends_count": 291, "statues_count": 263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DeKalb, IL", "id": "76cec30d4c553725", "name": "DeKalb", "place_type": "city", "bounding_box": rectangle("-88.792846,41.883102 -88.714573,41.972512") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1719161, "cityName": "DeKalb" } }
+{ "create_at": datetime("2016-02-23T00:00:00.000Z"), "id": 702040197370806272, "text": "Damn @suhmeduh dayum daniel remix is nearing 1 million plays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1110182340 }}, "user": { "id": 109690801, "name": "Alexander Tanaka", "screen_name": "NgoJustNgo", "lang": "en", "location": "San Jose, Fremont, Miami ", "create_at": date("2010-01-29"), "description": "The very basic core of a man's living spirit is his passion for adventure.", "followers_count": 924, "friends_count": 670, "statues_count": 28450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040197639249920, "text": "Goodnight! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832257439, "name": "Kakashi", "screen_name": "Angel76Flores", "lang": "en", "location": "Mount Pleasant, TX", "create_at": date("2012-09-18"), "description": "| إيمان | Art, fashion, nature, bands, vinyl", "followers_count": 427, "friends_count": 147, "statues_count": 23791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040197752422400, "text": "NIGHT EVERYONE IM OUUUUTTTTTT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.17937,33.002857"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363262720, "name": "CYBZZZZZ", "screen_name": "CybilHella", "lang": "en", "location": "Rancho Santa Fe, CA", "create_at": date("2014-02-26"), "description": "Is it Civil, like the Civil war? No. ///Actress?sorta. http://www.imdb.com/name/nm6474205/", "followers_count": 671, "friends_count": 873, "statues_count": 7320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Santa Fe (Fairbanks Ranch)", "id": "07d9e77c1a883000", "name": "Rancho Santa Fe (Fairbanks Ranch)", "place_type": "poi", "bounding_box": rectangle("-117.1793701,33.0028569 -117.17937,33.002857") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040197811167232, "text": "@ebbtideapp Tide in Dover Bridge, Maryland 02/23/2016\nHigh 6:30am 1.5\n Low 12:34pm -0.4\nHigh 7:13pm 1.7\n Low 1:20am -0.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.9983,38.7567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 33, "friends_count": 1, "statues_count": 10100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24041, "countyName": "Talbot" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040197974900737, "text": "Barberton Oh Temp:27.5°F Wind:0 mph Dir:NNE Baro:Falling slowly Rain2day:0.00in Hum:65% UV:0.0 @ 03:00 02/23/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040198096400384, "text": "Thank you god for blessing me with 21 years of life �������� #WhatsNinePlusTen #Legal #TU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhatsNinePlusTen", "Legal", "TU" }}, "user": { "id": 89851698, "name": "FEB23", "screen_name": "patrickoliza23", "lang": "en", "location": "562", "create_at": date("2009-11-13"), "description": "IG/SC: itspatrickstarr", "followers_count": 589, "friends_count": 618, "statues_count": 13449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040198339637248, "text": "I can't breathe https://t.co/RnwkzRCGR3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3061937874, "name": "M", "screen_name": "no_si_dam", "lang": "en", "location": "null", "create_at": date("2015-03-04"), "description": "Five foot one inch & Nocturnal ♻️♻️♻️ 1/10/07", "followers_count": 346, "friends_count": 265, "statues_count": 4430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Battle Ground, WA", "id": "a5457fd8ca260f1b", "name": "Battle Ground", "place_type": "city", "bounding_box": rectangle("-122.579747,45.763975 -122.516349,45.799033") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5304475, "cityName": "Battle Ground" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040198515822592, "text": "I ain't fw no rn. Like if i talk to you it's all just on a friendly level. So don't get my kindness mixed up with me flirting lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594318511, "name": "riah ✨", "screen_name": "riahsthename_", "lang": "en", "location": "null", "create_at": date("2012-05-29"), "description": "18 ♐. \n\n\n#RIPJessieHernandez ❤", "followers_count": 2020, "friends_count": 1262, "statues_count": 28596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CO", "id": "3f871475c095f94f", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-105.148925,39.819133 -104.987771,39.97944") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 883835, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040199019126784, "text": "it's so cute when geoff gets mad \nヽ(´o`;", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346335348, "name": "☁️hoe dameron☁️", "screen_name": "brideofdeIuxe", "lang": "en", "location": "conley|20|cu boulder|she/her", "create_at": date("2011-07-31"), "description": "do you mean like.... forever ever? forever ever? forever ever? forever ever? ... sure!", "followers_count": 278, "friends_count": 1010, "statues_count": 9730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040199178506244, "text": "I deff need a vacation already ��✈️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2915344909, "name": "naomi.gonzalezz", "screen_name": "naomii_gonzalez", "lang": "en", "location": "Los Organos, Guanajuato", "create_at": date("2014-11-30"), "description": "snapchat: naomi.gonzalezz", "followers_count": 200, "friends_count": 242, "statues_count": 4061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willits, CA", "id": "7f871a056f466c29", "name": "Willits", "place_type": "city", "bounding_box": rectangle("-123.364007,39.376929 -123.335284,39.424874") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6045, "countyName": "Mendocino", "cityID": 685600, "cityName": "Willits" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040199656771585, "text": "02/23@03:00 - Temp 36.8F, WC 36.8F. Wind 1.0mph NE, Gust 2.0mph. Bar 30.229in, Falling slowly. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040199765762048, "text": "Scum of the �� https://t.co/qQSmS3CXsM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3520563319, "name": "Unbothered..", "screen_name": "ShesFree_", "lang": "en", "location": "null", "create_at": date("2015-09-10"), "description": "Reinvented, Redefined, and Riveting #UNCO19 . SC: Shesangs", "followers_count": 233, "friends_count": 397, "statues_count": 2790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040199979786241, "text": "0h 10m wait time at UNIVERSITY HOSPITAL & MEDICAL CENTER. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.252623,26.211459"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 360, "friends_count": 884, "statues_count": 2788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamarac, FL", "id": "fdddb6198f42cf7d", "name": "Tamarac", "place_type": "city", "bounding_box": rectangle("-80.298308,26.178836 -80.162961,26.230749") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1270675, "cityName": "Tamarac" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040200092864512, "text": "I love surprises ☺️����❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25748301, "name": "Meghan Sanett", "screen_name": "mmsanett", "lang": "en", "location": "los angeles", "create_at": date("2009-03-21"), "description": "dancer. coffee addict. certified weirdo. instagram & vine: @mmsanett || bookings: meghan@boomopolis.com #yaaaas #werk", "followers_count": 15716, "friends_count": 540, "statues_count": 26122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040200164175872, "text": "Wind 4.0 mph SSE. Barometer 30.078 in, Falling slowly. Temperature 32.3 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040200562798592, "text": "Wind 3.0 mph NE. Barometer 30.010 in, Falling slowly. Temperature 37.6 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-23T00:00:01.000Z"), "id": 702040201586196480, "text": "#SupportOriginMelissa 40.5°F Wind:0.0mph Pressure: 29.91hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 310575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040201904820224, "text": "@ryannyelder �������� gotta be saucy walkin up to the plate", "in_reply_to_status": 702040026939396096, "in_reply_to_user": 2432420480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2432420480 }}, "user": { "id": 100907757, "name": "kenz", "screen_name": "Keennzz_", "lang": "en", "location": "ShineInPeaceToAllMyAngels", "create_at": date("2009-12-31"), "description": "#TEAMJOEY #ripJKM. freedaedae❤️", "followers_count": 965, "friends_count": 1071, "statues_count": 74154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040202135535616, "text": "5. & then!!!! I've had this uncomfortable pain on the right side of my lower stomach since I got home from school", "in_reply_to_status": 702039951131570176, "in_reply_to_user": 3014137003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3014137003, "name": "LoLo ✨", "screen_name": "_loowwiee", "lang": "en", "location": "Norwalk, CA", "create_at": date("2015-02-08"), "description": "notice me // take my hand", "followers_count": 679, "friends_count": 635, "statues_count": 6282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040202630471680, "text": "@fcksnk TEHE c:", "in_reply_to_status": 702040049517535232, "in_reply_to_user": 4030718539, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 4030718539 }}, "user": { "id": 3184997011, "name": "kales!", "screen_name": "kayliejzt13", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-05-04"), "description": "god is enough", "followers_count": 326, "friends_count": 120, "statues_count": 6344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040203142115330, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1043430642, "name": "$", "screen_name": "bsevxo", "lang": "en", "location": "From The West Side With Love", "create_at": date("2012-12-28"), "description": "yours truly", "followers_count": 544, "friends_count": 413, "statues_count": 47454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040203230371841, "text": "Wind 0.0 mph ---. Barometer 30.037 in, Steady. Temperature 33.6 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204174082048, "text": "Temp: 62.1°F Wind:0.0mph Pressure: 29.787hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204207648768, "text": "Wind 0.0 mph ---. Barometer 29.84 in, Falling. Temperature 61.7 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204354461696, "text": "That's art, all jokes aside. https://t.co/Q7yTy0ZNJk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3054198217, "name": "Tune.☁️", "screen_name": "ImTune_YouJig", "lang": "en", "location": "Conway, AR", "create_at": date("2015-03-01"), "description": "What's the problem? Why you not smiling, we're in the prime of our life.. Snapchat: Montorey_j #UCA", "followers_count": 1379, "friends_count": 1032, "statues_count": 6228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204522102788, "text": "59.8F (Feels: 59.8F) - Humidity: 99% - Wind: 0.7mph E - Gust: 0.7mph - Pressure: 1017.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 227999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204681424896, "text": "@Cam_Trill cam with the slam?", "in_reply_to_status": 702039880004554752, "in_reply_to_user": 168965995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168965995 }}, "user": { "id": 251870722, "name": "Justin", "screen_name": "okayjayyjones", "lang": "en", "location": "HTX", "create_at": date("2011-02-13"), "description": "University of Houston", "followers_count": 2650, "friends_count": 1392, "statues_count": 71908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204719226880, "text": "In that 90210", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4264733474, "name": "Conrad", "screen_name": "DConrad0", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2015-11-16"), "description": "Sowela. #Trump2016. https://vine.co/v/iqBiqupMvVI https://twitter.com/someonesanidiot/status/694181289398763520", "followers_count": 234, "friends_count": 612, "statues_count": 4442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prien, LA", "id": "00a028c043bb68f0", "name": "Prien", "place_type": "city", "bounding_box": rectangle("-93.283003,30.106667 -93.213369,30.168682") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2262647, "cityName": "Prien" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040204840931328, "text": "@KelseyAlvarez you follow the wrong people.", "in_reply_to_status": 702040016281858052, "in_reply_to_user": 258179117, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 258179117 }}, "user": { "id": 555517603, "name": "Views x3", "screen_name": "VersaceSilk", "lang": "en", "location": "Jersey", "create_at": date("2012-04-16"), "description": "I love all my exes. The feeling isn't mutual", "followers_count": 10335, "friends_count": 1578, "statues_count": 89482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hackensack, NJ", "id": "a64b09ad02cb5c61", "name": "Hackensack", "place_type": "city", "bounding_box": rectangle("-74.06836,40.858471 -74.024444,40.913461") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3428680, "cityName": "Hackensack" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040205012766721, "text": "I sleep now and sleep for 4 hours or I don't sleep at all and just stay up all day again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2999777171, "name": "Havyn Ramirez", "screen_name": "havynnn", "lang": "en", "location": "some place higher", "create_at": date("2015-01-26"), "description": "musical anarchist/the originator lover", "followers_count": 510, "friends_count": 428, "statues_count": 19811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040205071671297, "text": "�� https://t.co/fvY0Dxq20a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 264568504, "name": "Dutty Boi D", "screen_name": "execteamcoo", "lang": "en", "location": "Greensboro, NC", "create_at": date("2011-03-11"), "description": "Heavy is the head that wears the crown. C.O.O of @Executive_Team. Proud member of PHI SLAMMA JAMMA #MisFit I R.I.P Grandma I 2 2000+ PARTIES THROWN", "followers_count": 17878, "friends_count": 14189, "statues_count": 281952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040205188997120, "text": "SMH.....this is a thot...... https://t.co/UeQjtrhzTo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 757509738, "name": "Sir Katz to you!", "screen_name": "KatzSbk94", "lang": "en", "location": "California", "create_at": date("2012-08-14"), "description": "21 Fullerton College. Smile it'll confuse people! It's the little things that matter most!", "followers_count": 479, "friends_count": 844, "statues_count": 28144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040205264556033, "text": "#fitnessmotivation https://t.co/iZ62rj8gRG https://t.co/sOpWP4zgVs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "fitnessmotivation" }}, "user": { "id": 1664691378, "name": "Lola Luna", "screen_name": "LolaLunaEurope", "lang": "pl", "location": "Global", "create_at": date("2013-08-12"), "description": "Welcome to the official feed for Lola Luna lingerie and beachwear brand synonymous with Made in France luxury.", "followers_count": 5099, "friends_count": 5584, "statues_count": 6288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:02.000Z"), "id": 702040205709156353, "text": "Travelling to Pass Christian or just twittering about Pass Christian? https://t.co/Wv90nv6e5N #Pass Christian", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.2475,30.3158"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Pass" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1028, "friends_count": 311, "statues_count": 2613134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pass Christian, MS", "id": "06e9911667152ffa", "name": "Pass Christian", "place_type": "city", "bounding_box": rectangle("-89.287081,30.304097 -89.186749,30.343339") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2855400, "cityName": "Pass Christian" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040206031982592, "text": "future or thug https://t.co/DrD4RRewRc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1466295068, "name": "mckenzie", "screen_name": "mckeeenzie", "lang": "en", "location": "Sacramento, CA", "create_at": date("2013-05-28"), "description": "robert phillips is my boo thang", "followers_count": 1054, "friends_count": 611, "statues_count": 26830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Highlands, CA", "id": "ee9341f90ab895d5", "name": "North Highlands", "place_type": "city", "bounding_box": rectangle("-121.424332,38.638355 -121.335815,38.704481") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 651924, "cityName": "North Highlands" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040206157996035, "text": "meanwhile you fuckheads out here growing white hairs trying to make a hit https://t.co/XRzhzpQDtC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 104171936, "name": "Ock Religion", "screen_name": "ockreligion", "lang": "en", "location": "TOKYO", "create_at": date("2010-01-12"), "description": "こんにちは混合トレンドの世界へ", "followers_count": 351, "friends_count": 263, "statues_count": 12272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040206346563585, "text": "Nightly promise: \nI promise to get up as soon as my alarm clock goes off the first time. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2281645422, "name": "{Crystal}", "screen_name": "crys9041", "lang": "en", "location": "null", "create_at": date("2014-01-07"), "description": "Expert at being a cotton headed ninny muggin'", "followers_count": 207, "friends_count": 287, "statues_count": 4564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040206816350208, "text": "hey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2580713891, "name": "lex", "screen_name": "lexieepitcherr", "lang": "en", "location": "null", "create_at": date("2014-06-03"), "description": "just be happy", "followers_count": 1241, "friends_count": 1000, "statues_count": 7029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040207164485632, "text": "Wind 0.7 mph NE. Barometer 30.38 in, Steady. Temperature 18.1 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 112, "statues_count": 157729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040207441330177, "text": "@NotJoseph_ you like to be a bitch and comment unnecessary things", "in_reply_to_status": 702040051509653505, "in_reply_to_user": 618436569, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 618436569 }}, "user": { "id": 981832944, "name": "$ushi Papi", "screen_name": "KawaiiKalasa", "lang": "en", "location": "null", "create_at": date("2012-11-30"), "description": "Sad Dad Co • Photographer • UrbanAC", "followers_count": 855, "friends_count": 671, "statues_count": 32315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040207634231297, "text": "I was gone #imessage you , but idk about you , you might spazz on me https://t.co/wrQZ9tHRL8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "imessage" }}, "user": { "id": 1471399370, "name": "Im Splashin On Yu", "screen_name": "Jose__Gordo", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2013-05-30"), "description": "saucing & splashing in high fashion!", "followers_count": 633, "friends_count": 630, "statues_count": 8850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040207822958592, "text": "Full moon warms me in blues....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281650443, "name": "Sariyah Idan", "screen_name": "sariyahidan", "lang": "en", "location": "LA based, NY bred", "create_at": date("2011-04-13"), "description": "My Weapon Is Love | singer-songwriter | actor-playwright | Deeper Than Skin #EP on iTunes | Not Afraid #mixtape on Soundcloud & Bandcamp | reclaiming #jewess", "followers_count": 3244, "friends_count": 1176, "statues_count": 4229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040207822970884, "text": "@Wrath2X IMO Snyder made the best possible Watchmen movie aside from Ozy casting. I'll give this a shot, even though I hated Man of Steel.", "in_reply_to_status": 702039539624366080, "in_reply_to_user": 246226332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 246226332 }}, "user": { "id": 47787970, "name": "Rob Young", "screen_name": "polyh3dron", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-16"), "description": "http://prodby.altervista.org/producer_list.php?producer=Polyhedron\nI also write for http://CombatPress.com\nOpinions are my own.", "followers_count": 636, "friends_count": 1325, "statues_count": 29461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040208003305472, "text": "@Vandiekins22 that was just slightly over kill.", "in_reply_to_status": -1, "in_reply_to_user": 444319535, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 444319535 }}, "user": { "id": 21712366, "name": "JoAnna Montoya", "screen_name": "TeamFraserAZ", "lang": "en", "location": "Corner of Hell and BFE!", "create_at": date("2009-02-23"), "description": "Fan of Movies, Tv Shows, Books, and Video Games. Supernatural, Arrow, Outlander, OUAT, Doctor Who, and much more!", "followers_count": 321, "friends_count": 293, "statues_count": 12014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040208200437760, "text": "U gon regret not inviting me to your birthday when I become famous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842621053, "name": "yashm", "screen_name": "venuseos", "lang": "en", "location": "aztlan", "create_at": date("2012-09-23"), "description": "existo porque resisto | Wixáritari y Otomí", "followers_count": 1441, "friends_count": 538, "statues_count": 88547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040208280170496, "text": "the broski came thru wit panda ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3194944860, "name": "Erik Torres", "screen_name": "wassup_831", "lang": "en", "location": "Watsonville, CA", "create_at": date("2015-05-13"), "description": "hatred and attitude tear us entirely", "followers_count": 173, "friends_count": 177, "statues_count": 14689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040208724742144, "text": "Be patient. \nGod isn't finished with you yet.\nPhilippians 1:6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 121, "friends_count": 235, "statues_count": 172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040209077067776, "text": "he so beautiful https://t.co/cc6Zq4DWe0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3220994388, "name": "ä", "screen_name": "vibewithash", "lang": "en", "location": "bay area", "create_at": date("2015-05-19"), "description": "cancer", "followers_count": 277, "friends_count": 186, "statues_count": 8993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Pablo, CA", "id": "04016a4890553832", "name": "San Pablo", "place_type": "city", "bounding_box": rectangle("-122.358758,37.948637 -122.318412,37.981558") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 668294, "cityName": "San Pablo" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040209244995585, "text": "@heathurrr_xo you should be a housewife with this last statement!", "in_reply_to_status": 702040097428938752, "in_reply_to_user": 1017102937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1017102937 }}, "user": { "id": 35165330, "name": "Uh Huh. Ok.", "screen_name": "mrcorkles", "lang": "en", "location": "Chicago", "create_at": date("2009-04-24"), "description": "Don't get mad. I'm not serious. GMO and gluten free sustainable Af Am homo here. Game Of Thrones. Getting On. Jessica Jones. Narcos. HTGAWM. RH. Transparent", "followers_count": 226, "friends_count": 548, "statues_count": 10938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040209794334721, "text": "2:00 am ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2655908610, "name": "n", "screen_name": "Nayelaay_", "lang": "en", "location": "atx", "create_at": date("2014-07-17"), "description": "❤️", "followers_count": 648, "friends_count": 253, "statues_count": 26957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-23T00:00:03.000Z"), "id": 702040209945276418, "text": "they're so fly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81464760, "name": "J.", "screen_name": "_JadaJ", "lang": "en", "location": "bmtx | pv18", "create_at": date("2009-10-10"), "description": "is what it is. &#127803; sc:jadaa_j", "followers_count": 2035, "friends_count": 999, "statues_count": 122253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040210075353090, "text": "Hbd dad ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2453571235, "name": "Pricila ❥", "screen_name": "pricila_lopez3", "lang": "en", "location": "null", "create_at": date("2014-04-19"), "description": "null", "followers_count": 610, "friends_count": 330, "statues_count": 18216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040210515701760, "text": "@Rtfor3 let the scorecard show, I beat you by strokes. My bars are like lava, your raps are a hoax. #RyanSupaNot", "in_reply_to_status": 702038177591111681, "in_reply_to_user": 325916392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RyanSupaNot" }}, "user_mentions": {{ 325916392 }}, "user": { "id": 523918205, "name": "Jake Green", "screen_name": "Jake_AD28", "lang": "en", "location": "null", "create_at": date("2012-03-13"), "description": "Life is a garden, dig it - Joe Dirt", "followers_count": 245, "friends_count": 202, "statues_count": 1206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clear Lake, IA", "id": "4a349d6d2dcedf6e", "name": "Clear Lake", "place_type": "city", "bounding_box": rectangle("-93.438289,43.107835 -93.320024,43.169698") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19033, "countyName": "Cerro Gordo", "cityID": 1914025, "cityName": "Clear Lake" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040210876538880, "text": "@saeromz just letting you know you misspelled something, but k", "in_reply_to_status": 702039887160152065, "in_reply_to_user": 240061265, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240061265 }}, "user": { "id": 1428354494, "name": "Min Kim (화랑)", "screen_name": "Hwarangatang", "lang": "en", "location": "null", "create_at": date("2013-05-14"), "description": "null", "followers_count": 152, "friends_count": 119, "statues_count": 3822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040211174199296, "text": "When my best friend texts me \"hi\" at 1am >>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72686493, "name": "whitney", "screen_name": "whitneyburke11", "lang": "en", "location": "null", "create_at": date("2009-09-08"), "description": "smoke fruit, eat pot", "followers_count": 825, "friends_count": 344, "statues_count": 19863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040211224530944, "text": "macklin... u son of a bitch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141626420, "name": "softest hakyeon stan", "screen_name": "softhakyeon", "lang": "en", "location": "missouri | he/him | ♊ | INFP ", "create_at": date("2010-05-08"), "description": "adrian. elf, st☆rlight, bestiny. x-men. DIGIMON. i talk abt rp a lot. gender: kim heechul | i'm in vixx hell and i love it | #10yearswithsuju | i love moonbyul", "followers_count": 342, "friends_count": 901, "statues_count": 123534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrensburg, MO", "id": "95f975ecaafb76ad", "name": "Warrensburg", "place_type": "city", "bounding_box": rectangle("-93.800937,38.72244 -93.69101,38.785966") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29101, "countyName": "Johnson", "cityID": 2977092, "cityName": "Warrensburg" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040211270688770, "text": "Temp: 32.0°F - Dew Point: 27.8° - Wind: 0.5 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 30.09in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 13841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040211308544000, "text": "Wind 0.0 mph N. Barometer 30.409 in, Steady. Temperature 4.4 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040211467796480, "text": "These 2 big ass pimples on my forehead need to go..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1592656904, "name": "dama ❥", "screen_name": "damabepoppin", "lang": "en", "location": "null", "create_at": date("2013-07-13"), "description": "♏️", "followers_count": 632, "friends_count": 436, "statues_count": 26532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Live Oak, CA", "id": "6669a9f1a235a4bb", "name": "Live Oak", "place_type": "city", "bounding_box": rectangle("-121.678704,39.255526 -121.645035,39.291653") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 641936, "cityName": "Live Oak" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040212046639104, "text": "https://t.co/XDkevCFMfH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3025306807, "name": "tuh-la-n-uh", "screen_name": "lgbqtomlinson", "lang": "en", "location": "up harry's ass", "create_at": date("2015-02-18"), "description": "louis twinklinson", "followers_count": 386, "friends_count": 442, "statues_count": 11148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040212227153920, "text": "@RobertBayBay just hmu bb", "in_reply_to_status": 702040158309191680, "in_reply_to_user": 542819335, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 542819335 }}, "user": { "id": 799624982, "name": "Your Dad", "screen_name": "alexis143u1", "lang": "en", "location": "Napa,Caliii", "create_at": date("2012-09-02"), "description": "SC:Alexis143u1 #ViscaBarca", "followers_count": 388, "friends_count": 282, "statues_count": 17291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040212570963968, "text": "I love ww2 documentaries so much! Just history fascinates me so much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2232062102, "name": "nutterbutterbuster", "screen_name": "biggpurro47", "lang": "en", "location": "bootyville usa", "create_at": date("2013-12-05"), "description": "solo do lo.\nAmerican Airman", "followers_count": 95, "friends_count": 180, "statues_count": 2461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tomball, TX", "id": "00028b6863e4a97b", "name": "Tomball", "place_type": "city", "bounding_box": rectangle("-95.662144,30.066754 -95.576814,30.135969") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4873316, "cityName": "Tomball" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040212839518208, "text": "Wind 0 mph W. Barometer 29.99 in, Rising slowly. Temperature 66.6 °F. Rain today 0.02 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 55, "friends_count": 72, "statues_count": 25920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040213019824128, "text": "What a waste of a night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23211353, "name": "sarah strange", "screen_name": "_nighthawk69_", "lang": "en", "location": "Astoria, Queens, NY", "create_at": date("2009-03-07"), "description": "I have an affinity for all things that resonate with my last name. scorpio.", "followers_count": 739, "friends_count": 2097, "statues_count": 41407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040213854392320, "text": "@FUNimation \"I was young! I needed the Dragon Balls!\" https://t.co/akhZjKwPy6", "in_reply_to_status": -1, "in_reply_to_user": 17444764, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17444764 }}, "user": { "id": 614005930, "name": "Matthew Johnson", "screen_name": "CrookBu41", "lang": "en", "location": "null", "create_at": date("2012-06-20"), "description": "null", "followers_count": 26, "friends_count": 263, "statues_count": 172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-23T00:00:04.000Z"), "id": 702040214013739009, "text": "Wind 3.0 mph ENE. Barometer 30.098 in, Steady. Temperature 28.6 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 379, "friends_count": 291, "statues_count": 8485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040214433181697, "text": "happy birthday to this beautiful girl that I aspire to be the most, I love you so much @dixonrian ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615824654 }}, "user": { "id": 3338965274, "name": "bessie dixon", "screen_name": "alien_mom99", "lang": "en", "location": "CA", "create_at": date("2015-08-25"), "description": "LHHS & WACC", "followers_count": 694, "friends_count": 224, "statues_count": 1241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040214806507520, "text": "I thought completely wrong lmao but that's what I get just hope you're happy over at your moms. .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2602223909, "name": "1RuinedTib_", "screen_name": "deleon_ezequiel", "lang": "en", "location": "Wichita M'F Kansas", "create_at": date("2014-06-11"), "description": "null", "followers_count": 995, "friends_count": 1630, "statues_count": 4777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040214827458560, "text": "Welcome to the land of 6000 degrees", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2896251019, "name": "Captain $auce❄️", "screen_name": "Jewlian_626", "lang": "en", "location": "Rialto/La puente", "create_at": date("2014-11-10"), "description": "lascia Che io ti ami. #GetBuck (sc: sexy_bitch626) {ig:king_jewlian} #StreetWarrior #GothMoneySouljah", "followers_count": 201, "friends_count": 455, "statues_count": 6517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040215624380416, "text": "@TokerGal my Colorado vacation smoking spot �� well 1 of many https://t.co/tJeiiTrr9O", "in_reply_to_status": 702030409845051392, "in_reply_to_user": 4531101323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4531101323 }}, "user": { "id": 1374749341, "name": "Bryson Kounovsky", "screen_name": "BrysonJames18", "lang": "en", "location": "Omaha, NE", "create_at": date("2013-04-23"), "description": "Blessed from up above", "followers_count": 577, "friends_count": 1053, "statues_count": 19158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040215800647681, "text": "Join the them.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4573379414, "name": "DJ Slim Goody", "screen_name": "EarsToThaStreet", "lang": "en", "location": "null", "create_at": date("2015-12-16"), "description": "Mixtape and Radio DJ. Inquiries (304)635-8581", "followers_count": 282, "friends_count": 614, "statues_count": 2679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinsburg, WV", "id": "d5039bee9eb93f2c", "name": "Martinsburg", "place_type": "city", "bounding_box": rectangle("-78.028084,39.426663 -77.94245,39.494633") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54003, "countyName": "Berkeley", "cityID": 5452060, "cityName": "Martinsburg" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040215804706816, "text": "Gosh damn I also have a weirdo stalker hashtag #CanISpotyou lol uhhh no ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CanISpotyou" }}, "user": { "id": 153060884, "name": "Evony", "screen_name": "MoonlightBeb", "lang": "en", "location": " †snap:@evonyxoxo Ig: Indieevv", "create_at": date("2010-06-07"), "description": "null", "followers_count": 237, "friends_count": 345, "statues_count": 3580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040216232550401, "text": "let me eat ya ass baby boy https://t.co/fZw6o8Zs4B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938530766, "name": "sassy sweetheart", "screen_name": "chelsthepimp", "lang": "en", "location": "ling ling in the cut ", "create_at": date("2014-12-23"), "description": "just lemme luhh you baby boy • chamorro", "followers_count": 1005, "friends_count": 650, "statues_count": 13243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040216429862912, "text": "@vanesssaa13 @Gee_Bites @Mermaid_1126 lol.", "in_reply_to_status": 702039698835779584, "in_reply_to_user": 91408168, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 91408168, 120911055, 576024819 }}, "user": { "id": 1716002671, "name": "Daniel Rivas", "screen_name": "daniel_rivas13", "lang": "en", "location": "null", "create_at": date("2013-08-31"), "description": "Army Solider & 2015 WesCon Champions", "followers_count": 713, "friends_count": 170, "statues_count": 6622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040217063026688, "text": "@aaronshatto_ @BaSourD vegetable ��", "in_reply_to_status": 702040072216940544, "in_reply_to_user": 41685452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41685452, 3017748843 }}, "user": { "id": 365834668, "name": "dude", "screen_name": "AtNameChef", "lang": "en", "location": "Houston, TX", "create_at": date("2011-08-31"), "description": "Emma. With birds i share this lonely view. Auburn/Rockets/Panthers/Astros", "followers_count": 1191, "friends_count": 601, "statues_count": 51733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040217100820480, "text": "happy birthday boo�� luvv you so much, seriously don't know what I'd do w/o you. I hope you have a gr8 day❣@ih8acrx https://t.co/wUokceFXBP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2244805958 }}, "user": { "id": 2535896765, "name": "jasmine", "screen_name": "jaaasmineg", "lang": "en", "location": "bay area ", "create_at": date("2014-05-08"), "description": "fredy nieto♡", "followers_count": 263, "friends_count": 228, "statues_count": 6081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040217289490432, "text": "As time goes by ....���� @PHORAONE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68907936 }}, "user": { "id": 844848776, "name": "Juan-800", "screen_name": "Juan_ocho5", "lang": "en", "location": "Riverdale", "create_at": date("2012-09-24"), "description": "/ Life's a trip Snapchat / : @Juanito_FPR \n#FCC #PreludeNation / Car Enthusiast /", "followers_count": 500, "friends_count": 341, "statues_count": 21401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverdale, CA", "id": "0028563c445e3c67", "name": "Riverdale", "place_type": "city", "bounding_box": rectangle("-119.878071,36.423728 -119.845512,36.438219") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 661096, "cityName": "Riverdale" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040217520242688, "text": "yea it's in my mouth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3571454423, "name": "Brian Dawson", "screen_name": "TheNatural301", "lang": "en", "location": "Maryland, USA", "create_at": date("2015-09-06"), "description": "playing the corners where the washed up old heads chant tunes by The Whispers", "followers_count": 126, "friends_count": 415, "statues_count": 1560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederick, MD", "id": "ec3b8b62828d8f6a", "name": "Frederick", "place_type": "city", "bounding_box": rectangle("-77.476711,39.369128 -77.359293,39.494945") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2430325, "cityName": "Frederick" } }
+{ "create_at": datetime("2016-02-23T00:00:05.000Z"), "id": 702040217721516033, "text": "63-0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 168981992, "name": "Don Don", "screen_name": "_Donnnnn", "lang": "en", "location": "null", "create_at": date("2010-07-21"), "description": "I knew I was gonna be something because I came from nothing! #LSU16 #FutureSportsAgent", "followers_count": 1988, "friends_count": 1692, "statues_count": 56916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040219562938368, "text": "Happy birthday to the man who never fails to put a smile on my face. Randall, every single… https://t.co/9dYzeV0A5y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.30145709,38.3003649"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4194547093, "name": "Becz", "screen_name": "theweedfairy209", "lang": "en", "location": "California, USA", "create_at": date("2015-11-10"), "description": "The mind is everything. What you think, you become -Buddha.\n#wubz #squadventures #takemetothebeach", "followers_count": 94, "friends_count": 201, "statues_count": 653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040219856429057, "text": "I'm not cut out for this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2358873038, "name": "kayso❥", "screen_name": "katiejgoss", "lang": "en", "location": "the great outdoors", "create_at": date("2014-02-23"), "description": "ramble on✿MVC✿ @jdumas12 ♡", "followers_count": 590, "friends_count": 331, "statues_count": 16764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040220179439617, "text": "Haven't been to Lees class for like 3 weeks lmao i just leave after 4th", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2473303524, "name": "Goofy Jeo", "screen_name": "Alvawrexz", "lang": "en", "location": "null", "create_at": date("2014-05-01"), "description": "Life Motto: A Hoe Never Gets Cold", "followers_count": 130, "friends_count": 109, "statues_count": 5590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040220196147200, "text": "@Palaaardaniella like a small D or something I want them to look natural but we'll see what the doctor says I should go with ����", "in_reply_to_status": 702039825663156224, "in_reply_to_user": 2256205867, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2256205867 }}, "user": { "id": 2263945309, "name": "Jasmine", "screen_name": "jasmine_smithhh", "lang": "en", "location": "null", "create_at": date("2013-12-26"), "description": "Paradise", "followers_count": 137, "friends_count": 204, "statues_count": 7064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, CA", "id": "8eccead85cc1b6c0", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-121.260264,38.621443 -121.085432,38.714331") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040220468797442, "text": "@TravisHolcombe new Tweet", "in_reply_to_status": 702032169271578625, "in_reply_to_user": 158850481, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 158850481 }}, "user": { "id": 2482920776, "name": "Kharis Catchings", "screen_name": "Bullish72", "lang": "en", "location": "Galactic ", "create_at": date("2014-05-07"), "description": "I am a Taurus and that's all you need to know.", "followers_count": 315, "friends_count": 2192, "statues_count": 490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040220489928708, "text": "I can't ever sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572148988, "name": "colby", "screen_name": "cxlby", "lang": "en", "location": "DFW", "create_at": date("2012-05-05"), "description": "The champion. / Z33", "followers_count": 1574, "friends_count": 531, "statues_count": 17379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040220972118016, "text": "������ avi ������ https://t.co/TWHxVn6D0V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1561575578, "name": "veronica", "screen_name": "viewsfromveroni", "lang": "en", "location": "TX ✈️ UT", "create_at": date("2013-07-01"), "description": "hogwarts school of witchcraft and wizardry || @sandovaleslie", "followers_count": 586, "friends_count": 217, "statues_count": 15701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millcreek, UT", "id": "5af91023ed39df89", "name": "Millcreek", "place_type": "city", "bounding_box": rectangle("-111.921221,40.664889 -111.839259,40.712147") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4950150, "cityName": "Millcreek" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221139927041, "text": "Fuck https://t.co/CdhcAcUkDO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3058681316, "name": "haley", "screen_name": "hhhales", "lang": "en", "location": "null", "create_at": date("2015-03-02"), "description": "ig: hhhhales", "followers_count": 530, "friends_count": 371, "statues_count": 6529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221165039616, "text": "or your white vans https://t.co/ZYqoSUI7oz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 89391412, "name": "yesenia victoria", "screen_name": "yesvictoria", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-11-11"), "description": "http://ycastorena.tumblr.com", "followers_count": 1250, "friends_count": 372, "statues_count": 13019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221458759680, "text": "Gotta mean what you say", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2934740309, "name": "BiNo", "screen_name": "lavishBiNoBBG", "lang": "en", "location": "wea da cash @", "create_at": date("2014-12-20"), "description": "wen i was younger i was told that i could have it all, so that's exactly what I'm gon get", "followers_count": 822, "friends_count": 672, "statues_count": 16946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221542539265, "text": "@SpelunkerSal I contributed!", "in_reply_to_status": 702037818332180480, "in_reply_to_user": 199369138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 199369138 }}, "user": { "id": 2859842273, "name": "Wibbeh", "screen_name": "WbbFuzz", "lang": "en", "location": "Tustin, CA", "create_at": date("2014-11-03"), "description": "Now with 150% more fluff!", "followers_count": 609, "friends_count": 108, "statues_count": 2650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221697748992, "text": "(#Christ) 20:20 #Hecatean #Tree\n#Ni #Me K♠\n#Ko #Ra 8♠,10♦\n#Hu #By 9♦,10♠,8♦\n#Vy #Su 9♣,9♠,K♦,A♦\n66/6\n♠♥♦♣\nMonica Crowley 808\n#Short 99\n#Pods", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Christ", "Hecatean", "Tree", "Ni", "Me", "Ko", "Ra", "Hu", "By", "Vy", "Su", "Short", "Pods" }}, "user": { "id": 70626091, "name": "Left Hand Pan", "screen_name": "666Beastism999", "lang": "en", "location": "Spokane, WA", "create_at": date("2009-09-01"), "description": "Messiah, The Christ, Tezcatlipoca Card, Ace of Jihad, Sanctified Interface", "followers_count": 358, "friends_count": 276, "statues_count": 216318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040221982916609, "text": "Wind 0.0 mph E. Barometer 30.295 in, Steady. Temperature 19.9 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-23T00:00:06.000Z"), "id": 702040222368980992, "text": "lmaooo, you aint real dawg https://t.co/EylaUqyRFp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53000782, "name": "G® From OP‼️", "screen_name": "JG_Trilll", "lang": "en", "location": "Opelousas☀️", "create_at": date("2009-07-02"), "description": "been jiggy, been pretty | YWA | #LSU19 ❌", "followers_count": 2910, "friends_count": 2097, "statues_count": 48038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-23T00:00:07.000Z"), "id": 702040223664832512, "text": "����omg nooo https://t.co/YExMLlEg3R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3115740859, "name": "val", "screen_name": "ValllValerie", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "proverbs 4:23 ✨", "followers_count": 917, "friends_count": 797, "statues_count": 24526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402581679742976, "text": "03:00:02 |Temp: 63.5ºF | Wind Chill 63.5ºF |Dew Point 60.2ºF | Rain today: 0.31 inches | Wind: 6.0 mph from the ESE, Gusting to 10.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402582317289473, "text": "So Ima win regardless", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116862205, "name": "Curve-A-Saurus Rex", "screen_name": "CoolGuyWho", "lang": "en", "location": "California's Dungeon ", "create_at": date("2010-02-23"), "description": "A Real Man handles Business...", "followers_count": 867, "friends_count": 386, "statues_count": 53893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402582854168576, "text": "このあとアイフォン盗まれるとはまだ知りまてん https://t.co/9F7oc6wETQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 567864085, "name": "HARUKA", "screen_name": "harukyaaaaa", "lang": "ja", "location": "Santa Barbara", "create_at": date("2012-04-30"), "description": "Seize the Day.", "followers_count": 679, "friends_count": 546, "statues_count": 15155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Barbara, CA", "id": "f6ebc676e5cde864", "name": "Santa Barbara", "place_type": "city", "bounding_box": rectangle("-119.763341,34.395525 -119.639931,34.464245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669070, "cityName": "Santa Barbara" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402582900318210, "text": "A personal mentor would be niiiiiiiiiiice right about now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253295004, "name": "Mariah Turner", "screen_name": "msmariahh_", "lang": "en", "location": "San Jose, CA ", "create_at": date("2011-02-16"), "description": "SJSU 2016 • Public Relations • President @PRSSASJSU • Self Motivated • #DUBNATION", "followers_count": 832, "friends_count": 530, "statues_count": 16223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583151910916, "text": "Over this shit always popping your gums some times you should just shut the fuck up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114954730, "name": "Misunderstood!", "screen_name": "ChunkyandPretty", "lang": "en", "location": "American Canyon, CA", "create_at": date("2010-02-16"), "description": "null", "followers_count": 825, "friends_count": 1160, "statues_count": 18749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583349088256, "text": "Dope https://t.co/TMKvDuntdc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64237538, "name": "Mace Windu", "screen_name": "HumbleButHUNGRY", "lang": "en", "location": "Camden, New Jersey", "create_at": date("2009-08-09"), "description": "23 Football. Film. Photography. Bryant University Alum. Personal account", "followers_count": 1284, "friends_count": 912, "statues_count": 107694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlton, NJ", "id": "860637a8dc169e00", "name": "Marlton", "place_type": "city", "bounding_box": rectangle("-74.957163,39.855644 -74.862132,39.919853") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3444100, "cityName": "Marlton" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583370047488, "text": "The most painful goodbyes are the ones that are never said and never explained.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560803026, "name": "mairead", "screen_name": "MaireadDeCourse", "lang": "en", "location": "null", "create_at": date("2012-04-22"), "description": "b-town", "followers_count": 325, "friends_count": 225, "statues_count": 28776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summit, NJ", "id": "789a596e07b40baf", "name": "Summit", "place_type": "city", "bounding_box": rectangle("-74.402023,40.685322 -74.330621,40.739209") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3471430, "cityName": "Summit" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583797899268, "text": "⚾️����: I think you're a stranger and that should change ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4434810973, "name": "TrillJes", "screen_name": "jcano98", "lang": "en", "location": "right thurr hawmie", "create_at": date("2015-12-02"), "description": "null", "followers_count": 200, "friends_count": 149, "statues_count": 417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583827255297, "text": "�� https://t.co/WZD17y0ssk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 497655897, "name": "fanni", "screen_name": "fanfandoe", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "lypophrenia", "followers_count": 178, "friends_count": 131, "statues_count": 20719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402583978221568, "text": "@Upcoming_HipHop @Eminem fucking pain", "in_reply_to_status": 702402531851374592, "in_reply_to_user": 4022177650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 741511332, 22940219 }}, "user": { "id": 4022177650, "name": "Ben", "screen_name": "Ben1NOT2BFORGOT", "lang": "en", "location": "null", "create_at": date("2015-10-22"), "description": "Poet, aspiring gym rat, artist. Just another American hoping we can come together. Together we stand divided we fall. #notimpressedwitheitherparty", "followers_count": 417, "friends_count": 1239, "statues_count": 626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jeffersontown, KY", "id": "258c7428a48a8cfa", "name": "Jeffersontown", "place_type": "city", "bounding_box": rectangle("-85.60709,38.143069 -85.508139,38.246418") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2140222, "cityName": "Jeffersontown" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402584129216513, "text": "@momoiktkr yumimania nonchachacha perform together for the @anisonusa grand finale at the… https://t.co/y1w7r2VLO1", "in_reply_to_status": -1, "in_reply_to_user": 66045742, "favorite_count": 0, "coordinate": point("-118.27221,34.06717"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66045742, 2807165582 }}, "user": { "id": 85225087, "name": "Hupiom", "screen_name": "Hupiom", "lang": "en", "location": "CA, USA", "create_at": date("2009-10-25"), "description": "DERP.", "followers_count": 204, "friends_count": 746, "statues_count": 5727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:00.000Z"), "id": 702402584364105728, "text": "Gotta get every dollar��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422204123, "name": "..NastyDealerひ", "screen_name": "ThaRuler_LTF", "lang": "en", "location": "Queen City, 4⃣0⃣0⃣", "create_at": date("2011-11-26"), "description": "Psalm 25:1-5✊. .#LTF #GloryBoy SC:ThaRuler_LTF", "followers_count": 1565, "friends_count": 709, "statues_count": 23349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, MS", "id": "57dc070bcd27882d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-88.763404,32.312389 -88.655698,32.443049") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2846640, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402585924395009, "text": "@VICELAND @vine https://t.co/LqrwkjHYOz", "in_reply_to_status": 700741235674320898, "in_reply_to_user": 3806553495, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3806553495, 586671909 }}, "user": { "id": 2752477352, "name": "Web_Bests", "screen_name": "Web_Bests", "lang": "en", "location": "RTs-Replies-Follows≠Endorsmnts", "create_at": date("2014-08-21"), "description": "Businesswoman-Kids advocate-Maverick-ENTJ-Innovator-Perfectionist-Internet equalizer-Respects all religions & ethnics-No politics-I WORK HARD!", "followers_count": 139, "friends_count": 2401, "statues_count": 26714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402586004033536, "text": "Another big mouth!! Yea, this is SO what this show needed..God take this show & Tank it!!!#RHOBH https://t.co/1vf0M5VADA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RHOBH" }}, "user": { "id": 909378985, "name": "Bren", "screen_name": "brink20556", "lang": "en", "location": "United States", "create_at": date("2012-10-27"), "description": "Fighting the disease, I won't let it win!Had my picture up, but stolen twice. Beware of the mean girl groups. Quite a few on Twitter!", "followers_count": 324, "friends_count": 370, "statues_count": 6895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402586255749120, "text": "Wind 9.0 mph N. Barometer 1014.16 mb, Steady. Temperature 40.6 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402586444500993, "text": "what would I do w/o soundcloud", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367193795, "name": "tori.", "screen_name": "DopeInDisguise", "lang": "en", "location": "vcu", "create_at": date("2011-09-03"), "description": "you're more than just a person , you're an experience so make sure you're a good one", "followers_count": 754, "friends_count": 589, "statues_count": 17596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402586599673856, "text": "#SupportOriginMelissa 54.3°F Wind:7.4mph Pressure: 29.38hpa Falling very rapidly Rain Today 0.25in. Forecast: Stormy, much precipitation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 310667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402586780033025, "text": "@IbankoMedia feel dumb for asking but how does one find this in the app?", "in_reply_to_status": 702392413613121541, "in_reply_to_user": 947734266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 947734266 }}, "user": { "id": 644113, "name": "Noel Cornell", "screen_name": "noelc", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-01-15"), "description": "Formerly Twitter, Apple. From Omaha. I care a lot.", "followers_count": 616, "friends_count": 399, "statues_count": 12713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402587694338049, "text": "@ebbtideapp Tide in Darien, Georgia 02/24/2016\n Low 4:19am -0.5\nHigh 10:07am 7.6\n Low 4:36pm -0.2\nHigh 10:30pm 7.3\n Low 4:57am -0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.4333,31.3667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 33, "friends_count": 1, "statues_count": 10376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Darien, GA", "id": "0131ffdc15c2bf88", "name": "Darien", "place_type": "city", "bounding_box": rectangle("-81.441325,31.364032 -81.39937,31.412011") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13191, "countyName": "McIntosh", "cityID": 1321716, "cityName": "Darien" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402587748917248, "text": "02/24@03:00 - Temp 41.3F, WC 41.3F. Wind 2.8mph NE, Gust 10.0mph. Bar 30.030in, Falling. Rain 0.01in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402587983802373, "text": "Bunny https://t.co/a6fKYZb2Vv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111826829, "name": "Sara Thornton", "screen_name": "SaraAnnThornton", "lang": "en", "location": "Aruba/Los Angeles ☀️", "create_at": date("2010-02-05"), "description": "Treat others as you would like to be treated.", "followers_count": 6964, "friends_count": 187, "statues_count": 3533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402588340260864, "text": "I'm 18 years old and growing pains still keep me up at night���� #dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dying" }}, "user": { "id": 416886476, "name": "Esme➹", "screen_name": "esmeraldagarc44", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "lufkin tx,| c/o 2015", "followers_count": 1150, "friends_count": 1351, "statues_count": 16012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lufkin, TX", "id": "ef5f7bc53f4594ea", "name": "Lufkin", "place_type": "city", "bounding_box": rectangle("-94.796056,31.264964 -94.669141,31.400021") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48005, "countyName": "Angelina", "cityID": 4845072, "cityName": "Lufkin" } }
+{ "create_at": datetime("2016-02-24T00:00:01.000Z"), "id": 702402589443366912, "text": "I'm so stressed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465051699, "name": "DG", "screen_name": "DKGballer", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-01-15"), "description": "Daniel Gee 123", "followers_count": 283, "friends_count": 195, "statues_count": 4151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402589632172032, "text": "Wind 1.1 mph SSE. Barometer 29.51 in, Steady. Temperature 62.4 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402589774716928, "text": "*watches Forrest Gump*\n*has to watch something funny to recover*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407527922, "name": "Dallas Roark", "screen_name": "DallasRoark", "lang": "en", "location": "Monmouth, OR", "create_at": date("2011-11-07"), "description": "ig: dallasroark College Entrepreneur Slime Season 3", "followers_count": 689, "friends_count": 423, "statues_count": 4908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402590009630720, "text": "Wind 6.0 mph NW. Barometer 29.634 in, Falling Rapidly. Temperature 37.2 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402590374498306, "text": "Gn (@ Meridian Place in Northridge, CA) https://t.co/hUz3pEAx5P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 369, "friends_count": 339, "statues_count": 25608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402590919815169, "text": "Temp: 54.8°F Wind:10.7mph Pressure: 29.499hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402590953312257, "text": "@justIN_Case06 you hell you to old to be on the site son ��������", "in_reply_to_status": 702401190349430784, "in_reply_to_user": 2320266161, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2320266161 }}, "user": { "id": 2922487598, "name": "Freddie P.", "screen_name": "FreddiePrincee", "lang": "en", "location": "null", "create_at": date("2014-12-08"), "description": "null", "followers_count": 401, "friends_count": 459, "statues_count": 2180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bogalusa, LA", "id": "0d86f1118064d85f", "name": "Bogalusa", "place_type": "city", "bounding_box": rectangle("-89.896103,30.746749 -89.833306,30.812685") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22117, "countyName": "Washington", "cityID": 2208150, "cityName": "Bogalusa" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402591028805633, "text": "@TROY_theGreat ����������", "in_reply_to_status": 702402475131858944, "in_reply_to_user": 614548585, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 614548585 }}, "user": { "id": 2949219788, "name": "❣", "screen_name": "yourgoddesssss", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "null", "followers_count": 18287, "friends_count": 8041, "statues_count": 53337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402591049830400, "text": "1.\"@IMRoseMcIver\"\n#Life Is A @Porn, Why Not Be In This #Porn Together?\n��Love - #Hubby��\n[@CWiZombie.@CWiZombieINDO].2 https://t.co/A55PoFtuts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Life", "Porn", "Hubby" }}, "user_mentions": {{ 2155365456, 2343591, 2474719489, 2508085358 }}, "user": { "id": 3185728714, "name": "8", "screen_name": "plasmadice2", "lang": "en", "location": "Scranton, PA", "create_at": date("2015-04-19"), "description": "God", "followers_count": 2129, "friends_count": 5000, "statues_count": 11776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scranton, PA", "id": "8e67b1e195b34dd8", "name": "Scranton", "place_type": "city", "bounding_box": rectangle("-75.719751,41.369442 -75.617789,41.469377") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42069, "countyName": "Lackawanna", "cityID": 4269000, "cityName": "Scranton" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402591758614532, "text": "1 Long-eared Owl (Asio otus) - Buenos Aires NWR--Arivaca Cienega - 2016-02-23 13:17 https://t.co/lzDoQqVa6Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.3244,31.572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12216 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 403320, "cityName": "Arivaca" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402591980961792, "text": "@DuflockaTapp Frfr it's so watered down now I can't mess with it", "in_reply_to_status": 702402367279525891, "in_reply_to_user": 166010755, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 166010755 }}, "user": { "id": 247618246, "name": "#JonesSystem", "screen_name": "SAY_D_JAYY3X", "lang": "en", "location": "NLR✈️DTX✈️HTX", "create_at": date("2011-02-04"), "description": "D.Jones •God is my idol, mentor, and motivation •IG:@DariusJayy ✡DDP✡ R.I.P Rosie Lee 07/02/2014 #LoveLiveTank #FreeLilRed #FreeYvette", "followers_count": 1322, "friends_count": 1159, "statues_count": 40879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402592224243713, "text": "Wind 1.6 mph N. Barometer 30.22 in, Falling. Temperature 32.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 111, "statues_count": 157751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402592509444096, "text": "Wind 7.0 mph ENE. Barometer 29.425 in, Falling very rapidly. Temperature 47.3 °F. Rain today 0.59 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402592798871552, "text": "Will I ever do my laundry? The saga continues", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155314543, "name": "kaitlyn", "screen_name": "k8lyn96", "lang": "en", "location": "null", "create_at": date("2010-06-13"), "description": "wild", "followers_count": 301, "friends_count": 495, "statues_count": 5168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402592891084802, "text": "where do broken hearts go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2430412128, "name": "amykaori", "screen_name": "ammyyy_nardi", "lang": "en", "location": "null", "create_at": date("2014-04-06"), "description": "instagram: amykaori", "followers_count": 123, "friends_count": 120, "statues_count": 937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipio, HI", "id": "00b9f0520832fe1b", "name": "Waipio", "place_type": "city", "bounding_box": rectangle("-158.026656,21.397631 -157.98498,21.458421") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579860, "cityName": "Waipio" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402592899506177, "text": "@IAMJRoddyRod like me", "in_reply_to_status": 702401464690417664, "in_reply_to_user": 63257395, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63257395 }}, "user": { "id": 3029626059, "name": "Des", "screen_name": "goddess_des", "lang": "en", "location": "CA", "create_at": date("2015-02-10"), "description": "Afakasi. SC - theylovedess", "followers_count": 493, "friends_count": 157, "statues_count": 3710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402593037926400, "text": "Don't mistake my whistling for me being in a good mood. The whistling just helps drown out the voices I hear in my head.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17768924, "name": "Joe Kelley", "screen_name": "talkradiojoe", "lang": "en", "location": "Orlando, FL", "create_at": date("2008-11-30"), "description": "The guy from the thing. Web stuff. Something.\n\nNewsDir and morning radio host @ WDBO/Orlando", "followers_count": 10655, "friends_count": 10527, "statues_count": 13314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Butler, FL", "id": "0052cdde03f2eadb", "name": "Lake Butler", "place_type": "city", "bounding_box": rectangle("-81.611279,28.424587 -81.483386,28.536837") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12125, "countyName": "Union", "cityID": 1237650, "cityName": "Lake Butler" } }
+{ "create_at": datetime("2016-02-24T00:00:02.000Z"), "id": 702402593256038400, "text": "Bruhhh I turned my phone off and on and then checked my App Store ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415125521, "name": "Kia", "screen_name": "dopefumes", "lang": "en", "location": "NARNIA. UMES19", "create_at": date("2011-11-17"), "description": "MERMAID KIA", "followers_count": 1811, "friends_count": 1038, "statues_count": 44969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princess Anne, MD", "id": "0098bec2526560be", "name": "Princess Anne", "place_type": "city", "bounding_box": rectangle("-75.715802,38.152156 -75.668895,38.236042") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24039, "countyName": "Somerset", "cityID": 2464000, "cityName": "Princess Anne" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402593847447552, "text": "Faccttttts https://t.co/crn2aZsdoh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389261737, "name": "Pablito", "screen_name": "Alexf4__", "lang": "en", "location": "The Jungle ", "create_at": date("2011-10-11"), "description": "The 6 is watching", "followers_count": 8132, "friends_count": 1856, "statues_count": 22486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402593939701760, "text": "@mypaperheart16 I love that song", "in_reply_to_status": 702402263827017730, "in_reply_to_user": 178121798, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178121798 }}, "user": { "id": 3118020221, "name": "Angelina Cruzers", "screen_name": "angelinacruzers", "lang": "en", "location": "Beverly Hills CA", "create_at": date("2015-03-26"), "description": "please boys don't follow me I hate everyone of u", "followers_count": 341, "friends_count": 168, "statues_count": 4792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402594241695748, "text": "@ChadR73 @america_trump @anthony5505USN", "in_reply_to_status": 702378678047674370, "in_reply_to_user": 366015392, "favorite_count": 0, "coordinate": point("-75.05800981,40.06745804"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 366015392, 68698563 }}, "user": { "id": 260369607, "name": "ACG5505USN", "screen_name": "anthony5505", "lang": "en", "location": "null", "create_at": date("2011-03-03"), "description": "null", "followers_count": 491, "friends_count": 301, "statues_count": 11888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402594367524865, "text": "Wind 6.0 mph ENE. Barometer 29.972 in, Falling. Temperature 34.4 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402594459746305, "text": "Savage.���� RT @bobsaget: I need every bad bitch up in Equinox. I need to know right now if you're a freak or not.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38536306 }}, "user": { "id": 403099560, "name": "PIMPSTER ®", "screen_name": "pimpsterbrand", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-11-01"), "description": "Pimpster Brand premium collection.\nInstagram: pimpsterbrand #PimpsterBrand contact us: sales@pimpsterbrand.com", "followers_count": 544, "friends_count": 180, "statues_count": 8985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402594849865728, "text": "I wanna put it on snap but one of them has me on snap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581529380, "name": "REAL OAKLAND", "screen_name": "Susboytay", "lang": "en", "location": "North Oakland/Los Angeles CA", "create_at": date("2012-05-15"), "description": "I do my daily service just to see the gates of afterlife| CSU Northridge C/O '19", "followers_count": 266, "friends_count": 262, "statues_count": 4892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402595046998018, "text": "@Skrillex Mew from Pokemon is a real creature?!? ������", "in_reply_to_status": 702370330447368192, "in_reply_to_user": 18825961, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18825961 }}, "user": { "id": 26752714, "name": "Kayla Vee", "screen_name": "kaylatehehe", "lang": "en", "location": "Boulder, CO", "create_at": date("2009-03-26"), "description": "I eat my breakfast the night before.", "followers_count": 114, "friends_count": 299, "statues_count": 661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402595470581760, "text": "Temp: 32.2°F - Dew Point: 25.2° - Wind: 5.2 mph - Gust: 7.6 - Rain Today: 0.00in. - Pressure: 29.87in, - Trend: Falling quickly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 13889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402595705516034, "text": "46.3F (Feels: 46.3F) - Humidity: 78% - Wind: 20.6mph W - Gust: 29.8mph - Pressure: 1020.0mb - Rain: 0.52\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402596456243200, "text": "I can write papers better than anything when it comes to school, but this one I'm trying to get a ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 974971375, "name": "Pleasure Z‼️", "screen_name": "_ZaeSavage", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "Ralphy World | Just a kid trying to make it ✊| #VSU19 | 757", "followers_count": 920, "friends_count": 785, "statues_count": 30438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ettrick, VA", "id": "68a0bfb12798b6fc", "name": "Ettrick", "place_type": "city", "bounding_box": rectangle("-77.443568,37.223646 -77.414759,37.265595") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51041, "countyName": "Chesterfield", "cityID": 5126128, "cityName": "Ettrick" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402596871528449, "text": "Craving a nice salad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405843568, "name": "Jess", "screen_name": "Jess_Brooke52", "lang": "en", "location": "null", "create_at": date("2011-11-05"), "description": "null", "followers_count": 434, "friends_count": 327, "statues_count": 18789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorketown, NJ", "id": "005f52e06b9c06c8", "name": "Yorketown", "place_type": "city", "bounding_box": rectangle("-74.362666,40.270616 -74.282922,40.344614") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3483245, "cityName": "Yorketown" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402597030907904, "text": "This next album crazy yo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54106166, "name": "Teddy West$ide", "screen_name": "KoVuMusic", "lang": "en", "location": "Chicago, IL | XVI ", "create_at": date("2009-07-05"), "description": "The New Fresh Prince || IG: @versacepapa kovumusic14@gmail.com || Runaway Brand Media", "followers_count": 1344, "friends_count": 623, "statues_count": 79712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, IL", "id": "9c908d38b1b53897", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-89.028906,39.785267 -88.844461,39.923903") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17115, "countyName": "Macon", "cityID": 1718823, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402597370601473, "text": "@scouse_hanky yes he is always at it too", "in_reply_to_status": 702392588280733696, "in_reply_to_user": 66971832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66971832 }}, "user": { "id": 553180562, "name": "Wayne Harrison", "screen_name": "WayneHarrison9", "lang": "en", "location": "California", "create_at": date("2012-04-13"), "description": "FOUNDER SOCCER AWARENESS Since 1996 UEFA A , NSCAA Prem Diploma Ex-Pro, 14 best selling Books, 34 E Books, many videos, trying to make a difference in the game", "followers_count": 13237, "friends_count": 7080, "statues_count": 14468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402597500674049, "text": "Kelas aku ada neelopa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 910967077, "name": "محمد قيم", "screen_name": "qayyimlopoh", "lang": "en", "location": "selangor, MAS", "create_at": date("2012-10-28"), "description": "null", "followers_count": 442, "friends_count": 477, "statues_count": 23571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-24T00:00:03.000Z"), "id": 702402597504811008, "text": "I can't believe this.. https://t.co/eabgvsJPbl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2479398421, "name": "Melissa Ramirez", "screen_name": "msmelly1994", "lang": "en", "location": "null", "create_at": date("2014-05-05"), "description": "OKO ❤️", "followers_count": 273, "friends_count": 209, "statues_count": 11204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402598217850880, "text": "@Igorvm777 @AMEHXOTEn_CYKA https://t.co/ENVRN6DFAE", "in_reply_to_status": 702397796872273920, "in_reply_to_user": 786674540, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 786674540, 3366425169 }}, "user": { "id": 1866709338, "name": "МаринаMarianna", "screen_name": "Mariannfog", "lang": "ru", "location": "Санкт-Петербург, Россия", "create_at": date("2013-09-15"), "description": "Желающим научить меня, что писать и с кем общаться:Проходите-проходите!Здесь дует!", "followers_count": 2800, "friends_count": 370, "statues_count": 51132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402598620569605, "text": "@kanyewest maybe you are not 'for all markets'? #shakespeare", "in_reply_to_status": 702354017725390848, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shakespeare" }}, "user_mentions": {{ 169686021 }}, "user": { "id": 19513873, "name": "Pahtrisha", "screen_name": "Pahtrisha", "lang": "en", "location": "NZ/USA", "create_at": date("2009-01-25"), "description": "Outspoken Liberal/Social Capitalist, Educator, Consultant and Actress, lover of Street Tacos, Opera and Politics, trying to change the world!", "followers_count": 179, "friends_count": 504, "statues_count": 4402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402598847062017, "text": "@jastheasshole Wtf Jasmine", "in_reply_to_status": -1, "in_reply_to_user": 1177831436, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 1177831436 }}, "user": { "id": 575764265, "name": "$HADEDESZA", "screen_name": "wyfinistriflin", "lang": "en", "location": "planting palm treees for u", "create_at": date("2012-05-09"), "description": "SUGA SUGA HOW YOU GET SO FLY", "followers_count": 761, "friends_count": 874, "statues_count": 25267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402599702679553, "text": "Wind 1.0 mph SSE. Barometer 30.274 in, Falling. Temperature 31.8 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600436703233, "text": "@TheGuyWhoSmiles lmao wouldn't you like to know!", "in_reply_to_status": 702402484560592896, "in_reply_to_user": 65244875, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65244875 }}, "user": { "id": 428039088, "name": "ana", "screen_name": "fggtyanne", "lang": "en", "location": "Long Beach, CA", "create_at": date("2011-12-04"), "description": "@justinbieber ❣", "followers_count": 651, "friends_count": 374, "statues_count": 63627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600570908674, "text": "it's LIT now! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1129996656, "name": "carlos", "screen_name": "DOPEITSPEPPER", "lang": "en", "location": "RiteNext2ThePacific2BeSpecific", "create_at": date("2013-01-28"), "description": "positive vibrations gonna set your mind free #PPP", "followers_count": 686, "friends_count": 451, "statues_count": 34570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600705069061, "text": "@dylanprohaska my pantry is much less extensive tho I'll be able to infuse it only w stolen K cup powder and a single banana laffy taffy", "in_reply_to_status": 702402259343314944, "in_reply_to_user": 842036882, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 128422661 }}, "user": { "id": 842036882, "name": "Kendall Anne", "screen_name": "CommonKendall", "lang": "en", "location": "az/sd", "create_at": date("2012-09-23"), "description": "sdsu deegee", "followers_count": 688, "friends_count": 537, "statues_count": 11709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600843534336, "text": "Wait. I mean an actual banana. As in the fruit. I forget how perverted y'all are.", "in_reply_to_status": 702402418542247936, "in_reply_to_user": 287882043, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287882043, "name": "Tira ", "screen_name": "AtiRANDOMness", "lang": "en", "location": "East Tennessee ", "create_at": date("2011-04-25"), "description": "Unless someone like you cares a whole awful lot, Nothing is going to get better. It's not -Dr. Seuss", "followers_count": 1423, "friends_count": 831, "statues_count": 184030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600885420034, "text": "@CorpusBaseball thanks for the interview! Let's show 'em what we're up to! #GoDers https://t.co/KvMr7v6LTM", "in_reply_to_status": -1, "in_reply_to_user": 52329806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoDers" }}, "user_mentions": {{ 52329806 }}, "user": { "id": 295729993, "name": "jamiefink", "screen_name": "JK_Fink", "lang": "en", "location": "TAMUCC / McAlester", "create_at": date("2011-05-09"), "description": "Think For Yourself & Question Authority. Give a damn. Many damns. More damns than anyone. Journalist | Opportunist | Redeemed", "followers_count": 485, "friends_count": 865, "statues_count": 14072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600927436800, "text": "Barberton Oh Temp:38.6°F Wind:0 mph Dir:NE Baro:Falling Rain2day:0.00in Hum:81% UV:0.0 @ 03:00 02/24/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 235, "friends_count": 228, "statues_count": 114875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402600948338688, "text": "@dihorchata @foIloweed naice", "in_reply_to_status": 702278843961180160, "in_reply_to_user": 3766421174, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3766421174, 3191956537 }}, "user": { "id": 207228898, "name": "Duro", "screen_name": "ElDJDURO", "lang": "en", "location": "Miami-NYC", "create_at": date("2010-10-24"), "description": "DJ/Producer invented Jerseyton. Resident DJ at @MoombahMansion . \r\n@LaClinicaRecs \r\nPromo bienduropromo@gmail.com", "followers_count": 2580, "friends_count": 936, "statues_count": 61667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naranjito, PR", "id": "013cfcb2685426b0", "name": "Naranjito", "place_type": "city", "bounding_box": rectangle("-66.314136,18.241203 -66.204836,18.334653") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72105, "countyName": "Naranjito", "cityID": 7256129, "cityName": "Naranjito" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402601099337728, "text": "I think I am gonna minor in Sociology because I can do it in a way where it is only one extra class", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1088179531, "name": "Kiersten Fields", "screen_name": "Fields_Kiersten", "lang": "en", "location": "Hillsboro, OR", "create_at": date("2013-01-13"), "description": "You only have a few blue moon moments in your life. Don't let those blue moon moment pass you by.", "followers_count": 174, "friends_count": 416, "statues_count": 14416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-02-24T00:00:04.000Z"), "id": 702402601355243522, "text": "@DrawekimMike i snore.. Specially when im really tired.. Haha its pretty bad i hear", "in_reply_to_status": -1, "in_reply_to_user": 1428792992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1428792992 }}, "user": { "id": 63885431, "name": "Omar Handsome", "screen_name": "Omar5091", "lang": "en", "location": "Connecticut", "create_at": date("2009-08-07"), "description": "#teamtaurus.followed by @JLo may31,2015 @dawnrobinson_ @KellyPrice4Real @AubreyOday life keeps getting better.. etc IG: handsomeguy509", "followers_count": 983, "friends_count": 1616, "statues_count": 54526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Haven, CT", "id": "795003fb11ee9829", "name": "New Haven", "place_type": "city", "bounding_box": rectangle("-72.998069,41.246404 -72.860248,41.350384") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402603452403712, "text": "PLAYOFFS TOMORROW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262509919, "name": "Pindo Drammeh", "screen_name": "AirAfrica24", "lang": "en", "location": "Anchorage, AK/ California", "create_at": date("2011-03-07"), "description": "Sierra College Basketball Player", "followers_count": 2828, "friends_count": 1808, "statues_count": 75226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402603666313216, "text": "Shit Cruz just past you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4886678508, "name": "bill hood", "screen_name": "billhood1966", "lang": "en", "location": "null", "create_at": date("2016-02-09"), "description": "null", "followers_count": 3, "friends_count": 3, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mooresville, IN", "id": "002ff610122dafad", "name": "Mooresville", "place_type": "city", "bounding_box": rectangle("-86.417461,39.56948 -86.263008,39.632796") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18109, "countyName": "Morgan", "cityID": 1850976, "cityName": "Mooresville" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402604010176513, "text": "#JustTruckThings #V8 #Underwater #NoSnorkel https://t.co/gxAfFjKxLU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "JustTruckThings", "V8", "Underwater", "NoSnorkel" }}, "user": { "id": 1004290663, "name": "Adrian Acevedo", "screen_name": "BigAvocado21", "lang": "en", "location": "null", "create_at": date("2012-12-11"), "description": "Live life however you can SC: bigavocado21", "followers_count": 129, "friends_count": 262, "statues_count": 8949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402604614164480, "text": "I need water.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075373611, "name": "Obi Wan CUHnobi", "screen_name": "TheBlackHippiii", "lang": "en", "location": "HTX - LBK", "create_at": date("2013-01-09"), "description": "Lord Knows | Texas Tech University", "followers_count": 1165, "friends_count": 855, "statues_count": 77606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402605327192064, "text": "Happy birthday to this GILF @emilysfeed , you're the most splendid bestie in the world & deserve the best��⛽️������ https://t.co/iSxTTpOLg2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1210858327 }}, "user": { "id": 3306548768, "name": "Trent", "screen_name": "trentsentell", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2015-08-04"), "description": "Proud owner of a puppy", "followers_count": 86, "friends_count": 140, "statues_count": 700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402605906055168, "text": "3.1 magnitude #earthquake. 77 km from #Y, AK, United States https://t.co/VUFD1O8k09", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-150.551,62.767"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "Y" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 65146, "friends_count": 10, "statues_count": 96870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna" } }
+{ "create_at": datetime("2016-02-24T00:00:05.000Z"), "id": 702402606262566912, "text": "temperature down 34°F -> 25°F\nhumidity up 38% -> 47%\nwind 5mph -> 3mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.07626,40.5815"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130695262, "name": "Fort Collins Weather", "screen_name": "_FortCollinsCO", "lang": "en", "location": "Fort Collins, CO", "create_at": date("2010-04-07"), "description": "Weather updates, forecast, warnings and information for Fort Collins, CO. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 121, "friends_count": 1, "statues_count": 27041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Collins, CO", "id": "b2e4e65d7b80d2c1", "name": "Fort Collins", "place_type": "city", "bounding_box": rectangle("-105.148074,40.47168 -104.979811,40.656701") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 827425, "cityName": "Fort Collins" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402606614892544, "text": "@tattedkisspers @shellheroy", "in_reply_to_status": 698334989239369729, "in_reply_to_user": 2853936016, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2853936016, 395741776 }}, "user": { "id": 323638856, "name": "Haley Heroy", "screen_name": "hheroy", "lang": "en", "location": "las vegas", "create_at": date("2011-06-24"), "description": "LV Hair stylist text\\call for appointments 702.521.6644", "followers_count": 947, "friends_count": 429, "statues_count": 6626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402606908489728, "text": "TBH https://t.co/bB7cJVtJiU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 358502061, "name": "j mama", "screen_name": "GrandePapiJota", "lang": "en", "location": "null", "create_at": date("2011-08-19"), "description": "I love it when they call me j mama", "followers_count": 731, "friends_count": 609, "statues_count": 47342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherryland, CA", "id": "3b93b1d81c90ff1e", "name": "Cherryland", "place_type": "city", "bounding_box": rectangle("-122.122215,37.666826 -122.090037,37.688086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 612902, "cityName": "Cherryland" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607030083585, "text": "Happy birthday to the most real man I know! Wouldn't be here without him ������ may life bring you more amazing moments https://t.co/22gt0abDSt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3363092991, "name": "Gina_valentinaxxx", "screen_name": "GValentinaxxx", "lang": "en", "location": "Miami, FL", "create_at": date("2015-07-06"), "description": "5'2 Brazilian✨ management Eastcoasttalents XXX ✨✨ serious inquiry only personal video request only email gvalentinaxxx@gmail.com", "followers_count": 21210, "friends_count": 366, "statues_count": 1273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ojus, FL", "id": "cc7ba09ef43845d3", "name": "Ojus", "place_type": "city", "bounding_box": rectangle("-80.185106,25.936226 -80.147035,25.974421") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1251125, "cityName": "Ojus" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607092998144, "text": "@eslaaayy you need this https://t.co/GITFrv5Dgy", "in_reply_to_status": -1, "in_reply_to_user": 2871547386, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2871547386 }}, "user": { "id": 1353079741, "name": "hilda", "screen_name": "hilgyyy", "lang": "en", "location": "Dallas, Texas", "create_at": date("2013-04-14"), "description": "thanks for keeping up with me", "followers_count": 321, "friends_count": 234, "statues_count": 16305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607424389120, "text": "NADINE & JAMES SRSLY NEEDS TO BE ON ANOTHER SHOW SOMETIME SOON. ����❤️✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2302101252, "name": "J©", "screen_name": "jhoannactn", "lang": "en", "location": "null", "create_at": date("2014-01-20"), "description": "Leaning on God's understanding", "followers_count": 262, "friends_count": 259, "statues_count": 7917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607608950784, "text": "No I put it on too haha���� https://t.co/vcWFeZlmDN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1099943485, "name": "Sanamسشوشى", "screen_name": "SanamBoo", "lang": "en", "location": "Bay Area", "create_at": date("2013-01-17"), "description": "I like light", "followers_count": 861, "friends_count": 395, "statues_count": 16004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607621476352, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 624263023, "name": "boss of bosses", "screen_name": "ForeignMamacita", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "Reason why Your Hating ass Mad", "followers_count": 1023, "friends_count": 399, "statues_count": 34347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Lawn, NJ", "id": "d9da081322efe6e7", "name": "Fair Lawn", "place_type": "city", "bounding_box": rectangle("-74.148182,40.917486 -74.089421,40.955019") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3422470, "cityName": "Fair Lawn" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607726370816, "text": "@ClumsyyyyCassss GO TO HIS HOUSE", "in_reply_to_status": 702402111489904641, "in_reply_to_user": 321185456, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321185456 }}, "user": { "id": 177074557, "name": "Paula Taguinod", "screen_name": "paulawallabear", "lang": "en", "location": "cali ", "create_at": date("2010-08-10"), "description": "when life throws you lemons, make margaritas!", "followers_count": 121, "friends_count": 212, "statues_count": 5079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607944437760, "text": "I love this haha https://t.co/FYL20O334X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1531989290, "name": "ㅤㅤㅤ", "screen_name": "prisillaserrano", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2013-06-19"), "description": "it really go down in the DMs", "followers_count": 327, "friends_count": 221, "statues_count": 20115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402607986450432, "text": "2.5 magnitude #earthquake. 72 km from Y, AK, #UnitedStates https://t.co/7rWKpjZLG3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-149.584,62.799"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 65146, "friends_count": 10, "statues_count": 96871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402608946876416, "text": "I geh a puss ah food a feed a sweet ting a main ting and a side ting yea ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403050615, "name": "Keeping up with Nell", "screen_name": "FunkyFresh_Kidd", "lang": "en", "location": "null", "create_at": date("2011-11-01"), "description": "I Love Booty. . Virgin Island #340☀️ 11/29♐️", "followers_count": 1128, "friends_count": 960, "statues_count": 59125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford Mill, MD", "id": "dc811f15be302120", "name": "Milford Mill", "place_type": "city", "bounding_box": rectangle("-76.795559,39.320502 -76.744784,39.369482") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2452562, "cityName": "Milford Mill" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402609584410629, "text": "Wind 0.6 mph E. Barometer 30.108 in, Falling Rapidly. Temperature 31.6 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-24T00:00:06.000Z"), "id": 702402610167488516, "text": "@CaseyCakes9496 also \"Ke$ha Deconstructed\"", "in_reply_to_status": 702402407364485120, "in_reply_to_user": 109938520, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 109938520 }}, "user": { "id": 109938520, "name": "Casey Nicole", "screen_name": "CaseyCakes9496", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-01-30"), "description": "Thespian - Fangirl - Corgi Enthusiast 19 - Cornish College of the Arts, class of 2019", "followers_count": 138, "friends_count": 178, "statues_count": 1321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402610536521731, "text": "Sheesh https://t.co/JKxN0Jl8cN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 535845167, "name": "Ike Riddick", "screen_name": "IkeBoutChecks", "lang": "en", "location": "252", "create_at": date("2012-03-24"), "description": "#CarolinaDreaming #252 King of 85, #757", "followers_count": 609, "friends_count": 784, "statues_count": 13301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402611757105153, "text": "That shit ain't me, it ain't in me��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298793440, "name": "❣Kay'Nicole❣", "screen_name": "_PhenomenalBoss", "lang": "en", "location": "Greenville, MS✈️Starkville, MS", "create_at": date("2011-05-14"), "description": "IG: phenomenal_boss1 SC: phenomenal_boss ‼️Rest Easy Daddy ILY❤️✊", "followers_count": 1013, "friends_count": 962, "statues_count": 18592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, MS", "id": "15afdc7eb03e43ba", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-91.10053,33.336092 -90.98932,33.451107") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28151, "countyName": "Washington", "cityID": 2829180, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402611992010753, "text": "@O1Promotions IT GOES DOWN N D #STUDIO. ������������U just might not be ready4 dis ������ ‼ #TTE #PEDAL2METAL ������\nhttps://t.co/NdUlqmDtot", "in_reply_to_status": -1, "in_reply_to_user": 3028273816, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STUDIO", "TTE", "PEDAL2METAL" }}, "user_mentions": {{ 3028273816 }}, "user": { "id": 82721349, "name": "BAMA CHEWING #TTE", "screen_name": "jeanettagraham", "lang": "en", "location": "Gadsden, Alabama", "create_at": date("2009-10-15"), "description": "Manager &Promoter for TYME TA EAT RECORDS . Send email to jeanettagraham24@gmail.com for booking shows & doing features. (256)478-1463 *SERIOUS CALLS ONLY", "followers_count": 1646, "friends_count": 2633, "statues_count": 58614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gadsden, AL", "id": "8dac588c46ed3591", "name": "Gadsden", "place_type": "city", "bounding_box": rectangle("-86.079043,33.933091 -85.934616,34.073159") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1055, "countyName": "Etowah", "cityID": 128696, "cityName": "Gadsden" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402612260425728, "text": "Umm . He's in love ? AWURADE!! �� wat did I do to deserve this? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 773034446, "name": "precious:)", "screen_name": "Urz_Truthfully", "lang": "en", "location": "EKB❤️", "create_at": date("2012-08-21"), "description": "✨Psalms 34:18 The LORD is near to the brokenhearted and saves the crushed in spirit.✨♎️✨", "followers_count": 633, "friends_count": 536, "statues_count": 24653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groveton, VA", "id": "522b36589a90e2b5", "name": "Groveton", "place_type": "city", "bounding_box": rectangle("-77.133063,38.739648 -77.059357,38.797762") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5133584, "cityName": "Groveton" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402612424019968, "text": "Wind 5.0 mph ENE. Barometer 29.721 in, Falling Rapidly. Temperature 33.4 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 381, "friends_count": 291, "statues_count": 8510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402612486864896, "text": "If you found your friends boyfriend on tinder do you tell her or leave it alone?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37744024, "name": "kate", "screen_name": "younglungss", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-05-04"), "description": "I'm the girl with crooked pinkies.", "followers_count": 442, "friends_count": 398, "statues_count": 9131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402613162156032, "text": "He would be an excellent VP choice. Go Julián! https://t.co/nw0WD6OGCP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34189221, "name": "Oscar Arana", "screen_name": "oscar_arana", "lang": "en", "location": "Portland, OR", "create_at": date("2009-04-21"), "description": "Hispanic Chamber Prez, strategist, negotiator, communicator, recreational kickball and soccer player. My ideas = my own, not my employer.", "followers_count": 256, "friends_count": 465, "statues_count": 137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402613321531392, "text": "2/24/2016 - 02:00\nTemp: 41.0F \nHum: 94%\nWind: 5.0 mph\nBaro: 29.634in. & Rising\nRain: 0.02 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 50258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402613325778944, "text": "Ripley SW Limestone Co. Temp: 56.8°F Wind:6.0mph Pressure: 973.0mb Falling slowly Rain Today 0.04in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 49367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402613929746432, "text": "Heavenly father ���� https://t.co/NJ4b6yhenK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 608763547, "name": "tyler", "screen_name": "tyler216_", "lang": "en", "location": "Richmond, VA", "create_at": date("2012-06-14"), "description": "CLEVELAND✈ VA \n#VCU\n \n \nNational Society of Black Engineers (NSBE)", "followers_count": 309, "friends_count": 256, "statues_count": 1906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402614491783169, "text": "@Snikle79 Thanks for the follow! #LegalizeIt https://t.co/fY27n3jpj1 https://t.co/etXD2Em1rS", "in_reply_to_status": -1, "in_reply_to_user": 3219671509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LegalizeIt" }}, "user_mentions": {{ 3219671509 }}, "user": { "id": 2668511677, "name": "Higher Ground", "screen_name": "highergtv", "lang": "en", "location": "Seattle", "create_at": date("2014-07-21"), "description": "Higher Ground is the World’s first satirical news program highlighting the revolutionary changes related to the legalization of marijuana.", "followers_count": 7165, "friends_count": 7314, "statues_count": 41952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-24T00:00:07.000Z"), "id": 702402614512783360, "text": "been there done that, so embarrassing. thought about throwing myself in front of a moving bus https://t.co/W2mC10a7Te", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359814734, "name": "Jamie Bauer", "screen_name": "jaybauerr", "lang": "en", "location": "stockton '16", "create_at": date("2011-08-21"), "description": "| happiest girls are the prettiest |", "followers_count": 497, "friends_count": 358, "statues_count": 23928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, NJ", "id": "cff113be4012e4ce", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-74.624348,39.439255 -74.526829,39.511343") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3460030, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969058820096, "text": "@DSprayberry Congrats on the nomination. And them spelling your name right. Quite a few on the list can't say the same.", "in_reply_to_status": 702761491687038976, "in_reply_to_user": 423781033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423781033 }}, "user": { "id": 36380135, "name": "NurseBrak", "screen_name": "NurseBrak", "lang": "en", "location": "null", "create_at": date("2009-04-29"), "description": "Fly Jedi", "followers_count": 213, "friends_count": 317, "statues_count": 33249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nazareth, PA", "id": "7583e05c6989c72a", "name": "Nazareth", "place_type": "city", "bounding_box": rectangle("-75.330793,40.730297 -75.297315,40.751385") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4252808, "cityName": "Nazareth" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969138528256, "text": "@bUrn_01 I mean it really don't matter though for some people that's the thing�� it's just funny to poke fun at", "in_reply_to_status": 702764529839685632, "in_reply_to_user": 240014836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240014836 }}, "user": { "id": 182888841, "name": "28 FEBRUARY", "screen_name": "enimsahJ__x", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-08-25"), "description": "22. Navy Vet. On the left. Not trying to be your bitch so move.", "followers_count": 2657, "friends_count": 2553, "statues_count": 174947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willoughby Hills, OH", "id": "3e16246d8205b329", "name": "Willoughby Hills", "place_type": "city", "bounding_box": rectangle("-81.488121,41.569695 -81.389499,41.621429") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985512, "cityName": "Willoughby Hills" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969343848448, "text": "Happy Birthday best friend I love you so much have a great day don't let no putas ruin that Shit ���� @_xo_lc ps I'll be better by Saturday ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 377730862 }}, "user": { "id": 2442723746, "name": "yessenia", "screen_name": "fatmuffin3", "lang": "en", "location": "Pomona, CA", "create_at": date("2014-04-13"), "description": "Mt. SAC Nursing, Loops is my bff", "followers_count": 169, "friends_count": 39, "statues_count": 10248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969373376512, "text": "I'm so awake fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 127415877, "name": "Matt", "screen_name": "MATTicus_Fiinch", "lang": "en", "location": "Dublin City, Ireland", "create_at": date("2010-03-28"), "description": "I hate Maroon 5 and onions • Northeastern University • TAKEN by hashbrowns", "followers_count": 687, "friends_count": 558, "statues_count": 24534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969427869696, "text": "I've tried many drugs hoping that I could find one to make me more social. A drug for \"mingling\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.232644,34.076518"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1678785049, "name": "affordable mansions", "screen_name": "havinghad", "lang": "en", "location": "null", "create_at": date("2013-08-17"), "description": "Eden O. Orr-Kerr", "followers_count": 253, "friends_count": 213, "statues_count": 3231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Solano Canyon", "id": "07d9e38cf7c82001", "name": "Solano Canyon", "place_type": "poi", "bounding_box": rectangle("-118.23264409999999,34.0765179 -118.232644,34.076518") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969444691968, "text": "Lighting yourself on fire won't make you LOOK any hotter!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 225548413, "name": "ҠŔЗѠ™", "screen_name": "KR3Wmatic", "lang": "en", "location": "Hip Hop State of Mind", "create_at": date("2010-12-11"), "description": "I love Hip Hop but I'm still single. Looking for Cleopatra. African queen, yo look at me I'm a bachelor. A sarcastic bastard, I hope you have a sense of humor.", "followers_count": 60147, "friends_count": 6813, "statues_count": 47819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764969536847872, "text": "@LahavHarkov @thenation equating Hillary with conservative Rep is hateful. What the hell conservative Rep? They are all radicals.", "in_reply_to_status": 702756592333275136, "in_reply_to_user": 315779591, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 315779591, 1947301 }}, "user": { "id": 266570946, "name": "Shmuel Rotenstreich", "screen_name": "koshem9", "lang": "en", "location": "Berkeley, CA", "create_at": date("2011-03-15"), "description": "Workers' rights, free speech, equality and support for the disadvantaged", "followers_count": 81, "friends_count": 148, "statues_count": 4786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764970631696384, "text": "I came, I saw, I conquered! Time to tone this body down. @ Life Time… https://t.co/J5cPr1JUSp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.6326294,29.6053905"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235724534, "name": "maggie z", "screen_name": "Maggie_Z_Artist", "lang": "en", "location": "houston tx", "create_at": date("2011-01-08"), "description": "Artist/awesome", "followers_count": 5, "friends_count": 4, "statues_count": 17 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Land, TX", "id": "7a41192a2879ee24", "name": "Sugar Land", "place_type": "city", "bounding_box": rectangle("-95.686106,29.543372 -95.577273,29.663556") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4870808, "cityName": "Sugar Land" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764971239706624, "text": "Ganon ba talaga siya ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 257367896, "name": "g○yena", "screen_name": "goyenana", "lang": "en", "location": "New York, USA", "create_at": date("2011-02-24"), "description": "null", "followers_count": 915, "friends_count": 245, "statues_count": 65721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764971382288384, "text": "https://t.co/JeM4euWRQ7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 311030731, "name": "gee✨", "screen_name": "geraldiineee_", "lang": "en", "location": "Lancaster, Ca", "create_at": date("2011-06-04"), "description": "eleven/sixteen.", "followers_count": 275, "friends_count": 277, "statues_count": 10517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764971575414784, "text": "Barberton Oh Temp:35.0°F Wind:4 mph Dir:W Baro:Rising slowly Rain2day:0.06in Hum:84% UV:0.0 @ 03:00 02/25/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 234, "friends_count": 228, "statues_count": 114899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764972162428928, "text": "Wind 1.0 mph N. Barometer 1023.47 mb, Rising. Temperature 36.2 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764972766568449, "text": "When the doc gives you that good good cough syrup that's supposed to knock you TF out but then you can't sleep when you take it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27568257, "name": "Heather Turner", "screen_name": "heatheraturner_", "lang": "en", "location": "Kansas, USA", "create_at": date("2009-03-29"), "description": "26 || KU Alum || Part-Time Traveler", "followers_count": 169, "friends_count": 200, "statues_count": 7798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olathe, KS", "id": "06d6054b42e6575f", "name": "Olathe", "place_type": "city", "bounding_box": rectangle("-94.901766,38.825503 -94.714404,38.947743") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2052575, "cityName": "Olathe" } }
+{ "create_at": datetime("2016-02-25T00:00:00.000Z"), "id": 702764972892250112, "text": "getcho ass https://t.co/hTv5261khx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236699978, "name": "miss savage", "screen_name": "mailala_", "lang": "en", "location": "null", "create_at": date("2013-12-21"), "description": "stay out my DMs.", "followers_count": 972, "friends_count": 156, "statues_count": 177616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764973190049792, "text": "@deansikon @markingtuan #nohill #feelthebern #firedebbie#imnotyourfirewall#bernie2016#notuswe#$shillarysucks#berniesrev\nolution#nosuperpacs", "in_reply_to_status": 702762956463845376, "in_reply_to_user": 3163975003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nohill", "feelthebern" }}, "user_mentions": {{ 3163975003, 704530686 }}, "user": { "id": 1652838942, "name": "jcazes54@gmail.com", "screen_name": "jcazen57s54", "lang": "en", "location": "Louisiana, USA", "create_at": date("2013-08-07"), "description": "null", "followers_count": 142, "friends_count": 196, "statues_count": 3833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Iberia, LA", "id": "7d8e6592712895b9", "name": "New Iberia", "place_type": "city", "bounding_box": rectangle("-91.892077,29.941102 -91.731432,30.04583") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22045, "countyName": "Iberia", "cityID": 2254035, "cityName": "New Iberia" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974150721536, "text": "@KatiieFiischer thought of you https://t.co/ueXGC5fCSg", "in_reply_to_status": -1, "in_reply_to_user": 91823730, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 91823730 }}, "user": { "id": 102197406, "name": "Angela Sasso", "screen_name": "Ang_XoXo17", "lang": "en", "location": "Long Island, New York", "create_at": date("2010-01-05"), "description": "Instagram: xo_angela_xo #AngelaSassoPhotography #Model #Photographer #KrisAllen #MusicIsLife #CarsAreLife #Hope #Rare #Dreamer #Fighter", "followers_count": 639, "friends_count": 862, "statues_count": 30805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindenhurst, NY", "id": "85644511d3f21570", "name": "Lindenhurst", "place_type": "city", "bounding_box": rectangle("-73.393981,40.665788 -73.351882,40.708732") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3642554, "cityName": "Lindenhurst" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974217756672, "text": "#AboutTheBusinessBET Eva Marcelle bringing back that Eve look w/ her blonde short hair girl you better work it LOL :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AboutTheBusinessBET" }}, "user": { "id": 2269420068, "name": "SharonLadyCapricorn", "screen_name": "dimepiece103", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-12-30"), "description": "#MusicLover (Rap, R&B, Reggae etc) I read books #SoapFan Y&R B&B #Navery #Avery #Tabby #Lope #Hope #TeamCapricorn #SuckaFreeIn2016", "followers_count": 5272, "friends_count": 4673, "statues_count": 2899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974238597120, "text": "TRU AF https://t.co/A8FGswkUCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 263214940, "name": "Monica D", "screen_name": "Monica_Dhiman", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-03-09"), "description": "UTD. Sharara. Biology is Life. Bollywood is Zindagi. Music is One Love. Dance is First Love. Enthusiast. Sorta Crazy. Punjabi. Dreams in the making.", "followers_count": 285, "friends_count": 172, "statues_count": 2716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974398005248, "text": "And hungry. My stomach is telling me to feed it but its 2am. I ain't getting outta bed I'm comfy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198958973, "name": "Ashley®", "screen_name": "whatever_idiot", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-11-16"), "description": "Ashley Cuevas. #EndWarOnDrugs. Average. Human. Female. Brunette. Gay. Joshua ❤️", "followers_count": 634, "friends_count": 465, "statues_count": 10608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974796640256, "text": "Fuck it I'm pulling an all-nighter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457688177, "name": "christina leader", "screen_name": "followda_l3ader", "lang": "en", "location": "Margaritaville", "create_at": date("2012-01-07"), "description": "I'd rather be in Philly with my cats", "followers_count": 460, "friends_count": 359, "statues_count": 11482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millersville, PA", "id": "6651a2bba1f95658", "name": "Millersville", "place_type": "city", "bounding_box": rectangle("-76.367261,39.983906 -76.334787,40.023056") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42071, "countyName": "Lancaster", "cityID": 4249728, "cityName": "Millersville" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764974809161728, "text": "03:00:01 |Temp: 44.5ºF | Wind Chill 44.5ºF |Dew Point 37.4ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WNW, Gusting to 14.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975052304384, "text": "@ebbtideapp Tide in Woodbury Creek, New Jersey 02/25/2016\n Low 9:35am -0.3\nHigh 3:13pm 5.6\n Low 9:59pm -0.2\nHigh 3:34am 5.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.1867,39.86"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 34, "friends_count": 1, "statues_count": 10663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "National Park, NJ", "id": "a24942445f37c4a7", "name": "National Park", "place_type": "city", "bounding_box": rectangle("-75.196292,39.856789 -75.169014,39.875787") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3449680, "cityName": "National Park" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975245250560, "text": "@CaliSimba about time, you should've been asleep . goodnight ! Text me when you go to work . Oops I'm the one that forgot to reply ����", "in_reply_to_status": 702764350268780544, "in_reply_to_user": 430878362, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430878362 }}, "user": { "id": 330636743, "name": "Briana", "screen_name": "brianaxloveee", "lang": "en", "location": "Dreamville, LA ", "create_at": date("2011-07-06"), "description": "MY LIFE BE LIKE ❤️✌️", "followers_count": 449, "friends_count": 349, "statues_count": 23458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975312363524, "text": "#SupportOriginMelissa 33.8°F Wind:3.1mph Pressure: 29.68hpa Rising Rain Today 0.00in. Forecast: Fairly fine, possible showers early", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 310761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975463526400, "text": "I haven't had a phone for a week son", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 741787496, "name": "ryan", "screen_name": "Scienceinameric", "lang": "en", "location": "Degobah System", "create_at": date("2012-08-06"), "description": "the wrong version", "followers_count": 833, "friends_count": 1057, "statues_count": 21043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975815839749, "text": "I don't think its healthy that lola farts more than anyone I know, & every time she does she gives me a \"it wasn't me look\" NIGGA I HEARD IT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335607094, "name": "Tommy Trimarco", "screen_name": "TommyTrimarco", "lang": "en", "location": "19 Booze Court ", "create_at": date("2011-07-14"), "description": "I live my life a quarter of a mile at a time.", "followers_count": 310, "friends_count": 112, "statues_count": 16747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764975928967168, "text": "0h 30m wait time at Strathcona Community Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffE2jF! #SherwoodPark", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.277154,53.568215"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "SherwoodPark" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 360, "friends_count": 884, "statues_count": 2812 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Strathcona County, Alberta", "id": "79a60b51078a624f", "name": "Strathcona County", "place_type": "city", "bounding_box": rectangle("-113.402012,53.354476 -112.840174,53.893471") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976038092800, "text": "Wind 0.0 mph ---. Barometer 30.13 in, Rising slowly. Temperature 41.7 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976176496640, "text": "Vegas working on a new business deal! Launching a new system that will create massive… https://t.co/aVgkDbFATC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17453438,36.10968678"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43755592, "name": "Kai Lo", "screen_name": "kaiyabunga", "lang": "en", "location": "Philadelphia", "create_at": date("2009-05-31"), "description": "No Credit Card Needed - COMPLETE 14-Day Internet Marketing Training Course in bio link", "followers_count": 10648, "friends_count": 85, "statues_count": 2878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976247857152, "text": "02/25@03:00 - Temp 58.0F, WC 56.7F. Wind 7.2mph SSW, Gust 15.0mph. Bar 29.296in, Falling slowly. Rain 0.02in. Hum 75%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976268713985, "text": "@kanyewest Romeo and Juliet? Nah Kanye and Kim", "in_reply_to_status": -1, "in_reply_to_user": 169686021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169686021 }}, "user": { "id": 2181673352, "name": "Kevin", "screen_name": "13Kevinturtles", "lang": "en", "location": "washington ", "create_at": date("2013-11-07"), "description": "dallas cowboys and new england patriots are life", "followers_count": 28, "friends_count": 221, "statues_count": 271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976402878464, "text": "@uniqueessenceee @asifuentes14_", "in_reply_to_status": 702762067330097152, "in_reply_to_user": 2619256642, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2619256642, 3265771662 }}, "user": { "id": 569350377, "name": "Nashville", "screen_name": "JULIANNASH24", "lang": "en", "location": "Wherever life takes me. ", "create_at": date("2012-05-02"), "description": "Real Men Feel Nothing.", "followers_count": 571, "friends_count": 711, "statues_count": 21730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brawley, CA", "id": "8da9aff1e34d7b52", "name": "Brawley", "place_type": "city", "bounding_box": rectangle("-115.568757,32.953097 -115.506903,33.004816") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 608058, "cityName": "Brawley" } }
+{ "create_at": datetime("2016-02-25T00:00:01.000Z"), "id": 702764976679710720, "text": "@JetBlackForum ������", "in_reply_to_status": -1, "in_reply_to_user": 4822446302, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4822446302 }}, "user": { "id": 4822446302, "name": "SLFL 186 Days ☀︎", "screen_name": "JetBlackForum", "lang": "en", "location": "Olympia/Seattle, WA", "create_at": date("2016-01-27"), "description": "Two girls trying to see four boys at the Forum in LA, California. There's going to be a lot of fucks. Personals: @AllycatReynolds @SammmyT19 ♪", "followers_count": 801, "friends_count": 1286, "statues_count": 1045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764977422094337, "text": "@AliShahpoory کابوس من تا ۳ سال بعد از فارغ التحصیلی!‌ خواب می‌دیدم زنگ زدن می‌گن پاشو بیا وصایا پاس نکردی!", "in_reply_to_status": 702403977581895681, "in_reply_to_user": 3501825737, "favorite_count": 0, "retweet_count": 0, "lang": "fa", "is_retweet": false, "user_mentions": {{ 3501825737 }}, "user": { "id": 2328961339, "name": "Ida Nikou", "screen_name": "idanikou", "lang": "en", "location": "Houston, TX", "create_at": date("2014-02-05"), "description": "I'm here to avoid Facebook.", "followers_count": 284, "friends_count": 741, "statues_count": 723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764977627729921, "text": "Wind 1.5 mph SSE. Barometer 29.202 in, Falling Rapidly. Temperature 47.4 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764977753554945, "text": "GN��❤️ (@ Meridian Place in Northridge, CA) https://t.co/33VMkS77Mc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.5371086,34.24156181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391738949, "name": "Fatoma AlSairaFi", "screen_name": "A6oo6a_", "lang": "en", "location": "CA 91324", "create_at": date("2011-10-15"), "description": "an Engineer to be", "followers_count": 369, "friends_count": 339, "statues_count": 25611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764977854099456, "text": "������ https://t.co/b8fTpfoyDW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1731698340, "name": "ㅤ", "screen_name": "yvanadelrosario", "lang": "en", "location": "Houston, TX", "create_at": date("2013-09-05"), "description": "❃ snapchat - yvanadelrosario || swift • goulding • sheeran • lovato • maiden ❃", "followers_count": 1020, "friends_count": 430, "statues_count": 60528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764977963274240, "text": "Temp: 38.4°F Wind:0.0mph Pressure: 30.134hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764978072190978, "text": "#MuchGratitude @thecavalier @moviereviewnmor for the #awesome #opportunities! #GreatFriend #Colleague #Host #Correspondent @BeloitFilmFest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MuchGratitude", "awesome", "opportunities", "GreatFriend", "Colleague", "Host", "Correspondent" }}, "user_mentions": {{ 16943121, 1263093206, 189709797 }}, "user": { "id": 37479334, "name": "Anne McDaniels", "screen_name": "annemcdaniels", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-03"), "description": "Host/Actress/Model: Host for @MovieReviewsandMore @HollywoodFest. #Boxing #HardRock #Education #GivingBack 4eva. Spirit Animal = @SharonStone #CatherineTramell", "followers_count": 25500, "friends_count": 14140, "statues_count": 4652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764978323922944, "text": "What's the difference between @SamsungPayUS and @googlewallet? Aren't they both @Android OS?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4236702519, 309827046, 382267114 }}, "user": { "id": 27407704, "name": "Add Your Name", "screen_name": "shawnallenlive", "lang": "en", "location": "nowhere", "create_at": date("2009-03-29"), "description": "snapchat: shawnallenlive", "followers_count": 4418, "friends_count": 409, "statues_count": 175230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764978890117120, "text": "This month seems to be going by so slow!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 390765611, "name": "angela gonzalez", "screen_name": "angig81", "lang": "en", "location": "Mendota, CA ", "create_at": date("2011-10-14"), "description": "God.Mom.Lover.Daydreamer. Mendota.CA.FarmersInsuranceMendota.'81", "followers_count": 302, "friends_count": 464, "statues_count": 18225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mendota, CA", "id": "407f4285b8cc50e7", "name": "Mendota", "place_type": "city", "bounding_box": rectangle("-120.405019,36.74579 -120.368636,36.772713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 646828, "cityName": "Mendota" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764978894462976, "text": "Wind 3.0 mph NW. Barometer 29.938 in, Steady. Temperature 38.2 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764979217375232, "text": "Wind 6.0 mph W. Barometer 29.700 in, Rising. Temperature 33.6 °F. Rain today 0.01 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764979431174145, "text": "@skw907 @HarvestSnaps @joanna_markell I don't blame you...wasabi ranch snapeas changed my life", "in_reply_to_status": 702735123012124674, "in_reply_to_user": 241002140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 241002140, 1132772443, 102923323 }}, "user": { "id": 22634715, "name": "Lilah Walker", "screen_name": "lilah907", "lang": "en", "location": "Anchorage, AK", "create_at": date("2009-03-03"), "description": "null", "followers_count": 45, "friends_count": 167, "statues_count": 64 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764979540369409, "text": "Feel like I have so much stress��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331408947, "name": "saintxsantos", "screen_name": "prodsantos65", "lang": "en", "location": "null", "create_at": date("2011-07-07"), "description": "null", "followers_count": 363, "friends_count": 439, "statues_count": 17819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764979661869056, "text": "@TheSamuraiRose @NGBJoshPerez https://t.co/NKYYHFhV2Y", "in_reply_to_status": 702764471677104128, "in_reply_to_user": 395599586, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 395599586, 2668890321 }}, "user": { "id": 262483701, "name": "Adrianna Pearl", "screen_name": "HeyItsAddie", "lang": "en", "location": "Round Rock, TX", "create_at": date("2011-03-07"), "description": "I'm Addie. I'm 22 and I play video games too much. I also like tea. Potential animator in the making.\nI do art sometimes.\nhttp://AdriannaPearl.tumblr.com", "followers_count": 133, "friends_count": 276, "statues_count": 2050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764980194643968, "text": "���� damn https://t.co/GQscQz3vdi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311629669, "name": "Pearl ✨", "screen_name": "_ImVeryLoyal", "lang": "en", "location": "229 ✈ 912", "create_at": date("2011-06-05"), "description": "I just want everything I deserve !\n#SSU", "followers_count": 2086, "friends_count": 2074, "statues_count": 38567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764980500738048, "text": "Up late realizing that I live very near to more than one active volcano. At least they're very picturesque..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41970879, "name": "Krista Maxine", "screen_name": "kristamaxine", "lang": "en", "location": "Portland, OR", "create_at": date("2009-05-22"), "description": "Just a gal who likes to photograph and enjoys music. Instagram: Krista.Maxine", "followers_count": 142, "friends_count": 96, "statues_count": 6715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tualatin, OR", "id": "ee4298ad58771a16", "name": "Tualatin", "place_type": "city", "bounding_box": rectangle("-122.818314,45.349158 -122.721832,45.403778") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4174950, "cityName": "Tualatin" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764980513304576, "text": "Temp: 32.6°F - Dew Point: 22.1° - Wind: 19.8 mph - Gust: 30.4 - Rain Today: 0.00in. - Pressure: 29.70in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 13936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764980714602496, "text": "I'm so sad:(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1031847066, "name": "Hannah Lewis", "screen_name": "hannah_lewis97", "lang": "en", "location": "null", "create_at": date("2012-12-23"), "description": "1/3 of the v cats// Bakersfield College // I'm short, I know", "followers_count": 200, "friends_count": 181, "statues_count": 4052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764981004189696, "text": "@A_M_OR @Mechis_M @AdryMMP Chugging water is just as essential during cold-weather runs\n\nhttps://t.co/RvLxFw8MzI https://t.co/8WKq85rbEf", "in_reply_to_status": 702713639032193026, "in_reply_to_user": 2183423438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2183423438, 1565249677, 1320883860 }}, "user": { "id": 372864987, "name": "Nightingale Care", "screen_name": "CareNightingale", "lang": "en", "location": "Florida", "create_at": date("2011-09-13"), "description": "Senior care at home", "followers_count": 613, "friends_count": 388, "statues_count": 11611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobe Sound, FL", "id": "7d50ab28068709e7", "name": "Hobe Sound", "place_type": "city", "bounding_box": rectangle("-80.210383,27.043306 -80.116244,27.128247") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12085, "countyName": "Martin", "cityID": 1230975, "cityName": "Hobe Sound" } }
+{ "create_at": datetime("2016-02-25T00:00:02.000Z"), "id": 702764981394108416, "text": "low key miss high school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1094078694, "name": "Gabby Nelson❁", "screen_name": "GabbyNelsonnn", "lang": "en", "location": "Orange County -- Flagstaff", "create_at": date("2013-01-15"), "description": "NAU '19 || ☀️C || SLS 05•18•07 || 2 Cor. 5:7", "followers_count": 219, "friends_count": 326, "statues_count": 3301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764981557686272, "text": "I just can't fuck with people who ain't gon be 100 wit me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056619452, "name": "BRAZY", "screen_name": "BradyFHannah", "lang": "en", "location": "618", "create_at": date("2013-01-02"), "description": "members only.", "followers_count": 867, "friends_count": 619, "statues_count": 1831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764981612367873, "text": "Wo liegt Portland? https://t.co/QFWRoM3QVt #Portland #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.67,45.5239"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Portland", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 431, "friends_count": 114, "statues_count": 1873513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764981746466817, "text": "@MacMiller ������", "in_reply_to_status": -1, "in_reply_to_user": 23065354, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 23065354 }}, "user": { "id": 3356144091, "name": "ace boogie", "screen_name": "alexadenilsa", "lang": "en", "location": "Hartford, CT", "create_at": date("2015-07-02"), "description": "null", "followers_count": 62, "friends_count": 39, "statues_count": 1287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartford, CT", "id": "61c225139f635563", "name": "Hartford", "place_type": "city", "bounding_box": rectangle("-72.718386,41.723759 -72.643547,41.807475") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937000, "cityName": "Hartford" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764981985611776, "text": "She straight up killed it on Main Event from charisma, move set , taunts everything was on point https://t.co/4CEGsaKsCg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3345214551, "name": "NaomiWWE", "screen_name": "slayomi_mob", "lang": "en", "location": "null", "create_at": date("2015-06-25"), "description": "Whos the best.... Queen of the mob @NaomiWWE Fear the rear", "followers_count": 241, "friends_count": 172, "statues_count": 894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lima, OH", "id": "6b1aa33507f2e472", "name": "Lima", "place_type": "city", "bounding_box": rectangle("-84.215102,40.687562 -84.012941,40.817349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39003, "countyName": "Allen", "cityID": 3943554, "cityName": "Lima" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764982325235712, "text": "Ig too much rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2150125614, "name": "Oscar Sanders", "screen_name": "Oscarin_95", "lang": "en", "location": "chillin", "create_at": date("2013-10-22"), "description": "snapchat : oscarinnn95 ig : Oscarin_95", "followers_count": 427, "friends_count": 393, "statues_count": 19776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764982723710976, "text": "@Anvb25 ������������❄️ https://t.co/4V7rNvFu2r", "in_reply_to_status": -1, "in_reply_to_user": 2806328142, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2806328142 }}, "user": { "id": 2694311829, "name": "BlAS'E", "screen_name": "MuniroFromThe6", "lang": "en", "location": "Renton, WA", "create_at": date("2014-07-10"), "description": "I slay you bitches ✨ sc:Muniro_hussien #JUSTICE4HAMZA", "followers_count": 727, "friends_count": 396, "statues_count": 7727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Renton, WA", "id": "a96f188f5de647b5", "name": "Renton", "place_type": "city", "bounding_box": rectangle("-122.253717,47.423263 -122.135079,47.539933") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357745, "cityName": "Renton" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983285784576, "text": "Want In-n-Out.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3029382771, "name": "Varsha Pandey", "screen_name": "varshapandey13", "lang": "en", "location": "null", "create_at": date("2015-02-10"), "description": "null", "followers_count": 44, "friends_count": 71, "statues_count": 586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983294103553, "text": "Andy is always my favorite idc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3733345693, "name": "Avery Nicole", "screen_name": "Aves_nicole97", "lang": "en", "location": "Houston, TX", "create_at": date("2015-09-29"), "description": "know yourself, know your worth", "followers_count": 289, "friends_count": 319, "statues_count": 79 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983466102784, "text": "@modene1 @Campaign_Trump Stop using my dog's picture.", "in_reply_to_status": 702759461958496256, "in_reply_to_user": 21423073, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21423073, 3314758074 }}, "user": { "id": 4855679640, "name": "AllAboutTruth", "screen_name": "NoMorePolitical", "lang": "en", "location": "Southern CA", "create_at": date("2016-02-03"), "description": "null", "followers_count": 138, "friends_count": 236, "statues_count": 1321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983495499776, "text": "We're all a little bit fucked up but that's okay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1539624019, "name": "Christina Sanchez", "screen_name": "christinaallena", "lang": "en", "location": "Colorado, USA", "create_at": date("2013-06-22"), "description": "snap: christinaallena", "followers_count": 434, "friends_count": 342, "statues_count": 6351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highlands Ranch, CO", "id": "2571b7720cd62ad3", "name": "Highlands Ranch", "place_type": "city", "bounding_box": rectangle("-105.053666,39.5033 -104.899868,39.566287") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 836410, "cityName": "Highlands Ranch" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983524814852, "text": "One of the most creative things I've ever seen and it's an opera. Thank you captainpatten for… https://t.co/M7q00TjK5A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.24861111,34.05638889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39014360, "name": "Chiara Gillette", "screen_name": "ChiaraGillette", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-10"), "description": "DreamWorker Troll * Maker of movies * Writer of crap * Eater of purple people * Watcher of shows * Consumer of things * Reader of novels * Crier of tears", "followers_count": 228, "friends_count": 805, "statues_count": 6073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983541575681, "text": "Love love ������������������ – ♫Maometto II: Overture by Academy of St. Martin in the Fields & Sir Neville Marriner, from #SoundHound", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SoundHound" }}, "user": { "id": 254791401, "name": "Elias Mbvukuta", "screen_name": "mbvukutaphiri", "lang": "en", "location": "Davis, California, USA", "create_at": date("2011-02-19"), "description": "Music nut and news junkie. Father of three and citizen of the world.", "followers_count": 1493, "friends_count": 2195, "statues_count": 468489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davis, CA", "id": "1994142e26ba7127", "name": "Davis", "place_type": "city", "bounding_box": rectangle("-121.803252,38.526843 -121.675074,38.590264") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 618100, "cityName": "Davis" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983579316225, "text": "⚡️ “Here's Kanye West's entire Grammys tweetstorm ” by @micnews\n\nhttps://t.co/0ne4LHh1xW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139909832 }}, "user": { "id": 45027998, "name": "Yuriy Rzhemovskiy", "screen_name": "yuriyr", "lang": "en", "location": "San Francisco", "create_at": date("2009-06-05"), "description": "Adventuring, learning and humanizing technology. Designer at @twitter.", "followers_count": 3146, "friends_count": 705, "statues_count": 6065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764983889694720, "text": "@emily_0013 hoooo ma god��", "in_reply_to_status": 702764910732677120, "in_reply_to_user": 1059484526, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1059484526 }}, "user": { "id": 846824299, "name": "Victoria➰", "screen_name": "IckyVickyyyyy", "lang": "en", "location": "Los Banos, CA", "create_at": date("2012-09-25"), "description": "lbhs", "followers_count": 522, "friends_count": 466, "statues_count": 13596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Banos, CA", "id": "b1b6fc646de75904", "name": "Los Banos", "place_type": "city", "bounding_box": rectangle("-120.879833,37.033745 -120.803053,37.088185") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 644028, "cityName": "Los Banos" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764984825065472, "text": "Maybe you should ask yourself that https://t.co/dWwdtNWGY8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1133093275, "name": "Travis", "screen_name": "Yungtrav93", "lang": "en", "location": "null", "create_at": date("2013-01-29"), "description": "Don't waste my time. 22 years young. 580✈️918 Phil 4:13. On the road to be successful.", "followers_count": 1091, "friends_count": 1000, "statues_count": 40651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764985122840576, "text": "oh geez", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574697328, "name": "melee melee melee", "screen_name": "ScumBog", "lang": "en", "location": "San Diego", "create_at": date("2012-05-08"), "description": "ehwah || SCUM SQUAD || I main Falco :^)", "followers_count": 613, "friends_count": 133, "statues_count": 37944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764985349500928, "text": "I might as well call it a night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421264231, "name": "g0ldTop", "screen_name": "g0ldenHighTop", "lang": "en", "location": "where there's ☮ ", "create_at": date("2011-11-25"), "description": "fiu18", "followers_count": 1774, "friends_count": 744, "statues_count": 65133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naranja, FL", "id": "00531e7c63964b3d", "name": "Naranja", "place_type": "city", "bounding_box": rectangle("-80.453594,25.504351 -80.405043,25.536465") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1247700, "cityName": "Naranja" } }
+{ "create_at": datetime("2016-02-25T00:00:03.000Z"), "id": 702764985361903616, "text": "@Kuwait_CU \n@malek_alothman \nhttps://t.co/Rn8PDny0Wt", "in_reply_to_status": 700068493408890880, "in_reply_to_user": 2434094460, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2434094460, 2631478722 }}, "user": { "id": 243370617, "name": "Bader Abuqamaz", "screen_name": "baderbq", "lang": "en", "location": "Boulder, CO", "create_at": date("2011-01-26"), "description": "Never Stop Dreaming", "followers_count": 403, "friends_count": 476, "statues_count": 5094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764986234372096, "text": "I reminisce when I get high", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2694762565, "name": "al〽ightylex ひ", "screen_name": "kenshinalexa", "lang": "en", "location": "dallas flexa$ - dirty lew", "create_at": date("2014-07-30"), "description": "heart full of pride & a mouth full of gold", "followers_count": 1044, "friends_count": 677, "statues_count": 14169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764986649681920, "text": "I'm still dead https://t.co/5vIOm3zhVd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289129660, "name": "Mal of Dem Boy$", "screen_name": "ImDatNigga_Mal", "lang": "en", "location": "NawfDallas ✈ PV", "create_at": date("2011-04-27"), "description": "these people think they classy. well that's the class im skippin |IG: DatBoi_Mal |", "followers_count": 3042, "friends_count": 1130, "statues_count": 201332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764986880229376, "text": "Wind 6.0 mph NNE. Barometer 30.012 in, Rising slowly. Temperature 32.6 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764986951725057, "text": "Don't beg for my time, then waste it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3126525459, "name": "Co-Co ❤️", "screen_name": "SpeakingonTee", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "Not Average ❄️", "followers_count": 1510, "friends_count": 1001, "statues_count": 20959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764987119333376, "text": "@JuiZeeen__ A", "in_reply_to_status": -1, "in_reply_to_user": 2873617172, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2873617172 }}, "user": { "id": 520883187, "name": "Sombre Djak's", "screen_name": "Alfoussen_94", "lang": "fr", "location": "Phoenix, AZ", "create_at": date("2012-03-10"), "description": "Tous conçu de la même façon donc pourquoi aurais je peur de l'un d'eux ! Sénégalais-Guinéen Djack's ❌ c'est bloqué ❌", "followers_count": 1347, "friends_count": 993, "statues_count": 48229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764987371008001, "text": "IF AN ANIMAL HAS BEEN ABUSED U NEED TO SEND IT TO ME I WILL MAKE SURE IT FORGETS ITS EVER BEEN IN PAIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 556413365, "name": "Lil", "screen_name": "LillyMolerr", "lang": "en", "location": "Hillsboro, OR", "create_at": date("2012-04-17"), "description": "Oregon | Southridge Senior", "followers_count": 814, "friends_count": 390, "statues_count": 26679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764987668758528, "text": "BBC News - Why do stuntmen not have an Oscar? https://t.co/q4Rd8MSj5l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22703884, "name": "Juliana Potter", "screen_name": "jtpotter", "lang": "en", "location": "null", "create_at": date("2009-03-03"), "description": "mini stuntwoman. kid double, actress, dancer, teacher. Lover of all things blissful, challenging and wild.", "followers_count": 317, "friends_count": 669, "statues_count": 475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermosa Beach, CA", "id": "4baf4d09759d33e4", "name": "Hermosa Beach", "place_type": "city", "bounding_box": rectangle("-118.409579,33.85185 -118.384866,33.877823") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 633364, "cityName": "Hermosa Beach" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764988146937856, "text": "Walked outside and walked right back inside boyyyy it's cold as pie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195589801, "name": "Hook-Island", "screen_name": "Respect_TheHook", "lang": "en", "location": "Richmond Tx", "create_at": date("2010-09-26"), "description": "*Strive For Greatness* Photographer TDK Semi Pro Football *DBU* N.O.C. - Philippians 4-13... https://www.youtube.com/channel/UCIbOVSnDRiTpAD3CBjFP9lg", "followers_count": 888, "friends_count": 559, "statues_count": 33257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764988427993088, "text": "@NotChrisSmith Please watch -- This guy is black & 10X more articulate than ill ever be. https://t.co/edBizf8N23 @mattkola1776 @teyegirlily", "in_reply_to_status": 702761880725561344, "in_reply_to_user": 36375465, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36375465, 3377100196, 823809320 }}, "user": { "id": 824761309, "name": "ﻥ Derek", "screen_name": "dwp1970", "lang": "en", "location": "Eugene, Or. (Go Ducks!)", "create_at": date("2012-09-15"), "description": "God. Country. Family. Conservative. RIGHTS SETTLED 1791.Obama's USA=Failure,Surrender,Dysfunction. Web/Net(IT)Guitar!Tweets=me,RT≠ #2A #AmYisraelChai ♥RIP D.J.♥", "followers_count": 2395, "friends_count": 2631, "statues_count": 19788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764989036130304, "text": "It's onnnnnnnn ���� https://t.co/Ho7a4s4HsT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2904166988, "name": "BANGBROS", "screen_name": "vales_josh21", "lang": "en", "location": "San Jose, CA", "create_at": date("2014-11-18"), "description": "R.I.P Walter Matthew Vales// R.I.P Philip Thomas Vales ✊ ISAIAH57:1-2", "followers_count": 314, "friends_count": 310, "statues_count": 1720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764989308915714, "text": "@amandaluella_ grow up", "in_reply_to_status": -1, "in_reply_to_user": 2906562495, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2906562495 }}, "user": { "id": 317944472, "name": "Connor Jones", "screen_name": "RealConnorJones", "lang": "en", "location": "Kickin' It With Chevy Chase", "create_at": date("2011-06-15"), "description": "'And if you ain't a hoe, get up out my traphouse' -Bobby Shmurda", "followers_count": 522, "friends_count": 443, "statues_count": 4748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Carlos Park, FL", "id": "46bf69ae15f33611", "name": "San Carlos Park", "place_type": "city", "bounding_box": rectangle("-81.845887,26.452722 -81.797192,26.492899") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1263425, "cityName": "San Carlos Park" } }
+{ "create_at": datetime("2016-02-25T00:00:04.000Z"), "id": 702764989900148736, "text": "I'm not even shocked anymore when people let me down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2916731419, "name": "gabyyyy", "screen_name": "gabydavenia", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-12-02"), "description": "20 & Blessed", "followers_count": 537, "friends_count": 364, "statues_count": 1569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764990034419713, "text": "@bellahredahh @rachelkobe literally this has pissed me off for months but idk who to take that up with ����", "in_reply_to_status": 702751894154964993, "in_reply_to_user": 815586488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 815586488, 2203729350 }}, "user": { "id": 725295895, "name": "beck beck", "screen_name": "sparkleina", "lang": "en", "location": "Alsip, IL", "create_at": date("2012-07-29"), "description": "20 • great things never came from comfort zones •", "followers_count": 678, "friends_count": 700, "statues_count": 4346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alsip, IL", "id": "9fdc3d1edf51a0a0", "name": "Alsip", "place_type": "city", "bounding_box": rectangle("-87.778536,41.650997 -87.694882,41.690972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1701010, "cityName": "Alsip" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764990126624768, "text": "ADELA. Minsan magaling pero Madalas Napakagaling. Happy Birthday Maricel Soriano! #HBDDiamondStarMS https://t.co/F5ZYYdYtDk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "HBDDiamondStarMS" }}, "user": { "id": 114409693, "name": "Maricel Soriano News", "screen_name": "DiamondStarNews", "lang": "en", "location": "ÜT: 33.879317,-117.880425", "create_at": date("2010-02-15"), "description": "Tweets & opinions of this FANPAGE does not reflect that of Maricel Soriano or her management. News & updates about the Philippines' Diamond Star,Maricel Soriano", "followers_count": 11227, "friends_count": 2055, "statues_count": 66528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764990961299457, "text": "@pinkgoldclang the classic movie, \"I, Clangbot\"", "in_reply_to_status": 702761259146616832, "in_reply_to_user": 4827574266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4827574266 }}, "user": { "id": 19745748, "name": "Stu Senpai", "screen_name": "StuSenpai", "lang": "en", "location": "Not Quite Montana-Land", "create_at": date("2009-01-29"), "description": "A real life spooky spook", "followers_count": 211, "friends_count": 168, "statues_count": 25214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991204630528, "text": "An other success! @burkhalterdiary rocks! @rawartistsla \nPh. stevemanivongstudios… https://t.co/gs87KyMSD0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2540283,34.0439682"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1779553970, 225524046 }}, "user": { "id": 111388519, "name": "Stephanie Burkhalter", "screen_name": "StephBurkhalter", "lang": "en", "location": "Los Angeles", "create_at": date("2010-02-04"), "description": "Burkhalter Couture Creative Director @BurkhalterDiary - Model", "followers_count": 1466, "friends_count": 442, "statues_count": 6321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991296868352, "text": "Death by clarinet...that would be awesome...#bones", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.81144724,33.37129783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bones" }}, "user": { "id": 375690790, "name": "Suri Reale", "screen_name": "SuriReale", "lang": "en", "location": "Phoenix", "create_at": date("2011-09-18"), "description": "The Painful Truth", "followers_count": 208, "friends_count": 187, "statues_count": 6681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991397568512, "text": "Bout to watch da bulls game was sleep earlier but gotta watch my team get this very much needed win to stay in this playoff hunt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41296878, "name": "Deon Jones", "screen_name": "ten6forever", "lang": "en", "location": "chicago", "create_at": date("2009-05-19"), "description": "chi-town 10-6 wentworth all day grindin R.I.P King Kujo RIP Dub and Slim #⛽️ang⛽️ #SquadShit Free Dem Demons", "followers_count": 164, "friends_count": 988, "statues_count": 759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Chicago, IN", "id": "41bc24e82dfe5a2c", "name": "East Chicago", "place_type": "city", "bounding_box": rectangle("-87.490523,41.609953 -87.406819,41.689616") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1819486, "cityName": "East Chicago" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991523323904, "text": "@christheguard I'm just going to stop. �������� - you're a goon brother, you're a goon.", "in_reply_to_status": 702764326805843968, "in_reply_to_user": 3147022350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3147022350 }}, "user": { "id": 1260677726, "name": "ZAY", "screen_name": "Izzy_Q3", "lang": "en", "location": "The Deepest Part of the Sea", "create_at": date("2013-03-11"), "description": "Follow, let's talk| You tweet to enjoy, I tweet to die. | writer. | whiskey. | PNW. | 206 | Poseidon | Water Bender | Eat her up like mochi | V X 3", "followers_count": 582, "friends_count": 572, "statues_count": 17885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340391 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991758270464, "text": "Love never gives up, \nnever loses faith,\nalways hopeful,\nand endures through\nevery circumstance.\n1 Corinthians 13:7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 123, "friends_count": 237, "statues_count": 181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764991762419713, "text": "Unappreciated.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2242727360, "name": "A", "screen_name": "lisheeeeee", "lang": "en", "location": "SD", "create_at": date("2013-12-12"), "description": "from the westside w. love // Jacob ✨", "followers_count": 837, "friends_count": 225, "statues_count": 37343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764992546754560, "text": "@Cassmontes_ oh yeah lol", "in_reply_to_status": 702764820613869568, "in_reply_to_user": 2253775357, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2253775357 }}, "user": { "id": 1414192818, "name": "Garcia lll ⚡️", "screen_name": "GarciaGiggles6", "lang": "en", "location": "Hawaii ", "create_at": date("2013-05-08"), "description": "#NoFlyZone. #6", "followers_count": 494, "friends_count": 498, "statues_count": 48578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764992920100864, "text": "Never do, never do. https://t.co/gU9wtuJPto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156491180, "name": "Tyler Ziskin", "screen_name": "TZiskBuff", "lang": "en", "location": "Denver, CO", "create_at": date("2010-06-16"), "description": "#CUBuffs fanatic. Big dude, brutally honest. Pride, respect, and family and you know me. I tweet my truth. Fan Correspondent for @buff_stampede", "followers_count": 1300, "friends_count": 1179, "statues_count": 30242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-02-25T00:00:05.000Z"), "id": 702764993276547073, "text": "Today is the last of five open houses .@RideSacRT will hold on a proposed fare increase. Will you join us at 5:30pm? https://t.co/TDQEjRQTEj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316621010 }}, "user": { "id": 1381072680, "name": "Ride Downtown 916", "screen_name": "RideDowntown916", "lang": "en", "location": "Downtown Sacramento, CA 95811", "create_at": date("2013-04-25"), "description": "A grassroots organization showing support for using public transportation throughout the 7-County, 29-City @SACOG & @SJCOG Regions to & from @docosacramento.", "followers_count": 934, "friends_count": 1714, "statues_count": 24063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764994656468993, "text": "Cutie got me an edible ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4086125116, "name": "lenasa", "screen_name": "Lenasalinas_", "lang": "en", "location": "East Los Angeles, CA", "create_at": date("2015-10-31"), "description": "21", "followers_count": 41, "friends_count": 33, "statues_count": 887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764994786504704, "text": "White ppl staying ruining everything �� https://t.co/AIxOXUVPG6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2298792746, "name": "❄️Aflex❄️", "screen_name": "shelovesAflex", "lang": "en", "location": "wolverines home court", "create_at": date("2014-01-18"), "description": "Evergreen high school 6'0 PG/SG class of 18 #ogadenia SC: abdifatah-206 @xoxoaliaa❤️", "followers_count": 650, "friends_count": 435, "statues_count": 10652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Center, WA", "id": "3a201fc1ccc493b8", "name": "White Center", "place_type": "city", "bounding_box": rectangle("-122.374253,47.488444 -122.324769,47.517465") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5378225, "cityName": "White Center" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764995256393728, "text": "No ahi nadie aquí", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1700451121, "name": "Marie Xioana ✨", "screen_name": "Xioana_Marie", "lang": "es", "location": "Waterbury, CT", "create_at": date("2013-08-25"), "description": "Redes ➰♦Sc- xio321♦ Phhhoto- Xioana19 ♦ Ig- xioana_15 ♦Tumblr- xio321", "followers_count": 1135, "friends_count": 1375, "statues_count": 30499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterbury, CT", "id": "f9a4ed4d94a9b93f", "name": "Waterbury", "place_type": "city", "bounding_box": rectangle("-73.093845,41.513971 -72.950371,41.619186") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 980000, "cityName": "Waterbury" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764995709313024, "text": "Long days call for little sleep ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490121142, "name": "DK", "screen_name": "dylank_125", "lang": "en", "location": "Brookings, So.Dak.", "create_at": date("2012-02-11"), "description": "Denver Broncos/Ohio State Fanatic. SDSU. Jackrabbit Club Baseball #11 ⚾️ #ShockTheNation", "followers_count": 924, "friends_count": 600, "statues_count": 15045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookings, SD", "id": "e460ea685ce41cb4", "name": "Brookings", "place_type": "city", "bounding_box": rectangle("-96.824461,44.253728 -96.744992,44.33") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46011, "countyName": "Brookings", "cityID": 4607580, "cityName": "Brookings" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764996019798016, "text": "@KBakaTHEBEST23 3much exclusives pls chill pls. plssssssssss", "in_reply_to_status": 702763302422761472, "in_reply_to_user": 585198137, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 585198137 }}, "user": { "id": 1037820210, "name": "Omarii", "screen_name": "_omlar", "lang": "en", "location": "Indiana, USA", "create_at": date("2012-12-26"), "description": "sc: omarfinna ~ U4:EA", "followers_count": 761, "friends_count": 618, "statues_count": 7757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobart, IN", "id": "5264b8cb7f94920d", "name": "Hobart", "place_type": "city", "bounding_box": rectangle("-87.317258,41.499162 -87.209636,41.565107") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1834114, "cityName": "Hobart" } }
+{ "create_at": datetime("2016-02-25T00:00:06.000Z"), "id": 702764996934004736, "text": "@BookMartialArts I greatly appreciate����the follow today & I highly value you doing so! I'm also on #Instagram https://t.co/RkvhG9PDu5", "in_reply_to_status": -1, "in_reply_to_user": 287864718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Instagram" }}, "user_mentions": {{ 287864718 }}, "user": { "id": 132249194, "name": "#DJ2B #UFCLondon", "screen_name": "DJ_2BoUnCe", "lang": "en", "location": "#SpinningTracks", "create_at": date("2010-04-12"), "description": "#UFConFox19 #CFFC56 #FMD9 #RFA35 #WomenKickAss #InvictaFC16 #WSOFGlobal #TUF23 #WMMA #WWE #Glory27 @LFC51 #ONEFC #IMMAF #WSOF29 #UFC196 #UFCFightNight84", "followers_count": 9549, "friends_count": 8935, "statues_count": 203606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127357566033920, "text": "I want to make Mandy my queen but she not feeling me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593162543, "name": "Box Connoisseur", "screen_name": "MalcolmTooXtra", "lang": "en", "location": "San Antonio, Texas ", "create_at": date("2012-05-28"), "description": "The following tweets do not represent the views of my employers nor do they represent the views of this tweeter. Viewer discretion advised", "followers_count": 1707, "friends_count": 995, "statues_count": 161499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scenic Oaks, TX", "id": "006397db03a23ca1", "name": "Scenic Oaks", "place_type": "city", "bounding_box": rectangle("-98.693075,29.65431 -98.631762,29.738805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4866089, "cityName": "Scenic Oaks" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127357662453761, "text": "Wind 0.0 mph ---. Barometer 1034.48 mb, Steady. Temperature 24.4 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127357851197440, "text": "I just threw up my bdubs I'm sad :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 378956331, "name": "valarie.", "screen_name": "ChuntiVal", "lang": "en", "location": "null", "create_at": date("2011-09-23"), "description": "null", "followers_count": 1029, "friends_count": 516, "statues_count": 51478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127357909897217, "text": "@joebombs25 it was awful. The baby and I were awake at 3 in morning both of us crying bc it wouldn't shut up. Good luck", "in_reply_to_status": 703127167664656384, "in_reply_to_user": 85939674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85939674 }}, "user": { "id": 438002627, "name": "maddie koby", "screen_name": "maddieamaya", "lang": "en", "location": "walla walla, wa", "create_at": date("2011-12-15"), "description": "I do not care for your casual misogyny || CWU '19 || Instagram: @maddieamaya", "followers_count": 286, "friends_count": 495, "statues_count": 6708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellensburg, WA", "id": "c95cdb2a983262e5", "name": "Ellensburg", "place_type": "city", "bounding_box": rectangle("-120.582586,46.958017 -120.49726,47.028542") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53037, "countyName": "Kittitas", "cityID": 5321240, "cityName": "Ellensburg" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358144794625, "text": "#tappycoloring #coloringtherapy #onmyphone #itsfun #pinks #insomnia #kisses #�� #cbt… https://t.co/EqHcKvM8xz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.27924256,43.07694022"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "tappycoloring", "coloringtherapy", "onmyphone", "itsfun", "pinks", "insomnia", "kisses", "cbt" }}, "user": { "id": 61050268, "name": "LisaLisa Pearlman", "screen_name": "nocultjam", "lang": "en", "location": "Thru the Vatican?", "create_at": date("2009-07-28"), "description": "Looking@abyss Destination: Cat Heaven Island.", "followers_count": 51, "friends_count": 639, "statues_count": 1043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wisconsin, USA", "id": "7dc5c6d3bfb10ccc", "name": "Wisconsin", "place_type": "admin", "bounding_box": rectangle("-92.889433,42.491921 -86.24955,47.309715") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5548000, "cityName": "Madison" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358547427328, "text": "I'm not trying to spend the night baby, some good head and a fat ass for this long dick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3190496185, "name": "Cesar", "screen_name": "YoungCeazerXXX", "lang": "en", "location": "Watts, CA", "create_at": date("2015-05-09"), "description": "It all started from an idea... Retirement at 30 is the goal... Fuck only bad bitches but complain I want a good girl... #YC #NiceGuy #BadInfluence", "followers_count": 55, "friends_count": 106, "statues_count": 2118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358555828224, "text": "@ForsakenTongues https://t.co/Tc8yzt23EG", "in_reply_to_status": 703109857864364032, "in_reply_to_user": 2249537973, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2249537973 }}, "user": { "id": 2314512294, "name": "*$aucyNigga✈️⛽️", "screen_name": "StayTalented_", "lang": "en", "location": "Chasin Bread ✈️", "create_at": date("2014-01-27"), "description": "[GAYY]✨ , •\\ employed⚡️ / 071215⚓️", "followers_count": 3139, "friends_count": 2977, "statues_count": 38658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358597795840, "text": "If she dealing with me Just know Imma Treat you the Best", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63381077, "name": "Blueprint", "screen_name": "TheRealTlove_", "lang": "en", "location": "Bank", "create_at": date("2009-08-05"), "description": "When God is with you who can be against you?They say you are what you eat..(That's funny) I don't remember eating a goat!!", "followers_count": 1088, "friends_count": 706, "statues_count": 62883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358698450944, "text": "I wonder if the people on Naboo knew about Padme's husband and children.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2809645833, "name": "Sir Frye", "screen_name": "Jager_Bradley", "lang": "en", "location": "San Francisco,CA", "create_at": date("2014-10-06"), "description": "Hardcore AC enthusiast, padawan learner, shoddy artist, real person. I talk a lot. PSN:Lollisdo Instagram:Jager_Bradley", "followers_count": 117, "friends_count": 825, "statues_count": 3171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127358920794112, "text": "I cannot wait to go to Norway", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19266489, "name": "Aaryn", "screen_name": "aaryn_boldt", "lang": "en", "location": "null", "create_at": date("2009-01-20"), "description": "she smells of daisies", "followers_count": 667, "friends_count": 125, "statues_count": 508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barberton, WA", "id": "56ea62265afdd34d", "name": "Barberton", "place_type": "city", "bounding_box": rectangle("-122.639363,45.693051 -122.583735,45.730653") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5304195, "cityName": "Barberton" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127359176577024, "text": "They are just dominating! https://t.co/V9rRQpfWlt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122721074, "name": "Ric Meyer©", "screen_name": "HooliganHawk", "lang": "en", "location": "The Lilac City ", "create_at": date("2010-03-13"), "description": "DIE HARD SEAHAWKS FAN. #GoHawks Love other sports too. Ecclesiastes 8:15 Happily married with 3 kids.", "followers_count": 1587, "friends_count": 1489, "statues_count": 23190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127359231135745, "text": "Feels so good to be home �� @vctreng", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2207863664 }}, "user": { "id": 3648923417, "name": "Madelyn Wightman", "screen_name": "MWightman06", "lang": "en", "location": "Providence, RI", "create_at": date("2015-09-13"), "description": "I never feared death or dying\nI only fear never trying\nI am whatever I am,\nOnly God can judge me now\n\n2 Chainz - We Own It", "followers_count": 88, "friends_count": 186, "statues_count": 369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127359415648256, "text": "@itstaylorbabeee go to sushi loco best place ever changes lives", "in_reply_to_status": 703127170806259712, "in_reply_to_user": 493700147, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493700147 }}, "user": { "id": 439930322, "name": "Cordell Tidwell", "screen_name": "_groovyC", "lang": "en", "location": "Alliance, OH", "create_at": date("2011-12-18"), "description": "702 Youngin From the CITY Mount Union D3 National CHAMPIONS Football", "followers_count": 1536, "friends_count": 1260, "statues_count": 37155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green, OH", "id": "009103ea59aa47b3", "name": "Green", "place_type": "city", "bounding_box": rectangle("-81.54525,40.872987 -81.416329,40.997434") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3931860, "cityName": "Green" } }
+{ "create_at": datetime("2016-02-26T00:00:00.000Z"), "id": 703127360325824513, "text": "������ https://t.co/c6Dm23QXaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1307558317, "name": "Matt Silverman", "screen_name": "mattsilverman55", "lang": "en", "location": "Lost in the Sauce", "create_at": date("2013-03-27"), "description": "|All Section First Team Tweeter|56|I hate kids that run to class|Co-Prez of NSPA|snapchat: msilverman|", "followers_count": 754, "friends_count": 632, "statues_count": 4964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127361378611201, "text": "we just need a face 2 face, u could pick the time & the placeeeee!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264245230, "name": "only nannie matter❤️", "screen_name": "_nekkac", "lang": "en", "location": "null", "create_at": date("2011-03-11"), "description": "null", "followers_count": 3486, "friends_count": 826, "statues_count": 42847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127361638703104, "text": "happy anniversary to us. mallique your honestly the biggest pain in the ass I've ever met but I love it. I love you�� https://t.co/o9qY4EYJvk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2984779688, "name": "Heavitaaa", "screen_name": "bachelmataa", "lang": "en", "location": "null", "create_at": date("2015-01-15"), "description": "cold as the winter wind when it breeze yo • @king_heavyy •", "followers_count": 582, "friends_count": 448, "statues_count": 7742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127361756094464, "text": "https://t.co/Rbqds2iCkP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 401710943, "name": "MarkGeoffreyKirshner", "screen_name": "MarkGKirshner", "lang": "en", "location": "NY", "create_at": date("2011-10-30"), "description": "#Art #vaw #UN #HeForShe https://t.co/1B3kyHW7j2 http://t.co/veJ4VxKmu7 \n https://t.co/tdAFudkoRH \n http://t.co/Y3Z4waGRW4 \n http://t.co/uUOIuWalTD", "followers_count": 137261, "friends_count": 71635, "statues_count": 1086889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Uniondale, NY", "id": "6ea95c38a0f6ebfe", "name": "Uniondale", "place_type": "city", "bounding_box": rectangle("-73.609294,40.684017 -73.57501,40.722199") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3676089, "cityName": "Uniondale" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127361869389824, "text": "@rescuethedonut damn!������ https://t.co/ZFIKlyhXBJ", "in_reply_to_status": 703127159225757696, "in_reply_to_user": 1940105654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1940105654 }}, "user": { "id": 1940105654, "name": "anastasia egorova", "screen_name": "rescuethedonut", "lang": "ru", "location": "Raleigh, NC", "create_at": date("2013-10-06"), "description": "0100010101101110011101000110010101110010001000000111010001101000011001010010000001010110011011110110100101100100", "followers_count": 174, "friends_count": 99, "statues_count": 1717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127361974177792, "text": "03:00:02 |Temp: 35.5ºF | Wind Chill 35.5ºF |Dew Point 29.9ºF | Rain today: 0.00 inches | Wind: 2.0 mph from the NNW, Gusting to 3.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127362238414848, "text": "@YFNLUCCI #WishMeWell2 @ Lawndale, California https://t.co/c7JWJXOioO", "in_reply_to_status": -1, "in_reply_to_user": 407329869, "favorite_count": 0, "coordinate": point("-118.354,33.8868"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "hashtags": {{ "WishMeWell2" }}, "user_mentions": {{ 407329869 }}, "user": { "id": 1968064338, "name": "lEMONS$infinity710", "screen_name": "Plugz18", "lang": "en", "location": "Greece", "create_at": date("2013-10-17"), "description": "https://t.co/nTUyVm7Acc", "followers_count": 811, "friends_count": 3242, "statues_count": 839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawndale, CA", "id": "cce33d74ceffbe08", "name": "Lawndale", "place_type": "city", "bounding_box": rectangle("-118.369186,33.872914 -118.343796,33.902665") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640886, "cityName": "Lawndale" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127362414612480, "text": "Barberton Oh Temp:22.5°F Wind:6 mph Dir:NW Baro:Rising slowly Rain2day:0.00in Hum:79% UV:0.0 @ 03:00 02/26/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 229, "statues_count": 114927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127362653687809, "text": "And with a slight bump, I've arrived. Treat me well, Portland (and Calamackas)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540973445, "name": "Toler Webb", "screen_name": "Dimsunlight", "lang": "en", "location": "North Carolina", "create_at": date("2012-03-30"), "description": "Pokemon VGC 2012 Senior World & 2015 Masters National Champion. Boiler Room coach. High schooler, singer, runner, and uh.. nerd.", "followers_count": 1109, "friends_count": 193, "statues_count": 751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127362657918976, "text": "3:00am Temp: 31.7°F (High 33.2/Low 31.7) Wind: WNW 7mph Wind Gust: 18 Pressure: 29.796in. UV Index: 0.0 Rain: 0.0mm. #MDwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.54472222,39.7025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MDwx" }}, "user": { "id": 702673227945021444, "name": "Josh Henderson", "screen_name": "SVObservatoryWX", "lang": "en", "location": "Maryland, USA", "create_at": date("2016-02-24"), "description": "Amateur Astronomer, Storm Chaser Meteorologist. Providing weather conditions for Harford County. MD, USA NWS, NOAA, CWOP, CoCoRaHS, ACON JHU Applied Physics Lab", "followers_count": 0, "friends_count": 7, "statues_count": 84 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127362662068227, "text": "I'm just walking through Stillwater by myself and no one will answer my call to make sure I'm safe. Cool friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343040837, "name": "Sheradon Smith", "screen_name": "SheradonSmith", "lang": "en", "location": "Arkansas, USA", "create_at": date("2011-07-26"), "description": "classy but I cuss a little", "followers_count": 1208, "friends_count": 455, "statues_count": 20334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127363102441472, "text": "Friday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2422607269, "name": "M.A.C", "screen_name": "LilTuroAii", "lang": "en", "location": "null", "create_at": date("2014-04-01"), "description": "You See I'm From The Bay Biiiiiiiitch", "followers_count": 291, "friends_count": 196, "statues_count": 7132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127363358330880, "text": "02/26@03:00 - Temp 36.7F, WC 30.6F. Wind 7.9mph WNW, Gust 19.0mph. Bar 29.795in, Rising. Rain 0.00in. Hum 64%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127363962273793, "text": "@_baybayallday_ o", "in_reply_to_status": 701984972140187649, "in_reply_to_user": 2463821646, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2463821646 }}, "user": { "id": 85973278, "name": "alley", "screen_name": "AlleyWehrman", "lang": "en", "location": "Anchorage, AK", "create_at": date("2009-10-28"), "description": "18", "followers_count": 1225, "friends_count": 808, "statues_count": 36557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364159442944, "text": "@ebbtideapp Tide in Southwest Pass, Louisiana 02/26/2016\n Low 4:35am 0.2\nHigh 12:01pm 0.4\n Low 6:49pm 0.1\nHigh 12:58am 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-89.4283,28.9317"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 36, "friends_count": 1, "statues_count": 10944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22075, "countyName": "Plaquemines" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364180402176, "text": "If young metro don't trust you....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2234171880, "name": "HBK Jay", "screen_name": "HBKjayy", "lang": "en", "location": "Lemoore, CA", "create_at": date("2013-12-07"), "description": "I'm just worried about my momma worrying less. -TrillerHive", "followers_count": 987, "friends_count": 769, "statues_count": 16277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364302012416, "text": "Temp: 28.8°F - Dew Point: 13.0° - Wind: 9.3 mph - Gust: 13.9 - Rain Today: 0.00in. - Pressure: 30.17in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 13983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364583043073, "text": "#NowPlaying on @SXMTheHighway: I'm listening to Backroad Song by @GrangerSmith ❤❤������❤❤ https://t.co/XlYDgQn7YT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NowPlaying" }}, "user_mentions": {{ 69188130, 25901146 }}, "user": { "id": 2215012364, "name": "Debbie Leon", "screen_name": "DebbieLeon31", "lang": "en", "location": "Frisco, TX", "create_at": date("2013-11-25"), "description": "Love all kinds of Music! From Country 2 Rock.Give respect & U will get it back. Remember it is the fans that can make you or break you. Be Nice to your fans!", "followers_count": 242, "friends_count": 447, "statues_count": 2822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364796944384, "text": "give me three wishes, I wish, I wish, I wish, you would b*tch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359759434, "name": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "screen_name": "daniellexolovee", "lang": "en", "location": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀westeros", "create_at": date("2011-08-21"), "description": "null", "followers_count": 23692, "friends_count": 11126, "statues_count": 1270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127364964737025, "text": "THANK YOU RAVN FOR CASHING OUT MY PTO. ���� RICH FOR A MIN.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414394852, "name": "ALFALFA", "screen_name": "afaiosefa", "lang": "en", "location": "Anchorage Alaska", "create_at": date("2011-11-16"), "description": "I probably checked you in for a flight.", "followers_count": 145, "friends_count": 110, "statues_count": 7081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127365019242496, "text": "Wind 0.0 mph ---. Barometer 30.34 in, Rising slowly. Temperature 38.5 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-26T00:00:01.000Z"), "id": 703127365094772736, "text": "Wind 3.0 mph NW. Barometer 30.347 in, Rising. Temperature 33.2 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127365996556288, "text": "Someone is getting shot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176561260, "name": "Patty Mills", "screen_name": "mrplanman_", "lang": "en", "location": "Dreamville, TX. ", "create_at": date("2010-08-09"), "description": "We gon' be alright", "followers_count": 1038, "friends_count": 1616, "statues_count": 28852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127366764113920, "text": "i did some classic hank williams sr last night was my own personal hank williams sr classic country radio station i think he would love it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4817373982, "name": "JOHNNY BLAZZE", "screen_name": "cowboyjohnnyjg", "lang": "en", "location": "null", "create_at": date("2016-01-15"), "description": "Father of 2. Country boy. I have my own business. Italian & Irish. Animal lover/owner. Budding country singer/songwriter! Follow me for covers & original music!", "followers_count": 1632, "friends_count": 3906, "statues_count": 874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hallandale Beach, FL", "id": "253e07b9636d0061", "name": "Hallandale Beach", "place_type": "city", "bounding_box": rectangle("-80.207047,25.972852 -80.11721,25.997497") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1228452, "cityName": "Hallandale Beach" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127366873120769, "text": "Who was better ?!?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218669094, "name": "Evan", "screen_name": "EvanGonzalez_", "lang": "en", "location": "Fresno/Firebaugh", "create_at": date("2010-11-22"), "description": "Fresno State", "followers_count": 1266, "friends_count": 962, "statues_count": 25204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127366881554433, "text": "That's what I want ☝��️��, I want pizza ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 588610761, "name": "T. ☯", "screen_name": "__TrevorS", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2012-05-23"), "description": "null", "followers_count": 15204, "friends_count": 13376, "statues_count": 26220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367024140288, "text": "Temp: 36.0°F Wind:0.0mph Pressure: 30.375hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367103819778, "text": ".@marcorubio says Republicans are the party of diversity not the democrats - no seriously, he said that!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15745368 }}, "user": { "id": 22699560, "name": "Cindy Leinwand", "screen_name": "CindyLeinwand12", "lang": "en", "location": "Los Angeles", "create_at": date("2009-03-03"), "description": "Mom to Sweet Mini Long-Haired Doxies, Music/TV Prod Assistant, Luv to Talk Politics & Current Events.... @Cher Fan @HillaryClinton #ImWithHer @TheHRCSuperVols", "followers_count": 1731, "friends_count": 1850, "statues_count": 60085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367233851393, "text": "Wind 0.0 mph WNW. Barometer 30.208 in, Rising slowly. Temperature 33.9 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367430967296, "text": "More rain & wind yay ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357982869, "name": "Carlita", "screen_name": "simplycarli", "lang": "en", "location": "New York, NY", "create_at": date("2011-08-18"), "description": "Skills: Taking Selfies & Being Sassy AF #youcantsitwithus | IG: nyc.carla", "followers_count": 295, "friends_count": 197, "statues_count": 5802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367447777280, "text": "@TudoSobreQuase @nomnomnami ENOUGH WITH THESE BOT TWEETS OMG", "in_reply_to_status": 703127178511122432, "in_reply_to_user": 3438425337, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3438425337, 185752453 }}, "user": { "id": 71825739, "name": "Nick", "screen_name": "HaloMillennium", "lang": "en", "location": "New Jersey", "create_at": date("2009-09-05"), "description": "#XboxLive #Steam: HaloMillennium | Gaming YouTuber (500 subs) | Tumblr: http://halomillennium.tumblr.com | I'm a hug nerd", "followers_count": 207, "friends_count": 419, "statues_count": 53050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Browns Mills, NJ", "id": "d6a0035751296dd8", "name": "Browns Mills", "place_type": "city", "bounding_box": rectangle("-74.603325,39.948198 -74.520915,39.990313") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3408455, "cityName": "Browns Mills" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367779164160, "text": "�� https://t.co/zbeWthdBHf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 164054256, "name": "SyDell", "screen_name": "Isinglikefuture", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-07-07"), "description": "⚔When He-Man said he had the power he was referring to his white male privilege⚔ LOL", "followers_count": 2200, "friends_count": 1571, "statues_count": 101133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367804284928, "text": "I'm officially 16!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2354026855, "name": "Rhi", "screen_name": "rhiannonhubach", "lang": "en", "location": "phs sophomore", "create_at": date("2014-02-20"), "description": "PHS '18 || Chase❤️", "followers_count": 268, "friends_count": 479, "statues_count": 4313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waller, WA", "id": "58bbeb0f59fec425", "name": "Waller", "place_type": "city", "bounding_box": rectangle("-122.397234,47.18062 -122.321424,47.236512") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5375905, "cityName": "Waller" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127367938510850, "text": "up every night. late night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2505072566, "name": "run it up randy.", "screen_name": "emptygyms", "lang": "en", "location": "switzerland", "create_at": date("2014-05-18"), "description": "ig & sc: kodalace", "followers_count": 643, "friends_count": 593, "statues_count": 4160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127368060174336, "text": "Wind 4.1 mph NW. Barometer 29.490 in, Rising Rapidly. Temperature 21.9 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127368328544261, "text": "The bartender at The O told me I intimidated him �� idk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248005980, "name": "Dem", "screen_name": "demitsengas", "lang": "en", "location": "null", "create_at": date("2011-02-05"), "description": "Tburg ➡️ KSU⚡• Nursing •️ Everything happens for a reason •", "followers_count": 677, "friends_count": 701, "statues_count": 30251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127369314217984, "text": "Stay fly ✨������������ #FLYGRLSLA #lovemade #lovemademedoit #LAnights #DTLA #womensupportingwomen… https://t.co/LCSLszMhXr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2478027,34.0469818"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FLYGRLSLA", "lovemade", "lovemademedoit", "LAnights", "DTLA", "womensupportingwomen" }}, "user": { "id": 1311295693, "name": "MissMegan_B", "screen_name": "MissMegan_B", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-03-28"), "description": "Los Angeles is my city. Music is my boyfriend. Girl, you got style.", "followers_count": 121, "friends_count": 332, "statues_count": 2937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:02.000Z"), "id": 703127369343569923, "text": "I swear across my heart , I ain't gotta lie to not one soul ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328802740, "name": "K I N G ☔️.〽️.", "screen_name": "MillerMade", "lang": "en", "location": "UTSA", "create_at": date("2011-07-03"), "description": "#LADPromo / ♊️ / Mechanical Engineer .... RIP Coach Green", "followers_count": 1297, "friends_count": 1049, "statues_count": 36808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127369448435717, "text": "Pandemonium https://t.co/YFGcYudS0I #pandemonium #parcdattraction #sanantonio #texas #unitedstates https://t.co/QtwcmuHbht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.607269,29.595156"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pandemonium", "parcdattraction", "sanantonio", "texas", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 827, "friends_count": 1256, "statues_count": 3214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127369800749058, "text": "45.6F (Feels: 45.6F) - Humidity: 63% - Wind: 5.4mph N - Gust: 5.4mph - Pressure: 1040.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127369859485701, "text": "And then this happened @nickcarter @skulleeroz #AllAmericanTour #shakeit #backstreetboys #booty https://t.co/VZyxnapjnq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.37492,34.064585"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AllAmericanTour", "shakeit", "backstreetboys", "booty" }}, "user_mentions": {{ 24325547, 161211454 }}, "user": { "id": 21608812, "name": "Miranda", "screen_name": "Lildeepmind", "lang": "en", "location": "San Fernando Valley", "create_at": date("2009-02-22"), "description": "Social Worker/Children's therapist. UC Berkeley & USC alum. LAKERS fan. Traveling is my self-care", "followers_count": 198, "friends_count": 423, "statues_count": 1270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saban Theater", "id": "07d9f325f2881001", "name": "Saban Theater", "place_type": "poi", "bounding_box": rectangle("-118.3749201,34.0645849 -118.37492,34.064585") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127370224369664, "text": "For those of us slaves to time (well, Wine-o-Clock) ;-) 17 Truths Every Wine Lover Knows https://t.co/BjfM2dUXpx via @VinePair @AdeldMeyer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1217373696, 151805556 }}, "user": { "id": 18854728, "name": "Jeff Dagley", "screen_name": "jdags", "lang": "en", "location": "Washington, DC", "create_at": date("2009-01-10"), "description": "Crafting #highered comm/SM content: @gwcareercenter. Passions: writing that knocks you out, history/preservation, wine, classic film, art, twisted humor", "followers_count": 1765, "friends_count": 3973, "statues_count": 2703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, VA", "id": "319ee7b36c9149da", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-77.172219,38.827378 -77.031779,38.934311") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51013, "countyName": "Arlington", "cityID": 5103000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127370295685126, "text": "ooh la la ive fallen in love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47695730, "name": "✩ sarah cera ✩", "screen_name": "zefrron", "lang": "en", "location": "austin tx ", "create_at": date("2009-06-16"), "description": "aquarius | boring | s.e.m. ♡", "followers_count": 270, "friends_count": 417, "statues_count": 19717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127370320875520, "text": "Fuller House is about to be out on Netflix. IM NOT GOING TO BEC UNTIL I WATCH ALL OF IT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323413132, "name": "//eryn rose//43", "screen_name": "BYEeryn5SOS", "lang": "en", "location": "front row//center stage", "create_at": date("2011-06-24"), "description": "did you ask for extra sassy? because you've came to the right fucking place, my friend.", "followers_count": 1107, "friends_count": 243, "statues_count": 47074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bear Creek, FL", "id": "00f547f36abae34d", "name": "Bear Creek", "place_type": "city", "bounding_box": rectangle("-82.732594,27.74548 -82.724244,27.763054") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1204735, "cityName": "Bear Creek" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127370526371840, "text": "Highkey wanna be in love again that shit was clutch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384046916, "name": "MillionDollarDreams⛽", "screen_name": "____vintvgee", "lang": "en", "location": "♡ Memphis✈Chicago ♡ ", "create_at": date("2011-10-02"), "description": "IG _milliondollardreams_ \n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nSC jackkiee0", "followers_count": 3297, "friends_count": 810, "statues_count": 88033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downers Grove, IL", "id": "6af99a29bfae42a2", "name": "Downers Grove", "place_type": "city", "bounding_box": rectangle("-88.096689,41.744098 -87.983315,41.83907") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1720591, "cityName": "Downers Grove" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127371314962433, "text": "@IMDb an EDM database? I love your guys' website and I think you guys have the right tools, tech, and know how to make this possible.", "in_reply_to_status": 703127180121747457, "in_reply_to_user": 3233256156, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17602896 }}, "user": { "id": 3233256156, "name": "MIKEOL", "screen_name": "DJMIKEOLMUSIC", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-06-02"), "description": "EDM ARTIST/UPCOMING DJ, AUTHOR, CG ARTIST, HOPEFUL FUTURE VIDEO GAME MAKER, AND ILLUMINATI SUPPORTER.", "followers_count": 1267, "friends_count": 3685, "statues_count": 17438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127372480925696, "text": "Don't trust a mf who let the microwave go to 0:00 after midnight https://t.co/86GI1NtuXa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331489905, "name": "Cam and 99 others", "screen_name": "Stuss_swank", "lang": "en", "location": "DTX 2 HTX", "create_at": date("2011-07-08"), "description": "R.I.P. Cameron Ellis July 16, 1997 - Today Dville Alumni", "followers_count": 1149, "friends_count": 701, "statues_count": 7787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127372904595456, "text": "Wind 5.4 mph NW. Barometer 29.59 in, Rising quickly. Temperature 30.0 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 111, "statues_count": 157807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127373235945472, "text": "Stankin ass hallways����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1483145155, "name": "The Real P.SiNG #FMS", "screen_name": "x_PGP", "lang": "en", "location": "Lincoln College", "create_at": date("2013-06-04"), "description": "Baby Official Music Video on YouTube Now! link in Bio! | http://youtu.be/j_2CINMLHs8 | Booking & Features contact: parissingleton11@gmail.com |", "followers_count": 778, "friends_count": 466, "statues_count": 8791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, IL", "id": "013b9d696e5665d0", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-89.416938,40.08307 -89.330625,40.168573") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17107, "countyName": "Logan", "cityID": 1743536, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-02-26T00:00:03.000Z"), "id": 703127373479194624, "text": "“Enlightenment is man’s emergence from his self-imposed immaturity.”\n—Immanuel Kant", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17797951, "name": "sean rose", "screen_name": "sean_a_rose", "lang": "en", "location": "san francisco, ca", "create_at": date("2008-12-01"), "description": "product at slack. \n\noptimist.", "followers_count": 5539, "friends_count": 185, "statues_count": 11719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayes Valley, San Francisco", "id": "1aab1e3216d9c231", "name": "Hayes Valley", "place_type": "neighborhood", "bounding_box": rectangle("-122.437582,37.769551 -122.421482,37.780719") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127373701472257, "text": "And I'm about to get paid in a few minutes��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174400828, "name": "Baecon⚽", "screen_name": "CerealSerg", "lang": "en", "location": "null", "create_at": date("2010-08-03"), "description": "IG: @barcafan_\nSC: b4rcafan", "followers_count": 609, "friends_count": 540, "statues_count": 89750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jurupa Valley, CA", "id": "015b658472edd3dc", "name": "Jurupa Valley", "place_type": "city", "bounding_box": rectangle("-117.551185,33.955517 -117.410795,34.034306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 637692, "cityName": "Jurupa Valley" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127374179643393, "text": "Very content with how my life is going rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615037090, "name": "Wyatt from WalMart Ω", "screen_name": "ZeniM0f0", "lang": "en", "location": "oregun", "create_at": date("2012-06-22"), "description": "http://www.livemixtapes.com/mixtapes/33866/billionaire-black-out-the-blue.html #MENINIST |541| snap me | wyattz69 |", "followers_count": 489, "friends_count": 335, "statues_count": 10005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lebanon, OR", "id": "a12170d40e438f19", "name": "Lebanon", "place_type": "city", "bounding_box": rectangle("-122.934374,44.508029 -122.876675,44.561163") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4141650, "cityName": "Lebanon" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127374234198016, "text": "She like that certain type of nigga and it's clear I'm not that guy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1517736145, "name": "Beto JawQ", "screen_name": "Jacquez023", "lang": "en", "location": "Furthest thing from perfect. H", "create_at": date("2013-06-14"), "description": "#7 #23⚽️Coahuila. Houston.", "followers_count": 281, "friends_count": 190, "statues_count": 20219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127374502563840, "text": "@IsmailiVjollca @valbona_ismaili @v23dbest what's cracking cuzzos vali nuk u nale sot", "in_reply_to_status": 703125152620347392, "in_reply_to_user": 2365018190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2365018190, 2371707903, 268862133 }}, "user": { "id": 1269086863, "name": "akaDonnieDoesIt", "screen_name": "LiridonZeciri", "lang": "en", "location": "Anchorage, AK", "create_at": date("2013-03-15"), "description": "Engaged ❤️ Albanian CHI-AK", "followers_count": 225, "friends_count": 144, "statues_count": 10492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375152680961, "text": "Damn Daniel! Back at it again with the Reptar 11s #DamnDaniel #Reptile #omg #whatarethose #clawsout https://t.co/FyJh80hFNZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DamnDaniel", "Reptile", "omg", "whatarethose", "clawsout" }}, "user": { "id": 1107568387, "name": "Jarren Jones", "screen_name": "jarrenjones94", "lang": "en", "location": "Snapchat: jarren_jones96 ", "create_at": date("2013-01-20"), "description": "Huntingdon College| Student Athlete | GOD 1st| Family 2nd| BRUH MAN FROM DA FIF FLO", "followers_count": 341, "friends_count": 337, "statues_count": 2670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Hill, FL", "id": "68042baafa8b7e0a", "name": "Spring Hill", "place_type": "city", "bounding_box": rectangle("-82.639883,28.433671 -82.428825,28.534024") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12053, "countyName": "Hernando", "cityID": 1268350, "cityName": "Spring Hill" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375224016897, "text": "Wind 2.0 mph NNE. Barometer 30.333 in, Falling slowly. Temperature 30.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375307939840, "text": "explain @ArmaniDeanX https://t.co/g68GINwcOv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1698458575 }}, "user": { "id": 2323385372, "name": "Justin Octave", "screen_name": "Jdynasty__", "lang": "en", "location": "Liemert Park ", "create_at": date("2014-02-01"), "description": "money bag money bag money bag", "followers_count": 179, "friends_count": 218, "statues_count": 2955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375374999553, "text": "because the colors are just... fabulous;\n\n\"Half eye\"\n\n#red #yellow #blue #aden #adenfilter… https://t.co/Zj48MEZuoh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "red", "yellow", "blue", "aden", "adenfilter" }}, "user": { "id": 419715864, "name": "Katie Booth", "screen_name": "whatthehellKB", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-11-23"), "description": "actress | likes politics & meltdowns | defender of the crazies | animal avenger (stolen from @rickygervais) | insta me at: https://www.instagram.com/boothk2/", "followers_count": 3551, "friends_count": 1725, "statues_count": 2509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375953862656, "text": "Awwwww wish I could see him too�� https://t.co/1LuXfx8JuW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1839788586, "name": "desarae", "screen_name": "dutchessdes_", "lang": "en", "location": "null", "create_at": date("2013-09-08"), "description": "#CAKETEAM", "followers_count": 1628, "friends_count": 1395, "statues_count": 81340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127375962189825, "text": "ғav ғor a твн @ 12:00 @ nιgнт caυѕe yoυr gιrl can'т ever ѕeeм тo ѕleep aт a decenт тιмe ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cs", "is_retweet": false, "user": { "id": 2940212556, "name": "ѕaвrιna❁", "screen_name": "sabrinaaalee2", "lang": "en", "location": "living life", "create_at": date("2014-12-22"), "description": "ιт ιѕ wнaт ιт ιѕ вaвy| R.I.P- c.a.l.❤️| sc: s_castro", "followers_count": 565, "friends_count": 1121, "statues_count": 8779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127376452956160, "text": "I love my girls @ChrissiRose23 & @Rosalina_TW so much ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 963988939, 713120934 }}, "user": { "id": 319499865, "name": "Jaylexis", "screen_name": "LoseMyMind_4_TW", "lang": "en", "location": "That ice cube down Jay's pants", "create_at": date("2011-06-17"), "description": "Love is the only thing in this world that does not hurt ♥ @TheJayC_V ☯ 10/25 • @TrevorandAustin x8 • STL for @HoosekiLeckey & @PhaseVMusic • Viber Royalty | x18", "followers_count": 14481, "friends_count": 11657, "statues_count": 88759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127376637460481, "text": "Fr ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.21993025,38.75927907"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2810929805, "name": "tayyy", "screen_name": "Taythatgal", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "With naya , and jerm❤", "followers_count": 410, "friends_count": 453, "statues_count": 14382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple Hills, MD", "id": "50ab1362909ec83d", "name": "Temple Hills", "place_type": "city", "bounding_box": rectangle("-76.973441,38.802185 -76.93105,38.818865") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2477100, "cityName": "Temple Hills" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127377065291777, "text": "Being single is the worst shit ever. Dating is the worst lol like y'all have no idea how many dates I've gone on & there's no 2nd date", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325834262, "name": "Jess", "screen_name": "MuffyDoodle", "lang": "en", "location": "Pasadena,TX|San Marcos,TX", "create_at": date("2013-04-03"), "description": "I'm like a cat, always ready to pounce.", "followers_count": 355, "friends_count": 293, "statues_count": 34562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127377069494272, "text": "Happy birthday to my person my number 1 I'm blessed to have you in my life! Let's fuck shit up on Saturday! Big 21���� https://t.co/Ce4AuCtcsw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3953045414, "name": "CT", "screen_name": "CeeTeeBabbeey", "lang": "en", "location": "null", "create_at": date("2015-10-19"), "description": "BBC | 19 | if you had a twin, I'll still choose you", "followers_count": 172, "friends_count": 168, "statues_count": 3192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montclair, CA", "id": "3153d677b795e293", "name": "Montclair", "place_type": "city", "bounding_box": rectangle("-117.721072,34.046668 -117.680734,34.096817") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 648788, "cityName": "Montclair" } }
+{ "create_at": datetime("2016-02-26T00:00:04.000Z"), "id": 703127377551843329, "text": "I feel like I won the #oscars with the ultimate #VFSC gifts #bloggerbabes @thebloggerbabes thx… https://t.co/eMGPi83mm1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3881607,34.0267601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oscars", "VFSC", "bloggerbabes" }}, "user_mentions": {{ 1692830768 }}, "user": { "id": 15685664, "name": "Cherry the Geek", "screen_name": "cherry_LA", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-07-31"), "description": "Geek Chick talk about tech, cars, business, travel, entertainment & lifestyle http://www.youtube.com/user/cherrylosangeles", "followers_count": 2045, "friends_count": 977, "statues_count": 76218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127378466242560, "text": "@EyeZick https://t.co/oqDQKTtS45", "in_reply_to_status": 703126394155302912, "in_reply_to_user": 38095772, "favorite_count": 0, "coordinate": point("-86.34075314,41.94110697"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 38095772 }}, "user": { "id": 34618846, "name": "new Pablo who dis", "screen_name": "SaigeSmoove", "lang": "en", "location": "NYC", "create_at": date("2009-04-23"), "description": "im woke and im savage #Futurehive #Yeezyhive [SC: t00faded]", "followers_count": 649, "friends_count": 566, "statues_count": 22565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berrien Springs, MI", "id": "003fcca428581309", "name": "Berrien Springs", "place_type": "city", "bounding_box": rectangle("-86.36849,41.89966 -86.305321,41.976116") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26021, "countyName": "Berrien", "cityID": 2607860, "cityName": "Berrien Springs" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127379472855040, "text": "scariest moment of my life ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284257444, "name": "maranda", "screen_name": "marandajoycee", "lang": "en", "location": "selvaggio e meraviglioso, WV", "create_at": date("2011-04-18"), "description": "☺️ wvu '18 ΑΞΔ", "followers_count": 675, "friends_count": 487, "statues_count": 14632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127379657371649, "text": "@_ceray_ lol me rn", "in_reply_to_status": 702307155362320384, "in_reply_to_user": 424573815, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 424573815 }}, "user": { "id": 2351311093, "name": "PrincessTiana", "screen_name": "dvliyvh", "lang": "en", "location": "CA to VA ", "create_at": date("2014-02-18"), "description": "• QTX ⚓️ HU19 • GatesMillenniumScholar2015 •", "followers_count": 866, "friends_count": 854, "statues_count": 8332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127380034912256, "text": "Lol I'm 18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2371613245, "name": "denali", "screen_name": "flannelgab", "lang": "en", "location": "Neo Cali", "create_at": date("2014-03-04"), "description": "mexicana ❁ I'm crazy, but it's always justifiably crazy ❉ viva la raza", "followers_count": 185, "friends_count": 210, "statues_count": 6809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127380198498304, "text": "daddy ���� https://t.co/WB6j9Enoim", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 499423588, "name": "Ᏼ.", "screen_name": "BilqisOlympia", "lang": "en", "location": "null", "create_at": date("2012-02-21"), "description": "re$t easy dad", "followers_count": 4835, "friends_count": 3337, "statues_count": 58307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-02-26T00:00:05.000Z"), "id": 703127380743696385, "text": "Now we just wait for Shinjis lol https://t.co/EVazgkloth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1727319666, "name": "Quintin K", "screen_name": "_QKing_", "lang": "en", "location": "Des Moines, IA", "create_at": date("2013-09-03"), "description": "Oak Park IL ➡ Des Moines IA \nGrand View University \n\nWasting my youth on this app but it's whatever \n\n#BleachHive", "followers_count": 559, "friends_count": 843, "statues_count": 9954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127382320742402, "text": "temperature down 26°F -> 22°F\nhumidity down 95% -> 86%\nwind 10mph -> 14mph\npressure 29.71in rising\nvisibility 2mi -> 4mi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.14739,43.04999"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161728160, "name": "Syracuse Weather", "screen_name": "SyracuseNY", "lang": "en", "location": "Syracuse, NY", "create_at": date("2010-07-01"), "description": "Weather updates, forecast, warnings and information for Syracuse, NY. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 85, "friends_count": 2, "statues_count": 20860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127382656311296, "text": "You made me a GIF! Amazeballs! https://t.co/QRQiB1hyL4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700405107263803392, "name": "Patrick Sullivan", "screen_name": "PSullivanMusic", "lang": "en", "location": "Los Angeles ", "create_at": date("2016-02-18"), "description": "soulpopmachine | Cali transplant | bka @Klokwize with the rhymes", "followers_count": 1080, "friends_count": 33, "statues_count": 24 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127382681481216, "text": "When he can't understand the hints you're throwing at him. ��������<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3028298028, "name": "Chantay West", "screen_name": "tvywest", "lang": "en", "location": "#FleekSqwad", "create_at": date("2015-02-18"), "description": "Unfriendly Black Hottie // sfsu", "followers_count": 186, "friends_count": 185, "statues_count": 5208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daly City, CA", "id": "6a71821001635bbd", "name": "Daly City", "place_type": "city", "bounding_box": rectangle("-122.500164,37.649122 -122.405233,37.708437") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 617918, "cityName": "Daly City" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127382937329665, "text": "Didn't get @fukevin's beanie though", "in_reply_to_status": 703127042796048384, "in_reply_to_user": 16440706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24341293 }}, "user": { "id": 16440706, "name": "Mikayla", "screen_name": "mikaylax", "lang": "en", "location": "Orlando, FL", "create_at": date("2008-09-24"), "description": "I do merch & makeup / @licknord's social media handler", "followers_count": 2767, "friends_count": 547, "statues_count": 48894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127383037997056, "text": "About to stay up to watch #FullerHouse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FullerHouse" }}, "user": { "id": 44024913, "name": "Melissa M. Galindo", "screen_name": "meli20_01", "lang": "en", "location": "Salem, OR", "create_at": date("2009-06-01"), "description": "Oregon Born.. California Raised", "followers_count": 32, "friends_count": 92, "statues_count": 763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127383046430720, "text": "4 months!! ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2153663768, "name": "Aaron Ward", "screen_name": "ayee_cee14", "lang": "en", "location": "Montgomery, AL", "create_at": date("2013-10-24"), "description": "LBHS Graduate Alumni c/o '14 | SCC| GK| Music Major| Mary Dorantes (: ♥| Alabama Life", "followers_count": 1108, "friends_count": 2133, "statues_count": 15913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127383646146560, "text": "@jennyyloup I can't wait to get home and see my kitty babies tomorrow ����������", "in_reply_to_status": 685006838169874432, "in_reply_to_user": 28942928, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28942928 }}, "user": { "id": 28942928, "name": "jennifer ˚✵✺✧*✷", "screen_name": "jennyyloup", "lang": "en", "location": "michigan", "create_at": date("2009-04-04"), "description": "cancer sun / scorpio moon. enfp. lover. kitty mom. I put the hot in psychotic.", "followers_count": 710, "friends_count": 308, "statues_count": 28997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodhaven, MI", "id": "00487e63709614f2", "name": "Woodhaven", "place_type": "city", "bounding_box": rectangle("-83.306006,42.109216 -83.215583,42.18442") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2688380, "cityName": "Woodhaven" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127384887656449, "text": "Interested in a #Finance #job near #JerseyCity, New Jersey? This could be a great fit: https://t.co/ljp9yRCpIB #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0776417,40.7281575"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Finance", "job", "JerseyCity", "Hiring" }}, "user": { "id": 3836985865, "name": "BOC Staffing", "screen_name": "BOCstaffing", "lang": "en", "location": "NYC & Jersey City area", "create_at": date("2015-10-09"), "description": "BOC is a niche specialty provider, serving the Tri-State area's Buy-side/Sell-side #Financial Services community’s #staffing requirements.#finance #job #NYC #NJ", "followers_count": 34, "friends_count": 54, "statues_count": 62 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-02-26T00:00:06.000Z"), "id": 703127385533591552, "text": "Don't let negative people fuck up ur blessings, let go and let God bless you as he blessed me. I… https://t.co/WUanEY9jpH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.8587,33.5992"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3366747226, "name": "Mr. Andrews", "screen_name": "MrAndrews5", "lang": "en", "location": "〽️ars", "create_at": date("2015-07-08"), "description": "My Heart is trapped in #ZordoraWorld", "followers_count": 49, "friends_count": 119, "statues_count": 902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, GA", "id": "005f284c5f674191", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-83.891081,33.569455 -83.813866,33.637881") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13217, "countyName": "Newton", "cityID": 1320064, "cityName": "Covington" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127388100567040, "text": "Since it'll probs be my last time being in town for it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356445386, "name": "Linz♊️", "screen_name": "Linzwearsprada", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2011-08-16"), "description": "FAU 19' | Fly High Hope", "followers_count": 340, "friends_count": 447, "statues_count": 11605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127388683567108, "text": "Houston Hospitals Get the Jump on Rapid Tests for Zika Virus https://t.co/9Tgu6esEyP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.08709717,29.51160049"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2866011003, "name": "League City, TX News", "screen_name": "BLifeLeagueCity", "lang": "en", "location": "League City, Texas", "create_at": date("2014-11-07"), "description": "League City BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/Ew7CTPM7Zf.", "followers_count": 137, "friends_count": 122, "statues_count": 8484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "League City, TX", "id": "cf3b45e29c2c319b", "name": "League City", "place_type": "city", "bounding_box": rectangle("-95.178987,29.438994 -94.995071,29.55532") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841980, "cityName": "League City" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127388738048005, "text": "Gym in the AM. Goodnight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155366234, "name": "Alexa Andrade✌", "screen_name": "ayo_lexaa", "lang": "en", "location": "El Paso, TX", "create_at": date("2010-06-13"), "description": "A little bit of devil in her angel eyes.", "followers_count": 1874, "friends_count": 769, "statues_count": 46306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127388859707392, "text": "@TownBoyLoweSoo ��", "in_reply_to_status": 703127261650681856, "in_reply_to_user": 2951455235, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2951455235 }}, "user": { "id": 3303478807, "name": "TooMuchh", "screen_name": "Destoooooooooo", "lang": "en", "location": "null", "create_at": date("2015-08-01"), "description": "null", "followers_count": 376, "friends_count": 372, "statues_count": 5344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127388910022656, "text": "Wind 3.0 mph NW. Barometer 30.121 in, Rising. Temperature 26.4 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 382, "friends_count": 291, "statues_count": 8562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127389530824704, "text": "Wow I love life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1122561804, "name": "holls", "screen_name": "HollyWiemann", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "null", "followers_count": 1333, "friends_count": 357, "statues_count": 14957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-02-26T00:00:07.000Z"), "id": 703127389711126529, "text": "Back on do not disturb.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393816368, "name": "Darv.", "screen_name": "imJustDarv", "lang": "en", "location": "oh tuskegee. ", "create_at": date("2011-10-18"), "description": "ion think I'll ever be shit foreal.", "followers_count": 1428, "friends_count": 612, "statues_count": 114762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuskegee, AL", "id": "28df0fa3673b5b49", "name": "Tuskegee", "place_type": "city", "bounding_box": rectangle("-85.747927,32.386278 -85.663058,32.460448") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1087, "countyName": "Macon", "cityID": 177304, "cityName": "Tuskegee" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489744789962752, "text": "#UltimateCollectionTourQuesto sarà un altro paio di scarpe che Anastacia indosserà per il suo… https://t.co/WO7BnnD3UQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.35544,34.0539"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "UltimateCollectionTourQuesto" }}, "user": { "id": 192918396, "name": "Anastacia Ankh&Wings", "screen_name": "AnkhandWings", "lang": "it", "location": "Italy", "create_at": date("2010-09-20"), "description": "Anastacia Unofficial Italian Fanclub, recognized by Anastacia herself, born in Rome in 2009! Our passion is the little white lady with big black voice!", "followers_count": 805, "friends_count": 734, "statues_count": 4493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489744911540224, "text": "wow, there’s an official steam controller configuration for blops3 by treyarch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27403903, "name": "TimeDoctor", "screen_name": "TimeDoctor", "lang": "en", "location": "Assembled in Philadelphia, PA. Living in Oakland, CA", "create_at": date("2009-03-29"), "description": "Husband of @MrsTimeDoctor \nand also part of these fine things: http://timedoctor.org @ioquake3", "followers_count": 2030, "friends_count": 1326, "statues_count": 16659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489745377079296, "text": "Want to work in #Harrisburg, PA? View our latest opening: https://t.co/nHFreMpXEh #Nursing #RN #Nursing #Job #Jobs https://t.co/TbqNKcNqjf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.8844179,40.2737002"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Harrisburg", "Nursing", "RN", "Nursing", "Job", "Jobs" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 163, "friends_count": 99, "statues_count": 2078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pennsylvania, USA", "id": "dd9c503d6c35364b", "name": "Pennsylvania", "place_type": "admin", "bounding_box": rectangle("-80.519851,39.719801 -74.689517,42.516072") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42043, "countyName": "Dauphin", "cityID": 4232800, "cityName": "Harrisburg" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489745498681344, "text": "@aletihadae @Forsan_UAE رحمة الله على شهداءالامارات", "in_reply_to_status": 703489014893043712, "in_reply_to_user": 119756265, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 119756265, 1321331874 }}, "user": { "id": 4548521292, "name": "Nader", "screen_name": "NaderghosnPsp", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "null", "followers_count": 13, "friends_count": 69, "statues_count": 72 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489745846796288, "text": "Woof. I have a lot of thoughts. \nI gotta take a break from seeing theatre, I think.\nToo much disappointment.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99033899, "name": "Rachel Delmar", "screen_name": "RachelDelmar", "lang": "en", "location": "Seattle", "create_at": date("2009-12-23"), "description": "Theatre maker & marketer in #SEAthtr. @PlayInProg. Poet. Yogi in Training. Adventure is out there. Loves: @Seahawks, Running, Cocktails, & #HouseDelmarston.", "followers_count": 1701, "friends_count": 953, "statues_count": 27968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489747612602368, "text": "We on the come up https://t.co/PIRQus81bV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2543566071, "name": "Kaylah J", "screen_name": "gaugedearsh4wty", "lang": "en", "location": "MO/TX", "create_at": date("2014-05-12"), "description": "I draw a lil", "followers_count": 605, "friends_count": 296, "statues_count": 41409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489747839295488, "text": "@greencheech movies. Not cool \n https://t.co/CzcgbxKA4h", "in_reply_to_status": -1, "in_reply_to_user": 36702377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36702377 }}, "user": { "id": 635604581, "name": "nope", "screen_name": "cornugget35", "lang": "en", "location": "null", "create_at": date("2012-07-14"), "description": "null", "followers_count": 149, "friends_count": 318, "statues_count": 1106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489747864322048, "text": "When you realize there are those with common sense. #NeverTrump https://t.co/ajk6atyI6N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NeverTrump" }}, "user": { "id": 1976820770, "name": "William Flores", "screen_name": "William_EFlores", "lang": "en", "location": "Bay Area ", "create_at": date("2013-10-20"), "description": "Scientist at Stanford...just doing my part in this world one day at a time. We can make a difference. Cali", "followers_count": 7962, "friends_count": 8203, "statues_count": 11350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489748116103168, "text": "@br_andy13 brilliant", "in_reply_to_status": 703488959582633984, "in_reply_to_user": 787470342, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 787470342 }}, "user": { "id": 324551710, "name": "Taylor Short", "screen_name": "taylor_short22", "lang": "en", "location": "Apple Valley-Fargo", "create_at": date("2011-06-26"), "description": "#SkolVikes", "followers_count": 1525, "friends_count": 334, "statues_count": 8139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fargo, ND", "id": "5c8fea63bc3faa5b", "name": "Fargo", "place_type": "city", "bounding_box": rectangle("-96.903853,46.787881 -96.756332,46.941884") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38017, "countyName": "Cass", "cityID": 3825700, "cityName": "Fargo" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489748254351360, "text": "I just wanna cuddle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370418936, "name": "heffley", "screen_name": "HeighleyDoria", "lang": "en", "location": "V-town", "create_at": date("2011-09-08"), "description": "hedonist", "followers_count": 354, "friends_count": 320, "statues_count": 5496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489748518604800, "text": "Baby that belongs to me. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418304469, "name": "Nita", "screen_name": "Nitaababe", "lang": "en", "location": "South Sac.", "create_at": date("2011-11-21"), "description": "Coco , Team Breezy, & Dem Dallas Cowboys Baby❤", "followers_count": 618, "friends_count": 551, "statues_count": 18664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-27T00:00:00.000Z"), "id": 703489748833144832, "text": "I wish it was that easy to just let go of someone who means so much to you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396081988, "name": "Cinthia", "screen_name": "Riosc22", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-10-22"), "description": "Thats my person & that will always be my person", "followers_count": 604, "friends_count": 619, "statues_count": 21753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489749160325120, "text": "Three hour life talks with Uncle Pat at Village Inn are my favorite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 254863552, "name": "Anthony Kennedy", "screen_name": "Aka_AK20", "lang": "en", "location": "El Paso, TX", "create_at": date("2011-02-19"), "description": "null", "followers_count": 1092, "friends_count": 799, "statues_count": 18686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Cruces, NM", "id": "f515486276aa6192", "name": "Las Cruces", "place_type": "city", "bounding_box": rectangle("-106.843427,32.238361 -106.63145,32.446238") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3539380, "cityName": "Las Cruces" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489749370015744, "text": "@MashupMusics @martinlili44", "in_reply_to_status": 698514931008741381, "in_reply_to_user": 2449829138, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2449829138, 1733947188 }}, "user": { "id": 701217248799649792, "name": "adon", "screen_name": "iamTIDES", "lang": "en", "location": "Northern Bay", "create_at": date("2016-02-20"), "description": "back on this shit for the 3rd time", "followers_count": 11, "friends_count": 19, "statues_count": 37 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489749390995456, "text": "shout out to fake ids", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275868597, "name": "steezus//steele", "screen_name": "OGsteele", "lang": "en", "location": "washington/minnesota", "create_at": date("2011-04-01"), "description": "i take park laps and hikes and dabs university of washington.", "followers_count": 266, "friends_count": 182, "statues_count": 5700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burien, WA", "id": "531b8f14eaf6c48e", "name": "Burien", "place_type": "city", "bounding_box": rectangle("-122.382962,47.430814 -122.311608,47.499304") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5308850, "cityName": "Burien" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489750326497280, "text": "I need to explore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304392938, "name": "Jacob Blain", "screen_name": "theohiohippie", "lang": "en", "location": "☯HI☮", "create_at": date("2011-05-24"), "description": "20 Good Vibes Child of the Universe .Tuffy ♥♥♥", "followers_count": 867, "friends_count": 206, "statues_count": 7313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489750339084288, "text": "02/27@03:00 - Temp 28.7F, WC 28.7F. Wind 1.3mph W, Gust 2.0mph. Bar 30.182in, Rising slowly. Rain 0.00in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489750569619457, "text": "@GloriaTrevi gloriaaaa falto yo.. Buenas noches queridísima.. Tkm un abrazote!! ��������������❤️", "in_reply_to_status": -1, "in_reply_to_user": 86119466, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 86119466 }}, "user": { "id": 2175104438, "name": "Monserrat moreno", "screen_name": "monse_more", "lang": "es", "location": "Palm Springs, CA", "create_at": date("2013-11-04"), "description": "#Trevilander❤️AMORBRAVIO#deportista#sialostoros.San MIGUEL EL ALTO,JALISCO#GOD1st#Winelover#AmoAGLORIATREVI instagram:MONSEE_MORENO @GloriaTrevi❤️ @silvnavarro", "followers_count": 152, "friends_count": 562, "statues_count": 870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489751190343680, "text": "You said I was the most exotic flower\nHolding me tight in our final hour", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2422687993, "name": "Katherine Todd Rose", "screen_name": "Katherineadam0", "lang": "en", "location": "Adam Lambert", "create_at": date("2014-04-01"), "description": "I'm Singer/Songwriter/Dancer/. ❤❤❤❤ ⭐ ♦♥ ♪ Glambert Girls and I love to party with anyone. I love Adam Lambert more then anything. ❤ ⭐ ♦♥ ♪", "followers_count": 389, "friends_count": 1029, "statues_count": 2336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New London, CT", "id": "6e9c054da607b539", "name": "New London", "place_type": "city", "bounding_box": rectangle("-72.12852,41.304762 -72.087716,41.385244") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 952280, "cityName": "New London" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489751467208704, "text": "Running around naked in my apartment by myself it feels alright", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502333540, "name": "gloria susana", "screen_name": "G_ThangBaby", "lang": "en", "location": "Kansas, USA", "create_at": date("2012-02-24"), "description": "bæsic selfless promiscuous. i am not in danger, i am the danger. we are all searching for someone whose demons play well with ours ♡ Snapchat&IG: @g_thangbaby", "followers_count": 529, "friends_count": 493, "statues_count": 26092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, KS", "id": "4ccacb29ffb3b3df", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-94.724428,37.367357 -94.668019,37.444178") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20037, "countyName": "Crawford", "cityID": 2056025, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489751614029824, "text": "I drank a lot of green tea so I can't sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2233636489, "name": "chanelyyy", "screen_name": "chaneeelll", "lang": "en", "location": "LA / لبنانية Lebanese", "create_at": date("2013-12-06"), "description": "snapchat: chaneeelll98 • @justinbieber since 2009", "followers_count": 952, "friends_count": 346, "statues_count": 28079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489751857258497, "text": "@MooseJuan wow!", "in_reply_to_status": 703489452883087360, "in_reply_to_user": 557294904, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 557294904 }}, "user": { "id": 1511306342, "name": "Kyle Evangelista", "screen_name": "kyyyllleee310", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "@andreagancenia ♡", "followers_count": 111, "friends_count": 110, "statues_count": 724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489752431853568, "text": "Cussed my boyfriend out now I can't sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 906689593, "name": "Alexus Strickland⛽", "screen_name": "Funsized_Lexy", "lang": "en", "location": "Dade City ' Fl", "create_at": date("2012-10-26"), "description": "R.I.P \nTuvarrion Sirmons\n&\nJordan Abigail Ivie", "followers_count": 1790, "friends_count": 1493, "statues_count": 4154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dade City, FL", "id": "b22602b152b8f982", "name": "Dade City", "place_type": "city", "bounding_box": rectangle("-82.239194,28.324759 -82.166298,28.408986") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1216125, "cityName": "Dade City" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489752457043968, "text": "I hope the Kylie lip kits don't sell out hella quick this time around. I'm trynna buy like all of them ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 826064892, "name": "Mami", "screen_name": "melinuuuh", "lang": "en", "location": "null", "create_at": date("2012-09-15"), "description": "Take a breath & let the rest come easy.", "followers_count": 227, "friends_count": 299, "statues_count": 8123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "American Canyon, CA", "id": "f28195f8b183abcd", "name": "American Canyon", "place_type": "city", "bounding_box": rectangle("-122.283944,38.154977 -122.229741,38.204868") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 601640, "cityName": "American Canyon" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489752511619072, "text": "Bahahahahahahaha hes a ducks fan. (I go on tinder when im bored) #dontjudgeme #ImSooooSingle https://t.co/X1se1d8JG9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dontjudgeme", "ImSooooSingle" }}, "user": { "id": 190124038, "name": "Cece", "screen_name": "cecetapatio", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-09-12"), "description": "20. Animal lover. Disney enthusiast. LA Kings!!!\nMay the force be with you, ya dig.", "followers_count": 549, "friends_count": 304, "statues_count": 34699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-27T00:00:01.000Z"), "id": 703489752566108160, "text": "Like what do I have to do to look my age?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1638569828, "name": "You Already Know", "screen_name": "TaliaKaljian", "lang": "en", "location": "out of your league", "create_at": date("2013-08-01"), "description": "Somewhere between a sweetheart and a complete asshole.", "followers_count": 91, "friends_count": 95, "statues_count": 2330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489753178578945, "text": "Wind 2.0 mph SE. Barometer 30.180 in, Falling. Temperature 32.7 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489753455443969, "text": "Wind 0.0 mph ---. Barometer 30.31 in, Rising slowly. Temperature 37.9 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489753581117442, "text": "@ebbtideapp Tide in Hudson, New York 02/27/2016\nHigh 5:58am 3.9\n Low 12:17pm 0.3\nHigh 6:10pm 3.6\n Low 12:29am 0.3\nHigh 6:37am 3.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.8,42.25"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 36, "friends_count": 1, "statues_count": 11221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, NY", "id": "59eaa92f1e96c615", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-73.807203,42.235321 -73.763695,42.262946") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36021, "countyName": "Columbia", "cityID": 3635969, "cityName": "Hudson" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489753983918080, "text": "Temp: 34.3°F Wind:0.0mph Pressure: 30.338hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489754197659649, "text": "Check out my selfie with Regina! https://t.co/NkshpCLGpC https://t.co/qX70xv7Qbb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152843313, "name": "Tracy Boyle", "screen_name": "Oopsey0022", "lang": "en", "location": "null", "create_at": date("2010-06-06"), "description": "null", "followers_count": 13, "friends_count": 75, "statues_count": 190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lee's Summit, MO", "id": "6d91b7c06b86638b", "name": "Lee's Summit", "place_type": "city", "bounding_box": rectangle("-94.45916,38.805502 -94.297303,39.026684") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2941348, "cityName": "Lee's Summit" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489754654846980, "text": "@madelynekarina I'm winning", "in_reply_to_status": 703489620529405952, "in_reply_to_user": 2264003780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2264003780 }}, "user": { "id": 3103881194, "name": "ClarenceIsaiahJohns", "screen_name": "King__Clarence", "lang": "en", "location": "West Covina, CA", "create_at": date("2015-03-22"), "description": "null", "followers_count": 285, "friends_count": 314, "statues_count": 9288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489754797465601, "text": "@traffichaze I'm gonna go drink some water now thanks", "in_reply_to_status": 703489395941208064, "in_reply_to_user": 45040094, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45040094 }}, "user": { "id": 100702708, "name": "Zachary Simpson", "screen_name": "ZacharyXCIII", "lang": "en", "location": "Sacramento", "create_at": date("2009-12-30"), "description": "Sacramento, CA. Liverpool FC. Hockey.", "followers_count": 660, "friends_count": 477, "statues_count": 29782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galt, CA", "id": "6f49c528bb0c1a33", "name": "Galt", "place_type": "city", "bounding_box": rectangle("-121.337371,38.236791 -121.264316,38.294229") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 628112, "cityName": "Galt" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489755086876672, "text": "Turn a 5 star hotel to a trap house.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232976323, "name": "JP3™", "screen_name": "jtown2123", "lang": "en", "location": "In My Own Head", "create_at": date("2011-01-01"), "description": "God blessing all the trap niggas. #EVOL #WATTBA MidAmerica Nazarene University", "followers_count": 1361, "friends_count": 624, "statues_count": 7162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lenexa, KS", "id": "010ca0c52ba7443f", "name": "Lenexa", "place_type": "city", "bounding_box": rectangle("-94.844203,38.915313 -94.704952,38.993369") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2039350, "cityName": "Lenexa" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489755414032386, "text": "#3AMWeMeetAgain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "3AMWeMeetAgain" }}, "user": { "id": 1075248840, "name": "LENNY: The Red Mamba", "screen_name": "redheadLEN", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-01-09"), "description": "IG: redheadLEN • @WorldFamousNYC • #HorrorWorld • #WorldBBQ • #WorldLove •#TheGreatestDayEver • #HoopsShowdown • #FlagFootballShowdown • #3AmWeMeetAgain", "followers_count": 1553, "friends_count": 1001, "statues_count": 59435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489755825229824, "text": "Wind 2.0 mph WNW. Barometer 30.232 in, Falling slowly. Temperature 34.7 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756559228928, "text": "\"Go for the six, back the eff off.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4720571297, "name": "Hailey Ross", "screen_name": "hailey_rozay", "lang": "en", "location": "null", "create_at": date("2016-01-05"), "description": "null", "followers_count": 40, "friends_count": 100, "statues_count": 446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksville, TN", "id": "52f4a98d03e4ea4f", "name": "Clarksville", "place_type": "city", "bounding_box": rectangle("-87.486546,36.4602 -87.157791,36.641767") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47125, "countyName": "Montgomery", "cityID": 4715160, "cityName": "Clarksville" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756584288256, "text": "42.1F (Feels: 42.1F) - Humidity: 99% - Wind: 1.6mph E - Gust: 1.6mph - Pressure: 1038.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756756193280, "text": "Dads locked up and this shit just kills , wanna Od so I grab a couple pills", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4418522599, "name": "Ϣ Σ Σ Z̤̮ Ꮍ", "screen_name": "LuissPerezzjr", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-12-01"), "description": "#ℒℐᎾℕℒℐℱℰ #ℂℋᎯЅℐℕᎶⅅℛℰᎯℳℤ", "followers_count": 214, "friends_count": 202, "statues_count": 3578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ennis, TX", "id": "054df5106ee9caf2", "name": "Ennis", "place_type": "city", "bounding_box": rectangle("-96.667447,32.289048 -96.571268,32.368525") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4824348, "cityName": "Ennis" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756768776192, "text": "WORK FROM FUCKING HOME SLAYS!!!!!!! Not up for discussion either y'all! Thanks for the bop @FifthHarmony", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 872374136 }}, "user": { "id": 85725078, "name": "#B9", "screen_name": "luvmeoniceBOMT", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-10-27"), "description": "You'll have to wait and see. Founder of the Britney Army Prayer Circle. Followed by my idol the legendary @britneyspears. Fan since '98 ✌️", "followers_count": 1038, "friends_count": 593, "statues_count": 19787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756798124032, "text": "Temp: 27.7°F - Dew Point: 21.5° - Wind: 6.6 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 29.98in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489756945059840, "text": "Wind 0.7 mph E. Barometer 30.10 in, Steady. Temperature 18.1 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 111, "statues_count": 157831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-27T00:00:02.000Z"), "id": 703489757016293377, "text": "@LolaPerez97 ����@EmpireFOX @FOXTV", "in_reply_to_status": 703233290103205889, "in_reply_to_user": 2374176276, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 344520949, 2374176276, 16537989 }}, "user": { "id": 2364917419, "name": "Kaylee❤️", "screen_name": "__Kaylee_rie", "lang": "en", "location": "null", "create_at": date("2014-02-27"), "description": "fuck bitches and get money", "followers_count": 205, "friends_count": 358, "statues_count": 1130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489757444182016, "text": "Snapchat names?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2484645474, "name": "Tyler-morgan", "screen_name": "Teeerodriguez", "lang": "en", "location": "803", "create_at": date("2014-05-08"), "description": "@beyonce ♥️ @officialamiyah .", "followers_count": 1325, "friends_count": 1745, "statues_count": 41593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dentsville, SC", "id": "6adfad7bd6c1e881", "name": "Dentsville", "place_type": "city", "bounding_box": rectangle("-80.995487,34.051162 -80.912497,34.11936") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4519285, "cityName": "Dentsville" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489757792202752, "text": "ive never been this sick from a hangover �� i just wanna be pampered", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3750908424, "name": "DREAAAAA", "screen_name": "dreaadiaz", "lang": "en", "location": "arizona state university", "create_at": date("2015-10-01"), "description": "u do u boo boo", "followers_count": 631, "friends_count": 568, "statues_count": 4965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758157144064, "text": "Airplane mode ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2384773675, "name": "Chip Skylark", "screen_name": "__SkylarMims", "lang": "en", "location": "null", "create_at": date("2014-03-11"), "description": "Proverbs 3:5-6 ULM '18", "followers_count": 237, "friends_count": 118, "statues_count": 2286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, LA", "id": "e60ab9aacc0e96af", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-92.135057,32.429655 -92.018346,32.612831") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2251410, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758161457152, "text": "Wind 0.0 mph ---. Barometer 30.081 in, Steady. Temperature 2.9 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 8932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758224216064, "text": "gn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2285795340, "name": "jimenez", "screen_name": "ammy_jimenez", "lang": "en", "location": "null", "create_at": date("2014-01-10"), "description": "fútbol", "followers_count": 551, "friends_count": 393, "statues_count": 8758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758274539520, "text": "This shouldn't be funny but it is �� @realDonaldTrump https://t.co/D0I0go2E4I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 382672955, "name": "Ashley Nicole ", "screen_name": "Trashthatass", "lang": "en", "location": "gym", "create_at": date("2011-09-30"), "description": "soy Mexicana y Cabrona", "followers_count": 1111, "friends_count": 919, "statues_count": 30948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Cruces, NM", "id": "f515486276aa6192", "name": "Las Cruces", "place_type": "city", "bounding_box": rectangle("-106.843427,32.238361 -106.63145,32.446238") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3539380, "cityName": "Las Cruces" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758488629248, "text": "@ItalynRedGlass @goodrich_linda Oh, I thought she looked familiar!", "in_reply_to_status": 702949323382853633, "in_reply_to_user": 3226843425, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3226843425, 327819855 }}, "user": { "id": 3904270037, "name": "CassieJo Miller", "screen_name": "luv4Billy", "lang": "en", "location": "the South", "create_at": date("2015-10-08"), "description": "Huge fan of the most adorable, award winning actor, Billy Miller. All Billy, all the time.", "followers_count": 16, "friends_count": 0, "statues_count": 165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489758815621120, "text": "I love him! ������ https://t.co/GO7UZgcCv2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 221994776, "name": "LatinaMomInLA", "screen_name": "LatinaMomInLA", "lang": "en", "location": "Los Angeles", "create_at": date("2010-12-01"), "description": "Mom..Wife..Sister...Friend... BP child's Mom/Activist ~ Womens rights r Human rights! Oh and I LOVE MUSIC!!! working on being a better ME - PERSEVERE!", "followers_count": 188, "friends_count": 506, "statues_count": 6623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489759088349184, "text": "Workflow © 2016 @ Harlem Uptown https://t.co/LYhtQ5zsIp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9568741,40.818324"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 136955038, "name": "Jack", "screen_name": "yrface_", "lang": "en", "location": "NYC", "create_at": date("2010-04-25"), "description": "YOUR FACE / Graphic Designer", "followers_count": 293, "friends_count": 108, "statues_count": 14731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489759264374784, "text": "A little faith will bring your soul\nto heaven, but a lot of faith will\nbring heaven to your soul.\nAuthor Unknown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 124, "friends_count": 238, "statues_count": 188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489759520292864, "text": "No plans for Tm Fml .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2839633207, "name": "Ł.ū.ì.$", "screen_name": "Legit_patina", "lang": "en", "location": "null", "create_at": date("2014-10-03"), "description": "Shit is real in the field, kill or be killed", "followers_count": 149, "friends_count": 170, "statues_count": 5262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489759885152256, "text": "Fuller house corny af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377639270, "name": "Jonathan Askew", "screen_name": "AyooAskew", "lang": "en", "location": "Loitering at a local store", "create_at": date("2011-09-21"), "description": "SHS Senior // Active Participant in Coonery", "followers_count": 196, "friends_count": 158, "statues_count": 971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489760199749632, "text": "So today we had an crazy day with my bestie no badthings happen we just enjoy the day love you bestie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279838878, "name": "♡ Chinitaprincess♡", "screen_name": "alynnaaaaaaaa", "lang": "en", "location": "Los Angeles Ca ", "create_at": date("2011-04-09"), "description": "♡ Chinitaprincess ♡ ♡ ♡ ♡ ♡ KIMXI and Kim Chiu forever ♡ ♡ ♡ ♡ and also i love janella and i love miles ocampo", "followers_count": 672, "friends_count": 1458, "statues_count": 28681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489760338116608, "text": "Slide so you can lose 5 . Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362217888, "name": "GoldBearKev", "screen_name": "FilthyKev415", "lang": "en", "location": "The Bay", "create_at": date("2011-08-25"), "description": "Hi I'm Kevin. #GoodVibes", "followers_count": 172, "friends_count": 596, "statues_count": 19857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489760606576641, "text": "@NekronStaghorn now you can harass me on ALL social media sites :-)", "in_reply_to_status": 703489399778971648, "in_reply_to_user": 59046751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 59046751 }}, "user": { "id": 4265078418, "name": "♡ alyssa ♡", "screen_name": "_dece4sed", "lang": "en", "location": "reeferside", "create_at": date("2015-11-16"), "description": "null", "followers_count": 159, "friends_count": 184, "statues_count": 1847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489761181184001, "text": "He just has to like baseball as much as I do, or at least enjoy listening to me when I can't shut up about it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341735855, "name": "Paradise ✨", "screen_name": "little_brenna", "lang": "en", "location": "O.co", "create_at": date("2011-07-24"), "description": "Netflix and Oakland A's is where my heart is at.", "followers_count": 405, "friends_count": 640, "statues_count": 20656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489761277661185, "text": "I literally KEN-NAHHHTT!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2340821767, "name": "Denise Rivera", "screen_name": "DeniseBRR", "lang": "en", "location": "I'm from everywhere. ", "create_at": date("2014-02-12"), "description": "~~~~~ Videographer ~ Editor ~ Gamer ~ Human ~~~~~ Twitch and YouTube: @DeniseBRR Business Inquiries: denisebrivera@gmail.com SnapChat: @island_kisses", "followers_count": 54, "friends_count": 92, "statues_count": 192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489761277845508, "text": "I know I haven't been active on Instagram lately. But I do want to show you this photo… https://t.co/g0s2qhMP6O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2585831,39.7290497"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43704262, "name": "Daniel Kelson Lacy", "screen_name": "electricblue91", "lang": "en", "location": "Indianapolis,Indiana,U.S.A.", "create_at": date("2009-05-31"), "description": "Cosplayer, gamer, photographer, machinima director, and college student. Should I go for more? Just ask me nicely about me.", "followers_count": 235, "friends_count": 230, "statues_count": 3411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianapolis, IN", "id": "018929347840059e", "name": "Indianapolis", "place_type": "city", "bounding_box": rectangle("-86.348441,39.631677 -85.937404,39.927448") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1836003, "cityName": "Indianapolis city (balance)" } }
+{ "create_at": datetime("2016-02-27T00:00:03.000Z"), "id": 703489761370050561, "text": "Why can't I find a game in matchmaking now!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1331946560, "name": "cαѕѕαndrα", "screen_name": "laceupcassie", "lang": "en", "location": "New York", "create_at": date("2013-04-06"), "description": "✖️✖️xQueenCatoriX formerly known as xMsLeGiTBarbiie✖️✖️", "followers_count": 341, "friends_count": 313, "statues_count": 1821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mastic, NY", "id": "ffaba230a4e0ec74", "name": "Mastic", "place_type": "city", "bounding_box": rectangle("-72.866558,40.775992 -72.814025,40.834075") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3646074, "cityName": "Mastic" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489761994895360, "text": "baby b my diamonds & gold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3227607450, "name": "✨", "screen_name": "ammyyg_", "lang": "en", "location": "null", "create_at": date("2015-05-26"), "description": "A.♡", "followers_count": 175, "friends_count": 99, "statues_count": 2995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763018448896, "text": "@GeminiStarlet than k y o u", "in_reply_to_status": 703489559238148096, "in_reply_to_user": 844266476, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 844266476 }}, "user": { "id": 339252447, "name": "Natalie", "screen_name": "Mismembered", "lang": "en", "location": "#OffendCisPeople", "create_at": date("2011-07-20"), "description": "| She/Her | White, Trans, Lesbian (Bi), Poly, Game designer, Student, Feminist | NSFWish | might have 4 arms | ShapeShifter | Has blue hair now", "followers_count": 583, "friends_count": 396, "statues_count": 19283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, VT", "id": "8f07721fe6eb5c41", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-73.276995,44.445714 -73.17455,44.539485") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5010675, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763437727744, "text": "You won't feel any different, but happy birthday �� https://t.co/ZsyE1L2cs2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131760866, "name": "Guissel", "screen_name": "AsapGigi", "lang": "en", "location": "bay area ", "create_at": date("2010-04-11"), "description": "chicana.", "followers_count": 1002, "friends_count": 234, "statues_count": 92599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763467247617, "text": "Boomerang: Coast to Coaster https://t.co/rB8wJaD8xI #boomerangcoasttocoaster #parcdattraction https://t.co/n7uenRGtf6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.610725,29.598547"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "boomerangcoasttocoaster", "parcdattraction" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 839, "friends_count": 1257, "statues_count": 3241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763634974720, "text": "3:00am Temp: 25.0°F (High 26.4/Low 25.0) Wind: WNW 0mph Wind Gust: 4 Pressure: 30.163in. UV Index: 0.0 Rain: 0.0mm. #MDwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.54472222,39.7025"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MDwx" }}, "user": { "id": 702673227945021444, "name": "Josh Henderson", "screen_name": "SVObservatoryWX", "lang": "en", "location": "White Hall, MD", "create_at": date("2016-02-24"), "description": "Amateur Astronomer, Storm Chaser Meteorologist. Providing weather conditions for Harford County. MD, USA NWS, NOAA, CWOP, CoCoRaHS, ACON JHU Applied Physics Lab", "followers_count": 0, "friends_count": 9, "statues_count": 112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763785859072, "text": "������⛽️⛽️ https://t.co/gQeGVlSqN3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1262369904, "name": "MR. PRESIDENT", "screen_name": "AlexSantana0924", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-03-12"), "description": "defensive back | #FreeMax", "followers_count": 352, "friends_count": 295, "statues_count": 22149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489763823767552, "text": "Goodnight y'all ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 578365411, "name": "PRINCESS M", "screen_name": "Trunaaaaeeee", "lang": "en", "location": "Alabama, USA", "create_at": date("2012-05-12"), "description": "2016 Is My Year For New Promises❤️ .. God First Tho. #AAMU", "followers_count": 2143, "friends_count": 2367, "statues_count": 18747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexander City, AL", "id": "1d50e070bad3f2e8", "name": "Alexander City", "place_type": "city", "bounding_box": rectangle("-86.008053,32.906441 -85.920097,32.977895") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1123, "countyName": "Tallapoosa", "cityID": 101132, "cityName": "Alexander City" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489764125638656, "text": "https://t.co/ESEveD4m7b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 289109365, "name": "$tory", "screen_name": "1story_", "lang": "en", "location": "Houston, TX", "create_at": date("2011-04-27"), "description": "marathon... wip anthony story #freecwill", "followers_count": 679, "friends_count": 379, "statues_count": 9538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489764465336320, "text": "@moiegy sisi! https://t.co/MgIOfQs4yo", "in_reply_to_status": 703487546362351616, "in_reply_to_user": 295113790, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 295113790 }}, "user": { "id": 2752477352, "name": "Web_Bests", "screen_name": "Web_Bests", "lang": "en", "location": "RTs-Replies-Follows≠Endorsmnts", "create_at": date("2014-08-21"), "description": "Businesswoman-Kids advocate-Maverick-ENTJ-Innovator-Perfectionist-Internet equalizer-Respects all religions & ethnics-No politics-I WORK HARD!", "followers_count": 138, "friends_count": 2401, "statues_count": 27112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489765157408768, "text": "同い年である叔父同士は同じ学校に通った。世間は「あら、似ていない双子ね」。大笑いだ。", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 107890250, "name": "自称Busker♪=DrMarks本人", "screen_name": "MarkWatermanPhD", "lang": "en", "location": "LosAngeles-Heidelberg-Caesarea", "create_at": date("2010-01-23"), "description": "A good-for-nothing crusty old bugger (˘Ω˘) 要するに、役立たずで嫌われ者の頑固ジジイということ(٭ڋ٭) Science editor and theologian/biblical scholar with PhD became a would-be busker♫", "followers_count": 534, "friends_count": 45, "statues_count": 49532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489765224615936, "text": "Who up? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275831066, "name": "Kinan Humphrey", "screen_name": "Kinan_Humphrey", "lang": "en", "location": " MIA ✈️ ATL ✈️ VA ", "create_at": date("2011-04-01"), "description": "2Love✌️❤️ | #RIPKC | #FreeTee | #LongLiveMike | sc:kingleo4 | #NSU19", "followers_count": 2355, "friends_count": 2383, "statues_count": 23524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489765556015104, "text": "Young AJ & Nina #FBF https://t.co/7wEmNd6y64", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FBF" }}, "user": { "id": 141484416, "name": "Alessandro Joel", "screen_name": "alessandrojoel", "lang": "en", "location": "Los Angeles", "create_at": date("2010-05-07"), "description": "null", "followers_count": 1895, "friends_count": 802, "statues_count": 8400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-27T00:00:04.000Z"), "id": 703489765644099584, "text": "@TERRANCEESCOBAR this Nigga https://t.co/6w3PJb2BQg", "in_reply_to_status": -1, "in_reply_to_user": 256167687, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256167687 }}, "user": { "id": 312994109, "name": "Dj Gorgi Rocks", "screen_name": "41JUICE", "lang": "en", "location": "Uber Black", "create_at": date("2011-06-07"), "description": "Praise to our creator. D.E.A.T.H gorgirocks@gmail.com", "followers_count": 683, "friends_count": 115, "statues_count": 9378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489765778165765, "text": "Wind 2.0 mph WNW. Barometer 30.045 in, Falling slowly. Temperature 32.4 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489765803294720, "text": "@Sxrfa oh shit really damn", "in_reply_to_status": 703489303121252352, "in_reply_to_user": 131395119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 131395119 }}, "user": { "id": 2326631202, "name": "Nat the cat! ¨̮", "screen_name": "RickysVLine", "lang": "en", "location": "Los Angeles, CA || ricky af", "create_at": date("2014-02-03"), "description": "ricky's car broke down, but don't worry, we saved him in our school bus", "followers_count": 8424, "friends_count": 450, "statues_count": 150952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489766205952000, "text": "I think I'm dead and walking around my house a ghost right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069720748, "name": "hattie", "screen_name": "Muthersh1p", "lang": "en", "location": "null", "create_at": date("2013-01-07"), "description": "I have a sock tan", "followers_count": 126, "friends_count": 98, "statues_count": 4728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489766407311361, "text": "@Gen_uinee will eat all of this �� https://t.co/dY2RLbKnFn", "in_reply_to_status": -1, "in_reply_to_user": 1525869062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1525869062 }}, "user": { "id": 2437697054, "name": "PRETTYBITCH", "screen_name": "_eeeeeeeb", "lang": "en", "location": "null", "create_at": date("2014-04-10"), "description": "19&grown", "followers_count": 486, "friends_count": 475, "statues_count": 6328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489766440833025, "text": "Ain't nothing changed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348657837, "name": "Matthew Perera", "screen_name": "matperera", "lang": "en", "location": "California", "create_at": date("2011-08-04"), "description": "Stay tuned for greatness///APU Class of 2019", "followers_count": 391, "friends_count": 345, "statues_count": 3985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489767497867264, "text": "so tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520742888, "name": "damn linda", "screen_name": "thefuckitslinda", "lang": "en", "location": "MINNESOTA", "create_at": date("2012-03-10"), "description": "get wit lin he ain't never going back", "followers_count": 1945, "friends_count": 789, "statues_count": 132961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn Park, MN", "id": "1b86771ff62f45fb", "name": "Brooklyn Park", "place_type": "city", "bounding_box": rectangle("-93.402498,45.063712 -93.279015,45.152479") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2707966, "cityName": "Brooklyn Park" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489767690903553, "text": "@lizzseymour https://t.co/gajCvhOPwH", "in_reply_to_status": -1, "in_reply_to_user": 390704544, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 390704544 }}, "user": { "id": 1380946400, "name": "minaaaa❥", "screen_name": "marinavitalich", "lang": "en", "location": "CA✈️VA", "create_at": date("2013-04-25"), "description": "GMUSB '19", "followers_count": 378, "friends_count": 291, "statues_count": 6041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "George Mason, VA", "id": "0042deed0e8f466a", "name": "George Mason", "place_type": "city", "bounding_box": rectangle("-77.345038,38.824012 -77.288248,38.851728") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5130618, "cityName": "George Mason" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489768575758337, "text": "@titititytia pretty girl������", "in_reply_to_status": 703488306311397376, "in_reply_to_user": 940633526, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 940633526 }}, "user": { "id": 4891290493, "name": "Marie Tiatia", "screen_name": "tiatiababayyy", "lang": "en", "location": "Utah, USA", "create_at": date("2016-02-10"), "description": "Samoan/Puerto Rican •• my son will always be my #1✊ SC: marietiatia", "followers_count": 208, "friends_count": 155, "statues_count": 1002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylorsville, UT", "id": "4b26b09f8108c4e8", "name": "Taylorsville", "place_type": "city", "bounding_box": rectangle("-111.986946,40.627557 -111.907843,40.686322") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4975360, "cityName": "Taylorsville" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489768731049984, "text": "Add me on �� paola001", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 348294665, "name": "Paola Gomez", "screen_name": "paolangomez", "lang": "en", "location": "null", "create_at": date("2011-08-03"), "description": "null", "followers_count": 890, "friends_count": 388, "statues_count": 21568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489768806453249, "text": "Welcome to the Lubbock trailer hood and get ready for some Tau-ladega nights. ���� https://t.co/3XhhBVLTul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343861493, "name": "Taylar-Jayde", "screen_name": "TaylarJayde", "lang": "en", "location": "ENGLAND | HOUSTON | LUBBOCK", "create_at": date("2011-07-27"), "description": "↠Texas Tech↞ Id rather be at a baseball game", "followers_count": 565, "friends_count": 463, "statues_count": 18207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489769297330177, "text": "And when the lights start flashing like a photo booth \nAnd the stars exploding we'll be… https://t.co/QAjzuTeU8u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.92437348,33.87042289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125993560, "name": "Mark Christopherr", "screen_name": "MCAlviz", "lang": "en", "location": "The OC & Manila is ❤️", "create_at": date("2010-03-24"), "description": "I'm a hopeless romantic. There, I said it! I'm also sexy. There, I lied! I'm also slutty! That, you wish!", "followers_count": 268, "friends_count": 158, "statues_count": 13770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489769435734017, "text": "Damn NYC is kinda lit but it's something new to me idk I feel judged af here��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313137808, "name": "Chalupa Batman", "screen_name": "OmarMagana", "lang": "en", "location": "ATL", "create_at": date("2011-06-07"), "description": "probably stumbling around faded attempting to play guitar in @CYTATL gt:omarbadblood", "followers_count": 11592, "friends_count": 8118, "statues_count": 66883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-27T00:00:05.000Z"), "id": 703489769804800000, "text": "@sxphivmindful word", "in_reply_to_status": 703489678331273216, "in_reply_to_user": 3054038268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3054038268 }}, "user": { "id": 104927561, "name": "Kenny H", "screen_name": "fuckennyh", "lang": "en", "location": "NY / Tampa", "create_at": date("2010-01-14"), "description": "@cruelartmedia", "followers_count": 1706, "friends_count": 718, "statues_count": 29788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489770832396288, "text": "@PJStrikeForce What a bunch of BS, think trump is going to solve our problems? Nobody can overnight or in a day, week But give Cruz chance", "in_reply_to_status": 703488320077111296, "in_reply_to_user": 121656624, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 121656624 }}, "user": { "id": 22184436, "name": "Born 2 Seek", "screen_name": "TrthSeeker", "lang": "en", "location": "America, Gods Country", "create_at": date("2009-02-27"), "description": "A Conservative Activist. and PROUD Supporter of our TROOPS", "followers_count": 1119, "friends_count": 1567, "statues_count": 10626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489771558084608, "text": "@jennalmh This bout me? Lol", "in_reply_to_status": 703443506765783041, "in_reply_to_user": 51197033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51197033 }}, "user": { "id": 499069388, "name": "Ty", "screen_name": "_UNOCEROTRES_", "lang": "en", "location": "Warrenton, MO", "create_at": date("2012-02-21"), "description": "Feelin' flawless, livin' lawless. Moto ✊", "followers_count": 678, "friends_count": 601, "statues_count": 21329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrenton, MO", "id": "ea5894bfe2b5d77f", "name": "Warrenton", "place_type": "city", "bounding_box": rectangle("-91.179334,38.784445 -91.098133,38.859414") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29219, "countyName": "Warren", "cityID": 2977128, "cityName": "Warrenton" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489773046931456, "text": "Literally the best thing in the world is having nice soft legs and fuzzy socks����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1571214938, "name": "Kayla Carranza", "screen_name": "kayla_carranza", "lang": "en", "location": "null", "create_at": date("2013-07-05"), "description": "Sassy & 17", "followers_count": 110, "friends_count": 143, "statues_count": 1778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489773243994112, "text": "Wind 4.0 mph W. Barometer 30.144 in, Steady. Temperature 30.0 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 383, "friends_count": 291, "statues_count": 8583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489773491478530, "text": "Spent like 500$ fuck it one night not tripping it was cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2298263368, "name": "Rob", "screen_name": "Jays4days_htx", "lang": "en", "location": "Second Ward, Houston", "create_at": date("2014-01-22"), "description": "Use to have nothing... Now I got a whole lot of everything #Blessed #GhettoDreams #2k16 ♊️", "followers_count": 556, "friends_count": 399, "statues_count": 2382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489773688586242, "text": "2 Lil' lopes listening to Fetty Wap and Post Malone �� https://t.co/QxE5iMA3er", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 865562256, "name": "Sarah Jackson", "screen_name": "SarahhhJackson3", "lang": "en", "location": "CA||AZ", "create_at": date("2012-10-06"), "description": "||Insta/snapchat: sarahhhjacksonn||GCU'19", "followers_count": 1237, "friends_count": 1094, "statues_count": 8279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-02-27T00:00:06.000Z"), "id": 703489773986385920, "text": "@vKratic isnt backwards comp coming soon", "in_reply_to_status": 703489577051262976, "in_reply_to_user": 617827229, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617827229 }}, "user": { "id": 1014376770, "name": "Chandler", "screen_name": "lRichyy", "lang": "en", "location": "Canada", "create_at": date("2012-12-15"), "description": "Runescape is the best game ever created and if you've never played it im truly sorry | was once amazing at COD | duo- @Icy_Rapture | Computer Engineer @ ISU", "followers_count": 2040, "friends_count": 265, "statues_count": 4054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-02-27T00:00:07.000Z"), "id": 703489774414237696, "text": "@SpencerGrefe you're my hero", "in_reply_to_status": -1, "in_reply_to_user": 2722916364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2722916364 }}, "user": { "id": 2331047508, "name": "vannah", "screen_name": "savannahwesttt", "lang": "en", "location": "(◠‿◠✿)", "create_at": date("2014-02-06"), "description": "good girl & you know it // sc: savannahwestt", "followers_count": 362, "friends_count": 285, "statues_count": 6460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Elsinore, CA", "id": "80cf9987ff7e9762", "name": "Lake Elsinore", "place_type": "city", "bounding_box": rectangle("-117.413156,33.618447 -117.216785,33.713284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 639486, "cityName": "Lake Elsinore" } }
+{ "create_at": datetime("2016-02-27T00:00:07.000Z"), "id": 703489775114686464, "text": "World at War is my favorite Call of Duty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153282458, "name": "Big Nashty", "screen_name": "Gotlemonicedtea", "lang": "en", "location": "null", "create_at": date("2010-06-07"), "description": "Oh well, whatever, nevermind #Uic #Chicago", "followers_count": 633, "friends_count": 428, "statues_count": 39135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-27T00:00:07.000Z"), "id": 703489775173443584, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 412774172, "name": "DÄDDŸ 〽️ÊŸ✨", "screen_name": "Queen_Folarin", "lang": "en", "location": "California, USA", "create_at": date("2011-11-14"), "description": "If ya mind was ya body...wouldn't nobody wanna be beside it. - @Wale ❤️ #TrxllGXDS", "followers_count": 1690, "friends_count": 1271, "statues_count": 30000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-27T00:00:07.000Z"), "id": 703489775202865152, "text": "Gotta thank god ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3693093676, "name": "• Lilo", "screen_name": "YessyKennia", "lang": "en", "location": "Queens, NY", "create_at": date("2015-09-17"), "description": "null", "followers_count": 45, "friends_count": 36, "statues_count": 1380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-02-27T00:00:07.000Z"), "id": 703489775324553217, "text": "@illanikz Zella Day at Mountain Winery (Aug 21, 2016) https://t.co/epzCRtqspA https://t.co/gPWoNNRIZe", "in_reply_to_status": -1, "in_reply_to_user": 333416012, "favorite_count": 0, "coordinate": point("-122.0553675,37.2590928"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333416012 }}, "user": { "id": 343197788, "name": "Songkick alerts", "screen_name": "kickalert", "lang": "en", "location": "At a gig", "create_at": date("2011-07-26"), "description": "Get automatic @ mentions for your tracked artists & locations on @songkick. A project by @mario. To start, register at http://kickalert.me", "followers_count": 1634, "friends_count": 1, "statues_count": 162924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saratoga, CA", "id": "b46f044cb28493c6", "name": "Saratoga", "place_type": "city", "bounding_box": rectangle("-122.068181,37.237066 -121.989501,37.296386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 670280, "cityName": "Saratoga" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852132974211076, "text": "actually a frat boy when it comes to my hat game https://t.co/iBFFsrK9ei", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178821800, "name": "lil_lit b", "screen_name": "jinnjuicee", "lang": "en", "location": "Missouri, USA", "create_at": date("2010-08-15"), "description": "CEO, Dream girl, Drug dealaaaaah #spittinbarzzzz. \nfeminist as fuck", "followers_count": 459, "friends_count": 493, "statues_count": 13595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gladstone, MO", "id": "e5731aae9bb5e805", "name": "Gladstone", "place_type": "city", "bounding_box": rectangle("-94.593389,39.173738 -94.529799,39.235151") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29047, "countyName": "Clay", "cityID": 2927190, "cityName": "Gladstone" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852133271998464, "text": "@_diggydigs ahahahah fuck your header bitch", "in_reply_to_status": -1, "in_reply_to_user": 3140962484, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3140962484 }}, "user": { "id": 1016590117, "name": "SAV", "screen_name": "savanahvanzee_", "lang": "en", "location": " JWW♡", "create_at": date("2012-12-16"), "description": "look good feel good", "followers_count": 1310, "friends_count": 316, "statues_count": 39137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valle Vista, CA", "id": "6504ec137a7160ee", "name": "Valle Vista", "place_type": "city", "bounding_box": rectangle("-116.919834,33.729635 -116.85798,33.765951") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 681708, "cityName": "Valle Vista" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852133418868738, "text": "���� https://t.co/ch3M2GBpW6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 346406641, "name": "H△⃒⃘n△⃒⃘n Kourtu", "screen_name": "Hanan1001K", "lang": "en", "location": "hogwarts", "create_at": date("2011-07-31"), "description": "it's hananoween", "followers_count": 260, "friends_count": 315, "statues_count": 5257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852133632704512, "text": "I miss my best friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262989637, "name": "Franci Elle", "screen_name": "FranciBurton", "lang": "en", "location": "basically from louisville", "create_at": date("2011-03-08"), "description": "KKΓ • seek the beautiful and the good • Semper Fi, Jackson Cade Ragains is my US Marine", "followers_count": 909, "friends_count": 1008, "statues_count": 9818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852133745995776, "text": "\"With one hand the past moves us forward, with the other it holds us back.\" #Frasier", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Frasier" }}, "user": { "id": 169355376, "name": "Brandon, Son of Pen.", "screen_name": "PensonBrandon", "lang": "en", "location": "East Moline, IL", "create_at": date("2010-07-21"), "description": "Proud geek, Film nerd, and Quad Cities native. Lover of cars, comic books, the Bulls, the Blackhawks, coffee and Playboy Playmates. Be well and live awesome!", "followers_count": 776, "friends_count": 2027, "statues_count": 20414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Moline, IL", "id": "6539db53e6d91750", "name": "East Moline", "place_type": "city", "bounding_box": rectangle("-90.462938,41.469736 -90.366762,41.551533") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17161, "countyName": "Rock Island", "cityID": 1722073, "cityName": "East Moline" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852134471602178, "text": "if you're going to start telling me something then say \"nvm\" or just change your mind then DON'T bring it up in the first place ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2575584316, "name": "Maria", "screen_name": "haii_itz_maria", "lang": "en", "location": "null", "create_at": date("2014-06-01"), "description": "null", "followers_count": 307, "friends_count": 169, "statues_count": 8104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852134777823232, "text": "Listen to I Wanna Rock (Remix) by Young Meez #np on #SoundCloud\nhttps://t.co/NuteUlUYRF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9810287,40.9005554"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "np", "SoundCloud" }}, "user": { "id": 457469251, "name": "Young Playboy Porter", "screen_name": "MeezPorter", "lang": "en", "location": "Wolf City !", "create_at": date("2012-01-07"), "description": "EP OUT NOW!! MixtapeiNMotion #JERSEY", "followers_count": 214, "friends_count": 335, "statues_count": 987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englewood, NJ", "id": "65e3a577f1940550", "name": "Englewood", "place_type": "city", "bounding_box": rectangle("-73.994557,40.865695 -73.949202,40.913406") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3421480, "cityName": "Englewood" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852134966497280, "text": "@cxweak erick u silly bo billy", "in_reply_to_status": 703851893152321536, "in_reply_to_user": 2999064752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2999064752 }}, "user": { "id": 509809912, "name": "antonio", "screen_name": "steezelo", "lang": "en", "location": "Fresno, CA", "create_at": date("2012-02-29"), "description": "null", "followers_count": 1677, "friends_count": 840, "statues_count": 41305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852134991695875, "text": "Ask me how!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476359789, "name": "Jon Massey⚾️", "screen_name": "J_massey4", "lang": "en", "location": "null", "create_at": date("2012-01-27"), "description": "http://God.Family.Baseball ✨Vegas✨", "followers_count": 836, "friends_count": 795, "statues_count": 9956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852135948034048, "text": "@patriziaandreoz ho già avuto la mia dose di fortuna a trovare un lavoro X una azienda americana, ora sta a me,ma X ora gli servo in Europa.", "in_reply_to_status": 703851720523132928, "in_reply_to_user": 573821701, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 573821701 }}, "user": { "id": 401256157, "name": "Me, Myself & Mario®", "screen_name": "FassoliMario", "lang": "it", "location": "un po' qua.... un po' là.", "create_at": date("2011-10-30"), "description": "definitevi voi... nel frattempo io rido ..", "followers_count": 219, "friends_count": 165, "statues_count": 24576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spry, PA", "id": "0cfbc6c688507cb8", "name": "Spry", "place_type": "city", "bounding_box": rectangle("-76.704543,39.893732 -76.661363,39.932708") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4273528, "cityName": "Spry" } }
+{ "create_at": datetime("2016-02-28T00:00:00.000Z"), "id": 703852136119971840, "text": "@perfect9y7 STOP RTING FIGHTS THEY HURT MY HEART", "in_reply_to_status": -1, "in_reply_to_user": 2653471228, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2653471228 }}, "user": { "id": 1582624866, "name": "g-lion", "screen_name": "_jayllian", "lang": "en", "location": "Area 6 • D5170", "create_at": date("2013-07-10"), "description": "a vessel of morbid humor, self-deprecating outbursts, & genuine vitality // bobbysoxer", "followers_count": 324, "friends_count": 287, "statues_count": 13184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852136988172291, "text": "@trinaaryana you didn't like one of them", "in_reply_to_status": 703851818816655361, "in_reply_to_user": 3061062344, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3061062344 }}, "user": { "id": 324830322, "name": "Rilla.", "screen_name": "LalalaLiaCakes", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "@joeyrice15 is mine USE UBER CODE: liav78ue", "followers_count": 163, "friends_count": 278, "statues_count": 5030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852137055281152, "text": "Wish you were here @HTC_Chelsea @HTC_Kelly @HTC_Emily @HTC_AshleyR @HTC_Mikaley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3146243787, 3146721548, 3102712483, 3145749712, 2433112993 }}, "user": { "id": 2990752895, "name": "Dallas Thompson", "screen_name": "dallasdaddy321", "lang": "en", "location": "null", "create_at": date("2015-01-21"), "description": "Ive only been gluten-free for a week but im already really annoying", "followers_count": 126, "friends_count": 188, "statues_count": 645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852137227251712, "text": "@ebbtideapp Tide in Fort Hamilton, New York 02/28/2016\n Low 5:17am 0.5\nHigh 11:21am 4.1\n Low 5:22pm 0.5\nHigh 11:50pm 4.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.035,40.6083"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 36, "friends_count": 1, "statues_count": 11503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, NY", "id": "27485069891a7938", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-74.255641,40.495865 -73.699793,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852137554399233, "text": "Hmm its 12 in cali �� Its my bb's bday �� happy birthday joel ��❤️ @itsjoelpimentel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3191117990 }}, "user": { "id": 4329292812, "name": "alexa", "screen_name": "c_alexxaa", "lang": "en", "location": "cali", "create_at": date("2015-11-22"), "description": "@itsjoelpimentel", "followers_count": 32, "friends_count": 98, "statues_count": 1322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852137625726977, "text": "02/28@03:00 - Temp 33.6F, WC 33.6F. Wind 1.0mph SW, Gust 3.0mph. Bar 30.053in, Falling slowly. Rain 0.00in. Hum 74%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138032533504, "text": "Love you too https://t.co/FKpIqrcER6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2656188883, "name": "nation✨", "screen_name": "BOMB__diggidy", "lang": "en", "location": "Tx", "create_at": date("2014-07-18"), "description": "wintertime zi", "followers_count": 607, "friends_count": 245, "statues_count": 30245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138175156224, "text": "HAPPY BIRTHDAY TO ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276819128, "name": "Zachary Gaynor", "screen_name": "z_gaynor", "lang": "en", "location": "null", "create_at": date("2011-04-03"), "description": "#USMC", "followers_count": 466, "friends_count": 224, "statues_count": 7605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevenson Ranch, CA", "id": "00ab310716dc2504", "name": "Stevenson Ranch", "place_type": "city", "bounding_box": rectangle("-118.622276,34.366596 -118.560263,34.418064") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 674130, "cityName": "Stevenson Ranch" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138456182786, "text": "View leaving the Walter Pyramid after the game. Sunset for UC Riverside. #GoBeach @ Walter… https://t.co/9qwcAT1R0V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.11501861,33.78746484"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoBeach" }}, "user": { "id": 2238391627, "name": "Phil Cox", "screen_name": "PhilCox58", "lang": "en", "location": "null", "create_at": date("2013-12-09"), "description": "null", "followers_count": 76, "friends_count": 243, "statues_count": 4798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138703654921, "text": "@nicpeps Happy 21st Birthday!!!No doubt in my mind Flagstaff is treating you well������...please no calls from jail", "in_reply_to_status": -1, "in_reply_to_user": 1479894576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1479894576 }}, "user": { "id": 1205663868, "name": "Tricia Pepe", "screen_name": "TriciaPepe", "lang": "en", "location": "California, USA", "create_at": date("2013-02-21"), "description": "Producer for America Now with Meghan McCain @premierenetwork & @iHeartRadio", "followers_count": 137, "friends_count": 209, "statues_count": 404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seal Beach, CA", "id": "d35feae19d268b09", "name": "Seal Beach", "place_type": "city", "bounding_box": rectangle("-118.117365,33.723095 -118.041655,33.787561") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 670686, "cityName": "Seal Beach" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138724626432, "text": "Barberton Oh Temp:37.3°F Wind:3 mph Dir:S Baro:Falling slowly Rain2day:0.00in Hum:70% UV:0.0 @ 03:00 02/28/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 229, "statues_count": 114975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138745634817, "text": "Wind 0.0 mph ---. Barometer 30.22 in, Falling slowly. Temperature 42.6 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852138833661954, "text": "Like what's good https://t.co/Rs772cG9n4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374311006, "name": "Lauren", "screen_name": "Lwiild", "lang": "en", "location": "null", "create_at": date("2011-09-15"), "description": "#girlpower. ♐", "followers_count": 302, "friends_count": 395, "statues_count": 7156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "York, PA", "id": "39c84b689e24ab85", "name": "York", "place_type": "city", "bounding_box": rectangle("-76.762559,39.942947 -76.699457,39.991071") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4287048, "cityName": "York" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852139366338560, "text": "Hey kiefer sutherland, go home already so I can finish my job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1895626844, "name": "Stacy Elena", "screen_name": "Fatbottomgirlx", "lang": "en", "location": "California", "create_at": date("2013-09-22"), "description": "ig: stasssyx", "followers_count": 64, "friends_count": 70, "statues_count": 6984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852139936808960, "text": "happy birthday @natwhatevs ��✨✨���� I love u ��⭐️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2833356094 }}, "user": { "id": 2223077760, "name": "p.", "screen_name": "_falconprincess", "lang": "en", "location": "IG:falconprincess//SC:pdfalcon", "create_at": date("2013-11-30"), "description": "this must be my dream", "followers_count": 2236, "friends_count": 415, "statues_count": 56338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852140435910656, "text": "In a mood for a taco run ahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1556535542, "name": "Coda", "screen_name": "BestCoda", "lang": "en", "location": "null", "create_at": date("2013-06-29"), "description": "6'1 , Future physician", "followers_count": 1363, "friends_count": 1195, "statues_count": 37628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852140570107904, "text": "#SupportOriginMelissa 43.5°F Wind:4.7mph Pressure: 30.06hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852140666589185, "text": "@BeyondScaredd that kid is life omf", "in_reply_to_status": 697583131013050370, "in_reply_to_user": 4687623691, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4687623691 }}, "user": { "id": 2676507285, "name": "Go Away.", "screen_name": "KServio", "lang": "en", "location": "null", "create_at": date("2014-07-05"), "description": "null", "followers_count": 131, "friends_count": 133, "statues_count": 385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-02-28T00:00:01.000Z"), "id": 703852140767227908, "text": "same Shwayze ���� https://t.co/kwlYcaI7ge", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327642832, "name": "makenzie ✾", "screen_name": "makenziechae", "lang": "en", "location": "Park City, MT", "create_at": date("2011-07-01"), "description": "smoke yr cigarette, I hope you choke", "followers_count": 445, "friends_count": 426, "statues_count": 4702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bozeman, MT", "id": "807584e8539d2a7e", "name": "Bozeman", "place_type": "city", "bounding_box": rectangle("-111.185677,45.625349 -110.980181,45.750351") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30031, "countyName": "Gallatin", "cityID": 3008950, "cityName": "Bozeman" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852141694169091, "text": "Wind 3.0 mph SE. Barometer 29.866 in, Falling. Temperature 53.0 °F. Rain today 0.00 in. Humidity 47%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852141828411392, "text": "Temp: 42.1°F Wind:0.0mph Pressure: 30.248hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852142474334208, "text": "Can't stop watching ������ https://t.co/EVR8i6fdEu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243002287, "name": "Cheyenne❣", "screen_name": "__xSimplyMe", "lang": "en", "location": "College girl", "create_at": date("2011-01-25"), "description": "#BlackLivesMatter", "followers_count": 2679, "friends_count": 1970, "statues_count": 81758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852142705020929, "text": "facetime me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 942407540, "name": "jacqueline", "screen_name": "PlazaJackie", "lang": "en", "location": "N. Long Beach ", "create_at": date("2012-11-11"), "description": "null", "followers_count": 1007, "friends_count": 398, "statues_count": 13612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852142814035968, "text": "I want a panini from Broadway diner", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1210562070, "name": "SaxCr", "screen_name": "sherylromwah", "lang": "en", "location": "NJ", "create_at": date("2013-02-22"), "description": "cuddles, sleep & qdoba is all I really need", "followers_count": 298, "friends_count": 240, "statues_count": 9983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayonne, NJ", "id": "3d3c56338b6a3b4a", "name": "Bayonne", "place_type": "city", "bounding_box": rectangle("-74.146932,40.643773 -74.0658,40.697794") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3403580, "cityName": "Bayonne" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852143132807169, "text": "MY PIZZA IS DONE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 208764598, "name": "Claire Porter", "screen_name": "clairelela", "lang": "en", "location": "null", "create_at": date("2010-10-27"), "description": "quantum superstate feminist", "followers_count": 106, "friends_count": 179, "statues_count": 1587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852143501914114, "text": "Wind 1.0 mph SW. Barometer 30.041 in, Falling slowly. Temperature 41.4 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852143577473026, "text": "wow drunk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46533052, "name": "jordyn", "screen_name": "jorrrrrrrrd", "lang": "en", "location": "null", "create_at": date("2009-06-11"), "description": "tysm.", "followers_count": 401, "friends_count": 391, "statues_count": 12407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Idaho Falls, ID", "id": "953525ef572ed959", "name": "Idaho Falls", "place_type": "city", "bounding_box": rectangle("-112.094583,43.444371 -111.974283,43.553475") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16019, "countyName": "Bonneville", "cityID": 1639700, "cityName": "Idaho Falls" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852143606771713, "text": "can never go wrong with pizza & fries at 3am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2186994494, "name": "fęličiå", "screen_name": "KayaKycat", "lang": "en", "location": "null", "create_at": date("2013-11-10"), "description": "love yourself❤️ young thug❣", "followers_count": 1155, "friends_count": 1102, "statues_count": 20987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852143942377473, "text": "I fer real can't sleep with all this ringingin my ears", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 599626840, "name": "Zuri", "screen_name": "yo_boi_zuri", "lang": "en", "location": "Mo-Town ", "create_at": date("2012-06-04"), "description": "glo up failed", "followers_count": 526, "friends_count": 433, "statues_count": 28871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852144344961024, "text": "@XRamon21 ���������� https://t.co/ob8Ox37uZG", "in_reply_to_status": -1, "in_reply_to_user": 703813270654746624, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 703813270654746624 }}, "user": { "id": 1401104137, "name": "MamiBri", "screen_name": "_yadirabby", "lang": "en", "location": "null", "create_at": date("2013-05-03"), "description": "Austin ,Tx 18'", "followers_count": 575, "friends_count": 519, "statues_count": 14335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852144462434309, "text": "@parseltung training bra typa boobs so prob like 12 lol", "in_reply_to_status": 703851964904308736, "in_reply_to_user": 2419529305, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2419529305 }}, "user": { "id": 3434557933, "name": "jay", "screen_name": "aisooos", "lang": "en", "location": "502", "create_at": date("2015-09-03"), "description": "wrestlers do it best", "followers_count": 323, "friends_count": 301, "statues_count": 3367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Pointe, HI", "id": "013abac55b5e444f", "name": "Ocean Pointe", "place_type": "city", "bounding_box": rectangle("-158.044905,21.303603 -158.013973,21.328864") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1556685, "cityName": "Ocean Pointe" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852144508579841, "text": "how do any of you want Trump???? how do you want the richest man in the world to represent YOU??? what the fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 215453816, "name": "Haley Sarvis", "screen_name": "sarvishaley", "lang": "en", "location": "null", "create_at": date("2010-11-13"), "description": "rather be a rebel than a slave", "followers_count": 745, "friends_count": 155, "statues_count": 24901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-02-28T00:00:02.000Z"), "id": 703852144533700608, "text": "And we know that in all things God works for the good of those who love him, who have been called according to his purpose.\nRomans 8:28", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 126, "friends_count": 240, "statues_count": 190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852145708126208, "text": "47.7F (Feels: 47.7F) - Humidity: 86% - Wind: 0.7mph E - Gust: 3.1mph - Pressure: 1034.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852145892659200, "text": "@DKnucklesFerg not hungry; Caesars for dinner. ����", "in_reply_to_status": 703849558829195268, "in_reply_to_user": 322495549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 322495549 }}, "user": { "id": 2609179484, "name": "Damian Phillips", "screen_name": "AceKillaThrilla", "lang": "en", "location": "Gallup, NM & Las Vegas, NV", "create_at": date("2014-07-07"), "description": "Poker Player, future husband, life enthusiast. #OnToTheNextOne", "followers_count": 380, "friends_count": 294, "statues_count": 2468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146010107908, "text": "truth be told I never was yours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242998635, "name": "savage T ☯", "screen_name": "tonigirlfml", "lang": "en", "location": "Bridgeport, CT", "create_at": date("2015-05-08"), "description": "I've got more issues then vogue", "followers_count": 146, "friends_count": 82, "statues_count": 690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgeport, CT", "id": "7714f2c70ed66321", "name": "Bridgeport", "place_type": "city", "bounding_box": rectangle("-73.244348,41.141738 -73.153889,41.229611") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 908000, "cityName": "Bridgeport" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146161090560, "text": "Good evening #NorthDakota! Sun will set in 30 minutes (local time 18:30) #Sunset #Photo #Photography [28.02.2016]", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.77778,47.26417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NorthDakota", "Sunset", "Photo", "Photography" }}, "user": { "id": 4567387397, "name": "PhotoSchedule", "screen_name": "PhotoSchedule", "lang": "en", "location": "Finland", "create_at": date("2015-12-22"), "description": "We bring up sunrise and sunset around the world so you can get your camera ready or just enjoy the view. Developed by @JanneHonkonen / @TheOwlkeep", "followers_count": 184, "friends_count": 3, "statues_count": 55040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beulah, ND", "id": "ad0db983ad079634", "name": "Beulah", "place_type": "city", "bounding_box": rectangle("-101.791111,47.253694 -101.753992,47.286459") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38057, "countyName": "Mercer", "cityID": 3806660, "cityName": "Beulah" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146366623745, "text": "You say I'm not alone, but I'm petrified.. |-/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2853550387, "name": "Jacqueline", "screen_name": "yahque_lean", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "follow me, I'm funny. Well, kind of..", "followers_count": 32, "friends_count": 157, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146530193408, "text": "@CityOnFireNews @easternfilmfans @hkfilmnet @lovehkfilm @VCFilmFest @vcinemahsow Second item. https://t.co/7CqSUzd6Q6", "in_reply_to_status": -1, "in_reply_to_user": 324745210, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 324745210, 236374513, 22206617, 35205162, 111688399 }}, "user": { "id": 58693179, "name": "Daniel Zelter", "screen_name": "ZERUDAH", "lang": "en", "location": "WeHo", "create_at": date("2009-07-20"), "description": "Me.", "followers_count": 919, "friends_count": 593, "statues_count": 477305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146651828225, "text": "My cousin called Jason \"brother in law \" WHERRE WHEN LMFAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2455999370, "name": "Summer16", "screen_name": "giselle_cerv21", "lang": "en", "location": "null", "create_at": date("2014-04-20"), "description": "I got niggas trynna end me", "followers_count": 83, "friends_count": 97, "statues_count": 2316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146748366848, "text": "Wind 1.6 mph S. Barometer 29.86 in, Steady. Temperature 30.2 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 111, "statues_count": 157856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852146848956419, "text": "Everything..........that's a lot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1161086594, "name": "RollMaster Wawa", "screen_name": "alessprim", "lang": "en", "location": "Blunt City, USA", "create_at": date("2013-02-08"), "description": "complicated & classy", "followers_count": 64, "friends_count": 123, "statues_count": 1449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Gables, FL", "id": "778af41ffb719450", "name": "Coral Gables", "place_type": "city", "bounding_box": rectangle("-80.297165,25.627038 -80.245009,25.771859") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214250, "cityName": "Coral Gables" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852147050311681, "text": "My feet hurt od", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 485190850, "name": "PrincessCiaraa", "screen_name": "PrincessCiaraa_", "lang": "en", "location": "null", "create_at": date("2012-02-06"), "description": "pcruz894@gmail.com", "followers_count": 1467, "friends_count": 295, "statues_count": 53817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852147171983360, "text": "happy birthday to u �� have a great day! love u ������ https://t.co/75IgIB5rRK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275629075, "name": "allyson", "screen_name": "allyson108_", "lang": "en", "location": "washington", "create_at": date("2011-04-01"), "description": "null", "followers_count": 729, "friends_count": 329, "statues_count": 25413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Puyallup, WA", "id": "000dea80079d8b64", "name": "Puyallup", "place_type": "city", "bounding_box": rectangle("-122.336103,47.145624 -122.232425,47.207294") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356695, "cityName": "Puyallup" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852147293564928, "text": "\"I'm just waiting for you to come back\" - R. \nThose words made my heart melt. The best words I've heard so far. ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1136956573, "name": "Tracy", "screen_name": "Trenggggg", "lang": "en-gb", "location": "null", "create_at": date("2013-01-31"), "description": "• 19", "followers_count": 607, "friends_count": 257, "statues_count": 37821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852147943735296, "text": "I love a \"stayed down til we came up\" couple. Yassss. You better stick it out.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22382385, "name": "METRA BOOMIN", "screen_name": "TheeHeartthrob", "lang": "en", "location": "Mississippi ✈️ Atlanta", "create_at": date("2009-03-01"), "description": "Chic.. where the hood and high class meets.. I'm everybody's type. I go with everybody's life. I got what everybody like.", "followers_count": 6555, "friends_count": 5221, "statues_count": 110531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148040142848, "text": "#3AMWeMeetAgain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "3AMWeMeetAgain" }}, "user": { "id": 1075248840, "name": "LENNY: The Red Mamba", "screen_name": "redheadLEN", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-01-09"), "description": "IG: redheadLEN • @WorldFamousNYC • #HorrorWorld • #WorldBBQ • #WorldLove •#TheGreatestDayEver • #HoopsShowdown • #FlagFootballShowdown • #3AmWeMeetAgain", "followers_count": 1551, "friends_count": 1000, "statues_count": 59474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148467986432, "text": "I love Bloomsburggggggggggg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132353715, "name": "Hannah Goralczyk", "screen_name": "HmgHanMackenzie", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2010-04-12"), "description": "18 |NCHS '15|Bloomsburg University '19|SC: Hanmack01 Insta:hannah_goralczyk97", "followers_count": 310, "friends_count": 196, "statues_count": 15622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomsburg, PA", "id": "1862782bb20740eb", "name": "Bloomsburg", "place_type": "city", "bounding_box": rectangle("-76.477539,40.977883 -76.428725,41.025192") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42037, "countyName": "Columbia", "cityID": 4207128, "cityName": "Bloomsburg" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148841263104, "text": "Steele Hughes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 124331818, "name": "Connor Reap", "screen_name": "Connor_Reap", "lang": "en", "location": "MTSU '19", "create_at": date("2010-03-18"), "description": "• @madisonetaylor visits me at work • Future Powerball Winner • Mac n Cheese Enthusiast •", "followers_count": 1004, "friends_count": 667, "statues_count": 9775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green Hill, TN", "id": "bf4f35ca65ddb50c", "name": "Green Hill", "place_type": "city", "bounding_box": rectangle("-86.594998,36.215808 -86.531261,36.250715") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47189, "countyName": "Wilson", "cityID": 4731100, "cityName": "Green Hill" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148883222528, "text": "More #bestfriendshit - yeah, you know my #dancingsoulmate alexspady. �������� @ Euphoria Nightclub https://t.co/H2CBqOuFqh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6631,45.52051"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bestfriendshit", "dancingsoulmate" }}, "user": { "id": 34987806, "name": "Cassie Lovett", "screen_name": "igotawesome", "lang": "en", "location": "orrrrregon", "create_at": date("2009-04-24"), "description": "it's in the name. everyone in my life is the same- plain awesome.\n\n*these tweets rep me, not any institution with which i may be affiliated.", "followers_count": 169, "friends_count": 653, "statues_count": 2004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148967100416, "text": "@longlive_alexx money talks fam", "in_reply_to_status": 703850507270377473, "in_reply_to_user": 342562070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342562070 }}, "user": { "id": 2610852628, "name": "Brennddaa✨", "screen_name": "Brennndda_", "lang": "en", "location": "South Gate, CA", "create_at": date("2014-06-15"), "description": "null", "followers_count": 340, "friends_count": 318, "statues_count": 6781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852148979687424, "text": "Like who the fuck made your stupid ass??????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3300948137, "name": "Lexi Egnor ✨", "screen_name": "lexi_egnor1224", "lang": "en", "location": "null", "create_at": date("2015-05-27"), "description": "RIP Skyler // 6.9.15 ❤️❤️. RIP Devan // 2.11.16 ❤️❤️.", "followers_count": 252, "friends_count": 320, "statues_count": 4980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Van Wert, OH", "id": "ed241b86b5da2a41", "name": "Van Wert", "place_type": "city", "bounding_box": rectangle("-84.627834,40.842859 -84.552539,40.893959") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39161, "countyName": "Van Wert", "cityID": 3979562, "cityName": "Van Wert" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852149101309952, "text": "Wind 2.0 mph S. Barometer 29.896 in, Falling. Temperature 39.2 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 382, "friends_count": 291, "statues_count": 8609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852149113884672, "text": "@LSDbrzy very relevant", "in_reply_to_status": 647869189642940416, "in_reply_to_user": 2901934458, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2901934458 }}, "user": { "id": 2901934458, "name": "briana", "screen_name": "LSDbrzy", "lang": "en", "location": "null", "create_at": date("2014-11-16"), "description": "bladadahh", "followers_count": 401, "friends_count": 214, "statues_count": 74 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasanton, CA", "id": "ad4876a662119b74", "name": "Pleasanton", "place_type": "city", "bounding_box": rectangle("-121.956864,37.621859 -121.798606,37.704036") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 657792, "cityName": "Pleasanton" } }
+{ "create_at": datetime("2016-02-28T00:00:03.000Z"), "id": 703852149311025153, "text": "�� https://t.co/4sZ1IU9uIx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3409458317, "name": "Mary K. Karcher ☻", "screen_name": "maky_ray", "lang": "en", "location": "North Carolina, USA", "create_at": date("2015-08-08"), "description": "you are a child of the universe, no less than the trees and the stars // #ECU", "followers_count": 284, "friends_count": 241, "statues_count": 1728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, NC", "id": "f7c78236e397acd7", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-83.436067,35.149662 -83.347114,35.220192") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37113, "countyName": "Macon", "cityID": 3724640, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852149361344513, "text": "@SoDamnTrue this quite made me rethink my relationship, he didn't really love me if he hurt me. Thank you", "in_reply_to_status": -1, "in_reply_to_user": 219049532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219049532 }}, "user": { "id": 699760123539886080, "name": "Hailey Neal", "screen_name": "HaileyNeal9", "lang": "en", "location": "Vancouver, WA", "create_at": date("2016-02-16"), "description": "Single with a gypsy soul.", "followers_count": 66, "friends_count": 301, "statues_count": 26 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852149457817600, "text": "Wind 0.1 mph W. Barometer 29.756 in, Steady. Temperature 33.9 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 8956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852149457858560, "text": "Im up��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189528231, "name": "La' Perla Figi", "screen_name": "LaMadrina____", "lang": "en", "location": "chasing waterfalls", "create_at": date("2010-09-11"), "description": "cloth cut from a differnt frabic. I am nothing like them.", "followers_count": 891, "friends_count": 208, "statues_count": 44020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852149571059712, "text": "so annoyed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864818355, "name": "crystal.", "screen_name": "crystalliann5", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "null", "followers_count": 350, "friends_count": 274, "statues_count": 6468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852150514814976, "text": "@BruceWayne_Ave I get it", "in_reply_to_status": 703851578495627265, "in_reply_to_user": 40765409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40765409 }}, "user": { "id": 153998008, "name": "Just Here", "screen_name": "UR_DSYRE", "lang": "en", "location": "null", "create_at": date("2010-06-09"), "description": "I'm a fake controversial tweeter, people just won't allow my washed up ass to tweet in peace", "followers_count": 1750, "friends_count": 1413, "statues_count": 190126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852150787428352, "text": "#DirectTraffic at N Orange Ave & E Central Blvd. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3790352,28.5421102"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DirectTraffic", "orlpol", "opd" }}, "user": { "id": 39049373, "name": "Police Calls 32801", "screen_name": "orlpol32801", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 532, "friends_count": 1, "statues_count": 100598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852151169093632, "text": "Wind 4.0 mph SSW. Barometer 29.709 in, Falling. Temperature 29.9 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852151332679681, "text": "The niggas was out tonight chile��☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2707132301, "name": "||L||", "screen_name": "__KingHAJ", "lang": "en", "location": "null", "create_at": date("2014-07-13"), "description": "5/19/14 The Day My Life Changed Forever (Sa'Haj Keeper)", "followers_count": 209, "friends_count": 331, "statues_count": 6707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852151685013505, "text": "Thank you so much babes love you ❤️❤️ https://t.co/AEaaPviOs4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342588244, "name": "kalene", "screen_name": "kaleneperez", "lang": "en", "location": "venus", "create_at": date("2011-07-26"), "description": "reality is a lovely place, but I wouldn't want to live there.", "followers_count": 1047, "friends_count": 695, "statues_count": 30824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152008024064, "text": "I want tacos w/ rice & beans ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1834548998, "name": "C Dogg", "screen_name": "__X0Kitty", "lang": "en", "location": "null", "create_at": date("2013-09-08"), "description": "♋️ PRC Gremlins ♥️️ Link Crew ♋️", "followers_count": 309, "friends_count": 422, "statues_count": 1266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152179920897, "text": "These chicks get 100 like on IG and forget they got roaches running out their cereal box. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260529457, "name": "lancaslloyd", "screen_name": "lloydcal20", "lang": "en", "location": "Lancaster, PA", "create_at": date("2011-03-03"), "description": "#ThirstyThursdays @Chameleon_Club Lancaster Pa. Tylea.Adore Feb.25TH Follow On Instagram: @Budd_320 Tickets for #PhillyInvadesLaunch http://bit.ly/yazz-launch", "followers_count": 768, "friends_count": 1891, "statues_count": 1458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, PA", "id": "0774aa7280ccd221", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-76.346653,40.006885 -76.254112,40.073016") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42071, "countyName": "Lancaster", "cityID": 4241216, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152234463232, "text": "It's my bday. I'm stoned. I'm happy �� https://t.co/00dMx0YBVK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2730533138, "name": "luc", "screen_name": "luquinh0xxx", "lang": "en", "location": "null", "create_at": date("2014-08-13"), "description": "Brazilian • UW", "followers_count": 102, "friends_count": 94, "statues_count": 1720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152368697345, "text": "@CVC541 Check!", "in_reply_to_status": 703830007190200320, "in_reply_to_user": 4222254388, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4222254388 }}, "user": { "id": 4195263914, "name": "Kirstie Fankhauser", "screen_name": "KirstieFankhau4", "lang": "en", "location": "null", "create_at": date("2015-11-10"), "description": "I live in California. l love Bernie Sanders and music with a good message!", "followers_count": 711, "friends_count": 953, "statues_count": 2713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nipomo, CA", "id": "51ca80066df113e2", "name": "Nipomo", "place_type": "city", "bounding_box": rectangle("-120.533212,35.003173 -120.447468,35.052688") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 651476, "cityName": "Nipomo" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152687493120, "text": "Wind 0.0 mph ---. Barometer 30.195 in, Steady. Temperature 36.2 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852152712613890, "text": "BBC logic video headed to YouTube tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2460812582, "name": "Dalauan Sparrow", "screen_name": "LOWTI3RGOD", "lang": "en", "location": "Lancaster ca", "create_at": date("2014-04-23"), "description": "Actor/fighting game player/gym rat", "followers_count": 2061, "friends_count": 268, "statues_count": 8032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852153228492800, "text": "Temp: 41.0°F | Humidity: 97% | Wind: --- @ 0.0 mph | Barometer: 30.27 in | Dewpoint: 40.2°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 27, "friends_count": 1, "statues_count": 165501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-02-28T00:00:04.000Z"), "id": 703852153325027328, "text": "@adrianachechik @Erika4Jordan @michellemaylene @realmarycarey @NCrystean @Karliemontana Please @VividLiveHou @adamevehouston bring 2 #HTown", "in_reply_to_status": 703850452249505792, "in_reply_to_user": 65156041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HTown" }}, "user_mentions": {{ 1219304198, 55126114, 18576552, 19201923, 26498679, 20396889, 2647036693, 228211898 }}, "user": { "id": 65156041, "name": "Paul D. Feltovich", "screen_name": "pfeltovich", "lang": "en", "location": "Houston, TX", "create_at": date("2009-08-12"), "description": "null", "followers_count": 594, "friends_count": 2456, "statues_count": 6969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852153601785856, "text": "#Battery at 525 Ross Pl. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.408024,28.55009"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Battery", "orlpol", "opd" }}, "user": { "id": 39049856, "name": "Police Calls 32805", "screen_name": "orlpol32805", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 249, "friends_count": 1, "statues_count": 112780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852153601810436, "text": "@La_poderosaa happy bday lover", "in_reply_to_status": -1, "in_reply_to_user": 536905801, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 536905801 }}, "user": { "id": 716989848, "name": "uNWorthy", "screen_name": "larsdarling", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "fl / surviving / @mournfulFL", "followers_count": 317, "friends_count": 153, "statues_count": 16749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154142855171, "text": "@evanjaquez its 3am and im stalking ur account.....why", "in_reply_to_status": -1, "in_reply_to_user": 605903420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605903420 }}, "user": { "id": 3386075955, "name": "Joia(JOY-UH)", "screen_name": "KWAYNTjoia", "lang": "en", "location": "New York", "create_at": date("2015-07-21"), "description": "im a singing grapefruit // XD // we loves it tho", "followers_count": 535, "friends_count": 1095, "statues_count": 12333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154478399493, "text": "@3rdwardAngel all y'all do is smoke n chill in the car and go to the store and buy a gar", "in_reply_to_status": 703846964006289409, "in_reply_to_user": 248478292, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248478292 }}, "user": { "id": 2961960336, "name": "josh♎️", "screen_name": "bwa_joshh", "lang": "en", "location": "null", "create_at": date("2015-01-06"), "description": "Money,Power,and Respect follow me on instagram and snapchat josh_justdoit", "followers_count": 53, "friends_count": 80, "statues_count": 550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154625208321, "text": "And I love eating", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155747462, "name": "Henderson", "screen_name": "LehendersonP", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-06-14"), "description": "null", "followers_count": 711, "friends_count": 664, "statues_count": 14079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154683961344, "text": "girls really just enjoy arguments just bc that's the only time niggas express how they really feel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2217088734, "name": "ke.", "screen_name": "keiara_og", "lang": "en", "location": "cedarhill,tx.", "create_at": date("2013-11-26"), "description": "singledab. #PVAMU19", "followers_count": 588, "friends_count": 500, "statues_count": 9678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154730057728, "text": "& no one is replying to my snaps.... is it that late???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1672599860, "name": "SHEILATEQUILA⭐️", "screen_name": "sheilsforreals", "lang": "en", "location": "null", "create_at": date("2013-08-15"), "description": "19 | Sanjo | #laxislyfe | people always leave.", "followers_count": 287, "friends_count": 213, "statues_count": 5871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852154969174017, "text": "Mental block. Ugh.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2161949196, "name": "Kate ☪", "screen_name": "kayteee55", "lang": "en", "location": "adventure awaits ➵", "create_at": date("2013-10-28"), "description": "dogs. adventures. & a good cup of coffee.", "followers_count": 288, "friends_count": 419, "statues_count": 5051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arden-Arcade, CA", "id": "b90f2a335f8565c0", "name": "Arden-Arcade", "place_type": "city", "bounding_box": rectangle("-121.423941,38.562585 -121.327437,38.645482") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602553, "cityName": "Arden-Arcade" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852155338272768, "text": "#CommercialAlarm at 6130 Edgewater Dr, Lockhart, FL 32810. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4175693,28.6121097"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CommercialAlarm", "orlpol", "opd" }}, "user": { "id": 39133076, "name": "Police Calls 32810", "screen_name": "orlpol32810", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 79, "friends_count": 1, "statues_count": 11358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852155719929856, "text": "Fuck Sleep ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151309627, "name": "Durk Ward", "screen_name": "POV_CITY", "lang": "en", "location": "Southside Jamaica ,Queens ", "create_at": date("2010-06-02"), "description": "Shot My First Gun At This Old Head Niggas Knew Than I Wasn't No B****", "followers_count": 879, "friends_count": 906, "statues_count": 17624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saratoga Springs, NY", "id": "d606af411edd5297", "name": "Saratoga Springs", "place_type": "city", "bounding_box": rectangle("-73.848875,43.020588 -73.684637,43.11327") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3665255, "cityName": "Saratoga Springs" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852155782828032, "text": "❄️❄️❄️ Like Minnesota @lilyachty ⛵️ https://t.co/srmGemThhO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 175198393 }}, "user": { "id": 2342812272, "name": "$$$BACKWØØD BENNY$$$", "screen_name": "JamezLaFlvme", "lang": "en", "location": "✨ Made In Flexico ✨", "create_at": date("2014-02-13"), "description": "• VISUAL ARTIST + CREATOR • Eye Take Pictures &' Shit . ➡️ | Photos x Videos Contact JamezLaFlame@Gmail.com | ⬅️ •#RareBoyz ✨. [ $TONEYGANG ]", "followers_count": 1967, "friends_count": 794, "statues_count": 122555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hyattsville, MD", "id": "f3f3e88ab848b86b", "name": "Hyattsville", "place_type": "city", "bounding_box": rectangle("-76.974612,38.942717 -76.936986,38.978966") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2441250, "cityName": "Hyattsville" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852155807993856, "text": "Yes I am a Sagittarius ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1353241218, "name": "ૐ❁๓αgร๓αяïร๑ℓ❁ૐ", "screen_name": "magsmarisolestr", "lang": "en", "location": "with him☝", "create_at": date("2013-04-14"), "description": "Bae... @s13_chris✨ ❃♡②ⓦⓔⓝⓣⓨ♡❃....✊ Xicana Sin Verguenza", "followers_count": 539, "friends_count": 524, "statues_count": 15202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852156332343296, "text": "Awww lol ���� https://t.co/dHeSi7X0hW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547202442, "name": "Proton John", "screen_name": "1Chocolate_Rain", "lang": "en", "location": "COS", "create_at": date("2012-04-06"), "description": "My thing is just staying postive", "followers_count": 587, "friends_count": 555, "statues_count": 32493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852156684664832, "text": "Steph Curry's talent is the only thing that all of America can agree on.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386241127, "name": "Allen", "screen_name": "allentmartin", "lang": "en", "location": "ACU", "create_at": date("2011-10-06"), "description": "to strive, to seek, to find, & not to yield", "followers_count": 738, "friends_count": 454, "statues_count": 19895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-02-28T00:00:05.000Z"), "id": 703852157141843968, "text": "#RecklessVehicle at Tradeport Dr & Express St. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3346014,28.4266453"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RecklessVehicle", "orlpol", "opd" }}, "user": { "id": 39057416, "name": "Police Calls 32827", "screen_name": "orlpol32827", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 287, "friends_count": 1, "statues_count": 32209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-02-28T00:00:06.000Z"), "id": 703852158337179648, "text": "⛳️\n\n ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 417450284, "name": "Sheorge", "screen_name": "GTSETSAKOS15", "lang": "en", "location": "StatenIsland, NY #OSU #NYY 312", "create_at": date("2011-11-20"), "description": "Why you heff to be mad? yessurr", "followers_count": 503, "friends_count": 479, "statues_count": 27317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-02-28T00:00:06.000Z"), "id": 703852158601420801, "text": "And he snoring loud af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163131909, "name": "Jordon", "screen_name": "JordonHaley", "lang": "en", "location": "null", "create_at": date("2010-07-05"), "description": "null", "followers_count": 238, "friends_count": 145, "statues_count": 9287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catonsville, MD", "id": "dd03cd2e1b3ad5fa", "name": "Catonsville", "place_type": "city", "bounding_box": rectangle("-76.794408,39.22709 -76.688592,39.294914") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2414125, "cityName": "Catonsville" } }
+{ "create_at": datetime("2016-02-28T00:00:06.000Z"), "id": 703852158739820545, "text": "Wind 11.0 mph WSW. Barometer 29.526 in, Falling. Temperature 54.8 °F. Rain today 0.00 in. Humidity 43%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214520797868033, "text": "!!!! https://t.co/6s9pgH69nP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2763537798, "name": "brooks", "screen_name": "ItsDonnieBruh", "lang": "en", "location": "Swamp Lands", "create_at": date("2014-08-24"), "description": "you wanna be a coffee table? be a coffee table. get on your hands and knees bitch. **SWAMPLANDS DREAMS WILL BE A REALITY**", "followers_count": 135, "friends_count": 67, "statues_count": 1175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Oroville, CA", "id": "cdf85de2e3b9f268", "name": "South Oroville", "place_type": "city", "bounding_box": rectangle("-121.561348,39.458049 -121.504208,39.500607") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 673178, "cityName": "South Oroville" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214521619943424, "text": "@SumeetMa It's questions like this that keep me up at night.", "in_reply_to_status": 704199943842496512, "in_reply_to_user": 17335239, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17335239 }}, "user": { "id": 24656661, "name": "Yonadav Tsuna", "screen_name": "YTsuna", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-15"), "description": "Sales Ops @Twitter. Founder, Empire Biscuit. Ex-New Yorker. Runner. Film enthusiast. Opera lover. Ex-banker. NYU grad.", "followers_count": 800, "friends_count": 1349, "statues_count": 11403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214521972445184, "text": "@Bayleef69 plz", "in_reply_to_status": 704185997202948100, "in_reply_to_user": 421289206, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 421289206 }}, "user": { "id": 421289206, "name": "Baeleef", "screen_name": "Bayleef69", "lang": "en", "location": "null", "create_at": date("2011-11-25"), "description": "Lookin' for love but not so sure.\nSpace-Time Jumper #PLF #BNF", "followers_count": 967, "friends_count": 828, "statues_count": 34234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, AL", "id": "deb349182b3f42bb", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-85.569969,32.532449 -85.413112,32.662041") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1081, "countyName": "Lee", "cityID": 103076, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214522169528321, "text": "Son las 3AM en #NYC estoy con mi amigo Jetlag yeah ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "NYC" }}, "user": { "id": 145741250, "name": "Chechu Lina", "screen_name": "_chechulina", "lang": "es", "location": "Ramos Mejía", "create_at": date("2010-05-19"), "description": "Hotelera. Quemera. Fan ❤ backstreet's back alright! ❤", "followers_count": 202, "friends_count": 575, "statues_count": 3605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214522798723072, "text": "Barberton Oh Temp:48.5°F Wind:5 mph Dir:SW Baro:Steady Rain2day:0.04in Hum:79% UV:0.0 @ 03:00 02/29/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 229, "statues_count": 114999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214523847151616, "text": "���������� https://t.co/euCo52zjoS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3096849090, "name": "celia", "screen_name": "raeeeraeee__", "lang": "en", "location": "null", "create_at": date("2015-03-18"), "description": "phs softball", "followers_count": 469, "friends_count": 495, "statues_count": 5405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-02-29T00:00:00.000Z"), "id": 704214524597932032, "text": "体力と行動力ある方だし、病気しないし、長期間留守にするの平気だし、どっちかと言えば勉強してもできる方なのでジャーナリストやってみたいけど、コミュ力が足りない", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 1359132072, "name": "ハードコア", "screen_name": "oil_of_the_goma", "lang": "ja", "location": "null", "create_at": date("2013-04-17"), "description": "https://Instagram.com/otakuhouseofficial/", "followers_count": 468, "friends_count": 369, "statues_count": 25869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214525067730944, "text": "Kelly Rowland feat. Lil Wayne x Motivation https://t.co/9Nt8WUkBds", "in_reply_to_status": 704210778476441600, "in_reply_to_user": 296565555, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296565555, "name": "#LAZYBITCH", "screen_name": "BeyoncesHubby", "lang": "en", "location": "DC Chillin, PG Chillin", "create_at": date("2011-05-10"), "description": "I'm just a regular degular shmegular guy from the 3-0-1. You want dis ghetto dick or dat ova da counter shit? #BlackLivesMatter", "followers_count": 9056, "friends_count": 1329, "statues_count": 344215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxon Hill, MD", "id": "003b0a6b6b3eca0e", "name": "Oxon Hill", "place_type": "city", "bounding_box": rectangle("-77.001308,38.760754 -76.93722,38.83211") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2459500, "cityName": "Oxon Hill" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214525155803136, "text": "��™ https://t.co/7RrhdlpH9E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 406624177, "name": "♣♠NaeNae™♣♠", "screen_name": "Heartless23_", "lang": "en", "location": "Sacramento, CA", "create_at": date("2011-11-06"), "description": "#Sagittarius ♐ #Broncos #Lakers #Raiders ..\n18 ......class of 2016 ............ #FreeWiZz", "followers_count": 3332, "friends_count": 3174, "statues_count": 3580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214525264863232, "text": "You heard the man.. Start sending https://t.co/dGVSJefbDF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455471085, "name": "¿Sean?", "screen_name": "SeanIsDad", "lang": "en", "location": "Washington", "create_at": date("2012-01-04"), "description": "Blackout boyz // Active Yelp User", "followers_count": 846, "friends_count": 300, "statues_count": 48504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salmon Creek, WA", "id": "01d1bc4e2f2710a8", "name": "Salmon Creek", "place_type": "city", "bounding_box": rectangle("-122.713017,45.692689 -122.620464,45.765171") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5361000, "cityName": "Salmon Creek" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214525445279744, "text": "https://t.co/OH71Q9dLyI https://t.co/e4Tg67gP0v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.6578039,31.2754365"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3430920274, "name": "Karla Hinson", "screen_name": "karlacurlshins5", "lang": "en", "location": "Georgia", "create_at": date("2015-08-18"), "description": "null", "followers_count": 182, "friends_count": 240, "statues_count": 860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13071, "countyName": "Colquitt" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214526116298752, "text": "I love my mom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3151787924, "name": "Grams", "screen_name": "sabinaaaa11", "lang": "en", "location": "California, USA", "create_at": date("2015-04-12"), "description": "I do theatre and go to RHS that's all you need to know", "followers_count": 353, "friends_count": 449, "statues_count": 2683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214526539894784, "text": "@RaymondsRad nvm it's Fetty Wap not stay home", "in_reply_to_status": 704213666548166656, "in_reply_to_user": 313080511, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313080511 }}, "user": { "id": 313080511, "name": "Yung Blood", "screen_name": "RaymondsRad", "lang": "en", "location": "The Dirt ", "create_at": date("2011-06-07"), "description": "One cold night in October", "followers_count": 744, "friends_count": 431, "statues_count": 53302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214526829338625, "text": "I'm so excited about my project/video �� Can't wait to share it with y'all!! #ccs10b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ccs10b" }}, "user": { "id": 602426327, "name": "Lupe", "screen_name": "_MariaS192", "lang": "en", "location": "null", "create_at": date("2012-06-07"), "description": "cuando crezca quiero ser como mi mamá", "followers_count": 380, "friends_count": 370, "statues_count": 26395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214527005614080, "text": "Wind 1.0 mph NW. Barometer 30.013 in, Rising. Temperature 43.5 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214527076765696, "text": "É tempo de valorizar de verdade as pessoas q estão na nossa vida.Um dia podem ja nao estar perto de nós. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 268195348, "name": "Ary Cosme", "screen_name": "ary_cosme", "lang": "en", "location": "Tulsa, OK", "create_at": date("2011-03-18"), "description": "Always do your best. What you plant now, you will harvest later.", "followers_count": 283, "friends_count": 178, "statues_count": 2443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214527869476864, "text": "She shallow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300317070, "name": "FondioLeo", "screen_name": "Avenger136", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "They don't want you to see it....", "followers_count": 91, "friends_count": 178, "statues_count": 1674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528423124992, "text": "@ebbtideapp Tide in Inlet, Delaware 02/29/2016\n Low 6:28am 0.5\nHigh 12:41pm 2.2\n Low 6:26pm 0.3\nHigh 1:18am 2.4\n Low 7:23am 0.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.07,38.61"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 38, "friends_count": 1, "statues_count": 11776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528469282816, "text": "I'm feeling this girl.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1228306058, "name": "Hugo Alonso", "screen_name": "Hlopez14Hugo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-02-28"), "description": "Philippians 4:13 Pierce ✏️.", "followers_count": 295, "friends_count": 149, "statues_count": 3457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528603475968, "text": "Annnnd everybody fell asleep at once ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339409857, "name": "Jeeves", "screen_name": "LaBearIs", "lang": "en", "location": "United States", "create_at": date("2011-07-20"), "description": "Dont come over here being irrational", "followers_count": 728, "friends_count": 993, "statues_count": 86799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528649617408, "text": "If you listen to Saosin and Mos Def you understand my musical variety", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 586685768, "name": "CLtwïtterActivist®", "screen_name": "willgriffinnig", "lang": "en", "location": "your dm", "create_at": date("2012-05-21"), "description": "Don't let your dreams be memes/cartoon connoisseur/I peaked in high school.", "followers_count": 1160, "friends_count": 336, "statues_count": 31671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528670629888, "text": "My brother @_websterX had Minneapolis turnt tonight! #JustAKidFromMilwaukee https://t.co/hehZ89h0TY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JustAKidFromMilwaukee" }}, "user_mentions": {{ 335134816 }}, "user": { "id": 2733830311, "name": "Isiah Allen", "screen_name": "isiahallen43", "lang": "en", "location": "worldwide, milwaukee! ", "create_at": date("2014-08-14"), "description": "I'm on the pursuit of happiness. #JustAKidFromMilwaukee #TravelGawd", "followers_count": 214, "friends_count": 595, "statues_count": 1851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-02-29T00:00:01.000Z"), "id": 704214528792330240, "text": "02/29@03:00 - Temp 48.7F, WC 48.7F. Wind 1.0mph WSW, Gust 2.0mph. Bar 29.704in, Falling. Rain 0.00in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214529199071233, "text": "#SupportOriginMelissa 54.3°F Wind:3.8mph Pressure: 29.76hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers likely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214529199251460, "text": "I'm at Metro Vista in Orlando, FL https://t.co/bdUNm57H1o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.47316582,28.53692675"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44133645, "name": "Finz72", "screen_name": "Finz72", "lang": "en", "location": "Orlando, Fl", "create_at": date("2009-06-02"), "description": "null", "followers_count": 29, "friends_count": 133, "statues_count": 103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlovista, FL", "id": "48ddd956df593fc0", "name": "Orlovista", "place_type": "city", "bounding_box": rectangle("-81.475973,28.53214 -81.446957,28.55261") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253100, "cityName": "Orlovista" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214529463492612, "text": "3h 5m wait time at University of Alberta Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.520907,53.520659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 361, "friends_count": 884, "statues_count": 2859 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214529790631936, "text": "Temp: 53.8°F Wind:0.0mph Pressure: 30.130hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214529916284929, "text": "Smh https://t.co/gO1Pk9buBJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 323880821, "name": "W$P", "screen_name": "Khristhehomie", "lang": "en", "location": "Inglewood, CA", "create_at": date("2011-06-25"), "description": "null", "followers_count": 843, "friends_count": 554, "statues_count": 21150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214530134405120, "text": "56.7F (Feels: 56.7F) - Humidity: 99% - Wind: 1.6mph E - Gust: 2.2mph - Pressure: 1030.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214530629435392, "text": "Good evening #NorthDakota! Sun will set in 30 minutes (local time 18:30) #Sunset #Photo #Photography [29.02.2016]", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.29917,47.11639"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NorthDakota", "Sunset", "Photo", "Photography" }}, "user": { "id": 4567387397, "name": "PhotoSchedule", "screen_name": "PhotoSchedule", "lang": "en", "location": "Finland", "create_at": date("2015-12-22"), "description": "We bring up sunrise and sunset around the world so you can get your camera ready or just enjoy the view. Developed by @JanneHonkonen / @TheOwlkeep", "followers_count": 184, "friends_count": 3, "statues_count": 55869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Dakota, USA", "id": "7d893ca2441b0c21", "name": "North Dakota", "place_type": "admin", "bounding_box": rectangle("-104.048915,45.935021 -96.554508,49.000693") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38065, "countyName": "Oliver", "cityID": 3813180, "cityName": "Center" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214531107491840, "text": "@maheaz4196 that's what my friend said! I never knew that hahah", "in_reply_to_status": -1, "in_reply_to_user": 2900227464, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2900227464 }}, "user": { "id": 988370448, "name": "Taylor Mendiola", "screen_name": "_taayrose", "lang": "en", "location": "hawaiii", "create_at": date("2012-12-04"), "description": "null", "followers_count": 243, "friends_count": 195, "statues_count": 8093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipio Acres, HI", "id": "44cec4558c57418e", "name": "Waipio Acres", "place_type": "city", "bounding_box": rectangle("-158.028507,21.458343 -158.00428,21.489971") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1580000, "cityName": "Waipio Acres" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214531564707840, "text": "Night ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551916212, "name": "King N ✊", "screen_name": "NBrotherns", "lang": "en", "location": "Moreno Valley, CA", "create_at": date("2012-04-12"), "description": "They Never Told Me When You Get The Crown, It's Going To Take Some Getting Use To ✊", "followers_count": 756, "friends_count": 680, "statues_count": 31406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214531636121600, "text": "Wind 5.0 mph W. Barometer 29.846 in, Rising quickly. Temperature 47.8 °F. Rain today 0.20 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 17998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214531719888897, "text": "Play with fire, you get burned ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622775193, "name": "Jeff Salonga", "screen_name": "JeffgotBARZ", "lang": "en", "location": "null", "create_at": date("2012-06-30"), "description": "null", "followers_count": 104, "friends_count": 89, "statues_count": 6326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214532449824768, "text": "Wind 2.2 mph S. Barometer 29.51 in, Falling. Temperature 46.6 °F. Rain today 0.00 in. Humidity 52%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 112, "statues_count": 157880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214532512583680, "text": "Always been a player but now I catch my self slipping catching feelings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144343413, "name": "Pastor The Shooter", "screen_name": "iSeeYouBish", "lang": "en", "location": "Raleigh, NC", "create_at": date("2010-05-15"), "description": "Pville..Kinston..Raleigh ✌️5⃣ ✌️ & 9⃣1⃣9⃣", "followers_count": 1480, "friends_count": 748, "statues_count": 210721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apex, NC", "id": "d36f171e278ac981", "name": "Apex", "place_type": "city", "bounding_box": rectangle("-78.915463,35.678557 -78.794693,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214532743245824, "text": "@berlinbowie I love this gif so much", "in_reply_to_status": 704213033078272000, "in_reply_to_user": 453465877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453465877 }}, "user": { "id": 463995420, "name": "boy loves me", "screen_name": "adultsupermart", "lang": "en", "location": "California, USA", "create_at": date("2012-01-14"), "description": "I'm not asking you to forget, I'm only asking to be forgiven. Keep ur memories good or bad in your head, I don't care about that it's your heart I wanna live in", "followers_count": 667, "friends_count": 194, "statues_count": 11585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214532793565188, "text": "On god https://t.co/7yz01pmmVZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193952601, "name": "FEB 28TH", "screen_name": "ObeyMarcuss", "lang": "en", "location": "MKE ✈️ HOU ", "create_at": date("2010-09-22"), "description": "pvamu | just chillin", "followers_count": 4927, "friends_count": 2410, "statues_count": 60543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-02-29T00:00:02.000Z"), "id": 704214532990836736, "text": "Can you find Brentwood on the map? Just try it at https://t.co/49S4YFaeTR #Brentwood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.2462,40.7812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Brentwood" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2624835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, NY", "id": "5fe049183a425395", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-73.295325,40.754764 -73.209269,40.807341") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608026, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214533066215425, "text": "he said not to subtweet so I'll @@@@ you instead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551124504, "name": "McKenzy ✨", "screen_name": "KenzyGutierrez", "lang": "en", "location": "Mesa, AZ", "create_at": date("2012-04-11"), "description": "null", "followers_count": 592, "friends_count": 643, "statues_count": 14102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214533481431040, "text": "https://t.co/2VSKJw0J2A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2836264831, "name": "isabella", "screen_name": "amandaisebella", "lang": "en", "location": "Aries", "create_at": date("2014-09-30"), "description": "bio? more like bye yo #RDA", "followers_count": 623, "friends_count": 1183, "statues_count": 7705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214533661806592, "text": "I hate those ' I'm always here for you ' ppl cause they're never ever around when you really need somebody.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4471765812, "name": "Stephany✨", "screen_name": "stephanythebae", "lang": "en", "location": "somewhere in Oregon", "create_at": date("2015-12-05"), "description": "b l e s s e d | 17", "followers_count": 166, "friends_count": 155, "statues_count": 2023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214534458732545, "text": "@_ThirstTrap_ this is such a trash account and I should know because I'm a trash account", "in_reply_to_status": -1, "in_reply_to_user": 704202107130253312, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 704202107130253312 }}, "user": { "id": 1395760015, "name": "cole.", "screen_name": "thelatinoosux", "lang": "en", "location": "SUPREAM, AZ", "create_at": date("2013-05-01"), "description": "null", "followers_count": 1304, "friends_count": 764, "statues_count": 45623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214534626676737, "text": "Travelling to Brentwood or just twittering about Brentwood? https://t.co/49S4YFaeTR #Brentwood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.2462,40.7812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Brentwood" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1031, "friends_count": 311, "statues_count": 2624836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, NY", "id": "5fe049183a425395", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-73.295325,40.754764 -73.209269,40.807341") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608026, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214534995611648, "text": "@annalyseeee_ use too work there till I got injured", "in_reply_to_status": 704210407867568128, "in_reply_to_user": 1430945730, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1430945730 }}, "user": { "id": 4502512214, "name": "C Mysael", "screen_name": "cesarpedals", "lang": "en", "location": "Santa Cruz, CA", "create_at": date("2015-12-08"), "description": "S A D B O Y H O T B O Y", "followers_count": 69, "friends_count": 158, "statues_count": 949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214535452782592, "text": "Wind 5.0 mph S. Barometer 29.505 in, Falling Rapidly. Temperature 48.8 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214535771586560, "text": "Don't fuckin text me and tell me to like and comment on yo shit ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470324463, "name": "baby j", "screen_name": "JahzeelJurado", "lang": "en", "location": "West Coast", "create_at": date("2012-01-21"), "description": "god is good fam.", "followers_count": 667, "friends_count": 287, "statues_count": 2653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214535775727616, "text": "@Ookayx @YELLOWCLAW @OliverHeldens @snailmusic @Firebeatz Goals ��", "in_reply_to_status": 704213167279198208, "in_reply_to_user": 821119939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 821119939, 235542752, 256569627, 504890998, 32351549 }}, "user": { "id": 3354150372, "name": "ODIR", "screen_name": "dj_odir", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-08-26"), "description": "Out now ! LA PEPA https://m.soundcloud.com/odirdj/la-pepa. Whatever it is, it sounds Awful !", "followers_count": 55, "friends_count": 148, "statues_count": 178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214535922520068, "text": "Watch me forget.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2606016648, "name": "◡̈", "screen_name": "k_pvtta", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2014-07-05"), "description": "laughing at you.", "followers_count": 158, "friends_count": 148, "statues_count": 7518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-02-29T00:00:03.000Z"), "id": 704214536266473473, "text": "Very True https://t.co/u6MbbigaGJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101950174, "name": "$300 An Hour", "screen_name": "KingKillaQuin", "lang": "en", "location": "Clutch City", "create_at": date("2010-01-04"), "description": "#HomeComingKingTour✈️ | The VOICE Of TEXAS | #TxSU | Event & Concert Host | For All Inquiries 832-964-6426 | ΦΒΣ | ΒΔ | TailLife5", "followers_count": 2932, "friends_count": 1320, "statues_count": 123484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214537407328256, "text": "@MalikBusari @KinggChip �������� it's a joke, relax", "in_reply_to_status": 704213826158272512, "in_reply_to_user": 1408929482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1408929482, 377714550 }}, "user": { "id": 783415656, "name": "chinonye .", "screen_name": "its_catherinee", "lang": "en", "location": "htx | denton", "create_at": date("2012-08-26"), "description": "nigerian . | teen spirit .", "followers_count": 1742, "friends_count": 996, "statues_count": 37220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214537524752385, "text": "swear to god if I get yelled at for being awake and doing homework and just wanting a bowl of cereal I'm throwing it at the window", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172307372, "name": "KyotE", "screen_name": "LsdZepppelin", "lang": "en", "location": "null", "create_at": date("2010-07-29"), "description": "art pimp, dealer, hoe, 90's hip hop geek, peace frog", "followers_count": 2093, "friends_count": 628, "statues_count": 114312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214537558360064, "text": "Im on juan tonight mannn. Its the chocolate bar and pineapple gfuel i had before i played", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2150081454, "name": "PndaKalvin", "screen_name": "SenpaiKrizBe", "lang": "en", "location": "PSN: HokageKrizBe", "create_at": date("2013-10-22"), "description": "KeyBlade Master/Hokage/Senpai for @PndaGaming #KrizBaes \nhttp://Twitch.tv/HokageKrizBe", "followers_count": 1551, "friends_count": 984, "statues_count": 24579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheeling, IL", "id": "f2da3efc48696715", "name": "Wheeling", "place_type": "city", "bounding_box": rectangle("-87.961117,42.094938 -87.886833,42.154943") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1781087, "cityName": "Wheeling" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214538107883521, "text": "Wind 2.9 mph SSE. Barometer 29.427 in, Falling Rapidly. Temperature 49.5 °F. Rain today 0.00 in. Humidity 52%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 8978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214538271518720, "text": "@WrestlingInc i love yall article", "in_reply_to_status": -1, "in_reply_to_user": 34668551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34668551 }}, "user": { "id": 1431521558, "name": "Eric Melendez Battle", "screen_name": "ericbattle5th", "lang": "en", "location": "BLADENSBURG ", "create_at": date("2013-05-15"), "description": "Eric Battle is a huge wrestling fan enjoys playing video games n watching anime including Dragon Ball Z. He's on facebook google I.g youtube n twitter", "followers_count": 638, "friends_count": 5002, "statues_count": 2616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Spring, MD", "id": "6417871953fa5e86", "name": "Silver Spring", "place_type": "city", "bounding_box": rectangle("-77.064086,38.979735 -76.97162,39.036964") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2472450, "cityName": "Silver Spring" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214538325983233, "text": "A fun time thecomedystore listening to #BedtimeStories. Please Sign me up, Kyle, to share one… https://t.co/yPX5RrDdXx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.37394,34.0952"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BedtimeStories" }}, "user": { "id": 19836394, "name": "Sarah G", "screen_name": "SarahGraham69", "lang": "en", "location": "UK & LA", "create_at": date("2009-01-31"), "description": "CEO Sarah Graham Solutions #Sober 12.12.01 British #intersex. #RainbowList #2 'most influential' #LGBTi. Scotty #Beagle's mum. @AddictionExpert & stand-up comic", "followers_count": 2740, "friends_count": 2761, "statues_count": 11032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214540062433281, "text": "My mind is saying fuck you, keep overthinking your not getting any sleep tonight, good luck tomorrow. ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1122896774, "name": "Alexandria Schneck", "screen_name": "AllieSchneck", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "wine please.", "followers_count": 656, "friends_count": 403, "statues_count": 6046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214540712435712, "text": "\"I'm awake\" \n*snap*\n\"getting dress for work\"\n*snap*\n\"In the car\"\n*snap*\n\"Getting off the car\"\n*snap*\n\"At work\"\n*snap*\n\"Working\"\n*snap*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296535604, "name": "Bella", "screen_name": "Barbiegurlbella", "lang": "en", "location": "null", "create_at": date("2011-05-10"), "description": "We are each our own devil, and we make this world our hell. -Oscar Wilde", "followers_count": 401, "friends_count": 318, "statues_count": 26690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patterson, CA", "id": "bcb6b4eebbf9b55c", "name": "Patterson", "place_type": "city", "bounding_box": rectangle("-121.172793,37.453362 -121.098968,37.517058") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 656112, "cityName": "Patterson" } }
+{ "create_at": datetime("2016-02-29T00:00:04.000Z"), "id": 704214541014437888, "text": "Pretty cool DIY #lightbox for video but lot of work. https://t.co/q7iDoEe1xE #youtube", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lightbox", "youtube" }}, "user": { "id": 33429714, "name": "John ", "screen_name": "rhcp011235", "lang": "en", "location": "Shenzhen", "create_at": date("2009-04-19"), "description": "Owner of http://t.co/7MvsPTfBkG http://t.co/HgXPFJBzBZ #android #ios #osx iMessage: rhcp011235@me.com kik: rhcp011235 google+: http://t.co/FJYAQowvLy", "followers_count": 2204, "friends_count": 766, "statues_count": 67945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214541710843904, "text": "\"what did we do to deserve dogs\" we fucking killed a bunch of them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24387971, "name": "+", "screen_name": "plussable", "lang": "en", "location": "chicago", "create_at": date("2009-03-14"), "description": "+ aka Plus Sign. God MC, Anarchist Pop Star! Seventeen albums n runnin coms@tenderdiscovery.com", "followers_count": 915, "friends_count": 910, "statues_count": 52452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214541794586625, "text": "@YUNGSWAGMANTHA @kourtneykardash they need to get over tht shit ain't realize that diabetes is no joke,Chyna is doin something they couldn't", "in_reply_to_status": 704214139774722048, "in_reply_to_user": 1628595222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1628595222, 23617610 }}, "user": { "id": 522994306, "name": "zitlalhy⭐️", "screen_name": "zeeeezeeee_", "lang": "en", "location": "null", "create_at": date("2012-03-13"), "description": "Houston. 20. Graduate Nurse.", "followers_count": 1293, "friends_count": 526, "statues_count": 52018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214542557904896, "text": "اللهم طهر جسد عمي سليمان من الالم وابعد عنه متاعب الدنيا يا رب", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2817615980, "name": "شيخه", "screen_name": "shralsulaiman", "lang": "en", "location": "KW|LA⛅️", "create_at": date("2014-09-18"), "description": "اللهم استرني فوق الارض و تحت الارض و يوم العرض عليك. اللهم ارحم جدي وجدتي واغفر لهم♡", "followers_count": 152, "friends_count": 190, "statues_count": 17272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214543052861440, "text": "I Could really use a massage right about now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1206996193, "name": "Logan L-W", "screen_name": "blackmomba0514", "lang": "en", "location": "null", "create_at": date("2013-02-21"), "description": "null", "followers_count": 199, "friends_count": 242, "statues_count": 4232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214543174533121, "text": "Shit is fire cuz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842904577, "name": "KayyBee", "screen_name": "renblvnco", "lang": "en", "location": "SCREWSTONE, TX ", "create_at": date("2012-09-23"), "description": "null", "followers_count": 277, "friends_count": 150, "statues_count": 10520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214543426314240, "text": "مثل م اهتمينا نقدر نهمل ، ومثل م تعودنا نقدر نتخلى����!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 4143432803, "name": "♡Eng.smyer alhajri⚓️", "screen_name": "smyerq8", "lang": "en", "location": "Los Angeles, CA ,K'278", "create_at": date("2015-11-08"), "description": "Snapchat:Smyerq820 http://ask.fm/Smyerq8", "followers_count": 514, "friends_count": 1524, "statues_count": 1055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214543560417280, "text": "Wind 9.0 mph WNW. Barometer 29.737 in, Rising Rapidly. Temperature 38.8 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 382, "friends_count": 291, "statues_count": 8636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214544411815937, "text": "1.9 magnitude #earthquake. 5 km from Volcano, HI, #UnitedStates https://t.co/J6s52jWz3Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-155.266,19.404"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 65364, "friends_count": 10, "statues_count": 97341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaii, USA", "id": "9dafd05b1158873b", "name": "Hawaii", "place_type": "admin", "bounding_box": rectangle("-178.443593,18.86546 -154.755792,28.517269") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214544722350080, "text": "Instagram been wack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418332255, "name": "car fox", "screen_name": "ror925", "lang": "en", "location": "first reply", "create_at": date("2011-11-21"), "description": "and if ya don't know now you are aware of my opinion of that topic, nigga\n#SitAroundAndWaitToBeOffendedTwitter", "followers_count": 984, "friends_count": 865, "statues_count": 39823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214545347190784, "text": "Don't let people change you, be you!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 202019535, "name": "SpanishShawwty", "screen_name": "_A6696", "lang": "en", "location": "Westest of Coasts", "create_at": date("2010-10-12"), "description": "Alejandra, sup bitch✨", "followers_count": 425, "friends_count": 387, "statues_count": 10084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:05.000Z"), "id": 704214545418444800, "text": "https://t.co/HWl0Gx0gSd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 227901767, "name": "latrommi.", "screen_name": "salthetaco", "lang": "en", "location": "AZ", "create_at": date("2010-12-17"), "description": "lost between reality & imagination", "followers_count": 482, "friends_count": 295, "statues_count": 26729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214546915799048, "text": "@ashleycookcox fix her", "in_reply_to_status": 704214383585419264, "in_reply_to_user": 2269353579, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2269353579 }}, "user": { "id": 845985205, "name": "Lexi", "screen_name": "LexiTubbs", "lang": "en", "location": "Turlock, CA", "create_at": date("2012-09-25"), "description": "| NNU Basketball & Track | IG: lexi_moze | ❤️ •BeautyInTheStruggle• CA➰ID", "followers_count": 656, "friends_count": 652, "statues_count": 10808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nampa, ID", "id": "7325f3442fd87621", "name": "Nampa", "place_type": "city", "bounding_box": rectangle("-116.637186,43.523489 -116.502031,43.634335") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16027, "countyName": "Canyon", "cityID": 1656260, "cityName": "Nampa" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214547444461568, "text": "Wind 1.0 mph NNE. Barometer 29.609 in, Falling Rapidly. Temperature 27.7 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214547599503360, "text": "@tbfhnia @_jaiimii *cough cough* gold digger", "in_reply_to_status": 655859392114327554, "in_reply_to_user": 780534732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 780534732, 531745900 }}, "user": { "id": 2953035409, "name": "Eileen", "screen_name": "paredeseileen", "lang": "en", "location": "Pico Rivera, CA", "create_at": date("2014-12-30"), "description": "ERHS|| Senior|| Chris❤️", "followers_count": 253, "friends_count": 243, "statues_count": 6019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214547758915586, "text": "Leap year bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594678760, "name": "♡", "screen_name": "yooniggaamy", "lang": "en", "location": "null", "create_at": date("2012-05-30"), "description": "raul garcia♡", "followers_count": 326, "friends_count": 207, "statues_count": 5201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, CA", "id": "b2fda7385aa963dd", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-118.317776,33.930952 -118.291549,33.959516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684592, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548291547136, "text": "We're #hiring! Read about our latest #job opening here: IT Applications - SAP - https://t.co/ob01FJmDLH #Lexington, KY #IT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.4951359,38.0317136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Lexington", "IT" }}, "user": { "id": 26266228, "name": "Lexmark Careers", "screen_name": "lexmarkcareers", "lang": "en", "location": "Lexington, KY", "create_at": date("2009-03-24"), "description": "Careers at Lexmark -- Open the possibilities", "followers_count": 1684, "friends_count": 1142, "statues_count": 572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21067, "countyName": "Fayette", "cityID": 2146027, "cityName": "Lexington-Fayette" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548392202240, "text": "Wind 4.0 mph WSW. Barometer 29.915 in, Falling slowly. Temperature 32.7 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548400762881, "text": "Wth did i just step on ���� https://t.co/4YhhorUalb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41873734, "name": "Lord Blaze", "screen_name": "ThatKiddDeonRaw", "lang": "en", "location": "null", "create_at": date("2009-05-22"), "description": "615✈407 | #LXRDFAM | R.I.P. Dianna |", "followers_count": 4854, "friends_count": 3925, "statues_count": 11869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melbourne Beach, FL", "id": "74a9b3383262e5de", "name": "Melbourne Beach", "place_type": "city", "bounding_box": rectangle("-80.572714,28.053161 -80.551389,28.078255") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1244000, "cityName": "Melbourne Beach" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548429950976, "text": "@nanahensley he's scared of me now! Whenever I walk past the house he runs away. ��", "in_reply_to_status": 704214313288888320, "in_reply_to_user": 148599467, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148599467 }}, "user": { "id": 456530404, "name": "RaShan", "screen_name": "ThatDudeShan", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-01-06"), "description": "The San Francisco Giants won the World Series on my birthday. 10-29-14 #RIPDad 11-18-15", "followers_count": 1489, "friends_count": 762, "statues_count": 148649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548568481794, "text": "@ClareElaine14 like true nerds do... get that paper...lol", "in_reply_to_status": 704206708831342592, "in_reply_to_user": 1537694894, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1537694894 }}, "user": { "id": 323841356, "name": "Yoku Masaki™", "screen_name": "YokuMasaki", "lang": "en", "location": "Gotham City to Planet Jurai", "create_at": date("2011-06-25"), "description": "Photographer/YouTuber/Musician ^S^ One Day I Put on Cat Ears and Weird Shit Happens. Geek of Many Talents. Founder of #JuraiMedia. #DirtyDice #OtakuGang", "followers_count": 1177, "friends_count": 2150, "statues_count": 87225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548656435200, "text": "Interested in a #Sales #job near #CranberryTownship, PA? This could be a great fit: https://t.co/jjUlmVVHUI #Hiring https://t.co/w7MuFPqQXC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.119951,40.708957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "CranberryTownship", "Hiring" }}, "user": { "id": 455971331, "name": "Continuum", "screen_name": "FollowContinuum", "lang": "en", "location": "Boston, MA", "create_at": date("2012-01-05"), "description": "A unified managed services platform that enables MSPs to profitably backup, monitor, troubleshoot and maintain IT environments.", "followers_count": 2670, "friends_count": 2032, "statues_count": 11092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cranberry, PA", "id": "01b347e54132a78d", "name": "Cranberry", "place_type": "city", "bounding_box": rectangle("-80.160831,40.640453 -79.995749,40.776081") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler" } }
+{ "create_at": datetime("2016-02-29T00:00:06.000Z"), "id": 704214548710993920, "text": "sooo insecure hahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2808987150, "name": "Jassy", "screen_name": "Jasmine_x00", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "est;12516@7;43am ❤️", "followers_count": 287, "friends_count": 443, "statues_count": 10444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214550762000384, "text": "2/29/2016 - 02:00\nTemp: 48.0F \nHum: 95%\nWind: 0.0 mph\nBaro: 30.066in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 50409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214551223349248, "text": "We're #hiring! Click to apply: Call Center Sales Representative - Part Time - SiriusXM - https://t.co/had0SZDKg6 https://t.co/pTlItk4P4p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2361744,39.8998074"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 320, "friends_count": 215, "statues_count": 2013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianapolis, IN", "id": "018929347840059e", "name": "Indianapolis", "place_type": "city", "bounding_box": rectangle("-86.348441,39.631677 -85.937404,39.927448") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1836003, "cityName": "Indianapolis city (balance)" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214552183836672, "text": "Winn Dixie: Seafood Associate (#HOOVER, AL) https://t.co/IQQY2HJgCr #Retail #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.6951827,33.4166785"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HOOVER", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 529, "friends_count": 251, "statues_count": 17427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoover, AL", "id": "56c9f474a934e9fe", "name": "Hoover", "place_type": "city", "bounding_box": rectangle("-86.912935,33.294784 -86.607247,33.476326") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 135896, "cityName": "Hoover" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214552544579585, "text": "@trancetherapy1 wait what i stayed up to watch you damn.... everything alright though?", "in_reply_to_status": 704198363520978944, "in_reply_to_user": 2238745693, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2238745693 }}, "user": { "id": 562972038, "name": "Vitalitycros", "screen_name": "Vitalitycross", "lang": "en", "location": "Glendale, CA", "create_at": date("2012-04-25"), "description": "Heroes of the Storm Rank 1 HL player and Rank 2 TL", "followers_count": 89, "friends_count": 145, "statues_count": 2059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214552905256961, "text": "daas my sooong !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2471864360, "name": "miina.", "screen_name": "Minawescoast", "lang": "en", "location": "null", "create_at": date("2014-04-30"), "description": "leave em on that need to know .", "followers_count": 434, "friends_count": 278, "statues_count": 11416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214552930447360, "text": "niggas love to overhype their sex", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46942102, "name": "kitty", "screen_name": "_edizzzzzzzle", "lang": "en", "location": "heaven ", "create_at": date("2009-06-13"), "description": "supa chillin", "followers_count": 799, "friends_count": 432, "statues_count": 14830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214552959791104, "text": "�� honestly?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260497487, "name": "BettyLou", "screen_name": "BoottyLou", "lang": "en", "location": "null", "create_at": date("2011-03-03"), "description": "Independent. Confident. Courageous. NAU pre-med ΔΔΔ", "followers_count": 491, "friends_count": 251, "statues_count": 11881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214553043845120, "text": "@GulabJamunHigh hahah ✌��️", "in_reply_to_status": 704214074146594816, "in_reply_to_user": 2298067076, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 2298067076 }}, "user": { "id": 133879137, "name": "Osama Bin Liaqat Mir", "screen_name": "oblmir", "lang": "en", "location": "Islamabad/Connecticut", "create_at": date("2010-04-16"), "description": "A power engineer, an airplane geek & an armchair cricket expert who jumps at slightest opportunity to travel. Pakistan Zindabad. IG/SC: oblmir #NeverForget144", "followers_count": 2920, "friends_count": 773, "statues_count": 71254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CT", "id": "1cfa9ac27a1e93bd", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-73.502636,41.055206 -73.379243,41.257357") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 955990, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214553643524096, "text": "������������������ https://t.co/t16mxgyT2O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3111024788, "name": "❤", "screen_name": "Taebarnes120697", "lang": "en", "location": "null", "create_at": date("2015-03-27"), "description": "LongLiveeMoosee❤", "followers_count": 359, "friends_count": 301, "statues_count": 5987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214553677025280, "text": "u ever burn a candle and then get too lazy to blow it out?? this is wat i get for wanting my room to smell like vanilla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195557131, "name": "SOSO", "screen_name": "hellodaanielle", "lang": "en", "location": "SD,CA ", "create_at": date("2010-09-26"), "description": "typa way", "followers_count": 210, "friends_count": 161, "statues_count": 1333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214553828024320, "text": "And here comes the silence.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1274008182, "name": "Karimel", "screen_name": "karieeexo", "lang": "en", "location": "null", "create_at": date("2013-03-16"), "description": "always be impeccable with your word", "followers_count": 169, "friends_count": 143, "statues_count": 8244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaiian Gardens, CA", "id": "bbdaba1428d18b5c", "name": "Hawaiian Gardens", "place_type": "city", "bounding_box": rectangle("-118.08284,33.822551 -118.061795,33.838733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632506, "cityName": "Hawaiian Gardens" } }
+{ "create_at": datetime("2016-02-29T00:00:07.000Z"), "id": 704214553920319488, "text": "Don't even get mad at this troll guys ? https://t.co/EzHbElOAm9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195977525, "name": "woof", "screen_name": "KiingDustin", "lang": "en", "location": "Shitsville ,Ca", "create_at": date("2010-09-27"), "description": "what y'all want", "followers_count": 614, "friends_count": 257, "statues_count": 85551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214554159374336, "text": "Temp: 31.7°F - Dew Point: 22.0° - Wind: 19.0 mph - Gust: 30.4 - Rain Today: 0.00in. - Pressure: 29.76in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214555015032832, "text": "This interview is the best thing that happened to me all day ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281081323, "name": "✾Viv✾", "screen_name": "ViviCecy", "lang": "en", "location": "null", "create_at": date("2011-04-12"), "description": "19. Sociology Major. Double minor Philosophy and Business. Buddhist. Vegan.", "followers_count": 354, "friends_count": 563, "statues_count": 10572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214555174445057, "text": "#trendstacking Trendstacking...something I original heard from @tailopez has shifted the way I think about things.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "trendstacking" }}, "user_mentions": {{ 1691181 }}, "user": { "id": 205238001, "name": "David Madison/Nate", "screen_name": "knavethemage", "lang": "en", "location": "Boulder, CO", "create_at": date("2010-10-20"), "description": "ENFP Marine Was Foster Kid Homeless 4 Yrs ago Ideas + Action = Result Raising $ 4 Dreams Pblm Solver Ppl ConXion and Empathy = Better World", "followers_count": 42, "friends_count": 24, "statues_count": 548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214555757391872, "text": "Love this! ✌️����@aNgie41610112", "in_reply_to_status": 702968631811022848, "in_reply_to_user": 783307615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 783307615 }}, "user": { "id": 210681879, "name": "The Well Pennies", "screen_name": "thewellpennies", "lang": "en", "location": "Los Angeles", "create_at": date("2010-10-31"), "description": "Music for lovers & dreamers... New album ENDLINGS available now! Tweets mostly by Bryan...", "followers_count": 1032, "friends_count": 710, "statues_count": 2031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214556592246784, "text": "I'm at @Yeowoosai in Chicago, IL https://t.co/FbkvwZkIzy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6996642,41.99518881"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150554522 }}, "user": { "id": 240193993, "name": "Gabriela Wanderlust", "screen_name": "GabrielaWndrlst", "lang": "en", "location": "Chicago", "create_at": date("2011-01-19"), "description": "Passionate travel journalist ✈️✨♐️ M.A. in I.C. A dreamer, yogui, foodie & music lover! A Chilean girl traveling the word! #GabrielaWanderlust", "followers_count": 353, "friends_count": 769, "statues_count": 1238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-02-29T00:00:08.000Z"), "id": 704214557254746112, "text": "سلم الاولويات رماني في اسفل السلم", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 3066416797, "name": "محمّد العامري", "screen_name": "_TheAmazingMk", "lang": "en", "location": "Abu Dhabi | San Diego", "create_at": date("2015-03-07"), "description": "مخلوق لجل ارضي رب البريه ، و آمانت ان اللي تسويه تلاقيه ، من حشمني فوق الراس حطيته ، والردي دعست حذايه ما تكفيه | ❤️mba", "followers_count": 657, "friends_count": 101, "statues_count": 9451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-02-29T00:00:09.000Z"), "id": 704214558764707840, "text": "Procrastination at its finest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1964793626, "name": "lupita", "screen_name": "lupita_barajas1", "lang": "en", "location": "null", "create_at": date("2013-10-16"), "description": "null", "followers_count": 330, "friends_count": 224, "statues_count": 11332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576909309444096, "text": "Disneyland next weekend ? I think he'll yes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 851791429, "name": "NINO", "screen_name": "ninno___", "lang": "en", "location": "LA TO LV", "create_at": date("2012-09-28"), "description": "Heartless", "followers_count": 1285, "friends_count": 296, "statues_count": 45498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576910156832769, "text": "Barberton Oh Temp:31.1°F Wind:1 mph Dir:ENE Baro:Falling slowly Rain2day:0.00in Hum:74% UV:0.0 @ 03:00 03/01/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 229, "statues_count": 115023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576910408458240, "text": "this couldn't be any truer https://t.co/j3x10QCKIX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467933867, "name": "pao", "screen_name": "paolatabares_", "lang": "en", "location": "null", "create_at": date("2012-01-18"), "description": "medellin born, boston living. #COL || O'B", "followers_count": 1022, "friends_count": 683, "statues_count": 25147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576910433591296, "text": "��❤️❤️❤️ https://t.co/YatS5T9eIs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 599513974, "name": "Kimmy K.", "screen_name": "JUST_Kayshon", "lang": "en", "location": "A T L A N T A ", "create_at": date("2012-06-04"), "description": "#FutureHive S|C: Kayshonnnn 4 N's", "followers_count": 1699, "friends_count": 534, "statues_count": 32017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576911062732800, "text": "https://t.co/UeVnq3cdOX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2896941082, "name": "Cherokeedass", "screen_name": "therealdassxxx", "lang": "en", "location": "All Over Da Globe", "create_at": date("2014-11-28"), "description": "YRK PRODUCTIONS ➖ http://Cherokeesfetishes.com ➖ http://Clubcherokeedass.com ➖ http://ChatStar.com/Cherokeedass ➖ Book: bookingcherokee@gmail.com", "followers_count": 20186, "friends_count": 1105, "statues_count": 13655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576911289221120, "text": "@savannuhhhleigh ��������", "in_reply_to_status": -1, "in_reply_to_user": 702218132585373696, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 702218132585373696 }}, "user": { "id": 52659765, "name": "DIRTY-DAN", "screen_name": "1Mook_", "lang": "en", "location": "☁️We$t Orlando ☁️", "create_at": date("2009-07-01"), "description": "i dont get mad, i just get paid\r\n |$| | #Zoelife | #ForeverLiveOnBJ | #FlyyLyfe", "followers_count": 2378, "friends_count": 1071, "statues_count": 160235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504956,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576911784046592, "text": "Back in my happy place :) #thepit #musicaltheaterlife #orchestra… https://t.co/GDV9KMDJlR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-152.39339893,57.79347486"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "thepit", "musicaltheaterlife", "orchestra" }}, "user": { "id": 21470048, "name": "Nicole Clark", "screen_name": "soyperiodista89", "lang": "en", "location": "Kodiak, Alaska", "create_at": date("2009-02-20"), "description": "26. Oregon native finding herself on an island in Alaska. Lives at KMXT Kodiak 100.1 FM. Player of many instruments. I've never met a drum I didn't like. :)", "followers_count": 474, "friends_count": 2173, "statues_count": 14286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kodiak, AK", "id": "8713c0c86ea50da9", "name": "Kodiak", "place_type": "city", "bounding_box": rectangle("-152.455159,57.771608 -152.325453,57.835754") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2150, "countyName": "Kodiak Island", "cityID": 240950, "cityName": "Kodiak" } }
+{ "create_at": datetime("2016-03-01T00:00:00.000Z"), "id": 704576912127979520, "text": "I can literally squeeze the sweat off my sports bra. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832517084, "name": "Alma Sánchez Lozano", "screen_name": "almaaceciliaa", "lang": "en", "location": "Long Beach", "create_at": date("2012-09-18"), "description": "Jalisco ; Guanajuato ; Me Encantan los Corridos ; LVN Graduate ; Future RN 〽️ ; & i Workk", "followers_count": 42, "friends_count": 135, "statues_count": 1136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576912643985408, "text": "im gone play my position let you catch what you been missing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700558838, "name": "kelsey", "screen_name": "ialways_Wynn", "lang": "en", "location": "Columbus, GA", "create_at": date("2012-07-17"), "description": "I'm gold, but some people prefer silver and that's okay, cause I'm still gold at the end of the day ✨ rest easy oma and grammy #freeWOP #freeDHall", "followers_count": 1174, "friends_count": 767, "statues_count": 52319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, GA", "id": "7969d4d66b90a22e", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-84.513157,33.412687 -84.40952,33.492489") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13113, "countyName": "Fayette", "cityID": 1328968, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576913054904320, "text": "happy birthday baby @justinbieber ��✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 3467401392, "name": "stephanie❣", "screen_name": "heyimssteph", "lang": "en", "location": "null", "create_at": date("2015-09-05"), "description": "null", "followers_count": 97, "friends_count": 82, "statues_count": 2032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576913327710208, "text": "@_only1ti_ where ya sista ay", "in_reply_to_status": 704575189565247488, "in_reply_to_user": 238032426, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 238032426 }}, "user": { "id": 48711873, "name": "YM", "screen_name": "Young_Mar215", "lang": "en", "location": "Hansberry", "create_at": date("2009-06-19"), "description": "null", "followers_count": 1738, "friends_count": 511, "statues_count": 43797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576913377898500, "text": "@cedrictjohnson it's 4 lol", "in_reply_to_status": 704576522078724096, "in_reply_to_user": 2604688374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2604688374 }}, "user": { "id": 3068704945, "name": "too_smooth_wallace", "screen_name": "Wallace2Kasey", "lang": "en", "location": "null", "create_at": date("2015-03-08"), "description": "Sierra college⚾️ R.I.P. Lynda", "followers_count": 335, "friends_count": 401, "statues_count": 951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576914032177152, "text": "@MarinaaNikolee lucky", "in_reply_to_status": 704576800928563201, "in_reply_to_user": 4634518206, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4634518206 }}, "user": { "id": 2880271992, "name": "Jakobe", "screen_name": "jakobe980", "lang": "en", "location": "null", "create_at": date("2014-10-27"), "description": "fuck it", "followers_count": 458, "friends_count": 212, "statues_count": 26654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576914095267840, "text": "@TrentDopeAf log out", "in_reply_to_status": 704575062804971521, "in_reply_to_user": 311154471, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311154471 }}, "user": { "id": 3428612037, "name": "Jada Alexander", "screen_name": "Jada__Jayyy", "lang": "en", "location": "Atlanta, GA", "create_at": date("2015-08-17"), "description": "New twitter..... IG: Jada_Jayyy", "followers_count": 298, "friends_count": 199, "statues_count": 1052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576914187419648, "text": "@samanthalouie starbs run?", "in_reply_to_status": 704575489621364736, "in_reply_to_user": 83987222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83987222 }}, "user": { "id": 43636609, "name": "Maddy Kotoshirodo", "screen_name": "maaaddyK", "lang": "en", "location": "Hawaii · Las Vegas", "create_at": date("2009-05-30"), "description": "time for the charm bomb to explode", "followers_count": 258, "friends_count": 267, "statues_count": 5564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576914544062464, "text": "Wind 0.0 mph ---. Barometer 30.07 in, Falling slowly. Temperature 59.2 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576914753589249, "text": "2:00 am thoughts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2800749464, "name": "c", "screen_name": "caseywilcox_", "lang": "en", "location": "Granbury, TX", "create_at": date("2014-09-09"), "description": "you bloom like a flower; slowly but beautifully", "followers_count": 606, "friends_count": 303, "statues_count": 8568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granbury, TX", "id": "013401bd2ad7a235", "name": "Granbury", "place_type": "city", "bounding_box": rectangle("-97.822813,32.39284 -97.698331,32.506334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48221, "countyName": "Hood", "cityID": 4830416, "cityName": "Granbury" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915084939264, "text": "@_reshuffled holy shit", "in_reply_to_status": 704576743361740804, "in_reply_to_user": 3229235827, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3229235827 }}, "user": { "id": 39092860, "name": "Elia Cat", "screen_name": "eggmeowth", "lang": "en", "location": "Los Angeles/Santa Rosa", "create_at": date("2009-05-10"), "description": "elia cat | 21 | Aspiring tattoo artist!! | Painter & multi media | contact for commissions/work: eliacattanese@gmail.com", "followers_count": 271, "friends_count": 307, "statues_count": 11034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915290644480, "text": "03/01@03:00 - Temp 36.2F, WC 36.2F. Wind 1.5mph SSW, Gust 2.0mph. Bar 30.078in, Rising slowly. Rain 0.00in. Hum 82%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915605028864, "text": "@legallag2 Close but, ahem, it's \"Brew Dawg Tyme\"", "in_reply_to_status": 704503142864424960, "in_reply_to_user": 4846800878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4846800878 }}, "user": { "id": 1002415759, "name": "Wimps", "screen_name": "TheseWimps", "lang": "en", "location": "Seattle", "create_at": date("2012-12-10"), "description": "wimps are a punk band on Kill Rock Stars, End of Time and Help Yourself Records. Album 'Suitcase' available now at http://killrockstars.com/suitcase", "followers_count": 1291, "friends_count": 512, "statues_count": 4834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915609235456, "text": "@b_vinson22 Morgan is home :-)", "in_reply_to_status": 704576566550900737, "in_reply_to_user": 3254953188, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3254953188 }}, "user": { "id": 292675411, "name": "bo", "screen_name": "brooke_pleak", "lang": "en", "location": "franklin•indy", "create_at": date("2011-05-03"), "description": "null", "followers_count": 1562, "friends_count": 935, "statues_count": 25988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, IN", "id": "271351c1a7aca06e", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.096551,39.441909 -86.010161,39.542581") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18081, "countyName": "Johnson", "cityID": 1825450, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915781382145, "text": "Wind 0.0 mph ---. Barometer 29.959 in, Falling. Temperature 40.5 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576915936407552, "text": "#HappyBirthdayJustinBieber ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "HappyBirthdayJustinBieber" }}, "user": { "id": 2953390285, "name": "Dessie", "screen_name": "__dessireee", "lang": "en", "location": "Hacienda Heights, CA ", "create_at": date("2014-12-31"), "description": "null", "followers_count": 175, "friends_count": 79, "statues_count": 80 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576916200620032, "text": "Legit skipped school and everything lmao https://t.co/wqTo1sUoJu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3631837878, "name": "alaska ☄", "screen_name": "whatkindofkaren", "lang": "en", "location": "null", "create_at": date("2015-09-20"), "description": "cancer crybaby and sometimes writer.", "followers_count": 373, "friends_count": 426, "statues_count": 8532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-01T00:00:01.000Z"), "id": 704576916490039296, "text": "In high school I wanted to fw her soooo bad mane...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91214712, "name": "L.$hort", "screen_name": "LS3__", "lang": "en", "location": "Grambling State University", "create_at": date("2009-11-19"), "description": "Charming with an attitude...", "followers_count": 1748, "friends_count": 881, "statues_count": 29623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grambling, LA", "id": "7659c27719e754b6", "name": "Grambling", "place_type": "city", "bounding_box": rectangle("-92.734385,32.50363 -92.669504,32.551821") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2230515, "cityName": "Grambling" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576916779507712, "text": "@ebbtideapp Tide in Wingham Island, Alaska 03/01/2016\nHigh 5:21am 8.9\n Low 12:25pm 2.4\nHigh 6:39pm 6.6\n Low 11:52pm 4.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-144.4,60.05"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 38, "friends_count": 1, "statues_count": 12049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2261, "countyName": "Valdez-Cordova" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576916800540672, "text": "#SupportOriginMelissa 41.7°F Wind:3.1mph Pressure: 29.91hpa Falling Rain Today 0.00in. Forecast: Occasional precipitation, worsening", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576916855132160, "text": "#3AMWeMeetAgain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "3AMWeMeetAgain" }}, "user": { "id": 1075248840, "name": "LENNY: The Red Mamba", "screen_name": "redheadLEN", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-01-09"), "description": "IG: redheadLEN • @WorldFamousNYC • Shisha Sundays • #HorrorWorld • #WorldBBQ • #TheGreatestDayEver • #HoopsShowdown • #FlagFootballShowdown • #3AmWeMeetAgain", "followers_count": 1558, "friends_count": 1002, "statues_count": 59597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576917006057473, "text": "Wind 2.0 mph SE. Barometer 29.804 in, Falling Rapidly. Temperature 52.7 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576917068972032, "text": "Only mistake I ever made was trusting you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38887130, "name": "ComeForMe", "screen_name": "DuckDuckkGoose", "lang": "en", "location": "Tampa Flaaawda ", "create_at": date("2009-05-09"), "description": "Sc:Duckie813 / Ig:Duckie813 , Im Broke", "followers_count": 1728, "friends_count": 1081, "statues_count": 110772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576917484077056, "text": "I don't even wanna talk about it I don't even wanna talk about it I don't even wanna say nothin everybody gonna say somethin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2727190000, "name": "brenna", "screen_name": "brennnzz", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-07-28"), "description": "without you, life will never be the same Oscar Florczyk -------- Drahnell Macaraeg ❤️", "followers_count": 274, "friends_count": 219, "statues_count": 8780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576917601710081, "text": "but ay i swear that its like pro rapperz raised me, &nobody'll ever b able2stop how id snap so amazingly,+i mean imuh do good w/my ppl ob-v.", "in_reply_to_status": 704563651290767360, "in_reply_to_user": 587523869, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587523869, "name": "K1NG J-O-E", "screen_name": "MriNoHow2Disser", "lang": "en", "location": "$$$CROWN$$$MY$$$FLY$$$ASS$$$", "create_at": date("2012-05-22"), "description": "Joe Mondrella~Rapper/Mu$1c Producer~thaprok1ngjoe@gmail.com ~ Potent1al Mov1e $tar #Let$Let1tFlyHomie! #iRap₩₩€$-t-y-l-e CatchMeN#ThaBurb$Or#ThaCh1 8159315004", "followers_count": 524, "friends_count": 595, "statues_count": 48917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lockport, IL", "id": "014241bf2253c205", "name": "Lockport", "place_type": "city", "bounding_box": rectangle("-88.087085,41.553694 -87.94345,41.641802") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1744225, "cityName": "Lockport" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576918801281025, "text": "\"Leave the door unlocked\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2245068486, "name": "kourt..", "screen_name": "KourtneiCt", "lang": "en", "location": "null", "create_at": date("2013-12-13"), "description": "21. Dalla$ Miss Purple & Gold 2015-2016", "followers_count": 1330, "friends_count": 740, "statues_count": 31963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshall, TX", "id": "0139a7c035c529cd", "name": "Marshall", "place_type": "city", "bounding_box": rectangle("-94.406743,32.491967 -94.29016,32.571239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48203, "countyName": "Harrison", "cityID": 4846776, "cityName": "Marshall" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576918876766208, "text": "Temp: 58.6°F Wind:0.0mph Pressure: 30.051hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576919270981633, "text": "got my nigga fucked up... https://t.co/Am0iF0WyM9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700855103, "name": "JFinesse", "screen_name": "Maine100k", "lang": "en", "location": "null", "create_at": date("2013-10-03"), "description": "⚔Louisburg College '17 |⚔18. Future U.S. Soldier", "followers_count": 501, "friends_count": 497, "statues_count": 5171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisburg, NC", "id": "0c88e2eadb640de9", "name": "Louisburg", "place_type": "city", "bounding_box": rectangle("-78.31891,36.07379 -78.277434,36.117072") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37069, "countyName": "Franklin", "cityID": 3739360, "cityName": "Louisburg" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576919287799808, "text": "@KyleBornheimer congrats!! https://t.co/OyN13SU0Pg #charitycase", "in_reply_to_status": -1, "in_reply_to_user": 126805530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "charitycase" }}, "user_mentions": {{ 126805530 }}, "user": { "id": 41134691, "name": "jeff meshel", "screen_name": "jammesh", "lang": "en", "location": "Fort Lauderdale", "create_at": date("2009-05-19"), "description": "null", "followers_count": 531, "friends_count": 337, "statues_count": 3427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland Park, FL", "id": "b230af2256484995", "name": "Oakland Park", "place_type": "city", "bounding_box": rectangle("-80.196625,26.150489 -80.117027,26.209352") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1250575, "cityName": "Oakland Park" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576919895814144, "text": "@WysteriaWhisper You too Wysteria. Sweet dreams.", "in_reply_to_status": 704576718120431618, "in_reply_to_user": 3287196289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3287196289 }}, "user": { "id": 3464930718, "name": "Marty Resetar", "screen_name": "RetiredSkeptic", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-09-05"), "description": "Retired from Las Vegas gaming industry after 36 years. Vietnam veteran. Married. Oakland Raider fan. Love learning about anything interesting or paranormal!", "followers_count": 1004, "friends_count": 1204, "statues_count": 4547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576920436895744, "text": "happy birthday to my baby!!!❤️❤️❤️❤️ @justinbieber https://t.co/XPeQpefPF3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 1039122762, "name": "ash", "screen_name": "ashleynavarro_", "lang": "en", "location": "null", "create_at": date("2012-12-27"), "description": "null", "followers_count": 362, "friends_count": 238, "statues_count": 4318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-03-01T00:00:02.000Z"), "id": 704576920604639232, "text": "One of my not so secret talents is turning in assignments is turning in assignments due at midnight......at 11:59", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1465938936, "name": "KariAnna Kuklinski", "screen_name": "k_kuklinski", "lang": "en", "location": "George Fox University ", "create_at": date("2013-05-28"), "description": "where the spirit of the Lord is, there is freedom", "followers_count": 253, "friends_count": 224, "statues_count": 6599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newberg, OR", "id": "7f3bf05bec93c209", "name": "Newberg", "place_type": "city", "bounding_box": rectangle("-122.989925,45.28369 -122.925614,45.328231") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill", "cityID": 4152100, "cityName": "Newberg" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576920969568256, "text": "59.4F (Feels: 59.4F) - Humidity: 99% - Wind: 4.5mph SE - Gust: 5.4mph - Pressure: 1026.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 228811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576921292705792, "text": "Wind 0.7 mph WSW. Barometer 30.09 in, Rising very rapidly. Temperature 27.1 °F. Rain today 0.00 in. Humidity 51%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 112, "statues_count": 157908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576921326264320, "text": "Wind 0 mph ---. Barometer 30.16 in, Rising. Temperature 50.9 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 56, "friends_count": 72, "statues_count": 26088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576921384824833, "text": "H A P P Y B I R T H D A Y @justinbieber ���� my first celebrity love�� 7 years & counting❤️ you are perfection�� https://t.co/ifFJNlxSaW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 559372323, "name": "K H A R E C E✨", "screen_name": "__kharece", "lang": "en", "location": "Cheney, WA", "create_at": date("2012-04-21"), "description": "everyone loves a good jazz square sc: kharecekathleen", "followers_count": 288, "friends_count": 499, "statues_count": 25453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576922198519809, "text": "#faceswap #whoworeitbest https://t.co/oR6i92aW95", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "faceswap", "whoworeitbest" }}, "user": { "id": 132338702, "name": "Laura Hartzheim", "screen_name": "LauraHartzheim", "lang": "en", "location": "Anaheim Hills", "create_at": date("2010-04-12"), "description": "Hi my name is Laura and I love America and alcohol", "followers_count": 222, "friends_count": 140, "statues_count": 6553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576922500509696, "text": "All yo tweets are bout me @girlswhogrowwee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4809803655 }}, "user": { "id": 134970873, "name": "Sugar", "screen_name": "SugarJayAre", "lang": "en", "location": "Clovis, CA", "create_at": date("2010-04-19"), "description": "Come at meh?", "followers_count": 517, "friends_count": 131, "statues_count": 19810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576923066761216, "text": "@anna_lutherr @50shadesofdank girl a year ago I had about $6 to my name. Today I live a comfy lifestyle and make more $ than my mom does.", "in_reply_to_status": 704576206302085120, "in_reply_to_user": 887493696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 887493696, 592285742 }}, "user": { "id": 191279750, "name": "kitty wap", "screen_name": "BK_mindtricks", "lang": "en", "location": "EPT///HTX", "create_at": date("2010-09-15"), "description": "I use my cat to pick up men", "followers_count": 1134, "friends_count": 852, "statues_count": 45157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576923192713216, "text": "@wickedshrapnel tits mcgee, right?", "in_reply_to_status": 704556754189725696, "in_reply_to_user": 200291506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 200291506 }}, "user": { "id": 453611632, "name": "Brian", "screen_name": "briocon7", "lang": "en", "location": "Livonia, MI", "create_at": date("2012-01-02"), "description": "On PSN & Xbox One: briocon7", "followers_count": 363, "friends_count": 70, "statues_count": 14724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livonia, MI", "id": "07b9307419f9363f", "name": "Livonia", "place_type": "city", "bounding_box": rectangle("-83.433894,42.352453 -83.312805,42.442178") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2649000, "cityName": "Livonia" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576923876225025, "text": "Alright it's midnight && still no J.Cole Album ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302168400, "name": "Miguel SZN", "screen_name": "MiguelRITG", "lang": "en", "location": "LA", "create_at": date("2015-07-31"), "description": "Real Boasy", "followers_count": 100, "friends_count": 80, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-03-01T00:00:03.000Z"), "id": 704576924408901632, "text": "BREE PUT MY LIFE INTO WORDS JUST NOW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 956467050, "name": "badie", "screen_name": "madiemaresh", "lang": "en", "location": "why you need to find me anyway", "create_at": date("2012-11-18"), "description": "it you didn't read it how Simple Plan sang it we can't be friends.", "followers_count": 519, "friends_count": 227, "statues_count": 19838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576925390413825, "text": "I can't fucking believe this...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2596875595, "name": "Global Boyz ", "screen_name": "beffo_joe", "lang": "en", "location": "Marvins Room ", "create_at": date("2014-06-30"), "description": "HOST OF 4/20, 3/22, & 2/20 ||18 AND BOOMIN", "followers_count": 457, "friends_count": 279, "statues_count": 31952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926053113856, "text": "Wind 4.3 mph NW. Barometer 30.080 in, Rising Rapidly. Temperature 15.3 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926082412544, "text": "i'm cryin' my baby is finally 22 now���� @justinbieber https://t.co/J5ulybHYVO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 2927456942, "name": "beyonce", "screen_name": "malyiaaf", "lang": "en", "location": "in my bed", "create_at": date("2014-12-12"), "description": "probably sleeping", "followers_count": 447, "friends_count": 1190, "statues_count": 2326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926183268352, "text": "@Trumpwall2016 @msann43 https://t.co/ACGl5xoaYH", "in_reply_to_status": 704575106752720897, "in_reply_to_user": 4854732196, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4854732196, 876040219 }}, "user": { "id": 819577338, "name": "Tony Zraiqat", "screen_name": "TonyZraiqat", "lang": "en", "location": "Texas", "create_at": date("2012-09-12"), "description": "The power of the people is stronger than the people in power ,Politicians R+D is corrupt", "followers_count": 277, "friends_count": 266, "statues_count": 3614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926287958017, "text": "HAPPY 17TH BFF @KateDoonan ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298810948 }}, "user": { "id": 363901092, "name": "Lindsey", "screen_name": "lindseyyannaa", "lang": "en", "location": "PDX", "create_at": date("2011-08-28"), "description": "bears, beets, battlestar galactica", "followers_count": 321, "friends_count": 275, "statues_count": 4674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926409740288, "text": "And on that note, I think Imma call it a night.sweet dreams https://t.co/W6ImmW2OsN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76360295, "name": "Barbara Dajour", "screen_name": "wantmorethrills", "lang": "en", "location": "aint nobody comin 2 see u otis", "create_at": date("2009-09-22"), "description": "I act as old as my name. I got way too much soul for me to sell it. #nccu", "followers_count": 4113, "friends_count": 1692, "statues_count": 105457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926481031169, "text": "#MissingPerson at 4500-4549 Silver Star Rd, Pine Hills, FL 32808. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4400027,28.5779251"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MissingPerson", "orlpol", "ocso" }}, "user": { "id": 39047091, "name": "Police Calls 32808", "screen_name": "orlpol32808", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 219, "friends_count": 1, "statues_count": 79387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576926963384321, "text": "Nah baby I ain't want no wap I just want the fetty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406364890, "name": "Ricky Bautista", "screen_name": "platanodickrick", "lang": "en", "location": "Springfield, ma", "create_at": date("2011-11-06"), "description": "null", "followers_count": 729, "friends_count": 568, "statues_count": 24444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576927466549248, "text": "Better Call Saul is literally such a good show", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434512241, "name": "Dan Dan™", "screen_name": "dandandailyy", "lang": "en", "location": "Azerbaijan", "create_at": date("2011-12-11"), "description": "not flawless but I'm gorgeous", "followers_count": 461, "friends_count": 313, "statues_count": 7169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitney, NV", "id": "cb5ac954e9b77fba", "name": "Whitney", "place_type": "city", "bounding_box": rectangle("-115.064609,36.074534 -115.010215,36.137196") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3283800, "cityName": "Whitney" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576927873433601, "text": "I'm legit pissed rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 742254470, "name": "Michael", "screen_name": "mike_lupercio", "lang": "en", "location": "null", "create_at": date("2012-08-06"), "description": "can't knock the hustle", "followers_count": 368, "friends_count": 334, "statues_count": 8004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576928016019456, "text": "@worddiction @SSanchez210", "in_reply_to_status": 704188131097108480, "in_reply_to_user": 454939382, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 454939382, 393790060 }}, "user": { "id": 587139693, "name": "adreeanna lee", "screen_name": "leeadrianna", "lang": "en", "location": "null", "create_at": date("2012-05-21"), "description": "John 13:7.karen is bae.robert is kool.it's just that simple", "followers_count": 817, "friends_count": 773, "statues_count": 17154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576928297058304, "text": "Someone please bring medicine that will knock me out ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56784371, "name": "Heat Nation Kaela", "screen_name": "KaelaKolossal", "lang": "en", "location": "SoCal Shawty ", "create_at": date("2009-07-14"), "description": "I like basketball, porn, and cussing you out in Spanish. Kaela all on ya mouth like liquor. #FaithfulDominicansOfTwitter #HeatNATION (33-26)", "followers_count": 5602, "friends_count": 862, "statues_count": 201939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576928607502336, "text": "@ZoomTV @Aminaramayaz @kalisaisaacbet @eddykenzoficial drop Nanziri https://t.co/eEXOfz47aV https://t.co/QCeMXMQOz9", "in_reply_to_status": 704558161374502912, "in_reply_to_user": 18354016, "favorite_count": 0, "retweet_count": 0, "lang": "sl", "is_retweet": false, "user_mentions": {{ 18354016, 3316627178, 4031798128, 2496290942 }}, "user": { "id": 3439722501, "name": "Morosi Kalya", "screen_name": "mogariate", "lang": "en", "location": "null", "create_at": date("2015-08-25"), "description": "null", "followers_count": 1502, "friends_count": 4998, "statues_count": 18531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-01T00:00:04.000Z"), "id": 704576928636805120, "text": "These words are knives that often leave scars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3144997170, "name": "Erin Flynn", "screen_name": "erinmflynn98", "lang": "en", "location": "null", "create_at": date("2015-04-08"), "description": "always trying to look at the bright side of things", "followers_count": 52, "friends_count": 44, "statues_count": 579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576929416908800, "text": "@whoDAfuckisTIKI @Breionjazmine I have no memory of what happened, I was hacked, we were talking about someone else, pick 1 to believe...", "in_reply_to_status": 704576536750391300, "in_reply_to_user": 2886371272, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2886371272, 1029170106 }}, "user": { "id": 221636660, "name": "The Company Hitman", "screen_name": "tRYANbeme", "lang": "en", "location": "Behind Rallys on Claiborne ", "create_at": date("2010-11-30"), "description": "REDTEAM Prostitution Company! Mention me for a bitch.", "followers_count": 689, "friends_count": 161, "statues_count": 50531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Destrehan, LA", "id": "97aba0c2400af778", "name": "Destrehan", "place_type": "city", "bounding_box": rectangle("-90.392327,29.93704 -90.335777,29.987893") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2220820, "cityName": "Destrehan" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576929794379776, "text": "Wind 4.9 mph NW. Barometer 30.127 in, Rising Rapidly. Temperature 20.2 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576929802752000, "text": "@YoungButAble shit up", "in_reply_to_status": 704570276919414784, "in_reply_to_user": 453794714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453794714 }}, "user": { "id": 404574164, "name": "Hugh Hefner Jr.", "screen_name": "TheTruth_99", "lang": "en", "location": "null", "create_at": date("2011-11-03"), "description": "Honorary member of #FleekSquad, #ICEEaddictsanonymous and #TangGang Love you Folabi R.I.P 2/27/13 A great friend but an even better brother #TAMU17", "followers_count": 588, "friends_count": 400, "statues_count": 21887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576929987371008, "text": "Just had someone ask me if my boobs were real...#illtakethatasacompliment #lasvegas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "illtakethatasacompliment", "lasvegas" }}, "user": { "id": 2961445440, "name": "Audrey Sisson", "screen_name": "audreyfsisson", "lang": "en", "location": "Missouri", "create_at": date("2015-01-06"), "description": "vegan. midwestern. domestic goddess. sine prole. future court reporter. marketing representative.", "followers_count": 30, "friends_count": 60, "statues_count": 178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, NV", "id": "37d88f13e7a85f14", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-115.173994,36.128077 -115.083699,36.144748") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3284600, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576929999884288, "text": "@westkid559 nothing ����", "in_reply_to_status": 704576706590302208, "in_reply_to_user": 53612128, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53612128 }}, "user": { "id": 385807408, "name": "JT.", "screen_name": "Jontayyylor", "lang": "en", "location": "null", "create_at": date("2011-10-05"), "description": "didn't they tell you that I was a savage ?", "followers_count": 926, "friends_count": 698, "statues_count": 13806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindsay, CA", "id": "1c7fc1369e419742", "name": "Lindsay", "place_type": "city", "bounding_box": rectangle("-119.120488,36.181549 -119.060335,36.237053") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 641712, "cityName": "Lindsay" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576930071375872, "text": "@GovernorPerry I am quite disturbed by your questioning of the Biblical knowledge of Dr Jerry Falwell JR. U R NO PREACHER U R A Politician.", "in_reply_to_status": -1, "in_reply_to_user": 18906561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18906561 }}, "user": { "id": 47759398, "name": "Harry Burdette", "screen_name": "shoecop", "lang": "en", "location": "Evansville IN", "create_at": date("2009-06-16"), "description": "Last American Hero; in my own mind. Conservative man.", "followers_count": 31, "friends_count": 138, "statues_count": 635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576930377388032, "text": "I thought about her every single day that summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951611208, "name": "Stephanie", "screen_name": "SstephFernanda", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 303, "friends_count": 227, "statues_count": 5143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576930461454337, "text": "Yasssss���������� https://t.co/d4w5UNtmnU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2389958232, "name": "heather✝", "screen_name": "heatherbrunett1", "lang": "en", "location": "517-337", "create_at": date("2014-03-14"), "description": "null", "followers_count": 417, "friends_count": 444, "statues_count": 14588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moss Bluff, LA", "id": "01d0182d2509e0e3", "name": "Moss Bluff", "place_type": "city", "bounding_box": rectangle("-93.255102,30.293616 -93.162702,30.369199") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2252425, "cityName": "Moss Bluff" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576930868125696, "text": "It's game, das me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1014583254, "name": "MB.ricky", "screen_name": "RickyChavez12", "lang": "en", "location": "the 6", "create_at": date("2012-12-15"), "description": "know yourself know your worth", "followers_count": 212, "friends_count": 291, "statues_count": 3135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576930931036160, "text": "When I cut everybody off for you and tell you I love you I mean that shit Fr Fr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263439930, "name": "El Kenn❤️", "screen_name": "kenneic", "lang": "en", "location": "Dallas, TX ✈️ Tyler, TX", "create_at": date("2011-03-09"), "description": "19 | #DabChicksPromo | #TJC17 | #FREEMYBROTHERTAY", "followers_count": 2866, "friends_count": 1677, "statues_count": 39151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576931510018049, "text": "Well I guess you'd say, what can make me feel this way. My girl, my girl, my girl, talking bout my girl.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 610927647, "name": "Travis Solberg", "screen_name": "TravieeBoyy", "lang": "en", "location": "Westland, MI", "create_at": date("2012-06-17"), "description": "Travis Solberg. Jghs 14. Scraft. I have become comfortably numb. #h2h. Leaving Sep 6th to become a United States Marine.", "followers_count": 1616, "friends_count": 859, "statues_count": 29199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westland, MI", "id": "879dfee0187b884a", "name": "Westland", "place_type": "city", "bounding_box": rectangle("-83.430458,42.267434 -83.308978,42.355177") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2686000, "cityName": "Westland" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576931535060993, "text": "I'm so excited for Skizzy Mars tomorrow!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 956881434, "name": "King Andy", "screen_name": "TheBrutalSavage", "lang": "en", "location": "M.A.A.D City", "create_at": date("2012-11-18"), "description": "You'd be nervous. I'd be chill. #SmoothSavage SnapChat ~ andy_magana", "followers_count": 1015, "friends_count": 254, "statues_count": 13896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576932067872769, "text": "Great Raiz Brazil Samba Show in Private Party this Monday evening at Kravings Fusion Brazilian… https://t.co/aK2yoFQ9RI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.54054367,34.17089032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 945331694, "name": "RaizBrazil Dance Co", "screen_name": "RaizBrazilCo", "lang": "en", "location": "Los Angeles/California/USA", "create_at": date("2012-11-12"), "description": "Brazilian Dance Company that entertain the audiences, bringing a lot of soul and culture behind the feathers, glitter and the soultry moves!", "followers_count": 79, "friends_count": 308, "statues_count": 478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576932189372416, "text": "When your laptop shuts down in the middle of you writing an essay ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2181032814, "name": "K", "screen_name": "_marrypoppinns", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "null", "followers_count": 502, "friends_count": 313, "statues_count": 7366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576932579581954, "text": "@faragi11 Ima call you around noon tomorrow coach .", "in_reply_to_status": -1, "in_reply_to_user": 29049662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29049662 }}, "user": { "id": 620135849, "name": "☁️✨ Marcus ✨☁️", "screen_name": "PaperChaser_901", "lang": "en", "location": "Memphis , TN ", "create_at": date("2012-06-27"), "description": "Bookings/Promos/Inquires/Performaces .... Email: marcusabston14@gmail.com", "followers_count": 2663, "friends_count": 5001, "statues_count": 10386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576932596174848, "text": "my hair is so close to being ass length good thing im ready to chop it off ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2995330350, "name": "leese", "screen_name": "scumzillla", "lang": "en", "location": "the land", "create_at": date("2015-01-24"), "description": "only happy when it rains", "followers_count": 73, "friends_count": 117, "statues_count": 3939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, OH", "id": "888482aa70a3bc61", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-81.8375,41.463245 -81.768603,41.49759") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3941664, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576932835295232, "text": "@Carvers_Deli The Reuben Sandwich was great. Tnx. I was the one wearing a plaid flannel shirt", "in_reply_to_status": 702196760890974208, "in_reply_to_user": 1572048427, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1572048427 }}, "user": { "id": 3118782261, "name": "Steve Marshall", "screen_name": "MSO59803", "lang": "en", "location": "Missoula, MT", "create_at": date("2015-03-26"), "description": "5'5 274# Practice TKD & Judo here in Missoula, at Missoula Taekwondo Center. Currently Blue & closer to Senior Blue. Went to Lng Bch Poly in CA", "followers_count": 6, "friends_count": 103, "statues_count": 227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missoula, MT", "id": "00427d4a5c4a1fc3", "name": "Missoula", "place_type": "city", "bounding_box": rectangle("-114.120325,46.780202 -113.941975,46.977998") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30063, "countyName": "Missoula", "cityID": 3050200, "cityName": "Missoula" } }
+{ "create_at": datetime("2016-03-01T00:00:05.000Z"), "id": 704576933074317312, "text": "Temp: 23.0°F - Dew Point: 17.2° - Wind: 11.8 mph - Gust: 19.7 - Rain Today: 0.00in. - Pressure: 29.92in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934034874368, "text": "Ladies y'all still faking orgasms so you don't hurt ya niggas feelings bc he got weak dick������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2349058622, "name": "Finesse prince of LA", "screen_name": "yaboii_matt", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-02-17"), "description": "Jerkin ain't dead #scumsquad", "followers_count": 63, "friends_count": 53, "statues_count": 4065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934106128384, "text": "Wind 4.0 mph ENE. Barometer 29.974 in, Falling. Temperature 36.0 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 384, "friends_count": 292, "statues_count": 8661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934231945216, "text": "bby boy is 22 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1170088878, "name": "HOPE", "screen_name": "hopealvarezz", "lang": "en", "location": "lqhs", "create_at": date("2013-02-11"), "description": "livin life lavish.", "followers_count": 393, "friends_count": 198, "statues_count": 7277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934349578240, "text": "NOT JUST AN AMNESTY MADMAN: Est Rubio Proves He's A Big Govt Neo-Con Interventionist rt @Jamie_Weinstein @potusCRUZ https://t.co/WyuGf4ABkL", "in_reply_to_status": 703058847921876992, "in_reply_to_user": 33938700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33938700, 3631111281 }}, "user": { "id": 19850687, "name": "ConserValidity", "screen_name": "ConserValidity", "lang": "en", "location": "NEW YORK, NY ", "create_at": date("2009-01-31"), "description": "Constitutional Conservative\n#TCOT", "followers_count": 62946, "friends_count": 68905, "statues_count": 73460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934357766144, "text": "I was told this astronomy class would just be a lecture class and be easy. It's my hardest class. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133467834, "name": "Jordan Ader", "screen_name": "HawkeySZN", "lang": "en", "location": "Commerce, TX", "create_at": date("2010-04-15"), "description": "Hello, I'm Sigma Chi pledge Jordan Ader, and I'm more funny on the Internet than I am in person • Addicted to Chicago • TAMUC • SC: AderJ05", "followers_count": 1506, "friends_count": 818, "statues_count": 14941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934768803840, "text": "I'm about to run it right now!! https://t.co/yqFLdJd7Ab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2301827244, "name": "I Feel Like Pablo☔️", "screen_name": "DebonairFresH", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-20"), "description": "National Weightlifter || Covered || Gentleman || Positive Vibes", "followers_count": 805, "friends_count": 1162, "statues_count": 10237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576934987112452, "text": "I don't get how rape is funny wether it happens to a man of a woman. These ppl are dumb https://t.co/Dwz17D9e3q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2441996595, "name": "Lady Bowie.", "screen_name": "JunoCassandra", "lang": "en", "location": "Dragonstone. ☄", "create_at": date("2014-03-28"), "description": "• I will NEVER apologize for being an outspoken boss, because I don't owe anyone an apology. •", "followers_count": 1588, "friends_count": 952, "statues_count": 105178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576935062409216, "text": "@RyanSebesta https://t.co/zb5mchfEZV", "in_reply_to_status": 704576752245460992, "in_reply_to_user": 380660622, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 380660622 }}, "user": { "id": 844901059, "name": "Shelby Ashford", "screen_name": "ashfordshelby", "lang": "en", "location": "Chimy's ", "create_at": date("2012-09-24"), "description": "Margaritas and John Elway; College Station/Keller TX/Colorado snapchat: ashfordshelby", "followers_count": 1020, "friends_count": 526, "statues_count": 39422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576935431643136, "text": "I grind till I own it������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2722686970, "name": "Hailz", "screen_name": "haileystraderrr", "lang": "en", "location": "null", "create_at": date("2014-07-23"), "description": "null", "followers_count": 228, "friends_count": 305, "statues_count": 1514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madisonville, KY", "id": "0ca7a5e41a672fe8", "name": "Madisonville", "place_type": "city", "bounding_box": rectangle("-87.545278,37.281802 -87.452009,37.3971") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21107, "countyName": "Hopkins", "cityID": 2149368, "cityName": "Madisonville" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576935993569280, "text": "#HappyBirthdayJustinBieber you're 22 now �� I love you forever!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyBirthdayJustinBieber" }}, "user": { "id": 291159024, "name": "Krisha", "screen_name": "krishaogarcia", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-05-01"), "description": "3/9 | tm | sc: krishaogarcia", "followers_count": 711, "friends_count": 699, "statues_count": 12361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576936144572416, "text": "@leemon_j wow", "in_reply_to_status": 704576806901252096, "in_reply_to_user": 3148553852, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3148553852 }}, "user": { "id": 3292411931, "name": "Menéndez", "screen_name": "morganthelion", "lang": "en", "location": "San Marcos, TX", "create_at": date("2015-05-20"), "description": "The greatest gift is to love and to be loved in return. Snap Chat @morganthelion", "followers_count": 189, "friends_count": 249, "statues_count": 675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576936576577536, "text": "It's fucking March.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17764856, "name": "Emmm A. Deee", "screen_name": "shelldash", "lang": "en", "location": "New York, NY", "create_at": date("2008-11-30"), "description": "My defense attorney said the less I say here the better.", "followers_count": 5068, "friends_count": 3431, "statues_count": 45107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altadena, CA", "id": "fb76b3aa366004c9", "name": "Altadena", "place_type": "city", "bounding_box": rectangle("-118.177296,34.167539 -118.095091,34.220078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601290, "cityName": "Altadena" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576936664649728, "text": "@HMTylerHenry I'm in awe with your gift, such power and gentleness combine.", "in_reply_to_status": -1, "in_reply_to_user": 3322895292, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3322895292 }}, "user": { "id": 19316168, "name": "Marius Andrei", "screen_name": "MARIUS_DTOWN", "lang": "en", "location": "ÜT: 34.036252,-118.236045", "create_at": date("2009-01-21"), "description": "Jack of all trades, master of none. I tweet at my own risk, 140 characters at a time. Recovering Catholic. Underwear Salesman.", "followers_count": 1551, "friends_count": 996, "statues_count": 48240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central City East, Los Angeles", "id": "61d3a50809879273", "name": "Central City East", "place_type": "neighborhood", "bounding_box": rectangle("-118.239675,34.014714 -118.215014,34.071979") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576936853372929, "text": "Truuuuuee https://t.co/f3NV77Fo3e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1113450896, "name": "Yuri", "screen_name": "Ramirezbruhh_", "lang": "en", "location": "null", "create_at": date("2013-01-22"), "description": "null", "followers_count": 277, "friends_count": 159, "statues_count": 14237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-03-01T00:00:06.000Z"), "id": 704576937562234880, "text": "@_AyooMolly you know why ��", "in_reply_to_status": 704576875306295296, "in_reply_to_user": 330576330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 330576330 }}, "user": { "id": 2774539715, "name": "✌️❄️1⃣", "screen_name": "toocold_1", "lang": "en", "location": "null", "create_at": date("2014-09-18"), "description": "#336", "followers_count": 699, "friends_count": 995, "statues_count": 3704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-03-01T00:00:07.000Z"), "id": 704576938724098049, "text": "Happy Birthday @Aisling24778819!!����❤️ you're my best friend through everything and I love you lots cuzz. Have fun being 16!!!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4217299633 }}, "user": { "id": 3018023654, "name": "Maddy", "screen_name": "murphy_maddy", "lang": "en", "location": "Boncord, CA", "create_at": date("2015-02-12"), "description": "null", "followers_count": 166, "friends_count": 216, "statues_count": 632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-03-01T00:00:07.000Z"), "id": 704576939558871040, "text": "need an exotic baby animal or i will cry https://t.co/7N6nScmzul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1853009864, "name": "b", "screen_name": "xchnll", "lang": "en", "location": "null", "create_at": date("2013-09-10"), "description": "null", "followers_count": 681, "friends_count": 104, "statues_count": 35795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermitage, PA", "id": "008a6343d42a42f3", "name": "Hermitage", "place_type": "city", "bounding_box": rectangle("-80.519092,41.187685 -80.427557,41.279389") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42085, "countyName": "Mercer", "cityID": 4234064, "cityName": "Hermitage" } }
+{ "create_at": datetime("2016-03-01T00:00:07.000Z"), "id": 704576939596652544, "text": "@GabeKindaCares join the pure side", "in_reply_to_status": 704576840577511425, "in_reply_to_user": 1152403088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1152403088 }}, "user": { "id": 2915933862, "name": "Bullet Club 4-4-4-4", "screen_name": "Marcus_Mims213", "lang": "en", "location": "Killeen, TX", "create_at": date("2014-12-01"), "description": "23 from Killeen TX Hall Of Pain", "followers_count": 522, "friends_count": 804, "statues_count": 30488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-03-01T00:00:07.000Z"), "id": 704576939907014656, "text": "Wind 7.0 mph WNW. Barometer 29.796 in, Rising Rapidly. Temperature 31.0 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939296193699840, "text": "�� https://t.co/l6OILoHq5S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 139228142, "name": "✿ Pope of Dope ✿", "screen_name": "templeshields", "lang": "en", "location": "19 | uncool", "create_at": date("2010-05-01"), "description": "null", "followers_count": 425, "friends_count": 252, "statues_count": 19618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brea, CA", "id": "341aa0cdb944022e", "name": "Brea", "place_type": "city", "bounding_box": rectangle("-117.928789,33.905579 -117.800848,33.947725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608100, "cityName": "Brea" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939297170915328, "text": "Xans got my eyes crossing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57682255, "name": "✨", "screen_name": "queenlegacie", "lang": "en", "location": "da south", "create_at": date("2009-07-17"), "description": "the reason I act superior to everyone is bc I am. #NCAT @youurridol #futurepsychiatrist", "followers_count": 1159, "friends_count": 582, "statues_count": 69744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939298680922112, "text": "or is it no ones job to deal with that friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258015876, "name": "charles", "screen_name": "Rels_LT", "lang": "en", "location": "Echo Park", "create_at": date("2011-02-26"), "description": "I'm not punk, please don't tell anyone", "followers_count": 184, "friends_count": 404, "statues_count": 56806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939299289100288, "text": "@angdegrazia https://t.co/FA78G1nBaO", "in_reply_to_status": 704939034242580480, "in_reply_to_user": 60510605, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 60510605 }}, "user": { "id": 619586953, "name": "Jenny", "screen_name": "jenbask", "lang": "en", "location": "null", "create_at": date("2012-06-26"), "description": "I'm a hummusexual", "followers_count": 534, "friends_count": 350, "statues_count": 5550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939299414929410, "text": "The litty committee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.654427,41.841912"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95947142, "name": "Otto Rocket", "screen_name": "WhoButAR", "lang": "en", "location": "#TheRidge", "create_at": date("2009-12-10"), "description": "Mood: http://tinyurl.com/ko6lecd\n\nSongwriter/feature inquiries \nwhobutar@gmail.com\n\nCheck out TheGoodz/MouthfulaGOLD \non sound cloud link below", "followers_count": 1027, "friends_count": 527, "statues_count": 59479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939299494621186, "text": "Wizkayy stay losing boi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1668356684, "name": "Joel", "screen_name": "InFamous773", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-08-13"), "description": "Duo:@EZG_Mayhem My SnD Duo: @Godlo_\nStreamer for @cE_eSports", "followers_count": 2174, "friends_count": 981, "statues_count": 33738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-02T00:00:00.000Z"), "id": 704939299842756610, "text": "@BasedDodgersFan https://t.co/1vvFfZ1LIH", "in_reply_to_status": 704939111812042752, "in_reply_to_user": 2807991804, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2807991804 }}, "user": { "id": 184630285, "name": "BOBBY VALENTINE", "screen_name": "MakeMericaGr8t", "lang": "en", "location": "Denton, TX", "create_at": date("2010-08-29"), "description": "Real Estate Major #SpursNation #CowboysNation #UNT19 ONLY THE EDUCATED ARE FREE! DOES THAT INCLUDE YOU? IF CASH AIN'T KING IT'S DAMN SURE THE INCENTIVE", "followers_count": 648, "friends_count": 521, "statues_count": 22365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939300631285760, "text": "\"Vai, lasciati alle spalle passato e origini. Parti, per dove poco conta, esplora, osa ma… https://t.co/opIiCEUKYv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-109.74222222,38.50833333"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 51420606, "name": "Silvia Blazina", "screen_name": "SilviaBlazina", "lang": "it", "location": "Spilimbergo", "create_at": date("2009-06-27"), "description": "Graphic designer, mogger and life lover: kite, stray dog, walking travesty, smiling on everything. FriuliVeneziaGiulia I T / http://theeatculture.com", "followers_count": 211, "friends_count": 568, "statues_count": 748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utah, USA", "id": "1879ace9e02ace61", "name": "Utah", "place_type": "admin", "bounding_box": rectangle("-114.052999,36.997905 -109.041059,42.001619") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49019, "countyName": "Grand" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939300643803136, "text": "@K1ng_Al23 @b_frauenheim lets hope another mean girls 2 doesn't happen again", "in_reply_to_status": 704939074159845381, "in_reply_to_user": 2570059401, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2570059401, 2727208957 }}, "user": { "id": 2299220870, "name": "riana nicole", "screen_name": "rianamartinezzz", "lang": "en", "location": "Hanford, CA", "create_at": date("2014-01-18"), "description": "fun&fifteen", "followers_count": 232, "friends_count": 324, "statues_count": 1280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301155561472, "text": "03/02@03:00 - Temp 56.3F, WC 53.9F. Wind 9.8mph SW, Gust 21.0mph. Bar 29.554in, Falling slowly. Rain 0.07in. Hum 87%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301189058561, "text": "@SefDogg_ lmfao ��", "in_reply_to_status": 704939222373933057, "in_reply_to_user": 1524120708, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1524120708 }}, "user": { "id": 1022438443, "name": "Kristaps GLOzingis", "screen_name": "21Joseph", "lang": "en", "location": "Lakewood, CA", "create_at": date("2012-12-19"), "description": "#MacMillerHive | Cubs back2back champs 1907/08 | Mommas Boy", "followers_count": 779, "friends_count": 580, "statues_count": 34750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301436575744, "text": "Why do cheating ass niggas always got some loyal ass shit to say?? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3821114293, "name": "TreBeenHim✨", "screen_name": "daSNATCHgod", "lang": "en", "location": "DryAss Modesto, California", "create_at": date("2015-10-07"), "description": "SnapChat: Tretrill510 | IG: yourkvng.tre | @QueenAsh1_ my dark chocolate thuggaaaa ✨ | MJC student | BraceFace | #BlackMafia | I have MCMs and WCMs #Bi", "followers_count": 1294, "friends_count": 1624, "statues_count": 3808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301453299712, "text": "\"Because you never truly know abandonment and lonliness until you've felt the opposite.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2983249237, "name": "Sara Bae Mitchell", "screen_name": "mischiefhandled", "lang": "en", "location": "Under4layersofcarhartt, Utah", "create_at": date("2015-01-14"), "description": "Fuckin Orcas. Am I right?", "followers_count": 330, "friends_count": 756, "statues_count": 4514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ogden, UT", "id": "4980bc1544652d22", "name": "Ogden", "place_type": "city", "bounding_box": rectangle("-112.025874,41.149594 -111.904265,41.286003") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4955980, "cityName": "Ogden" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301478465541, "text": "\"And I brought you into a plentiful country, to eat the fruit thereof and the goodness thereof; ...\" Jeremiah 2:7 (KJV)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59940799, "name": "Valeri Dvnae", "screen_name": "Valeyah", "lang": "en", "location": "Washington State", "create_at": date("2009-07-24"), "description": "Pro Israel, Pro Humanity, Pro Life, Pro Jesus , Pro Truth, Pro God, Pro the Holy Ghost/Holy Ghost, God love you more than anyone you know", "followers_count": 222, "friends_count": 905, "statues_count": 7741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301562404865, "text": "HAPPY BIRTHDAY TO MY BEST FRIEND IN THE ENTIRE WORLD ILYSM CAT! I HOPE YOU HAVE AN AMAZING DAY����1️⃣4️⃣ @catholt21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2569724432 }}, "user": { "id": 2666889296, "name": "Gigi Papasavvas™", "screen_name": "GigiPapasavvas_", "lang": "en", "location": "null", "create_at": date("2014-07-21"), "description": "I love @theskylerngo❤", "followers_count": 408, "friends_count": 318, "statues_count": 1109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301755346944, "text": "New Midori Traveler's Notebook delivered today! Very excited. It's all setup and ready for use starting tomorrow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35916158, "name": "Danny Rehr", "screen_name": "danrehr", "lang": "en", "location": "Vancouver, WA", "create_at": date("2009-04-27"), "description": "MBA candidate", "followers_count": 23, "friends_count": 12, "statues_count": 314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301851758592, "text": "The Lakers are not a good NBA basketball team..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1084185427, "name": "Dontonio Jordan", "screen_name": "DonJayDeuceTre", "lang": "en", "location": "AR ✈️ TX ✈️ CA ✈️ ?", "create_at": date("2013-01-12"), "description": "Former Stanford Football Wide Receiver | Ωψφ", "followers_count": 603, "friends_count": 256, "statues_count": 5324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stanford, CA", "id": "ed0a823456907b4c", "name": "Stanford", "place_type": "city", "bounding_box": rectangle("-122.191291,37.406339 -122.149162,37.443731") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 673906, "cityName": "Stanford" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301851815936, "text": "@LovelyJeenn did we have homework for stockweezie lol", "in_reply_to_status": 704938382217093120, "in_reply_to_user": 329473847, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 329473847 }}, "user": { "id": 883617421, "name": "James", "screen_name": "James_soup2", "lang": "en", "location": "Jefferson Park, Los Angeles", "create_at": date("2012-10-15"), "description": "sc: James_sup", "followers_count": 117, "friends_count": 78, "statues_count": 2679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939301851828224, "text": "i got anxiety bad ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2258384905, "name": "Jaz'Myn Collins", "screen_name": "__jazzmynn", "lang": "en", "location": "North Carolina", "create_at": date("2013-12-22"), "description": "IG : theyadorejas | SC : j_azmynm", "followers_count": 1466, "friends_count": 1272, "statues_count": 33192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939303680532480, "text": "Lil Uzi be turnt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 46305981, "name": "Jack of DemBoy$", "screen_name": "ImDatNigga_Jack", "lang": "en", "location": "PVAMU", "create_at": date("2009-06-10"), "description": "The Father Of Slaughter Dem Boy$ IG:imdatnigga_jack #PVNation #HighLifePromos #LongLiveChopo Free @JoeSwaggerRight #LongLiveClyde", "followers_count": 5760, "friends_count": 4015, "statues_count": 333024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939303743430657, "text": "Call me maybe. #GimmieLoveTour @ Wonder Ballroom https://t.co/wZfRJRnZ3D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.66338,45.54063"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GimmieLoveTour" }}, "user": { "id": 34830170, "name": "The Naz", "screen_name": "nazpyro", "lang": "en", "location": "Techlandia", "create_at": date("2009-04-23"), "description": "Geek. Drank. Dance. Teemo.", "followers_count": 390, "friends_count": 2068, "statues_count": 11873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-02T00:00:01.000Z"), "id": 704939304179601408, "text": "What if Trump just names his son as his Vp candidate and just turns turns the White House into a dynasty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497522452, "name": "Brian Crowley", "screen_name": "RUN_BRC", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "Go Tribe", "followers_count": 198, "friends_count": 294, "statues_count": 3699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williamsburg, VA", "id": "e39414c8fa720aae", "name": "Williamsburg", "place_type": "city", "bounding_box": rectangle("-76.741123,37.247278 -76.665387,37.31071") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51830, "countyName": "Williamsburg", "cityID": 5186160, "cityName": "Williamsburg" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305039486977, "text": "@martos_andrew oakley. Where you at?", "in_reply_to_status": 704939187158589441, "in_reply_to_user": 2496828355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2496828355 }}, "user": { "id": 175874505, "name": ".", "screen_name": "TardyDom", "lang": "en", "location": "null", "create_at": date("2010-08-07"), "description": "Yay Area. 6 foot and... well. yeah", "followers_count": 451, "friends_count": 358, "statues_count": 25204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Creek, CA", "id": "a35b62af9d82aa08", "name": "Walnut Creek", "place_type": "city", "bounding_box": rectangle("-122.10003,37.842368 -121.980829,37.944965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 683346, "cityName": "Walnut Creek" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305060474880, "text": "@ebbtideapp Tide in Rockport, Texas 03/02/2016\n Low 7:03am -0.2\nHigh 10:22pm 0.0\n Low 7:59am -0.2\nHigh 11:07pm 0.0\n Low 8:55am -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-97.0467,28.0217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 40, "friends_count": 1, "statues_count": 12331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48007, "countyName": "Aransas", "cityID": 4862804, "cityName": "Rockport" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305232429056, "text": "I love you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176874611, "name": "Anthony Gonzales", "screen_name": "AntsRGon", "lang": "en", "location": "Thornton, Colorado", "create_at": date("2010-08-10"), "description": "Open Sociopath, producer, singer/rapper, vlogger, artist. Taking the world by storm, one step at a time. just kidding I play video games.", "followers_count": 318, "friends_count": 277, "statues_count": 7299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305354051584, "text": "57.1F (Feels: 57.1F) - Humidity: 90% - Wind: 6.9mph NW - Gust: 8.3mph - Pressure: 1030.9mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 228937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305597321217, "text": "@aztiitza @yerangie_ lmaooo did it hurt a lot", "in_reply_to_status": 704931420276391936, "in_reply_to_user": 1141211617, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1141211617, 2965234851 }}, "user": { "id": 985912608, "name": "Ivan", "screen_name": "_IvanTheCreator", "lang": "en", "location": "TheFlood2015", "create_at": date("2012-12-02"), "description": "Silver Tongued Devil", "followers_count": 299, "friends_count": 268, "statues_count": 35552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cicero, IL", "id": "50f296a1bd4198b6", "name": "Cicero", "place_type": "city", "bounding_box": rectangle("-87.77903,41.82158 -87.738571,41.865972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714351, "cityName": "Cicero" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939305605709824, "text": "Shii you right lol https://t.co/xWnaeVoGAW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64064346, "name": "Jesus Shuttlesworth", "screen_name": "ThatBoyGQ", "lang": "en", "location": "Moreno Valley, CA", "create_at": date("2009-08-08"), "description": "Move in Silence", "followers_count": 1298, "friends_count": 1057, "statues_count": 46845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939306008363008, "text": "Wind 0.0 mph NW. Barometer 30.224 in, Steady. Temperature 26.7 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939306184544257, "text": "It is not only about you stop being a lot little bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2295201326, "name": ".", "screen_name": "youngsterpunk", "lang": "en", "location": "Lacey, WA", "create_at": date("2014-01-16"), "description": "senior@tline", "followers_count": 135, "friends_count": 100, "statues_count": 2883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939306448785408, "text": "Wind 0.0 mph ---. Barometer 30.16 in, Rising slowly. Temperature 54.9 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939306964664320, "text": "Wind 4.0 mph WNW. Barometer 30.124 in, Rising. Temperature 28.6 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939307723784193, "text": "I hate feeling nauseous �� its scary cuz u dont know if ur going to throw up or not.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2183661721, "name": "Briana_Banana❤[3•3]", "screen_name": "Rebel_Girl_2205", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "Enjoy your Life ❤✨\nBBC❤ VIP❤RoyalTreasure", "followers_count": 31, "friends_count": 346, "statues_count": 4117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939307996467200, "text": "Happy Birthday to the ���� Dr. Seuss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2953258514, "name": "Nick", "screen_name": "nick_park2", "lang": "en", "location": "null", "create_at": date("2014-12-30"), "description": "boomin", "followers_count": 224, "friends_count": 191, "statues_count": 6392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939308164255744, "text": "@Thuggnificent45 ����������������������������", "in_reply_to_status": 704939167097221122, "in_reply_to_user": 494413701, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 494413701 }}, "user": { "id": 467086715, "name": "Natasha Marie", "screen_name": "NarieMatasha", "lang": "en", "location": "null", "create_at": date("2012-01-17"), "description": "null", "followers_count": 500, "friends_count": 327, "statues_count": 55232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-02T00:00:02.000Z"), "id": 704939308608786436, "text": "Son of a bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470498736, "name": "Chris Johnson", "screen_name": "chrisj2211", "lang": "en", "location": "Dearborn", "create_at": date("2012-01-21"), "description": "instagram @pherballs", "followers_count": 462, "friends_count": 416, "statues_count": 11891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn, MI", "id": "339bd7ae6a55ba9f", "name": "Dearborn", "place_type": "city", "bounding_box": rectangle("-83.287094,42.277554 -83.14002,42.35191") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621000, "cityName": "Dearborn" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939308927623170, "text": "Should have formed an offshore corporation...#fraudulent #conveyance", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.73794715,34.2552932"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fraudulent", "conveyance" }}, "user": { "id": 1059335766, "name": "Jeff Hancock", "screen_name": "xpertss97_jeff", "lang": "en", "location": "Simi Valley, CA", "create_at": date("2013-01-03"), "description": "Just a single man trying to change the world but the cops are preventing me from doing anything!", "followers_count": 171, "friends_count": 236, "statues_count": 10644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939309221212160, "text": "Kindergarteners in 7 hours\nyeehah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.4421514,53.56350448"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458161643, "name": "Dave Walker", "screen_name": "kickstart_life", "lang": "en", "location": "Edmonton, Alberta", "create_at": date("2012-01-07"), "description": "Reply Hazy Try Again", "followers_count": 860, "friends_count": 807, "statues_count": 19978 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939309628002306, "text": "\"It's very important you vote this year Carolina, trump can't win he's like Hitler\" true mom true", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 931225418, "name": "Carolina Castellanos", "screen_name": "CarolinaC68", "lang": "en", "location": "PS / SDSU", "create_at": date("2012-11-06"), "description": "null", "followers_count": 659, "friends_count": 475, "statues_count": 6107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939309829328896, "text": "@InThose_Jeans stoppppp ������", "in_reply_to_status": 704934557901053952, "in_reply_to_user": 244277687, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 244277687 }}, "user": { "id": 339218847, "name": "bri baby", "screen_name": "briannaaaaaaax", "lang": "en", "location": "null", "create_at": date("2011-07-20"), "description": "towson university", "followers_count": 493, "friends_count": 479, "statues_count": 19831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939309967745024, "text": "See you in the am LA @ Honolulu International Airport https://t.co/IGwEnF9jv3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.9225,21.31861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78211773, "name": "Whitney Taber", "screen_name": "MsTaber", "lang": "en", "location": " Los Angeles to Seattle", "create_at": date("2009-09-28"), "description": "Radio host on @dash_radio #Hotbutton #VoR host @AllDefDigital #TheCooler Production/ VIP Client coordinator at @recordplant Recording studio. ✌️❤️", "followers_count": 2839, "friends_count": 730, "statues_count": 29625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939309980377090, "text": "L m a o �� https://t.co/2uebKG0uBe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 1368762876, "name": "maaaannni", "screen_name": "edubbzzz", "lang": "en", "location": "Lit Angeles", "create_at": date("2013-04-20"), "description": "null", "followers_count": 1925, "friends_count": 1584, "statues_count": 24829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939310248824833, "text": "Wind 0.0 mph NE. Barometer 29.44 in, Falling very rapidly. Temperature 31.1 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 112, "statues_count": 157932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939310919917568, "text": "SOS why isn't my license plate # on my license this is so upsetting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344785981, "name": "yasmin", "screen_name": "YazBoloori", "lang": "en", "location": "Austin, TX", "create_at": date("2011-07-29"), "description": "If you want crappy things to stop happening to you, then stop accepting crap and demand something more. -Christina Yang", "followers_count": 980, "friends_count": 362, "statues_count": 18997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939311381291008, "text": "I miss watching anime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150156651, "name": "Alex", "screen_name": "MLGSynthetics", "lang": "en", "location": "Los Angeles/London ", "create_at": date("2010-05-30"), "description": "I write my thoughts down and some people like it I guess? Immortal thanks to denial.Gamer/Skater/Music lover. I HATE @_bay21 |League IGN: Synthetics|", "followers_count": 613, "friends_count": 429, "statues_count": 13230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939311574159360, "text": "@GeorgiieDuZziT *16", "in_reply_to_status": 704939148302536704, "in_reply_to_user": 404629464, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 404629464 }}, "user": { "id": 984115110, "name": "Hectuhh", "screen_name": "HectorrTweets", "lang": "en", "location": "California, USA", "create_at": date("2012-12-01"), "description": "Keep it Real Major Key. Proud Bolt #StayInSD", "followers_count": 231, "friends_count": 199, "statues_count": 9325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939311586738176, "text": "@iambeckyg HAPY BIRTHDAY!", "in_reply_to_status": -1, "in_reply_to_user": 128023009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 128023009 }}, "user": { "id": 73147758, "name": "iamAide", "screen_name": "polaxroid", "lang": "en", "location": "ILOVEYOUANDRES", "create_at": date("2009-09-10"), "description": "Justin follows Becky &RahRah are my world and this is andres too", "followers_count": 608, "friends_count": 2062, "statues_count": 27276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell Gardens, CA", "id": "9da8666a1bd226d8", "name": "Bell Gardens", "place_type": "city", "bounding_box": rectangle("-118.170748,33.951888 -118.128456,33.97938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604996, "cityName": "Bell Gardens" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939311901319170, "text": "Guys with a good smile always have my heart❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056484104, "name": "itselveraa♡", "screen_name": "ElveraKali", "lang": "en", "location": "balifornia", "create_at": date("2013-01-02"), "description": "Bitches Everywhere", "followers_count": 511, "friends_count": 294, "statues_count": 13969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, ID", "id": "afc5c9ce888b958d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-116.461472,43.561187 -116.337922,43.662958") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1652120, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939312048173056, "text": "\"What are u doing\" https://t.co/USNnMC3vLZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2392367329, "name": "Fran Grolnick", "screen_name": "Powerhouse548", "lang": "en", "location": "Marina, CA", "create_at": date("2014-03-16"), "description": "I advocate for animals in high kill shelters to get them adoptef in loving furever homes,just a small part.I am also! a die hard TWD lover and fan.", "followers_count": 667, "friends_count": 2106, "statues_count": 1256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marina, CA", "id": "7a3fb038dd5b70e7", "name": "Marina", "place_type": "city", "bounding_box": rectangle("-121.817399,36.652284 -121.741857,36.707451") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 645778, "cityName": "Marina" } }
+{ "create_at": datetime("2016-03-02T00:00:03.000Z"), "id": 704939312484368385, "text": "@FoxNews @realDonaldTrump \"Abysmal\": @ChrisChristie's face during the hostage crisis", "in_reply_to_status": 704937492756062208, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531, 25073877, 1347285918 }}, "user": { "id": 43263991, "name": "Stacy Lillis", "screen_name": "stacylillis", "lang": "en", "location": "Colleyville, TX", "create_at": date("2009-05-28"), "description": "Jesus Freak. Entrepreneur. Sister. Wife. Mom. Want my kids, all kids, to have the free America I did, and better. #CruzCrew", "followers_count": 172, "friends_count": 233, "statues_count": 1869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Lavaca, TX", "id": "3d8ff8d4ade0ee73", "name": "Port Lavaca", "place_type": "city", "bounding_box": rectangle("-96.658423,28.587114 -96.611409,28.649616") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48057, "countyName": "Calhoun", "cityID": 4858916, "cityName": "Port Lavaca" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939313323237377, "text": "#SuperTuesday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SuperTuesday" }}, "user": { "id": 1491374864, "name": "Daniel C.", "screen_name": "_CancholaDaniel", "lang": "en", "location": "Disneyland", "create_at": date("2013-06-07"), "description": "If you don't like Disneyland then I don't think we were ever really friends... I❤️OSO", "followers_count": 281, "friends_count": 59, "statues_count": 25613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939313759444992, "text": "Creep Hours.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.343708,42.450435"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2405721630, "name": "K•", "screen_name": "KennyWarBucks", "lang": "en", "location": "New York, USA", "create_at": date("2014-03-22"), "description": "I'm Just In A Different Space And I Choose To Embrace It | \nIG: kennywarbucks", "followers_count": 293, "friends_count": 289, "statues_count": 378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fredonia, NY", "id": "8d267f94bab958c0", "name": "Fredonia", "place_type": "city", "bounding_box": rectangle("-79.362899,42.414703 -79.294678,42.45723") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36013, "countyName": "Chautauqua", "cityID": 3627419, "cityName": "Fredonia" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939313918844930, "text": "Come near to God and he will come nearby to you.\nJames 4:8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 131, "friends_count": 245, "statues_count": 198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939314430484480, "text": "#HouseBusinessCheck at #FloridaHospital, 601 E Rollins St. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3694779,28.5744461"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HouseBusinessCheck", "FloridaHospital", "orlpol", "opd" }}, "user": { "id": 39049582, "name": "Police Calls 32803", "screen_name": "orlpol32803", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 681, "friends_count": 1, "statues_count": 70318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315214868482, "text": "Thick thighs matter https://t.co/YL52IBytiQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189528231, "name": "La' Perla Figi", "screen_name": "LaMadrina____", "lang": "en", "location": "chasing waterfalls", "create_at": date("2010-09-11"), "description": "cloth cut from a differnt frabic. I am nothing like them. MTC..STAMPED IT!", "followers_count": 906, "friends_count": 218, "statues_count": 44312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315462348800, "text": "U tha fuckin best", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243953133, "name": "dilly", "screen_name": "BasedDilly", "lang": "en", "location": "null", "create_at": date("2011-01-27"), "description": "21 z33 @icylifebrand", "followers_count": 2297, "friends_count": 256, "statues_count": 125155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clover Creek, WA", "id": "6a3c00987122ff28", "name": "Clover Creek", "place_type": "city", "bounding_box": rectangle("-122.415738,47.110842 -122.357057,47.159203") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5313215, "cityName": "Clover Creek" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315655266304, "text": "you know what they say about men with small hands\n\nthey can't pull this country up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116308543, "name": "Elite Four Kyle", "screen_name": "kylegaddo", "lang": "en", "location": "Addison, IL", "create_at": date("2010-02-21"), "description": "Stuff at @novyunlimited. CM at @11x2. Wordmancer at @SaveContinue. Dad at @HeavyBlog. Opinions here are garbage, completely unrelatable, and gluten-free.", "followers_count": 1554, "friends_count": 1377, "statues_count": 143065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315676258306, "text": "#VehicleAccident at 201-598 S Observatory Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4680542,28.5360229"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VehicleAccident", "orlpol", "ocso" }}, "user": { "id": 39060093, "name": "Police Calls 32835", "screen_name": "orlpol32835", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 262, "friends_count": 1, "statues_count": 43782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlovista, FL", "id": "48ddd956df593fc0", "name": "Orlovista", "place_type": "city", "bounding_box": rectangle("-81.475973,28.53214 -81.446957,28.55261") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253100, "cityName": "Orlovista" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315701391361, "text": "Smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1416858571, "name": "GLO☀️", "screen_name": "__blvckjesus", "lang": "en", "location": "Heights", "create_at": date("2013-05-09"), "description": "Its me and my jewelry Im anti-security. Aries♈️", "followers_count": 789, "friends_count": 659, "statues_count": 42676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Euclid, OH", "id": "19cdbb0c29dbc3a5", "name": "South Euclid", "place_type": "city", "bounding_box": rectangle("-81.54638,41.501345 -81.506679,41.54648") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3973264, "cityName": "South Euclid" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315730780160, "text": "@cooolhads you and me both", "in_reply_to_status": 704902830684028928, "in_reply_to_user": 304185261, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304185261 }}, "user": { "id": 894539474, "name": "Alex Crouch", "screen_name": "AlexCrouch2", "lang": "en", "location": "The U.S.A.", "create_at": date("2012-10-20"), "description": "Ole Miss ΠKΦ", "followers_count": 492, "friends_count": 336, "statues_count": 3146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MS", "id": "b86cd6ef7a54d640", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-89.571643,34.321375 -89.495897,34.387965") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28071, "countyName": "Lafayette", "cityID": 2854840, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315734908929, "text": "@emnhill_ is my favorite person to watch scary movies with bc she doesn't scream.", "in_reply_to_status": -1, "in_reply_to_user": 3804695055, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3804695055 }}, "user": { "id": 617263826, "name": "hunty", "screen_name": "HJA903", "lang": "en", "location": "wherever im at, tx ", "create_at": date("2012-06-24"), "description": "Don't stress about life, you don't make it out alive anyway.", "followers_count": 882, "friends_count": 431, "statues_count": 26803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939315818799104, "text": "Temp: 51.9°F Wind:0.7mph Pressure: 30.146hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939316276031488, "text": "Wind 6.0 mph SE. Barometer 29.443 in, Falling Rapidly. Temperature 33.0 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939316582195201, "text": "Can't for April! Dodgers are back and NBA playoffs ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2700203971, "name": "Anthony orduna", "screen_name": "Mvp_024", "lang": "en", "location": "null", "create_at": date("2014-08-02"), "description": "Rapper/producer/Csun", "followers_count": 287, "friends_count": 717, "statues_count": 3719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:04.000Z"), "id": 704939316900978689, "text": "@SelinaMinx Nooooo! You can't make me! �� ....wait.", "in_reply_to_status": 704935193145188352, "in_reply_to_user": 1315483848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1315483848 }}, "user": { "id": 3113198734, "name": "Chrissy_Pup", "screen_name": "Chrissy_Pup", "lang": "en", "location": "Planet with the water on it. ", "create_at": date("2015-03-24"), "description": "+18| An obedient, SOMETIMES bratty puppy. Lover of CBT, chastity, tease/denial and boot licking. Movies, video games, basketball & comics nerd.", "followers_count": 126, "friends_count": 228, "statues_count": 1934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939318742220800, "text": "I got it.. So I guess im a #pierce https://t.co/eMTnGPqVRS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pierce" }}, "user": { "id": 260889098, "name": "QUEEN DILLY", "screen_name": "SimplyDillan", "lang": "en", "location": "null", "create_at": date("2011-03-04"), "description": "relax ya life", "followers_count": 1453, "friends_count": 591, "statues_count": 84064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939318763253761, "text": "Si tuviera los chavos apostaria todo a McGregor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1254496236, "name": "☠ØVMM 6|96☣♏️", "screen_name": "Osvaldo_VJ", "lang": "en", "location": "Sanford, FL", "create_at": date("2013-03-09"), "description": "Forgive me Father , for I am a sinner . Kēêp ît 8+92 , | IG : Osvaldo_Vicente 'lyricist' UC6|96", "followers_count": 1401, "friends_count": 1287, "statues_count": 41825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanford, FL", "id": "e55e8f90b3127585", "name": "Sanford", "place_type": "city", "bounding_box": rectangle("-81.347775,28.738984 -81.218468,28.829384") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1263650, "cityName": "Sanford" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939318859722752, "text": "This day done. Rest now. Tomorrow comes swiftly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278223827, "name": "Kyle Eason", "screen_name": "eason_kyle", "lang": "en", "location": "Denver, CO", "create_at": date("2011-04-06"), "description": "Creative Director Writer Branding Video #VR #PR Software Web Apps #growth @datacultureco @inceptionim @rockethaus @milehighhouse @purpleballllc @platform88net", "followers_count": 3683, "friends_count": 3766, "statues_count": 5781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939319442673665, "text": "We just need s face to face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286029649, "name": "queen", "screen_name": "Stephanie836", "lang": "en", "location": "Wilmington, Los Angeles", "create_at": date("2011-04-21"), "description": "kill them with kindness", "followers_count": 403, "friends_count": 302, "statues_count": 13769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939319459500032, "text": "@iam_miguel_ https://t.co/2hboOKxlSO", "in_reply_to_status": -1, "in_reply_to_user": 2236685298, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2236685298 }}, "user": { "id": 1112608326, "name": "brookę ❧", "screen_name": "brookeavery_", "lang": "en", "location": "hogwarts", "create_at": date("2013-01-22"), "description": "Miguel is the honey", "followers_count": 426, "friends_count": 580, "statues_count": 44779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939319597867008, "text": "@Hillaryclinton https://t.co/NTgB6xqes8", "in_reply_to_status": -1, "in_reply_to_user": 1339835893, "favorite_count": 0, "coordinate": point("-72.895697,41.3182636"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1339835893 }}, "user": { "id": 2899891384, "name": "Den Bac Bacardi", "screen_name": "bac_dennplhpklp", "lang": "en", "location": "denverbacardi@mail.com", "create_at": date("2014-11-30"), "description": "null", "followers_count": 7, "friends_count": 27, "statues_count": 4511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Haven, CT", "id": "795003fb11ee9829", "name": "New Haven", "place_type": "city", "bounding_box": rectangle("-72.998069,41.246404 -72.860248,41.350384") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939320197697536, "text": "\"How easy do you think it would be to sneak into the zoo? I have to see some penguins like RIGHT now.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226737857, "name": "Moon", "screen_name": "Taytay_moon", "lang": "en", "location": "City of Angels", "create_at": date("2010-12-14"), "description": "I'm probably crying about one thing or another...", "followers_count": 401, "friends_count": 737, "statues_count": 19432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939320587784192, "text": "No https://t.co/8hkbdCRYhN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2989079241, "name": "Negrita ❁", "screen_name": "LeslieGochez", "lang": "en", "location": "valleys", "create_at": date("2015-01-20"), "description": "r.i.p ; Jorge Antonio Gochez/grandpa ... • ⚽️⚽️", "followers_count": 508, "friends_count": 460, "statues_count": 3931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-03-02T00:00:05.000Z"), "id": 704939320914878464, "text": "Imma go back to this tweet when Marco Rubio wins the presidential election.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195470923, "name": "jinchuuriki", "screen_name": "RodriguezX_18", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "null", "followers_count": 238, "friends_count": 231, "statues_count": 2709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939321795727360, "text": "2016/04/23: Milwaukee, WI, United States: RUSSIAN ROULETTE Vol. 3 https://t.co/VFI86BQtPT https://t.co/bWZt1z7Pzy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.8814302,43.075164"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 203930370, "name": "DanceDeets", "screen_name": "dancedeets", "lang": "en", "location": "Worldwide", "create_at": date("2010-10-17"), "description": "Tweeing about upcoming events around the world, right when they're added to DanceDeets!", "followers_count": 987, "friends_count": 791, "statues_count": 18783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939322252857344, "text": "going through pictures and seeing how happy we used to be really fucks you up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3716452572, "name": "e.jay_", "screen_name": "ej_milwhore", "lang": "en", "location": "null", "create_at": date("2015-09-28"), "description": "null", "followers_count": 184, "friends_count": 305, "statues_count": 601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanish Springs, NV", "id": "2ef2e027146be550", "name": "Spanish Springs", "place_type": "city", "bounding_box": rectangle("-119.735925,39.621614 -119.675337,39.6768") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268350, "cityName": "Spanish Springs" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939322559094784, "text": "San Diego round 2 this weekend ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700192608, "name": "ca_isaac", "screen_name": "IsaacRobles15", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "FUCK IT UP, FUCK IT UP @Forever_Sam_", "followers_count": 1005, "friends_count": 1114, "statues_count": 11148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939322802315265, "text": "Temp: 11.7°F - Dew Point: -3.3° - Wind: 7.9 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 30.05in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939322882048000, "text": "Redds are fucking disgusting.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220273933, "name": "tgram$", "screen_name": "TgramsThaGreat", "lang": "en", "location": "$$$", "create_at": date("2010-11-27"), "description": "in tha mix'", "followers_count": 1859, "friends_count": 95, "statues_count": 98503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939323200827392, "text": "Happy birthday babe!!!!���� hope you have an amazing day! You deserve it �� https://t.co/BQqWjI3m7t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3232981494, "name": "Jordan", "screen_name": "JordanF1738", "lang": "en", "location": "Long Beach, CA", "create_at": date("2015-06-01"), "description": "@littlekugler❤️", "followers_count": 120, "friends_count": 420, "statues_count": 2249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939323527925762, "text": "I have to be up in three and a half hours. What's sleep???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146839015, "name": "Cam:)", "screen_name": "CamrynKeller99", "lang": "en", "location": "null", "create_at": date("2015-04-09"), "description": "☕️☕️☕️", "followers_count": 197, "friends_count": 192, "statues_count": 920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324085821441, "text": "I'm so fucking mentally and physically tired and exhausted... I need a break from reality.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3556826173, "name": "stephanie olivas", "screen_name": "_stephfoshizzle", "lang": "en", "location": "sc: stephfoshizzle", "create_at": date("2015-09-13"), "description": "null", "followers_count": 723, "friends_count": 207, "statues_count": 1740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324182233090, "text": "@iambeckyg love you so much B �� https://t.co/fd2m3SBDMo", "in_reply_to_status": -1, "in_reply_to_user": 128023009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 128023009 }}, "user": { "id": 2714785449, "name": "Leslie", "screen_name": "leslieelias12", "lang": "en", "location": "null", "create_at": date("2014-07-16"), "description": "\\Why should I change myself to be what everybody else wants me to be/ ~@iambeckyg", "followers_count": 105, "friends_count": 266, "statues_count": 4193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324190625792, "text": "If you're a #Sales professional in #CT, check out this #job: https://t.co/6sUN8uqiJA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.087749,41.6032207"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "CT", "job", "Hiring" }}, "user": { "id": 2835627385, "name": "Sunrise UK Jobs", "screen_name": "SunriseUKjobs", "lang": "en", "location": "null", "create_at": date("2014-09-29"), "description": "We are fully committed to quality care. That's why we need quality people. Follow us to learn more about our current opportunities.", "followers_count": 9, "friends_count": 13, "statues_count": 297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakville, CT", "id": "a91a6db1722d29ce", "name": "Oakville", "place_type": "city", "bounding_box": rectangle("-73.106256,41.568958 -73.057384,41.615593") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9005, "countyName": "Litchfield", "cityID": 956690, "cityName": "Oakville" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324379422728, "text": "Last Week Tonight with John Oliver: Donald Trump (HBO) https://t.co/lLPe1cDEnh #Republican", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Republican" }}, "user": { "id": 3591975733, "name": "Mitzi Aimee Gonzalez", "screen_name": "miaigo", "lang": "en", "location": "null", "create_at": date("2015-09-17"), "description": "#PR student at CSUF | Major interest in current #socialissues | #Opinionated | | If you have a voice, use it! ✊", "followers_count": 89, "friends_count": 142, "statues_count": 344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324933083136, "text": "@bayleeluge Ewww they actually made you pay?? Emily got locked out once and they didn't charge her ��", "in_reply_to_status": 704935743219695617, "in_reply_to_user": 445768089, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 445768089 }}, "user": { "id": 865562256, "name": "Sarah Jackson", "screen_name": "SarahhhJackson3", "lang": "en", "location": "CA||AZ", "create_at": date("2012-10-06"), "description": "||Insta/snapchat: sarahhhjacksonn||GCU'19", "followers_count": 1242, "friends_count": 1106, "statues_count": 8331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-02T00:00:06.000Z"), "id": 704939324958253056, "text": "I NEED TO FIND THESE ASGFJKL �� https://t.co/O8OH57XpiK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1611125173, "name": "Corey", "screen_name": "corneliousdawwg", "lang": "en", "location": "null", "create_at": date("2013-07-21"), "description": "I like to make money, get turnttttttt", "followers_count": 464, "friends_count": 386, "statues_count": 8101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939325901905920, "text": "Guess ion get no attention tonight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4099886925, "name": "Antt Alexander", "screen_name": "glo_juggman", "lang": "en", "location": "Huntsville, AL", "create_at": date("2015-11-02"), "description": "SC : anttgloo_25 ⛽ \n#48HGGM #SouthSide #TeamGwop \nFollow Or Not Ion Give Af....", "followers_count": 259, "friends_count": 628, "statues_count": 2492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939326359076864, "text": "Dos dos https://t.co/tPiebf9NR0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 4471775832, "name": "M E L A N I E", "screen_name": "vixaaan", "lang": "en", "location": "TwoTre", "create_at": date("2015-12-05"), "description": "What that do ?", "followers_count": 75, "friends_count": 62, "statues_count": 4613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939326438809606, "text": "This #Sales #job might be a great fit for you: Call Center Supervisor - https://t.co/ddfuWyiuB5 #jobs #Fairlawn, OH https://t.co/Nz7CN25xCL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.609844,41.127833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sales", "job", "jobs", "Fairlawn" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 316, "friends_count": 215, "statues_count": 2014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairlawn, OH", "id": "4277b55a20443b84", "name": "Fairlawn", "place_type": "city", "bounding_box": rectangle("-81.658213,41.096709 -81.587404,41.148921") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3926166, "cityName": "Fairlawn" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939327667761152, "text": "@AriGotDaJuiceee ������ no , just getting ready to hop in the shower", "in_reply_to_status": 704939076961607681, "in_reply_to_user": 317841834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 317841834 }}, "user": { "id": 229921741, "name": "fortunate.", "screen_name": "mynameSHAWN", "lang": "en", "location": "null", "create_at": date("2010-12-23"), "description": "Founder of ADDICT$HUN. LongLiveQueenAngie ❤️", "followers_count": 965, "friends_count": 709, "statues_count": 25677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939327747448832, "text": "Girl, no. He's Hispanic �� & you know I try and stay away from them bc I heard all about that lmao https://t.co/VHefocKblD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1651948009, "name": "peaches", "screen_name": "Meeuhh", "lang": "en", "location": "this is deeper than rap.", "create_at": date("2013-08-06"), "description": "emotional shawty", "followers_count": 1009, "friends_count": 592, "statues_count": 67840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939328145858560, "text": "3/2/2016 - 02:00\nTemp: 35.2F \nHum: 91%\nWind: 0.0 mph\nBaro: 30.223in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 109, "friends_count": 53, "statues_count": 50470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939328678588416, "text": "Your vibes are...vibey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497450324, "name": "bumbler bee", "screen_name": "Eating_Stars", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "@dispirittx", "followers_count": 503, "friends_count": 566, "statues_count": 53804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939328875712512, "text": "Meet America’s First #Asexual Politician https://t.co/yJQOechNwg via @thedailybeast", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Asexual" }}, "user_mentions": {{ 16012783 }}, "user": { "id": 2206520060, "name": "Jenn Carson", "screen_name": "8675309Tea", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-11-20"), "description": "Lover of pop culture, progressive thought, tea & cats. Advocate for children of prisoners & crime victims. Alum of Baylor, GWU & TFA.", "followers_count": 909, "friends_count": 1601, "statues_count": 965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-03-02T00:00:07.000Z"), "id": 704939329383178240, "text": "HAPPY BIRTHDAY NORMITABISH I love u so much, wish I was there to celebrate ❗️���� Hope it's the best��❤️���� @Ayeenormss https://t.co/HVGXxvP8CV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493723513 }}, "user": { "id": 1110668634, "name": "Breanna Hernandez", "screen_name": "Breannaah", "lang": "en", "location": "null", "create_at": date("2013-01-21"), "description": "SLO down girl #wasntme", "followers_count": 830, "friends_count": 508, "statues_count": 27540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-03-02T00:00:08.000Z"), "id": 704939330364710912, "text": "Wind 0.0 mph N. Barometer 30.196 in, Falling slowly. Temperature 15.8 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-03-02T00:00:08.000Z"), "id": 704939332507926528, "text": "@iambeckyg #HappyBirthdayBeckyYou're a very special person to me u have change my life i love u�� https://t.co/5eGdj7feGf", "in_reply_to_status": -1, "in_reply_to_user": 128023009, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyBirthdayBeckyYou" }}, "user_mentions": {{ 128023009 }}, "user": { "id": 3153107442, "name": "5H2-7/27", "screen_name": "lauren26884113", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-04-13"), "description": "Harmonizer", "followers_count": 260, "friends_count": 351, "statues_count": 602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-03-02T00:00:08.000Z"), "id": 704939332839329792, "text": "@valerialsv haha I will", "in_reply_to_status": 704939084595257345, "in_reply_to_user": 111165920, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 111165920 }}, "user": { "id": 759978301, "name": "edgar zuniga®©", "screen_name": "edgar_elpapii", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "Tiffany is the best girlfriend I could ever ask for she's always make smile when I'm sad she's my everything", "followers_count": 255, "friends_count": 1379, "statues_count": 2935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bithlo, FL", "id": "98ce00cd948ac284", "name": "Bithlo", "place_type": "city", "bounding_box": rectangle("-81.129645,28.543428 -81.09278,28.591199") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1206625, "cityName": "Bithlo" } }
+{ "create_at": datetime("2016-03-02T00:00:08.000Z"), "id": 704939333707538434, "text": "Wind 2.0 mph ENE. Barometer 29.817 in, Falling Rapidly. Temperature 27.8 °F. Rain today 0.08 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301684151558144, "text": "And y'all niggas not going to Canada so stop it...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378787824, "name": "Vandal Savage", "screen_name": "MikeLee______", "lang": "en", "location": "Richmond, VA", "create_at": date("2014-03-08"), "description": "God First Family Second 2⃣4⃣♈️ Canes| Fins | Pistons| Tarheel Basketball| Cubs", "followers_count": 994, "friends_count": 1340, "statues_count": 87270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301684331765760, "text": "@XXL @TefPoe speaks conviction & solidarity within his lyrics/actions - in and outside the booth #FreePalestine https://t.co/OBv6UsRCfE", "in_reply_to_status": -1, "in_reply_to_user": 14629315, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FreePalestine" }}, "user_mentions": {{ 14629315, 40293464 }}, "user": { "id": 169175670, "name": "Alexander Trey ✌️", "screen_name": "AlexzanderTMcD", "lang": "en", "location": "Topeka, Native America", "create_at": date("2010-07-21"), "description": "The Dreamer, The Believer.", "followers_count": 207, "friends_count": 226, "statues_count": 4214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Topeka, KS", "id": "835f1b2948575c51", "name": "Topeka", "place_type": "city", "bounding_box": rectangle("-95.809606,38.971524 -95.571859,39.103634") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20177, "countyName": "Shawnee", "cityID": 2071000, "cityName": "Topeka" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301684625354752, "text": "so cute that @allyyduffy and I have someone who truly cares about our lives more than their own ❤️������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1869767857 }}, "user": { "id": 842790079, "name": "ry", "screen_name": "ryleegrace_15", "lang": "en", "location": "Hanford, CA", "create_at": date("2012-09-23"), "description": "tomorrow will be better || B&A", "followers_count": 1674, "friends_count": 843, "statues_count": 35453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301684843520000, "text": "I'm the most clumsiest person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1970557729, "name": "A.", "screen_name": "_alanisgrijalva", "lang": "en", "location": "null", "create_at": date("2013-10-18"), "description": "Play the dirty version, not the clean one", "followers_count": 312, "friends_count": 262, "statues_count": 15441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301684864548864, "text": "@CBBXpert they are better next year tho just look at rosters", "in_reply_to_status": 705301410078990336, "in_reply_to_user": 3996337415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3996337415 }}, "user": { "id": 69352359, "name": "Zach Braziller", "screen_name": "NYPost_Brazille", "lang": "en", "location": "astoria ", "create_at": date("2009-08-27"), "description": "Sports reporter for the New York Post. Astorian. Google me.", "followers_count": 7402, "friends_count": 2860, "statues_count": 95587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301685074153472, "text": "@t_itran plus this rain too that's coming������", "in_reply_to_status": 705301227114860544, "in_reply_to_user": 333807103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333807103 }}, "user": { "id": 1158659394, "name": "••Rìtçhàá⥕•", "screen_name": "ClubXHype", "lang": "en", "location": "SnapChat-RitcheRichSison", "create_at": date("2013-02-07"), "description": "Yesterday is history, Today is a gift, Tomorrow is a mystery⚠️ #HDYNATION #RaveBooty http://WeTheBestStore.com", "followers_count": 353, "friends_count": 209, "statues_count": 26380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301685996953600, "text": "@cr_nna loving your posts", "in_reply_to_status": 704728035459801088, "in_reply_to_user": 542914958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 542914958 }}, "user": { "id": 2501185880, "name": "Julie Fan of Madonna", "screen_name": "julieroot4", "lang": "en", "location": "Santa Rosa C.A..", "create_at": date("2014-05-17"), "description": "MADONNA FAN FOR 32YRS - love art - gothic Lifestyle - for gay rights own a Shih Tzu I like all types of music & movies - for animal Rights", "followers_count": 9688, "friends_count": 10657, "statues_count": 176732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301686449889280, "text": "������\n.\n.\n.\n#snapchat #movie https://t.co/Qd765V2eVt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "snapchat", "movie" }}, "user": { "id": 1079126886, "name": "anna_fujita⭐藤田杏奈", "screen_name": "anna_fujita", "lang": "ja", "location": "日本 東京都", "create_at": date("2013-01-11"), "description": "A-LIGHT(エーライツ)所属\n\n\n\n\n\n1994.9.22(20) 168cm 股下 90cm \n♡JELLY♡model\n Instagram→@anna_fujita", "followers_count": 24396, "friends_count": 307, "statues_count": 5531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301686542168064, "text": "@dajad_ ain't put you to sleep �� https://t.co/rWrrSDuQkF", "in_reply_to_status": -1, "in_reply_to_user": 2993199345, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2993199345 }}, "user": { "id": 333664755, "name": "Poor Lu'RichGurl", "screen_name": "KeepinUpWitZa_", "lang": "en", "location": "Riley World", "create_at": date("2011-07-11"), "description": "#LiveLoveRileyArmani", "followers_count": 684, "friends_count": 386, "statues_count": 26493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301686835765250, "text": "All the things we could've did and places we could've gone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503570951, "name": "Tatum Preister", "screen_name": "tatumpreister", "lang": "en", "location": "null", "create_at": date("2012-02-25"), "description": "null", "followers_count": 1086, "friends_count": 706, "statues_count": 4261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301687943057408, "text": "bd94015b6ade59dd720a2cc48d06734f3b50c4ff48b73d5be35e61266618394a20d27b8a42d419485d92d29d3ece5671687828577fd1c9e09f8ac708450201b6d20fa0000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.20484234,47.52345516"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 144, "friends_count": 1, "statues_count": 809566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53027, "countyName": "Grays Harbor" } }
+{ "create_at": datetime("2016-03-03T00:00:00.000Z"), "id": 705301688135974912, "text": "Want to work in #Gainesville, GA? View our latest opening: https://t.co/J0duaZ6enU #Nursing #CathLab #NurseLife #Job https://t.co/e2MhD0JE3n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.8240663,34.2978794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gainesville", "Nursing", "CathLab", "NurseLife", "Job" }}, "user": { "id": 234960361, "name": "TravelNursesPHP", "screen_name": "TravelNurseWork", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-01-06"), "description": "Premier Healthcare Professionals is an industry leader in domestic & international healthcare staffing. We have a client base of thousands of hospitals.", "followers_count": 4548, "friends_count": 4830, "statues_count": 5078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, GA", "id": "ec6dac47648ca27f", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-83.89971,34.238672 -83.762889,34.362593") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall", "cityID": 1331908, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301688446357505, "text": "Wouldn't be too upset if someone wanted to order me a pizza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352080106, "name": "Kylo Ren", "screen_name": "warboy__", "lang": "en", "location": "bbb 1991.2012", "create_at": date("2011-08-09"), "description": "UofU premed | NASM CPT | VASA Fitness personal trainer | professional musician | seahawks | nap king | sith lord | nerd god", "followers_count": 795, "friends_count": 268, "statues_count": 10217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301688492556288, "text": "High schoolers should be taught how to play beer pong to learn proper sportsmanship for the future.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2983249237, "name": "Sara Bae Mitchell", "screen_name": "mischiefhandled", "lang": "en", "location": "Under4layersofcarhartt, Utah", "create_at": date("2015-01-14"), "description": "Fuckin Orcas. Am I right?", "followers_count": 330, "friends_count": 755, "statues_count": 4536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ogden, UT", "id": "4980bc1544652d22", "name": "Ogden", "place_type": "city", "bounding_box": rectangle("-112.025874,41.149594 -111.904265,41.286003") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4955980, "cityName": "Ogden" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301688878497793, "text": "All the time ! Lol https://t.co/t6KjqZ3YPb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455366012, "name": "Destiny Reyes", "screen_name": "TheNotoriousDES", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2012-01-04"), "description": "Cuse 2018 - Sup", "followers_count": 262, "friends_count": 256, "statues_count": 4849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301689679544320, "text": "If a person fucks with you they gonna fuck with you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711882464, "name": "Stephon Dupree'", "screen_name": "ContactSwad3", "lang": "en", "location": "Ky ✈️ LA", "create_at": date("2012-07-22"), "description": "Management @yotranemusic | PR consultant | stephondupreepr@gmail.com", "followers_count": 84902, "friends_count": 5632, "statues_count": 53156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301690912649216, "text": "Twitter is dry��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1014687828, "name": "Father", "screen_name": "notoriousfather", "lang": "en", "location": "Manteca, CA", "create_at": date("2012-12-15"), "description": "keep it G'in", "followers_count": 639, "friends_count": 506, "statues_count": 16038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301691055222784, "text": "Waiting on that @FreddieGibbs & @Ye_Ali", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18888713, 2786770105 }}, "user": { "id": 784147746, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "MCMXCV10", "lang": "en", "location": "null", "create_at": date("2012-08-27"), "description": "Planetary Devastation", "followers_count": 563, "friends_count": 454, "statues_count": 1609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301691206234112, "text": "@Qualityislife im better than yoh", "in_reply_to_status": 705298570581159936, "in_reply_to_user": 1406205360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1406205360 }}, "user": { "id": 2842908828, "name": "jeffrey", "screen_name": "GAMZY1210", "lang": "en", "location": "Dallas, Tx ", "create_at": date("2014-10-06"), "description": "im a freak and a coming up AM with a dream to lead period..... hot cheetos", "followers_count": 636, "friends_count": 634, "statues_count": 13578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flower Mound, TX", "id": "d6bbfe2356aebaab", "name": "Flower Mound", "place_type": "city", "bounding_box": rectangle("-97.178734,32.981704 -97.027543,33.078462") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4826232, "cityName": "Flower Mound" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301691621462017, "text": "@JohnnyF_2124 swoop let's go", "in_reply_to_status": 705301656439648256, "in_reply_to_user": 3246447349, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3246447349 }}, "user": { "id": 1043378264, "name": "isaac", "screen_name": "Isaaccguapo", "lang": "en", "location": "Yay Area", "create_at": date("2012-12-28"), "description": "IM ME, I DO ME, AND I CHILL", "followers_count": 1085, "friends_count": 662, "statues_count": 40757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301691713888256, "text": "0h 30m wait time at Northeast Community Health Centre. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.418304,53.604512"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 367, "friends_count": 884, "statues_count": 2894 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301691986358272, "text": "2:00 am. The earliest I'll ever go to sleep this semester...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172099262, "name": "Lilo ⚓️", "screen_name": "EmelyVentura7", "lang": "en", "location": "Kingsville, TX", "create_at": date("2010-07-28"), "description": "I have a pop punk personality with a hardcore soul.", "followers_count": 597, "friends_count": 656, "statues_count": 8175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301692011651072, "text": "Wind 0.0 mph ---. Barometer 30.07 in, Falling slowly. Temperature 41.7 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301692250775552, "text": "#SupportOriginMelissa 30.9°F Wind:1.3mph Pressure: 30.12hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-03T00:00:01.000Z"), "id": 705301692305125376, "text": "it's 12am so technically it's day #18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3086390065, "name": "E V A N", "screen_name": "PrinceTulip_", "lang": "en", "location": "South Reno", "create_at": date("2015-03-15"), "description": "sadboyz:(", "followers_count": 269, "friends_count": 169, "statues_count": 3359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301692837961729, "text": "wonder if i was to text oomf would i get ripped on��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71096318, "name": "Chevy C&R™", "screen_name": "ceedeez_nuts", "lang": "en", "location": "#251 #36617 ", "create_at": date("2009-09-02"), "description": "finesser professor . connected&respected™", "followers_count": 1672, "friends_count": 792, "statues_count": 54043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301692980416512, "text": "Wind 5.0 mph SSE. Barometer 1012.26 mb, Falling Rapidly. Temperature 48.7 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693043310592, "text": "My View �� ∞ https://t.co/8yI3GSPGGg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576436876, "name": "Jérôme Cadiz", "screen_name": "JerThatBear", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "Save life. Best friends: @MelodyWhiskers @dubble_0 Realizer | Runner | Wannabe Model | Dancer | Believer", "followers_count": 118, "friends_count": 105, "statues_count": 1439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693089587200, "text": "I had an attitude the whole day even while I was smacced ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4187938995, "name": "Markee Carroll", "screen_name": "Markee_carroll", "lang": "en", "location": "null", "create_at": date("2015-11-14"), "description": "http://www.iamkittycash.com", "followers_count": 76, "friends_count": 41, "statues_count": 1106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693282578432, "text": "Temp: 44.3°F Wind:0.0mph Pressure: 30.060hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693630558208, "text": "But it was you\nSo I was with it\nThen tell you the truth\nWish we never did it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069350162, "name": "Cedillo✌️", "screen_name": "_Kvssvndru", "lang": "en", "location": "LongLiveSapio", "create_at": date("2013-01-07"), "description": "you a hater hoe n ima greater hoe", "followers_count": 1243, "friends_count": 502, "statues_count": 202417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693630697472, "text": "03/03@03:00 - Temp 28.7F, WC 21.5F. Wind 7.0mph NNW, Gust 12.0mph. Bar 30.223in, Rising slowly. Rain 0.00in. Hum 53%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301693920104448, "text": "Wind 1.0 mph E. Barometer 30.171 in, Falling slowly. Temperature 32.3 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301694142357504, "text": "Oh me too https://t.co/GOaFq1VWte", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346582690, "name": "THRIFTSTOREJUNKIEY_", "screen_name": "Beenbahd_DUH", "lang": "en", "location": "inyourBITCHESmouth", "create_at": date("2011-08-01"), "description": "sprinkling fairy dust on yo bitch☄✨", "followers_count": 1293, "friends_count": 1399, "statues_count": 77376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucker, GA", "id": "a4e8b6fd61c8026b", "name": "Tucker", "place_type": "city", "bounding_box": rectangle("-84.252764,33.815828 -84.173182,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301694175776769, "text": "https://t.co/aN2EmwlMnj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 363439529, "name": "m", "screen_name": "ITAL1ANSTALLION", "lang": "en", "location": "DALLAS", "create_at": date("2011-08-27"), "description": "really freakin awkward", "followers_count": 432, "friends_count": 215, "statues_count": 25597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301694188527616, "text": "Wind 0.0 mph ---. Barometer 29.970 in, Falling Rapidly. Temperature 40.0 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695010439168, "text": "The changes I'm going through are incredible idk why I didn't do this earlier", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550523971, "name": "Andres", "screen_name": "ballinandy24", "lang": "en", "location": "Lynwood,CA", "create_at": date("2012-04-10"), "description": "Austin 3:16 Know your role and shut your mouth", "followers_count": 763, "friends_count": 376, "statues_count": 211547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695140487168, "text": "I love when my dog wakes up and snuggles up closer to me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414440021, "name": "majairaa⚾️", "screen_name": "majairaa", "lang": "en", "location": "null", "create_at": date("2011-11-16"), "description": "ma-highhh-rah! life is a short hop, DIG IT ⚾️ IG/SC: majairaa crazy dog lady", "followers_count": 482, "friends_count": 373, "statues_count": 43886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695681564673, "text": "Happy birthday to my best friend @HartzheimTyler #ttl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ttl" }}, "user_mentions": {{ 507792369 }}, "user": { "id": 515019263, "name": "Cowey", "screen_name": "ColeyOBrienRui1", "lang": "en", "location": "Simi Valley Ca", "create_at": date("2012-03-04"), "description": "#dodgers #lakers\nEmployed at Glendale Infiniti. Come see me!", "followers_count": 278, "friends_count": 260, "statues_count": 12731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695719321600, "text": "@NegativexxaM lmaooooo okay ����", "in_reply_to_status": 705301460297216000, "in_reply_to_user": 220231979, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 220231979 }}, "user": { "id": 154384518, "name": "⠀⠀⠀⠀", "screen_name": "xjamiemartinez", "lang": "en", "location": "lawrence ❥", "create_at": date("2010-06-10"), "description": "null", "followers_count": 1753, "friends_count": 503, "statues_count": 24014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695736258560, "text": "Red Gorilla Festival, Austin, TX https://t.co/pXpkgLOzzn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2466843846, "name": "OHLAYINDIGO", "screen_name": "OHLAYINDIGO", "lang": "en-gb", "location": "San Francisco/Oslo", "create_at": date("2014-04-27"), "description": "Offical twitter page for Norwegian Electronic Pop artist @Ohlayindigo // http://www.ohlayindigo.com", "followers_count": 854, "friends_count": 324, "statues_count": 745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695765417985, "text": "Wind 0.7 mph WSW. Barometer 30.14 in, Rising slowly. Temperature 18.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 112, "statues_count": 157956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301695920631808, "text": "Want to work in #Newton, IA? View our latest opening: https://t.co/qmLp4EwVCr #Nursing #RN #Nursing #Job #Jobs https://t.co/Nwa2TKrvVz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.0469444,41.6986111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Newton", "Nursing", "RN", "Nursing", "Job", "Jobs" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 172, "friends_count": 99, "statues_count": 3009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, IA", "id": "227489a8b6bf0d8d", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-93.084674,41.674076 -92.999909,41.729725") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19099, "countyName": "Jasper", "cityID": 1956505, "cityName": "Newton" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301696163901440, "text": "Auhhhh screaming ��������, All That Damn Money, If She Don't Buy Him Some Retrospect �� https://t.co/MtTXQFliAl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2436628256, "name": "Chosen One", "screen_name": "P_Body5", "lang": "en", "location": "College Of Canyons", "create_at": date("2014-04-09"), "description": "Snapchat: Pjwhookid -Im The Next Big Thing In The Family, Athlete: Football And Basketball :19years Old #RipKyree Ill See You Soon #UglyBoyzEnt #ProEra", "followers_count": 792, "friends_count": 907, "statues_count": 28672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-03-03T00:00:02.000Z"), "id": 705301696436523008, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/2hwvjXC0z3 #GoldenValley, MN #Hiring https://t.co/EnmdVrJdLH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.3600034,44.9917149"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "GoldenValley", "Hiring" }}, "user": { "id": 2327706649, "name": "Sunrise Careers", "screen_name": "Sunrise_Careers", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "We are always seeking exceptional people with a passion for working with seniors, and dedicated to caring for others.", "followers_count": 365, "friends_count": 9, "statues_count": 3115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Golden Valley, MN", "id": "e314d126e15cdbfa", "name": "Golden Valley", "place_type": "city", "bounding_box": rectangle("-93.400969,44.966485 -93.318318,45.016929") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2724308, "cityName": "Golden Valley" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301696692547585, "text": "03:00:02 |Temp: 33.2ºF | Wind Chill 33.2ºF |Dew Point 30.0ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NW, Gusting to 1.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301696709206016, "text": "�������� https://t.co/mjdzQRlmkZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 497553314, "name": "Papí Charles ♿️", "screen_name": "GoldPackNation", "lang": "en", "location": "Cisco, TX", "create_at": date("2012-02-19"), "description": "Got everything in my mama name but that's okay cuh I'm still fly! Mathematics major ⚫️ OKC 41-17 Peace Be Upon You", "followers_count": 1066, "friends_count": 1060, "statues_count": 65699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cisco, TX", "id": "721e5b312b06ad49", "name": "Cisco", "place_type": "city", "bounding_box": rectangle("-99.004915,32.374537 -98.956093,32.401675") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48133, "countyName": "Eastland", "cityID": 4815004, "cityName": "Cisco" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301696902074368, "text": "61.6F (Feels: 61.6F) - Humidity: 89% - Wind: 7.6mph E - Gust: 11.4mph - Pressure: 1025.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 229081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301697090850816, "text": "Goodnight fam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535657576, "name": "♎️", "screen_name": "ManzoLowksz", "lang": "en", "location": "null", "create_at": date("2012-03-24"), "description": "DWMTM", "followers_count": 403, "friends_count": 373, "statues_count": 12591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301697472569344, "text": "HAPPY BDAY TO MY FAV PERSON EVER ADRIANA CASTILLO (ELLIE) have a great day love you ❤️ @adrianaellie https://t.co/lboMgSHtQk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1340605760 }}, "user": { "id": 2848378249, "name": "andres", "screen_name": "andresrubioo7", "lang": "en", "location": "RM❤️", "create_at": date("2014-10-08"), "description": "suh dude @mikaylarice_ @hollyn_deakins", "followers_count": 1543, "friends_count": 911, "statues_count": 2075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301697514475523, "text": "@BootsBryantPR ugh I know the feeling I've been banned from a lot of websites. It got so bad I still have furries ask me if I still do art.", "in_reply_to_status": 705300245920555008, "in_reply_to_user": 155700583, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155700583 }}, "user": { "id": 30214298, "name": "Lewd", "screen_name": "Monsterbait", "lang": "en", "location": "Denver", "create_at": date("2009-04-10"), "description": "Monsterbait.me encompasses the erotic art of Lewd. Your source for demons, alien creatures, and other beastly denizens of the perverted underworld!", "followers_count": 916, "friends_count": 1756, "statues_count": 7484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301697854189568, "text": "@Imaginedragons it was so good guys!!! I sang and movie theater chair danced the whole time!!", "in_reply_to_status": 705281005368774656, "in_reply_to_user": 75916180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75916180 }}, "user": { "id": 73050435, "name": "Jolene Painter", "screen_name": "jolenedpainter", "lang": "en", "location": "San Diego CA", "create_at": date("2009-09-09"), "description": "null", "followers_count": 74, "friends_count": 198, "statues_count": 1428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Encinitas, CA", "id": "1f6b47c3f3352385", "name": "Encinitas", "place_type": "city", "bounding_box": rectangle("-117.312091,32.999469 -117.195721,33.090549") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622678, "cityName": "Encinitas" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301697900339200, "text": "I died when I saw the interview where KHLOE said she acted like a dog as a kid and one time she bit a mans leg on a cruise ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460626537, "name": "A l e x a n d r a", "screen_name": "alexxalex33", "lang": "en", "location": "SanAntonio texas/210", "create_at": date("2012-01-10"), "description": "Southern ~ I like adventures|My goal is to meet khloe kardashian|18years|Texas| I make beats|BHS|RAPPERS HMU FOR BEATS", "followers_count": 1121, "friends_count": 2247, "statues_count": 40792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301698093273088, "text": "Love must be sincere.\nHate what is evil; \nCling to what is good.\nRomans 12:9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 133, "friends_count": 247, "statues_count": 202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301698168770560, "text": "I don't lose sleep over the opinion of sheep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4740480438, "name": "13 DAYS", "screen_name": "LeggoMyDiegoo_", "lang": "en", "location": "Tucson, AZ", "create_at": date("2016-01-10"), "description": "God is love ❤️", "followers_count": 330, "friends_count": 294, "statues_count": 1420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301698844033024, "text": "Olympus Corporation of the Americas and Subsidiary Reach Settlement Agreements with the DOJ - CENTER VALLEY, Pa., .. https://t.co/r9OAUUX5Dj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598077903, "name": "Angel Gabriel Loya", "screen_name": "angelloya_loya", "lang": "en", "location": "San Jose, Ca ", "create_at": date("2012-06-02"), "description": "null", "followers_count": 321, "friends_count": 1212, "statues_count": 10275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301698940633089, "text": "Can you find Savannah on the map? Just try it at https://t.co/eoyuorvZWJ #Savannah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.0998,32.0835"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Savannah" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1032, "friends_count": 311, "statues_count": 2629837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301699158605825, "text": "@KDTrey5 make sure yall keep playing all around b ball for all 4quarters,trust ur team more����keep the faith thunder up����", "in_reply_to_status": -1, "in_reply_to_user": 35936474, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35936474 }}, "user": { "id": 2865281663, "name": "J VAL King", "screen_name": "jvalking7_j", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "producer/artist", "followers_count": 726, "friends_count": 2162, "statues_count": 4531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301699322204160, "text": "a2870ad300p75A53", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-163.25554709,56.95524815"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 9, "friends_count": 0, "statues_count": 2397 }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301699674578945, "text": "Honestly I love that my schedule is Monday-Friday at my job. And it's night shift lol I love night shifts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311030731, "name": "gee✨", "screen_name": "geraldiineee_", "lang": "en", "location": "Lancaster, Ca", "create_at": date("2011-06-04"), "description": "eleven/sixteen.", "followers_count": 279, "friends_count": 281, "statues_count": 10566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301699724881925, "text": ".@iamKennyHolland's snapchat is always a work of art! It's too hard to pick out a fav moment!\nSc: kennyholland https://t.co/ImZimjJsk5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 434700167 }}, "user": { "id": 970843968, "name": "Bryan Johnson", "screen_name": "Bryan_L_Johnson", "lang": "en", "location": "Lake Forest, CA", "create_at": date("2012-11-25"), "description": "Loves Collins Key, Blizzard games, Dylan Dauzat, Weekly Chris, comedies, Star Trek, Breaking Bad, Walking Dead, Marvel movies/TV, 1D, 5SOS.\n\nA proud nerd", "followers_count": 1684, "friends_count": 840, "statues_count": 56020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, CA", "id": "3a16272f05577d7e", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-117.72136,33.606781 -117.621862,33.685914") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639496, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301700165259264, "text": "Lol I feel like shit now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406828812, "name": "Chhrisstiina⛹", "screen_name": "Chhrisstiina", "lang": "en", "location": "Eagle Rock, C.A", "create_at": date("2011-11-06"), "description": "ERHS| Var Bball #12|", "followers_count": 478, "friends_count": 662, "statues_count": 4238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, CA", "id": "e6fc035b8939bd3b", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-118.06946,34.087059 -117.99218,34.180507") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602462, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-03-03T00:00:03.000Z"), "id": 705301700735664128, "text": "�� https://t.co/lhKRe997Fb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2339785644, "name": "offline", "screen_name": "RairOnline", "lang": "en", "location": "null", "create_at": date("2014-02-11"), "description": "gonebad.usa", "followers_count": 460, "friends_count": 395, "statues_count": 13914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301703554310144, "text": "Today is historically the worst day of my life but hey, I'm smart, cute, my dick is huge and I'm a compulsive liar.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296565875, "name": "Tones", "screen_name": "whatyallldoin", "lang": "en", "location": "@herbology99", "create_at": date("2011-05-10"), "description": "20. Very cute and popular. In love with my husband @amgeeked. @smokindicks fan account. Property of the JizzLord™ @BBWslayer666 @realsunglassdog @darrylyrn_", "followers_count": 1121, "friends_count": 605, "statues_count": 47705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704254685184, "text": "Here we go..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3080263400, "name": "Mama ❤", "screen_name": "KingZarya", "lang": "en", "location": "null", "create_at": date("2015-03-14"), "description": "...", "followers_count": 773, "friends_count": 1100, "statues_count": 1997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberal, KS", "id": "61fd29934c414a67", "name": "Liberal", "place_type": "city", "bounding_box": rectangle("-100.976544,37.017814 -100.895167,37.070314") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20175, "countyName": "Seward", "cityID": 2039825, "cityName": "Liberal" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704279875584, "text": "I can't sing w Bryson Tiller, he be doing too much��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326951030, "name": "Feb.9th", "screen_name": "NiaLCreecy", "lang": "en", "location": "null", "create_at": date("2011-06-30"), "description": "19. UCSB'19. Dallas ✈️ Santa Barbara, CA☀️ @todd_rachel is the best friend.", "followers_count": 1171, "friends_count": 1081, "statues_count": 19707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704321818625, "text": "@rich0H It's a test.", "in_reply_to_status": 705297521535381504, "in_reply_to_user": 359853655, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 359853655 }}, "user": { "id": 796601, "name": "Andrey Petrov", "screen_name": "shazow", "lang": "en", "location": "Canadian in Bayrea, California", "create_at": date("2007-02-26"), "description": "Go, Python, Javascript, and Pilot G-Tec-C4 scribe. I build many things—mostly open source, some for profit, all for fun. YC alum and Xoogler. Cat person.", "followers_count": 2875, "friends_count": 451, "statues_count": 26124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704800116736, "text": "@rachkautz @Megliss I'm just over here at work bored talking to myself because you like your puppies more than late night Twitter #hurt", "in_reply_to_status": 705301005538177024, "in_reply_to_user": 262530377, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hurt" }}, "user_mentions": {{ 262530377, 1583379060 }}, "user": { "id": 240842420, "name": "Kevin Fink", "screen_name": "KFink61", "lang": "en", "location": "Eastlake, OH", "create_at": date("2011-01-20"), "description": "Im pretty much a big deal.", "followers_count": 695, "friends_count": 1110, "statues_count": 18142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentor, OH", "id": "a47c7932f8997705", "name": "Mentor", "place_type": "city", "bounding_box": rectangle("-81.391672,41.640241 -81.288901,41.75771") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3949056, "cityName": "Mentor" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704858796032, "text": "Enough is enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58703165, "name": "King B of UPTOWN", "screen_name": "realkingb", "lang": "en", "location": "VA-ATL Upt. to Black Hollywood", "create_at": date("2009-07-20"), "description": "#DGE Contact: kingbofuptown@gmail.com PLAN B the EP coming soon", "followers_count": 450, "friends_count": 477, "statues_count": 20465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-03-03T00:00:04.000Z"), "id": 705301704934182912, "text": "My new @ is @MyxomatosisLuke and I'm really fxckin proud. @Luke5SOS I hope you're proud too. I really do because Radiohead is life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3625108334, 403245020 }}, "user": { "id": 3625108334, "name": "Lumpy Space Dork", "screen_name": "MyxomatosisLuke", "lang": "en", "location": "Hogwarts", "create_at": date("2015-09-20"), "description": "^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^", "followers_count": 1176, "friends_count": 3402, "statues_count": 1869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301705768960001, "text": "Wind 4.1 mph NW. Barometer 30.142 in, Rising. Temperature 7.8 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301706339282945, "text": "Saving this pic for future references https://t.co/GTsQp8r1np", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 398383067, "name": "Demarkus.", "screen_name": "NoGoldChains", "lang": "en", "location": "dallas,tx", "create_at": date("2011-10-25"), "description": "I aint shit...... yet, but im boosted.", "followers_count": 1391, "friends_count": 987, "statues_count": 103075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301706410582016, "text": "@cassidy_andrea @GMB @GerardButler X", "in_reply_to_status": 705121068135129089, "in_reply_to_user": 1919073720, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1919073720, 19424427, 1161493932 }}, "user": { "id": 223399512, "name": "Ross King", "screen_name": "TheRossKing", "lang": "en", "location": "Los Angeles", "create_at": date("2010-12-06"), "description": "TV & Radio Host Actor Producer,failed footballer,best selling Author (Taking Hollywood Shari King ...I'm the King half!) Good Morning Britain ITV Lorraine.", "followers_count": 79694, "friends_count": 805, "statues_count": 30553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301706859360256, "text": "@Cargorabbit you have them say a list of things.", "in_reply_to_status": 705300909551554560, "in_reply_to_user": 2316027446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2316027446 }}, "user": { "id": 2201987750, "name": "✧ maz!", "screen_name": "mazzaful", "lang": "en", "location": "[47.4886° N, 117.5786° W], WA ", "create_at": date("2013-11-18"), "description": "Artist and computer support technician.", "followers_count": 464, "friends_count": 328, "statues_count": 20968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301707048165376, "text": "Thanks Justin! ���� https://t.co/RYkcqJqBUg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737956489, "name": "Livid", "screen_name": "P_A_S_I_O_N_", "lang": "en", "location": "Bay Area", "create_at": date("2012-08-04"), "description": "5'4ever | I love Z4brina", "followers_count": 442, "friends_count": 336, "statues_count": 11598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301707069128704, "text": "@amin2074 :|", "in_reply_to_status": 703240280963559425, "in_reply_to_user": 2824602254, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2824602254 }}, "user": { "id": 2824602254, "name": "امـــين⊙⊙", "screen_name": "amin2074", "lang": "en", "location": "karmania", "create_at": date("2014-09-21"), "description": "⭐️⭐️⭐️⭐️⭐️", "followers_count": 3888, "friends_count": 1264, "statues_count": 25780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Campbell, CA", "id": "0354c827bfda68de", "name": "Campbell", "place_type": "city", "bounding_box": rectangle("-121.991728,37.254665 -121.918729,37.307009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 610345, "cityName": "Campbell" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301707685756928, "text": "my playlist freaky as shit lawdcy lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29922435, "name": "Jj.", "screen_name": "BITCHUKNEWW", "lang": "en", "location": "null", "create_at": date("2009-04-08"), "description": "ain't no time bitch.", "followers_count": 669, "friends_count": 77, "statues_count": 34468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Miami, FL", "id": "cae987165260f81f", "name": "North Miami", "place_type": "city", "bounding_box": rectangle("-80.227398,25.880982 -80.127938,25.930229") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1249450, "cityName": "North Miami" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301707807264768, "text": "RT @WellandGoodNYC Flywheel is making big changes to “Power Your Life” https://t.co/m1UUi1j0Kb #Good_Sweat #healthy #nutrition", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.71894531,45.53900337"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Good_Sweat", "healthy", "nutrition" }}, "user_mentions": {{ 50292966 }}, "user": { "id": 55055830, "name": "Sean Harvey", "screen_name": "OneMileMarathon", "lang": "en", "location": "Portland, OR", "create_at": date("2009-07-08"), "description": "Tired of being a 470 lb man, I decided to fight back. Since 04/2010, I am down over 165 lbs & counting. I am a loving husband & father, clean eater & sports fan", "followers_count": 2275, "friends_count": 2259, "statues_count": 15005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301708063244289, "text": "Wind 7.3 mph WNW. Barometer 30.232 in, Rising Rapidly. Temperature 15.8 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301708067332096, "text": "Last tweet as a 17 year old goes out to my boy @Bryce_parsons11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2470100564 }}, "user": { "id": 2511721319, "name": "The GaZelle", "screen_name": "Mr_nava_", "lang": "en", "location": "Santee, CA", "create_at": date("2014-04-26"), "description": "Hey I'm ugly but I bet I'll make you laugh", "followers_count": 625, "friends_count": 569, "statues_count": 10221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santee, CA", "id": "04ed8141edf1aba0", "name": "Santee", "place_type": "city", "bounding_box": rectangle("-117.039886,32.814991 -116.936159,32.887369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 670224, "cityName": "Santee" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301708474327040, "text": "VirtueGirls #Share #MarchMadness #GospelMusic tune in & get your praise on with new artist & new music https://t.co/dGy8aq1Xh6 #Retweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Share", "MarchMadness", "GospelMusic", "Retweet" }}, "user": { "id": 27826482, "name": "Andr'e L. Carter", "screen_name": "Synergy1Radio", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-03-30"), "description": "Editor for #Gospel Synergy Magazine. Program Director for #Synergy1Radio. http://www.gospelsynergy.com Advertise, Market & Promote your business!", "followers_count": 17426, "friends_count": 17261, "statues_count": 542025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calumet City, IL", "id": "6b745382f2cdbb67", "name": "Calumet City", "place_type": "city", "bounding_box": rectangle("-87.579081,41.581483 -87.525064,41.644809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1710487, "cityName": "Calumet City" } }
+{ "create_at": datetime("2016-03-03T00:00:05.000Z"), "id": 705301709094932480, "text": "Right back at you lil moma ✨ https://t.co/Qm0gK06tZ4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2480918190, "name": "3LY$$A", "screen_name": "elyssah8tsu", "lang": "en", "location": "null", "create_at": date("2014-05-06"), "description": "GGHS", "followers_count": 429, "friends_count": 406, "statues_count": 8223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709195587584, "text": "@Alex_Wiley ��", "in_reply_to_status": 705238945676681218, "in_reply_to_user": 131087290, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 131087290 }}, "user": { "id": 18586171, "name": "fetti fitz", "screen_name": "thomasfitzner", "lang": "en", "location": "chamber of secrets", "create_at": date("2009-01-03"), "description": "lost boy", "followers_count": 593, "friends_count": 974, "statues_count": 18966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709275312128, "text": "#3 TEXAS PREMIERE LOOK was just slayin me it gave me New York Fashion week vibes and I was loving that! @ThorgyThor https://t.co/DdkYaUz2mo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 336338772 }}, "user": { "id": 2523839736, "name": "silicone_barbie", "screen_name": "DragRace_Fan", "lang": "en", "location": "null", "create_at": date("2014-05-25"), "description": "All About #RPDR", "followers_count": 848, "friends_count": 899, "statues_count": 2341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709480792064, "text": "massage my back for a blunt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 817718653, "name": "captain jake", "screen_name": "jake_ledesma", "lang": "en", "location": "San Benito, TX", "create_at": date("2012-09-11"), "description": "Fuck yes, im so odd...yamum......uhhhhh leader of the sasquatch gang...oh oh and blunts!", "followers_count": 628, "friends_count": 380, "statues_count": 22893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Benito, TX", "id": "b8f3a7fd432ec1f1", "name": "San Benito", "place_type": "city", "bounding_box": rectangle("-97.684281,26.07277 -97.575927,26.172319") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4865036, "cityName": "San Benito" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709564846080, "text": "I'm always up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595118877, "name": "♬Lil'J♛YoungKillaz♬", "screen_name": "LilJ_ABeast", "lang": "en", "location": "HTX ", "create_at": date("2012-05-30"), "description": "#YoungKillazGang #YKG #StillW8N OUT NOW GO DOWNLOAD MY SHYT❗❗❗ Follow me on IG: @Lil_J_YKG \nNew Video out now click the link", "followers_count": 6265, "friends_count": 4512, "statues_count": 27399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709585670144, "text": "These the type of females who listen to Kodak Black https://t.co/S8yAlN0GFB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1607357000, "name": "Azie", "screen_name": "josedagawd", "lang": "en", "location": "IE", "create_at": date("2013-07-19"), "description": "only interested in @AlexissxChanel", "followers_count": 579, "friends_count": 369, "statues_count": 44828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301709963145216, "text": "#3AMWeMeetAgain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "3AMWeMeetAgain" }}, "user": { "id": 1075248840, "name": "LENNY: The Red Mamba", "screen_name": "redheadLEN", "lang": "en", "location": "Bronx, NY", "create_at": date("2013-01-09"), "description": "IG: redheadLEN • @WorldFamousNYC • Shisha Sundays • #HorrorWorld • #WorldBBQ • #TheGreatestDayEver • #HoopsShowdown • #FlagFootballShowdown • #3AmWeMeetAgain", "followers_count": 1562, "friends_count": 1001, "statues_count": 59606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301711674433536, "text": "friends can really turn your day around ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593174948, "name": "victoria", "screen_name": "victoriaolavide", "lang": "en", "location": "camarillo || carlsbad", "create_at": date("2012-05-28"), "description": "(๑˃̵ᴗ˂̵)و", "followers_count": 179, "friends_count": 162, "statues_count": 2072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301711800242177, "text": "Temp: 22.1°F - Dew Point: 10.4° - Wind: 2.5 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 30.06in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301711842246656, "text": "Cause I can pull a butch wit my eyes closed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1363693711, "name": "#laf ひ", "screen_name": "lafinestt", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "my life is like a sandwich either way you flip it my bread come first", "followers_count": 1178, "friends_count": 692, "statues_count": 29279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301711884144645, "text": "Omg why does this always happen!!!! Always being put to a test with making the right decision ugh. Whhhyyyy??!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3189318601, "name": "Jae Bee", "screen_name": "jaziejayy", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 121, "friends_count": 348, "statues_count": 1898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301712202891265, "text": "@Neva_Sarahi stfu", "in_reply_to_status": 705301669190340608, "in_reply_to_user": 2406332455, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2406332455 }}, "user": { "id": 4406049972, "name": "Alex Campos", "screen_name": "AlexCam_97", "lang": "en", "location": "Austin, TX", "create_at": date("2015-11-29"), "description": "18. Student.", "followers_count": 50, "friends_count": 53, "statues_count": 360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.50077") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301712530251776, "text": "@Kikierawilli SAMEEEEE https://t.co/5eU72sXisk", "in_reply_to_status": 705301037335371776, "in_reply_to_user": 2499923529, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2499923529 }}, "user": { "id": 262037774, "name": "+Elizabeth Jane+", "screen_name": "itslizzyjane", "lang": "en", "location": "Las Vegas, NV. UNLV Film Dept.", "create_at": date("2011-03-06"), "description": "Photographer - Actress - Musician - Film Maker - Artist - Marketing Coordinator - instagram: itslizzyjane", "followers_count": 5282, "friends_count": 898, "statues_count": 2597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301712643313664, "text": "Woke up and can't fall back asleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240578568, "name": "Be Great", "screen_name": "SmooveReggie", "lang": "en", "location": "Dallas, tx", "create_at": date("2011-01-19"), "description": "Im just a basic guy with extrodinary qualities . I set goal and then acheive them. #SFA #MOA #LimitlessMinistry ⚠️caution its just twitter⚠️", "followers_count": 724, "friends_count": 641, "statues_count": 25369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-03-03T00:00:06.000Z"), "id": 705301712953716736, "text": "Lmao 'que estas mirando Jazmin'", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 482640889, "name": "Jazmin Aviña", "screen_name": "jazzzzxoxo", "lang": "en", "location": "Winters, CA", "create_at": date("2012-02-03"), "description": "sc: jazminavinaa", "followers_count": 596, "friends_count": 323, "statues_count": 26999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-03-03T00:00:07.000Z"), "id": 705301713729818624, "text": "#DesignatedPatrolAreaAvailable at Curry Ford Rd & Morristown Ave. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.344462,28.5241652"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DesignatedPatrolAreaAvailable", "orlpol", "opd" }}, "user": { "id": 39055366, "name": "Police Calls 32806", "screen_name": "orlpol32806", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 563, "friends_count": 1, "statues_count": 47928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-03T00:00:07.000Z"), "id": 705301713931083776, "text": "@PapaSwampp I am man !!", "in_reply_to_status": 705301170915577856, "in_reply_to_user": 554343211, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 554343211 }}, "user": { "id": 270687789, "name": "$asha", "screen_name": "_xADORESasha", "lang": "en", "location": "ig: _slimgawdess", "create_at": date("2011-03-22"), "description": "⚡️ goin brazy with the cash flow #bitchesisbittermywaistgettingslimmer #getwitthewinningteam @maniarianna_ ❤️ #CCSU19", "followers_count": 2828, "friends_count": 1369, "statues_count": 171101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Britain, CT", "id": "f080a098cad5b04d", "name": "New Britain", "place_type": "city", "bounding_box": rectangle("-72.826538,41.643749 -72.748011,41.715995") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 950370, "cityName": "New Britain" } }
+{ "create_at": datetime("2016-03-03T00:00:07.000Z"), "id": 705301714090340352, "text": "“peace and resentment cannot inhabit the same body.”", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548773915, "name": "• Alexandria Smith •", "screen_name": "AyeTigerLily", "lang": "en", "location": "LA to LV ", "create_at": date("2012-04-08"), "description": "My friends call me Tiger . Instagram•@ayetigerlily ⠀⠀Contact Email : Ayetigerlily@gmail.com", "followers_count": 457, "friends_count": 132, "statues_count": 8549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-03T00:00:07.000Z"), "id": 705301714853695493, "text": "@jessie__shields @SalasSadie yes", "in_reply_to_status": 705269629627736065, "in_reply_to_user": 859383205, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 859383205, 1385829666 }}, "user": { "id": 594340460, "name": "The Original Bunting", "screen_name": "lukeak65", "lang": "en", "location": "kodiak, Alaska", "create_at": date("2012-05-29"), "description": "tonight is what you make it, take it don't waste it", "followers_count": 333, "friends_count": 218, "statues_count": 6606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072113455104, "text": "#HolyIntel #JesusIsGodMadeFlesh #CHINA #NKorea #SoCHINASEA\n\nPropheticWord for #USA from Jesus to me for all #Jesus\n\nhttps://t.co/IOkFZeBk0V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HolyIntel", "JesusIsGodMadeFlesh", "CHINA", "NKorea", "SoCHINASEA", "USA", "Jesus" }}, "user": { "id": 37398878, "name": "michelle meyer", "screen_name": "MichelleMeyer10", "lang": "en", "location": "North East Coast", "create_at": date("2009-05-03"), "description": "Singer/Songwriter/Actress/Entrepreneur/Mom/Warrior/Friend/Sister/ChildofGod/TeaPartyLeader/FFLDCTY #CCOT #Jesus #1 @PushBackNow @DrDaveOrts =My Husband", "followers_count": 6065, "friends_count": 6671, "statues_count": 65230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, MN", "id": "62619a76134ad05e", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-92.282745,46.650616 -91.962935,46.866053") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2717000, "cityName": "Duluth" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072365142016, "text": "@Samanthaaxmarie https://t.co/3Ha1i66XF1", "in_reply_to_status": 705663834405470208, "in_reply_to_user": 126968172, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 126968172 }}, "user": { "id": 36509209, "name": "S. Kardashian-West", "screen_name": "MrKingWoodrow", "lang": "en", "location": "University of North Texas", "create_at": date("2009-04-29"), "description": "Sigma Nu President • BSU • NT40 • Leadership Fanatic I'm an enigma, wrapped in sass, inside a baddie.", "followers_count": 908, "friends_count": 407, "statues_count": 62166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072402927616, "text": "It's all a scheme....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 121854073, "name": "El Ever", "screen_name": "Ever9393", "lang": "en", "location": "Denver, Co.", "create_at": date("2010-03-10"), "description": "22 | Saxofonista de Peligro Norte! | Music | Santa Maria Del Oro | Durango | #Followme| =) | Instagram: ever9393 | https://www.facebook.com/Ever9393", "followers_count": 1214, "friends_count": 1019, "statues_count": 17675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072440803328, "text": "What are you doing?? https://t.co/45IZzhDZAE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1485552289, "name": "Justin Barley", "screen_name": "justin_barley", "lang": "en", "location": "null", "create_at": date("2013-06-05"), "description": "WKU - Finance - ΣΦΕ VP of Programming", "followers_count": 553, "friends_count": 486, "statues_count": 3772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072503549952, "text": "So Kailee and I are officially moving to Zootopia! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599363732, "name": "Lindsy Thorwaldson", "screen_name": "lindsymarrin", "lang": "en", "location": "null", "create_at": date("2013-07-16"), "description": "Clear Eyes • Full Hearts • Can't Lose", "followers_count": 137, "friends_count": 235, "statues_count": 368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072524668928, "text": "f76c2fda458f0ea59e7c6cffa52fa52a17d19b5ebbb36e3610277a276f87b537d7679705ccc41d5138b0dd864e0a55b7632cbc95c635b733403c94e70c9d38fc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-107.364175,51.267375"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 429803867, "name": "MarsBots", "screen_name": "MarsBots", "lang": "en", "location": "Mars", "create_at": date("2011-12-06"), "description": "null", "followers_count": 88, "friends_count": 0, "statues_count": 454499 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "King George No. 256, Saskatchewan", "id": "15814b210f755033", "name": "King George No. 256", "place_type": "city", "bounding_box": rectangle("-107.678903,51.008583 -107.259921,51.271063") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072553902082, "text": ". https://t.co/sDR5mYmygF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2263207304, "name": "sarah❣", "screen_name": "mochaa_hontas", "lang": "en", "location": "with drake❤️", "create_at": date("2013-12-26"), "description": "views. #LU19", "followers_count": 462, "friends_count": 491, "statues_count": 7324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664072797171712, "text": "Greys anatomy time! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543224793, "name": "Can2", "screen_name": "Milio_2", "lang": "en", "location": "Dreamville.Dallas.Detroit.", "create_at": date("2012-04-01"), "description": "they say anything is possible, you gotta dream like you never seen obstacles. #crazyfelicialove", "followers_count": 442, "friends_count": 569, "statues_count": 15951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664073552105472, "text": "Grown man in my suit and tie��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486230218, "name": "Cierra✨", "screen_name": "OneToEnvy_XO", "lang": "en", "location": "412|Pgh", "create_at": date("2012-02-07"), "description": "IUP| SC: cierraloveeee", "followers_count": 1831, "friends_count": 985, "statues_count": 76355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664073740849152, "text": "If you're a #Nursing professional in #Houston, TX, check out this #job: https://t.co/izMNLmYivg #MedField #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.366043,29.7482407"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "Houston", "job", "MedField", "Hiring" }}, "user": { "id": 4586339233, "name": "IASIS Careers", "screen_name": "IASISCareers", "lang": "en", "location": "null", "create_at": date("2015-12-17"), "description": "When you join IASIS, you become part of a team that is committed to caring. Our goal is to approach every situation and every patient with a caring heart.", "followers_count": 69, "friends_count": 214, "statues_count": 418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664073799639040, "text": "Want to work at Lahey Health? We're #hiring in #Burlington, MA! Click for details: https://t.co/N822WDS01C #Clerical #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.195611,42.5048167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Burlington", "Clerical", "Job", "Jobs" }}, "user": { "id": 3011627064, "name": "Lahey Careers", "screen_name": "LaheyCareers", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "null", "followers_count": 43, "friends_count": 32, "statues_count": 446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664073875087361, "text": "Ok let me chill and watch my show��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 764637198, "name": "Adrianna", "screen_name": "Adriilomeli", "lang": "en", "location": "chillin", "create_at": date("2012-08-17"), "description": "I got me", "followers_count": 417, "friends_count": 161, "statues_count": 10216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664073912832000, "text": "@FriezzB4Guys regular niggaz", "in_reply_to_status": 705663655514230785, "in_reply_to_user": 410365018, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 410365018 }}, "user": { "id": 38813183, "name": "Kerrie ", "screen_name": "KdubSoSolid", "lang": "en", "location": "Sacramento CA ✈️ Houston TX", "create_at": date("2009-05-08"), "description": "All Glory to God IG:Kdubsosolid LakerNation PatsNation TSU - SAC STATE", "followers_count": 1198, "friends_count": 462, "statues_count": 175084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664074420453376, "text": "I'm not built for swing shift no mo!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132036630, "name": "Mookie", "screen_name": "MonSeanDon", "lang": "en", "location": "216 Honeymoon Ave - CLE", "create_at": date("2010-04-11"), "description": "These girls born in the 90s are dangerous. ✨", "followers_count": 1500, "friends_count": 712, "statues_count": 63633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664074437120000, "text": "My boyfriend tells me to close my eyes and tells me he has a surprise for me and i hear FREAK and see the video on tv... �� he's perfect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1100000886, "name": "jazzy✨", "screen_name": "_jadedelrey", "lang": "en", "location": "null", "create_at": date("2013-01-17"), "description": "You have to be odd to be number one. - Dr. Seuss", "followers_count": 620, "friends_count": 581, "statues_count": 35330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664075028508673, "text": "ima shoot myself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2909426773, "name": "sam", "screen_name": "samanthivve", "lang": "en", "location": "Pomona, CA", "create_at": date("2014-11-24"), "description": "null", "followers_count": 354, "friends_count": 239, "statues_count": 10513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-03-04T00:00:00.000Z"), "id": 705664075305320448, "text": "5447affbb7aa9c1e776c9fcd134e6f61f3f135db6bad574dd6ba4cbcb8cadbef13cd37db05668424308b4196bfa53ccb22d8ba1b955c5e4e312a05e764de016faf2ef0000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.52131879,27.88415914"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 144, "friends_count": 1, "statues_count": 811725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Micco, FL", "id": "51519f82b776ac84", "name": "Micco", "place_type": "city", "bounding_box": rectangle("-80.541212,27.835505 -80.490908,27.910291") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1245275, "cityName": "Micco" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664076228104193, "text": "@therealBreminem we share the same name, strange. I've never met anyone with out last name lol I know this is random but so so weird", "in_reply_to_status": -1, "in_reply_to_user": 421516411, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 421516411 }}, "user": { "id": 4109848396, "name": "Josh Bremerman", "screen_name": "Bremerman_Music", "lang": "en", "location": "Portland, OR", "create_at": date("2015-11-03"), "description": "Guitarist/Singer/Songwriter--Portland, OR. Keep In Touch!", "followers_count": 1031, "friends_count": 1985, "statues_count": 118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664076249059328, "text": "Wind 0.0 mph ---. Barometer 1024.28 mb, Steady. Temperature 37.9 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664076404240384, "text": "same https://t.co/rLfhFDtkry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317559432, "name": "conner", "screen_name": "IHATN", "lang": "en", "location": "texas", "create_at": date("2011-06-14"), "description": "you will never catch me without a hat on and without a smile on my face. speaking of hats , I make them .", "followers_count": 2812, "friends_count": 658, "statues_count": 196092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664076886609920, "text": "I always get excited about what kind of dreams i'm going to have������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20174823, "name": "Jason Finau", "screen_name": "jfinau", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-02-05"), "description": "helping to change the world for the better one person at a time...starting with me. #SocialWorker", "followers_count": 480, "friends_count": 531, "statues_count": 12515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664076999819264, "text": "@NeoRaven oh my god absolutely", "in_reply_to_status": 705660088955084800, "in_reply_to_user": 12164832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12164832 }}, "user": { "id": 589232732, "name": "Dara Khan", "screen_name": "palakchaval", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-05-24"), "description": "Stories & Songs & Feelpieces & Tweets", "followers_count": 162, "friends_count": 394, "statues_count": 7197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664077075361793, "text": "shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336392067, "name": "tom", "screen_name": "mr_rager60", "lang": "en", "location": "null", "create_at": date("2011-07-15"), "description": "null", "followers_count": 224, "friends_count": 137, "statues_count": 5300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664077322846208, "text": "@ebbtideapp Tide in Summit Bridge, Delaware 03/04/2016\nHigh 6:18am 3.5\n Low 12:57pm 0.2\nHigh 6:58pm 3.2\n Low 1:02am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.7333,39.5333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 45, "friends_count": 1, "statues_count": 12881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664077608067072, "text": "Can you recommend anyone for this #FacilitiesMgmt #job? https://t.co/8Iy7C9oCzD #nowhiring #Melville, NY #Hiring https://t.co/GzgiM9nJ9E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.4151214,40.7934322"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FacilitiesMgmt", "job", "nowhiring", "Melville", "Hiring" }}, "user": { "id": 30056162, "name": "AvalonBay Jobs", "screen_name": "workatavalonbay", "lang": "en", "location": "null", "create_at": date("2009-04-09"), "description": "The foundation of our success is our team of talented associates. Continued learning & new career challenges is a major focus. Join us in building a future!", "followers_count": 227, "friends_count": 32, "statues_count": 364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melville, NY", "id": "9f23cc99c0ba48db", "name": "Melville", "place_type": "city", "bounding_box": rectangle("-73.448769,40.751317 -73.368128,40.815432") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3646514, "cityName": "Melville" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664077654261760, "text": "@McCluskyJoe @LeahR77 @LindaSuhler @suscitate", "in_reply_to_status": 705660108697632768, "in_reply_to_user": 2320404222, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2320404222, 393190233, 347627434, 209484116 }}, "user": { "id": 3181439997, "name": "نHarbaugh Unhooked", "screen_name": "Brother4Blue", "lang": "en", "location": "Muskegon, MI", "create_at": date("2015-04-18"), "description": "Life long Michigan fan. Third shift production welder at Jost Int. I do not do Facebook. Very active on Google Plus. Life long Christian. JEW lover.", "followers_count": 622, "friends_count": 988, "statues_count": 14790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muskegon, MI", "id": "012b416c2c776945", "name": "Muskegon", "place_type": "city", "bounding_box": rectangle("-86.339696,43.201538 -86.144095,43.264395") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26121, "countyName": "Muskegon", "cityID": 2656320, "cityName": "Muskegon" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664077998198784, "text": "03/04@03:00 - Temp 31.2F, WC 31.2F. Wind 0.3mph ENE, Gust 3.0mph. Bar 30.123in, Falling. Rain 0.00in. Hum 94%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078312816640, "text": "And…it's 3am.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4239737843, "name": "NIÑO", "screen_name": "elninoem", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2015-11-20"), "description": "imma talk my shit on here, so fuck up.", "followers_count": 55, "friends_count": 70, "statues_count": 396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078367301632, "text": "Wind 0.0 mph NNW. Barometer 30.07 in, Steady. Temperature 47.1 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078425886721, "text": "This Kenyan woman looks like Megan Good at this party and I was mesmerized.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457048099, "name": "কোলবি", "screen_name": "_Colb24_", "lang": "en", "location": " Cleveland, OH ", "create_at": date("2012-01-06"), "description": "Saint Eds Alum | Ohio University 2017| Cavs | Browns |Indians | LAKERS | #ThankYouKobe |", "followers_count": 611, "friends_count": 416, "statues_count": 28510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078824529920, "text": "#SupportOriginMelissa 35.1°F Wind:0.0mph Pressure: 30.00hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 311478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078862278656, "text": "@HelenaSaynsber1", "in_reply_to_status": 705643908340944896, "in_reply_to_user": 701526905028222976, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4417853355 }}, "user": { "id": 701526905028222976, "name": "Andrzej J DABEK", "screen_name": "dabek_j", "lang": "en", "location": "null", "create_at": date("2016-02-21"), "description": "null", "followers_count": 45, "friends_count": 151, "statues_count": 75 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garfield, NJ", "id": "86fc60f26e1639cc", "name": "Garfield", "place_type": "city", "bounding_box": rectangle("-74.129119,40.861524 -74.091977,40.89369") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3425770, "cityName": "Garfield" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664078904033286, "text": "���� https://t.co/WrSGRTNjm3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2953907252, "name": "Dev ❤️", "screen_name": "DevieGee", "lang": "en", "location": "Palmdale, CA", "create_at": date("2014-12-31"), "description": "- Photography email: devongoodphotography@yahoo.com", "followers_count": 824, "friends_count": 765, "statues_count": 20572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664079482982400, "text": "Wind 1.0 mph NW. Barometer 30.159 in, Rising. Temperature 37.9 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-04T00:00:01.000Z"), "id": 705664080229445633, "text": "You know what means @1future ? https://t.co/wOL9yUI4mv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51742969 }}, "user": { "id": 412774172, "name": "DÄDDŸ 〽️ÊŸ✨", "screen_name": "Queen_Folarin", "lang": "en", "location": "California, USA", "create_at": date("2011-11-14"), "description": "⚓️ #NavyGirl", "followers_count": 1713, "friends_count": 1311, "statues_count": 30400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664080309305350, "text": "Forever hate people and love animals lolz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385636083, "name": "Point God", "screen_name": "YoPrimetimeee", "lang": "en", "location": "erlandooo", "create_at": date("2011-10-05"), "description": "Just give me the ball. BML 12.10.95 - 5.21.13", "followers_count": 678, "friends_count": 491, "statues_count": 24396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Springs, FL", "id": "6241025d720c4971", "name": "Coral Springs", "place_type": "city", "bounding_box": rectangle("-80.297884,26.228692 -80.201661,26.31183") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1214400, "cityName": "Coral Springs" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664080342683652, "text": "@iStoleFreeHugs you ain't even in Denton", "in_reply_to_status": 705662689201123328, "in_reply_to_user": 38886446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38886446 }}, "user": { "id": 239250751, "name": "Joe.", "screen_name": "Fly_Air_Walker", "lang": "en", "location": "Denton", "create_at": date("2011-01-16"), "description": "Where ever the Lord takes me, ill make the best of it. #UNT. IG same as my twitter.", "followers_count": 2533, "friends_count": 1528, "statues_count": 176416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664080573411328, "text": "I can't believe his friends actually thought he just threw his head back and went to sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344795177, "name": "her.", "screen_name": "KeAshaaa_", "lang": "en", "location": "New York ✈️", "create_at": date("2011-07-29"), "description": "19 ✨ || Single and poppin ' #ImHer", "followers_count": 2698, "friends_count": 1932, "statues_count": 60810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hot Springs, AR", "id": "208f4f6ad65bb7c9", "name": "Hot Springs", "place_type": "city", "bounding_box": rectangle("-93.12832,34.434369 -92.914247,34.538694") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5051, "countyName": "Garland", "cityID": 533400, "cityName": "Hot Springs" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664080825036801, "text": "All irrelevant �� what a joke�� https://t.co/e3qiPqg8px", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2254122205, "name": "Tina", "screen_name": "ScottRyanLarson", "lang": "en", "location": "West Jordan, UT", "create_at": date("2013-12-19"), "description": "just trying to figure out life like everyone else.", "followers_count": 345, "friends_count": 408, "statues_count": 6278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664081160708101, "text": "1h 40m wait time at WestView Health Centre. Browse wait times or share your own with #WaitShare at https://t.co/rv1fffVDIf! #StonyPlain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.978334,53.536777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "StonyPlain" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 368, "friends_count": 884, "statues_count": 2906 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Stony Plain, Alberta", "id": "6b297b6cc01bdffa", "name": "Stony Plain", "place_type": "city", "bounding_box": rectangle("-114.051347,53.498167 -113.946572,53.556168") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664081311576064, "text": "I rather u keep shit 100 and tell me your intensions than waste my time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2959313437, "name": "Princess Naomi", "screen_name": "HeLoveNomi", "lang": "en", "location": "null", "create_at": date("2015-01-04"), "description": "solid bitches don't crumble.", "followers_count": 2363, "friends_count": 757, "statues_count": 42192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664081475325952, "text": "03:00:02 |Temp: 42.9ºF | Wind Chill 42.9ºF |Dew Point 40.2ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NW, Gusting to 5.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664081714356224, "text": "Wind 0.0 mph SW. Barometer 30.066 in, Rising slowly. Temperature 36.4 °F. Rain today 0.01 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664081970143232, "text": "Chris Black �� https://t.co/AIVBry8OoO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3291468848, "name": "Ty™", "screen_name": "THCTyTy", "lang": "en", "location": "null", "create_at": date("2015-07-24"), "description": "I don't lie anymore™ Strength,Discipline,Integrity,Loyalty,Family", "followers_count": 946, "friends_count": 864, "statues_count": 18474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain House, CA", "id": "014b25c473600d50", "name": "Mountain House", "place_type": "city", "bounding_box": rectangle("-121.557176,37.766633 -121.533126,37.795236") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 649582, "cityName": "Mountain House" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664082083360768, "text": "@b_spectabilis @PollsterPatrick A lot can happen before November. I still have hope.", "in_reply_to_status": 705662729286123520, "in_reply_to_user": 189137892, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 189137892, 314019215 }}, "user": { "id": 15755364, "name": "David R Strong", "screen_name": "davidrstrong", "lang": "en", "location": "78757", "create_at": date("2008-08-06"), "description": "Aficionado, local personality, benign, randomly curious, love fine & performing arts, acting, music; responsive audience member too darost@earthlink.net", "followers_count": 1053, "friends_count": 2080, "statues_count": 23360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664082138025985, "text": "@BAF757 very true unfortunately I don't know how too", "in_reply_to_status": 705663328195108864, "in_reply_to_user": 494611323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 494611323 }}, "user": { "id": 376449852, "name": "College Street Kids", "screen_name": "collegeSTkids", "lang": "en", "location": "detroit/cleveland ", "create_at": date("2011-09-19"), "description": "G Huff & Tony Montana", "followers_count": 1989, "friends_count": 2127, "statues_count": 10757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, MI", "id": "bf02d3f5b54f0641", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-83.486832,42.358503 -83.44554,42.38445") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2665060, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664082393698304, "text": "I love this OMG https://t.co/ZoXJzKwmcm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285446119, "name": "BITCHASS PRINCESS♔", "screen_name": "Rawritsaubrey", "lang": "en", "location": "null", "create_at": date("2011-04-20"), "description": "sassy af", "followers_count": 211, "friends_count": 143, "statues_count": 16226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664082808967168, "text": "My remote just died and I'm stuck on Fox News and my building's only two stories high so I can't even plunge to my death.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15490533, "name": "Laura Silverman", "screen_name": "LauraJSilverman", "lang": "en", "location": "Los Angeles", "create_at": date("2008-07-18"), "description": "Dr. Katz, Half Baked, The Comeback (Jane, Jane) The SSP, Nurse Jackie, Andy of Andy and Ollie on FOX's Bob's Burgers. BOSTON BRUINS FAN. ❤️LA Kings way too much", "followers_count": 46476, "friends_count": 364, "statues_count": 3885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664082876067841, "text": "he ain't never go past SS2 https://t.co/8qjmqvp4B1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269200310, "name": "JayZOverrated", "screen_name": "JayZOverrated", "lang": "en", "location": "South Louisiana", "create_at": date("2011-03-20"), "description": "null", "followers_count": 8205, "friends_count": 896, "statues_count": 317171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayou Cane, LA", "id": "b5b20fbf38f349af", "name": "Bayou Cane", "place_type": "city", "bounding_box": rectangle("-90.810681,29.591044 -90.691976,29.661513") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2205210, "cityName": "Bayou Cane" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664083018833921, "text": "Temp: 49.9°F Wind:0.0mph Pressure: 30.083hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664083857518592, "text": "Be STRONG and COURAGEOUS.\nDo not be afraid; do not be\ndiscouraged for the LORD your GOD will be with you wherever you go.\nJoshua 1:9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 133, "friends_count": 248, "statues_count": 204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-04T00:00:02.000Z"), "id": 705664084159700993, "text": "\"you real bih\" https://t.co/CbsgpU3Vhk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1650849492, "name": "5'10 & Gifted", "screen_name": "notnyah", "lang": "en", "location": "jerz✈️chi", "create_at": date("2013-08-06"), "description": "depaul university | diamond in the rough", "followers_count": 552, "friends_count": 295, "statues_count": 30544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664084637827072, "text": "Insomnia ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 539390540, "name": "Stan Arwell", "screen_name": "stanarwell", "lang": "en", "location": "New York, NY", "create_at": date("2012-03-28"), "description": "'Flare' is out now on Armada Trice: https://ArmadaTrice.lnk.to/OMSAFlare. Send your promos to promos@stanarwell.com.", "followers_count": 3439, "friends_count": 210, "statues_count": 5273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664084667080704, "text": "@matt_zeidman Conner is inside that retarded beaners head!", "in_reply_to_status": 705662406765051904, "in_reply_to_user": 4077335894, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4077335894 }}, "user": { "id": 1574546449, "name": "Angelo", "screen_name": "apitillo818", "lang": "en", "location": "AJ is strait, America!", "create_at": date("2013-07-07"), "description": "Not my first rodeo", "followers_count": 572, "friends_count": 469, "statues_count": 9491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664084692246529, "text": "55.5F (Feels: 55.5F) - Humidity: 76% - Wind: 2.2mph N - Gust: 3.1mph - Pressure: 1028.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 229226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664084721557504, "text": "店長のLINEみて一人で感動して、なに勘違いしてたんやろうとか後悔した。なんか本物の人狼みたい。わて市民店長騎士あやつ人狼ふぃーっあ。てきな", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 2936683188, "name": "risa", "screen_name": "risapero24", "lang": "ja", "location": "♡げんきのお隣さーん♡", "create_at": date("2014-12-19"), "description": "♡女子大1年目♡専攻は英語♡只今アメリカに留学中です♡好きなことは、食べる、寝る、適当に歌う、です♡写真は得意ですが動画は不得意です笑♡どうぞよろしく♡", "followers_count": 487, "friends_count": 389, "statues_count": 2339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664085380227072, "text": "\"While all my friends are out partying....\" \n\n#nicethings #ChillsBirthday #happybirthdaytome… https://t.co/sP60gobfOK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98882157,40.76008402"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nicethings", "ChillsBirthday", "happybirthdaytome" }}, "user": { "id": 19013418, "name": "Chill Moody", "screen_name": "ChillMoody", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2009-01-14"), "description": "HipHop Artist & Philadelphia Music Ambassador. Inquiries hit nicethings@nicethingsMUSIC.com | http://ChillMoody.com follow @DrunkChill & @nicethingsMUSIC", "followers_count": 12927, "friends_count": 3564, "statues_count": 149105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664085678026752, "text": "Wind 0.7 mph ENE. Barometer 30.22 in, Falling. Temperature 17.1 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 112, "statues_count": 157983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664086470578176, "text": "I did It with both my phones it work �� https://t.co/nSABnChL5r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1669392642, "name": "flip ✨", "screen_name": "almighty_Ingy", "lang": "en", "location": "baltimore ", "create_at": date("2013-08-13"), "description": "410✈️805 can't chase you I'm chasing dreams baby.. #maurstrong", "followers_count": 2990, "friends_count": 1826, "statues_count": 37203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664087322185728, "text": "I'm in love with music...listen to it from when i wake up till i go to sleep...love analyzing it as well...can't live without it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456314678, "name": "ARI$", "screen_name": "aseferlis23", "lang": "en", "location": "IONA MENS SOCCER '19", "create_at": date("2012-01-05"), "description": "Trouble in Paradise™", "followers_count": 426, "friends_count": 304, "statues_count": 24642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Rochelle, NY", "id": "11cdb607ab2f17bb", "name": "New Rochelle", "place_type": "city", "bounding_box": rectangle("-73.807197,40.881294 -73.755414,40.988329") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3650617, "cityName": "New Rochelle" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664087330410496, "text": "It's cold af ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93750822, "name": "༄", "screen_name": "__ItsMARKIE", "lang": "en", "location": "Ypsilanti, ND", "create_at": date("2009-11-30"), "description": "| It's the experience that matters | DL•WB•MB | Eastern Michigan University '19 |", "followers_count": 764, "friends_count": 459, "statues_count": 26704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ypsilanti, MI", "id": "0049088b1906e51b", "name": "Ypsilanti", "place_type": "city", "bounding_box": rectangle("-83.661476,42.165629 -83.542129,42.325797") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2016-03-04T00:00:03.000Z"), "id": 705664088253325312, "text": "#Litty ✨�� @ The Mansion Elan Nightclub (Atlanta) https://t.co/4dVePA7Mcx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.3077206,33.86484634"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Litty" }}, "user": { "id": 191877050, "name": "DJ Eboogie", "screen_name": "iamdjeboogie", "lang": "en", "location": "LA ✈ ATL", "create_at": date("2010-09-17"), "description": "I Do Dope Shit.. \n ♌✌\nbookdjeboogie@gmail.com", "followers_count": 3621, "friends_count": 3272, "statues_count": 17178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664088941002752, "text": "Play my games babygirl https://t.co/LjNN5GvVpx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613736254, "name": "bae", "screen_name": "babysoymilk", "lang": "en", "location": "Bay Area, CA ", "create_at": date("2012-06-20"), "description": "full time cat and gecko mom", "followers_count": 875, "friends_count": 506, "statues_count": 18209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Discovery Bay, CA", "id": "309e4f83032e9413", "name": "Discovery Bay", "place_type": "city", "bounding_box": rectangle("-121.641476,37.889142 -121.586181,37.932788") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 619339, "cityName": "Discovery Bay" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664088941039616, "text": "WHAAAAAAAT......��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298850869, "name": "Taylan", "screen_name": "TaylanIliff", "lang": "en", "location": "null", "create_at": date("2011-05-14"), "description": "#iHoop. #FitFam. #2018. #Oklahoma. College Athlete at #NWOSU. IG: taylan_5 UCO BRONCHOS Next Fall", "followers_count": 1291, "friends_count": 979, "statues_count": 13126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alva, OK", "id": "4101ca1974f7262a", "name": "Alva", "place_type": "city", "bounding_box": rectangle("-98.691224,36.783091 -98.630554,36.812206") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40151, "countyName": "Woods", "cityID": 4001800, "cityName": "Alva" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664088962039809, "text": "@itsNotMirna @queen_gagarina @TRabbitMovement AMAZING", "in_reply_to_status": 705663984171655168, "in_reply_to_user": 1035717240, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1035717240, 1344920658, 1107907603 }}, "user": { "id": 384486160, "name": "странник ヽ(゚〇゚)ノ", "screen_name": "shining_nick", "lang": "en", "location": "probably running late", "create_at": date("2011-10-03"), "description": "♒️ | US/FI | ♂ Finn-tastic, pop-pönk, wannabe hater skater. #TeamMadCraft, ESC fan, ATL Hustler. European heart. UT Dallas student. http://bit.ly/1QnMWzq", "followers_count": 748, "friends_count": 591, "statues_count": 45136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089536630784, "text": "HAPPY BIRTHDAY TO ONE OF MY FAV SHAWTZ ������ HAVE A BOMB DAY ILY @rudimentarypete https://t.co/MD9iVFSJ1e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2471722698 }}, "user": { "id": 1895741568, "name": "mac", "screen_name": "esocr4m", "lang": "en", "location": " new west ", "create_at": date("2013-09-22"), "description": "#dontdropthebar", "followers_count": 343, "friends_count": 313, "statues_count": 17742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089565978624, "text": "Kendrick ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 478326542, "name": "Jacoby ⚾️", "screen_name": "ayomunoz", "lang": "en", "location": "null", "create_at": date("2012-01-29"), "description": "I hate losing more than I love winning", "followers_count": 971, "friends_count": 931, "statues_count": 24365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089662451713, "text": "@maaarlie he's so cute", "in_reply_to_status": 705641866947997696, "in_reply_to_user": 252450742, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 252450742 }}, "user": { "id": 252826132, "name": "Ashley Forster", "screen_name": "ashleyforster", "lang": "en", "location": "Calabasas, CA", "create_at": date("2011-02-15"), "description": "withering old person", "followers_count": 219, "friends_count": 363, "statues_count": 21376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089729531905, "text": "@toph42 Yes. As I have a million times. Since it’s my business address it’s no secret. Send me a burrito if you’d like!", "in_reply_to_status": 705562588772474880, "in_reply_to_user": 28542707, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28542707 }}, "user": { "id": 2911221, "name": "Wil Shipley", "screen_name": "wilshipley", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-03-29"), "description": "I WRITE SOFTWARE. I'M OBSESSIVE-COMPULSIVE. I LIKE GOOD STUFF. KITTENS ARE FUZZY.", "followers_count": 20971, "friends_count": 92, "statues_count": 42411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089767301120, "text": "@liveitupnw oh spit for the weekend?", "in_reply_to_status": 705663844467630080, "in_reply_to_user": 331385726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 331385726 }}, "user": { "id": 347663196, "name": "Ned Stark", "screen_name": "d_wright25", "lang": "en", "location": "Mesquite", "create_at": date("2011-08-02"), "description": "I do it for Ms. Gladys boy. #StayBlessed #YSBH", "followers_count": 1140, "friends_count": 1159, "statues_count": 54893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Balch Springs, TX", "id": "9935c102706fb55a", "name": "Balch Springs", "place_type": "city", "bounding_box": rectangle("-96.639116,32.69025 -96.574335,32.748536") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4805372, "cityName": "Balch Springs" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664089968779264, "text": "Travelling to Oakland or just twittering about Oakland? https://t.co/JgAAmAxCl1 #Oakland", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.271,37.8044"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oakland" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1036, "friends_count": 311, "statues_count": 2631727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664090518089728, "text": "@_jesus_mendez_ oh my god ��", "in_reply_to_status": 705646259407749125, "in_reply_to_user": 146229752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 146229752 }}, "user": { "id": 738585474, "name": "➳ veronica ♡", "screen_name": "punkbynaturee", "lang": "en", "location": "Chicago, IL ", "create_at": date("2012-08-05"), "description": "Pogo Jr. ⛅ Huggin' since '95. Bayley is da main squeeze. met my idol 1/16/16. ❤ Too sweeted Finn Bálor once so I'm cool. BayMella™", "followers_count": 2011, "friends_count": 1233, "statues_count": 39757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664090702647296, "text": "Another late night session. Let's go!!!!! #musicislife #1205amproductions", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "musicislife", "1205amproductions" }}, "user": { "id": 204117945, "name": "jeremy seaton", "screen_name": "JSeezie1205", "lang": "en", "location": "detroit, michigan", "create_at": date("2010-10-17"), "description": "Pianist, producer, musical illustrator", "followers_count": 196, "friends_count": 506, "statues_count": 2520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664091470192640, "text": "lol. yeah iighhh. I ain't even gone stress it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483458616, "name": "GG", "screen_name": "getemgg", "lang": "en", "location": "null", "create_at": date("2012-02-04"), "description": "I'm living good.", "followers_count": 798, "friends_count": 801, "statues_count": 33285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664091583553537, "text": "@EyyFrancisco10 HAPPY BIRTHDAY !! ���� have a gr8 day friend �� finally 18.. i think lmao", "in_reply_to_status": -1, "in_reply_to_user": 1414521860, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1414521860 }}, "user": { "id": 742306532, "name": "Barla ♛", "screen_name": "princessbarla", "lang": "en", "location": "null", "create_at": date("2012-08-06"), "description": "Eyeliner so sharp I can & will cut a bitch. |DHS| ♡DG♡", "followers_count": 711, "friends_count": 616, "statues_count": 14148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664091705094149, "text": "All from different mothers, but treat Each Other like brothers.. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2814231894, "name": "L.Auggie", "screen_name": "lukeauggie", "lang": "en", "location": "United States", "create_at": date("2014-09-16"), "description": "thoughtz in these tweetz", "followers_count": 546, "friends_count": 446, "statues_count": 2213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado Hills, CA", "id": "ab2b3353d37b40e6", "name": "El Dorado Hills", "place_type": "city", "bounding_box": rectangle("-121.111768,38.621986 -121.027263,38.743729") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 621880, "cityName": "El Dorado Hills" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664092057444352, "text": "I'm lowkey highkey still tryna do something & I just got back to my room lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 126826288, "name": "sunflower", "screen_name": "anijahboyd", "lang": "en", "location": "null", "create_at": date("2010-03-26"), "description": "Hip-Hop womanist.", "followers_count": 1809, "friends_count": 622, "statues_count": 56402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664092057444353, "text": "Even though, I say I need sleep I'm still up studying this mess. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371009085, "name": "breeeee☁️", "screen_name": "sheCutethough", "lang": "en", "location": "Ktown", "create_at": date("2011-09-09"), "description": "BLESSED | MIXED; BLACK & MEXICAN | USC❤️", "followers_count": 478, "friends_count": 1911, "statues_count": 14000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-04T00:00:04.000Z"), "id": 705664092573315072, "text": "Going to watch that tonight. #WhyNot", "in_reply_to_status": 705663958351360000, "in_reply_to_user": 149254310, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhyNot" }}, "user": { "id": 149254310, "name": "RayLe' SinClair", "screen_name": "IamRayle", "lang": "en", "location": "Wisconsin", "create_at": date("2010-05-28"), "description": "College Student #UWW | Dreamer | Avid sports fan. Packers/Bucks/Brewers/Badgers | Pro Wrestling is Life | MMA Fan | I'm a HUGE nerd for all things Marvel & DC.", "followers_count": 1711, "friends_count": 1692, "statues_count": 83877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monona, WI", "id": "756059b9675acdae", "name": "Monona", "place_type": "city", "bounding_box": rectangle("-89.359164,43.032813 -89.300252,43.086622") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55025, "countyName": "Dane", "cityID": 5553675, "cityName": "Monona" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664093030453248, "text": "Gym in 6 hours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 494789159, "name": "MustBeNice!", "screen_name": "FernBern24", "lang": "en", "location": "null", "create_at": date("2012-02-17"), "description": "Fernando/23/Instagram: NANDO_1993_", "followers_count": 262, "friends_count": 254, "statues_count": 4214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midvale, UT", "id": "ea98531b10405c6a", "name": "Midvale", "place_type": "city", "bounding_box": rectangle("-111.921741,40.593357 -111.855658,40.631074") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4949710, "cityName": "Midvale" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664093428936705, "text": "Passed some old blood just now. They said that would happen. Still gotta wear my pads. Yay. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37882628, "name": "Lisa Had Surgery", "screen_name": "SamuraiLisa", "lang": "en", "location": "Newton, IL", "create_at": date("2009-05-05"), "description": "Time Magazine Person Of The Year 2011. Waiting 4 skinny jeans 2 die a slow painful death. #YesAllWomen #BlackLivesMatter #FeelTheBern", "followers_count": 1310, "friends_count": 2099, "statues_count": 64547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, IL", "id": "b5783ad9ea014b3d", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-88.207281,38.972737 -88.14386,38.998089") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17079, "countyName": "Jasper", "cityID": 1752844, "cityName": "Newton" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664093475049472, "text": "Gn boe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1925587794, "name": "zaÿ⃒⃘o", "screen_name": "zay_cx", "lang": "en", "location": "lost", "create_at": date("2013-10-01"), "description": "⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀⠀ ⠀grind$", "followers_count": 814, "friends_count": 1246, "statues_count": 8980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise Hills, NM", "id": "00899a1c3f2d6c7d", "name": "Paradise Hills", "place_type": "city", "bounding_box": rectangle("-106.719935,35.188473 -106.68448,35.203753") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3555270, "cityName": "Paradise Hills" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664094301503489, "text": "They meant the truth will set you up.. And it's free https://t.co/iG9tFHWJUM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619238043, "name": "HolyBoobs", "screen_name": "CYNinNYC", "lang": "en", "location": "United States", "create_at": date("2012-06-26"), "description": "Always stay gracious, best revenge is your paper.", "followers_count": 651, "friends_count": 383, "statues_count": 25536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johnstown, OH", "id": "7997fe03e375aad1", "name": "Johnstown", "place_type": "city", "bounding_box": rectangle("-82.708443,40.136421 -82.666826,40.165073") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39089, "countyName": "Licking", "cityID": 3939340, "cityName": "Johnstown" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664094443950080, "text": "Trashhhhhh is recyclable", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.738366,30.266796"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3850256060, "name": "Guneez Ibrahim", "screen_name": "guneezzz", "lang": "en", "location": "Austin, TX", "create_at": date("2015-10-10"), "description": "i like my baby heir with baby hair and afros | \n\nBusiness Development at #besomebody.", "followers_count": 163, "friends_count": 356, "statues_count": 591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "6th Street", "id": "07d9c9e9ebc87007", "name": "6th Street", "place_type": "poi", "bounding_box": rectangle("-97.7383661,30.266795899999998 -97.738366,30.266796") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664094989217794, "text": "gotta wake up early tomorrow fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548751259, "name": "nvst", "screen_name": "_brxndvthxgod", "lang": "en", "location": "null", "create_at": date("2012-04-08"), "description": "work hard in silence, let your success be your noise", "followers_count": 988, "friends_count": 696, "statues_count": 46934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664094993575937, "text": "Everything is gay nowadays https://t.co/Q5cGmJEEqI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155127923, "name": "G you funny as hell", "screen_name": "WatchMeDestroy", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-06-12"), "description": "null", "followers_count": 9410, "friends_count": 3991, "statues_count": 128264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095039549440, "text": "#Team\nKeep it going, #Warriors!\n@Lilbsgv @BeccaaaLazzzuka @_naannna\n#TheHuntress #FireAway", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Team", "Warriors", "TheHuntress", "FireAway" }}, "user_mentions": {{ 355353209, 2211076423, 1581023510 }}, "user": { "id": 77332490, "name": "KathyRocks LASparks", "screen_name": "jaqlinnikkels", "lang": "en", "location": "#SoCalGal", "create_at": date("2009-09-25"), "description": "#My name is Kathy. #KathyRocks", "followers_count": 663, "friends_count": 2084, "statues_count": 49179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095068884992, "text": "Somebody ft me?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176733664, "name": "Wavy", "screen_name": "Dorian__insane", "lang": "en", "location": "huntington beach ", "create_at": date("2010-08-10"), "description": "Sc & Insta: Thatkidddorian / Chi-town✈️Cali / 19Years-old", "followers_count": 1503, "friends_count": 880, "statues_count": 34674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095081492480, "text": "Lol homer cracks me up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1350513642, "name": "Summer Roxburgh", "screen_name": "summerroxburgh", "lang": "en", "location": "Austin, Texas", "create_at": date("2013-04-13"), "description": "May I never forget, on my best day, that I still need God as desperately as I did on my worst day. #UT19", "followers_count": 419, "friends_count": 230, "statues_count": 13042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095085658112, "text": "Wind 1.0 mph NW. Barometer 30.169 in, Falling slowly. Temperature 30.3 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095333134336, "text": "Honestly need a new phone so if anyone has a Verizon 6 I would love you 4ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321756934, "name": "Lauren", "screen_name": "Its_LaurenU", "lang": "en", "location": "Washington, USA", "create_at": date("2011-06-21"), "description": "Gamma Phi Beta", "followers_count": 307, "friends_count": 323, "statues_count": 10876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095333306368, "text": "Bae tried to play me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2158538793, "name": "_LiyaTajee", "screen_name": "HighThenGoLow", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2013-10-29"), "description": "Know yourself, Know your worth. West Philly. Sc @highthen", "followers_count": 216, "friends_count": 199, "statues_count": 2338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664095589163009, "text": "Wind 0.0 mph ---. Barometer 30.269 in, Falling. Temperature 2.4 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664096184754176, "text": "Yes, I do. I believe that one day I will be right there where I was, right next to you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1232683842, "name": "Amy✨", "screen_name": "amiikins", "lang": "en", "location": "Tx", "create_at": date("2013-03-01"), "description": "Happy atm", "followers_count": 1253, "friends_count": 500, "statues_count": 74619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664096281079808, "text": "Happy birthday to my precious little blonde princess ����✨ pop lock and drop that thang tonight bitch I luv u!!!�� https://t.co/ULswDrU6uw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2478945649, "name": "phenomeBONG", "screen_name": "annanongx", "lang": "en", "location": "somewhere cloudy", "create_at": date("2014-05-05"), "description": "null", "followers_count": 706, "friends_count": 393, "statues_count": 33137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664096927092736, "text": "Sometimes I wonder if I put too much effort into somebody. When I should've been just half-assing it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1972166928, "name": "G. Mendoza", "screen_name": "OG_Gill", "lang": "en", "location": "Crane, Tx ⏩ York, Nebraska", "create_at": date("2013-10-19"), "description": "Insta: o.g._gill & SC: og_gill | Stay Golden | York College Baseball #19 | Just looking for the right pitch to hit |", "followers_count": 559, "friends_count": 502, "statues_count": 11391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "York, NE", "id": "0be973b828c2f491", "name": "York", "place_type": "city", "bounding_box": rectangle("-97.608606,40.83569 -97.565422,40.900497") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31185, "countyName": "York", "cityID": 3154045, "cityName": "York" } }
+{ "create_at": datetime("2016-03-04T00:00:05.000Z"), "id": 705664097006833664, "text": "@ShaneCampbell44 go to sleep Shane.", "in_reply_to_status": 705662193061257216, "in_reply_to_user": 418343932, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418343932 }}, "user": { "id": 586108815, "name": "Chris Boisvert", "screen_name": "BOvi_Chris", "lang": "en", "location": "Mission Hill, Boston", "create_at": date("2012-05-20"), "description": "My healies have no breaks because my swag rides on forever. Bruins, Wild, Sox and Pats. I'm slightly obsessed with music. SJCP '14 - WIT '18", "followers_count": 529, "friends_count": 1661, "statues_count": 2827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-03-04T00:00:06.000Z"), "id": 705664097195565056, "text": "Ion trust that shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605137059, "name": "LilBrooke✨", "screen_name": "alwaysbrooke_", "lang": "en", "location": "Gucci World", "create_at": date("2012-06-10"), "description": "All I CAN Be Is Real. •UNT19|DTX• #RHQ #Aubrey❤️", "followers_count": 1462, "friends_count": 768, "statues_count": 30539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-04T00:00:06.000Z"), "id": 705664098068008960, "text": "RT https://t.co/39yVOS0Cul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 229705800, "name": "Darien Grant", "screen_name": "_TruChainz", "lang": "en", "location": "Old Dominion University ", "create_at": date("2010-12-22"), "description": "Founder of @Dusaamerica | Urban Philosopher | IG: Goldgrant", "followers_count": 2441, "friends_count": 1280, "statues_count": 83110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-03-04T00:00:06.000Z"), "id": 705664098130907136, "text": "Freaked out by that damn movie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3294819412, "name": "baligay✨", "screen_name": "Forevanaae", "lang": "en", "location": "The Swamp", "create_at": date("2015-05-22"), "description": "pray about it as much as you think about it!", "followers_count": 402, "friends_count": 242, "statues_count": 2284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026460461436928, "text": "Accused murder accomplice accepts plea deal: Mustapha Jacobs at his 2010 arraignment at the N... https://t.co/IL3Qt2IK9a #stamford #topix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.5387341,41.0534302"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stamford", "topix" }}, "user": { "id": 115851436, "name": "StamfordCTNews", "screen_name": "stamfordctnews", "lang": "en", "location": "Stamford, CT", "create_at": date("2010-02-19"), "description": "null", "followers_count": 840, "friends_count": 0, "statues_count": 33450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stamford, CT", "id": "468bfe7c705fbc37", "name": "Stamford", "place_type": "city", "bounding_box": rectangle("-73.606436,41.016898 -73.501758,41.179771") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 973000, "cityName": "Stamford" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026460721577984, "text": "She is a terrible candidate. Blamed GEB on Nader supt. Gore was terrible and lost TN! https://t.co/67Hq0IFwOB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3383747345, "name": "Rita", "screen_name": "ritakiser25", "lang": "en", "location": "Richmond, VA", "create_at": date("2015-07-19"), "description": "62 yr old mom of 4 grandma of 6. iT support.", "followers_count": 329, "friends_count": 778, "statues_count": 4688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026461048647680, "text": "I'm gone have every one saying kinky", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393425348, "name": "Playa Ass Diego™", "screen_name": "AFatChickHero", "lang": "en", "location": "Rest with God lil homie Ebby", "create_at": date("2011-10-18"), "description": "| I'm The greatest thing that happened to black women since weave glue | Just an average man with an average penis |Nurse Diego one day | SD✈️ATX✈️DFW | #TAMUC", "followers_count": 844, "friends_count": 245, "statues_count": 82141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026461170241536, "text": "These high school wannabe Steph Curry's out there tryna run their mouth. Haha, sit down with your 5'7 ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2310274974, "name": "Jess¡e", "screen_name": "__Segovia", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-01-25"), "description": "Young Sinatra | 19", "followers_count": 141, "friends_count": 124, "statues_count": 5306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026461329793024, "text": "https://t.co/bP7y3wbqsC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 603837472, "name": "Cavemanrobot", "screen_name": "Cavemanrobot", "lang": "en", "location": "Monumenta", "create_at": date("2012-06-09"), "description": "Caveman Robot, hero of past, present, and future!", "followers_count": 93, "friends_count": 445, "statues_count": 48957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026462164426753, "text": "Whatever bruh��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2533257841, "name": "Cobbs!", "screen_name": "1cobbs_", "lang": "en", "location": "null", "create_at": date("2014-05-29"), "description": "stay down", "followers_count": 633, "friends_count": 288, "statues_count": 4162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euclid, OH", "id": "59ed3d5f720ebdf0", "name": "Euclid", "place_type": "city", "bounding_box": rectangle("-81.552943,41.545336 -81.488014,41.631584") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3925704, "cityName": "Euclid" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026462541819904, "text": "Wind 4.0 mph SW. Barometer 1023.78 mb, Rising. Temperature 47.3 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026462562885632, "text": "@SkinnyMech_ ������", "in_reply_to_status": 706026387681861632, "in_reply_to_user": 137918656, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 137918656 }}, "user": { "id": 29946900, "name": "Guess Who", "screen_name": "Mr_ITYC", "lang": "en", "location": "804,VA to 704,NC", "create_at": date("2009-04-09"), "description": "Marketing Connoisseur!!", "followers_count": 1694, "friends_count": 237, "statues_count": 109934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026462684573696, "text": "rs! https://t.co/2OY02AAmYy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 516947960, "name": "arrogant", "screen_name": "elovewithme", "lang": "en", "location": "null", "create_at": date("2012-03-06"), "description": "take me as I am or leave me where I'm at.", "followers_count": 1743, "friends_count": 292, "statues_count": 54032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026463007526912, "text": "Lol Wednesday that is someone's husband ��������, Nigerian men are not like your people ������ https://t.co/O6BNtpZKwi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 199198124, "name": "YEYEOGE AWELEWA 1", "screen_name": "GossipGirliee", "lang": "en", "location": "District of Columbia, USA", "create_at": date("2010-10-06"), "description": "when you need massive awareness for your business, services, music, etc. I am who you should holla @ info@gossipgirlee.com http://www.facebook.com/gossipgirlee", "followers_count": 56166, "friends_count": 10205, "statues_count": 304497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Laurel, MD", "id": "0270fb8d47bab72f", "name": "South Laurel", "place_type": "city", "bounding_box": rectangle("-76.885346,39.033137 -76.810791,39.096875") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2473650, "cityName": "South Laurel" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026463053545472, "text": "I'm going to get my paper done tomorrow and still have fun what's good ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255800234, "name": "Danime", "screen_name": "DaynaaNiggah", "lang": "en", "location": " lucy❤️", "create_at": date("2011-02-21"), "description": "18. I ain't got no issues, I'm just doin' my thing. Hope you're doin' your thing too.", "followers_count": 401, "friends_count": 401, "statues_count": 20195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026463158374400, "text": "Happy Birthday to a very special girl!@allegra_grace ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 361678740, "name": "Neverett", "screen_name": "LucasEverett15", "lang": "en", "location": "Fullerton, Cali", "create_at": date("2011-08-24"), "description": "Sonora | Sophomore | God First | The Uprising", "followers_count": 445, "friends_count": 219, "statues_count": 1658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-03-05T00:00:00.000Z"), "id": 706026463561056256, "text": "@Sakthi_Twitz ������", "in_reply_to_status": 706026259378151425, "in_reply_to_user": 3279494845, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3279494845 }}, "user": { "id": 499692587, "name": "நீ எழுதா❄கவிதையிவன்✒", "screen_name": "smg_s", "lang": "en", "location": "என் அம்மா❤இதயத்தில் மட்டும்", "create_at": date("2012-02-22"), "description": "«««கோவக்காரன், பாசக்காரன் கொஞ்சம் ரொம்பவே ரோசக்காரன், மனசால ரொம்ப நல்லது நினைப்பான் எதையும் பண்ணாம அமைதியா இருப்பான்»»»", "followers_count": 9823, "friends_count": 2420, "statues_count": 69821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026463980494849, "text": "@JoselineZamora ��", "in_reply_to_status": 706026310561234944, "in_reply_to_user": 312136151, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 312136151 }}, "user": { "id": 86420066, "name": "aaliyah ♛", "screen_name": "AaliyahhRenee", "lang": "en", "location": "ig/sc: aaliyahhrenee", "create_at": date("2009-10-30"), "description": "Adecyn Grace♕", "followers_count": 1346, "friends_count": 903, "statues_count": 29283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026464508932096, "text": "Happy Birthday Beauty!❤️ I love you and I hope your birthday is as amazing as you are.������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4164904346, "name": "liv", "screen_name": "LiviaaMichellee", "lang": "en", "location": "silver lakes", "create_at": date("2015-11-07"), "description": "| C/O 2016 | Future Nurse Anesthetist❤️ | Excelsior Varsity Volleyball | Happiness is key | #SJSharks fan |", "followers_count": 204, "friends_count": 211, "statues_count": 2460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Lakes, CA", "id": "0144019445c157c9", "name": "Silver Lakes", "place_type": "city", "bounding_box": rectangle("-117.367738,34.726505 -117.330275,34.7622") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 671964, "cityName": "Silver Lakes" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026465125605376, "text": "03:00:02 |Temp: 35.6ºF | Wind Chill 35.6ºF |Dew Point 32.7ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the N, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 90882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026465133928449, "text": "Hahaahah lmfao!!! https://t.co/1GQNTyRTwe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513803408, "name": "zayyyy", "screen_name": "WIZDOM_zayt", "lang": "en", "location": "Austin, TX", "create_at": date("2012-03-03"), "description": "Dress like sinatra but act like you listen to pac! #FREEPL4Y", "followers_count": 594, "friends_count": 345, "statues_count": 16773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pflugerville, TX", "id": "b5613ac46d587422", "name": "Pflugerville", "place_type": "city", "bounding_box": rectangle("-97.664611,30.420118 -97.549428,30.50077") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4857176, "cityName": "Pflugerville" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026466056613889, "text": "I knew 50 cent wouldn't do me like that ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2457572712, "name": "sara", "screen_name": "cryfce", "lang": "en", "location": "LA", "create_at": date("2014-04-21"), "description": "I drink till I'm mad and I love being sad", "followers_count": 470, "friends_count": 217, "statues_count": 17373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026466178420737, "text": "Ripley SW Limestone Co. Temp: 33.1°F Wind:0.0mph Pressure: 1000.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 47, "friends_count": 32, "statues_count": 49847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026466379747328, "text": "@mtranquilnight History will show @SenMikeLee and the #GOP lacks of compassion for human kind in 2016 America. (flint gets new pipes)", "in_reply_to_status": 706025385360646144, "in_reply_to_user": 525785725, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GOP" }}, "user_mentions": {{ 525785725, 88784440 }}, "user": { "id": 220262110, "name": "Flint's best friend", "screen_name": "C1TYofFL1NT", "lang": "en", "location": "Flint, MI ", "create_at": date("2010-11-26"), "description": "'A wise man changes his mind, a fool never' ~ #anonymous ( #OpFlint founder)", "followers_count": 3803, "friends_count": 491, "statues_count": 12980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026466459443200, "text": "Wind 1.0 mph ESE. Barometer 30.073 in, Falling. Temperature 39.8 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026466673295361, "text": "03/05@03:00 - Temp 29.7F, WC 29.7F. Wind 1.0mph N, Gust 2.0mph. Bar 30.196in, Rising slowly. Rain 0.00in. Hum 76%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026467130380288, "text": "@DarlinBelieveMe You're so hot, aguamenti couldn't even put you out", "in_reply_to_status": 706024438345523200, "in_reply_to_user": 3007194118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3007194118 }}, "user": { "id": 3398116745, "name": "Beff", "screen_name": "Aisllyn7", "lang": "en", "location": "Oswego, NY", "create_at": date("2015-07-31"), "description": "My cat is Rayita May, follow her on insta: https://instagram.com/rayita_kitty/ Follow my twitch stream: http://twitch.tv/aisllyn7 for cats and fun ♥", "followers_count": 136, "friends_count": 202, "statues_count": 2252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oswego, NY", "id": "2bf0b14f23e6d6b8", "name": "Oswego", "place_type": "city", "bounding_box": rectangle("-76.565589,43.428101 -76.437307,43.494352") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36075, "countyName": "Oswego", "cityID": 3655574, "cityName": "Oswego" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026467163897856, "text": "♫ Played Beverly Hills by @Weezer, 2 Heads by Coleman Hell and 6 more songs at Anytime Fitness Chico : https://t.co/8BOGBs4Ciq @GetRockbot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.795738,39.718334"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16685316, 91192339 }}, "user": { "id": 32825474, "name": "Gloria", "screen_name": "gloria_olivarez", "lang": "en", "location": "N 39°45' 0'' / W 122°10' 0''", "create_at": date("2009-04-18"), "description": "null", "followers_count": 16, "friends_count": 107, "statues_count": 262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026467184881664, "text": "Wishing the BEST big brother a happy happy 21st!!!!!!!!! Have the best night & love you lots @Tshaww32����❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 138582124 }}, "user": { "id": 879264163, "name": "Alexa Shaw", "screen_name": "alexaashaw", "lang": "en", "location": "null", "create_at": date("2012-10-13"), "description": "Sportsaholic•#D1Bound Sideline Reporter•TCU Sports Broadcasting•ZTA / http://Instagram.com/alexxashaw", "followers_count": 1705, "friends_count": 477, "statues_count": 2004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026467306541056, "text": "@jaymesnevis hbd I said it first that means I'm your best friend", "in_reply_to_status": -1, "in_reply_to_user": 2210228762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2210228762 }}, "user": { "id": 2258493859, "name": "Ryan", "screen_name": "ryanteck99", "lang": "en", "location": "null", "create_at": date("2013-12-22"), "description": "Track Pants enthusiast", "followers_count": 219, "friends_count": 364, "statues_count": 791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:01.000Z"), "id": 706026468128661505, "text": "I'll vote for you @MittRomney", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50055701 }}, "user": { "id": 51277182, "name": "Juan Carlo Sarinas", "screen_name": "Real_JC_Sarinas", "lang": "en", "location": "Imus, Cavite - Bellingham, WA", "create_at": date("2009-06-26"), "description": "taas noo kahit kanino", "followers_count": 402, "friends_count": 620, "statues_count": 39426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026468762062848, "text": "You think you hot as me hold on you must be smokin nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487775217, "name": "Sofyoncé", "screen_name": "SofiaDrewniany", "lang": "en", "location": "Middletown, CT", "create_at": date("2012-02-09"), "description": "those who mind don't matter and those who matter don't mind ~ xo till we od", "followers_count": 521, "friends_count": 271, "statues_count": 7689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, CT", "id": "8c814b789b5f7574", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-72.752943,41.494706 -72.58268,41.604104") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 947290, "cityName": "Middletown" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026469051514880, "text": "Wind 0.0 mph SE. Barometer 30.180 in, Falling slowly. Temperature 31.3 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026469399535617, "text": "They need another season of bms", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2696786214, "name": "hail storm", "screen_name": "hayleyymcdonald", "lang": "en", "location": "scv allstars r5 ", "create_at": date("2014-07-31"), "description": "I think I'm hilarious", "followers_count": 444, "friends_count": 586, "statues_count": 1265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026469701632000, "text": "I'm sleepy but ion wanna go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 450384357, "name": "NEW✨❣‼️", "screen_name": "__New2x", "lang": "en", "location": "Clarksdale, MS", "create_at": date("2011-12-29"), "description": "Rip Auntie Key '09 / Rip Grandaddy '15 ❤️ / SC: CallMee_NewNew ❤️‼️", "followers_count": 1104, "friends_count": 1023, "statues_count": 34903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarksdale, MS", "id": "aff7d5b6120f9190", "name": "Clarksdale", "place_type": "city", "bounding_box": rectangle("-90.611078,34.16622 -90.538506,34.23361") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28027, "countyName": "Coahoma", "cityID": 2813820, "cityName": "Clarksdale" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026469969915904, "text": "It's like your over it all https://t.co/7dNnU15zXB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276141036, "name": "ReddgotdaJUICE", "screen_name": "REDDwhiteNblue", "lang": "en", "location": "New Orleans ", "create_at": date("2011-04-02"), "description": "23", "followers_count": 1353, "friends_count": 922, "statues_count": 62486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026470196424705, "text": "Mosh Pitting during #statechamps @state_champs @ Masquerade https://t.co/pgWYamp2Cj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.36751084,33.86814756"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "statechamps" }}, "user_mentions": {{ 150041309 }}, "user": { "id": 57519990, "name": "Rebecca ✨", "screen_name": "itsgottabluke", "lang": "en", "location": "Dacula, GA", "create_at": date("2009-07-16"), "description": "Can I interest you in a sarcastic comment?", "followers_count": 294, "friends_count": 107, "statues_count": 10619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026470401925120, "text": "52.2F (Feels: 52.2F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1030.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 229370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026470670524417, "text": "Temp: 44.4°F Wind:0.0mph Pressure: 30.175hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026470947164160, "text": "I had so much fun just playing and making my son laugh so much today, like it's crazy now that he's asleep my room is no quiet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 472204289, "name": "⚽️⚽️Kevin ⚽️ ⚽", "screen_name": "yooitzkevin", "lang": "en", "location": "null", "create_at": date("2012-01-23"), "description": "father to a wonderful boy named adrian that runs my wolrd. and has my heart ❤ ❤\nsingle father", "followers_count": 276, "friends_count": 539, "statues_count": 5294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026471593107456, "text": "������ https://t.co/BHFH9FjNPJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1462546994, "name": "briana", "screen_name": "brianapalacios_", "lang": "en", "location": "Haltom City, TX", "create_at": date("2013-05-27"), "description": "Luis Lopez • 11/10", "followers_count": 585, "friends_count": 351, "statues_count": 20049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haltom City, TX", "id": "69644b8de3ee73aa", "name": "Haltom City", "place_type": "city", "bounding_box": rectangle("-97.293831,32.779549 -97.236256,32.861171") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4831928, "cityName": "Haltom City" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026471605846018, "text": "Wind 0.0 mph N. Barometer 30.20 in, Steady. Temperature 18.1 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 112, "statues_count": 158007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-05T00:00:02.000Z"), "id": 706026472092250112, "text": "what we do �� https://t.co/FRFSMNSltj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2730918611, "name": "lame af.", "screen_name": "__iamqueenk", "lang": "en", "location": "where ever the money is , TX", "create_at": date("2014-08-01"), "description": "Dominican princess", "followers_count": 1486, "friends_count": 858, "statues_count": 32119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge North, TX", "id": "0180324e104dd7b5", "name": "Oak Ridge North", "place_type": "city", "bounding_box": rectangle("-95.452477,30.076095 -95.349469,30.173617") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4853190, "cityName": "Oak Ridge North" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026472348061696, "text": "��", "in_reply_to_status": 584866764275585024, "in_reply_to_user": 2396773525, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2396773525, "name": "$ Pãpi J $", "screen_name": "papikeraj", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-03-18"), "description": "~ Don't be the nigga she tweetin about. Be the nigga she talkin shit about.~ •DaGram: __keraj__ •Snapchat: daddykeraj •Periscope: papikeraj", "followers_count": 27064, "friends_count": 27160, "statues_count": 9764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026472742453248, "text": "@beerenah @emileecnossen I'm George", "in_reply_to_status": 705979298855915520, "in_reply_to_user": 117272729, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 117272729, 54312525 }}, "user": { "id": 216431321, "name": "Taylor Sheehan", "screen_name": "taysheehan", "lang": "en", "location": "Grand Rapids, Michigan ", "create_at": date("2010-11-16"), "description": "I hear her hair is insured for $10,000", "followers_count": 202, "friends_count": 365, "statues_count": 7743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026473535057920, "text": "@NotDomingoAyaIa thank for the follow bro . Made my day https://t.co/j22DuGvFIv", "in_reply_to_status": -1, "in_reply_to_user": 3272313104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3272313104 }}, "user": { "id": 3311342720, "name": "Brandon Rothell", "screen_name": "BrandonRothell", "lang": "en", "location": "null", "create_at": date("2015-08-10"), "description": "On my way to make my dream a reality ⚾️ And on the search for that one girl", "followers_count": 90, "friends_count": 339, "statues_count": 824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera Acres, CA", "id": "75161a014946f779", "name": "Madera Acres", "place_type": "city", "bounding_box": rectangle("-120.109807,36.98237 -120.056201,37.03494") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645050, "cityName": "Madera Acres" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026474013335552, "text": "I tried finding lewd pics of @Daniel_Splatter on Derpibooru but all I got was a virus.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3622655833 }}, "user": { "id": 115997117, "name": "Mechadragon", "screen_name": "MechaDragon101", "lang": "en", "location": "A new York that open up", "create_at": date("2010-02-20"), "description": "Some mechanic dragon who loves card games, video games, Wander Over Yonder, Pokemon, and Applejack...alot :D. Follow for some madness.", "followers_count": 2477, "friends_count": 1356, "statues_count": 134911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026474646609921, "text": "sometimes ya just gotta build a bridge", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2392878487, "name": "jenn", "screen_name": "xojennagxoxo", "lang": "en", "location": "null", "create_at": date("2014-03-16"), "description": "believe in the good of the people", "followers_count": 216, "friends_count": 208, "statues_count": 1700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026474894024705, "text": "Faith goes up the stairs that love has made and looks out of the Windows which hope has opened.\nCharles Spurgeon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 134, "friends_count": 249, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026475078553600, "text": "Look at my dab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1951058239, "name": "CA♛", "screen_name": "CHANEL1700", "lang": "en", "location": "null", "create_at": date("2013-10-09"), "description": "•gwhs•", "followers_count": 692, "friends_count": 673, "statues_count": 20837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026475259084801, "text": "watching Unfriended with my friends!!! never seen it before!! heard there's a really cool blender in it though", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182337487, "name": "cc", "screen_name": "_champagnecc", "lang": "en", "location": "null", "create_at": date("2010-08-24"), "description": "IG: champagnecc_", "followers_count": 917, "friends_count": 219, "statues_count": 71270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southgate, MI", "id": "f359dbed0d2029e0", "name": "Southgate", "place_type": "city", "bounding_box": rectangle("-83.229831,42.184275 -83.179587,42.227987") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2674960, "cityName": "Southgate" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026475661561856, "text": "11:59 pm https://t.co/spGc5Mds3n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2620850336, "name": "ᶜᴼᴸᴸᴱᴱᴺᶜᴴᴿᴵˢᵀᴵᴺᴬ", "screen_name": "TheColleenXtina", "lang": "en", "location": "Whittier, CA", "create_at": date("2014-07-12"), "description": "ωαιтιиg fσя ѕυммα!!!!", "followers_count": 339, "friends_count": 356, "statues_count": 9314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-03-05T00:00:03.000Z"), "id": 706026475766452224, "text": "I'm not trying to be cocky but I know I'm gonna be sexy asf in a few years same thing with my brother lol we're gonna kill shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1345247737, "name": "A$apSoto⚡️", "screen_name": "Stevensoto69", "lang": "en", "location": "Surprise, AZ", "create_at": date("2013-04-11"), "description": "(Senior) Rest In Paradise: My Brother Damien Serrahn", "followers_count": 750, "friends_count": 547, "statues_count": 4004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026477926481921, "text": "ha ha you disgust me ha ha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872846354, "name": "sami", "screen_name": "samicasteel", "lang": "en", "location": "null", "create_at": date("2012-10-10"), "description": "• unchain your brain •", "followers_count": 415, "friends_count": 127, "statues_count": 7431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clayton, MO", "id": "a5c0eba140e25cbb", "name": "Clayton", "place_type": "city", "bounding_box": rectangle("-90.353953,38.633586 -90.303397,38.658236") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2914572, "cityName": "Clayton" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026478190796802, "text": "if today goes as planned... omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951139368, "name": "TEEZUS", "screen_name": "MariahLives23", "lang": "en", "location": "E. CLT", "create_at": date("2014-12-29"), "description": "null", "followers_count": 61, "friends_count": 42, "statues_count": 7863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026478777950208, "text": "Happy birthday to one of my bestfriends and boyfriend�� I love you to the moon and back❤️�� HAPPY 19th hunny�� https://t.co/9ORw2Qzpzp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3231817418, "name": "Destiny Moore", "screen_name": "Destinyyymoore", "lang": "en", "location": "null", "create_at": date("2015-05-31"), "description": "MS ❤️", "followers_count": 276, "friends_count": 188, "statues_count": 3656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026478882918400, "text": "Let it rain...let it pour...if she don't love me anymore...let it come down on me... @ John T.… https://t.co/AMcnPMIzn2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.6904297,29.5781193"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37079385, "name": "Kelli", "screen_name": "tx_kelli", "lang": "en", "location": "null", "create_at": date("2009-05-01"), "description": "almost never hangs on forever", "followers_count": 640, "friends_count": 1641, "statues_count": 7889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helotes, TX", "id": "00a1b28251c671b8", "name": "Helotes", "place_type": "city", "bounding_box": rectangle("-98.764721,29.495566 -98.661448,29.604798") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026478975238144, "text": "@KennethPogue @BurnsvillePD Kill Em All album.", "in_reply_to_status": 706026176158892032, "in_reply_to_user": 2833021878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2833021878, 829533036 }}, "user": { "id": 466778850, "name": "Andrew in Minnesota", "screen_name": "NOuse4Username7", "lang": "en", "location": "Minnesota, USA", "create_at": date("2012-01-17"), "description": "23 #Irish #Israeli #Smartass #Polyglot #Comedian #HopelessRomantic #HockeyPerson #punkrocker #Chiver #KCCO NY Rangers, Wild, Mets, Badgers, and Vikings fan", "followers_count": 1264, "friends_count": 2161, "statues_count": 45915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burnsville, MN", "id": "5f2ad2e8059e53c5", "name": "Burnsville", "place_type": "city", "bounding_box": rectangle("-93.329811,44.716911 -93.222028,44.828464") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2708794, "cityName": "Burnsville" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026479025573888, "text": "If youon learn then shame on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304816967, "name": "Applied Pressure", "screen_name": "SavageInkk_", "lang": "en", "location": "404 ✈️ 214 ✈️ 229", "create_at": date("2011-05-24"), "description": "Stay Solid & Stay Free. VSU16", "followers_count": 1373, "friends_count": 962, "statues_count": 78453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valdosta, GA", "id": "5e1c91065bc30991", "name": "Valdosta", "place_type": "city", "bounding_box": rectangle("-83.374825,30.762483 -83.230199,30.949686") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1378800, "cityName": "Valdosta" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026479197528064, "text": "2 &, 4. https://t.co/SnG8Wx7mNd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2471891528, "name": "shannon", "screen_name": "shanympho", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2014-04-30"), "description": "carlos p. ❣ #stoptravelingfordick2K16", "followers_count": 809, "friends_count": 641, "statues_count": 9121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026479717478400, "text": "Yes all of next week will be storms ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863950254, "name": "JACKS➰N", "screen_name": "jacksonhatesyou", "lang": "en", "location": "Keller, TX", "create_at": date("2014-10-18"), "description": "idk", "followers_count": 185, "friends_count": 164, "statues_count": 3517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026479931363328, "text": "Wind 3.0 mph NNW. Barometer 30.142 in, Falling. Temperature 17.1 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026480099332096, "text": "Omg @jul_lowe https://t.co/O8xqr0sozN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 257741597 }}, "user": { "id": 377712172, "name": "✝aylor", "screen_name": "prrincesstaylor", "lang": "en", "location": "neverland ", "create_at": date("2011-09-21"), "description": "RIP Bizzy", "followers_count": 200, "friends_count": 193, "statues_count": 15165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentor-on-the-Lake, OH", "id": "8e912902951846d5", "name": "Mentor-on-the-Lake", "place_type": "city", "bounding_box": rectangle("-81.3888,41.704391 -81.348368,41.72827") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3949098, "cityName": "Mentor-on-the-Lake" } }
+{ "create_at": datetime("2016-03-05T00:00:04.000Z"), "id": 706026480296284161, "text": "@AdammrdchAdam no way i wish i was there Adam �� i will be pulling all the girls", "in_reply_to_status": 706023652479754241, "in_reply_to_user": 1093620770, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1093620770 }}, "user": { "id": 3074448601, "name": "Yung DC", "screen_name": "theboiyung", "lang": "en", "location": "Clear Lake, Houston", "create_at": date("2015-03-11"), "description": "20. HCC 16 I'm a Houston Texans fan at heart I like gurls that are fun & Educated and I can rap soon to be a UH Cougar", "followers_count": 286, "friends_count": 530, "statues_count": 9250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026481659412480, "text": "#SupportOriginMelissa 31.8°F Wind:1.3mph Pressure: 30.12hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 311558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026482104012800, "text": "@Amanijaber13 happy birthday pretty lady! Have a great day love you �������� https://t.co/YVlQNC1HQf", "in_reply_to_status": -1, "in_reply_to_user": 4087284255, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4087284255 }}, "user": { "id": 2398679892, "name": "Adrianna", "screen_name": "Adri_Xoxo_", "lang": "en", "location": "null", "create_at": date("2014-03-19"), "description": "17 • Palestinian• Buffalo, New york • Insta / @adrianna_xoxo_", "followers_count": 602, "friends_count": 424, "statues_count": 21978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026482221473792, "text": "#19��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 961112912, "name": "Karsyn Moore", "screen_name": "Karsynn_", "lang": "en", "location": "yucaipa, ca", "create_at": date("2012-11-20"), "description": "all I want in life is to fulfill my dream of being a mermaid", "followers_count": 438, "friends_count": 117, "statues_count": 31312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026482615758848, "text": "Lol fucking bug bites ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2853500263, "name": "Chef Ayeeeee", "screen_name": "axnuwin", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "http://phhho.to/AXNUWIN?t=o", "followers_count": 382, "friends_count": 284, "statues_count": 6701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483202920449, "text": "Is it a bad idea to wear a t shirt dress to CT??:/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 830119562, "name": "Mia☔️", "screen_name": "Bitcheslovemia", "lang": "en", "location": "mpc", "create_at": date("2012-09-17"), "description": "sc/ig/twitter: bitcheslovemia", "followers_count": 975, "friends_count": 402, "statues_count": 22922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, AZ", "id": "2db584821a82475e", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-111.425637,33.006636 -111.34843,33.078782") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 423760, "cityName": "Florence" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483286994948, "text": "Y'all always being extra tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.00102875,40.01036063"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35129467, "name": "Coconut Weezay", "screen_name": "Tweetin_Yo_Girl", "lang": "en", "location": "Columbus, OH", "create_at": date("2009-04-24"), "description": "I do what I love...and I don't think you should know about it all. Make it light up", "followers_count": 448, "friends_count": 512, "statues_count": 32077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483563798528, "text": "Bernieeee https://t.co/vgW92LfCkR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 63626823, "name": "Essence.", "screen_name": "AmbitioussAries", "lang": "en", "location": "UNC Greensboro.", "create_at": date("2009-08-06"), "description": "The short, curvy girl who looks mean with a bomb personality & an amazing playlist filled with trap music & old school love songs.", "followers_count": 2091, "friends_count": 2075, "statues_count": 46116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483597217792, "text": "To my good pal Isaac for always keepin it real. Happy 18th birthday bud. Make it a good one!��������@magicike1 https://t.co/TJFfOhpz0v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4016084954 }}, "user": { "id": 1266138146, "name": "Samuel", "screen_name": "Samuelittle", "lang": "en", "location": "Edgewood", "create_at": date("2013-03-13"), "description": "PHS", "followers_count": 264, "friends_count": 287, "statues_count": 509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, WA", "id": "af582c6bfd1b91eb", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-122.326451,47.203736 -122.2489,47.257551") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5320645, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483798507520, "text": "Yo I want some carne asada fries from La Vic's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2561510796, "name": "ßig〽️ondo", "screen_name": "MondoPando", "lang": "en", "location": "Straight Outta Fremont", "create_at": date("2014-06-11"), "description": "Living each day like it's my last. Can't dwell on the past and most certainly not going to let my life go by fast.", "followers_count": 173, "friends_count": 144, "statues_count": 8486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026483928711168, "text": "Ayo I'm really turnt watch my story on snapxhat I'm dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4609072277, "name": "#2", "screen_name": "Cashmere__22", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2015-12-26"), "description": "SC: Prettyboitae. Calu", "followers_count": 171, "friends_count": 213, "statues_count": 1823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026484134117376, "text": "Someone smoke this blunt with me :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932853753, "name": "QOE. ✨", "screen_name": "Stick_011", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2013-10-03"), "description": "TGOD. whatevers clever. tamucc.", "followers_count": 1092, "friends_count": 548, "statues_count": 19401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026484666867712, "text": "@LilMoneyMonk if we being honest you the lick", "in_reply_to_status": 706026215052845056, "in_reply_to_user": 476405257, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 476405257 }}, "user": { "id": 317618743, "name": "heaven", "screen_name": "heavstro", "lang": "en", "location": "ohio state university", "create_at": date("2011-06-14"), "description": "philosophy major. osu19. \nbig philosopher big logician big lawyer", "followers_count": 1882, "friends_count": 693, "statues_count": 77725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026484721299456, "text": "Happy bday Lor!! One minute till it's not but I just wanted to say I love and miss you so much!! I hope 16 is bomb AF @LaurenLindholm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1507310426 }}, "user": { "id": 1491921840, "name": "Joy Day", "screen_name": "itsaJoyDayyy13", "lang": "en", "location": "null", "create_at": date("2013-06-07"), "description": "INSTA: itsjoyday • SNAPCHAT: joy13day99", "followers_count": 162, "friends_count": 141, "statues_count": 712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-03-05T00:00:05.000Z"), "id": 706026484905877506, "text": "https://t.co/Al4ISIc5uG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2463286764, "name": "liz", "screen_name": "lookin4revenge", "lang": "en", "location": "Los Angeles", "create_at": date("2014-04-25"), "description": "17 / phuck you", "followers_count": 1360, "friends_count": 766, "statues_count": 39117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026485077942272, "text": "Fishing �� Stunning sunsets on Marathon Key, Florida, USA ����☀️ #sunset #colors #miami… https://t.co/DHxAYWm43v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.93647175,24.77210455"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sunset", "colors", "miami" }}, "user": { "id": 26307457, "name": "Martin Kup", "screen_name": "Housik", "lang": "en", "location": "Uherské Hradiště", "create_at": date("2009-03-24"), "description": "Webdeveloper, Marketer, Photograper, Traveler, http://martinkup.cz & http://digidy.cz", "followers_count": 26, "friends_count": 46, "statues_count": 942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marathon, FL", "id": "301bd8da9d45994b", "name": "Marathon", "place_type": "city", "bounding_box": rectangle("-81.124761,24.688245 -80.923527,24.777279") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12087, "countyName": "Monroe", "cityID": 1243000, "cityName": "Marathon" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026485879062528, "text": "Wind 0.0 mph ---. Barometer 30.232 in, Steady. Temperature 6.6 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026485946019840, "text": "��✨ we never talk anymore lol I would say hi but I left Qhhs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2281419704, "name": "Seth Montes", "screen_name": "SethM__24", "lang": "en", "location": "A Galaxy Far Far Away ", "create_at": date("2014-01-07"), "description": "null", "followers_count": 257, "friends_count": 181, "statues_count": 1367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quartz Hill, CA", "id": "cbac03fb139017a6", "name": "Quartz Hill", "place_type": "city", "bounding_box": rectangle("-118.236164,34.6308 -118.201289,34.680279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 659052, "cityName": "Quartz Hill" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026486164094977, "text": "It's always too good to be true", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3682618453, "name": "Hannah Taylor", "screen_name": "hlaynetaylor", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-09-25"), "description": "It's all make believe, isn't it?", "followers_count": 174, "friends_count": 165, "statues_count": 81 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026487590170626, "text": "���� https://t.co/XIAwmdq8FC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2472628124, "name": "⚓️Rashawn4⃣", "screen_name": "rashawnd88", "lang": "en", "location": "null", "create_at": date("2014-05-01"), "description": "DAY 673: I still don't own a sea otter", "followers_count": 691, "friends_count": 1278, "statues_count": 6163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tanque Verde, AZ", "id": "598f1e0ca6a69b63", "name": "Tanque Verde", "place_type": "city", "bounding_box": rectangle("-110.812889,32.23542 -110.72088,32.308673") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 472000, "cityName": "Tanque Verde" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026487615348736, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 765079267, "name": "The Dragon", "screen_name": "dannytdinh", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-08-17"), "description": "Wait for me UFC | Singha MMA | UW", "followers_count": 362, "friends_count": 206, "statues_count": 21369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tukwila, WA", "id": "9883a4be0cbd497c", "name": "Tukwila", "place_type": "city", "bounding_box": rectangle("-122.318673,47.407147 -122.241356,47.531957") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5372625, "cityName": "Tukwila" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026487929933826, "text": "Temp: 29.7°F - Dew Point: 26.3° - Wind: 6.9 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 29.96in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026488278204416, "text": "@bret_28 it was an accident hoe", "in_reply_to_status": 706026419285983232, "in_reply_to_user": 117205983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 117205983 }}, "user": { "id": 154282163, "name": "brianna lopez", "screen_name": "BriE_MINE", "lang": "en", "location": "null", "create_at": date("2010-06-10"), "description": "V24", "followers_count": 1409, "friends_count": 820, "statues_count": 40115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026488420823041, "text": "Forum focuses on hurricane protection options https://t.co/E53HQ66bXd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.04820251,29.68300056"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2866050172, "name": "La Porte, TX News", "screen_name": "BLifeLaPorte", "lang": "en", "location": "La Porte, Texas", "create_at": date("2014-11-07"), "description": "La Porte BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/IzWtG570IM.", "followers_count": 144, "friends_count": 122, "statues_count": 5307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026488437436416, "text": "@DaaanielSan LETS. Is your number still the same??", "in_reply_to_status": 706013373415034882, "in_reply_to_user": 36807896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36807896 }}, "user": { "id": 202531006, "name": "Cali Canary", "screen_name": "EnvyMV", "lang": "en", "location": "San Diego ☀️", "create_at": date("2010-10-14"), "description": "•Call me Marnise •Humor, Anime, Shows, and Music keep me sane #BlackLivesMatter", "followers_count": 141, "friends_count": 116, "statues_count": 4224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026488458383360, "text": "Surrounded by love tonight�� #ThankYouJesus", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThankYouJesus" }}, "user": { "id": 219177181, "name": "Shanae Cole", "screen_name": "idance_365", "lang": "en", "location": "null", "create_at": date("2010-11-23"), "description": "I'm a superhero • James 1:12 ✝", "followers_count": 1549, "friends_count": 1494, "statues_count": 7232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-03-05T00:00:06.000Z"), "id": 706026488911429633, "text": "Wind 3.0 mph W. Barometer 30.055 in, Falling slowly. Temperature 35.5 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 1653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026489318240256, "text": "Wind 2.3 mph WNW. Barometer 30.249 in, Steady. Temperature 16.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 43, "statues_count": 6980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026489444061184, "text": "I wnna do somethingggg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 746459174, "name": "Mikey ♚", "screen_name": "MikeyyGee_562", "lang": "en", "location": "null", "create_at": date("2012-08-08"), "description": "19♉️", "followers_count": 1125, "friends_count": 553, "statues_count": 64447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026489855094785, "text": "They want Capital to make a statement. I say shovel all the coal in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2985272402, "name": "Jeffrey Kudo", "screen_name": "kudojeff", "lang": "en", "location": "Oxnard Ca", "create_at": date("2015-01-15"), "description": "null", "followers_count": 86, "friends_count": 621, "statues_count": 5520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026490262122496, "text": "Not to mention, the best things in life usually come free, or are in the form of car parts. https://t.co/IY9ggGfk3e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 996053042, "name": "Jesse", "screen_name": "RVLTD_S13", "lang": "en", "location": "Back Home", "create_at": date("2012-12-07"), "description": "I live my life a 1/4 mile at a time & If one day the speed kills me, do not cry, because I was smiling.", "followers_count": 408, "friends_count": 327, "statues_count": 31274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethel, CT", "id": "4e873217de913edf", "name": "Bethel", "place_type": "city", "bounding_box": rectangle("-73.445754,41.310356 -73.341668,41.428451") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 904790, "cityName": "Bethel" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026490652188672, "text": "idefk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 422228095, "name": "Neftali Valdivia", "screen_name": "Neftalivaldivia", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-11-26"), "description": "my neck my back. | 14 | :): | BKB ✏️ | snapchat : neftalivaldivia | ✌️✌ | #FreeDaiJanae | Panocha", "followers_count": 151, "friends_count": 1016, "statues_count": 3219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026490668843008, "text": "@denisemariebook yeas! I want ALL THE POPS!!!", "in_reply_to_status": 706022140919742464, "in_reply_to_user": 4369725912, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4369725912 }}, "user": { "id": 3973426034, "name": "Brekke El", "screen_name": "BrekkeWrites", "lang": "en", "location": "California", "create_at": date("2015-10-21"), "description": "word tinker. thought thinker. coffee drinker. clumsy thumbs. #amwriting", "followers_count": 194, "friends_count": 452, "statues_count": 1184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026491407003649, "text": "Who is bankroll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247091784, "name": "Miss Pam Anderson", "screen_name": "Titsandlickher", "lang": "en", "location": "Kemet", "create_at": date("2011-02-03"), "description": "null", "followers_count": 7239, "friends_count": 132, "statues_count": 49902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026491658825728, "text": "@ComplexMusic @BlowMu_Blushhhh", "in_reply_to_status": 706014159796899840, "in_reply_to_user": 243163874, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 243163874, 160797227 }}, "user": { "id": 2242174132, "name": "cleopatra ✨", "screen_name": "aurorakween", "lang": "en", "location": "working at the pyramid tonight", "create_at": date("2013-12-24"), "description": "trynna steal dat dick, wyd? | 20 | feeling good, living better ❤️", "followers_count": 2006, "friends_count": 688, "statues_count": 283680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026491851571200, "text": "@StephyRoseG https://t.co/EBGKiqxDmF", "in_reply_to_status": 706025873774751744, "in_reply_to_user": 1054557488, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1054557488 }}, "user": { "id": 3244472202, "name": "tyler hendren", "screen_name": "tyhen2208", "lang": "en", "location": "California, USA", "create_at": date("2015-06-13"), "description": "onward and upward ⬆️", "followers_count": 203, "friends_count": 198, "statues_count": 4862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Calaveras, CA", "id": "6efe67c93b20d55a", "name": "Rancho Calaveras", "place_type": "city", "bounding_box": rectangle("-120.879833,38.091186 -120.825605,38.160607") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6009, "countyName": "Calaveras", "cityID": 659426, "cityName": "Rancho Calaveras" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026491864154112, "text": "@itsskionnaa_ i already did now im up", "in_reply_to_status": 706026230038994948, "in_reply_to_user": 4903362853, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4903362853 }}, "user": { "id": 348247641, "name": "Real 1", "screen_name": "uarenotworthy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-08-03"), "description": "Shut the fuck up and enjoy the greatness.", "followers_count": 27178, "friends_count": 9223, "statues_count": 43172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026492329910272, "text": "I hate y'all stoner bitches on Snapchat �� all y'all do is cry and smoke weed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4712771655, "name": "TRISGOD", "screen_name": "_xcvi", "lang": "en", "location": "xxx", "create_at": date("2016-01-05"), "description": "aspiring chef, I do art sometimes.. @de_jpg ❤", "followers_count": 55, "friends_count": 121, "statues_count": 1028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Socastee, SC", "id": "0043ce7c1cad5c16", "name": "Socastee", "place_type": "city", "bounding_box": rectangle("-79.06754,33.614208 -78.921739,33.731483") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4567390, "cityName": "Socastee" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026492577206272, "text": "3/5/2016 - 02:00\nTemp: 41.1F \nHum: 95%\nWind: 0.0 mph\nBaro: 30.160in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 109, "friends_count": 53, "statues_count": 50564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026492808011776, "text": "Thanks @mkfrench for the follow! Have a wonderful weekend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22077812 }}, "user": { "id": 17247683, "name": "James Wagner", "screen_name": "James6495", "lang": "en", "location": "Cheviot, OH", "create_at": date("2008-11-08"), "description": "#Blogger , #Entrepreneur,#homerunsolos owner #Bengals #reds fan #NetworkMarketer #Dad #seo geek #logo design #teeshirt design #proud American #ordained minister", "followers_count": 3300, "friends_count": 3713, "statues_count": 12114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheviot, OH", "id": "c5fa66be451d6b3f", "name": "Cheviot", "place_type": "city", "bounding_box": rectangle("-84.626222,39.148265 -84.602684,39.168738") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3914128, "cityName": "Cheviot" } }
+{ "create_at": datetime("2016-03-05T00:00:07.000Z"), "id": 706026493168619520, "text": "@Kit_Pocket gn!!!! https://t.co/8xerWyv858", "in_reply_to_status": 706025819307528193, "in_reply_to_user": 1153248234, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1153248234 }}, "user": { "id": 2327656434, "name": "✨carrots✨", "screen_name": "agentpleakley", "lang": "en", "location": "under the sea", "create_at": date("2014-02-04"), "description": "@mermistas's side/more personal/shitposty/fandomy account i live tweet here a lot", "followers_count": 257, "friends_count": 198, "statues_count": 22551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388848801161216, "text": "03:00:01 |Temp: 38.7ºF | Wind Chill 38.7ºF |Dew Point 35.7ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WSW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 96, "friends_count": 21, "statues_count": 90930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388848855506944, "text": "@taobby @ChanTaoLuWu I know where that is imma go find his ass", "in_reply_to_status": 706319344125382656, "in_reply_to_user": 2770663739, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2770663739, 1867927812 }}, "user": { "id": 3037556102, "name": "jaimz", "screen_name": "romanticjinki", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-02-22"), "description": "*jinki voice* noona you're my benchwarmer", "followers_count": 865, "friends_count": 586, "statues_count": 39702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388849744740352, "text": "#MyPartnerMustHave a vacuum for when I don't want to clean the house. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MyPartnerMustHave" }}, "user": { "id": 51974112, "name": "☠Βαρβάρα Λάμπερτ", "screen_name": "iPrincessBitch", "lang": "en", "location": "Keene, New Hampshire", "create_at": date("2009-06-28"), "description": "♏ I'm not really here. I'm just a figment of your imagination.™️ Not sure anymore. https://iprincessbitch.wordpress.com/", "followers_count": 1045, "friends_count": 2517, "statues_count": 18816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keene, NH", "id": "5d7310873de470f7", "name": "Keene", "place_type": "city", "bounding_box": rectangle("-72.347356,42.910955 -72.221936,42.982474") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33005, "countyName": "Cheshire", "cityID": 3339300, "cityName": "Keene" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388849895735296, "text": "@SkizzyMars hmu with the addy 3176950800", "in_reply_to_status": 706387724505186304, "in_reply_to_user": 139869817, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139869817 }}, "user": { "id": 253248340, "name": "Madalyn Norman", "screen_name": "Madddnor", "lang": "en", "location": "Avon, IN", "create_at": date("2011-02-16"), "description": "(-':", "followers_count": 698, "friends_count": 495, "statues_count": 7923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avon, IN", "id": "6126fccc2f229071", "name": "Avon", "place_type": "city", "bounding_box": rectangle("-86.457278,39.733571 -86.326391,39.800876") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1802908, "cityName": "Avon" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388849904095233, "text": "House of Cards better be lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372029480, "name": "SEB", "screen_name": "ChaseMeyer1", "lang": "en", "location": "Fort Worth, TX ✈️ Moberly, MO", "create_at": date("2011-09-11"), "description": "I'm a man of many talents. Wrestling. Football. Pursuing doctor/surgeon Instagram @Chase_Meyerrr SC: Chasemeyer1", "followers_count": 756, "friends_count": 798, "statues_count": 16374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moberly, MO", "id": "65af6d81d67ec5f8", "name": "Moberly", "place_type": "city", "bounding_box": rectangle("-92.481848,39.358718 -92.411635,39.458601") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29175, "countyName": "Randolph", "cityID": 2949034, "cityName": "Moberly" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388850113798145, "text": "@gabrielherrera killer. I wish I had a deck irl.", "in_reply_to_status": 706388421250568192, "in_reply_to_user": 16172556, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16172556 }}, "user": { "id": 78108635, "name": "Edward Hoffman", "screen_name": "E_L_Hoffman", "lang": "en", "location": "null", "create_at": date("2009-09-28"), "description": "null", "followers_count": 88, "friends_count": 99, "statues_count": 5761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388850164129794, "text": "Nope I meant ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179596553, "name": "Patti", "screen_name": "_retropetro", "lang": "en", "location": "TX", "create_at": date("2010-08-17"), "description": "our hearts are heavy burdens we shouldn't have to bear alone.", "followers_count": 182, "friends_count": 277, "statues_count": 6980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388850478690304, "text": "Wind 5.0 mph SE. Barometer 1024.76 mb, Steady. Temperature 53.2 °F. Rain today 0.00 in. Humidity 39%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388850558390272, "text": "This Giveaway ends soon. 4 more days. So go RT to enter!!!\n\n@PndaCodeE @ShoutGamers @KILLCLIFF @DynastyKaYco https://t.co/TlLXBpRAFE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1335626024, 1345471728, 57141484, 2752388574 }}, "user": { "id": 47599831, "name": "#FreeDragz", "screen_name": "UnF_Syco", "lang": "en", "location": "Cancer Survivor ", "create_at": date("2009-06-16"), "description": "| Owner-@WereUnForgiven | GM-@InControlHQ | Sis-@Nogy_Sway | @UnF_OLT | @UnF_Pubs | @UnFStudios | DMV Homie-@Breenade", "followers_count": 2660, "friends_count": 1411, "statues_count": 29196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388850818600960, "text": "Bedtime!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1549894706, "name": "♎Kaylaaa⚓", "screen_name": "Kassidys_mommyy", "lang": "en", "location": "Sweet Home Alabama ❤", "create_at": date("2013-06-26"), "description": "✨Let your faith be bigger than your fear❤\nSC: Kayluh_22", "followers_count": 251, "friends_count": 616, "statues_count": 13012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chickasaw, AL", "id": "395b77bf2d7d1e9f", "name": "Chickasaw", "place_type": "city", "bounding_box": rectangle("-88.102366,30.753344 -88.068151,30.79154") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 114392, "cityName": "Chickasaw" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388851028320256, "text": "December '16 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1937709493, "name": "κingτee.", "screen_name": "__TLamar", "lang": "en", "location": "StraightOuttaDega ", "create_at": date("2013-10-05"), "description": "Army | Political Scientist | #BlackExcellence *RT/Likes ≠ endorsement*", "followers_count": 1507, "friends_count": 1084, "statues_count": 43034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-86.418059,32.284593 -86.071398,32.443697") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388851229523968, "text": "Bet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2191200902, "name": "ñicolas", "screen_name": "nickhtx", "lang": "en", "location": "Life ", "create_at": date("2013-11-12"), "description": "I'll be fine // byan :) // https://www.youtube.com/watch?v=oCd6SQ67k-Y", "followers_count": 335, "friends_count": 290, "statues_count": 5692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388851732779010, "text": "Last seconds of my birthday...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1970379288, "name": "LordPrettyFlackoJoey", "screen_name": "brick_wall02", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-10-18"), "description": "I have no life. @torie_677 ❤️", "followers_count": 347, "friends_count": 330, "statues_count": 8568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-03-06T00:00:00.000Z"), "id": 706388851774918657, "text": "#FACTS https://t.co/kAM7UqUa8Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FACTS" }}, "user": { "id": 2441996595, "name": "Lady Bowie.", "screen_name": "JunoCassandra", "lang": "en", "location": "Dragonstone. ☄", "create_at": date("2014-03-28"), "description": "• I will NEVER apologize for being an outspoken boss, because I don't owe anyone an apology. •", "followers_count": 1581, "friends_count": 898, "statues_count": 105983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388852185780225, "text": "U can see her huh? https://t.co/KsPGElk9E1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.26218655,36.30073417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3276477930, "name": "Ana Steele", "screen_name": "AnaStee19484959", "lang": "en", "location": "Heaven ASAP", "create_at": date("2015-07-11"), "description": "By blood rite: king of all Earth. Angel of Allah y Yahway. Buddah by enlightement under the Bohdi Tree; direct Decendent Christ. Prof. singer actor dir writer", "followers_count": 1752, "friends_count": 2145, "statues_count": 118137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388852261429248, "text": "@BabyAnimalPics @IsaHurairah", "in_reply_to_status": 706289468081377280, "in_reply_to_user": 1372975219, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1372975219, 222689420 }}, "user": { "id": 33242024, "name": "OLA AHMED", "screen_name": "OLAAHMEDNY", "lang": "en", "location": "NEW YORK, NY", "create_at": date("2009-04-19"), "description": "INTERIOR DESIGN", "followers_count": 774, "friends_count": 40, "statues_count": 285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388852718620672, "text": "Live happy and free ✌��️\n\n#positivevibes #postivity #powerofpositivity @ Tres Gringos Cabo Cantina https://t.co/Y05fT3StdJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.8884125,37.3352013"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "positivevibes", "postivity", "powerofpositivity" }}, "user": { "id": 274833831, "name": "Shane A. DeHart", "screen_name": "ShaneADeHart", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-03-30"), "description": "Follow our Twitter @realadversaries | Follow our Instagram @theadversaries | For beat inquiries contact: prodbytheadversaries@gmail.com | #NOFREEBEATS |", "followers_count": 485, "friends_count": 362, "statues_count": 3996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388853364432896, "text": "I know I didn't even read it I just wanted to tell u to shut up lol https://t.co/qfJ5qmkdyV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932681667, "name": "night.lebell", "screen_name": "kaylebb_", "lang": "en", "location": "null", "create_at": date("2013-10-03"), "description": "#whooty", "followers_count": 290, "friends_count": 327, "statues_count": 4055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388853473595392, "text": "Shout out to @its_kathyn because yes ❤️ #robeitup #itspastmybedtime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "robeitup", "itspastmybedtime" }}, "user_mentions": {{ 575803017 }}, "user": { "id": 3314812316, "name": "Amanda K Sellman", "screen_name": "A_Man_Duh_moo", "lang": "en", "location": "Bellingham, WA", "create_at": date("2015-08-13"), "description": "insta: amanda.sellman tumblr: http://www.a-man-duh-moo.tumblr.com ❤️ Hello Seattle ❤️", "followers_count": 52, "friends_count": 154, "statues_count": 464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388853477679104, "text": "Just turned 18��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1306264278, "name": "Young Metro", "screen_name": "esiason_r", "lang": "en", "location": "null", "create_at": date("2013-03-26"), "description": "shopin in Dubai feeling fly", "followers_count": 693, "friends_count": 871, "statues_count": 2261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388853653958657, "text": "Ripley SW Limestone Co. Temp: 45.0°F Wind:0.7mph Pressure: 1003.4mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 47, "friends_count": 32, "statues_count": 49892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388853901430785, "text": "= https://t.co/duxpKBTMFW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2571613183, "name": "papi.", "screen_name": "MarlinsRoom", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "i am the real papi shampoo schemin with good intentions", "followers_count": 254, "friends_count": 242, "statues_count": 5081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mooresville, NC", "id": "eb5c40f3d165a855", "name": "Mooresville", "place_type": "city", "bounding_box": rectangle("-80.961004,35.495207 -80.768109,35.641662") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37097, "countyName": "Iredell", "cityID": 3744220, "cityName": "Mooresville" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388854492639232, "text": "I closed my eyes for 5 seconds, opened them, and @ellieachapman was literally hovering over me a centimeter away from my face ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 865669470 }}, "user": { "id": 745961564, "name": "αυввѕ", "screen_name": "AubreeGardner3", "lang": "en", "location": "Raymond, WA", "create_at": date("2012-08-08"), "description": "junior @ RHS", "followers_count": 451, "friends_count": 364, "statues_count": 15414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388854492811264, "text": "@ashh_mariee1 fuckkkkk thatttttt", "in_reply_to_status": 706376461200936960, "in_reply_to_user": 446517480, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 446517480 }}, "user": { "id": 327811149, "name": "Meg Ryan", "screen_name": "_mryan_", "lang": "en", "location": "null", "create_at": date("2011-07-01"), "description": "Ithaca '19", "followers_count": 614, "friends_count": 532, "statues_count": 9425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, NY", "id": "988d82c27e9587c8", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-76.523386,42.398337 -76.452907,42.431088") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36109, "countyName": "Tompkins", "cityID": 3669199, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388854626869248, "text": "Dude I havent seen you since middle school hfs, imy and happy birthday b������ @jasmineDANG6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1152386593 }}, "user": { "id": 2853500263, "name": "Annie", "screen_name": "axnuwin", "lang": "en", "location": "null", "create_at": date("2014-10-12"), "description": "http://phhho.to/AXNUWIN?t=o", "followers_count": 381, "friends_count": 284, "statues_count": 6729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388854799015937, "text": "@beauty_necia hurry back ��", "in_reply_to_status": -1, "in_reply_to_user": 2757837868, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2757837868 }}, "user": { "id": 626900569, "name": "#KeepingUpW/Kay", "screen_name": "EatMyiii_Tweet", "lang": "en", "location": "Meridian, MS", "create_at": date("2012-07-04"), "description": "What You Allow Will Only Continue. mines;Z.R.C.❤️", "followers_count": 2609, "friends_count": 1431, "statues_count": 96932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, MS", "id": "57dc070bcd27882d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-88.763404,32.312389 -88.655698,32.443049") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2846640, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388854979186688, "text": "@ebbtideapp Tide in Padre Island, Texas 03/06/2016\n Low 6:34am -0.3\nHigh 3:00pm 1.3\n Low 8:02pm 1.1\nHigh 11:13pm 1.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-97.1567,26.0683"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 43, "friends_count": 1, "statues_count": 13432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855147077632, "text": "Wind 0.0 mph ---. Barometer 30.293 in, Steady. Temperature 34.3 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855335817217, "text": "03/06@03:00 - Temp 32.3F, WC 32.3F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.232in, Falling slowly. Rain 0.00in. Hum 85%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855352659968, "text": "Wind 0.0 mph NE. Barometer 30.306 in, Rising slowly. Temperature 33.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855448948736, "text": "@uhh_sammi the first day I laid my eyes on your beautiful face <3 https://t.co/pvbHIxy6Pg", "in_reply_to_status": 706388371828973568, "in_reply_to_user": 422352840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 422352840 }}, "user": { "id": 1599765992, "name": "sergio", "screen_name": "xxs3rgioxx", "lang": "en", "location": "Inglewood", "create_at": date("2013-07-16"), "description": "null", "followers_count": 170, "friends_count": 175, "statues_count": 5286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855662866432, "text": "00:00", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 301757608, "name": "☄Dånïeł Çørbïeł", "screen_name": "JustDallasB", "lang": "en", "location": "♦️⚪️Dixie State Univ.⚪️♦️", "create_at": date("2011-05-19"), "description": "Cuse--STG I'm going to be that dude you love to hate in the future, if I'm not already.", "followers_count": 1215, "friends_count": 449, "statues_count": 42093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-06T00:00:01.000Z"), "id": 706388855704784896, "text": "Happy 21st birthday to my beautiful sister @alexhbergeson love you to the moon and back 482x ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 408857760 }}, "user": { "id": 1043006436, "name": "holly bergeson", "screen_name": "hollandbergeson", "lang": "en", "location": "Washington ", "create_at": date("2012-12-28"), "description": "I'm a 43 year old ambidextrous butterfly who speaks 6 or 7 languages and specializes in RIPre Calculus, AP BYEology, and LITerature. hmu sometime ;-)", "followers_count": 735, "friends_count": 632, "statues_count": 18539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, WA", "id": "9904dedb2c0909ab", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-123.86033,46.95147 -123.778204,47.001666") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53027, "countyName": "Grays Harbor", "cityID": 5300100, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388856082403328, "text": "Suhhhhhhhh dude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 381220227, "name": "katie culbertson", "screen_name": "kaytaykinzz", "lang": "en", "location": "null", "create_at": date("2011-09-27"), "description": "don't follow me unless you're Matthew Gray Gubler / MD / OC / US Navy / Syd", "followers_count": 1464, "friends_count": 971, "statues_count": 34179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fallston, MD", "id": "01a68c0903e42cf6", "name": "Fallston", "place_type": "city", "bounding_box": rectangle("-76.514298,39.498425 -76.368986,39.570417") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2427700, "cityName": "Fallston" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388856275226625, "text": "Trump just stays looking constipated 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1226906952, "name": "Balreet Singh", "screen_name": "BalreetIsLife", "lang": "en", "location": "null", "create_at": date("2013-02-27"), "description": "Life consists of two dates with a dash in between. Make the dash count. - Stuart Scott", "followers_count": 524, "friends_count": 586, "statues_count": 30027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388856447172609, "text": "Snap me! @frenchchantana ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 303317200 }}, "user": { "id": 303317200, "name": "Mo Mo⭐️", "screen_name": "FrenchChantana", "lang": "en", "location": "TX", "create_at": date("2011-05-22"), "description": "Elijah McKinnie.❤️", "followers_count": 1327, "friends_count": 994, "statues_count": 20454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388856707403776, "text": "*sings songs to Sierra at 3am because I can't sleep. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 809825304, "name": "Madeline", "screen_name": "MadelinePostmus", "lang": "en", "location": "michigan ", "create_at": date("2012-09-07"), "description": "null", "followers_count": 219, "friends_count": 150, "statues_count": 4342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walker, MI", "id": "c9d8828f259682c3", "name": "Walker", "place_type": "city", "bounding_box": rectangle("-85.787954,42.915342 -85.661793,43.059577") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2682960, "cityName": "Walker" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388856845660167, "text": "this is so cute https://t.co/8UEPQT9WJI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 201026762, "name": "♡", "screen_name": "niggggyyy", "lang": "en", "location": "null", "create_at": date("2010-10-10"), "description": "5'3 but my attitude 6'1", "followers_count": 1052, "friends_count": 187, "statues_count": 108440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857277681664, "text": "I had fun tonight !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1948376018, "name": "JayBam", "screen_name": "BBCjayy", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "My Twitter so I Tweet what I want !", "followers_count": 539, "friends_count": 349, "statues_count": 12255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857437233152, "text": "Temp: 47.2°F Wind:0.0mph Pressure: 30.246hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 60991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857852395521, "text": "#SupportOriginMelissa 32.7°F Wind:0.0mph Pressure: 30.24hpa Rising Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857860698112, "text": "My mouth has no filter ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 507736718, "name": "Leyc", "screen_name": "X_IIVXV", "lang": "en", "location": "Buffalo✈️North Carolina", "create_at": date("2012-02-28"), "description": "Single & Gay ❣", "followers_count": 2532, "friends_count": 1788, "statues_count": 77427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857940418561, "text": "Happy birthday to my beautiful bestfriend @Missxchristine !!!!!���������� I love you Esperanza�������� cant wait to see you & tu at BWWs������ ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4725758275 }}, "user": { "id": 2360295608, "name": "MyraTheGee✌️", "screen_name": "boo_xomrey", "lang": "en", "location": "null", "create_at": date("2014-02-24"), "description": "null", "followers_count": 286, "friends_count": 223, "statues_count": 15537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388857969741827, "text": "I'm such a dick ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2287135964, "name": "travi$", "screen_name": "BOOTYHUNTER_69", "lang": "en", "location": "at the rodeo", "create_at": date("2014-01-11"), "description": "doing life one blunt at a time", "followers_count": 406, "friends_count": 261, "statues_count": 13996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388858162663424, "text": "USGS reports a M1.33 #earthquake 6km NW of The Geysers, California on 3/6/16 @ 7:57:39 UTC https://t.co/AAh5zUkkNX #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8083344,38.8193321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 5517, "friends_count": 17, "statues_count": 144941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388858384941057, "text": "@AuldDragon @FrankConniff ok, what's the evidence to the contrary?", "in_reply_to_status": 706386073648304128, "in_reply_to_user": 1308824695, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1308824695, 34556109 }}, "user": { "id": 2337137324, "name": "John Blackman", "screen_name": "John_D_Blackman", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-02-10"), "description": "Atheist only describes something I don't believe. More important to my identity is what I do believe. Secular Humanist is the label that comes closest.", "followers_count": 652, "friends_count": 366, "statues_count": 20502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388858502426624, "text": "53.8F (Feels: 53.8F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 2.2mph - Pressure: 1033.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 229514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388858573721604, "text": "I love everything about this picture https://t.co/mmsFmU8Bp7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25862611, "name": "Andrew Iorio", "screen_name": "Cooksta77", "lang": "en", "location": "Southern California", "create_at": date("2009-03-22"), "description": "Meh", "followers_count": 212, "friends_count": 657, "statues_count": 35251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388858812805120, "text": "������ https://t.co/EbCrFUvZzf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 710149882, "name": "Eat Her Ass", "screen_name": "PrinceGuwop", "lang": "en", "location": "Norfolk, VA", "create_at": date("2013-10-11"), "description": "no hoes and no smoke .....", "followers_count": 1089, "friends_count": 769, "statues_count": 34898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859001577472, "text": "maybe I made the wrong decision here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1577028800, "name": "gabs (:(:", "screen_name": "gabbyyy_perez", "lang": "en", "location": "floresville or lampasas tx ", "create_at": date("2013-07-08"), "description": "dark & twisty", "followers_count": 675, "friends_count": 576, "statues_count": 25809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Floresville, TX", "id": "3f53ed8eba395810", "name": "Floresville", "place_type": "city", "bounding_box": rectangle("-98.179363,29.115435 -98.135405,29.157281") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48493, "countyName": "Wilson", "cityID": 4826160, "cityName": "Floresville" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859068624896, "text": "I fw lil uzi vert", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 316159834, "name": "aaliyah doucet♍", "screen_name": "Liyah_EPIC", "lang": "en", "location": "Baton Rouge, La ", "create_at": date("2011-06-12"), "description": "#GANGSHIT", "followers_count": 792, "friends_count": 186, "statues_count": 22819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859328659456, "text": "A ti todo te puede vale madre", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2464149108, "name": "♛CynthiaSoltero♛", "screen_name": "Girl39Lover", "lang": "en", "location": "null", "create_at": date("2014-04-25"), "description": "Never fear the unknown", "followers_count": 159, "friends_count": 131, "statues_count": 2653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859412557825, "text": "Who is cuter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 829929246, "name": "giselle", "screen_name": "giselleeehh", "lang": "en", "location": "San Jose", "create_at": date("2012-09-17"), "description": "Chico State | ΣΚ", "followers_count": 393, "friends_count": 206, "statues_count": 16928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859441942529, "text": "This app teaches Australia about its 500 Indigenous first nations https://t.co/JgAyNBfeRb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17550, "friends_count": 17561, "statues_count": 68892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859546775552, "text": "USGS reports a M1.33 #earthquake 6km NW of The Geysers, California on 3/6/16 @ 7:57:39 UTC https://t.co/Wb0f8oWc0l #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8083344,38.8193321"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 2654450299, "name": "NorCal Earthquakes", "screen_name": "NorCalEq", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "null", "followers_count": 81, "friends_count": 0, "statues_count": 7716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388859970387968, "text": "Go to bed melo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 945130273, "name": "phatty", "screen_name": "symphonic_ksb", "lang": "en", "location": "null", "create_at": date("2012-11-12"), "description": "innappropriate laugher, '02 Nat. Jr Olympics Ping Pong Champion, Bobcat Preview Co-Chair ig➡️therealsteelebolton", "followers_count": 600, "friends_count": 349, "statues_count": 44884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-03-06T00:00:02.000Z"), "id": 706388860054466560, "text": "Np have a good day �� https://t.co/KkQ34WhU06", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2935693085, "name": "✨", "screen_name": "xx_jayla_xx", "lang": "en", "location": "null", "create_at": date("2014-12-21"), "description": "M❤️", "followers_count": 1014, "friends_count": 556, "statues_count": 10374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newington, VA", "id": "76394a9282b85d3e", "name": "Newington", "place_type": "city", "bounding_box": rectangle("-77.248713,38.707371 -77.161399,38.75928") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5155752, "cityName": "Newington" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388860218023936, "text": "Wind 0.0 mph ---. Barometer 30.26 in, Rising slowly. Temperature 24.4 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 114, "statues_count": 158033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388860595511296, "text": "every time i lick it, you be losin' it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 504683621, "name": "bree.", "screen_name": "BreeDollas__", "lang": "en", "location": "JNT ❤️", "create_at": date("2012-02-26"), "description": "sc: breedoeee | ⚢ | @domiidoe | @anaesee_", "followers_count": 2996, "friends_count": 818, "statues_count": 81789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, NJ", "id": "cff113be4012e4ce", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-74.624348,39.439255 -74.526829,39.511343") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3460030, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388860729561088, "text": "Could've had you. But oh well,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4575878801, "name": "lula✨", "screen_name": "a_badasss", "lang": "en", "location": "null", "create_at": date("2015-12-22"), "description": "spoiled.", "followers_count": 351, "friends_count": 291, "statues_count": 2868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Riverdale, MD", "id": "2163e3af2838ce1c", "name": "East Riverdale", "place_type": "city", "bounding_box": rectangle("-76.930889,38.94415 -76.891633,38.972339") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2424650, "cityName": "East Riverdale" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388860821852161, "text": "St. Albert Houses for rent on RentBoard.ca https://t.co/rwMHtpDvuD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.625642,53.6304753"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31169027, "name": "Rent Board of Canada", "screen_name": "rentboard", "lang": "en", "location": "Canada", "create_at": date("2009-04-14"), "description": "Apartments and Houses for Rent in Canada", "followers_count": 316, "friends_count": 0, "statues_count": 117325 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "St. Albert, Alberta", "id": "56316c24480c1a3a", "name": "St. Albert", "place_type": "city", "bounding_box": rectangle("-113.707088,53.599415 -113.565747,53.681859") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388861442772992, "text": "My cuz Madonna trini_n_tobago_gal reppin' with her gangsta chick pose. #gowiz @ Verizon Center https://t.co/CFTXtJFZlb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.0209198,38.8981285"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gowiz" }}, "user": { "id": 127062525, "name": "Jay Woolridge", "screen_name": "jay_woolridge", "lang": "en", "location": "Maryland", "create_at": date("2010-03-27"), "description": "I help businesses grow! Advise companies on how to utilize the mobile platform to help them be more efficient, competitive and profitable.", "followers_count": 178, "friends_count": 125, "statues_count": 4447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 11, "stateName": "District of Columbia", "countyID": 11001, "countyName": "District of Columbia", "cityID": 1150000, "cityName": "Washington" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388861501353985, "text": "1️⃣6️⃣����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3280059434, "name": "kaitlyn♓️", "screen_name": "kaldrete36", "lang": "en", "location": "null", "create_at": date("2015-07-14"), "description": "EDHS", "followers_count": 888, "friends_count": 1074, "statues_count": 2497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388862315032577, "text": "Hahahaaaaaa ew", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 615055698, "name": "Drew Scott", "screen_name": "imdrewscott", "lang": "en", "location": "Los Angeles", "create_at": date("2012-06-22"), "description": "Online blogger, Picture Taker and Video maker from Los Angeles. Studying at FIDM while attempting to adopt a french bulldog named Timmy!", "followers_count": 1392, "friends_count": 96, "statues_count": 2107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388862705258496, "text": "Who tf reports a tweet about my birthday ? Get a life��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324635002, "name": "Kev✌", "screen_name": "K_Millzzz_", "lang": "en", "location": "Sloot City, Ohigho", "create_at": date("2011-06-26"), "description": "I don't do drugs .", "followers_count": 513, "friends_count": 469, "statues_count": 31756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parma, OH", "id": "dbd1fd3168748e88", "name": "Parma", "place_type": "city", "bounding_box": rectangle("-81.785053,41.350358 -81.68457,41.419489") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3961000, "cityName": "Parma" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388862818344964, "text": "@ermniggkohl so like the best classic rock", "in_reply_to_status": 706388768119361537, "in_reply_to_user": 623443568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 623443568 }}, "user": { "id": 350982914, "name": "jøsie", "screen_name": "phoxxi_", "lang": "en", "location": " sømewhere drinkin' wine ", "create_at": date("2011-08-08"), "description": "~lil twisted tipsy gypsyஜ~ sweet like sugar venøm~ ~føxy enchantress ~ music junky~ @thesmokeguru cooks bomb food - I sell gorgeous vintage jewelry -", "followers_count": 2536, "friends_count": 985, "statues_count": 75786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863124570112, "text": "Dre sleep now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273451686, "name": "__ popthathater", "screen_name": "_ibeenkoolin", "lang": "en", "location": "THE BANK ", "create_at": date("2011-03-28"), "description": "K M I C H E L L E - REALEST FAN", "followers_count": 853, "friends_count": 1071, "statues_count": 28172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethesda, MD", "id": "864ff125241f172f", "name": "Bethesda", "place_type": "city", "bounding_box": rectangle("-77.158594,38.940225 -77.078411,39.022449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863179067392, "text": "@mpoindc we just passed that anniversary. Still a great story.", "in_reply_to_status": 706160807013265408, "in_reply_to_user": 18909534, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18909534 }}, "user": { "id": 1009631790, "name": "Max Eddy", "screen_name": "wmaxeddy", "lang": "en", "location": "NYC", "create_at": date("2012-12-13"), "description": "Analyst for @pcmag. Huge, very buzzworthy and cool. Former write-in candidate for Washtenaw Country drain commissioner.", "followers_count": 996, "friends_count": 380, "statues_count": 21598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863196008448, "text": "Damn Air Freshener now sold here. \n#palaceinn #greenspoint #northside #damnsoldhere #inn… https://t.co/Fp4t4VWXP1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.3687668,29.9382992"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "palaceinn", "greenspoint", "northside", "damnsoldhere", "inn" }}, "user": { "id": 35265018, "name": "Damn Air Freshener", "screen_name": "damnairfresh", "lang": "en", "location": "Houston, TX", "create_at": date("2009-04-25"), "description": "It's the Freshest One Shot Air Freshener that will make you say DAMN!\n Damn, Your Premium Choice.", "followers_count": 7815, "friends_count": 3642, "statues_count": 16724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aldine, TX", "id": "836634cecebe2de4", "name": "Aldine", "place_type": "city", "bounding_box": rectangle("-95.408756,29.891304 -95.355057,29.939071") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4801696, "cityName": "Aldine" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863472631808, "text": "@averyjones23 ��➡️��", "in_reply_to_status": 706388660841635840, "in_reply_to_user": 911426059, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 911426059 }}, "user": { "id": 876628560, "name": "ROBO", "screen_name": "503MichaelR", "lang": "en", "location": "Sherwood, OR", "create_at": date("2012-10-12"), "description": "invest in yourself", "followers_count": 770, "friends_count": 453, "statues_count": 12731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherwood, OR", "id": "6870523b6f8d96e6", "name": "Sherwood", "place_type": "city", "bounding_box": rectangle("-122.869771,45.345593 -122.805551,45.376774") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4167100, "cityName": "Sherwood" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863757918208, "text": "I just smelt pizza and was confused and got up to make a pizza bc it sounded good but then remembered I was already making a pizza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543176160, "name": "mirella❃", "screen_name": "_mireyarenae", "lang": "en", "location": "anywhere but here", "create_at": date("2012-04-01"), "description": "suh dude", "followers_count": 716, "friends_count": 521, "statues_count": 18577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rapid City, SD", "id": "4dcfc855e2614f09", "name": "Rapid City", "place_type": "city", "bounding_box": rectangle("-103.315567,44.01364 -103.151254,44.136814") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46103, "countyName": "Pennington", "cityID": 4652980, "cityName": "Rapid City" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388863971766272, "text": "@larriest HAPPY 19th BIRTHDAY!!!! I LOVE YOU SO MUCH ALICIA ��❤️�������� https://t.co/TY41ZPHk8Y", "in_reply_to_status": -1, "in_reply_to_user": 140269727, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 140269727 }}, "user": { "id": 1371183582, "name": "Edgar Allan Hoe", "screen_name": "carla_altomare", "lang": "en", "location": "null", "create_at": date("2013-04-21"), "description": "one time I met panic! at the disco and Scott Hoying followed me i cry UCSB c/o 2019 ☀️ I stan for Zaddy", "followers_count": 169, "friends_count": 365, "statues_count": 2650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388864085012480, "text": "Makes first pong shot after 30 tries... \"clutch\"...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3676321032, "name": "Nick Spagnola", "screen_name": "spagnolagranola", "lang": "en", "location": "null", "create_at": date("2015-09-24"), "description": "underrated", "followers_count": 282, "friends_count": 265, "statues_count": 712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antelope, CA", "id": "013214de57b4da2e", "name": "Antelope", "place_type": "city", "bounding_box": rectangle("-121.403842,38.702271 -121.30897,38.728927") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602210, "cityName": "Antelope" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388864269578240, "text": "It's tough, but KING still has the best album this year so far to me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 266282582, "name": "Jay", "screen_name": "SarkastikDemon", "lang": "en", "location": "New Orleans", "create_at": date("2011-03-14"), "description": "Don't follow me, I'm an asshole.", "followers_count": 549, "friends_count": 211, "statues_count": 90225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-06T00:00:03.000Z"), "id": 706388864277938176, "text": "@Shiftinator @xParttyy @SawExact skype?", "in_reply_to_status": -1, "in_reply_to_user": 704956222051581952, "favorite_count": 0, "retweet_count": 0, "lang": "lv", "is_retweet": false, "user_mentions": {{ 704956222051581952, 2922269958, 1918826508 }}, "user": { "id": 2426034080, "name": "ᖇᕮGᗩᒪ ᗰᗪᗰ", "screen_name": "ii_MDM", "lang": "en", "location": "200+ SUBSCRIBERS!", "create_at": date("2014-04-03"), "description": "| Max | 17 | YouTuber, Entertainer | Director for @TheRegalReserve | SUBSCRIBE TO MY YOUTUBE! |", "followers_count": 604, "friends_count": 356, "statues_count": 8553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388864944861184, "text": "Up watching lie to me on netflix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4779401647, "name": "PLAYGOD", "screen_name": "Irolldat", "lang": "en", "location": "null", "create_at": date("2016-01-18"), "description": "Who cares?", "followers_count": 74, "friends_count": 127, "statues_count": 307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865150365696, "text": "Happy Birthday @tianaabrownn ❤️ \nI LOVE YOU SO MUCH!!��\nbffs �� finally ur 18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1018404032 }}, "user": { "id": 726070687, "name": "jaenee♡", "screen_name": "jaeneeleee", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "ig: @jaeneeleee ㅡ sc: @jaeneeleee", "followers_count": 703, "friends_count": 418, "statues_count": 18469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865255391232, "text": "Wind 0.0 mph NNW. Barometer 30.230 in, Rising. Temperature 23.1 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865620127744, "text": "Ain't that some shit. https://t.co/R0nBdwsi3J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902940894, "name": "GoDiegoGo", "screen_name": "fernandoo_1904", "lang": "en", "location": "Fresno, CA", "create_at": date("2012-10-24"), "description": "Why you creepin' just follow me damn it.|♌|Soon to be something i haven't had growing up...A Father", "followers_count": 115, "friends_count": 121, "statues_count": 2145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865657995264, "text": "Temp: 48.9°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 30.14 in | Dewpoint: 48.7°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 165532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865700012032, "text": "@kittybrucknell where u gone? I'm next to the guys flashing their Todgers! Come back!!!!!", "in_reply_to_status": -1, "in_reply_to_user": 70767382, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 70767382 }}, "user": { "id": 1405023014, "name": "Rob Cooper", "screen_name": "RobCooperPR", "lang": "en", "location": "London / Los Angeles", "create_at": date("2013-05-05"), "description": "Publicist & Celebrity Promoter. Rob@RobCooperPR.com", "followers_count": 88617, "friends_count": 1493, "statues_count": 411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865896976385, "text": "Reasons why I don't go.to sleep early. I wake up at random times", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134890268, "name": "Lily", "screen_name": "ItsSuperLilyBro", "lang": "en", "location": "Laredo, TX", "create_at": date("2010-04-19"), "description": "tweet", "followers_count": 154, "friends_count": 270, "statues_count": 9085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865934708736, "text": "Bought a cool cat triptych in an art gallery for Cellar Cat partner's birthday. @ Cellar Cat https://t.co/7HrIKys9FS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.1066208,44.6374588"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356433209, "name": "Holly Evans-White", "screen_name": "LeCellarCat", "lang": "en", "location": "Albany, Oregon", "create_at": date("2011-08-16"), "description": "Owner of eclectic wine-centric Cellar Cat (bistro & bar) tweeting from the glorious Willamette Valley Also @CellarCat1", "followers_count": 3609, "friends_count": 4189, "statues_count": 7959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albany, OR", "id": "6083b03ae37cd913", "name": "Albany", "place_type": "city", "bounding_box": rectangle("-123.159583,44.577589 -123.032415,44.684678") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4101000, "cityName": "Albany" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388865959927809, "text": "@Arteezy You should take notes from him XD", "in_reply_to_status": 706388536992215040, "in_reply_to_user": 139169385, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139169385 }}, "user": { "id": 560374836, "name": "Elijah", "screen_name": "elijahantor", "lang": "en", "location": "California, USA", "create_at": date("2012-04-22"), "description": "Picture me rollin'", "followers_count": 207, "friends_count": 205, "statues_count": 7895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388866484187137, "text": "��do I want e...?? no ����������I don't think I do ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365039196, "name": "shy.", "screen_name": "shaniaa_c", "lang": "en", "location": "r.i.p papa⛅️ ", "create_at": date("2011-08-30"), "description": "snapchat; xoshanialove ☺️", "followers_count": 2268, "friends_count": 578, "statues_count": 94455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388866689691649, "text": "Ruby Room Wants To Give You A Special Gift For Your Birthday...https://t.co/wKZ01aWp4A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6716,41.9032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27957411, "name": "Ruby Room", "screen_name": "rubyroomchicago", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2009-03-31"), "description": "to leave a beauty mark on the face of the wellness industry through our commitment to healing, energy and all things positive.", "followers_count": 1120, "friends_count": 1714, "statues_count": 1879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388866870046720, "text": "This #Retail #job might be a great fit for you: Manager Trainee-Retail Sales - https://t.co/6tyU53z6Sf #Athens, TN #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5929898,35.4428512"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "Athens", "Hiring" }}, "user": { "id": 3144822634, "name": "Speedway Jobs", "screen_name": "SpeedwayJobs", "lang": "en", "location": "Nationwide", "create_at": date("2015-04-07"), "description": "Rethink Speedway. It's more than a convenience store... It's your career opportunity!", "followers_count": 215, "friends_count": 27, "statues_count": 6427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, TN", "id": "35f73f0698fa5f98", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-84.688591,35.398719 -84.525246,35.492382") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47107, "countyName": "McMinn", "cityID": 4702320, "cityName": "Athens" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388866895323136, "text": "@Grubeee_ get out my Twitter mentions��", "in_reply_to_status": 706388721709481984, "in_reply_to_user": 227918190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 227918190 }}, "user": { "id": 4785089890, "name": "J", "screen_name": "JasonJohnsJr", "lang": "en", "location": "null", "create_at": date("2016-01-12"), "description": "#LowLife - #LackOfWorries - #LivingInFearlessEfforts", "followers_count": 164, "friends_count": 172, "statues_count": 709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388867281199104, "text": "Fall of the hulks @ City Of Antioch, CA. https://t.co/2zilght5BL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.7806877,37.96597107"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2891535980, "name": "Jonathan Bruno", "screen_name": "JgJohnnycage", "lang": "en", "location": "null", "create_at": date("2014-11-05"), "description": "anime geek and a gamer.", "followers_count": 62, "friends_count": 229, "statues_count": 805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388867314688001, "text": "@veezy_SQ sick game tonight homie. Good seeing you again.", "in_reply_to_status": -1, "in_reply_to_user": 77031637, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 77031637 }}, "user": { "id": 2633990989, "name": "nick.", "screen_name": "nickgasparyan", "lang": "en", "location": "Los Angeles, CA, USA", "create_at": date("2014-07-12"), "description": "Devil in the City of Angels.", "followers_count": 70, "friends_count": 50, "statues_count": 22 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388867646169088, "text": "#GeneralDisturbance at 1800-1999 Blossom Terrace. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4036172,28.4789035"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GeneralDisturbance", "orlpol", "ocso" }}, "user": { "id": 39065901, "name": "Police Calls 32839", "screen_name": "orlpol32839", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 153, "friends_count": 1, "statues_count": 38794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge, FL", "id": "aa9eedbd533da630", "name": "Oak Ridge", "place_type": "city", "bounding_box": rectangle("-81.442112,28.450553 -81.396669,28.487021") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1250638, "cityName": "Oak Ridge" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388867906146305, "text": "B DAY TWERK OR NAH @kmichelle ���� @ BLISS Nightclub https://t.co/3Jg62VUe78", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.9716034,38.9197998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28442098 }}, "user": { "id": 31346948, "name": "PARIS LONDON", "screen_name": "PARISPLONDON", "lang": "en", "location": "M-TOWN", "create_at": date("2009-04-14"), "description": "♕ #TEAMPLONDON #GEMINI #108WAPG FOR FEATURES, HOSTING, OR PERFORMANCE Bookplondon@gmail.com {FEMALE RAP ARTIST}", "followers_count": 13807, "friends_count": 9219, "statues_count": 87342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 11, "stateName": "District of Columbia", "countyID": 11001, "countyName": "District of Columbia", "cityID": 1150000, "cityName": "Washington" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388868057034753, "text": "@SvbTweetGod lmao we really do though", "in_reply_to_status": 706388751459549184, "in_reply_to_user": 408921616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 408921616 }}, "user": { "id": 2368211832, "name": "ULT CHEF CJ", "screen_name": "cj_all_day_01", "lang": "en", "location": "Yakima, WA", "create_at": date("2014-03-01"), "description": "20.\nYVCC\n@Izabel_9", "followers_count": 565, "friends_count": 528, "statues_count": 20507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388868296085505, "text": "Simply Hard Rock @ Hard Rock Cafe Houston https://t.co/8KTcDhqgZJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.36697476,29.7627046"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3045641017, "name": "Johann Merida Perez", "screen_name": "johannmerida75", "lang": "es", "location": "null", "create_at": date("2015-02-26"), "description": "null", "followers_count": 627, "friends_count": 1387, "statues_count": 281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-06T00:00:04.000Z"), "id": 706388868585562112, "text": "#ShoutOut to @NateDiaz209 .... Keeps it too real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ShoutOut" }}, "user_mentions": {{ 52181976 }}, "user": { "id": 314155396, "name": "T-Bob Hebert", "screen_name": "TBob53", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-06-09"), "description": "I played football for LSU, I love video games (LoL, Hearthstone), comic books, and LoTR; Host of Night Show Double Coverage on WWL 870 in Nola BILBOSWAGGIN$", "followers_count": 16501, "friends_count": 1533, "statues_count": 30297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-03-06T00:00:05.000Z"), "id": 706388868992540673, "text": "This nigga is lame lol he can't hang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1426632968, "name": "Rachel", "screen_name": "rachelsargentt", "lang": "en", "location": "point pleasant, WV", "create_at": date("2013-05-13"), "description": "null", "followers_count": 544, "friends_count": 340, "statues_count": 12870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Point Pleasant, WV", "id": "62093956700bed6e", "name": "Point Pleasant", "place_type": "city", "bounding_box": rectangle("-82.141247,38.823843 -82.105925,38.882844") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54053, "countyName": "Mason", "cityID": 5464708, "cityName": "Point Pleasant" } }
+{ "create_at": datetime("2016-03-06T00:00:05.000Z"), "id": 706388869386674177, "text": "Finished some custom pieces tonight. Shells from the beach of couples honeymoon. #shells #beach #jewelry https://t.co/B0daLK8eVG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shells", "beach", "jewelry" }}, "user": { "id": 21555831, "name": "Ruth Tillman", "screen_name": "Ruth_Tillman", "lang": "en", "location": "Bay Area, CA", "create_at": date("2009-02-22"), "description": "Yarn artist, designer, teacher, mother of three teens and in love with anything artsy.", "followers_count": 200, "friends_count": 159, "statues_count": 840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alameda, CA", "id": "000e96b4e9f8503f", "name": "Alameda", "place_type": "city", "bounding_box": rectangle("-122.332411,37.720367 -122.224562,37.797229") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 600562, "cityName": "Alameda" } }
+{ "create_at": datetime("2016-03-06T00:00:05.000Z"), "id": 706388869478912000, "text": "@realgay22 knows how to serve #Cakes #Gay #Gogo #GogoDancer #Instagay #Seattle #Nightlife #Twerk… https://t.co/jZbJ0ufF05", "in_reply_to_status": -1, "in_reply_to_user": 948911227, "favorite_count": 0, "coordinate": point("-122.32353169,47.61518329"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Cakes", "Gay", "Gogo", "GogoDancer", "Instagay", "Seattle", "Nightlife", "Twerk" }}, "user_mentions": {{ 948911227 }}, "user": { "id": 36615782, "name": "Eric Martin", "screen_name": "riqmartin", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "The one and only me! Living, loving, and making the best of every day!", "followers_count": 109, "friends_count": 282, "statues_count": 781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-06T00:00:05.000Z"), "id": 706388869676052480, "text": "Let's hear it for @AgwaDeBolivia & let's hear it for @Jankie2099 @XXXDaytonR @drsuzy he says it's the weirdest book #promo he's ever done.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "promo" }}, "user_mentions": {{ 24349451, 59958830, 1566783126, 14355949 }}, "user": { "id": 2703342968, "name": "Chelsea Demoiselle", "screen_name": "chelseabonobo", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-08-02"), "description": "Eco-Priestess, Sacred BodyCare, Free Hugz onVenice Boardwalk #@ #Vegan", "followers_count": 589, "friends_count": 1218, "statues_count": 4439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-06T00:00:05.000Z"), "id": 706388870091431936, "text": "Ha ha McGregor lost", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231262275, "name": "Kevin", "screen_name": "YeaaahKevin", "lang": "en", "location": "Winterfell, North", "create_at": date("2010-12-27"), "description": "Why your shirt dusty, boy?", "followers_count": 468, "friends_count": 227, "statues_count": 29991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Spring, MD", "id": "6417871953fa5e86", "name": "Silver Spring", "place_type": "city", "bounding_box": rectangle("-77.064086,38.979735 -76.97162,39.036964") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2472450, "cityName": "Silver Spring" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751235710300160, "text": "@SkizzyMars https://t.co/zXONsEFWIO", "in_reply_to_status": -1, "in_reply_to_user": 139869817, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 139869817 }}, "user": { "id": 2547699717, "name": "jon", "screen_name": "Chiverss_", "lang": "en", "location": "California ", "create_at": date("2014-05-14"), "description": "#ProjectLG", "followers_count": 602, "friends_count": 590, "statues_count": 12705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751236192780288, "text": "@akbullets what about bang 3?", "in_reply_to_status": 706751040897441792, "in_reply_to_user": 51495746, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51495746 }}, "user": { "id": 3035518853, "name": "神", "screen_name": "NICK_EXT", "lang": "en", "location": "Stony Brook, NY", "create_at": date("2015-02-13"), "description": "null", "followers_count": 220, "friends_count": 350, "statues_count": 4 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stony Brook, NY", "id": "8241555e75571517", "name": "Stony Brook", "place_type": "city", "bounding_box": rectangle("-73.151367,40.868719 -73.096505,40.948108") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3671608, "cityName": "Stony Brook" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751236314259456, "text": "@nottodaysatan23 hell yeah bih ����", "in_reply_to_status": 706746285412319232, "in_reply_to_user": 2806987219, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 2806987219 }}, "user": { "id": 213073542, "name": "YUNGNINJAPRINCESS✨", "screen_name": "princesavesta", "lang": "en", "location": "Neverland", "create_at": date("2010-11-07"), "description": "Jhamari ❣", "followers_count": 402, "friends_count": 285, "statues_count": 62484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751236691726337, "text": "Rainy night, Los Feliz!! Wine and cheese at Figaro cafe! https://t.co/X7i8Fg8tQ7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120170855, "name": "Adam Guthrie", "screen_name": "AdmGuth3", "lang": "en", "location": "Los Angeles CA", "create_at": date("2010-03-05"), "description": "Actor, Film Maker, Comedian, Drummer, Gamer, Artist, Lover: these are the words I write across my forehead.", "followers_count": 7709, "friends_count": 3501, "statues_count": 575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751236981157889, "text": "How I feel about my homework rn https://t.co/ZtoRr90k6k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350627956, "name": "тαyℓσя ♍️", "screen_name": "TaylorCPage", "lang": "en", "location": "Stillwater, OK", "create_at": date("2011-08-07"), "description": "I love cats, life & michael. OSU ΑΞΔ", "followers_count": 510, "friends_count": 383, "statues_count": 14280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751237140537344, "text": "Just posted a photo @ Erin's Bar https://t.co/Yl6wfIYsXK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.533493,31.8325996"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043564484, "name": "RediscoveringAmerica", "screen_name": "RediscovAmerica", "lang": "en", "location": "America", "create_at": date("2012-12-28"), "description": "Travelling the Highways and Byways of America and Rediscovering it one place at a time.", "followers_count": 35, "friends_count": 51, "statues_count": 481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751237929115648, "text": "@_UrSoREPLACED ����", "in_reply_to_status": 706751158014992384, "in_reply_to_user": 319150371, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 319150371 }}, "user": { "id": 106027471, "name": "3/25 #BootCamp2k16", "screen_name": "DjDoesItAll", "lang": "en", "location": "Dallas, TX", "create_at": date("2010-01-17"), "description": "@DefinitionDjs | DjDoesitall@yahoo.com | 4X DJ Battle Champion | #CKB | #TAMUC | #1525 #RIPDJKIDD #RIPCLYDE", "followers_count": 4864, "friends_count": 3802, "statues_count": 159419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751238461784064, "text": "Lil nigga got me up https://t.co/q25uIlsTlc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177362114, "name": "Michael Anthony", "screen_name": "MichaelAnth0ny_", "lang": "en", "location": "null", "create_at": date("2010-08-11"), "description": "lord forgive me for my sins || 18 ||", "followers_count": 844, "friends_count": 586, "statues_count": 11915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751238491123712, "text": "Lights will guide you home.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3299975443, "name": "fire", "screen_name": "shurrrrdannnn", "lang": "en", "location": "Moscow, ID", "create_at": date("2015-07-28"), "description": "maybe I should pack my things and move on //", "followers_count": 297, "friends_count": 336, "statues_count": 6781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moscow, ID", "id": "be3410e825fe4fa4", "name": "Moscow", "place_type": "city", "bounding_box": rectangle("-117.039699,46.710846 -116.961769,46.758567") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16057, "countyName": "Latah", "cityID": 1654550, "cityName": "Moscow" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751238650474497, "text": "Wind 13.0 mph SSW. Barometer 1012.60 mb, Falling. Temperature 62.4 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751238839279616, "text": "Million x Tink>>>>> ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324074501, "name": "BeautifulBliss✨", "screen_name": "Pearlfect_Ky", "lang": "en", "location": "OKC ✈️ Las Vegas", "create_at": date("2011-06-25"), "description": "Psalm 23:4 1 9 0 8 Θβ", "followers_count": 908, "friends_count": 511, "statues_count": 29728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751239220916224, "text": "I CORRECTED MY SELF BEFORE UR BITXH ASS COULD OMG https://t.co/UPcQCGo0P7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3053537701, "name": "jennifer armstrong", "screen_name": "l0ljenniferr", "lang": "en", "location": "null", "create_at": date("2015-03-01"), "description": "null", "followers_count": 200, "friends_count": 226, "statues_count": 2832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751239254462464, "text": "����- you're my bestfriend & I miss N love you ���� btw you're selfies fleeky n hmu I wanna hangout ! N I need to come over to bless ur new home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3005123283, "name": "-", "screen_name": "joxhva", "lang": "en", "location": "Medford, OR", "create_at": date("2015-01-29"), "description": "content", "followers_count": 238, "friends_count": 232, "statues_count": 3068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, OR", "id": "7520fc0be21c62bf", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-122.96154,42.288726 -122.776437,42.398452") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4147000, "cityName": "Medford" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751239346720768, "text": "⚔���� https://t.co/HpNKcjRQB9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 32243077, "name": "Mandy Vecks", "screen_name": "mandyvecks", "lang": "en", "location": "SD/LA", "create_at": date("2009-04-16"), "description": "#1daf | Instagram: @mandyvecks", "followers_count": 475, "friends_count": 156, "statues_count": 14912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-03-07T00:00:00.000Z"), "id": 706751239606951936, "text": "Ripley SW Limestone Co. Temp: 50.5°F Wind:2.2mph Pressure: 1000.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 49925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751240105889793, "text": "why can't I sleep ��/ 11:59 https://t.co/SGVgBJ3DDC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2852178745, "name": "jas", "screen_name": "jjasminemmarie", "lang": "en", "location": "null", "create_at": date("2014-10-11"), "description": "lhs", "followers_count": 112, "friends_count": 105, "statues_count": 1028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751240214941696, "text": "Haven't slept since I've been in Florida and I'm gonna go nuts I need my bed back ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403618951, "name": "sunshine", "screen_name": "KostopoulosAnna", "lang": "en", "location": "Fort Myers, FL", "create_at": date("2011-11-02"), "description": "FSW radiology major always in my thoughts rest easy brothas TA &MCJ snapchattt; rastaaahbby97", "followers_count": 375, "friends_count": 648, "statues_count": 6669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westbrook, ME", "id": "0105162a703442f4", "name": "Westbrook", "place_type": "city", "bounding_box": rectangle("-70.391816,43.644549 -70.311199,43.75233") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2382105, "cityName": "Westbrook" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751240559001600, "text": "Getting intimate on acid is 2 euphoric https://t.co/zuXl16fGjU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521863038, "name": "Haile Selassie", "screen_name": "TrillTeo", "lang": "en", "location": "Rest in peace Monroe", "create_at": date("2012-03-11"), "description": "The more you educate yourself the more you understand where things come from the more obvious things become and you begin to see lies everywhere.", "followers_count": 1394, "friends_count": 1006, "statues_count": 52597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751240655368192, "text": "Throw that @ https://t.co/Nrotdx6YMb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 768244532, "name": "Gilbert ⚽️", "screen_name": "G_Rivers97", "lang": "en", "location": "Becky ", "create_at": date("2012-08-19"), "description": "18|♓️|⚽️|'13 Civic|'10 Genesis Coupe", "followers_count": 1456, "friends_count": 1203, "statues_count": 59767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751240781172736, "text": "Ash thought I was flirting with the worker from the gym like can I not make friends ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134249592, "name": "Vanessa", "screen_name": "_vanessssa14", "lang": "en", "location": "Texas, USA", "create_at": date("2010-04-17"), "description": "Live like there's no midnight. ✨ sc: vanessag014", "followers_count": 627, "friends_count": 466, "statues_count": 78527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751241225789440, "text": "@theericoo hey boo", "in_reply_to_status": 706751179452100608, "in_reply_to_user": 240112777, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240112777 }}, "user": { "id": 2421408218, "name": "no", "screen_name": "sadalieee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-03-31"), "description": "I don't know.", "followers_count": 1183, "friends_count": 648, "statues_count": 60504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751241309687808, "text": "@kaileyygladden �� https://t.co/h0W8i2esEz", "in_reply_to_status": -1, "in_reply_to_user": 574830333, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 574830333 }}, "user": { "id": 544833192, "name": "Cameron", "screen_name": "_Mr_Brooks", "lang": "en", "location": "HG 903 ✈ Den10 ", "create_at": date("2012-04-03"), "description": "RIP E-Love, Granny, Eva, & Tootie | UNT", "followers_count": 3193, "friends_count": 2910, "statues_count": 111554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751241599102976, "text": "Me encanta Jorge Bucay acompañado de un buen té", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 979631832, "name": "♧", "screen_name": "JrGervi", "lang": "es", "location": "El Paso TX", "create_at": date("2012-11-29"), "description": "Snapchat:GervyL", "followers_count": 436, "friends_count": 301, "statues_count": 8308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751241682968576, "text": "@ebbtideapp Tide in Honolulu, Hawaii 03/07/2016\nHigh 3:03am 2.0\n Low 9:41am -0.2\nHigh 3:19pm 1.2\n Low 9:05pm -0.3\nHigh 3:42am 2.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-157.8667,21.3067"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 41, "friends_count": 1, "statues_count": 13713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaii, USA", "id": "9dafd05b1158873b", "name": "Hawaii", "place_type": "admin", "bounding_box": rectangle("-178.443593,18.86546 -154.755792,28.517269") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242006040576, "text": "Lord if I don't say this enough I just want to say thank you for everything you do! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348586487, "name": "Zeús⚡️", "screen_name": "JRsmooth_01", "lang": "en", "location": "Vidalia, Ga", "create_at": date("2011-08-04"), "description": "Following God's plan he has in store for me. #vsufootball #VSU19 @beautiful_emmm ❤️", "followers_count": 660, "friends_count": 539, "statues_count": 4480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valdosta, GA", "id": "5e1c91065bc30991", "name": "Valdosta", "place_type": "city", "bounding_box": rectangle("-83.374825,30.762483 -83.230199,30.949686") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1378800, "cityName": "Valdosta" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242035449857, "text": "0h 30m wait time at Strathcona Community Hospital. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #SherwoodPark", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.277154,53.568215"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "SherwoodPark" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 369, "friends_count": 884, "statues_count": 2942 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Strathcona County, Alberta", "id": "79a60b51078a624f", "name": "Strathcona County", "place_type": "city", "bounding_box": rectangle("-113.402012,53.354476 -112.840174,53.893471") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242064809984, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 366897412, "name": "Cash Money", "screen_name": "Cash2_Cool", "lang": "en", "location": "DTX ", "create_at": date("2011-09-02"), "description": "#NC'17 Live.Learn.Succeed", "followers_count": 1470, "friends_count": 641, "statues_count": 129765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corsicana, TX", "id": "faa25283bbfe186a", "name": "Corsicana", "place_type": "city", "bounding_box": rectangle("-96.537914,32.054664 -96.415163,32.131294") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48349, "countyName": "Navarro", "cityID": 4817060, "cityName": "Corsicana" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242391826436, "text": "You deserve the same love you've given others.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2169008217, "name": "Vane", "screen_name": "sanches3396", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "null", "followers_count": 266, "friends_count": 234, "statues_count": 6452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242668650496, "text": "Investments the time in a good way is lead you to success . ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1145987202, "name": "Salman fahad", "screen_name": "Aldoossary819", "lang": "ar", "location": "Pullman, WA", "create_at": date("2013-02-03"), "description": "null", "followers_count": 50, "friends_count": 87, "statues_count": 348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242752626688, "text": "Wind 5.0 mph SE. Barometer 29.893 in, Falling. Temperature 59.3 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751242911887360, "text": "@miagold23 RETWEET A MILLION GODDAMN TIMES DUDE PREACH����", "in_reply_to_status": 706745773778538497, "in_reply_to_user": 2195192587, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2195192587 }}, "user": { "id": 2278905229, "name": "Kayyleanna Ridens", "screen_name": "kayleannuhh", "lang": "en", "location": "Gold Beach, OR", "create_at": date("2014-01-06"), "description": "9/20/15 Rocky Lilly has my heart❤️", "followers_count": 285, "friends_count": 368, "statues_count": 2363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gold Beach, OR", "id": "01f6cdd5442c1cf9", "name": "Gold Beach", "place_type": "city", "bounding_box": rectangle("-124.427604,42.388923 -124.406375,42.424825") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41015, "countyName": "Curry", "cityID": 4129900, "cityName": "Gold Beach" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751243092267008, "text": "Then he told me he snatched him right out of the car and was about to stomp his head on the sidewalk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332100770, "name": "David", "screen_name": "Davidbodybuilds", "lang": "en", "location": "The gym", "create_at": date("2011-07-08"), "description": "BodyBuilding, not physique", "followers_count": 321, "friends_count": 282, "statues_count": 6247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751243272765440, "text": "Wind 0.0 mph SSE. Barometer 30.101 in, Falling slowly. Temperature 42.9 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751243322961920, "text": "I just want a girl with a big forehead and a big heart", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3260794502, "name": "Johnny Manziel", "screen_name": "AaronJones4Real", "lang": "en", "location": "Planet Vegeta", "create_at": date("2015-06-29"), "description": "Wooooooooooo!! *Ric Flair Voice", "followers_count": 456, "friends_count": 492, "statues_count": 21883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-03-07T00:00:01.000Z"), "id": 706751243595620352, "text": "#SupportOriginMelissa 46.8°F Wind:2.9mph Pressure: 30.12hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751244023554049, "text": "03/07@03:00 - Temp 27.9F, WC 27.9F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.314in, Falling slowly. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751244136677376, "text": "That's what big's are for �� thanks biggie smalls ❤️�� https://t.co/iBf9TNcG7R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54170152, "name": "ⓡⓞⓢⓔ❥❁", "screen_name": "dearroselyn", "lang": "en", "location": "SD", "create_at": date("2009-07-06"), "description": "CSUSM ♔ Alpha Omicron Pi", "followers_count": 425, "friends_count": 283, "statues_count": 11393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751245428523012, "text": "@IAMZAYJONES https://t.co/EsfQ1aP4Yy", "in_reply_to_status": -1, "in_reply_to_user": 114078409, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 114078409 }}, "user": { "id": 309842848, "name": "T Rodgers", "screen_name": "TrillxRico", "lang": "en", "location": "Dallas, TX Houston, TX Tyler, TX", "create_at": date("2011-06-02"), "description": "Free Troy Free A Dunn Free Streetz Free D Nick RIP Money Mayo RIp Blyde 11/8 #PlugBrothaz #TJC", "followers_count": 2743, "friends_count": 358, "statues_count": 175049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751245621596160, "text": "Temp: 47.9°F Wind:0.1mph Pressure: 30.156hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751245722103808, "text": "03:00 45.5°F Feels:45.5°F (Hi48.4°F/Lo42.4°F) Hum:63% Wnd:--- 0.0MPH Baro:29.76in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 190, "friends_count": 262, "statues_count": 26588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751245793566721, "text": "Barberton Oh Temp:34.4°F Wind:3 mph Dir:S Baro:Falling slowly Rain2day:0.00in Hum:81% UV:0.0 @ 03:00 03/07/15 #weather #Ohio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.57861111,40.98861111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "Ohio" }}, "user": { "id": 14460569, "name": "miles m", "screen_name": "weather_outside", "lang": "en", "location": "Barberton, Ohio", "create_at": date("2008-04-21"), "description": "Weather data from Barberton / Portage Lakes / Coventry Township Ohio auto tweeted unless I got nothing to say then I will tweet that personally.", "followers_count": 233, "friends_count": 229, "statues_count": 115167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751245915201541, "text": "Donald Trump cost the party nothing \nNot a dime Romney you are so out of line", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3339087916, "name": "3306856363", "screen_name": "claytons1952171", "lang": "en", "location": "null", "create_at": date("2015-06-21"), "description": "null", "followers_count": 518, "friends_count": 435, "statues_count": 8665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massillon, OH", "id": "005f11de9931c8a4", "name": "Massillon", "place_type": "city", "bounding_box": rectangle("-81.631799,40.732694 -81.432311,40.896962") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3948244, "cityName": "Massillon" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751246338826240, "text": "Hungry but whatever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242099346, "name": "Luca Bondanelli Jr.", "screen_name": "Lbond47", "lang": "en", "location": "Iron Paradise", "create_at": date("2011-01-23"), "description": "God is gangsta #YeezySeason #FreeRetch", "followers_count": 545, "friends_count": 397, "statues_count": 51135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florham Park, NJ", "id": "cf24f6a1fc150ee7", "name": "Florham Park", "place_type": "city", "bounding_box": rectangle("-74.440536,40.750397 -74.35756,40.798281") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris", "cityID": 3423910, "cityName": "Florham Park" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751246388953089, "text": "@realDonaldTrump LEAVE MY COUNTRY, U IMMIGRANT GET THE FUCK OUT OF MY COUNTRY U WILL NEVER BE PRESIDENT #FACT", "in_reply_to_status": -1, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FACT" }}, "user_mentions": {{ 25073877 }}, "user": { "id": 383445309, "name": "EminemsRealWife", "screen_name": "EminemsRealMrs", "lang": "en", "location": "East Pointe, Detroit MI", "create_at": date("2011-10-01"), "description": "Eminem's Real Mrs. 42 yrs old #Aquarius wit a Moon in #Gemini, My own #TeamShady Detroit, Mother of a 20 yr old & a 17 yr old son,massage therapist", "followers_count": 5978, "friends_count": 3094, "statues_count": 176452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastpointe, MI", "id": "0969de5c93c437c4", "name": "Eastpointe", "place_type": "city", "bounding_box": rectangle("-82.971839,42.449927 -82.916743,42.48053") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2624290, "cityName": "Eastpointe" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751246938451968, "text": "Fuck my name on little liquor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 933301027, "name": "omar", "screen_name": "Itsomar2", "lang": "en", "location": "null", "create_at": date("2012-11-07"), "description": "htx", "followers_count": 237, "friends_count": 102, "statues_count": 5231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247336910849, "text": "Wind 0.0 mph W. Barometer 30.18 in, Falling slowly. Temperature 27.7 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 114, "statues_count": 158057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247454330880, "text": "61.0F (Feels: 61.0F) - Humidity: 87% - Wind: 9.8mph E - Gust: 9.8mph - Pressure: 1014.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 229656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247542452224, "text": "Oomf's tattoos >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497569302, "name": "ad", "screen_name": "adrianagarcia5", "lang": "en", "location": "Lackland Air Force Base, TX", "create_at": date("2012-02-19"), "description": "Drink one more cold one // Texas Forever", "followers_count": 354, "friends_count": 1318, "statues_count": 14882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247697612800, "text": "'Pizza in a cup': Aussies come up with ingenious inventions in #ideasboom https://t.co/pDe3IFG1FD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ideasboom" }}, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17555, "friends_count": 17562, "statues_count": 68921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247924072448, "text": "Literally can't sleep beacuse im anxious of whats to come", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832600398, "name": "bram", "screen_name": "babydiaz_", "lang": "en", "location": "null", "create_at": date("2012-09-18"), "description": "null", "followers_count": 297, "friends_count": 144, "statues_count": 24247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rodeo, CA", "id": "712d61ba26321517", "name": "Rodeo", "place_type": "city", "bounding_box": rectangle("-122.274848,38.014234 -122.238027,38.057435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 662490, "cityName": "Rodeo" } }
+{ "create_at": datetime("2016-03-07T00:00:02.000Z"), "id": 706751247961837568, "text": "I'm not tired or sleepy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1161694147, "name": "Deams", "screen_name": "deamberparker", "lang": "en", "location": "Duarte, CA ", "create_at": date("2013-02-08"), "description": "19. How you gon win if you aint right within", "followers_count": 191, "friends_count": 129, "statues_count": 24452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Monrovia Island, CA", "id": "012ee70c473b33ea", "name": "South Monrovia Island", "place_type": "city", "bounding_box": rectangle("-118.003851,34.117437 -117.987719,34.130449") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673167, "cityName": "South Monrovia Island" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751248842665985, "text": "Need a nose hair trimmer like shit lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352052368, "name": "A.J W.⚡️", "screen_name": "ItsAlwaysAJ", "lang": "en", "location": "Colorado, USA", "create_at": date("2011-08-09"), "description": "IG/SC @itsalwaysaj, college hooper, sneaker enthusiast.", "followers_count": 623, "friends_count": 772, "statues_count": 24983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alamosa, CO", "id": "fb4cac3f448732d3", "name": "Alamosa", "place_type": "city", "bounding_box": rectangle("-105.912016,37.440834 -105.856303,37.494073") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8003, "countyName": "Alamosa", "cityID": 801090, "cityName": "Alamosa" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751248964395008, "text": "@theywantdarell ain't tell me bout this https://t.co/mpcdNuvCQU", "in_reply_to_status": -1, "in_reply_to_user": 376458612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 376458612 }}, "user": { "id": 228934761, "name": "IG @_taetoofine", "screen_name": "taetooFINE", "lang": "en", "location": "null", "create_at": date("2010-12-20"), "description": "hey there, hey there #SU", "followers_count": 1365, "friends_count": 612, "statues_count": 127914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751248968454144, "text": "And meek mill continues to meek mill...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428230810, "name": ",,", "screen_name": "masontheafakasi", "lang": "en", "location": "Anchorage, AK", "create_at": date("2014-04-04"), "description": "w$ t$ j$ ™", "followers_count": 1291, "friends_count": 1334, "statues_count": 19474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751249014624256, "text": "@Quincy is daddy ��", "in_reply_to_status": -1, "in_reply_to_user": 22900332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22900332 }}, "user": { "id": 1950157842, "name": "elaina", "screen_name": "lainnersss", "lang": "en", "location": "Lompoc, CA", "create_at": date("2013-10-09"), "description": "20 ♏️ Employed", "followers_count": 175, "friends_count": 130, "statues_count": 20816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lompoc, CA", "id": "68108bea30da38c3", "name": "Lompoc", "place_type": "city", "bounding_box": rectangle("-120.507681,34.619205 -120.432998,34.689644") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 642524, "cityName": "Lompoc" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751249417420801, "text": "1 Baltimore Oriole (Icterus galbula) - Sweetwater Wetlands - 2016-03-06 07:30 https://t.co/vtOnCLFDzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12606 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751249421434880, "text": "@jessicarivera__ @bbysofs me ��������", "in_reply_to_status": 706751128457711616, "in_reply_to_user": 826648867, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 826648867, 1576725020 }}, "user": { "id": 2568396764, "name": "Brazy Brazil ☀️", "screen_name": "Devanicolelayda", "lang": "en", "location": "null", "create_at": date("2014-06-14"), "description": "sc: devanicole insta: devss11 From a sunny state of no chill", "followers_count": 697, "friends_count": 515, "statues_count": 25168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollister, CA", "id": "2417f030a58de0ce", "name": "Hollister", "place_type": "city", "bounding_box": rectangle("-121.450786,36.816282 -121.363572,36.874852") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6069, "countyName": "San Benito", "cityID": 634120, "cityName": "Hollister" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751249467764736, "text": "�� https://t.co/IBlLLKCdeE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2844244946, "name": "cayla", "screen_name": "caylamiragliaaa", "lang": "en", "location": "516-941-850", "create_at": date("2014-10-07"), "description": "UWF19. Public Relations Major. CAB. SAA. Catholic Argos. European Spanish Italian", "followers_count": 372, "friends_count": 426, "statues_count": 9054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferry Pass, FL", "id": "33a522a8e7317b6e", "name": "Ferry Pass", "place_type": "city", "bounding_box": rectangle("-87.247666,30.479609 -87.158257,30.562505") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1222275, "cityName": "Ferry Pass" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751249949941760, "text": "When bae sticks his big ass Jew nose in and out your pussy ���������� YAAAS DADDY https://t.co/9yowj7901T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3732424334, "name": "Kucci Mane", "screen_name": "slutmuffinx", "lang": "en", "location": "null", "create_at": date("2015-09-29"), "description": "crip gang or die", "followers_count": 175, "friends_count": 223, "statues_count": 1499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751250121949184, "text": "@a__butter happy 20th big man! Hope it's a slam dunk! https://t.co/gN6ShKGAYH", "in_reply_to_status": -1, "in_reply_to_user": 2295666643, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2295666643 }}, "user": { "id": 258440439, "name": "hayden", "screen_name": "yruhayden", "lang": "en", "location": "Keizer, OR", "create_at": date("2011-02-27"), "description": "null", "followers_count": 764, "friends_count": 273, "statues_count": 37995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keizer, OR", "id": "fd5dc77c129022ee", "name": "Keizer", "place_type": "city", "bounding_box": rectangle("-123.05417,44.972903 -122.99073,45.038125") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4138500, "cityName": "Keizer" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751250839138305, "text": "I'm irritated with multiple things atm. I just want sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 181007172, "name": "Breanna Navar", "screen_name": "AyeBreanna", "lang": "en", "location": "stuck in vegas", "create_at": date("2010-08-20"), "description": "journalism major, unlv // 19. // hopeless opus", "followers_count": 618, "friends_count": 545, "statues_count": 21607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751250843340800, "text": "@wrestlerpro15 https://t.co/HVUHneqz8v", "in_reply_to_status": 706750963869024256, "in_reply_to_user": 607928164, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 607928164 }}, "user": { "id": 258483800, "name": "Josh Hernandez", "screen_name": "_JoshEHernandez", "lang": "en", "location": "Kalamazoo, MI", "create_at": date("2011-02-27"), "description": "Stay chill fam Snapchat: joshehernandez", "followers_count": 434, "friends_count": 285, "statues_count": 16209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Juan Capistrano, CA", "id": "4aea239b6a146e77", "name": "San Juan Capistrano", "place_type": "city", "bounding_box": rectangle("-117.686553,33.466555 -117.611591,33.54673") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 668028, "cityName": "San Juan Capistrano" } }
+{ "create_at": datetime("2016-03-07T00:00:03.000Z"), "id": 706751251581526018, "text": "@TheSolarSpirit @zackjfrank @RBReich Simple, but doesn'tmake upfor lackof knowledgeof theissues.", "in_reply_to_status": 706672132667772929, "in_reply_to_user": 177357302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177357302, 435215755, 148529707 }}, "user": { "id": 4274523498, "name": "janet gilles", "screen_name": "GillesJanet", "lang": "en", "location": "San Francisco/Austin", "create_at": date("2015-11-17"), "description": "Running quarterhorses, Odessa, anti war 1966, History of Science, Universityof Texas, real estate, Austin, sequestering carbon inthe soil, where it belongs Yes", "followers_count": 37, "friends_count": 222, "statues_count": 229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pacifica, CA", "id": "a769d8f00b12878d", "name": "Pacifica", "place_type": "city", "bounding_box": rectangle("-122.515276,37.565624 -122.442863,37.666566") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 654806, "cityName": "Pacifica" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751253469069312, "text": "This the life that i live this the life that i chose #Gutta21 #RIP #Missyouhomie https://t.co/8aZFGt5NgW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Gutta21", "RIP", "Missyouhomie" }}, "user": { "id": 573865709, "name": "Sam LoBosco", "screen_name": "LoBoMusic__", "lang": "en", "location": "CHICAGO, IL", "create_at": date("2012-05-07"), "description": "Producer/Engineer from Chicago #SAEAlumni Contact: Lobo.music.productions@gmail.com The Alpha EP Coming Soon", "followers_count": 2494, "friends_count": 1143, "statues_count": 37592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomingdale, IL", "id": "a3c3c2f3573eacf7", "name": "Bloomingdale", "place_type": "city", "bounding_box": rectangle("-88.153241,41.931309 -88.049768,41.974694") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1706587, "cityName": "Bloomingdale" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751253762551808, "text": "2:00 o'clock eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2225935428, "name": "Emma Hagen", "screen_name": "emmahagen6", "lang": "en", "location": "null", "create_at": date("2013-12-01"), "description": "Rawr", "followers_count": 1306, "friends_count": 643, "statues_count": 2210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeville, MN", "id": "a8d13ffe83f33ce5", "name": "Lakeville", "place_type": "city", "bounding_box": rectangle("-93.3184,44.640924 -93.156249,44.717832") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2735180, "cityName": "Lakeville" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751254815309824, "text": "No I'm not Jordan but I am the goat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292551346, "name": "Leontae'", "screen_name": "JustTevon_", "lang": "en", "location": "La Marque, TX", "create_at": date("2011-05-03"), "description": "Just a nigga trying to chase the paper.", "followers_count": 1374, "friends_count": 1169, "statues_count": 30038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Marque, TX", "id": "62d6ba41ace74599", "name": "La Marque", "place_type": "city", "bounding_box": rectangle("-95.051744,29.348343 -94.946934,29.403387") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841116, "cityName": "La Marque" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751254899367936, "text": "Untitled 02", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170910381, "name": "lit like vic", "screen_name": "Tatianaa_Carter", "lang": "en", "location": "Huntsville, AL", "create_at": date("2010-07-25"), "description": "just milly rocking my way to the top ✨", "followers_count": 627, "friends_count": 518, "statues_count": 8094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751255020830720, "text": "@ry__yo stop tweeting", "in_reply_to_status": 706745932558110720, "in_reply_to_user": 37513497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37513497 }}, "user": { "id": 3310808634, "name": "dad machine ˚₊* ✩‧₊˚", "screen_name": "53npai", "lang": "en", "location": "null", "create_at": date("2015-08-09"), "description": "【=◈︿◈=】lian // snapchat: senpaidesu // video game characters aren't real but my feelings are", "followers_count": 121, "friends_count": 139, "statues_count": 1996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751255046000640, "text": "Damnnnn������ https://t.co/nMZYVynJCV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560622718, "name": "sabrinnnna™♊️", "screen_name": "sabrinal0525", "lang": "en", "location": "null", "create_at": date("2012-04-22"), "description": "•sa•bree•nuhh• ◡̈ bees kneeze", "followers_count": 244, "friends_count": 478, "statues_count": 3027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751255054450688, "text": "Goodnight to the pimps all across America ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2345861431, "name": "T R A Y ✨", "screen_name": "LilDarkskinnnnn", "lang": "en", "location": "Seattle", "create_at": date("2014-02-15"), "description": "17 ♑ Model ✨", "followers_count": 976, "friends_count": 811, "statues_count": 8213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751255180288000, "text": "ITS MY BIRTHDAY BITCHES!! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505250088, "name": "Mikel AnnMarie Clark", "screen_name": "Mikel_AnnMarie", "lang": "en", "location": "Your Moms House. ", "create_at": date("2012-02-26"), "description": "I can't feel anything anymore.", "followers_count": 304, "friends_count": 374, "statues_count": 9309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taft, CA", "id": "5296438a6967a6fe", "name": "Taft", "place_type": "city", "bounding_box": rectangle("-119.483186,35.122091 -119.430928,35.172321") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 677574, "cityName": "Taft" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751255943581696, "text": "I wish I had people to call but I be deleting everybody number ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3014021412, "name": "...®", "screen_name": "slim9_", "lang": "en", "location": "null", "create_at": date("2015-02-08"), "description": "null", "followers_count": 446, "friends_count": 242, "statues_count": 15389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-07T00:00:04.000Z"), "id": 706751256006664192, "text": "Book of Isaiah Chapter 6.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4006968231, "name": "Ace Boogie", "screen_name": "MannyLo079", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-10-20"), "description": "And I heard the voice of the Lord saying, “Whom shall I send, and who will go for us?” Then I said, “Here I am! Send me.”", "followers_count": 72, "friends_count": 237, "statues_count": 2589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carbondale, IL", "id": "20d0abc87d11501a", "name": "Carbondale", "place_type": "city", "bounding_box": rectangle("-89.294485,37.670875 -89.151934,37.764433") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17077, "countyName": "Jackson", "cityID": 1711163, "cityName": "Carbondale" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751256887480321, "text": "#SuspiciousIncident at 117 S Summerlin Ave. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3683092,28.5407939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousIncident", "orlpol", "opd" }}, "user": { "id": 39049373, "name": "Police Calls 32801", "screen_name": "orlpol32801", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 533, "friends_count": 1, "statues_count": 100923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751257227108352, "text": "The best part about messing something up is having the Curb Your Enthusiasm theme playing while I think about how much I screwed up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 610534725, "name": "Chris", "screen_name": "DeniedByQuick", "lang": "en", "location": "Los Angeles, California.", "create_at": date("2012-06-16"), "description": "Comic & film enthusiast. [Ravens. LA Kings. Football. (as in soccer) Dodgers. Galaxy. Lakers]", "followers_count": 2408, "friends_count": 1838, "statues_count": 100396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751257344512001, "text": "Damn I thought that was a cookie too. https://t.co/vi6KrsT3iC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2890092445, "name": "Sexy Flexy.", "screen_name": "thefadedreality", "lang": "en", "location": "Houston, TX", "create_at": date("2014-11-04"), "description": "Vous me manques encore? Who depends on another man's table often dines late. 214. #WWPD", "followers_count": 1482, "friends_count": 833, "statues_count": 54910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751257386455041, "text": "happy birthday babe�� https://t.co/5aZhWCMv6C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2292298424, "name": "Ⓟarker Ⓙames", "screen_name": "Mottram28", "lang": "en", "location": "Southern California", "create_at": date("2014-01-14"), "description": "yet its just the beginning.", "followers_count": 342, "friends_count": 256, "statues_count": 1257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751257692622848, "text": "Best friends finally 18�� ! Congrats bby here's some amazing memories of https://t.co/VYvOGhAXZS you. @sxummerkvrinv https://t.co/KjnsLZaPHU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1052076997 }}, "user": { "id": 2792653468, "name": "♡Alfa Perales♡", "screen_name": "heyitsalfaa", "lang": "en", "location": "شاهزاده خانم", "create_at": date("2014-09-29"), "description": "figure me out", "followers_count": 271, "friends_count": 206, "statues_count": 4794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751257759739905, "text": "@chevalier_jake @braddder u screenshot it. Its on you", "in_reply_to_status": 706750277903233024, "in_reply_to_user": 1692692744, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1692692744, 1554482605 }}, "user": { "id": 3310211550, "name": "Mr. Fantastic", "screen_name": "isaiah_jeong", "lang": "en", "location": "japanese poonami", "create_at": date("2015-08-08"), "description": "(R)ice", "followers_count": 133, "friends_count": 118, "statues_count": 1601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751258024144896, "text": "#UnknownTrouble at 4509 Crimson Ct. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.439269,28.593223"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "UnknownTrouble", "orlpol", "opd" }}, "user": { "id": 39047091, "name": "Police Calls 32808", "screen_name": "orlpol32808", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 219, "friends_count": 1, "statues_count": 79742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751258519068673, "text": "LMFAOOOOOOO I WOULD BE IN HEAVEN WEAK AF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36832622, "name": "Ginaaaa", "screen_name": "BehindHaSHADES_", "lang": "en", "location": "NYC", "create_at": date("2009-04-30"), "description": "20 . ✒️ Pushing Pens One Post At A Time. #TM Always Lurking ...", "followers_count": 1568, "friends_count": 873, "statues_count": 69329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751258812674048, "text": "Travelling to Greensboro or just twittering about Greensboro? https://t.co/WvIVW353X6 #Greensboro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.792,36.0726"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Greensboro" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1028, "friends_count": 311, "statues_count": 2636150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751259055939584, "text": "Wind 2.0 mph SSW. Barometer 30.195 in, Falling. Temperature 25.0 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751259160674304, "text": "@kiiingbravo47 you're the most annoying human being I've ever encountered", "in_reply_to_status": 706750313705791488, "in_reply_to_user": 614661501, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 614661501 }}, "user": { "id": 129335837, "name": "catfish", "screen_name": "cynnerrr", "lang": "en", "location": "La", "create_at": date("2010-04-03"), "description": "so bleachy // tattoos and psycho vision shirts // sc:cynflowers", "followers_count": 621, "friends_count": 281, "statues_count": 23359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751259257262080, "text": "#VerbalDisturbance at 3520-3651 Westland Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4766898,28.5885153"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VerbalDisturbance", "orlpol", "ocso" }}, "user": { "id": 39133730, "name": "Police Calls 32818", "screen_name": "orlpol32818", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 49, "friends_count": 1, "statues_count": 9086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504956,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751259739561984, "text": "@MisterMetokur @MittRomney Isn't Mitt already and embarrassment? I mean, he's a Mormon AND an establishment cuck. Lose/lose imo.", "in_reply_to_status": 706750835292655617, "in_reply_to_user": 3070114050, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3070114050, 50055701 }}, "user": { "id": 180293731, "name": "mrbraindead", "screen_name": "mrbraindead12", "lang": "en", "location": "null", "create_at": date("2010-08-19"), "description": "null", "followers_count": 491, "friends_count": 677, "statues_count": 6266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, SC", "id": "8eb7d0abedc4817b", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-82.434848,34.687331 -82.249689,34.904552") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4530850, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751259915608064, "text": "@lilyaniris definitely agree with you. Bernie strives for peace and unity. Hillary strives for war and violence", "in_reply_to_status": 706750920063778816, "in_reply_to_user": 217677021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 217677021 }}, "user": { "id": 1146946760, "name": "NeilLee", "screen_name": "CaptainNeil96", "lang": "en", "location": "CALIFORNIA", "create_at": date("2013-02-03"), "description": "4/26/96, and i love positivity.#uniteblue i officially #Feelthebern full-time student at Pasadena city college. #PCC4BERNIE", "followers_count": 1791, "friends_count": 3678, "statues_count": 48009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751260314230785, "text": "Can you find Greensboro on the map? Just try it at https://t.co/WvIVW353X6 #Greensboro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.792,36.0726"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Greensboro" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1028, "friends_count": 311, "statues_count": 2636151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751260561637378, "text": "#Trespasser at 5515 International Dr. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4548249,28.4627293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trespasser", "orlpol", "opd" }}, "user": { "id": 39050965, "name": "Police Calls 32819", "screen_name": "orlpol32819", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 289, "friends_count": 1, "statues_count": 71509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751260586692608, "text": "Miss gym and you so much :( I hope I get to see you soon and also hope I didn't let you guys down too much when I quit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1534490724, "name": "sierra", "screen_name": "sierrafiala", "lang": "en", "location": "null", "create_at": date("2013-06-20"), "description": "I don't go to Skyview", "followers_count": 2162, "friends_count": 781, "statues_count": 22656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salmon Creek, WA", "id": "01d1bc4e2f2710a8", "name": "Salmon Creek", "place_type": "city", "bounding_box": rectangle("-122.713017,45.692689 -122.620464,45.765171") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5361000, "cityName": "Salmon Creek" } }
+{ "create_at": datetime("2016-03-07T00:00:05.000Z"), "id": 706751260624556032, "text": "At every instance your doubted for your choices, if you're not secure enough it cause resentment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51554214, "name": "Vicki", "screen_name": "VeeEmEss", "lang": "en", "location": "Charlotte, NC", "create_at": date("2009-06-27"), "description": "Hampton '14 Syracuse '15 [Obnoxious Optimist]\nI'm only getting started. #JKIW #JWOI #WorkInProgress", "followers_count": 595, "friends_count": 512, "statues_count": 60584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751261257945088, "text": "Chris over here sleeping good while I'm mad at him and shit .. I should slap him ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3163055821, "name": "♎️", "screen_name": "faithwright65", "lang": "en", "location": "Kerrville, TX", "create_at": date("2015-04-18"), "description": "Got FaiTh? |", "followers_count": 535, "friends_count": 504, "statues_count": 6767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kerrville, TX", "id": "9e1bc6eba1d7d97c", "name": "Kerrville", "place_type": "city", "bounding_box": rectangle("-99.22603,29.983238 -99.098045,30.094147") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48265, "countyName": "Kerr", "cityID": 4839040, "cityName": "Kerrville" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751261375389696, "text": "Somehow by loving myself, I made you love me more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340758809, "name": "p.a", "screen_name": "pageari", "lang": "en", "location": "The Magic City", "create_at": date("2011-07-22"), "description": "snap me picari", "followers_count": 208, "friends_count": 163, "statues_count": 17088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall, FL", "id": "9b46dccb3cfb880c", "name": "Kendall", "place_type": "city", "bounding_box": rectangle("-80.389344,25.628844 -80.304896,25.715128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236100, "cityName": "Kendall" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751261417148416, "text": "Kan ku jadikan kau kenangan yang terindah dalam hidupku ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2271168068, "name": "shazwan", "screen_name": "shzwn_azr", "lang": "en", "location": "Colorado, USA", "create_at": date("2013-12-31"), "description": "'A'", "followers_count": 210, "friends_count": 179, "statues_count": 6122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751261660487680, "text": "取り込み中のようでしたので", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-140.0766,56.8037"), "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 1148156568, "name": "アルフィン", "screen_name": "AL_FiN_07839216", "lang": "ja", "location": "飛空巡航艦FS-317", "create_at": date("2013-02-04"), "description": "こちらは『シャイニング・フォース フェザー』に登場するコアユニット、アルフィンbotです。\n反応ワードなどはURLから。非公式全自動botの不具合・反応の追加などDMでお知らせ頂ければ幸いです。Shining Force Feather SEGA®", "followers_count": 2152, "friends_count": 2270, "statues_count": 37009 }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751262381834240, "text": "why you gotta act shady baby i wanted to be your friend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 216615458, "name": "gabriel €.", "screen_name": "lilbuzzo_", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-11-16"), "description": "♏️ - hate me all you want its bad for your health - ☮", "followers_count": 508, "friends_count": 63, "statues_count": 21501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winter Gardens, CA", "id": "0164a99dd4ce91a4", "name": "Winter Gardens", "place_type": "city", "bounding_box": rectangle("-116.946888,32.819949 -116.903401,32.857262") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 685992, "cityName": "Winter Gardens" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751262734344194, "text": "@L_MarysEyebrows Stopped myself from paraphrasing WWI quote: \"The lamps are going out all over Downton....\"", "in_reply_to_status": 706750402902016000, "in_reply_to_user": 440567869, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 440567869 }}, "user": { "id": 762237372, "name": "Downton Abbey Books", "screen_name": "DowntonBooks", "lang": "en", "location": "NYC", "create_at": date("2012-08-16"), "description": "Mad Anglophile/@StMartinsPress editor for brand-new official books The Wit & Wisdom of Downton Abbey and Downton Abbey: A Celebration by @JessicaFellowes", "followers_count": 4142, "friends_count": 813, "statues_count": 12375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751262998470656, "text": "@EuroDude92 sad life ����", "in_reply_to_status": 706750983905214464, "in_reply_to_user": 485132848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 485132848 }}, "user": { "id": 1353013356, "name": "⚜ Elvedina ⚜", "screen_name": "deeeena_xo", "lang": "en", "location": "Gresham, OR", "create_at": date("2013-04-14"), "description": "Irnes Paco ❤️ @amelakazic is my bff #Blessup", "followers_count": 1735, "friends_count": 886, "statues_count": 29177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751263409639424, "text": "She's getting sooo big. #longpuppy But she'll always be my baby girl. #cavpuppy… https://t.co/pdhEohWqCB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.827,47.6397"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "longpuppy", "cavpuppy" }}, "user": { "id": 16521177, "name": "ThisChelsea", "screen_name": "reeder_chelsea", "lang": "en", "location": "CWU ", "create_at": date("2008-09-29"), "description": "Official page of Chelsea Reeder", "followers_count": 49, "friends_count": 231, "statues_count": 501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5362120, "cityName": "Seabeck" } }
+{ "create_at": datetime("2016-03-07T00:00:06.000Z"), "id": 706751263535296512, "text": "Temp: 43.9°F - Dew Point: 35.1° - Wind: 12.8 mph - Gust: 21.5 - Rain Today: 0.00in. - Pressure: 29.80in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113623680737281, "text": "Every time I think of something random enough for Twitter, I save it in my drafts, but gah lee I can't believe half the shit I think of ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1643643666, "name": "ash", "screen_name": "ash2fab", "lang": "en", "location": "NM", "create_at": date("2013-08-03"), "description": "null", "followers_count": 1746, "friends_count": 288, "statues_count": 19529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113624209215488, "text": "It's heart medicine, she has been taking since 06 and it wasn't banned until Jan 2016. No chill. https://t.co/OCyCBxLVdT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24799301, "name": "9thIslandMoke", "screen_name": "kylemontoya", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "null", "followers_count": 135, "friends_count": 93, "statues_count": 80 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113624213434369, "text": "I'm rushing home because my curfew 2:30 ☹☹☹☹☹ and I'm all the way in North Dallas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236108418, "name": "Champagne Barbie ॐ", "screen_name": "BrichelleBeBomb", "lang": "en", "location": "Dallas, TX ", "create_at": date("2011-01-09"), "description": "EROTIC ANGEL; MELANIN POPPING FEMINIST (gender equality) & art major at #TJC17 #RipVell DTX ॐ IG & Snapchat @brichellebebomb", "followers_count": 2413, "friends_count": 1293, "statues_count": 57049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113624351809537, "text": "Aligning yourself by acknowledging the #TrumpHateMachine is tantamount to political suicide especially given the bright future @marcorubio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TrumpHateMachine" }}, "user_mentions": {{ 15745368 }}, "user": { "id": 509800941, "name": "Misbah Siddique", "screen_name": "professormisbah", "lang": "en", "location": "null", "create_at": date("2012-02-29"), "description": "null", "followers_count": 82, "friends_count": 157, "statues_count": 934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113625517826050, "text": "@erhsryan @TheBravesGuy I'm off at 12", "in_reply_to_status": 707112000560562176, "in_reply_to_user": 1210771070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1210771070, 355381658 }}, "user": { "id": 477941855, "name": "ELMER FUDD™", "screen_name": "ClassKing13", "lang": "en", "location": "Snow Bunny City", "create_at": date("2012-01-29"), "description": "In search of the perfect snow bunny. I am the human gif button.", "followers_count": 1298, "friends_count": 509, "statues_count": 65260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113625580756992, "text": "Wind 7.0 mph SW. Barometer 1015.14 mb, Rising Rapidly. Temperature 56.1 °F. Rain today 0.32 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 126, "statues_count": 12964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113625585115136, "text": "At the end of the day, no matter what shitty things have happened, you are lucky enough to be alive. Don't take this life for granted.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 777163250, "name": "jake", "screen_name": "JakeShebish", "lang": "en", "location": "Chicago", "create_at": date("2012-08-23"), "description": "be happy. #EDC2016 #Lolla", "followers_count": 598, "friends_count": 993, "statues_count": 11427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113626382049280, "text": "@BetteMidler I'm ur doppelgänger. Was getting my hair done at salon & sone guy said I look like u ❤️❤️❤️����", "in_reply_to_status": -1, "in_reply_to_user": 139823781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139823781 }}, "user": { "id": 4256149305, "name": "Nancy Lasocki", "screen_name": "nancylasocki", "lang": "en", "location": "Illinois, USA", "create_at": date("2015-11-23"), "description": "Wife,Mom,Grandma & business owner/Pest Control. Simple things make me happy! Animal lover, Metal detecting,Casino lover", "followers_count": 642, "friends_count": 1091, "statues_count": 7429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channahon, IL", "id": "9ae1ad28e07154d9", "name": "Channahon", "place_type": "city", "bounding_box": rectangle("-88.25119,41.377462 -88.13539,41.492478") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1712476, "cityName": "Channahon" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113626436505600, "text": "Up all night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3054198217, "name": "Tune.☁️", "screen_name": "ImTune_YouJig", "lang": "en", "location": "Conway, AR", "create_at": date("2015-03-01"), "description": "What's the problem? Why you not smiling, we're in the prime of our life.. Snapchat: Montorey_j #UCA", "followers_count": 1421, "friends_count": 1088, "statues_count": 6517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113626541228032, "text": "Yes, Bernie! https://t.co/56hd5kb4BX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21176973, "name": "Lauren Hayes", "screen_name": "laurenahayes", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-02-17"), "description": "reinventing the vitamin @ritualvitamins. model @industry_models. previously @scopely & @startenginela. IG: @laurenahayes", "followers_count": 3490, "friends_count": 1829, "statues_count": 17599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:00.000Z"), "id": 707113626700656640, "text": "@dodgerjazzcougs he was being honest.", "in_reply_to_status": 707109467259691008, "in_reply_to_user": 58609941, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 58609941 }}, "user": { "id": 145814126, "name": "Jeremiah Jensen", "screen_name": "JJSportsBeat", "lang": "en", "location": "Salt Lake City", "create_at": date("2010-05-19"), "description": "KSL Sports Anchor from Star Valley, Wyoming.", "followers_count": 8617, "friends_count": 644, "statues_count": 15666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113627581480960, "text": "lol @ the song", "in_reply_to_status": 706981395319619585, "in_reply_to_user": 401661510, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 401661510, "name": "bambi ♡", "screen_name": "izy_espinoza", "lang": "en", "location": "Stephenville, TX", "create_at": date("2011-10-30"), "description": "tarleton state • delta phi epsilon • I'm a princess/sasshole ≫it'll buff≪", "followers_count": 912, "friends_count": 382, "statues_count": 27953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granbury, TX", "id": "013401bd2ad7a235", "name": "Granbury", "place_type": "city", "bounding_box": rectangle("-97.822813,32.39284 -97.698331,32.506334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48221, "countyName": "Hood", "cityID": 4830416, "cityName": "Granbury" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113628424544256, "text": "@rrrosie__ ��", "in_reply_to_status": 707113352112115712, "in_reply_to_user": 2686716438, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2686716438 }}, "user": { "id": 4160637198, "name": "Tony", "screen_name": "tony_vmaya", "lang": "en", "location": "Long Beach, CA", "create_at": date("2015-11-07"), "description": "I'm trusted by Young Metro .", "followers_count": 95, "friends_count": 77, "statues_count": 1675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113628504162308, "text": "#BatmanvSuperman Who will win? #LexLuthor https://t.co/4gSZa5HxzV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BatmanvSuperman", "LexLuthor" }}, "user": { "id": 2857978582, "name": "DCEU Enterprises", "screen_name": "dcmarvel88", "lang": "en", "location": "Chandler, AZ", "create_at": date("2014-11-02"), "description": "We just have a bad history with freaks dressed like clowns.", "followers_count": 1035, "friends_count": 557, "statues_count": 16080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113628533530624, "text": "I wouldn't be upset if @KimKardashian and @kanyewest both just got absolutely rocked with a Mike Tyson right hook in his prime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25365536, 169686021 }}, "user": { "id": 435397155, "name": "AJ Fredrickson", "screen_name": "AJ_FREDRICKSON", "lang": "en", "location": "Fridley, MN/SCSU 19'", "create_at": date("2011-12-12"), "description": "D1 skate sharpener, inventor of the Whip Pitch, 2nd best underwater basket weaver in the state #GETIII #ScottFliesForever", "followers_count": 641, "friends_count": 591, "statues_count": 9858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fridley, MN", "id": "612139aad8c98456", "name": "Fridley", "place_type": "city", "bounding_box": rectangle("-93.290637,45.035336 -93.227152,45.125115") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2722814, "cityName": "Fridley" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113628571471872, "text": "03/08@03:00 - Temp 44.7F, WC 44.7F. Wind 0.0mph SW, Gust 1.0mph. Bar 30.121in, Falling slowly. Rain 0.00in. Hum 83%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113628625997824, "text": "Ripley SW Limestone Co. Temp: 55.0°F Wind:6.0mph Pressure: 1000.5mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 49959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113629041033216, "text": "03:00 61.5°F Feels:61.5°F (Hi61.7°F/Lo59.7°F) Hum:42% Wnd:WSW 5.0MPH Baro:29.78in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 190, "friends_count": 262, "statues_count": 26630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113629628272640, "text": "Waistin all of this time but I don't mind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2829884417, "name": "Megan Brooking", "screen_name": "megan_brooking", "lang": "en", "location": "null", "create_at": date("2014-10-14"), "description": "snap ' Meganbrookingg :)))", "followers_count": 566, "friends_count": 860, "statues_count": 6950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113629653602304, "text": "@pau_masson lmao you'll have to excuse me I'm lit", "in_reply_to_status": 707113427945308161, "in_reply_to_user": 548836816, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 548836816 }}, "user": { "id": 1328167862, "name": "Madeline", "screen_name": "madfullerxo", "lang": "en", "location": "508|912", "create_at": date("2013-04-04"), "description": "ga southern'19 #hailsouthern #freebandz", "followers_count": 762, "friends_count": 483, "statues_count": 10138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113629695537152, "text": "Wind 7.0 mph SE. Barometer 29.879 in, Steady. Temperature 64.6 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113630291140608, "text": "Temp: 59.6°F Wind:1.9mph Pressure: 30.111hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113630836400128, "text": "I miss cuddling braaaah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24125771, "name": "lor dr. love", "screen_name": "_freakydre", "lang": "en", "location": "The 410", "create_at": date("2009-03-12"), "description": "towson. photographer. writer for the @thedemotape_. filmmaker kinda. #HeavyBoys SC: juicelessdre", "followers_count": 2374, "friends_count": 1671, "statues_count": 167161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113631037566980, "text": "@ebbtideapp Tide in North Newport River, Georgia 03/08/2016\nHigh 7:54am 8.4\n Low 2:15pm -1.2\nHigh 8:09pm 8.4\n Low 2:35am -1.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.2,31.6833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 40, "friends_count": 1, "statues_count": 13995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13191, "countyName": "McIntosh" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113631146618881, "text": "@lingaya あ、そうですか。一回全タブ閉じてはみたんですが、。。。まあ、IEでアクセスできるから特に困っていないので、そのままにしています。数日前まではなんら問題無かったんですが。", "in_reply_to_status": 707112719271337985, "in_reply_to_user": 119672311, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 119672311 }}, "user": { "id": 547153442, "name": "BlogFromAmerica", "screen_name": "BlogFromAmerica", "lang": "en", "location": "Colorado, USA", "create_at": date("2012-04-06"), "description": "アメリカより (携帯・モバイル&コロラド・デンバーブログ) http://BlogFromAmerica.comの管理人です。 Please, I only tweet in Japanese, and my followers are almost all Japanese.", "followers_count": 1529, "friends_count": 8, "statues_count": 7096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-03-08T00:00:01.000Z"), "id": 707113631331262464, "text": "#SupportOriginMelissa 57.4°F Wind:2.2mph Pressure: 30.09hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113631868198913, "text": "I'm always going to appreciate Dillon.\nI discovered him back in 2010, and he was my first show back in 2013.\nSeen him 7 times & not done", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3014162335, "name": "chacón", "screen_name": "machacon__", "lang": "en", "location": "Riverside, CA", "create_at": date("2015-02-09"), "description": "19. Groovy tunes. RCC. #YETHSQUAD E.R", "followers_count": 691, "friends_count": 706, "statues_count": 37727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jurupa Valley, CA", "id": "015b658472edd3dc", "name": "Jurupa Valley", "place_type": "city", "bounding_box": rectangle("-117.551185,33.955517 -117.410795,34.034306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 637692, "cityName": "Jurupa Valley" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113632400695296, "text": "65.4F (Feels: 65.4F) - Humidity: 93% - Wind: 9.8mph NE - Gust: 10.7mph - Pressure: 1015.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 229788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113632509853696, "text": "Wind 0.0 mph WSW. Barometer 30.090 in, Rising slowly. Temperature 53.4 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113632618786816, "text": "https://t.co/XCjAzpkfqe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 244338620, "name": "イエスのオスナ", "screen_name": "JezuzOzuna", "lang": "en", "location": "CALIFORNIA", "create_at": date("2011-01-28"), "description": "RELAX 1991 CA", "followers_count": 491, "friends_count": 336, "statues_count": 57601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113632669278208, "text": "���� https://t.co/oXj3nccICp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 511165463, "name": "Swaggy P", "screen_name": "NateFromPeak", "lang": "en", "location": "null", "create_at": date("2012-03-01"), "description": "20. Sophomore guard at Rockford University. #815", "followers_count": 1546, "friends_count": 563, "statues_count": 85169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113633180950530, "text": "@TamiahRomae imym :'(", "in_reply_to_status": 707113557880471552, "in_reply_to_user": 602342194, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user_mentions": {{ 602342194 }}, "user": { "id": 539740304, "name": "Barlow☁️", "screen_name": "Carlos_Pwnzz", "lang": "en", "location": "In The Clouds, CO", "create_at": date("2012-03-28"), "description": "| Just a young nigga trying to be happy | VR Kings | Cars. Culture. Lifestyle. | R.I.P. Megan Pacheco❤️ |", "followers_count": 1327, "friends_count": 862, "statues_count": 89433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cimarron Hills, CO", "id": "6ba10b142c887fdb", "name": "Cimarron Hills", "place_type": "city", "bounding_box": rectangle("-104.720989,38.831707 -104.681102,38.900855") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 814587, "cityName": "Cimarron Hills" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113634305064960, "text": "How the fuck do I sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2607708796, "name": "Yendis", "screen_name": "sidpullsgbs", "lang": "en", "location": "Charlotte, NC", "create_at": date("2014-06-13"), "description": "Hi http://daisiesandrainb0ws.tumblr.com 9/12/14❤️", "followers_count": 292, "friends_count": 274, "statues_count": 16794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113634573340677, "text": "@Julian_hol put your grandmas glasses away", "in_reply_to_status": 707113245555843073, "in_reply_to_user": 1003310798, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1003310798 }}, "user": { "id": 3228532838, "name": "bren", "screen_name": "brenditaibarraa", "lang": "en", "location": "Oxnard, CA", "create_at": date("2015-05-27"), "description": "si se puede", "followers_count": 469, "friends_count": 423, "statues_count": 3702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113634858569728, "text": "KIM LITERALLY DRAGGING EVERYONE SAVAGE AF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333574037, "name": "ㅤ ㅤ", "screen_name": "biebschaneI", "lang": "en", "location": "LA", "create_at": date("2011-07-11"), "description": "ig: mccanns.queen ✨", "followers_count": 51616, "friends_count": 9504, "statues_count": 14720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113634917289984, "text": "China is going crazy over this cute little mermaid https://t.co/EDKWKRhgOM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17560, "friends_count": 17560, "statues_count": 68967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635164901376, "text": "Wind 0.7 mph N. Barometer 30.12 in, Rising slowly. Temperature 39.6 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 115, "statues_count": 158081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635194122241, "text": "@cloternamian L'ecole est pas le feu... C'est trop ennuyeux..", "in_reply_to_status": 707113195505328128, "in_reply_to_user": 3067333915, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 3067333915 }}, "user": { "id": 500551782, "name": "Se Min Park", "screen_name": "ItsTGTC6", "lang": "en", "location": "The Peak", "create_at": date("2012-02-22"), "description": "Know your worth and stay relevant", "followers_count": 272, "friends_count": 115, "statues_count": 7199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635336867840, "text": "Don't mind my face but it feels damn good to start lifting again and transforming my body. Been… https://t.co/jXvVMbl2DC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.9296799,46.5359497"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 763194865, "name": "Brian Reynolds", "screen_name": "BrianRR33", "lang": "en", "location": "morton wa", "create_at": date("2012-08-17"), "description": "Never know what's ahead of you ✌️Snap:Reynolds33", "followers_count": 923, "friends_count": 694, "statues_count": 8388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53041, "countyName": "Lewis" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635500433408, "text": "I got home from work late and I have an early morning ahead of me but I can't go to sleep because the new into it over it record is so fucki", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3877824857, "name": "Matt", "screen_name": "slee_patterns", "lang": "en", "location": "Greensboro, NC", "create_at": date("2015-10-05"), "description": "legalize RANCH", "followers_count": 94, "friends_count": 66, "statues_count": 215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635697397760, "text": "Stfu �� https://t.co/PEdGVaIA31", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1245511158, "name": "Glen Coco", "screen_name": "MeganMoore__", "lang": "en", "location": "senior yr baby;;;))) GO LAMBS", "create_at": date("2013-03-05"), "description": "SJGC 4evr", "followers_count": 986, "friends_count": 503, "statues_count": 39454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-03-08T00:00:02.000Z"), "id": 707113635722596352, "text": "In case no ones told you lately.. You are one of the realist & down to mars girl ever. #�� https://t.co/GeHsUzCNbM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150001030, "name": "Suavè", "screen_name": "Why_Meek", "lang": "en", "location": "null", "create_at": date("2010-05-30"), "description": "The bees knees, the cats pajamas, & cooler than the other side of the pillow.", "followers_count": 558, "friends_count": 524, "statues_count": 13276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113636347715585, "text": "Jennifer Thompson bringing it! #losangelespoetsociety #womentakeover #killradio #fullspectrum… https://t.co/I3FkgUCRd2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2920685,34.0761299"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "losangelespoetsociety", "womentakeover", "killradio", "fullspectrum" }}, "user": { "id": 2916107696, "name": "LA Poet Society", "screen_name": "LosAngelesPoetS", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-12-01"), "description": "We are the bridge to your creative communities in Los Angeles. Read more about our mission here: http://www.lapoetsociety.org/#!about/c24vq", "followers_count": 248, "friends_count": 541, "statues_count": 552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113636771266560, "text": "Bro I can't fall back asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2293819004, "name": "Joakim Noah✨", "screen_name": "AndreaMacias60", "lang": "en", "location": "null", "create_at": date("2014-01-15"), "description": "Ball don't lie - Joakim Noah", "followers_count": 157, "friends_count": 127, "statues_count": 9034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melrose Park, IL", "id": "b23478d3162b126e", "name": "Melrose Park", "place_type": "city", "bounding_box": rectangle("-87.903492,41.888952 -87.824303,41.919399") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1748242, "cityName": "Melrose Park" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113636838318080, "text": "@iamblackbear facemask right now on snapchat I love it so much kinda reminds me of the joker because of the smile haha", "in_reply_to_status": -1, "in_reply_to_user": 17364412, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17364412 }}, "user": { "id": 2286061783, "name": "G.T.", "screen_name": "gaaaaabbbbyy", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2014-01-10"), "description": "19.Blackhawks.7/11/15.\nBroken boy meets broken girl", "followers_count": 2901, "friends_count": 3388, "statues_count": 25735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113636846645248, "text": "I need to make time for check up. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351762679, "name": "B.", "screen_name": "byngxx_", "lang": "en", "location": "21", "create_at": date("2011-08-09"), "description": "contented | hopeless romantic pero scared", "followers_count": 207, "friends_count": 279, "statues_count": 5776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113636905365504, "text": "@UNCLE__DRUW @DestMonteon if I'm alive tomorrow, you two will have a role in the movie", "in_reply_to_status": 707113458836221952, "in_reply_to_user": 1633899978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1633899978, 2366865061 }}, "user": { "id": 124540088, "name": "snavs", "screen_name": "real_nava", "lang": "en", "location": "null", "create_at": date("2010-03-19"), "description": "18 | Rosemead | STAR$ | We dont brag. We flex | @Marilynnbay", "followers_count": 988, "friends_count": 805, "statues_count": 16877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113637694087168, "text": "Omg�������������� https://t.co/vm9d22VuEw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 157859298, "name": "☀️", "screen_name": "DevenMarie456", "lang": "en", "location": "Minnesota", "create_at": date("2010-06-20"), "description": "21. i love whosah. 9/14/12. 2/8/16", "followers_count": 974, "friends_count": 254, "statues_count": 64809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkins, MN", "id": "2d5b67122572d42f", "name": "Hopkins", "place_type": "city", "bounding_box": rectangle("-93.429448,44.905986 -93.370824,44.949276") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2730140, "cityName": "Hopkins" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113637807333376, "text": "Wind 0 mph ---. Barometer 30.09 in, Rising. Temperature 52.5 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 57, "friends_count": 73, "statues_count": 26256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113638084136960, "text": "@RemingtonWRX I wish I could retweet this 1 million times. He is slept on hard", "in_reply_to_status": 707112751475376129, "in_reply_to_user": 1485674059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1485674059 }}, "user": { "id": 360212392, "name": "Catchphrase Jones", "screen_name": "TheShamster3000", "lang": "en", "location": "Runnin' Thru The 6 ", "create_at": date("2011-08-22"), "description": "No Limit, But I Get My Roll On- Don't Get Lost In The Sauce- ΣΧ at TTU -sc- jadonrichmond3", "followers_count": 1058, "friends_count": 1110, "statues_count": 8387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cookeville, TN", "id": "3448f4f82d2187bc", "name": "Cookeville", "place_type": "city", "bounding_box": rectangle("-85.601741,36.046986 -85.418673,36.232069") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47141, "countyName": "Putnam", "cityID": 4716920, "cityName": "Cookeville" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113638188982272, "text": "strawberry applesauce >>>>> https://t.co/lkLsqpe48R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2242174132, "name": "cleopatra ✨", "screen_name": "aurorakween", "lang": "en", "location": "working at the pyramid tonight", "create_at": date("2013-12-24"), "description": "trynna steal dat dick, wyd? | 20 | feeling good, living better ❤️", "followers_count": 2013, "friends_count": 688, "statues_count": 284389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113638411182080, "text": "So let me get this straight...yall LOVE the snap chat puppy filter but hate being called a bitch...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338217299, "name": ".", "screen_name": "karen_dot_com", "lang": "en", "location": "somewhere between TX and MX", "create_at": date("2011-07-19"), "description": "null", "followers_count": 677, "friends_count": 610, "statues_count": 29202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Braunfels, TX", "id": "c807df9bb77ea0b3", "name": "New Braunfels", "place_type": "city", "bounding_box": rectangle("-98.214955,29.64544 -98.031544,29.765788") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48091, "countyName": "Comal", "cityID": 4850820, "cityName": "New Braunfels" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113639111581696, "text": "@CakesGod don't give in bro", "in_reply_to_status": 707113545998020608, "in_reply_to_user": 296592184, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296592184 }}, "user": { "id": 344851171, "name": "Ev ⚜", "screen_name": "JustDoItEverett", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-07-29"), "description": "Let Go And Let God || 6'5 || IG: JustDoItEv || SC: JustDoItEvv", "followers_count": 1906, "friends_count": 480, "statues_count": 105622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenn Heights, TX", "id": "63634cb1acb5eff7", "name": "Glenn Heights", "place_type": "city", "bounding_box": rectangle("-96.891055,32.518192 -96.821826,32.568093") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829840, "cityName": "Glenn Heights" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113639430463489, "text": "@jessica_zitka thanks baby", "in_reply_to_status": 707110073756217344, "in_reply_to_user": 4461071477, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4461071477 }}, "user": { "id": 478816351, "name": "J", "screen_name": "joemfing_thomas", "lang": "en", "location": "null", "create_at": date("2012-01-30"), "description": "18.", "followers_count": 1311, "friends_count": 1130, "statues_count": 74073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, MI", "id": "7b082d03ee1b544d", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-82.968959,42.479787 -82.902872,42.539749") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2669800, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-03-08T00:00:03.000Z"), "id": 707113639803654145, "text": "I'm not gon judge you off your past cause first of all that ain't right secondly I wasn't there so it dnt even matter.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231997207, "name": "SxFx", "screen_name": "cravebuttvh", "lang": "en", "location": "balifornia", "create_at": date("2010-12-29"), "description": "forgive me", "followers_count": 2725, "friends_count": 1663, "statues_count": 129542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113640608989185, "text": "It means they look at everyone (big board) and then find their right fit. https://t.co/KVz0dHIuJV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407329611, "name": "Blake Allen Murphy", "screen_name": "blakemurphy7", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2011-11-07"), "description": "ASU Marketing Graduate. #BirdGang. Former Arizona Rattlers Scouting Intern. I talk nothing but #sports from this account. Lover of Mac & Cheese with Ketchup", "followers_count": 1126, "friends_count": 2022, "statues_count": 81031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113640898531328, "text": "I'm fucked ip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90284195, "name": "Irving", "screen_name": "Irvingmadden", "lang": "en", "location": "Where the Lions be", "create_at": date("2009-11-15"), "description": "#⃣FTF ,Psalms 46:1 Grind when nobody is looking!", "followers_count": 846, "friends_count": 409, "statues_count": 27962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113641166770177, "text": "@__jrdg appteciate it man, I'm laser focused these days.", "in_reply_to_status": 707113361213763584, "in_reply_to_user": 197707145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 197707145 }}, "user": { "id": 381169723, "name": "Samurai Del", "screen_name": "samuraidel", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-09-27"), "description": "Producer | Founder of Northern Natives | Booking/Inquiries delfonicbeats@gmail.com", "followers_count": 1043, "friends_count": 475, "statues_count": 36975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113641582039040, "text": "@theejackieg if I made YouTube makeup tutorials lol jk https://t.co/PJZkXDSI5U", "in_reply_to_status": 706955380434214912, "in_reply_to_user": 53826431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53826431 }}, "user": { "id": 53826431, "name": "Jackie G", "screen_name": "theejackieg", "lang": "en", "location": "Salinas ✈Los Angeles", "create_at": date("2009-07-04"), "description": "21 and chillin'.", "followers_count": 401, "friends_count": 282, "statues_count": 18082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113641582075906, "text": "@EwItsLexi ugh...I am so fucking jealous :( I guess when I go to state I can ask for advice along with you, if you go ��", "in_reply_to_status": 707113434584846336, "in_reply_to_user": 96027172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 96027172 }}, "user": { "id": 1370993904, "name": "Austin Jones", "screen_name": "AustinBonees4L", "lang": "en", "location": "샌디애고", "create_at": date("2013-04-21"), "description": "null", "followers_count": 369, "friends_count": 232, "statues_count": 35586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113641993089024, "text": "@mehornybot ������wtf is your account", "in_reply_to_status": -1, "in_reply_to_user": 707043091874643968, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 707043091874643968 }}, "user": { "id": 2423822327, "name": "Yannah", "screen_name": "XOyannahOX", "lang": "en", "location": "Houston, TX", "create_at": date("2014-03-19"), "description": "Muke clemmings \nLarry stylinson \nLiam... motherfucking.... Payne \nOKKAY now you know my whole life story...", "followers_count": 57, "friends_count": 157, "statues_count": 644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conroe, TX", "id": "361610d5aa1f6719", "name": "Conroe", "place_type": "city", "bounding_box": rectangle("-95.679001,30.23105 -95.397267,30.443978") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4816432, "cityName": "Conroe" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113641997369344, "text": "@BlackBerry where is the service center in new York, my phone is under warranty and Its not recognising any sim, plz help", "in_reply_to_status": -1, "in_reply_to_user": 14580438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14580438 }}, "user": { "id": 181695653, "name": "Theja Airala", "screen_name": "princetheja", "lang": "en", "location": "Boston ,USA", "create_at": date("2010-08-22"), "description": "#Dermatologist #Cricketer #TelevisionNetwork #HarvardUniv #PartyLife #thuglife", "followers_count": 79, "friends_count": 304, "statues_count": 255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113642618060800, "text": "I'm with you. �� https://t.co/jipmrADKMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2763236836, "name": "Minaa Alvarado", "screen_name": "minaa_alvarado", "lang": "en", "location": "somewhere in tx", "create_at": date("2014-09-06"), "description": "null", "followers_count": 221, "friends_count": 75, "statues_count": 3309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113642626449408, "text": "Why is it so hard to KO before midnight man.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 126157084, "name": "メロディー", "screen_name": "infinitelyjoy_", "lang": "en", "location": "Bay Area, CA", "create_at": date("2010-03-24"), "description": "Constantly caffeinated and endlessly petty. But also a recklessly adventurous college kid.", "followers_count": 174, "friends_count": 154, "statues_count": 19155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daly City, CA", "id": "6a71821001635bbd", "name": "Daly City", "place_type": "city", "bounding_box": rectangle("-122.500164,37.649122 -122.405233,37.708437") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 617918, "cityName": "Daly City" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113642903216128, "text": "@MoralesXavior check your porch", "in_reply_to_status": -1, "in_reply_to_user": 1498769509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1498769509 }}, "user": { "id": 2161978332, "name": "CL", "screen_name": "CielleHaynes", "lang": "en", "location": "null", "create_at": date("2013-10-28"), "description": "null", "followers_count": 475, "friends_count": 400, "statues_count": 14208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113643201138688, "text": "✌️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 470796902, "name": "AnAddiction☘", "screen_name": "Heyverde", "lang": "en", "location": "✈️ Through the ⭐️", "create_at": date("2012-01-21"), "description": "• Ver-day ☘ , 19 ⚗⛽️ |Romans 8:18 ⛪️ ❤️ #FayState19 #CSM ✍ #Single STEM #LimitedLoyalty ✂️❤️ # LoveisLove #ShortHair❣#FeVerde ✉️⌨❣ Ig & SC :// HeyVerde ☘", "followers_count": 2723, "friends_count": 2335, "statues_count": 37384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, NC", "id": "4548df3d99bb4891", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-79.195342,34.936245 -78.808883,35.173009") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3722920, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113643393941505, "text": "Wind 0.0 mph ---. Barometer 30.040 in, Steady. Temperature 51.0 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 385, "friends_count": 292, "statues_count": 8841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113643654160384, "text": "#CommercialAlarm at 610 N Orange Ave. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3794877,28.5512677"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "CommercialAlarm", "orlpol", "opd" }}, "user": { "id": 39049373, "name": "Police Calls 32801", "screen_name": "orlpol32801", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 533, "friends_count": 1, "statues_count": 100969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113643721170944, "text": "I really used to have no morals on Twitter ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299942114, "name": "Lul Tay⚡️", "screen_name": "KoochieKinte", "lang": "en", "location": "H O U $ T O N ✈ Tyler, TX ", "create_at": date("2011-05-16"), "description": "Why follow your dreams when you can follow me?", "followers_count": 2084, "friends_count": 776, "statues_count": 33784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113644115398656, "text": "I really enjoy cropping people's faces and putting them on other pictures. I die of laughter at each one of them. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2293337360, "name": "Sylvia", "screen_name": "princesssayaa", "lang": "en", "location": "Denver", "create_at": date("2014-01-15"), "description": "Yo Yo Yiggity Yo!", "followers_count": 43, "friends_count": 95, "statues_count": 153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CO", "id": "3f871475c095f94f", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-105.148925,39.819133 -104.987771,39.97944") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 883835, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-03-08T00:00:04.000Z"), "id": 707113644136345600, "text": "temperature up 56°F -> 59°F\nhumidity down 47% -> 41%\nwind 6mph -> 10mph\npressure 30.02in falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.53424,39.16659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128069267, "name": "Bloomington Weather", "screen_name": "Bloomington", "lang": "en", "location": "Bloomington, IN", "create_at": date("2010-03-30"), "description": "Weather updates, forecast, warnings and information for Bloomington, IN. Sources: Yahoo! Weather, NOAA, USGS.", "followers_count": 275, "friends_count": 3, "statues_count": 21586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18105, "countyName": "Monroe", "cityID": 1805860, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113645004554240, "text": "Need to stop looking for excuses ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1582021561, "name": "damaris", "screen_name": "m0onchild_", "lang": "en", "location": "Porterville, CA", "create_at": date("2013-07-09"), "description": "°19 °adventurous✈°Cali living\n°College", "followers_count": 1131, "friends_count": 1117, "statues_count": 10200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113645528915968, "text": "Wife came home #BadSexIn3Words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BadSexIn3Words" }}, "user": { "id": 346340843, "name": "AntonioAngelo21", "screen_name": "AntonioAngelo21", "lang": "en", "location": "corner stall in an airport", "create_at": date("2011-07-31"), "description": "A serial flirt who really loves you and his wife and probably someone else. not safe for anywhere decent. please consider in lieu of resume", "followers_count": 2521, "friends_count": 2792, "statues_count": 50805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646107680768, "text": "@dollaas_ I stopped to think about this...and you have a point ��", "in_reply_to_status": 707061322723143681, "in_reply_to_user": 1423818548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1423818548 }}, "user": { "id": 2540776561, "name": "Joey", "screen_name": "joeypineo", "lang": "en", "location": "Private Island ", "create_at": date("2014-06-01"), "description": "Superhero", "followers_count": 330, "friends_count": 235, "statues_count": 3165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646271270913, "text": "That was not Cydney who posted that.. Stop being weird.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2959313437, "name": "Princess Naomi", "screen_name": "HeLoveNomi", "lang": "en", "location": "null", "create_at": date("2015-01-04"), "description": "solid bitches don't crumble. focused.", "followers_count": 2384, "friends_count": 772, "statues_count": 42515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646535680000, "text": "What's the storyline behind her sneakers? https://t.co/VaEXsRHPRU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2438987390, "name": "⠀", "screen_name": "Betz_DGAF", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-04-11"), "description": "#FeelingTheBern", "followers_count": 1028, "friends_count": 444, "statues_count": 49306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646636138496, "text": "@thebritofdallas I've noticed, for one reason or another, that I just miss a lot when they're not on. It's ritual at this point.", "in_reply_to_status": 707112272741556224, "in_reply_to_user": 141635026, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 141635026 }}, "user": { "id": 255155219, "name": "ʕ•ᴥ•ʔ", "screen_name": "MoTheBear", "lang": "en", "location": "Dallus, Tessus", "create_at": date("2011-02-20"), "description": "24, Know It All, Incredibly Underrated Tweeter. My Glass Is Always Half Full.", "followers_count": 735, "friends_count": 410, "statues_count": 54045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646678286337, "text": "\"YOU CAN MAKE A VAGINA OUT OF PLAY-DOH.\"\n\"IT'S A CHILD'S TOY, YOU SICK FUCK.\" \nGame night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65169444, "name": "Mama Paige", "screen_name": "egiapwieland", "lang": "en", "location": "Grover Beach, CA", "create_at": date("2009-08-12"), "description": "Anxiety and Self-Loathing in Paigestralia.", "followers_count": 336, "friends_count": 1350, "statues_count": 15280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arroyo Grande, CA", "id": "3ece1bab2041284e", "name": "Arroyo Grande", "place_type": "city", "bounding_box": rectangle("-120.60993,35.101686 -120.551631,35.146993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 602868, "cityName": "Arroyo Grande" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646745194496, "text": "@Sunset_SC2 LOL what?", "in_reply_to_status": 707113470827737088, "in_reply_to_user": 275790517, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 275790517 }}, "user": { "id": 132684863, "name": "tifapriori", "screen_name": "tifapriori", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-04-13"), "description": "Merchandising Associate | @Twitch. Formerly at Blizzard Entertainment making #blizzardgear & @IGNProleague. Tweets are my own!", "followers_count": 1065, "friends_count": 455, "statues_count": 6928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113646850072577, "text": "I really shouldn't be hungry right now, but I'm hungry right now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3187030004, "name": "Sharon Sumrit", "screen_name": "sharonsumrit", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "Tech PR @waltcomm. @CSUF Alum. Under promise, over deliver.", "followers_count": 165, "friends_count": 239, "statues_count": 332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113647861075968, "text": "I'm at Monarch Hotel and Convention Center in Clackamas, OR https://t.co/0WqaCU2192", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.56708604,45.43063939"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197812755, "name": "Flyboi01", "screen_name": "JoshLynch6", "lang": "en", "location": "null", "create_at": date("2010-10-02"), "description": "null", "followers_count": 465, "friends_count": 695, "statues_count": 12168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Happy Valley, OR", "id": "00bda52c7eddb627", "name": "Happy Valley", "place_type": "city", "bounding_box": rectangle("-122.581184,45.40302 -122.477198,45.461537") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4132050, "cityName": "Happy Valley" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113648007856129, "text": "@AmericanPhenom_ nope����", "in_reply_to_status": 707103859878076416, "in_reply_to_user": 233020958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 233020958 }}, "user": { "id": 312952871, "name": "Cam Carter", "screen_name": "thatguy2follow", "lang": "en", "location": "Hampton, VA", "create_at": date("2011-06-07"), "description": "Cam 7⃣0⃣4⃣✈️7⃣✋7⃣ Hampton University ⚓️ I Strive to be the best. Writer for PeachState College Sports", "followers_count": 2058, "friends_count": 1979, "statues_count": 119563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113648339230722, "text": "I wanna be a hockey scout. More than anything else in the world.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227564967, "name": "Allman Miller", "screen_name": "Its_Allman_Doe", "lang": "en", "location": "FunkyTown, USA || Tallahassee ", "create_at": date("2010-12-16"), "description": "Former front-man of groundbreaking alternative pop-reggae band Los Lonely Boys. State ball, baby.", "followers_count": 211, "friends_count": 184, "statues_count": 27248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113648376799235, "text": "the fact that I can't go back to sleep is so annoying. My only night to sleep in too ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336386674, "name": "asg", "screen_name": "ashlysgxoxo", "lang": "en", "location": "Katy, texas", "create_at": date("2011-07-15"), "description": "♕ |ig/phhhoto: ashlysgxoxo ❃", "followers_count": 1439, "friends_count": 601, "statues_count": 41890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-08T00:00:05.000Z"), "id": 707113648502611969, "text": "Temp: 55.3°F - Dew Point: 48.2° - Wind: 8.3 mph - Gust: 16.6 - Rain Today: 0.00in. - Pressure: 29.83in, - Trend: Rising slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113648674705414, "text": "�� thinkin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874457857, "name": "morgie", "screen_name": "MorgsMorton", "lang": "en", "location": "Ohio", "create_at": date("2014-11-12"), "description": "That grind is precious baby. Keep Hustlin", "followers_count": 12662, "friends_count": 8560, "statues_count": 3264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elida, OH", "id": "d9b1a8780fc61ff7", "name": "Elida", "place_type": "city", "bounding_box": rectangle("-84.21664,40.767243 -84.181568,40.797427") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39003, "countyName": "Allen", "cityID": 3924808, "cityName": "Elida" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113649228230656, "text": "19 ?? �������� https://t.co/RbSCAXQWJJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 718843267, "name": "❗️OVO BIGMIKE❗️", "screen_name": "trillyaquarius", "lang": "en", "location": "Houston, TX", "create_at": date("2012-07-26"), "description": "Follow me @trillyaquarius I'll follow back. PROMISE", "followers_count": 4972, "friends_count": 4792, "statues_count": 17116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113650478186496, "text": "@live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 #live955 ✨", "in_reply_to_status": -1, "in_reply_to_user": 299917756, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955", "live955" }}, "user_mentions": {{ 299917756 }}, "user": { "id": 86738008, "name": "ㅤㅤㅤㅤㅤㅤ", "screen_name": "viniciusarruda", "lang": "pt", "location": "Portland, OR", "create_at": date("2009-11-01"), "description": "ㅤㅤㅤㅤㅤㅤ", "followers_count": 1383, "friends_count": 327, "statues_count": 25039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113650733985792, "text": "Boutta get my ass are by my Bae @IamthequeenofTX ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312923219 }}, "user": { "id": 1375255879, "name": "Jose", "screen_name": "Obeyjosee214", "lang": "en", "location": "null", "create_at": date("2013-04-23"), "description": "null", "followers_count": 79, "friends_count": 146, "statues_count": 445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113651111641088, "text": "Wind 3.0 mph NNW. Barometer 30.062 in, Steady. Temperature 37.3 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 4882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-03-08T00:00:06.000Z"), "id": 707113651623165952, "text": "@ScousePower8 I don't follow tennis; does her explanation make sense? Or is it closer to so many other sports \"mea culpas\"/excuses?", "in_reply_to_status": 707095298338721792, "in_reply_to_user": 760521134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 760521134 }}, "user": { "id": 15682105, "name": "Keith Palau⭐️", "screen_name": "kpduck", "lang": "en", "location": "Portland (Westside), Oregon", "create_at": date("2008-07-31"), "description": "Husband, father, devoted follower of the beautiful game (PTFC-style), the Oregon Ducks, & Jesus. Not necessarily in that order. #TimbersArmy #GoDucks", "followers_count": 1319, "friends_count": 1139, "statues_count": 48230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockcreek, OR", "id": "6431c7b02057bcca", "name": "Rockcreek", "place_type": "city", "bounding_box": rectangle("-122.899663,45.538108 -122.855391,45.566322") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4163010, "cityName": "Rockcreek" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113652906688512, "text": "@jucaferreira CELEBRATE INTERNATIONAL WOMEN'S DAY! MARCH 8! \nhttps://t.co/8O6yZIyDY3", "in_reply_to_status": -1, "in_reply_to_user": 56513415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56513415 }}, "user": { "id": 19210349, "name": "ARTISTS PR", "screen_name": "ARTISTSPR", "lang": "en", "location": "Beverly Hills,California ", "create_at": date("2009-01-19"), "description": "I am PR of Artist / Philanthropist METIN BEREKETLI \nhttp://WWW.HOLLYWOODPAINTER.COM", "followers_count": 861, "friends_count": 1035, "statues_count": 13491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly Hills, CA", "id": "741e9df4d2522275", "name": "Beverly Hills", "place_type": "city", "bounding_box": rectangle("-118.427344,34.052559 -118.371934,34.112434") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 606308, "cityName": "Beverly Hills" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113653657427968, "text": "@velvetspiiice Lmaoo aww don't be like that", "in_reply_to_status": 707113596098981888, "in_reply_to_user": 2195044543, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2195044543 }}, "user": { "id": 46305981, "name": "Jack of DemBoy$", "screen_name": "ImDatNigga_Jack", "lang": "en", "location": "PVAMU", "create_at": date("2009-06-10"), "description": "The Father Of Slaughter Dem Boy$ IG:imdatnigga_jack #PVNation #HighLifePromos #LongLiveChopo Free @JoeSwaggerRight #LongLiveClyde", "followers_count": 5766, "friends_count": 4019, "statues_count": 333921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113653774974976, "text": "Where is Bonita Springs on the map? Play the game at https://t.co/EB8eqjH3RP #Bonita Springs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.7787,26.3398"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bonita" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1027, "friends_count": 311, "statues_count": 2638088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonita Springs, FL", "id": "3fbcab53578e6f0b", "name": "Bonita Springs", "place_type": "city", "bounding_box": rectangle("-81.878279,26.316205 -81.737244,26.406516") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1207525, "cityName": "Bonita Springs" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113654047485952, "text": "My ankle won't stop hurting & my toes are freezing even though I'm wearing those fuzzy socks ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1491774176, "name": "Sara-hí", "screen_name": "SarahiGallegos9", "lang": "en", "location": "California, USA", "create_at": date("2013-06-07"), "description": "My joy is boundless My soul knows it's worth", "followers_count": 538, "friends_count": 420, "statues_count": 21374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113654185922560, "text": "3/8/2016 - 02:00\nTemp: 62.9F \nHum: 79%\nWind: 4.0 mph\nBaro: 29.949in. & Steady\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 109, "friends_count": 53, "statues_count": 50659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113654794125312, "text": "Who's up?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244302588, "name": "Young Simba", "screen_name": "YoungSimbaSH", "lang": "en", "location": "null", "create_at": date("2011-01-28"), "description": "17.|Space Head Ent.| For Bookings & Features: spaceheads.ent@gmail.com @SpaceHeadsent #UCLA #1 https://m.soundcloud.com/youngsimbasounds", "followers_count": 3404, "friends_count": 279, "statues_count": 10536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113655322673152, "text": "Fr me to https://t.co/5leAL28Dzp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483627721, "name": "Daddy's Girl FL ✨", "screen_name": "Forever_Shyaa", "lang": "en", "location": "IG X_Shyaa & snapchat x_shyaa ", "create_at": date("2012-02-04"), "description": "I Count My Smiles Instead Of My Tears , & Count My Courage Instead Of My Fears", "followers_count": 2294, "friends_count": 2087, "statues_count": 35557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113655687458820, "text": "One thing Americans do know, is how to do make amazing cars. Made for ELVIS. Would love to drive it! @ClassicCar_TV https://t.co/2wVuvQVqsa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2576699208 }}, "user": { "id": 465412176, "name": "SPARKY FILM", "screen_name": "SPARKYFILM", "lang": "en", "location": "London", "create_at": date("2012-01-16"), "description": "London Video Production Company", "followers_count": 252, "friends_count": 74, "statues_count": 508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-08T00:00:07.000Z"), "id": 707113655746187264, "text": "Tickle Me, Elmo #BadSexIn3Words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BadSexIn3Words" }}, "user": { "id": 17775088, "name": "Charles Williams", "screen_name": "TMPChuck", "lang": "en", "location": "Southern California", "create_at": date("2008-12-01"), "description": "We done hid the monsters in the floor.", "followers_count": 174, "friends_count": 431, "statues_count": 3337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Irwin, CA", "id": "01118e5072ab7b7e", "name": "Fort Irwin", "place_type": "city", "bounding_box": rectangle("-116.707901,35.244849 -116.670256,35.274459") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 625114, "cityName": "Fort Irwin" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476011516960768, "text": "No drive.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429903372, "name": "D'Ann!", "screen_name": "DAnnParks", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-12-06"), "description": "Follow me on instagram: @DAnnParks Booking/Features: @GoldMindz_ Click the link in my bio to hear my music! ⬇⬇⬇⬇⬇", "followers_count": 1207, "friends_count": 715, "statues_count": 15115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476011663888385, "text": "Meeeeeeeeeeeeeee https://t.co/IQKTdjA5ww", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2844250976, "name": "mel$", "screen_name": "queeeenmel", "lang": "en", "location": "probably the gym", "create_at": date("2014-10-07"), "description": "I just wanna make it big w Aaron", "followers_count": 228, "friends_count": 196, "statues_count": 8034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476011970002944, "text": "@hi_itsthomas it's #liberating", "in_reply_to_status": 707475682956279809, "in_reply_to_user": 619845158, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "liberating" }}, "user_mentions": {{ 619845158 }}, "user": { "id": 847671642, "name": "Bernie Xanders", "screen_name": "bellatrillman", "lang": "en", "location": "xansas ", "create_at": date("2012-09-26"), "description": "for president", "followers_count": 286, "friends_count": 226, "statues_count": 4581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476012255162368, "text": "Oh hell no don't ever call me Shan lol foh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1435689444, "name": "sha", "screen_name": "shannen0hwhy", "lang": "en", "location": "(678)999-8212", "create_at": date("2013-05-17"), "description": "J. Sanesanong #4♡ -- 4'8 but my ego 6'5 / $$$", "followers_count": 446, "friends_count": 278, "statues_count": 20248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476012909469696, "text": "Start texting her all day every day and making her feel special. Then just ignore her and text her less #wastehertime2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wastehertime2016" }}, "user": { "id": 142893457, "name": "Allajziah", "screen_name": "ComeClean15", "lang": "en", "location": "null", "create_at": date("2010-05-11"), "description": "I've tried fixing people and people have tried to fix me. I can say that neither is possible because someone always ends up more broken than when they started.", "followers_count": 790, "friends_count": 1420, "statues_count": 5032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476012918022146, "text": "Sister time, watching @girlsHBO. Str8hollas. https://t.co/D5jtJtkcDj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 282789898 }}, "user": { "id": 3906083060, "name": "anna_sasin", "screen_name": "xanna_sasinx", "lang": "en", "location": "null", "create_at": date("2015-10-15"), "description": "THE ONLY DISABILITY IN LIFE IS A BAD ATTITUDE", "followers_count": 44, "friends_count": 28, "statues_count": 1017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alum Rock, CA", "id": "277b4360183b5d75", "name": "Alum Rock", "place_type": "city", "bounding_box": rectangle("-121.842175,37.35398 -121.812158,37.38102") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 601458, "cityName": "Alum Rock" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013010329602, "text": "�� https://t.co/bateX9v6Go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1338383336, "name": "Byron Sanchez", "screen_name": "byronsanchezzz", "lang": "en", "location": "null", "create_at": date("2013-04-08"), "description": "don't keep me in one place for too long.", "followers_count": 235, "friends_count": 246, "statues_count": 6874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Sacramento, CA", "id": "009243ac50ad0f26", "name": "West Sacramento", "place_type": "city", "bounding_box": rectangle("-121.584282,38.516197 -121.506483,38.603777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 684816, "cityName": "West Sacramento" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013186306048, "text": "Well this is my résumé to be @danisnotonfire 's wife RT for moral support of my unrealistic dreams I have issues k https://t.co/kmoWef6acA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40059553 }}, "user": { "id": 96924019, "name": "Dans Whorell", "screen_name": "slaymedanhowell", "lang": "en", "location": "#DAYLA", "create_at": date("2009-12-14"), "description": "i just want @danisnotonfire to touch my butt is that too much to ask? (soz I'm not Dan but ily anyway)", "followers_count": 621, "friends_count": 327, "statues_count": 3197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goleta, CA", "id": "0019e2618c34b6b2", "name": "Goleta", "place_type": "city", "bounding_box": rectangle("-119.913268,34.40581 -119.744123,34.473116") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 630378, "cityName": "Goleta" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013236654080, "text": "*i like it when it rains", "in_reply_to_status": 707475877639168000, "in_reply_to_user": 1328836172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1328836172, "name": "idk.", "screen_name": "ciankd0r0the0", "lang": "en", "location": "null", "create_at": date("2013-04-05"), "description": "AJ my BAABY", "followers_count": 471, "friends_count": 429, "statues_count": 18598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013500997632, "text": "@TadoeTv_ you good black ass ��", "in_reply_to_status": 707475863881682944, "in_reply_to_user": 3853293072, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3853293072 }}, "user": { "id": 2894202564, "name": "dezz.", "screen_name": "WNBADestined", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "you lurking?", "followers_count": 902, "friends_count": 865, "statues_count": 25006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013811408900, "text": "chaos in my veins \nkeeping me awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031736381, "name": "eye•scream•sunday", "screen_name": "artstarved", "lang": "en-GB", "location": "fuck you, that's where", "create_at": date("2015-02-11"), "description": "all monsters are human. my girlfriend is an otherworldly, punk rock, goddess creature. art hurts. [I AM GENERATION DOOM O7EP]", "followers_count": 443, "friends_count": 781, "statues_count": 18113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, GA", "id": "ec6dac47648ca27f", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-83.89971,34.238672 -83.762889,34.362593") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall", "cityID": 1331908, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476013895307264, "text": "I just wanna be a spicy potato soft taco", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433949415, "name": "Jake Ford", "screen_name": "ford_jake", "lang": "en", "location": "ECU", "create_at": date("2011-12-10"), "description": "null", "followers_count": 566, "friends_count": 194, "statues_count": 15176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Panama City Beach, FL", "id": "9ebd5acfac2301ba", "name": "Panama City Beach", "place_type": "city", "bounding_box": rectangle("-85.95802,30.165061 -85.786077,30.266595") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1254725, "cityName": "Panama City Beach" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476014360821761, "text": "robines water 7 outfit tho A+++", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159024741, "name": "Jill @ OP Ep 227", "screen_name": "PiratessUnluck", "lang": "en", "location": "So Cal", "create_at": date("2010-06-24"), "description": "---------Learn from the bones---------Twitter, Tumblr, and Skype are all the same handle.", "followers_count": 197, "friends_count": 204, "statues_count": 28270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476014528647168, "text": "I hope she knows what shes getting her self into", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 522699427, "name": "Shamayaaa", "screen_name": "_humble_02", "lang": "en", "location": "null", "create_at": date("2012-03-12"), "description": "#Libra", "followers_count": 1675, "friends_count": 1366, "statues_count": 41869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-03-09T00:00:00.000Z"), "id": 707476014948077568, "text": "Wonder if I can get to a 4 before IFO ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548522715, "name": "∆ ∇ Christian ∇ ∆", "screen_name": "CTizii", "lang": "en", "location": "Surprise, AZ", "create_at": date("2012-04-08"), "description": "Ever since this began, I was blessed with a curse. - San Diego, CA - Surprise, AZ", "followers_count": 336, "friends_count": 519, "statues_count": 19311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476015518326784, "text": "There's something about worship music I just adore https://t.co/AppXR5tB31", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 573082515, "name": "Jen", "screen_name": "jennihuntley", "lang": "en", "location": "Oroville, CA", "create_at": date("2012-05-06"), "description": "Betty White's stunt double.", "followers_count": 345, "friends_count": 264, "statues_count": 19766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oroville East, CA", "id": "2a152e11137e65d0", "name": "Oroville East", "place_type": "city", "bounding_box": rectangle("-121.538364,39.479173 -121.456825,39.53656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 654388, "cityName": "Oroville East" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476015640092672, "text": "#stayhealthy #stayhappy ✌️ https://t.co/yBTXz8HpeI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "stayhealthy", "stayhappy" }}, "user": { "id": 238620310, "name": "Shinde_Official", "screen_name": "123prasha", "lang": "en", "location": "Richardson, TX", "create_at": date("2011-01-15"), "description": "https://instagram.com/shinde_official/", "followers_count": 178, "friends_count": 561, "statues_count": 135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476015803666432, "text": "Why ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2681496999, "name": "Dominique", "screen_name": "Niqueeeeeeeee__", "lang": "en", "location": "Columbus, OH", "create_at": date("2014-07-06"), "description": "Rest up E! gone but NEVER forgotten.", "followers_count": 433, "friends_count": 317, "statues_count": 10284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476015887556609, "text": "@MAINEstersPamp @StanofMaine @mainedcm PARANG KAMAG ANAK NI MR. Grinch..LOL #ALDUBTheCompromise", "in_reply_to_status": 707473311094145024, "in_reply_to_user": 4411437678, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBTheCompromise" }}, "user_mentions": {{ 4411437678, 1937760000, 63701775 }}, "user": { "id": 3669938115, "name": "Aldub Global NewYork", "screen_name": "aldubglobalnyc", "lang": "en", "location": "Queens, NY", "create_at": date("2015-09-15"), "description": "@aldenrichards02 and @mainedcm FRIENDS @ TRI-STATE AREA (NEW YORK CITY, NEW JERSEY, & CONNECTICUT) -Only All good vibes & Day To Day Inspirations", "followers_count": 4757, "friends_count": 1422, "statues_count": 13959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476016235716609, "text": "����- I thinc yo lips are as big as mine ������ #rare", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rare" }}, "user": { "id": 705376831, "name": "Leveon Bell 2017 MVP", "screen_name": "Leel_hussle", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-07-19"), "description": "Stay Hungry Stay Humble. I'm tryna get to the league. #RIPSNUPE #RIPPOPS #steelers #yankees #lakers CSUB '19 #FREEJOHN", "followers_count": 1109, "friends_count": 826, "statues_count": 26197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476016680312832, "text": "Happy birthday to my favorite whore. Wish I was there to dance&blast 22 in the car with you, cause I know you will. @mystery_smith ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2429367895 }}, "user": { "id": 1710918356, "name": "ⓓⓐⓡⓐ", "screen_name": "iDarianStanard", "lang": "en", "location": "l♱❁♪☯☠l", "create_at": date("2013-08-29"), "description": "Stealing my best friends last name, αρяιℓ ❶❻ᴛʜ❥ @michelle_walls7 is my person", "followers_count": 131, "friends_count": 317, "statues_count": 8568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nipomo, CA", "id": "51ca80066df113e2", "name": "Nipomo", "place_type": "city", "bounding_box": rectangle("-120.533212,35.003173 -120.447468,35.052688") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 651476, "cityName": "Nipomo" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476016713826306, "text": "@mylife �������� https://t.co/H0F0aie6lV", "in_reply_to_status": -1, "in_reply_to_user": 7520962, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 7520962 }}, "user": { "id": 1232910158, "name": "KįñgPAPI", "screen_name": "Spaced_Mind", "lang": "en", "location": "null", "create_at": date("2013-03-01"), "description": "♑️Megz just stay roaming .... Sharper12", "followers_count": 161, "friends_count": 245, "statues_count": 777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017221394432, "text": "Ripley SW Limestone Co. Temp: 60.6°F Wind:3.8mph Pressure: 998.4mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 49995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017347043329, "text": "@ebbtideapp Tide in Cedar Creek entrance, New Jersey 03/09/2016\n Low 3:46am -0.7\nHigh 9:32am 6.7\n Low 4:16pm -0.6\nHigh 9:56pm 6.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.2467,39.2983"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 41, "friends_count": 1, "statues_count": 14274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Jersey, USA", "id": "65b4760a2b411e11", "name": "New Jersey", "place_type": "admin", "bounding_box": rectangle("-75.563587,38.788657 -73.88506,41.357424") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34011, "countyName": "Cumberland" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017527463936, "text": "@VANS_66 Can the prison issue just come back please ��", "in_reply_to_status": -1, "in_reply_to_user": 15383636, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15383636 }}, "user": { "id": 2558291406, "name": "Jerome", "screen_name": "ComiBoog", "lang": "en", "location": "Marysville, WA", "create_at": date("2014-06-09"), "description": "Member of ManimeTV", "followers_count": 96, "friends_count": 173, "statues_count": 1783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017628254209, "text": "Anyway. I really am gonna go to bed now. (Dammit, Hawaii.)\n\n#NeverTrump", "in_reply_to_status": 707468640619274241, "in_reply_to_user": 15990298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NeverTrump" }}, "user": { "id": 15990298, "name": "Mile High Brendan", "screen_name": "MileHighBrendan", "lang": "en", "location": "Colorado, USA", "create_at": date("2008-08-25"), "description": "Politics, sports, weather, LOTR, #PANIC, #DOOM, #dadlife. #ResistTrump #AgainstTrump #NeverTrump", "followers_count": 5479, "friends_count": 5797, "statues_count": 178553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stapleton, Denver", "id": "389b6bffced8dd19", "name": "Stapleton", "place_type": "neighborhood", "bounding_box": rectangle("-104.903602,39.747411 -104.827773,39.812981") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017737244672, "text": "Wind 0.0 mph SE. Barometer 29.826 in, Rising Rapidly. Temperature 61.8 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476017821184000, "text": "I just need to be held.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1580725874, "name": "manda", "screen_name": "mandymoo1342", "lang": "en", "location": "AHS/PNW", "create_at": date("2013-07-09"), "description": "\\\\ Insta: mandy_moo13 // \\\\ Snap: mandamurphy // || Skylar John | 8•18•15 ||", "followers_count": 429, "friends_count": 381, "statues_count": 35159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, WA", "id": "744fbe9224233893", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-122.199229,48.133613 -122.10191,48.203671") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5302585, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476018064445441, "text": "03/09@03:00 - Temp 42.4F, WC 42.4F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.133in, Falling slowly. Rain 0.00in. Hum 84%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-09T00:00:01.000Z"), "id": 707476019402256384, "text": "Social experiment time! Who can use the n-word??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347609983, "name": "Indi Wap", "screen_name": "Indigill", "lang": "en", "location": "Simi Valley, CA", "create_at": date("2011-08-02"), "description": "2015 was a fucking disaster, but 2016?? yeah that's gonna be a disaster too. let's be real.", "followers_count": 425, "friends_count": 545, "statues_count": 16640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476020065144832, "text": "#SupportOriginMelissa 56.1°F Wind:2.9mph Pressure: 30.06hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 311930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476020165746688, "text": "I WANT THIS!!!!!!! https://t.co/V0Ac8fT2iC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2150026003, "name": "princess || 8 DAYS", "screen_name": "BR0NIES", "lang": "en", "location": "Grand Junction, CO", "create_at": date("2013-10-22"), "description": "❥❥❥ the appropriation of brony culture. || bgd mom™ || #mommysquad", "followers_count": 1381, "friends_count": 154, "statues_count": 132223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Junction, CO", "id": "3b2155c98f7ba6a7", "name": "Grand Junction", "place_type": "city", "bounding_box": rectangle("-108.662336,39.033602 -108.486897,39.149899") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8077, "countyName": "Mesa", "cityID": 831660, "cityName": "Grand Junction" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476020912328704, "text": "Wind 1.0 mph SSE. Barometer 30.046 in, Rising slowly. Temperature 58.8 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476021520506880, "text": "Temp: 63.5°F Wind:3.4mph Pressure: 29.986hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476021927419905, "text": "@Slick_Teddy93 I'm a faithful black man I only get pics from one person lol", "in_reply_to_status": 707475789567148032, "in_reply_to_user": 528754939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 528754939 }}, "user": { "id": 261102661, "name": "New Bae®", "screen_name": "TheRealYoungZel", "lang": "en", "location": "DFW,Texas ✈️ HTX", "create_at": date("2011-03-04"), "description": "I make music on drugs enjoy. #FreeStreetz. Follow my backup @HighNiggaZel if you real. SC: indiana_stoned. Business inquiries: youngzelmusic@gmail.com", "followers_count": 13747, "friends_count": 1792, "statues_count": 114137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476022296322048, "text": "Mi dolor de cabeza era por no haber comido", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 283281083, "name": "Noelia Cabrera ❁", "screen_name": "NoeliaCabrera27", "lang": "en", "location": "Mission Viejo, CA", "create_at": date("2011-04-16"), "description": "no one cares, neither do I.", "followers_count": 875, "friends_count": 2000, "statues_count": 12360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Viejo, CA", "id": "6b55df049258bade", "name": "Mission Viejo", "place_type": "city", "bounding_box": rectangle("-117.695948,33.534941 -117.617598,33.672075") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 648256, "cityName": "Mission Viejo" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476022644506624, "text": "Heavy rain hitting the roof creates a majestic sound imo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 576783363, "name": "Fargo Argentina™", "screen_name": "patpatbush", "lang": "en", "location": "At the rim ", "create_at": date("2012-05-10"), "description": "God first then hoop dreams", "followers_count": 831, "friends_count": 429, "statues_count": 35043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476022791446528, "text": "We grow in this great life by making room for Jesus Christ in our outlook on everything.\nOswald Chambers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 139, "friends_count": 257, "statues_count": 221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476022808190977, "text": "I'm gonna look like Jenny's ratchet ass bc I broke a nail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18184353, "name": "Sammie", "screen_name": "MissCago", "lang": "en", "location": "Schaumburg, IL", "create_at": date("2008-12-16"), "description": "26. Ellen Degeneres number 1 fan", "followers_count": 102, "friends_count": 189, "statues_count": 2990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wauconda, IL", "id": "01fbe65a0127f8ca", "name": "Wauconda", "place_type": "city", "bounding_box": rectangle("-88.164314,42.241083 -88.108609,42.306471") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1779267, "cityName": "Wauconda" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023152017408, "text": "Omg when your bf's mom is so cute I think I'm in love https://t.co/nVuc5G0xQi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2960799918, "name": "baecey", "screen_name": "trayceecook", "lang": "en", "location": "#GoDawgs", "create_at": date("2015-01-06"), "description": "aoπ | @aaronsdrops ❤️ | life is plurfect", "followers_count": 1271, "friends_count": 956, "statues_count": 55829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023219097600, "text": "I was addicted to you, but shit just like drugs, it ain't never good for ya.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2904166988, "name": "BANGBROS", "screen_name": "vales_josh21", "lang": "en", "location": "lost", "create_at": date("2014-11-18"), "description": "sc: joshvales99", "followers_count": 322, "friends_count": 273, "statues_count": 1948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023223394305, "text": "I've always checked, and there's a reason why.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2365838703, "name": "Gillian Foster.", "screen_name": "deworlddegirl", "lang": "en", "location": "Univ of Maryland Eastern Shore", "create_at": date("2014-02-25"), "description": "Grange Hill, Jamaica ♏️ Chicago, Illinois @takecaredevin ❤️", "followers_count": 1812, "friends_count": 2008, "statues_count": 22669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princess Anne, MD", "id": "0098bec2526560be", "name": "Princess Anne", "place_type": "city", "bounding_box": rectangle("-75.715802,38.152156 -75.668895,38.236042") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24039, "countyName": "Somerset", "cityID": 2464000, "cityName": "Princess Anne" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023387033601, "text": "@iDontAnswerHoes oh yea Ify... My best essays are the ones I write 2 hours before it's due ��������", "in_reply_to_status": 707475751340265472, "in_reply_to_user": 359059925, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 359059925 }}, "user": { "id": 261549348, "name": "Bella Noché 02/26", "screen_name": "internet_threat", "lang": "en", "location": "Tuscaloosa, AL", "create_at": date("2011-03-05"), "description": "Ok you don't give a damn??? Neither do I but somebody's gone come out hea and they gone see dat duck it dat pool IG: dar_thegoat. \n❤RIP Charity", "followers_count": 2090, "friends_count": 1127, "statues_count": 135960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northport, AL", "id": "003779400ddc8a06", "name": "Northport", "place_type": "city", "bounding_box": rectangle("-87.644402,33.207777 -87.540968,33.333623") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 155200, "cityName": "Northport" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023709974528, "text": "Don't get disrespectful under posts you never know who knows the person your judging and never… https://t.co/vhg6QhfcrI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3758,28.5335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20290065, "name": "Star Cavalli", "screen_name": "StarCavalli", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2009-02-06"), "description": "Add my station http://Pandora.com/Star-Cavalli #HipHopsFavLesbo #ShoreSideLife CEO @KayozUncut MGR #hollywoodvalli #southbeachvalli http://kayozuncut.com", "followers_count": 26398, "friends_count": 11090, "statues_count": 64090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-09T00:00:02.000Z"), "id": 707476023751917568, "text": "these headaches & sharp pains i've been getting<<<����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2474581614, "name": "catalina☹️", "screen_name": "catalinaaa16", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "null", "followers_count": 394, "friends_count": 176, "statues_count": 41312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476024095744001, "text": "Norb, Ryan, Jr & Ty https://t.co/lWu32sq2ga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 574975851, "name": "josh", "screen_name": "madmaxhengst", "lang": "en", "location": "HTX✈️LBK", "create_at": date("2012-05-08"), "description": "I'm the fish dude", "followers_count": 744, "friends_count": 231, "statues_count": 39803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476024976547840, "text": "Why he gotta be a dust monkey? I always take Malcs side ���� https://t.co/DdnMBc4fSM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191649388, "name": "Roxanne, Roxanne", "screen_name": "Cali_StyleJas", "lang": "en", "location": "Bay ✈️ USC", "create_at": date("2010-09-16"), "description": "Daughter to the spinnin post. Queen of 2,000 moons. Sister to the distant yet risin' star.. I'm just here to tweet lyrics. Unapologetically black ✨", "followers_count": 3166, "friends_count": 2827, "statues_count": 262165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476025169600513, "text": "Wind 0.7 mph SW. Barometer 30.07 in, Steady. Temperature 37.8 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 116, "statues_count": 158107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476025249112064, "text": "@cummies_ LMFAO", "in_reply_to_status": 707475834987257857, "in_reply_to_user": 1902168013, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1902168013 }}, "user": { "id": 990116622, "name": "jeb is a mess", "screen_name": "kinksamer", "lang": "en", "location": "welcome to hell", "create_at": date("2012-12-04"), "description": "Hey there, my name's Alex! I tweet way too much and I should be stopped // He/They pronouns please // icon by @shoegazewolf", "followers_count": 582, "friends_count": 870, "statues_count": 50354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476026427723776, "text": "\"You coming home tonight\" �� my dad swear I have a man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 364577353, "name": "K.Monaè", "screen_name": "_PinkGrenades", "lang": "en", "location": "null", "create_at": date("2011-08-29"), "description": "FAMU17|BSLS...Fierce64|ΦΑΔFall 2k15|SC:@shunaaaaa", "followers_count": 1679, "friends_count": 1254, "statues_count": 41387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Miami Beach, FL", "id": "045badf8fedd9c63", "name": "North Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.208932,25.914146 -80.130673,25.957137") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1249475, "cityName": "North Miami Beach" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476026801127425, "text": "I have nothing against strippers. But I can't stand the petty ass conceited ones.�� Sweetie, say sorry to your parents & go home boo��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2367508640, "name": "Melissa D Heng", "screen_name": "melissadheng", "lang": "en", "location": "las vegas", "create_at": date("2014-03-01"), "description": "Yes, I'm fancy Drake.", "followers_count": 350, "friends_count": 265, "statues_count": 3312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-03-09T00:00:03.000Z"), "id": 707476027304517634, "text": "@Baeshia_ ������", "in_reply_to_status": 707468062803554308, "in_reply_to_user": 289077843, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 289077843 }}, "user": { "id": 228908265, "name": "Steady Mobbin", "screen_name": "KAMsta180", "lang": "en", "location": "Norfolk, VA", "create_at": date("2010-12-20"), "description": "21 ODU♦️ΗΩ Νupe @TheComeUp_KAM (IG)", "followers_count": 841, "friends_count": 851, "statues_count": 19320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476028059467777, "text": "my name isn't Ash ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 973140626, "name": "hannah gilliland ☺️", "screen_name": "dummyswag_", "lang": "en", "location": "nebraskiiii ", "create_at": date("2012-11-26"), "description": "gangsta rap made me do it.", "followers_count": 3626, "friends_count": 3605, "statues_count": 13753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476028059475968, "text": "You don't even have to pull the carb all you do is press down �� https://t.co/x6tIMOHdkq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1164720325, "name": "⚡️", "screen_name": "rachaelnelsonn", "lang": "en", "location": "null", "create_at": date("2013-02-09"), "description": "If you ain't talking documentaries I don't wanna talk #SFA18 \n\n\n1 of 2", "followers_count": 298, "friends_count": 659, "statues_count": 7614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476028134793217, "text": "@Evvaaa29 chill out lil one https://t.co/vzkfljY5iH", "in_reply_to_status": 707475726149279744, "in_reply_to_user": 3287305872, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3287305872 }}, "user": { "id": 3102230533, "name": "Brandn", "screen_name": "brandendonttrip", "lang": "en", "location": "null", "create_at": date("2015-03-21"), "description": "digging it", "followers_count": 187, "friends_count": 150, "statues_count": 4527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476029904982016, "text": "I need to stop taking four hour naps in the middle of the day....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407445914, "name": "Ella Miceli", "screen_name": "ellabells10", "lang": "en", "location": "Akron, Ohio", "create_at": date("2011-11-07"), "description": "21. University of Akron. ✌", "followers_count": 196, "friends_count": 264, "statues_count": 3562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476030525612032, "text": "Damn im 21 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488209803, "name": "Luigi(:", "screen_name": "LueyGeee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-02-09"), "description": "null", "followers_count": 334, "friends_count": 276, "statues_count": 15962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476030546694145, "text": "Got me feeling like the last man standing..smh\n#FreeRoscoe… https://t.co/WzolQOoLuE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.8127,33.5248"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FreeRoscoe" }}, "user": { "id": 159987891, "name": "YOUNG 2ThaPj", "screen_name": "young2thapj", "lang": "en", "location": "BIRMINGHAM,ALABAMA", "create_at": date("2010-06-26"), "description": "205 REPRESENTA..SONGWRITER/ARTIST,PRODUCER,MOGUL N DA MAKING FROM BHAM,AL...#RollTideGang CONTACT ME @ BEATS4YOUNGPJ@GMAIL.COM", "followers_count": 1217, "friends_count": 998, "statues_count": 10442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476031427350528, "text": "Wind 1.6 mph W. Barometer 29.64 in, Rising. Temperature 41.2 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 40611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-03-09T00:00:04.000Z"), "id": 707476031721119744, "text": "@annajamson88 \nBreakfast, lunch and dinner baby!", "in_reply_to_status": 707369525562515456, "in_reply_to_user": 3372774807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3372774807 }}, "user": { "id": 34973230, "name": "ron", "screen_name": "freewhilly13", "lang": "en", "location": "null", "create_at": date("2009-04-24"), "description": "null", "followers_count": 48, "friends_count": 171, "statues_count": 103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476032463380481, "text": "@nessa__cisneros wish I had one ��", "in_reply_to_status": 707475742880215041, "in_reply_to_user": 2298744662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2298744662 }}, "user": { "id": 214306436, "name": "grassshoppper", "screen_name": "LinaGolub", "lang": "en", "location": "null", "create_at": date("2010-11-10"), "description": "whatever 4ever", "followers_count": 192, "friends_count": 137, "statues_count": 2686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, WA", "id": "397854265ee65aa6", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-122.144491,47.345144 -122.058504,47.395662") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5315290, "cityName": "Covington" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476032798990336, "text": "Travelling to Myrtle Beach or just twittering about Myrtle Beach? https://t.co/6KzuSI3oHy #Myrtle Beach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.8867,33.6891"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Myrtle" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2639674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476032824053760, "text": "@whatcedsays https://t.co/mcpw6YSlpy", "in_reply_to_status": -1, "in_reply_to_user": 535067880, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 535067880 }}, "user": { "id": 2905535293, "name": "Justin Aguilar", "screen_name": "jma_justin", "lang": "en", "location": "Tulsa, OK", "create_at": date("2014-11-20"), "description": "Movin' On Up", "followers_count": 91, "friends_count": 101, "statues_count": 2227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476033319084034, "text": "@Zlatan_Ivancic might come down Friday night cuz I wanna wake up and get to boozin instantly", "in_reply_to_status": 707471690268327936, "in_reply_to_user": 374282444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 374282444 }}, "user": { "id": 559765317, "name": "Ko$ey", "screen_name": "R_R_R_Rosa", "lang": "en", "location": "The Cabin", "create_at": date("2012-04-21"), "description": "I'm too nice wit it right wit. #BIXB Artist. Pull up & fuccs with the kid kosey2nice@gmail.com #37", "followers_count": 959, "friends_count": 1243, "statues_count": 15416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willoughby, OH", "id": "88dfdf4f649a0ba2", "name": "Willoughby", "place_type": "city", "bounding_box": rectangle("-81.452726,41.603963 -81.370884,41.707193") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985484, "cityName": "Willoughby" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476033574842368, "text": "もう寝よ、、、", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 1313161688, "name": "Mizuna Inoue", "screen_name": "MizunaInoue", "lang": "ja", "location": "null", "create_at": date("2013-03-29"), "description": "城東23期→ニューヨーク州立大学 Geneseo校 / international relations major/引き寄せ❤️", "followers_count": 282, "friends_count": 317, "statues_count": 7361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Geneseo, NY", "id": "fa70259289515d5a", "name": "Geneseo", "place_type": "city", "bounding_box": rectangle("-77.828767,42.772513 -77.778016,42.815875") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36051, "countyName": "Livingston", "cityID": 3628618, "cityName": "Geneseo" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476033860018176, "text": "\"So you're friends with @12Rudyjr right...?\" .... #OKAYYY #YALLSLEEP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OKAYYY", "YALLSLEEP" }}, "user_mentions": {{ 617645560 }}, "user": { "id": 36257810, "name": "Luisa Castro", "screen_name": "LlamaLuisa", "lang": "en", "location": "Spurs Nation, Texas", "create_at": date("2009-04-28"), "description": "#YeezyTaughtMe• UTSA AlphaSig • Self-proclaimed Sports Analyst • #GoRunners • #DezCaughtIt •", "followers_count": 935, "friends_count": 589, "statues_count": 15329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476034011144192, "text": "@Andrew_Laeddis https://t.co/fqyDflgc3s", "in_reply_to_status": 707458710600355840, "in_reply_to_user": 27600961, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 27600961 }}, "user": { "id": 979477476, "name": "damn kaylee", "screen_name": "kayleeellsworth", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-11-29"), "description": "wish u were beer", "followers_count": 774, "friends_count": 537, "statues_count": 41032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476034032111616, "text": "يلا زين بدال لاتصرف فلوسك على سفره وخرابيط https://t.co/eKaHsTtq1S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 391251542, "name": "بو يعقوب", "screen_name": "Y_Alshama3", "lang": "en", "location": "ALZAHARA❤️BARCELONA", "create_at": date("2011-10-15"), "description": "USA-ARIZONA اللهم ارحم جدتي", "followers_count": 381, "friends_count": 96, "statues_count": 9276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476034401075200, "text": "I might have to hit up cafe clem tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 583406117, "name": "Kadijah Means", "screen_name": "SpecialKay00", "lang": "en", "location": "510", "create_at": date("2012-05-17"), "description": "Say. Her. Name. @ABlackCoalition RTs ≠ endorsements She/Her kadijah.means@gmail.com", "followers_count": 1084, "friends_count": 519, "statues_count": 29314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476034455740416, "text": "Wind 2.0 mph S. Barometer 30.011 in, Steady. Temperature 57.2 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 385, "friends_count": 292, "statues_count": 8867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476035713916928, "text": "@youngsterjiji https://t.co/9OECMH7iU4", "in_reply_to_status": 707475024953872385, "in_reply_to_user": 3182190481, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3182190481 }}, "user": { "id": 20407320, "name": "vape daddy", "screen_name": "KevCantHang", "lang": "en", "location": " ¯\\_(ツ)_/¯", "create_at": date("2009-02-08"), "description": "Fuckboi Of The Year 2K14", "followers_count": 1502, "friends_count": 578, "statues_count": 28465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476035932176384, "text": "Someone wants to go to Japan with us... �������� Leaving my heart it Vallejo.. �� @ Winterfell https://t.co/JTMWXcN8lg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.21633833,38.07647217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38138503, "name": "Via F", "screen_name": "swimmalita", "lang": "en", "location": "NorCal", "create_at": date("2009-05-06"), "description": "Digital Artist. Pixar. Corgis. Unicorns. Harry Potter. \r\n\r\nI'm iMDb-able :)", "followers_count": 121, "friends_count": 219, "statues_count": 14258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-03-09T00:00:05.000Z"), "id": 707476035936382977, "text": "\"I put it on my hair products\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 708757931, "name": "KM", "screen_name": "KenMonai", "lang": "en", "location": "null", "create_at": date("2013-10-10"), "description": "$ cosmetologist FSU'17", "followers_count": 376, "friends_count": 130, "statues_count": 2401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frostburg, MD", "id": "105ebe2ad97a9958", "name": "Frostburg", "place_type": "city", "bounding_box": rectangle("-78.948683,39.633535 -78.902675,39.676134") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24001, "countyName": "Allegany", "cityID": 2430900, "cityName": "Frostburg" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476036485681152, "text": "@cheikhmoses1 I got you I'll find one", "in_reply_to_status": 707475822291099648, "in_reply_to_user": 3433074492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3433074492 }}, "user": { "id": 3238012269, "name": "lil chano from 79", "screen_name": "livvinlegndd", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-05-05"), "description": "Jr. ¯\\_(ツ)_/¯ Future sun devil I promise it", "followers_count": 298, "friends_count": 466, "statues_count": 1431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476036628385792, "text": "@xfranman He is already toast, a FL win is only to block DT #CruzOrLose", "in_reply_to_status": 707475067077267456, "in_reply_to_user": 21553487, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CruzOrLose" }}, "user_mentions": {{ 21553487 }}, "user": { "id": 232712997, "name": "Cats Endorsing Cats", "screen_name": "GeenaMidtown", "lang": "en", "location": "Atlanta - sometimes Houston", "create_at": date("2010-12-31"), "description": "Devon Cat of the world, follows hooman culture, politics, finance, and sports; Prowling Midtown Atlanta; Defender of cats, dogs, and anipals of all ideologies", "followers_count": 782, "friends_count": 1029, "statues_count": 11032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, GA", "id": "ae8761907bd9c420", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-84.315827,33.750938 -84.277985,33.793477") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1322052, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476036657647616, "text": "Temp: 57.2°F - Dew Point: 55.2° - Wind: 10.3 mph - Gust: 21.5 - Rain Today: 0.04in. - Pressure: 29.75in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476036884107264, "text": "Overwhelmed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477332547, "name": "Mariana", "screen_name": "holamariana_", "lang": "en", "location": "null", "create_at": date("2012-01-28"), "description": "it's a good day to have a good day • csuf", "followers_count": 281, "friends_count": 243, "statues_count": 15395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476037647605760, "text": "I hate when you don't talk to someone for a while and they hit you with the worst action \"doddaye what happen to you disappeared\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3747330682, "name": "Brobiwan Kenobi", "screen_name": "BrightonWJ", "lang": "en", "location": "Kings Landing, Westeros", "create_at": date("2015-09-23"), "description": "These are not the broids you're looking for..", "followers_count": 127, "friends_count": 230, "statues_count": 799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgewood, NJ", "id": "38718ccc91e76508", "name": "Ridgewood", "place_type": "city", "bounding_box": rectangle("-74.149096,40.955019 -74.077902,41.007882") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3463000, "cityName": "Ridgewood" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476037819441152, "text": "52 more dollars left to make for Florida ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317998017, "name": "Mer.", "screen_name": "Idella__", "lang": "en", "location": "Manhattan, NY", "create_at": date("2011-06-15"), "description": "T H R I V I N G APMR Forever. #UTA", "followers_count": 840, "friends_count": 563, "statues_count": 78427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476038041731072, "text": "@mrlorenzosnow @LexxxieMae @BryndenXXX @VivianIsNotArt @DesTitsAndAss @Xutjja @PleasntlyPlump", "in_reply_to_status": 707474181093326848, "in_reply_to_user": 3051748622, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2736580887, 3238387692, 3163190046, 1145497350, 142950680, 1177013730, 23394918 }}, "user": { "id": 3051748622, "name": "ZoUndergroundStudio™", "screen_name": "ZoUnderground", "lang": "en", "location": "North Texas and ✈️✈️", "create_at": date("2015-02-28"), "description": "Cinematic Alternative Dark/Suspence/Goth/Reality/Grindhouse film studios.#zoangels #teamunderground http://www.manyvids.com/Profile/550162/Zo-UndergroudStudios/", "followers_count": 1537, "friends_count": 2677, "statues_count": 11448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476038188556288, "text": "Interested in a #Retail #job near #MIAMI, FL? This could be a great fit: https://t.co/B8SCy7o7C3 #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.256252,25.781368"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "MIAMI", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 539, "friends_count": 251, "statues_count": 17715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476039157596160, "text": "Yo I dead ass just heard claws scratch my carpet under my bed. Wtf do I do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191741390, "name": "Sav", "screen_name": "savaaage_", "lang": "en", "location": "null", "create_at": date("2015-05-10"), "description": "@paidfamray is all me #fuckdonaldtrump", "followers_count": 331, "friends_count": 296, "statues_count": 3880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476039191101440, "text": "At tired ramp to myself and OHONE bout to die. What better way of I die tonight :-) https://t.co/SiU6YtNP2R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.00896344,42.34854327"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 512650421, "name": "Guy with the orange", "screen_name": "JMilosek", "lang": "en", "location": "null", "create_at": date("2012-03-02"), "description": "null", "followers_count": 727, "friends_count": 563, "statues_count": 10901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476039455395840, "text": "Okay I really miss my long hair.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 942899126, "name": "Savage Sev", "screen_name": "SevannahMirii", "lang": "en", "location": "Upland, CA", "create_at": date("2012-11-11"), "description": "chillin'", "followers_count": 647, "friends_count": 948, "statues_count": 10814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476040050974720, "text": "Lol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 480054626, "name": "Miranda Creggar", "screen_name": "MirandaCreggar", "lang": "en", "location": "Murfreesboro, TN", "create_at": date("2012-01-31"), "description": "From Daytona Beach to Murfreesboro. Let's have a conversation.", "followers_count": 344, "friends_count": 534, "statues_count": 5447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ormond Beach, FL", "id": "1e12587898ca9a9e", "name": "Ormond Beach", "place_type": "city", "bounding_box": rectangle("-81.061031,29.264995 -81.027452,29.318406") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1253150, "cityName": "Ormond Beach" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476040222957568, "text": "If you like, the way you look that much, baby you should go and FUCK yourself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 602953582, "name": "Jack", "screen_name": "DaCubsFan13", "lang": "en", "location": "Wrigleyville, Il.", "create_at": date("2012-06-08"), "description": "Proud constitutionalist and die hard Cubbies fan. #CDTDCC #JLM", "followers_count": 472, "friends_count": 328, "statues_count": 10748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winnebago, IL", "id": "7f0e28305e192abf", "name": "Winnebago", "place_type": "city", "bounding_box": rectangle("-89.25175,42.251747 -89.173538,42.278086") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1782491, "cityName": "Winnebago" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476040281690112, "text": "@droy_j @C_lewis8 cya never", "in_reply_to_status": 707475903253708800, "in_reply_to_user": 512611155, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 512611155, 563039303 }}, "user": { "id": 572431362, "name": "Brian Schrader", "screen_name": "BrianSchra", "lang": "en", "location": "MI", "create_at": date("2012-05-05"), "description": "Miller Lite Employee | Gym Rat", "followers_count": 496, "friends_count": 387, "statues_count": 1801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476040507985921, "text": "I fail at trying to sleep early", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 598897013, "name": "Melisa Garcia", "screen_name": "ayo_shawty1", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "Always Working But Still Broke Af ✨ -Nate // Small Girl With Big Dreams ✨", "followers_count": 292, "friends_count": 280, "statues_count": 15502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-03-09T00:00:06.000Z"), "id": 707476040529084416, "text": "When your crush brag about her sex game on Twitter https://t.co/Smg4PEtBDg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334803893, "name": "Hennëssy Papì", "screen_name": "jaydaddy__", "lang": "en", "location": "Orlando, Fl", "create_at": date("2011-07-13"), "description": "#SqueezyGang", "followers_count": 532, "friends_count": 498, "statues_count": 6733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alafaya, FL", "id": "00d511d335cd9fb6", "name": "Alafaya", "place_type": "city", "bounding_box": rectangle("-81.244501,28.489488 -81.125632,28.568794") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1200410, "cityName": "Alafaya" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476040986152960, "text": "As if I couldn't LOVE -#HelenMirren any more than I already do, she did THAT to @StephenAtHome! Atta girl! #LifeGoals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HelenMirren", "LifeGoals" }}, "user_mentions": {{ 16303106 }}, "user": { "id": 27829587, "name": "Wilson Cruz", "screen_name": "wcruz73", "lang": "en", "location": "Los Angeles", "create_at": date("2009-03-30"), "description": "Actorvist! \n\nViews are MY OWN!", "followers_count": 28934, "friends_count": 1499, "statues_count": 14862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476041309167616, "text": "Absolutely devastated to hear of #GeorgeMartin passing on - #thebeatles were everything because of him. #RIPGeorgeMartin !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GeorgeMartin", "thebeatles", "RIPGeorgeMartin" }}, "user": { "id": 327142672, "name": "Dream Alive", "screen_name": "DreamAliveMusic", "lang": "en", "location": "Los Angeles", "create_at": date("2011-06-30"), "description": "High-energy rock band with influences from Pink Floyd, The Beatles, Rush, Black Sabbath, and Symphonic Rock", "followers_count": 6557, "friends_count": 5154, "statues_count": 929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476041435103232, "text": "[17:00:06] 209.126.97.240:5076 >> :5060 (UDP)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.1922,38.6312"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 550261599, "name": "infosrv", "screen_name": "infosrv", "lang": "ja", "location": "null", "create_at": date("2012-04-10"), "description": "SERVER NOTIFICATIONS", "followers_count": 1, "friends_count": 1, "statues_count": 424583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Louis, MO", "id": "0570f015c264cbd9", "name": "St Louis", "place_type": "city", "bounding_box": rectangle("-90.320516,38.533193 -90.175132,38.774349") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29510, "countyName": "St. Louis", "cityID": 2965000, "cityName": "St. Louis" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476042462715904, "text": "@FauxMcCaskey My point is, he's a pussy.", "in_reply_to_status": 707475951102390273, "in_reply_to_user": 2922998997, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2922998997 }}, "user": { "id": 84926600, "name": "Matthew Reynolds", "screen_name": "MJR747", "lang": "en", "location": "Oak Lawn, IL", "create_at": date("2009-10-24"), "description": "I'm a nerd/sports nut, and love all things sports, science, aviation, weather, and U2. Go White Sox! Go Stewart-Haas Racing! Go Bears! Go Blackhawks!", "followers_count": 1123, "friends_count": 2108, "statues_count": 91108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Lawn, IL", "id": "5cba3a162965b0e2", "name": "Oak Lawn", "place_type": "city", "bounding_box": rectangle("-87.798798,41.68331 -87.720268,41.734598") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1754820, "cityName": "Oak Lawn" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476043125293056, "text": "It's slowly but surely coming to an end, day by day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 840378055, "name": "Thomas", "screen_name": "TJM_OHB", "lang": "en", "location": "VA✈️CA", "create_at": date("2012-09-22"), "description": "I've never backed down from a challenge | OHB | Snapchat: lil5ivecount | IG: tjm_ohb", "followers_count": 409, "friends_count": 87, "statues_count": 10761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado Hills, CA", "id": "ab2b3353d37b40e6", "name": "El Dorado Hills", "place_type": "city", "bounding_box": rectangle("-121.111768,38.621986 -121.027263,38.743729") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 621880, "cityName": "El Dorado Hills" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476043456761856, "text": "3/9/2016 - 02:00\nTemp: 59.5F \nHum: 98%\nWind: 3.0 mph\nBaro: 29.830in. & Rising\nRain: 0.02 in.\nhttps://t.co/aTIxgK3mTv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 109, "friends_count": 53, "statues_count": 50690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476043632918528, "text": "If you don't treat me well you can go fuck yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619082662, "name": "Natasha", "screen_name": "NatLantica", "lang": "en", "location": "Ravenswood, WV", "create_at": date("2012-06-26"), "description": "CNA. Kids. Old people. Recovery. Coffee.", "followers_count": 10134, "friends_count": 9897, "statues_count": 5911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ravenswood, WV", "id": "e71c49af0da03c17", "name": "Ravenswood", "place_type": "city", "bounding_box": rectangle("-81.777029,38.933941 -81.744035,38.965043") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54035, "countyName": "Jackson", "cityID": 5467108, "cityName": "Ravenswood" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476044119408641, "text": "1 time my friends and i wanted drama so we invited this kid who bragged ab lean so we gave him walmart grape soda and he acted fucked up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2772354229, "name": "KETAMINE DREAM", "screen_name": "XANANGEL", "lang": "en", "location": "LAS VEGAS", "create_at": date("2014-08-26"), "description": "PRESCRIPTION KITTEN", "followers_count": 424, "friends_count": 131, "statues_count": 3583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-09T00:00:07.000Z"), "id": 707476044756856834, "text": "This killed me ahahaha �� https://t.co/geMgo2QPcw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 531429485, "name": "PㄣT • ⊥ㄣԀ", "screen_name": "_PatPat28", "lang": "en", "location": "null", "create_at": date("2012-03-20"), "description": "•fuck off•", "followers_count": 296, "friends_count": 287, "statues_count": 22257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-03-09T00:00:08.000Z"), "id": 707476044958339072, "text": "Omg ur 20!!@vndromedv ily big sis even tho I'm your headache ����❤️ wish u the best day ever bc that's all you deserve https://t.co/wQZ8RwXfnv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2792653468, "name": "♡Alfa Perales♡", "screen_name": "heyitsalfaa", "lang": "en", "location": "شاهزاده خانم", "create_at": date("2014-09-29"), "description": "figure me out", "followers_count": 270, "friends_count": 206, "statues_count": 4805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-03-09T00:00:08.000Z"), "id": 707476045234962432, "text": "@DaleGodboldo @ACSFX Me Too..", "in_reply_to_status": 707416592796094464, "in_reply_to_user": 83304161, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83304161, 2966965225 }}, "user": { "id": 636586407, "name": "irma rojas", "screen_name": "irmarojas19", "lang": "en", "location": "I live in Motley Crue Land:)", "create_at": date("2012-07-15"), "description": "Huge ROCK N ROLL FAN. LOVE MOTLEY CRUE. HUGE NIKKI SIXX FAN:) LOVE SPORTS. FAV TEAM: LA LAKERS:) MUSIC AND BOOKS, AND BIG DOGS ARE MY PASSION:)", "followers_count": 920, "friends_count": 1603, "statues_count": 23664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399105912832, "text": "Diamante Hollis https://t.co/AEU85ozHZe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212119193, "name": "〽️onte.", "screen_name": "I_Glewupp", "lang": "en", "location": "Columbus, MS", "create_at": date("2013-11-24"), "description": "KNOWN ‼️. NEMCC", "followers_count": 705, "friends_count": 618, "statues_count": 16304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Booneville, MS", "id": "72ab484194c7557b", "name": "Booneville", "place_type": "city", "bounding_box": rectangle("-88.586849,34.633999 -88.526676,34.685938") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28117, "countyName": "Prentiss", "cityID": 2807780, "cityName": "Booneville" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399156195328, "text": "@jerichovil LOL KOJIMA-SAN, NOTICE ME SENPAII", "in_reply_to_status": 707838283385085954, "in_reply_to_user": 29091910, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 29091910 }}, "user": { "id": 843997759, "name": "Adam 【=◈︿◈=】", "screen_name": "BaconPancaaakes", "lang": "en", "location": "null", "create_at": date("2012-09-24"), "description": "kind of sort of an oaky afterbirth | UIUC '19", "followers_count": 198, "friends_count": 158, "statues_count": 8318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399206457344, "text": "Spectacular birthday dinner for Allison tonight with @zavertnik At @trwpdx tonight. From… https://t.co/swNrFqfZh3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6644058,45.5152359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17185447, 3621562814 }}, "user": { "id": 304133829, "name": "Chris DArcy", "screen_name": "Chris_DArcy_Art", "lang": "en", "location": "Oregon. Beyond. ", "create_at": date("2011-05-23"), "description": "Arts. Culture. Ideas. Strategies. Action.", "followers_count": 386, "friends_count": 615, "statues_count": 1755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399403532288, "text": "@smd_imviviana wait are we on the same page ������", "in_reply_to_status": 707838243845214209, "in_reply_to_user": 1123587408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1123587408 }}, "user": { "id": 2282989141, "name": "Margarita Arias", "screen_name": "_mxggie", "lang": "en", "location": "null", "create_at": date("2014-01-08"), "description": "null", "followers_count": 515, "friends_count": 390, "statues_count": 12719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399638441984, "text": "@deadamss I'm baaaaack", "in_reply_to_status": 707835625324457984, "in_reply_to_user": 2356903453, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2356903453 }}, "user": { "id": 723207950, "name": "Elizabeth Sander", "screen_name": "___eliz____", "lang": "en", "location": "null", "create_at": date("2012-07-28"), "description": "null", "followers_count": 340, "friends_count": 332, "statues_count": 2214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838399734910976, "text": "This Shit Annoying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2898030596, "name": "•Rawboi•〽️", "screen_name": "abforr3", "lang": "en", "location": "null", "create_at": date("2014-11-12"), "description": "Jabb✊4⃣", "followers_count": 202, "friends_count": 217, "statues_count": 5052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838400594751488, "text": "@Therealmaw8 text me", "in_reply_to_status": 707838298635436032, "in_reply_to_user": 3066569145, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3066569145 }}, "user": { "id": 2379197478, "name": "J O E ❗️", "screen_name": "Hotboyjoee", "lang": "en", "location": "SomewhereMakingMoney", "create_at": date("2014-03-08"), "description": "Father of a Goddess ❣| Free Chamid9️⃣", "followers_count": 722, "friends_count": 401, "statues_count": 22510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838400649256960, "text": "@laurapvera good idea, I'll just plug my headphones into my computer. bye babe, see you when I ace my test & kill this essay. All bc of you", "in_reply_to_status": 707832967951949824, "in_reply_to_user": 431294187, "favorite_count": 0, "coordinate": point("-96.33858268,30.60453807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 431294187 }}, "user": { "id": 407433031, "name": "Lacy", "screen_name": "AlexisTracy", "lang": "en", "location": "null", "create_at": date("2011-11-07"), "description": "College Station, TX // Matthew // ily Jake // Instagram: lexitracy", "followers_count": 1060, "friends_count": 388, "statues_count": 24348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838400745746432, "text": "��������this is great https://t.co/Nsm6N139HL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 996549324, "name": "⚾️Brando⚾️", "screen_name": "HBrando686", "lang": "en", "location": "null", "create_at": date("2012-12-07"), "description": "No -Rosa Parks", "followers_count": 278, "friends_count": 281, "statues_count": 5543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Grove, OR", "id": "beedf9d8e2499b64", "name": "Forest Grove", "place_type": "city", "bounding_box": rectangle("-123.15354,45.501953 -123.070256,45.542318") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4126200, "cityName": "Forest Grove" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838400863346688, "text": "@Dementia_Grimm ... you got that right.... I'll go with hot chocolate and work my way up to that pound bag of M & M's for my dark fix...", "in_reply_to_status": 707837861400330240, "in_reply_to_user": 2472800849, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2472800849 }}, "user": { "id": 91226096, "name": "Damned Demon", "screen_name": "Dave_Wilhelm", "lang": "en", "location": "..where you least expect...", "create_at": date("2009-11-19"), "description": "I'll slice you for your soul......HorrorSquadWW", "followers_count": 2387, "friends_count": 2166, "statues_count": 63195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, WA", "id": "f479e40901a48515", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-122.266398,47.257029 -122.144477,47.356233") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5303180, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838401639124993, "text": "@SFWater thanks for all you do. But ur digital payment system is confusing and indecipherable. Please simplify so we can easily pay online!", "in_reply_to_status": -1, "in_reply_to_user": 23103448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23103448 }}, "user": { "id": 136030721, "name": "R y ∧ n Clifford", "screen_name": "ryantclifford", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-04-22"), "description": "Run on digital, marketing, biking, SF, NYC, inventive multimedia, my dog, kid, dining, exploring, friends & YOU, Have fun. @Atlassian former @Yahoo", "followers_count": 974, "friends_count": 425, "statues_count": 3201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838401760858112, "text": "high af ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43768003, "name": "97 ☾", "screen_name": "fatbootybella", "lang": "en", "location": "Spice World", "create_at": date("2009-05-31"), "description": "yez the booty fat. haters gone hate, potatoes gone potate • tomboy Spice girl • amateur photographer/ filmographer • #ssu • savages only • StunnaHD photography", "followers_count": 2076, "friends_count": 498, "statues_count": 177182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838401861406721, "text": "felt this in my soul https://t.co/ru9eAcKkbq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4659532163, "name": "Jaide", "screen_name": "jayangel___", "lang": "en", "location": "null", "create_at": date("2015-12-30"), "description": "a kid with a kid. new member of NappyTwitter ✊", "followers_count": 180, "friends_count": 186, "statues_count": 5313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838401920143360, "text": "#MWMadness: @UNLVLadyRebels @aley_bo_bally and @Dollbaby_duo (Kota) join @CoachKO in the postgame presser. #UNLVWBB https://t.co/xhgMXLaGRo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MWMadness", "UNLVWBB" }}, "user_mentions": {{ 861798690, 311855099, 728951072, 36729777 }}, "user": { "id": 452411446, "name": "W.G. Ramirez", "screen_name": "WillieGRamirez", "lang": "en", "location": "www.perfictlifestyle.com", "create_at": date("2012-01-01"), "description": "God • Jordin • Loyalty • So. Nevada sports correspondent for @AP_Sports; scrub sports writer for others • Co-Founder @perFIcTlifestyl • #TeamPerFIcT", "followers_count": 2631, "friends_count": 1122, "statues_count": 90536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838401958047744, "text": "I'm stubborn asf i can stay mad for a long time.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2851056080, "name": "SC : Jalesiiaaa", "screen_name": "Only1jalesia_", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-10-10"), "description": "A Girl That' Smiles Through It All Deserves A Crown.", "followers_count": 400, "friends_count": 326, "statues_count": 3218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838402012393472, "text": "00:00", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 428662053, "name": "Chicharo", "screen_name": "JuanMoreTime69", "lang": "en", "location": "burro, Oregon ", "create_at": date("2011-12-04"), "description": "#DareToYuki", "followers_count": 513, "friends_count": 160, "statues_count": 16645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838402020782080, "text": "@SeeDisD literally laid on the floor for a good two minutes contemplating what my life has come to.\nAnd I'm STILL not asleep #whydoidothis", "in_reply_to_status": 707838018674040832, "in_reply_to_user": 2691614466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "whydoidothis" }}, "user_mentions": {{ 2691614466 }}, "user": { "id": 1406443730, "name": "angie", "screen_name": "jaws0flife", "lang": "en", "location": "Subtweet Nation", "create_at": date("2013-05-05"), "description": "UT 2017. ΣφΩ SP14.", "followers_count": 348, "friends_count": 296, "statues_count": 5253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838402226294788, "text": "Craving a 4 day weekend so might as well take Friday and Monday off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1199207352, "name": "MarHernandez", "screen_name": "mvreuh", "lang": "en", "location": "null", "create_at": date("2013-02-19"), "description": "bclhs '16. Matthew 7:7 ✨", "followers_count": 910, "friends_count": 772, "statues_count": 38062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838402654294018, "text": "Wish I hadn't eaten 27 mini Reese's cups and drank a large Gatorade at midnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45983621, "name": "Kayla Farris", "screen_name": "kkaayyyyy_", "lang": "en", "location": "null", "create_at": date("2009-06-09"), "description": "I'd rather be in Haiti | http://mycalltoloveblog.wordpress.com", "followers_count": 778, "friends_count": 354, "statues_count": 17236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fruit Cove, FL", "id": "439beb879c4ddecc", "name": "Fruit Cove", "place_type": "city", "bounding_box": rectangle("-81.670267,30.054866 -81.55475,30.133637") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12109, "countyName": "St. Johns", "cityID": 1224925, "cityName": "Fruit Cove" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838402926870529, "text": "these girls are smart G-Stack these girls are smart", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 474481885, "name": "Childish Globino", "screen_name": "DopeeMoneyy96", "lang": "en", "location": "The New World", "create_at": date("2012-01-25"), "description": "Simba", "followers_count": 581, "friends_count": 343, "statues_count": 39206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabeth, NJ", "id": "b74cebcb62a1a686", "name": "Elizabeth", "place_type": "city", "bounding_box": rectangle("-74.254211,40.634285 -74.138838,40.690673") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3421000, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838403044204544, "text": "@MissTreglia ��", "in_reply_to_status": 707739196421574656, "in_reply_to_user": 226061642, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 226061642 }}, "user": { "id": 92730370, "name": "R.J. Knapp", "screen_name": "naR__J__Kapp", "lang": "en", "location": "California", "create_at": date("2009-11-26"), "description": "Nothing here to see, just a kid like me, trying to cuss and see, trying to figure it out", "followers_count": 164, "friends_count": 107, "statues_count": 3045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-03-10T00:00:00.000Z"), "id": 707838403082063872, "text": "I told nick to stop in college station on his way to Houston to rest for the night but he kept driving and ended up wrecking his car :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 494298173, "name": "vs angel", "screen_name": "Gaberzzzzz", "lang": "en", "location": "Houston/College Station, TX", "create_at": date("2012-02-16"), "description": "in love with God and myself", "followers_count": 803, "friends_count": 637, "statues_count": 70572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838403912409088, "text": "Billy Fishkins from my math class asked me out to pizza and to play baseball tomorrow. Witch yo ol fashioned cute ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46587715, "name": "tequilianna", "screen_name": "kieaunnacooley", "lang": "en", "location": "null", "create_at": date("2009-06-11"), "description": "in a relationship w @LVAC", "followers_count": 156, "friends_count": 78, "statues_count": 3093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838404113915904, "text": "Wind 2.8 mph WNW. Barometer 29.909 in, Rising. Temperature 56.0 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838404248150016, "text": "God is Great! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42518039, "name": "Nick Belfort", "screen_name": "NickTrillions", "lang": "en", "location": "Ohio, USA", "create_at": date("2009-05-25"), "description": "God First | Multi-Billionaire | KSU Engineer", "followers_count": 849, "friends_count": 370, "statues_count": 9835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, OH", "id": "45a0ea3329c38f9f", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-81.392451,41.11766 -81.308418,41.202841") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39133, "countyName": "Portage", "cityID": 3939872, "cityName": "Kent" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838404403265537, "text": "https://t.co/WGuX9pET8e������this is ��������..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 38414735, "name": "Sahara Aziz", "screen_name": "tumighty", "lang": "en", "location": "Oakland Tha Bay Midwest Atl AZ", "create_at": date("2009-05-07"), "description": "I am a strong immeasurable limitless beautiful Black ♀ mother of 2.Preschool teacher, activist, ghostwriter. Owner of Square Bizz Cleaning...#BlackMafia", "followers_count": 2130, "friends_count": 4383, "statues_count": 4469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838404550000640, "text": "https://t.co/WGuX9pWtZM������this is ��������..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 38414735, "name": "Sahara Aziz", "screen_name": "tumighty", "lang": "en", "location": "Oakland Tha Bay Midwest Atl AZ", "create_at": date("2009-05-07"), "description": "I am a strong immeasurable limitless beautiful Black ♀ mother of 2.Preschool teacher, activist, ghostwriter. Owner of Square Bizz Cleaning...#BlackMafia", "followers_count": 2130, "friends_count": 4383, "statues_count": 4469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838404625604608, "text": "#SupportOriginMelissa 66.9°F Wind:4.3mph Pressure: 30.03hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 312024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838405133078528, "text": "03/10@03:00 - Temp 57.8F, WC 57.8F. Wind 2.6mph SW, Gust 4.0mph. Bar 30.049in, Falling slowly. Rain 0.00in. Hum 73%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838405296558080, "text": "2:00 am. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 132634628, "name": "bwa.", "screen_name": "_Goldddie", "lang": "en", "location": "Little Rock, AR", "create_at": date("2010-04-13"), "description": "2⃣1⃣ year old money chaser, #GTT〽️. RIL GrannyGranny. #LongLiveTisha | #LongLiveHoward | #LongLiveTreQuis | #LongLiveDarnelleBandz. ❤ -snapchat // alexthomass", "followers_count": 1966, "friends_count": 1485, "statues_count": 124069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838405313306624, "text": "\"show me all of your tattoos\" hours.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268137295, "name": "FiLTH GOD", "screen_name": "inHERribs", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-03-17"), "description": "people have sex, record it & send it to me. I'm not sure why. • contact: iFiLTHGOD@gmail.com", "followers_count": 141507, "friends_count": 313, "statues_count": 98803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838405678268416, "text": "\"I CAN STAND NO MATTER WHAT I'M FACING, CARRY ON\"!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.31968973,34.26007828"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27860174, "name": "carol wagner", "screen_name": "carolmwagner", "lang": "en", "location": "California", "create_at": date("2009-03-31"), "description": "I Love Jesus!", "followers_count": 398, "friends_count": 353, "statues_count": 64682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406030548992, "text": "Wind 0.7 mph W. Barometer 29.99 in, Steady. Temperature 32.7 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 40635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406206816257, "text": "Ripley SW Limestone Co. Temp: 68.4°F Wind:13.0mph Pressure: 996.6mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 50033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406277996544, "text": "s/o to @xxReagannxx bc if she hadn't got me hooked on oth idk what i would be doing this week����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2203591790 }}, "user": { "id": 2856054588, "name": "madison!!", "screen_name": "madisons2345", "lang": "en", "location": "Forney, TX", "create_at": date("2014-10-14"), "description": "spirit lead me where my trust is without borders | nfhs'18 | texas a&m |", "followers_count": 331, "friends_count": 214, "statues_count": 2206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406424862721, "text": "@ebbtideapp Tide in Bon Secour, Alabama 03/10/2016\n Low 7:34am 0.4\nHigh 2:16pm 1.0\n Low 8:00pm 0.3\nHigh 3:16am 0.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-87.7333,30.3"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 41, "friends_count": 1, "statues_count": 14554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 132272, "cityName": "Gulf Shores" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406663884801, "text": "@suuunkissd I think I have bloop#d pressure problems", "in_reply_to_status": 707838241257295872, "in_reply_to_user": 408250313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 408250313 }}, "user": { "id": 53577121, "name": "EsteVan", "screen_name": "MeaningTheCrown", "lang": "en", "location": "9¾ ", "create_at": date("2009-07-03"), "description": "A little bit of a dreamer held down by the little bit of a realist he is. @suunkissd", "followers_count": 543, "friends_count": 290, "statues_count": 82156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838406710067200, "text": "¿Cómo se sentiría el saber que todos los 10 de cada mes tus gastos y mantencion del hogar ya estén cubiertos 100%? ¿No sería bonito? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 154648796, "name": "Milton Pacas", "screen_name": "miltonpacas", "lang": "en", "location": "Buena Park, CA", "create_at": date("2010-06-11"), "description": "Emprendedor de Corazon , Padre y Esposo Estudiante y Coach de la Ciencia y el Arte de Liderazgo, Rebelde con Causa, Mentor para unos y Amigo Para Todos.", "followers_count": 125, "friends_count": 112, "statues_count": 507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838407251255296, "text": "I remember sleeping in them stolos pulling all nighters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3628233856, "name": "l a ' f l y y 1 k", "screen_name": "Lafly1K", "lang": "en", "location": "state to stateee w/ kiaaa....", "create_at": date("2015-09-11"), "description": "best thing I ever did was find another plug ❗️ | free da guys ❤️| worst thing they ever did was kill my la thugs . . . rest up doodoo & dre", "followers_count": 826, "friends_count": 427, "statues_count": 4644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Orange, FL", "id": "018244f41aa15ac3", "name": "Port Orange", "place_type": "city", "bounding_box": rectangle("-81.078928,29.027688 -80.967617,29.156983") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1258575, "cityName": "Port Orange" } }
+{ "create_at": datetime("2016-03-10T00:00:01.000Z"), "id": 707838407364505601, "text": "not sleepy..��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750367861, "name": "lol...", "screen_name": "_AyannaD", "lang": "en", "location": "null", "create_at": date("2012-08-10"), "description": "Snapchat; ayanna_duncan", "followers_count": 747, "friends_count": 515, "statues_count": 37079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838407788072960, "text": "@JoshWillisUPH hahahaha!!!!", "in_reply_to_status": 707804385460543488, "in_reply_to_user": 250994481, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 250994481 }}, "user": { "id": 430775605, "name": "Markus Eve To Adam", "screen_name": "MarkusDrive", "lang": "en", "location": "Queens, NY", "create_at": date("2011-12-07"), "description": "Guitar/Vox at Eve To Adam. Start a fire, start a war, stand up for what you believe in.", "followers_count": 752, "friends_count": 355, "statues_count": 1835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838408069148673, "text": "Wind 1.0 mph NW. Barometer 29.989 in, Rising. Temperature 55.8 °F. Rain today 0.04 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838408371150848, "text": "Тем временем ,так как мы не работаем больше с предыдущим промоутером,мы начали работать с другим,пришли в клуб-а там все уже в курсе что мы", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 114810204, "name": "Irina", "screen_name": "someone__new", "lang": "en", "location": "New York City (born in Russia)", "create_at": date("2010-02-16"), "description": "They said this day wouldn't come we refuse to run ☼ @jtimberlake , @thewanted, @OneRepublic make my life complete", "followers_count": 1014, "friends_count": 158, "statues_count": 128166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838408501014528, "text": "Wind 0.0 mph SSE. Barometer 29.997 in, Falling slowly. Temperature 58.3 °F. Rain today 0.03 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838409285316609, "text": "@Boys_AHS_Soccer shout out @maxkrepple", "in_reply_to_status": 707773848435822592, "in_reply_to_user": 4703424103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4703424103, 1120704930 }}, "user": { "id": 99015502, "name": "Otilio Guerrero", "screen_name": "tillyplz", "lang": "en", "location": "Austin, TX", "create_at": date("2009-12-23"), "description": "I'm old Gregg Rip Ezra", "followers_count": 527, "friends_count": 417, "statues_count": 15779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838409755267072, "text": "Temp: 67.9°F Wind:2.6mph Pressure: 29.963hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838409973170176, "text": "70.0F (Feels: 70.0F) - Humidity: 97% - Wind: 10.7mph SE - Gust: 14.5mph - Pressure: 1009.3mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 230039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410027769856, "text": "Help I'm hyper�� It's 1am...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 859134230, "name": "jøhnmøffitt", "screen_name": "johnmoffitt29", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2012-10-02"), "description": "They call me Peanut Butter // ΒΘΠ", "followers_count": 367, "friends_count": 268, "statues_count": 1338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410237550592, "text": "Damn, last episode of Californication I'm tight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 106532972, "name": "Mango", "screen_name": "kingwzrddd", "lang": "en", "location": "null", "create_at": date("2010-01-19"), "description": "47", "followers_count": 245, "friends_count": 223, "statues_count": 7753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410493267968, "text": "Rain today. Still, almost 12 degrees above average. See you 4-7 am..(gosh that's early) WTHRSunrise. @ChuckWTHR https://t.co/LZDrGGZ1YS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.15650321,39.76690667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347160044 }}, "user": { "id": 62169540, "name": "Indianapolis CP", "screen_name": "IndianapolisCP", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2009-08-01"), "description": "Indianapolis City And Press", "followers_count": 1859, "friends_count": 3, "statues_count": 222334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianapolis, IN", "id": "018929347840059e", "name": "Indianapolis", "place_type": "city", "bounding_box": rectangle("-86.348441,39.631677 -85.937404,39.927448") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1836003, "cityName": "Indianapolis city (balance)" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410833076224, "text": "@alohaakytiaa *sends wisdom care package*", "in_reply_to_status": 707838203936387073, "in_reply_to_user": 2462811024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2462811024 }}, "user": { "id": 197719120, "name": "❁ wizzzzz ❁", "screen_name": "baybeewiz", "lang": "en", "location": "• CA ↣ AZ • ☼☾☪☯ϟ❁", "create_at": date("2010-10-02"), "description": "Wis∂σm. 19. ♛. gcu cℓasss of 2019. ♡. #†яackizℓyғe. #carliewilsonforever rest easy baby girl ♡.", "followers_count": 1214, "friends_count": 743, "statues_count": 59406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410967351296, "text": "Wind 0.0 mph W. Barometer 29.98 in, Steady. Temperature 60.4 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 116, "statues_count": 158133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838410971426816, "text": "I'm always hungry this late at night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542900266, "name": "Melissa", "screen_name": "_xmelissa16", "lang": "en", "location": "D a l l a s", "create_at": date("2012-04-01"), "description": "texas state university", "followers_count": 1136, "friends_count": 759, "statues_count": 20087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838411080540160, "text": "I want a thousand Twitter followers by the end of next year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2226042074, "name": "bhavez", "screen_name": "RomannnChavez", "lang": "en", "location": "most likely here, on twitter", "create_at": date("2013-12-01"), "description": "bahs | Don't take my tweets seriously, it's the Internet", "followers_count": 574, "friends_count": 555, "statues_count": 10760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-03-10T00:00:02.000Z"), "id": 707838411248283648, "text": "I'm bored ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 857895900, "name": "Beelo.", "screen_name": "_Beelo", "lang": "en", "location": "null", "create_at": date("2012-10-02"), "description": "Heav,Kellz,&Jay❤️ Sc:bbyronnnnn", "followers_count": 1226, "friends_count": 2204, "statues_count": 13366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodmere, LA", "id": "25041b89e1b96118", "name": "Woodmere", "place_type": "city", "bounding_box": rectangle("-90.089118,29.838214 -90.066639,29.874275") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2283002, "cityName": "Woodmere" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838411760115712, "text": "Spring around the Penthouse. @ Aubrey's Palatial Penthouse https://t.co/trvFQ5U8Hv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.40258853,37.7957193"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1709019582, "name": "Aubrey Brewster", "screen_name": "AubreyBrewster", "lang": "en", "location": "SF | London | Paris | NoLa", "create_at": date("2013-08-28"), "description": "Aubrey is a product of his enviroment. Allow Aubrey be your guide to fashion, entertaining, recipes, and socialite lifestyle coloured with a taste of satire.", "followers_count": 80, "friends_count": 144, "statues_count": 603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838412389126145, "text": "My mama told me that I was different the moment I was invented Estranged baby, no I'm not ashamed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585351607, "name": "Marty", "screen_name": "Dreamingst_", "lang": "en", "location": "Lewisville, TX", "create_at": date("2012-05-19"), "description": "The presence of me is the same as it leaves.", "followers_count": 376, "friends_count": 289, "statues_count": 15347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838412401872896, "text": "Check out my 2nd article! @EBHSprinc @debra_gulick Integrating Tech: Primary Source Analysis using Google Forms https://t.co/V5pmAVMstF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 923503405, 3110693452 }}, "user": { "id": 2153272486, "name": "Ms. Grotrian", "screen_name": "MsGrotrian", "lang": "en", "location": "New Jersey, USA", "create_at": date("2013-10-25"), "description": "Social Studies Teacher at East Brunswick High School, Avid Activity Designer, Professional Development Enthusiast https://t.co/LCy2qTNlJL #VBQs #EdPrepChat", "followers_count": 671, "friends_count": 818, "statues_count": 3528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englishtown, NJ", "id": "00fc227e25eae376", "name": "Englishtown", "place_type": "city", "bounding_box": rectangle("-74.379277,40.286363 -74.35297,40.31666") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3421570, "cityName": "Englishtown" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838412531720192, "text": "@allipaige I neeeed.", "in_reply_to_status": 707453177868771332, "in_reply_to_user": 15099250, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15099250 }}, "user": { "id": 35752797, "name": "Kyle Hulett", "screen_name": "KyleHvlett", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-27"), "description": "Music Social Marketer at @TotalAssault. Community Director for @flippenmusic. Espresso & Aesthetics. #ItsAllHappening", "followers_count": 9860, "friends_count": 757, "statues_count": 42741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838412930359296, "text": "@HillaryClinton @BRios82 YES HE DID!!!! So quit with your lies #Bernie", "in_reply_to_status": 707758877253767168, "in_reply_to_user": 1339835893, "favorite_count": 0, "coordinate": point("-97.5272341,35.48361118"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Bernie" }}, "user_mentions": {{ 1339835893, 117983479 }}, "user": { "id": 981050484, "name": "Brenda Hawkins", "screen_name": "iheartwestbrook", "lang": "en", "location": "null", "create_at": date("2012-11-30"), "description": "Retired Sp. Ed. Teacher, Special Olympics coach, CASA, MADD, O-State Alum, rancher, mom, and full-time Thunder fan!!! Whatever you are, be a good one", "followers_count": 406, "friends_count": 360, "statues_count": 27985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838413001478144, "text": "@mewingwang happy birthday eat it yeaaa!!!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": 2565458206, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2565458206 }}, "user": { "id": 3243921962, "name": "Chris spades", "screen_name": "cjsantos322", "lang": "en", "location": "null", "create_at": date("2015-06-12"), "description": "I'm a small youtuber that loves professional wrestling,gaming,creepy stories,dogs and stuff that glow in the dark I'm a Nintendo guy & I am also (straight edge)", "followers_count": 48, "friends_count": 391, "statues_count": 444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838414322663424, "text": "@jaredcw15 that was a metaphor first off and I took government freshman year and I got an A.", "in_reply_to_status": 707837982275862528, "in_reply_to_user": 756732578, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 756732578 }}, "user": { "id": 591305599, "name": "P£T£R", "screen_name": "PeteyTerrazas", "lang": "en", "location": "California, USA", "create_at": date("2012-05-26"), "description": "5'10 G for Rancho Elite #StriveForGreatness. I'm missing you and it's Killing me.", "followers_count": 173, "friends_count": 208, "statues_count": 4287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838414607884288, "text": "\"Teach me to do your will, for you are my God. \nMay your gracious spirit lead me on higher ground.\"\nPsalm 143:10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 140, "friends_count": 258, "statues_count": 225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838415077646336, "text": "@ForeverIrish_ let me see it", "in_reply_to_status": 707838030422216704, "in_reply_to_user": 3169065486, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3169065486 }}, "user": { "id": 2915045132, "name": "Wolf Of All Streets", "screen_name": "Brob2times", "lang": "en", "location": "Boom Town ", "create_at": date("2014-11-30"), "description": "#PTnationn #youafoolchallenge #PTFL run with us or get ran over", "followers_count": 844, "friends_count": 602, "statues_count": 19640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-10T00:00:03.000Z"), "id": 707838415459385345, "text": "b7f4f81000pE21DF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.02684698,30.77035461"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 11, "friends_count": 0, "statues_count": 4413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13101, "countyName": "Echols" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838416302448640, "text": "@drewny1995 im not sure, I can ask my dad tomorrow he does it all the time.", "in_reply_to_status": 707838157111230464, "in_reply_to_user": 1059891722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1059891722 }}, "user": { "id": 919341854, "name": "jarred", "screen_name": "JayyRaww12", "lang": "en", "location": "California, USA", "create_at": date("2012-11-01"), "description": "College Kid", "followers_count": 351, "friends_count": 211, "statues_count": 17626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838416625381377, "text": "\"They have repu's in Hawaii? @politico: #Breaking: Donald Trump wins the Hawaii Republican caucuses https://t.co/OthqDYs9mT | AP Photo\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Breaking" }}, "user_mentions": {{ 9300262 }}, "user": { "id": 3590383037, "name": "Catherine", "screen_name": "acdeece1", "lang": "en", "location": "Washington, USA", "create_at": date("2015-09-08"), "description": "Journalist/Author/90's SEA/LA music biz, neuro researcher, EMT, med therapist, political prog activist, amateur comedian, Mom, trauma survivor, busy person...", "followers_count": 40, "friends_count": 179, "statues_count": 946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838416688451584, "text": "#ResidentialAlarm at Aztec Rose Ln. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2686908,28.3850954"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ResidentialAlarm", "orlpol", "opd" }}, "user": { "id": 39057416, "name": "Police Calls 32827", "screen_name": "orlpol32827", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 289, "friends_count": 1, "statues_count": 32300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838417015541760, "text": "This nigga talking my ear off tf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3165523371, "name": "mom✨", "screen_name": "anasjahere", "lang": "en", "location": "New York, USA", "create_at": date("2015-04-13"), "description": "Romans 12:12", "followers_count": 1886, "friends_count": 857, "statues_count": 34785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838417116106754, "text": "Couldn't even tell ya how blessed iam. https://t.co/zBSls9VuEa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 604726747, "name": "A$AP", "screen_name": "rockyhanson1", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "Never question greatness", "followers_count": 966, "friends_count": 1389, "statues_count": 6893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson City, NV", "id": "f2390622c570faf1", "name": "Carson City", "place_type": "city", "bounding_box": rectangle("-119.814173,39.08497 -119.670954,39.216676") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32510, "countyName": "Carson City", "cityID": 3209700, "cityName": "Carson City" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838418114387968, "text": "Wow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1881311966, "name": "⚡Bam Bam⚡", "screen_name": "VarelaNoah", "lang": "en", "location": "LBC✈AZ", "create_at": date("2013-09-18"), "description": "Higley Football", "followers_count": 618, "friends_count": 409, "statues_count": 6833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queen Creek, AZ", "id": "01cb573821d94344", "name": "Queen Creek", "place_type": "city", "bounding_box": rectangle("-111.686314,33.196614 -111.582748,33.288127") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 458150, "cityName": "Queen Creek" } }
+{ "create_at": datetime("2016-03-10T00:00:04.000Z"), "id": 707838419104301056, "text": "this is why I hate the rap game now https://t.co/SBbG9Ir7Fv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456054352, "name": "walking licc.", "screen_name": "svrfthegod", "lang": "en", "location": "Richmond, VA", "create_at": date("2012-01-05"), "description": "ig: Lxrd_Svrf", "followers_count": 486, "friends_count": 406, "statues_count": 24932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suffolk, VA", "id": "6e0ed207f0c2d36b", "name": "Suffolk", "place_type": "city", "bounding_box": rectangle("-76.668194,36.698303 -76.402862,36.92173") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51800, "countyName": "Suffolk", "cityID": 5176432, "cityName": "Suffolk" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838420320522240, "text": "@PrincesssVivian you have a bae now ��", "in_reply_to_status": 707838096688095232, "in_reply_to_user": 362217669, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 362217669 }}, "user": { "id": 1156336916, "name": "Juan Orozco", "screen_name": "juan_juanorozco", "lang": "en", "location": "null", "create_at": date("2013-02-06"), "description": "null", "followers_count": 235, "friends_count": 200, "statues_count": 3459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838420823838720, "text": "Lol what? https://t.co/jnRHLCt5qS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265597807, "name": "Eligwe 2x", "screen_name": "SNigerianPrince", "lang": "en", "location": "Fort Worth Tx Eastside 817 ", "create_at": date("2011-03-13"), "description": "Tarleton state university God always first", "followers_count": 3257, "friends_count": 2660, "statues_count": 53616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838420840656896, "text": "@thisizcoco interracial relationship probs lol", "in_reply_to_status": 707837950831144961, "in_reply_to_user": 80905095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80905095 }}, "user": { "id": 235808287, "name": "shai'-ann", "screen_name": "littlepinkkbow", "lang": "en", "location": "Here.There.Everywhere ", "create_at": date("2011-01-08"), "description": "There's no one I'd rather be, than me.", "followers_count": 400, "friends_count": 370, "statues_count": 26246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838422002618368, "text": "Can you find Tallahassee on the map? Just try it at https://t.co/FZIUEAsQtu #Tallahassee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.2807,30.4383"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tallahassee" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2641452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838422212349952, "text": "When the �� hits your �� like a big ��, that's amore!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.48224,37.796741"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226550231, "name": "Amore Explainer", "screen_name": "amore_explainer", "lang": "en", "location": "Naples, Campania", "create_at": date("2015-05-01"), "description": "Heard about amore, but unsure what it is? Seguimi! // f=5 hours // made by @andrewbadr", "followers_count": 22, "friends_count": 31, "statues_count": 1570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838423084769280, "text": "@kausmickey @ThePlumLineGS GOP race is Millers Crossing. Rubio is Turturro. Trump is Polito. Cruz is Gabriel Byrne.", "in_reply_to_status": 707806900151308289, "in_reply_to_user": 30534986, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30534986, 20508720 }}, "user": { "id": 2936965923, "name": "Rob Yarbrough", "screen_name": "RobertKYarbro", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "“Let me have my own way in exactly everything and a sunnier and pleasanter creature does not exist.”\n― Thomas Carlyle", "followers_count": 58, "friends_count": 218, "statues_count": 2927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manassas, VA", "id": "1f703799b3940c20", "name": "Manassas", "place_type": "city", "bounding_box": rectangle("-77.526464,38.715315 -77.447589,38.7814") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51683, "countyName": "Manassas", "cityID": 5148952, "cityName": "Manassas" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838423197831168, "text": "Me rn in this library https://t.co/jMjFqyQdaX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1081765278, "name": "Blacksbi Hendrix", "screen_name": "MattWakhu", "lang": "en", "location": "Denton, TX", "create_at": date("2013-01-11"), "description": "Matt Wakhu | Future GentlΣmaΝ | Mr. UNT | Kenyan | Follow my music page @MattWakhuMusic!", "followers_count": 3419, "friends_count": 2267, "statues_count": 120628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838423214596096, "text": "@SNigerianPrince if u over 18 u can get it that simple", "in_reply_to_status": 707838221644791808, "in_reply_to_user": 265597807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 265597807 }}, "user": { "id": 548921445, "name": "Tay", "screen_name": "kid_Lante5", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-04-08"), "description": "Sc LanteGlizzy , Ig LanteGlizzy_", "followers_count": 2136, "friends_count": 2070, "statues_count": 45041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-10T00:00:05.000Z"), "id": 707838423776632832, "text": "Gotta double check my backpack for drugs before I go to the airport, college problems.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3384190558, "name": "Rob Goerss", "screen_name": "RGoerss", "lang": "en", "location": "San Mateo, CA", "create_at": date("2015-07-19"), "description": "Cali ✈️ Utah UVU", "followers_count": 80, "friends_count": 145, "statues_count": 447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838424397426688, "text": "Blessings on blessings on blessings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2942367258, "name": "tatiana☀", "screen_name": "tatianadshai_", "lang": "en", "location": "Monroe, Tx", "create_at": date("2014-12-24"), "description": "honestly, im very disrespectful", "followers_count": 735, "friends_count": 399, "statues_count": 27489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Angleton, TX", "id": "fe6fdbea8b8a1476", "name": "Angleton", "place_type": "city", "bounding_box": rectangle("-95.454639,29.10696 -95.390443,29.205233") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4803264, "cityName": "Angleton" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838424539987968, "text": "Jordan Clarkson to Sacramento ? ���� https://t.co/qoD1tBtSnK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33339701, "name": "Keith Jouganatos", "screen_name": "KeithJouganatos", "lang": "en", "location": "Sacramento, CA ya dig", "create_at": date("2009-04-19"), "description": "22 years young. College Journalism Major, Kicx TV reporter 'Bout that action boss", "followers_count": 1432, "friends_count": 589, "statues_count": 20315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838425160880128, "text": "wth this girl talkin bout...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53000782, "name": "G® From OP‼️", "screen_name": "JG_Trilll", "lang": "en", "location": "Opelousas☀️", "create_at": date("2009-07-02"), "description": "been jiggy, been pretty | YWA | #LSU19 ❌ Promoter ❌ Engineer ❌ #BootUp", "followers_count": 2985, "friends_count": 2155, "statues_count": 48554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838425655672832, "text": "Temp: 42.8°F - Dew Point: 38.9° - Wind: 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 29.99in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838426469376000, "text": "������ https://t.co/FuRYDdANlk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 258565314, "name": "abbbbs", "screen_name": "abbyxx0", "lang": "en", "location": "bronx", "create_at": date("2011-02-27"), "description": "hunter college", "followers_count": 1019, "friends_count": 800, "statues_count": 28043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838426507116544, "text": "I feel you �� https://t.co/DsJaMl0dvn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184171238, "name": "champagnekate", "screen_name": "katieayala16", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "null", "followers_count": 156, "friends_count": 180, "statues_count": 701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altadena, CA", "id": "fb76b3aa366004c9", "name": "Altadena", "place_type": "city", "bounding_box": rectangle("-118.177296,34.167539 -118.095091,34.220078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601290, "cityName": "Altadena" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838426868011009, "text": "The staples show was amazing despite not having my best friends there @BlackbeardTCSW @TOTEMROMULUS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 799018027, 378500952 }}, "user": { "id": 1915035577, "name": "Eclipso Amore", "screen_name": "EclipsoForever", "lang": "en", "location": "Dayton, OH", "create_at": date("2013-09-28"), "description": "It's a New Day& I'm a new Gabe", "followers_count": 379, "friends_count": 307, "statues_count": 27085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838427044118529, "text": "MOBBIN' @ Downtown Cincinnati https://t.co/U6rbRlMVhc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51666667,39.1"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368772740, "name": "BiG ChinGS", "screen_name": "BiGChinGSdotCOM", "lang": "en", "location": "Las Cruces NM", "create_at": date("2011-09-05"), "description": "LISTEN TO MY MUZIK!!!!!!!", "followers_count": 843, "friends_count": 1028, "statues_count": 1626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838427165667328, "text": "Taken by the best�� @_theolang_ https://t.co/nvSsEbaAxl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3254108635 }}, "user": { "id": 3473185392, "name": "lynn ash", "screen_name": "ashlynnnoble", "lang": "en", "location": "null", "create_at": date("2015-09-06"), "description": "2•27•16\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nTheo ♡", "followers_count": 149, "friends_count": 119, "statues_count": 2769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lamont, CA", "id": "ee121100f5e9087b", "name": "Lamont", "place_type": "city", "bounding_box": rectangle("-118.932111,35.234117 -118.894307,35.310691") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 640088, "cityName": "Lamont" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838427631202304, "text": "its like every time iwalk into Eli's house iautomatically get high . \n����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1147546562, "name": "_angeladee. ♡ ∞ ✞", "screen_name": "itss_dee3", "lang": "en", "location": "null", "create_at": date("2013-02-04"), "description": "♪ | h-town ' 96 | †", "followers_count": 172, "friends_count": 186, "statues_count": 230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838427719450624, "text": "���������� �������� #Afterwork_Wednesdays @ The New Esavoy Lounge https://t.co/OkZOrx3aRA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.93043,40.61654"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Afterwork_Wednesdays" }}, "user": { "id": 4341345801, "name": "SuckerFreeWes", "screen_name": "WesFromCI", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2015-12-01"), "description": "Handsome Musician trapped in Ugly Promoter's Body #VMNation • Sunday Brunch/Vivid Cafe 11am - 4pm • brooklynsouth@gmail.com", "followers_count": 521, "friends_count": 627, "statues_count": 8711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838427824300032, "text": "@Salembabie please inform youself https://t.co/VvBHK5itYn", "in_reply_to_status": 707837485586321409, "in_reply_to_user": 19317371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19317371 }}, "user": { "id": 365770118, "name": "maria r munoz", "screen_name": "mrserranom", "lang": "en", "location": "Plano, Texas", "create_at": date("2011-08-31"), "description": "Latina legally inmigrated 2 US,proud American,Capitalist,freedom lover,hate political correctness,communism/progressivism/socialism,media bias,& rude people", "followers_count": 222, "friends_count": 402, "statues_count": 2898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838428193234944, "text": "3/10/2016 - 02:00\nTemp: 59.7F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.867in. & Falling\nRain: 0.18 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 50720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-03-10T00:00:06.000Z"), "id": 707838428197486592, "text": "Wind 0.0 mph ---. Barometer 29.998 in, Falling. Temperature 56.5 °F. Rain today 0.06 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 385, "friends_count": 292, "statues_count": 8892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-10T00:00:07.000Z"), "id": 707838428734312448, "text": "night #2 of @NatalieMerchant's #soldout #paradiseistheretour at the lovely #orpheumtheatre @LAOrpheum #losangeles https://t.co/8hWZcpk9kH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "soldout", "paradiseistheretour", "orpheumtheatre", "losangeles" }}, "user_mentions": {{ 126389931, 92401690 }}, "user": { "id": 29287908, "name": "Justin Higuchi", "screen_name": "jus10h", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-06"), "description": "professional social media music photographer... it's a thing. email: justin.higuchi (at) gmail (dot) com", "followers_count": 444, "friends_count": 404, "statues_count": 6671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-10T00:00:07.000Z"), "id": 707838429363449856, "text": "Fuck a vacay I feel better at work.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21321925, "name": "e.", "screen_name": "EO7", "lang": "en", "location": "null", "create_at": date("2009-02-19"), "description": "life enthusiast #ripJMAC LA➡️Tha Bay", "followers_count": 353, "friends_count": 282, "statues_count": 13800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-03-10T00:00:07.000Z"), "id": 707838429799776256, "text": "@andradaymusic @ 3rd & Lindsley Nashville https://t.co/4A3yvoeI69", "in_reply_to_status": -1, "in_reply_to_user": 815537202, "favorite_count": 0, "coordinate": point("-86.77006899,36.15225801"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 815537202 }}, "user": { "id": 351920308, "name": "Cecil Withrow", "screen_name": "CecilWithrow", "lang": "en", "location": "Nashville, TN", "create_at": date("2011-08-09"), "description": "Artist", "followers_count": 32, "friends_count": 43, "statues_count": 494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashville, TN", "id": "00ab941b685334e3", "name": "Nashville", "place_type": "city", "bounding_box": rectangle("-87.022482,35.994568 -86.560616,36.405448") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47037, "countyName": "Davidson", "cityID": 4752006, "cityName": "Nashville-Davidson metropolitan government (balance)" } }
+{ "create_at": datetime("2016-03-10T00:00:07.000Z"), "id": 707838431364251648, "text": "When you just get in bed and have to pee. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54624185, "name": "IG: emmanuel.navedo", "screen_name": "Emmanuelsogr8", "lang": "en", "location": "| Miami |", "create_at": date("2009-07-07"), "description": "live long and prosper", "followers_count": 278, "friends_count": 255, "statues_count": 4546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-03-10T00:00:07.000Z"), "id": 707838431448072193, "text": "Happy birthday to my homie @Pereidaa_951 the best crippled bp partner I have �� love you G ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1078295743 }}, "user": { "id": 2955370135, "name": "L", "screen_name": "__siul", "lang": "en", "location": "null", "create_at": date("2015-01-01"), "description": "17. what doesn't kill you. fucks you up, mentally ➰", "followers_count": 366, "friends_count": 349, "statues_count": 7132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200787415937024, "text": "@_Shuntanevitt @ItsMy_Dimples @_terionb so tagging my name idk what's going on", "in_reply_to_status": 708200702187712512, "in_reply_to_user": 1966230774, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1966230774, 92439134, 282648906 }}, "user": { "id": 247611282, "name": "RIP-DJ&E&E", "screen_name": "HubbDaGreat", "lang": "en", "location": "shreveport LA", "create_at": date("2011-02-04"), "description": "#SAU19 snapchat: hubbwizzle ig: underrated15", "followers_count": 605, "friends_count": 551, "statues_count": 13537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magnolia, AR", "id": "71e16cd97a552f44", "name": "Magnolia", "place_type": "city", "bounding_box": rectangle("-93.267708,33.23834 -93.208361,33.309264") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5027, "countyName": "Columbia", "cityID": 543460, "cityName": "Magnolia" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200787847962624, "text": "@StationAgent00 I see you down with Apache �� https://t.co/dK2nMmDtRl", "in_reply_to_status": -1, "in_reply_to_user": 937162045, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 937162045 }}, "user": { "id": 16099395, "name": "The Good Sheppard", "screen_name": "SeanSheppard", "lang": "en", "location": "San Diego", "create_at": date("2008-09-02"), "description": "Native East Coaster, Founder of Embrace, Sons of the American Legion, Honorary Buffalo Soldier. Just here, in God's name, to be a blessing to millions of people", "followers_count": 501, "friends_count": 244, "statues_count": 6135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200787994738688, "text": "I Hate Being By Myself At Home I Feel Like I Be Hearing Shit. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1274229709, "name": "XXVIII-XII", "screen_name": "Damaris_Marieee", "lang": "en", "location": "San Diego", "create_at": date("2013-03-16"), "description": "Baaaaaad Boooooyyyyyy|East Coast| Ask , Believe And You Shall Receive| KDS✊❤️", "followers_count": 342, "friends_count": 531, "statues_count": 10329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200788800057345, "text": "-5 https://t.co/PwXfknNHeF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2384561665, "name": "maxe™", "screen_name": "Maxwellminus", "lang": "en", "location": "Mesquite➡️Forney, TX", "create_at": date("2014-03-11"), "description": "future something current garbage NFHS '16 Chi", "followers_count": 732, "friends_count": 550, "statues_count": 20848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forney, TX", "id": "0059ce31696ebaf7", "name": "Forney", "place_type": "city", "bounding_box": rectangle("-96.491676,32.69623 -96.387296,32.778267") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4826604, "cityName": "Forney" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200789399904257, "text": "@essjayeff @ruminski my thongs?", "in_reply_to_status": 708200166751232005, "in_reply_to_user": 12084692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12084692, 20903992 }}, "user": { "id": 62200057, "name": "Cha's Kitchen", "screen_name": "chaskitchen", "lang": "en", "location": "Melbourne ✈️ Portland", "create_at": date("2009-08-01"), "description": "Wealthy Industrialist, bow tie connoisseur and the worlds foremost bubble bath enthusiast. I like cheap beer.", "followers_count": 3784, "friends_count": 1284, "statues_count": 27455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200789567606784, "text": "2 am and my mind is on ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2903123533, "name": "Alexis $andoval", "screen_name": "_35alexis", "lang": "en", "location": "Mission, TX", "create_at": date("2014-11-17"), "description": "idea mission// varsity basketball", "followers_count": 212, "friends_count": 100, "statues_count": 4732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission, TX", "id": "77633125ba089dcb", "name": "Mission", "place_type": "city", "bounding_box": rectangle("-98.363219,26.155046 -98.272146,26.262558") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4848768, "cityName": "Mission" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200789852946432, "text": "Time is not only a magazine, it's also made by man. #sundial #firstclock #whattimeisit #time… https://t.co/ivGwqLsWPr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1222573,38.8438325"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sundial", "firstclock", "whattimeisit", "time" }}, "user": { "id": 39474967, "name": "Sancho", "screen_name": "phogletaker", "lang": "en", "location": "USA", "create_at": date("2009-05-12"), "description": "I am Sancho", "followers_count": 485, "friends_count": 2007, "statues_count": 29980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bailey's Crossroads, VA", "id": "c933b99b367f0b5b", "name": "Bailey's Crossroads", "place_type": "city", "bounding_box": rectangle("-77.147585,38.832027 -77.110316,38.861746") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5104088, "cityName": "Bailey's Crossroads" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200789987053568, "text": "My favorite line of all time https://t.co/FINjWaLZKJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240670810, "name": "Danny Rios V", "screen_name": "fuckdjdd", "lang": "en", "location": "Schaumburg, IL", "create_at": date("2011-01-20"), "description": "null", "followers_count": 50, "friends_count": 117, "statues_count": 1617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoffman Estates, IL", "id": "01df0964763e9f17", "name": "Hoffman Estates", "place_type": "city", "bounding_box": rectangle("-88.243545,42.026652 -88.060119,42.117916") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1735411, "cityName": "Hoffman Estates" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200789987078144, "text": ":(((( https://t.co/wImn9FFeFq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2365015868, "name": "sakura uchiha", "screen_name": "jadaIndigo", "lang": "en", "location": "the cut", "create_at": date("2014-02-27"), "description": "don't dm me i'm a whore.", "followers_count": 1348, "friends_count": 1218, "statues_count": 58957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennesaw, GA", "id": "c4cd1df048841e00", "name": "Kennesaw", "place_type": "city", "bounding_box": rectangle("-84.64724,33.955258 -84.577544,34.066895") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1343192, "cityName": "Kennesaw" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200790700089344, "text": "Smokies Lee I'd be your trap queen any day.\nMarch 4 at 3:25pm https://t.co/Nv5yXp25Z9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3903951312, "name": "Jayy Sorath LaVeyGOD", "screen_name": "Jayy_LaVey", "lang": "en", "location": "Franklin, TN", "create_at": date("2015-10-15"), "description": "#illuminati. IM FUNNY. IM SARCASTIC. #starvingartist King, Of Tennessee. Married. 25. BLACKBILLY. I'm Silly. SloppyTweeter. Kid Komedian: IN PROGRESS. #BOSSING.", "followers_count": 278, "friends_count": 163, "statues_count": 72896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, TN", "id": "cc631a80adacd459", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.935419,35.85036 -86.766934,36.019674") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47187, "countyName": "Williamson", "cityID": 4727740, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200790800859136, "text": "@talkingcongas so are you a ducks fan then? Also what’s the hockey culture like in Cali? Hahaha", "in_reply_to_status": -1, "in_reply_to_user": 35012599, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35012599 }}, "user": { "id": 722901415, "name": "Seraj", "screen_name": "Serajfarabi", "lang": "en", "location": "Minneapolis, Minnesota", "create_at": date("2012-07-28"), "description": "Photographer, Hockey Fan,  Apple Geek, Technology Lover, World Traveler.", "followers_count": 216, "friends_count": 943, "statues_count": 5859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200791052460032, "text": "My safe ride driver has to pee. God help us. #DAH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DAH" }}, "user": { "id": 973113468, "name": "UNATH", "screen_name": "DanePiazza", "lang": "en", "location": "Oregon State University", "create_at": date("2012-11-26"), "description": "Discover Oceans #DaneAfterHours", "followers_count": 539, "friends_count": 538, "statues_count": 5933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-03-11T00:00:00.000Z"), "id": 708200791085948928, "text": "https://t.co/3YFj5UlGWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3042597607, "name": "Andrew Apodaca", "screen_name": "andrewapodaca47", "lang": "en", "location": "Arvada, Colorado", "create_at": date("2015-02-25"), "description": "my name is Andrew I'm a single 23 year old I work as a cashier I'm in college and I love sports green bay packers colorado avalanche Denver nuggets and Rockies", "followers_count": 635, "friends_count": 2102, "statues_count": 1069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arvada, CO", "id": "c02e66a5016d732f", "name": "Arvada", "place_type": "city", "bounding_box": rectangle("-105.214417,39.783802 -105.04196,39.856611") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 803455, "cityName": "Arvada" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200791538925568, "text": "@TonyDulalia yeah same �� we should get together for a set.", "in_reply_to_status": 708199090346037248, "in_reply_to_user": 147768852, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 147768852 }}, "user": { "id": 224837606, "name": "Jesüs Hernandez", "screen_name": "Jedi_Jesus77", "lang": "en", "location": "Palm Springs, Ca ", "create_at": date("2010-12-09"), "description": "DW", "followers_count": 560, "friends_count": 474, "statues_count": 35916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200791908208641, "text": "@TheMikeEads \"Where an excess of power prevails, no man is safe in his opinions, his person, his faculties or his possessions.” Madison", "in_reply_to_status": -1, "in_reply_to_user": 17596826, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17596826 }}, "user": { "id": 1497568382, "name": "Jerome Huyler, PhD.", "screen_name": "huylerje", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2013-06-09"), "description": "Fmr Asst Professor Seton Hall University. Author: Locke in America: The Moral Philosophy of the Founding Era and Everything You Have: The Case Against Welfare.", "followers_count": 5096, "friends_count": 5583, "statues_count": 64992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200791979532288, "text": "@TerrierHockey 's Adam Clendening had an assist as Edmonton won 2-1 at Minnesota #Oilers #NHL #BU #HockeyEast", "in_reply_to_status": -1, "in_reply_to_user": 558835990, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Oilers", "NHL", "BU", "HockeyEast" }}, "user_mentions": {{ 558835990 }}, "user": { "id": 308892843, "name": "Richard Slate", "screen_name": "RichSlate", "lang": "en", "location": "Beverly, MA", "create_at": date("2011-05-31"), "description": "B's beat writer & sportswriter @MetroBOS; Twitter-obsessed Boston sports fan; @Beverly_High & @ProvidenceCol grad; beer snob; amateur music, TV & film critic", "followers_count": 936, "friends_count": 2000, "statues_count": 462490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beverly, MA", "id": "7dea5f8c907d1ea2", "name": "Beverly", "place_type": "city", "bounding_box": rectangle("-70.923007,42.53915 -70.793176,42.593439") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2505595, "cityName": "Beverly" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792050835456, "text": "@diosprimero32 you slid", "in_reply_to_status": 708200661758906368, "in_reply_to_user": 2384228061, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2384228061 }}, "user": { "id": 3449137287, "name": "Bigmansavage95", "screen_name": "KyleShortridge", "lang": "en", "location": "null", "create_at": date("2015-08-26"), "description": "young man tryna get rich", "followers_count": 400, "friends_count": 434, "statues_count": 246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792189042688, "text": "Wind 2.7 mph NW. Barometer 29.940 in, Rising. Temperature 33.1 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792218411009, "text": "@ebbtideapp Tide in Bangor, Maine 03/11/2016\n Low 6:23am -1.6\nHigh 12:10pm 16.3\n Low 6:47pm -1.4\nHigh 12:33am 16.4\n Low 7:15am -1.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-68.7717,44.795"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 39, "friends_count": 1, "statues_count": 14833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bangor, ME", "id": "bd73972b51647e05", "name": "Bangor", "place_type": "city", "bounding_box": rectangle("-68.872064,44.768404 -68.71876,44.855108") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23019, "countyName": "Penobscot", "cityID": 2302795, "cityName": "Bangor" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792272932868, "text": "@Jonathan_126 �� https://t.co/vezkGATqzf", "in_reply_to_status": -1, "in_reply_to_user": 378233280, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 378233280 }}, "user": { "id": 276853862, "name": "Dan", "screen_name": "Dan0nfire", "lang": "en", "location": "Bay Area, CA", "create_at": date("2011-04-03"), "description": "Four strands of rope!!!", "followers_count": 115, "friends_count": 431, "statues_count": 8164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Mateo, CA", "id": "432daa3153c5fef9", "name": "San Mateo", "place_type": "city", "bounding_box": rectangle("-122.355038,37.512567 -122.275805,37.592395") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 668252, "cityName": "San Mateo" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792461737984, "text": "yooo, I'm pretty sure I got insomnia cause I can't ever fall sleep at night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1072709839, "name": "natatata", "screen_name": "_natalyaaa_", "lang": "en", "location": "Houston, TX", "create_at": date("2013-01-08"), "description": "null", "followers_count": 1421, "friends_count": 456, "statues_count": 13108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200792809992192, "text": "who would've known?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 894574922, "name": "⛑ luv$.", "screen_name": "TezzDaGod", "lang": "en", "location": "null", "create_at": date("2012-10-20"), "description": "601.", "followers_count": 1138, "friends_count": 404, "statues_count": 59215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200793476698112, "text": "Going thru crates right now... I miss back in the day listening to @xzibit in middle school. First intro to @DJFINGAZ was back then too ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18845207, 22102755 }}, "user": { "id": 352128647, "name": "DJ HeartAttak", "screen_name": "DJHeartAttak", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-08-09"), "description": "...just pass me the aux cord • IG: djHeartAttak", "followers_count": 1977, "friends_count": 300, "statues_count": 14266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200793623531521, "text": "@tropicunta BITCH IM@FUCKING SHY EXHNSNFNMGJB", "in_reply_to_status": 708200562156658688, "in_reply_to_user": 3026549518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3026549518 }}, "user": { "id": 3037556102, "name": "jinki's jaimz", "screen_name": "romanticjinki", "lang": "en", "location": "yixing❤️", "create_at": date("2015-02-22"), "description": "*jinki voice* noona you're my benchwarmer", "followers_count": 887, "friends_count": 482, "statues_count": 41113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200793946521601, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 376384047, "name": "EuphoricChaos", "screen_name": "_DistantUtopia", "lang": "en", "location": "Miami, Fl", "create_at": date("2011-09-19"), "description": "Sith Lord. Los Lobo 21 NSU/MDC/FIU Miami Heat, and Golden State. . We loved with a love that was more than love. -EAP", "followers_count": 516, "friends_count": 699, "statues_count": 36694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westview, FL", "id": "7b0797409cd1e6de", "name": "Westview", "place_type": "city", "bounding_box": rectangle("-80.259871,25.864967 -80.226027,25.897642") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1276950, "cityName": "Westview" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200794047119360, "text": "�� https://t.co/QsR3HOa01l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 588819741, "name": "čhåñçë", "screen_name": "T_Town19", "lang": "en", "location": "null", "create_at": date("2012-05-23"), "description": "18.THS wrestler. work hard and play harder. Are you happy with who you are?", "followers_count": 465, "friends_count": 811, "statues_count": 4681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tooele, UT", "id": "5f72d488385b8ced", "name": "Tooele", "place_type": "city", "bounding_box": rectangle("-112.331612,40.507327 -112.266954,40.569357") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49045, "countyName": "Tooele", "cityID": 4976680, "cityName": "Tooele" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200794105851908, "text": "We're #hiring! Read about our latest #job opening here: Registered Nurse | RN | PEDS... - https://t.co/2sLFu3YQUj https://t.co/XqxGIEmsK8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.4139393,37.2295733"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 185, "friends_count": 99, "statues_count": 3696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blacksburg, VA", "id": "820684853e0f1eb6", "name": "Blacksburg", "place_type": "city", "bounding_box": rectangle("-80.47611,37.185195 -80.381618,37.273387") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51121, "countyName": "Montgomery", "cityID": 5107784, "cityName": "Blacksburg" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200794282012672, "text": "When a thot tells you she's pregnant but you wore a rubber https://t.co/2QOq1o5biu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1527006450, "name": "Fried Chicken Papi", "screen_name": "SeanLWashington", "lang": "en", "location": "North West ☔️", "create_at": date("2013-06-18"), "description": "If young thugga don't trust you, Ima shoot you", "followers_count": 1104, "friends_count": 672, "statues_count": 29518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-11T00:00:01.000Z"), "id": 708200795246764032, "text": "@eeedesu @J4Mane https://t.co/wUc1pmRFYA", "in_reply_to_status": -1, "in_reply_to_user": 3161430822, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3161430822, 3351190390 }}, "user": { "id": 2996135683, "name": "RAGE STAR", "screen_name": "TUJARZZ", "lang": "en", "location": "We're fashion and art fuse ", "create_at": date("2015-01-25"), "description": "contact email watdatour@gmail.com #walkingarttour #ragenroll\nAll the love that we had and the love that we hide Who will bury us when we die?", "followers_count": 1549, "friends_count": 2884, "statues_count": 4131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200795385274368, "text": "Wind 2.0 mph NE. Barometer 30.156 in, Rising slowly. Temperature 49.7 °F. Rain today 0.02 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200795754340352, "text": "Goodnight ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585106188, "name": "Quasiaaaa Hodgesss", "screen_name": "Quasia_Hodges", "lang": "en", "location": "38801", "create_at": date("2012-05-19"), "description": "null", "followers_count": 1607, "friends_count": 1189, "statues_count": 15746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fulton, MS", "id": "01ac83b656e89fa8", "name": "Fulton", "place_type": "city", "bounding_box": rectangle("-88.418119,34.229051 -88.368362,34.286157") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28057, "countyName": "Itawamba", "cityID": 2826300, "cityName": "Fulton" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200795825541120, "text": "This my favorite move RT @cummings_lexie: Niggas break ya heart & burn it then wanna hit you up later talmbout \"I hope you're doing good\"...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1071865142 }}, "user": { "id": 911909497, "name": "Walley™ ‏", "screen_name": "ImJonDuhh", "lang": "en", "location": "Los Angeles, California", "create_at": date("2012-10-29"), "description": "#Justice4Tyler \n\nIG: Walley1017", "followers_count": 5037, "friends_count": 2625, "statues_count": 77563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200795959742464, "text": "66.4F (Feels: 66.4F) - Humidity: 99% - Wind: 6.9mph NE - Gust: 7.6mph - Pressure: 1017.5mb - Rain: 0.01\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 219, "friends_count": 18, "statues_count": 230149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796211580928, "text": "03/11@03:00 - Temp 62.1F, WC 62.1F. Wind 0.9mph WNW, Gust 3.0mph. Bar 29.928in, Steady. Rain 0.00in. Hum 80%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796484149248, "text": "1h 30m wait time at Northeast Community Health Centre. Browse or share wait times with #WaitShare at https://t.co/rv1fffVDIf! #Edmonton", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.418304,53.604512"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitShare", "Edmonton" }}, "user": { "id": 2444180670, "name": "WaitShare", "screen_name": "WaitShareApp", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "WaitShare is a free app that lets you share your wait from ERs, clinics, and pharmacies. Make informed health care decisions because others have shared.", "followers_count": 371, "friends_count": 883, "statues_count": 2990 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796677079040, "text": "Wind 0.0 mph ---. Barometer 30.260 in, Rising. Temperature 41.1 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796806979588, "text": "REAL NIGGA HOURS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409080423, "name": "Side Nigga Manuel", "screen_name": "Grand__papi", "lang": "en", "location": "Lancaster, CA", "create_at": date("2013-05-06"), "description": "God, 19", "followers_count": 744, "friends_count": 558, "statues_count": 60374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796916211713, "text": "1 Scott's Oriole (Icterus parisorum) - Florida Canyon--lower - 2016-03-10 09:45 https://t.co/8az3CiQDK7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12761 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796945571840, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Florida Canyon--lower - 2016-03-10 10:55 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12761 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796949716992, "text": "1 Baltimore Oriole (Icterus galbula) - Sweetwater Wetlands - 2016-03-10 11:20 https://t.co/vtOnCLFDzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12761 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200796991705088, "text": "1 White-throated Sparrow (Zonotrichia albicollis) - Angel's Corner - 2016-03-10 17:15 https://t.co/m3XsLOTrMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9120538,32.508082"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12761 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 411160, "cityName": "Catalina" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797062868992, "text": "1 Hepatic Tanager (Piranga flava) - Florida Canyon--lower - 2016-03-10 09:45 https://t.co/TCWyz5wNo4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12761 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797205614592, "text": "@Greenskull wait. Are you on the Ubisoft AND 343 payroll?", "in_reply_to_status": 708198212612399104, "in_reply_to_user": 5623612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5623612 }}, "user": { "id": 30723950, "name": "Tyler Owens", "screen_name": "heytred_", "lang": "en", "location": "Kirkland, WA", "create_at": date("2009-04-12"), "description": "Halo 5 // ex-343 employee and current 343 shill", "followers_count": 999, "friends_count": 670, "statues_count": 5758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797373239296, "text": "no one go to school tom. problem solved.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2872559144, "name": "katie", "screen_name": "katieconzelman", "lang": "en", "location": "null", "create_at": date("2014-10-22"), "description": "sassyass", "followers_count": 418, "friends_count": 555, "statues_count": 3845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rohnert Park, CA", "id": "5cda0a6ac9cf8725", "name": "Rohnert Park", "place_type": "city", "bounding_box": rectangle("-122.732141,38.314281 -122.666695,38.372609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 662546, "cityName": "Rohnert Park" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797398372353, "text": "I'm gonna try to sleep on a different side of my bed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2199226207, "name": "camila", "screen_name": "champagnecamile", "lang": "en", "location": "661", "create_at": date("2013-11-17"), "description": "Alicia's my right hand", "followers_count": 882, "friends_count": 559, "statues_count": 27377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lamont, CA", "id": "ee121100f5e9087b", "name": "Lamont", "place_type": "city", "bounding_box": rectangle("-118.932111,35.234117 -118.894307,35.310691") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 640088, "cityName": "Lamont" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797499031555, "text": "Happy birthday EggJ!!!!!!! ��\n@uglyweatherman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2998098199 }}, "user": { "id": 964951525, "name": "ok", "screen_name": "enthusedx", "lang": "en", "location": "rhc", "create_at": date("2012-11-22"), "description": "less malcontent than before, still malcontent • @facexghost_ & @FacexFart are everything •", "followers_count": 398, "friends_count": 308, "statues_count": 5031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797641711616, "text": "#SupportOriginMelissa 58.8°F Wind:0.0mph Pressure: 30.12hpa Rising Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 312073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797754892290, "text": "I know you on here lurking https://t.co/ypK5Ohb18X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1133093275, "name": "Travis", "screen_name": "Yungtrav93", "lang": "en", "location": "null", "create_at": date("2013-01-29"), "description": "Don't waste my time. 22 years young. 580✈️918 Phil 4:13. On the road to be successful.", "followers_count": 1097, "friends_count": 1021, "statues_count": 40885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797956403200, "text": "1 Baltimore Oriole (Icterus galbula) - Sweetwater Wetlands - 2016-03-10 08:50 https://t.co/vtOnCLFDzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12766 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200797989957632, "text": "1 Baltimore Oriole (Icterus galbula) - Sweetwater Wetlands - 2016-03-10 09:22 https://t.co/vtOnCLFDzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12766 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798002487296, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Florida Canyon--lower - 2016-03-10 09:15 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12766 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798115659776, "text": "Temp: 64.7°F Wind:1.3mph Pressure: 30.095hpa Steady Rain Today 0.33in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798132436993, "text": "@CFWPirate Not seen you around lately Jenn .......ANYBODY HOLDING YOU HOSTAGE?", "in_reply_to_status": 708200300876668928, "in_reply_to_user": 2607804924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2607804924 }}, "user": { "id": 4861280291, "name": "TrumpWhiteHouseBitch", "screen_name": "ItsRickBitches", "lang": "en", "location": "null", "create_at": date("2016-01-29"), "description": "null", "followers_count": 742, "friends_count": 729, "statues_count": 9831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamilton, OH", "id": "3a32d101fe825fd9", "name": "Hamilton", "place_type": "city", "bounding_box": rectangle("-84.64717,39.350135 -84.474638,39.457988") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3933012, "cityName": "Hamilton" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798161768448, "text": "1 Black-and-white Warbler (Mniotilta varia) - Sweetwater Wetlands - 2016-03-10 08:50 https://t.co/dbHJYjQq4B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12766 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798442885121, "text": "1 Rufous-backed Robin (Turdus rufopalliatus) - Catalina SP - 2016-03-10 08:32 https://t.co/7ll1ihhmu1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9293064,32.4186422"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12768 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798526640128, "text": "Holy fuck I wish my friends had twitters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546823617, "name": "Ludwig", "screen_name": "slaymama", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "ॐSweeter than a swisher.", "followers_count": 678, "friends_count": 96, "statues_count": 20263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798635687937, "text": "#pawnstars is now trending in #Columbus https://t.co/EsqXXTf4jQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pawnstars", "Columbus" }}, "user": { "id": 132354213, "name": "Trendsmap Columbus", "screen_name": "TrendsColumbus", "lang": "en", "location": "Columbus, OH, USA", "create_at": date("2010-04-12"), "description": "Real-time Columbus, OH Twitter trends", "followers_count": 1519, "friends_count": 438, "statues_count": 10497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798950465536, "text": "2 White-throated Sparrow (Zonotrichia albicollis) - Catalina SP - 2016-03-10 08:32 https://t.co/m3XsLOTrMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9293064,32.4186422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12770 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200798979821568, "text": "1 Baltimore Oriole (Icterus galbula) - Sweetwater Wetlands - 2016-03-10 08:30 https://t.co/vtOnCLFDzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12770 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200799340515328, "text": "1 Black-and-white Warbler (Mniotilta varia) - Sweetwater Wetlands - 2016-03-10 08:30 https://t.co/dbHJYjyPd3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12773 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200799361441793, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Florida Canyon--lower - 2016-03-10 07:53 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12773 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200799449571328, "text": "People are shitty.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3063756429, "name": "Katie Selway", "screen_name": "kselw2", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "null", "followers_count": 83, "friends_count": 66, "statues_count": 132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, IL", "id": "ce1765e3abafe93e", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-89.773186,39.673272 -89.546193,39.87542") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17167, "countyName": "Sangamon", "cityID": 1772000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-03-11T00:00:02.000Z"), "id": 708200799466196993, "text": "@Drob0G wyd papa", "in_reply_to_status": 708200671149854720, "in_reply_to_user": 764111965, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user_mentions": {{ 764111965 }}, "user": { "id": 738280860, "name": ":)", "screen_name": "PaulTweets2Much", "lang": "en", "location": "Queens, NY", "create_at": date("2012-08-05"), "description": "i Tweet Too Much | I'll probably upset you at some point | ❤️@PaulTweets2Much❤️ | i had sexual intercourse with Tom Brady & gave birth to @hsmitty3 | A❤️", "followers_count": 8770, "friends_count": 791, "statues_count": 314637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200799655043072, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Madera Canyon--Proctor Rd. - 2016-03-10 07:45 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.886147,31.739938"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12773 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200799776550912, "text": "@logo_07 happy birthday brother����������✊��", "in_reply_to_status": -1, "in_reply_to_user": 447730953, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 447730953 }}, "user": { "id": 590944993, "name": "Duhh Dummy™", "screen_name": "TeamNiike48", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "Follow Me On IG: OsoArrogantTre_ Wisconsin/Galveston [ØBN]", "followers_count": 1352, "friends_count": 895, "statues_count": 15069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galveston, TX", "id": "632eeebc87aecd57", "name": "Galveston", "place_type": "city", "bounding_box": rectangle("-94.880809,29.239602 -94.764742,29.335548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4828068, "cityName": "Galveston" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200799868985348, "text": "2 Cassin's Sparrow (Peucaea cassinii) - Grasslands between Proctor Rd and Florida Wash - 2016-03-10 07:20 https://t.co/pcqlSaIkFj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8873272,31.769332"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200799940247552, "text": "1 White-crowned Sparrow (oriantha) (Zonotrichia leucophrys) - Seneca/Ridgeway area - 2016-03-10 06:59 https://t.co/KmIYE1DiTc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8653452,32.2475404"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800120516608, "text": "Daaamn James �� love this song already❤️❤️❤️\n#Forever\n#ReigNadine \nhttps://t.co/zVOTxeo9iU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Forever", "ReigNadine" }}, "user": { "id": 92711659, "name": "voтe nadιne ғor мyх", "screen_name": "kjcxrasalan", "lang": "en", "location": "jadιne&вaιlona ғan & oтwolιѕтa", "create_at": date("2009-11-26"), "description": "ĸapag мaнal ĸa, вaвalιĸan ĸa? ĸυng мaнal ĸa nιya тalaga, ιιwan ĸa вa nιya? -clarĸ м.", "followers_count": 635, "friends_count": 318, "statues_count": 68842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kapolei, HI", "id": "01932ef001ef7c53", "name": "Kapolei", "place_type": "city", "bounding_box": rectangle("-158.090772,21.323228 -158.048087,21.365894") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1530300, "cityName": "Kapolei" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800330317824, "text": "1 Black-and-white Warbler (Mniotilta varia) - Sweetwater Wetlands - 2016-03-10 06:54 https://t.co/dbHJYjyPd3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.022495,32.278549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12778 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800338710528, "text": "1 White-throated Sparrow (Zonotrichia albicollis) - Coachline Gravel Pit (El Rio Open Space Preserve) - 2016-03-09 07:46", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.136,32.394"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12778 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 444270, "cityName": "Marana" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800456089600, "text": "I'm exhausted should I sleep or watch my show", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1967132196, "name": "Said", "screen_name": "said_abdisalam", "lang": "en", "location": "Dubai ✈️ Ohio ✈️ Minneapolis ", "create_at": date("2013-10-17"), "description": "Sanisa. sc: saidoo16", "followers_count": 404, "friends_count": 271, "statues_count": 9613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maplewood, MN", "id": "fea1c768c9a65cc7", "name": "Maplewood", "place_type": "city", "bounding_box": rectangle("-93.105955,44.890773 -92.984027,45.036878") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2740382, "cityName": "Maplewood" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800523169793, "text": "#chumley is now trending in #Columbus https://t.co/EsqXXTf4jQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "chumley", "Columbus" }}, "user": { "id": 132354213, "name": "Trendsmap Columbus", "screen_name": "TrendsColumbus", "lang": "en", "location": "Columbus, OH, USA", "create_at": date("2010-04-12"), "description": "Real-time Columbus, OH Twitter trends", "followers_count": 1519, "friends_count": 438, "statues_count": 10498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800602988544, "text": "1 Rufous-backed Robin (Turdus rufopalliatus) - Catalina SP - 2016-03-09 07:30 https://t.co/7ll1ihhmu1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9293064,32.4186422"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12778 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800636416001, "text": "�������� https://t.co/8bQeXeQc79", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 492802930, "name": "Solomon M.", "screen_name": "HeftyHendrix", "lang": "en", "location": "College Station, TX • SATX", "create_at": date("2012-02-14"), "description": "JOURNALISM MAJOR • PR MINOR • THE FAT R.KELLY • TAMU '17 • #THEYOUTHALWAYSWINS", "followers_count": 1085, "friends_count": 299, "statues_count": 55415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800733011968, "text": "Wonderful night with awesome friends. �� can't beat that stuff", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321221241, "name": "JPenn", "screen_name": "JPenn1123", "lang": "en", "location": "Bottom of the Mitten ", "create_at": date("2011-06-20"), "description": "RIP pops. I paint shoes Instagram-jpcustomshoes-jordanpennington11", "followers_count": 579, "friends_count": 394, "statues_count": 14117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Adrian, MI", "id": "4286bea8be81e904", "name": "Adrian", "place_type": "city", "bounding_box": rectangle("-84.094415,41.860395 -84.004134,41.941268") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2600440, "cityName": "Adrian" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800837881856, "text": "1 White-throated Sparrow (Zonotrichia albicollis) - Catalina SP - 2016-03-09 07:30 https://t.co/m3XsLOTrMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.9293064,32.4186422"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 12780 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800946753539, "text": "All through her mind she can't sleep lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155115676, "name": "OSOArrogant", "screen_name": "KingPratt95", "lang": "en", "location": "Bahamas", "create_at": date("2010-06-12"), "description": "IG: @bahamian_nightmare242 ; ||c/o 2014 || Live where you vacay Bahamas ☀️ #KingPratt #OsoArrogant #FTF the Movement", "followers_count": 1431, "friends_count": 1165, "statues_count": 61836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200800971935745, "text": "King lil g ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615401566, "name": "Emily✨", "screen_name": "EmilyLovessyuh", "lang": "en", "location": "California ✈️ WA ", "create_at": date("2012-06-22"), "description": "doing , living it for JO ‼️❤️❤❤️ sc / emilygutierrez & im coolin sets", "followers_count": 509, "friends_count": 511, "statues_count": 12114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederickson, WA", "id": "1dc0a9a76e5081b3", "name": "Frederickson", "place_type": "city", "bounding_box": rectangle("-122.400295,47.062069 -122.314857,47.127584") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5325475, "cityName": "Frederickson" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200801122918401, "text": "@Nudah @SequoiaRayne", "in_reply_to_status": 707784798996049920, "in_reply_to_user": 774626688, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 774626688, 431366638 }}, "user": { "id": 450983484, "name": "Nonie-Ann Grant", "screen_name": "Nonie_Grant96", "lang": "en", "location": "null", "create_at": date("2011-12-30"), "description": "Fresno Pacific Water polo c/o 2019 | ZW ♡ Proverbs 31:25", "followers_count": 179, "friends_count": 153, "statues_count": 2697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200801554960386, "text": "Honestly have mad love for Ju and Ness ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2683994989, "name": "slinthia", "screen_name": "cynthjagonzalez", "lang": "en", "location": "null", "create_at": date("2014-07-26"), "description": "bahs. Because we completely believe in the impossible. We believe that we can change the world.", "followers_count": 250, "friends_count": 357, "statues_count": 2720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200802079248385, "text": "Wind 0.7 mph WSW. Barometer 29.91 in, Steady. Temperature 52.5 °F. Rain today 0.02 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 116, "statues_count": 158157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200802372825089, "text": "In just a week I'll be in San Diego ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051888195, "name": "drey", "screen_name": "its__andrea", "lang": "en", "location": "null", "create_at": date("2012-12-31"), "description": "lhs // soccer", "followers_count": 669, "friends_count": 740, "statues_count": 18100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200802536402945, "text": "I get on here when snap chat is dry lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155804881, "name": "Boss Lady", "screen_name": "N0W0rries06", "lang": "en", "location": "Jonesboro, AR", "create_at": date("2010-06-14"), "description": "Instagram: ms_loyal_one", "followers_count": 915, "friends_count": 816, "statues_count": 35456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, AR", "id": "35189a8fcb82b788", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-90.782468,35.763136 -90.611214,35.894116") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5031, "countyName": "Craighead", "cityID": 535710, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200802792247297, "text": "�������� https://t.co/Xo3wcB4lxB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 7455622, "name": "Tyler Scott Sanchez", "screen_name": "PlayerOneTyler", "lang": "en", "location": "Burbank, CA", "create_at": date("2007-07-13"), "description": "@DisneyInfinity Community Team - YouTuber - Disney Parks / Pokemon Enthusiast - Statements don't reflect those of Disney.", "followers_count": 963, "friends_count": 411, "statues_count": 7808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200803245236225, "text": "Check out super awesome #director #producer @durruck and his film #dreality at #caamfest2016 on… https://t.co/3aW4VLX9gh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.41638475,37.78019375"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "director", "producer", "dreality", "caamfest2016" }}, "user_mentions": {{ 338873984 }}, "user": { "id": 25407663, "name": "Larry Wong", "screen_name": "elarrywong", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-19"), "description": "Serious Photographer, Traveler, Connoisseur of all things beautiful. \nMy Cities: San Francisco | Los Angeles | New York | Seattle | \nBrands: @maybach @7x7", "followers_count": 832, "friends_count": 1906, "statues_count": 2497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200803350085636, "text": "He said rent ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236746318, "name": "Portgas D. Ace", "screen_name": "bootywranglers", "lang": "en", "location": "Norco, CA", "create_at": date("2013-12-08"), "description": "Just a loser Otaku who likes to work out . \n \n\n\n \nSponsored by Brazzers", "followers_count": 630, "friends_count": 343, "statues_count": 39955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200803547291648, "text": "I love you baby ������ https://t.co/Erz9l7x1Gr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2833351105, "name": "Genesis", "screen_name": "lowkeygenesis", "lang": "en", "location": "Cole World", "create_at": date("2014-09-26"), "description": "#FreeGuwop | Life is what you make it", "followers_count": 4369, "friends_count": 2055, "statues_count": 52640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-03-11T00:00:03.000Z"), "id": 708200803584978947, "text": "LMFAOOOOO https://t.co/0tI9OcShsY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1081765278, "name": "Blacksbi Hendrix", "screen_name": "MattWakhu", "lang": "en", "location": "Denton, TX", "create_at": date("2013-01-11"), "description": "Matt Wakhu | Future GentlΣmaΝ | Mr. UNT | Kenyan | Follow my music page @MattWakhuMusic!", "followers_count": 3424, "friends_count": 2269, "statues_count": 120864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200804256063488, "text": "Happy birthday Lakelyn!������ @lakelynbagge", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 529863948 }}, "user": { "id": 240911879, "name": "Baela_Kitty", "screen_name": "daela_haerle", "lang": "en", "location": "null", "create_at": date("2011-01-20"), "description": "Day-Luh // Cali Reckless & Plastix", "followers_count": 811, "friends_count": 450, "statues_count": 10613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200804281229312, "text": "Lmaooooo https://t.co/DnxxDrCa7W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.25376142,33.85502054"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 594306460, "name": "ElijahSho™", "screen_name": "ElijahSho", "lang": "en", "location": "Carson, CA", "create_at": date("2012-05-29"), "description": "Anything legal for da uchies | #LakerNation | Nigerian | Ruth ❤️ | Rip Gio #LongLiveGio", "followers_count": 1072, "friends_count": 394, "statues_count": 102957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200804520304640, "text": "@_Cohesza I post that once a year... He bout to cus me out", "in_reply_to_status": 708200695782965248, "in_reply_to_user": 2352109885, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2352109885 }}, "user": { "id": 44467060, "name": "Trillyonce", "screen_name": "KissChelleV", "lang": "en", "location": "Sippin' on Port Arthur", "create_at": date("2009-06-03"), "description": "Automatic Supersonic Hypnotic Funky Fresh. IG: KissChelleV UCO 11.15 Scorpio Brown Suga", "followers_count": 1384, "friends_count": 947, "statues_count": 105584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200804625219589, "text": "I miss Mexico ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 273210560, "name": "sunshine", "screen_name": "heynelly_", "lang": "en", "location": "null", "create_at": date("2011-03-27"), "description": "Golden.", "followers_count": 700, "friends_count": 380, "statues_count": 23454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200804889468928, "text": "Lmao people call my bluff and I show them wrong.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2184644046, "name": "James Ryan", "screen_name": "JTownsend96", "lang": "en", "location": "Houston, TX", "create_at": date("2013-11-09"), "description": "U of H Coog • ΠΚΑ • #GeauxTigers #GoCoogs", "followers_count": 463, "friends_count": 339, "statues_count": 12588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200805065568257, "text": "Who all remember Zieger Sunday ? We was going up on Sundays !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597476767, "name": "Willy ⚔ Will", "screen_name": "WillTheGoat", "lang": "en", "location": "Prairie View, TX", "create_at": date("2012-06-02"), "description": "null", "followers_count": 732, "friends_count": 615, "statues_count": 7411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200806462283776, "text": "Jb next week wtFFFFFF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2246115956, "name": "gretchen (︶︿︶メ)╭∩╮", "screen_name": "gntran", "lang": "en", "location": "null", "create_at": date("2013-12-14"), "description": "null", "followers_count": 204, "friends_count": 224, "statues_count": 12065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200806537830400, "text": "Always me, myself, & I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2739009481, "name": "Scott Pilgrim", "screen_name": "BitchMando", "lang": "en", "location": "Bronx, NY", "create_at": date("2014-08-16"), "description": "Up with hope, down with dope.", "followers_count": 96, "friends_count": 64, "statues_count": 1511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200806743277568, "text": "Armenians have a type of hospitality that most races really lack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2559560820, "name": "Christine", "screen_name": "kisocolee", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "ⓁⒶ ~ Dreamville ~ ♒️", "followers_count": 495, "friends_count": 331, "statues_count": 18878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200807045267456, "text": "The coolest kids you know! #erobyouth #elevate16 @ Crowne Plaza Columbus-Dublin https://t.co/5DxpDVF1gb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.13322012,40.09635601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "erobyouth", "elevate16" }}, "user": { "id": 92708896, "name": "Teracyn Rich", "screen_name": "teracynrich", "lang": "en", "location": "Wheeling, WV", "create_at": date("2009-11-25"), "description": "22. Coffee. Grace. Jesus. Photography. Wanderlust. Etc...", "followers_count": 182, "friends_count": 176, "statues_count": 544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dublin, OH", "id": "2cb954ccdc0dba5f", "name": "Dublin", "place_type": "city", "bounding_box": rectangle("-83.18921,40.067746 -83.089352,40.182801") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3922694, "cityName": "Dublin" } }
+{ "create_at": datetime("2016-03-11T00:00:04.000Z"), "id": 708200807246667776, "text": "How much you Tryna pay? https://t.co/9LThY319dh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2508519234, "name": "#MetroTrustMe", "screen_name": "PoAFoWitYaHoe", "lang": "en", "location": "San Francisco, CA", "create_at": date("2014-05-19"), "description": "young water god ➕ pro hoe snatcher.", "followers_count": 24791, "friends_count": 16798, "statues_count": 6508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-11T00:00:05.000Z"), "id": 708200808228069376, "text": "@ChrisTreherne IT IS NOT TIME TO WAKE UP YET, STOP PLAYING DISNEY TUNES", "in_reply_to_status": -1, "in_reply_to_user": 36395938, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36395938 }}, "user": { "id": 172928095, "name": "Devin Renee Duesing", "screen_name": "DevinDuesing", "lang": "en", "location": "Birmingham, AL", "create_at": date("2010-07-30"), "description": "ACE Cheer Company Coach. Lover of Jesus, GIGs, and all things Crimson. Roll Tide!", "followers_count": 738, "friends_count": 555, "statues_count": 22373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563174950211584, "text": "@KZeenni fuck you", "in_reply_to_status": 708561418174332928, "in_reply_to_user": 1387247966, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1387247966 }}, "user": { "id": 2224065074, "name": "mo swirls", "screen_name": "moman208", "lang": "en", "location": "null", "create_at": date("2013-11-30"), "description": "if you a square eat a pear", "followers_count": 177, "friends_count": 199, "statues_count": 2539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563176720302080, "text": "FLIM FLAM BIM BAM OLE MISS BY DAMN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92664480, "name": "ann marie", "screen_name": "_theycallmeAM", "lang": "en", "location": "Oxford, MS", "create_at": date("2009-11-25"), "description": "I've been thinking too much. | tøp", "followers_count": 681, "friends_count": 386, "statues_count": 51728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muscle Shoals, AL", "id": "74d460734c3eca69", "name": "Muscle Shoals", "place_type": "city", "bounding_box": rectangle("-87.680639,34.705252 -87.617362,34.762527") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1033, "countyName": "Colbert", "cityID": 153016, "cityName": "Muscle Shoals" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563177043144704, "text": "In this negative world https://t.co/83KmOvUdFR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2581279676, "name": "HONCHO", "screen_name": "1bigkamm", "lang": "en", "location": "H Town Vibes", "create_at": date("2014-06-21"), "description": "|R.I.P Q WILL #31|SUPER SARID #28|\n ((HEAD HONCHO))", "followers_count": 307, "friends_count": 455, "statues_count": 1871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563177047330818, "text": "@DannicaKelley what do u want ��", "in_reply_to_status": 708563018146119680, "in_reply_to_user": 1364598673, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1364598673 }}, "user": { "id": 337993942, "name": "young hecty", "screen_name": "hector_upm", "lang": "en", "location": "CA.", "create_at": date("2011-07-18"), "description": "CSUF. πΚα. #FantoFamily", "followers_count": 1448, "friends_count": 935, "statues_count": 39467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563177429082112, "text": "let me skip my way into his dm������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1957800055, "name": "21", "screen_name": "yandollazzz", "lang": "en", "location": "null", "create_at": date("2013-10-12"), "description": "I play all day", "followers_count": 652, "friends_count": 239, "statues_count": 28455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563177609437184, "text": "now plz https://t.co/MThlJZgaV7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2492425405, "name": "jenna!", "screen_name": "_jenro", "lang": "en", "location": "null", "create_at": date("2014-05-12"), "description": "whatta?", "followers_count": 316, "friends_count": 311, "statues_count": 943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563177848483840, "text": "@blurryfced �� por que", "in_reply_to_status": 708562675354226688, "in_reply_to_user": 317573813, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 317573813 }}, "user": { "id": 270808874, "name": "diana", "screen_name": "temporaryfixmp3", "lang": "en", "location": "alejandra", "create_at": date("2011-03-23"), "description": "nearly always looks bored, or disapproving", "followers_count": 923, "friends_count": 154, "statues_count": 49450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563178171437057, "text": "@BetteMidler https://t.co/FHX4G06jjF", "in_reply_to_status": 708543395547848704, "in_reply_to_user": 139823781, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 139823781 }}, "user": { "id": 1412172103, "name": "Sister Lawna Jocqui", "screen_name": "SisterLawna", "lang": "en", "location": "Dallas, Texas", "create_at": date("2013-05-07"), "description": "Fully Professed Sister with The DFW Sisters of Perpetual Indulgence (R), a part of the Worldwide Order of the Sisters of Perpetual Indulgence (c).", "followers_count": 903, "friends_count": 1711, "statues_count": 3669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563178301448192, "text": "It's my birthday bitch ������������������������goodnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2894245985, "name": "kenszn.", "screen_name": "kennbeeeezzy", "lang": "en", "location": "null", "create_at": date("2014-11-26"), "description": "null", "followers_count": 353, "friends_count": 331, "statues_count": 6418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:00.000Z"), "id": 708563178746064896, "text": "Happy Birthday to my best friend and brother @Crespin_09 and happy birthday to my other brother @Anthoneee21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2369012456, 331179257 }}, "user": { "id": 1528587757, "name": "Allan Arevalo", "screen_name": "Alan_0708", "lang": "en", "location": "null", "create_at": date("2013-06-18"), "description": "null", "followers_count": 222, "friends_count": 213, "statues_count": 1827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563179777822720, "text": "@ebbtideapp Tide in Longview, Washington 03/12/2016\n Low 1:34am 0.0\nHigh 5:54am 0.0\n Low 2:16pm -0.0\nHigh 6:36pm 0.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.95,46.1"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 40, "friends_count": 1, "statues_count": 15111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rainier, OR", "id": "d338e67d518d0ff2", "name": "Rainier", "place_type": "city", "bounding_box": rectangle("-122.978416,46.081078 -122.913137,46.10765") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41009, "countyName": "Columbia", "cityID": 4160850, "cityName": "Rainier" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180201488384, "text": "Goodnight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341394622, "name": "WillyBandz", "screen_name": "DamnnWill", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-07-24"), "description": "Dont hate the sauce cause you got lost #RestInParadiseJuneMarie #Mosthated™ #Thepack |", "followers_count": 1884, "friends_count": 1613, "statues_count": 68438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180205662208, "text": "summer wya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1327896758, "name": "jo beans", "screen_name": "pintofrijol", "lang": "en", "location": "null", "create_at": date("2013-04-04"), "description": "try me", "followers_count": 398, "friends_count": 270, "statues_count": 34561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180486787072, "text": "03/12@03:00 - Temp 41.2F, WC 41.2F. Wind 1.0mph ENE, Gust 2.0mph. Bar 30.365in, Falling slowly. Rain 0.00in. Hum 76%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180654501888, "text": "@Maddiedigsyou_ kinda think we should hangout soon ��", "in_reply_to_status": 708353575093555204, "in_reply_to_user": 798638378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 798638378 }}, "user": { "id": 327265275, "name": "Andrea Wilkinson", "screen_name": "Ann_Dree_Uhhh", "lang": "en", "location": "null", "create_at": date("2011-07-01"), "description": "Dre", "followers_count": 1137, "friends_count": 863, "statues_count": 10441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180675411968, "text": "Don't fuck w/ those who don't save my number��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316154975, "name": "KΞV", "screen_name": "kevinsalinas98", "lang": "en", "location": " ig || kevinsalinas98", "create_at": date("2011-06-12"), "description": "|нтχ| Kchs 17'", "followers_count": 1227, "friends_count": 283, "statues_count": 1132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563180792971264, "text": "hope sleep finds me soon.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414304133, "name": "darling nikki✨", "screen_name": "NoHablaFXCKSHXT", "lang": "en", "location": "finding myself.", "create_at": date("2011-11-16"), "description": "young and ambitious with big dreams ✨ she believed she could so she did. | u a queen girl. | Psalms 46:5 | your future CRNA ✂ | #AUM19 | #RipTevin", "followers_count": 2047, "friends_count": 1012, "statues_count": 234764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phenix City, AL", "id": "66c034beedd48294", "name": "Phenix City", "place_type": "city", "bounding_box": rectangle("-85.06093,32.345401 -84.9918,32.531343") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1113, "countyName": "Russell", "cityID": 159472, "cityName": "Phenix City" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563181136965632, "text": "Wind 0.0 mph WNW. Barometer 30.122 in, Falling Rapidly. Temperature 52.5 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563181300396032, "text": "Congrats on the win @ShaneMosley_Jr https://t.co/VJ6dhV1P1g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 198764039 }}, "user": { "id": 38552438, "name": "Mike Brown", "screen_name": "MikeBrownMOD", "lang": "en", "location": "La Habra, CA", "create_at": date("2009-05-07"), "description": "Entrepreneur, Competitor, Business Builder, and Mentor. Internet Marketing & Automotive Customization Expert. Co-Founder & CEO of @ModBargains", "followers_count": 257, "friends_count": 279, "statues_count": 931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563181384429568, "text": "Wind 0.0 mph ---. Barometer 30.327 in, Falling. Temperature 23.8 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563181770297344, "text": "1d3f7bc85bb07272d964e2760d70f1de1af55414bf237d2fad88104932529e75f613b52bb0773a89eade48f787e2f4cfa2ad92bde97d3d896be8852a3e3e03ba", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-144.376394,63.687055"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 429803867, "name": "MarsBots", "screen_name": "MarsBots", "lang": "en", "location": "Mars", "create_at": date("2011-12-06"), "description": "null", "followers_count": 88, "friends_count": 0, "statues_count": 459190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2240, "countyName": "Southeast Fairbanks", "cityID": 219720, "cityName": "Dot Lake" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563182193872896, "text": "Wind 2.0 mph ENE. Barometer 30.198 in, Falling. Temperature 52.5 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563182214848512, "text": "@SoundRemedy you ever gonna drop a heavy song?don't get me wrong I'm fuckin in love with ur sound, but with ur goal for all these new songs,", "in_reply_to_status": 708541785056739328, "in_reply_to_user": 50130045, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50130045 }}, "user": { "id": 1648795891, "name": "(^^^) craig (◕,,,◕)", "screen_name": "gregums_gloves", "lang": "en", "location": "Lonely Mountain", "create_at": date("2013-08-05"), "description": "Wether you're being laughed at, or laughed with, you made someone smile. Be fuckin' weird.| Finger stylist/tutter/glover/Dancer #EDMfam", "followers_count": 318, "friends_count": 167, "statues_count": 18112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563182764343296, "text": "Dancing with the DJ! @ Subrosa https://t.co/5RUTiHtHVY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0073364,40.7395135"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19343695, "name": "Royal Singleton", "screen_name": "royalsingleton", "lang": "en", "location": "New York City", "create_at": date("2009-01-22"), "description": "♫ Record Producer, Audio Engineer, DJ, Songwriter. EmCee, CEO of Shabar Music Entertainment, from the BX, NYC ∞POWER∞ https://soundcloud.com/royalsingleton", "followers_count": 1087, "friends_count": 2028, "statues_count": 5652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563182869151744, "text": "Temp: 60.8°F Wind:0.0mph Pressure: 30.039hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-12T00:00:01.000Z"), "id": 708563183187988480, "text": "#SupportOriginMelissa 51.4°F Wind:0.0mph Pressure: 30.15hpa Falling slowly Rain Today 0.00in. Forecast: Fine, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 312164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563183313616897, "text": "Fuck 12 https://t.co/TtYAVx59JC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342954877, "name": "Yeezu$ Hendrix", "screen_name": "Jesus_512", "lang": "en", "location": "null", "create_at": date("2011-07-26"), "description": "I stay highed up roasted, plotting on my fortune - Young Roddy #JetLife ✈", "followers_count": 742, "friends_count": 1041, "statues_count": 8257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor, TX", "id": "7793efb73edea52a", "name": "Taylor", "place_type": "city", "bounding_box": rectangle("-97.446645,30.536252 -97.382435,30.614587") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4871948, "cityName": "Taylor" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563183925989376, "text": "what a great busy day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337549133, "name": "Jules", "screen_name": "Checknojulio", "lang": "en", "location": "Missing Khayyam", "create_at": date("2014-02-10"), "description": "null", "followers_count": 551, "friends_count": 803, "statues_count": 9175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563184680964096, "text": "It would start pouring as soon as I get to the H.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2288506380, "name": "AiteLilBro", "screen_name": "ayethatsplay", "lang": "en", "location": "Commerce ,Tx", "create_at": date("2014-01-12"), "description": "Made it out of poverty", "followers_count": 550, "friends_count": 423, "statues_count": 6456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563184723034112, "text": "Don't wanna go home lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 588326300, "name": "⚡️", "screen_name": "DiegoC_19", "lang": "en", "location": "SW Detroit", "create_at": date("2012-05-23"), "description": "Durango Detroit ✈️ Colorado", "followers_count": 788, "friends_count": 694, "statues_count": 8922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563184878116864, "text": "Lol gotta stop staying up this late", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377722322, "name": "Franky J.", "screen_name": "FrankyJohnson19", "lang": "en", "location": "Sinking Spring, PA-Prov, RI", "create_at": date("2011-09-21"), "description": "“No person has the right to rain on your dreams” ― Martin Luther King Jr. JWU '17", "followers_count": 558, "friends_count": 521, "statues_count": 21609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185146552321, "text": "I fucked up on a 10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326571952, "name": "Jeremy Hatley", "screen_name": "AyyeeJerm94", "lang": "en", "location": "null", "create_at": date("2011-06-29"), "description": "professional pb&j maker, burger connoisseur #TAMUCC16 #BrokePhiBroke", "followers_count": 1219, "friends_count": 898, "statues_count": 33832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185238933504, "text": "Everything is spinning.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 867259908, "name": "Mayla Purpura", "screen_name": "Maylarpurpura", "lang": "en", "location": "null", "create_at": date("2012-10-07"), "description": "Tampa", "followers_count": 236, "friends_count": 341, "statues_count": 3658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185507303424, "text": "@MsGirlAlmighty I liked his hand one actually. Face is just my limit. I just can't haha \nI don't understand why he would do that?", "in_reply_to_status": 708562969752260608, "in_reply_to_user": 52627823, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52627823 }}, "user": { "id": 317377372, "name": "Danielle", "screen_name": "Danielle_Joy1", "lang": "en", "location": "null", "create_at": date("2011-06-14"), "description": "21 &1D AF. Avid enthusiast of Harry Styles. Preorder Made In The A.M. (Deluxe Edition) by One Direction https://itun.es/us/S-7_9", "followers_count": 2429, "friends_count": 2585, "statues_count": 29250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185515671553, "text": "Oh my God. I've been waiting so long to see @jamielawsonuk ������������ https://t.co/5r5aDTPxA8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57678632 }}, "user": { "id": 3633904932, "name": "Sharmina", "screen_name": "sharminamolla", "lang": "en", "location": "null", "create_at": date("2015-09-20"), "description": "null", "followers_count": 13, "friends_count": 33, "statues_count": 236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Creek, CA", "id": "a35b62af9d82aa08", "name": "Walnut Creek", "place_type": "city", "bounding_box": rectangle("-122.10003,37.842368 -121.980829,37.944965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 683346, "cityName": "Walnut Creek" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185528213504, "text": "I've been at work since 2...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61131749, "name": "paige", "screen_name": "paigero13", "lang": "en", "location": "null", "create_at": date("2009-07-28"), "description": "null", "followers_count": 278, "friends_count": 166, "statues_count": 7509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185771544576, "text": "I want movie night but it's 2 am wah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2333913300, "name": "Maria Matt", "screen_name": "mariamatt2015", "lang": "en", "location": "The Burg", "create_at": date("2014-02-08"), "description": "AOII•UCM•Mac Miller•Bob's Burgers• \n@JoeSpreck", "followers_count": 566, "friends_count": 444, "statues_count": 6147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warrensburg, MO", "id": "95f975ecaafb76ad", "name": "Warrensburg", "place_type": "city", "bounding_box": rectangle("-93.800937,38.72244 -93.69101,38.785966") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29101, "countyName": "Johnson", "cityID": 2977092, "cityName": "Warrensburg" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563185956048897, "text": "@oKielvhecklero @audreyisfab @NicholasBager we know kiel", "in_reply_to_status": 708562964840747008, "in_reply_to_user": 37353790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37353790, 1032339835, 2148057414 }}, "user": { "id": 123472105, "name": "Rog dad", "screen_name": "Browndaddy69", "lang": "en", "location": "null", "create_at": date("2010-03-15"), "description": "Flooring professional. take care of your wife or Rog will", "followers_count": 144, "friends_count": 190, "statues_count": 13125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563186463731712, "text": "Los Angeles Memorial Coliseum (capacity 93,607) https://t.co/yPRiAnSKQG #stade #losangeles #california #unitedstates https://t.co/Z0s7pvP5nC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.287781,34.014168"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stade", "losangeles", "california", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 890, "friends_count": 1296, "statues_count": 3637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563186732158976, "text": "Need parts in all that.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513374468, "name": "A large black male", "screen_name": "ilovebiggirls_", "lang": "en", "location": "the prison in my mind ", "create_at": date("2012-03-03"), "description": "Do you want to be happy...", "followers_count": 1548, "friends_count": 2053, "statues_count": 75618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563187050934272, "text": "I think the greatest thing about college is getting woken up at 3 am by a fire alarm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1029673130, "name": "Emily Haglund", "screen_name": "em8_15ily", "lang": "en", "location": "null", "create_at": date("2012-12-22"), "description": "W&J '19|you can probably find me in my bed watching Mulan", "followers_count": 190, "friends_count": 205, "statues_count": 5573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, PA", "id": "0097a1de4d520b9a", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-80.327525,40.131709 -80.208588,40.200796") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4281328, "cityName": "Washington" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563187168313344, "text": "Yeeees! https://t.co/GgKxhKVHNF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 358022496, "name": "Melissa McCarty", "screen_name": "MelissaMcCarty1", "lang": "en", "location": "Fantasy Island", "create_at": date("2011-08-19"), "description": "#TVHost, #Reporter, #Actress & #Author of 'NEWS GIRLS DON'T CRY', #HuffPostBlogger #Addiction & #Mental #Illness advocate, Mind Ninja!", "followers_count": 8667, "friends_count": 517, "statues_count": 2757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:02.000Z"), "id": 708563187222781952, "text": "@matthewdavidsc Being hungry", "in_reply_to_status": 708521435732381696, "in_reply_to_user": 799603645, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 799603645 }}, "user": { "id": 701496975296835585, "name": "Nadine Singson", "screen_name": "singsonadine", "lang": "en", "location": "Houston, TX", "create_at": date("2016-02-21"), "description": "Live passionately ❤️", "followers_count": 95, "friends_count": 115, "statues_count": 135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563187679924224, "text": "Wind 0.0 mph SW. Barometer 30.40 in, Steady. Temperature 29.8 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563188091101185, "text": "And getting superiors at high school performance evaluations https://t.co/tB09lKAXKM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58549460, "name": "血液ムーン Pt. 2", "screen_name": "kerrybtone", "lang": "en", "location": "Deep C swats ", "create_at": date("2009-07-20"), "description": "Organized Chaos is a Virtue filled with Naïve arrogance and stoic resolution. It's not Yellow fever, it's small pox. It's not a Worm, it's just Malware.", "followers_count": 690, "friends_count": 388, "statues_count": 96913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Point, GA", "id": "dfb4df427ea8a2d6", "name": "East Point", "place_type": "city", "bounding_box": rectangle("-84.535534,33.609487 -84.415048,33.705685") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1325720, "cityName": "East Point" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563188397305857, "text": "Can you find Palo Alto on the map? Just try it at https://t.co/ddgu5J9Ne0 #Palo Alto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.143,37.4419"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Palo" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2644822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palo Alto, CA", "id": "3ad0f706b3fa62a8", "name": "Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.190523,37.362824 -122.097537,37.465918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 655282, "cityName": "Palo Alto" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563188749500419, "text": "@onpaperplanes https://t.co/HhABO5wZN1", "in_reply_to_status": 708559700699127808, "in_reply_to_user": 324746497, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 324746497 }}, "user": { "id": 452767127, "name": "Liitle Delicious", "screen_name": "WhitegirlKy", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "UNLV's Finest ,Crenshaw Alumni", "followers_count": 1857, "friends_count": 748, "statues_count": 6499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563189319897088, "text": "I'm definitely gonna be at the movie theaters when the new Captain America movie comes out����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940655052, "name": "Ronaldo⚡️", "screen_name": "_fakeasseddie", "lang": "en", "location": "null", "create_at": date("2014-12-23"), "description": "LHA C/O '15 IG: _fakeasseddie | BlowMoney", "followers_count": 248, "friends_count": 219, "statues_count": 7935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563189546364928, "text": "@KingMiller2012 one of these days you'll get it lmao.", "in_reply_to_status": 708562807667568640, "in_reply_to_user": 586972531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 586972531 }}, "user": { "id": 2890309572, "name": "Caty", "screen_name": "82498_", "lang": "en", "location": "Canton, TX", "create_at": date("2014-11-04"), "description": "Free Taylor D", "followers_count": 475, "friends_count": 437, "statues_count": 7140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canton, TX", "id": "461f451fe99bab6b", "name": "Canton", "place_type": "city", "bounding_box": rectangle("-95.893341,32.523476 -95.844021,32.588065") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48467, "countyName": "Van Zandt", "cityID": 4812496, "cityName": "Canton" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563189756112896, "text": "They're certain songs that I can't listen too Bc of my ex .. Like don't put them on Bc the river starts flowing �� even hip hop songs ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2441079511, "name": "jazzlizle", "screen_name": "jassleenpeace", "lang": "en", "location": "Del Rio, TX", "create_at": date("2014-04-12"), "description": "If we breakup or are no longer speaking,stay the fuck away from all my social media| I just like to lift and eat Pizza", "followers_count": 1033, "friends_count": 845, "statues_count": 21165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del Rio, TX", "id": "e34ace6177dd4848", "name": "Del Rio", "place_type": "city", "bounding_box": rectangle("-100.936202,29.332641 -100.808438,29.420173") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48465, "countyName": "Val Verde", "cityID": 4819792, "cityName": "Del Rio" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563189764456448, "text": "Be there soon enough. ���� https://t.co/oYRdQi64mp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184897732, "name": "The Kid ☄", "screen_name": "MjTheKid_", "lang": "en", "location": "Bay Area, USA", "create_at": date("2013-02-15"), "description": "#vT It was fun for a while then I got over it. #MD2B", "followers_count": 737, "friends_count": 515, "statues_count": 15814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563189894483968, "text": "Hindi Po Lola wahahaha aayyy joke Lang Po #VoteALDENonMYX https://t.co/bDA1ZkCf93", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "VoteALDENonMYX" }}, "user": { "id": 87395753, "name": "lashei", "screen_name": "SheaDMe", "lang": "en", "location": "Duarte, CA", "create_at": date("2009-11-04"), "description": "Dreamer.... Member of @ofcaldub_calif , Sub chapter @ofcaldub_LAcali", "followers_count": 97, "friends_count": 289, "statues_count": 5165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563190934679552, "text": "Happy birthday @allihurd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2664703364 }}, "user": { "id": 1016430151, "name": "Johnny Ramirez", "screen_name": "Johram515", "lang": "en", "location": "Hacienda Heights ", "create_at": date("2012-12-16"), "description": "null", "followers_count": 300, "friends_count": 116, "statues_count": 251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563190980812801, "text": "Miss him, but also hopes he gets a mildly irritating STD....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327460342, "name": "simplyyysabrinaaa", "screen_name": "itssabrinabetch", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "Almost always avoid alliteration", "followers_count": 179, "friends_count": 283, "statues_count": 2737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Viejo, CA", "id": "6b55df049258bade", "name": "Mission Viejo", "place_type": "city", "bounding_box": rectangle("-117.695948,33.534941 -117.617598,33.672075") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 648256, "cityName": "Mission Viejo" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563191274442752, "text": "Fun night and in bed by midnight, ideal old man timeline. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33738121, "name": "Andrew Graham", "screen_name": "MistaGraham", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2009-04-20"), "description": "Talent manager, co-founder of @heardwell and @NewhouseSU alum. The sense and nonsense here is solely mine.", "followers_count": 9891, "friends_count": 888, "statues_count": 10346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:03.000Z"), "id": 708563191513686016, "text": "Thank you so much Jazmineos for coming to support me at the dance… https://t.co/bz3ZiUAMTz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.09659827,33.88566201"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365541497, "name": "Juanita Angelina", "screen_name": "LuvDance07", "lang": "en", "location": "Santa Fe Springs, CA", "create_at": date("2011-08-31"), "description": "Teacher Nanny Dancer Live, Love, Dance♡", "followers_count": 138, "friends_count": 201, "statues_count": 3848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563191748542464, "text": "I'm at J&S No. 3 in Montebello, CA https://t.co/StadYaxYWj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.12962623,34.03221766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 10929372, "name": "TsunamiSkweek", "screen_name": "TsunamiSkweek", "lang": "en", "location": "null", "create_at": date("2007-12-06"), "description": "Eating in a dream world, with a belly bursting.", "followers_count": 271, "friends_count": 949, "statues_count": 23785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563191840710658, "text": "can't tel y'all how many relationships I've ruined. but I don't even try to. I fucc with a girl & take her out then find out she HAD a bf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1835582624, "name": "JayKen ++®", "screen_name": "JayTweetsWavy", "lang": "en", "location": "810•VEGAS ", "create_at": date("2013-09-08"), "description": "Graphic Designer • 2020®Clothing • #200kClub Contact: 6ickdesigns2020@gmail.com ", "followers_count": 3559, "friends_count": 122, "statues_count": 80444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563192293666817, "text": "Ily ���� https://t.co/OekEOA0HW0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2213371099, "name": "Sarah Marangon", "screen_name": "sarahmarangon", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "WSU•AGD", "followers_count": 599, "friends_count": 320, "statues_count": 7942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563192348278785, "text": "Travelling to Palo Alto or just twittering about Palo Alto? https://t.co/ddgu5J9Ne0 #Palo Alto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.143,37.4419"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Palo" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2644823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palo Alto, CA", "id": "3ad0f706b3fa62a8", "name": "Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.190523,37.362824 -122.097537,37.465918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 655282, "cityName": "Palo Alto" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563192700481537, "text": "She will never be me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2964461666, "name": "Mariah", "screen_name": "7_fijiwater_7", "lang": "en", "location": "null", "create_at": date("2015-01-07"), "description": "just your typical Aquarius", "followers_count": 490, "friends_count": 371, "statues_count": 10468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563192838901760, "text": "i never be up this late", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2414666458, "name": "yfn tre", "screen_name": "trapAvillieTre", "lang": "en", "location": "out west boolin", "create_at": date("2014-03-16"), "description": "self motivated.... nmsu commit✈️. #nmsu20", "followers_count": 450, "friends_count": 285, "statues_count": 2813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563193182838786, "text": "Y'all think I look Mexican?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2978621354, "name": "Clecka", "screen_name": "blecka6ix", "lang": "en", "location": "#DaClub", "create_at": date("2015-01-12"), "description": "Single moms hmu *hint hint*", "followers_count": 871, "friends_count": 236, "statues_count": 20290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563193237344256, "text": "I have a lot of passion about hating things", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 213384454, "name": "christina schroeter", "screen_name": "nineteenfemale", "lang": "en", "location": "los angeles", "create_at": date("2010-11-08"), "description": "the girl in family of the year. snap nineteenfemale", "followers_count": 1409, "friends_count": 333, "statues_count": 9212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563193270919168, "text": "Everyone about to hit up Cazino to see Froy lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951387856, "name": "Paulina❥", "screen_name": "paulinaaxoxo___", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 1121, "friends_count": 431, "statues_count": 16813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitney, NV", "id": "cb5ac954e9b77fba", "name": "Whitney", "place_type": "city", "bounding_box": rectangle("-115.064609,36.074534 -115.010215,36.137196") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3283800, "cityName": "Whitney" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563194659274752, "text": "https://t.co/gZsS731Wzp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3164731538, "name": "corrupt of the state", "screen_name": "corrupdate", "lang": "en", "location": "INSIDE", "create_at": date("2015-04-19"), "description": "lead of the state", "followers_count": 584, "friends_count": 146, "statues_count": 25847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:04.000Z"), "id": 708563195380703232, "text": "there's so many frat boys at this Whataburger where is @KaylinJonesy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71652420 }}, "user": { "id": 3378518712, "name": "party boy", "screen_name": "starlust96", "lang": "en", "location": "Austin, TX", "create_at": date("2015-08-28"), "description": "I hate dolphins", "followers_count": 512, "friends_count": 446, "statues_count": 9372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563196005654528, "text": "63.0F (Feels: 63.0F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1021.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 230295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563196358094848, "text": "I need sleep but I need to stay awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2928992000, "name": "Sandy", "screen_name": "chavezsandyy", "lang": "en", "location": "Nebraska Wesleyan University", "create_at": date("2014-12-13"), "description": "snapchat&instagram| Chavezsandy", "followers_count": 338, "friends_count": 231, "statues_count": 1259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Des Moines, IA", "id": "4a9aecc7ad6b85e6", "name": "West Des Moines", "place_type": "city", "bounding_box": rectangle("-93.846184,41.528008 -93.703326,41.600507") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1983910, "cityName": "West Des Moines" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563197209370624, "text": "@TommyGunnX how I will look when I'm coming up the the person the killed my niece", "in_reply_to_status": 708560579942678528, "in_reply_to_user": 22815836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22815836 }}, "user": { "id": 22815836, "name": "TOMMY GUNN", "screen_name": "TommyGunnX", "lang": "en", "location": "Pasadena CA ", "create_at": date("2009-03-04"), "description": "Adult Film Star, Beginning in1994 Sober since 6/1/88,Guitar Player,Biker,Original 9 of Son's Of Anarchy.extra on,CSI Las Vegas,The Mentalist,Gray's Anatomy", "followers_count": 31793, "friends_count": 11596, "statues_count": 17420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altadena, CA", "id": "fb76b3aa366004c9", "name": "Altadena", "place_type": "city", "bounding_box": rectangle("-118.177296,34.167539 -118.095091,34.220078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601290, "cityName": "Altadena" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563197574254592, "text": "He retweeting everything ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271236592, "name": "brittanyyw. ➰", "screen_name": "Brittanyyw__", "lang": "en", "location": "null", "create_at": date("2011-03-23"), "description": "shreveport. | iG: @brittanyyw__ | snapchat: @britnicwash |", "followers_count": 953, "friends_count": 636, "statues_count": 43009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563197842710528, "text": "@vickypadron417 but like more like how you came into mine tho", "in_reply_to_status": 708561962204966913, "in_reply_to_user": 510360008, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 344547163 }}, "user": { "id": 510360008, "name": "Camille De Angelo", "screen_name": "camilledeangelo", "lang": "en", "location": "Miami, FL", "create_at": date("2012-03-01"), "description": "tracin' her way through the constellation", "followers_count": 214, "friends_count": 141, "statues_count": 13968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall, FL", "id": "9b46dccb3cfb880c", "name": "Kendall", "place_type": "city", "bounding_box": rectangle("-80.389344,25.628844 -80.304896,25.715128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236100, "cityName": "Kendall" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198018985984, "text": "if you ain't climbing up the latter then hey it's time to reevaluate yourself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 508161507, "name": "LowerLowkey", "screen_name": "LowkeyTresside", "lang": "en", "location": "null", "create_at": date("2012-02-28"), "description": "@InTressWeTrust. ΩΔ #GrowWithMe", "followers_count": 473, "friends_count": 283, "statues_count": 26445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198090354688, "text": "I didnt tweet anything gross tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371087026, "name": "Disaronno Papi", "screen_name": "JtTheDream86", "lang": "en", "location": "Atlanta, GA via Charleston, SC", "create_at": date("2011-09-09"), "description": "R.I.P. #CharlestonNine\nHeart of a Lion, Mind of a Madman, Aquarius, Healthcare Worker, Uplifter, Life Lover, 80's Baby, Geechee Boy", "followers_count": 1358, "friends_count": 495, "statues_count": 91949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198270689281, "text": "Playing how many drunks can I trick into buying my friends and I drinks is working", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403089135, "name": "Jill", "screen_name": "jill_hoppe", "lang": "en", "location": "null", "create_at": date("2011-11-01"), "description": "null", "followers_count": 230, "friends_count": 284, "statues_count": 8624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198517989376, "text": "03:00 47.5°F Feels:47.5°F (Hi47.7°F/Lo45.0°F) Hum:89% Wnd:--- 0.0MPH Baro:29.94in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 192, "friends_count": 260, "statues_count": 26788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198648180737, "text": "Done Drinking If Its Not A Drink When Im Out To Eat I Dont Want It", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88458959, "name": "Gianni", "screen_name": "7thletter_gee", "lang": "en", "location": "null", "create_at": date("2009-11-08"), "description": "null", "followers_count": 1352, "friends_count": 458, "statues_count": 39102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buffalo, NY", "id": "a307591cd0413588", "name": "Buffalo", "place_type": "city", "bounding_box": rectangle("-78.912276,42.826008 -78.79485,42.966451") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3611000, "cityName": "Buffalo" } }
+{ "create_at": datetime("2016-03-12T00:00:05.000Z"), "id": 708563198769627136, "text": "Playing yourself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157221496, "name": "chief.", "screen_name": "JohnnySwvnk", "lang": "en", "location": "Nigeria", "create_at": date("2010-06-18"), "description": "roty.", "followers_count": 1317, "friends_count": 986, "statues_count": 44516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563200187437056, "text": "@MeninistTweet @ArmandoJalou", "in_reply_to_status": 708562377495617537, "in_reply_to_user": 2870078111, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2870078111, 402994106 }}, "user": { "id": 870272533, "name": "arie jordan", "screen_name": "aMargielaSnake", "lang": "en", "location": "ahhh, troy", "create_at": date("2012-10-09"), "description": "Everyone you will ever meet knows something that you don't.", "followers_count": 346, "friends_count": 168, "statues_count": 16715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, MI", "id": "4e284ea3fff91c09", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-83.209206,42.533782 -83.086881,42.624224") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2680700, "cityName": "Troy" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563201273692160, "text": "San Francisco: pt. 2 https://t.co/YEKNkSYdwP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 1035273132, "name": "Alyssah Patrice", "screen_name": "apftolentino", "lang": "en", "location": "16 months", "create_at": date("2012-12-25"), "description": "mutuals w/ Yan // IG: apftolentino", "followers_count": 376, "friends_count": 221, "statues_count": 2493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563202230001664, "text": "I'm stranded ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2639414619, "name": "torieyy", "screen_name": "vic_toriia3", "lang": "en", "location": "null", "create_at": date("2014-06-24"), "description": "null", "followers_count": 988, "friends_count": 713, "statues_count": 12893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563202259296256, "text": "@1AlejandraF_ I'm sorry ��", "in_reply_to_status": 708563150514167813, "in_reply_to_user": 2158189904, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2158189904 }}, "user": { "id": 4740480438, "name": "4 Days", "screen_name": "LeggoMyDiegoo_", "lang": "en", "location": "Tucson, AZ", "create_at": date("2016-01-10"), "description": "God is love ❤️", "followers_count": 360, "friends_count": 305, "statues_count": 1856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563202271936512, "text": "Here y'all go with tha \"wanna be laid up\" tweets.......if y'all don't lay y'all uglass down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35073721, "name": "$lim of Dem Boy$", "screen_name": "1Jimmy__", "lang": "en", "location": "Dallas, TX/Commerce, TX", "create_at": date("2009-04-24"), "description": "#LongLiveClyde / Dem Boy$ Live Forever / #TAMUC / Beats, Features, Bookings : 1SlimJimmy@gmail.com", "followers_count": 4093, "friends_count": 1027, "statues_count": 284305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563202573926400, "text": "Temp: 34.7°F - Dew Point: 29.2° - Wind: 2.2 mph - Gust: 4.5 - Rain Today: 0.00in. - Pressure: 30.19in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 6, "friends_count": 11, "statues_count": 14670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563202854952960, "text": "I'm falling asleep to the thought of hibachi grills tomorrow ���� https://t.co/R4TTCWsiUs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339537666, "name": "pimples and dimples", "screen_name": "irishniall_lu", "lang": "en", "location": "why do you want know", "create_at": date("2014-02-11"), "description": "I lost count of how many days till Sounds Live Feels Live so I'm just kinda sitting here", "followers_count": 583, "friends_count": 747, "statues_count": 7987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563203068801025, "text": "CHVRCHES & BAAUER SAVED ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 973084867, "name": "$lime", "screen_name": "dabossboobooo", "lang": "en", "location": "nola/br", "create_at": date("2012-11-26"), "description": "no water in the water fountain", "followers_count": 1163, "friends_count": 573, "statues_count": 48682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563203228246016, "text": "Soo bored ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433124617, "name": "El Domo", "screen_name": "DomFlash5", "lang": "en", "location": "Came From Da Bottom Of Mankind", "create_at": date("2011-12-09"), "description": "I just wanna be happy and watch Dragon Ball Z", "followers_count": 781, "friends_count": 842, "statues_count": 102590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elkhart, IN", "id": "43aa6d32149a6772", "name": "Elkhart", "place_type": "city", "bounding_box": rectangle("-86.061524,41.620816 -85.855849,41.739181") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18039, "countyName": "Elkhart", "cityID": 1820728, "cityName": "Elkhart" } }
+{ "create_at": datetime("2016-03-12T00:00:06.000Z"), "id": 708563203752665090, "text": "I've been awake since 6:50 am but I'm wide awake��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1513734672, "name": "Gεorgιε Sταmατιs™", "screen_name": "GeorgieStamatis", "lang": "en", "location": "Chicago", "create_at": date("2013-06-13"), "description": "Instagram & Snapchat- georgiestamatis", "followers_count": 714, "friends_count": 385, "statues_count": 4324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563204696186885, "text": "Wind 0.0 mph ---. Barometer 30.236 in, Falling Rapidly. Temperature 36.6 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 385, "friends_count": 292, "statues_count": 8932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563204700397568, "text": "meditation is ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 310920237, "name": "ct", "screen_name": "aerbalac", "lang": "en", "location": "Texas, USA", "create_at": date("2011-06-04"), "description": "worrying about me & mines @ShaneLaredo be groovy.", "followers_count": 1748, "friends_count": 688, "statues_count": 123737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, TX", "id": "012a07a105d10298", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-97.186747,32.530915 -97.046821,32.616182") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4846452, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563204721352704, "text": "3/11 gang you know how we rockin' ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785010888, "name": "QZ", "screen_name": "qcaylao3", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "Colorado WBB '20 #DI4I #TLF ig: quflossa", "followers_count": 1073, "friends_count": 554, "statues_count": 8427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563205178544128, "text": "3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1569853009, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "kylemetoyer_", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "i hate @sadieestricklin", "followers_count": 400, "friends_count": 293, "statues_count": 3542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563205212086273, "text": "@AlexOsorio_3 stop I hate you ��", "in_reply_to_status": 708559845981442048, "in_reply_to_user": 66489267, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66489267 }}, "user": { "id": 874637826, "name": "Jess ❄", "screen_name": "jessiguhh_", "lang": "en", "location": "htx // cstat", "create_at": date("2012-10-11"), "description": "let's avocuddle #TAMU19", "followers_count": 607, "friends_count": 442, "statues_count": 22319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563205614862336, "text": "PANIC IN L.A. W/ DJ Premier & DJ Craze #noeltruickphotography #iphoneography #true129 #preemo… https://t.co/Pjg0i6sZGk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.25924,34.04025"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "hashtags": {{ "noeltruickphotography", "iphoneography", "true129", "preemo" }}, "user": { "id": 102747425, "name": "Noel Truick", "screen_name": "true129", "lang": "en", "location": "Los Angeles", "create_at": date("2010-01-07"), "description": "DJ - Photographer -Graphic Artist - Grillmaster - Common Senser -", "followers_count": 251, "friends_count": 228, "statues_count": 5119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563205723967488, "text": "@Annalisamariahh this girl that do hair �� she scary I sent all that and she told me kiss her ass", "in_reply_to_status": 708562848721543168, "in_reply_to_user": 2934118572, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2934118572 }}, "user": { "id": 298367232, "name": "Jonadeezy", "screen_name": "jaayjaywest", "lang": "en", "location": "Los Angeles☀️✈⚜️New Orleans", "create_at": date("2011-05-13"), "description": "18| LA to the Core| SC: jaayjaywest", "followers_count": 2671, "friends_count": 1752, "statues_count": 129366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563206231314432, "text": "Essence>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 79555276, "name": "jo ☯", "screen_name": "josephinekris", "lang": "en", "location": "Manhattan, NY", "create_at": date("2009-10-03"), "description": "null", "followers_count": 545, "friends_count": 635, "statues_count": 8353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563207158255616, "text": "@latsondheimer Isn't it at The Pyramid...Isn't that big enough?", "in_reply_to_status": 708540025986678786, "in_reply_to_user": 50158116, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50158116 }}, "user": { "id": 735755186, "name": "Steve Ramirez", "screen_name": "SteveRRamirez", "lang": "en", "location": "Covina, CA", "create_at": date("2012-08-03"), "description": "Steve Ramirez writes for the Los Angeles News Group, covering high school sports. The opinions are my own.", "followers_count": 1927, "friends_count": 143, "statues_count": 20867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563207162609664, "text": "@olivia_roccos yeah melanin is what makes dark eyes or dark skin that's why usually white ppl people have blue eyes bc they have no melanin", "in_reply_to_status": 708562915553579008, "in_reply_to_user": 2152288282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2152288282 }}, "user": { "id": 3755255173, "name": "Romeo Anthonyz", "screen_name": "romeo_anthonys", "lang": "en", "location": "Manhattan, NY", "create_at": date("2015-10-01"), "description": "null", "followers_count": 188, "friends_count": 143, "statues_count": 4053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563207229730816, "text": "I wish the party didn't get shut down ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4174582573, "name": "Daddy", "screen_name": "flashbaker2", "lang": "en", "location": "Lost in the sauce ", "create_at": date("2015-11-08"), "description": "Ashtyn gets her wetter | Triple D | It's easy!snapchat:Flashbaker", "followers_count": 1405, "friends_count": 1150, "statues_count": 8451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-03-12T00:00:07.000Z"), "id": 708563207250530304, "text": "all niggas do is smoke all day & lie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290900988, "name": "twin.", "screen_name": "dnashaaaa", "lang": "en", "location": "Laffy, Louisiana", "create_at": date("2011-04-30"), "description": "sc: dnashasherae", "followers_count": 2349, "friends_count": 2158, "statues_count": 54740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925563100987393, "text": "Oh lawdd, I wish! https://t.co/K5iAvJd9BY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 1272752894, "name": "The Don", "screen_name": "RaffiG_Luxuries", "lang": "en", "location": "John Burroughs High School", "create_at": date("2013-03-16"), "description": "Inch asem", "followers_count": 243, "friends_count": 188, "statues_count": 28191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925563914866688, "text": "Malo je previše glupa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 469759063, "name": "Kevin Princ", "screen_name": "doktorcoa", "lang": "en", "location": "Odžino selo, Chicago IL", "create_at": date("2012-01-20"), "description": "Учим школу да будем директор и јебем секретарицу. Just don't tell anyone! http://favstar.fm/users/doktorcoa http://instagram.com/doktorcoa#", "followers_count": 754, "friends_count": 465, "statues_count": 21980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925564132806657, "text": "1/3 ain't bad https://t.co/1VDkpJDNLz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44252886, "name": "Marcus Stokes", "screen_name": "PopeQuanPaul", "lang": "en", "location": "Bronx 1st. Now Atlanta", "create_at": date("2009-06-02"), "description": "Beat it Feds. Don't fuck with me if you don't fuck with me. BLACK MAN", "followers_count": 1734, "friends_count": 948, "statues_count": 161055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925564476784640, "text": "I crave sex ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477916206, "name": "Taylor Adams", "screen_name": "Taylor_James19", "lang": "en", "location": "Houston", "create_at": date("2012-01-29"), "description": "Professional Golfer | UH Chemical Engineering", "followers_count": 1698, "friends_count": 128, "statues_count": 4706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Katy, TX", "id": "c484d6c5cd72beb0", "name": "Katy", "place_type": "city", "bounding_box": rectangle("-95.87417,29.736773 -95.774936,29.831219") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4838476, "cityName": "Katy" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925565013598208, "text": "We won the game!!! #StillGot #BeerPong #Yasss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StillGot", "BeerPong", "Yasss" }}, "user": { "id": 79890934, "name": "Bobii Wilson", "screen_name": "bobiiwlsn", "lang": "en", "location": "Henderson, NV", "create_at": date("2009-10-04"), "description": "I am one of the best people you'll ever meet.... and no matter what I can put a smile on your face ¯\\_(ツ)_/¯", "followers_count": 52, "friends_count": 67, "statues_count": 9342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bountiful, UT", "id": "d10ead505fcb25cb", "name": "Bountiful", "place_type": "city", "bounding_box": rectangle("-111.906716,40.834807 -111.834245,40.909783") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4907690, "cityName": "Bountiful" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925565013778432, "text": "@realDonaldTrump. Exceptional head swiveling by #Trump. Gotta be like #TomBradyBitch. They coming from all angles �� https://t.co/AkO4d01K5q", "in_reply_to_status": -1, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump", "TomBradyBitch" }}, "user_mentions": {{ 25073877 }}, "user": { "id": 150116475, "name": "Dr. siiig", "screen_name": "siigmund", "lang": "en", "location": "look behind U", "create_at": date("2010-05-30"), "description": "A life lived in fear is a life half lived ♑6'5 love beaches whisky bourbon & bitches. Sustaining desire by moving on... Until our paths cross again & so on...", "followers_count": 560, "friends_count": 1752, "statues_count": 16328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meadowbrook, AL", "id": "809982c795c5b7c8", "name": "Meadowbrook", "place_type": "city", "bounding_box": rectangle("-86.734213,33.369478 -86.673656,33.419962") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1117, "countyName": "Shelby", "cityID": 147740, "cityName": "Meadowbrook" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925565022109696, "text": "so tell me why I just randomly woke up & now I'm wide awake.. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 463821963, "name": "sydddd☻", "screen_name": "sydney_jordan22", "lang": "en", "location": "null", "create_at": date("2012-01-14"), "description": "snapchat// sydney_jordan22", "followers_count": 737, "friends_count": 583, "statues_count": 7858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buford, GA", "id": "5c002c164438e699", "name": "Buford", "place_type": "city", "bounding_box": rectangle("-84.051863,34.067677 -83.941599,34.168489") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1311784, "cityName": "Buford" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925565265272833, "text": "Happy birthday ❤️❤️❤️❤️ @madsnguyen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2179484827 }}, "user": { "id": 2996682578, "name": "Joshua Carrera", "screen_name": "JoshuaCarrera22", "lang": "en", "location": "null", "create_at": date("2015-01-25"), "description": "Play every game like it's your last", "followers_count": 296, "friends_count": 379, "statues_count": 2915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925565441589248, "text": "@SilkenSoul Yeah, definitely jealous then lol Gorgeous view!", "in_reply_to_status": 708924873318338560, "in_reply_to_user": 434356583, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 434356583 }}, "user": { "id": 1361698914, "name": "DH Gamer", "screen_name": "DHGamerMR", "lang": "en", "location": "Drifting through Limbo", "create_at": date("2013-04-18"), "description": "Enemy of the 37th Oompa Loompa Legion. Champion of Valhalla. #Coffee addict. #SnarkAlec No, really. Bring me #coffee.", "followers_count": 1845, "friends_count": 1964, "statues_count": 30417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethlehem, PA", "id": "128ae72e3854b273", "name": "Bethlehem", "place_type": "city", "bounding_box": rectangle("-75.4314,40.578043 -75.302993,40.672508") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4206088, "cityName": "Bethlehem" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925566360133632, "text": "Im gettin sick a this ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3120877469, "name": "TheBigAssCashKardash", "screen_name": "savannahhjbs", "lang": "en", "location": "Clearwater, FL", "create_at": date("2015-03-27"), "description": "Girls will be girls, & boys will be Ass holes.", "followers_count": 395, "friends_count": 190, "statues_count": 5098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearwater, FL", "id": "700eeb799fa55a4b", "name": "Clearwater", "place_type": "city", "bounding_box": rectangle("-82.831674,27.935178 -82.679007,28.050243") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1212875, "cityName": "Clearwater" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925566490030080, "text": "@woolystyles @jennallard she's asleep so I'll say it for her. HOLY COW SOPHIE YOURE AMAZING ✨", "in_reply_to_status": 708922440915689473, "in_reply_to_user": 1724786545, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1724786545, 23349549 }}, "user": { "id": 179797536, "name": "Gordy", "screen_name": "mytchsmitty", "lang": "en", "location": "null", "create_at": date("2010-08-17"), "description": "null", "followers_count": 1389, "friends_count": 529, "statues_count": 19668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, UT", "id": "f02825518e8d0371", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-111.699365,40.027014 -111.639757,40.081695") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4965770, "cityName": "Salem" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925566515216385, "text": "@edddiemarquez ������", "in_reply_to_status": 708925511200673792, "in_reply_to_user": 1118577769, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1118577769 }}, "user": { "id": 262233673, "name": "Q.", "screen_name": "_akathatNigga", "lang": "en", "location": "carson CA ", "create_at": date("2011-03-07"), "description": "@Hippie_Gabbaay❤️", "followers_count": 365, "friends_count": 473, "statues_count": 27798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925566628536320, "text": "Can yall tell me what's wrong with this picture? https://t.co/owOVaTIfIw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 740877866, "name": "Limpia Su Boca", "screen_name": "whizkhidK", "lang": "en", "location": "With The Old Young Folks", "create_at": date("2012-08-06"), "description": "Wash that dirt from ya mouth gente sucio!", "followers_count": 348, "friends_count": 193, "statues_count": 80097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Memphis, AR", "id": "017db07b3676900f", "name": "West Memphis", "place_type": "city", "bounding_box": rectangle("-90.237784,35.120479 -90.121655,35.231923") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5035, "countyName": "Crittenden", "cityID": 574540, "cityName": "West Memphis" } }
+{ "create_at": datetime("2016-03-13T00:00:00.000Z"), "id": 708925566846672896, "text": "https://t.co/8JJ7eEyhDB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3033702513, "name": "CJ Gray", "screen_name": "CJFron", "lang": "en", "location": "NOVA", "create_at": date("2015-02-12"), "description": "chill fam✌️", "followers_count": 36, "friends_count": 36, "statues_count": 128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leesburg, VA", "id": "594ec79d61a7f536", "name": "Leesburg", "place_type": "city", "bounding_box": rectangle("-77.602414,39.067605 -77.503682,39.136067") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5144984, "cityName": "Leesburg" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925567157051392, "text": "Out of many....\n#LitAF @ The New Parish https://t.co/GxW15U2xuG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.27275499,37.80782058"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LitAF" }}, "user": { "id": 17418108, "name": "Big Sista Gorgeous 1", "screen_name": "11Zette17", "lang": "en", "location": "L'amour n'a pas de, frontiere", "create_at": date("2008-11-15"), "description": "Product of Jamaica + Real Down 2 Mars Girl+ Scorpio + Facesty SOUTHERN Gurl + Univ of FL Alum + 6'1.5 Barefoot + Renegade + Creator+ Wordsmith= Zette...kinda", "followers_count": 1599, "friends_count": 1188, "statues_count": 119200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925567593132032, "text": "THIS BUS WE ARE HAVING IS ABOUT TO BE SO ON ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609414906, "name": "Based Boody", "screen_name": "BOODYPRIDE", "lang": "en", "location": "TAKEOVER HQ", "create_at": date("2012-06-15"), "description": "Supreme Leader of Chino Hills High School. We Are One", "followers_count": 1235, "friends_count": 744, "statues_count": 14447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925567924436992, "text": "My getting my life together and let me tell you how good it feels��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2252493240, "name": "Thi Tran", "screen_name": "thi_hee", "lang": "en", "location": "null", "create_at": date("2013-12-18"), "description": "Laugh and the world laughs with you.... Snore and you sleep alone", "followers_count": 591, "friends_count": 524, "statues_count": 4561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925568058839040, "text": "@Sasseeloo I need caffeine so bad lol", "in_reply_to_status": 708925496210358272, "in_reply_to_user": 135799674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 135799674 }}, "user": { "id": 486911156, "name": "Alexis", "screen_name": "LexieKOfficial", "lang": "en", "location": "Always Traveling", "create_at": date("2012-02-08"), "description": "It is what it is....", "followers_count": 1234, "friends_count": 1096, "statues_count": 77364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson, LA", "id": "782ef2b9d89f68d2", "name": "Jefferson", "place_type": "city", "bounding_box": rectangle("-90.208303,29.946409 -90.128094,29.97364") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2238145, "cityName": "Jefferson" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925568289533952, "text": "Just posted a photo @ Hawaii Convention Center https://t.co/tjquPMaMN2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.83659,21.28965"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 468811241, "name": "Sie Sie", "screen_name": "Siesie_p", "lang": "en", "location": "Hawaii, Honolulu", "create_at": date("2012-01-19"), "description": "Ashtin & Anastie 'Rae Madre Gym rat! Love being healthy and strong minded Positive aspect in life Smile through the rain", "followers_count": 263, "friends_count": 318, "statues_count": 4724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925568507584512, "text": "03/13@04:00 - Temp 47.7F, WC 47.7F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.030in, Falling slowly. Rain 0.00in. Hum 84%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925569099030529, "text": "Black to tally Monday #FAMU_RYS16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FAMU_RYS16" }}, "user": { "id": 1106442950, "name": "〽️♥️", "screen_name": "M_Nicolee18", "lang": "en", "location": "null", "create_at": date("2013-01-20"), "description": "#FAMU19", "followers_count": 1182, "friends_count": 764, "statues_count": 89646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925569153376257, "text": "@TBN https://t.co/qof77lBTWs https://t.co/e6w0yPWMpd", "in_reply_to_status": -1, "in_reply_to_user": 186613457, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 186613457 }}, "user": { "id": 4802259732, "name": "YAHSOUHA CHRIST", "screen_name": "YahsouhaChrist", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2016-01-23"), "description": "NUCLEAR WWIII FOR (KILLING AND HIDING) (THE 2ND COMING OF THE ETHIOPIAN (UFO) (2ND NUCLEAR CHRIST) OF ARMAGEDDON, (BRITISH COLONY OF ADEN) OF 1967 (SOUTH YEMEN)", "followers_count": 345, "friends_count": 2661, "statues_count": 16603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925569312759808, "text": "USGS reports a M0.46 #earthquake 6km NW of The Geysers, California on 3/13/16 @ 7:56:34 UTC https://t.co/Ba2mvO9O1M #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8061676,38.8160019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 5539, "friends_count": 17, "statues_count": 145939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925569824591872, "text": "Who up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381203134, "name": "23", "screen_name": "___Blove", "lang": "en", "location": "Badlandz®", "create_at": date("2011-09-27"), "description": "God... Family... Football", "followers_count": 1401, "friends_count": 1352, "statues_count": 12894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925569992237056, "text": "And boom I just lost an hour!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132340954, "name": "#TrumpsRacist", "screen_name": "From_Nothing", "lang": "en", "location": "Chicago, IL (River North)", "create_at": date("2010-04-12"), "description": "Husband | Father | Lover | ENFP | Speaker | Poly |#BlackLivesMatter | Love is the answer, we fail because we fear the equation. | #RunwayBella 4/11/16", "followers_count": 6126, "friends_count": 1381, "statues_count": 41975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570067763200, "text": "@ebbtideapp Tide in Vaughns Landing, Florida 03/13/2016\nHigh 5:44am 3.2\n Low 11:35am 0.7\nHigh 5:35pm 2.9\n Low 11:56pm 0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.5767,30.63"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 40, "friends_count": 1, "statues_count": 15389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yulee, FL", "id": "8a02ad7fbbf2bc8b", "name": "Yulee", "place_type": "city", "bounding_box": rectangle("-81.614086,30.569122 -81.565295,30.64398") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12089, "countyName": "Nassau", "cityID": 1279175, "cityName": "Yulee" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570067865600, "text": "#SupportOriginMelissa 61.5°F Wind:0.0mph Pressure: 29.91hpa Falling Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 312251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570541756417, "text": "Death can't stop the Dandy, baby. https://t.co/ySVcp9o5qn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 96256810, "name": "Jake V", "screen_name": "MetalShadowX", "lang": "en", "location": "Humble, TX", "create_at": date("2009-12-11"), "description": "A 24yo aspie with very simple interests: sarcasm, vidya games, cartoons, superheroes, comedy. Pokéfan & Dragon Ball nerd since 1998/99. A Smark as of Summer '15", "followers_count": 556, "friends_count": 550, "statues_count": 155770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570650755072, "text": "Let's not look past Bangladesh. Win vs Bangladesh and then worry about India. #WT20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WT20" }}, "user": { "id": 201814200, "name": "Adil Momin", "screen_name": "adilmomin786", "lang": "en", "location": "Houston, Texas, USA", "create_at": date("2010-10-12"), "description": "Benjamin Franklin — 'Those who surrender freedom for security will not have, nor do they deserve, either one.'", "followers_count": 584, "friends_count": 802, "statues_count": 45443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570768220160, "text": "3 https://t.co/VMKnS6sPCq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 468887300, "name": "Shawdy Kar$on✨", "screen_name": "LOVESHAWDY", "lang": "en", "location": "#TAMUC16 ", "create_at": date("2012-01-19"), "description": "21 | cancer ♋️ | sc: love.shawdy", "followers_count": 1006, "friends_count": 400, "statues_count": 64269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570789191680, "text": "Nobody knows how many times my xbox has been kicked over when I was a kid while in mid sesh. I've lost like 20 games from being scratched", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1832897132, "name": "JVelasquez", "screen_name": "Jbvelasquez4949", "lang": "en", "location": "Los Angeles, CA / Seattle, WA", "create_at": date("2013-09-08"), "description": "On your left. |Instagram|:@JVELASQUEZ24 @Kawaiith0t ❤️", "followers_count": 498, "friends_count": 673, "statues_count": 36205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Sacramento, CA", "id": "009243ac50ad0f26", "name": "West Sacramento", "place_type": "city", "bounding_box": rectangle("-121.584282,38.516197 -121.506483,38.603777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 684816, "cityName": "West Sacramento" } }
+{ "create_at": datetime("2016-03-13T00:00:01.000Z"), "id": 708925570805960704, "text": "@MaddieTumser MADDIE MY HEART���� https://t.co/3t7CGyvJFt", "in_reply_to_status": -1, "in_reply_to_user": 2760768424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2760768424 }}, "user": { "id": 768631656, "name": "shmemily☀️", "screen_name": "Emily_Bonser", "lang": "en", "location": "California, USA", "create_at": date("2012-08-19"), "description": "Be free.", "followers_count": 489, "friends_count": 993, "statues_count": 4595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925571078565888, "text": "USGS reports a M0.46 #earthquake 6km NW of The Geysers, California on 3/13/16 @ 7:56:34 UTC https://t.co/FDxPdYQBWr #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8061676,38.8160019"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 2654450299, "name": "NorCal Earthquakes", "screen_name": "NorCalEq", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "null", "followers_count": 82, "friends_count": 0, "statues_count": 8070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925571107950592, "text": "11:59 to 1 AM like that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167299274, "name": "i.", "screen_name": "olivjuic", "lang": "en", "location": "null", "create_at": date("2013-02-10"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀her olive.", "followers_count": 321, "friends_count": 151, "statues_count": 27501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlingame, CA", "id": "9cee92fec370baf9", "name": "Burlingame", "place_type": "city", "bounding_box": rectangle("-122.403936,37.568111 -122.330819,37.604114") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 609066, "cityName": "Burlingame" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925571124695040, "text": "\"Man has two great spiritual needs.\nOne is for forgiveness. \nThe other is for goodness.\"\nBilly Graham", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 141, "friends_count": 259, "statues_count": 236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925571703504896, "text": "3/13/16 3:00am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 446849734, "name": "Sarah Fuhrmeister", "screen_name": "iamsarahsmiles", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "you can do the impossible, because you've been through the unimaginable.", "followers_count": 570, "friends_count": 306, "statues_count": 34593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murphy, TX", "id": "015f7b57645e0238", "name": "Murphy", "place_type": "city", "bounding_box": rectangle("-96.642394,32.987106 -96.594535,33.049041") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4850100, "cityName": "Murphy" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925571858714625, "text": "Cutie���� @JacobDaneJohn https://t.co/1gRW7k6dy5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 395045988 }}, "user": { "id": 1437169004, "name": "♦️Kyn♦️", "screen_name": "KyndallWomble", "lang": "en", "location": "null", "create_at": date("2013-05-17"), "description": "null", "followers_count": 262, "friends_count": 191, "statues_count": 1513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572014067712, "text": "Temp: 62.8°F Wind:0.4mph Pressure: 29.878hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572047572993, "text": "Wind 0.0 mph N. Barometer 29.979 in, Rising. Temperature 32.9 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572382994433, "text": "Pongan su TV al 49.1 Si viven entre SD & Tijuana Xq #HoyLasPrincesas d @InfamesLaSerie como @Erendiritas están d regreso AHORA en @MundoMax!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "HoyLasPrincesas" }}, "user_mentions": {{ 459786026, 13663052, 27042973 }}, "user": { "id": 218949984, "name": "Alberto", "screen_name": "314LoveMachine", "lang": "es", "location": "Mexicali,BC,Mexico/SanDiego,CA", "create_at": date("2010-11-23"), "description": "I Don't use... / De paso yo No uso : Instagram. http://Facebook.com/314LoveMachine", "followers_count": 17745, "friends_count": 712, "statues_count": 22357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572634685440, "text": "USGS reports a M2.5 #earthquake 2km E of Hennessey, Oklahoma on 3/13/16 @ 7:40:18 UTC https://t.co/yWLtwHa1zR #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.8739,36.1112"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 5539, "friends_count": 17, "statues_count": 145940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma, USA", "id": "bd3d2074a33fbd06", "name": "Oklahoma", "place_type": "admin", "bounding_box": rectangle("-103.002652,33.615765 -94.431332,37.002328") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40073, "countyName": "Kingfisher" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572722786305, "text": "1:59 ➡️ 3:00 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 396038989, "name": "Mohamed A.", "screen_name": "Masryyy_", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-10-22"), "description": "University of New Orleans '19. SC: Masry09", "followers_count": 552, "friends_count": 642, "statues_count": 28288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Timberlane, LA", "id": "6e137e292946d911", "name": "Timberlane", "place_type": "city", "bounding_box": rectangle("-90.044486,29.8481 -90.01741,29.898203") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2275740, "cityName": "Timberlane" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925572857131008, "text": "Cute ginger guy cool pits ������ https://t.co/v95mWbPsAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4210, "friends_count": 3103, "statues_count": 44628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573033111553, "text": "It was too funny tho ���� https://t.co/cAoCXeLPxT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4658346271, "name": "Ugly Antho", "screen_name": "yt__antho", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "stay true to yourself", "followers_count": 1073, "friends_count": 898, "statues_count": 3879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573062524929, "text": "*heart cracks*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727129465, "name": "jessica", "screen_name": "xo___jess", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "you was just another nigga on my hit list", "followers_count": 722, "friends_count": 834, "statues_count": 7512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573255569408, "text": "Okay, I'm using snapchat yet again. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2350754280, "name": "lush n.", "screen_name": "hvlvt", "lang": "en", "location": "null", "create_at": date("2014-02-18"), "description": "Life aspiration is to be as brillant as Michael Pollan", "followers_count": 151, "friends_count": 199, "statues_count": 2011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, GA", "id": "005f284c5f674191", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-83.891081,33.569455 -83.813866,33.637881") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13217, "countyName": "Newton", "cityID": 1320064, "cityName": "Covington" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573536415744, "text": "@MAHOTONNN はい。YOUTUBEのコミュニティ・ガイドライン\nhttps://t.co/H4aGsJLGLi https://t.co/eSHcB7Zys9", "in_reply_to_status": 708922365296574464, "in_reply_to_user": 160560121, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 160560121 }}, "user": { "id": 2758475808, "name": "こっしー@自作PC(とは言っても組み立て", "screen_name": "hajimebucho", "lang": "ja", "location": "カンボジア", "create_at": date("2014-08-23"), "description": "フォローを整理しました。気に入った方(可愛い、機械好き、文房具好き、Youtuber)はフォローします LINE友達になれる人リプかDMお願いします。/人間/東京都の左のほう住み/文房具、機械好き/中2/府中八中", "followers_count": 430, "friends_count": 161, "statues_count": 2310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brazil, IN", "id": "c04138d310229f7e", "name": "Brazil", "place_type": "city", "bounding_box": rectangle("-87.188047,39.503132 -87.098537,39.538108") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18021, "countyName": "Clay", "cityID": 1807174, "cityName": "Brazil" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573557559296, "text": "Wind 1.0 mph ---. Barometer 29.794 in, Falling Rapidly. Temperature 57.7 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573570170882, "text": "life is great dude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4093176969, "name": "j-wheezy", "screen_name": "jordannstanleyy", "lang": "en", "location": "Evansville, IN", "create_at": date("2015-11-01"), "description": "trust no one", "followers_count": 230, "friends_count": 196, "statues_count": 2507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573595136000, "text": "Another birthday celebrated on the wavy west #blessed #22", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.239448,32.797358"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed" }}, "user": { "id": 458814557, "name": "DR", "screen_name": "1Drizz", "lang": "en", "location": " Long Island 631", "create_at": date("2012-01-08"), "description": "some study the roots while others pick the fruits. One:1 #ganggreen", "followers_count": 188, "friends_count": 203, "statues_count": 20966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925573997973504, "text": "Wind 0.0 mph ---. Barometer 29.900 in, Falling slowly. Temperature 61.2 °F. Rain today 0.03 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925574056509440, "text": "WAY FUN NIGHT https://t.co/x3f9ueinMQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2943414679, "name": "Sav", "screen_name": "sav_morris33", "lang": "en", "location": "null", "create_at": date("2014-12-25"), "description": "I mean I could but why would I want to", "followers_count": 226, "friends_count": 284, "statues_count": 167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925574308298753, "text": "60.3F (Feels: 60.3F) - Humidity: 99% - Wind: 3.1mph SE - Gust: 6.0mph - Pressure: 1017.6mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 230436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925574803283970, "text": "what do you see?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2267875881, "name": "otw.", "screen_name": "thempaws", "lang": "en", "location": "United States", "create_at": date("2014-01-07"), "description": "big bear. snapchat • east.memphis #TooUglyEnt", "followers_count": 205, "friends_count": 232, "statues_count": 12593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:02.000Z"), "id": 708925574983450624, "text": "https://t.co/zpleYDOc5W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 348779089, "name": "Chris Randle", "screen_name": "ChrisRandle_", "lang": "en", "location": "Dallas/Denton, TX", "create_at": date("2011-08-04"), "description": "DeAndre Jordan is a coward. #UNT #KeepPounding #MFFL", "followers_count": 1528, "friends_count": 880, "statues_count": 66607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925576032034817, "text": "Not into entertaining temporary people.. No point of even texting/calling/speaking/talking or hanging out. Too many snakes in the grass..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129384495, "name": "Walt of Dem Boy$", "screen_name": "AMBITIONBELL24", "lang": "en", "location": "Texas, USA", "create_at": date("2010-04-03"), "description": "Jarvis Christian College basketball | #LongLiveClyde Dem Boy$ Forever!", "followers_count": 1835, "friends_count": 908, "statues_count": 128932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Colony, TX", "id": "5f6eb89a27398e90", "name": "The Colony", "place_type": "city", "bounding_box": rectangle("-96.932509,33.048438 -96.850968,33.10959") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4872530, "cityName": "The Colony" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925576371777536, "text": "@dccdestiny I told you you can't fucking drive", "in_reply_to_status": 708925366534930432, "in_reply_to_user": 880760360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 880760360 }}, "user": { "id": 2817722680, "name": "Aaron Salazar", "screen_name": "aarons_44", "lang": "en", "location": "null", "create_at": date("2014-10-08"), "description": "I want it all and then some. SC aaron_s24 #JeepNation", "followers_count": 873, "friends_count": 715, "statues_count": 9400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victoria, TX", "id": "70f5587b3e27a105", "name": "Victoria", "place_type": "city", "bounding_box": rectangle("-97.045657,28.709293 -96.900168,28.895775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48469, "countyName": "Victoria", "cityID": 4875428, "cityName": "Victoria" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925576409526272, "text": "@Tiffy4u if it's any condolences sparkledragon, i don't get another free day til april 22nd", "in_reply_to_status": 708925311245680640, "in_reply_to_user": 15211585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15211585 }}, "user": { "id": 34259389, "name": "lance newman", "screen_name": "engineerhunter", "lang": "en", "location": "Santa Monica, CA 90405", "create_at": date("2009-04-22"), "description": "I tweet about staffing & sports usually. Used to springboard & platform dive for the University of Hawaii. Presently doing cool stuff @ http://tsearch.com", "followers_count": 1050, "friends_count": 1018, "statues_count": 40225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925576875253760, "text": "gente mia #serie56 #sanfranciscodemacoris - #leruffo gracias x el apoyo @ 809 Bar & Grill https://t.co/4OJCPTe4Lq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9253387,40.8619118"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "serie56", "sanfranciscodemacoris", "leruffo" }}, "user": { "id": 58402955, "name": "THIS SAT@809-", "screen_name": "LERUFFO", "lang": "en", "location": "new york", "create_at": date("2009-07-19"), "description": "EVENTS NYC-HEAD PROMOTOR@809lounge DAYS:THURSDAYS/SATURDAYS@809LOUNGE NY.FOLLOW ME@ INSTAGRAM@LERUFFO /ph-whatsapp 646-651-8791", "followers_count": 1936, "friends_count": 1666, "statues_count": 27664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925577223344128, "text": "@Nanam_xoxo ����", "in_reply_to_status": 708925531907956737, "in_reply_to_user": 139932266, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 139932266 }}, "user": { "id": 155118292, "name": "Rdrick B. Williams", "screen_name": "iAM_RodWill", "lang": "en", "location": "Shreveport/Grambling, LA", "create_at": date("2010-06-12"), "description": "Visual Arts major(digital). Stay humble, stay hungry. PatsNation. KnicksTape. ChaserGANG #GramFam", "followers_count": 4833, "friends_count": 1983, "statues_count": 455795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925577651216384, "text": "@zahrarsr عظم الله اجركم ورحم فقيدكم و غفر له ����", "in_reply_to_status": 708912431729401856, "in_reply_to_user": 3248232307, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 3248232307 }}, "user": { "id": 399629204, "name": "لؤي", "screen_name": "loayalzaabi", "lang": "en", "location": "University City, Philadelphia", "create_at": date("2011-10-27"), "description": "رب اوزعني ان اشكر نعمتك التي انعمت علي.", "followers_count": 1537, "friends_count": 2103, "statues_count": 6964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925577672003585, "text": "@HaileNation for $0.99 in-app purchase you also get a ranking of toilet paper quality", "in_reply_to_status": 708920936670502912, "in_reply_to_user": 567836046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 567836046 }}, "user": { "id": 327178001, "name": "Noah Johnson", "screen_name": "NizzTweets", "lang": "en", "location": "Lino Lakes, MN • Tempe, AZ", "create_at": date("2011-06-30"), "description": "My occasionally serious--often satirical--thoughts and rhetoric on sports, politics, pop-culture, and the like.", "followers_count": 657, "friends_count": 300, "statues_count": 24014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925577692971008, "text": "#tattoo #goals https://t.co/7j1PWt8ZG0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "tattoo", "goals" }}, "user": { "id": 19069942, "name": "RestivoYoga", "screen_name": "arestivo16", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-01-16"), "description": "Yoga For The REST Of Your Life! Recovery, Expression, Strength, Tadasana! http://Setandflow.com", "followers_count": 487, "friends_count": 1070, "statues_count": 1192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925577810419713, "text": "Wind 1.6 mph W. Barometer 30.01 in, Steady. Temperature 47.1 °F. Rain today 0.00 in. Humidity 48%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925578024452100, "text": "This nigh do everything lol & he was committed to Ohio state ������ https://t.co/sC4rc8qQ1p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977289264, "name": "TRAVION", "screen_name": "NoLimitTra", "lang": "en", "location": "null", "create_at": date("2012-11-28"), "description": "18 . NLMB", "followers_count": 503, "friends_count": 463, "statues_count": 12838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lee, VA", "id": "af543ab305d8cf8d", "name": "Fort Lee", "place_type": "city", "bounding_box": rectangle("-77.356787,37.212618 -77.307936,37.290882") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51149, "countyName": "Prince George", "cityID": 5129152, "cityName": "Fort Lee" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925578129203200, "text": "Fuck the time, I just want some head", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2263742480, "name": "shawty of god", "screen_name": "basicparis", "lang": "en", "location": "San Mo", "create_at": date("2013-12-26"), "description": "allahu akbar | #blacklivesmatter | texas state '19", "followers_count": 1650, "friends_count": 1193, "statues_count": 13773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925578183778305, "text": "As fuck. https://t.co/Q5vfhbDf4W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3062242471, "name": "Ⓜ️ook ✈️", "screen_name": "RellBandss_", "lang": "en", "location": "Oakland, CA", "create_at": date("2015-03-04"), "description": "Free D.a", "followers_count": 556, "friends_count": 584, "statues_count": 14663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925578653474817, "text": "Oh shit I just timed traveled!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1932558176, "name": "Elias", "screen_name": "Prietochiwas", "lang": "en", "location": "null", "create_at": date("2013-10-03"), "description": "Arriba la Sierra!! #FHSU", "followers_count": 188, "friends_count": 281, "statues_count": 3043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hays, KS", "id": "b0e50abc7e3f7363", "name": "Hays", "place_type": "city", "bounding_box": rectangle("-99.354684,38.849348 -99.280695,38.914552") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20051, "countyName": "Ellis", "cityID": 2031100, "cityName": "Hays" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579161112576, "text": "@RINOwithHorns @kevysmalls I WASNT SAYING UR INCORRECT. I STATED THAT SUCH A PROGRAM IS NOT RIGHT (APPROPRIATE)", "in_reply_to_status": 708924709828763648, "in_reply_to_user": 703366485297729539, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 703366485297729539, 191542255 }}, "user": { "id": 2393063119, "name": "Susan V Brown", "screen_name": "SassCBrown", "lang": "en", "location": "null", "create_at": date("2014-03-16"), "description": "#tcot #Christian Sinner saved by grace watchwoman Fed up with commie Taking America back 4 God Caregiver 2 mom with Alzheimer's Vets Wife Jewelry Designer #WAAR", "followers_count": 131793, "friends_count": 110416, "statues_count": 50392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579181973506, "text": "If that counts as excellent judgement, no wonder cops are gunning down so many innocent people. https://t.co/VgMHrST6xz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3293178390, "name": "Jim's Junk Box", "screen_name": "JamesIsaak2", "lang": "en", "location": "Contiguous USA and Canada", "create_at": date("2015-07-25"), "description": "Driving around making deliveries of stuff to people around the country. I have a lot of time to let my mind wander. That's rarely a good thing.", "followers_count": 237, "friends_count": 364, "statues_count": 13254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579244867585, "text": "@pascaljosh says the \"queen\" who probably hasn't driven 1k kms in her life", "in_reply_to_status": 708924720717037569, "in_reply_to_user": 2606447100, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2606447100 }}, "user": { "id": 383323, "name": "vacation boy", "screen_name": "zzap", "lang": "en", "location": "Melbourne, Australia", "create_at": date("2006-12-30"), "description": "buy me a coffee and I'm yours", "followers_count": 6960, "friends_count": 670, "statues_count": 133092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579345698816, "text": "Hamilton Pool (lac dans l'état du Texas) https://t.co/9L9dFzXaKP #hamiltonpool #lacetang #drippingsprings #texas https://t.co/VrqQTm5Boo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.12693,30.34227"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "hamiltonpool", "lacetang", "drippingsprings", "texas" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 894, "friends_count": 1299, "statues_count": 3666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579362476033, "text": "Can you find Atlanta on the map? Just try it at https://t.co/T9s29PRmIT #Atlanta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.388,33.749"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Atlanta" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2646093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-13T00:00:03.000Z"), "id": 708925579442032641, "text": "2 https://t.co/P8oERTEBoT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1569853009, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "kylemetoyer_", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "i hate @sadieestricklin", "followers_count": 404, "friends_count": 295, "statues_count": 3560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925580025028608, "text": "@TeeqoFaZe Hey Teeqo, Do you think that @FaZeClan should create personal FaZe Shoes? I think they would look dope af! Roshes, Air, etc. etc.", "in_reply_to_status": -1, "in_reply_to_user": 2389888200, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2389888200, 238431491 }}, "user": { "id": 3301823029, "name": "Luke Rivera", "screen_name": "LukeSkybreaker", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "#GodFirst Love one another, just how God Loves you. I love hanging out with my Amazing Friends #Isabellai #Angela #Faith #Rebecca #Kay #Arch #TD #Mikal #Costco", "followers_count": 309, "friends_count": 1030, "statues_count": 4864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Diamond Bar, CA", "id": "771f8196d3598fe8", "name": "Diamond Bar", "place_type": "city", "bounding_box": rectangle("-117.86323,33.959529 -117.767626,34.042345") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619192, "cityName": "Diamond Bar" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925580138323968, "text": "time to go bike to my old avi lls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597626721, "name": "ChuckDaCondorFanAcco", "screen_name": "_______lalo", "lang": "en", "location": " ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "create_at": date("2012-06-02"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 282, "friends_count": 77, "statues_count": 37610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soquel, CA", "id": "01f8789e60f75ea1", "name": "Soquel", "place_type": "city", "bounding_box": rectangle("-121.971311,36.982749 -121.92327,37.013149") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 672688, "cityName": "Soquel" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925580230598657, "text": "@Iam360WISE @MCHammer @GentryKozia @AceKayo your hope didn't work. Your change didn't work. Time to move on......and not dot org or BLM.", "in_reply_to_status": 708902757131001856, "in_reply_to_user": 282827622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 282827622, 6273552, 160306026, 142997115 }}, "user": { "id": 473085522, "name": "Matt Raley", "screen_name": "MattRaley81", "lang": "en", "location": "Florida", "create_at": date("2012-01-24"), "description": "US Marine who is currently a Banker. My favorite things in life are: 1.My Wife 2.My Son 3.Coaching my son in baseball 4.All LSU Sports", "followers_count": 50, "friends_count": 285, "statues_count": 386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frisco, TX", "id": "00c3e46c3d0cac08", "name": "Frisco", "place_type": "city", "bounding_box": rectangle("-96.937783,33.081206 -96.732715,33.219073") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4827684, "cityName": "Frisco" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925580482195456, "text": "@Lisa_Haven \nRevelation 1:7 - Behold, he cometh with clouds; and every eye shall see him, and they [also] which pierced him: and all", "in_reply_to_status": -1, "in_reply_to_user": 2290728422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2290728422 }}, "user": { "id": 56595028, "name": "Thea F.", "screen_name": "theame2", "lang": "en", "location": "null", "create_at": date("2009-07-13"), "description": "null", "followers_count": 198, "friends_count": 123, "statues_count": 15160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925580813725696, "text": "@FuckkAssNick need more reasons to use this picture. https://t.co/oJFfrUn07x", "in_reply_to_status": 708880193142689792, "in_reply_to_user": 310698221, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310698221 }}, "user": { "id": 2151850946, "name": "G.W.T.B.S", "screen_name": "ryantherkildsen", "lang": "en", "location": "null", "create_at": date("2013-10-23"), "description": "I fucking hate you", "followers_count": 226, "friends_count": 266, "statues_count": 1067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925581115568128, "text": "@fuckyouimcamron stuffy", "in_reply_to_status": 708924918759424000, "in_reply_to_user": 1069704794, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1069704794 }}, "user": { "id": 1069704794, "name": "camron", "screen_name": "fuckyouimcamron", "lang": "en", "location": "pacific beach, ca", "create_at": date("2013-01-07"), "description": "i am the liquor", "followers_count": 2945, "friends_count": 188, "statues_count": 53143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Bear Lake, CA", "id": "17fe6350a3570c69", "name": "Big Bear Lake", "place_type": "city", "bounding_box": rectangle("-116.962579,34.223549 -116.842586,34.262952") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 606434, "cityName": "Big Bear Lake" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925581140701184, "text": "WHY THESE NYGGAS BE GETTING SO PERSONAL.\nDONT NO ONE CARE BOUT U LMAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1632229310, "name": "$XSW|shyne", "screen_name": "OGShyne", "lang": "en", "location": "HTX", "create_at": date("2013-07-29"), "description": "FEMALE RAPPER. MAJOR PAYNE IS NOW ON ITUNES.MDRNSNDS.HOUSTOCK.|http://soundcloud.com/ogshyne", "followers_count": 1886, "friends_count": 923, "statues_count": 31529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925581216190465, "text": "Boys# she wildin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348785842, "name": "lil Stevie", "screen_name": "Stevie_kiss", "lang": "en", "location": "Texas sc: steven_kiss2", "create_at": date("2011-08-04"), "description": "SKINNY NIGGA RUNNING SHIT", "followers_count": 737, "friends_count": 619, "statues_count": 11049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925581539155968, "text": "THE BLOB FISH FH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3105343944, "name": "ㅤㅤ", "screen_name": "JorgeDeIuxe", "lang": "en", "location": "#BlackLivesMatter", "create_at": date("2015-03-23"), "description": "null", "followers_count": 633, "friends_count": 33, "statues_count": 15682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925581698662400, "text": "Temp: 66.2°F | Humidity: 95% | Wind: --- @ 0.0 mph | Barometer: 30.03 in | Dewpoint: 64.8°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 165835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925582571020290, "text": "I swear I'm always blamed for everything and I don't even know half the things that are happening��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2973418384, "name": "#iplay4dana", "screen_name": "jadaacruzz", "lang": "en", "location": "con mí chorros", "create_at": date("2015-01-11"), "description": "Dana Grace Housley", "followers_count": 408, "friends_count": 355, "statues_count": 832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925583045038081, "text": "GRACE IS A MEAN GIRL☹", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198458849, "name": "deyj", "screen_name": "deyjaahmccoo", "lang": "en", "location": "AZ", "create_at": date("2013-11-16"), "description": "dancin&finessin", "followers_count": 743, "friends_count": 303, "statues_count": 15353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925583107989506, "text": "This #woman @SimoneGiertz is building #BadRobots - Business Insider #Robots #STEM https://t.co/89IEiaAG2b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "woman", "BadRobots", "Robots", "STEM" }}, "user_mentions": {{ 897861037 }}, "user": { "id": 363240308, "name": "eastvillagetwt", "screen_name": "eastvillagetwt", "lang": "en", "location": "East Village, New York, NY", "create_at": date("2011-08-27"), "description": "Highlighting interesting things about the EastVillage, world, politics, memes, things to do in New York City, important social causes/issues and the arts.", "followers_count": 1015, "friends_count": 2702, "statues_count": 22721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925583250616321, "text": "�� https://t.co/kEvGTe4iKc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2558102395, "name": "ShQunnä", "screen_name": "Bbyy_G", "lang": "en", "location": "L-town , nc", "create_at": date("2014-06-09"), "description": "Positivity equals growth.", "followers_count": 350, "friends_count": 240, "statues_count": 3742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-03-13T00:00:04.000Z"), "id": 708925583632306176, "text": "Seriously so pissed off ✌��️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 707339494, "name": "paige zuck", "screen_name": "zuck_paige", "lang": "en", "location": "null", "create_at": date("2013-10-09"), "description": "Let them haters hate! ❤️", "followers_count": 81, "friends_count": 377, "statues_count": 1219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clarion, PA", "id": "01c0627815037ea9", "name": "Clarion", "place_type": "city", "bounding_box": rectangle("-79.400677,41.177945 -79.335984,41.226688") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42031, "countyName": "Clarion", "cityID": 4213800, "cityName": "Clarion" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925583837626368, "text": "aw hell yeah lol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3243512720, "name": "bonita applebum", "screen_name": "rachod3", "lang": "en", "location": "null", "create_at": date("2015-06-12"), "description": "life goes on pimpin, the wise don't doubt it", "followers_count": 132, "friends_count": 161, "statues_count": 1797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925584299171840, "text": "Wind 2.0 mph SW. Barometer 29.827 in, Steady. Temperature 61.7 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925584554926081, "text": "@Zdic26 how many debates have you watched", "in_reply_to_status": 708925340324917248, "in_reply_to_user": 1329380444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1329380444 }}, "user": { "id": 3426461661, "name": "Kelsey Kochman", "screen_name": "kelkochmann", "lang": "en", "location": "null", "create_at": date("2015-08-16"), "description": "you can find me at your nearest coffee shop, listening to John Mayer, or laughing at things no one else finds funny. taken by AGD and Mel", "followers_count": 487, "friends_count": 608, "statues_count": 4897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925584991080448, "text": "Stop Messing With These Lil Ass Boys That Got You Blind To What A Real Nigga Can Show You", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4103448618, "name": "HollyHoodCurt", "screen_name": "OG_Uncle_Curtis", "lang": "en", "location": "NBA Arena", "create_at": date("2015-11-02"), "description": "Soon To Be 2016 TSU College Graduate", "followers_count": 263, "friends_count": 399, "statues_count": 880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925585070788608, "text": "@camilacabello97@AllyBrooke@dinahjane97@LaurenJauregui @NormaniKordei@FifthHarmony Congrats on the big win������ #KCAs https://t.co/NXgYGJyOCl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KCAs" }}, "user": { "id": 1973555665, "name": "Officially Kiara", "screen_name": "411_hollywood", "lang": "en", "location": "null", "create_at": date("2013-10-19"), "description": "Freelance Hollywood Reporter/ Production Assistant ... TV Luver. Movie Luver. Music Luver.", "followers_count": 28, "friends_count": 276, "statues_count": 158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925585217564672, "text": "So @DisneyZootopia was so amazing it's making me question my favorite Disney movie.. That's a big deal. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2830724243 }}, "user": { "id": 1700613204, "name": "Ashli", "screen_name": "ashlirh_", "lang": "en", "location": "null", "create_at": date("2013-08-25"), "description": "*The past can hurt. But from the way I see it, you can either run from it or learn from it*", "followers_count": 155, "friends_count": 487, "statues_count": 5762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Provo, UT", "id": "665409336d2b20a0", "name": "Provo", "place_type": "city", "bounding_box": rectangle("-111.740949,40.18565 -111.602629,40.311641") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4962470, "cityName": "Provo" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925585490190338, "text": "@SwaginNando you https://t.co/aKVdvnRPTd", "in_reply_to_status": -1, "in_reply_to_user": 276830666, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 276830666 }}, "user": { "id": 774541098, "name": "Jr", "screen_name": "JuniorEsv", "lang": "en", "location": "Vallejo Ca", "create_at": date("2012-08-22"), "description": "null", "followers_count": 494, "friends_count": 420, "statues_count": 40341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925585901223936, "text": "fuck what they talkin bout on your timeline", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013791744, "name": "alina", "screen_name": "lowkeyalina", "lang": "en", "location": "Fresno, CA", "create_at": date("2015-02-08"), "description": "I go to a community college", "followers_count": 388, "friends_count": 308, "statues_count": 12643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Malaga, CA", "id": "0128d1be33bc0d8c", "name": "Malaga", "place_type": "city", "bounding_box": rectangle("-119.736767,36.669989 -119.71559,36.69306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 645232, "cityName": "Malaga" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925586320658432, "text": "�� stupid girl https://t.co/TKiKMOl0bh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1682360245, "name": "mary", "screen_name": "Crownroyale__", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2013-08-18"), "description": "Mck Senior| 18 | single ✨", "followers_count": 728, "friends_count": 634, "statues_count": 12598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925586618494976, "text": "Damn haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 227918440, "name": "Anibal Hernandez", "screen_name": "Anibalh215", "lang": "en", "location": "Los Angeles", "create_at": date("2010-12-17"), "description": "A Socialist living in the Most Hated Capitalist Country in the World.", "followers_count": 75, "friends_count": 106, "statues_count": 1812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-13T00:00:05.000Z"), "id": 708925586664607744, "text": "2.5 magnitude #earthquake. 2 km from Hennessey, OK, #UnitedStates https://t.co/1k1NMyZB0E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.874,36.111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 66454, "friends_count": 10, "statues_count": 98424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma, USA", "id": "bd3d2074a33fbd06", "name": "Oklahoma", "place_type": "admin", "bounding_box": rectangle("-103.002652,33.615765 -94.431332,37.002328") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40073, "countyName": "Kingfisher" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272851065524224, "text": "@Kingkoopaaa2 @Will_ofRights @1000_Myles but look at the quality of food lol that's like comparing being a cook at a top notch place to MCD", "in_reply_to_status": 709272595699539968, "in_reply_to_user": 368720409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 368720409, 343307918, 87855724 }}, "user": { "id": 240745108, "name": "Akeem Echols", "screen_name": "PimpinWody", "lang": "en", "location": "Baltimore/Raleigh", "create_at": date("2011-01-20"), "description": "Jesus Christ I gotta get right before I get left!! ShawUBasketball #0", "followers_count": 1723, "friends_count": 1545, "statues_count": 101966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272851258449920, "text": "Who is awake and in MN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455228794, "name": "Whitney Anderson", "screen_name": "WhittaMarie", "lang": "en", "location": "null", "create_at": date("2012-01-04"), "description": "University of Wisconsin Madison 2020", "followers_count": 365, "friends_count": 177, "statues_count": 5485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champlin, MN", "id": "84464e4839625b9d", "name": "Champlin", "place_type": "city", "bounding_box": rectangle("-93.422453,45.150614 -93.332378,45.209599") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2710846, "cityName": "Champlin" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272851799355392, "text": "@SMGB25 yeeeeet~ meanwhile I'm about to go to bed", "in_reply_to_status": 709272651726852096, "in_reply_to_user": 58388348, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 58388348 }}, "user": { "id": 18495431, "name": "vehunnie", "screen_name": "sukiilu", "lang": "en", "location": "River of Golden Dreams", "create_at": date("2008-12-30"), "description": "いつもドキドキ", "followers_count": 385, "friends_count": 776, "statues_count": 21026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272852051021825, "text": "@SamuelLJackson Take me under your wing and teach me the craft", "in_reply_to_status": -1, "in_reply_to_user": 75974281, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75974281 }}, "user": { "id": 213050398, "name": "MR.BOGARD", "screen_name": "ESEANDUB", "lang": "en", "location": "UNIVERSAL", "create_at": date("2010-11-07"), "description": "Crows and Pigeons are not best friends.", "followers_count": 45, "friends_count": 323, "statues_count": 225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272852856311808, "text": "Everyone cuffedddd �� where my poly boys @ ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1953212023, "name": "Bynthiaaa", "screen_name": "_CynnnBaileyyy", "lang": "en", "location": "null", "create_at": date("2013-10-10"), "description": "NHS | Ke Akua | RIP Isa / RIP Uncle / RIP G-Ma", "followers_count": 963, "friends_count": 826, "statues_count": 36379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272852915232768, "text": "YOOOOOOOOO. FUCKING CANT DEAL. https://t.co/LZIjwsOtIZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134602560, "name": "Wish I Was A Raccoon", "screen_name": "LuceFerre", "lang": "en", "location": "NYC", "create_at": date("2010-04-18"), "description": "A Life of Perpetual Sorrow. Singer/Songwriter. Happily in love with a vampire platypus @thegaymike", "followers_count": 1767, "friends_count": 788, "statues_count": 88057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272853284298752, "text": "and @QueennBreee", "in_reply_to_status": 709272396201639936, "in_reply_to_user": 2411758669, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1237162292 }}, "user": { "id": 2411758669, "name": "Holly", "screen_name": "hollyisking96", "lang": "en", "location": "St Louis, MO", "create_at": date("2014-03-25"), "description": "fall in love with being alive", "followers_count": 1234, "friends_count": 736, "statues_count": 14054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granite City, IL", "id": "83a8abf4f6ece8db", "name": "Granite City", "place_type": "city", "bounding_box": rectangle("-90.185267,38.686586 -90.038683,38.799749") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1730926, "cityName": "Granite City" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272853321920512, "text": "@anderson_tierra us in history everyday!! https://t.co/F8a7lbkGe1", "in_reply_to_status": -1, "in_reply_to_user": 122795882, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 122795882 }}, "user": { "id": 2584973424, "name": "Tiya❤️", "screen_name": "tiya_taylor", "lang": "en", "location": "ETX✈️DTX ", "create_at": date("2014-06-23"), "description": "#UTD19", "followers_count": 475, "friends_count": 341, "statues_count": 9568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Longview, TX", "id": "d4157f8e4bd01598", "name": "Longview", "place_type": "city", "bounding_box": rectangle("-94.844303,32.42091 -94.618379,32.58565") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48183, "countyName": "Gregg", "cityID": 4843888, "cityName": "Longview" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272853376409600, "text": "Happy Birthday @AnselElgort I love you!! Have a great day ❤❤ https://t.co/PsQvAsKmkT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216201838 }}, "user": { "id": 2528900077, "name": "Monica", "screen_name": "MonicaHansen77_", "lang": "en", "location": "null", "create_at": date("2014-05-27"), "description": "EUHS 2017 ❤ Ansel Elgort is my husband", "followers_count": 164, "friends_count": 122, "statues_count": 417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272853376466944, "text": "@fettywap NOTICE BBY WAAAAP @_NawDeeUhh �������������������� https://t.co/cXgcCdqhDA", "in_reply_to_status": -1, "in_reply_to_user": 2601175671, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2601175671, 58325629 }}, "user": { "id": 2999547126, "name": "JD ✌", "screen_name": "littleeejaaay", "lang": "en", "location": "suncity tx ☀️", "create_at": date("2015-01-28"), "description": "Jacob Matthew Carrillo ❤️", "followers_count": 448, "friends_count": 689, "statues_count": 14703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272853850382336, "text": "I will make somethin of myself cause I couldn't give up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596276676, "name": "Bryce.", "screen_name": "Brryceee_", "lang": "en", "location": "null", "create_at": date("2012-05-31"), "description": "ʷʰʸ ᵗʰᵉ fᵘͨᵏ ʸºᵘ ˡʸʸʸ'ⁿ, ʷʰʸ ʸºᵘ ᵃˡʷᵃʸˢ ˡʸʸʸ'ⁿ, ᵐᵐᵐᵐᵐᵐ ºʰ ᵐʸ ᵍºᵈ ˢᵗºp fᵘͨᵏ'ⁿ ˡʸʸʸ'ⁿ", "followers_count": 1331, "friends_count": 406, "statues_count": 75761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272854387240961, "text": "Im penny proud, I'm cute and I'm loud and I got. it. going on lol. https://t.co/9ZMoq1A6tW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34398473, "name": "Paris", "screen_name": "_PARISnoHilton", "lang": "en", "location": "null", "create_at": date("2009-04-22"), "description": "23. College grad. #ULAlumna. Hair-doer. Makeup lover. YouTube beauty blogger (ParisDanielle)", "followers_count": 1713, "friends_count": 1095, "statues_count": 65238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272854525661184, "text": "It's 2am on a day after time change. I know I'm not the only person on earth wide awake #stupidtimechange #nightowl #coffeemonday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stupidtimechange", "nightowl", "coffeemonday" }}, "user": { "id": 701935753811525632, "name": "Elizabeth Godfrey", "screen_name": "LizOnceBitten", "lang": "en", "location": "Bossier City, LA", "create_at": date("2016-02-22"), "description": "Author of Once Bitten Twice Cursed for bookings contact oncebittennovel@Gmail.com", "followers_count": 75, "friends_count": 268, "statues_count": 111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bossier City, LA", "id": "018617530de2ac1c", "name": "Bossier City", "place_type": "city", "bounding_box": rectangle("-93.74855,32.419303 -93.584206,32.630042") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22015, "countyName": "Bossier", "cityID": 2208920, "cityName": "Bossier City" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272854802653184, "text": "Uuuh is anybody else watching #LittleBigShots this 4 yr old has been playing for 3 months & is fan-freaking-tactic #classical tunes by ear!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LittleBigShots", "classical" }}, "user": { "id": 103147590, "name": "Elliott Annissa", "screen_name": "nissakaye", "lang": "en", "location": "ATL GA", "create_at": date("2010-01-08"), "description": "i love self-motivated people!!", "followers_count": 841, "friends_count": 1378, "statues_count": 9122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-14T00:00:00.000Z"), "id": 709272855209320449, "text": "Be sure to take advantage of the discount rate for custom units time is running out. Can text,… https://t.co/knbH4eeL9D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.225621,36.128097"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189328768, "name": "❤TRU❤", "screen_name": "enviedbyu", "lang": "en", "location": "Las Vegas, New Orleans", "create_at": date("2010-09-10"), "description": "❤God, my family, & Wigs/Hair Extensions. Hair Extension Specialist/ Wigmaker http://www.trutouch.mayvenn.com", "followers_count": 115, "friends_count": 284, "statues_count": 9188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3268585, "cityName": "Spring Valley" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272855335153664, "text": "@carolinaa_o99 welcome to the club ��", "in_reply_to_status": 709272442636603395, "in_reply_to_user": 2383050890, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2383050890 }}, "user": { "id": 3235375500, "name": "halani", "screen_name": "halanirdz", "lang": "en", "location": "Houston, TX", "create_at": date("2015-06-03"), "description": "null", "followers_count": 174, "friends_count": 302, "statues_count": 3399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellaire, TX", "id": "bc39561011b12bc6", "name": "Bellaire", "place_type": "city", "bounding_box": rectangle("-95.484647,29.689372 -95.447422,29.725996") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4807300, "cityName": "Bellaire" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272855394041856, "text": "my sleeping schedule is all fucked up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981884229, "name": "katty", "screen_name": "cruz_katarina", "lang": "en", "location": "Staten Island, NY", "create_at": date("2015-01-16"), "description": "I have only one life, I'm trying to live it #1005", "followers_count": 130, "friends_count": 447, "statues_count": 4546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorketown, NJ", "id": "005f52e06b9c06c8", "name": "Yorketown", "place_type": "city", "bounding_box": rectangle("-74.362666,40.270616 -74.282922,40.344614") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3483245, "cityName": "Yorketown" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272855402262529, "text": "Queen�������� https://t.co/e89PogTLju", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708680834, "name": "Natasha Angel Nieva", "screen_name": "NatashaNieva", "lang": "en", "location": "South San Francisco, CA", "create_at": date("2013-08-28"), "description": "The world is our stage |nursing| |theatre| |1D| |Little Mix| IG: angelnieva_", "followers_count": 469, "friends_count": 536, "statues_count": 5192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272856421478400, "text": "@shak me too. Would be happy to have someone take over but have yet to encounter the right mix of passion/price", "in_reply_to_status": 709272434134753280, "in_reply_to_user": 17977475, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17977475 }}, "user": { "id": 15416686, "name": "Stephen Tatton", "screen_name": "TattonTreks", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2008-07-13"), "description": "Ent/Tech. Stones, Music, Coffee. Sailor. Adventure sports. Life. Entrepreneur.", "followers_count": 16820, "friends_count": 2020, "statues_count": 39236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mid-Wilshire, Los Angeles", "id": "0775fcd6eb188d7c", "name": "Mid-Wilshire", "place_type": "neighborhood", "bounding_box": rectangle("-118.361388,34.043829 -118.309037,34.083516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272856492974080, "text": "ssup l.a []", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2906160926, "name": "キング Kylie", "screen_name": "kylizxzle", "lang": "en", "location": "nyc", "create_at": date("2014-11-21"), "description": "❝Live a life you will remember❞ · [Kylie Jenner's duplicate] · [xA_H-Ravenclaw's Pride] [wsq;baratsq]", "followers_count": 521, "friends_count": 542, "statues_count": 10725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272856534749184, "text": "@Team_Sed shit jus coolin, nice long vacay... had to get away from bham and get my mind right I got shot in January", "in_reply_to_status": 709272673852006401, "in_reply_to_user": 256663298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 256663298 }}, "user": { "id": 473597317, "name": "Chavo✡", "screen_name": "2kChavo", "lang": "en", "location": "Alabama", "create_at": date("2012-01-24"), "description": "Money on my mind.$$", "followers_count": 1337, "friends_count": 1181, "statues_count": 79814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272856639574016, "text": "@TDRico cus Dexter was starting to judge me ☹️", "in_reply_to_status": 709261912664461313, "in_reply_to_user": 105674620, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 105674620 }}, "user": { "id": 2823294300, "name": "JR", "screen_name": "Skrillahead_", "lang": "en", "location": "Fullerton,CA", "create_at": date("2014-09-20"), "description": "I'm not the prettiest you've ever seen but on good days I'm charming as fuck.", "followers_count": 84, "friends_count": 175, "statues_count": 3225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272857214373888, "text": "..ima eat me some snacks && force myself to gts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 482331527, "name": "GLODAY MARCH27th❤️", "screen_name": "NeshaPrettyAss_", "lang": "en", "location": "null", "create_at": date("2012-02-03"), "description": "#CollegeGirl #SIUC18 follow me on snap chat: @neshaprettyass #Aries♈️ 3/27/96", "followers_count": 1075, "friends_count": 976, "statues_count": 61134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carbondale, IL", "id": "20d0abc87d11501a", "name": "Carbondale", "place_type": "city", "bounding_box": rectangle("-89.294485,37.670875 -89.151934,37.764433") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17077, "countyName": "Jackson", "cityID": 1711163, "cityName": "Carbondale" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272857516191744, "text": "At least I tried ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 901819981, "name": "That Saxophone Guy", "screen_name": "JoshClements403", "lang": "en", "location": "null", "create_at": date("2012-10-24"), "description": "Saxophone Performance and Music Ed at UNCO! Go Bears!", "followers_count": 251, "friends_count": 344, "statues_count": 1448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272857855922178, "text": "Didn't I tell you not to tell me good night if you not going to sleep? @NadiaSmiles_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353040981 }}, "user": { "id": 290830699, "name": "D'Marcus Lacy II", "screen_name": "AyeChop_YEEEE", "lang": "en", "location": "Galveston/Lake Travis/PV", "create_at": date("2011-04-30"), "description": "only time will tell.", "followers_count": 1548, "friends_count": 569, "statues_count": 40133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Marque, TX", "id": "62d6ba41ace74599", "name": "La Marque", "place_type": "city", "bounding_box": rectangle("-95.051744,29.348343 -94.946934,29.403387") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841116, "cityName": "La Marque" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272858023895040, "text": "03/14@03:00 - Temp 47.9F, WC 47.2F. Wind 3.1mph ENE, Gust 9.0mph. Bar 30.017in, Falling slowly. Rain 0.03in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272858178904064, "text": "1 https://t.co/7rsyW7SLYw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1569853009, "name": "ㅤㅤㅤㅤㅤㅤㅤㅤㅤ", "screen_name": "kylemetoyer_", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "i hate @sadieestricklin", "followers_count": 404, "friends_count": 199, "statues_count": 3575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272858313228288, "text": "Just watched the #ESPN #30for30 #FantasticLies one of the best ones I've watched ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ESPN", "30for30", "FantasticLies" }}, "user": { "id": 809583666, "name": "I Am Bane", "screen_name": "PatStat22", "lang": "en", "location": "null", "create_at": date("2012-09-07"), "description": "West Virgina born living in Greenville SC #TarheelNation #ClemsonFootball #RedSox #Reds #CharlotteHornets IG @PATSTAT22", "followers_count": 1420, "friends_count": 656, "statues_count": 28263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, SC", "id": "8eb7d0abedc4817b", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-82.434848,34.687331 -82.249689,34.904552") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4530850, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272858422333440, "text": "Wind 0.0 mph ---. Barometer 29.673 in, Falling. Temperature 55.5 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272858573279233, "text": "This about to be me https://t.co/oQ1VxQve7V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1547517618, "name": "MoneyMakingMitch", "screen_name": "_Blvckhype", "lang": "en", "location": "low", "create_at": date("2013-06-26"), "description": "somewhere fucking up a check....Ladies don't stare to long I belong to Ariella Wanzo❤️", "followers_count": 1238, "friends_count": 1354, "statues_count": 10136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272859382669312, "text": "@SavageKaraa idk what you're talking about.", "in_reply_to_status": 709272657875705856, "in_reply_to_user": 170522242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 170522242 }}, "user": { "id": 2739334052, "name": "DRE - CEO", "screen_name": "xKoah", "lang": "en", "location": "Add me on Snapchat: Deon_SC", "create_at": date("2014-08-17"), "description": "Founder/CEO of @TheRegalReserve /\nCo-Owner of @ControllerUnion /\nMedia manager for @Looks /\nGeneral manager for @8berryBranding /\nMarketing manager - @Everwarmc", "followers_count": 63828, "friends_count": 5294, "statues_count": 18009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aptos, CA", "id": "09d7cfd44170d537", "name": "Aptos", "place_type": "city", "bounding_box": rectangle("-121.933333,36.967212 -121.863133,37.016726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 602378, "cityName": "Aptos" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272859386851328, "text": "None of my streaks on snapchat are changing ☹☹", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304205532, "name": "mikayla", "screen_name": "Mikaaaylaaa__", "lang": "en", "location": "null", "create_at": date("2011-05-23"), "description": "831", "followers_count": 972, "friends_count": 559, "statues_count": 36684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "King City, CA", "id": "0f119d2beee20b90", "name": "King City", "place_type": "city", "bounding_box": rectangle("-121.148723,36.195958 -121.111924,36.235099") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 638520, "cityName": "King City" } }
+{ "create_at": datetime("2016-03-14T00:00:01.000Z"), "id": 709272859403616256, "text": "I'm still waiting for the rt ? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3112703747, "name": "k lyn", "screen_name": "KailynGarcia1", "lang": "en", "location": "w/Brit or Boo", "create_at": date("2015-03-24"), "description": "@kaylaafaustt", "followers_count": 398, "friends_count": 676, "statues_count": 2533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272859827253249, "text": "You right brethren. https://t.co/n6vaKno8kY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1699824037, "name": "pimp pimp☯", "screen_name": "nonchalanty_", "lang": "en", "location": "null", "create_at": date("2013-08-25"), "description": "#BigRingsEnt the hitman", "followers_count": 858, "friends_count": 548, "statues_count": 60160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272859911172096, "text": "@beautifullNbold but Ima just feel bad about eating it", "in_reply_to_status": 709272784514392064, "in_reply_to_user": 282393285, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 282393285 }}, "user": { "id": 282393285, "name": "Raven Williams", "screen_name": "beautifullNbold", "lang": "en", "location": "Houston, TX", "create_at": date("2011-04-14"), "description": "Future Doctor ,With hard work and god it'll happen #TxSU19, SC: williamsraven", "followers_count": 2952, "friends_count": 970, "statues_count": 53703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272860104200192, "text": "Don't measure the amount I care for you by my absence", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388363439, "name": "Sherene", "screen_name": "ShereneLoves", "lang": "en", "location": "null", "create_at": date("2011-10-10"), "description": "sometimes you have to be kinder than how you actually feel", "followers_count": 200, "friends_count": 216, "statues_count": 2619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterbury, CT", "id": "f9a4ed4d94a9b93f", "name": "Waterbury", "place_type": "city", "bounding_box": rectangle("-73.093845,41.513971 -72.950371,41.619186") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 980000, "cityName": "Waterbury" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272860217323520, "text": "Being petty cause I want toooooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2469596623, "name": "ash ♓️", "screen_name": "_Ashlynnn", "lang": "en", "location": "DTX to PVTX", "create_at": date("2014-04-29"), "description": "#PVAMU19 #God1st #FreeMyBrothas", "followers_count": 1441, "friends_count": 1407, "statues_count": 24886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272860393672706, "text": "Graffiti Tracking: No GPS, Bakersfield", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.11886,35.31595167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55761838, "name": "594 Graffiti", "screen_name": "594Graffiti", "lang": "en", "location": "ÜT: 33.564277,-117.709513", "create_at": date("2009-07-10"), "description": "594 Graffiti- Providers of TAGRS a web-based Graffiti Tracking Solution - TAGRS, Tracking and Automated Graffiti Reporting System, Uses Smartphones to Catch Du", "followers_count": 905, "friends_count": 2, "statues_count": 325448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272860406243328, "text": "Me too boo. https://t.co/TtItmNu0Pf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37129716, "name": "billy hough", "screen_name": "billyhough", "lang": "en", "location": "god knows", "create_at": date("2009-05-01"), "description": "Head asshole of SCREAM ALONG WITH BILLY, GARAGEDOGS, TIME OUT OF MIND (2015), THE GOLD DUST ORPHANS, SURRENDER DOROTHY, proud uncle. http://www.billyhough.com", "followers_count": 283, "friends_count": 253, "statues_count": 2407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861077331968, "text": "@freddystaco @Netflixhelps this is why I love you lmfao ����", "in_reply_to_status": 709205360125485057, "in_reply_to_user": 2722779139, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2722779139, 21454307 }}, "user": { "id": 1921689349, "name": "les❁ loves rose", "screen_name": "cuddlyalonso", "lang": "en", "location": "diana kyla flo :) ♕", "create_at": date("2013-09-30"), "description": "ma pussy b poppin 25/8 || my @ doesn't limit who I love || cnco Johann and yoyo :)", "followers_count": 3772, "friends_count": 454, "statues_count": 56767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861081350144, "text": "@SOB_COMIX @ohcararara @allaloam What are you working on?", "in_reply_to_status": 709272810632273920, "in_reply_to_user": 54845467, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24423719, 40152883, 237919122 }}, "user": { "id": 54845467, "name": "Leia Weathington", "screen_name": "solmaru", "lang": "en", "location": "Portland, Oregon", "create_at": date("2009-07-08"), "description": "Writer for @DeepEngines, Smut Peddler, The Legend of Bold Riley and host of A Happy Go Lucky Podcast. Never safe for work.", "followers_count": 2456, "friends_count": 361, "statues_count": 47991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861140230144, "text": "Temp: 65.8°F Wind:1.2mph Pressure: 29.848hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861190590464, "text": "@SynergyGymsNYC please call me 917-804-3365.", "in_reply_to_status": 709272769784061952, "in_reply_to_user": 23880149, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1266248826 }}, "user": { "id": 23880149, "name": "BestPersonalTrainer", "screen_name": "BestTrainer", "lang": "en", "location": "New York", "create_at": date("2009-03-11"), "description": "My Clients consider me Best Personal Trainer in the World. I try to Live Up to That. My Web Site is: http://BestPersonalTrainer.Weebly.Com", "followers_count": 261, "friends_count": 211, "statues_count": 5634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861484064768, "text": "If we gotta study 9-10 hours straight SO BE IT! If it'll make us brilliant in the end why not sacrifice that time for the career we desire?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90114100, "name": "Dora Owusu", "screen_name": "owusulovesyou", "lang": "en", "location": "null", "create_at": date("2009-11-14"), "description": "19 yrs.| Instagram:@doraowusu| Tumblr:owusulovesyou| Care free Black girl✊| Benjamin Bowen❤️", "followers_count": 898, "friends_count": 467, "statues_count": 7933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861622607872, "text": "Wind 0.0 mph SW. Barometer 29.761 in, Rising slowly. Temperature 58.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861765013505, "text": "I L O V E listening to my boyfriend play piano ��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393212807, "name": "Amanda", "screen_name": "mmanderzz", "lang": "en", "location": "null", "create_at": date("2011-10-17"), "description": "null", "followers_count": 136, "friends_count": 376, "statues_count": 2745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861836386305, "text": "@GarrettIzCool cuz some annoying ass female fan of hers would have said something", "in_reply_to_status": 709271306391277569, "in_reply_to_user": 3179521718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3179521718 }}, "user": { "id": 26642527, "name": "fernando", "screen_name": "robymac369", "lang": "en", "location": "OKC/Edmond", "create_at": date("2009-03-25"), "description": "suh dude", "followers_count": 857, "friends_count": 502, "statues_count": 57470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272861936996354, "text": "@DannyBoy133100 lol yes I forgot about that", "in_reply_to_status": 709272660547514369, "in_reply_to_user": 4052066772, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4052066772 }}, "user": { "id": 4256149305, "name": "Nancy Lasocki", "screen_name": "nancylasocki", "lang": "en", "location": "Illinois, USA", "create_at": date("2015-11-23"), "description": "Wife,Mom,Grandma & business owner/Pest Control. Simple things make me happy! Animal lover, Metal detecting,Casino lover", "followers_count": 683, "friends_count": 1114, "statues_count": 8154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minooka, IL", "id": "00bc588143233567", "name": "Minooka", "place_type": "city", "bounding_box": rectangle("-88.337511,41.390769 -88.232545,41.484099") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17063, "countyName": "Grundy", "cityID": 1749607, "cityName": "Minooka" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272862687944704, "text": "Morning #workout done, new #blog is up, back to the #hustle! https://t.co/JNpxNT1WYQ #twitter #monday #madness https://t.co/jVTpXMf0el", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "workout", "blog", "hustle", "twitter", "monday", "madness" }}, "user": { "id": 1853434640, "name": "James Carian", "screen_name": "James_Carian", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2013-09-10"), "description": "Taking ideas seriously. Your #philosophy creates your life: existence, rationality, selfishness. Fighting for Objectivism not by faith, but by reason.", "followers_count": 2429, "friends_count": 2313, "statues_count": 3264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272862872342532, "text": "Sunday the 13th is scarier than Friday tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1950333666, "name": "kaneki", "screen_name": "jonna_boy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-10-09"), "description": "Beaming beauty badass", "followers_count": 256, "friends_count": 178, "statues_count": 2079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272863363047424, "text": "https://t.co/9Exp9bBXW8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1069819256, "name": "KANSAS/SEATTLE", "screen_name": "CAT1PRO", "lang": "en", "location": "SEATTLE,WA", "create_at": date("2013-01-07"), "description": "BORN IN KANSAS-RAISED ON A NW RANCH ,SEEN 20 NATIONS,CATERPILLAR TECH.,EET-, TRAINED USA COMBAT SOLDIERS,HORSES&DOGS- LOVE LIFE-JESUS IS MY LORD", "followers_count": 23118, "friends_count": 24200, "statues_count": 85688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272863367372800, "text": "Wind 1.6 mph NE. Barometer 30.16 in, Steady. Temperature 40.8 °F. Rain today 0.00 in. Humidity 47%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272863375646721, "text": "68.4F (Feels: 68.4F) - Humidity: 94% - Wind: 9.8mph S - Gust: 13.0mph - Pressure: 1016.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 230594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-14T00:00:02.000Z"), "id": 709272863618928640, "text": "In case you missed it earlier this evening...Bernie Sanders was up first at the CNN Town Hall... https://t.co/uuEMQdIPqR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2160736098, "name": "Janis Jones", "screen_name": "11papyrus11", "lang": "en", "location": "Los Angeles", "create_at": date("2013-10-28"), "description": "Films/Music/Screenwriting/Art/Dogs/XX Chromosomes", "followers_count": 1218, "friends_count": 996, "statues_count": 25902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272864340516865, "text": "Wind 0.0 mph ---. Barometer 30.166 in, Steady. Temperature 25.6 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272864361467904, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1855167824, "name": "MJR", "screen_name": "ifoundm0lly", "lang": "en", "location": "null", "create_at": date("2013-09-11"), "description": "null", "followers_count": 293, "friends_count": 113, "statues_count": 3541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, ME", "id": "4958d94f9cbd52b5", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-70.30035,44.022138 -70.213412,44.144619") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin", "cityID": 2302060, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272864600539136, "text": "قال علي بن أبي طالب\nجالس الفقراء تزداد شكراً", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 418946780, "name": "محمد الفضلي", "screen_name": "abooabdullahq8", "lang": "en", "location": "الكويت (( دار ابن صباح ))", "create_at": date("2011-11-22"), "description": "ماجستير قانون وباحث دكتوراه & مُحكمْ معتمد من الجهات الكويتيه المختصه & رئيس الجمعية الكويتيه لمناهضة التمييز العنصري & عضو بجمعية الأداء البرلماني 96567717764+", "followers_count": 5297, "friends_count": 498, "statues_count": 11726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272864671707136, "text": "Incident on I-94 EB before 35th St #traffic https://t.co/nohdC1DOBx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.95774,43.03225"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "traffic" }}, "user": { "id": 24953488, "name": "TTN Milwaukee", "screen_name": "TotalTrafficMKE", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2009-03-17"), "description": "Total Traffic Network serving Milwaukee on WMIL, WKKV, WRNW, WRIT, WOKY, WISN\r\n\r\nTo report accidents/delays, call the Tipline at 414-944-5111!", "followers_count": 1457, "friends_count": 48, "statues_count": 28377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272864717807616, "text": "Everybody go wish @razmeezy a happy birthday, she finally 12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728035488 }}, "user": { "id": 634282327, "name": "Yasin West", "screen_name": "ThatAfgKid", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-07-12"), "description": "Just a young white boy trying to make it big in these streets", "followers_count": 393, "friends_count": 193, "statues_count": 37962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272865409859584, "text": "There's a massive mural of Kim Kardashian's nude selfie in Melbourne https://t.co/WTFVkmB5To", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17566, "friends_count": 17573, "statues_count": 69242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272865560862721, "text": "@bassomar_omy @HillaryClinton ALL THESE LIES R FROM THE GD GOP!!", "in_reply_to_status": 709180939885277184, "in_reply_to_user": 2369688244, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2369688244, 1339835893 }}, "user": { "id": 328336153, "name": "linda conaway", "screen_name": "lconaway53", "lang": "en", "location": "null", "create_at": date("2011-07-02"), "description": "null", "followers_count": 111, "friends_count": 143, "statues_count": 7888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866030809088, "text": "Hatians out here put me on all the raw ass Kodak����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353474651, "name": "JIG™", "screen_name": "BossKj_", "lang": "en", "location": "Chicago ✈️ Miami", "create_at": date("2011-08-11"), "description": "null", "followers_count": 1779, "friends_count": 744, "statues_count": 58542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ives Estates, FL", "id": "b6e48266218d4454", "name": "Ives Estates", "place_type": "city", "bounding_box": rectangle("-80.199715,25.947387 -80.16523,25.973778") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1234400, "cityName": "Ives Estates" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866043265024, "text": "Love the sound of the hubby and daughter laughing! @BaileyBogan ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 794592494 }}, "user": { "id": 30375690, "name": "Sarah Durst", "screen_name": "bbratty", "lang": "en", "location": "Citrus Heights, CA", "create_at": date("2009-04-10"), "description": "Hmmmm.. you tell me about me?", "followers_count": 45, "friends_count": 143, "statues_count": 3242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Heights, CA", "id": "a89175c4c91f45a3", "name": "Citrus Heights", "place_type": "city", "bounding_box": rectangle("-121.331355,38.663673 -121.242804,38.722779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 613588, "cityName": "Citrus Heights" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866362011650, "text": "Trump all the way ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434072731, "name": "Uncle illest", "screen_name": "illest1507", "lang": "en", "location": "Ym's house ", "create_at": date("2011-12-11"), "description": "null", "followers_count": 501, "friends_count": 406, "statues_count": 4605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866479411200, "text": "Dear insomnia, \nlet's break up \nSincerely,\nThe girl still awake on a school night ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4254294372, "name": "Sierra", "screen_name": "luke_hemmotron", "lang": "en", "location": "probably a concert", "create_at": date("2015-11-15"), "description": "who am I when I don't know myself? who am I... who am I... Invisible!!! #SLFLRidgefield #5sosfam #potatofam", "followers_count": 932, "friends_count": 1945, "statues_count": 11111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napavine, WA", "id": "011c17273af10c3e", "name": "Napavine", "place_type": "city", "bounding_box": rectangle("-122.929299,46.569058 -122.871049,46.623027") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53041, "countyName": "Lewis", "cityID": 5347980, "cityName": "Napavine" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866525749248, "text": "@FoodPornsx @LindsGilchrist7", "in_reply_to_status": 709258337989369856, "in_reply_to_user": 1130356441, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1130356441, 1630701438 }}, "user": { "id": 586203578, "name": "Little RedRiding Cam", "screen_name": "cameroncassi", "lang": "en", "location": "Montana, USA", "create_at": date("2012-05-20"), "description": "406 - MSUB", "followers_count": 606, "friends_count": 446, "statues_count": 34358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Billings, MT", "id": "662aa8db4557a744", "name": "Billings", "place_type": "city", "bounding_box": rectangle("-108.692879,45.723722 -108.432965,45.871169") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3006550, "cityName": "Billings" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866848571392, "text": "#SupportOriginMelissa 56.5°F Wind:1.1mph Pressure: 29.79hpa Steady Rain Today 0.00in. Forecast: Changeable, some precipitation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 312342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272866961952768, "text": "08294f8000p8E9EF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.47909108,35.55580559"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 12, "friends_count": 0, "statues_count": 5553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Carolina, USA", "id": "3b98b02fba3f9753", "name": "North Carolina", "place_type": "admin", "bounding_box": rectangle("-84.321948,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37045, "countyName": "Cleveland" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867272269824, "text": "Wind 4.3 mph SW. Barometer 29.768 in, Steady. Temperature 68.0 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867288928256, "text": "In other news, this episode of Last Week Tonight from last month is pretty great. https://t.co/rEl7iPYdAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51989025, "name": "Saul Badillo", "screen_name": "SauBadillo", "lang": "en", "location": "Over, There", "create_at": date("2009-06-29"), "description": "I'm not very interesting.", "followers_count": 85, "friends_count": 664, "statues_count": 7688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sudden Valley, WA", "id": "397ffcacbb3f95cf", "name": "Sudden Valley", "place_type": "city", "bounding_box": rectangle("-122.393441,48.70284 -122.309991,48.745106") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5368200, "cityName": "Sudden Valley" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867372838912, "text": "I been really into lip glosses lately #malipglossispoppin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "malipglossispoppin" }}, "user": { "id": 98761619, "name": "hastie", "screen_name": "hasnassty", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-12-22"), "description": "la vida enferma", "followers_count": 265, "friends_count": 95, "statues_count": 23400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867569950720, "text": "If I don't past my behind the wheel test tmrw im crying idc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1713877548, "name": "antonio.", "screen_name": "P1ZZATEEN", "lang": "en", "location": "somewhere i can feel safe ", "create_at": date("2013-08-30"), "description": "chicano // pisces // feminist", "followers_count": 291, "friends_count": 282, "statues_count": 25081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867590971393, "text": "My dad came home brought me a burrito and my car �� #Blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Blessed" }}, "user": { "id": 3327906073, "name": "Gigi", "screen_name": "gisseeeellll", "lang": "en", "location": "null", "create_at": date("2015-08-23"), "description": "18", "followers_count": 361, "friends_count": 316, "statues_count": 2286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867616075776, "text": "D-Savage 3900��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2557975014, "name": "K", "screen_name": "Uribaay", "lang": "en", "location": "California, USA", "create_at": date("2014-06-09"), "description": "DOOMSDAY", "followers_count": 128, "friends_count": 99, "statues_count": 5389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867632971776, "text": "Definitely not a beer drinker", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4891722065, "name": "papi Glo", "screen_name": "DaWiz_Jiz", "lang": "en", "location": "null", "create_at": date("2016-02-09"), "description": "null", "followers_count": 119, "friends_count": 143, "statues_count": 1430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-03-14T00:00:03.000Z"), "id": 709272867771387904, "text": "@carlaxxxo_ �� I miss you too!!", "in_reply_to_status": 709272771654627328, "in_reply_to_user": 2328091273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2328091273 }}, "user": { "id": 76770020, "name": "wendy", "screen_name": "___wendyy", "lang": "en", "location": "isu / chicago", "create_at": date("2009-09-23"), "description": "lose yourself to improve yourself", "followers_count": 706, "friends_count": 530, "statues_count": 39557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868333297666, "text": "Wish someone cared", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1424374722, "name": "Michaela Jo(:", "screen_name": "MichaelaJo", "lang": "en", "location": "jacobs bae", "create_at": date("2013-05-12"), "description": "RIP 3-25-14 & 10-25-14♡", "followers_count": 516, "friends_count": 980, "statues_count": 5828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868438147072, "text": "I'm so much cuter on the Internet https://t.co/fYc73TNgV8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1300618728, "name": "jub", "screen_name": "jubilee_jaira", "lang": "en", "location": "ak", "create_at": date("2013-03-25"), "description": "I'm kinda a cool person if you close your eyes and think of someone else.", "followers_count": 296, "friends_count": 329, "statues_count": 13113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868568174592, "text": "really mad and I need somebody to vent to.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332597845, "name": "angel✨", "screen_name": "youl0ve_a", "lang": "en", "location": "on ya mind", "create_at": date("2011-07-09"), "description": "ℓσυιѕιαиα gαℓ ✨ | .. ĸeep ιт cυтe.| snapchat: zatgirl.angel | JUN17R☺️ | deja❤️", "followers_count": 2306, "friends_count": 1546, "statues_count": 61836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bastrop, LA", "id": "76a017bc522edb14", "name": "Bastrop", "place_type": "city", "bounding_box": rectangle("-91.953951,32.749751 -91.868468,32.803653") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22067, "countyName": "Morehouse", "cityID": 2204685, "cityName": "Bastrop" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868761128962, "text": "bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 717214195, "name": "lex", "screen_name": "_lexx559", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "null", "followers_count": 744, "friends_count": 717, "statues_count": 20351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Firebaugh, CA", "id": "1bf4409ce9f41ef2", "name": "Firebaugh", "place_type": "city", "bounding_box": rectangle("-120.491468,36.83684 -120.428897,36.88301") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 624134, "cityName": "Firebaugh" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868773691393, "text": "Ugh why can't I sleep? #perfectTiming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "perfectTiming" }}, "user": { "id": 744225391, "name": "Jake From State Farm", "screen_name": "MastersPhoto74", "lang": "en", "location": "Morgan Hill, CA", "create_at": date("2012-08-07"), "description": "Masters Photography #ThePhotoGuy", "followers_count": 261, "friends_count": 554, "statues_count": 6921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgan Hill, CA", "id": "e872bcd2497287a7", "name": "Morgan Hill", "place_type": "city", "bounding_box": rectangle("-121.70014,37.088404 -121.583333,37.16931") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649278, "cityName": "Morgan Hill" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272868920500227, "text": "Beauty in profile. More pictures to come... @ Ayres Hotel & Suites Costa Mesa/Newport Beach https://t.co/RMdpwQNLd3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.88177052,33.66549564"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101128816, "name": "Lauren Salter", "screen_name": "lsesalter", "lang": "en", "location": "Lake Placid, NY", "create_at": date("2010-01-01"), "description": "USA Skeleton. Lover of theatre and film. Passionate women's soccer fan. Harborer of a continuous desire to play outside. Proud alumna of NAU and πβφ.", "followers_count": 593, "friends_count": 291, "statues_count": 23044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272869092659200, "text": "Aaaand that was on the #trail trying to find that #beautiful #beach ��#Exploring #Washington… https://t.co/igTBKt2ODn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.43333317,47.6610266"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "trail", "beautiful", "beach", "Exploring", "Washington" }}, "user": { "id": 434179604, "name": "Alaa Bukhari", "screen_name": "Allooosh96", "lang": "en", "location": "Jeddah, SA / Pullman, Wa", "create_at": date("2011-12-11"), "description": "Leaving footprints all over the world, and doing my best to live each moment of my life . I tweet what I like, I feel, and what I want to. #Adventurer #مبتعث", "followers_count": 528, "friends_count": 155, "statues_count": 14208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272869151191040, "text": "Kourtney still cute tho https://t.co/P1AaSL28cZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175190341, "name": "Carla", "screen_name": "cvrlv_mvrtinez", "lang": "en", "location": "Oakland • San Gaspar • UCSB", "create_at": date("2010-08-05"), "description": "null", "followers_count": 562, "friends_count": 387, "statues_count": 24867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272869293977602, "text": "Too tired to study but to awake to sleep ?¿?¿?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735898587, "name": "Ari", "screen_name": "la_rosaa8", "lang": "en", "location": "Hartford, CT", "create_at": date("2014-08-06"), "description": "Never not making my dreams a reality.", "followers_count": 321, "friends_count": 233, "statues_count": 4960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hartford, CT", "id": "485ebc6dbebdbf32", "name": "West Hartford", "place_type": "city", "bounding_box": rectangle("-72.786564,41.717959 -72.713899,41.806675") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 982660, "cityName": "West Hartford" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272869419622400, "text": "@milkmanslick I don't care what you say,but I'm old enough to remember when America never shut her doors closed to people in dire need.", "in_reply_to_status": 709270914613903360, "in_reply_to_user": 3160110589, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4840757947 }}, "user": { "id": 3160110589, "name": "Ricky Nixon", "screen_name": "rmiltonnixon", "lang": "en", "location": "null", "create_at": date("2015-04-16"), "description": "I'm a retired desert storm veteran who's seen and fought the enemy in the desert storm. I'm only out for freedom and justice and liberty.Ideas many died for.", "followers_count": 559, "friends_count": 1179, "statues_count": 7015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, OR", "id": "7520fc0be21c62bf", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-122.96154,42.288726 -122.776437,42.398452") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4147000, "cityName": "Medford" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272870174597120, "text": "@alisoncollins6 or your story �������� this is me on the daily lmao", "in_reply_to_status": 709272496256593920, "in_reply_to_user": 521802627, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521802627 }}, "user": { "id": 1721290063, "name": "MØMØ", "screen_name": "mo_wenner5874", "lang": "en", "location": "WA", "create_at": date("2013-09-01"), "description": "GREEK&GERMAN • EVENT MARKETING • BARISTA • 206&360 • SC moniquewenner XOXO", "followers_count": 270, "friends_count": 233, "statues_count": 5858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272870199758848, "text": "@dommm_c wow cool man", "in_reply_to_status": 709272790281490432, "in_reply_to_user": 122868327, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 122868327 }}, "user": { "id": 181386144, "name": "♛poeticprincess", "screen_name": "blasianpapii", "lang": "en", "location": "IE", "create_at": date("2010-08-21"), "description": "As Salaamu Alaykum ✨ (May Peace Be With You) Poetry & ++ Vibes are my thing", "followers_count": 4534, "friends_count": 3677, "statues_count": 36645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272870430580736, "text": "Setrumomeverbh00ib https://t.co/tb8y2XSvPo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2555606919, "name": "#TrulyPureBeauty_", "screen_name": "YouniquebyKerri", "lang": "en", "location": "#FL #USA #CA #AU #NZ #UK #WW", "create_at": date("2014-05-20"), "description": "#Pure #EcoGlow #EarthFriendly #Clean #Green #Beauty #Natural #Safe #Vegan #Nontoxic #CrueltyFree #ChemicalFree #GlutenFree #join #host #shop", "followers_count": 678, "friends_count": 2705, "statues_count": 2074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Harbor, FL", "id": "c5a8a3c8523b835e", "name": "Palm Harbor", "place_type": "city", "bounding_box": rectangle("-82.786128,28.048627 -82.707574,28.1231") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1254350, "cityName": "Palm Harbor" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272870539538432, "text": "Why am I always the one to be pushed away?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334295813, "name": "carlie hauck", "screen_name": "carlaayyyhauckk", "lang": "en", "location": "null", "create_at": date("2011-07-12"), "description": "catch me at Disneyland with @princessevison ✨", "followers_count": 544, "friends_count": 494, "statues_count": 18815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun Village, CA", "id": "01f12d076169f380", "name": "Sun Village", "place_type": "city", "bounding_box": rectangle("-117.99682,34.528244 -117.916342,34.58717") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 677308, "cityName": "Sun Village" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272871047028736, "text": "My back will never be the same. Dance is tougher than ever now��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2867150225, "name": "& Cola! ✨", "screen_name": "ReezeDollaz", "lang": "en", "location": "Los Angeles, CA☀️", "create_at": date("2014-11-08"), "description": "LIVING | FFC16 AA | snapchat:Deniseromans", "followers_count": 1721, "friends_count": 784, "statues_count": 9246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272871063961600, "text": "I mean, it's more akin to fixing myself after the depression builds up and I fall apart. Or after panic attacks.", "in_reply_to_status": 709272355219087360, "in_reply_to_user": 118628464, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 118628464, "name": "Brian White", "screen_name": "TheWhyOfBri", "lang": "en", "location": "NYC", "create_at": date("2010-03-01"), "description": "Video games writer/dev+other stuff. INTJ. Editor @currentdigital, GOTW guy on @Kotaku TAY. DM's open. Tweets about games and mental health. I think you're cool.", "followers_count": 296, "friends_count": 494, "statues_count": 9637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272871600824320, "text": "Higher grades .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258153053, "name": "Jay.", "screen_name": "JoclandLauren", "lang": "en", "location": "East Atlanta Zone 6 ", "create_at": date("2011-02-26"), "description": "20|Clark Atlanta#hbcu | motivated |", "followers_count": 2277, "friends_count": 2360, "statues_count": 103989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272871613280257, "text": "Whoopty whoop Blahzay blah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446483037, "name": "feel.x", "screen_name": "YungFelix_", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "Texsus '16", "followers_count": 200, "friends_count": 341, "statues_count": 5832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-03-14T00:00:04.000Z"), "id": 709272871634411520, "text": "Nope fucking nope https://t.co/n0hjQgN0ly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521674954, "name": "Yassssica ☯", "screen_name": "ISpam_JessiMar", "lang": "en", "location": "302", "create_at": date("2012-03-11"), "description": "Play me on league of legends - Saber Goddess Insta- ispam_jessimar", "followers_count": 2125, "friends_count": 713, "statues_count": 37118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wyoming, DE", "id": "ca92369c50478227", "name": "Wyoming", "place_type": "city", "bounding_box": rectangle("-75.568007,39.107585 -75.547386,39.127739") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1081350, "cityName": "Wyoming" } }
+{ "create_at": datetime("2016-03-14T00:00:05.000Z"), "id": 709272872338874368, "text": "@n8_speedo ����", "in_reply_to_status": 709272669988847618, "in_reply_to_user": 3312731707, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3312731707 }}, "user": { "id": 792601412, "name": "Desiree Avalos", "screen_name": "DesireeAvalos", "lang": "en", "location": "United States of America", "create_at": date("2012-08-30"), "description": "C/O 2016", "followers_count": 464, "friends_count": 334, "statues_count": 2661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-03-14T00:00:05.000Z"), "id": 709272872389255168, "text": "Believe it or not... That is NOT a travel https://t.co/SfdLcfNtXO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247154952, "name": "D-Conn™", "screen_name": "DCONN24", "lang": "en", "location": "Mountain Home", "create_at": date("2011-02-03"), "description": "Asst. Coach for @MHHSBBall. Announcer for @RazorbackHockey Cubs/Bulls/Blackhawks/Razorbacks/Bombers/Bears. Bat Flip Choreographer. You got games on your phone?", "followers_count": 1208, "friends_count": 2374, "statues_count": 102278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain Home, AR", "id": "69e8d4996cf22753", "name": "Mountain Home", "place_type": "city", "bounding_box": rectangle("-92.436037,36.309943 -92.312154,36.382252") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5005, "countyName": "Baxter", "cityID": 547390, "cityName": "Mountain Home" } }
+{ "create_at": datetime("2016-03-14T00:00:05.000Z"), "id": 709272872838103040, "text": "I'm not tryna cum home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49859259, "name": "WHEW", "screen_name": "TheRealWhew", "lang": "en", "location": "Lil Bel Air ", "create_at": date("2009-06-22"), "description": "I need every dollar Ig: @TheRealWhew 1manArmy", "followers_count": 1168, "friends_count": 488, "statues_count": 20357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-03-14T00:00:05.000Z"), "id": 709272872934445058, "text": "Larry Kramer Sighting: New York, New York/ The Public Theater https://t.co/BwotNKjsRC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.991759,40.728957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81237494, "name": "StarSightings", "screen_name": "ssbot", "lang": "en", "location": "Everywhere celebs are", "create_at": date("2009-10-09"), "description": "ALL CONFIRMED celebrity and public figure sightings posted here in near real-time as possible. Get the mobile app: http://www.starsightings.com/download.html", "followers_count": 2137, "friends_count": 1, "statues_count": 261314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-14T00:00:05.000Z"), "id": 709272873286799361, "text": "I can't believe I was in the same room as @justinbieber tonight ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 402245879, "name": "Lauren Conway", "screen_name": "laurenkconway", "lang": "en", "location": "the palace", "create_at": date("2011-10-31"), "description": "metro trusts me", "followers_count": 1476, "friends_count": 1250, "statues_count": 1544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635239069351936, "text": "But how da boyfriend so perfect ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61119096, "name": "Byrdie", "screen_name": "hannahbyrd", "lang": "en", "location": "null", "create_at": date("2009-07-28"), "description": "Made ya look", "followers_count": 266, "friends_count": 264, "statues_count": 8361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635239455272960, "text": "My kids moms are cool\n#Mykidsmomsarecool\n#SkyLeeVague \n#SONSOFLA… https://t.co/J1eGZC2nBy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.07042439,33.7176981"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Mykidsmomsarecool", "SkyLeeVague", "SONSOFLA" }}, "user": { "id": 2472906745, "name": "Endele Wilson", "screen_name": "SKYLEEVAGUE", "lang": "en", "location": "null", "create_at": date("2014-05-01"), "description": "null", "followers_count": 250, "friends_count": 357, "statues_count": 922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635240352854017, "text": "2️⃣0️⃣0️⃣?¿", "in_reply_to_status": 702376178246615040, "in_reply_to_user": 139201977, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 139201977, "name": "Lex", "screen_name": "Lexciii_", "lang": "en", "location": "Federal Way, WA", "create_at": date("2010-05-01"), "description": "IG: alexciateang_ SC: alexcia_teang Cambodian #LongLiveLorenzo Beamer", "followers_count": 1417, "friends_count": 771, "statues_count": 75815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635240763858944, "text": "@wyatthillard if I had a dollar for every time I've almost fallen asleep reading to little kids..... ��", "in_reply_to_status": 709635035029118977, "in_reply_to_user": 1878093535, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1878093535 }}, "user": { "id": 557268914, "name": "Tay's Anatomy", "screen_name": "tay_erickson", "lang": "en", "location": "☂ Pacific Northwest ☂ ", "create_at": date("2012-04-18"), "description": "I love good coffee, good books, and good people in that particular order // ✨Senior at Hockinson✨", "followers_count": 680, "friends_count": 478, "statues_count": 11278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camas, WA", "id": "219460d3060c1d2e", "name": "Camas", "place_type": "city", "bounding_box": rectangle("-122.474225,45.57579 -122.367318,45.633039") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5309480, "cityName": "Camas" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635240881364992, "text": "it ain't where she's at , it's where she wanna be . �� and that's with me . ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1381315238, "name": "king dyke virgin❗️", "screen_name": "__iamjazz", "lang": "en", "location": "SUNSHINE , LOUISIANA", "create_at": date("2013-04-26"), "description": "rest in peace toad and uncle malcolm . ❤️ . 11-30-97 | 5-10-15 . ima keep yo name alive . ❤️ #SMOOVEWAY❗️king$moove beedy❤️", "followers_count": 1149, "friends_count": 728, "statues_count": 110883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635241233784832, "text": "Deadass. �� this is what the world has come to though? https://t.co/iqz8DODV37", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114797767, "name": "Kay✌️", "screen_name": "QueenKaaay__", "lang": "en", "location": "null", "create_at": date("2010-02-16"), "description": "Kik: Kr213 IG:kaaaadijah_____ & snapchat: kaaaadijah", "followers_count": 1275, "friends_count": 2343, "statues_count": 15551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Station, NY", "id": "37d3d85288d83e69", "name": "Huntington Station", "place_type": "city", "bounding_box": rectangle("-73.432183,40.828166 -73.370619,40.866629") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3637044, "cityName": "Huntington Station" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635241376284672, "text": "Lmao stfu https://t.co/JMgtS2Rj6u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544342024, "name": "the god", "screen_name": "hoez4sale", "lang": "en", "location": "Hennessy Beach", "create_at": date("2012-04-03"), "description": "I think pimp, therefore I am pimp. #longlivegotti @1waynostra @gbabythageneral", "followers_count": 9439, "friends_count": 1043, "statues_count": 93628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Germantown, MD", "id": "0b8f86488a6ebab7", "name": "Germantown", "place_type": "city", "bounding_box": rectangle("-77.31732,39.12466 -77.21136,39.215563") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2432025, "cityName": "Germantown" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635241384787968, "text": "@manerayo go to sleep", "in_reply_to_status": 709634825959813120, "in_reply_to_user": 50819985, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 50819985 }}, "user": { "id": 431025761, "name": "kristen arias", "screen_name": "kriiistenarias", "lang": "en", "location": "NY", "create_at": date("2011-12-07"), "description": "null", "followers_count": 1337, "friends_count": 353, "statues_count": 48995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635241783091200, "text": "Stop Jeffn, they don't fwu ... ®☥®M™", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 400448621, "name": "♫Mr. Carradine ®☥®M™", "screen_name": "CarradineShaad", "lang": "en", "location": "♫ Kansas City, Missouri♫", "create_at": date("2011-10-28"), "description": "♌ CEO #ReaLifeRealMuzic Singer/Rapper #YoungEntrepreneur Everyday I'm on a mission. With God All Things are Possible. 19:26 IG carradineshaad", "followers_count": 585, "friends_count": 520, "statues_count": 20955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635242064224256, "text": "@cocaingel_ ��", "in_reply_to_status": 709605341944631297, "in_reply_to_user": 958197228, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 958197228 }}, "user": { "id": 3243990601, "name": "andrew lowkzzz", "screen_name": "andrewlowkzz", "lang": "en", "location": "West Whittier, CA", "create_at": date("2015-06-12"), "description": "tomorrow is not promised", "followers_count": 256, "friends_count": 524, "statues_count": 4270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seal Beach, CA", "id": "d35feae19d268b09", "name": "Seal Beach", "place_type": "city", "bounding_box": rectangle("-118.117365,33.723095 -118.041655,33.787561") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 670686, "cityName": "Seal Beach" } }
+{ "create_at": datetime("2016-03-15T00:00:00.000Z"), "id": 709635242332520448, "text": "I just need to take breather sometimes so I won't go off lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350409230, "name": "Diane Pimentel", "screen_name": "Causeimdianee", "lang": "en", "location": "null", "create_at": date("2011-08-07"), "description": "Pain is just for an instance. ill be good no matter what im faced with. #BRONCOS", "followers_count": 184, "friends_count": 178, "statues_count": 8539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635243519528961, "text": "I DONT CARE WHAT YOU DO HAVE FUN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2718433818, "name": "h", "screen_name": "hvnnvhr_", "lang": "en", "location": "null", "create_at": date("2014-08-08"), "description": "age ❤️", "followers_count": 337, "friends_count": 201, "statues_count": 2042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helotes, TX", "id": "00a1b28251c671b8", "name": "Helotes", "place_type": "city", "bounding_box": rectangle("-98.764721,29.495566 -98.661448,29.604798") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635243653861377, "text": "bet you wish you were done ������ https://t.co/O1eTIZ7rCP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146653497, "name": "Nicoley Powers", "screen_name": "Nikkay1447", "lang": "en", "location": "Baldwinsville, NY", "create_at": date("2010-05-21"), "description": "nicole// I'm just striving to be a better person//", "followers_count": 331, "friends_count": 742, "statues_count": 3931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canandaigua, NY", "id": "248fb663ecee6ab6", "name": "Canandaigua", "place_type": "city", "bounding_box": rectangle("-77.316269,42.839834 -77.239245,42.927235") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36069, "countyName": "Ontario", "cityID": 3612144, "cityName": "Canandaigua" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635243716640769, "text": "Wanna see this �� https://t.co/x7ojjBdsPP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2159964236, "name": "c太rlos", "screen_name": "Carlossssss__", "lang": "en", "location": "null", "create_at": date("2013-10-27"), "description": "17 | Speed kills #34", "followers_count": 517, "friends_count": 191, "statues_count": 18409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Benito, TX", "id": "b8f3a7fd432ec1f1", "name": "San Benito", "place_type": "city", "bounding_box": rectangle("-97.684281,26.07277 -97.575927,26.172319") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4865036, "cityName": "San Benito" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635243750203392, "text": "@gudkidoari_ have fun", "in_reply_to_status": 709635067472023553, "in_reply_to_user": 204398978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 204398978 }}, "user": { "id": 4658346271, "name": "Ugly Antho", "screen_name": "yt__antho", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "stay true to yourself", "followers_count": 1086, "friends_count": 907, "statues_count": 4194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635244001865728, "text": "If he had his way, @GiladAtzmon would prefer that no Jews existed and that is not an exaggeration. @IdoDaniel https://t.co/VDaGW41gg1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42750334, 349047223 }}, "user": { "id": 36164779, "name": "Steve Kassel", "screen_name": "TaxNegotiator", "lang": "en", "location": "Burlingame, CA, USA", "create_at": date("2009-04-28"), "description": "Nation's Leading Tax Negotiator. Good Morning America & O'Reilly Factor. Maryland bball & tennis mgr. Defender of Israel & US-Israel relationship. Go Ravens!", "followers_count": 18979, "friends_count": 1116, "statues_count": 4196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635244249391104, "text": "Can someone please hang out with me rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2423088883, "name": "E.R.G.D", "screen_name": "emsicklecell", "lang": "en", "location": "90006/91325", "create_at": date("2014-04-01"), "description": "breathtakingly fucked", "followers_count": 107, "friends_count": 94, "statues_count": 8673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635244798902272, "text": "@kokiecola u can do it�� I believe in u", "in_reply_to_status": 709634984320049152, "in_reply_to_user": 400422577, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 400422577 }}, "user": { "id": 1250318246, "name": "RAE", "screen_name": "Raeocchipinti", "lang": "en", "location": "A queen living in Queens", "create_at": date("2013-03-07"), "description": "you'll see. info.raeocchipinti@gmail.com", "followers_count": 21276, "friends_count": 265, "statues_count": 15132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635244870111237, "text": "@JosalynGiraffe omg yes girl ��", "in_reply_to_status": 709634705973387264, "in_reply_to_user": 213032482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 213032482 }}, "user": { "id": 210608976, "name": "Taco Belle ✨", "screen_name": "_adsoul", "lang": "en", "location": "Ellensburg, WA", "create_at": date("2010-10-31"), "description": "✝✨", "followers_count": 754, "friends_count": 539, "statues_count": 30104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellensburg, WA", "id": "c95cdb2a983262e5", "name": "Ellensburg", "place_type": "city", "bounding_box": rectangle("-120.582586,46.958017 -120.49726,47.028542") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53037, "countyName": "Kittitas", "cityID": 5321240, "cityName": "Ellensburg" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245088190464, "text": "@ebbtideapp Tide in Pearlington, Mississippi 03/15/2016\n Low 6:50am -0.3\nHigh 8:06pm 1.1\n Low 7:54am -0.3\nHigh 9:15pm 1.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-89.615,30.24"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 41, "friends_count": 1, "statues_count": 15931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28045, "countyName": "Hancock", "cityID": 2855920, "cityName": "Pearlington" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245125992452, "text": "�������� https://t.co/O8XltKnoWL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 631620342, "name": "Lubna", "screen_name": "SweetassLu", "lang": "en", "location": "Racine", "create_at": date("2012-07-09"), "description": "null", "followers_count": 1213, "friends_count": 686, "statues_count": 45599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245235023872, "text": "@ShawnMendes I love you.", "in_reply_to_status": -1, "in_reply_to_user": 379408088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 379408088 }}, "user": { "id": 299279321, "name": "layne", "screen_name": "TheFast_Layne", "lang": "en", "location": "null", "create_at": date("2011-05-15"), "description": "null", "followers_count": 1588, "friends_count": 394, "statues_count": 68201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland Heights, MO", "id": "fb26223d27a11fbb", "name": "Maryland Heights", "place_type": "city", "bounding_box": rectangle("-90.500805,38.675851 -90.407348,38.756043") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2946586, "cityName": "Maryland Heights" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245352464384, "text": "@pretty_visitor_ https://t.co/gZmj001CUC", "in_reply_to_status": -1, "in_reply_to_user": 1119590077, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1119590077 }}, "user": { "id": 1611674137, "name": "deep voice", "screen_name": "vladimirsork", "lang": "ru", "location": "Hollywood, Los Angeles", "create_at": date("2013-07-21"), "description": "The Lee Strasberg Theatre and Film Institute. Vladimir Sork. ♥️@pretty_visitor_♥️", "followers_count": 42, "friends_count": 48, "statues_count": 822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245352611840, "text": "Real friends is a song that's gonna last many MANY years..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63084512, "name": "huntər", "screen_name": "HNEU2495", "lang": "en", "location": "null", "create_at": date("2009-08-05"), "description": "make your own luck --- love&loyalty #NFLSU", "followers_count": 524, "friends_count": 430, "statues_count": 10527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oshkosh, WI", "id": "01e2555c756cd031", "name": "Oshkosh", "place_type": "city", "bounding_box": rectangle("-88.653224,43.963944 -88.477447,44.102608") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55139, "countyName": "Winnebago", "cityID": 5560500, "cityName": "Oshkosh" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245398560768, "text": "if someone is pissed at you, let it be their problem, not yours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 895979953, "name": "trevor", "screen_name": "TrevorStathatos", "lang": "en", "location": "Flower Mound, TX", "create_at": date("2012-10-21"), "description": "mission is D3", "followers_count": 379, "friends_count": 207, "statues_count": 1112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flower Mound, TX", "id": "d6bbfe2356aebaab", "name": "Flower Mound", "place_type": "city", "bounding_box": rectangle("-97.178734,32.981704 -97.027543,33.078462") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4826232, "cityName": "Flower Mound" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635245977436161, "text": "Happy birthday to my Bestfriend ❤️�� @MMB_21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 494920619 }}, "user": { "id": 3096888860, "name": "Zeus ⚡️", "screen_name": "brian_coreas7", "lang": "en", "location": "South Hill, WA", "create_at": date("2015-03-18"), "description": "| LA ✈️ WA | Semi pro⚽️⚽️ | sc : briancoreas69", "followers_count": 236, "friends_count": 212, "statues_count": 4942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246409568256, "text": "Never Respect No one Who Doesn't Respect U.....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242025847, "name": "Best DJ In The South", "screen_name": "DJJukeboxlive", "lang": "en", "location": "Mississippi, USA", "create_at": date("2011-01-23"), "description": "Young DJ From Mississippi | This Month Bangers |#OfficialDJ | For Slots Or Booking - DJJukebox601@gmail.com", "followers_count": 5292, "friends_count": 3742, "statues_count": 57662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hattiesburg, MS", "id": "2b34df148a211c3e", "name": "Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.427669,31.241189 -89.248409,31.380085") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28035, "countyName": "Forrest", "cityID": 2831020, "cityName": "Hattiesburg" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246417907712, "text": "Ripley SW Limestone Co. Temp: 52.5°F Wind:0.0mph Pressure: 990.4mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 47, "friends_count": 32, "statues_count": 50227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246485016576, "text": "03/15@03:00 - Temp 44.7F, WC 44.7F. Wind 1.0mph NNW, Gust 4.0mph. Bar 29.860in, Falling slowly. Rain 0.00in. Hum 98%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246489141249, "text": "Thank you boogs. https://t.co/aRfd4cvXDj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2244675805, "name": "YungFreemason", "screen_name": "AlessioNaji", "lang": "en", "location": "Houston, TX", "create_at": date("2013-12-13"), "description": "null", "followers_count": 602, "friends_count": 455, "statues_count": 22388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Territory, TX", "id": "9ddf6d74b857d09a", "name": "New Territory", "place_type": "city", "bounding_box": rectangle("-95.715664,29.571852 -95.659771,29.608493") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4851366, "cityName": "New Territory" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246711410689, "text": "I'm crazy about chu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1326232015, "name": "Liam Apple", "screen_name": "soyelchamp", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "change your perspective", "followers_count": 244, "friends_count": 399, "statues_count": 670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246719963140, "text": "Wind 3.0 mph ESE. Barometer 29.703 in, Falling. Temperature 58.6 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246782685185, "text": "#SupportOriginMelissa 55.6°F Wind:0.0mph Pressure: 29.85hpa Rising slowly Rain Today 0.00in. Forecast: Becoming fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 312434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635246837403648, "text": "Temp: 62.3°F Wind:0.0mph Pressure: 29.862hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-15T00:00:01.000Z"), "id": 709635247101480960, "text": "Young metro young metro young metro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1357221258, "name": "Joe Joe", "screen_name": "LuceroJoey", "lang": "en", "location": "null", "create_at": date("2013-04-16"), "description": "San Diego", "followers_count": 546, "friends_count": 529, "statues_count": 8486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635247416029184, "text": "#6Days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "6Days" }}, "user": { "id": 147826252, "name": "T'Challa", "screen_name": "BigDoeb", "lang": "en", "location": "Upland, CA", "create_at": date("2010-05-24"), "description": "Loading...", "followers_count": 351, "friends_count": 147, "statues_count": 67861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635247428644865, "text": "Hi I am speaking fwhite and I make this for my mom every Tuesday https://t.co/XqcevQBKIV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207324262, "name": "breana", "screen_name": "BreanaLeaBailey", "lang": "en", "location": "null", "create_at": date("2010-10-24"), "description": "y'all ain't my type", "followers_count": 687, "friends_count": 314, "statues_count": 6131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vidor, TX", "id": "f9c24307c964621c", "name": "Vidor", "place_type": "city", "bounding_box": rectangle("-94.039128,30.079067 -93.944553,30.187285") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48361, "countyName": "Orange", "cityID": 4875476, "cityName": "Vidor" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635247529267200, "text": "'Scuse me, just askin' .. thought I was in 'merica last time I looked.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374901847, "name": "Sauk River Review", "screen_name": "OldSaukRiver", "lang": "en", "location": "#FeelTheBern #USA", "create_at": date("2011-09-16"), "description": "American imagination artist living in the Pacific Northwest. Visit http://oldsaukriver.blogspot.com/ and hang out with me by the riverside. Peace.", "followers_count": 2136, "friends_count": 1935, "statues_count": 50312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635247923539970, "text": "If you down for me I will always hit you up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1164362888, "name": "Erabeezy", "screen_name": "EAurnab", "lang": "en", "location": "Houston, TX", "create_at": date("2013-02-09"), "description": "Certified Mind Blower", "followers_count": 17, "friends_count": 109, "statues_count": 541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635248074543105, "text": "66.6F (Feels: 66.6F) - Humidity: 92% - Wind: 12.1mph SW - Gust: 15.2mph - Pressure: 1017.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 230748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635248510795776, "text": "I've finally come to terms that once again I was the side hoe all along��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3430835052, "name": "asia", "screen_name": "isaleche", "lang": "en", "location": "null", "create_at": date("2015-09-02"), "description": "☹", "followers_count": 60, "friends_count": 45, "statues_count": 16 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635248540098560, "text": "@YuhBoiiFerd @phillywin faggots", "in_reply_to_status": 709635188926472192, "in_reply_to_user": 2405233236, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2405233236, 1615967912 }}, "user": { "id": 438668607, "name": "March 15th wya", "screen_name": "_Patel18", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2011-12-16"), "description": "You asked for my hustle, I gave you my heart -Kobe Bryant", "followers_count": 457, "friends_count": 374, "statues_count": 9638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635248896626692, "text": "Happy birthday @nickjoseph80", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2960251358 }}, "user": { "id": 604978078, "name": "Jazzy♡", "screen_name": "JazminGurrola0", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "I luv pugs", "followers_count": 801, "friends_count": 219, "statues_count": 6329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635248951267329, "text": "eyeshadow not even blended... And people commented on this picture complimenting her \"skills\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91959239, "name": "Bre", "screen_name": "breannaleeivey", "lang": "en", "location": "null", "create_at": date("2009-11-22"), "description": "null", "followers_count": 1023, "friends_count": 754, "statues_count": 25143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249068597250, "text": "Do your best and let God do \nthe rest.\nBen Carson", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 143, "friends_count": 260, "statues_count": 244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249173565440, "text": "@cashakazaam i watched it high as balls and i couldnt even disagree w her", "in_reply_to_status": 709635039965941760, "in_reply_to_user": 1676699318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1676699318 }}, "user": { "id": 429847964, "name": "soba", "screen_name": "Sober_Lama", "lang": "en", "location": "somewhere being a nuisance", "create_at": date("2011-12-06"), "description": "Dont get lost in the sauce.", "followers_count": 1123, "friends_count": 849, "statues_count": 47983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miramar, FL", "id": "5ca2cc5afa0894df", "name": "Miramar", "place_type": "city", "bounding_box": rectangle("-80.40816,25.956799 -80.205968,25.995449") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1245975, "cityName": "Miramar" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249240539136, "text": "I give more advice to people than I do myself. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175934167, "name": "kevs", "screen_name": "YNGKEVS", "lang": "en", "location": "Lincoln, CA", "create_at": date("2010-08-07"), "description": "Está Quemado Familia. #RDMTM", "followers_count": 312, "friends_count": 271, "statues_count": 17988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, CA", "id": "0026c52b089aab9c", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-121.340149,38.839106 -121.222598,38.911502") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 641474, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249341247488, "text": "Yesss and u think so hard that U can't sleep and u have to tweet about it�� https://t.co/7lzz6lLtUU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3168076567, "name": "Kalea Caggan", "screen_name": "k_kurlz", "lang": "en", "location": "null", "create_at": date("2015-04-22"), "description": "Dream big or don't dream at all...", "followers_count": 351, "friends_count": 365, "statues_count": 2892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249471291392, "text": "#bitchperfect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "bitchperfect" }}, "user": { "id": 149997183, "name": "Her", "screen_name": "_DevonNicole", "lang": "en", "location": "West Coast (818)", "create_at": date("2010-05-30"), "description": "She a canine at heart, she a dog.", "followers_count": 237, "friends_count": 295, "statues_count": 9561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249626599424, "text": "Wind 0.0 mph ---. Barometer 29.807 in, Rising slowly. Temperature 51.3 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635249769156608, "text": "It's 3am doe ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2408140069, "name": "Champagne Mami ✨", "screen_name": "Thereal6goddess", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-03-23"), "description": "fuck em we ball", "followers_count": 2270, "friends_count": 999, "statues_count": 71398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Valley, GA", "id": "5fb7d0858055aa39", "name": "Fort Valley", "place_type": "city", "bounding_box": rectangle("-83.914883,32.521974 -83.865501,32.574552") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13225, "countyName": "Peach", "cityID": 1331096, "cityName": "Fort Valley" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635250448572416, "text": "Dammm this boy asked me for my number ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2866308553, "name": "$tezzy-a", "screen_name": "amandasolano63", "lang": "en", "location": "null", "create_at": date("2014-10-19"), "description": "avoid the negative & you lit. | wolf | tuc town", "followers_count": 336, "friends_count": 475, "statues_count": 3676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635250687639552, "text": "hardest stage ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155719291, "name": "myrna gonzalez cantu", "screen_name": "myrnagzzcantu", "lang": "en", "location": "monterrey; n.l. mexico", "create_at": date("2010-06-14"), "description": "Periodontist, Implantologist, ❤️ #dentalsurgery #loveyourgums #surgerypassionate #fashiondentistry", "followers_count": 192, "friends_count": 416, "statues_count": 606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635250993762304, "text": "You might be from Illinois if you've heard the term \"Chicagoland.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403977318, "name": "Bill Graner ☕️ [GDC]", "screen_name": "funwithbill", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-11-03"), "description": "Game design, Twitch casting, jokes, and strife. Next Mostly Walking: 3/21/16!", "followers_count": 3659, "friends_count": 727, "statues_count": 23594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635251065135104, "text": "This last 2\\11 will be the death of me ....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252089520, "name": "Tysaiah Haley", "screen_name": "Tah_Haley1", "lang": "en", "location": "null", "create_at": date("2011-02-14"), "description": "P.T.K or DIE", "followers_count": 929, "friends_count": 890, "statues_count": 21269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minot, ND", "id": "2ae0ed991bd2e8d6", "name": "Minot", "place_type": "city", "bounding_box": rectangle("-101.357743,48.182195 -101.178137,48.283425") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38101, "countyName": "Ward", "cityID": 3853380, "cityName": "Minot" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635251262193667, "text": "@Kasaikage Filthy Frank described you three to a tee, fam...", "in_reply_to_status": 709635158131933184, "in_reply_to_user": 310952715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310952715 }}, "user": { "id": 572981684, "name": "KUOKCWSUKCSTL8556", "screen_name": "brian8556utube", "lang": "en", "location": "Bel Aire, KS, USA", "create_at": date("2012-05-06"), "description": "God first, Family second, Sports third. God is my Rock. #RockChalk #WatchUs #ForeverRoyal #ChiefsKingdom #ThunderUp #LGB #USMNT #SKC #BlitzTeam #Libertarian", "followers_count": 1587, "friends_count": 2615, "statues_count": 185866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bel Aire, KS", "id": "8530e930d5428ce9", "name": "Bel Aire", "place_type": "city", "bounding_box": rectangle("-97.284762,37.752216 -97.244558,37.78135") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2005337, "cityName": "Bel Aire" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635251396542464, "text": "Wind 0.0 mph ---. Barometer 29.812 in, Steady. Temperature 58.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635251409059844, "text": "@tmjr_ ion see it lol", "in_reply_to_status": 709635194962243585, "in_reply_to_user": 4573856234, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4573856234 }}, "user": { "id": 319441270, "name": "JORDΛN", "screen_name": "__jwells2", "lang": "en", "location": "Houston, TX", "create_at": date("2011-06-17"), "description": "20. God first & foremost. #PVAMU", "followers_count": 2167, "friends_count": 1657, "statues_count": 14468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-03-15T00:00:02.000Z"), "id": 709635251476123648, "text": "who can I joce 2am beside my pillow?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2328071447, "name": "0620.", "screen_name": "_femalepac", "lang": "en", "location": "null", "create_at": date("2014-02-06"), "description": "peepshitologist..", "followers_count": 1281, "friends_count": 1124, "statues_count": 29390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635251610345472, "text": "@melinatequila @ceeewanss I am", "in_reply_to_status": 709633474383073281, "in_reply_to_user": 2383002871, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2383002871, 434014247 }}, "user": { "id": 2829180976, "name": "Alpha.", "screen_name": "killahnoe", "lang": "en", "location": "null", "create_at": date("2014-10-14"), "description": "null", "followers_count": 169, "friends_count": 90, "statues_count": 2153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waikele, HI", "id": "00b1ba9a29775fba", "name": "Waikele", "place_type": "city", "bounding_box": rectangle("-158.018703,21.390995 -157.991593,21.415112") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1576250, "cityName": "Waikele" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635252042354690, "text": "San Antonio sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2666916084, "name": "Chihuahua Girl", "screen_name": "lanabananao", "lang": "en", "location": "Houston, TX", "create_at": date("2014-07-21"), "description": "i misspell most things.", "followers_count": 120, "friends_count": 68, "statues_count": 7873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635252243841024, "text": "I hate im careless I think it stops me from a lot of things ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630827754, "name": "Tot ❣", "screen_name": "_nashayy", "lang": "en", "location": "Mississippi, USA", "create_at": date("2012-07-08"), "description": "Tot ~", "followers_count": 2523, "friends_count": 2239, "statues_count": 15952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635252851879936, "text": "this used to be my shit when I was like 6 ������ https://t.co/3xq5vw9ulL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3037361270, "name": "6 i x", "screen_name": "Jocc51o", "lang": "en", "location": "Hayward, CA", "create_at": date("2015-02-22"), "description": "@Drake ❤️", "followers_count": 187, "friends_count": 130, "statues_count": 6665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635252906385408, "text": "Feel weird holding my phone with my right hand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 135635165, "name": "tink.", "screen_name": "_MissTink", "lang": "en", "location": "Houston, TX", "create_at": date("2010-04-21"), "description": "someone rt'd me onto your tl & now here you are. might as well follow me | #Beyhive", "followers_count": 19856, "friends_count": 503, "statues_count": 105227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253095170048, "text": "my friend met this dude that drops 42gs every year on season tickets for the Warriors. like wtf that's insane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3105688136, "name": "Sam Hawkins", "screen_name": "samlhawkins_", "lang": "en", "location": "Fresno, CA", "create_at": date("2015-03-23"), "description": "null", "followers_count": 115, "friends_count": 71, "statues_count": 1554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253103632384, "text": "Airports make me feel so much better ��✈️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 741590994, "name": "Cindy", "screen_name": "MabelDuran12", "lang": "en", "location": "North Carolina, USA", "create_at": date("2012-08-06"), "description": "sometimes I miss Dallas, most times I don't.", "followers_count": 329, "friends_count": 332, "statues_count": 14050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253514559488, "text": "Posting about my man because at the end of the day he's always there for me and that makes me pretty lucky https://t.co/82hmqpciat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 590808138, "name": "vivianna beltran", "screen_name": "Vivaaaay", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "happily in love with migi❤️", "followers_count": 230, "friends_count": 268, "statues_count": 9736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253543968769, "text": "No I never reach that point ppl always change when I'm in the middle of falling for them https://t.co/d1PURH2mrC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332638685, "name": "¢ℓιиσмαиια", "screen_name": "xoxoaliiyah", "lang": "en", "location": "i'll ruin your life", "create_at": date("2011-07-09"), "description": "RIPDami✨@menacing32", "followers_count": 1729, "friends_count": 1279, "statues_count": 62588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253602639873, "text": "Its funny how you destroy my friendship with some people that I used to chill wit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4846604567, "name": "〽artin Perez ☁", "screen_name": "MP_Justice1", "lang": "en", "location": "Reedley, CA", "create_at": date("2016-01-25"), "description": "If you wanna stand out, be outstanding\n#Eliteofthekings", "followers_count": 107, "friends_count": 109, "statues_count": 252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reedley, CA", "id": "f434511e8f0c8aa4", "name": "Reedley", "place_type": "city", "bounding_box": rectangle("-119.4699,36.575137 -119.421897,36.618737") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 660242, "cityName": "Reedley" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253623595008, "text": "@Alfredomusic “Adios Nonino” from Tocororo (CD, Album, Jazz, 2016) on @MackAvenueMusic New", "in_reply_to_status": -1, "in_reply_to_user": 24847668, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 24847668, 33686150 }}, "user": { "id": 134863265, "name": "Jim Bennett", "screen_name": "formsfeelings", "lang": "en", "location": "null", "create_at": date("2010-04-19"), "description": "null", "followers_count": 351, "friends_count": 1119, "statues_count": 5391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasanton, CA", "id": "ad4876a662119b74", "name": "Pleasanton", "place_type": "city", "bounding_box": rectangle("-121.956864,37.621859 -121.798606,37.704036") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 657792, "cityName": "Pleasanton" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253745242113, "text": "I just really need your ass with me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2707329992, "name": "kotf", "screen_name": "carlos_hndz7", "lang": "en", "location": "Stanford, CA", "create_at": date("2014-08-04"), "description": "run happy", "followers_count": 151, "friends_count": 144, "statues_count": 13253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange Cove, CA", "id": "461c7cf6e96d2d78", "name": "Orange Cove", "place_type": "city", "bounding_box": rectangle("-119.331688,36.610175 -119.303893,36.638109") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 654008, "cityName": "Orange Cove" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253854285824, "text": "#GreenwoodVillage, CO #security #Job: Security Analyst - SOC at Trustwave https://t.co/vmLTEUf3DX #infosec #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0969343,38.2050032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreenwoodVillage", "security", "Job", "infosec", "Jobs", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 292, "friends_count": 160, "statues_count": 575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado, USA", "id": "e21c8e4914eef2b3", "name": "Colorado", "place_type": "admin", "bounding_box": rectangle("-109.060257,36.992427 -102.041524,41.003445") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8027, "countyName": "Custer" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253908819968, "text": "@marcelferg23 ��", "in_reply_to_status": 709634976635990017, "in_reply_to_user": 3171469885, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3171469885 }}, "user": { "id": 3775581852, "name": "M", "screen_name": "maliabananola", "lang": "en", "location": "Tacoma, WA", "create_at": date("2015-10-03"), "description": "// makeup enthusiast // YouTuber // IG & snapchat: maliabananola //", "followers_count": 329, "friends_count": 251, "statues_count": 4305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253929844737, "text": "@tommy__GUNNS @slightlywavey lol", "in_reply_to_status": 709627577703354369, "in_reply_to_user": 550634842, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 550634842, 378331552 }}, "user": { "id": 2730700927, "name": "Izabel Zambrano", "screen_name": "IzabelZambrano", "lang": "en", "location": "Kansas - Las Vegas", "create_at": date("2014-08-13"), "description": "MARK MY WORDS IMMA BALL WITHOUT YOU", "followers_count": 4767, "friends_count": 3228, "statues_count": 77001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635253954940928, "text": "Want a banana split from that fried Oreos place at the rodeo mmm��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 498441099, "name": "⠀", "screen_name": "lilvibky", "lang": "en", "location": "null", "create_at": date("2012-02-20"), "description": "Blessed. lakers. Rip bbyzach ❤️ | Liongang | LLG", "followers_count": 1352, "friends_count": 229, "statues_count": 136216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635254491996160, "text": "Wind 2.7 mph SE. Barometer 29.981 in, Falling. Temperature 37.1 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635254655520768, "text": "@OfficialKei all about vibes", "in_reply_to_status": 709635161021743104, "in_reply_to_user": 361409830, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 361409830 }}, "user": { "id": 4693243333, "name": "//G.V.O//", "screen_name": "_mube", "lang": "en", "location": "Zambia", "create_at": date("2016-01-01"), "description": "#MOD", "followers_count": 377, "friends_count": 360, "statues_count": 1910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellicott City, MD", "id": "515586774402eacb", "name": "Ellicott City", "place_type": "city", "bounding_box": rectangle("-76.952172,39.215288 -76.763767,39.324062") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2426000, "cityName": "Ellicott City" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635254672175104, "text": "going to bed in a bad mood is not the biiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2205872876, "name": "Cami Cota", "screen_name": "camcota", "lang": "en", "location": "Eugene, OR.", "create_at": date("2013-11-20"), "description": "null", "followers_count": 1427, "friends_count": 484, "statues_count": 1638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635254814941184, "text": "Wind 0.7 mph N. Barometer 29.98 in, Falling. Temperature 39.0 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635255318093824, "text": "We're #hiring! Read about our latest #job opening here: Inside Sales Representative -... - https://t.co/BCrFhtmVWL https://t.co/aL7R9ulCVL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.7195451,28.2716755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job" }}, "user": { "id": 835931426, "name": "DialAmerica Jobs", "screen_name": "DialAmericaJobs", "lang": "en", "location": "null", "create_at": date("2012-09-20"), "description": "Come for a job, stay for a career. Follow us for info on career opportunities at DialAmerica. Apply today and find out why we’re the best job in town!", "followers_count": 323, "friends_count": 215, "statues_count": 2020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Richey, FL", "id": "66f5a20eae65a37d", "name": "Port Richey", "place_type": "city", "bounding_box": rectangle("-82.744399,28.263172 -82.707212,28.287695") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1258600, "cityName": "Port Richey" } }
+{ "create_at": datetime("2016-03-15T00:00:03.000Z"), "id": 709635255582351360, "text": "I AM SO HAPPY FOR LAUREN!!!! #TheBachelor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheBachelor" }}, "user": { "id": 2876012680, "name": "alyssa", "screen_name": "Alyssacarsonn", "lang": "en", "location": "gymcats", "create_at": date("2014-11-13"), "description": "insta////alyssacarsonn", "followers_count": 70, "friends_count": 165, "statues_count": 1554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635256031141888, "text": "350+ miles away, and you still find a way to get mad at me and make me feel like shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700263823, "name": "Ziddhartha Juatama", "screen_name": "jzaragoza320", "lang": "en", "location": "Brownsville, Texas", "create_at": date("2012-07-16"), "description": "Never have I dealt with anything more difficult than my own soul.", "followers_count": 199, "friends_count": 199, "statues_count": 1175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635256295550978, "text": "https://t.co/msd33j0vlE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 482551394, "name": "Uncle Drew", "screen_name": "andrewandderson", "lang": "en", "location": "null", "create_at": date("2012-02-03"), "description": "CSI/NEXTROUND/FAMILY FIRST", "followers_count": 1435, "friends_count": 796, "statues_count": 56497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635256496721920, "text": "\"You have a drug problem\" \nAt Least I'm not cheating on my boyfriend who wants to get married, with a married man who has children my age.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 432161192, "name": "unnie", "screen_name": "whosautumn", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-08"), "description": "가을. Grandmas hate me [gavin screaming]", "followers_count": 470, "friends_count": 159, "statues_count": 10859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257075535873, "text": "Do you think I'll ever need your love more than you need me?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1332656977, "name": "Joss", "screen_name": "_josselinnn", "lang": "en", "location": "93927 ", "create_at": date("2013-04-06"), "description": "sc: josselinnng", "followers_count": 622, "friends_count": 509, "statues_count": 10887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, CA", "id": "6684d4e60ddfc668", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-121.260686,36.307617 -121.226796,36.339522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 630994, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257310515200, "text": "Night like this I know I can't keep doing what I am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4354918516, "name": "Dip", "screen_name": "VonOutThe4", "lang": "en", "location": "Charlotte, NC", "create_at": date("2015-12-02"), "description": "Student Artist #WSSU #NOMERCY", "followers_count": 160, "friends_count": 134, "statues_count": 2580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257553723392, "text": "Got the #pullupgang baseball jersey just came in...s/o my designer… https://t.co/aRUYg5bfRa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.48003921,29.73257379"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pullupgang" }}, "user": { "id": 28158640, "name": "IG:MRWIREDUPOHBOY", "screen_name": "MrWiredUpOhBoy", "lang": "en", "location": "Your Favorite Strip Club", "create_at": date("2009-04-01"), "description": "#TRPGANG #GODJS #FLEETDJS #SHAKELIFESHAWTY\nBOOKING/FEATURES BOOKMRWIREDUP@GMAIL.COM #DOUBLECUPSANDSWISHAS OUT NOW!!!", "followers_count": 6646, "friends_count": 6453, "statues_count": 101347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257721421826, "text": "The gavel for Oregon FFA state convention drops in 4 days! Can't believe it's here, who's excited?!��#illuminateORFFA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "illuminateORFFA" }}, "user": { "id": 768109951, "name": "Addie Howell", "screen_name": "muddyprincess24", "lang": "en", "location": "null", "create_at": date("2012-08-19"), "description": "Unless someone like you cares a whole awful lot, nothings going to get better. It's not. Blessed to serve as the 2015-16 Oregon State Vice President", "followers_count": 461, "friends_count": 373, "statues_count": 2624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257734144000, "text": "�� https://t.co/389IfAqyfb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 414320313, "name": "Кαуℓуи∂ Мι¢нαєℓ•", "screen_name": "Kaybaybay096", "lang": "en", "location": "null", "create_at": date("2011-11-16"), "description": "Does she realize she looks like a sunflower, ready to rain sunlight on all who look down upon her• •1Corιnтнιanѕ 16:13-14•", "followers_count": 663, "friends_count": 484, "statues_count": 26227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview Heights, IL", "id": "4fe9b33eae7d1221", "name": "Fairview Heights", "place_type": "city", "bounding_box": rectangle("-90.063103,38.564588 -89.961348,38.625848") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1725141, "cityName": "Fairview Heights" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635257968922625, "text": "@jimmyfallon is always like 10 times more excited about his games than his guest is", "in_reply_to_status": -1, "in_reply_to_user": 15485441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15485441 }}, "user": { "id": 1586573821, "name": "Gary", "screen_name": "_garrebear", "lang": "en", "location": "null", "create_at": date("2013-07-11"), "description": "omnivore", "followers_count": 237, "friends_count": 218, "statues_count": 2356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego Country Estates, CA", "id": "5b151cafa875f47c", "name": "San Diego Country Estates", "place_type": "city", "bounding_box": rectangle("-116.843644,32.991819 -116.72703,33.035831") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666004, "cityName": "San Diego Country Estates" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635258593861633, "text": "@Jal_236 @yoalbie Tfti", "in_reply_to_status": 709633997156954112, "in_reply_to_user": 720846060, "favorite_count": 0, "retweet_count": 0, "lang": "is", "is_retweet": false, "user_mentions": {{ 720846060, 582465560 }}, "user": { "id": 2222255790, "name": "Ivan", "screen_name": "i_rivera34", "lang": "en", "location": "Norwalk, CA", "create_at": date("2013-11-29"), "description": "null", "followers_count": 273, "friends_count": 272, "statues_count": 2815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635258660990976, "text": "Heroes and Villains was the most lit album of my childhood \n\n(In case anyones wondering that's the soundtrack to the powerpuff girls movie)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2267833177, "name": "créme fresh", "screen_name": "meezy_g", "lang": "en", "location": "Reseda, Los Angeles", "create_at": date("2013-12-29"), "description": "back by unpopular demand", "followers_count": 206, "friends_count": 128, "statues_count": 14984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635258715508736, "text": "@sandraDodd7 @marcorubio @FirstOrlando @PastorDavidUth", "in_reply_to_status": 709068821131350022, "in_reply_to_user": 3344459109, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3344459109, 15745368, 16082094, 2834383559 }}, "user": { "id": 235086306, "name": "Gail", "screen_name": "1classysinger", "lang": "en", "location": "California, USA", "create_at": date("2011-01-07"), "description": "Mom, Grandmom, classically trained singer, love Jesus, conservative, patriot, Mayflower descendant.", "followers_count": 65, "friends_count": 79, "statues_count": 2048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635258984099840, "text": "@ItsAshlynBrooke @hayleyyjordyn I'm crying bc it's my fault ��", "in_reply_to_status": 709635139886784512, "in_reply_to_user": 593142832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 593142832, 1535448997 }}, "user": { "id": 2811422131, "name": "madi k", "screen_name": "kimbro_2", "lang": "en", "location": "Daphne, AL", "create_at": date("2014-09-15"), "description": "sc: madisonkimbro", "followers_count": 986, "friends_count": 871, "statues_count": 2361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daphne, AL", "id": "d4aa1d0bb51bccfd", "name": "Daphne", "place_type": "city", "bounding_box": rectangle("-87.92086,30.575695 -87.765512,30.684177") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 119648, "cityName": "Daphne" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635259059429377, "text": "In the mood to eat at zippys ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3180909721, "name": "Asia Canon", "screen_name": "AsiaCanon", "lang": "en", "location": "null", "create_at": date("2015-04-30"), "description": "@rusty_crowder ❤️ NS | HI", "followers_count": 173, "friends_count": 157, "statues_count": 2090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Pointe, HI", "id": "013abac55b5e444f", "name": "Ocean Pointe", "place_type": "city", "bounding_box": rectangle("-158.044905,21.303603 -158.013973,21.328864") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1556685, "cityName": "Ocean Pointe" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635259097174016, "text": "It's midnight and I found out I have a midterm tomorrow :')", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58352207, "name": "Janelle Jankowski", "screen_name": "janellejankkk", "lang": "en", "location": "null", "create_at": date("2009-07-19"), "description": "SJSU. Delta Gamma. corgis ⚓️", "followers_count": 667, "friends_count": 906, "statues_count": 14171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-15T00:00:04.000Z"), "id": 709635259327840256, "text": "Wind 2.0 mph WNW. Barometer 29.789 in, Rising. Temperature 52.2 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 385, "friends_count": 292, "statues_count": 9009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-15T00:00:05.000Z"), "id": 709635259931844608, "text": "@bravoproducer what is @twschwa wearing?!?! @MusicKillsKate I hope this didn't make the style blog.", "in_reply_to_status": 709455847059886084, "in_reply_to_user": 117506202, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 117506202, 260822065, 185409011 }}, "user": { "id": 149967651, "name": "clee", "screen_name": "cmclee", "lang": "en", "location": "Merica!", "create_at": date("2010-05-30"), "description": "Chico State Grad, Dobby Look-A-Like, Cheese Addict, Country Music Fan, World Traveler, Lady. Figuring Out Where I'm Goin'. I'm Young, and the World's Wide Open!", "followers_count": 117, "friends_count": 472, "statues_count": 1429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-15T00:00:05.000Z"), "id": 709635259969642497, "text": "Like Hannah Montana once said, life's a climb but the view is great", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2275250941, "name": "vaeda ramirez", "screen_name": "litttlevae_", "lang": "en", "location": "null", "create_at": date("2014-01-03"), "description": "just a little girl with a big personality((:", "followers_count": 594, "friends_count": 469, "statues_count": 12137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-15T00:00:05.000Z"), "id": 709635260162543616, "text": "I hope Carson is in charge of agricultural - don't they determine how grain is stored? https://t.co/xejizOAURx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74667358, "name": "Tim Berfield", "screen_name": "taberfield", "lang": "en", "location": "Parts Unknown", "create_at": date("2009-09-15"), "description": "Here's the thing about #evolution... It happened. #Secular #HumanRights #Logic #CTO #Programmer #Entrepreneur #BodyBuilding #fitfam", "followers_count": 1766, "friends_count": 2044, "statues_count": 3319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997626947719168, "text": "soul sick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291926319, "name": "based visionary :+)", "screen_name": "yungmalpal_", "lang": "en", "location": "s e a t t l e w a ", "create_at": date("2011-05-02"), "description": "tybg\n basedgod velli \n positivity", "followers_count": 326, "friends_count": 294, "statues_count": 7254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997627312627712, "text": "Sen gizli bir homosun lütfen bizden saklama", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 2361268402, "name": "KRALİÇE ELVİRA", "screen_name": "kraliceelvira", "lang": "tr", "location": "Antalya, Türkiye", "create_at": date("2014-02-23"), "description": "Sistemi ben kölem yapıp zorlayacağım , çalışmaya.", "followers_count": 547, "friends_count": 179, "statues_count": 27308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997627396509697, "text": "Nope https://t.co/5NRfXfmRg5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23799737, "name": "♍TakiChan", "screen_name": "TakiOkamiChan", "lang": "en", "location": "Tallahassee", "create_at": date("2009-03-11"), "description": "♍️girl. ❤️anime, music, and video games. I miss you grandma❤ (1/7/34 - 4/18/11)️! #GoNoles #NoleNation", "followers_count": 533, "friends_count": 560, "statues_count": 46367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997627497189376, "text": "If I wrote you a lover letter, would you write back ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289013877, "name": "Dennis", "screen_name": "D91Barca", "lang": "en", "location": "West Adams. ", "create_at": date("2011-04-27"), "description": "snapchat :d91barca", "followers_count": 229, "friends_count": 193, "statues_count": 16412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997627987853312, "text": "Happy birthday Kris", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487119794, "name": "Meta-Grandpa", "screen_name": "KrisPNW", "lang": "en", "location": "Tacoma", "create_at": date("2012-02-08"), "description": "Photographer//Biomedicine Major//Music Maker", "followers_count": 190, "friends_count": 141, "statues_count": 13569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997628080189440, "text": "My snapchst is lit add me u won't be disappointed https://t.co/huCJuHHqX8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74330653, "name": "omar", "screen_name": "omizzlee", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-09-14"), "description": "R.I.P Hakeem Samara 1/3/2014", "followers_count": 903, "friends_count": 710, "statues_count": 47597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Panama City Beach, FL", "id": "9ebd5acfac2301ba", "name": "Panama City Beach", "place_type": "city", "bounding_box": rectangle("-85.95802,30.165061 -85.786077,30.266595") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1254725, "cityName": "Panama City Beach" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997628533116928, "text": "Just read about Anonymous. What a group. The good guys. #OpTrump\nhttps://t.co/RUjUJYA6N8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OpTrump" }}, "user": { "id": 14711728, "name": "John Brosnan", "screen_name": "birdflyoverlake", "lang": "en", "location": "Eugene", "create_at": date("2008-05-09"), "description": "Liberal from Minn now settled in Oregon. Father of three girls; two lost to parental alienation, which I write about on my blog. Deadhead #VoteBlueNoMatterWho", "followers_count": 3149, "friends_count": 3151, "statues_count": 27779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997629699170304, "text": "big things are coming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 303569778, "name": "1999 derek jeter", "screen_name": "trillvnthropy", "lang": "en", "location": "626", "create_at": date("2011-05-22"), "description": "i watch the sun rise then i watch the sun fall, studied the son of god and still don't recognize my flaws @Arsenal @nyjets @nyknicks @Yankees @USC_Athletics", "followers_count": 1286, "friends_count": 751, "statues_count": 58754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997629745336320, "text": "@daniellepizzaa my life is just a series of unfortunate yet entertaining events", "in_reply_to_status": 709992884456591361, "in_reply_to_user": 383991767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 383991767 }}, "user": { "id": 272564821, "name": "justina", "screen_name": "JustinaBonita_", "lang": "en", "location": "California, USA", "create_at": date("2011-03-26"), "description": "hydrate your hustle/keep an open mind/improve your perspective/stay empowered", "followers_count": 1177, "friends_count": 361, "statues_count": 18662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997629955031040, "text": "Fiorina murdered Hilary with straight up facts ahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2892568364, "name": "jes.", "screen_name": "Jesslvx_", "lang": "en", "location": "csula", "create_at": date("2014-11-07"), "description": "call it how I see it.", "followers_count": 260, "friends_count": 154, "statues_count": 8574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997629971808256, "text": "@KeithDW_ swear", "in_reply_to_status": 709996199953211393, "in_reply_to_user": 156539952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156539952 }}, "user": { "id": 291904893, "name": "Ameerah ❤️", "screen_name": "Ameerah_XOXO", "lang": "en", "location": "detroit ✈️ new york city", "create_at": date("2011-05-02"), "description": "Queen|Wanderlust|Introvert|Cass Tech Alumna|Michigan State University '19 #BlackLivesMatter", "followers_count": 1926, "friends_count": 896, "statues_count": 123522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997630328328192, "text": "Everyone act like they look so good in the airport im just clearly thinking about how my shorts are too big", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4768689595, "name": "gretchλn", "screen_name": "notsowild_cat", "lang": "en", "location": "the sump, arizona", "create_at": date("2016-01-16"), "description": "both sides of me are evenly odd\n http://meaninglesstattoos.wordpress.com @dailywildcat reporter and editor noob", "followers_count": 110, "friends_count": 201, "statues_count": 2124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997630475145217, "text": "Paranoid and Dizzy doesn't sound like either of those things ☹️ https://t.co/Vlvzew9mjB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 898161108, "name": "Wolf Crown", "screen_name": "DoMeAbdiel", "lang": "en", "location": "Ace Hotel", "create_at": date("2012-10-22"), "description": "Leo", "followers_count": 492, "friends_count": 177, "statues_count": 45283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lehigh Acres, FL", "id": "95fea8413e475d95", "name": "Lehigh Acres", "place_type": "city", "bounding_box": rectangle("-81.759179,26.513337 -81.563869,26.686278") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1239925, "cityName": "Lehigh Acres" } }
+{ "create_at": datetime("2016-03-16T00:00:00.000Z"), "id": 709997630684790784, "text": "@susannaesqui https://t.co/1mlANwlsGO", "in_reply_to_status": 709997061140303873, "in_reply_to_user": 61001029, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 61001029 }}, "user": { "id": 1363083985, "name": "Stephen Jacobs", "screen_name": "stephenjacobs27", "lang": "en", "location": "Murray State University", "create_at": date("2013-04-18"), "description": "GoodVibeTribe", "followers_count": 843, "friends_count": 755, "statues_count": 19443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murray, KY", "id": "ca0d320dd40f586b", "name": "Murray", "place_type": "city", "bounding_box": rectangle("-88.361326,36.572627 -88.287973,36.652072") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21035, "countyName": "Calloway", "cityID": 2154642, "cityName": "Murray" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997631397822464, "text": "At the end of the day, no one honestly cares��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123443133, "name": "Nita.", "screen_name": "NitaaCarter", "lang": "en", "location": "Change Locations", "create_at": date("2010-03-15"), "description": "|SOU '19| ~Sacramento Raised Me~", "followers_count": 242, "friends_count": 358, "statues_count": 2944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997631490154496, "text": "25 NA BA https://t.co/DDCSj6J3sg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1731698340, "name": "ㅤ", "screen_name": "yvanadelrosario", "lang": "en", "location": "Houston, TX", "create_at": date("2013-09-05"), "description": "❃ snapchat - yvanadelrosario || swift • goulding • sheeran • lovato • maichard ❃ @yvanaxmaichard", "followers_count": 2057, "friends_count": 467, "statues_count": 75856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997632073158656, "text": "justin and these gifs are killing me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 280647294, "name": "damn cam ✌️", "screen_name": "ayooocam", "lang": "en", "location": "Athens, OH", "create_at": date("2011-04-11"), "description": "Everyone calls me Cam, but you can call me tonight ;) // Combo Guard @ Ohio University Lancaster!! LOADING..... #30", "followers_count": 14156, "friends_count": 11180, "statues_count": 105769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pickerington, OH", "id": "3995cc1483801d24", "name": "Pickerington", "place_type": "city", "bounding_box": rectangle("-82.797752,39.841431 -82.684335,39.939034") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3962498, "cityName": "Pickerington" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997632366710784, "text": "���� https://t.co/Hu5NzFBmKq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1835582624, "name": "JayKen ++®", "screen_name": "JayTweetsWavy", "lang": "en", "location": "810•VEGAS ", "create_at": date("2013-09-08"), "description": "Graphic Designer • 2020®Clothing • #200kClub Contact: 6ickdesigns2020@gmail.com ", "followers_count": 3615, "friends_count": 121, "statues_count": 81155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997632542920704, "text": "EHELP thAtt s s reaal y weard feeling i ?? eeeverything was black and iiidknw knwow whasgts happened OK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4206912852, "name": "ki th goblin", "screen_name": "bakugoui", "lang": "en", "location": "null", "create_at": date("2015-11-11"), "description": "im ki and i fucking hate karamatsu", "followers_count": 54, "friends_count": 103, "statues_count": 3009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Collins, CO", "id": "b2e4e65d7b80d2c1", "name": "Fort Collins", "place_type": "city", "bounding_box": rectangle("-105.148074,40.47168 -104.979811,40.656701") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 827425, "cityName": "Fort Collins" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997633155239937, "text": "They Murded this !! https://t.co/5dMgjualK8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3235524626, "name": "Mia♡", "screen_name": "mfreinaa", "lang": "en", "location": "null", "create_at": date("2015-06-03"), "description": "CH$", "followers_count": 281, "friends_count": 240, "statues_count": 5681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997633180459009, "text": "I didn't even know you could do that ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 753464395, "name": "don", "screen_name": "_bootyislyfe", "lang": "en", "location": "htx", "create_at": date("2012-08-12"), "description": "Houston", "followers_count": 343, "friends_count": 350, "statues_count": 14281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997633234935808, "text": "#SupportOriginMelissa 70.3°F Wind:2.9mph Pressure: 29.62hpa Falling slowly Rain Today 0.00in. Forecast: Precipitation at times, very unsett", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SupportOriginMelissa" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 79, "friends_count": 17, "statues_count": 312524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997633629245441, "text": "@VANS_66 rocking vans since I was a baby! Never disappointed. Happy Birthday I'll be supporting y'all as long as I live #BringAnother50 ������", "in_reply_to_status": -1, "in_reply_to_user": 15383636, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BringAnother50" }}, "user_mentions": {{ 15383636 }}, "user": { "id": 2838811681, "name": "bmertz", "screen_name": "demii4lifee", "lang": "en", "location": "null", "create_at": date("2014-10-02"), "description": "my heart belongs to Lily Loveless", "followers_count": 487, "friends_count": 190, "statues_count": 34177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997633734111232, "text": "03/16@03:00 - Temp 43.6F, WC 43.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.852in, Falling slowly. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634090618880, "text": "Fav for A Dm ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2955468253, "name": "lex☮", "screen_name": "rainbow_lex_", "lang": "en", "location": "Weslaco, TX", "create_at": date("2015-01-01"), "description": "17|| wehs", "followers_count": 752, "friends_count": 1099, "statues_count": 13811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mercedes, TX", "id": "c9af03f7af638bed", "name": "Mercedes", "place_type": "city", "bounding_box": rectangle("-97.958308,26.123747 -97.863735,26.17475") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4847700, "cityName": "Mercedes" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634195492865, "text": "@ebbtideapp Tide in Laie Bay, Hawaii 03/16/2016\nHigh 10:24pm 1.9\n Low 3:01pm 0.2\nHigh 11:25pm 2.0\n Low 6:38am 0.6\nHigh 10:30am 1.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-157.9333,21.65"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 41, "friends_count": 1, "statues_count": 16213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laie, HI", "id": "e9cba3f5c03c8c20", "name": "Laie", "place_type": "city", "bounding_box": rectangle("-157.938319,21.625099 -157.912018,21.661968") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1543250, "cityName": "Laie" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634329710592, "text": "Max has so much built up test", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 436238408, "name": "Kevin", "screen_name": "poptartkevin11", "lang": "en", "location": "null", "create_at": date("2011-12-13"), "description": "mevin in the streets r.i.p honda accord", "followers_count": 542, "friends_count": 347, "statues_count": 15672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634359042048, "text": "@gentry_kline30 but then we wouldn't have each other...such a hard thing to hate a place and love it too.", "in_reply_to_status": 709969058951798784, "in_reply_to_user": 836820816, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 836820816 }}, "user": { "id": 1925465251, "name": "Kayla Butler", "screen_name": "beginswithK", "lang": "en", "location": "Evansville.", "create_at": date("2013-10-01"), "description": "Waffle House Expert", "followers_count": 299, "friends_count": 145, "statues_count": 4391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634640060416, "text": "Wind 5.0 mph WSW. Barometer 29.571 in, Rising slowly. Temperature 73.3 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-16T00:00:01.000Z"), "id": 709997634791018498, "text": "@vxxivn what period do you have her?", "in_reply_to_status": 709997558999998465, "in_reply_to_user": 3606356900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3606356900 }}, "user": { "id": 1135992751, "name": "Adumb", "screen_name": "itsadum", "lang": "en", "location": "CA", "create_at": date("2013-01-30"), "description": "OHS", "followers_count": 516, "friends_count": 478, "statues_count": 12262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997635286011905, "text": "Wind 0.0 mph SW. Barometer 29.908 in, Rising. Temperature 67.9 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997636040966144, "text": "Wind 2.0 mph WNW. Barometer 29.695 in, Rising. Temperature 53.4 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997636179369984, "text": "Temp: 68.8°F Wind:0.9mph Pressure: 29.923hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997636275802112, "text": "\"I feel bad for the people around me because I totally started barking during the concert\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2314495812, "name": "ábbíé", "screen_name": "abbierothschild", "lang": "en", "location": "null", "create_at": date("2014-01-27"), "description": "zlessed", "followers_count": 289, "friends_count": 396, "statues_count": 7432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Heights, CA", "id": "a89175c4c91f45a3", "name": "Citrus Heights", "place_type": "city", "bounding_box": rectangle("-121.331355,38.663673 -121.242804,38.722779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 613588, "cityName": "Citrus Heights" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997636779184128, "text": "@lilmeder happy birthday Lil!!����", "in_reply_to_status": -1, "in_reply_to_user": 2816645645, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2816645645 }}, "user": { "id": 2596496213, "name": "Maz", "screen_name": "DylaMasanotti", "lang": "en", "location": "300hunna bang ", "create_at": date("2014-06-09"), "description": "GHS", "followers_count": 1067, "friends_count": 669, "statues_count": 33789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997636833689600, "text": "girl fucking boo, gtfoh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1012235941, "name": "whip", "screen_name": "Miracleeeee__", "lang": "en", "location": "null", "create_at": date("2012-12-14"), "description": "null", "followers_count": 1049, "friends_count": 933, "statues_count": 30722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del City, OK", "id": "7e831b14a06a5759", "name": "Del City", "place_type": "city", "bounding_box": rectangle("-97.459517,35.420738 -97.423502,35.478707") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4019900, "cityName": "Del City" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637009854466, "text": "Selfish wit ma woman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270716871, "name": "The Wavy Crocket", "screen_name": "gang_greenE", "lang": "en", "location": "null", "create_at": date("2011-03-22"), "description": "Love my girl to death, on occasion still fuck a bitch", "followers_count": 2211, "friends_count": 990, "statues_count": 92846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637139865601, "text": "@dominiquemead33 lmaoooo damn why is dis so close I'm sad AF", "in_reply_to_status": 709990708636225538, "in_reply_to_user": 1584805532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1584805532 }}, "user": { "id": 1584805532, "name": "dom", "screen_name": "dominiquemead33", "lang": "en", "location": "null", "create_at": date("2013-07-10"), "description": "null", "followers_count": 836, "friends_count": 352, "statues_count": 43439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637177581568, "text": "Wasn't trying to hit the gym today, but im just too used to going before work now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191639158, "name": "✭ Roney ✭", "screen_name": "RONiTORiVERO", "lang": "en", "location": "Mount Ωlympus ", "create_at": date("2010-09-16"), "description": "Tishman Construction Electrical Engineer ✳️ #SaiyanArmy #AriesNation Si Vis Pacem Para Bellum // IG: r_rivero", "followers_count": 444, "friends_count": 249, "statues_count": 41672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dumont, NJ", "id": "b45ae2b5def8048f", "name": "Dumont", "place_type": "city", "bounding_box": rectangle("-74.010799,40.9354 -73.975579,40.955124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3418400, "cityName": "Dumont" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637274046464, "text": "@LIZiverson15 @Nooner56 Liz stfu.", "in_reply_to_status": 709964036910272513, "in_reply_to_user": 41493414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41493414, 433802852 }}, "user": { "id": 168543471, "name": "J. Harris", "screen_name": "HARR_I_IS", "lang": "en", "location": "Satisfaction since 1822 ", "create_at": date("2010-07-19"), "description": "We catch and release in fried grease", "followers_count": 613, "friends_count": 570, "statues_count": 19163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637378949120, "text": "67.3F (Feels: 67.3F) - Humidity: 99% - Wind: 2.2mph SW - Gust: 2.2mph - Pressure: 1018.2mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 230885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637408313344, "text": "Star wars porn https://t.co/9wH6OzNWDu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1403234154, "name": "#1 dad", "screen_name": "supream_sam", "lang": "en", "location": "Whittier, CA", "create_at": date("2013-05-04"), "description": "null", "followers_count": 662, "friends_count": 457, "statues_count": 24678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637638950913, "text": "@kinggcabellos", "in_reply_to_status": 709997409632444416, "in_reply_to_user": 2421408218, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2275249164 }}, "user": { "id": 2421408218, "name": "peach", "screen_name": "sadalieee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-03-31"), "description": "you still have your place in the garden", "followers_count": 1274, "friends_count": 688, "statues_count": 63249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997637672509441, "text": "You can be Famous with no Shows ... Not making shit .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239994851, "name": "APimpNameDJTatTat", "screen_name": "DJTatTat", "lang": "en", "location": "3rd Ward | Houston,Tx  ", "create_at": date("2011-01-18"), "description": "Houston Texas @DJTatTat #SpaceCity ..... #Scarface ICON TOUR Bookings: 713.933.5039 • Yo GAL FAVORITE DJ #AYoungLegend #DJTatTat #TravelLifee", "followers_count": 10611, "friends_count": 7334, "statues_count": 121888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, KY", "id": "9c38c3bbd6c69084", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-84.554442,38.964564 -84.479712,39.095317") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2117848, "cityName": "Covington" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997638083551232, "text": "I thought GDC would be learning about the next gen of video games but really its just getting really drunk and making new industry friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 395715944, "name": "Jack", "screen_name": "jackquade", "lang": "en", "location": "Boston, MA", "create_at": date("2011-10-21"), "description": "a nervous wreck & sound designer/musician", "followers_count": 92, "friends_count": 67, "statues_count": 3134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997638213640194, "text": "Females Be Picky And Still Pick The Wrong Nigga ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2535936528, "name": "Cash✨", "screen_name": "JalynnCash", "lang": "en", "location": "SnooWorld,California", "create_at": date("2014-05-30"), "description": "SnapChat: DaddyJalynn", "followers_count": 459, "friends_count": 620, "statues_count": 4155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997638943424512, "text": "@lovebugg_97 get off my shit man", "in_reply_to_status": 709997411964473345, "in_reply_to_user": 99771924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99771924 }}, "user": { "id": 1556407507, "name": "Isaiah Ray", "screen_name": "Kingray_3", "lang": "en", "location": "Spencer, OK Mid Dub", "create_at": date("2013-06-29"), "description": "null", "followers_count": 842, "friends_count": 409, "statues_count": 32656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997638951804929, "text": "@VancityBuzz What's happening in Vancouver this weekend? Looking forward to seeing the city again!", "in_reply_to_status": -1, "in_reply_to_user": 19314850, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19314850 }}, "user": { "id": 16835903, "name": "Christopher P. Moore", "screen_name": "christopherpmo", "lang": "en", "location": "Portland, OR", "create_at": date("2008-10-17"), "description": "Morehouse Alumnus | Nupe | PHA F&AM | Adidas Collector | Martial Artist | Southern Gentleman | West Coast Living", "followers_count": 1031, "friends_count": 1075, "statues_count": 7107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethany, OR", "id": "002b68e9273298f0", "name": "Bethany", "place_type": "city", "bounding_box": rectangle("-122.867608,45.542616 -122.805462,45.568094") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105950, "cityName": "Bethany" } }
+{ "create_at": datetime("2016-03-16T00:00:02.000Z"), "id": 709997638960156672, "text": "I could honestly give you the world not just materialistically but with your heart but you stuck on dumb niggas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263441307, "name": "Also known as Mungia", "screen_name": "DylanMungia", "lang": "en", "location": "null", "create_at": date("2013-03-12"), "description": "#CiscoCollege Kinesiology major I can do all things through Christ who makes it possible IN GOD I TRUST #SingleasaPringle!", "followers_count": 664, "friends_count": 1538, "statues_count": 2290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cisco, TX", "id": "721e5b312b06ad49", "name": "Cisco", "place_type": "city", "bounding_box": rectangle("-99.004915,32.374537 -98.956093,32.401675") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48133, "countyName": "Eastland", "cityID": 4815004, "cityName": "Cisco" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997639501221888, "text": "#MyHeart #MyLove #MyPromise #MyBest #MommieLyfe #ilovemylyfenow #living #milkinghuman #JettDevyn… https://t.co/ODHvXTtUal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.176984,39.937115"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MyHeart", "MyLove", "MyPromise", "MyBest", "MommieLyfe", "ilovemylyfenow", "living", "milkinghuman", "JettDevyn" }}, "user": { "id": 311145162, "name": "tiyethompson", "screen_name": "tiyethompson", "lang": "en", "location": "team.thirtysixth.first", "create_at": date("2011-06-04"), "description": "the elected official loving x living x giving. the OVERCOMER w|a reason. basking in #GODsPromise, jubilantly!#GODFamilyCommunity #ForeverInThatOrder", "followers_count": 415, "friends_count": 1401, "statues_count": 9342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997639664803841, "text": "does the word \"cracker\" have a history of oppression and slavery??? no bc I eat crackers wtf??? https://t.co/R0lbR29gWx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2960799918, "name": "baecey", "screen_name": "trayceecook", "lang": "en", "location": "#GoDawgs", "create_at": date("2015-01-06"), "description": "18 | @aaronsdrops ❤️ | life is plurfect", "followers_count": 1286, "friends_count": 951, "statues_count": 56398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997639744495616, "text": "Looked out my window&I see a whole pack of coyotes roaming around my street. Thnks 4 waking me up your howls sound like the devil laughing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2797047105, "name": "Emma Teager", "screen_name": "TeagerEmma", "lang": "en", "location": "THHS", "create_at": date("2014-09-30"), "description": "artsy♏️", "followers_count": 200, "friends_count": 444, "statues_count": 1663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Santa Margarita, CA", "id": "728359fcdfad2b43", "name": "Rancho Santa Margarita", "place_type": "city", "bounding_box": rectangle("-117.638013,33.585875 -117.553442,33.672953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 659587, "cityName": "Rancho Santa Margarita" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997639895543808, "text": "Wind 0.0 mph N. Barometer 29.87 in, Rising slowly. Temperature 46.6 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640256262145, "text": "Lmao https://t.co/InpAVxSGdg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 312913788, "name": "RADRIC DAVIS", "screen_name": "TEVO_SPRITE", "lang": "en", "location": "CHICAGO - E N G L E W O O D", "create_at": date("2011-06-07"), "description": "BLAME THE CITY IM A PRODUCT OF IT / RIP BERNIE MAC / RIP TALL", "followers_count": 10570, "friends_count": 2553, "statues_count": 285892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640340082689, "text": "Graffiti Tracking: No GPS, Bakersfield", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.11886,35.31595167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55761838, "name": "594 Graffiti", "screen_name": "594Graffiti", "lang": "en", "location": "ÜT: 33.564277,-117.709513", "create_at": date("2009-07-10"), "description": "594 Graffiti- Providers of TAGRS a web-based Graffiti Tracking Solution - TAGRS, Tracking and Automated Graffiti Reporting System, Uses Smartphones to Catch Du", "followers_count": 905, "friends_count": 2, "statues_count": 325829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640566632448, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 309263073, "name": "....", "screen_name": "nigeyyy_", "lang": "en", "location": "null", "create_at": date("2011-06-01"), "description": "Follow my snapchat // nigeyyyy #riptre #treway", "followers_count": 1437, "friends_count": 682, "statues_count": 45880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thibodaux, LA", "id": "1a40ecb19d710091", "name": "Thibodaux", "place_type": "city", "bounding_box": rectangle("-90.881732,29.753203 -90.784444,29.835904") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22057, "countyName": "Lafourche", "cityID": 2275425, "cityName": "Thibodaux" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640575033344, "text": "Who even is u @violentfushcia https://t.co/qBip5OsDj3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2697085451 }}, "user": { "id": 3302333252, "name": "Mason", "screen_name": "Dr_twatson_", "lang": "en", "location": "null", "create_at": date("2015-07-31"), "description": "skate n shit @daddiesboardshop", "followers_count": 46, "friends_count": 105, "statues_count": 400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640889536514, "text": "@_HoesNTattoos oh really now ����", "in_reply_to_status": 709997366280122368, "in_reply_to_user": 469848749, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 469848749 }}, "user": { "id": 239076168, "name": "Princess", "screen_name": "_Yellacocainem", "lang": "en", "location": "Peoria ✈kansas✈texas✈colorado", "create_at": date("2011-01-16"), "description": "Curve queen ✨ Gypsy My lover @kamikazizee #pvamu19 #blinn16 in brenham", "followers_count": 2070, "friends_count": 922, "statues_count": 46246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640939872256, "text": "Makonnen at sx on a Tuesday https://t.co/gdKLQFQ1t0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331183546, "name": "Max", "screen_name": "kosusnik", "lang": "en", "location": "Austin TX", "create_at": date("2011-07-07"), "description": "21. Scorpio. Austin Tx. I once owned a pair of Circa's autographed by OJ Simpson and Les Claypool.", "followers_count": 243, "friends_count": 449, "statues_count": 11387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997640948322305, "text": "don't be a scary a bitch and go private to talk shit lmao )):", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 749343967, "name": "jaz", "screen_name": "hctiblil", "lang": "en", "location": "htx", "create_at": date("2012-08-10"), "description": "null", "followers_count": 3754, "friends_count": 195, "statues_count": 49416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997641002840065, "text": "Oh cool. More grey hair.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362452182, "name": "avocodrowe", "screen_name": "CodyxRowe", "lang": "en", "location": "JXN HC", "create_at": date("2011-08-26"), "description": "veg lyfe. I bleed green. The chlorophyll thrives in my veins. Avocado on the brain.", "followers_count": 266, "friends_count": 562, "statues_count": 7254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MS", "id": "bdaf4d6412ba7c95", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-90.367999,32.301932 -90.274255,32.370893") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2814420, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997641111904256, "text": "should've stayed @ emilys again tonight, I'm all bored @ home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 629943222, "name": "F", "screen_name": "asvpfelic", "lang": "en", "location": "tx ", "create_at": date("2012-07-07"), "description": "her heart was a secret garden & the walls were very high.", "followers_count": 1362, "friends_count": 547, "statues_count": 67584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robstown, TX", "id": "fb66c781b044ddb1", "name": "Robstown", "place_type": "city", "bounding_box": rectangle("-97.685957,27.768882 -97.648646,27.811362") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4862600, "cityName": "Robstown" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997641569013761, "text": "Dreaming of dark chocolate covered raisins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 790870987, "name": "Kaylee", "screen_name": "KayleeMaxwell99", "lang": "en", "location": "OC || SJSU '18 ", "create_at": date("2012-08-29"), "description": "Why have a thigh gap when you can thigh clap", "followers_count": 458, "friends_count": 220, "statues_count": 26898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997641803894784, "text": "Next year we coming deep to this island!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 195589801, "name": "Hook-Island", "screen_name": "Respect_TheHook", "lang": "en", "location": "Richmond Tx", "create_at": date("2010-09-26"), "description": "*G.O.A.T* *Texas Dark Knight SemiPro Football* *Pro Photographer* *Strive 4 Greatness* *Philippians 4-13* *D.B.U* HTX", "followers_count": 904, "friends_count": 558, "statues_count": 34189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Padre Island, TX", "id": "1d1f665883989434", "name": "South Padre Island", "place_type": "city", "bounding_box": rectangle("-97.177252,26.078455 -97.1597,26.151016") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48261, "countyName": "Kenedy", "cityID": 4869140, "cityName": "South Padre Island" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997641866805249, "text": "It's easy to look at ourselves\nand see ourselves as what we\nare now.\nGod looks at us and sees what\nwe can become.\nSarah Ann Neil", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 144, "friends_count": 262, "statues_count": 246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642164609025, "text": "HAPPY BIRTHDAY ������ may you have a great day chalupa batman ���� https://t.co/tw1nxwo7Ai", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.800206,36.653573"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232938445, "name": "swamp thing", "screen_name": "shad0w0wl", "lang": "en", "location": "Fremont, CA", "create_at": date("2011-01-01"), "description": "stay wild, moon child", "followers_count": 501, "friends_count": 368, "statues_count": 9005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tortuga Hall", "id": "07d9ca2398881003", "name": "Tortuga Hall", "place_type": "poi", "bounding_box": rectangle("-121.8002061,36.6535729 -121.800206,36.653573") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642189824001, "text": "I just ruined some of New Girl for Morgan and I feel like a piece of shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64821824, "name": "Cameron DuBay", "screen_name": "Passdadooby", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "Sunglasses emoji.", "followers_count": 397, "friends_count": 275, "statues_count": 36698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642370125826, "text": "\"Everybody\nIs somebody\nAnd anybody is you\nI own my story\nI won't say sorry\nAnd neither should you\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 626906712, "name": "Carl Davis", "screen_name": "Caaaaaaarlllll", "lang": "en", "location": "Kirksville, MO", "create_at": date("2012-07-04"), "description": "Director. Writer. IT Nerd. Gamer. Music enthusiast. Adventurer. Hunter. I am who i am and nothing more.", "followers_count": 87, "friends_count": 251, "statues_count": 2401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirksville, MO", "id": "7e403d6a5b07bc42", "name": "Kirksville", "place_type": "city", "bounding_box": rectangle("-92.601741,40.15184 -92.553156,40.238028") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29001, "countyName": "Adair", "cityID": 2939026, "cityName": "Kirksville" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642504413185, "text": "It's something about you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2546050839, "name": "$avageWorldTour", "screen_name": "MadamBoatswain", "lang": "en", "location": "SomePlace.", "create_at": date("2014-05-13"), "description": "Aggie Pride.", "followers_count": 948, "friends_count": 676, "statues_count": 88273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642571456512, "text": "They took my bottle on 6th. I'm sad.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54504333, "name": "pimp c", "screen_name": "CaylaVuitton", "lang": "en", "location": "null", "create_at": date("2009-07-07"), "description": "y'all are too serious 4 me.", "followers_count": 4331, "friends_count": 998, "statues_count": 3492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642596618240, "text": "hopefully I don't die in the middle of the night, if I do, it was lit bye hoes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2749441073, "name": "desiree nichole.", "screen_name": "sorryifangirl", "lang": "en", "location": "i love hannah nichole.", "create_at": date("2014-08-21"), "description": "not good enough, but almost.", "followers_count": 7239, "friends_count": 470, "statues_count": 54962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Levelland, TX", "id": "f3f6359397117e7d", "name": "Levelland", "place_type": "city", "bounding_box": rectangle("-102.403865,33.561058 -102.310997,33.603758") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48219, "countyName": "Hockley", "cityID": 4842448, "cityName": "Levelland" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997642714107904, "text": "I SEE @justinbieber TOMORROW ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27260086 }}, "user": { "id": 1629468912, "name": "SAMMIE", "screen_name": "iamsamieduh", "lang": "en", "location": "philippians 4:13", "create_at": date("2013-07-28"), "description": "Screw spring break summer where you @ !?!?", "followers_count": 1602, "friends_count": 898, "statues_count": 26356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997643158700032, "text": "acid gives me bad trips no thx https://t.co/sbgETloncn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2388059498, "name": "lil quesadilla", "screen_name": "_aubz", "lang": "en", "location": "utsa satx", "create_at": date("2014-03-13"), "description": "think of all the beauty still left around you and be happy", "followers_count": 1348, "friends_count": 647, "statues_count": 54682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997643162914816, "text": "�� https://t.co/8PHJTGFw6C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2411905627, "name": "VictoriaAlexis †", "screen_name": "_iBeVictoria", "lang": "en", "location": "Sacramento, CA ", "create_at": date("2014-03-25"), "description": "SnapChat @ WeedIsBeautiful • Makeup ✨", "followers_count": 153, "friends_count": 74, "statues_count": 4294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-03-16T00:00:03.000Z"), "id": 709997643360038913, "text": "@YoungHenddog I hate you ��", "in_reply_to_status": 709997512262815744, "in_reply_to_user": 2735791120, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2735791120 }}, "user": { "id": 737466236, "name": "NBA & Chill", "screen_name": "_EmptyCupOfCare", "lang": "en", "location": "null", "create_at": date("2012-08-04"), "description": "God Comes First. Siempre. Clemson Sophomore Miami✈️SC", "followers_count": 376, "friends_count": 396, "statues_count": 17871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemson, SC", "id": "01ff1adb1d047095", "name": "Clemson", "place_type": "city", "bounding_box": rectangle("-82.853949,34.658367 -82.776522,34.729476") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45077, "countyName": "Pickens", "cityID": 4514950, "cityName": "Clemson" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997643724890114, "text": "@DweenieeTodd yes ��", "in_reply_to_status": 709948709568643072, "in_reply_to_user": 392335354, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 392335354 }}, "user": { "id": 278141481, "name": "Allegorithmic", "screen_name": "Allegorithmic", "lang": "fr", "location": "Clermont-Ferrand, France", "create_at": date("2011-04-06"), "description": "Allegorithmic is the developer of Substance, the killer app when it comes to texturing the next wave of 3D.", "followers_count": 7556, "friends_count": 336, "statues_count": 5742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997643850780672, "text": "Can't stop, won't stop @LighterKick https://t.co/UVkQdqOe0U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 302957810 }}, "user": { "id": 701759848, "name": "Calvin Espinoza", "screen_name": "espys0910", "lang": "en", "location": "null", "create_at": date("2013-10-04"), "description": "ATU19 ΣΦΕ", "followers_count": 767, "friends_count": 592, "statues_count": 7091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Russellville, AR", "id": "c09505551daee91a", "name": "Russellville", "place_type": "city", "bounding_box": rectangle("-93.205955,35.220654 -93.056961,35.336506") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5115, "countyName": "Pope", "cityID": 561670, "cityName": "Russellville" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997644014358528, "text": "Never https://t.co/dTBIew79Sv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329424716, "name": "queen", "screen_name": "lacey_feagley", "lang": "en", "location": "the 6ix w y(our) nigga", "create_at": date("2011-07-04"), "description": "stop lurkin | roll tide ❤️ | I do what I want cuz im poppin", "followers_count": 1207, "friends_count": 2247, "statues_count": 51228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Springs, VA", "id": "08b233eda1164a49", "name": "Highland Springs", "place_type": "city", "bounding_box": rectangle("-77.375572,37.521142 -77.270531,37.576879") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5137032, "cityName": "Highland Springs" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997644819668993, "text": "Midnight facetimes w/ you are the v best ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3297070364, "name": "Natt", "screen_name": "MojarroNathalia", "lang": "en", "location": "null", "create_at": date("2015-07-27"), "description": "s.a ❤️", "followers_count": 173, "friends_count": 318, "statues_count": 2597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997645050339328, "text": "Wind 0.0 mph ---. Barometer 29.859 in, Steady. Temperature 37.9 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997645322911744, "text": "@BernieSanders spend some money on advertising...you're not getting the attention you and this country deserve! #Sanders2016", "in_reply_to_status": -1, "in_reply_to_user": 216776631, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sanders2016" }}, "user_mentions": {{ 216776631 }}, "user": { "id": 134257947, "name": "Jack Wong", "screen_name": "PDXJack", "lang": "en", "location": "Pacific northfresh ", "create_at": date("2010-04-17"), "description": "It's the happiness of the pursuit. Do you want to come along for the ride?", "followers_count": 183, "friends_count": 362, "statues_count": 7599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997645339693060, "text": "@EdelCaregiver What #Caregivers Need to Know about Qualifying for #AssistedLiving\n\nhttps://t.co/5bS089q9Vv https://t.co/Bs5liAajZq", "in_reply_to_status": 709904885152665600, "in_reply_to_user": 3418478489, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Caregivers", "AssistedLiving" }}, "user_mentions": {{ 3418478489 }}, "user": { "id": 51185319, "name": "SilverCensus", "screen_name": "SilverCensus", "lang": "en", "location": "Palm Beach Gardens", "create_at": date("2009-06-26"), "description": "http://SilverCensus.com - The Premier Directory for Senior Living & Health Services offering free placement assistance and job recruitment", "followers_count": 884, "friends_count": 543, "statues_count": 15296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Palm Beach, FL", "id": "4de072969805ac41", "name": "West Palm Beach", "place_type": "city", "bounding_box": rectangle("-80.2006,26.644671 -80.044639,26.799121") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1276600, "cityName": "West Palm Beach" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997645788487685, "text": "I can't sleepp��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2865481992, "name": "jenah♕", "screen_name": "jeennaahhhhh", "lang": "en", "location": "null", "create_at": date("2014-10-19"), "description": "null", "followers_count": 166, "friends_count": 251, "statues_count": 712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hughson, CA", "id": "62548cdeae2a4b43", "name": "Hughson", "place_type": "city", "bounding_box": rectangle("-120.879351,37.589058 -120.851611,37.609315") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 634904, "cityName": "Hughson" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997645968896000, "text": "If you made an actual change in your life, you'd be blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 906978859, "name": "Gordita. ❣", "screen_name": "Manderzz123", "lang": "en", "location": "Alice, TX", "create_at": date("2012-10-26"), "description": "chin up buttercup.", "followers_count": 390, "friends_count": 213, "statues_count": 7433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alice, TX", "id": "8de14ceebb414fe3", "name": "Alice", "place_type": "city", "bounding_box": rectangle("-98.109116,27.713198 -98.036162,27.784932") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48249, "countyName": "Jim Wells", "cityID": 4801852, "cityName": "Alice" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997646384144384, "text": "27 days. Do what's necessary. Push what's possible. Watch what people think is impossible. \n\nFuture - March Madness https://t.co/tj0Fesutoc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4795815447, "name": "Will Kampfman", "screen_name": "will_kampfman", "lang": "en", "location": "#CharmCity", "create_at": date("2016-01-13"), "description": "Field Director @mosbyformayor", "followers_count": 30, "friends_count": 24, "statues_count": 75 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997647210401792, "text": "PERSIAN DELIGHT https://t.co/kL4QyCBO6t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 937720771, "name": "Peggy VanDeWeghe♈", "screen_name": "PeggyVDW", "lang": "en", "location": " LA via NOLA by way of NYC", "create_at": date("2012-11-09"), "description": "Structural Engr Designer #LSUAlum Cajun Foodie Fashionista Avid Sports Fan Reece's Mom ❤Kiki's Wife also follow @SecondChancesDN", "followers_count": 2247, "friends_count": 1877, "statues_count": 33220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997647285886977, "text": "Ripley SW Limestone Co. Temp: 71.8°F Wind:3.8mph Pressure: 988.7mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 47, "friends_count": 32, "statues_count": 50277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-16T00:00:04.000Z"), "id": 709997647554351104, "text": "Oh yea && I can't see them emojis ����me trying to see them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3995071107, "name": "Birthday Bussy♓", "screen_name": "Im_bull_forreal", "lang": "en", "location": "Northside of Philly ", "create_at": date("2015-10-19"), "description": "Need I say more, when my name && @ states the obvious✌ Oh && another thing don't follow me if you not gone rt my shit!☺", "followers_count": 435, "friends_count": 698, "statues_count": 43725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997648263172097, "text": "Pls do it so I can buy you giant sushi rolls https://t.co/ynXdG8EfI3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1170759290, "name": "Charlotte Guerrero", "screen_name": "cocogee95", "lang": "en", "location": "null", "create_at": date("2013-02-11"), "description": "love my life on the highest of keys #KL❤️", "followers_count": 357, "friends_count": 682, "statues_count": 10142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonita, CA", "id": "dfb4dd1ba29fead1", "name": "Bonita", "place_type": "city", "bounding_box": rectangle("-117.065683,32.638243 -116.991198,32.691595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 607414, "cityName": "Bonita" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997648602927108, "text": "The media is portraying Trump as a horrible racist guy when he's not. He's more presidential now and he will make america great again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 793032193, "name": "alexis", "screen_name": "PokemonAlexx", "lang": "en", "location": "US: 37.766697,-122.208225", "create_at": date("2012-08-30"), "description": "null", "followers_count": 334, "friends_count": 258, "statues_count": 41558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997649651490817, "text": "not even a lil https://t.co/ODWecnPxWc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219085869, "name": "j.coles gf", "screen_name": "sydalx", "lang": "en", "location": "boolin w/ 54", "create_at": date("2010-11-23"), "description": "too black for the white kids, & too white for tha blacks", "followers_count": 512, "friends_count": 501, "statues_count": 14177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650066677760, "text": "I don't know how to handle this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1273746036, "name": "Izzy Bates☁️", "screen_name": "Izzyb96Isabell", "lang": "en", "location": "null", "create_at": date("2013-03-16"), "description": "Psalm 91:4 He will cover you with his feathers, and under his wing you will find refuge;", "followers_count": 200, "friends_count": 688, "statues_count": 9163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650079322112, "text": "WHY YOU ACTING CRAZY !? I SWEARRR THIS NIGGA TRIPPIN OFF SUM !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3586190892, "name": "Izzzzzy", "screen_name": "__sheabaddiee", "lang": "en", "location": "OUT N' ABOUT ", "create_at": date("2015-09-16"), "description": "All I Care Bout Is Ma Bitchesss", "followers_count": 444, "friends_count": 164, "statues_count": 3848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Coast, FL", "id": "6613abf67d2b0d39", "name": "Palm Coast", "place_type": "city", "bounding_box": rectangle("-81.279617,29.424649 -81.158741,29.627243") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12035, "countyName": "Flagler", "cityID": 1254200, "cityName": "Palm Coast" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650209345537, "text": "Booty pics from seriaha ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3252971467, "name": "❣bria❣", "screen_name": "im_just_bria", "lang": "en", "location": "CLOUD 19", "create_at": date("2015-06-22"), "description": "Cali Always. TrvpSoul bc Bryson . WAVY bc Kehlani . TGOD . What's a Queen w/ out a Queen. .LittyCommittee.", "followers_count": 285, "friends_count": 260, "statues_count": 7675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650335105024, "text": "come back heyaaa bae", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2597112445, "name": "daniella hernandez ✨", "screen_name": "dahyella_", "lang": "en", "location": "pierce is daddi", "create_at": date("2014-06-30"), "description": "♥️ 02-20-16 ♥️", "followers_count": 282, "friends_count": 245, "statues_count": 4806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650393899013, "text": "@KatieLinendoll that cat looks like it's ready to star some trouble...idk if i wamt trouble", "in_reply_to_status": 709997154694262784, "in_reply_to_user": 15965822, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15965822 }}, "user": { "id": 25177195, "name": "JaredMartinez™", "screen_name": "JaredMartinezVO", "lang": "en", "location": "null", "create_at": date("2009-03-18"), "description": "27, Taken =), 3x Warrior, 12x Spartan, Lover of Hip Hop/Wrestling/Retro Video Games. Up & coming Voice Actor..gotta keep pushin =) SC: JaredAKAMe", "followers_count": 401, "friends_count": 1322, "statues_count": 23693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Mohegan, NY", "id": "4c6a3f7e3092600d", "name": "Lake Mohegan", "place_type": "city", "bounding_box": rectangle("-73.861989,41.294835 -73.832928,41.336163") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3640689, "cityName": "Lake Mohegan" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650708459520, "text": "Wind 4.0 mph SSE. Barometer 29.544 in, Falling Rapidly. Temperature 60.0 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 386, "friends_count": 292, "statues_count": 9034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997650817458177, "text": "sup @fucktyler", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 166747718 }}, "user": { "id": 1559543996, "name": "electric girl", "screen_name": "shay_simpauco", "lang": "en", "location": "null", "create_at": date("2013-06-30"), "description": "in search of sugar daddy", "followers_count": 452, "friends_count": 276, "statues_count": 4742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Proctor, MN", "id": "b5a3b03027147380", "name": "Proctor", "place_type": "city", "bounding_box": rectangle("-92.240693,46.720702 -92.216981,46.764132") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2752630, "cityName": "Proctor" } }
+{ "create_at": datetime("2016-03-16T00:00:05.000Z"), "id": 709997651828285440, "text": ":)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 565896258, "name": "cōll⛈", "screen_name": "collard_greens", "lang": "en", "location": "CA/IL", "create_at": date("2012-04-28"), "description": "this my boyfriend ☞ @from__myspace", "followers_count": 1379, "friends_count": 2247, "statues_count": 49176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Beloit, IL", "id": "005667134a7dbda4", "name": "South Beloit", "place_type": "city", "bounding_box": rectangle("-89.085077,42.458347 -88.992955,42.496354") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1770590, "cityName": "South Beloit" } }
+{ "create_at": datetime("2016-03-16T00:00:06.000Z"), "id": 709997652075790337, "text": "@HoopsOverHoes_ something you could do instead of getting beaten up \nhttps://t.co/a2wEdotd8J", "in_reply_to_status": 709997241969336321, "in_reply_to_user": 388390496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 388390496 }}, "user": { "id": 4848155246, "name": "Michelle Martin", "screen_name": "michell97246087", "lang": "en", "location": "United States", "create_at": date("2016-02-01"), "description": "Fasion just", "followers_count": 32, "friends_count": 66, "statues_count": 379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015190646785, "text": "If you still have your grandparents talk to them and enjoy there presence, don't take them for granted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977520150, "name": "T.L.O.J", "screen_name": "_Jova15", "lang": "en", "location": "null", "create_at": date("2012-11-28"), "description": "I'm on my way to heaven... |CSUSB|", "followers_count": 237, "friends_count": 217, "statues_count": 7499 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015350079488, "text": "@_RichieSteez already ✊��", "in_reply_to_status": 710359892519886853, "in_reply_to_user": 320923219, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320923219 }}, "user": { "id": 341861060, "name": "Ahnté Harkey", "screen_name": "TaeDay20", "lang": "en", "location": "Del City ", "create_at": date("2011-07-24"), "description": "Hillsdale Baptist Basketball Snapchat: TrickDaddyTae #LongLiveAlleyKat @FBMOOklahoma", "followers_count": 1344, "friends_count": 795, "statues_count": 41828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015517790208, "text": "@f_hikmat always will be a beautiful morning with kind ,nice and friendly good peoples such your self.", "in_reply_to_status": 710346706005659648, "in_reply_to_user": 961668908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 961668908 }}, "user": { "id": 2187447985, "name": "KAMAL", "screen_name": "kamalthefunman", "lang": "en", "location": "Marina del Rey, CA USA", "create_at": date("2013-11-10"), "description": "male", "followers_count": 3168, "friends_count": 4982, "statues_count": 45823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marina del Rey, CA", "id": "38ea9782c8d83a4b", "name": "Marina del Rey", "place_type": "city", "bounding_box": rectangle("-118.463481,33.964019 -118.432199,33.98647") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645806, "cityName": "Marina del Rey" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015614431232, "text": "necesito darle follow a gente que estè despierta a esta hora", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 4656014322, "name": "la jodienda", "screen_name": "2jevaaa", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "rg4l", "followers_count": 3587, "friends_count": 800, "statues_count": 17084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lower Allen, PA", "id": "73e9489fbb2465e4", "name": "Lower Allen", "place_type": "city", "bounding_box": rectangle("-76.932746,40.213637 -76.868095,40.238874") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4244824, "cityName": "Lower Allen" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015626850304, "text": "@yungjustinrhode @grave_waves yes!!!", "in_reply_to_status": 710359734470230016, "in_reply_to_user": 198009106, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 198009106, 2410755006 }}, "user": { "id": 2590892510, "name": "rae", "screen_name": "eyehaterae", "lang": "en", "location": "Seattle, WA", "create_at": date("2014-06-26"), "description": "quiet vegan loser", "followers_count": 394, "friends_count": 342, "statues_count": 12914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340391 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360015744471040, "text": "Tryna party", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548132685, "name": "Tyler Marlow", "screen_name": "TysDaName", "lang": "en", "location": "Temecula, CA / Las Vegas, NV", "create_at": date("2012-04-07"), "description": "Marlow is the funniest kid you'll ever meet - Tyler Marlow", "followers_count": 92, "friends_count": 228, "statues_count": 5197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360016142729216, "text": "Thank you @DarkMatterPoets for your performance and support/giving us the bravery to speak in front of our white peers. So much gratitude!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1292418902 }}, "user": { "id": 3477218952, "name": "Estefanía", "screen_name": "queen_nalgona", "lang": "en", "location": "Portland, OR", "create_at": date("2015-09-06"), "description": "You can't spell scholar without chola", "followers_count": 100, "friends_count": 152, "statues_count": 1006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360016176349184, "text": "@PeterBeinart 4 PRESIDENT OBAMA! ALL OVER THE COUNTRY ON SOCIAL MEDIA THERE'S A MOVEMENT STARTED 2 SPREAD THE WORD THAT WE'RE ALL VOTING", "in_reply_to_status": -1, "in_reply_to_user": 114870265, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 114870265 }}, "user": { "id": 328336153, "name": "linda conaway", "screen_name": "lconaway53", "lang": "en", "location": "null", "create_at": date("2011-07-02"), "description": "null", "followers_count": 111, "friends_count": 143, "statues_count": 8062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360016553783296, "text": "@yaya_paz dude she works at MS too i hate her. She complains and tries to get everyone in trouble for EVERRRRRRRYTHING", "in_reply_to_status": 710359774353698816, "in_reply_to_user": 488659777, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 488659777 }}, "user": { "id": 249730235, "name": "Chris", "screen_name": "Gospurs21", "lang": "en", "location": "null", "create_at": date("2011-02-09"), "description": "Probably upset cause of the Cowboys", "followers_count": 653, "friends_count": 383, "statues_count": 57591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360016708960256, "text": "oomf still hasn't hmu smh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1250650309, "name": "LM $$", "screen_name": "QueenBeeLuz", "lang": "en", "location": "713", "create_at": date("2013-03-07"), "description": "K9 @ heart", "followers_count": 469, "friends_count": 334, "statues_count": 20473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360016922873856, "text": "@swavey_cuhh miss you too man!! 4 more days", "in_reply_to_status": 710359875226697731, "in_reply_to_user": 779524884, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 779524884 }}, "user": { "id": 582446360, "name": "Rocky", "screen_name": "PullGod", "lang": "en", "location": "Texas, USA", "create_at": date("2012-05-16"), "description": "Yo Adrian. I have a hot girlfriend", "followers_count": 450, "friends_count": 349, "statues_count": 28533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360017531183104, "text": "Wind 0.0 mph ---. Barometer 29.933 in, Steady. Temperature 60.7 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360017749192704, "text": "@mkskee_ Bhudda Bless", "in_reply_to_status": -1, "in_reply_to_user": 4176964039, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user_mentions": {{ 4176964039 }}, "user": { "id": 1297997263, "name": "Jonasty Vaifala Papi", "screen_name": "_Jooownaaaw", "lang": "en", "location": "null", "create_at": date("2013-03-24"), "description": "Jonah Stonah. My Blood is Hawaiian, but my Stomach is Samoan. [No. I'm not Jonah from Tonga]", "followers_count": 431, "friends_count": 284, "statues_count": 11151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Kunia, HI", "id": "000cd0bd39281e46", "name": "Royal Kunia", "place_type": "city", "bounding_box": rectangle("-158.039748,21.396379 -158.023455,21.409007") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1568815, "cityName": "Royal Kunia" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360017925332992, "text": "Yee https://t.co/0jj49RuPf3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2509633862, "name": "Unkle Pedialyte", "screen_name": "Original_KEL", "lang": "en", "location": "Playlist Central", "create_at": date("2014-05-20"), "description": "• O G + K E L • aka DJ Juntao • M O R E V I B E $ •⚡️", "followers_count": 585, "friends_count": 879, "statues_count": 8884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cerrito, CA", "id": "6407905aa8012e44", "name": "El Cerrito", "place_type": "city", "bounding_box": rectangle("-122.323541,37.89782 -122.280875,37.939284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621810, "cityName": "El Cerrito" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360018495741954, "text": "@cheynanaginss *thanks��", "in_reply_to_status": 710359921003343872, "in_reply_to_user": 605795678, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1514717761 }}, "user": { "id": 605795678, "name": "Anisah", "screen_name": "anisahh_9", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "@joshmares_28", "followers_count": 854, "friends_count": 628, "statues_count": 24705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360018516717568, "text": "Two full geared up cops came into the gym and starting working out. Hard core ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 586932738, "name": "Mandy Bolstad", "screen_name": "MandyBolstad", "lang": "en", "location": "null", "create_at": date("2012-05-21"), "description": "null", "followers_count": 215, "friends_count": 294, "statues_count": 827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest Park, WA", "id": "c3f0bca4262b5ef8", "name": "Lake Forest Park", "place_type": "city", "bounding_box": rectangle("-122.31356,47.732932 -122.263384,47.777413") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5337270, "cityName": "Lake Forest Park" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360018642546690, "text": "3 hours down, 8 remaining ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3490234940, "name": "Em", "screen_name": "emsherick", "lang": "en", "location": "null", "create_at": date("2015-09-07"), "description": "null", "followers_count": 172, "friends_count": 111, "statues_count": 616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minden, LA", "id": "b48cb85462d3e599", "name": "Minden", "place_type": "city", "bounding_box": rectangle("-93.318942,32.558986 -93.248155,32.65746") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22119, "countyName": "Webster", "cityID": 2250885, "cityName": "Minden" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360018671931393, "text": "@rs_nigel stop", "in_reply_to_status": 710359813268447233, "in_reply_to_user": 153135899, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 153135899 }}, "user": { "id": 788431886, "name": "TheFakeDaniel™", "screen_name": "x_Dxnny", "lang": "en", "location": "CA", "create_at": date("2012-08-28"), "description": "________________________________________________________", "followers_count": 290, "friends_count": 180, "statues_count": 27680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-03-17T00:00:00.000Z"), "id": 710360018718105600, "text": "What's so great about love?\n\n Nothing, it's just the people who make you love them effortlessly.\n#wordstoliveby #love", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wordstoliveby", "love" }}, "user": { "id": 37731208, "name": "Paras Sayed", "screen_name": "ParasSayed", "lang": "en", "location": "Houston Texas", "create_at": date("2009-05-04"), "description": "Artist.", "followers_count": 484, "friends_count": 116, "statues_count": 5084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360018944536577, "text": "I kinda forgot what it felt like to have no beard", "in_reply_to_status": 710357034009387010, "in_reply_to_user": 461762277, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 461762277, "name": "Dylan", "screen_name": "dylan_275", "lang": "en", "location": "Austin, TX", "create_at": date("2012-01-11"), "description": "I once served a hot dog to Tom Delay.", "followers_count": 1056, "friends_count": 1049, "statues_count": 146626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360019573669894, "text": "@Dylan_Schaefer8 cal is doomed", "in_reply_to_status": -1, "in_reply_to_user": 231914266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 231914266 }}, "user": { "id": 215461962, "name": "last minute T", "screen_name": "turniee25", "lang": "en", "location": "ballin no NBA", "create_at": date("2010-11-13"), "description": "Somewhere smokin on dope with joey....... FREE DA KID #YoungETrap #NTL #poppy TRAPPATONI", "followers_count": 615, "friends_count": 714, "statues_count": 12820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360019733188608, "text": "Don't you have a test in the morning ?? https://t.co/xVEhoVKcJP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 720136919, "name": "TheWifeOfPablo", "screen_name": "Pink_buggattii", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-17"), "description": "Big ❤️ Forever smiling .... The nicest person you'll ever meet Nomatter what happens in life being good to people is a wonderful legacy to leave behind", "followers_count": 4264, "friends_count": 673, "statues_count": 96850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360019993145344, "text": "@Olxandra1 @deras_abigail @DeleonJoseline omg nooo���� is he in town?����", "in_reply_to_status": 710359855333187585, "in_reply_to_user": 3443648833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3443648833, 4914066001, 700920097196519424 }}, "user": { "id": 731207976, "name": "katalina.", "screen_name": "kgp97", "lang": "en", "location": "Kansas City, MO", "create_at": date("2012-08-01"), "description": "does it even matter?", "followers_count": 204, "friends_count": 336, "statues_count": 3941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020026793985, "text": "I'm so happy we aren't paying $4 a gallon for gas anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2863482650, "name": "Godzilla", "screen_name": "TyloTweets", "lang": "en", "location": "null", "create_at": date("2014-10-18"), "description": "TLOT", "followers_count": 219, "friends_count": 128, "statues_count": 1113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redwood, OR", "id": "c77b5fda9c52afc5", "name": "Redwood", "place_type": "city", "bounding_box": rectangle("-123.420861,42.397624 -123.348505,42.437368") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41033, "countyName": "Josephine", "cityID": 4161250, "cityName": "Redwood" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020156690432, "text": "@rachaellepapaya I heard food", "in_reply_to_status": 710343744097099776, "in_reply_to_user": 411712252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 411712252 }}, "user": { "id": 274321360, "name": "KChoBoom.", "screen_name": "kchoboom", "lang": "en", "location": " In Costco trying free samples", "create_at": date("2011-03-29"), "description": "* walk fast, look busy* Nordy.", "followers_count": 580, "friends_count": 470, "statues_count": 23921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020530110464, "text": "@_emilyglen_ lol what does she want you to do then?", "in_reply_to_status": 710359748168847361, "in_reply_to_user": 1033957861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1033957861 }}, "user": { "id": 381684786, "name": "emmy ⚡️", "screen_name": "emilykayviar", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-09-28"), "description": "stay weird & roll tide | Be yourself, everyone else is a twat | I'm outdoorsy in the sense that I like to get drunk on on patios△⃒⃘", "followers_count": 400, "friends_count": 929, "statues_count": 13262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020739694592, "text": "��#wednesday #style #new #music #single #tearin #up #my #heart #itunes #baltimore #maryland… https://t.co/8FDzvGewt3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.6125,39.2903"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "wednesday", "style", "new", "music", "single", "tearin", "up", "my", "heart", "itunes", "baltimore", "maryland" }}, "user": { "id": 4652964553, "name": "Panda", "screen_name": "josephliam4", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-12-25"), "description": "⇨ Music is the hidden language of the soul", "followers_count": 13, "friends_count": 118, "statues_count": 3 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020836335617, "text": "03/17@03:00 - Temp 45.9F, WC 45.9F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.881in, Rising slowly. Rain 0.00in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 47754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020848762880, "text": "@arroyoerick101 ����", "in_reply_to_status": 710359902749741056, "in_reply_to_user": 2719031496, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2719031496 }}, "user": { "id": 2287135964, "name": "travi$", "screen_name": "BOOTYHUNTER_69", "lang": "en", "location": "at the rodeo", "create_at": date("2014-01-11"), "description": "doing life one blunt at a time", "followers_count": 400, "friends_count": 257, "statues_count": 14138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020987150336, "text": "The Great American West", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 444285951, "name": "SANCHO VILLA", "screen_name": "Burning_Dan", "lang": "en", "location": "Los Angeles", "create_at": date("2011-12-22"), "description": "Man Against Humanity", "followers_count": 319, "friends_count": 208, "statues_count": 18573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawndale, CA", "id": "cce33d74ceffbe08", "name": "Lawndale", "place_type": "city", "bounding_box": rectangle("-118.369186,33.872914 -118.343796,33.902665") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640886, "cityName": "Lawndale" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360020999745536, "text": "@ebbtideapp Tide in Puddledock Sand & Gravel, Virginia 03/17/2016\n Low 5:54am 0.1\nHigh 12:05pm 3.1\n Low 6:31pm 0.2\nHigh 12:31am 2.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-77.3717,37.2667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 45, "friends_count": 1, "statues_count": 16493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51570, "countyName": "Colonial Heights", "cityID": 5118448, "cityName": "Colonial Heights" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360021003931649, "text": "Tim's dad will truly be missed..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418241196, "name": "christianjgg", "screen_name": "Whippinbitches", "lang": "en", "location": "null", "create_at": date("2011-11-21"), "description": "IG: christianjgg", "followers_count": 577, "friends_count": 335, "statues_count": 6086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheaton, MD", "id": "0183569b04a64d1f", "name": "Wheaton", "place_type": "city", "bounding_box": rectangle("-77.109128,39.024229 -77.023211,39.093731") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2483775, "cityName": "Wheaton" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360021012500480, "text": "Wind 1.0 mph NW. Barometer 29.959 in, Steady. Temperature 52.5 °F. Rain today 0.00 in. Humidity 46%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 111816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360021352075265, "text": "#Trump in 2016 Temp:45.1°F Wind:0.0mph Pressure: 29.94hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 78, "friends_count": 17, "statues_count": 312621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360021503049728, "text": "I'm everywhere and nowhere at the same time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513534970, "name": "Lobo", "screen_name": "_ericb3", "lang": "en", "location": "gone", "create_at": date("2012-03-03"), "description": "null", "followers_count": 379, "friends_count": 336, "statues_count": 17434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360021746458624, "text": "Wind 2.0 mph W. Barometer 29.937 in, Falling slowly. Temperature 54.1 °F. Rain today 0.00 in. Humidity 48%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022157553664, "text": "Temp: 63.9°F Wind:0.0mph Pressure: 29.973hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 63, "friends_count": 23, "statues_count": 61741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022203535361, "text": "@matthewperuu @brackhailey", "in_reply_to_status": 710359962589892608, "in_reply_to_user": 3050274600, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3050274600, 2592533984 }}, "user": { "id": 3050274600, "name": "Matty P Raps", "screen_name": "matthewperuu", "lang": "en", "location": "Home", "create_at": date("2015-02-28"), "description": "My slide to my dms is closed (except to Hailey)", "followers_count": 246, "friends_count": 194, "statues_count": 1148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saddlebrooke, AZ", "id": "00b97a050e9ad203", "name": "Saddlebrooke", "place_type": "city", "bounding_box": rectangle("-110.928274,32.511182 -110.855567,32.554549") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 461940, "cityName": "Saddlebrooke" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022488756224, "text": "@Lucayee_ any day next week except Thursday! (I'm on spring break wassup��)", "in_reply_to_status": 710359231262629889, "in_reply_to_user": 1974451982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1974451982 }}, "user": { "id": 2460699638, "name": "broke briicat", "screen_name": "softgreymatter", "lang": "en", "location": "null", "create_at": date("2014-04-23"), "description": "wut", "followers_count": 423, "friends_count": 281, "statues_count": 11812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022585180160, "text": "Matter fact I don't care where she is", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3396183082, "name": "ph0run", "screen_name": "ph0run", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "hi I like dragon ball and some anime I'm Tom Ford on Facebook as well look me up", "followers_count": 112, "friends_count": 0, "statues_count": 28859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022719463425, "text": "How the fuck do I use whatsapp?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218742779, "name": "Kai", "screen_name": "LeelaLovee", "lang": "en", "location": "null", "create_at": date("2010-11-22"), "description": "null", "followers_count": 4069, "friends_count": 452, "statues_count": 11278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coon Rapids, MN", "id": "850e74c08b577616", "name": "Coon Rapids", "place_type": "city", "bounding_box": rectangle("-93.374494,45.118611 -93.265764,45.212876") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2713114, "cityName": "Coon Rapids" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022723637248, "text": "HAPPY BIRTHDAY! ���� Happy 21st,�� I hope you a great day and I can't wait to see you, I miss you.�� Party hard.�� @pelizabeth86", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 754580990 }}, "user": { "id": 2813507161, "name": "Aavontiana", "screen_name": "_aavontiana", "lang": "en", "location": "null", "create_at": date("2014-09-16"), "description": "null", "followers_count": 172, "friends_count": 272, "statues_count": 469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fortuna Foothills, AZ", "id": "ea5ccbacf82bdef6", "name": "Fortuna Foothills", "place_type": "city", "bounding_box": rectangle("-114.462723,32.624634 -114.375516,32.696713") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 425030, "cityName": "Fortuna Foothills" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022786580481, "text": "I DID IT BEFORE SHE FIXING TO CATCH THIS KANYE UNDERCOVER FADE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585734947, "name": "Nesto", "screen_name": "Ernest_Mtz_I", "lang": "en", "location": "San Antonio Tx", "create_at": date("2012-05-20"), "description": "#GoSpursGo⚫⚪\nKinesiology major. Dab on 'em folk", "followers_count": 554, "friends_count": 498, "statues_count": 18622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022966927361, "text": "Miso cod. The ladies ate this one. #jacksoysterbar #prehawaiidinner… https://t.co/qSS0LorK4A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.27550301,37.79417156"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jacksoysterbar", "prehawaiidinner" }}, "user": { "id": 164161890, "name": "Tae Kim", "screen_name": "OaklandTae", "lang": "en", "location": "Oakland, CA", "create_at": date("2010-07-07"), "description": "All Your Base Are Belong To Us!!!!", "followers_count": 44, "friends_count": 56, "statues_count": 1717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-03-17T00:00:01.000Z"), "id": 710360022979489792, "text": "I like this show Suspect on MTV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67981986, "name": "ℬri", "screen_name": "Westside_briiii", "lang": "en", "location": "Los Angeles", "create_at": date("2009-08-22"), "description": "XXlll└A Where the wetha is betta ☀️ Gemini ♊️ #LAKERGANG IG: simplybriiii_ SC: Califaniabri", "followers_count": 3010, "friends_count": 1259, "statues_count": 273411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360023088701440, "text": "Ripley SW Limestone Co. Temp: 52.3°F Wind:1.6mph Pressure: 995.4mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 46, "friends_count": 32, "statues_count": 50308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360023092862976, "text": "@Bakersman_Joe lol I sold that bih the next day too", "in_reply_to_status": 710359905165651969, "in_reply_to_user": 338876136, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 338876136 }}, "user": { "id": 38887130, "name": "ComeForMe", "screen_name": "DuckDuckkGoose", "lang": "en", "location": "Tampa Flaaawda ", "create_at": date("2009-05-09"), "description": "Sc:Duckie813 / Ig:Duckie813 , Im Broke", "followers_count": 1738, "friends_count": 1090, "statues_count": 111433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360023281569792, "text": "My TL so inappropriate at this time��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368679879, "name": "babygirl✨", "screen_name": "Cassie_Lovette", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "There is beauty in broken✨", "followers_count": 3487, "friends_count": 1900, "statues_count": 193061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360023562592256, "text": "Wind 0.0 mph ---. Barometer 29.765 in, Falling. Temperature 38.7 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360023734591488, "text": "and Joy is the real MVP she's doing the most.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1451900904, "name": "צרות באות בצרורות", "screen_name": "Lilmin93", "lang": "en", "location": "NYC.212.", "create_at": date("2013-05-23"), "description": "❣☹ http://goo.gl/8jcnr2", "followers_count": 154, "friends_count": 290, "statues_count": 3590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360024086749185, "text": "lol, maybe I should go to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2772048410, "name": "{ E S S }", "screen_name": "essence_nycole7", "lang": "en", "location": "cache, oklahoma. ⛅️", "create_at": date("2014-08-26"), "description": "don't let people make you sad-mad ok. special olympics volunteer. ❤️", "followers_count": 295, "friends_count": 465, "statues_count": 9231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cache, OK", "id": "01a482d3115479ef", "name": "Cache", "place_type": "city", "bounding_box": rectangle("-98.65041,34.621523 -98.615046,34.640185") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40031, "countyName": "Comanche", "cityID": 4010700, "cityName": "Cache" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360024086806528, "text": "Tempted to get into Greys Anatomy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2181145988, "name": "MaryLou", "screen_name": "x3maryloufm", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "❤️God, love | #⃣Hiphop | Makeup & stuff | EG ➡️ LA", "followers_count": 178, "friends_count": 146, "statues_count": 9591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360024460054529, "text": "These papaw tweets ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 517063312, "name": "ens", "screen_name": "ensleyacosta", "lang": "en", "location": "null", "create_at": date("2012-03-06"), "description": "Be strong and courageous! Deuteronomy 31:6✨", "followers_count": 762, "friends_count": 397, "statues_count": 20849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360024661499904, "text": "Wind 0.0 mph ---. Barometer 29.867 in, Rising. Temperature 43.1F. Rain today 0.00 in. Humidity 98% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 613, "friends_count": 813, "statues_count": 39086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360025080864769, "text": "69.3F (Feels: 69.3F) - Humidity: 99% - Wind: 1.6mph SE - Gust: 3.1mph - Pressure: 1017.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 231028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360025173065728, "text": "\"trumpalumpa\" im crying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338074467, "name": "tedrazzini", "screen_name": "TeddytheTuba", "lang": "en", "location": "Bryan High - Coe College", "create_at": date("2011-07-18"), "description": "19 y/o tuba + banjo player, barbershopper (bass), equestrian, sommelier-in-training, future fireman, mac n' cheese enthusiast, brother of ΦΜΑ.", "followers_count": 139, "friends_count": 137, "statues_count": 23976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360025336774656, "text": "@_itsdabney what happened to you ?", "in_reply_to_status": -1, "in_reply_to_user": 1317413036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1317413036 }}, "user": { "id": 1232880157, "name": "#BrydgeEntLEDROIT", "screen_name": "bossman_LEDROIT", "lang": "en", "location": "Boxing Ring #DC2⃣0⃣2⃣", "create_at": date("2013-03-01"), "description": "#RipRay #RipIndia #Entreprenuer| COO of nirvana properties | profession: boxer | #BrydgeEnt #LaylaLounge #Vita #Midtown | #GODsWill | StandOnPrinciples", "followers_count": 1321, "friends_count": 862, "statues_count": 82606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Park, MD", "id": "e4c17912c815124d", "name": "College Park", "place_type": "city", "bounding_box": rectangle("-76.965596,38.971053 -76.903378,39.022888") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2418750, "cityName": "College Park" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360025894494208, "text": "Dress hangin off your shoulder barely sober ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3252971467, "name": "❣bria❣", "screen_name": "im_just_bria", "lang": "en", "location": "CLOUD 19", "create_at": date("2015-06-22"), "description": "Cali Always. TrvpSoul bc Bryson . WAVY bc Kehlani . TGOD . What's a Queen w/ out a Queen. .LittyCommittee.", "followers_count": 292, "friends_count": 262, "statues_count": 7761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360026137776128, "text": "Why does it have to be so loud ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71724843, "name": "b u t t e r f l y ✨", "screen_name": "RenaeeLovee", "lang": "en", "location": "mother", "create_at": date("2009-09-04"), "description": "A'rya & Aaliyah ❣", "followers_count": 824, "friends_count": 517, "statues_count": 63738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360026301513728, "text": "Main bohut tumse pyar karta hoon\n^_^ ( i love u very much) king at mirchi awards\n@iamsrk… https://t.co/L5YbfYclNl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.99824,21.3868699"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101311381 }}, "user": { "id": 2321607871, "name": "❤ Iamsrk Fans ❤", "screen_name": "Eva_Srk", "lang": "en", "location": "jakarta selatan. indonesia", "create_at": date("2014-01-31"), "description": "am srkian$ from medan (indonesia). i love srk a lot wanna meet him in real . kiss him touch him and hug him(srks my future husband)if u haters dont follow me ok", "followers_count": 1162, "friends_count": 180, "statues_count": 15844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360026913722369, "text": "Your barz cheesy and wack af cuh trippen ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 564688432, "name": "✖", "screen_name": "ayyyye_Nick", "lang": "en", "location": "Okaga, CA", "create_at": date("2012-04-27"), "description": "Blaxican | Long Beach", "followers_count": 74, "friends_count": 171, "statues_count": 4053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-03-17T00:00:02.000Z"), "id": 710360027068895232, "text": "Worst https://t.co/gigOct6npA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300166584, "name": "Janky-J Stasiak ◬", "screen_name": "jadinstasiak", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "a bitch is a bitch", "followers_count": 289, "friends_count": 282, "statues_count": 4039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fishers, IN", "id": "55bb81e82d2aa1aa", "name": "Fishers", "place_type": "city", "bounding_box": rectangle("-86.076196,39.926769 -85.823905,40.000753") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1823278, "cityName": "Fishers" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360027400249345, "text": "For NOTHING will be impossible\nwith God.\nLuke 1:37", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I strive to live each and every passing day, according to Gods ways and will for my life.", "followers_count": 144, "friends_count": 262, "statues_count": 245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360027526090753, "text": "When people know they want get what they want and go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506660476, "name": "Sañchez", "screen_name": "_ALDON", "lang": "en", "location": "Kansas City", "create_at": date("2012-02-27"), "description": "null", "followers_count": 192, "friends_count": 289, "statues_count": 1119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360027622584320, "text": "������ how gang better be if someone jump in https://t.co/7EuwAgB5Jj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1554460536, "name": "Kai Williams ", "screen_name": "RIPCpayne", "lang": "en", "location": "null", "create_at": date("2013-06-28"), "description": "Owner of @Spiffy_Mane2016 | God ,Family,Football | Live life to the fullest | RIP CPC & CARL| Kai Williams don't come at me with no BS Dms closed", "followers_count": 1309, "friends_count": 982, "statues_count": 11089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kemah, TX", "id": "1470167084d62696", "name": "Kemah", "place_type": "city", "bounding_box": rectangle("-95.036881,29.514855 -95.00182,29.549073") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4838776, "cityName": "Kemah" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360028004294657, "text": "@Michael_Husnik this is really random but I found your student ID from last year in one of mrs. Nygren's books", "in_reply_to_status": 710341870723837952, "in_reply_to_user": 596590802, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 596590802 }}, "user": { "id": 474329023, "name": "anj", "screen_name": "ayeits_anj", "lang": "en", "location": "stayin in my lane , mn", "create_at": date("2012-01-25"), "description": "(on-juh-lee) | black lives still matter | everything is fine", "followers_count": 535, "friends_count": 397, "statues_count": 7735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, MN", "id": "1f7188ae9079a893", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-93.197082,44.623363 -93.121633,44.688673") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2720618, "cityName": "Farmington" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360028503535616, "text": "@tylerderickson late?? I been bumpin them since they came out tf you mean", "in_reply_to_status": 710359872399740928, "in_reply_to_user": 909264697, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 909264697 }}, "user": { "id": 382983158, "name": "Nick Milligan", "screen_name": "Kerzyr", "lang": "en", "location": "Columbus, OH", "create_at": date("2011-09-30"), "description": "Introvert. Columbus State Comm. College.\nFunk Volume for life, idc that it's over. Ily Hop, Dizzy, Jarren, SwizZz, Hoppa.", "followers_count": 286, "friends_count": 361, "statues_count": 24381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Worthington, OH", "id": "3de739b7819a3b2d", "name": "Worthington", "place_type": "city", "bounding_box": rectangle("-83.055384,40.074153 -82.948592,40.133801") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3986604, "cityName": "Worthington" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360028700475392, "text": "@_vampsam I want to go super light :/", "in_reply_to_status": 710359813121638400, "in_reply_to_user": 1436806118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1436806118 }}, "user": { "id": 1474139797, "name": "FACTS", "screen_name": "brokeboiiiiiiii", "lang": "en", "location": "over here ", "create_at": date("2013-06-01"), "description": "il y a une place en enfer juste pour moi", "followers_count": 574, "friends_count": 299, "statues_count": 72469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360029031870465, "text": "Homework will be the death of me lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2915042856, "name": "Jessica Gallardo", "screen_name": "jessigallardo6", "lang": "en", "location": "null", "create_at": date("2014-11-30"), "description": "|Dinuba High|", "followers_count": 323, "friends_count": 394, "statues_count": 299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360029380157440, "text": "Alecia the realist ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2641767101, "name": "§helly", "screen_name": "soshellyy", "lang": "en", "location": "San Tan Valley , AZ ☀️", "create_at": date("2014-06-25"), "description": "chapter 18 ... continue reading MY story is only getting better", "followers_count": 744, "friends_count": 316, "statues_count": 3681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360029501652992, "text": "Tonight I watched an ep of 2 Guys and a Girl on YouTube. Obvs @VancityReynolds was on there but completely forgot @NathanFillion was too.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2893511188, 31353077 }}, "user": { "id": 392296412, "name": "Nate Manzo", "screen_name": "SoonerNate67", "lang": "en", "location": "Raleigh", "create_at": date("2011-10-16"), "description": "I like sports and tv and movies and music and funny things. #unique", "followers_count": 123, "friends_count": 846, "statues_count": 3124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wake Forest, NC", "id": "01e2c29e3bad4178", "name": "Wake Forest", "place_type": "city", "bounding_box": rectangle("-78.561149,35.904286 -78.469525,36.013761") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3770540, "cityName": "Wake Forest" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360029581283328, "text": "@thisbesalma HAHAHAHAHAHAHAHA https://t.co/7bNLwxNOaY", "in_reply_to_status": -1, "in_reply_to_user": 827836154, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 827836154 }}, "user": { "id": 2547064033, "name": "DamnDedrick", "screen_name": "boydedrick", "lang": "en", "location": "Santa Clara, CA", "create_at": date("2014-06-04"), "description": "nah", "followers_count": 231, "friends_count": 213, "statues_count": 4524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360029866500096, "text": "#WhichWaytotheFront #TCMParty everybody disses this film - I love it - true first 15 minutes are very slow but after like Hogans Heroes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WhichWaytotheFront", "TCMParty" }}, "user": { "id": 3632614273, "name": "Robert Waggoner", "screen_name": "sgtrobwag", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-09-20"), "description": "US ARMY - 9 years --- MST3K Freak - RIFFTRAX freak - Svengoolie freak - Classic Movies/Shows freak --- so basically a freak", "followers_count": 31, "friends_count": 160, "statues_count": 616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030122356737, "text": "Girl what he did? https://t.co/M13CroNWnk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2486085186, "name": "Jordan Wagner", "screen_name": "jkw2598", "lang": "en", "location": "Spring, TX", "create_at": date("2014-05-09"), "description": "John 21:17", "followers_count": 180, "friends_count": 336, "statues_count": 2386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030176915456, "text": "@jimmyfallon I'm at the Motel 6 in Vallejo 'North Bay Bitch' and I'll have to say, you can't find a better shitty hotel. #DontJudgeMe #420", "in_reply_to_status": -1, "in_reply_to_user": 15485441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DontJudgeMe" }}, "user_mentions": {{ 15485441 }}, "user": { "id": 2557480471, "name": "JDW Robinson", "screen_name": "JDWRobinson", "lang": "en", "location": "USA", "create_at": date("2014-06-09"), "description": "Your #Global insights and opinions interest me. Geopolitical advisor, equestrian, and native traveler. Lost in #Harmony; found by #Grace.", "followers_count": 5088, "friends_count": 382, "statues_count": 9311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030470602752, "text": "And no, for once it's not FUNi being...FUNi with a harem anime, this is on the money. #Vandread", "in_reply_to_status": 710359943522754561, "in_reply_to_user": 16859893, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Vandread" }}, "user": { "id": 16859893, "name": "DigiRanma", "screen_name": "DigiRanma", "lang": "en", "location": "Woodstock, IL", "create_at": date("2008-10-19"), "description": "An anime fan who watches and reviews bad movies... why, I don't know.", "followers_count": 485, "friends_count": 238, "statues_count": 69008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodstock, IL", "id": "6ce71712d41e1929", "name": "Woodstock", "place_type": "city", "bounding_box": rectangle("-88.491765,42.270987 -88.392355,42.350431") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1783349, "cityName": "Woodstock" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030667796480, "text": "Travelling to Miami or just twittering about Miami? https://t.co/W564yd619W #Miami", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.2264,25.789"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami" }}, "user": { "id": 163590921, "name": "MapGame", "screen_name": "Map_Game", "lang": "en", "location": "null", "create_at": date("2010-07-06"), "description": "The Map Game is a free geography quiz based on Google Maps. No flash, no need to register, just start finding places on a blind map!", "followers_count": 1026, "friends_count": 311, "statues_count": 2652369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030676000768, "text": "@brookllewis �������� https://t.co/l3ygBF6hRq", "in_reply_to_status": -1, "in_reply_to_user": 1576323086, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1576323086 }}, "user": { "id": 1213885214, "name": "D Rae", "screen_name": "_dezarae_", "lang": "en", "location": "Moore , Ok ❤️", "create_at": date("2013-02-23"), "description": "You win some. You lose some. Rest easy Pawpaw", "followers_count": 461, "friends_count": 585, "statues_count": 14690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030743146496, "text": "EVERYBODY HURTS SOMETIMES https://t.co/vG24dfoSxM via @wordpressdotcom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 823905 }}, "user": { "id": 26012231, "name": "Felicia Reevers", "screen_name": "MsFelicia", "lang": "en", "location": "Arizona", "create_at": date("2009-03-23"), "description": "Where am I going and why am I carrying this handbasket? Romance is my favorite genre, but love a good mystery, multiracial, and anything with a Cowboy!", "followers_count": 1585, "friends_count": 2207, "statues_count": 5094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360030999158784, "text": "I just \"fluffed\" a pillow for the first time in my life. Who sleeps on pillows that need to be fluffed regularly?! #latenightthoughts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "latenightthoughts" }}, "user": { "id": 855513606, "name": "Anna", "screen_name": "anm31797", "lang": "en", "location": "Manhattan, KS", "create_at": date("2012-09-30"), "description": "KSU '19 | ΓΦΒ | Matthew 7:25", "followers_count": 522, "friends_count": 502, "statues_count": 10686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Destin, FL", "id": "18a461be4c0c2033", "name": "Destin", "place_type": "city", "bounding_box": rectangle("-86.515771,30.37865 -86.39721,30.417706") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1217325, "cityName": "Destin" } }
+{ "create_at": datetime("2016-03-17T00:00:03.000Z"), "id": 710360031296823296, "text": "Soooo glad the quarter is finally over", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1621432062, "name": "Brooks", "screen_name": "Brrookss", "lang": "en", "location": "Camas, Washington", "create_at": date("2013-07-25"), "description": "University of Washington | #Barça", "followers_count": 451, "friends_count": 291, "statues_count": 1082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camas, WA", "id": "219460d3060c1d2e", "name": "Camas", "place_type": "city", "bounding_box": rectangle("-122.474225,45.57579 -122.367318,45.633039") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5309480, "cityName": "Camas" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360031720579073, "text": "Wind 0.7 mph SSE. Barometer 29.85 in, Steady. Temperature 42.6 °F. Rain today 0.01 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 117, "statues_count": 158328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360032471154688, "text": "Demon to lean on https://t.co/LTxAc9jhEK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 867207482, "name": "Trashley", "screen_name": "bombdigity13", "lang": "en", "location": "null", "create_at": date("2012-10-07"), "description": "in my head it's a hell hole", "followers_count": 343, "friends_count": 309, "statues_count": 670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon Lake, CA", "id": "85072b5a406cd0ff", "name": "Canyon Lake", "place_type": "city", "bounding_box": rectangle("-117.283988,33.670173 -117.234601,33.71489") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 610928, "cityName": "Canyon Lake" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360032634740736, "text": "13 hours left in Cali ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455242408, "name": "Kaptain", "screen_name": "KurtKrauss", "lang": "en", "location": "Toms River, NJ", "create_at": date("2012-01-04"), "description": "#SB- WLTBWNLAT Down for whateva It's what you do in the dark that puts you in the light", "followers_count": 478, "friends_count": 474, "statues_count": 14352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360032706080768, "text": "esta lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 261896718, "name": "vicky", "screen_name": "victooooria09", "lang": "en", "location": "null", "create_at": date("2011-03-06"), "description": "ling ling", "followers_count": 663, "friends_count": 2135, "statues_count": 4097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conroe, TX", "id": "361610d5aa1f6719", "name": "Conroe", "place_type": "city", "bounding_box": rectangle("-95.679001,30.23105 -95.397267,30.443978") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4816432, "cityName": "Conroe" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360032764887040, "text": "Thank you to all my customers for supporting on buying… https://t.co/gueqEJMWS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.378,34.1722"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2393332273, "name": "hacwearcompany", "screen_name": "hacwearcompany", "lang": "en", "location": "null", "create_at": date("2014-03-16"), "description": "Hmu wen yu want to buy one of these @hacsupply Hats(20ea.)\nSnapbacks & T-Shirts(10ea.)\n12918 Vanowen ave. NorthHollywood Ca 91605 Callme (818)821-2938", "followers_count": 370, "friends_count": 1937, "statues_count": 1013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360032919965696, "text": "@OutThisJAM nope, still in Wisconsin :/", "in_reply_to_status": 710359901143437312, "in_reply_to_user": 314364445, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314364445 }}, "user": { "id": 3613054756, "name": "Sarah Hon", "screen_name": "sarahhon01", "lang": "en", "location": "Beloit, WI", "create_at": date("2015-09-10"), "description": "Assistant Women's Lacrosse Coach at Beloit College", "followers_count": 164, "friends_count": 463, "statues_count": 235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beloit, WI", "id": "000fc3e67b627897", "name": "Beloit", "place_type": "city", "bounding_box": rectangle("-89.079237,42.496019 -88.948817,42.610064") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55105, "countyName": "Rock", "cityID": 5506500, "cityName": "Beloit" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360033477808128, "text": "damn...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2565737519, "name": "karen espinoza.", "screen_name": "kaaarenespinoza", "lang": "en", "location": "chino", "create_at": date("2014-05-26"), "description": "suh dude", "followers_count": 259, "friends_count": 150, "statues_count": 11932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360033515601921, "text": "Girls https://t.co/6ZfEF0k56b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439379717, "name": "вяιʌи ✟", "screen_name": "VSLX3_", "lang": "en", "location": "null", "create_at": date("2014-04-11"), "description": "#36 E-Block or No Block #MG", "followers_count": 207, "friends_count": 101, "statues_count": 5227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360034685820928, "text": "2.0 magnitude #earthquake. 7 mi from Morongo Valley, #CA, United States https://t.co/6Afm1Lm5Il", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.705,34.018"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "CA" }}, "user": { "id": 715541250, "name": "Southern CA Quakes", "screen_name": "quakesLA", "lang": "en", "location": "Los Angeles", "create_at": date("2012-07-24"), "description": "Southern California earthquake alerts based on USGS data. 2.5 magnitude and higher in Los Angeles and Southern California.", "followers_count": 17295, "friends_count": 15, "statues_count": 11150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360034706911233, "text": "I wanna bang like a pistol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464383955, "name": "Fujiwara", "screen_name": "jean_zorrilla", "lang": "en", "location": "Clifton, NJ", "create_at": date("2012-01-14"), "description": "Pure crush groovin, Nj sc: jzorrilla", "followers_count": 509, "friends_count": 501, "statues_count": 21906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clifton, NJ", "id": "7356b662670b2c31", "name": "Clifton", "place_type": "city", "bounding_box": rectangle("-74.197728,40.82028 -74.118533,40.899384") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3413690, "cityName": "Clifton" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360034752921600, "text": "4. More. Days.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1646959292, "name": "Milsa Reyes(:", "screen_name": "MilsaBiebz18", "lang": "en", "location": "null", "create_at": date("2013-08-04"), "description": "Justin is my fav.\nLA Kings!", "followers_count": 335, "friends_count": 150, "statues_count": 14266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-03-17T00:00:04.000Z"), "id": 710360035298246656, "text": "Jk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 548132685, "name": "Tyler Marlow", "screen_name": "TysDaName", "lang": "en", "location": "Temecula, CA / Las Vegas, NV", "create_at": date("2012-04-07"), "description": "Marlow is the funniest kid you'll ever meet - Tyler Marlow", "followers_count": 92, "friends_count": 228, "statues_count": 5198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360035847569408, "text": "Leave her alone lol https://t.co/6CzusFgDfy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142799192, "name": " Jasmine", "screen_name": "JReneex", "lang": "en", "location": "BeachyCalifornia", "create_at": date("2010-05-11"), "description": "Forever alone giant .....", "followers_count": 773, "friends_count": 532, "statues_count": 93741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360035914743808, "text": "Yea I'm hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614781194, "name": "dont", "screen_name": "TorresNahum", "lang": "en", "location": "Coachella, CA", "create_at": date("2012-06-21"), "description": "null", "followers_count": 156, "friends_count": 168, "statues_count": 6088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360036279754752, "text": "2.0 magnitude #earthquake. 12 km from Morongo Valley, CA, #UnitedStates https://t.co/1TjL445G0J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.705,34.018"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 66581, "friends_count": 10, "statues_count": 98734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360036384448512, "text": "@PainInTheSash I’m trying very hard not to let my enjoyment of the Madness rest on our Hawkeyes performance.", "in_reply_to_status": 710359807522279424, "in_reply_to_user": 2718355034, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2718355034 }}, "user": { "id": 67316708, "name": "Go Hawks", "screen_name": "HawkeyeMommeye", "lang": "en", "location": "Chi-town's North Shore", "create_at": date("2009-08-20"), "description": "HawkeyeGirleye on @BHGP, SAHM/soccer mom, Fitbit addict, bleed black & gold, Blackhawk red & Cubbie blue", "followers_count": 488, "friends_count": 332, "statues_count": 58179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360036397023233, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 585386619, "name": "Hoezay", "screen_name": "JoseMartineezz", "lang": "en", "location": "661,California ", "create_at": date("2012-05-19"), "description": "null", "followers_count": 1268, "friends_count": 654, "statues_count": 28287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360036812304386, "text": "Thank you �� https://t.co/Ul5rCxEdJp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60203175, "name": "icy rivers.", "screen_name": "mikesmovement", "lang": "en", "location": "null", "create_at": date("2009-07-25"), "description": "null", "followers_count": 550, "friends_count": 169, "statues_count": 51613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-03-17T00:00:05.000Z"), "id": 710360036900405249, "text": "@xexiamor Mamita chula que tengas una noche linda, te quiero mucho hermosa bella y relájate Y descanse con paz����������ya todo se va a mejorar", "in_reply_to_status": -1, "in_reply_to_user": 3463332674, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3463332674 }}, "user": { "id": 78389807, "name": "hugo salazar", "screen_name": "netio29", "lang": "en", "location": "Culver City, CA", "create_at": date("2009-09-29"), "description": "null", "followers_count": 16, "friends_count": 175, "statues_count": 128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635459397967872, "text": "Similar styles found at @Kohls #LakeNona https://t.co/XSu6GbsfaT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LakeNona" }}, "user_mentions": {{ 35880417 }}, "user": { "id": 4519865729, "name": "Lake Nona Small Biz", "screen_name": "NonaSmallBiz", "lang": "en", "location": "Orlando, FL", "create_at": date("2015-12-17"), "description": "Find services and best deals in the Lake Nona area. Enjoy the small businesses that make our community a very unique place to work, play and live!", "followers_count": 203, "friends_count": 764, "statues_count": 788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635459544592384, "text": "No one Thot https://t.co/aAmB3yxlAC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 566038076, "name": "Victoria Secret", "screen_name": "_victoriuh", "lang": "en", "location": "ktx", "create_at": date("2012-04-28"), "description": "I ruin lives.", "followers_count": 4052, "friends_count": 841, "statues_count": 62898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635459779649540, "text": "@maxp____ @ARDXLINO", "in_reply_to_status": 715635271874842624, "in_reply_to_user": 172653202, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 172653202, 1099519890 }}, "user": { "id": 702307921, "name": "ap", "screen_name": "_palomb0", "lang": "en", "location": "tallahassee/ wellington", "create_at": date("2012-07-17"), "description": "http://1017.com", "followers_count": 178, "friends_count": 269, "statues_count": 8568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460022865920, "text": "@suzimcc That's my viewpoint. @iSocialFanz and I may not differ much - I just think there's always some strategy, never out the door.", "in_reply_to_status": 715634621703196673, "in_reply_to_user": 20621761, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20621761, 633287684 }}, "user": { "id": 18791723, "name": "Chris Barrows", "screen_name": "CBarrows", "lang": "en", "location": "NY/NJ", "create_at": date("2009-01-08"), "description": "SM Strategist (#HESM, @NYUDigiComm) | Telling stories on @WhyISocial (Sponsor: @Zoomph) | @EduWebConf Chair | #Tabletop #PeopleFirst \n\nSnap or Insta: CGBarrows", "followers_count": 10129, "friends_count": 7864, "statues_count": 59540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460400226304, "text": "When you finally reach a year ���� feeling accomplished https://t.co/Zl9Xgsumw6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416774241, "name": "Katie Schuler", "screen_name": "kay_schuler", "lang": "en", "location": "Oxford, MS", "create_at": date("2011-11-19"), "description": "hotty toddy", "followers_count": 611, "friends_count": 455, "statues_count": 4133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MS", "id": "b86cd6ef7a54d640", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-89.571643,34.321375 -89.495897,34.387965") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28071, "countyName": "Lafayette", "cityID": 2854840, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460517732352, "text": "The Vitamin Shoppe #Retail #Job: Store Manager (#Appleton, WI) https://t.co/e2LkBJ9RPJ #Jobs #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.4699601,44.2730307"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Appleton", "Jobs", "Hiring", "CareerArc" }}, "user": { "id": 59784301, "name": "TMJ-WI Retail Jobs", "screen_name": "tmj_wi_retail", "lang": "en", "location": "Wisconsin", "create_at": date("2009-07-24"), "description": "Follow this account for geo-targeted Retail job tweets in Wisconsin Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 368, "friends_count": 309, "statues_count": 540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Chute, WI", "id": "01de44ff61a82ea1", "name": "Grand Chute", "place_type": "city", "bounding_box": rectangle("-88.496842,44.243882 -88.385483,44.329156") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55087, "countyName": "Outagamie" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460702228480, "text": "Reopened in #Miami on I-95 Express NB from Downtown Miami Entrance to Golden Glades Express Exit #SFLtraffic https://t.co/US2j7Ix0rb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.20557,25.8023"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami", "SFLtraffic" }}, "user": { "id": 249831776, "name": "TT+WN South Florida", "screen_name": "TotalTrafficMIA", "lang": "en", "location": "Miami, FL", "create_at": date("2011-02-09"), "description": "Total Traffic and Weather Network.Hitting the road? Follow us for the latest road conditions, accidents and delays on your South Florida drive.", "followers_count": 28124, "friends_count": 189, "statues_count": 89103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460739981312, "text": "Wouldn't trade my relationship with my bros for anything in the world.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2528261444, "name": "King Lake", "screen_name": "king__lake", "lang": "en", "location": "out here", "create_at": date("2014-05-27"), "description": "GooD ViBe TRiBe.\n\nWWKD", "followers_count": 295, "friends_count": 214, "statues_count": 2192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenwood, IN", "id": "008f15ef5cfd041a", "name": "Greenwood", "place_type": "city", "bounding_box": rectangle("-86.25064,39.529358 -86.040001,39.636719") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18081, "countyName": "Johnson", "cityID": 1829898, "cityName": "Greenwood" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460802945024, "text": "We're #hiring! Read about our latest #job opening here: Cook - https://t.co/fdHCJqDxdl #Denver, CO #Hospitality #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.9847034,39.7391536"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Denver", "Hospitality", "CareerArc" }}, "user": { "id": 22487719, "name": "Denver Hospitality", "screen_name": "tmj_den_hrta", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-02"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in Denver, CO. Need help? Tweet us at @CareerArc!", "followers_count": 516, "friends_count": 293, "statues_count": 1069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460865806336, "text": "AP voters must've missed most of the regular season and the Tournament. Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 533765305, "name": "Jared McCausland", "screen_name": "JaredMcCausland", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-03-22"), "description": "Alumnus of The University of Oklahoma. God Bless Texas. History Fanatic. #MUFC #ThunderUp #GoStars #CowboysNation #DTID", "followers_count": 178, "friends_count": 229, "statues_count": 3590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635460979224577, "text": "�� https://t.co/Tegw3gCQk8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 459680528, "name": "Liz Santana", "screen_name": "lizettex_n", "lang": "en", "location": "BOS", "create_at": date("2012-01-09"), "description": "null", "followers_count": 758, "friends_count": 375, "statues_count": 21859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynn, MA", "id": "3e711c16431907f4", "name": "Lynn", "place_type": "city", "bounding_box": rectangle("-71.01153,42.444954 -70.92218,42.51252") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2537490, "cityName": "Lynn" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461146804224, "text": "Congratulations to my old team at @SeeMonterey (and @simpleview) on the fab new site! Very engaging, great design. https://t.co/qpg9SIlW12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18149301, 18192950 }}, "user": { "id": 1254751, "name": "Nicole Gustas", "screen_name": "rednikki", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2007-03-15"), "description": "A collection of odd facts and quirky knowledge. Digital Marketing Manager for Tourism Australia. All opinions are my own.", "followers_count": 1471, "friends_count": 1135, "statues_count": 12824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461260058625, "text": "AJ Hinch says he knows what he's going to do concerning his closer. Wasn't ready to announce it & may not. Doesn't consider titles important", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78515008, "name": "Mark Berman", "screen_name": "MarkBermanFox26", "lang": "en", "location": "Houston, Tx", "create_at": date("2009-09-29"), "description": "Sports Director at Fox 26 (KRIV)", "followers_count": 16628, "friends_count": 839, "statues_count": 18016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461427970055, "text": "Thankful to former students. Matthew & Andi, who came to the Scholar-Teacher symposium to cheer… https://t.co/5JkOBq8SgJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.34722041,35.56064082"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2381981, "name": "Will Banks", "screen_name": "rhetboi", "lang": "en", "location": "Greenville, NC", "create_at": date("2007-03-26"), "description": "null", "followers_count": 500, "friends_count": 473, "statues_count": 6971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Carolina, USA", "id": "3b98b02fba3f9753", "name": "North Carolina", "place_type": "admin", "bounding_box": rectangle("-84.321948,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37147, "countyName": "Pitt", "cityID": 3728080, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461537005570, "text": "Sushi date night with my roommates", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287955952, "name": "veez nutz", "screen_name": "veandrasak", "lang": "en", "location": "cleveland ", "create_at": date("2011-04-25"), "description": "ICASI pastry & Daniel", "followers_count": 158, "friends_count": 53, "statues_count": 12076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middleburg Heights, OH", "id": "6f9917397013c785", "name": "Middleburg Heights", "place_type": "city", "bounding_box": rectangle("-81.855797,41.350571 -81.784556,41.389426") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3949644, "cityName": "Middleburg Heights" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461537067009, "text": "Things to Do in Denver When You're With Your Dad #MovingHomeFilms https://t.co/ylyuAYTOrw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MovingHomeFilms" }}, "user": { "id": 1655909672, "name": "Feral Urkel", "screen_name": "twitsnotnice", "lang": "en", "location": "Hades ", "create_at": date("2013-08-08"), "description": "according to a famous linguist, the two most mellifluous words in the English language are Feral Urkle. #YokoBono", "followers_count": 1331, "friends_count": 671, "statues_count": 8845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461549494272, "text": "Ermac https://t.co/abWdGnMtVO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1947483524, "name": "#GoonSquad", "screen_name": "TheeGDawg", "lang": "en", "location": "Under₩ater, TX", "create_at": date("2013-10-08"), "description": "i only care about basketball, hip-hop, and anime philosopher/philanthropist Indigoism #RIPTree", "followers_count": 1406, "friends_count": 945, "statues_count": 85783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635461826281472, "text": "Join the Kalispell Regional Healthcare team! See our latest #Nursing #job opening here: https://t.co/DXWQkVDbfj #Kalispell, MT #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.3168131,48.1919889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Kalispell", "Hiring" }}, "user": { "id": 67083158, "name": "TMJ-MT Nursing Jobs", "screen_name": "tmj_mt_nursing", "lang": "en", "location": "Montana", "create_at": date("2009-08-19"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in Montana Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 338, "friends_count": 292, "statues_count": 530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalispell, MT", "id": "8598f8dca7729aae", "name": "Kalispell", "place_type": "city", "bounding_box": rectangle("-114.384176,48.159653 -114.276672,48.255224") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30029, "countyName": "Flathead", "cityID": 3040075, "cityName": "Kalispell" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462132539394, "text": "I'm a soaked Peter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540279455, "name": "PΣTΣS", "screen_name": "tjpeters27", "lang": "en", "location": "Probably at work", "create_at": date("2012-03-29"), "description": "Rip AJ #1 #blessed yah know?", "followers_count": 670, "friends_count": 334, "statues_count": 9628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mishawaka, IN", "id": "d04cf2a772b88bba", "name": "Mishawaka", "place_type": "city", "bounding_box": rectangle("-86.216235,41.621439 -86.090052,41.727645") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1849932, "cityName": "Mishawaka" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462350716930, "text": "For a second year, Eyes on Main Street photo festival will be back. April 9 - July 2016. https://t.co/zWpuzpPc7i", "in_reply_to_status": 714798361535193088, "in_reply_to_user": 2492262133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2492262133, "name": "Historic DT Wilson", "screen_name": "DowntownWilson", "lang": "en", "location": "Wilson, NC ", "create_at": date("2014-05-12"), "description": "Crossroads of Charm and Innovation", "followers_count": 181, "friends_count": 183, "statues_count": 311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilson, NC", "id": "551f529f172a516a", "name": "Wilson", "place_type": "city", "bounding_box": rectangle("-77.990547,35.682149 -77.872166,35.795207") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37195, "countyName": "Wilson", "cityID": 3774540, "cityName": "Wilson" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462694502400, "text": "#CapeMay, NJ #Nursing #Job: Registered Nurse (RN) / Telemetry at Accountable Healthcare Staffing https://t.co/NzUUGTMLCs #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.9060053,38.9351125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CapeMay", "Nursing", "Job", "Jobs", "Hiring" }}, "user": { "id": 120507895, "name": "TMJ-NJ Nursing Jobs", "screen_name": "tmj_NJ_NURSING", "lang": "en", "location": "New Jersey", "create_at": date("2010-03-06"), "description": "Follow this account for geo-targeted Healthcare-Nursing job tweets in New Jersey Non-Metro. Need help? Tweet us at @CareerArc!", "followers_count": 462, "friends_count": 292, "statues_count": 478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape May, NJ", "id": "a4cbd877de20a757", "name": "Cape May", "place_type": "city", "bounding_box": rectangle("-74.938857,38.928771 -74.871078,38.954047") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3410270, "cityName": "Cape May" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462799536130, "text": "I have an obsession with bralettes ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418509026, "name": "Court ❁ ❁ ❁", "screen_name": "Courtneyyyy_19", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2011-11-22"), "description": "FSU ↣ // 21 // Proverbs 3 5-6 ✟", "followers_count": 1125, "friends_count": 1424, "statues_count": 7445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462896029696, "text": "@DXXMXNXY pregnant", "in_reply_to_status": 715463421152485377, "in_reply_to_user": 372612662, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 372612662 }}, "user": { "id": 1135821728, "name": "✨adriana villarreal", "screen_name": "ogadri_", "lang": "en", "location": "san antonio tx", "create_at": date("2013-01-30"), "description": "Brack", "followers_count": 1935, "friends_count": 490, "statues_count": 51202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635462988304384, "text": "I agree with him. https://t.co/RZ779Efg3J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15852646, "name": "Yves Darbouze", "screen_name": "YvesDarbouze", "lang": "en", "location": "BROOKLYN", "create_at": date("2008-08-14"), "description": "http://yvesdarbouze.com", "followers_count": 901, "friends_count": 532, "statues_count": 67359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635463197884416, "text": "#Serendipity #Streetgazing I'm #shooting their #engagement #dinner tomorrow and just met #Fiancé… https://t.co/xebGl5148C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.17131792,39.94980432"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Serendipity", "Streetgazing", "shooting", "engagement", "dinner", "Fiancé" }}, "user": { "id": 227376364, "name": "Reuben Harley", "screen_name": "BigRubeHarley", "lang": "en", "location": "Philly // New York", "create_at": date("2010-12-16"), "description": "Photovoyeur.Lifestyle Guru. Brand Marketer. Blogger. Freelance @phillydailynews Street Style columnist and Chef.", "followers_count": 2677, "friends_count": 1198, "statues_count": 17027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635463294361600, "text": "Disabled vehicle, right lane blocked in #Baldwin on I-10 EB near US-301 #traffic https://t.co/xLiEqnZoyg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.98299,30.28881"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Baldwin", "traffic" }}, "user": { "id": 250304006, "name": "TT+WN JAX", "screen_name": "TotalTrafficJAX", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-02-10"), "description": "null", "followers_count": 113, "friends_count": 43, "statues_count": 10782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southwest, FL", "id": "15c127e5cf660d95", "name": "Southwest", "place_type": "neighborhood", "bounding_box": rectangle("-82.049504,30.185738 -81.751879,30.373713") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1235000, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-03-31T13:22:44.000Z"), "id": 715635463361544193, "text": "@JessicaUlrich22 @yelyahwilliams I still do ��", "in_reply_to_status": 715193827666079749, "in_reply_to_user": 1284826350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1284826350, 40981798 }}, "user": { "id": 542893150, "name": "Kennadie Lawson", "screen_name": "Kennadie_Lawson", "lang": "en", "location": "null", "create_at": date("2012-04-01"), "description": "ⓚ | twenty | øhiø", "followers_count": 125, "friends_count": 351, "statues_count": 2674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenhills, OH", "id": "c4cb0cb2b921e9d2", "name": "Greenhills", "place_type": "city", "bounding_box": rectangle("-84.536341,39.259634 -84.503097,39.275616") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3932158, "cityName": "Greenhills" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463575437316, "text": "sumbodyy come smoke blunts wit me on thisss lovely daayyy��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1038289920, "name": "#️⃣7️⃣3️⃣0️⃣", "screen_name": "CraGotDatThaang", "lang": "en", "location": "Long Branch, NJ", "create_at": date("2012-12-26"), "description": "TRYNAA GETT RICHHH", "followers_count": 642, "friends_count": 498, "statues_count": 12253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Branch, NJ", "id": "97e1b68100f6629d", "name": "Long Branch", "place_type": "city", "bounding_box": rectangle("-74.012435,40.260557 -73.974657,40.328156") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3441310, "cityName": "Long Branch" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463579676673, "text": "Just my point of view ������ #nyc #lowereastside #taylersapartment #whenwiththemodels @ Lower East… https://t.co/SL0VbiGkr2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98972222,40.71722222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nyc", "lowereastside", "taylersapartment", "whenwiththemodels" }}, "user": { "id": 22822759, "name": "Jennifer Doll", "screen_name": "TheJenniferDoll", "lang": "en", "location": "Toronto, Ontario", "create_at": date("2009-03-04"), "description": "CEO of G of G Inc (www.twitter/GofGInc) IG: thejenniferdoll FB:http://www.facebook.com/jennhing YYZ living but I've left my heart in so many places", "followers_count": 1182, "friends_count": 654, "statues_count": 6444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463579688960, "text": "SK gets runners on 3rd and 2nd but can't bring home a run. Stamford coming up for its first at bats.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.620734,42.423657"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3654738017, "name": "Daily Star Sports", "screen_name": "DS_sportsnews", "lang": "en", "location": "Oneonta, NY", "create_at": date("2015-09-14"), "description": "The official Twitter account of The Daily Star sports department (Oneonta, NY).", "followers_count": 665, "friends_count": 137, "statues_count": 6262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Archibald Field", "id": "07d9f67efb482001", "name": "Archibald Field", "place_type": "poi", "bounding_box": rectangle("-74.62073409999999,42.4236569 -74.620734,42.423657") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36025, "countyName": "Delaware" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463663394816, "text": "@Its_Kelton @_tytoonice_ @BigNeechi boy I watched it on the snap �� this man Neechi despises Androids ����", "in_reply_to_status": 715634981985366016, "in_reply_to_user": 1046966923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1046966923, 2783400962, 288513282 }}, "user": { "id": 587950640, "name": "Tommy Strong", "screen_name": "TimWoods43", "lang": "en", "location": "817...", "create_at": date("2012-05-22"), "description": "#LongLiveCT SC: t_woods45", "followers_count": 1047, "friends_count": 729, "statues_count": 27584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463671910401, "text": "For someone who LOVES attention, she sure is camera shy. #daddysgirl #nakia #shydog #centralpark… https://t.co/JPRjoEPqPf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.95211085,40.79789429"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "daddysgirl", "nakia", "shydog", "centralpark" }}, "user": { "id": 4688141719, "name": "James A Pierce III", "screen_name": "jamesapierceIII", "lang": "en", "location": "New York, NY", "create_at": date("2016-01-01"), "description": "http://www.gaymerstv.com", "followers_count": 229, "friends_count": 493, "statues_count": 190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635463948775424, "text": "When you accidentally fall sleep for 14 hours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262539400, "name": "Austin Skinner", "screen_name": "austindskinner", "lang": "en", "location": "Cali / Illinois", "create_at": date("2013-12-26"), "description": "null", "followers_count": 159, "friends_count": 129, "statues_count": 2157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Forest, IL", "id": "0fb8d7b7e21d86a5", "name": "River Forest", "place_type": "city", "bounding_box": rectangle("-87.834577,41.879203 -87.80495,41.908842") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1764304, "cityName": "River Forest" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464078622720, "text": "Turn my headphones up!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346645755, "name": "AJ Camarota", "screen_name": "AJ_Camarota", "lang": "en", "location": "State College, PA", "create_at": date("2011-08-01"), "description": "The Pennsylvania State University//If you can't have fun, there's no sense in doing it #maximumeffort", "followers_count": 358, "friends_count": 521, "statues_count": 16521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "State College, PA", "id": "22c613c36f32f0d1", "name": "State College", "place_type": "city", "bounding_box": rectangle("-77.917295,40.749326 -77.798924,40.817749") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42027, "countyName": "Centre", "cityID": 4273808, "cityName": "State College" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464158519296, "text": "Breckinridge County Middle School officials say kids took shelter, no injuries, now being sent home. @WHAS11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20563513 }}, "user": { "id": 279720011, "name": "Shane Simmons", "screen_name": "shanesimmonstv", "lang": "en", "location": "Louisville", "create_at": date("2011-04-09"), "description": "@WHAS11 Emmy-nominated storyteller who shoots, writes and edits | Goal Chaser | Dictionary Geek | Golfer | Hoosier | RT's ❌ endorsements. | IN ✈️ IL ✈️ KY", "followers_count": 1012, "friends_count": 332, "statues_count": 9935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464250601472, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/immxI6GIYJ #parttime #Herndon, VA #Veterans https://t.co/m9tqio4PFU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.3956698,38.9494473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "parttime", "Herndon", "Veterans" }}, "user": { "id": 404878140, "name": "Panera Careers", "screen_name": "PaneraCareers", "lang": "en", "location": "null", "create_at": date("2011-11-04"), "description": "#nowhiring #parttime #hourlyjobs", "followers_count": 204, "friends_count": 0, "statues_count": 12759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McNair, VA", "id": "00e516e00f21079b", "name": "McNair", "place_type": "city", "bounding_box": rectangle("-77.429394,38.941438 -77.388917,38.962919") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5148450, "cityName": "McNair" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464267558912, "text": "Epic #Powder day at #vail @ Belle's Camp - Vail, CO https://t.co/ydHciYZq9D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.32854141,39.56302087"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Powder", "vail" }}, "user": { "id": 19292699, "name": "Joey Kercher", "screen_name": "joeykerch", "lang": "en", "location": "Denver, CO", "create_at": date("2009-01-21"), "description": "President/Founder of @airfreshmarket & @isnowreport. Entrepreneur, SAG Actor, Identical Triplet. As Seen on TV. @YEC & @Forbescouncils Member", "followers_count": 1637, "friends_count": 2061, "statues_count": 1740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado, USA", "id": "e21c8e4914eef2b3", "name": "Colorado", "place_type": "admin", "bounding_box": rectangle("-109.060257,36.992427 -102.041524,41.003445") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8037, "countyName": "Eagle" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464292552704, "text": "Feels like classes are getting longer and longer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632528087, "name": "A-A-Ron❄️", "screen_name": "AaronCanepa", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2012-07-10"), "description": "Don't ever, for any reason, do anything to anyone, for any reason, ever, no matter what. CA✈️AZ #Loading...", "followers_count": 418, "friends_count": 197, "statues_count": 36698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464527552512, "text": "I literally hear this everyday. I hate you. @MadisenMaring https://t.co/8Avxp1QQ9y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2770099949 }}, "user": { "id": 699121426016686080, "name": "Laurrr:-))", "screen_name": "laurenmaherrr", "lang": "en", "location": "woodhaven mi", "create_at": date("2016-02-14"), "description": "instagram: @llaurenmaherr", "followers_count": 268, "friends_count": 284, "statues_count": 833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464619692032, "text": "I can't wait to get to my dorm and take my pants off.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2347100846, "name": "K", "screen_name": "karenanalyn", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "Texas| #Tamuk19| Psychology Major", "followers_count": 387, "friends_count": 371, "statues_count": 9777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464762339329, "text": "I missed my phone so much ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4203191533, "name": "Gabriela Segura ✨", "screen_name": "_valesegura", "lang": "en", "location": "htx ", "create_at": date("2015-11-11"), "description": "null", "followers_count": 310, "friends_count": 251, "statues_count": 3900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464829476864, "text": "I've been feeling sick for so long now , I'm tired of it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2196589878, "name": "Cat", "screen_name": "CatherineAnn717", "lang": "en", "location": "null", "create_at": date("2013-11-15"), "description": "My life is about as organized as the $5 DVD bin at Wal-Mart", "followers_count": 802, "friends_count": 386, "statues_count": 24667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635464926052353, "text": "I AM THE BASIC https://t.co/o30crJh8Bf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348605131, "name": "jennifart", "screen_name": "justjennixo", "lang": "en", "location": "north carolina ", "create_at": date("2011-08-04"), "description": "schs '16 // romans 5:8", "followers_count": 1750, "friends_count": 857, "statues_count": 124750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flat Rock, NC", "id": "7cc8dde74254fbeb", "name": "Flat Rock", "place_type": "city", "bounding_box": rectangle("-80.600997,36.435953 -80.53222,36.536559") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37171, "countyName": "Surry", "cityID": 3723640, "cityName": "Flat Rock" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635465416794113, "text": "@NicoleTurner5 @litmusapp Congrats Nikki! Your going to Boston. how cool is that. #LitmusLive", "in_reply_to_status": 715589101970436097, "in_reply_to_user": 27664352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LitmusLive" }}, "user_mentions": {{ 27664352, 14180282 }}, "user": { "id": 154140884, "name": "Peter Ansbacher", "screen_name": "MeetPostivePete", "lang": "en", "location": "Raleigh, NC", "create_at": date("2010-06-10"), "description": "Email Marketing Guru, 2006 President of the Raleigh Jaycees, All-Around good guy.", "followers_count": 413, "friends_count": 600, "statues_count": 5233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635465798332416, "text": "A hoe gon be a hoe, and a lame gon be a lame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2442290849, "name": "Caleña", "screen_name": "valenbernaal", "lang": "en", "location": "Cali, Colombia", "create_at": date("2014-03-28"), "description": "Lady Bison Basketball", "followers_count": 1559, "friends_count": 656, "statues_count": 63858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doral, FL", "id": "053ab8e6678199ed", "name": "Doral", "place_type": "city", "bounding_box": rectangle("-80.41729,25.780974 -80.321016,25.870313") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1217935, "cityName": "Doral" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635465882198016, "text": "#CareerArc #Retail #Job alert: Assistant Store Manager | The Vitamin Shoppe | #Lynchburg, VA https://t.co/5MpENpiCWt #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.1844969,37.350229"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CareerArc", "Retail", "Job", "Lynchburg", "Jobs", "Hiring" }}, "user": { "id": 28705647, "name": "TMJ-ROA Retail Jobs", "screen_name": "tmj_roa_retail", "lang": "en", "location": "Roanoke, VA", "create_at": date("2009-04-03"), "description": "Follow this account for geo-targeted Retail job tweets in Roanoke, VA. Need help? Tweet us at @CareerArc!", "followers_count": 335, "friends_count": 308, "statues_count": 173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynchburg, VA", "id": "2490949a68631669", "name": "Lynchburg", "place_type": "city", "bounding_box": rectangle("-79.257789,37.332496 -79.100237,37.469415") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51680, "countyName": "Lynchburg", "cityID": 5147672, "cityName": "Lynchburg" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635466112868353, "text": "We're #hiring! Read about our latest #job opening here: Host - Captain Morgan Club - https://t.co/mUxX3e9BWM #Arlington, TX #Hospitality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.1080656,32.735687"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Arlington", "Hospitality" }}, "user": { "id": 22454135, "name": "Dallas Hospitality", "screen_name": "tmj_dfw_hrta", "lang": "en", "location": "DFW, Texas", "create_at": date("2009-03-01"), "description": "Follow this account for geo-targeted Hospitality/Restaurant/Tourism job tweets in DFW, Texas. Need help? Tweet us at @CareerArc!", "followers_count": 631, "friends_count": 294, "statues_count": 3134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635466377248768, "text": "@partyvowels ahh yesss", "in_reply_to_status": 715635372978540544, "in_reply_to_user": 3109246497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3109246497 }}, "user": { "id": 377727657, "name": "justine", "screen_name": "longl0st", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-09-21"), "description": "musician, music journalist, & dog lover.", "followers_count": 348, "friends_count": 642, "statues_count": 10291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635466943344644, "text": "@KCDeanGirl @ljwinterh @PradasMinion @s_verasani @JensenAckles This is the funniest shit I will see today. Probably this week. ����", "in_reply_to_status": 715620957331677185, "in_reply_to_user": 512761168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 512761168, 237151547, 2427802808, 2767256731, 2743113943 }}, "user": { "id": 17829124, "name": "Candice ❤'s #SPN11", "screen_name": "clmccallie", "lang": "en", "location": "Probably watching Supernatural", "create_at": date("2008-12-02"), "description": "The world isn't black and white. It's a much more interesting place. Over-thinking, independent weirdo. In awe of the social media power of the #SPNFamily", "followers_count": 167, "friends_count": 439, "statues_count": 2236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chattanooga, TN", "id": "3b3916ee31cfc9e6", "name": "Chattanooga", "place_type": "city", "bounding_box": rectangle("-85.404424,34.983674 -85.080704,35.200235") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635466968522752, "text": "Road construction, right lane closed in #GandyBridge on Gandy Blvd EB between Gandy Brg and Selmon Expy, stop and... https://t.co/KJgf5PdTvw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.56758,27.88338"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GandyBridge" }}, "user": { "id": 249881371, "name": "TT+WN Tampa", "screen_name": "TotalTrafficTPA", "lang": "en", "location": "Tampa, FL", "create_at": date("2011-02-09"), "description": "null", "followers_count": 331, "friends_count": 66, "statues_count": 30279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-03-31T13:22:45.000Z"), "id": 715635467320823809, "text": "https://t.co/coEXAN4b6p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2376666876, "name": "a$htray", "screen_name": "Madbhillin", "lang": "en", "location": "N. Hollywood, Ca", "create_at": date("2014-03-06"), "description": "null", "followers_count": 783, "friends_count": 255, "statues_count": 18844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635467648110593, "text": "literally just met #hilaryclinton ���� she took my phone & everything���� https://t.co/aY7U4flb1e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.703993,41.048744"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hilaryclinton" }}, "user": { "id": 109137309, "name": "Kane S. Montan", "screen_name": "kanesebastian", "lang": "en", "location": "null", "create_at": date("2010-01-27"), "description": "Musical.ly 10k @kanesebastian - shimmur -funimate", "followers_count": 139, "friends_count": 76, "statues_count": 4 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SUNY Purchase", "id": "07d9ce744e881003", "name": "SUNY Purchase", "place_type": "poi", "bounding_box": rectangle("-73.70399309999999,41.0487439 -73.703993,41.048744") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3632402, "cityName": "Harrison" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635467702702082, "text": "���������� https://t.co/yjV0DnSJrD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 216905071, "name": "shawtymane", "screen_name": "shawtyyyy69", "lang": "en", "location": "3☹5", "create_at": date("2010-11-17"), "description": "| ice house inc | glass gardens | 1-800-YOU-WISH ✨SHAWTYMANELAFLARE@GMAIL.COM✨ ✨iCEHOUSEiNC@GMAIL.COM✨~ for all ur quality jewelry needs", "followers_count": 1251, "friends_count": 824, "statues_count": 47111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468163895296, "text": "Interested in a #Engineering #job near #Cincinnati, OH? This could be a great fit: https://t.co/dJQ4CNgQhC #Veterans #Hiring #CareerArc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.5120196,39.1031182"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Engineering", "job", "Cincinnati", "Veterans", "Hiring", "CareerArc" }}, "user": { "id": 23366425, "name": "TMJ-CIN Engin. Jobs", "screen_name": "tmj_cin_eng", "lang": "en", "location": "Cincinnati, OH", "create_at": date("2009-03-08"), "description": "Follow this account for geo-targeted Engineering job tweets in Cincinnati, OH. Need help? Tweet us at @CareerArc!", "followers_count": 313, "friends_count": 227, "statues_count": 99 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468189245440, "text": "*Ally how's that diet going* https://t.co/CgfNNaaTt4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410286318, "name": "Ally Senecal", "screen_name": "Ally_Senecal09", "lang": "en", "location": "null", "create_at": date("2011-11-11"), "description": "null", "followers_count": 774, "friends_count": 579, "statues_count": 25176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manorville, NY", "id": "4d2b1566262ae881", "name": "Manorville", "place_type": "city", "bounding_box": rectangle("-72.865145,40.819884 -72.739139,40.903776") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3645139, "cityName": "Manorville" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468365213696, "text": "@C_rawr14 Jayden made it before me dude ��", "in_reply_to_status": 715634984413835264, "in_reply_to_user": 844816939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 844816939 }}, "user": { "id": 3224668926, "name": "estela", "screen_name": "_itzapizza", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-05-23"), "description": "18. ♑️ ig : a_estelaaa / sc: e.stelaaa", "followers_count": 622, "friends_count": 535, "statues_count": 5560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468574924800, "text": "I'm just studying something right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3396183082, "name": "ph0run", "screen_name": "ph0run", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "hi I like dragon ball and some anime I'm Tom Ford on Facebook as well look me up", "followers_count": 109, "friends_count": 0, "statues_count": 30375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468583325696, "text": "@TTettDuroy miss you", "in_reply_to_status": 715634654942928896, "in_reply_to_user": 2579537874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2579537874 }}, "user": { "id": 3533763079, "name": "Kylee Dungan", "screen_name": "DunganKylee", "lang": "en", "location": "null", "create_at": date("2015-09-11"), "description": "{•I don't know exactly where I am going from here, but I can guarantee it won't be boring•}", "followers_count": 211, "friends_count": 275, "statues_count": 532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yukon, OK", "id": "4370a1ec03a1674a", "name": "Yukon", "place_type": "city", "bounding_box": rectangle("-97.777656,35.478851 -97.706928,35.52241") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40017, "countyName": "Canadian", "cityID": 4082950, "cityName": "Yukon" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468587692034, "text": "Feel better soon! https://t.co/i7toosUvqv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2905430351, "name": "Christian Leonel", "screen_name": "leonel0487", "lang": "en", "location": "Florida, USA", "create_at": date("2014-12-04"), "description": "-Singer, Nature, Life!", "followers_count": 68, "friends_count": 245, "statues_count": 567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468633841664, "text": "I want all of Kylie's glosses #like #literally #socute @kyliecosmetics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "like", "literally", "socute" }}, "user_mentions": {{ 4038362244 }}, "user": { "id": 279576357, "name": "Kimberly Childers", "screen_name": "kimberlaayjo", "lang": "en", "location": "null", "create_at": date("2011-04-09"), "description": "Seriously, don't take everything so seriously.", "followers_count": 301, "friends_count": 362, "statues_count": 5015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haddon Heights, NJ", "id": "d0e0f06490a8e595", "name": "Haddon Heights", "place_type": "city", "bounding_box": rectangle("-75.085432,39.869303 -75.046058,39.889882") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3428800, "cityName": "Haddon Heights" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468642025472, "text": "Congrats to Oli and the boys. That's exciting ������ https://t.co/s8mJ7vktZJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343862852, "name": "Murry Laumau", "screen_name": "SkurryMurry_23", "lang": "en", "location": "760CEANSIDE ", "create_at": date("2011-07-27"), "description": "TT & Elijah❤️ sc- murrybitxh", "followers_count": 1560, "friends_count": 1083, "statues_count": 116220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635468965117952, "text": "Live and learn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301443629, "name": "Kashman", "screen_name": "Kashews10", "lang": "en", "location": "null", "create_at": date("2011-05-19"), "description": "Soundcloud: kashrules_everything, snapchat: kashrule_s, Instagram: Sheem", "followers_count": 133, "friends_count": 298, "statues_count": 2260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635469397180417, "text": "@frankibaldini @kkayleighh_410 @cmarkabawi15 @ Cheryl Norton, we have some problems", "in_reply_to_status": 715635232158912516, "in_reply_to_user": 1639713481, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1639713481, 501208433, 1888489747 }}, "user": { "id": 494626943, "name": "dad", "screen_name": "elijah_frazier4", "lang": "en", "location": "null", "create_at": date("2012-02-16"), "description": "pizza SRU '19", "followers_count": 463, "friends_count": 334, "statues_count": 16369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slippery Rock, PA", "id": "87dd0b38d4a6f883", "name": "Slippery Rock", "place_type": "city", "bounding_box": rectangle("-80.071952,41.038567 -80.031521,41.084549") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler", "cityID": 4271184, "cityName": "Slippery Rock" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635469443276801, "text": "3 years today rip kevin ware's leg that snapped in half��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351958651, "name": "zoe soilis", "screen_name": "ZZZZOEE_S", "lang": "en", "location": "Freeport, PA", "create_at": date("2011-08-09"), "description": "baller on a budget", "followers_count": 660, "friends_count": 522, "statues_count": 8743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freeport, PA", "id": "4a7f3711d4dd0a9b", "name": "Freeport", "place_type": "city", "bounding_box": rectangle("-79.73543,40.669698 -79.673485,40.706161") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42005, "countyName": "Armstrong", "cityID": 4227784, "cityName": "Freeport" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635469531394049, "text": "https://t.co/7TIm61r3Ga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 411956311, "name": "Bae J", "screen_name": "Agecage229", "lang": "en", "location": "Buffalo-Brockport", "create_at": date("2011-11-13"), "description": "Snowboarding Meteorology student who likes cars and golf", "followers_count": 211, "friends_count": 198, "statues_count": 4794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brockport, NY", "id": "01fbdc902515ef96", "name": "Brockport", "place_type": "city", "bounding_box": rectangle("-77.996912,43.192641 -77.89015,43.226323") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608466, "cityName": "Brockport" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635469778718721, "text": "@itsife_love LOL rude and reckless", "in_reply_to_status": 715634427024556032, "in_reply_to_user": 1046142116, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1046142116 }}, "user": { "id": 720136919, "name": "TheWifeOfPablo", "screen_name": "Pink_buggattii", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-17"), "description": "Big ❤️ Forever smiling .... The nicest person you'll ever meet Nomatter what happens in life being good to people is a wonderful legacy to leave behind", "followers_count": 4464, "friends_count": 700, "statues_count": 101450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635469921361920, "text": "https://t.co/NgP2Gt1rsV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1258373544, "name": "Pōnce", "screen_name": "brandonponcee", "lang": "en", "location": "CA, USA", "create_at": date("2013-03-10"), "description": "'95 | TPMB x WC. http://www.TheProcessMusicBlog.com", "followers_count": 217, "friends_count": 126, "statues_count": 5635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470055579648, "text": "グレハン、完凸できたのは嬉しいけど…", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 4885977133, "name": "星ドラひろつん。", "screen_name": "hirotsun7", "lang": "en", "location": "null", "create_at": date("2016-02-09"), "description": "とりあえず無課金でやってます。", "followers_count": 54, "friends_count": 70, "statues_count": 66 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470055686144, "text": "@AAcidland is she yanking his branch", "in_reply_to_status": 715608375694852096, "in_reply_to_user": 1090218493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1090218493 }}, "user": { "id": 3239347282, "name": "gz", "screen_name": "edguygz", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "metal head ,sports fan ,#TWD fan ,horror fan", "followers_count": 1520, "friends_count": 884, "statues_count": 72102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470114414592, "text": "Side effect of smoking weed is hating the Jews. That's why Hitler was born on 4/20. It all makes sense.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2147843972, "name": "Andrew Q. Stookey", "screen_name": "stookmeister", "lang": "en", "location": "NKY/USA", "create_at": date("2013-10-21"), "description": "Author. Runner. Penguin Enthusiast. NKU 2019.", "followers_count": 247, "friends_count": 230, "statues_count": 3075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Heights, KY", "id": "7e3677074757a991", "name": "Highland Heights", "place_type": "city", "bounding_box": rectangle("-84.478667,39.011993 -84.434744,39.054807") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21037, "countyName": "Campbell", "cityID": 2136604, "cityName": "Highland Heights" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470131003392, "text": "the last one ���� https://t.co/gudRPrbhx2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1383112808, "name": "emotional shawty", "screen_name": "yungfijiwater_", "lang": "en", "location": "null", "create_at": date("2013-04-26"), "description": "it's deeper than that jb/pm/t$/ar/kc/rw rip wade rip tae #eblock", "followers_count": 812, "friends_count": 795, "statues_count": 38845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470248579072, "text": "My '44 make sure all your kids don't grow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 710219258, "name": "bambi", "screen_name": "___KissMyClass", "lang": "en", "location": "westend", "create_at": date("2012-07-21"), "description": "$olid.", "followers_count": 4883, "friends_count": 4624, "statues_count": 66445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470269440000, "text": "@landondonovan @cynthianatalie LD is for equal pay b/c fairness=equality", "in_reply_to_status": 715615476261359617, "in_reply_to_user": 85633543, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85633543, 21103938 }}, "user": { "id": 508992733, "name": "Andrew J Ash", "screen_name": "AndrewJAsh", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-02-29"), "description": "Reaganite, bluecollar advocate, soccer, music, philosophy, TX! followed by @RickSantorum All American with Cuban/Irish heritage. Block-walker extraordinaire!", "followers_count": 535, "friends_count": 888, "statues_count": 5059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470370152448, "text": "Accident in #Louisville on 71 NB at Barbour Ln overpass, stop and go traffic back to Blankenbaker Ln Brg, delay of 9 mins #SDFtraffic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.6066,38.3024"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Louisville", "SDFtraffic" }}, "user": { "id": 249827241, "name": "TTWN Louisville", "screen_name": "TotalTrafficSDF", "lang": "en", "location": "Louisville, KY", "create_at": date("2011-02-09"), "description": "null", "followers_count": 515, "friends_count": 10, "statues_count": 18119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hills and Dales, KY", "id": "003eacb12cf11406", "name": "Hills and Dales", "place_type": "city", "bounding_box": rectangle("-85.642921,38.293893 -85.606607,38.314652") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2136865, "cityName": "Hills and Dales" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635470730854400, "text": "@oceanfaerie yaaaaaaas", "in_reply_to_status": 715635291222970369, "in_reply_to_user": 3505449672, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3505449672 }}, "user": { "id": 3479819894, "name": "Conrad Kippes", "screen_name": "conradulation", "lang": "en", "location": "null", "create_at": date("2015-09-07"), "description": "Sometimes I do pretty impressive stuff", "followers_count": 88, "friends_count": 133, "statues_count": 131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodcrest, CA", "id": "4b68a015eaeb9b4f", "name": "Woodcrest", "place_type": "city", "bounding_box": rectangle("-117.418374,33.828971 -117.313882,33.917206") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 686244, "cityName": "Woodcrest" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635471351681024, "text": "@oliver_karlos", "in_reply_to_status": 715635409833836544, "in_reply_to_user": 3150521243, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3150521243 }}, "user": { "id": 3150521243, "name": "KARLOS ANTONY OLIVER", "screen_name": "oliver_karlos", "lang": "en", "location": "New York, NY", "create_at": date("2015-04-08"), "description": "null", "followers_count": 25, "friends_count": 299, "statues_count": 1641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-03-31T13:22:46.000Z"), "id": 715635471754346497, "text": "@mandy_484 thank you so much!!", "in_reply_to_status": 715635402174889984, "in_reply_to_user": 338073568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 338073568 }}, "user": { "id": 298199860, "name": "Bailey Goforth", "screen_name": "BaileyGoforth", "lang": "en", "location": "Somewhere in my own world.", "create_at": date("2011-05-13"), "description": "null", "followers_count": 903, "friends_count": 403, "statues_count": 9630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntersville, NC", "id": "27fbd5412c269208", "name": "Huntersville", "place_type": "city", "bounding_box": rectangle("-80.957062,35.366148 -80.8109,35.454554") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3733120, "cityName": "Huntersville" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635471842455552, "text": "@JakeTrimble7 is finally coming up to the boro tomorrow ����", "in_reply_to_status": -1, "in_reply_to_user": 1912784700, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1912784700 }}, "user": { "id": 2975606919, "name": "Shannon Wagner", "screen_name": "shannon_wagner1", "lang": "en", "location": "null", "create_at": date("2015-01-12"), "description": "Georgia Southern University | Bio Major | JT", "followers_count": 132, "friends_count": 144, "statues_count": 548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635471959855104, "text": "K Michelle new album is my favorite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2284595714, "name": "Melanin✨", "screen_name": "tynirene", "lang": "en", "location": "null", "create_at": date("2014-01-09"), "description": "null", "followers_count": 1131, "friends_count": 850, "statues_count": 7181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472047820800, "text": "Yup Buzzin is still a dope album all these years later thanks for this @shwayze", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7119102 }}, "user": { "id": 227148849, "name": "Don Marquez", "screen_name": "AustinM_Butler", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2010-12-15"), "description": "snapchat: Legendary_A3 #JuiceUp", "followers_count": 1450, "friends_count": 998, "statues_count": 57305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472060407811, "text": "@NatalieeRamoss yeah I think , mine are together though so I just came to med term ��", "in_reply_to_status": 715635307287154688, "in_reply_to_user": 2245752650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2245752650 }}, "user": { "id": 1431776731, "name": "X", "screen_name": "XelesteMelendez", "lang": "en", "location": "null", "create_at": date("2013-05-15"), "description": "✈️ PHS", "followers_count": 2253, "friends_count": 1712, "statues_count": 17359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472098152448, "text": "عيشتي دون وصلت ماعليها حلا \nحسبي الله على صدك ونعم الوكيل", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 431905473, "name": "Ahmad", "screen_name": "a_alkhatlan", "lang": "en", "location": "Pomona, CA ", "create_at": date("2011-12-08"), "description": "instagram:a_alkhatlan", "followers_count": 602, "friends_count": 253, "statues_count": 7085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472178003968, "text": "#mcgreekweek16 banner competition: @atobetarho and @chiomegamc! #mcgreeklife #MariettaCollege @… https://t.co/7Mw1axOHDk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.44901916,39.41698315"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mcgreekweek16", "mcgreeklife", "MariettaCollege" }}, "user_mentions": {{ 2975548175, 517039017 }}, "user": { "id": 2744888738, "name": "Marietta Greek Life", "screen_name": "MCGreekLife", "lang": "en", "location": "Marietta College, Ohio", "create_at": date("2014-08-19"), "description": "Keep up with Fraternity & Sorority Life at @MariettaCollege. Use #MCGreekLife to follow along with @PioneerPanhel and @MariettaIFC chapters. #gogreek", "followers_count": 129, "friends_count": 105, "statues_count": 317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marietta, OH", "id": "33833c3421cbb25e", "name": "Marietta", "place_type": "city", "bounding_box": rectangle("-81.502155,39.377658 -81.395312,39.491007") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39167, "countyName": "Washington", "cityID": 3947628, "cityName": "Marietta" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472337252352, "text": "@_CollegeHumor_ @RomyMorsy", "in_reply_to_status": 715626090568531969, "in_reply_to_user": 724223456, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 724223456, 1479752474 }}, "user": { "id": 307000153, "name": "Plank", "screen_name": "Jake_Plank", "lang": "en", "location": "null", "create_at": date("2011-05-28"), "description": "• Sports Enthusiast • Sneakerhead •", "followers_count": 270, "friends_count": 309, "statues_count": 4034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilkes-Barre, PA", "id": "3fdd3a62ae058de9", "name": "Wilkes-Barre", "place_type": "city", "bounding_box": rectangle("-75.924804,41.215416 -75.826756,41.272993") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4285152, "cityName": "Wilkes-Barre" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472765100032, "text": "To go or not to go to ilovemakonnen tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70039532, "name": "Annie", "screen_name": "anniedets", "lang": "en", "location": "null", "create_at": date("2009-08-29"), "description": "null", "followers_count": 702, "friends_count": 577, "statues_count": 20632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472811175936, "text": "@maddymarkks I was reading your tweets last night and I was dying ������", "in_reply_to_status": 715635328426467328, "in_reply_to_user": 2899835192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2899835192 }}, "user": { "id": 3237278622, "name": "Dominic Martinez", "screen_name": "Dominicmar9", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "Chillin", "followers_count": 256, "friends_count": 225, "statues_count": 577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635472974766080, "text": "Mood: https://t.co/yESWVuJvjB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35631003, "name": "#ChosenOne", "screen_name": "Peti___", "lang": "en", "location": "Silsbee, Texas", "create_at": date("2009-04-26"), "description": "#RIPJakaylaB", "followers_count": 1232, "friends_count": 582, "statues_count": 34109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silsbee, TX", "id": "001e60954611de22", "name": "Silsbee", "place_type": "city", "bounding_box": rectangle("-94.2286,30.32076 -94.141621,30.437904") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48199, "countyName": "Hardin", "cityID": 4867832, "cityName": "Silsbee" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635473234800640, "text": "@lexi_shyanne yessss tell me more", "in_reply_to_status": 715630927104184320, "in_reply_to_user": 3526104497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3526104497 }}, "user": { "id": 1429127684, "name": "Samantha RaVonne", "screen_name": "Sam_Trick15", "lang": "en", "location": "United States", "create_at": date("2013-05-14"), "description": "null", "followers_count": 380, "friends_count": 196, "statues_count": 12541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coos Bay, OR", "id": "00e27ccc4e07ff06", "name": "Coos Bay", "place_type": "city", "bounding_box": rectangle("-124.298278,43.342917 -124.205934,43.411311") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41011, "countyName": "Coos", "cityID": 4115250, "cityName": "Coos Bay" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635473406930944, "text": "@jondaiello I don’t think I played at all last summer, but I’ve been playing on and off since high school.", "in_reply_to_status": 715635356662697984, "in_reply_to_user": 408611390, "favorite_count": 0, "coordinate": point("-84.15361695,39.72125449"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 408611390 }}, "user": { "id": 13075822, "name": "Michael Yockey", "screen_name": "myockey", "lang": "en", "location": "Ohio", "create_at": date("2008-02-04"), "description": "I still like taking things apart. Developer @hearsparkbox", "followers_count": 339, "friends_count": 493, "statues_count": 3195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, OH", "id": "8d742fb555fbff21", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-84.307688,39.695193 -84.093044,39.865523") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3921000, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635473503260672, "text": "No oink for me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2832524640, "name": "justin", "screen_name": "champagnejusti", "lang": "en", "location": "washington state ", "create_at": date("2014-09-25"), "description": "live your best life. ig: fakaleiti", "followers_count": 617, "friends_count": 372, "statues_count": 20039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635473587109888, "text": "@rendeeros562 Arrived new cars bruh", "in_reply_to_status": -1, "in_reply_to_user": 2866928625, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2866928625 }}, "user": { "id": 3666998000, "name": "Patrick Johnson", "screen_name": "pflako7", "lang": "es", "location": "Los Angeles ", "create_at": date("2015-09-23"), "description": "ليث A Bullet Proof Glass! You Can Luve Me Or Hate Me... #MadeinLosAngeles", "followers_count": 2, "friends_count": 8, "statues_count": 90 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635473746624512, "text": "Lehigh Valley Mall to add more upscale outdoor shops?\nhttps://t.co/DpoVGR3f8v https://t.co/V48Ny0zTwA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15431914, "name": "The Morning Call", "screen_name": "mcall", "lang": "en", "location": "Allentown, PA", "create_at": date("2008-07-14"), "description": "We are a newspaper serving the greater Lehigh Valley area in eastern PA. (This account is updated by online producers inside our newsroom) RTs not endorsements", "followers_count": 44528, "friends_count": 4692, "statues_count": 57391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allentown, PA", "id": "cfcf6b96b61e7b11", "name": "Allentown", "place_type": "city", "bounding_box": rectangle("-75.548035,40.547616 -75.419836,40.636034") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4202000, "cityName": "Allentown" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474325344257, "text": "@Real_Foolish @jerryswag80 @saavvvyy_ they started it wtf", "in_reply_to_status": 715635227855441920, "in_reply_to_user": 2551680462, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2551680462, 3196326078, 3290840588 }}, "user": { "id": 2718753705, "name": "Ärīštôtłę", "screen_name": "JamesGibert98", "lang": "en", "location": "justin, TX ", "create_at": date("2014-07-20"), "description": "I play football at northwest", "followers_count": 305, "friends_count": 494, "statues_count": 390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roanoke, TX", "id": "001a980b0bf26339", "name": "Roanoke", "place_type": "city", "bounding_box": rectangle("-97.340667,32.988286 -97.21085,33.049565") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4862504, "cityName": "Roanoke" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474405130242, "text": "Maluma jamás morirá https://t.co/LDLitNxRQV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 27306508, "name": "Paula Chávez", "screen_name": "boomboompau", "lang": "en", "location": "Miami, FL claro que sí", "create_at": date("2009-03-28"), "description": "The coolest amongst us shed light.", "followers_count": 7195, "friends_count": 1602, "statues_count": 42589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474467913729, "text": "Everyone is so angry in this butterfly effect movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 585547934, "name": "マリ", "screen_name": "Maridrawsdotcom", "lang": "en", "location": "California & 台灣", "create_at": date("2012-05-20"), "description": "愛畫畫的女孩 ☆ ENG/中文/日本語 OK! ☆", "followers_count": 468, "friends_count": 235, "statues_count": 8682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474719703041, "text": "“Mistakes are the portals of discovery.” - James Joyce. @AIGAdesign. https://t.co/FWD0ReiGxX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19619047 }}, "user": { "id": 629880674, "name": "Justin Macuga", "screen_name": "askmehowto", "lang": "en", "location": "#Pittsburgh.", "create_at": date("2012-07-07"), "description": "@UCLA_Alumni.", "followers_count": 763, "friends_count": 2184, "statues_count": 26555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensburg, PA", "id": "2d2cc38bfa01da4a", "name": "Greensburg", "place_type": "city", "bounding_box": rectangle("-79.596255,40.262029 -79.485605,40.348027") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42129, "countyName": "Westmoreland", "cityID": 4231200, "cityName": "Greensburg" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474862243841, "text": "Join the Supplemental Health Care team! See our latest #Healthcare #job opening here: https://t.co/tNBwfgQoWF https://t.co/74sUUfNErk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-109.0764828,35.7444602"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 2587789764, "name": "SHC Careers", "screen_name": "WorkWithSHC", "lang": "en", "location": "null", "create_at": date("2014-06-25"), "description": "Work for the Best! Whether you want to work across town or across the country, we have thousands of great health care jobs available at top facilities.", "followers_count": 907, "friends_count": 1, "statues_count": 100089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Defiance, AZ", "id": "01d193800fec82d6", "name": "Fort Defiance", "place_type": "city", "bounding_box": rectangle("-109.084446,35.72354 -109.046154,35.762213") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4001, "countyName": "Apache", "cityID": 424460, "cityName": "Fort Defiance" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635474933555200, "text": "@lolo__monroe for info!", "in_reply_to_status": 715623343831982080, "in_reply_to_user": 172579567, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332343485 }}, "user": { "id": 172579567, "name": "tupaclilsister♏️", "screen_name": "kieralovealways", "lang": "en", "location": "dallas,texas⛽️", "create_at": date("2010-07-29"), "description": "The game is to be sold, not told.. #SFA18", "followers_count": 801, "friends_count": 483, "statues_count": 12840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635475319504897, "text": "@OnlyThePoets @MrFlyyyGuyyy lmao! I'm for it ALL! I was trying to get Bae a scuba suit for Puerto Rico! ������", "in_reply_to_status": 715625951699271681, "in_reply_to_user": 42144390, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42144390, 29973030 }}, "user": { "id": 25644851, "name": "KyleΔlexanderMurray", "screen_name": "murraysmyname", "lang": "en", "location": "on a flight back to Charlotte", "create_at": date("2009-03-20"), "description": "An artistic, witty, mostly silly, social media guy with a stylists touch, locs and a degree, who dreams excessively, but prays much more. kyle@thespreadmag.com", "followers_count": 2580, "friends_count": 676, "statues_count": 176601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-03-31T13:22:47.000Z"), "id": 715635475675947008, "text": "These #STAAR tweets are killing me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "STAAR" }}, "user": { "id": 707945704535171072, "name": "$avanna", "screen_name": "Vannaazz", "lang": "en", "location": "null", "create_at": date("2016-03-10"), "description": "null", "followers_count": 35, "friends_count": 33, "statues_count": 102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795833350959104, "text": "@_smilecassiee didn't see your texttttt got Cha now ��", "in_reply_to_status": 715793630380834817, "in_reply_to_user": 351078155, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351078155 }}, "user": { "id": 258166086, "name": "⚡️", "screen_name": "Darriatenae_", "lang": "en", "location": "va", "create_at": date("2011-02-26"), "description": "null", "followers_count": 1472, "friends_count": 891, "statues_count": 84476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suffolk, VA", "id": "6e0ed207f0c2d36b", "name": "Suffolk", "place_type": "city", "bounding_box": rectangle("-76.668194,36.698303 -76.402862,36.92173") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51800, "countyName": "Suffolk", "cityID": 5176432, "cityName": "Suffolk" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795833682309124, "text": "I low key feel like Jacob retweets certain things bc he knows I lurk on him ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 836697186, "name": "bibi", "screen_name": "ThatChickLisz_", "lang": "en", "location": "Reno, NV - Guadalajara, JAL ", "create_at": date("2012-09-20"), "description": "jacob strickland ❤️", "followers_count": 660, "friends_count": 604, "statues_count": 27365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun Valley, NV", "id": "8f6f3f1941bc1936", "name": "Sun Valley", "place_type": "city", "bounding_box": rectangle("-119.807877,39.556726 -119.732974,39.625005") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3271600, "cityName": "Sun Valley" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795834013683713, "text": "Mix between routine and random as hell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.40185204,29.69332294"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33921088, "name": "..STRangeFRuiT", "screen_name": "DyemonDzN_PeRlz", "lang": "en", "location": "ALPHA. Female ", "create_at": date("2009-04-21"), "description": "I do what I want to do..,I say what I want to say.", "followers_count": 421, "friends_count": 407, "statues_count": 9181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795834395340800, "text": "@Strezzy_ I asked a question, just wanted an answer. That's all ��", "in_reply_to_status": 715795564907073536, "in_reply_to_user": 455730290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 455730290 }}, "user": { "id": 80085508, "name": "Leah B.", "screen_name": "Mc__Leah", "lang": "en", "location": "RIP Buddha|Adam|", "create_at": date("2009-10-05"), "description": "Stay focused", "followers_count": 1046, "friends_count": 1011, "statues_count": 56651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Campbell North, KY", "id": "006cfe05510849b3", "name": "Fort Campbell North", "place_type": "city", "bounding_box": rectangle("-87.514325,36.64013 -87.436613,36.689327") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21047, "countyName": "Christian", "cityID": 2128486, "cityName": "Fort Campbell North" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835305459716, "text": "I have to work in 7 hours. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2354161514, "name": "Kay Renwick. ☼", "screen_name": "_kayxoo", "lang": "en", "location": "Pickerington, OH ", "create_at": date("2014-02-20"), "description": "|| 18 || The sun sees your body, the moon sees your soul. || ☼☯ ||", "followers_count": 330, "friends_count": 312, "statues_count": 19886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pickerington, OH", "id": "3995cc1483801d24", "name": "Pickerington", "place_type": "city", "bounding_box": rectangle("-82.797752,39.841431 -82.684335,39.939034") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3962498, "cityName": "Pickerington" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835376762880, "text": "@hlluang @W0RLDSTARHlPHOP it does look like him", "in_reply_to_status": 715794344121409536, "in_reply_to_user": 2347304058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2347304058, 1390969753 }}, "user": { "id": 564778428, "name": "Willie P", "screen_name": "WillPerkins3", "lang": "en", "location": "Ruston, LA", "create_at": date("2012-04-27"), "description": "We in the prime of our life, supposed to be having the time of our life.", "followers_count": 459, "friends_count": 397, "statues_count": 7597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835435528192, "text": "Temp 61.8° Hi/Lo 63.2/61.6 Rng 1.6° WC 61.6° Hmd 91% Rain 0.01\" Storm 0.00\" BAR 29.755 Falling DP 59.1° Wnd 7mph Dir W Gst 24mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every 20 min. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 64, "friends_count": 116, "statues_count": 17335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835519377409, "text": "�������� cause these girls ain't intertaining me https://t.co/TJz1EDHrmW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1485396523, "name": "SNUPE", "screen_name": "YvngZachh", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-06-05"), "description": "Artist|Actor ☆ DM [V] | ApeLife / TribeShxt | I GOT A LIVIN TESTIMONY", "followers_count": 1473, "friends_count": 582, "statues_count": 17232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835573964801, "text": "@jovanni_12 you're trippin man, this is why we aren't friends no more.", "in_reply_to_status": 715795740728102912, "in_reply_to_user": 2826766756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2826766756 }}, "user": { "id": 2869645760, "name": "LC.", "screen_name": "LEISTEEZEY_", "lang": "en", "location": "wherever you're not", "create_at": date("2014-10-21"), "description": "life messier than mud pies.", "followers_count": 558, "friends_count": 446, "statues_count": 26933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795835645202432, "text": "@papiblythe hoWDY pArTNeR ;-)))", "in_reply_to_status": 715795452764012545, "in_reply_to_user": 2284180302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2284180302 }}, "user": { "id": 598519285, "name": "erin", "screen_name": "erin_corrine", "lang": "en", "location": "x • xvii • xv", "create_at": date("2012-06-03"), "description": "totally lacking the wow factor // it's a fabulous life, mr. whale emoji.", "followers_count": 288, "friends_count": 386, "statues_count": 4705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sahuarita, AZ", "id": "010e9b9269df1e78", "name": "Sahuarita", "place_type": "city", "bounding_box": rectangle("-111.004339,31.895159 -110.964233,32.001439") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 462140, "cityName": "Sahuarita" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795836114964480, "text": "@ReannaSpoon you can't do an entire week without me ��", "in_reply_to_status": 715795430819434496, "in_reply_to_user": 439740434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 439740434 }}, "user": { "id": 305227817, "name": "Giselle", "screen_name": "MissGiselle_H", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-05-25"), "description": "Too blessed to be stressed", "followers_count": 426, "friends_count": 330, "statues_count": 17130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795836295385088, "text": "'Cause nobody in his right mind would've left her. I had to be crazy to say goodbye ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28865723, "name": "August Zaragoza", "screen_name": "augustzaragoza", "lang": "en", "location": "Fort Smith, AR", "create_at": date("2009-04-04"), "description": "Find yourself and be good at it. SC: augustzaragoza {keep it simple}", "followers_count": 1312, "friends_count": 1396, "statues_count": 33018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Smith, AR", "id": "a5cb06fbef643420", "name": "Fort Smith", "place_type": "city", "bounding_box": rectangle("-94.435514,35.276033 -94.305102,35.438472") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5131, "countyName": "Sebastian", "cityID": 524550, "cityName": "Fort Smith" } }
+{ "create_at": datetime("2016-04-01T00:00:00.000Z"), "id": 715795836731584513, "text": "I hate that my notifications are off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1054203631, "name": "MEERKAT", "screen_name": "jvaldez2669", "lang": "en", "location": "El Monte, CA", "create_at": date("2013-01-01"), "description": "That's on God", "followers_count": 622, "friends_count": 504, "statues_count": 10781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795837046104065, "text": "Yall need to listen to @nfrealmusic mans too cold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335540047 }}, "user": { "id": 1592154332, "name": "King Mo Money", "screen_name": "Illmatic_Mo", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-07-13"), "description": "Snapchat: Mc_Illmatic hip hop artist|Instagram: MC_illmatic!| | Bay Area| Arizona| Virgo 20 | Party/Event Planner| Promoter | ASU | Entrepreneur", "followers_count": 702, "friends_count": 597, "statues_count": 17957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795837176135681, "text": "@Ordinary_Chickk @britttanyyyyyy ������ what u told him", "in_reply_to_status": 715795739411132417, "in_reply_to_user": 505119430, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 505119430, 706641535853596672 }}, "user": { "id": 292940164, "name": "NeishaFineA$$.", "screen_name": "_yourstruuly_", "lang": "en", "location": "cuttin' up w dreka fine azz❤️", "create_at": date("2011-05-04"), "description": "For anybody else that's saying don't forget you, my advice is that you just be unforgettable ✨. 18YearsYoung, ClassOf2016.", "followers_count": 1831, "friends_count": 1370, "statues_count": 95979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bogalusa, LA", "id": "0d86f1118064d85f", "name": "Bogalusa", "place_type": "city", "bounding_box": rectangle("-89.896103,30.746749 -89.833306,30.812685") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22117, "countyName": "Washington", "cityID": 2208150, "cityName": "Bogalusa" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795837264244736, "text": "@baseballfan1324 I wish it was that easy", "in_reply_to_status": 715795046734413824, "in_reply_to_user": 863141028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 863141028 }}, "user": { "id": 628887834, "name": "Sam G", "screen_name": "Saaam_Gorski", "lang": "en", "location": "The Dirty C", "create_at": date("2012-07-06"), "description": "always give more than you take | @Mas_Stachowiak ❤️", "followers_count": 739, "friends_count": 350, "statues_count": 20606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cudahy, WI", "id": "484f13f6c61b96d0", "name": "Cudahy", "place_type": "city", "bounding_box": rectangle("-87.88769,42.929977 -87.842893,42.966744") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5517975, "cityName": "Cudahy" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795837465587716, "text": "I need a new fuckin job", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315050376, "name": "6age", "screen_name": "GageTheGreatt", "lang": "en", "location": "null", "create_at": date("2011-06-10"), "description": "#LVL\nWell there goes our security deposit.", "followers_count": 1047, "friends_count": 786, "statues_count": 79062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sahuarita, AZ", "id": "010e9b9269df1e78", "name": "Sahuarita", "place_type": "city", "bounding_box": rectangle("-111.004339,31.895159 -110.964233,32.001439") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 462140, "cityName": "Sahuarita" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795837645889538, "text": "@Indihaitian need u to make sure im up lmao, im writing this paper but im dumb tired ���� lmao just like call my phone before u go to sleep ��", "in_reply_to_status": 715795657261494273, "in_reply_to_user": 46586125, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46586125 }}, "user": { "id": 301769299, "name": "#Kaizen", "screen_name": "_eugenewalls_", "lang": "en", "location": "The Town ✈ The City", "create_at": date("2011-05-19"), "description": "A chance. Its all I need. |#DC3Football| SHiNE . Avant Garde. - PrayForMerakol", "followers_count": 1146, "friends_count": 1121, "statues_count": 46927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dodge City, KS", "id": "1b49a9a5d4fd7baf", "name": "Dodge City", "place_type": "city", "bounding_box": rectangle("-100.056008,37.724925 -99.964975,37.792451") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20057, "countyName": "Ford", "cityID": 2018250, "cityName": "Dodge City" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795838056980481, "text": "@zodiak_iller yeeeeeah! ������������", "in_reply_to_status": 715768853024911361, "in_reply_to_user": 3065194896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3065194896 }}, "user": { "id": 318164922, "name": "LAMEBOT", "screen_name": "LAMEBOTMUSIC", "lang": "en", "location": "Miami, FL", "create_at": date("2011-06-15"), "description": "Bass Music | US Booking: Booking@lamebotsucks.com | UK & Abroad: http://universaltalents.com/artist.php?lamebot", "followers_count": 1591, "friends_count": 362, "statues_count": 22842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795838182760448, "text": "Happy New Year", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576436876, "name": "Jerome Cadiz", "screen_name": "JeromeExists", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "Save life. Realizer | Runner | Wannabe Model | '80s Blood | Dancer | Believer", "followers_count": 123, "friends_count": 107, "statues_count": 1718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795838338002944, "text": "นี่ขนาดโดนเบลอเอาไว้ก่อนกูยังสัมผัสได้ถึงความมุ้งมิ้งขององค์ชาย10ทะลุมาเลยค่ะ ตัวเล็กอะไรขนาดนั้นนนนนนนนนนน;ㅅ; https://t.co/Qo0yd5G0Oy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user": { "id": 1410288966, "name": "peterpan;'ㅅ'", "screen_name": "p_papannn", "lang": "en", "location": "McKinney, TX |", "create_at": date("2013-05-07"), "description": "上癮网络剧", "followers_count": 1609, "friends_count": 696, "statues_count": 76497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839034269696, "text": "We're good incles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2749764130, "name": "andrea", "screen_name": "majesticincle", "lang": "en", "location": "null", "create_at": date("2014-08-21"), "description": "Girl Groups. Winner.", "followers_count": 767, "friends_count": 220, "statues_count": 30254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839063613441, "text": "it's already 12? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2275145155, "name": "alizah", "screen_name": "alizahh__", "lang": "en", "location": "at practice or frosties ", "create_at": date("2014-01-03"), "description": "null", "followers_count": 720, "friends_count": 447, "statues_count": 15057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839084576769, "text": "I hella thought I was tight wit that ringtone tho ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316515134, "name": "-Kiara❤️", "screen_name": "__lilki", "lang": "en", "location": "Jonesboro, LA", "create_at": date("2011-06-13"), "description": "longliveSnupe", "followers_count": 1524, "friends_count": 448, "statues_count": 35516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, LA", "id": "7a305189b509c584", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-92.726864,32.212426 -92.686115,32.260875") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22049, "countyName": "Jackson", "cityID": 2238670, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839109742593, "text": "All smiles ✌️������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 245843590, "name": "Omg its Stacee☺️", "screen_name": "omg_stace", "lang": "en", "location": "chicago ", "create_at": date("2011-02-01"), "description": "(I -Am) Stacee ! God is first", "followers_count": 294, "friends_count": 436, "statues_count": 3319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839185207296, "text": "Wind 0.0 mph SSE. Barometer 29.729 in, Rising. Temperature 49.2 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839185235968, "text": "#Trump in 2016 Temp:60.6°F Wind:2.5mph Pressure: 29.62hpa Falling slowly Rain Today 0.00in. Forecast: Precipitation at times, very unsettl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 313973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839239766016, "text": "ouuu jesus fix it pls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2403617873, "name": "cat", "screen_name": "catlaluv", "lang": "en", "location": "somewhere getting high ", "create_at": date("2014-03-11"), "description": "get high get high get high✨", "followers_count": 484, "friends_count": 463, "statues_count": 14951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laplace, LA", "id": "005beffd77be6ac9", "name": "Laplace", "place_type": "city", "bounding_box": rectangle("-90.519583,30.031013 -90.435378,30.105989") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2242030, "cityName": "Laplace" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839382339584, "text": "One time for my LA hoess https://t.co/rXHgsINZQ4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.253807,33.212039"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408925611, "name": "sandwi̤̮ch", "screen_name": "kyleeocampo", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-11-09"), "description": "EXPRESS YOURSELF", "followers_count": 2030, "friends_count": 136, "statues_count": 25013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles Ave", "id": "07d9e79ccf881000", "name": "Los Angeles Ave", "place_type": "poi", "bounding_box": rectangle("-117.25380709999999,33.212038899999996 -117.253807,33.212039") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839453642752, "text": "@YRRLFATV_JJ can he even rap ?��", "in_reply_to_status": 715780203293908992, "in_reply_to_user": 351288639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351288639 }}, "user": { "id": 240118778, "name": "WZRD K3lly", "screen_name": "Kfost3", "lang": "en", "location": "I'm off on an Adventure", "create_at": date("2011-01-18"), "description": "My DNA is DMT I'm so rare", "followers_count": 544, "friends_count": 583, "statues_count": 14346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawrence, IN", "id": "08e2b37735301da6", "name": "Lawrence", "place_type": "city", "bounding_box": rectangle("-86.055608,39.827737 -85.93803,39.913058") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1842426, "cityName": "Lawrence" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839482986496, "text": "April fools �� #itsonPhil", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "itsonPhil" }}, "user": { "id": 1159151298, "name": "Ghino Esteban", "screen_name": "ghinohidalgo", "lang": "en", "location": "Chula Vista, CA", "create_at": date("2013-02-07"), "description": "Eastlake HS | 1997 | Eastlake Dance Co. | EAM Captain | 24/7 SD | #SQ7 | NDMA", "followers_count": 200, "friends_count": 132, "statues_count": 1942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839495577601, "text": "I wish all good things could last but unfortunately that just isn't how life works out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2346903594, "name": "ariana miesse ☼", "screen_name": "theofficial_Ari", "lang": "en", "location": "unc charlotte", "create_at": date("2014-02-16"), "description": "high key sassy with a bulletproof logic ¯\\_(ツ)_/¯", "followers_count": 604, "friends_count": 615, "statues_count": 9406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839747276800, "text": "@ebbtideapp Tide in Elbow Bay, Alaska 04/01/2016\n Low 1:17am 4.7\nHigh 7:21am 10.7\n Low 2:34pm 1.7\nHigh 9:24pm 9.8\n Low 2:49am 4.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-132.65,54.9"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 46, "friends_count": 1, "statues_count": 20677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795839894036480, "text": "@marjoriesba @jeannemfl @TerryMcAuliffe @TFFVA @GovernorVA -#pro-Choice must prevail-thank you TM", "in_reply_to_status": 715208718556733440, "in_reply_to_user": 522526229, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pro" }}, "user_mentions": {{ 522526229, 237872656, 19471123, 18147727, 104198706 }}, "user": { "id": 21765409, "name": "shereelee21", "screen_name": "LennardSbkohut", "lang": "en", "location": "null", "create_at": date("2009-02-24"), "description": "SVP Media Sales, World Health Networks, the game changing product innovator of non-invasive, biometric health screening device products and consumer insights", "followers_count": 571, "friends_count": 2068, "statues_count": 4910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840066068484, "text": "ain't a good night until some beef is goin down on twitter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1121108960, "name": "Lilbug ♡", "screen_name": "PriincesLilly", "lang": "en", "location": "pnw 360 ", "create_at": date("2013-01-25"), "description": "twitter addict from the upper left of the USA #ANGELa #KonnerJHouse", "followers_count": 847, "friends_count": 341, "statues_count": 28270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840124760065, "text": "Missin' Slicky a little extra tonight��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1412051988, "name": "Lexa", "screen_name": "Lexarlara", "lang": "en", "location": "Monmouth, OR", "create_at": date("2013-05-07"), "description": "Everyones favorite baby • @2BigSlick❤️", "followers_count": 1563, "friends_count": 508, "statues_count": 36932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840166662144, "text": "I feel so bad for Zoe rn :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 809815417, "name": "Megan Johnson", "screen_name": "baconnandmegs", "lang": "en", "location": "210/806", "create_at": date("2012-09-07"), "description": "Texas Tech ΔΓ • insta/snapchat: @meganjohnsonn", "followers_count": 440, "friends_count": 224, "statues_count": 23470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840485486593, "text": "Happy mother fucking birthday to my pimp @BaeSick_Mariah ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301228562 }}, "user": { "id": 390451514, "name": "Jailene Rizo", "screen_name": "Jayyy_diizzlle", "lang": "en", "location": "Selma, CA", "create_at": date("2011-10-13"), "description": "Ya tu sabes Yeezy 2020", "followers_count": 680, "friends_count": 665, "statues_count": 21252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840573579265, "text": ".@adamfgoldberg I apologize for being late to the party, but \"The Goldbergs\" is absolutely fantastic. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236930610 }}, "user": { "id": 22836586, "name": "Michael Malone", "screen_name": "MaloneComedy", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-04"), "description": "Award Winning Comedian/Director | FOX TV| WGN|ChoppingBlock| Bethlehem| SICC Winner| Best Selling iTunes Artist| HOT Comic to Watch 2013| XM Radio| Dish Nation|", "followers_count": 6819, "friends_count": 1831, "statues_count": 30756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840686759936, "text": "04/01@03:00 - Temp 67.7F, WC 67.7F. Wind 5.0mph SSW, Gust 10.0mph. Bar 29.756in, Falling slowly. Rain 0.00in. Hum 88%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840875540480, "text": "Me bc that direct deposit hit ������ https://t.co/OAXTxEIukk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389749590, "name": "Lexi", "screen_name": "LexiNelson__", "lang": "en", "location": "Joliet", "create_at": date("2011-10-12"), "description": "It's a beautiful day to save lives || 18 || JWHS", "followers_count": 536, "friends_count": 305, "statues_count": 16159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shorewood, IL", "id": "0a15199a1ef555b1", "name": "Shorewood", "place_type": "city", "bounding_box": rectangle("-88.254057,41.492184 -88.178709,41.545149") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1769758, "cityName": "Shorewood" } }
+{ "create_at": datetime("2016-04-01T00:00:01.000Z"), "id": 715795840909107200, "text": "Hashtag salty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 751585933, "name": "Bailey", "screen_name": "baileeeeys", "lang": "en", "location": "Texas", "create_at": date("2012-08-11"), "description": "null", "followers_count": 521, "friends_count": 277, "statues_count": 31180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795841294999553, "text": "I should kms", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3526784301, "name": "Steven Hernandez", "screen_name": "steviebruh", "lang": "en", "location": "null", "create_at": date("2015-09-02"), "description": "PHS '19 // instagram: stevie.bruh // freshman are friends, not food", "followers_count": 90, "friends_count": 156, "statues_count": 492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottage Grove, MN", "id": "8e24cdcee62a4347", "name": "Cottage Grove", "place_type": "city", "bounding_box": rectangle("-92.98399,44.800931 -92.903515,44.855344") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27163, "countyName": "Washington", "cityID": 2713456, "cityName": "Cottage Grove" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795841416622080, "text": "https://t.co/sBT8UA7U6M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 717284456, "name": "Escobar De La Verga", "screen_name": "chrissydawg_219", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "University of Lightskin '19", "followers_count": 667, "friends_count": 428, "statues_count": 22688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kearny, NJ", "id": "d5140ca0d08eadc7", "name": "Kearny", "place_type": "city", "bounding_box": rectangle("-74.164089,40.717135 -74.076066,40.786287") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436510, "cityName": "Kearny" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795841542447104, "text": "@sammfenz deadass", "in_reply_to_status": 715774152230232066, "in_reply_to_user": 24467602, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24467602 }}, "user": { "id": 251223593, "name": "Khalil edwards", "screen_name": "_lilmatic", "lang": "en", "location": "null", "create_at": date("2011-02-12"), "description": "null", "followers_count": 643, "friends_count": 634, "statues_count": 8023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, PA", "id": "d88a5def1d7e9609", "name": "Indiana", "place_type": "city", "bounding_box": rectangle("-79.213942,40.586024 -79.089163,40.656614") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42063, "countyName": "Indiana", "cityID": 4236816, "cityName": "Indiana" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795841781510152, "text": "70.2F (Feels: 70.2F) - Humidity: 99% - Wind: 3.1mph SE - Gust: 3.8mph - Pressure: 1003.6mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 233210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795843044020224, "text": "Wind 0.0 mph W. Barometer 29.655 in, Rising slowly. Temperature 54.8 °F. Rain today 0.23 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 56, "friends_count": 27, "statues_count": 18765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795843215986688, "text": "Temp: 72.5°F Wind:0.4mph Pressure: 29.771hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 66, "friends_count": 24, "statues_count": 59934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795843455066112, "text": "@LJ__2305 Yes �� I currently have 0 words ��", "in_reply_to_status": 715795112387862528, "in_reply_to_user": 3634102572, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3634102572 }}, "user": { "id": 1358484840, "name": "Arnulfo Hernandez", "screen_name": "arnulfo__79", "lang": "en", "location": "null", "create_at": date("2013-04-16"), "description": "Live everyday like it's your last.", "followers_count": 100, "friends_count": 190, "statues_count": 583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-01T00:00:02.000Z"), "id": 715795844168081409, "text": "It's a crazy life, but I'm juuusssttt fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1663295839, "name": "kam", "screen_name": "kammeegonzalez", "lang": "en", "location": "null", "create_at": date("2013-08-11"), "description": "you just wasted your time reading this", "followers_count": 915, "friends_count": 923, "statues_count": 27817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795845342474240, "text": "https://t.co/lNcrwfLwfQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1276671728, "name": "mamí", "screen_name": "princessovsass", "lang": "en", "location": "null", "create_at": date("2013-03-17"), "description": "281 to my city", "followers_count": 393, "friends_count": 104, "statues_count": 25943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795845413801985, "text": "People be calling me ugly. I post a selfie and get compliments because I'm not ugly, the same people call me thirsty. K.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24982298, "name": "Hoe-Jo", "screen_name": "Mariah_Cara", "lang": "en", "location": "Home of James Dean", "create_at": date("2009-03-17"), "description": "Cara. 22. Sir Hoe. Womanist. Aspie. Hoosier. Hip-Hop and culture critic. LGBQTIA+. #BlackLivesMatter. Disability activist. Mariah Carey. Babies. Puppies. Cats.", "followers_count": 1541, "friends_count": 986, "statues_count": 106338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795845531189248, "text": "I'm gonna be people. Y'all be cool!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103937832, "name": "Deondre", "screen_name": "Deondre017", "lang": "en", "location": "Kansas City, MO", "create_at": date("2010-01-11"), "description": "I'm just here to add more stamps to my passport, and keep my beard connected. #BeardGameStrong #PassportGameStrong", "followers_count": 386, "friends_count": 336, "statues_count": 61323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795845686419456, "text": "I'm so thankful for my mom������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1691266512, "name": "anahi", "screen_name": "Anahiii_mtz", "lang": "en", "location": "jan'06.16❤️", "create_at": date("2013-08-22"), "description": "null", "followers_count": 792, "friends_count": 452, "statues_count": 10858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795845728305153, "text": "������ https://t.co/7YgnPA7jX8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 917673133, "name": "atciii", "screen_name": "itsatcifooo", "lang": "en", "location": "null", "create_at": date("2012-10-31"), "description": "19 biotchessss", "followers_count": 480, "friends_count": 885, "statues_count": 23222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846072242177, "text": "Nobody Better April Fools Prank Me Cause I Get Mad Fast lmao Shits Childish Cause I'm Gullible ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075818284, "name": "not papí", "screen_name": "ZapotecaRob", "lang": "en", "location": "criptorville", "create_at": date("2013-01-09"), "description": "shs c/o 2016 | #FUCKCANCER | Employed & Enlisted", "followers_count": 544, "friends_count": 322, "statues_count": 28230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846269444098, "text": "best movie ever https://t.co/rTLKnrLo1Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1115831736, "name": "nicki minaj", "screen_name": "nichole_gennock", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2013-01-23"), "description": "I just love Alec Martinez", "followers_count": 925, "friends_count": 428, "statues_count": 48206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846344929281, "text": "Wind 4.2 mph SSE. Barometer 29.464 in, Falling. Temperature 62.8 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846365855745, "text": "I also provide the voice of numerous other characters you may be familiar with.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317652275, "name": "Problematic_Smith", "screen_name": "Pauncho_Smith", "lang": "en", "location": "The Banana Republic of Florida", "create_at": date("2011-06-15"), "description": "A Classic Clusterfuck", "followers_count": 88, "friends_count": 343, "statues_count": 23723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846697193472, "text": "@stnkmsterflex I would have helped.", "in_reply_to_status": 715791799227383808, "in_reply_to_user": 286993737, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 286993737 }}, "user": { "id": 8649652, "name": "Hashtag Slut", "screen_name": "xthree", "lang": "en", "location": "Hermosa Beach", "create_at": date("2007-09-04"), "description": "#JudgeMe #ZiveZife #TeamCali #ForeverChi", "followers_count": 321, "friends_count": 629, "statues_count": 27848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermosa Beach, CA", "id": "4baf4d09759d33e4", "name": "Hermosa Beach", "place_type": "city", "bounding_box": rectangle("-118.409579,33.85185 -118.384866,33.877823") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 633364, "cityName": "Hermosa Beach" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846747582464, "text": "Get ur head out of ur ass and give me the explanation I deserve like FUCK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 706455129, "name": "keylee", "screen_name": "Keyleeshayee", "lang": "en", "location": "front row ", "create_at": date("2013-10-09"), "description": "Spalding University - concert junkie - whatever", "followers_count": 405, "friends_count": 270, "statues_count": 9526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795846877601794, "text": "Smoking my pet out when I get one. La nigga gone be looking like Chief keef dog messing with me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632633278, "name": "Rika", "screen_name": "XOLove_RikaXO", "lang": "en", "location": "Louisiana, USA", "create_at": date("2012-07-10"), "description": "Let's Kreate and be perfect. #LSU", "followers_count": 2800, "friends_count": 1541, "statues_count": 60425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795847020154880, "text": "@yuhgurlnya https://t.co/qFyMKJ0uoI", "in_reply_to_status": -1, "in_reply_to_user": 3379868937, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3379868937 }}, "user": { "id": 877786944, "name": "taylor ❤️", "screen_name": "trillestcurls_", "lang": "en", "location": "Laurel, MD", "create_at": date("2012-10-13"), "description": "poppin in md | #VirginAffairsPromo", "followers_count": 1818, "friends_count": 1398, "statues_count": 25358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Laurel, MD", "id": "0270fb8d47bab72f", "name": "South Laurel", "place_type": "city", "bounding_box": rectangle("-76.885346,39.033137 -76.810791,39.096875") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2473650, "cityName": "South Laurel" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795847041191938, "text": "Good morning Nicole ✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3508500494, "name": "Celeste", "screen_name": "pepsicolaqueen_", "lang": "en", "location": "North Las Vegas, NV", "create_at": date("2015-09-09"), "description": "I require a lot of attention", "followers_count": 263, "friends_count": 282, "statues_count": 2851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795847749959680, "text": "Ripley SW Limestone Co. Temp: 64.9°F Wind:2.2mph Pressure: 985.7mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 44, "friends_count": 33, "statues_count": 51213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795848077123584, "text": "Wtf..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 363677747, "name": "jorrd", "screen_name": "Jorrd_", "lang": "en", "location": "Johnstown-Slippery Rock", "create_at": date("2011-08-28"), "description": "☼❁☾| SRU 19' | ΔZ", "followers_count": 1533, "friends_count": 1666, "statues_count": 21446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slippery Rock, PA", "id": "87dd0b38d4a6f883", "name": "Slippery Rock", "place_type": "city", "bounding_box": rectangle("-80.071952,41.038567 -80.031521,41.084549") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42019, "countyName": "Butler", "cityID": 4271184, "cityName": "Slippery Rock" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795848387567616, "text": "lemme stop ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 930907632, "name": "ka$h b", "screen_name": "pettyassiyanah", "lang": "en", "location": "7$7", "create_at": date("2012-11-06"), "description": "Kaedyn & Bailee ❤️", "followers_count": 1350, "friends_count": 246, "statues_count": 18848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, VA", "id": "754f6c33c758f636", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-76.969533,36.647507 -76.909734,36.708694") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51620, "countyName": "Franklin", "cityID": 5129600, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795849247326210, "text": "My thoughts on abortion are... Let's all have a good time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2707241119, "name": "Ryn McHaggis", "screen_name": "TheRuckus_", "lang": "en", "location": "The Internet", "create_at": date("2014-08-04"), "description": "My safe word is Topanga", "followers_count": 27, "friends_count": 63, "statues_count": 644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naperville, IL", "id": "3227bcceb76b6a37", "name": "Naperville", "place_type": "city", "bounding_box": rectangle("-88.259598,41.665584 -88.063045,41.831408") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1751622, "cityName": "Naperville" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795849272537088, "text": "@MSLisaChang no idea. That was his first and only message. No pic. No response.", "in_reply_to_status": 715795744058392577, "in_reply_to_user": 47731709, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47731709 }}, "user": { "id": 5994222, "name": "iconick™", "screen_name": "NICKWALSH", "lang": "en", "location": "Sherman Oaks, CA", "create_at": date("2007-05-12"), "description": "snapchat: toepick. Hillary Clinton's #1 fan. #risePOTUS. Writing the Pepper Potts movie, 1 tweet at a time. I teach selfie classes at the annex.", "followers_count": 4471, "friends_count": 817, "statues_count": 57048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-01T00:00:03.000Z"), "id": 715795849356427265, "text": "She gave in I give up...and we just live in the moment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101897996, "name": "dixie rose", "screen_name": "rosedixie", "lang": "en", "location": "Trinidad/New York", "create_at": date("2010-01-04"), "description": "I don't mind being hated, but I hate being misunderstood.....♓️Pisces!", "followers_count": 1351, "friends_count": 1526, "statues_count": 44850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795849478021120, "text": "RIP Zaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 430317878, "name": "Brooks Caton", "screen_name": "B_Caton82", "lang": "en", "location": "College Station", "create_at": date("2011-12-06"), "description": "chme time is the best time", "followers_count": 291, "friends_count": 256, "statues_count": 4666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795849499041793, "text": "I'm not your friend ���� https://t.co/GaSzghQWDY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 892445437, "name": "cardigan queen", "screen_name": "isawthewolff", "lang": "en", "location": "Austin, TX", "create_at": date("2012-10-19"), "description": "Somebody stop me", "followers_count": 464, "friends_count": 516, "statues_count": 10827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795849993981952, "text": "Damn, these my shits https://t.co/MG1lwcDYBo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560316958, "name": "Alexa Mescudi", "screen_name": "alexapadilla80", "lang": "en", "location": "Chicago", "create_at": date("2014-06-10"), "description": "stay ✨ CUDFAM UIC '19 Beauty IG: beautyxalexa IG: alexa_mescudi", "followers_count": 1113, "friends_count": 425, "statues_count": 21449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795850027483136, "text": "You guys have no idea how much i love Alicia keys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543502510, "name": "Jade", "screen_name": "xanaxnymph", "lang": "en", "location": "null", "create_at": date("2012-04-02"), "description": "LA ✈️PORTLAND .STAY TUNED", "followers_count": 315, "friends_count": 107, "statues_count": 33111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795850249814018, "text": "@beehoopsx15 sucia ��", "in_reply_to_status": 715794050046099456, "in_reply_to_user": 3285153253, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3285153253 }}, "user": { "id": 390615099, "name": "Charles Two Bears", "screen_name": "LOStheBB", "lang": "en", "location": "San Gabriel Valley,California ", "create_at": date("2011-10-14"), "description": "Mexican-Native American|PT/Strength & Conditioning Coach|History Educator|Motivational Speaker|Poet|Aesthetics/MMA|Changing lives 1 day @ a time!", "followers_count": 469, "friends_count": 828, "statues_count": 3860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795850589519872, "text": "Can't sleep. #insomniac #wip #sacredgeometry #sacredgeometryart #geometry #geometric… https://t.co/if1IEjVC6t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.62476717,41.71644897"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "insomniac", "wip", "sacredgeometry", "sacredgeometryart", "geometry", "geometric" }}, "user": { "id": 19216755, "name": "J.R. Avaritt", "screen_name": "JIMBOBWE", "lang": "en", "location": "Toledo,Ohio", "create_at": date("2009-01-19"), "description": "Tattooed, Cigar Smoking, Geocaching, Artist,Painter,Paperhanger,Coffee Addict,Nature Junkie. #addictedtoclouds", "followers_count": 1768, "friends_count": 2071, "statues_count": 9541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795850786643973, "text": "You hoes can keep yo fake feelings. . . We don't need yo sympathy ✋������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310655866, "name": "Poochie ☄", "screen_name": "lovemedwn_", "lang": "en", "location": "HTX ", "create_at": date("2011-06-03"), "description": "AJones8:7❤ Mesha12:12✨ Swift12:29❤️ #WakeEmUp", "followers_count": 2898, "friends_count": 2065, "statues_count": 98617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795851092824064, "text": "@asiraM__ what one ?", "in_reply_to_status": 715795784847998976, "in_reply_to_user": 326537195, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326537195 }}, "user": { "id": 750043393, "name": "✨", "screen_name": "kesharnahorne", "lang": "en", "location": "#FreeTravis✨", "create_at": date("2012-08-10"), "description": "collegeFRESHMAN | employed‼️", "followers_count": 1273, "friends_count": 710, "statues_count": 47795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795851159986177, "text": "Happy birthday to the best Spanish buddy out there! Have a great day I love you! ������ @lolaolalere https://t.co/y9WzTkSKjN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2182337959 }}, "user": { "id": 3222551563, "name": "Rachel Rash", "screen_name": "rachel_rash", "lang": "en", "location": "null", "create_at": date("2015-05-21"), "description": "Disneyland| Soccer| Friends", "followers_count": 299, "friends_count": 324, "statues_count": 1816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795851931688960, "text": "Jordan pic got me so weak����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2321577769, "name": "Red❣", "screen_name": "_Samira2", "lang": "en", "location": "sc sami_dinkins", "create_at": date("2014-01-31"), "description": "Money is the motive", "followers_count": 895, "friends_count": 674, "statues_count": 21324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795852007227393, "text": "Howell, NJ | Wind 2.0 mph SSW. Baro 29.749 in, Falling. Temp 63.9F. Rain today 0.00 in. Humidity 92% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 621, "friends_count": 816, "statues_count": 39896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795852195995655, "text": "It's been released into the wild that is my garden okay cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2952318994, "name": "Laura Elizabeth", "screen_name": "cndrblck_grdn", "lang": "en", "location": "NYC", "create_at": date("2014-12-30"), "description": "Music, reading, writing, art, and nutella obsessed teenage band girl.", "followers_count": 213, "friends_count": 424, "statues_count": 2150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795852216893440, "text": "Kevin Lynn Reiman, Manassas, VA, 20109 - https://t.co/2zJccy6Lw7 #kevinlynnreiman #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.513961,38.786861"), "retweet_count": 0, "lang": "fi", "is_retweet": false, "hashtags": {{ "kevinlynnreiman", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 250, "friends_count": 695, "statues_count": 99 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Gate, VA", "id": "552db388e30f5030", "name": "West Gate", "place_type": "city", "bounding_box": rectangle("-77.51417,38.770372 -77.473057,38.794403") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5111230, "cityName": "Bull Run" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795852971917312, "text": "@teenytailz faves ����✨", "in_reply_to_status": 715436578366050304, "in_reply_to_user": 3198365509, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3198365509 }}, "user": { "id": 2535017543, "name": "Rasta☯Junes", "screen_name": "rastajunes", "lang": "en", "location": "home grown, not flown", "create_at": date("2014-05-08"), "description": "yung metro trusts me.", "followers_count": 306, "friends_count": 222, "statues_count": 8482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nanakuli, HI", "id": "78ba3e8c91201199", "name": "Nanakuli", "place_type": "city", "bounding_box": rectangle("-158.176921,21.368987 -158.125777,21.404694") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1553900, "cityName": "Nanakuli" } }
+{ "create_at": datetime("2016-04-01T00:00:04.000Z"), "id": 715795853399756804, "text": "18 https://t.co/chlJg2uUSH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2187137935, "name": "Braden", "screen_name": "Killembraden", "lang": "en", "location": "Norf Tucson ", "create_at": date("2013-11-10"), "description": "Relax homie, it's a bad day not a bad life.", "followers_count": 700, "friends_count": 538, "statues_count": 7755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795853752012801, "text": "@SophiaHelwani I couldn't have ever imagined having to choose between them.", "in_reply_to_status": 715771308999581697, "in_reply_to_user": 2927284436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2927284436 }}, "user": { "id": 18431199, "name": "haiLife", "screen_name": "haiLife", "lang": "en", "location": "San Diego", "create_at": date("2008-12-28"), "description": "null", "followers_count": 65, "friends_count": 101, "statues_count": 193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795854263717890, "text": "I want my tattoo already ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2843302693, "name": "❄E.Jayy❄", "screen_name": "Thiinlee_22", "lang": "en", "location": "Deming, NM", "create_at": date("2014-10-07"), "description": "R.I.P Spooky Ray #DirtySouth #StayUp", "followers_count": 601, "friends_count": 368, "statues_count": 9718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deming, NM", "id": "82c42a788b149dfc", "name": "Deming", "place_type": "city", "bounding_box": rectangle("-107.786105,32.236025 -107.702888,32.286845") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35029, "countyName": "Luna", "cityID": 3520270, "cityName": "Deming" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795854695776257, "text": "@a_man_in_yellow Also 30K is \"Not that much money.\"", "in_reply_to_status": 715792112277700608, "in_reply_to_user": 1531368576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1531368576 }}, "user": { "id": 114584062, "name": "Mac Daddy Maggie", "screen_name": "Battybuddy", "lang": "en", "location": "Seattle WA", "create_at": date("2010-02-15"), "description": "One day, very soon, you're going to wake up and find your face has been ripped off... and the last thing you'll hear in this life will be my laughter...", "followers_count": 1781, "friends_count": 1270, "statues_count": 93077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795854737743873, "text": "I'm so tired being 2nd class in everything.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62087040, "name": "Victor Garcia", "screen_name": "Victorrito", "lang": "en", "location": "East Troy, WI", "create_at": date("2009-08-01"), "description": "Freshman at Stevens Point, Double music major, Point Rugby, Phi Mu Alpha Sinfonia.", "followers_count": 172, "friends_count": 258, "statues_count": 1887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevens Point, WI", "id": "c0b44c42d36404e3", "name": "Stevens Point", "place_type": "city", "bounding_box": rectangle("-89.60726,44.493975 -89.491635,44.586049") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55097, "countyName": "Portage", "cityID": 5577200, "cityName": "Stevens Point" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795854855114752, "text": "17f85a6000p12543", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.389599,53.380188"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 15, "friends_count": 0, "statues_count": 10735 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Garden River No. 490, Saskatchewan", "id": "4f008915ee56e675", "name": "Garden River No. 490", "place_type": "city", "bounding_box": rectangle("-105.543536,53.229401 -105.073015,53.453164") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795855022899201, "text": "@aaronrobinson69 must be nice having long hair", "in_reply_to_status": 715794953289539584, "in_reply_to_user": 3311930754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3311930754 }}, "user": { "id": 743871115, "name": "Gabe", "screen_name": "GJCintheOKC", "lang": "en", "location": "null", "create_at": date("2012-08-07"), "description": "OCCC Bound athlete #CivicGang", "followers_count": 127, "friends_count": 241, "statues_count": 5475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795855320752128, "text": "@HannahReanneP_ damn my forehead phat as fuck lmaoooo fuck a 4-head I got that 10-head��������������", "in_reply_to_status": 714918198324424705, "in_reply_to_user": 2428300777, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2428300777 }}, "user": { "id": 2428300777, "name": "Its Hanns Bitch.", "screen_name": "HannahReanneP_", "lang": "en", "location": "G Shiit", "create_at": date("2014-04-04"), "description": "null", "followers_count": 875, "friends_count": 797, "statues_count": 10655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulf Hills, MS", "id": "d8573d17f48a61a5", "name": "Gulf Hills", "place_type": "city", "bounding_box": rectangle("-88.84974,30.419425 -88.793304,30.456269") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28059, "countyName": "Jackson", "cityID": 2829620, "cityName": "Gulf Hills" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795856155348995, "text": "���� https://t.co/VL5tvnorR9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 254893248, "name": "B*I*P $*O*D", "screen_name": "1stLADYMONEY", "lang": "en", "location": "null", "create_at": date("2011-02-19"), "description": "SHANICE vs 1st LADY ALL SHE KNW IS WIN", "followers_count": 280, "friends_count": 372, "statues_count": 3195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795856256008192, "text": "Wind 2.2 mph S. Barometer 29.58 in, Falling. Temperature 62.8 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 158725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795856289636357, "text": "bitch you got me fucked up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256841399, "name": "chel belle ♡", "screen_name": "Itzzmichelle", "lang": "en", "location": "null", "create_at": date("2011-02-23"), "description": "xoxo gossip girl", "followers_count": 1059, "friends_count": 772, "statues_count": 14606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795856616775682, "text": "Popeyes want y'all to be fat.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236699978, "name": "thickums", "screen_name": "mailala_", "lang": "en", "location": "null", "create_at": date("2013-12-21"), "description": "null", "followers_count": 1016, "friends_count": 158, "statues_count": 187835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795857107496960, "text": "@OdalisLeon111 https://t.co/EVRCLbQt9s", "in_reply_to_status": 715794309476401152, "in_reply_to_user": 404523989, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 404523989 }}, "user": { "id": 823287396, "name": "El Chaco Gimenez", "screen_name": "MarcoGee___", "lang": "en", "location": "707 // 916", "create_at": date("2012-09-14"), "description": "Football Is Freedom ⚽️// CSUS", "followers_count": 373, "friends_count": 335, "statues_count": 16024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795857208176640, "text": "Hangin' out, Manhattan Beach @ Manhattan Beach Pier https://t.co/WSNb20ei1m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.41361,33.88374"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21664357, "name": "Mikki Ansin", "screen_name": "maggya", "lang": "en", "location": "Cambridge, MA usa", "create_at": date("2009-02-23"), "description": "photographer filmmaker traveler adventurer reader skier swimmer", "followers_count": 549, "friends_count": 914, "statues_count": 1880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-04-01T00:00:05.000Z"), "id": 715795857581408257, "text": "@jordynnconlonn let's do it", "in_reply_to_status": 715795437756751872, "in_reply_to_user": 2548655696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2548655696 }}, "user": { "id": 289050451, "name": "Rami", "screen_name": "RamiMichelle", "lang": "en", "location": "null", "create_at": date("2011-04-27"), "description": "snapchat- ramimichelle", "followers_count": 1215, "friends_count": 376, "statues_count": 20259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell Canyon, CA", "id": "014fc73a72a07ae5", "name": "Bell Canyon", "place_type": "city", "bounding_box": rectangle("-118.712222,34.19484 -118.667705,34.21607") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 604938, "cityName": "Bell Canyon" } }
+{ "create_at": datetime("2016-04-01T00:00:06.000Z"), "id": 715795857971539969, "text": "#Hospitality #Job alert: Cook - PM Cook / Bob Evans Express | HMSHost | #Columbus, OH https://t.co/MS3RegAcW0 #restaurantlife #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.9987942,39.9611755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "Job", "Columbus", "restaurantlife", "Jobs", "Hiring" }}, "user": { "id": 701776772057141248, "name": "HMSHost Jobs", "screen_name": "HMSHostCareers", "lang": "en", "location": "null", "create_at": date("2016-02-22"), "description": "HMSHost is part of the world’s largest provider of food & beverage services for travelers. Explore our various hourly and management positions in US & Canada!", "followers_count": 210, "friends_count": 199, "statues_count": 1912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158221178634241, "text": "@blitznaomz I know I won't do Santa Barbara because their majors are too limited for me, if I do Davis they gave me extra grant money", "in_reply_to_status": 716157616217456640, "in_reply_to_user": 1302002431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1302002431 }}, "user": { "id": 89602870, "name": "Roxanna Castañeda", "screen_name": "nerdyroxy", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2009-11-12"), "description": "I'm a spicy Latina.", "followers_count": 403, "friends_count": 393, "statues_count": 12595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158221375791104, "text": "You the one I think about all day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3270702337, "name": "BrazyRach❤️", "screen_name": "babyraachaael", "lang": "en", "location": "bay area ", "create_at": date("2015-07-06"), "description": "AyePlaya", "followers_count": 348, "friends_count": 261, "statues_count": 15856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Lorenzo, CA", "id": "3656cfbea78908af", "name": "San Lorenzo", "place_type": "city", "bounding_box": rectangle("-122.161568,37.661213 -122.106552,37.685884") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668112, "cityName": "San Lorenzo" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158221430480896, "text": "He reminds me of a female troll hiding under the bridge ������ https://t.co/5Je4512flo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223205844, "name": "Nick Schnabel", "screen_name": "Nicks_era", "lang": "en", "location": "Talbott Tn.", "create_at": date("2010-12-05"), "description": "I'm the one that has to die when it's time for me to die, so let me live my life, the way I want to.\n\n-Jimi Hendrix", "followers_count": 323, "friends_count": 199, "statues_count": 15921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport, TN", "id": "9ac8cf3351eefab7", "name": "Newport", "place_type": "city", "bounding_box": rectangle("-83.282821,35.920355 -83.129391,35.991106") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47029, "countyName": "Cocke", "cityID": 4753000, "cityName": "Newport" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158221585518593, "text": "I'm always the one who's heartbroken in the end. Always. That's not fair", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 349459783, "name": "killer titties", "screen_name": "NiggaDawwwg", "lang": "en", "location": "Tucson", "create_at": date("2011-08-05"), "description": "18.", "followers_count": 1261, "friends_count": 701, "statues_count": 112524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Drexel Heights, AZ", "id": "19b90ef8d6e28eab", "name": "Drexel Heights", "place_type": "city", "bounding_box": rectangle("-111.097562,32.09131 -111.006161,32.176145") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 420540, "cityName": "Drexel Heights" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158221795209218, "text": "Wow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 509554810, "name": "Josh", "screen_name": "JoshGreeeen", "lang": "en", "location": "Houston", "create_at": date("2012-02-29"), "description": "SC Joshmj23green Hungry for Greatness Yall gone know my name. Student Athlete (Louisiana Breed)", "followers_count": 392, "friends_count": 370, "statues_count": 8220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158222604877824, "text": "A little piece of the #Opry was on stage with @carrieunderwood tonight.��#ACMs #ACMParty https://t.co/NRDV2f18aN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Opry", "ACMs", "ACMParty" }}, "user_mentions": {{ 386244525 }}, "user": { "id": 19772559, "name": "Grand Ole Opry", "screen_name": "opry", "lang": "en", "location": "Nashville, TN", "create_at": date("2009-01-30"), "description": "See The Show That Made Country Music Famous", "followers_count": 413613, "friends_count": 1504, "statues_count": 18377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, NV", "id": "37d88f13e7a85f14", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-115.173994,36.128077 -115.083699,36.144748") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3284600, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158222688583680, "text": "@WaubonsieBand #WVNOLA Dancing with the owner of Palm Court Cafe. �� https://t.co/thyP0midrm", "in_reply_to_status": -1, "in_reply_to_user": 2368488048, "favorite_count": 0, "coordinate": point("-90.059229,29.960871"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WVNOLA" }}, "user_mentions": {{ 2368488048 }}, "user": { "id": 59547160, "name": "Gregory Kontos", "screen_name": "gkontos642", "lang": "en", "location": "null", "create_at": date("2009-07-23"), "description": "null", "followers_count": 33, "friends_count": 84, "statues_count": 76 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Court Jazz Cafe", "id": "07d9f31a86083000", "name": "Palm Court Jazz Cafe", "place_type": "poi", "bounding_box": rectangle("-90.0592291,29.9608709 -90.059229,29.960871") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158222898302977, "text": "Js. https://t.co/SVyI53Iy11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1026948174, "name": "Markayla.", "screen_name": "_queendenaaye", "lang": "en", "location": "null", "create_at": date("2012-12-21"), "description": "Luke Thomas . Maaseiah Lael .", "followers_count": 670, "friends_count": 548, "statues_count": 18826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158223208857600, "text": "@jaeteez_ lmfaoooooo", "in_reply_to_status": 716157934846263297, "in_reply_to_user": 184160028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184160028 }}, "user": { "id": 483624765, "name": "lexx✨", "screen_name": "Eleexiss_xOxO", "lang": "en", "location": "getting dis money.", "create_at": date("2012-02-04"), "description": "always glo'n up I'm never glo'n down ✨", "followers_count": 1078, "friends_count": 934, "statues_count": 100353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Greenwich, CT", "id": "004c65550e85ca55", "name": "Old Greenwich", "place_type": "city", "bounding_box": rectangle("-73.585481,41.000823 -73.554537,41.05948") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 956900, "cityName": "Old Greenwich" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158224534253569, "text": "I've got problem, you've got a problem why don't we solve together?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223979785, "name": "toxic 卑劣な人", "screen_name": "drawtoxic", "lang": "en", "location": "chicago", "create_at": date("2010-12-07"), "description": "infrequent babbles of a dangerous mind", "followers_count": 225, "friends_count": 384, "statues_count": 5620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158224618037248, "text": "Bro where your rhythm at �� https://t.co/o7EcIAwwBo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 563541795, "name": "NO⭐️RUBE", "screen_name": "Rubanzo11", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "AZ✈️IA. Tough times don't last but tough people do.", "followers_count": 774, "friends_count": 593, "statues_count": 7393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valencia West, AZ", "id": "0172552d7a380653", "name": "Valencia West", "place_type": "city", "bounding_box": rectangle("-111.16794,32.104267 -111.086119,32.155128") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 478760, "cityName": "Valencia West" } }
+{ "create_at": datetime("2016-04-02T00:00:00.000Z"), "id": 716158224676720640, "text": "https://t.co/Qzibsqz5GZ I miss ot5 so much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2537737320, "name": "milly", "screen_name": "saucymilly", "lang": "en", "location": "XO", "create_at": date("2014-05-31"), "description": "null", "followers_count": 1727, "friends_count": 238, "statues_count": 62384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158224827813888, "text": "@tofurose ily", "in_reply_to_status": 716157938927185920, "in_reply_to_user": 621655024, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 621655024 }}, "user": { "id": 2962926910, "name": "neeners", "screen_name": "ninanevillee", "lang": "en", "location": "San Clemente, CA", "create_at": date("2015-01-05"), "description": "I have nothing to say", "followers_count": 700, "friends_count": 767, "statues_count": 3702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Juliet, TN", "id": "165269430f88d842", "name": "Mount Juliet", "place_type": "city", "bounding_box": rectangle("-86.583643,36.12784 -86.456775,36.315578") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47189, "countyName": "Wilson", "cityID": 4750780, "cityName": "Mount Juliet" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158225020628996, "text": "���� https://t.co/UDHQ9mURRs", "in_reply_to_status": 716140082336497664, "in_reply_to_user": 3100440998, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3100440998, "name": "Lil Aaron", "screen_name": "TheKingAJC", "lang": "en", "location": "Louisiana ✈️ Texas ", "create_at": date("2015-03-20"), "description": "R.I.P. Chris 8-29-13 #TJC17 Football", "followers_count": 1381, "friends_count": 1282, "statues_count": 1657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158225196908545, "text": "Temp 52.1° Hi/Lo 55.8/52.1 Rng 3.7° WC 52.1° Hmd 56% Rain 0.00\" Storm 0.09\" BAR 29.635 Falling DP 36.9° Wnd 0mph Dir --- Gst 12mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every 20 min. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 64, "friends_count": 116, "statues_count": 17412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158225456959488, "text": "@EASPORTS_MUT someone please buy my 96 tj yeldon he's cheapest one up there at 85k the closest is 99k", "in_reply_to_status": -1, "in_reply_to_user": 405027739, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 405027739 }}, "user": { "id": 379920750, "name": "Billy gib", "screen_name": "B_SAXY_G", "lang": "en", "location": "maryland", "create_at": date("2011-09-25"), "description": "Fairy tales do not tell children the dragons exist. Children already know that dragons exist. Fairy tales tell children the dragons can be killed.\n\nG.K. Chester", "followers_count": 203, "friends_count": 534, "statues_count": 10710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethesda, MD", "id": "864ff125241f172f", "name": "Bethesda", "place_type": "city", "bounding_box": rectangle("-77.158594,38.940225 -77.078411,39.022449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158225582665728, "text": "หน้านี้ที่อยากเห็นมานานมากกกกก หน้ายันๆแบบนี้แหละ!!!!!!!!!! https://t.co/yYUZWsKnbo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user": { "id": 4329647598, "name": "许馨玲 ♡\\(*ˊᗜˋ*)/♡緑高", "screen_name": "ShinyaNightSky", "lang": "th", "location": "Washington, USA", "create_at": date("2015-11-22"), "description": "Hi~สวัสดีจร้ายินดีที่รู้จักเน้อ ฟอลได้ ทักได้ รีได้จ๊ะ はじめまして。よろしくね 〜 私はタイ人です。アニメとまんがが大好き lไทย l English l日本語がすこししかはなせません |English speaker as my second language", "followers_count": 86, "friends_count": 744, "statues_count": 6206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226006282240, "text": "����. ���� https://t.co/JA7HxvunT7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 93240564, "name": "Faithonna.➰", "screen_name": "Faaithslurp", "lang": "en", "location": "null", "create_at": date("2009-11-28"), "description": "8teenaf dime // Been that thang since 97'", "followers_count": 1213, "friends_count": 1000, "statues_count": 28024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Grove, IL", "id": "0010043e0083ea4a", "name": "Sugar Grove", "place_type": "city", "bounding_box": rectangle("-88.489185,41.747143 -88.375384,41.805351") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1773391, "cityName": "Sugar Grove" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226077761536, "text": "Wind 1.0 mph NW. Barometer 29.986 in, Steady. Temperature 43.8 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226174197760, "text": "����Hanging out with tha Spaghetti's now oh yeah! Crustie hangers on ������ bring le pain oh it's… https://t.co/b11oOFyn6B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.7639,30.2672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249966449, "name": "veganskullsnbones", "screen_name": "clydegovegan", "lang": "en", "location": "Austin, TX", "create_at": date("2011-02-09"), "description": "Clyde&Yorkie✌️tweet food music movies culture robots records books spells teas &tales #vegansofinstagram", "followers_count": 625, "friends_count": 1082, "statues_count": 4662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226623045633, "text": "04/02@03:00 - Temp 62.0F, WC 62.0F. Wind 0.0mph N, Gust 1.0mph. Bar 29.643in, Falling slowly. Rain 0.00in. Hum 67%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226807463939, "text": "@TolarianCollege \n\nThere's an honourable, take-no-bullshit quality to you I admire.", "in_reply_to_status": 716157166306983936, "in_reply_to_user": 2442432306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2442432306 }}, "user": { "id": 219909729, "name": "Miranda Mei", "screen_name": "FairyFatale", "lang": "en", "location": "San Francisco", "create_at": date("2010-11-25"), "description": "小百合。Founder of the SF Lady Planeswalkers. Fairy Stepmother. Pixel Pixie. Neurodiverse homemaker, educator, and gamer. Married to @al2der. Plural. Poly. Pan.", "followers_count": 89, "friends_count": 75, "statues_count": 2136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158226941632514, "text": "@ebbtideapp Tide in Barren Island, Maryland 04/02/2016\n Low 3:51am 0.1\nHigh 10:16am 1.4\n Low 5:09pm 0.2\nHigh 10:45pm 1.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-76.265,38.3417"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 48, "friends_count": 1, "statues_count": 20958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24019, "countyName": "Dorchester" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158227247939584, "text": "That concert was fucking nasty, I've never been happier ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457688177, "name": "christina", "screen_name": "followda_l3ader", "lang": "en", "location": "Margaritaville", "create_at": date("2012-01-07"), "description": "I'd rather be in Philly with my cats", "followers_count": 466, "friends_count": 358, "statues_count": 11780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158227289874432, "text": "@morg_hayes you are trippin fam https://t.co/OQTvK265AO", "in_reply_to_status": 715974796287336449, "in_reply_to_user": 116264260, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 116264260 }}, "user": { "id": 404267977, "name": "Malik Ojuri", "screen_name": "Malik_a_Freak", "lang": "en", "location": "New York | Indiana", "create_at": date("2011-11-03"), "description": "@bsu_taus. BSU '17. Professional Selling. My pep talk turn into a pep rally. @youngthug's stylist.", "followers_count": 1231, "friends_count": 1081, "statues_count": 15906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muncie, IN", "id": "01c14352f8d6ca6e", "name": "Muncie", "place_type": "city", "bounding_box": rectangle("-85.461887,40.121825 -85.320813,40.272656") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18035, "countyName": "Delaware", "cityID": 1851876, "cityName": "Muncie" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158227382018049, "text": "#Trump in 2016 Temp:48.6°F Wind:0.9mph Pressure: 29.82hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158227553984512, "text": "I HAVE 1 MINUTE LEFT TO PRANK SOMEONE FCKU!!!!!!!!?!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139515825, "name": "aye bay bay", "screen_name": "BaileyGosling", "lang": "en", "location": "null", "create_at": date("2010-05-02"), "description": "legend in the making", "followers_count": 262, "friends_count": 110, "statues_count": 8938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SeaTac, WA", "id": "c8b06a459cc8f78a", "name": "SeaTac", "place_type": "city", "bounding_box": rectangle("-122.326102,47.396426 -122.266875,47.48874") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5362288, "cityName": "SeaTac" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158228254498816, "text": "There is a shift coming.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711197438, "name": "iii", "screen_name": "tiers_yeux", "lang": "en", "location": "casa grande ", "create_at": date("2012-07-22"), "description": "null", "followers_count": 375, "friends_count": 156, "statues_count": 15291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casa Grande, AZ", "id": "fbb3d1e41acab043", "name": "Casa Grande", "place_type": "city", "bounding_box": rectangle("-111.791608,32.858246 -111.670779,32.992892") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 410530, "cityName": "Casa Grande" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158228338376704, "text": "flex up & win.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99314963, "name": "Darrion.", "screen_name": "DarrionPollard", "lang": "en", "location": "Gilmer, TX - Houston, TX", "create_at": date("2009-12-25"), "description": "@DezPollard8 // 4️⃣Nem // RiceU '16", "followers_count": 1809, "friends_count": 618, "statues_count": 21871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-02T00:00:01.000Z"), "id": 716158228409819136, "text": "No, the water looks dirty https://t.co/vELuUE8UW0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711901735, "name": "Gladiator", "screen_name": "willie_romero", "lang": "en", "location": "null", "create_at": date("2012-07-22"), "description": "Family", "followers_count": 633, "friends_count": 575, "statues_count": 25393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West New York, NJ", "id": "8479859139338fb7", "name": "West New York", "place_type": "city", "bounding_box": rectangle("-74.023591,40.778186 -73.99818,40.796544") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3479610, "cityName": "West New York" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229558919168, "text": "Bri called me a bad bittchh, I love her ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496178689, "name": "emily nicole", "screen_name": "emilync_", "lang": "en", "location": "Big Spring, TX", "create_at": date("2012-02-18"), "description": "tsunami mob", "followers_count": 738, "friends_count": 1209, "statues_count": 16258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Spring, TX", "id": "06997f7cb5324da0", "name": "Big Spring", "place_type": "city", "bounding_box": rectangle("-101.530226,32.203834 -101.354511,32.286026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48227, "countyName": "Howard", "cityID": 4808236, "cityName": "Big Spring" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229617586177, "text": "sabes que vale verga cuando te dicen que no quieren nada de nadie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 700836227482714112, "name": "sofia cano", "screen_name": "sofiacanojc", "lang": "es", "location": "Cananea, Sonora & Tucson Az", "create_at": date("2016-02-19"), "description": "null", "followers_count": 48, "friends_count": 59, "statues_count": 234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229810536448, "text": "I know you're pretty close to nat and we should srsly talk sometime!! Btw ur so pretty ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3383531653, "name": "C. SHIN", "screen_name": "clairebelleee", "lang": "en", "location": "null", "create_at": date("2015-08-29"), "description": "null", "followers_count": 256, "friends_count": 223, "statues_count": 2620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229827297282, "text": "It's my birthday ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 398351680, "name": "Robles", "screen_name": "andresrobles9", "lang": "en", "location": "Arizona ", "create_at": date("2011-10-25"), "description": "Philippians 4:13", "followers_count": 557, "friends_count": 195, "statues_count": 16203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229948932096, "text": "my dog, Oscar, got out around 2:30 this morning, behind Main Street. if you see him contact me please, thank you. https://t.co/IgeVfu1wQS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.482726,39.578915"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1863676596, "name": "Camryn Bradford", "screen_name": "camrynblake__", "lang": "en", "location": "null", "create_at": date("2013-09-14"), "description": "null", "followers_count": 438, "friends_count": 191, "statues_count": 1494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Main Street Grill", "id": "07d9f3ac2dc81000", "name": "Main Street Grill", "place_type": "poi", "bounding_box": rectangle("-86.4827261,39.5789149 -86.482726,39.578915") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18109, "countyName": "Morgan", "cityID": 1850580, "cityName": "Monrovia" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158229978460160, "text": "Temp: 54.2°F Wind:4.1mph Pressure: 29.957hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 66, "friends_count": 24, "statues_count": 60030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158230053855232, "text": "@TheAtlantic ...Elmo Lincoln had it all... https://t.co/OFEudQDhzy", "in_reply_to_status": 716071654565273600, "in_reply_to_user": 35773039, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35773039 }}, "user": { "id": 4071458352, "name": "Aceairdog", "screen_name": "Aceairdog11", "lang": "en", "location": "null", "create_at": date("2015-10-30"), "description": "null", "followers_count": 45, "friends_count": 136, "statues_count": 180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158230670532608, "text": "Brazy https://t.co/KqnZi5KsbZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316924203, "name": "RellChapoJr", "screen_name": "luhrell2x", "lang": "en", "location": "Stl", "create_at": date("2011-06-13"), "description": "quiet wit it i just ride wit it. Rip fred.", "followers_count": 1340, "friends_count": 695, "statues_count": 24489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belleville, IL", "id": "01d4e7421ef07326", "name": "Belleville", "place_type": "city", "bounding_box": rectangle("-90.099191,38.4682 -89.905307,38.59106") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1704845, "cityName": "Belleville" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231102423040, "text": "@bbygirlbetho_ ���������� watch out lul bihh �� bet bye see you Monday", "in_reply_to_status": 716157420217577472, "in_reply_to_user": 709453217977864192, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 709453217977864192 }}, "user": { "id": 267870177, "name": "Kayla ❣", "screen_name": "ZhaneCreates", "lang": "en", "location": "somewhere buying makeup ", "create_at": date("2011-03-17"), "description": "#PVAMU18 #Htown closer to my dreams ❤️ #fortheloveofDRAKE #ripuncleray 9:14 #glitterqueen", "followers_count": 2750, "friends_count": 1473, "statues_count": 117127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231136047104, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake SP--Birding Trail - 2016-04-01 08:09 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8489448,31.4967181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13495 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231152877569, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake SP--Birding Trail - 2016-04-01 08:04 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8489448,31.4967181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13495 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231152877570, "text": "1 White-throated Sparrow (Zonotrichia albicollis) - Santa Cruz River--Palo Parado Rd - 2016-03-31 13:20 https://t.co/m3XsLOTrMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.0165882,31.5309397"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13495 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz", "cityID": 460180, "cityName": "Rio Rico" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231161266177, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake State Park - 2016-04-01 08:00 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8566971,31.4951992"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13495 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231257694208, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake State Park - 2016-04-01 07:28 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8566971,31.4951992"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13495 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231702290433, "text": "Wind 0.7 mph SW. Barometer 29.69 in, Falling slowly. Temperature 42.8 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 34, "friends_count": 118, "statues_count": 158749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231702347776, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 122757785, "name": "Crystal", "screen_name": "crystalnico1e", "lang": "en", "location": "South Alabama ", "create_at": date("2010-03-13"), "description": "instagrammm crystalnico1e @SwisherSweeets twin babymama @MorganBogolin", "followers_count": 2388, "friends_count": 1405, "statues_count": 70317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231718993921, "text": "@4bands_ HAPPY BIRTHDAY ����❣", "in_reply_to_status": -1, "in_reply_to_user": 2524252772, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2524252772 }}, "user": { "id": 1918536402, "name": "LaniBandz➰", "screen_name": "alanikiaraa", "lang": "en", "location": "riddgggeee", "create_at": date("2013-09-29"), "description": "Queen Lani ✨", "followers_count": 1479, "friends_count": 852, "statues_count": 41004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231807000576, "text": "I got a bad habit at replying back and saving numbers ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587148099, "name": "Josh U Whaaaa?", "screen_name": "_ChichiBoy", "lang": "en", "location": "san antonio, Tx", "create_at": date("2012-05-21"), "description": "Twenty20 || IG: chachivalles || SC: chachivalles", "followers_count": 1148, "friends_count": 1752, "statues_count": 16016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158231974948864, "text": "Wind 3.0 mph W. Barometer 29.849 in, Falling slowly. Temperature 45.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 56, "friends_count": 27, "statues_count": 18778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232281161728, "text": "1 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake SP--Birding Trail - 2016-03-31 08:30 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8489448,31.4967181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13500 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232289550337, "text": "1 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake SP--Birding Trail - 2016-03-31 09:15 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8489448,31.4967181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13500 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232373432320, "text": "Cute as fuck farm boy ���������� https://t.co/ig33pMqJwn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4306, "friends_count": 3118, "statues_count": 45498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232486625280, "text": "1 Black-capped Gnatcatcher (Polioptila nigriceps) - Patagonia Lake State Park - 2016-03-31 07:28 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8566971,31.4951992"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13500 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232520040448, "text": "52.9F (Feels: 52.9F) - Humidity: 78% - Wind: 11.4mph N - Gust: 11.4mph - Pressure: 1013.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 233359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232532635650, "text": "Rihanna is one sexy ass bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555527394, "name": "Teddi", "screen_name": "teddibearwoood", "lang": "en", "location": "null", "create_at": date("2012-04-16"), "description": "@caiitlinwinn", "followers_count": 705, "friends_count": 615, "statues_count": 4815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, IA", "id": "00bd9eff10d959e9", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-93.707382,41.467999 -93.654531,41.512103") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19181, "countyName": "Warren", "cityID": 1957675, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232591523840, "text": "1 White-throated Sparrow (Zonotrichia albicollis) - Patagonia Lake SP--Birding Trail - 2016-03-30 07:29 https://t.co/m3XsLOTrMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8489448,31.4967181"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13500 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232733954048, "text": "Went through a whole lot of obstacles .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228101868, "name": "Miguel", "screen_name": "Hvlfbaked", "lang": "en", "location": "The Ville ✈ Houston", "create_at": date("2010-12-18"), "description": "null", "followers_count": 868, "friends_count": 376, "statues_count": 15393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-02T00:00:02.000Z"), "id": 716158232918495233, "text": "fav if up and wanna chat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 739492460, "name": "ㅤㅤㅤ", "screen_name": "____ConJ____", "lang": "en", "location": "Lehi, UT", "create_at": date("2012-08-05"), "description": "you dont know", "followers_count": 936, "friends_count": 419, "statues_count": 3122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lehi, UT", "id": "ed3b23f667186d1f", "name": "Lehi", "place_type": "city", "bounding_box": rectangle("-111.914775,40.357876 -111.818332,40.456264") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4944320, "cityName": "Lehi" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158233388392448, "text": "Lmaoo niggas hacked @hurricanechris6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 366830630 }}, "user": { "id": 39193120, "name": "pablo", "screen_name": "BoloSaidIt", "lang": "en", "location": "CA", "create_at": date("2009-05-10"), "description": "#Team5iveStar !!!! #NoLimitSoldier", "followers_count": 973, "friends_count": 902, "statues_count": 111446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Point, GA", "id": "dfb4df427ea8a2d6", "name": "East Point", "place_type": "city", "bounding_box": rectangle("-84.535534,33.609487 -84.415048,33.705685") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1325720, "cityName": "East Point" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158233639985152, "text": "Some sleep would be nice...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2665213344, "name": "Dominique Rodriguez", "screen_name": "dominique_rr", "lang": "en", "location": "null", "create_at": date("2014-07-21"), "description": "Don't trip chocolate chip", "followers_count": 307, "friends_count": 312, "statues_count": 2482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158233656713216, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2865544416, "name": "cece™", "screen_name": "c_e_c_e_6", "lang": "en", "location": "somewhere only we know", "create_at": date("2014-10-19"), "description": "doin my own thing", "followers_count": 709, "friends_count": 893, "statues_count": 8027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158233728196608, "text": "Why am I still out?!? SquadMoves all night long lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 80706814, "name": "Harris (LUEcreative)", "screen_name": "HarrisLue", "lang": "en", "location": "Charlotte, NC", "create_at": date("2009-10-07"), "description": "Graphic Designer for NASCAR, Car Photographer, ECU Alumni, Art Is My Passion... I Love Life! #NASCAR #DRDcrew #ECU #Stilllife #Foxbody #TeamLMR", "followers_count": 1704, "friends_count": 2073, "statues_count": 35861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158233895944192, "text": "@Ravenspoetry333 We r \"driven ...by the horrible feeling that [our cherished] society may cease 2 exist.\" https://t.co/P1qOWk21MB", "in_reply_to_status": -1, "in_reply_to_user": 21962160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21962160 }}, "user": { "id": 1497568382, "name": "Jerome Huyler, PhD.", "screen_name": "huylerje", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2013-06-09"), "description": "Fmr Asst Professor Seton Hall University. Author: Locke in America: The Moral Philosophy of the Founding Era and Everything You Have: The Case Against Welfare.", "followers_count": 5267, "friends_count": 5748, "statues_count": 68855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158234055323648, "text": "Up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 621466482, "name": "Young Escobar", "screen_name": "WeBallll", "lang": "en", "location": "The West Side ", "create_at": date("2012-06-28"), "description": "snapchat: jacobinabors", "followers_count": 1423, "friends_count": 1044, "statues_count": 37161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158234076258304, "text": "@eRaDismiss please bio me baby <3", "in_reply_to_status": -1, "in_reply_to_user": 1465801070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1465801070 }}, "user": { "id": 710238687934554112, "name": "jackie (: (: (: (:", "screen_name": "jackieuriosteui", "lang": "en", "location": "United States", "create_at": date("2016-03-16"), "description": "@mrsdillxn is boo @buttahflyfly is my boo(:@eRadismiss is mine(:", "followers_count": 667, "friends_count": 2495, "statues_count": 1937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naperville, IL", "id": "3227bcceb76b6a37", "name": "Naperville", "place_type": "city", "bounding_box": rectangle("-88.259598,41.665584 -88.063045,41.831408") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1751622, "cityName": "Naperville" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158234369728512, "text": "@RyanSBieber lmfao Ry you drunk as hell ���� I love you sweets ����", "in_reply_to_status": 716138174452523008, "in_reply_to_user": 83714806, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83714806 }}, "user": { "id": 268597316, "name": "hun⚓️", "screen_name": "Hunnnaaa", "lang": "en", "location": "chicago ", "create_at": date("2011-03-18"), "description": "| U of I '19 | CPD lifeguard", "followers_count": 875, "friends_count": 615, "statues_count": 39344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235074400257, "text": "Your lie in April episode 1: the feels are already real ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1613273780, "name": "Dat Ass Nice So I", "screen_name": "lordoftheb00ty", "lang": "en", "location": "trying my hardest ", "create_at": date("2013-07-22"), "description": "the Weeknds little brother ♏️", "followers_count": 880, "friends_count": 711, "statues_count": 7751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orangevale, CA", "id": "d950fc7dc2cd14f9", "name": "Orangevale", "place_type": "city", "bounding_box": rectangle("-121.262498,38.637436 -121.187521,38.718208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 654092, "cityName": "Orangevale" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235120545792, "text": "I want Dutch Bros!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1472116938, "name": "Eunice", "screen_name": "rosalesecinue12", "lang": "en", "location": "Grandview, WA", "create_at": date("2013-05-31"), "description": "17", "followers_count": 903, "friends_count": 461, "statues_count": 12460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grandview, WA", "id": "36360005eeb536a8", "name": "Grandview", "place_type": "city", "bounding_box": rectangle("-119.942937,46.235098 -119.873337,46.281903") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5327925, "cityName": "Grandview" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235217170432, "text": "\"If there were no children on this earth. If someone said all kids are dead. I'd jump off the balcony immediately I'm done\" Michael Jackson", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1217244780, "name": "Michael Jackson", "screen_name": "ToriTinashe", "lang": "en", "location": "11/16/14 1:22 AM", "create_at": date("2013-02-24"), "description": "Still I'll sit here waiting all alone by the phone for you @TinasheButera\n\nMichael Jackson/Tori Kelly/Tinashe/Ariana Grande\n\nSinger/Songwriter", "followers_count": 15108, "friends_count": 9687, "statues_count": 156874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235258978306, "text": "Don't understand why they wouldn't want to get a car. #mindboggling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mindboggling" }}, "user": { "id": 294377808, "name": "Big Boys Daddy", "screen_name": "Lauvao88", "lang": "en", "location": "Santa Ana", "create_at": date("2011-05-06"), "description": "Happily married to KAHLEETHA LAUVAO with 6 beautiful children. Miss n love you to the fullest son Louis Joseph Augustino Lauvao aka Big Boy. Until we meet again", "followers_count": 90, "friends_count": 163, "statues_count": 1164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235317637120, "text": "@modica24 been Chillin wbu", "in_reply_to_status": 716158151339302912, "in_reply_to_user": 1734066085, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1734066085 }}, "user": { "id": 2564486189, "name": "Michael Drouillard", "screen_name": "mdrouillard10", "lang": "en", "location": "null", "create_at": date("2014-05-25"), "description": "null", "followers_count": 67, "friends_count": 58, "statues_count": 523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendora, CA", "id": "eb1bb64775708bc1", "name": "Glendora", "place_type": "city", "bounding_box": rectangle("-117.890263,34.10549 -117.809111,34.165551") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 630014, "cityName": "Glendora" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235577683969, "text": "17 ice creams Nichole https://t.co/iaNFTkgI5d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2457731790, "name": "NeptunoⓂ️", "screen_name": "neptuno30", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-04-21"), "description": "One God Only", "followers_count": 5006, "friends_count": 5005, "statues_count": 166868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheridan, CO", "id": "3cfce73ef3a505af", "name": "Sheridan", "place_type": "city", "bounding_box": rectangle("-105.034755,39.631416 -104.999537,39.660504") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 869645, "cityName": "Sheridan" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235779194880, "text": "Currently someone left their car parked in the middle of the parking lot blocking the entrance ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3045299735, "name": "Zoey Whittington", "screen_name": "its_zogurttt", "lang": "en", "location": "null", "create_at": date("2015-02-19"), "description": "Pitt Track &nd Field .. imagine these moments in vibrant hues ❤️", "followers_count": 285, "friends_count": 271, "statues_count": 2566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235850383360, "text": "& you should know that the lies won't hide your flaws.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2851248433, "name": "K.Cole", "screen_name": "kaycole0", "lang": "en", "location": "ocean drive ", "create_at": date("2014-10-10"), "description": "TAMU-CC.", "followers_count": 734, "friends_count": 484, "statues_count": 22093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158235921649664, "text": "@unstoppabull24 thank you sharmoot❤️", "in_reply_to_status": 716158059114967040, "in_reply_to_user": 714045397, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 714045397 }}, "user": { "id": 535553188, "name": "Sarsoor❤️", "screen_name": "saraamawi", "lang": "en", "location": "Plano, TX", "create_at": date("2012-03-24"), "description": "sensational // فلسطين", "followers_count": 758, "friends_count": 491, "statues_count": 29463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158236206870528, "text": "b my friend on miitomo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 931252831, "name": "sleepy jess", "screen_name": "yhessibaby", "lang": "en", "location": "null", "create_at": date("2012-11-06"), "description": "if jessicakes got so much money then why she robbin' -Fern", "followers_count": 484, "friends_count": 407, "statues_count": 19329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale, AZ", "id": "0015d9147cee6907", "name": "Avondale", "place_type": "city", "bounding_box": rectangle("-112.357999,33.384785 -112.272424,33.493806") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 404720, "cityName": "Avondale" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158237028933632, "text": "����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1291870304, "name": "Jay", "screen_name": "kingroshy", "lang": "en", "location": "T.D.E", "create_at": date("2013-03-23"), "description": "Top Dawg Entertainment", "followers_count": 388, "friends_count": 512, "statues_count": 5510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-04-02T00:00:03.000Z"), "id": 716158237087764480, "text": "Wind 2.6 mph NW. Barometer 29.604 in, Steady. Temperature 39.5 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237293342721, "text": "Yo https://t.co/zTzhOx4Ebs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1393261064, "name": "Aaron Simpson", "screen_name": "_asimp_", "lang": "en", "location": "null", "create_at": date("2013-04-30"), "description": "Football || Baseball⚾️ #2", "followers_count": 1240, "friends_count": 895, "statues_count": 3620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indian Wells, CA", "id": "00af7bd33e05ab00", "name": "Indian Wells", "place_type": "city", "bounding_box": rectangle("-116.374119,33.671815 -116.294972,33.743643") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636434, "cityName": "Indian Wells" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237398007808, "text": "https://t.co/n9cWepYWmy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 385114882, "name": "Juan Cena", "screen_name": "juanchain_", "lang": "en", "location": "san antonio ✈️ los angeles", "create_at": date("2011-10-04"), "description": "I DO NOT HAVE HOES #cowboynation #gatornation", "followers_count": 1474, "friends_count": 765, "statues_count": 105351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple City, CA", "id": "206e932cc0e89fa2", "name": "Temple City", "place_type": "city", "bounding_box": rectangle("-118.081392,34.085395 -118.028425,34.120984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 678148, "cityName": "Temple City" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237418987520, "text": "I almost had a heart attack, but my baby is home now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312920034, "name": "Gabriella Di Caprio", "screen_name": "Gdicaprio", "lang": "en", "location": "Pasadena, CA", "create_at": date("2011-06-07"), "description": "live in the now 20 English & Pre-Law major make-up addict ❤️ TAKEN 3/16/15❤️instagram: @gaboutloud you don't want to be tangled up by roads you never traveled", "followers_count": 142, "friends_count": 261, "statues_count": 1244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237809180672, "text": "Friday feels. #woodhoops \n--\n#groove #tone #chop #drum #drummer #drumming #drumset #drumlife… https://t.co/OWmdc9yC5C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.36953964,34.16649068"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "woodhoops", "groove", "tone", "chop", "drum", "drummer", "drumming", "drumset", "drumlife" }}, "user": { "id": 22186364, "name": "Blake Paulson", "screen_name": "blakepaulson", "lang": "en", "location": "North Hollywood, CA", "create_at": date("2009-02-27"), "description": "Music is my passion: Drummer. Producer. Educator. Author. | #MakeMusic @LosAngeles | Info at my website.", "followers_count": 404, "friends_count": 660, "statues_count": 1270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237867773952, "text": "https://t.co/rED5bVCRYv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 929058469, "name": "Chris Wilcox", "screen_name": "SuperCFlash", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "BYU Signee || SCF Beats ♨️|| (scfbeats@gmail.com)", "followers_count": 849, "friends_count": 450, "statues_count": 12381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158237989609472, "text": "Just got asked if I was \"trans\" so there goes my life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1171207592, "name": "kelizzle", "screen_name": "KKLLCCC", "lang": "xx-lc", "location": "ATX", "create_at": date("2013-02-11"), "description": "I dont have a lisp when i type", "followers_count": 137, "friends_count": 164, "statues_count": 7733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158238165630976, "text": "https://t.co/ANdW7UWHrS\n\nI support their efforts. Uniting against the proposed pipeline.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20206173, "name": "Carol D", "screen_name": "Mommyof7", "lang": "en", "location": "United States", "create_at": date("2009-02-05"), "description": "I don't go along to get along.", "followers_count": 28, "friends_count": 23, "statues_count": 4458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158238199250945, "text": "��", "in_reply_to_status": 711398141761888256, "in_reply_to_user": 590902435, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 590902435, "name": "sol", "screen_name": "Marisol_Trevino", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "@trvisXX", "followers_count": 1208, "friends_count": 344, "statues_count": 41684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen Park, MI", "id": "0c937cb917334546", "name": "Allen Park", "place_type": "city", "bounding_box": rectangle("-83.231897,42.222765 -83.183819,42.301134") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2601380, "cityName": "Allen Park" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158238551445504, "text": "ALGO TIME!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3007218264, "name": "Riddim Jake", "screen_name": "jdawgg713", "lang": "en", "location": "null", "create_at": date("2015-02-01"), "description": "Corn make me Happy :) HTX.Corn Enthusiast. Ask me for my Corn ;) Fractals n Rage, forget chill! #TXRaveChasers", "followers_count": 766, "friends_count": 1472, "statues_count": 12595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158238861824000, "text": "Makayla ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 2462726622, "name": "✨H O N E Y✨", "screen_name": "KaHevW40", "lang": "en", "location": "In her pants", "create_at": date("2014-04-24"), "description": "'17 | Cassie Marie 3.26.16", "followers_count": 481, "friends_count": 444, "statues_count": 8604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158239524655108, "text": "Still floating", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 996299329, "name": "ㅤㅤㅤㅤㅤ", "screen_name": "1fatnif", "lang": "en", "location": "Philadelphia, PA•", "create_at": date("2012-12-07"), "description": "null", "followers_count": 1848, "friends_count": 883, "statues_count": 14455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158239608418305, "text": "That's the funniest shit I've seen all day. https://t.co/RY3G53Lrfy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1123944800, "name": "Wise Guy", "screen_name": "whos_tinauris", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "I'm here, aren't I?", "followers_count": 198, "friends_count": 126, "statues_count": 7719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158239918858240, "text": "Been struggling hella hard these past days to catch some ZZZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1556407507, "name": "RAY", "screen_name": "Og_zay3", "lang": "en", "location": "Somewhere waitin on oomf ", "create_at": date("2013-06-29"), "description": "null", "followers_count": 867, "friends_count": 428, "statues_count": 33387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spencer, OK", "id": "c84ea8299b18df59", "name": "Spencer", "place_type": "city", "bounding_box": rectangle("-97.388653,35.49295 -97.344491,35.528897") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4069200, "cityName": "Spencer" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158240246116352, "text": "@RABIDPANIC no other energy drink has had me this fucked. I feel like I can fly", "in_reply_to_status": 716158102563856385, "in_reply_to_user": 228563431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 115842291 }}, "user": { "id": 228563431, "name": "SCORPION", "screen_name": "zacharyhatfield", "lang": "en", "location": "WV", "create_at": date("2010-12-19"), "description": "@leftbehind304 @tosoundthealarm", "followers_count": 1370, "friends_count": 496, "statues_count": 25250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carnuel, NM", "id": "663f7377b8cd0592", "name": "Carnuel", "place_type": "city", "bounding_box": rectangle("-106.484896,35.056541 -106.424867,35.074659") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3512430, "cityName": "Carnuel" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158240611049472, "text": "Starbucks break with @RuminateMag editors Brianna Van Dyke and Renee Long. #AWP16 https://t.co/UWpg0196qI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "AWP16" }}, "user_mentions": {{ 21103145 }}, "user": { "id": 19916228, "name": "Sophfronia Scott", "screen_name": "Sophfronia", "lang": "en", "location": "Sandy Hook, CT", "create_at": date("2009-02-02"), "description": "Author of novel All I Need to Get By. Faculty @RegisUniversity @MileHighMFA. Writing in @SaranacReview, @RuminateMag, @BStormJournal. Agent @Brettne @thebookgrp", "followers_count": 6518, "friends_count": 4343, "statues_count": 30154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158240866721793, "text": "In first or second grade https://t.co/N6ZBPbaHIc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712124551916720128, "name": "Sarai", "screen_name": "ssaaraai_", "lang": "en", "location": "null", "create_at": date("2016-03-21"), "description": "EAHS• Peace,Love,and Positivity", "followers_count": 96, "friends_count": 112, "statues_count": 492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158240892010497, "text": "@navohnaiviv BC THEY GAVE ME ALFREDO W NO SAUCE", "in_reply_to_status": 716158137275801600, "in_reply_to_user": 2249554159, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2249554159 }}, "user": { "id": 3287114202, "name": "Katelyn Vuong", "screen_name": "VuongKatelyn", "lang": "en", "location": "null", "create_at": date("2015-07-21"), "description": "null", "followers_count": 44, "friends_count": 70, "statues_count": 660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158241156116480, "text": "See you soon ���� https://t.co/6M54coIORU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2716207674, "name": "jessica", "screen_name": "doitbig___", "lang": "en", "location": "HOUSTON❤️", "create_at": date("2014-08-07"), "description": "let's get lit", "followers_count": 240, "friends_count": 278, "statues_count": 13165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158241156149249, "text": "@GeKaay dammmmmmmmmm whatever", "in_reply_to_status": 716158145257541632, "in_reply_to_user": 3016472723, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3016472723 }}, "user": { "id": 3233642018, "name": "♡", "screen_name": "__danaaaaaa", "lang": "en", "location": "null", "create_at": date("2015-06-02"), "description": "chs ✨", "followers_count": 101, "friends_count": 147, "statues_count": 502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158241244196864, "text": "Laughing in the face of death", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1433753582, "name": "Blake", "screen_name": "blakemalana", "lang": "en", "location": "Ontario, CA", "create_at": date("2013-05-16"), "description": "null", "followers_count": 1263, "friends_count": 795, "statues_count": 31689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-02T00:00:04.000Z"), "id": 716158241374216192, "text": "@yessiaguilar_00 are you okay?", "in_reply_to_status": 716147338914344960, "in_reply_to_user": 587086483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 587086483 }}, "user": { "id": 360213934, "name": "❥ Jenn ❥", "screen_name": "_jenniferest", "lang": "en", "location": "Oregon, USA", "create_at": date("2011-08-22"), "description": "null", "followers_count": 247, "friends_count": 218, "statues_count": 6766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Dalles, OR", "id": "0003cfce3c426793", "name": "The Dalles", "place_type": "city", "bounding_box": rectangle("-121.229115,45.585142 -121.121632,45.634037") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41065, "countyName": "Wasco", "cityID": 4172950, "cityName": "The Dalles" } }
+{ "create_at": datetime("2016-04-02T00:00:05.000Z"), "id": 716158241646845956, "text": "do you remember them nights, we be fussin & fighting an be fucking all night ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323035356, "name": "taylor.", "screen_name": "ripshonne2_", "lang": "en", "location": "null", "create_at": date("2011-06-23"), "description": "✊ ʝαиυαяу 21ѕт 2015 ❣ zуѕнσииєѕ кєєρєя ✨", "followers_count": 1496, "friends_count": 1239, "statues_count": 25865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520608746450944, "text": "He didn't even wait for me to get inside after he dropped me off sooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2945459786, "name": "☼Elizah☼", "screen_name": "chill_itseliza", "lang": "en", "location": "Mesa, AZ", "create_at": date("2014-12-27"), "description": "We need more independent thinkers, we need more lovers, more healers, more poets, more artists. We need more people that actually care.", "followers_count": 208, "friends_count": 260, "statues_count": 10234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520608801103872, "text": "my black friends I grew up with I was turning too white my white friend's never understood why everytime I got drunk I went on fuck wp rants", "in_reply_to_status": 716520418467778560, "in_reply_to_user": 768311516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 768311516, "name": "Zeroni", "screen_name": "Misszeroni", "lang": "en", "location": "Washington, DC", "create_at": date("2012-08-19"), "description": "Pisces. Songbird. Womanist. ig: @Zeronimusic", "followers_count": 3747, "friends_count": 1891, "statues_count": 113574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520609199497216, "text": "Butthurt they didn't give me a shirt ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051305344, "name": "擦来了么大大的y", "screen_name": "jaasminel_24", "lang": "en", "location": "San Diego", "create_at": date("2012-12-31"), "description": "null", "followers_count": 157, "friends_count": 84, "statues_count": 5115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520609237237760, "text": "Coolest dish of all time. @ Disneyland https://t.co/KojdQsynct", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.91990057,33.8107798"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1933535305, "name": "shad newton", "screen_name": "shadsfc", "lang": "en", "location": "null", "create_at": date("2013-10-04"), "description": "Trois Mec", "followers_count": 49, "friends_count": 243, "statues_count": 100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520609254154240, "text": "@jvnghoseoks him but then run my fingers through his hair and kiss his face and make sure hes rlly okay Damn", "in_reply_to_status": 716520453381218305, "in_reply_to_user": 2194988149, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2194988149 }}, "user": { "id": 2194988149, "name": "ash", "screen_name": "jvnghoseoks", "lang": "en", "location": "sylvia, kylie", "create_at": date("2013-11-14"), "description": "@siopaojimin: imagine ashley and hoseok together ..... a Great Concept ™", "followers_count": 1744, "friends_count": 98, "statues_count": 133712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Park, TX", "id": "5c937363480a2c19", "name": "Cedar Park", "place_type": "city", "bounding_box": rectangle("-97.8817,30.455536 -97.74559,30.556381") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4813552, "cityName": "Cedar Park" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520609975414784, "text": "Yes https://t.co/gkHFlrQVMd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 496501160, "name": "Ari ☜♡☞", "screen_name": "arianaortiz96", "lang": "en", "location": "Southern California, USA", "create_at": date("2012-02-18"), "description": "tell me who the fuck you wanna be.", "followers_count": 644, "friends_count": 1845, "statues_count": 35340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wildomar, CA", "id": "5080f2ebcf3d5cfd", "name": "Wildomar", "place_type": "city", "bounding_box": rectangle("-117.308832,33.575507 -117.20561,33.656019") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 685446, "cityName": "Wildomar" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520610717786112, "text": "I wanna go to sleep but my comforter & bed sheets are washing :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3130201571, "name": "Priscilla Villarreal", "screen_name": "pvillarreal1l", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "I'm not that great", "followers_count": 262, "friends_count": 191, "statues_count": 309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520610944442369, "text": "Temp 37.6° Hi/Lo 38.6/37.3 Rng 1.3° WC 37.6° Hmd 97% Rain 0.00\" Storm 0.40\" BAR 29.250 Falling DP 36.8° Wnd 1mph Dir S Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 64, "friends_count": 116, "statues_count": 17471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520611380535297, "text": "It's officially the start of the @MLB season!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18479513 }}, "user": { "id": 2160106555, "name": "Bradley", "screen_name": "bradleyheckman8", "lang": "en", "location": "El Segundo ", "create_at": date("2013-10-27"), "description": "I never quite figured this out", "followers_count": 462, "friends_count": 380, "statues_count": 4171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Segundo, CA", "id": "0654b676d0359a31", "name": "El Segundo", "place_type": "city", "bounding_box": rectangle("-118.429843,33.901804 -118.370685,33.931493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622412, "cityName": "El Segundo" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520611451768832, "text": "Watching previous KS episodes while I wait for #MAICHARDonSPS replay haha #atat ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MAICHARDonSPS", "atat" }}, "user": { "id": 373859494, "name": "Cheena Marie David", "screen_name": "CiCiDavid", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2011-09-15"), "description": "CiCi | Beauty, books and everything in between.", "followers_count": 222, "friends_count": 164, "statues_count": 26805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520611867021312, "text": "Bruh I swear Amber dropped the money on this lens but she basically put our future in my hands because fuck these photos just sky rocketed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46977872, "name": "Desirée Guerra °OØO", "screen_name": "DesAesthetic", "lang": "en", "location": "Texas, USA", "create_at": date("2009-06-13"), "description": "DM for any inquiries please. Beast without the Hype. Owner of Fair'Nuff #FRNF // Member of °OØO // #TeamAmber", "followers_count": 2035, "friends_count": 1591, "statues_count": 27770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612047421441, "text": "#شعيب_شالسالفة ؟ @Sh3aaiB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "شعيب_شالسالفة" }}, "user_mentions": {{ 223904948 }}, "user": { "id": 313530859, "name": "Ahmed E. Al-eid", "screen_name": "AhmedEssa07", "lang": "en", "location": "Wichita, KS", "create_at": date("2011-06-08"), "description": "here is where I write what's on my mind. snapchat: AhmedEssa07", "followers_count": 206, "friends_count": 164, "statues_count": 1069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612198363138, "text": "↑", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1576436876, "name": "Jerome Cadiz", "screen_name": "JeromeExists", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "Save life. Realizer | Runner | Wannabe Model | '80s Blood | Dancer | Believer", "followers_count": 123, "friends_count": 107, "statues_count": 1721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612211073024, "text": "@aka_Eldrick @BriarWynn1 man what a time! Coleton is not MIA anymore", "in_reply_to_status": 716520410834034688, "in_reply_to_user": 431321618, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 431321618, 808053966 }}, "user": { "id": 363437651, "name": "Tyson Tucker", "screen_name": "TysonATucker", "lang": "en", "location": "null", "create_at": date("2011-08-27"), "description": "null", "followers_count": 637, "friends_count": 299, "statues_count": 17239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creston, IA", "id": "20bd1a83594beaa7", "name": "Creston", "place_type": "city", "bounding_box": rectangle("-94.385733,41.042901 -94.332785,41.074839") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19175, "countyName": "Union", "cityID": 1917265, "cityName": "Creston" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612336828416, "text": "⚠️⚠️⚠️⚠️⚠️ https://t.co/nBAbtJn0JO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 310691746, "name": "dog", "screen_name": "Jjaaaypaz", "lang": "en", "location": "Reno, NV", "create_at": date("2011-06-03"), "description": "all that is gold does not glitter; not all those who wander are lost.", "followers_count": 639, "friends_count": 613, "statues_count": 18063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612374650880, "text": "Ripley SW Limestone Co. Temp: 48.9°F Wind:1.6mph Pressure: 998.6mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612395503616, "text": "He's so upset at me. �� https://t.co/FWLQDbLyOD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34013653, "name": "Chamaine", "screen_name": "cbekius", "lang": "en", "location": "Kennydale", "create_at": date("2009-04-21"), "description": "MN native, Hawks, Sounders, Reign and Chelsea fan. #EBFG , dogs, Seattle, Favre lover and all things Batman....Instagram is cbekius1 snapchat is cbekius", "followers_count": 1463, "friends_count": 1613, "statues_count": 37219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Renton, WA", "id": "a96f188f5de647b5", "name": "Renton", "place_type": "city", "bounding_box": rectangle("-122.253717,47.423263 -122.135079,47.539933") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357745, "cityName": "Renton" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612416647168, "text": "Operation long john silver", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229794351, "name": "Matt Schroth", "screen_name": "Schroth21", "lang": "en", "location": "CLE/CBUS", "create_at": date("2010-12-23"), "description": "If coach would've put me in i know we would've won state.", "followers_count": 698, "friends_count": 567, "statues_count": 19818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-03T00:00:00.000Z"), "id": 716520612542320640, "text": "I think life goals is Mathew mcconaughey driving with the two Great Danes in the Lincoln", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547616573, "name": "Nick Merlino", "screen_name": "merlino_nick", "lang": "en", "location": "Stillwater, USA", "create_at": date("2012-04-07"), "description": "Oklahoma State", "followers_count": 253, "friends_count": 392, "statues_count": 982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520612747804672, "text": "just caught my little brother trying to sneak out right now????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302796835, "name": "princess jasmine", "screen_name": "mariajesseee", "lang": "en", "location": "null", "create_at": date("2015-07-31"), "description": "snapchat maria.jesse", "followers_count": 400, "friends_count": 301, "statues_count": 3142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520612756193280, "text": "it's me https://t.co/sY6MhKd3pF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2201987750, "name": "Maz @ NWCampCon", "screen_name": "mazzaful", "lang": "en", "location": "[47.4886° N, 117.5786° W], WA ", "create_at": date("2013-11-18"), "description": "Androgynous DFAB artist, web designer, and computer support technician. Possibly a meme.", "followers_count": 493, "friends_count": 350, "statues_count": 22189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520612789878784, "text": "ANOTHER bulb blew on my car. I JUST CHANGED SOME ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35261215, "name": "b♡", "screen_name": "brandiseon", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-25"), "description": "NYG • VW • Bleed Blue", "followers_count": 601, "friends_count": 479, "statues_count": 35155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawrenceville, GA", "id": "181444e0e9e17e20", "name": "Lawrenceville", "place_type": "city", "bounding_box": rectangle("-84.046802,33.910542 -83.951606,33.990728") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1345488, "cityName": "Lawrenceville" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520612945117184, "text": "Temp 32.5°F Wind Chill 25.6°F RH 51% Wind 7.8 W Gust 20.0 W SLP 29.923 in Rising quickly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 63, "statues_count": 34428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520612957659136, "text": "I have to stop deleting people's number when I'm mad because they text me and I'm just like ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3340498468, "name": "Michaela", "screen_name": "Dipped_ngold_", "lang": "en", "location": "null", "create_at": date("2015-06-21"), "description": "#NCAT \n856✈336 \nBiology/Pre-Med \nIG:_dipped_ngold_", "followers_count": 569, "friends_count": 507, "statues_count": 1824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520613125472256, "text": "let God get them �� https://t.co/ErAujqWwuC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363865316, "name": "shenell waugh", "screen_name": "shenellvintage", "lang": "en", "location": "florida atlantic university ", "create_at": date("2011-08-28"), "description": "made of brown sugar, cocoa, honey & gold.", "followers_count": 874, "friends_count": 813, "statues_count": 34589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520613901275137, "text": "you guys are so funny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171607784, "name": "Harneet ♡", "screen_name": "yourgirlneet", "lang": "en", "location": "American Canyon", "create_at": date("2010-07-27"), "description": "null", "followers_count": 1942, "friends_count": 922, "statues_count": 27306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "American Canyon, CA", "id": "f28195f8b183abcd", "name": "American Canyon", "place_type": "city", "bounding_box": rectangle("-122.283944,38.154977 -122.229741,38.204868") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 601640, "cityName": "American Canyon" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614190653445, "text": "If you don't feel The Beatles I don't feel you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 850072699, "name": "kaylene", "screen_name": "affleck_kaylene", "lang": "en", "location": "null", "create_at": date("2012-09-27"), "description": "i live to relax", "followers_count": 341, "friends_count": 405, "statues_count": 2646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cottonwood Heights, UT", "id": "1bbe64ec5c8ce842", "name": "Cottonwood Heights", "place_type": "city", "bounding_box": rectangle("-111.856645,40.585778 -111.776806,40.637499") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4916270, "cityName": "Cottonwood Heights" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614207459329, "text": "Epic Noodle �� || MongolianBBQ��#KeepinItReal #FRESH #GoGreen \n #Healthy #NoMSG #FlexibleDiet… https://t.co/Hr6vmSTzYz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.8310776,33.8995895"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KeepinItReal", "FRESH", "GoGreen", "Healthy", "NoMSG", "FlexibleDiet" }}, "user": { "id": 2476254139, "name": "EPIC NOODLE", "screen_name": "EPICNOODLEOC", "lang": "en", "location": "17401 Bastanchury Rd 92886", "create_at": date("2014-05-03"), "description": "EPIC NOODLE MONGOLIAN BBQ http://EPICNOODLE.COM", "followers_count": 55, "friends_count": 62, "statues_count": 1196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614329065472, "text": "Wind 3.0 mph SSW. Barometer 29.955 in, Falling. Temperature 35.6 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 2357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614371180545, "text": "Wind 0.0 mph ---. Barometer 30.124 in, Steady. Temperature 38.4 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614371192832, "text": "03:00:01 |Temp: 52.4ºF | Wind Chill 52.4ºF |Dew Point 38.5ºF | Rain today: 0.00 inches | Wind: 2.0 mph from the WNW, Gusting to 5.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614647861250, "text": "รักอ่ะ มีแมวมานอนข้างๆ ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user": { "id": 361823993, "name": "NN ◡̈", "screen_name": "ahanut_", "lang": "en", "location": "Texas, USA", "create_at": date("2011-08-25"), "description": "#Giforce ( จะรีบกลับไทยนะขอเวลาหน่อย ) ข้าอยากอยู่กับเองนานๆ ♡", "followers_count": 474, "friends_count": 149, "statues_count": 92291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614677372929, "text": "04/03@03:00 - Temp 42.0F, WC 36.1F. Wind 10.0mph WSW, Gust 28.0mph. Bar 29.481in, Rising. Rain 0.03in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614698229760, "text": "Don't mean ima stop tho fuck yall", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315510857, "name": "piece of shit.", "screen_name": "Sebcessa_", "lang": "en", "location": "US Navy ", "create_at": date("2011-06-11"), "description": "17 idk still figuring shit out.", "followers_count": 914, "friends_count": 604, "statues_count": 30102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520614840836096, "text": "All of the above �� https://t.co/a6C9BVH9o7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1345808557, "name": "$", "screen_name": "queenashton_", "lang": "en", "location": "in sephora ", "create_at": date("2013-04-11"), "description": "Yes Daddy I Do", "followers_count": 825, "friends_count": 1000, "statues_count": 16444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615017062400, "text": "Listen to Way Down We Go by Kaleo on @AppleMusic. https://t.co/Cf3E3huQZe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74580436 }}, "user": { "id": 2509266223, "name": "JRG ", "screen_name": "joegimler", "lang": "en", "location": "Boston, MA", "create_at": date("2014-05-19"), "description": "Pharmacy Student", "followers_count": 170, "friends_count": 246, "statues_count": 198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615105040384, "text": "Like if you're going to act single just be single don't play with anyone's feelings...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197484931, "name": "YoGurlWithCurls", "screen_name": "ALEHHH_44", "lang": "en", "location": "Dominican Republic", "create_at": date("2010-10-01"), "description": "Ms.Colomé", "followers_count": 1205, "friends_count": 1580, "statues_count": 29913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615163760641, "text": "I would say so �� https://t.co/CiNwkUT164", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2468498598, "name": "ash", "screen_name": "ash_villanueva_", "lang": "en", "location": "null", "create_at": date("2014-04-28"), "description": "null", "followers_count": 373, "friends_count": 407, "statues_count": 8177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615193157633, "text": "@RunJessRunn lol what makes you cool", "in_reply_to_status": 716520531688685569, "in_reply_to_user": 628173840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 628173840 }}, "user": { "id": 2549082211, "name": "Brad Woodard", "screen_name": "5for6", "lang": "en", "location": "Houston, TX", "create_at": date("2014-06-05"), "description": "it's all about finesse. TSU football #NoPressure #MoneyTime", "followers_count": 1849, "friends_count": 1093, "statues_count": 288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dickinson, TX", "id": "1eac21aedfcc3668", "name": "Dickinson", "place_type": "city", "bounding_box": rectangle("-95.120569,29.422182 -95.010182,29.491047") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4820344, "cityName": "Dickinson" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615226658816, "text": "Basically the movie should of just been called was DC's Dawn Of Justice. Instead of Batman Vs Superman: Dawn Of Justice.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84892289, "name": "⚡️Electro⚡️ (Bryant)", "screen_name": "Ice100x", "lang": "en", "location": "The ATX! SC: TheChargedGod⚡️", "create_at": date("2009-10-24"), "description": "#TeamSagittarius♐️ #TeamTruth #Fashion #Optimistic #CHARGED⚡ #PositiveVibes☀️ #Love❤️ #God #IDontgettired #Flex #Fighter #Loyal #Friendships #Music #Dance #Sing", "followers_count": 1262, "friends_count": 1163, "statues_count": 23876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leander, TX", "id": "be1ec4774a795507", "name": "Leander", "place_type": "city", "bounding_box": rectangle("-97.897898,30.524702 -97.795581,30.612656") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4842016, "cityName": "Leander" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615331500035, "text": "@ayanamariee_ aw!! It came out really good!!", "in_reply_to_status": 716498622678839299, "in_reply_to_user": 2558291431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2558291431 }}, "user": { "id": 2369348144, "name": "madds", "screen_name": "maddsthebadds", "lang": "en", "location": "lbp", "create_at": date("2014-03-02"), "description": "do ur work. dont b stupid.", "followers_count": 442, "friends_count": 289, "statues_count": 15896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615469953024, "text": "happy I saw @alig319 and @abby_cragun tonight������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2213605874, 2269575377 }}, "user": { "id": 1423926896, "name": "jessicar", "screen_name": "jessie_noellle", "lang": "en", "location": "null", "create_at": date("2013-05-12"), "description": "@tysonagla", "followers_count": 739, "friends_count": 860, "statues_count": 6880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615683837952, "text": "Unplanned matching �� ���� #luau https://t.co/X9SqxFNeo7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.733795,33.556643"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "luau" }}, "user": { "id": 520615725, "name": "セレステ", "screen_name": "MarquezCeleste", "lang": "en", "location": "Location: by the food", "create_at": date("2012-03-10"), "description": "Proverbs 3:5-6//Amat '15//SUA '19", "followers_count": 216, "friends_count": 203, "statues_count": 4049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soka University", "id": "07d9e3bc11087000", "name": "Soka University", "place_type": "poi", "bounding_box": rectangle("-117.7337951,33.5566429 -117.733795,33.556643") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639248, "cityName": "Laguna Niguel" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615721771008, "text": "finally figured my way to one up you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269998529, "name": "la.", "screen_name": "laurensimpson2", "lang": "en", "location": "912", "create_at": date("2011-03-21"), "description": "ᗷE YOᑌᖇᔕEᒪᖴ, EᐯEᖇYOᑎE EᒪᔕE Iᔕ ᗩᒪᖇEᗩᗪY TᗩKEᑎ -if you can make a girl laugh, you can make her do anything", "followers_count": 742, "friends_count": 1767, "statues_count": 16817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crescent Beach, FL", "id": "92f03e393e8ada04", "name": "Crescent Beach", "place_type": "city", "bounding_box": rectangle("-81.257529,29.707376 -81.226152,29.771667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12109, "countyName": "St. Johns", "cityID": 1215325, "cityName": "Crescent Beach" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520615822430208, "text": "Wind 0.0 mph WNW. Barometer 30.106 in, Rising. Temperature 33.9 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520616136867840, "text": "Me and the youngin had a campout in the living room tonight, since I been working alot lately. Big Hero 6 and hella popcorn was the move", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101524778, "name": "Scrill Murray", "screen_name": "Im_VelvetJones", "lang": "en", "location": "Long Beach", "create_at": date("2010-01-03"), "description": "I talk sports, music, and shit. Not in that order #AirForce #TheColi #TarHeels\nIG: Im_VelvetJones", "followers_count": 704, "friends_count": 1239, "statues_count": 85317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520616208113664, "text": "Reverb on guitar plucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1098492817, "name": "Max Wickens", "screen_name": "maxwellwickens", "lang": "en", "location": "null", "create_at": date("2013-01-17"), "description": "null", "followers_count": 815, "friends_count": 264, "statues_count": 3816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520616451399681, "text": "@saxonjohn666 @marzipan_man", "in_reply_to_status": 716520522016665600, "in_reply_to_user": 2472019135, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2472019135, 713971794122711040 }}, "user": { "id": 2472019135, "name": "Nicotine Crustacean", "screen_name": "saxonjohn666", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2014-04-30"), "description": "18/Pansexual/I'm the fucking Lizard King", "followers_count": 564, "friends_count": 399, "statues_count": 20329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520616518488065, "text": "@ebbtideapp Tide in Dauphin Island, Alabama 04/03/2016\n Low 5:58am -0.0\nHigh 8:07pm 1.0\n Low 6:38am 0.1\nHigh 9:28pm 0.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-88.075,30.25"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 49, "friends_count": 1, "statues_count": 21228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 119744, "cityName": "Dauphin Island" } }
+{ "create_at": datetime("2016-04-03T00:00:01.000Z"), "id": 716520616736587777, "text": "@Ianlopez_4 aw c'mon you know I'm playing", "in_reply_to_status": 716520511598043136, "in_reply_to_user": 621322988, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 621322988 }}, "user": { "id": 954038250, "name": "lexy gonzalez", "screen_name": "lexygonzalez99", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-11-17"), "description": "taft raider band flute | cahs '17 | jeremiah 29:11", "followers_count": 2465, "friends_count": 1327, "statues_count": 8494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helotes, TX", "id": "00a1b28251c671b8", "name": "Helotes", "place_type": "city", "bounding_box": rectangle("-98.764721,29.495566 -98.661448,29.604798") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520616887586816, "text": "Talking all that shit but after I hit....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2256005796, "name": "Papi Keef", "screen_name": "Noahkeefnochief", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "Free El Chapo", "followers_count": 401, "friends_count": 423, "statues_count": 6042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520616900313094, "text": "My auntie is like really...... ANNOYING TF OUT OF ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36777629, "name": "sc • itsRAE", "screen_name": "Uluvraee", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-04-30"), "description": "This ain't a shoulder with a chip or an ego", "followers_count": 1216, "friends_count": 760, "statues_count": 33086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, WV", "id": "71f2805dd75bc147", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-81.707175,38.293241 -81.560235,38.401731") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54039, "countyName": "Kanawha", "cityID": 5414600, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617059749888, "text": "Temp: 43.3°F Wind:0.0mph Pressure: 30.166hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617617457152, "text": "Yep. He will get drafted higher then most think @BillLatin I ain't tripping @Nick_DFWPulse https://t.co/Q2JFPqaj0C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34161980, 636657461 }}, "user": { "id": 315016589, "name": "NFLJoshua Johnson", "screen_name": "Josh_DFWPulse", "lang": "en", "location": "Sioux Falls South Dakota", "create_at": date("2011-06-10"), "description": "Host of #DynastyPulsePodcast (http://www.blogtalkradio.com/dfwarehouse) | by #DFW | #IDP #Dynasty #DFW16 #NFLDraft #DFW36 #DFW48 #RaiderNation", "followers_count": 2564, "friends_count": 4998, "statues_count": 37251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617667743744, "text": "Then the answer is yes to the no. https://t.co/hncLMY5AvZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76862633, "name": "Katt Funny", "screen_name": "KattFunny", "lang": "en", "location": "Portland, OR", "create_at": date("2009-09-23"), "description": "Host of the #KattFunnyTag, a pop-up hashtag game for the underdog.", "followers_count": 8346, "friends_count": 8191, "statues_count": 39800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukie, OR", "id": "0834e7769aa05fce", "name": "Milwaukie", "place_type": "city", "bounding_box": rectangle("-122.655374,45.424593 -122.588791,45.461537") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4148650, "cityName": "Milwaukie" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617676156928, "text": "@_ohdearsol sol-Ea 5ever �� https://t.co/wG0ME0NCYM", "in_reply_to_status": 716520241828761601, "in_reply_to_user": 2560815356, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2560815356 }}, "user": { "id": 3852862093, "name": "matteimasu", "screen_name": "leacaguioa", "lang": "en", "location": "null", "create_at": date("2015-10-10"), "description": "kōfuku o matteimasu", "followers_count": 142, "friends_count": 111, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617747488768, "text": "There is no way of escaping the facts of life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49197388, "name": "WorldzSavior", "screen_name": "WolvieWalls", "lang": "en", "location": "The Habitat/Neptune", "create_at": date("2009-06-20"), "description": "Innovata/Entreprenegro 不 Artist/Visionary 人 Hip Hop, Marijuana, & Comedy Enthusiast #BlackSpiderMan #SniperGameWolvie★彡", "followers_count": 917, "friends_count": 913, "statues_count": 10569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520617810333697, "text": "@azamuddinadnan @AufaAzmin aku tau aufa loser tpi tktau plak yg dia sehina ni", "in_reply_to_status": 716516691799179264, "in_reply_to_user": 441740957, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 441740957, 451182506 }}, "user": { "id": 430615778, "name": "هايكل هيشام", "screen_name": "haikal_hisham", "lang": "en", "location": "Shah Alam City", "create_at": date("2011-12-07"), "description": "Aerospace Engineer in the future", "followers_count": 360, "friends_count": 242, "statues_count": 25719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520618313719809, "text": "@FindKee_ dnt imagine �� stay you g enjoy it", "in_reply_to_status": 716520318475481088, "in_reply_to_user": 222314226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 222314226 }}, "user": { "id": 44245298, "name": "A Faithful Black Man", "screen_name": "QDOTjones", "lang": "en", "location": "New Orleans, Louisiana", "create_at": date("2009-06-02"), "description": "Quinton Jones - REST Moms - Videographer - Cinematographer - Photographer - USARMY University of New Orleans - IG: QDOTJONES http://qdotjones.com", "followers_count": 3048, "friends_count": 2205, "statues_count": 176906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520618540138498, "text": "Didn't get ready for nothing ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 583336458, "name": "Lyss", "screen_name": "younglyssma", "lang": "en", "location": "null", "create_at": date("2012-05-17"), "description": "null", "followers_count": 1571, "friends_count": 485, "statues_count": 49081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520618586476544, "text": "Damn my mans arms so fucking sexy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2963647253, "name": "woke", "screen_name": "lovlifelivelov", "lang": "en", "location": "God", "create_at": date("2015-01-05"), "description": "null", "followers_count": 172, "friends_count": 162, "statues_count": 18287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pompano Beach, FL", "id": "fa8f8f24dc772cc0", "name": "Pompano Beach", "place_type": "city", "bounding_box": rectangle("-80.195561,26.206136 -80.08173,26.297654") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1258050, "cityName": "Pompano Beach" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520619400036352, "text": "Here we gooo, Ben fuckin Klock!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1375527470, "name": "Erika", "screen_name": "escapistdancer", "lang": "en", "location": "the DARK SIDE", "create_at": date("2013-04-23"), "description": "Too weird to live. Too rare to die.", "followers_count": 376, "friends_count": 347, "statues_count": 24236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520619815219200, "text": "49.3F (Feels: 49.3F) - Humidity: 85% - Wind: 0.0mph --- - Gust: 3.1mph - Pressure: 1018.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 233503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520619966275584, "text": "I'm chillin on snapchat to catch my brother rage. ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2598625242, "name": "Andres Suchil", "screen_name": "Suchi0220", "lang": "en", "location": "Andrews, TX", "create_at": date("2014-07-01"), "description": "The Come Up - RATT Pack - ⚾️", "followers_count": 76, "friends_count": 82, "statues_count": 1429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Andrews, TX", "id": "4c34ce2e6b4721d9", "name": "Andrews", "place_type": "city", "bounding_box": rectangle("-102.573252,32.296046 -102.501004,32.354076") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48003, "countyName": "Andrews", "cityID": 4803216, "cityName": "Andrews" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520619995566080, "text": "#gloryhole #foodie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "gloryhole", "foodie" }}, "user": { "id": 15534499, "name": "Karen Lopez", "screen_name": "datachick", "lang": "en", "location": "null", "create_at": date("2008-07-22"), "description": "Digital Smartieskirt, Architect, Consultant | Speaker | Space Enthusiast | Data Evangelist | Microsoft MVP | TFD I want you to Love Your Data! A bit snarky.", "followers_count": 11819, "friends_count": 3308, "statues_count": 208553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620150886401, "text": "I don't get why people hate FaceTiming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247904396, "name": "han ❁", "screen_name": "hannah_soltis", "lang": "en", "location": "Ohio, USA", "create_at": date("2011-02-05"), "description": "| Ohio University '20 | 1•1•14❤️", "followers_count": 1102, "friends_count": 779, "statues_count": 29777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redington Shores, FL", "id": "77f7a6acbae585d6", "name": "Redington Shores", "place_type": "city", "bounding_box": rectangle("-82.838441,27.822097 -82.818852,27.837206") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1259750, "cityName": "Redington Shores" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620184379393, "text": "WHAT THE FUCK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2894889066, "name": "morg", "screen_name": "morgannnlyonsss", "lang": "en", "location": "Christopher, IL", "create_at": date("2014-11-09"), "description": "sc: morganlyonsss", "followers_count": 473, "friends_count": 2009, "statues_count": 1814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chester, IL", "id": "0034ee0c179f0035", "name": "Chester", "place_type": "city", "bounding_box": rectangle("-89.859287,37.888979 -89.783947,37.938962") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17157, "countyName": "Randolph", "cityID": 1713139, "cityName": "Chester" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620373073920, "text": "@_sophfranco wth lets go", "in_reply_to_status": 716520575292690432, "in_reply_to_user": 3119815493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3119815493 }}, "user": { "id": 2335137532, "name": "Tonny Gomez", "screen_name": "TonnnyG_", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-02-09"), "description": "null", "followers_count": 214, "friends_count": 428, "statues_count": 7117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620402454528, "text": "I know of you.�� and I seem you around GR. https://t.co/95uz1E9Mem", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2665110518, "name": "juicy jas✨", "screen_name": "bvbyjas", "lang": "en", "location": "null", "create_at": date("2014-07-20"), "description": "null", "followers_count": 330, "friends_count": 413, "statues_count": 10871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greatwood, TX", "id": "6ef928bfe164fa3c", "name": "Greatwood", "place_type": "city", "bounding_box": rectangle("-95.74722,29.535478 -95.637861,29.583376") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4830806, "cityName": "Greatwood" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620519870464, "text": "You're so stupid ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319489234, "name": "POORRASS", "screen_name": "james_1618", "lang": "en", "location": "El Paso, TX", "create_at": date("2011-06-17"), "description": "Rochester, New York! Future National Geographic Photographer", "followers_count": 271, "friends_count": 500, "statues_count": 5190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620557635584, "text": "come and see me for once, you don't ever come to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2196936487, "name": "ian", "screen_name": "shwaggyian", "lang": "en", "location": "City of Angels ∞ ", "create_at": date("2013-11-15"), "description": "Appreciate the really, really tough times as much you appreciate the great times. - Kobe Bryant", "followers_count": 215, "friends_count": 199, "statues_count": 1208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:02.000Z"), "id": 716520620624744448, "text": "This guy playing ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447223527, "name": "Princess Adri ♡", "screen_name": "___WildOne", "lang": "en", "location": "KC ❤️", "create_at": date("2011-12-26"), "description": "Kia & Kyessa Bffl", "followers_count": 2379, "friends_count": 2964, "statues_count": 25217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621027500032, "text": "The need to make love and track ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261363865, "name": "D'Eldrick Campbell", "screen_name": "__DC9", "lang": "en", "location": "Florida, USA", "create_at": date("2011-03-05"), "description": "#TeamCampbell I do it for my dad R.I.P ❤️", "followers_count": 846, "friends_count": 514, "statues_count": 8894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Babson Park, FL", "id": "0196331f2ab777b0", "name": "Babson Park", "place_type": "city", "bounding_box": rectangle("-81.543622,27.82104 -81.520875,27.853979") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1202900, "cityName": "Babson Park" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621291741184, "text": "Wind 0.7 mph WSW. Barometer 29.34 in, Falling. Temperature 36.0 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621346140160, "text": "I just don't get it... What do you gain from lying???", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2900829534, "name": "Funny Guy", "screen_name": "A_aronBryant", "lang": "en", "location": "Nowhere", "create_at": date("2014-11-15"), "description": "Life may suck, but at least you're still here to do something about it", "followers_count": 94, "friends_count": 139, "statues_count": 2689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmundson, MO", "id": "6c44861562beca23", "name": "Edmundson", "place_type": "city", "bounding_box": rectangle("-90.371121,38.730548 -90.360263,38.740028") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2921376, "cityName": "Edmundson" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621694328834, "text": "Ugly garden https://t.co/dAhxtZwsuE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.545758,33.800433"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328856509, "name": "britknee", "screen_name": "Hellobreenyy", "lang": "en", "location": "Earth", "create_at": date("2011-07-03"), "description": "womyn not woman.", "followers_count": 397, "friends_count": 576, "statues_count": 12542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moorten Botanical Garden", "id": "07d9dcaa34c87000", "name": "Moorten Botanical Garden", "place_type": "poi", "bounding_box": rectangle("-116.5457581,33.8004329 -116.545758,33.800433") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621719425025, "text": "@justkhourtni_ it says you opened my snap at 6:55 pm https://t.co/kKQLE1HtaA", "in_reply_to_status": 716519374232752130, "in_reply_to_user": 459828429, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 459828429 }}, "user": { "id": 286396687, "name": "DJ Wally West", "screen_name": "TheMixedWonder", "lang": "en", "location": "Countdown City, Texas ", "create_at": date("2011-04-22"), "description": "Saved by God thru Jesus Christ. DJ. Marketing Manager for @OrderUpStilly *Tweets do not reflect the viewpoint of my employer* Pre-Law.", "followers_count": 1548, "friends_count": 1145, "statues_count": 48939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621719609345, "text": "#Trump in 2016 Temp:34.7°F Wind:0.0mph Pressure: 30.06hpa Rising Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621774123008, "text": "But wtf about me and how I feel.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 266149211, "name": "Edin Hodžić", "screen_name": "EdinHodzic6_", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-03-14"), "description": "| Chicago | LUC '19 | 18 | #NLBB", "followers_count": 3277, "friends_count": 2045, "statues_count": 47152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Skokie, IL", "id": "4b3758bc32868265", "name": "Skokie", "place_type": "city", "bounding_box": rectangle("-87.780869,42.0032 -87.708543,42.065033") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1770122, "cityName": "Skokie" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520621912403968, "text": "Behold the glory of Peter Dinklage's 'SNL' space pants https://t.co/FDNh02dGHL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17623, "friends_count": 17560, "statues_count": 70133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622017417216, "text": "fighting back against the browser back button w/ @randfish #cxllive https://t.co/oNMAq0VrEn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cxllive" }}, "user_mentions": {{ 6527972 }}, "user": { "id": 14139941, "name": "Leho Kraav", "screen_name": "lkraav", "lang": "en", "location": "Tallinn, Estonia", "create_at": date("2008-03-13"), "description": "I make websites sell and I make high-selling websites. Owner of http://t.co/Q9tUT7qELx website development co, partner at http://t.co/c80g1yOnpS optimization co", "followers_count": 307, "friends_count": 439, "statues_count": 1147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622361194496, "text": "I don't know and I don't want to know https://t.co/1ETe6Yh0Rp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1723727052, "name": "joanna", "screen_name": "joannnnax", "lang": "en", "location": "corcoran ", "create_at": date("2013-09-02"), "description": "love yourself ️", "followers_count": 3835, "friends_count": 232, "statues_count": 29194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622411501568, "text": "I'm so sensitive. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715396054, "name": "blade", "screen_name": "bladeraynelle", "lang": "en", "location": "North Hollywood, CA", "create_at": date("2013-10-14"), "description": "null", "followers_count": 1030, "friends_count": 559, "statues_count": 26012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622650695680, "text": "Take easy opportunities to succeed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1206622332, "name": "Santiago", "screen_name": "santi_71", "lang": "en", "location": "Miami, FL", "create_at": date("2013-02-21"), "description": "Argentina | Miami", "followers_count": 170, "friends_count": 171, "statues_count": 3739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Biscayne Park, FL", "id": "000db6740b1fec59", "name": "Biscayne Park", "place_type": "city", "bounding_box": rectangle("-80.191065,25.87231 -80.160462,25.887555") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1206600, "cityName": "Biscayne Park" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622793338880, "text": "I just ordered a pizza for myself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30394122, "name": "aandreaa", "screen_name": "andreadarmody", "lang": "en-gb", "location": "Stanford, CA", "create_at": date("2009-04-10"), "description": "Thank you for your interest in Andrea. Please know that Andrea is very interested in you.", "followers_count": 236, "friends_count": 228, "statues_count": 2182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stanford, CA", "id": "ed0a823456907b4c", "name": "Stanford", "place_type": "city", "bounding_box": rectangle("-122.191291,37.406339 -122.149162,37.443731") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 673906, "cityName": "Stanford" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520622940151808, "text": "Nardy ft rich boy // home of murder an Mardi Gras ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1108248138, "name": "كل شيء ممكن", "screen_name": "glo_gvng_gabe", "lang": "en", "location": "null", "create_at": date("2013-01-20"), "description": "ROMANS 8:18 THE PAIN YOU BEEN FEELING CAN NOT COMPARE TO THE JOY THATS COMMING #504-#251", "followers_count": 985, "friends_count": 362, "statues_count": 43074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520623187472384, "text": "Same! But with some gummies :) https://t.co/Ub5eVVnV2h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2661356412, "name": "selma", "screen_name": "ogeeechi_", "lang": "en", "location": "htx", "create_at": date("2014-07-19"), "description": "Nigerian , SC: itsselmaa , #prolife", "followers_count": 928, "friends_count": 819, "statues_count": 6614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greatwood, TX", "id": "6ef928bfe164fa3c", "name": "Greatwood", "place_type": "city", "bounding_box": rectangle("-95.74722,29.535478 -95.637861,29.583376") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4830806, "cityName": "Greatwood" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520624164720641, "text": "Just posted a photo @ Downtown Los Angeles https://t.co/2mThcYf3aH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.25117066,34.04630156"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19537057, "name": "Holli Duncan", "screen_name": "meemite", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-01-26"), "description": "null", "followers_count": 72, "friends_count": 62, "statues_count": 3266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-03T00:00:03.000Z"), "id": 716520624357650432, "text": "���� https://t.co/W4xSDBOxV0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3032089517, "name": "Chandni Patel", "screen_name": "chandnnni", "lang": "en", "location": "null", "create_at": date("2015-02-11"), "description": "null", "followers_count": 290, "friends_count": 270, "statues_count": 1850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520625288794114, "text": "@FoolyMike I have no clue ��", "in_reply_to_status": 716520517033861120, "in_reply_to_user": 446099318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 446099318 }}, "user": { "id": 1716379016, "name": ".", "screen_name": "TristanAngell", "lang": "en", "location": "null", "create_at": date("2013-08-31"), "description": "null", "followers_count": 910, "friends_count": 591, "statues_count": 82770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferris, TX", "id": "01bb16e5794f8f38", "name": "Ferris", "place_type": "city", "bounding_box": rectangle("-96.681464,32.527813 -96.649186,32.546059") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4825752, "cityName": "Ferris" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520625641119744, "text": "do not come for me if you don't even know me. you're irrelevant as helllll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4705102279, "name": "Alynna✨", "screen_name": "alynnaa_", "lang": "en", "location": "Riverside, CA", "create_at": date("2016-01-03"), "description": "null", "followers_count": 1159, "friends_count": 463, "statues_count": 6352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520625808887808, "text": "Happy Birthday Kam! Have fun being 16 and being able to drive me around������ love ya! https://t.co/bWVQWH0Qrs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2408368968, "name": "Champ", "screen_name": "ChampGrayson", "lang": "en", "location": "null", "create_at": date("2014-03-23"), "description": "Kamiakin", "followers_count": 557, "friends_count": 362, "statues_count": 4104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, WA", "id": "7291a25672e0d4b1", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-122.1649,47.626845 -121.958642,47.73078") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357535, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520625913749504, "text": "songs that give you chills ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2147952553, "name": "marisa ✨", "screen_name": "MarisaWatts_", "lang": "en", "location": "w/ my woes", "create_at": date("2013-10-21"), "description": "// i got hot sauce in my bag \\\\", "followers_count": 238, "friends_count": 102, "statues_count": 14621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Travis Ranch, TX", "id": "0081a5ba6572e762", "name": "Travis Ranch", "place_type": "city", "bounding_box": rectangle("-96.486355,32.791913 -96.456803,32.812339") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4873562, "cityName": "Travis Ranch" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520626018721792, "text": "Happy Birthday my baby love! Love you Tara. Too bad you deactivated your Instagram. ❤️ @ Gagopa… https://t.co/SgkNTSuIIc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.98701,40.7477"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331273222, "name": "Martina Barksdale", "screen_name": "_martiBarksdale", "lang": "en", "location": " New York, NY", "create_at": date("2011-07-07"), "description": "http://Actress.Host. Just trying to develop insomnia to keep up with this city that never sleeps.", "followers_count": 455, "friends_count": 335, "statues_count": 5141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520626215735296, "text": "HAPPY BIRTHDAY @BrennenTaylor hope you have the best day ever��❤️�� and bring me some cake!!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 396866544 }}, "user": { "id": 3366973168, "name": "Brenda ∞", "screen_name": "BrennensbooMaya", "lang": "en", "location": "San Francisco, CA", "create_at": date("2015-07-08"), "description": "|Just a Normal Fan Girl| Brenda is my real name sadly what is A Bio? *Brennen Girl* #1D", "followers_count": 185, "friends_count": 167, "statues_count": 5862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520626438078464, "text": "Definitely had the best date to the ΚΣ formal, and you're without a… https://t.co/xt708QVCWD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-100.43474587,31.46194178"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2421319333, "name": "Dev", "screen_name": "DevonNorwood", "lang": "en", "location": "San Angelo, TX", "create_at": date("2014-03-31"), "description": "Proverbs 31:25", "followers_count": 493, "friends_count": 508, "statues_count": 4705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Angelo, TX", "id": "4cb7f66ae6f6d487", "name": "San Angelo", "place_type": "city", "bounding_box": rectangle("-100.52941,31.335769 -100.376768,31.545276") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48451, "countyName": "Tom Green", "cityID": 4864472, "cityName": "San Angelo" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520626702266368, "text": "See yall tomorow!!!! @fatlace @fatlace @thesonnyb @romeov #YoungHawaii @ Ke‘ehi Lagoon Park https://t.co/jtF1ssIkzr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.89677625,21.33175532"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "YoungHawaii" }}, "user_mentions": {{ 15141980, 15141980, 28252086, 36804172 }}, "user": { "id": 3085911495, "name": "Kaipo", "screen_name": "kaipo__kapua", "lang": "en", "location": "Hawai'i", "create_at": date("2015-03-11"), "description": "Local Musician from Hawai'i... To book Kaipo Kapua email at Booking.Kaipo@Gmail.Com IG:@KaipoOfficial FB:KaipoKapua Youtube:KaipoKapua SnapChat:KaipoOfficial", "followers_count": 366, "friends_count": 293, "statues_count": 1015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520627000094720, "text": "Woowwww. https://t.co/0FrRYWuRX9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 414535734, "name": "BIG OG", "screen_name": "iamogkidd", "lang": "en", "location": "Houston, TX", "create_at": date("2011-11-16"), "description": "OG KIDD | IG: @Trill_Gxdd | Booking: 1400booking@gmail.com", "followers_count": 3194, "friends_count": 2156, "statues_count": 16170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520627247534081, "text": "@babyyflacko figures", "in_reply_to_status": 716520440496140288, "in_reply_to_user": 2941307030, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2941307030 }}, "user": { "id": 4663572109, "name": "Raya the Sagicorn", "screen_name": "_RayaTheGOAT24", "lang": "en", "location": "6thSideOfPicoRivera", "create_at": date("2015-12-27"), "description": "null", "followers_count": 585, "friends_count": 776, "statues_count": 23483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-04-03T00:00:04.000Z"), "id": 716520627469946880, "text": "من اللازم عليك تخطي الحُطام الذي يمر بك وكأنه لم يكن،الحياة لا تُقدم لك رسالة إعتذار وتأسف على مايجري ولا توعدك إن هذه أخر مره.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 1624388216, "name": "rodina", "screen_name": "y__r2", "lang": "ar", "location": "Nashville, TN", "create_at": date("2013-07-26"), "description": "null", "followers_count": 174, "friends_count": 44, "statues_count": 4666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Hill, TN", "id": "008300a34795739b", "name": "Spring Hill", "place_type": "city", "bounding_box": rectangle("-86.983702,35.664463 -86.867521,35.79496") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47119, "countyName": "Maury", "cityID": 4770580, "cityName": "Spring Hill" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882996301705218, "text": "it's all fun & games until you catch feelings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36581667, "name": "Makenna", "screen_name": "makkkymak", "lang": "en", "location": "oregon state university", "create_at": date("2009-04-29"), "description": "no more parties in LA", "followers_count": 870, "friends_count": 398, "statues_count": 13801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882996326838274, "text": "Exactly https://t.co/sRb11EZdT2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 84778100, "name": "Hector Ligsay Jr.", "screen_name": "Hectorium", "lang": "en", "location": "Oxnard, CA", "create_at": date("2009-10-23"), "description": "Instagram @Hectorligsayjr || I like to cook", "followers_count": 932, "friends_count": 686, "statues_count": 30319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882996612104192, "text": "soooooo sleepy gn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2719979400, "name": "paola salcedo", "screen_name": "paaowow_", "lang": "en", "location": "Coachella, CA/Fullerton, CA", "create_at": date("2014-08-09"), "description": "something cool", "followers_count": 709, "friends_count": 314, "statues_count": 28765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882997627072512, "text": "pretty chill https://t.co/lIUPlm2eht", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2746581902, "name": "TY", "screen_name": "TyMvtthews", "lang": "en", "location": "null", "create_at": date("2014-08-19"), "description": "null", "followers_count": 214, "friends_count": 158, "statues_count": 1846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882997669183492, "text": "@dreadhead31988 @KellyJalen dancing on these hoes https://t.co/nEWTCue6FM", "in_reply_to_status": 716882351918292992, "in_reply_to_user": 432214066, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 432214066, 1375909399 }}, "user": { "id": 1206719448, "name": "sQuale", "screen_name": "dxxtvylor", "lang": "en", "location": "Japan ✈ SoCal ✈ Raleigh ", "create_at": date("2013-02-21"), "description": "#TMT | 1/3 Da Frenchies©", "followers_count": 1829, "friends_count": 316, "statues_count": 44580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882997786517506, "text": "I'm hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244299594, "name": "Just ~Jaimmie", "screen_name": "Jaimmieee", "lang": "en", "location": "Bellflower, CA", "create_at": date("2011-01-28"), "description": "I may not be perfect, but at least I'm not you.", "followers_count": 256, "friends_count": 185, "statues_count": 7285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882998013009920, "text": "03:00 57.6°F Feels:55.4°F (Hi58.8°F/Lo57.6°F) Hum:55% Wnd:WSW 10.1MPH Baro:29.58in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 193, "friends_count": 263, "statues_count": 27866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882998965080064, "text": "Savage ass attitude & sensitive ass heart ��❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715142560, "name": "Zaryia ✨", "screen_name": "stunna_a1", "lang": "en", "location": "401 ⚓ Prov ", "create_at": date("2013-10-14"), "description": "SC : Shawtyx17 | ♋ | My Own Wce ❤", "followers_count": 1529, "friends_count": 1508, "statues_count": 19654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882998973444102, "text": "Opening night is finally here @Athletics let's go Oakland", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19607400 }}, "user": { "id": 300027487, "name": "Ryantdalton", "screen_name": "ryantdalton1", "lang": "en", "location": "null", "create_at": date("2011-05-16"), "description": "Im 23 love the A's warriors sharks and raiders #sharksfam #nfwbge warriors nba champions #dubnation", "followers_count": 568, "friends_count": 1970, "statues_count": 86511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999057522689, "text": "Let's take a trip to boca �� https://t.co/Vd7FEFYTIp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108759431, "name": "Petty Ci ❤️", "screen_name": "Ayeee_Cici", "lang": "en", "location": "null", "create_at": date("2010-01-26"), "description": "IG : @ayeee_cici SC: @ayeee_cici.", "followers_count": 1944, "friends_count": 936, "statues_count": 104998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Miami Heights, FL", "id": "576bb66a5b95f65a", "name": "South Miami Heights", "place_type": "city", "bounding_box": rectangle("-80.414746,25.566015 -80.356751,25.627599") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1267575, "cityName": "South Miami Heights" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999518699520, "text": "@SPYKXD @AcidAlien_ totally agree we should ��", "in_reply_to_status": 716882491680817152, "in_reply_to_user": 451835024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 451835024, 870714421 }}, "user": { "id": 851791429, "name": "NINO", "screen_name": "ninno___", "lang": "en", "location": "LA TO LV", "create_at": date("2012-09-28"), "description": "Heartless", "followers_count": 1307, "friends_count": 302, "statues_count": 46195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999577587712, "text": "03:00:01 |Temp: 42.8ºF | Wind Chill 42.8ºF |Dew Point 39.5ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999904579584, "text": "This is more impressive than steph curry to me... https://t.co/7rLT96jif0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505061272, "name": "Alejandro Vela", "screen_name": "aj_vela_", "lang": "en", "location": "null", "create_at": date("2012-02-26"), "description": "You've got to learn to live with what you can't rise above.\n#UH\n#GoPack \n#HookEm", "followers_count": 242, "friends_count": 423, "statues_count": 37866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999912980480, "text": "@MatthewMailen it can be mutiple things but I get death whabblez like a bitch now going like 35", "in_reply_to_status": 716882726956187648, "in_reply_to_user": 572864804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 572864804 }}, "user": { "id": 549758771, "name": "Richter", "screen_name": "BrandonRichter4", "lang": "en", "location": "Oak Creek Wisconsin ", "create_at": date("2012-04-09"), "description": "§tay l¤wkey & let them assμme.\n~sc richter_brandon", "followers_count": 457, "friends_count": 398, "statues_count": 18058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Creek, WI", "id": "05d07f0891432eaa", "name": "Oak Creek", "place_type": "city", "bounding_box": rectangle("-87.951617,42.83611 -87.842053,42.930439") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5558800, "cityName": "Oak Creek" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716882999963422720, "text": "Temp 22.2° Hi/Lo 25.8/22.2 Rng 3.6° WC 22.2° Hmd 78% Rain 0.00\" Storm 0.85\" BAR 30.056 Falling DP 16.4° Wnd 0mph Dir --- Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 67, "friends_count": 117, "statues_count": 17509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716883000114327552, "text": "@emigyaru i know! Especially the cookie!!", "in_reply_to_status": 716882920586223617, "in_reply_to_user": 197342089, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 197342089 }}, "user": { "id": 2740532664, "name": "✨ k o u ✨", "screen_name": "PetrifiedKou", "lang": "en-gb", "location": "Welcome to Kou's Apartment", "create_at": date("2014-08-17"), "description": "Airport sock enthusiast that is too poor for the socks |AX14-16 | disney | coffee&tea | @kevinkev1337 ♡| Haruhichan Writer | EN LLSIF139| terminal shortness |", "followers_count": 447, "friends_count": 326, "statues_count": 41853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716883000126930944, "text": "Many people posting about Osaka aquarium today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3260156390, "name": "Qyle James", "screen_name": "qylejames", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2015-06-29"), "description": "I honestly have a thing for rice crackers", "followers_count": 124, "friends_count": 125, "statues_count": 1667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Beach, HI", "id": "6bd73386ffaba450", "name": "Ewa Beach", "place_type": "city", "bounding_box": rectangle("-158.028613,21.306027 -157.990042,21.332114") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507450, "cityName": "Ewa Beach" } }
+{ "create_at": datetime("2016-04-04T00:00:00.000Z"), "id": 716883000235954176, "text": "@kyrajaronn text me I'm up", "in_reply_to_status": 716882842349756416, "in_reply_to_user": 2670788436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2670788436 }}, "user": { "id": 1547186419, "name": "ol'nappyhead✨", "screen_name": "boricuaenid", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "Joey ♡", "followers_count": 492, "friends_count": 374, "statues_count": 28268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883000605196289, "text": "04/04@03:00 - Temp 40.1F, WC 40.1F. Wind 1.2mph ENE, Gust 3.0mph. Bar 29.962in, Falling. Rain 0.00in. Hum 66%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001078984704, "text": "Sad spring break is coming to an end but then remembered professors are going on strike in a couple days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2684080500, "name": "Kyle Parra", "screen_name": "K_Parra_", "lang": "en", "location": "null", "create_at": date("2014-07-26"), "description": "CSULB", "followers_count": 447, "friends_count": 382, "statues_count": 10959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001137692672, "text": "���� Shuuut TheFuckUp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207843969, "name": "Olivia Basden", "screen_name": "OP_livvv", "lang": "en", "location": "Killa4nia.", "create_at": date("2010-10-25"), "description": "Get It Right, Get It Tight.", "followers_count": 326, "friends_count": 265, "statues_count": 52487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001280446464, "text": "Ripley SW Limestone Co. Temp: 44.8°F Wind:0.0mph Pressure: 998.5mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001401999360, "text": "Vote #Libertarian. Dump the R & D. https://t.co/LR8hngs7cv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Libertarian" }}, "user": { "id": 567930956, "name": "Craig Bowden", "screen_name": "CraigBowden2020", "lang": "en", "location": "North Ogden, UT", "create_at": date("2012-04-30"), "description": "Family, God, Country. Served in the Marines, author, Libertarian Candidate for Utah's 1st Congressional District.", "followers_count": 7246, "friends_count": 7928, "statues_count": 54477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Ogden, UT", "id": "381a4280f594326c", "name": "North Ogden", "place_type": "city", "bounding_box": rectangle("-111.987476,41.283801 -111.930827,41.338144") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4955100, "cityName": "North Ogden" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001435525120, "text": "I need sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2533663772, "name": "susana", "screen_name": "_princessusana", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-05-29"), "description": "18 || love yourself", "followers_count": 1241, "friends_count": 871, "statues_count": 13181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001813049344, "text": "this is fucking disgusting https://t.co/AgWEx8dk60", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3115893470, "name": "liyaa", "screen_name": "liya_cakes", "lang": "en", "location": "✨with thanyy✨", "create_at": date("2015-03-29"), "description": "gas, break, dip", "followers_count": 794, "friends_count": 520, "statues_count": 27309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Banos, CA", "id": "b1b6fc646de75904", "name": "Los Banos", "place_type": "city", "bounding_box": rectangle("-120.879833,37.033745 -120.803053,37.088185") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 644028, "cityName": "Los Banos" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883001867513856, "text": "I met blasé by the music building and we walked to the business in the cut and lordt he got all touchy feely my heart stopped and shit", "in_reply_to_status": 716882628800937985, "in_reply_to_user": 330653023, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330653023, "name": "The 25th Of April.", "screen_name": "JalynPersuasion", "lang": "en", "location": "Snatching Souls & Eating Fries", "create_at": date("2011-07-06"), "description": "Delicate Flower. A Queen Lil Bish.", "followers_count": 3566, "friends_count": 2837, "statues_count": 64727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daytona Beach, FL", "id": "5876e96f70283826", "name": "Daytona Beach", "place_type": "city", "bounding_box": rectangle("-81.133668,29.127565 -81.003444,29.252881") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1216525, "cityName": "Daytona Beach" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883002203045888, "text": "I'm very happy I love my friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 876132266, "name": "Blake", "screen_name": "b_naiman", "lang": "en", "location": "Nashville, TN", "create_at": date("2012-10-12"), "description": "19 | Belmont University | Prov. 3:3 | Irony", "followers_count": 421, "friends_count": 349, "statues_count": 13419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883002265985024, "text": "Who's the real #Democrat? #Clinton and #Clooney raise funds for the 'revolution'; Sanders attacks them\n\nhttps://t.co/wVo5kcCUSc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Democrat", "Clinton", "Clooney" }}, "user": { "id": 430325527, "name": "tandy key culpepper", "screen_name": "andytandy3", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-06"), "description": "A little of this, a little of that. Stirred, rarely shaken. Journalist, shill, enthusiast, iconoclast. Hollywood emigre.", "followers_count": 621, "friends_count": 1986, "statues_count": 993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883002349891584, "text": "I've been working 9-5s almost everyday. It kinda sucks. Definitely something I don't want to do in life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53766136, "name": "diana", "screen_name": "gl1ttr", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-07-04"), "description": "sc: glitterbreeze", "followers_count": 725, "friends_count": 135, "statues_count": 32845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883002815418369, "text": "@MarkYoung77 you gon spend da night sum time dis week or wha", "in_reply_to_status": 716882893818109952, "in_reply_to_user": 1363597279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1363597279 }}, "user": { "id": 4548824473, "name": "Laura", "screen_name": "laura_sandbox_", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "I like big butts. Why would anyone want to lie about liking big butts?", "followers_count": 130, "friends_count": 393, "statues_count": 198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valencia, NM", "id": "426cd232649d7461", "name": "Valencia", "place_type": "city", "bounding_box": rectangle("-106.715305,34.768766 -106.65837,34.817804") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35061, "countyName": "Valencia", "cityID": 3581800, "cityName": "Valencia" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883002840584192, "text": "@ebbtideapp Tide in Jamaica Beach, Texas 04/04/2016\nHigh 4:12am 0.9\n Low 11:59am 0.2\nHigh 6:07pm 0.9\n Low 12:21am 0.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-94.9833,29.2"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 49, "friends_count": 1, "statues_count": 21509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4837252, "cityName": "Jamaica Beach" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883003427856385, "text": "Today would have been 5 years of Kalin and Myles but …… its not I'm sad ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3138401622, "name": "K&M", "screen_name": "yourgirlaliice", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-04-03"), "description": "Bddt Bddt", "followers_count": 574, "friends_count": 315, "statues_count": 9702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883004052914176, "text": "Always wake up in the middle of the night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219846053, "name": "B", "screen_name": "BriHammm", "lang": "en", "location": "Garfield Heights, Ohio", "create_at": date("2010-11-25"), "description": "null", "followers_count": 4343, "friends_count": 2120, "statues_count": 201921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garfield Heights, OH", "id": "4bcff2f443c05374", "name": "Garfield Heights", "place_type": "city", "bounding_box": rectangle("-81.637998,41.384569 -81.56761,41.441812") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3929428, "cityName": "Garfield Heights" } }
+{ "create_at": datetime("2016-04-04T00:00:01.000Z"), "id": 716883004271013888, "text": "Con el hombre fuerte de la x96.3 @elpadrinoradio @ Oro Bar and Restaurant https://t.co/PC00wWuXBc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.90039825,40.82200241"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 112575837 }}, "user": { "id": 148650253, "name": "leticia serrano", "screen_name": "leslymusic", "lang": "en", "location": "null", "create_at": date("2010-05-27"), "description": "Soy una mujere decidida a Alcanzar mis metas y objetivos", "followers_count": 202, "friends_count": 193, "statues_count": 1938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883004644196352, "text": "I would honestly rather go to school tomorrow instead of feeling like shit from going a little to hard on springbreak����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4761781572, "name": "lil_lewi", "screen_name": "king_lewi365", "lang": "en", "location": "null", "create_at": date("2016-01-14"), "description": "FUCK A TIT", "followers_count": 221, "friends_count": 303, "statues_count": 286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanish Springs, NV", "id": "2ef2e027146be550", "name": "Spanish Springs", "place_type": "city", "bounding_box": rectangle("-119.735925,39.621614 -119.675337,39.6768") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268350, "cityName": "Spanish Springs" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883004904214528, "text": "Julion Alvarez is my favorite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1030899996, "name": "Ashley", "screen_name": "AshleyCuevaass", "lang": "en", "location": "w. claris", "create_at": date("2012-12-23"), "description": "null", "followers_count": 452, "friends_count": 296, "statues_count": 16096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883004963090433, "text": "Temp: 48.5°F Wind:0.0mph Pressure: 30.147hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005059543040, "text": "Can I smash these buns in peace?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1206719448, "name": "sQuale", "screen_name": "dxxtvylor", "lang": "en", "location": "Japan ✈ SoCal ✈ Raleigh ", "create_at": date("2013-02-21"), "description": "#TMT | 1/3 Da Frenchies©", "followers_count": 1829, "friends_count": 316, "statues_count": 44581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005202038784, "text": "YASSSSS https://t.co/FAIdKvnp8r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 482640889, "name": "Jazmin Aviña", "screen_name": "jazzzzxoxo", "lang": "en", "location": "Winters, CA", "create_at": date("2012-02-03"), "description": "sc: jazminavinaa", "followers_count": 618, "friends_count": 329, "statues_count": 27552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winters, CA", "id": "05ecc92d5f11c46a", "name": "Winters", "place_type": "city", "bounding_box": rectangle("-121.993012,38.513068 -121.953083,38.538069") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686034, "cityName": "Winters" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005227286528, "text": "Temp 34.8°F Wind Chill 34.8°F RH 79% Wind 0.0 --- Gust 0.0 --- SLP 30.087 in Steady Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 106, "friends_count": 63, "statues_count": 34452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005239721985, "text": "��dogs https://t.co/6FLn5yV1tq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1853246622, "name": "ZORPTHELIZARDGOD", "screen_name": "kingfresa123", "lang": "en", "location": "null", "create_at": date("2013-09-10"), "description": "⚒⚒⚒ Orgullo Mexicano", "followers_count": 252, "friends_count": 458, "statues_count": 17470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Puente, CA", "id": "e04653ebc39b100c", "name": "La Puente", "place_type": "city", "bounding_box": rectangle("-117.98558,34.009223 -117.920899,34.057298") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640340, "cityName": "La Puente" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005277663232, "text": "Wind 4.0 mph SW. Barometer 29.929 in, Falling slowly. Temperature 56.5 °F. Rain today 0.00 in. Humidity 49%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883005374132224, "text": "Wind 3.0 mph S. Barometer 29.833 in, Steady. Temperature 66.8 °F. Rain today 0.00 in. Humidity 35%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883006246400000, "text": "☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️☹️", "in_reply_to_status": 716882841020211200, "in_reply_to_user": 60438802, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 60438802, "name": "Augy ✨", "screen_name": "okaygayy", "lang": "en", "location": "Cedar Falls, Iowa", "create_at": date("2009-07-26"), "description": "Sales Manager, EXP. Iowa captive. University of Northern Iowa. Genderqueer- they/them/their. I sometimes tweet funny things, and sometimes I'm a rude bitch.", "followers_count": 292, "friends_count": 321, "statues_count": 12909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Falls, IA", "id": "8ddaffc6aef377de", "name": "Cedar Falls", "place_type": "city", "bounding_box": rectangle("-92.499974,42.464484 -92.406204,42.567828") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1911755, "cityName": "Cedar Falls" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883006267400193, "text": "أرى الإعلام يرفع كل وغد ويخفض كل ذي زنة شريفة", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 340601288, "name": "أحمد الزهيري", "screen_name": "a3z_11", "lang": "ar", "location": "Foster City, CA", "create_at": date("2011-07-22"), "description": "Love & Peace الحب والسلام", "followers_count": 580, "friends_count": 233, "statues_count": 12852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Foster City, CA", "id": "59bb4e6ce17a8b66", "name": "Foster City", "place_type": "city", "bounding_box": rectangle("-122.288439,37.533497 -122.245136,37.575839") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 625338, "cityName": "Foster City" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883006774837249, "text": "Haven't turned up in a minuteeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312162854, "name": "Ivan", "screen_name": "Enriquez10__", "lang": "en", "location": "milehigh is where Im stationed", "create_at": date("2015-08-10"), "description": "null", "followers_count": 570, "friends_count": 591, "statues_count": 2806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007152455680, "text": "@summerrowenss prom hair 2k16? https://t.co/pNKl5SJWtv", "in_reply_to_status": -1, "in_reply_to_user": 611139197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 611139197 }}, "user": { "id": 411864583, "name": "SassQueenSavageOwens", "screen_name": "Shannon_O3", "lang": "en", "location": "null", "create_at": date("2011-11-13"), "description": "______less drama and more bellinis______ _________________VII.XXII.IV_______________ _____Scottsdale, AZ || Florence, Italy____", "followers_count": 586, "friends_count": 391, "statues_count": 10168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007173496832, "text": "Wind 0.0 mph W. Barometer 30.01 in, Falling. Temperature 32.5 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007454351362, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 466006841, "name": ".", "screen_name": "AllOfTheGhouls", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "My eyes don't recognize you no more", "followers_count": 503, "friends_count": 594, "statues_count": 67823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007802478594, "text": "Wind 3.0 mph N. Barometer 29.891 in, Rising. Temperature 48.8 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 2381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007873818624, "text": "@NaRiNe_94 HAPPY BIRTHDAY NARINE !!!! ������ HAVE A GOOD ONE AND ASTVATS ORNI BAHI KEZ", "in_reply_to_status": -1, "in_reply_to_user": 1638399468, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1638399468 }}, "user": { "id": 2436215135, "name": "KOSRAE", "screen_name": "_utwe", "lang": "en", "location": "Hawaii University ", "create_at": date("2014-03-25"), "description": "Stay Positive Fam", "followers_count": 379, "friends_count": 217, "statues_count": 11068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883007940861952, "text": "young niggaz without kids , baby girl ima meal ticket ��⚾️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170222422, "name": "jah•bless ▪️", "screen_name": "KiNGJarren", "lang": "en", "location": "Separate Myself : NO✈️HTX", "create_at": date("2010-07-24"), "description": "null", "followers_count": 696, "friends_count": 117, "statues_count": 14810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008037470208, "text": "Still. https://t.co/5JpdAUlYa8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413258219, "name": "CaseynAmari", "screen_name": "CaseAmari", "lang": "en", "location": "Tennessee", "create_at": date("2011-11-15"), "description": "Prayer & Patience.", "followers_count": 5907, "friends_count": 294, "statues_count": 6835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008062509056, "text": "@dessy_smalls ��������", "in_reply_to_status": 716882846556643328, "in_reply_to_user": 3305820542, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3305820542 }}, "user": { "id": 2950940744, "name": "lorena", "screen_name": "ddylauren", "lang": "en", "location": "Pico Rivera, CA", "create_at": date("2014-12-29"), "description": "cheesin'", "followers_count": 326, "friends_count": 276, "statues_count": 8710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008146386944, "text": "Ladies Imma need y'alls help for this Gucci Mane concert... pray that we get it. #ThisWeek ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThisWeek" }}, "user": { "id": 118237954, "name": "Cotton Boulevardez", "screen_name": "Shawn_Cotton", "lang": "en", "location": "South Venice Beach", "create_at": date("2010-02-27"), "description": "I'm a bum.. i have 4 kids & I don't have a job", "followers_count": 9334, "friends_count": 1012, "statues_count": 192088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008356102144, "text": "@Ahmed_Mutiny omg I hope y'all aren't naked", "in_reply_to_status": 716882423275868162, "in_reply_to_user": 558863615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 558863615 }}, "user": { "id": 606638955, "name": "Sandbag", "screen_name": "PrenupShawty", "lang": "en", "location": "Unfortunate, Texas", "create_at": date("2012-06-12"), "description": "//19//i love my dog more than anything in the world// I do makeup// snap: @jinasoccer25 //follow my makeup insta. link down below", "followers_count": 1387, "friends_count": 279, "statues_count": 80659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008427413505, "text": "Colin hates me and has his window open on the highway. I'm so cold.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322303543, "name": "Dancing Queen", "screen_name": "MirBishop", "lang": "en", "location": "Butte Montana ", "create_at": date("2011-06-22"), "description": "eating pizza and fucking shit up since '97", "followers_count": 269, "friends_count": 214, "statues_count": 5581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-04T00:00:02.000Z"), "id": 716883008620343297, "text": "@Pawelotti Hahahaha. They beat my ass. #fact", "in_reply_to_status": 716882884213272576, "in_reply_to_user": 18107306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fact" }}, "user_mentions": {{ 18107306 }}, "user": { "id": 594280670, "name": "Justin Caouette", "screen_name": "JustinCaouette", "lang": "en", "location": "Calgary, Alberta", "create_at": date("2012-05-29"), "description": "PhD candidate in Philosophy (Massachusetts native) Tweeting: Ethics, Enhancement, Free Will, Mind, Emotions, Punishment, Academia, Psychology, Neuroscience.", "followers_count": 2487, "friends_count": 982, "statues_count": 26110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009018814465, "text": "@CassidyKlise bless https://t.co/9gNcnI0UjG", "in_reply_to_status": -1, "in_reply_to_user": 2167747400, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2167747400 }}, "user": { "id": 2936676349, "name": "chayse marie", "screen_name": "chayse_russell_", "lang": "en", "location": "gemini ", "create_at": date("2014-12-19"), "description": "daddy's little girl ain't a girl no more", "followers_count": 649, "friends_count": 276, "statues_count": 11483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redwood, OR", "id": "c77b5fda9c52afc5", "name": "Redwood", "place_type": "city", "bounding_box": rectangle("-123.420861,42.397624 -123.348505,42.437368") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41033, "countyName": "Josephine", "cityID": 4161250, "cityName": "Redwood" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009392091139, "text": "Jumping up & running out the room https://t.co/HAvVxnaa32", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2293096613, "name": "Lexx❤", "screen_name": "slimmsexyy", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "Real antisocial . Ion fuck around ✌ Instagram: @__lexxiiissss", "followers_count": 2318, "friends_count": 1282, "statues_count": 87960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baker, LA", "id": "ef948a8dce60ec09", "name": "Baker", "place_type": "city", "bounding_box": rectangle("-91.21205,30.55635 -91.115189,30.612501") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2203985, "cityName": "Baker" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009459200001, "text": "Just tried to say \"Mark\" and \"Jack\" at the same time, & it inexplicably came out as \"Mike.\" I don't even know. @markiplier @Jack_Septic_Eye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.14599879,34.12612613"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517077573, 77596200 }}, "user": { "id": 14410226, "name": "Kameko Hamato", "screen_name": "HamatoKameko", "lang": "en", "location": "Los Angeles, California, USA", "create_at": date("2008-04-16"), "description": "32. Professional fangirl. Obsessions include Steven Universe, Ninja Turtles, Sonic the Hedgehog, Invader Zim, Lordi, Minecraft, Markiplier. Also known as Ink.", "followers_count": 80, "friends_count": 180, "statues_count": 8967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009585152000, "text": "I'm up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 938173387, "name": "Smokey", "screen_name": "DannyBoyy", "lang": "en", "location": "null", "create_at": date("2012-11-09"), "description": "null", "followers_count": 743, "friends_count": 912, "statues_count": 40448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009652166656, "text": "51.5F (Feels: 51.5F) - Humidity: 85% - Wind: 0.0mph --- - Gust: 1.6mph - Pressure: 1017.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 233647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009706700801, "text": "I'd do you just right��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1353559987, "name": "maryfer", "screen_name": "maryferl60", "lang": "en", "location": "with my bby cay", "create_at": date("2013-04-14"), "description": "Made in the USA with Mexican parts || JGHS // XC,Track, Soccer // RIP BWS❤️", "followers_count": 1512, "friends_count": 1474, "statues_count": 23460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westland, MI", "id": "879dfee0187b884a", "name": "Westland", "place_type": "city", "bounding_box": rectangle("-83.430458,42.267434 -83.308978,42.355177") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2686000, "cityName": "Westland" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009761255424, "text": "My nipple has been itching for the past week or so and I'm about ready to rip this piercing off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3561682272, "name": "Ivana Rey.", "screen_name": "_holaaivanitaaa", "lang": "en", "location": "San Marcos, TX", "create_at": date("2015-09-14"), "description": "Young Metro trust me..", "followers_count": 568, "friends_count": 549, "statues_count": 7885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kyle, TX", "id": "94ef42cc204d5195", "name": "Kyle", "place_type": "city", "bounding_box": rectangle("-97.896063,29.942852 -97.830853,30.035633") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4839952, "cityName": "Kyle" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883009845092352, "text": "Nah family guy still on https://t.co/WbHcUyFQlP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1357221258, "name": "Joe Joe", "screen_name": "LuceroJoey", "lang": "en", "location": "null", "create_at": date("2013-04-16"), "description": "San Diego", "followers_count": 587, "friends_count": 544, "statues_count": 9022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010042212353, "text": "Nick's mom made me some bomb ass tea tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532404100, "name": "BABYM.XVII", "screen_name": "MarinaNicoleRic", "lang": "en", "location": "West Valley City, UT", "create_at": date("2013-06-19"), "description": "Прочность", "followers_count": 1272, "friends_count": 980, "statues_count": 29959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010054782977, "text": "Join the The TJX Companies team! See our latest #BusinessMgmt #job opening here: https://t.co/2Reu0ZEW3d #Pittston, PA #Veterans #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.7893604,41.3259134"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BusinessMgmt", "job", "Pittston", "Veterans", "Hiring" }}, "user": { "id": 2591769673, "name": "Jobs at TJX", "screen_name": "JobsatTJX", "lang": "en", "location": "null", "create_at": date("2014-06-27"), "description": "@TJXCareers is the leading off-price apparel and home fashions retailer worldwide. You can join us by applying to our open #jobs here.", "followers_count": 361, "friends_count": 2, "statues_count": 4625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittston, PA", "id": "2ad6133c9988753b", "name": "Pittston", "place_type": "city", "bounding_box": rectangle("-75.806941,41.31025 -75.775879,41.344956") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4261048, "cityName": "Pittston" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010080079872, "text": "@_Tiffaney15 @nwgtp_bri nasty", "in_reply_to_status": 716882720073388032, "in_reply_to_user": 2835624656, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user_mentions": {{ 2835624656, 565907175 }}, "user": { "id": 377676446, "name": "Kellz", "screen_name": "KSwank_", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "@IndigoElephants❤ #NCAT", "followers_count": 350, "friends_count": 298, "statues_count": 10977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010235174912, "text": "@hokagebacy , Ima start , soon", "in_reply_to_status": 716882363297320960, "in_reply_to_user": 1093584012, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1093584012 }}, "user": { "id": 333222073, "name": "X", "screen_name": "Leeean_", "lang": "en", "location": "Houston, TX", "create_at": date("2011-07-10"), "description": "#TxSU19 | wait on it ... | #Durty4", "followers_count": 1913, "friends_count": 892, "statues_count": 36668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010277093377, "text": "someone message me �� or call bc texting gets annoying after a while �� (preferably ladies, ukhtis)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3926658312, "name": "carina", "screen_name": "habibti_carina", "lang": "en", "location": "Hempstead, NY", "create_at": date("2015-10-17"), "description": "17 | Qur'an 93:7 | a Salvadoran revert to islam ☪ #Latina | LongIsland , NY 》DTX", "followers_count": 198, "friends_count": 130, "statues_count": 4278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010352717824, "text": "We are crazy, I swear! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1673892086, "name": "C . Ѡ ∈ § †", "screen_name": "i_CeceB", "lang": "en", "location": "Lil Haiti, Miami", "create_at": date("2013-08-15"), "description": "God First ✊| ♒9+10=21 | Haitian Queen | SC: CeJizzle", "followers_count": 2385, "friends_count": 2108, "statues_count": 27805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010428252160, "text": "A little of both https://t.co/r3AOzKit9M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3924629895, "name": "{BabyJesus}", "screen_name": "rdygambino", "lang": "en", "location": "null", "create_at": date("2015-10-10"), "description": "null", "followers_count": 74, "friends_count": 148, "statues_count": 1563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Point, GA", "id": "dfb4df427ea8a2d6", "name": "East Point", "place_type": "city", "bounding_box": rectangle("-84.535534,33.609487 -84.415048,33.705685") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1325720, "cityName": "East Point" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010465849345, "text": "#Trump in 2016 Temp:51.8°F Wind:1.3mph Pressure: 29.97hpa Falling slowly Rain Today 0.00in. Forecast: Precipitation at times, very unsettl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883010809765888, "text": "@500daysofallen he still hasn't texted me..��", "in_reply_to_status": 716862356194594818, "in_reply_to_user": 842210983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 842210983 }}, "user": { "id": 842210983, "name": "ferris", "screen_name": "500daysofallen", "lang": "en", "location": "santa monica", "create_at": date("2012-09-23"), "description": "✨catch me at #EDCLV16! snap: kickassallen", "followers_count": 657, "friends_count": 131, "statues_count": 44075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Park, CA", "id": "7b85d19c09869d20", "name": "Walnut Park", "place_type": "city", "bounding_box": rectangle("-118.230699,33.959223 -118.210743,33.974727") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683402, "cityName": "Walnut Park" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883012076457984, "text": "https://t.co/z53VfQwaKR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 879249948, "name": "Andy Baughman", "screen_name": "andy_baughman", "lang": "en", "location": "null", "create_at": date("2012-10-13"), "description": "You know, I like his tweets, except for that nervous fellow who's always in them", "followers_count": 352, "friends_count": 447, "statues_count": 11862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catalina Foothills, AZ", "id": "5a51ae834390a143", "name": "Catalina Foothills", "place_type": "city", "bounding_box": rectangle("-110.980121,32.250309 -110.805635,32.355554") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 411230, "cityName": "Catalina Foothills" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883012235857921, "text": "Happy birthday grandma, forever missing you �� got but never forgotten ����✝", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1087569740, "name": "Adriana❤️", "screen_name": "adribear_", "lang": "en", "location": "Tacoma, WA", "create_at": date("2013-01-13"), "description": "snapchat: Adriana-2 || 49ers", "followers_count": 344, "friends_count": 430, "statues_count": 4008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883012487491584, "text": "@babygurlaby brat", "in_reply_to_status": 716882920414138368, "in_reply_to_user": 3790950739, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3790950739 }}, "user": { "id": 2959518963, "name": "breezy a. lara", "screen_name": "breezycassh", "lang": "en", "location": "null", "create_at": date("2015-01-03"), "description": "they all hate me cause I got the city wavy", "followers_count": 315, "friends_count": 404, "statues_count": 2429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-04T00:00:03.000Z"), "id": 716883012865032193, "text": "So my lead training started today and it wasn't all that bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2266762362, "name": "John", "screen_name": "IvarraJohn", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-12-28"), "description": "Low Life", "followers_count": 1277, "friends_count": 841, "statues_count": 54351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883013745795073, "text": "I have a feeling they dont talk xcept for d simple text b4 Eat Bulaga para meron pagusapan, kaya nga may nagreklamo �� #ALDUBTheSearchIsOver", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBTheSearchIsOver" }}, "user": { "id": 23565558, "name": "Xtine Mac", "screen_name": "xtine76", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-09"), "description": "a daughter, a sister, a cool aunt, a crazy friend, a singer, a culture vulture, a follower of Pop Culture...ex Professional Bugaw - Think Happy :)", "followers_count": 196, "friends_count": 701, "statues_count": 2275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883013993271296, "text": "iCE Is not a noun !\nit's a life$tyle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268134032, "name": "iNDIKA", "screen_name": "sambamlocc", "lang": "en", "location": "blowing up", "create_at": date("2011-03-17"), "description": "BASEBOY. Insta & SC equals Indikasam JIGGYNIGGAiBeenIcinSWAGSEXUAL✌️ https://m.soundcloud.com/sambam-2/india-waseem", "followers_count": 1769, "friends_count": 1036, "statues_count": 51477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014001623040, "text": "Want change I'll change", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1022761242, "name": "Princess Brenda", "screen_name": "Brendacab94", "lang": "en", "location": "null", "create_at": date("2012-12-19"), "description": "God❤️ Robert Marin❤️ Have patience God isnt finished yet Philippians 1:6", "followers_count": 159, "friends_count": 267, "statues_count": 5637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alondra Park, CA", "id": "19e03c5f2d867530", "name": "Alondra Park", "place_type": "city", "bounding_box": rectangle("-118.343843,33.877306 -118.326441,33.901741") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601150, "cityName": "Alondra Park" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014039396352, "text": "too many to even answer that �� https://t.co/selkThxvC2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1274136878, "name": "Spence", "screen_name": "sbh127", "lang": "en", "location": "null", "create_at": date("2013-03-16"), "description": "• Albany, OR •", "followers_count": 562, "friends_count": 288, "statues_count": 1305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albany, OR", "id": "6083b03ae37cd913", "name": "Albany", "place_type": "city", "bounding_box": rectangle("-123.159583,44.577589 -123.032415,44.684678") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4101000, "cityName": "Albany" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014265884672, "text": "I'm crying https://t.co/ZvsObc9q4i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146508935, "name": "Lil Sad Poet Boi", "screen_name": "DNLCHRKSHYN", "lang": "en", "location": "Boolin' with my girl", "create_at": date("2015-04-07"), "description": "I'm just living life, y'all punk motherfuckers hatin.", "followers_count": 159, "friends_count": 135, "statues_count": 15868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014374952960, "text": "but then I get to find out what the hell is going on with me, plus?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2680749604, "name": "Tina ✨", "screen_name": "tinaa2ootruu", "lang": "en", "location": "null", "create_at": date("2014-07-06"), "description": "null", "followers_count": 253, "friends_count": 233, "statues_count": 4563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014920196097, "text": "No one talk to me at school tomorrow tbh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236705118, "name": "Toño", "screen_name": "Bermudez2k", "lang": "en", "location": "Quartz Hill, CA", "create_at": date("2013-12-08"), "description": "Evolution S.C.//", "followers_count": 399, "friends_count": 488, "statues_count": 5173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883014991552512, "text": "I was never perfect nor do I aim to be. I give my best to anything & everything that comes my way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3417118546, "name": "Graciela", "screen_name": "ciela____", "lang": "en", "location": "null", "create_at": date("2015-08-11"), "description": "null", "followers_count": 242, "friends_count": 239, "statues_count": 2674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883015025037313, "text": "It gets me hyped when people say I'm not down , lmfao say I'm not down and I'll do it fasho !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272273182, "name": "Diana Rodriguez", "screen_name": "eyy_diana", "lang": "en", "location": "Fontana, CA", "create_at": date("2014-01-01"), "description": "milly mouse ❤️ || sc ; eyy_diana", "followers_count": 370, "friends_count": 360, "statues_count": 8555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883015297736704, "text": "Strong believer in second chances, we all fuck up at some point and sometimes you just have to forgive and forget. No ones perfect.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 137553355, "name": "Andrea Gomez", "screen_name": "AndreaxAlicia", "lang": "en", "location": " Anaheim, CA", "create_at": date("2010-04-26"), "description": "null", "followers_count": 566, "friends_count": 373, "statues_count": 10421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883015905976320, "text": "Wish I had a cuddly little cat to sleep with rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 435376933, "name": "Emily LoRusso", "screen_name": "emmm_azing", "lang": "en", "location": "Oak Forest, IL / NIU AΣA", "create_at": date("2011-12-12"), "description": "I talk way too much", "followers_count": 1678, "friends_count": 902, "statues_count": 27925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DeKalb, IL", "id": "76cec30d4c553725", "name": "DeKalb", "place_type": "city", "bounding_box": rectangle("-88.792846,41.883102 -88.714573,41.972512") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1719161, "cityName": "DeKalb" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883016233017344, "text": "#SpringBreak2016 #Jamaica #CostaMaya #Cozumel https://t.co/eBof2gddrb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SpringBreak2016", "Jamaica", "CostaMaya", "Cozumel" }}, "user": { "id": 99989392, "name": "Ke", "screen_name": "TeamMinaj_HTown", "lang": "en", "location": "htx", "create_at": date("2009-12-28"), "description": "the Pinkprint", "followers_count": 3263, "friends_count": 348, "statues_count": 141900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883016551759872, "text": "I'm so happy right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486257019, "name": "salem", "screen_name": "loovejewels", "lang": "en", "location": "LA", "create_at": date("2012-02-07"), "description": "losing trust in human touch. Michelle is my best friend.", "followers_count": 992, "friends_count": 2427, "statues_count": 47350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883017059291139, "text": "ALWAYS LMAO https://t.co/VSakrVV38K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2901963929, "name": "alejandra", "screen_name": "alenicole___", "lang": "en", "location": "FW WF", "create_at": date("2014-12-01"), "description": "always stay humble and kind.", "followers_count": 771, "friends_count": 620, "statues_count": 18528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-04-04T00:00:04.000Z"), "id": 716883017155887104, "text": "I don't love myself fr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2177528111, "name": "Mahdi Mohamed", "screen_name": "NaSirYara", "lang": "en", "location": "Boston, MA", "create_at": date("2013-11-11"), "description": "~Somali", "followers_count": 530, "friends_count": 484, "statues_count": 6555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883017302548480, "text": "Idk how work is gonna go for me tomorrow ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175963101, "name": "LexxLaLatina", "screen_name": "Babyyy_Lexx", "lang": "en", "location": "null", "create_at": date("2010-08-07"), "description": "Richmond,CA", "followers_count": 549, "friends_count": 462, "statues_count": 16941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883017994608640, "text": "Its Bedtime Now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218253279, "name": "MK ⚓️", "screen_name": "ThaRealMoesha", "lang": "en", "location": "null", "create_at": date("2010-11-21"), "description": "B E A U T Y| #Alcorn19", "followers_count": 1947, "friends_count": 2046, "statues_count": 64434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallulah, LA", "id": "635b800a9767abcf", "name": "Tallulah", "place_type": "city", "bounding_box": rectangle("-91.208068,32.386504 -91.169316,32.425418") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22065, "countyName": "Madison", "cityID": 2274690, "cityName": "Tallulah" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883018015719424, "text": "Wind 0.0 mph ---. Barometer 30.039 in, Steady. Temperature 10.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 5, "friends_count": 2, "statues_count": 9784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883018162417665, "text": "Carajo es esto?? Los polluelos llevan 4 derrotas dolorosas...#BeisbolAA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.2478284,18.1482305"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "BeisbolAA" }}, "user": { "id": 129961241, "name": "Noel", "screen_name": "YoSoyNoelJr", "lang": "en", "location": "#Ingobernables , Puerto Rico", "create_at": date("2010-04-05"), "description": "Wrestling,Music,Anime lover, Ocassional Gamer,songwriter,music editor ,Content editor in CornerTv, hablo español,english and some 日本人", "followers_count": 530, "friends_count": 928, "statues_count": 18058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Caonillas, Puerto Rico", "id": "bbb1f49702216665", "name": "Caonillas", "place_type": "city", "bounding_box": rectangle("-66.281898,18.143808 -66.246697,18.17373") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72009, "countyName": "Aibonito" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883018162438145, "text": "#Rockaway, NJ #SkilledTrade #Job: Project Coordinator at Tyco https://t.co/2uIwUj7UZT #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.5143232,40.9012101"), "retweet_count": 0, "lang": "da", "is_retweet": false, "hashtags": {{ "Rockaway", "SkilledTrade", "Job", "Jobs", "Hiring" }}, "user": { "id": 1011811424, "name": "Tyco Careers", "screen_name": "TycoCareers", "lang": "en", "location": "North America", "create_at": date("2012-12-14"), "description": "The world’s largest pure-play fire and security company is looking for talented people. Join the winning team!", "followers_count": 1069, "friends_count": 137, "statues_count": 9332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockaway, NJ", "id": "ea767b0daf0dda36", "name": "Rockaway", "place_type": "city", "bounding_box": rectangle("-74.547994,40.87907 -74.496653,40.912041") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34027, "countyName": "Morris", "cityID": 3464050, "cityName": "Rockaway" } }
+{ "create_at": datetime("2016-04-04T00:00:05.000Z"), "id": 716883018611208192, "text": "\"Do my makeup next!\" https://t.co/iSHfQ0DfdE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1719810122, "name": "Alexa", "screen_name": "poubellealexa", "lang": "en", "location": "Modesto, CA", "create_at": date("2013-09-01"), "description": "null", "followers_count": 575, "friends_count": 273, "statues_count": 21755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245385140342784, "text": "@BoxFred it's looks like it ! That's awesome though how many states are you going to ?", "in_reply_to_status": 717244927885770752, "in_reply_to_user": 382518291, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 382518291 }}, "user": { "id": 166315772, "name": "Monica Renee", "screen_name": "iam_medusa", "lang": "en", "location": "null", "create_at": date("2010-07-13"), "description": "null", "followers_count": 191, "friends_count": 107, "statues_count": 10599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245385299726336, "text": "I'm really good now at making urban dictionary definitions about people :)))", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075838659, "name": "Omieeeee", "screen_name": "omie__g", "lang": "en", "location": "Glendora, CA", "create_at": date("2013-01-09"), "description": "Δ Σ Δ", "followers_count": 568, "friends_count": 325, "statues_count": 8398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245386608381953, "text": "#Healthcare in #Washington, DC: CLINICAL DIETITIAN / WASHINGTON, DC. at Compass Group USA https://t.co/avQcu8hCQq #jobhunt #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.6637765,39.244853"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Washington", "jobhunt", "Hiring" }}, "user": { "id": 126371773, "name": "CompassUSAJobBoard", "screen_name": "CompassJobBoard", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-03-25"), "description": "Welcome to Compass Group USA's job board. To learn about our people and careers go to @CGCareersUSA", "followers_count": 830, "friends_count": 124, "statues_count": 16727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, WV", "id": "64f5357060ac4b3c", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-81.681549,39.225056 -81.645467,39.253489") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54107, "countyName": "Wood", "cityID": 5484724, "cityName": "Washington" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245386658697216, "text": "I'm tweeting too much. Time to go ✌", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362646529, "name": "✨Daj", "screen_name": "_LoveMyAmbition", "lang": "en", "location": "Cleveland To BG Temporarily", "create_at": date("2011-08-26"), "description": "I Am A Child Of God, A Strong-Minded & Beautiful Woman! Known To Be Ambitious Because I Have A Desire For Success! 23 Years Old, Entrepreneur", "followers_count": 1088, "friends_count": 642, "statues_count": 38775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245386662871040, "text": "We're #hiring! Read about our latest #job opening here: Center Store Associate - https://t.co/QzLTUDkKqg #HAMMOND, LA #Retail", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.4868444,30.502434"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "HAMMOND", "Retail" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 530, "friends_count": 251, "statues_count": 18066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245386864222208, "text": "fuck PND https://t.co/vBThLAWtDs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2682014444, "name": "WBA", "screen_name": "TheRealWBA", "lang": "en", "location": "Attic", "create_at": date("2014-07-26"), "description": "mafia", "followers_count": 302, "friends_count": 385, "statues_count": 6984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245386994368512, "text": "#Dallas (@ Dallas, TX in Dallas, TX) https://t.co/q0C13j2JWz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.79948568,32.7805637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dallas" }}, "user": { "id": 6256732, "name": "M. Orozco", "screen_name": "MOrozcoSecurity", "lang": "en", "location": "Southern California Based (LA)", "create_at": date("2007-05-23"), "description": "Executive Protection Professional 10 years • Tour Security Professional 4 years • US Navy Veteran 8 years • Law Enforcement 4 years • Bilingual (Spanish)", "followers_count": 467, "friends_count": 1021, "statues_count": 7109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387090690048, "text": "Stop tweeting take ya ass to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502008150, "name": "Sedddd", "screen_name": "Mc_DIMPLESS", "lang": "en", "location": " ✈️ In Commerce Learning ", "create_at": date("2012-02-24"), "description": "Crimonology|Fall '15| De2ce Club|MuZeta|ΣΓΡ", "followers_count": 986, "friends_count": 946, "statues_count": 29887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387451408384, "text": "WHO WOULDN'T BE MOTIVATED TO WORK OUT?!?!?!!!! https://t.co/XoOGC10rmM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 629132160, "name": "wildfox", "screen_name": "chaneldnglsn", "lang": "en", "location": "Milpitas, CA", "create_at": date("2012-07-06"), "description": "a little more kindess, a little less judgement. \nsc:chanel01\n#08112016", "followers_count": 331, "friends_count": 511, "statues_count": 18188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387547873280, "text": "Like most of the time I'm just pissed and petty and shit talking but in the end I'm fucking hurt because idk where I keep going wrong.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2468421254, "name": "peanut butter ass", "screen_name": "baddriverAF", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-04-28"), "description": "19. if feminism makes you uncomfortable turn back now. 28064212. mama to the most perfect mutt on the face of the planet & married to @bentdicktip", "followers_count": 495, "friends_count": 223, "statues_count": 33582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387703259137, "text": "@bulimics lmaoo we used to be so rude to them in the past", "in_reply_to_status": 717244284810895360, "in_reply_to_user": 30300282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30300282 }}, "user": { "id": 189951706, "name": "Cathy Fans Somalia", "screen_name": "fainvr", "lang": "en", "location": "null", "create_at": date("2010-09-12"), "description": "@thetennisisland contributor. medill student. my hobbies include crying over popstars, movies, tv shows, soccer, and tennis.", "followers_count": 1038, "friends_count": 353, "statues_count": 53906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evanston, IL", "id": "8b351eeb91372dc7", "name": "Evanston", "place_type": "city", "bounding_box": rectangle("-87.733451,42.019151 -87.66494,42.071723") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1724582, "cityName": "Evanston" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387841474560, "text": "nicky�� https://t.co/vllKi17gVV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2475100304, "name": "sebastian loera", "screen_name": "kingsimba_11", "lang": "en", "location": "South Gate, California", "create_at": date("2014-05-03"), "description": "fear none.", "followers_count": 294, "friends_count": 426, "statues_count": 2518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-04-05T00:00:00.000Z"), "id": 717245387984113664, "text": "I have no feeling for no one anymore. I love everybody though . I got my eye on something though .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174483668, "name": "Chef R. Cole IV", "screen_name": "_ImSooLOST", "lang": "en", "location": "Houston, TX", "create_at": date("2010-08-03"), "description": "#Founder: IVFourth's by Chef R. Cole IV | #TheBiggTastingParty | #WhatsNextChef Party Series || Business Inquiries- ivfourthsbooking@gmail.com", "followers_count": 2395, "friends_count": 3706, "statues_count": 128067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245388542099457, "text": "04/05@03:00 - Temp 33.2F, WC 27.6F. Wind 6.1mph NNW, Gust 17.0mph. Bar 30.131in, Rising. Rain 0.00in. Hum 70%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245388843982848, "text": "put the worst one first �������� https://t.co/7PWGQH1jEk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290900988, "name": ".", "screen_name": "dnashaaaa", "lang": "en", "location": "Laffy, Louisiana", "create_at": date("2011-04-30"), "description": "18... sc: dnashasherae", "followers_count": 2372, "friends_count": 2182, "statues_count": 55835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245388952985600, "text": "I want to end up like Ellie and Carl in Up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262513798, "name": "baby", "screen_name": "connizle_", "lang": "en", "location": "Los Angeles, California", "create_at": date("2013-12-26"), "description": "null", "followers_count": 236, "friends_count": 228, "statues_count": 2784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245389154312192, "text": "@paulwiggins In other words, the Times is asking, \"Why didn't we get it?\"", "in_reply_to_status": 717245035033432064, "in_reply_to_user": 10482452, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 10482452 }}, "user": { "id": 16375096, "name": "maryrduan", "screen_name": "maryrduan", "lang": "en", "location": "Salinas, CA", "create_at": date("2008-09-19"), "description": "Editor-in-chief at the Monterey County Weekly, world's greatest living expert on the film Con Air. Brown Act and PRA lover. http://www.montereycountyweekly.com.", "followers_count": 1806, "friends_count": 2014, "statues_count": 7407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245389666123776, "text": "Ripley SW Limestone Co. Temp: 55.8°F Wind:3.8mph Pressure: 1002.0mb Rising Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245390387605504, "text": "https://t.co/vEBaKJv4wb", "in_reply_to_status": 690277094132224000, "in_reply_to_user": 324823622, "favorite_count": 0, "coordinate": point("-83.73165105,34.3422113"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 324823622, "name": "HOGG McSadden", "screen_name": "HellonDaScale", "lang": "en", "location": "stone creek thats me& bruh, GA", "create_at": date("2011-06-27"), "description": "I'm not ur friend cause I follow u... R.i.P. Nanna...R.i.P. Beavs", "followers_count": 2476, "friends_count": 666, "statues_count": 239572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13139, "countyName": "Hall" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245390697926656, "text": "Temp 24.4° Hi/Lo 24.7/24.4 Rng 0.3° WC 24.4° Hmd 87% Rain 0.00\" Storm 0.00\" BAR 30.108 Rising DP 21.1° Wnd 2mph Dir N Gst 11mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 117, "statues_count": 17548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245390890766336, "text": "��✊�� https://t.co/aeYTiQAfF0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 331755648, "name": "Poppin", "screen_name": "Howyoufeel3x", "lang": "en", "location": "null", "create_at": date("2011-07-08"), "description": "she ain't got no heart shawty like the tinman tell her anything she wont feel it like its thin air| bff - @yallknowrhi_ ✨❄️", "followers_count": 1520, "friends_count": 1154, "statues_count": 35205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245391129993216, "text": "bruh why can't I joan w out sounding either too white or awkward", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1397921618, "name": "Abs", "screen_name": "Anna_badger7", "lang": "en", "location": "256", "create_at": date("2013-05-02"), "description": "@_JVICE_❣", "followers_count": 327, "friends_count": 614, "statues_count": 429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weaver, AL", "id": "0f174a3dc7510419", "name": "Weaver", "place_type": "city", "bounding_box": rectangle("-85.847512,33.735457 -85.759796,33.783318") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 180352, "cityName": "Weaver" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245391138267136, "text": "happy birthday G! ���� thanks for everything and I hope you have a great day ������ @giaxnna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1403739199 }}, "user": { "id": 3152267286, "name": "christian..", "screen_name": "tromiemowgli", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "@alexacaldera ❤️", "followers_count": 236, "friends_count": 287, "statues_count": 495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245391364694016, "text": "this is the month i started loving you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2609931627, "name": "em", "screen_name": "kayitsem", "lang": "en", "location": "null", "create_at": date("2014-06-14"), "description": "I am stupid and courageous", "followers_count": 615, "friends_count": 427, "statues_count": 18198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Elsinore, CA", "id": "80cf9987ff7e9762", "name": "Lake Elsinore", "place_type": "city", "bounding_box": rectangle("-117.413156,33.618447 -117.216785,33.713284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 639486, "cityName": "Lake Elsinore" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245391847051265, "text": "@iHartbeat @harto ����", "in_reply_to_status": 717244970734718976, "in_reply_to_user": 57760544, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 57760544, 14166096 }}, "user": { "id": 83040723, "name": "Helen", "screen_name": "ever4far", "lang": "en", "location": "Palmdale, CA", "create_at": date("2009-10-16"), "description": "LA Have a Hart Day City Captain, Internet addict, Travel enthusiast", "followers_count": 298, "friends_count": 456, "statues_count": 3086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245392157614080, "text": "Temp: 51.1°F Wind:0.0mph Pressure: 30.168hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245392253890560, "text": "I wanted to go pool tomorrow, now I gotta debate on tomorrow's plans", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3087670926, "name": "kaimanaaaaa", "screen_name": "kaimanaaaaa", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2015-03-15"), "description": "poi poundah", "followers_count": 169, "friends_count": 85, "statues_count": 4937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Beach, HI", "id": "6bd73386ffaba450", "name": "Ewa Beach", "place_type": "city", "bounding_box": rectangle("-158.028613,21.306027 -157.990042,21.332114") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507450, "cityName": "Ewa Beach" } }
+{ "create_at": datetime("2016-04-05T00:00:01.000Z"), "id": 717245392421670914, "text": "@ebbtideapp Tide in Nawiliwili, Hawaii 04/05/2016\nHigh 2:16am 1.6\n Low 8:33am -0.1\nHigh 2:48pm 1.4\n Low 8:40pm -0.1\nHigh 2:54am 1.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-159.3567,21.955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 21790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lihue, HI", "id": "c291c36c05006a4a", "name": "Lihue", "place_type": "city", "bounding_box": rectangle("-159.390295,21.946465 -159.329778,21.995001") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15007, "countyName": "Kauai", "cityID": 1545200, "cityName": "Lihue" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245392727965696, "text": "@harrison_barron what shout the Blues? ��", "in_reply_to_status": 717236521007497216, "in_reply_to_user": 240694112, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 240694112 }}, "user": { "id": 2344926979, "name": "Honey Badger", "screen_name": "MattLegit10", "lang": "en", "location": "null", "create_at": date("2014-02-15"), "description": "I am the master of my fate: I am the Captain's of my soul. #Invictus #KiaKaha STL ↔️ Louisville", "followers_count": 968, "friends_count": 1990, "statues_count": 46464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245393331990528, "text": "Wind 1.0 mph NNE. Barometer 30.317 in, Rising slowly. Temperature 34.7 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245393583480832, "text": "@DRich59 I had that earlier haha", "in_reply_to_status": 717241221614149632, "in_reply_to_user": 417651410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417651410 }}, "user": { "id": 402491827, "name": "scriggs✊", "screen_name": "Brysonbriggs", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2011-10-31"), "description": "null", "followers_count": 1238, "friends_count": 1001, "statues_count": 16435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245393738661888, "text": "@cassodinero ok. Always in my prayers family. Anything I can do to help??", "in_reply_to_status": 717245234153795584, "in_reply_to_user": 35844551, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35844551 }}, "user": { "id": 36603820, "name": "Wes Parrish", "screen_name": "Wes_Nixon", "lang": "en", "location": "The Nickel ", "create_at": date("2009-04-29"), "description": "Vicious", "followers_count": 680, "friends_count": 740, "statues_count": 37241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245393776410624, "text": "Cancer sticks and airport bars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144666110, "name": "shannon", "screen_name": "shannicotine", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-05-16"), "description": "you just gotta say fuck it", "followers_count": 213, "friends_count": 204, "statues_count": 8544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245393919164416, "text": "Dunno if I can go to NWM anymore, got hit with some unexpected expenses =\\", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22677124, "name": "Burst Burglar", "screen_name": "TheBurstBurglar", "lang": "en", "location": "the 30H3", "create_at": date("2009-03-03"), "description": "Former writer for The Game Reviews, Indie game developer.", "followers_count": 267, "friends_count": 301, "statues_count": 50523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245394174906372, "text": "God would probably have me on some real strict shit https://t.co/utWjHsFaDY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69192117, "name": "Ryan fuckin swift;)", "screen_name": "niggaryan", "lang": "en", "location": "cali", "create_at": date("2009-08-26"), "description": "Fuck it. stay lit.", "followers_count": 264, "friends_count": 277, "statues_count": 1501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245394267144192, "text": "u can't just put me down and pick me up when you want me lmfao fuck u thought this was ????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1537801592, "name": "gyp$y✨", "screen_name": "letstryacid", "lang": "en", "location": "idfk", "create_at": date("2013-06-21"), "description": "I'll be gone for awhile//", "followers_count": 4504, "friends_count": 3012, "statues_count": 56582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245394409889792, "text": "Wind 0.0 mph ---. Barometer 30.351 in, Rising. Temperature 40.9 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245394489516032, "text": "Wanna wish one of my closest friend @nemesisnoyolam a happy 22nd birthday ����hope all your dreams and wishes come true❤#turnupthisweekend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "turnupthisweekend" }}, "user_mentions": {{ 448525791 }}, "user": { "id": 1552393236, "name": "Felipe Soto", "screen_name": "Pipe08_", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-06-27"), "description": "On the path to success", "followers_count": 167, "friends_count": 121, "statues_count": 1095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245395529666561, "text": "������ dead, DIDNT U FREAKING TAKE JAPANESE?? https://t.co/ehE0Am0qMu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2764721796, "name": "Miss Congeniality ✨", "screen_name": "_Trinibee", "lang": "en", "location": "doing AP probs ", "create_at": date("2014-08-24"), "description": "I am genuinely happy rn, so pls leave your negativity at the door", "followers_count": 206, "friends_count": 149, "statues_count": 4868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245395588374528, "text": "That moment of nirvana when your eyes caught mine��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331472230, "name": "Tristin Gallo", "screen_name": "TGall24", "lang": "en", "location": "OHIO - CALI - MMHS", "create_at": date("2011-07-07"), "description": "Procrastination at its finest", "followers_count": 259, "friends_count": 257, "statues_count": 396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245396028751872, "text": "George's full name is George Theodore Washington, but I'm thinking about calling him Theo from now on cuz he's getting kinda angsty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1000983271, "name": "hamster mom", "screen_name": "Ew_IHateIt", "lang": "en", "location": "null", "create_at": date("2012-12-09"), "description": "it ain't that serious", "followers_count": 1040, "friends_count": 690, "statues_count": 21740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McPherson, KS", "id": "8e170916da1cb9fb", "name": "McPherson", "place_type": "city", "bounding_box": rectangle("-97.70368,38.35104 -97.621063,38.400169") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20113, "countyName": "McPherson", "cityID": 2043950, "cityName": "McPherson" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245396028944385, "text": "Wind 0.0 mph NW. Barometer 30.21 in, Rising slowly. Temperature 13.3 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245396557369344, "text": "America gives too much importance to mundane, non relevant things.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298807875, "name": "S3", "screen_name": "Shetty_S3", "lang": "en", "location": "New York", "create_at": date("2011-05-14"), "description": "Its only when the Phoenix submits itself, that it is able to rise from its ashes", "followers_count": 103, "friends_count": 126, "statues_count": 9510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-05T00:00:02.000Z"), "id": 717245396641320960, "text": "why is it sssssoooooo haaarrrdddd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252892906, "name": "ky", "screen_name": "kylaorozco", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-02-15"), "description": "unavailable", "followers_count": 1275, "friends_count": 248, "statues_count": 24120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245396922212354, "text": "@janessarihana lmfao!!!", "in_reply_to_status": 717245309781303297, "in_reply_to_user": 2579877493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2579877493 }}, "user": { "id": 1043378264, "name": "isaac", "screen_name": "Isaaccguapo", "lang": "en", "location": "Yay Area", "create_at": date("2012-12-28"), "description": "IM ME, I DO ME, AND I CHILL", "followers_count": 1131, "friends_count": 661, "statues_count": 42001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245396955701248, "text": "@Wonderland_248 #ScreenShotsSakiJapan https://t.co/Vwg5OkYxs8", "in_reply_to_status": -1, "in_reply_to_user": 2934423326, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ScreenShotsSakiJapan" }}, "user_mentions": {{ 2934423326 }}, "user": { "id": 3282946934, "name": "Michael Campbell", "screen_name": "MichaelTnDisney", "lang": "en", "location": "East Ridge, TN", "create_at": date("2015-07-17"), "description": "Chattanooga Tennessee / Tennessee / Georgia State line / Born July 16th. @ Fort Benning, Georgia / love Disney World, Orlando!!!!", "followers_count": 115, "friends_count": 217, "statues_count": 2790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Ridge, TN", "id": "f3d2895a43058632", "name": "East Ridge", "place_type": "city", "bounding_box": rectangle("-85.275605,34.984972 -85.172342,35.020961") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4722720, "cityName": "East Ridge" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245397056356352, "text": "Tongue in the butt https://t.co/sj26rXCGep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3316343414, "name": "rosecrans stupid!", "screen_name": "ComptonStupid", "lang": "en", "location": "Compton, CA", "create_at": date("2015-08-15"), "description": "all my life I hustled. now I uber everywhere. I'm coming for whoever. I'm really from Compton tho.", "followers_count": 107, "friends_count": 121, "statues_count": 1942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245397337395200, "text": "It's hard to lift others up, when they fight against you. #alone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alone" }}, "user": { "id": 4826736014, "name": "Lee Ann", "screen_name": "MusicLvrUT", "lang": "en", "location": "Washington, USA", "create_at": date("2016-01-28"), "description": "Musical mom who loves her God, her family, her life!", "followers_count": 13, "friends_count": 30, "statues_count": 166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Graham, WA", "id": "08a26cd724eca51d", "name": "Graham", "place_type": "city", "bounding_box": rectangle("-122.338034,47.016521 -122.206872,47.09723") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5327785, "cityName": "Graham" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245397882658817, "text": "Happy birthday, me.", "in_reply_to_status": 717244476167618560, "in_reply_to_user": 38748359, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38748359, "name": "Erin❤️", "screen_name": "_CallmeShelly_", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-05-08"), "description": "50% anxiety, 50% sarcasm. #Feminist #squad |-/", "followers_count": 1278, "friends_count": 1068, "statues_count": 141816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245398088359936, "text": "Wind 0.0 mph ---. Barometer 30.145 in, Steady. Temperature 48.8 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245398155288576, "text": "Legends https://t.co/czFrMcPoAx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2283061166, "name": "Barrett Kelly", "screen_name": "barrettkellly", "lang": "en", "location": "California, USA", "create_at": date("2014-01-08"), "description": "SC: Barrettkellyy", "followers_count": 822, "friends_count": 521, "statues_count": 1197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245398390321152, "text": "Temp 33.6°F Wind Chill 31.1°F RH 71% Wind 3.0 W Gust 9.0 W SLP 30.131 in Rising Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 106, "friends_count": 63, "statues_count": 34476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245398801391616, "text": "@Mercymercmerc @RaulMtz_Mier he is lol", "in_reply_to_status": 717245233394679808, "in_reply_to_user": 713122434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 713122434, 1882016264 }}, "user": { "id": 1703495180, "name": "willy rodriguez", "screen_name": "mcwilly12", "lang": "en", "location": "null", "create_at": date("2013-08-26"), "description": "King Kong ain't got shit on me", "followers_count": 103, "friends_count": 263, "statues_count": 750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245399405187072, "text": "so it begins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434459246, "name": "Elizabeth", "screen_name": "leesuuuuh", "lang": "en", "location": "Hogsmeade / Los Angeles", "create_at": date("2011-12-11"), "description": "hold yer hippogriffs", "followers_count": 493, "friends_count": 165, "statues_count": 56799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245399644278784, "text": "@TheyLoveTraviee oops �� how lame of me.", "in_reply_to_status": 717245282845605888, "in_reply_to_user": 297797175, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 297797175 }}, "user": { "id": 1232683842, "name": "4/30 ✨", "screen_name": "amiikins", "lang": "en", "location": "Arlington, TX", "create_at": date("2013-03-01"), "description": "it's pretty much whatever at this point.", "followers_count": 1453, "friends_count": 580, "statues_count": 80828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245400164515840, "text": "@DejaVuXo �������� https://t.co/prmr2BQbAg", "in_reply_to_status": -1, "in_reply_to_user": 2694321680, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2694321680 }}, "user": { "id": 2408411132, "name": "Phina✨", "screen_name": "soursera", "lang": "en", "location": "Federal Way☔️ • SC/IG:SourSera", "create_at": date("2014-03-23"), "description": "Big Dreams, Good Music, & Expensive Taste.", "followers_count": 1119, "friends_count": 690, "statues_count": 15936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245400223059969, "text": "Wind 4.0 mph E. Barometer 30.327 in, Rising slowly. Temperature 33.5 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 15, "friends_count": 15, "statues_count": 2405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245400269230080, "text": "I'm very ugly also so thank you God", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1031937450, "name": "hollaback gurl", "screen_name": "phatbinch", "lang": "en", "location": "Tucson, AZ", "create_at": date("2012-12-23"), "description": "I'm just a fucking ugli toad", "followers_count": 469, "friends_count": 337, "statues_count": 39307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245400336314368, "text": "Yawn. Time to go back to work tomorrow!!!! ❤️������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26697372, "name": "Jeh⭐️Knee", "screen_name": "IMterriblyvexed", "lang": "en", "location": "Pemberley", "create_at": date("2009-03-25"), "description": "Research Analyst & Adventurer. Tatted Trivia Nerd. HondaLyft. Elite 16. Kinsey 3. 3rd gen Portuguese #SFGiants fan. #Faithful #SacKings #CSUS '11. ❤books. ⚾️", "followers_count": 530, "friends_count": 720, "statues_count": 32087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-05T00:00:03.000Z"), "id": 717245400839753732, "text": "Lily Aldridge: dalla prima alla terza con un click. https://t.co/jLBpIQ8T8w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 437491646, "name": "Duk the doc", "screen_name": "rikiduk", "lang": "it", "location": "milano", "create_at": date("2011-12-15"), "description": "Dottore Chiacchierone, lobi frontali iposviluppati mi permettono di dire tutto ciò che penso e pensare solo cose stupidamente primordiali.", "followers_count": 104, "friends_count": 104, "statues_count": 4816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Presidio, San Francisco", "id": "df51dec6f4ee2b2c", "name": "Presidio", "place_type": "neighborhood", "bounding_box": rectangle("-122.489133,37.786925 -122.446306,37.812868") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245401439449090, "text": "Had so much fun today! Love my neighborhood!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4676096694, "name": "Leslie Villa", "screen_name": "lushlie23", "lang": "en", "location": "Chula Vista, CA", "create_at": date("2015-12-29"), "description": "null", "followers_count": 80, "friends_count": 244, "statues_count": 74 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245401506516992, "text": "57.6F (Feels: 57.6F) - Humidity: 98% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1017.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 233790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245401565368320, "text": "@JacksonLeach27 hands down one of the best friends I have much love for her and I know she'll always be there. ��", "in_reply_to_status": 717243521640505344, "in_reply_to_user": 875630900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 875630900 }}, "user": { "id": 2912205315, "name": "Sav", "screen_name": "YungSavv16", "lang": "en", "location": "Sc: s_amial16", "create_at": date("2014-12-08"), "description": "let's get lost. madera.", "followers_count": 845, "friends_count": 536, "statues_count": 14688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera Acres, CA", "id": "75161a014946f779", "name": "Madera Acres", "place_type": "city", "bounding_box": rectangle("-120.109807,36.98237 -120.056201,37.03494") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645050, "cityName": "Madera Acres" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245401838002176, "text": "������ fuck it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2368153948, "name": "you know me.", "screen_name": "brxvh_", "lang": "en", "location": "getmoney.com", "create_at": date("2014-02-26"), "description": "null", "followers_count": 1147, "friends_count": 961, "statues_count": 23363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245402659958785, "text": "Ruby Room Wants To Give You A Special Gift For Your Birthday...https://t.co/STvwerxrqK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6716,41.9032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27957411, "name": "Ruby Room", "screen_name": "rubyroomchicago", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2009-03-31"), "description": "to leave a beauty mark on the face of the wellness industry through our commitment to healing, energy and all things positive.", "followers_count": 1125, "friends_count": 1712, "statues_count": 1941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245402987249664, "text": "Finally hit level 200. 800 more levels to go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447290500, "name": "Alex Hill", "screen_name": "ImDiabetus", "lang": "en", "location": "Valentine Detective Agency ", "create_at": date("2011-12-26"), "description": "Proud Member of @Axon_Gaming | Podcaster | Filmmaker | Steelers/Lakers/Pirates fan | Constitutionalist | Huge Star Wars Nerd", "followers_count": 1084, "friends_count": 353, "statues_count": 79131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Curwensville, PA", "id": "0dfb88e628be713c", "name": "Curwensville", "place_type": "city", "bounding_box": rectangle("-78.540995,40.960463 -78.497918,40.986955") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42033, "countyName": "Clearfield", "cityID": 4217840, "cityName": "Curwensville" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245403595276293, "text": "mbn���� https://t.co/i1PncltULA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 280818328, "name": "Alejandra", "screen_name": "Alejandra_OVOXO", "lang": "en", "location": "Oregon", "create_at": date("2011-04-11"), "description": "|| 20 ||", "followers_count": 1371, "friends_count": 1997, "statues_count": 46874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, OR", "id": "06bafceaaf071d8a", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-123.349064,44.903121 -123.274943,44.937956") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4117700, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404232847360, "text": ".@Lilbumblebear game, work, eat, work, game, work, gym, work, game, work. Now it's time to sleep! xD", "in_reply_to_status": 717209338973175808, "in_reply_to_user": 1901029892, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1901029892 }}, "user": { "id": 72240034, "name": "Spencer Bing", "screen_name": "notspencer", "lang": "en", "location": "Los Angeles, California", "create_at": date("2009-09-07"), "description": "Travel, gaming, and Chelsea boy. Content Manager and Support Lead for http://Proguides.com", "followers_count": 524, "friends_count": 398, "statues_count": 8589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404308320256, "text": "@Trephillips05 get out your feelings", "in_reply_to_status": 717245339904782340, "in_reply_to_user": 455875605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 455875605 }}, "user": { "id": 484502345, "name": "Jack White", "screen_name": "JaCk_olatern", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-02-05"), "description": "I believe what doesn't kill you, simply makes you.. stranger-The Joker | Cam 8-21-14 | Fullerton JuCo. 2016 qualifier •CHI ✈️ CALI• #Tomitha #JUCOPRODUCT", "followers_count": 1425, "friends_count": 587, "statues_count": 32785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404430082048, "text": "@devstev_ still need to catch up on that", "in_reply_to_status": 717243506989903872, "in_reply_to_user": 909174258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 909174258 }}, "user": { "id": 350608608, "name": "Manny Ortiz", "screen_name": "MannyOrtiz3", "lang": "en", "location": "Dearborn, Michigan ", "create_at": date("2011-08-07"), "description": "UofM-Dearborn, , Super Smash Bros Champion", "followers_count": 500, "friends_count": 630, "statues_count": 2688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn, MI", "id": "339bd7ae6a55ba9f", "name": "Dearborn", "place_type": "city", "bounding_box": rectangle("-83.287094,42.277554 -83.14002,42.35191") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621000, "cityName": "Dearborn" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404572569600, "text": "@BrendaSPino you say this like 5 times a day", "in_reply_to_status": 717224937233231872, "in_reply_to_user": 230859767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 230859767 }}, "user": { "id": 24298089, "name": "piño", "screen_name": "crystalpino_", "lang": "en", "location": "null", "create_at": date("2009-03-13"), "description": "She got Latin roots, she got everything", "followers_count": 329, "friends_count": 276, "statues_count": 14287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404610363392, "text": "HAPPY BIRTHDAY!! @JoshuaDtown @MissRBaller ���������������������� hope u both have the best day ever https://t.co/PKs9lVXWaY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 302841137, 468957900 }}, "user": { "id": 3258340345, "name": "Colleens cookie", "screen_name": "trinab1233", "lang": "en", "location": "colleens corner", "create_at": date("2015-06-27"), "description": "met colleen josh and Rebecca 1•14•16|| josh followed 2•20•16||Colleen followed 1•18•16||", "followers_count": 368, "friends_count": 308, "statues_count": 8738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245404874678272, "text": "She's the smollest kid https://t.co/du964pCXii", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394325917, "name": "James Forret", "screen_name": "JamesForret", "lang": "en", "location": "Clearwater, FL", "create_at": date("2011-10-19"), "description": "I'm James Forret, not against it @LizBrauer is my angel.", "followers_count": 291, "friends_count": 951, "statues_count": 2931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearwater, FL", "id": "700eeb799fa55a4b", "name": "Clearwater", "place_type": "city", "bounding_box": rectangle("-82.831674,27.935178 -82.679007,28.050243") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1212875, "cityName": "Clearwater" } }
+{ "create_at": datetime("2016-04-05T00:00:04.000Z"), "id": 717245405033943040, "text": "So do females https://t.co/ewKSiwVQff", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3618693914, "name": "eeryT", "screen_name": "Tyree8Taylor_", "lang": "en", "location": "null", "create_at": date("2015-09-19"), "description": "All State DB // Dysart High School Varsity Quarterback //http://www.hudl.com/athlete/3019553/highlights/311958415/v2", "followers_count": 729, "friends_count": 843, "statues_count": 11815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Mirage, AZ", "id": "0de4c71dbfcd2c32", "name": "El Mirage", "place_type": "city", "bounding_box": rectangle("-112.342111,33.579997 -112.302246,33.630786") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 422220, "cityName": "El Mirage" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245405361102848, "text": "@Rjayeeeee happy bday bro ����", "in_reply_to_status": -1, "in_reply_to_user": 1286385764, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1286385764 }}, "user": { "id": 2316472118, "name": "Fuentez", "screen_name": "Fuentez_06", "lang": "en", "location": "Ontario, CA", "create_at": date("2014-01-28"), "description": "I refuse to change what I am, A lion has to eat. So run with me, or Run from me Sc/ Fuentez06", "followers_count": 1109, "friends_count": 1021, "statues_count": 60667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245405369667584, "text": "#Trespasser at Boardwalk At Alafaya Trail, 11801 Boardwalk Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.205597,28.586342"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trespasser", "orlpol", "ocso" }}, "user": { "id": 39134484, "name": "Police Calls 32826", "screen_name": "orlpol32826", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 52, "friends_count": 1, "statues_count": 5539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "01f196c135e37b16", "name": "University", "place_type": "city", "bounding_box": rectangle("-81.224485,28.56471 -81.170066,28.61218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245406778945536, "text": "Who up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49332051, "name": "Eric.", "screen_name": "LaceDownForWhat", "lang": "en", "location": "Basement", "create_at": date("2009-06-21"), "description": "19XX Hello world, I received your application, but I'm sorry to inform you that you do not meet my qualifications.", "followers_count": 427, "friends_count": 178, "statues_count": 14232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, IL", "id": "c3f1deee8671c492", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-88.565734,41.645151 -88.487888,41.684729") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17093, "countyName": "Kendall", "cityID": 1760352, "cityName": "Plano" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245406913175552, "text": "Why am I still awake ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 501034682, "name": "Allie Burkhardt", "screen_name": "allieburk", "lang": "en", "location": "null", "create_at": date("2012-02-23"), "description": "Every day should feel this good!☀️", "followers_count": 108, "friends_count": 220, "statues_count": 1189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orono, ME", "id": "01564b04448716aa", "name": "Orono", "place_type": "city", "bounding_box": rectangle("-68.702721,44.845528 -68.639048,44.912888") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23019, "countyName": "Penobscot", "cityID": 2355575, "cityName": "Orono" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245406996930560, "text": "@JaylaLashe Bet That! ��", "in_reply_to_status": 717243162234716160, "in_reply_to_user": 286920374, "favorite_count": 0, "coordinate": point("-92.264139,34.710596"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 286920374 }}, "user": { "id": 165521516, "name": "#StreetMoney", "screen_name": "CJ_HallStar", "lang": "en", "location": "Trap", "create_at": date("2010-07-11"), "description": "#TeamLakers #TeamTitans #Arkansas #Razorbacks #WPS #HallStarTV #OneLove", "followers_count": 1862, "friends_count": 1960, "statues_count": 44238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245407567347712, "text": "Temp: 23.9°F - Dew Point: 15.6° - Wind: 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.41in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 15819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245408184033280, "text": "@BruceBlitz @VirtuosoSports @Jthames101193 Another Gem from @HempshallAndy: https://t.co/N1e4Z3GjqJ", "in_reply_to_status": 717243925183008768, "in_reply_to_user": 25653324, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 142341666, 1315380474, 2380911098, 2571268133 }}, "user": { "id": 25653324, "name": "Karen Catizone", "screen_name": "Mugsysam", "lang": "en", "location": "Boston, MA", "create_at": date("2009-03-21"), "description": "Don't take life so seriously, live it 2 it's fullest & enjoy it because none of us R getting out of here alive..........", "followers_count": 3335, "friends_count": 3224, "statues_count": 83495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245408301461504, "text": "Wind 0.0 mph ---. Barometer 30.224 in, Rising. Temperature 8.6 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245408523608064, "text": "@kaykamimura thanks kaykay. Miss u ��", "in_reply_to_status": 717204928721264640, "in_reply_to_user": 46841317, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 46841317 }}, "user": { "id": 304452957, "name": "hippo", "screen_name": "vanessatatts", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-05-24"), "description": "http://vanessatheartist.com/portfolio", "followers_count": 1091, "friends_count": 453, "statues_count": 1828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245408699789313, "text": "1.7 magnitude #earthquake. 21 km from Hawthorne, NV, #UnitedStates https://t.co/hfcuNP76Mo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.781,38.672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "UnitedStates" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 67424, "friends_count": 10, "statues_count": 100233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6051, "countyName": "Mono" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245408896921600, "text": "Just believe in the vision", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3123986180, "name": "Yin.", "screen_name": "SmoothTheGreat", "lang": "en", "location": "Somewhere Sewing Fabric", "create_at": date("2015-04-01"), "description": "My Time is approaching.. Founder Of God Made Clothing . Ig: GodMadeDes", "followers_count": 680, "friends_count": 617, "statues_count": 3924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Barbara, CA", "id": "f6ebc676e5cde864", "name": "Santa Barbara", "place_type": "city", "bounding_box": rectangle("-119.763341,34.395525 -119.639931,34.464245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669070, "cityName": "Santa Barbara" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245409089839104, "text": "You'd think after 17 years of school I'd be better at it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 154067739, "name": "Robbie", "screen_name": "Rob_Lulay", "lang": "en", "location": "Corvallis, OR", "create_at": date("2010-06-10"), "description": "⚯͛ △⃒⃘  I just tweet what's on my mind", "followers_count": 197, "friends_count": 137, "statues_count": 6700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245409236754432, "text": "Poke bowl �� #nomnom #salmon #miso #wasabi @ Aloha Fresh https://t.co/My3p2TC3no", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.0181808,37.3227348"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "nomnom", "salmon", "miso", "wasabi" }}, "user": { "id": 184567047, "name": "Feevee", "screen_name": "feeveeowe", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-08-29"), "description": "IG: fibia89", "followers_count": 1278, "friends_count": 239, "statues_count": 11173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cupertino, CA", "id": "36237ab3643ff2be", "name": "Cupertino", "place_type": "city", "bounding_box": rectangle("-122.091151,37.285786 -121.995595,37.34038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 617610, "cityName": "Cupertino" } }
+{ "create_at": datetime("2016-04-05T00:00:05.000Z"), "id": 717245409240854529, "text": "Finding a job has always been hard for me ���� so irritating", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576687790, "name": "Shelby-Lynn", "screen_name": "UnkoBuenz", "lang": "en", "location": "null", "create_at": date("2013-07-07"), "description": "Isaiah Kahoonei", "followers_count": 118, "friends_count": 169, "statues_count": 4002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimanalo Beach, HI", "id": "6c95e47bf6bd5ff4", "name": "Waimanalo Beach", "place_type": "city", "bounding_box": rectangle("-157.71502,21.316707 -157.669774,21.361859") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1578200, "cityName": "Waimanalo Beach" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245409593167872, "text": "Late Night Stream! | Come Chill and Chat! \nhttps://t.co/ndu9kMmTSx\n\n@RoyalFlushSeven @RF7Gilyphix @NightRTs @GamerRTer @CircaValen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2295530581, 1084950475, 2613982245, 2942587130, 2183594612 }}, "user": { "id": 2860037708, "name": "Kyle Miller", "screen_name": "RF7Native", "lang": "en", "location": "Chicago, IL ", "create_at": date("2014-10-17"), "description": "COD Fanboy | Lead Content Creator/ Streamer for @RoyalFlushSeven | Affiliated w/ @GorillaGGC and @GamerGearStore", "followers_count": 2547, "friends_count": 931, "statues_count": 13381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245409844793345, "text": "regretting so much rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2668795987, "name": "Princess Tija", "screen_name": "princessstija", "lang": "en", "location": "null", "create_at": date("2014-07-22"), "description": "it's pink", "followers_count": 244, "friends_count": 145, "statues_count": 1538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245410092433408, "text": "I'm hilarious, I always have myself dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228161955, "name": "Backyard Jules", "screen_name": "SwAgMaN_24", "lang": "en", "location": "The Backyard ", "create_at": date("2010-12-18"), "description": "#Txst19 Thank God for the Waters! #SpursNation 210✈️✈️Ak✈️✈️ San Mo Snapchat: swagman_24. Julian is my name, taking L's is my game", "followers_count": 843, "friends_count": 700, "statues_count": 51502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245410230730752, "text": "Good night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 635774893, "name": "⚡️ Zari", "screen_name": "Shedidthaat_", "lang": "en", "location": "in texa$", "create_at": date("2012-07-14"), "description": "I'm young..... let me live.| TView High", "followers_count": 1493, "friends_count": 734, "statues_count": 30144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245410230845442, "text": "I am a sorority girl, and I make fun of sorority girls. So basically, I pay a lot of money to do things that I make fun of myself for.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 203651166, "name": "Claire.", "screen_name": "ClFollis", "lang": "en", "location": "null", "create_at": date("2010-10-16"), "description": "Texas Forever||University of Illinois ΚΚΓ '19", "followers_count": 245, "friends_count": 197, "statues_count": 5911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Urbana, IL", "id": "497789c72fadba82", "name": "Urbana", "place_type": "city", "bounding_box": rectangle("-88.238573,40.072763 -88.162095,40.157407") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1777005, "cityName": "Urbana" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245411254075392, "text": "@FearonJames great band", "in_reply_to_status": 717245321357594624, "in_reply_to_user": 603884850, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 603884850 }}, "user": { "id": 118516415, "name": "NTRL", "screen_name": "xLeNaturel", "lang": "en", "location": "Hotel Erotica , TX", "create_at": date("2010-02-28"), "description": "Oooooof", "followers_count": 981, "friends_count": 277, "statues_count": 92590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245412466294784, "text": "@machinegunlacey strip, I make that in a week easy lol", "in_reply_to_status": 717228744709128192, "in_reply_to_user": 492976194, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 492976194 }}, "user": { "id": 341309563, "name": "rebecca mcclelland", "screen_name": "thaatbitchbecca", "lang": "en", "location": "Akron, OH", "create_at": date("2011-07-23"), "description": "UA '19 biochemistry (pre-med) & international business majors #LongLiveMatt RIP Matt & Devon", "followers_count": 3971, "friends_count": 827, "statues_count": 107067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Canton, OH", "id": "00bd609424733145", "name": "North Canton", "place_type": "city", "bounding_box": rectangle("-81.442077,40.853129 -81.323313,40.917247") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3956294, "cityName": "North Canton" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245412793397248, "text": "I'll suck some toes before I eat some booty https://t.co/XX3lcToSRi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2594146814, "name": "E. Boogie", "screen_name": "itsEricbruhh", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2014-06-28"), "description": "In BMG, We Trust", "followers_count": 724, "friends_count": 409, "statues_count": 47780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maricopa, AZ", "id": "001b67fd5761210e", "name": "Maricopa", "place_type": "city", "bounding_box": rectangle("-112.079946,33.029009 -111.944584,33.087983") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 444410, "cityName": "Maricopa" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245413028331520, "text": "☺️☺️☺️ https://t.co/W82pECEPzY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 299756152, "name": "QUEEN DANGER", "screen_name": "Therealdang3r", "lang": "en", "location": "null", "create_at": date("2011-05-16"), "description": "#FollowTheCheck // no bio, meet me in real life // DominicanRepublic ✈️ HTX", "followers_count": 2869, "friends_count": 829, "statues_count": 126856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245413137518592, "text": "Love Me Or Forever Hold Your Hate...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 611612658, "name": "TheRealMarco༄$oule¿", "screen_name": "MarcoSoule", "lang": "en", "location": "Saginaw, MI", "create_at": date("2012-06-18"), "description": "THE FUNKY SAPIOSEXUAL, THE INTELLECTUAL ORIGINATOR ❌OakU'19 ❌ SC: originalfreshhh POETRY•IS•LIFE", "followers_count": 1684, "friends_count": 809, "statues_count": 43852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn Hills, MI", "id": "1d09535781e3b870", "name": "Auburn Hills", "place_type": "city", "bounding_box": rectangle("-83.32891,42.619824 -83.209206,42.709545") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2604105, "cityName": "Auburn Hills" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245413145845761, "text": "Hope they don't put you on list for a liver transplant when you need it in the future @RandPaul https://t.co/49X8H2M9np", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216881337 }}, "user": { "id": 707256854448107520, "name": "Sheyma Gates, PhD", "screen_name": "784578GGBATski", "lang": "en", "location": "Pickerington, OH", "create_at": date("2016-03-08"), "description": "Non of your business Tweets may be Archived used for my delight", "followers_count": 9, "friends_count": 230, "statues_count": 3604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pickerington, OH", "id": "3995cc1483801d24", "name": "Pickerington", "place_type": "city", "bounding_box": rectangle("-82.797752,39.841431 -82.684335,39.939034") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3962498, "cityName": "Pickerington" } }
+{ "create_at": datetime("2016-04-05T00:00:06.000Z"), "id": 717245413309423616, "text": "@ScreamngMantis Boogie woogie is grooooooooovy. *goes grab beers*", "in_reply_to_status": 717236885005934592, "in_reply_to_user": 197593265, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38318388 }}, "user": { "id": 197593265, "name": "James Peralta", "screen_name": "jnp_08", "lang": "en", "location": "Juneau, Alaska", "create_at": date("2010-10-01"), "description": "A titanium thwomp with a good heart. From the one cold state where everyone lives in igloos. Even the internet is made of a igloo. Not a lot of people know that", "followers_count": 44, "friends_count": 237, "statues_count": 6154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607772418883584, "text": "@ikamatsu_ I'm sorry! But that screen shot is hilarious.", "in_reply_to_status": 717604971093434368, "in_reply_to_user": 4640722404, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4640722404 }}, "user": { "id": 140823673, "name": "チョロ松ZOMBIE", "screen_name": "zombie_boogie", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-05-06"), "description": "(Kelly) I put the romance back in necromancer. R-18!, complaints, http://thiscrush.com/~pregnantzombie 爆ツイ, BL松 OK ENG OK, 日本語ギリギリOK -おそ松さん, 文句だよ-", "followers_count": 386, "friends_count": 449, "statues_count": 24302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607772708179968, "text": "I need to sleep I have a damn midterm mananananananananana", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1926025201, "name": "L. LUNA", "screen_name": "vivalalupee", "lang": "en", "location": "Moyahua zacatecas ", "create_at": date("2013-10-02"), "description": "BAY AREA", "followers_count": 415, "friends_count": 757, "statues_count": 10202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607772913811456, "text": "When you thought your snapchat selfie was trash, but people send you heart eyes>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394458001, "name": "Ayanna Byrd", "screen_name": "ayanna_byrd", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-10-19"), "description": "Single black female addicted to retail. ~UMD 2019", "followers_count": 317, "friends_count": 274, "statues_count": 5919 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Park, MD", "id": "e4c17912c815124d", "name": "College Park", "place_type": "city", "bounding_box": rectangle("-76.965596,38.971053 -76.903378,39.022888") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2418750, "cityName": "College Park" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607773714841600, "text": "@msketteringham Her voice, the way she expressed herself.", "in_reply_to_status": 717607057587773441, "in_reply_to_user": 393053384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393053384 }}, "user": { "id": 1078046984, "name": "Marla Simone", "screen_name": "MarlaSimone", "lang": "en", "location": "Los Angeles ", "create_at": date("2013-01-10"), "description": "An all around Bon Vivant actress, content creator and writer. Life is a banquet. Enjoy it. Be yourself.", "followers_count": 6136, "friends_count": 4950, "statues_count": 21490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607773731565568, "text": "@JeremyChoi1 mile and half each way", "in_reply_to_status": 717607695084232704, "in_reply_to_user": 416569553, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 416569553 }}, "user": { "id": 812666263, "name": "Robert J King III", "screen_name": "rking425369", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2012-09-09"), "description": "I'm Hungry", "followers_count": 602, "friends_count": 579, "statues_count": 17735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607774511706112, "text": "On god ���� https://t.co/63hjZUG4nS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35478824, "name": "7", "screen_name": "AppleciderPapi", "lang": "en", "location": "LA", "create_at": date("2009-04-26"), "description": "18. Muslim. Beautiful. Genius. ⭐️", "followers_count": 6189, "friends_count": 946, "statues_count": 168674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607774671114240, "text": "S/o to kris for dragging my ass to the gym tonight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1143996798, "name": "Bean", "screen_name": "DeanBasa", "lang": "en", "location": "null", "create_at": date("2013-02-02"), "description": "On the pursuit of happiness.", "followers_count": 136, "friends_count": 166, "statues_count": 2018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607774679465985, "text": "Excited as fuck for this coming week ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219110627, "name": "urfavbbw", "screen_name": "M4NI4C_", "lang": "en", "location": "null", "create_at": date("2010-11-23"), "description": "Arielle / babygirl", "followers_count": 570, "friends_count": 316, "statues_count": 63461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Gabriel, CA", "id": "2d6aea733609e3e4", "name": "South San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.108297,34.040493 -118.081891,34.055365") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673276, "cityName": "South San Gabriel" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607774767677440, "text": "Seriously. How much can one person cry during this damn show?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36758392, "name": "Taylor Shay", "screen_name": "Tayx3720", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "I always tell the girls to never take it seriously, if you never take it seriously, you never get hurt & if you never get hurt, you always have fun.", "followers_count": 447, "friends_count": 797, "statues_count": 18374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastwood, MI", "id": "5f12b6c0c6c23ac0", "name": "Eastwood", "place_type": "city", "bounding_box": rectangle("-85.56327,42.287426 -85.530762,42.321691") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2624500, "cityName": "Eastwood" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607775426240512, "text": "Ripley SW Limestone Co. Temp: 41.2°F Wind:1.6mph Pressure: 1000.1mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607775501586432, "text": "Bye Mom and Dad ������������ https://t.co/llms1rsq8w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2211504204, "name": "awi", "screen_name": "AriannaLoera", "lang": "en", "location": "LA", "create_at": date("2013-11-23"), "description": "life's too short to be anything but happy", "followers_count": 225, "friends_count": 406, "statues_count": 2016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monrovia, CA", "id": "174309ec90de94d0", "name": "Monrovia", "place_type": "city", "bounding_box": rectangle("-118.023461,34.099706 -117.975357,34.178637") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648648, "cityName": "Monrovia" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607775619002370, "text": "Laying in bed being paranoid now that I saw a spider. Where there's one spider there's four more.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28506129, "name": "christina ference", "screen_name": "c_ference", "lang": "en", "location": "las vegas", "create_at": date("2009-04-02"), "description": "I'll probably like your dog more than you", "followers_count": 189, "friends_count": 143, "statues_count": 8254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607775673720833, "text": "When your up alone �� cause babe knocked !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715142560, "name": "Zaryia ✨", "screen_name": "stunna_a1", "lang": "en", "location": "401 ⚓ Prov ", "create_at": date("2013-10-14"), "description": "SC : Shawtyx17 | ♋ | My Own Wce ❤", "followers_count": 1534, "friends_count": 1512, "statues_count": 19699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-04-06T00:00:00.000Z"), "id": 717607775711268865, "text": "Come on casting agents I can't afford 2 update a photo Now I have to buy an afro puff hair piece from d beauty supply to match my headshot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29626666, "name": "CushGotIssues", "screen_name": "SELF2LUV", "lang": "en", "location": " CaLiForNiA", "create_at": date("2009-04-07"), "description": "✋Yoohoo Actress| Chicago native & we slash tires.|I can be a serious Cicely Tyson & funny Kim Wayans| Goals: training to become a bodybuilder", "followers_count": 797, "friends_count": 2869, "statues_count": 5100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607776172642304, "text": "Got a lot of things accomplished at the good ole house hold meeting tonight #minusone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "minusone" }}, "user": { "id": 709126882998943744, "name": "Kelz", "screen_name": "kshan__", "lang": "en", "location": "null", "create_at": date("2016-03-13"), "description": "null", "followers_count": 59, "friends_count": 114, "statues_count": 43 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607776944455680, "text": "@chelbsmarie then watch it!!", "in_reply_to_status": 717603894390624256, "in_reply_to_user": 47565736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47565736 }}, "user": { "id": 296597421, "name": "Nyal", "screen_name": "NyalRJBarnes", "lang": "en", "location": "Allen, TX/ San Marcos, TX", "create_at": date("2011-05-10"), "description": "|214⛵️512| Texas State '18 | snapchat: nyalbarnes Professional Hairologist, Makeupologist; up and coming photoologist. DM for more info; serious inquiries only", "followers_count": 1034, "friends_count": 699, "statues_count": 24400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777011572738, "text": "Wind 13.0 mph SSW. Barometer 1011.52 mb, Falling Rapidly. Temperature 64.3 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 13655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777196236800, "text": "��️��️ https://t.co/OlNPSCtSKh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2828097478, "name": "DⓂ️oney", "screen_name": "DxMoney_", "lang": "en", "location": "With My Brother Mari", "create_at": date("2014-10-13"), "description": "Dah-v-yah-na not Day-v-ana | sc:lady.louie / Richards 18", "followers_count": 653, "friends_count": 572, "statues_count": 11048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calumet Park, IL", "id": "e81d7291bd2ba45e", "name": "Calumet Park", "place_type": "city", "bounding_box": rectangle("-87.670865,41.654734 -87.641524,41.677541") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1710513, "cityName": "Calumet Park" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777221353472, "text": "Transformation Tuesday. If fuller natural looking eyebrows is what you're after, look no… https://t.co/eHb0YtJxlP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.38252924,33.87304756"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951454840, "name": "Rumany Long", "screen_name": "Rumanystudio", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 2, "friends_count": 48, "statues_count": 154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777409978368, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 735916417, "name": "The Wizard™", "screen_name": "Tha_Great_14", "lang": "en", "location": "Mars", "create_at": date("2012-08-03"), "description": "University of California-Berkeley | Football |DB 14 | Sophomore | Show No Emotion... Feel No Pain...", "followers_count": 766, "friends_count": 447, "statues_count": 11656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777699540992, "text": "No hammer huh meet @myfabolouslife & @DJCLUE no excuse nerds https://t.co/Ra6xFC2RpV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.926606,41.297782"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21278412, 24719760 }}, "user": { "id": 2544795223, "name": "DONRIKOCZAE", "screen_name": "MR8DONRIKOCZAE", "lang": "en", "location": "MRDONRIKOCZAE@AOL.COM", "create_at": date("2014-06-03"), "description": "http://KICKSTARTER.COM/PROOFOFLYFE", "followers_count": 151, "friends_count": 291, "statues_count": 33512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union News Of New Haven", "id": "07d9dbdefac83002", "name": "Union News Of New Haven", "place_type": "poi", "bounding_box": rectangle("-72.9266061,41.2977819 -72.926606,41.297782") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777711955968, "text": "@ebbtideapp Tide in Eatons Neck Point, New York 04/06/2016\n Low 5:05am -0.9\nHigh 11:06am 8.3\n Low 5:26pm -0.9\nHigh 11:27pm 8.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.4,40.9533"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 52, "friends_count": 1, "statues_count": 22073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3602737, "cityName": "Asharoken" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777728770050, "text": "I should be sleeping but here I am reading about Michael Jackson's chimpanzee because I was worried about him", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1579559184, "name": "Run_Away_With_Me.mp3", "screen_name": "skreechybat", "lang": "en", "location": "Heaven or Las Vegas ", "create_at": date("2013-07-08"), "description": "Luna de miel, rosa pastel, cliches y tonterias (He/Him/They)", "followers_count": 444, "friends_count": 502, "statues_count": 46943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607777875533824, "text": "@PamEngel12 https://t.co/zFiprlcOfd", "in_reply_to_status": -1, "in_reply_to_user": 55756399, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 55756399 }}, "user": { "id": 4862697011, "name": "Stephen Cooper", "screen_name": "SteveCooperEsq", "lang": "en", "location": "Woodland Hills, Los Angeles", "create_at": date("2016-01-29"), "description": "Writer & Lawyer", "followers_count": 37, "friends_count": 178, "statues_count": 5569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607778206920704, "text": "wow, did Golden State really lose to the Wolves?? what the hell the wolves doing trying to win anyways?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246027990, "name": "Rell The Soundbender", "screen_name": "RelltheSB", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-02-01"), "description": "Panamanian Diablo. #WAYMO", "followers_count": 3291, "friends_count": 99, "statues_count": 2608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607779419041792, "text": "I'm meaner to the people I love the most ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2946798764, "name": "04/14✨", "screen_name": "yasmeenlegit_", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "lmao bro idc", "followers_count": 474, "friends_count": 291, "statues_count": 23962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duncanville, TX", "id": "c9ff03f5c5cb510a", "name": "Duncanville", "place_type": "city", "bounding_box": rectangle("-96.943349,32.617554 -96.882757,32.676694") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4821628, "cityName": "Duncanville" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607779444375552, "text": "04/06@03:00 - Temp 27.0F, WC 27.0F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.409in, Rising slowly. Rain 0.00in. Hum 79%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607779507249152, "text": "Wind 8.0 mph SE. Barometer 29.851 in, Falling Rapidly. Temperature 54.5 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607780064976897, "text": "@lourdessgee_ there's cream for stretch marks & to prevent stretch marks��", "in_reply_to_status": 717607451999100928, "in_reply_to_user": 3301945728, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3301945728 }}, "user": { "id": 2471370313, "name": "Edgar", "screen_name": "EdgarR_925", "lang": "en", "location": "null", "create_at": date("2014-04-30"), "description": "⚽", "followers_count": 115, "friends_count": 99, "statues_count": 3262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-04-06T00:00:01.000Z"), "id": 717607780086128640, "text": "Wind 1.0 mph SE. Barometer 30.137 in, Falling. Temperature 38.9 °F. Rain today 0.00 in. Humidity 58%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607780580982784, "text": "�������� https://t.co/B9ktKUH7g8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 787895840, "name": "5/054️⃣ Best❣", "screen_name": "yasmine67581884", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-08-28"), "description": "Albert Earl Stevenson", "followers_count": 184, "friends_count": 177, "statues_count": 1241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flossmoor, IL", "id": "97ae6559498bd23d", "name": "Flossmoor", "place_type": "city", "bounding_box": rectangle("-87.713559,41.527807 -87.655073,41.55185") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1726571, "cityName": "Flossmoor" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607780828381185, "text": "@tarachatzakis Thanks Tara you too :)", "in_reply_to_status": 717595327818244096, "in_reply_to_user": 82046896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82046896 }}, "user": { "id": 403600558, "name": "keith roland", "screen_name": "keithroland1", "lang": "en", "location": "milwaukee wisconsin", "create_at": date("2011-11-02"), "description": "Electrical Engineer love meeting new people and love to laugh would love to have you follow me", "followers_count": 4852, "friends_count": 4610, "statues_count": 93918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607780849299456, "text": "Jade: Finally meets her cosplay idol at Katsu 2015\nCosplay Idol: Oh yeah, you made those Homestuck Horns?\nJade: https://t.co/zTGhKYpCxI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2918241822, "name": "drop the wolf", "screen_name": "Luuz0r", "lang": "en", "location": "Minneapolis, MN ", "create_at": date("2014-12-03"), "description": "Corvin (they/them), Bird Nerd who is learning how to draw things with moderate success on good days --\n\nArt Twitter over at @Corvart", "followers_count": 349, "friends_count": 241, "statues_count": 27719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607780903858177, "text": "Apparently.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230854666, "name": "cocoa baby.", "screen_name": "juju_beansweet", "lang": "en", "location": "Dallas/FortWorth| UNT", "create_at": date("2010-12-26"), "description": "null", "followers_count": 1591, "friends_count": 623, "statues_count": 107331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607781369380865, "text": "The Warriors won't break the Bulls record now. The Spurs and Grizzlies will split these last 4. I'm disappointed......and I'm a Bulls fan.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77353783, "name": "H. T®°¥", "screen_name": "HelixTroysworld", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-09-25"), "description": "Bars and Loops.", "followers_count": 475, "friends_count": 152, "statues_count": 6474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607781449277440, "text": "I want chips and dip�� but I don't feel like going down my stairs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242038085, "name": "neishaNeish❣", "screen_name": "NeishaX3_", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "Fuck a bio! I dont need one foreal", "followers_count": 594, "friends_count": 480, "statues_count": 21645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607781738655744, "text": "Temp: 57.5°F Wind:0.0mph Pressure: 30.130hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607782401245185, "text": "@MadJoe13 I love you sweetheart ����", "in_reply_to_status": -1, "in_reply_to_user": 1201995282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1201995282 }}, "user": { "id": 1206490400, "name": "Dalton Rotramel", "screen_name": "daltonjr13", "lang": "en", "location": "Benton, IL", "create_at": date("2013-02-21"), "description": "I'm Dalton Rotramel, I have two boys, Noah and Nolan. Wife Madeline, and I'm a tire builder at Continental Tire. Thats about it!", "followers_count": 28, "friends_count": 131, "statues_count": 194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benton, IL", "id": "74d3114b43ba4c2c", "name": "Benton", "place_type": "city", "bounding_box": rectangle("-88.935521,37.981799 -88.889872,38.023957") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17055, "countyName": "Franklin", "cityID": 1705300, "cityName": "Benton" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607783168782336, "text": "@AirDmaz23 @Mylos562 https://t.co/z63xGdxTUL", "in_reply_to_status": 717607608937422848, "in_reply_to_user": 302762673, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 302762673, 741630212 }}, "user": { "id": 1347332672, "name": "Sin Amor", "screen_name": "ctinoco562", "lang": "en", "location": "null", "create_at": date("2013-04-12"), "description": "tu es belle", "followers_count": 163, "friends_count": 164, "statues_count": 12769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607783177150464, "text": "يبقى (حيدر) بلسم قلوب الحيارى .. \nو استراحوا آدم و نوح بـ جواره ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 732017786, "name": "زَهْرة إيِليـّا", "screen_name": "z_z_saffar", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-08-01"), "description": "أنا : روحٌ نفخَها الله، عقلٌ غذَّاهُ مُحمَّد، وقلبٌ استوطنَهُ علي ••• وحشني شوووف الكوووويت •••", "followers_count": 253, "friends_count": 94, "statues_count": 4548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607783269437440, "text": "why is everything that happens to me the most ridiculous scenario possible", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26707785, "name": "mel", "screen_name": "melparkerrr", "lang": "en", "location": "Danville, CA", "create_at": date("2009-03-26"), "description": "same", "followers_count": 802, "friends_count": 653, "statues_count": 6969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, CA", "id": "aa30747001a23f03", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-122.027412,37.779803 -121.89165,37.847751") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 617988, "cityName": "Danville" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607783621914624, "text": "Wind 0.1 mph SSW. Barometer 30.052 in, Falling. Temperature 63.0 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607783999217664, "text": "Who is this irresistible creature who has an insatiable love for the dead ? Living Dead gurl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543502510, "name": "Jade", "screen_name": "xanaxnymph", "lang": "en", "location": "null", "create_at": date("2012-04-02"), "description": "LA ✈️PORTLAND .STAY TUNED", "followers_count": 328, "friends_count": 111, "statues_count": 33485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607784120848384, "text": "GINOBILIIIIIIIII! https://t.co/8BtKtui99m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "eu", "is_retweet": false, "user": { "id": 456530404, "name": "RaShan", "screen_name": "ThatDudeShan", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-01-06"), "description": "The San Francisco Giants won the World Series on my birthday. 10-29-14 #RIPDad 11-18-15", "followers_count": 1552, "friends_count": 833, "statues_count": 154655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-06T00:00:02.000Z"), "id": 717607784200540160, "text": "In too deep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1714335050, "name": "Jose", "screen_name": "JMacias21", "lang": "en", "location": "null", "create_at": date("2013-08-30"), "description": "Subi Hatch ❤️", "followers_count": 136, "friends_count": 139, "statues_count": 8607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Jose Hills, CA", "id": "945d4c47ec5036f9", "name": "South San Jose Hills", "place_type": "city", "bounding_box": rectangle("-117.923259,34.003576 -117.88006,34.026435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673290, "cityName": "South San Jose Hills" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607785035395072, "text": "Temp 19.3° Hi/Lo 23.3/19.3 Rng 4.0° WC 19.3° Hmd 82% Rain 0.00\" Storm 0.22\" BAR 30.367 Rising DP 14.7° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 117, "statues_count": 17596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607785157025792, "text": "Wind 0.0 mph W. Barometer 30.37 in, Steady. Temperature 22.8 °F. Rain today 0.00 in. Humidity 55%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607785395978240, "text": "58.0F (Feels: 58.0F) - Humidity: 99% - Wind: 1.6mph S - Gust: 1.6mph - Pressure: 1015.5mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 233934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607786486497281, "text": "����happy birthday my lovely sunflower, @RenaLovelis ������ love you so much, beautiful. ����\n\nnow go legally buy yourself a lotto ticket, lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 305916419 }}, "user": { "id": 2222005818, "name": "jess//michaelgc☁️", "screen_name": "mychemical_muke", "lang": "en", "location": "LA, California ☹", "create_at": date("2013-11-29"), "description": "pretty lame. pretty weird. but not pretty. thankfully met ashton irwin once tho & hey violet. 5/4-hv.", "followers_count": 4329, "friends_count": 4270, "statues_count": 55640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607786742308864, "text": "#Trump in 2016 Temp:39.0°F Wind:2.9mph Pressure: 30.12hpa Falling Rain Today 0.00in. Forecast: Fairly fine, showery later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 314424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607786914295808, "text": "High school was the best time of my life but I hated the actually school part. My friends made it worth it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 495439240, "name": "Noah", "screen_name": "nstertz", "lang": "en", "location": "Louisville, OH", "create_at": date("2012-02-17"), "description": "dropped out of college in advance", "followers_count": 696, "friends_count": 285, "statues_count": 57836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, OH", "id": "01624fab56cace5d", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-81.309006,40.80682 -81.225325,40.873524") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3945094, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607787296079877, "text": "@gabbirelly thnx boo ❤️", "in_reply_to_status": 717606266462404608, "in_reply_to_user": 326765051, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326765051 }}, "user": { "id": 2993490646, "name": "Mama Ganja.", "screen_name": "__AbelFleur", "lang": "en", "location": "Eastham, MA", "create_at": date("2015-01-22"), "description": "Jamaican.", "followers_count": 1022, "friends_count": 701, "statues_count": 28821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orleans, MA", "id": "01ab9a62635a2984", "name": "Orleans", "place_type": "city", "bounding_box": rectangle("-70.022426,41.756918 -69.936405,41.810268") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2551405, "cityName": "Orleans" } }
+{ "create_at": datetime("2016-04-06T00:00:03.000Z"), "id": 717607788654931968, "text": "@Mehrija7 oh my god I'm already on season 3", "in_reply_to_status": 716854209946095617, "in_reply_to_user": 3297144768, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3297144768 }}, "user": { "id": 3297144768, "name": "young yeezus", "screen_name": "Mehrija7", "lang": "en", "location": "no upper lip ", "create_at": date("2015-07-27"), "description": "heritage high school '18", "followers_count": 854, "friends_count": 293, "statues_count": 3101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Five Corners, WA", "id": "5d2456d6a8ee247d", "name": "Five Corners", "place_type": "city", "bounding_box": rectangle("-122.606953,45.658578 -122.552528,45.707934") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5324188, "cityName": "Five Corners" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607788977913857, "text": "Ahahhahahah!!! https://t.co/lVxagkA6eZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 464908761, "name": "Deevz", "screen_name": "Baybe_Diva", "lang": "en", "location": "Gold River,Ca", "create_at": date("2012-01-15"), "description": "Young Body, Mature Mind, Old Soul / SC: Divagotdem_madd", "followers_count": 518, "friends_count": 496, "statues_count": 4630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gold River, CA", "id": "90291f178137701d", "name": "Gold River", "place_type": "city", "bounding_box": rectangle("-121.269185,38.608255 -121.212728,38.638108") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 630345, "cityName": "Gold River" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607789204406272, "text": "Wind 3.0 mph SSE. Barometer 29.647 in, Falling. Temperature 42.4 °F. Rain today 0.26 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607790387208194, "text": "@Mariah_love21 smile bitch you have a fat ass ��", "in_reply_to_status": 717587225056153601, "in_reply_to_user": 1159719343, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1159719343 }}, "user": { "id": 733631786, "name": "Jocelyn", "screen_name": "lynnlarks", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-08-02"), "description": "1/2 Lit Bitch 1/2 Emotional Shawty", "followers_count": 1042, "friends_count": 766, "statues_count": 27545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607790550847489, "text": "New Salem Chiropractic, New Salem, ND, 58563 - https://t.co/dEDQURXmGK\n\n#newsalemchiropractic #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.417807,46.856997"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "newsalemchiropractic", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 393, "friends_count": 397, "statues_count": 176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Dakota, USA", "id": "7d893ca2441b0c21", "name": "North Dakota", "place_type": "admin", "bounding_box": rectangle("-104.048915,45.935021 -96.554508,49.000693") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38059, "countyName": "Morton", "cityID": 3856700, "cityName": "New Salem" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791112896512, "text": "Meanwhile it is 2 o'clock in the morning and I'm still up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263282350, "name": "G00TIME", "screen_name": "tooTall_Halie", "lang": "en", "location": "New Orleans ", "create_at": date("2011-03-09"), "description": "#⃣0⃣0⃣ Lady Privateers #UNO18 Biological Science Major", "followers_count": 2469, "friends_count": 2119, "statues_count": 43844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791263789056, "text": "HAPPY BIRTHDAY TO THIS OLD LADY I LOVE YOU APRIL OHH WHAT WOULD I DO WITHOUT YOU (: !!!! @AprilHernandezz https://t.co/ccsiLDa8AR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3671854459 }}, "user": { "id": 3467112674, "name": "nina", "screen_name": "dammarlette", "lang": "en", "location": "Soccer™", "create_at": date("2015-09-05"), "description": "I have a twin (: / AMK ¢", "followers_count": 149, "friends_count": 137, "statues_count": 1116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791381258240, "text": "Everybody has gone through something that has changed them in a way that they could never go back to the person they once were.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2620316065, "name": "Joel Aguilar", "screen_name": "ayyee__joelcx", "lang": "en", "location": "Norwalk, CA", "create_at": date("2014-07-11"), "description": "✝ GRAND CANYON UNIVERSITY C/O 2020 ✝ Psalm 188: 14-16", "followers_count": 590, "friends_count": 2016, "statues_count": 7513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791444144128, "text": "@SteezyKobe wow dude this tweet reminded me I have cookies thank u", "in_reply_to_status": 717604147906740226, "in_reply_to_user": 206541280, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 206541280 }}, "user": { "id": 958758360, "name": "=(", "screen_name": "Tonaay_", "lang": "en", "location": "Fresno", "create_at": date("2012-11-19"), "description": "Dunder Mifflin this is Pam", "followers_count": 511, "friends_count": 394, "statues_count": 39804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791574196224, "text": "Glad you liked it! #musubilove for my babe! �� and hahaha for the biebs �� @hipstertacos https://t.co/SvjkqEYT4P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "musubilove" }}, "user_mentions": {{ 2191385863 }}, "user": { "id": 3044577476, "name": "Farah Gee", "screen_name": "farahgee808", "lang": "en", "location": "Oahu, Hawaii", "create_at": date("2015-02-26"), "description": "null", "followers_count": 141, "friends_count": 155, "statues_count": 1025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791616139264, "text": "I'm so glad OJ was found not guilty on #ThePeopleVsOJSimpson. It wasn't looking good for a while there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThePeopleVsOJSimpson" }}, "user": { "id": 28042270, "name": "Clint Childs", "screen_name": "clintchilds", "lang": "en", "location": "Eastside of LA", "create_at": date("2009-03-31"), "description": "Wolf of Silver Lake Blvd. I learned everything I know in the PG County, MD Public School System.", "followers_count": 632, "friends_count": 414, "statues_count": 5195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607791746134017, "text": "@yg_carter happppppy birthday Bestfriend ���������� I HOPE YOU HAVE AN AMAZING 21ST BDAY!!!!! ����", "in_reply_to_status": 717607018844917760, "in_reply_to_user": 4271850571, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4271850571 }}, "user": { "id": 50122267, "name": "Blair Waldorf.", "screen_name": "NiiiJones_", "lang": "en", "location": "Berkeley.", "create_at": date("2009-06-23"), "description": "UCB'17 | Track&Field • Don't get lost in the sauce.", "followers_count": 1581, "friends_count": 365, "statues_count": 153377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607792379453440, "text": "Day of Birth �� Skr Skr \n#TwoUno #vscocam #randxmphotography #nikond3100 #createexploretakeover @… https://t.co/UVj7J29RSN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.225368,33.93487342"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TwoUno", "vscocam", "randxmphotography", "nikond3100", "createexploretakeover" }}, "user": { "id": 3035628894, "name": "Superman'sDad.", "screen_name": "tinorelz", "lang": "en", "location": "Moval, Cali", "create_at": date("2015-02-21"), "description": "Aye on my momma nigga, imma be the greatest to ever do this shit. -K.Dot\n#RANDXM", "followers_count": 161, "friends_count": 250, "statues_count": 2715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607792392151040, "text": "@parat99 @linaiolanda @potato_samel @mucupurulent показываем. Весь мир показывает.", "in_reply_to_status": 717607543862870016, "in_reply_to_user": 2478610898, "favorite_count": 0, "retweet_count": 0, "lang": "ru", "is_retweet": false, "user_mentions": {{ 2478610898, 2434963412, 363646239, 173476620 }}, "user": { "id": 577855660, "name": "Roman Abramenko", "screen_name": "ralogic", "lang": "en", "location": "Dublin", "create_at": date("2012-05-12"), "description": "Жiдобандера.", "followers_count": 420, "friends_count": 140, "statues_count": 21407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-06T00:00:04.000Z"), "id": 717607792757108736, "text": "@NotQuirky @HPRSays I doubt apna HPR will reply re. Inactive he's been", "in_reply_to_status": 717601806298251264, "in_reply_to_user": 831096883, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 831096883, 2840523450 }}, "user": { "id": 906602304, "name": "Pushpak Jain", "screen_name": "Pushpak168", "lang": "en", "location": "New York, NY", "create_at": date("2012-10-26"), "description": "Management student at New York University. A marketing fan. Love cracking jokes and making funny vines! Manchester United all the way! #GGMU #NYU", "followers_count": 346, "friends_count": 104, "statues_count": 5682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607793109413888, "text": "Temp 37.0°F Wind Chill 37.0°F RH 42% Wind 1.6 NNE Gust 5.0 NNE SLP 30.293 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 107, "friends_count": 63, "statues_count": 34500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607793486745600, "text": "2 in the morning my mind is on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2572417284, "name": "haley", "screen_name": "haley__estes", "lang": "en", "location": "Round Rock, TX", "create_at": date("2014-06-17"), "description": "crhs", "followers_count": 206, "friends_count": 235, "statues_count": 4836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607793658691585, "text": "@garyvee hey Gary saw the snap let me know how I can acquire that signed copy!", "in_reply_to_status": -1, "in_reply_to_user": 5768872, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5768872 }}, "user": { "id": 46730215, "name": "Tim Gatto", "screen_name": "Tim_Gatto", "lang": "en", "location": "Napa, CA", "create_at": date("2009-06-12"), "description": "Branding and Packaging for Wine, Spirits & Beer", "followers_count": 497, "friends_count": 577, "statues_count": 3647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607793910374400, "text": "Cause I knew you were being sarcastic ������ https://t.co/vc2nTVhKy9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 113248706, "name": "VNA", "screen_name": "veeemariee_", "lang": "en", "location": "null", "create_at": date("2010-02-10"), "description": "9teen. | ♒️", "followers_count": 453, "friends_count": 425, "statues_count": 27375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607794027794432, "text": "@WTFitzMariaTho @stephmontemayor WHAAAAAAA FRL????", "in_reply_to_status": 717606542908854272, "in_reply_to_user": 497682677, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 497682677, 1573599180 }}, "user": { "id": 154258690, "name": "Jay R. Sanidad", "screen_name": "jaysaniFATHER", "lang": "en", "location": "Stockton, CA ", "create_at": date("2010-06-10"), "description": "Make every day the best day that you've ever had. #LifeIsAmazing", "followers_count": 601, "friends_count": 507, "statues_count": 21379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607794124304384, "text": "@OGweswalker is the mans", "in_reply_to_status": -1, "in_reply_to_user": 239097614, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 239097614 }}, "user": { "id": 445447759, "name": "Bo Anderson", "screen_name": "boanderson12", "lang": "en", "location": "Bakersfield • Tucson ", "create_at": date("2011-12-24"), "description": "University of Arizona | ΘΔΧ", "followers_count": 619, "friends_count": 204, "statues_count": 7728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607794690490368, "text": "Gunna get my hair color touched up and I have 3 styles in mind but don't quite know what u wanna get :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2400444702, "name": "Lexie ♡", "screen_name": "lexietekk98", "lang": "en", "location": "null", "create_at": date("2014-03-20"), "description": "just ur average stressed out mom w back problems", "followers_count": 380, "friends_count": 501, "statues_count": 5120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607794992525313, "text": "@ryanirelan @ianlandsman $5 is same as free, which is just how I get treated. Charge me $50 and provide real service and some guarantees.", "in_reply_to_status": 717558524050432000, "in_reply_to_user": 13544, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 13544, 1383161 }}, "user": { "id": 15353121, "name": "Swizec", "screen_name": "Swizec", "lang": "en", "location": "San Francisco / LJ", "create_at": date("2008-07-08"), "description": "A geek with a hat", "followers_count": 5164, "friends_count": 1111, "statues_count": 46680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Beach, San Francisco", "id": "5c92ab5379de3839", "name": "South Beach", "place_type": "neighborhood", "bounding_box": rectangle("-122.401378,37.777625 -122.380984,37.798014") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607795353255936, "text": "I owe them nothing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252954044, "name": "Dj Cmix aka DJ Guru", "screen_name": "DjCmix_", "lang": "en", "location": "Baton Rouge, Louisiana", "create_at": date("2011-02-15"), "description": "Owner of C - Mix Productions. We do weddings, clubs,parties, proms, etc.. Full sound system & lights\r\nFor bookings email : cmixbookings@gmail.com\r\nIG: Djcmix", "followers_count": 5191, "friends_count": 4923, "statues_count": 8886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607795848138753, "text": "But could you imagine taking time out of your life, preparing remarks and then being denied your opportunity? https://t.co/YLgm5P3LdD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23096795, "name": "TDL", "screen_name": "tdlove5", "lang": "en", "location": "Oakland, California", "create_at": date("2009-03-06"), "description": "Policy, Health, #Oakland, humor and good times. I live-tweet #Oakmtg. Thanks for following! :)", "followers_count": 1987, "friends_count": 962, "statues_count": 38774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607796158496768, "text": "No one ever asks me how I feel about shit and that makes me feel kinda shitty tbh lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338035359, "name": "leslie gonzalez", "screen_name": "leslieistoocool", "lang": "en", "location": "splendora/san marcos TX", "create_at": date("2011-07-18"), "description": "la vida puede ser hermosa, si tu la dejas ser. #TXST", "followers_count": 2517, "friends_count": 2084, "statues_count": 26934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-06T00:00:05.000Z"), "id": 717607796275982344, "text": "@tay1o it's like.. Wtf you suck & take big dicks for a living.. Why tf you sucking a tiny dick for the snap ��", "in_reply_to_status": 717606992932511744, "in_reply_to_user": 229614421, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 229614421 }}, "user": { "id": 1628316638, "name": "M.", "screen_name": "MsMESHELLVEE", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "20 | HTX | Viet & Trini", "followers_count": 1384, "friends_count": 583, "statues_count": 52681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607797345619968, "text": "Jst leaving the gym", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1972948244, "name": "✨", "screen_name": "sunflowur_", "lang": "en", "location": "#myasu20", "create_at": date("2013-10-19"), "description": "vibin", "followers_count": 4644, "friends_count": 3638, "statues_count": 70331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, MD", "id": "01c689cd0c26beaa", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-76.200658,39.451104 -76.09734,39.544301") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24025, "countyName": "Harford", "cityID": 2400125, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607797618114560, "text": "lmaooooooooooooooooooooooooooooooooooooooooo https://t.co/PSOZv1JvM5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 228761373, "name": "كوثر", "screen_name": "kowtharhussein", "lang": "en", "location": "null", "create_at": date("2010-12-20"), "description": "lord knows,", "followers_count": 1036, "friends_count": 538, "statues_count": 4872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607798016704512, "text": "This highlight live https://t.co/u2pkOfGhAV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554528097, "name": "Å (aka Josh)", "screen_name": "og_moves", "lang": "en", "location": "null", "create_at": date("2012-04-15"), "description": "try to kill my vibe but the ego resurrected", "followers_count": 1352, "friends_count": 1191, "statues_count": 31487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607798167572480, "text": "I can't take another breath without feeling the regret of my jealousy. Horrified that you might… https://t.co/0x4zqAU9nv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.98445056,36.85584295"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 206814512, "name": "bayli", "screen_name": "bayliface", "lang": "en", "location": "null", "create_at": date("2010-10-23"), "description": "Just let it go ➵ just let it be✌️", "followers_count": 209, "friends_count": 215, "statues_count": 6686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arizona, USA", "id": "a612c69b44b2e5da", "name": "Arizona", "place_type": "admin", "bounding_box": rectangle("-114.818269,31.332246 -109.045153,37.004261") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607798712958976, "text": "Wind 0.0 mph ---. Barometer 30.347 in, Steady. Temperature 12.4 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607799111294977, "text": "@jojothevoice https://t.co/4R3tDIkGL4", "in_reply_to_status": 717607312727416833, "in_reply_to_user": 2492025535, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2492025535 }}, "user": { "id": 4677025382, "name": "Kiara Holliday", "screen_name": "kiholliday", "lang": "en", "location": "null", "create_at": date("2015-12-30"), "description": "UL| A simple hello could lead to a million things.", "followers_count": 67, "friends_count": 79, "statues_count": 1296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607799119855617, "text": "❤️✊�� https://t.co/jbKsnXeluB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 228222937, "name": "Momma", "screen_name": "AMirrorMurders", "lang": "en", "location": "Cleveland, OH", "create_at": date("2010-12-18"), "description": "Lil philosopher. Big attorney. PLAI MAKR. 4/2 ✨", "followers_count": 1242, "friends_count": 637, "statues_count": 134334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607799509811200, "text": "@jade_alyshaa @lmao_sav that girl is crazy and @snowfllake_ and I called the police on her once", "in_reply_to_status": 701138206977208320, "in_reply_to_user": 1619852580, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1619852580, 1465906099, 1312973706 }}, "user": { "id": 3098827964, "name": "Dyllón Patterson", "screen_name": "DyllonDelReyy", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2015-03-19"), "description": "•Lisa J Carter Spokesmodel• Mysterious glamorous gay that always sparks your attention. #veggie #feelthebern #starbucksbarista", "followers_count": 694, "friends_count": 227, "statues_count": 4125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607800239591424, "text": "@samanthadel_ the world may never know :(", "in_reply_to_status": 717606609103392768, "in_reply_to_user": 2849719933, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2849719933 }}, "user": { "id": 513700737, "name": "kayleen.", "screen_name": "kay_weenie", "lang": "en", "location": "sc: kay-weenie", "create_at": date("2012-03-03"), "description": "• pretty girls make graves •", "followers_count": 628, "friends_count": 453, "statues_count": 36794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607800583495680, "text": "@DarkMattersProj #FreeTheBrows", "in_reply_to_status": 717603038198894594, "in_reply_to_user": 2433667077, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FreeTheBrows" }}, "user_mentions": {{ 2433667077 }}, "user": { "id": 25080927, "name": "Wicked Annie", "screen_name": "fangirlsmash", "lang": "en", "location": "SAN FRANCISCO. ENVY ME.", "create_at": date("2009-03-18"), "description": "Semiotic Terrorist/Majestrix of the Shi'ar Empire/Pangolin Sentience Appreciator Fangirl Rage Obviated Here!", "followers_count": 587, "friends_count": 1847, "statues_count": 11206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607800684171264, "text": "Seems legit. �� https://t.co/4Rr0cGQNxz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 625416175, "name": "Santiago Esparza", "screen_name": "IrascibleSavage", "lang": "en", "location": "Watching @OscineGehenna's six.", "create_at": date("2012-07-03"), "description": "❝You want me to do what, sir?❞ — Engineer. Technopath. I make things go boom. Sometimes. [♕] 「Agent of SHIELD.」 || #Danti AF. || #Asp", "followers_count": 622, "friends_count": 214, "statues_count": 7038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607800893931520, "text": "Window shopping.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262872641, "name": "Luna✨", "screen_name": "StefiePie_", "lang": "en", "location": "RockCity✈️Wadadli✈️SoFlo", "create_at": date("2011-03-08"), "description": "Feisty|FUN|Lovable  #AriesGang ♈", "followers_count": 1013, "friends_count": 749, "statues_count": 224139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miramar, FL", "id": "5ca2cc5afa0894df", "name": "Miramar", "place_type": "city", "bounding_box": rectangle("-80.40816,25.956799 -80.205968,25.995449") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1245975, "cityName": "Miramar" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607800902328320, "text": "@danyovalera I H8 U ��", "in_reply_to_status": 717606501888602112, "in_reply_to_user": 79834388, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 79834388 }}, "user": { "id": 3685892534, "name": "stef", "screen_name": "stefl0uise", "lang": "en", "location": "Hawaii", "create_at": date("2015-09-25"), "description": "nah, don't sweat it", "followers_count": 181, "friends_count": 195, "statues_count": 2948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aiea, HI", "id": "001d16420d512eaf", "name": "Aiea", "place_type": "city", "bounding_box": rectangle("-157.943639,21.374177 -157.85149,21.421297") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1500550, "cityName": "Aiea" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607801279754241, "text": "Temp: 38.9°F - Dew Point: 36.2° - Wind: 16.7 mph - Gust: 27.3 - Rain Today: 0.33in. - Pressure: 29.82in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 15867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-06T00:00:06.000Z"), "id": 717607801317502976, "text": "it's your day girl! Im gonna make sure you have a good one thanks 4 dealing w/my bs you're a real one so glad to be friends w/u Luv you��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2950581529, "name": "analysa", "screen_name": "analysamarie_", "lang": "en", "location": "Palm Desert, CA", "create_at": date("2014-12-29"), "description": "spoiled Brat who just wants some Luv & Attention", "followers_count": 402, "friends_count": 233, "statues_count": 15977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607802294771712, "text": "COLLEGE PAPERS ARE LIT SUH DUDE. https://t.co/6XbfgMES7Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2820103578, "name": "MD", "screen_name": "MarshalDeaton", "lang": "en", "location": "Lafayette, LA", "create_at": date("2014-09-19"), "description": "I CAN DO BETTER THAN THIS. I PROMISE I LOOK BETTER IN REAL LIFE THAN I DO ON HERE.", "followers_count": 242, "friends_count": 327, "statues_count": 67 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607802588409856, "text": "no one �� https://t.co/Pi2oQIHQvW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875861535, "name": "Carrie", "screen_name": "cawwiebear", "lang": "en", "location": "null", "create_at": date("2014-11-13"), "description": "null", "followers_count": 366, "friends_count": 326, "statues_count": 2456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607802672398337, "text": "I'm playing dragon age 2 rock out to tdg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3372115218, "name": "Steven Williams emo", "screen_name": "WilliamsSteven0", "lang": "en", "location": "Lothian, MD", "create_at": date("2015-08-28"), "description": "game nerd & anime guy. I like rock music Wrestling I am 24 I have a love and I have furry wolf girl not my art Skyrim lover and Dragon age I'm a furry", "followers_count": 1732, "friends_count": 4987, "statues_count": 5537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forestville, MD", "id": "0173c19a1ce41de0", "name": "Forestville", "place_type": "city", "bounding_box": rectangle("-76.902529,38.825445 -76.829437,38.880084") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2429000, "cityName": "Forestville" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607802705981440, "text": "@_TriggaTrent ok", "in_reply_to_status": 717607002277416960, "in_reply_to_user": 494195823, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 494195823 }}, "user": { "id": 237051181, "name": "Notorious", "screen_name": "50ShadesOfClair", "lang": "en", "location": "NY✈️GA✈️Valdosta", "create_at": date("2011-01-11"), "description": "God Got Me | GT-Trini | #VSU17 #NAACP #HabitatForHumanity | Honorary Jackson", "followers_count": 5332, "friends_count": 5105, "statues_count": 250559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valdosta, GA", "id": "5e1c91065bc30991", "name": "Valdosta", "place_type": "city", "bounding_box": rectangle("-83.374825,30.762483 -83.230199,30.949686") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1378800, "cityName": "Valdosta" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607802718576640, "text": "@JabariTheGreat @TeannaTrump she ignores me to dont worry", "in_reply_to_status": 699691124110204928, "in_reply_to_user": 363379988, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363379988, 2278455931 }}, "user": { "id": 261066340, "name": "Danny", "screen_name": "FadedAmbitions", "lang": "en", "location": "DMarylandV", "create_at": date("2011-03-04"), "description": "RIP Joel • @_PoloLoco • @DopeDpㅤ⠀⠀⠀⠀⠀⠀", "followers_count": 2900, "friends_count": 466, "statues_count": 86596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Landover, MD", "id": "0051db7004c337bf", "name": "Landover", "place_type": "city", "bounding_box": rectangle("-76.917955,38.907419 -76.851914,38.945052") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445325, "cityName": "Landover" } }
+{ "create_at": datetime("2016-04-06T00:00:07.000Z"), "id": 717607803096023040, "text": "#GeneralDisturbance at 813 Avondale Ave. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.387236,28.5319212"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "GeneralDisturbance", "orlpol", "opd" }}, "user": { "id": 39049856, "name": "Police Calls 32805", "screen_name": "orlpol32805", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 247, "friends_count": 1, "statues_count": 115588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970159994929153, "text": "@nathan_bevan @cher love love love that song !!!", "in_reply_to_status": 717968398680862722, "in_reply_to_user": 325815263, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 325815263, 124003770 }}, "user": { "id": 1523985152, "name": "Russell G. Looney", "screen_name": "Llynnfan68G", "lang": "en", "location": "Lee's Summit Mo.", "create_at": date("2013-06-16"), "description": "Lover of music ! Movies / Musical's I'm 47 year old male ! Me & My Spouse Dean Celebrated 22 yrs. 5/1/15", "followers_count": 267, "friends_count": 587, "statues_count": 2977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lee's Summit, MO", "id": "6d91b7c06b86638b", "name": "Lee's Summit", "place_type": "city", "bounding_box": rectangle("-94.45916,38.805502 -94.297303,39.026684") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2941348, "cityName": "Lee's Summit" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970160246611968, "text": "日本からの留学生が日本ネタで学位論文を書かないことはむしろ少ない。また、そのように教授が薦めるから致し方ないのかも。だけどね_| ̄|○\nいや、人文・社会科学系での話です。", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 107890250, "name": "自称Busker♪=DrMarks本人", "screen_name": "MarkWatermanPhD", "lang": "en", "location": "LosAngeles-Heidelberg-Caesarea", "create_at": date("2010-01-23"), "description": "A good-for-nothing crusty old bugger (˘Ω˘) 要するに、役立たずで嫌われ者の頑固ジジイということ(٭ڋ٭) Science editor and theologian/biblical scholar with PhD became a would-be busker♫", "followers_count": 545, "friends_count": 56, "statues_count": 50710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970160556986368, "text": "Brian need to chill with tweeting these song lyrics acting like its 2013", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4478877493, "name": "Chris Rendon", "screen_name": "Chris310Rendon", "lang": "en", "location": "Inglewood, CA", "create_at": date("2015-12-06"), "description": "new account who dis", "followers_count": 162, "friends_count": 231, "statues_count": 2196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970160905297920, "text": "@Happy_Gollidays �� https://t.co/jAZHDndPoQ", "in_reply_to_status": -1, "in_reply_to_user": 546409372, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 546409372 }}, "user": { "id": 3171442918, "name": "Wes Jones", "screen_name": "wes_won", "lang": "en", "location": "null", "create_at": date("2015-04-15"), "description": "null", "followers_count": 118, "friends_count": 287, "statues_count": 359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allendale, MI", "id": "5b0caebf50f0bf61", "name": "Allendale", "place_type": "city", "bounding_box": rectangle("-85.973029,42.943183 -85.876713,42.99425") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2601340, "cityName": "Allendale" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970160938651648, "text": "What or who did you find ? LOL https://t.co/AhPl2ysVGm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2612106570, "name": "Rodney K", "screen_name": "hotrod6273", "lang": "en", "location": "California ", "create_at": date("2014-07-08"), "description": "From Los Angeles born and raised. Living in the bay area of California. Ex military/Air Force AVI is myself and my husband. #betrue #noh8", "followers_count": 1186, "friends_count": 2389, "statues_count": 10416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970161056124929, "text": "I be missing home the moment I hop on the plane ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41010845, "name": "o_O", "screen_name": "daaavie", "lang": "en", "location": "Long Beach | LA ,CA ✈️ OR ", "create_at": date("2009-05-18"), "description": "#teamG | free my brother @slim218st", "followers_count": 921, "friends_count": 353, "statues_count": 119559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970161991438336, "text": "BK is overall trash, chicken fries & $1 icees are good tho", "in_reply_to_status": 717970021465468928, "in_reply_to_user": 298944084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298944084, "name": "Batfleck", "screen_name": "_RaymondFranko", "lang": "en", "location": "lost", "create_at": date("2011-05-14"), "description": "The revolution will not be televised. The revolution is here.", "followers_count": 717, "friends_count": 287, "statues_count": 48235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970162385883136, "text": "Wind 3.3 mph WSW. Barometer 29.83 in, Rising slowly. Temperature 42.8 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 23, "friends_count": 0, "statues_count": 31859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970162423504900, "text": "@ThatBoyEman Emmanuel", "in_reply_to_status": 717970051815645184, "in_reply_to_user": 288002176, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 288002176 }}, "user": { "id": 2658353896, "name": "allison", "screen_name": "herbalsn", "lang": "en", "location": "4/20", "create_at": date("2014-06-30"), "description": "doin this thang for my guardian angel", "followers_count": 678, "friends_count": 338, "statues_count": 33812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970162771759104, "text": "Wind 2.3 mph SE. Barometer 29.795 in, Falling Rapidly. Temperature 36.0 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 4, "friends_count": 2, "statues_count": 9854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970162977218560, "text": "Ripley SW Limestone Co. Temp: 47.3°F Wind:0.0mph Pressure: 991.1mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970163031752704, "text": "@JSNathalie no eres un perro, Jaque querida ��", "in_reply_to_status": 717969396736479232, "in_reply_to_user": 3215405974, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3215405974 }}, "user": { "id": 3103508387, "name": "Jak Metheny", "screen_name": "anticruzmissile", "lang": "en", "location": "Trump Towers", "create_at": date("2015-03-21"), "description": "What’s It All About?", "followers_count": 1356, "friends_count": 998, "statues_count": 42594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-07T00:00:00.000Z"), "id": 717970163648212992, "text": "Been not so motivated recently with lifting & real estate. Realized I don't have goals to reach. Goals are what drive you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506266252, "name": "Robby James Delwarte", "screen_name": "RobbyyJames", "lang": "en", "location": "null", "create_at": date("2012-02-27"), "description": "Real Estate the best investment | Olson Agency BRE: 01997116", "followers_count": 193, "friends_count": 285, "statues_count": 1596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164046630912, "text": "@SalterLeilani still relevant btw", "in_reply_to_status": 692950335812411392, "in_reply_to_user": 2194933761, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2194933761 }}, "user": { "id": 2194933761, "name": "lani", "screen_name": "SalterLeilani", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "I really like enchiladas. Long Beach, CA. ▶️ Pendleton, OR. || Blue Mountain Softball ||", "followers_count": 564, "friends_count": 481, "statues_count": 20131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pendleton, OR", "id": "2fb80bcccadf3ab4", "name": "Pendleton", "place_type": "city", "bounding_box": rectangle("-118.871352,45.640013 -118.747355,45.704491") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41059, "countyName": "Umatilla", "cityID": 4157150, "cityName": "Pendleton" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164164259840, "text": "Let's Grill! #8551 @AnyaeBell @thd278 https://t.co/2NITyZDIcB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27340541, 3359243789 }}, "user": { "id": 2794649604, "name": "Paris Sachek", "screen_name": "parisaubrey10", "lang": "en", "location": "South Riding, VA", "create_at": date("2014-09-06"), "description": "Live, Laugh and Love what you do!", "followers_count": 101, "friends_count": 153, "statues_count": 78 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Riding, VA", "id": "012bcc3675fdcc96", "name": "South Riding", "place_type": "city", "bounding_box": rectangle("-77.545878,38.887183 -77.473381,38.938016") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5174100, "cityName": "South Riding" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164185047040, "text": "Happy birthday Bae @SirajTheMirage ❤️ I'm first bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 627892080 }}, "user": { "id": 336997035, "name": "Vérsace Tristan", "screen_name": "TKTwilligear", "lang": "en", "location": "Pullman⛅️", "create_at": date("2011-07-17"), "description": "bishop | broke college student & professional sign stealer #BoomSquad | ΔΨ | Marvins Room |", "followers_count": 1451, "friends_count": 515, "statues_count": 77403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164487159808, "text": "Idk why I'm up I got school in the AM. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 529034622, "name": "❗️Deejaye❗️", "screen_name": "Lucx_brxsi", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2012-03-18"), "description": "C/O:17| Free jameshunter| July22| 7Teen||", "followers_count": 347, "friends_count": 225, "statues_count": 1237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Charles, LA", "id": "21b6f72f9fdaeb06", "name": "Lake Charles", "place_type": "city", "bounding_box": rectangle("-93.314852,30.124859 -93.080595,30.280683") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2241155, "cityName": "Lake Charles" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164533342208, "text": "@YungSavage_3 no ��... Never ��", "in_reply_to_status": 717969433105305600, "in_reply_to_user": 956754463, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 956754463 }}, "user": { "id": 832648602, "name": "Ro...", "screen_name": "DarionT12", "lang": "en", "location": "Corsicana, TX", "create_at": date("2012-09-18"), "description": "|R.I.P. Mother| I tweet what's on my mind .. Open minded and one of the last nice niggas on earth lmao .. Enjoy life ..young mind becoming a leader", "followers_count": 2077, "friends_count": 1754, "statues_count": 64002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corsicana, TX", "id": "faa25283bbfe186a", "name": "Corsicana", "place_type": "city", "bounding_box": rectangle("-96.537914,32.054664 -96.415163,32.131294") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48349, "countyName": "Navarro", "cityID": 4817060, "cityName": "Corsicana" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970164742889472, "text": "got out of bed at 10 to hoop now I'm going to Marie's lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509496850, "name": "Kevin C.", "screen_name": "KevChowPow", "lang": "en", "location": "on the one ", "create_at": date("2012-02-29"), "description": "just hold a nigga down through this bad weather.", "followers_count": 640, "friends_count": 384, "statues_count": 43988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970165015650305, "text": "Please answer https://t.co/RphRN8oEVh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2381394596, "name": "Chauce da Honest One", "screen_name": "Chaucerjr", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-03-09"), "description": "21 #ArtnSoul #ChicagoBred None of this juice matters if no worthy candidate partakes of the chalice", "followers_count": 1145, "friends_count": 1000, "statues_count": 91020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970165044895744, "text": "Holy Christ, sports are easier. #FeelTheBern #gopscaresthefuckouttame", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FeelTheBern", "gopscaresthefuckouttame" }}, "user": { "id": 4102778599, "name": "TXJayhawk", "screen_name": "jo_washburn", "lang": "en", "location": "Arlington, TX", "create_at": date("2015-11-02"), "description": "Lover of all things sports - esp. KU hoops. Also love funny people, happy people, liberal people.", "followers_count": 62, "friends_count": 199, "statues_count": 254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Desert, CA", "id": "b075d78174642ce2", "name": "Palm Desert", "place_type": "city", "bounding_box": rectangle("-116.421818,33.670765 -116.300776,33.810312") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655184, "cityName": "Palm Desert" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970165284077568, "text": "Still up wit da deals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62995465, "name": "Beezy", "screen_name": "All_Hale_Nikki", "lang": "en", "location": "SouthC. /Virgina", "create_at": date("2009-08-04"), "description": "Stay real Neva fold | Norfolk Virgina | Put Yourself On Ep Coming Soon | nikkibrgss@gmail.com", "followers_count": 929, "friends_count": 811, "statues_count": 18620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970165376221184, "text": "Wind 0.0 mph SW. Barometer 1015.82 mb, Falling. Temperature 46.2 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 13679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970166492086272, "text": "Wind 1.0 mph NW. Barometer 29.738 in, Steady. Temperature 50.0 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970166919905281, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3849123436, "name": "francesca", "screen_name": "frankierae_", "lang": "en", "location": "Shippensburg University '19", "create_at": date("2015-10-02"), "description": "basic ✨", "followers_count": 811, "friends_count": 540, "statues_count": 31571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shippensburg, PA", "id": "a3d888f1d0b5d5a2", "name": "Shippensburg", "place_type": "city", "bounding_box": rectangle("-77.549194,40.03646 -77.499257,40.067857") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4270352, "cityName": "Shippensburg" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970167200763904, "text": "@PrisilaChavarin okay forsure ��", "in_reply_to_status": 717969930230968320, "in_reply_to_user": 2423530500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2423530500 }}, "user": { "id": 3287243407, "name": "Grecia Saavedra", "screen_name": "greciarsaavedra", "lang": "en", "location": "null", "create_at": date("2015-07-21"), "description": "null", "followers_count": 322, "friends_count": 188, "statues_count": 1775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970167947476992, "text": "I could never lead nobody on I'm to honest ���� idk way to say out my mouth either so Imma tell u something I ain't wanna hear but o well", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53594778, "name": "Humble G", "screen_name": "MYKELLLLL", "lang": "en", "location": "Baltimore , MD", "create_at": date("2009-07-03"), "description": "LONDON's father |NLMB never leave my brother|NBA #longlivejolley #longlivekoby fly high young soldiers", "followers_count": 4582, "friends_count": 2101, "statues_count": 112591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-07T00:00:01.000Z"), "id": 717970167981031424, "text": "Wind 0.0 mph ---. Barometer 29.933 in, Steady. Temperature 54.3 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970168593440768, "text": "04/07@03:00 - Temp 49.5F, WC 49.5F. Wind 2.0mph SSE, Gust 5.0mph. Bar 29.846in, Falling. Rain 0.00in. Hum 88%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970168727650304, "text": "Wind 4.0 mph W. Barometer 29.669 in, Falling slowly. Temperature 48.7 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169264476161, "text": "I need to take a piss but my dog is sleeping on my butt :///", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 436322969, "name": "kim♉", "screen_name": "fetushippie", "lang": "en", "location": "intersectional feminist ", "create_at": date("2011-12-13"), "description": "cute xicana", "followers_count": 715, "friends_count": 454, "statues_count": 24757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylors, SC", "id": "0eec6a9635525944", "name": "Taylors", "place_type": "city", "bounding_box": rectangle("-82.396441,34.881534 -82.26651,34.996401") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4571395, "cityName": "Taylors" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169268600833, "text": "KH is on android too now\nhttps://t.co/fLPyCMf4OD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119291248, "name": "Luis Alamilla", "screen_name": "Luis_GVN", "lang": "en", "location": "LA , California", "create_at": date("2010-03-02"), "description": "Youtube Content Creator & Streamer on Youtube Gaming. #GvnFam \n\nBusiness Email - luis@gamingvlognetwork.com", "followers_count": 2125, "friends_count": 673, "statues_count": 53970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169335689216, "text": "@MosyeAbr You sure do love to generalize. I guess this is what you meant between elite and non-elite Eritreans. @Erena1991 @nicolasF24", "in_reply_to_status": 717843872513134594, "in_reply_to_user": 1016273988, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1016273988, 1965097128, 50590713 }}, "user": { "id": 1122961266, "name": "The Truth", "screen_name": "ConsciousEri", "lang": "en", "location": "null", "create_at": date("2013-01-26"), "description": "Exposing lies about Eritrea and all nations who do not cave to hegemonic powers.", "followers_count": 1028, "friends_count": 225, "statues_count": 28036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169348427777, "text": "@_pimpzaddysheaa don't count", "in_reply_to_status": 717970031691350020, "in_reply_to_user": 43219068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43219068 }}, "user": { "id": 236156697, "name": "Kenny", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | #EnginEars | #KennythingCanHappen | Mix x Master $40 Per Song | excusetherant@gmail.com", "followers_count": 791, "friends_count": 386, "statues_count": 107146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169482645504, "text": "I hate my eating habits. That's literally the only thing killing me right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430293782, "name": "Christian Garcia", "screen_name": "CougarTron5000", "lang": "en", "location": "null", "create_at": date("2011-12-06"), "description": "Grind ti'll you succeed. Never give up. You got 1 life. Live it. No Regrets. \n Lit snaps @Cougartron5000 \nLit IG @Christian14g", "followers_count": 212, "friends_count": 173, "statues_count": 6704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169511809024, "text": "61.9F (Feels: 61.9F) - Humidity: 96% - Wind: 6.0mph NW - Gust: 6.0mph - Pressure: 1011.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 234077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169650282496, "text": "soy bean put her toys in my suitcase and keeps going back inside my suitcase as I'm packing ������ https://t.co/a5jLYK1agM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544518424, "name": "farquad", "screen_name": "ricerealnice", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2012-04-03"), "description": "Comtemplating life from my Bathtub", "followers_count": 1088, "friends_count": 145, "statues_count": 11311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970169880924161, "text": "Thankssss doll ��, thasss why I pick u ! https://t.co/PA6t2UnpCn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3613194919, "name": "k e f e", "screen_name": "allyvaga1", "lang": "en", "location": "Seattle, WA", "create_at": date("2015-09-18"), "description": "x v a a v l i e e r r i a e", "followers_count": 335, "friends_count": 293, "statues_count": 7624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Renton, WA", "id": "a96f188f5de647b5", "name": "Renton", "place_type": "city", "bounding_box": rectangle("-122.253717,47.423263 -122.135079,47.539933") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357745, "cityName": "Renton" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170187112448, "text": "�� sounds intense https://t.co/k6X96t7EEq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4556137952, "name": "&E Jimenez", "screen_name": "Andy_wrx", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-12-14"), "description": "I'm a mess", "followers_count": 262, "friends_count": 282, "statues_count": 1004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell Gardens, CA", "id": "9da8666a1bd226d8", "name": "Bell Gardens", "place_type": "city", "bounding_box": rectangle("-118.170748,33.951888 -118.128456,33.97938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604996, "cityName": "Bell Gardens" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170266853376, "text": "Happy C-Day big sis !!!!!!!!! @cherryMight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2299339046 }}, "user": { "id": 2374703971, "name": "Anakin", "screen_name": "BRenteria_45", "lang": "en", "location": "null", "create_at": date("2014-03-05"), "description": "#98'LBC \n《BpHs》", "followers_count": 461, "friends_count": 582, "statues_count": 16033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170434564096, "text": "You're a Halsey fan? ������ @kiwikiid https://t.co/zl0GSq4sPV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2263597634 }}, "user": { "id": 2799196884, "name": "Jose Vazquez", "screen_name": "whyalwaysnano", "lang": "en", "location": "some place higher", "create_at": date("2014-09-08"), "description": "Just learn to love pain and be patient", "followers_count": 342, "friends_count": 494, "statues_count": 8968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170489102336, "text": "&& i hope you believe me..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3098624180, "name": "P R I N C € $ S", "screen_name": "50shadesoftiara", "lang": "en", "location": "null", "create_at": date("2015-03-19"), "description": "Tyrell Davis ❤", "followers_count": 292, "friends_count": 328, "statues_count": 6666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170514268160, "text": "Temp: 61.4°F Wind:0.0mph Pressure: 29.929hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170673815552, "text": "Wake TF up ! @Shakiyaaa_xoxo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 406014451 }}, "user": { "id": 160308745, "name": "Just Call me Cam", "screen_name": "_CamRon__", "lang": "en", "location": "757 ✈️ 919 ✈️ 404", "create_at": date("2010-06-27"), "description": "Lowkey |", "followers_count": 1118, "friends_count": 1092, "statues_count": 44164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Candler-McAfee, GA", "id": "7202b2c7918d203e", "name": "Candler-McAfee", "place_type": "city", "bounding_box": rectangle("-84.309736,33.712266 -84.230819,33.739899") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1312834, "cityName": "Candler-McAfee" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970170694746112, "text": "It's time to #Vegas @thekillers #ZapposSecretShow @BunkhouseLV #TheKillers #OnTop https://t.co/TYB214BBkn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Vegas", "ZapposSecretShow", "TheKillers", "OnTop" }}, "user_mentions": {{ 19661294, 355838324 }}, "user": { "id": 45927998, "name": "Shaun Leonard", "screen_name": "shaun_leonard", "lang": "en", "location": "Las Vegas", "create_at": date("2009-06-09"), "description": "Irish writer in America", "followers_count": 273, "friends_count": 330, "statues_count": 1103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970171004981249, "text": "Try to say I'm still him from the past", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476524002, "name": "Angel Dorado", "screen_name": "AngelTheSexyOne", "lang": "en", "location": "null", "create_at": date("2012-01-27"), "description": "null", "followers_count": 133, "friends_count": 228, "statues_count": 1509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Home Gardens, CA", "id": "821a205015ebda82", "name": "Home Gardens", "place_type": "city", "bounding_box": rectangle("-117.532621,33.871339 -117.495868,33.88969") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 634302, "cityName": "Home Gardens" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970171193724929, "text": "I wanna do a photoshoot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2430724230, "name": "Baz_Boy", "screen_name": "anthonybasaldua", "lang": "en", "location": "SC: anthony_baz", "create_at": date("2014-04-06"), "description": "A day without laughter is a day wasted", "followers_count": 843, "friends_count": 720, "statues_count": 12881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atwater, CA", "id": "676c3e6a1e0642db", "name": "Atwater", "place_type": "city", "bounding_box": rectangle("-120.641299,37.325641 -120.555357,37.374957") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 603162, "cityName": "Atwater" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970171298598913, "text": "@delaneybruckner HE BETTER CATCH THESE FUCKING FISTS RN", "in_reply_to_status": 717970111869521920, "in_reply_to_user": 3643385654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3643385654 }}, "user": { "id": 20344109, "name": "Kristin Onias", "screen_name": "Krisserbombs", "lang": "en", "location": "Rowland Heights", "create_at": date("2009-02-07"), "description": "It's okay, I creep too. HTH bellhop. chipotle is my life. I demand people to fight me on the regular. Snapchat:krisserbombs", "followers_count": 405, "friends_count": 390, "statues_count": 14248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970171663482880, "text": "If you're a #Retail professional in #Dearborn, MI, check out this #job: https://t.co/MOgo6dxOa2 #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2007066,42.3272182"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Dearborn", "job", "Hiring" }}, "user": { "id": 128246428, "name": "Citi Trends Jobs", "screen_name": "cititrendsjobs", "lang": "en", "location": "null", "create_at": date("2010-03-31"), "description": "Citi Trends has Sales Associate Jobs, Store Manager Jobs, Distribution center Jobs and other great career opportunities.", "followers_count": 1751, "friends_count": 1887, "statues_count": 13719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn, MI", "id": "339bd7ae6a55ba9f", "name": "Dearborn", "place_type": "city", "bounding_box": rectangle("-83.287094,42.277554 -83.14002,42.35191") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621000, "cityName": "Dearborn" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970172024373248, "text": "Temp 44.5° Hi/Lo 44.5/39.6 Rng 4.9° WC 44.5° Hmd 86% Rain 0.00\" Storm 0.28\" BAR 29.947 Falling DP 40.6° Wnd 0mph Dir --- Gst 10mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 117, "statues_count": 17626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970172154261505, "text": "its me and @queefz one year anniversary today https://t.co/iWwjRyVthV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2991564046 }}, "user": { "id": 306853279, "name": "ash", "screen_name": "fuckashIey", "lang": "en", "location": "z", "create_at": date("2011-05-28"), "description": "null", "followers_count": 2620, "friends_count": 111, "statues_count": 84978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970172183642112, "text": "Well im 18 now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1395158748, "name": "_DA9briian", "screen_name": "brianjorge04", "lang": "en", "location": "null", "create_at": date("2013-05-01"), "description": "Senior, SC: briiann99", "followers_count": 242, "friends_count": 228, "statues_count": 5088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-04-07T00:00:02.000Z"), "id": 717970172355551232, "text": "@_WolfHayleyyy that's the reason I'll be in jail. For putting my foot knee high in yo ass.", "in_reply_to_status": 717969959796641792, "in_reply_to_user": 181470958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181470958 }}, "user": { "id": 2292000883, "name": "Charles Gotti", "screen_name": "PapiChuloCharlz", "lang": "en", "location": "null", "create_at": date("2014-01-14"), "description": "R.I.P Jared T. Fair | #HTTR", "followers_count": 278, "friends_count": 188, "statues_count": 14541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970172426866688, "text": "Omg these other apartments are hideous yet way cheaper but i refuse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261463861, "name": "Theyoncé➰", "screen_name": "the0nna_xo", "lang": "en", "location": "Raleigh, NC", "create_at": date("2011-03-05"), "description": "Confidence is the real key, bitch! • aren't you the girl who makes cupcakes? • | sc: the0nna_xo ✨| master mind of @TheyoncePastry_", "followers_count": 1818, "friends_count": 1036, "statues_count": 334587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970172632367105, "text": "Nigga whaa? I'm innat hoe https://t.co/tvKmjAH7ab", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323409233, "name": "Tommy Strawn", "screen_name": "LoLife_", "lang": "en", "location": "S.W H O U S T O N Tx", "create_at": date("2011-06-24"), "description": "My worst fear is going broke cause I'm bad with money....", "followers_count": 732, "friends_count": 597, "statues_count": 120909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970172959588352, "text": "Who would of known that it would turn out this way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3192888727, "name": "Eyeecandyyyy✨", "screen_name": "Lashabadbutt", "lang": "en", "location": "null", "create_at": date("2015-05-11"), "description": "bae area", "followers_count": 259, "friends_count": 196, "statues_count": 2815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970173064515584, "text": "you a fuckin creep���������� https://t.co/L6SpyrijIU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69229846, "name": "Lougotca$h", "screen_name": "Lougotcash", "lang": "en", "location": "NY ✈✈ATL", "create_at": date("2009-08-27"), "description": "I RAP #DREAMSFROMTHEBRICKS MIXTAPE NOW ON SPINRILLA http://spnr.la/IBD1xPHX #Vstate Rip Swint Rip Nana #FreeRico #FreeGunna", "followers_count": 5185, "friends_count": 3313, "statues_count": 143777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970173278482432, "text": "Wo liegt Indianapolis? https://t.co/YP70k8Syc2 #Indianapolis #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1563,39.767"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Indianapolis", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 460, "friends_count": 116, "statues_count": 1901026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianapolis, IN", "id": "018929347840059e", "name": "Indianapolis", "place_type": "city", "bounding_box": rectangle("-86.348441,39.631677 -85.937404,39.927448") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1836003, "cityName": "Indianapolis city (balance)" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970173555261440, "text": "Wind 0.7 mph S. Barometer 29.80 in, Falling. Temperature 42.4 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970173609676800, "text": "I love my gf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45948852, "name": "Dammit Alex", "screen_name": "ItsAlejandrohhh", "lang": "en", "location": "20", "create_at": date("2009-06-09"), "description": "Server at Pizza Hut @OhwowValerie_ is the gal.", "followers_count": 444, "friends_count": 334, "statues_count": 37970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174020726784, "text": "Aye bout to hit up The Donut Man ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372061695, "name": "Aimee Rodriguez", "screen_name": "AimeeRodz25", "lang": "en", "location": "Burbank, California", "create_at": date("2011-09-11"), "description": "Mount SAC", "followers_count": 278, "friends_count": 259, "statues_count": 12323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut, CA", "id": "2fd93ef04260afec", "name": "Walnut", "place_type": "city", "bounding_box": rectangle("-117.895753,34.00286 -117.825003,34.063424") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683332, "cityName": "Walnut" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174054371328, "text": "3am��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2911624941, "name": "Wolftylah.", "screen_name": "xCoolin_", "lang": "en", "location": "The 617", "create_at": date("2014-12-08"), "description": "North side coolin.", "followers_count": 150, "friends_count": 134, "statues_count": 8792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milton, MA", "id": "a1bc60ce60b4052b", "name": "Milton", "place_type": "city", "bounding_box": rectangle("-71.130699,42.200659 -71.031508,42.277667") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2541725, "cityName": "Milton" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174150905860, "text": "#Trump in 2016 Temp:48.2°F Wind:2.9mph Pressure: 29.71hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers likely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 314508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174167687168, "text": "I wonder who gone be here in the end ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2869335060, "name": "Most Hated ✨❣", "screen_name": "_brianna365", "lang": "en", "location": "null", "create_at": date("2014-10-21"), "description": "new twitter follow me", "followers_count": 1870, "friends_count": 1097, "statues_count": 35571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174196879360, "text": "Thanks #siriusxm #williesroadhouse for playing #MerleHaggard all day. #OnlyOneMerle #april62016 https://t.co/MtVGmuJrMa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "siriusxm", "williesroadhouse", "MerleHaggard", "OnlyOneMerle", "april62016" }}, "user": { "id": 96714165, "name": "Cari Forrester", "screen_name": "SnowhiteXO", "lang": "en", "location": "Murrieta, CA", "create_at": date("2009-12-13"), "description": "null", "followers_count": 19, "friends_count": 21, "statues_count": 35 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174423355392, "text": "I've done so much for them. I never expected anything in return but I didn't expect to get used either", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2555648234, "name": "Vyrrah", "screen_name": "Vyyrrah", "lang": "en", "location": "Long Beach, CA/San Diego, CA", "create_at": date("2014-06-08"), "description": "Ward Maiden. I play #leagueoflegends too much. CSULB 2017.", "followers_count": 28, "friends_count": 195, "statues_count": 742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174503030785, "text": "Lol maybe it's just me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276398380, "name": "❣zeeѕнawтy❣", "screen_name": "_zaarriiaa_", "lang": "en", "location": "null", "create_at": date("2011-04-03"), "description": "тαι❤️", "followers_count": 268, "friends_count": 290, "statues_count": 2891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174582755328, "text": "Eat a pepperoni pizza pretzel hotdog at this Australian stoner food restaurant https://t.co/3dGieZYVCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17625, "friends_count": 17555, "statues_count": 70306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174586978304, "text": "When I was 20 I learned trying to one up everyone will just cause bridges to burn. \nBuild others while building yourself.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 541672514, "name": "James Curley", "screen_name": "JurleyCames", "lang": "en", "location": "mIcHiGaN ", "create_at": date("2012-03-31"), "description": "Grand Rapids Forever. +radical+", "followers_count": 570, "friends_count": 177, "statues_count": 2068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970174821793792, "text": "@PACIFYftHER it was v v v v v v v v v v good. she was incredible.", "in_reply_to_status": 717968844325675008, "in_reply_to_user": 1110583105, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1110583105 }}, "user": { "id": 1601827489, "name": "john ❩", "screen_name": "kissyoutwice", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-17"), "description": "here is home", "followers_count": 258, "friends_count": 212, "statues_count": 3735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970175493074945, "text": "This finna be the first year that I don't care about the Clippers v. Memphis series.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54781136, "name": "the real Tropicana", "screen_name": "Question_Mid", "lang": "en", "location": "IG: TropicanaKenny", "create_at": date("2009-07-07"), "description": "It's only fair that I warn ya, rap's my new hustle. I'm treatin' it like the corner. Fuck with more if you wanna.", "followers_count": 303, "friends_count": 207, "statues_count": 26715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-07T00:00:03.000Z"), "id": 717970176277262336, "text": "Tbh lol nvm ever grade has their WYD......moments", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3379415001, "name": "a-sizzle☺️", "screen_name": "AreeMurray", "lang": "en", "location": "somewhere doing my makeup", "create_at": date("2015-07-16"), "description": "I hate your stinking guts", "followers_count": 282, "friends_count": 298, "statues_count": 834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastmont, WA", "id": "002d396055a6adf8", "name": "Eastmont", "place_type": "city", "bounding_box": rectangle("-122.234054,47.860402 -122.14427,47.921824") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5319630, "cityName": "Eastmont" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177057562624, "text": "Raymond Michael Ruszkowski, Homestead, FL, 33030 - https://t.co/FReRkLPCI8\n\n#raymondmichaelruszkowski #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.46701,25.474514"), "retweet_count": 0, "lang": "pl", "is_retweet": false, "hashtags": {{ "raymondmichaelruszkowski", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 410, "friends_count": 411, "statues_count": 200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homestead, FL", "id": "9519f2018bb6585b", "name": "Homestead", "place_type": "city", "bounding_box": rectangle("-80.526237,25.440567 -80.377996,25.518331") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1232275, "cityName": "Homestead" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177418104833, "text": "12:00 I'm ending", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1928858317, "name": "yung chemist", "screen_name": "bIackyxchan_", "lang": "en", "location": "Elk Grove, CA", "create_at": date("2013-10-02"), "description": "not possible| #creekmob | illiterate&sad&broke|", "followers_count": 360, "friends_count": 306, "statues_count": 9889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177606832128, "text": "It lit, sike I'm lit ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2916731419, "name": "gabyyyy", "screen_name": "gabydavenia", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-12-02"), "description": "20 & Blessed", "followers_count": 576, "friends_count": 391, "statues_count": 1955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177644707840, "text": "help.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 588398482, "name": "em⚢", "screen_name": "emilyugh_", "lang": "en", "location": "probably reading, crying too", "create_at": date("2012-05-23"), "description": "heroin took my hero // 1.17.16 // dedicated to professional wrestling. // literature enthusiast // girls are rad // “Keep going. The journey never stops.”", "followers_count": 2202, "friends_count": 2454, "statues_count": 64212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stow, OH", "id": "5920ec228f7cd83f", "name": "Stow", "place_type": "city", "bounding_box": rectangle("-81.491087,41.135566 -81.392049,41.2175") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3974944, "cityName": "Stow" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177736847360, "text": "Fave! Hella funny and cool but we don't really talk that much anymore https://t.co/myzjfI4hHi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1223359442, "name": "ludaCRIS", "screen_name": "__cristinamaria", "lang": "en", "location": "Oakland, CA", "create_at": date("2013-02-26"), "description": "das a cutie", "followers_count": 514, "friends_count": 676, "statues_count": 5301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970177871118337, "text": "@LoneVaultWander @G27Status https://t.co/sk0axx5fx6", "in_reply_to_status": 717952894247374848, "in_reply_to_user": 1560120186, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1560120186, 322129286 }}, "user": { "id": 58513403, "name": "Ruthven78", "screen_name": "ruthven78", "lang": "en", "location": "Spokane, WA", "create_at": date("2009-07-20"), "description": "A Salty Hardcore Casual Gamer @789gaming", "followers_count": 88, "friends_count": 161, "statues_count": 3931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970178458263556, "text": "FACTS https://t.co/8ZMF6jxbKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319393651, "name": "PABLÖ", "screen_name": "ElijahThaGod", "lang": "en", "location": "San Jose, CA", "create_at": date("2011-06-17"), "description": "And I'm doing pretty hood in my pink polo", "followers_count": 1352, "friends_count": 980, "statues_count": 90294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970178835779585, "text": "@jazziee17 I feel special��", "in_reply_to_status": 717952862764924928, "in_reply_to_user": 795238075, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 795238075 }}, "user": { "id": 1693336710, "name": "Andrea", "screen_name": "Andrea_Arias91", "lang": "en", "location": "Los Angeles,CA", "create_at": date("2013-08-23"), "description": "null", "followers_count": 109, "friends_count": 130, "statues_count": 3510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179087400962, "text": "@AngelCorisha hahahah baddddie ����", "in_reply_to_status": 717970003778121728, "in_reply_to_user": 283249649, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user_mentions": {{ 283249649 }}, "user": { "id": 1488316861, "name": "c. Snow ❄️", "screen_name": "_SoUKnowCaitlyn", "lang": "en", "location": "Denton, TX", "create_at": date("2013-06-06"), "description": "My son Isaiah is my everything ❤️ / IG: same as twitter@ / SC: Caitlynann01 / University of North Texas '17 / Aspiring physical therapist", "followers_count": 930, "friends_count": 580, "statues_count": 23739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179137777664, "text": "Still wed https://t.co/N31ZvJyE6U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189423268, "name": "dlo", "screen_name": "Dlo159", "lang": "en", "location": "null", "create_at": date("2010-09-10"), "description": "imma sleeper", "followers_count": 201, "friends_count": 137, "statues_count": 1529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179259379712, "text": "Someone entertain me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1126932372, "name": "Aba", "screen_name": "DanielBenzikri", "lang": "en", "location": "Los Angeles ", "create_at": date("2013-01-27"), "description": "Fucking Sick Man Mixtape - https://soundcloud.com/dbenz-2/sets/lfm", "followers_count": 1311, "friends_count": 906, "statues_count": 34161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179603505152, "text": "Wind 0 mph SE. Barometer 30.07 in, Rising slowly. Temperature 70.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 59, "friends_count": 76, "statues_count": 26904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179620151296, "text": "Wow @jacksontr964 why you so obsessed with me �� https://t.co/aiokHqxMBd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3302415220 }}, "user": { "id": 463089759, "name": "Grace Ryan", "screen_name": "itsgraaaciie", "lang": "en", "location": "TX", "create_at": date("2012-01-13"), "description": "and those who were seen dancing were thought to be insane by those who could not hear the music☪", "followers_count": 531, "friends_count": 676, "statues_count": 3748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179855097856, "text": "@bringyofriends oh factss lml wasnt hearing it", "in_reply_to_status": 717966174215147521, "in_reply_to_user": 133030019, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 133030019 }}, "user": { "id": 237461916, "name": "Battle", "screen_name": "yvngwavybattle", "lang": "en", "location": "ATL", "create_at": date("2011-01-12"), "description": "19. #UWG19 . sc-yvngwavybattle", "followers_count": 1624, "friends_count": 936, "statues_count": 110433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollton, GA", "id": "7bc33682b217b5a1", "name": "Carrollton", "place_type": "city", "bounding_box": rectangle("-85.128063,33.52676 -85.009976,33.633403") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13045, "countyName": "Carroll", "cityID": 1313492, "cityName": "Carrollton" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970179892715520, "text": "Wind 6.0 mph WNW. Barometer 29.782 in, Rising slowly. Temperature 39.0 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970180337504256, "text": "3:00AM revelation: Life is weird. #gr8thnx #thishasbeenHo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gr8thnx", "thishasbeenHo" }}, "user": { "id": 1413887108, "name": "Sammie Ho", "screen_name": "sammiehohoho", "lang": "en", "location": "null", "create_at": date("2013-05-08"), "description": "Born in NYC, raised in Chicago, living in New Jersey. Go Cuse. | @NewhouseSU | @fypsyracuse | @spoonuniversity | @trailerparkinc |", "followers_count": 193, "friends_count": 589, "statues_count": 1279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970180706402304, "text": "Momma will forever be my biggest blessing, i love you so dearly ❣", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453811199, "name": "saucy chriss", "screen_name": "chrissdawgg_", "lang": "en", "location": "West Chicago, IL", "create_at": date("2012-01-03"), "description": "CSUSB ; SC: ayeeechriss ; fuck em", "followers_count": 851, "friends_count": 376, "statues_count": 34279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-07T00:00:04.000Z"), "id": 717970180769341440, "text": "I'm curious to find out https://t.co/Hjqz9Acn3Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2720273349, "name": "Chandler Cossio", "screen_name": "chango1348", "lang": "en", "location": "null", "create_at": date("2014-07-21"), "description": "Life is nothing but shits and giggles.", "followers_count": 152, "friends_count": 233, "statues_count": 1872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alpine, CA", "id": "ff5e82bba182df0b", "name": "Alpine", "place_type": "city", "bounding_box": rectangle("-116.816165,32.802788 -116.71178,32.852325") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 601192, "cityName": "Alpine" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970180932960256, "text": "Bout to be pretty lit weekend das fasho ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2166961572, "name": "LittleMermaid", "screen_name": "Lolitssammy_", "lang": "en", "location": "null", "create_at": date("2013-10-31"), "description": "Puerto Rican & Domincan Chipotle Gang RIP Tito ❤️", "followers_count": 423, "friends_count": 402, "statues_count": 9030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970181020999680, "text": "Your mcm drives a minivan that smells like shcungey pizza and has air soft bb's spilled everywhere", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2816529067, "name": "rake", "screen_name": "jake_reesman", "lang": "en", "location": "mung", "create_at": date("2014-09-17"), "description": "isthatthingfast", "followers_count": 632, "friends_count": 386, "statues_count": 5717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970181209792512, "text": "Same https://t.co/Kcl9DrF0sj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66022858, "name": "Barry", "screen_name": "kevman1017", "lang": "en", "location": "S.T.A.R. Labs", "create_at": date("2009-08-15"), "description": "We are all lost", "followers_count": 697, "friends_count": 552, "statues_count": 43392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970182187036673, "text": "@Lessslie_16 �� secrets out", "in_reply_to_status": 717963378333851648, "in_reply_to_user": 1061829866, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1061829866 }}, "user": { "id": 853476631, "name": "Kara Hugghins", "screen_name": "karahugghins", "lang": "en", "location": "null", "create_at": date("2012-09-29"), "description": "Texas A&M University '18", "followers_count": 610, "friends_count": 504, "statues_count": 3644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryan, TX", "id": "9ce0f1d2eb5998ce", "name": "Bryan", "place_type": "city", "bounding_box": rectangle("-96.439644,30.609214 -96.255918,30.722503") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4810912, "cityName": "Bryan" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970182753296384, "text": "@aghastlygalaxy @vegassist keep in mind that doctors aren't nutritionalists. They only take like one short course on nutrition.", "in_reply_to_status": 717963023206322176, "in_reply_to_user": 717834566606528512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 717834566606528512, 3847478477 }}, "user": { "id": 3379495929, "name": "Brittni Kristine", "screen_name": "BrittniKristine", "lang": "en", "location": "null", "create_at": date("2015-07-16"), "description": "23, vegan, LA based, dog mom ☀️❤️ Business related: brittni_kristine@yahoo.com", "followers_count": 688, "friends_count": 276, "statues_count": 2774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970183667625984, "text": "IM FUCKING 21 AMERICA!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16523618, "name": "Hannah Montana", "screen_name": "Patrickrazzi", "lang": "en", "location": "Orange Coünty, California", "create_at": date("2008-09-29"), "description": "Monster, and lover of life. Smiles are my favorite currency. Catholic", "followers_count": 4399, "friends_count": 1426, "statues_count": 115085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-07T00:00:05.000Z"), "id": 717970184804368384, "text": "Temp 45.3°F Wind Chill 45.3°F RH 84% Wind 2.6 WNW Gust 11.0 WNW SLP 29.719 in Falling Rain 0.14 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 106, "friends_count": 63, "statues_count": 34524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-07T00:00:06.000Z"), "id": 717970185047511041, "text": "i was asked what i did today and i sent an essay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 864053239, "name": "andrea", "screen_name": "OROZKOOL", "lang": "en", "location": "null", "create_at": date("2012-10-05"), "description": "jdb | ovo", "followers_count": 603, "friends_count": 100, "statues_count": 56720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-04-07T00:00:06.000Z"), "id": 717970185055920132, "text": "@JuannTonnn sowwy I just love attention. https://t.co/f1cpj4CxaX", "in_reply_to_status": 717965106630369280, "in_reply_to_user": 614485782, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 614485782 }}, "user": { "id": 614485782, "name": "RockLee", "screen_name": "JuannTonnn", "lang": "en", "location": "Houston, TX", "create_at": date("2012-06-21"), "description": "incognito", "followers_count": 875, "friends_count": 289, "statues_count": 39769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-04-07T00:00:06.000Z"), "id": 717970185483726848, "text": "@jaaymaama bye", "in_reply_to_status": 717969643793547268, "in_reply_to_user": 2936986323, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2936986323 }}, "user": { "id": 3499498152, "name": "pugaevan", "screen_name": "pugaevan_", "lang": "en", "location": "null", "create_at": date("2015-09-08"), "description": "null", "followers_count": 350, "friends_count": 257, "statues_count": 329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332548129157120, "text": "�� https://t.co/xhlY8MUdlI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 265077412, "name": "D.Leg™", "screen_name": "YungJefeLeg", "lang": "en", "location": "Jaylon Demond Steele.", "create_at": date("2011-03-12"), "description": "Baby'Jesus is what they call me. Prince of Dallas! #THEYSLEEP 214'The OakCliff Holy One. NLMB #GB #YoungOGoat #OSU ✈️ #TAMUC", "followers_count": 2171, "friends_count": 1121, "statues_count": 84345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332549647437824, "text": "Wind 1.0 mph WNW. Barometer 1023.44 mb, Rising. Temperature 45.6 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 13703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332549911724032, "text": "Keep it �� and be persistent ������ #GoodNight #LateNightHumor #Tequila #ShotsShotsShots #Taylor✌��️… https://t.co/23DrCVTOen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.419957,30.567987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoodNight", "LateNightHumor", "Tequila", "ShotsShotsShots", "Taylor" }}, "user": { "id": 243425853, "name": "Taylor Murphy", "screen_name": "T_Bird36", "lang": "en", "location": "Atlanta, Georgia", "create_at": date("2011-01-26"), "description": "Radio Personality for Instinct Radio | Blogger of http://Guesswhatiheardblog.com Follow for the tea! Email Taylorgangandfriends@gmail.com for sponsorships✌️", "followers_count": 237, "friends_count": 526, "statues_count": 6301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor, TX", "id": "7793efb73edea52a", "name": "Taylor", "place_type": "city", "bounding_box": rectangle("-97.446645,30.536252 -97.382435,30.614587") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4871948, "cityName": "Taylor" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332549928521730, "text": "@emmalicious713 #SilverCensus has developed a complete and comprehensive nursing home\n\nhttps://t.co/af8N6fghcK https://t.co/0tCMSSlWV3", "in_reply_to_status": 715604056337620993, "in_reply_to_user": 3357962350, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SilverCensus" }}, "user_mentions": {{ 3357962350 }}, "user": { "id": 51185319, "name": "SilverCensus", "screen_name": "SilverCensus", "lang": "en", "location": "Palm Beach Gardens", "create_at": date("2009-06-26"), "description": "http://SilverCensus.com - The Premier Directory for Senior Living & Health Services offering free placement assistance and job recruitment", "followers_count": 925, "friends_count": 560, "statues_count": 16250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Palm Beach, FL", "id": "4de072969805ac41", "name": "West Palm Beach", "place_type": "city", "bounding_box": rectangle("-80.2006,26.644671 -80.044639,26.799121") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1276600, "cityName": "West Palm Beach" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332550117203968, "text": "Late night craving�� https://t.co/0Qk9Q4Lva3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41006016, "name": "Czarina❤️‍", "screen_name": "Czariegirl", "lang": "en", "location": "null", "create_at": date("2009-05-18"), "description": "And I will love with urgency, but not with haste.", "followers_count": 298, "friends_count": 104, "statues_count": 2176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332550771515392, "text": "@eai463 @DannyGradio Nice & Smooth....the guitar is Tracy somebody.", "in_reply_to_status": 718325553623904256, "in_reply_to_user": 493371078, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 493371078, 736301784 }}, "user": { "id": 270958352, "name": "denverrob", "screen_name": "ROBDOG0721", "lang": "en", "location": "Denver Colorado", "create_at": date("2011-03-23"), "description": "if you don't like it...change it...if you can't change it...find a way to deal with it....letting go is much healthier than hanging on to anger", "followers_count": 82, "friends_count": 301, "statues_count": 13581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332550985420800, "text": "Let Vic Give You Head @o_liv33 #DoItForState", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DoItForState" }}, "user_mentions": {{ 892440654 }}, "user": { "id": 126157793, "name": "Ms. Jane✈™", "screen_name": "Jet_Setter_Life", "lang": "en", "location": "New Orleans,LA", "create_at": date("2010-03-24"), "description": "La Nouvelle-Orléans ♔ Young Savage Girl Lost Among The Lily Pads 8-29-2005 #XULA19", "followers_count": 4458, "friends_count": 4624, "statues_count": 8706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332551446806528, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 398400889, "name": "Alex Hefner ❤", "screen_name": "QueenAlxandraa", "lang": "en", "location": "Shreveport, LA", "create_at": date("2011-10-25"), "description": "yes , I'm the Alex with Cancer ❤️", "followers_count": 1685, "friends_count": 1692, "statues_count": 88595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332551618826242, "text": "@Heather_Poole https://t.co/xbJFvbpH0C", "in_reply_to_status": -1, "in_reply_to_user": 16688389, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 16688389 }}, "user": { "id": 17383566, "name": "Penny M", "screen_name": "CalliopeCo", "lang": "en", "location": "Santa Cruz County CA", "create_at": date("2008-11-13"), "description": "Boston-born, Silicon Valley SW QA engineer/writer. Digital artisan w/views on gender & class. Cinephile. Speaks plusieurs Sprachen. Collects miles w/ @rjmatteis", "followers_count": 5617, "friends_count": 6181, "statues_count": 223614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aptos, CA", "id": "09d7cfd44170d537", "name": "Aptos", "place_type": "city", "bounding_box": rectangle("-121.933333,36.967212 -121.863133,37.016726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 602378, "cityName": "Aptos" } }
+{ "create_at": datetime("2016-04-08T00:00:00.000Z"), "id": 718332551883067393, "text": "Temp: 34.5F W C: 29.3F Wind:NE at 5.1kts Baro: 1012.6mb and Rising slowly Rain today: 0.00in R H: 94% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 207, "friends_count": 219, "statues_count": 101851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332551891443712, "text": "Today is the day!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1836873517, "name": "champagne mami", "screen_name": "gurqueen", "lang": "en", "location": "Trang's House", "create_at": date("2013-09-08"), "description": "RHS | D8 Key Club | @twotrangs", "followers_count": 850, "friends_count": 626, "statues_count": 11767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332551924957184, "text": "@rockolla @MinhP765 @niseishonagon that's lewd", "in_reply_to_status": 718330629667045377, "in_reply_to_user": 16217639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16217639, 14606436, 575206417 }}, "user": { "id": 230044983, "name": "Evermatsu@househunt", "screen_name": "everfeather", "lang": "en", "location": "North Metro Atlanta", "create_at": date("2010-12-23"), "description": "Vodka clad Valkyrie wielding Google Drive necromancy on a warpath to get my monster girls back from Obama", "followers_count": 399, "friends_count": 268, "statues_count": 82895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dania Beach, FL", "id": "2b8922cbe7f16337", "name": "Dania Beach", "place_type": "city", "bounding_box": rectangle("-80.208086,26.033366 -80.127345,26.091444") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1216335, "cityName": "Dania Beach" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332552541564930, "text": "13 hour haul to Kansas City complete. Walmart parking lots are going to be our best friend.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362107706, "name": "David Kleinebreil", "screen_name": "NLTGdavid", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-08-25"), "description": "I sing/play in @NLTGaz. I love puns and Brand New. Always quoting SpongeBob & The Office. #TheWalkingDead #RiseUpRedSea", "followers_count": 27544, "friends_count": 6425, "statues_count": 10469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332552688377862, "text": "Happiness is all the avocados being ripe at the grocery store", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 818268566, "name": "MVRiC", "screen_name": "omgitsdylanalba", "lang": "en", "location": "Irvine, CA", "create_at": date("2012-09-11"), "description": "the toughest SOB in the WWF. #austin3:16", "followers_count": 255, "friends_count": 305, "statues_count": 3662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553074241537, "text": "Ripley SW Limestone Co. Temp: 46.8°F Wind:0.0mph Pressure: 993.8mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553116168193, "text": "This headache ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2174877944, "name": "Super Savage Snoop⚜", "screen_name": "Snoopaman_", "lang": "en", "location": "Gym regaurdless.", "create_at": date("2013-11-04"), "description": "Everything i do is for my mom❤ ™ #LionHearted ® #RipNittyG #RipLilGerld #RipDmos #RipDad Ⓜob babbyyy ♠ on a mission to get everything they said I couldn't have.", "followers_count": 651, "friends_count": 490, "statues_count": 7149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, IL", "id": "013b9d696e5665d0", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-89.416938,40.08307 -89.330625,40.168573") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17107, "countyName": "Logan", "cityID": 1743536, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553330077696, "text": "Wow my charger has gone missing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356447347, "name": "alexandra valenzuela", "screen_name": "itsmealexandrav", "lang": "en", "location": "Coachella, CA", "create_at": date("2011-08-16"), "description": "Pop that pussy like it's worth something @marielaayala5 ❤️", "followers_count": 958, "friends_count": 291, "statues_count": 63303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553342631938, "text": "Uh oh. Take time to make the plan. Then #worktheplan https://t.co/TFat50YJUx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "worktheplan" }}, "user": { "id": 68910618, "name": "Paul Cattin", "screen_name": "paulcattin", "lang": "en", "location": "Denver, CO", "create_at": date("2009-08-25"), "description": "I love marketing & entrepreneurial strategy for innovation and consumer experiences. Life balance is through tea, coffee and soccer -Find passion in all you do!", "followers_count": 116, "friends_count": 171, "statues_count": 1662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553707528193, "text": "@ebbtideapp Tide in Galveston Bay Entrance, Texas 04/08/2016\nHigh 5:37am 2.0\n Low 11:36am 0.9\nHigh 4:39pm 1.5\n Low 11:01pm -0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-94.725,29.3583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 50, "friends_count": 1, "statues_count": 22628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332553770438656, "text": "�� https://t.co/X0LZZCfYLM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1173753901, "name": "RJ", "screen_name": "WavyRJ", "lang": "en", "location": "IE,", "create_at": date("2013-02-12"), "description": "#Lux #FEAR", "followers_count": 1651, "friends_count": 1465, "statues_count": 46827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332554110201856, "text": "Temp 45.6° Hi/Lo 48.3/42.5 Rng 5.8° WC 45.6° Hmd 97% Rain 0.00\" Storm 1.36\" BAR 29.429 Falling DP 44.8° Wnd 3mph Dir WNW Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 67, "friends_count": 117, "statues_count": 17665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332554143772673, "text": "@Arum_MHutami Hai kak kita jual domo, stitch, doraemon dan kartun lainnya loh. Cek twitter dan ig kita yuk kak. Banyak stuff menarik", "in_reply_to_status": -1, "in_reply_to_user": 1036518007, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 1036518007 }}, "user": { "id": 1597576466, "name": "IG : @Blueonshoppp", "screen_name": "blueonshoppp", "lang": "id", "location": "Jakarta Capital Region", "create_at": date("2013-07-15"), "description": "NEW PIN - 5BCE0B27 \nLINE? @FXQ1836X (pakai @ depannya) ADD FB : Blueonshop", "followers_count": 4508, "friends_count": 1177, "statues_count": 87994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332554378616832, "text": "@annabellalynnn https://t.co/pz8F5mVlj1", "in_reply_to_status": -1, "in_reply_to_user": 2874667119, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2874667119 }}, "user": { "id": 324953359, "name": "I.sa.k", "screen_name": "deadice69", "lang": "en", "location": "hell has no location", "create_at": date("2011-06-27"), "description": "48/16 on a Aventón Mataro I'm a danger to society because my cycling skills make me a better person", "followers_count": 243, "friends_count": 399, "statues_count": 11277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332554697433088, "text": "Wind 0.0 mph ---. Barometer 29.324 in, Steady. Temperature 33.3 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555238486016, "text": "ANSWER MY CALL �� https://t.co/vUSOir11aS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3177370770, "name": "Tray ⚠️", "screen_name": "YouSleepOnTray", "lang": "en", "location": "UWG/Atlanta", "create_at": date("2015-04-26"), "description": "SC: tray_grover #LongLiveStreetMoneyBankroll Ugly Nigga W/ A Good Heart", "followers_count": 1119, "friends_count": 772, "statues_count": 25872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollton, GA", "id": "7bc33682b217b5a1", "name": "Carrollton", "place_type": "city", "bounding_box": rectangle("-85.128063,33.52676 -85.009976,33.633403") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13045, "countyName": "Carroll", "cityID": 1313492, "cityName": "Carrollton" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555498561536, "text": "Why is this so accurate. im an Aquarius https://t.co/u5gFZ1B4eO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331053504, "name": "L.y.v.e.e♒", "screen_name": "ItsLyvee", "lang": "en", "location": "That Place Where No One Looks", "create_at": date("2011-07-07"), "description": "Let Your Vision Empower Everything #Dodielipbrand #LyveeBrand\nArtist\nDJ\nMotivational Speaker\nPhotographer\nLyveevm@gmail.com", "followers_count": 1080, "friends_count": 1008, "statues_count": 36210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555523657728, "text": "Well Happy Birthday to me!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2300898078, "name": "Emma", "screen_name": "emma_jacque", "lang": "en", "location": "Oro Valley, AZ", "create_at": date("2014-01-19"), "description": "19 living life to the fullest with no regrets", "followers_count": 556, "friends_count": 436, "statues_count": 6639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555532050433, "text": "04/08@03:00 - Temp 44.7F, WC 42.3F. Wind 4.6mph WSW, Gust 10.0mph. Bar 29.603in, Rising slowly. Rain 0.01in. Hum 75%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555741802496, "text": "Fo life https://t.co/uD1Cx3G35n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1857078464, "name": "Brandon Kaleikini", "screen_name": "Kal_Brando", "lang": "en", "location": "Cali, Bay Area", "create_at": date("2013-09-12"), "description": "3rd St Boxing IG: @Brando_Kaleikini", "followers_count": 274, "friends_count": 409, "statues_count": 9706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555758583809, "text": "Reminds me of @Jeremiah2Dope ���� https://t.co/1mrExkUZL7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174400481 }}, "user": { "id": 3019410999, "name": "jenn", "screen_name": "umbyeefeliciaa", "lang": "en", "location": "null", "create_at": date("2015-02-04"), "description": "careless", "followers_count": 285, "friends_count": 243, "statues_count": 11725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-08T00:00:01.000Z"), "id": 718332555976642560, "text": "Nothing coming close to changing #SFPD #howmuchmorecantheygetawaywith #justiceforthemurdered #putthemurderercopsbehindbars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SFPD", "howmuchmorecantheygetawaywith", "justiceforthemurdered", "putthemurderercopsbehindbars" }}, "user": { "id": 289665268, "name": "Matt", "screen_name": "MatthewWinslow", "lang": "en", "location": "Middletown, CT", "create_at": date("2011-04-28"), "description": "Wesleyan Class of 2015, Mendo, UEA Pirate, Browns, 49ers, Decoptus Floreo", "followers_count": 209, "friends_count": 636, "statues_count": 850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332556547067904, "text": "@northwest_gay shel just wants to play lol", "in_reply_to_status": 718331739848982528, "in_reply_to_user": 2350933628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2350933628 }}, "user": { "id": 4010504413, "name": "Ian", "screen_name": "ICC_00", "lang": "en", "location": "Seattle, WA", "create_at": date("2015-10-24"), "description": "Seahawks, Mariners, Sounders, Huskies, Warriors, and Sharks", "followers_count": 893, "friends_count": 840, "statues_count": 14374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332556832350209, "text": "Wind 0.7 mph SW. Barometer 29.956 in, Rising. Temperature 55.9 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332556916211713, "text": "59.6F (Feels: 59.6F) - Humidity: 84% - Wind: 4.5mph W - Gust: 4.5mph - Pressure: 1011.6mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 234221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332557574696961, "text": "@Feelin_onika me too lol", "in_reply_to_status": 718332437856653312, "in_reply_to_user": 3053525070, "favorite_count": 0, "coordinate": point("-96.67923432,30.97536313"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3053525070 }}, "user": { "id": 564924670, "name": "#MoreIssuesThanVogue", "screen_name": "Zacch_Minaj", "lang": "en", "location": "W | Y(our) Bae", "create_at": date("2012-04-27"), "description": "IG ; @Zaccchaaaeuuus | SC ; @iAmTootieB | TX", "followers_count": 2105, "friends_count": 996, "statues_count": 71361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48395, "countyName": "Robertson", "cityID": 4811992, "cityName": "Calvert" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332557578932224, "text": "Temp: 63.6°F Wind:4.2mph Pressure: 29.940hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 60604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332557813751808, "text": "I forget this too often. https://t.co/rVLPGuE0zA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15886234, "name": "Stephanie Munoz", "screen_name": "StephanieMMunoz", "lang": "en", "location": "Arizona", "create_at": date("2008-08-17"), "description": "Digital marketing and social media nerd with a K-Pop and K-Drama habit who's an inadvertent hipster.", "followers_count": 195, "friends_count": 296, "statues_count": 1713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332558069612544, "text": "Wind 0.0 mph ---. Barometer 29.910 in, Rising slowly. Temperature 29.8 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 57, "friends_count": 27, "statues_count": 18927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332558442958848, "text": "Wind 4.0 mph NW. Barometer 29.965 in, Steady. Temperature 45.1 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332558539407360, "text": "fr just wanna do some reckless shit but.. priorities.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732028112, "name": "young reezy", "screen_name": "_mariahhx", "lang": "en", "location": "null", "create_at": date("2012-08-01"), "description": "viewer discretion is advised.", "followers_count": 709, "friends_count": 844, "statues_count": 9005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MI", "id": "c309c6a6f85110cd", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-82.973298,42.539309 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559055306752, "text": "@staiboy_west I really do lmao", "in_reply_to_status": 718332490763644928, "in_reply_to_user": 459706337, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 459706337 }}, "user": { "id": 349479002, "name": "semhar", "screen_name": "semharfitwi", "lang": "en", "location": "ARIZONA ☀️", "create_at": date("2011-08-05"), "description": "Eritrean . 18", "followers_count": 647, "friends_count": 415, "statues_count": 43979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559072096256, "text": "เหนื่อยจะตายแล้ว", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user": { "id": 1725782592, "name": "che", "screen_name": "cherriejubu", "lang": "th", "location": "null", "create_at": date("2013-09-03"), "description": "บิ๊กบอสคือมึงน่าเย็ดไปอะ", "followers_count": 87, "friends_count": 198, "statues_count": 4760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559109799936, "text": "Rehearsal was dope... This movie going to be amazing. The cast is great, I'm having a ball as… https://t.co/UKUIdV3qdx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.4304199,38.6070747"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 66721208, "name": "Anderi Bailey", "screen_name": "andrebfunny", "lang": "en", "location": "Sac, Ca", "create_at": date("2009-08-18"), "description": "I Love to see people smile and my brand of Comedy happens to be clean.", "followers_count": 1174, "friends_count": 2014, "statues_count": 3589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559139172352, "text": "@OfficialGizelle I Miss You ������", "in_reply_to_status": 718330901214732288, "in_reply_to_user": 289623256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 289623256 }}, "user": { "id": 4178568022, "name": "#TeamLoso", "screen_name": "ace_27st", "lang": "en", "location": "null", "create_at": date("2015-11-13"), "description": "#FFF", "followers_count": 86, "friends_count": 179, "statues_count": 574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559457984512, "text": "HAPPY BIRTHDAY I LOVE YOU TO THE FRICKEN MOON AND BACK BRO ��@lyndseaaluddine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1244966467 }}, "user": { "id": 881181948, "name": "beth", "screen_name": "bethmartinnn", "lang": "en", "location": "hb ♡ ", "create_at": date("2012-10-14"), "description": "only those who risk going too far can possibly find out how far they can go", "followers_count": 513, "friends_count": 448, "statues_count": 7381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559558578177, "text": "@Dominican_33 not even close girl", "in_reply_to_status": 718332438276124672, "in_reply_to_user": 3531264978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3531264978 }}, "user": { "id": 614485782, "name": "RockLee", "screen_name": "JuannTonnn", "lang": "en", "location": "Houston, TX", "create_at": date("2012-06-21"), "description": "incognito", "followers_count": 878, "friends_count": 292, "statues_count": 39817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-04-08T00:00:02.000Z"), "id": 718332559730614273, "text": "#Trump in 2016 Temp:36.1°F Wind:0.0mph Pressure: 29.85hpa Rising slowly Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 314596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332560443600897, "text": "He preaching tho https://t.co/AOp2QQthT7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320560188, "name": "Tay West", "screen_name": "iHateTayWest", "lang": "en", "location": "Sacramento, CA", "create_at": date("2011-06-19"), "description": "just a Nigga With Attitude", "followers_count": 11998, "friends_count": 6562, "statues_count": 71181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332560988893184, "text": "�� even when you tried and make up for being an asshole still isn't good enough for you is it!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339916404, "name": "Cece's World", "screen_name": "Chillsurfa", "lang": "en", "location": "Space ", "create_at": date("2014-02-11"), "description": "catch waves not feeling. ______Just your local average Fu(k€d up individual that ruins anything good going in life", "followers_count": 225, "friends_count": 788, "statues_count": 4187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del Aire, CA", "id": "af1bd9a7cea6c77e", "name": "Del Aire", "place_type": "city", "bounding_box": rectangle("-118.378565,33.902435 -118.361276,33.929174") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 618352, "cityName": "Del Aire" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332561265717248, "text": "@Thesupercole your love for Jesus inspires me so much whenever i see it on my feed. thank you for being an awesome voice for Christ✝��", "in_reply_to_status": -1, "in_reply_to_user": 280711094, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 280711094 }}, "user": { "id": 631092950, "name": "becca lahr", "screen_name": "rebeccamaelahr", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "God's not dead He's surely alive✨", "followers_count": 369, "friends_count": 579, "statues_count": 4644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332561471225857, "text": "@1FaceByGrace still packing", "in_reply_to_status": 718331688238112769, "in_reply_to_user": 30949323, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30949323 }}, "user": { "id": 51471374, "name": "Kandi Conda", "screen_name": "KCbizBOSS", "lang": "en", "location": "CHICAGO & THE WORLD", "create_at": date("2009-06-27"), "description": "HALL OF FAMER 4Womens Pro BB. Brand Consultant | Author | Speaker | - MGR. for DEMI LOBO Celeb Host/Media Personality /Booking: HALLofFAMERKANDI@gmail.com", "followers_count": 7594, "friends_count": 1932, "statues_count": 26457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332561592872961, "text": "@NylaVSTheWorld won't happen again ����", "in_reply_to_status": 718332415509467136, "in_reply_to_user": 1493448512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1493448512 }}, "user": { "id": 144038538, "name": "MILLYIONS", "screen_name": "TheJamilBell", "lang": "en", "location": "BROOKLYN ", "create_at": date("2010-05-14"), "description": "TUHD Clothing© 730bg", "followers_count": 2403, "friends_count": 1867, "statues_count": 71069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332562003910656, "text": "Fucking night shift��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123178359, "name": "Elijah Funtiveros", "screen_name": "lamebuttts", "lang": "en", "location": "New Jersey, USA", "create_at": date("2010-03-15"), "description": "Hi im lame.", "followers_count": 127, "friends_count": 170, "statues_count": 1878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332562335264769, "text": "Wind 2.2 mph W. Barometer 29.42 in, Steady. Temperature 54.3 °F. Rain today 0.19 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 34, "friends_count": 118, "statues_count": 158900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332562658209792, "text": "FACT: if he brownskin/darkskin just know he about to waste your time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 899942156, "name": "mamacita.", "screen_name": "olyyrose", "lang": "en", "location": "California, USA", "create_at": date("2012-10-23"), "description": "null", "followers_count": 2108, "friends_count": 712, "statues_count": 15663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amarillo, TX", "id": "52a0517c2183cf22", "name": "Amarillo", "place_type": "city", "bounding_box": rectangle("-101.96758,35.098693 -101.664604,35.323538") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48375, "countyName": "Potter", "cityID": 4803000, "cityName": "Amarillo" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332562825953280, "text": "@PettyLikeLondyn I'll ring the local authorities. JUST KIDDING lol", "in_reply_to_status": 718332299218173954, "in_reply_to_user": 260045410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260045410 }}, "user": { "id": 207203414, "name": "black dennis", "screen_name": "OGBEARD", "lang": "en", "location": "null", "create_at": date("2010-10-24"), "description": "A coworker told me bitches on twitter love beards so im here...", "followers_count": 24540, "friends_count": 5635, "statues_count": 416085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332563199238144, "text": "Rubber neck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2845690694, "name": "yaaahhh", "screen_name": "Humaje14", "lang": "en", "location": "Pacific Ocean", "create_at": date("2014-10-07"), "description": "just a young savage in the 916", "followers_count": 245, "friends_count": 199, "statues_count": 9118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Foothill Farms, CA", "id": "3dc5823629ba0580", "name": "Foothill Farms", "place_type": "city", "bounding_box": rectangle("-121.373031,38.66773 -121.326256,38.705859") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624722, "cityName": "Foothill Farms" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332563333509120, "text": "He wanted to be an artist.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1564957980, "name": "APΘīΐîîîΐههة", "screen_name": "Illumeye", "lang": "en", "location": "Fairview, TX", "create_at": date("2013-07-02"), "description": "Vive La #fR∀nÇe #TuRkeʎ #ÇhEʎen∀ #EʎeR∀q #EʎeR∀n #Russì∀ #BrEʎeŽe∀l #N.S.KṓrEʎe∀ #Jap∀in Handle #Tex∀s", "followers_count": 21, "friends_count": 13, "statues_count": 2549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, TX", "id": "506bc3b13b02edd6", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-96.659709,33.122254 -96.572672,33.165515") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4825224, "cityName": "Fairview" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332563341910017, "text": "flexin on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2668747549, "name": "9", "screen_name": "breckenn", "lang": "en", "location": "null", "create_at": date("2014-07-22"), "description": "null", "followers_count": 717, "friends_count": 272, "statues_count": 6771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-04-08T00:00:03.000Z"), "id": 718332563975249920, "text": "@PricePolley I haven't had cold stone in literally yearsssss but you're so right", "in_reply_to_status": 718332430734790656, "in_reply_to_user": 811233979, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 811233979 }}, "user": { "id": 982905144, "name": "Theressa Marie", "screen_name": "TshirtB", "lang": "en", "location": "Arizona State University", "create_at": date("2012-12-01"), "description": "|18|lds|better believe I keep my demons to myself", "followers_count": 494, "friends_count": 305, "statues_count": 6516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332564671447040, "text": "Why do I tweet so much when I'm hyper", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 724796066, "name": "Meghan Giroux", "screen_name": "meglizgiroux", "lang": "en", "location": "Mercy", "create_at": date("2012-07-29"), "description": "Goals isn't as goalz as goalz -anon | @clairebeau16 @oliviaramirez27", "followers_count": 472, "friends_count": 445, "statues_count": 2152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Novi, MI", "id": "9458d33c93e47ce4", "name": "Novi", "place_type": "city", "bounding_box": rectangle("-83.555202,42.436044 -83.432974,42.527116") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2659440, "cityName": "Novi" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332564717584384, "text": "I called him to tell him about this guy I've been seeing and how hes acting shady.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 463055889, "name": "Sarah Graves", "screen_name": "SarahhGravess", "lang": "en", "location": "null", "create_at": date("2012-01-13"), "description": "null", "followers_count": 150, "friends_count": 149, "statues_count": 1689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevenson Ranch, CA", "id": "00ab310716dc2504", "name": "Stevenson Ranch", "place_type": "city", "bounding_box": rectangle("-118.622276,34.366596 -118.560263,34.418064") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 674130, "cityName": "Stevenson Ranch" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332564902121472, "text": "���������� https://t.co/FktnvmvdJZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 592365161, "name": "B.Hill", "screen_name": "Smoovebe_", "lang": "en", "location": "null", "create_at": date("2012-05-27"), "description": "Byrnes high'16", "followers_count": 622, "friends_count": 461, "statues_count": 5728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565061529600, "text": "Zach too turnt for Twitter pt. 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3254340357, "name": "Lewis", "screen_name": "wcu_blackice", "lang": "en", "location": "null", "create_at": date("2015-05-14"), "description": "As human beings, it is inevitable that we make mistakes. Those that learn from these mistakes separate themselves from the rest.\n#WCU16", "followers_count": 176, "friends_count": 178, "statues_count": 3432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cullowhee, NC", "id": "4e7a5a46faf1356c", "name": "Cullowhee", "place_type": "city", "bounding_box": rectangle("-83.200975,35.29043 -83.159039,35.332791") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37099, "countyName": "Jackson", "cityID": 3715880, "cityName": "Cullowhee" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565460008961, "text": "رحم الله ضحكات لا تُنسى وملامح لا تغيب عن البال وحديثاً اشتقنا لسماعه رحم الله كل روح غاليه تحت الثرى #هشام_الخشتي", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "hashtags": {{ "هشام_الخشتي" }}, "user": { "id": 242642430, "name": "فهد الخشتي", "screen_name": "FH_alkhashti", "lang": "en", "location": "Tempe,Az | Kuwait☀️", "create_at": date("2011-01-24"), "description": "اللهم ارحم ابي هشام الخشتي و اسكنه فسيح جناتك ASU - O|||||||O", "followers_count": 411, "friends_count": 397, "statues_count": 15365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565485142020, "text": "I haven't heard Finally Famous Vol. 3 in so long. Its such a lit tape. Wow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1397940204, "name": "Master Bugs ✌", "screen_name": "BugsVeggies", "lang": "en", "location": "Gotham City ", "create_at": date("2013-05-02"), "description": "sc: bugsveggies\nStay Gold.", "followers_count": 559, "friends_count": 649, "statues_count": 67845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565665488896, "text": "Try https://t.co/Tq66XDxcq4 While Keith accomplished many remarkable achievements, none are more valuable to the Beverage #ice #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.846485,32.403773"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ice", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4379, "friends_count": 876, "statues_count": 413058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565845880835, "text": "Wind 4.0 mph WSW. Barometer 29.341 in, Steady. Temperature 47.3 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 5602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332565996834818, "text": "Ummmm @RobertNBCLA no bolt of lightning or glasses...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 78368586 }}, "user": { "id": 1403611226, "name": "Rusty Three", "screen_name": "RustyThree", "lang": "en", "location": "null", "create_at": date("2013-05-04"), "description": "null", "followers_count": 152, "friends_count": 302, "statues_count": 6710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332566063968258, "text": "@IVIIXCIII_ k, I willl", "in_reply_to_status": 718332389265711104, "in_reply_to_user": 310521334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310521334 }}, "user": { "id": 4881713064, "name": "Karlie Kahn", "screen_name": "_ksophia", "lang": "en", "location": "Chico, CA", "create_at": date("2016-02-08"), "description": "ig: k.sophia_ - 22 • Biology • Runner", "followers_count": 94, "friends_count": 101, "statues_count": 1102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332566240108544, "text": "Catch some waves at Australia's first surf park https://t.co/pNPuWGWxAH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17621, "friends_count": 17556, "statues_count": 70351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332566714118144, "text": "Hey nvm fuck you all I get paid tonight Hahahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1368292861, "name": "daddio pablito", "screen_name": "Kingpablito5112", "lang": "en", "location": "90210", "create_at": date("2013-04-20"), "description": "null", "followers_count": 316, "friends_count": 290, "statues_count": 21772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332566806380544, "text": "@LindseyFranco97 my fish both got eaten alive so I've been pretty lonely ever since", "in_reply_to_status": 718332337616982016, "in_reply_to_user": 774955524, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 774955524 }}, "user": { "id": 826571898, "name": "Kayla", "screen_name": "championkayla", "lang": "en", "location": "inside ur mind", "create_at": date("2012-09-15"), "description": "yes, my last name really is champion", "followers_count": 517, "friends_count": 322, "statues_count": 5584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rohnert Park, CA", "id": "5cda0a6ac9cf8725", "name": "Rohnert Park", "place_type": "city", "bounding_box": rectangle("-122.732141,38.314281 -122.666695,38.372609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 662546, "cityName": "Rohnert Park" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332566928011264, "text": "\"Oooh she be gettin' them zeta zeta zetas!\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226003935, "name": "Stephanie Slocum", "screen_name": "stephsloke", "lang": "en", "location": "Los Angeles", "create_at": date("2010-12-12"), "description": "funny, fresh meat from that town with the Rocky Horror production issues and apparently now where Soulja Boy got arrested. Way to go Carrollton, Georgia!", "followers_count": 98, "friends_count": 159, "statues_count": 3832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Culver City, CA", "id": "a75bc1fb166cd594", "name": "Culver City", "place_type": "city", "bounding_box": rectangle("-118.448451,33.976882 -118.369529,34.035143") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 617568, "cityName": "Culver City" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332567271915520, "text": "Long deep conversations.. >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 247407557, "name": "•AmberJoi•", "screen_name": "signedAMBER_", "lang": "en", "location": "null", "create_at": date("2011-02-04"), "description": "QueenⒶ snapchat : amberjoi1004", "followers_count": 3325, "friends_count": 2917, "statues_count": 46523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maywood, IL", "id": "b98acb4e130f102d", "name": "Maywood", "place_type": "city", "bounding_box": rectangle("-87.863231,41.86419 -87.824599,41.898867") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1747774, "cityName": "Maywood" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332567397756930, "text": "03:00:01 |Temp: 54.2ºF | Wind Chill 54.2ºF |Dew Point 49.1ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the N, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332567565504512, "text": "@RLRKilger @Kssek DITTO", "in_reply_to_status": 718325604286885888, "in_reply_to_user": 522881502, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 522881502, 2175526026 }}, "user": { "id": 263331062, "name": "Chris Williams", "screen_name": "c__williams", "lang": "en", "location": "ryan's house", "create_at": date("2011-03-09"), "description": "According to Matt I've never been right", "followers_count": 136, "friends_count": 390, "statues_count": 17358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332567687143424, "text": "Shit at least I understand her https://t.co/QQY043COFX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 121920558, "name": "Paz", "screen_name": "whybang", "lang": "en", "location": "null", "create_at": date("2010-03-10"), "description": "level 82 warlock", "followers_count": 509, "friends_count": 335, "statues_count": 40939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blue Island, IL", "id": "fb0971608411ae20", "name": "Blue Island", "place_type": "city", "bounding_box": rectangle("-87.709565,41.630338 -87.651678,41.677417") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1706704, "cityName": "Blue Island" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332568156942336, "text": "Today is seriously not my day lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240120102, "name": "Anissa Pena", "screen_name": "anissapenaa", "lang": "en", "location": "null", "create_at": date("2011-01-18"), "description": "The One and Only | Instagram & SC: anissaap", "followers_count": 511, "friends_count": 320, "statues_count": 17738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332568614084608, "text": "☺️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 316032204, "name": "Shelby Wooten", "screen_name": "shelbyjunee", "lang": "en", "location": "null", "create_at": date("2011-06-12"), "description": "410", "followers_count": 1449, "friends_count": 654, "statues_count": 30950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-08T00:00:04.000Z"), "id": 718332568656027648, "text": "@destmodesto happy birthday girly hope you have a good day!������", "in_reply_to_status": -1, "in_reply_to_user": 2762080082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2762080082 }}, "user": { "id": 3018101791, "name": "shantel♒️", "screen_name": "littlefalconnnn", "lang": "en", "location": "Clovis/Reedley", "create_at": date("2015-02-12"), "description": "sc : bby.falconn // gym // success is the best revenge.", "followers_count": 566, "friends_count": 384, "statues_count": 20222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332569151021056, "text": "Thirst trappin' https://t.co/07OxBz0YCx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 153531540, "name": "prettymickiee", "screen_name": "MickieeMoscoto", "lang": "en", "location": "NY ✈️ ATL ✈️ LA", "create_at": date("2010-06-08"), "description": "a niggas dream & bitches nightmare bookingmickiee@gmail.com", "followers_count": 81388, "friends_count": 57644, "statues_count": 91942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332569289388033, "text": "@MamaBear4Cruz @Network_Citizen @ChuckNellis You mean link all the sources I provided that you ignored? LOL", "in_reply_to_status": 718332388456140800, "in_reply_to_user": 17694306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17694306, 3069361299, 315878798 }}, "user": { "id": 1287321284, "name": "Christopher Logan", "screen_name": "NAInfidels", "lang": "en", "location": "USA", "create_at": date("2013-03-21"), "description": "President of the America's most assertive anti-Islam organization. Founder of Logan's Warning.", "followers_count": 5121, "friends_count": 5623, "statues_count": 105945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332569629167617, "text": "#MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #MadayGHUS #GranHermanoUS 286", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "MadayGHUS", "GranHermanoUS" }}, "user": { "id": 505884498, "name": "Licet casado", "screen_name": "licetcasado", "lang": "es", "location": "BRONX", "create_at": date("2012-02-27"), "description": "http://facebook.com/CasadoShopOnli…", "followers_count": 124, "friends_count": 221, "statues_count": 19021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332569788551172, "text": "Football starts Monday #springball2016 https://t.co/UiUUYm7dt3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "springball2016" }}, "user": { "id": 2207941866, "name": "return of R.J5⃣", "screen_name": "_iamthetruth7", "lang": "en", "location": "null", "create_at": date("2013-11-21"), "description": "19 God #1 R.i.Paradise Marion ig- iamthetruth7 snapchat- younghamp. Yp. #footballLife. R.I.PGrabdpa B", "followers_count": 782, "friends_count": 698, "statues_count": 18291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570149265408, "text": "@TeeaamEdward DONE", "in_reply_to_status": 718332239898157056, "in_reply_to_user": 86079444, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 86079444 }}, "user": { "id": 76790297, "name": ".", "screen_name": "xBarrel_Riderx", "lang": "en", "location": "San Diego", "create_at": date("2009-09-23"), "description": "Viventes puellae mortuae -- IG: @xbarrel_riderx -- insane. Imperfect. Broken. || Singer. Writer. Coffee Addict|| I see and feel dead people.", "followers_count": 490, "friends_count": 357, "statues_count": 43802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570224762881, "text": "Meeeee �� https://t.co/RuxNwnUn4H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 261463861, "name": "Theyoncé➰", "screen_name": "the0nna_xo", "lang": "en", "location": "Raleigh, NC", "create_at": date("2011-03-05"), "description": "Confidence is the real key, bitch! • aren't you the girl who makes cupcakes? • | sc: the0nna_xo ✨| master mind of @TheyoncePastry_", "followers_count": 1819, "friends_count": 1039, "statues_count": 334947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570254057473, "text": "@ashhkardashh_ Who? https://t.co/WiIhW8DqmP", "in_reply_to_status": 718331258951045123, "in_reply_to_user": 3719950032, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3719950032 }}, "user": { "id": 83273852, "name": "Nov 23.", "screen_name": "Shardeee_", "lang": "en", "location": "null", "create_at": date("2009-10-17"), "description": "null", "followers_count": 1524, "friends_count": 1240, "statues_count": 45886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570291843072, "text": "@klosedkaskets @JacksonThings I'm dead ����", "in_reply_to_status": 718332368952692737, "in_reply_to_user": 393663113, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393663113, 943609752 }}, "user": { "id": 3368045757, "name": "Phoenix Down", "screen_name": "rogueee_", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-07-09"), "description": "I hope I die of laughter.", "followers_count": 297, "friends_count": 348, "statues_count": 6423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570577039364, "text": "I'd rather suck Bruce Jenner's dick than fuck Caitlyn Jenner. \n\nThat's not true. I was tryin to be dramatic. I'd suck both of their dicks.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 707956002, "name": "Flakka Martini", "screen_name": "ThirstNhowllll", "lang": "en", "location": "Fort Lauderdale, Florida", "create_at": date("2012-07-20"), "description": "I kicked my mother in the stomach but I was still born.", "followers_count": 1685, "friends_count": 232, "statues_count": 30291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570740658178, "text": "Wit a stripper & we in da church.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506514833, "name": "De'Andre Joseph", "screen_name": "d_jsph", "lang": "en", "location": "Bartlesville, Oklahoma", "create_at": date("2012-02-27"), "description": "//Freelance Videography// 2015 Show Reel ⏩https://youtu.be/C3o27fWnWSg Contact : d_jsph70@yahoo.com", "followers_count": 326, "friends_count": 245, "statues_count": 5518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bartlesville, OK", "id": "8de43ba3f996df05", "name": "Bartlesville", "place_type": "city", "bounding_box": rectangle("-96.012722,36.699006 -95.899191,36.786088") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40147, "countyName": "Washington", "cityID": 4004450, "cityName": "Bartlesville" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332570837102592, "text": "TAKE ME HERE ASAP I AM SO EXCITED OMG https://t.co/4qm0YJtSMr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 998825268, "name": "Maxine △⃒⃘", "screen_name": "n000dle", "lang": "en", "location": "Redondo Beach, CA", "create_at": date("2012-12-09"), "description": "✨⛈☠", "followers_count": 121, "friends_count": 112, "statues_count": 7503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redondo Beach, CA", "id": "e1e35d357ceefa52", "name": "Redondo Beach", "place_type": "city", "bounding_box": rectangle("-118.401931,33.814689 -118.352695,33.894649") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 660018, "cityName": "Redondo Beach" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332571512369152, "text": "2 days!! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1461708565, "name": "Kilimon", "screen_name": "KayeLimon", "lang": "en", "location": "Seattle, WA", "create_at": date("2013-05-27"), "description": "young and resilient | Snapchat: kayelimon ❤️", "followers_count": 422, "friends_count": 310, "statues_count": 20361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tukwila, WA", "id": "9883a4be0cbd497c", "name": "Tukwila", "place_type": "city", "bounding_box": rectangle("-122.318673,47.407147 -122.241356,47.531957") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5372625, "cityName": "Tukwila" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332572141490176, "text": "@GummyOctopus why :(", "in_reply_to_status": 718332255173742593, "in_reply_to_user": 34855967, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 34855967 }}, "user": { "id": 3118420572, "name": "KGB | Cakes", "screen_name": "SSBMcakes", "lang": "en", "location": "uuurrrvine", "create_at": date("2015-03-30"), "description": "competitive ssbm player", "followers_count": 458, "friends_count": 99, "statues_count": 680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332572363841536, "text": "5efcc78c00p7ECCF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-127.592653,55.02436"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 18, "friends_count": 0, "statues_count": 12748 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Kitimat-Stikine B, British Columbia", "id": "3479e679ae40df02", "name": "Kitimat-Stikine B", "place_type": "city", "bounding_box": rectangle("-128.950071,54.993627 -126.643762,56.083996") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332572544196608, "text": "When daddy...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546463071, "name": "Rachelle", "screen_name": "213Rachelle", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "20❤", "followers_count": 733, "friends_count": 314, "statues_count": 4914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-08T00:00:05.000Z"), "id": 718332572854579200, "text": "this is NOT what I meant by \"spread the word\" ���������� https://t.co/dXOqJoKzIR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1498854516, "name": "brat", "screen_name": "kiddree", "lang": "en", "location": "sc: kiddree", "create_at": date("2013-06-10"), "description": "it's never personal.", "followers_count": 288, "friends_count": 254, "statues_count": 9612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-08T00:00:06.000Z"), "id": 718332573005561856, "text": "Check out me and my buddies vine @neky dek", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.4829041,37.5700304"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16562373 }}, "user": { "id": 2898814763, "name": "Austin", "screen_name": "EgoTrip96", "lang": "en", "location": "VA", "create_at": date("2014-11-29"), "description": "Tattoo Apprentice @LiquidMetalTattoo in westpoint VA // DM me for custom Artwork and prices", "followers_count": 203, "friends_count": 533, "statues_count": 1076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51073, "countyName": "Gloucester" } }
+{ "create_at": datetime("2016-04-08T00:00:06.000Z"), "id": 718332573081067520, "text": "Finally 21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 714954819308380160, "name": "Morgan Stalker", "screen_name": "Captain_Morgan0", "lang": "en", "location": "Pahrump, NV", "create_at": date("2016-03-29"), "description": "null", "followers_count": 35, "friends_count": 68, "statues_count": 36 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pahrump, NV", "id": "0128fa53607453b1", "name": "Pahrump", "place_type": "city", "bounding_box": rectangle("-116.103394,36.096268 -115.902526,36.307718") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32023, "countyName": "Nye", "cityID": 3253800, "cityName": "Pahrump" } }
+{ "create_at": datetime("2016-04-08T00:00:06.000Z"), "id": 718332573122998272, "text": "2603 Castello Plan, Katy, TX 77494, $394,990 5 beds, 4.5 baths https://t.co/wwDfVQhwif", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.8302002,29.74020004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864061615, "name": "Katy, TX News", "screen_name": "BLifeKaty", "lang": "en", "location": "Katy, Texas", "create_at": date("2014-11-06"), "description": "Katy BubbleLife features community news, photos and events. Share your business, organization or personal news and events at http://katytx.bubblelife.com.", "followers_count": 291, "friends_count": 1103, "statues_count": 22003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694935755489280, "text": "I'm so sad that I'm not with Kiki while she's drunk rn ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 582282805, "name": "Gabbi✨", "screen_name": "_gabbim", "lang": "en", "location": "null", "create_at": date("2012-05-16"), "description": "sugar honey iced tea", "followers_count": 372, "friends_count": 197, "statues_count": 33811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694935864541184, "text": "Wind 2.0 mph E. Barometer 1023.91 mb, Steady. Temperature 44.8 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 13726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694936007335936, "text": "@its_slimx3 hi bitch", "in_reply_to_status": -1, "in_reply_to_user": 33635033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33635033 }}, "user": { "id": 49332051, "name": "Eric.", "screen_name": "LaceDownForWhat", "lang": "en", "location": "Basement", "create_at": date("2009-06-21"), "description": "19XX Hello world, I received your application, but I'm sorry to inform you that you do not meet my qualifications.", "followers_count": 428, "friends_count": 179, "statues_count": 14274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, IL", "id": "c3f1deee8671c492", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-88.565734,41.645151 -88.487888,41.684729") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17093, "countyName": "Kendall", "cityID": 1760352, "cityName": "Plano" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694936254656514, "text": "I'm tired.. I laughed a bit too much from that anime. I think this is gonna be the first time I slept with high spirits.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64442916, "name": "Phoenix Starrwind", "screen_name": "PhoenixStarwind", "lang": "en", "location": "San Diego, California", "create_at": date("2009-08-10"), "description": "I like to draw digitally and I play online and 3DS games. I'm a mixed breed so i'm neither black nor white. I like to have fun basically so let's get along.", "followers_count": 52, "friends_count": 56, "statues_count": 7026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694936866983936, "text": "God is all I have", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228993827, "name": "#BlackSheepDontGrin", "screen_name": "KingMotaw", "lang": "en", "location": "Carbondale, IL", "create_at": date("2010-12-20"), "description": "God 1st & Family 2nd | I'm just trying to get right it's so much stuff going wrong | #GrindHard #BWA | #FreeTheGuys #TeamLEO♌ #TeamLAKERS", "followers_count": 1796, "friends_count": 1574, "statues_count": 77676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belleville, IL", "id": "01d4e7421ef07326", "name": "Belleville", "place_type": "city", "bounding_box": rectangle("-90.099191,38.4682 -89.905307,38.59106") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1704845, "cityName": "Belleville" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694937433358336, "text": "Temp 30.5° Hi/Lo 33.3/30.5 Rng 2.8° WC 30.5° Hmd 82% Rain 0.00\" Storm 0.00\" BAR 29.738 Falling DP 25.7° Wnd 0mph Dir --- Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 67, "friends_count": 117, "statues_count": 17694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694937462730752, "text": "Wind 0.0 mph ---. Barometer 30.118 in, Steady. Temperature 45.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 60700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694937496170497, "text": "@RandallJSanders @Sweet_Loops @ChiBDM @midway_brennan @cubsbeisbol https://t.co/XRJwi4sBku", "in_reply_to_status": -1, "in_reply_to_user": 161782126, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 161782126, 52749730, 53315324, 405469139, 2722572896 }}, "user": { "id": 247154952, "name": "D-Conn™", "screen_name": "DCONN24", "lang": "en", "location": "Mountain Home", "create_at": date("2011-02-03"), "description": "Asst. Coach for @MHHSBBall. Announcer for @RazorbackHockey Cubs/Bulls/Blackhawks/Razorbacks/Bombers/Bears. Bat Flip Choreographer. Cubs Record: 2-0", "followers_count": 1249, "friends_count": 2438, "statues_count": 104215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain Home, AR", "id": "69e8d4996cf22753", "name": "Mountain Home", "place_type": "city", "bounding_box": rectangle("-92.436037,36.309943 -92.312154,36.382252") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5005, "countyName": "Baxter", "cityID": 547390, "cityName": "Mountain Home" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694937672486912, "text": "03:00:01 |Temp: 46.7ºF | Wind Chill 46.7ºF |Dew Point 39.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the S, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694938246914049, "text": "There some places where icyhot should never be.. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1457339628, "name": "kuya_zoomz", "screen_name": "carman233010", "lang": "en", "location": "null", "create_at": date("2013-05-25"), "description": "Be the fastest turtle out there", "followers_count": 303, "friends_count": 464, "statues_count": 638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694938259562496, "text": "When me and @rbaty1995 hang out https://t.co/xPubeqWa73", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2546124439 }}, "user": { "id": 238711625, "name": "JK", "screen_name": "JakeV_BU13", "lang": "en", "location": "null", "create_at": date("2011-01-15"), "description": "null", "followers_count": 1299, "friends_count": 1292, "statues_count": 48112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694938653773825, "text": "When u take 8 balloons and half a bottle to the dome! #wavy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wavy" }}, "user": { "id": 2893852910, "name": "Gator", "screen_name": "_niiickah", "lang": "en", "location": "null", "create_at": date("2014-11-08"), "description": "GHS'16 chef big tymin S❤", "followers_count": 255, "friends_count": 316, "statues_count": 3722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus, CA", "id": "04088141121c7398", "name": "Citrus", "place_type": "city", "bounding_box": rectangle("-117.899428,34.105384 -117.876959,34.127081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 613560, "cityName": "Citrus" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694939077517312, "text": "I'm just adjusting to my new life. It's the best time of my life— ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2758334794, "name": "NC", "screen_name": "_NicoleCynthia", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "null", "followers_count": 505, "friends_count": 192, "statues_count": 936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Holland, IL", "id": "f1dfa58974449052", "name": "South Holland", "place_type": "city", "bounding_box": rectangle("-87.636467,41.576797 -87.563393,41.62263") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1770850, "cityName": "South Holland" } }
+{ "create_at": datetime("2016-04-09T00:00:00.000Z"), "id": 718694939232567296, "text": "Life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576436876, "name": "Jerome Cadiz", "screen_name": "JeromeExists", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "Save life. Realizer | Runner | Wannabe Model | '80s Blood | Dancer | Believer", "followers_count": 123, "friends_count": 107, "statues_count": 1724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694939928895488, "text": "@superjoshyyy hindi nga e ��", "in_reply_to_status": 718659348394717185, "in_reply_to_user": 315492671, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 315492671 }}, "user": { "id": 93341808, "name": "Yelle Del Rosario", "screen_name": "yelleroyce", "lang": "en", "location": "ph✈️us", "create_at": date("2009-11-28"), "description": "trust no one • 18 • feumnl • snap/ig: yelleroyce", "followers_count": 7996, "friends_count": 1028, "statues_count": 88541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694940084011008, "text": "Well April 8th you were a day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273039140, "name": "Chey$ilver", "screen_name": "cheyenneevelynw", "lang": "en", "location": "working on TPSP", "create_at": date("2011-03-27"), "description": "Really good. Really evil. At the same damn time. I'm not a concept", "followers_count": 727, "friends_count": 299, "statues_count": 183833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694940096782337, "text": "���� Lucci", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 287296109, "name": "™FluieAss Bandz™", "screen_name": "ttu_bandz", "lang": "en", "location": "Wilkinsburg, PA", "create_at": date("2011-04-24"), "description": "T.R.◬.P.N.✯.☥.i #LLJ #LLF #LLT", "followers_count": 1677, "friends_count": 1407, "statues_count": 16462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694940205780992, "text": "Just a wee bit excited for this ���� https://t.co/r4jgukPkeP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 400531420, "name": "➳Krissy➳", "screen_name": "Krissy_Kris06", "lang": "en", "location": "null", "create_at": date("2011-10-28"), "description": "Heart is taken by a sailor ⚓ GVSU '19 snapchat: @krissy_kris06 Instagram: @snowboardchick06 #Delta2021", "followers_count": 688, "friends_count": 415, "statues_count": 3194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allendale, MI", "id": "5b0caebf50f0bf61", "name": "Allendale", "place_type": "city", "bounding_box": rectangle("-85.973029,42.943183 -85.876713,42.99425") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26139, "countyName": "Ottawa", "cityID": 2601340, "cityName": "Allendale" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694940566376448, "text": "@youngkeeesh @tsiannabackus you lyin", "in_reply_to_status": 718691825838501888, "in_reply_to_user": 2768479327, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2768479327, 4469197153 }}, "user": { "id": 2906671922, "name": "Madame Zeroni", "screen_name": "AchmedAlani", "lang": "en", "location": "arabdingo", "create_at": date("2014-11-21"), "description": "damn near the funniest person I know", "followers_count": 439, "friends_count": 425, "statues_count": 5326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694941031931905, "text": "@starlust96 still luv u tho", "in_reply_to_status": 718686048427978752, "in_reply_to_user": 3378518712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3378518712 }}, "user": { "id": 28742539, "name": "professional negro", "screen_name": "HandsomeAssOreo", "lang": "en", "location": "Texas", "create_at": date("2009-04-03"), "description": "not a chipmunk | ig: @handsomeassoreo", "followers_count": 1613, "friends_count": 1049, "statues_count": 50278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen, TX", "id": "22d928cbeab790ad", "name": "Allen", "place_type": "city", "bounding_box": rectangle("-96.736596,33.066464 -96.608938,33.158169") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4801924, "cityName": "Allen" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694941317271552, "text": "Ripley SW Limestone Co. Temp: 44.6°F Wind:0.0mph Pressure: 997.3mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694941422006273, "text": "@asvpamber_ my world lol", "in_reply_to_status": 718694807560790017, "in_reply_to_user": 3884532204, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3884532204 }}, "user": { "id": 3884532204, "name": ".", "screen_name": "asvpamber_", "lang": "en", "location": "w. my girl ", "create_at": date("2015-10-13"), "description": "null", "followers_count": 232, "friends_count": 187, "statues_count": 10272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694941518651393, "text": "03:00 31.8°F Feels:24.3°F (Hi34.0°F/Lo31.8°F) Hum:67% Wnd:NW 8.6MPH Baro:29.73in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 194, "friends_count": 263, "statues_count": 28136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694942080499712, "text": "Im gonna start saving money right now for October when the rangers are in the World Series so I can go to every home game!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417198746, "name": "Pony Boy", "screen_name": "hawthorne__2", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "¯\\_(ツ)_/¯", "followers_count": 749, "friends_count": 98, "statues_count": 24010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694942537723904, "text": "You're not fine, Feel better soon :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572753844, "name": "Henry", "screen_name": "ImHengry", "lang": "en", "location": "University of Utah", "create_at": date("2012-05-06"), "description": "I'm a wizard, deal with it | Leveled", "followers_count": 408, "friends_count": 319, "statues_count": 5617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694942713847809, "text": "@dylancecot hi!", "in_reply_to_status": -1, "in_reply_to_user": 2618580021, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2618580021 }}, "user": { "id": 299324957, "name": "sarah", "screen_name": "sarahraykhman", "lang": "en", "location": "null", "create_at": date("2011-05-15"), "description": "null", "followers_count": 546, "friends_count": 238, "statues_count": 7838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694942793678852, "text": "Up like I don't get my hair cut at 11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 974971375, "name": "Almighty ✨", "screen_name": "_ZaeSavage", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "Ralphy World | Just a kid trying to make it ✊| #VSU19 | 757", "followers_count": 990, "friends_count": 837, "statues_count": 31542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton, VA", "id": "2f5f5ba43ec2f5a9", "name": "Hampton", "place_type": "city", "bounding_box": rectangle("-76.451006,36.989568 -76.273546,37.112225") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51650, "countyName": "Hampton", "cityID": 5135000, "cityName": "Hampton" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694943145988096, "text": "Wind 1.0 mph NW. Barometer 30.171 in, Rising. Temperature 37.6 °F. Rain today 0.00 in. Humidity 46%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694943238307840, "text": "04/09@03:00 - Temp 34.1F, WC 34.1F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.769in, Falling slowly. Rain 0.00in. Hum 86%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694943665954817, "text": "I got my 21st next week and the week after I got coachella ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 387851120, "name": "Δ-9", "screen_name": "BluntMePlease", "lang": "en", "location": "California, USA", "create_at": date("2011-10-09"), "description": "Dodgers Lakers Chargers UA Wildcats", "followers_count": 109, "friends_count": 182, "statues_count": 4081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-09T00:00:01.000Z"), "id": 718694943804358657, "text": "@ebbtideapp Tide in Pearlington, Mississippi 04/09/2016\nHigh 4:05pm 1.2\n Low 2:20am -0.1\nHigh 4:45pm 1.4\n Low 3:39am -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-89.615,30.24"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 22906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28045, "countyName": "Hancock", "cityID": 2855920, "cityName": "Pearlington" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694943989096448, "text": "Temp: 47.0°F Wind:0.0mph Pressure: 30.148hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 60701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944236392452, "text": "As in, I probably would've bought him a gundam. But he fucked that all up so hahahaha I have one. You don't.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43255130, "name": "Sailor Neptune ✨", "screen_name": "thidatotter", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-28"), "description": "Falling down the rabbit hole.", "followers_count": 537, "friends_count": 220, "statues_count": 55160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944525918210, "text": "Wow lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302232742, "name": "♍️Xernie Banders⚽️", "screen_name": "Paisa_Papiii", "lang": "en", "location": "PA | NYC | Medellin, Colombia", "create_at": date("2011-05-20"), "description": "20 | (B)uilt (O)n (S)elf (S)uccess | XO | RIP JD | #PLUR #PrayForPeace", "followers_count": 772, "friends_count": 728, "statues_count": 124254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, PA", "id": "307a7ae81e051945", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-75.344806,40.631608 -75.306619,40.662596") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4249144, "cityName": "Middletown" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944580378625, "text": "it's called good coffee for a reason☕️�� https://t.co/JMdcr89C8A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.61348,45.505344"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 956666030, "name": "Bailey Sarah", "screen_name": "supremebay", "lang": "en", "location": "portlandia", "create_at": date("2012-11-18"), "description": "Narcissistic Wednesday", "followers_count": 442, "friends_count": 301, "statues_count": 6945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Good Coffee", "id": "07d9ec35ca087000", "name": "Good Coffee", "place_type": "poi", "bounding_box": rectangle("-122.61348009999999,45.5053439 -122.61348,45.505344") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944664326144, "text": "I love that I grew up in donora", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379116391, "name": "Mike", "screen_name": "MikeBeveridge44", "lang": "en", "location": "412", "create_at": date("2011-09-24"), "description": "Rest in peace Nick Landman!!", "followers_count": 612, "friends_count": 403, "statues_count": 10150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, PA", "id": "2cea0164374447b6", "name": "California", "place_type": "city", "bounding_box": rectangle("-79.923076,40.025892 -79.880322,40.084191") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4210768, "cityName": "California" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944790044672, "text": "itea bonding at cash cash https://t.co/fMmRK6OVuR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 124649804, "name": "j swole", "screen_name": "studpham", "lang": "en", "location": "4 0 8 b o y s", "create_at": date("2010-03-19"), "description": "whack ass history projects smh", "followers_count": 176, "friends_count": 235, "statues_count": 178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694944895066112, "text": "Wind 3.0 mph W. Barometer 29.985 in, Rising slowly. Temperature 32.9 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694945490481152, "text": "You girls make me jealous!!! Enjoy!! https://t.co/Lc60K3ulBG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 936478014, "name": "Delia Lara", "screen_name": "DeliaLara1888", "lang": "en", "location": "null", "create_at": date("2012-11-09"), "description": "null", "followers_count": 77, "friends_count": 225, "statues_count": 93 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694945498882048, "text": "Idk what to eat but I need to eat something bc this headache is only getting worse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734783736, "name": "rae.", "screen_name": "2fabKass", "lang": "en", "location": "null", "create_at": date("2012-08-03"), "description": "jayme ❤", "followers_count": 527, "friends_count": 245, "statues_count": 43254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur, LA", "id": "253b154805ec7fbb", "name": "Sulphur", "place_type": "city", "bounding_box": rectangle("-93.427304,30.183555 -93.257523,30.277601") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2273640, "cityName": "Sulphur" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694945591140353, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 812589752, "name": "Madds", "screen_name": "maddiegates2", "lang": "en", "location": "California ✈️ Oregon", "create_at": date("2012-09-09"), "description": "| Our soul knows the geography of our destiny |party on | Psalm 51:10 | ☼ |", "followers_count": 847, "friends_count": 578, "statues_count": 58716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694946161762304, "text": "Wind 0.0 mph ---. Barometer 29.754 in, Steady. Temperature 20.6 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694946841108480, "text": "@isabella_gaavin @its_emily58 okay okay:(", "in_reply_to_status": 718694782625644544, "in_reply_to_user": 2249243479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2249243479, 1584696530 }}, "user": { "id": 1514277433, "name": "blaz", "screen_name": "BlaiseBonomo", "lang": "en", "location": "Carson City, NV", "create_at": date("2013-06-13"), "description": "watching movies and grabbing booties", "followers_count": 479, "friends_count": 513, "statues_count": 4332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson City, NV", "id": "f2390622c570faf1", "name": "Carson City", "place_type": "city", "bounding_box": rectangle("-119.814173,39.08497 -119.670954,39.216676") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32510, "countyName": "Carson City", "cityID": 3209700, "cityName": "Carson City" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694947449208833, "text": "@Mtvmtv8 here u go this is the answer https://t.co/nXvUIkuf0u", "in_reply_to_status": 718693770099863552, "in_reply_to_user": 2940926166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2940926166 }}, "user": { "id": 604964149, "name": "hooligan", "screen_name": "nemonipples", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "I don't tell my friends happy birthday", "followers_count": 742, "friends_count": 131, "statues_count": 18147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-04-09T00:00:02.000Z"), "id": 718694947742855168, "text": "@Mannie_fresco happy birthday, mannie!!!", "in_reply_to_status": 718694637163032576, "in_reply_to_user": 1533635522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1533635522 }}, "user": { "id": 708637742, "name": "Bella", "screen_name": "leboobslebae", "lang": "en", "location": "San Marcos, TX", "create_at": date("2012-07-21"), "description": "huskies, Wingstop & naps.", "followers_count": 1058, "friends_count": 608, "statues_count": 67560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694948363603969, "text": "@SPACEGIRLGEMMY gah bless", "in_reply_to_status": 718642244866015233, "in_reply_to_user": 3020262043, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3020262043 }}, "user": { "id": 1413702708, "name": "Pixelon™", "screen_name": "OfficialPixelon", "lang": "en", "location": "Anime", "create_at": date("2013-05-08"), "description": "@RoraCollective Co-Founder • Media Guy • 3DS#0405-0004-9450 • Twt me for inquiries!", "followers_count": 350, "friends_count": 656, "statues_count": 11643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694948409749505, "text": "someone facetime me 5306139419", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1377059492, "name": "mac bay", "screen_name": "baileyyxavier", "lang": "en", "location": "null", "create_at": date("2013-04-24"), "description": "null", "followers_count": 1673, "friends_count": 271, "statues_count": 14263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado Hills, CA", "id": "ab2b3353d37b40e6", "name": "El Dorado Hills", "place_type": "city", "bounding_box": rectangle("-121.111768,38.621986 -121.027263,38.743729") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 621880, "cityName": "El Dorado Hills" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694948472807424, "text": "@VerizonSupport Unlock the FM chip on all @Verizon phones so I can listen for free to @NextRadioApp! #unlockFM #freeradio", "in_reply_to_status": -1, "in_reply_to_user": 37435412, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "unlockFM", "freeradio" }}, "user_mentions": {{ 37435412, 59889953, 891575689 }}, "user": { "id": 28003616, "name": "Jon Lee On Air", "screen_name": "jonleeradio", "lang": "en", "location": "Beaverton", "create_at": date("2009-03-31"), "description": "Nightside Radio Station Host. Sports Producer. Hear me weekdays 12-5am on 93.9 KPDQ-FM Portland-Vanc. and http://www.kpdq.com", "followers_count": 305, "friends_count": 2053, "statues_count": 4999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694948942516224, "text": "Wo liegt Newark? https://t.co/slCNr9n2Wb #Newark #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1722,40.7356"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Newark", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 462, "friends_count": 116, "statues_count": 1902090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694948984324097, "text": "18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3235713642, "name": "Emily", "screen_name": "a__miillii", "lang": "en", "location": "null", "create_at": date("2015-06-03"), "description": "IG - amilia.emily || Paralegal", "followers_count": 728, "friends_count": 184, "statues_count": 3440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949097771008, "text": "#ritzselfie #theritzsj #sanjosenightlife @ The Ritz https://t.co/lrwIp2HyOC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.8855133,37.3300514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ritzselfie", "theritzsj", "sanjosenightlife" }}, "user": { "id": 563656677, "name": "GingerMonkey", "screen_name": "m0nk4y4g", "lang": "en", "location": "Sunnyvale, CA", "create_at": date("2012-04-26"), "description": "Just a Monkey with a keyboard... Ang Mo Kui", "followers_count": 252, "friends_count": 790, "statues_count": 1247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949235982336, "text": "#Trump in 2016 Temp:36.1°F Wind:2.7mph Pressure: 29.94hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 314686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949252784130, "text": "54.7F (Feels: 54.7F) - Humidity: 66% - Wind: 2.2mph E - Gust: 2.2mph - Pressure: 1016.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 234365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949416484864, "text": "Wind 0.0 mph W. Barometer 29.75 in, Falling slowly. Temperature 27.9 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 35, "friends_count": 118, "statues_count": 158924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949714198529, "text": "@HALEiGHNiCOLE19 NO!!! I've tried the last 4 times she's restocked and have yet to get one��", "in_reply_to_status": 718626586124292101, "in_reply_to_user": 387241391, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 387241391 }}, "user": { "id": 396941929, "name": "Taylor Foster", "screen_name": "Taylor_Foster95", "lang": "en", "location": "null", "create_at": date("2011-10-23"), "description": "Arlington,TX Tarleton2018", "followers_count": 192, "friends_count": 204, "statues_count": 2748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694949751906304, "text": "Popping bottles because I'm bottling in my feelings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 391037827, "name": "cynthia rodriguez", "screen_name": "Cynteaahh", "lang": "en", "location": "ig / sc : cynteaahh", "create_at": date("2011-10-14"), "description": "romantic poetical type. CSUF 19", "followers_count": 1238, "friends_count": 668, "statues_count": 39716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South El Monte, CA", "id": "122aef8d6840943b", "name": "South El Monte", "place_type": "city", "bounding_box": rectangle("-118.075186,34.028137 -118.021298,34.064367") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 672996, "cityName": "South El Monte" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694950104338433, "text": "I hope you all notice the two bubbles in this photo! :) #bubbles… https://t.co/QqCecxziQ2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.65778721,40.3136582"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bubbles" }}, "user": { "id": 702748221580251136, "name": "Perry Joseph Olsen", "screen_name": "PerryJosephOls3", "lang": "en", "location": "null", "create_at": date("2016-02-24"), "description": "null", "followers_count": 123, "friends_count": 418, "statues_count": 420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694950330720257, "text": "Nikes on my feet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1552451269, "name": "Kyle Hyer", "screen_name": "HyerKyle", "lang": "en", "location": "null", "create_at": date("2013-06-28"), "description": "Everybody believes in something, I believe I'll have another beer. PNW", "followers_count": 318, "friends_count": 190, "statues_count": 5862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walla Walla, WA", "id": "3f449e4a20757d86", "name": "Walla Walla", "place_type": "city", "bounding_box": rectangle("-118.379086,46.022857 -118.263051,46.106424") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53071, "countyName": "Walla Walla", "cityID": 5375775, "cityName": "Walla Walla" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694950561406976, "text": "he's fineeee omg but pass he look like a hoe https://t.co/mgjQ3KgWUH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3240514970, "name": "samsam", "screen_name": "samsamamohamud", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "3900", "followers_count": 813, "friends_count": 661, "statues_count": 17947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694950708187136, "text": "https://t.co/3kHNWqludb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 22094990, "name": "Caitlin", "screen_name": "uhhuhcait", "lang": "en", "location": "Media, Delco, PA, USA. ", "create_at": date("2009-02-26"), "description": "Tweets about music, hockey, and dumb things I do. Philly. Flyers & Kings. I'm a wild child, baby.", "followers_count": 146, "friends_count": 492, "statues_count": 5458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermosa Beach, CA", "id": "4baf4d09759d33e4", "name": "Hermosa Beach", "place_type": "city", "bounding_box": rectangle("-118.409579,33.85185 -118.384866,33.877823") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 633364, "cityName": "Hermosa Beach" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694950909521920, "text": "My bitch I buy her jimmy choos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2569442030, "name": "let me gas you up", "screen_name": "taewest", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-06-15"), "description": "fuck vh1", "followers_count": 517, "friends_count": 549, "statues_count": 36532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951261851648, "text": "@_2strange4u haha ohh ��", "in_reply_to_status": 718694740279959552, "in_reply_to_user": 261274457, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 261274457 }}, "user": { "id": 3189605172, "name": "michαɘℓ", "screen_name": "kekadur", "lang": "en", "location": "Tucson, AZ", "create_at": date("2015-05-09"), "description": "The Exotic Is My Everyday Breathe ☪", "followers_count": 1129, "friends_count": 946, "statues_count": 5009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flowing Wells, AZ", "id": "e09e7a006eb223b2", "name": "Flowing Wells", "place_type": "city", "bounding_box": rectangle("-111.04135,32.279249 -110.980032,32.310435") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 423960, "cityName": "Flowing Wells" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951505260545, "text": "Being this drunk and you gotta get even more drunk the next day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285929746, "name": "chris waddy", "screen_name": "champcalledfuzz", "lang": "en", "location": "cincinnati", "create_at": date("2011-04-21"), "description": "UC.", "followers_count": 821, "friends_count": 693, "statues_count": 8526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951538728960, "text": "HAPPY BIRTHDAY TO THE YENNA YADAMS! I love you so much and I'm honored to be your foster child ��. You're legal! �� https://t.co/TELyeTUTkT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1833227323, "name": "el oh el", "screen_name": "ronikastonee", "lang": "en", "location": "null", "create_at": date("2013-09-08"), "description": "Universitizzle of Oregon '20", "followers_count": 800, "friends_count": 554, "statues_count": 9302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951719202816, "text": "@garpike28 Hey if you like Jon Connor make sure you hear this new freestyle It's Sick!!!https://t.co/MvPBF6yaYY", "in_reply_to_status": 718105990592348160, "in_reply_to_user": 466834322, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 466834322 }}, "user": { "id": 98454190, "name": "Nizo Pavarotti", "screen_name": "iamnizo", "lang": "en", "location": "Flint M.i.✈️✈️✈️Los Angeles ", "create_at": date("2009-12-21"), "description": "Lil brother \r\nAnd\r\nProtege of #JonConnor", "followers_count": 434, "friends_count": 268, "statues_count": 12162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951740006401, "text": "Fuck you stupid whore I miss you come sit on my face bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2165514800, "name": "Quan", "screen_name": "ratchet_quan", "lang": "en", "location": "United States", "create_at": date("2013-10-30"), "description": "If you're reading this there's a 94% chance you're ratchet", "followers_count": 570, "friends_count": 493, "statues_count": 8506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951836524544, "text": "@ALEYNALMAO foh and shove a foot in your ass��", "in_reply_to_status": 718694749050277888, "in_reply_to_user": 2588802403, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2588802403 }}, "user": { "id": 1575869305, "name": "kenneth ng", "screen_name": "kennethh_ngg", "lang": "en", "location": "san francisco, ca", "create_at": date("2013-07-07"), "description": "all $milez.", "followers_count": 294, "friends_count": 392, "statues_count": 3590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daly City, CA", "id": "6a71821001635bbd", "name": "Daly City", "place_type": "city", "bounding_box": rectangle("-122.500164,37.649122 -122.405233,37.708437") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 617918, "cityName": "Daly City" } }
+{ "create_at": datetime("2016-04-09T00:00:03.000Z"), "id": 718694951882633216, "text": "They was out here like roaches tonight @ The Biltmore https://t.co/C1N66e1rk5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.50053,38.03637"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120198402, "name": "Ike Love", "screen_name": "CRIMEBOSS752", "lang": "en", "location": "page st ", "create_at": date("2010-03-05"), "description": "My father was a legend", "followers_count": 470, "friends_count": 939, "statues_count": 34991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlottesville, VA", "id": "531a6942bb8056b9", "name": "Charlottesville", "place_type": "city", "bounding_box": rectangle("-78.528507,38.009553 -78.446389,38.07052") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51540, "countyName": "Charlottesville", "cityID": 5114968, "cityName": "Charlottesville" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694952457273344, "text": "Happy 18th birthday to my bestfriend �������� @kenz_getch https://t.co/bwvIEzWHTZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 569542076 }}, "user": { "id": 954719737, "name": "Samantha Mayfield", "screen_name": "sammayfieldd", "lang": "en", "location": "Washington", "create_at": date("2012-11-17"), "description": "null", "followers_count": 686, "friends_count": 276, "statues_count": 25868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.567788,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694953044418561, "text": "Lmao taje �� https://t.co/pRbUIsjLB8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 3521580854, "name": "heartbreak suzy ✞", "screen_name": "UncleTrapQueen", "lang": "en", "location": "rich city ", "create_at": date("2015-09-10"), "description": "18 • YNK GIRL GANG ✨ • SC: styrfry96", "followers_count": 180, "friends_count": 459, "statues_count": 3194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Sobrante, CA", "id": "af57c8053272899f", "name": "El Sobrante", "place_type": "city", "bounding_box": rectangle("-122.318538,37.938766 -122.244018,37.991108") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 622457, "cityName": "El Sobrante" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694953505792001, "text": "@InsideSoCalSpts shocking that Crawfish is injured again", "in_reply_to_status": 718693773027450880, "in_reply_to_user": 2309040030, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2309040030 }}, "user": { "id": 1918772167, "name": "Russell Small", "screen_name": "brewnnut", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-09-29"), "description": "Certified Public Accountant (CPA). Alumnus and fan of UCLA Bruins; Los Angeles Dodgers; Los Angeles Kings; Los Angeles Rams (welcome home!); and NY Giants.", "followers_count": 25, "friends_count": 129, "statues_count": 529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954055262208, "text": "@billmaher @GOP So be it.", "in_reply_to_status": 718658831098769412, "in_reply_to_user": 19697415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19697415, 11134252 }}, "user": { "id": 48861915, "name": "Anne M Hughes", "screen_name": "AnneMHughes", "lang": "en", "location": "Jurupa Valley, CA 92509 USA", "create_at": date("2009-06-19"), "description": "null", "followers_count": 99, "friends_count": 158, "statues_count": 441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954067828736, "text": "Doesn't drink on a Friday night, still ends up throwing up in the bathroom. Yay migraines. ��������������������������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 904680116, "name": "WhiteCandy", "screen_name": "CassidyGlasmann", "lang": "en", "location": "null", "create_at": date("2012-10-25"), "description": "SC-- cashxoxo27 #20", "followers_count": 354, "friends_count": 424, "statues_count": 5187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ogden, UT", "id": "4980bc1544652d22", "name": "Ogden", "place_type": "city", "bounding_box": rectangle("-112.025874,41.149594 -111.904265,41.286003") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4955980, "cityName": "Ogden" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954462089216, "text": "I want there to be a way of making a living off of just traveling the world riding rollercoasters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251812968, "name": "Kristi McLaughlin", "screen_name": "kristimacattack", "lang": "en", "location": "Scottsdale, AZ", "create_at": date("2011-02-13"), "description": "Hi", "followers_count": 1358, "friends_count": 594, "statues_count": 8545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954491633664, "text": "@_Jvyy ��it is but .... I'm on a lil drought and it's by choice so that should count ��", "in_reply_to_status": 718694436511858688, "in_reply_to_user": 521459689, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521459689 }}, "user": { "id": 53630929, "name": "JT", "screen_name": "jarel22", "lang": "en", "location": "null", "create_at": date("2009-07-04"), "description": "Cleveland", "followers_count": 1682, "friends_count": 1524, "statues_count": 148550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euclid, OH", "id": "59ed3d5f720ebdf0", "name": "Euclid", "place_type": "city", "bounding_box": rectangle("-81.552943,41.545336 -81.488014,41.631584") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3925704, "cityName": "Euclid" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954579685376, "text": "Nathan Andrew Zimring, Oak Brook, IL, 60523 - https://t.co/I9npy7lpYz\n\n#nathanandrewzimring #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.98317,41.845676"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nathanandrewzimring", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 422, "friends_count": 428, "statues_count": 237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Brook, IL", "id": "04c0488a01ad05ce", "name": "Oak Brook", "place_type": "city", "bounding_box": rectangle("-88.003635,41.812505 -87.900262,41.862029") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1754534, "cityName": "Oak Brook" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954734714880, "text": "pass https://t.co/vA2PVPm3MJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056619452, "name": "BUTTERCUP", "screen_name": "BradyFHannah", "lang": "en", "location": "618", "create_at": date("2013-01-02"), "description": "take yo bitch she need some milk.", "followers_count": 924, "friends_count": 642, "statues_count": 2183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694954835402752, "text": "Walking around campus today seeing sporting events, student life events & even discover events & realizing how amazing it is here\nI love GCU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551362817, "name": "Emily Sanda", "screen_name": "EmilySanda", "lang": "en", "location": "null", "create_at": date("2012-04-11"), "description": "I'm probably thinking about French Fries. GCU", "followers_count": 652, "friends_count": 485, "statues_count": 11246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694955191914496, "text": "@avigheyapetrik @wizardingomez I know ����", "in_reply_to_status": 718694880877215744, "in_reply_to_user": 2959102554, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2959102554, 193021663 }}, "user": { "id": 896369742, "name": "Tori♛", "screen_name": "toriarmendariz", "lang": "en", "location": "Renton Wa", "create_at": date("2012-10-21"), "description": "life is complicated, but it's beautiful. I'm not as rude as I look", "followers_count": 575, "friends_count": 810, "statues_count": 31181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694955259047937, "text": "My phone is on 4% !! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3776106972, "name": "Angel Rae", "screen_name": "AngelRaeee", "lang": "en", "location": "null", "create_at": date("2015-10-03"), "description": "new account", "followers_count": 362, "friends_count": 200, "statues_count": 5629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenville, TX", "id": "d3aa6dad1368841f", "name": "Greenville", "place_type": "city", "bounding_box": rectangle("-96.162071,33.064175 -96.049505,33.171817") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4830920, "cityName": "Greenville" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694955808460800, "text": "Cudi all night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648416896, "name": "DANIEL", "screen_name": "Lackey28", "lang": "en", "location": "Oxnard, CA", "create_at": date("2013-08-05"), "description": "null", "followers_count": 641, "friends_count": 525, "statues_count": 7448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694955913449473, "text": "Pictures of guests? No, my Kodak moments are meals that I make from… https://t.co/WpBImCWI07", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.02638889,36.97194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421566124, "name": "Aprí Medina", "screen_name": "uhpree", "lang": "en", "location": "null", "create_at": date("2011-11-25"), "description": "null", "followers_count": 10, "friends_count": 80, "statues_count": 385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Cruz, CA", "id": "3c9e627dd6b55d9e", "name": "Santa Cruz", "place_type": "city", "bounding_box": rectangle("-122.076144,36.948098 -121.986229,37.010652") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 669112, "cityName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694956060291072, "text": "@olamide_YBNL @iamDo2dtun @ Houston, Texas https://t.co/PmjmPTMUzo", "in_reply_to_status": -1, "in_reply_to_user": 190481460, "favorite_count": 0, "coordinate": point("-95.3832,29.7629"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 190481460, 35963976 }}, "user": { "id": 175316806, "name": "ButaneGrant", "screen_name": "biyione", "lang": "en", "location": "ÜT: 6.552121,3.2693087", "create_at": date("2010-08-06"), "description": "#TeamManutd Music person #Islam,Chemical Engineer Sport Sport instagram butanegrant", "followers_count": 1262, "friends_count": 1007, "statues_count": 106330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694956181770241, "text": "So excited for #ATL May 1st ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ATL" }}, "user": { "id": 80759500, "name": "Ashlee Elizabeth", "screen_name": "miss_mulatto", "lang": "en", "location": "Bay Area, CA ", "create_at": date("2009-10-07"), "description": "Make up artist | hair styling | Male grooming | occasional model ☺️", "followers_count": 1698, "friends_count": 413, "statues_count": 29387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-09T00:00:04.000Z"), "id": 718694956240478209, "text": "3 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 381700361, "name": "Jordyn Yeager", "screen_name": "JordynYeager", "lang": "en", "location": "Bloomsburg University", "create_at": date("2011-09-28"), "description": "null", "followers_count": 1104, "friends_count": 362, "statues_count": 15177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonwald, PA", "id": "01b184931c03b159", "name": "Jacksonwald", "place_type": "city", "bounding_box": rectangle("-75.85984,40.311306 -75.822024,40.337735") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4237584, "cityName": "Jacksonwald" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694956563607552, "text": "Wind 4.0 mph WNW. Barometer 29.635 in, Steady. Temperature 31.9 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 5626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694957154832384, "text": "Well I think it's time to quit........", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42999566, "name": "sid", "screen_name": "SidneyBiondi", "lang": "en", "location": "1020 Dancer, Cedar Falls, IA", "create_at": date("2009-05-27"), "description": "Same", "followers_count": 356, "friends_count": 298, "statues_count": 40549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694957222129664, "text": "#pcb2k16 them boys https://t.co/s8w8T7gYbW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "pcb2k16" }}, "user": { "id": 301668743, "name": "B2G", "screen_name": "T3ASING_D3ATH", "lang": "en", "location": "UnderCover", "create_at": date("2011-05-19"), "description": "Half Human/Half God. it's luh Bran Man nice ta meet ya. - Business/Gah Damn.", "followers_count": 2078, "friends_count": 1517, "statues_count": 23532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Panama City Beach, FL", "id": "9ebd5acfac2301ba", "name": "Panama City Beach", "place_type": "city", "bounding_box": rectangle("-85.95802,30.165061 -85.786077,30.266595") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1254725, "cityName": "Panama City Beach" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694957314232320, "text": "They weren't ready for the model pose ! Love them ������ made my 21 great ☺️! https://t.co/pb9MMAD4Lz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 106258110, "name": "Kassy", "screen_name": "ThatWeirdRIco", "lang": "en", "location": "California ", "create_at": date("2010-01-18"), "description": "College althete, track nation, business major", "followers_count": 1040, "friends_count": 838, "statues_count": 9059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694957406662656, "text": "current hate everyoNe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381307869, "name": "dani california", "screen_name": "danielIegarcia", "lang": "en", "location": "ny", "create_at": date("2011-09-27"), "description": "snap me: danielleegarcia", "followers_count": 1188, "friends_count": 649, "statues_count": 46692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694957465346049, "text": "ETR MIAH. [@ogmuchee] https://t.co/dOP6TLVKDN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 171134786 }}, "user": { "id": 236156697, "name": "Kenny", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | #EnginEars | #KennythingCanHappen | Mix x Master $40 Per Song | excusetherant@gmail.com", "followers_count": 796, "friends_count": 387, "statues_count": 107428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694958191013889, "text": "on me fuck these other hoes https://t.co/DXTjxe7dwi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327158364, "name": "jordan ✨", "screen_name": "DaePhillips", "lang": "en", "location": "somewhere salty", "create_at": date("2011-06-30"), "description": "idc, kiss my ass.", "followers_count": 3213, "friends_count": 772, "statues_count": 160912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694958320889856, "text": "@jimmmerzz drier*", "in_reply_to_status": 718694848283303936, "in_reply_to_user": 1199593842, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 555783343 }}, "user": { "id": 1199593842, "name": "Kass", "screen_name": "kaassx_", "lang": "en", "location": "bay area", "create_at": date("2013-02-19"), "description": "I'm high rn", "followers_count": 508, "friends_count": 297, "statues_count": 38537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694958526431232, "text": "once I became single.. I started slipping when it came to taking nudes lol them hoes rare now", "in_reply_to_status": 718694805870514176, "in_reply_to_user": 422120654, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422120654, "name": "Hilary Banks ✨", "screen_name": "sadebaby__", "lang": "en", "location": "♋️", "create_at": date("2011-11-26"), "description": "null", "followers_count": 1967, "friends_count": 910, "statues_count": 43116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694958857863169, "text": "Surprise me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148891868, "name": "SOLAECLIPSE", "screen_name": "DrinkSolaPop", "lang": "en", "location": "A train to Far Rockaway. NYC.", "create_at": date("2010-05-27"), "description": "Na⚓️vY", "followers_count": 1200, "friends_count": 668, "statues_count": 34844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694959734337537, "text": "@LayneeGaGa Bro it's 12:59.", "in_reply_to_status": 718694763474526208, "in_reply_to_user": 253257736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 253257736 }}, "user": { "id": 97270758, "name": "Steph", "screen_name": "OhItsStephen", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2009-12-16"), "description": "null", "followers_count": 84, "friends_count": 117, "statues_count": 10372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694959923093504, "text": "Wind 0 mph ---. Barometer 1008.1 hPa, Falling. Temperature 71.1 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 152029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694959927283713, "text": "Cleansing myself of any and all leftover bitterness. Let's be kind to each other. I love you all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3298231411, "name": "Paige Perez", "screen_name": "paigepurrez", "lang": "en", "location": "null", "create_at": date("2015-07-27"), "description": "just me and my little plant friends. still a registered mouth-breather", "followers_count": 118, "friends_count": 137, "statues_count": 1215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694959944044544, "text": "let's break the surface and make a little happiness", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2745720803, "name": ".", "screen_name": "alondrahurr", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "null", "followers_count": 346, "friends_count": 408, "statues_count": 7806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-09T00:00:05.000Z"), "id": 718694960116146176, "text": "I love driving through Atlanta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351618806, "name": "Dani", "screen_name": "DaniHughes2", "lang": "en", "location": "null", "create_at": date("2011-08-09"), "description": "What doesnt kill you only makes you blonder ⠀✟senior @ ohs", "followers_count": 480, "friends_count": 431, "statues_count": 13513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057323486875648, "text": "Wind 10.0 mph S. Barometer 1016.43 mb, Falling. Temperature 55.9 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 13748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057323692412930, "text": "@chiefpoloo @MrRedMartian mainly bc the nails and fingers have a lot of germs... The vagina is a sensitive canal. I like 2 be clean", "in_reply_to_status": 719054765817249792, "in_reply_to_user": 2728457183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728457183, 78222718 }}, "user": { "id": 453136295, "name": "triippy S K I P P Y", "screen_name": "highlyxsedatedx", "lang": "en", "location": "Houston, TX.", "create_at": date("2012-01-02"), "description": "'i fancy pot' ❤ // #goodvibetribe #DirtySouth #GEAUXSAINTS", "followers_count": 1431, "friends_count": 2252, "statues_count": 21470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057323772088320, "text": "@ladylaceup42 nah bruh you have pipes don't play ya self", "in_reply_to_status": 719056915720839168, "in_reply_to_user": 1518223466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1518223466 }}, "user": { "id": 29938394, "name": "YA STEP FAVA", "screen_name": "FUCCMYKE", "lang": "en", "location": "Cali ☁", "create_at": date("2009-04-08"), "description": "▲ Ig: FUCCMYKE Snap: FUCCMYKE", "followers_count": 779, "friends_count": 615, "statues_count": 28053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057324086681600, "text": "\"Some new policeses\" �������������� https://t.co/unUI0VdlBj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357353279, "name": "Alondra T", "screen_name": "alondratobias", "lang": "en", "location": "Texas, USA", "create_at": date("2011-08-18"), "description": "null", "followers_count": 674, "friends_count": 470, "statues_count": 25861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057324392841216, "text": "@moniquemarvez Two hours of KFI Saturday date night w/Mo. Then, an hour of live streaming! How much mo' Mo can we take? A whole lotta Mo!", "in_reply_to_status": -1, "in_reply_to_user": 38699138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38699138 }}, "user": { "id": 2657250254, "name": "carrascopaula", "screen_name": "CarrascoPaula11", "lang": "en", "location": "null", "create_at": date("2014-07-18"), "description": "null", "followers_count": 14, "friends_count": 27, "statues_count": 671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057324594188289, "text": "You're a poor excuse for a \"grown man\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1977583226, "name": "Brudder Bear 2", "screen_name": "HBK_Casillas", "lang": "en", "location": "null", "create_at": date("2013-10-20"), "description": "Taylor Showers ❤️.", "followers_count": 463, "friends_count": 391, "statues_count": 7316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325059780608, "text": "i drank a half gallon of milk in less than 24 hours. am i 12 again ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347675525, "name": "OG Mudbownas", "screen_name": "johnissimba", "lang": "en", "location": "Denver, Callierado", "create_at": date("2011-08-02"), "description": "fav movie is rush hour 2\n#BernieSanders2016 #LostintheSauce", "followers_count": 345, "friends_count": 302, "statues_count": 9116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325068189696, "text": "@xChicagoo https://t.co/FdeSLK8tox", "in_reply_to_status": -1, "in_reply_to_user": 284285972, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 284285972 }}, "user": { "id": 952965408, "name": "kennedy", "screen_name": "kgraham2015", "lang": "en", "location": "Spring, TX", "create_at": date("2012-11-16"), "description": "sc; kennedygraham ig; kennedygraham7 • I'm Jorge's fav :D", "followers_count": 1163, "friends_count": 424, "statues_count": 39711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325353476096, "text": "all this pain in my heart but I still manage to smile ‼️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4325541934, "name": "געלט", "screen_name": "TCampaign__", "lang": "en", "location": "null", "create_at": date("2015-11-29"), "description": "null", "followers_count": 333, "friends_count": 184, "statues_count": 3127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walker, MI", "id": "c9d8828f259682c3", "name": "Walker", "place_type": "city", "bounding_box": rectangle("-85.787954,42.915342 -85.661793,43.059577") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2682960, "cityName": "Walker" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325726650368, "text": "Life is so precious, yet people still don't cherish it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 175893605, "name": "Joshua Rounds", "screen_name": "JROUNDS25", "lang": "en", "location": "New Orleans", "create_at": date("2010-08-07"), "description": "Tulane Football player #25", "followers_count": 561, "friends_count": 398, "statues_count": 9391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325735026689, "text": "@caroooposttt https://t.co/dq3ggOdPCr", "in_reply_to_status": 719056732899553283, "in_reply_to_user": 1262748680, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1262748680 }}, "user": { "id": 426233551, "name": "Aliyah Gonzalez", "screen_name": "aliyah_gonz23", "lang": "en", "location": "probably taking a nap ", "create_at": date("2011-12-01"), "description": "Arizona State | ΑΧΩ", "followers_count": 750, "friends_count": 325, "statues_count": 12261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325797953538, "text": "@Fatassah234 I WANT TO", "in_reply_to_status": 719057270437351424, "in_reply_to_user": 998686278, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 998686278 }}, "user": { "id": 62182028, "name": "Hannahconda", "screen_name": "Lady_Santillan_", "lang": "en", "location": "Glendale, AZ", "create_at": date("2009-08-01"), "description": "20, Single, Pisces...wait this isn't a dating app Snapchat: raeray16", "followers_count": 440, "friends_count": 385, "statues_count": 35230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057325965844480, "text": "No fast food week. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325896315, "name": "kodak flack❗️", "screen_name": "_Ciphelo", "lang": "en", "location": "Tha Jungle", "create_at": date("2011-06-28"), "description": "(704) R.I.P Wes Hillman Where I came from humbled me, Employed . Believer, Son, Brother, Student. North Carolina A&T 17' #TeamiPhone ✊ #Wolfpack #FlyHigh5", "followers_count": 1754, "friends_count": 1645, "statues_count": 65043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057326242668544, "text": "Triplets! nopityvan #RCTID #PTFC #timbersarmy #timbersaway #squad @ Long Beach, California https://t.co/Ce33fI9qzu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.158,33.8041"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RCTID", "PTFC", "timbersarmy", "timbersaway", "squad" }}, "user": { "id": 524930452, "name": "Nancy", "screen_name": "randomnotions", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "Soccer, Dogs, Travel, Books, Plants, whiskey. Oregonian running amok in Texas #RCTID #TimbersArmy #LSB", "followers_count": 114, "friends_count": 164, "statues_count": 1220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057326397911040, "text": "Wind 0.0 mph ---. Barometer 30.16 in, Steady. Temperature 55.4 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057326485934080, "text": "Ripley SW Limestone Co. Temp: 41.5°F Wind:2.2mph Pressure: 1001.5mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057326678884353, "text": "���� @lilimoose https://t.co/oaYhvF6gIw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1583841937 }}, "user": { "id": 20870924, "name": "Ang Ela", "screen_name": "poppet77", "lang": "en", "location": "null", "create_at": date("2009-02-14"), "description": "null", "followers_count": 131, "friends_count": 224, "statues_count": 2489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilson, PA", "id": "fb3e8933bc780f93", "name": "Wilson", "place_type": "city", "bounding_box": rectangle("-75.252443,40.664321 -75.231448,40.702235") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4285592, "cityName": "Wilson" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057327144464384, "text": "First ever Spring Dance Concert ✔️❣ @ Nate Holden Performing Arts Center https://t.co/7LPG1hSdTT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.34200619,34.03981776"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4771559065, "name": "Connor Miles Freeman", "screen_name": "connfree_", "lang": "en", "location": "null", "create_at": date("2016-01-16"), "description": "LACHSA 2019 | LADM | SSDC | Dancer", "followers_count": 58, "friends_count": 146, "statues_count": 127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:00.000Z"), "id": 719057327320670208, "text": "Temp 30.3° Hi/Lo 35.7/30.3 Rng 5.4° WC 30.3° Hmd 56% Rain 0.00\" Storm 0.00\" BAR 30.045 Rising DP 16.5° Wnd 3mph Dir N Gst 16mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 117, "statues_count": 17724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057327769436160, "text": "What I wish for my kids #ketchikan #Alaska #beach #quote #wisdom @ North Point Higgins Beach https://t.co/R1AVk6QVFQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-131.70986247,55.35680349"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ketchikan", "Alaska", "beach", "quote", "wisdom" }}, "user": { "id": 452562083, "name": "Bobbi Keuken", "screen_name": "mCklarken", "lang": "en", "location": "Ketchikan, AK", "create_at": date("2012-01-01"), "description": "... #wiscogirl ...", "followers_count": 75, "friends_count": 113, "statues_count": 730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2130, "countyName": "Ketchikan Gateway", "cityID": 238970, "cityName": "Ketchikan" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057327786106880, "text": "That song is still hard Wayne the goat for that https://t.co/KtauBcuc7t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1268105978, "name": "MinuteMan", "screen_name": "YoungAndWise83", "lang": "en", "location": "Los Angeles", "create_at": date("2013-03-14"), "description": "Don't save her she don't wanna be saved", "followers_count": 426, "friends_count": 331, "statues_count": 34752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057328255795200, "text": "Whisper words of wisdom let it be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1384674656, "name": "Haelee", "screen_name": "haeeeelee", "lang": "en", "location": "CA", "create_at": date("2013-04-27"), "description": "sc///haelee13 • Aries • what would hank moody do ?", "followers_count": 661, "friends_count": 553, "statues_count": 12916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon Lake, CA", "id": "85072b5a406cd0ff", "name": "Canyon Lake", "place_type": "city", "bounding_box": rectangle("-117.283988,33.670173 -117.234601,33.71489") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 610928, "cityName": "Canyon Lake" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057328557989888, "text": "04/10@03:00 - Temp 33.0F, WC 33.0F. Wind 1.8mph WSW, Gust 3.0mph. Bar 30.227in, Rising. Rain 0.01in. Hum 60%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057328960643072, "text": "Oh boy 3 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271338310, "name": "David Knight", "screen_name": "DavidKaNight", "lang": "en", "location": "Toledo, OH", "create_at": date("2014-01-01"), "description": "null", "followers_count": 468, "friends_count": 348, "statues_count": 6086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057329010814976, "text": "https://t.co/cLc1mYqC3K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3706327574, "name": "Alexander Manuel", "screen_name": "AlexManuel420", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-09-27"), "description": "null", "followers_count": 101, "friends_count": 539, "statues_count": 32 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057329061126144, "text": "@nytimes And most of the cool ones support the Jew from The Bronx, go figure.", "in_reply_to_status": 719055582448259072, "in_reply_to_user": 807095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 807095 }}, "user": { "id": 1158691333, "name": "Alex Arter", "screen_name": "arter_alex", "lang": "en", "location": "Lincoln, NE", "create_at": date("2013-02-07"), "description": "Husker fan, blue dot in a red state. Science is not up for debate — interpret facts, don't make up ur own.", "followers_count": 64, "friends_count": 262, "statues_count": 619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057329065299969, "text": "https://t.co/wQ4M2VStk3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 343681483, "name": "$krillla", "screen_name": "turkeyluvrr", "lang": "en", "location": "Nuketown,NV", "create_at": date("2011-07-27"), "description": "just a ghost being ghostly mostly", "followers_count": 331, "friends_count": 174, "statues_count": 41474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057329078038528, "text": "that's why I don't speak my feelings and let it be.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3240931874, "name": "mei❤️", "screen_name": "Makiyaa_B", "lang": "en", "location": " Detroit", "create_at": date("2015-06-09"), "description": "#RIPLON", "followers_count": 1639, "friends_count": 1488, "statues_count": 7815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wixom, MI", "id": "17a49f2ba5a82b26", "name": "Wixom", "place_type": "city", "bounding_box": rectangle("-83.559041,42.492847 -83.499279,42.563759") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2688140, "cityName": "Wixom" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057330558656513, "text": "@sprinklingdust I totally and utterly agree with you! #NCT", "in_reply_to_status": 719049537256468481, "in_reply_to_user": 101244645, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NCT" }}, "user_mentions": {{ 101244645 }}, "user": { "id": 266419431, "name": "Karina Rampal", "screen_name": "kapoop_panda", "lang": "en", "location": "null", "create_at": date("2011-03-14"), "description": "Don't let the mirror define you.", "followers_count": 64, "friends_count": 65, "statues_count": 2194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgewood, NJ", "id": "38718ccc91e76508", "name": "Ridgewood", "place_type": "city", "bounding_box": rectangle("-74.149096,40.955019 -74.077902,41.007882") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3463000, "cityName": "Ridgewood" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057330688491522, "text": "This bitch Kenya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 722614064, "name": "B.", "screen_name": "yufckinlovebre", "lang": "en", "location": "Trill, Texas", "create_at": date("2012-07-28"), "description": "LU. 22, Smartass.", "followers_count": 818, "friends_count": 350, "statues_count": 56869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Island, TX", "id": "00e8a59f3ce3ab5b", "name": "Pine Island", "place_type": "city", "bounding_box": rectangle("-96.037802,30.057846 -96.010235,30.082302") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4857615, "cityName": "Pine Island" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057330751545344, "text": "me\n\n╭┈┈┈┈╯   ╰┈┈┈╮\n\n ╰┳┳╯    ╰┳┳╯\n   ��       ��\n\n   ��       ��\n      ╰┈┈╯\n   ��╰━━━━━╯��\n     ┈┈┈┈\n   ��       ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1389662263, "name": "mami ✨", "screen_name": "camycicala", "lang": "en", "location": "theweeknd ♡", "create_at": date("2013-04-29"), "description": "unbothered.", "followers_count": 777, "friends_count": 906, "statues_count": 9877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Drexel Hill, PA", "id": "4eaabc683e109220", "name": "Drexel Hill", "place_type": "city", "bounding_box": rectangle("-75.331497,39.911076 -75.282044,39.964566") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4219920, "cityName": "Drexel Hill" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331053416448, "text": "In-N-Out RN����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1293191376, "name": "Daniela Martínez", "screen_name": "Daniiii_apple", "lang": "en", "location": "null", "create_at": date("2013-03-23"), "description": "null", "followers_count": 140, "friends_count": 170, "statues_count": 4539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331279896576, "text": "@ebbtideapp Tide in Stockton, California 04/10/2016\n Low 3:08am 0.7\nHigh 8:25am 4.4\n Low 4:18pm -0.3\nHigh 10:06pm 3.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-121.29,37.9583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 23183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331615444993, "text": "#goldilocks ���� https://t.co/8ZL2FJtiVG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "goldilocks" }}, "user": { "id": 1600930958, "name": "Adrian", "screen_name": "adrianzoescottt", "lang": "en", "location": "null", "create_at": date("2013-07-17"), "description": "null", "followers_count": 214, "friends_count": 341, "statues_count": 9061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331640799232, "text": "I'm cryin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384062464, "name": "kam", "screen_name": "Kam_uran", "lang": "en", "location": "dirty jersey", "create_at": date("2011-10-02"), "description": "18 / avid coffee enthusiast / us marine / cinephile / mv", "followers_count": 954, "friends_count": 195, "statues_count": 77744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland Park, NJ", "id": "77ed64eec363b46c", "name": "Highland Park", "place_type": "city", "bounding_box": rectangle("-74.446897,40.488049 -74.410916,40.514422") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3431470, "cityName": "Highland Park" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331686916097, "text": "2:00 a.m ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 327206952, "name": "Dee ✨❤️", "screen_name": "__swaggydee__", "lang": "en", "location": "Richmond, LA", "create_at": date("2011-06-30"), "description": "#LakerNation. R.I.P Madea & Uncle Jimmy!", "followers_count": 1482, "friends_count": 1461, "statues_count": 57914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vicksburg, MS", "id": "bdaa181492bdd711", "name": "Vicksburg", "place_type": "city", "bounding_box": rectangle("-90.92885,32.223388 -90.821771,32.392365") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28149, "countyName": "Warren", "cityID": 2876720, "cityName": "Vicksburg" } }
+{ "create_at": datetime("2016-04-10T00:00:01.000Z"), "id": 719057331716235264, "text": "Wind 4.0 mph ESE. Barometer 30.025 in, Falling Rapidly. Temperature 44.5 °F. Rain today 0.00 in. Humidity 52%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057332026609665, "text": "@whcarmichael \n#BurnDownTheGOP", "in_reply_to_status": 719057099263750145, "in_reply_to_user": 3201104837, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "BurnDownTheGOP" }}, "user_mentions": {{ 3201104837 }}, "user": { "id": 554672034, "name": "♦sally♦sez♦", "screen_name": "ClownGroovy", "lang": "en", "location": "NYC & Bremerton & TexasOrBust", "create_at": date("2012-04-15"), "description": "♦♦Harlequin♦Quipster♦♦\n#Descendant #FramerSherman Signed #DECLARATION_ARTICLES ElegantAgression w/ VerbalBrutality #TSoulTrain #Peace #ΜΟΛΩΝ_ΛΑΒΕ #WarriorPoet ♒", "followers_count": 1954, "friends_count": 3072, "statues_count": 16370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057332341059584, "text": "Registered Nurse | RN |... - Favorite Health Care Staffing: (#Springfield, MO) https://t.co/1PUntaedIU #Nursing #RN https://t.co/1BcPmYY4Ig", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.2922989,37.2089572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Springfield", "Nursing", "RN" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 188, "friends_count": 99, "statues_count": 3779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri, USA", "id": "2526edd24c06e60c", "name": "Missouri", "place_type": "admin", "bounding_box": rectangle("-95.774704,35.995476 -89.098843,40.613641") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29077, "countyName": "Greene", "cityID": 2970000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057332412497920, "text": "Temp: 60.4°F Wind:0.0mph Pressure: 30.151hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 60797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057332534185984, "text": "I'm so tipsy right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251901767, "name": "✨-LeaveItToWeaver-✨", "screen_name": "_IAmLamar_", "lang": "en", "location": "Southern Raised Mississippian ", "create_at": date("2011-02-13"), "description": "null", "followers_count": 957, "friends_count": 888, "statues_count": 17684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057332706144256, "text": "Wind 0.0 mph SSE. Barometer 30.257 in, Falling slowly. Temperature 31.0 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 18986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333024866304, "text": "it's a boy and a girl thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3252538168, "name": "Elaina Carson", "screen_name": "ecarsonn", "lang": "en", "location": "neverland", "create_at": date("2015-05-13"), "description": "21, I like pizza, ♋️", "followers_count": 371, "friends_count": 162, "statues_count": 15053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doctor Phillips, FL", "id": "3b05a9986be2419b", "name": "Doctor Phillips", "place_type": "city", "bounding_box": rectangle("-81.508635,28.420712 -81.474987,28.493573") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1217725, "cityName": "Doctor Phillips" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333058281472, "text": "Freshmen year of college almost over and it has been greatttt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2785034772, "name": "tylahwylah ❤️", "screen_name": "MoorerTyler", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "18. blessed. just living. 205✈️601 #JSU19 #JacksonStateVolleyball #jsuhonorscollege", "followers_count": 1539, "friends_count": 1002, "statues_count": 24768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333096026112, "text": "@lazyavery exactly!", "in_reply_to_status": 719057249470025729, "in_reply_to_user": 3196243986, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3196243986 }}, "user": { "id": 1672051572, "name": "mother laura", "screen_name": "hatelaura_", "lang": "en", "location": "1998", "create_at": date("2013-08-14"), "description": "sugar spice n everything nice", "followers_count": 301, "friends_count": 319, "statues_count": 9356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flower Mound, TX", "id": "d6bbfe2356aebaab", "name": "Flower Mound", "place_type": "city", "bounding_box": rectangle("-97.178734,32.981704 -97.027543,33.078462") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4826232, "cityName": "Flower Mound" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333100408832, "text": "@JusAllahMMA What's good?", "in_reply_to_status": 719055993599107072, "in_reply_to_user": 26096020, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 102271499 }}, "user": { "id": 26096020, "name": "L.", "screen_name": "MeanestManAlive", "lang": "en", "location": "Boston, MA", "create_at": date("2009-03-23"), "description": "Facing a Life Sentence for committing ThoughtCrime.", "followers_count": 360, "friends_count": 599, "statues_count": 8555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333163167744, "text": "Doing squats have done me ALOT of justice!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2919974558, "name": "ღ", "screen_name": "queeenmorocco", "lang": "en", "location": "Stockton, CA", "create_at": date("2014-12-05"), "description": "don't be bitter, be better❥", "followers_count": 201, "friends_count": 180, "statues_count": 3108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333544878081, "text": "Joey thought he did a good job on my hair!!���� https://t.co/B9JierYfcV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1917903384, "name": "Alina Daelyn Ashe", "screen_name": "alinadaelyn12", "lang": "en", "location": "Naselle ", "create_at": date("2013-09-29"), "description": "always pissed, or pissing people off", "followers_count": 153, "friends_count": 149, "statues_count": 2016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333892952065, "text": "Starting 2 get over that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429721013, "name": "criss.", "screen_name": "SsircBitch", "lang": "en", "location": "Harbor City, Cå", "create_at": date("2011-12-06"), "description": "20.", "followers_count": 258, "friends_count": 230, "statues_count": 13097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057333989613569, "text": "@everytwitpic haha this is weird but I'm into it.", "in_reply_to_status": -1, "in_reply_to_user": 2780249319, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2780249319 }}, "user": { "id": 23077552, "name": "Logan", "screen_name": "dirtynewyorkr", "lang": "en", "location": "ÜT: 40.649434,-74.077438", "create_at": date("2009-03-06"), "description": "All City. @JungleGymMag Co-founder/Editor//@dollar_bin (Manager) Pre-order The King of Styles EP http://apple.co/1T4fcsP | SC + IG dirtynewyorkr", "followers_count": 1579, "friends_count": 671, "statues_count": 73355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057334366900224, "text": "59.8F (Feels: 59.8F) - Humidity: 82% - Wind: 1.6mph E - Gust: 5.4mph - Pressure: 1015.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 234509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057335096909824, "text": "Wind 3.0 mph NW. Barometer 30.138 in, Rising Rapidly. Temperature 23.0 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057335126130688, "text": "I'm LK hella defiant", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1733332274, "name": "CAPRICE", "screen_name": "capricekiyomi", "lang": "en", "location": "Hayward CA,OaklandCA", "create_at": date("2013-09-05"), "description": "Just get me out the house and we coo lol.. Virgo EHS, hooper,c/o 2016 SC: kikicaprice1", "followers_count": 429, "friends_count": 338, "statues_count": 1593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057335243640833, "text": "Wind 0.0 mph SW. Barometer 30.14 in, Rising. Temperature 28.9 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 158948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057335532924928, "text": "I'm not allowed to talk to my big starting at midnight and I'm sad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344588043, "name": "V ❃", "screen_name": "vanessdiaaz", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-07-29"), "description": "SF State • ΦΣΣ ♡", "followers_count": 1090, "friends_count": 723, "statues_count": 32127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-10T00:00:02.000Z"), "id": 719057335755210752, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 295421937, "name": "⭐~De'Von~⭐™", "screen_name": "YungSav94", "lang": "en", "location": "Cuba, AL", "create_at": date("2011-05-08"), "description": "2⃣0⃣5⃣⬅AAMU - SAINTS ~WHO DAT!!!! ROLL TIDE #95. MAVS #41 Strive for greatness. SUMTER COUNTY ✈️ HSV", "followers_count": 1797, "friends_count": 1957, "statues_count": 15215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336023851008, "text": "\"why do i work so much??\"\n*gets paid*\noh, right", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410445325, "name": "trophy", "screen_name": "verytiredboy", "lang": "en", "location": "phl", "create_at": date("2011-11-11"), "description": "activist/dead inside", "followers_count": 1525, "friends_count": 482, "statues_count": 38936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336455659522, "text": "@Ticobell And Kroger brand mint oreo knockoffs. That's my jam.", "in_reply_to_status": 719056952723054592, "in_reply_to_user": 86005005, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 86005005 }}, "user": { "id": 33729317, "name": "CA Blocke", "screen_name": "afullmargin", "lang": "en", "location": "Arizona", "create_at": date("2009-04-20"), "description": "Fangirl, parent, writer, and friend. Queer-identified gender non-conforming female. I swear, ship, ramble, write smut, and verbify nouns. You've been warned.", "followers_count": 259, "friends_count": 331, "statues_count": 68336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336828952576, "text": "Hanging out with that little girl from New York tonight.\n\nI don't think I like her boyfriend. This is probably going to end poorly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542119755, "name": "brushhhhhh", "screen_name": "Razzable", "lang": "en", "location": "Portland, OR", "create_at": date("2012-03-31"), "description": "Moderately Informed : Unreasonably Optimistic", "followers_count": 106, "friends_count": 111, "statues_count": 16630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336896081920, "text": "@dankmomma i blame u for it being 2am!", "in_reply_to_status": 719054067431899136, "in_reply_to_user": 1864668296, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1864668296 }}, "user": { "id": 23161285, "name": "Young Wikid", "screen_name": "YoungWikid", "lang": "en", "location": "Dallas, Tx Shreveport", "create_at": date("2009-03-06"), "description": "Artist 1st ,#MFFL 2ND https://soundcloud.com/young-wikid", "followers_count": 511, "friends_count": 440, "statues_count": 24673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336917106688, "text": "Poor baby ☹️ https://t.co/jwohG1CBAd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 839231436, "name": "♛", "screen_name": "autumndawndiaz", "lang": "en", "location": "somewhere near seattle", "create_at": date("2012-09-21"), "description": "money talks", "followers_count": 296, "friends_count": 96, "statues_count": 24809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057336967499776, "text": "anti-Assad vluchtelingen zelfde geloofsgroep die Alawieten vervolgde tot Assad daar in 1970 een eind aan maakte. Welkom vluchtelingen", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 254078352, "name": "BillSmith", "screen_name": "melanchton1", "lang": "en", "location": "Bonaire, Sint Eustatius and Sa", "create_at": date("2011-02-18"), "description": "Realist, Socio-libertijn, maar géén socialist, Atheist (vanzelfsprekend). Rechts noemt my links, Links noemt mij ultra rechtse reactionair. Vader van 2 kinderen", "followers_count": 455, "friends_count": 189, "statues_count": 13053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Princeton, NJ", "id": "c833fbabba6fe48e", "name": "Princeton", "place_type": "city", "bounding_box": rectangle("-74.710297,40.32495 -74.619759,40.37435") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3460900, "cityName": "Princeton" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337072427008, "text": "@LaurenRenschler my heart goes out to the gentle giant Will god bless his soul in heaven", "in_reply_to_status": -1, "in_reply_to_user": 16563690, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16563690 }}, "user": { "id": 81469847, "name": "Kevin Davis", "screen_name": "Karceno", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-10-10"), "description": "VP of Operations & Co-Owner of http://BoxingSocialist.com/ Business Connoisseur/Sports & Film Savant/Gentleman of Lesuire", "followers_count": 3170, "friends_count": 778, "statues_count": 50742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Lake, IL", "id": "014cdf2b0d6404fa", "name": "Round Lake", "place_type": "city", "bounding_box": rectangle("-88.140904,42.309615 -88.082053,42.368764") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1766027, "cityName": "Round Lake" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337093193729, "text": "Bout to delete my Twitter again soon ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.891485,32.995337"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326027612, "name": "☔️", "screen_name": "HoneyysLove", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-29"), "description": "null", "followers_count": 1843, "friends_count": 27, "statues_count": 197176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337265205248, "text": "@VanessaHudgens fuck it up , yasssss !!!!", "in_reply_to_status": 718965580519383040, "in_reply_to_user": 274119641, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 274119641 }}, "user": { "id": 220900763, "name": "anthony", "screen_name": "ObeyLunatic", "lang": "en", "location": "1503 :.", "create_at": date("2010-11-28"), "description": "21 | ♑️ , 4x World Dance Champion . i love to rave , what can i say", "followers_count": 2531, "friends_count": 1433, "statues_count": 44720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337533599745, "text": "Breakfast an chill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 817718653, "name": "captain jake", "screen_name": "jake_ledesma", "lang": "en", "location": "San Benito, TX", "create_at": date("2012-09-11"), "description": "Fuck yes, im so odd...yamum......uhhhhh leader of the sasquatch gang...oh oh and blunts!", "followers_count": 638, "friends_count": 380, "statues_count": 23842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Benito, TX", "id": "b8f3a7fd432ec1f1", "name": "San Benito", "place_type": "city", "bounding_box": rectangle("-97.684281,26.07277 -97.575927,26.172319") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4865036, "cityName": "San Benito" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337814761472, "text": "@FADhanoa was sent to me at 10 pm", "in_reply_to_status": 719057174878691328, "in_reply_to_user": 101703533, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101703533 }}, "user": { "id": 174867187, "name": "maige pac", "screen_name": "pagemasterpaige", "lang": "en", "location": "null", "create_at": date("2010-08-04"), "description": "Fuhgeddaboudit.", "followers_count": 145, "friends_count": 455, "statues_count": 7805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massapequa Park, NY", "id": "c03748ea7a3b06c0", "name": "Massapequa Park", "place_type": "city", "bounding_box": rectangle("-73.464521,40.652099 -73.436512,40.705188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3645997, "cityName": "Massapequa Park" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337881735168, "text": "So good to be home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187334663, "name": "K.B.", "screen_name": "karlee364", "lang": "en", "location": "null", "create_at": date("2013-11-10"), "description": "null", "followers_count": 387, "friends_count": 208, "statues_count": 745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Payson, UT", "id": "3a9c54ff2723b0b2", "name": "Payson", "place_type": "city", "bounding_box": rectangle("-111.768548,40.007774 -111.684985,40.058035") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4958730, "cityName": "Payson" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057337965613057, "text": "@_jaazlynn Tf up", "in_reply_to_status": 719057272387690496, "in_reply_to_user": 2935525712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2935525712 }}, "user": { "id": 2944751256, "name": "andree", "screen_name": "chico_8o5", "lang": "en", "location": "null", "create_at": date("2014-12-27"), "description": "null", "followers_count": 224, "friends_count": 274, "statues_count": 1509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338015977472, "text": "@ginacaliente https://t.co/GboD2iET8c", "in_reply_to_status": 719057059149262849, "in_reply_to_user": 267017403, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 267017403 }}, "user": { "id": 620783209, "name": "Bae Doona", "screen_name": "kazanderr", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-06-28"), "description": "Straight Edge One Direction fan.", "followers_count": 271, "friends_count": 221, "statues_count": 11490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338037059584, "text": "i have work at 6:30 i am so upset with myself right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311074532, "name": "stocrates", "screen_name": "stocktoncobb", "lang": "en", "location": "austin, texas", "create_at": date("2011-06-04"), "description": "my dad calls me rockton because i rock!", "followers_count": 1377, "friends_count": 1334, "statues_count": 50768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338125152256, "text": "Dyckman is where it's at https://t.co/XbnPPWGibC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1398651938, "name": "Murcielago", "screen_name": "_YouLoveYamile", "lang": "en", "location": "null", "create_at": date("2013-05-02"), "description": "Some girls are born with glitter in their veins", "followers_count": 281, "friends_count": 223, "statues_count": 10002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338515202048, "text": "@Smileyeli0121 I feel you ��", "in_reply_to_status": -1, "in_reply_to_user": 1578800047, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1578800047 }}, "user": { "id": 2510673090, "name": "shabba", "screen_name": "partyoutsiide", "lang": "en", "location": "Washington, DC", "create_at": date("2014-05-20"), "description": "null", "followers_count": 3997, "friends_count": 1931, "statues_count": 62484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338536058881, "text": "@benlandis Surely. Like everyone that's taken Dilaudid in the hospital and never wanted to take it afterwards.", "in_reply_to_status": 719053749142953986, "in_reply_to_user": 37599351, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 37599351 }}, "user": { "id": 5103, "name": "Noël Jackson", "screen_name": "noel", "lang": "en", "location": "everywhere", "create_at": date("2006-08-31"), "description": "The answer is to be the answer. n@noeljackson.com / @weareexquisite / @hypertonemusic", "followers_count": 3000, "friends_count": 1218, "statues_count": 9093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338582179842, "text": "Stop trying to be a fucking motivational speaker ����������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 337997365, "name": "El Chapo Jr", "screen_name": "ChaseCadena", "lang": "en", "location": "Texa$", "create_at": date("2011-07-18"), "description": "Justice League , Unite! Never Lackin' On God. Ignorant . Certified Martian . Stupid . Yung Savage", "followers_count": 801, "friends_count": 666, "statues_count": 15327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur Springs, TX", "id": "17b1345969dabc8d", "name": "Sulphur Springs", "place_type": "city", "bounding_box": rectangle("-95.65806,33.091542 -95.556096,33.158547") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48223, "countyName": "Hopkins", "cityID": 4870904, "cityName": "Sulphur Springs" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338733187076, "text": "��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.50542822,38.58255279"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 359286894, "name": "Sway", "screen_name": "josalv3", "lang": "en", "location": "Global.", "create_at": date("2011-08-21"), "description": "Resource finder.\r\nENFJ.", "followers_count": 569, "friends_count": 44, "statues_count": 7419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338770935809, "text": "The bartender just said \"hey Bree, I'm surprised I haven't seen you in awhile.\" Lol same", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331663325, "name": "Bree Jacoba", "screen_name": "BreeJacoba", "lang": "en", "location": "University of Iowa", "create_at": date("2011-07-08"), "description": "† Philippians 4:13", "followers_count": 1250, "friends_count": 754, "statues_count": 26149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338817097731, "text": "Ima catch a body if i hear that bullshit ass nigga put his hands on ny bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236208018, "name": "lEO(POLANSKI) suavé", "screen_name": "_Brooo_", "lang": "en", "location": "Like my #BLOG (link below) ", "create_at": date("2011-01-09"), "description": "18 . 03131997\nClass of '15 \nCertified CARE GIVER for\nAccent Care Inc. .\n#iFollowBack \n#BADBOY\n#TeamREAL", "followers_count": 824, "friends_count": 878, "statues_count": 36257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas City, TX", "id": "05959d7d3c4d4c27", "name": "Texas City", "place_type": "city", "bounding_box": rectangle("-95.049499,29.33728 -94.886484,29.422811") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4872392, "cityName": "Texas City" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057338896744448, "text": "I may be making a Rumlow brother. THE POLL KNOWS.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 821028307, "name": "Viper Frost.", "screen_name": "EverTheViper", "lang": "en", "location": "G e o n o s i s.", "create_at": date("2012-09-12"), "description": "Supreme Bitch King. Sweet Moon Child. One Nice Motherfucker. Till the end of the line, @VivaLaGloriaV? 「#Zishy #Vicon #Elviper #Lyper」 #Asp #FrostBites", "followers_count": 1777, "friends_count": 939, "statues_count": 96495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057339249065984, "text": "Im hanging up on u rn �������� https://t.co/YHkm1fIjXE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3055298058, "name": "Amari Ransom", "screen_name": "AmariAr22", "lang": "en", "location": "Seaching For My Dreams", "create_at": date("2015-03-01"), "description": "Dtf or nah follow @cuddiejen", "followers_count": 125, "friends_count": 167, "statues_count": 127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057339320569856, "text": "@AskAboutNookie ima slap u if u did wat i think u did ��", "in_reply_to_status": 719055018058457089, "in_reply_to_user": 313739329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313739329 }}, "user": { "id": 2963345248, "name": "that nigga.", "screen_name": "haughtyd", "lang": "en", "location": "Atlanta, GA", "create_at": date("2015-01-05"), "description": "Instagram 1haughty.d", "followers_count": 409, "friends_count": 134, "statues_count": 286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057339328831488, "text": "Free drinks. Free entry to the club.... Life is grand at the moment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47802732, "name": "angelica", "screen_name": "xgels", "lang": "en", "location": "Palm Springs ", "create_at": date("2009-06-16"), "description": "We just tryna get even that's why we at odds..", "followers_count": 555, "friends_count": 538, "statues_count": 30222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-10T00:00:03.000Z"), "id": 719057339874054144, "text": "Brant came over to watch a movie and basically slept thru it all �� @brant_mortonnnn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1959522104 }}, "user": { "id": 1156075218, "name": "alia", "screen_name": "_alia_13", "lang": "en", "location": "qc", "create_at": date("2013-02-06"), "description": "qchs varsity cheer @brant_mortonnnn", "followers_count": 813, "friends_count": 762, "statues_count": 8218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057340671008768, "text": "Much to learn you still have", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165543989, "name": "so harkless", "screen_name": "thomas_ikehara", "lang": "en", "location": "#ReyHive", "create_at": date("2010-07-11"), "description": "i'm just a gangsta i suppose", "followers_count": 494, "friends_count": 979, "statues_count": 72656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341086183424, "text": "smash obvi https://t.co/QV3RLewDTi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2495346452, "name": "soph♡♡", "screen_name": "sophieebobal", "lang": "en", "location": "legacy volleyball ", "create_at": date("2014-05-14"), "description": "wrhs & legacy volleyball #21 @jackmcnamara28 @anna_kyker @mayaafam", "followers_count": 441, "friends_count": 293, "statues_count": 2104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341098758144, "text": "I only have 5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 952631636, "name": "Armando", "screen_name": "josebeltran68", "lang": "en", "location": "San Juan, TX", "create_at": date("2012-11-16"), "description": "I'm going wherever the money takes me.", "followers_count": 485, "friends_count": 215, "statues_count": 16119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Juan, TX", "id": "6c253d30b4b6dbd4", "name": "San Juan", "place_type": "city", "bounding_box": rectangle("-98.17347,26.153868 -98.11375,26.244297") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4865532, "cityName": "San Juan" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341404983296, "text": "\"I'm not racist, I just hate everybody equally", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1901725148, "name": "KING", "screen_name": "KMFS92", "lang": "en", "location": "Ames • Cedar Rapids • IC ", "create_at": date("2013-09-24"), "description": "Shub is my spirit animal •• Snapchat: Keiloferd ••", "followers_count": 233, "friends_count": 94, "statues_count": 5640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341476278272, "text": "@_princesspuno5 basically", "in_reply_to_status": 719057096998649857, "in_reply_to_user": 1455532286, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1455532286 }}, "user": { "id": 2608174975, "name": "liiiiiinz", "screen_name": "liggitylinz", "lang": "en", "location": "USD '20", "create_at": date("2014-07-06"), "description": "gainz city / kinda cute", "followers_count": 981, "friends_count": 667, "statues_count": 14455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341484638208, "text": "@celriflo il a neigé ?", "in_reply_to_status": 719057223545200640, "in_reply_to_user": 236456726, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 236456726 }}, "user": { "id": 337614706, "name": "Léo", "screen_name": "1L30", "lang": "fr", "location": "Montpellier/Denver(CO)", "create_at": date("2011-07-18"), "description": "Passionné d’info de 20 ans\nMême que je code ! (Swift, C, C++...)\n\nCréateur de @lecafedugeek\n\nGlandeur et Gamer à temps perdu \n\nVivre pour manger\n\nTek 2018 !", "followers_count": 425, "friends_count": 80, "statues_count": 42681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341669380096, "text": "@AbbyGailMeyer as well", "in_reply_to_status": 719057219531227136, "in_reply_to_user": 178589944, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178589944 }}, "user": { "id": 506554169, "name": "Kade", "screen_name": "kbrotsky27", "lang": "en", "location": "null", "create_at": date("2012-02-27"), "description": "null", "followers_count": 272, "friends_count": 292, "statues_count": 1930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pierre, SD", "id": "3f42527ec99e95d1", "name": "Pierre", "place_type": "city", "bounding_box": rectangle("-100.371969,44.341218 -100.269096,44.396649") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46065, "countyName": "Hughes", "cityID": 4649600, "cityName": "Pierre" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341950386176, "text": "*prepares to fall asleep in makeup*\n*continues to wonder why my skin breaks out a little more every morning*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232271967, "name": "Chef D.", "screen_name": "_dasiavuu", "lang": "en", "location": "the 215", "create_at": date("2010-12-30"), "description": "it's ALL happening || 1cor 13:4-8 || temple18?\nsc: dasiaaavuu || 7515", "followers_count": 741, "friends_count": 589, "statues_count": 44620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057341979623424, "text": "Wind 9.6 mph SSE. Barometer 29.954 in, Falling slowly. Temperature 59.9 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057342344667136, "text": "When u lit but the fish bitin https://t.co/T5qIVNaWiS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069704794, "name": "camron", "screen_name": "fuckyouimcamron", "lang": "en", "location": "pacific beach, ca", "create_at": date("2013-01-07"), "description": "i am the liquor", "followers_count": 2951, "friends_count": 197, "statues_count": 53918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057342386417668, "text": "I like giving Otis a little voice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497450324, "name": "tonyboi", "screen_name": "Eating_Stars", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "@dispirttx SoundCloudy: Eating Stars. Blood Type: O", "followers_count": 504, "friends_count": 584, "statues_count": 54730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057342810034176, "text": "@ortizxsam HAPPY 16 TO THIS UGLY PRETTY GIRL��������ily!! her snaps always bomb too. https://t.co/jQdrr4fNv1", "in_reply_to_status": -1, "in_reply_to_user": 2764050783, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2764050783 }}, "user": { "id": 1242642787, "name": "pau", "screen_name": "paulinaflopez", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "@arielmiranda_", "followers_count": 924, "friends_count": 869, "statues_count": 4572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057342952812544, "text": "#Spaceage https://t.co/OyAYQnNnKv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Spaceage" }}, "user": { "id": 2745997576, "name": "Phillip Smith", "screen_name": "phillipsmith836", "lang": "en", "location": "erlanger Ky", "create_at": date("2014-08-17"), "description": "null", "followers_count": 22, "friends_count": 87, "statues_count": 119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, KY", "id": "f2ac7339a6059e50", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-84.413523,38.920717 -84.3378,38.997124") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21037, "countyName": "Campbell", "cityID": 2100802, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057342977826816, "text": "I fuckn hate you churro https://t.co/xdIDftt4SD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 750758544, "name": "LIT LI$$", "screen_name": "alissa_vizcaino", "lang": "en", "location": "null", "create_at": date("2012-08-10"), "description": "null", "followers_count": 630, "friends_count": 417, "statues_count": 13163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057343040851968, "text": "#Trump in 2016 Temp:29.5°F Wind:2.7mph Pressure: 30.18hpa Falling Rain Today 0.00in. Forecast: Unsettled, precipitation later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057343711842304, "text": "#u know how WE do it #always DODGERS DOYERS & Blue������HEAVEN W VIN! #so please give it up4 Vince right Magic!!?? https://t.co/cGnlBVPn9r", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "u", "always", "so" }}, "user": { "id": 2735996547, "name": "Manuel From Gardena", "screen_name": "manuelingardena", "lang": "en", "location": "Gardena,CA", "create_at": date("2014-08-07"), "description": "Mohrrior4Life,YoungMohrriors4life,Proud Father2Lil Manuel, Analisa, Raider Joe, IN JAY MOHRS HOF, FoxSportsRadioAddict, 70's&80's, Worship God,Jesus,HolySpirit", "followers_count": 1432, "friends_count": 1973, "statues_count": 13726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-10T00:00:04.000Z"), "id": 719057343724564480, "text": "It's 3 AM, I must be lonely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342478345, "name": "Sharon Blackstone", "screen_name": "SharonnBrooke", "lang": "en", "location": "Barnesville, Ohio", "create_at": date("2011-07-25"), "description": "West Virginia University '19 ❁", "followers_count": 284, "friends_count": 166, "statues_count": 9537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419711373463552, "text": "Trust https://t.co/mMzMwIJ17i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2877768962, "name": "♐Mani Tsunami♐", "screen_name": "youngmani98", "lang": "en", "location": "check the clouds", "create_at": date("2014-10-25"), "description": "♻", "followers_count": 360, "friends_count": 211, "statues_count": 26590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419711377682432, "text": "Find Chelsea lmao @godlychill @missnicoleuzo https://t.co/OoxWdPh4fT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1295487698, 618432256 }}, "user": { "id": 195589801, "name": "Jerry LockDown Mbu", "screen_name": "Respect_TheHook", "lang": "en", "location": "Richmond Tx", "create_at": date("2010-09-26"), "description": "*G.O.A.T* *Texas Dark Knight SemiPro Football* *Pro Photographer* *Strive 4 Greatness* *Philippians 4-13* *D.B.U* HTX", "followers_count": 931, "friends_count": 577, "statues_count": 35261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419711595761664, "text": "@chribitchhh https://t.co/gbt3qQjMJa", "in_reply_to_status": -1, "in_reply_to_user": 633943349, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 633943349 }}, "user": { "id": 524907570, "name": "Shadyn", "screen_name": "Blacckness", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "trap flower", "followers_count": 575, "friends_count": 262, "statues_count": 28936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego Country Estates, CA", "id": "5b151cafa875f47c", "name": "San Diego Country Estates", "place_type": "city", "bounding_box": rectangle("-116.843644,32.991819 -116.72703,33.035831") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666004, "cityName": "San Diego Country Estates" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419711901999104, "text": "@NiccAtNightt THENNNN eat pasta lol", "in_reply_to_status": 719419514098569216, "in_reply_to_user": 3796972333, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3796972333 }}, "user": { "id": 75325502, "name": "amber ✨", "screen_name": "prettydolledup", "lang": "en", "location": "IG: theprettydolledup", "create_at": date("2009-09-18"), "description": "null", "followers_count": 1156, "friends_count": 388, "statues_count": 34818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419712023605252, "text": "@JillComptn hurry back", "in_reply_to_status": 719416357775745024, "in_reply_to_user": 4427648232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4427648232 }}, "user": { "id": 931364815, "name": "N.A.P", "screen_name": "patergator40", "lang": "en", "location": "Fresno Pacific University ", "create_at": date("2012-11-06"), "description": "Baseball is Life ⚾ #TeamMelanie #FPU #Sunbirds", "followers_count": 443, "friends_count": 739, "statues_count": 1755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419712224911361, "text": "This is soo freaking cute ������ https://t.co/3uR8IEh2lM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2975240070, "name": "Jadamae✨.", "screen_name": "Jayyybabiee__", "lang": "en", "location": "null", "create_at": date("2015-01-11"), "description": "Sc: @jatonya4 freeedaddy , Jake & Choc 4️⃣ !", "followers_count": 1369, "friends_count": 1094, "statues_count": 15193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419712275243008, "text": "damn all these sad and creepy things are happening:( I just wanna go home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732335654, "name": "b r i a n a .", "screen_name": "brianaamariee_", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "sanmarcos,tx", "followers_count": 1194, "friends_count": 588, "statues_count": 14126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419712384290816, "text": "Wind 3.8 mph WSW. Barometer 29.68 in, Rising slowly. Temperature 33.8 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 41399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419712812101633, "text": "�������������� https://t.co/tGxZzY1vEN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 315588375, "name": "Jaylon Jones™♊️", "screen_name": "_cametoofar", "lang": "en", "location": "Houston ", "create_at": date("2011-06-11"), "description": "It Is What It Is #TJC", "followers_count": 782, "friends_count": 1371, "statues_count": 33155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419713223139328, "text": "Smash or pass? Why are you guys trying to make Twitter into Facebook smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312697050, "name": "Alex", "screen_name": "alexavila555", "lang": "en", "location": "null", "create_at": date("2015-08-11"), "description": "@kiimmmye's peasant❤️", "followers_count": 192, "friends_count": 172, "statues_count": 2141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419713646948352, "text": "All the Entities involved have put in Incarnations as Human Flesh & understand the differences between Species & recognized Issues.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 82078715, "name": "Miki Terry", "screen_name": "silentsound49", "lang": "en", "location": "Waxhaw,NC,USA", "create_at": date("2009-10-13"), "description": "http://t.co/kO32aRjInn Silent(subliminal)Sound of 17khz - Speaker Adapters - perps at 7010C Fernwood Dr 28211 & Waxhaw NC Stanton Dr Sharon Dr Waterbell area", "followers_count": 675, "friends_count": 1162, "statues_count": 41121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419713948925952, "text": "Ripley SW Limestone Co. Temp: 59.2°F Wind:7.6mph Pressure: 999.1mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419714116710400, "text": "im just v ready to reunite with my fav on tuesday ok", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1145329543, "name": "brannan ◡̈", "screen_name": "_brannancarter", "lang": "en", "location": "LA", "create_at": date("2013-02-03"), "description": "album 1 track 10 // psalm 91:4 // https://twitter.com/torikelly/status/352296610597842946", "followers_count": 11814, "friends_count": 255, "statues_count": 179340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419714431115264, "text": "Females don't understand some niggas want love too ...every nigga ain't worried bout ya lace panties", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 502126895, "name": "wonder why...", "screen_name": "_KodakFashion", "lang": "en", "location": "Nawfside (HTX)", "create_at": date("2012-02-24"), "description": "#LongLiveBankRoll", "followers_count": 3257, "friends_count": 2895, "statues_count": 59782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419714594668544, "text": "@UltraGrobanite Alrighty then.", "in_reply_to_status": 719418512830279680, "in_reply_to_user": 1215978740, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1215978740 }}, "user": { "id": 1689920359, "name": "#TakeNoPrisoners", "screen_name": "NoPrisoners08", "lang": "en", "location": "The Great USA", "create_at": date("2013-08-21"), "description": "I believe in the cause, not the man. My opinion will change when facts present themselves. Devoutly American. My support of Trump is unwielding", "followers_count": 1601, "friends_count": 596, "statues_count": 30400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-11T00:00:00.000Z"), "id": 719419715165241344, "text": "@blitzflip yo flex in cool sound on the crew https://t.co/91DZvlWlFE���� drop today! advertising available! all music drops!����", "in_reply_to_status": -1, "in_reply_to_user": 1170513302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1170513302 }}, "user": { "id": 2314337120, "name": "Submit Be Heard", "screen_name": "DailyHipHopJamz", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-01-27"), "description": "A platform for all hip hop artists anywhere. Submit today ⤵️ #DHHJCREW", "followers_count": 5647, "friends_count": 1751, "statues_count": 214283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419715492257792, "text": "She's beautiful in her simple little way", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2557483446, "name": "♡Hannah Renae♡", "screen_name": "BramanHannah", "lang": "en", "location": "Carlinville, IL", "create_at": date("2014-06-09"), "description": "I'm a blue eyed girl from a small town. 21. Blackburn college. I'm classy and sassy.", "followers_count": 155, "friends_count": 621, "statues_count": 2921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlinville, IL", "id": "2e9ffc7a520f5d4e", "name": "Carlinville", "place_type": "city", "bounding_box": rectangle("-89.899886,39.263281 -89.858299,39.296632") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17117, "countyName": "Macoupin", "cityID": 1711202, "cityName": "Carlinville" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419715492397057, "text": "Like dumbass I always defend u but u got my brother high af tonight and let him get a tattoo omg so angry rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284228637, "name": "s h e l b y", "screen_name": "soupbitchbaddd", "lang": "en", "location": "rocktown", "create_at": date("2011-04-18"), "description": "love makes you forget the difference between right and wrong", "followers_count": 548, "friends_count": 348, "statues_count": 25169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Loves Park, IL", "id": "9bfddbd45776b2bc", "name": "Loves Park", "place_type": "city", "bounding_box": rectangle("-89.069204,42.293874 -88.964153,42.365885") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1745031, "cityName": "Loves Park" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419715647463424, "text": "@alec6lee https://t.co/LgEbtS66HP", "in_reply_to_status": -1, "in_reply_to_user": 1370721638, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1370721638 }}, "user": { "id": 461747578, "name": "John Savage", "screen_name": "john_savage11", "lang": "en", "location": "null", "create_at": date("2012-01-11"), "description": "Wear your halo like a hat, that's like the latest fashion I got angels all around me they keep me surrounded", "followers_count": 169, "friends_count": 192, "statues_count": 282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419716167577601, "text": "This week calls for ����&�� @JacklynLovee ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 166405424 }}, "user": { "id": 186650185, "name": "G A R C I A", "screen_name": "gennnxo", "lang": "en", "location": "559/408", "create_at": date("2010-09-03"), "description": "SJSU", "followers_count": 1694, "friends_count": 781, "statues_count": 71891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419716368867329, "text": "My face be so oily now ugh!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155804881, "name": "Boss Lady", "screen_name": "N0W0rries06", "lang": "en", "location": "Jonesboro, AR", "create_at": date("2010-06-14"), "description": "Instagram: ms_loyal_one", "followers_count": 916, "friends_count": 816, "statues_count": 35475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, AR", "id": "35189a8fcb82b788", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-90.782468,35.763136 -90.611214,35.894116") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5031, "countyName": "Craighead", "cityID": 535710, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419716805091328, "text": "Dude I don't even know why I do this to myself? Why do i insist on listening to sad music when I'm sad?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634230407, "name": "gèminis rodriguez", "screen_name": "GemiGeminis", "lang": "en", "location": "Ontario, CA", "create_at": date("2012-07-12"), "description": "I have a craving for the things that shouldn't be done ✨", "followers_count": 511, "friends_count": 608, "statues_count": 9158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419717019136001, "text": "Wind 1.0 mph ESE. Barometer 29.813 in, Rising Rapidly. Temperature 58.4 °F. Rain today 0.22 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419717035753472, "text": "body rock", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2828233357, "name": "Caden Weible", "screen_name": "cbearfool", "lang": "en", "location": "surfing", "create_at": date("2014-09-23"), "description": "ILS", "followers_count": 773, "friends_count": 456, "statues_count": 5889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419717103038464, "text": "04/11@03:00 - Temp 43.1F, WC 43.1F. Wind 2.0mph SE, Gust 4.0mph. Bar 30.276in, Falling. Rain 0.00in. Hum 72%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419717455384576, "text": "@KrahVickson https://t.co/mUN9MHSPPa", "in_reply_to_status": -1, "in_reply_to_user": 2443086897, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2443086897 }}, "user": { "id": 586276932, "name": "Cody O'Connor", "screen_name": "oconnor_06", "lang": "en", "location": "South Plainfield NJ", "create_at": date("2012-05-20"), "description": "Esu '18", "followers_count": 453, "friends_count": 326, "statues_count": 3052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Stroudsburg, PA", "id": "7dc76c5fd4c473fb", "name": "East Stroudsburg", "place_type": "city", "bounding_box": rectangle("-75.19626,40.983946 -75.161432,41.02404") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42089, "countyName": "Monroe", "cityID": 4221872, "cityName": "East Stroudsburg" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419717874688002, "text": "@wtfanita @tillxodawn @MattySupaSavage ihy", "in_reply_to_status": 719418577820909568, "in_reply_to_user": 110912771, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 110912771, 2198270760, 2870024123 }}, "user": { "id": 492665377, "name": "pound", "screen_name": "NonraciaInigger", "lang": "en", "location": "null", "create_at": date("2012-02-14"), "description": "I don't have hoes. V.P. of #CellyRuHive.", "followers_count": 14012, "friends_count": 809, "statues_count": 60579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419718172454912, "text": "Te quiero a morir // Banda El Recodo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1332098886, "name": "HashSlingingSlasher", "screen_name": "joannaxert_", "lang": "en", "location": "null", "create_at": date("2013-04-06"), "description": "jacob garcia is better than you.", "followers_count": 481, "friends_count": 334, "statues_count": 31312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419718562488320, "text": "@ebbtideapp Tide in Montauk, New York 04/11/2016\n Low 7:24am -0.4\nHigh 12:53pm 2.3\n Low 7:28pm 0.1\nHigh 1:21am 2.8\n Low 8:25am -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.96,41.0483"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 23457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3648054, "cityName": "Montauk" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419718906560512, "text": "@itsablank https://t.co/OSvycSWY2y", "in_reply_to_status": -1, "in_reply_to_user": 497624398, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 497624398 }}, "user": { "id": 463320609, "name": "_briquelle_", "screen_name": "_briquelle_", "lang": "en", "location": "Louisville, Kentucky", "create_at": date("2012-01-13"), "description": "Just a simple girl who loves Jesus & wants to save the world.", "followers_count": 327, "friends_count": 245, "statues_count": 4193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grayson, KY", "id": "01461cde6c5130e8", "name": "Grayson", "place_type": "city", "bounding_box": rectangle("-82.97325,38.30953 -82.882641,38.360784") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21043, "countyName": "Carter", "cityID": 2132572, "cityName": "Grayson" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419718952710144, "text": "Im so thristy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173345433, "name": "big honey", "screen_name": "_TheDaeshGodd", "lang": "en", "location": "#919", "create_at": date("2010-07-31"), "description": "✨never shady; keeps it 3eighty; [+] vibe$ #nccu JAP #MainEvent #BeatABitchUpYup buy me bundles ;)", "followers_count": 1471, "friends_count": 1298, "statues_count": 69607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-04-11T00:00:01.000Z"), "id": 719419719044829184, "text": "Fuck studying but fuck Bering broke ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.46624,34.044807"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278775345, "name": "Jez c.", "screen_name": "official_jayyy", "lang": "en", "location": "Irene ❤️", "create_at": date("2011-04-07"), "description": "these dreams im always trynna chase", "followers_count": 499, "friends_count": 427, "statues_count": 40287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southridge (Fontana)", "id": "07d9e5ae46885000", "name": "Southridge (Fontana)", "place_type": "poi", "bounding_box": rectangle("-117.4662401,34.0448069 -117.46624,34.044807") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419719682359297, "text": "Happy birthday @jacquelyn_c10 @__luke_4 ��������❤️��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 711069692991905792, 2904961117 }}, "user": { "id": 4600396034, "name": "kelly", "screen_name": "kkeeellyy", "lang": "en", "location": "null", "create_at": date("2015-12-18"), "description": "student athlete", "followers_count": 134, "friends_count": 177, "statues_count": 546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419719686733824, "text": "could use some sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2582388625, "name": "philippine✨princess", "screen_name": "sophiaa_cezaar", "lang": "en", "location": "Baltimore, MD", "create_at": date("2014-06-22"), "description": "God ❤️ M.A. | PHS | what's your mindset?", "followers_count": 1045, "friends_count": 1060, "statues_count": 10585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middle River, MD", "id": "010f203624f4bfd3", "name": "Middle River", "place_type": "city", "bounding_box": rectangle("-76.487921,39.308163 -76.338006,39.384988") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2452300, "cityName": "Middle River" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419719804198913, "text": "If the power goes out in this apartment complex one more time, setting off all the alarms, I am going to burn it down.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397025935, "name": "Cailey", "screen_name": "cboyyyd", "lang": "en", "location": "knox", "create_at": date("2011-10-23"), "description": "as much as she stumbles she's running, as much as she runs she's still here", "followers_count": 1057, "friends_count": 1453, "statues_count": 8890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720072454144, "text": "Hmmm probably smash. But where the personality at...�� https://t.co/yARHfn9pNC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286273452, "name": "Jack Stewart", "screen_name": "JackTHEwhiteGUY", "lang": "en", "location": "California, USA", "create_at": date("2011-04-22"), "description": "OHS senior // 18 \\\\ 3000GT Cars&Guitars . Snapchat:JackWStewart", "followers_count": 378, "friends_count": 546, "statues_count": 9818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Heights, CA", "id": "a89175c4c91f45a3", "name": "Citrus Heights", "place_type": "city", "bounding_box": rectangle("-121.331355,38.663673 -121.242804,38.722779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 613588, "cityName": "Citrus Heights" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720118566913, "text": "YOU SO POPPIN https://t.co/JBiGWEdAAt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2933262657, "name": "HER", "screen_name": "gabriellaaaa7", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-12-20"), "description": "tropical mexican", "followers_count": 757, "friends_count": 742, "statues_count": 27682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720135532545, "text": "Temp: 57.9°F Wind:0.5mph Pressure: 30.105hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 60893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720160657409, "text": "Wind 0.0 mph SE. Barometer 30.13 in, Falling slowly. Temperature 59.4 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 21990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720236146688, "text": "@amberjuliablack Thank you. Happy belated to you as well!", "in_reply_to_status": 719342216490192896, "in_reply_to_user": 325281159, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 325281159 }}, "user": { "id": 38737044, "name": "Nick Graziadei", "screen_name": "NickLoungecat", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-05-08"), "description": "I'm Nick. I like to make jokes, play guitar, and give bad advice. Universal Studios and Disney junkie.", "followers_count": 179, "friends_count": 78, "statues_count": 2126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720252989440, "text": "when did it become 2am lmfaooooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3120118612, "name": "abby", "screen_name": "abbyrautmann", "lang": "en", "location": "Muskego, WI", "create_at": date("2015-03-26"), "description": "snapchat ll ajrautmann5", "followers_count": 236, "friends_count": 203, "statues_count": 2254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muskego, WI", "id": "17879f10e478956c", "name": "Muskego", "place_type": "city", "bounding_box": rectangle("-88.189959,42.842802 -88.069408,42.930888") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5555275, "cityName": "Muskego" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720286339072, "text": "@intuition68 devlet daha nasil sahip ciksin ? You need to pull your head out of the sand.", "in_reply_to_status": 719417227020222466, "in_reply_to_user": 1515605197, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1515605197 }}, "user": { "id": 60722509, "name": "yurda budak", "screen_name": "yurdabudak", "lang": "en", "location": "San Francisco", "create_at": date("2009-07-27"), "description": "RTs =/= endorsements; Tweets in English and Turkish", "followers_count": 1005, "friends_count": 403, "statues_count": 14480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720387067904, "text": "1 boy No feelings https://t.co/djoNAEhDyw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3410450354, "name": "Alexandria", "screen_name": "2sweetalex", "lang": "en", "location": "Cali", "create_at": date("2015-08-31"), "description": "C.R.E.A.M", "followers_count": 314, "friends_count": 294, "statues_count": 3559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419720793862144, "text": "my asthma gets so bad at night wut the hell :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184702472, "name": "april 20 ✨", "screen_name": "princessammm", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "huatabampo sonora", "followers_count": 1205, "friends_count": 1006, "statues_count": 79820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419721221713920, "text": "green https://t.co/3ja5Uk2ljY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2823373232, "name": "jordann", "screen_name": "curlyheadjordan", "lang": "en", "location": "null", "create_at": date("2014-09-20"), "description": "insta curlyheadjordann snap kingg_owens", "followers_count": 249, "friends_count": 199, "statues_count": 8780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419722043756544, "text": "Window poking a sport we keep a puncher ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331916965, "name": "#FreebandzTalibanCEO", "screen_name": "BalloutWebbie", "lang": "en", "location": "BALLOUTWORLD w/ Money & Nem", "create_at": date("2011-07-08"), "description": "Free The Guyz Rip The Guyz #SSD #NLMB #OTF #2FFE #YFNBC #MurdaTeam", "followers_count": 1378, "friends_count": 1124, "statues_count": 120845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419722203324416, "text": "Parc national des Everglades https://t.co/iix2T9QaxO #parcnationaldeseverglades #unesco #ochopee #florida #unitedstates", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.99639,25.55444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "parcnationaldeseverglades", "unesco", "ochopee", "florida", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 921, "friends_count": 1295, "statues_count": 4454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12087, "countyName": "Monroe" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419722324807680, "text": "Hell nahhhhh https://t.co/seN5AV37D4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1028005188, "name": "lil pou", "screen_name": "tesfayeeeee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-12-21"), "description": "if it don't make money, it don't make sense", "followers_count": 1284, "friends_count": 2040, "statues_count": 31770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419722522095616, "text": "Temp 40.5° Hi/Lo 40.5/40.2 Rng 0.3° WC 38.4° Hmd 67% Rain 0.00\" Storm 0.00\" BAR 30.299 Falling DP 30.4° Wnd 1mph Dir WNW Gst 14mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 118, "statues_count": 17753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419722656124928, "text": "Wind 7.0 mph SW. Barometer 29.927 in, Rising slowly. Temperature 60.0 °F. Rain today 0.00 in. Humidity 51%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419723298037762, "text": "Wind 2.2 mph S. Barometer 30.16 in, Falling. Temperature 41.9 °F. Rain today 0.00 in. Humidity 46%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 158972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419723343994885, "text": "Pass , but he seem nice lol https://t.co/STekCItgS1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2616085561, "name": "Sheri.", "screen_name": "sherilaii", "lang": "en", "location": "null", "create_at": date("2014-07-10"), "description": "all my socials are @ sherilaii", "followers_count": 943, "friends_count": 183, "statues_count": 15554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:02.000Z"), "id": 719419723759296512, "text": "ok well i love louis he's so amazing???? djdjdj goodnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2532821202, "name": "jazmin", "screen_name": "aimhparis", "lang": "en", "location": "null", "create_at": date("2014-05-29"), "description": "@aimhmatty could we ever be enough?", "followers_count": 3609, "friends_count": 98, "statues_count": 137101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harvey, LA", "id": "337d304222d75060", "name": "Harvey", "place_type": "city", "bounding_box": rectangle("-90.092431,29.836718 -90.035095,29.916898") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2233245, "cityName": "Harvey" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419723893633024, "text": "@Vivas_Wendy ���������� https://t.co/4xqmkGMvnh", "in_reply_to_status": -1, "in_reply_to_user": 419244783, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 419244783 }}, "user": { "id": 265631464, "name": "Keni Marie", "screen_name": "ClassyAssBitch_", "lang": "en", "location": "Milwaukee & Titletown", "create_at": date("2011-03-13"), "description": "She had the need to feel the thunder, chase the lightening from the sky. Watch the storm with all it's wonder, raging in her lovers eyes. #ThatSummer #Always⚡", "followers_count": 339, "friends_count": 403, "statues_count": 28423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green Bay, WI", "id": "648f332371670aae", "name": "Green Bay", "place_type": "city", "bounding_box": rectangle("-88.139804,44.47174 -87.876593,44.574607") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5531000, "cityName": "Green Bay" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419723927003136, "text": "Smasshhhhhhhh https://t.co/7KPlaSVEIV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3396142829, "name": "Chris", "screen_name": "Chrriisss6", "lang": "en", "location": "null", "create_at": date("2015-07-30"), "description": "null", "followers_count": 195, "friends_count": 166, "statues_count": 1245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419724249976832, "text": "I miss when I used to get my clothes ready the night before for school in HS lol now I just grab the first tshirt and jeans I see . .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3184577947, "name": "Aundrea", "screen_name": "dreavxo", "lang": "en", "location": "Cypress, CA ", "create_at": date("2015-05-03"), "description": "Be where you want to be", "followers_count": 182, "friends_count": 183, "statues_count": 5745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cypress, CA", "id": "a077793faceeda6f", "name": "Cypress", "place_type": "city", "bounding_box": rectangle("-118.063298,33.791963 -118.010668,33.845856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 617750, "cityName": "Cypress" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419724317089792, "text": "#NationalSiblingDay @its_Deltaco58 @jruiz7658 @YoBoyOchoa @DolanCobie #luis", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "NationalSiblingDay", "luis" }}, "user_mentions": {{ 2560100737, 4919627544, 954262177, 2401041355 }}, "user": { "id": 530877613, "name": "Hip Hop Head", "screen_name": "joshuabailey15", "lang": "en", "location": " (54.654197, -8.110546)", "create_at": date("2012-03-19"), "description": "My mama always used to tell me: 'If you can't find somethin' to live for, you best find somethin' to die for.'", "followers_count": 300, "friends_count": 671, "statues_count": 4191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419724728180736, "text": "65.5F (Feels: 65.5F) - Humidity: 93% - Wind: 5.4mph SE - Gust: 8.3mph - Pressure: 1011.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 234656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419725030105088, "text": "#SFA I start in the fall ���� https://t.co/UX7LLxirTr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SFA" }}, "user": { "id": 358007611, "name": "#TeamLilBooty", "screen_name": "_ThaDopest", "lang": "en", "location": "Etx made. ", "create_at": date("2011-08-18"), "description": "Don't . I'll snatch your soul ...", "followers_count": 3708, "friends_count": 2996, "statues_count": 98584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Center, TX", "id": "7a15f6b26b4b752e", "name": "Center", "place_type": "city", "bounding_box": rectangle("-94.207311,31.766635 -94.154063,31.825256") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48419, "countyName": "Shelby", "cityID": 4813732, "cityName": "Center" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419725134974977, "text": "@alexjohnsonhaha �� right?", "in_reply_to_status": -1, "in_reply_to_user": 182549810, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 182549810 }}, "user": { "id": 1484062141, "name": "Beating Lights", "screen_name": "BeatingLights", "lang": "en", "location": "currently: Los Angeles, CA", "create_at": date("2013-06-04"), "description": "Los Angeles-based event producer/promoter, content creator, music blog and friend. Run by @shabferdowsi", "followers_count": 714, "friends_count": 1662, "statues_count": 1681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419725436989440, "text": "@BabyyLicia Hope you're doing well sweets ��", "in_reply_to_status": 719419671267553280, "in_reply_to_user": 449057355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1323754969 }}, "user": { "id": 449057355, "name": "Courtney Pandol", "screen_name": "CourtneyPandol", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "null", "followers_count": 721, "friends_count": 408, "statues_count": 21681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419725449592833, "text": "Oh my god I haven't been on my frank ocean vibes in so long", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1936091773, "name": "chuck", "screen_name": "charlie_hagerty", "lang": "en", "location": "probs chillin", "create_at": date("2013-10-04"), "description": "real recognize real #nbs", "followers_count": 649, "friends_count": 510, "statues_count": 9381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419725772693505, "text": "Wind 0.1 mph ESE. Barometer 30.185 in, Falling. Temperature 30.8 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419726036774913, "text": "Holy thunderstorms ��⛈", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2200481101, "name": "Brandon Dodson", "screen_name": "BrandonDodson23", "lang": "en", "location": "Bixby, OK", "create_at": date("2013-11-17"), "description": "04/03/97 fitness is my passion", "followers_count": 158, "friends_count": 137, "statues_count": 163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bixby, OK", "id": "736dc4af8e68929c", "name": "Bixby", "place_type": "city", "bounding_box": rectangle("-95.904492,35.907134 -95.851283,36.017384") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4006400, "cityName": "Bixby" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419726556889088, "text": "Wind 12.8 mph S. Barometer 29.827 in, Falling slowly. Temperature 68.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419726787518464, "text": "@haknoeb_ why is this so accurate ������ https://t.co/dYFQIFSjXn", "in_reply_to_status": -1, "in_reply_to_user": 2587201836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2587201836 }}, "user": { "id": 2206584644, "name": "Jen⚽️", "screen_name": "jennyrz_xo", "lang": "en", "location": "Perris, CA", "create_at": date("2013-11-20"), "description": "null", "followers_count": 373, "friends_count": 288, "statues_count": 10588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419726837886976, "text": "@Pitman_Track could they have the throwers area any further away from the actual track? https://t.co/S9QhU564RY", "in_reply_to_status": 719417799060299776, "in_reply_to_user": 2354500502, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2354500502 }}, "user": { "id": 751579573, "name": "Ay-ayron", "screen_name": "kingkatahdins", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "null", "followers_count": 32, "friends_count": 336, "statues_count": 306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419726871396352, "text": "@shunterrical for 2 weeks you trying to join", "in_reply_to_status": 719419625860018176, "in_reply_to_user": 3111487378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3111487378 }}, "user": { "id": 3185628421, "name": "FREAKY P", "screen_name": "pauleadenise_", "lang": "en", "location": "null", "create_at": date("2015-05-04"), "description": "Future Educator| ΔΣΘ| #TAMUC", "followers_count": 415, "friends_count": 371, "statues_count": 3963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419727190368256, "text": "The other day I was having a conversation with this guy about some girl having a fat ass & he goes \"so do you like guys to or just girls\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 166620021, "name": "LO ❄️", "screen_name": "lonthecelebrity", "lang": "en", "location": "null", "create_at": date("2010-07-14"), "description": "eye candy & soul food.", "followers_count": 3211, "friends_count": 2858, "statues_count": 27569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MS", "id": "b86cd6ef7a54d640", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-89.571643,34.321375 -89.495897,34.387965") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28071, "countyName": "Lafayette", "cityID": 2854840, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419727202791424, "text": "@athenajasvier this is soooo tru that it's scary. Like how did they get this video of us??? Idk", "in_reply_to_status": 719419404803387392, "in_reply_to_user": 603274436, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 603274436 }}, "user": { "id": 2667408776, "name": "Angel Feliciano", "screen_name": "AstoundingAF", "lang": "en", "location": "Bay Area ", "create_at": date("2014-07-21"), "description": "ΦΣΣ-Theta Mu. Rest in peace, Papi. Ori 120614", "followers_count": 531, "friends_count": 340, "statues_count": 18297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419727534120961, "text": "It's been almost a year ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 478298048, "name": "k. dot ♊", "screen_name": "Krys_Official", "lang": "en", "location": "runnin thru the 6 wit my woes", "create_at": date("2012-01-29"), "description": "I'm afraid we'll always be a book with the end pages ripped out. m.k. 他妈的关闭 | 8.7.13 | nwts |", "followers_count": 614, "friends_count": 384, "statues_count": 67723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maywood, CA", "id": "8a8b2699803bff27", "name": "Maywood", "place_type": "city", "bounding_box": rectangle("-118.204687,33.979979 -118.169985,33.996711") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 646492, "cityName": "Maywood" } }
+{ "create_at": datetime("2016-04-11T00:00:03.000Z"), "id": 719419728024854528, "text": "Pass �� https://t.co/F2NfavIczO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1373266256, "name": "nori ♡", "screen_name": "noooridooori", "lang": "en", "location": "somewhere w britney ", "create_at": date("2013-04-22"), "description": "what it do babyboo✨", "followers_count": 742, "friends_count": 315, "statues_count": 15054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419728272306177, "text": "#tile <3 https://t.co/quT0WJ9jBY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "tile" }}, "user": { "id": 3228356030, "name": "Laith Ben Leov", "screen_name": "LeovBen", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-05-27"), "description": "Coffee-Drinker, Music Addict, Music producer. I’m very busy and awesome guy! I’m actually not funny. I’m really mean and people think I am joking! Awkward! #LFC", "followers_count": 69, "friends_count": 168, "statues_count": 1122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419728469446656, "text": "@el_butterball I know right?! He's such an amazing guy ��", "in_reply_to_status": 719419647616004096, "in_reply_to_user": 3258100758, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3258100758 }}, "user": { "id": 365170052, "name": "MMM", "screen_name": "FoolishMireya", "lang": "en", "location": "la/sf", "create_at": date("2011-08-30"), "description": "that was then this is now", "followers_count": 882, "friends_count": 669, "statues_count": 31504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729191006210, "text": "@DaniMo330 oh snap �� well i guess you do what u want when u poppin.", "in_reply_to_status": 719419394259054592, "in_reply_to_user": 1370917478, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1370917478 }}, "user": { "id": 175300229, "name": "big daddy t ♍️", "screen_name": "__tinytee", "lang": "en", "location": "big daddy t-ville", "create_at": date("2010-08-05"), "description": "warnerrobins ✈️ |VSU to THE fvsu| im yo big daddy t, respect. RIP unk, RIP gma. love yall. ✊", "followers_count": 2009, "friends_count": 823, "statues_count": 41766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Valley, GA", "id": "5fb7d0858055aa39", "name": "Fort Valley", "place_type": "city", "bounding_box": rectangle("-83.914883,32.521974 -83.865501,32.574552") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13225, "countyName": "Peach", "cityID": 1331096, "cityName": "Fort Valley" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729430110208, "text": "Airdrie Automotive, Airdrie, AB, T4A 2J9 - https://t.co/ZfgseL4Iqz\n\n#airdrieautomotive #serviceshop #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.983363,51.291412"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "airdrieautomotive", "serviceshop", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 434, "friends_count": 433, "statues_count": 274 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Airdrie, Alberta", "id": "46cfc7fda28c4d88", "name": "Airdrie", "place_type": "city", "bounding_box": rectangle("-114.048657,51.241238 -113.969823,51.308137") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729484636160, "text": "4 Wilson's Phalarope (Phalaropus tricolor) - Black Mountain Power ponds - 2016-04-10 17:10 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-114.161414,35.03486"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13800 }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729543180288, "text": "@C4shhh lmao I'm lyin bro", "in_reply_to_status": 719419671737335808, "in_reply_to_user": 99890148, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99890148 }}, "user": { "id": 269354944, "name": "TreBall", "screen_name": "TrayTheTruth3", "lang": "en", "location": "Dreamville, Tx", "create_at": date("2011-03-20"), "description": "Chase your passion Humble,Tx 21 years young #RIPJERAY", "followers_count": 1524, "friends_count": 1085, "statues_count": 44175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729778061314, "text": "#Trump in 2016 Temp:54.0°F Wind:3.1mph Pressure: 30.00hpa Falling slowly Rain Today 0.00in. Forecast: Precipitation at times, very unsettl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419729861980162, "text": "stfu https://t.co/YH9lyFNfXJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1360961455, "name": "Jessica", "screen_name": "jessicawalker04", "lang": "en", "location": "Torrance, CA", "create_at": date("2013-04-17"), "description": "nhs volleyball", "followers_count": 131, "friends_count": 263, "statues_count": 266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419730109435904, "text": "@kass_pusscake ������������ stats, communications and a film studies class, and I got political science for the summer ☺️", "in_reply_to_status": 719418822432653313, "in_reply_to_user": 1582111574, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1582111574 }}, "user": { "id": 1621564639, "name": "Gabby✨", "screen_name": "GabbyMedinaa", "lang": "en", "location": "Wasco, California", "create_at": date("2013-07-25"), "description": "null", "followers_count": 410, "friends_count": 363, "statues_count": 2916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasco, CA", "id": "01643e7e5fce28b7", "name": "Wasco", "place_type": "city", "bounding_box": rectangle("-119.42052,35.572513 -119.328147,35.610926") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 683542, "cityName": "Wasco" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419730172350464, "text": "this hoe needa chill ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2471562242, "name": "jen", "screen_name": "jenpaolaa", "lang": "en", "location": "null", "create_at": date("2014-04-30"), "description": "@_luispulido is daddy", "followers_count": 521, "friends_count": 328, "statues_count": 15498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419730537230337, "text": "@YungKeem4rmLB ehh. I liked Em both but the manga was good.", "in_reply_to_status": 719419588467798016, "in_reply_to_user": 487173130, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 487173130 }}, "user": { "id": 423659807, "name": "Shaun.", "screen_name": "AmericanChink", "lang": "en", "location": "null", "create_at": date("2011-11-28"), "description": "All homo intended. Scissor Bang Sisters. I am RoboBitch, stop in the name of feminism.", "followers_count": 405, "friends_count": 234, "statues_count": 40560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring Valley Lake, CA", "id": "01bf3d0d1ae1118c", "name": "Spring Valley Lake", "place_type": "city", "bounding_box": rectangle("-117.28777,34.476065 -117.251962,34.533456") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 673700, "cityName": "Spring Valley Lake" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419730927468544, "text": "@a1fbgmn @_iTrevii the hate be so real", "in_reply_to_status": 719371608750731264, "in_reply_to_user": 531567264, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 531567264, 1518087734 }}, "user": { "id": 3004153021, "name": "WANDERLUST", "screen_name": "__shevibes", "lang": "en", "location": "null", "create_at": date("2015-01-30"), "description": "null", "followers_count": 603, "friends_count": 808, "statues_count": 2498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plymouth, NC", "id": "882fcf6b4add2a3d", "name": "Plymouth", "place_type": "city", "bounding_box": rectangle("-76.77167,35.827535 -76.719525,35.878825") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37187, "countyName": "Washington", "cityID": 3753040, "cityName": "Plymouth" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419731002793984, "text": "HAPPY 18th BITCH❤️ love you forever and always trash bag https://t.co/eiQJgl76ha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1042901983, "name": "hailz", "screen_name": "hailey_golec", "lang": "en", "location": "California, USA", "create_at": date("2012-12-28"), "description": "passionate/HIU ⚽️'20", "followers_count": 1194, "friends_count": 790, "statues_count": 25572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419731413901312, "text": "������������������ https://t.co/tzzvRzL98C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1576273884, "name": "cap", "screen_name": "shmadamm", "lang": "en", "location": "ʞays4days.", "create_at": date("2013-07-07"), "description": "I have a dream but no time to sleep. [HEADMODE]", "followers_count": 1251, "friends_count": 813, "statues_count": 6323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-11T00:00:04.000Z"), "id": 719419731925708800, "text": "Hell yeah https://t.co/iLTLhMKJwg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415920039, "name": "Hoodrich Pablo", "screen_name": "OlDirtyPablo", "lang": "en", "location": "FL ", "create_at": date("2011-11-18"), "description": "don't call me friend I don't fuck wit you like that", "followers_count": 2225, "friends_count": 811, "statues_count": 313905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419732391247872, "text": "Just released #Tied on #soundcloud! Been #songwriting so much recently thanks to #waynelim.… https://t.co/U9VWlvQFQk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tied", "soundcloud", "songwriting", "waynelim" }}, "user": { "id": 2597025872, "name": "Devon Perry", "screen_name": "DevonTylerPerry", "lang": "en", "location": "Los Angeles ", "create_at": date("2014-06-30"), "description": "Singer/Songwriter, producer, and translator of the heart.", "followers_count": 319, "friends_count": 307, "statues_count": 1710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419733204840448, "text": "My iCloud in my bio you can email me, FaceTime me, dm me for my number, it's whatever", "in_reply_to_status": 719419384737832960, "in_reply_to_user": 443348706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 443348706, "name": "+", "screen_name": "SnowboyHennessy", "lang": "en", "location": "California, USA", "create_at": date("2011-12-21"), "description": "For The Kids By The Kids//Creating Tomorrow//SnowboyHennessy@icloud.com", "followers_count": 1410, "friends_count": 643, "statues_count": 109279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419733339078658, "text": "Madami (puro ia na :( ) https://t.co/xwU4fZ6mR1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1420084910, "name": "el ; im okay?¿", "screen_name": "indiewink", "lang": "en", "location": "a j m m a r j e ☁︎", "create_at": date("2013-05-11"), "description": "malec slash shumdario slash saphael trash", "followers_count": 7032, "friends_count": 734, "statues_count": 24227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419733766832128, "text": "Wind 0.0 mph ---. Barometer 29.981 in, Rising. Temperature 50.5 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 267197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419733989203968, "text": "@purple_diiiva https://t.co/NKcIn5aigs", "in_reply_to_status": 356198849360044032, "in_reply_to_user": 1276302300, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1276302300 }}, "user": { "id": 16091981, "name": "Rain B☮jangles &: {)", "screen_name": "Rainbojangles", "lang": "en", "location": "SE Portland PDX", "create_at": date("2008-09-01"), "description": "I don't buy followers! If they can't swallow facts, let them eat fiction. ~Rain Bojangles #goodthingscoming I block follower-sellers who follow me.", "followers_count": 12971, "friends_count": 13600, "statues_count": 61575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419734010146816, "text": "@obeezyyyy you've got worse puns than a white dad", "in_reply_to_status": 719415679087030272, "in_reply_to_user": 2929042752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2929042752 }}, "user": { "id": 1343469457, "name": "madz", "screen_name": "mad_salz", "lang": "en", "location": "LA / santa cruz", "create_at": date("2013-04-10"), "description": "forget it dad, life's all right (͡° ͜ʖ ͡°)", "followers_count": 538, "friends_count": 502, "statues_count": 16112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Cruz, CA", "id": "3c9e627dd6b55d9e", "name": "Santa Cruz", "place_type": "city", "bounding_box": rectangle("-122.076144,36.948098 -121.986229,37.010652") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 669112, "cityName": "Santa Cruz" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419734341517312, "text": "@neweragodd annnnnd again", "in_reply_to_status": 719359957506764800, "in_reply_to_user": 4913844674, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4913844674 }}, "user": { "id": 4913844674, "name": "Makalih Cobb", "screen_name": "neweragodd", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2016-02-15"), "description": "null", "followers_count": 122, "friends_count": 108, "statues_count": 47 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419735167737856, "text": "@matoswk75 same thing tho", "in_reply_to_status": 719418854372352004, "in_reply_to_user": 127463226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 127463226 }}, "user": { "id": 892441, "name": "jonathan zwickel", "screen_name": "zwickelicious", "lang": "en", "location": "Capitol Hill, Seattle", "create_at": date("2007-03-10"), "description": "Self-facilitating media node :: Senior editor, City Arts Magazine", "followers_count": 3022, "friends_count": 2129, "statues_count": 11966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419735184519168, "text": "I don't trust people who don't like dogs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 359611613, "name": "britt", "screen_name": "Bertneey_", "lang": "en", "location": "Los Angeles ", "create_at": date("2011-08-21"), "description": "BAHS '17", "followers_count": 957, "friends_count": 780, "statues_count": 11578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419735524245504, "text": "Temp: 46.4°F - Dew Point: 45.6° - Wind: 9.1 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 29.71in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419735532707840, "text": "Send Mario a dm saying \"your hand\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059839250, "name": "Andrew", "screen_name": "drew_310", "lang": "en", "location": "#SAVAGESZN", "create_at": date("2013-01-03"), "description": "m.r | DBU | Young King with Big Dreams", "followers_count": 308, "friends_count": 274, "statues_count": 7010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-04-11T00:00:05.000Z"), "id": 719419736275046400, "text": "there was a me before there was a me n u & when there's not a me n u there's still gonna be a me so y would I listen to what u tell me 2 do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2459146142, "name": "jade ♡", "screen_name": "huneyglttergurl", "lang": "en", "location": "null", "create_at": date("2014-04-22"), "description": "null", "followers_count": 306, "friends_count": 197, "statues_count": 6095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-04-11T00:00:06.000Z"), "id": 719419736748982273, "text": "03:00 51.8°F Feels:51.8°F (Hi52.0°F/Lo50.7°F) Hum:89% Wnd:--- 0.0MPH Baro:29.57in. Prcp:0.28in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 196, "friends_count": 263, "statues_count": 28249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782099222310912, "text": "I hope feminists find this offensive!!!!!! Another thing for them to hate. https://t.co/jOAQAekpdQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233434908, "name": "Potatodidlydo", "screen_name": "clitclat", "lang": "en", "location": "Boulder, CO", "create_at": date("2011-01-02"), "description": "haters gonna hate, potatoes gonna potate", "followers_count": 359, "friends_count": 258, "statues_count": 16088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782100128309248, "text": "@Grizztof Also \"somewhere out there\" from American Tale, such beautiful and heartwarming music.", "in_reply_to_status": 719781827859210240, "in_reply_to_user": 141089665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 418322949 }}, "user": { "id": 141089665, "name": "M. Takahashi ZPD", "screen_name": "matoakit", "lang": "en", "location": "Foxe Basin, Canada", "create_at": date("2010-05-06"), "description": "Frolicking in bamboo forests and the watery shores of the sea. Always SFW! If you don't feel the inner fire, then pretend as if you did. #PoetryFox", "followers_count": 669, "friends_count": 858, "statues_count": 27131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782100275175424, "text": "@margs9184 https://t.co/xUgk0w5xKg", "in_reply_to_status": -1, "in_reply_to_user": 30093056, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 30093056 }}, "user": { "id": 1872548804, "name": "alex woodworth", "screen_name": "slightlywetsock", "lang": "en", "location": "Upper Sandusky, OH", "create_at": date("2013-09-16"), "description": "karaoke angel", "followers_count": 199, "friends_count": 100, "statues_count": 23390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upper Sandusky, OH", "id": "12198c1d40c289f0", "name": "Upper Sandusky", "place_type": "city", "bounding_box": rectangle("-83.297819,40.805837 -83.221139,40.857981") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39175, "countyName": "Wyandot", "cityID": 3979044, "cityName": "Upper Sandusky" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782101344608256, "text": "HAPPY LATE BIRTHDAY @MarissaJensen29 ❤️ wouldn't want anyone else as my mom https://t.co/lIoeBL6Buc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 881815454 }}, "user": { "id": 1489606230, "name": "kors", "screen_name": "GoerlitzKori", "lang": "en", "location": "ptown", "create_at": date("2013-06-06"), "description": "null", "followers_count": 1021, "friends_count": 563, "statues_count": 7494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782101394989057, "text": "@Tynene_ thanks", "in_reply_to_status": 719763743446437888, "in_reply_to_user": 295898381, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 295898381 }}, "user": { "id": 314888650, "name": "April 12 ❣", "screen_name": "TaelorTheBOSS", "lang": "en", "location": "null", "create_at": date("2011-06-10"), "description": "follow me on snapchat: taelortheboss ❤️", "followers_count": 575, "friends_count": 444, "statues_count": 30957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782101722136576, "text": "Me trying to knock @JTMcLuckey over during bubble soccer: \n\nhttps://t.co/3Jl2QlXcT9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 892737434 }}, "user": { "id": 625293099, "name": "Diana Gonzalez", "screen_name": "MissDee_94", "lang": "en", "location": "La Verne, CA", "create_at": date("2012-07-02"), "description": "And what do we say about coincidences? The universe is rarely so lazy. -Mycroft Holmes.", "followers_count": 127, "friends_count": 127, "statues_count": 1845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Verne, CA", "id": "7183cae332544afd", "name": "La Verne", "place_type": "city", "bounding_box": rectangle("-117.80156,34.086183 -117.733148,34.165358") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640830, "cityName": "La Verne" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102053486593, "text": "Future my nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2607034306, "name": "Jamal the great⚫️♠️", "screen_name": "dea_babyjay", "lang": "en", "location": "southpark, TX", "create_at": date("2014-06-13"), "description": "jamal♥R.I.P leroy, lil jimmy,lil roe, lil dex ♥ #deadendtx", "followers_count": 839, "friends_count": 723, "statues_count": 28898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102523203584, "text": "めっちゃラブライブ爆音で流してて島風ちゃん車に貼ってたけど、元海兵隊で50キャリバー撃ちまくってる動画見せてくれてギャップが凄い https://t.co/Ee1xHvtI7O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.5565452,34.0683934"), "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 577093870, "name": "ショーン君さん", "screen_name": "sean1208", "lang": "ja", "location": "17 English school race dept.", "create_at": date("2012-05-11"), "description": "艦これ 加賀さん シルビア セロー AC/DC BLUESROCK G&L", "followers_count": 349, "friends_count": 422, "statues_count": 14742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102548508673, "text": "Ripley SW Limestone Co. Temp: 56.7°F Wind:0.7mph Pressure: 997.5mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102632239104, "text": "going to sleeeeppppp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3005569268, "name": "ty!", "screen_name": "_typimpin", "lang": "en", "location": "null", "create_at": date("2015-01-31"), "description": "Minding my business and shit", "followers_count": 353, "friends_count": 296, "statues_count": 18850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102678384641, "text": "This guys butthurt is too funny. For fucks sake anyone gets offended about anything. \nhttps://t.co/DZBf3aUEYt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4581433934, "name": "Rex", "screen_name": "W3bclone", "lang": "en", "location": "Houston, TX", "create_at": date("2015-12-16"), "description": "The name is Rex, 21 years old, Rotarian at D5890, Ingress player for the Enlightened (L13), Major phone lover! (LG V10) (#DammitRex™) #XFACorGTFO ( ͡° ͜ʖ ͡°) ", "followers_count": 35, "friends_count": 137, "statues_count": 388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102699417602, "text": "@amandaalcortaa I LOVE UUUUUU ����", "in_reply_to_status": 719702983487791105, "in_reply_to_user": 2221953326, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2221953326 }}, "user": { "id": 535770700, "name": "xandrie", "screen_name": "xandreezus", "lang": "en", "location": "LMHS ", "create_at": date("2012-03-24"), "description": "what's poppin' ♒️", "followers_count": 746, "friends_count": 669, "statues_count": 17559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102753894400, "text": "MAKE SENSE NIGGA https://t.co/2uOcDSf2tx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 113030654, "name": "Reggie Davis II.", "screen_name": "naldodavis", "lang": "en", "location": "Lansing, MI", "create_at": date("2010-02-10"), "description": "Everything I'm not made me everything I am. #DooDooRecords #DubNation", "followers_count": 1713, "friends_count": 931, "statues_count": 213285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waverly, MI", "id": "011071d81596fbba", "name": "Waverly", "place_type": "city", "bounding_box": rectangle("-84.720053,42.681889 -84.60259,42.78436") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26045, "countyName": "Eaton", "cityID": 2684800, "cityName": "Waverly" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782102946988032, "text": "Wind 0.0 mph ---. Barometer 30.10 in, Falling slowly. Temperature 61.9 °F. Rain today 0.04 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-12T00:00:00.000Z"), "id": 719782103039156224, "text": "I'm guessing a teaser image of that supposed MMO Neptunia game. Or just Vert showcasing a sword and tits. https://t.co/nqjsGPjeDJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397542399, "name": "Marc Dichoco", "screen_name": "MindofMarc", "lang": "en", "location": "My Thoughtful Spot", "create_at": date("2011-10-24"), "description": "Really talented at picking out the voices of VAs in Dubbed Anime/Games. Distro Supporter/English Dub Fan. Also, I don't tweet...I chirp! Care to listen?", "followers_count": 189, "friends_count": 860, "statues_count": 8887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daly City, CA", "id": "6a71821001635bbd", "name": "Daly City", "place_type": "city", "bounding_box": rectangle("-122.500164,37.649122 -122.405233,37.708437") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 617918, "cityName": "Daly City" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782103638913024, "text": "�� https://t.co/LXGWJQh2ca", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4504135932, "name": "Jessica Poindexter", "screen_name": "BbyCakez707", "lang": "en", "location": "Ukiah, CA", "create_at": date("2015-12-08"), "description": "let the good times roll ||♡", "followers_count": 14, "friends_count": 51, "statues_count": 78 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ukiah, CA", "id": "0742517d53e00505", "name": "Ukiah", "place_type": "city", "bounding_box": rectangle("-123.232051,39.116505 -123.18786,39.174311") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6045, "countyName": "Mendocino", "cityID": 681134, "cityName": "Ukiah" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782103643127808, "text": "I want to wish the beautiful and talented @reverieLOVE a happy birthday! Have a safe trip pimpin ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36018979 }}, "user": { "id": 2973886184, "name": "Geo", "screen_name": "geo_da_barber", "lang": "en", "location": "Boyle Heights, Los Angeles", "create_at": date("2015-01-10"), "description": "Goodfellas barbershop", "followers_count": 42, "friends_count": 222, "statues_count": 60 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boyle Heights, Los Angeles", "id": "03143399d0b3b8ff", "name": "Boyle Heights", "place_type": "neighborhood", "bounding_box": rectangle("-118.223776,34.01253 -118.191293,34.066015") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782103907348480, "text": "This is a big pill for me to swallow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242079215, "name": "MarcusTheDoctor™", "screen_name": "TheDoc__", "lang": "en", "location": "Isolated ", "create_at": date("2011-01-23"), "description": "Doc season 2016 | UNLV", "followers_count": 2060, "friends_count": 1557, "statues_count": 31554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782104196845568, "text": "04/12@03:00 - Temp 61.7F, WC 61.2F. Wind 7.3mph SSW, Gust 15.0mph. Bar 29.929in, Falling slowly. Rain 0.00in. Hum 77%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782104649834496, "text": "Just posted a video @ The Village Underground NYC https://t.co/b7BFthxXMN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0009079,40.7306099"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49187382, "name": "MAURICE SMITH", "screen_name": "MOEBETTAR", "lang": "en", "location": "BROOKLYN, NY", "create_at": date("2009-06-20"), "description": "Singer/ Praise & Worship Leader/ Songwriter/Performer FACEBOOK:MOEBETTAR/ MYSPACE:MAURICE SMITH/ IG:MAURICE_SMITH", "followers_count": 690, "friends_count": 843, "statues_count": 7386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782104901545984, "text": "Wind 0.0 mph ---. Barometer 30.242 in, Rising. Temperature 39.9 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782104993763328, "text": "Temp 56.4° Hi/Lo 56.7/54.9 Rng 1.8° WC 55.1° Hmd 78% Rain 0.00\" Storm 0.03\" BAR 29.997 Falling DP 49.6° Wnd 8mph Dir SW Gst 27mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782105224314880, "text": "@raddsabb ok slayage", "in_reply_to_status": 719768907800678400, "in_reply_to_user": 61028355, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 61028355 }}, "user": { "id": 2240207328, "name": "shut up", "screen_name": "eelenagil", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2013-12-10"), "description": "tell my baby I'm back in town", "followers_count": 687, "friends_count": 407, "statues_count": 15676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782105350189056, "text": "Bye lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 3077599386, "name": "Rosalinda", "screen_name": "rosawuvsyou", "lang": "en", "location": "Galt, CA", "create_at": date("2015-03-13"), "description": "don't trip chocolate chip", "followers_count": 321, "friends_count": 252, "statues_count": 2716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galt, CA", "id": "6f49c528bb0c1a33", "name": "Galt", "place_type": "city", "bounding_box": rectangle("-121.337371,38.236791 -121.264316,38.294229") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 628112, "cityName": "Galt" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782105475973121, "text": "@ebbtideapp Tide in Rowayton, Connecticut 04/12/2016\nHigh 3:42am 8.1\n Low 10:20am -0.3\nHigh 4:21pm 7.3\n Low 10:38pm 0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.445,41.065"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 23734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Connecticut, USA", "id": "e86b380cfefcced5", "name": "Connecticut", "place_type": "admin", "bounding_box": rectangle("-73.727776,40.950918 -71.786994,42.050588") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 918920, "cityName": "Darien" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782106017058817, "text": "Tom Coffin is the man.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 590345914, "name": "Sumatra", "screen_name": "DakotaSpani", "lang": "en", "location": "Bellingham, WA", "create_at": date("2012-05-25"), "description": "Don't cry......m'lady....", "followers_count": 215, "friends_count": 205, "statues_count": 4615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782106050613248, "text": "Always counting on Manny for an invigorating, thought provoking, philosophical, open-minded conversation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2936698279, "name": "KaylaBayla", "screen_name": "doe_eyed_me", "lang": "en", "location": "null", "create_at": date("2014-12-19"), "description": "A soul that sought serendipity", "followers_count": 302, "friends_count": 243, "statues_count": 6458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmichael, CA", "id": "b37d3f2758ab6f61", "name": "Carmichael", "place_type": "city", "bounding_box": rectangle("-121.3461,38.591142 -121.291344,38.691468") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 611390, "cityName": "Carmichael" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782106075795456, "text": "Ritas are so nasty but I'll drink one if it's there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259602911, "name": "isabe", "screen_name": "xcieckret_", "lang": "en", "location": "From The Westside, With Love", "create_at": date("2011-03-01"), "description": "null", "followers_count": 860, "friends_count": 430, "statues_count": 15231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782106365341696, "text": "���� @ Northern Illinois University https://t.co/tLIGjaSUuE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.780201,41.9308762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2487184946, "name": "Chris'", "screen_name": "itsmemylet", "lang": "en", "location": "null", "create_at": date("2014-05-10"), "description": "Not just some meat covered skeleton made up of stardust ⚓ IITian. Nursing.Typography. Art. Travel. ✈️ @Chris_Homura's", "followers_count": 296, "friends_count": 311, "statues_count": 11979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DeKalb, IL", "id": "76cec30d4c553725", "name": "DeKalb", "place_type": "city", "bounding_box": rectangle("-88.792846,41.883102 -88.714573,41.972512") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1719161, "cityName": "DeKalb" } }
+{ "create_at": datetime("2016-04-12T00:00:01.000Z"), "id": 719782106553954304, "text": "Haaaaaate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2724755577, "name": "Carmen Cardenas", "screen_name": "cardenascarmen_", "lang": "en", "location": "Washington, USA", "create_at": date("2014-07-25"), "description": "You give me Purpose.", "followers_count": 164, "friends_count": 201, "statues_count": 8442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyside, WA", "id": "00cce068663f6743", "name": "Sunnyside", "place_type": "city", "bounding_box": rectangle("-120.046373,46.286764 -119.964934,46.343583") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5368750, "cityName": "Sunnyside" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782107560542208, "text": "Morning Beautiful Lady���� https://t.co/OO5bgooUP4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 708061168745615360, "name": "Weed Beat", "screen_name": "TheWeedBeat", "lang": "en", "location": "Washington, USA/ World Wide", "create_at": date("2016-03-10"), "description": "Lover of all things Weed!! Promoter, Sales,Pitch & Booth man Events,Fairs & Festivals. Reviewer of Great Weed Products &Cannabis Cuisine Fun! 18+ Let's Enjoy!!", "followers_count": 789, "friends_count": 425, "statues_count": 6828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782107875151874, "text": "Stress eat week", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1675150430, "name": "reshley andrew", "screen_name": "reshleyy", "lang": "en-gb", "location": "Provo, UT", "create_at": date("2013-08-16"), "description": "my mama don't like you", "followers_count": 217, "friends_count": 756, "statues_count": 993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant Grove, UT", "id": "b7bf044c46621317", "name": "Pleasant Grove", "place_type": "city", "bounding_box": rectangle("-111.774284,40.351435 -111.678506,40.413594") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4960930, "cityName": "Pleasant Grove" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782108038885376, "text": "Wind 0.0 mph NNE. Barometer 30.133 in, Rising slowly. Temperature 37.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782108474908672, "text": "@Sk15zy @Susgodfizzy_ sus nice bro I don't know about that", "in_reply_to_status": 719778390627131397, "in_reply_to_user": 2369971106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2369971106, 1731077982 }}, "user": { "id": 2983591536, "name": "Luis", "screen_name": "Lcardenas_4", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "That's cool and whatever #MercFam", "followers_count": 265, "friends_count": 212, "statues_count": 2779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782108735127552, "text": "just spent like an hour trying to figure out our new coffee machine jfc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1036157365, "name": "Cam☽✧", "screen_name": "cammmylynn", "lang": "en", "location": "null", "create_at": date("2012-12-25"), "description": "eat ya ♡ out", "followers_count": 1503, "friends_count": 1093, "statues_count": 64929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardner, MA", "id": "f529ed8d70851f9e", "name": "Gardner", "place_type": "city", "bounding_box": rectangle("-72.046012,42.535734 -71.934819,42.614744") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2525485, "cityName": "Gardner" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782109439664130, "text": "Very informative video on Islam in the UK. https://t.co/nTmiKsNIMz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36707754, "name": "Franklin", "screen_name": "Farkal", "lang": "en", "location": "Boca Raton, Florida", "create_at": date("2009-04-30"), "description": "❤️, CEO, self made Millionaire, Type A, Christian, unafraid, Honoring Vets, I love ppl who ❤️America.", "followers_count": 1163, "friends_count": 1522, "statues_count": 8688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782109821411328, "text": "Temp: 61.0°F Wind:0.2mph Pressure: 30.068hpa Falling Rain Today 0.03in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 60987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782109951295490, "text": "When no one remembers we put ice cream on my backpack's side pocket after lunch ������ @Desia_J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 315649724 }}, "user": { "id": 454516225, "name": "Aphrodite ☪", "screen_name": "Beankaaaa", "lang": "en", "location": "IG: @beankaa", "create_at": date("2012-01-03"), "description": "''Who? Who? What are you, a fuckin' owl?''", "followers_count": 553, "friends_count": 452, "statues_count": 26259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782110022672384, "text": "69.3F (Feels: 69.3F) - Humidity: 99% - Wind: 2.2mph SE - Gust: 3.8mph - Pressure: 1011.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 234800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782110240907268, "text": "@ion26_exbb グロガバのゼミ受かる受からないってどういうシステム?演習の授業を受講できればいいってこと?", "in_reply_to_status": 719746013561393152, "in_reply_to_user": 1388898044, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 1388898044 }}, "user": { "id": 2384602086, "name": "M.I", "screen_name": "baseeballabc052", "lang": "ja", "location": "Murfreesboro, TN", "create_at": date("2014-03-11"), "description": "Saitama Univ. (Japan) ⇄ Middle Tennessee State University (US) / Be sure to taste your words before you spit them out.", "followers_count": 298, "friends_count": 277, "statues_count": 5090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782110563663873, "text": "IM FINALLY 17 BLESSED ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240043115, "name": "Banana✨", "screen_name": "hai_brianna", "lang": "en", "location": "nevada", "create_at": date("2011-01-18"), "description": "spectrum | mads | lds | I'm a spoiled brat", "followers_count": 851, "friends_count": 259, "statues_count": 6658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782110844719104, "text": "Lost in the sea ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3258497714, "name": "ELI", "screen_name": "anthony_elija", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-06-27"), "description": "♓️", "followers_count": 21, "friends_count": 41, "statues_count": 477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782111037624321, "text": "I needa meet some new people", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389235241, "name": "Midget Mike", "screen_name": "MidgetMike41", "lang": "en", "location": "null", "create_at": date("2011-10-11"), "description": "#BarberGang #RestInPeaceTorry", "followers_count": 580, "friends_count": 565, "statues_count": 11038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-04-12T00:00:02.000Z"), "id": 719782111209775104, "text": "Wind 0.0 mph SSW. Barometer 29.87 in, Falling. Temperature 46.4 °F. Rain today 0.01 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 158999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782111821955072, "text": "@jesssaj_ happy birthday ��", "in_reply_to_status": 719775546637361152, "in_reply_to_user": 476092530, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 476092530 }}, "user": { "id": 3237298009, "name": "lucid", "screen_name": "fcknphenomenal_", "lang": "en", "location": "y'know?", "create_at": date("2015-06-05"), "description": "null", "followers_count": 77, "friends_count": 57, "statues_count": 1814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782111863934976, "text": "@StephGotHerOwn smh sadly ��", "in_reply_to_status": 719781764751724544, "in_reply_to_user": 53557861, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53557861 }}, "user": { "id": 3418880772, "name": "Gods Most High", "screen_name": "Saintamiss", "lang": "en", "location": "null", "create_at": date("2015-09-01"), "description": "Don't Be a...", "followers_count": 201, "friends_count": 194, "statues_count": 2835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782111893405696, "text": "Ever can't sleep and you think about everything? Now I can't sleep cause now I'm pissed about everything. Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4283674,26.7100194"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 719626409455566849, "name": "Michael Guy", "screen_name": "Michael66827378", "lang": "en", "location": "Labelle, FL", "create_at": date("2016-04-11"), "description": "Chillin it", "followers_count": 3, "friends_count": 55, "statues_count": 1 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Labelle, FL", "id": "ad62ac9099a62f75", "name": "Labelle", "place_type": "city", "bounding_box": rectangle("-81.465813,26.736572 -81.421531,26.769311") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12051, "countyName": "Hendry", "cityID": 1237225, "cityName": "LaBelle" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782112161873921, "text": "@pandemona kind of agree, but aren't all computers just a series of ANDs and ORs though?", "in_reply_to_status": 719708067340759040, "in_reply_to_user": 18337283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18337283 }}, "user": { "id": 7644032, "name": "Omar Bohsali", "screen_name": "omarish", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-07-22"), "description": "Explorer, engineer, entrepreneur. Now: Retail Investing Platform @LendingHome. Previously: @Priceonomics. @ycombinator / University of Virginia alum.", "followers_count": 1050, "friends_count": 690, "statues_count": 2437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782112677593088, "text": "how crazy is that i read about plants and fungi and i kept seeing an imagine in my head and wrote an intense fucking poem", "in_reply_to_status": 719781818447167488, "in_reply_to_user": 372011659, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372011659, "name": "naomi✨", "screen_name": "teninchpriestly", "lang": "en", "location": "flo rida", "create_at": date("2011-09-11"), "description": "we live and breathe words", "followers_count": 527, "friends_count": 918, "statues_count": 30556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melbourne, FL", "id": "d970e7fd69eaf4ec", "name": "Melbourne", "place_type": "city", "bounding_box": rectangle("-80.706074,28.03509 -80.593324,28.200658") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1243975, "cityName": "Melbourne" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782112686018560, "text": "Happy birthday @morgie_rae96 ! Glad we've made so many awesome memories since we've known each other �������� https://t.co/km0mIlVIee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 991955148 }}, "user": { "id": 1368934872, "name": "walle.", "screen_name": "writneyballe", "lang": "en", "location": "null", "create_at": date("2013-04-21"), "description": "@dodgers", "followers_count": 317, "friends_count": 168, "statues_count": 32393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782113025740800, "text": "Try https://t.co/VcYkflu57L All-natural grilled chicken breast, artichoke hearts, chickpeas, roasted red #grilled #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.66949094,41.90527363"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "grilled", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4386, "friends_count": 876, "statues_count": 414927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782113134792704, "text": "Perhaps there was a better place to spend time for someone in NFL drug program than a place where marijuana is legal #wtf @TonyGrossi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wtf" }}, "user_mentions": {{ 33424006 }}, "user": { "id": 163580156, "name": "Carl Bee", "screen_name": "bigbear318", "lang": "en", "location": "Munroe Falls, Ohio", "create_at": date("2010-07-06"), "description": "Largest Brown's fan in Ohio, both physically and fanatacism", "followers_count": 67, "friends_count": 160, "statues_count": 655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Munroe Falls, OH", "id": "99e006baa05003eb", "name": "Munroe Falls", "place_type": "city", "bounding_box": rectangle("-81.452727,41.123841 -81.41407,41.152152") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3953312, "cityName": "Munroe Falls" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782113579442176, "text": "Why ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1479076026, "name": "Bee .", "screen_name": "Belinda_Mariee", "lang": "en", "location": "null", "create_at": date("2013-06-03"), "description": "relax , it's just Twitter ✨", "followers_count": 392, "friends_count": 247, "statues_count": 13148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782113948475393, "text": "@penaashley_ FaceTime me tmr after school", "in_reply_to_status": 719781927222267904, "in_reply_to_user": 2500868498, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2500868498 }}, "user": { "id": 1173693043, "name": "Sarah Valenzuela", "screen_name": "seewa13", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-02-12"), "description": "PHS//Vball//Music", "followers_count": 222, "friends_count": 309, "statues_count": 8186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782114237894656, "text": "@WhoopiGoldberg I really want to be your friend in real life. You are an inspiration. #TrueStory", "in_reply_to_status": -1, "in_reply_to_user": 284602545, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TrueStory" }}, "user_mentions": {{ 284602545 }}, "user": { "id": 14915052, "name": "Jefferson", "screen_name": "JeffersonsBeard", "lang": "en", "location": "Detroit, Michigan", "create_at": date("2008-05-26"), "description": "I'm Jefferson.", "followers_count": 460, "friends_count": 616, "statues_count": 3243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferndale, MI", "id": "5aed094b21b72a54", "name": "Ferndale", "place_type": "city", "bounding_box": rectangle("-83.17198,42.445669 -83.109587,42.476085") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2627880, "cityName": "Ferndale" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782114892320768, "text": "The Accuracy with this tweet �� https://t.co/xJEIM2HVdZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162633005, "name": "Malicious Chiko", "screen_name": "djdroidnexus", "lang": "en", "location": "dope spot near you", "create_at": date("2010-07-04"), "description": "null", "followers_count": 154, "friends_count": 295, "statues_count": 12812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Little River, FL", "id": "4ba5710f3448a6d8", "name": "West Little River", "place_type": "city", "bounding_box": rectangle("-80.260185,25.845593 -80.196918,25.871522") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1276487, "cityName": "West Little River" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782114967748608, "text": "Girls changing their bios faster than bitches running back to their ex", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3100875382, "name": "Trev✈️", "screen_name": "trevorkemngang", "lang": "en", "location": "9⃣0⃣9⃣", "create_at": date("2015-03-20"), "description": "6'3/Leo♌️/Im a twin stop asking/ SC: trevor_4hunnid", "followers_count": 643, "friends_count": 445, "statues_count": 36642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782115311640576, "text": ".@CaptainAmerica I stand with #TeamIronMan. #TweetMe https://t.co/zB9yVNjgvK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamIronMan", "TweetMe" }}, "user_mentions": {{ 701615052 }}, "user": { "id": 1870050206, "name": "KK⚡️", "screen_name": "Koldhearted_kel", "lang": "en", "location": "Ontario, CA", "create_at": date("2013-09-15"), "description": "sc:// jesuss_sonn #SHSU", "followers_count": 942, "friends_count": 640, "statues_count": 21539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782115357958144, "text": "�� RT @_jonnie29: 6. I can’t name one flipper who isn’t hit", "in_reply_to_status": 719781773807366144, "in_reply_to_user": 296626523, "favorite_count": 0, "coordinate": point("-91.18223426,30.41910951"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296626523 }}, "user": { "id": 236156697, "name": "Kenny", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | #EnginEars | #KennythingCanHappen | Mix x Master $25 Per Song | excusetherant@gmail.com", "followers_count": 793, "friends_count": 390, "statues_count": 108010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782115496202240, "text": "*lobs to TL and goes to bed* https://t.co/l8JzThYhI5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282894832, "name": "unpopular nobody.", "screen_name": "HBfromKC", "lang": "en", "location": "Kansas City, MO", "create_at": date("2011-04-15"), "description": "*terms & conditions may apply. K=Fuck You. Beards Are Life. Black is beautiful. Founder of #PeepDontSpeak #BUSGF #TimsRightHand #BBW DMs Closed", "followers_count": 2689, "friends_count": 1818, "statues_count": 178386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782115718467585, "text": "@TheMercedesXXX @RaboCaliente1 only time I'll send this dick, https://t.co/VMJi1St9xZ", "in_reply_to_status": 719773948594946049, "in_reply_to_user": 2666936809, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2666936809, 748995164 }}, "user": { "id": 928112988, "name": "Kevin Maginnis", "screen_name": "kmag94", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "null", "followers_count": 48, "friends_count": 264, "statues_count": 867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newcastle, CA", "id": "000a100640d82df1", "name": "Newcastle", "place_type": "city", "bounding_box": rectangle("-121.164586,38.83937 -121.115485,38.879613") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 651000, "cityName": "Newcastle" } }
+{ "create_at": datetime("2016-04-12T00:00:03.000Z"), "id": 719782115907252224, "text": "If I had a dollar every time some one asked about my race, I'd be Rich AF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419948146, "name": "Kinaa Marie", "screen_name": "ToniMarie117", "lang": "en", "location": "Phx, Arizona ", "create_at": date("2011-11-23"), "description": "20 | Az | GCU | IG: KinaMariie | SC: tonimarie1716", "followers_count": 720, "friends_count": 454, "statues_count": 14327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782116016463872, "text": "Wind 1.0 mph WSW. Barometer 30.016 in, Rising. Temperature 61.9 °F. Rain today 0.44 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 60988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782116242911233, "text": "#sushi with my boy his last night (@ Umi Sake House in Seattle, WA w/ @moeriyami) https://t.co/ZrGnD7wb3e https://t.co/asZVZ5wyTn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.34602183,47.6132696"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sushi" }}, "user_mentions": {{ 201860897 }}, "user": { "id": 31267110, "name": "junior Eluhu", "screen_name": "sweluhu", "lang": "en", "location": "iPhone: 30.175048,-85.664351", "create_at": date("2009-04-14"), "description": "NC State University Alumni - Electrical & Computer Engineering | Cerner Corporation - Devreg1:/ # whoami -- root | Gadget Freak.", "followers_count": 219, "friends_count": 250, "statues_count": 1802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782116637155329, "text": "LIT https://t.co/e6UYXcTCva", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 129403616, "name": "Sebas (Which / One)", "screen_name": "seabreezy46", "lang": "en", "location": "Sebastian, FL", "create_at": date("2010-04-03"), "description": "Yeezy SZN is in full effect #RockySZN", "followers_count": 457, "friends_count": 400, "statues_count": 7506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782116666580992, "text": "Par Auto Parts, Airdrie, AB, T4A 2G8 - https://t.co/XQbSd17xvw\n\n#parautoparts #serviceshop #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.991038,51.29636"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "parautoparts", "serviceshop", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 439, "friends_count": 432, "statues_count": 298 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Airdrie, Alberta", "id": "46cfc7fda28c4d88", "name": "Airdrie", "place_type": "city", "bounding_box": rectangle("-114.048657,51.241238 -113.969823,51.308137") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782116750274560, "text": "Girl on my floor: \"This dorm is a cesspool\" you should definitely see the weekends then.. good lord", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506939459, "name": "daniel", "screen_name": "danielpoonslayr", "lang": "en", "location": "Washington State University", "create_at": date("2012-02-27"), "description": "it ain't Ralph tho", "followers_count": 390, "friends_count": 297, "statues_count": 2593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782117035614208, "text": "#Trump in 2016 Temp:44.8°F Wind:0.0mph Pressure: 30.12hpa Rising Rain Today 0.08in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 314968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782117706567681, "text": "@Truman_Black luv u bby boi https://t.co/O35st7gP49", "in_reply_to_status": -1, "in_reply_to_user": 227324947, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 227324947 }}, "user": { "id": 2655047622, "name": "angel", "screen_name": "angeImtz", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "J", "followers_count": 115, "friends_count": 106, "statues_count": 1593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calexico, CA", "id": "93b15c1a419833c4", "name": "Calexico", "place_type": "city", "bounding_box": rectangle("-115.5336,32.662133 -115.459983,32.701648") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 609710, "cityName": "Calexico" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782117979398144, "text": "��F4FB ����\n\n@sparklebabies2\n@clupko ��\n@Azulolho4835\n@bojanek_riri\n@RobStroup6\n@freakyyy_com\n@yagurlnika\n@Me_Gofficial https://t.co/aHvtdPGb6t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 719373342004740097, 28507777, 27919628, 2897598341, 4590174269, 455689579, 47271556, 2784170003 }}, "user": { "id": 938330827, "name": "ClaudyThe Artist", "screen_name": "ClaudyTheArtist", "lang": "en", "location": "Tampa, Florida USA", "create_at": date("2012-11-09"), "description": "Contemporary Impressionistic Artist // Composer // World Critic", "followers_count": 141946, "friends_count": 152696, "statues_count": 224718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "553011a282a18fb5", "name": "University", "place_type": "city", "bounding_box": rectangle("-82.45491,28.054805 -82.409812,28.098918") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118025342977, "text": "I tagged nipsey hussle instead of sal ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455043153, "name": "J", "screen_name": "jasleenxm", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-05-24"), "description": "null", "followers_count": 705, "friends_count": 396, "statues_count": 31582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118293811200, "text": "HBD @mvilenerose! ����������We kick it everyday so you already what it is���� https://t.co/CdcFzSyc9H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1110638288 }}, "user": { "id": 1942277532, "name": "( ੭ ・ᴗ・ )੭", "screen_name": "thejtrizzles", "lang": "en", "location": "San Jose", "create_at": date("2013-10-06"), "description": "carpe omnia", "followers_count": 493, "friends_count": 425, "statues_count": 13029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118302220289, "text": "@yagirl_lizzy hot wings cafe", "in_reply_to_status": 719781468470255616, "in_reply_to_user": 2988238477, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2988238477 }}, "user": { "id": 2441242561, "name": "Lawn", "screen_name": "Londagod_", "lang": "en", "location": "Playboy Mansion ", "create_at": date("2014-04-13"), "description": "Call me Lon or Lonnie idc .", "followers_count": 1047, "friends_count": 890, "statues_count": 36619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118319112192, "text": "03:00 34.7°F Feels:34.7°F (Hi38.3°F/Lo34.7°F) Hum:80% Wnd:--- 0.0MPH Baro:29.89in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 197, "friends_count": 262, "statues_count": 28305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118474260480, "text": "Real \"LOYALTY\" is hard to find �� if you do... Don't cheat yourself and do sum fuck shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3397847278, "name": "ར ɖɨƈᏦ ɦɛʀ ɖօառ", "screen_name": "Shutuplilthot", "lang": "en", "location": "Trap House ", "create_at": date("2015-07-31"), "description": "just a fly brother talking facts and making jokes.... No instagram OR facebook if yall see me its a fake!! snapchat: slim-tatted #NoChillCrew", "followers_count": 2047, "friends_count": 1781, "statues_count": 4817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118834851840, "text": "On another note, today was a good day ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2175502760, "name": "Sabrina N. Colegio", "screen_name": "sabrinajaberi", "lang": "en", "location": "Parlier, CA", "create_at": date("2013-11-04"), "description": "#STAYBLEST", "followers_count": 434, "friends_count": 322, "statues_count": 6151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parlier, CA", "id": "b3c579dbd6e6f98b", "name": "Parlier", "place_type": "city", "bounding_box": rectangle("-119.562738,36.603513 -119.51629,36.621062") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 655856, "cityName": "Parlier" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782118868389888, "text": "Temp: 28.8°F - Dew Point: 18.2° - Wind: 3.0 mph - Gust: 4.5 - Rain Today: 0.00in. - Pressure: 30.18in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782119443197952, "text": "@annieebeeee I was in too deep when I first made my Pinterest and I didn't wanna go through and make a new board and aerate everything ��", "in_reply_to_status": 719772778354741248, "in_reply_to_user": 491046393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 491046393 }}, "user": { "id": 219914009, "name": "Elli Giacomazzi ♡", "screen_name": "ElliGiacomazzi", "lang": "en", "location": "Auburn University", "create_at": date("2010-11-25"), "description": "fashionably late", "followers_count": 672, "friends_count": 383, "statues_count": 27823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, AL", "id": "deb349182b3f42bb", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-85.569969,32.532449 -85.413112,32.662041") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1081, "countyName": "Lee", "cityID": 103076, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782119770173441, "text": "MY DMS ARE INSANELY OPEN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1611290058, "name": "same", "screen_name": "sarah_lawwwd", "lang": "en", "location": "null", "create_at": date("2013-07-21"), "description": "pro-happiness. semi-charmed life. body luv. self luv. luv rules.", "followers_count": 2521, "friends_count": 699, "statues_count": 116628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron, TX", "id": "922185539a1a7a3b", "name": "Cameron", "place_type": "city", "bounding_box": rectangle("-97.003013,30.841928 -96.958853,30.882807") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48331, "countyName": "Milam", "cityID": 4812040, "cityName": "Cameron" } }
+{ "create_at": datetime("2016-04-12T00:00:04.000Z"), "id": 719782120021819392, "text": "@politcalparrots This is just smart and the DNC and state parties benefit", "in_reply_to_status": 719780301191249920, "in_reply_to_user": 4900603814, "favorite_count": 0, "coordinate": point("-103.32680733,34.18321452"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4900603814 }}, "user": { "id": 562595617, "name": "Cheri Quinn", "screen_name": "DrCheriQuinn", "lang": "en", "location": "null", "create_at": date("2012-04-24"), "description": "Professor, concerned citizen, left of Jesus.", "followers_count": 368, "friends_count": 597, "statues_count": 8134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portales, NM", "id": "d38e64abf084c996", "name": "Portales", "place_type": "city", "bounding_box": rectangle("-103.365502,34.156721 -103.309031,34.200853") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35041, "countyName": "Roosevelt", "cityID": 3559260, "cityName": "Portales" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782120256839680, "text": "Wind 1.4 mph SSE. Barometer 29.824 in, Falling Rapidly. Temperature 48.1 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782120378355712, "text": "@Joan_senesh @BlissTabitha @3rdeye49 America,so sad,sold down the river by its own politicians just for a few more dollars...ABSOLUTE GREED!", "in_reply_to_status": 719367693636407296, "in_reply_to_user": 712732447599947778, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 712732447599947778, 999453985, 2153666664 }}, "user": { "id": 576478396, "name": "ARTURO", "screen_name": "acard090", "lang": "en", "location": "Texas, USA", "create_at": date("2012-05-10"), "description": "LipanApache,USNVet(Honor, Courage&Commitment).Ret.Tx.Master LEO.Grandma's words of adviseAlways worshipJesusChrist &work hard,never be a sloth or a beggar.", "followers_count": 487, "friends_count": 746, "statues_count": 7318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782120399351809, "text": "The nighttime setting on the iphone is so nice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276861040, "name": "Jared Bair", "screen_name": "JaredBair_", "lang": "en", "location": "San Diego, Ca", "create_at": date("2011-04-04"), "description": "The focus of a monk with none of the empathy", "followers_count": 341, "friends_count": 315, "statues_count": 4055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782120734859266, "text": "if i find the bitch though, you know these hoes on twitter & ig look nothing like they do in person.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3740640073, "name": "Zaibaa Fakiri", "screen_name": "2aibaa", "lang": "en", "location": "heartbreak hotel", "create_at": date("2015-09-30"), "description": "فتاة حزينة للحياة", "followers_count": 263, "friends_count": 199, "statues_count": 4249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dublin, CA", "id": "003253f0107acd32", "name": "Dublin", "place_type": "city", "bounding_box": rectangle("-121.979522,37.697406 -121.849535,37.733656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 620018, "cityName": "Dublin" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782121686962176, "text": "Wind 0.0 mph ---. Barometer 30.106 in, Steady. Temperature 52.0 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 267455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782121699577856, "text": "@Gladdxo twas weird af", "in_reply_to_status": 719780217070288896, "in_reply_to_user": 1388275332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1388275332 }}, "user": { "id": 2673446184, "name": "ang", "screen_name": "angms_", "lang": "en", "location": "sc//:angxxmm", "create_at": date("2014-07-23"), "description": "miserable & magical.", "followers_count": 458, "friends_count": 166, "statues_count": 9837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782122009919488, "text": "And the entire London-Wall Street financial system of the trans-Atlantic countries is heading toward a complete collapse. -Lyndon LaRouche", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151552406, "name": "Richard O'Donnell", "screen_name": "Bartonesque", "lang": "en", "location": "null", "create_at": date("2010-06-03"), "description": "Charlemagne Is the Head of Gold (Daniel 2:31-45)", "followers_count": 2651, "friends_count": 2651, "statues_count": 75585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782122043482112, "text": "I'm gonna have some massive problems w/ my brain if I don't stop having nightmares.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229561622, "name": "Jacob Rushing", "screen_name": "jakeuff", "lang": "en", "location": "null", "create_at": date("2010-12-22"), "description": "I'm probably reading, drawing or gaming. \nLife is an incessant struggle for existence.\nAsk me to recite pi lol.", "followers_count": 164, "friends_count": 294, "statues_count": 1819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782122563633152, "text": "@dannyx416 damn were you high ��", "in_reply_to_status": 719781786226548736, "in_reply_to_user": 372058544, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 372058544 }}, "user": { "id": 394372741, "name": "seth. Ü", "screen_name": "sethdiaz_", "lang": "en", "location": "San Antonio, TX - Odessa, TX", "create_at": date("2011-10-19"), "description": "you is kind, you is smart, you is important. @SierraDiaz_ is my twin. | SPURS NATION.", "followers_count": 1489, "friends_count": 742, "statues_count": 66955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782122945388544, "text": "Just posted a video @ The Village Underground NYC https://t.co/JAhMec9kzO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0009079,40.7306099"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 49187382, "name": "MAURICE SMITH", "screen_name": "MOEBETTAR", "lang": "en", "location": "BROOKLYN, NY", "create_at": date("2009-06-20"), "description": "Singer/ Praise & Worship Leader/ Songwriter/Performer FACEBOOK:MOEBETTAR/ MYSPACE:MAURICE SMITH/ IG:MAURICE_SMITH", "followers_count": 690, "friends_count": 843, "statues_count": 7387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782123205443584, "text": "@iMile_ user?", "in_reply_to_status": 719781320868634624, "in_reply_to_user": 353233995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353233995 }}, "user": { "id": 1071515168, "name": "Dylan F/A", "screen_name": "Xytonic", "lang": "en", "location": "Murda Mitten, USA", "create_at": date("2013-01-08"), "description": "Competitve FPS Player • @OnslaughtClan #pake Bae: ?", "followers_count": 7701, "friends_count": 782, "statues_count": 22699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northview, MI", "id": "ee3de4f6fdd7de3e", "name": "Northview", "place_type": "city", "bounding_box": rectangle("-85.659488,43.024098 -85.543489,43.06309") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2658945, "cityName": "Northview" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782123591307264, "text": "My mom just said at least I got one child correct and looks at my sister... Wtf is that supposed to mean?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 545316106, "name": "Teyanna", "screen_name": "renae_teyanna", "lang": "en", "location": "Arizona", "create_at": date("2012-04-04"), "description": "suh dude. ASU '19 TGOG⛳♌", "followers_count": 570, "friends_count": 356, "statues_count": 33508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apache Junction, AZ", "id": "bf09d4c99c2d845c", "name": "Apache Junction", "place_type": "city", "bounding_box": rectangle("-111.587098,33.378739 -111.469058,33.465988") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 402830, "cityName": "Apache Junction" } }
+{ "create_at": datetime("2016-04-12T00:00:05.000Z"), "id": 719782124010668032, "text": "Can you recommend anyone for this #security #job? https://t.co/yKrcY9ygTl #infosec #GreenwoodVillage, CO #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.0969343,38.2050032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "security", "job", "infosec", "GreenwoodVillage", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 299, "friends_count": 160, "statues_count": 584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado, USA", "id": "e21c8e4914eef2b3", "name": "Colorado", "place_type": "admin", "bounding_box": rectangle("-109.060257,36.992427 -102.041524,41.003445") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8027, "countyName": "Custer" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782124425863168, "text": "@briadunnn and how much you got left ? Cause you need a good night sleep.", "in_reply_to_status": 719781860818071552, "in_reply_to_user": 1885350769, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1885350769 }}, "user": { "id": 2291378581, "name": "PrettyBlack_", "screen_name": "RoRoLicious_", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2014-01-14"), "description": "Scorpion.♏️", "followers_count": 906, "friends_count": 903, "statues_count": 11054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782124614627328, "text": "@jaymowingurlawn omg where the fuck have you been?!", "in_reply_to_status": 719752616368955392, "in_reply_to_user": 716150092843974656, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 716150092843974656 }}, "user": { "id": 420470339, "name": "✨Shannon✨", "screen_name": "shannbabyy22", "lang": "en", "location": "50 shades of fucked up", "create_at": date("2011-11-24"), "description": "lifes a bitch but so am i", "followers_count": 887, "friends_count": 753, "statues_count": 46210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782124677521408, "text": "Happy bday Maddie, have a good day�� https://t.co/lA3HnHjs0k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2233984627, "name": "logan", "screen_name": "loganbathke", "lang": "en", "location": "null", "create_at": date("2013-12-06"), "description": "Oregon Football Commit", "followers_count": 620, "friends_count": 236, "statues_count": 1285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782124924964864, "text": "same. https://t.co/I8hYnUlwu5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352852490, "name": "von", "screen_name": "VonBroo", "lang": "en", "location": "757. ", "create_at": date("2011-08-10"), "description": "SC: von_broo", "followers_count": 3102, "friends_count": 2537, "statues_count": 72506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782124987875328, "text": "@aamalki76 ادخل الهاشتاق واقرا", "in_reply_to_status": 719777679206182917, "in_reply_to_user": 475188436, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 475188436 }}, "user": { "id": 2734175413, "name": "مبارك المهنديⓂ️", "screen_name": "bin_ma6wi", "lang": "en", "location": "null", "create_at": date("2014-08-15"), "description": "(( الله نُورُ السّمَاوآتِ والأرضْ))", "followers_count": 189, "friends_count": 175, "statues_count": 463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782126019682305, "text": "Wind 3.2 mph NNE. Barometer 30.061 in, Rising slowly. Temperature 60.9 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-12T00:00:06.000Z"), "id": 719782126149832704, "text": "#SuspiciousPerson at #7Eleven, 938 W Colonial Dr. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3929567,28.552591"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousPerson", "7Eleven", "orlpol", "opd" }}, "user": { "id": 39061451, "name": "Police Calls 32804", "screen_name": "orlpol32804", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 562, "friends_count": 1, "statues_count": 50830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144487243120640, "text": "I almost shat when I saw that. Finished in 1 sitting. #90sKid @BreezyPuffs @netflix", "in_reply_to_status": 719750574292348929, "in_reply_to_user": 189019079, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "90sKid" }}, "user_mentions": {{ 189019079, 16573941 }}, "user": { "id": 480620032, "name": "Dante Marcy", "screen_name": "DanteShmante", "lang": "en", "location": "Disneyland on Crack, NV", "create_at": date("2012-02-01"), "description": "Actor. Improvisist. Funny man. Bartender. Nerd. Sploosh.", "followers_count": 200, "friends_count": 675, "statues_count": 800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144487700258816, "text": "@Mariee_Loveee it's always raining in Alabama & Mississippi & it be hot as hell I always see cars over heating on the side of the road ��", "in_reply_to_status": 720143811175788545, "in_reply_to_user": 51875908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51875908 }}, "user": { "id": 1628316638, "name": "M.", "screen_name": "MsMESHELLVEE", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "20 | HTX | Viet & Trini", "followers_count": 1398, "friends_count": 584, "statues_count": 53199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144487968743424, "text": "My quick trip to the library turned into an hour long trip ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368169154, "name": "krob", "screen_name": "kaileyrobinson1", "lang": "en", "location": "Pacific Lutheran University ", "create_at": date("2011-09-04"), "description": "Embrace the glorious mess that you are", "followers_count": 886, "friends_count": 654, "statues_count": 42610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parkland, WA", "id": "69d2c4f187635997", "name": "Parkland", "place_type": "city", "bounding_box": rectangle("-122.469677,47.118559 -122.389204,47.161211") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5353335, "cityName": "Parkland" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144488321056768, "text": "@ogchubby did i ask?", "in_reply_to_status": 720144237061357568, "in_reply_to_user": 1856292230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1856292230 }}, "user": { "id": 604764226, "name": "sickassvero", "screen_name": "yoniggavero", "lang": "en", "location": "null", "create_at": date("2012-06-10"), "description": "amat senior | 7.11.14 ♡ | i LOVE dogs especially mine | phhhoto: veronicalyss_", "followers_count": 710, "friends_count": 604, "statues_count": 25865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144488790790145, "text": "I want to fight you mentally, emotionally and sexually.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278853962, "name": "princessaaidni ✨", "screen_name": "_iloveyoufood", "lang": "en", "location": "null", "create_at": date("2011-04-07"), "description": "Get money by any means", "followers_count": 1149, "friends_count": 1172, "statues_count": 48114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144488987910144, "text": "i can't wait to go back to El Salvador and spend another 1-2 months over there ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2558300159, "name": "sara", "screen_name": "sarauhling", "lang": "en", "location": "null", "create_at": date("2014-05-21"), "description": "18", "followers_count": 355, "friends_count": 202, "statues_count": 10434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489080184832, "text": "Honestly contemplating moving back home...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4169405892, "name": "winona ryder", "screen_name": "xthrashlee", "lang": "en", "location": "null", "create_at": date("2015-11-08"), "description": "does anyone even read this?", "followers_count": 384, "friends_count": 271, "statues_count": 2821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alta Sierra, CA", "id": "63acbaab4eec6ae7", "name": "Alta Sierra", "place_type": "city", "bounding_box": rectangle("-121.110186,39.09943 -121.028434,39.192114") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 601360, "cityName": "Alta Sierra" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489222914048, "text": "Lisamarie Joyce can get it any day of the week������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544004434, "name": "Jeremy⚾️Arzuaga(6-1)", "screen_name": "jeremyarzuaga2", "lang": "en", "location": "The Friendly Confines", "create_at": date("2012-04-02"), "description": "RIPPapa|SanLorenzoPR✈️ChiTownIL|DieHardCubsFan|#GoCubsGo|#DreamVille|Ye|OVO|YoungTilla|JuicyJ|Weeknd|Wiz|Travi$|YeezySeason|#SeeRed|NWA|BHS'16 EIU'20|", "followers_count": 527, "friends_count": 391, "statues_count": 47702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carol Stream, IL", "id": "1aa7803ca6707875", "name": "Carol Stream", "place_type": "city", "bounding_box": rectangle("-88.179339,41.887811 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489575088128, "text": "\"Bro she look like a peanut\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2358564228, "name": "Federico Gabriel", "screen_name": "federicong_", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": "null", "followers_count": 102, "friends_count": 76, "statues_count": 1640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489600266240, "text": "T.Is family cause they look so fun lmao https://t.co/346uv7UiMa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3240514970, "name": "samsam", "screen_name": "samsamamohamud", "lang": "en", "location": "null", "create_at": date("2015-06-08"), "description": "3900", "followers_count": 837, "friends_count": 668, "statues_count": 18623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489764036608, "text": "Wind 0.0 mph ---. Barometer 30.13 in, Falling slowly. Temperature 57.9 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144489789009920, "text": "You just look dumb in the end", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268134032, "name": "iNDIKA", "screen_name": "sambamlocc", "lang": "en", "location": "blowing up", "create_at": date("2011-03-17"), "description": "BASEBOY. Insta & SC equals Indikasam JIGGYNIGGAiBeenIcinSWAGSEXUAL✌️ https://m.soundcloud.com/sambam-2/india-waseem", "followers_count": 1770, "friends_count": 1026, "statues_count": 51316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144490023923712, "text": "@Yank_76 @balnibarbi bye my sweet", "in_reply_to_status": 720143762681270272, "in_reply_to_user": 254561731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 254561731, 134649253 }}, "user": { "id": 376481110, "name": "✨ashley✨", "screen_name": "deckthisout", "lang": "en", "location": "MINNEAPOLIS", "create_at": date("2011-09-19"), "description": "shouting fall out boy to the heavens and working for the park board + anime con life", "followers_count": 563, "friends_count": 298, "statues_count": 12367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144490984521729, "text": "Sucking Dick for 5 hours!!!!!!! Who's got that much power in them? https://t.co/u7eqY6ts0a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29902828, "name": "Stevie not Steebie ", "screen_name": "StevieSoFetch_", "lang": "en", "location": "LONG ISLAND, NY . $$", "create_at": date("2009-04-08"), "description": "Aries ♈️|@TamarBraxtonHer STAN. | Instagram : StevieSoFetch_ | Snapchat : StevieS0Fetch #TheShadyBunch", "followers_count": 10854, "friends_count": 3824, "statues_count": 426875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inwood, NY", "id": "edd2aa964fa61ec9", "name": "Inwood", "place_type": "city", "bounding_box": rectangle("-73.77069,40.610349 -73.731496,40.632508") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 3637583, "cityName": "Inwood" } }
+{ "create_at": datetime("2016-04-13T00:00:00.000Z"), "id": 720144491072602112, "text": "Ripley SW Limestone Co. Temp: 49.3°F Wind:3.1mph Pressure: 1000.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144491609399296, "text": "@kayyfiree HOW THE FUCK", "in_reply_to_status": 720091397408169985, "in_reply_to_user": 124660227, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 124660227 }}, "user": { "id": 283279325, "name": "nicotine fiend", "screen_name": "SickxImpatient", "lang": "en", "location": "Michigan // Iowa", "create_at": date("2011-04-16"), "description": "I just wanna break you down so badly. @spoopygrandma", "followers_count": 2326, "friends_count": 514, "statues_count": 94931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garner, IA", "id": "2a91eda4814fc291", "name": "Garner", "place_type": "city", "bounding_box": rectangle("-93.616451,43.083149 -93.588309,43.113425") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19081, "countyName": "Hancock", "cityID": 1929955, "cityName": "Garner" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144492225892352, "text": "@Natdaddy__ and I are so out of shape we just did 50 squats and I feel like I'm about to go into a cardiac arrest hahahhaha", "in_reply_to_status": -1, "in_reply_to_user": 381224328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 381224328 }}, "user": { "id": 195239069, "name": "Carina Amador", "screen_name": "Careena_96", "lang": "en", "location": "SoCal-East Coast ", "create_at": date("2010-09-25"), "description": "Aries 19", "followers_count": 559, "friends_count": 468, "statues_count": 30589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escondido, CA", "id": "00610f4d3a382ec1", "name": "Escondido", "place_type": "city", "bounding_box": rectangle("-117.146102,33.057782 -116.998535,33.182936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622804, "cityName": "Escondido" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144492477706240, "text": "Temp 29.4° Hi/Lo 33.5/29.4 Rng 4.1° WC 29.4° Hmd 92% Rain 0.00\" Storm 0.19\" BAR 30.317 Rising DP 27.4° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144493152894976, "text": "@ebbtideapp Tide in Oceanic Bridge, New Jersey 04/13/2016\n Low 9:37am 0.1\nHigh 3:08pm 3.4\n Low 9:46pm 0.4\nHigh 3:26am 3.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.015,40.3767"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 51, "friends_count": 1, "statues_count": 24012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Jersey, USA", "id": "65b4760a2b411e11", "name": "New Jersey", "place_type": "admin", "bounding_box": rectangle("-75.563587,38.788657 -73.88506,41.357424") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3465130, "cityName": "Rumson" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144493761048576, "text": "Wind 3.0 mph SSW. Barometer 1024.83 mb, Steady. Temperature 48.9 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144494448934912, "text": "@JakeAndrew24 damn even worse, you gonna cry in front of everybody lol", "in_reply_to_status": 720143979426095104, "in_reply_to_user": 353927756, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 353927756 }}, "user": { "id": 474329628, "name": "Jonathan Bautista", "screen_name": "JohnBautista23", "lang": "en", "location": "Houston, Tx ", "create_at": date("2012-01-25"), "description": "6'2 Hispanic! #INSTAGRAM: JonathanBautista23 #SnapChat -JohnBautista23 MANAGER/HEAD PROMOTER AT CLUB FUEGO, HMU FOR MORE INFO!! #FUEGOPROMO", "followers_count": 1874, "friends_count": 1535, "statues_count": 54690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144494667108353, "text": "Temp: 59.8°F Wind:1.0mph Pressure: 30.103hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144494960750592, "text": "Wind 0.0 mph ---. Barometer 30.266 in, Falling. Temperature 41.3 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-13T00:00:01.000Z"), "id": 720144495069773824, "text": "Wind 1.0 mph ENE. Barometer 30.327 in, Falling slowly. Temperature 39.9 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144495673745408, "text": "04/13@03:00 - Temp 39.8F, WC 39.8F. Wind 0.2mph NW, Gust 2.0mph. Bar 30.359in, Rising slowly. Rain 0.00in. Hum 64%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144495778476032, "text": "Now it's officially #MambaDay. Unreal that this is Kobe's last game. Always hated Kobe because he was Kobe, but always respected a great.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MambaDay" }}, "user": { "id": 2779043736, "name": "Devon", "screen_name": "devonharipal", "lang": "en", "location": "Queens, NY", "create_at": date("2014-08-29"), "description": "| SFSU '19 | Hip-Hop & Sports Connoisseur | Snapchat & IG: devonharipal |", "followers_count": 313, "friends_count": 163, "statues_count": 4306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496168607745, "text": "Im getting better I'm in practice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 591298593, "name": "JT", "screen_name": "tweet_king2015", "lang": "en", "location": "null", "create_at": date("2012-05-26"), "description": "All Glory To God #Longlivethekidd\n#TAMUC", "followers_count": 1027, "friends_count": 846, "statues_count": 14666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bossier City, LA", "id": "018617530de2ac1c", "name": "Bossier City", "place_type": "city", "bounding_box": rectangle("-93.74855,32.419303 -93.584206,32.630042") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22015, "countyName": "Bossier", "cityID": 2208920, "cityName": "Bossier City" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496315371520, "text": "it's fascinating how people can tweet and act like I never texted them.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3285864956, "name": "zoë", "screen_name": "itszcrew_", "lang": "en", "location": "702", "create_at": date("2015-07-20"), "description": "stay paid, stay pretty.", "followers_count": 492, "friends_count": 338, "statues_count": 4721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496390877184, "text": "65.0F (Feels: 65.0F) - Humidity: 94% - Wind: 3.8mph E - Gust: 6.0mph - Pressure: 1012.1mb - Rain: 0.01\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 234942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496516681728, "text": "#Trump in 2016 Temp:37.6°F Wind:0.0mph Pressure: 30.27hpa Steady Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 315060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496646688768, "text": "Rt☹️ https://t.co/vLmnuyGaed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 550669276, "name": "♡Justine♡", "screen_name": "SimplyJustinee", "lang": "en", "location": "null", "create_at": date("2012-04-10"), "description": "null", "followers_count": 1897, "friends_count": 972, "statues_count": 48426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496885805056, "text": "I'm too litty rn��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335096314, "name": "تحب", "screen_name": "a1quanso", "lang": "en", "location": "NJ✈️LA", "create_at": date("2011-07-13"), "description": "#ALLAHUAKBAR #IG : MR.A1JEFE", "followers_count": 5833, "friends_count": 970, "statues_count": 131096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144496927838208, "text": "@QuanC_ @killitkiyyy_ Y'all Lil Ass Heads ����", "in_reply_to_status": 720144338601316352, "in_reply_to_user": 554239441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 554239441, 35240342 }}, "user": { "id": 865580564, "name": "Stuckalicious", "screen_name": "StuckeyLove", "lang": "en", "location": "Bite Me ", "create_at": date("2012-10-06"), "description": "ShawU19, Nj ✈ NC Diamond Sensation Dancer, SC:Raniaya RipJaybff❤❤ Miss 1923", "followers_count": 1777, "friends_count": 1581, "statues_count": 55558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144497183600640, "text": "Never underestimate those who you scar\nCause karma, karma, karma comes back to you hard!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19692555, "name": "Nick Simmons", "screen_name": "NorthwestNick", "lang": "en", "location": "LA", "create_at": date("2009-01-28"), "description": "Snap/Insta: NorthwestNick | Kim & Kanye named North after me | record label guy | lover of bacon | not related to Russell or Gene | #TheU", "followers_count": 3553, "friends_count": 917, "statues_count": 22642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144497384955905, "text": "wow. a snickers sounds soopp good right now ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316466994, "name": "Mariah Balmaceda", "screen_name": "mariahhhbeee", "lang": "en", "location": "Honolulu, HI", "create_at": date("2011-06-13"), "description": "what it do, baby boo? #kj ❥", "followers_count": 593, "friends_count": 366, "statues_count": 12430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144498156838912, "text": "������ https://t.co/FPQVUkRxzV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 569238776, "name": "Amaηda ♢", "screen_name": "amanda_stoinich", "lang": "en", "location": "null", "create_at": date("2012-05-02"), "description": "Åℓαs₭α ☄ sc & insta - amanda_stoinich", "followers_count": 407, "friends_count": 320, "statues_count": 15192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144498454503424, "text": "@jcyrusmusic COOL BAND, BRO", "in_reply_to_status": -1, "in_reply_to_user": 155322130, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155322130 }}, "user": { "id": 708646108591824896, "name": "annoying af lol", "screen_name": "CaseyBrittsan", "lang": "en", "location": "Wilton, CA", "create_at": date("2016-03-12"), "description": "null", "followers_count": 181, "friends_count": 966, "statues_count": 1178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144498676752384, "text": "Forget about past failures", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2992663357, "name": "Sam Cooper", "screen_name": "scoooop4", "lang": "en", "location": "null", "create_at": date("2015-01-22"), "description": "I just wanna travel the world", "followers_count": 231, "friends_count": 238, "statues_count": 1780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144499498958849, "text": "Wind 0.0 mph ---. Barometer 30.056 in, Steady. Temperature 61.1 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-13T00:00:02.000Z"), "id": 720144499532570625, "text": "@balagopalks @jetairways @WeAreTrivandrum @ShashiTharoor Direct connectivity is a must 2 promote IT & domestic tourism @IndiGo6E", "in_reply_to_status": 720080722044686337, "in_reply_to_user": 43828556, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43828556, 14918367, 2725318658, 24705126, 1467228402 }}, "user": { "id": 35181416, "name": "Ajay Prasad", "screen_name": "Ajaypp", "lang": "en", "location": "Cambridge, MA", "create_at": date("2009-04-25"), "description": "Trivandrumite | Development Evangelist | Country Managing Director, India at Taurus Investment Holdings | MIT IIMC CET Alum", "followers_count": 1108, "friends_count": 666, "statues_count": 807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cambridge, MA", "id": "8193d87541f11dfb", "name": "Cambridge", "place_type": "city", "bounding_box": rectangle("-71.160356,42.352429 -71.064398,42.403966") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2511000, "cityName": "Cambridge" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144500140687360, "text": "Para @sport, @tv3cat, @mundodeportivo, @Rac1, @diariARA, @elpuntavui, @govern y el interminable etcétera:\n\nhttps://t.co/0PzCVkoKIu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 19343904, 28373820, 15748165, 18577646, 169538164, 12723412, 145670919 }}, "user": { "id": 2349941124, "name": "(Meri)", "screen_name": "Meritxell2014", "lang": "es", "location": "Fargo.", "create_at": date("2014-02-18"), "description": "Catalana, española y europea porque nací en Barcelona. Lo único que tengo es lo que he dado. Creo.", "followers_count": 236, "friends_count": 389, "statues_count": 4392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, FL", "id": "2bc7c264a080898b", "name": "Hollywood", "place_type": "city", "bounding_box": rectangle("-80.248975,25.986228 -80.107808,26.093132") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1232000, "cityName": "Hollywood" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144500266405888, "text": "@Remarkable_____ exactly ! & I gotta get up early ��", "in_reply_to_status": 720142077598629888, "in_reply_to_user": 3150553970, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3150553970 }}, "user": { "id": 100847159, "name": "rae.", "screen_name": "AshleyReshun", "lang": "en", "location": "Houston, TX", "create_at": date("2009-12-31"), "description": "#PVAMU19 • R.I.P Daddy •. 19", "followers_count": 451, "friends_count": 406, "statues_count": 14019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144500950052864, "text": "Sometimes I think I should do nail art videos for instagram but then I remember I'm not as talented as everyone on it so I get over it lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537850860, "name": "Anahi ♥", "screen_name": "anahilynn", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "20 | UTRGV | Pedro Maciel", "followers_count": 210, "friends_count": 155, "statues_count": 5190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144501273141248, "text": "You already know.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1606769736, "name": "Mia.", "screen_name": "ncriherd", "lang": "en", "location": "Claremore, OK", "create_at": date("2013-07-19"), "description": "Insta and SC: mia_riherd", "followers_count": 728, "friends_count": 299, "statues_count": 19243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Claremore, OK", "id": "b92d59dc197635bf", "name": "Claremore", "place_type": "city", "bounding_box": rectangle("-95.686645,36.264238 -95.517488,36.344263") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40131, "countyName": "Rogers", "cityID": 4014700, "cityName": "Claremore" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144502061543425, "text": "How can you live a life full of hate & negativity??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 942928902, "name": "Catalina", "screen_name": "CatMelgo12", "lang": "en", "location": "Kansas City", "create_at": date("2012-11-11"), "description": "19 || College Freshman.", "followers_count": 548, "friends_count": 844, "statues_count": 11582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, KS", "id": "1276a44d783a529d", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-96.649744,39.159985 -96.499443,39.259872") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20161, "countyName": "Riley", "cityID": 2044250, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144502170648580, "text": "@babygoatt https://t.co/fdF5E2GUpo", "in_reply_to_status": 720142444642197504, "in_reply_to_user": 3389610561, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3389610561 }}, "user": { "id": 48240246, "name": "kaydubz", "screen_name": "KayDubz_", "lang": "en", "location": "CA", "create_at": date("2009-06-17"), "description": "psych - that's a potato gun", "followers_count": 895, "friends_count": 420, "statues_count": 30946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144502371913728, "text": "Wind 3.2 mph NE. Barometer 30.069 in, Falling slowly. Temperature 53.9 °F. Rain today 0.06 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144502594228224, "text": "Chicken https://t.co/QW5CZJcj5c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3142784592, "name": "Antoine", "screen_name": "TrocheManuel23", "lang": "en", "location": "null", "create_at": date("2015-04-06"), "description": "Happiness is not looking at a price tag and just buying it", "followers_count": 462, "friends_count": 505, "statues_count": 2712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144502728450048, "text": "Being broke is seriously the worst shit ever!! Smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47154199, "name": "Cindy G.", "screen_name": "cindy1017_", "lang": "en", "location": "Pomona, CA", "create_at": date("2009-06-14"), "description": "null", "followers_count": 712, "friends_count": 551, "statues_count": 68739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144503332425728, "text": "Couldn't be any happier #grateful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "grateful" }}, "user": { "id": 714537011269341184, "name": "Faduma", "screen_name": "officialfadumaa", "lang": "en", "location": "San Diego, CA", "create_at": date("2016-03-28"), "description": "aspiring journalist ✍|fashionista|Young East African Queen| #ripdad #ripgrandpa", "followers_count": 238, "friends_count": 135, "statues_count": 69 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144503730999296, "text": "#SuspiciousPerson at 2022-2199 Buchanan Bay Cir. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4075497,28.4936699"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousPerson", "orlpol", "ocso" }}, "user": { "id": 39065901, "name": "Police Calls 32839", "screen_name": "orlpol32839", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 151, "friends_count": 1, "statues_count": 40947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-13T00:00:03.000Z"), "id": 720144503789649920, "text": "I had to cute Tierney off as my friend tonight . #RIP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RIP" }}, "user": { "id": 3188308380, "name": "Dj YungBake♈️", "screen_name": "DjYungbake_", "lang": "en", "location": "Fuckin Yo Friends w/o u knowin", "create_at": date("2015-05-07"), "description": "DJ • GFX Designer | Bookings: djyungbake@gmail.com | #FlyNewLifestyle | IG: @dj_yungbake | L's are for Learning, Not Losing | #BLM✊", "followers_count": 637, "friends_count": 494, "statues_count": 14722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144504049651713, "text": "FaReal man Watch out on some real shyt I'm not with that gay shyt you doing too much now Nigga I don't role that way https://t.co/KFXuPbS6Es", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4083176474, "name": "Scooba", "screen_name": "Young_Scooba07", "lang": "en", "location": "Claremore, OK", "create_at": date("2015-10-31"), "description": "I claim it right Now 2016 Champs-OKC#ThunderUp 55-27 (Go to RSU) SnapChat(Scooba0007", "followers_count": 130, "friends_count": 132, "statues_count": 298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Claremore, OK", "id": "b92d59dc197635bf", "name": "Claremore", "place_type": "city", "bounding_box": rectangle("-95.686645,36.264238 -95.517488,36.344263") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40131, "countyName": "Rogers", "cityID": 4014700, "cityName": "Claremore" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144504154562560, "text": "@MattJGoodwin sike", "in_reply_to_status": 720144338072834048, "in_reply_to_user": 1012391400, "favorite_count": 0, "retweet_count": 0, "lang": "no", "is_retweet": false, "user_mentions": {{ 1012391400 }}, "user": { "id": 621482941, "name": "C A M", "screen_name": "CamsInPdx", "lang": "en", "location": "Portland, OR", "create_at": date("2012-06-28"), "description": "18 | PCC | Photographer | Portland Trailblazers | United States Marine Corps", "followers_count": 773, "friends_count": 156, "statues_count": 16503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144504179847168, "text": "Wind 0.7 mph W. Barometer 30.36 in, Rising slowly. Temperature 33.1 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 159023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144504188219393, "text": "Sally Friar, Jamaica Plain, MA, 02130 - https://t.co/BsiWIDBVEH\n\n#sallyfriar #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.112799,42.317453"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sallyfriar", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 442, "friends_count": 432, "statues_count": 322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144504922193920, "text": "Nah some things I have to speak on. https://t.co/8jnHh0bbcS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417399480, "name": "#FREE9", "screen_name": "istealTWINKIES", "lang": "en", "location": "Homicide Hartford ", "create_at": date("2011-11-20"), "description": "i got some problems only lord knows️", "followers_count": 2276, "friends_count": 1782, "statues_count": 50639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartford, CT", "id": "61c225139f635563", "name": "Hartford", "place_type": "city", "bounding_box": rectangle("-72.718386,41.723759 -72.643547,41.807475") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937000, "cityName": "Hartford" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505014480896, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 15:58 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13849 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505027051520, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 15:55 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13849 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505035493376, "text": "1 Yellow-rumped Warbler (Myrtle x Audubon's) (Setophaga coronata) - My Patch - 2016-04-12 17:00 https://t.co/DPLjNucq7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.4787998,33.3952608"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13849 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505048055811, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 18:57 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13849 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505056423936, "text": "1 Broad-billed Hummingbird (Cynanthus latirostris) - Needle Rock Recreation Area - 2016-04-12 13:45 https://t.co/Peb2sS4Use", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.6755104,33.7585986"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13849 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505245044736, "text": "@REWROTE bro...i feel the sadness and despair in this post a thousand miles away....", "in_reply_to_status": 720098789726429184, "in_reply_to_user": 20825793, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20825793 }}, "user": { "id": 548839616, "name": "Chamberlain", "screen_name": "YungOrca", "lang": "en", "location": "Los Angeles CA and Portland OR", "create_at": date("2012-04-08"), "description": "MANAGEMENT (Booking/General Info): Info@JAMSWORLDmgmt.com", "followers_count": 653, "friends_count": 873, "statues_count": 8042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505811275776, "text": "Happy Birthday baby girl, shhhaaooowww!❤️ @losario_R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 814287409 }}, "user": { "id": 2165676938, "name": "ѵίɕ", "screen_name": "_VictorMendez_", "lang": "en", "location": "Ukiah, CA", "create_at": date("2013-10-30"), "description": "null", "followers_count": 737, "friends_count": 269, "statues_count": 17081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ukiah, CA", "id": "0742517d53e00505", "name": "Ukiah", "place_type": "city", "bounding_box": rectangle("-123.232051,39.116505 -123.18786,39.174311") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6045, "countyName": "Mendocino", "cityID": 681134, "cityName": "Ukiah" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505857552384, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 11:10 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13854 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505916104704, "text": "\"I just wanna give everyone a special individual hug just for them so they know they're special- QUIT LAUGHING IM BEIN REAL.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770207168, "name": "Katie", "screen_name": "Kativanius", "lang": "en", "location": "New Mexico, USA", "create_at": date("2012-08-20"), "description": "I am the long worlds gentleman, and I share my bed with Capricorn and Cancer.", "followers_count": 180, "friends_count": 357, "statues_count": 2514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144505979211777, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 08:25 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13854 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506184671232, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-12 06:45 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13855 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506201489408, "text": "1 House Finch (Common) (Haemorhous mexicanus) - BWestern 2425 S 24th St. Phoenix - 2016-04-12 07:00 https://t.co/4eFriPL9Iq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.0283154,33.4240732"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13855 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506230706177, "text": "@IAmRocioRivera lolololol ����", "in_reply_to_status": 720143947473952768, "in_reply_to_user": 216942505, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 216942505 }}, "user": { "id": 305386331, "name": "V", "screen_name": "Vianettt", "lang": "en", "location": "null", "create_at": date("2011-05-25"), "description": "The past cannot be changed, forgotten, edited, or erased; it can only be accepted.❤", "followers_count": 200, "friends_count": 160, "statues_count": 7579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506302177280, "text": "7 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-11 15:54 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13856 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506381668352, "text": "National Kobe Day #24 #8 #BlackMamba #MambaDay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackMamba", "MambaDay" }}, "user": { "id": 379247806, "name": "Jonathan Aguayo", "screen_name": "xJonnyPonnyx", "lang": "en", "location": "San Jose, CA", "create_at": date("2011-09-24"), "description": "que", "followers_count": 263, "friends_count": 403, "statues_count": 4844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506461401089, "text": "I turned up for a check ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237490111, "name": "A.waaaassh ☀️", "screen_name": "xoxo_dreahh", "lang": "en", "location": "snapchat : xo.dreahh", "create_at": date("2011-01-12"), "description": "19 , long story short I'm the shit .", "followers_count": 1943, "friends_count": 1191, "statues_count": 27830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506469756928, "text": "never enough time in the day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1582401319, "name": "Max Murray", "screen_name": "maximusprimed", "lang": "en", "location": "iammaxmurray@gmail.com", "create_at": date("2013-07-09"), "description": "null", "followers_count": 297, "friends_count": 304, "statues_count": 1135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506667069441, "text": "Facts https://t.co/aWYhoTHGau", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30992427, "name": "nelly", "screen_name": "NellyBabyyyy", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-13"), "description": "19. Made in Jamaica. Brownskin Beauty. Future Nurse Anesthesist. #GSU18", "followers_count": 1628, "friends_count": 729, "statues_count": 47922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506683813888, "text": "COME MANGO WITH US ����It's your One-Stop-Shop for all your Hookah/Accessory needs #mangoscafe… https://t.co/ropJZi9uCs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0063171,39.9881401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mangoscafe" }}, "user": { "id": 383010151, "name": "Mango's Cafe", "screen_name": "MangosCafecbus", "lang": "en", "location": "21 Smith Place. Columbus, OHIO", "create_at": date("2011-09-30"), "description": "google us & check out what others have to say 6144293694 http://www.mangoscafe.orgus⬇️ https://www.facebook.com/pages/Mangos-Cafe/108546569254369", "followers_count": 10577, "friends_count": 502, "statues_count": 28600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506734161920, "text": "1 Canvasback (Aythya valisineria) - Salt River--Granite Reef Recreation Area - 2016-04-11 08:35 https://t.co/XfMO275JPO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.6830875,33.5138839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13858 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506755133440, "text": "The Noise Presents NOFX - Hepatitis Bathtub Tour @ The UC Theater https://t.co/O0XxwRZodA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.269959,37.871601"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99802706, "name": "Luis Campos II", "screen_name": "luis_campos2", "lang": "en", "location": "Bay Area, California", "create_at": date("2009-12-27"), "description": "null", "followers_count": 51, "friends_count": 234, "statues_count": 941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506759286784, "text": "3 Olive Warbler (Peucedramus taeniatus) - Mt. Ord--FR 1688 - 2016-04-11 09:00 https://t.co/NORtBqvc1Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.4143384,33.9211769"), "retweet_count": 0, "lang": "da", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13858 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144506834796544, "text": "https://t.co/RYhSTsgXK4 \nHow did I miss this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53297859, "name": "annjeanette", "screen_name": "ajaymarie", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-07-02"), "description": "you get the W2, you get the W9, but where do you get 3-8?", "followers_count": 764, "friends_count": 1125, "statues_count": 7221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Skokie, IL", "id": "4b3758bc32868265", "name": "Skokie", "place_type": "city", "bounding_box": rectangle("-87.780869,42.0032 -87.708543,42.065033") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1770122, "cityName": "Skokie" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507069530113, "text": "\"Shut up Megan, just because you have no friends...\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2281555591, "name": "marvin", "screen_name": "sandinyoface", "lang": "en", "location": "Lakewood, WA", "create_at": date("2014-01-07"), "description": "dog mom", "followers_count": 320, "friends_count": 329, "statues_count": 4766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198337") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507128430592, "text": "2 Gray Vireo (Vireo vicinior) - Mt. Ord--lower (Maricopa Co.) - 2016-04-11 07:30 https://t.co/xu5gBgWGSs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.4428341,33.9171171"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13860 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507266850816, "text": "1 Vaux's Swift (Chaetura vauxi) - Glendale Recharge Ponds - 2016-04-11 06:45 https://t.co/Pjgn5s5LMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13860 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507325517824, "text": "#podcast 838 #shoutouts to @FloridaMarc @PhantasOrlando We play #nerd #geek #trivia & have a sample of #otr #radio https://t.co/GOG6xNpaZ7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "podcast", "shoutouts", "nerd", "geek", "trivia", "otr", "radio" }}, "user_mentions": {{ 19153078, 2775954659 }}, "user": { "id": 98288086, "name": "Riley and Kimmy Show", "screen_name": "RileyandKimmy", "lang": "en", "location": "Orlando, Florida", "create_at": date("2009-12-20"), "description": "Daily Podcasts about all things Nerd Freaky & Fun: Comic Books, Superheroes, Cosplay Sci-Fi Movies TV Horror Action Figures/Toys visit: http://t.co/ILGA7TyebC", "followers_count": 1184, "friends_count": 1077, "statues_count": 10981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507346554880, "text": "5 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-11 06:45 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13860 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507442896896, "text": "@KingGerrad what do you drink it with ?", "in_reply_to_status": 720144174872285184, "in_reply_to_user": 82192399, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82192399 }}, "user": { "id": 460696537, "name": "stank daddy", "screen_name": "KatanaKez", "lang": "en", "location": "Las Vegas", "create_at": date("2012-01-10"), "description": "i ain't shy or quiet. im just chillin bitch 神", "followers_count": 1314, "friends_count": 322, "statues_count": 22344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507476557824, "text": "Wind 0.2 mph WNW. Barometer 30.349 in, Rising Rapidly. Temperature 31.8 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 9991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507560402945, "text": "2 Nanday Parakeet (Aratinga nenday) - Nanday Parakeet stakeout, 90th Place, Chandler - 2016-04-10 09:53 https://t.co/GUnvToJRTy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.8883812,33.3366057"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13861 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507602243586, "text": "Temp: 36.0°F - Dew Point: 26.8° - Wind: 6.0 mph - Gust: 8.9 - Rain Today: 0.00in. - Pressure: 30.34in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507757395968, "text": "@lanadelcunt @PitbullIsKing let's go meet at our secret headquarters. It's that time", "in_reply_to_status": -1, "in_reply_to_user": 972859459, "favorite_count": 0, "coordinate": point("-117.128046,32.709359"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 972859459, 2309469320 }}, "user": { "id": 66060399, "name": "p r i s c i l l a", "screen_name": "HECHlCERA", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-08-15"), "description": "IG: yaval_", "followers_count": 5654, "friends_count": 137, "statues_count": 16070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slave Abduction Headquarters", "id": "07d9e48716887001", "name": "Slave Abduction Headquarters", "place_type": "poi", "bounding_box": rectangle("-117.12804609999999,32.7093589 -117.128046,32.709359") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 616378, "cityName": "Coronado" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507912597505, "text": "Gang Gang Gang up in here once again @Jae_Patin @EMurphjr @Claydale138", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 521932720, 1153324322, 527090972 }}, "user": { "id": 441415725, "name": "Del", "screen_name": "FidelMartinez_1", "lang": "en", "location": "Ridgecrest, CA", "create_at": date("2011-12-19"), "description": "⚽️", "followers_count": 703, "friends_count": 978, "statues_count": 22286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgecrest, CA", "id": "17c023bb615a7e48", "name": "Ridgecrest", "place_type": "city", "bounding_box": rectangle("-117.72304,35.573286 -117.624503,35.666117") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 660704, "cityName": "Ridgecrest" } }
+{ "create_at": datetime("2016-04-13T00:00:04.000Z"), "id": 720144507967246336, "text": "2 Gray Hawk (Buteo plagiatus) - Hassayampa River Preserve - 2016-04-10 08:30 https://t.co/CEjtgJtO2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.693,33.933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13864 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508130881542, "text": "1 Red-shouldered Hawk (Buteo lineatus) - Hassayampa River Preserve - 2016-04-10 08:30 https://t.co/xOB8yIQkc2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.693,33.933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13864 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508210425856, "text": "@Monstersx33 I thought u was about that Greek life I was trying to celebrate and break the cup duh", "in_reply_to_status": 720144382276407296, "in_reply_to_user": 332750384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332750384 }}, "user": { "id": 3059128507, "name": "Junior", "screen_name": "JRtoobig34", "lang": "en", "location": "null", "create_at": date("2015-03-02"), "description": "the gym is the only love in my life #gainz you'll always be the one for me ❤️", "followers_count": 295, "friends_count": 525, "statues_count": 7073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508248129536, "text": "It's officially Mamba Day...�� #413", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3040948705, "name": "Jacob", "screen_name": "ygk_jacobito", "lang": "en", "location": "null", "create_at": date("2015-02-24"), "description": "Philippians 4:13 | Alicia❤️", "followers_count": 257, "friends_count": 239, "statues_count": 5815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508256722944, "text": "1 Yellow-rumped Warbler (Myrtle x Audubon's) (Setophaga coronata) - Tempe Town Lake - 2016-04-10 08:15 https://t.co/DPLjNucq7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9186044,33.4332333"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13865 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508323774464, "text": "1 Vaux's Swift (Chaetura vauxi) - Tempe Town Lake - 2016-04-10 08:15 https://t.co/Pjgn5s5LMt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.9186044,33.4332333"), "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13866 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508613226496, "text": "I have to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 111698229, "name": "madisunnnn ☀️", "screen_name": "MsBehavingMad", "lang": "en", "location": "null", "create_at": date("2010-02-05"), "description": "she got a body & she's naughty.", "followers_count": 1163, "friends_count": 1140, "statues_count": 50313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508747411456, "text": "5 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-10 07:45 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13868 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-13T00:00:05.000Z"), "id": 720144508856442881, "text": "5 Wilson's Phalarope (Phalaropus tricolor) - Glendale Recharge Ponds - 2016-04-09 17:00 https://t.co/F7qmsyP87u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 2, "friends_count": 0, "statues_count": 13868 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506875096276992, "text": "I'm at @Wingstop in Tempe, AZ https://t.co/JddzEEfCNE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.92547607,33.43461919"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89858807 }}, "user": { "id": 252237204, "name": "Mohammed Hamad", "screen_name": "MHAlamiri", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-02-14"), "description": "I mean it..", "followers_count": 129, "friends_count": 71, "statues_count": 7794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506875847053312, "text": "@andrewtylenol @djangobitch1 it was pretty awkward ��������", "in_reply_to_status": 720506643616817152, "in_reply_to_user": 3244757526, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3244757526, 3028045231 }}, "user": { "id": 242041046, "name": "Katie Baker", "screen_name": "KatieSavesLives", "lang": "en", "location": "null", "create_at": date("2011-01-23"), "description": "There is sunshine in my soul today", "followers_count": 404, "friends_count": 248, "statues_count": 20046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial, CA", "id": "e6aaf7cb58e7857c", "name": "Imperial", "place_type": "city", "bounding_box": rectangle("-115.594963,32.817908 -115.544378,32.876462") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 636280, "cityName": "Imperial" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506875968675840, "text": "The NBA isn't going to be same w/out u. Congrats on 60 tonight Kobe!! @Lakers #MambaOut #Kobe #KB20 ���������� https://t.co/mIYtldZxwz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MambaOut", "Kobe", "KB20" }}, "user_mentions": {{ 20346956 }}, "user": { "id": 1898962592, "name": "Bri", "screen_name": "BriannaOard", "lang": "en", "location": "LA ", "create_at": date("2013-09-23"), "description": "adventure", "followers_count": 111, "friends_count": 96, "statues_count": 354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506876291588097, "text": "Temp 31.9° Hi/Lo 33.5/31.7 Rng 1.8° WC 31.9° Hmd 88% Rain 0.00\" Storm 0.00\" BAR 30.339 Falling DP 28.8° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506876476174336, "text": "Really need some dutches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1647987667, "name": "AtƱAk", "screen_name": "___tiz", "lang": "en", "location": "Location Off", "create_at": date("2013-08-05"), "description": "#RIPBREE #RIPMEAT #FREEMILKMAN", "followers_count": 524, "friends_count": 396, "statues_count": 13601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Haven, CT", "id": "795003fb11ee9829", "name": "New Haven", "place_type": "city", "bounding_box": rectangle("-72.998069,41.246404 -72.860248,41.350384") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506877017243648, "text": "Ripley SW Limestone Co. Temp: 58.6°F Wind:0.0mph Pressure: 998.4mb Falling slowly Rain Today 0.01in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 51984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506877340164097, "text": "@ScHoolBoyQ snaps be having me dead! xD", "in_reply_to_status": -1, "in_reply_to_user": 22736017, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 22736017 }}, "user": { "id": 711406560228945920, "name": "Anthony Alejo", "screen_name": "thewhale_34", "lang": "en", "location": "South Central Los Angeles, CA", "create_at": date("2016-03-19"), "description": "Diego Rivera / Wayne State / Compton CC", "followers_count": 291, "friends_count": 718, "statues_count": 1564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wayne, NE", "id": "78235f6b3fb2469d", "name": "Wayne", "place_type": "city", "bounding_box": rectangle("-97.037277,42.224239 -96.989749,42.250055") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31179, "countyName": "Wayne", "cityID": 3151840, "cityName": "Wayne" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506877726048256, "text": "https://t.co/Yw8F2wDYre", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2187447985, "name": "KAMAL", "screen_name": "kamalthefunman", "lang": "en", "location": "Marina del Rey, CA USA", "create_at": date("2013-11-10"), "description": "male", "followers_count": 3244, "friends_count": 4999, "statues_count": 50826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marina del Rey, CA", "id": "38ea9782c8d83a4b", "name": "Marina del Rey", "place_type": "city", "bounding_box": rectangle("-118.463481,33.964019 -118.432199,33.98647") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645806, "cityName": "Marina del Rey" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506877847670785, "text": "Last game feel like Kobe had to ball out. Pussy so good I forgot to pull out ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1836393090, "name": "Ozzy", "screen_name": "yzzzzo", "lang": "en", "location": "Brownsville, TX", "create_at": date("2013-09-08"), "description": "Siempre soy lindo. I dj in Brownsville weekly. Basically... I'm a catch.", "followers_count": 289, "friends_count": 425, "statues_count": 10802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506878460080128, "text": "@Nyla__tishe come my child and I shall show you", "in_reply_to_status": 720505789299998720, "in_reply_to_user": 1651812979, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1651812979 }}, "user": { "id": 1691489310, "name": "Camille", "screen_name": "camyb00", "lang": "en", "location": "At the bottom of a tulip bed", "create_at": date("2013-08-22"), "description": "The feeler's, feel her", "followers_count": 177, "friends_count": 268, "statues_count": 2181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506878522966018, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 63.1 °F. Rain today 0.09 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506878703349761, "text": "@mersaaydes happy birthday ��", "in_reply_to_status": -1, "in_reply_to_user": 478268355, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 478268355 }}, "user": { "id": 3305769584, "name": "austin hardy", "screen_name": "AustinHardy20", "lang": "en", "location": "Pullman, WA", "create_at": date("2015-08-03"), "description": "WAZZU | 19 | Packers | ConAgra Farms", "followers_count": 252, "friends_count": 261, "statues_count": 657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-04-14T00:00:00.000Z"), "id": 720506878749487105, "text": "03:00:01 |Temp: 57.1ºF |Dew Point 53.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NE, Gusting to 1.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506879567347716, "text": "Todo hasta un día.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1546635588, "name": "Mister Cap", "screen_name": "MisaelCap__", "lang": "es", "location": "Manhattan, NY", "create_at": date("2013-06-25"), "description": "null", "followers_count": 779, "friends_count": 215, "statues_count": 37024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506879802241024, "text": "@chavezjoshuaiii but he shot 44% !", "in_reply_to_status": 720496039527104512, "in_reply_to_user": 4648677390, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4648677390 }}, "user": { "id": 3213048267, "name": "Hendrixx", "screen_name": "Davonted11x", "lang": "en", "location": "null", "create_at": date("2015-04-27"), "description": "10•15 Army strong", "followers_count": 297, "friends_count": 300, "statues_count": 2082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale, AZ", "id": "0015d9147cee6907", "name": "Avondale", "place_type": "city", "bounding_box": rectangle("-112.357999,33.384785 -112.272424,33.493806") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 404720, "cityName": "Avondale" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506879915487233, "text": "The baby not mine but this fit was clean https://t.co/YZwdxwPxXA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 993815623, "name": "pvpi champû", "screen_name": "hollywood_jay", "lang": "en", "location": "The Throne", "create_at": date("2012-12-06"), "description": "null", "followers_count": 629, "friends_count": 331, "statues_count": 40201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506880028733440, "text": "This is not who I am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2833724991, "name": "hetha", "screen_name": "_anuhealoha", "lang": "en", "location": "Makakilo, HI", "create_at": date("2014-10-16"), "description": "God wasn't done working on you when you gave up", "followers_count": 226, "friends_count": 210, "statues_count": 6482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Makakilo, HI", "id": "3aff180d3a9e53d5", "name": "Makakilo", "place_type": "city", "bounding_box": rectangle("-158.115354,21.332074 -158.057413,21.396068") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1547600, "cityName": "Makakilo" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506880825696256, "text": "Wind 2.0 mph S. Barometer 1022.52 mb, Steady. Temperature 51.0 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506881169629184, "text": "04/14@03:00 - Temp 36.4F, WC 36.4F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.327in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 67, "friends_count": 8, "statues_count": 48424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506881723232256, "text": "@cbryant_21 well that timer takes really great pictures..", "in_reply_to_status": 720506243937386496, "in_reply_to_user": 1927098811, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1927098811 }}, "user": { "id": 589807602, "name": "Haylee", "screen_name": "Hayleee_Mariee", "lang": "en", "location": "null", "create_at": date("2012-05-24"), "description": "Proverbs 31:25", "followers_count": 632, "friends_count": 483, "statues_count": 7514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506881781997568, "text": "formation in a few weeks https://t.co/Uim3WlGk5N", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1332400526, "name": "twenty88", "screen_name": "2sassy4u123", "lang": "en", "location": "on an ultralight beam", "create_at": date("2013-04-06"), "description": "i'm tryna keep my faith.....but i'm looking for more.....somewhere i can feel safe.....and end my holy war. UCLA17", "followers_count": 810, "friends_count": 485, "statues_count": 41934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882281107456, "text": "Your past mistakes are meant to guide you, not define you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142860977, "name": "Nikki Falash", "screen_name": "nikkifalash", "lang": "en", "location": "Washington, USA", "create_at": date("2010-05-11"), "description": "I'm exotic. Like ranch dressing", "followers_count": 174, "friends_count": 143, "statues_count": 10209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Felida, WA", "id": "3990e08a4d4cdafe", "name": "Felida", "place_type": "city", "bounding_box": rectangle("-122.734598,45.693162 -122.682473,45.733492") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5323550, "cityName": "Felida" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882377588737, "text": "���������� https://t.co/MYB72dnBax", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 181413781, "name": "Black Is Beautiful", "screen_name": "KingTaddx", "lang": "en", "location": "#AppleCity #SpiceLn #YVGGA", "create_at": date("2010-08-21"), "description": "At least I'm not a ordinary ass guy putting some ignorant shit here", "followers_count": 2006, "friends_count": 1926, "statues_count": 149884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882528538624, "text": "@tater182 soy pendejo amigos", "in_reply_to_status": 720495095557058560, "in_reply_to_user": 371132984, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 371132984 }}, "user": { "id": 371132984, "name": "Andrew Hernández", "screen_name": "tater182", "lang": "en", "location": "Seaside, CA", "create_at": date("2011-09-10"), "description": "When you do things right, people won't be sure you've done anything at all", "followers_count": 188, "friends_count": 302, "statues_count": 15483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seaside, CA", "id": "89bac4213b1b5525", "name": "Seaside", "place_type": "city", "bounding_box": rectangle("-121.860374,36.596815 -121.789746,36.654798") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882822168577, "text": "I can't wait till I get to cut someone open like I can just imagine the adrenaline rush I'll be getting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411837879, "name": "Emely", "screen_name": "BaezEmely", "lang": "en", "location": "NYC•", "create_at": date("2011-11-13"), "description": "Ig: anumati__", "followers_count": 1676, "friends_count": 591, "statues_count": 31992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882826366976, "text": "Lmfaoo https://t.co/gJ5ylXf3Ph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1917473324, "name": "Bam", "screen_name": "killaxcam_", "lang": "en", "location": "704", "create_at": date("2013-09-29"), "description": "kush to face, that's the antidote", "followers_count": 1089, "friends_count": 591, "statues_count": 59205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882918494208, "text": "Wind 0.0 mph ---. Barometer 30.415 in, Steady. Temperature 24.7 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506882947985408, "text": "#Trump in 2016 Temp:44.6°F Wind:0.0mph Pressure: 30.09hpa Falling Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 315143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506883140939777, "text": "Lol fuck long distance relationships man.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287331678, "name": "Lil", "screen_name": "Liliana_rose20", "lang": "en", "location": "Seattle washington ", "create_at": date("2011-04-24"), "description": "I'm that one good girl that's worth 1,000 bitches -Sustainability & Dogs- WWU", "followers_count": 209, "friends_count": 538, "statues_count": 4183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506883187077121, "text": "Wind 0.0 mph ---. Barometer 30.146 in, Steady. Temperature 47.0 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-14T00:00:01.000Z"), "id": 720506883199614976, "text": "بالله يا احبابنا عودوا لنا\nفقد كفى ماقد جرى من أدمعي!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 482088849, "name": "MAB96", "screen_name": "M7MDAl3azmiiii", "lang": "en", "location": "Kuwait / USA, Ohio, Marietta", "create_at": date("2012-02-03"), "description": "Milan/ يوسف المطرف", "followers_count": 251, "friends_count": 149, "statues_count": 8006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marietta, OH", "id": "33833c3421cbb25e", "name": "Marietta", "place_type": "city", "bounding_box": rectangle("-81.502155,39.377658 -81.395312,39.491007") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39167, "countyName": "Washington", "cityID": 3947628, "cityName": "Marietta" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506883388403712, "text": "Didn't know I had so many rave friends till my snap chat timeline was floodEd by JackÜ stories.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 822713756, "name": "madeinus", "screen_name": "mvdeinus", "lang": "en", "location": "California, USA", "create_at": date("2012-09-13"), "description": "ᴇɴɪɢᴍᴀᴛɪᴄ ᴅʀᴀᴡɴ ᴛᴏ ᴀᴇsᴛʜᴇᴛɪᴄs", "followers_count": 724, "friends_count": 222, "statues_count": 80557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506883396800512, "text": "same asf https://t.co/v6tKMpvW3Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42346076, "name": "AC", "screen_name": "_TripleOGangsta", "lang": "en", "location": "null", "create_at": date("2009-05-24"), "description": "god . | pk . | lv ✈️ tx | sc: tripleogangsta3", "followers_count": 1266, "friends_count": 1195, "statues_count": 34006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506883489013762, "text": "@ebbtideapp Tide in Canaday Landing, South Carolina 04/14/2016\nHigh 6:56am 0.9\n Low 2:42pm 0.2\nHigh 7:36pm 0.9\n Low 3:03am 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-80.4067,32.8133"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 69, "friends_count": 1, "statues_count": 24285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Carolina, USA", "id": "6057f1e35bcc6c20", "name": "South Carolina", "place_type": "admin", "bounding_box": rectangle("-83.353955,32.04683 -78.499301,35.215449") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506883539410945, "text": "3:00 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 420068190, "name": "➰", "screen_name": "Sedaliaaa", "lang": "en", "location": "Detroit, MI", "create_at": date("2011-11-23"), "description": "SC: Sedaliaaa #WSU19", "followers_count": 1326, "friends_count": 784, "statues_count": 44238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westland, MI", "id": "879dfee0187b884a", "name": "Westland", "place_type": "city", "bounding_box": rectangle("-83.430458,42.267434 -83.308978,42.355177") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2686000, "cityName": "Westland" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506883774226432, "text": "Wind 4.0 mph ENE. Barometer 30.181 in, Falling slowly. Temperature 46.9 °F. Rain today 0.00 in. Humidity 58%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506884344713217, "text": "I don't know what I'm sitting around waiting for", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 909051602, "name": "Kirstie Johnson", "screen_name": "kirstiej2112", "lang": "en", "location": "Issaquah, WA", "create_at": date("2012-10-27"), "description": "snapchat: kirstiej2112", "followers_count": 577, "friends_count": 389, "statues_count": 807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Issaquah, WA", "id": "8ab9a87db23e5a16", "name": "Issaquah", "place_type": "city", "bounding_box": rectangle("-122.100358,47.508261 -121.984731,47.577367") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5333805, "cityName": "Issaquah" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506884759887872, "text": "21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 443103958, "name": "Ray Hudson", "screen_name": "rayhudson11", "lang": "en", "location": "Bay Area Raised ", "create_at": date("2011-12-21"), "description": "University of California Berkeley #11 #HoldTheRope", "followers_count": 1006, "friends_count": 211, "statues_count": 7450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506885024149504, "text": "Temp: 63.0°F Wind:0.0mph Pressure: 30.019hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506885166788609, "text": "@LoveeCinthyaa I looked for it too!!! I thought it was on Facebook but I guess not ��", "in_reply_to_status": 720502299400019968, "in_reply_to_user": 1340966689, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1340966689 }}, "user": { "id": 35930362, "name": "pervy sage", "screen_name": "kissmyana", "lang": "en", "location": "Sacramento State ", "create_at": date("2009-04-27"), "description": "“It’s foolish to fear what we’ve yet to see and know.”✨", "followers_count": 542, "friends_count": 135, "statues_count": 47220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearlake, CA", "id": "7c340bbe75b12e62", "name": "Clearlake", "place_type": "city", "bounding_box": rectangle("-122.699517,38.921813 -122.602268,38.994881") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6033, "countyName": "Lake", "cityID": 613945, "cityName": "Clearlake" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506886240518144, "text": "Why am I friends with this bitch... https://t.co/H8GnpqfRFx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2149426086, "name": "Jasleen", "screen_name": "jasleenjezelle", "lang": "en", "location": "OC", "create_at": date("2013-10-22"), "description": "There’s a little bit of devil in her angel eyes, she’s a little bit of heaven with a wild side.\nB. Paqua ❥", "followers_count": 222, "friends_count": 248, "statues_count": 10408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506886261489664, "text": "Wind 0.0 mph NW. Barometer 30.39 in, Steady. Temperature 34.0 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 159047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506886538313728, "text": "23415 Tirino Shores Dr, Katy, TX 77493, $258,122 3 beds, 2.5 baths https://t.co/2qFPg1fqWa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.8302002,29.74020004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864061615, "name": "Katy, TX News", "screen_name": "BLifeKaty", "lang": "en", "location": "Katy, Texas", "create_at": date("2014-11-06"), "description": "Katy BubbleLife features community news, photos and events. Share your business, organization or personal news and events at http://katytx.bubblelife.com.", "followers_count": 294, "friends_count": 1103, "statues_count": 22212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506886726901760, "text": "Im Gonna Do Whatever Keeps Me Happy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 475045899, "name": "❣Moe❣", "screen_name": "Mokaaaa____", "lang": "en", "location": "null", "create_at": date("2012-01-26"), "description": "♒️ SC:allwell_moe |Kayla's Bestfriend|", "followers_count": 2161, "friends_count": 1121, "statues_count": 66199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506886966145027, "text": "Anna just run away run faraway #G H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "G" }}, "user": { "id": 302788970, "name": "FBM", "screen_name": "FBM2005", "lang": "en", "location": "Pt. Charles Chronicles", "create_at": date("2011-05-21"), "description": "Because life in River City's a soap with a cast of thousands #IEStrong #StandwithSanBernardino", "followers_count": 566, "friends_count": 470, "statues_count": 64985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506887029047297, "text": "65.5F (Feels: 65.5F) - Humidity: 93% - Wind: 3.1mph E - Gust: 4.5mph - Pressure: 1009.4mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 235086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506887226179585, "text": "Can't sleep...yay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338121898, "name": "Kaylaa", "screen_name": "Kayla_Brooke15", "lang": "en", "location": "Nacogdoches, TX", "create_at": date("2011-07-18"), "description": "#SFA19 | Nineteen", "followers_count": 739, "friends_count": 495, "statues_count": 10895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-04-14T00:00:02.000Z"), "id": 720506887435915265, "text": "We know Jack... We know https://t.co/enfkrpQCk3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3109014954, "name": "Alé", "screen_name": "thisisjack96", "lang": "en", "location": "Lubbock, TX", "create_at": date("2015-03-26"), "description": "TTU C/O '19. embracing the inappropriate since 1996. kinda love music more than people tbh. ~ 9-4-15 ~ ❤ ~10-17-15~", "followers_count": 1167, "friends_count": 663, "statues_count": 6628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506887486181376, "text": "woohoo got waitlisted for housing next year.\nfuck.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2156135798, "name": "stephen", "screen_name": "LolclothingZX", "lang": "en", "location": "null", "create_at": date("2013-10-25"), "description": "Worst Zed Player NA™. I play League & Smash a lot. Melee: Marth/Falco SSB4: Sheik/Lucina Ahri/Kotori/Cordelia/Lucina/Felicia ♥ ~occasionally NSFW~", "followers_count": 120, "friends_count": 381, "statues_count": 14381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506887687548928, "text": "Wind 0.0 mph ---. Barometer 29.961 in, Falling. Temperature 63.2 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506887968595970, "text": "studying in bed and christine casually starts having a conversation with herself in her sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336731267, "name": "Kali Perfetti", "screen_name": "kaliperfettti", "lang": "en", "location": "null", "create_at": date("2011-07-16"), "description": "Cleveland | Ohio University", "followers_count": 1262, "friends_count": 638, "statues_count": 20792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, OH", "id": "77aa8a0640caa98c", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-82.173479,39.27973 -82.01933,39.361038") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39009, "countyName": "Athens", "cityID": 3902736, "cityName": "Athens" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506888266387456, "text": "Guess I'm partying with Persians next Friday ���� بیا بریم", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 24486438, "name": "Lana Minas", "screen_name": "LanaMinas", "lang": "en", "location": "LA+SD, California", "create_at": date("2009-03-14"), "description": "social media addict | self-proclaimed music video critic | reluctant college senior", "followers_count": 131, "friends_count": 99, "statues_count": 3276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506889293967360, "text": "Facts. https://t.co/GLCfGU32pe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 439796693, "name": "Nakamura Hiro ;", "screen_name": "_PhoenixKing", "lang": "en", "location": "JA ✈️NY", "create_at": date("2011-12-17"), "description": "enigmatic persona, twisted mind ; fwm. || Young entrepreneur, poet, fire dancer /breather.", "followers_count": 526, "friends_count": 653, "statues_count": 70606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890371903488, "text": "Jacklyn Joyce Nogan, Levittown, NY, 11756 - https://t.co/w976DUYNI4\n\n#jacklynjoycenogan #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.537405,40.724961"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jacklynjoycenogan", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 449, "friends_count": 432, "statues_count": 346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Levittown, NY", "id": "364a45c10832ed51", "name": "Levittown", "place_type": "city", "bounding_box": rectangle("-73.545679,40.699993 -73.484061,40.74357") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3642081, "cityName": "Levittown" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890472570880, "text": "Just posted a video @ Rusty Mullet https://t.co/S6YGJFVXn5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.335971,34.101919"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33272246, "name": "Marck Gomez", "screen_name": "fannum", "lang": "en", "location": " West Covina, 91792", "create_at": date("2009-04-19"), "description": "null", "followers_count": 153, "friends_count": 502, "statues_count": 2021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890476724224, "text": "This girl at work is such a fucking kiss ass to the managers ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3164599597, "name": "Julian", "screen_name": "king_julian____", "lang": "en", "location": "El Paso, TX", "create_at": date("2015-04-19"), "description": "great story in the making ✍", "followers_count": 132, "friends_count": 109, "statues_count": 38 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890669703169, "text": "What the?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1113360182, "name": "christi•ana", "screen_name": "christeanaaaaa", "lang": "en", "location": "null", "create_at": date("2013-01-22"), "description": "19 |-/ ig:christiana.maria", "followers_count": 589, "friends_count": 648, "statues_count": 11805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890707447808, "text": "Get on twitch see @LilyStellaRosa is streaming hyped and then she gets off... #FeelsBadMan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FeelsBadMan" }}, "user_mentions": {{ 2945668470 }}, "user": { "id": 438752597, "name": "Alec", "screen_name": "OneShotNeededTV", "lang": "en", "location": "California, USA", "create_at": date("2011-12-16"), "description": "Grad Student RU Alumnus, New streamer on twitch. I almost like to cook food as much as I like to eat it. #HouseTargaryen", "followers_count": 2452, "friends_count": 823, "statues_count": 7518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890757808128, "text": "@katie_bowennn keep goin babe❤️", "in_reply_to_status": 720501086554075136, "in_reply_to_user": 470402855, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 470402855 }}, "user": { "id": 404579074, "name": "THICKSHADY", "screen_name": "KalobCent", "lang": "en", "location": "KB❤️", "create_at": date("2011-11-03"), "description": "RIP James Thomas Grady. CARDED", "followers_count": 327, "friends_count": 212, "statues_count": 4802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albany, OR", "id": "6083b03ae37cd913", "name": "Albany", "place_type": "city", "bounding_box": rectangle("-123.159583,44.577589 -123.032415,44.684678") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4101000, "cityName": "Albany" } }
+{ "create_at": datetime("2016-04-14T00:00:03.000Z"), "id": 720506890933923841, "text": "Ms Badu followed me, Kobe dropped 60 in his last game, I'm laying next to my lady ...life's good ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23717054, "name": "Rellington Beats", "screen_name": "TyRellington", "lang": "en", "location": "Washington DC", "create_at": date("2009-03-10"), "description": "(official twitter of Rellington Beats) all business inquires send to Tyrelldc23@gmail.com @SyceGame #SBSB #TOOSWEET", "followers_count": 3250, "friends_count": 3312, "statues_count": 219123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506891818962944, "text": "@aintnodocta no", "in_reply_to_status": 720506262354599936, "in_reply_to_user": 246291678, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 246291678 }}, "user": { "id": 56519463, "name": "△▽", "screen_name": "illestflow", "lang": "en", "location": "null", "create_at": date("2009-07-13"), "description": "wassup ninj.", "followers_count": 599, "friends_count": 397, "statues_count": 22291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kings Park West, VA", "id": "011bcdef2abd93ca", "name": "Kings Park West", "place_type": "city", "bounding_box": rectangle("-77.321491,38.798964 -77.273962,38.827547") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5142680, "cityName": "Kings Park West" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892066418688, "text": "BFF! No doubt! #wifeysforlife #spring2016 #dirtythirties @ Playground Bar & Lounge https://t.co/ZNcy8n6SBN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.96730068,32.22196298"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wifeysforlife", "spring2016", "dirtythirties" }}, "user": { "id": 90428851, "name": "Tara E. Neal", "screen_name": "TaraENeal", "lang": "en", "location": "ÜT: 32.246775,-110.861286", "create_at": date("2009-11-16"), "description": "Loving living life.", "followers_count": 83, "friends_count": 225, "statues_count": 773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892250787840, "text": "Wind 1.6 mph WNW. Barometer 30.023 in, Falling slowly. Temperature 55.7 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892401950724, "text": "Rambo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1648416896, "name": "DANIEL", "screen_name": "Lackey28", "lang": "en", "location": "Oxnard, CA", "create_at": date("2013-08-05"), "description": "null", "followers_count": 643, "friends_count": 528, "statues_count": 7472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892443848704, "text": "That's a huge disappointment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 950792762, "name": "J", "screen_name": "joshhh_rush", "lang": "en", "location": "sc: realjoshuarush", "create_at": date("2012-11-15"), "description": "Senior LHS | Elyse is my girl❤️", "followers_count": 171, "friends_count": 263, "statues_count": 1856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892603301888, "text": "Ohhhh shit ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967519963, "name": "Brooke", "screen_name": "brookiemoss", "lang": "en", "location": "Alaska", "create_at": date("2012-11-23"), "description": "I got hot sauce in my bag, swag.", "followers_count": 237, "friends_count": 164, "statues_count": 5018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506892917846016, "text": "Boyyyyy do I hate when a huge chunk of my check is taken out for taxes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547059817, "name": "JB.", "screen_name": "JenaBasile", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2012-04-06"), "description": "20.", "followers_count": 1011, "friends_count": 653, "statues_count": 62332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Allis, WI", "id": "27e4a2a92b9c3f65", "name": "West Allis", "place_type": "city", "bounding_box": rectangle("-88.068598,42.980825 -87.975863,43.0314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5585300, "cityName": "West Allis" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506893349830656, "text": "@AlezaeJazzy I swear working in the food industry people get mad over the stupidest shit ������", "in_reply_to_status": 720500164214013952, "in_reply_to_user": 2904265807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2904265807 }}, "user": { "id": 1569351151, "name": "•Cassidy•", "screen_name": "CassidyMeadows", "lang": "en", "location": "null", "create_at": date("2013-07-04"), "description": "in love with @MattDelaney19 ❤️", "followers_count": 814, "friends_count": 151, "statues_count": 23689 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506894138351617, "text": "Temp: 35.6°F - Dew Point: 29.8° - Wind: ESE @ 5.1 mph - Gust: 6.3 - Rain Today: 0.00in. - Pressure: 30.21in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506894348128258, "text": "#GeneralDisturbance at 4422 Barley St. #orlpol #opd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.435169,28.540024"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GeneralDisturbance", "orlpol", "opd" }}, "user": { "id": 39049097, "name": "Police Calls 32811", "screen_name": "orlpol32811", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 263, "friends_count": 1, "statues_count": 93851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506894402646019, "text": "@andreaoverboard fucking much about an individual who doesn't think twice about whether I'm doing ok or if my feelings are hurt", "in_reply_to_status": 720504834177306624, "in_reply_to_user": 351999215, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351999215 }}, "user": { "id": 545482007, "name": "lil emotional baby", "screen_name": "dietxmtnxdew", "lang": "en", "location": "SD", "create_at": date("2012-04-04"), "description": "hearts too big for my body", "followers_count": 378, "friends_count": 262, "statues_count": 28267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506894754951170, "text": "Are you happy with yourself?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715159484, "name": "al1cat", "screen_name": "astovee", "lang": "en", "location": "anchorage ", "create_at": date("2012-07-24"), "description": "Econ major", "followers_count": 301, "friends_count": 234, "statues_count": 2674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506894826225664, "text": "@knarsu3 No man. There's no one who would get this level of reverence. Not yet. Not even Bron. This is 20 years. Man.", "in_reply_to_status": 720504371973398529, "in_reply_to_user": 281228552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 281228552 }}, "user": { "id": 296592157, "name": "AlexANDRE", "screen_name": "monsieuralexxx", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-05-10"), "description": "IG: monsieuralexmosaic \nOddish's doppelgänger.\nDominator of the Universe.", "followers_count": 272, "friends_count": 469, "statues_count": 17172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506895430238208, "text": "Let's be grown here kids", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2292272193, "name": "#MambaDay", "screen_name": "_DomKM", "lang": "en", "location": "Raleigh, NC", "create_at": date("2014-01-19"), "description": "chicken wingz & audizine", "followers_count": 1189, "friends_count": 963, "statues_count": 82386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506895669338112, "text": "That's a sensitive subject though.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352052368, "name": "A.J W.⚡️", "screen_name": "ItsAlwaysAJ", "lang": "en", "location": "Colorado, USA", "create_at": date("2011-08-09"), "description": "IG/SC @itsalwaysaj, college hooper, sneaker enthusiast.", "followers_count": 629, "friends_count": 792, "statues_count": 26656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alamosa, CO", "id": "fb4cac3f448732d3", "name": "Alamosa", "place_type": "city", "bounding_box": rectangle("-105.912016,37.440834 -105.856303,37.494073") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8003, "countyName": "Alamosa", "cityID": 801090, "cityName": "Alamosa" } }
+{ "create_at": datetime("2016-04-14T00:00:04.000Z"), "id": 720506895728041984, "text": "Annie thinks it's August", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1472696802, "name": "JessieFromTheBlock", "screen_name": "JessieMaloney3", "lang": "en", "location": "Macomb, IL Western Illinois ", "create_at": date("2013-05-31"), "description": "WIU Cheerleader || Snapchat: jessiemaloney3", "followers_count": 439, "friends_count": 655, "statues_count": 7934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Macomb, IL", "id": "1af01854e44e009a", "name": "Macomb", "place_type": "city", "bounding_box": rectangle("-90.72417,40.433688 -90.624248,40.48538") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17109, "countyName": "McDonough", "cityID": 1745889, "cityName": "Macomb" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506896097157120, "text": "This Lou Williams vs. Shady McCoy twitter beef is a hilarious end to the night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18971284, "name": "pat muldowney", "screen_name": "patmuldowney", "lang": "en", "location": "LA now, pittsburgh always", "create_at": date("2009-01-14"), "description": "sr. manager, @foxsports social. @jayanddan podcast. previously, @espn. snapchat & instagram: patmuldowney", "followers_count": 10049, "friends_count": 1862, "statues_count": 143497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506896805969920, "text": "literally so hungry I'm crying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2349937306, "name": "abs ✠", "screen_name": "a_opiee", "lang": "en", "location": "sc: yooitsopiee", "create_at": date("2014-02-17"), "description": "the darkest nights produce the brightest stars", "followers_count": 534, "friends_count": 979, "statues_count": 4735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzalez, FL", "id": "1a0cf025d0716f3d", "name": "Gonzalez", "place_type": "city", "bounding_box": rectangle("-87.325789,30.547501 -87.260678,30.619707") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1226700, "cityName": "Gonzalez" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506896965373952, "text": "3.1 magnitude #earthquake. 12 km from #Eureka, NV, United States https://t.co/FDGwXaX7EL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.85,39.581"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "Eureka" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 68503, "friends_count": 10, "statues_count": 101002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32033, "countyName": "White Pine" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506897170759680, "text": "@casaaaaandra @JaredHill96 DL", "in_reply_to_status": 720494790178140161, "in_reply_to_user": 380751334, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 380751334, 276386531 }}, "user": { "id": 337993942, "name": "fez", "screen_name": "hector_upm", "lang": "en", "location": "CA.", "create_at": date("2011-07-18"), "description": "CSUF. πΚα. INO #FantoFamily", "followers_count": 1442, "friends_count": 944, "statues_count": 40094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506897347035136, "text": "@anna_thomp8 @umfuckno @alexthompson14 no chance", "in_reply_to_status": 720415063140671488, "in_reply_to_user": 412375789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 412375789, 1685863872, 88852635 }}, "user": { "id": 404931107, "name": "Marteen Weeds #8", "screen_name": "JackWietlispach", "lang": "en", "location": "Norwood Park, Chicago", "create_at": date("2011-11-04"), "description": "Living the life of Christ, Trying to make the best of everyday. #8 from Chicago. Psalm 56:3 When I am afraid, I put my trust in you..#IAMSecond", "followers_count": 1159, "friends_count": 1002, "statues_count": 9458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orland Park, IL", "id": "0da8a6c990f02eed", "name": "Orland Park", "place_type": "city", "bounding_box": rectangle("-87.911936,41.552464 -87.790471,41.65669") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1756640, "cityName": "Orland Park" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506897774833664, "text": "i need a fucking gf.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321672539, "name": "stewart", "screen_name": "yahurdmeh_xx", "lang": "en", "location": "null", "create_at": date("2011-06-21"), "description": "21. #resteasyslice", "followers_count": 1669, "friends_count": 1010, "statues_count": 78527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506897959362561, "text": "So in love with this face https://t.co/stGtkleXwf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1207099321, "name": "☪Jesssss", "screen_name": "itsjay_jay", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2013-02-21"), "description": "... but if you walk the footsteps of a stranger, you'll learn things you never knew -Pocahontas", "followers_count": 243, "friends_count": 471, "statues_count": 6272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goleta, CA", "id": "0019e2618c34b6b2", "name": "Goleta", "place_type": "city", "bounding_box": rectangle("-119.913268,34.40581 -119.744123,34.473116") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 630378, "cityName": "Goleta" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898055897089, "text": "@ChrisCanRunn https://t.co/xMDasnzZT7", "in_reply_to_status": -1, "in_reply_to_user": 938052097, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 938052097 }}, "user": { "id": 1273107884, "name": "maegan", "screen_name": "_maegatron", "lang": "en", "location": "null", "create_at": date("2013-03-16"), "description": "Terminally chill. @ChrisCanRunn is my twin.", "followers_count": 337, "friends_count": 231, "statues_count": 9357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898156531712, "text": "What's luv?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336285967, "name": "♻️", "screen_name": "Sal_Adame", "lang": "en", "location": "probably @ a concert/event", "create_at": date("2011-07-15"), "description": "Concerts & Gym || snapchat: sal_adame", "followers_count": 616, "friends_count": 562, "statues_count": 23388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898345287681, "text": "@TacoBellGawd credit cards in the scannas", "in_reply_to_status": 720506249712766976, "in_reply_to_user": 167310339, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 167310339 }}, "user": { "id": 3228350234, "name": "ATM", "screen_name": "Austin_M80_", "lang": "en", "location": "tornado alley, OK", "create_at": date("2015-05-27"), "description": "|| I just found out my birthday is on the SAME day that I was born.... life is crazy man........|#OHB| (Average joes 1-0)", "followers_count": 2192, "friends_count": 1926, "statues_count": 34598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898349441024, "text": "I just don't even know how to feel right now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 814108070, "name": "Desiree Tekulve", "screen_name": "DesireeTekulve", "lang": "en", "location": "null", "create_at": date("2012-09-09"), "description": "null", "followers_count": 268, "friends_count": 376, "statues_count": 2525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898387230720, "text": "This Road Leads To Worlds Of Motion... @ Test Track https://t.co/AqUeuGtDs3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.54783118,28.37335043"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164761758, "name": "Marcus Womble", "screen_name": "mwepcot82", "lang": "en", "location": "Wichita Falls, TX", "create_at": date("2010-07-09"), "description": "Nurse, D23 Member, WDW 2005 CP, Disneyland AP, Whovian, Married in Vegas to Nichole & Father!", "followers_count": 468, "friends_count": 539, "statues_count": 14338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1204150, "cityName": "Bay Lake" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506898580180992, "text": "@PedroNeira_ Feliz jueves Pedro.. que lo disfrutes..!!!", "in_reply_to_status": 720476752578002949, "in_reply_to_user": 229777133, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 229777133 }}, "user": { "id": 1726104156, "name": "SusyM.", "screen_name": "susyvassallo", "lang": "en", "location": "california", "create_at": date("2013-09-03"), "description": "All you need is love... and a cup of coffee..Always forgive, but never forget. Learn from mistakes, but never regret.", "followers_count": 595, "friends_count": 620, "statues_count": 5297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506899314122753, "text": "If Kobe woulda dunked tonight I woulda went vegan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212100547, "name": "Troiter VVS Vega", "screen_name": "7mileTroiter", "lang": "en", "location": "Traplanta", "create_at": date("2010-11-04"), "description": "Detroit Made me Atlanta raised me", "followers_count": 524, "friends_count": 369, "statues_count": 6726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryville, TN", "id": "01092ff657add392", "name": "Maryville", "place_type": "city", "bounding_box": rectangle("-84.120656,35.663386 -83.85217,35.788977") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47009, "countyName": "Blount", "cityID": 4746380, "cityName": "Maryville" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506899754545155, "text": "Wind 5.0 mph S. Barometer 29.985 in, Falling. Temperature 47.1 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 267999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-14T00:00:05.000Z"), "id": 720506899981037568, "text": "@JeffAyresDaily Sorry short of 2m for my 20min wish4u.But I m still proud of u. Clips count on u 4great practice thru playoffs.GoJeff!", "in_reply_to_status": -1, "in_reply_to_user": 636979061, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 636979061 }}, "user": { "id": 1246999968, "name": "Candice Lee", "screen_name": "lee_candice8888", "lang": "en", "location": "null", "create_at": date("2013-03-06"), "description": "null", "followers_count": 12, "friends_count": 77, "statues_count": 255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506900073357312, "text": "These niggas got patron they're gonna be wasted lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284228637, "name": "s h e l b y", "screen_name": "soupbitchbaddd", "lang": "en", "location": "rocktown", "create_at": date("2011-04-18"), "description": "love makes you forget the difference between right and wrong", "followers_count": 546, "friends_count": 349, "statues_count": 25213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Machesney Park, IL", "id": "41beb41a3e14dab0", "name": "Machesney Park", "place_type": "city", "bounding_box": rectangle("-89.077972,42.32505 -88.985474,42.401396") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1745726, "cityName": "Machesney Park" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506900958339072, "text": "Wind 0 mph ---. Barometer 1006.0 hPa, Falling slowly. Temperature 73.2 °F. Rain today 0.00 in. Humidity 35%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 152723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506901100957696, "text": "I'm so done with twitter������������ https://t.co/lNGzlcSSu7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 518994514, "name": "ari", "screen_name": "Nanci_WOL", "lang": "en", "location": "null", "create_at": date("2012-03-08"), "description": "null", "followers_count": 749, "friends_count": 728, "statues_count": 95759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ivanhoe, CA", "id": "00c70026791b84e5", "name": "Ivanhoe", "place_type": "city", "bounding_box": rectangle("-119.233863,36.348997 -119.197619,36.399505") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 636910, "cityName": "Ivanhoe" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506901172264962, "text": "I really need to go to sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1434635162, "name": "Danny Phantom", "screen_name": "Danny_Phantom_Z", "lang": "en", "location": "Steelernation ", "create_at": date("2013-05-16"), "description": "one day my dream will come true", "followers_count": 559, "friends_count": 415, "statues_count": 12514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506901654585344, "text": "#NY VOTE TRUMP!CruzSaysAnything►https://t.co/lQpqPBgn2L American?CanadaBorn►https://t.co/EYhcDfpxjE @SEC_SAM @KamVTV https://t.co/gN6p51htAU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NY" }}, "user_mentions": {{ 188807646, 3995778614 }}, "user": { "id": 255645890, "name": "U.S. Army Veteran", "screen_name": "Natire2u", "lang": "en", "location": "United States", "create_at": date("2011-02-21"), "description": "Return America to Americans that we all remember America used to be! Pro-American & Pro-Veteran! No Islam, No Illegal Immigrants & No Anti-American propaganda!", "followers_count": 4345, "friends_count": 5002, "statues_count": 23234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palo Alto, CA", "id": "3ad0f706b3fa62a8", "name": "Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.190523,37.362824 -122.097537,37.465918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 655282, "cityName": "Palo Alto" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506902636007424, "text": "in the shadows �� @marvin24jr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2940147774 }}, "user": { "id": 968239484, "name": "DAN18LLE", "screen_name": "itsdanieelll", "lang": "en", "location": "null", "create_at": date("2012-11-24"), "description": "SadBoyz Est. 08' | ihs 18' | #1", "followers_count": 546, "friends_count": 775, "statues_count": 6140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506903088930816, "text": "Give it 2 weeks before Soulja Boy drops a song called Kobe Bryant & makes me reconsidering if Kobe was that nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227148849, "name": "Don Marquez", "screen_name": "AustinM_Butler", "lang": "en", "location": "Oklahoma City ✈️ Indiana", "create_at": date("2010-12-15"), "description": "Cause this the summer that our life change snapchat: Legendary_A3", "followers_count": 1455, "friends_count": 1009, "statues_count": 57805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-04-14T00:00:06.000Z"), "id": 720506903541981185, "text": "Things should've turned out differently. They should've", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2578969358, "name": "Leia Organa", "screen_name": "giannasty", "lang": "en", "location": "null", "create_at": date("2014-06-20"), "description": "the crazy ones change the world - Scott Mescudi • ♏️ sun & moon ♈️ rising", "followers_count": 805, "friends_count": 295, "statues_count": 47836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869263913783296, "text": "posted about a million snapchat videos ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1496173976, "name": "lil danielle", "screen_name": "daniellerosenn", "lang": "en", "location": "wyatt", "create_at": date("2013-06-09"), "description": "null", "followers_count": 572, "friends_count": 296, "statues_count": 1546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869264316452864, "text": "Honestly quite jealous naelie SaladBar got to see the huevo today����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619547711, "name": "Mari", "screen_name": "Marii_Salud", "lang": "en", "location": "null", "create_at": date("2012-06-26"), "description": "null", "followers_count": 336, "friends_count": 236, "statues_count": 9645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arbuckle, CA", "id": "00a988565b5b2a29", "name": "Arbuckle", "place_type": "city", "bounding_box": rectangle("-122.073714,39.006723 -122.050665,39.028345") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6011, "countyName": "Colusa", "cityID": 602420, "cityName": "Arbuckle" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869264400384001, "text": "Favoriting my stuff but dont text me back ��. How that work?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2908119088, "name": "11/25 ♻️", "screen_name": "Bmoney___", "lang": "en", "location": "null", "create_at": date("2014-12-06"), "description": "18.", "followers_count": 690, "friends_count": 514, "statues_count": 9262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frostburg, MD", "id": "105ebe2ad97a9958", "name": "Frostburg", "place_type": "city", "bounding_box": rectangle("-78.948683,39.633535 -78.902675,39.676134") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24001, "countyName": "Allegany", "cityID": 2430900, "cityName": "Frostburg" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869264677154817, "text": "You a fool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3100560661, "name": "Méłissa maríah", "screen_name": "darlinmel_x", "lang": "en", "location": "null", "create_at": date("2015-03-20"), "description": "j", "followers_count": 121, "friends_count": 123, "statues_count": 3274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869264953966593, "text": "@SuperheroFeed compare it to BvS to get the comments stirred up", "in_reply_to_status": 720867531771097089, "in_reply_to_user": 2896294831, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2896294831 }}, "user": { "id": 447290500, "name": "Alex Hill", "screen_name": "ImDiabetus", "lang": "en", "location": "Valentine Detective Agency ", "create_at": date("2011-12-26"), "description": "Proud Member of @Axon_Gaming | Podcaster | Filmmaker | Steelers/Lakers/Pirates fan | Constitutionalist | Huge Star Wars Nerd", "followers_count": 1094, "friends_count": 355, "statues_count": 79736 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Curwensville, PA", "id": "0dfb88e628be713c", "name": "Curwensville", "place_type": "city", "bounding_box": rectangle("-78.540995,40.960463 -78.497918,40.986955") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42033, "countyName": "Clearfield", "cityID": 4217840, "cityName": "Curwensville" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869265365012483, "text": "@SaraiSkywalker we need to write a book", "in_reply_to_status": 720868198778675200, "in_reply_to_user": 39391076, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39391076 }}, "user": { "id": 4137377719, "name": "supernova girl.", "screen_name": "spaceeflower", "lang": "en", "location": "null", "create_at": date("2015-11-05"), "description": "a little bit of an asshole, but not really.", "followers_count": 156, "friends_count": 114, "statues_count": 3726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869265411153920, "text": "@YnnsPhilippakis i think u and @BadgeofShame should have a @WuTangClan nerd off on @midnight", "in_reply_to_status": -1, "in_reply_to_user": 111376222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 111376222, 143351067, 26946348, 1418745782 }}, "user": { "id": 2896066195, "name": "Kevin thå Snowman,dr", "screen_name": "kevin_gonzo", "lang": "en", "location": "SoCal/Randeberg/Denver/CT/WA", "create_at": date("2014-11-10"), "description": "booking at raindropsonrosesmedia at gmail award winning actor, writer and general freak of nature. i love cock and satan. legalize lsd free leonard pickard.", "followers_count": 339, "friends_count": 905, "statues_count": 13206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869266602348545, "text": "@chaso_JOP you Na fi dey sleep ?", "in_reply_to_status": 720869042878226432, "in_reply_to_user": 49062175, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 49062175 }}, "user": { "id": 3236379991, "name": "✨MisFit✨", "screen_name": "colorme_ali", "lang": "en", "location": "null", "create_at": date("2015-06-04"), "description": "No such thing as a life that's better than yours", "followers_count": 231, "friends_count": 156, "statues_count": 33880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-15T00:00:00.000Z"), "id": 720869266820431872, "text": "@viamaeg @Yung_kswiss ��", "in_reply_to_status": 720869224273432577, "in_reply_to_user": 217291655, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 217291655, 3742392198 }}, "user": { "id": 79820174, "name": "Darren", "screen_name": "ddarrenizzles", "lang": "en", "location": "SJ", "create_at": date("2009-10-04"), "description": "simplicity is all i ask for | SC: ddarrenizzles", "followers_count": 383, "friends_count": 344, "statues_count": 13167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869267139198976, "text": "@frostyplum Not with that spelling, missy.", "in_reply_to_status": 720869139405869057, "in_reply_to_user": 46538927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46538927 }}, "user": { "id": 14195458, "name": "Spencer D. Williams", "screen_name": "Devilturnip", "lang": "en", "location": "Huntsville, AL", "create_at": date("2008-03-21"), "description": "There is nothing to say about Spencer D. Williams.", "followers_count": 1249, "friends_count": 279, "statues_count": 10833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869267785125888, "text": "03:00:01 |Temp: 51.1ºF |Dew Point 48.0ºF | Rain today: 0.12 inches | Wind: 0.0 mph from the E, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869267911024640, "text": "Temp 29.1° Hi/Lo 31.7/29.0 Rng 2.7° WC 29.1° Hmd 87% Rain 0.00\" Storm 0.00\" BAR 30.420 Rising DP 25.7° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869268141662208, "text": "#Trump in 2016 Temp:50.4°F Wind:0.0mph Pressure: 30.09hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming less settled", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 315236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869268481413120, "text": "I could watch modern family ever day all day ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2207577884, "name": "Tawts", "screen_name": "Tawtsss", "lang": "en", "location": "null", "create_at": date("2013-11-21"), "description": "rags to riches", "followers_count": 904, "friends_count": 1080, "statues_count": 5116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Auburn, CA", "id": "0024bea413aa7102", "name": "North Auburn", "place_type": "city", "bounding_box": rectangle("-121.128403,38.901257 -121.035533,38.993117") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 651637, "cityName": "North Auburn" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869268535914496, "text": "@ebbtideapp Tide in USCG Freeport, Texas 04/15/2016\n Low 5:02am 0.3\nHigh 1:11pm 1.7\n Low 7:51pm 1.0\nHigh 11:36pm 1.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-95.3017,28.9433"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 71, "friends_count": 1, "statues_count": 24561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4827420, "cityName": "Freeport" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869269139927040, "text": "Wind 0.0 mph ---. Barometer 29.871 in, Falling. Temperature 62.4 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 61275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869269345427457, "text": "Just let that shit go ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 530815360, "name": "Donovan", "screen_name": "blvckcaesar", "lang": "en", "location": "null", "create_at": date("2012-03-19"), "description": "Goucher university ☄/ I'm just chilling", "followers_count": 1609, "friends_count": 1206, "statues_count": 9088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869270125563905, "text": "04/15@03:00 - Temp 37.6F, WC 37.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.392in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869270440161280, "text": "Livin' my life til I die..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34398473, "name": "Paris", "screen_name": "_PARISnoHilton", "lang": "en", "location": "null", "create_at": date("2009-04-22"), "description": "23. College grad. #ULAlumna. Mental Health Specialist. Hair-doer. Makeup lover. YouTube beauty blogger (ParisDanielle)", "followers_count": 1721, "friends_count": 1094, "statues_count": 65977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869270494707712, "text": "Wind 2.7 mph ESE. Barometer 30.04 in, Steady. Temperature 61.3 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869270708559873, "text": "Wind 2.0 mph ENE. Barometer 30.116 in, Falling slowly. Temperature 54.4 °F. Rain today 0.00 in. Humidity 48%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-15T00:00:01.000Z"), "id": 720869270796697600, "text": "18 years of annoying the shit out of the people I love, thanks for putting up with me people. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197704907, "name": "chase", "screen_name": "ChaseRyanPalmer", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-10-01"), "description": "la based photographer chase@chasepalmer.photography for business inquiries", "followers_count": 1031, "friends_count": 256, "statues_count": 14843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869271207710720, "text": "Jared is ham ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160113698, "name": "dominick klockeman", "screen_name": "domklockeman", "lang": "en", "location": "null", "create_at": date("2010-06-26"), "description": "Nothing matters without the man above. Gechelle is my 1 and only.", "followers_count": 390, "friends_count": 150, "statues_count": 27082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanaway, WA", "id": "21ea0a8db9b8d520", "name": "Spanaway", "place_type": "city", "bounding_box": rectangle("-122.464673,47.053113 -122.389138,47.126056") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5366255, "cityName": "Spanaway" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869271299948544, "text": "OMG ... That poor kid �� https://t.co/M1wO29AaS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129402604, "name": "April16th.", "screen_name": "AuviOhh_AndXO", "lang": "en", "location": "Vallejo, CA", "create_at": date("2010-04-03"), "description": "you's kind . you's smart . you's important.", "followers_count": 790, "friends_count": 1129, "statues_count": 18316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869271388028928, "text": "Want to work at Molina Healthcare? We're #hiring in #LongBeach, CA! Click for details: https://t.co/GiML1jS2ND https://t.co/hEYJrzxLqW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.1937395,33.7700504"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "LongBeach" }}, "user": { "id": 3022832918, "name": "Molina Careers", "screen_name": "MolinaCareers", "lang": "en", "location": "Nationwide", "create_at": date("2015-02-16"), "description": "Molina is growing rapidly & we’re looking for talented people who share our values. Check out our current opportunities at http://MolinaHealthcare.com/Careers.", "followers_count": 234, "friends_count": 1, "statues_count": 844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869271849402369, "text": "@Selenaa_suzette thats my boy dude....", "in_reply_to_status": 720869032501444608, "in_reply_to_user": 2369183701, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2369183701 }}, "user": { "id": 1213863559, "name": "pheekx.", "screen_name": "swampgooch", "lang": "en", "location": "null", "create_at": date("2013-02-23"), "description": "Skate // GOS // San Jose Local // Hip Hop // Krevchenko snapchat : @jamesdafames", "followers_count": 332, "friends_count": 449, "statues_count": 14209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272017195008, "text": "Wind 0.0 mph ---. Barometer 30.041 in, Steady. Temperature 57.4 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272394727426, "text": "Faith that is proven through action. By being a doer of the Word, not just a hearer (James 1:22).", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392241216, "name": "Kevin J Holmes", "screen_name": "kevinh75051", "lang": "en", "location": "null", "create_at": date("2011-10-16"), "description": "There's No Mission 2 Difficult No Sacrifice 2 Great! Philippians 4:13 I can do all things through Jesus Christ who strengthens me.", "followers_count": 3438, "friends_count": 3444, "statues_count": 23961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272486948869, "text": "@Rosie2588 @CaptainAmerica Wuuuuut!!??\npffffffftttt #TeamIronMan \nWait.... is there a Side called \"Underoos\"? #Underoos", "in_reply_to_status": 720866504586072064, "in_reply_to_user": 79183748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamIronMan", "Underoos" }}, "user_mentions": {{ 79183748, 701615052 }}, "user": { "id": 33717452, "name": "Alex Melendez", "screen_name": "SenorMelendez", "lang": "en", "location": "McAllen Texas ", "create_at": date("2009-04-20"), "description": "Oh Hai!!", "followers_count": 149, "friends_count": 212, "statues_count": 2142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272487010304, "text": "When your daily horoscope is on fleek��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2495922116, "name": "☮ Krista ☮", "screen_name": "xo_kriss14", "lang": "en", "location": "null", "create_at": date("2014-05-15"), "description": "#Reciprocity", "followers_count": 359, "friends_count": 631, "statues_count": 5828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atwater, CA", "id": "676c3e6a1e0642db", "name": "Atwater", "place_type": "city", "bounding_box": rectangle("-120.641299,37.325641 -120.555357,37.374957") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 603162, "cityName": "Atwater" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272650539009, "text": "66.3F (Feels: 66.3F) - Humidity: 99% - Wind: 1.6mph E - Gust: 2.2mph - Pressure: 1005.9mb - Rain: 0.08\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272889593856, "text": "my roommate who usually never comes out of her room or talks is taking shots with her loud friend right now & woke me up out of a dead sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4253800414, "name": "Kayla ⭐️", "screen_name": "kaylakristine24", "lang": "en", "location": "Tampa, FL", "create_at": date("2015-11-22"), "description": "''I should've known you were a lesbian, I saw you drink Bud Light once'' - Maria", "followers_count": 63, "friends_count": 164, "statues_count": 1069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869272948334592, "text": "#DemDebate #imwithjane https://t.co/Kbpglgjoz9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "DemDebate", "imwithjane" }}, "user": { "id": 96648433, "name": "Sara Spector", "screen_name": "Miriam2626", "lang": "en", "location": "Hondo, Texas", "create_at": date("2009-12-13"), "description": "Criminal Defense Attorney. http://Hondolawfirm.com. Passionate advocate.", "followers_count": 6105, "friends_count": 5940, "statues_count": 26068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hondo, TX", "id": "6521a8878080ca86", "name": "Hondo", "place_type": "city", "bounding_box": rectangle("-99.200407,29.332533 -99.122347,29.373723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48325, "countyName": "Medina", "cityID": 4834676, "cityName": "Hondo" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273183236096, "text": "Temp: 62.1°F Wind:0.3mph Pressure: 29.969hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 61276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273376153601, "text": "Wind 0.0 mph ---. Barometer 30.468 in, Steady. Temperature 25.7 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273409708033, "text": "Iron Man trilogy is so not good :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46745375, "name": "Latina Whisperer", "screen_name": "YoungaviatorJ", "lang": "en", "location": "Manteca, CA", "create_at": date("2009-06-12"), "description": "21. Gamer. Nerd. Rest in Paradise Fabrise Munoz. ReelSquad Cofounder. @TheReelSquad", "followers_count": 1314, "friends_count": 795, "statues_count": 250753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273745252352, "text": "It seriously feels like someone's stabbing me in the head wtf dude", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75631580, "name": "Carissa ♡", "screen_name": "Carissa_Swain", "lang": "en", "location": "az", "create_at": date("2009-09-19"), "description": "No such thing as a life that's better than yours ●\nUptown server ● ♒", "followers_count": 758, "friends_count": 732, "statues_count": 17162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273783001089, "text": "The cross has made you flawless", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 548874375, "name": "Reagan", "screen_name": "_Reaganhollan_", "lang": "en", "location": "admit nothing deny everything ", "create_at": date("2012-04-08"), "description": "Been featured on Old Row & that's all u need to know. You know what I find offensive? Not being able to freely offend someone.", "followers_count": 873, "friends_count": 250, "statues_count": 59862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869273963335681, "text": "Obeying God is only solutions for what evils awaits the generations of blacks, then he will heal the community & raise real leaders worthy!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 511597571, "name": "Phillips McCune /2", "screen_name": "McCune2", "lang": "en", "location": "Global", "create_at": date("2012-03-02"), "description": "Creator Prototype 800b-99521 Technology, in development: Created Blk History CD ROM, Tic TacToe Knowledge Game for schools/ Talking Technology Interfaces &", "followers_count": 930, "friends_count": 1875, "statues_count": 2137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869274076590080, "text": "@iiroberto_ wya?", "in_reply_to_status": 720862130275164160, "in_reply_to_user": 407461381, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 407461381 }}, "user": { "id": 998138484, "name": "Superman™♊", "screen_name": "yes_its_james", "lang": "en", "location": "The Murda", "create_at": date("2012-12-08"), "description": "there are two important days in our lives. the day we are born and the day we find out why.\n\n..CPP BRONCO..\n\nLions don't lose sleep over the opinion of sheep", "followers_count": 851, "friends_count": 690, "statues_count": 46050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869274156290049, "text": "Ain't drama when people hope to pass away .. Life is way tough .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2156102887, "name": "MCMXCIV.77♚", "screen_name": "Bader_kurdi", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-10-25"), "description": "Fight Until You Can't Fight Again - (اللهم ارحم ابي حبيبي و نور له قبر)", "followers_count": 369, "friends_count": 185, "statues_count": 8352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869274873499648, "text": "Wind 0.7 mph W. Barometer 30.43 in, Steady. Temperature 33.8 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 159072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-15T00:00:02.000Z"), "id": 720869274894467073, "text": "K gn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1523191784, "name": "Allessandra✨", "screen_name": "1allessandra", "lang": "en", "location": "y(our)bitchhouse", "create_at": date("2013-06-16"), "description": "lil xan .KE", "followers_count": 354, "friends_count": 320, "statues_count": 7558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apex, NC", "id": "d36f171e278ac981", "name": "Apex", "place_type": "city", "bounding_box": rectangle("-78.915463,35.678557 -78.794693,35.767843") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3701520, "cityName": "Apex" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869275393654785, "text": "Santa Monica #santamonica #santamonicabeach #WeAreOriginalMY… https://t.co/ivS9jm0YCh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.49737934,34.00922415"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "santamonica", "santamonicabeach", "WeAreOriginalMY" }}, "user": { "id": 252937838, "name": "monotonous", "screen_name": "hattadolmatHDC", "lang": "en", "location": "malaysia/40000 shah alam ", "create_at": date("2011-02-15"), "description": "instagram : @hattadolmat_ @hattadolmatofficial snapchat: hattadolmat #monotonous email: hattadolmat.rtw@gmail.com", "followers_count": 54096, "friends_count": 656, "statues_count": 82556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869275494277122, "text": "Ha ha what a bad joke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351126243, "name": "nay", "screen_name": "nayday_", "lang": "en", "location": "Pasadena, CA", "create_at": date("2011-08-08"), "description": "tune out, drop in • IG: _nayre", "followers_count": 1107, "friends_count": 793, "statues_count": 43923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altadena, CA", "id": "fb76b3aa366004c9", "name": "Altadena", "place_type": "city", "bounding_box": rectangle("-118.177296,34.167539 -118.095091,34.220078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 601290, "cityName": "Altadena" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869276006023168, "text": "Wind 0.0 mph ---. Barometer 29.909 in, Falling slowly. Temperature 52.8 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869277604061185, "text": "@AbsolutBex Liz lost her obsession and she's got nobody. She even lost her house.", "in_reply_to_status": 720866899194548224, "in_reply_to_user": 32446433, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 32446433 }}, "user": { "id": 25890025, "name": "LM", "screen_name": "shayne571", "lang": "en", "location": "Los Angeles", "create_at": date("2009-03-22"), "description": "Techie. I do database stuff. Ex-Canadian living in SoCal.\n\nI love to snark. Don't like it? You know where the unfollow button is.", "followers_count": 285, "friends_count": 229, "statues_count": 71119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869278107369472, "text": "nigga where the plug", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 255263207, "name": "BMBG TATE", "screen_name": "NewAgeTate", "lang": "en", "location": "In The Kitchen , TX", "create_at": date("2011-02-20"), "description": "MASTERCHEFTATE | @BMBGMUSIC", "followers_count": 1124, "friends_count": 749, "statues_count": 19331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869278208004097, "text": "When the person that you're watching on YouTube watches your snapchat story where there's a snap about him ������ https://t.co/wybwrUhMq6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352033099, "name": "Devyn", "screen_name": "_GalaxyGamer", "lang": "en", "location": "At my Drumset ✌️", "create_at": date("2011-08-09"), "description": "17 | Drummer | Youtube | Twitch | PurFate | ~I have the heart of a lion and the drums are my jungle~ My band's first music video link below c:", "followers_count": 1010, "friends_count": 308, "statues_count": 42955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869278740643840, "text": "@idaliareyes41 @Angelitosalvaje @drajanetalvarez @lakes1947 @Yoleys_ DLB a todas ��❤️ https://t.co/2hwNMtCgMh", "in_reply_to_status": 720722150991114240, "in_reply_to_user": 1284739453, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1284739453, 181927687, 84806538, 368755316, 182623092 }}, "user": { "id": 241389892, "name": "tila iraheta", "screen_name": "clotildeiraheta", "lang": "es", "location": "USA", "create_at": date("2011-01-21"), "description": "Amo a Dios primeramente, despues a mis hijos, nietos, y a los amigos que El Senor ha puesto en mi camino!!", "followers_count": 1680, "friends_count": 2003, "statues_count": 131061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:03.000Z"), "id": 720869278971330561, "text": "@Staygoldsamm bye Felicia", "in_reply_to_status": 720868694998429696, "in_reply_to_user": 170941554, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 170941554 }}, "user": { "id": 2364832134, "name": "J.Sabajan", "screen_name": "JSabajan", "lang": "en", "location": "null", "create_at": date("2014-02-27"), "description": "you made a choice today", "followers_count": 222, "friends_count": 132, "statues_count": 6156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869280233828357, "text": "��������", "in_reply_to_status": 720050846751326208, "in_reply_to_user": 316735921, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 316735921, "name": "♿️", "screen_name": "DarelBeChillin", "lang": "en", "location": "Kansas City✈️ Austin, TX/Den10", "create_at": date("2011-06-13"), "description": "Don't take me seriously......unless I'm serious | #DILFHive | #BLACKLIVESMATTER", "followers_count": 2017, "friends_count": 890, "statues_count": 108847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869280607117313, "text": "But does it really matter?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319935020, "name": "Vonn", "screen_name": "itsVonn", "lang": "en", "location": "SOMEWHERE", "create_at": date("2011-06-18"), "description": "Dying of laughter", "followers_count": 571, "friends_count": 372, "statues_count": 23843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869280741330945, "text": "@CameronMoulene nice name", "in_reply_to_status": -1, "in_reply_to_user": 1426674912, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1426674912 }}, "user": { "id": 3313813441, "name": "Cameron Hale", "screen_name": "Camerondabawmb", "lang": "en", "location": "Illahee, WA", "create_at": date("2015-08-12"), "description": "Hey I do YouTube videos. \nCheck out my channel .youtube.com/channel/UCe9Kv…", "followers_count": 129, "friends_count": 57, "statues_count": 750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869280955256832, "text": "@lupitamaru3 yea same it's the struggle ��", "in_reply_to_status": 720868289669242880, "in_reply_to_user": 1056883188, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1056883188 }}, "user": { "id": 1170815252, "name": "alannaj", "screen_name": "jeffcoatmarie13", "lang": "en", "location": "Abilene, TX", "create_at": date("2013-02-11"), "description": "Nursing Student Jville||Abilene", "followers_count": 686, "friends_count": 306, "statues_count": 17574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869281303384068, "text": "I only have one class tomorrrrrow����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1096423796, "name": "Gianna Lucchetti", "screen_name": "gbaaaby2", "lang": "en", "location": "Manteca, CA", "create_at": date("2013-01-16"), "description": "@thetreyroberts, baseball & true crime books", "followers_count": 714, "friends_count": 363, "statues_count": 54136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282008080384, "text": "�������������� https://t.co/hvBSY6fVQj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 378254614, "name": "Carlos Aguilar", "screen_name": "CarlosTheFiasco", "lang": "en", "location": "null", "create_at": date("2011-09-22"), "description": "null", "followers_count": 643, "friends_count": 418, "statues_count": 20300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Katy, TX", "id": "c484d6c5cd72beb0", "name": "Katy", "place_type": "city", "bounding_box": rectangle("-95.87417,29.736773 -95.774936,29.831219") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4838476, "cityName": "Katy" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282037387265, "text": "James, you have a very soothing voice. @JamesRAHendry #SafariLive @WildEarth Thanks for the drive.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.95255493,41.36426633"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SafariLive" }}, "user_mentions": {{ 45083301, 14691503 }}, "user": { "id": 292404883, "name": "Sherry Kamphus", "screen_name": "marshy5365", "lang": "en", "location": "Olmsted Falls, OH", "create_at": date("2011-05-03"), "description": "This is my official page.., African wildlife lover, Stepmom, Aneurysm survivor, Y&R, B&B, Dubstep, and loving life.", "followers_count": 455, "friends_count": 763, "statues_count": 20996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olmsted Falls, OH", "id": "01c554ba145fdc29", "name": "Olmsted Falls", "place_type": "city", "bounding_box": rectangle("-81.98075,41.314081 -81.876105,41.39158") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3958422, "cityName": "Olmsted Falls" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282196762624, "text": "Temp: 38.3°F - Dew Point: 31.5° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.19in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282561662976, "text": "#diabetic #t1d Les Merveilleux Bienfaits Du Bicarbonate De Soude: Les Merveilleux… https://t.co/cn10bGvtoU #type1diabetes #diabetes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99841309,40.70640872"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "diabetic", "t1d", "type1diabetes", "diabetes" }}, "user": { "id": 4835709563, "name": "James Wolter", "screen_name": "Diabetes_Newzz", "lang": "en", "location": "United States", "create_at": date("2016-01-22"), "description": "Hi, James here, I focus on providing a comprehensive, supportive and independent experience for my twitter visitors around Diabetes and help for Diabetes.", "followers_count": 683, "friends_count": 1811, "statues_count": 19982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282624606209, "text": "Should I watch ALL of @KimmySchmidt tomorrow or space if out? Probably binge watch it over and over.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2438808708 }}, "user": { "id": 313117217, "name": "Jade Ingersoll", "screen_name": "MylifeasJade", "lang": "en", "location": "Berkeley, CA", "create_at": date("2011-06-07"), "description": "Academy of Art University 2016 ✨ SF Giants baseball ⚾️", "followers_count": 207, "friends_count": 469, "statues_count": 18478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282838482950, "text": "Maybe it's for the better lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2407770288, "name": "ANNAXOR", "screen_name": "roxy_thotsy", "lang": "en", "location": "null", "create_at": date("2014-03-23"), "description": "null", "followers_count": 222, "friends_count": 181, "statues_count": 10850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282955984896, "text": "@DCMG_Mikee hell yeah", "in_reply_to_status": 720869171718791168, "in_reply_to_user": 305446862, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 305446862 }}, "user": { "id": 331916965, "name": "#FreebandzTalibanCEO", "screen_name": "BalloutWebbie", "lang": "en", "location": "BALLOUTWORLD w/ Money & Nem", "create_at": date("2011-07-08"), "description": "Free The Guyz Rip The Guyz #SSD #NLMB #OTF #2FFE #YFNBC #MurdaTeam", "followers_count": 1385, "friends_count": 1134, "statues_count": 122102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-15T00:00:04.000Z"), "id": 720869282981109761, "text": "\"Don't turn your back on bears, men you have wronged, or the dominant turkey during mating season.\"\n-Dwight K. Schrute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2365099627, "name": "Haiden Drake", "screen_name": "drake_haiden", "lang": "en", "location": "null", "create_at": date("2014-02-27"), "description": "voted most likely to brighten your day", "followers_count": 169, "friends_count": 152, "statues_count": 914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869284675657728, "text": "@lafinestt https://t.co/tQxXFblFHt", "in_reply_to_status": 720869051094859777, "in_reply_to_user": 1363693711, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1363693711 }}, "user": { "id": 707848551951249408, "name": "JIGGY", "screen_name": "AndreaMcCabe_", "lang": "en", "location": "null", "create_at": date("2016-03-10"), "description": "#longlivegio #longlivelance #longlivepaul #freeWill", "followers_count": 679, "friends_count": 477, "statues_count": 3392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869284952453120, "text": "I've got a long term plan with short term fixes♥︎", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1376193055, "name": "hanna", "screen_name": "hanna_gatpo", "lang": "en", "location": "☁︎9", "create_at": date("2013-04-23"), "description": "♡♥︎♡♥︎♡♥︎♡♥︎♡♥︎♡♥︎♡♥︎♡♥︎♡♥︎♡", "followers_count": 302, "friends_count": 433, "statues_count": 2270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869284977639424, "text": "yeppp.... my life is a COMPLETE joke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119048981, "name": "lælæ™", "screen_name": "lainiesmith1", "lang": "en", "location": "new orleans,louisiana", "create_at": date("2010-03-02"), "description": "a beautiful mind with a hell of a grind", "followers_count": 787, "friends_count": 494, "statues_count": 15376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869284981792772, "text": "I was excitedly tweeting my new apartment so much than @Snapchat locked me out of my account for spam:( meanwhile. Snapchat me: adam_hawk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 376502929 }}, "user": { "id": 389945480, "name": "Hawks☕️", "screen_name": "TheAdamHawk", "lang": "en", "location": "Pitt✈️LA / Subscribe ⬇️", "create_at": date("2011-10-13"), "description": "I'm an Idea Guy. Internet Personality. Twitch Broadcaster. Coffee Addict. Check out my @YouTube below! #Actor #Creator", "followers_count": 42347, "friends_count": 1419, "statues_count": 61887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869285011165184, "text": "W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 288534221, "name": "Fanelle", "screen_name": "JanelleCarty", "lang": "en", "location": "null", "create_at": date("2011-04-26"), "description": "Htx @KingxJeff_ ❤️ #islandraised", "followers_count": 1445, "friends_count": 716, "statues_count": 44347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, TX", "id": "8935eb0e13a342db", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-95.505362,29.507337 -95.434776,29.567483") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4827540, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869285166391296, "text": "Waves up to 22 feet forecast to hit Bay Area https://t.co/CkmPbWHRPO ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 8200072, "name": "Steven BRODY Stevens", "screen_name": "BrodyismeFriend", "lang": "en", "location": "THE San Fernando Valley/LA ☀️", "create_at": date("2007-08-15"), "description": "Comedian from The United States and Tarzana, California. Pitcher & Percussionist. Actor & Personality.", "followers_count": 94579, "friends_count": 4721, "statues_count": 56609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869285585756160, "text": "Bates motel .. Carrie diaries https://t.co/l7kgYe9dhN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 314850287, "name": "⚫️", "screen_name": "Jakyiahg", "lang": "en", "location": "null", "create_at": date("2011-06-10"), "description": "the blacker the berry, the sweeter the juice", "followers_count": 1895, "friends_count": 897, "statues_count": 30998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869286303047681, "text": "@BelleChere I believe it's the positive vibe you give as well as the awesome costumes.", "in_reply_to_status": 720655857445416961, "in_reply_to_user": 15760042, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15760042 }}, "user": { "id": 134352996, "name": "Frank", "screen_name": "jokersole", "lang": "en", "location": "Some where under a bridge.", "create_at": date("2010-04-17"), "description": "I am the background noise. Listen and you just might hear me.", "followers_count": 78, "friends_count": 210, "statues_count": 1098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lexington, NC", "id": "0912175f15659c1e", "name": "Lexington", "place_type": "city", "bounding_box": rectangle("-80.367573,35.749299 -80.187124,35.870998") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37057, "countyName": "Davidson", "cityID": 3738060, "cityName": "Lexington" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869286659563521, "text": "Your WCW the type to talk shit about herself just so you'd compliment her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1051927436, "name": "Robin Quinanola", "screen_name": "robinquinanola", "lang": "en", "location": "null", "create_at": date("2012-12-31"), "description": "null", "followers_count": 445, "friends_count": 24, "statues_count": 25907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869286810492930, "text": "@manikhanfar remember this https://t.co/NmEfUTqtbu", "in_reply_to_status": -1, "in_reply_to_user": 161387934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161387934 }}, "user": { "id": 2226087727, "name": "Yani", "screen_name": "EriyanaWard", "lang": "en", "location": "Anthem, AZ", "create_at": date("2013-12-01"), "description": "trigger fingers // Life's most persistent and urgent question is, 'What are you doing for others?", "followers_count": 311, "friends_count": 179, "statues_count": 2579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anthem, AZ", "id": "01d5798f59d51d79", "name": "Anthem", "place_type": "city", "bounding_box": rectangle("-112.164272,33.784511 -112.012981,33.885193") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 402430, "cityName": "Anthem" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869286860816384, "text": "Lil Bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1915613984, "name": "Shinobool Sensui", "screen_name": "sklaw__suseJ", "lang": "en", "location": "Newark, NJ", "create_at": date("2013-09-28"), "description": "White Bitches And Money All I Care For", "followers_count": 3536, "friends_count": 345, "statues_count": 16464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869287448018944, "text": "Uhhhhh �� https://t.co/dEplQt7UdO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 712173447867043840, "name": "R.I.P. Lonnie❤️", "screen_name": "queen_typeshii", "lang": "en", "location": "Dallas, TX", "create_at": date("2016-03-22"), "description": "SC: queen_typeshii, pizza is life", "followers_count": 71, "friends_count": 53, "statues_count": 845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869287750045697, "text": "@Marianaasofia https://t.co/9x8mxQpMSP", "in_reply_to_status": -1, "in_reply_to_user": 364666528, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 364666528 }}, "user": { "id": 2930265468, "name": "KarenYP", "screen_name": "karenyesell", "lang": "en", "location": "null", "create_at": date("2014-12-14"), "description": "Psalm 23:4 ✨ Instagram: karenyesell | snapchat: kareennn5", "followers_count": 264, "friends_count": 287, "statues_count": 9906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869287792025600, "text": "#diabetic #t1d Tiny House Plans: Tiny House Plans Earn 50% Commission Promoting Tiny House… https://t.co/QvkiNpzdUE #type1diabetes #diabetes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99841309,40.70640872"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "diabetic", "t1d", "type1diabetes", "diabetes" }}, "user": { "id": 4835709563, "name": "James Wolter", "screen_name": "Diabetes_Newzz", "lang": "en", "location": "United States", "create_at": date("2016-01-22"), "description": "Hi, James here, I focus on providing a comprehensive, supportive and independent experience for my twitter visitors around Diabetes and help for Diabetes.", "followers_count": 683, "friends_count": 1811, "statues_count": 19983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-15T00:00:05.000Z"), "id": 720869287812997121, "text": "@ActuallyCryBaby yes or no smh", "in_reply_to_status": 720869081205727233, "in_reply_to_user": 1005883915, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1005883915 }}, "user": { "id": 1326173832, "name": "⛓ peachy mamí ⛓", "screen_name": "babexfett", "lang": "en", "location": "DR PEPPER YOUTH CREW", "create_at": date("2013-04-03"), "description": "spookshow baby", "followers_count": 1504, "friends_count": 938, "statues_count": 27090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869288131698691, "text": "Anyone know how to fix iPhone screens and wants to hook a nigga up?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270692197, "name": "Bolton Bombs", "screen_name": "Based_Colton", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2011-03-22"), "description": "#BASEDWORLD Lil B fucked my bitch", "followers_count": 502, "friends_count": 391, "statues_count": 12896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869288341471232, "text": "The only thing that counts is faith expressing itself through love.\nGalatians 5:6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I live each and every passing day, according to Gods ways and will for my life.", "followers_count": 174, "friends_count": 293, "statues_count": 329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869288525975552, "text": "Wind 5.0 mph N. Barometer 30.438 in, Steady. Temperature 37.0 °F. Rain today 0.00 in. Humidity 38%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 5770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869288609878016, "text": "Bernie is neck and neck with Hillary simply off the people of our nation. That should be enough to prove that he deserves the nomination.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42293008, "name": "Pure Passion", "screen_name": "RealPurePassion", "lang": "en", "location": "Yonkers", "create_at": date("2009-05-24"), "description": "Rapper/Song Writer out of Yonkers, NY. (Peace Uniting Righteous Eternally Pushing All Soul Survivors In Our Nation) #PUREPASSION", "followers_count": 7131, "friends_count": 5422, "statues_count": 22333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869288895062017, "text": "����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3786539233, "name": "Tkevah Morton", "screen_name": "Keke23Queen", "lang": "en", "location": "null", "create_at": date("2015-10-04"), "description": "Que future wifey nursing #strong cool pretty smart Sing n the choir #strong BLACK women!don't bring that drama shit in my life.", "followers_count": 838, "friends_count": 2464, "statues_count": 14154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869289750695937, "text": "How fitting it is that you're a rihanna Stan with bad taste. https://t.co/adsrACPKGg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411754636, "name": "albino alligators.", "screen_name": "beyoncehatesme", "lang": "en", "location": "null", "create_at": date("2011-11-13"), "description": "and Beyoncé said, Let there be light and there was light", "followers_count": 18871, "friends_count": 323, "statues_count": 111908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869290073665536, "text": "@TycineC @selena_aaliyaah @JoinGreenRush Thank you ! https://t.co/vB174t8qbJ", "in_reply_to_status": 720868893200244737, "in_reply_to_user": 2303946955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2303946955, 1577159876, 2594142181 }}, "user": { "id": 4639094582, "name": "NÏA", "screen_name": "Tsnmi_nia", "lang": "en", "location": "null", "create_at": date("2015-12-23"), "description": "22. college. TSNMIMOB. music is life. ball is life", "followers_count": 90, "friends_count": 116, "statues_count": 3584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869290174361604, "text": "Why do you hate me?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2886923479, "name": "#ThankYouKobe", "screen_name": "ahlisamichelle", "lang": "en", "location": "Wonderland ", "create_at": date("2014-11-01"), "description": "Filipino/White/Hawaiian - 2⃣3️⃣ - ⚓️ - Aleena's GodMommy ♡ #LakerGang", "followers_count": 36, "friends_count": 86, "statues_count": 1978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869290774175746, "text": "same https://t.co/rJeTGPPIL2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2284968950, "name": "✨daddio gladio✨", "screen_name": "redserenade", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-10"), "description": "noel|19|her/him|ENG/한국어 OK|i love money, power, and big daddies ♔ local kazuhira miller lover ♔ personal: @ocelhiras / nsfw: @pink_concerto", "followers_count": 207, "friends_count": 652, "statues_count": 36154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869291038343169, "text": "Happy birthday @CrawfordCollins ur trying 19 what since its your birthday I made this for you eat a lot of cake�� https://t.co/CmBGdnOsCK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 355795737 }}, "user": { "id": 696474534421987329, "name": "Melenie", "screen_name": "Melenie523", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2016-02-07"), "description": "I'm living life right now man cause @weeklychris & @crawfordcollins followed me", "followers_count": 435, "friends_count": 818, "statues_count": 615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869291256512513, "text": "Who Knows She Might Not and You Might Get An Accepted By Another Companion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35126535, "name": "King Tyrant", "screen_name": "SaintJamesKing", "lang": "en", "location": "null", "create_at": date("2009-04-24"), "description": "_The Young Mans King & The Old Mans Tyrant_Louis XII", "followers_count": 711, "friends_count": 1649, "statues_count": 19593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869291461988353, "text": "Pressed Juice \n#losangles #la #hollywood #juice @ The Americana at Brand https://t.co/OjpZpjkd3s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.25693512,34.14407719"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "losangles", "la", "hollywood", "juice" }}, "user": { "id": 92213372, "name": "Turki Faisal", "screen_name": "Turkifaisal", "lang": "en", "location": "Qatar قطر", "create_at": date("2009-11-23"), "description": "‏‏رئيس جهاز كرة السله بالنادي العربي القطري Head of basketball sector Al Arabi sport club", "followers_count": 427, "friends_count": 778, "statues_count": 2144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869291625590784, "text": "@mformarj cge! Asa mn? Ibaligya nko akong atay��", "in_reply_to_status": 720869100403068928, "in_reply_to_user": 470317443, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 470317443 }}, "user": { "id": 3229941572, "name": "kirky", "screen_name": "kjpilaaaaaar", "lang": "en", "location": "DAVAO CITY ✖️ NEW JERSEY", "create_at": date("2015-05-29"), "description": "SC: pilarkirk17 IG: kirkpilar ❤️@mikkahmilan❤️", "followers_count": 369, "friends_count": 583, "statues_count": 6639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sayreville, NJ", "id": "7c09539af94ab80b", "name": "Sayreville", "place_type": "city", "bounding_box": rectangle("-74.385802,40.441457 -74.258203,40.509016") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3465790, "cityName": "Sayreville" } }
+{ "create_at": datetime("2016-04-15T00:00:06.000Z"), "id": 720869292028223489, "text": "Throwback�� https://t.co/KgS9B9mkA7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 592068220, "name": "Christian Alejandro", "screen_name": "crisbriseno", "lang": "en", "location": "null", "create_at": date("2012-05-27"), "description": "In love with Elissa ❤", "followers_count": 281, "friends_count": 273, "statues_count": 5685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869292250517504, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3092341674, "name": "BriargateAssJayoo⛽️", "screen_name": "Trapjayoo", "lang": "en", "location": "MoCity Tx.", "create_at": date("2015-03-16"), "description": "#813Tee #414Tyree #LongLiveDex53 #KIPKEYANNA #TeamSS #JugginByThaTexaco |Lil Craft Jr.|", "followers_count": 498, "friends_count": 395, "statues_count": 5744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293039050752, "text": "I love Sade.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1929888823, "name": "Daniella Rodriguez", "screen_name": "LAwomann_", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-10-03"), "description": "Classic rock, films, and my man.", "followers_count": 933, "friends_count": 973, "statues_count": 61163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293127127040, "text": "@_brittlaurennn don't make me change my mind", "in_reply_to_status": 720869236470513664, "in_reply_to_user": 2727793872, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2727793872 }}, "user": { "id": 750813218, "name": "E.", "screen_name": "EricAndHisAss", "lang": "en", "location": "Texas", "create_at": date("2012-08-10"), "description": "I change tires and go to school. 20.", "followers_count": 621, "friends_count": 444, "statues_count": 65114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293215186945, "text": "Taco Bell and a movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224333845, "name": "Perkycallin", "screen_name": "DontHoldbacShit", "lang": "en", "location": "New York, USA", "create_at": date("2010-12-08"), "description": "#creeplife #thunderup #teamiphone #team1kid #wildboy #GND #realnigga #familyaboveeverything #rip travon #MMG #YMCMB", "followers_count": 253, "friends_count": 344, "statues_count": 30619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293458522112, "text": "Wind 0.0 mph ---. Barometer 30.077 in, Rising. Temperature 39.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 204, "friends_count": 59, "statues_count": 268286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293726912512, "text": "@D1_AndOnlyx14 Chopatti bread and butter ��������", "in_reply_to_status": 720861365456359429, "in_reply_to_user": 571070605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 571070605 }}, "user": { "id": 310240690, "name": "Sohwon Kim", "screen_name": "So1Kim3", "lang": "en", "location": "null", "create_at": date("2011-06-03"), "description": "DMV to Miami USA TaeKwonDo, #15,", "followers_count": 820, "friends_count": 770, "statues_count": 32822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Walk, FL", "id": "971b4b7c352f379f", "name": "Country Walk", "place_type": "city", "bounding_box": rectangle("-80.456153,25.624969 -80.41449,25.641133") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1215055, "cityName": "Country Walk" } }
+{ "create_at": datetime("2016-04-15T00:00:07.000Z"), "id": 720869293752094722, "text": "L's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 33712831, "name": "P Á B Ł Õ", "screen_name": "erose000", "lang": "en", "location": "United States", "create_at": date("2009-04-20"), "description": "310", "followers_count": 642, "friends_count": 451, "statues_count": 12416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231651112488961, "text": "@yagirlclair @whtevername @1208Hernandez same kids (:", "in_reply_to_status": 721231571248754688, "in_reply_to_user": 1650996457, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1650996457, 3178863470, 3240167022 }}, "user": { "id": 3256396692, "name": "yani", "screen_name": "yaannnniiii", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2015-06-26"), "description": "gvhs", "followers_count": 451, "friends_count": 268, "statues_count": 7593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231651288690690, "text": "Things will never be the same between us but I guess it's still worth a try .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1276234033, "name": "shameika haynes", "screen_name": "HaynesShameika", "lang": "en", "location": "null", "create_at": date("2013-03-17"), "description": "you are braver than you believe, stronger than you seem , and smarter than you think.", "followers_count": 215, "friends_count": 189, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231651590635520, "text": "@allihurd YESSS!!! MA'AM!", "in_reply_to_status": 721231084541726724, "in_reply_to_user": 2664703364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2664703364 }}, "user": { "id": 3612531254, "name": "Anthony Carrillo", "screen_name": "antcar344", "lang": "en", "location": "null", "create_at": date("2015-09-18"), "description": "//LAHS//", "followers_count": 161, "friends_count": 262, "statues_count": 261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Puente Valley, CA", "id": "b5b11d0f5cc91e3c", "name": "West Puente Valley", "place_type": "city", "bounding_box": rectangle("-117.990493,34.033699 -117.94947,34.066049") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684774, "cityName": "West Puente Valley" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231651762638848, "text": "@catteomac @halsteadsass IT'S SO GROOOOSSSS", "in_reply_to_status": 721133304611667968, "in_reply_to_user": 371622698, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371622698, 2705207426 }}, "user": { "id": 15079084, "name": "therese", "screen_name": "bamfpire", "lang": "en", "location": "arizona", "create_at": date("2008-06-10"), "description": "founded @nerdophiles. my opinions, obnoxious as they might be, are my own. i review #AoS, #shadowhunters, #outlander, #WynonnaEarp. #spoilers", "followers_count": 424, "friends_count": 714, "statues_count": 23042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231651917803520, "text": "currently v confused at my life's situation at this exact moment in time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258175169, "name": "Kacie Van Stiphout", "screen_name": "KacieVS", "lang": "en", "location": "aka: kacie van bean sprout ", "create_at": date("2011-02-26"), "description": "I could probably go for an acaí bowl or some almond butter right now. hummus also flows through my veins. //KKG @ UO\\\\", "followers_count": 396, "friends_count": 540, "statues_count": 5273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231652081381376, "text": "انا مسيّر وانت مثلي مسيّر\nتبيني وتقفي .. وأقفّي وانا ابيك \n.\nأنا لو إني في حياتي مخيّر\nما كان خلّيتك وأنا خاطري فيك", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 295769741, "name": "طـلال الـكــنـدري", "screen_name": "TalalAlKandari", "lang": "en", "location": "Q8 - Fl - TX", "create_at": date("2011-05-09"), "description": "A Marketing student at @UTSA & Assistant Executive Director at the Retreat SA #Free_Palestine #Humanity الدين معاملة •", "followers_count": 813, "friends_count": 700, "statues_count": 11725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231652559523840, "text": "Interested in a #Nursing #job near #Burlington, MA? This could be a great fit: https://t.co/RNlOZ5wRJk #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1956205,42.5047161"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "Burlington", "Hiring" }}, "user": { "id": 3011627064, "name": "Lahey Careers", "screen_name": "LaheyCareers", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "null", "followers_count": 41, "friends_count": 32, "statues_count": 457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231652991545344, "text": "Temp: 41.2F W C: 41.2F Wind:--- at 0.0kts Baro: 1034.3mb and Steady Rain today: 0.00in R H: 83% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 209, "friends_count": 219, "statues_count": 102441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231653595533313, "text": "Life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1576436876, "name": "Jerome Cadiz", "screen_name": "JeromeExists", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "Save life. Realizer | Runner | Wannabe Model | '80s Blood | Dancer | Believer", "followers_count": 124, "friends_count": 109, "statues_count": 1729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231653612314624, "text": "@EstoyQueEnamoro me los quite y no te leo", "in_reply_to_status": 721231139453538304, "in_reply_to_user": 137213091, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 137213091 }}, "user": { "id": 121486744, "name": "Arte Carnal", "screen_name": "ArteCarnal", "lang": "en", "location": "null", "create_at": date("2010-03-09"), "description": "Me gusta pintar mis sueños y mis deseos y lo que no e tenido. #ArteCarnal #MiArte", "followers_count": 784, "friends_count": 1272, "statues_count": 9859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis, AZ", "id": "6c6a7773365bea00", "name": "San Luis", "place_type": "city", "bounding_box": rectangle("-114.797084,32.479046 -114.75227,32.511968") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 463470, "cityName": "San Luis" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231653675241473, "text": "my bed has magic sobering powers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 494100655, "name": "kat.", "screen_name": "HurricaneK96", "lang": "en", "location": "null", "create_at": date("2012-02-16"), "description": "Snapchat: trinababay Insta: kat_tobekitten", "followers_count": 546, "friends_count": 242, "statues_count": 10140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231653704638464, "text": "I love you too my baby. ������ https://t.co/itcoqs7A4J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1112032405, "name": "Breee.", "screen_name": "breeconnell4", "lang": "en", "location": "Damon, Tx", "create_at": date("2013-01-22"), "description": "Ü Name's Breanna but known as Breee. I'm 20, graduated from Odem High School '15 . Capri♌️ Blue and #4 are my F A V. JUSTIN DREW BIEBER is mine. SINGLE.", "followers_count": 523, "friends_count": 467, "statues_count": 10757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wild Peach Village, TX", "id": "7ab8e8bc726e5e64", "name": "Wild Peach Village", "place_type": "city", "bounding_box": rectangle("-95.651447,29.057766 -95.584987,29.113626") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4879192, "cityName": "Wild Peach Village" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231653771718656, "text": "@robert3barajas true sorry /:", "in_reply_to_status": 721228299444203520, "in_reply_to_user": 2391821857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2391821857 }}, "user": { "id": 324168454, "name": "P♕", "screen_name": "paaulinaaiscool", "lang": "en", "location": "null", "create_at": date("2011-06-25"), "description": "#fuckcancer", "followers_count": 1203, "friends_count": 977, "statues_count": 26231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231654187098112, "text": "The boy in the striped pajamas kills me����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1605030264, "name": "alexisss", "screen_name": "_alexisharper_", "lang": "en", "location": "null", "create_at": date("2013-07-18"), "description": "live life with no worries", "followers_count": 140, "friends_count": 213, "statues_count": 140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-16T00:00:00.000Z"), "id": 721231654585561088, "text": "mas aí, are you living your life?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336055241, "name": "⋁", "screen_name": "vkzera", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-07-15"), "description": "Ima fan of the turns that the world gives", "followers_count": 5201, "friends_count": 290, "statues_count": 53920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231654891560960, "text": "@JkJahmal happy birthday catch these hands", "in_reply_to_status": 721230771290148864, "in_reply_to_user": 2413374378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2413374378 }}, "user": { "id": 390896927, "name": "drew", "screen_name": "DrewHladek", "lang": "en", "location": "LV ✈️ LA", "create_at": date("2011-10-14"), "description": "Cerritos College Track Squad | #WOOFGANG | 2014 State Champion | 4.58", "followers_count": 937, "friends_count": 386, "statues_count": 19159 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231655055175680, "text": "@yungfrecks what Is it", "in_reply_to_status": 721231283888652288, "in_reply_to_user": 326499389, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326499389 }}, "user": { "id": 428655701, "name": "iMinikon | ▲", "screen_name": "iMinikon", "lang": "en", "location": "RI Made ✈ (HEB) TX Raised", "create_at": date("2011-12-04"), "description": "• 23 Traveling Videographer • All Social Media: @iMinikon || A RETWEET OF A VIDEO ON @IMINIKONVIDS GOES A LONG WAY || #UNT", "followers_count": 1737, "friends_count": 363, "statues_count": 18887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656162500609, "text": "Mood https://t.co/BnD5jw3Jr4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2597118679, "name": "✨Addison✨", "screen_name": "Addi_boy99", "lang": "en", "location": "Evansville, IN", "create_at": date("2014-06-30"), "description": "So if a girl swallows ya kids is that a kids meal?", "followers_count": 1964, "friends_count": 969, "statues_count": 16986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656179269632, "text": "#Trump in 2016 Temp:56.3°F Wind:3.6mph Pressure: 30.21hpa Rising Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 315326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656376373248, "text": "I LOVE YEW HE IS A BIG BABY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 794155483, "name": "give her shoes", "screen_name": "ZEPHlRlN", "lang": "en", "location": "@MUNECHlKA", "create_at": date("2012-08-31"), "description": "Star/F/22 I scream about otome games, Rps @ CITTA ENG // 日本語 OK!! Seiza Marebito @ Hyperion", "followers_count": 190, "friends_count": 352, "statues_count": 44156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656405770240, "text": "@haileylsimons happy birthday! Love you, sowwy it's so late! ����", "in_reply_to_status": 721225724103819264, "in_reply_to_user": 2836491823, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2836491823 }}, "user": { "id": 161431770, "name": "Courtney Lynn", "screen_name": "simply_clx", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-06-30"), "description": "null", "followers_count": 595, "friends_count": 407, "statues_count": 14924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656737091584, "text": "happy birthday @sofiaaaa240 you deserve nothing but the best. enjoy your day. love you������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2875130826 }}, "user": { "id": 3087735512, "name": "jamieteresa", "screen_name": "jamiesterr", "lang": "en", "location": "walking with jesus christ ", "create_at": date("2015-03-15"), "description": "null", "followers_count": 668, "friends_count": 541, "statues_count": 4296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fowler, CA", "id": "002a82ff3d3b9828", "name": "Fowler", "place_type": "city", "bounding_box": rectangle("-119.693182,36.597948 -119.637458,36.642439") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 625436, "cityName": "Fowler" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231656930119680, "text": "Wind 0.4 mph WNW. Barometer 29.972 in, Steady. Temperature 61.3 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231657337008128, "text": "Temp: 60.6°F Wind:0.4mph Pressure: 30.063hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231657857064960, "text": "uhhh wtffff was thaaat ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3113499900, "name": "yvette", "screen_name": "titusyvette", "lang": "en", "location": "null", "create_at": date("2015-03-28"), "description": "null", "followers_count": 356, "friends_count": 474, "statues_count": 6448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madras, OR", "id": "363a256b7c2c3e3c", "name": "Madras", "place_type": "city", "bounding_box": rectangle("-121.150688,44.607204 -121.109014,44.647335") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41031, "countyName": "Jefferson", "cityID": 4145250, "cityName": "Madras" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231657877905410, "text": "So if anyone would like to buy either one of these for me, I would accept them in a heartbeat. https://t.co/ojjlcyxuQV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 527929512, "name": "Cadence Mae✨", "screen_name": "Cadence_mae0106", "lang": "en", "location": "null", "create_at": date("2012-03-17"), "description": "MSU coed cheer❤️", "followers_count": 668, "friends_count": 368, "statues_count": 5291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa Park, TX", "id": "010b5dffb4a0973b", "name": "Iowa Park", "place_type": "city", "bounding_box": rectangle("-98.696673,33.935929 -98.655598,33.976054") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4836104, "cityName": "Iowa Park" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231658028916736, "text": "Ying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2497500571, "name": "Perry", "screen_name": "BrockLashley1", "lang": "en", "location": "null", "create_at": date("2014-05-15"), "description": "Swag money //\\\\ Andover high school", "followers_count": 612, "friends_count": 719, "statues_count": 1139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231658100240384, "text": "Fingers on my Butterfinger #IrritateMeIn4Words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IrritateMeIn4Words" }}, "user": { "id": 413178295, "name": "Mike DeRusha", "screen_name": "Mike_DeRusha", "lang": "en", "location": "Albany, OR", "create_at": date("2011-11-15"), "description": "Wannabe comedian and writer.\nFor real musician & photographer.\n\nIf I'm older than your mom, you probably suck.", "followers_count": 199, "friends_count": 112, "statues_count": 8840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albany, OR", "id": "6083b03ae37cd913", "name": "Albany", "place_type": "city", "bounding_box": rectangle("-123.159583,44.577589 -123.032415,44.684678") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4101000, "cityName": "Albany" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231658112970752, "text": "@9944Farah ��good night friend��✨����✨��������✨", "in_reply_to_status": -1, "in_reply_to_user": 1704950851, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1704950851 }}, "user": { "id": 748675542, "name": "Cristobal Reyes", "screen_name": "crisf2525", "lang": "es", "location": "null", "create_at": date("2012-08-09"), "description": "null", "followers_count": 1314, "friends_count": 1338, "statues_count": 15356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231658230218753, "text": "@gillianzeee yeah they're both named after our dad ������", "in_reply_to_status": 721231516617953280, "in_reply_to_user": 1967632080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1967632080 }}, "user": { "id": 2583335587, "name": "Auddsss", "screen_name": "celeste_audrey", "lang": "en", "location": "Rialto, CA", "create_at": date("2014-06-22"), "description": "@delgui_21 is soooo cute , I think Imma keep him | chs", "followers_count": 236, "friends_count": 204, "statues_count": 677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-04-16T00:00:01.000Z"), "id": 721231658238795776, "text": "Wind 4.9 mph E. Barometer 30.13 in, Falling slowly. Temperature 60.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659023101952, "text": "Temp 30.8° Hi/Lo 34.6/30.8 Rng 3.8° WC 30.8° Hmd 83% Rain 0.00\" Storm 0.00\" BAR 30.454 Falling DP 26.2° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659039895552, "text": "04/16@03:00 - Temp 38.9F, WC 38.9F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.414in, Falling slowly. Rain 0.00in. Hum 82%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659069100033, "text": "@ebbtideapp Tide in Stanwood, Washington 04/16/2016\nHigh 2:50am 6.7\n Low 11:02am 1.3\nHigh 2:16pm 5.2\n Low 10:14pm 0.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.3667,48.2333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 72, "friends_count": 1, "statues_count": 24842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53029, "countyName": "Island", "cityID": 5309365, "cityName": "Camano" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659610165248, "text": "Same af �� https://t.co/G2H0brpOL0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298846640, "name": "Buddy™", "screen_name": "BDugasss", "lang": "en", "location": "Where Playa's get chose ", "create_at": date("2011-05-14"), "description": "A LEGEND at 20| Rest Easy Pops & Joe L. NeighborhoodSupaStar ✨. Rayne, Louisiana where I'm located bitch", "followers_count": 1936, "friends_count": 888, "statues_count": 135594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rayne, LA", "id": "807281e8b753fb7e", "name": "Rayne", "place_type": "city", "bounding_box": rectangle("-92.296633,30.198776 -92.20768,30.264282") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22001, "countyName": "Acadia", "cityID": 2263645, "cityName": "Rayne" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659815813120, "text": "Wind 0.0 mph ESE. Barometer 30.095 in, Steady. Temperature 57.4 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231659920723968, "text": "Wind 1.0 mph ESE. Barometer 30.168 in, Falling slowly. Temperature 59.9 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660134457344, "text": "The thing is I can play it better ��✌", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3708930973, "name": "AprilO'Nealchannel6", "screen_name": "AprilLoveJohn", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-09-27"), "description": "Guess the only way to survive in this world is with a cold heart.", "followers_count": 59, "friends_count": 112, "statues_count": 138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660147023873, "text": "Actually I had a very eventful day & talked to a lot of ppl �� stop with the assumptions before it make u look stupid https://t.co/elpVXUoq3q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466159249, "name": "Lil Saint", "screen_name": "TheReal_JayLo", "lang": "en", "location": "St. Louis, Missouri", "create_at": date("2012-01-16"), "description": "19 | Child of God | Kent State '19 The Fashion School | IG/SC @mickeyss_wifee.", "followers_count": 967, "friends_count": 820, "statues_count": 27844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, OH", "id": "45a0ea3329c38f9f", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-81.392451,41.11766 -81.308418,41.202841") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39133, "countyName": "Portage", "cityID": 3939872, "cityName": "Kent" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660348346369, "text": "dawggg ������ https://t.co/0Jg10nPakB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2197256603, "name": "phenomenal.", "screen_name": "__kingdalon", "lang": "en", "location": "MCMVI", "create_at": date("2013-11-26"), "description": "Stay Humble & Focused | Romans 8:18 | ΑΦΑ | UL", "followers_count": 1255, "friends_count": 784, "statues_count": 24835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monticello, LA", "id": "01417b3c030308be", "name": "Monticello", "place_type": "city", "bounding_box": rectangle("-91.060564,30.473007 -91.032397,30.505716") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2251645, "cityName": "Monticello" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660394491904, "text": "\"how did a girl just beat me in lane? why are u top lane and not support? u must be ugly\" - this just made my night~ ʱªʱªʱª (ᕑᗢूᓫ∗)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 110918879, "name": "♡ нᎧℓℓყ ♡ ٩(╹౪╹๑)۶", "screen_name": "CEOofCute", "lang": "en", "location": "Houston, TX", "create_at": date("2010-02-02"), "description": "徳永千奈美 推し~! Cuteness! Charm! & Potty Humor! Loud mouth artist from Jersey! I ♡ Berryz工房, LoL, LL & dogs.★★★★★★★★★★ KF ♡ 藤井梨央 MM ♡ 辻希美 & 加護亜依 & 工藤遥 TF ♡ 岸本ゆめの", "followers_count": 788, "friends_count": 288, "statues_count": 6204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660813922304, "text": "If I don't get these yeezys tmw..��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403915588, "name": "Jeremiah Pruitt", "screen_name": "_SFS_20", "lang": "en", "location": "Colorado Springs, CO", "create_at": date("2011-11-02"), "description": "#teamlowkey 480✈️719 F.C.H.W/DG I aint s#!t......yet ⚠️I do it for my brother..⚠️#Anapaulastrong", "followers_count": 733, "friends_count": 248, "statues_count": 12961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660814069764, "text": "I may not be there yet, but I’m closer than I was yesterday.\n#ALDUB9thMonthsary", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ALDUB9thMonthsary" }}, "user": { "id": 3669938115, "name": "Aldub Global NewYork", "screen_name": "aldubglobalnyc", "lang": "en", "location": "Queens, NY", "create_at": date("2015-09-15"), "description": "@aldenrichards02 and @mainedcm FRIENDS @ TRI-STATE AREA (NEW YORK CITY, NEW JERSEY, & CONNECTICUT) -Only All good vibes & Day To Day Inspirations", "followers_count": 4817, "friends_count": 1399, "statues_count": 14641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660851798016, "text": "I keep selling myself short with fuc niggas only cause da real thing requires the type of work Im not willing to do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.62435642,39.2972973"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25367676, "name": "MLSB", "screen_name": "MLSBLUVSFASHION", "lang": "en", "location": "ÜT: 39.312191,-76.608024off ", "create_at": date("2009-03-19"), "description": "Hard act to follow.....", "followers_count": 357, "friends_count": 486, "statues_count": 16385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mid-Town Belvedere, Baltimore", "id": "6d3faa62f1db26ee", "name": "Mid-Town Belvedere", "place_type": "neighborhood", "bounding_box": rectangle("-76.621013,39.299568 -76.611236,39.30948") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231660902002688, "text": "Omg twins win twins win", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328021485, "name": "Roy G. Biv", "screen_name": "Willynumnum", "lang": "en", "location": "preferrebly a buffet", "create_at": date("2011-07-02"), "description": "(insert life story)", "followers_count": 1066, "friends_count": 961, "statues_count": 26215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decorah, IA", "id": "fcb1bf1bf526aec7", "name": "Decorah", "place_type": "city", "bounding_box": rectangle("-91.811973,43.282274 -91.763174,43.331261") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19191, "countyName": "Winneshiek", "cityID": 1919405, "cityName": "Decorah" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231661095088128, "text": "Call on Him!! Amen @_lovecollide @ First Assembly of God (Hawaii) https://t.co/tGNNCJAx5Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.9020538,21.3653507"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 263910603 }}, "user": { "id": 1862378095, "name": "Michelle Ebalaroza", "screen_name": "MEbalaroza", "lang": "en", "location": "Hawaii, USA", "create_at": date("2013-09-13"), "description": "null", "followers_count": 202, "friends_count": 376, "statues_count": 4537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231662231592961, "text": "@TC_Instruction Yes please do let me know. ����", "in_reply_to_status": 721231323629637633, "in_reply_to_user": 745367053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 745367053 }}, "user": { "id": 39138136, "name": "Kara Hoover", "screen_name": "KaraKoo", "lang": "en", "location": "Dallas", "create_at": date("2009-05-10"), "description": "Sooner, Sports Nerd & Social Media Junkie. Obsessed w/: Blackhawks, Twins, FC Dallas & PGA Tour. Also tweet about: Golf Apparel (Instagram=@Fairwaythreads)", "followers_count": 1053, "friends_count": 807, "statues_count": 15420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Addison, TX", "id": "570002ce3d2a7ac1", "name": "Addison", "place_type": "city", "bounding_box": rectangle("-96.856247,32.932133 -96.807772,32.987442") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4801240, "cityName": "Addison" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231662303035392, "text": "The calm before the storm.., \n\n#makerfaireMV #minimakerfaire #makerfaire https://t.co/knmZj4Lyn9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.227337,43.136945"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "makerfaireMV", "minimakerfaire", "makerfaire" }}, "user": { "id": 4872921297, "name": "MakerFaireMV", "screen_name": "makerfaireMV", "lang": "en", "location": "Utica, NY", "create_at": date("2016-02-02"), "description": "The Mohawk Valley Mini Maker Faire is a celebration of the creativity, ingenuity, and diversity of makers throughout the Mohawk Valley region. #makerfairemv", "followers_count": 56, "friends_count": 152, "statues_count": 55 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SUNY IT", "id": "07d9ed0c53887002", "name": "SUNY IT", "place_type": "poi", "bounding_box": rectangle("-75.2273371,43.136944899999996 -75.227337,43.136945") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida" } }
+{ "create_at": datetime("2016-04-16T00:00:02.000Z"), "id": 721231662437113856, "text": "66.3F (Feels: 66.3F) - Humidity: 92% - Wind: 7.6mph E - Gust: 7.6mph - Pressure: 1008.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663267737600, "text": "Wind 0.0 mph NW. Barometer 30.47 in, Steady. Temperature 72.0 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 159096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663410204672, "text": "@ChrisMFkinTrinh @xsupafreshx ima be changing my name", "in_reply_to_status": 721229851462479874, "in_reply_to_user": 1733649828, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1733649828, 1452766400 }}, "user": { "id": 2350535210, "name": "Yung Fukurou", "screen_name": "tristfudge", "lang": "en", "location": "null", "create_at": date("2014-02-18"), "description": "#DamatoCrew #mfhomie$ UNDXRGR0VND 275 $vN D!€g0 ima be a waterboy till my soul floats away X NewEraGang", "followers_count": 409, "friends_count": 418, "statues_count": 3652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663569747970, "text": "03:00:01 |Temp: 55.9ºF |Dew Point 51.8ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663653519360, "text": "I love scary movies but I hate watching scary movies. Do you see my dilemma?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346166539, "name": "Ratchel", "screen_name": "RachelKoljesky", "lang": "en", "location": "Monmouth, OR", "create_at": date("2011-07-31"), "description": "Come correct or don't come at all", "followers_count": 325, "friends_count": 275, "statues_count": 13701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monmouth, OR", "id": "943b8d68d76b386c", "name": "Monmouth", "place_type": "city", "bounding_box": rectangle("-123.248721,44.836591 -123.208941,44.863769") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41053, "countyName": "Polk", "cityID": 4149550, "cityName": "Monmouth" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663754125313, "text": "why are boys so frustrating.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2833364071, "name": "ambs✨", "screen_name": "amberrupee_", "lang": "en", "location": "null", "create_at": date("2014-09-26"), "description": "red mountain high school//class of 2018. i'm highly obsessed with frozen and cats.", "followers_count": 526, "friends_count": 1134, "statues_count": 12933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231663997423616, "text": "@estradakaela beautiful goals ihy ��", "in_reply_to_status": 721220983445520384, "in_reply_to_user": 2868684040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2868684040 }}, "user": { "id": 626699139, "name": "victoria", "screen_name": "Victoriaswagga", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-07-04"), "description": "always know you deserve more", "followers_count": 338, "friends_count": 492, "statues_count": 5368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231664374874113, "text": "@TraeisMalice as your bio does mine ��", "in_reply_to_status": 721231334534955009, "in_reply_to_user": 1401041792, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1401041792 }}, "user": { "id": 282894832, "name": "the bad guy.", "screen_name": "HBfromKC", "lang": "en", "location": "Kansas City, MO", "create_at": date("2011-04-15"), "description": "*terms & conditions may apply. K=Fuck You. Beards Are Life. Black is beautiful. Founder of #PeepDontSpeak #BUSGF #TimsRightHand #BBW DMs Closed", "followers_count": 2718, "friends_count": 1826, "statues_count": 180584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231664526008320, "text": "Wind 0.0 mph N. Barometer 30.485 in, Steady. Temperature 30.1 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231664802828288, "text": "NY's yearly visit of his friends at Central Park Zoo #newyorknaruepa… https://t.co/W4Zj92QeB5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.97181217,40.7675423"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "newyorknaruepa" }}, "user": { "id": 56046418, "name": "Pla Prinda", "screen_name": "plaprinda", "lang": "en", "location": "null", "create_at": date("2009-07-12"), "description": "The godmother of NEW YORK", "followers_count": 807, "friends_count": 116, "statues_count": 2612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231664869875712, "text": "shoutout to everyone who performed at SD live, you all did great����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1856311135, "name": "•Victoria•", "screen_name": "hey_imvictoria", "lang": "en", "location": "snapchat: vee_rrr ✨", "create_at": date("2013-09-11"), "description": "•http://So.Cal• ~SDHS '16~", "followers_count": 75, "friends_count": 91, "statues_count": 3892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231664991571969, "text": "42 ��⚾️ @ Dodger Stadium https://t.co/5xrcVSW9PB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.24037936,34.07390024"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1443044364, "name": "Stephanie Cristina", "screen_name": "DodgerDashGirl", "lang": "en", "location": "null", "create_at": date("2013-05-20"), "description": "null", "followers_count": 80, "friends_count": 375, "statues_count": 984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231665360539648, "text": "@yassssica omg that vid", "in_reply_to_status": 721231620984979456, "in_reply_to_user": 14129705, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14129705 }}, "user": { "id": 161782758, "name": "Sandra", "screen_name": "sxndravides", "lang": "en", "location": "null", "create_at": date("2010-07-01"), "description": "pushing through the black and white", "followers_count": 304, "friends_count": 267, "statues_count": 15890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231665713057792, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.393 in, Falling. Temp 34.7F. Rain today 0.00 in. Humidity 90% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 614, "friends_count": 821, "statues_count": 40811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666040020992, "text": "why do cute people snap me when I look awful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1356183649, "name": "mandy", "screen_name": "amandaaman_", "lang": "en", "location": "probably listening to j. cole", "create_at": date("2013-04-15"), "description": "virgo/libra / rest in peace niko", "followers_count": 648, "friends_count": 987, "statues_count": 6237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Shore, WA", "id": "8be0d1aae242df9f", "name": "Lake Shore", "place_type": "city", "bounding_box": rectangle("-122.706188,45.678391 -122.679496,45.707311") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5337830, "cityName": "Lake Shore" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666144882688, "text": "Debating whether to stay in LA or leave to AZ.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3073736624, "name": "•Sarah Kardashian•", "screen_name": "saraagxox_", "lang": "en", "location": "LA", "create_at": date("2015-03-11"), "description": "Cause' somebody stole my car radio and now I just sit in silence. #FifthHarmony #1D #TwentyOnePilots #zquad", "followers_count": 228, "friends_count": 196, "statues_count": 8067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666438496256, "text": "���������� https://t.co/8qa5b1NaBW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2638774787, "name": "GEMINI", "screen_name": "thereaaldee", "lang": "en", "location": "null", "create_at": date("2014-06-24"), "description": "null", "followers_count": 1029, "friends_count": 477, "statues_count": 34161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666518310912, "text": "@Cyrecyka this run I'm gonna go devourer, rage blade, wits end and frozen heart", "in_reply_to_status": 721231531688271873, "in_reply_to_user": 3307846126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3307846126 }}, "user": { "id": 3289402951, "name": "jrd", "screen_name": "Majinbuuu__", "lang": "en", "location": "null", "create_at": date("2015-07-23"), "description": "21 , nerd pretty much", "followers_count": 2374, "friends_count": 1040, "statues_count": 106253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666748887040, "text": "����‼️ https://t.co/qHjH9kchA4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3244027849, "name": "Valeace ❤", "screen_name": "pretttyyvee", "lang": "en", "location": "Jackson, MS", "create_at": date("2015-06-13"), "description": "Very Beautiful⭐️| ❤️✨Sister Bee @_fineeeeeXB page too ❤️✨|", "followers_count": 1283, "friends_count": 1097, "statues_count": 17617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666748923904, "text": "Wind 6.4 mph E. Barometer 29.839 in, Falling slowly. Temperature 60.9 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-16T00:00:03.000Z"), "id": 721231666988118017, "text": "Free tht bitch Niya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257960070, "name": "sc:kashmoneyky", "screen_name": "MrWellpaid", "lang": "en", "location": "null", "create_at": date("2011-02-26"), "description": "Tryna obtain a million #LSU19", "followers_count": 366, "friends_count": 358, "statues_count": 5370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231667831054336, "text": "Austin makes me so happy :,)) ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2600232133, "name": "Jordyn Farley", "screen_name": "farley_jordyn", "lang": "en", "location": "las vegas ", "create_at": date("2014-07-02"), "description": "avhs~ @austiniskent❤️", "followers_count": 520, "friends_count": 369, "statues_count": 14362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231667868737536, "text": "Thems my girls! ������ https://t.co/7ToXp7SZ2X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3101909576, "name": "Jay baby ♥️", "screen_name": "Hooperjay1", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2015-03-21"), "description": "IG : ambitious_hooper sc : that_hooperjay1", "followers_count": 2249, "friends_count": 1519, "statues_count": 3794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231668380569600, "text": "July Films: 1 -Moonstruck, 8 -Diner, 15 -The Sound of Music, 22 -More than a Miracle, 29 -The Good, the Bad and the Ugly", "in_reply_to_status": 721230013601837057, "in_reply_to_user": 3616702522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3616702522, "name": "Little Italy, MD", "screen_name": "LittleItaly_BAL", "lang": "en", "location": "Little Italy, Baltimore", "create_at": date("2015-09-10"), "description": "null", "followers_count": 80, "friends_count": 120, "statues_count": 55 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231669819256833, "text": "Just posted a photo @ Texas Butter - txbut https://t.co/9bUA2hq3dM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.202829,29.6657711"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2837466572, "name": "wJ", "screen_name": "wJTEEVEE", "lang": "en", "location": "null", "create_at": date("2014-10-01"), "description": "null", "followers_count": 298, "friends_count": 943, "statues_count": 6953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231669890387968, "text": "Oh well", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2992901840, "name": "Fudge Me!", "screen_name": "_SavagesSavages", "lang": "en", "location": "null", "create_at": date("2015-01-22"), "description": "Something cool", "followers_count": 53, "friends_count": 50, "statues_count": 5538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231670343569408, "text": "It's soooooo earlyyyyyy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4686470352, "name": "man made seizure", "screen_name": "definitelyntali", "lang": "en", "location": "fruit bowl ", "create_at": date("2015-12-31"), "description": "I'm a grape", "followers_count": 104, "friends_count": 157, "statues_count": 668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopedale, MA", "id": "daf32d97fbea4de2", "name": "Hopedale", "place_type": "city", "bounding_box": rectangle("-71.564257,42.09931 -71.499704,42.153349") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2530980, "cityName": "Hopedale" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231670767017984, "text": "Mix the dark and light I will act a plum fool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1115641050, "name": "Scoop", "screen_name": "_STAYJORDAN", "lang": "en", "location": "1 minute past 3", "create_at": date("2013-01-23"), "description": "null", "followers_count": 1170, "friends_count": 588, "statues_count": 73517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowie, MD", "id": "369842624da7239c", "name": "Bowie", "place_type": "city", "bounding_box": rectangle("-76.800034,38.8898 -76.693243,39.037765") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2408775, "cityName": "Bowie" } }
+{ "create_at": datetime("2016-04-16T00:00:04.000Z"), "id": 721231670783959040, "text": "I was fine when I fell asleep but I just now woke up dying", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396971588, "name": "brooklyne ♢", "screen_name": "sbrooklyne_15", "lang": "en", "location": "Carlisle, OH", "create_at": date("2011-10-23"), "description": "your future psychiatrist", "followers_count": 502, "friends_count": 291, "statues_count": 11102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miamisburg, OH", "id": "6252d06c3e5bb9f9", "name": "Miamisburg", "place_type": "city", "bounding_box": rectangle("-84.30945,39.584523 -84.202828,39.675365") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3949434, "cityName": "Miamisburg" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231671589097472, "text": "This rant was the best! I'm relaxing at the rental while the friends are at Coachella - it's Couchella, w/ a pool. https://t.co/3ETvXvCJCR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25245954, "name": "Ashley", "screen_name": "ItsJustMeAshley", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2009-03-18"), "description": "Personal assistant extraordinaire and rescuer of animals with a fondness for Schnauzers and Brussels Griffons. IG: ItsJustMeAshley", "followers_count": 193, "friends_count": 319, "statues_count": 7557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.567788,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231671668805632, "text": "Throwback songs, always get me in the feels tbh. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263040236, "name": "tmsn.", "screen_name": "tmn_23", "lang": "en", "location": "Arizona, USA", "create_at": date("2013-03-12"), "description": "null", "followers_count": 308, "friends_count": 218, "statues_count": 17870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231671886897152, "text": "If anyone else wants me to add their https://t.co/xXc90dYv1l into let me know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15877967, "name": "mattmagician", "screen_name": "mattmagician", "lang": "en", "location": "Cleveland, OH", "create_at": date("2008-08-16"), "description": "Pixlbot Team Lead, Nerd, Friend.", "followers_count": 556, "friends_count": 1086, "statues_count": 19446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willoughby, OH", "id": "88dfdf4f649a0ba2", "name": "Willoughby", "place_type": "city", "bounding_box": rectangle("-81.452726,41.603963 -81.370884,41.707193") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985484, "cityName": "Willoughby" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231672964997120, "text": "I'm at V-Nyce's Crib in Valley Stream, NY https://t.co/fBWEC1stWt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.71836391,40.68262247"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481675603, "name": "Vincent Edwards III", "screen_name": "ItzTrikkz", "lang": "en", "location": "Valley Stream, NY", "create_at": date("2012-02-02"), "description": "null", "followers_count": 68, "friends_count": 537, "statues_count": 2925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Valley Stream, NY", "id": "cdc37bae2a45fc33", "name": "North Valley Stream", "place_type": "city", "bounding_box": rectangle("-73.726621,40.675855 -73.679609,40.696918") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3653748, "cityName": "North Valley Stream" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231673069678592, "text": "the beach I drove to at 2 in the morning to have one of the weirdest and funniest conversations with justin I've ever had..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 79, "friends_count": 220, "statues_count": 124312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231674101510144, "text": "@hsndrn78 Kadınları diri diri kara çuvala gömmeyi din sanan bir sünni olduğun için erdemli bir kadına sataşıyorsun. Düşün.\n@ozlemavci114", "in_reply_to_status": 721189598261559300, "in_reply_to_user": 3129166635, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 3129166635, 2906229669 }}, "user": { "id": 238776270, "name": "Edip Yüksel", "screen_name": "edipyuksel", "lang": "en", "location": "Arizona, USA", "create_at": date("2011-01-15"), "description": "For English tweets follow: @19org Read\nQuran: a Reformist Translation + \nManifesto for Islamic Reform\n+1 520 481 1919", "followers_count": 45980, "friends_count": 205, "statues_count": 32518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231674256842753, "text": "Nike too https://t.co/bDnvGcVoSC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949245558, "name": "Vlone Thug", "screen_name": "Yung_fendi", "lang": "en", "location": "the tommy sto", "create_at": date("2014-12-28"), "description": "you a lightweight and you can't finesse. IG: fendiiiii_ sc-wavyshit", "followers_count": 318, "friends_count": 517, "statues_count": 12042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville, IL", "id": "1c41ff1e4f918408", "name": "Jacksonville", "place_type": "city", "bounding_box": rectangle("-90.292217,39.712209 -90.171866,39.759846") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17137, "countyName": "Morgan", "cityID": 1738115, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231674453934080, "text": "Awesomesauce! @ French Camp, California https://t.co/sUivebd6Xa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.28,37.8827"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1946305274, "name": "julio lopez", "screen_name": "juliolomlopez81", "lang": "en", "location": "FC", "create_at": date("2013-10-08"), "description": "TWEETLEDEEEE....", "followers_count": 597, "friends_count": 1927, "statues_count": 4227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Camp, CA", "id": "223dd5d8e83a71b8", "name": "French Camp", "place_type": "city", "bounding_box": rectangle("-121.305096,37.855527 -121.264446,37.897212") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 626028, "cityName": "French Camp" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231674663501824, "text": "sigh local gays are a curse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1654627068, "name": "emotional shawty", "screen_name": "daviflapjacks", "lang": "en", "location": "Vancouver, WA", "create_at": date("2013-08-07"), "description": "i make it look easy cause it is to me @gothmoneytkeyah", "followers_count": 1121, "friends_count": 600, "statues_count": 36662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231675347247104, "text": "https://t.co/M5pZNlOqrM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2421396558, "name": "HaleyG", "screen_name": "Haleyyranee", "lang": "en", "location": "null", "create_at": date("2014-03-31"), "description": "null", "followers_count": 450, "friends_count": 249, "statues_count": 173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231675359793152, "text": "Adjustando algunos detalles para proyecto Vesperia #FandubLatino https://t.co/ZV7dFXxCbF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "FandubLatino" }}, "user": { "id": 483443887, "name": "Chris, El Juve", "screen_name": "Soohable", "lang": "en", "location": "United States", "create_at": date("2012-02-04"), "description": "Passionate Java programmer, web developer, and systems engineer.", "followers_count": 39, "friends_count": 18, "statues_count": 876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-16T00:00:05.000Z"), "id": 721231675628257280, "text": "@YamanAlAhmed probably deserved it", "in_reply_to_status": 721225099404152833, "in_reply_to_user": 554870804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 554870804 }}, "user": { "id": 862659301, "name": "Young $!Mba", "screen_name": "malikq99", "lang": "en", "location": "null", "create_at": date("2012-10-05"), "description": "Fresno state. Ride or Die!", "followers_count": 530, "friends_count": 277, "statues_count": 7092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231676731297792, "text": "THANKS CHANCYYY ���� https://t.co/qdM1RwJk2i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1326630397, "name": "jerms", "screen_name": "jeremyvchack", "lang": "en", "location": "Murrieta, CA", "create_at": date("2013-04-04"), "description": "null", "followers_count": 341, "friends_count": 218, "statues_count": 1696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231676777431040, "text": "Send spam messages", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120293736, "name": "ひ", "screen_name": "thealmightyluis", "lang": "en", "location": "null", "create_at": date("2010-03-05"), "description": "#ThuggerHive//Spurs//Sooners//Seahawks", "followers_count": 1094, "friends_count": 820, "statues_count": 23697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231677138141188, "text": "@sumairaaraynee daaaamm I just started watching a new show on Hulu and it's super good", "in_reply_to_status": 721228767331377152, "in_reply_to_user": 629869076, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 629869076 }}, "user": { "id": 3279837762, "name": "jess", "screen_name": "jessicaLstotts", "lang": "en", "location": "Mission Viejo, CA", "create_at": date("2015-07-14"), "description": "people occasionally call me jessicake || @VirtueRiley", "followers_count": 147, "friends_count": 123, "statues_count": 4462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231677318561792, "text": "I jus talk to you when I don't want youuuu https://t.co/9io0HVmWRI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 286401743, "name": "Morena Roas", "screen_name": "MorenaRoas", "lang": "en", "location": "United States of Houston", "create_at": date("2011-04-22"), "description": "Miss Worldwide n Mcee Performer n Artist so basically Talented hv a Olympic Medal in backhanding.Wat u say bt my Mama", "followers_count": 915, "friends_count": 706, "statues_count": 14158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231677608050688, "text": "Temp: 71.4°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.97 in | Dewpoint: 71.2°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 166629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231677800841216, "text": "Hate when people stare and don't speak. WELL HELLO tf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579300723, "name": "Jaren ➰", "screen_name": "yeah_itsjay", "lang": "en", "location": "214✈️713", "create_at": date("2012-05-13"), "description": "eighteen, dancer, student | #uh19", "followers_count": 1016, "friends_count": 716, "statues_count": 13846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231678228836352, "text": "yeezus christ @ Coachella https://t.co/8rh9zglYu9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.237221,33.679974"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40556097, "name": "Gabi Feingold", "screen_name": "gabiiifeingold", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-16"), "description": "California State University, Northridge '19 Music Enthusiast ♬", "followers_count": 299, "friends_count": 477, "statues_count": 8150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231678522273792, "text": "And I also love being quoted for things I never said", "in_reply_to_status": 721231520174747648, "in_reply_to_user": 2500408218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2500408218, "name": "Maya", "screen_name": "princessMayabug", "lang": "en", "location": "Anaheim", "create_at": date("2014-05-16"), "description": "I'm a mess who got her camera stolen at the happiest place on earth but I try to be cool about it", "followers_count": 158, "friends_count": 129, "statues_count": 2343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231678891548672, "text": "She now be cursed \nAnd it is true her world become smaller \nAnd she become weaker \nThe reason y, yee sand before curses \nYee know my son", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257448798, "name": "Cat Morre", "screen_name": "eyebelike3", "lang": "en", "location": "Hamilton, New Jersey", "create_at": date("2011-02-25"), "description": "null", "followers_count": 695, "friends_count": 1550, "statues_count": 7782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mercerville, NJ", "id": "00d52fb84a327694", "name": "Mercerville", "place_type": "city", "bounding_box": rectangle("-74.752772,40.184949 -74.655192,40.270908") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3445480, "cityName": "Mercerville" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231679029784576, "text": "Temp: 40.9°F - Dew Point: 34.5° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.23in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-16T00:00:06.000Z"), "id": 721231679671525378, "text": "You can't save them all...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16286476, "name": "R. Angermuller", "screen_name": "angermuller", "lang": "en", "location": "San Francisco", "create_at": date("2008-09-14"), "description": "Partner / Creative Director @ Math Times Joy, a San Francisco based design studio.", "followers_count": 1262, "friends_count": 382, "statues_count": 14507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594038634164224, "text": "can't even lie ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2677057096, "name": "T o r i ♥️..", "screen_name": "Torianaaa__", "lang": "en", "location": "null", "create_at": date("2014-07-05"), "description": "null", "followers_count": 1173, "friends_count": 670, "statues_count": 17525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Horn Lake, MS", "id": "5a8a3331fa3a363a", "name": "Horn Lake", "place_type": "city", "bounding_box": rectangle("-90.096466,34.918916 -89.996918,34.977346") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28033, "countyName": "DeSoto", "cityID": 2833700, "cityName": "Horn Lake" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594038764363776, "text": "@starlitup great lol", "in_reply_to_status": 721593878487310336, "in_reply_to_user": 17936432, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17936432 }}, "user": { "id": 24299238, "name": "KU®T SN❄W", "screen_name": "HDKG", "lang": "en", "location": "C o l u m b u s , O H 1 0", "create_at": date("2009-03-13"), "description": "I tweet all lies please don't believe none of this shit. #MFFL #TheColi", "followers_count": 1033, "friends_count": 775, "statues_count": 83033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594038982344705, "text": "My sleeping schedule is all the way messed up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55309292, "name": "Monty Mayfield", "screen_name": "ItsMontyCurtis", "lang": "en", "location": "Los Angeles, CA✈Wichita, KS", "create_at": date("2009-07-09"), "description": "R.I.P. Caleb Mayfield R.I.P. Grandpa Mayfield South Central L.A. \nF.O.E.", "followers_count": 510, "friends_count": 264, "statues_count": 38775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594039254913026, "text": "@lesleyy_xoxo which one?? I used to work at a Denny's �� and I hated it lmao", "in_reply_to_status": 721593642960334848, "in_reply_to_user": 2559896329, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2559896329 }}, "user": { "id": 628148405, "name": "jas ✨", "screen_name": "jasmartinez_", "lang": "en", "location": "null", "create_at": date("2012-07-06"), "description": "Kanye attitude with drake feelings", "followers_count": 1307, "friends_count": 566, "statues_count": 17412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594039410122753, "text": "I really want to fall asleep in this car", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2276837720, "name": "Bobby❄", "screen_name": "erriiccccc", "lang": "en", "location": "null", "create_at": date("2014-01-04"), "description": "Ain't nun.", "followers_count": 578, "friends_count": 319, "statues_count": 9657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del Aire, CA", "id": "af1bd9a7cea6c77e", "name": "Del Aire", "place_type": "city", "bounding_box": rectangle("-118.378565,33.902435 -118.361276,33.929174") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 618352, "cityName": "Del Aire" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594039686922240, "text": "@NikkilasLemear I'm living, maintaining, so I have no complaints. You going home at all this summer?", "in_reply_to_status": 721593319764037632, "in_reply_to_user": 441401330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 441401330 }}, "user": { "id": 523957794, "name": "$lim Jimmy", "screen_name": "Slim_Boyiee", "lang": "en", "location": "Austin,TX", "create_at": date("2012-03-13"), "description": "Somewhere on a court feeding boys buckets! Chase your dreams 'cause no one else will. '95. Brooklyn ✈️ Texas. #PV17", "followers_count": 1617, "friends_count": 1155, "statues_count": 47145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594039968120832, "text": "@Cyber_Gnarledge I have a research paper due Wednesday I'm going to recycle one from my capstone", "in_reply_to_status": 721589167482732546, "in_reply_to_user": 1406973319, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1406973319 }}, "user": { "id": 484311821, "name": "Orion #Bernie2k16", "screen_name": "reluctantowls", "lang": "en", "location": "Washington, DC", "create_at": date("2012-02-05"), "description": "Bronx Strong || Eph. 2:8 || JD / MPA '17 @americanu || Games, cosplays and animal rescues || Opinions here are garbage, completely unrelatable, and gluten-free.", "followers_count": 1323, "friends_count": 526, "statues_count": 3261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanover, PA", "id": "acddd1c346101ae2", "name": "Hanover", "place_type": "city", "bounding_box": rectangle("-77.000327,39.791053 -76.963141,39.832454") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42133, "countyName": "York", "cityID": 4232448, "cityName": "Hanover" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594040077180928, "text": "Sometimes you realize & it's too late, but then sometimes you find the person willing to let you right your wrongs. Never let that go.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3045658293, "name": "Mad Lover", "screen_name": "mesplyler15", "lang": "en", "location": "•4/06/16•", "create_at": date("2015-02-19"), "description": "I just drooled on myself ❤️in love with @_haleydarling❤️", "followers_count": 363, "friends_count": 681, "statues_count": 2551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pretty Bayou, FL", "id": "a478833db7bfc4fa", "name": "Pretty Bayou", "place_type": "city", "bounding_box": rectangle("-85.721111,30.189631 -85.676169,30.210841") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1258962, "cityName": "Pretty Bayou" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594040303493120, "text": "@krislin21 from tanning in a bed bro ������", "in_reply_to_status": 721593215271370753, "in_reply_to_user": 423164571, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423164571 }}, "user": { "id": 101381243, "name": "jess", "screen_name": "jessicaharris_1", "lang": "en", "location": "pnw", "create_at": date("2010-01-02"), "description": "western washington university", "followers_count": 624, "friends_count": 333, "statues_count": 31776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594040693592064, "text": "McDonald's breakfast @ 11 pm ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2383363098, "name": "LauraGomez", "screen_name": "lauragomezzzz", "lang": "en", "location": "Anchorage, AK", "create_at": date("2014-03-11"), "description": "1998", "followers_count": 287, "friends_count": 160, "statues_count": 803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041205260288, "text": "Beyond grateful for today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2805978907, "name": "ariana", "screen_name": "arixnaramirez", "lang": "en", "location": "null", "create_at": date("2014-09-12"), "description": "cogito ergo sum", "followers_count": 410, "friends_count": 407, "statues_count": 7372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenfield, CA", "id": "6684d4e60ddfc668", "name": "Greenfield", "place_type": "city", "bounding_box": rectangle("-121.260686,36.307617 -121.226796,36.339522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 630994, "cityName": "Greenfield" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041419206656, "text": "This #Healthcare #job might be a great fit for you: OPA Health Unit Coordinator - Mercy Medical Center - 28... - https://t.co/1XvUalX4u8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.5426136,44.0247062"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job" }}, "user": { "id": 4137429793, "name": "Ministry Health Jobs", "screen_name": "JobsAtMinistry", "lang": "en", "location": "Central and Northern Wisconsin", "create_at": date("2015-11-05"), "description": "Follow this page for open healthcare positions at @Ministry_Health We are #hiring in #Wisconsin! #jobs #job", "followers_count": 46, "friends_count": 45, "statues_count": 429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oshkosh, WI", "id": "01e2555c756cd031", "name": "Oshkosh", "place_type": "city", "bounding_box": rectangle("-88.653224,43.963944 -88.477447,44.102608") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55139, "countyName": "Winnebago", "cityID": 5560500, "cityName": "Oshkosh" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041645686784, "text": "@spikezezel2 yep. I lost my dad at 33.", "in_reply_to_status": 721593230329032704, "in_reply_to_user": 60298957, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60298957 }}, "user": { "id": 50462319, "name": "Dbright21", "screen_name": "Dbright21", "lang": "en", "location": "Sunny SoCal!", "create_at": date("2009-06-24"), "description": "all about my #Bills #BillsMafia", "followers_count": 2628, "friends_count": 3197, "statues_count": 83676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041779924992, "text": "@myrnaaascats OMG ILYSM", "in_reply_to_status": 721467671322079232, "in_reply_to_user": 1704914676, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 1704914676 }}, "user": { "id": 1048672747, "name": "Phoebo", "screen_name": "crystalsirrasin", "lang": "en", "location": "Placentia CA", "create_at": date("2012-12-30"), "description": "why don't you get on the 10 and get outta here", "followers_count": 131, "friends_count": 165, "statues_count": 1733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041968668673, "text": "Who won prom king and queen??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4848083241, "name": "mads", "screen_name": "__maddietaylor", "lang": "en", "location": "null", "create_at": date("2016-01-25"), "description": "I enjoy coffee and performing, in that order", "followers_count": 346, "friends_count": 322, "statues_count": 1383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594041997996034, "text": "If you snooze you'll lose. You'll be in the past, not the future.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388450833, "name": "Pete Puebla Jr.", "screen_name": "PeteyPablo623", "lang": "en", "location": "All over Southern California", "create_at": date("2011-10-10"), "description": "I'm in the business of helping people and cleaning California while doing it. #SoCal #SanDiego #ChulaVista #ElCajon #Escondito #Poway #Santee #RanchoBenardo", "followers_count": 1714, "friends_count": 2085, "statues_count": 120360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escondido, CA", "id": "00610f4d3a382ec1", "name": "Escondido", "place_type": "city", "bounding_box": rectangle("-117.146102,33.057782 -116.998535,33.182936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622804, "cityName": "Escondido" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042056904704, "text": "Told Tha bih you muthafuckin right", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176950801, "name": "Portia", "screen_name": "PortiaRi", "lang": "en", "location": "Texas", "create_at": date("2010-08-10"), "description": "#FollowME @PortiaRi on Instagram|Snapchat|Twitter *http://Fashion.Fun.Faith* #TeamLoyalFollowers #WhatModelsWant Change the World, with Confidence! God 1st", "followers_count": 2074, "friends_count": 767, "statues_count": 34005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042086076417, "text": "@JdotKimmy yea. But not rn.", "in_reply_to_status": 721593978945024001, "in_reply_to_user": 20971702, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20971702 }}, "user": { "id": 383463364, "name": "tAylor aRe", "screen_name": "TRichardsonWA", "lang": "en", "location": "null", "create_at": date("2011-10-01"), "description": "Hopeless Mariner Fan. WA State. Bachelors in Business Marketing.", "followers_count": 410, "friends_count": 372, "statues_count": 39072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042396454912, "text": "@rachelsheetss come over whenevr", "in_reply_to_status": 721593631581188097, "in_reply_to_user": 2576519665, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2576519665 }}, "user": { "id": 24832799, "name": "I Died In The Bible", "screen_name": "DieAbelDie", "lang": "en", "location": "Gallifrey, AZ", "create_at": date("2009-03-16"), "description": "Veteran, Korean War (North Korean Army)\nI'm a madman with a box, without a box\nI strum, speak, and yell in @foxhoundaz. @dragonbby_'s jerk. #FRABEL", "followers_count": 1158, "friends_count": 899, "statues_count": 68261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042501337089, "text": "Prayers and condolences go out to the Sula aiga in Kalihi. Rest easy Wesley. Ia manuia lau malaga. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1111230812, "name": "daONIoneyouneed", "screen_name": "KEONIKIAAAI", "lang": "en", "location": "Waianae, HI ✈️ Las Vegas, NV", "create_at": date("2013-01-22"), "description": "people say you are watchu eat. guess that's explains why im such a dick | SNAPCHAT: HEISKEONI | #lovewins", "followers_count": 1080, "friends_count": 1066, "statues_count": 51345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042551832577, "text": "I'm mad asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553565687, "name": "Shawdyyyy❣", "screen_name": "BlackGal___", "lang": "en", "location": "w/ tha gang ", "create_at": date("2012-04-14"), "description": "It's CamWorld or No World. Never let your faith die. sc: shaniecebabe. Bae Bae @_WarReadyy❤️", "followers_count": 1740, "friends_count": 1649, "statues_count": 56803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homewood, AL", "id": "d7bd87a33673fb63", "name": "Homewood", "place_type": "city", "bounding_box": rectangle("-86.84954,33.414818 -86.762633,33.488982") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 135800, "cityName": "Homewood" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042644082688, "text": "@TearraU counting all these bands aye, who da fuck are u aye ��", "in_reply_to_status": 721590289044017152, "in_reply_to_user": 3389626132, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3389626132 }}, "user": { "id": 1579184636, "name": "Connerrr.", "screen_name": "zoej__", "lang": "en", "location": "Richmond, IN ✈️", "create_at": date("2013-07-08"), "description": "null", "followers_count": 660, "friends_count": 414, "statues_count": 9856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-17T00:00:00.000Z"), "id": 721594042673426433, "text": "Tfw when you come to a party and its shut down��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3084926740, "name": "Shawn Island", "screen_name": "FastBreak_53", "lang": "en", "location": "west palm bitch ", "create_at": date("2015-03-10"), "description": "Live frat, die young.", "followers_count": 65, "friends_count": 81, "statues_count": 1765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594043054952448, "text": "It might be the hormones speaking but I wanna watch a sad movie lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33333558, "name": "Baddie B", "screen_name": "_britt2trilll", "lang": "en", "location": "null", "create_at": date("2009-04-19"), "description": "HOU. 23. UH. You ain't gotta like me, I like me.", "followers_count": 583, "friends_count": 460, "statues_count": 40083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channelview, TX", "id": "eabe7b6fd4504fff", "name": "Channelview", "place_type": "city", "bounding_box": rectangle("-95.176002,29.735548 -95.061098,29.851809") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4814236, "cityName": "Channelview" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594043424227329, "text": "Temp 30.5° Hi/Lo 34.0/30.5 Rng 3.5° WC 30.5° Hmd 86% Rain 0.00\" Storm 0.00\" BAR 30.471 Falling DP 26.8° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 118, "statues_count": 17946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594044116287488, "text": "Worse? When the girl in the uber pool chews gum louder than ever imagined? All because the 4 isn't running.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58227630, "name": "Rebecca Flynn", "screen_name": "rebeflynn", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-07-19"), "description": "Coffee lover, craft beer enthusiast, wine yogi | Beverage Assistant for Cushman Concepts and @oyanyc Server", "followers_count": 641, "friends_count": 675, "statues_count": 3240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594044137082882, "text": "#Retail #Job in #WALTERBORO, SC: Bakery Associate at BI-LO https://t.co/Q0fpbIdZNU #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.6681815,32.9203943"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "WALTERBORO", "Jobs", "Hiring" }}, "user": { "id": 2706829250, "name": "BI-LO Jobs", "screen_name": "BiLoJobs", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "null", "followers_count": 4, "friends_count": 0, "statues_count": 12178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walterboro, SC", "id": "93cd66c83e6d95e7", "name": "Walterboro", "place_type": "city", "bounding_box": rectangle("-80.721781,32.874749 -80.642543,32.934116") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45029, "countyName": "Colleton", "cityID": 4574275, "cityName": "Walterboro" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594044363747329, "text": "They don't hear you!! https://t.co/coNyedzbK6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 853272330, "name": "Maddy", "screen_name": "maddyfrank24", "lang": "en", "location": "cuse | oneonta", "create_at": date("2012-09-29"), "description": "•xo•", "followers_count": 601, "friends_count": 407, "statues_count": 8784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oneonta, NY", "id": "ff9cc975fb01f747", "name": "Oneonta", "place_type": "city", "bounding_box": rectangle("-75.108773,42.433438 -74.980422,42.478019") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36077, "countyName": "Otsego", "cityID": 3654881, "cityName": "Oneonta" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594044598599680, "text": "Temp 42.2°F Wind Chill 42.2°F RH 70% Wind 0.0 --- Gust 0.0 --- SLP 30.374 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 106, "friends_count": 63, "statues_count": 34764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045009530880, "text": "@ebbtideapp Tide in Summit Bridge, Delaware 04/17/2016\nHigh 8:10am 3.7\n Low 2:29pm 0.1\nHigh 8:40pm 3.8\n Low 2:47am 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.7333,39.5333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 72, "friends_count": 1, "statues_count": 25117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045080977409, "text": "04/17@03:00 - Temp 41.1F, WC 41.1F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.464in, Falling slowly. Rain 0.00in. Hum 80%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045173137408, "text": "After the $uicideboy$ concert, I now I have 666 photos https://t.co/Goz6Ev6uSM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 216275510, "name": "$lumpBoy", "screen_name": "thelouie_", "lang": "en", "location": "South Central, Los Angeles", "create_at": date("2010-11-15"), "description": "Get to know me, you will either end up loving or hating me.", "followers_count": 1521, "friends_count": 1359, "statues_count": 46991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045206700032, "text": "I'm gonna pierce my nose this week lgi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1709346294, "name": "Xander", "screen_name": "_ComplexAlex", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-08-29"), "description": "null", "followers_count": 435, "friends_count": 249, "statues_count": 44465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045357760512, "text": "Wind 0.0 mph E. Barometer 30.25 in, Steady. Temperature 63.3 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045466738688, "text": "@imreallyCARLITA I have my tongue pierced 4 a reason mama", "in_reply_to_status": 721593954764857344, "in_reply_to_user": 621623824, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 621623824 }}, "user": { "id": 730423830, "name": "adri.", "screen_name": "adriana_mariiie", "lang": "en", "location": "hiymrah, tx ", "create_at": date("2012-08-01"), "description": "it's a bad day, not a bad life. @jordynyoung_ is my other half. #RIPMichael #RIPBigV ❤️", "followers_count": 1720, "friends_count": 1517, "statues_count": 49901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045471006722, "text": "Wind 1.0 mph ESE. Barometer 30.224 in, Steady. Temperature 64.2 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594045491859456, "text": "my baby��❣❣ https://t.co/Yl6jhT0A3n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3271672159, "name": "lil niña", "screen_name": "gretchenelizaa", "lang": "en", "location": "null", "create_at": date("2015-07-07"), "description": "null", "followers_count": 511, "friends_count": 515, "statues_count": 8393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594046183911424, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2193580874, "name": "RENE", "screen_name": "rene_marquez37", "lang": "en", "location": "null", "create_at": date("2013-11-13"), "description": "Fontana,ca", "followers_count": 524, "friends_count": 400, "statues_count": 10656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594046460870656, "text": "Now Watching: Comfirmation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146892710, "name": "versace hotty", "screen_name": "JBTaughtYa", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2015-04-09"), "description": "work hard. grind till you OWN IT.", "followers_count": 343, "friends_count": 497, "statues_count": 50788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594046502825984, "text": "don't know what to say lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2763544793, "name": "Sarah Barger", "screen_name": "bargerr_sarahh", "lang": "en", "location": "Stafford, VA", "create_at": date("2014-09-06"), "description": "♉️", "followers_count": 486, "friends_count": 521, "statues_count": 7857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boswell's Corner, VA", "id": "01c683231f7f814e", "name": "Boswell's Corner", "place_type": "city", "bounding_box": rectangle("-77.388712,38.484693 -77.361413,38.519935") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51179, "countyName": "Stafford", "cityID": 5108708, "cityName": "Boswell's Corner" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594046704013313, "text": "I gave my nephew hella candy and donuts and played/ tickled him so much that he threw up ������ oops!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1606408170, "name": "tats", "screen_name": "_tatianamarquez", "lang": "en", "location": "not interested", "create_at": date("2013-07-19"), "description": "as the wallet is emptied, the heart is filled #sleepislife http://instagram.com/_tatianamarquez", "followers_count": 580, "friends_count": 524, "statues_count": 14987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-17T00:00:01.000Z"), "id": 721594046716575744, "text": "@Downey_Fresh_ hooah❤️��", "in_reply_to_status": 721579616373121024, "in_reply_to_user": 1171064516, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 1171064516 }}, "user": { "id": 344804612, "name": "Trent", "screen_name": "TeeeAllen", "lang": "en", "location": "Army National Guard", "create_at": date("2011-07-29"), "description": "Queen of battle follow me", "followers_count": 1037, "friends_count": 604, "statues_count": 36193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denham Springs, LA", "id": "56f397b1076d7ce2", "name": "Denham Springs", "place_type": "city", "bounding_box": rectangle("-90.991153,30.356008 -90.838776,30.571121") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22063, "countyName": "Livingston", "cityID": 2220435, "cityName": "Denham Springs" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047341731840, "text": "Temp: 62.9°F Wind:0.5mph Pressure: 30.170hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047345901568, "text": "Mangos Cafe ���������������� Outdoor PATIO ����������������No matter what the weather conditions are, you can… https://t.co/qvKssNhLqL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0063171,39.9881401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383010151, "name": "Mango's Cafe", "screen_name": "MangosCafecbus", "lang": "en", "location": "21 Smith Place. Columbus, OHIO", "create_at": date("2011-09-30"), "description": "google us & check out what others have to say 6144293694 http://www.mangoscafe.orgus⬇️ https://www.facebook.com/pages/Mangos-Cafe/108546569254369", "followers_count": 10575, "friends_count": 502, "statues_count": 28772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047480012801, "text": "CAN YOU BELIEVE I'M SEEING MY FAVORITE BAND IN THE WHOLE WORLD, TWENTY ONE PILOTS, AGAIN IN ONE HUNDRED DAYS!!! WHAT THE HECK!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3037005079, "name": "bee movie fanatic", "screen_name": "taylorcIark", "lang": "en", "location": "phx", "create_at": date("2015-02-22"), "description": "twenty one pilots enthusiast | 32 | 101", "followers_count": 800, "friends_count": 100, "statues_count": 110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047643553794, "text": "@Sophie_Dopie hahaha its like having a large", "in_reply_to_status": 721589652860219393, "in_reply_to_user": 782943086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 782943086 }}, "user": { "id": 445901897, "name": "Callye", "screen_name": "callyeanna", "lang": "en", "location": "null", "create_at": date("2011-12-24"), "description": "blessed // snapchat: Callye", "followers_count": 312, "friends_count": 379, "statues_count": 4477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Granite Bay, CA", "id": "01b4481d283c3d09", "name": "Granite Bay", "place_type": "city", "bounding_box": rectangle("-121.226663,38.712987 -121.105833,38.84012") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 630693, "cityName": "Granite Bay" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047891173376, "text": "It's real life feels! https://t.co/K32SRemYur", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416783369, "name": "Alyssa Gatlin Hall", "screen_name": "AlyssaLeeGatlin", "lang": "en", "location": "Hazel Green, Alabama ", "create_at": date("2011-11-19"), "description": "Alyssa Lee. 22. Married to Marc Alan Hall❤️ Die hard Swiftie. Thanks a ton if you follow me✌", "followers_count": 95, "friends_count": 260, "statues_count": 948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594047941349376, "text": "@DinoDotta lol niggaaa you feeel me? I was over here 1/2 lit wit the squinty \"not sure if\" eyes, & hell yea I was about ready!! Lol", "in_reply_to_status": 721593623851077633, "in_reply_to_user": 27106051, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27106051 }}, "user": { "id": 2646780836, "name": "C.L. Smooth", "screen_name": "chelslaurice", "lang": "en", "location": "null", "create_at": date("2014-07-14"), "description": "God. all Love. #Finessekitten.", "followers_count": 497, "friends_count": 499, "statues_count": 6851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594048281251842, "text": "Wind 0.0 mph ---. Barometer 30.384 in, Rising slowly. Temperature 50.8 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 41, "friends_count": 4, "statues_count": 26758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594048474148869, "text": "Wind 0.0 mph ---. Barometer 30.299 in, Rising slowly. Temperature 57.9 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 58, "friends_count": 27, "statues_count": 19144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594048478384128, "text": "Ripley SW Limestone Co. Temp: 55.9°F Wind:0.7mph Pressure: 1005.2mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594048578871297, "text": "Satisfaction at zero percent", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2385576841, "name": "Zero", "screen_name": "SenorRamirez", "lang": "en", "location": "Lost", "create_at": date("2014-03-12"), "description": "doing something you wouldnt dare to do", "followers_count": 213, "friends_count": 205, "statues_count": 10131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burien, WA", "id": "531b8f14eaf6c48e", "name": "Burien", "place_type": "city", "bounding_box": rectangle("-122.382962,47.430814 -122.311608,47.499304") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5308850, "cityName": "Burien" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594048914427905, "text": "Love: the most irrational force under the sun.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23840526, "name": "James A. Wooten III", "screen_name": "James_Wooten", "lang": "en", "location": "null", "create_at": date("2009-03-11"), "description": "Man of formidable and distinguished talents. Lover of good story, flawlessly executed ideas, fascinating people and wine.", "followers_count": 980, "friends_count": 99, "statues_count": 32650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "NoHo, Manhattan", "id": "62cbc76c48110f14", "name": "NoHo", "place_type": "neighborhood", "bounding_box": rectangle("-73.992641,40.721534 -73.982952,40.726425") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594049023647744, "text": "Wind 0.7 mph WNW. Barometer 30.45 in, Steady. Temperature 73.6 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 159120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594049166241793, "text": "03:00:01 |Temp: 45.7ºF |Dew Point 41.8ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594049417711617, "text": "67.2F (Feels: 67.2F) - Humidity: 89% - Wind: 6.9mph E - Gust: 10.7mph - Pressure: 1011.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594050214649857, "text": "My boyfriend just broke my heart he didn't remember our anniversary may 18 but remembers his best friends birthday may 17 smh...... ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1416856063, "name": "Monique▼", "screen_name": "Moniixquee", "lang": "en", "location": "null", "create_at": date("2013-05-09"), "description": "Jonathan && Disneyland ♡", "followers_count": 102, "friends_count": 36, "statues_count": 36 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594050315362305, "text": "About to get food but I feel like the person that's working is gonna tell me that their system is down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3531325154, "name": "8", "screen_name": "Rubio95s", "lang": "en", "location": "Houston, TX", "create_at": date("2015-09-11"), "description": "new Twitter", "followers_count": 327, "friends_count": 230, "statues_count": 3309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594050780901377, "text": "Can't wait until they come to Philly https://t.co/9Dq4maQew9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1713197749, "name": "Harry Cottrill", "screen_name": "harrycottrill1", "lang": "en", "location": "L-Dub", "create_at": date("2013-08-30"), "description": "Biology", "followers_count": 485, "friends_count": 380, "statues_count": 3529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Towson, MD", "id": "ac31053cb03c56b2", "name": "Towson", "place_type": "city", "bounding_box": rectangle("-76.667184,39.371735 -76.563177,39.420969") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2478425, "cityName": "Towson" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594050860584960, "text": "Donald Trump won the state primary votes but didn't get delegates? Foolish the voters?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 713404274064031744, "name": "Christina", "screen_name": "ChristinaCao110", "lang": "en", "location": "USA", "create_at": date("2016-03-25"), "description": "Chinese-American. Independent Woman. Wife. Mother. Worker. Artist. Loving everything I have!", "followers_count": 122, "friends_count": 147, "statues_count": 302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-17T00:00:02.000Z"), "id": 721594051041083392, "text": "Fr https://t.co/rqVCVCpE89", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 308181555, "name": "OG", "screen_name": "_CoreyRosado", "lang": "en", "location": "null", "create_at": date("2011-05-30"), "description": "snapchat : Corey_Rosado IG : _CoreyRosado G.D❤️", "followers_count": 1705, "friends_count": 1608, "statues_count": 47874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594051737350144, "text": "#Trump in 2016 Temp:47.7°F Wind:0.0mph Pressure: 30.30hpa Steady Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 315420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594051913465856, "text": "@MahkaylaMonster I'm not even sure", "in_reply_to_status": 721593975224856576, "in_reply_to_user": 746510305, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 746510305 }}, "user": { "id": 3873607847, "name": "Jay Papí", "screen_name": "jeremyncolon", "lang": "en", "location": "Union Beach, NJ", "create_at": date("2015-10-05"), "description": "Work harder than your hardest. - Flex Luther", "followers_count": 480, "friends_count": 472, "statues_count": 5326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keyport, NJ", "id": "36763957ba75d166", "name": "Keyport", "place_type": "city", "bounding_box": rectangle("-74.217207,40.422607 -74.183613,40.447219") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3436810, "cityName": "Keyport" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594052249001984, "text": "��������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 942241830, "name": "Emily Wilson", "screen_name": "thisisemilyokay", "lang": "en", "location": "beaufort, south carolina", "create_at": date("2012-11-11"), "description": "stop asking me what's going on. I don't know either.", "followers_count": 218, "friends_count": 164, "statues_count": 2363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel Bay, SC", "id": "cfae1c9a88960db7", "name": "Laurel Bay", "place_type": "city", "bounding_box": rectangle("-80.809517,32.441983 -80.752113,32.482502") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45013, "countyName": "Beaufort", "cityID": 4540525, "cityName": "Laurel Bay" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594052307759104, "text": "Yeah I know that karmas too real so I hope you doin cool. But I still ain't fuckin with youuuuuu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 584906156, "name": "Lauren Eline", "screen_name": "LaLaa_Lauren17", "lang": "en", "location": "null", "create_at": date("2012-05-19"), "description": "Even the moon has a dark side. ⭐️ Never let anything get in the way of your dreams ⭐️ Darling I'm a nightmare dressed like a daydream ⚡️", "followers_count": 377, "friends_count": 555, "statues_count": 15974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Front Royal, VA", "id": "172f8c423c3bad36", "name": "Front Royal", "place_type": "city", "bounding_box": rectangle("-78.220205,38.896761 -78.142948,38.955969") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51187, "countyName": "Warren", "cityID": 5129968, "cityName": "Front Royal" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594052341211137, "text": "It's 3 years since I saw Justin for the first time at Believe Tour in Norway����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1612429393, "name": "Karina", "screen_name": "4eKidrauhl", "lang": "en", "location": "Norwegian in LA", "create_at": date("2013-07-22"), "description": "Life Is Worth Living", "followers_count": 535, "friends_count": 123, "statues_count": 4284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594052668358656, "text": "Round 2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271460723, "name": "Jacob Starcer", "screen_name": "JStarcer_15", "lang": "en", "location": "Santa Clarita, CA", "create_at": date("2014-01-01"), "description": "chs football", "followers_count": 269, "friends_count": 222, "statues_count": 1062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594052832071685, "text": "I don't even know why I still have a timehop. Its just a constant reminder of how ugly I've become.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27754112, "name": "Kristi", "screen_name": "kristiannamaria", "lang": "en", "location": "null", "create_at": date("2009-03-30"), "description": "Yes.", "followers_count": 253, "friends_count": 266, "statues_count": 3507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053029076992, "text": "@BreannaLabbe wyd", "in_reply_to_status": 721593992329043969, "in_reply_to_user": 2870091235, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2870091235 }}, "user": { "id": 3178532664, "name": "alicia", "screen_name": "whorjas", "lang": "en", "location": "337", "create_at": date("2015-04-28"), "description": "I use :-) a lot", "followers_count": 204, "friends_count": 528, "statues_count": 1883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prien, LA", "id": "00a028c043bb68f0", "name": "Prien", "place_type": "city", "bounding_box": rectangle("-93.283003,30.106667 -93.213369,30.168682") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2262647, "cityName": "Prien" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053117083648, "text": "You're not my problem to fix anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977550540, "name": "ninja C", "screen_name": "chandougie", "lang": "en", "location": "downtown santa ana", "create_at": date("2012-11-28"), "description": "dark angel • balance of opposites • space dude • sun model • aspiring artists", "followers_count": 628, "friends_count": 525, "statues_count": 29436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053284876288, "text": "casi empieza la pelea de @oscarvaldez56 x @HBOLatino! Pero primero...mira esta entrevista: https://t.co/fWJmM91h2n\nDale un \"like\" amigos! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 277194514, 182059443 }}, "user": { "id": 240842438, "name": "Jessica Rosales", "screen_name": "JessRosales1", "lang": "en", "location": "Los Angeles", "create_at": date("2011-01-20"), "description": "null", "followers_count": 4743, "friends_count": 766, "statues_count": 8381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053331169281, "text": "Casually makes spaghetti at 3AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 387277746, "name": "Badeline ♐️", "screen_name": "MadelineMH_", "lang": "en", "location": "In my room", "create_at": date("2011-10-08"), "description": "my friends like to call me Baddie and I like to call them daddy", "followers_count": 359, "friends_count": 541, "statues_count": 2028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kettering, OH", "id": "a121bc3ed630a35e", "name": "Kettering", "place_type": "city", "bounding_box": rectangle("-84.218859,39.654418 -84.088199,39.736536") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3940040, "cityName": "Kettering" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053599588352, "text": "Lit lit lit lit lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1216166696, "name": "bri loude", "screen_name": "brineebooboo", "lang": "en", "location": "null", "create_at": date("2013-02-24"), "description": "@andrewgilby13 ♡", "followers_count": 372, "friends_count": 300, "statues_count": 3048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ham Lake, MN", "id": "2dece6b71d2fab70", "name": "Ham Lake", "place_type": "city", "bounding_box": rectangle("-93.27277,45.211237 -93.175842,45.312567") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2726738, "cityName": "Ham Lake" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594053616410624, "text": "https://t.co/scqR6j7gUf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2720437863, "name": "hhwtil", "screen_name": "hhwtil", "lang": "en", "location": "null", "create_at": date("2014-07-21"), "description": "null", "followers_count": 54, "friends_count": 0, "statues_count": 10072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ben Avon, PA", "id": "f123b0605844d37d", "name": "Ben Avon", "place_type": "city", "bounding_box": rectangle("-80.093354,40.500323 -80.06987,40.522121") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4205504, "cityName": "Ben Avon" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054002155521, "text": "Wind 14.4 mph E. Barometer 29.910 in, Steady. Temperature 68.8 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054106951682, "text": "oomf fine asf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2652572154, "name": "Yung Dill", "screen_name": "iceyboydill", "lang": "en", "location": "Arlington, TX", "create_at": date("2014-07-16"), "description": "null", "followers_count": 413, "friends_count": 345, "statues_count": 7874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054170013696, "text": "حقيقة مرة:\n\nما نفع كل نية طيبة بالتبادل؟ في النهاية تبقى الأمور على حالها: الواحد لا يطيق رائحة الآخر.\n . \n\n Friedrich Nietzsche", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 130922404, "name": "Atheer", "screen_name": "AtheerShawai", "lang": "en", "location": "Washington DC", "create_at": date("2010-04-08"), "description": "Journalist, Writer and Student. ............. صحفي في راديو سوا Love your life. .اعشق حياتك", "followers_count": 550, "friends_count": 248, "statues_count": 3089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, VA", "id": "d6819fe60643ebc1", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-77.144435,38.789907 -77.037304,38.844853") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5101000, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054350348288, "text": "https://t.co/IwZLCSTfZ3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 70059594, "name": "Johns", "screen_name": "eboicey", "lang": "en", "location": "Philadelphia // West Chester", "create_at": date("2009-08-29"), "description": "Spanking the money.", "followers_count": 511, "friends_count": 213, "statues_count": 11844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054425853952, "text": "And I was off, but i DNT got shit else to do tomorrow might as well go get some extra hours right? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34857419, "name": "CODE RED", "screen_name": "BdotMARTIN", "lang": "en", "location": "ATL", "create_at": date("2009-04-23"), "description": "#NewLife", "followers_count": 975, "friends_count": 974, "statues_count": 60251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054803398656, "text": "Short clip of the #Denials @overtimetheater #satx #improv @ The Denials https://t.co/UJk31Ne727", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-98.4834366,29.4398899"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Denials", "satx", "improv" }}, "user_mentions": {{ 16530521 }}, "user": { "id": 43729001, "name": "Jenny Taylor", "screen_name": "JennyTaylor1123", "lang": "en", "location": "San Antonio, Texas", "create_at": date("2009-05-31"), "description": "Cuckoo\nby Jack Prelutsky\nThe cuckoo in our cuckoo clock\nwas wedded to an octopus\nshe laid a singal wooden egg\nand hatched a cuckooclotopus!", "followers_count": 88, "friends_count": 176, "statues_count": 1450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594054958407680, "text": "I hate not being able to sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595678041, "name": "allison", "screen_name": "allison_lumley", "lang": "en", "location": "null", "create_at": date("2012-05-31"), "description": "Jesus & food lover. ΑΞΔ #HSU19❤️", "followers_count": 1141, "friends_count": 1326, "statues_count": 6710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkadelphia, AR", "id": "bdefad825ffb7daa", "name": "Arkadelphia", "place_type": "city", "bounding_box": rectangle("-93.093798,34.091696 -93.044691,34.161433") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5019, "countyName": "Clark", "cityID": 501870, "cityName": "Arkadelphia" } }
+{ "create_at": datetime("2016-04-17T00:00:03.000Z"), "id": 721594055251992578, "text": "What the fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2839497694, "name": "Micah Ono", "screen_name": "Micka808", "lang": "en", "location": "null", "create_at": date("2014-10-21"), "description": "Saved by the barbell", "followers_count": 218, "friends_count": 222, "statues_count": 1282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594055570898944, "text": "@ParisSlaysUrFav she'll explain it eventually she knows we're E notionally invested. i rly didn't fe lik he was THE ONE tho so idk", "in_reply_to_status": 721593624287293440, "in_reply_to_user": 1564577515, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1564577515 }}, "user": { "id": 4915537953, "name": "hollywood hot mess", "screen_name": "popculturecunt", "lang": "en", "location": "90210", "create_at": date("2016-02-15"), "description": "do they sell marc jacobs or chanel in this grocery store", "followers_count": 279, "friends_count": 147, "statues_count": 409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saugerties South, NY", "id": "8c0ebdf91696edf8", "name": "Saugerties South", "place_type": "city", "bounding_box": rectangle("-73.965643,42.053048 -73.938792,42.072675") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36111, "countyName": "Ulster", "cityID": 3665310, "cityName": "Saugerties South" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594055638065152, "text": "Wind 0.7 mph W. Barometer 30.090 in, Steady. Temperature 62.0 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594055780503552, "text": "Welp ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477229784, "name": "iAmKarma .", "screen_name": "BabieeCuuhz", "lang": "en", "location": "In The Clouds ..", "create_at": date("2012-01-28"), "description": "What Goes Around Comes Around && Naa iAint Talkin Bout Propellers . Im Too Dope , Got Him Addicted ✌", "followers_count": 9416, "friends_count": 8767, "statues_count": 52355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parkway, CA", "id": "00c4e030703d58e2", "name": "Parkway", "place_type": "city", "bounding_box": rectangle("-121.477338,38.481561 -121.429218,38.53946") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 655800, "cityName": "Parkway" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594056166408192, "text": "I was really craving all that Mexican food at fiesta. Like floutas, tacos y elotes �� makes me miss the ville", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3561682272, "name": "Ivana Rey.", "screen_name": "_holaaivanitaaa", "lang": "en", "location": "San Marcos, TX", "create_at": date("2015-09-14"), "description": "20✨", "followers_count": 591, "friends_count": 493, "statues_count": 8905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594056267014145, "text": "Wind 0.0 mph ---. Barometer 30.439 in, Steady. Temperature 31.8 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594056673853440, "text": "#CincodeMega en #Escapade2001houston\nDomingo 1 de Mayo \n@PLANBLIVE @TEGOCALDERON @zionylennoxpr @jquiles \n2813721100 https://t.co/omdGFNnnzs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "CincodeMega", "Escapade2001houston" }}, "user_mentions": {{ 61452308, 25594518, 40307393, 101303833 }}, "user": { "id": 65768493, "name": "Escapade 2001", "screen_name": "Escapade2001", "lang": "en", "location": "Houston, TX", "create_at": date("2009-08-14"), "description": "Locación:\r\nEscapade 2001\r\n11903 Eastex Fwy. (59 Norte salida Aldine Mail Rt.)\r\nHouston, TX 77039\r\n\r\nInformes y Reservaciones:\r\n281-372-1100", "followers_count": 6252, "friends_count": 1721, "statues_count": 3216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594056925511680, "text": "If only you knew.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1370141988, "name": "Lyssa♡♍", "screen_name": "lyssamae_xoxo", "lang": "en", "location": "Az", "create_at": date("2013-04-21"), "description": "c/o 17 \nbe a gangsta. don't have feelings.\n•snap & insta: lovelylyssa13.x", "followers_count": 349, "friends_count": 382, "statues_count": 3676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594057382850560, "text": "@EZG_Frozone @Observe_Talentz @Solidestwand you guys are all gods.", "in_reply_to_status": 721593902688436224, "in_reply_to_user": 4636048381, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4636048381, 3002764340, 3082809257 }}, "user": { "id": 1454138750, "name": "Raine™", "screen_name": "Unfamilliarr", "lang": "en", "location": "Clarksville, TN", "create_at": date("2013-05-24"), "description": "17. Always on the grind • GOON $QUAD™ • Owner of @Taking_Flyte. I only spread Good vibes, and Positivity. #TheUnfamilia", "followers_count": 2061, "friends_count": 1092, "statues_count": 10030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Campbell North, KY", "id": "006cfe05510849b3", "name": "Fort Campbell North", "place_type": "city", "bounding_box": rectangle("-87.514325,36.64013 -87.436613,36.689327") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21047, "countyName": "Christian", "cityID": 2128486, "cityName": "Fort Campbell North" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594057399513088, "text": "Happy birthday to the best sister anyone could ask for ���� I love you Skyita ������ @natalieeskyy https://t.co/c23RSqOhT0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1566626960 }}, "user": { "id": 2737545270, "name": "sam", "screen_name": "Samantheerrss", "lang": "en", "location": "null", "create_at": date("2014-08-16"), "description": "ni santa , ni puta , solo mujer y chingona", "followers_count": 347, "friends_count": 331, "statues_count": 4125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594057433096196, "text": "@TexanTan why am I just seeing this��", "in_reply_to_status": 721008292936306688, "in_reply_to_user": 575712500, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 575712500 }}, "user": { "id": 3267736051, "name": "Cynthia Renea.", "screen_name": "cynthia_reneaa", "lang": "en", "location": "Stephenville, TX", "create_at": date("2015-07-03"), "description": "null", "followers_count": 140, "friends_count": 395, "statues_count": 1921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594057491750912, "text": "My bf called me on FaceTime and showed me through his closet with like over 10 shirts he's giving me to sleep in ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2686606057, "name": "hannah", "screen_name": "hannahmorgan024", "lang": "en", "location": "null", "create_at": date("2014-07-27"), "description": "null", "followers_count": 200, "friends_count": 268, "statues_count": 3679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594058364317696, "text": "@jewelisaxo stop", "in_reply_to_status": 721594011073572864, "in_reply_to_user": 3999090586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3999090586 }}, "user": { "id": 4374110356, "name": "Casey Cat.", "screen_name": "princessHarls", "lang": "en", "location": "null", "create_at": date("2015-12-04"), "description": "mother | professional sleepyhead", "followers_count": 94, "friends_count": 97, "statues_count": 731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594058380939264, "text": "one time I low key ate an entire fern plant and I'm not even ashamed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2382788646, "name": "ryan", "screen_name": "ryaaaann___", "lang": "en", "location": "in eve ramirez's pants ", "create_at": date("2014-03-10"), "description": "yea something like that", "followers_count": 1019, "friends_count": 843, "statues_count": 8913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valinda, CA", "id": "b1bbf8e2ef22573a", "name": "Valinda", "place_type": "city", "bounding_box": rectangle("-117.94891,34.024734 -117.912772,34.053145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 681638, "cityName": "Valinda" } }
+{ "create_at": datetime("2016-04-17T00:00:04.000Z"), "id": 721594059165315072, "text": "GRIMES AND ROSES!!!!!! #GNR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GNR" }}, "user": { "id": 55681363, "name": "Ryan Fox", "screen_name": "Ry_Fox", "lang": "en", "location": "EC Wisconsin", "create_at": date("2009-07-10"), "description": "welcome to the end of the thought process", "followers_count": 94, "friends_count": 99, "statues_count": 1954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eau Claire, WI", "id": "6f58cacd3741baa6", "name": "Eau Claire", "place_type": "city", "bounding_box": rectangle("-91.601375,44.759861 -91.418063,44.879183") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55035, "countyName": "Eau Claire", "cityID": 5522300, "cityName": "Eau Claire" } }
+{ "create_at": datetime("2016-04-17T00:00:05.000Z"), "id": 721594060381794304, "text": "@defectivepearl yes!!!", "in_reply_to_status": 721562692335570944, "in_reply_to_user": 3303281717, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3303281717 }}, "user": { "id": 1463546047, "name": "big T", "screen_name": "bobby_nap", "lang": "en", "location": "null", "create_at": date("2013-05-27"), "description": "I'm Tristan | 20 | he/him | I like cute shit and pretty colors | dogs are everything | be a sexy tiger |", "followers_count": 334, "friends_count": 169, "statues_count": 50850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Melbourne, FL", "id": "d970e7fd69eaf4ec", "name": "Melbourne", "place_type": "city", "bounding_box": rectangle("-80.706074,28.03509 -80.593324,28.200658") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12009, "countyName": "Brevard", "cityID": 1243975, "cityName": "Melbourne" } }
+{ "create_at": datetime("2016-04-17T00:00:05.000Z"), "id": 721594060486549504, "text": "When guys OVER sexualize things out of nowhere https://t.co/o3Lsw1qsfJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458147769, "name": "bugatti hottie", "screen_name": "chellae_rae", "lang": "en", "location": "Inglewood, CA", "create_at": date("2012-01-07"), "description": "I'm Marsha you're Jan ✨", "followers_count": 861, "friends_count": 655, "statues_count": 34804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-17T00:00:05.000Z"), "id": 721594060524400640, "text": "If they didn't say I fouled on my last jump��... But OU this Saturday... Happy to be back in action��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2371054526, "name": "⚡️IJump/Sprint⚡️", "screen_name": "poloholic_214", "lang": "en", "location": "Denton, TX", "create_at": date("2014-03-03"), "description": "|✨UNT Track & Field✨|D1 Student Athlete✨|Blessed✨|USATF|⚡️Ijump/Sprint⚡️|#UNT19|#HornAlumni|", "followers_count": 1203, "friends_count": 984, "statues_count": 2489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-17T00:00:05.000Z"), "id": 721594060687949824, "text": "HELLO GOOD AFTERNOON https://t.co/aPkDmm2Roz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338703835, "name": "mac leon", "screen_name": "kingxnoel", "lang": "en", "location": "Hangin' in a chow line ", "create_at": date("2011-07-19"), "description": "20. Washed.", "followers_count": 926, "friends_count": 496, "statues_count": 61940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956427716124673, "text": "Wind 3.8 mph NNE. Barometer 30.28 in, Rising slowly. Temperature 41.5 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 41566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956427749728256, "text": "Thomas fucking us all up https://t.co/8wL3CzlRvk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 568764709, "name": "Sr. Class Pres Tank", "screen_name": "alextankbarajas", "lang": "en", "location": "San Jacinto, CA", "create_at": date("2012-05-01"), "description": "Tank. C/O 2016........... Hey School administration stop lurking on my Profile. Rice&Pussy", "followers_count": 671, "friends_count": 665, "statues_count": 8472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jacinto, CA", "id": "0068bc1a061e33c3", "name": "San Jacinto", "place_type": "city", "bounding_box": rectangle("-117.039483,33.751357 -116.919422,33.818476") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 667112, "cityName": "San Jacinto" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956428135710720, "text": "Has a angelic face but is a troll https://t.co/iDT32ay87J", "in_reply_to_status": 721952392477089792, "in_reply_to_user": 719098675, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 719098675, "name": "Malec's Canary Sheet", "screen_name": "Malec_Updates", "lang": "en", "location": "Malec's loft", "create_at": date("2012-07-26"), "description": "#MalecTrashSquad ... In love with a troll who plays a awkward shadowhunter and a precious flower who plays a sassy warlock. Also runs @HarryShumUpdate", "followers_count": 1705, "friends_count": 348, "statues_count": 23464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956428194254848, "text": "@RoyoGrandee @_Roni_Boni https://t.co/hCRcxEw6HD", "in_reply_to_status": 721955282289434624, "in_reply_to_user": 327737875, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 327737875, 729939277 }}, "user": { "id": 1970557729, "name": "A.", "screen_name": "_alanisgrijalva", "lang": "en", "location": "null", "create_at": date("2013-10-18"), "description": "You know, feelin' good livin' better", "followers_count": 326, "friends_count": 282, "statues_count": 17579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956429003927553, "text": "I wish I could get tattoos.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 469554937, "name": "♠️", "screen_name": "TheRealUsama", "lang": "en", "location": "Had to change location", "create_at": date("2012-01-20"), "description": "Flacco is elite", "followers_count": 341, "friends_count": 301, "statues_count": 18987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Burnie, MD", "id": "cad757363c1a85df", "name": "Glen Burnie", "place_type": "city", "bounding_box": rectangle("-76.644717,39.131259 -76.563196,39.207912") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2432650, "cityName": "Glen Burnie" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956429066690560, "text": "#loveyourselffirst https://t.co/gn6DPukgw2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "loveyourselffirst" }}, "user": { "id": 73778146, "name": "SASHASAYS", "screen_name": "SASHASAYS01", "lang": "en", "location": "new orleans", "create_at": date("2009-09-12"), "description": "No Weapon Formed Against Me Shall Prosper! I AM HERE TO LOVE LAUGH AND LIVE OUT LOUD!!", "followers_count": 857, "friends_count": 1400, "statues_count": 5402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956429326725120, "text": "my chin hairs bussin but my mustache still only visible when wet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32939076, "name": "Eric", "screen_name": "SLIKKRIC", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-04-18"), "description": "Chicago ✈️ New Orleans #XULA", "followers_count": 1195, "friends_count": 571, "statues_count": 148368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956429339484160, "text": "shush nigga damn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2458913772, "name": "ROCAIZE", "screen_name": "Rocaize", "lang": "en", "location": "Dreamville", "create_at": date("2014-04-22"), "description": "ㅤㅤㅤㅤㅤㅤㅤNACKㅤㅤㅤㅤㅤㅤㅤㅤㅤFuture Big Dick Pornstar", "followers_count": 1006, "friends_count": 359, "statues_count": 10933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ann Arbor, MI", "id": "92b2158a6d509767", "name": "Ann Arbor", "place_type": "city", "bounding_box": rectangle("-83.800262,42.215087 -83.659681,42.326592") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2603000, "cityName": "Ann Arbor" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956429616254976, "text": "Temp 40.8° Hi/Lo 46.3/40.8 Rng 5.5° WC 40.8° Hmd 71% Rain 0.00\" Storm 0.00\" BAR 30.226 Falling DP 32.1° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 118, "statues_count": 17977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956430169833472, "text": "What to watch on Netflix?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614769565, "name": "|DJ Tobes Boomin|✈️", "screen_name": "1chaiin", "lang": "en", "location": "sek", "create_at": date("2012-06-21"), "description": "✊ #ClassOf2k15 SNAPCHAT ➡️➡️ kobie_ray11 ⚡️|I Am The HYPE-GOD|Stay Boomin|☁️", "followers_count": 709, "friends_count": 1934, "statues_count": 4546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, KS", "id": "2cb7103b8bddaea6", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-95.751487,37.192852 -95.677879,37.263548") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20125, "countyName": "Montgomery", "cityID": 2033875, "cityName": "Independence" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956430274650112, "text": "@brittani_muiter @JaylonNelson that was my point", "in_reply_to_status": 721956313790451712, "in_reply_to_user": 2266290360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2266290360, 2886519300 }}, "user": { "id": 634354096, "name": "AB", "screen_name": "ABrooks0604", "lang": "en", "location": "USA", "create_at": date("2012-07-12"), "description": "Dropped outta school now we dumb rich |snapchat: a_brooks0604", "followers_count": 375, "friends_count": 426, "statues_count": 6180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-04-18T00:00:00.000Z"), "id": 721956430329196545, "text": "Just woke up outta nowhere", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293357597, "name": "BrokeWristBaba.", "screen_name": "EgusiFiend", "lang": "en", "location": "from tha 281 to tha 817", "create_at": date("2011-05-04"), "description": "Samuel OluwaMayokun Awolope. Nigerian (Yoruba.) Jesus Christ will forever be my Lord and Savior. Snapchat: unclesamisback. #1120AlwaysAndForever #TCU19", "followers_count": 2187, "friends_count": 1761, "statues_count": 82440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956430778081280, "text": "I see ya ����☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 485176781, "name": "Dre-Lord ✨", "screen_name": "KinqqDre_", "lang": "en", "location": "null", "create_at": date("2012-02-06"), "description": "Good, better, best. ✨ Never let it rest. 'Til your good is better and your better is best. ✊\r\n-King Dre", "followers_count": 704, "friends_count": 408, "statues_count": 18599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kannapolis, NC", "id": "3562f19d95536f2f", "name": "Kannapolis", "place_type": "city", "bounding_box": rectangle("-80.678004,35.435255 -80.561482,35.54115") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37025, "countyName": "Cabarrus", "cityID": 3735200, "cityName": "Kannapolis" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956430807339008, "text": "Bruh..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473451199, "name": "Premium.G", "screen_name": "BrysonG_", "lang": "en", "location": "null", "create_at": date("2012-01-24"), "description": "null", "followers_count": 1462, "friends_count": 51, "statues_count": 8599 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956431205781506, "text": "@reyes_morado #nofilter", "in_reply_to_status": 721956358677880833, "in_reply_to_user": 842790079, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "nofilter" }}, "user_mentions": {{ 732034112 }}, "user": { "id": 842790079, "name": "ry", "screen_name": "ryleegrace_15", "lang": "en", "location": "Hanford, CA", "create_at": date("2012-09-23"), "description": "csusm || B&A", "followers_count": 1730, "friends_count": 845, "statues_count": 37178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956431298174976, "text": "Wind 1.3 mph NE. Barometer 30.28 in, Steady. Temperature 61.7 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956431352700928, "text": "04/18@03:00 - Temp 44.3F, WC 44.3F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.271in, Falling slowly. Rain 0.00in. Hum 80%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956431507750912, "text": "HAPPY 16TH BIRTHDAY @sammyt2000 hope u have a great day!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2688886933 }}, "user": { "id": 1950252110, "name": "Mariah Villanueva", "screen_name": "Mariahmariiee", "lang": "en", "location": "null", "create_at": date("2013-10-09"), "description": "idgaf", "followers_count": 339, "friends_count": 267, "statues_count": 714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plumas Lake, CA", "id": "00aaaf1766b35652", "name": "Plumas Lake", "place_type": "city", "bounding_box": rectangle("-121.561363,38.978053 -121.544054,39.02631") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6115, "countyName": "Yuba", "cityID": 657829, "cityName": "Plumas Lake" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956431726014467, "text": "@kelvinfranklin_ #ChrisP #KevinGates \"IDK\" #NewMusic https://t.co/F0T5LRAJ7K", "in_reply_to_status": -1, "in_reply_to_user": 572398444, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "ChrisP", "KevinGates", "NewMusic" }}, "user_mentions": {{ 572398444 }}, "user": { "id": 111640056, "name": "Mr. CEO", "screen_name": "Waddup_Hill", "lang": "en", "location": "Tx to ✈ ATL", "create_at": date("2010-02-05"), "description": "#Utsa Live life With Purpose & Understanding. Born 2 Entertain, Mc, Promoter, CEO of #AllHillEnt, YouTube us All Hill Ent TheNewP1@yahoo.com", "followers_count": 2803, "friends_count": 1263, "statues_count": 47231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956432111751170, "text": "grey's anatomy ... just caught up https://t.co/QAVmrwactt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354344210, "name": "The Legendary Uterus", "screen_name": "jamaicaklove", "lang": "en", "location": "arizona ", "create_at": date("2011-08-13"), "description": "| janet × michael | snapchat & ig: jamaicaklove | l.u.t.f.a | baby al mana is coming |", "followers_count": 768, "friends_count": 219, "statues_count": 35517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956432258596864, "text": "I'm dating your father :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45215551, "name": "Jake Lang", "screen_name": "FakeWang", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2009-06-06"), "description": "I sell t-shirts because I desperately want teenagers to think I'm cool. snapchat:Fakewang", "followers_count": 3534, "friends_count": 722, "statues_count": 46625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Des Moines, IA", "id": "4a9aecc7ad6b85e6", "name": "West Des Moines", "place_type": "city", "bounding_box": rectangle("-93.846184,41.528008 -93.703326,41.600507") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1983910, "cityName": "West Des Moines" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956432459882497, "text": "@ebbtideapp Tide in Scotland, Virginia 04/18/2016\n Low 4:33am 0.3\nHigh 10:21am 1.9\n Low 4:55pm 0.2\nHigh 10:39pm 2.0\n Low 5:21am 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-76.7833,37.185"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 73, "friends_count": 1, "statues_count": 25394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51181, "countyName": "Surry", "cityID": 5170715, "cityName": "Scotland" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956432711655425, "text": "03:00:01 |Temp: 44.0ºF |Dew Point 40.7ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the NE, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956432963194880, "text": "Do you ever have one of those days that doesn't seem like it even happened? Ya that's me right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413587228, "name": "dad", "screen_name": "XavierZelis", "lang": "en", "location": "Collinsville, OK", "create_at": date("2011-11-15"), "description": "I've gotten several rec league MVP's", "followers_count": 298, "friends_count": 329, "statues_count": 2570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collinsville, OK", "id": "196ca031cf73a199", "name": "Collinsville", "place_type": "city", "bounding_box": rectangle("-95.874154,36.336519 -95.800951,36.38018") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4016350, "cityName": "Collinsville" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956433223258112, "text": "Trying so hard not to fall asleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.593796,45.589169"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711401444360851456, "name": "ألبيرتو", "screen_name": "otreblazneas", "lang": "en", "location": "Pacific Northwest", "create_at": date("2016-03-19"), "description": "Life imitates art, and bitches love to talk shit.", "followers_count": 46, "friends_count": 142, "statues_count": 70 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland International Airport (PDX)", "id": "07d9f31b52c87004", "name": "Portland International Airport (PDX)", "place_type": "poi", "bounding_box": rectangle("-122.59379609999999,45.5891689 -122.593796,45.589169") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956433399427072, "text": "So am I going to get paid to take this picture down? https://t.co/x1IDLHvD0q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1906272632, "name": "Son Vu", "screen_name": "SonLVu", "lang": "en", "location": "null", "create_at": date("2013-09-25"), "description": "null", "followers_count": 374, "friends_count": 287, "statues_count": 3564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956433609277442, "text": "@OG_TheMacc lol hell yea", "in_reply_to_status": 721956322971820032, "in_reply_to_user": 283715503, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 283715503 }}, "user": { "id": 430875079, "name": "Glorious", "screen_name": "C__LVW", "lang": "en", "location": "DDD Hoe", "create_at": date("2011-12-07"), "description": "stacking & maintaining", "followers_count": 1153, "friends_count": 706, "statues_count": 28083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956433860767744, "text": "@XOR3_ sucker ����", "in_reply_to_status": 721944307964313600, "in_reply_to_user": 4331528594, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4331528594 }}, "user": { "id": 1451539723, "name": "mariah", "screen_name": "riah_perezz", "lang": "en", "location": "null", "create_at": date("2013-05-23"), "description": "chillin with kens", "followers_count": 303, "friends_count": 288, "statues_count": 2968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434112614405, "text": "تصبحون على خير جميعا", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 919144776, "name": "KH.Al3jmi", "screen_name": "al_3jmi93", "lang": "en", "location": "usa/q8 (egaila)", "create_at": date("2012-11-01"), "description": "milanista احب تفاصيل كرة القدم ، لا اميل الى السياسه ، كسول جدا", "followers_count": 684, "friends_count": 228, "statues_count": 13037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cookeville, TN", "id": "3448f4f82d2187bc", "name": "Cookeville", "place_type": "city", "bounding_box": rectangle("-85.601741,36.046986 -85.418673,36.232069") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47141, "countyName": "Putnam", "cityID": 4716920, "cityName": "Cookeville" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434158559232, "text": "@ADarkSkin_Guy glad I can help ����������", "in_reply_to_status": 721956371709734912, "in_reply_to_user": 3170708920, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3170708920 }}, "user": { "id": 370924974, "name": "Gangsta Leannnnn", "screen_name": "_NiaJoy_", "lang": "en", "location": "Instagram: _NiaJoy_", "create_at": date("2011-09-09"), "description": "Harrisburg, PA|New Orleans| Xavier University of Louisiana #LibraryAndChill #BiologyIsLiterallyLife #xula19", "followers_count": 3942, "friends_count": 1918, "statues_count": 119244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434158596096, "text": "@Arietachung @A_B_hoops @ThatboidDj we do it on purpose", "in_reply_to_status": 721954118860812292, "in_reply_to_user": 2175344791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2175344791, 2753763636, 1494634136 }}, "user": { "id": 1494634136, "name": "sleejus", "screen_name": "ThatboidDj", "lang": "en", "location": "null", "create_at": date("2013-06-08"), "description": "T.R.O.Y", "followers_count": 473, "friends_count": 432, "statues_count": 7341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434158690304, "text": "Wind 0.0 mph ---. Barometer 30.298 in, Falling slowly. Temperature 51.2 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 59, "friends_count": 27, "statues_count": 19179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434343120896, "text": "@Mary_Beth_Pike https://t.co/ASWyMPTsKc", "in_reply_to_status": 721956186279579648, "in_reply_to_user": 379369388, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 379369388 }}, "user": { "id": 842055126, "name": "Nolan Arnold", "screen_name": "nolanarnoldzzz", "lang": "en", "location": "Southlake, TX", "create_at": date("2012-09-23"), "description": "they call me The Boat", "followers_count": 149, "friends_count": 149, "statues_count": 985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434347360261, "text": "@TheFizzyBubbly she's getting it last lol", "in_reply_to_status": 721955983971319808, "in_reply_to_user": 922185967, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 922185967 }}, "user": { "id": 227217648, "name": "kikee", "screen_name": "CubbsDog", "lang": "en", "location": "San Bernardino", "create_at": date("2010-12-15"), "description": "#RIPCyleeLockhart #RipJefferyMorris #RIPKMAC", "followers_count": 2604, "friends_count": 364, "statues_count": 36968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-18T00:00:01.000Z"), "id": 721956434389250051, "text": "#Trump in 2016 Temp:50.0°F Wind:0.0mph Pressure: 30.30hpa Steady Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 17, "statues_count": 315510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956434808848384, "text": "03:00 52.3°F Feels:52.3°F (Hi59.7°F/Lo52.3°F) Hum:59% Wnd:--- 0.0MPH Baro:30.08in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 199, "friends_count": 262, "statues_count": 28533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956435106664448, "text": "Wind 0.0 mph ---. Barometer 30.242 in, Steady. Temperature 58.8 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956436029235201, "text": "\"Become wise by walking with the wise\" https://t.co/97Kt3ivoNI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540511826, "name": "Diego_Martinez", "screen_name": "The_Guatemalan", "lang": "en", "location": "Hollywood", "create_at": date("2012-03-29"), "description": "Representing the heart of God. #MyOasisLA", "followers_count": 651, "friends_count": 393, "statues_count": 13618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956436087955456, "text": "Cry and chill ??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983990953, "name": "mari", "screen_name": "Thizzymarley", "lang": "en", "location": "Bay Area", "create_at": date("2012-12-01"), "description": "sc:lilcroptop https://twitter.com/fucktyler/status/33648200095113216", "followers_count": 2228, "friends_count": 413, "statues_count": 19826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437119729664, "text": "Temp: 59.9°F Wind:0.0mph Pressure: 30.233hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437337870337, "text": "Just keep it real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 947765816, "name": "Randy", "screen_name": "BuffaloRandy_", "lang": "en", "location": "null", "create_at": date("2012-11-14"), "description": "......", "followers_count": 282, "friends_count": 265, "statues_count": 20204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437551751168, "text": ".@kristeligt #Denmark wants more articles like these please! #media cover #animal issues! https://t.co/j2DmvgGX9i https://t.co/LH5BLZ7qwo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Denmark", "media", "animal" }}, "user_mentions": {{ 21741359 }}, "user": { "id": 1380652956, "name": "Cindy Zimmermann", "screen_name": "govnmtgirl", "lang": "en", "location": "California, USA", "create_at": date("2013-04-25"), "description": "I am the voice for all animals.", "followers_count": 166, "friends_count": 250, "statues_count": 10067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Imperial Beach, CA", "id": "5cc07810d110956f", "name": "Imperial Beach", "place_type": "city", "bounding_box": rectangle("-117.133501,32.555434 -117.099564,32.591984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 636294, "cityName": "Imperial Beach" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437623066624, "text": "68.1F (Feels: 68.1F) - Humidity: 86% - Wind: 6.9mph E - Gust: 10.7mph - Pressure: 1015.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437778243585, "text": "@sofiecosmic I don't remember this picture ��������", "in_reply_to_status": 721544560006930432, "in_reply_to_user": 2275913576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275913576 }}, "user": { "id": 826643514, "name": "Alba Rodriguez", "screen_name": "albaaa443", "lang": "en", "location": "null", "create_at": date("2012-09-15"), "description": "вay area lιvιn ✨", "followers_count": 230, "friends_count": 210, "statues_count": 6385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956437950337024, "text": "I can't sleep and my IV is old and sore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152821342, "name": "nash the unkindled", "screen_name": "vulcanscience", "lang": "en", "location": "Manhattan, NY", "create_at": date("2010-06-06"), "description": "26f / nyc riffraff / trekker / vulcan / socialist / dark souls / we're gonna rattle this ghost town", "followers_count": 316, "friends_count": 400, "statues_count": 24256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956438176731137, "text": "DJI sells new filmmaking drone with rooftop karate and stunt racing https://t.co/5Im50vy9Uo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17609, "friends_count": 17549, "statues_count": 70752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956438285881344, "text": "Temp 41.7°F Wind Chill 41.7°F RH 83% Wind 0.0 --- Gust 0.0 --- SLP 30.343 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 107, "friends_count": 63, "statues_count": 34788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-18T00:00:02.000Z"), "id": 721956438671667200, "text": "@RobertGreene Ninon says that an enemy of seduction is disillusionment... if one is disillusioned, can they be re-enchanted later?", "in_reply_to_status": -1, "in_reply_to_user": 3763761, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3763761 }}, "user": { "id": 434366392, "name": "Danisha Carter", "screen_name": "DanishaCarter4", "lang": "en", "location": "null", "create_at": date("2011-12-11"), "description": "Singer || Creator || Callipygian || Egypt || LV✈️✈✈️️NYC I can sing like I found Jesus | ملك", "followers_count": 11009, "friends_count": 27, "statues_count": 87844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956439154102272, "text": "Ripley SW Limestone Co. Temp: 51.1°F Wind:0.7mph Pressure: 1005.8mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956439540002816, "text": "I can't believe I left these clothes over here. I'm livid and lazy as fuck right now. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2147950884, "name": "Johnny", "screen_name": "johnnylenell", "lang": "en", "location": "null", "create_at": date("2013-10-21"), "description": "null", "followers_count": 617, "friends_count": 232, "statues_count": 31663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956439980244992, "text": "lol no fucks given about you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1406042588, "name": "reinä g", "screen_name": "megangonzaaalez", "lang": "en", "location": "null", "create_at": date("2013-05-05"), "description": "lil emotions but shawty still bad af", "followers_count": 145, "friends_count": 59, "statues_count": 60 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duarte, CA", "id": "ddf8317a51899bd8", "name": "Duarte", "place_type": "city", "bounding_box": rectangle("-117.990614,34.118256 -117.929718,34.165263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619990, "cityName": "Duarte" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956440009781248, "text": "@AyooCharlie_ @Itzrayuuu", "in_reply_to_status": 524259653455060992, "in_reply_to_user": 480059324, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 480059324, 2842608032 }}, "user": { "id": 604770307, "name": "$norlax", "screen_name": "Ayo_Nando_", "lang": "en", "location": "SomewhereDrivingMyCivic", "create_at": date("2012-06-10"), "description": "Snapchat: @Crxlazaro Dead Roses....[OlllllO] Im the Guy that drives the sedan ;)", "followers_count": 339, "friends_count": 272, "statues_count": 51481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956440248684545, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 588054611, "name": "bailee wishart", "screen_name": "BaileeWishart", "lang": "en", "location": "ASU☀️", "create_at": date("2012-05-22"), "description": "swooofy || curly || #TeamIronMan || #TeamJesus || Respect Movement ASU", "followers_count": 516, "friends_count": 319, "statues_count": 29814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956440953524225, "text": "Yes he is fantastic.�� https://t.co/Aq2ecP9lGT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1073963870, "name": "♡Honeybear¤AM♡", "screen_name": "Autumnal89", "lang": "en", "location": "Fremont MI", "create_at": date("2013-01-09"), "description": "Small town girl. Trying to live my best life❤", "followers_count": 148, "friends_count": 415, "statues_count": 1355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, MI", "id": "90d30af2d1ff26ee", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-85.988894,43.438258 -85.920051,43.485624") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26123, "countyName": "Newaygo", "cityID": 2630700, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956441175769088, "text": "@Chloe_Parris awww dang man. It's 3... They're closed now", "in_reply_to_status": 721955937968353280, "in_reply_to_user": 35346196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35346196 }}, "user": { "id": 175234820, "name": "lexi.", "screen_name": "_AlexisSpokeIt", "lang": "en", "location": "Charlotte, NC", "create_at": date("2010-08-05"), "description": "null", "followers_count": 2144, "friends_count": 1636, "statues_count": 58557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956441310031872, "text": "Wind 0.0 mph ---. Barometer 30.174 in, Steady. Temperature 59.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956441527967744, "text": "He pushes my buttons but I'd do anything for him https://t.co/rrhqkj7hOR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 544764055, "name": "Em", "screen_name": "woaheyitsemily", "lang": "en", "location": "yay area ⛅", "create_at": date("2012-04-03"), "description": "I don't cause commotions I am one", "followers_count": 423, "friends_count": 773, "statues_count": 20972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956441666355201, "text": "Having A Photographic Memory Is A Blessing & A Curse... Its Harder To Let Go Of Things/People When Their Image Is Embedded Like A Fkn Virus.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4740413558, "name": "Cat Soup", "screen_name": "izzzzyyyy_____", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2016-01-10"), "description": "21, Oilfield Crew Pusher/Operator", "followers_count": 29, "friends_count": 56, "statues_count": 336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956441683312640, "text": "Wind 1.6 mph WNW. Barometer 30.27 in, Falling slowly. Temperature 52.5 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 159142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956442287185920, "text": "Bad Day, Hold Tight, & Change Me ❤ https://t.co/Pa2t6tzeqh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311176177, "name": "Princess", "screen_name": "princess_alyxx", "lang": "en", "location": "Oklahoma, USA", "create_at": date("2011-06-04"), "description": "@justinbieber❤❤", "followers_count": 832, "friends_count": 747, "statues_count": 16413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAlester, OK", "id": "d1bb2686669067d6", "name": "McAlester", "place_type": "city", "bounding_box": rectangle("-95.82443,34.882342 -95.726154,34.964015") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40121, "countyName": "Pittsburg", "cityID": 4044800, "cityName": "McAlester" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956442995986432, "text": "you'll never make genuine memories like we did but I will with someone else", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1188392442, "name": "Salome Duenes", "screen_name": "sduenes", "lang": "en", "location": "Ventura, CA", "create_at": date("2013-02-16"), "description": "little minds are tamed and subdued by misfortune, but great minds rise above them. ~ WI", "followers_count": 161, "friends_count": 517, "statues_count": 3816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-18T00:00:03.000Z"), "id": 721956443100827648, "text": "Me AF https://t.co/D7izMGBnkj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21272414, "name": "Madiiiii", "screen_name": "madisonxmorgan", "lang": "en", "location": "chino hills", "create_at": date("2009-02-18"), "description": "not looking to be found, just want to feel (un)lost.", "followers_count": 747, "friends_count": 331, "statues_count": 38878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956443155398656, "text": "@asdfghjkaylene happy birthday beautiful! Have a wonderful day", "in_reply_to_status": -1, "in_reply_to_user": 3192884905, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3192884905 }}, "user": { "id": 904799502, "name": "Alora✨", "screen_name": "A_TizzleYo", "lang": "en", "location": "Indio, CA", "create_at": date("2012-10-25"), "description": "ig/sc: missypink11", "followers_count": 1508, "friends_count": 1104, "statues_count": 49570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956443583291392, "text": "if u got ya pussy all out and shit, hell yea imma look and stare and say shit..shid, I'm gay as hell, he'll yea��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1007861780, "name": "kayy.", "screen_name": "Callmegunz__", "lang": "en", "location": "null", "create_at": date("2012-12-12"), "description": "STAY W IT!!", "followers_count": 5399, "friends_count": 4165, "statues_count": 124511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956443654615040, "text": "Hopefully I get to find the one for me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4827590013, "name": "DollyGirl Missy", "screen_name": "DanielleMissy2", "lang": "en", "location": "null", "create_at": date("2016-01-19"), "description": "its better off letting go #loganAmir Mommy God&Money", "followers_count": 221, "friends_count": 295, "statues_count": 2157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956443910328320, "text": "idk if its to early to start asking my bf to prom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2347748130, "name": "MA", "screen_name": "marleenaranda", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "null", "followers_count": 194, "friends_count": 216, "statues_count": 1340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956444581584896, "text": "https://t.co/unPSv15mjc Jennifer Lopez - Waiting For Tonight remix", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155910701, "name": "Chris R", "screen_name": "PsychicStoner", "lang": "en", "location": "Gulf Breeze, FL, USA", "create_at": date("2010-06-15"), "description": "Filthy, unfiltered thoughts, dirty jokes, deviant, metal, Nihilist, free thinker, Marvel and Anarchy. Follow at your own risk. No minors admitted", "followers_count": 3158, "friends_count": 3027, "statues_count": 231185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midway, FL", "id": "00030a773a4ffe7a", "name": "Midway", "place_type": "city", "bounding_box": rectangle("-87.136336,30.367273 -86.950356,30.442973") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1245475, "cityName": "Midway" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956444862423040, "text": "@ThatRuben would I lie to u ��", "in_reply_to_status": 721956050824355840, "in_reply_to_user": 33720266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33720266 }}, "user": { "id": 553270273, "name": "lil brown eyez", "screen_name": "almaskeet", "lang": "en", "location": "sur califas", "create_at": date("2012-04-13"), "description": "soñadora", "followers_count": 127, "friends_count": 115, "statues_count": 36648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445135089664, "text": "@cee_laura @Zveronica27 everyone is gonna get new brushes, i saw that on the moments page", "in_reply_to_status": 721956296929337344, "in_reply_to_user": 1252429172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1252429172, 3018617580 }}, "user": { "id": 194434683, "name": "brian", "screen_name": "easybreezy41", "lang": "en", "location": "South Central Los Angeles, CA", "create_at": date("2010-09-23"), "description": "i comment on everything", "followers_count": 538, "friends_count": 499, "statues_count": 20209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445294436353, "text": "@brother_dag u like it", "in_reply_to_status": 721955641812627456, "in_reply_to_user": 97009441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97009441 }}, "user": { "id": 30264426, "name": "emily", "screen_name": "THADRIVEIN", "lang": "en", "location": "LA", "create_at": date("2009-04-10"), "description": "Emily", "followers_count": 7937, "friends_count": 1034, "statues_count": 90433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445441294336, "text": "���������� https://t.co/fSLLuVdO6G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1616693364, "name": "Simaine Stewart", "screen_name": "king_stu3", "lang": "en", "location": "Pasadena, CA", "create_at": date("2013-07-23"), "description": "Tryin to make it in the game of life Psalms 145:3", "followers_count": 89, "friends_count": 161, "statues_count": 1242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monrovia, CA", "id": "174309ec90de94d0", "name": "Monrovia", "place_type": "city", "bounding_box": rectangle("-118.023461,34.099706 -117.975357,34.178637") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648648, "cityName": "Monrovia" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445495934976, "text": "\"Pagsisipingin ba ang papel at tinta?\"\nSer, napahanga mo ako sayong talinghaga. https://t.co/TbHt5FtjVy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 4332655353, "name": "Jess", "screen_name": "kebemamma", "lang": "en", "location": "Broadway, VA", "create_at": date("2015-11-30"), "description": "Kitchen warrior by day, fangirling insomniac at night. Fan of love expressed in any art form! Mirrors on WP: http://w.tt/1XBt4dr", "followers_count": 1844, "friends_count": 90, "statues_count": 12952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broadway, VA", "id": "b5f2d122e31983a1", "name": "Broadway", "place_type": "city", "bounding_box": rectangle("-78.815941,38.574337 -78.783927,38.623188") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51165, "countyName": "Rockingham", "cityID": 5110040, "cityName": "Broadway" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445894225920, "text": "The Bitch nail broke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3122664098, "name": "5tr8drop5piffy", "screen_name": "5tr8drop5piffy1", "lang": "en", "location": "Acres Homes Tx ", "create_at": date("2015-03-31"), "description": "#DontSmokeWithReggieHoe #TheAudioDopeDealer #MILEHIGHCLUB #STEELERNATION #OKCTHUNDER IG:5tr8drop5piffy #WH400P‼️#LIBRA @5tr8drop5piffy new account", "followers_count": 688, "friends_count": 1554, "statues_count": 2205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956445923766276, "text": "Wanna see a body ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2289157013, "name": "#CTC DeathAss ™ ❄️❄️", "screen_name": "DeathAssPinnup", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2014-01-17"), "description": "⚠️ NO DM's ⚠️", "followers_count": 518, "friends_count": 302, "statues_count": 23937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glassport, PA", "id": "c62b4a2c861eeeac", "name": "Glassport", "place_type": "city", "bounding_box": rectangle("-79.897591,40.311242 -79.872314,40.345456") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4229432, "cityName": "Glassport" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956446364000256, "text": "@manchild1077 https://t.co/O1bGxwuGky", "in_reply_to_status": 721944269460611072, "in_reply_to_user": 55019353, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 34832649 }}, "user": { "id": 55019353, "name": "Mark El Tigre", "screen_name": "SSFCFOTY13", "lang": "en", "location": "Kirkland, WA", "create_at": date("2009-07-08"), "description": "Cougs, Sounders FC, Mountaineer, Dave Matthews Band, Beer & Wine. #FOTY13, SCT, Real Oviedo shareholder #YNYA tweets are my own", "followers_count": 1601, "friends_count": 1809, "statues_count": 73630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everest, Kirkland", "id": "7e8260459b2324d2", "name": "Everest", "place_type": "neighborhood", "bounding_box": rectangle("-122.199612,47.666245 -122.184671,47.679358") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335940, "cityName": "Kirkland" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956446682755072, "text": "@_niiico97 lmao https://t.co/jrhUsoc6Hy", "in_reply_to_status": 721955520228171780, "in_reply_to_user": 636673526, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 636673526 }}, "user": { "id": 119209516, "name": "manda", "screen_name": "manndapanda_", "lang": "en", "location": "LA", "create_at": date("2010-03-02"), "description": "null", "followers_count": 1478, "friends_count": 987, "statues_count": 23269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956447097978880, "text": "Wind 0.0 mph ---. Barometer 30.348 in, Steady. Temperature 56.0 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 390, "friends_count": 292, "statues_count": 9880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956447102361600, "text": "Wind 0.0 mph ---. Barometer 30.255 in, Falling. Temperature 38.3 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-18T00:00:04.000Z"), "id": 721956447169441792, "text": "Yerba Buena Island (Goat Island) (1875) https://t.co/QpCW5sjQYl #yerbabuenaisland #phares #sanfrancisco #california https://t.co/9gWLuceuXn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.362274,37.807419"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "yerbabuenaisland", "phares", "sanfrancisco", "california" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 936, "friends_count": 1294, "statues_count": 4642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956447668404224, "text": "@FlexLuthor17 tamales are my happy place", "in_reply_to_status": 721955221409124353, "in_reply_to_user": 1444612879, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1444612879 }}, "user": { "id": 26538412, "name": "ƎYΛ5U ΛKΛ J05H", "screen_name": "efelleke", "lang": "en", "location": "Whereintheworldiscarmensd?", "create_at": date("2009-03-25"), "description": "USC Grad // Coffee Afficionado // JS Dev http://JoshuasRoasting.com", "followers_count": 303, "friends_count": 575, "statues_count": 2617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956448209608704, "text": "Lmao bro how https://t.co/dvE3nAiQsF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1671501980, "name": "Ⓜ️omo❄️", "screen_name": "sheaintmomo", "lang": "en", "location": "nunya", "create_at": date("2013-08-14"), "description": "rip mike mike ❤️ sc: thamomoshow", "followers_count": 1243, "friends_count": 1121, "statues_count": 29512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956448213688320, "text": "@daniellelcosta UGH", "in_reply_to_status": 721887375081648129, "in_reply_to_user": 3192915290, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3192915290 }}, "user": { "id": 1210685600, "name": "jayyy", "screen_name": "jassyverdult", "lang": "en", "location": "Montecito, CA", "create_at": date("2013-02-22"), "description": "pasta with garlic sauce and chill?", "followers_count": 186, "friends_count": 197, "statues_count": 1715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montecito, CA", "id": "2ddc5725a8d3924e", "name": "Montecito", "place_type": "city", "bounding_box": rectangle("-119.675838,34.415858 -119.575876,34.456163") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 648844, "cityName": "Montecito" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956448830300160, "text": "Wind 1.6 mph W. Barometer 30.080 in, Rising. Temperature 61.5 °F. Rain today 0.19 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956449295806465, "text": "Wind 0.0 mph ---. Barometer 30.020 in, Steady. Temperature 56.2 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 269138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956449379840000, "text": "@swiss_wali hell the book of Mormon is a musical created by the writers of south park that literally makes fun of the actual religion", "in_reply_to_status": 721956021124669440, "in_reply_to_user": 1117802160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1117802160 }}, "user": { "id": 455325612, "name": "RetroBark!", "screen_name": "RetroPupbark", "lang": "en", "location": "Dayton, OH", "create_at": date("2012-01-04"), "description": "Let's Go on An Adventure .\n Fursuit by @donthugcacti", "followers_count": 5315, "friends_count": 2163, "statues_count": 48034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daytona Beach, FL", "id": "5876e96f70283826", "name": "Daytona Beach", "place_type": "city", "bounding_box": rectangle("-81.133668,29.127565 -81.003444,29.252881") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1216525, "cityName": "Daytona Beach" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956449740390400, "text": "If not I feel unappreciated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 998742720, "name": "fi$$h", "screen_name": "supfissh17", "lang": "en", "location": "null", "create_at": date("2012-12-08"), "description": "skurrrt skurrrt", "followers_count": 707, "friends_count": 692, "statues_count": 39641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arvin, CA", "id": "22d1f67e5dc77410", "name": "Arvin", "place_type": "city", "bounding_box": rectangle("-118.860132,35.18714 -118.805953,35.230936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 602924, "cityName": "Arvin" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956449769754627, "text": "�������� this is life in one message https://t.co/idE33bWLxs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3297526868, "name": "Kanyen", "screen_name": "kguice_", "lang": "en", "location": "Cove", "create_at": date("2015-07-27"), "description": "19. It is what it is", "followers_count": 324, "friends_count": 350, "statues_count": 1362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Copperas Cove, TX", "id": "f0eebea3b08e0158", "name": "Copperas Cove", "place_type": "city", "bounding_box": rectangle("-97.950002,31.081381 -97.867527,31.16156") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48099, "countyName": "Coryell", "cityID": 4816624, "cityName": "Copperas Cove" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956449778143233, "text": "LaVergne for Congress 3rd cd NJ. The real 3rd CD Candidate. 5 stepped up. LaVergne won. Support LaVergne. https://t.co/mQcd32dnac Retweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23772575, "name": "Scott Neuman", "screen_name": "kickyourace", "lang": "en", "location": "Manchester Twp, NJ", "create_at": date("2009-03-11"), "description": "Scott Neuman - President - Recordweb Communications - Running for Ocean County Board of Chosen Freeholders - Blogger - Corruption Fighter", "followers_count": 621, "friends_count": 1939, "statues_count": 8471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Lake Park, NJ", "id": "007998004ba3efc0", "name": "Pine Lake Park", "place_type": "city", "bounding_box": rectangle("-74.288073,39.987032 -74.231391,40.032622") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3458830, "cityName": "Pine Lake Park" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956450126340096, "text": "Real nigga hours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2166075008, "name": "FRANNY", "screen_name": "xFrancisRoque", "lang": "en", "location": "NINA, CA", "create_at": date("2013-10-30"), "description": "null", "followers_count": 798, "friends_count": 262, "statues_count": 13049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956450260504577, "text": "I feel it #relatable https://t.co/XGq8qNhJ5C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "relatable" }}, "user": { "id": 1607676073, "name": "☁️jessí☁️", "screen_name": "yungjxssi", "lang": "en", "location": "FresYES, ca", "create_at": date("2013-07-20"), "description": "be my friend?", "followers_count": 445, "friends_count": 315, "statues_count": 12588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956450273067008, "text": "Welp. Ended up in MoVal ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2633533054, "name": ".sharaya", "screen_name": "__swiirl", "lang": "en", "location": "c a l i f o r n i a ", "create_at": date("2014-06-22"), "description": "I myself am made entirely of flaws, stitched together with good intentions.", "followers_count": 97, "friends_count": 92, "statues_count": 8128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956450285649920, "text": "wildin tonight fuck it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422291723, "name": "BREADWINNERJΛ$H ♻️", "screen_name": "joshMOB_", "lang": "en", "location": "somewhere packflippin", "create_at": date("2011-11-26"), "description": "blinded by $uccessful thoughts • #txst • HTMB", "followers_count": 745, "friends_count": 672, "statues_count": 22055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956450835169281, "text": "Daniel. Anatoly. Gabriel. ���� https://t.co/uE8L0Om13E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 821028307, "name": "Viper Frost", "screen_name": "EverTheViper", "lang": "en", "location": "G e o n o s i s.", "create_at": date("2012-09-12"), "description": "Folkie. Lover of the Light. Supreme Bitch King. Sweet Moon Child. Till the end of the line, @VivaLaGloriaV? 「#Zishy #Vicon #Ciper #Veri #Elviper」 #FrostBites", "followers_count": 1779, "friends_count": 943, "statues_count": 97272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956451099365376, "text": "Bro ������ https://t.co/XOxhLHfPlY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 31571933, "name": "assholes anonymous", "screen_name": "St33zyxSaleen", "lang": "en", "location": "Old Dominion University", "create_at": date("2009-04-15"), "description": "Only the strong survive...", "followers_count": 719, "friends_count": 642, "statues_count": 21705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-04-18T00:00:05.000Z"), "id": 721956451455881216, "text": "FYI if you follow me on snapchat you'll love the 4 songs I snapped", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226616445, "name": "LL BOOL J", "screen_name": "3_Two_Juan", "lang": "en", "location": "Georgia, USA", "create_at": date("2010-12-14"), "description": "Snapchat: iateassonce|GA is home|Imported from SC|Music saved my life|Baptized by Rap|Above the influence|Black Hippie lifestyle|Kendrick's Ghostwriter|", "followers_count": 1242, "friends_count": 757, "statues_count": 140784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warner Robins, GA", "id": "4caf02a6f195ddab", "name": "Warner Robins", "place_type": "city", "bounding_box": rectangle("-83.728243,32.495868 -83.554155,32.669392") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1380508, "cityName": "Warner Robins" } }
+{ "create_at": datetime("2016-04-18T00:00:06.000Z"), "id": 721956451564957697, "text": "@paucasillas2 pero saludas!!!! Eh?", "in_reply_to_status": 721955080660852736, "in_reply_to_user": 363401965, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 363401965 }}, "user": { "id": 2878294421, "name": "Fer Perez", "screen_name": "fregandiux", "lang": "es-MX", "location": "De Mexico, Monterrey y Cordoba", "create_at": date("2014-11-15"), "description": "guitar tech, stage manager, chelero, buen amigo, melomano, hard-glam rocker, viajero frecuente y Americanista.!! Ha*Ash crew...!!", "followers_count": 925, "friends_count": 688, "statues_count": 958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Cruces, NM", "id": "f515486276aa6192", "name": "Las Cruces", "place_type": "city", "bounding_box": rectangle("-106.843427,32.238361 -106.63145,32.446238") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3539380, "cityName": "Las Cruces" } }
+{ "create_at": datetime("2016-04-18T00:00:06.000Z"), "id": 721956452613554176, "text": "@itismarcoL chill why", "in_reply_to_status": 721956317066178561, "in_reply_to_user": 719633115380084736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 719633115380084736 }}, "user": { "id": 636006351, "name": "Turi", "screen_name": "aangeles300", "lang": "en", "location": "Montebello, CA", "create_at": date("2012-07-14"), "description": "null", "followers_count": 553, "friends_count": 398, "statues_count": 29073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-04-18T00:00:06.000Z"), "id": 721956453007773696, "text": "Just got a W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132322406, "name": "Matthew Bennett", "screen_name": "Istayraw58", "lang": "en", "location": "null", "create_at": date("2010-04-12"), "description": "suhhh dude", "followers_count": 692, "friends_count": 616, "statues_count": 14106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-04-18T00:00:06.000Z"), "id": 721956453033123840, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.237 in, Falling. Temp 39.6F. Rain today 0.00 in. Humidity 66% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 616, "friends_count": 823, "statues_count": 40917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-18T00:00:06.000Z"), "id": 721956453142036480, "text": "@samklakulak perf ����", "in_reply_to_status": 721956390231609344, "in_reply_to_user": 596575946, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 596575946 }}, "user": { "id": 3108868459, "name": "Tay", "screen_name": "taylorleonard97", "lang": "en", "location": "null", "create_at": date("2015-03-26"), "description": "Live a life worth living", "followers_count": 422, "friends_count": 398, "statues_count": 2324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318814898065408, "text": "All they wanna know is what I got to give", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386319848, "name": "Avery Urban", "screen_name": "23urb23", "lang": "en", "location": "Kaneohe Station, HI", "create_at": date("2011-10-06"), "description": "snapchat- avery_urban23", "followers_count": 875, "friends_count": 838, "statues_count": 8554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaneohe Station, HI", "id": "df32dd0ac7c1cfc2", "name": "Kaneohe Station", "place_type": "city", "bounding_box": rectangle("-157.778654,21.42733 -157.722166,21.463074") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1528400, "cityName": "Kaneohe Station" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318814910676992, "text": "I reallly don't wanna get a one bedroom , but at the same time I don't wanna go backwards and get an apartment that isn't as nice as mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059863485, "name": "JVN", "screen_name": "jahhv_", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-01-04"), "description": "#UTSA #ToyotaOffroad #Subieflow #Campvibes ~ SC:jvn48 ~IG:Jahhv ♒️", "followers_count": 464, "friends_count": 473, "statues_count": 8578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318815074336768, "text": "@cri3s_r3al plz support and i show love. comment and share on youtube JDAMEANOR sorry if u saw post\nhttps://t.co/ZzOK6ocs3A", "in_reply_to_status": -1, "in_reply_to_user": 1546868966, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1546868966 }}, "user": { "id": 2183091704, "name": "jemel booker jr", "screen_name": "JDameanor", "lang": "en", "location": "Woodbridge VA", "create_at": date("2013-11-08"), "description": "Whats good world im jemel a music producer and emcee. I moved out here from Philadelphia pa but im from Brooklyn bedstuy the best buy whats goodie!!!!!!", "followers_count": 4848, "friends_count": 2492, "statues_count": 123296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318815363735552, "text": "26402 Banning Park Ln, Katy, TX 77494, $289,000 4 beds, 2.5 baths https://t.co/nLpA4pBxBE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.8302002,29.74020004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864061615, "name": "Katy, TX News", "screen_name": "BLifeKaty", "lang": "en", "location": "Katy, Texas", "create_at": date("2014-11-06"), "description": "Katy BubbleLife features community news, photos and events. Share your business, organization or personal news and events at http://katytx.bubblelife.com.", "followers_count": 296, "friends_count": 1103, "statues_count": 22339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318815447490560, "text": "Why are you the only person tweeting right now? Tf?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 484179157, "name": "Nonie", "screen_name": "angelina_fenner", "lang": "en", "location": "Gulf Breeze, FL", "create_at": date("2012-02-05"), "description": "22. Gamer. Ravenclaw. PreMed College Student. Floridian. Future Crazy Cat Lady. Bartender.", "followers_count": 113, "friends_count": 89, "statues_count": 4849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oriole Beach, FL", "id": "007239c16f7898bf", "name": "Oriole Beach", "place_type": "city", "bounding_box": rectangle("-87.106064,30.370878 -87.086168,30.386218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12113, "countyName": "Santa Rosa", "cityID": 1252250, "cityName": "Oriole Beach" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318815917318144, "text": "Late night me just wants to tell everyone to fuck off...I'm sure early tomorrow morning me will be much more pleasant.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288007464, "name": "Stephen Cline", "screen_name": "scline87", "lang": "en", "location": "Youngstown, OH", "create_at": date("2011-04-25"), "description": "pick a star on the dark horizon, and follow the light.", "followers_count": 182, "friends_count": 326, "statues_count": 17525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland Heights, OH", "id": "aa7defe13028d41f", "name": "Cleveland Heights", "place_type": "city", "bounding_box": rectangle("-81.603358,41.482742 -81.529651,41.545274") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916014, "cityName": "Cleveland Heights" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318816584314880, "text": "all Katy ISD campuses and school facilities are closed Monday, April 18, due to weather https://t.co/SjvWMneWH5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.8302002,29.74020004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864061615, "name": "Katy, TX News", "screen_name": "BLifeKaty", "lang": "en", "location": "Katy, Texas", "create_at": date("2014-11-06"), "description": "Katy BubbleLife features community news, photos and events. Share your business, organization or personal news and events at http://katytx.bubblelife.com.", "followers_count": 296, "friends_count": 1103, "statues_count": 22340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318816940666880, "text": "03:00 57.7°F Feels:57.7°F (Hi63.1°F/Lo57.6°F) Hum:63% Wnd:--- 0.0MPH Baro:29.99in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 199, "friends_count": 262, "statues_count": 28564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318817066491906, "text": "Why the hell do I hear birds chirping ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503293862, "name": "♠️", "screen_name": "Mattythegypsy", "lang": "en", "location": "beaumont, Ca", "create_at": date("2012-02-25"), "description": "the sun and moon are like us .", "followers_count": 2145, "friends_count": 2049, "statues_count": 28250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318817091825664, "text": "03:00:01 |Temp: 47.8ºF |Dew Point 44.4ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the W, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 95, "friends_count": 21, "statues_count": 92981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318817431388160, "text": "@CallMeAlhaji yeah you're nice AF if we r being honest", "in_reply_to_status": 722318579056713728, "in_reply_to_user": 1408929482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1408929482 }}, "user": { "id": 238822713, "name": "Nsogbu IFY", "screen_name": "ifeomaO16", "lang": "en", "location": "Pla-no games, Texas", "create_at": date("2011-01-15"), "description": "PLANO/DTX |NIGERIAN| #TTU18", "followers_count": 1087, "friends_count": 572, "statues_count": 21853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318817804754944, "text": "At least I can watch tangled again.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 74572046, "name": "Marissa Patterson", "screen_name": "marissapaige13", "lang": "en", "location": "Cali ", "create_at": date("2009-09-15"), "description": "An April Ludgate that's inspiring to be Olivia Pope", "followers_count": 184, "friends_count": 116, "statues_count": 12630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318817808883712, "text": "Somethings got to give soon, or I'm gonna lose it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1973977356, "name": "The Masked Chode", "screen_name": "_Codyne", "lang": "en", "location": "null", "create_at": date("2013-10-19"), "description": "null", "followers_count": 1256, "friends_count": 492, "statues_count": 14460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318818098327552, "text": "This mcmuffin is so clutch rnnnn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336431451, "name": "Jasmine Rendon '19", "screen_name": "jas97rendon", "lang": "en", "location": "Port Lavaca ➡️ College Station", "create_at": date("2011-07-16"), "description": "TAMU '19 • Chemical Engineering major • Always moving forward while somehow stuck in my old ways.", "followers_count": 1236, "friends_count": 806, "statues_count": 11574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-04-19T00:00:00.000Z"), "id": 722318818349953025, "text": "If you call me daddy I have the right to claim you on my taxes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2999074784, "name": "Basketballs Hokage", "screen_name": "Satnjooh", "lang": "en", "location": "null", "create_at": date("2015-01-27"), "description": "Man , Visionary , God Fearing (Not the conversation type) Power Lvl 9000+", "followers_count": 511, "friends_count": 524, "statues_count": 19045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318818484195328, "text": "I tried tossing @poisonplnt onto my bed earlier and she bounced headfirst into the wall #smooth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "smooth" }}, "user_mentions": {{ 1431504205 }}, "user": { "id": 517235176, "name": "Morgan", "screen_name": "pastmycurfew", "lang": "en", "location": "some show", "create_at": date("2012-03-06"), "description": "late nite organizer @pastyourcurfew", "followers_count": 286, "friends_count": 172, "statues_count": 5638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318818509332480, "text": "@frncissdominc ��������✨��", "in_reply_to_status": -1, "in_reply_to_user": 35949400, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35949400 }}, "user": { "id": 2281279033, "name": "Ricky Lozano", "screen_name": "_RickyLozano_", "lang": "en", "location": "main street usa", "create_at": date("2014-01-07"), "description": "God can restore what is broken and change it into something amazing. All you need is faith. joel 2:25 365 days of magic annual pass holder", "followers_count": 3970, "friends_count": 4015, "statues_count": 6899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318819192999936, "text": "@JenniferrPham hopefully soon!! ��", "in_reply_to_status": 722317958760890368, "in_reply_to_user": 40719316, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40719316 }}, "user": { "id": 2299229784, "name": "Lucy Nguyen", "screen_name": "lucynguyeen", "lang": "en", "location": "null", "create_at": date("2014-01-18"), "description": "htx | uh", "followers_count": 249, "friends_count": 206, "statues_count": 4661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318819314655232, "text": "how every guy should be https://t.co/wsIHCszTAL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2955764983, "name": "Ingrid Diaz", "screen_name": "__ingriddiaz", "lang": "en", "location": "Houston, TX", "create_at": date("2015-01-01"), "description": "null", "followers_count": 477, "friends_count": 368, "statues_count": 17653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318819528679424, "text": "Temp 48.4° Hi/Lo 48.5/46.5 Rng 2.0° WC 48.4° Hmd 58% Rain 0.00\" Storm 0.00\" BAR 30.017 Falling DP 34.3° Wnd 2mph Dir S Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 68, "friends_count": 118, "statues_count": 18010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318819994120192, "text": "24 days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1621168292, "name": "N8 the Gr8", "screen_name": "_nathaaaan_", "lang": "en", "location": "Arizøna ", "create_at": date("2013-07-25"), "description": "Beauty is everywhere • |-/", "followers_count": 2881, "friends_count": 985, "statues_count": 26943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820233228289, "text": "Ugly mf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 834722622, "name": "Mo", "screen_name": "MoDollaz11", "lang": "en", "location": "Denver, CO", "create_at": date("2012-09-19"), "description": "B R A T • SC: morganencinias • i heart chocolate boys", "followers_count": 692, "friends_count": 543, "statues_count": 34060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce City, CO", "id": "58fe996bbe3a4048", "name": "Commerce City", "place_type": "city", "bounding_box": rectangle("-104.96747,39.783752 -104.771597,39.922981") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 816495, "cityName": "Commerce City" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820359086080, "text": "@queenevaaaa for real lol", "in_reply_to_status": 722318676439904256, "in_reply_to_user": 314235595, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314235595 }}, "user": { "id": 2349805225, "name": "Space Cowboy", "screen_name": "Thevman98", "lang": "en", "location": "Lamecaster ", "create_at": date("2014-02-18"), "description": "All life is precious", "followers_count": 680, "friends_count": 455, "statues_count": 39879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820426297344, "text": "�������� https://t.co/M82ts6Fxxg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 182986657, "name": "A1_PrettyFigure", "screen_name": "Pink_Kitty17", "lang": "en", "location": "Akron, OH", "create_at": date("2010-08-25"), "description": "null", "followers_count": 309, "friends_count": 730, "statues_count": 1850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820552015872, "text": "When me and emery wait until the night before to start writing our 8 page essay....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350603190, "name": "Grayson Van Meter", "screen_name": "GraysonVanMeter", "lang": "en", "location": "Huntsville, TX", "create_at": date("2011-08-07"), "description": "SHSU, Zeta Tau Alpha", "followers_count": 960, "friends_count": 364, "statues_count": 12668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820615065600, "text": "Wind 0.0 mph E. Barometer 30.137 in, Falling. Temperature 64.2 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820635910144, "text": "What to wear tomorrow for @the1975?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545443555 }}, "user": { "id": 21156000, "name": "Sabrina", "screen_name": "eyeamsabrina", "lang": "en", "location": "LA / NYC ", "create_at": date("2009-02-17"), "description": "24. Born and raised New Yawker. Music & food junkie. Hoarder of memories. Probably smell like a thrift store. Soul is in California. Sel Q inspired always ❤️", "followers_count": 1463, "friends_count": 2016, "statues_count": 52255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318820950556672, "text": "you ain't funny https://t.co/j0WnwYvbHC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 558287394, "name": "kooolinnnnn⛹", "screen_name": "JALENBOE", "lang": "en", "location": "North Carolina, USA", "create_at": date("2012-04-19"), "description": "yo soy muy guapo #ncat boyz", "followers_count": 8529, "friends_count": 5710, "statues_count": 194507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821034430464, "text": "04/19@03:00 - Temp 51.1F, WC 51.1F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.059in, Falling slowly. Rain 0.00in. Hum 78%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821072179201, "text": "had the worst dream ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2886794118, "name": "bbyray", "screen_name": "zaddyyray", "lang": "en", "location": "w/ kodak", "create_at": date("2014-11-01"), "description": "ima cool dude baby", "followers_count": 1335, "friends_count": 982, "statues_count": 28531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Laurel, MD", "id": "5642221f5e474985", "name": "West Laurel", "place_type": "city", "bounding_box": rectangle("-76.909238,39.098797 -76.873651,39.131178") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2483025, "cityName": "West Laurel" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821097410560, "text": "Wind 0.0 mph ---. Barometer 30.22 in, Falling slowly. Temperature 55.6 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821151739904, "text": "@jscelfo66 come to Nebraska! #GBR", "in_reply_to_status": -1, "in_reply_to_user": 212263382, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GBR" }}, "user_mentions": {{ 212263382 }}, "user": { "id": 335443514, "name": "Max", "screen_name": "max_kinney_", "lang": "en", "location": "the compound", "create_at": date("2011-07-14"), "description": "Mike Riley Supporter", "followers_count": 533, "friends_count": 450, "statues_count": 4461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821202243584, "text": "PLEASE MF TELL ME ����♑️♑️♑️♑️♑️♑️��✔️���� https://t.co/f61pR7ATX0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4188782325, "name": "GoodBandz™ ENT", "screen_name": "RealLulChino069", "lang": "en", "location": "173rd Tarkington Ave ♿️", "create_at": date("2015-11-14"), "description": "GoodBandz™ Music Producer Frm ClevelandOhio #RIUC Business Only @lulchino069gmail.com FOLLOW MY ARTIST @iamdeenogotti21 Free My Bro Fattz ♿️ 3 18 9 16 ♿️ TG4L", "followers_count": 389, "friends_count": 858, "statues_count": 752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Randall, OH", "id": "6414cc5b330e6e32", "name": "North Randall", "place_type": "city", "bounding_box": rectangle("-81.541516,41.424002 -81.5174,41.439976") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3956924, "cityName": "North Randall" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318821961416704, "text": "#Trump in 2016 Temp:52.5°F Wind:0.0mph Pressure: 30.18hpa Falling slowly Rain Today 0.00in. Forecast: Fairly fine, showery later", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 17, "statues_count": 315602 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318822061924352, "text": "���� https://t.co/luP6kuWw4u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 624342957, "name": "DopeBoí Dìno", "screen_name": "Don_Dino8", "lang": "en", "location": "843✈️813", "create_at": date("2012-07-01"), "description": "SC-dopeasschico82 IG-don_dino8 #LifeOfAhDon The Mixtape #BBG #BroLifeEnt", "followers_count": 933, "friends_count": 526, "statues_count": 51511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomingdale, FL", "id": "75a3d7a592190ec9", "name": "Bloomingdale", "place_type": "city", "bounding_box": rectangle("-82.29976,27.857254 -82.211304,27.893744") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1206875, "cityName": "Bloomingdale" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318822125019136, "text": "High up in the clouds is where I want to be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 457657831, "name": "Just.Chris", "screen_name": "lilchrissKB", "lang": "en", "location": "Detroit, MI ✋", "create_at": date("2012-01-07"), "description": "〽ichigan 〽ade\nSnapchat me- c_layow", "followers_count": 225, "friends_count": 153, "statues_count": 3290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn Heights, MI", "id": "59dab0b1b9779311", "name": "Dearborn Heights", "place_type": "city", "bounding_box": rectangle("-83.312805,42.268212 -83.217437,42.357044") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621020, "cityName": "Dearborn Heights" } }
+{ "create_at": datetime("2016-04-19T00:00:01.000Z"), "id": 722318822263234561, "text": "@TrochezATX :-) r wii listenin 2 da same song?", "in_reply_to_status": 722318632764637185, "in_reply_to_user": 442175510, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 442175510 }}, "user": { "id": 871985094, "name": "Saint E-ast", "screen_name": "ESolano_7", "lang": "en", "location": "Austin, TX / Sao Paulo, BR", "create_at": date("2012-10-10"), "description": "Joga Bonito.", "followers_count": 284, "friends_count": 391, "statues_count": 15232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318822674313217, "text": "ok. let me study I already planned to be up til about 4 studying anyways & watch I still don't get what I'm striving for ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228230203, "name": "nns.", "screen_name": "SearchME_NikiaS", "lang": "en", "location": "Somewhere Finessin'", "create_at": date("2010-12-18"), "description": "SouthernUBR | ❤️ | iSupportJawuan ✍", "followers_count": 3250, "friends_count": 2807, "statues_count": 125098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318822816948224, "text": "@ebbtideapp Tide in Newbold, Pennsylvania 04/19/2016\n Low 8:30am -0.1\nHigh 2:02pm 8.5\n Low 8:55pm -0.1\nHigh 2:25am 8.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.7517,40.1367"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 74, "friends_count": 1, "statues_count": 25674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pennsylvania, USA", "id": "dd9c503d6c35364b", "name": "Pennsylvania", "place_type": "admin", "bounding_box": rectangle("-80.519851,39.719801 -74.689517,42.516072") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318822846246912, "text": "Im going to sleep like a baby tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 76534505, "name": "Juan Velazquez⚽", "screen_name": "JVelazquez06", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-09-22"), "description": "Ilse Cetina ❤", "followers_count": 405, "friends_count": 363, "statues_count": 11013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, NV", "id": "37d88f13e7a85f14", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-115.173994,36.128077 -115.083699,36.144748") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3284600, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318823626579969, "text": "Wind 0.0 mph ---. Barometer 30.170 in, Falling slowly. Temperature 54.4 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824091967488, "text": "My back pack smells like spray paint.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2276782304, "name": "jess", "screen_name": "__Jessaayy", "lang": "en", "location": "null", "create_at": date("2014-01-04"), "description": "ok?", "followers_count": 666, "friends_count": 657, "statues_count": 4465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824209403904, "text": "I want you to be the arms that hold me after a hard day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4052760800, "name": "Denise Polanco", "screen_name": "Polanco2Denise", "lang": "en", "location": "null", "create_at": date("2015-10-28"), "description": "- Humble, with just a hint of Kanye - UNM - 19 -", "followers_count": 222, "friends_count": 400, "statues_count": 245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824356204544, "text": "Invest in you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467141375, "name": "Wisdom In Chains", "screen_name": "MrMunyoni", "lang": "en", "location": "Alaska, USA", "create_at": date("2012-01-17"), "description": "♈ FNO CNC #Nautical #MA #AK to #HI #PatsNation #BoatLife #JA PMA Don't fit the mold be above that.. Be water my friend 1.618", "followers_count": 349, "friends_count": 795, "statues_count": 27738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824653991936, "text": "Las Vegas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2328001795, "name": "☮❤☯⛤", "screen_name": "Always_Lizzi", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "null", "followers_count": 195, "friends_count": 148, "statues_count": 5446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824947773442, "text": "I'm starving run", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52306707, "name": "Cwēny✨", "screen_name": "Ihonestly_dgaf", "lang": "en", "location": "In Formation ✨", "create_at": date("2009-06-29"), "description": "sinful angel | cheerleader | Queen ♒️ #TeamShayStar ✨", "followers_count": 2068, "friends_count": 2238, "statues_count": 58915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318824993878017, "text": "Temp: 55.0°F Wind:0.0mph Pressure: 30.193hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318825081835520, "text": "Fave babes https://t.co/GItaC2OlQ5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2931679875, "name": "Alika Tree", "screen_name": "Thealikatree", "lang": "en", "location": "null", "create_at": date("2014-12-19"), "description": "null", "followers_count": 41, "friends_count": 29, "statues_count": 1637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodinville, WA", "id": "59454533edec2571", "name": "Woodinville", "place_type": "city", "bounding_box": rectangle("-122.176406,47.723831 -122.108973,47.776251") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5379590, "cityName": "Woodinville" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318825501229056, "text": "Estas pastillas me tiene bien atarantada �� pero se siente bien ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2189221765, "name": "Daisy Morales", "screen_name": "daiiisymo", "lang": "en", "location": "Riverside, CA ", "create_at": date("2013-11-11"), "description": "la vida te da y también te lo quita", "followers_count": 171, "friends_count": 129, "statues_count": 2570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318826105384960, "text": "Lmao the kind that get you stalked obviously �� https://t.co/1h0Cxt5yT6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326951358, "name": "G Or D", "screen_name": "taBoy_Geordi", "lang": "en", "location": "478! Haddock, Ga/Jones County/", "create_at": date("2011-06-30"), "description": "ΚΔ, Fall14 ΦΒΣ,6'2,Success is not given... Valdosta State.. IG&SC: attaboy_geordi", "followers_count": 2190, "friends_count": 1946, "statues_count": 80674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Remerton, GA", "id": "bdadf12f22b3405f", "name": "Remerton", "place_type": "city", "bounding_box": rectangle("-83.31399,30.840384 -83.303978,30.846943") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1364596, "cityName": "Remerton" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318826491150337, "text": "Follow along nigga! https://t.co/KcLqTdIin7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324074501, "name": "BeautifulBliss✨", "screen_name": "Pearlfect_Ky", "lang": "en", "location": "OKC ✈️ Las Vegas", "create_at": date("2011-06-25"), "description": "Psalm 23:4 1 9 0 8 Θβ", "followers_count": 917, "friends_count": 520, "statues_count": 30828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318826717597696, "text": "I literally have no support from anyone ... No one understands what I wanna do which fires me up to become successful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3767305032, "name": "Edgar Salazar", "screen_name": "Edgarsalazar40", "lang": "en", "location": "null", "create_at": date("2015-10-02"), "description": "Sponsored Athlete for Nutrition Zone , Entrepreneur, 18 years old", "followers_count": 189, "friends_count": 313, "statues_count": 825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-19T00:00:02.000Z"), "id": 722318826730160128, "text": "68.2F (Feels: 68.2F) - Humidity: 94% - Wind: 5.4mph E - Gust: 6.9mph - Pressure: 1013.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318826960986112, "text": "Wind 0.0 mph ---. Barometer 30.126 in, Steady. Temperature 62.0 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827204128768, "text": "@Heike11276 @Jordygirl8 @ValeBOSTON @LoveEternal_D yes need it , xoxoxo", "in_reply_to_status": 722308774569058305, "in_reply_to_user": 1035796340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1035796340, 964223024, 29191761, 2276465935 }}, "user": { "id": 85999500, "name": "JordansGotMyHeart", "screen_name": "horttim", "lang": "en", "location": "Washington, USA", "create_at": date("2009-10-28"), "description": "❤NKOTB /Jordan since '88, BH 4 life! Flwd by DannyWd 12/19/15DonnieWahlberg 1/14/16 AaronCarter 12/23/15Tiffany 2/12/16 Daniel Newman1/5,DebbieGibson2/19/16", "followers_count": 967, "friends_count": 1206, "statues_count": 20386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827443339265, "text": "Temp 44.7°F Wind Chill 44.7°F RH 82% Wind 0.0 --- Gust 0.0 --- SLP 30.194 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 107, "friends_count": 63, "statues_count": 34812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827589996544, "text": "@swaggnificientt https://t.co/mjWfoZQCjm", "in_reply_to_status": 722318709092581376, "in_reply_to_user": 346665700, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 346665700 }}, "user": { "id": 961249453, "name": "August", "screen_name": "AugustineNnodim", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-11-20"), "description": "God Bless. Cjword/FreeVell", "followers_count": 543, "friends_count": 441, "statues_count": 6635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Angelo, TX", "id": "4cb7f66ae6f6d487", "name": "San Angelo", "place_type": "city", "bounding_box": rectangle("-100.52941,31.335769 -100.376768,31.545276") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48451, "countyName": "Tom Green", "cityID": 4864472, "cityName": "San Angelo" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827791392768, "text": "Everything is perfect rn ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1070173550, "name": "❥k", "screen_name": "bitchyykarenn", "lang": "en", "location": "null", "create_at": date("2013-01-07"), "description": "null", "followers_count": 112, "friends_count": 192, "statues_count": 2533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827837636608, "text": "Wind 0.0 mph NNE. Barometer 30.00 in, Falling. Temperature 51.1 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 159168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318827933962240, "text": "Come join the stream and play with us!!! @Mojang #Minecraft #twitch https://t.co/2XPYjSmZF2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Minecraft", "twitch" }}, "user_mentions": {{ 282348945 }}, "user": { "id": 415313363, "name": "Lord Reptilia", "screen_name": "Lord_Reptilia", "lang": "en", "location": "A Galaxy Far, Far Away", "create_at": date("2011-11-17"), "description": "And suddenly, you are home. Organismal Equality. YouTuber. Gamer. Twitch Streamer. http://YouTube.com/c/LordReptilia", "followers_count": 290, "friends_count": 637, "statues_count": 5192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318828038828032, "text": ".@CaptainAmerica I stand with #TeamIronMan. #TweetMe https://t.co/MPgZhG9FvT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamIronMan", "TweetMe" }}, "user_mentions": {{ 701615052 }}, "user": { "id": 3696730813, "name": "Wilson", "screen_name": "WilsonDoan", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-09-26"), "description": "Dubs taking W's ||", "followers_count": 18, "friends_count": 19, "statues_count": 130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318828219277312, "text": "Teach ur kids to respect the freedom Elephants & other wildlife deserve. Riding them is wrong! Bad parenting. Dad ���� https://t.co/rPrBEkZNGc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4837228803, "name": "jimendres2 #CherCrew", "screen_name": "JimEndres2", "lang": "en", "location": "United States", "create_at": date("2016-01-22"), "description": "Animal Advocate, Pro Choice, Pro Peace, Anti Abuse of Children, Women, Animals & Elderly, Anti Guns, Democrat , For freedom, love, art, music, Cher fan 4ever", "followers_count": 1095, "friends_count": 1542, "statues_count": 25410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haddonfield, NJ", "id": "26f7e7c6999f79a7", "name": "Haddonfield", "place_type": "city", "bounding_box": rectangle("-75.051452,39.873035 -75.016426,39.916621") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3428770, "cityName": "Haddonfield" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318828248571904, "text": "I hate everyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1258213363, "name": "Al Capone", "screen_name": "keviikinsCh", "lang": "en", "location": "North Hollyhood", "create_at": date("2013-03-10"), "description": "6'11", "followers_count": 463, "friends_count": 294, "statues_count": 53927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318828445835265, "text": "Ripley SW Limestone Co. Temp: 51.6°F Wind:0.0mph Pressure: 1002.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318829070774272, "text": "I have so much to fucking do tomorrow it's not even funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378787824, "name": "Lee", "screen_name": "MikeLee______", "lang": "en", "location": "Richmond, VA", "create_at": date("2014-03-08"), "description": "God First Family Second 2⃣4⃣♈️ Canes| Fins | Pistons| Tarheel Basketball| Cubs", "followers_count": 1047, "friends_count": 1415, "statues_count": 92209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318829699764226, "text": "Queefs are just a large scale version of that Noise Putty stuff.....a perfect parallel...........God made this world with such care", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 879249948, "name": "Andy Baughman", "screen_name": "andy_baughman", "lang": "en", "location": "null", "create_at": date("2012-10-13"), "description": "So long gay bowser", "followers_count": 351, "friends_count": 453, "statues_count": 11942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catalina Foothills, AZ", "id": "5a51ae834390a143", "name": "Catalina Foothills", "place_type": "city", "bounding_box": rectangle("-110.980121,32.250309 -110.805635,32.355554") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 411230, "cityName": "Catalina Foothills" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318830077235200, "text": "Wind 0.0 mph NE. Barometer 30.086 in, Falling slowly. Temperature 63.2 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-19T00:00:03.000Z"), "id": 722318830664441856, "text": "Nights like this leave my mind racing��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2474701957, "name": "Maria Ivana Bill", "screen_name": "MariaIvana22", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "Tarleton State '18 sc: maria_ivana", "followers_count": 493, "friends_count": 484, "statues_count": 853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318831268585472, "text": "122 days until I move out!!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 469031267, "name": "lesly juárez", "screen_name": "yunglesly", "lang": "en", "location": "california", "create_at": date("2012-01-19"), "description": "San Francisco State University", "followers_count": 420, "friends_count": 109, "statues_count": 31347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hughson, CA", "id": "62548cdeae2a4b43", "name": "Hughson", "place_type": "city", "bounding_box": rectangle("-120.879351,37.589058 -120.851611,37.609315") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 634904, "cityName": "Hughson" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318831411048448, "text": "���� https://t.co/hsWlbsSIsv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 448735448, "name": "Hanson Ruff", "screen_name": "PancakeHomie", "lang": "en", "location": "Imagine Nation", "create_at": date("2011-12-28"), "description": "~ aspiring globemaster ~", "followers_count": 183, "friends_count": 254, "statues_count": 2775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318831901745152, "text": "Have INSTAGRAM? Need help. Would you \"Like\" this post on my account official_carolyn_dodd ? PLEASE. ��❤️ Thank you! https://t.co/0VEttPyMQn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605552222, "name": "Carolyn Dodd", "screen_name": "cakedodd", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-06-11"), "description": "Green eyed ACTOR on a mission to bring smiles. Writer. Great imagination. YouTube Host. Momheld. Mgr: @dbatalent. http://www.imdb.me/carolyndodd", "followers_count": 658, "friends_count": 781, "statues_count": 1718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318831998230528, "text": "I see everything lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2409615042, "name": "Maine ✨", "screen_name": "GucciMaine_1", "lang": "en", "location": "null", "create_at": date("2014-03-24"), "description": "Daddy D. Drizzle ❤", "followers_count": 1506, "friends_count": 856, "statues_count": 13016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318832505741313, "text": "���� https://t.co/Jhmiu84Alb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 111977123, "name": "Kprincess", "screen_name": "kprincess_", "lang": "en", "location": "San Jose, CA", "create_at": date("2010-02-06"), "description": "God. Lift Heavy. SJSU. 22. Just me.", "followers_count": 235, "friends_count": 272, "statues_count": 13990 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318832870694912, "text": "Love this! spreading_JAM @lyokana59 @TEAMBOOST", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1287467406, 36598772 }}, "user": { "id": 613123995, "name": "Ellen Deutscher", "screen_name": "lndeutsch", "lang": "en", "location": "sf bay area", "create_at": date("2012-06-19"), "description": "20+yr publicschool T,DesignThinking&Creativity Consultant,@teachersguild Coach, #DTK12chat, Co-Creator #DesignDots Shakinitupw/Shakespeare,RubberChickensinSpace", "followers_count": 2340, "friends_count": 2375, "statues_count": 12095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Carlos, CA", "id": "c7d2493232f133a0", "name": "San Carlos", "place_type": "city", "bounding_box": rectangle("-122.303432,37.47453 -122.237163,37.521347") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665070, "cityName": "San Carlos" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833097179136, "text": "Thats righhhhtt ������ https://t.co/A1QQTcIj2l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 892701230, "name": "R U B I", "screen_name": "roobzzz_", "lang": "en", "location": "Downey, CA", "create_at": date("2012-10-20"), "description": "null", "followers_count": 753, "friends_count": 564, "statues_count": 8086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833361362945, "text": "@amandaraelong ✌��️�� https://t.co/9OjFZGvFQM", "in_reply_to_status": -1, "in_reply_to_user": 2189625107, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2189625107 }}, "user": { "id": 274454178, "name": "Shannon ✌", "screen_name": "OohManItsShan", "lang": "en", "location": "Portland, OR", "create_at": date("2011-03-30"), "description": "Life's not out to get you.", "followers_count": 373, "friends_count": 756, "statues_count": 11631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833701240832, "text": "I feel like it's different. Whether it's good or bad is another story.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1542239492, "name": "Hunter Ristau", "screen_name": "hunter_ristau", "lang": "en", "location": "null", "create_at": date("2013-06-23"), "description": "*Romans 8:31* What, then, shall we say in response to these things? If God is for us, who can be against us?", "followers_count": 746, "friends_count": 646, "statues_count": 7099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lackawanna, NY", "id": "47275aa57a6ff215", "name": "Lackawanna", "place_type": "city", "bounding_box": rectangle("-78.872789,42.803551 -78.793014,42.834283") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3640189, "cityName": "Lackawanna" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833784987649, "text": "no other bitch can do it to ya like I can", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2511513080, "name": "banana♍️", "screen_name": "hfonsecaa_", "lang": "en", "location": "shs senior", "create_at": date("2014-05-20"), "description": "kill em w/ kindness", "followers_count": 1401, "friends_count": 1042, "statues_count": 26462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833826959360, "text": "WHICH ONE OF YOU WEAK ASS NIGGAS NEEDS TO UNFOLLOW ME ��", "in_reply_to_status": 722314170436161537, "in_reply_to_user": 587900993, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587900993, "name": "young metro", "screen_name": "johny2shoess", "lang": "en", "location": "Houston, TX", "create_at": date("2012-05-22"), "description": "I have asthma, can't be chasing these hoes", "followers_count": 728, "friends_count": 690, "statues_count": 28808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318833873059840, "text": "Back in vegas.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396813326, "name": "OG Bre ❤", "screen_name": "labreeanah", "lang": "en", "location": "Los Angeles/Las Vegas ", "create_at": date("2011-10-23"), "description": "visual artist", "followers_count": 924, "friends_count": 275, "statues_count": 22251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-04-19T00:00:04.000Z"), "id": 722318834720329728, "text": "I deserve to be happy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2972887010, "name": "Tiffany", "screen_name": "TiffanyMTrevino", "lang": "en", "location": "null", "create_at": date("2015-01-10"), "description": "null", "followers_count": 149, "friends_count": 66, "statues_count": 4356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alice, TX", "id": "8de14ceebb414fe3", "name": "Alice", "place_type": "city", "bounding_box": rectangle("-98.109116,27.713198 -98.036162,27.784932") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48249, "countyName": "Jim Wells", "cityID": 4801852, "cityName": "Alice" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318835223818240, "text": "@nekeysForever that's why I'm about to stop ��", "in_reply_to_status": 722318480750571520, "in_reply_to_user": 1363634299, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1363634299 }}, "user": { "id": 231984509, "name": "$haadyRack$", "screen_name": "shaady_racks", "lang": "en", "location": "westbagdad ,la", "create_at": date("2010-12-29"), "description": "Rapper/songwriter/motivationalSpeaker #TeamRacks for inquires email: shaady70058@gmail.com or Dm me", "followers_count": 904, "friends_count": 862, "statues_count": 15453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318835660038144, "text": "ITS MY BIRTHDAY ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 329208895, "name": "ye", "screen_name": "JuicyYe_", "lang": "en", "location": "outchaaaa", "create_at": date("2011-07-04"), "description": "live and maintain", "followers_count": 597, "friends_count": 243, "statues_count": 16568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318835844517888, "text": "Slowly drifting away", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 918296240, "name": "Logan Thompson", "screen_name": "RealGinsanity", "lang": "en", "location": "North Olmsted, Ohio", "create_at": date("2012-10-31"), "description": "Loading....☝. ️snapchat: loganthompson23 Instagram: loganthompson23", "followers_count": 515, "friends_count": 320, "statues_count": 22132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Olmsted, OH", "id": "aa0056ab084f5da5", "name": "North Olmsted", "place_type": "city", "bounding_box": rectangle("-81.970835,41.375218 -81.874236,41.448187") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3956882, "cityName": "North Olmsted" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318836049915906, "text": "Tomorrow will be a better day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1031917891, "name": "joel", "screen_name": "Joellhernandezz", "lang": "en", "location": "null", "create_at": date("2012-12-23"), "description": "we're not here for a long time , we're here for a good time", "followers_count": 260, "friends_count": 182, "statues_count": 25404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318836125601792, "text": "@slicc101 plz support and i show love. comment and share on youtube JDAMEANOR sorry if u saw post\nhttps://t.co/ZzOK6ocs3A", "in_reply_to_status": -1, "in_reply_to_user": 716677617202626560, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 716677617202626560 }}, "user": { "id": 2183091704, "name": "jemel booker jr", "screen_name": "JDameanor", "lang": "en", "location": "Woodbridge VA", "create_at": date("2013-11-08"), "description": "Whats good world im jemel a music producer and emcee. I moved out here from Philadelphia pa but im from Brooklyn bedstuy the best buy whats goodie!!!!!!", "followers_count": 4848, "friends_count": 2492, "statues_count": 123297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837308395520, "text": "Wind 2.8 mph NW. Barometer 30.037 in, Falling Rapidly. Temperature 40.1 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837367050240, "text": "@TonyMazur The REALEST First Black President.", "in_reply_to_status": 722314712986337280, "in_reply_to_user": 18200549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18200549 }}, "user": { "id": 2479125236, "name": "Mack Webbahoe", "screen_name": "Clevetroit", "lang": "en", "location": "Clevemada, MI ", "create_at": date("2014-05-05"), "description": "Father, Husband, Son, Brother, Army Vet • Family is Life • Mantua/Nelson, OH expat • Browns/Tribe/Cavaliers/Buckeyes/Herd • 18,739 Days Without A Championship..", "followers_count": 760, "friends_count": 1554, "statues_count": 10805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sterling Heights, MI", "id": "23e0af972b9a2e8e", "name": "Sterling Heights", "place_type": "city", "bounding_box": rectangle("-83.091535,42.534811 -82.969611,42.627801") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2676460, "cityName": "Sterling Heights" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837379522560, "text": "@JoeyRuotolo I went to the event but it didn't give me an option to sell them", "in_reply_to_status": 722318358406803458, "in_reply_to_user": 1450564118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1450564118 }}, "user": { "id": 3353841268, "name": "Kel", "screen_name": "kcsnooks", "lang": "en", "location": "null", "create_at": date("2015-07-01"), "description": "if you say you're cooler than me, does that make me hotter than you?", "followers_count": 49, "friends_count": 121, "statues_count": 1741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837421486080, "text": "Haaaaaa I'm not tired & I think I should be worried.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1636074487, "name": "brook ♛", "screen_name": "woahhbrook", "lang": "en", "location": "dallas", "create_at": date("2013-07-31"), "description": "lchs jv cheerleader | sc - loxbrooklyn | kelvin's girl", "followers_count": 575, "friends_count": 278, "statues_count": 2112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837614399489, "text": "My dog's snoring won't let me sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53733071, "name": "AlbayMyself", "screen_name": "albaisela", "lang": "en", "location": "Denton,Tx.", "create_at": date("2009-07-04"), "description": "22 | UNT | Engineering | ZTA | SC:thatoneuntgirl | IG:albaisela | #makeup | #frystreet", "followers_count": 187, "friends_count": 179, "statues_count": 845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837769728000, "text": "Dahlonega Mine Train https://t.co/mmB85bpwlM #dahlonegaminetrain #parcdattraction https://t.co/dYq9wBfMM6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.552177,33.767944"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "dahlonegaminetrain", "parcdattraction" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 935, "friends_count": 1294, "statues_count": 4673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318837803143168, "text": "Wind 0.0 mph ---. Barometer 30.186 in, Steady. Temperature 57.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 391, "friends_count": 292, "statues_count": 9906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318838306598912, "text": "Wind 0.0 mph ---. Barometer 30.014 in, Steady. Temperature 55.7 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 269421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318838478479360, "text": "@SidLeSquid_ @FlashSimeon23 ����", "in_reply_to_status": 722318758816063488, "in_reply_to_user": 1970017993, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1970017993, 55353937 }}, "user": { "id": 302762516, "name": "tHE Perfect Guy ", "screen_name": "KingSeun24", "lang": "en", "location": "Denton, TX", "create_at": date("2011-05-21"), "description": "Future CMO| ΑΦΑ | Your PHavorite Alpha Man | ΗΕ TR3 | #UNT | Nigerian | IG: kingseun24", "followers_count": 5071, "friends_count": 2632, "statues_count": 222310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318838541459456, "text": "I miss you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284183191, "name": "Remy", "screen_name": "x_JohnDoee", "lang": "en", "location": "null", "create_at": date("2011-04-18"), "description": "null", "followers_count": 1550, "friends_count": 510, "statues_count": 98498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hallandale Beach, FL", "id": "253e07b9636d0061", "name": "Hallandale Beach", "place_type": "city", "bounding_box": rectangle("-80.207047,25.972852 -80.11721,25.997497") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1228452, "cityName": "Hallandale Beach" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318838985990145, "text": "@OSV https://t.co/Xfomxu0qwJ", "in_reply_to_status": -1, "in_reply_to_user": 16630895, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 16630895 }}, "user": { "id": 4862697011, "name": "Stephen Cooper", "screen_name": "SteveCooperEsq", "lang": "en", "location": "Woodland Hills, Los Angeles", "create_at": date("2016-01-29"), "description": "Writer & Lawyer", "followers_count": 42, "friends_count": 179, "statues_count": 7319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318839128547328, "text": "God sacrificed his own son Jesus who came into our world in human form. I understand the meaning behind \"The Gospel\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1195502532, "name": "JARVO", "screen_name": "Jarvosworld", "lang": "en", "location": "null", "create_at": date("2013-02-18"), "description": "God. Family. Basketball. No weapon formed against me shall prosper", "followers_count": 325, "friends_count": 369, "statues_count": 7580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oskaloosa, IA", "id": "1ab231a01bbda86f", "name": "Oskaloosa", "place_type": "city", "bounding_box": rectangle("-92.690714,41.2563 -92.611063,41.33583") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19123, "countyName": "Mahaska", "cityID": 1959925, "cityName": "Oskaloosa" } }
+{ "create_at": datetime("2016-04-19T00:00:05.000Z"), "id": 722318839141130240, "text": "Wanna tweet a lot, shouldn't tweet a lot, and that's where we're at.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2167809835, "name": "megs", "screen_name": "itsmegmorg", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2013-10-31"), "description": "NAU '19, I like science I guess , Carlos is my fav & my man❤️ ps- are you still reading this?", "followers_count": 353, "friends_count": 783, "statues_count": 10398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-19T00:00:06.000Z"), "id": 722318839715737600, "text": "Looks like an angel, so picturesque, liked she walked right out of the Wild Wild West", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 425438452, "name": "ㄒ₳ㄚㄚ", "screen_name": "TayyShyy", "lang": "en", "location": "Reno, NV", "create_at": date("2011-11-30"), "description": "6 shots of espresso minimum", "followers_count": 430, "friends_count": 786, "statues_count": 18497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681202109521920, "text": "Being bored and wide awake at this time is the worst", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880271992, "name": "Jacob San Miguel", "screen_name": "Thatniggajakobe", "lang": "en", "location": "null", "create_at": date("2014-10-27"), "description": "fuck it", "followers_count": 531, "friends_count": 254, "statues_count": 32601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681202134642688, "text": "The new bo3 dlc barely got on 2 GBs of downloading in 8 hours but downstairs got to 5 GBs in less than 45 mins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191545016, "name": "kitty destroyer", "screen_name": "jedisteven0", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-05-10"), "description": "null", "followers_count": 117, "friends_count": 93, "statues_count": 3227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681202658906113, "text": "Who smoking?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314393933, "name": "#SmokeBettaBlues", "screen_name": "OGSpazz", "lang": "en", "location": "Houston, Texas", "create_at": date("2011-06-09"), "description": "Booking/Inquiries : bookingspazz@gmail.com", "followers_count": 1882, "friends_count": 940, "statues_count": 39151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681203011280896, "text": "Fuck living in a dorm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 800964642, "name": "lorde purdy flacko", "screen_name": "Dpurdy10", "lang": "en", "location": "null", "create_at": date("2012-09-03"), "description": "college makes you want to cry", "followers_count": 1185, "friends_count": 977, "statues_count": 29326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681203283902464, "text": "I like my job. I like my community. My mortgage isn’t soul crushing & most of my people are here.\nGoogle’s just not enough.\n\nTesla might be.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 7982372, "name": "Marcus Ross", "screen_name": "MarcusRoss", "lang": "en", "location": "Omaha, Nebraska USA", "create_at": date("2007-08-05"), "description": "Software Developer (@BCBSNebraska)\nTabletop Game Designer (w/@CaraHeacock - @waterbeargames)\nLibrarian/Spielmason (@Spielbound)\nEV Enthusiast\nAcerbic\nGeek\nHuman", "followers_count": 1093, "friends_count": 1023, "statues_count": 9064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681203636178945, "text": "I swear I sound annoying and whiny in any past message threads I read. I promise it's just the anxiety talking.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1120071284, "name": "caity", "screen_name": "caity_marieS", "lang": "en", "location": "SF. LA.", "create_at": date("2013-01-25"), "description": "outer space is cool. marvel maniac. kid at heart. // 19", "followers_count": 396, "friends_count": 2264, "statues_count": 14464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681204923854848, "text": "4/20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 540437716, "name": "Wiz Colifa", "screen_name": "Based_Col", "lang": "en", "location": "Chiraq", "create_at": date("2012-03-29"), "description": "God | Family | EWU '19 | Chicago |", "followers_count": 945, "friends_count": 549, "statues_count": 22089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681205150322688, "text": "I'm so tired of people just saying what you wanna hear.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2728471859, "name": "Jaden", "screen_name": "jadenkraemer", "lang": "en", "location": "null", "create_at": date("2014-07-29"), "description": "null", "followers_count": 177, "friends_count": 165, "statues_count": 3261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681205666234368, "text": "Wind 0.0 mph ---. Barometer 30.05 in, Falling slowly. Temperature 58.5 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 32138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681205964083200, "text": "My man @AaronSmw knows how to fucking BibleThump https://t.co/eOzjJY3N9W", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 382930740 }}, "user": { "id": 1548856422, "name": "Deuceler", "screen_name": "Deuceler", "lang": "en", "location": "Texas", "create_at": date("2013-06-26"), "description": "speed games / DeuceCon host / pug dad", "followers_count": 666, "friends_count": 391, "statues_count": 8243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasilla, AK", "id": "00549eae15fe11a3", "name": "Wasilla", "place_type": "city", "bounding_box": rectangle("-149.555619,61.552885 -149.358889,61.599832") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 283080, "cityName": "Wasilla" } }
+{ "create_at": datetime("2016-04-20T00:00:00.000Z"), "id": 722681206123454465, "text": "IM TRYNNA GET HIGH AF NOW, FUCK YOU MEAN BITCHHHHHHHHHHHHHHH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 949021070, "name": "FÉRINA", "screen_name": "NathvnVo", "lang": "en", "location": "lost", "create_at": date("2012-11-14"), "description": "sc: pho-king | B00N1ë B0y$", "followers_count": 931, "friends_count": 368, "statues_count": 45906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681206291185664, "text": "you ain't gotta say too much, from the look in your eyes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2299774374, "name": "Frank Underwood", "screen_name": "thebombdotcomm9", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "UH18| snapchat: thebombdotcom9", "followers_count": 1377, "friends_count": 563, "statues_count": 53803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681206735810560, "text": "HAPPY BIRTHDAY!!!! @freshqueenariel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2225490979 }}, "user": { "id": 1077697668, "name": "Jonnie Basketball", "screen_name": "Jonnie007", "lang": "en", "location": "Auburn, WA", "create_at": date("2013-01-10"), "description": "Student-Athlete at THEE Auburn Highschool", "followers_count": 279, "friends_count": 218, "statues_count": 1351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pacific, WA", "id": "2481e36886b27a9b", "name": "Pacific", "place_type": "city", "bounding_box": rectangle("-122.271705,47.257227 -122.223815,47.274297") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5352495, "cityName": "Pacific" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681207113297920, "text": "Temp 37.7° Hi/Lo 41.5/36.8 Rng 4.7° WC 37.7° Hmd 72% Rain 0.00\" Storm 0.00\" BAR 30.118 Rising DP 29.5° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 118, "statues_count": 18036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681207125880833, "text": "Ready to knock these classes out so I can enjoy 4/20 with my niggas. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 308537800, "name": "B'$ UP!", "screen_name": "CONVERSE_NoTlkn", "lang": "en", "location": "The Sip ", "create_at": date("2011-05-31"), "description": "Chelz!! #BLOWGang #BGONG #TGOD Gary Lee Rich 4eva w/ Li B Doing BIG things ❤️ #TheeILove #JSU", "followers_count": 967, "friends_count": 604, "statues_count": 94150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681207352397825, "text": "Blaze it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2806897964, "name": "ReiLord", "screen_name": "brokenNose03", "lang": "en", "location": "Maple Valley, WA", "create_at": date("2014-09-12"), "description": "hey-0,", "followers_count": 51, "friends_count": 110, "statues_count": 863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681207767621633, "text": "Wind 0.0 mph ---. Barometer 30.10 in, Falling slowly. Temperature 66.2 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681207872438272, "text": "The Vitamin Shoppe #Retail #Job: Assistant Store Manager (#BigFlats, NY) https://t.co/97PB5FEAAD #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.8724564,42.1533711"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "BigFlats", "Jobs", "Hiring" }}, "user": { "id": 2362771358, "name": "Vitamin Shoppe Jobs", "screen_name": "VSIcareers", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "At #VitaminShoppe we are lovers of fitness, health, vitamins, helping you on your wellness journey and motivating others. Apply to our #jobs below.", "followers_count": 606, "friends_count": 96, "statues_count": 3461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Flats, NY", "id": "0077f3ae1b34401c", "name": "Big Flats", "place_type": "city", "bounding_box": rectangle("-76.966178,42.127128 -76.860399,42.20027") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36015, "countyName": "Chemung", "cityID": 3606464, "cityName": "Big Flats" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681208195457024, "text": "Staying confused", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207282401, "name": "Jack Wilkins™", "screen_name": "_jack_wilkins_", "lang": "en", "location": "Houston,Texas", "create_at": date("2010-10-24"), "description": "| Sader Varsity Football | Class of 2017 |", "followers_count": 305, "friends_count": 332, "statues_count": 873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681208421945344, "text": "@Zachary_Todd @mcckkaayyllaa not even hungry, next time for sure", "in_reply_to_status": 722680894583128064, "in_reply_to_user": 2240164584, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1018612340, 2850796741 }}, "user": { "id": 2240164584, "name": "Steve", "screen_name": "SteveChahkah", "lang": "en", "location": "Scranton, PA", "create_at": date("2013-12-10"), "description": "beets. bears. battlestar galactica - co owner of Shrute farms", "followers_count": 379, "friends_count": 418, "statues_count": 5396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681208644218881, "text": "SOS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 419809000, "name": "ariel", "screen_name": "_arielaubrey_", "lang": "en", "location": "usu", "create_at": date("2011-11-23"), "description": ":)", "followers_count": 825, "friends_count": 390, "statues_count": 9954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Logan, UT", "id": "e3d31ee0aea2ee7b", "name": "Logan", "place_type": "city", "bounding_box": rectangle("-111.879045,41.679328 -111.779715,41.797389") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49005, "countyName": "Cache", "cityID": 4945860, "cityName": "Logan" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209181048832, "text": "Wind 0.0 mph WSW. Barometer 30.099 in, Steady. Temperature 61.3 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209441161217, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":56.8°F Wind:0.0mph Pressure: 30.12hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 17, "statues_count": 315697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209457917952, "text": "Wind 0.0 mph ---. Barometer 30.010 in, Falling. Temperature 63.2 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209474695168, "text": "Temp: 63.9°F Wind:0.0mph Pressure: 30.060hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209621463041, "text": "Howell, NJ | Wind 1.0 mph NNW. Baro 30.185 in, Rising. Temp 51.1F. Rain today 0.00 in. Humidity 41% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 613, "friends_count": 824, "statues_count": 41038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209709555712, "text": "I need ��, that's my problem..��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290081547, "name": "PLAY", "screen_name": "DJCanShoot_32", "lang": "en", "location": "Jonesboro, NC", "create_at": date("2011-04-29"), "description": "MU ➡️ NCCU ✈️Statesboro #Major2016", "followers_count": 1612, "friends_count": 1577, "statues_count": 44344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Statesboro, GA", "id": "90c543b181a7c56e", "name": "Statesboro", "place_type": "city", "bounding_box": rectangle("-81.869986,32.360463 -81.731171,32.520558") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13031, "countyName": "Bulloch", "cityID": 1373256, "cityName": "Statesboro" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681209927700485, "text": "@lcm1986 Shannon must have had the inning off. It was John Rooney on the call https://t.co/VUVjAnhzNr", "in_reply_to_status": 722680250723868674, "in_reply_to_user": 16531969, "favorite_count": 0, "coordinate": point("-97.39641551,27.6933377"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16531969 }}, "user": { "id": 295390785, "name": "Jeff Aronson", "screen_name": "jamfan40", "lang": "en", "location": "Corpus Christi, TX", "create_at": date("2011-05-08"), "description": "24. Cubs, Bears, Bulls, Blackhawks, Southern Illinois Salukis, Illinois Fighting Illini. I used to read Word Up! Magazine. Follow my alternate @KanyeWest", "followers_count": 1378, "friends_count": 2195, "statues_count": 104142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-04-20T00:00:01.000Z"), "id": 722681210149986305, "text": "Sad cuz I'm reminiscing about how fucking radical my first n second 4/20s were n now this one is gonna be so teeny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1861233842, "name": "satan", "screen_name": "blowjobs4pizza", "lang": "en", "location": "LB, HELL", "create_at": date("2013-09-13"), "description": "premature son of satan // 666 —snapchat:blowjobs4pizza", "followers_count": 682, "friends_count": 432, "statues_count": 24405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681210623913985, "text": "@__tcal HAPPY BIRTHDAY BRUH HAVE A GREAT DAY!��������������", "in_reply_to_status": -1, "in_reply_to_user": 996305052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 996305052 }}, "user": { "id": 1387863552, "name": "FITO⚽️", "screen_name": "arnulfo_garibay", "lang": "en", "location": "null", "create_at": date("2013-04-28"), "description": "soccer is my thing⚽️ senior at mckay", "followers_count": 465, "friends_count": 253, "statues_count": 16375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681210623942658, "text": "fuck it's 12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3910429700, "name": "Vanessa Rivera", "screen_name": "okvanessaaa", "lang": "en", "location": "Gilbert, AZ", "create_at": date("2015-10-15"), "description": "I'm just chilling.", "followers_count": 188, "friends_count": 193, "statues_count": 4314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681210695237632, "text": "Wind 0.0 mph ---. Barometer 30.029 in, Falling. Temperature 58.3 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681210917490689, "text": "Dogs are taking over my home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238009865, "name": "Javi Gold", "screen_name": "javigreenshoes", "lang": "en", "location": "five oh filthy", "create_at": date("2011-01-13"), "description": "We Spaniards know a sickness of the heart that only gold can cure\n|UNM| #MeatLife", "followers_count": 588, "friends_count": 515, "statues_count": 14577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681210959474688, "text": "68.2F (Feels: 68.2F) - Humidity: 95% - Wind: 3.1mph SE - Gust: 5.4mph - Pressure: 1011.5mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 222, "friends_count": 18, "statues_count": 235956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681211739586563, "text": "@ebbtideapp Tide in Prince Point, Maine 04/20/2016\n Low 4:57am 0.4\nHigh 11:07am 9.3\n Low 5:12pm 0.6\nHigh 11:22pm 9.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.1733,43.7617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 76, "friends_count": 1, "statues_count": 25956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yarmouth, ME", "id": "009940e854c4ca59", "name": "Yarmouth", "place_type": "city", "bounding_box": rectangle("-70.247725,43.738287 -70.146746,43.828961") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2387810, "cityName": "Yarmouth" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681211836084224, "text": "04/20@03:00 - Temp 51.3F, WC 51.3F. Wind 1.0mph NW, Gust 3.0mph. Bar 30.228in, Rising slowly. Rain 0.00in. Hum 42%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681212016402433, "text": "Ugh, my bae has a bae ������ https://t.co/XHda711bJx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2952438839, "name": "Meli G.", "screen_name": "MeliGraciela_", "lang": "en", "location": "null", "create_at": date("2014-12-30"), "description": "null", "followers_count": 252, "friends_count": 239, "statues_count": 2932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Coral, FL", "id": "fbd1502e5edce6bf", "name": "Cape Coral", "place_type": "city", "bounding_box": rectangle("-82.069567,26.531814 -81.904101,26.741146") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1210275, "cityName": "Cape Coral" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681212066770944, "text": "I never have much things to tweet about when it comes to personal stuff. I'm happy & content w/ the way my life is & how things are going.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320094664, "name": "Marisa", "screen_name": "MSqueochs20", "lang": "en", "location": "null", "create_at": date("2011-06-19"), "description": "21. Native American. Mother. Medical Assistant.", "followers_count": 567, "friends_count": 366, "statues_count": 22945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681212486184960, "text": "@Daynuh_Dances je suis désolé mon amour", "in_reply_to_status": 722676471521783808, "in_reply_to_user": 623334254, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user_mentions": {{ 623334254 }}, "user": { "id": 173232398, "name": "Savanna Rose Louise", "screen_name": "HonestlySavanna", "lang": "en", "location": "null", "create_at": date("2010-07-31"), "description": "Whatever you're going to do, do it well || Peet's Coffee Barista ☕️", "followers_count": 459, "friends_count": 1324, "statues_count": 45575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681212582641664, "text": "@MrJohnLawton thanks man! Will do.", "in_reply_to_status": 722591621414547457, "in_reply_to_user": 520539828, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 520539828 }}, "user": { "id": 520133783, "name": "Paul Guglielmo", "screen_name": "PaulyGoogs", "lang": "en", "location": "Rochester, NY/Conneaut, OH", "create_at": date("2012-03-09"), "description": "Radio guy. Sauce maker. Engaged to @RyannEliz.", "followers_count": 2157, "friends_count": 151, "statues_count": 11703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brighton, NY", "id": "94779fa8bb1f79a8", "name": "Brighton", "place_type": "city", "bounding_box": rectangle("-77.694219,43.087472 -77.516194,43.167065") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608257, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681212851068929, "text": "@gerraarrdo I swore this said nudes", "in_reply_to_status": 722680981988253697, "in_reply_to_user": 3251414204, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3251414204 }}, "user": { "id": 2454303750, "name": "Daniela", "screen_name": "wtvrdaniela", "lang": "en-gb", "location": "S.T.A.R. Labs ", "create_at": date("2014-04-19"), "description": "Peruvian Mountain Princess //64", "followers_count": 405, "friends_count": 321, "statues_count": 6719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Jose Hills, CA", "id": "945d4c47ec5036f9", "name": "South San Jose Hills", "place_type": "city", "bounding_box": rectangle("-117.923259,34.003576 -117.88006,34.026435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673290, "cityName": "South San Jose Hills" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681213195055104, "text": "Mood https://t.co/qL6bCboIlx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1048928491, "name": "Sarahi ✨", "screen_name": "YessicaPerez_", "lang": "en", "location": "Houston, Texas", "create_at": date("2012-12-30"), "description": "| I Dance | 16 | IG : sarahihtx | Sc : sarahilovesroda c:", "followers_count": 383, "friends_count": 914, "statues_count": 4330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681213648023554, "text": "Wind 1.6 mph WNW. Barometer 30.20 in, Steady. Temperature 43.9 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 159194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681214063243264, "text": "BADDEST BITCH ON THE BLOCK & HER EYEBROWS ARE FLEEKY 24/7 ������ https://t.co/OiQyJkVdsh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1260427711, "name": "jess", "screen_name": "jsca1057", "lang": "en", "location": "Petty Island", "create_at": date("2013-03-11"), "description": "NAU | biomed | pre-med", "followers_count": 1269, "friends_count": 489, "statues_count": 44610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-20T00:00:02.000Z"), "id": 722681214289711104, "text": "Wind 2.0 mph NW. Barometer 30.265 in, Steady. Temperature 41.1 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 9, "friends_count": 44, "statues_count": 7459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681214902083584, "text": "420 glaze it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2298756242, "name": "G", "screen_name": "Steven8th", "lang": "en", "location": "Chino, CA", "create_at": date("2014-01-18"), "description": "Im the seventh letter in the alphabet.", "followers_count": 183, "friends_count": 252, "statues_count": 1382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681214944088065, "text": "Ripley SW Limestone Co. Temp: 61.2°F Wind:0.0mph Pressure: 998.6mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681215048904704, "text": "Happy birthday have a great day see you Sunday ���� @Breeflores15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 438594801 }}, "user": { "id": 2523686015, "name": "Nik", "screen_name": "_nik04_", "lang": "en", "location": "null", "create_at": date("2014-05-02"), "description": "Sucka Free for Life", "followers_count": 340, "friends_count": 313, "statues_count": 3337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681215061504009, "text": "yoooo it's 420", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2558180778, "name": "bella prieto", "screen_name": "bella__prieto", "lang": "en", "location": "null", "create_at": date("2014-06-09"), "description": "lucys '19 • sc bella_prieto", "followers_count": 450, "friends_count": 392, "statues_count": 3119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Claremont, CA", "id": "dbd7fea9eedaecd0", "name": "Claremont", "place_type": "city", "bounding_box": rectangle("-117.750807,34.079477 -117.681539,34.158556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 613756, "cityName": "Claremont" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681215506112512, "text": "Ya private boat rides are nice and all but I lost my eyelashes and 2 nails so fuck u.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323469506, "name": "jess lister.", "screen_name": "jlis16", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "I feel her filth in my bones. -the lumineers.", "followers_count": 915, "friends_count": 289, "statues_count": 29041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681215787130880, "text": "Need more BBC for the summer.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45050830, "name": "最高", "screen_name": "miahsito", "lang": "en", "location": "midgar.", "create_at": date("2009-06-05"), "description": "love | peace | prosperity | chaos | rage . university of south alabama .", "followers_count": 1984, "friends_count": 1595, "statues_count": 113213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681216877637632, "text": "I'm not lol. But super relieved https://t.co/UQAhTHFz0X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3687613694, "name": "Rich Porter $", "screen_name": "punny_lava", "lang": "en", "location": "null", "create_at": date("2015-09-25"), "description": "had a good snooze?", "followers_count": 94, "friends_count": 90, "statues_count": 367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Acres, CA", "id": "01f76332643a43c6", "name": "Garden Acres", "place_type": "city", "bounding_box": rectangle("-121.256401,37.941173 -121.167581,38.01443") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 628182, "cityName": "Garden Acres" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681218127499264, "text": "@oomf https://t.co/LuopQeW8fW", "in_reply_to_status": -1, "in_reply_to_user": 624964580, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 624964580 }}, "user": { "id": 3150778381, "name": "Deandra Cisneros", "screen_name": "deeanndruhhh", "lang": "en", "location": "Kingsville, TX", "create_at": date("2015-04-11"), "description": "~ sassy | snapchat: princess.deee | good vibessss ~", "followers_count": 588, "friends_count": 1152, "statues_count": 3539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681218177875969, "text": "Happy Birthday Hitler!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 947124180, "name": "Sean Downey", "screen_name": "theseandowney", "lang": "en", "location": "Whittier", "create_at": date("2012-11-13"), "description": "Now that I've found all the answers, I realize that what I was living for were the questions! (I'm an asshole)", "followers_count": 302, "friends_count": 438, "statues_count": 2129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681218228166657, "text": "Today I start a long journey into a series of drunken mistakes and future alcoholic rehabilitation �� #finally21", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "finally21" }}, "user": { "id": 524879555, "name": "TZA", "screen_name": "_Tezzy23", "lang": "en", "location": "Memphis, TN", "create_at": date("2012-03-14"), "description": "〽\nAmos: Antnply63 \n\nInstagram: TZA__23\n\n*Follow me on Soundcloud*", "followers_count": 455, "friends_count": 559, "statues_count": 6503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millington, TN", "id": "017c94be7f336363", "name": "Millington", "place_type": "city", "bounding_box": rectangle("-89.950564,35.263115 -89.824818,35.373704") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4749060, "cityName": "Millington" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681218278518785, "text": "That gal is still defending herself and Shaw. Wow. You have a real gem #Blackhawks fans. https://t.co/Lp6zoaD1NQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Blackhawks" }}, "user": { "id": 274329154, "name": "Bob Dobalina", "screen_name": "outonthepatio", "lang": "en", "location": "Inner Loop - Space City - USA", "create_at": date("2011-03-29"), "description": "I help produce sports on your TV. I like to travel and drink beer, but I drink more beer than I travel. Married to @Jae_Bee76, but #WayneTrain is my man crush.", "followers_count": 2153, "friends_count": 2295, "statues_count": 75218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:03.000Z"), "id": 722681218404368384, "text": "Happy 420 guys ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 196412503, "name": "Paraguay", "screen_name": "LeStoner_", "lang": "en", "location": "Gotham City Hoodlum", "create_at": date("2010-09-28"), "description": "when nothing goes right, make a left", "followers_count": 1130, "friends_count": 1962, "statues_count": 96117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681218983182337, "text": "Just went 34-9. You can say @jollyrichardLP and I are getting back into the groove of Battlefront! @EA @starwars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 78116815, 15234657, 20106852 }}, "user": { "id": 59720180, "name": "Cody Wilson", "screen_name": "CodyTheTank91", "lang": "en", "location": "Hot Springs, Arkansas", "create_at": date("2009-07-24"), "description": "Comic Lover. Gamer. Wannabe Writer. Science Lover. I Sell Video Games. Basketball/Soccer/Gaming Obsessed. Reading is a Drug that I'm addicted to.", "followers_count": 226, "friends_count": 337, "statues_count": 6382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hot Springs, AR", "id": "208f4f6ad65bb7c9", "name": "Hot Springs", "place_type": "city", "bounding_box": rectangle("-93.12832,34.434369 -92.914247,34.538694") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5051, "countyName": "Garland", "cityID": 533400, "cityName": "Hot Springs" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681219142524929, "text": "420 BLAZE IT FAGGOT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 962967806, "name": "pupper", "screen_name": "GrowlerPincher", "lang": "en", "location": "Anaheim, CA", "create_at": date("2012-11-21"), "description": "I own Duncan Construction", "followers_count": 181, "friends_count": 184, "statues_count": 53173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681219310346240, "text": "and at the end of the day, I'll still always love you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1014265304, "name": "maddie osborne ❂", "screen_name": "mlo_04", "lang": "en", "location": "null", "create_at": date("2012-12-15"), "description": "kind words are a honeycomb. sweet to the soul and healing for the body. proverbs 16:24", "followers_count": 500, "friends_count": 302, "statues_count": 4528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southside, AL", "id": "0060a15f2aabf113", "name": "Southside", "place_type": "city", "bounding_box": rectangle("-86.070513,33.852599 -85.985964,33.946651") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1055, "countyName": "Etowah", "cityID": 171832, "cityName": "Southside" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681219666821121, "text": "@imHALFof_Hilan ��", "in_reply_to_status": 722681062099435520, "in_reply_to_user": 280114575, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 280114575 }}, "user": { "id": 278823801, "name": "4/23♉️", "screen_name": "Traeskii__", "lang": "en", "location": "round the way 4️⃣0️⃣0️⃣", "create_at": date("2011-04-07"), "description": "get high to balance my lows ... sc: traepound19", "followers_count": 1372, "friends_count": 1308, "statues_count": 92992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681219671027712, "text": "4/20 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2287831153, "name": "jammin jamin", "screen_name": "jamintheasian", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-12"), "description": "Beast Coast // LA // NY // Street Wear // Pop Punk Let's Explore the Unknown // 98 'Til Infinity", "followers_count": 533, "friends_count": 497, "statues_count": 14234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Puente, CA", "id": "e04653ebc39b100c", "name": "La Puente", "place_type": "city", "bounding_box": rectangle("-117.98558,34.009223 -117.920899,34.057298") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640340, "cityName": "La Puente" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681219708764160, "text": "Wind 3.2 mph E. Barometer 29.900 in, Falling. Temperature 66.0 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681220035948548, "text": "it's time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460786767, "name": "nathan", "screen_name": "pnwnd", "lang": "en", "location": "541", "create_at": date("2012-01-10"), "description": "17", "followers_count": 1217, "friends_count": 689, "statues_count": 25538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681220493144064, "text": "Or at least try ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1629563678, "name": "Dennise", "screen_name": "_denniiiise", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "null", "followers_count": 231, "friends_count": 156, "statues_count": 8763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681220711215104, "text": "What I would give, to not go back in there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542119755, "name": "brushhhhhh", "screen_name": "Razzable", "lang": "en", "location": "Portland, OR", "create_at": date("2012-03-31"), "description": "Moderately Informed : Unreasonably Optimistic", "followers_count": 112, "friends_count": 112, "statues_count": 17636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681220920918018, "text": "Close eyes, see things", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3101876474, "name": "jose", "screen_name": "jose_0901_", "lang": "en", "location": "null", "create_at": date("2015-03-21"), "description": "kfhs", "followers_count": 133, "friends_count": 100, "statues_count": 4067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681221336145920, "text": "������ #bluesky #dayoff #fun �� @ Palm Springs, California https://t.co/3ULHqTrWAU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.53,33.824"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bluesky", "dayoff", "fun" }}, "user": { "id": 340684466, "name": "⊂∂яL os", "screen_name": "Caarloz83", "lang": "en", "location": "Palm Springs ca. ", "create_at": date("2011-07-22"), "description": "●๋•:@monicanaranjo #Minage #Sushi #Whiskey #Palmsprings #KarloOfarrell http://Instagram.com/caarloz83", "followers_count": 2873, "friends_count": 2843, "statues_count": 71052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.567788,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681222338592769, "text": "Temp 50.7°F Wind Chill 50.7°F RH 81% Wind 0.0 --- Gust 0.0 --- SLP 30.120 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 107, "friends_count": 63, "statues_count": 34836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681222388912129, "text": "@CrispenJessy3 okay do that", "in_reply_to_status": 722681038435151878, "in_reply_to_user": 1885026907, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1885026907 }}, "user": { "id": 1009016924, "name": "alexis taylor.", "screen_name": "lexxbrooke20", "lang": "en", "location": "Texas, USA", "create_at": date("2012-12-13"), "description": "null", "followers_count": 548, "friends_count": 318, "statues_count": 8897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownfield, TX", "id": "f7bb6752fbd343e9", "name": "Brownfield", "place_type": "city", "bounding_box": rectangle("-102.309211,33.146988 -102.244024,33.202964") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48445, "countyName": "Terry", "cityID": 4810720, "cityName": "Brownfield" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681222426705920, "text": "the real reason people like thick thighs https://t.co/Ca4kKIoo16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 869004372, "name": "livvay", "screen_name": "olivia_darragh", "lang": "en", "location": "with y(our) man", "create_at": date("2012-10-08"), "description": "emancipate yourself from mental slavery// you will have freed your mind", "followers_count": 150, "friends_count": 243, "statues_count": 2381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Collins, CO", "id": "b2e4e65d7b80d2c1", "name": "Fort Collins", "place_type": "city", "bounding_box": rectangle("-105.148074,40.47168 -104.979811,40.656701") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 827425, "cityName": "Fort Collins" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681222846160896, "text": "https://t.co/DgoFLNdLGU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 166685041, "name": "Lindsey Hughes", "screen_name": "beautybaby44", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-07-14"), "description": "Lindsey | Youtube vlogger | 21 years young | fucking obsessed with @bassnectar | livin with love | Snapchat: beautybaby | Instagram: livinlikelindsey", "followers_count": 253085, "friends_count": 375, "statues_count": 29928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681222909059074, "text": "@xomarleigh LMAOOO you'll judge hard", "in_reply_to_status": 722681164343996417, "in_reply_to_user": 285974025, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 285974025 }}, "user": { "id": 438793524, "name": "brett", "screen_name": "GoIdenSkans", "lang": "en", "location": "Illinois ", "create_at": date("2011-12-16"), "description": "hey dudes are you ready to", "followers_count": 1761, "friends_count": 632, "statues_count": 82263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, IL", "id": "4117469e512e486a", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-89.62277,40.628142 -89.569333,40.661063") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17179, "countyName": "Tazewell", "cityID": 1717549, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-20T00:00:04.000Z"), "id": 722681223030681600, "text": "I used to be so tan:( https://t.co/tOrK7uZIHu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 159233825, "name": "Dani HomeBaker", "screen_name": "Mask12_1", "lang": "en", "location": "watching blackhawks, NJN", "create_at": date("2010-06-24"), "description": "the man in my header is my home. being in college is great because everyone leaves you alone. aka don't fwm thnx xoxo LDS", "followers_count": 919, "friends_count": 1634, "statues_count": 46223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681223227785217, "text": "so blessed to have such a great partner by my side ✨✨✨✨��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3113072361, "name": "Perla", "screen_name": "peaarls__", "lang": "en", "location": "null", "create_at": date("2015-03-24"), "description": "blessed w the best", "followers_count": 112, "friends_count": 53, "statues_count": 3587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681224284790784, "text": "Making pizza rolls at 3 in the morning bc why the hell not ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272518084, "name": "Capricia Gant ✝", "screen_name": "primo110696", "lang": "en", "location": "null", "create_at": date("2014-01-01"), "description": "We don't remember days, we remember moments ✴\nBest friend is Emily Kay ✊", "followers_count": 842, "friends_count": 2160, "statues_count": 11685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southfield, MI", "id": "ed8334ef408add50", "name": "Southfield", "place_type": "city", "bounding_box": rectangle("-83.319904,42.442178 -83.200644,42.517441") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2674900, "cityName": "Southfield" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681224561618946, "text": "Happy birthday ���������������� @DorahBugg turnuh https://t.co/q1HXhf74fW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 399715320 }}, "user": { "id": 516309360, "name": "✨April 18th✨", "screen_name": "RancyBalboa", "lang": "en", "location": "null", "create_at": date("2012-03-05"), "description": "You Get A Thousand Things Everyday Just Make Sure This Is One Of Them ✨✨✨✨Snapchat:RancyBalboa|✨✨", "followers_count": 869, "friends_count": 960, "statues_count": 18632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681224658096129, "text": "@kcdyer is it a humorous romantic story", "in_reply_to_status": 722567990223482880, "in_reply_to_user": 18090941, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18090941 }}, "user": { "id": 24623450, "name": "Fern Manor", "screen_name": "FManor75", "lang": "en", "location": "Long Island", "create_at": date("2009-03-15"), "description": "Physical Therapist private practice, Beta reader, ARC reader & reviewer of awesome books by amazing authors.", "followers_count": 537, "friends_count": 777, "statues_count": 18844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roslyn Heights, NY", "id": "0d12de399d3f8b55", "name": "Roslyn Heights", "place_type": "city", "bounding_box": rectangle("-73.658206,40.759648 -73.623811,40.793575") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3663814, "cityName": "Roslyn Heights" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681225161347073, "text": "@veevutton will do ����", "in_reply_to_status": 722680457784139776, "in_reply_to_user": 335850878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335850878 }}, "user": { "id": 340117925, "name": "Leblanc ✈️", "screen_name": "LeblancMusic_", "lang": "en", "location": "Kenner, LA", "create_at": date("2011-07-21"), "description": "Features/Bookings ➡️ leblancmusic@yahoo.com | #Itgetsbetter #IGB!", "followers_count": 16575, "friends_count": 2439, "statues_count": 24144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681225266249731, "text": "@cagednephlyn @NCIS_CBS they mentioned delilah I was hoping they wouldn't", "in_reply_to_status": 722530470550970371, "in_reply_to_user": 2755553739, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2755553739, 68795677 }}, "user": { "id": 275378900, "name": "Robert L E", "screen_name": "vegasdude83", "lang": "en", "location": "null", "create_at": date("2011-03-31"), "description": "it is the duty of every citizen to challenge their leaders,to keep them honest & hold them accountable when they aren't-Ahsoka Tano", "followers_count": 1364, "friends_count": 1595, "statues_count": 144090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681226000277504, "text": "@_shanataafuaaaa ��", "in_reply_to_status": 722677921211674624, "in_reply_to_user": 2992460870, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2992460870 }}, "user": { "id": 2697361940, "name": "EARTHLY♻️", "screen_name": "earthlydevon", "lang": "en", "location": "Belize to Long Beach Ca", "create_at": date("2014-07-31"), "description": "#earthlythemovement ♻️", "followers_count": 1836, "friends_count": 1270, "statues_count": 26700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681226549661697, "text": "@Baldobinoo I'm free the next 3 days ...", "in_reply_to_status": 722680816715870208, "in_reply_to_user": 2355940410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2355940410 }}, "user": { "id": 2820743040, "name": "Celeste", "screen_name": "celestea__", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-09-19"), "description": "null", "followers_count": 245, "friends_count": 260, "statues_count": 5688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681226692272129, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 621655024, "name": "tiffy", "screen_name": "tofurose", "lang": "en", "location": "null", "create_at": date("2012-06-29"), "description": "flash a-ah", "followers_count": 1092, "friends_count": 434, "statues_count": 9092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681226830749696, "text": "I used to have a lot of ride or dies,, now we bygones ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1343402844, "name": "KayKay2Timess", "screen_name": "KaylaMarleyBudd", "lang": "en", "location": "Naperville, IL", "create_at": date("2013-04-10"), "description": "Tell me what your life likee. KayKay2Times on Instagram !", "followers_count": 227, "friends_count": 71, "statues_count": 9063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Naperville, IL", "id": "3227bcceb76b6a37", "name": "Naperville", "place_type": "city", "bounding_box": rectangle("-88.259598,41.665584 -88.063045,41.831408") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1751622, "cityName": "Naperville" } }
+{ "create_at": datetime("2016-04-20T00:00:05.000Z"), "id": 722681226872635397, "text": "4/20", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2571746839, "name": "alexis galindo", "screen_name": "alexisgalindoo", "lang": "en", "location": "insta: alexissgalindoo", "create_at": date("2014-06-16"), "description": "sc: alexisgalindoo", "followers_count": 1035, "friends_count": 435, "statues_count": 14157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681227279523840, "text": "Can @CurrenSy_Spitta match one for 420 Htx ��☹", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20851642 }}, "user": { "id": 340039837, "name": "MachineGun Murda", "screen_name": "HOUlalalalalife", "lang": "en", "location": "✈ Go shawty this yo world ", "create_at": date("2011-07-21"), "description": "IG: Her.84gmc", "followers_count": 1395, "friends_count": 954, "statues_count": 59193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681227665358848, "text": "It's my middle name https://t.co/AIzgtFd6KY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2510740661, "name": "sc: pilar.sims28", "screen_name": "Sadeee_xo", "lang": "en", "location": "Charleston, WV ", "create_at": date("2014-04-26"), "description": "No Bitch Intimidates me, sorry.", "followers_count": 2453, "friends_count": 962, "statues_count": 47799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, WV", "id": "71f2805dd75bc147", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-81.707175,38.293241 -81.560235,38.401731") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54039, "countyName": "Kanawha", "cityID": 5414600, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681227736662017, "text": "@LouannTexas Be safe!", "in_reply_to_status": 722680774307266560, "in_reply_to_user": 236170649, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236170649 }}, "user": { "id": 97248301, "name": "HarveyDent's Mama", "screen_name": "ladyleesa", "lang": "en", "location": "Englewood, CO", "create_at": date("2009-12-16"), "description": "END BSL! PRO NO-KILL. Mama of my 4 legged son, Harvey Dent. ANIMAL RIGHTS/DEATHROW DOG RT'r! Proud Lakota woman. ✌#AdoptDontShop #StopYulinForever", "followers_count": 858, "friends_count": 1087, "statues_count": 16038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englewood, CO", "id": "7fdcecb477b5ddd7", "name": "Englewood", "place_type": "city", "bounding_box": rectangle("-105.03495,39.616584 -104.95943,39.678686") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 824785, "cityName": "Englewood" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681227766075392, "text": "YESssssss https://t.co/ux8YSC9XBp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 459550480, "name": "Aquarius", "screen_name": "CarriDinsmore", "lang": "en", "location": "Cincinnati Ohio", "create_at": date("2012-01-09"), "description": "null", "followers_count": 1368, "friends_count": 1244, "statues_count": 22282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosamond, CA", "id": "6c6677c9b39c3c6d", "name": "Rosamond", "place_type": "city", "bounding_box": rectangle("-118.242397,34.820596 -118.142104,34.88272") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 662826, "cityName": "Rosamond" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681227950600192, "text": "@socalmike_SD @stompcure @Cernovich then don't support a man whose whole business career and four bankruptcies are about crony-Crapitalism", "in_reply_to_status": 722680861372653569, "in_reply_to_user": 832886564, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 832886564, 2816371, 358545917 }}, "user": { "id": 261362196, "name": "JimWilemon", "screen_name": "JimWilemon", "lang": "en", "location": "SOUTHWEST-USA", "create_at": date("2011-03-05"), "description": "Constitutionalist..Classical Liberal (look it up)..1st ID/4th CAV(67-68)..Retired Radar Air Traffic Controller..Afghanistan ATC Contractor 05/06", "followers_count": 667, "friends_count": 738, "statues_count": 4722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Fe, NM", "id": "ebdd9f30d3abfc43", "name": "Santa Fe", "place_type": "city", "bounding_box": rectangle("-106.10308,35.586659 -105.895133,35.726444") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35049, "countyName": "Santa Fe", "cityID": 3570500, "cityName": "Santa Fe" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681228697169922, "text": "Wo liegt Phoenix? https://t.co/i6ulLa5cnW #Phoenix #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.074,33.4483"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Phoenix", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 464, "friends_count": 116, "statues_count": 1909799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681229615755264, "text": "Temp: 45.9°F - Dew Point: 44.6° - Wind: ESE @ 7.1 mph - Gust: 10.7 - Rain Today: 0.00in. - Pressure: 30.15in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 11, "statues_count": 16507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681230072934400, "text": "forever trying to turn my life around and do better but I'm always being tested�� #helpmeloooooordt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "helpmeloooooordt" }}, "user": { "id": 260889098, "name": "QUEEN DILLY", "screen_name": "SimplyDillan", "lang": "en", "location": "null", "create_at": date("2011-03-04"), "description": "relax ya life", "followers_count": 1469, "friends_count": 596, "statues_count": 84665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681230395879424, "text": "Wind 3.0 mph E. Barometer 30.203 in, Steady. Temperature 51.1 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 391, "friends_count": 292, "statues_count": 9931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681230609801218, "text": "4/20/2016 - 02:00\nTemp: 60.0F \nHum: 99%\nWind: 0.0 mph\nBaro: 30.017in. & Falling\nRain: 0.00 in.\nhttps://t.co/6r89Idko4M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 51948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-04-20T00:00:06.000Z"), "id": 722681231360581632, "text": "Time to study for gov!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 738046495, "name": "Hadison Motchkiss", "screen_name": "madhotchkiss", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-08-04"), "description": "SCHS, Stephen Curry high school (; #dubnation #districtswagcali", "followers_count": 544, "friends_count": 508, "statues_count": 12651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-20T00:00:07.000Z"), "id": 722681232040071168, "text": "\"She got money, huh?\" https://t.co/Y9XivQbaLg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2594146814, "name": "E. Boogie", "screen_name": "itsEricbruhh", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2014-06-28"), "description": "Twitter Feminists are NOT welcomed in my mentions", "followers_count": 736, "friends_count": 430, "statues_count": 49144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-20T00:00:07.000Z"), "id": 722681232073617409, "text": "@BurtLevine James Partsch-Galvan the ex-spouse of Richard William Partsch and Green Party Candidate for US Congress Texas 29th", "in_reply_to_status": 722674379168079872, "in_reply_to_user": 442725335, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 442725335 }}, "user": { "id": 237438306, "name": "James Partsch-Galvan", "screen_name": "MayorGalvan2011", "lang": "en", "location": "Houston/Galveston, Texas USA", "create_at": date("2011-01-12"), "description": "The time has come to galvanize ! http://www.mayorgalvan.com", "followers_count": 233, "friends_count": 507, "statues_count": 2899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-20T00:00:07.000Z"), "id": 722681232639815680, "text": "Wind 0.0 mph ---. Barometer 29.859 in, Steady. Temperature 57.5 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 269702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-20T00:00:07.000Z"), "id": 722681233541623808, "text": "every hoe, cheater, trollup and slow texter pay rent at bayou oaks. they prolly even room together. https://t.co/ispQjpbq5L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341312888, "name": "23 de abril ✨.", "screen_name": "brimarie423", "lang": "en", "location": "nola ➡️ nacghanistan ↔️ h-town", "create_at": date("2011-07-23"), "description": "bougie on a budget. a perpetually petty person. part-time twitter activist. university of houston. rest in paradise marie. #blacklivesmatter #UH18", "followers_count": 1036, "friends_count": 518, "statues_count": 99012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043589991751680, "text": "[frantically searches through contacts to spread the news about Chyna]", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3154758212, "name": "albert", "screen_name": "alaterbloom", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-04-13"), "description": "null", "followers_count": 102, "friends_count": 83, "statues_count": 117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043590348234753, "text": "@tana2turnt @bfsmith45 me", "in_reply_to_status": 696920054110273538, "in_reply_to_user": 2995976598, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2995976598, 2327599452 }}, "user": { "id": 38748512, "name": "jaycie", "screen_name": "jayciekitty", "lang": "en", "location": "portland", "create_at": date("2009-05-08"), "description": "whs '16 / uo '20", "followers_count": 372, "friends_count": 379, "statues_count": 4597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043590365011970, "text": "I wanna follow more people on snap ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2241235975, "name": "Cubaña", "screen_name": "Vickkay__", "lang": "en", "location": "Miami/Cuba", "create_at": date("2013-12-11"), "description": "#GOLDEN; Terrance ❣", "followers_count": 572, "friends_count": 418, "statues_count": 14237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indian Head, MD", "id": "01b43c60341ab52b", "name": "Indian Head", "place_type": "city", "bounding_box": rectangle("-77.210341,38.561964 -77.137122,38.608403") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24017, "countyName": "Charles", "cityID": 2441500, "cityName": "Indian Head" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043590482460672, "text": "Thank you to everybody who said Happy Birthday ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1925604145, "name": "M A R V I N", "screen_name": "MarvinH_000", "lang": "en", "location": "#YayArea", "create_at": date("2013-10-01"), "description": "Life is never gonna go the way you want it...", "followers_count": 309, "friends_count": 272, "statues_count": 11198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043590784446465, "text": "Yall add me on sc: j.gerald10\n or drop yours��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321819183, "name": "Gerald Jalipa", "screen_name": "GJALIPA_10", "lang": "en", "location": "Carson, California", "create_at": date("2011-06-21"), "description": "#10 From Carson", "followers_count": 255, "friends_count": 223, "statues_count": 3093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043591615078400, "text": "I woke up out of no where thinking its time to get ready for school ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3028202490, "name": "malaysia❤️✨", "screen_name": "Deeezyx1", "lang": "en", "location": "Trenton, NJ", "create_at": date("2015-02-18"), "description": "sc: deeezyx1", "followers_count": 369, "friends_count": 353, "statues_count": 3087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Trenton, NJ", "id": "74767081120f9b42", "name": "Trenton", "place_type": "city", "bounding_box": rectangle("-74.817535,40.183967 -74.729138,40.248298") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34021, "countyName": "Mercer", "cityID": 3474000, "cityName": "Trenton" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043592348962816, "text": "Como chingas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1691266512, "name": "anahi", "screen_name": "Anahiii_mtz", "lang": "en", "location": "jan'06❤️", "create_at": date("2013-08-22"), "description": "null", "followers_count": 790, "friends_count": 450, "statues_count": 11196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043592718032896, "text": "Jane the v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75696479, "name": "Veronica Miranda", "screen_name": "saayveroh", "lang": "en", "location": "Mid-City, Los Angeles", "create_at": date("2009-09-19"), "description": "Natrual Redhead | 20 | CRomero |", "followers_count": 438, "friends_count": 381, "statues_count": 21997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043592969838592, "text": "Jackson or no jackson dave ur still tha man #HolyWars(ThePunishmentDue) https://t.co/NcYCi1qEmo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HolyWars" }}, "user": { "id": 2463706598, "name": "JESUS➕", "screen_name": "JESUS_6_KALING", "lang": "en", "location": "i am within your heart", "create_at": date("2014-04-25"), "description": "YOUR SAVIOR! Followed by model @NikiTaylorInc #storagewars @MissyElliott @mrchuckd @nancysinatra \nI love the human heart & a special lady named @mindykaling", "followers_count": 1373, "friends_count": 2482, "statues_count": 41633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043593452052480, "text": "Amazing Re-MIX of @JohnnyCash song. https://t.co/kh8Xh9DZV8 Watch it. :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 546941942 }}, "user": { "id": 46028446, "name": "Elliot Gray VanOrman", "screen_name": "ElliotVanOrman", "lang": "en", "location": "Detroit,Michigan.USA:TWDFamily", "create_at": date("2009-06-09"), "description": "30-Bipolar-Filmmaker-CEO/Owner of TWDFamily: http://t.co/dJNUTdjvmN http://t.co/AZzgOoVw44 http://t.co/0Q3MVQO06d http://t.co/7wiiCVkk1E http://t.co/HE8KBlAUB1", "followers_count": 750, "friends_count": 438, "statues_count": 9178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-04-21T00:00:00.000Z"), "id": 723043593603022848, "text": "O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3246514496, "name": "7|2♋️", "screen_name": "FuckYaTWEET_", "lang": "en", "location": "BigMoney Tx ", "create_at": date("2015-06-15"), "description": "7.30.15 ❣", "followers_count": 582, "friends_count": 538, "statues_count": 2836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043595079426048, "text": "@AaronJones4Real orale homes", "in_reply_to_status": 723043418016903170, "in_reply_to_user": 3260794502, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3260794502 }}, "user": { "id": 879447702, "name": "Ramon", "screen_name": "tall_ramon", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2012-10-13"), "description": "don't wait on it\nCeres➡Santa Barbara", "followers_count": 622, "friends_count": 571, "statues_count": 15378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Barbara, CA", "id": "f6ebc676e5cde864", "name": "Santa Barbara", "place_type": "city", "bounding_box": rectangle("-119.763341,34.395525 -119.639931,34.464245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669070, "cityName": "Santa Barbara" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043595213656065, "text": "Miss my nephew ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232379905, "name": "Drizzy On The Line♎️", "screen_name": "drizzybethename", "lang": "en", "location": "Kansas City, MO", "create_at": date("2010-12-30"), "description": "Est 93' ♎️. 10/20. Jesus is the Reason. #KCistheTown. Dreamville. i'm mad cool. I go by Dre. 1906", "followers_count": 1237, "friends_count": 1282, "statues_count": 35730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043595570180097, "text": "@villanuevac con el tiempo te darás cuenta que fue lo mejor", "in_reply_to_status": 723041073220571136, "in_reply_to_user": 84209502, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 84209502 }}, "user": { "id": 116261443, "name": "Andy Saldívar", "screen_name": "AndresSaldivar", "lang": "es", "location": "Arizona, USA", "create_at": date("2010-02-21"), "description": "Soy como los hombres del norte grandote y buenote. http://Instagram.com/andresitosaldivar", "followers_count": 1159, "friends_count": 1902, "statues_count": 23393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043596224614402, "text": "04/21@03:00 - Temp 44.2F, WC 44.2F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.231in, Falling slowly. Rain 0.00in. Hum 81%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043596241227776, "text": "My lip keeps burning and I keep forgetting there's a wad of sriacha on it ;(((((", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341842058, "name": "Liz Newcomer", "screen_name": "liznewcomer", "lang": "en", "location": "CA✈️WA", "create_at": date("2011-07-24"), "description": "wΑshingtΟΠ", "followers_count": 1433, "friends_count": 360, "statues_count": 6538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043596312698880, "text": "Wind 0.0 mph SSW. Barometer 29.879 in, Falling. Temperature 61.0 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043596384014336, "text": "Goodnight for now. The grind starts again tomorrow ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284485241, "name": "Curtis_Woods", "screen_name": "curt_woods", "lang": "en", "location": "#EATGREEDY", "create_at": date("2011-04-19"), "description": "☝God #RIPSilver ❤ Miller Grove Alumni #UWGFOOTBALL #DekalbRaised", "followers_count": 476, "friends_count": 406, "statues_count": 13101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollton, GA", "id": "7bc33682b217b5a1", "name": "Carrollton", "place_type": "city", "bounding_box": rectangle("-85.128063,33.52676 -85.009976,33.633403") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13045, "countyName": "Carroll", "cityID": 1313492, "cityName": "Carrollton" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043596623077376, "text": "- مرات يوم أكون مضايقه ماحب اشكي الا بالانقليزي ، ماعرف اذا هالحاله طبيعيه ولا (: !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 1072562017, "name": "فطيمالبرشلونيّه", "screen_name": "iFbm__", "lang": "en", "location": "Houston, TX", "create_at": date("2013-01-08"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀ .كتاباتي بالمفضله - واصرف عن أمي شرّ مآ قضيت", "followers_count": 557, "friends_count": 205, "statues_count": 30780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043597315137536, "text": "Wind 0.7 mph WSW. Barometer 29.941 in, Steady. Temperature 64.9 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043597533208577, "text": "Wind 0.0 mph ---. Barometer 29.918 in, Falling slowly. Temperature 63.7 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043597654888448, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":69.1°F Wind:0.0mph Pressure: 30.00hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 315788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043597776375808, "text": "@EdelizPerez Kim, people are dying.", "in_reply_to_status": 720749172677160961, "in_reply_to_user": 1069917325, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1069917325 }}, "user": { "id": 4764291078, "name": "Alyssa why Saenz", "screen_name": "saenzalyssaa", "lang": "en", "location": "Abilene, TX", "create_at": date("2016-01-15"), "description": "20 | | ACU Theatre | | Unqualified Air Pilot \nHost of a podcast for @acutheatre... link below!!", "followers_count": 95, "friends_count": 270, "statues_count": 217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043598036410368, "text": "Last minute blaze", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3185789862, "name": "dal pal", "screen_name": "doctordallin", "lang": "en", "location": "sleeping", "create_at": date("2015-05-04"), "description": "I like to wear maroon @dallintphoto", "followers_count": 262, "friends_count": 213, "statues_count": 2785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-21T00:00:01.000Z"), "id": 723043598300790784, "text": "I can eat a large papa johns right about now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58043106, "name": "dude", "screen_name": "iEmelyn", "lang": "en", "location": "North Carolina, USA", "create_at": date("2009-07-18"), "description": "http://snapchat.com/add/Trapxgawwd", "followers_count": 6833, "friends_count": 2926, "statues_count": 56239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sharpsburg, NC", "id": "8af26701aaafb31b", "name": "Sharpsburg", "place_type": "city", "bounding_box": rectangle("-77.840203,35.854897 -77.817104,35.87648") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37127, "countyName": "Nash", "cityID": 3761060, "cityName": "Sharpsburg" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043598460055552, "text": "@loumongous Is there a video? I personally was looking forward to fighting Void myself cause I love vs Sheik but good stuff to him.", "in_reply_to_status": 722974094900109312, "in_reply_to_user": 459708394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 459708394 }}, "user": { "id": 522605684, "name": "CLG Keitaro", "screen_name": "KeitaroTime", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-03-12"), "description": "Video Production Specialist for @clgaming. Co-creator of Rush Hour Smash, member of LoF crew. Smash Bros Commentator Business Email keitaro@clgaming.net", "followers_count": 16607, "friends_count": 446, "statues_count": 6113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, CA", "id": "e6fc035b8939bd3b", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-118.06946,34.087059 -117.99218,34.180507") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602462, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043598711689216, "text": "@ebbtideapp Tide in Marion, Massachusetts 04/21/2016\nHigh 8:21am 3.9\n Low 1:37pm 0.2\nHigh 8:39pm 4.5\n Low 2:14am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.76,41.72"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 78, "friends_count": 1, "statues_count": 26238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marion Center, MA", "id": "1589fbb50112c19a", "name": "Marion Center", "place_type": "city", "bounding_box": rectangle("-70.771833,41.687636 -70.751888,41.721551") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2538575, "cityName": "Marion Center" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043599688982529, "text": "USGS reports a M0.7 #earthquake Nevada on 4/21/16 @ 6:58:08 UTC https://t.co/nWOF1nYFP9 #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.9644,36.8344"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 5728, "friends_count": 17, "statues_count": 151711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nevada, USA", "id": "d374fb61a20fb74f", "name": "Nevada", "place_type": "admin", "bounding_box": rectangle("-120.00574,35.002086 -114.039649,42.002208") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6027, "countyName": "Inyo" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043600175607808, "text": "Temp: 62.7°F Wind:0.0mph Pressure: 30.008hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043600469204992, "text": "All I really need is your undivided", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487775217, "name": "burnt", "screen_name": "SofiaDrewniany", "lang": "en", "location": "Middletown, CT", "create_at": date("2012-02-09"), "description": "those who mind don't matter and those who matter don't mind ~ xo till we od", "followers_count": 529, "friends_count": 270, "statues_count": 8049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, CT", "id": "8c814b789b5f7574", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-72.752943,41.494706 -72.58268,41.604104") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 947290, "cityName": "Middletown" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043601135984641, "text": "I have literally applied to like 15 places by my new house. Please hire me people ��☺", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3015953184, "name": "allie", "screen_name": "allisonrita96", "lang": "en", "location": "null", "create_at": date("2015-02-09"), "description": "GCU '17// Is it nap time yet?", "followers_count": 378, "friends_count": 231, "statues_count": 7212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043601152880640, "text": "Qué comience (@ Monty Bar - @themontybar in Los Angeles, CA w/ @vcastaa) https://t.co/Ytdx9MWipf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.26668917,34.0516931"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 603022726, 99436996 }}, "user": { "id": 194262285, "name": "Arturo Luna", "screen_name": "alunavarrete", "lang": "en", "location": "Mexico City", "create_at": date("2010-09-23"), "description": "PR Manager & Embajador Casa Pedro Domecq\r\n@DomecqInD_House", "followers_count": 604, "friends_count": 212, "statues_count": 14562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043601261830145, "text": "@palebluejonny bless you", "in_reply_to_status": 723042494796034048, "in_reply_to_user": 3738792492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3738792492 }}, "user": { "id": 64868248, "name": "Haley McIntyre", "screen_name": "haleymcintyre", "lang": "en", "location": "null", "create_at": date("2009-08-11"), "description": "•you probably don't care•", "followers_count": 473, "friends_count": 239, "statues_count": 9284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043601945481216, "text": "63.2F (Feels: 63.2F) - Humidity: 99% - Wind: 6.9mph S - Gust: 12.1mph - Pressure: 1008.8mb - Rain: 0.26\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 236103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043602201456640, "text": "Ripley SW Limestone Co. Temp: 64.4°F Wind:2.2mph Pressure: 995.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-21T00:00:02.000Z"), "id": 723043602356531200, "text": "@_dazedd123 https://t.co/FF5O99Yg7w", "in_reply_to_status": 723043043025154048, "in_reply_to_user": 2193538394, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2193538394 }}, "user": { "id": 2372871062, "name": "anthony", "screen_name": "anthonyyemm", "lang": "en", "location": "null", "create_at": date("2014-03-04"), "description": "she's gay", "followers_count": 146, "friends_count": 139, "statues_count": 3569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043602700591104, "text": "It was the best I ever had !!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1694513221, "name": "Yadira Vargas", "screen_name": "Yadiiiirrrraaa", "lang": "en", "location": "Oregon City ", "create_at": date("2013-08-23"), "description": "enjoy my tweets", "followers_count": 461, "friends_count": 76, "statues_count": 11257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043603371544577, "text": "@etnow We Knew Miley Cyrus would NEVER Be a Kennedy!", "in_reply_to_status": 723038598778486784, "in_reply_to_user": 23603092, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23603092 }}, "user": { "id": 462923401, "name": "Kimberley Monari", "screen_name": "KimberleyMonari", "lang": "en", "location": "The Best Coast", "create_at": date("2012-01-13"), "description": "Broadcast/Print Journalist, Associated Press Mark Twain Award Winner, Voice Over, Avid SF Giants/49ers Fan, 1-time Password Plus Contestant, Avowed Contrarian", "followers_count": 940, "friends_count": 467, "statues_count": 104480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043603405099008, "text": "/ / o u t n o w / / //// #PATCHMEUP https://t.co/RLnIDEmu23", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PATCHMEUP" }}, "user": { "id": 382154913, "name": "The Y Axes", "screen_name": "TheYAxes", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-09-29"), "description": "/ / tech pop / / moon rock / / post-americana / /", "followers_count": 5100, "friends_count": 4522, "statues_count": 1627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043603480776704, "text": "Temp 39.4° Hi/Lo 43.5/39.3 Rng 4.2° WC 39.4° Hmd 78% Rain 0.00\" Storm 0.00\" BAR 30.195 Falling DP 33.1° Wnd 0mph Dir --- Gst 6mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 118, "statues_count": 18065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043603954561024, "text": "@Babby_Geee ��������☠", "in_reply_to_status": 723043296071704577, "in_reply_to_user": 931227872, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 931227872 }}, "user": { "id": 514909170, "name": "Aimée", "screen_name": "imehbaez", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "M.A.M ❌⭕️", "followers_count": 371, "friends_count": 287, "statues_count": 28664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043604114079745, "text": "Wind 0.0 mph ---. Barometer 30.18 in, Falling slowly. Temperature 39.7 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 33, "friends_count": 118, "statues_count": 159218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043604504006656, "text": "What to watch on Amazon Prime in May https://t.co/QB8un055Rj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17573, "friends_count": 17585, "statues_count": 70902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043604806115328, "text": "Wind 0.0 mph ---. Barometer 30.145 in, Falling. Temperature 40.5 °F. Rain today 0.00 in. Humidity 57%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 9, "friends_count": 44, "statues_count": 7483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043605372395520, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.215 in, Steady. Temp 39.9F. Rain today 0.00 in. Humidity 87% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 614, "friends_count": 825, "statues_count": 41101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043605535813634, "text": "They're* damnit I can't spell tonight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2200605212, "name": "Phil", "screen_name": "JCuellar06", "lang": "en", "location": "817", "create_at": date("2013-11-17"), "description": "The Fort", "followers_count": 253, "friends_count": 295, "statues_count": 23392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043606181908480, "text": "MY SUNSHINE THOUGH ��", "in_reply_to_status": 722877720699514881, "in_reply_to_user": 25876319, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25876319, "name": "Sarah ✨", "screen_name": "QueenofRider", "lang": "en", "location": "Rider tweeted me on 12/13/14", "create_at": date("2009-03-22"), "description": "18 | I'm literal trash for a dork named Rider Strong | Just one more year and then you'll be happy...", "followers_count": 2604, "friends_count": 611, "statues_count": 84183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheektowaga, NY", "id": "d3f378f122d4f4a7", "name": "Cheektowaga", "place_type": "city", "bounding_box": rectangle("-78.799913,42.863516 -78.696885,42.950897") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3615000, "cityName": "Cheektowaga" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043606542442496, "text": "necessary. https://t.co/htbsM3HoFq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 410987203, "name": "haylee stubblefield", "screen_name": "hayleestubbs", "lang": "en", "location": "T town/Stilly ", "create_at": date("2011-11-12"), "description": "tell me what you know bout dreams", "followers_count": 385, "friends_count": 281, "statues_count": 6999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043606664253440, "text": "@AQ_227 @Alialkandari_ @A_failchawe @smb965 \n\n١:٥٠ موعده وين بيروح مني", "in_reply_to_status": 723043423922597888, "in_reply_to_user": 1382801700, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 1382801700, 269941994, 4610185757, 184859192 }}, "user": { "id": 348084039, "name": "يوسف يتحدث :)", "screen_name": "VAIKINGb", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-08-03"), "description": "PhotoGrapher .... Instagram:yousefalkandari_", "followers_count": 620, "friends_count": 103, "statues_count": 35200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-21T00:00:03.000Z"), "id": 723043606697668609, "text": "What even is sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2881216592, "name": "lizbeth", "screen_name": "sorryimlizbeth", "lang": "en", "location": "Euless, TX", "create_at": date("2014-10-28"), "description": "16||iFollowback||sc:lizbethvega", "followers_count": 2491, "friends_count": 2644, "statues_count": 12753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euless, TX", "id": "5e2d6c67e728cca5", "name": "Euless", "place_type": "city", "bounding_box": rectangle("-97.15606,32.81088 -97.032953,32.881593") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4824768, "cityName": "Euless" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043606735540224, "text": "���������� https://t.co/3mEkgBlxWo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2635843889, "name": "raiden cote", "screen_name": "Raiden_rolls0ne", "lang": "en", "location": "Miriam ❤", "create_at": date("2014-06-23"), "description": "'Juggin Is My Hobby '\nsc; raiden016", "followers_count": 240, "friends_count": 257, "statues_count": 4259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043607192698880, "text": "Temp: 66.2°F | Humidity: 76% | Wind: --- @ 0.0 mph | Barometer: 30.07 in | Dewpoint: 58.4°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 166610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043607410708480, "text": "I think imma have to get her ����she still eating it up https://t.co/dbQu890nfk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3184326128, "name": "•Dess•", "screen_name": "queeenn__d", "lang": "en", "location": "null", "create_at": date("2015-05-03"), "description": "HOES", "followers_count": 737, "friends_count": 926, "statues_count": 2547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Monroe, LA", "id": "574af63ba69840d6", "name": "West Monroe", "place_type": "city", "bounding_box": rectangle("-92.192108,32.481896 -92.113927,32.57157") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2280955, "cityName": "West Monroe" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043607557627904, "text": "Wind 0 mph SE. Barometer 30.01 in, Rising slowly. Temperature 70.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 59, "friends_count": 76, "statues_count": 27218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043607641378817, "text": "6️⃣1️⃣ https://t.co/iylVbQ9Tlt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 58993073, "name": "Heyleen☕️", "screen_name": "heyleenseleena", "lang": "en", "location": "Montebello, CA.", "create_at": date("2009-07-21"), "description": "22.", "followers_count": 609, "friends_count": 181, "statues_count": 60168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043607763021824, "text": "@ThatKevinSmith get @jayoliva1 back onto fatman on batman and talk #BatmanvSuperman. He'll unpack the nightmare scene, blow your mind!", "in_reply_to_status": -1, "in_reply_to_user": 14681605, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BatmanvSuperman" }}, "user_mentions": {{ 14681605, 849458468 }}, "user": { "id": 211617790, "name": "Kerry Vanderberg", "screen_name": "kmvanderberg", "lang": "en", "location": "Mountain View, CA", "create_at": date("2010-11-03"), "description": "I follow Jesus, love teaching the Bible, and teaching and mentoring high school students. I play drums, love watching superhero movies, and reading comics.", "followers_count": 47, "friends_count": 190, "statues_count": 975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain View, CA", "id": "b19a2cc5134b7e0a", "name": "Mountain View", "place_type": "city", "bounding_box": rectangle("-122.117916,37.356771 -122.044969,37.436935") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608031457282, "text": "... ... WORLDLY PARALLELS OF TRANSFORMATION... https://t.co/H7lyRmymMB ... ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 360837046, "name": "Andre' Harris", "screen_name": "AndreHarris4", "lang": "en", "location": "Baton Rouge, Louisiana", "create_at": date("2011-08-23"), "description": "MUSIC GENRE: MIDLAND HOP", "followers_count": 1065, "friends_count": 5004, "statues_count": 126078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608031539201, "text": "no one cares about your deep life issues and how \"he treated you\" go on omegle stupid bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347051482, "name": "sαvαnnα ❂", "screen_name": "always_f0reverr", "lang": "en", "location": "Rowan University✌️", "create_at": date("2011-08-01"), "description": "whether or not it is clear to you, no doubt the universe is unfolding as it should", "followers_count": 280, "friends_count": 375, "statues_count": 5641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glassboro, NJ", "id": "55352b8c5e28010f", "name": "Glassboro", "place_type": "city", "bounding_box": rectangle("-75.168425,39.679348 -75.081224,39.73751") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3426340, "cityName": "Glassboro" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608245338112, "text": "@NottexEterna ��������\n\nWATCH THE HAIR MUSIC VIDEO! ITS AMAZZIINNGGGG", "in_reply_to_status": 723043244225916929, "in_reply_to_user": 45247024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45247024 }}, "user": { "id": 91282251, "name": "mayra∵", "screen_name": "LarryDickSquad", "lang": "en", "location": "Simi Valley, California ت", "create_at": date("2009-11-19"), "description": "20 ※Pansexual※ || Little Mix follows 11/7/15♡♡ || http://redbubble.com/people/larrydicksquad", "followers_count": 2468, "friends_count": 1882, "statues_count": 147250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608316649472, "text": "This seniors memory book has me simpin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634976076, "name": "Harrison", "screen_name": "hbeemiller7", "lang": "en", "location": "irhs ", "create_at": date("2012-07-13"), "description": "SD Mines Football", "followers_count": 1063, "friends_count": 697, "statues_count": 13563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oro Valley, AZ", "id": "23ddee699e919f28", "name": "Oro Valley", "place_type": "city", "bounding_box": rectangle("-111.02149,32.357649 -110.928016,32.477644") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608538931204, "text": "All those white peoples was rolling of a X on Project X ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417615011, "name": "Des Otto ❗️", "screen_name": "214LilDesmond", "lang": "en", "location": "DTX ", "create_at": date("2011-11-20"), "description": "Wolf of 4 Street #SHSU", "followers_count": 2065, "friends_count": 815, "statues_count": 66586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608555741184, "text": "Wind 0.0 mph ---. Barometer 29.927 in, Falling slowly. Temperature 57.3 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608778010624, "text": "Depends on which stage of talking but I'm sensitive so for me it's a bad idea https://t.co/MVD5SvbjKe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458147769, "name": "bugatti hottie", "screen_name": "chellae_rae", "lang": "en", "location": "Inglewood, CA", "create_at": date("2012-01-07"), "description": "I'm Marsha you're Jan ✨", "followers_count": 876, "friends_count": 663, "statues_count": 35117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043608962560000, "text": "@kayfaraday jazz fusion group obsessed with icp", "in_reply_to_status": 723041491174596608, "in_reply_to_user": 43754642, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 43754642 }}, "user": { "id": 1570682910, "name": "NARFNra - hi", "screen_name": "NARFNra", "lang": "en", "location": "who knows", "create_at": date("2013-07-05"), "description": "It's Narf!", "followers_count": 108, "friends_count": 104, "statues_count": 14479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043609226911744, "text": "@sabbella_sab @silverheartss https://t.co/Nx6Eac8EvD", "in_reply_to_status": 723042494175277057, "in_reply_to_user": 3323772393, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3323772393, 2185424438 }}, "user": { "id": 48875310, "name": "Alhaj Nasir Uddin", "screen_name": "MrNaassir", "lang": "en", "location": "Manhattan, New York. U.S.A.", "create_at": date("2009-06-19"), "description": "Muslim,Graduated from Karachi,Pakistan & U.S.Citizen.English Urdu Arabic Please Type MrNaassir in Google & see my 16000 pics ThanksNO DM Pl or U will be Blocked", "followers_count": 10788, "friends_count": 6344, "statues_count": 190408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043609637851136, "text": "Wide awake ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 969294254, "name": "lil Frankie", "screen_name": "lil__nene", "lang": "en", "location": "Denver, CO", "create_at": date("2012-11-24"), "description": "My advise to you, is get that rope wrap it around your neck, & jump", "followers_count": 1420, "friends_count": 1360, "statues_count": 29365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broomfield, CO", "id": "00f8c733400e497a", "name": "Broomfield", "place_type": "city", "bounding_box": rectangle("-105.147267,39.894185 -104.997195,40.000314") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8014, "countyName": "Broomfield", "cityID": 809280, "cityName": "Broomfield" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043609700761600, "text": "I can't stop laughing rn ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2167817636, "name": "Ruru", "screen_name": "Ruuubbbyy_07", "lang": "en", "location": "Cesars Bed", "create_at": date("2013-10-31"), "description": "CA ❤️", "followers_count": 718, "friends_count": 523, "statues_count": 13742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seaside, CA", "id": "89bac4213b1b5525", "name": "Seaside", "place_type": "city", "bounding_box": rectangle("-121.860374,36.596815 -121.789746,36.654798") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-04-21T00:00:04.000Z"), "id": 723043609918988288, "text": "Nerd n chuck bass https://t.co/t4hQeB0KSl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226034553, "name": "rocky", "screen_name": "rockydavids_1", "lang": "en", "location": "Greenwich Village, Manhattan", "create_at": date("2015-05-01"), "description": "No Snakes In The Grass Cause I Mowed It niggah", "followers_count": 293, "friends_count": 239, "statues_count": 10805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043610988433409, "text": "Move something tho all that don't move me til u pull up n get@me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248277888, "name": "❣TiA~ORTIZ❣", "screen_name": "KarmenBangz", "lang": "en", "location": "null", "create_at": date("2011-02-06"), "description": "I ain't perfect but I'm not that messy miserable low self esteem ratchett that checks my page daily!", "followers_count": 562, "friends_count": 91, "statues_count": 5220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043611705663488, "text": "I can tell they scared from the way they talk!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2500136551, "name": "FRêęJR", "screen_name": "JET_GrEEDy", "lang": "en", "location": "California, USA", "create_at": date("2014-05-16"), "description": "R.I.P BALLY .....FREE Jr out the jam✊", "followers_count": 916, "friends_count": 640, "statues_count": 15916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043612141834241, "text": "@junyongie he knows he really knows and it's so unfair of him ��", "in_reply_to_status": 722946277760200704, "in_reply_to_user": 1250984911, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1250984911 }}, "user": { "id": 295358382, "name": "For Chan. #ChainedUp", "screen_name": "Ieohyuk", "lang": "en", "location": "Puerto Rico", "create_at": date("2011-05-08"), "description": "himup and leohyuk~ #ForeverWithBAP", "followers_count": 383, "friends_count": 242, "statues_count": 127114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemont, IL", "id": "f48e110211eee6ef", "name": "Rosemont", "place_type": "city", "bounding_box": rectangle("-87.896102,41.973825 -87.855303,42.009717") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1765819, "cityName": "Rosemont" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043612607537152, "text": "Temp 59.3°F Wind Chill 59.0°F RH 50% Wind 5.4 W Gust 16.0 W SLP 30.065 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 107, "friends_count": 63, "statues_count": 34860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043613345648641, "text": "Wont be no scoring..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 606820219, "name": "Shiaaa✨", "screen_name": "shia_uniquee", "lang": "en", "location": "Nacogdoches, TX", "create_at": date("2012-06-12"), "description": "Just a goofy girl, who doesn't change for nobody. #SFA #ASO Longview ✈️ Nacogdoches .", "followers_count": 659, "friends_count": 651, "statues_count": 8310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043613408661504, "text": "Come #GetShiftFaced with us this Saturday at #BattlefortheDanceBelt!… https://t.co/eon0NkUgz2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.31721396,47.61588659"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GetShiftFaced", "BattlefortheDanceBelt" }}, "user": { "id": 103769232, "name": "Xaviera Vandermay", "screen_name": "3rdShiftDance", "lang": "en", "location": "Seattle, WA", "create_at": date("2010-01-10"), "description": "This is the twitter page for Seattle contemporary jazz dance company 3rd Shift Dance directed by Xaviera Vandermay", "followers_count": 400, "friends_count": 1071, "statues_count": 1488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043613719056388, "text": "@MeganShine_ I just finished studying...", "in_reply_to_status": 723042625092128769, "in_reply_to_user": 261400929, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261400929 }}, "user": { "id": 452652626, "name": "Jill Bradley", "screen_name": "_JillBradley_", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "337•ULL•ΦΜ", "followers_count": 780, "friends_count": 443, "statues_count": 15717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043613760954368, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 63.7 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043614088028161, "text": "@giannamaii amazing news i didnt", "in_reply_to_status": 723042096580423680, "in_reply_to_user": 751951651, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 751951651 }}, "user": { "id": 2203960962, "name": "gonzo", "screen_name": "Inaleeeigh", "lang": "en", "location": "SF CA", "create_at": date("2013-11-19"), "description": "for the time being yes i am yours", "followers_count": 612, "friends_count": 608, "statues_count": 8858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043614335488000, "text": "2.1 magnitude #earthquake. 110km NW of Talkeetna, Alaska https://t.co/Rmbh6E8KeE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-151.437,63.109"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 70852, "friends_count": 10, "statues_count": 101734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043614436159489, "text": "Done smoking as of righhht now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 814263860, "name": "موب مايك", "screen_name": "mikey_mf_brown", "lang": "en", "location": "ひ", "create_at": date("2012-09-09"), "description": "null", "followers_count": 515, "friends_count": 281, "statues_count": 5775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-04-21T00:00:05.000Z"), "id": 723043614503268352, "text": "@emilynpyatt @hanmgrant @CalebSWilliams1 this is true. all in this together", "in_reply_to_status": 723043566407262208, "in_reply_to_user": 187441936, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 187441936, 320457905, 97693337 }}, "user": { "id": 298035515, "name": "bri", "screen_name": "brianna_norrell", "lang": "en", "location": "Guntersville, AL", "create_at": date("2011-05-13"), "description": "ΑΟΠ at JSU", "followers_count": 1064, "friends_count": 974, "statues_count": 15325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jacksonville, AL", "id": "d5dbaf62e7106dc4", "name": "Jacksonville", "place_type": "city", "bounding_box": rectangle("-85.800502,33.752607 -85.724207,33.840282") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 138272, "cityName": "Jacksonville" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043616042569728, "text": "Smh �� https://t.co/DWEw3jPhrw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2849818363, "name": "OG", "screen_name": "TrillaDa6Foota", "lang": "en", "location": "The Westside ☀️", "create_at": date("2014-10-09"), "description": "My names Rod... And I like to party", "followers_count": 587, "friends_count": 563, "statues_count": 59668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043616382324736, "text": "your tongue say killa then kill my mood.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317265430, "name": "pearce", "screen_name": "pearcegossett", "lang": "en", "location": "null", "create_at": date("2011-06-14"), "description": "null", "followers_count": 817, "friends_count": 549, "statues_count": 42941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, MI", "id": "4e284ea3fff91c09", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-83.209206,42.533782 -83.086881,42.624224") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2680700, "cityName": "Troy" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043616587845632, "text": "@tjironaddict LOTS OF ACTION! I WANT TO SCHOCK ADDICTS WITH MY CONDITION AND FIGHTING ABILITY. I WANT THEM TO SAY \"THAT MUTHA CANT BE 57!\"", "in_reply_to_status": 723041065335287809, "in_reply_to_user": 4805862858, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4805862858 }}, "user": { "id": 1278377149, "name": "CT Fletcher", "screen_name": "CTFletcherISYMF", "lang": "en", "location": "null", "create_at": date("2013-03-18"), "description": "OFFICIAL CT FLETCHER TWITTER ACCOUNT\r\nCompeted in Powerlifting for 30 yrs. Won 3 world “drug free” bench titles, 3 World Strict Curl titles. ISATORI DISCOUNT CO", "followers_count": 104803, "friends_count": 3973, "statues_count": 28128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043616843685888, "text": "@V2Brie ask her to let me take it for a spin", "in_reply_to_status": 723041614189334528, "in_reply_to_user": 489967962, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 489967962 }}, "user": { "id": 342535530, "name": "Antonio", "screen_name": "ItsAntonioJQ", "lang": "en", "location": "Menifee, CA", "create_at": date("2011-07-25"), "description": "US Air National Guard.", "followers_count": 768, "friends_count": 471, "statues_count": 57094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043616847962112, "text": "So true�� https://t.co/wqLZQVsGTr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3129919696, "name": "✨Ancelia✨", "screen_name": "AnceliaNimely", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "LIBERIAN_Girl( I Ain't your MAMA)", "followers_count": 401, "friends_count": 673, "statues_count": 2034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043617485500416, "text": "#Felony at N Chickasaw Trail & Valencia College Ln. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2777014,28.553749"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Felony", "orlpol", "ocso" }}, "user": { "id": 39134311, "name": "Police Calls 32825", "screen_name": "orlpol32825", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 97, "friends_count": 1, "statues_count": 6773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043617808351232, "text": "Happy birthday to this beautiful man who speaks to my soul. I love you, Robert Smith. https://t.co/LiHUZe561c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261407095, "name": "chris", "screen_name": "yourdaddychris", "lang": "en", "location": "null", "create_at": date("2011-03-05"), "description": "@_coolkidadriana, The Cure, and pugs.", "followers_count": 807, "friends_count": 166, "statues_count": 102745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043617904828418, "text": "This is the most I've tweeted and spent time on social media in god knows how long. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 573334894, "name": "Zuleivy.", "screen_name": "_Zuzu92", "lang": "en", "location": "Houston, Tx.", "create_at": date("2012-05-06"), "description": "null", "followers_count": 962, "friends_count": 406, "statues_count": 25584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043618341171200, "text": "We both know I don't lift weights. I'm the cardio queen.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251004964, "name": "Kelly MacDonald", "screen_name": "Kelly82495", "lang": "en", "location": "null", "create_at": date("2011-02-11"), "description": "TXST • Dog Mom", "followers_count": 813, "friends_count": 579, "statues_count": 24780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043618714488832, "text": "I know when that hotline bling...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51651004, "name": "Miss Wells ✌️", "screen_name": "_SuchADreamer", "lang": "en", "location": "Snapchat: ericadwells", "create_at": date("2009-06-27"), "description": "Cause darling, I'm a nightmare dressed like a daydream.", "followers_count": 578, "friends_count": 524, "statues_count": 38555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043618823483392, "text": "#SuspiciousIncident at 5543-5598 Orange Blossom Trail. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.3970327,28.4787767"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SuspiciousIncident", "orlpol", "ocso" }}, "user": { "id": 39065901, "name": "Police Calls 32839", "screen_name": "orlpol32839", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 151, "friends_count": 1, "statues_count": 41435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge, FL", "id": "aa9eedbd533da630", "name": "Oak Ridge", "place_type": "city", "bounding_box": rectangle("-81.442112,28.450553 -81.396669,28.487021") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1250638, "cityName": "Oak Ridge" } }
+{ "create_at": datetime("2016-04-21T00:00:06.000Z"), "id": 723043619003723776, "text": "Temp: 56.2°F - Dew Point: 56.2° - Wind: S @ 5.3 mph - Gust: 10.7 - Rain Today: 0.04in. - Pressure: 29.83in, - Trend: Falling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 12, "statues_count": 16555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043619960020995, "text": "*Goes to sleep to interstellar soundtrack every night.* ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565987702, "name": "Cedar Stone", "screen_name": "stone_cedar", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-04-28"), "description": "Since '97", "followers_count": 1249, "friends_count": 374, "statues_count": 13671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waxahachie, TX", "id": "45ff3ab084a1f419", "name": "Waxahachie", "place_type": "city", "bounding_box": rectangle("-96.880016,32.316564 -96.79913,32.494604") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4876816, "cityName": "Waxahachie" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043620551548928, "text": "First stop. Let's go. @ Night + Market Song https://t.co/GumRh9gski", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.27588,34.08688"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 364482473, "name": "Laura Carlson", "screen_name": "Lecarlso", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2011-08-29"), "description": "Bourbon lover. Wine enthusiast.", "followers_count": 77, "friends_count": 223, "statues_count": 849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043620903747584, "text": "@melaniejoyceee too bad almost there", "in_reply_to_status": 723043422781595649, "in_reply_to_user": 339466353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 339466353 }}, "user": { "id": 1065020474, "name": "meliss", "screen_name": "nig_lissa", "lang": "en", "location": "SD|LA", "create_at": date("2013-01-05"), "description": "Jesus is my sugar daddy", "followers_count": 110, "friends_count": 105, "statues_count": 5357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043620987641858, "text": "Omg I miss Malia https://t.co/CDGzu2yEvl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238780510, "name": "shaybæ", "screen_name": "mer_shay", "lang": "en", "location": "Phoenix, AZ ", "create_at": date("2011-01-15"), "description": "future Mrs. Owings. life is the bubbles", "followers_count": 809, "friends_count": 581, "statues_count": 22300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043621058973697, "text": "4/21/2016 - 02:00\nTemp: 58.4F \nHum: 98%\nWind: 0.0 mph\nBaro: 29.931in. & Falling\nRain: 0.00 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 51983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043621117644800, "text": "LOL well deserved; great taste and price https://t.co/y1G9Kwbjre", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141086619, "name": "IG: dennispastorizo", "screen_name": "dennispastorizo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-06"), "description": "Host of YouTube's Dennis Does Gossip//Entertainment Reporter- @LATV En La Zona//Co-host, Mornings With Fousey// HGTV's @crowdedhousetv / SC: dennisdoesla", "followers_count": 11201, "friends_count": 1970, "statues_count": 19021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043621121843201, "text": "@jacobvest2 calls me in the morning to make sure I wake up on time https://t.co/LgERbJogac", "in_reply_to_status": -1, "in_reply_to_user": 2874683829, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2874683829 }}, "user": { "id": 2219317693, "name": "Brititty", "screen_name": "Brititknee", "lang": "en", "location": "AK // WA", "create_at": date("2013-11-28"), "description": "if we die, we die.", "followers_count": 1156, "friends_count": 518, "statues_count": 16875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakes, AK", "id": "6de06a92516436ce", "name": "Lakes", "place_type": "city", "bounding_box": rectangle("-149.360927,61.563018 -149.249951,61.642803") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 242832, "cityName": "Lakes" } }
+{ "create_at": datetime("2016-04-21T00:00:07.000Z"), "id": 723043621755195393, "text": "@HoolMcBride LMAO good. ��", "in_reply_to_status": 723042261609521152, "in_reply_to_user": 562543585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 562543585 }}, "user": { "id": 129645532, "name": "Southside Sizz", "screen_name": "iamsizzlemcfly", "lang": "en", "location": "The Zae, CA.", "create_at": date("2010-04-04"), "description": "Bookings/Features: iamsizzlemcfly@gmail.com Owner/Designer of Pop the Trunk Apparel | PTTA Contact: popthetrunkapparel@gmail.com https://t.co/eMNUfq4jMh", "followers_count": 418, "friends_count": 413, "statues_count": 6351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405978645893121, "text": "There's some serious sketch balls out there mayne", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317350263, "name": "LoganTaylor", "screen_name": "Lots_of_Log", "lang": "en", "location": "null", "create_at": date("2011-06-14"), "description": "Get it how you live. Northwest Missouri State University class of 2018.", "followers_count": 759, "friends_count": 589, "statues_count": 21864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryville, MO", "id": "d2e5e6b4697f41a7", "name": "Maryville", "place_type": "city", "bounding_box": rectangle("-94.894994,40.318088 -94.831714,40.372994") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29147, "countyName": "Nodaway", "cityID": 2946640, "cityName": "Maryville" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405978993991680, "text": "@_jennmorgann congrats����", "in_reply_to_status": 723404105729761281, "in_reply_to_user": 725588306, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 725588306 }}, "user": { "id": 124540088, "name": "snavs", "screen_name": "real_nava", "lang": "en", "location": "null", "create_at": date("2010-03-19"), "description": "18 | All Area QB | STAR$ | @Marilynnbay", "followers_count": 1027, "friends_count": 850, "statues_count": 17725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Gabriel, CA", "id": "efdc0bd6b3e2a8fb", "name": "San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.120827,34.071162 -118.073374,34.115328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 667042, "cityName": "San Gabriel" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405978994020353, "text": "when you check your bank account & see that you got your tax money, got good tips at work & got a paycheck from your two jobs :')", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 477262038, "name": "elbaddy", "screen_name": "elbaamejia", "lang": "en", "location": "null", "create_at": date("2012-01-28"), "description": "People usually need a couple drinks just to get through a conversation with me", "followers_count": 192, "friends_count": 141, "statues_count": 4618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jamesburg, NJ", "id": "007f540206ac15d0", "name": "Jamesburg", "place_type": "city", "bounding_box": rectangle("-74.471178,40.327303 -74.376034,40.377375") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3434890, "cityName": "Jamesburg" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405979421806592, "text": "@nataliematian oh good�� this person has expensive taste tho lol", "in_reply_to_status": 723405474654097408, "in_reply_to_user": 636359696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 636359696 }}, "user": { "id": 2769302797, "name": "young hot ebony✨", "screen_name": "haddiethebaddie", "lang": "en", "location": "408", "create_at": date("2014-08-25"), "description": "cutie with a booty", "followers_count": 428, "friends_count": 287, "statues_count": 10362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cupertino, CA", "id": "36237ab3643ff2be", "name": "Cupertino", "place_type": "city", "bounding_box": rectangle("-122.091151,37.285786 -121.995595,37.34038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 617610, "cityName": "Cupertino" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405979883180032, "text": "i would literally read ANYTHING \nhttps://t.co/QTv42pJghf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560697155, "name": "starfire", "screen_name": "dalysdynamite", "lang": "en", "location": "Hogwarts, Scotland ", "create_at": date("2014-06-10"), "description": "ambitious.", "followers_count": 353, "friends_count": 249, "statues_count": 11801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980109705216, "text": "Sucks https://t.co/CNErQtZwE5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458518183, "name": "Leon FreeSpitta", "screen_name": "ThatGuyFreeman", "lang": "en", "location": "null", "create_at": date("2012-01-08"), "description": "null", "followers_count": 491, "friends_count": 421, "statues_count": 14359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, TX", "id": "8935eb0e13a342db", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-95.505362,29.507337 -95.434776,29.567483") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4827540, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980147437573, "text": "@SamuelLJackson cc @NICKIMINAJ. https://t.co/1cmRHALRMh", "in_reply_to_status": -1, "in_reply_to_user": 75974281, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 75974281, 35787166 }}, "user": { "id": 2896066195, "name": "Kevin thå Snowman,dr", "screen_name": "kevin_gonzo", "lang": "en", "location": "SoCal/Randeberg/Denver/CT/WA", "create_at": date("2014-11-10"), "description": "booking at raindropsonrosesmedia at gmail award winning actor, writer and general freak of nature. i love cock and satan. legalize lsd free leonard pickard.", "followers_count": 374, "friends_count": 990, "statues_count": 17463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980164198400, "text": "i love ah freak hoe tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167960560, "name": "lil' baby✨", "screen_name": "A_YungQueen", "lang": "en", "location": "Illinois, USA", "create_at": date("2010-07-17"), "description": "#RipDom❤", "followers_count": 1885, "friends_count": 1775, "statues_count": 61690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980466212864, "text": "i'm going to hate myself in the morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3386210294, "name": "marcela ✨", "screen_name": "AyeYoMarcelaa", "lang": "en", "location": "WEST$IDE", "create_at": date("2015-08-29"), "description": "new twitter & shit", "followers_count": 122, "friends_count": 117, "statues_count": 958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980734644224, "text": "@kaustinsoccer tonight I found a fellow Arsenal supporter and conversed about wilshere's return", "in_reply_to_status": 723405455741972480, "in_reply_to_user": 257142684, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 257142684 }}, "user": { "id": 257142684, "name": "Kylie Austin", "screen_name": "kaustinsoccer", "lang": "en", "location": "Houston, TX // Wichita Falls", "create_at": date("2011-02-24"), "description": "now I don't wanna talk like a boss but we both know you know who I am || Midwestern State #15 ⚽️|| ΑΦ || cancer survivor", "followers_count": 650, "friends_count": 442, "statues_count": 47324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980868857856, "text": "Some men are so lonely around this time of the night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1164115974, "name": "☾Michelle☽", "screen_name": "MichelleCCain", "lang": "en", "location": " NYC", "create_at": date("2013-02-09"), "description": "NYC • Leo • Mixed ✨ Snap: Michelleccain", "followers_count": 3119, "friends_count": 195, "statues_count": 32083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405980889800710, "text": "My back hurts. Like always. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371519335, "name": "Dustin", "screen_name": "dustinmcbride75", "lang": "en", "location": "Corcoran", "create_at": date("2011-09-10"), "description": "6'4 ~ Work Hard, Nap Hard", "followers_count": 845, "friends_count": 589, "statues_count": 91663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405981011472385, "text": "Wind 3.1 mph N. Barometer 30.04 in, Rising slowly. Temperature 37.4 °F. Rain today 0.01 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 41662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405981242155009, "text": "@Juuged https://t.co/pJ75mVkuPS", "in_reply_to_status": 723402810671271937, "in_reply_to_user": 352561529, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 352561529 }}, "user": { "id": 319850776, "name": ".", "screen_name": "Nyxnyl", "lang": "en", "location": "WE$T COA$T", "create_at": date("2011-06-18"), "description": "I miss the old Lynnie | sc: nokoleelyn", "followers_count": 2073, "friends_count": 743, "statues_count": 109125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405981347008512, "text": "@Billy_mo24 ya my momma's gunna love you ❤️", "in_reply_to_status": 723405643218976768, "in_reply_to_user": 480835308, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 480835308 }}, "user": { "id": 276317689, "name": "Megan Noel Heaton", "screen_name": "megannoelheaton", "lang": "en", "location": "tempe, arizona", "create_at": date("2011-04-02"), "description": "null", "followers_count": 935, "friends_count": 434, "statues_count": 10114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-22T00:00:00.000Z"), "id": 723405981594472449, "text": "@Alexis_grombol omg i love you ❤️", "in_reply_to_status": 723405603658289152, "in_reply_to_user": 1913091103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1913091103 }}, "user": { "id": 1233004795, "name": "tabs", "screen_name": "tabithabellis", "lang": "en", "location": "null", "create_at": date("2013-03-01"), "description": "null", "followers_count": 338, "friends_count": 177, "statues_count": 9385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405982349418496, "text": "When you have to be up for work in 3 hours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2962125466, "name": "matos", "screen_name": "matosnyc", "lang": "en", "location": "NYC", "create_at": date("2015-01-05"), "description": "null", "followers_count": 181, "friends_count": 123, "statues_count": 1550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405982705930241, "text": "La niggas be the craziest.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 774964543, "name": "Michelleee", "screen_name": "shelly_medina", "lang": "en", "location": "Whittier, CA ", "create_at": date("2012-08-22"), "description": "FOCUS BABYGIRL. @YG SC: Michelle_030 IG: Emm_medina", "followers_count": 256, "friends_count": 186, "statues_count": 8293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405982949212161, "text": "Coffee date with this cutie @michalmesa #fiancee #pdxbabes #mrsandmrs #michalmesa #andro… https://t.co/jPHd7ovAep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.682,45.52"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fiancee", "pdxbabes", "mrsandmrs", "michalmesa", "andro" }}, "user_mentions": {{ 2333784552 }}, "user": { "id": 112362139, "name": "Natasha Knight", "screen_name": "Team_Tasha", "lang": "en", "location": "Portland, OR", "create_at": date("2010-02-07"), "description": "©1985 loves:Mixolgy-Cocktail Creation-Photography-Fashion-Music-Snowboarding-Dance-Traveling-Vegetarianism-Pugs | Dislikes:Chewing with your mouth open", "followers_count": 191, "friends_count": 375, "statues_count": 3935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405983356055552, "text": "@itsjustjasmin you this Saturday vs. next Saturday https://t.co/vgBjxweXm9", "in_reply_to_status": -1, "in_reply_to_user": 2983315104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2983315104 }}, "user": { "id": 2324870474, "name": "Emily Rodriguez", "screen_name": "KillaMily", "lang": "en", "location": "null", "create_at": date("2014-02-02"), "description": "null", "followers_count": 500, "friends_count": 445, "statues_count": 7641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405983569960961, "text": "04/22@03:00 - Temp 57.5F, WC 57.5F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.967in, Falling slowly. Rain 0.00in. Hum 87%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405983767105537, "text": "@ebbtideapp Tide in Great Shoals Light, Maryland 04/22/2016\n Low 9:33am 0.1\nHigh 3:19pm 2.3\n Low 9:32pm 0.2\nHigh 3:32am 2.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.8833,38.2167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 80, "friends_count": 1, "statues_count": 26516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24039, "countyName": "Somerset" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405983771295744, "text": "@Ryan_Lough https://t.co/NS0stak88f", "in_reply_to_status": -1, "in_reply_to_user": 348297320, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 348297320 }}, "user": { "id": 2174947074, "name": "hannnnssss", "screen_name": "hannnnssss", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "Ryan Lough ❤️", "followers_count": 261, "friends_count": 175, "statues_count": 557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405984052310017, "text": "So many crazy news today... First I hear about @loveline ending and then finding out Prince passed away.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15614819 }}, "user": { "id": 16658371, "name": "Meowricio", "screen_name": "TheRealChato", "lang": "en", "location": "South El Monte, CA", "create_at": date("2008-10-08"), "description": "WC '15. Iced coffee drinker. Oh, music is pretty awesome. 69. My twitter, my views. IG: chatoxred PHHHOTO: Chatoxred", "followers_count": 574, "friends_count": 551, "statues_count": 33643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South El Monte, CA", "id": "122aef8d6840943b", "name": "South El Monte", "place_type": "city", "bounding_box": rectangle("-118.075186,34.028137 -118.021298,34.064367") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 672996, "cityName": "South El Monte" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405984173957121, "text": "Wind 0.0 mph ---. Barometer 29.924 in, Falling. Temperature 63.8 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405984274649090, "text": "19 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 471484136, "name": "Feathery Falcon", "screen_name": "NathanRojas3", "lang": "en", "location": "null", "create_at": date("2012-01-22"), "description": "HIU 19' // Active RS // J.C.", "followers_count": 685, "friends_count": 738, "statues_count": 3414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405984526262272, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":59.0°F Wind:0.0mph Pressure: 29.91hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 315884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405984891174912, "text": "Temp: 56.7F W C: 56.7F Wind:--- at 0.0kts Baro: 1020.6mb and Steady Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 207, "friends_count": 219, "statues_count": 102960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405985209929728, "text": "العمر كله ف طاعة الرحمن حبيبة قلبي �� @Maitha_tk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 368528116 }}, "user": { "id": 404863599, "name": "سكينة", "screen_name": "Skown_", "lang": "en", "location": "Arizona, USA", "create_at": date("2011-11-04"), "description": "A Future Meteorologist.", "followers_count": 1901, "friends_count": 451, "statues_count": 26483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405985344147457, "text": "Grumpy Cat, in the flesh. https://t.co/e0HkG2uSsk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17693390, "name": "Zoomshyne", "screen_name": "Zoomshyne", "lang": "en", "location": "Texas", "create_at": date("2008-11-27"), "description": "Raging sunshine!", "followers_count": 2392, "friends_count": 2630, "statues_count": 26385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-04-22T00:00:01.000Z"), "id": 723405985503551489, "text": "Should I k/o v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543502510, "name": "Jade", "screen_name": "xanaxnymph", "lang": "en", "location": "null", "create_at": date("2012-04-02"), "description": "LA ✈️SAN FRANCISCO .STAY TUNED", "followers_count": 335, "friends_count": 122, "statues_count": 34087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405986287882240, "text": "@elmakias hey bro you need an extra photographer to help you shoot Coachella? Or maybe even do behind the scenes stuff of you shooting?", "in_reply_to_status": 723342113459425280, "in_reply_to_user": 20785288, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20785288 }}, "user": { "id": 2807943590, "name": "Jason Walton", "screen_name": "jasonwaltonfilm", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-09-13"), "description": "Photographer // Music Lover // Casual Explorer Looking to go on tour with some dope bands! let's Hang out and snap some pics!", "followers_count": 86, "friends_count": 111, "statues_count": 1278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405986602487808, "text": "But Right Now I GOTTA GET THIS �� �� �� UP at #JOB2 #overnightshift… https://t.co/YoDiJdg9bL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.11595204,40.72813189"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JOB2", "overnightshift" }}, "user": { "id": 187805963, "name": "BIRTHDAYKING", "screen_name": "IAMV_DUB", "lang": "en", "location": "East Orange, NJ", "create_at": date("2010-09-06"), "description": "7YEARS AND STIL GOING ..", "followers_count": 1404, "friends_count": 1552, "statues_count": 46709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kearny, NJ", "id": "d5140ca0d08eadc7", "name": "Kearny", "place_type": "city", "bounding_box": rectangle("-74.164089,40.717135 -74.076066,40.786287") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436510, "cityName": "Kearny" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405986807992321, "text": "Temp: 64.9°F Wind:0.0mph Pressure: 29.959hpa Falling Rain Today 0.01in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 61948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405986988318720, "text": "Pointless", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 446604924, "name": "Deidra kay", "screen_name": "Deidragrand", "lang": "en", "location": "texas state ", "create_at": date("2011-12-25"), "description": "http://eye.donut.care", "followers_count": 534, "friends_count": 1019, "statues_count": 1597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405987055427584, "text": "I'm hella missing football right now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 424664756, "name": "Whoredan", "screen_name": "Jpostleydoe", "lang": "en", "location": "CA / Dreamville", "create_at": date("2011-11-29"), "description": "@Amyevelyn96", "followers_count": 765, "friends_count": 782, "statues_count": 52904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405987286151172, "text": "This remix I'm doing to wildcard by KSHMR is coming out so sick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1424125014, "name": "Jorge Toscano", "screen_name": "JorgeToscanoTho", "lang": "en", "location": "San Antonio! TX", "create_at": date("2013-05-12"), "description": "DJ • Producer || Supported by. Adventure Club, Victor Niglio, ETC || http://soundcloud.com/jorgetoscanomusic || booking MGMTjorgetoscano@gmail.com", "followers_count": 547, "friends_count": 795, "statues_count": 8223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405987529416706, "text": "@FrostedFrank yeah I didn't want a invite to in n out ��", "in_reply_to_status": 723405684797116417, "in_reply_to_user": 235369065, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 235369065 }}, "user": { "id": 417685562, "name": "God", "screen_name": "BryantChapel", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "Get out your feelings young nigga let's get it ❤️Sox 7-8", "followers_count": 982, "friends_count": 424, "statues_count": 124577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Park, CA", "id": "7b85d19c09869d20", "name": "Walnut Park", "place_type": "city", "bounding_box": rectangle("-118.230699,33.959223 -118.210743,33.974727") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 683402, "cityName": "Walnut Park" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405988217257984, "text": "Wind 1.0 mph WNW. Barometer 29.863 in, Steady. Temperature 58.4 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405988657664000, "text": "���� https://t.co/Hc0axbQceA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2784787212, "name": "SML", "screen_name": "sage_L22", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "Bad moon rising", "followers_count": 130, "friends_count": 178, "statues_count": 2086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405988796100608, "text": "Yep I think I got me a keeper ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 965124043, "name": "Alexandriaxoxo", "screen_name": "AlexArchuleta2", "lang": "en", "location": "null", "create_at": date("2012-11-22"), "description": "| Making my way back home to Cali one day at a time | Snapchat: Alexandriaxoxo2 |", "followers_count": 130, "friends_count": 145, "statues_count": 3985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405989026766848, "text": "yea nigga fuck Donald Trump", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21278217, "name": "FUEGO FLORES", "screen_name": "FUEGOFLORES", "lang": "en", "location": "Virginia, USA", "create_at": date("2009-02-18"), "description": "don't chase money chase yo passion #BlackLivesMatter", "followers_count": 994, "friends_count": 231, "statues_count": 150973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stone Ridge, VA", "id": "0026a6714f154760", "name": "Stone Ridge", "place_type": "city", "bounding_box": rectangle("-77.571292,38.9097 -77.540992,38.942057") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5175702, "cityName": "Stone Ridge" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405989647503360, "text": "I haven't had my homemade brownies in so long ��\nSmh I need to start baking again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423232790, "name": "Mama Assidy", "screen_name": "BigMelon_booty", "lang": "en", "location": "3 Job Shawty", "create_at": date("2011-11-27"), "description": "Official Co_Head Promoter For Royal Empire Ent. Ya girl was Born From a Boombox. I can dance better than I can walk.", "followers_count": 1577, "friends_count": 1257, "statues_count": 43332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405989685268480, "text": "Wind 0.0 mph WSW. Barometer 29.868 in, Steady. Temperature 56.1 °F. Rain today 0.05 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405989861433345, "text": "Temp 56.3° Hi/Lo 56.4/54.1 Rng 2.3° WC 56.3° Hmd 62% Rain 0.00\" Storm 0.00\" BAR 29.948 Falling DP 43.4° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 118, "statues_count": 18094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-22T00:00:02.000Z"), "id": 723405990306041856, "text": "@1Mauriez Available now on SPOTIFY, iTUNES, & APPLE MUSIC #BANKROLL\nhttps://t.co/vmtgyK4Crd", "in_reply_to_status": -1, "in_reply_to_user": 26917603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BANKROLL" }}, "user_mentions": {{ 26917603 }}, "user": { "id": 143445518, "name": "Stack Dollaz", "screen_name": "djstackdollaz", "lang": "en", "location": "Atlantic Station, Atlanta", "create_at": date("2010-05-13"), "description": "New single out now! #BANKROLL #BANKROLL Produced by XXXProductionz", "followers_count": 2544, "friends_count": 73, "statues_count": 6065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405990419255299, "text": "Ripley SW Limestone Co. Temp: 63.3°F Wind:6.9mph Pressure: 993.6mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405991241388033, "text": "61.9F (Feels: 61.9F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1009.9mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 236252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405991497195520, "text": "Wind 0.0 mph ESE. Barometer 29.901 in, Falling. Temperature 56.7 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405992466112512, "text": "always be my favorite player #ThankYouPavel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThankYouPavel" }}, "user": { "id": 1556852952, "name": "The Bautista Show", "screen_name": "_ericbautista", "lang": "en", "location": "null", "create_at": date("2013-06-29"), "description": "null", "followers_count": 203, "friends_count": 195, "statues_count": 2413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405992583524356, "text": "you're welcome ������������ https://t.co/cuULXToqtS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235816305, "name": "®.", "screen_name": "blacmattel", "lang": "en", "location": "da' factory ", "create_at": date("2011-01-08"), "description": "↞••••↢ ☾↣••••↠", "followers_count": 4142, "friends_count": 1070, "statues_count": 214962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405992612909057, "text": "Wind 0.0 mph SW. Barometer 29.94 in, Falling slowly. Temperature 60.8 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 118, "statues_count": 159242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405993099419649, "text": "Wind 0.0 mph W. Barometer 30.029 in, Rising slowly. Temperature 60.6 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 18, "friends_count": 54, "statues_count": 7901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405993107820545, "text": "I need me some of you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2269864902, "name": "¶layboi Ërick", "screen_name": "Epic_Erick", "lang": "en", "location": "California, USA", "create_at": date("2013-12-30"), "description": "God is who's going to guide you warrior", "followers_count": 362, "friends_count": 320, "statues_count": 10458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405993435009024, "text": "@sydneybachand @me��", "in_reply_to_status": 723341915282804736, "in_reply_to_user": 2376212539, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2376212539, 3291466538 }}, "user": { "id": 2943144060, "name": "Isaac", "screen_name": "IsaacXnegrete", "lang": "en", "location": "Highland, CA", "create_at": date("2014-12-25"), "description": "Rhs |", "followers_count": 511, "friends_count": 767, "statues_count": 763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405993602772992, "text": "@alyssaleilani_ https://t.co/mTBXBA9AFO", "in_reply_to_status": 723405660067532800, "in_reply_to_user": 332147865, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 332147865 }}, "user": { "id": 928855268, "name": "MG", "screen_name": "TheRealGee_Q", "lang": "en", "location": "W A V E L O R D", "create_at": date("2012-11-05"), "description": "SC michaelgalvan7 IG: TheRealGQ_13 •WAVY•", "followers_count": 732, "friends_count": 713, "statues_count": 7396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405993904742400, "text": "Hmm cold weather tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648416896, "name": "DANIEL", "screen_name": "Lackey28", "lang": "en", "location": "Oxnard, CA", "create_at": date("2013-08-05"), "description": "null", "followers_count": 652, "friends_count": 535, "statues_count": 7526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-04-22T00:00:03.000Z"), "id": 723405994571665408, "text": "@PhilShaw go back on flurry I'm bored", "in_reply_to_status": -1, "in_reply_to_user": 316262088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 316262088 }}, "user": { "id": 2756554852, "name": "tyra", "screen_name": "okoktyra", "lang": "en", "location": "enchanted forest", "create_at": date("2014-08-30"), "description": "I'd probably still adore you with your hands around my neck", "followers_count": 2086, "friends_count": 499, "statues_count": 44317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405994810732545, "text": "Need female friends in the east coast", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268892785, "name": "✨THE BLACK WIDOW ✨", "screen_name": "SOICEYTRAP", "lang": "en", "location": "NYC , LA ", "create_at": date("2011-03-19"), "description": "female rapper , fake model . ask for email", "followers_count": 4364, "friends_count": 527, "statues_count": 53246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clifton, NJ", "id": "7356b662670b2c31", "name": "Clifton", "place_type": "city", "bounding_box": rectangle("-74.197728,40.82028 -74.118533,40.899384") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3413690, "cityName": "Clifton" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405994819080194, "text": "He's a rapper from Florida lol I aint know of him til the beginning of last month�� https://t.co/nU8w0tI4DJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167434829, "name": "Man of The Year", "screen_name": "almightykenny", "lang": "en", "location": "Lost in the sauce", "create_at": date("2010-07-16"), "description": "UL | Taken | RIP Mom | IG: @kenny", "followers_count": 10506, "friends_count": 4603, "statues_count": 71478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405995460829186, "text": "@YahooSports I love the pics of a young Prince playing basketball.", "in_reply_to_status": 723389360532393984, "in_reply_to_user": 7302282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7302282 }}, "user": { "id": 336813148, "name": "WIZARD ENTERTAINMENT", "screen_name": "Wizardiris", "lang": "en", "location": "FRESNO, CALIFORNIA", "create_at": date("2011-07-16"), "description": "WIZARD ENTERTAINMENT AND SOUND.\r\nIRIS LEVESQUE, SECRETARY TO COACH JERRY TARKANIAN\r\nEVENTS, CLUBS, ARENAS,WEDDINGS, CORPORATE EVENTS,ETC\r\niris@wizardsound.com", "followers_count": 577, "friends_count": 2251, "statues_count": 5345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405995943194625, "text": "@mmunozzzz28 thats not the last time i seen you but thats like the last time we went somewhere", "in_reply_to_status": 723405131228712961, "in_reply_to_user": 2413594934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2413594934 }}, "user": { "id": 347687895, "name": "Ruben$ito", "screen_name": "Ruben__5", "lang": "en", "location": "The Springs", "create_at": date("2011-08-02"), "description": "18.OpM.", "followers_count": 764, "friends_count": 573, "statues_count": 16272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405996270342144, "text": "Absolutely love this https://t.co/8SmEtNPXNE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1564713655, "name": "Sunnn☀", "screen_name": "Chloeherrera33", "lang": "en", "location": "snap me - chloeherrera33", "create_at": date("2013-07-02"), "description": "#SHSU19 ~ Gio", "followers_count": 346, "friends_count": 371, "statues_count": 7059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405996358397955, "text": "@taytristan95 for real though ��", "in_reply_to_status": 723405918315024385, "in_reply_to_user": 2395368608, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2395368608 }}, "user": { "id": 706123718, "name": "Shae ✨", "screen_name": "ShannonElaineeW", "lang": "en", "location": "Canyon, TX", "create_at": date("2012-07-19"), "description": "•Sass Master•", "followers_count": 470, "friends_count": 633, "statues_count": 11949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon, TX", "id": "309059ff6710946f", "name": "Canyon", "place_type": "city", "bounding_box": rectangle("-101.954673,34.960525 -101.878133,35.017533") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48381, "countyName": "Randall", "cityID": 4812532, "cityName": "Canyon" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405996672966656, "text": "https://t.co/eCtYNhd9Mu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 945239035, "name": "Light", "screen_name": "Teamsolomidmatt", "lang": "en", "location": "Dunwall ", "create_at": date("2012-11-12"), "description": "I love God, people, video games,Music, anime, sports, food..... you get the gist. Victoria kirk❤️Cedar 1 #TeamYunqSuave", "followers_count": 861, "friends_count": 692, "statues_count": 18952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Graham, WA", "id": "08a26cd724eca51d", "name": "Graham", "place_type": "city", "bounding_box": rectangle("-122.338034,47.016521 -122.206872,47.09723") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5327785, "cityName": "Graham" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405996740108292, "text": "People can forget what you said People can forget what you did But they'll always remember how you made them feel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1508768742, "name": "Jas Kooner", "screen_name": "Kooner00", "lang": "en", "location": "null", "create_at": date("2013-06-11"), "description": "null", "followers_count": 221, "friends_count": 195, "statues_count": 1652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405997406986240, "text": "Arcuri Raimonda, Lancaster, CA, 93534 - https://t.co/q3tH8auelt\n\n#arcuriraimonda #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.159483,34.689733"), "retweet_count": 0, "lang": "it", "is_retweet": false, "hashtags": {{ "arcuriraimonda", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 485, "friends_count": 483, "statues_count": 416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405997482467332, "text": "\"Compassion is an action word with no boundaries.\" �� -Prince", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69442159, "name": "ashley", "screen_name": "x_iceberg", "lang": "en", "location": "Boulder, Colorado", "create_at": date("2009-08-27"), "description": "live with compassion ;", "followers_count": 238, "friends_count": 110, "statues_count": 19918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405997608300544, "text": "@readraines95 are you walking through the drive thru lane?", "in_reply_to_status": 723403628145299456, "in_reply_to_user": 313775001, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313775001 }}, "user": { "id": 458062723, "name": "Jamie Yeates", "screen_name": "jamjammzz", "lang": "en", "location": "null", "create_at": date("2012-01-07"), "description": "| Texas A&M | HTX |", "followers_count": 272, "friends_count": 208, "statues_count": 2716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryan, TX", "id": "9ce0f1d2eb5998ce", "name": "Bryan", "place_type": "city", "bounding_box": rectangle("-96.439644,30.609214 -96.255918,30.722503") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4810912, "cityName": "Bryan" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405997688025089, "text": "@NikkiLucca happy birthday bro ��", "in_reply_to_status": -1, "in_reply_to_user": 177448927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177448927 }}, "user": { "id": 1545242275, "name": "augustine", "screen_name": "AuggieM", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "khs | #Patriots • #Snowman", "followers_count": 1533, "friends_count": 1206, "statues_count": 39834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405998031921152, "text": "MTSWM����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 320041545, "name": "ÐE$PEƦλÐØ", "screen_name": "OHmitchieOHmy", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-06-18"), "description": "12.23 ♑️ #NIU19 ............................................................ A woman whose heart is hollow.", "followers_count": 1075, "friends_count": 870, "statues_count": 13750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "DeKalb, IL", "id": "76cec30d4c553725", "name": "DeKalb", "place_type": "city", "bounding_box": rectangle("-88.792846,41.883102 -88.714573,41.972512") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17037, "countyName": "DeKalb", "cityID": 1719161, "cityName": "DeKalb" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405998254223360, "text": "@ArionCrenshaw just cause", "in_reply_to_status": 723405961356943362, "in_reply_to_user": 2763637588, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2763637588 }}, "user": { "id": 577353879, "name": "Aaliyah Crenshaw", "screen_name": "Aaliiyyaahhh", "lang": "en", "location": "Prairie View, TX", "create_at": date("2012-05-11"), "description": "Prairie View A&M University'19, Nursing Major. Austin✈️Houston.", "followers_count": 1358, "friends_count": 814, "statues_count": 24814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-22T00:00:04.000Z"), "id": 723405998623326210, "text": "Me & @kdc are drunk soooo Taco In A Bag just happened. �� https://t.co/oYGQkXvnsa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.686048,41.965091"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14114579 }}, "user": { "id": 12346782, "name": "Tom Hardly", "screen_name": "softjunebreeze", "lang": "en", "location": "Detroit, MI", "create_at": date("2008-01-16"), "description": "You think I’ll be the dark sky so you can be the star? I’ll swallow you whole. © Warsan Shire", "followers_count": 1739, "friends_count": 1506, "statues_count": 108698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taco In a Bag", "id": "0816f71b5ed66001", "name": "Taco In a Bag", "place_type": "poi", "bounding_box": rectangle("-87.6860481,41.9650909 -87.686048,41.965091") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723405999508316160, "text": "all roads lead to Indio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54729961, "name": "Stephanie Zee", "screen_name": "silly_starfish", "lang": "en", "location": "Los Angeles, California", "create_at": date("2009-07-07"), "description": "I've never been there but the brochure looks nice", "followers_count": 239, "friends_count": 306, "statues_count": 4520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Diamond Bar, CA", "id": "771f8196d3598fe8", "name": "Diamond Bar", "place_type": "city", "bounding_box": rectangle("-117.86323,33.959529 -117.767626,34.042345") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619192, "cityName": "Diamond Bar" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723405999692873731, "text": "HAPPY 21st MF BIRTHDAY TO MY BESTFRIEND, @TabithaNevarez �� Have an amazing day but stay safe. I love and miss you �� https://t.co/APCSnyZXYe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38076261 }}, "user": { "id": 361728708, "name": "❁", "screen_name": "SamanthaRaeee24", "lang": "en", "location": "CA", "create_at": date("2011-08-25"), "description": "Tables turn, bridges burn", "followers_count": 448, "friends_count": 574, "statues_count": 13611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406000112340993, "text": "@hollyrpeete \"Could you be the most beautiful girl in the world?\"\n- Prince Rogers Nelson \nI know how much you love that song.", "in_reply_to_status": -1, "in_reply_to_user": 30274144, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30274144 }}, "user": { "id": 98439664, "name": "Damon Roberson", "screen_name": "droberson10", "lang": "en", "location": "Baltimore, Maryland", "create_at": date("2009-12-21"), "description": "I'm 43 years old. I don't tolerate negative people. I enjoy sci-fi, superheroes, anime, and video games; and I have autism. Asperger's that is.", "followers_count": 389, "friends_count": 1277, "statues_count": 846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406000212955137, "text": "Im taking so many Ls tonight. This girl just dogged my entire sex life. Whet?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36188267, "name": "Bre (Vibez)", "screen_name": "brevibez", "lang": "en", "location": "Bre WORLD", "create_at": date("2009-04-28"), "description": "Bre. 20.Lyricist. Cleveland HTS. #Vibrationz #NovaNauts Contact me: brevibez@gmail.com. Cudi Gang.", "followers_count": 789, "friends_count": 917, "statues_count": 15340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland Heights, OH", "id": "aa7defe13028d41f", "name": "Cleveland Heights", "place_type": "city", "bounding_box": rectangle("-81.603358,41.482742 -81.529651,41.545274") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916014, "cityName": "Cleveland Heights" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406000347189248, "text": "Like I LOVE kids but having my own ������������ nah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46576572, "name": "❥hayley", "screen_name": "princesshayleyy", "lang": "en", "location": "SEA", "create_at": date("2009-06-11"), "description": "take me to the sunset ❁", "followers_count": 445, "friends_count": 212, "statues_count": 34895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Palos Verdes, CA", "id": "905627646db90fef", "name": "Rancho Palos Verdes", "place_type": "city", "bounding_box": rectangle("-118.418623,33.722004 -118.301098,33.795538") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 659514, "cityName": "Rancho Palos Verdes" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406001022464001, "text": "I just updated my shit on the guys ���� https://t.co/T9sMYatus6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24445668, "name": "312C.O.O.", "screen_name": "ArieTheHomie", "lang": "en", "location": "Chicago", "create_at": date("2009-03-14"), "description": "Section 312 #UB4L #EvilRegals #SwanQueen Rest In Peace Cassius E. White", "followers_count": 792, "friends_count": 1109, "statues_count": 67697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, IL", "id": "013b9d696e5665d0", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-89.416938,40.08307 -89.330625,40.168573") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17107, "countyName": "Logan", "cityID": 1743536, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406001328648192, "text": "@Allegra__Nicole you gotta work ��", "in_reply_to_status": 723405728388521984, "in_reply_to_user": 545642749, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545642749 }}, "user": { "id": 31638022, "name": "diamond.", "screen_name": "_Brilyante", "lang": "en", "location": "usvi.", "create_at": date("2009-04-15"), "description": "#VState19. #BeyHive & you can't sit with us.", "followers_count": 2230, "friends_count": 1552, "statues_count": 102459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valdosta, GA", "id": "5e1c91065bc30991", "name": "Valdosta", "place_type": "city", "bounding_box": rectangle("-83.374825,30.762483 -83.230199,30.949686") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1378800, "cityName": "Valdosta" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406001483878400, "text": "Are you gonna be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2874731375, "name": "Eskimo J", "screen_name": "Jay_Jataa", "lang": "en", "location": "Starkville, MS; Pontotoc, MS", "create_at": date("2014-11-12"), "description": "Mississippi State University 16 #PHA /G\\", "followers_count": 550, "friends_count": 756, "statues_count": 14960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Starkville, MS", "id": "ed1736d76c1e007c", "name": "Starkville", "place_type": "city", "bounding_box": rectangle("-88.866974,33.406706 -88.776403,33.506302") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28105, "countyName": "Oktibbeha", "cityID": 2870240, "cityName": "Starkville" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406001555181570, "text": "Just saw on Facebook some guy named his son Skeet �� wtf why lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368541992, "name": "Andrew", "screen_name": "andreaux_a", "lang": "en", "location": "that ultralight beam", "create_at": date("2011-09-05"), "description": "UCR. stay humble✌", "followers_count": 316, "friends_count": 286, "statues_count": 28674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406001999769600, "text": "@KvngScott https://t.co/dKzh3sfZt6", "in_reply_to_status": 723385635927523328, "in_reply_to_user": 127625220, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 127625220 }}, "user": { "id": 328558133, "name": "toki-e", "screen_name": "baddietoks", "lang": "en", "location": "lagos - houston ", "create_at": date("2011-07-03"), "description": "excited, activated get ignited", "followers_count": 1838, "friends_count": 868, "statues_count": 60387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Katy, TX", "id": "c484d6c5cd72beb0", "name": "Katy", "place_type": "city", "bounding_box": rectangle("-95.87417,29.736773 -95.774936,29.831219") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4838476, "cityName": "Katy" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406002016526339, "text": "patiently waiting on 7am ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397785999, "name": "Khadijah Lashay", "screen_name": "K_Lashayyy", "lang": "en", "location": "null", "create_at": date("2011-10-24"), "description": "M x K Taken Humble", "followers_count": 1390, "friends_count": 893, "statues_count": 77854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406002486288384, "text": "Killed my vibe ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415140338, "name": "Victoria", "screen_name": "Vicky_Toriaaa", "lang": "en", "location": "htx", "create_at": date("2011-11-17"), "description": "✨19 & blessed✨", "followers_count": 496, "friends_count": 334, "statues_count": 13921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406002616307712, "text": "same af https://t.co/V2bXv6Ju3s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317559432, "name": "2am conner", "screen_name": "IHATN", "lang": "en", "location": "texas", "create_at": date("2011-06-14"), "description": "my ultimate goal is to go out in public and see a random wearing one of my cool ass hats (:", "followers_count": 3286, "friends_count": 743, "statues_count": 208093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-22T00:00:05.000Z"), "id": 723406002670858240, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Steady. Temperature 64.9 °F. Rain today 0.46 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406003081863168, "text": "الحيا والخجل. أمر مؤسف وصل لمرحلة الانقراض. #شي_مات_في_مجتمعنا", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.441002,34.059712"), "retweet_count": 0, "lang": "ar", "is_retweet": false, "hashtags": {{ "شي_مات_في_مجتمعنا" }}, "user": { "id": 1423897800, "name": "KSA", "screen_name": "1iSaudi", "lang": "en", "location": "null", "create_at": date("2013-05-12"), "description": "‏‏‏‏ الله ثم الملك ثم الوطن اللهم إنا نستودعك ملكنا وجنودنا | نحن مستهدفين في ديننا وبلادنا وشبابنا | دائماً وأبداً الوطن أولاً ..", "followers_count": 468, "friends_count": 499, "statues_count": 416 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Legacy at Westwood", "id": "07d9d06ecc886000", "name": "Legacy at Westwood", "place_type": "poi", "bounding_box": rectangle("-118.44100209999999,34.059711899999996 -118.441002,34.059712") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406003375509504, "text": "Let me explain how I wash my clothes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3876466700, "name": "Audrey Nguyen", "screen_name": "audidasu", "lang": "en", "location": "Houston, TX", "create_at": date("2015-10-12"), "description": "UH '20 | utmb research | professional crawfish critic & quirk fanatic", "followers_count": 41, "friends_count": 59, "statues_count": 439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greatwood, TX", "id": "6ef928bfe164fa3c", "name": "Greatwood", "place_type": "city", "bounding_box": rectangle("-95.74722,29.535478 -95.637861,29.583376") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4830806, "cityName": "Greatwood" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406003585216512, "text": "La cabeza me quiere explotar", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 985420669, "name": "BJB ✌⚾", "screen_name": "BJBenitez16", "lang": "es", "location": "Des Plaines, IL", "create_at": date("2012-12-02"), "description": "Josué 1: 8-9 \n2|20|2013\nOakton Community College", "followers_count": 522, "friends_count": 366, "statues_count": 49980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Plaines, IL", "id": "f04813ea26f51186", "name": "Des Plaines", "place_type": "city", "bounding_box": rectangle("-87.950838,41.991443 -87.83701,42.080839") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1719642, "cityName": "Des Plaines" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406004038168576, "text": "Fuck it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 541203207, "name": "✨Jayla Wayla✨", "screen_name": "Jayllaaa_", "lang": "en", "location": "TEXAS", "create_at": date("2012-03-30"), "description": "Just Trynna make somethin outta nothin | SENIOR '16 | 281-330-8004", "followers_count": 1087, "friends_count": 1111, "statues_count": 32965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ennis, TX", "id": "054df5106ee9caf2", "name": "Ennis", "place_type": "city", "bounding_box": rectangle("-96.667447,32.289048 -96.571268,32.368525") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4824348, "cityName": "Ennis" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406004751196160, "text": "I'm gonna start free balling all the time. I'm just afraid of my dick getting all hard in inappropriate places. I get horny a lot lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3009165881, "name": "nicca", "screen_name": "nikolleye", "lang": "en", "location": "North Carolina, USA", "create_at": date("2015-01-31"), "description": "i am gemini. i am versatility.", "followers_count": 2569, "friends_count": 2168, "statues_count": 2522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, NC", "id": "52cd3346625199d1", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-79.556369,36.028627 -79.379255,36.127415") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37001, "countyName": "Alamance", "cityID": 3709060, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406004923158528, "text": "all I do is cry and complain because second's not the same.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411777996, "name": "Abraham Mendez", "screen_name": "Broham95", "lang": "en", "location": "null", "create_at": date("2011-11-13"), "description": "Loser & a crybaby", "followers_count": 104, "friends_count": 192, "statues_count": 2019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avondale, AZ", "id": "0015d9147cee6907", "name": "Avondale", "place_type": "city", "bounding_box": rectangle("-112.357999,33.384785 -112.272424,33.493806") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 404720, "cityName": "Avondale" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406005195808769, "text": "@KrisJenner if i were high with Kim&God this would turn me on. To see Kim�� sucking a dick while i have mine in her���� https://t.co/5FisPdA2hv", "in_reply_to_status": -1, "in_reply_to_user": 23613479, "favorite_count": 0, "coordinate": point("-118.349428,34.097885"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23613479 }}, "user": { "id": 3265715965, "name": "CurseOfHAM-HAMisPIG!", "screen_name": "BlackDeath6371", "lang": "en", "location": "Calabasas, CA", "create_at": date("2015-07-01"), "description": "God is ending the #BadRace #EndOfBlackPigDogWormRace!!! #BlackPeopleAreUglyByDefault!!! #TrustMyTeeth71 #IamDaSUNGOD☆ #LMFAO #HAHAHA #JAJAJA #KAKAKA #LOL #BURN!", "followers_count": 94, "friends_count": 51, "statues_count": 7855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pinkberry", "id": "07d9f31c61c88001", "name": "Pinkberry", "place_type": "poi", "bounding_box": rectangle("-118.3494281,34.0978849 -118.349428,34.097885") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406005350981632, "text": "Seeing my 1st godson come back to ministry for the first time in 3 years was the highlight of my night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3152386165, "name": "Elmer Itchon", "screen_name": "eaitchon", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "null", "followers_count": 134, "friends_count": 118, "statues_count": 263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406005460070401, "text": "Is @hovoxoligan's bf okay? I'm a little worried after watching her snap story ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156432759 }}, "user": { "id": 48623858, "name": "Holly Golightly", "screen_name": "anamarbxo", "lang": "en", "location": "Searching for enlightenment", "create_at": date("2009-06-18"), "description": "Lover. Yogi.", "followers_count": 650, "friends_count": 226, "statues_count": 45560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellwood, IL", "id": "c226bc3c790795d8", "name": "Bellwood", "place_type": "city", "bounding_box": rectangle("-87.898589,41.870812 -87.858873,41.893069") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1704975, "cityName": "Bellwood" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406006311473152, "text": "@Flawless_gifts م يصطلبن ياخ", "in_reply_to_status": 723405629306470401, "in_reply_to_user": 721884059727753216, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 721884059727753216 }}, "user": { "id": 436767664, "name": "الامير النائم ➰", "screen_name": "Sul6an_96", "lang": "en", "location": "Virginia, USA", "create_at": date("2011-12-14"), "description": "from Abu Dhabi/Dubai / living that tajneed life instagram/ its_sal6oon", "followers_count": 3304, "friends_count": 766, "statues_count": 67549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrifield, VA", "id": "d9444ad74f3ab7f4", "name": "Merrifield", "place_type": "city", "bounding_box": rectangle("-77.274342,38.86503 -77.217659,38.890139") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5151192, "cityName": "Merrifield" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406006378602496, "text": "@fvck_bob_saget @iAmNelyk @CJSTR013 ������������", "in_reply_to_status": 723405409172623360, "in_reply_to_user": 2281688569, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2281688569, 3253075809, 1447592970 }}, "user": { "id": 3100186525, "name": "crock", "screen_name": "ItsCrock_", "lang": "en", "location": "Ducked Off", "create_at": date("2015-03-20"), "description": "Photographer for Jetsonz Music Media. Musician. Business Inquiries : bookinginfojcrock@gmail.com", "followers_count": 1642, "friends_count": 1172, "statues_count": 22807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alliance, OH", "id": "f3d62daa712bbb93", "name": "Alliance", "place_type": "city", "bounding_box": rectangle("-81.162586,40.872386 -81.062267,40.962933") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3901420, "cityName": "Alliance" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406006617706496, "text": "MPBSW����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 4731092598, "name": "July16", "screen_name": "prettylilron", "lang": "en", "location": "null", "create_at": date("2016-01-08"), "description": "hating hoes ain't happy happy hoes ain't hating.", "followers_count": 433, "friends_count": 374, "statues_count": 3152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-22T00:00:06.000Z"), "id": 723406006730924032, "text": "@catrific they still out to get me, they don't get it...greatest two bars ever?", "in_reply_to_status": 723392115019927552, "in_reply_to_user": 15993167, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15993167 }}, "user": { "id": 40276769, "name": "Brian Miggels", "screen_name": "BMiggs", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-05-15"), "description": "Community & Studio Brand Strategist, @SHGames.", "followers_count": 3971, "friends_count": 534, "statues_count": 2957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768365982994432, "text": "@NYRangersZone was there tonight..2nd period was amazing.. Then the Sharks got hungry and stayed that way.. Just someone best the caps pls!", "in_reply_to_status": 723744036465438720, "in_reply_to_user": 429468742, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429468742 }}, "user": { "id": 47451835, "name": "Christine Larson", "screen_name": "NYRangerGirl12", "lang": "en", "location": "Los Angeles", "create_at": date("2009-06-15"), "description": "New York girl at heart ~ paranormal investigator ~ hopeless romantic ~ I love hockey, heavy metal and travelling", "followers_count": 578, "friends_count": 499, "statues_count": 3846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768366264123393, "text": "I can stand a corny ass person. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 856940612, "name": "Mary", "screen_name": "marrrybbb", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-10-01"), "description": "SC: Recklesslymary", "followers_count": 877, "friends_count": 320, "statues_count": 42039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altamonte Springs, FL", "id": "e40595796da14879", "name": "Altamonte Springs", "place_type": "city", "bounding_box": rectangle("-81.443607,28.639994 -81.343699,28.689613") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1200950, "cityName": "Altamonte Springs" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768366909902848, "text": "I've always had a thing for guys that wear glasses", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 904799502, "name": "Alora✨", "screen_name": "A_TizzleYo", "lang": "en", "location": "Indio, CA", "create_at": date("2012-10-25"), "description": "ig/sc: missypink11", "followers_count": 1519, "friends_count": 1107, "statues_count": 49743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768367190925312, "text": "Everyday you're alive is another step closer to recovery. The road can be long, but having patience will be enough.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258202891, "name": "MaKayla.", "screen_name": "Ehmsssss", "lang": "en", "location": "Salt Lake City, UT", "create_at": date("2011-02-26"), "description": "meow.", "followers_count": 632, "friends_count": 468, "statues_count": 14568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grantsville, UT", "id": "d74234f0f6d9d6b8", "name": "Grantsville", "place_type": "city", "bounding_box": rectangle("-112.503828,40.563399 -112.421893,40.621603") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49045, "countyName": "Tooele", "cityID": 4931120, "cityName": "Grantsville" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768367367122944, "text": "@IsaacPantke I remember when we had a candle lit dinner", "in_reply_to_status": 723725515882786816, "in_reply_to_user": 607540999, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 607540999 }}, "user": { "id": 2346573222, "name": "Adam's slave", "screen_name": "JEakett", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "pj❤️. Las Vegas storm hockey", "followers_count": 657, "friends_count": 677, "statues_count": 3653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768367929253889, "text": "Temp 59.7° Hi/Lo 60.4/59.3 Rng 1.1° WC 59.7° Hmd 97% Rain 0.05\" Storm 0.20\" BAR 29.801 Falling DP 58.8° Wnd 0mph Dir --- Gst 5mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 118, "statues_count": 18121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768368201916418, "text": "@sweet_fanta u really didn't", "in_reply_to_status": 723753579299901440, "in_reply_to_user": 2456304671, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2456304671 }}, "user": { "id": 2942007750, "name": "BOSSMAN DonTana", "screen_name": "_Don2x", "lang": "en", "location": "N Block 614", "create_at": date("2014-12-24"), "description": "The world is yours R.I.P Snupe #FreeMedBoss #Y.I.C. R.I.P LIL BRO BOSSMAN Inshalla we will meet again", "followers_count": 499, "friends_count": 384, "statues_count": 7652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768368256307200, "text": "Temp: 52.3F W C: 50.7F Wind:N at 4.8kts Baro: 1022.1mb and Rising slowly Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 208, "friends_count": 219, "statues_count": 103047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768368562466817, "text": "when he falls asleep while talking on the phone ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1215935876, "name": "✨", "screen_name": "ChulaKayla", "lang": "en", "location": "null", "create_at": date("2013-02-24"), "description": "null", "followers_count": 542, "friends_count": 565, "statues_count": 16181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768369053196288, "text": "Y no le pido al cielo mas castigo para ti. Por que Estas durmiendo con Ella y soñando conmigo.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 878687234, "name": "Erendida Saldaña", "screen_name": "erendida_saldaa", "lang": "en", "location": "La Luna", "create_at": date("2012-10-13"), "description": "Daughter☄Sister☄Aunt☄Friend☄Pet Lover☄Self-employed☄Latina☄Hopeless romantic☄God is my savior☄Advocate for Animals rights☄Fashion lover☄his favorite", "followers_count": 59, "friends_count": 138, "statues_count": 572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-04-23T00:00:00.000Z"), "id": 723768369145507844, "text": "https://t.co/1HXA3szzhr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 872554621, "name": "JAYKIE JAKE", "screen_name": "itsjacob3", "lang": "en", "location": "Roma, TX ", "create_at": date("2012-10-10"), "description": "Life can only be lived forwards but only understood backwards", "followers_count": 787, "friends_count": 361, "statues_count": 26696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roma, TX", "id": "01e0e9a5007db306", "name": "Roma", "place_type": "city", "bounding_box": rectangle("-99.032671,26.368658 -98.887064,26.433349") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4863020, "cityName": "Roma" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768370303094784, "text": "���� https://t.co/hAwGCC2zKC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 894763399, "name": "Aderrial", "screen_name": "AderrialH", "lang": "en", "location": "Midland, Texas", "create_at": date("2012-10-20"), "description": "I seem cool, but I'm really a lame. bitsm", "followers_count": 575, "friends_count": 484, "statues_count": 11295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midland, TX", "id": "a3df7faf80ceea6b", "name": "Midland", "place_type": "city", "bounding_box": rectangle("-102.229316,31.923379 -101.979938,32.065221") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48329, "countyName": "Midland", "cityID": 4848072, "cityName": "Midland" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768370579931137, "text": "How mine and Rebzys join funeral is going to go down.\nEveryone will be sad.\nThen the pastor will read my dying wish:\nPlay lane boy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1287875778, "name": "LAN(EMM) BO(Y)", "screen_name": "907bis", "lang": "en", "location": "FusedattheWrist ", "create_at": date("2013-03-21"), "description": "+ @tylerrjoseph just told me: YOU CAN KEEP YOUR SOUL AND TYLER WE ALL KNOW YOU KNOWNWHAT ACOUSTIC IS JUST DO IT AND DO IT WITH JOSH ☀️ LINK IN BIO", "followers_count": 2584, "friends_count": 1314, "statues_count": 50267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768370697515008, "text": "04/23@03:00 - Temp 63.9F, WC 63.9F. Wind 1.1mph SSW, Gust 4.0mph. Bar 29.811in, Falling. Rain 0.00in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768371670446080, "text": "@JayceLeslie @DaniellaCaesare @SofiaGabriella_ @AlexandraReneee @smariepierce @glitter_cunt you didn't pull thru", "in_reply_to_status": 723765854228242432, "in_reply_to_user": 4265875273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4265875273, 2197932241, 196049547, 215822080, 1462809944, 1541830136 }}, "user": { "id": 2793718383, "name": "allie❥", "screen_name": "Rogers14Allie", "lang": "en", "location": "null", "create_at": date("2014-09-29"), "description": "heard that.", "followers_count": 182, "friends_count": 270, "statues_count": 1409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petaluma, CA", "id": "239aa72871ae24ab", "name": "Petaluma", "place_type": "city", "bounding_box": rectangle("-122.682383,38.205307 -122.584356,38.283601") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 656784, "cityName": "Petaluma" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768371745976320, "text": "@iThrashHoe_ I'll Pull Up", "in_reply_to_status": 723768131890470913, "in_reply_to_user": 392490047, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 392490047 }}, "user": { "id": 302047041, "name": "#SunnySide_Jon", "screen_name": "_Thrashtastic", "lang": "en", "location": "Sunnyside, Houston", "create_at": date("2011-05-20"), "description": "SC: JöñPøppï", "followers_count": 1069, "friends_count": 718, "statues_count": 34851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768372647759872, "text": "@ebbtideapp Tide in Ocean City, Maryland 04/23/2016\n Low 3:27am 0.0\nHigh 9:31am 2.2\n Low 3:28pm 0.2\nHigh 9:48pm 2.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.09,38.3317"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 79, "friends_count": 1, "statues_count": 26794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean City, MD", "id": "759c5982d28a7b07", "name": "Ocean City", "place_type": "city", "bounding_box": rectangle("-75.091333,38.32431 -75.04914,38.451299") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24047, "countyName": "Worcester", "cityID": 2458225, "cityName": "Ocean City" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768372723355648, "text": "Wind 0.0 mph ---. Barometer 30.105 in, Steady. Temperature 53.3 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-23T00:00:01.000Z"), "id": 723768373041979393, "text": "I think there's a ghost that's fighting me during the night because I keep getting random bruises", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119552624, "name": "#NTXvSTX", "screen_name": "The_Regulator44", "lang": "en", "location": "Texas", "create_at": date("2010-03-03"), "description": "#Texans #FarmTeam #SFA17 Its LIT!! . jj_thatrillest on Snapchat", "followers_count": 205, "friends_count": 138, "statues_count": 10047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374157668352, "text": "Should've have went all the way back there ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2277362852, "name": ":)", "screen_name": "al0ndraramirez_", "lang": "en", "location": "j u s t i n b i e b e r ♡", "create_at": date("2014-01-05"), "description": "Omar Salcedo ♏️", "followers_count": 2077, "friends_count": 2009, "statues_count": 28051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Camp, CA", "id": "223dd5d8e83a71b8", "name": "French Camp", "place_type": "city", "bounding_box": rectangle("-121.305096,37.855527 -121.264446,37.897212") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 626028, "cityName": "French Camp" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374531076096, "text": "Wind 0.0 mph NNW. Barometer 30.025 in, Rising slowly. Temperature 54.3 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374740688897, "text": "IM 18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 702861890465050624, "name": "NAY~Z", "screen_name": "nayliacostaa", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2016-02-25"), "description": "null", "followers_count": 318, "friends_count": 318, "statues_count": 4487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374862311425, "text": "Watch Prince’s Final Performance of ‘Purple Rain’ https://t.co/3wDmKaVhfA via @UsWeekly #amazing #prince #itslikeheknew ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "amazing", "prince", "itslikeheknew" }}, "user_mentions": {{ 20012204 }}, "user": { "id": 411222191, "name": "michelle gilchrist", "screen_name": "mrs__gilchrist", "lang": "en", "location": "HOUSTON, TEXAS ", "create_at": date("2011-11-12"), "description": "@will_gilchristt MOM•Tony's wife• Reality Junkie• Friend•SOS•Teacher•Sports Fan•Animal Lover• #Fibromyalgia #Arthritis #FootballMom #OLineMom #BlinnMom", "followers_count": 310, "friends_count": 912, "statues_count": 2120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374912643074, "text": "But she right lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2297190727, "name": "p", "screen_name": "prrlz", "lang": "en", "location": "Long Beach", "create_at": date("2014-01-17"), "description": "⚽️", "followers_count": 363, "friends_count": 98, "statues_count": 24131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768374971502592, "text": "The robe. Honestly. #TheOtherPrince https://t.co/d8YunpCYYf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheOtherPrince" }}, "user": { "id": 17821643, "name": "Jon Carroll", "screen_name": "jdcarroll88", "lang": "en", "location": "MA/DC/VA/MD Etc.", "create_at": date("2008-12-02"), "description": "Musician, writer etc. I've yack and opinion steering. I mostly follow journalists and comics. Bless 'em. http://joncarroll.org http://joncarroll.bandcamp.com", "followers_count": 706, "friends_count": 1340, "statues_count": 6106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northampton, MA", "id": "c27aeb5a2089da3c", "name": "Northampton", "place_type": "city", "bounding_box": rectangle("-72.729839,42.28419 -72.615605,42.380065") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25015, "countyName": "Hampshire", "cityID": 2546330, "cityName": "Northampton" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768375025930240, "text": "I wish I knew what the right decision was", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 425438452, "name": "ㄒ₳ㄚㄚ", "screen_name": "TayyShyy", "lang": "en", "location": "Reno, NV", "create_at": date("2011-11-30"), "description": "6 shots of espresso minimum", "followers_count": 433, "friends_count": 787, "statues_count": 18562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768375084765185, "text": "@jadaterrio : shatara sing me to sleep \n@shat_stet : no all these bitches won't shut the fuck up", "in_reply_to_status": -1, "in_reply_to_user": 1113016086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1113016086, 3431552183 }}, "user": { "id": 953857375, "name": "Carla Woznick", "screen_name": "carla_2456", "lang": "en", "location": "603", "create_at": date("2012-11-17"), "description": "queen", "followers_count": 1066, "friends_count": 1589, "statues_count": 10226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, NH", "id": "7b179e1b1fdd80d0", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-71.622585,43.159703 -71.462538,43.295436") }, "geo_tag": { "stateID": 33, "stateName": "New Hampshire", "countyID": 33013, "countyName": "Merrimack", "cityID": 3314200, "cityName": "Concord" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768375311237121, "text": "Boss my throat lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 729382134, "name": "Alaina", "screen_name": "_SimplyLaurie", "lang": "en", "location": "Antigua and Barbuda :)", "create_at": date("2012-07-31"), "description": "Wadadli Pikney|| #ColeWorld #USF18", "followers_count": 522, "friends_count": 407, "statues_count": 21755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tampa, FL", "id": "dc62519fda13b4ec", "name": "Tampa", "place_type": "city", "bounding_box": rectangle("-82.620093,27.821353 -82.265295,28.171836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271000, "cityName": "Tampa" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768375441252353, "text": "@ryuuhibi it's a might. Hard to say. Megan is a very big source of inspiration.", "in_reply_to_status": 723768186450092032, "in_reply_to_user": 4865684170, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4865684170 }}, "user": { "id": 140823673, "name": "チョロ松ZOMBIE", "screen_name": "zombie_boogie", "lang": "en", "location": "Chicago, IL", "create_at": date("2010-05-06"), "description": "(Kelly/Zombie) ☆pregnantzombie-ao3☆ I put the romance back in necromancer♡ R-18! BL松 OK, ENG OK, 日本語ギリギリOK 《○I complain a lot/たくさん文句だよ○》header:@trashy_sin", "followers_count": 419, "friends_count": 458, "statues_count": 25901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768376233861120, "text": "@SanneBorsti @elke5180 @DutchSassenach Wow! She's gorgeous, isn't she? I'm so happy she's our Claire. Your edit is terrific!!! ������", "in_reply_to_status": 723749337772204032, "in_reply_to_user": 3001107519, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3001107519, 2853128476, 3200956623 }}, "user": { "id": 25093882, "name": "Bonnie Klein", "screen_name": "SkinnyBon", "lang": "en", "location": "Dallas, TX USA", "create_at": date("2009-03-18"), "description": "Cancer survivor, widow, mom, minister, teacher, counselor, Christian mystic, writer, lover of good books, Outlander fanatic, hopeless romantic", "followers_count": 1726, "friends_count": 3115, "statues_count": 4811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768376548569088, "text": "Getting ready to leave Miami ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362203677, "name": "HER. ✨", "screen_name": "_Tamerre_", "lang": "en", "location": "On The Way To Success❤️", "create_at": date("2011-08-25"), "description": "Chapter 16 ❤️ 2017 My Year ❤️DUCKED ❤️✌️", "followers_count": 465, "friends_count": 445, "statues_count": 45659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768376586194944, "text": "Well ok then", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3845630786, "name": "KRYSTAL", "screen_name": "Knxfjx", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-10-10"), "description": "null", "followers_count": 42, "friends_count": 110, "statues_count": 841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768376590405632, "text": "beers w/ my mom are always the best ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46455101, "name": "mamacita", "screen_name": "wildhaze_", "lang": "en", "location": "Alhambra, CA", "create_at": date("2009-06-11"), "description": "beer me", "followers_count": 605, "friends_count": 441, "statues_count": 32288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alhambra, CA", "id": "d563cf7307e4ba95", "name": "Alhambra", "place_type": "city", "bounding_box": rectangle("-118.165119,34.05998 -118.108233,34.111213") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600884, "cityName": "Alhambra" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768377219538944, "text": "Temp: 57.3°F Wind:0.0mph Pressure: 30.004hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768377810939904, "text": "61.8F (Feels: 61.8F) - Humidity: 91% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1012.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 236393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768378154991617, "text": "Wind 1.6 mph NNE. Barometer 29.85 in, Steady. Temperature 60.3 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 118, "statues_count": 159266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-23T00:00:02.000Z"), "id": 723768378159046656, "text": "صباح الخير أحبائنا ..\nالأردن \nعمان https://t.co/s0wn2HYQMq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2187447985, "name": "KAMAL", "screen_name": "kamalthefunman", "lang": "en", "location": "Marina del Rey, CA USA", "create_at": date("2013-11-10"), "description": "male", "followers_count": 3258, "friends_count": 5009, "statues_count": 53596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marina del Rey, CA", "id": "38ea9782c8d83a4b", "name": "Marina del Rey", "place_type": "city", "bounding_box": rectangle("-118.463481,33.964019 -118.432199,33.98647") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645806, "cityName": "Marina del Rey" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768378704318464, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3231911988, "name": "Hannah Patton", "screen_name": "hannahxxpatton", "lang": "en", "location": "null", "create_at": date("2015-05-31"), "description": "engaged to my best friend @nscott755", "followers_count": 105, "friends_count": 112, "statues_count": 3674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379035684864, "text": "@LK3185 shhhh lol", "in_reply_to_status": 723765797898723328, "in_reply_to_user": 155437977, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155437977 }}, "user": { "id": 319365044, "name": "kerry", "screen_name": "minniepearl74", "lang": "en", "location": "MA", "create_at": date("2011-06-17"), "description": "love music new england sports soaps ship many couples co host @soapparty411", "followers_count": 1374, "friends_count": 1738, "statues_count": 46140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fall River, MA", "id": "1496f43fd8a3341f", "name": "Fall River", "place_type": "city", "bounding_box": rectangle("-71.195676,41.603903 -71.101605,41.768298") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2523000, "cityName": "Fall River" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379417358337, "text": "i'm yelling bc he actually looked like this lmao!!!!!!! https://t.co/yVOYb9Ye0g", "in_reply_to_status": 723767773751447552, "in_reply_to_user": 238335327, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238335327, "name": "jasmine", "screen_name": "snowberger_", "lang": "en", "location": "austin, tx||unt19", "create_at": date("2011-01-14"), "description": "big fan of dogs & harry potter", "followers_count": 494, "friends_count": 323, "statues_count": 8286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379564240896, "text": "@bennymitch05 @autano34 if that had a little more lift and you caught it I would be beyond pissed.", "in_reply_to_status": 723768207987892224, "in_reply_to_user": 472446235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 472446235, 588612884 }}, "user": { "id": 290099520, "name": "AJ Bialota", "screen_name": "AJTheBasedGod", "lang": "en", "location": "Ohio University", "create_at": date("2011-04-29"), "description": "I'm just tryna' grow up old and rich. The Cubs are 13-4.", "followers_count": 653, "friends_count": 429, "statues_count": 13843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, OH", "id": "77aa8a0640caa98c", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-82.173479,39.27973 -82.01933,39.361038") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39009, "countyName": "Athens", "cityID": 3902736, "cityName": "Athens" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379673350144, "text": "Ripley SW Limestone Co. Temp: 57.9°F Wind:0.7mph Pressure: 995.7mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379853565952, "text": "late night gym session, building that muscle ������ #fitnessmotivation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.048064,33.916931"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fitnessmotivation" }}, "user": { "id": 30818731, "name": "bekz", "screen_name": "Bekahroo3", "lang": "en", "location": "null", "create_at": date("2009-04-12"), "description": "null", "followers_count": 50, "friends_count": 79, "statues_count": 425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "24 Hour Fitness", "id": "07d9f31cfe882002", "name": "24 Hour Fitness", "place_type": "poi", "bounding_box": rectangle("-118.04806409999999,33.9169309 -118.048064,33.916931") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768379908055040, "text": "16 Years Ago the most Legendary twins were born ������ #April23 idk who is who https://t.co/SVeXyw6MdC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "April23" }}, "user": { "id": 1473910915, "name": "Eye-Zick", "screen_name": "isaaac_11", "lang": "en", "location": "Hayward, CA", "create_at": date("2013-05-31"), "description": "http://youtu.be/sV6lyUdmRj4 Also ball up for Tecos ⚽️", "followers_count": 315, "friends_count": 442, "statues_count": 944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768380168134660, "text": "@txtrue @phil200269 @CarmineZozzora is changed to only allow abortions strictly to save the life of the mother, and make those that act", "in_reply_to_status": 723761125154672640, "in_reply_to_user": 793103334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 793103334, 1936536769, 2167674121 }}, "user": { "id": 2843858606, "name": "Robert", "screen_name": "rachal_robert", "lang": "en", "location": "null", "create_at": date("2014-10-07"), "description": "I often fall short of God's target, He forgives,I thank him for daily blessings. Its a badge of honor that obama and liberals hate my christianity. FREEDOM!", "followers_count": 2068, "friends_count": 2401, "statues_count": 25831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768380491079681, "text": "Better sweet moment of having to leave my Finishline team", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1616628666, "name": "Yesenia Jaimes", "screen_name": "_yesssssy", "lang": "en", "location": "null", "create_at": date("2013-07-23"), "description": "null", "followers_count": 221, "friends_count": 126, "statues_count": 3721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stanton, CA", "id": "35fea2c31d822962", "name": "Stanton", "place_type": "city", "bounding_box": rectangle("-118.017331,33.773894 -117.96762,33.81573") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 673962, "cityName": "Stanton" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768380587560961, "text": "Running w some steppas who might stretch yah if you trip .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228101868, "name": "Miguel", "screen_name": "Hvlfbaked", "lang": "en", "location": "The Ville ✈ Houston", "create_at": date("2010-12-18"), "description": "null", "followers_count": 874, "friends_count": 381, "statues_count": 16478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768380956676096, "text": "LA has been crazy honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219509941, "name": "Christopher", "screen_name": "DaaddyChris", "lang": "en", "location": "Wa✈️ Az ", "create_at": date("2010-11-24"), "description": "#ASU19", "followers_count": 1539, "friends_count": 964, "statues_count": 75360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381032124416, "text": "#FF: @TheAdvocateMag @TuckMusical @DW_Angel3 @DanielBishopXXX @_adriangarcia @TannerSharpXXX @gageowensxxx @KadenPorterBSB @KylePorterBSB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FF" }}, "user_mentions": {{ 21692297, 544437603, 776607248, 1552325318, 319335741, 795007183, 2704848943, 4004626599, 4004671018 }}, "user": { "id": 345548735, "name": "Chris Bright", "screen_name": "C_Bright1988", "lang": "en", "location": "South Bend,Indiana", "create_at": date("2011-07-30"), "description": "Loyal friend, college grad, political science nerd, LGBT activist, and family man. Uncensored thoughts beware.", "followers_count": 576, "friends_count": 2075, "statues_count": 19907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381149564928, "text": "feeling pretty shitty lately", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3164577576, "name": "drew", "screen_name": "DWaelde", "lang": "en", "location": "Murrieta, CA", "create_at": date("2015-04-19"), "description": "just livin life", "followers_count": 127, "friends_count": 208, "statues_count": 720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381250260992, "text": "@KOINNews how could I send you guys a picture from the crazy weather last night??", "in_reply_to_status": -1, "in_reply_to_user": 18228178, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18228178 }}, "user": { "id": 452895184, "name": "Drew", "screen_name": "DrewScott21", "lang": "en", "location": "Snapchat: scott_robinson", "create_at": date("2012-01-02"), "description": "PNW | Supervisor at Pumpkin Ridge Golf Club | It's nice to be important, but it's more important to be nice | Cancer♋️ | Proud owner of Zoey the 350z |", "followers_count": 553, "friends_count": 923, "statues_count": 33424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Grove, OR", "id": "beedf9d8e2499b64", "name": "Forest Grove", "place_type": "city", "bounding_box": rectangle("-123.15354,45.501953 -123.070256,45.542318") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4126200, "cityName": "Forest Grove" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381262848000, "text": "@destinygapultos DES!!!! �������������� thanks so much cutie Ilysm & I miss you with allll my soul bb ��✨", "in_reply_to_status": 723765705821147136, "in_reply_to_user": 382513755, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 382513755 }}, "user": { "id": 1363063656, "name": "heizel zuniga", "screen_name": "heizelclam", "lang": "en", "location": "sc/insta: heizelzuniga", "create_at": date("2013-04-18"), "description": "¿¿¿livin' la vida loca???", "followers_count": 688, "friends_count": 488, "statues_count": 8984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381275561985, "text": "Add me on Snapchat! Username: erm0ji https://t.co/KDB3ccd0RW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73936146, "name": "Amaury Diaz ''Moji''", "screen_name": "Amaury_Moji", "lang": "en", "location": "ÜT: 18.510047,-69.938284", "create_at": date("2009-09-13"), "description": "Dominican photographer in NY", "followers_count": 498, "friends_count": 311, "statues_count": 5713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768381292175360, "text": "@PedernalesBrew @browndist hosting #TapTakeOver Sat April 23. 6pm. \n@BrewStuds @AustinTx_RR @ATXBeerReleases https://t.co/LaDOkf0Cb9", "in_reply_to_status": -1, "in_reply_to_user": 599713893, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TapTakeOver" }}, "user_mentions": {{ 599713893, 102510460, 613497601, 2951568349, 1329540054 }}, "user": { "id": 1972120988, "name": "HCM GastroPub", "screen_name": "HCMGastroPub", "lang": "en", "location": "3701 Gattis School Rd, RR, TX.", "create_at": date("2013-10-19"), "description": "Gastro Pub Dinér offering Burgers, NYstyle Pizza, Sandwiches, B'fast Taco's. Conv' Market w 200+ Beers and Wines. Wine & Beer Bars w Craft Beers on tap.", "followers_count": 410, "friends_count": 743, "statues_count": 1326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768382181371904, "text": "Over here banging on doors and windows shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481529368, "name": "Mads", "screen_name": "MadisonObryan", "lang": "en", "location": "Wichita | El Dorado", "create_at": date("2012-02-02"), "description": "I'm not special", "followers_count": 4566, "friends_count": 536, "statues_count": 20248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado, KS", "id": "c8dcca76c8150e99", "name": "El Dorado", "place_type": "city", "bounding_box": rectangle("-96.905493,37.788334 -96.823135,37.850332") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20015, "countyName": "Butler", "cityID": 2020075, "cityName": "El Dorado" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768382303002625, "text": "Happy birthday to the stupid ass hope you have a good one!!����@RID3R23 https://t.co/SFNJmovDuW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 963342378 }}, "user": { "id": 3232017918, "name": "Joey", "screen_name": "joeyfrazier52", "lang": "en", "location": "null", "create_at": date("2015-05-31"), "description": "null", "followers_count": 110, "friends_count": 162, "statues_count": 293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-04-23T00:00:03.000Z"), "id": 723768382357561344, "text": "tonight was good ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702029556, "name": "breanna", "screen_name": "brreannaa_", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "it is what it is", "followers_count": 449, "friends_count": 264, "statues_count": 23616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Puente, CA", "id": "e04653ebc39b100c", "name": "La Puente", "place_type": "city", "bounding_box": rectangle("-117.98558,34.009223 -117.920899,34.057298") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640340, "cityName": "La Puente" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768382483521536, "text": "party was actually fun tn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274192928, "name": "Nyjil", "screen_name": "Nyjil_", "lang": "en", "location": "Dover, DE", "create_at": date("2011-03-29"), "description": "#DSU18. IG:NyjilH / SC:Nyjil01", "followers_count": 1366, "friends_count": 1181, "statues_count": 40062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dover, DE", "id": "2ca1e1d1d0fae614", "name": "Dover", "place_type": "city", "bounding_box": rectangle("-75.586247,39.108566 -75.449548,39.20982") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1021200, "cityName": "Dover" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768382844080129, "text": "@iHustle1108 You already know", "in_reply_to_status": 723767922821206020, "in_reply_to_user": 31245235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31245235 }}, "user": { "id": 257276567, "name": "Earl, Crewneck", "screen_name": "_IAintShitSon", "lang": "en", "location": "Mt. Silver 757", "create_at": date("2011-02-24"), "description": "You niggas can't be this fucking stupid. #DogHiveCEO", "followers_count": 5936, "friends_count": 2259, "statues_count": 235523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383066394626, "text": "Happy birthday to my bro @_franco5! Always got you my dude ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3287171010 }}, "user": { "id": 3152267286, "name": "christian..", "screen_name": "tromiemowgli", "lang": "en", "location": "null", "create_at": date("2015-04-12"), "description": "@alexacaldera ❤️", "followers_count": 246, "friends_count": 289, "statues_count": 547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383183822848, "text": "Come and hangout @HoloGrunt 's awesome stream https://t.co/yc7cM9n9bZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 479024488 }}, "user": { "id": 3198465138, "name": "Jorge Guerra", "screen_name": "mellemiumb", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-05-16"), "description": "null", "followers_count": 35, "friends_count": 103, "statues_count": 108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383494361092, "text": "I honestly don't mind this lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2530951676, "name": "Bri-ANA", "screen_name": "Briana_Ruiz", "lang": "en", "location": "Dallas, TX", "create_at": date("2014-05-28"), "description": "You can't stop the change anymore than you can stop the suns from setting-Shmi Skywalker NMHS CG captain #TSU20 #Tarleton20", "followers_count": 587, "friends_count": 421, "statues_count": 38973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murphy, MO", "id": "c79eed7858abe65e", "name": "Murphy", "place_type": "city", "bounding_box": rectangle("-90.529632,38.437456 -90.423923,38.541479") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson", "cityID": 2950834, "cityName": "Murphy" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383678738432, "text": "I will be up all night hungry but too lazy to go get food. #tragic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tragic" }}, "user": { "id": 39359380, "name": "Torrance Williams✨", "screen_name": "TAW2011", "lang": "en", "location": "ALABAMA", "create_at": date("2009-05-11"), "description": "The goal is to be better then you was yesterday. IG: Taw92 SC: Taw2011", "followers_count": 914, "friends_count": 924, "statues_count": 15904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anniston, AL", "id": "48c472deb9cb7d02", "name": "Anniston", "place_type": "city", "bounding_box": rectangle("-85.86503,33.615344 -85.75944,33.74226") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 101852, "cityName": "Anniston" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383791992836, "text": "Miami hospital fires doctor who attacked Uber driverFox https://t.co/ZaDJ2KdVxv \n@khalid_alqadi she is a neurology 4th year resident ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 594140074 }}, "user": { "id": 323013512, "name": "Dr. Yazid Almalky", "screen_name": "dr_almalky", "lang": "en", "location": "Bowie, MD", "create_at": date("2011-06-23"), "description": "شخابيط سعودية في دفترٍ امريكي. طبيب استشاري حاصل على البورد الامريكي في طب الطوارئ ومهتم بالجودة الطبية وسلامة المرضى. snap:dr_almalky", "followers_count": 1261, "friends_count": 367, "statues_count": 3521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chestertown, MD", "id": "a39e306e0377ae02", "name": "Chestertown", "place_type": "city", "bounding_box": rectangle("-76.094819,39.188684 -76.049122,39.239173") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24029, "countyName": "Kent", "cityID": 2416225, "cityName": "Chestertown" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768383888564226, "text": "@carolaltoe é o que?", "in_reply_to_status": 723742377970675717, "in_reply_to_user": 2296784079, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 2296784079 }}, "user": { "id": 3349270767, "name": "〽️", "screen_name": "marloonlucas", "lang": "pt", "location": "Florida, USA", "create_at": date("2015-06-28"), "description": "porque és pó e ao pó tornarás", "followers_count": 554, "friends_count": 291, "statues_count": 15250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coconut Creek, FL", "id": "0af0a491a5194920", "name": "Coconut Creek", "place_type": "city", "bounding_box": rectangle("-80.203196,26.213436 -80.152557,26.327949") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1213275, "cityName": "Coconut Creek" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384119173120, "text": "It's back!!! https://t.co/pAWqMW0f54", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 47796730, "name": "Chelsea Hope", "screen_name": "Chachachelsea", "lang": "en", "location": "34.141995,-118.65087", "create_at": date("2009-06-16"), "description": "just some random, crazy, gay, nerd and her really weird times.", "followers_count": 204, "friends_count": 127, "statues_count": 4752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384307912704, "text": "@odamnwolf only cracked open 3 boxes so far, 1 regular avacyn in each, last one also had a foil one xP", "in_reply_to_status": 723767730122399744, "in_reply_to_user": 370720955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 370720955 }}, "user": { "id": 178906701, "name": "DT", "screen_name": "darktigger13", "lang": "en", "location": "Northville, MI", "create_at": date("2010-08-15"), "description": "Electrical Engineering blue Tiger sometimes running around rampant as a fluffy Husky in SE Michigan", "followers_count": 319, "friends_count": 405, "statues_count": 3811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northville, MI", "id": "006a69011e2bfd34", "name": "Northville", "place_type": "city", "bounding_box": rectangle("-83.551193,42.393398 -83.430722,42.444865") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2658980, "cityName": "Northville" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384307924993, "text": "Everything about Tyler makes me sohappy aw I'm so lucky ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2209887324, "name": "ℓєχιє ʝαє", "screen_name": "lexie_jaee", "lang": "en", "location": "null", "create_at": date("2013-11-22"), "description": "coffee & tyler", "followers_count": 541, "friends_count": 179, "statues_count": 10526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384308031488, "text": "Amundson Law Office, Eagan, MN, 55122 - https://t.co/l1BYDyD1uz\n\n#amundsonlawoffice #lawyer #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.222282,44.787721"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "amundsonlawoffice", "lawyer", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 490, "friends_count": 482, "statues_count": 440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagan, MN", "id": "5d441fd5487fc133", "name": "Eagan", "place_type": "city", "bounding_box": rectangle("-93.228436,44.774013 -93.104796,44.862942") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2717288, "cityName": "Eagan" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384471502848, "text": "ミクの腋", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ja", "is_retweet": false, "user": { "id": 293388138, "name": "なお@サンタミクシコ", "screen_name": "nao_square", "lang": "ja", "location": "ミクダヨーさんの中", "create_at": date("2011-05-05"), "description": "通知が遅れて飛んでくるので、リプに気づかない事が多々あります。", "followers_count": 1355, "friends_count": 523, "statues_count": 130579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384484077568, "text": "Okay, my favorite type of show is where there are like four or more fairly prominent villains and they all hella hate each other.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17897038, "name": "Shabazz Luhrmann", "screen_name": "ThomasAwful", "lang": "en", "location": "Baltimore, MD", "create_at": date("2008-12-05"), "description": "I've got this feeling I'm totally, totally fucked. sometimes writes @Classical @ViceSports @GQMagazine @NewYorker @city_paper @TheCauldron and your mama.", "followers_count": 1789, "friends_count": 1315, "statues_count": 47296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384635064320, "text": "Wind 0.0 mph ---. Barometer 30.068 in, Rising slowly. Temperature 60.0 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768384865738752, "text": "Josh Norman just RTed me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193141678, "name": "Matt Geving", "screen_name": "MattGeving", "lang": "en", "location": "The Heart Of Broncos Country", "create_at": date("2010-09-20"), "description": "Season ticket holder of the 3 time Super Bowl champion Denver Broncos. Partial owner of @Broncos_Trivia. I scream on Sundays & celebrate the Orange Crush.", "followers_count": 2696, "friends_count": 1545, "statues_count": 33056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Pinery, CO", "id": "6171173764a0c932", "name": "The Pinery", "place_type": "city", "bounding_box": rectangle("-104.78938,39.42758 -104.693481,39.463965") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 877235, "cityName": "The Pinery" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768385507610626, "text": "Wind 5.2 mph NW. Barometer 29.887 in, Steady. Temperature 49.2 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768385566183424, "text": "@alyybongo @MattEatsMochi not even close.", "in_reply_to_status": 723768231475863553, "in_reply_to_user": 123740283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 123740283, 107882881 }}, "user": { "id": 40328833, "name": "Joeduck", "screen_name": "g3nuinejoe", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2009-05-15"), "description": "Will cook for cuddles.\n\n***WARNING: Sometimes I tweet pictures of food.***", "followers_count": 7598, "friends_count": 630, "statues_count": 27162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768385742364672, "text": "I love my Addams Family @gm72798 @DanielleRaesin @nschlaffer @heyjudetanner Antonia Vivino @heytherediem @foreveronj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 471472469, 615622120, 1923476263, 490540786, 2304238165, 1016136541 }}, "user": { "id": 401849923, "name": "Grandmama Addams", "screen_name": "mel_shilkie", "lang": "en", "location": "Simi Valley CA", "create_at": date("2011-10-30"), "description": "SSHS senior, legal adult, actress, official cat whisperer", "followers_count": 419, "friends_count": 1087, "statues_count": 1996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768385893326848, "text": "Don't know if I should sleep for a couple of hours or pull an all nighter................", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307594373, "name": "alyssa", "screen_name": "alyssubmarine", "lang": "en", "location": "null", "create_at": date("2011-05-29"), "description": "never worry about me nigga. #AlyssaPls", "followers_count": 988, "friends_count": 379, "statues_count": 21019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768386266750976, "text": "But the way he set up that'll never happen ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81711578, "name": "TAYBAE™ ♡", "screen_name": "TayTaught_YOU", "lang": "en", "location": "null", "create_at": date("2009-10-11"), "description": "living everyday is a blessing , follow me☺️ #PSU18 || BoogaCBaby | Pgh,PA", "followers_count": 3902, "friends_count": 2308, "statues_count": 148297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "State College, PA", "id": "22c613c36f32f0d1", "name": "State College", "place_type": "city", "bounding_box": rectangle("-77.917295,40.749326 -77.798924,40.817749") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42027, "countyName": "Centre", "cityID": 4273808, "cityName": "State College" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768386409365504, "text": "I don't want NONE of the niggas I use to mess with�� keep my name out y'all fucking mouth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1263378434, "name": "ll . Xll . XVl", "screen_name": "Cora_Andrew", "lang": "en", "location": "205 ✈️ 334", "create_at": date("2013-03-12"), "description": "#myASU19 | Rehabilitation Services Major .. @DEHTKIDDDEON ✨ | Taurus ♉️", "followers_count": 1883, "friends_count": 1108, "statues_count": 68572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-86.418059,32.284593 -86.071398,32.443697") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768386463760384, "text": "fuuuuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3287956664, "name": "dani", "screen_name": "danivscz", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "hhs 17 yaaaa", "followers_count": 820, "friends_count": 707, "statues_count": 9285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hondo, TX", "id": "6521a8878080ca86", "name": "Hondo", "place_type": "city", "bounding_box": rectangle("-99.200407,29.332533 -99.122347,29.373723") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48325, "countyName": "Medina", "cityID": 4834676, "cityName": "Hondo" } }
+{ "create_at": datetime("2016-04-23T00:00:04.000Z"), "id": 723768386522501120, "text": "@Yeahzaco @EmilyBett @ReneeMulholand Yeah, Emily would definitely win, sorry!!", "in_reply_to_status": 723768066241241088, "in_reply_to_user": 19737133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19737133, 159311024, 2682142295 }}, "user": { "id": 30496696, "name": "Shauna Sullivan", "screen_name": "ShaunaBear22", "lang": "en", "location": "Michigan", "create_at": date("2009-04-11"), "description": "BOOM CLAP Big Bawls", "followers_count": 621, "friends_count": 1851, "statues_count": 33336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MI", "id": "c309c6a6f85110cd", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-82.973298,42.539309 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768386656690176, "text": "Eh ung siguradong jinojoke nya ng kagat-labi si Rj ng dalawa lang sila...\n\nSTAY STRONG RICHARD ��\n #ALDUBLiveWorkshop https://t.co/VT0DsQwEFK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBLiveWorkshop" }}, "user": { "id": 3767382252, "name": "MrMrsF ♡", "screen_name": "mrmrsfaulkerson", "lang": "en", "location": "Chicago, IL, USA", "create_at": date("2015-10-03"), "description": "MaiChard ❤️ | Orient kita: The views & opinions expressed in my account does not necessarily reflect the opinions of Eat Bulaga nor the AlDub Nation as a whole.", "followers_count": 1753, "friends_count": 388, "statues_count": 17526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768386673610753, "text": "That post calling r*nch cinnamon rolls and root and shaw sinnamon rolls .........", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3537808942, "name": "jazzy j", "screen_name": "semenshaw", "lang": "en", "location": "hopefully not gotham", "create_at": date("2015-09-03"), "description": "hi im jasmine and i like person of interest, & chicken nuggets. DC owns my ass.", "followers_count": 154, "friends_count": 99, "statues_count": 4715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Town 'n' Country, FL", "id": "9e27ffe7ca397c0a", "name": "Town 'n' Country", "place_type": "city", "bounding_box": rectangle("-82.649284,27.968367 -82.525493,28.039978") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1272145, "cityName": "Town 'n' Country" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768386686210049, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Rising slowly. Temperature 62.1 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768387407470593, "text": "It's okay you weren't the one that fell on your ass and had the workers laugh at you ���� https://t.co/I8b0FC0lHf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71914427, "name": "AriSafari✨", "screen_name": "AriannaMariaa14", "lang": "en", "location": "null", "create_at": date("2009-09-05"), "description": "Portuguese & Mexican l Sonoma State l Hula l", "followers_count": 647, "friends_count": 1069, "statues_count": 13304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768388112232448, "text": "Woahhhh 3 a.m. where did you come from", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248506023, "name": "Brock Nettleton", "screen_name": "BNettleton25", "lang": "en", "location": "Columbus, OH", "create_at": date("2011-02-06"), "description": "Sophomore @ The Ohio State University #JesusSaves #LaFamilia #SmithvilleAlum", "followers_count": 1006, "friends_count": 585, "statues_count": 11082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768388477034496, "text": "DnF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 519965489, "name": "Endo", "screen_name": "TrojanStretcher", "lang": "en", "location": "Mesa, AZ", "create_at": date("2012-03-09"), "description": "Hippies x Goonz is the movement.", "followers_count": 346, "friends_count": 263, "statues_count": 36897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768388573491200, "text": "perfect. perfect. perfect", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1708697551, "name": "Kayyy.♡", "screen_name": "KaMii28_", "lang": "en", "location": "Dinuba, CA", "create_at": date("2013-08-28"), "description": "I love animals, duh", "followers_count": 756, "friends_count": 873, "statues_count": 13940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768388640604160, "text": "When @coldcompanion90 messes up your rant chain smh https://t.co/dbcCAib9xr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4766938904 }}, "user": { "id": 2225337948, "name": "j e s s a", "screen_name": "JessaTrophy", "lang": "en", "location": "houston ", "create_at": date("2013-12-01"), "description": "the evil that i see is taking over me", "followers_count": 593, "friends_count": 475, "statues_count": 1647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aldine, TX", "id": "836634cecebe2de4", "name": "Aldine", "place_type": "city", "bounding_box": rectangle("-95.408756,29.891304 -95.355057,29.939071") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4801696, "cityName": "Aldine" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768388758052864, "text": "Like... Make a line to sit on face", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 303975005, "name": "Jay", "screen_name": "sarcastic_soul_", "lang": "en", "location": "Home of Uncle Luke", "create_at": date("2011-05-23"), "description": "lover & lifestyle blogger in the making.", "followers_count": 1147, "friends_count": 597, "statues_count": 45492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galveston, TX", "id": "632eeebc87aecd57", "name": "Galveston", "place_type": "city", "bounding_box": rectangle("-94.880809,29.239602 -94.764742,29.335548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4828068, "cityName": "Galveston" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768389139873792, "text": "�� https://t.co/ZUXU7h4lPT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 531904941, "name": "Pnutt Perezツ", "screen_name": "PnuttPerez", "lang": "en", "location": "mexican ting", "create_at": date("2012-03-20"), "description": "Too determined to be defeated✞ BGSU", "followers_count": 1018, "friends_count": 336, "statues_count": 55957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768389592731648, "text": "@Miikey_09 ugly", "in_reply_to_status": 723767217637052416, "in_reply_to_user": 939622158, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 939622158 }}, "user": { "id": 628105580, "name": "Brutus Maximus", "screen_name": "DrugDealerx", "lang": "en", "location": "SC: notcriss", "create_at": date("2012-07-05"), "description": "Mentally UCLA, Physically Community College\n Sylmar Wrestling 2013-2015", "followers_count": 284, "friends_count": 177, "statues_count": 19134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768389613682688, "text": "@ryry_sawyer @madisonblehm Yo mads 2 q's , how ur ass so fat? n wheres the blow?", "in_reply_to_status": 723752842742231040, "in_reply_to_user": 2990555502, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2990555502, 3063480991 }}, "user": { "id": 3310400953, "name": "The Kenzie Krab", "screen_name": "kenzers_29", "lang": "en", "location": "Arlington, WA", "create_at": date("2015-08-09"), "description": "They bring you so much pain. Are they seriously worth thinking about this much?", "followers_count": 254, "friends_count": 451, "statues_count": 4357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768390645481474, "text": "Wind 1.0 mph SE. Barometer 30.090 in, Steady. Temperature 47.1 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-23T00:00:05.000Z"), "id": 723768390762958848, "text": "@1077TheEnd goodnight #2minutepromise", "in_reply_to_status": -1, "in_reply_to_user": 17531722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "2minutepromise" }}, "user_mentions": {{ 17531722 }}, "user": { "id": 30996738, "name": "Sibrina from Seattle", "screen_name": "bye_matty", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-04-13"), "description": "Currently listening to the 1975, the japanese house, gallant, balam acab, and fickle friends // side account so I don't annoy my friends", "followers_count": 46, "friends_count": 69, "statues_count": 501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "00d4bfeb72515895", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-122.212834,47.430091 -122.108701,47.471158") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-04-23T00:00:06.000Z"), "id": 723768391534661632, "text": "i want a year of eye contacts for my birthday, thanks ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16154135, "name": "Heather Hartman", "screen_name": "heatheranmariee", "lang": "en", "location": "Paramount, California", "create_at": date("2008-09-05"), "description": "19 / i'd rather be at the gym / Mood: fierce / Most of my tweets are notes to myself / 03.29.12 #lovelifeleadership", "followers_count": 541, "friends_count": 473, "statues_count": 7664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-04-23T00:00:06.000Z"), "id": 723768391614513152, "text": "Please excuse me while I get the sand out of my ears, but it might take a few months.… https://t.co/ag0YaujhKz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.445,33.9489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16541331, "name": "liz", "screen_name": "dirtytucson", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2008-10-01"), "description": "Production Assistant/Researcher @ The Dr Phil Show, 26, writer, producer, television junkie. I have an instagram problem.", "followers_count": 244, "friends_count": 170, "statues_count": 10536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-23T00:00:06.000Z"), "id": 723768392122028032, "text": "#Trespasser at 7201-7998 International Dr. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.4707994,28.4529741"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trespasser", "orlpol", "ocso" }}, "user": { "id": 39050965, "name": "Police Calls 32819", "screen_name": "orlpol32819", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-10"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 294, "friends_count": 1, "statues_count": 74588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-04-23T00:00:06.000Z"), "id": 723768392239321090, "text": "You may not always end up where you thought you were going, but you will always end up where you are meant to be��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 522507781, "name": "nada a7med", "screen_name": "nadooooooooosha", "lang": "ar", "location": "corvallis, OR", "create_at": date("2012-03-12"), "description": "I do not believe in taking the right decision, I take a decision and make it right", "followers_count": 246, "friends_count": 183, "statues_count": 3308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-04-23T00:00:06.000Z"), "id": 723768392285454336, "text": "@Love4Military thank you", "in_reply_to_status": -1, "in_reply_to_user": 2410866554, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2410866554 }}, "user": { "id": 712690051201695744, "name": "Jim Bott# TRUMP", "screen_name": "jamesbott007", "lang": "en", "location": "wilds of new Mexico.USA", "create_at": date("2016-03-23"), "description": "retired lumberman please don't ad to any list.thank you!!!", "followers_count": 820, "friends_count": 1498, "statues_count": 4125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Cruces, NM", "id": "f515486276aa6192", "name": "Las Cruces", "place_type": "city", "bounding_box": rectangle("-106.843427,32.238361 -106.63145,32.446238") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3539380, "cityName": "Las Cruces" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130753819324416, "text": "i mean what can I say���� #prom2k16 #codered https://t.co/x3cw5ZiYuX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "prom2k16", "codered" }}, "user": { "id": 705345917, "name": "antwonette", "screen_name": "yvng_netty", "lang": "en", "location": "null", "create_at": date("2013-10-08"), "description": "i'd b jealous of me too!", "followers_count": 942, "friends_count": 394, "statues_count": 20693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130753903099904, "text": "@irahmonge yea, and the freaking cheeseburger w/o cheese!", "in_reply_to_status": 724130481671790592, "in_reply_to_user": 615890678, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615890678 }}, "user": { "id": 2334579624, "name": "Jeadrin Ady", "screen_name": "jeadrinxx", "lang": "en", "location": "Anchorage, AK", "create_at": date("2014-02-08"), "description": "3 26 99", "followers_count": 390, "friends_count": 363, "statues_count": 10023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130753924239360, "text": "It's alright to pop out now and then ����I like home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209793660, "name": "©〰", "screen_name": "JetPlaneCaine_", "lang": "en", "location": "getting to the money ", "create_at": date("2010-10-29"), "description": "Bands my only motivation '16", "followers_count": 1629, "friends_count": 1343, "statues_count": 19340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130754137980929, "text": "@BriannaAlysse where u at �� im in Murrieta lol", "in_reply_to_status": -1, "in_reply_to_user": 498422915, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 498422915 }}, "user": { "id": 258194000, "name": "Sirena Nessa✨", "screen_name": "vnodal", "lang": "en", "location": "Oxnard, CA", "create_at": date("2011-02-26"), "description": "all the gods, all the heavens, all the hells are within you.", "followers_count": 927, "friends_count": 941, "statues_count": 43317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130754419027968, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 574615290, "name": "Rosemary Hess", "screen_name": "RosemaryEllenn", "lang": "en", "location": "null", "create_at": date("2012-05-08"), "description": "|SC:RosemaryEllenn ➳ ATX//CC|", "followers_count": 681, "friends_count": 302, "statues_count": 36937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130755719233536, "text": "It's about prison abolition. Super interesting stuff but ������������", "in_reply_to_status": 724130614517993472, "in_reply_to_user": 300113561, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300113561, "name": "Jeremy, B.A.", "screen_name": "KermitTheHermit", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2011-05-16"), "description": "Graduated from Cal State Fullerton with my sexy English degree. Now i'm a grad student. The Angels did good one year.", "followers_count": 411, "friends_count": 2065, "statues_count": 16042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130755845050368, "text": "nick: PREGNANT https://t.co/iH2WZcKHEw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327656434, "name": "✨gay luigi?✨", "screen_name": "agentpleakley", "lang": "en", "location": "far far away", "create_at": date("2014-02-04"), "description": "@mermistas's side/more personal/shitposty/fandomy account i live tweet here a lot", "followers_count": 327, "friends_count": 215, "statues_count": 29063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130756646211586, "text": "Can you recommend anyone for this #Hospitality #job? https://t.co/rSYp6ifvD3 #restaurantlife #Chicago, IL #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "restaurantlife", "Chicago", "Hiring" }}, "user": { "id": 701776772057141248, "name": "HMSHost Jobs", "screen_name": "HMSHostCareers", "lang": "en", "location": "null", "create_at": date("2016-02-22"), "description": "HMSHost is part of the world’s largest provider of food & beverage services for travelers. Explore our various hourly and management positions in US & Canada!", "followers_count": 295, "friends_count": 220, "statues_count": 2133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130756994437120, "text": "saving silverman just came on hbo, you're welcome", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317446845, "name": "kevin casey", "screen_name": "exile_on_2nd_st", "lang": "en", "location": "philadelphia, pa", "create_at": date("2011-06-14"), "description": "i only pin deleted tweets", "followers_count": 430, "friends_count": 296, "statues_count": 9525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-24T00:00:00.000Z"), "id": 724130757396959233, "text": "i'm just gonna drink tea, eat pizza, write these essays, and then clean tomorrow before mom comes home", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60817926, "name": "myluv", "screen_name": "mylaorr", "lang": "en", "location": "stayin healthy on a punk bitch", "create_at": date("2009-07-27"), "description": "you my candy, willy wonka | sc: myluh", "followers_count": 1192, "friends_count": 315, "statues_count": 40852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parkland, WA", "id": "69d2c4f187635997", "name": "Parkland", "place_type": "city", "bounding_box": rectangle("-122.469677,47.118559 -122.389204,47.161211") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5353335, "cityName": "Parkland" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130757904502784, "text": "@ebbtideapp Tide in Keyport, New Jersey 04/24/2016\n Low 4:14am 0.1\nHigh 9:56am 4.9\n Low 4:15pm 0.5\nHigh 10:08pm 5.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-74.1983,40.44"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 79, "friends_count": 1, "statues_count": 27078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Jersey, USA", "id": "65b4760a2b411e11", "name": "New Jersey", "place_type": "admin", "bounding_box": rectangle("-75.563587,38.788657 -73.88506,41.357424") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3436810, "cityName": "Keyport" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758017732609, "text": "Count on that alcohol to numb them feelings", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418372935, "name": "Andoru", "screen_name": "ANDORUSUMMERS", "lang": "en", "location": "Charlotte, NC", "create_at": date("2011-11-21"), "description": "Soundcloud/Youtube: andorutunes", "followers_count": 1192, "friends_count": 968, "statues_count": 46896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758089015296, "text": "how'd I get so lucky", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2187198822, "name": "һѧʟi", "screen_name": "halimaee", "lang": "en", "location": "| COᒪOᖇAᗪO ᔕPRΙNGᔕ |", "create_at": date("2013-11-10"), "description": "| DOD - CTU Software Programming | 22 |✌️cute4u | | NOLA & CO | ❣ @daltonhayess |", "followers_count": 553, "friends_count": 368, "statues_count": 26361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758617489408, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 545515118, "name": "monique", "screen_name": "_moniquerrenee", "lang": "en", "location": "null", "create_at": date("2012-04-04"), "description": ":)))))))) ((((((((: amat", "followers_count": 343, "friends_count": 316, "statues_count": 5481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758630076416, "text": "My ex is texting me. My Napohoe is texting me. Me PoF is texting me... Sorry about it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131312622, "name": "MrPopKills", "screen_name": "MrPopKills", "lang": "en", "location": "An LA dancefloor.", "create_at": date("2010-04-09"), "description": "I'm the dandy highwayman that you're too scared to mention. Need a pop DJ in LA? Hit me up. #popmusicfiend", "followers_count": 353, "friends_count": 373, "statues_count": 9231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758688907264, "text": "Temp 37.7°F Wind Chill 37.7°F RH 92% Wind 0.0 --- Gust 0.0 --- SLP 30.088 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 111, "friends_count": 63, "statues_count": 34932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758814617601, "text": "Only way to say you got it.. Is to get it!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419850919, "name": "Lightskin Terrance", "screen_name": "Pat_Ayyee", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "Faith in God †", "followers_count": 241, "friends_count": 146, "statues_count": 76 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Park, GA", "id": "1703b859c254a0f9", "name": "College Park", "place_type": "city", "bounding_box": rectangle("-84.512982,33.593318 -84.427795,33.669237") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1317776, "cityName": "College Park" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130758856683521, "text": "04/24@03:00 - Temp 47.9F, WC 47.9F. Wind 1.5mph NNW, Gust 4.0mph. Bar 30.033in, Rising slowly. Rain 0.00in. Hum 39%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130759645270016, "text": "I kinda appreciate the small town feel of Greenville now that I've lived in larger cities and college towns", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354025750, "name": "Fezz", "screen_name": "amifezz", "lang": "en", "location": "Greenville MSU Kalamazoo", "create_at": date("2011-08-12"), "description": "I ain't rich, but I damn sure wanna be", "followers_count": 288, "friends_count": 561, "statues_count": 20559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130759716405248, "text": "Had me a good little night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93510504, "name": "Ambie", "screen_name": "agdoll", "lang": "en", "location": "null", "create_at": date("2009-11-29"), "description": "nyc x ohio", "followers_count": 696, "friends_count": 149, "statues_count": 17377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130759796219905, "text": "I cannot hang", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572767639, "name": "alison", "screen_name": "nicole_arias", "lang": "en", "location": "New Brunswick, NJ", "create_at": date("2012-05-06"), "description": "RU '18", "followers_count": 314, "friends_count": 203, "statues_count": 13664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130759867555841, "text": "Somebody gone appreciate me one day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949219788, "name": "5lim✨", "screen_name": "yourgoddesssss", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "#SSU18❤", "followers_count": 23272, "friends_count": 8227, "statues_count": 78157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130759955484676, "text": "@TheresaMechele @stinnett_doug So Lyin Ted why are you accusing Trump of donating to Hillary when you do it too!", "in_reply_to_status": 723835825964744705, "in_reply_to_user": 928618363, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 928618363, 4411076609 }}, "user": { "id": 2814595145, "name": "linda ikeda", "screen_name": "lindaikedatko", "lang": "en", "location": "null", "create_at": date("2014-10-07"), "description": "null", "followers_count": 47, "friends_count": 123, "statues_count": 735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760593014784, "text": "Lmaoo set up a dick appointment https://t.co/uiiFr9mUYU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3100875382, "name": "Trev✈️", "screen_name": "trevorkemngang", "lang": "en", "location": "9⃣0⃣9⃣", "create_at": date("2015-03-20"), "description": "6'3/Leo♌️/Im a twin stop asking/ SC: trevor_4hunnid", "followers_count": 674, "friends_count": 460, "statues_count": 38050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760614133760, "text": "I'm with that ���� https://t.co/7BqNdKXBDF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2239235791, "name": "April 24", "screen_name": "stvf__almvghty", "lang": "en", "location": "Ohio, USA", "create_at": date("2013-12-10"), "description": "St.Clair. Ignatius '17.Watusi Lifestyle", "followers_count": 691, "friends_count": 551, "statues_count": 34930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euclid, OH", "id": "59ed3d5f720ebdf0", "name": "Euclid", "place_type": "city", "bounding_box": rectangle("-81.552943,41.545336 -81.488014,41.631584") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3925704, "cityName": "Euclid" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760777572352, "text": "She went into a kush time machine and woke up the next day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496318216, "name": "Sosa Marvin Gaye", "screen_name": "YL14_", "lang": "en", "location": "cudderland ", "create_at": date("2012-02-18"), "description": "wish i was jigga man!! ASN", "followers_count": 493, "friends_count": 299, "statues_count": 47066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shiloh, IL", "id": "c4396e759c0f5fae", "name": "Shiloh", "place_type": "city", "bounding_box": rectangle("-89.966385,38.533628 -89.87644,38.580671") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17163, "countyName": "St. Clair", "cityID": 1769524, "cityName": "Shiloh" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760903389186, "text": "You'll know..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 561624145, "name": "Juliaa", "screen_name": "JuliaKnapsted", "lang": "en", "location": "null", "create_at": date("2012-04-23"), "description": "not everything's so serious", "followers_count": 477, "friends_count": 296, "statues_count": 8474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Celebration, FL", "id": "01bbe9ba4078361c", "name": "Celebration", "place_type": "city", "bounding_box": rectangle("-81.577489,28.276141 -81.521152,28.332902") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1211285, "cityName": "Celebration" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760957943809, "text": "Left on read��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2888706012, "name": "J POUNDS", "screen_name": "Jayy_patrick", "lang": "en", "location": "null", "create_at": date("2014-11-03"), "description": "don't just read my bio and not follow me.", "followers_count": 372, "friends_count": 233, "statues_count": 3988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130760999899137, "text": "ミライスタート@BABSCon https://t.co/Ixk5m3LkX9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.365289,37.594078"), "retweet_count": 0, "lang": "ja", "is_retweet": false, "user_mentions": {{ 1515688483 }}, "user": { "id": 65728121, "name": "タワー | Tower", "screen_name": "nullpotower", "lang": "ja", "location": "えこだ", "create_at": date("2009-08-14"), "description": "Japanese brony. MLPの話題で騒ぎます。BlenderとUE4を申し訳程度に触るVR信者です。", "followers_count": 596, "friends_count": 493, "statues_count": 24645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hyatt Regency San Francisco Airport", "id": "07d9cb1354c84001", "name": "Hyatt Regency San Francisco Airport", "place_type": "poi", "bounding_box": rectangle("-122.3652891,37.5940779 -122.365289,37.594078") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 609066, "cityName": "Burlingame" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761050349568, "text": "Wind 3.0 mph ESE. Barometer 29.978 in, Steady. Temperature 57.5 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761079562240, "text": "@PimpinCLYDE here u go..", "in_reply_to_status": 724130510855794688, "in_reply_to_user": 265668274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 265668274 }}, "user": { "id": 91478411, "name": "Honey✨", "screen_name": "e__god", "lang": "en", "location": "E Dolla $ign", "create_at": date("2009-11-20"), "description": "Spanish mami", "followers_count": 1815, "friends_count": 360, "statues_count": 86534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761394257920, "text": "I wanna change my Avi but Carla is gonna punch me in the cooter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32476246, "name": "tellhaCARLAsentcha", "screen_name": "MessiahMARIAH", "lang": "en", "location": "null", "create_at": date("2009-04-17"), "description": "null", "followers_count": 4345, "friends_count": 2794, "statues_count": 257109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761494781953, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":45.7°F Wind:0.0mph Pressure: 30.03hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 316069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761566121985, "text": "@karinathegiant happy birthday pretty girl ��", "in_reply_to_status": -1, "in_reply_to_user": 2722232180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2722232180 }}, "user": { "id": 2344648686, "name": "Mariah✨❤️", "screen_name": "mariahguzman78", "lang": "en", "location": "Orosi, CA", "create_at": date("2014-02-14"), "description": "OHS Cheerleader❤️", "followers_count": 601, "friends_count": 702, "statues_count": 9912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761742241792, "text": "I purposely give FAU pens to customers with their bills so they know the struggle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56129649, "name": "Turmaine", "screen_name": "Jorddanian", "lang": "en", "location": "null", "create_at": date("2009-07-12"), "description": "Avada Kedavra", "followers_count": 317, "friends_count": 191, "statues_count": 12682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coconut Creek, FL", "id": "0af0a491a5194920", "name": "Coconut Creek", "place_type": "city", "bounding_box": rectangle("-80.203196,26.213436 -80.152557,26.327949") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1213275, "cityName": "Coconut Creek" } }
+{ "create_at": datetime("2016-04-24T00:00:01.000Z"), "id": 724130761817747456, "text": "Omg he's ridiculously so cute and smart! https://t.co/P14FF6wIIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417588520, "name": "Kelsey Amber Robles", "screen_name": "kelseyamber_14", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "Texas born and raised", "followers_count": 145, "friends_count": 293, "statues_count": 1752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130762077786113, "text": "I don't understand.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3045146646, "name": "Darius", "screen_name": "DariusJ_29", "lang": "en", "location": "Find your own path. ", "create_at": date("2015-02-26"), "description": "copper hills baseball⚾", "followers_count": 601, "friends_count": 446, "statues_count": 949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130762405097472, "text": "@_matthewpalmer i'm hoping it works more like a news report – I don't necessarily want to have to search for something every time.", "in_reply_to_status": 724130558763237376, "in_reply_to_user": 937781815, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 212522309 }}, "user": { "id": 937781815, "name": "MichaelPrenezIsbell", "screen_name": "MichaelIsbell", "lang": "en", "location": "New York, New York", "create_at": date("2012-11-09"), "description": "Editor, Writer, Software Dvlpr, Sober New Yorker, Vegan, Media Technologist, Urban Cyclist, Prudent Optimist.", "followers_count": 470, "friends_count": 2122, "statues_count": 46 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130762551779330, "text": "Instrument Technician - #CHOMP: (#Monterey, CA) https://t.co/uotTFElHFJ #Healthcare #MedField #NorCal #ScrubLife https://t.co/SmIZ9Mnfef", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.913238,36.5790057"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "CHOMP", "Monterey", "Healthcare", "MedField", "NorCal", "ScrubLife" }}, "user": { "id": 3132594138, "name": "CHOMP", "screen_name": "CHOMPJobs", "lang": "en", "location": "Monterey, CA", "create_at": date("2015-04-02"), "description": "Community Hospital of the Monterey Peninsula is #hiring! Apply to our open #jobs here.", "followers_count": 65, "friends_count": 27, "statues_count": 98 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey, CA", "id": "49af5b43d4963f4c", "name": "Monterey", "place_type": "city", "bounding_box": rectangle("-121.927092,36.573772 -121.830974,36.618826") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 648872, "cityName": "Monterey" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130762572763136, "text": "@PaigeVanzantUFC's snaps are so cute ��", "in_reply_to_status": -1, "in_reply_to_user": 136165096, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 136165096 }}, "user": { "id": 2333938752, "name": "whitttttt. ✌️", "screen_name": "xoxowhittt", "lang": "en", "location": "Charleston✈️Myrtle Beach✌️", "create_at": date("2014-02-08"), "description": "red, white and brew.", "followers_count": 1339, "friends_count": 790, "statues_count": 22642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Hill, SC", "id": "001e2a4a7b2f19a5", "name": "Red Hill", "place_type": "city", "bounding_box": rectangle("-79.052947,33.694141 -78.899479,33.838132") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4559190, "cityName": "Red Hill" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130762874822656, "text": "Wind 0.0 mph E. Barometer 30.062 in, Rising slowly. Temperature 51.1 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130763034107905, "text": "I don't go out anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173988218, "name": "Gaby Jacinto", "screen_name": "jacinto_gaby", "lang": "en", "location": "null", "create_at": date("2013-02-12"), "description": "Sc: gabyylove", "followers_count": 155, "friends_count": 274, "statues_count": 16171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130763227033600, "text": "Can you recommend anyone for this #job? Manager, Provider Engagement - https://t.co/j3uSN7Eh7p #Irving, TX #Hiring https://t.co/KRg551Oz7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.9488945,32.8140177"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "job", "Irving", "Hiring" }}, "user": { "id": 3022832918, "name": "Molina Careers", "screen_name": "MolinaCareers", "lang": "en", "location": "Nationwide", "create_at": date("2015-02-16"), "description": "Molina is growing rapidly & we’re looking for talented people who share our values. Check out our current opportunities at http://MolinaHealthcare.com/Careers.", "followers_count": 240, "friends_count": 1, "statues_count": 672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130763315126273, "text": "lmt I want to ft someone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2269730995, "name": "J@ÿdęń", "screen_name": "jayden828", "lang": "en", "location": "null", "create_at": date("2013-12-30"), "description": "Desert Ridge||Photography||Symphony||Swim", "followers_count": 493, "friends_count": 207, "statues_count": 3225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130763550142464, "text": "Temp: 56.9°F Wind:0.0mph Pressure: 29.985hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130763608854528, "text": "Good eats at @bluestem_kc #shrimpandgrits #yesplease @ bluestem https://t.co/kMZqzSKSwr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.597023,39.0508919"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "shrimpandgrits", "yesplease" }}, "user_mentions": {{ 2648031517 }}, "user": { "id": 2820114762, "name": "Christie Hammond", "screen_name": "ChristieInKC", "lang": "en", "location": "Kansas City", "create_at": date("2014-09-19"), "description": "null", "followers_count": 138, "friends_count": 185, "statues_count": 290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130764556640256, "text": "and Hunter too, I saw him about 4 times tonight. and Blake about 7.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2477170020, "name": "syd | TODAAAAY!!!!!!", "screen_name": "KINKYMILES", "lang": "en", "location": "Hou$ton // intoxicated hoes", "create_at": date("2014-05-04"), "description": "respect america - jacob sartorius", "followers_count": 4346, "friends_count": 385, "statues_count": 42554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130764695187457, "text": "thank you love you ��������R.O.D. https://t.co/uc6u7H77m6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2411677323, "name": "tori..", "screen_name": "alextorain", "lang": "en", "location": "null", "create_at": date("2014-03-14"), "description": "I get it", "followers_count": 667, "friends_count": 361, "statues_count": 5760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130764741292032, "text": "Wind 0.0 mph ---. Barometer 30.138 in, Falling slowly. Temperature 44.7 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 41, "friends_count": 4, "statues_count": 26927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765097717761, "text": "@vicerylle 4th week of May....next year ��", "in_reply_to_status": 724130599217172481, "in_reply_to_user": 500889259, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 500889259 }}, "user": { "id": 2685772777, "name": "jose", "screen_name": "fourteenvk", "lang": "en", "location": "adv", "create_at": date("2014-07-27"), "description": "jmbv and akpt both have clASS", "followers_count": 4492, "friends_count": 158, "statues_count": 38026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765152391168, "text": "Ripley SW Limestone Co. Temp: 53.8°F Wind:0.0mph Pressure: 996.2mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765219356673, "text": "@tripleogyan I'm legit so sad about that. ��", "in_reply_to_status": 724130665000669184, "in_reply_to_user": 2290511246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2290511246 }}, "user": { "id": 80759500, "name": "Ashlee Elizabeth", "screen_name": "miss_mulatto", "lang": "en", "location": "Bay Area, CA ", "create_at": date("2009-10-07"), "description": "Make up artist | hair styling | Male grooming | occasional model ☺️", "followers_count": 1699, "friends_count": 412, "statues_count": 29462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765240410112, "text": "*cuts all my hoes off for my damn self*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262895985, "name": ".", "screen_name": "valcvpone", "lang": "en", "location": "null", "create_at": date("2011-03-08"), "description": ".", "followers_count": 4652, "friends_count": 1848, "statues_count": 79175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765282402305, "text": "Not true lol https://t.co/uFgSvGTzAA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368679879, "name": "babygirl cass", "screen_name": "Cassie_Lovette", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "♒||baby| stressed but mad blessed✨", "followers_count": 3509, "friends_count": 1908, "statues_count": 194716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765303209987, "text": "@brothersxblood I ONCE WATCHED A MAN LOSE HIS DICK GETTING RAN OVER BY A FORKLIFT, ALSO HE WAS GAY AFTER THAT LOOK OUT FOR FORK LIFTS", "in_reply_to_status": 724130188410249216, "in_reply_to_user": 867310585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 867310585 }}, "user": { "id": 867310585, "name": "Coolguy McGillicudy", "screen_name": "brothersxblood", "lang": "en", "location": "IG: nicolasxburroughs ", "create_at": date("2012-10-07"), "description": "I like my pug.", "followers_count": 1366, "friends_count": 859, "statues_count": 58781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765353537538, "text": "God damn! Lol! Grandma got me against the ropes. Ha. Props to you! https://t.co/Kpexm6B80D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3185451961, "name": "GummyWormKnight", "screen_name": "gshortey13", "lang": "en", "location": "null", "create_at": date("2015-05-04"), "description": "Stranger in a Strange Land. Comics, film, heavy metal, punk, rockabilly, guns, swords, hot rods, motorcycles, tattoos, IPA, hot wings, horror movies and chicks", "followers_count": 1327, "friends_count": 2863, "statues_count": 2971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmington, NM", "id": "21fb3163863b6d42", "name": "Farmington", "place_type": "city", "bounding_box": rectangle("-108.283101,36.705314 -108.103633,36.809526") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35045, "countyName": "San Juan", "cityID": 3525800, "cityName": "Farmington" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765420802048, "text": "tbh all yall niggas the same", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2727093227, "name": "Iz.⚜", "screen_name": "lit_iz_", "lang": "en", "location": "dancing my A$$ off ", "create_at": date("2014-07-28"), "description": "God.❤RIP Kenton", "followers_count": 1322, "friends_count": 606, "statues_count": 25982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, LA", "id": "e60ab9aacc0e96af", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-92.135057,32.429655 -92.018346,32.612831") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2251410, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765462753280, "text": "Wind 0.0 mph W. Barometer 30.07 in, Steady. Temperature 39.4 °F. Rain today 0.00 in. Humidity 49%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 118, "statues_count": 159290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130765508882432, "text": "I just got soooo aggravated outta nowhere thinking bout shit ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2375887981, "name": "Ace Boogie ♠️⚡️", "screen_name": "omgits_aniyah", "lang": "en", "location": "w/ ya moms ‼️", "create_at": date("2014-03-06"), "description": "aniyah ❤️| basketball ❤️| rip jozy ❤️| 08901 ❤️", "followers_count": 539, "friends_count": 432, "statues_count": 19858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130766016241664, "text": "This bitch talk ridiculously sickly .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228101868, "name": "Miguel", "screen_name": "Hvlfbaked", "lang": "en", "location": "The Ville ✈ Houston", "create_at": date("2010-12-18"), "description": "null", "followers_count": 874, "friends_count": 377, "statues_count": 16669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-24T00:00:02.000Z"), "id": 724130766112866304, "text": "Me encontre con el verdadero tiguere #backstage @barriosray #miami #miamibash2016 @ South Beach… https://t.co/a75KnMEjfy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.12855755,25.79414235"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "backstage", "miami", "miamibash2016" }}, "user_mentions": {{ 137091642 }}, "user": { "id": 34526772, "name": "GUILLERMO MORDAN", "screen_name": "PinkyMM", "lang": "en", "location": "NEW YORK /HARLEM/WORLDWIDE", "create_at": date("2009-04-22"), "description": "PINKY THE BRAIN!! PRODUCER/ OWNER OF @MORDANMUSIC INC....MANAGER OF @EDDYVOLER FOR BOOKING EMAIL ME MORDANMUSIC@YAHOO.COM . WHATSAPP: +19175188774", "followers_count": 2466, "friends_count": 402, "statues_count": 10667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130766616031232, "text": "Wind 3.0 mph S. Barometer 1015.82 mb, Steady. Temperature 60.8 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130766859333633, "text": "62.1F (Feels: 62.1F) - Humidity: 79% - Wind: 0.0mph --- - Gust: 3.1mph - Pressure: 1010.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 236536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130767412961285, "text": "I feeling like getting on somebody ass ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2958657134, "name": "SmoothYoungPlaya", "screen_name": "doitfordrevon", "lang": "en", "location": "selinsky", "create_at": date("2015-01-04"), "description": "7:14. Bip Dre | Bip Zuba. | South Side Nigga", "followers_count": 2467, "friends_count": 481, "statues_count": 49623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130767505223680, "text": "I can't believe everyone's phones broke today causing no one to see my insta pic and like it...crazy stuff����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3087197292, "name": "Zackary Drysdale", "screen_name": "ZackDrysdale", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "NMSU", "followers_count": 203, "friends_count": 211, "statues_count": 2101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, NM", "id": "76a51908fb719a47", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-106.767744,32.262483 -106.731575,32.28619") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3581030, "cityName": "University Park" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130767714930688, "text": "Will & I raced to the car because we both wanted to drive, then we got to chicken express and realized we left our shoes.........", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 764477233, "name": "Haley Smith", "screen_name": "haleymsmith00", "lang": "en", "location": "null", "create_at": date("2012-08-17"), "description": "null", "followers_count": 462, "friends_count": 148, "statues_count": 33609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Oak, TX", "id": "d91428b130ec1da5", "name": "Red Oak", "place_type": "city", "bounding_box": rectangle("-96.839528,32.484371 -96.70256,32.547292") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4861196, "cityName": "Red Oak" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130767966625792, "text": "me tooo�� https://t.co/Q0almGFEXH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322158737, "name": "Rosy", "screen_name": "RosyMedina_", "lang": "en", "location": "Houston ,Tx", "create_at": date("2011-06-22"), "description": "Nobody understands me but i like it that way|| instagram @Rosymedina_||sc;rosymedina18", "followers_count": 518, "friends_count": 478, "statues_count": 37425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aldine, TX", "id": "836634cecebe2de4", "name": "Aldine", "place_type": "city", "bounding_box": rectangle("-95.408756,29.891304 -95.355057,29.939071") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4801696, "cityName": "Aldine" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130768293761024, "text": "The more things change, the more they stay the same...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296498328, "name": "Juan Asenjo", "screen_name": "JuanFAsenjo", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-05-10"), "description": "null", "followers_count": 315, "friends_count": 118, "statues_count": 18853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Timberlake, OH", "id": "d0593167a3061508", "name": "Timberlake", "place_type": "city", "bounding_box": rectangle("-81.45063,41.663312 -81.435512,41.668159") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3976834, "cityName": "Timberlake" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130768297959425, "text": "I have to wake up super early tomorrow what am I doing awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2806434747, "name": "lyz", "screen_name": "crybabylyz", "lang": "en", "location": "null", "create_at": date("2014-10-04"), "description": "Llorona pero chingona", "followers_count": 260, "friends_count": 301, "statues_count": 7434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130768318930944, "text": "What is a passover", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2444694847, "name": "MISSEDTHEMETERp", "screen_name": "Gohomepriscilla", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "Sirius", "followers_count": 188, "friends_count": 261, "statues_count": 16580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130768776110085, "text": "She got dumb ass ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277229068, "name": "So Cold ❄️", "screen_name": "MoneyMakin_K", "lang": "en", "location": "The Ville ", "create_at": date("2011-04-04"), "description": "Say It Wit Muh Chest, Yea Bitch I'm From Flawda ☀️ 2⃣2️⃣ Snapchat:MoneyMakin_K IG:_KDawgg", "followers_count": 1502, "friends_count": 1997, "statues_count": 26781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130768952410112, "text": "Wind 0.0 mph SSE. Barometer 29.910 in, Falling slowly. Temperature 61.8 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130769623379973, "text": "@EvannShelton happy birthday!!", "in_reply_to_status": -1, "in_reply_to_user": 553842722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 553842722 }}, "user": { "id": 2986743870, "name": "Randi Nicole | 3", "screen_name": "randii_nicolee", "lang": "en", "location": "null", "create_at": date("2015-01-17"), "description": "luke 1:37 || rock chalk || volleyball&track", "followers_count": 569, "friends_count": 1142, "statues_count": 2486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valley Center, KS", "id": "6d1d07db5725ba60", "name": "Valley Center", "place_type": "city", "bounding_box": rectangle("-97.394821,37.795856 -97.348935,37.853577") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2073250, "cityName": "Valley Center" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130769946308608, "text": "At canes we have a list of sayings to greet the customers with at drive & my manager gave me $20 to say all 19 of them :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90221813, "name": "Mel", "screen_name": "meliza_aguilera", "lang": "en", "location": "Houston, TX", "create_at": date("2009-11-15"), "description": "imy Mikey", "followers_count": 286, "friends_count": 81, "statues_count": 32611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-24T00:00:03.000Z"), "id": 724130770202173440, "text": "And I hope I didn't cause any bad vibes to anyone tonight. I let loose and I hope y'all did too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 271735528, "name": "Rolly ✨ [✖️〰✖️]", "screen_name": "roelferrales", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2011-03-24"), "description": "Lights and sick drops. Let's find ourselves under the electric sky more often....there's no place I'd rather be... #MelloGang SC: rferrales95", "followers_count": 803, "friends_count": 1141, "statues_count": 33587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roanoke, TX", "id": "001a980b0bf26339", "name": "Roanoke", "place_type": "city", "bounding_box": rectangle("-97.340667,32.988286 -97.21085,33.049565") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4862504, "cityName": "Roanoke" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130770655178752, "text": "happy birthday to my favorite of all time. Thank you for 5 years, as I continue to support you. https://t.co/kn1GKm6JYq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2398484768, "name": "Naya", "screen_name": "NayaFlowers", "lang": "en", "location": "the bay", "create_at": date("2014-03-19"), "description": "majestic casual | always in a marc e. bassy mood", "followers_count": 1303, "friends_count": 994, "statues_count": 44777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130770663559169, "text": "@Tristan_Marquis @zoned_out15 @Making_Moves40 @AmayaC23 @camruncam @KaylaJackson47", "in_reply_to_status": 724130280500383744, "in_reply_to_user": 2538452121, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2538452121, 244592966, 3016008910, 1053540474, 311804753, 626204079 }}, "user": { "id": 2538452121, "name": "t", "screen_name": "Tristan_Marquis", "lang": "en", "location": "503 ✈ 918", "create_at": date("2014-05-09"), "description": "ORU | instagram: tristanmarquis | bless up", "followers_count": 294, "friends_count": 361, "statues_count": 2759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130771200430080, "text": "Today. Or yesterday https://t.co/xMBgiLmE3K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3029266293, "name": "KING.", "screen_name": "flexforjay", "lang": "en", "location": "the clarke", "create_at": date("2015-02-10"), "description": "skinny nigga but my pockets obese.", "followers_count": 841, "friends_count": 721, "statues_count": 14699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130771385090049, "text": "@SALAMXIII \nحمار للأبد", "in_reply_to_status": 724082742716194816, "in_reply_to_user": 232604780, "favorite_count": 0, "coordinate": point("-87.61861141,41.89209091"), "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 232604780 }}, "user": { "id": 200083896, "name": "Śaeed", "screen_name": "_sa09", "lang": "en", "location": "Chicago/ Abha ", "create_at": date("2010-10-08"), "description": "{وَإِذَا سَأَلَكَ عِبَادِي عَنِّي فَإِنِّي قَرِيبٌ أُجِيبُ دَعْوَةَ الدَّاعِي إِذَا دَعَانِ} \n(يوماً مـا)", "followers_count": 115, "friends_count": 90, "statues_count": 14841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130771716304896, "text": "15 lads", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1494253124, "name": "artush", "screen_name": "gasparyann_", "lang": "en", "location": "orwp//\\bhs polo ", "create_at": date("2013-06-08"), "description": "my archive", "followers_count": 119, "friends_count": 102, "statues_count": 422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130771947151360, "text": "@maddypollick ����������", "in_reply_to_status": 724130642812792832, "in_reply_to_user": 758708629, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 758708629 }}, "user": { "id": 701409944, "name": "Tink ♤♡♢♧", "screen_name": "D_Martink", "lang": "en", "location": "Pittsburgh, PA Highlands High", "create_at": date("2012-07-17"), "description": "Loyalty brings forth Royalty,\nAmbition is priceless, it's something that's in ya veins, #R4L #R.I.P Danny and Ryan", "followers_count": 1646, "friends_count": 2240, "statues_count": 20999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brackenridge, PA", "id": "b28a9b2f4e0de28e", "name": "Brackenridge", "place_type": "city", "bounding_box": rectangle("-79.750791,40.602192 -79.733798,40.615603") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4207976, "cityName": "Brackenridge" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130772014125056, "text": "@51Jnash I was :(", "in_reply_to_status": 724130719396585475, "in_reply_to_user": 2905616715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2905616715 }}, "user": { "id": 151437124, "name": "babygirl", "screen_name": "paigeimmanivong", "lang": "en", "location": "houston", "create_at": date("2010-06-03"), "description": "when Vietnam met the hood we got you - @versacesir | #tcsquad5ever | sc: @paigeimmanivong", "followers_count": 283, "friends_count": 321, "statues_count": 20049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130772077015042, "text": "Happy birthday beautiful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2901791399, "name": "Noe Ocampo", "screen_name": "fkn_noe", "lang": "en", "location": "West Los Angeles, Los Angeles", "create_at": date("2014-12-01"), "description": "null", "followers_count": 65, "friends_count": 75, "statues_count": 344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130772760727552, "text": "03:00 47.3°F Feels:47.3°F (Hi52.2°F/Lo47.3°F) Hum:86% Wnd:--- 0.0MPH Baro:29.81in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 199, "friends_count": 262, "statues_count": 28713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130772911792128, "text": "@ninfias CUTE", "in_reply_to_status": 724130397978763264, "in_reply_to_user": 2952918340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2952918340 }}, "user": { "id": 1950897883, "name": "✨Alpha Omega Pivi✨", "screen_name": "MakaiWars", "lang": "en", "location": "East Rochester, NY", "create_at": date("2013-10-09"), "description": "Hey! I'm Pivi! ◆19/M ◆ Artist ◆Splatoon + Disgaea Enthusiast ◆ College Freshman ◆ ディスガイアをプレーしてください !◆", "followers_count": 332, "friends_count": 403, "statues_count": 33169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brighton, NY", "id": "94779fa8bb1f79a8", "name": "Brighton", "place_type": "city", "bounding_box": rectangle("-77.694219,43.087472 -77.516194,43.167065") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608257, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773008306177, "text": "Adelman Paul Law Office, Anchorage, AK, 99501 - https://t.co/krCFurWXTD\n\n#adelmanpaullawoffice #lawyer #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-149.895342,61.2178883"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "hashtags": {{ "adelmanpaullawoffice", "lawyer", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 495, "friends_count": 482, "statues_count": 464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773104766976, "text": "Wind 0.0 mph WNW. Barometer 30.104 in, Steady. Temperature 31.9 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773184462848, "text": "2:44am (@ Prince Watchcottage in Deer Park, NY, NY) https://t.co/3OtaHlR86t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.32960373,40.75410274"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273302802, "name": "LennyLynn P.", "screen_name": "Pantherstar22", "lang": "en", "location": "Lennyville, NY", "create_at": date("2011-03-27"), "description": "I'm a church-goer, loves to shop, loves my boyfriend, family and friends. Loves God the most. Loves to listen to anything but rap. Loves to travel anywhere.", "followers_count": 387, "friends_count": 2058, "statues_count": 63876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deer Park, NY", "id": "081822bec6f679d4", "name": "Deer Park", "place_type": "city", "bounding_box": rectangle("-73.354311,40.740662 -73.295069,40.783594") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3619972, "cityName": "Deer Park" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773599707139, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Falling slowly. Temperature 58.5 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773737984000, "text": "Whoever Tf thought it was funny to try and fuck with my house made a huge mistake. Caught you fools and made y'all look like bitches.✌��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371021460, "name": "J. Belfort", "screen_name": "King_Joey15", "lang": "en", "location": "HTown", "create_at": date("2011-09-09"), "description": "I'm All Bite No Bark", "followers_count": 809, "friends_count": 1471, "statues_count": 27210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773817765888, "text": "where can I go get iphone screen fixed?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1546671228, "name": "derionnnn❤️", "screen_name": "_beaaautiful", "lang": "en", "location": "Marrero, LA", "create_at": date("2013-06-25"), "description": "null", "followers_count": 599, "friends_count": 481, "statues_count": 21623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130773889064960, "text": "can't stand a grown person who act more like a child than I do.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3140904306, "name": "susie carmichael", "screen_name": "lyssasowavey", "lang": "en", "location": "clt", "create_at": date("2015-04-05"), "description": "@antdavis23 the loml ❣", "followers_count": 661, "friends_count": 404, "statues_count": 45229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130774103023616, "text": "always stay on top", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4884840346, "name": "babygirl ❣", "screen_name": "princesslege_", "lang": "en", "location": "in da books", "create_at": date("2016-02-07"), "description": "Mrs.Lawrimore...only fwu if u real", "followers_count": 315, "friends_count": 301, "statues_count": 4126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130774354694144, "text": "You think it would be polite to leave a note. Fuck It.\nOrale Harriet. https://t.co/SFetwp0fdc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1978641510, "name": "RickVoid", "screen_name": "rick_void", "lang": "en", "location": "{Chentalonia} Manassas, VA", "create_at": date("2013-10-21"), "description": "null", "followers_count": 43, "friends_count": 69, "statues_count": 759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linton Hall, VA", "id": "000ab6b1404a5cd8", "name": "Linton Hall", "place_type": "city", "bounding_box": rectangle("-77.622033,38.717097 -77.534718,38.78464") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5145957, "cityName": "Linton Hall" } }
+{ "create_at": datetime("2016-04-24T00:00:04.000Z"), "id": 724130774421782528, "text": "I think I overdosed on caffeine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 276756995, "name": "Drew", "screen_name": "PabstDrewRibbon", "lang": "en", "location": "Potomac/Coral Gables/Rehoboth", "create_at": date("2011-04-03"), "description": "null", "followers_count": 156, "friends_count": 278, "statues_count": 6665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Gables, FL", "id": "778af41ffb719450", "name": "Coral Gables", "place_type": "city", "bounding_box": rectangle("-80.297165,25.627038 -80.245009,25.771859") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214250, "cityName": "Coral Gables" } }
+{ "create_at": datetime("2016-04-24T00:00:05.000Z"), "id": 724130775269052416, "text": "RT @hikellz \"Imma chicken nugget\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3158570119 }}, "user": { "id": 91930859, "name": "Nikki", "screen_name": "ElizabethNW_", "lang": "en", "location": "null", "create_at": date("2009-11-22"), "description": "O-H BGSU TALONS UP!!", "followers_count": 292, "friends_count": 253, "statues_count": 10433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-04-24T00:00:05.000Z"), "id": 724130775331807234, "text": "When @Borgore played Forbes I completely when ape shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 66995579 }}, "user": { "id": 2918308492, "name": "DAMN JOSE", "screen_name": "depazapooluza", "lang": "en", "location": "arlington, tx", "create_at": date("2014-12-12"), "description": "PLUR, Honda, and THC. What more do you need?", "followers_count": 226, "friends_count": 227, "statues_count": 1818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-24T00:00:05.000Z"), "id": 724130775394738177, "text": "Have patience, God isn't finished yet.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4333887075, "name": "Alex Myers", "screen_name": "AlpacaMaster8", "lang": "en", "location": "null", "create_at": date("2015-11-30"), "description": "•confidence•faith•sacrifice•", "followers_count": 133, "friends_count": 308, "statues_count": 357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clifton, VA", "id": "bbb2331ac69ffdd9", "name": "Clifton", "place_type": "city", "bounding_box": rectangle("-77.391606,38.774892 -77.380254,38.784956") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5117376, "cityName": "Clifton" } }
+{ "create_at": datetime("2016-04-24T00:00:05.000Z"), "id": 724130775529062400, "text": "@popyonce def! Why do I feel like Freedom shoulda been the opening track", "in_reply_to_status": 724130489930383360, "in_reply_to_user": 3190427365, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3190427365 }}, "user": { "id": 545701924, "name": "Shit Kicker", "screen_name": "Imma_Twit_On_U", "lang": "en", "location": "West Palm Beach, FL", "create_at": date("2012-04-04"), "description": "#Lemonade", "followers_count": 483, "friends_count": 442, "statues_count": 29583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Palm Beach, FL", "id": "4de072969805ac41", "name": "West Palm Beach", "place_type": "city", "bounding_box": rectangle("-80.2006,26.644671 -80.044639,26.799121") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1276600, "cityName": "West Palm Beach" } }
+{ "create_at": datetime("2016-04-24T00:00:05.000Z"), "id": 724130775583453185, "text": "crazy 4 u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396958110, "name": "sam keepit 1thousand", "screen_name": "samanthafuller8", "lang": "en", "location": "atx", "create_at": date("2011-10-23"), "description": "null", "followers_count": 443, "friends_count": 533, "statues_count": 5379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493141533925378, "text": "wtff ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2597738929, "name": "Ivana S. Zelaya♡", "screen_name": "ivanaszelaya", "lang": "en", "location": "Land O' Lakes, FL 504✈️813", "create_at": date("2014-07-01"), "description": "•Hondureña♡♕• lolhs'16✨• rip dev♡", "followers_count": 692, "friends_count": 403, "statues_count": 9431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Connerton, FL", "id": "00b6a1c7611dcf24", "name": "Connerton", "place_type": "city", "bounding_box": rectangle("-82.520132,28.251311 -82.441213,28.324493") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1213965, "cityName": "Connerton" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493141563314176, "text": "virgin gang or no gang �������� https://t.co/90DvSSIcZ9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 4212304889, "name": "sc: tell_ybhi", "screen_name": "haha_sole_yb", "lang": "en", "location": "null", "create_at": date("2015-11-17"), "description": "null", "followers_count": 137, "friends_count": 209, "statues_count": 747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493141806559232, "text": "Temp 44.2° Hi/Lo 44.5/42.5 Rng 2.0° WC 44.2° Hmd 79% Rain 0.00\" Storm 0.00\" BAR 30.059 Falling DP 38.1° Wnd 0mph Dir --- Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493141970018305, "text": "@juwwiebear FUCKING SAME������", "in_reply_to_status": 724489454459592705, "in_reply_to_user": 595748311, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 595748311 }}, "user": { "id": 1368014071, "name": "γαμώ u", "screen_name": "HarebearIII", "lang": "en", "location": "null", "create_at": date("2013-04-20"), "description": "σας μισώ όλους", "followers_count": 193, "friends_count": 167, "statues_count": 13538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493142045528065, "text": "Been playing for 15 minutes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261912267, "name": "Këńñ☸", "screen_name": "Frootynloops", "lang": "en", "location": "null", "create_at": date("2011-03-06"), "description": "May your neighbors respect you☮Trouble neglect you☯Angels protect you✳️ and heaven accept you✝", "followers_count": 217, "friends_count": 140, "statues_count": 35676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493142045552640, "text": "So much love, transformation, gratitude, grace and generosity oozing from this group of Akashic… https://t.co/vavba9dMQH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.241,33.1936"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403638850, "name": "HollyHawkins Marwood", "screen_name": "HollyHMarwood", "lang": "en", "location": "Escondido, California USA", "create_at": date("2011-11-02"), "description": "Owner: Soul Genesis: Advanced Certified Akashic Teacher & Practitioner; Channel for High Council of Orion #akashicrecords #highcounciloforion #channeling", "followers_count": 616, "friends_count": 702, "statues_count": 1562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493142125363200, "text": "Edibles got me lit��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1419165540, "name": "Rico", "screen_name": "Rich_Romney", "lang": "en", "location": "Philadelphia PA", "create_at": date("2013-05-10"), "description": "Aℹ️️P #LongLiveAllThoseIveLost Just living my life", "followers_count": 604, "friends_count": 539, "statues_count": 16777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493142569807872, "text": "@denalifm I had about 2k rpm left but decided to stop and get the kit haha", "in_reply_to_status": 724492811228180481, "in_reply_to_user": 2848250268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2848250268 }}, "user": { "id": 310751535, "name": "Dalton James", "screen_name": "DaltonDesrocher", "lang": "en", "location": "null", "create_at": date("2011-06-04"), "description": "Apprentice electrician for the IBEW Snapchat: daltondesrocher", "followers_count": 354, "friends_count": 378, "statues_count": 4868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493143022796801, "text": "Why did I just hear like 5 loud ass gun shot....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 489803351, "name": "MAKENNA LOPEZ!!", "screen_name": "lopez_makenna", "lang": "en", "location": "Tyler, TX", "create_at": date("2012-02-11"), "description": "19! NICK❤️❤️❤️ #TJC17", "followers_count": 1854, "friends_count": 901, "statues_count": 56259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493144071409664, "text": "Yasssss https://t.co/52gH3Y2PSd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 35478824, "name": "7", "screen_name": "AppleciderPapi", "lang": "en", "location": "LA", "create_at": date("2009-04-26"), "description": "18. Muslim. Beautiful. Genius. ⭐️", "followers_count": 6128, "friends_count": 954, "statues_count": 169805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493144608251904, "text": "ωєℓℓ ѕнє ρи тнє νєяgє", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "uk", "is_retweet": false, "user": { "id": 626929308, "name": "kenJ", "screen_name": "KendallJamal23", "lang": "en", "location": "Mention4FollowBackButFollow", "create_at": date("2012-07-04"), "description": "The Name Is Ken, that's all ya need to know ⚡️ #SEN16R ‼️ #BLVS23D wife: @briaaa23 ❤️ Rip Granddad ✨", "followers_count": 1320, "friends_count": 2126, "statues_count": 41169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallulah, LA", "id": "635b800a9767abcf", "name": "Tallulah", "place_type": "city", "bounding_box": rectangle("-91.208068,32.386504 -91.169316,32.425418") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22065, "countyName": "Madison", "cityID": 2274690, "cityName": "Tallulah" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493144616755200, "text": "�� @ The Fletcher Home https://t.co/Auuoi5g7Bt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.31737431,36.30440829"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252334082, "name": "SabrinaLee Fletcher", "screen_name": "brinalee08", "lang": "en", "location": "Visalia ", "create_at": date("2011-02-14"), "description": "Enjoying life Living for God & loving my husband ❤️", "followers_count": 223, "friends_count": 375, "statues_count": 1606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-04-25T00:00:00.000Z"), "id": 724493145459810304, "text": "I. Am. Done. @mwatkins32 https://t.co/GpEt4brlJd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39099545 }}, "user": { "id": 30546905, "name": "Leighton Parton", "screen_name": "LeightonParton", "lang": "en", "location": "Southside, AL", "create_at": date("2009-04-11"), "description": "I really really love my dog and sloths. I kinda like @mwatkins32.", "followers_count": 950, "friends_count": 694, "statues_count": 47220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Southside, AL", "id": "0060a15f2aabf113", "name": "Southside", "place_type": "city", "bounding_box": rectangle("-86.070513,33.852599 -85.985964,33.946651") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1055, "countyName": "Etowah", "cityID": 171832, "cityName": "Southside" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493146139185155, "text": "03:00 57.4°F Feels:57.4°F (Hi59.9°F/Lo57.4°F) Hum:73% Wnd:--- 0.0MPH Baro:29.67in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 199, "friends_count": 262, "statues_count": 28734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493146378248193, "text": "What upsets me, is how guys normally treat me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40109482, "name": "John Cruz", "screen_name": "_JohnnyCouture", "lang": "en", "location": "Palmdale,Ca", "create_at": date("2009-05-14"), "description": "The Best At Being Average", "followers_count": 331, "friends_count": 451, "statues_count": 13745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493146747326465, "text": "I need to learn how to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3566340254, "name": "Toucan Sam", "screen_name": "sammiesulkin", "lang": "en", "location": "null", "create_at": date("2015-09-14"), "description": "I had a cute matching bio with @ewpriscilla but she ruined it", "followers_count": 123, "friends_count": 150, "statues_count": 1835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493147410030592, "text": "Bout to lucid dream brah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2859531515, "name": "GΔβΣ MΩRGΔΠ™", "screen_name": "waffelgang", "lang": "en", "location": "WAZZZ UPPPP", "create_at": date("2014-11-03"), "description": "Seeking bliss, I'm here to talk. so talk to me", "followers_count": 1052, "friends_count": 395, "statues_count": 3329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enumclaw, WA", "id": "1b209a6091ecebd0", "name": "Enumclaw", "place_type": "city", "bounding_box": rectangle("-122.020908,47.176005 -121.952293,47.226032") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5322045, "cityName": "Enumclaw" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493147535859713, "text": "@ebbtideapp Tide in Strawberry Bay, Washington 04/25/2016\n Low 1:37am 5.2\nHigh 6:43am 7.1\n Low 1:36pm -0.3\nHigh 9:08pm 7.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.7167,48.5667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 79, "friends_count": 1, "statues_count": 27360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493147548557312, "text": "My heart is broken", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770375371, "name": "Mollie J", "screen_name": "wellheyMJ", "lang": "en", "location": "the whee WCU ● potm ", "create_at": date("2012-08-20"), "description": "yaaa baby ● sh // ka // sc ♡ ● dreams big, govt small", "followers_count": 837, "friends_count": 1154, "statues_count": 13525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cullowhee, NC", "id": "4e7a5a46faf1356c", "name": "Cullowhee", "place_type": "city", "bounding_box": rectangle("-83.200975,35.29043 -83.159039,35.332791") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37099, "countyName": "Jackson", "cityID": 3715880, "cityName": "Cullowhee" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493147573768192, "text": "04/25@03:00 - Temp 42.8F, WC 42.8F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.052in, Falling slowly. Rain 0.00in. Hum 84%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493147875614720, "text": "\"Keeps the TL lit\"\n\n�� https://t.co/XTa5htMWKJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1922054466, "name": "Nigromancyee", "screen_name": "NextRollBounce", "lang": "en", "location": "Lewisville, TX", "create_at": date("2013-09-30"), "description": "⠀⠀⠀⠀⠀⠀backup @TheRollBounce⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀#ThuggerHive⠀#IgnantTribe⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ #Elite4 ⠀⠀⠀⠀⠀⠀⠀⠀⠀#FakeSleep", "followers_count": 9484, "friends_count": 4839, "statues_count": 99075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493148378918913, "text": "No power outage here ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85112252, "name": "Mat", "screen_name": "mattagrat", "lang": "en", "location": "null", "create_at": date("2009-10-25"), "description": "null", "followers_count": 503, "friends_count": 432, "statues_count": 12985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493148748058625, "text": "Someone tickle my back so i can fall asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132700389, "name": "YOSS", "screen_name": "yossitzel", "lang": "en", "location": "Tucson, AZ", "create_at": date("2010-04-13"), "description": "takes a young queen to know one • DA • SC/IG: yosssitzel", "followers_count": 1056, "friends_count": 535, "statues_count": 35106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Douglas, AZ", "id": "52d04cba03ddef94", "name": "Douglas", "place_type": "city", "bounding_box": rectangle("-109.568104,31.334002 -109.511419,31.371493") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise", "cityID": 420050, "cityName": "Douglas" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493148756418560, "text": "Happy birthday to my other half �� �� ✨ @PeaceLoveRuby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2878862816 }}, "user": { "id": 2707534668, "name": "Audrey ♡", "screen_name": "floresaudrey04", "lang": "en", "location": "null", "create_at": date("2014-08-04"), "description": "bphs, Alex ✨", "followers_count": 448, "friends_count": 325, "statues_count": 2638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493148991320064, "text": "G Herbo x Pain ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4272890716, "name": "scottieo", "screen_name": "_scottieoo", "lang": "en", "location": "null", "create_at": date("2015-11-25"), "description": "203✈️904✈️619.....TBA R.I.P Markel Byrd", "followers_count": 894, "friends_count": 716, "statues_count": 3762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493149393948674, "text": "Wind 1.0 mph SW. Barometer 29.581 in, Falling slowly. Temperature 58.6 °F. Rain today 0.09 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-25T00:00:01.000Z"), "id": 724493149658189824, "text": "@isthatxander Happy Birthday my love ��", "in_reply_to_status": -1, "in_reply_to_user": 2458985190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2458985190 }}, "user": { "id": 2348042856, "name": "bri", "screen_name": "brii_breyy", "lang": "en", "location": "California, USA", "create_at": date("2014-02-16"), "description": "Im a screamer. Not sexually, just in general.", "followers_count": 434, "friends_count": 343, "statues_count": 13064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493151080091648, "text": "Mads not good...again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 528793670, "name": "Madison Mahnesmith", "screen_name": "Madisonanneeee", "lang": "en", "location": "Northern Arizona University☼", "create_at": date("2012-03-18"), "description": "miley what's good? || delta delta delta", "followers_count": 880, "friends_count": 491, "statues_count": 55335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493151205871617, "text": "My sleeping schedule is so fucked up . I'm always tired ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384128106, "name": "Ruellane Anne ♡", "screen_name": "ItsLilRue", "lang": "en", "location": "null", "create_at": date("2011-10-02"), "description": "SouthStockton", "followers_count": 1204, "friends_count": 878, "statues_count": 16549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493151378001920, "text": "Wind 0.0 mph ESE. Barometer 29.790 in, Steady. Temperature 66.5 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493151461842944, "text": "My will be done in Malkuth as it is in Yesod.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20169442, "name": "Aaron David", "screen_name": "hodosholos", "lang": "en", "location": "NC", "create_at": date("2009-02-05"), "description": "Currently in prevocation, with avocations, seeking vocation.", "followers_count": 373, "friends_count": 904, "statues_count": 2116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weaverville, NC", "id": "0003a816002eaa32", "name": "Weaverville", "place_type": "city", "bounding_box": rectangle("-82.62914,35.668393 -82.515121,35.754667") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37021, "countyName": "Buncombe", "cityID": 3771560, "cityName": "Weaverville" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493151520612352, "text": "Loool @GoodieBennett you agree? https://t.co/pLeu1DZEbm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 364680967 }}, "user": { "id": 439796693, "name": "Ash Ketchum ;", "screen_name": "_PhoenixKing", "lang": "en", "location": "JA ✈️NY", "create_at": date("2011-12-17"), "description": "enigmatic persona, twisted mind ; fwm. || Young entrepreneur, poet, fire dancer /breather.", "followers_count": 546, "friends_count": 686, "statues_count": 71858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493152049061889, "text": "@ASAPFiRECROTCH @NBA haha yeah it should be a close series, saying they make it past the mavs tomorrow", "in_reply_to_status": 724492815061770240, "in_reply_to_user": 946808754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 946808754, 19923144 }}, "user": { "id": 272536457, "name": "Blaine Harriman", "screen_name": "Bballerblaine", "lang": "en", "location": "ATX~~~Cstat", "create_at": date("2011-03-26"), "description": "you can find a pot of gold on the other side of the blainebow", "followers_count": 486, "friends_count": 336, "statues_count": 6070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493152300584960, "text": "Temp: 63.2°F Wind:0.0mph Pressure: 29.939hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493152413974529, "text": "Wind 0.0 mph ---. Barometer 29.916 in, Falling slowly. Temperature 56.4 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153017851904, "text": "Who is this boy �� https://t.co/CpT5yqu8At", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333577108, "name": "Valerie", "screen_name": "CheekyVal_7", "lang": "en", "location": "null", "create_at": date("2011-07-11"), "description": "♡ focused on loving myself ♡", "followers_count": 333, "friends_count": 1009, "statues_count": 23131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153156272128, "text": "66.3F (Feels: 66.3F) - Humidity: 86% - Wind: 0.7mph E - Gust: 2.2mph - Pressure: 1007.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 236680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153412231169, "text": "seriously this is how they act in every damn episode ctfu https://t.co/3tGNyS47m5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3849123436, "name": "francesca", "screen_name": "frankierae_", "lang": "en", "location": "Philly", "create_at": date("2015-10-02"), "description": "Starbucks soothes the soul. ✨ Shippensburg University ⚓️", "followers_count": 903, "friends_count": 613, "statues_count": 33821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shippensburg, PA", "id": "a3d888f1d0b5d5a2", "name": "Shippensburg", "place_type": "city", "bounding_box": rectangle("-77.549194,40.03646 -77.499257,40.067857") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4270352, "cityName": "Shippensburg" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153575694337, "text": "LMFAOOOO @ leeko pic https://t.co/WlXjbc9CGE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605507434, "name": "✨", "screen_name": "adnariv", "lang": "en", "location": "UCSB isla vista✨", "create_at": date("2012-06-11"), "description": "half super woman but down to earth , blessed LongLiveGio✨", "followers_count": 2205, "friends_count": 984, "statues_count": 130541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153596768256, "text": "Temp 43.7°F Wind Chill 43.7°F RH 89% Wind 0.0 --- Gust 0.0 --- SLP 30.018 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 111, "friends_count": 63, "statues_count": 34956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-25T00:00:02.000Z"), "id": 724493153898635264, "text": "@Tiffanycarlie5 I love you tiffy��", "in_reply_to_status": 724492541626748928, "in_reply_to_user": 453323602, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453323602 }}, "user": { "id": 1046802504, "name": "kar", "screen_name": "KaraDianthia", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-12-29"), "description": "ripyanni•ripray•ripnick•ripadriana", "followers_count": 660, "friends_count": 588, "statues_count": 22347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493154334855168, "text": "Ohhh it's a shame you let this good love go to waste. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43721946, "name": "carol. ", "screen_name": "_iAmThugga", "lang": "en", "location": "IG: @_iAmThugga", "create_at": date("2009-05-31"), "description": "null", "followers_count": 1193, "friends_count": 852, "statues_count": 39239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493154531975168, "text": "Wind 2.0 mph S. Barometer 1010.74 mb, Steady. Temperature 58.6 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493154708238337, "text": "Manoa Sampler #appetizers #LocalKineGrinds @ Tokoname https://t.co/8Ws83Ej3U1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.81057932,21.30826529"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "appetizers", "LocalKineGrinds" }}, "user": { "id": 41039648, "name": "Jason M. Nishimura", "screen_name": "jnishimura", "lang": "en", "location": "Honolulu, Hawaii", "create_at": date("2009-05-18"), "description": "Digital Media Specialist - Auto Gearhead - Moderate Health Nut", "followers_count": 631, "friends_count": 613, "statues_count": 14380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493154884317184, "text": "Couldn't be happier with life rn :)))", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3238188424, "name": "Ashley", "screen_name": "Ashluuu_", "lang": "en", "location": "paradise", "create_at": date("2015-05-05"), "description": "mamí", "followers_count": 562, "friends_count": 594, "statues_count": 1470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155157041152, "text": "Hustling getting my funds right . a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2922388197, "name": "MaJuan Bates Jr", "screen_name": "JuanTheTruth", "lang": "en", "location": "St. Louis ", "create_at": date("2014-12-14"), "description": "Just trying to make a difference. Grind x Pray™ Blessed and Ambitious.", "followers_count": 242, "friends_count": 255, "statues_count": 2022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Shores, FL", "id": "77ad88ac9caf6874", "name": "Miami Shores", "place_type": "city", "bounding_box": rectangle("-80.210559,25.851456 -80.164547,25.886421") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245175, "cityName": "Miami Shores" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155459059713, "text": "Ripley SW Limestone Co. Temp: 62.2°F Wind:0.7mph Pressure: 993.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155559702528, "text": "Wind 0.0 mph N. Barometer 30.07 in, Steady. Temperature 47.5 °F. Rain today 0.00 in. Humidity 32%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 118, "statues_count": 159315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155718950912, "text": "Couldn't be more content with life recently ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967423232, "name": "JMilllzz", "screen_name": "jordanmiller719", "lang": "en", "location": "Sequim, WA", "create_at": date("2012-11-23"), "description": "null", "followers_count": 663, "friends_count": 446, "statues_count": 4682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sequim, WA", "id": "7e05a40db75ea147", "name": "Sequim", "place_type": "city", "bounding_box": rectangle("-123.154156,48.054375 -123.039812,48.12363") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53009, "countyName": "Clallam", "cityID": 5363385, "cityName": "Sequim" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155727343617, "text": "We need to take pics lol I'm mad I don't have our bomb old ones BUT HAPPY BIRTHDAY CRAZY A** RY I LOVE U������ https://t.co/0BnBNAYWiB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 707848551951249408, "name": "MAY 18TH", "screen_name": "AndreaMcCabe_", "lang": "en", "location": "null", "create_at": date("2016-03-10"), "description": "#longlivegio #longlivelance #longlivepaul", "followers_count": 820, "friends_count": 520, "statues_count": 5045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493155958026240, "text": "Im here for good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1691369947, "name": "Daddy ⭐️", "screen_name": "Pappi_Luiss", "lang": "en", "location": "null", "create_at": date("2013-08-22"), "description": "Los Angeles , CA", "followers_count": 264, "friends_count": 165, "statues_count": 18973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156100767744, "text": "Nah. https://t.co/aNTvgYb7Eu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 780644084, "name": "ANM", "screen_name": "_aasiaa2", "lang": "en", "location": "Tuscaloosa, AL", "create_at": date("2012-08-25"), "description": "Rest Peacefully Malissa..️❣#Stillman20", "followers_count": 10518, "friends_count": 10215, "statues_count": 89945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northport, AL", "id": "003779400ddc8a06", "name": "Northport", "place_type": "city", "bounding_box": rectangle("-87.644402,33.207777 -87.540968,33.333623") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 155200, "cityName": "Northport" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156222423040, "text": "Adam Davis Attorney At Law - https://t.co/N7JNIM1bOZ\n\n#adamdavisattorneyatlaw #lawyer #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.918776,33.583231"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "adamdavisattorneyatlaw", "lawyer", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 502, "friends_count": 482, "statues_count": 488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156272631808, "text": "I just finished my series I've been reading since 2012.. Heart break.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1342975886, "name": "Brittney Bates •", "screen_name": "brittneybates_4", "lang": "en", "location": "Tyler, TX", "create_at": date("2013-04-10"), "description": "its all happening | I just want to see the world •living my life by my soul, happily •", "followers_count": 672, "friends_count": 475, "statues_count": 6893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156411035648, "text": "There's Been A Startling Rise In Suicide Rates In The U.S. https://t.co/lMKkYmXm74", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336269937, "name": "Cesar Rivadeneyra", "screen_name": "CeRivadeneyra", "lang": "en", "location": "Northridge,CA", "create_at": date("2011-07-15"), "description": "Real Estate Agent helping home owners and buyers in their real estate needs. Helping under water home owners avoiding foreclosure. Short Sale Specialist .", "followers_count": 580, "friends_count": 1445, "statues_count": 3621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156683649024, "text": "Nightly searches for a-bed and I just came off tour with Troy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1499570108, "name": "Stale", "screen_name": "Justinstal", "lang": "en", "location": "Tucson, AZ", "create_at": date("2013-06-10"), "description": "Arizona Wildcats c/o '19 #BLM LA Confidential", "followers_count": 592, "friends_count": 387, "statues_count": 27582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156742389761, "text": "** a week ago \nMe: Ok I won't stay up til 3AM working on my research project\n** a week later \nMe: ... I really need to stop ginxing myself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 742160838, "name": "Sarah McClenaghan", "screen_name": "smmcclen", "lang": "en", "location": "null", "create_at": date("2012-08-06"), "description": "Psalm 128:1|Eph 2:1-10|Col 3:12-17|Psalm 16:19|Making you laugh is my obligation my dear||LHU '19", "followers_count": 442, "friends_count": 420, "statues_count": 5743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownstown, PA", "id": "001890eb0ef2d342", "name": "Brownstown", "place_type": "city", "bounding_box": rectangle("-76.237444,40.112486 -76.197477,40.142248") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42071, "countyName": "Lancaster", "cityID": 4209416, "cityName": "Brownstown" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156905930753, "text": "Lord knows I needed this one.. ❤️���� \nhttps://t.co/UyWNM8pNKt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258425474, "name": "Liv", "screen_name": "OliviaFennig", "lang": "en", "location": "null", "create_at": date("2011-02-27"), "description": "TX ✈️ IN - WTX ✈️ ATX", "followers_count": 265, "friends_count": 548, "statues_count": 1083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493156956299265, "text": "if somebody does not want to be w/ u or around u they will make it their business to find another way in life to not deal w/ yo ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630779917, "name": "cyn", "screen_name": "cynncardenas", "lang": "en", "location": "null", "create_at": date("2012-07-08"), "description": "layin low • town bizz", "followers_count": 1054, "friends_count": 304, "statues_count": 17873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157103206400, "text": "This paper fucking me up man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270096879, "name": "Ken", "screen_name": "KenballsNYoJawz", "lang": "en", "location": "null", "create_at": date("2011-03-21"), "description": "null", "followers_count": 2246, "friends_count": 1777, "statues_count": 77750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carbondale, IL", "id": "20d0abc87d11501a", "name": "Carbondale", "place_type": "city", "bounding_box": rectangle("-89.294485,37.670875 -89.151934,37.764433") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17077, "countyName": "Jackson", "cityID": 1711163, "cityName": "Carbondale" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157199552513, "text": "❤️ my freshman bf's glasses������ https://t.co/o8XeVHVo6l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2214712386, "name": "christina", "screen_name": "youungzoo_", "lang": "en", "location": "bay area", "create_at": date("2013-11-25"), "description": "4114 • 7313 snap: christinazuniga", "followers_count": 477, "friends_count": 707, "statues_count": 16439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157262581761, "text": "@krista_noel https://t.co/zXsMWNo2GZ", "in_reply_to_status": -1, "in_reply_to_user": 549291979, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 549291979 }}, "user": { "id": 308534843, "name": "Briyoncé", "screen_name": "briiixo22", "lang": "en", "location": "Coastal Carolina", "create_at": date("2011-05-31"), "description": "It's kind of fun to do the impossible. #ΦΣΣ", "followers_count": 864, "friends_count": 784, "statues_count": 24707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Hill, SC", "id": "001e2a4a7b2f19a5", "name": "Red Hill", "place_type": "city", "bounding_box": rectangle("-79.052947,33.694141 -78.899479,33.838132") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4559190, "cityName": "Red Hill" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157434560513, "text": "That one night we tried the condom challenge like 30092928 times https://t.co/Bfn5J1uqYs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 900805110, "name": "Maddie Bowie", "screen_name": "MaddieGBowie", "lang": "en", "location": "Virginia, USA", "create_at": date("2012-10-23"), "description": "RU' 19 \n♦Special Education♦", "followers_count": 261, "friends_count": 185, "statues_count": 2216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Radford, VA", "id": "a265d5d344dbd61d", "name": "Radford", "place_type": "city", "bounding_box": rectangle("-80.595589,37.090022 -80.513486,37.146017") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51750, "countyName": "Radford", "cityID": 5165392, "cityName": "Radford" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157728026624, "text": "���� https://t.co/2GBRRIggzx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 59176407, "name": "Bari⛹", "screen_name": "JabariMoore_", "lang": "en", "location": "Garland, TX", "create_at": date("2009-07-22"), "description": "Chill vibes only | Sc: Jabarimoore3 RIP DAD | Libra♎️ | 20", "followers_count": 3435, "friends_count": 971, "statues_count": 165754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493157971333120, "text": "it shows up on all my pics too.... it's new from today & won't go away with a reset. @AppleSupport @Apple https://t.co/uf7Yn7Q4kC", "in_reply_to_status": 724492837274832896, "in_reply_to_user": 33818434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3309375033, 380749300 }}, "user": { "id": 33818434, "name": "k8 ☆", "screen_name": "dolewhipfloat", "lang": "en", "location": "disneyland native / 22 / ENFJ", "create_at": date("2009-04-20"), "description": "i am the plastic fried egg in captain EO .", "followers_count": 618, "friends_count": 219, "statues_count": 5421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-04-25T00:00:03.000Z"), "id": 724493158155845634, "text": "Wind 9.6 mph SSE. Barometer 29.765 in, Falling slowly. Temperature 65.9 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493158361403396, "text": "@LordShockwave cutie������", "in_reply_to_status": 724492634614431744, "in_reply_to_user": 419113148, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 419113148 }}, "user": { "id": 986874618, "name": "Diegz", "screen_name": "DiegoBurrows", "lang": "en", "location": "mattyb's crib", "create_at": date("2012-12-03"), "description": "Lit like bic #hhh", "followers_count": 556, "friends_count": 522, "statues_count": 5769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493158550147073, "text": "Real shit https://t.co/wroSzmKxRO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3286033861, "name": "$adthotty", "screen_name": "champagnkisses", "lang": "en", "location": "San Jose, CA", "create_at": date("2015-07-20"), "description": "follow me on Ig an snap ho", "followers_count": 36, "friends_count": 40, "statues_count": 93 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493158931800064, "text": "How she paused before pronouncing apple jacks �� https://t.co/mxoE5F4MWv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977329266, "name": "Raymond", "screen_name": "yungjetta", "lang": "en", "location": "flexico", "create_at": date("2012-11-28"), "description": "all my role models died", "followers_count": 1328, "friends_count": 698, "statues_count": 34386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493158952767489, "text": "#MCE\nI can honestly say I love this kid with all my heart.���� @SamRossxx https://t.co/DkHgOkWc9a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MCE" }}, "user_mentions": {{ 2232405589 }}, "user": { "id": 879422076, "name": "Shai✨", "screen_name": "shaiannedomingu", "lang": "en", "location": "null", "create_at": date("2012-10-13"), "description": "FDT", "followers_count": 280, "friends_count": 695, "statues_count": 1263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493159061966848, "text": "@photosandbacon the Royal Canadian Mounties and armed forces will rule the world mahuahahah", "in_reply_to_status": 724478743780036608, "in_reply_to_user": 33160996, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33160996 }}, "user": { "id": 462460274, "name": "Ryan", "screen_name": "RyanBermuda97", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2012-01-12"), "description": "http://imgur.com/gallery/Ww89N Bermuda Bermuda Bermuda http://imgur.com/gallery/iQKIa", "followers_count": 960, "friends_count": 292, "statues_count": 87586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493159175053312, "text": "McGregor is a terrible sportsman he dogged interviews and still getting paid. Diaz in three RDS! https://t.co/ycgFX5xieL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716731314855133184, "name": "Jimmy Albanese", "screen_name": "JamesAlbanese69", "lang": "en", "location": "Pittsburgh, PA", "create_at": date("2016-04-03"), "description": "in a relationship!", "followers_count": 77, "friends_count": 172, "statues_count": 624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493159317835776, "text": "hahahahah omg so young and innocent https://t.co/DKRiVAgjDa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365807157, "name": "Bianca Carrera", "screen_name": "bianc_taina", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-08-31"), "description": "20 | Ecuadorian | Arizona State '18 | ΣΚ", "followers_count": 278, "friends_count": 275, "statues_count": 9067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493159506579458, "text": "Doctor need to prescribe me some xans asum cause I be bored af after like 2:13am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419149456, "name": "1700 Lil Malc", "screen_name": "GasMan_", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-11-22"), "description": "im too high fa dis shit. WHS Alumni '15 #ssu. #DRVG! #VPRACT #SWATS", "followers_count": 1842, "friends_count": 928, "statues_count": 89354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493159691083776, "text": "Wind 0.0 mph ---. Barometer 29.891 in, Steady. Temperature 60.5 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493160609533952, "text": "My emotions just did a whole 360", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1703660401, "name": "Sydney", "screen_name": "sydneyljordan", "lang": "en", "location": "null", "create_at": date("2013-08-26"), "description": "null", "followers_count": 643, "friends_count": 283, "statues_count": 10467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493160743878656, "text": "Wind 0.0 mph ---. Barometer 30.071 in, Falling. Temperature 43.0 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493160802459648, "text": "You = ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 734733948, "name": "Kate Montemayor", "screen_name": "_AlyshaKate", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-08-03"), "description": "XO | TESFAYE | SENIOR |", "followers_count": 1927, "friends_count": 1363, "statues_count": 34012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493160836112384, "text": "He traveled the ocean, & took the risk. Just to be able to see her again. #TSOUItsComplicated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TSOUItsComplicated" }}, "user": { "id": 4465757356, "name": "Jen❤️NY...", "screen_name": "itsmejenkx", "lang": "en", "location": "New York", "create_at": date("2015-12-12"), "description": "~ a story about family, first loves, second chances, and the moments in life that leads you back home ❤️ The Story of Us on Primetime Bida! 9:30 pm", "followers_count": 124, "friends_count": 35, "statues_count": 5692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493161301581824, "text": "As much as you pay for school you don't know any useful life skills", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141014427, "name": "CHRON", "screen_name": "chrondizzo", "lang": "en", "location": "L.A / the middle grounds", "create_at": date("2010-05-06"), "description": "Searching for a life worth living", "followers_count": 194, "friends_count": 151, "statues_count": 30132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493161741979648, "text": "Wind 0.0 mph ---. Barometer 29.888 in, Steady. Temperature 42.6 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 203, "friends_count": 59, "statues_count": 271109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493161935048704, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Falling slowly. Temperature 64.9 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-25T00:00:04.000Z"), "id": 724493162006208512, "text": "@xantia8 \nGRMA \n#SaoirseAbú", "in_reply_to_status": 724491455553781760, "in_reply_to_user": 117539485, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "SaoirseAbú" }}, "user_mentions": {{ 117539485 }}, "user": { "id": 28024611, "name": "tuigim", "screen_name": "Tuigim", "lang": "ga", "location": "Skype: gaeilgebheo", "create_at": date("2009-03-31"), "description": "Éireannach i gCathair na nAingeal, ag imeacht le sruth: suim agam i dteangachaí, nádúr, is cearta daonna http://alturl.com/s2dz7 m'achainí", "followers_count": 4010, "friends_count": 3921, "statues_count": 92167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493162744410112, "text": "I don't have a problem with black men dating outside their race.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2241790543, "name": "Queen", "screen_name": "SoulSister93", "lang": "en", "location": "Texas, USA", "create_at": date("2013-12-11"), "description": "22 | College Student | Majoring in Nursing | Taurus | Pan-Africanist | Activist | Pro-Black | Unapologetically Black | #BlackLivesMatter", "followers_count": 15078, "friends_count": 7977, "statues_count": 22107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493162857652226, "text": "Happy Birthday to the best friend in my world now and forever!!… https://t.co/xNM4qHGO0V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.02212653,33.95113219"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193375693, "name": "Sarah Church", "screen_name": "sportsfanvball", "lang": "en", "location": "Beaumont CA", "create_at": date("2010-09-21"), "description": "Live Life to the Fullest! Hug and Kiss your Loved Ones Every Single Day❤️", "followers_count": 213, "friends_count": 338, "statues_count": 10665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493163092537344, "text": "@HosaIsDope @KylieFurr @madddie_af ummmm WHERE YOU GOING BOIII you're apart of this too ���� https://t.co/qYQ89Jp5XW", "in_reply_to_status": 724491278071697408, "in_reply_to_user": 150303871, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150303871, 2597491141, 1913008933 }}, "user": { "id": 224892209, "name": "Kendall Hill", "screen_name": "ken_of_the_hill", "lang": "en", "location": "University of Houston", "create_at": date("2010-12-09"), "description": "Behind every successful person is a substantial amount of coffee. | ΠΚΦ Βeta Nu | #HtownTakeOver", "followers_count": 622, "friends_count": 598, "statues_count": 32526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493163277082624, "text": "me 24/7 https://t.co/S9goNSwR8K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 535770700, "name": "xandrie", "screen_name": "xandreezus", "lang": "en", "location": "LMHS ", "create_at": date("2012-03-24"), "description": "live long and prosper ♒️", "followers_count": 754, "friends_count": 680, "statues_count": 17850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493163340009472, "text": "2️⃣6️⃣������������ https://t.co/p2l2XX4O0X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 365249109, "name": "Gremlin $av", "screen_name": "Twon_Turner", "lang": "en", "location": "Bay Area, CA", "create_at": date("2011-08-30"), "description": "Antoine Turner aka Tron at Revere Tattoo shop.... I like spam musubi.... IG: Twon_Turner", "followers_count": 785, "friends_count": 711, "statues_count": 28423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493164585713664, "text": "21. I'm always goofy, so make sure you can take a joke. (Especially corny ones)", "in_reply_to_status": 724492830375190528, "in_reply_to_user": 248893204, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248893204, "name": "lil mama", "screen_name": "_SincerelyA_22", "lang": "en", "location": "WA", "create_at": date("2011-02-07"), "description": "I'm not latina", "followers_count": 1266, "friends_count": 482, "statues_count": 46198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Graham, WA", "id": "08a26cd724eca51d", "name": "Graham", "place_type": "city", "bounding_box": rectangle("-122.338034,47.016521 -122.206872,47.09723") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5327785, "cityName": "Graham" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493164782845952, "text": "she coo and shit , but I'm not bouta go in full detail ���� just gotta wait and see what happens ������ https://t.co/6NVx3dGUwH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2951321564, "name": "dudd2time$", "screen_name": "rjd4l", "lang": "en", "location": "null", "create_at": date("2014-12-29"), "description": "null", "followers_count": 176, "friends_count": 130, "statues_count": 453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rodeo, CA", "id": "712d61ba26321517", "name": "Rodeo", "place_type": "city", "bounding_box": rectangle("-122.274848,38.014234 -122.238027,38.057435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 662490, "cityName": "Rodeo" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493164854185984, "text": "Why am I up right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323008567, "name": "Linaaa.", "screen_name": "Lil_Linaaa", "lang": "en", "location": "vegasssss", "create_at": date("2011-06-23"), "description": "wax on wax off. SC: lil_linaaa", "followers_count": 1022, "friends_count": 915, "statues_count": 31670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493165143552001, "text": "@princesofimperf Good.", "in_reply_to_status": 724493104904990720, "in_reply_to_user": 251918855, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251918855 }}, "user": { "id": 262056626, "name": "KingCas.", "screen_name": "AyeRamra", "lang": "en", "location": "vibin wid da hunnies .", "create_at": date("2011-03-06"), "description": "20.", "followers_count": 511, "friends_count": 475, "statues_count": 24501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Frankfort, IL", "id": "ed29aaa34aadef2b", "name": "West Frankfort", "place_type": "city", "bounding_box": rectangle("-88.970144,37.873175 -88.894807,37.928015") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17055, "countyName": "Franklin", "cityID": 1780333, "cityName": "West Frankfort" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493165722390528, "text": "@iamatheistgirl my sister adopted her belief & truly does only good. No proselytizing. Don't know why she 'needs' it, being science-y, etc.", "in_reply_to_status": 724492653966954496, "in_reply_to_user": 3911216058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3911216058 }}, "user": { "id": 108813540, "name": "Barbara McCarren", "screen_name": "barbaramccarren", "lang": "en", "location": "Venice, California", "create_at": date("2010-01-26"), "description": "Educated, opinionated, over-heated & under-rated. Artist, feminist, world-traveler, animal lover & liberal. Obamabot forever.", "followers_count": 1381, "friends_count": 2246, "statues_count": 8168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493165894344704, "text": "https://t.co/zBanFiWKRT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 598786789, "name": "Evan Ryan", "screen_name": "Lil_BonQuiQui", "lang": "en", "location": "null", "create_at": date("2012-06-03"), "description": "y'allready know me. E. V. A. N.", "followers_count": 189, "friends_count": 231, "statues_count": 12382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lahaina, HI", "id": "249325c821f8b89a", "name": "Lahaina", "place_type": "city", "bounding_box": rectangle("-156.690186,20.855885 -156.657928,20.941656") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1542950, "cityName": "Lahaina" } }
+{ "create_at": datetime("2016-04-25T00:00:05.000Z"), "id": 724493166175485952, "text": "ANA AND I ARE TWINS BISH OMG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2793775223, "name": "Viachumba", "screen_name": "Shortgoku", "lang": "en", "location": "Planeta Namek", "create_at": date("2014-09-29"), "description": "Burning Calories by Jumping into Conclusions", "followers_count": 132, "friends_count": 132, "statues_count": 5019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-25T00:00:06.000Z"), "id": 724493166913544192, "text": "@ImJessiLang Very nice", "in_reply_to_status": 724425217406173184, "in_reply_to_user": 17949607, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17949607 }}, "user": { "id": 40513813, "name": "RockV.", "screen_name": "RockyVe", "lang": "en", "location": "FL, CT, GA & CA", "create_at": date("2009-05-16"), "description": "This is Rock (my real name...)...Weight training, Filipino martial arts, Sports Car racing , My Jeep...and satisfying my desire for animated comedy.", "followers_count": 148, "friends_count": 162, "statues_count": 2586 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-25T00:00:06.000Z"), "id": 724493167693688832, "text": "This headache ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 402786332, "name": "Rich Homie Romie", "screen_name": "Rommmmmie", "lang": "en", "location": "null", "create_at": date("2011-11-01"), "description": "Being confident in yourself is so rare that people actually get mad at you if you are. #ImWayyyBlessed", "followers_count": 449, "friends_count": 268, "statues_count": 25537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wylie, TX", "id": "5b96c12e41e49aa5", "name": "Wylie", "place_type": "city", "bounding_box": rectangle("-96.596044,32.981938 -96.489165,33.054983") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4880356, "cityName": "Wylie" } }
+{ "create_at": datetime("2016-04-25T00:00:06.000Z"), "id": 724493167718879233, "text": "I miss watching the Dave Chappelle show", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1547173530, "name": "Nando", "screen_name": "CoFernando7", "lang": "en", "location": "Jalisco ✈️ Bay | MHS 16' ", "create_at": date("2013-06-25"), "description": "null", "followers_count": 431, "friends_count": 411, "statues_count": 13727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-04-25T00:00:06.000Z"), "id": 724493167882473472, "text": "Should be ready to kms any day now, thanks for asking ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486174530, "name": "Princess Consuela", "screen_name": "rylee_lewis04", "lang": "en", "location": "Portland, OR", "create_at": date("2012-02-07"), "description": "Oh well.\nfake it till ya make it", "followers_count": 274, "friends_count": 252, "statues_count": 20016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tigard, OR", "id": "d1f122645b638aec", "name": "Tigard", "place_type": "city", "bounding_box": rectangle("-122.840168,45.393108 -122.74354,45.456634") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4173650, "cityName": "Tigard" } }
+{ "create_at": datetime("2016-04-25T00:00:06.000Z"), "id": 724493167886761984, "text": "@marieosmond my name is candace hearrell please view all of my videos? thank you https://t.co/TMyQhxlFNW", "in_reply_to_status": 724483374379872256, "in_reply_to_user": 18538268, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18538268 }}, "user": { "id": 594294538, "name": "candace hearrell", "screen_name": "godisreal06", "lang": "en", "location": "null", "create_at": date("2012-05-29"), "description": "http://www.youtube.com/watch?v=oDMnxFNmcWE&feature=relmfu\r\nhttp://www.youtube.com/watch?v=6OYE2TWlyLQ&feature=relmfu", "followers_count": 137, "friends_count": 539, "statues_count": 10189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austintown, OH", "id": "4e2d49af4742dcf4", "name": "Austintown", "place_type": "city", "bounding_box": rectangle("-80.811312,41.060737 -80.711005,41.134204") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3903184, "cityName": "Austintown" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855529445662720, "text": "Wind 0.0 mph ---. Barometer 29.917 in, Falling. Temperature 36.3 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855529974157313, "text": "@Rachel_Roy I wish I could shoot you fake ass bitch", "in_reply_to_status": 723308001717792768, "in_reply_to_user": 31119337, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31119337 }}, "user": { "id": 2192128270, "name": "Jazmine Morales", "screen_name": "JazmineMorale17", "lang": "en", "location": "null", "create_at": date("2013-11-22"), "description": "libra, ♎️latina, IG: xxjazzykinsxx", "followers_count": 200, "friends_count": 181, "statues_count": 1252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855530527821824, "text": "Sex & super smash bros", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240889451, "name": "Jay Smallwood", "screen_name": "Jsmallsiii", "lang": "en", "location": "RPAC", "create_at": date("2011-01-20"), "description": "The Ohio State University. Trusted by young metro", "followers_count": 664, "friends_count": 898, "statues_count": 18227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855530653634560, "text": "No lol https://t.co/OFtloVBPpK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46745375, "name": "Born Again Fuckboy", "screen_name": "YoungaviatorJ", "lang": "en", "location": "Manteca, CA", "create_at": date("2009-06-12"), "description": "21. Gamer. Heat. Kings. Niners. Trojans. Rest in Paradise Fabrise Munoz. ReelSquad Cofounder. @TheReelSquad", "followers_count": 1339, "friends_count": 782, "statues_count": 252917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855531287007233, "text": "So get yo ass in the car and come thru ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78355084, "name": "Meryl V.", "screen_name": "merylvelezzz", "lang": "en", "location": "MNL✖️CA", "create_at": date("2009-09-29"), "description": "Don't go with the flow ✌️ | IG/Snap: merylvelez", "followers_count": 279, "friends_count": 172, "statues_count": 41203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Foster City, CA", "id": "59bb4e6ce17a8b66", "name": "Foster City", "place_type": "city", "bounding_box": rectangle("-122.288439,37.533497 -122.245136,37.575839") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 625338, "cityName": "Foster City" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855531861610496, "text": "Any range from Michael B Jordan to Odell Beckham Jr. https://t.co/KOXIhPpjkY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4701015865, "name": "Dont hurt yourself", "screen_name": "TrashyYonce", "lang": "en", "location": "Michigan, USA", "create_at": date("2016-01-02"), "description": "Boy you know I give you life", "followers_count": 458, "friends_count": 896, "statues_count": 3445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln Park, MI", "id": "1769b19bf68bc543", "name": "Lincoln Park", "place_type": "city", "bounding_box": rectangle("-83.202957,42.220851 -83.155381,42.271565") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2647800, "cityName": "Lincoln Park" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855533111496705, "text": "@NiaSVaughn @xojeanaaa Ebonics is a lack of culture. Choosing to mimic people who can't speak proper English isn't culture.", "in_reply_to_status": 620700451076501505, "in_reply_to_user": 2579048617, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2579048617, 2853356900 }}, "user": { "id": 2571895692, "name": "Gary Hamilton", "screen_name": "Hamilton2R", "lang": "en", "location": "null", "create_at": date("2014-06-16"), "description": "38 years old. Love sports, economics, politics, and people watching.", "followers_count": 8, "friends_count": 23, "statues_count": 237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855533199593472, "text": "04/26@03:00 - Temp 57.7F, WC 57.7F. Wind 0.0mph SSW, Gust 1.0mph. Bar 29.750in, Falling. Rain 0.02in. Hum 94%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-26T00:00:00.000Z"), "id": 724855533375774722, "text": "I also ain't tryna spend a whole day at a boo boo ass park , Idc if it's senior memories.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1226241854, "name": "x.", "screen_name": "xdoee", "lang": "en", "location": "hmu for GBs. ", "create_at": date("2013-02-27"), "description": "fuck what yah heard. PSN: Jxvx_", "followers_count": 407, "friends_count": 167, "statues_count": 24161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855533782622208, "text": "All Summer 16.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3016579937, "name": "Shay", "screen_name": "ShayStC", "lang": "en", "location": "Los Angeles ", "create_at": date("2015-02-03"), "description": "*God first* singer/songwriter |business email : shaystcbiz@gmail.com", "followers_count": 83, "friends_count": 86, "statues_count": 824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855533803548673, "text": "@ebbtideapp Tide in Pearl Harbor Entrance, Hawaii 04/26/2016\n Low 1:54am 0.2\nHigh 6:39am 0.6\n Low 11:52am 0.0\nHigh 7:35pm 1.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-157.9667,21.33"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 78, "friends_count": 1, "statues_count": 27636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaii, USA", "id": "9dafd05b1158873b", "name": "Hawaii", "place_type": "admin", "bounding_box": rectangle("-178.443593,18.86546 -154.755792,28.517269") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1514200, "cityName": "Hickam Housing" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855534101401600, "text": "it's my birthday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3007346990, "name": "sarah◡̈", "screen_name": "drunkinsarah", "lang": "en", "location": "null", "create_at": date("2015-02-01"), "description": "momma this surely is a dream", "followers_count": 820, "friends_count": 812, "statues_count": 11136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855534151688193, "text": "Temp 47.4° Hi/Lo 51.8/46.9 Rng 4.9° WC 47.4° Hmd 65% Rain 0.00\" Storm 0.00\" BAR 29.890 Falling DP 36.2° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855534839586816, "text": "@BrianaScribbles @alchymies Well it makes me feel better.", "in_reply_to_status": 724855455865028609, "in_reply_to_user": 371109953, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371109953, 132688519 }}, "user": { "id": 86997068, "name": "Bekalou @ KrakenCon", "screen_name": "Beka_Lou", "lang": "en", "location": "CA Bay Area", "create_at": date("2009-11-02"), "description": "NorCal Cosplay Gathering Coordinator", "followers_count": 227, "friends_count": 180, "statues_count": 44003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain View, CA", "id": "b19a2cc5134b7e0a", "name": "Mountain View", "place_type": "city", "bounding_box": rectangle("-122.117916,37.356771 -122.044969,37.436935") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855535045070848, "text": "Every time I open my photo gallery I see my boobs and I'm not sure I like that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 524722108, "name": "MOMMY", "screen_name": "sadieyourladyy", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "ripped at every edge, but still a masterpiece.\nsc» sadierf", "followers_count": 783, "friends_count": 233, "statues_count": 11827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, UT", "id": "01042b286713fe40", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-113.536534,37.080024 -113.444687,37.167408") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49053, "countyName": "Washington", "cityID": 4981960, "cityName": "Washington" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855535137333248, "text": "@xxLousa I did ������ I started feeling bad", "in_reply_to_status": 724855302240264192, "in_reply_to_user": 266360190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266360190 }}, "user": { "id": 3009165881, "name": "nincumpoop", "screen_name": "nikolleye", "lang": "en", "location": "North Carolina, USA", "create_at": date("2015-01-31"), "description": "i am gemini. i am versatility.", "followers_count": 2589, "friends_count": 2178, "statues_count": 3191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, NC", "id": "52cd3346625199d1", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-79.556369,36.028627 -79.379255,36.127415") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37001, "countyName": "Alamance", "cityID": 3709060, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855535812632576, "text": "Yonce knew exactly the reaction she was going to get from her fans when making Lemonade", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1036322004, "name": "Mariah Gomez", "screen_name": "MariahhAnnBby", "lang": "en", "location": "null", "create_at": date("2012-12-25"), "description": "You're a horcrux, Harry!", "followers_count": 358, "friends_count": 287, "statues_count": 10927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, CA", "id": "b820f5ac98af3903", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-121.315922,38.084259 -121.241288,38.155089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 642202, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855535900712960, "text": "It feels great to know that people at church look out for me ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2464033106, "name": "Jerissa Ann ♕", "screen_name": "jerissannx", "lang": "en", "location": "waipahu ", "create_at": date("2014-04-25"), "description": "Joven Genesis Sambrana", "followers_count": 418, "friends_count": 291, "statues_count": 12218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855536362131457, "text": "������ https://t.co/GyNyWPOCkl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1567710397, "name": "Buckwheat.", "screen_name": "ohtaayyy", "lang": "en", "location": "Louisiana, USA", "create_at": date("2013-07-04"), "description": "You took me serious and that's where you fucked up || #UL19", "followers_count": 465, "friends_count": 343, "statues_count": 10502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855536492134401, "text": "@princessponce_ @brenaenaefoo :(((", "in_reply_to_status": 724855083960246272, "in_reply_to_user": 2285907966, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2285907966, 436320567 }}, "user": { "id": 2355511950, "name": "JULE", "screen_name": "yuliithebratt_", "lang": "en", "location": "LBCC", "create_at": date("2014-02-21"), "description": "•18•#20⚾️•IG: yuliithebratt_", "followers_count": 874, "friends_count": 961, "statues_count": 15834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Fe Springs, CA", "id": "00b028b400039d97", "name": "Santa Fe Springs", "place_type": "city", "bounding_box": rectangle("-118.098575,33.882744 -118.028833,33.975123") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669154, "cityName": "Santa Fe Springs" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855536915722244, "text": "Temp: 64.8°F Wind:0.0mph Pressure: 29.902hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855537452613632, "text": "NFL Draft is almost here ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587813494, "name": "fernie ✍", "screen_name": "_kingfernie", "lang": "en", "location": "6", "create_at": date("2012-05-22"), "description": "17 // Mountain View HS // instagram: _kingfernie", "followers_count": 896, "friends_count": 421, "statues_count": 25859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-04-26T00:00:01.000Z"), "id": 724855537527988224, "text": "super soothing �� https://t.co/iYolk2wiQt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4746545376, "name": "✨Metzli Adrianna ✨", "screen_name": "AdriannaMetzli", "lang": "en", "location": "Somewhere in California", "create_at": date("2016-01-11"), "description": "You're on my profile so you might as well creep around.", "followers_count": 217, "friends_count": 194, "statues_count": 849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgecrest, CA", "id": "17c023bb615a7e48", "name": "Ridgecrest", "place_type": "city", "bounding_box": rectangle("-117.72304,35.573286 -117.624503,35.666117") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 660704, "cityName": "Ridgecrest" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855537888825344, "text": "Some people read 2000 year old writings written into books over & over again. I read my spaghetti-o's (more current) https://t.co/vSqjkRS1WI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 86387357, "name": "Macky", "screen_name": "sacmac11", "lang": "en", "location": "Fun, Humor, & JOY in Living", "create_at": date("2009-10-30"), "description": "Happy go lucky truth Sayer. *ATHEIST* Don't believe in fairytales. We're conceived, we're born, we live, we die (in between - PARTY & ENJOY LIFE) ...\nYES I AM!", "followers_count": 281, "friends_count": 334, "statues_count": 6140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antelope, CA", "id": "013214de57b4da2e", "name": "Antelope", "place_type": "city", "bounding_box": rectangle("-121.403842,38.702271 -121.30897,38.728927") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602210, "cityName": "Antelope" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855538043985921, "text": "Wind 3.0 mph S. Barometer 1008.94 mb, Steady. Temperature 66.7 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855538090147840, "text": "Just wanna say that when Trump referenced 9/11 as 7/11 he was at the first Niagara center in Buffalo, nothing ever goes right there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 435483436, "name": "Tyler Vaow", "screen_name": "TyVaow96", "lang": "en", "location": "Newark, NY", "create_at": date("2011-12-12"), "description": "20. Snap me! @TyVaow96 #BillsMafia", "followers_count": 420, "friends_count": 388, "statues_count": 27125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NY", "id": "10cda831e1140cfd", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-77.12731,43.012621 -77.035974,43.068197") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36117, "countyName": "Wayne", "cityID": 3649891, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855538174013440, "text": "Wind 0.0 mph SW. Barometer 29.797 in, Falling slowly. Temperature 64.3 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855538421514240, "text": "Y'all don't know how bad I'm craving aguachiles but they don't have any food places here in SB ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160829553, "name": "marla", "screen_name": "marlaa_m", "lang": "en", "location": "los angeles", "create_at": date("2010-06-28"), "description": "ucsb | ig: @marla_ym | what you see is what you get|", "followers_count": 611, "friends_count": 568, "statues_count": 60527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855538543153152, "text": "70.4F (Feels: 70.4F) - Humidity: 97% - Wind: 3.1mph E - Gust: 5.4mph - Pressure: 1005.4mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 236822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855539067420673, "text": "Wind 0.0 mph SE. Barometer 29.730 in, Steady. Temperature 72.3 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855539310706690, "text": "Snapchat: Tgotti02", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 124384609, "name": "BigGucciGotti", "screen_name": "T_Gotti_Swajjur", "lang": "en", "location": "Dallas,Tx ✈️ Tyler,Tx", "create_at": date("2010-03-18"), "description": "Living Everyday Like Its My Last! Patience Is The Key ... Texas College Student-Athlete I Make This Shit Look Too Easy x #PlugBrothaz Shit‼️ {Playa Potnaz}", "followers_count": 2044, "friends_count": 2513, "statues_count": 28710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855539688194048, "text": "@naivelocin lol that was you that brought it up funny ass", "in_reply_to_status": 724854690081607680, "in_reply_to_user": 4158838966, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4158838966 }}, "user": { "id": 2166075008, "name": "FRANNY", "screen_name": "xFrancisRoque", "lang": "en", "location": "NINA, CA", "create_at": date("2013-10-30"), "description": "null", "followers_count": 798, "friends_count": 263, "statues_count": 13130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855540279611392, "text": "Wind 1.0 mph ENE. Barometer 29.736 in, Falling Rapidly. Temperature 46.0 °F. Rain today 0.01 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855540501860352, "text": "@daisyee72 maybe", "in_reply_to_status": 724853935794692096, "in_reply_to_user": 3016501504, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3016501504 }}, "user": { "id": 303571746, "name": "Tony #17", "screen_name": "Tonaye_17", "lang": "en", "location": "@Tonaye_17 follows you", "create_at": date("2011-05-22"), "description": "Los Angeles | Take Advantage Of An Opportunity Of A Lifetime In A Lifetime Of The Opportunity", "followers_count": 499, "friends_count": 439, "statues_count": 35625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855540904505344, "text": "Thanks to marvinkwhite's vision (& challenge), the book section of the #Berkeley #Goodwill has… https://t.co/zp3v87Hoqh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.26926,37.87168"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Berkeley", "Goodwill" }}, "user": { "id": 85309536, "name": "tdub", "screen_name": "toddwhitley", "lang": "en", "location": "Berkeley, CA", "create_at": date("2009-10-26"), "description": "#BlackLivesMatter | @PSRberkeley #MAST student | ♥ @miggs70 | #GayDad/4 sons + 2 grands | #ENFJ | blogger | #BayArea via #Dallas", "followers_count": 2694, "friends_count": 2930, "statues_count": 117580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855541563056130, "text": "Ripley SW Limestone Co. Temp: 62.1°F Wind:0.0mph Pressure: 992.1mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-26T00:00:02.000Z"), "id": 724855541760184321, "text": "\"Headstone: Monday\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21926284, "name": "Ryan David Sandoval", "screen_name": "ryandsandoval", "lang": "en", "location": "The Woodwork", "create_at": date("2009-02-25"), "description": "child of the hydra; writer @tvdotcom", "followers_count": 355, "friends_count": 122, "statues_count": 10429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542112501760, "text": "@DaaaymJina bet", "in_reply_to_status": 724855233260695553, "in_reply_to_user": 230846064, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 230846064 }}, "user": { "id": 356613398, "name": "bae laden", "screen_name": "Juhhriz", "lang": "en", "location": "null", "create_at": date("2011-08-16"), "description": "null", "followers_count": 1904, "friends_count": 932, "statues_count": 119411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542376763393, "text": "Wind 0.3 mph WSW. Barometer 29.838 in, Falling. Temperature 65.7 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 62332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542397702146, "text": "Wind 1.0 mph N. Barometer 29.721 in, Rising slowly. Temperature 54.4 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542536146944, "text": "Ana is a real one for her little talk with me today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2956285180, "name": "lizzette llanos❤️", "screen_name": "lillizz_22", "lang": "en", "location": "null", "create_at": date("2015-01-02"), "description": "20| your desire for success should be greater than your fear of failure| Chaffey college ❤️", "followers_count": 180, "friends_count": 140, "statues_count": 5527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542720634880, "text": "Temp 50.4°F Wind Chill 50.4°F RH 88% Wind 0.0 --- Gust 0.0 --- SLP 29.884 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 110, "friends_count": 63, "statues_count": 34980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542766788608, "text": "I love when songs can bring me back great memories :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1667033245, "name": "ev", "screen_name": "foreverevieaa", "lang": "en", "location": "eg", "create_at": date("2013-08-12"), "description": "sc: Foreverevieaa", "followers_count": 680, "friends_count": 298, "statues_count": 7396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542766796803, "text": "keep it a buck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2860015631, "name": "A'fari", "screen_name": "ronnieshust", "lang": "en", "location": " 98023, WA", "create_at": date("2014-11-03"), "description": "20. Clubhouse Collective.", "followers_count": 271, "friends_count": 153, "statues_count": 5411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855542905188356, "text": "@IVIIXCIII_ that's money!", "in_reply_to_status": 724855357407944704, "in_reply_to_user": 310521334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 310521334 }}, "user": { "id": 120888351, "name": "E.J. Epps", "screen_name": "EricJEpps", "lang": "en", "location": "Inglewood", "create_at": date("2010-03-07"), "description": "•Equinox Fitness Club- South Bay. •Future US NAVY SAILOR", "followers_count": 819, "friends_count": 391, "statues_count": 53252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855543106514944, "text": "Disney land, dodger game and finally car looking ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2357784128, "name": "Mäxx", "screen_name": "maaxx_02", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": "Psalm 46:5 All in good faith", "followers_count": 671, "friends_count": 557, "statues_count": 9007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Maria, CA", "id": "dee4ad8775ff8102", "name": "Santa Maria", "place_type": "city", "bounding_box": rectangle("-120.482386,34.875868 -120.356782,34.988866") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669196, "cityName": "Santa Maria" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855543333019648, "text": "Wind 3.8 mph NE. Barometer 29.90 in, Falling slowly. Temperature 43.5 °F. Rain today 0.00 in. Humidity 58%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 119, "statues_count": 159339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855543534354432, "text": "Wind 11.2 mph SSE. Barometer 29.694 in, Rising slowly. Temperature 73.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 7997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855543601455104, "text": "@lekz55 se soia! You paved the way, showed me it was possible. Possible to do another set of reps in the gym, to go & do. Love you fam! ��������", "in_reply_to_status": 724855075580088321, "in_reply_to_user": 79963289, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 79963289 }}, "user": { "id": 55655129, "name": "Nicole Su'apaia", "screen_name": "ntapasa", "lang": "en", "location": "HNL➔ ANC➔ LAS➔ SLC sc:ntapasa", "create_at": date("2009-07-10"), "description": "Just a stay at home daughter who's prone to wander. Faith. Family. Gratitude ♡. Social Media Marketing.", "followers_count": 795, "friends_count": 479, "statues_count": 48584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855544218062848, "text": "@feedingjoy Ask not, \"What can I get?\" but \"What can I give?\" True #success", "in_reply_to_status": 724833009912913922, "in_reply_to_user": 2759000172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "success" }}, "user_mentions": {{ 2759000172 }}, "user": { "id": 18655837, "name": "Kelli Maroney", "screen_name": "Kellimaroney", "lang": "en", "location": "Los Angeles, CA.", "create_at": date("2009-01-05"), "description": "http://t.co/fMmV7CSkbU, http://t.co/GRoCxpSTpr http://t.co/qabM27o2EF, http://t.co/i1lVWWvKX2 \nActor, V.O., Animal Lover, Yogini, Armchair Philosopher", "followers_count": 2456, "friends_count": 1341, "statues_count": 14296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855544960409601, "text": "Happy birthday to my best fucking friend! 16 babygirl fuck it up�������� @devinnealubong", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1661448685 }}, "user": { "id": 1512387210, "name": "jules", "screen_name": "julianna_lua", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "RHS", "followers_count": 908, "friends_count": 461, "statues_count": 4026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855545417596928, "text": "'clippers' is now trending in #SanJose https://t.co/A799vGmEdj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SanJose" }}, "user": { "id": 132350009, "name": "Trendsmap San Jose", "screen_name": "TrendsSanJose", "lang": "en", "location": "San Jose, USA", "create_at": date("2010-04-12"), "description": "Real-time San Jose Twitter trends", "followers_count": 235, "friends_count": 75, "statues_count": 2890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-26T00:00:03.000Z"), "id": 724855545979613184, "text": "@jslayys come home", "in_reply_to_status": 724855246321799170, "in_reply_to_user": 2890249728, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2890249728 }}, "user": { "id": 330705839, "name": "Sha'la", "screen_name": "__queenmookmook", "lang": "en", "location": "COME FIND ME ", "create_at": date("2011-07-06"), "description": "IG:itsshakundra❗️SNAPCHAT:younganddope2x❗️FREE NICHOLAS❗️R.I.P FRUIT❤️R.I.P To my KASSY POOH❣#JaxSU19", "followers_count": 1102, "friends_count": 1304, "statues_count": 14819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855546482974720, "text": "Mcintosh Troy Atc, Lancaster, CA, 93534 - https://t.co/7JhRAYsMoH\n\n#mcintoshtroyatc #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.147213,34.685688"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mcintoshtroyatc", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 504, "friends_count": 482, "statues_count": 512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855546738778112, "text": "Howell, NJ | Wind 0.0 mph S. Baro 29.745 in, Falling Rapidly. Temp 58.9F. Rain today 0.00 in. Humidity 87% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 619, "friends_count": 825, "statues_count": 41343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855547330224128, "text": "Forgot to shoutout robs sister naoki for being a savage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2175352914, "name": "Jasmin", "screen_name": "jasminGsilva", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "null", "followers_count": 447, "friends_count": 367, "statues_count": 15321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855547481214976, "text": "@lyriumidol watch game of thrones", "in_reply_to_status": 724855443537977344, "in_reply_to_user": 1475544368, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1475544368 }}, "user": { "id": 2776093724, "name": "konrad", "screen_name": "chriispratt", "lang": "en", "location": "(cis, he/him) new york", "create_at": date("2014-08-28"), "description": "18 // valar morghulis // PSN: Redania", "followers_count": 246, "friends_count": 511, "statues_count": 13194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855547829325825, "text": "Si hace frio me protejo y aunqe esta caliente salgo \nsi algun dia voy a morirme para qe me cuido… https://t.co/arNTfO8GQ4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.019,35.3733"), "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 125099758, "name": "JR", "screen_name": "Michuypromotion", "lang": "es", "location": "soy vago ando de arriva abajo", "create_at": date("2010-03-21"), "description": "Nacido en Culiacan Sinaloa", "followers_count": 450, "friends_count": 2088, "statues_count": 15850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855548668219392, "text": "@R0YALJIMIN thank you for this, but you didn't have to say anything. ��", "in_reply_to_status": 724855143913672704, "in_reply_to_user": 1293927426, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1293927426 }}, "user": { "id": 267428716, "name": "Gina Steger", "screen_name": "missginaface", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-03-16"), "description": "TW, Palaye Royale, MNR, 5SOS, FS, and more=Love. 4/6 TW follow. 3/5 MNR follow. 4/4 Palaye Royale follow #TWFanmilyForever", "followers_count": 1418, "friends_count": 1391, "statues_count": 28447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855548684984320, "text": "I'm so done �� https://t.co/9cxCkqhUEm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1537951243, "name": "nicholas", "screen_name": "nick_russellll", "lang": "en", "location": "Santa Clara, CA", "create_at": date("2013-06-21"), "description": "schs ' 18", "followers_count": 425, "friends_count": 389, "statues_count": 9882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855549171486720, "text": "Wind 3.0 mph WSW. Barometer 29.721 in, Falling. Temperature 66.8 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 393, "friends_count": 293, "statues_count": 10072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855549205069827, "text": "I don't even think I like you when I'm sober", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1034905117, "name": "jules", "screen_name": "daavidson_julia", "lang": "en", "location": "null", "create_at": date("2012-12-25"), "description": "ball is pike", "followers_count": 625, "friends_count": 729, "statues_count": 45432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855549544783872, "text": "@ebuttas you needa do a this or that tweets", "in_reply_to_status": -1, "in_reply_to_user": 178445160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178445160 }}, "user": { "id": 2439153234, "name": "ally_woo", "screen_name": "_ayoubani", "lang": "en", "location": "null", "create_at": date("2014-04-11"), "description": "if you ain't with me then you against me Ig @ ally_wooo sc @ ally_wooo ⬅️", "followers_count": 215, "friends_count": 242, "statues_count": 1097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855549850980356, "text": "@zachias_ https://t.co/HwmqT2R4av", "in_reply_to_status": 724797565267529728, "in_reply_to_user": 2986839501, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2986839501 }}, "user": { "id": 2938833794, "name": "silky regret", "screen_name": "gayswine", "lang": "en", "location": "Colorado, USA", "create_at": date("2014-12-21"), "description": "feed me", "followers_count": 414, "friends_count": 391, "statues_count": 4980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855550006157317, "text": "Taking a shower with no hot water at 2 am ain't no mf joke man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4469338094, "name": "jimmy dean", "screen_name": "BrandoNSmtHjlr", "lang": "en", "location": "breakfast ", "create_at": date("2015-12-05"), "description": "send booty pics 2253135074", "followers_count": 142, "friends_count": 217, "statues_count": 3426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lutcher, LA", "id": "b0fb1c9443ff0f48", "name": "Lutcher", "place_type": "city", "bounding_box": rectangle("-90.716815,30.033381 -90.689036,30.075667") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22093, "countyName": "St. James", "cityID": 2246720, "cityName": "Lutcher" } }
+{ "create_at": datetime("2016-04-26T00:00:04.000Z"), "id": 724855550241046528, "text": "@wanderlust132 ����������bitch I can't lmaooooo I love you! I'm coming to STUNT ON THESE HOS with you ��", "in_reply_to_status": 724855035386040320, "in_reply_to_user": 3476907072, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3476907072 }}, "user": { "id": 349537541, "name": "$$$ALVY", "screen_name": "_The3rd", "lang": "en", "location": "Vegas ", "create_at": date("2011-08-06"), "description": "bitch i came to stunt", "followers_count": 1313, "friends_count": 747, "statues_count": 111593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855550526238723, "text": "Shout out to the mvp of the night @J3lly_B3aN2012", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101318535 }}, "user": { "id": 46761040, "name": "beth onofre", "screen_name": "LizGomezOnofre", "lang": "en", "location": "oc", "create_at": date("2009-06-12"), "description": "I have G O A L S", "followers_count": 325, "friends_count": 285, "statues_count": 4881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855550673035264, "text": "I guess this bump on my head is never gonna go away��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55972757, "name": "Randers", "screen_name": "JasmineMcM", "lang": "en", "location": "Stephenville, TX", "create_at": date("2009-07-11"), "description": "|#BombSquad|Skeleton Clique{|-/}|Snapchat:jweezy_23|", "followers_count": 246, "friends_count": 234, "statues_count": 8508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551000236035, "text": "@skylarrebeccaj me too girl, it's been 5 days with out her and im going crazy.. I couldn't imagine a lifetime. She has to wake up.", "in_reply_to_status": 724855370326351872, "in_reply_to_user": 545692612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545692612 }}, "user": { "id": 2579915107, "name": "kassandra", "screen_name": "kassiebee123", "lang": "en", "location": "The Boonies ", "create_at": date("2014-06-20"), "description": "on my come up. 17 y/o. NPC bikini.", "followers_count": 1142, "friends_count": 451, "statues_count": 21850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551289602049, "text": "Wind 0.0 mph ---. Barometer 29.95 in, Steady. Temperature 67.5 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551365140480, "text": "Wind 0.0 mph ---. Barometer 29.996 in, Steady. Temperature 43.7 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 204, "friends_count": 59, "statues_count": 271391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551465783296, "text": "I started m to catch the feels but I shook it off #toocloseforconfort", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "toocloseforconfort" }}, "user": { "id": 446585182, "name": "leo yang", "screen_name": "yang_leonard", "lang": "en", "location": "null", "create_at": date("2011-12-25"), "description": "this one time I ran a 5k", "followers_count": 136, "friends_count": 112, "statues_count": 2060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551570661376, "text": "Redix :oh about to be the big 19? I remember when I was that age\nMe: that was last year .... ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1656387018, "name": "kathleen", "screen_name": "TheGreatGatita", "lang": "en", "location": "my feelings probably", "create_at": date("2013-08-08"), "description": "The first letter of my name ends relationships", "followers_count": 161, "friends_count": 258, "statues_count": 4383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, KY", "id": "dcd73905ee565216", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-84.706797,38.917578 -84.606098,39.060938") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21015, "countyName": "Boone", "cityID": 2127982, "cityName": "Florence" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551616753664, "text": "All of y'all some hoe ass niggas on me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 832315812, "name": "$ensei", "screen_name": "EightyNxne", "lang": "en", "location": "null", "create_at": date("2012-09-18"), "description": "I'm just tryna prosper", "followers_count": 1163, "friends_count": 904, "statues_count": 23500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855551704879104, "text": "Pple hittin me up like \"lets go to Faded in the Park\" or \"lehgo to Pemberton\" hoe I got a case pending I cant go to no Canadian festival tf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 636795414, "name": "Yuriy", "screen_name": "SeattleThot", "lang": "en", "location": "Lynnwood, WA", "create_at": date("2012-07-15"), "description": "Call me when yo man ain't home #SpursNation #Armenian", "followers_count": 741, "friends_count": 415, "statues_count": 30045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountlake Terrace, WA", "id": "d1280141e5f979cf", "name": "Mountlake Terrace", "place_type": "city", "bounding_box": rectangle("-122.333394,47.777172 -122.28146,47.807896") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5347490, "cityName": "Mountlake Terrace" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855552417906688, "text": "@Runitback_Rob bruh", "in_reply_to_status": -1, "in_reply_to_user": 307229437, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 307229437 }}, "user": { "id": 2379302060, "name": "BC", "screen_name": "bclinks17", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-03-08"), "description": "CCSF DB #citysituation", "followers_count": 792, "friends_count": 594, "statues_count": 11385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553034457089, "text": "Still super salt drake isn't coming to Denver for his summer 16 tour. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 199542468, "name": "J O S S I", "screen_name": "jossichavez", "lang": "en", "location": "null", "create_at": date("2010-10-06"), "description": "Denver, Co✨", "followers_count": 559, "friends_count": 619, "statues_count": 23645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553042833408, "text": "@MyLifeAsHowie @Donmccarthyjr @DanielSantos805 Donny is a robot. He favorites everything and anything", "in_reply_to_status": 724854941215510529, "in_reply_to_user": 517085234, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517085234, 298867595, 1022821206 }}, "user": { "id": 88760833, "name": "LAndy", "screen_name": "LAndyman_", "lang": "en", "location": "Bula, fighting by way of LA", "create_at": date("2009-11-09"), "description": "NE OHIO guy in L.A. ~ I still dislike John Elway ~ #Tribe Baseball enthusiast ~ Hollywood #Browns Backer ~ I can't ever sit still ~ Serial RT'r ~ #Cavs #GoBucks", "followers_count": 2962, "friends_count": 2367, "statues_count": 87666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553294536705, "text": "@esmefelipe @is_anyone_up", "in_reply_to_status": 724844087162097666, "in_reply_to_user": 1939920817, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1939920817, 864186378 }}, "user": { "id": 35752797, "name": "Danny Fenton", "screen_name": "KyleHvlett", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-04-27"), "description": "Music Social Marketer at @TotalAssault. Community Director for @flippenmusic. #ItsAllHappening", "followers_count": 9842, "friends_count": 833, "statues_count": 43403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553298673664, "text": "Cp3 played his last game as a clipper ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35478824, "name": "7", "screen_name": "AppleciderPapi", "lang": "en", "location": "LA", "create_at": date("2009-04-26"), "description": "18. Muslim. Beautiful. Genius. ⭐️", "followers_count": 6123, "friends_count": 954, "statues_count": 169913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553667780608, "text": "@IAMKendraYHill @JonettaMDavis Honesty Hour?", "in_reply_to_status": 724855269847654401, "in_reply_to_user": 27031619, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27031619, 28056649 }}, "user": { "id": 3291905311, "name": "Yazmine L. Green ❤️", "screen_name": "chosen4gr8nesss", "lang": "en", "location": "null", "create_at": date("2015-07-24"), "description": "I just want to please Jesus with my life #EverydayGoal #LifetimeGoal ✊ | 21 | Worshipper | Intercessor | Heir of Christ | #Freedomite | #UAB18 | Isaiah 40:31 ✨", "followers_count": 509, "friends_count": 427, "statues_count": 11390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553843970049, "text": "Bennie And The Jets by Elton John is #nowplaying in Thumb Coast Brewing Co., Port Huron.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.422367,42.975489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox", "screen_name": "VirtualJukebox", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1704, "friends_count": 0, "statues_count": 1615119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Huron, MI", "id": "010750c237347ebb", "name": "Port Huron", "place_type": "city", "bounding_box": rectangle("-82.514554,42.935388 -82.418396,43.040084") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2665820, "cityName": "Port Huron" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855553965580288, "text": "I guess I'm just a shitty individual.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2392369679, "name": "Jessika", "screen_name": "jessyjams10", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "Basketball | Becca❤️", "followers_count": 493, "friends_count": 413, "statues_count": 8230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-04-26T00:00:05.000Z"), "id": 724855554112409600, "text": "Ok sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23525855, "name": "Natalie", "screen_name": "NatalieNClark", "lang": "en", "location": "Lake Oswego, Oregon", "create_at": date("2009-03-09"), "description": "College football, Oregon Ducks, Justin Bieber, and maybe like 3 people. Currently employed at @BurgervilleUSA West Linn location :) #ThatGirlAtBurgerville", "followers_count": 166, "friends_count": 959, "statues_count": 9520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Oswego, OR", "id": "386b4dc0b52f8b48", "name": "Lake Oswego", "place_type": "city", "bounding_box": rectangle("-122.750195,45.382216 -122.642587,45.44028") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4140550, "cityName": "Lake Oswego" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855554485673984, "text": "I just got accepted into west valley", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 536790383, "name": "yung anj", "screen_name": "Amysxwhinehouse", "lang": "en", "location": "null", "create_at": date("2012-03-25"), "description": "gay af ☯", "followers_count": 853, "friends_count": 617, "statues_count": 33228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alum Rock, CA", "id": "277b4360183b5d75", "name": "Alum Rock", "place_type": "city", "bounding_box": rectangle("-121.842175,37.35398 -121.812158,37.38102") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 601458, "cityName": "Alum Rock" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855554707959808, "text": "@JacobKnocks good nite", "in_reply_to_status": 724855088779542529, "in_reply_to_user": 150858848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150858848 }}, "user": { "id": 150858848, "name": "oomf ⛳️", "screen_name": "JacobKnocks", "lang": "en", "location": "AUSTIN TEXAS", "create_at": date("2010-06-01"), "description": "wyd | SEU18", "followers_count": 297, "friends_count": 57, "statues_count": 1582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855554733137920, "text": "@iFeeLikeKobe screamin", "in_reply_to_status": 724855040419176448, "in_reply_to_user": 2904751410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2904751410 }}, "user": { "id": 2316472118, "name": "Fuentez➕⚜", "screen_name": "Fuentez_06", "lang": "en", "location": "Tweeting ", "create_at": date("2014-01-28"), "description": "I do me and chill #WhoDat", "followers_count": 1155, "friends_count": 769, "statues_count": 61894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855555454582784, "text": "Aw u kappin https://t.co/RWeogmKbyE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146831774, "name": "Trap Brady", "screen_name": "Intertraptional", "lang": "en", "location": "New Orleans _ Dallas ", "create_at": date("2010-05-22"), "description": "Plugged In With A Chico. . #AZ #FreeStreetz #TrapNation #NewMoney #TXST17", "followers_count": 1306, "friends_count": 829, "statues_count": 71366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855555488108544, "text": "@chavezjoshuaiii *an, and advanced p.e. don't count��", "in_reply_to_status": 724853080500613121, "in_reply_to_user": 4648677390, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4648677390 }}, "user": { "id": 3213048267, "name": "Hendrixx", "screen_name": "Davonted11x", "lang": "en", "location": "null", "create_at": date("2015-04-27"), "description": "10•15 Army strong", "followers_count": 295, "friends_count": 298, "statues_count": 2231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855556024995840, "text": "@nytimes Too much violence in sports. Too much sexism, drug culture. Save American families. Stop pro sports. Help your wife at home. KRC", "in_reply_to_status": 724801940195041280, "in_reply_to_user": 807095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 807095 }}, "user": { "id": 121637095, "name": "Kathleen Cunningham", "screen_name": "Rosewater2016", "lang": "en", "location": "Denver, CO", "create_at": date("2010-03-09"), "description": "null", "followers_count": 10, "friends_count": 39, "statues_count": 424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855556033372161, "text": "@LongwayNigga lol nope I need it more then them", "in_reply_to_status": 724855455860793345, "in_reply_to_user": 68111983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 68111983 }}, "user": { "id": 524820136, "name": "Cee2x", "screen_name": "Therealqscee", "lang": "en", "location": "null", "create_at": date("2012-03-14"), "description": "null", "followers_count": 1832, "friends_count": 804, "statues_count": 42892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855556821889024, "text": "Just posted a photo @ Ferally Normal Clayworks https://t.co/F04vfXvEYy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.08609,45.40783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53927345, "name": "Amanda Igaki-Aribon", "screen_name": "HulaSan", "lang": "en", "location": "Gaston, OR", "create_at": date("2009-07-05"), "description": "null", "followers_count": 90, "friends_count": 226, "statues_count": 766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon, USA", "id": "df7fd3a3b9eff7ee", "name": "Oregon", "place_type": "admin", "bounding_box": rectangle("-124.703541,41.991795 -116.463262,46.299078") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41071, "countyName": "Yamhill" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855557421682688, "text": "Top Thrill Dragster https://t.co/CU77MaMlmI #topthrilldragster #parcdattraction https://t.co/cVQjTM4YJv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.686348,41.483929"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "topthrilldragster", "parcdattraction" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 949, "friends_count": 1293, "statues_count": 4868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandusky, OH", "id": "fcc54ac4568181a4", "name": "Sandusky", "place_type": "city", "bounding_box": rectangle("-82.78773,41.409372 -82.617802,41.492373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39043, "countyName": "Erie", "cityID": 3970380, "cityName": "Sandusky" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855557497180161, "text": "Temp: 52.6°F - Dew Point: 49.1° - Wind: NW @ 7.3 mph - Gust: 12.1 - Rain Today: 0.00in. - Pressure: 29.66in, - Trend: Rising", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 12, "statues_count": 16794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855558256386054, "text": "Go on IG they steady trying to turn your $200 to $2000 https://t.co/Oqc2UIX5rq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38813183, "name": "Kerrie ", "screen_name": "KdubSoSolid", "lang": "en", "location": "Sacramento CA ✈️ Houston TX", "create_at": date("2009-05-08"), "description": "All Glory to God IG:Kdubsosolid LakerNation PatsNation TSU - SAC STATE", "followers_count": 1196, "friends_count": 457, "statues_count": 179495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855558428319745, "text": "@DallasCowboys draft this man. https://t.co/MjxbKM3jX6", "in_reply_to_status": -1, "in_reply_to_user": 8824902, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 8824902 }}, "user": { "id": 62013596, "name": "Brandon Cox", "screen_name": "HEEL_BLC", "lang": "en", "location": "Lubbock Tx", "create_at": date("2009-08-01"), "description": "#GoSpursGo #WreckEmTech #TexasRangers #WWEFanatic Favorite Wrestler Sasha Fucking Banks #SashaBanksMark", "followers_count": 947, "friends_count": 1697, "statues_count": 60250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Littlefield, TX", "id": "fd4a2f8903d8a73d", "name": "Littlefield", "place_type": "city", "bounding_box": rectangle("-102.355122,33.89945 -102.311606,33.939433") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48279, "countyName": "Lamb", "cityID": 4843024, "cityName": "Littlefield" } }
+{ "create_at": datetime("2016-04-26T00:00:06.000Z"), "id": 724855558533185537, "text": "I never get tired of ok go music videos tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2287030892, "name": "#1 Tully Monster Fan", "screen_name": "fossilfriendly", "lang": "en", "location": "somewhere in the Pleistocine", "create_at": date("2014-01-11"), "description": "Official #TullyMonster2016 fan account http://www.theatlantic.com/science/archive/2016/03/solving-the-mystery-of-the-tully-monster/473823/ I'm a Girl", "followers_count": 627, "friends_count": 872, "statues_count": 102536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metzger, OR", "id": "28578d0449d066e5", "name": "Metzger", "place_type": "city", "bounding_box": rectangle("-122.774163,45.44123 -122.751697,45.458208") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4147800, "cityName": "Metzger" } }
+{ "create_at": datetime("2016-04-26T00:00:07.000Z"), "id": 724855558638030849, "text": "@drewsmithvideo great video! Just so much to go thru lol not even done logging and I have 15 pages of notes. Eek! Script by Friday prob ��", "in_reply_to_status": 724838423144615936, "in_reply_to_user": 807477721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 807477721 }}, "user": { "id": 15906463, "name": "Kevin Torres", "screen_name": "kevinltorres", "lang": "en", "location": "Denver, CO", "create_at": date("2008-08-19"), "description": "7x Emmy-Award winning Reporter for FOX31 Denver & Channel 2 News. Hiker, Skier, Netflixer, XboxOner. I'm also afraid of whales. Fo realsies.", "followers_count": 8170, "friends_count": 1918, "statues_count": 19330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-04-26T00:00:07.000Z"), "id": 724855559049109504, "text": "@StreamMeWaffles if it sucks I'm spamming you", "in_reply_to_status": 724855138473639937, "in_reply_to_user": 142776849, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 142776849 }}, "user": { "id": 28933718, "name": "Sith Lord Trell", "screen_name": "iTrap4TheHokage", "lang": "en", "location": "Detroit,MI", "create_at": date("2009-04-04"), "description": "3-D character artist\n Xbox live GT: HMzero2 / PSN:Trellmix223", "followers_count": 1919, "friends_count": 1192, "statues_count": 57581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-04-26T00:00:07.000Z"), "id": 724855559103631362, "text": "Wind 0 mph ---. Barometer 1009.3 hPa, Falling. Temperature 75.4 °F. Rain today 0.00 in. Humidity 40%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 154167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217917277626368, "text": "@kouotsu HAHAHA WHOOPS", "in_reply_to_status": 725217867545858048, "in_reply_to_user": 15305193, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 15305193 }}, "user": { "id": 15960033, "name": "clover & sealife", "screen_name": "cloversealife", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-08-23"), "description": "✨ jami carignan 〜 trackmaker 〜 future girl jimmy buffett / my new record, “a sentimental cry” out now!!! http://bit.ly/1L7H7pw", "followers_count": 2222, "friends_count": 344, "statues_count": 109631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belltown, Seattle", "id": "2b6abedf069864d1", "name": "Belltown", "place_type": "neighborhood", "bounding_box": rectangle("-122.362276,47.6086 -122.328744,47.619927") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217917478957056, "text": "@KayleeMingss I really have ni idea who ur tweet was towards I just wanted to get ur back��", "in_reply_to_status": 725217413092921346, "in_reply_to_user": 905183258, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 905183258 }}, "user": { "id": 416878880, "name": "Chad Nishioka", "screen_name": "nishiHIstyle", "lang": "en", "location": "probably at the skatepark", "create_at": date("2011-11-19"), "description": "Chad Nishioka, Senior@Shs, skater and musician/songwriter. snap me @ nishistyle RIP tersee | https://m.youtube.com/watch?v=agYuFtkGUBQ", "followers_count": 411, "friends_count": 578, "statues_count": 2322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherwood, OR", "id": "6870523b6f8d96e6", "name": "Sherwood", "place_type": "city", "bounding_box": rectangle("-122.869771,45.345593 -122.805551,45.376774") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4167100, "cityName": "Sherwood" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217918636576769, "text": "I didn't like one dance until I heard it high and it tickled me .. I've been listening to it everyday since", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750849597, "name": "Alton Williams", "screen_name": "Simply_AhhWill", "lang": "en", "location": "null", "create_at": date("2014-08-24"), "description": "I'm ahead of my time", "followers_count": 287, "friends_count": 281, "statues_count": 6254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217919144054788, "text": "@bwiyurin are you w yr BF", "in_reply_to_status": 725217856019812352, "in_reply_to_user": 1116004040, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1116004040 }}, "user": { "id": 396416798, "name": "Tomas", "screen_name": "tolivefreeordie", "lang": "en", "location": "null", "create_at": date("2011-10-23"), "description": "In the dragonless age, this knight, who long guarded the ruined cathedral, left the land in search of the nameless king", "followers_count": 1003, "friends_count": 550, "statues_count": 105719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, MI", "id": "c309c6a6f85110cd", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-82.973298,42.539309 -82.855854,42.630481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2616480, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217919416832000, "text": "@QaizerSoze @MrsJelly@TazsAngels the bikini not the problem it's your grip of the club and keeping club straight https://t.co/u2611XgAXR", "in_reply_to_status": 725165345145798658, "in_reply_to_user": 145427762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 145427762 }}, "user": { "id": 257467181, "name": "arnold johnson", "screen_name": "arnojohnson", "lang": "en", "location": "New Jersey, USA", "create_at": date("2011-02-25"), "description": "twitter is my official designated start info seeking. and job opportunities", "followers_count": 1064, "friends_count": 3121, "statues_count": 12754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217919471226881, "text": "It's warriors gameday #DubNation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DubNation" }}, "user": { "id": 300027487, "name": "Ryantdalton", "screen_name": "ryantdalton1", "lang": "en", "location": "null", "create_at": date("2011-05-16"), "description": "Im 23 love the A's warriors sharks and raiders #sharksfam #nfwbge warriors nba champions #dubnation", "followers_count": 585, "friends_count": 1984, "statues_count": 87433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217919496388610, "text": "my girlfriend �� https://t.co/gksBnFcl6E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760700304, "name": "andrea.", "screen_name": "niallsnandos13_", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "make tacos, not war", "followers_count": 3129, "friends_count": 3050, "statues_count": 41309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217920322674688, "text": "Wind 0.0 mph ---. Barometer 29.907 in, Rising. Temperature 24.6 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217920419172358, "text": "Wrist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3330736574, "name": "Sawoop", "screen_name": "SaulSawoop", "lang": "en", "location": "ReefaBöÿz", "create_at": date("2015-08-24"), "description": "LOWKEY IM MAD HORNY", "followers_count": 421, "friends_count": 291, "statues_count": 7378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217920624660480, "text": "PULL UP ON 5GUYS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513460778, "name": "kyle.", "screen_name": "krispykay_", "lang": "en", "location": "Detroit to New Orleans", "create_at": date("2014-04-27"), "description": "ugly, cass tech alum, public relations major #DU18", "followers_count": 4297, "friends_count": 827, "statues_count": 42957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-27T00:00:00.000Z"), "id": 725217920817618944, "text": "best feeling is going to sleep in warm clean sheets ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65271826, "name": "goose mama", "screen_name": "paigeeykinns", "lang": "en", "location": "fresNo", "create_at": date("2009-08-12"), "description": "equestrian | my heart belongs to hondo | book worm | Racey ❤️", "followers_count": 213, "friends_count": 168, "statues_count": 30020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217921660678144, "text": "@edgar_deleon10 you can do it! You are a meatball", "in_reply_to_status": 725215122180861952, "in_reply_to_user": 31023415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31023415 }}, "user": { "id": 575015297, "name": "Lily", "screen_name": "LilyPop_21", "lang": "en", "location": "TX/CO", "create_at": date("2012-05-08"), "description": "#SpursNation Still trying to get over game 6", "followers_count": 525, "friends_count": 360, "statues_count": 16811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217922092769280, "text": "04/27@03:00 - Temp 51.1F, WC 51.1F. Wind 2.9mph E, Gust 7.0mph. Bar 29.867in, Rising. Rain 0.00in. Hum 95%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217922122018816, "text": "i'm not even gonna try finding love anymore, if it finds me great if not coo i'll wait.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 612332271, "name": "ang", "screen_name": "angzzz_", "lang": "en", "location": "palmdale, ca", "create_at": date("2012-06-18"), "description": "growing", "followers_count": 1820, "friends_count": 984, "statues_count": 20832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217922256388097, "text": "Temp 34.3° Hi/Lo 37.1/34.3 Rng 2.8° WC 34.3° Hmd 94% Rain 0.00\" Storm 0.33\" BAR 29.897 Falling DP 32.8° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217922466103297, "text": "Dusty Rhodes matches for the rest of the night over here brother.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453644637, "name": "Brian Szybowski", "screen_name": "Bri_Ski28", "lang": "en", "location": "null", "create_at": date("2012-01-02"), "description": "Daytrading the market. Professional sports fan. Wrestling fan.", "followers_count": 79, "friends_count": 212, "statues_count": 2185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217922537410560, "text": "@DrThomasParker1 I am going to give it a go next year....I want to try DinaFem Purple Afghan Kush....Done end of Sept too....", "in_reply_to_status": 725217606446116864, "in_reply_to_user": 373629913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373629913 }}, "user": { "id": 373629913, "name": "Dr. Thomas Parker", "screen_name": "DrThomasParker1", "lang": "en", "location": "Corvallis, Oregon", "create_at": date("2011-09-14"), "description": "Hi,\r\nMy name is Dr. Thomas Parker. I am a scientist living in Corvallis, Oregon. I have a PhD in Crop and Soil Science from Oregon State University.", "followers_count": 4607, "friends_count": 5053, "statues_count": 61446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217923078352896, "text": "God is really looking out for me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65993492, "name": "Alysia", "screen_name": "alysiacastro", "lang": "en", "location": "PHX, Arizona", "create_at": date("2009-08-15"), "description": "Asking God for mercy", "followers_count": 875, "friends_count": 483, "statues_count": 32571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217923611168768, "text": "me as I submit said paper at 3am https://t.co/mXAIreFqap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351809468, "name": "Sarah Elizabeth", "screen_name": "skawinskk", "lang": "en", "location": "ΦΣΣ | NJ | CT", "create_at": date("2011-08-09"), "description": "@erynelly is the patrick to my spongebob", "followers_count": 465, "friends_count": 486, "statues_count": 8896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Haven, CT", "id": "c783883e2edd7454", "name": "West Haven", "place_type": "city", "bounding_box": rectangle("-72.994697,41.233074 -72.919497,41.313267") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 982800, "cityName": "West Haven" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217923933949952, "text": "I still am dude https://t.co/uxZyeX7NpE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 466161567, "name": "Brianna Valencia", "screen_name": "valenciabri", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "❥", "followers_count": 640, "friends_count": 562, "statues_count": 20056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217924198354944, "text": "Wind 0.0 mph SE. Barometer 29.705 in, Rising. Temperature 60.5 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217924332417024, "text": "Took a shower & was calm for a sec but I'm really still not in the mood Tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3346868200, "name": "Nicole'™", "screen_name": "AshleyN09265167", "lang": "en", "location": "Conway, AR", "create_at": date("2015-06-26"), "description": "SC @nicolegurle IG @ashnicolesmith\n#FREEDEX'016 Steven&Ashley™", "followers_count": 867, "friends_count": 1418, "statues_count": 10991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217924386934784, "text": "I literally only watch black movies. I don't have an interest in other kind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2696692639, "name": "Vanity", "screen_name": "CindyrellaOG", "lang": "en", "location": "Plano, TX", "create_at": date("2014-07-31"), "description": "† Ogechi ka nma| Nigerian| UT Dallas co'18| Rest In Peace Newton ❤️ YouTube: Cindyrella OG", "followers_count": 2428, "friends_count": 899, "statues_count": 58529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217924995121153, "text": "They're either using you, helping you, or watching you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 68506224, "name": "Magnus", "screen_name": "FreshmeOMG", "lang": "en", "location": "LA", "create_at": date("2009-08-24"), "description": "null", "followers_count": 239, "friends_count": 532, "statues_count": 3298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217925125181440, "text": "@midjaii this is so bo3������ https://t.co/JRlJGopZaw", "in_reply_to_status": -1, "in_reply_to_user": 874496220, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 874496220 }}, "user": { "id": 2200605212, "name": "Phil", "screen_name": "JCuellar06", "lang": "en", "location": "817", "create_at": date("2013-11-17"), "description": "The Fort", "followers_count": 258, "friends_count": 299, "statues_count": 23510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217925133537281, "text": "Needed to read this. For so long I've been giving all of my energy to an empty vessel. No more. ���� https://t.co/tLccGAgNKf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 627738269, "name": "kailey manz", "screen_name": "kaileyyymanz", "lang": "en", "location": "null", "create_at": date("2012-07-05"), "description": "University of Northern Colorado //// You'll catch flies with honey but you'll catch more hunnies lookin fly.", "followers_count": 1103, "friends_count": 1105, "statues_count": 7004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-04-27T00:00:01.000Z"), "id": 725217925326467072, "text": "@ebbtideapp Tide in Marcus Hook, Pennsylvania 04/27/2016\nHigh 3:48am 6.3\n Low 11:34am 0.5\nHigh 4:34pm 5.2\n Low 11:24pm 0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.41,39.8117"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 80, "friends_count": 1, "statues_count": 27915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pennsylvania, USA", "id": "dd9c503d6c35364b", "name": "Pennsylvania", "place_type": "admin", "bounding_box": rectangle("-80.519851,39.719801 -74.689517,42.516072") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4247344, "cityName": "Marcus Hook" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217925615865857, "text": "The best pina colada in SF. #trickdog #dranks #infatuationsf #sf @ Trick Dog https://t.co/gdLLf7r4ya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.41121966,37.75922161"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "trickdog", "dranks", "infatuationsf", "sf" }}, "user": { "id": 24658819, "name": "Samantha Kharma", "screen_name": "SamKharma", "lang": "en", "location": "San Fran", "create_at": date("2009-03-15"), "description": "restaurant manager, soon to be bar owner, chef of kharma catering", "followers_count": 163, "friends_count": 201, "statues_count": 17309 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217925636956162, "text": "Temp: 65.8°F Wind:0.0mph Pressure: 29.781hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217925741699072, "text": "Wind 3.0 mph NW. Barometer 1008.77 mb, Rising Rapidly. Temperature 58.4 °F. Rain today 0.62 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217925758504960, "text": "@KaylaDTD @salthetaco AHHAH I'm sorry it's not :( try looking at the pattern", "in_reply_to_status": 725217753510998016, "in_reply_to_user": 187712274, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 187712274, 227901767 }}, "user": { "id": 89986656, "name": "XObeyTheBarrelX", "screen_name": "OnlyyOsasee", "lang": "en", "location": "Tolleson, AZ", "create_at": date("2009-11-14"), "description": "Naruto and One Piece fan boy. Add my Snapchat: Osasen #BuffetBoyz", "followers_count": 298, "friends_count": 94, "statues_count": 19646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217925997678592, "text": "Wind 2.0 mph S. Barometer 29.749 in, Falling slowly. Temperature 61.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217926257729536, "text": "Wind 1.1 mph WSW. Barometer 29.723 in, Falling. Temperature 70.6 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217926513463296, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":60.8°F Wind:1.1mph Pressure: 29.82hpa Falling Rain Today 0.24in. Forecast: Precipitation, very unse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 316344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217926668636160, "text": "School isn't over yet, but I'm already making a list of shows I'm going to watch during the summer hehhe...summa come sooner! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75715167, "name": "Jermaine Lark", "screen_name": "heyitsjermaine", "lang": "en", "location": "SAN DIEGO,CA", "create_at": date("2009-09-19"), "description": "Let me enlighten you with my wild and beautiful mind", "followers_count": 197, "friends_count": 188, "statues_count": 6557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217926752555008, "text": "They just wanna brag", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2344337132, "name": "yego of tfti", "screen_name": "DiegoNigguhh", "lang": "en", "location": "9's", "create_at": date("2014-02-14"), "description": "I eat ice cream with my chicken", "followers_count": 1067, "friends_count": 518, "statues_count": 67433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217926828163072, "text": "“Courage doesn’t always roar. Sometimes courage is the quiet voice at the end of the day saying, ‘I will try again tomorrow.’” —unknown", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967244772, "name": "Cupcake12424", "screen_name": "nicolek14584793", "lang": "en", "location": "null", "create_at": date("2012-11-23"), "description": "new video: https://youtu.be/2TePHqriooM", "followers_count": 463, "friends_count": 1229, "statues_count": 206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lee, NJ", "id": "609b31e0a50b4474", "name": "Fort Lee", "place_type": "city", "bounding_box": rectangle("-73.991203,40.827763 -73.953707,40.868904") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3424420, "cityName": "Fort Lee" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217927176306688, "text": "I gotta get up a little earlier than usual", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263912857, "name": ". ShanDaddy✨™", "screen_name": "___Shannonnn", "lang": "en", "location": "662 ✈️ 601 Hattiesburg, MS ", "create_at": date("2011-03-10"), "description": "#USM18 .", "followers_count": 1537, "friends_count": 890, "statues_count": 54869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hattiesburg, MS", "id": "2b34df148a211c3e", "name": "Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.427669,31.241189 -89.248409,31.380085") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28035, "countyName": "Forrest", "cityID": 2831020, "cityName": "Hattiesburg" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217928015040512, "text": "73.6F (Feels: 73.6F) - Humidity: 93% - Wind: 7.6mph S - Gust: 9.8mph - Pressure: 1000.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 236965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217928082153472, "text": "Happy 22nd birthday! Thank you for being the coolest older brother ever������ https://t.co/txTszP5I9d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 752338194, "name": "Oh", "screen_name": "micheleshirley6", "lang": "en", "location": "Rexburg, ID", "create_at": date("2012-08-11"), "description": "Idunno Prolly", "followers_count": 287, "friends_count": 208, "statues_count": 9958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rexburg, ID", "id": "da61090cbdc5350e", "name": "Rexburg", "place_type": "city", "bounding_box": rectangle("-111.818018,43.79717 -111.757595,43.842019") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16065, "countyName": "Madison", "cityID": 1667420, "cityName": "Rexburg" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217928136658944, "text": "Levi isn't cutting it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1792852434, "name": "Vanessa Maynes", "screen_name": "vmmaynes_", "lang": "en", "location": "UTSA ", "create_at": date("2013-09-07"), "description": "null", "followers_count": 615, "friends_count": 514, "statues_count": 5029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-27T00:00:02.000Z"), "id": 725217928803549184, "text": "12 am and I'm already done with today ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119957209, "name": "Emily", "screen_name": "EmilyNicole112", "lang": "en", "location": "null", "create_at": date("2010-03-04"), "description": "5'10 with some attitude. sc-- @emilynicole112", "followers_count": 518, "friends_count": 496, "statues_count": 11089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tumwater, WA", "id": "12148f3ffcceb203", "name": "Tumwater", "place_type": "city", "bounding_box": rectangle("-123.010086,46.913138 -122.860365,47.029499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930158313472, "text": "@Caesears oui", "in_reply_to_status": 725217861141045249, "in_reply_to_user": 1587266395, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1587266395 }}, "user": { "id": 793032193, "name": "alexis", "screen_name": "PokemonAlexx", "lang": "en", "location": "US: 37.766697,-122.208225", "create_at": date("2012-08-30"), "description": "null", "followers_count": 335, "friends_count": 252, "statues_count": 42106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930275778560, "text": "Freddy Frito", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 82009830, "name": "agatha ordinario", "screen_name": "HernameisEe", "lang": "en", "location": "in my feelings most likely ", "create_at": date("2009-10-12"), "description": "pedestal so high you niggas can't reach it. snapchat:anaisamore", "followers_count": 366, "friends_count": 332, "statues_count": 7982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930347081728, "text": "@shroobchomp clawhauser: IT IS A MIRACLE!!!!!", "in_reply_to_status": 725217441039548416, "in_reply_to_user": 2327656434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2647007906 }}, "user": { "id": 2327656434, "name": "✨belle✨", "screen_name": "agentpleakley", "lang": "en", "location": "far far away", "create_at": date("2014-02-04"), "description": "@mermistas's side/more personal/shitposty/fandomy account i live tweet here a lot", "followers_count": 332, "friends_count": 217, "statues_count": 29589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Woodlands, TX", "id": "5280739904d0ef48", "name": "The Woodlands", "place_type": "city", "bounding_box": rectangle("-95.661376,30.10764 -95.418329,30.254863") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4872656, "cityName": "The Woodlands" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930481266688, "text": "@juan_jaime11 you said it not me", "in_reply_to_status": 725217875900788736, "in_reply_to_user": 605836829, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605836829 }}, "user": { "id": 758526847, "name": "ECW world champ", "screen_name": "Juhhayy", "lang": "en", "location": "chicago ", "create_at": date("2012-08-14"), "description": "snapchat-justin.orozco21//// You're not the only one trying to be the only one", "followers_count": 398, "friends_count": 194, "statues_count": 9803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orland, CA", "id": "670fb5b9ac1f023e", "name": "Orland", "place_type": "city", "bounding_box": rectangle("-122.238913,39.721575 -122.150193,39.769545") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6021, "countyName": "Glenn", "cityID": 654274, "cityName": "Orland" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930607095809, "text": "I wiish I could sing this to someone and mean it https://t.co/1kGIBESKnr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1297997263, "name": "Jonasty Vaifala Papi", "screen_name": "_Jooownaaaw", "lang": "en", "location": "null", "create_at": date("2013-03-24"), "description": "Jonah Stonah. My Blood is Hawaiian, but my Stomach is Samoan. [No. I'm not Jonah from Tonga]", "followers_count": 466, "friends_count": 296, "statues_count": 12118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royal Kunia, HI", "id": "000cd0bd39281e46", "name": "Royal Kunia", "place_type": "city", "bounding_box": rectangle("-158.039748,21.396379 -158.023455,21.409007") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1568815, "cityName": "Royal Kunia" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930737119232, "text": "Looking at you through the glass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133821041, "name": "Funky Funk", "screen_name": "FunkyDikidd", "lang": "en", "location": "Enjoy Life & Get Funky Wid It", "create_at": date("2010-04-16"), "description": "22, Songwriter, StoryTeller, Food Eater! LA Bound, Gamer Somewhat XBOX GT: SWE MrClutch", "followers_count": 1060, "friends_count": 856, "statues_count": 52724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217930883911680, "text": "@cristianisdead contemplating ...", "in_reply_to_status": 725217429974962178, "in_reply_to_user": 3112073788, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3112073788 }}, "user": { "id": 2338783778, "name": "Daisy", "screen_name": "dvisyy__", "lang": "en", "location": "null", "create_at": date("2014-02-11"), "description": "slowly but surely getting my shit together (:", "followers_count": 403, "friends_count": 317, "statues_count": 18217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217931227852800, "text": "@ColtonBlakley incase you missed it and need a laugh ��", "in_reply_to_status": 725124686812712961, "in_reply_to_user": 1643915736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 709283976 }}, "user": { "id": 1643915736, "name": "Teresa aguilar", "screen_name": "Teresa24Aguilar", "lang": "en", "location": "Choctaw, OK", "create_at": date("2013-08-03"), "description": "Your choices, your actions, your life. Live it the way you want, and don't let anyone tell you otherwise.", "followers_count": 1804, "friends_count": 687, "statues_count": 68778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217931836166144, "text": "@RachRandall HAHAHAHA", "in_reply_to_status": 725216090654167040, "in_reply_to_user": 351261434, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 351261434 }}, "user": { "id": 239998178, "name": "Stephen Ysl", "screen_name": "VoteForStephen", "lang": "en", "location": "Lewisham, London", "create_at": date("2011-01-18"), "description": "Sigma Alpha Epsilon | The Ohio University", "followers_count": 813, "friends_count": 592, "statues_count": 48801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Athens, OH", "id": "77aa8a0640caa98c", "name": "Athens", "place_type": "city", "bounding_box": rectangle("-82.173479,39.27973 -82.01933,39.361038") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39009, "countyName": "Athens", "cityID": 3902736, "cityName": "Athens" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932054302720, "text": "I hate waking up in the middle of the night hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19510750, "name": "Riot Henry", "screen_name": "RiotHenry", "lang": "en", "location": "Charlotte NC", "create_at": date("2009-01-25"), "description": "Boy who cried Wolf... Young Ebenezer", "followers_count": 1039, "friends_count": 351, "statues_count": 74733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932209442820, "text": "Ripley SW Limestone Co. Temp: 66.7°F Wind:1.6mph Pressure: 989.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932213506048, "text": "@basesfuckingego so annoyed man ):", "in_reply_to_status": 725216841321222145, "in_reply_to_user": 1639292520, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1639292520 }}, "user": { "id": 898747476, "name": "Delmy", "screen_name": "delmysanchez1", "lang": "en", "location": "null", "create_at": date("2012-10-22"), "description": "♏️", "followers_count": 553, "friends_count": 425, "statues_count": 26573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932339503107, "text": "Wind 0.0 mph N. Barometer 29.93 in, Steady. Temperature 36.9 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932394020864, "text": "https://t.co/RTiiuoVLPa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3960331295, "name": "Patty lemmon....Frye", "screen_name": "PattyFrye12", "lang": "en", "location": "null", "create_at": date("2015-10-14"), "description": "born ohio.....always TRUMP.....retired timken co 30yrs..a boy scout...love my grandchildren...TRUMP supporter", "followers_count": 437, "friends_count": 732, "statues_count": 6711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perry Heights, OH", "id": "ad5246057045cc0e", "name": "Perry Heights", "place_type": "city", "bounding_box": rectangle("-81.498112,40.767243 -81.425054,40.820745") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3962134, "cityName": "Perry Heights" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932888969216, "text": "Yall thought I was sleep huh �� https://t.co/1pCFpMKbxp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419149456, "name": "1700 Lil Malc", "screen_name": "GasMan_", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-11-22"), "description": "im too high fa dis shit. WHS Alumni '15 #ssu. #DRVGxVPRACT! #SWATS", "followers_count": 1838, "friends_count": 928, "statues_count": 89538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217932951851008, "text": "You're my world but you're too blind to see that...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1253073830, "name": "Hannah Breezy⚪️", "screen_name": "HannahTrevino21", "lang": "en", "location": "Probably sleeping", "create_at": date("2013-03-08"), "description": "⚪️Sophmore at Case⚪️262⚪️", "followers_count": 347, "friends_count": 291, "statues_count": 10130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217933211914240, "text": "Happy birthday to one of my favoritest people ever STACECHANG!! I know you don't read so I'll… https://t.co/E24NOUzxdN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.25606876,37.87536209"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142494507, "name": "Sheezy", "screen_name": "OhMyKaution", "lang": "en", "location": "Bae Area, Cali Love. ", "create_at": date("2010-05-10"), "description": "I am the King of the World. \nকলকাতা \nUC Berkeley till I drop out.", "followers_count": 1896, "friends_count": 166, "statues_count": 169126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217933723471873, "text": "It was turnt https://t.co/HbF6plaIKw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2999074784, "name": "T.Kelly", "screen_name": "Satnjooh", "lang": "en", "location": "null", "create_at": date("2015-01-27"), "description": "Man , Visionary , God Fearing (Not the conversation type) Power Lvl 9000+", "followers_count": 507, "friends_count": 524, "statues_count": 19163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-04-27T00:00:03.000Z"), "id": 725217933849317376, "text": "Wind 12.0 mph E. Barometer 29.765 in, Rising slowly. Temperature 51.2 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217933924818944, "text": "@chaviraalex_xo miss you ��", "in_reply_to_status": 725217843487232000, "in_reply_to_user": 965468826, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 965468826 }}, "user": { "id": 596906439, "name": "Jen ☯", "screen_name": "_Jennifeeeeeer", "lang": "en", "location": "htx.", "create_at": date("2012-06-01"), "description": "null", "followers_count": 1755, "friends_count": 1557, "statues_count": 45986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934251958272, "text": "2day @ bigscoop a lady said she is allergic to nuts and her son says \" you must not be that allergic to nuts you had 6 kids\" it was lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 754656871, "name": "Ali Heyntsen", "screen_name": "aliheyntsen", "lang": "en", "location": "null", "create_at": date("2012-08-13"), "description": "WA sc: aliheyntsen", "followers_count": 759, "friends_count": 594, "statues_count": 6262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, WA", "id": "ac2bdb59bf2a437e", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-122.357149,48.445589 -122.291309,48.495195") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit", "cityID": 5308920, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934361157632, "text": "You guys are awesome! Thanks for hanging out and making my night a good one. Goodnight��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3040053241, "name": "John Kniffin", "screen_name": "JohnKniffin", "lang": "en", "location": "null", "create_at": date("2015-02-24"), "description": "23 ~ Singer-Songwriter / Insta & Snapchat: johnkniffin", "followers_count": 620, "friends_count": 63, "statues_count": 1350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Day Valley, CA", "id": "0d902f3b8af34c66", "name": "Day Valley", "place_type": "city", "bounding_box": rectangle("-121.900769,36.981533 -121.804421,37.016077") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 618153, "cityName": "Day Valley" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934566653952, "text": "I really like @KodakBlack1k hair tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2163267319 }}, "user": { "id": 2477650537, "name": "YSL.CB", "screen_name": "Christa_Bosh", "lang": "en", "location": "null", "create_at": date("2014-05-04"), "description": "317! Young Thug! Snapchat:cbosh317", "followers_count": 93, "friends_count": 79, "statues_count": 574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincennes, IN", "id": "bfadcb4bfcf83cef", "name": "Vincennes", "place_type": "city", "bounding_box": rectangle("-87.547173,38.625181 -87.433471,38.736046") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18083, "countyName": "Knox", "cityID": 1879208, "cityName": "Vincennes" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934658904064, "text": "Your bitch came to my house to get fucked by my roster my team homie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397985265, "name": "Chief", "screen_name": "wallsjon_", "lang": "en", "location": "null", "create_at": date("2011-10-25"), "description": "life isn't wasted if you spend it getting wasted 05/01/96", "followers_count": 538, "friends_count": 545, "statues_count": 21631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shinnston, WV", "id": "ff45015ab17cfddf", "name": "Shinnston", "place_type": "city", "bounding_box": rectangle("-80.314936,39.374649 -80.280183,39.41007") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54033, "countyName": "Harrison", "cityID": 5473636, "cityName": "Shinnston" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934675705857, "text": "Wind 1.0 mph WNW. Barometer 29.968 in, Rising. Temperature 32.2 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.33805556,42.12361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19719569, "name": "Dean Lawton Sr.", "screen_name": "Buseedean", "lang": "en", "location": "null", "create_at": date("2009-01-29"), "description": "null", "followers_count": 8, "friends_count": 44, "statues_count": 7627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36107, "countyName": "Tioga" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217934872813568, "text": "@Redlite0_0 good looking", "in_reply_to_status": 725217881374367744, "in_reply_to_user": 2850785252, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2850785252 }}, "user": { "id": 27246424, "name": "Emergency Contact", "screen_name": "HOLA_MELLITO", "lang": "en", "location": "East Flatbush, Brooklyn", "create_at": date("2009-03-28"), "description": "#TheSuiteLife", "followers_count": 1098, "friends_count": 1075, "statues_count": 153185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217935057285120, "text": "Wind 1.6 mph S. Barometer 29.556 in, Falling slowly. Temperature 60.9 °F. Rain today 0.04 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217935338270720, "text": "This rain is amazing ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549614400, "name": "Adriiii ❤️", "screen_name": "Adri_Christie", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "null", "followers_count": 793, "friends_count": 352, "statues_count": 30403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Killeen, TX", "id": "a27a0542e774e138", "name": "Killeen", "place_type": "city", "bounding_box": rectangle("-97.848463,31.014356 -97.659217,31.144257") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4839148, "cityName": "Killeen" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217935594258433, "text": "Temp 58.5°F Wind Chill 58.5°F RH 94% Wind 0.0 --- Gust 0.0 --- SLP 29.794 in Falling slowly Rain 0.09 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 111, "friends_count": 63, "statues_count": 35004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217935933894656, "text": "ice breaker: what's your favorite dinosaur", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1211216947, "name": "poison heart", "screen_name": "talontrot", "lang": "en", "location": "somewhere california", "create_at": date("2013-02-23"), "description": "i play monster hunter. rice is forever and yayoi is the truth. 3DS: 1349-7491-1066 IG: gruntyindustries LINE: sadskeleton", "followers_count": 126, "friends_count": 178, "statues_count": 7492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217936101646340, "text": "@VictoriasSecret when are you gonna come up with new designs? they're getting boring.", "in_reply_to_status": -1, "in_reply_to_user": 16193578, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16193578 }}, "user": { "id": 954897516, "name": "tamela", "screen_name": "ughleiest", "lang": "en", "location": "on your boy's dick", "create_at": date("2012-11-17"), "description": "fuck outta my life", "followers_count": 414, "friends_count": 508, "statues_count": 31526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianola, WA", "id": "204cf8d953aff300", "name": "Indianola", "place_type": "city", "bounding_box": rectangle("-122.555441,47.742554 -122.470205,47.780466") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53035, "countyName": "Kitsap", "cityID": 5333280, "cityName": "Indianola" } }
+{ "create_at": datetime("2016-04-27T00:00:04.000Z"), "id": 725217937817108480, "text": "@_jasmineveee oh u right FUCKITYOLO LETS GO", "in_reply_to_status": 725217857286471681, "in_reply_to_user": 605968757, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605968757 }}, "user": { "id": 142510561, "name": "Chelsea Martinez", "screen_name": "seamzchel", "lang": "en", "location": "null", "create_at": date("2010-05-10"), "description": "I'm 20 and I like lasagna", "followers_count": 260, "friends_count": 250, "statues_count": 6131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217938563813378, "text": "I miss my momma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419619470, "name": "~ART~", "screen_name": "BornADancer_97", "lang": "en", "location": "North Carolina", "create_at": date("2011-11-23"), "description": "Keeping God first. Dance is my passion. eXpressions Dancer. #FayState19", "followers_count": 828, "friends_count": 980, "statues_count": 13150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, NC", "id": "4548df3d99bb4891", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-79.195342,34.936245 -78.808883,35.173009") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3722920, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217938786099200, "text": "Wind 0.0 mph ---. Barometer 29.86 in, Falling slowly. Temperature 68.0 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217938899267584, "text": "@Addingbread �������� bruh https://t.co/Z6NDRhZBRo", "in_reply_to_status": -1, "in_reply_to_user": 420932716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 420932716 }}, "user": { "id": 225989023, "name": "hamilton, jaylin", "screen_name": "jaayzus", "lang": "en", "location": "null", "create_at": date("2010-12-12"), "description": "keep it g, nigga ⚓️ | $G❤️", "followers_count": 387, "friends_count": 411, "statues_count": 19409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217939423518720, "text": "puñeta con una bellaquera como toño bicicleta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1322726774, "name": "• tila • ☠", "screen_name": "mama_rica", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-04-02"), "description": "D R U M & B A S S H E A D", "followers_count": 1828, "friends_count": 536, "statues_count": 111944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217939448844288, "text": "okay night ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487984150, "name": "your fav princess", "screen_name": "ashhllyyyy__", "lang": "en", "location": "somewhere princessy", "create_at": date("2012-02-09"), "description": "sparkly things and diamond rings | #LC", "followers_count": 555, "friends_count": 616, "statues_count": 40410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salisbury, NC", "id": "6d788d8a748a314a", "name": "Salisbury", "place_type": "city", "bounding_box": rectangle("-80.61977,35.591709 -80.437656,35.713629") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37159, "countyName": "Rowan", "cityID": 3758860, "cityName": "Salisbury" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217939880710144, "text": "Wind 2.0 mph SSE. Barometer 29.976 in, Steady. Temperature 51.2 °F. Rain today 0.00 in. Humidity 58%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 204, "friends_count": 59, "statues_count": 271673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217940082020353, "text": "@Captian_Brock they do exist !?������", "in_reply_to_status": 725217837887852544, "in_reply_to_user": 3096676134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3096676134 }}, "user": { "id": 4699236991, "name": "alexandra", "screen_name": "alexwerhas", "lang": "en", "location": "null", "create_at": date("2016-01-02"), "description": "♡♡♡ // sc: alexwerhas", "followers_count": 131, "friends_count": 97, "statues_count": 1053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217940195299328, "text": "OL is losing‼️ Shadowhunters voting nonstop. If we don't power vote, J&C will lose. Sam watches these.\nPlease Vote! https://t.co/JQslXInuOQ…", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25093882, "name": "Bonnie Klein", "screen_name": "SkinnyBon", "lang": "en", "location": "Dallas, TX USA", "create_at": date("2009-03-18"), "description": "Cancer survivor, widow, mom, minister, teacher, counselor, Christian mystic, writer, lover of good books, Outlander fanatic, hopeless romantic", "followers_count": 1767, "friends_count": 3156, "statues_count": 4936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217940799279104, "text": "@JoshuaLeGusta you came in here hoe", "in_reply_to_status": 725217532085305344, "in_reply_to_user": 178971056, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 178971056 }}, "user": { "id": 485390253, "name": "jordan", "screen_name": "draper_23", "lang": "en", "location": "null", "create_at": date("2012-02-06"), "description": "null", "followers_count": 443, "friends_count": 328, "statues_count": 38566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217940887359488, "text": "@420snipesalot ugh yes so much to catch up on������������", "in_reply_to_status": 725217242435055617, "in_reply_to_user": 149696332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 149696332 }}, "user": { "id": 360213934, "name": "❥ Jenn ❥", "screen_name": "_jenniferest", "lang": "en", "location": "Oregon, USA", "create_at": date("2011-08-22"), "description": "null", "followers_count": 253, "friends_count": 222, "statues_count": 6839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Dalles, OR", "id": "0003cfce3c426793", "name": "The Dalles", "place_type": "city", "bounding_box": rectangle("-121.229115,45.585142 -121.121632,45.634037") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41065, "countyName": "Wasco", "cityID": 4172950, "cityName": "The Dalles" } }
+{ "create_at": datetime("2016-04-27T00:00:05.000Z"), "id": 725217942229536769, "text": "I love it when you make a new friend and you just click and it's like you've been friends your whole life����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3708024072, "name": "Ems", "screen_name": "evu817", "lang": "en", "location": "California, USA", "create_at": date("2015-09-27"), "description": "I love puppies and sandwiches", "followers_count": 204, "friends_count": 320, "statues_count": 2899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217943403929601, "text": "The fucking audacity", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48377424, "name": "JULIEANNE", "screen_name": "jul1eanne", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-06-18"), "description": "@GabeCacayuran ❤️", "followers_count": 271, "friends_count": 139, "statues_count": 58606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "National City, CA", "id": "7905cfee1600eb70", "name": "National City", "place_type": "city", "bounding_box": rectangle("-117.124452,32.64634 -117.049103,32.69328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 650398, "cityName": "National City" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217944070840321, "text": "4/27/2016 - 02:00\nTemp: 69.0F \nHum: 96%\nWind: 0.0 mph\nBaro: 29.688in. & Rising\nRain: 0.00 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 110, "friends_count": 53, "statues_count": 52183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217944196673536, "text": "And I don't want it to end ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1390437192, "name": "666", "screen_name": "Joaquin_313", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-04-29"), "description": "| 我充满仇恨 |", "followers_count": 673, "friends_count": 820, "statues_count": 35460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217944381214720, "text": "Temp: 41.4°F - Dew Point: 38.4° - Wind: E @ 8.9 mph - Gust: 16.6 - Rain Today: 0.00in. - Pressure: 29.91in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 12, "statues_count": 16842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217945010335744, "text": "@flacqua Clinton takes four states, limiting chances of Sanders nomination https://t.co/Xw6jE2icCQ", "in_reply_to_status": -1, "in_reply_to_user": 63222576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63222576 }}, "user": { "id": 264189033, "name": "W. Clayton", "screen_name": "weciv01", "lang": "en", "location": "null", "create_at": date("2011-03-11"), "description": "null", "followers_count": 143, "friends_count": 270, "statues_count": 43503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217945064873984, "text": "@ZACoberly but the roses are wilting, and the violets are dead, sugar bowls empty, just like his head.", "in_reply_to_status": 725217832745594881, "in_reply_to_user": 307160650, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 476262842 }}, "user": { "id": 307160650, "name": "Adara Kaylee Jones", "screen_name": "adarakayleebaby", "lang": "en", "location": "null", "create_at": date("2011-05-28"), "description": "http://vsco.co/adarakayleebaby Boston☺️✌️snapchat & Instagram: adarakayleebaby", "followers_count": 308, "friends_count": 131, "statues_count": 11157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, UT", "id": "a35adb837e057d3b", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-111.830541,40.399965 -111.751966,40.464607") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4935190, "cityName": "Highland" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217945517842432, "text": "Wind 6.0 mph ENE. Barometer 29.892 in, Steady. Temperature 44.4 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 392, "friends_count": 293, "statues_count": 10097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217946037972993, "text": "coming home & seeing my mom crying is probably the worse thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198084400, "name": "linda", "screen_name": "sandovaleslie", "lang": "en", "location": "NYC | SLC ", "create_at": date("2013-11-16"), "description": "rest easy raul armando angel ❤️", "followers_count": 769, "friends_count": 874, "statues_count": 19703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millcreek, UT", "id": "5af91023ed39df89", "name": "Millcreek", "place_type": "city", "bounding_box": rectangle("-111.921221,40.664889 -111.839259,40.712147") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4950150, "cityName": "Millcreek" } }
+{ "create_at": datetime("2016-04-27T00:00:06.000Z"), "id": 725217946314874880, "text": "I didn't like this for some reason https://t.co/y9jNMMFwqw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155189978, "name": "G5 THA GEEK", "screen_name": "CXNCO", "lang": "en", "location": "upt dc", "create_at": date("2010-06-13"), "description": "dont let the thot form from anger Chrisbrownthians (3:17) B.S; Business Admin #GØDBODY #ROSMG+RADICAL+", "followers_count": 4035, "friends_count": 2672, "statues_count": 178694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217947493502976, "text": "Howell, NJ | Wind 1.0 mph ENE. Baro 29.884 in, Steady. Temp 47.8F. Rain today 0.00 in. Humidity 89% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 617, "friends_count": 827, "statues_count": 41435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217947866685440, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1661448685, "name": "dev", "screen_name": "devinnealubong", "lang": "en", "location": "null", "create_at": date("2013-08-10"), "description": "null", "followers_count": 960, "friends_count": 548, "statues_count": 18534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217948034551808, "text": "@BarbieKeleigh Online comments seem the same as psychic attacks. No? If so, then protect by whatever means suitable. I plan to, from now on.", "in_reply_to_status": 725174524858093568, "in_reply_to_user": 35655149, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35655149 }}, "user": { "id": 2490816690, "name": "David Stewart", "screen_name": "dast323", "lang": "en", "location": "Austin, TX", "create_at": date("2014-05-11"), "description": "Life is a dream. Become a good dreamer. Dream about Liberty. Self-reliance. Soul is the dreamer of all things. #Longhorns #2A #4A USAF Vet #MAGA", "followers_count": 110, "friends_count": 169, "statues_count": 316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217948168638464, "text": "@Ofcourseanna ��", "in_reply_to_status": -1, "in_reply_to_user": 420458773, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 420458773 }}, "user": { "id": 992337506, "name": "ㅤㅤㅤㅤ-Ryan Gindhart ®", "screen_name": "zQuakke", "lang": "en", "location": "Arkansas, USA .19 UAFS ", "create_at": date("2012-12-05"), "description": "{Life is all Alcohol & Booty calls} @packers #ProMarijuana #BeardGang", "followers_count": 1792, "friends_count": 645, "statues_count": 110500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paris, AR", "id": "4328bf892b89f825", "name": "Paris", "place_type": "city", "bounding_box": rectangle("-93.760904,35.273479 -93.691152,35.306553") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5083, "countyName": "Logan", "cityID": 553480, "cityName": "Paris" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217948395139072, "text": "@ReBL_MikeLee Thank you Mikey ☺️", "in_reply_to_status": 725210152425541632, "in_reply_to_user": 223611152, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223611152 }}, "user": { "id": 141040755, "name": "Stacy Barrios", "screen_name": "Stacy94_", "lang": "en", "location": "null", "create_at": date("2010-05-06"), "description": "null", "followers_count": 187, "friends_count": 144, "statues_count": 21396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Paula, CA", "id": "4991c17aaa783ed6", "name": "Santa Paula", "place_type": "city", "bounding_box": rectangle("-119.097335,34.33027 -119.037479,34.389759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 670042, "cityName": "Santa Paula" } }
+{ "create_at": datetime("2016-04-27T00:00:07.000Z"), "id": 725217948827156482, "text": "@Hochatime roaming is good therapy you instantly lose track of the world, and enter a new world of PEACE!!", "in_reply_to_status": 725035478848671744, "in_reply_to_user": 3254793013, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3254793013 }}, "user": { "id": 4031155099, "name": "Gary Lee", "screen_name": "leedeangary9360", "lang": "en", "location": "Longview, TX", "create_at": date("2015-10-26"), "description": "COALMINER FOR 31yrs. Love NASCAR, camping, hiking etc.", "followers_count": 65, "friends_count": 159, "statues_count": 453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Longview, TX", "id": "d4157f8e4bd01598", "name": "Longview", "place_type": "city", "bounding_box": rectangle("-94.844303,32.42091 -94.618379,32.58565") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48183, "countyName": "Gregg", "cityID": 4843888, "cityName": "Longview" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305139118080, "text": "@masoooo_ you got a beating yo ? ��", "in_reply_to_status": 725580168648122368, "in_reply_to_user": 1331989292, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1331989292 }}, "user": { "id": 278823801, "name": "Rambo", "screen_name": "Traeskii__", "lang": "en", "location": "round the way 4️⃣0️⃣0️⃣", "create_at": date("2011-04-07"), "description": "get high to balance my lows ... sc: traepound19", "followers_count": 1383, "friends_count": 1316, "statues_count": 93435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305147396096, "text": "Fucking psycho .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718313875297460224, "name": "•", "screen_name": "carloscollazo_", "lang": "en", "location": "null", "create_at": date("2016-04-07"), "description": "| 20 | Sparks , NV |", "followers_count": 47, "friends_count": 66, "statues_count": 179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winnemucca, NV", "id": "807efdfb21bfb93f", "name": "Winnemucca", "place_type": "city", "bounding_box": rectangle("-117.809335,40.907926 -117.699767,40.993246") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32013, "countyName": "Humboldt", "cityID": 3284800, "cityName": "Winnemucca" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305189326848, "text": "esp with 91 lp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2566658708, "name": "♡enzo♡", "screen_name": "fernby_", "lang": "en", "location": "seattle", "create_at": date("2014-06-14"), "description": "♡ pnw ♡ 18 ♡ misnyandrist ♡ nsfw ♡", "followers_count": 155, "friends_count": 364, "statues_count": 11821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305289998336, "text": "@AmazonianBeauti well damn lol", "in_reply_to_status": 725579913709809664, "in_reply_to_user": 216676562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216676562 }}, "user": { "id": 163755160, "name": "RELL.", "screen_name": "DeronTesfay", "lang": "en", "location": "Some Where In Seattle ☔", "create_at": date("2010-07-06"), "description": "Currently... Im High off Knowledge, Overdosing on Confidence & Drunk off Ideas", "followers_count": 1589, "friends_count": 976, "statues_count": 74200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305709469696, "text": "@krillho it's good", "in_reply_to_status": 725567051771715584, "in_reply_to_user": 1968471384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1968471384 }}, "user": { "id": 20481102, "name": "Matt Napoletano", "screen_name": "BeardedBrewer87", "lang": "en", "location": "Fresno, Ca", "create_at": date("2009-02-09"), "description": "Hospitality at @TiogaSequoia / Voice on @PerfectPourPod / Coffee Drinker / Beer Sipper /Salad Bars / Pizza / Cars / Sassmaster of 2015", "followers_count": 551, "friends_count": 988, "statues_count": 10438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305868804096, "text": "You want her, you need her, & I'll never be her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550396285, "name": "Bree♡", "screen_name": "Breeluuv", "lang": "en", "location": "LA - UCM", "create_at": date("2012-04-10"), "description": "17 | Carrera is my person.", "followers_count": 857, "friends_count": 783, "statues_count": 53189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580305994768384, "text": "My bd always making me smoke ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2361725651, "name": "Red head Hottie", "screen_name": "ShareeceJ", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": "null", "followers_count": 340, "friends_count": 279, "statues_count": 12029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580306116317184, "text": "*listening to my old music* \n\n\"I feel like I'm going to get a virus on my computer listening to this\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 586207846, "name": "bæźä", "screen_name": "BaezaAdam", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-05-20"), "description": "Las Vegas", "followers_count": 928, "friends_count": 93, "statues_count": 104718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580306233708545, "text": "@khalidkhu not on the album", "in_reply_to_status": 725579927639068672, "in_reply_to_user": 163015660, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163015660 }}, "user": { "id": 382093845, "name": "Salam Al-Abdullatif", "screen_name": "Salzcamino", "lang": "en", "location": "null", "create_at": date("2011-09-29"), "description": "I do a cappella Slipknot", "followers_count": 179, "friends_count": 392, "statues_count": 18415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580306472833024, "text": "�� https://t.co/kK3Tovog7M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 185503803, "name": "not a boy", "screen_name": "imsohoustonn", "lang": "en", "location": "slapAhoe,htx", "create_at": date("2010-08-31"), "description": "I am mine before I am anyone else's", "followers_count": 1156, "friends_count": 615, "statues_count": 39640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580306758156289, "text": "@MorganFlournory but do you even watch the show.....", "in_reply_to_status": 725577219372945408, "in_reply_to_user": 424417789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 424417789 }}, "user": { "id": 2485024581, "name": "darc", "screen_name": "darcychaffin", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2014-04-14"), "description": "• florida state university • insta: darcychaffin •", "followers_count": 655, "friends_count": 286, "statues_count": 4004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580306997084160, "text": "@rileybuttler @Coachellas16 how dare you compare the two https://t.co/lzSObIsVZ4", "in_reply_to_status": 725576700298334210, "in_reply_to_user": 384673029, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 384673029, 3153284224 }}, "user": { "id": 586231360, "name": "Malerey Villasenor", "screen_name": "MalVillasenor", "lang": "en", "location": "san diego", "create_at": date("2012-05-20"), "description": "null", "followers_count": 397, "friends_count": 186, "statues_count": 2290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580307290714112, "text": "Follow me on #Google #plus ��\nCarmen Retief-Alharbi #WorkAtHome #Entrepreneurs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Google", "plus", "WorkAtHome", "Entrepreneurs" }}, "user": { "id": 4003919657, "name": "Umm Jana", "screen_name": "Cralharbi", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-10-20"), "description": "#Avonrep serving the San Antonio area. Become a beauty advisor for Avon http://www.startavon.com Reference Code: carmenretiefalharbi", "followers_count": 59, "friends_count": 222, "statues_count": 421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580307412467712, "text": "Wind 0.0 mph NNE. Barometer 29.980 in, Rising. Temperature 23.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580308226011136, "text": "Really need some Ixtapa in my life right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043782166, "name": "Ben Snyder⚽", "screen_name": "bensnyder04", "lang": "en", "location": "null", "create_at": date("2012-12-28"), "description": "I'm probably at a Sounders game ⚽️", "followers_count": 139, "friends_count": 152, "statues_count": 4206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580308414763008, "text": "@Giorgio_Thomas @BiagioCollura1 GUYS NO", "in_reply_to_status": 725579543948349440, "in_reply_to_user": 740124270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 740124270, 899046128 }}, "user": { "id": 3418203134, "name": "Ally White", "screen_name": "allywhte", "lang": "en", "location": "Tucson, AZ", "create_at": date("2015-09-01"), "description": "UofA • Black N Blue Hip Hop Crew", "followers_count": 352, "friends_count": 311, "statues_count": 2444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580308624629760, "text": "@Cassthecatmom how long until you're out of school?", "in_reply_to_status": 725579543134670850, "in_reply_to_user": 442342941, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 442342941 }}, "user": { "id": 537562937, "name": "Eddie", "screen_name": "MuscleManTaylor", "lang": "en", "location": "Louisiana Tech Library or Gym", "create_at": date("2012-03-26"), "description": "The Emøtiøn ł #PsychologyMajorsMatter", "followers_count": 1311, "friends_count": 890, "statues_count": 112040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580308771282944, "text": "Anything ���� https://t.co/qjolMigKMU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437066229, "name": "Jordyn Paige ➰", "screen_name": "Forever_Jordyn_", "lang": "en", "location": "104", "create_at": date("2011-12-14"), "description": "Single black female addicted to retail", "followers_count": 683, "friends_count": 474, "statues_count": 29516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mankato, MN", "id": "a2b439a23220cb96", "name": "Mankato", "place_type": "city", "bounding_box": rectangle("-94.06457,44.119612 -93.933999,44.210668") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27013, "countyName": "Blue Earth", "cityID": 2739878, "cityName": "Mankato" } }
+{ "create_at": datetime("2016-04-28T00:00:00.000Z"), "id": 725580309132013568, "text": "@enews They fuckin'.", "in_reply_to_status": 725572909947826177, "in_reply_to_user": 2883841, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2883841 }}, "user": { "id": 24138696, "name": "Seth Arrons", "screen_name": "SethArrons", "lang": "en", "location": "Reno, NV", "create_at": date("2009-03-12"), "description": "Opinions? I got 'em. You definitely won't like them. Sit the fuck down.", "followers_count": 529, "friends_count": 168, "statues_count": 6074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580309283147776, "text": "@HeymanHustle do you like Calvin Klein jeans Paul?", "in_reply_to_status": 725579965404700672, "in_reply_to_user": 23089228, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23089228 }}, "user": { "id": 27991343, "name": "Chris Mendez", "screen_name": "thechrismendez", "lang": "en", "location": "Brooklyn ", "create_at": date("2009-03-31"), "description": "Creative: @wknyc", "followers_count": 1454, "friends_count": 649, "statues_count": 26128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580309350244352, "text": "Shit keeping me up right now https://t.co/tUH4JD7Dj7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164918167, "name": "Rawn the uh, yeah.", "screen_name": "TMTWW_", "lang": "en", "location": "Ni(RVA)na", "create_at": date("2010-07-09"), "description": "Modern Day Buddhist. Yung Siddhartha. I Don't Lie.", "followers_count": 788, "friends_count": 663, "statues_count": 84762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580309538828288, "text": "@ebbtideapp Tide in Guilford Harbor, Connecticut 04/28/2016\nHigh 3:42am 5.3\n Low 10:03am 0.4\nHigh 4:23pm 4.9\n Low 10:21pm 1.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-72.6667,41.2717"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 79, "friends_count": 1, "statues_count": 28186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Guilford, CT", "id": "0136444405eb084a", "name": "Guilford", "place_type": "city", "bounding_box": rectangle("-72.746818,41.243282 -72.631735,41.403321") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 935020, "cityName": "Guilford Center" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580309597577216, "text": "You're just a good girl, &you know it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634349465, "name": "Lo.", "screen_name": "LJFamous", "lang": "en", "location": "null", "create_at": date("2012-07-12"), "description": "I got beef with being broke.", "followers_count": 1120, "friends_count": 878, "statues_count": 36921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580310105063424, "text": "Aku ijek ndue akeh waktu nggo nunggu kowe����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 427039020, "name": "Awall ☀️", "screen_name": "awallawall", "lang": "id", "location": "Surabaya", "create_at": date("2011-12-02"), "description": "jamiah science tambakberas", "followers_count": 412, "friends_count": 274, "statues_count": 5859 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580310172327937, "text": "04/28@03:00 - Temp 46.5F, WC 46.5F. Wind 0.0mph SW, Gust 1.0mph. Bar 29.964in, Steady. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580310314766338, "text": "https://t.co/CNdQTUibkh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2989348742, "name": "jacqueline", "screen_name": "forreverxjackie", "lang": "en", "location": "null", "create_at": date("2015-01-18"), "description": "sc: jackiebabe00 ✨", "followers_count": 1174, "friends_count": 729, "statues_count": 7791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580310331723776, "text": ".@CaptainAmerica I stand with #TeamIronMan. #TweetMe\nhttps://t.co/E8Usvxe6jd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamIronMan", "TweetMe" }}, "user_mentions": {{ 701615052 }}, "user": { "id": 125496501, "name": "FATAL", "screen_name": "FANATICALLYSICK", "lang": "en", "location": "Imma Be Where I'm At! ", "create_at": date("2010-03-22"), "description": "I shadowbox behind the scenes!!! For those that get it, get it!!!", "followers_count": 329, "friends_count": 2061, "statues_count": 28741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goldenrod, FL", "id": "4e644cf413b4c021", "name": "Goldenrod", "place_type": "city", "bounding_box": rectangle("-81.309484,28.597471 -81.26758,28.625333") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1226475, "cityName": "Goldenrod" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311556415488, "text": "So the trip begins", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289638648, "name": "JP Kehoe", "screen_name": "StudKehoe", "lang": "en", "location": "Auburn Newyork", "create_at": date("2011-04-28"), "description": "#VSB #VSBGODB", "followers_count": 253, "friends_count": 122, "statues_count": 6279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Solvay, NY", "id": "a0501795ed164a24", "name": "Solvay", "place_type": "city", "bounding_box": rectangle("-76.231345,43.041804 -76.192235,43.068445") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3668286, "cityName": "Solvay" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311673737217, "text": "Wind 0.0 mph ---. Barometer 1012.53 mb, Rising Rapidly. Temperature 54.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 13999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311732609024, "text": "#LetThatHurtGo https://t.co/y8Bv2kQye8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LetThatHurtGo" }}, "user": { "id": 2292272193, "name": "Dom Corleone ⌚️", "screen_name": "_DomKM", "lang": "en", "location": "Raleigh, NC", "create_at": date("2014-01-19"), "description": "chicken wingz & audizine #CowboysNation", "followers_count": 1213, "friends_count": 978, "statues_count": 84798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311921201152, "text": "Hugs from you will always be my favorite. ���� https://t.co/BjOZi819ay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405758597, "name": "Leesha", "screen_name": "MermaidxTears", "lang": "en", "location": "null", "create_at": date("2011-11-05"), "description": "A diamond in the sand, but you cant tell the difference on a beach full of rhinestones.", "followers_count": 0, "friends_count": 0, "statues_count": 772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311929610240, "text": "@leafpool18 yeah no way i;ll make this at all ... about a hour latter my skype on my phone goes off..", "in_reply_to_status": 725580145822572544, "in_reply_to_user": 2568261181, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2568261181 }}, "user": { "id": 2568261181, "name": "leafpool12", "screen_name": "leafpool18", "lang": "en", "location": "null", "create_at": date("2014-06-14"), "description": "female youtuber and minecraft player", "followers_count": 31, "friends_count": 93, "statues_count": 407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Windsor, CA", "id": "cf574ccb037d4ff3", "name": "Windsor", "place_type": "city", "bounding_box": rectangle("-122.83677,38.515011 -122.772142,38.566956") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 685922, "cityName": "Windsor" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580311942279168, "text": "Wind 0.0 mph ---. Barometer 29.779 in, Rising. Temperature 66.1 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580312516907009, "text": "@HanadA25 Prince Track Ft. Kodak Black & Ganja Banks - D. F. T. M. ( No Friends No Homies ) https://t.co/7WJNX1szoo", "in_reply_to_status": 725570464723009536, "in_reply_to_user": 3223968655, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3223968655 }}, "user": { "id": 52934921, "name": "MrGoldenTicket", "screen_name": "GoldenTicket954", "lang": "en", "location": "SOUTH FLORIDA ", "create_at": date("2009-07-01"), "description": "null", "followers_count": 2250, "friends_count": 1120, "statues_count": 54862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise, FL", "id": "14441682ff5a9fa3", "name": "Sunrise", "place_type": "city", "bounding_box": rectangle("-80.363112,26.119383 -80.226673,26.193854") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1269700, "cityName": "Sunrise" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580312609181696, "text": "Pt. 4 https://t.co/O4hv8971DQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 348770666, "name": "$BB.", "screen_name": "payohlaa_", "lang": "en", "location": "null", "create_at": date("2011-08-04"), "description": "null", "followers_count": 1918, "friends_count": 605, "statues_count": 49768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Haven, CT", "id": "795003fb11ee9829", "name": "New Haven", "place_type": "city", "bounding_box": rectangle("-72.998069,41.246404 -72.860248,41.350384") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 952000, "cityName": "New Haven" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580312655331328, "text": "Who needs friends when you have flowers?? Just kidding, my other DZs were watching as I was… https://t.co/QyTMEE55aR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.449898,37.77636"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258655012, "name": "champagne mami", "screen_name": "AnnaaBorsoss", "lang": "en", "location": "null", "create_at": date("2011-02-27"), "description": "Confessions of a twenty something drama queen", "followers_count": 264, "friends_count": 159, "statues_count": 13493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580312793636864, "text": "@YoureSexxy this b u https://t.co/jix4tMep0d", "in_reply_to_status": -1, "in_reply_to_user": 387252334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 387252334 }}, "user": { "id": 497512937, "name": "MOANica.", "screen_name": "xvoxv_", "lang": "en", "location": "w/ hannah. ❤️", "create_at": date("2012-02-19"), "description": "stay out of my messages, my girl crazy.", "followers_count": 2105, "friends_count": 645, "statues_count": 271303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580312814608387, "text": "Temp: 66.1°F Wind:0.0mph Pressure: 29.831hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-28T00:00:01.000Z"), "id": 725580313066242049, "text": "Coachella camping: perpetually grimy & dry. https://t.co/beIdZbKFAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212814627, "name": "grace louise quist", "screen_name": "TheTwistQuist", "lang": "en", "location": "B", "create_at": date("2010-11-06"), "description": "gave me cookie, got you cookie", "followers_count": 738, "friends_count": 222, "statues_count": 12031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blythe, CA", "id": "790308d3f2816c37", "name": "Blythe", "place_type": "city", "bounding_box": rectangle("-114.635632,33.574798 -114.531797,33.632599") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 607218, "cityName": "Blythe" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314010071040, "text": "What's left of my 3 wooden baseball bats... @tennymatt_2012 @ Carrollton, Texas https://t.co/CiL2ftm4jY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.8933,32.9901"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 498459660 }}, "user": { "id": 498459660, "name": "Matt D. Tennyson", "screen_name": "tennymatt_2012", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-02-20"), "description": "I Am Just Living Life To The Fullest Or I Am Going To Die Trying I'm A Photographer, A Skateboarder, A Guitar Player And A Baseball Player... - MATT", "followers_count": 442, "friends_count": 2486, "statues_count": 439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314542637056, "text": "I wanna buy 100 pairs of these socks just cause the sweet little note with everyone �� https://t.co/ARvm2s7x6c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325601674, "name": "Tate", "screen_name": "tatummcelroy", "lang": "en", "location": "null", "create_at": date("2013-04-03"), "description": "A little sweet & A little salty.KAHS. ♓️", "followers_count": 645, "friends_count": 335, "statues_count": 4211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richland, WA", "id": "0dd0c9c93b5519e1", "name": "Richland", "place_type": "city", "bounding_box": rectangle("-119.348075,46.164988 -119.211248,46.351367") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5358235, "cityName": "Richland" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314647523328, "text": "Deyy cute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310743798, "name": "Ricky Richard", "screen_name": "SlickRich22", "lang": "en", "location": "Arlington, TX", "create_at": date("2011-06-04"), "description": "Look inside of my soul and you can find gold and maybe get rich.. #BlackLivesMatter #Dreamville", "followers_count": 1148, "friends_count": 697, "statues_count": 82667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314685272064, "text": "To be mad or happy�� https://t.co/lJoSa6Mqyo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3298537542, "name": "LAI", "screen_name": "miisslai", "lang": "en", "location": "Long Beach, CA", "create_at": date("2015-07-27"), "description": "null", "followers_count": 191, "friends_count": 92, "statues_count": 491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314744131584, "text": "Wind 1.0 mph NW. Barometer 29.717 in, Rising Rapidly. Temperature 62.4 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314752348160, "text": "monday a friend of mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191450150, "name": "Vandreiz Thurmond", "screen_name": "vandreizx", "lang": "en", "location": "null", "create_at": date("2010-09-16"), "description": "null", "followers_count": 1130, "friends_count": 470, "statues_count": 78909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Savannah, GA", "id": "00e9226863a6e5a4", "name": "Savannah", "place_type": "city", "bounding_box": rectangle("-81.188663,31.922596 -81.03057,32.111973") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1369000, "cityName": "Savannah" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580314840539136, "text": "Wind 0.0 mph SE. Barometer 29.760 in, Falling slowly. Temperature 60.3 °F. Rain today 0.00 in. Humidity 100%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580315595436032, "text": "@SPORTalkDucks let him go please!!Good coach for regular season but playoff hockey is a whole different animal!! He is not up to the task!", "in_reply_to_status": 725550415538884608, "in_reply_to_user": 3398119492, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3398119492 }}, "user": { "id": 342455832, "name": "Tomi Maiers", "screen_name": "TomiMaiers", "lang": "en", "location": "California", "create_at": date("2011-07-25"), "description": "null", "followers_count": 105, "friends_count": 172, "statues_count": 4978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580315654275072, "text": "Temp 38.2° Hi/Lo 47.5/38.2 Rng 9.3° WC 38.2° Hmd 60% Rain 0.00\" Storm 0.00\" BAR 29.965 Rising DP 25.5° Wnd 0mph Dir --- Gst 9mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580315847073793, "text": "@Mariela96_ to me it should be mutual honestly, just go back and forth you know?", "in_reply_to_status": 725580093754503168, "in_reply_to_user": 549729930, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 549729930 }}, "user": { "id": 2208639128, "name": "Young Neil", "screen_name": "Origyoungneil", "lang": "en", "location": "null", "create_at": date("2013-11-22"), "description": "Its ya boi Young Neil.", "followers_count": 467, "friends_count": 398, "statues_count": 10563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580316086136833, "text": "71.5F (Feels: 71.5F) - Humidity: 97% - Wind: 0.0mph --- - Gust: 0.0mph - Pressure: 1003.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580316665090048, "text": "#catfish i know why you did what you did. Handled well", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "catfish" }}, "user": { "id": 871207010, "name": "Kevin sparks", "screen_name": "KevinSparks2000", "lang": "en", "location": "null", "create_at": date("2012-10-09"), "description": "null", "followers_count": 6, "friends_count": 36, "statues_count": 18 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Zanesville, OH", "id": "0100091077be4c6f", "name": "North Zanesville", "place_type": "city", "bounding_box": rectangle("-82.064354,39.969935 -81.991305,40.046871") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39119, "countyName": "Muskingum", "cityID": 3957218, "cityName": "North Zanesville" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580316946104320, "text": "Thursday 4-28-16 https://t.co/oGAZaEqg6Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1139, "friends_count": 322, "statues_count": 3565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580317264769024, "text": "@_ThatGirlMeagan *cough* your bestfriend has been not up to date in her bestfriends relationship ��", "in_reply_to_status": 725554726322884608, "in_reply_to_user": 603817353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 603817353 }}, "user": { "id": 2910583386, "name": "Bianca", "screen_name": "b_magallanez", "lang": "en", "location": "Houston, TX", "create_at": date("2014-11-25"), "description": "Quantum Numbers will help you find Me in the Orbitals", "followers_count": 445, "friends_count": 297, "statues_count": 9876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:02.000Z"), "id": 725580317503819777, "text": "Don'twatchmewatchtv����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188432409, "name": "Zbaby", "screen_name": "ZiriyahCakesss", "lang": "en", "location": "Kaylees' World ", "create_at": date("2010-09-08"), "description": "null", "followers_count": 2324, "friends_count": 1238, "statues_count": 45492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580317688487936, "text": "AHHHHHHHHH �� https://t.co/HVhAn6cLIs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 58318120, "name": "SH∆NE", "screen_name": "ShaneMacMiller", "lang": "en", "location": "ॐ", "create_at": date("2009-07-19"), "description": "all is not lost", "followers_count": 452, "friends_count": 542, "statues_count": 9868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, DE", "id": "23aaa87ea90e76a4", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-75.788669,39.6431 -75.723528,39.715087") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1050670, "cityName": "Newark" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580317730455552, "text": "Happy birthday to my lovebug!!!����������You have absolutely no idea how amazing you are ❤️ You make… https://t.co/z82CuGdfQq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.84288281,33.25862617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178231672, "name": "Melina", "screen_name": "Mellyybon", "lang": "en", "location": "NY/AZ", "create_at": date("2010-08-13"), "description": "The University of Arizona☀️ Snapchat: Melinaaxp", "followers_count": 1044, "friends_count": 460, "statues_count": 19281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580318376251392, "text": "@_derek_stewart_ oh my god", "in_reply_to_status": 725496549044674561, "in_reply_to_user": 1037830346, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1037830346 }}, "user": { "id": 2573663904, "name": "David Collins", "screen_name": "DavidpCollins28", "lang": "en", "location": "null", "create_at": date("2014-06-17"), "description": "null", "followers_count": 186, "friends_count": 461, "statues_count": 1108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580318401515520, "text": "classic stripes https://t.co/EW7LG2kEJZ #ontheblog #fashionblogger #whatiwore #ootd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1242881,39.7419493"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontheblog", "fashionblogger", "whatiwore", "ootd" }}, "user": { "id": 65940718, "name": "Brett Elizabeth", "screen_name": "BrettEParker", "lang": "en", "location": "NYC", "create_at": date("2009-08-15"), "description": "i ❤ nyc.", "followers_count": 143, "friends_count": 226, "statues_count": 2176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianapolis, IN", "id": "018929347840059e", "name": "Indianapolis", "place_type": "city", "bounding_box": rectangle("-86.348441,39.631677 -85.937404,39.927448") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18097, "countyName": "Marion", "cityID": 1836003, "cityName": "Indianapolis city (balance)" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580318795685888, "text": "AINT NO SUNSHINE WHEN SHES GONE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977569656, "name": "Alabideezus", "screen_name": "zanerbonaner", "lang": "en", "location": "null", "create_at": date("2012-11-28"), "description": "Memories don't live like people do, They always 'member you, Whether things are good or bad, It's just the memories that you have - Beenie Man", "followers_count": 481, "friends_count": 764, "statues_count": 5418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319013920768, "text": "Wind 1.6 mph NNW. Barometer 29.99 in, Rising slowly. Temperature 39.0 °F. Rain today 0.00 in. Humidity 47%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319131213824, "text": "TBH �� even low key relationships deserves to be showed off �� https://t.co/iLm7ZivDOO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630182948, "name": "Princess ♡", "screen_name": "kimbuurlyy", "lang": "en", "location": "Ca ✈️ Tx", "create_at": date("2012-07-08"), "description": "@kobebryant & Kari ❤️ & Aldi", "followers_count": 940, "friends_count": 616, "statues_count": 55737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319433187328, "text": "Every night I try to go to bed by 10 Then everyone starts getting off work and group chats are \"lit\" until 2am. I know... SILENT... But FOMO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71923815, "name": "STACY", "screen_name": "STACYYA", "lang": "en", "location": "ÜT: 34.158639,-118.83082", "create_at": date("2009-09-05"), "description": "IG: Stacyya Snapchat: Stacyya ••OG||ΑΦ••", "followers_count": 229, "friends_count": 271, "statues_count": 302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Agoura Hills, CA", "id": "0d9a7f9b8f0baf12", "name": "Agoura Hills", "place_type": "city", "bounding_box": rectangle("-118.794237,34.125821 -118.715023,34.168336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600394, "cityName": "Agoura Hills" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319877902336, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":59.2°F Wind:2.0mph Pressure: 29.82hpa Falling Rain Today 0.21in. Forecast: Precipitation, very unse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 316430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319940710402, "text": "I'm not kidding y'all I'm literally sleeping underneath my desk #FinalsGotMeLike.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FinalsGotMeLike" }}, "user": { "id": 1432120148, "name": "aLy", "screen_name": "AlyTmz", "lang": "en", "location": "Iowa City / Chicago", "create_at": date("2013-05-15"), "description": "null", "followers_count": 581, "friends_count": 357, "statues_count": 10550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580319949242368, "text": "Ripley SW Limestone Co. Temp: 68.4°F Wind:4.5mph Pressure: 989.3mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580320108515328, "text": "Texting bae", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3224079386, "name": "April 25th", "screen_name": "Jrs_Mommy", "lang": "en", "location": "with JJ", "create_at": date("2015-05-23"), "description": "null", "followers_count": 299, "friends_count": 1289, "statues_count": 763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580320397877248, "text": "I need to take spell correct off,it keep changing my words. I know wtf I was Tryna put.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2317837945, "name": "PapaShot™", "screen_name": "Kingggmee2x", "lang": "en", "location": "H O U S T ON S C R E W S T O N", "create_at": date("2014-01-29"), "description": "Only way to get it is to get off ya ass and make it happen. AcreshomeTx #Pvnation PapaShot™ Management/Consultant EventPlanner/Host #OnThaSpot #SoulSnatchers", "followers_count": 1762, "friends_count": 1725, "statues_count": 46891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580320708431872, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 29.945 in, Steady. Temp 44.5F. Rain today 0.00 in. Humidity 97% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 618, "friends_count": 827, "statues_count": 41483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580320838291456, "text": "Birthdays in 8 days n I still have no plans����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2730997982, "name": "Tanayaa", "screen_name": "tanaya_ramirez", "lang": "en", "location": "null", "create_at": date("2014-08-13"), "description": "null", "followers_count": 163, "friends_count": 107, "statues_count": 562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Fe Springs, CA", "id": "00b028b400039d97", "name": "Santa Fe Springs", "place_type": "city", "bounding_box": rectangle("-118.098575,33.882744 -118.028833,33.975123") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669154, "cityName": "Santa Fe Springs" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580320993468416, "text": "Hahahahaha I was so surprised and waiting for u to get mad https://t.co/56RjZVacDE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172198709, "name": "Kristina Mai", "screen_name": "KristinaJuice7", "lang": "en", "location": "saugus varsity cheer", "create_at": date("2010-07-28"), "description": "always given everyone love❤️❤️❤️❤️❤️ @lillianhillian @ally_estes @notoriousbigal1", "followers_count": 1195, "friends_count": 467, "statues_count": 32329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-28T00:00:03.000Z"), "id": 725580321526157312, "text": "@iamnickam @huntforthe fam", "in_reply_to_status": 725579593827160064, "in_reply_to_user": 100399581, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 100399581, 86196325 }}, "user": { "id": 38204241, "name": "BoKnows", "screen_name": "BoTriplett", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-05-06"), "description": "Professionally C3. Personally just a man from the Carolina's. @alinabaraz @timgunter", "followers_count": 4413, "friends_count": 421, "statues_count": 4438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322117701632, "text": "If you're reading this I love you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338868202, "name": "Coltin Henderson", "screen_name": "ColtinHenderson", "lang": "en", "location": "Hayward", "create_at": date("2011-07-19"), "description": "Lets make this simple.. I love God, sports, skating, fast cars and pretty girls.", "followers_count": 221, "friends_count": 336, "statues_count": 1496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, WI", "id": "00e21a81751b0883", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-91.519529,45.986148 -91.461948,46.027024") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55113, "countyName": "Sawyer", "cityID": 5533450, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322134319105, "text": "If this ain't me in my room with all those dance choreo's lmao https://t.co/pJ2apgU42Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2473019964, "name": "DJ SarahBow", "screen_name": "SarahLopezz_", "lang": "en", "location": "Arizona.", "create_at": date("2014-05-01"), "description": "ASU '18 ☀️", "followers_count": 269, "friends_count": 451, "statues_count": 6480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322268569600, "text": "Wind 0.0 mph ---. Barometer 29.770 in, Rising slowly. Temperature 56.4 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322511835136, "text": "@daizsababy lmao shut up negro", "in_reply_to_status": 725580018512891904, "in_reply_to_user": 44255483, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44255483 }}, "user": { "id": 2599343401, "name": "Bo Wilson ➰", "screen_name": "bobbygee__", "lang": "en", "location": "lvnd of the snakes", "create_at": date("2014-07-02"), "description": "I told you it was beautiful", "followers_count": 1132, "friends_count": 895, "statues_count": 12652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322608283649, "text": "Wind 2.0 mph S. Barometer 29.642 in, Rising slowly. Temperature 47.3 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580322801213441, "text": "Does smoking make anyone else wanna take a dump", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45948852, "name": "Dammit Alex", "screen_name": "ItsAlejandrohhh", "lang": "en", "location": "20", "create_at": date("2009-06-09"), "description": "Server at Pizza Hut @OhwowValerie_ is the gal.", "followers_count": 454, "friends_count": 335, "statues_count": 38203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580324709748736, "text": "@Cody_schaefer Prince Track Ft. Kodak Black & Ganja Banks - D. F. T. M. ( No Friends No Homies ) https://t.co/7WJNX1szoo", "in_reply_to_status": 725570399585488896, "in_reply_to_user": 617915359, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617915359 }}, "user": { "id": 52934921, "name": "MrGoldenTicket", "screen_name": "GoldenTicket954", "lang": "en", "location": "SOUTH FLORIDA ", "create_at": date("2009-07-01"), "description": "null", "followers_count": 2250, "friends_count": 1120, "statues_count": 54863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise, FL", "id": "14441682ff5a9fa3", "name": "Sunrise", "place_type": "city", "bounding_box": rectangle("-80.363112,26.119383 -80.226673,26.193854") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1269700, "cityName": "Sunrise" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580324856455168, "text": "@ashleyxmaeee miss debs appreciates you for coming in Ashley never forget that!", "in_reply_to_status": 725579472921976832, "in_reply_to_user": 251751360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251751360 }}, "user": { "id": 417335825, "name": "Reese's Pieces Bitch", "screen_name": "Tinaafish", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "Gabriel C. ❤️", "followers_count": 333, "friends_count": 277, "statues_count": 16874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Carson, CA", "id": "1d62d313a45d799f", "name": "West Carson", "place_type": "city", "bounding_box": rectangle("-118.299695,33.797809 -118.285186,33.846342") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684144, "cityName": "West Carson" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580325267496960, "text": "#Trump is 2 Million votes behind #Hillary", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump", "Hillary" }}, "user": { "id": 861388620, "name": "T Gard", "screen_name": "Michiganborn58", "lang": "en", "location": "Colorado/Michigan", "create_at": date("2012-10-04"), "description": "Trying to keep myself and others on the right path.", "followers_count": 2818, "friends_count": 2874, "statues_count": 51718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Collins, CO", "id": "b2e4e65d7b80d2c1", "name": "Fort Collins", "place_type": "city", "bounding_box": rectangle("-105.148074,40.47168 -104.979811,40.656701") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 827425, "cityName": "Fort Collins" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580325364076544, "text": "Wind 0.0 mph ---. Barometer 29.89 in, Steady. Temperature 68.2 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580325372334080, "text": "Bruh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352128647, "name": "DJ HeartAttak", "screen_name": "DJHeartAttak", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-08-09"), "description": "...just pass me the aux cord • IG: djHeartAttak : contact@Djheartattak.com", "followers_count": 1999, "friends_count": 278, "statues_count": 15261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-28T00:00:04.000Z"), "id": 725580325565272068, "text": "@BubblyMandy1 @MysticFaeArt @Alex_Polinsky @ladyaquitane something.", "in_reply_to_status": 725580301590650880, "in_reply_to_user": 1486217978, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3875830455, 59704512, 464312468, 4834492455 }}, "user": { "id": 1486217978, "name": "Shayne Cohen", "screen_name": "Talentedkatz", "lang": "en", "location": "Houston, TX ", "create_at": date("2013-06-05"), "description": "Supergirl fan, Once upon a time fan, Artist, my cats perform tricks, love Disney animation & sci fi... spread the #smiles...it contagious", "followers_count": 238, "friends_count": 358, "statues_count": 11540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stafford, TX", "id": "822f7a173519a8dd", "name": "Stafford", "place_type": "city", "bounding_box": rectangle("-95.597893,29.598122 -95.526995,29.648822") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4869908, "cityName": "Stafford" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580326680940544, "text": "We made art. I can't stop. You go away, away, away. I don't play no more I'm not a kid anymore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3285162536, "name": "cathalina", "screen_name": "SEASLUT420", "lang": "en", "location": "Hungary", "create_at": date("2015-07-20"), "description": "mind your own bidness bitch.", "followers_count": 174, "friends_count": 121, "statues_count": 1075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580326743871489, "text": "Chris Stapleton is just country father John Misty I love it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2959929008, "name": "Delia Lara stan", "screen_name": "abel_figgy", "lang": "en", "location": "Chino Hills & Fullerton", "create_at": date("2015-01-05"), "description": "Corey Seager stan as well.", "followers_count": 275, "friends_count": 438, "statues_count": 16551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580326760644609, "text": "Gotta delete the whole tweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330653023, "name": "I'sanah.", "screen_name": "JalynPersuasion", "lang": "en", "location": "Snatching Souls & Eating Fries", "create_at": date("2011-07-06"), "description": "Muse. Delicate Flower. Liberated Butterfly Applying Pressure .", "followers_count": 3608, "friends_count": 2786, "statues_count": 67551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daytona Beach, FL", "id": "5876e96f70283826", "name": "Daytona Beach", "place_type": "city", "bounding_box": rectangle("-81.133668,29.127565 -81.003444,29.252881") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1216525, "cityName": "Daytona Beach" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327142354944, "text": "I need to eat a whole pizza by myself. Soon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737837148, "name": "dra", "screen_name": "DraStevenson", "lang": "en", "location": "Wichita Falls, TX", "create_at": date("2012-08-04"), "description": "Knotty Head. Rule #95 #1Life #AK F.O.E.", "followers_count": 614, "friends_count": 600, "statues_count": 4520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327234596864, "text": "Yo 6am hurry tf up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 421585673, "name": "Young Dabber", "screen_name": "YoungDabber710", "lang": "en", "location": "DabCity, TX", "create_at": date("2011-11-25"), "description": "Stay high, 710", "followers_count": 1371, "friends_count": 902, "statues_count": 28674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327419174912, "text": "Happy #VIEWS Eve����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VIEWS" }}, "user": { "id": 2828888774, "name": "6ixGodSaul", "screen_name": "YOUNGTRAPKING98", "lang": "en", "location": "null", "create_at": date("2014-09-23"), "description": "Know Yourself Know your Worth - Drake", "followers_count": 135, "friends_count": 164, "statues_count": 2149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327599636480, "text": "my lifestyle ain't cheap g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274315144, "name": "Dyren", "screen_name": "sbh_dee", "lang": "en", "location": "orlando", "create_at": date("2011-03-29"), "description": "entrepreneur.", "followers_count": 1167, "friends_count": 291, "statues_count": 13483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327796748288, "text": "Temp 63.6°F Wind Chill 63.6°F RH 83% Wind 1.6 NW Gust 6.0 NW SLP 29.798 in Falling slowly Rain 0.01 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 111, "friends_count": 63, "statues_count": 35028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580327817732096, "text": "3:00 Wind:Calm Dry 66F RH:96% Dp:65F BP:29.7inHg Mn:Last Quarter #CarolinaWx #ClaytonNC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.328611,35.625556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CarolinaWx", "ClaytonNC" }}, "user": { "id": 180365358, "name": "Clayton, NC Weather", "screen_name": "CarolinaWx", "lang": "en", "location": "Clayton, NC ", "create_at": date("2010-08-19"), "description": "Updated using WxTweeter by PAROLE Software", "followers_count": 272, "friends_count": 37, "statues_count": 775725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Carolina, USA", "id": "3b98b02fba3f9753", "name": "North Carolina", "place_type": "admin", "bounding_box": rectangle("-84.321948,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37101, "countyName": "Johnston" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580328295755776, "text": "Loveeee my residents! @sami_michelle & @ShelbyClanin picked me up from work, they are the real MVP's☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 844747537, 1118164321 }}, "user": { "id": 128773709, "name": "Jenna Girdosky❁", "screen_name": "girdosky", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2010-04-01"), "description": "✨Just livin' life✨", "followers_count": 179, "friends_count": 234, "statues_count": 10447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580329067646978, "text": "@Yokobaby I am!", "in_reply_to_status": 702927737024077825, "in_reply_to_user": 48622077, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 48622077 }}, "user": { "id": 14164848, "name": "Manda", "screen_name": "pandanima", "lang": "en", "location": "null", "create_at": date("2008-03-17"), "description": "Bibliophile. Traveler. Geek. Incurable insomniac. \nI flip tables when I don't win Monopoly and playing Risk with me is a life or death affair.", "followers_count": 140, "friends_count": 224, "statues_count": 8452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawrenceville, GA", "id": "181444e0e9e17e20", "name": "Lawrenceville", "place_type": "city", "bounding_box": rectangle("-84.046802,33.910542 -83.951606,33.990728") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1345488, "cityName": "Lawrenceville" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580329180909568, "text": "@DJ_Romestallion yo shoot in some work! https://t.co/91DZvlWlFE�� all music drops! were fuled by artists!��", "in_reply_to_status": -1, "in_reply_to_user": 52524715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52524715 }}, "user": { "id": 2314337120, "name": "Submit Be Heard", "screen_name": "DailyHipHopJamz", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-01-27"), "description": "The marketing tool for hip hop. Submit now⤵️ #DHHJCREW", "followers_count": 5961, "friends_count": 1994, "statues_count": 240940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580329700876288, "text": "you made it??? what serum patch did you use?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 861669320, "name": "☹", "screen_name": "JuanJChaca", "lang": "en", "location": "null", "create_at": date("2012-10-04"), "description": "take this more serious please", "followers_count": 479, "friends_count": 578, "statues_count": 6466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-04-28T00:00:05.000Z"), "id": 725580329981865984, "text": "Happy last minute of your bday I love you a lot and you make me laugh and nothing brings me more joy than watching you dance <3 @kwmccurdy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2834259228 }}, "user": { "id": 2934137035, "name": "Sim", "screen_name": "ColwillSimone", "lang": "en", "location": "WA", "create_at": date("2014-12-17"), "description": "get s'money", "followers_count": 234, "friends_count": 288, "statues_count": 252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountlake Terrace, WA", "id": "d1280141e5f979cf", "name": "Mountlake Terrace", "place_type": "city", "bounding_box": rectangle("-122.333394,47.777172 -122.28146,47.807896") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5347490, "cityName": "Mountlake Terrace" } }
+{ "create_at": datetime("2016-04-28T00:00:06.000Z"), "id": 725580330363551748, "text": "Jack in the Box tacos with ranch ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158281026, "name": "Taylor Garfield", "screen_name": "TJGarf", "lang": "en", "location": "253", "create_at": date("2010-06-22"), "description": "Proud WSU Alum and sports aficionado. I might flood your feed with Sports Stuff. Sorry. #GoCougs #NotHomeless", "followers_count": 387, "friends_count": 1540, "statues_count": 5157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-04-28T00:00:06.000Z"), "id": 725580331575824384, "text": "Wind 1.0 mph ENE. Barometer 29.832 in, Falling. Temperature 50.4 °F. Rain today 0.26 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 41, "friends_count": 4, "statues_count": 27023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-28T00:00:06.000Z"), "id": 725580331856728064, "text": "\"No L's\" is ������ af �� lol������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3113877230, "name": "TGOD", "screen_name": "__panggaa", "lang": "en", "location": "Durant, OK", "create_at": date("2015-03-28"), "description": "null", "followers_count": 252, "friends_count": 366, "statues_count": 1315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durant, OK", "id": "fe21a4e65fc5a590", "name": "Durant", "place_type": "city", "bounding_box": rectangle("-96.454342,33.979306 -96.341734,34.050118") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40013, "countyName": "Bryan", "cityID": 4022050, "cityName": "Durant" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942693415636992, "text": "I hate him so much.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57495826, "name": "poppin", "screen_name": "_ajyat", "lang": "en", "location": "Lafayette, LA", "create_at": date("2009-07-16"), "description": "UL❤️ single for no reason. just a force to be reckoned with ya know #MUA", "followers_count": 1402, "friends_count": 1147, "statues_count": 32659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942693616963584, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2239841876, "name": "Abbie", "screen_name": "abbiefriedrich", "lang": "en", "location": "Lubbock, TX", "create_at": date("2013-12-10"), "description": "2 lit 2 quit - YAM CLAN", "followers_count": 732, "friends_count": 163, "statues_count": 8576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942694267092994, "text": "Look at followers websites before you follow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3189433165, "name": "Clark Moses", "screen_name": "ScratchycClark", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 382, "friends_count": 440, "statues_count": 6439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942695005327360, "text": "@Bojalicious them are the Jesus force 12s", "in_reply_to_status": 725942145253658624, "in_reply_to_user": 2503017576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2503017576 }}, "user": { "id": 2414889733, "name": "TaiL ^v^", "screen_name": "TforTaiL", "lang": "en", "location": "Missouri", "create_at": date("2014-03-27"), "description": "C.R.E.A.M", "followers_count": 734, "friends_count": 191, "statues_count": 66587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wildwood, MO", "id": "22c7e8e90ce80ef3", "name": "Wildwood", "place_type": "city", "bounding_box": rectangle("-90.677998,38.542551 -90.581245,38.649521") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2979820, "cityName": "Wildwood" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942695437295616, "text": "You know I'm a rider, nothin left to hide, yo Otha nigga tired.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 829945110, "name": "free lil' kel", "screen_name": "libbs_452", "lang": "en", "location": "guwops trap", "create_at": date("2012-09-17"), "description": "$|um £0rd C/O '16 | 513 / 615 / 817 | R.I.P. Ian.| Free my brother Kel man ⛓", "followers_count": 696, "friends_count": 684, "statues_count": 15331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942695454085120, "text": "like it's fr cold in Newport tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1058877212, "name": "Des", "screen_name": "desbushh", "lang": "en", "location": "Houston, TX ✈ Newport Beach,CA", "create_at": date("2013-01-03"), "description": "#BANDANASEASON\n\nWatch My Life Change x Bandana", "followers_count": 164, "friends_count": 207, "statues_count": 378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport Beach, CA", "id": "0ce4c27ce26030aa", "name": "Newport Beach", "place_type": "city", "bounding_box": rectangle("-117.958219,33.563618 -117.789845,33.671658") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 651182, "cityName": "Newport Beach" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942695911284739, "text": "@JPBlanchette @SRpreps Petrino will bail", "in_reply_to_status": 725916391144988674, "in_reply_to_user": 214654106, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 214654106, 160999592 }}, "user": { "id": 1270734109, "name": "michael anderson", "screen_name": "manderpile", "lang": "en", "location": "Spokane, washington", "create_at": date("2013-03-15"), "description": "Most important things in life....FAMILY and ZAGS!!!!", "followers_count": 45, "friends_count": 68, "statues_count": 575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairwood, WA", "id": "013a0d802e0e0e6b", "name": "Fairwood", "place_type": "city", "bounding_box": rectangle("-117.453957,47.751098 -117.367548,47.826215") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5323165, "cityName": "Fairwood" } }
+{ "create_at": datetime("2016-04-29T00:00:00.000Z"), "id": 725942696771084288, "text": "Damn, I'm old now. #26", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39202816, "name": "Stephen Silver", "screen_name": "StephenSilver2", "lang": "en", "location": "null", "create_at": date("2009-05-10"), "description": "Instagram: stevie_f_baby", "followers_count": 360, "friends_count": 296, "statues_count": 14798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevenson Ranch, CA", "id": "00ab310716dc2504", "name": "Stevenson Ranch", "place_type": "city", "bounding_box": rectangle("-118.622276,34.366596 -118.560263,34.418064") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 674130, "cityName": "Stevenson Ranch" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942697324765184, "text": "Can't sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1840519640, "name": "Jəsseひ", "screen_name": "_NikeGod_", "lang": "en", "location": "Corcoran, CA", "create_at": date("2013-09-08"), "description": "when much is given, much is required.", "followers_count": 469, "friends_count": 329, "statues_count": 21037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942697622560768, "text": "Wind 2.0 mph ENE. Barometer 1017.31 mb, Steady. Temperature 58.2 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942697727430656, "text": "Time to give #Reckless a listen. @martinamcbride", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Reckless" }}, "user_mentions": {{ 14444926 }}, "user": { "id": 1525444386, "name": "NETSUA", "screen_name": "Austen127", "lang": "en", "location": "Spartanburg, South Carolina ", "create_at": date("2013-06-17"), "description": "South Carolina fan. Duke fan. SPURS. DREAM. BRAVES. COLTS. PREDATORS. COLUMBUS CREW. PS CARRIE UNDERWOOD IS QUEEN !!", "followers_count": 1480, "friends_count": 2203, "statues_count": 95485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, SC", "id": "004734e06fbc5da1", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-82.00618,34.945529 -81.980427,34.975224") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4502080, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942697760968704, "text": "The cops ended the party talkin bout if you not 21+ you gotta go lmfaooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1066433622, "name": "HulkOnCreatine", "screen_name": "BIGBERTIETV", "lang": "en", "location": "Waterbury, CT", "create_at": date("2013-01-06"), "description": "Snapchat : BigBertie_Ty. AIC'19", "followers_count": 1360, "friends_count": 982, "statues_count": 41721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942697836470273, "text": "Howell, NJ | Wind 0.0 mph ENE. Baro 30.047 in, Falling. Temp 44.4F. Rain today 0.00 in. Humidity 98% | https://t.co/jrBhN6IRrK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 620, "friends_count": 828, "statues_count": 41530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942698130075648, "text": "Rihanna and Drake track is nice, nothing special.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1033526005, "name": "Drought", "screen_name": "_Tonywolves", "lang": "en", "location": "Cesspool known as las vegas ", "create_at": date("2012-12-24"), "description": "Full-Time Aux Cord Dj.... FIRE.JAWNZ.ONLY......", "followers_count": 940, "friends_count": 790, "statues_count": 107410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942698234896385, "text": "Wind 0.0 mph ---. Barometer 29.873 in, Steady. Temperature 72.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942698394288128, "text": "Omg @SierraNeudeck & @BigWillSimmons, laughing at your Twitter back and forth and please tell me there are gifs of our video somewhere!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 410501973, 593512230 }}, "user": { "id": 39701585, "name": "Ryan Parma", "screen_name": "RyanParma", "lang": "en", "location": "null", "create_at": date("2009-05-12"), "description": "Video Director of musicals, music and dance videos. Director on @LifetimeTV's new show #DanceVideoThrowdown. Instagram: @ryanparma UT RTF Alum", "followers_count": 5697, "friends_count": 786, "statues_count": 11275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942698499141633, "text": "Temp 41.4° Hi/Lo 46.0/41.4 Rng 4.6° WC 41.4° Hmd 78% Rain 0.00\" Storm 0.00\" BAR 30.121 Rising DP 35.1° Wnd 0mph Dir --- Gst 6mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942698973122560, "text": "Felt like being wild ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1242777746, "name": "Ny-ee-luh", "screen_name": "Nailah_Iman", "lang": "en", "location": "DMV ✈️ ATL ", "create_at": date("2013-03-04"), "description": "snapchat: nailah.iman | Somewhere with Future | #Spelmanite", "followers_count": 1114, "friends_count": 1091, "statues_count": 9447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699312836608, "text": "Don't get mad at someone for not giving you enough effort when you don't give any to them", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 808888286, "name": "Bvilles Fresh Prince", "screen_name": "XREY27", "lang": "en", "location": "null", "create_at": date("2012-09-07"), "description": "The Next Episode", "followers_count": 434, "friends_count": 505, "statues_count": 1375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Ogden, UT", "id": "3149bbc966846740", "name": "South Ogden", "place_type": "city", "bounding_box": rectangle("-111.982891,41.146514 -111.914557,41.197898") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4970960, "cityName": "South Ogden" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699518398465, "text": "04/29@03:00 - Temp 45.2F, WC 45.2F. Wind 1.2mph NNE, Gust 3.0mph. Bar 30.048in, Falling slowly. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699564490752, "text": "k gn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2730638515, "name": "isa", "screen_name": "belladryl", "lang": "en", "location": "null", "create_at": date("2014-08-13"), "description": "null", "followers_count": 350, "friends_count": 266, "statues_count": 7671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699665154048, "text": "Having to choose between watching .@jimmyfallon on #TheTonightShow or \"Fever Pitch\" is the reason why DVR was invented", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheTonightShow" }}, "user_mentions": {{ 15485441 }}, "user": { "id": 592448559, "name": "Jordan Maydole", "screen_name": "JordEMay", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-05-27"), "description": "|ⓤⓒⓛⓐ '14 alumna| |USF Sport Management Grad Student|", "followers_count": 154, "friends_count": 200, "statues_count": 1357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699782631424, "text": "significant weather advisory for southwestern little river... central bowie and northwestern cass counties until... https://t.co/Y5ltlwvqGp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.03217,33.43092"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 236640778, "name": "Texarkana Weather", "screen_name": "TexarkanaAR", "lang": "en", "location": "Texarkana, AR", "create_at": date("2011-01-10"), "description": "Weather updates, forecast, warnings and information for Texarkana, AR. Sources: http://OpenWeatherMap.org, NOAA, USGS.", "followers_count": 148, "friends_count": 7, "statues_count": 20684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5091, "countyName": "Miller", "cityID": 568810, "cityName": "Texarkana" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942699883270144, "text": "Your newest Bengal,Can't wait to see him play on Sunday's.(Chose #33 because he wore 3 in college) @fantastik_willo https://t.co/t21d6sN5Ka", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 518696203 }}, "user": { "id": 3552927072, "name": "Mike Stadlman", "screen_name": "MikesGraffix", "lang": "en", "location": "Nashville, TN", "create_at": date("2015-09-13"), "description": "I'm the best at what I do. Designer for @BuckeyesNews. DM me if interested in an edit or Highlight! *MUST BE FOLLOWING TO GET ONE MADE #GeauxTigers", "followers_count": 1078, "friends_count": 654, "statues_count": 2543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dickson, TN", "id": "01bf2cf4716b1185", "name": "Dickson", "place_type": "city", "bounding_box": rectangle("-87.44198,35.994499 -87.303256,36.110694") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47043, "countyName": "Dickson", "cityID": 4720620, "cityName": "Dickson" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700160114690, "text": "It", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1576436876, "name": "Jerome Cadiz", "screen_name": "JeromeExists", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-07"), "description": "null", "followers_count": 125, "friends_count": 108, "statues_count": 1729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700378198016, "text": "@Goin_in_Goins @FazedDesigns of course", "in_reply_to_status": 725941996754333697, "in_reply_to_user": 348244689, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 348244689, 976359530 }}, "user": { "id": 91529986, "name": "Justin", "screen_name": "justinagustin_", "lang": "en", "location": "Edwardsville, IL", "create_at": date("2009-11-21"), "description": "You judged me before you met me, yeah it figures.", "followers_count": 1601, "friends_count": 673, "statues_count": 59952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edwardsville, IL", "id": "bd480515dd04af98", "name": "Edwardsville", "place_type": "city", "bounding_box": rectangle("-90.038697,38.761368 -89.906547,38.850738") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1722697, "cityName": "Edwardsville" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700474654721, "text": "Omg Seattle heads are too nice. I fucking love all of you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 569809512, "name": "Alz ✈️ NWM8", "screen_name": "AlzarathEX", "lang": "en", "location": "Brooklyn, NYC / Savage Land", "create_at": date("2012-05-02"), "description": "宗主", "followers_count": 1182, "friends_count": 232, "statues_count": 50176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirkland, WA", "id": "ba4471e5bc687736", "name": "Kirkland", "place_type": "city", "bounding_box": rectangle("-122.240397,47.642154 -122.163413,47.718484") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335940, "cityName": "Kirkland" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700558553088, "text": "3:25-3:48 on 'U With Me?' Is the standout moment on #Views, nothing better on the album than that. Wish it would've lasted longer.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Views" }}, "user": { "id": 68586539, "name": "Ralph Chiarella", "screen_name": "ralphchiarella", "lang": "en", "location": "Atlantic City, NJ", "create_at": date("2009-08-24"), "description": "bird can't fly in a cage", "followers_count": 562, "friends_count": 1396, "statues_count": 15644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlantic City, NJ", "id": "7ad0e3081108f4ba", "name": "Atlantic City", "place_type": "city", "bounding_box": rectangle("-74.467407,39.342291 -74.40702,39.386729") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3402080, "cityName": "Atlantic City" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700579557376, "text": "Temp: 47.1F W C: 47.1F Wind:N at 1.7kts Baro: 1021.6mb and Steady Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 210, "friends_count": 219, "statues_count": 103622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700600483842, "text": "21 is fucking amazing.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476939407, "name": "Austin", "screen_name": "AustinMarshall_", "lang": "en", "location": "California", "create_at": date("2012-01-28"), "description": "Be a proliferator of good vibes.", "followers_count": 307, "friends_count": 257, "statues_count": 16861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, KS", "id": "1276a44d783a529d", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-96.649744,39.159985 -96.499443,39.259872") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20161, "countyName": "Riley", "cityID": 2044250, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942700885712896, "text": "@ayeeshar650 ������ https://t.co/X5g6UQpsQi", "in_reply_to_status": -1, "in_reply_to_user": 1466831240, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1466831240 }}, "user": { "id": 2854010730, "name": "Nikki De Leon", "screen_name": "nikkideleonn", "lang": "en", "location": "null", "create_at": date("2014-10-13"), "description": "Team Wolf | snapchat : nikkideleonn | Jadine |", "followers_count": 44, "friends_count": 63, "statues_count": 3374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-29T00:00:01.000Z"), "id": 725942701242216448, "text": "Drake blessed us with another album full of IG and Snap captions...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259139798, "name": "Daniel McDowell, MSA", "screen_name": "DMcDIII", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2011-02-28"), "description": "WKU Graduate Student/Assistant Founder of #LIFE Non-profit SC: Danielson213", "followers_count": 910, "friends_count": 851, "statues_count": 47557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942702458572802, "text": "❤️❤️❤️ @Drake https://t.co/JmZAZMUA06", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 27195114 }}, "user": { "id": 116362282, "name": "SARA J", "screen_name": "nothinbut_nipz", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-02-21"), "description": "Arab American girl from the future ✨", "followers_count": 371, "friends_count": 215, "statues_count": 20403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942702668271618, "text": "Wind 0.0 mph ---. Barometer 30.018 in, Steady. Temperature 54.0 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942702718652416, "text": "I know you don't, you love me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 938317284, "name": "Old Soul.", "screen_name": "_ThatGuyRaul", "lang": "en", "location": "null", "create_at": date("2012-11-09"), "description": "We're far from good not good from far With the top down screaming We don't give a fuck! Drink my 40 ounce of freedom while I roll a blunt", "followers_count": 494, "friends_count": 368, "statues_count": 16514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942702726995968, "text": "debating if i should go to bww after work tomorrow.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32625199, "name": "✨kawaii basshead✨", "screen_name": "toxic_jackie", "lang": "en", "location": "Dodgertown ♥", "create_at": date("2009-04-17"), "description": "21 |【=◈︿◈=】| my sun and stars @illegalstylez ❤️ 4.5.15 ☀️⭐️", "followers_count": 472, "friends_count": 314, "statues_count": 74749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942702978662401, "text": "Wind 0.0 mph NNW. Barometer 30.153 in, Steady. Temperature 25.6 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703033180160, "text": "You'll get what you did to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519840609, "name": "ѕmv ☸", "screen_name": "ItsLinnnaaaBihh", "lang": "en", "location": "Lansing, Mi ✈️ Dallas, Tx", "create_at": date("2012-03-09"), "description": "20 yrs old sc: iamfleektf #LongLiveSliccG #LongLiveDmoney ❤️ ☮", "followers_count": 1717, "friends_count": 1492, "statues_count": 42611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703481966593, "text": "@allen_gera stop Bruce you're great", "in_reply_to_status": 725939901498150913, "in_reply_to_user": 700274528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 700274528 }}, "user": { "id": 1619211072, "name": "Austin Riegler", "screen_name": "austinriegler15", "lang": "en", "location": "Athens, OH", "create_at": date("2013-07-24"), "description": "Ohio University", "followers_count": 537, "friends_count": 260, "statues_count": 6169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703557513216, "text": "Fwm i'll put her on it lol https://t.co/5LjRmbBBA7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 428466165, "name": "KING✨", "screen_name": "nrf1993", "lang": "en", "location": "Eunice, La", "create_at": date("2011-12-04"), "description": "Chasin this money follow me and i'll get you there", "followers_count": 1776, "friends_count": 2162, "statues_count": 87249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eunice, LA", "id": "24e9f647ae9be121", "name": "Eunice", "place_type": "city", "bounding_box": rectangle("-92.500072,30.45788 -92.363908,30.516903") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22097, "countyName": "St. Landry", "cityID": 2224565, "cityName": "Eunice" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703653974017, "text": "I just want to be invited to stay inside the castle, is that too much to ask? �� disneyland… https://t.co/EhjsxoE9LF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.925217,33.815354"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24474361, "name": "Melinda", "screen_name": "Jay_Meli", "lang": "en", "location": "null", "create_at": date("2009-03-14"), "description": "Single mom, model and animal lover. Now, let's make some stories.", "followers_count": 395, "friends_count": 574, "statues_count": 1881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703767195649, "text": "3:00 AM ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 332034886, "name": "Brenda ❁", "screen_name": "JanieValverde14", "lang": "en", "location": "Fayetteville ,NC", "create_at": date("2011-07-08"), "description": "null", "followers_count": 605, "friends_count": 572, "statues_count": 12451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fayetteville, NC", "id": "4548df3d99bb4891", "name": "Fayetteville", "place_type": "city", "bounding_box": rectangle("-79.195342,34.936245 -78.808883,35.173009") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37051, "countyName": "Cumberland", "cityID": 3722920, "cityName": "Fayetteville" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703863648256, "text": "Happy Birthday to this gorgeous beautiful girl. I hope it's as amazing and beautiful you are. I love you��������������������❤️ https://t.co/FRORxTKb4l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735791437, "name": "Connor J Kmetic", "screen_name": "connorkmetic", "lang": "en", "location": "null", "create_at": date("2014-08-06"), "description": "10/10", "followers_count": 394, "friends_count": 366, "statues_count": 1245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703897243649, "text": "Killed it bruhhh �������� https://t.co/o05ywLCpir", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163972971, "name": "mobywagas", "screen_name": "MobyWagas", "lang": "en", "location": "null", "create_at": date("2010-07-07"), "description": "null", "followers_count": 253, "friends_count": 261, "statues_count": 3188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942703922372609, "text": "Hit my old nigga now she thinking that he played me ���� bihh you got my old freak����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453398815, "name": "QueenDee❤️", "screen_name": "SheGetMoney_", "lang": "en", "location": "Prairie View, TX", "create_at": date("2012-01-02"), "description": "AIN'T WORRIED ABOUT MUCH ➰PVAMU19", "followers_count": 1186, "friends_count": 881, "statues_count": 69965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942704052400128, "text": "Temp: 69.1°F Wind:0.0mph Pressure: 29.949hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942704245362688, "text": "Wind 0.0 mph ---. Barometer 29.966 in, Rising slowly. Temperature 53.7 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942704467677187, "text": "Wind 1.6 mph NNE. Barometer 30.11 in, Steady. Temperature 39.0 °F. Rain today 0.00 in. Humidity 48%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942704517967872, "text": "74.0F (Feels: 74.0F) - Humidity: 96% - Wind: 8.3mph SE - Gust: 9.2mph - Pressure: 1005.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942704920662016, "text": "������ https://t.co/CDG8DLOMit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 465272297, "name": "Hope Dealer", "screen_name": "MadeMen94", "lang": "en", "location": "Anderson, SC", "create_at": date("2012-01-15"), "description": "God is Real.Romans 8:18 - IRON ADDICT C.O.D.E FITNESS/ IG:CODEFITNESSUSA/Army Veteran", "followers_count": 1396, "friends_count": 903, "statues_count": 68017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anderson, SC", "id": "42d68bc4d1591d12", "name": "Anderson", "place_type": "city", "bounding_box": rectangle("-82.707012,34.447616 -82.548858,34.610762") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45007, "countyName": "Anderson", "cityID": 4501360, "cityName": "Anderson" } }
+{ "create_at": datetime("2016-04-29T00:00:02.000Z"), "id": 725942705059057665, "text": "How come all the pretty girls like you are taken baby ���� https://t.co/UcX7Pac5NZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2946630427, "name": "Joaquin Pasillas", "screen_name": "The_kingg24", "lang": "en", "location": "u.s army ", "create_at": date("2014-12-28"), "description": "18//Work hard for what you want because it won't come to you without a fight// Military Bound", "followers_count": 233, "friends_count": 402, "statues_count": 3187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Linda, CA", "id": "58dfcfad72e4ccdb", "name": "Linda", "place_type": "city", "bounding_box": rectangle("-121.592289,39.10136 -121.504874,39.143222") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6115, "countyName": "Yuba", "cityID": 641572, "cityName": "Linda" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942705684008960, "text": "why this no titty ass bitch follow me... again ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2249826439, "name": "Da Plug", "screen_name": "Han_Glolo", "lang": "en", "location": "405", "create_at": date("2013-12-16"), "description": "I got broads in Atlanta", "followers_count": 475, "friends_count": 420, "statues_count": 3456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942705755291649, "text": "@__JayMarie21 https://t.co/m6g7DGtOUI", "in_reply_to_status": 725942556316442624, "in_reply_to_user": 2291798875, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2291798875 }}, "user": { "id": 333210512, "name": "Cesar Escobedo", "screen_name": "CesarEscobedo94", "lang": "en", "location": "Chicago,Il", "create_at": date("2011-07-10"), "description": "null", "followers_count": 294, "friends_count": 533, "statues_count": 7083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706011172866, "text": "Nice shirt https://t.co/yRhh6GefuH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2149794835, "name": "Chris Candy", "screen_name": "ChrisCandy4u", "lang": "en", "location": "Los Angeles", "create_at": date("2013-10-22"), "description": "no flipping. I act and play music.", "followers_count": 633, "friends_count": 375, "statues_count": 1325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706044690433, "text": "I think I punched myself in the eye on accident ?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1310112067, "name": "Arielleee c;", "screen_name": "HaroArielle", "lang": "en", "location": "null", "create_at": date("2013-03-28"), "description": "null", "followers_count": 313, "friends_count": 178, "statues_count": 3881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706157948929, "text": "Super happy I got to hang with @RioCoyote and @wolfknight1990 @HigsbyTheDeer etc etc in Atlanta. Will defs come back :3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2701045180, 156287305, 2253713785 }}, "user": { "id": 398548641, "name": "Sky Cheetah", "screen_name": "sky_cheetah", "lang": "en", "location": "Orange County CA", "create_at": date("2011-10-25"), "description": "An Aussie cheet living (lost) in California. Car cat. Avionics Engineer, Traveller, World Citizen. #THEEE", "followers_count": 1780, "friends_count": 513, "statues_count": 3296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johns Creek, GA", "id": "00975c9578f9e109", "name": "Johns Creek", "place_type": "city", "bounding_box": rectangle("-84.286258,33.985534 -84.097879,34.0905") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1342425, "cityName": "Johns Creek" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706187341824, "text": "all these people dropping albums but frank ocean is like dead or something", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2291769840, "name": "cyberbully", "screen_name": "mercedesbenzos", "lang": "en", "location": "slide thru", "create_at": date("2014-01-14"), "description": "digital dirtbag", "followers_count": 171, "friends_count": 106, "statues_count": 5275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Verano, CA", "id": "a9f8f95142f106f6", "name": "El Verano", "place_type": "city", "bounding_box": rectangle("-122.503028,38.286955 -122.476403,38.307254") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 622510, "cityName": "El Verano" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706287996928, "text": "Y'all power went out?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354769519, "name": "Buy Me Shoes", "screen_name": "_JohnBeChillin", "lang": "en", "location": "UNT, Texas", "create_at": date("2011-08-14"), "description": "Gambian #ColeWorld ΑΚΨ", "followers_count": 895, "friends_count": 635, "statues_count": 37252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706631897089, "text": "@nataliebbrown13 LMAOOOOOOOOOOOOOOOOOOOOOOOOOOOOO", "in_reply_to_status": 725924506523459588, "in_reply_to_user": 173678432, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 173678432 }}, "user": { "id": 45899080, "name": "Andrea", "screen_name": "andimacchiato", "lang": "en", "location": "somewhere singing", "create_at": date("2009-06-09"), "description": "|@BaylorZTA|@BaylorVirtuOSO|Apparel Design & Entrepreneurship|RIP JM|", "followers_count": 964, "friends_count": 421, "statues_count": 30023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706778693632, "text": "Kid Frankie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3308287993, "name": "$", "screen_name": "vnndreeeew", "lang": "en", "location": "Salinas, CA", "create_at": date("2015-08-06"), "description": "null", "followers_count": 253, "friends_count": 198, "statues_count": 1464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942706896171009, "text": "Exactly �� exactly exactly exactly. https://t.co/sdraBuPot1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155712061, "name": "fenty.", "screen_name": "robinfenty_", "lang": "en", "location": "null", "create_at": date("2010-06-14"), "description": "God is still working on me.", "followers_count": 2705, "friends_count": 936, "statues_count": 95622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942707181363200, "text": "Happy birthday to my day one!! Finally 17 eeey! Listen to views and stay up! Have a good one Cuddy! @jmclarke5 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1594043118 }}, "user": { "id": 533752707, "name": "JMoney4 ⚡️", "screen_name": "Jaden1027", "lang": "en", "location": "Portland, OR", "create_at": date("2012-03-22"), "description": "WHS class of 2017 snapchat @jaden1027 instagram @Jaden2798 And mama don't cry, your son can handle his - Lil Wayne", "followers_count": 731, "friends_count": 513, "statues_count": 6367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethany, OR", "id": "002b68e9273298f0", "name": "Bethany", "place_type": "city", "bounding_box": rectangle("-122.867608,45.542616 -122.805462,45.568094") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105950, "cityName": "Bethany" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942707353329664, "text": "MarcFord #elijahford Kirstenford #antoinearvizu at #theprospector in #longbeach #california on… https://t.co/6welmTE3Qy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.16315424,33.77520701"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "elijahford", "antoinearvizu", "theprospector", "longbeach", "california" }}, "user": { "id": 17745859, "name": "HairyBerries", "screen_name": "Nomad66", "lang": "en", "location": "Long Beach, California, USA", "create_at": date("2008-11-29"), "description": "A free spirit fighting the fetters of society. Once every 60 years, the Fire Horse is born. Beware! Capturing the live jive, that's vibe certified.", "followers_count": 225, "friends_count": 630, "statues_count": 3723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942707386863616, "text": "Bitch posts her titties for all of Twitter and people are defending her like no stop. She's a lil hoe. End of story", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1917774930, "name": "Chris Collier", "screen_name": "chriscollier777", "lang": "en", "location": "null", "create_at": date("2013-09-29"), "description": "vhs", "followers_count": 674, "friends_count": 504, "statues_count": 6690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942707521089536, "text": "Ripley SW Limestone Co. Temp: 59.9°F Wind:0.0mph Pressure: 994.6mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942707579805697, "text": "@chris_mahan Be careful. They're working on the Burbank exits.", "in_reply_to_status": 725942474271662080, "in_reply_to_user": 806585, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 806585 }}, "user": { "id": 37882628, "name": "Lisa Is A Farmer", "screen_name": "SamuraiLisa", "lang": "en", "location": "Newton, IL", "create_at": date("2009-05-05"), "description": "Time Magazine Person Of The Year 2011. Waiting 4 strawberries and cucumbers to be in season. #YesAllWomen #BlackLivesMatter #FeelTheBern", "followers_count": 1345, "friends_count": 2141, "statues_count": 66691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newton, IL", "id": "b5783ad9ea014b3d", "name": "Newton", "place_type": "city", "bounding_box": rectangle("-88.207281,38.972737 -88.14386,38.998089") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17079, "countyName": "Jasper", "cityID": 1752844, "cityName": "Newton" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942708393529350, "text": "Jared Goff's interview rn got me rollin on the floor laughing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 21357586, "name": "Zarabe", "screen_name": "deenutty", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-02-19"), "description": "Fighting modern day gladiator wars ♌️ IG:DarthDreah #BORNRADICALS", "followers_count": 393, "friends_count": 190, "statues_count": 14425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, CA", "id": "8eccead85cc1b6c0", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-121.260264,38.621443 -121.085432,38.714331") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942708401885185, "text": "Temp: 67.8°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.98 in | Dewpoint: 67.6°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 166939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942708448043009, "text": "I fall for fictional characters too hard and I expect people to be that for me. That's my problem.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 527114186, "name": "Tania", "screen_name": "taniawonders", "lang": "en", "location": "null", "create_at": date("2012-03-16"), "description": "null", "followers_count": 286, "friends_count": 187, "statues_count": 15176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942708506746880, "text": "BACK AT IT AGAIN DRAKE WITH THE LINES AGAIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 981821948, "name": "Jair Z", "screen_name": "Gucci_Diamond13", "lang": "en", "location": "@ The Rodeo ", "create_at": date("2012-11-30"), "description": "Senior 2016, Utah State University bound, President of United Latinos of Utah, Musician, LULAC member, Chicano, Arleth Valeria Garcia is my girlfriend", "followers_count": 205, "friends_count": 263, "statues_count": 13806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kearns, UT", "id": "16f11b6e2e5b8cc1", "name": "Kearns", "place_type": "city", "bounding_box": rectangle("-112.043213,40.638527 -111.986427,40.667799") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4940470, "cityName": "Kearns" } }
+{ "create_at": datetime("2016-04-29T00:00:03.000Z"), "id": 725942709060390912, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":55.2°F Wind:1.3mph Pressure: 30.00hpa Rising slowly Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 316525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942709710548994, "text": "#IdLikeToAnnounce that depression sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IdLikeToAnnounce" }}, "user": { "id": 68013016, "name": "Grumpy Dawnie", "screen_name": "DawnMuses", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2009-08-22"), "description": "Rainbow of fruit flavors. \nAutism Warrior Mom. Hashtag game Junky. Shakespeare taste on a Fart Joke budget.", "followers_count": 1132, "friends_count": 2194, "statues_count": 6024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942709924425730, "text": "Wind 0.0 mph ---. Barometer 29.728 in, Falling slowly. Temperature 70.3 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942711115636737, "text": "Sex with me so amazing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334816918, "name": "Brazy Bray", "screen_name": "Fastdontlie9", "lang": "en", "location": "just know the water is still", "create_at": date("2011-07-13"), "description": "Let go and let God | #OOB | #FARWG | Oklahoma State 18' | Oriental | ILO | you like bagels? | i kinda wanna be a promoter |", "followers_count": 2194, "friends_count": 978, "statues_count": 52652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942711161757696, "text": "@DescribeMe_RARE great mindset", "in_reply_to_status": 725941651714138112, "in_reply_to_user": 254227679, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 254227679 }}, "user": { "id": 2182584477, "name": "Flippa McFadden", "screen_name": "_mcfadden01", "lang": "en", "location": "Orlando, FL", "create_at": date("2013-11-15"), "description": "I'm From A Place Called Curveway, FL SC:mcfadden01 : Valencia-UCF 18 IG: d.mcfadden", "followers_count": 1031, "friends_count": 990, "statues_count": 6496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "01f196c135e37b16", "name": "University", "place_type": "city", "bounding_box": rectangle("-81.224485,28.56471 -81.170066,28.61218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942711258226689, "text": "@shmarsen eep! https://t.co/TEVkeXiCqX", "in_reply_to_status": 725942513173811201, "in_reply_to_user": 633058916, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 633058916 }}, "user": { "id": 3383972031, "name": "Jønathan Flusser, Jr", "screen_name": "osxdude", "lang": "en", "location": "Lake Forest, IL", "create_at": date("2015-07-19"), "description": "21. He/him. Mental illness. Computer whisperer. Too hipster for photo filters.", "followers_count": 466, "friends_count": 1598, "statues_count": 8258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, IL", "id": "d50345354159b16c", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-87.942895,42.200288 -87.807195,42.280041") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1741105, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942712336154624, "text": "@iimaniixoxo hey mami", "in_reply_to_status": 725942605817647109, "in_reply_to_user": 48600431, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48600431 }}, "user": { "id": 2595561721, "name": "M", "screen_name": "xomirandaaa_", "lang": "en", "location": "Ceres, CA", "create_at": date("2014-06-29"), "description": "@stfuimcute ❤️", "followers_count": 1007, "friends_count": 619, "statues_count": 16330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942712457814016, "text": "@mermaidleelee it was years ago. Doesn't matter what people were saying. Those people are probably low life's in their 20s with no job.", "in_reply_to_status": 725942498669907968, "in_reply_to_user": 4572844230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4572844230 }}, "user": { "id": 301128301, "name": "Briana Alexis❥", "screen_name": "Bri_Fabreezy", "lang": "en", "location": "Los Angeles, California ", "create_at": date("2011-05-18"), "description": "Bri✖️.22. CSUF Psych Major. Preschool Teacher. Writer. Music is Life ♪ Lovestrong♡. I'm the sun, & he can go suck it! ☼ Somerholic ❤️ Instagram:@bri_fabreezy", "followers_count": 254, "friends_count": 311, "statues_count": 27543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942712692699139, "text": "Found 20$ on the floor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219172252, "name": "Jeremy Ubaldo", "screen_name": "YouBaldOh", "lang": "en", "location": "null", "create_at": date("2010-11-23"), "description": "null", "followers_count": 177, "friends_count": 164, "statues_count": 721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942713032413184, "text": "ok then drake , I'm fucking with it .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 488074351, "name": "BRE.", "screen_name": "lifesA_BREezee", "lang": "en", "location": "memphis .", "create_at": date("2012-02-09"), "description": "#mtsu", "followers_count": 1967, "friends_count": 1063, "statues_count": 33648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942713204400129, "text": "Friday 4-29-16 https://t.co/c2G6DI9X3n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1139, "friends_count": 322, "statues_count": 3568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942713309270016, "text": "NEW POST: 10 Digital Resources That Make Blogging A Breeze https://t.co/AgNZC3e7hC #fbloggers #bbloggers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fbloggers", "bbloggers" }}, "user": { "id": 257921418, "name": "Life of Ellie Grace", "screen_name": "elliegdickinson", "lang": "en", "location": "Manchester", "create_at": date("2011-02-26"), "description": "I spend my free time baking and blogging. Style, beauty and lifestyle blogger. UK Blog Award finalist. lifeofellieg@gmail.com | http://youtube.com/elliechatters", "followers_count": 2185, "friends_count": 479, "statues_count": 16749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-29T00:00:04.000Z"), "id": 725942713745440769, "text": "My study group is my lifeline ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67775647, "name": "G A B B Y", "screen_name": "GabriHellaGood", "lang": "en", "location": "Miami, FL", "create_at": date("2009-08-21"), "description": "a lover of the wild", "followers_count": 139, "friends_count": 154, "statues_count": 6765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942713879678979, "text": "@vogueprinsus take me out smh", "in_reply_to_status": 725942490285527040, "in_reply_to_user": 2581142089, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2581142089 }}, "user": { "id": 1925736288, "name": "oscar", "screen_name": "oscarsanchez___", "lang": "en", "location": "null", "create_at": date("2013-10-01"), "description": "null", "followers_count": 596, "friends_count": 591, "statues_count": 13358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942713934192640, "text": "@BrayanGee_ ���� smh", "in_reply_to_status": 725937572493455360, "in_reply_to_user": 3812257272, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3812257272 }}, "user": { "id": 3038974424, "name": "Madalyn", "screen_name": "maddi3_1", "lang": "en", "location": "with J Bug ", "create_at": date("2015-02-23"), "description": "null", "followers_count": 198, "friends_count": 145, "statues_count": 6752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714148118528, "text": "@keithxo_ just look at his hand lmao he made sure his hand started off the same way in each vine ������", "in_reply_to_status": 725942252917248000, "in_reply_to_user": 71334442, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71334442 }}, "user": { "id": 71334442, "name": "Keith", "screen_name": "keithxo_", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-09-03"), "description": "It's easy #3am", "followers_count": 3754, "friends_count": 1037, "statues_count": 92957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714190036994, "text": "Being farsighted is a struggle, I need glasses to see anything up close ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 267263309, "name": "papa frita", "screen_name": "anapmartz", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-03-16"), "description": "not your honey", "followers_count": 1071, "friends_count": 282, "statues_count": 44209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714282303489, "text": "How to deal with late hairdressers https://t.co/3I3wBYJBvW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322443544, "name": "Delforge + Co", "screen_name": "delforgeandco", "lang": "en", "location": "London", "create_at": date("2011-06-22"), "description": "Consultancy agency and Business Academy specialising in the Hair and Beauty industry", "followers_count": 313, "friends_count": 282, "statues_count": 348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714315886592, "text": "forcing myself back to sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4675399429, "name": "realmiaj", "screen_name": "lightthesplifff", "lang": "en", "location": "Houston, TX", "create_at": date("2015-12-29"), "description": "Freelance Model | ShesHappyHair BA | RIPPatchy", "followers_count": 785, "friends_count": 382, "statues_count": 3961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714315894785, "text": "New post: H&M Conscious Beauty... to Buy or Not to Buy? (Plus a Mini Review) https://t.co/KvAeQSmmZa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2203456731, "name": "katie", "screen_name": "KatieVibes", "lang": "en-gb", "location": "UK", "create_at": date("2013-12-01"), "description": "I write about books and vegan lipstick. Green beauty blogger. Entertainment Editor for @TridentMediaUK. English Lit & German student.", "followers_count": 826, "friends_count": 711, "statues_count": 6026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714382995456, "text": "Wind 4.0 mph NW. Barometer 30.064 in, Rising slowly. Temperature 46.8 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714412326914, "text": "Booked a trip to Florida. ab to be lit af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262925405, "name": "ㅤ", "screen_name": "ThisIsChasenn", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2011-03-08"), "description": "Meteorology Major @ WKU. #ColeWorld", "followers_count": 699, "friends_count": 200, "statues_count": 36777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714416566272, "text": "Insomniak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 2410838548, "name": "NoLimit Ty™", "screen_name": "VSBasedGod", "lang": "en", "location": "in the smoke chambers", "create_at": date("2014-03-14"), "description": "Big Dog. Secure your bag. straight cruddy IG:nolimit_roni", "followers_count": 1117, "friends_count": 543, "statues_count": 3921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942714601103362, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Falling slowly. Temperature 68.9 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942715339296768, "text": "Too Good https://t.co/NRP3DnqC26", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3223806835, "name": "Ashley✨", "screen_name": "ashserranomfg", "lang": "en", "location": "some place eating food", "create_at": date("2015-05-22"), "description": "RHS c/o '17 || VARSITY SONG ||⭐12.12.15 ❤", "followers_count": 170, "friends_count": 149, "statues_count": 2624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942715389599746, "text": "���� Pls go away.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.891475,32.995301"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326027612, "name": "Alexadrienne", "screen_name": "HoneyysLove", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-29"), "description": "null", "followers_count": 1849, "friends_count": 32, "statues_count": 196086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942715557384192, "text": "everyone on Twitter when I post a dam figure drawing that I put hours of hard work into https://t.co/9RrQzGmUMd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 954466460, "name": "mayonnaise dad", "screen_name": "_actuallyryan", "lang": "en", "location": "yeah yeah utah ", "create_at": date("2012-11-17"), "description": "average millennial trash • i make art", "followers_count": 398, "friends_count": 286, "statues_count": 19785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ephraim, UT", "id": "c6ede433427b8ece", "name": "Ephraim", "place_type": "city", "bounding_box": rectangle("-111.598906,39.338293 -111.559259,39.372886") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49039, "countyName": "Sanpete", "cityID": 4923530, "cityName": "Ephraim" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942715574194176, "text": "wish i had a blunt rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 164418498, "name": "©", "screen_name": "curteria_", "lang": "en", "location": "htx", "create_at": date("2010-07-08"), "description": "sc & insta: curteriaaa", "followers_count": 1210, "friends_count": 1034, "statues_count": 23860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942716102676481, "text": "Drip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 503919096, "name": "honey dip", "screen_name": "PrettyAleese", "lang": "en", "location": "DFW ✈️ SA ", "create_at": date("2012-02-25"), "description": "Legit could watch #ADifferentWorld and #Martin and listen to Badu everyday for the rest of my life #RIPCARL #UTSA19", "followers_count": 1223, "friends_count": 880, "statues_count": 47097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-29T00:00:05.000Z"), "id": 725942716131999744, "text": "@KurlyHairKillah stop unfollowing me lol", "in_reply_to_status": -1, "in_reply_to_user": 195884129, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 195884129 }}, "user": { "id": 1007948773, "name": "Corey M Thomas", "screen_name": "WifiAndPorn", "lang": "en", "location": "Chicago, IL", "create_at": date("2012-12-12"), "description": "No Ratchets Please....free agent in #DLeagueTwitter", "followers_count": 1658, "friends_count": 952, "statues_count": 138608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, Chicago", "id": "49e4d269b7206bc4", "name": "Austin", "place_type": "neighborhood", "bounding_box": rectangle("-87.806105,41.8654 -87.73934,41.923196") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305080828293120, "text": "keep screwing up��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2325852679, "name": "lalexus robertson", "screen_name": "yooitslexxy", "lang": "en", "location": "surprise, AZ", "create_at": date("2014-02-03"), "description": "nates my favorite//#6", "followers_count": 598, "friends_count": 931, "statues_count": 5538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305080845029376, "text": "When I'm alone I think of how much i miss you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 709415772502687744, "name": "Vane.", "screen_name": "VanessaGuajar16", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2016-03-14"), "description": "fw, tx | 20 | Leo", "followers_count": 21, "friends_count": 73, "statues_count": 59 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305080941510656, "text": "ill just call it a night lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1667292492, "name": "g", "screen_name": "misslandin", "lang": "en", "location": "null", "create_at": date("2013-08-13"), "description": "views", "followers_count": 1177, "friends_count": 681, "statues_count": 36826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305081038090240, "text": "Yay for thinking too much...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274731462, "name": "Ben Miller", "screen_name": "BMiller_57", "lang": "en", "location": "Wooster, OH", "create_at": date("2011-03-30"), "description": "What's life if we don't go out there and live it?Junior at THE Ohio State University. Animal Science", "followers_count": 611, "friends_count": 764, "statues_count": 14265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305081172365313, "text": "rivs fuckin around with my windows in my car, like it's cold ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437053317, "name": "honey haze", "screen_name": "Marie_Rissa", "lang": "en", "location": "the 330", "create_at": date("2011-12-14"), "description": "•LorASSa•UA'18•snap chat ya girl l_scott15 ◡̈⃝ im probably the pickiest person you'll ever meet", "followers_count": 653, "friends_count": 1149, "statues_count": 20707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305081201577984, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2341467391, "name": "M", "screen_name": "casas_melody", "lang": "en", "location": "null", "create_at": date("2014-02-12"), "description": "sucks to suck", "followers_count": 345, "friends_count": 392, "statues_count": 9422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Fernando, CA", "id": "aa755c5c1b8e341b", "name": "San Fernando", "place_type": "city", "bounding_box": rectangle("-118.456347,34.273334 -118.41567,34.304639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666140, "cityName": "San Fernando" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305081377710086, "text": "WHAT YHE FUCK IM LIVID", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2451842719, "name": "indy", "screen_name": "indyisla", "lang": "en", "location": "null", "create_at": date("2014-04-18"), "description": "null", "followers_count": 90, "friends_count": 215, "statues_count": 363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305081641951232, "text": "Can you recommend anyone for this #Retail #job? https://t.co/kYcFnXVwbE #AllenPark, MI #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2142854,42.2782999"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "AllenPark", "Hiring" }}, "user": { "id": 2362771358, "name": "Vitamin Shoppe Jobs", "screen_name": "VSIcareers", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "At #VitaminShoppe we are lovers of fitness, health, vitamins, helping you on your wellness journey and motivating others. Apply to our #jobs below.", "followers_count": 600, "friends_count": 96, "statues_count": 496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen Park, MI", "id": "0c937cb917334546", "name": "Allen Park", "place_type": "city", "bounding_box": rectangle("-83.231897,42.222765 -83.183819,42.301134") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2601380, "cityName": "Allen Park" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305082229190656, "text": "/not/ to but the only thing we can think of anymore is \"well carpe's gonna rip out our heart too because that's all that happens anymore\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 79, "friends_count": 219, "statues_count": 125878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305082346708992, "text": "My eyes so low ���� https://t.co/MLyXFVwfpy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3259242019, "name": ": A i", "screen_name": "niahlong", "lang": "en", "location": "where the cash at", "create_at": date("2015-06-28"), "description": "i had to grind like that to shine like this ;", "followers_count": 345, "friends_count": 172, "statues_count": 7998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305082451595265, "text": "Had fun w/ gabby but that's always ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354456111, "name": "BigBOOTYJudy", "screen_name": "imDOPE_GetHigh", "lang": "en", "location": "somewhere gettin money", "create_at": date("2011-08-13"), "description": "GET OUT YA FEELINGS YOUNG NIGGAH LETS GET IT | Money over Everything | I COOK and I do hair | sc: queenlachiefa21", "followers_count": 3185, "friends_count": 2123, "statues_count": 69923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phenix City, AL", "id": "66c034beedd48294", "name": "Phenix City", "place_type": "city", "bounding_box": rectangle("-85.06093,32.345401 -84.9918,32.531343") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1113, "countyName": "Russell", "cityID": 159472, "cityName": "Phenix City" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305082510217217, "text": "@gerichu just like Gaim x ToQger, and Driver x Ninninger", "in_reply_to_status": 726304466178203648, "in_reply_to_user": 2463718297, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2463718297 }}, "user": { "id": 59017382, "name": "Greg", "screen_name": "GregCayabyab", "lang": "en", "location": "Vallejo, CA", "create_at": date("2009-07-21"), "description": "null", "followers_count": 141, "friends_count": 381, "statues_count": 3482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305082610847744, "text": "@ebbtideapp Tide in Cathlamet, Washington 04/30/2016\n Low 4:08am 0.0\nHigh 8:46am 0.0\n Low 4:48pm 0.0\nHigh 10:38pm 0.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-123.3833,46.2"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 81, "friends_count": 1, "statues_count": 28743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53069, "countyName": "Wahkiakum", "cityID": 5356555, "cityName": "Puget Island" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083286233088, "text": "@Lucan_Sanchez I agree, fuck you Lucan ��", "in_reply_to_status": 726304971361243136, "in_reply_to_user": 3312272416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2364996589 }}, "user": { "id": 3312272416, "name": "Jeffrey Lupia", "screen_name": "JeffreyLupia", "lang": "en", "location": "null", "create_at": date("2015-06-07"), "description": "Jeffrey, Son of Joseph. Second son of the house Lupia", "followers_count": 102, "friends_count": 127, "statues_count": 40 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, NY", "id": "b1d34909d9f1a53e", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-73.939455,41.714502 -73.892094,41.749147") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3625120, "cityName": "Fairview" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083336495104, "text": "\"You look like the white kid who goes to black parties\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289542120, "name": "Rextopher", "screen_name": "rexeastepp", "lang": "en", "location": "Texas, USA", "create_at": date("2011-04-28"), "description": "null", "followers_count": 866, "friends_count": 544, "statues_count": 55964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083437113346, "text": "@JuanersRivera noodle head.", "in_reply_to_status": 726197736597389312, "in_reply_to_user": 434959290, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 434959290 }}, "user": { "id": 933687872, "name": "Vanilla Porter", "screen_name": "Trap_Potato17", "lang": "en", "location": "Baldwin City, KS", "create_at": date("2012-11-07"), "description": "PB&G", "followers_count": 275, "friends_count": 244, "statues_count": 1624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin City, KS", "id": "a071f54fdfe438f2", "name": "Baldwin City", "place_type": "city", "bounding_box": rectangle("-95.207162,38.765992 -95.165813,38.796737") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20045, "countyName": "Douglas", "cityID": 2003900, "cityName": "Baldwin City" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083533619200, "text": "What is life without my bestfriends honestly������☺️❤️ https://t.co/DN3NPiR3iX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4695706812, "name": "donna flores", "screen_name": "donnaflores39", "lang": "en", "location": "null", "create_at": date("2016-01-02"), "description": "null", "followers_count": 35, "friends_count": 51, "statues_count": 95 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennewick, WA", "id": "63d0671506628c8d", "name": "Kennewick", "place_type": "city", "bounding_box": rectangle("-119.264818,46.162262 -119.087063,46.236321") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5335275, "cityName": "Kennewick" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083848187904, "text": "@SamxAllStar ok wait really. YOURE SO LUCKY. I was wanting this black floral dress but it was still $121", "in_reply_to_status": 726257872955445248, "in_reply_to_user": 35312053, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35312053 }}, "user": { "id": 98199951, "name": "Acacia Brinley", "screen_name": "AcaciaBrinley", "lang": "en", "location": "null", "create_at": date("2009-12-20"), "description": "Trying to make my life and others a million times better, one day at a time. SNAPCHAT: acaciasnappy", "followers_count": 712402, "friends_count": 1219, "statues_count": 48242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305083940569088, "text": "\"How's that burger? Worth every calorie? Bet your bald ass it is.\"-Kathryn to some bald guy next to us in traffic.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48578669, "name": "Eddie Oldham", "screen_name": "Eazy_E1992", "lang": "en", "location": "Quincy,MA", "create_at": date("2009-06-18"), "description": "null", "followers_count": 317, "friends_count": 450, "statues_count": 16018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305084032860160, "text": "Wind 0.8 mph WNW. Barometer 29.744 in, Falling. Temperature 74.0 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305084112523264, "text": "@coolyounglou than that stay healthy. After Perriman's bs last year they needed a guy who can play today", "in_reply_to_status": 726304471752515584, "in_reply_to_user": 30987598, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30987598 }}, "user": { "id": 17412990, "name": "Andrew Maas", "screen_name": "AMAAS", "lang": "en", "location": "null", "create_at": date("2008-11-15"), "description": "I tweet about sports, life, tits, chicken & fat chicks. I rant a lot & crack jokes. Follow me if you like rollercoasters.", "followers_count": 3164, "friends_count": 2150, "statues_count": 405823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edwardsville, IL", "id": "bd480515dd04af98", "name": "Edwardsville", "place_type": "city", "bounding_box": rectangle("-90.038697,38.761368 -89.906547,38.850738") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1722697, "cityName": "Edwardsville" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305084229869568, "text": "Fuck man, I keep taking L's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 969153007, "name": "Bricks", "screen_name": "13Dickinson", "lang": "en", "location": "null", "create_at": date("2012-11-24"), "description": "null", "followers_count": 935, "friends_count": 346, "statues_count": 38781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parsons, KS", "id": "0a3e119020705b64", "name": "Parsons", "place_type": "city", "bounding_box": rectangle("-95.297025,37.321136 -95.2296,37.364043") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20099, "countyName": "Labette", "cityID": 2054675, "cityName": "Parsons" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305084422787072, "text": "*tweets something intellectual*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1472350436, "name": "Amelia Byrd", "screen_name": "AbyrdCom", "lang": "en", "location": "null", "create_at": date("2013-05-31"), "description": "null", "followers_count": 339, "friends_count": 319, "statues_count": 679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-04-30T00:00:00.000Z"), "id": 726305084687060992, "text": "@saucy_slimmm lol to follow rule #2", "in_reply_to_status": 726304710253105152, "in_reply_to_user": 531809692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 531809692 }}, "user": { "id": 130613835, "name": "Mike Rager", "screen_name": "MikeRagerxX", "lang": "en", "location": "Mo City, SFA", "create_at": date("2010-04-07"), "description": "HCS | 90.1 DJ for KSAU | Saturday & Sunday 6-8pm | The Addict. The North Face. General inquiries: highclvssbookings@Gmail.com", "followers_count": 4544, "friends_count": 3165, "statues_count": 63565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305085261668352, "text": "We live in a generation of not being in love, & not being together", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3325985989, "name": "Marissa!", "screen_name": "Reana__mari", "lang": "en", "location": "Mathis, TX", "create_at": date("2015-08-23"), "description": "•Happiness", "followers_count": 224, "friends_count": 275, "statues_count": 1459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mathis, TX", "id": "23ce8ba2308c3f30", "name": "Mathis", "place_type": "city", "bounding_box": rectangle("-97.839067,28.077909 -97.80972,28.112871") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4847040, "cityName": "Mathis" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305085647540225, "text": "Hux is so whinny I can't deal with this right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552908811, "name": "Anakin", "screen_name": "annabarnesyo", "lang": "en", "location": "null", "create_at": date("2012-04-13"), "description": "official pawnee goddess", "followers_count": 414, "friends_count": 358, "statues_count": 7954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Provo, UT", "id": "665409336d2b20a0", "name": "Provo", "place_type": "city", "bounding_box": rectangle("-111.740949,40.18565 -111.602629,40.311641") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4962470, "cityName": "Provo" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305085823709184, "text": "@NotBrandonWint depends on the drug in question, and how frequently said friend uses.", "in_reply_to_status": 726300112306114561, "in_reply_to_user": 71446121, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71446121 }}, "user": { "id": 567826466, "name": "QueenBrit", "screen_name": "ayyeitsbrit", "lang": "en", "location": "YNHSBN", "create_at": date("2012-04-30"), "description": "•Faith is the banishment of all doubt• '94s cutest baby• Staying Prayed Up• ISYN", "followers_count": 253, "friends_count": 189, "statues_count": 14920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086222139392, "text": "@jordanjfrasier this makes me want to die", "in_reply_to_status": 726280092322811904, "in_reply_to_user": 15606793, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15606793 }}, "user": { "id": 22313347, "name": "Jose A. DelReal", "screen_name": "jdelreal", "lang": "en", "location": "Washington, D.C.", "create_at": date("2009-02-28"), "description": "2016 campaign reporter for @WashingtonPost, following Donald Trump. Alaska grown. More coffee, please. jose.delreal@washpost.com || https://t.co/KNq6E2Muon", "followers_count": 5595, "friends_count": 2396, "statues_count": 14329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086352166913, "text": "Wind 6.0 mph SE. Barometer 1005.79 mb, Falling Rapidly. Temperature 62.9 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086519939072, "text": "@courtneeyshaye cat and bong smoke date soon you live in the inlet right ?", "in_reply_to_status": 726239573320302592, "in_reply_to_user": 61323993, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61323993 }}, "user": { "id": 299438033, "name": "samanta green", "screen_name": "Samanthaa_Gee", "lang": "en", "location": "Murrells Inlet, SC", "create_at": date("2011-05-15"), "description": "idc if the glass is half empty or half full, just as long as it's half wine", "followers_count": 862, "friends_count": 602, "statues_count": 23563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrells Inlet, SC", "id": "f5d9845d07a82c46", "name": "Murrells Inlet", "place_type": "city", "bounding_box": rectangle("-79.087697,33.523442 -79.016701,33.572178") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45043, "countyName": "Georgetown", "cityID": 4548985, "cityName": "Murrells Inlet" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086520066048, "text": "Drunk as shid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 461742692, "name": "Wavy Baby", "screen_name": "DoubleOSmoove", "lang": "en", "location": "The Ocean", "create_at": date("2012-01-11"), "description": "Artist. Business Inquiries: phiriant000@icloud.com", "followers_count": 1196, "friends_count": 778, "statues_count": 68184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086608015360, "text": "@luar_bago *Saca su encuesta y las técnicas cualitativas* ❤️��", "in_reply_to_status": 726304396439490561, "in_reply_to_user": 122408226, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 122408226 }}, "user": { "id": 119625190, "name": "Melissa Crespo", "screen_name": "crespomelissa", "lang": "es", "location": "Tijuana B.C. México", "create_at": date("2010-03-03"), "description": "Mercadóloga Tijuanense amante de la política y los deportes. Colaboro en @YQUMM @Xipalmarketing @Marcoirizar y @AytoTijuana. |080115|240116| A Título Personal", "followers_count": 1293, "friends_count": 772, "statues_count": 41122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086641561600, "text": "Swear if a bitch calls me with no caller ID again I'm gonna flip shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1622160080, "name": "Eric Lucero", "screen_name": "Ericlucero_4", "lang": "en", "location": "null", "create_at": date("2013-07-25"), "description": "God Is King", "followers_count": 231, "friends_count": 245, "statues_count": 1115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305086868066306, "text": "Going to Disney tomorrow for bats day!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407468794, "name": "Tilly", "screen_name": "wh0istilly", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2011-11-07"), "description": "I like aesthetic and indie art. Photography is my passion. I can make you laugh, kinda. Harry Potter is my obsession. Gryffindor. Jp has my heart.", "followers_count": 2119, "friends_count": 2537, "statues_count": 7270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305087321157632, "text": "Wind 1.0 mph ESE. Barometer 29.888 in, Falling. Temperature 57.9 °F. Rain today 0.29 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305087426035713, "text": "@mc_magic1887 ������ lol", "in_reply_to_status": 726304927396417536, "in_reply_to_user": 142758200, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 142758200 }}, "user": { "id": 172815511, "name": "Binge Watch Shawty", "screen_name": "4thQtr_Ninja", "lang": "en", "location": "Land of milk & honey", "create_at": date("2010-07-30"), "description": "Don't be naive all your life. Brainwashing is real.", "followers_count": 3926, "friends_count": 1168, "statues_count": 133731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305087543463936, "text": "Ripley SW Limestone Co. Temp: 61.5°F Wind:1.6mph Pressure: 991.5mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305087992172544, "text": "@beautfilms @Deepsext @Mads7501 @sammy_demay", "in_reply_to_status": 696132233829089280, "in_reply_to_user": 3319545392, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3319545392, 3051357597, 1591766862, 1735518534 }}, "user": { "id": 4275892218, "name": "Baneen Somji", "screen_name": "Baneen165", "lang": "en", "location": "Irvine Ca", "create_at": date("2015-11-17"), "description": "null", "followers_count": 117, "friends_count": 332, "statues_count": 894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305088160043008, "text": "Seems like a great idea", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319263889, "name": "Nick Fabian", "screen_name": "nfabz", "lang": "en", "location": "Sound Beach, NY", "create_at": date("2011-06-17"), "description": "6ix", "followers_count": 377, "friends_count": 405, "statues_count": 14607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sound Beach, NY", "id": "6ca98cf8235c4ef4", "name": "Sound Beach", "place_type": "city", "bounding_box": rectangle("-72.98701,40.944899 -72.941493,40.966781") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3668374, "cityName": "Sound Beach" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305088495489024, "text": "I get a bloody nose every damn day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1636393933, "name": "JISH", "screen_name": "Josh_Cantos", "lang": "en", "location": "Wherever the crowd is...", "create_at": date("2013-07-31"), "description": "Hebrews 13:5... #relentless #ASB #TLOP #GettingOutOurDreams #ascension #DONDA #DRIPP", "followers_count": 459, "friends_count": 418, "statues_count": 11568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305088915017728, "text": "I like this Judge!! What a horrible pair of devils these 2 are. Dispica https://t.co/mQ9lrB5GbN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1062225217, "name": "Danna Wells", "screen_name": "dannalynette", "lang": "en", "location": "SavannahGA•FtLaudFL•LexVilleKY", "create_at": date("2013-01-04"), "description": "•Southern #Vegan Democrat•\nRIP@Bellesbattle I❤•myPets•Animals•Family•@UKAthletics & @SEC FTBL! \n#BBN #Gilbert23\n#recycle♻\n▶Be like @CoryBooker & @LeilaniMunter◀", "followers_count": 4117, "friends_count": 4287, "statues_count": 28767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305088961040384, "text": "Puta el gordo me tiene chata con su weaita��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 122714389, "name": "Isi leon", "screen_name": "isiileon", "lang": "es", "location": "null", "create_at": date("2010-03-13"), "description": "Un ojo abierto, el otro soñando.. Canto que ha sido valiente, siempre sera cancion nuevas.. En la diversidad esta la perfección ❤️ || insta: isileon", "followers_count": 866, "friends_count": 684, "statues_count": 14738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest Park, WA", "id": "c3f0bca4262b5ef8", "name": "Lake Forest Park", "place_type": "city", "bounding_box": rectangle("-122.31356,47.732932 -122.263384,47.777413") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5337270, "cityName": "Lake Forest Park" } }
+{ "create_at": datetime("2016-04-30T00:00:01.000Z"), "id": 726305089045053440, "text": "I guess \"my name's Zac Underwood so get under my wood,\" isn't the best pickup line.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1877290220, "name": "Zac Underwood", "screen_name": "yaboyzac_", "lang": "en", "location": "MS / KY", "create_at": date("2013-09-17"), "description": "Memphis", "followers_count": 681, "friends_count": 471, "statues_count": 21011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paducah, KY", "id": "716a188f14ca331e", "name": "Paducah", "place_type": "city", "bounding_box": rectangle("-88.72501,37.039623 -88.561066,37.109891") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21145, "countyName": "McCracken", "cityID": 2158836, "cityName": "Paducah" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305089221177346, "text": "04/30@03:00 - Temp 46.7F, WC 46.7F. Wind 1.0mph NNE, Gust 2.0mph. Bar 30.160in, Steady. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305089971965952, "text": "Lol grandma forreal came out to play today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331261619, "name": "~Ingrid~", "screen_name": "isabelleenguyen", "lang": "en", "location": "San Jose || Isla Vista", "create_at": date("2011-07-07"), "description": "Audentes fortuna iuvat", "followers_count": 381, "friends_count": 412, "statues_count": 33610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305090017992704, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2556686080, "name": "jolly g", "screen_name": "bonura_gabby", "lang": "en", "location": "w/ kris or fefe", "create_at": date("2014-05-20"), "description": "r u cool enough to kick it?", "followers_count": 580, "friends_count": 397, "statues_count": 16657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305090022199297, "text": "@TaliuWilliams love ya too boo", "in_reply_to_status": 726302160653160452, "in_reply_to_user": 1716733928, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1716733928 }}, "user": { "id": 3196981992, "name": "Tatiana Makamae", "screen_name": "TatianaMakamae", "lang": "en-US", "location": "Honolulu, HI", "create_at": date("2015-05-15"), "description": "Taken by the best", "followers_count": 79, "friends_count": 152, "statues_count": 629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burien, WA", "id": "531b8f14eaf6c48e", "name": "Burien", "place_type": "city", "bounding_box": rectangle("-122.382962,47.430814 -122.311608,47.499304") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5308850, "cityName": "Burien" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305090026397696, "text": "it's 2 I need sleep rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2911452042, "name": "// Micah //", "screen_name": "1975micah", "lang": "en", "location": "Bryant, AR", "create_at": date("2014-11-26"), "description": "1/29/16", "followers_count": 825, "friends_count": 1116, "statues_count": 6049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benton, AR", "id": "e148c35b55163567", "name": "Benton", "place_type": "city", "bounding_box": rectangle("-92.651513,34.53414 -92.514669,34.650234") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline", "cityID": 505290, "cityName": "Benton" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305090064277504, "text": "Temp: 72.1°F Wind:1.0mph Pressure: 29.826hpa Falling Rapidly Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305090072502273, "text": "I like to keep our relationship interesting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25750758, "name": "ALAINE", "screen_name": "alainekristina", "lang": "en", "location": "LA", "create_at": date("2009-03-21"), "description": "Fashion & Lifestyle Blogger | Instagram:@thefashionbum | Snapchat: @alainekristina", "followers_count": 1947, "friends_count": 302, "statues_count": 13717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305091079163904, "text": "I swear I order some tacos at jacks& all you hear Frank saying is \"dam you know what I'm feeling some tacos too\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3552981434, "name": "Byran", "screen_name": "byranncito", "lang": "en", "location": "null", "create_at": date("2015-09-13"), "description": "LA living #DNC", "followers_count": 321, "friends_count": 308, "statues_count": 4129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell, CA", "id": "2c1b78093786a74a", "name": "Bell", "place_type": "city", "bounding_box": rectangle("-118.206291,33.95407 -118.152019,33.998359") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604870, "cityName": "Bell" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305091234340866, "text": "Honestly @natalieeochoa and @Thyalexismurder have been my friends longer than anyone ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1910189965, 704875003641106432 }}, "user": { "id": 707023406810574848, "name": "(יהודון) Jewboy✨", "screen_name": "Jewz1416", "lang": "en", "location": "Hawaii, USA", "create_at": date("2016-03-07"), "description": "Tucson AZ est99\nTEAM BACKWOOD\nSNAPCHAT:twdp1416\n\n\n~Shop with me im the plug", "followers_count": 142, "friends_count": 139, "statues_count": 2669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaiian Paradise Park, HI", "id": "1ff3790a528c2f97", "name": "Hawaiian Paradise Park", "place_type": "city", "bounding_box": rectangle("-155.024136,19.539572 -154.931512,19.619759") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii", "cityID": 1512600, "cityName": "Hawaiian Paradise Park" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305091691618304, "text": "Wind 1.0 mph E. Barometer 30.004 in, Steady. Temperature 59.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305091951595520, "text": "Today was our rest day lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2541207122, "name": "steph", "screen_name": "StephCaceres_", "lang": "en", "location": "3/11", "create_at": date("2014-06-02"), "description": "i only have 5 boys in my life my dad, my brothers and GOD", "followers_count": 291, "friends_count": 258, "statues_count": 5123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waialua, HI", "id": "7d69bb88fa2b2694", "name": "Waialua", "place_type": "city", "bounding_box": rectangle("-158.135962,21.554227 -158.101696,21.584665") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1574000, "cityName": "Waialua" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092131934208, "text": "@kiani_franco try harder ��", "in_reply_to_status": 726304774761512960, "in_reply_to_user": 3007296920, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3007296920 }}, "user": { "id": 2265672703, "name": "The whip", "screen_name": "BennettTamarcus", "lang": "en", "location": "Texas, USA", "create_at": date("2013-12-28"), "description": "If you send me a one worded text I will not reply...be interesting it's not hard #bowlme sc:yo-boi1", "followers_count": 1181, "friends_count": 1716, "statues_count": 16838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richardson, TX", "id": "bc7f3267d2efaf40", "name": "Richardson", "place_type": "city", "bounding_box": rectangle("-96.769003,32.923164 -96.612871,33.005805") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4861796, "cityName": "Richardson" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092316614656, "text": "Fuck �� I hate missing work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62706317, "name": "Jacob Routzong", "screen_name": "JacobRoutzong", "lang": "en", "location": "Eufaula, AL", "create_at": date("2009-08-03"), "description": "null", "followers_count": 314, "friends_count": 81, "statues_count": 2381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eufaula, AL", "id": "4777cdef36751c25", "name": "Eufaula", "place_type": "city", "bounding_box": rectangle("-85.180052,31.809606 -85.122095,31.922976") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1005, "countyName": "Barbour", "cityID": 124568, "cityName": "Eufaula" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092488560640, "text": "1 Black-capped Gnatcatcher (Polioptila nigriceps) - Florida Canyon--lower - 2016-04-28 08:30 https://t.co/h3cLVQrDPq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14541 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092576657408, "text": "Wind 0.7 mph NE. Barometer 30.16 in, Steady. Temperature 47.5 °F. Rain today 0.00 in. Humidity 60%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092744302592, "text": "@AustinKGodfrey I'm tired can I answer later", "in_reply_to_status": 726305029645172736, "in_reply_to_user": 259130048, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259130048 }}, "user": { "id": 2521680848, "name": "Moe-Jo Jojo", "screen_name": "moeeediggity", "lang": "en", "location": "WTWTA", "create_at": date("2014-05-24"), "description": "Moe . Nevada '19 . Loud-Mouth . Youtuber. Crazy . rest in paradise Dom ✨", "followers_count": 814, "friends_count": 386, "statues_count": 15608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305092928868354, "text": "73.8F (Feels: 73.8F) - Humidity: 92% - Wind: 14.5mph E - Gust: 16.8mph - Pressure: 1001.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305093188902913, "text": "Lol they gave Brady the same shot I got on my birthday and he's sauced ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 606174693, "name": "Mack Weezy", "screen_name": "Sub_Mack", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "Everybody's somebody's everything.", "followers_count": 431, "friends_count": 213, "statues_count": 16201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troutdale, OR", "id": "e0e015b833cb12b4", "name": "Troutdale", "place_type": "city", "bounding_box": rectangle("-122.431645,45.506422 -122.368489,45.561366") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4174850, "cityName": "Troutdale" } }
+{ "create_at": datetime("2016-04-30T00:00:02.000Z"), "id": 726305093226618881, "text": "@gllennn_ you got drugs", "in_reply_to_status": 726304804167815168, "in_reply_to_user": 1167180224, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1167180224 }}, "user": { "id": 2429804833, "name": "E♕", "screen_name": "elizzzabettth", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": "✨✨✨✨✨✨✨", "followers_count": 487, "friends_count": 171, "statues_count": 10288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305094057123840, "text": "���� https://t.co/brqMa7QwNl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 717552331, "name": "Manny Cruzzz", "screen_name": "SuaveCruz_", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "This is the cubs year", "followers_count": 495, "friends_count": 550, "statues_count": 18671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305094132596736, "text": "I need more BakuDeku fics in my life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2256885006, "name": "Wandering Wynn", "screen_name": "nanodayocchi", "lang": "en", "location": "Arizona, USA", "create_at": date("2013-12-21"), "description": "♀ ★ 17 ★ anime/manga lover ★ drawer/writer ★ aspiring director/screenwriter ★ too many husbandos ★ best kouhai ★ ☁ junior year blues ☁", "followers_count": 283, "friends_count": 944, "statues_count": 8935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Park, AZ", "id": "00e7b70e6cf7f64e", "name": "Citrus Park", "place_type": "city", "bounding_box": rectangle("-112.461589,33.508162 -112.426794,33.580736") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 413820, "cityName": "Citrus Park" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305094753484802, "text": "Temp 40.1° Hi/Lo 43.3/39.9 Rng 3.4° WC 40.1° Hmd 81% Rain 0.00\" Storm 0.00\" BAR 30.184 Rising DP 34.7° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305094799646721, "text": "There is nothing like holding your daughters tiny hand. #haydenwinter #love #mybaby #daughter… https://t.co/sfMmO2k0J6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.4861406,34.027401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "haydenwinter", "love", "mybaby", "daughter" }}, "user": { "id": 44428182, "name": "Betheny Zolt", "screen_name": "BethenyZ", "lang": "en", "location": "Venice, CA", "create_at": date("2009-06-03"), "description": "Voice Over Artist, Actress, Host, Eco Diva, Facilitator, and all around cool chick!", "followers_count": 305, "friends_count": 246, "statues_count": 402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305095202263040, "text": "Shots shots shots https://t.co/f041srpkG7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 808051033, "name": "Lo ✖️⭕️", "screen_name": "Lauren_Eggers", "lang": "en", "location": "Ohio, USA", "create_at": date("2012-09-06"), "description": "1738 AYEE #ripdaddy 1/25/13 #foreverinmyheart Criminal Justice . Ohio university/ future bobcat Single Life", "followers_count": 756, "friends_count": 2053, "statues_count": 14164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305095223152640, "text": "Wind 0.3 mph WNW. Barometer 30.167 in, Steady. Temperature 32.8 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305096410103811, "text": "Wow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4896724304, "name": "Chloe Rain", "screen_name": "ccchloeerainnn", "lang": "en", "location": "California", "create_at": date("2016-02-12"), "description": "I feel sick.", "followers_count": 237, "friends_count": 217, "statues_count": 1261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305096418603008, "text": "It's 3 a.m. but I'm too happy to fall asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1902667945, "name": "Lucas Smith", "screen_name": "lucassmitty5", "lang": "en", "location": "Miami, FL", "create_at": date("2013-09-24"), "description": "•The One And Only•", "followers_count": 500, "friends_count": 369, "statues_count": 3371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollwood, FL", "id": "5bf268a34de4d516", "name": "Carrollwood", "place_type": "city", "bounding_box": rectangle("-82.543989,28.033407 -82.479582,28.084798") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1210825, "cityName": "Carrollwood" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305096699506692, "text": "Happy Birthday to my boy @balvarez226 �� a.k.a pitohead #2 �� I love you bro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1669453988 }}, "user": { "id": 2590864188, "name": "Anthony", "screen_name": "AMG11__", "lang": "en", "location": "null", "create_at": date("2014-06-26"), "description": "Soccer's my passion ⚽️", "followers_count": 226, "friends_count": 288, "statues_count": 995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305096934522880, "text": "@EatHumans ��", "in_reply_to_status": 726302649906266112, "in_reply_to_user": 232384858, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 232384858 }}, "user": { "id": 79754671, "name": "S V T", "screen_name": "shanethacreator", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-10-04"), "description": "I got these strange voices inside my head. Keep a stack and .380 beside my bed.", "followers_count": 502, "friends_count": 240, "statues_count": 22688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305096972128256, "text": "HAPPY BIRTHDAY LA FLAME BIRDS IN THE TRAP SING MCKNIGHT ���� @trvisXX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 135019364 }}, "user": { "id": 2441242561, "name": "Lawn", "screen_name": "Londagod_", "lang": "en", "location": "Playboy Mansion ", "create_at": date("2014-04-13"), "description": "Horchata over Hoes .", "followers_count": 1059, "friends_count": 914, "statues_count": 37755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305097014079489, "text": "I forgot all about this. https://t.co/nPpp4aaglv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174353781, "name": "Simmons", "screen_name": "_MickeyDs", "lang": "en", "location": "Inglewood, CA", "create_at": date("2010-08-03"), "description": "null", "followers_count": 60, "friends_count": 47, "statues_count": 91 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:03.000Z"), "id": 726305097433542656, "text": "@symeo @MopUpReliever this isn’t football season??", "in_reply_to_status": 726304968282509313, "in_reply_to_user": 102012760, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 102012760, 96842488 }}, "user": { "id": 16549664, "name": "djcozmik", "screen_name": "djcozmik", "lang": "en", "location": "Los Angeles, CA / Salem, OR", "create_at": date("2008-10-01"), "description": "Science. Music. Sleep. Repeat.", "followers_count": 157, "friends_count": 77, "statues_count": 22264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palms, Los Angeles", "id": "5b1d6c8e87264039", "name": "Palms", "place_type": "neighborhood", "bounding_box": rectangle("-118.431436,34.011385 -118.387635,34.037107") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305097618055169, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":60.3°F Wind:1.3mph Pressure: 30.00hpa Falling slowly Rain Today 0.00in. Forecast: Unsettled, precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 316606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305097878102017, "text": "@byranncito so true lmao I call pink nobody fuking take the pink ranger or I swear!!! ������ frank wants to be Bryan so bad jk", "in_reply_to_status": 726304778653818880, "in_reply_to_user": 3552981434, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3552981434 }}, "user": { "id": 3510233832, "name": "Wendy Perez ☀️", "screen_name": "Wendy_Pendy", "lang": "en", "location": "Phoenix , AZ", "create_at": date("2015-09-09"), "description": "null", "followers_count": 152, "friends_count": 145, "statues_count": 1029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305097895055360, "text": "@PavelNunez en #Miami @HoycomoAyer https://t.co/TtPU6Ry6aN", "in_reply_to_status": -1, "in_reply_to_user": 149339448, "favorite_count": 0, "coordinate": point("-80.231159,25.765334"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Miami" }}, "user_mentions": {{ 149339448, 142015970 }}, "user": { "id": 16725012, "name": "Mariana Guerrero", "screen_name": "anairamg", "lang": "en", "location": "Miami, FL", "create_at": date("2008-10-13"), "description": "Communications/PR at @SAP. Techie. Photographer. #Millennial. My tweets reflect my thoughts. My motto:...Vision to Communicate!\r\nSoñadora, Luchadora...Guerrero!", "followers_count": 245, "friends_count": 608, "statues_count": 1099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hoy Como Ayer", "id": "07d9e38885c83001", "name": "Hoy Como Ayer", "place_type": "poi", "bounding_box": rectangle("-80.2311591,25.765333899999998 -80.231159,25.765334") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305097899118592, "text": "New vans ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2360265331, "name": "tegan", "screen_name": "teganxs", "lang": "en", "location": "Long Beach, CA", "create_at": date("2014-02-24"), "description": "b-rodddd❤️", "followers_count": 684, "friends_count": 504, "statues_count": 23063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305099627266050, "text": "Yep. Best seafood ever. https://t.co/oezRuXOp9B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 216403303, "name": "Dani", "screen_name": "TheDanielleNoel", "lang": "en", "location": "Guntucky.", "create_at": date("2010-11-16"), "description": "IG: TheDanielleNoel | Recording/Mix Engineer + Boss Lady @1StopStudiosLLC | Artist Management : DanielleNoelMGMT@gmail.com | #DaniEngineering", "followers_count": 1695, "friends_count": 465, "statues_count": 19098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Radcliff, KY", "id": "554fd0e3f306af2e", "name": "Radcliff", "place_type": "city", "bounding_box": rectangle("-86.005464,37.749997 -85.895924,37.864999") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21093, "countyName": "Hardin", "cityID": 2163912, "cityName": "Radcliff" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305099832692737, "text": "Glad my brother went to regionals but this nigga need to come home so we can laugh ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2205778028, "name": "lex", "screen_name": "ChiffonLexus", "lang": "en", "location": "null", "create_at": date("2013-11-20"), "description": "19. r.i.p dlyncia", "followers_count": 729, "friends_count": 576, "statues_count": 26660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100096933889, "text": "Wind 1.6 mph NW. Barometer 29.600 in, Rising slowly. Temperature 66.1 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100134764545, "text": "Saturday 4-30-16 https://t.co/6TBUXv0KY0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1139, "friends_count": 322, "statues_count": 3570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100352884736, "text": "I'm at The Patio in San Diego, Calif https://t.co/tWJaPCjsso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.077292,32.949481"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36723299, "name": "Ann Adamson", "screen_name": "GeekyGal8", "lang": "en", "location": "null", "create_at": date("2009-04-30"), "description": "Family. Food. Travel. And all of Life's other Good Things.", "followers_count": 40, "friends_count": 262, "statues_count": 1750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100373839872, "text": "Camouflage level: Expert https://t.co/yjTHJyog3C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 1432120148, "name": "aLy", "screen_name": "AlyTmz", "lang": "en", "location": "Iowa City / Chicago", "create_at": date("2013-05-15"), "description": "null", "followers_count": 583, "friends_count": 359, "statues_count": 10608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100709253120, "text": "You are ���� off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 333050843, "name": "Shar Berry⚓️", "screen_name": "BerrySharline6", "lang": "en", "location": "null", "create_at": date("2011-07-10"), "description": "Mad Hispanic woman with a big heart but a bigger attitude| IG: amayberry97", "followers_count": 651, "friends_count": 310, "statues_count": 9449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Falls, IA", "id": "8ddaffc6aef377de", "name": "Cedar Falls", "place_type": "city", "bounding_box": rectangle("-92.499974,42.464484 -92.406204,42.567828") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1911755, "cityName": "Cedar Falls" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305100801695745, "text": "I must say that I'm surprised what a good show James Bay put on tonight! I had no idea what to… https://t.co/XoOSsxpbvS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.67623259,45.52331335"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14885741, "name": "Burk Jackson", "screen_name": "burkjackson", "lang": "en", "location": "Portland, OR", "create_at": date("2008-05-23"), "description": "Creator | Curator | Connector", "followers_count": 1550, "friends_count": 1991, "statues_count": 7303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305101493608451, "text": "�� https://t.co/5q9KwNalRa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 783710210, "name": "Stephany", "screen_name": "lawfulsins", "lang": "en", "location": "null", "create_at": date("2012-08-26"), "description": "null", "followers_count": 109, "friends_count": 73, "statues_count": 2391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305101686525952, "text": "she is amazing �� https://t.co/FQpWUVppWi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.179348,36.090562"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54818959, "name": "Danee Jo Leary", "screen_name": "DaneeLeary", "lang": "en", "location": "Butte, MT", "create_at": date("2009-07-07"), "description": "snapchat: daneejo", "followers_count": 679, "friends_count": 731, "statues_count": 1818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mandalay Bay Arena", "id": "07d9e3fb5a085001", "name": "Mandalay Bay Arena", "place_type": "poi", "bounding_box": rectangle("-115.1793481,36.0905619 -115.179348,36.090562") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-04-30T00:00:04.000Z"), "id": 726305101690839040, "text": "@_Glenwood hit in work now https://t.co/91DZvlWlFE�� all music drops! were feuled by artists!��", "in_reply_to_status": -1, "in_reply_to_user": 341032148, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 341032148 }}, "user": { "id": 2314337120, "name": "Submit Be Heard", "screen_name": "DailyHipHopJamz", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-01-27"), "description": "The marketing tool for hip hop. Submit now⤵️ #DHHJCREW", "followers_count": 6015, "friends_count": 1784, "statues_count": 245239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305101795610624, "text": "???????? https://t.co/iqGQBziMjp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 44252886, "name": "Burt Maclin", "screen_name": "PopeQuanPaul", "lang": "en", "location": "Bronx 1st. Now Atlanta", "create_at": date("2009-06-02"), "description": "I'm waist deep in water while you waist deep in bills broke ass bitch ass hating ass nigga. All my Teams ass. Beat it Feds. BLACK MAN.", "followers_count": 1769, "friends_count": 958, "statues_count": 170692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305101820862464, "text": "@Reachout_mcd is it true every mcdonalds POS system \"restarts\" every night and shuts down for a few hours, rendering credit card useless?", "in_reply_to_status": -1, "in_reply_to_user": 148877103, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148877103 }}, "user": { "id": 630686648, "name": "Derek Dohrman", "screen_name": "DerekDohrman", "lang": "en", "location": "Flint, MI", "create_at": date("2012-07-08"), "description": "naturally selected | #theU | داريك | #Flint", "followers_count": 424, "friends_count": 359, "statues_count": 8092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305102424731648, "text": "more pics from today's 12-5 win https://t.co/q33uEOeAsY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3301309801, "name": "Vaca High Baseball", "screen_name": "VacaHiBaseball", "lang": "en", "location": "Vacaville, CA", "create_at": date("2015-07-30"), "description": "Official twitter of Vacaville High School Baseball. \nFour years of baseball, forty years of memories!", "followers_count": 168, "friends_count": 143, "statues_count": 591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305103133593600, "text": "Wow I have an online midterm on Wednesday but I work all day FML", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 961290480, "name": "lizzy navarro", "screen_name": "Lizzaye_", "lang": "en", "location": "the bay", "create_at": date("2012-11-20"), "description": "nothing's eazy", "followers_count": 1082, "friends_count": 860, "statues_count": 30417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305103884357632, "text": "Ugh I can never get enough of you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403882893, "name": "A", "screen_name": "_LHSP", "lang": "en", "location": "LV", "create_at": date("2011-11-02"), "description": "I like animals", "followers_count": 457, "friends_count": 312, "statues_count": 12065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305104069038080, "text": "Wind 3.6 mph E. Barometer 29.92 in, Falling. Temperature 68.4 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305104773693443, "text": "@samiaa19 tell Charley I miss her", "in_reply_to_status": 726304827295338496, "in_reply_to_user": 343863298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 343863298 }}, "user": { "id": 271632171, "name": "Kebab", "screen_name": "Kenank95", "lang": "en", "location": "Richmond Virginia", "create_at": date("2011-03-24"), "description": "Kenan #odu17 TKE.", "followers_count": 441, "friends_count": 406, "statues_count": 12983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305104945520640, "text": "We live in 2016 you'll either get one or the other... I'll take the first one �� https://t.co/D8TNUj5Iz9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2360513906, "name": "¢.", "screen_name": "_chanelalbright", "lang": "en", "location": "oakland, ca ☀️", "create_at": date("2014-02-24"), "description": "what it do?", "followers_count": 590, "friends_count": 267, "statues_count": 26830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-04-30T00:00:05.000Z"), "id": 726305105130098688, "text": "@monixoxx https://t.co/QnuTDU94BC", "in_reply_to_status": 726304998737354752, "in_reply_to_user": 702763168469155840, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 702763168469155840 }}, "user": { "id": 472702200, "name": "VINTAGE CREIGHT™", "screen_name": "MarcelloCRT", "lang": "en", "location": "ImaginationLand, Str8 Boolin", "create_at": date("2012-01-23"), "description": "Blessed Living ~ BrotherHood ~ TWC ~ #TheProcess ~ Nobody is me'er than me- Creight", "followers_count": 1742, "friends_count": 1007, "statues_count": 74989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-04-30T00:00:06.000Z"), "id": 726305106421944320, "text": "today's the big day, my big day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3350089493, "name": "MF SIRI", "screen_name": "hittii_", "lang": "en", "location": "OKAGA, CA", "create_at": date("2015-06-28"), "description": "null", "followers_count": 465, "friends_count": 395, "statues_count": 8413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667468861771776, "text": "@itsohsodemi I will be most definitely not...? Miss this LOL, and I bet her bottom not butt - dollar - not mine - that she will be rockin it", "in_reply_to_status": 725719952657895424, "in_reply_to_user": 148433437, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148433437 }}, "user": { "id": 4836077445, "name": "Brent Otter", "screen_name": "brent_otter7", "lang": "en", "location": "Durham, NC", "create_at": date("2016-01-22"), "description": "21 1/2 years as a Paratrooper In the US Army; MA in Psychology with minor in marriage and family therapy.loving kind respectful..messenger?too much to explain", "followers_count": 38, "friends_count": 174, "statues_count": 765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667468895178752, "text": "I'm going to show you better than I can tell you. & you're \"idgaf\" attitude go make it easier. I promise you only hurting yourself ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120933979, "name": "Andé: On-day", "screen_name": "sailingxsouls__", "lang": "en", "location": "C/O 2016 ❤️", "create_at": date("2010-03-07"), "description": "You don't get to choose if you get hurt in this world.. but you do have say in who hurts you..", "followers_count": 541, "friends_count": 371, "statues_count": 19674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerville, SC", "id": "676d5bedd73ca2ec", "name": "Summerville", "place_type": "city", "bounding_box": rectangle("-80.235287,32.944965 -80.109533,33.054993") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45035, "countyName": "Dorchester", "cityID": 4570270, "cityName": "Summerville" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667469411209216, "text": "@Ek_81_ haha just ignore me then.", "in_reply_to_status": 726667338938982400, "in_reply_to_user": 148444933, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148444933 }}, "user": { "id": 384443494, "name": "Nick Dvorak", "screen_name": "Nick_Dvorak", "lang": "en", "location": "River Falls, WI", "create_at": date("2011-10-03"), "description": "null", "followers_count": 499, "friends_count": 261, "statues_count": 32162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Falls, WI", "id": "fb0c8ec3e2d756b4", "name": "River Falls", "place_type": "city", "bounding_box": rectangle("-92.656794,44.829731 -92.55237,44.900999") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55093, "countyName": "Pierce", "cityID": 5568275, "cityName": "River Falls" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667469524471808, "text": "it's like being able to know everything but not having a brain to understand it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332593812, "name": "j", "screen_name": "joshtheghost", "lang": "en", "location": "月", "create_at": date("2011-07-09"), "description": "null", "followers_count": 164, "friends_count": 8, "statues_count": 10573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667469675352064, "text": "@brandon_ledet fuck you very much :-)", "in_reply_to_status": 726667272320770049, "in_reply_to_user": 1027509380, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1027509380 }}, "user": { "id": 90221813, "name": "Melvin", "screen_name": "meliza_aguilera", "lang": "en", "location": "Houston, TX", "create_at": date("2009-11-15"), "description": "imy Mikey", "followers_count": 286, "friends_count": 82, "statues_count": 32678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667470073798656, "text": "My life is so LIT FAM . (Sarcasm) Kind of getting over Instagram, but here's what my life has… https://t.co/xAZRWCGtoa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.09369947,36.0217391"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298188458, "name": "Soju boy", "screen_name": "AllSeeingJoven", "lang": "en", "location": "SinCity", "create_at": date("2011-05-13"), "description": "emotionally unstable", "followers_count": 211, "friends_count": 308, "statues_count": 22712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667470208139264, "text": "I'm not putting in no effort until I get a girl with a bubble butt , loyal , Lightskin , pretty & mean ��❌", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 416843406, "name": "✌️", "screen_name": "KingMarki__", "lang": "en", "location": "Chicago, IL ", "create_at": date("2011-11-19"), "description": "null", "followers_count": 5054, "friends_count": 2191, "statues_count": 113060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667470556160001, "text": "Lol there will be no alcohol at my wedding. None. Nope. Never.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99690234, "name": "Carlo Ren", "screen_name": "Carli_Elaine", "lang": "en", "location": "Irvine, CA", "create_at": date("2009-12-27"), "description": "Zachary Scott. Cal State Fullerton. Disneyland. Grey's Anatomy. And a love for Chewbacca that will never die. enjoy", "followers_count": 421, "friends_count": 382, "statues_count": 12791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667471302750208, "text": "Kissing him gave me butterflies ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1539684846, "name": "lex", "screen_name": "LexiQuevedo", "lang": "en", "location": "null", "create_at": date("2013-06-22"), "description": "that weird girl who can quote lord of the rings and spongebob in one sitting", "followers_count": 345, "friends_count": 172, "statues_count": 5593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667471617290240, "text": "what the fuck,\nmore like fuck the what.\nAM I RIGHT LADIES?\n@Sighlizzie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2183784360 }}, "user": { "id": 2694556806, "name": "Danielle", "screen_name": "DanielleKyes", "lang": "en", "location": "null", "create_at": date("2014-07-30"), "description": "(s)LAUGHTER", "followers_count": 67, "friends_count": 26, "statues_count": 1620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667471688597504, "text": "I HATE MOTHER NATURE BITCH ASS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288016736, "name": "♊️", "screen_name": "BreeeNicoleeeee", "lang": "en", "location": "Dallas TX ✈ LU ", "create_at": date("2011-04-25"), "description": "You Cannot Judge People Because They Sin Differently Than You.", "followers_count": 2148, "friends_count": 974, "statues_count": 166060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472074481664, "text": "@DagotheWolf awww we should have stayed for tattoos", "in_reply_to_status": 726663474378473473, "in_reply_to_user": 239652037, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 239652037 }}, "user": { "id": 237851518, "name": "HorseMeat", "screen_name": "ashlandpuppy", "lang": "en", "location": "Hemet, CA", "create_at": date("2011-01-13"), "description": "A powerlifting jock otterdog named Ashland. @BrutusMxms for jock things. Husband to @whitefoxhannah", "followers_count": 554, "friends_count": 262, "statues_count": 23379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472460455936, "text": "Ripley SW Limestone Co. Temp: 66.7°F Wind:0.0mph Pressure: 992.9mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472481329153, "text": "May", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.080523,34.400713"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2712549805, "name": "Atzel Solorza", "screen_name": "israelsolorza", "lang": "en", "location": "null", "create_at": date("2014-08-06"), "description": "17", "followers_count": 311, "friends_count": 309, "statues_count": 2513 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las vegas", "id": "07d9e3618f880002", "name": "Las vegas", "place_type": "poi", "bounding_box": rectangle("-118.0805231,34.4007129 -118.080523,34.400713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472540037121, "text": "Aww smh lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2942444242, "name": "Chief Hendrix", "screen_name": "TheGreat_Keeno", "lang": "en", "location": "Breaux Bridge", "create_at": date("2014-12-26"), "description": "Exit 1⃣0⃣9⃣..... #DukeNation...... LakerNation...... #BUCKETCREW2⃣2⃣........... #JayHawkNation", "followers_count": 1382, "friends_count": 979, "statues_count": 101960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cecilia, LA", "id": "00de095d5c203e1d", "name": "Cecilia", "place_type": "city", "bounding_box": rectangle("-91.864526,30.305524 -91.79393,30.36137") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22099, "countyName": "St. Martin", "cityID": 2213575, "cityName": "Cecilia" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472615559168, "text": "Wind 1.0 mph NW. Barometer 1016.46 mb, Rising. Temperature 52.8 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-01T00:00:00.000Z"), "id": 726667472720396288, "text": "@ebbtideapp Tide in Cornfield Harbor, Maryland 05/01/2016\nHigh 9:27am 1.5\n Low 3:21pm 0.2\nHigh 10:11pm 1.5\n Low 3:57am 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-76.3583,38.0617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 29018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maryland, USA", "id": "dea1eac2d7ef8878", "name": "Maryland", "place_type": "admin", "bounding_box": rectangle("-79.487651,37.886607 -74.986286,39.723622") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24037, "countyName": "St. Mary's" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667472951189504, "text": "Wind 0.0 mph SSE. Barometer 29.935 in, Falling slowly. Temperature 58.8 °F. Rain today 0.01 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 41, "friends_count": 4, "statues_count": 27095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667472955281409, "text": "Being faded and thinking of ya girl with another guy is sus smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2852428722, "name": "Nosaj", "screen_name": "Aritapl0x", "lang": "en", "location": "lost ", "create_at": date("2014-10-11"), "description": "A R I T A", "followers_count": 147, "friends_count": 48, "statues_count": 14672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667473064476672, "text": "Wind 1.6 mph S. Barometer 29.95 in, Steady. Temperature 65.7 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667473458622464, "text": "Sex with me so Amazing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 626252668, "name": "Cameron Larson Sr", "screen_name": "CammLarr1", "lang": "en", "location": "253", "create_at": date("2012-07-04"), "description": "#FBA #TTC #TKG Proper.Preparation.Prevents.Poor.Performance.", "followers_count": 681, "friends_count": 579, "statues_count": 52381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, WA", "id": "fedb5ad42e4b046a", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-122.579274,47.115869 -122.434284,47.198337") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5338038, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667473764913157, "text": "Should Hey Mercedes bring us on tour?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 221496459, "name": "TWIABP&IANLATD", "screen_name": "twiabp", "lang": "en", "location": "nowhere ", "create_at": date("2010-11-30"), "description": "Death to New Years. Long Live Happy Birthday.", "followers_count": 22481, "friends_count": 1263, "statues_count": 12605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667473848651776, "text": "@rchellesky I'm with him rn", "in_reply_to_status": 726667144285409280, "in_reply_to_user": 3067105489, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3067105489 }}, "user": { "id": 2733623018, "name": "Matt Duenas", "screen_name": "mattybobaaanga", "lang": "en", "location": "Bay Area, California", "create_at": date("2014-08-14"), "description": "Forging my own path | Hang Loose", "followers_count": 687, "friends_count": 982, "statues_count": 25938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "American Canyon, CA", "id": "f28195f8b183abcd", "name": "American Canyon", "place_type": "city", "bounding_box": rectangle("-122.283944,38.154977 -122.229741,38.204868") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 601640, "cityName": "American Canyon" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667474033238017, "text": "You don't know how to love me when you're sober...when the bottle's done you pull me closer. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1281303066, "name": "Carlos.", "screen_name": "CarlitosFaz", "lang": "en", "location": "null", "create_at": date("2013-03-19"), "description": "22. Happy Life.❤️", "followers_count": 633, "friends_count": 473, "statues_count": 36590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667474683359232, "text": "@Influenster i want one!!", "in_reply_to_status": 726172910877085701, "in_reply_to_user": 60152652, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60152652 }}, "user": { "id": 373157394, "name": "Breann E. Dixon", "screen_name": "Brebre916", "lang": "en", "location": "california", "create_at": date("2011-09-13"), "description": "im losst in music but it saves my soul", "followers_count": 35, "friends_count": 161, "statues_count": 161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Sacramento, CA", "id": "009243ac50ad0f26", "name": "West Sacramento", "place_type": "city", "bounding_box": rectangle("-121.584282,38.516197 -121.506483,38.603777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 684816, "cityName": "West Sacramento" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667475472027648, "text": "*my mom & brother get in my car*\nMy mom : \"oh shit ! Chris don't breath you're gonna get high\" \n������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 756517908, "name": "Cee-Cee", "screen_name": "sierraclem1", "lang": "en", "location": "outerspace", "create_at": date("2012-08-13"), "description": "null", "followers_count": 452, "friends_count": 746, "statues_count": 12980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667475509612544, "text": "Tell me why my bf never sleeps but when I'm talking to him he knocks out ): rip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2994898717, "name": "Kaylie", "screen_name": "kaylie_ee", "lang": "en", "location": "null", "create_at": date("2015-01-24"), "description": "Alejandro Andrade ❤️", "followers_count": 129, "friends_count": 139, "statues_count": 962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667475715264513, "text": "Temp: 63.4°F Wind:0.0mph Pressure: 29.896hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476101013505, "text": "Officially kicking off party season! Time to live it up Vegas style!!!! #lasvegas #glamlife #Drais https://t.co/m6zlSL0xZW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.172102,36.114979"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "lasvegas", "glamlife", "Drais" }}, "user": { "id": 594198746, "name": "Tresha Gazzola", "screen_name": "MakeupByTresha", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-05-29"), "description": "ᴍᴀᴋᴇᴜᴘ ᴀʀᴛɪsᴛ ℓαѕ νєgαѕ / нαωαιι", "followers_count": 245, "friends_count": 368, "statues_count": 6092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Drai's Beach Club • Nightclub", "id": "07d9f2ad97c88000", "name": "Drai's Beach Club • Nightclub", "place_type": "poi", "bounding_box": rectangle("-115.17210209999999,36.1149789 -115.172102,36.114979") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476298125313, "text": "Might take a nice hot shower maybe that will help me out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393677318, "name": "Bianca Vargas", "screen_name": "Chiquilina93", "lang": "en", "location": "Fresno", "create_at": date("2011-10-18"), "description": "|22 | Independent ||XOTWOD|| I'm just tryna live life through a new lens - The Weeknd", "followers_count": 60, "friends_count": 103, "statues_count": 1657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476470140928, "text": "�� https://t.co/UeAOVJoYW9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 348775520, "name": "tha worst", "screen_name": "dejawajaa", "lang": "en", "location": "MS", "create_at": date("2011-08-04"), "description": "God thinks I am to die for .. |luvyou2munii| #yellowworld", "followers_count": 1465, "friends_count": 560, "statues_count": 73304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Byram, MS", "id": "8c29db3fcbf22840", "name": "Byram", "place_type": "city", "bounding_box": rectangle("-90.31041,32.157413 -90.23648,32.239589") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2810140, "cityName": "Byram" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476549902336, "text": "He so fine ���� but yet so lightskin ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357609410, "name": "الأبيض جدا للأطفال أ", "screen_name": "LexyRenae_", "lang": "en", "location": "DET✈AL", "create_at": date("2011-08-18"), "description": "Miles College Junior #MC17 #PMM #Ego-Trip #ΤΒΣ Spr '15 Instagram: lexyrenae", "followers_count": 996, "friends_count": 1130, "statues_count": 29438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, AL", "id": "955c49d6773f00f0", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-86.951217,33.451056 -86.903856,33.500963") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 125120, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476700946432, "text": "#micdrop is now trending in #LA https://t.co/9vCPDzVdDL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "micdrop", "LA" }}, "user": { "id": 123790358, "name": "Trendsmap LosAngeles", "screen_name": "TrendsLA", "lang": "en", "location": "Los Angeles", "create_at": date("2010-03-16"), "description": "Real-time local Twitter trends for Los Angeles", "followers_count": 1959, "friends_count": 543, "statues_count": 59318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476759678976, "text": "We gotta abort the motion for the greater good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262284690, "name": "Brooke", "screen_name": "brookeave2", "lang": "en", "location": "null", "create_at": date("2013-12-25"), "description": "uofl, zta♛", "followers_count": 854, "friends_count": 478, "statues_count": 8318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476876988416, "text": "Just got out of work but I have to be back at 10 in the morning...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241438414, "name": "J. Ferrufino", "screen_name": "Trillassghost", "lang": "en", "location": "SC:PrinceJesse", "create_at": date("2011-01-21"), "description": "Est. 1994, Live the life that you have imagined #PartyFavors", "followers_count": 910, "friends_count": 2760, "statues_count": 20873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476956798976, "text": "can't wait to gtfo ship ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3849123436, "name": "5.14 ✨", "screen_name": "frankierae_", "lang": "en", "location": "Philly", "create_at": date("2015-10-02"), "description": "Starbucks soothes the soul. ✨ Shippensburg University ⛵️⚓️", "followers_count": 943, "friends_count": 635, "statues_count": 35687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shippensburg, PA", "id": "a3d888f1d0b5d5a2", "name": "Shippensburg", "place_type": "city", "bounding_box": rectangle("-77.549194,40.03646 -77.499257,40.067857") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4270352, "cityName": "Shippensburg" } }
+{ "create_at": datetime("2016-05-01T00:00:01.000Z"), "id": 726667476965163008, "text": "Nah I'm too skinny for that �� https://t.co/YrCJaKIN1f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278761062, "name": "JJtheeGreat", "screen_name": "TempFocTho", "lang": "en", "location": "null", "create_at": date("2011-04-07"), "description": "#JSU19 | Murrah'15 | #DWYC |", "followers_count": 658, "friends_count": 1040, "statues_count": 7233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vicksburg, MS", "id": "bdaa181492bdd711", "name": "Vicksburg", "place_type": "city", "bounding_box": rectangle("-90.92885,32.223388 -90.821771,32.392365") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28149, "countyName": "Warren", "cityID": 2876720, "cityName": "Vicksburg" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477028069376, "text": "Wind 0.0 mph ---. Barometer 29.834 in, Steady. Temperature 59.6 °F. Rain today 0.01 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477078384640, "text": "������ https://t.co/xt1GNqUFaM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 276847830, "name": "✨Emily✨", "screen_name": "Loveee_Stoned", "lang": "en", "location": "null", "create_at": date("2011-04-03"), "description": "Lips So French , Ass So Spanish", "followers_count": 804, "friends_count": 883, "statues_count": 40308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477158088704, "text": "05/01@03:00 - Temp 50.0F, WC 50.0F. Wind 0.3mph SE, Gust 2.0mph. Bar 30.182in, Falling. Rain 0.00in. Hum 80%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477195722752, "text": "I just watched this whole thing, and I can tell you it's worth it. On my list of most entertaining speeches. // https://t.co/S8WsDMiOQa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15494445, "name": "Jenna Kovalsky", "screen_name": "abcdefghijenna", "lang": "en", "location": "San Fransokyo", "create_at": date("2008-07-19"), "description": "I sweat the small stuff, and I try to look up a lot. Pro-Oxford comma. Content strategy at @Google.", "followers_count": 401, "friends_count": 394, "statues_count": 6904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477258670081, "text": "https://t.co/3AfNLQIyYL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1170792344, "name": "Daniel E. Barnes", "screen_name": "danielebarnes", "lang": "en", "location": "Los Angeles", "create_at": date("2013-02-11"), "description": "I am free in all of the ways that you are not.", "followers_count": 383, "friends_count": 98, "statues_count": 776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477598478336, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":62.2°F Wind:0.0mph Pressure: 29.91hpa Falling slowly Rain Today 0.33in. Forecast: Unsettled, precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 316699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477602713600, "text": "Fat crush on some marine ������ https://t.co/J2rs6MsBOZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338575712, "name": "Tatum Wright", "screen_name": "tatumwright4", "lang": "en", "location": "null", "create_at": date("2011-07-19"), "description": "22. dancer. JPD choreographer. crossfitfoothill. california dreamin☀️", "followers_count": 783, "friends_count": 711, "statues_count": 14069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Forest, CA", "id": "3a16272f05577d7e", "name": "Lake Forest", "place_type": "city", "bounding_box": rectangle("-117.72136,33.606781 -117.621862,33.685914") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639496, "cityName": "Lake Forest" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477808095232, "text": "I planned to pull an all nighter from 10 till tomorrow but United is playing at 8 so there goes my night.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172302796, "name": "Tobi", "screen_name": "t0bismith", "lang": "en", "location": "Denton, TX", "create_at": date("2010-07-29"), "description": "currently accepting donations in forms of pizza and tuition. writer: @zikokomag", "followers_count": 2591, "friends_count": 918, "statues_count": 124134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477875302400, "text": "I wish I could sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2314401469, "name": "Lady Kimberly", "screen_name": "kimberly_walski", "lang": "en", "location": "Illinois, USA", "create_at": date("2014-01-27"), "description": "Just a girl who loves anime and drawing! \nSingle \n19", "followers_count": 507, "friends_count": 744, "statues_count": 866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bourbonnais, IL", "id": "d4bae7fb9db0d218", "name": "Bourbonnais", "place_type": "city", "bounding_box": rectangle("-87.991251,41.139487 -87.848915,41.206121") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17091, "countyName": "Kankakee", "cityID": 1707471, "cityName": "Bourbonnais" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667477875343360, "text": "Tyna every babe ganna retweet this I hate my life. I hate being me https://t.co/7D1AWPifyR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3368822404, "name": "I lost my identity", "screen_name": "anthonylarrys1", "lang": "en", "location": "Staten Island, NY", "create_at": date("2015-07-09"), "description": "I don't even eat lever", "followers_count": 484, "friends_count": 241, "statues_count": 14879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667478357544960, "text": "I Couldn't make it TO Morelia MX to with Dad and pope Francis , and I'm so honored to step on holy Rome ��God 1st���� https://t.co/H3ezVdjQtH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3149543996, "name": ",S☀️L", "screen_name": "you_lissa", "lang": "en", "location": "CENTRAL,VALLEY,CA, S☀️NSHINE ", "create_at": date("2015-04-11"), "description": "Singer song writer,", "followers_count": 302, "friends_count": 36, "statues_count": 3039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanger, CA", "id": "cea1f774c62bb6fc", "name": "Sanger", "place_type": "city", "bounding_box": rectangle("-119.583292,36.677705 -119.538486,36.721241") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 667056, "cityName": "Sanger" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667478684819456, "text": "the feelin u b givin me tho ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2281338396, "name": "bri", "screen_name": "abriannaa16", "lang": "en", "location": "Willoughby, OH", "create_at": date("2014-01-07"), "description": "Beyond Blessed", "followers_count": 431, "friends_count": 179, "statues_count": 3106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowick, OH", "id": "df4f3fa83ca683de", "name": "Willowick", "place_type": "city", "bounding_box": rectangle("-81.488902,41.618052 -81.451305,41.652659") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985638, "cityName": "Willowick" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667478705774592, "text": "Temp 41.1° Hi/Lo 44.9/41.1 Rng 3.8° WC 41.1° Hmd 78% Rain 0.00\" Storm 0.00\" BAR 30.243 Falling DP 34.8° Wnd 0mph Dir --- Gst 8mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 66, "friends_count": 120, "statues_count": 18333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667478756028416, "text": "\"my numb is face\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3300375331, "name": "izybel", "screen_name": "hzmariahz", "lang": "en", "location": "null", "create_at": date("2015-07-29"), "description": "null", "followers_count": 118, "friends_count": 104, "statues_count": 1409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667479167188993, "text": "Ⓜ️arathon ⛽️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 238152143, "name": "SLICK", "screen_name": "Hoeuaint_SLICK", "lang": "en", "location": "Columbus, OH", "create_at": date("2011-01-14"), "description": "Progression ⏳ . . . R.I.P MOM and DAD", "followers_count": 3479, "friends_count": 2023, "statues_count": 73581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667479477415936, "text": "@LogoTV wishing @JLo and @McConaughey would've married! #weddingplanner #movienight #logotv", "in_reply_to_status": -1, "in_reply_to_user": 16688083, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weddingplanner", "movienight", "logotv" }}, "user_mentions": {{ 16688083, 85603854, 57097104 }}, "user": { "id": 72926830, "name": "AMBER", "screen_name": "ambermariefaith", "lang": "en", "location": "Houston, TX", "create_at": date("2009-09-09"), "description": "In order to be irreplaceable one must always be different TX ❤️TAMUCC '14 COMM @ambermariefaith @medleyinc", "followers_count": 180, "friends_count": 592, "statues_count": 311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667479678750720, "text": "heavy intensity rain -> moderate rain\nhumidity down 99% -> 98%\nwind 13mph -> 12mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.64,42.87"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 214306884, "name": "Kentwood Weather", "screen_name": "KentwoodMI", "lang": "en", "location": "Kentwood, MI", "create_at": date("2010-11-10"), "description": "Weather updates, forecast, warnings and information for Kentwood, MI. Sources: http://OpenWeatherMap.org, NOAA, USGS.", "followers_count": 23, "friends_count": 1, "statues_count": 20691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentwood, MI", "id": "638adca1ca0cd926", "name": "Kentwood", "place_type": "city", "bounding_box": rectangle("-85.665318,42.854479 -85.544605,42.971056") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2642820, "cityName": "Kentwood" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667479792013313, "text": "I will forever HATE Bombsell 2 in 1 Mascara. Was trying it out for prom..... And it's like glue. It won't come off my eyelashes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184902981, "name": "Gib-Gab Gunderson", "screen_name": "GabbyKouzoukas", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "Hi I'm Gabby I love cheese and I hate many people, except for @Jed0131", "followers_count": 389, "friends_count": 424, "statues_count": 12248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westchester, IL", "id": "2180fafeb49b5684", "name": "Westchester", "place_type": "city", "bounding_box": rectangle("-87.918968,41.83322 -87.867031,41.871262") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1780047, "cityName": "Westchester" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667479804698624, "text": "Wind 0.0 mph ---. Barometer 29.888 in, Falling slowly. Temperature 61.3 °F. Rain today 0.02 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480123449344, "text": "Wind 0.0 mph NNE. Barometer 30.19 in, Steady. Temperature 51.3 °F. Rain today 0.00 in. Humidity 37%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480303685632, "text": "I ate way too much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 370517262, "name": "✌Jaid ✌", "screen_name": "Jaid_Harris16", "lang": "en", "location": "future coug", "create_at": date("2011-09-08"), "description": "SHS Senior • Basketball & Softball", "followers_count": 664, "friends_count": 335, "statues_count": 10686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kennewick, WA", "id": "63d0671506628c8d", "name": "Kennewick", "place_type": "city", "bounding_box": rectangle("-119.264818,46.162262 -119.087063,46.236321") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5335275, "cityName": "Kennewick" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480349855744, "text": "https://t.co/9ExPerQfgD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 19743851, "name": "Zack Sestak", "screen_name": "e1nemo", "lang": "en", "location": "Hollywood", "create_at": date("2009-01-29"), "description": "I make it rain.", "followers_count": 115, "friends_count": 50, "statues_count": 4525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480471490560, "text": "19 BITCH�� https://t.co/4KOKIX3eDF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256841399, "name": "boolestmom", "screen_name": "Itzzmichelle", "lang": "en", "location": "null", "create_at": date("2011-02-23"), "description": "xoxo gossip girl", "followers_count": 1061, "friends_count": 783, "statues_count": 14911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480496631808, "text": "Take cute selfies w me and I'm literally happy like its not that hard to please me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 967283610, "name": "Darth Tater", "screen_name": "TheVajenna", "lang": "en", "location": "Boston, MA", "create_at": date("2012-11-23"), "description": "I love hiking, camping, classical music, and reading // I hold the high score in pacman at three separate locations // (she/her) //", "followers_count": 545, "friends_count": 399, "statues_count": 30333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haverhill, MA", "id": "fe52af1ec31cb540", "name": "Haverhill", "place_type": "city", "bounding_box": rectangle("-71.182362,42.734289 -71.000439,42.827267") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2529405, "cityName": "Haverhill" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480802848770, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1661637151, "name": "Tabbatha Riojas", "screen_name": "tabbbssssss_", "lang": "en", "location": "tx", "create_at": date("2013-08-10"), "description": "blessings on blessings", "followers_count": 1137, "friends_count": 546, "statues_count": 17292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainview, TX", "id": "6513b0430da3e9ac", "name": "Plainview", "place_type": "city", "bounding_box": rectangle("-101.760027,34.142451 -101.643481,34.235002") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48189, "countyName": "Hale", "cityID": 4857980, "cityName": "Plainview" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480869933056, "text": "<3 https://t.co/E10kHQf256", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2195376368, "name": "Gloria Garza", "screen_name": "Glogigglesria", "lang": "en", "location": "San antonio, TX", "create_at": date("2013-11-14"), "description": "Love the Cubbies !!!; Experience is what you get when you didn't get what you wanted.", "followers_count": 25, "friends_count": 61, "statues_count": 257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480869953537, "text": "@nailatre ������������������������", "in_reply_to_status": 726667029210492929, "in_reply_to_user": 624398685, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 624398685 }}, "user": { "id": 336796677, "name": "folami", "screen_name": "_FmTheGreat", "lang": "en", "location": "Houston ", "create_at": date("2011-07-16"), "description": "why read my bio when you could read a book", "followers_count": 2333, "friends_count": 899, "statues_count": 106357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667480907702273, "text": "72.4F (Feels: 72.4F) - Humidity: 99% - Wind: 6.9mph E - Gust: 12.1mph - Pressure: 1012.3mb - Rain: 0.12\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-01T00:00:02.000Z"), "id": 726667481058668544, "text": "Want to work at SONIC Drive-In? We're #hiring in #Amarillo, TX! Click for details: https://t.co/3CE8wyILEg #Hospitality #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.849987,35.2218186"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Amarillo", "Hospitality", "Job", "Jobs" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 309, "friends_count": 39, "statues_count": 19635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amarillo, TX", "id": "52a0517c2183cf22", "name": "Amarillo", "place_type": "city", "bounding_box": rectangle("-101.96758,35.098693 -101.664604,35.323538") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48375, "countyName": "Potter", "cityID": 4803000, "cityName": "Amarillo" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667481222397952, "text": "lmaoooo next episode is gonna be about fight stories ���� https://t.co/lWIe4OHLmo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 291163591, "name": "Michael B. Julian", "screen_name": "RobGotNext", "lang": "en", "location": "Southside, NJ", "create_at": date("2011-05-01"), "description": "yall done or yall finished?#ItsJustTwitterItsNotThatDeepTribe #dormshaker 11|14|08 dm me for promos", "followers_count": 49181, "friends_count": 40301, "statues_count": 13727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, NJ", "id": "0b54403ae9948a6c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-74.89812,40.024738 -74.797662,40.116498") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3408920, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667481230643200, "text": "@_SelfieQueennn is on my hit list ��", "in_reply_to_status": -1, "in_reply_to_user": 1869800893, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1869800893 }}, "user": { "id": 3316343414, "name": "rosecrans stupid!", "screen_name": "ComptonStupid", "lang": "en", "location": "Compton, CA", "create_at": date("2015-08-15"), "description": "all my life I hustled. now I uber everywhere. I'm coming for whoever. I'm really from Compton tho.", "followers_count": 123, "friends_count": 106, "statues_count": 2534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667482161778688, "text": "If you never get rid of the wrong people, you'll never meet the right people ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1075627501, "name": "WIIM aka Mr. PM", "screen_name": "WIIMtheMusician", "lang": "en", "location": "orange, CA", "create_at": date("2013-01-09"), "description": "Your friendly neighborhood rapper | Recording Engineer | For booking hit up wiimofficial@gmail.com", "followers_count": 4922, "friends_count": 4955, "statues_count": 1451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Costa Mesa, CA", "id": "2f97b9acb13ca665", "name": "Costa Mesa", "place_type": "city", "bounding_box": rectangle("-117.954201,33.625179 -117.85709,33.702038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 616532, "cityName": "Costa Mesa" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667482736545793, "text": "Map of the testimonies received so far following the #earthquake M3.5 in Western Kentucky 48 min ago https://t.co/7o26qjxHtt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.99,37.21"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake" }}, "user": { "id": 203226820, "name": "EMSC", "screen_name": "LastQuake", "lang": "fr", "location": "based in Paris, works globally", "create_at": date("2010-10-15"), "description": "Independant Scientifc Organization and provider of real-time earthquake warning. Share information with us through our app : https://t.co/AxO8bJddn5", "followers_count": 18918, "friends_count": 232, "statues_count": 13916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21007, "countyName": "Ballard" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667482811928576, "text": "if someone is embarrassed about talking to you and doesn't tell their friends about you what r u doing bro", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2810641260, "name": "seb", "screen_name": "SebastianVRL", "lang": "en", "location": "Bountiful, UT", "create_at": date("2014-09-14"), "description": "snapchat:th3sebastian", "followers_count": 206, "friends_count": 102, "statues_count": 608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bountiful, UT", "id": "d10ead505fcb25cb", "name": "Bountiful", "place_type": "city", "bounding_box": rectangle("-111.906716,40.834807 -111.834245,40.909783") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4907690, "cityName": "Bountiful" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483189399552, "text": "�� wyd? https://t.co/Yau1tkGhB1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 260545109, "name": "Mary J", "screen_name": "ItsMissMaryJ_", "lang": "en", "location": "Ft.Worth |SM #TxSt16", "create_at": date("2011-03-03"), "description": "#TXST16 | PR | Theatre |22| ✨", "followers_count": 1349, "friends_count": 1254, "statues_count": 21589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483520753664, "text": "Honestly EVERYONES looked bomb asf tonight ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2726121075, "name": "ѕι∂ тяαи", "screen_name": "sidneytran_", "lang": "en", "location": "null", "create_at": date("2014-07-27"), "description": "Clackamas Cavalette ♊️ Snapchat: Sidneytran_", "followers_count": 465, "friends_count": 584, "statues_count": 4084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483697012736, "text": "I CRIED ������������ https://t.co/q16cTwiexX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3236161412, "name": "Albarhoe", "screen_name": "AlvarhoeArzate", "lang": "en", "location": "Northlake, IL ", "create_at": date("2015-06-04"), "description": "18 just be happy yo i don't judge hmu", "followers_count": 302, "friends_count": 255, "statues_count": 3882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northlake, IL", "id": "d64afac908bf795d", "name": "Northlake", "place_type": "city", "bounding_box": rectangle("-87.92903,41.899096 -87.881665,41.937055") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1753871, "cityName": "Northlake" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483730612224, "text": "I got to work at 4pm and didn't get home until 2am so I sort of want2die", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272706086, "name": "Kelly Karausky", "screen_name": "Fearmadillo", "lang": "en", "location": "inaba, new jersey", "create_at": date("2011-03-26"), "description": "it's the nature of the beast | xxx | luke 10:19", "followers_count": 238, "friends_count": 203, "statues_count": 35646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Lake Park, NJ", "id": "007998004ba3efc0", "name": "Pine Lake Park", "place_type": "city", "bounding_box": rectangle("-74.288073,39.987032 -74.231391,40.032622") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean", "cityID": 3458830, "cityName": "Pine Lake Park" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483734773760, "text": "We're those troll dolls you used to play with. You know who you are.… https://t.co/6xmlps1tjD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.26020335,34.07762124"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441450014, "name": "Katie Moore", "screen_name": "ktmoorektmoore", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-19"), "description": "Yes, it's me. Beyoncé & Amy Schumer's love child. Mamba out.", "followers_count": 283, "friends_count": 338, "statues_count": 2059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667483835449344, "text": "I feel like if someone screamed in my face that they hated me 2 weekends in a row, I'd probably just leave them alone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284759483, "name": "Jenifer", "screen_name": "jeniferjones3", "lang": "en", "location": "Cedar Rapids, IA", "create_at": date("2011-04-19"), "description": "livin' it up", "followers_count": 320, "friends_count": 140, "statues_count": 9596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Robins, IA", "id": "76bf95914ac8eb10", "name": "Robins", "place_type": "city", "bounding_box": rectangle("-91.672009,42.058548 -91.643339,42.078598") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1967800, "cityName": "Robins" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667484485419008, "text": "���� https://t.co/RCOIfScacm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3067106243, "name": "Hanah Hulett", "screen_name": "hanahhhulett", "lang": "en", "location": "wingstop", "create_at": date("2015-03-02"), "description": "quality you can taste", "followers_count": 595, "friends_count": 364, "statues_count": 3077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667484556890113, "text": "Wind 0.0 mph ---. Barometer 30.183 in, Steady. Temperature 37.9 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667484577845248, "text": "f u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2958967893, "name": "Tianna DeFrancisco", "screen_name": "TiannaDefran", "lang": "en", "location": "null", "create_at": date("2015-01-03"), "description": "null", "followers_count": 610, "friends_count": 425, "statues_count": 555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Algonquin, IL", "id": "6d716125b43cb735", "name": "Algonquin", "place_type": "city", "bounding_box": rectangle("-88.383751,42.124838 -88.198981,42.241892") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17111, "countyName": "McHenry", "cityID": 1700685, "cityName": "Algonquin" } }
+{ "create_at": datetime("2016-05-01T00:00:03.000Z"), "id": 726667485043261440, "text": "@sstvrkk NOOOO", "in_reply_to_status": 726598886299566080, "in_reply_to_user": 615962220, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 615962220 }}, "user": { "id": 1452342955, "name": "(o´ω`o)", "screen_name": "springingleaks", "lang": "en", "location": "San Francisco", "create_at": date("2013-05-23"), "description": "you're gonna carry that weight", "followers_count": 229, "friends_count": 140, "statues_count": 13412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667485546598401, "text": "Why did @Meghan_Trainor eat @ToriKelly Grammy? @innoutburger must have been buzy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 254830969, 19017237, 16636772 }}, "user": { "id": 177087473, "name": "logan", "screen_name": "jumpedtrains", "lang": "en", "location": "null", "create_at": date("2010-08-10"), "description": "insta: jumpedtrains", "followers_count": 845, "friends_count": 265, "statues_count": 23785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, CA", "id": "51d80bae0d6cb453", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-122.072718,37.655585 -122.023477,37.696142") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 623350, "cityName": "Fairview" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667485928427520, "text": ":(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 174922926, "name": "Lauren Carney", "screen_name": "LaurenMCarney", "lang": "en", "location": "Dayton, KY", "create_at": date("2010-08-04"), "description": "Do what makes you happy, and do it unapologetically. #Trump2016", "followers_count": 1048, "friends_count": 575, "statues_count": 13287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, KY", "id": "30482134b73d7bc8", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-84.482506,39.101932 -84.444128,39.118814") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21000, "countyName": "Unknown", "cityID": 2120350, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667486087651329, "text": "IT'S 3 AM I MUST BE LONELY!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304707032, "name": "Madi Task", "screen_name": "alwaysoff_TASK", "lang": "en", "location": "The Ohio State University", "create_at": date("2011-05-24"), "description": "I am a nut || ALPHACHIp(OTL)e", "followers_count": 1146, "friends_count": 1034, "statues_count": 23582 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667486385594368, "text": "You ❤️ https://t.co/9bjoglGgei", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 282793330, "name": "Gabs", "screen_name": "ggabsc", "lang": "pt", "location": "null", "create_at": date("2011-04-15"), "description": "null", "followers_count": 254, "friends_count": 97, "statues_count": 5217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667486662369280, "text": "Bored at work, wasted love.�� https://t.co/t9Bfgx2Q7L", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58157906, "name": "✗♥O", "screen_name": "WickedGames__", "lang": "en", "location": "Electric Sky", "create_at": date("2009-07-19"), "description": "Let's Rave❣", "followers_count": 1333, "friends_count": 1170, "statues_count": 14075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doral, FL", "id": "053ab8e6678199ed", "name": "Doral", "place_type": "city", "bounding_box": rectangle("-80.41729,25.780974 -80.321016,25.870313") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1217935, "cityName": "Doral" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667486922313728, "text": "https://t.co/DYyjsBvS5K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2545288652, "name": "GUCCI", "screen_name": "serennie3", "lang": "en", "location": "null", "create_at": date("2014-06-03"), "description": "null", "followers_count": 139, "friends_count": 295, "statues_count": 1223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, CA", "id": "e6fc035b8939bd3b", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-118.06946,34.087059 -117.99218,34.180507") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602462, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667486934999040, "text": "They sure are https://t.co/StokLdP7XB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150401087, "name": "SLΛTER", "screen_name": "AmaniPunani", "lang": "en", "location": "null", "create_at": date("2010-05-31"), "description": "South Florida to Atlanta, INSTAGRAM: @instagodx #CAU17", "followers_count": 2816, "friends_count": 2032, "statues_count": 104461 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667487014694913, "text": "Someone bring me in & out ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614772012, "name": "Queen Dej ✨", "screen_name": "dejuhhhvoox3", "lang": "en", "location": "null", "create_at": date("2012-06-21"), "description": "null", "followers_count": 512, "friends_count": 371, "statues_count": 22403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667487102689280, "text": "#214 ���� https://t.co/IfEMe086Nc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1952955325, "name": "J Ā Y B.", "screen_name": "jah_briel", "lang": "en", "location": "TEXA$", "create_at": date("2013-10-10"), "description": "I'm cool and fun | #UofA '20", "followers_count": 304, "friends_count": 208, "statues_count": 8195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667487471763456, "text": "@chihiro_maria OMG!! ITS A WILD CHIHIRO ������������", "in_reply_to_status": 726667208961593344, "in_reply_to_user": 2666512410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2666512410 }}, "user": { "id": 2899463612, "name": "moii noonpoolaa", "screen_name": "meinonpala", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-11-14"), "description": "null", "followers_count": 99, "friends_count": 70, "statues_count": 3057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667487551442944, "text": "May Baby����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 799624982, "name": "Your Dad", "screen_name": "alexis143u1", "lang": "en", "location": "Napa,Caliii", "create_at": date("2012-09-02"), "description": "SC:Alexis143u1 #ViscaBarca", "followers_count": 421, "friends_count": 281, "statues_count": 19230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667488138788865, "text": "Easy nufta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 260908122, "name": "Anthony Pisani", "screen_name": "ampisani", "lang": "en", "location": "White Plains, NY", "create_at": date("2011-03-04"), "description": "Syracuse University 2012 & NY --- Live your life, be free, you know you can have it all", "followers_count": 294, "friends_count": 420, "statues_count": 4974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667488184905729, "text": "Wind 0.0 mph WSW. Barometer 29.836 in, Steady. Temperature 67.4 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667488822337536, "text": "Female shit...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 722430318259515396, "name": "Miel", "screen_name": "MsColorado_", "lang": "en", "location": "Denver, CO", "create_at": date("2016-04-19"), "description": "null", "followers_count": 143, "friends_count": 131, "statues_count": 1211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667488872656897, "text": "@zakkevanss @asiaxh how exciting", "in_reply_to_status": 726661891829956608, "in_reply_to_user": 573256508, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 573256508, 1192148786 }}, "user": { "id": 240437513, "name": "lily", "screen_name": "LilyBarek", "lang": "en", "location": "Irvine, CA", "create_at": date("2011-01-19"), "description": "null", "followers_count": 780, "friends_count": 441, "statues_count": 22771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667488952340481, "text": "Wind 0.0 mph ---. Barometer 29.855 in, Rising slowly. Temperature 59.0 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667489262731265, "text": "nah. https://t.co/sDe6dxWGB2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 313466164, "name": "✨London", "screen_name": "_LifesJustGood", "lang": "en", "location": "〽️to NOLA", "create_at": date("2011-06-08"), "description": "DillardUnivetsity'19 #DU19", "followers_count": 872, "friends_count": 942, "statues_count": 11263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-01T00:00:04.000Z"), "id": 726667489468375040, "text": "Oomf has the same name as me and idk why that's lit to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224120311, "name": "Dom", "screen_name": "Dom_Dalbert", "lang": "en", "location": "Austin, TX", "create_at": date("2010-12-07"), "description": "20. Young jiggy kid. Music is life.", "followers_count": 841, "friends_count": 845, "statues_count": 15067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-01T00:00:05.000Z"), "id": 726667490302947328, "text": "���������������� https://t.co/oPAb3TDXuM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 480157193, "name": "joy", "screen_name": "joyt__", "lang": "en", "location": "new mexico/oklahoma ", "create_at": date("2012-01-31"), "description": "uco biology/graphic design", "followers_count": 531, "friends_count": 241, "statues_count": 44666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edmond, OK", "id": "62ff27569714ddb3", "name": "Edmond", "place_type": "city", "bounding_box": rectangle("-97.557572,35.606823 -97.35391,35.783621") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4023200, "cityName": "Edmond" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029856836444161, "text": "@DevinFerrari Yessir!", "in_reply_to_status": 727018897791049728, "in_reply_to_user": 30548137, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30548137 }}, "user": { "id": 15896497, "name": "Mayor K Dot.", "screen_name": "KeithJonesJr", "lang": "en", "location": "(D)MV, Atlanta, Brooklyn", "create_at": date("2008-08-18"), "description": "Public Servant, Political Adviser, Social Activist for the LGBT movement, and MOREHOUSE Man '08. #RGODC #Leo Snapchat: Cuteblkguy", "followers_count": 5617, "friends_count": 3978, "statues_count": 237598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookhaven, GA", "id": "00c26afc77ee7aaa", "name": "Brookhaven", "place_type": "city", "bounding_box": rectangle("-84.348581,33.825123 -84.28156,33.923964") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1310944, "cityName": "Brookhaven" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029859298365440, "text": "Wind 1.0 mph WNW. Barometer 1022.42 mb, Steady. Temperature 52.4 °F. Rain today 0.01 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029859562573824, "text": "03:00 58.5°F Feels:58.1°F (Hi61.9°F/Lo58.5°F) Hum:86% Wnd:W 5.0MPH Baro:29.71in. Prcp:0.00in https://t.co/mEzzAZSI2h #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 199, "friends_count": 262, "statues_count": 29167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029859629830144, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Rising slowly. Temperature 50.5 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 32340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029859780825088, "text": "Time to do things better ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535843704, "name": "April 21st", "screen_name": "1YoungRichie", "lang": "en", "location": "WxstDistrvct™", "create_at": date("2012-03-24"), "description": "Music Artist: For Booking & Features| YoungRichie4@gmail.com| Broke Reality To Rich Dreams #DGMG", "followers_count": 1281, "friends_count": 1096, "statues_count": 21146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, OH", "id": "8d742fb555fbff21", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-84.307688,39.695193 -84.093044,39.865523") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3921000, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029859990573057, "text": "Dude no. Lmao I just want to be spoiled with food https://t.co/VTH881O17B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 560643402, "name": "Ricc ✨", "screen_name": "RiccFoxxx", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-04-22"), "description": "ya future navy sailor ⚓️", "followers_count": 1045, "friends_count": 629, "statues_count": 32782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-02T00:00:00.000Z"), "id": 727029860623732737, "text": "@ebbtideapp Tide in Jekyll Island Marina, Georgia 05/02/2016\nHigh 5:20am 7.4\n Low 12:00pm 0.2\nHigh 5:51pm 7.6\n Low 12:28am 0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.4233,31.0567"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 29291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13039, "countyName": "Camden" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029860967682048, "text": "Happy Birthday Sue! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 632413625, "name": "Mik", "screen_name": "noxiious_", "lang": "en", "location": "Peoria AZ ✈️ Portland OR", "create_at": date("2012-07-10"), "description": "rigamortus #Edgy2K16", "followers_count": 2110, "friends_count": 1697, "statues_count": 7310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029860984455168, "text": "Busy money moves tomorrow morning and more job applications to do so night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144263892, "name": "Bri", "screen_name": "Bri_annaa_", "lang": "en", "location": "Arizona, USA", "create_at": date("2010-05-15"), "description": "stay humble. grind & pray ♻️|20 year old| GOD 1st| Grand Canyon University'18| Cali ✈️Az", "followers_count": 487, "friends_count": 638, "statues_count": 22737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029861038989312, "text": "@LexingtonDrive_ that's when you find out who you with is real or not tho so it's needed", "in_reply_to_status": 727029592427499520, "in_reply_to_user": 299851446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 299851446 }}, "user": { "id": 90047402, "name": "Badruuu' ✌️", "screen_name": "omobaDRU", "lang": "en", "location": "stroking ya girl nostrels", "create_at": date("2009-11-14"), "description": "I'll make it some way. IG: _urdab The House #BonnetBoyz YorubaLand", "followers_count": 956, "friends_count": 900, "statues_count": 22642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029861072543744, "text": "who tf is up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1739048142, "name": "issa", "screen_name": "izuhhbelluhh", "lang": "en", "location": "null", "create_at": date("2013-09-06"), "description": "a positive life is a happy life/basha", "followers_count": 1410, "friends_count": 661, "statues_count": 9301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029861097721856, "text": "Picaditas\nMaiz\n#Morelos \n#Cuernavaca\n#Mexico\n#Comida\n#Tlaquiltenango… https://t.co/wiPL955fL2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.15157361,34.01930167"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Morelos", "Cuernavaca", "Mexico", "Comida", "Tlaquiltenango" }}, "user": { "id": 590516940, "name": "Tacos Cuernavaca .", "screen_name": "TacosCuernavaca", "lang": "es", "location": "5511 E , WHITTIER BLV CP.90022", "create_at": date("2012-05-25"), "description": "No Somos Los Unicos. Pero Si Los Mejores! We Are Not The Only Ones. But We Are The Best!", "followers_count": 198, "friends_count": 469, "statues_count": 2498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029861378875393, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Falling slowly. Temperature 66.6 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029861726953472, "text": "I guess no one is showing up for work today.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 73504918, "name": "Ashli", "screen_name": "AvengedAshli", "lang": "en", "location": "Cincinnati ", "create_at": date("2009-09-11"), "description": "20. Heavy music. Animal enthusiast. Marvel & Supernatural nerd. Vet Tech Major. SLFL 7/26", "followers_count": 996, "friends_count": 904, "statues_count": 25006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029862020444162, "text": "3 https://t.co/LrrJbAYjbS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1149823844, "name": "Lisette", "screen_name": "lisettemendoza_", "lang": "en", "location": "Bay area", "create_at": date("2013-02-04"), "description": "null", "followers_count": 674, "friends_count": 278, "statues_count": 14003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029862054125568, "text": "Ripley SW Limestone Co. Temp: 61.5°F Wind:0.0mph Pressure: 994.8mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029862179868673, "text": "In Jesus' Name I Pray", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2930556421, "name": "Saint", "screen_name": "D3RYC", "lang": "en", "location": "The 9", "create_at": date("2014-12-14"), "description": "blame the city, im a product of it", "followers_count": 121, "friends_count": 80, "statues_count": 111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Euless, TX", "id": "5e2d6c67e728cca5", "name": "Euless", "place_type": "city", "bounding_box": rectangle("-97.15606,32.81088 -97.032953,32.881593") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4824768, "cityName": "Euless" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029862985138176, "text": "I had sleep paralysis for the first like like 2 months ago and FUCK NO. Scariest shit ever. https://t.co/965lLyqEf0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2418321284, "name": "jada", "screen_name": "Jayduuh_", "lang": "en", "location": "chole", "create_at": date("2014-03-29"), "description": "gtfo", "followers_count": 224, "friends_count": 339, "statues_count": 6374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soledad, CA", "id": "642fbca6eea15240", "name": "Soledad", "place_type": "city", "bounding_box": rectangle("-121.338707,36.410518 -121.306524,36.445593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 672520, "cityName": "Soledad" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029863148826626, "text": "Temp 41.3° Hi/Lo 41.3/40.5 Rng 0.8° WC 41.3° Hmd 96% Rain 0.02\" Storm 0.08\" BAR 30.095 Falling DP 40.2° Wnd 0mph Dir --- Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 67, "friends_count": 120, "statues_count": 18360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029863308136448, "text": "All we have is time , what are you doing with yours ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229973250, "name": "Just Dom", "screen_name": "RealDMurray", "lang": "en", "location": "Denver, CO", "create_at": date("2010-12-23"), "description": "MSU Denver", "followers_count": 2360, "friends_count": 1572, "statues_count": 17337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029863542968324, "text": "Have you ever experienced a life review? (That is, has your life ever \"flashed before your eyes?\")", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324685664, "name": "Alex Yandell", "screen_name": "snowbird3097", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2011-06-26"), "description": "no batteries included and no strings attached", "followers_count": 141, "friends_count": 323, "statues_count": 1034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029863777992708, "text": "LMAO this all you Plainfield hoes https://t.co/YlWV1zZofi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2566063684, "name": "madison", "screen_name": "madisonnycole_", "lang": "en", "location": "815.", "create_at": date("2014-05-26"), "description": "In My Glo ✨ sc :madisonn.webb", "followers_count": 1290, "friends_count": 594, "statues_count": 60938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029863970807809, "text": "@_Clion_ let smoke", "in_reply_to_status": 727028505817440256, "in_reply_to_user": 479936808, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 479936808 }}, "user": { "id": 443458162, "name": "B-LU", "screen_name": "NikoNardo", "lang": "en", "location": "null", "create_at": date("2011-12-21"), "description": "null", "followers_count": 581, "friends_count": 500, "statues_count": 26714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Collins, CO", "id": "b2e4e65d7b80d2c1", "name": "Fort Collins", "place_type": "city", "bounding_box": rectangle("-105.148074,40.47168 -104.979811,40.656701") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8069, "countyName": "Larimer", "cityID": 827425, "cityName": "Fort Collins" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029864222576640, "text": "I refuse to believe that it's difficult for women to get sex no matter what they tell me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148891868, "name": "SOLAECLIPSE", "screen_name": "DrinkSolaPop", "lang": "en", "location": "A train to Far Rockaway. NYC.", "create_at": date("2010-05-27"), "description": "Na⚓️vY", "followers_count": 1244, "friends_count": 692, "statues_count": 35444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029864327323648, "text": "Wind 2.0 mph WNW. Barometer 30.108 in, Steady. Temperature 47.1 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029864381960192, "text": "Wind 1.0 mph WNW. Barometer 30.033 in, Rising. Temperature 53.0 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-02T00:00:01.000Z"), "id": 727029864482639872, "text": "05/02@03:00 - Temp 50.7F, WC 50.7F. Wind 1.5mph N, Gust 4.0mph. Bar 30.022in, Falling. Rain 0.01in. Hum 99%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029864927125504, "text": "I love making you happy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4018754473, "name": "Salvador Perez", "screen_name": "Sxxlvi_", "lang": "en", "location": "null", "create_at": date("2015-10-25"), "description": "null", "followers_count": 178, "friends_count": 193, "statues_count": 720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029865111670784, "text": "@ReenzKev lmao we on that hype again", "in_reply_to_status": 727029692117602304, "in_reply_to_user": 412877924, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 412877924 }}, "user": { "id": 360283512, "name": "Dominic", "screen_name": "hteam_domzz", "lang": "en", "location": "null", "create_at": date("2011-08-22"), "description": "Bouncer in SF down to SJ", "followers_count": 652, "friends_count": 627, "statues_count": 9571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029865233395712, "text": "Temp: 65.0°F Wind:0.0mph Pressure: 29.966hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029865870815233, "text": "I wanna kms https://t.co/6qfEIqcKWZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995796931, "name": "julie", "screen_name": "idioxsyncratic", "lang": "en", "location": "SLC", "create_at": date("2012-12-07"), "description": "18. out of mind, out of sight.", "followers_count": 1082, "friends_count": 1395, "statues_count": 39554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029865988263936, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":65.7°F Wind:0.9mph Pressure: 29.94hpa Falling slowly Rain Today 0.00in. Forecast: Unsettled, precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 316785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866265206784, "text": "If yo girl get your attention and say \"come here , look at my phone and tell me what this say\" \n\nRUN MY NIGGA !!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324051504, "name": "still Quen", "screen_name": "theOnlyQuen_PHE", "lang": "en", "location": "Dallas / SFA ", "create_at": date("2011-06-25"), "description": "#PHE - Welcome to Hustle University", "followers_count": 2709, "friends_count": 1031, "statues_count": 148715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866286223362, "text": "@iHustle1108 idk I fucks wit it", "in_reply_to_status": 727029771977125888, "in_reply_to_user": 31245235, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31245235 }}, "user": { "id": 415920039, "name": "Hoodrich Pablo", "screen_name": "OlDirtyPablo", "lang": "en", "location": "FL ", "create_at": date("2011-11-18"), "description": "vibes. #savageszn #shuttersquad", "followers_count": 2254, "friends_count": 813, "statues_count": 319902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866420281344, "text": "@brokenpjieces its midnight here, so happy birthday babe ☺", "in_reply_to_status": -1, "in_reply_to_user": 2889743287, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2889743287 }}, "user": { "id": 703676805, "name": "Julie//68 DAYS♡", "screen_name": "starbeauks", "lang": "en", "location": "She/Her", "create_at": date("2013-10-07"), "description": "Too many fandom's ! Zquad *ೃ༄ @JustinBieber My hero followed March 22, 2015 12:05pm| |8/12+Dolans|", "followers_count": 26883, "friends_count": 8926, "statues_count": 56662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866529361920, "text": "64.6F (Feels: 64.6F) - Humidity: 99% - Wind: 3.8mph SE - Gust: 3.8mph - Pressure: 1011.7mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866705653760, "text": "Dam marken got drafted to Vikings as wide receiver ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1063987070, "name": "fvckitup_cali™", "screen_name": "MARCOBOI6", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "Mind on my riches. 〽️⭕️B 19&up 305B&R fwm on ig: @str8cali_06 1⃣5⃣1⃣ #freesaquan#longlivepapi #rip dre", "followers_count": 215, "friends_count": 354, "statues_count": 9738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029866978234368, "text": "#krose2020 @krose2020 #cumulusclouds #white #blue #you #upintheair… https://t.co/jMGhiCgIEy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0522"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "krose2020", "cumulusclouds", "white", "blue", "you", "upintheair" }}, "user_mentions": {{ 24496545 }}, "user": { "id": 24496545, "name": "KYRON", "screen_name": "KRose2020", "lang": "en", "location": "iPhone: 42.982021,-78.893906", "create_at": date("2009-03-14"), "description": "null", "followers_count": 139, "friends_count": 109, "statues_count": 623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029867078926336, "text": "I love Spika & Sponge from James & The Giant Peach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 133151785, "name": "BEYS LEMON HEAD ASS!", "screen_name": "AntiJay_Fenty", "lang": "en", "location": "ANTI", "create_at": date("2010-04-14"), "description": "Explicit Gay! #Nhevi\nBuy @rihanna new singles\n#Work #KissItBetter #NeededMe \non iTunes. #AntiWorldTour date 4/09/16✨ \n#ANTI IS OUT NOW!⚓\n#TeamMya", "followers_count": 562, "friends_count": 579, "statues_count": 43351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029867099774976, "text": "@OhHaiRico fuck you're being real funny right now ��������", "in_reply_to_status": 727026415506325504, "in_reply_to_user": 287990353, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 287990353 }}, "user": { "id": 3012764257, "name": "Hails", "screen_name": "celiwest", "lang": "en", "location": "California, USA", "create_at": date("2015-02-07"), "description": "narry forever.", "followers_count": 155, "friends_count": 167, "statues_count": 12497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029867489939457, "text": "Wind 1.0 mph WNW. Barometer 29.907 in, Rising slowly. Temperature 63.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029867645116416, "text": "Wind 1.6 mph SE. Barometer 30.04 in, Falling slowly. Temperature 45.5 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029867863134208, "text": "This girl hella look like Wonder Woman though", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39091280, "name": "RETIRED", "screen_name": "IamAfricaBlack", "lang": "en", "location": "Oakland, CA", "create_at": date("2009-05-10"), "description": "Go download rich niggas don't wear socks 2 below.", "followers_count": 677, "friends_count": 499, "statues_count": 114518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029868081225728, "text": "I don't care how cheesy that shit sounds, it's true AF.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170572307, "name": "Enrique", "screen_name": "KikeAzul95", "lang": "en", "location": "Modesto, CA", "create_at": date("2010-07-24"), "description": "Habits reflect the mission. #WOYG #NinerNation #LakerNation #CarpeDiem #TeamVisionary #XX", "followers_count": 339, "friends_count": 830, "statues_count": 45118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029868353855488, "text": "ill never be able to sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572148988, "name": "colby", "screen_name": "cxlby", "lang": "en", "location": "DFW", "create_at": date("2012-05-05"), "description": "im a mess", "followers_count": 1622, "friends_count": 517, "statues_count": 17896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-02T00:00:02.000Z"), "id": 727029868567781376, "text": "Redemption on repeat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2194738993, "name": "Kodak Storm", "screen_name": "iAmSavage__TSOE", "lang": "en", "location": "Westside ", "create_at": date("2013-11-14"), "description": "#AOB #EatGoodENT #RIPUnc #RIPGranny #Westside #WUVU #OPM", "followers_count": 2107, "friends_count": 1242, "statues_count": 47267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869092196352, "text": "...bitch ew https://t.co/plEfvuZ53F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304439418, "name": "patriciapopthatpussy", "screen_name": "iRap4Food", "lang": "en", "location": "Richmond, VA", "create_at": date("2011-05-24"), "description": "vcu'19 | mass communications | hc vcu editorial team | online radio show [sundays-3 pm]", "followers_count": 2086, "friends_count": 453, "statues_count": 121679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869477957632, "text": "Yay just got home now time to knock the heck out!!����������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4051452313, "name": "Alex Hernandez", "screen_name": "alexh2342", "lang": "en", "location": "California, USA", "create_at": date("2015-10-28"), "description": "Proud to be Christian- \nsoccer-\nfcb-\n E❤", "followers_count": 97, "friends_count": 186, "statues_count": 1080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869746384897, "text": "@Sandra_Juarez goodnight", "in_reply_to_status": 727029659196485632, "in_reply_to_user": 264957363, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 264957363 }}, "user": { "id": 173269316, "name": "OoCap", "screen_name": "CVPSULE", "lang": "en", "location": "Austin, TX", "create_at": date("2010-07-31"), "description": "21 | MGMT & BOOKINGS : Contact Derek Gonzales cptv.promo@gmail.com IN AFFILIATION W/@GoodVibesPrsnts hmu for show info and banners", "followers_count": 2243, "friends_count": 87, "statues_count": 124679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869771677697, "text": "Wind 0.0 mph ---. Barometer 29.922 in, Steady. Temperature 62.9 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 62906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869830262784, "text": "oh my �� https://t.co/78LU6W5U0J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 8000812, "name": "Andrew S.", "screen_name": "shoutingboy", "lang": "en", "location": "Silicon Valley and environs", "create_at": date("2007-08-06"), "description": "Halfway from hither to yon", "followers_count": 495, "friends_count": 301, "statues_count": 20508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029869847007232, "text": "Fumble / Trey Songz ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 550527068, "name": "#VivianYourFriend", "screen_name": "BarrazaVivian", "lang": "en", "location": "null", "create_at": date("2012-04-10"), "description": "IG & phhhoto : freakinnveee", "followers_count": 455, "friends_count": 441, "statues_count": 19695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029870182584320, "text": "My eyes can barely stay open but I'm still on Twitter ������ https://t.co/ZXpoSCqotD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36508530, "name": "Ashhhh⚜", "screen_name": "PrincessAshhley", "lang": "en", "location": "San Marcos,TX", "create_at": date("2009-04-29"), "description": "Phil. 4:13 • Texas State 2017•Criminal Justice Major • Princess by Nature• ♥️⭐️☀️• SC: @ Ashley.Seeniorr #PILLOWTALK •", "followers_count": 1340, "friends_count": 1017, "statues_count": 17009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029870287421441, "text": "you put me on a feel that I never had", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322396179, "name": "Raul Alvarado", "screen_name": "RawlFucksYou", "lang": "en", "location": "Downey, CA", "create_at": date("2011-06-22"), "description": "CSULA", "followers_count": 615, "friends_count": 365, "statues_count": 31886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029870815911936, "text": "Lowkey I hate u now hahaha so dumb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103486997, "name": "expen$ive chola", "screen_name": "vanessaThief", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-01-09"), "description": "all i need is a co-pilot.", "followers_count": 1040, "friends_count": 1352, "statues_count": 94370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029870878924801, "text": "@Cara_Chea massages and spa stuff ��������������", "in_reply_to_status": -1, "in_reply_to_user": 40308139, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40308139 }}, "user": { "id": 23032503, "name": "Lauren Jacobs", "screen_name": "laurenjacobsnet", "lang": "en", "location": "atlanta , cincy , LA ", "create_at": date("2009-03-05"), "description": "For booking Lauren.Jacobs4@gmail.com. SAG actress. #virgo Atlanta /Cincinnati/ Los Angeles", "followers_count": 1945, "friends_count": 514, "statues_count": 12750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029870987898880, "text": "Wtf joe Jonas sings cake by the ocean. Didn't know it was by dnce ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25154309, "name": "Giselle ϟ", "screen_name": "Moricetsz", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-03-18"), "description": "this is where i come to complain and stalk yo ass! #FANGIRL. (BEWARNED) I hope no one I know finds this.", "followers_count": 567, "friends_count": 135, "statues_count": 17315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029871403130884, "text": "Can't stay in little ole Racine forever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2282717508, "name": "#JU_FOR_PROMKING2K18", "screen_name": "Julian_Miller00", "lang": "en", "location": "262", "create_at": date("2014-01-08"), "description": "Only thing I speak is the truth & my opinion on life & how things work #Loc2nem", "followers_count": 1291, "friends_count": 655, "statues_count": 42706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029871503806464, "text": "Please don't take the idea of \"upgrading\" as finding someone who is better looking, #breakupadvice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "breakupadvice" }}, "user": { "id": 311158275, "name": "Hector Aguirre", "screen_name": "hd_shine", "lang": "en", "location": "209,California. In God's Hands", "create_at": date("2011-06-04"), "description": "The truth will set you free", "followers_count": 735, "friends_count": 340, "statues_count": 26605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029871549935616, "text": "oh beybey yes https://t.co/RXIJCEMbNm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613972179, "name": "halona†", "screen_name": "sydneyhelme_", "lang": "en", "location": "anchorage, AK", "create_at": date("2012-06-20"), "description": "just another fat bitch from anchorage", "followers_count": 1349, "friends_count": 478, "statues_count": 28299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029871801552896, "text": "Wind 3.2 mph NNW. Barometer 30.027 in, Falling slowly. Temperature 60.5 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029871906430977, "text": "@Kingly_Strong96 then stop", "in_reply_to_status": 727028934139777024, "in_reply_to_user": 347667958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 347667958 }}, "user": { "id": 632573473, "name": "Zar ♡", "screen_name": "zarzar_15", "lang": "en", "location": "null", "create_at": date("2012-07-10"), "description": "•19•expect nothing, appreciate everything•curly hair enthusiast •fierce all stars• Natural hair videos•", "followers_count": 688, "friends_count": 571, "statues_count": 26469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charter Oak, CA", "id": "07432bdb1901f862", "name": "Charter Oak", "place_type": "city", "bounding_box": rectangle("-117.877463,34.093451 -117.837641,34.106755") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612734, "cityName": "Charter Oak" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872137134080, "text": "everytime I see someone with a Misfits shirt I feel compelled to ask them what era is their fave, & which horror film they like. Weird urges", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88610909, "name": "Cristina Rose", "screen_name": "lunarprisms", "lang": "en", "location": "Los Angeles", "create_at": date("2009-11-08"), "description": "I love nature, animals & sustain myself on coffee & paint. Animal & Environmental Activist. Book Worm. Poison Ivy is my hero.\nContact: cristinaroseart@gmail.com", "followers_count": 736, "friends_count": 447, "statues_count": 17688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872363630593, "text": "@_iamericb get ready for the cold �� it's not so much the city as it is my school!", "in_reply_to_status": 727029141602594816, "in_reply_to_user": 432228394, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 432228394 }}, "user": { "id": 220553282, "name": "cel", "screen_name": "ABCeleste_", "lang": "en", "location": "CA", "create_at": date("2010-11-27"), "description": "my flawless life with no problems: a story of perfection", "followers_count": 688, "friends_count": 247, "statues_count": 39844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872384593920, "text": "31 days ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4148184918, "name": "Xia Lucci ❣", "screen_name": "XiasSweets", "lang": "en", "location": "null", "create_at": date("2015-11-06"), "description": "20. Gemini. SnapChat: Xia_Lee | Facebook: Xia Sovereign | Instagram: XiTheDifference", "followers_count": 644, "friends_count": 602, "statues_count": 4462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872422342656, "text": "3693126b00p84CE7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-163.110687,60.30519"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 20, "friends_count": 0, "statues_count": 17943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2050, "countyName": "Bethel" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872757854208, "text": "@breeezybri_ she a baddie ? Tell her to hml pls ��", "in_reply_to_status": 727028321641320448, "in_reply_to_user": 4674543793, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4674543793 }}, "user": { "id": 586338438, "name": "Efrencito", "screen_name": "efriend_", "lang": "en", "location": "trap house", "create_at": date("2012-05-20"), "description": "was good", "followers_count": 681, "friends_count": 685, "statues_count": 26506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029872841904128, "text": "can't tweet much because my tweets offend certain people ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3116282809, "name": "$helle", "screen_name": "RAYSHELLEEEE", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "I love you uncle Shaun", "followers_count": 1012, "friends_count": 811, "statues_count": 11545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery Village, MD", "id": "017a75c0406737c7", "name": "Montgomery Village", "place_type": "city", "bounding_box": rectangle("-77.231293,39.152407 -77.112229,39.235691") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2453325, "cityName": "Montgomery Village" } }
+{ "create_at": datetime("2016-05-02T00:00:03.000Z"), "id": 727029873105989632, "text": "@_dinora_e haha im done ��", "in_reply_to_status": 727029616993427456, "in_reply_to_user": 595935083, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 595935083 }}, "user": { "id": 424791442, "name": "Rico Jr.", "screen_name": "HardTimesOf_RJ", "lang": "en", "location": "null", "create_at": date("2011-11-29"), "description": "Im just trying to live in the moment", "followers_count": 1194, "friends_count": 710, "statues_count": 48115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029873718382592, "text": "Nexus 6P International #Giveaway @androidauth https://t.co/Z5JfxEfDH7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Giveaway" }}, "user_mentions": {{ 95438524 }}, "user": { "id": 830138276, "name": "Eric Munoz", "screen_name": "EricMunoz98", "lang": "en", "location": "null", "create_at": date("2012-09-17"), "description": "What is happiness", "followers_count": 319, "friends_count": 307, "statues_count": 4472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029873978527744, "text": "Up like I don't have class in the morning ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 403825849, "name": "Chelsss.", "screen_name": "CheeChelsea", "lang": "en", "location": "null", "create_at": date("2011-11-02"), "description": "• CU ⚽️ • State Registered Nurse Aide ❤️ • Live life with no regrets ✨ |19| Instagram: callmechels", "followers_count": 363, "friends_count": 344, "statues_count": 4330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Campbellsville, KY", "id": "a0cf9725c14c88f3", "name": "Campbellsville", "place_type": "city", "bounding_box": rectangle("-85.382243,37.320508 -85.317,37.387589") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21217, "countyName": "Taylor", "cityID": 2112160, "cityName": "Campbellsville" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029874129391616, "text": "Healing, growing, loving.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 64244900, "name": "Alexandra Katherine", "screen_name": "allyruss55", "lang": "en", "location": "Aruba", "create_at": date("2009-08-09"), "description": "4 amazing lil munchkins call me Auntie❤️", "followers_count": 136, "friends_count": 117, "statues_count": 1054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, CO", "id": "72a91b4570317b68", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-105.190246,39.957602 -105.070651,40.0237") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 841835, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029874179723264, "text": "Happy birthday @jojoyo_ hope you have a wonderful day ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1166377831 }}, "user": { "id": 3920325019, "name": "Carisma", "screen_name": "carxsmaa_", "lang": "en", "location": "sleeping", "create_at": date("2015-10-16"), "description": "achs; c/o 2016", "followers_count": 159, "friends_count": 166, "statues_count": 1766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029874536251392, "text": "gahhhh damn https://t.co/0uNrkGaW3p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1167515984, "name": "lolo", "screen_name": "laurynbravo", "lang": "en", "location": "null", "create_at": date("2013-02-10"), "description": "o rlly?", "followers_count": 1197, "friends_count": 308, "statues_count": 14672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029874641256449, "text": "Oh, me? Just hanging with my s/heroes! Thank you #CollaborativeforUrbanAgroecologyLosAngeles for… https://t.co/mNMuc5WROg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2706,34.0315"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CollaborativeforUrbanAgroecologyLosAngeles" }}, "user": { "id": 38086499, "name": "Sarah Marie Leone", "screen_name": "sarahmleone", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-05-05"), "description": "I facilitate and participate in community building, gardening, thinking critically, and the creation of art and science.", "followers_count": 499, "friends_count": 1446, "statues_count": 684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029874875977728, "text": "Pass the herb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1297723850, "name": "OG Angel™", "screen_name": "chilllfam_", "lang": "en", "location": "Chicago", "create_at": date("2013-03-24"), "description": "occasional photographer and graphic designer, aviation is life, full time savage working with @royal_side |20|", "followers_count": 280, "friends_count": 228, "statues_count": 18790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schiller Park, IL", "id": "7c9d51a2fd0a7552", "name": "Schiller Park", "place_type": "city", "bounding_box": rectangle("-87.89071,41.94547 -87.846661,41.973297") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1768081, "cityName": "Schiller Park" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029875031216129, "text": "@NatbyNature Was really hoping you would become champ. Great seeing Bret. Charlotte doesn't make a very good heal.", "in_reply_to_status": 727029011193335808, "in_reply_to_user": 89014429, "favorite_count": 0, "coordinate": point("-79.83845436,42.21047081"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 89014429 }}, "user": { "id": 188913242, "name": "Wrestling Fan", "screen_name": "Gimpy002", "lang": "en", "location": "In Your Dreams", "create_at": date("2010-09-09"), "description": "null", "followers_count": 49, "friends_count": 543, "statues_count": 1280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North East, PA", "id": "f544b3de41f09319", "name": "North East", "place_type": "city", "bounding_box": rectangle("-79.855165,42.192986 -79.811061,42.236074") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4254952, "cityName": "North East" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029875492589568, "text": "Most of des bitches be skummy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193145118, "name": "TRVPKING.", "screen_name": "VoaanieToSmooth", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2010-09-20"), "description": "55 \n\n\n\nBOOKINGS:\nvoaanietosmooth@gmail.com sac ✈ vegas", "followers_count": 1167, "friends_count": 478, "statues_count": 88076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029875647766528, "text": "Less than 20 days for my admission exam, so fucking nervous...... #YoQuieroSerMedico", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "YoQuieroSerMedico" }}, "user": { "id": 711704005, "name": "Quetzi C", "screen_name": "quetzi_cornejo", "lang": "en", "location": "Pueblo Colorado", "create_at": date("2012-07-22"), "description": "Time to try and change life style. Enjoy every time. 17 years México, United States. Follow by: Sebas Arango y Screamau", "followers_count": 769, "friends_count": 1255, "statues_count": 8787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pueblo, CO", "id": "9d7b47e751be1551", "name": "Pueblo", "place_type": "city", "bounding_box": rectangle("-104.69356,38.200638 -104.551992,38.338462") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8101, "countyName": "Pueblo", "cityID": 862000, "cityName": "Pueblo" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029876146987008, "text": "Detail of a work featured in Metropolitan Arts Institute Student Showcase Coming to Bokeh… https://t.co/n4f8uTgFko", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.07048,33.45899"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1066820382, "name": "Nicole Royse", "screen_name": "NicoleRoyseArt", "lang": "en", "location": "Arizona", "create_at": date("2013-01-06"), "description": "Artist, Curator Shade Projects @themonOrchid, Freelance Writer, Wife, Mama of 3!", "followers_count": 3841, "friends_count": 3367, "statues_count": 43964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029876155371520, "text": "Wind 0.0 mph ---. Barometer 30.024 in, Falling. Temperature 41.4 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029876167864320, "text": "You did it to yo self \nNow you're all by yourself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 629132160, "name": "wildfox", "screen_name": "chaneldnglsn", "lang": "en", "location": "probably at the hospital LOL ", "create_at": date("2012-07-06"), "description": "people eventually stop asking if you're okay", "followers_count": 349, "friends_count": 554, "statues_count": 18450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029876197216257, "text": "Happy birthday @_Darrenhall you my brother, my ride or die... I hope you have a blessed one family������ https://t.co/Bu5a1K6R4C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1711699148 }}, "user": { "id": 2690356503, "name": "Quarterback", "screen_name": "zen_gonzales9", "lang": "en", "location": "Somewhere getting better", "create_at": date("2014-07-09"), "description": "Quarterback @ Rancho C/o 17. ||Breedup", "followers_count": 846, "friends_count": 968, "statues_count": 58 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029876679577600, "text": "Be with someone who will take care of you. Not materialistically, but take care of your soul, your well being, your heart, & everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119909274, "name": "Jay Cabanes ", "screen_name": "Jay_Cabanes7", "lang": "en", "location": "Community of Valencia, Spain", "create_at": date("2010-03-04"), "description": "To Live would be An awfully Big Adventure.", "followers_count": 116, "friends_count": 90, "statues_count": 11898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-05-02T00:00:04.000Z"), "id": 727029877136723968, "text": "appreciate those people that still try to be funny & make you laugh while they're going thru some stuff and are sad themselves", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2356567969, "name": "n a t a l i e", "screen_name": "xolovelynatalie", "lang": "en", "location": "null", "create_at": date("2014-02-22"), "description": "passionate about my goals, hbu?", "followers_count": 359, "friends_count": 264, "statues_count": 6099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Valley, CA", "id": "0196f2d33e5a1d73", "name": "French Valley", "place_type": "city", "bounding_box": rectangle("-117.137136,33.564564 -117.083609,33.627328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 626067, "cityName": "French Valley" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029877577240576, "text": "Monday 5-2-16 https://t.co/TMyxDBcRSZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1139, "friends_count": 321, "statues_count": 3573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029877782646785, "text": "@MazeOscar LOL", "in_reply_to_status": 727016558740967424, "in_reply_to_user": 1608605232, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1608605232 }}, "user": { "id": 134577480, "name": "E.", "screen_name": "eveiscruzzin", "lang": "en", "location": "Southern California ", "create_at": date("2010-04-18"), "description": "I accept apologies through PayPal", "followers_count": 414, "friends_count": 172, "statues_count": 23067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escondido, CA", "id": "00610f4d3a382ec1", "name": "Escondido", "place_type": "city", "bounding_box": rectangle("-117.146102,33.057782 -116.998535,33.182936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622804, "cityName": "Escondido" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878004977666, "text": "Ebay: https://t.co/t7ljUpXxaL\n\nFacebook: https://t.co/ZOAd0Nbsw7\n\n#Romance #faith #mayday #diamonds #engagementrings https://t.co/DWKz5a4THG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Romance", "faith", "mayday", "diamonds", "engagementrings" }}, "user": { "id": 505363635, "name": "Laura Adkism", "screen_name": "LauraAdkism", "lang": "en", "location": "United States", "create_at": date("2012-02-26"), "description": "A goal without a plan is just a wish.\n\n~ Antoine de Saint-Exuper", "followers_count": 858, "friends_count": 851, "statues_count": 5246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878088962048, "text": "take deep Breffs https://t.co/FRhGEgd6M4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 42135056, "name": "Hero", "screen_name": "TRENding_Topicc", "lang": "en", "location": "#JxnSU 17", "create_at": date("2009-05-23"), "description": "U.S Army R.I.P Dad & Grandpa ❤️. i love what i do. i do what i love", "followers_count": 1960, "friends_count": 1635, "statues_count": 145765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878260916224, "text": "floral shorts https://t.co/oIKd43RfA0 #ontheblog #blogger #fashion #fashioblogger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.7524482,39.4717184"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontheblog", "blogger", "fashion", "fashioblogger" }}, "user": { "id": 65940718, "name": "Brett Elizabeth", "screen_name": "BrettEParker", "lang": "en", "location": "NYC", "create_at": date("2009-08-15"), "description": "i ❤ nyc.", "followers_count": 143, "friends_count": 226, "statues_count": 2181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ohio, USA", "id": "de599025180e2ee7", "name": "Ohio", "place_type": "admin", "bounding_box": rectangle("-84.820309,38.403186 -80.518626,42.327133") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878319644672, "text": "Customer Review Mantras https://t.co/EKhytd3cwE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322443544, "name": "Delforge + Co", "screen_name": "delforgeandco", "lang": "en", "location": "London", "create_at": date("2011-06-22"), "description": "Consultancy agency and Business Academy specialising in the Hair and Beauty industry", "followers_count": 316, "friends_count": 285, "statues_count": 355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878353088513, "text": "You been on my mind lately ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 844848776, "name": "Juan-800", "screen_name": "Juan_ocho5", "lang": "en", "location": "Riverdale", "create_at": date("2012-09-24"), "description": "/ Life's a trip / Snapchat : @Juanito_FPR /\n#FCC / #PreludeNation / Car Enthusiast /", "followers_count": 502, "friends_count": 345, "statues_count": 21998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverdale, CA", "id": "0028563c445e3c67", "name": "Riverdale", "place_type": "city", "bounding_box": rectangle("-119.878071,36.423728 -119.845512,36.438219") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 661096, "cityName": "Riverdale" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878424489985, "text": "Suede Fringe Skirt https://t.co/bSjtQPxG8e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "da", "is_retweet": false, "user": { "id": 150926661, "name": "ℒiz ℒauren", "screen_name": "Disc0liz", "lang": "en", "location": "Bel Air, CA/Paris, France", "create_at": date("2010-06-01"), "description": "Fashion Blogger. #Liz\nFated to Pretend.", "followers_count": 1147, "friends_count": 1134, "statues_count": 9048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029878747340801, "text": "I haven't slept all weekend haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157714234, "name": "Carti $", "screen_name": "Desireeey_27", "lang": "en", "location": "Balifornia", "create_at": date("2010-06-20"), "description": "Puerto Rican / Volleyball", "followers_count": 1927, "friends_count": 1665, "statues_count": 62272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029879347118080, "text": "Haha what a rush wrote those bars down but I'm done for the night Goodnight and sweet dreams ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37480090, "name": "i'minvisible", "screen_name": "Mell0w_Le0", "lang": "en", "location": "UNKNOWN ", "create_at": date("2009-05-03"), "description": "I really am invisible, no one notice what I say or do it's kinda cool", "followers_count": 159, "friends_count": 82, "statues_count": 10174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029880035151872, "text": "It's not one day that goes by that I walk in the bathroom and it's not dirty ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506538345, "name": "✨QueenBree✨", "screen_name": "NiggasLove_Bree", "lang": "en", "location": "null", "create_at": date("2012-02-27"), "description": "M. Carroll ❤️ | #Miles19| 334 ✈️205", "followers_count": 1081, "friends_count": 1144, "statues_count": 26996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, AL", "id": "955c49d6773f00f0", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-86.951217,33.451056 -86.903856,33.500963") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 125120, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029880165003267, "text": "Lil niggas just not attractive, my nigga keep me warm as hell ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702227365359808512, "name": "riley .. new account", "screen_name": "sadityri", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 99, "friends_count": 75, "statues_count": 1423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029881112940544, "text": "2am and all I'm thinking about is the gym and sleep after school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4397082708, "name": "Dave Silva", "screen_name": "Goodoletexas", "lang": "en", "location": "MERICA ", "create_at": date("2015-11-29"), "description": "No pain. No gain. SHUT UP AND TRAIN!!\n\n\n Sc:Outlaw_696969", "followers_count": 418, "friends_count": 636, "statues_count": 6590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029881201147904, "text": "@DallasFanzine Is the New Dallas coming back to TV? It was a good show @LARZIK", "in_reply_to_status": -1, "in_reply_to_user": 58827348, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 58827348, 33469319 }}, "user": { "id": 33469319, "name": "GARY LARZIK", "screen_name": "LARZIK", "lang": "en", "location": "orlando, fl.", "create_at": date("2009-04-20"), "description": "i was marred to the most beautiful lady i every set my eye's on. we were happy for 35 years, until god took her away from me my heart Still's Cry's.", "followers_count": 761, "friends_count": 2038, "statues_count": 15827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sky Lake, FL", "id": "722e939516eff1cb", "name": "Sky Lake", "place_type": "city", "bounding_box": rectangle("-81.40094,28.450325 -81.384127,28.472722") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1266425, "cityName": "Sky Lake" } }
+{ "create_at": datetime("2016-05-02T00:00:05.000Z"), "id": 727029881284882433, "text": "4 total months on Tuesday. Fuck, time sure does fly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1264867704, "name": "Satanic Fresita", "screen_name": "unholyconfessor", "lang": "en", "location": "under your bed ", "create_at": date("2013-03-13"), "description": "I like serial killers.\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nJ♡", "followers_count": 327, "friends_count": 698, "statues_count": 14882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392244345397252, "text": "When people waste my time https://t.co/Gn72m41tTh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2737857231, "name": "Jacob Ali MacAulay", "screen_name": "AliSpeedVz", "lang": "en", "location": "Sandy, OR✈️Surprise, AZ", "create_at": date("2014-08-08"), "description": "I might fall and crumble but I'll rise up and make the world rumble (R.I.P. Damien) TNDO ADAO LackOfWorries 971 ~FootBallSoccerTrack~", "followers_count": 1142, "friends_count": 987, "statues_count": 4335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392244735594496, "text": "@_UFoolayBoolay https://t.co/rf5CIctegH", "in_reply_to_status": 727392131367604225, "in_reply_to_user": 210295002, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 210295002 }}, "user": { "id": 4352656812, "name": "Jerin Allen", "screen_name": "jayhendrix95", "lang": "en", "location": "null", "create_at": date("2015-11-24"), "description": "Isaiah 54:17. #HindsTrack. 3-time NJCAA All-American. #TrackNation. 228 bred. Trying to become a man.", "followers_count": 582, "friends_count": 564, "statues_count": 11169 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raymond, MS", "id": "01c58b1145869236", "name": "Raymond", "place_type": "city", "bounding_box": rectangle("-90.431739,32.249915 -90.404367,32.267662") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2861160, "cityName": "Raymond" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392245091995648, "text": "@Nancyy_97 I always deserve a nap!!!", "in_reply_to_status": 727388684333379584, "in_reply_to_user": 760773374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 760773374 }}, "user": { "id": 1312641205, "name": "Juan", "screen_name": "_juanmrtnz", "lang": "en", "location": "California, USA", "create_at": date("2013-03-28"), "description": "null", "followers_count": 114, "friends_count": 120, "statues_count": 4868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392245867892736, "text": "@yuikawaters OMG I totally forgot about Mother's Day fml ��", "in_reply_to_status": 727391805793165313, "in_reply_to_user": 2344412568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2344412568 }}, "user": { "id": 2314204723, "name": "k e n i c e ♌️", "screen_name": "keniceluv", "lang": "en", "location": "sd / murrieta ", "create_at": date("2014-01-27"), "description": "@savannahdeonne @JadaWoolsey", "followers_count": 798, "friends_count": 448, "statues_count": 30049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246031634432, "text": "Tiger been sleep for like 2 hours now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3322170009, "name": "April30♉️", "screen_name": "nicoleash__", "lang": "en", "location": "iup/610", "create_at": date("2015-06-12"), "description": "#behumble #havefaith", "followers_count": 304, "friends_count": 257, "statues_count": 6840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norristown, PA", "id": "6b4972f8e32f4e32", "name": "Norristown", "place_type": "city", "bounding_box": rectangle("-75.365138,40.105217 -75.31664,40.141599") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4254656, "cityName": "Norristown" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246081966080, "text": "Who daughter don't know Allah ? ���� cause mines will break out in salat soon as the athan go off ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2757131393, "name": "GianiVsVersace❤", "screen_name": "lilmissversace", "lang": "en", "location": "around ", "create_at": date("2014-08-31"), "description": "DD❤️ #FutureRCST", "followers_count": 709, "friends_count": 696, "statues_count": 25450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246249586688, "text": "@ebbtideapp Tide in Green Island, New York 05/03/2016\nHigh 6:35am 2.1\n Low 1:15pm -0.1\nHigh 7:09pm 2.3\n Low 1:51am -0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.5017,40.6233"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 29568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246316851200, "text": "@oh_daddi what about your liver? Doesn't your liver care?", "in_reply_to_status": 727392128385437696, "in_reply_to_user": 712545780918312961, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 712545780918312961 }}, "user": { "id": 3500123053, "name": "Alejandrito", "screen_name": "AlejandroJayy", "lang": "en", "location": "Tucson, AZ", "create_at": date("2015-09-08"), "description": "Educated..... not blackfished.", "followers_count": 221, "friends_count": 221, "statues_count": 1565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246333460480, "text": "Wind 0.0 mph ---. Barometer 1021.95 mb, Steady. Temperature 43.0 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246505476096, "text": "@runforkever https://t.co/yRogcuJfyP", "in_reply_to_status": 727391711333249024, "in_reply_to_user": 204085484, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 204085484 }}, "user": { "id": 60716966, "name": "Stevie, tbh.", "screen_name": "stvtbh", "lang": "en", "location": "null", "create_at": date("2009-07-27"), "description": "sometimes I get mad. so suck a dick.", "followers_count": 590, "friends_count": 603, "statues_count": 120775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzales, CA", "id": "19e04d406d643b64", "name": "Gonzales", "place_type": "city", "bounding_box": rectangle("-121.453749,36.491625 -121.426984,36.526757") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 630392, "cityName": "Gonzales" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392246681595906, "text": "update still awake", "in_reply_to_status": 727383451997892608, "in_reply_to_user": 1113109903, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1113109903, "name": "gill.", "screen_name": "gillyepperson", "lang": "en", "location": "Cleburne/Devine TX", "create_at": date("2013-01-22"), "description": "I really like outer space, probably not wearing pants // sc: itsgillianyo // #TXST20", "followers_count": 740, "friends_count": 1099, "statues_count": 14740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleburne, TX", "id": "70b538807e617654", "name": "Cleburne", "place_type": "city", "bounding_box": rectangle("-97.439627,32.298971 -97.349294,32.437917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48251, "countyName": "Johnson", "cityID": 4815364, "cityName": "Cleburne" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392247482830848, "text": "Wind 0.0 mph ---. Barometer 29.97 in, Steady. Temperature 50.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 32363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392247809867777, "text": "@RasiaZeus yes", "in_reply_to_status": 718515433914150913, "in_reply_to_user": 4167750881, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4167750881 }}, "user": { "id": 4553413093, "name": "william vann", "screen_name": "williamvann55", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "null", "followers_count": 213, "friends_count": 810, "statues_count": 2854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392248065830912, "text": "Ripley SW Limestone Co. Temp: 61.0°F Wind:3.1mph Pressure: 993.0mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392248141213696, "text": "@GetYourHoeMane_ ������", "in_reply_to_status": 727392024530292736, "in_reply_to_user": 235414979, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 235414979 }}, "user": { "id": 2314905150, "name": "im not albino", "screen_name": "damneablasian", "lang": "en", "location": "ugly corner blvd", "create_at": date("2014-01-27"), "description": "44 ❤️ #LongLiveJwo RIPSDF #pvamu", "followers_count": 1956, "friends_count": 1266, "statues_count": 16840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-03T00:00:00.000Z"), "id": 727392248153923585, "text": "Y'all high bruh? https://t.co/l7zHMQfWto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2320175300, "name": "Meek.", "screen_name": "__KamikaS", "lang": "en", "location": ".tuscaloosa❤️", "create_at": date("2014-01-30"), "description": "pussy not war | | ✨blackMagic", "followers_count": 622, "friends_count": 493, "statues_count": 10694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392248841703424, "text": "Happy birthday famo have a good one G ✊���� @LongLive_7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1977318367 }}, "user": { "id": 1177260595, "name": "Jüvën®", "screen_name": "stobie28", "lang": "en", "location": "SnapChat:stobie28", "create_at": date("2013-02-13"), "description": "|18| C/O 16 Citrusvalley highschool #28 /Empire rugby/http://www.hudl.com/athlete/1473098/stephen-plaisted", "followers_count": 1175, "friends_count": 892, "statues_count": 12853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentone, CA", "id": "6a4d9ceebcc15b7d", "name": "Mentone", "place_type": "city", "bounding_box": rectangle("-117.139033,34.041098 -117.075255,34.082948") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 646884, "cityName": "Mentone" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392249038786560, "text": "A blindfolded sailor-dive into a wood chipper...\n#SaferThanATrumpRally", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SaferThanATrumpRally" }}, "user": { "id": 4722573422, "name": "Pre Tentious", "screen_name": "GoPretentious", "lang": "en", "location": "Next door...", "create_at": date("2016-01-07"), "description": "Wizard/Paper folder/Rim Job Inventor/MILF Commander/Pornography Historian/Moonshiner/Bride Kidnapping Expert//Freelance Sperm Donor/BOSS/Crack Head/Baby Sitter.", "followers_count": 120, "friends_count": 400, "statues_count": 1363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392249126907904, "text": "18!!!! ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 338592822, "name": "Maddie Kyler", "screen_name": "maddiekyler_", "lang": "en", "location": "Mesa, AZ ", "create_at": date("2011-07-19"), "description": "Senior • Varsity Cheerleader • Captain • Photographer • Skyline • Future Terp❤️ •", "followers_count": 472, "friends_count": 1211, "statues_count": 6218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392249210769408, "text": "\"Show how you've felt about your high school experience\" https://t.co/yJOw8C6IGr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1147343390, "name": "Leonardo DiRappio", "screen_name": "sky_n_above", "lang": "en", "location": "California, USA", "create_at": date("2013-02-03"), "description": "As depressed as Edgar Allen Poe himself. Im delirious.", "followers_count": 1093, "friends_count": 1556, "statues_count": 27289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, CA", "id": "4337f2014a1d936b", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-117.072347,33.903209 -116.94645,33.968725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 604758, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392249588371456, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Falling slowly. Temperature 66.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392250095792129, "text": "It's my mfing birthday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2783334677, "name": "Paige", "screen_name": "paigehomad", "lang": "en", "location": "Raving ", "create_at": date("2014-09-24"), "description": "feel the Bern \\\\ idgafos // PNW", "followers_count": 574, "friends_count": 552, "statues_count": 3053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Klahanie, WA", "id": "00c27fb84946cf7f", "name": "Klahanie", "place_type": "city", "bounding_box": rectangle("-122.028038,47.554711 -121.98809,47.580823") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5336101, "cityName": "Klahanie" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392250586619904, "text": "When Beyoncé arrived i thought someone was dying https://t.co/hlKCCz7KPF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.962163,40.779022"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 140713463, "name": "♡tc♡", "screen_name": "taylorclark_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-05"), "description": "i like hot dadz", "followers_count": 494, "friends_count": 145, "statues_count": 6361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metropolitan Museum Steps", "id": "07d9e47c26888002", "name": "Metropolitan Museum Steps", "place_type": "poi", "bounding_box": rectangle("-73.9621631,40.7790219 -73.962163,40.779022") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392251446452224, "text": "Wind 0.0 mph NW. Barometer 29.906 in, Falling slowly. Temperature 54.1 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392251681206272, "text": "@Kenia__Mendoza lol hahahah its good right lol", "in_reply_to_status": 727392062006423552, "in_reply_to_user": 451668477, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 451668477 }}, "user": { "id": 2413097604, "name": "Michelle Carbajal", "screen_name": "Michyloves1D", "lang": "en", "location": "null", "create_at": date("2014-03-26"), "description": "Harry Styles❤️", "followers_count": 458, "friends_count": 492, "statues_count": 6199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, CA", "id": "7df6f50f15138f28", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-118.282262,33.901902 -118.222378,33.929527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685614, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392251865767936, "text": "@ASM_276 درر من اين لك هذا ياصديقي", "in_reply_to_status": 727391051380445184, "in_reply_to_user": 272936519, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 272936519 }}, "user": { "id": 167177441, "name": "محمد جمال اليوسـف", "screen_name": "ALYOUSEF94", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-07-15"), "description": "ACA Alumni 12' | Int.Peace&Conflict Studies-Politics at @SDSU | مهتم بالسياسة #ستُشرق | أشمئز من العنصرية بكل انواعها! |", "followers_count": 2152, "friends_count": 999, "statues_count": 58941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392251945459712, "text": "https://t.co/blJyeQGM5H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 552420640, "name": "Sara", "screen_name": "sarananigans", "lang": "en", "location": "null", "create_at": date("2012-04-12"), "description": "⭐️ long term goal: 108 years old, isolated from society, referred to by the children of nearby village folk only as “the crone,” holding a sword at all times", "followers_count": 121, "friends_count": 48, "statues_count": 9704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392252092248064, "text": "And another one came into my life without me wanting it #DescendantOfTheSun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DescendantOfTheSun" }}, "user": { "id": 1544609634, "name": "✴Wendy✴", "screen_name": "Datkidwendy", "lang": "en", "location": "null", "create_at": date("2013-06-24"), "description": "Hi I'm Wendy and I like K-pop", "followers_count": 345, "friends_count": 668, "statues_count": 2459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392252230664192, "text": "she says oww at the end lmaooo", "in_reply_to_status": 727391906456440833, "in_reply_to_user": 2421408218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2421408218, "name": "smad", "screen_name": "sadalieee", "lang": "en", "location": "Lynwood, CA", "create_at": date("2014-03-31"), "description": "lol no", "followers_count": 1400, "friends_count": 764, "statues_count": 72295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392252260155392, "text": "Wind 0.0 mph ---. Barometer 30.016 in, Steady. Temperature 52.7 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392252352409600, "text": "05/03@03:00 - Temp 57.9F, WC 57.9F. Wind 1.0mph ENE, Gust 3.0mph. Bar 29.837in, Falling. Rain 0.01in. Hum 98%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48879 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-03T00:00:01.000Z"), "id": 727392252419407872, "text": "@G_DatAsshole ❣", "in_reply_to_status": -1, "in_reply_to_user": 192642033, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 192642033 }}, "user": { "id": 2266432008, "name": "TKWG", "screen_name": "Tenesia_", "lang": "en", "location": "ᗪETᖇOIT, ᗰI ", "create_at": date("2013-12-28"), "description": "deтerмιned. opтιмιѕтιc. dreaмer. ι'м a ѕтrong drιven woмan тo ѕay тнe very leaѕт! new yorĸ ιnѕтιтυтe oғ pнoтograpнy ѕтυdenт!", "followers_count": 630, "friends_count": 265, "statues_count": 51776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392252796899328, "text": "The US' 30 year experiment with corporate medicine has been the worst failure in the history of medicine. Time to declare dead,then restore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 254719698, "name": "Gavin Preston, M.D.", "screen_name": "GavinPrestonMD", "lang": "en", "location": "California", "create_at": date("2011-02-19"), "description": "Author of the upcoming book: The Managed Care Blues: My journey through a broken health care system and how we can fix it. (34 yr cancer survivor).", "followers_count": 40847, "friends_count": 44902, "statues_count": 28986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Niguel, CA", "id": "ef28da43cdf17b3f", "name": "Laguna Niguel", "place_type": "city", "bounding_box": rectangle("-117.740002,33.487223 -117.672617,33.571675") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639248, "cityName": "Laguna Niguel" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392252830437376, "text": "Lmao I'm hairy & my arm hairs got swirls .. My side burns thick too .. It's that Dominican bullshit ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 985827038, "name": ".", "screen_name": "DanyellMiles_", "lang": "en", "location": "null", "create_at": date("2012-12-02"), "description": "#LongLiveDaddy .", "followers_count": 1606, "friends_count": 1163, "statues_count": 28971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392253241479168, "text": "@kmthnle ������", "in_reply_to_status": 727374798792232960, "in_reply_to_user": 88108975, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 88108975 }}, "user": { "id": 1362985832, "name": "hangry", "screen_name": "_jdao", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "¯\\( ˘͡ ˘̯)/¯", "followers_count": 896, "friends_count": 450, "statues_count": 32196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392253342191617, "text": "Hahahahah OMFG STAHP!!! https://t.co/QBQtuzAujt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3236298097, "name": "Liz Lopez", "screen_name": "HardcoreLizzy", "lang": "en", "location": "Orange, CA", "create_at": date("2015-06-04"), "description": "Welcome To The Jungle That Is My Mind. Excuse My Random, Emotional, Dirty, Weird As Fuck, & Etc Tweets", "followers_count": 228, "friends_count": 299, "statues_count": 2149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392253828698113, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":56.3°F Wind:0.0mph Pressure: 29.94hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 316874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392254038413313, "text": "Perfect way to unwind after a non-stop Monday...with my #Sol-mate! https://t.co/sIfADwKqfu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Sol" }}, "user": { "id": 26324624, "name": "Ruthie Flores Retana", "screen_name": "ruthieretana", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-03-24"), "description": "Advocate for Higher Education for All. Runner. Hiker. Yogi. Traveler. Cafeista. Besotted with my GSD #Sol. Tweets are my own.", "followers_count": 390, "friends_count": 192, "statues_count": 2233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392254726279168, "text": "Happy birthday to me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340155224, "name": "Jaz ✨", "screen_name": "_loovely", "lang": "en", "location": "null", "create_at": date("2011-07-21"), "description": "null", "followers_count": 1093, "friends_count": 804, "statues_count": 40455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392254747246594, "text": "Wind 2.0 mph SE. Barometer 30.022 in, Falling slowly. Temperature 45.7 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392255120535552, "text": "@BrianSheehy7 exactly!", "in_reply_to_status": 727392192927408129, "in_reply_to_user": 1213703370, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1213703370 }}, "user": { "id": 963198392, "name": "Danny Maldonado", "screen_name": "tat_man0069", "lang": "en", "location": "Temecula, CA", "create_at": date("2012-11-21"), "description": "Tattoo artist. Insta: dannymaldonado17", "followers_count": 612, "friends_count": 398, "statues_count": 27827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Valley, CA", "id": "0196f2d33e5a1d73", "name": "French Valley", "place_type": "city", "bounding_box": rectangle("-117.137136,33.564564 -117.083609,33.627328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 626067, "cityName": "French Valley" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392255170899969, "text": "HBD Mom ❤️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 232297031, "name": "Jay", "screen_name": "juliesaewut", "lang": "en", "location": "Fairfield, CA", "create_at": date("2010-12-30"), "description": "Rest In Peace Mama ❤️ #FUCKCANCER 05/03/73 - 02/23/16", "followers_count": 1203, "friends_count": 570, "statues_count": 86380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392255313465345, "text": "Temp: 66.3°F Wind:0.0mph Pressure: 29.936hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392255644848129, "text": "Happy Birthday mstorres52 �� Fashion Monday's was Lit tonight!! Good vibes all night long!! New… https://t.co/mVvt5PEO5p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.1440887,39.9493599"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55025223, "name": "DJSpontane", "screen_name": "djspontane", "lang": "en", "location": "Philly & Worldwide!", "create_at": date("2009-07-08"), "description": "Artist/Producer/DJ/Host/Actor Extrodinaire! I Do It All!! Believe Dat!!!", "followers_count": 1455, "friends_count": 580, "statues_count": 17099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392255842078720, "text": "Ight chano", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2304032877, "name": "May 5th", "screen_name": "createdmoody", "lang": "en", "location": "TheBoro", "create_at": date("2014-01-25"), "description": "#WSSU19", "followers_count": 2166, "friends_count": 1428, "statues_count": 44072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256144084993, "text": "#staywoke", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "staywoke" }}, "user": { "id": 4104168553, "name": "YUNG PABLO♨️", "screen_name": "nothingfugazyy", "lang": "en", "location": "Newark, NJ", "create_at": date("2015-11-02"), "description": "#StayBlessed", "followers_count": 116, "friends_count": 109, "statues_count": 3250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256341200897, "text": "@mcloven_youu so funny. ������", "in_reply_to_status": 727391836789178368, "in_reply_to_user": 236059931, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236059931 }}, "user": { "id": 93086632, "name": "menace2society", "screen_name": "__sweetnessss", "lang": "en", "location": "null", "create_at": date("2009-11-27"), "description": ":)", "followers_count": 990, "friends_count": 869, "statues_count": 38119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256362057729, "text": "66.6F (Feels: 66.6F) - Humidity: 95% - Wind: 2.2mph NE - Gust: 6.9mph - Pressure: 1012.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 237867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256458530816, "text": "First day at LA Fitness tomorrow��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3119060137, "name": "Arms", "screen_name": "armeniaquintero", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 280, "friends_count": 189, "statues_count": 8579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256727126016, "text": "Bye bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120531964, "name": "christian ovando", "screen_name": "christianovando", "lang": "en", "location": "null", "create_at": date("2010-03-06"), "description": "null", "followers_count": 69, "friends_count": 263, "statues_count": 871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newnan, GA", "id": "00a0d2e62d54ed89", "name": "Newnan", "place_type": "city", "bounding_box": rectangle("-84.832651,33.343601 -84.693369,33.416232") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13077, "countyName": "Coweta", "cityID": 1355020, "cityName": "Newnan" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256802492416, "text": "@a_asiaaaa I miss you too ����", "in_reply_to_status": 727392119812440064, "in_reply_to_user": 238366378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 238366378 }}, "user": { "id": 1210786171, "name": "la'hova", "screen_name": "_TheGreatPlay_", "lang": "en", "location": "New Orleans // Westbank", "create_at": date("2013-02-22"), "description": "#whatisyoudoing?", "followers_count": 322, "friends_count": 288, "statues_count": 4406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-03T00:00:02.000Z"), "id": 727392256840359937, "text": "| M o t i v a t i o n |\nRemembering that #rajnbow I saw last week on my #birthday it's… https://t.co/T3latuvNDl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3975983,33.9442368"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rajnbow", "birthday" }}, "user": { "id": 1129822118, "name": "Cory Green", "screen_name": "1stClassCory", "lang": "en", "location": "Los Angeles, California", "create_at": date("2013-01-28"), "description": "I write ✏ to free my mind, play tennis to fuel my body, and travel ✈️ to strengthen my soul", "followers_count": 336, "friends_count": 721, "statues_count": 1441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392257637146624, "text": "@TrillaDom why dis look so lit", "in_reply_to_status": 727391518122610688, "in_reply_to_user": 713841624183410689, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 713841624183410689 }}, "user": { "id": 607395926, "name": "John Proctor", "screen_name": "Devontooshort", "lang": "en", "location": "ATL hoe", "create_at": date("2012-06-13"), "description": "surviving ft. Trina", "followers_count": 2329, "friends_count": 898, "statues_count": 70805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucker, GA", "id": "a4e8b6fd61c8026b", "name": "Tucker", "place_type": "city", "bounding_box": rectangle("-84.252764,33.815828 -84.173182,33.884965") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1377652, "cityName": "Tucker" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392257758765059, "text": "�� https://t.co/KwCqsJp4Qv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1360598958, "name": "Beyoncé✨", "screen_name": "Miss_Ethiopiaa", "lang": "en", "location": "Dallas ♐️ College Station", "create_at": date("2013-04-17"), "description": "Motivate yo ass, call me Malcolm X | Remember when I coined #EastOnFleek bc same", "followers_count": 2752, "friends_count": 1413, "statues_count": 67664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392257846841345, "text": "Wind 0.0 mph ENE. Barometer 29.936 in, Falling. Temperature 63.1 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392258035699712, "text": "Grow a pair & own up to your shit ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165272020, "name": "Nina J.", "screen_name": "Nina_Jel", "lang": "en", "location": "Chicago/Konjic ", "create_at": date("2010-07-10"), "description": "IG: nina_jel", "followers_count": 249, "friends_count": 371, "statues_count": 6542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392258253684736, "text": "@cole_duprie spit / cough on her", "in_reply_to_status": 727392165148528640, "in_reply_to_user": 3318175271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3318175271 }}, "user": { "id": 3330519431, "name": "Marshall Ling", "screen_name": "marshallling", "lang": "en", "location": "Austin, TX", "create_at": date("2015-06-16"), "description": "Tubbs did nothing wrong #UT19", "followers_count": 215, "friends_count": 195, "statues_count": 2741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392258580848640, "text": "Has anyone else ever noticed random super weird things on the trending list with like 20 tweets or am I crazy bc wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 707079445346004992, "name": "Shay", "screen_name": "the100receipts", "lang": "en", "location": "and", "create_at": date("2016-03-07"), "description": "Constant updates and other things! If you have a ?/info/etc. message me! http://wedeservedbetter.com https://www.classy.org/fundraise?fcid=625415", "followers_count": 1604, "friends_count": 432, "statues_count": 1338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineyard, CA", "id": "01f9f8b01415323b", "name": "Vineyard", "place_type": "city", "bounding_box": rectangle("-121.371979,38.452572 -121.292468,38.485308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 682852, "cityName": "Vineyard" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392258790543360, "text": "I just woke up from my nap �� wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2941247616, "name": "JayPee", "screen_name": "jose_perez5743", "lang": "en", "location": "null", "create_at": date("2014-12-23"), "description": "null", "followers_count": 328, "friends_count": 209, "statues_count": 5180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392259000422404, "text": "Wind 0.7 mph N. Barometer 29.96 in, Steady. Temperature 46.8 °F. Rain today 0.02 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 119, "statues_count": 159508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392259612794880, "text": "@OG_Davae ����", "in_reply_to_status": 727391625698271232, "in_reply_to_user": 326227965, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 326227965 }}, "user": { "id": 269437158, "name": "I am.", "screen_name": "_kxll", "lang": "en", "location": "null", "create_at": date("2011-03-20"), "description": "well and wellness • relaxation and peace •", "followers_count": 1437, "friends_count": 1224, "statues_count": 61935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392259755241472, "text": "@dandean just finished this: https://t.co/vo7WO8SaRW", "in_reply_to_status": 727376274738401280, "in_reply_to_user": 9525212, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9525212 }}, "user": { "id": 14063139, "name": "Daniel and 10 others", "screen_name": "TechWraith", "lang": "en", "location": "San Francisco, CA", "create_at": date("2008-02-29"), "description": "Code Janitor. Creator of Atomify. CTO at @Getable. Previously at @Yammer and @Storify.", "followers_count": 2271, "friends_count": 385, "statues_count": 36794 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Emeryville, CA", "id": "99e789320196ef6a", "name": "Emeryville", "place_type": "city", "bounding_box": rectangle("-122.315509,37.827015 -122.27579,37.850085") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 622594, "cityName": "Emeryville" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392259793125377, "text": "@SWJae_CHAsh y'all tweeting me!! �� y'all niggas tripping", "in_reply_to_status": 727391891164155904, "in_reply_to_user": 284385172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 284385172 }}, "user": { "id": 102730180, "name": "OG", "screen_name": "_WillieBeamen_", "lang": "en", "location": "#WeWorkin #Eastside", "create_at": date("2010-01-07"), "description": "Detroit Made, Eastside Raised Just a cool wholesome dude trying to make it Go follow my AP @Just__Brit ❤ best sister I could ask for", "followers_count": 1895, "friends_count": 1472, "statues_count": 192260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392260145344512, "text": "I need to get out of Utah for a little while.. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 546436386, "name": "Alyssa Coxey", "screen_name": "LyssaSistah", "lang": "en", "location": "null", "create_at": date("2012-04-05"), "description": "good vibes is what I live by", "followers_count": 111, "friends_count": 178, "statues_count": 418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willard, UT", "id": "7b5bd31136f36c88", "name": "Willard", "place_type": "city", "bounding_box": rectangle("-112.051557,41.387164 -112.026914,41.444236") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49003, "countyName": "Box Elder", "cityID": 4984710, "cityName": "Willard" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392260355026944, "text": "Under My Umbrella by Incubus is #nowplaying in Thumb Coast Brewing Co., Port Huron.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.422367,42.975489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox", "screen_name": "VirtualJukebox", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1516, "friends_count": 0, "statues_count": 1632125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Huron, MI", "id": "010750c237347ebb", "name": "Port Huron", "place_type": "city", "bounding_box": rectangle("-82.514554,42.935388 -82.418396,43.040084") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2665820, "cityName": "Port Huron" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392260849987585, "text": "@Najmah_Omar yea wlh at rawdah������thought I was a convert", "in_reply_to_status": 727389068468822016, "in_reply_to_user": 363365447, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363365447 }}, "user": { "id": 2588842314, "name": "Mustafa", "screen_name": "Mustafa_Wideout", "lang": "en", "location": "Minneapolis, MN", "create_at": date("2014-06-25"), "description": "NDSCS '18 #JucoProduct", "followers_count": 1199, "friends_count": 440, "statues_count": 7944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392261026119681, "text": "my girl @dezthoo is officially a fellow Slytherin, I am proud. #sneksisters ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sneksisters" }}, "user_mentions": {{ 772568384 }}, "user": { "id": 1348443127, "name": "Liz", "screen_name": "lizalcarez", "lang": "en", "location": "null", "create_at": date("2013-04-12"), "description": "null", "followers_count": 335, "friends_count": 212, "statues_count": 13356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus, CA", "id": "04088141121c7398", "name": "Citrus", "place_type": "city", "bounding_box": rectangle("-117.899428,34.105384 -117.876959,34.127081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 613560, "cityName": "Citrus" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392261034508288, "text": "@olivverthekid @FeatSocks @fmlybnd SALT LAKE CITY BILLBOARD LIVE", "in_reply_to_status": 726832534659559424, "in_reply_to_user": 757801177, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 757801177, 2749990746, 1380349250 }}, "user": { "id": 563314854, "name": "jeff", "screen_name": "JeffreyRakes", "lang": "en", "location": "null", "create_at": date("2012-04-25"), "description": "null", "followers_count": 782, "friends_count": 394, "statues_count": 30785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-05-03T00:00:03.000Z"), "id": 727392261105811456, "text": "a MacBook is very distracting!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 147771077, "name": "Marco", "screen_name": "MarcoLoya37", "lang": "en", "location": "San Diego, CA", "create_at": date("2010-05-24"), "description": "any other social media @MarcoLoya37", "followers_count": 2858, "friends_count": 672, "statues_count": 32729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392261223260160, "text": "it's about damn time bruh lol enjoy ya time there https://t.co/36etmHnxwo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4865568114, "name": "justin.", "screen_name": "TakeYourLs", "lang": "en", "location": "Murrieta, CA", "create_at": date("2016-02-05"), "description": "I don't like people.", "followers_count": 228, "friends_count": 179, "statues_count": 8347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392261307142146, "text": "Little Lacey, aka LouLou is 1 already!❤️ https://t.co/S8a3gWzcia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2391320761, "name": "DreamChaser✝", "screen_name": "raeleigh2014", "lang": "en", "location": "Blanchard, OK", "create_at": date("2014-03-15"), "description": "•twenty//CRNA chasing// Integris SW// RIP Heather• {Seth has my heart}", "followers_count": 297, "friends_count": 461, "statues_count": 3403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arvada, CO", "id": "c02e66a5016d732f", "name": "Arvada", "place_type": "city", "bounding_box": rectangle("-105.214417,39.783802 -105.04196,39.856611") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 803455, "cityName": "Arvada" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392261546344449, "text": "I don't regret it, but I'm glad we're through", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 720781774, "name": "zoë101", "screen_name": "ZoeMarie_625", "lang": "en", "location": "null", "create_at": date("2013-10-18"), "description": "null", "followers_count": 252, "friends_count": 225, "statues_count": 7146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392261751754756, "text": "Wind 0.0 mph ---. Barometer 30.085 in, Falling slowly. Temperature 48.0 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392262116646913, "text": "@aleenanoelani that's not nice , we're friends", "in_reply_to_status": 727392186589802496, "in_reply_to_user": 151285640, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 151285640 }}, "user": { "id": 133614421, "name": "crishell.", "screen_name": "AlohaCrishell", "lang": "en", "location": "Kaneohe, HI", "create_at": date("2010-04-15"), "description": "los angeles || hawaii", "followers_count": 15679, "friends_count": 15026, "statues_count": 20337 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaneohe, HI", "id": "afdda35f375f55ae", "name": "Kaneohe", "place_type": "city", "bounding_box": rectangle("-157.830997,21.385091 -157.760186,21.428755") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1528250, "cityName": "Kaneohe" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392262368419841, "text": "Temp 46.5° Hi/Lo 47.3/46.5 Rng 0.8° WC 46.5° Hmd 97% Rain 0.00\" Storm 0.38\" BAR 29.944 Falling DP 45.7° Wnd 1mph Dir ESE Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 67, "friends_count": 120, "statues_count": 18387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392262859038720, "text": "@vaintfb it's a terrible feeling", "in_reply_to_status": 727392179157524480, "in_reply_to_user": 2937010909, "favorite_count": 0, "coordinate": point("-90.38440443,40.98463957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2937010909 }}, "user": { "id": 2402563498, "name": "Jvshy", "screen_name": "Obey_Jvshy", "lang": "en", "location": "null", "create_at": date("2014-03-11"), "description": "College baseball | Player for @TheObeyAlliance | GT: Obey Jshy", "followers_count": 1214, "friends_count": 387, "statues_count": 6305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Illinois, USA", "id": "f54a2170ff4b15f7", "name": "Illinois", "place_type": "admin", "bounding_box": rectangle("-91.51308,36.970298 -87.019935,42.508303") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17095, "countyName": "Knox", "cityID": 1728326, "cityName": "Galesburg" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392262963908608, "text": "Up listening to music and plotting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227291814, "name": "DChemist25", "screen_name": "DChemist25", "lang": "en", "location": "Top floor and beyond ", "create_at": date("2010-12-16"), "description": "Universal Family Connections.", "followers_count": 674, "friends_count": 1004, "statues_count": 31269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392263035297792, "text": "Soon as u see the text reply me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 227724619, "name": "Rave", "screen_name": "_prettygirlrave", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2010-12-17"), "description": "19yrsold. snapchat&IG: prettygirlraven", "followers_count": 2969, "friends_count": 1817, "statues_count": 77719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392263039541248, "text": "Tomorrows about to be so long ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 349377629, "name": "Stephanie", "screen_name": "stephm____", "lang": "en", "location": "Somewhere ", "create_at": date("2011-08-05"), "description": "Louisville✈️RichCity IG: stephm____ #MBNF", "followers_count": 1672, "friends_count": 1406, "statues_count": 61659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, KY", "id": "4c56854a831bdd43", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.37566,37.688339 -84.249178,37.790079") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21151, "countyName": "Madison", "cityID": 2165226, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392263114887168, "text": "Donna has a whole bottom tooth missing ������ #BlackInkCrew", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BlackInkCrew" }}, "user": { "id": 297634740, "name": "Diamond Wright ✌", "screen_name": "am_diamond", "lang": "en", "location": "Biloxi, MS", "create_at": date("2011-05-12"), "description": "Dallas #TeamBPace", "followers_count": 781, "friends_count": 438, "statues_count": 14011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Biloxi, MS", "id": "b92f67506b6a7eb8", "name": "Biloxi", "place_type": "city", "bounding_box": rectangle("-89.001504,30.386899 -88.856731,30.468736") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2806220, "cityName": "Biloxi" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392264406835200, "text": "Wouldn't last very long....Everybody gone have a story to tell �� https://t.co/PKNIHpFJSy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2889408358, "name": "B.A.M.M", "screen_name": "SauxeFaxtor__", "lang": "en", "location": "Splashville", "create_at": date("2014-11-23"), "description": "19| Now that's some heat™.......", "followers_count": 471, "friends_count": 367, "statues_count": 3848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeshore, LA", "id": "016b521306ea18dd", "name": "Lakeshore", "place_type": "city", "bounding_box": rectangle("-92.046263,32.524735 -92.01211,32.546714") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2241470, "cityName": "Lakeshore" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392264524193792, "text": "Wind 1 mph S. Barometer 1014.5 hPa, Falling. Temperature 77.2 °F. Rain today 0.00 in. Humidity 13%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 89, "friends_count": 264, "statues_count": 155068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392264754864129, "text": "A group of ugly bitches that sit around and hype each other up <<", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328802740, "name": "K I N G ☔️.〽️.", "screen_name": "MillerMade", "lang": "en", "location": "Htx✈️Sa", "create_at": date("2011-07-03"), "description": "♊️ / Mechanical Engineer .... RIP Coach Green", "followers_count": 1433, "friends_count": 1195, "statues_count": 39382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392264792641536, "text": "@NDNfrankenstein it's basic science, man", "in_reply_to_status": 727389068934287360, "in_reply_to_user": 1180704878, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1180704878 }}, "user": { "id": 6381322, "name": "Melissa Drew", "screen_name": "supjane", "lang": "en", "location": "NorCal // Sioux Falls", "create_at": date("2007-05-28"), "description": "Naked weirdo on the internet. Cat mom. Professional procrastinator.", "followers_count": 1220, "friends_count": 89, "statues_count": 3557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sioux Falls, SD", "id": "3605db70c687a01d", "name": "Sioux Falls", "place_type": "city", "bounding_box": rectangle("-96.839502,43.465641 -96.622783,43.616388") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46099, "countyName": "Minnehaha", "cityID": 4659020, "cityName": "Sioux Falls" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392264805183488, "text": "I hate questions . Leave me alone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163359139, "name": "sas queen", "screen_name": "YoungJacla", "lang": "en", "location": "null", "create_at": date("2010-07-05"), "description": "TRVPG0D", "followers_count": 586, "friends_count": 405, "statues_count": 121699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-03T00:00:04.000Z"), "id": 727392265287688192, "text": "I am saying everyone I give all my pay checks to is a strangers, you know everyone at the bank? https://t.co/XdSMAnSvnN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413470255, "name": "Timothy O'Donnell", "screen_name": "TimothyODonnel2", "lang": "en", "location": "#catholic Commonwealth of VA", "create_at": date("2011-11-15"), "description": "married, Army Veteran, Blocked by @KeithOlbermann, @monaeltahawy, @therealroseanne, @ijasonalexander, & @toure #TrumpLovesPecker #NeverTrump #TGDN", "followers_count": 5693, "friends_count": 6097, "statues_count": 66871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harrisonburg, VA", "id": "fb7c1db180c09183", "name": "Harrisonburg", "place_type": "city", "bounding_box": rectangle("-78.918345,38.387443 -78.824223,38.489169") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51660, "countyName": "Harrisonburg", "cityID": 5135624, "cityName": "Harrisonburg" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392265652559872, "text": "When you been working out with your friends, then you notice that one of your friends is getting more attractive. But y'all are FRIENDS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 539164752, "name": "King Jodye", "screen_name": "VoteforOshae", "lang": "en", "location": "Only God Knows", "create_at": date("2012-03-28"), "description": "People say you are what you eat, but that's weird because I don't remember eating a legend. Freelance photographer #TXST ✉:lifeless.aesthetics@gmail.com", "followers_count": 11508, "friends_count": 9738, "statues_count": 26835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392265698680832, "text": "❄️", "in_reply_to_status": 723216225912905732, "in_reply_to_user": 308390635, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 308390635, "name": "Eastside X", "screen_name": "ArkkAngelX", "lang": "en", "location": "Indianapolis, IN", "create_at": date("2011-05-31"), "description": "label // @arkkangelz™ | booking-features // arkkangelx@gmail.com | Brendon Harris // July 6th", "followers_count": 1565, "friends_count": 439, "statues_count": 81509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vincennes, IN", "id": "bfadcb4bfcf83cef", "name": "Vincennes", "place_type": "city", "bounding_box": rectangle("-87.547173,38.625181 -87.433471,38.736046") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18083, "countyName": "Knox", "cityID": 1879208, "cityName": "Vincennes" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392266222891008, "text": "ONE AM.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995796931, "name": "julie", "screen_name": "idioxsyncratic", "lang": "en", "location": "SLC", "create_at": date("2012-12-07"), "description": "18. out of mind, out of sight.", "followers_count": 1083, "friends_count": 1393, "statues_count": 39811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, UT", "id": "d254694c82a4473e", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-112.05574,41.074662 -111.979637,41.126319") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4913850, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392266436775942, "text": "@damannx3 hell yea those dragons bouta dominate", "in_reply_to_status": 727392009296584704, "in_reply_to_user": 850637126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 850637126 }}, "user": { "id": 1499570108, "name": "Stale", "screen_name": "Justinstal", "lang": "en", "location": "Tucson, AZ", "create_at": date("2013-06-10"), "description": "Arizona Wildcats c/o '19 #BLM LA Confidential", "followers_count": 594, "friends_count": 389, "statues_count": 27677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392266860503040, "text": "\"i love u to the whole world and never ever back\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2875710582, "name": "boy ryen", "screen_name": "yungdirtbagryen", "lang": "en", "location": "Arkansas", "create_at": date("2014-10-24"), "description": "writer/creator/black/creative curator/educator/professional bbysitta/always roast worthy/basically everything", "followers_count": 545, "friends_count": 497, "statues_count": 7620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riviera Beach, FL", "id": "ee2930da9f67751a", "name": "Riviera Beach", "place_type": "city", "bounding_box": rectangle("-80.119474,26.762211 -80.031242,26.815144") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1260975, "cityName": "Riviera Beach" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392266877210624, "text": "https://t.co/qSMqToDSM7 #Goodnightworld ✌����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Goodnightworld" }}, "user": { "id": 35005290, "name": "Moe Johnson", "screen_name": "Moe_the_Monster", "lang": "en", "location": "Where Fate & Opportunity Meet", "create_at": date("2009-04-24"), "description": "Goalgetter. RunnerUp in @HowardStern World's Biggest Most Beautiful Black Penis Contest ✊\nBookings: MoetheMonster13@Gmail.com Publicist: Misslainie2@gmail.com", "followers_count": 32023, "friends_count": 3019, "statues_count": 60922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392267279835138, "text": "HAPPY BIRTHDAY NATHALY ALVAREZ ILYSM ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195002737, "name": "sleeping beauty", "screen_name": "alondraxtamayo", "lang": "en", "location": "null", "create_at": date("2015-05-13"), "description": "Disney enthusiast", "followers_count": 179, "friends_count": 221, "statues_count": 6516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392267619721216, "text": "Pull up @TeamSCityy_Live katch the loop @illiotti Mixtapes, Dj's, Events, Artist..Link up & DM 4 info", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 758256768, 160783887 }}, "user": { "id": 160783887, "name": "illiotti inc.", "screen_name": "illiotti", "lang": "en", "location": "278 B.Q.", "create_at": date("2010-06-28"), "description": "#Qs5 #BBSs #QueensNation #DopeBitches #DopeModelsNYC #LKP #DopeMix Mg YFGang illiotti45@Gmail.com", "followers_count": 2794, "friends_count": 2895, "statues_count": 18663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268114501632, "text": "���������� https://t.co/mU9XwLPqug", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3241569016, "name": "sinii", "screen_name": "MMelesini", "lang": "en", "location": "null", "create_at": date("2015-05-07"), "description": "i like food more than i like people", "followers_count": 452, "friends_count": 333, "statues_count": 9419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268219355136, "text": "Future said you can do what you want when you poppin right?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 624424534, "name": "Samantha Charlene", "screen_name": "simply_samii95", "lang": "en", "location": "leesville louisiana", "create_at": date("2012-07-01"), "description": "LHS Alumni|\nNSU19\n|God First|\n|Loving Life|\n|Future social worker|\nsc:Sam_loves1995", "followers_count": 453, "friends_count": 466, "statues_count": 4568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Natchitoches, LA", "id": "28ee5ea3adb6eddd", "name": "Natchitoches", "place_type": "city", "bounding_box": rectangle("-93.128629,31.718432 -93.043947,31.806051") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22069, "countyName": "Natchitoches", "cityID": 2253545, "cityName": "Natchitoches" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268219387904, "text": "���� https://t.co/KpvG32BrCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1941910470, "name": "Kylli Lakatos", "screen_name": "KylliLakatos", "lang": "en", "location": "null", "create_at": date("2013-10-06"), "description": "null", "followers_count": 130, "friends_count": 151, "statues_count": 1647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Citrus Heights, CA", "id": "a89175c4c91f45a3", "name": "Citrus Heights", "place_type": "city", "bounding_box": rectangle("-121.331355,38.663673 -121.242804,38.722779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 613588, "cityName": "Citrus Heights" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268278071296, "text": "I swear I was crying and laughing at the same time all of Sunday lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251845438, "name": "jen coco", "screen_name": "Jen_Cocobaaa", "lang": "en", "location": "Arizona, Tucson", "create_at": date("2011-02-13"), "description": "power of makeup | be your own artist// snapc. jencocoba / dm me for makeup bookings", "followers_count": 1377, "friends_count": 1179, "statues_count": 20436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268320010240, "text": "I want a girl so bad I'm not even gonna front", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176561260, "name": "Patty Mills", "screen_name": "mrplanman_", "lang": "en", "location": "Dreamville, TX. ", "create_at": date("2010-08-09"), "description": "We gon' be alright", "followers_count": 1070, "friends_count": 1667, "statues_count": 29928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-03T00:00:05.000Z"), "id": 727392268793999360, "text": "Wind 0.3 mph NW. Barometer 29.938 in, Falling. Temperature 42.0 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754632425054209, "text": "Wind 1.0 mph SW. Barometer 1018.76 mb, Falling. Temperature 50.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754632458539009, "text": "@NevilleLouann https://t.co/0Hdr9hkKZE", "in_reply_to_status": 727634654828843008, "in_reply_to_user": 3091521770, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3091521770 }}, "user": { "id": 16683842, "name": "Jay Oliver Sax", "screen_name": "4POUNDTONGUE", "lang": "en", "location": "New York, NY", "create_at": date("2008-10-10"), "description": "WARNING: Harmful if swallowed, followed by myocardial infarction, or sexual intercourse... whichever comes first.", "followers_count": 757, "friends_count": 3015, "statues_count": 19367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754632596967425, "text": "Abúl q Kei me dejó sola!��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 289636391, "name": "*.* Jess P.❤️", "screen_name": "Jesscpt", "lang": "es", "location": "Lalalandia! :-}", "create_at": date("2011-04-28"), "description": "Persona SIMPLE con mente sumamente COMPLICADA! ;-) *Dios ❤️ *Familia ❤ #TeamPisces♓", "followers_count": 656, "friends_count": 319, "statues_count": 59508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754633091887104, "text": "@ReidMontano_ jk jk for reals tho let's be in movies tho", "in_reply_to_status": 727754119520358400, "in_reply_to_user": 1273970712, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1273970712 }}, "user": { "id": 1639083878, "name": "KK", "screen_name": "kelvin_kine", "lang": "en", "location": "Your moms house", "create_at": date("2013-08-01"), "description": "y'all really can't smoke like me", "followers_count": 593, "friends_count": 532, "statues_count": 8536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queen Creek, AZ", "id": "01cb573821d94344", "name": "Queen Creek", "place_type": "city", "bounding_box": rectangle("-111.686314,33.196614 -111.582748,33.288127") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 458150, "cityName": "Queen Creek" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754633398124544, "text": "@rc1963 @PeachyFiend \"it lacks the virtues of validity, believability, and beracity\" laugh all you want; i imaging an uncomfortable,", "in_reply_to_status": 727753473786314752, "in_reply_to_user": 205177161, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 205177161, 994637239 }}, "user": { "id": 956225894, "name": "Woody", "screen_name": "elbandito2016", "lang": "en", "location": "Nevada, USA", "create_at": date("2012-11-18"), "description": "Law Enforcement professional, veteran Marine, father and husband.", "followers_count": 125, "friends_count": 214, "statues_count": 2968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754633695920128, "text": "Hey y'all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298826309, "name": "Gene ♠️⛽️", "screen_name": "Foolio_Joshh", "lang": "en", "location": "Bowling Green, OH", "create_at": date("2011-05-14"), "description": "Fuck' with a real ass nigga. I'm a fuck how you feel ass nigga. ‼️#BGSU17 SC: DatNigga_Josh", "followers_count": 816, "friends_count": 384, "statues_count": 48085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754634564112384, "text": "I don't understand you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 729251227, "name": "prescilla", "screen_name": "_pdelgado97", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "Jbiebs is my boo ❤️", "followers_count": 567, "friends_count": 472, "statues_count": 11561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635323265025, "text": "I want innout", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 72693977, "name": "Christopher", "screen_name": "Chris_Ruiz_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-09-08"), "description": "null", "followers_count": 707, "friends_count": 387, "statues_count": 8875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635449139200, "text": "Those who are kind benefit themselves, but the cruel bring ruin on themselves. Proverbs 11:17", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574892254, "name": "lil' kinz", "screen_name": "idahowk", "lang": "en", "location": "null", "create_at": date("2012-05-08"), "description": "I love Jesus & sports ツ", "followers_count": 460, "friends_count": 380, "statues_count": 5547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rathdrum, ID", "id": "2a8e415ef970f42c", "name": "Rathdrum", "place_type": "city", "bounding_box": rectangle("-116.915792,47.788085 -116.869365,47.826625") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16055, "countyName": "Kootenai", "cityID": 1666340, "cityName": "Rathdrum" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635579170820, "text": "MAY THE 4TH BE WITH YOU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732125804, "name": "reena :)", "screen_name": "tapthatassash", "lang": "en", "location": "los angeles | m5™", "create_at": date("2012-08-01"), "description": "post break-up sex", "followers_count": 7761, "friends_count": 113, "statues_count": 80287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635612688386, "text": "STAP https://t.co/ErB9mtJ6hV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374278358, "name": "Tiffany", "screen_name": "disney_18", "lang": "en", "location": "Bronx, NY", "create_at": date("2011-09-15"), "description": "null", "followers_count": 1530, "friends_count": 931, "statues_count": 30345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635629449217, "text": "@imSeeRaw ������", "in_reply_to_status": 727752998340993025, "in_reply_to_user": 34482461, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 34482461 }}, "user": { "id": 40809894, "name": "ThatGoonSits", "screen_name": "ThatGoonsits", "lang": "en", "location": "Venice, Los Angeles", "create_at": date("2009-05-17"), "description": "IG: @Thatgoonsits @detoxsquad", "followers_count": 11571, "friends_count": 9643, "statues_count": 26501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635734290432, "text": "tempted to get some taco bell ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 439705443, "name": "león", "screen_name": "leonasaurus", "lang": "en", "location": "Whittier ", "create_at": date("2011-12-17"), "description": "I do what I want. Don't tell my mom.", "followers_count": 187, "friends_count": 153, "statues_count": 4343 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635746938880, "text": "2476aa115775b77e356d4a981b7318716f36cb682b2fbe1ad672c6f2d1ef266869bf14c3a527fe311ff04595463d5f78329fd995f82fbc910ec3efb615730116ba4f8f000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-109.68339527,56.26884294"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 156, "friends_count": 1, "statues_count": 943189 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Division No. 18, Unorganized, Saskatchewan", "id": "2cc34b9376b2a9a8", "name": "Division No. 18, Unorganized", "place_type": "city", "bounding_box": rectangle("-110.006084,53.625536 -101.768294,60.012537") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754635960791040, "text": "Ripley SW Limestone Co. Temp: 52.0°F Wind:0.0mph Pressure: 992.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 52855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754636069879808, "text": "@ebbtideapp Tide in Seavey Island, New Hampshire 05/04/2016\n Low 3:38am -0.2\nHigh 9:53am 9.1\n Low 4:01pm -0.3\nHigh 10:17pm 9.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.7417,43.08"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 29848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kittery, ME", "id": "56f17de01cf8fcc6", "name": "Kittery", "place_type": "city", "bounding_box": rectangle("-70.775038,43.074445 -70.702518,43.135878") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23031, "countyName": "York", "cityID": 2337235, "cityName": "Kittery" } }
+{ "create_at": datetime("2016-05-04T00:00:00.000Z"), "id": 727754636191510528, "text": "I'm so sore ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224120796, "name": "✨", "screen_name": "jennxo96", "lang": "en", "location": "null", "create_at": date("2010-12-07"), "description": "19.", "followers_count": 1291, "friends_count": 526, "statues_count": 44708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754636581580800, "text": "@cantstop_colvin best way to sleep ����", "in_reply_to_status": 727729522251079680, "in_reply_to_user": 517153189, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 517153189 }}, "user": { "id": 2421311130, "name": "Jordan Kearby", "screen_name": "jotaykaay", "lang": "en", "location": "219", "create_at": date("2014-03-31"), "description": "typical white girl, beach bum", "followers_count": 881, "friends_count": 601, "statues_count": 6457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberty, IN", "id": "00305c522a8b7101", "name": "Liberty", "place_type": "city", "bounding_box": rectangle("-87.124267,41.520668 -87.028491,41.57325") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18161, "countyName": "Union", "cityID": 1843434, "cityName": "Liberty" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754637059723264, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Steady. Temperature 61.0 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754637433016320, "text": "05/04@03:00 - Temp 51.6F, WC 51.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.807in, Falling slowly. Rain 0.00in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754637709840385, "text": "@MirLagerfield �� I'm right huh?", "in_reply_to_status": 727754504305836032, "in_reply_to_user": 99475247, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99475247 }}, "user": { "id": 231997207, "name": "cincodemayo ♉️", "screen_name": "cravebuttvh", "lang": "en", "location": "balifornia", "create_at": date("2010-12-29"), "description": "save yourself from me", "followers_count": 2838, "friends_count": 1732, "statues_count": 132674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754637848236032, "text": "in a couple of secs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 81683542, "name": "Kt the Qt.", "screen_name": "OhHaiLingLing", "lang": "en", "location": "Riverside, CA", "create_at": date("2009-10-11"), "description": "null", "followers_count": 1091, "friends_count": 513, "statues_count": 72959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754638137667584, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 113755064, "name": "natas", "screen_name": "nataaach", "lang": "en", "location": "Hometown USA", "create_at": date("2010-02-12"), "description": "SINK OR SWIM", "followers_count": 289, "friends_count": 170, "statues_count": 9197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Clemente, CA", "id": "97467526c304c5db", "name": "San Clemente", "place_type": "city", "bounding_box": rectangle("-117.666227,33.386645 -117.571015,33.490825") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 665084, "cityName": "San Clemente" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754638448074757, "text": "HAPPY 18TH BDAY TO NASIYA DELIANNA DIAZ CAUSE SHE MY BIH & LUH HER ❤️ SEE YOU IN TWO DAYS BBY & HAVE A GOOD DAY!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1294144729, "name": "keeya", "screen_name": "VXVXCIX", "lang": "en", "location": "沖縄 - PHX", "create_at": date("2013-03-24"), "description": "my bf @nsydz has the best wing tips | '16", "followers_count": 485, "friends_count": 294, "statues_count": 9258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754638531940352, "text": "@cortingthecourt @No_Era_Penal69 have y'all seen Khan do the thing with the milkshake bottle ?", "in_reply_to_status": 727754396533178368, "in_reply_to_user": 637091246, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 637091246, 1081517820 }}, "user": { "id": 1320984762, "name": "Diego Moreno", "screen_name": "eatpresh", "lang": "en", "location": "Kamino ", "create_at": date("2013-04-01"), "description": "you won't get my tweets | follow @QAJenkins !", "followers_count": 457, "friends_count": 477, "statues_count": 13406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754638783565824, "text": "you don't see this enough.. https://t.co/kXU1sOy79T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4841006609, "name": "Kortney Kling", "screen_name": "kortneykling", "lang": "en", "location": "null", "create_at": date("2016-01-23"), "description": "@LA_College /// @KALBSports", "followers_count": 137, "friends_count": 50, "statues_count": 1512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pineville, LA", "id": "a22746c88990f7a6", "name": "Pineville", "place_type": "city", "bounding_box": rectangle("-92.452373,31.303096 -92.34801,31.384652") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2260530, "cityName": "Pineville" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754639123333121, "text": "Night guys I'll be back Stay Real and Keep on Stroking ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4421, "friends_count": 3125, "statues_count": 47318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754639215562754, "text": "\"i just took the longest nap\"\n\nboy, you mean you slept?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231132096, "name": "bagel queen⁶", "screen_name": "nidoqueeen", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2010-12-27"), "description": "| twenty one | hockey | chicago | i'm probably crying about something |", "followers_count": 422, "friends_count": 351, "statues_count": 15136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754639307907072, "text": "@Virginers_ I'll take some chocolate covered strawberries pwease", "in_reply_to_status": 727754309685923840, "in_reply_to_user": 578847759, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 578847759 }}, "user": { "id": 366850701, "name": "miranda", "screen_name": "TheRandaRoo", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "There’s a little bit of devil in her angel eyes", "followers_count": 583, "friends_count": 433, "statues_count": 41214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-05-04T00:00:01.000Z"), "id": 727754640549359618, "text": "Celebrating not 1, but 2 birthdays!!!!! @ Farrell's Hawaii https://t.co/U3PaX88PZN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.94047,21.38485"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36887546, "name": "Sue B", "screen_name": "MomAndNavyWife", "lang": "en", "location": "Honolulu, HI/San Diego, CA", "create_at": date("2009-04-30"), "description": "Mom. Wife. Christian. Editor. Musically inclined. Laughter addict. California born, Hawaii living. For all my social media, use momandnavywife", "followers_count": 392, "friends_count": 246, "statues_count": 11214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754640914296832, "text": "Like a good neighbor, State Farm is there... with a nurse? @rypimentel", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304819342 }}, "user": { "id": 2517076249, "name": "Chelsea", "screen_name": "cjenkins7777", "lang": "en", "location": "Chico, CA", "create_at": date("2014-05-22"), "description": "Like a beach chair", "followers_count": 201, "friends_count": 340, "statues_count": 457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754640998141956, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":52.7°F Wind:0.0mph Pressure: 29.82hpa Falling slowly Rain Today 0.00in. Forecast: Occasional precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 316964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754641086222337, "text": "@Nostalgic_Jay too bad I gotta wait till Thursday to get lit and listen to it ����", "in_reply_to_status": 727754403529265152, "in_reply_to_user": 366712321, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 366712321 }}, "user": { "id": 366712321, "name": "Silly Bitch", "screen_name": "Nostalgic_Jay", "lang": "en", "location": "null", "create_at": date("2011-09-02"), "description": "null", "followers_count": 286, "friends_count": 234, "statues_count": 22845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sparks, NV", "id": "140800566259f12f", "name": "Sparks", "place_type": "city", "bounding_box": rectangle("-119.781386,39.511797 -119.640003,39.636636") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268400, "cityName": "Sparks" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754641124003842, "text": "The adventure begins. Next stop waltdisneyworld #disneysmmc @amtourister #packmorefun #travel… https://t.co/dZCFcmC7HD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.389544,37.615608"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "disneysmmc", "packmorefun", "travel" }}, "user_mentions": {{ 1252122272 }}, "user": { "id": 21841580, "name": "GayleT", "screen_name": "DisneyGals", "lang": "en", "location": "Educator/Freelance Writer", "create_at": date("2009-02-24"), "description": "Mommy #Educator #Writer #Explorer #Travel #ALIfeLessPlanned #DisneySMMC #PeanutsAmbassador ||Looking for the next big adventure one ice tea at a time", "followers_count": 74226, "friends_count": 67526, "statues_count": 71177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754641144950784, "text": "Slow down grasshopper. Patience is a virtue.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4171428433, "name": "Mr. Insatiable", "screen_name": "TheScottoShow", "lang": "en", "location": "null", "create_at": date("2015-11-08"), "description": "apply pressure.", "followers_count": 636, "friends_count": 624, "statues_count": 5558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, TX", "id": "85d3434ace478e35", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-95.600652,30.643296 -95.493887,30.768881") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48471, "countyName": "Walker", "cityID": 4835528, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754641337921537, "text": "Yatpu nanaman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 300161812, "name": "ㅤㅤjed", "screen_name": "jedpasco", "lang": "en", "location": "San Jose, CA", "create_at": date("2011-05-17"), "description": "null", "followers_count": 664, "friends_count": 655, "statues_count": 29797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754641656655873, "text": "Wind 1.0 mph SE. Barometer 29.812 in, Falling Rapidly. Temperature 52.4 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642017390593, "text": "#Azawesome I'm working up some good material, but won't get disqualified for jumping the gun... https://t.co/Z3y3Q2ngu1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Azawesome" }}, "user": { "id": 637298740, "name": "Hunter Moore", "screen_name": "ehuntermoore", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "Arizonan. Proud to be involved. All views are my own.", "followers_count": 206, "friends_count": 233, "statues_count": 614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642130649089, "text": "3am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 52804631, "name": "Alex Kern", "screen_name": "AlexxKerrrr", "lang": "en", "location": "Pennsylvania", "create_at": date("2009-07-01"), "description": "ESU '18. Pharmacy Tech. PRPA. Most likely sleeping.", "followers_count": 291, "friends_count": 251, "statues_count": 27954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Stroudsburg, PA", "id": "7dc76c5fd4c473fb", "name": "East Stroudsburg", "place_type": "city", "bounding_box": rectangle("-75.19626,40.983946 -75.161432,41.02404") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42089, "countyName": "Monroe", "cityID": 4221872, "cityName": "East Stroudsburg" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642130669569, "text": "Temp 44.0° Hi/Lo 45.4/44.0 Rng 1.4° WC 44.0° Hmd 95% Rain 0.00\" Storm 0.53\" BAR 29.871 Falling DP 42.7° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 121, "statues_count": 18414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642193534977, "text": "Wind 0.0 mph ---. Barometer 29.816 in, Falling. Temperature 44.8 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642290003970, "text": "Abercrombie will you sign off on this? https://t.co/0Zh7NyxP1a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1520656195, "name": "Alexis Turley-Young", "screen_name": "Turley_Young", "lang": "en", "location": "null", "create_at": date("2013-06-15"), "description": "hi mom", "followers_count": 509, "friends_count": 357, "statues_count": 4221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754642558464002, "text": "Try https://t.co/UbyiJHObN5 Four Piece Fried Chicken Dinner with cole slaw, baked beans and garlic #Fried #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.806241,41.40353"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Fried", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4399, "friends_count": 875, "statues_count": 424089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brook Park, OH", "id": "dac5f373b58024de", "name": "Brook Park", "place_type": "city", "bounding_box": rectangle("-81.885646,41.386334 -81.780301,41.42068") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3909288, "cityName": "Brook Park" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754643145633793, "text": "Temp: 55.3°F Wind:0.0mph Pressure: 29.961hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754643292426240, "text": "can't sleep bc there's too much on my mind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2961545341, "name": "jules", "screen_name": "izujulia", "lang": "en", "location": "null", "create_at": date("2015-01-06"), "description": "wrhs❤️ @catdeguzmann @aliza_hoskins @maddieponderr @mialachman @KarissaSeitz", "followers_count": 309, "friends_count": 308, "statues_count": 1178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754643518963713, "text": "@Elias_Sabbagh_ https://t.co/UehCTTM7rl", "in_reply_to_status": -1, "in_reply_to_user": 2358258337, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2358258337 }}, "user": { "id": 840963384, "name": "Filbert Tonyworthy", "screen_name": "andreylull", "lang": "en", "location": "Fountain Hills, AZ", "create_at": date("2012-09-22"), "description": "the secret to happiness is burying all of your true feelings, and living a life of bland compromise", "followers_count": 470, "friends_count": 364, "statues_count": 1764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Hills, AZ", "id": "5faafada28b440c3", "name": "Fountain Hills", "place_type": "city", "bounding_box": rectangle("-111.7834,33.567198 -111.698379,33.640623") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 425300, "cityName": "Fountain Hills" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754643892244483, "text": "Wind 0.0 mph ---. Barometer 29.953 in, Steady. Temperature 51.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754643992915968, "text": "62.5F (Feels: 62.5F) - Humidity: 86% - Wind: 3.8mph NW - Gust: 6.0mph - Pressure: 1013.8mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754644181680129, "text": "They're not expendable", "in_reply_to_status": 727754196024475649, "in_reply_to_user": 298319930, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298319930, "name": "Boy Namalipit", "screen_name": "I_Am_This_Tom", "lang": "en", "location": "http://tomnipotence.com", "create_at": date("2011-05-13"), "description": "see... what had happened was...", "followers_count": 109, "friends_count": 497, "statues_count": 49244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754644424949760, "text": "I had to let go of us, to show myself what I could do, and that juss didn't sit right with you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173640957, "name": "CT", "screen_name": "C_Topps", "lang": "en", "location": "Seattle. ", "create_at": date("2010-08-01"), "description": "2⃣3⃣ & Gettin It #TMC #GOLDEN #GoHawks ♊️", "followers_count": 968, "friends_count": 419, "statues_count": 93798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tukwila, WA", "id": "9883a4be0cbd497c", "name": "Tukwila", "place_type": "city", "bounding_box": rectangle("-122.318673,47.407147 -122.241356,47.531957") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5372625, "cityName": "Tukwila" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754644571734016, "text": "@JusTwoFive lmao, ok. my bad my bad����", "in_reply_to_status": 727754457530957824, "in_reply_to_user": 2733866215, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2733866215 }}, "user": { "id": 349899639, "name": "Caitlin Grayson", "screen_name": "KinggCaity", "lang": "en", "location": "livin' & ridin'", "create_at": date("2011-08-06"), "description": "SC: kvngcaity insta: caity.jayy", "followers_count": 1110, "friends_count": 644, "statues_count": 30740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-04T00:00:02.000Z"), "id": 727754644735283204, "text": "Sometimes I refuse to look at my bank account because I don't want to know how much I've spent...on food☹️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71179383, "name": "Evy", "screen_name": "TukanSamm", "lang": "en", "location": "the golden state ©@", "create_at": date("2009-09-02"), "description": "Chunky, yet funky.", "followers_count": 210, "friends_count": 301, "statues_count": 8440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754645582585857, "text": "Wind 0.0 mph NE. Barometer 29.83 in, Falling slowly. Temperature 49.6 °F. Rain today 0.01 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754645976842242, "text": "Wind 0.0 mph NE. Barometer 29.812 in, Falling. Temperature 36.2 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754646148796416, "text": "Ok but what this gotta do with ass https://t.co/6IMHTwcjkp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312049460, "name": "Dat Boi", "screen_name": "FinalxDoctrine", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-10"), "description": "Libertarian piece of shit", "followers_count": 1000, "friends_count": 669, "statues_count": 24978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754646379474944, "text": "@taytux HAPPY BIRTHDAY LOVE U", "in_reply_to_status": -1, "in_reply_to_user": 3168561489, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3168561489 }}, "user": { "id": 611186548, "name": "Sean Penn", "screen_name": "NatSom3", "lang": "en", "location": "cool dog rekords", "create_at": date("2012-06-17"), "description": "~~~~~~35 year old man~~~~~~im going places", "followers_count": 831, "friends_count": 366, "statues_count": 32497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport Beach, CA", "id": "0ce4c27ce26030aa", "name": "Newport Beach", "place_type": "city", "bounding_box": rectangle("-117.958219,33.563618 -117.789845,33.671658") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 651182, "cityName": "Newport Beach" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754646530445312, "text": "Early morning San Francisco from San Bruno Mountain #sanfrancisco #sanbruno #view #vista… https://t.co/uHjnrLckph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.42972222,37.68555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sanfrancisco", "sanbruno", "view", "vista" }}, "user": { "id": 2825385016, "name": "Bort Edwards", "screen_name": "BortEdwards", "lang": "en", "location": "Smithsonian Nat'l History Musm", "create_at": date("2014-10-12"), "description": "Australian evolutionary biologist, phylogeneticist, botanist, postdoc adrift in the US of A.", "followers_count": 475, "friends_count": 759, "statues_count": 1664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brisbane, CA", "id": "2ef9cb620c2bd681", "name": "Brisbane", "place_type": "city", "bounding_box": rectangle("-122.456052,37.66469 -122.380944,37.70837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 608310, "cityName": "Brisbane" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754646824038401, "text": "I just noticed I cleaned my room yesterday...it's junky again ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230009807, "name": "Tread", "screen_name": "_Ahdeyemi", "lang": "en", "location": "S.C.", "create_at": date("2010-12-23"), "description": "South Carolina -843- born and raised, #NCAT19 Aerospace Mechanical Engineering Major ✈️ Sway with the tide for good vibes ✌✊", "followers_count": 613, "friends_count": 517, "statues_count": 8723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754646983446529, "text": "@bigolmadison twitter better thank me for helping you choose which ones to post smh.", "in_reply_to_status": 726649659330957314, "in_reply_to_user": 3035318058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3035318058 }}, "user": { "id": 1228482799, "name": "jacob", "screen_name": "DuenezJacob", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-02-28"), "description": "19:: US Army POG.", "followers_count": 233, "friends_count": 96, "statues_count": 13629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prescott Valley, AZ", "id": "004eced64998b95b", "name": "Prescott Valley", "place_type": "city", "bounding_box": rectangle("-112.383963,34.547369 -112.258589,34.687925") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4025, "countyName": "Yavapai", "cityID": 457450, "cityName": "Prescott Valley" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754647247724548, "text": "Ringrazio @antifabiozeta per avermi consigliato questo libro straordinario https://t.co/YsIPfRRAlg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 561085864 }}, "user": { "id": 61162279, "name": "giulio verme", "screen_name": "zeropregi", "lang": "it", "location": "Roma", "create_at": date("2009-07-29"), "description": "Amo la digos, i carabinieri, gli scout e i cavalieri di Malta.", "followers_count": 24836, "friends_count": 2503, "statues_count": 212027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754647348326402, "text": "Sometimes love comes around and it knocks you down just get back up when it knocks you down", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490857477, "name": "stefmoren™", "screen_name": "stefani_erin", "lang": "en", "location": "null", "create_at": date("2012-02-12"), "description": "snap/insta- stefamoren. I also like @bigandsexy21 and @billdo10.", "followers_count": 891, "friends_count": 381, "statues_count": 6109 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Prairie, MN", "id": "008b76e07bb8f8da", "name": "La Prairie", "place_type": "city", "bounding_box": rectangle("-93.512102,47.220597 -93.476639,47.245231") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27061, "countyName": "Itasca", "cityID": 2735648, "cityName": "La Prairie" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754647398690817, "text": "Update: I dran all the lean in dead rising and ended up dropping a mixtape which cured the zombie apocalypse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2650090759, "name": "Jim Lahey", "screen_name": "_bad_brains", "lang": "en", "location": "satx", "create_at": date("2014-07-15"), "description": "up from the 36 chambers", "followers_count": 278, "friends_count": 601, "statues_count": 6890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754647495188480, "text": "@NickSwagyPYoung hey dad", "in_reply_to_status": 727753978876932096, "in_reply_to_user": 2242915207, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2242915207 }}, "user": { "id": 852342428, "name": "nick.", "screen_name": "TheRealNickGtz", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-09-28"), "description": "Feel The Bern!", "followers_count": 290, "friends_count": 272, "statues_count": 10754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754648099131393, "text": "It is what it is b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 942928902, "name": "Batalina YoungBlood", "screen_name": "CatMelgo12", "lang": "en", "location": "Kansas City", "create_at": date("2012-11-11"), "description": "19 || College Freshman.", "followers_count": 549, "friends_count": 849, "statues_count": 11944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, KS", "id": "1276a44d783a529d", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-96.649744,39.159985 -96.499443,39.259872") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20161, "countyName": "Riley", "cityID": 2044250, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754648212377600, "text": "One more day... ☝��️������❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 221563766, "name": "Abel Guzman", "screen_name": "Amazing_Abel93", "lang": "en", "location": "San Ysidro", "create_at": date("2010-11-30"), "description": "You don't want to get advice from me, trust me, you'll just end up drunk. Sangre Caliente. Snapchat: amazing_abel", "followers_count": 496, "friends_count": 251, "statues_count": 40413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754648237527040, "text": "Wind 0.0 mph ---. Barometer 30.058 in, Rising slowly. Temperature 51.5 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754648568893440, "text": "@FilthyTomLawlor @davemeltzerWON is the worlds biggest MARK what's he know? Says who ?", "in_reply_to_status": 727738586079109120, "in_reply_to_user": 40549967, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40549967, 799815672 }}, "user": { "id": 100678782, "name": "phil baroni", "screen_name": "philbaroni", "lang": "en", "location": "N 37� 13' 0'' / W 121� 46' 0''", "create_at": date("2009-12-30"), "description": "The Official Twitter Of The New York Bad Ass Phil Baroni.Check out my you tube & my site! http://youtube.com/thenewyorkbadass http://thenewyorkbadass.tumblr.com", "followers_count": 50834, "friends_count": 1552, "statues_count": 43878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-04T00:00:03.000Z"), "id": 727754648631816192, "text": "Yet another good one. What do you do when you miss your in-laws? Reload and fire again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.0831018,32.437873"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 876251324, "name": "Dave McIlreath", "screen_name": "djmsgm", "lang": "en", "location": "null", "create_at": date("2012-10-12"), "description": "null", "followers_count": 11, "friends_count": 20, "statues_count": 1229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marana, AZ", "id": "5ea979bd3146333d", "name": "Marana", "place_type": "city", "bounding_box": rectangle("-111.234057,32.308696 -111.036975,32.467327") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 444270, "cityName": "Marana" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754649248399360, "text": "@glnzs نه محدوده. گرونه ولی. الان یه ساله هیچ مشکلی نداشتم", "in_reply_to_status": 727754443333246976, "in_reply_to_user": 1281563635, "favorite_count": 0, "retweet_count": 0, "lang": "fa", "is_retweet": false, "user_mentions": {{ 1281563635 }}, "user": { "id": 1511480538, "name": "برنارد", "screen_name": "__Bernard_", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "من طرب اندر طربم", "followers_count": 1695, "friends_count": 142, "statues_count": 14905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754649252581377, "text": "Wind 8.0 mph W. Barometer 29.729 in, Falling slowly. Temperature 57.9 °F. Rain today 0.00 in. Humidity 47%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754649537761280, "text": "@thatboy_Donaldd no problem Donald!", "in_reply_to_status": 727726875951378432, "in_reply_to_user": 1041659180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1041659180 }}, "user": { "id": 1096904942, "name": "Dolobradda", "screen_name": "dolobradda", "lang": "en", "location": "null", "create_at": date("2013-01-16"), "description": "We're all in the same game, just different levels. Dealing with the same hell, just different devils.. R.I.P Linden boy", "followers_count": 673, "friends_count": 686, "statues_count": 19315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ko Olina, HI", "id": "011f8b0a63816c7a", "name": "Ko Olina", "place_type": "city", "bounding_box": rectangle("-158.1293,21.322963 -158.089509,21.349347") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1539400, "cityName": "Ko Olina" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754649592303616, "text": "@ChiefCanuck @GrimBrotherOne I. Love. Our. Community.", "in_reply_to_status": 727754540582359040, "in_reply_to_user": 219115948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219115948, 61598133 }}, "user": { "id": 1966060218, "name": "Kenny (Spartacat)", "screen_name": "TheSpartacat", "lang": "en", "location": "Seattle, WA", "create_at": date("2013-10-16"), "description": "YouTuber on @Ultimate_Halo | @PBLHalo Producer | Contact: ultimatehaloteam@gmail.com | 53K subscribers | #Caturday | #GoodMeowning", "followers_count": 2578, "friends_count": 588, "statues_count": 6440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754650615689216, "text": "oh my �� https://t.co/d0nS0wVqHm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122224260, "name": "psycho", "screen_name": "angelicaguting", "lang": "en", "location": "null", "create_at": date("2010-03-11"), "description": "rip ecd and glw ♡", "followers_count": 848, "friends_count": 528, "statues_count": 11942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754650632519680, "text": "some harold's would be so flame rn https://t.co/OOe5cIsG6E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 306771741, "name": "TPC King Kendall. ♎️", "screen_name": "TheQueenBBW", "lang": "en", "location": "708 to Macomb to Edwardsville ", "create_at": date("2011-05-28"), "description": "NappyTwitter Princess.❤️ unapologetically black & educated. sapiosexual. BBW. #WIU19 to #SIUe19 (soon). #BLM", "followers_count": 7927, "friends_count": 6778, "statues_count": 124259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Macomb, IL", "id": "1af01854e44e009a", "name": "Macomb", "place_type": "city", "bounding_box": rectangle("-90.72417,40.433688 -90.624248,40.48538") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17109, "countyName": "McDonough", "cityID": 1745889, "cityName": "Macomb" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754651202945024, "text": "WHY TF ARE WE GETTING PULLED OVER ???!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314061510, "name": "becky the headass.", "screen_name": "AlexandriaFlexx", "lang": "en", "location": "da naaasty ", "create_at": date("2011-06-09"), "description": "my main goal is to blow up & then act like i dont know nobody |19|", "followers_count": 2977, "friends_count": 975, "statues_count": 62295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754651496525824, "text": "Sc Cassidy.pauline", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 1115677908, "name": "Cass◡̈⃝", "screen_name": "Cassssidddyyy", "lang": "en", "location": "in bed", "create_at": date("2013-01-23"), "description": "❁❂✷✵ ◡̈♒️☠❁❂ sc: cassidy.pauline", "followers_count": 600, "friends_count": 376, "statues_count": 23269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754651890765824, "text": "@TriZTroCiTy @xComments_", "in_reply_to_status": 727754582059831297, "in_reply_to_user": 3075046449, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3075046449, 1206774540 }}, "user": { "id": 3075046449, "name": "TriZingis", "screen_name": "TriZTroCiTy", "lang": "en", "location": "Long Branch", "create_at": date("2015-03-06"), "description": "|| Ex Multi CoD Pro || Owner of DFS Domination the best DFS lineup provider http://bit.ly/1PPArNB || Avid Sport Fan || Yankees ll Buccs ll Knicks", "followers_count": 1207, "friends_count": 534, "statues_count": 12130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Branch, NJ", "id": "97e1b68100f6629d", "name": "Long Branch", "place_type": "city", "bounding_box": rectangle("-74.012435,40.260557 -73.974657,40.328156") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3441310, "cityName": "Long Branch" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754652108861440, "text": "@LILH4LFDEAD bruh im about to ko lmao", "in_reply_to_status": 727754435066249217, "in_reply_to_user": 615692271, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 615692271 }}, "user": { "id": 4779401647, "name": "PLAYGOD", "screen_name": "Irolldat", "lang": "en", "location": "null", "create_at": date("2016-01-18"), "description": "I eat ice cream with my chicken", "followers_count": 75, "friends_count": 144, "statues_count": 463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754652356382721, "text": "I've never had Popeyes but this is how I am with ice cream https://t.co/heqiCfQKIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 909228775, "name": "Armenian Princess ❁", "screen_name": "PrincessRajaaa", "lang": "en", "location": "Corona, CA", "create_at": date("2012-10-27"), "description": "R.I.P Grandma ❤. February 19th. snapchat : rajamariahhh", "followers_count": 1054, "friends_count": 638, "statues_count": 22908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754652553519105, "text": "@TDOLLAFROMTHEC hahahaha booshie bitch. My mind is boggled right now.", "in_reply_to_status": 727754355017957377, "in_reply_to_user": 234636834, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 234636834 }}, "user": { "id": 223000066, "name": "I'm not Mexican", "screen_name": "IzzzyIsaac", "lang": "en", "location": "Oakland, CA", "create_at": date("2010-12-04"), "description": "I get drunk alot. United States Army Vet. 22. #RaiderNation #OaklandAthletics #OaklandWarriors", "followers_count": 2598, "friends_count": 504, "statues_count": 62025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-04T00:00:04.000Z"), "id": 727754652628946945, "text": "Trying to hide everything through me smiling all the time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220834688, "name": "Rocket", "screen_name": "Rockhead_Dafool", "lang": "en", "location": "Ijebu-Ode, Nigeria", "create_at": date("2010-11-28"), "description": "Rest in peace Newton Ananti.", "followers_count": 1531, "friends_count": 1149, "statues_count": 65709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754653266513922, "text": "I'm up af. Like not sleepy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2214273984, "name": "champagne mami", "screen_name": "syramai", "lang": "en", "location": "Denver, CO", "create_at": date("2013-11-25"), "description": "Chase me, I'm chubby. #gramfam #LongLiveTra", "followers_count": 1205, "friends_count": 986, "statues_count": 63468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754653329436675, "text": "Earlier today I used the term \"hashtag blessed\" on my final exam essay and I still haven't decided if I love or hate myself for it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 369254563, "name": "Maddawg", "screen_name": "madison_lapke", "lang": "en", "location": "null", "create_at": date("2011-09-06"), "description": "be happy. be weird. Iowa State University || ΑΓΔ", "followers_count": 635, "friends_count": 493, "statues_count": 4558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ames, IA", "id": "d6794586b08b7f9c", "name": "Ames", "place_type": "city", "bounding_box": rectangle("-93.718311,41.986006 -93.560416,42.079007") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1901855, "cityName": "Ames" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754653627240448, "text": "@vaaannee_ word to Rob Dyrdek ( ͡° ͜ʖ ͡°)", "in_reply_to_status": 727754486735880193, "in_reply_to_user": 2956619178, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 2956619178 }}, "user": { "id": 359204673, "name": "( ͡° ͜ʖ ͡°)", "screen_name": "Brand0nGarcia", "lang": "en", "location": "null", "create_at": date("2011-08-20"), "description": "null", "followers_count": 454, "friends_count": 187, "statues_count": 86126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754653828526081, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 584217506, "name": "Jalin", "screen_name": "JHolden_04", "lang": "en", "location": "Converse, TX ➡️ Kerrville, Tx", "create_at": date("2012-05-18"), "description": "Just tryna be successful #SUBasketball (life is like photography you need the negatives in order to develop)", "followers_count": 869, "friends_count": 452, "statues_count": 36683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kerrville, TX", "id": "9e1bc6eba1d7d97c", "name": "Kerrville", "place_type": "city", "bounding_box": rectangle("-99.22603,29.983238 -99.098045,30.094147") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48265, "countyName": "Kerr", "cityID": 4839040, "cityName": "Kerrville" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754654319296513, "text": "@MsCBBoots @P0TUSTrump You mean you'll vote to cede #POTUS to #hillary2016 - because you hate your country. Just leave, eh?", "in_reply_to_status": 727753954541637632, "in_reply_to_user": 268477981, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "POTUS", "hillary2016" }}, "user_mentions": {{ 268477981, 2534195028 }}, "user": { "id": 299875937, "name": "Mark Becerra Sr.", "screen_name": "MarkBecerraSr", "lang": "en", "location": "Maui", "create_at": date("2011-05-16"), "description": "Intercessor, blissfully wed to Michelle. Dad & Papa, Gospel relief for homeless, golfer, surf, prolife, Maui! I believe in USA's Christian history. #Trump2016", "followers_count": 2625, "friends_count": 4993, "statues_count": 15310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pukalani, HI", "id": "2dab465cb5edceae", "name": "Pukalani", "place_type": "city", "bounding_box": rectangle("-156.359376,20.804118 -156.318233,20.850458") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1565900, "cityName": "Pukalani" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754654327689217, "text": "@Xtinaaa16 the Kat Von D contour palette is so much better��", "in_reply_to_status": 727754103665901569, "in_reply_to_user": 738326846, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 738326846 }}, "user": { "id": 235823744, "name": "Virginia P. Caldera", "screen_name": "virginia_loves", "lang": "en", "location": "null", "create_at": date("2011-01-08"), "description": "blessed", "followers_count": 626, "friends_count": 405, "statues_count": 31792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754654675767296, "text": "this is literally my day one, ride or die, my nigga, brother and anything.. straight family, happy bday❤️��☄�� #juice https://t.co/k4lJnGyv1E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "juice" }}, "user": { "id": 2828233357, "name": "Caden Weible", "screen_name": "cbearfool", "lang": "en", "location": "surfing", "create_at": date("2014-09-23"), "description": "ILS", "followers_count": 776, "friends_count": 463, "statues_count": 6045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754654856155141, "text": "Wow the dude working at Kati personally came and brought me two drinks when I only ordered one �� #qualityservice #keyToSuccess", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "qualityservice", "keyToSuccess" }}, "user": { "id": 44794444, "name": "Champagne Mamí", "screen_name": "shahrayray", "lang": "en", "location": "NY", "create_at": date("2009-06-04"), "description": "Change Is The Only Constant ◆ ΣΣΡ #RU", "followers_count": 469, "friends_count": 406, "statues_count": 13543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754655153917952, "text": "Liquor stores open 24 hrs, everyone on Bourbon being drunk and happy, and especially all the cute coffee places ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 134972025, "name": "soriano.", "screen_name": "cassayyye", "lang": "en", "location": "Texas, USA", "create_at": date("2010-04-19"), "description": "my dog is probs cooler than u #excelsior", "followers_count": 573, "friends_count": 430, "statues_count": 47548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754655619530754, "text": "终于! (at Squire Village) [pic] — https://t.co/9Fcx0Heo7v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.56382,42.44964"), "retweet_count": 0, "lang": "zh", "is_retweet": false, "user": { "id": 353140928, "name": "陆佰圆_theLegendLau", "screen_name": "NOS_ATX", "lang": "en", "location": "Beijing", "create_at": date("2011-08-11"), "description": "伪果粉,真Tifosi,流窜于各大社交网站,久居于傲娇百度贴吧!BTW: BBC F1 is better than Sky Sports", "followers_count": 106, "friends_count": 185, "statues_count": 2191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunderland, MA", "id": "00b5a72809afb292", "name": "Sunderland", "place_type": "city", "bounding_box": rectangle("-72.585253,42.409418 -72.540423,42.499613") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25015, "countyName": "Hampshire" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754655900549120, "text": "please fucking text me and then don't reply I love that shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99452431, "name": "Gaybee", "screen_name": "gabiebear", "lang": "en", "location": "null", "create_at": date("2009-12-25"), "description": "get drunk on the good life, i'll take you to paradise.", "followers_count": 370, "friends_count": 352, "statues_count": 20322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palm Springs, CA", "id": "4265ece9285a2872", "name": "Palm Springs", "place_type": "city", "bounding_box": rectangle("-116.567788,33.779426 -116.466791,33.870733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 655254, "cityName": "Palm Springs" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754656445771776, "text": "ugh ugh ugh i love anything that shows off ppls chests hehehhahh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3008834227, "name": "Ginger Wills", "screen_name": "KomunHorangi", "lang": "en", "location": "Oakland, CA", "create_at": date("2015-02-02"), "description": "into: DC/jason todd/OCs/vocaloid\n+\nmy girlfriend is @irleerah and i love her w my entire being", "followers_count": 746, "friends_count": 247, "statues_count": 2765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754656911339520, "text": "Wind 0 mph ESE. Barometer 1013.7 hPa, Falling. Temperature 76.8 °F. Rain today 0.00 in. Humidity 32%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 155210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-04T00:00:05.000Z"), "id": 727754657343414273, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 348262906, "name": "kitsune⚡️", "screen_name": "kamarisyrie", "lang": "en", "location": "Cali made...Texas raised", "create_at": date("2011-08-03"), "description": "#NZV #PGL", "followers_count": 1222, "friends_count": 942, "statues_count": 42521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hurst, TX", "id": "6b09f178b115813c", "name": "Hurst", "place_type": "city", "bounding_box": rectangle("-97.212945,32.804103 -97.135464,32.889346") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4835576, "cityName": "Hurst" } }
+{ "create_at": datetime("2016-05-04T00:00:06.000Z"), "id": 727754657842475009, "text": "Yesterday was a great day because of you @blakefriley7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1450448118 }}, "user": { "id": 3354219549, "name": "Cøtton", "screen_name": "Dillon16Joiner", "lang": "en", "location": "Meridian/Gulfport MS", "create_at": date("2015-07-01"), "description": "No such thing as a life that's better than yours.", "followers_count": 179, "friends_count": 260, "statues_count": 1011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, MS", "id": "57dc070bcd27882d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-88.763404,32.312389 -88.655698,32.443049") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28075, "countyName": "Lauderdale", "cityID": 2846640, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-05-04T00:00:06.000Z"), "id": 727754658115129344, "text": "I like the part in Views where it's like no no no no no no no no no", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455356330, "name": "Fire & Portayolo", "screen_name": "portayolo", "lang": "en", "location": "coog nation ", "create_at": date("2013-05-24"), "description": "yes plz", "followers_count": 911, "friends_count": 1363, "statues_count": 11801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-04T00:00:06.000Z"), "id": 727754658605830144, "text": "@saravalerianp yo no si no me contestas por wha��", "in_reply_to_status": 727725146870534144, "in_reply_to_user": 1062511554, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1062511554 }}, "user": { "id": 633503604, "name": "MG", "screen_name": "ArreolaMariana", "lang": "en", "location": "Tucson, AZ", "create_at": date("2012-07-11"), "description": "AZ|MX / SC&Instagram: arreolamariana / BUCEC", "followers_count": 495, "friends_count": 551, "statues_count": 7379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-04T00:00:06.000Z"), "id": 727754658979192832, "text": "Haven't seen @Jackieeegarcia_ in 5 years & of course the 1st time we hangout there's 2 football players going out of their way to hit on her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223005164 }}, "user": { "id": 909765667, "name": "Himani", "screen_name": "himannniiiii", "lang": "en", "location": "null", "create_at": date("2012-10-28"), "description": "null", "followers_count": 316, "friends_count": 227, "statues_count": 8488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-04T00:00:06.000Z"), "id": 727754659000094721, "text": "418 #NHL17Benn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "NHL17Benn" }}, "user": { "id": 17904694, "name": "April May", "screen_name": "musiclovingirl", "lang": "en", "location": "Dixon, Illinois", "create_at": date("2008-12-05"), "description": "MUSIC, Chicago Bears, Blackhawks. Rockford Icehogs, & Chicago Cubs, Horror movies, Penguins & Lions, im shy, but dont take it as a weakness. ZOMBIE CROTCH!", "followers_count": 386, "friends_count": 1079, "statues_count": 28351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dixon, IL", "id": "8f16af9f16f59631", "name": "Dixon", "place_type": "city", "bounding_box": rectangle("-89.520108,41.817427 -89.438924,41.900046") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17103, "countyName": "Lee", "cityID": 1720162, "cityName": "Dixon" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117020256964608, "text": "FUCK https://t.co/klm0eqfLFc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4215908778, "name": "WHATS2+2?", "screen_name": "JontayviaLove", "lang": "en", "location": "null", "create_at": date("2015-11-12"), "description": "Instagram @jontayviaaugustus Snapchat @jontayviabased I'm just a little lesbiana. 19. UNL.", "followers_count": 140, "friends_count": 114, "statues_count": 7754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117020621832192, "text": "Why are we like dis???! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3805888393, "name": "5'8 Chinxx", "screen_name": "Swweetchinxx", "lang": "en", "location": "Starbucks", "create_at": date("2015-10-06"), "description": "i enjoy long romantic walks........... to the fridge. Prince & Peypey's Mama | Don't DM me #TRESQUAH", "followers_count": 10624, "friends_count": 504, "statues_count": 48369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117021406191620, "text": "The media needs to hold Donald Trump accountable....Fox and Friends AND Morning Joe let all the outrageous stuff go unchallenged", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718831607688380416, "name": "Alexander Shaw", "screen_name": "Alexshawdc", "lang": "en", "location": "Washington, DC", "create_at": date("2016-04-09"), "description": "null", "followers_count": 9, "friends_count": 53, "statues_count": 3 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Washington, MD", "id": "2b8ddd2fad48eea2", "name": "Fort Washington", "place_type": "city", "bounding_box": rectangle("-77.037081,38.688359 -76.9478,38.817071") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2429525, "cityName": "Fort Washington" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117022630940673, "text": "@trackstarlife15 Aw okay lol", "in_reply_to_status": 728116514012811264, "in_reply_to_user": 261067859, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 261067859 }}, "user": { "id": 500321525, "name": "Aminah.", "screen_name": "Herrmosa_Reina", "lang": "en", "location": "University of Toledo 937 ✈ 419", "create_at": date("2012-02-22"), "description": "Blessed. ♍", "followers_count": 1932, "friends_count": 1838, "statues_count": 64909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117023113285633, "text": "#Maythe4thbewithyou everyone! #jedi might take today off but the hardworking #stormtroopers at… https://t.co/mvzX9rCgf9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.23611111,37.48277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Maythe4thbewithyou", "jedi", "stormtroopers" }}, "user": { "id": 785205565, "name": "Blu-Bin", "screen_name": "Blu_Bin", "lang": "en", "location": "United States", "create_at": date("2012-08-27"), "description": "Your friendly neighborhood 3D print shop! Making ideas into objects since 2012.", "followers_count": 802, "friends_count": 258, "statues_count": 936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redwood City, CA", "id": "a409256339a7c6a1", "name": "Redwood City", "place_type": "city", "bounding_box": rectangle("-122.28853,37.443954 -122.177339,37.550633") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 660102, "cityName": "Redwood City" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117023201316864, "text": "@ebbtideapp Tide in Skull Creek, South Carolina 05/05/2016\nHigh 8:01am 8.2\n Low 2:14pm -1.3\nHigh 8:24pm 9.3\n Low 2:51am -1.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-80.7717,32.2233"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 82, "friends_count": 1, "statues_count": 30125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Carolina, USA", "id": "6057f1e35bcc6c20", "name": "South Carolina", "place_type": "admin", "bounding_box": rectangle("-83.353955,32.04683 -78.499301,35.215449") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45013, "countyName": "Beaufort", "cityID": 4534045, "cityName": "Hilton Head Island" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117023671128065, "text": "Jk but I wish there was a store open rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377119335, "name": "Tatiana Flamenco✨", "screen_name": "TatiBabeee", "lang": "en", "location": "Covina / Disneyland", "create_at": date("2011-09-20"), "description": "Mt. SAC Cheer❤️ | Stubborn | 18♎️ | Take a walk on the wild side.", "followers_count": 1669, "friends_count": 1195, "statues_count": 26356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-05-05T00:00:00.000Z"), "id": 728117024233132032, "text": "@michael_odom24 thanks bb ��", "in_reply_to_status": 728106669016670208, "in_reply_to_user": 2746965374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2746965374 }}, "user": { "id": 1621070665, "name": "YVNGLLAMA", "screen_name": "Brianthellama", "lang": "en", "location": "Houston, TX", "create_at": date("2013-07-25"), "description": "look both ways before you cross my mind", "followers_count": 187, "friends_count": 513, "statues_count": 2989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cove, TX", "id": "011f5b2126127c09", "name": "Cove", "place_type": "city", "bounding_box": rectangle("-94.872268,29.783551 -94.801584,29.845599") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48071, "countyName": "Chambers", "cityID": 4817336, "cityName": "Cove" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117024514179073, "text": "Im at the point now where Idgaf. if you fwm then good, if you dont then dont lol cus ain forcing shit no mo ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235337856, "name": "Vonté World", "screen_name": "el_DineroBSM", "lang": "en", "location": "Jackson, MS/ U.S.A", "create_at": date("2011-01-07"), "description": "22 | Ⓜ️ississippi living | Sc: xtrillll | IG @xtrilll__| Ju'City", "followers_count": 1353, "friends_count": 1506, "statues_count": 15600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl, MS", "id": "04b55f8bc56b0df2", "name": "Pearl", "place_type": "city", "bounding_box": rectangle("-90.184695,32.23749 -90.037767,32.324508") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28121, "countyName": "Rankin", "cityID": 2855760, "cityName": "Pearl" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117024635805697, "text": "\"You can take a nigga out the ghetto, but you can't take the ghetto outta me\" https://t.co/oLw3wchfCu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 834788425, "name": "☹️無限☹️", "screen_name": "SaddBoiiDL", "lang": "en", "location": " Bay Area", "create_at": date("2012-09-19"), "description": "I just tweet, dont bother me| Susexual | #SaddBoiz X #IgnantTribe | DM's Closed | I'm not a crip", "followers_count": 5251, "friends_count": 3522, "statues_count": 138592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117024841289728, "text": "Tammy and Jim found at 33.905 Latitude, -84.306 Longitude. They have traveled 0.0 Miles since the last update. https://t.co/RyAM0ts3lY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.30575333,33.90484833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 726423052490276864, "name": "Tammy & Jim", "screen_name": "WhereRTammyNJim", "lang": "en", "location": "Chamblee, GA", "create_at": date("2016-04-30"), "description": "null", "followers_count": 4, "friends_count": 0, "statues_count": 61 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookhaven, GA", "id": "00c26afc77ee7aaa", "name": "Brookhaven", "place_type": "city", "bounding_box": rectangle("-84.348581,33.825123 -84.28156,33.923964") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1310944, "cityName": "Brookhaven" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117024841310208, "text": "You see id go to the casino or something but I have to open in the morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2217418550, "name": "Go Home", "screen_name": "JustIsaiahP", "lang": "en", "location": "Ruthless", "create_at": date("2013-11-27"), "description": "line up the Flex", "followers_count": 1555, "friends_count": 695, "statues_count": 44610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117024958746624, "text": "Me waiting for a certain review embargo to lift tonight...\n@Naughty_Dog @Neil_Druckmann @nolan_north @TroyBakerVA https://t.co/HKZCicBT8F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15222083, 33030256, 299776583, 434725454 }}, "user": { "id": 721600813, "name": "Nate McKinney", "screen_name": "Darth__Nater", "lang": "en", "location": "Valparaiso, IN", "create_at": date("2012-07-27"), "description": "Lover of video games. Movie buff. Nerd stuff. CCSJ alum, journalism degree. Alcohol enthusiast. Fan of all things Kinda Funny. \n PSN: Nater1228", "followers_count": 1083, "friends_count": 2103, "statues_count": 5854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valparaiso, IN", "id": "fcb1fb67eef60e00", "name": "Valparaiso", "place_type": "city", "bounding_box": rectangle("-87.124014,41.434556 -86.988343,41.52125") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18127, "countyName": "Porter", "cityID": 1878326, "cityName": "Valparaiso" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117025206231041, "text": "Ok fuck that im goin with you , Mexico is just a poor mans Columbia lmao https://t.co/aTe3YPawKe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3015176081, "name": "Cord Ryan Melancon", "screen_name": "ThisForCord", "lang": "en", "location": "From Liberty TX to San Marcos ", "create_at": date("2015-02-03"), "description": "Texas State University c/o 16' BA Political Science #RIPWilliam #RIPFitz #FreeRenzy #FreeTreyBoy #MavsFan #SaintsFan #HookEm", "followers_count": 669, "friends_count": 403, "statues_count": 31539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117026112180226, "text": "I'm so hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3091481395, "name": "anahi ✨", "screen_name": "babyanahiii", "lang": "en", "location": "null", "create_at": date("2015-03-16"), "description": "fútbol , volleyball", "followers_count": 531, "friends_count": 284, "statues_count": 23884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117026116378624, "text": "Slide into a UNT girl DMs 9/10 you got her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240083667, "name": "Notorious REX", "screen_name": "YurBoyRexDaBest", "lang": "en", "location": "Texas, USA", "create_at": date("2011-01-18"), "description": "Music Advisor, Journalist, Media Consultant, Blogger, Hip Hop Junkie, Sports Enthusiast #NotoriousREX | 8.3.16 ❤ Alayna Pops |@IamTreyDetroit Manager", "followers_count": 1478, "friends_count": 874, "statues_count": 14130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117026317721600, "text": "I just want to wish a Happy Birthday to someone that has meant a lot to me! I love you Allie hope you have a great birthday❤️@allieesmyth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1331973828 }}, "user": { "id": 1078432494, "name": "Billy Garber", "screen_name": "Billy_Garber", "lang": "en", "location": "Phoenix Arizona", "create_at": date("2013-01-10"), "description": "null", "followers_count": 565, "friends_count": 1081, "statues_count": 5163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117026351276037, "text": "This nigga dmed me saying \"are your parents retarded bc girl you special\" I'm logging off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1925008814, "name": "BabyHaiti", "screen_name": "RosesAndRellos", "lang": "en", "location": "SlaughteringSomeonesFather", "create_at": date("2013-10-01"), "description": "Embrace your slut ❣ | Soul Snatcher|", "followers_count": 1918, "friends_count": 1142, "statues_count": 42482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warner Robins, GA", "id": "4caf02a6f195ddab", "name": "Warner Robins", "place_type": "city", "bounding_box": rectangle("-83.728243,32.495868 -83.554155,32.669392") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1380508, "cityName": "Warner Robins" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117027160788992, "text": "Wind 0.0 mph ---. Barometer 29.839 in, Steady. Temperature 55.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117027664060416, "text": "Wind 0.0 mph ---. Barometer 29.89 in, Steady. Temperature 61.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117028041543680, "text": "Wind 3.0 mph WNW. Barometer 29.755 in, Falling. Temperature 46.4 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117028087726080, "text": "Sorry, But You Know How We Get Down. ���� LMAO. https://t.co/okUCktDhjG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2579273190, "name": "BL∆iR †", "screen_name": "_blvir", "lang": "en", "location": "null", "create_at": date("2014-06-20"), "description": "Every Body Wants To Be Popular IG: @_blvir •SavannahState• | #Fleeky SC: @og_blvir", "followers_count": 1352, "friends_count": 1011, "statues_count": 16442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117028158984193, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":42.8°F Wind:0.0mph Pressure: 29.76hpa Falling slowly Rain Today 0.05in. Forecast: Occasional precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 16, "statues_count": 317054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117028343549953, "text": "05/05@03:00 - Temp 50.2F, WC 50.2F. Wind 1.5mph NE, Gust 4.0mph. Bar 29.712in, Falling slowly. Rain 0.01in. Hum 95%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-05T00:00:01.000Z"), "id": 728117028419035136, "text": "������������ https://t.co/Ehpa20UlxZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2375759558, "name": "mo...... fuck it", "screen_name": "Hotsauce_____", "lang": "en", "location": "Dumb Ass East Piedmont, CA", "create_at": date("2014-03-06"), "description": "⛩...... お金を得る", "followers_count": 302, "friends_count": 435, "statues_count": 4512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117028477763586, "text": "Ripley SW Limestone Co. Temp: 50.4°F Wind:0.7mph Pressure: 990.3mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 52924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117028834283521, "text": "Trynna get my own place , trynna have my girl spend the night and shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1634581118, "name": "takeoffffff", "screen_name": "Fred_l16", "lang": "en", "location": "null", "create_at": date("2013-07-30"), "description": "drugs are bad", "followers_count": 88, "friends_count": 56, "statues_count": 8138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117029010464768, "text": "dream dream dream when I feel blue", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2670529736, "name": "lil moo", "screen_name": "marieghodsi", "lang": "en", "location": "Chatsworth, California ", "create_at": date("2014-07-22"), "description": "Pablo's girl", "followers_count": 247, "friends_count": 149, "statues_count": 20168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117029866110978, "text": "Wind 0.0 mph NW. Barometer 29.881 in, Steady. Temperature 51.4 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 112992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030176481280, "text": "@oddNESSSS suegra?? You better not have any one catch you ��", "in_reply_to_status": 728115747772227584, "in_reply_to_user": 219582266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219582266 }}, "user": { "id": 44501205, "name": "Anna", "screen_name": "annnaiscoool", "lang": "en", "location": "null", "create_at": date("2009-06-03"), "description": "happy soul", "followers_count": 234, "friends_count": 215, "statues_count": 25356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030230990852, "text": "44 more days until graduation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.20074053,34.18152461"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 945043566, "name": "JC", "screen_name": "janelle_corona", "lang": "en", "location": "null", "create_at": date("2012-11-12"), "description": "everybody dies but not everyone lives", "followers_count": 511, "friends_count": 370, "statues_count": 14729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030243569666, "text": "Not doing drugs is so great for your complexion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 484076264, "name": "666topher", "screen_name": "reqretting", "lang": "en", "location": "no longer a small loading dock", "create_at": date("2012-02-05"), "description": "• If you fall away, then you'll die •", "followers_count": 282, "friends_count": 205, "statues_count": 7320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030348460033, "text": "@johnytiger_ Personal Jesus ��", "in_reply_to_status": 728108402577055744, "in_reply_to_user": 23998055, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 23998055 }}, "user": { "id": 169534030, "name": "Tori Fernandez", "screen_name": "toriiiii666", "lang": "en", "location": "Miami, FL", "create_at": date("2010-07-22"), "description": "Tattoo artist. Bassist. Whiskey fanatic. All-around jackass. 219 NW 20 St, Miami, FL Instagram: @toriteratism Inqueries: toritattoos@gmail.com", "followers_count": 546, "friends_count": 264, "statues_count": 23176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Terrace, FL", "id": "d82177bd727bb74a", "name": "Coral Terrace", "place_type": "city", "bounding_box": rectangle("-80.321229,25.733083 -80.286514,25.763103") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214412, "cityName": "Coral Terrace" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030637854720, "text": "�� https://t.co/LDGs4rzrzP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2150125614, "name": "Hercules", "screen_name": "Oscarin_95", "lang": "en", "location": "chillin", "create_at": date("2013-10-22"), "description": "snapchat : oscarinnn95 ig : Oscarin_95", "followers_count": 451, "friends_count": 421, "statues_count": 21927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030679797761, "text": "text me at 2 am so i know it's real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2957776756, "name": "tk", "screen_name": "taykellly", "lang": "en", "location": "null", "create_at": date("2015-01-02"), "description": "benedictine college '18 | prior lake, mn | insta: taylorkelllly", "followers_count": 776, "friends_count": 539, "statues_count": 4624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atchison, KS", "id": "8279e351583dd1d9", "name": "Atchison", "place_type": "city", "bounding_box": rectangle("-95.163642,39.535619 -95.10563,39.594778") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20005, "countyName": "Atchison", "cityID": 2002900, "cityName": "Atchison" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030767894532, "text": "Real sees real enough said... Do fake wit me #tiredofthisshit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tiredofthisshit" }}, "user": { "id": 496480257, "name": "Luke Serbus T©'$", "screen_name": "SeababySerbus", "lang": "en", "location": "null", "create_at": date("2012-02-18"), "description": "Follow @truthcrew_ making dreams happen", "followers_count": 249, "friends_count": 245, "statues_count": 1804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brookings, SD", "id": "e460ea685ce41cb4", "name": "Brookings", "place_type": "city", "bounding_box": rectangle("-96.824461,44.253728 -96.744992,44.33") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46011, "countyName": "Brookings", "cityID": 4607580, "cityName": "Brookings" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117030851743744, "text": "I wanna see this movie even though it'd probs give me anxiety throughout the whole thing https://t.co/tKdIJh58gd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1606434468, "name": "crystal.", "screen_name": "crystalgetspapi", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-19"), "description": "I know the thought of me succeeding makes a lot of people nauseous #LasGueras✨", "followers_count": 635, "friends_count": 344, "statues_count": 43245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031090819072, "text": "67.0F (Feels: 67.0F) - Humidity: 77% - Wind: 6.9mph NW - Gust: 8.3mph - Pressure: 1011.1mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031157915648, "text": "@aIexanderjames I don't eat anyways lol sooooo I'll be good there", "in_reply_to_status": 728116825083404289, "in_reply_to_user": 2735696818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2735696818 }}, "user": { "id": 1270970666, "name": "Desirae B.", "screen_name": "Dessieboo_1624", "lang": "en", "location": "null", "create_at": date("2013-03-15"), "description": "Give me kisses and call me princess", "followers_count": 163, "friends_count": 197, "statues_count": 3179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031216680962, "text": "BONNIELYNN PHOTO ART. - BONNIELYNN PHOTOGRAPHY LA, CALIFORNIA : LA2016 https://t.co/1gEcw8Icz9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 248135355, "name": "Last Pirate in LA", "screen_name": "BONNIELYNN2015", "lang": "en", "location": "#DTLA ", "create_at": date("2011-02-06"), "description": "RN/BSN/CLE :Masters in psychology #DTLA, Actress, Author , stand up artist , photographer, film artist, creative film producer, Vet, USA ARMY,", "followers_count": 224044, "friends_count": 2532, "statues_count": 132216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031417958401, "text": "allie sukis's laugh brings me pure joy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420105404, "name": "leeham", "screen_name": "Liammurrayy", "lang": "en", "location": "brighton || kzoo", "create_at": date("2011-11-23"), "description": "I dress like a hip hobo & am probably watching CNN || western michigan university 19'", "followers_count": 795, "friends_count": 650, "statues_count": 9715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031547998208, "text": "#MericaMatters #NeverHillary https://t.co/GXp3DK3lsq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MericaMatters", "NeverHillary" }}, "user": { "id": 2686010270, "name": "Kevin Zueger", "screen_name": "kazueger1", "lang": "en", "location": "Lake Havasu City, AZ", "create_at": date("2014-07-27"), "description": "Pro Life, Pro God, Pro Guns...deal with it", "followers_count": 4754, "friends_count": 4489, "statues_count": 48775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Havasu City, AZ", "id": "5f83cd36a6136fd4", "name": "Lake Havasu City", "place_type": "city", "bounding_box": rectangle("-114.37582,34.449228 -114.245399,34.532878") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave", "cityID": 439370, "cityName": "Lake Havasu City" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117031761911808, "text": "Marisol is on some inspiring shit rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1208902590, "name": "Marcos Ochoa", "screen_name": "ayeenene_12", "lang": "en", "location": "null", "create_at": date("2013-02-22"), "description": "soccer♥ #9", "followers_count": 285, "friends_count": 251, "statues_count": 12886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117032164581377, "text": "Temp 42.4° Hi/Lo 42.7/42.3 Rng 0.4° WC 42.4° Hmd 97% Rain 0.02\" Storm 0.38\" BAR 29.788 Falling DP 41.6° Wnd 0mph Dir --- Gst 12mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 121, "statues_count": 18442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117032533663748, "text": "hey @TommyHilfiger https://t.co/nr984QlvdX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 30180137 }}, "user": { "id": 318289010, "name": "taybae", "screen_name": "tayMoniquee_", "lang": "en", "location": "w/ Lucci", "create_at": date("2011-06-16"), "description": "itGrl || houtx, TxSU19 bby @YFNLUCCI", "followers_count": 2058, "friends_count": 1249, "statues_count": 32074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-05T00:00:02.000Z"), "id": 728117032600788992, "text": "MY GOOFY MOMENT�� https://t.co/9uUVkSLtGM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712698039241162752, "name": "JuicyBooty4Life", "screen_name": "Juicybooty4life", "lang": "en", "location": "United States", "create_at": date("2016-03-23"), "description": "Business Inquiries (darealjuicybootyxxx@gmail.com)Also contact me about booking(xxxbackbreaker) SERIOUS INQUIRIES ONLY❗❗", "followers_count": 1312, "friends_count": 17, "statues_count": 939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-86.418059,32.284593 -86.071398,32.443697") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117032894402560, "text": "https://t.co/cmDD9uvHBr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3419529723, "name": "Valia", "screen_name": "786_4210", "lang": "es", "location": "Florida, USA", "create_at": date("2015-08-12"), "description": "null", "followers_count": 7, "friends_count": 16, "statues_count": 584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117032974090241, "text": "3:00", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 930291607, "name": "ℛᎾⅅᎶℰℛ ℳℰᎯⅅᎾᏇѕ", "screen_name": "brodd123", "lang": "en", "location": "Michigan", "create_at": date("2012-11-06"), "description": "ℙᎯℛᏆ Ꮎℱ ℳℰᏇᏆᎯℕᎶ. UℕᎾℋ ℛᎯℂℰℛѕ 19'. ℐ'Ꮙℰ ᎶᎾᏆ ᎽᎾUℛ ℬᎯℂᏦ ℐℱ ᎽᎾuᏉℰ ᎶᎾᏆ ℳℐℕℰ.", "followers_count": 833, "friends_count": 2139, "statues_count": 4769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Park, MI", "id": "f3e6cc3895998aff", "name": "Oak Park", "place_type": "city", "bounding_box": rectangle("-83.202528,42.444886 -83.153812,42.488703") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2659920, "cityName": "Oak Park" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117033099890688, "text": "Wind 0.7 mph N. Barometer 29.80 in, Falling slowly. Temperature 47.7 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117033338986500, "text": "@ArmandoSalguero gotta give Ross credit for loosening up?", "in_reply_to_status": -1, "in_reply_to_user": 46939857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46939857 }}, "user": { "id": 143215330, "name": "Michael Schwerdt", "screen_name": "23Redphin", "lang": "en", "location": "null", "create_at": date("2010-05-12"), "description": "null", "followers_count": 30, "friends_count": 151, "statues_count": 67 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117033926152192, "text": "All I want to do is go home and read books for fun �� #2moredays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "2moredays" }}, "user": { "id": 2213617134, "name": "Karli Wagatsuma", "screen_name": "kamw808", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "WSU☮ • HI☼ • hula☯", "followers_count": 113, "friends_count": 221, "statues_count": 1616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117033959759873, "text": "Girl you eat garbage quit playin ���� https://t.co/LSXVm9mYhl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 128679672, "name": "Albert Monteon", "screen_name": "AlbertMonteon", "lang": "en", "location": "null", "create_at": date("2010-04-01"), "description": "null", "followers_count": 295, "friends_count": 249, "statues_count": 4276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034026815488, "text": "@ItsUrDestinyy @Fatherbisbo ������", "in_reply_to_status": 728115282598727680, "in_reply_to_user": 519968396, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 519968396, 3164524507 }}, "user": { "id": 456530404, "name": "RaShan", "screen_name": "ThatDudeShan", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-01-06"), "description": "The San Francisco Giants won the World Series on my birthday. 10-29-14 #RIPDad 11-18-15", "followers_count": 1603, "friends_count": 763, "statues_count": 161562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034295250944, "text": "I have a little bit too much energy for 3 in the morning. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18915318, "name": "ᵴamantha ƙnecht ☾", "screen_name": "samknechtfour", "lang": "en", "location": "8️⃣4️⃣5️⃣", "create_at": date("2009-01-12"), "description": "It's not always rainbows and butterflies.", "followers_count": 952, "friends_count": 932, "statues_count": 8831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oneonta, NY", "id": "ff9cc975fb01f747", "name": "Oneonta", "place_type": "city", "bounding_box": rectangle("-75.108773,42.433438 -74.980422,42.478019") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36077, "countyName": "Otsego", "cityID": 3654881, "cityName": "Oneonta" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034488242179, "text": "the count down begins �� https://t.co/bt5vTbFOXI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4352153120, "name": "jeannica holland", "screen_name": "jeannicaholland", "lang": "en", "location": "160214 ", "create_at": date("2015-11-24"), "description": "dvc", "followers_count": 159, "friends_count": 178, "statues_count": 492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034555346944, "text": "@gigacake DOES IT FIT IN A BREAD BOX??", "in_reply_to_status": 728116853621460994, "in_reply_to_user": 2392174926, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2392174926 }}, "user": { "id": 1633914312, "name": "DAMIAN TEETH", "screen_name": "DAMIANTEETH", "lang": "en", "location": "CALIFORNIA", "create_at": date("2013-07-30"), "description": "#1 PUNK BOSS // XVX", "followers_count": 301, "friends_count": 177, "statues_count": 5176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034563690496, "text": "Being supportive and staying up for Aimee to finish her paper", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 516267417, "name": "natwithaY", "screen_name": "nataly_rendon", "lang": "en", "location": "null", "create_at": date("2012-03-05"), "description": "ninety five", "followers_count": 131, "friends_count": 73, "statues_count": 1026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Pablo, CA", "id": "04016a4890553832", "name": "San Pablo", "place_type": "city", "bounding_box": rectangle("-122.358758,37.948637 -122.318412,37.981558") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 668294, "cityName": "San Pablo" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034781810690, "text": "Bitches, I got that shit back today and I got an A. My prof said my shit was well developed and articulated. Hyfr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55765077, "name": "xingona", "screen_name": "chardonnaymami", "lang": "en", "location": "null", "create_at": date("2009-07-10"), "description": "like, ni siquiera puedo", "followers_count": 432, "friends_count": 170, "statues_count": 36509 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Puente, CA", "id": "e04653ebc39b100c", "name": "La Puente", "place_type": "city", "bounding_box": rectangle("-117.98558,34.009223 -117.920899,34.057298") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640340, "cityName": "La Puente" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117034781827073, "text": "@MaddiBragg @lightinthebox they're cheap like $28 & good !", "in_reply_to_status": 728116537056366592, "in_reply_to_user": 2316384601, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 182048795, 25082055 }}, "user": { "id": 2316384601, "name": "MT", "screen_name": "melisuuhhh", "lang": "en", "location": "null", "create_at": date("2014-01-28"), "description": "null", "followers_count": 335, "friends_count": 362, "statues_count": 6726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117035452878849, "text": "Happy birthday ���� @Jay_lew12 turn up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1879233042 }}, "user": { "id": 1718312774, "name": "Mel", "screen_name": "MyyMelodyyy", "lang": "en", "location": "null", "create_at": date("2013-09-01"), "description": "null", "followers_count": 1613, "friends_count": 736, "statues_count": 13716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117036262432768, "text": "My moms car would break down right now at Disneyland ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95008888, "name": "kaylee", "screen_name": "mermaidkayls", "lang": "en", "location": "Disneyland, CA", "create_at": date("2009-12-06"), "description": "damnit maddie", "followers_count": 353, "friends_count": 277, "statues_count": 5660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-05T00:00:03.000Z"), "id": 728117036337881089, "text": "3 hour phone calls... Tired but shoot it was worth it ����☺️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 378234503, "name": "hattie hupke", "screen_name": "hattieeehupkeee", "lang": "en", "location": "IA☀", "create_at": date("2011-09-22"), "description": "/ forget what hurt you; never forget what it taught you / waldorf university softball / #HNH / snapchat & instagram: hattieeehupkeee /", "followers_count": 729, "friends_count": 398, "statues_count": 15268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest City, IA", "id": "46b22d06c42579a4", "name": "Forest City", "place_type": "city", "bounding_box": rectangle("-93.656251,43.238506 -93.61692,43.278829") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19189, "countyName": "Winnebago", "cityID": 1928380, "cityName": "Forest City" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117037122260993, "text": "@Dr_Snekindekin you saying I'm a shame or", "in_reply_to_status": 728116913474146304, "in_reply_to_user": 2522447003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2522447003 }}, "user": { "id": 405262592, "name": "descartes", "screen_name": "dareustheblack", "lang": "en", "location": "Somewhere", "create_at": date("2011-11-04"), "description": "The butterfly and the caterpillar were one and the same...", "followers_count": 290, "friends_count": 226, "statues_count": 7219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117037411639296, "text": "#NF#motivated @ Gold's Gym Bear Creek https://t.co/C5NWFSNujn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.6472778,29.8348904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3148616461, "name": "Ednie Limage", "screen_name": "edny0313", "lang": "en", "location": "Houston, TX", "create_at": date("2015-04-10"), "description": "Professional & HWNT Goalkeeper & Team Captain. PT @ Gold's Gym.\nFuture NPC Competitor.", "followers_count": 21, "friends_count": 22, "statues_count": 116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117037914972160, "text": "So uhm @chancetherapper, its the 5th of May now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181572333 }}, "user": { "id": 66022858, "name": "⁶", "screen_name": "kevman1017", "lang": "en", "location": "S.T.A.R. Labs", "create_at": date("2009-08-15"), "description": "Truth is Power", "followers_count": 720, "friends_count": 572, "statues_count": 44784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117038242140161, "text": "Wind 0.0 mph W. Barometer 29.983 in, Rising slowly. Temperature 63.1 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117038346985473, "text": "Wind 0.0 mph ---. Barometer 29.805 in, Steady. Temperature 43.3 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117038602817538, "text": "Wind 1.0 mph NW. Barometer 29.954 in, Rising slowly. Temperature 50.2 °F. Rain today 0.00 in. Humidity 58%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46583333,41.63111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246464628, "name": "Jerry M", "screen_name": "midcon07", "lang": "en", "location": "Altoona, IA", "create_at": date("2011-02-02"), "description": "Weather Guy", "followers_count": 16, "friends_count": 15, "statues_count": 2882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altoona, IA", "id": "008d14b32ddf2fb6", "name": "Altoona", "place_type": "city", "bounding_box": rectangle("-93.519371,41.618766 -93.443318,41.671805") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1901630, "cityName": "Altoona" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117038636404736, "text": "god i fuckin love The Mist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 181342868, "name": "Gene Jacquet", "screen_name": "literalslime", "lang": "en", "location": "heck", "create_at": date("2010-08-21"), "description": "snailed it", "followers_count": 118, "friends_count": 308, "statues_count": 4130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edwardsburg, MI", "id": "0084e72c39f8fa1b", "name": "Edwardsburg", "place_type": "city", "bounding_box": rectangle("-86.098243,41.760518 -86.009845,41.810864") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26027, "countyName": "Cass", "cityID": 2625040, "cityName": "Edwardsburg" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117039215169537, "text": "Baby you a little mind reader", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 411177707, "name": "Natasha Rodriguez", "screen_name": "Tashaa_bre9", "lang": "en", "location": "null", "create_at": date("2011-11-12"), "description": "null", "followers_count": 1081, "friends_count": 344, "statues_count": 24753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117039986921473, "text": "@youknowyouredesi when these look like Gulaab Jamuns at a quick glance https://t.co/R5OOzChHzw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 339454739, "name": "Jasleen Kaur", "screen_name": "jasleenthequeen", "lang": "en", "location": "USA", "create_at": date("2011-07-20"), "description": "Just a princess and her tiara Snapchat: jk01 Instagram: @desi2lazy", "followers_count": 669, "friends_count": 1844, "statues_count": 2367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilkes-Barre, PA", "id": "3fdd3a62ae058de9", "name": "Wilkes-Barre", "place_type": "city", "bounding_box": rectangle("-75.924804,41.215416 -75.826756,41.272993") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4285152, "cityName": "Wilkes-Barre" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117040691580931, "text": "Love you too @mschroeder01 https://t.co/WpB9Y5OtIy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 409491470 }}, "user": { "id": 316856307, "name": "Ellen Brunner", "screen_name": "ebrunner_25", "lang": "en", "location": "null", "create_at": date("2011-06-13"), "description": "Home is where the heart is ♡ 815", "followers_count": 522, "friends_count": 404, "statues_count": 7540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117040939073536, "text": "Baddies... # @ ONO Nightclub https://t.co/Z0RFkSXWhN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.37868,28.54189"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53326070, "name": "Garuda Group", "screen_name": "GarudaGroupORL", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-07-03"), "description": "Premier events at only the top nightclubs and bars in Downtown Orlando. Our brand of party sets a new standard.. a higher one. #GarudaNightlife #NightlifeBetter", "followers_count": 454, "friends_count": 111, "statues_count": 332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-05-05T00:00:04.000Z"), "id": 728117041006149632, "text": "I just woke up and I think it finally hit me that I'm about to be a legit adult.. lol next joke I'm like 4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1610470784, "name": "vick", "screen_name": "lilvick0", "lang": "en", "location": "null", "create_at": date("2013-07-21"), "description": "yes I go to ND, yes I play football, yes we do love to party, no I am not on steriods, and yes I am a STATE CHAMP sc:lilvick0", "followers_count": 517, "friends_count": 417, "statues_count": 2693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117041861812224, "text": "@DemericanDream You really a dickrider its so sad. How you slime and a dickrider? I'm confused", "in_reply_to_status": 728116514583240704, "in_reply_to_user": 896549017, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 896549017 }}, "user": { "id": 87554095, "name": "Griselda Blanco", "screen_name": "GiovanniT_", "lang": "en", "location": "null", "create_at": date("2009-11-04"), "description": "And if it ain't about nothin', then she abort mission", "followers_count": 718, "friends_count": 30, "statues_count": 13257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brighton, NY", "id": "94779fa8bb1f79a8", "name": "Brighton", "place_type": "city", "bounding_box": rectangle("-77.694219,43.087472 -77.516194,43.167065") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3608257, "cityName": "Brighton" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117041983410176, "text": "Fun fact: If I could only drink one thing for the rest of my life it would be applied juice.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237417108, "name": "Chilo Dore", "screen_name": "cDore18", "lang": "en", "location": "null", "create_at": date("2015-06-05"), "description": "My Highlights: http://hudl.com/athlete/401512…", "followers_count": 64, "friends_count": 198, "statues_count": 199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117041996038144, "text": "@HartwellRayanne @_sariinnaa https://t.co/AaV2eSwJVo", "in_reply_to_status": -1, "in_reply_to_user": 3045332095, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3045332095, 2727663285 }}, "user": { "id": 2994947262, "name": "sammi♊️", "screen_name": "_saammii_", "lang": "en", "location": "NorCal", "create_at": date("2015-01-24"), "description": "We must take adventures in order to know where we truly belong.", "followers_count": 407, "friends_count": 367, "statues_count": 15276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Esparto, CA", "id": "00e3f74d48d6dea3", "name": "Esparto", "place_type": "city", "bounding_box": rectangle("-122.037257,38.684299 -121.983183,38.701284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 622846, "cityName": "Esparto" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117042260287488, "text": "What I find mind boggling are the sheer number of celebrities I've never heard of.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322183892, "name": "I Threw 89", "screen_name": "DartTalk", "lang": "en", "location": "Chicago, IL", "create_at": date("2011-06-22"), "description": "50 yr #Cubs fan, urbanite, raconteur, roustabout, gadfly, Zonker-ish Dude and @Cubs season ticket holder. I write for https://t.co/q4AOvnrKTw and I podcast too", "followers_count": 1309, "friends_count": 2904, "statues_count": 4897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117042331582466, "text": "I was looking at u and u wassss looking at me and we was thinkin..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4251080693, "name": "Chyfox", "screen_name": "thechyfoxshow", "lang": "en", "location": "Virginia, USA", "create_at": date("2015-11-22"), "description": "Youtube: TheChyfoxShow Email: contact@chyfox.com #blacklivesmatter RU18 Virgo", "followers_count": 183, "friends_count": 172, "statues_count": 1136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Radford, VA", "id": "a265d5d344dbd61d", "name": "Radford", "place_type": "city", "bounding_box": rectangle("-80.595589,37.090022 -80.513486,37.146017") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51750, "countyName": "Radford", "cityID": 5165392, "cityName": "Radford" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117042495115266, "text": "Ruby Room Wants To Give You A Special Gift For Your Birthday...https://t.co/wN7Wv4ODyh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6716,41.9032"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27957411, "name": "Ruby Room", "screen_name": "rubyroomchicago", "lang": "en", "location": "Chicago, Illinois", "create_at": date("2009-03-31"), "description": "to leave a beauty mark on the face of the wellness industry through our commitment to healing, energy and all things positive.", "followers_count": 1129, "friends_count": 1713, "statues_count": 1973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117042730012672, "text": "Wind 0 mph ---. Barometer 1012.4 hPa, Falling. Temperature 77.5 °F. Rain today 0.00 in. Humidity 32%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 155352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117043833143296, "text": "@mainedcm @aldenrichards02 Uyyyy! PAREHO silang kinikilig @NoemiManuel4 #ALDUB42ndWeeksary https://t.co/XdpQsI0MH4", "in_reply_to_status": -1, "in_reply_to_user": 63701775, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUB42ndWeeksary" }}, "user_mentions": {{ 63701775, 98310564, 3463479312 }}, "user": { "id": 3466444632, "name": "YolieQ.", "screen_name": "YQuietson", "lang": "en", "location": "null", "create_at": date("2015-09-05"), "description": "null", "followers_count": 80, "friends_count": 84, "statues_count": 17916 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117044873289728, "text": "This #Manufacturing #job might be a great fit for you: Production Supervisor - https://t.co/baMs22wdP1 #Breinigsville, PA #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.631484,40.5393961"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Manufacturing", "job", "Breinigsville", "Hiring" }}, "user": { "id": 562337812, "name": "NWNA Careers", "screen_name": "NWNACareers", "lang": "en", "location": "null", "create_at": date("2012-04-24"), "description": "Follow us for information on rewarding career opportunities at Nestlé Waters North America, the nation’s largest bottled water company. #workinyourelement", "followers_count": 281, "friends_count": 6, "statues_count": 2622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Breinigsville, PA", "id": "0166fb811a8b28c6", "name": "Breinigsville", "place_type": "city", "bounding_box": rectangle("-75.644341,40.519799 -75.613091,40.565181") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42077, "countyName": "Lehigh", "cityID": 4208392, "cityName": "Breinigsville" } }
+{ "create_at": datetime("2016-05-05T00:00:05.000Z"), "id": 728117045175320576, "text": "Drake | With You #Views", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Views" }}, "user": { "id": 374000190, "name": "Sweet Honey Ice Tea'", "screen_name": "_BeautyBlu", "lang": "en", "location": "null", "create_at": date("2011-09-15"), "description": "Professional #Model ✨ IG/SC: Royce2Fierce #LèBeaúty✨", "followers_count": 1512, "friends_count": 996, "statues_count": 52349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117045393424386, "text": "@cristinaodalis_ your better than any boy", "in_reply_to_status": 728116561899192320, "in_reply_to_user": 485171409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 485171409 }}, "user": { "id": 2895960624, "name": "Itzamary", "screen_name": "itzacutie", "lang": "en", "location": "null", "create_at": date("2014-11-10"), "description": "18", "followers_count": 628, "friends_count": 294, "statues_count": 18951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117045594750976, "text": "Jerk..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3039599353, "name": "nat.♥️", "screen_name": "nattaliemariee", "lang": "en", "location": "San Antonio", "create_at": date("2015-02-24"), "description": "IG:nattalieemariee", "followers_count": 432, "friends_count": 385, "statues_count": 7468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117046869786624, "text": "Everyone in the annex should stop talking", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 360224338, "name": "Morgan Skinner", "screen_name": "MO_SKINN", "lang": "en", "location": "HTX", "create_at": date("2011-08-22"), "description": "null", "followers_count": 989, "friends_count": 567, "statues_count": 15489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117046941097984, "text": "Alright I'm done I finally lost the exact amount of followers I was going for ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146553414, "name": "patrick.", "screen_name": "studio_dad", "lang": "en", "location": "San Francisco", "create_at": date("2015-04-09"), "description": "I am not a role model: Mixer / Producer / Current Owner of @differentfur / cheese filled food enthusiast....", "followers_count": 985, "friends_count": 1095, "statues_count": 6918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117047020806144, "text": "These classes gone be something serious. https://t.co/qI45Dv09fZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287869929, "name": "✨", "screen_name": "KristiaDBeauty", "lang": "en", "location": "Texas Woman's University ✨❤️", "create_at": date("2011-04-25"), "description": "Growing woman of God † Blogger. Afro-Latina-Creole. BSDH", "followers_count": 1585, "friends_count": 754, "statues_count": 79311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117047377334272, "text": "i think it's funny when ppl get annoyed and/or pissed off bc of something on Twitter. #itsneverthatserious ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "itsneverthatserious" }}, "user": { "id": 230851723, "name": "Timmy", "screen_name": "timothyburgos", "lang": "en", "location": "null", "create_at": date("2010-12-26"), "description": "you're corny", "followers_count": 502, "friends_count": 569, "statues_count": 22476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117047624761344, "text": "https://t.co/xNy6izYCU1", "in_reply_to_status": 728116794066534400, "in_reply_to_user": 375901863, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 375901863, "name": "¯\\_(ツ)_/¯", "screen_name": "cookidchey", "lang": "en", "location": "null", "create_at": date("2011-09-18"), "description": "the juice & da squeeze.", "followers_count": 2579, "friends_count": 625, "statues_count": 60315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117047758970883, "text": "The crazy part was that you thought so low of me when I was the one always there for you. Smh .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2918090736, "name": "Susana Villalobos", "screen_name": "Jiggaaboo_", "lang": "en", "location": "null", "create_at": date("2014-12-03"), "description": "Your opinion of me doesn't define who I am. Fuck outta here ✌️", "followers_count": 494, "friends_count": 644, "statues_count": 1647 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117048425881601, "text": "@Bangerr___ don't you watch this https://t.co/5u8mtGggcX", "in_reply_to_status": -1, "in_reply_to_user": 1445072336, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1445072336 }}, "user": { "id": 294631019, "name": "Oluwatobiloba", "screen_name": "___tlk", "lang": "en", "location": "Breaking Glass Ceilings", "create_at": date("2011-05-07"), "description": "thriving, popolicious youngster", "followers_count": 1150, "friends_count": 857, "statues_count": 29553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117048971157504, "text": "@thatgirlJOHNNI they don't hear me. Like we ain't like it for a reason", "in_reply_to_status": 728116857383755776, "in_reply_to_user": 362589536, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 362589536 }}, "user": { "id": 1500412051, "name": "valerie", "screen_name": "iamvx3_", "lang": "en", "location": "null", "create_at": date("2013-06-10"), "description": "Gods got this", "followers_count": 784, "friends_count": 435, "statues_count": 31139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117049302507520, "text": "https://t.co/Mua2CVCDp8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4192732934, "name": "ㅤ", "screen_name": "marcotics__", "lang": "en", "location": "818", "create_at": date("2015-11-10"), "description": "1/2DEAD. She think I'm the shit, he mad I'm the shit.", "followers_count": 259, "friends_count": 192, "statues_count": 3488 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-05T00:00:06.000Z"), "id": 728117049331851264, "text": "A ten.. A fuckin ten. #purenoiserecords #purenoisetour #rebellounge #hitthelights @ The Rebel… https://t.co/mkDECAqM3J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.03211,33.49453"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "purenoiserecords", "purenoisetour", "rebellounge", "hitthelights" }}, "user": { "id": 1396579418, "name": "Jessica", "screen_name": "JessicaxHavens", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2013-05-02"), "description": "you only live once, but if you do it right, once is enough - Mae West", "followers_count": 733, "friends_count": 1190, "statues_count": 20571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-05T00:00:07.000Z"), "id": 728117049520590849, "text": "5/5/2016 - 02:00\nTemp: 49.9F \nHum: 90%\nWind: 0.0 mph\nBaro: 29.948in. & Rising\nRain: 0.00 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 52443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-05T00:00:07.000Z"), "id": 728117050170740737, "text": "2am. Still studying for micro. My brain can no longer absorb any more info. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 168556957, "name": "E", "screen_name": "edenwhorton", "lang": "en", "location": "NSU-Tahlequah", "create_at": date("2010-07-19"), "description": "Marketing. 21. Waitress @ Iguana.", "followers_count": 808, "friends_count": 737, "statues_count": 15540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tahlequah, OK", "id": "c9bcc362968ea245", "name": "Tahlequah", "place_type": "city", "bounding_box": rectangle("-95.012128,35.856248 -94.932594,35.976425") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40021, "countyName": "Cherokee", "cityID": 4072100, "cityName": "Tahlequah" } }
+{ "create_at": datetime("2016-05-05T00:00:07.000Z"), "id": 728117050183294977, "text": "Time go to work���� https://t.co/PLW5Txp99c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.058626,34.22919"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2718124320, "name": "Jesse Cervantes", "screen_name": "jessse30", "lang": "en", "location": "null", "create_at": date("2014-08-08"), "description": "null", "followers_count": 226, "friends_count": 186, "statues_count": 6658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "United States Marine Corp Air Ground Combat Center", "id": "07d9c97c67486001", "name": "United States Marine Corp Air Ground Combat Center", "place_type": "poi", "bounding_box": rectangle("-116.0586261,34.2291899 -116.058626,34.22919") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 680994, "cityName": "Twentynine Palms" } }
+{ "create_at": datetime("2016-05-05T00:00:07.000Z"), "id": 728117050690805760, "text": "Thank you everyone for the happy birthday messages love y'all!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439588546, "name": "pey", "screen_name": "quintanapeyton", "lang": "en", "location": "sleeping ", "create_at": date("2014-04-11"), "description": "null", "followers_count": 621, "friends_count": 571, "statues_count": 8957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479407954763777, "text": "YG is trash", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 2421113472, "name": "1️⃣", "screen_name": "npickett1", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-03-31"), "description": "R.I.P. GRANNY", "followers_count": 2160, "friends_count": 545, "statues_count": 17827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479408269430784, "text": "������������ https://t.co/ERLnbH9GOg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 794961588, "name": ".", "screen_name": "WhoreHey_10", "lang": "en", "location": "Chicago Heights, IL", "create_at": date("2012-08-31"), "description": "null", "followers_count": 805, "friends_count": 485, "statues_count": 15955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago Heights, IL", "id": "55f894104157687c", "name": "Chicago Heights", "place_type": "city", "bounding_box": rectangle("-87.674421,41.481426 -87.596844,41.543213") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714026, "cityName": "Chicago Heights" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479408718123008, "text": "Still https://t.co/5IC4tKmfHT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394279975, "name": "PforShort", "screen_name": "Only4yubby", "lang": "en", "location": "Just a city girl ", "create_at": date("2011-10-19"), "description": "To my brother I Love You #teambey", "followers_count": 381, "friends_count": 347, "statues_count": 19690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479409020260353, "text": "International flags. International peoples. And lots of booze. @… https://t.co/FCQl5e2swA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.337942,47.606289"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 11968752, "name": "joaquin uy", "screen_name": "joaquinuy", "lang": "en", "location": "Seattle, WA", "create_at": date("2008-01-07"), "description": "Seattle-based communications professional & believer of the impossible. Views expressed are my own & do not reflect @IandRAffairs.", "followers_count": 1928, "friends_count": 1680, "statues_count": 15646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479409171140608, "text": "Holy shit. https://t.co/oYxVNpjaHY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2421351992, "name": "Robin", "screen_name": "Roobbiinnnn", "lang": "en", "location": "Houston, TX", "create_at": date("2014-03-31"), "description": "we are all in the gutter, but some of us are looking at the stars.", "followers_count": 53, "friends_count": 201, "statues_count": 872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479409871593473, "text": "‼️ https://t.co/VPXnLEmSTA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4498644986, "name": "KfcknNasty✨", "screen_name": "kashmafia00", "lang": "en", "location": "Houston, TX", "create_at": date("2015-12-08"), "description": "Beauty in the struggle.", "followers_count": 200, "friends_count": 188, "statues_count": 4730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479409972203520, "text": "@___iketani unless we runnin 2s", "in_reply_to_status": 728478927035850752, "in_reply_to_user": 100592882, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 100592882 }}, "user": { "id": 725026664, "name": "G.T.W.Y.C", "screen_name": "dhenley96", "lang": "en", "location": "null", "create_at": date("2012-07-29"), "description": "BANDANA B. BRICK SQUAD B. GET THERE B.", "followers_count": 147, "friends_count": 98, "statues_count": 26903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, CA", "id": "b2fda7385aa963dd", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-118.317776,33.930952 -118.291549,33.959516") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684592, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479411591225344, "text": "Lmao tonight was unexpected, and that's alright by me ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1905936350, "name": "Nate", "screen_name": "NateClax", "lang": "en", "location": "Nawf Tejas ", "create_at": date("2013-09-25"), "description": "http://So.Cal Native, I dig mountains, UNT Triathlon Team, ΠΚΦ, Bio major/ Pre-Med", "followers_count": 672, "friends_count": 243, "statues_count": 9994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479411784290304, "text": "Not only that my job was involved like tf come on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2741256371, "name": "AyoHoopsz", "screen_name": "princess_renaee", "lang": "en", "location": " ", "create_at": date("2014-08-12"), "description": "23[yung] ieatcake[♐29th]\nProudParnetof(3)❤\n(RIP)PrincessQuay!", "followers_count": 470, "friends_count": 470, "statues_count": 7686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, NC", "id": "f6239157c15e0d1f", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-80.648297,34.944713 -80.477093,35.061654") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37179, "countyName": "Union", "cityID": 3743920, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479411809456128, "text": "Attack, I'm COACH ��but ion play nun! 6️⃣th man too Boii ion play nun ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223635902, "name": "No Hesitation!", "screen_name": "BLOCK_Genesis", "lang": "en", "location": "somewhere yellin #FreeDev", "create_at": date("2010-12-06"), "description": "May our goals be BLESSED by the lord May our sins be forever forgiven. Let us stay humble in our times of distress. Keep us wise and give us understanding!", "followers_count": 635, "friends_count": 456, "statues_count": 13940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevens Point, WI", "id": "c0b44c42d36404e3", "name": "Stevens Point", "place_type": "city", "bounding_box": rectangle("-89.60726,44.493975 -89.491635,44.586049") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55097, "countyName": "Portage", "cityID": 5577200, "cityName": "Stevens Point" } }
+{ "create_at": datetime("2016-05-06T00:00:00.000Z"), "id": 728479412115623936, "text": "@localemogirl I'm in love w u!!!!!!!", "in_reply_to_status": 728479142363070464, "in_reply_to_user": 3069051242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3069051242 }}, "user": { "id": 2609911663, "name": "Brittany M. Ames", "screen_name": "brittypiexox", "lang": "en", "location": "508", "create_at": date("2014-07-07"), "description": "if you can't love yourself, how in the hell are you gonna love somebody else? • cuz I'm a child & soft", "followers_count": 88, "friends_count": 92, "statues_count": 2721 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, MA", "id": "90eed94925e42147", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-70.950694,42.48393 -70.864351,42.546174") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25009, "countyName": "Essex", "cityID": 2559105, "cityName": "Salem" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479412354732032, "text": "Temp 45.0° Hi/Lo 45.0/44.5 Rng 0.5° WC 45.0° Hmd 94% Rain 0.01\" Storm 0.47\" BAR 29.702 Falling DP 43.4° Wnd 0mph Dir --- Gst 14mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 121, "statues_count": 18469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479412358815744, "text": "Im sooooo littttttg omfggggg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2870096177, "name": "↑", "screen_name": "Cocohantess", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-11-09"), "description": "ERiTREAN", "followers_count": 314, "friends_count": 182, "statues_count": 5444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479412375654400, "text": "Wind 0.0 mph ENE. Barometer 29.970 in, Steady. Temperature 48.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479412866318337, "text": "Save the last dance is on Netflix �� that's classic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 150796492, "name": "#BeGreatWes", "screen_name": "Wes_Too_Smooth", "lang": "en", "location": "null", "create_at": date("2010-06-01"), "description": "As Real As They Get As Humble As They Come #GaState Psalms 34:4 #Team2Fye", "followers_count": 3617, "friends_count": 2812, "statues_count": 88709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anderson, SC", "id": "42d68bc4d1591d12", "name": "Anderson", "place_type": "city", "bounding_box": rectangle("-82.707012,34.447616 -82.548858,34.610762") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45007, "countyName": "Anderson", "cityID": 4501360, "cityName": "Anderson" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479413071810562, "text": "@ebbtideapp Tide in Waikane, Hawaii 05/06/2016\nHigh 1:15am 1.9\n Low 7:59am -0.4\nHigh 2:47pm 2.8\n Low 9:19pm 0.7\nHigh 1:50am 1.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-157.85,21.5"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 84, "friends_count": 1, "statues_count": 30404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaii, USA", "id": "9dafd05b1158873b", "name": "Hawaii", "place_type": "admin", "bounding_box": rectangle("-178.443593,18.86546 -154.755792,28.517269") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1575800, "cityName": "Waikane" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479413562564608, "text": "@ChrisCutless naw no backwoods welcomes lol", "in_reply_to_status": 728479293324562432, "in_reply_to_user": 268842989, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 268842989 }}, "user": { "id": 2362107105, "name": "LOVER", "screen_name": "InfamousEzzyGo", "lang": "en", "location": "Ft Pierce ✈️ Texas", "create_at": date("2014-02-23"), "description": "SAMO STONA ✨ G-Force shine brighter than diamonds", "followers_count": 401, "friends_count": 86, "statues_count": 8545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harker Heights, TX", "id": "006178844582e636", "name": "Harker Heights", "place_type": "city", "bounding_box": rectangle("-97.68605,31.006089 -97.623539,31.10068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4832312, "cityName": "Harker Heights" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479413785001984, "text": "05/06@03:00 - Temp 53.5F, WC 53.5F. Wind 0.8mph NNW, Gust 3.0mph. Bar 29.668in, Falling slowly. Rain 0.00in. Hum 86%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479413910835200, "text": "@AbdielValencia_ ��������", "in_reply_to_status": 728479364988346368, "in_reply_to_user": 625092382, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 625092382 }}, "user": { "id": 380150037, "name": "YE$$E", "screen_name": "Yeeezzuuss", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2011-09-25"), "description": "General roofing contractor ♋️ southpaw pro drinker #YankeeNation #cowboynation", "followers_count": 594, "friends_count": 467, "statues_count": 12962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414086852608, "text": "Preach it boo https://t.co/AUD7nKRxKb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1968051961, "name": "•Leah•", "screen_name": "arauza_leah03", "lang": "en", "location": "Rocklin, CA", "create_at": date("2013-10-17"), "description": "I'd rather be hated for who I am, than loved for who I am not. ↢Sierra College↢", "followers_count": 229, "friends_count": 473, "statues_count": 5829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414279757824, "text": "Did we even have a winter in Texas this year or what, I'm pissed. I only enjoy winter apparel.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1187933750, "name": "cassidy jewell", "screen_name": "AssidyJew", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "| Artemis is my fur baby | dfw | @stoopbrozay |", "followers_count": 617, "friends_count": 367, "statues_count": 27468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414409822210, "text": "Temp: 48.9F W C: 48.9F Wind:SE at 2.2kts Baro: 1017.1mb and Rising slowly Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 210, "friends_count": 218, "statues_count": 104286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414447685632, "text": "Ripley SW Limestone Co. Temp: 45.7°F Wind:0.0mph Pressure: 994.8mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 52978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414560821248, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":47.7°F Wind:0.0mph Pressure: 29.91hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 16, "statues_count": 317141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414779023360, "text": "Temp: 53.0°F Wind:0.0mph Pressure: 29.989hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479414829256704, "text": "emily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1685526697, "name": "カンエ", "screen_name": "cvlxste", "lang": "en", "location": "null", "create_at": date("2013-08-20"), "description": "the money is the motive ♕", "followers_count": 717, "friends_count": 431, "statues_count": 2848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479415303163906, "text": "@Mr_Pie @StevieEvilCat @LucyBusterBooks @luvpug25 @thedog_marlie @LBisaillon @FionaKenndal MOL! Dus u wants it in protein?Or flour & sugar?", "in_reply_to_status": 728307362017247232, "in_reply_to_user": 219273242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219273242, 2226701918, 1666723464, 3110974067, 4657141666, 496905664, 2376693973 }}, "user": { "id": 1020050376, "name": "Diego #RipPookie", "screen_name": "BarbaraBpiwetz", "lang": "en", "location": "US", "create_at": date("2012-12-18"), "description": "Territories include da magical forest, time&space and da mind palace, which some has mislabled as fantasy.Avi by @scullynoreen", "followers_count": 4899, "friends_count": 1101, "statues_count": 62312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479415441575936, "text": "@AmenOVO bro where & when does this happen I wanna hear this shit", "in_reply_to_status": 728479314795110400, "in_reply_to_user": 2951663083, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2951663083 }}, "user": { "id": 717079201, "name": "Brandon", "screen_name": "BrezzzyEzzzy41", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "Living Life To The Fullest. |SneakerHead. #RIPMalery #RIPBiggles #StayStrong7", "followers_count": 459, "friends_count": 376, "statues_count": 29898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479415529832448, "text": "Wind 0.0 mph NW. Barometer 29.944 in, Rising slowly. Temperature 47.7 °F. Rain today 0.01 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 60, "friends_count": 27, "statues_count": 19617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479415634550785, "text": "MMM mother fucking god! https://t.co/6VVOQVjUrp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25244854, "name": ".", "screen_name": "MusicalKidrauhl", "lang": "en", "location": "null", "create_at": date("2009-03-18"), "description": "I WILL FIND YOU AND BE KNOCKING ON YO DOOR AND SQUARING TF UP. JUSTIN DOESNT DESERVE THIS CRAP....", "followers_count": 6664, "friends_count": 441, "statues_count": 303187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-06T00:00:01.000Z"), "id": 728479416028798976, "text": "Please help us find the missing taco he was last seen near Taco Bell ewa beach https://t.co/kCDlkHg74a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560629679, "name": "DonaldnotTrump", "screen_name": "djbarker8", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2014-06-10"), "description": "Donald Barker", "followers_count": 237, "friends_count": 288, "statues_count": 6365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479416444178432, "text": "@BoutDatLifeBoi its cool lol", "in_reply_to_status": 728479282394206208, "in_reply_to_user": 150791506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150791506 }}, "user": { "id": 2806606791, "name": "Nicky", "screen_name": "Nicky216_", "lang": "en", "location": "null", "create_at": date("2014-10-04"), "description": "you gotta reposition yourself so you can receive all of your blessings.", "followers_count": 1741, "friends_count": 1265, "statues_count": 27274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479416817319936, "text": "BT THAT DOESNT STOP ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412387493, "name": "Amanatee", "screen_name": "ManndiPandi", "lang": "en", "location": "Camarillo, CA", "create_at": date("2011-11-14"), "description": "null", "followers_count": 259, "friends_count": 176, "statues_count": 22566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479416913891328, "text": "Hemlock Grove ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 280047787, "name": "Andre", "screen_name": "Andrekilgore", "lang": "en", "location": "Tennessee, USA", "create_at": date("2011-04-10"), "description": "null", "followers_count": 680, "friends_count": 229, "statues_count": 14632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsport, TN", "id": "00981cdd189cd103", "name": "Kingsport", "place_type": "city", "bounding_box": rectangle("-82.649893,36.43008 -82.39527,36.594906") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47163, "countyName": "Sullivan", "cityID": 4739560, "cityName": "Kingsport" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479417018798080, "text": "The sound of rain when you're tryna fall asleep is heavenly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184308561, "name": "Antoinette", "screen_name": "AnnieDazzo", "lang": "en", "location": "null", "create_at": date("2013-02-15"), "description": "18.♑ | NY | SJU/PharmD", "followers_count": 736, "friends_count": 215, "statues_count": 21699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479417153011713, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Rising slowly. Temperature 54.9 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479417459220482, "text": "Wind 0.0 mph ---. Barometer 30.033 in, Steady. Temperature 46.9 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479417744252928, "text": "60.9F (Feels: 60.9F) - Humidity: 71% - Wind: 1.6mph E - Gust: 2.2mph - Pressure: 1014.2mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479418398580736, "text": "Need now.. https://t.co/jFKg1zpctS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760388658, "name": "Cieeerrra", "screen_name": "cicithompson1", "lang": "en", "location": "Malcom, IA", "create_at": date("2012-08-15"), "description": "My avi shows my love of the mullet lifestyle.✌️/ @bkrieg5 is my fav adult. Mainly because of the mullet and snap filter usage. #work \\ Snap: cierra.thompson", "followers_count": 416, "friends_count": 647, "statues_count": 18578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479418432159744, "text": "@k_goddd ����", "in_reply_to_status": 728428586479144960, "in_reply_to_user": 2718532554, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2718532554 }}, "user": { "id": 326991466, "name": "Jared BK Hamilton", "screen_name": "Jaredbkhamilton", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-06-30"), "description": "Manager:@TribalTheory Booking:TribalTheoryMGMT@gmail.com CEO:@BambooSocks @LILO_LiveLove #LILO USD Alumni. Born in Laie Oahu Hawaii. LDS. #HFF", "followers_count": 1264, "friends_count": 668, "statues_count": 19829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479418520199168, "text": "Its amazing how just a bit of good news can take away massive amounts of stress.��#blessed #thanksjesus", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed", "thanksjesus" }}, "user": { "id": 1962092449, "name": "Mariah Monica Bravo", "screen_name": "mariahmonicab", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-10-14"), "description": "Your first draft was good, but do better.✍ \nCal Poly Alum '15", "followers_count": 44, "friends_count": 86, "statues_count": 92 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479418641846272, "text": "@TYPSYxGYPSY fuck your ex.", "in_reply_to_status": 728468260304211968, "in_reply_to_user": 460364940, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 460364940 }}, "user": { "id": 2728495793, "name": "JayMac", "screen_name": "joshuaaamranda", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-07-29"), "description": "I use twitter to archive my genius thoughts and hilarious jokes.", "followers_count": 117, "friends_count": 137, "statues_count": 1085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479419166138368, "text": "Hello 22!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38187356, "name": "Angela", "screen_name": "Deadly_Sinned", "lang": "en", "location": "Santa Clara,CA", "create_at": date("2009-05-06"), "description": "|02.15.2016❤️|Child Development Major| After School Leader| You want the moon? Just say the word and I'll throw a lasso around it and pull it down|", "followers_count": 161, "friends_count": 423, "statues_count": 4594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479419426201600, "text": "@KLnoona and a couple of great character actors added in don't hurt either", "in_reply_to_status": 728478994060873728, "in_reply_to_user": 355705276, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 942367014 }}, "user": { "id": 355705276, "name": "Meg'sOmma", "screen_name": "caahjumma", "lang": "en", "location": "Monterey, CA", "create_at": date("2011-08-15"), "description": "Mormon mom who loves her family & kdramas. I have a lawyer son w/a lovely wife & kid (and a half), and a beautiful 18 yo anime fan daughter.", "followers_count": 163, "friends_count": 134, "statues_count": 44474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elkhorn, CA", "id": "0e67ccb5685d3c6e", "name": "Elkhorn", "place_type": "city", "bounding_box": rectangle("-121.730782,36.794428 -121.675697,36.83089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 622034, "cityName": "Elkhorn" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479419493388288, "text": "My laptop running slow, I'm going to sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2165864844, "name": "Quay Baby ™", "screen_name": "baddiequay", "lang": "en", "location": "Kent State University '18 ⚡️", "create_at": date("2013-10-30"), "description": "20-year-old sex novelist. Creator and writer of Cheeseburgers, Chocolate, and Sex.", "followers_count": 1380, "friends_count": 986, "statues_count": 56943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, OH", "id": "45a0ea3329c38f9f", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-81.392451,41.11766 -81.308418,41.202841") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39133, "countyName": "Portage", "cityID": 3939872, "cityName": "Kent" } }
+{ "create_at": datetime("2016-05-06T00:00:02.000Z"), "id": 728479420307005440, "text": "I wish that I was what you wanted.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3059076133, "name": "Faith Alexandria", "screen_name": "faith_v21", "lang": "en", "location": "null", "create_at": date("2015-03-02"), "description": "Bellflower High School Varsity Softball #21⚾️ snapchat;faithvelasquez2", "followers_count": 182, "friends_count": 264, "statues_count": 5963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479420760084480, "text": "Wind 0.7 mph NNE. Barometer 29.74 in, Steady. Temperature 54.3 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 120, "statues_count": 159583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479420911128576, "text": "TOO MANY NIGGAS STEAING IDEAS AND OUTFITS I HATE Y'ALL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88778740, "name": "WAZ", "screen_name": "WazDOPE", "lang": "en", "location": "null", "create_at": date("2009-11-09"), "description": "All American LowLife", "followers_count": 1422, "friends_count": 446, "statues_count": 34057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479421586235392, "text": "This app led to the greatest car sex in history lmao https://t.co/nGq5c9ILm6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309922057, "name": "funny fat guy", "screen_name": "OMG_ItsKhairy", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-06-02"), "description": "Cartoon Connoisseur #FatBoyNation #FatBoyPresident email:M.Cr33k@yahoo.com", "followers_count": 6650, "friends_count": 3421, "statues_count": 78048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479421858897920, "text": "https://t.co/KIuk0qOVQz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 224387651, "name": "Yuck Fou", "screen_name": "Shreddy_69", "lang": "en", "location": "Fuck You ", "create_at": date("2010-12-08"), "description": "Tristen, 20, Just know I hate you mother fuckers and that's from the heart", "followers_count": 278, "friends_count": 339, "statues_count": 7358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentone, CA", "id": "6a4d9ceebcc15b7d", "name": "Mentone", "place_type": "city", "bounding_box": rectangle("-117.139033,34.041098 -117.075255,34.082948") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 646884, "cityName": "Mentone" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422114725889, "text": "https://t.co/7MKbVt5tjF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 920282186, "name": "Julio, from Houston", "screen_name": "squarian_kid", "lang": "en", "location": "HTX", "create_at": date("2012-11-01"), "description": "love yours", "followers_count": 596, "friends_count": 748, "statues_count": 33016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422118961152, "text": "3071c5c900p306E7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-101.784098,40.002816"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 21, "friends_count": 0, "statues_count": 19095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nebraska, USA", "id": "ac9b9070f6d17a9a", "name": "Nebraska", "place_type": "admin", "bounding_box": rectangle("-104.053515,39.999751 -95.30829,43.001708") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20023, "countyName": "Cheyenne" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422290894849, "text": "I always crave junk food at night. Like why? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1014396127, "name": "Karen Giselle", "screen_name": "HernandezKarenG", "lang": "en", "location": "null", "create_at": date("2012-12-15"), "description": "Though my heart and flesh may fail, there's an anchor for my soul. ⚓️", "followers_count": 176, "friends_count": 56, "statues_count": 8099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422341369856, "text": "Temp: 59.2°F | Humidity: 98% | Wind: --- @ 0.0 mph | Barometer: 29.86 in | Dewpoint: 58.6°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 167075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422462885888, "text": "Why am I crying over Michael leaving The Office", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2328050450, "name": "kat❁", "screen_name": "katie_clauer", "lang": "en", "location": "Folsom, CA", "create_at": date("2014-02-04"), "description": "princess", "followers_count": 226, "friends_count": 400, "statues_count": 3171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Folsom, CA", "id": "8eccead85cc1b6c0", "name": "Folsom", "place_type": "city", "bounding_box": rectangle("-121.260264,38.621443 -121.085432,38.714331") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 624638, "cityName": "Folsom" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479422618079232, "text": "#np Spirits // The Strumbellas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "np" }}, "user": { "id": 1012412172, "name": "sf", "screen_name": "sharmanfaiz", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-12-14"), "description": "Mt. Rainier", "followers_count": 555, "friends_count": 423, "statues_count": 31545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479423419305984, "text": "Hilton Doubletree Resort Yunnan by OAD https://t.co/xcWxW7Xbkq #InteriorDesign #Architecture https://t.co/BfSxFKN3Ph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.980904,40.758743"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "InteriorDesign", "Architecture" }}, "user": { "id": 16266811, "name": "Dexigner", "screen_name": "dexigner", "lang": "en", "location": "New York", "create_at": date("2008-09-12"), "description": "Dexigner is the leading online portal for designers, architects, illustrators, engineers, artists, and creatives of all kinds.", "followers_count": 231843, "friends_count": 205, "statues_count": 12945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479423754686464, "text": "HAPPY 20th birthday to this gem! I �� u & thank u for making my first year at state amazing �� @meghanramy ���� https://t.co/WnZKObkaUq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2548517317 }}, "user": { "id": 1276269312, "name": "Brooke", "screen_name": "brookevaron", "lang": "en", "location": "SDSU ", "create_at": date("2013-03-17"), "description": "Your mind is a powerful thing, when you fill it with positive thoughts, your life will start to change", "followers_count": 288, "friends_count": 304, "statues_count": 2757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479423763087361, "text": "It's cool though Ik Brandon's drivin����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 729988092, "name": "paige nicole dawson", "screen_name": "_paigedawson_", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "all night with bud light", "followers_count": 381, "friends_count": 790, "statues_count": 21821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monticello, LA", "id": "01417b3c030308be", "name": "Monticello", "place_type": "city", "bounding_box": rectangle("-91.060564,30.473007 -91.032397,30.505716") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2251645, "cityName": "Monticello" } }
+{ "create_at": datetime("2016-05-06T00:00:03.000Z"), "id": 728479423805063170, "text": "���� https://t.co/RcK70id2Y3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 33092338, "name": "Zi", "screen_name": "SincerelyLegit", "lang": "en", "location": "instagram: Sincerelylegit", "create_at": date("2009-04-18"), "description": "North Carolina Central University", "followers_count": 2486, "friends_count": 947, "statues_count": 217267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479424895537152, "text": "Круто, когда любимый человек разделяет твои странные хобби, вроде страсти к прогулкам по старым кладбищам �� https://t.co/UJjP9CGn1i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.849803,21.323279"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 477868249, "name": "קאָסטין", "screen_name": "febrarium", "lang": "en", "location": "Los Angeles", "create_at": date("2012-01-29"), "description": "Doshiraque-Perregaux", "followers_count": 340, "friends_count": 212, "statues_count": 13176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oahu Cemetery & Crematory", "id": "08161a59f256b001", "name": "Oahu Cemetery & Crematory", "place_type": "poi", "bounding_box": rectangle("-157.8498031,21.3232789 -157.849803,21.323279") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479424962662400, "text": "Sorry for the spam but we just had such an amazing time�� #bestbuyfriendever @ Studio Tour at… https://t.co/pJivlCF4YL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.35297558,34.13883672"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "bestbuyfriendever" }}, "user": { "id": 28395617, "name": "Martina Andoscia", "screen_name": "martinahv", "lang": "en", "location": "East Providence, Rhode Island", "create_at": date("2009-04-02"), "description": "null", "followers_count": 384, "friends_count": 258, "statues_count": 32546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425017225216, "text": "@BiankaCastaneda I'm not readyyyyyy������ https://t.co/rls239PLL2", "in_reply_to_status": 728478721208786944, "in_reply_to_user": 83982429, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 83982429 }}, "user": { "id": 60534622, "name": "Ellen Gonzalez", "screen_name": "ellengonzalez19", "lang": "en", "location": "Brownsville, TX / Austin, TX", "create_at": date("2009-07-27"), "description": "UT Austin Journalism\\m/ | Class of 2016 | ΚΔΧ | Project Votesmart Intern | Teach For America Corp member | Cat Mommy", "followers_count": 699, "friends_count": 799, "statues_count": 18241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425109450754, "text": "It began to rain the moment I began streaming my Pyromancer run, foreshadowing the struggle that was to come...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163978474, "name": "mottern mind", "screen_name": "motternmind", "lang": "en", "location": "California", "create_at": date("2010-07-07"), "description": "don't ever come near me or my robot daughter ever again // writing the chameleon creator // icon by gravity_cam", "followers_count": 141, "friends_count": 154, "statues_count": 18712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425315020800, "text": "\"I have seen it before, I get around\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167347684, "name": "Kelsey", "screen_name": "kelten8", "lang": "en", "location": "Outside Your Window", "create_at": date("2013-02-10"), "description": "Writer. Wanna be singer. Hella rad. In my 12th year of school. I play (and suck at) video games. Japanese candy, heelys, and my cat are all I need.", "followers_count": 118, "friends_count": 281, "statues_count": 1552 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425331900416, "text": "When you're hot and you know it @NUrollerhockey https://t.co/ukaghXWvFf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 61706964 }}, "user": { "id": 407444089, "name": "Bridgete", "screen_name": "BridgeteQuinn", "lang": "en", "location": "NU '18 ΑΣΑ", "create_at": date("2011-11-07"), "description": "I don't know. I think I might be asian.", "followers_count": 740, "friends_count": 507, "statues_count": 29802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425336123392, "text": "I Feel like a super Hero.... The way I rocked tonight.... @… https://t.co/11qAjMLQmM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.02849,38.91673"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1345165831, "name": "THE WORLD FAMOUS", "screen_name": "DJSHABLAST", "lang": "en", "location": "null", "create_at": date("2013-04-11"), "description": "Ceo/DJ/PRODUCER/Promotor/Fleetdj BlackoutFridays 7-8pm EVERYDAY 12:00pm @BLASTOFFRADIO1 IG:DJSHABLAST DJSHABLAST@Gmail.com http://www.trueschoolradio.net", "followers_count": 680, "friends_count": 937, "statues_count": 3112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 11, "stateName": "District of Columbia", "countyID": 11001, "countyName": "District of Columbia", "cityID": 1150000, "cityName": "Washington" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425411436544, "text": "https://t.co/NYOJZzQyw1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 58622310, "name": "Boogie B. ©uzzo", "screen_name": "BoogieFONTE", "lang": "en", "location": "5200 Grooveway Blvd. ", "create_at": date("2009-07-20"), "description": "ㅤㅤㅤ ㅤ ㅤ ㅤ ㅤ ㅤ ㅤㅤㅤ ㅤ ㅤ ㅤ IG: cle_boogie \n \n 2x All American Pill Poppin Champion", "followers_count": 1438, "friends_count": 683, "statues_count": 178502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conroe, TX", "id": "361610d5aa1f6719", "name": "Conroe", "place_type": "city", "bounding_box": rectangle("-95.679001,30.23105 -95.397267,30.443978") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4816432, "cityName": "Conroe" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479425524736001, "text": "Wind 0.0 mph ---. Barometer 30.016 in, Falling slowly. Temperature 54.3 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479426049085440, "text": "Them nxggas slide up on ya like a escalator, Saying run ya shit for dey AK ya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3012326567, "name": "Jus", "screen_name": "Justinissaih", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2015-02-02"), "description": "Money bestfriend that's the Program", "followers_count": 214, "friends_count": 117, "statues_count": 5358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479426074267649, "text": "@abbu25 @CNBCnow no Sr. Este no es mi candidato. No me representa. Ni al propio partido representa", "in_reply_to_status": 728384763153682434, "in_reply_to_user": 523443923, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 523443923, 26574283 }}, "user": { "id": 160819223, "name": "El Tavo", "screen_name": "tavo928", "lang": "es", "location": "Miami, FL. Montevideo. Colón", "create_at": date("2010-06-28"), "description": "Hincha de Peñarol. nada mas.", "followers_count": 1284, "friends_count": 1545, "statues_count": 47477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479426581659649, "text": "@lweatonNC @Amarillotxsguy @ByronYork as I stated. I gave Trump more than what he deserved", "in_reply_to_status": 728478883390066688, "in_reply_to_user": 20336168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20336168, 1686241021, 47739450 }}, "user": { "id": 32750528, "name": "###conservative", "screen_name": "randydeabay", "lang": "en", "location": "29486", "create_at": date("2009-04-17"), "description": "constitutional conservative--#nevereveretrump", "followers_count": 106, "friends_count": 135, "statues_count": 6041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerville, SC", "id": "676d5bedd73ca2ec", "name": "Summerville", "place_type": "city", "bounding_box": rectangle("-80.235287,32.944965 -80.109533,33.054993") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45035, "countyName": "Dorchester", "cityID": 4570270, "cityName": "Summerville" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479426657165312, "text": "@MrBrownThumb @ABCLiz perfect description", "in_reply_to_status": 728475726110920704, "in_reply_to_user": 15713562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15713562, 21461944 }}, "user": { "id": 18794520, "name": "ɐlzznq", "screen_name": "buzzLA", "lang": "en", "location": "NE Los Angeles", "create_at": date("2009-01-08"), "description": "Jack-of-all-media-trades. Interested in people and things. #LA #HighlandPark #90042 resident. #Producer #Editor #TVProduction #PostProduction", "followers_count": 651, "friends_count": 1049, "statues_count": 8664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479427085008896, "text": "Happy birthday @yojadeey ���� #19 https://t.co/hZHd2Zwnto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3320617087 }}, "user": { "id": 2411578183, "name": "Cristina", "screen_name": "Cristinnnnaaa", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "@YaaBoiEezus ♡", "followers_count": 404, "friends_count": 295, "statues_count": 5377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479427403780096, "text": "@Sonicfan71 I have the digital copy", "in_reply_to_status": 728410946259976192, "in_reply_to_user": 214356248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 214356248 }}, "user": { "id": 827789186, "name": "britney 007", "screen_name": "b_r_i_t_n_e_y__", "lang": "en", "location": "riverside california", "create_at": date("2012-09-16"), "description": "I'm a gamer/ shopper I like to create a individual look on playstation home ps3 rocks fashion diva on home as well ^_-", "followers_count": 461, "friends_count": 193, "statues_count": 17252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479427999338496, "text": "another attempt down the drain..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716825269, "name": "mel", "screen_name": "mxlvnie_", "lang": "en", "location": "javani newton ", "create_at": date("2012-07-25"), "description": "null", "followers_count": 238, "friends_count": 218, "statues_count": 14211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479428058046464, "text": "S/O to @Shaylen_Carroll for making such a lit track.Keep up the amazing work #KAMIKAZE \nhttps://t.co/BZnviZUFLf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KAMIKAZE" }}, "user_mentions": {{ 216807965 }}, "user": { "id": 145823309, "name": "DJ-K (KENJI) 賢", "screen_name": "Deejay_K88", "lang": "en", "location": "In The V.I.P Session", "create_at": date("2010-05-19"), "description": "Alaska-Born,Pansexual Male, #LGBTQ,Japanese/American,Bernie Supporter,ADHD/Autism,Mobile DJ,Disney Fan,Scratch DJ Academy in L.A (2018).", "followers_count": 1392, "friends_count": 2833, "statues_count": 24356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College, AK", "id": "a4fbc08102ee5632", "name": "College", "place_type": "city", "bounding_box": rectangle("-147.910047,64.795273 -147.75567,64.900722") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 216750, "cityName": "College" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479428485914624, "text": "just wanna go somewhere far, where no one can find me...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366381084, "name": "Erikaa❣", "screen_name": "rikaa_eayy23", "lang": "en", "location": "null", "create_at": date("2011-09-01"), "description": "null", "followers_count": 322, "friends_count": 249, "statues_count": 15900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-06T00:00:04.000Z"), "id": 728479428687347712, "text": "Like I'm short and that's extra work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 402946932, "name": "Eritrean Princess", "screen_name": "SinitGhebre", "lang": "en", "location": "Charlotte, NC", "create_at": date("2011-11-01"), "description": "NCAT$U. Stop stressin & start finessin", "followers_count": 1412, "friends_count": 1058, "statues_count": 29293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429324881920, "text": "Honestly, one of the best sites for porn is @HelixStudios", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 221838500 }}, "user": { "id": 345548735, "name": "Chris Bright", "screen_name": "C_Bright1988", "lang": "en", "location": "South Bend,Indiana", "create_at": date("2011-07-30"), "description": "Loyal friend, college grad, Gaymer, LGBTQ activist, and family man.", "followers_count": 585, "friends_count": 2088, "statues_count": 20485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Bend, IN", "id": "20a70247c3cbdd23", "name": "South Bend", "place_type": "city", "bounding_box": rectangle("-86.385306,41.598756 -86.19642,41.760555") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18141, "countyName": "St. Joseph", "cityID": 1871000, "cityName": "South Bend" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429333295105, "text": "Wind 0.8 mph NNW. Barometer 29.740 in, Steady. Temperature 46.0 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429442338816, "text": "It's 3 AM and I can't sleep����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2738530823, "name": "Alan", "screen_name": "AlanJ_15", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "null", "followers_count": 46, "friends_count": 151, "statues_count": 1615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, IN", "id": "df1b6e7143e9c8d4", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-86.92488,40.339754 -86.768625,40.474718") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18157, "countyName": "Tippecanoe", "cityID": 1840788, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429605777408, "text": "Temp: 44.3°F - Dew Point: 41.5° - Wind: W @ 1.8 mph - Gust: 4.5 - Rain Today: 0.00in. - Pressure: 29.93in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 12, "statues_count": 17267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429878493184, "text": "Stopping drunk people from fighting is not a part of my job description", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 601546737, "name": "Abhishek Vijay", "screen_name": "_abhishekvijay", "lang": "en", "location": "New Brunswick, NJ", "create_at": date("2012-06-06"), "description": "null", "followers_count": 476, "friends_count": 394, "statues_count": 14008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479429916139520, "text": "On second thought I'm going to sleep ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3239232163, "name": "XVI. ⚓️", "screen_name": "JodeeyJoe", "lang": "en", "location": "anywhere Darius goes ❤️", "create_at": date("2015-06-07"), "description": "God picked a beautiful flower✨DarrianAnthony", "followers_count": 783, "friends_count": 581, "statues_count": 20321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spring, TX", "id": "343ecdd7da8dfae0", "name": "Spring", "place_type": "city", "bounding_box": rectangle("-95.43718,30.011444 -95.318449,30.115585") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869596, "cityName": "Spring" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479430100832258, "text": "I'm not going hard enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 51502991, "name": "Dr.Jackson", "screen_name": "KingOf_AllKings", "lang": "en", "location": "Bacc Of The Line | Tail♦️", "create_at": date("2009-06-27"), "description": "Baltimore Born Baltimore Raised Lincoln University Class of 2016 Health Science Major Psychology Minor Rest in Peace Key Loyalty is Everything |NUPE| FA13|⭐️E", "followers_count": 1690, "friends_count": 1763, "statues_count": 56749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479430406856704, "text": "Peace out Des Moines. I may be headed to Germany tomorrow but I… https://t.co/WEfM1HrCyF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.61989304,41.59178977"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 462667169, "name": "*AhNJah*", "screen_name": "AnjaSchepp", "lang": "en", "location": "null", "create_at": date("2012-01-12"), "description": "Conform to non-conformity. Iowa State Civil Engineering. \nClass of 2019.", "followers_count": 347, "friends_count": 306, "statues_count": 2494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479430448943104, "text": "I've been dissolving caffeine pills in my energy drinks and if I could show how I felt it would look sorta like this https://t.co/I4R4cCiOPU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 184154057, "name": "jules", "screen_name": "JuliaLWasHere", "lang": "en", "location": "353 → 740 → 443", "create_at": date("2010-08-28"), "description": "to greatness, somehow. @dbk @umdwritersbloc", "followers_count": 285, "friends_count": 123, "statues_count": 3833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Park, MD", "id": "e4c17912c815124d", "name": "College Park", "place_type": "city", "bounding_box": rectangle("-76.965596,38.971053 -76.903378,39.022888") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2418750, "cityName": "College Park" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479430759174144, "text": "Wind 5 mph SE. Barometer 1011.2 hPa, Falling. Temperature 79.5 °F. Rain today 0.00 in. Humidity 25%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 155493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479430994059265, "text": "@Luke5SOS #riplukeslipring ��", "in_reply_to_status": -1, "in_reply_to_user": 403245020, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "riplukeslipring" }}, "user_mentions": {{ 403245020 }}, "user": { "id": 527019522, "name": "Destiny D.", "screen_name": "AyoodestinyKTA", "lang": "en", "location": "In my room reading smut", "create_at": date("2012-03-16"), "description": "Waves", "followers_count": 562, "friends_count": 1172, "statues_count": 11714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Hill, TX", "id": "dd8d6c178c390a37", "name": "Forest Hill", "place_type": "city", "bounding_box": rectangle("-97.291809,32.645534 -97.241254,32.688584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4826544, "cityName": "Forest Hill" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479431077941248, "text": "I'm strong I got this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2942773856, "name": "Rafa", "screen_name": "babyglexa", "lang": "en", "location": "Corona's Casa ", "create_at": date("2014-12-25"), "description": "null", "followers_count": 451, "friends_count": 434, "statues_count": 9521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479431606538240, "text": "@AndrewNalley ����", "in_reply_to_status": 728479284759801856, "in_reply_to_user": 35473364, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35473364 }}, "user": { "id": 35510134, "name": "Mackenzie Haines", "screen_name": "mackenziehaines", "lang": "en", "location": "Bowling Green, KY", "create_at": date("2009-04-26"), "description": "WKU | Auburn Vet School Bound | ΑΓΔ Alumna", "followers_count": 812, "friends_count": 621, "statues_count": 5703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, KY", "id": "390600307e9cd236", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-86.542319,36.890907 -86.328698,37.040435") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21227, "countyName": "Warren", "cityID": 2108902, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479432004931586, "text": "@Colleeng2000 this is me when you're in the car every day https://t.co/g1o1KJByln", "in_reply_to_status": -1, "in_reply_to_user": 1518040892, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1518040892 }}, "user": { "id": 1634773154, "name": "Katie Geraghty", "screen_name": "FeralCheese", "lang": "en", "location": "Hopefully Asleep ", "create_at": date("2013-07-30"), "description": "There's a 99% chance I have no idea what I'm doing", "followers_count": 145, "friends_count": 182, "statues_count": 1448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479432797786113, "text": "When you wish you hang females to chill with in the am ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54915290, "name": "AtlDJYungsta", "screen_name": "atldjyungsta", "lang": "en", "location": "nation wide", "create_at": date("2009-07-08"), "description": "#MixshowDJ x Host x Tour DJ .. Bookings atldjyungsta@gmail.com /6786208280 #FlyStyle #TeamPhlyteDJ #MyAsu, the real #EA6", "followers_count": 2635, "friends_count": 2177, "statues_count": 45025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-86.418059,32.284593 -86.071398,32.443697") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-05-06T00:00:05.000Z"), "id": 728479432957149184, "text": "Улыбка... Уличный фотопортрет красивой женщины. https://t.co/szkTn0bJa7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01196518,37.69916652"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 3829425134, "name": "Allen Collier", "screen_name": "colaroler", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "null", "followers_count": 6, "friends_count": 0, "statues_count": 24225 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-06T00:00:06.000Z"), "id": 728479433137348608, "text": "happy 21st birthday to my best friend!! I wish I was there to celebrate with you, but soon we… https://t.co/3hgZXnVkWf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.93606558,34.04526352"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 283836669, "name": "Cassidy Blumenthal", "screen_name": "cassidyyb13", "lang": "en", "location": "null", "create_at": date("2011-04-17"), "description": "JRK❥|| LA - AZ || AEΦ", "followers_count": 298, "friends_count": 186, "statues_count": 24868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-06T00:00:06.000Z"), "id": 728479433191919616, "text": "Thinking about Scarlett Johansson & @JLo tonight.\nGood thoughts.\n\nEquivalent of reaching out & stroking the side of your face thoughts.\n:-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 85603854 }}, "user": { "id": 454321859, "name": "MRJohn Laurich", "screen_name": "JohnLaurich", "lang": "en", "location": "Burbank, Ca", "create_at": date("2012-01-03"), "description": "I am the biggest independent film maker of the 21st Century. The movies I make are artwork for generations to come. timeless treasures. Invest in me...please.", "followers_count": 933, "friends_count": 2319, "statues_count": 129119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-06T00:00:06.000Z"), "id": 728479434525708288, "text": "7 1/2 ���� https://t.co/W2jUOL75Jz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2277034266, "name": "Dev", "screen_name": "devyfen_", "lang": "en", "location": "Kapogaaaaay", "create_at": date("2014-01-04"), "description": "FC ❤️", "followers_count": 930, "friends_count": 843, "statues_count": 20286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Pointe, HI", "id": "013abac55b5e444f", "name": "Ocean Pointe", "place_type": "city", "bounding_box": rectangle("-158.044905,21.303603 -158.013973,21.328864") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1556685, "cityName": "Ocean Pointe" } }
+{ "create_at": datetime("2016-05-06T00:00:06.000Z"), "id": 728479434575994885, "text": "I love seeing people doing good & being successful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1939865678, "name": "kri$", "screen_name": "_kristibruh", "lang": "en", "location": "with junior ", "create_at": date("2013-10-05"), "description": "Juniors girl ♡ 08/31/2015", "followers_count": 1275, "friends_count": 723, "statues_count": 20939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Watsonville, CA", "id": "ab2a6a25454fda97", "name": "Watsonville", "place_type": "city", "bounding_box": rectangle("-121.800366,36.894399 -121.732144,36.941555") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 683668, "cityName": "Watsonville" } }
+{ "create_at": datetime("2016-05-06T00:00:06.000Z"), "id": 728479434664116224, "text": "@RobStutzman @sactotrixie @RogerSalazar @madrid_mike @keckery @TimRosales reminds me of a whiffle ball game", "in_reply_to_status": 728443188004814848, "in_reply_to_user": 16641041, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16641041, 14329611, 32614987, 450341380, 19461723, 87058316 }}, "user": { "id": 809758578, "name": "Ed Emerson", "screen_name": "EdEmerson1", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-09-07"), "description": "Opinions are my own and you really should listen to me", "followers_count": 623, "friends_count": 1172, "statues_count": 1055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841796021682176, "text": "so my bf let me put make-up on him, after I'm done I give him the mirror so he can see, few mins later he says, \"I look like a model now\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 118292781, "name": "MariJ. Crvnts", "screen_name": "cbabby", "lang": "en", "location": "down on the west coast", "create_at": date("2010-02-27"), "description": "my only goal in life right now is to be happy. genuinely, intensely, consistently happy, regardless of what that looks like to others.\n#XOtillweOD ❤✌", "followers_count": 266, "friends_count": 498, "statues_count": 3230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841796252368898, "text": "Someone let me borrow some gloves", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 725568775286706176, "name": "Nicole", "screen_name": "MIDWEEKMOOP", "lang": "en", "location": "Pasadena, CA", "create_at": date("2016-04-27"), "description": "I'm funny and married af", "followers_count": 38, "friends_count": 86, "statues_count": 96 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841796319502336, "text": "Alberto's sounds so good :( :( :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971027918, "name": "sidney", "screen_name": "sidneydestinyxo", "lang": "en", "location": "Irvine, CA", "create_at": date("2012-11-25"), "description": "null", "followers_count": 553, "friends_count": 320, "statues_count": 12707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841796390785024, "text": "Once again, Twitter has been so helpful, thank u all", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1220216048, "name": "babygirl", "screen_name": "twozday__", "lang": "en", "location": "torrey ✨", "create_at": date("2013-02-25"), "description": "I ain't sorry, nigga, nah.", "followers_count": 983, "friends_count": 602, "statues_count": 35857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340391 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841796814548992, "text": "@blvckbeans lol lol lol. Ok, I'll be somewhere being a tall ass giraffe", "in_reply_to_status": 728841517289381889, "in_reply_to_user": 104680016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 104680016 }}, "user": { "id": 476554361, "name": "LDN x ATL", "screen_name": "LondonJvae", "lang": "en", "location": "Atlanta, GA", "create_at": date("2012-01-27"), "description": "Down to Mars guh from Zone4", "followers_count": 1166, "friends_count": 867, "statues_count": 23305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841797112205312, "text": "Temp: 51.4F W C: 51.4F Wind:--- at 0.0kts Baro: 1017.0mb and Steady Rain today: 0.00in R H: 95% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 210, "friends_count": 218, "statues_count": 104382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841797112365057, "text": "ily��❤️�� https://t.co/LF7ld3Px2Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 382122463, "name": "Nadine", "screen_name": "Go_Nades", "lang": "en", "location": "United States", "create_at": date("2011-09-29"), "description": "Jersey | Kutztown", "followers_count": 622, "friends_count": 553, "statues_count": 6891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kutztown, PA", "id": "76551cc477dcd48e", "name": "Kutztown", "place_type": "city", "bounding_box": rectangle("-75.798022,40.505567 -75.763846,40.534547") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42011, "countyName": "Berks", "cityID": 4240656, "cityName": "Kutztown" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841797493907456, "text": "@zealousNihilist d00d iz bedtime �� #sorry", "in_reply_to_status": 728841656624029696, "in_reply_to_user": 1903321250, "favorite_count": 0, "retweet_count": 0, "lang": "lv", "is_retweet": false, "hashtags": {{ "sorry" }}, "user_mentions": {{ 1903321250 }}, "user": { "id": 95753967, "name": "Tyler Eaves", "screen_name": "chilarome", "lang": "en", "location": "The Midwest Manhattan", "create_at": date("2009-12-09"), "description": "Singer/screamer/bassist. Aspiring professional history dude. General nerd. Happily married to @annagraceface. I did yoga once.", "followers_count": 560, "friends_count": 2015, "statues_count": 16862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, KS", "id": "1276a44d783a529d", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-96.649744,39.159985 -96.499443,39.259872") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20161, "countyName": "Riley", "cityID": 2044250, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841798462775296, "text": "https://t.co/JQD4B8rh5R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2787931316, "name": "Reynold Forman", "screen_name": "PPS_ELA_SUPV", "lang": "en", "location": "West Freehold, NJ", "create_at": date("2014-09-03"), "description": "District Supervisor of Language Arts, K-8- Princeton Public Schools forgive.grateful.believe.achieve.grow. purpose-driven-life.", "followers_count": 146, "friends_count": 328, "statues_count": 834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Freehold, NJ", "id": "01bb5d775f6ebee3", "name": "West Freehold", "place_type": "city", "bounding_box": rectangle("-74.332704,40.18095 -74.24345,40.252228") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3479100, "cityName": "West Freehold" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841799083560960, "text": "Now for movie night with the loml ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1660044115, "name": "DEBRA", "screen_name": "Debrareneee", "lang": "en", "location": "Edinburg, TX", "create_at": date("2013-08-10"), "description": "null", "followers_count": 594, "friends_count": 340, "statues_count": 8384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edinburg, TX", "id": "013fbf673c82e5e4", "name": "Edinburg", "place_type": "city", "bounding_box": rectangle("-98.225807,26.244127 -98.079031,26.450924") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4822660, "cityName": "Edinburg" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841799318413312, "text": "\"������✔️\" ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1597593096, "name": "Josue ⌚️", "screen_name": "Josue7ab", "lang": "en", "location": "null", "create_at": date("2013-07-15"), "description": "Habitual Flow", "followers_count": 78, "friends_count": 65, "statues_count": 193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841799570104320, "text": "@boalsr21 nope...im a rays and dodgers fan. The angels are always on here so I catch games", "in_reply_to_status": 728812280889094144, "in_reply_to_user": 373024571, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373024571 }}, "user": { "id": 27331462, "name": "Fredo Dooley", "screen_name": "AlfredoJDooley", "lang": "en", "location": "Los Angeles, California ", "create_at": date("2009-03-28"), "description": "Virginia born Gentleman, Proud USF Alumni; #CuriousSoul #WorldTraveler; Obsessive #Writer; & #ArmyVet. Lord of the land #Progressive #Obamacrat #ImWithHer HRC", "followers_count": 1251, "friends_count": 1116, "statues_count": 30265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-07T00:00:00.000Z"), "id": 728841799649787904, "text": "Lol so irritating https://t.co/bIkMRS4NSe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60487171, "name": "Nae from Aurora", "screen_name": "HolidaySeasonn", "lang": "en", "location": "Box State Shawty", "create_at": date("2009-07-26"), "description": "The highest form of human intelligence is to observe yourself without judgement. hippy ✌ #OMO #OVO", "followers_count": 2619, "friends_count": 1946, "statues_count": 123413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841800333574144, "text": "Wind 0.0 mph ---. Barometer 30.011 in, Steady. Temperature 47.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841800408965120, "text": "@ebbtideapp Tide in Cherry Point, Washington 05/07/2016\n Low 12:02am 4.6\nHigh 5:26am 9.1\n Low 12:28pm -1.9\nHigh 7:43pm 9.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.7583,48.8633"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 84, "friends_count": 1, "statues_count": 30683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, USA", "id": "bc3a38d3d5999b4b", "name": "Washington", "place_type": "admin", "bounding_box": rectangle("-124.848975,45.543542 -116.915989,49.002502") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841800983547904, "text": "Captain America: Civil War 10/10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1632381104, "name": "Jamal Shelton (SZN)", "screen_name": "JShelton713", "lang": "en", "location": "null", "create_at": date("2013-07-30"), "description": "Texas A&M University Class of 2016. Biology Major. Future Doctor. Gig 'em", "followers_count": 711, "friends_count": 513, "statues_count": 15526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryan, TX", "id": "9ce0f1d2eb5998ce", "name": "Bryan", "place_type": "city", "bounding_box": rectangle("-96.439644,30.609214 -96.255918,30.722503") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4810912, "cityName": "Bryan" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841801155514368, "text": "HAPPY BIRTHDAY SARAH! ❤️ Thank you for always putting a smile on my face after a shitty day :) love you ❤️❤️❤️ @SarahBaska", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1873398656 }}, "user": { "id": 1965655854, "name": "Ashley", "screen_name": "ashsuxs", "lang": "en", "location": "ashleyestrada6987@icloud.com", "create_at": date("2013-10-16"), "description": "tf do I put here", "followers_count": 574, "friends_count": 1256, "statues_count": 8628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841801533169664, "text": "Temp 45.3° Hi/Lo 45.5/45.3 Rng 0.2° WC 45.3° Hmd 97% Rain 0.00\" Storm 0.54\" BAR 29.906 Falling DP 44.5° Wnd 1mph Dir ENE Gst 6mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 121, "statues_count": 18493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841801621180416, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Steady. Temperature 54.7 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841801864495104, "text": "On @hulu #drama", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "drama" }}, "user_mentions": {{ 15033883 }}, "user": { "id": 1191713784, "name": "INSPIRE CHANGE", "screen_name": "12AND2CLOTHING", "lang": "en", "location": "DMV", "create_at": date("2013-02-17"), "description": "Streetwear for the TranzforMD. Weird. ◼2/2 #DMV", "followers_count": 131, "friends_count": 114, "statues_count": 3755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Severn, MD", "id": "1c98b097b6fcb4d6", "name": "Severn", "place_type": "city", "bounding_box": rectangle("-76.767088,39.094164 -76.642175,39.214142") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2471150, "cityName": "Severn" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841801990209536, "text": ".@CaptainAmerica I stand with #TeamCap. #TweetMe\nhttps://t.co/ncQhzXiKhY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeamCap", "TweetMe" }}, "user_mentions": {{ 701615052 }}, "user": { "id": 257274144, "name": "Randy F. Diaz", "screen_name": "randy1984d", "lang": "es", "location": "Panama", "create_at": date("2011-02-24"), "description": "Ing. Informática con énfasis en redes y programación, estudiante de Derecho y Ciencias Políticas.", "followers_count": 143, "friends_count": 231, "statues_count": 2019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841802527158273, "text": "In the need, not the want.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1099300826, "name": "Juvenile Philosopher", "screen_name": "Chip_Skyler", "lang": "en", "location": "(parts unknown)", "create_at": date("2013-01-17"), "description": "my crew is big, and it keeps getting bigger.", "followers_count": 265, "friends_count": 219, "statues_count": 7585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay City, MI", "id": "0121f9435fdae948", "name": "Bay City", "place_type": "city", "bounding_box": rectangle("-83.97498,43.538242 -83.842876,43.696322") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26017, "countyName": "Bay", "cityID": 2606020, "cityName": "Bay City" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841802539671552, "text": "Never realized how much stuff I have until now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 787704841, "name": "Emily Brautigam", "screen_name": "brautigam_emily", "lang": "en", "location": "null", "create_at": date("2012-08-28"), "description": "ΔΘΣ Lil Sis. Wilmington College.", "followers_count": 384, "friends_count": 376, "statues_count": 2203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilmington, OH", "id": "3499e4b1d67ce535", "name": "Wilmington", "place_type": "city", "bounding_box": rectangle("-83.862365,39.407985 -83.776472,39.463321") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39027, "countyName": "Clinton", "cityID": 3985792, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841802653011968, "text": "2 in the morning my mind is on you..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2436990569, "name": "Shelby", "screen_name": "shelbyblades15", "lang": "en", "location": "null", "create_at": date("2014-03-25"), "description": "null", "followers_count": 355, "friends_count": 475, "statues_count": 2543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Springs, MS", "id": "5a55712f28ac9c7e", "name": "Ocean Springs", "place_type": "city", "bounding_box": rectangle("-88.853041,30.382999 -88.698136,30.426897") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28059, "countyName": "Jackson", "cityID": 2853520, "cityName": "Ocean Springs" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841802883715072, "text": "Temp: 52.7°F Wind:0.0mph Pressure: 30.038hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841802917117952, "text": "@lcasares023 you fuckin with him? Get him off my phone", "in_reply_to_status": 728838886005350403, "in_reply_to_user": 2569751776, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2569751776 }}, "user": { "id": 1542200678, "name": "Kimberly❣", "screen_name": "KaayDoll_Xo", "lang": "en", "location": "Tx 4ever like Bun B", "create_at": date("2013-06-23"), "description": "| C/o 2015 |Instagram: Kimberlylove_xo | Sc: Kimberlyxo97|", "followers_count": 215, "friends_count": 229, "statues_count": 22186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841803026321410, "text": "03:00:01 |Temp: 44.8ºF |Dew Point 41.5ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SSW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 94, "friends_count": 21, "statues_count": 93785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841803068223489, "text": "Ripley SW Limestone Co. Temp: 51.1°F Wind:0.0mph Pressure: 994.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841803298947072, "text": "Wind 0.0 mph ---. Barometer 29.858 in, Falling slowly. Temperature 52.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841803319775233, "text": "I said nah I wanna see its eyes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4043194873, "name": "Watona", "screen_name": "_spacequeen505", "lang": "en", "location": "Tucson, AZ", "create_at": date("2015-10-27"), "description": "#Navajo #NativeAmerican #HipHopRomantic #OneLove #FOE #NMtoAZ✌ IG: _spacequeen505", "followers_count": 433, "friends_count": 691, "statues_count": 8052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841803412049920, "text": "You say you're loyal but you're in my Dms?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479986081, "name": "Eduardo Velazquez", "screen_name": "ChronicBeans_", "lang": "en", "location": "null", "create_at": date("2012-01-31"), "description": "Man Of The Year", "followers_count": 82, "friends_count": 122, "statues_count": 205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-07T00:00:01.000Z"), "id": 728841804070653954, "text": "Wind 0.0 mph ---. Barometer 29.895 in, Steady. Temperature 57.5 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841805219827714, "text": "Don't expect much from anyone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330813584, "name": "N-T", "screen_name": "ngrace_", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-07-06"), "description": "@ junior year: relax", "followers_count": 95, "friends_count": 182, "statues_count": 205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841805362515968, "text": "KD look like a dude cramming 5 minutes before a test and Russ look like he just walked outta yoga class. https://t.co/z1Nd1irQsu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1703740832, "name": "Kellen Becoats", "screen_name": "KellenBecoats", "lang": "en", "location": "Athens, OH", "create_at": date("2013-08-26"), "description": "Copy editor for http://www.thepostathens.com & writer for @BackdropMag | Formerly: @FOXSoccer | We are the roses. These are my damaged petals. | Ohio '17", "followers_count": 366, "friends_count": 352, "statues_count": 5782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Solon, OH", "id": "a440dcd36043e76b", "name": "Solon", "place_type": "city", "bounding_box": rectangle("-81.488956,41.348091 -81.353487,41.424696") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3972928, "cityName": "Solon" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841805383540737, "text": "YESSSSSS.... LITTT MOTHER FKER\nLOKILLO ENTERTAINMENT \nITS AT VACCA LOUNGE\n\nDINE, DRINK, TURN UP… https://t.co/YWOpFgWK1m", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.92000055,40.86138095"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 87135142, "name": "LokiLlo", "screen_name": "LokiLlo_R1", "lang": "en", "location": "ÜT: 40.7417,-73.858832", "create_at": date("2009-11-02"), "description": "Electrical, Mechanical & Music Engineer, Song Writer Party Animal, Outgoing, LiVin 2 Da Max DOmiNiCaNO HaStA La TamBorA RaTaCuChUPla!!!", "followers_count": 169, "friends_count": 82, "statues_count": 11593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841805505044481, "text": "Great night ⚡️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1194423744, "name": "Adam Gonzalez", "screen_name": "AdamGonzalez__", "lang": "en", "location": "Torrance, CA", "create_at": date("2013-02-18"), "description": "Snapchat : adam_gonzalez IG: adamgonzzzz WHS 17' ⚽️⚾️", "followers_count": 873, "friends_count": 890, "statues_count": 7071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841805739892736, "text": "Happy birthday @KayyChloee thank u for existing and being such a huge part of my life for a long time. I hope you're turnin tf up rn ������ ily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2576450045 }}, "user": { "id": 982409352, "name": "Baela", "screen_name": "AylaBaela", "lang": "en", "location": "San Diego", "create_at": date("2012-12-01"), "description": "null", "followers_count": 487, "friends_count": 525, "statues_count": 20738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santee, CA", "id": "04ed8141edf1aba0", "name": "Santee", "place_type": "city", "bounding_box": rectangle("-117.039886,32.814991 -116.936159,32.887369") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 670224, "cityName": "Santee" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841806092247040, "text": "https://t.co/6EzgQ1Zrn1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 233753182, "name": "♡♡Nathalie", "screen_name": "GLITTERBOMBn", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-01-03"), "description": "She's not heartless, she's cautious. She knows the value of her love. #TweetThruIt", "followers_count": 257, "friends_count": 256, "statues_count": 6046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841806486507521, "text": "61.6F (Feels: 61.6F) - Humidity: 79% - Wind: 0.0mph --- - Gust: 2.2mph - Pressure: 1015.4mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841806490701825, "text": "your actions not matching your words rn. u gon have to do more than just say it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521794269, "name": "Lucas Scott", "screen_name": "_JaredJD", "lang": "en", "location": "BAYAREAAAAA", "create_at": date("2012-03-11"), "description": "Jared Joseph D. (JD) | Herc Class of '15 | SC: jared-jd", "followers_count": 771, "friends_count": 499, "statues_count": 3577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hercules, CA", "id": "6fafb06c49df870f", "name": "Hercules", "place_type": "city", "bounding_box": rectangle("-122.2956,37.983956 -122.222315,38.035293") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 633308, "cityName": "Hercules" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841806721388544, "text": "Me regreso!!! @javiVillalpando @Tole08 @catarrin https://t.co/FMkWUM72wb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 57398508, 61035572, 54665639 }}, "user": { "id": 60604667, "name": "Daniel Margolin", "screen_name": "dmargolio", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-07-27"), "description": "San Diego State University SMBA 17 Candidate", "followers_count": 124, "friends_count": 147, "statues_count": 452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841806863949824, "text": "������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 394372741, "name": "seth. Ü", "screen_name": "sethdiaz_", "lang": "en", "location": "San Antonio, TX - Odessa, TX", "create_at": date("2011-10-19"), "description": "You is kind, You is smart, You is important. @SierraDiaz_ is my twin. | SPURS NATION.", "followers_count": 1517, "friends_count": 503, "statues_count": 67896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841807065276416, "text": "@RodriguezJ_7 happy birthday queer bag Hmu #18 #Anal #NoLube", "in_reply_to_status": -1, "in_reply_to_user": 270599446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Anal", "NoLube" }}, "user_mentions": {{ 270599446 }}, "user": { "id": 428662053, "name": "El Matador", "screen_name": "JuanMoreTime69", "lang": "en", "location": "Guadalajara , Oregon", "create_at": date("2011-12-04"), "description": "#JMFam", "followers_count": 508, "friends_count": 146, "statues_count": 17589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841807317114881, "text": "Lmao.... https://t.co/9XlABG6oIf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 60153108, "name": "Xavier LaFlamme", "screen_name": "JayVelez4L", "lang": "en", "location": "IG: JayVelez4L Snap: Squez", "create_at": date("2009-07-25"), "description": "null", "followers_count": 422, "friends_count": 186, "statues_count": 11780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841807627423744, "text": "I only want one person OMM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226029161, "name": "Zo Glo✨", "screen_name": "ZoieLaurenxo", "lang": "en", "location": "Chicago✈️ New Orleans ", "create_at": date("2010-12-12"), "description": "Glo'in and Grow'in. ✨ k116 #XULA19 Sc:ZoieLauren", "followers_count": 2419, "friends_count": 1156, "statues_count": 137931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-07T00:00:02.000Z"), "id": 728841808365621249, "text": "Тельбес - а130_stitch.jpg https://t.co/kgz7OKvb1P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01161346,37.6990307"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3829425134, "name": "Allen Collier", "screen_name": "colaroler", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "null", "followers_count": 6, "friends_count": 0, "statues_count": 24344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841808927674368, "text": "Is it just the moment or something beyond our concept of time?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549843191, "name": "Clayton Rodriguez", "screen_name": "TheHolyFlipster", "lang": "en", "location": "South Side Queens", "create_at": date("2012-04-09"), "description": "Perspective: We look up at the same stars and see such different things B \n\nIG: Same \n\nFlickr: https://www.flickr.com/photos/133299968@N06/", "followers_count": 253, "friends_count": 912, "statues_count": 5259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841810060185600, "text": "When you driving drunk & you pass my a police car ������ https://t.co/EDWXW9UVur", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324051504, "name": "still Quen", "screen_name": "theOnlyQuen_PHE", "lang": "en", "location": "Dallas / SFA ", "create_at": date("2011-06-25"), "description": "#PHE - Welcome to Hustle University", "followers_count": 2732, "friends_count": 1022, "statues_count": 149817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841810164875265, "text": "okkkkkk but who else besides wade has been there �� wade old ass ain't carrying them to no finals my nigga https://t.co/iz2G6qYhrk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750849597, "name": "Alton Williams", "screen_name": "Simply_AhhWill", "lang": "en", "location": "null", "create_at": date("2014-08-24"), "description": "I'm ahead of my time", "followers_count": 290, "friends_count": 253, "statues_count": 6844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841810475241472, "text": "@RedwoodGirl Also my internet just failed halfway through streaming an episode of something. FUCK ALL OF THIS.", "in_reply_to_status": 728839821662162944, "in_reply_to_user": 15174190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15174190 }}, "user": { "id": 51203, "name": "Anna Pickard", "screen_name": "annapickard", "lang": "en", "location": "San Francisco", "create_at": date("2006-12-08"), "description": "I do words and talking and fancy typing. I do them at @SlackHQ. Mother of a preschooler, lover of swearwords (and all other words)(and my husband)(obvs).", "followers_count": 4314, "friends_count": 143, "statues_count": 42707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841810542362624, "text": "I love the world", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542024544, "name": "analee", "screen_name": "AnaleeBonanno", "lang": "en", "location": "null", "create_at": date("2012-03-31"), "description": "null", "followers_count": 216, "friends_count": 137, "statues_count": 2664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pacific Grove, CA", "id": "1003b193f1cb0dcd", "name": "Pacific Grove", "place_type": "city", "bounding_box": rectangle("-121.942439,36.597763 -121.901121,36.638132") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 654848, "cityName": "Pacific Grove" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841810781437952, "text": "@babyy_ooscar you could've made it better ������", "in_reply_to_status": 728841711728754688, "in_reply_to_user": 1939726513, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1939726513 }}, "user": { "id": 988049778, "name": "Josh", "screen_name": "Sexy_Chenche", "lang": "en", "location": "VISCA BARCA ", "create_at": date("2012-12-03"), "description": "SC: Therealchenche", "followers_count": 441, "friends_count": 336, "statues_count": 14491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841811121340418, "text": "Wind 0.0 mph ---. Barometer 29.85 in, Steady. Temperature 52.0 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 120, "statues_count": 159607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841811620290560, "text": "@yungfuego so u can like my selfie but not come to show ok i see u", "in_reply_to_status": -1, "in_reply_to_user": 311755278, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311755278 }}, "user": { "id": 1034024960, "name": "v", "screen_name": "ew_vick", "lang": "en", "location": "clark ", "create_at": date("2012-12-24"), "description": "mucho trabajo poco dinero", "followers_count": 5395, "friends_count": 1174, "statues_count": 14200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841811842633728, "text": "I need above get away..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142156747, "name": "YoungCoCo", "screen_name": "HerNameIs_Amber", "lang": "en", "location": "Monroe, LA", "create_at": date("2010-05-09"), "description": "❤️", "followers_count": 1749, "friends_count": 794, "statues_count": 103434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-07T00:00:03.000Z"), "id": 728841811981033473, "text": "Wind 0.0 mph S. Barometer 30.000 in, Rising slowly. Temperature 61.1 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841812857659392, "text": "honestly thought we gon play it off coo but nah.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 758248225, "name": "νι ∂ιин", "screen_name": "ViDinh3278", "lang": "en", "location": "Tukwila, WA", "create_at": date("2012-08-14"), "description": "null", "followers_count": 449, "friends_count": 276, "statues_count": 10980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tukwila, WA", "id": "9883a4be0cbd497c", "name": "Tukwila", "place_type": "city", "bounding_box": rectangle("-122.318673,47.407147 -122.241356,47.531957") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5372625, "cityName": "Tukwila" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841813079924737, "text": "When you at a boring party. You get a couple drinking in you when you can feel your cheeks stuck on smile mode ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194412236, "name": "Junior", "screen_name": "J_Dizzle_520", "lang": "en", "location": "tยςร๏ภ, คгเz๏ภค", "create_at": date("2010-09-23"), "description": "Compassion is a weakness your enemies will not share. Thats why its so important, Its separates us from them -BatMan", "followers_count": 56, "friends_count": 212, "statues_count": 5897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841813327417344, "text": "Hungry as always", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3754206076, "name": "Alexa", "screen_name": "lexmercadoo", "lang": "en", "location": "South El Monte, CA", "create_at": date("2015-09-23"), "description": "living", "followers_count": 306, "friends_count": 212, "statues_count": 3571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841813423874048, "text": "The drive through girl at Taco Bell left her number on one of my burrito wrappers ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2263157161, "name": "Ha¥den", "screen_name": "Hayd3nG", "lang": "en", "location": "null", "create_at": date("2013-12-26"), "description": "hhs 16' • Never forget the good old days", "followers_count": 343, "friends_count": 494, "statues_count": 3440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841813474316288, "text": "Classic, the \"image\" @BarackObama that Satan breathed life into is condemning @realDonaldTrump. I love a good war of good vs evil. #Rev13", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rev13" }}, "user_mentions": {{ 813286, 25073877 }}, "user": { "id": 76481322, "name": "Joseywalesful Daniel", "screen_name": "Joseywalesful", "lang": "en", "location": "Carbondale, PA", "create_at": date("2009-09-22"), "description": "defender of the faithful Daniel 12:1", "followers_count": 956, "friends_count": 2264, "statues_count": 35987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leominster, MA", "id": "c4f1830ea4b8caaf", "name": "Leominster", "place_type": "city", "bounding_box": rectangle("-71.813501,42.4762 -71.702186,42.573956") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2535075, "cityName": "Leominster" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841813696520192, "text": "@Myaaaaahk scope", "in_reply_to_status": 728841774085480448, "in_reply_to_user": 2735955907, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1545072846 }}, "user": { "id": 2735955907, "name": "Davir Hamilton", "screen_name": "EspnHam9", "lang": "en", "location": "Compton ✈️ Salt Lake city ", "create_at": date("2014-08-15"), "description": "SAVAGE BUT KING", "followers_count": 2741, "friends_count": 740, "statues_count": 39960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841814598246401, "text": "#Motivation Get motivated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Motivation" }}, "user": { "id": 2932984711, "name": "susan graf/Cote", "screen_name": "CoteGraf", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-12-16"), "description": "I'm a Licensed MFT. http://UAreWell.com. will tell you about me. it will be up soon!", "followers_count": 1, "friends_count": 36, "statues_count": 5 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841814640332800, "text": "Watch your mouth.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2266083053, "name": "Kiiii", "screen_name": "ki_annaaa", "lang": "en", "location": "null", "create_at": date("2014-01-06"), "description": "new yawk ✈️MD coppinstateu.", "followers_count": 714, "friends_count": 448, "statues_count": 8708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841814866857985, "text": "I was no more good I'm still weak thinking about it bc you were dead serious lmaoooo https://t.co/1HaqjR7mXh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278818562, "name": "☔️☔️☔️", "screen_name": "RideOrNAI_", "lang": "en", "location": "VCU17", "create_at": date("2011-04-07"), "description": "They're pious here, but you and I we're pioneers.", "followers_count": 1203, "friends_count": 624, "statues_count": 76025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841815038656512, "text": "guess who's shooting tomorrow�� @ The Trap https://t.co/rFYnLmzxvt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.22735645,48.5143514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 556924517, "name": "Easton", "screen_name": "eastonlemos", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-04-18"), "description": "seattle based photographer • bookings: eastonalemos@gmail.com • snapchat & instagram: eastonlemos", "followers_count": 1537, "friends_count": 304, "statues_count": 9380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sedro-Woolley, WA", "id": "0eac77dee15e84b3", "name": "Sedro-Woolley", "place_type": "city", "bounding_box": rectangle("-122.291559,48.475472 -122.182096,48.543866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit", "cityID": 5363210, "cityName": "Sedro-Woolley" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841815864975360, "text": "@AlexChuyBerger never again", "in_reply_to_status": 728841733895655424, "in_reply_to_user": 993331076, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 993331076 }}, "user": { "id": 905177934, "name": "Max", "screen_name": "MaxMisner", "lang": "en", "location": "null", "create_at": date("2012-10-25"), "description": "Life is short, stunt it. Iowa State University class of 2018", "followers_count": 207, "friends_count": 236, "statues_count": 3605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodbury, MN", "id": "8439c955de1f8036", "name": "Woodbury", "place_type": "city", "bounding_box": rectangle("-92.984787,44.861854 -92.86283,44.948985") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27163, "countyName": "Washington", "cityID": 2771428, "cityName": "Woodbury" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841815982415873, "text": "Sometimes, I hate when I'm right ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75376313, "name": "Mehak Adamjee", "screen_name": "madamjee", "lang": "en", "location": "Houston, Texas", "create_at": date("2009-09-18"), "description": "I'm not just making a living, I'm making my life. MAM PLANNING | DESIGN", "followers_count": 543, "friends_count": 436, "statues_count": 18698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-07T00:00:04.000Z"), "id": 728841816540254209, "text": "It only seem like I got hoes cause I never erased the emojis after the break ups ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323015758, "name": "KushBoyWonder", "screen_name": "GiveMeDatN3ck", "lang": "en", "location": "Houston Spliff Texas", "create_at": date("2011-06-23"), "description": "It's All In The Spliff", "followers_count": 617, "friends_count": 401, "statues_count": 11526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841817324539904, "text": "��️ shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2260799971, "name": "alexis", "screen_name": "lexiss510", "lang": "en", "location": "null", "create_at": date("2013-12-24"), "description": "null", "followers_count": 127, "friends_count": 114, "statues_count": 1121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841817798610945, "text": "Jsu twitter be live wen it wanna be ������ Durin school dont nobody be mentioning nobody������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2297495932, "name": "ThuggaThugga", "screen_name": "1wardera", "lang": "en", "location": "East Atlanta", "create_at": date("2014-01-22"), "description": "•Sit back and let god take you through life..He'll take you to places you would've never imagined you would be•Defensive Back At Jackson State University•", "followers_count": 962, "friends_count": 798, "statues_count": 26202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Candler-McAfee, GA", "id": "7202b2c7918d203e", "name": "Candler-McAfee", "place_type": "city", "bounding_box": rectangle("-84.309736,33.712266 -84.230819,33.739899") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1312834, "cityName": "Candler-McAfee" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841818536738817, "text": "@notcayenne @gayblessing can I add this to my resume", "in_reply_to_status": 728841678971265024, "in_reply_to_user": 1396674440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1396674440, 4552819819 }}, "user": { "id": 18876633, "name": "Charlotte Peters", "screen_name": "suchsharpteeth", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-01-11"), "description": "Dragon-for-hire.", "followers_count": 343, "friends_count": 306, "statues_count": 8391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841818670923776, "text": "Why is the @DennysDiner Twitter the best thing to happen to me in years", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 23112346 }}, "user": { "id": 593963371, "name": "Amelia Madison", "screen_name": "ameeels_parkayy", "lang": "en", "location": "Tempe, AZ", "create_at": date("2012-05-29"), "description": "I like my whiskey straight, my wine chilled and my Taco Bell on the reg", "followers_count": 573, "friends_count": 523, "statues_count": 8146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841818696097792, "text": "happy 17th birthday to my love���� https://t.co/Lo10W2OfMA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3263002370, "name": "lex", "screen_name": "LexiDones", "lang": "en", "location": "somewhere watching anime", "create_at": date("2015-06-30"), "description": "@SusYayo", "followers_count": 245, "friends_count": 174, "statues_count": 1691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841819077758976, "text": "When all your thoughts come out all at once lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490846538, "name": "ℱ❥", "screen_name": "iaafaay", "lang": "en", "location": "Seattle, HI", "create_at": date("2012-02-12"), "description": "null", "followers_count": 297, "friends_count": 163, "statues_count": 24055 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, WA", "id": "f479e40901a48515", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-122.266398,47.257029 -122.144477,47.356233") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5303180, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841819790827520, "text": "Kick it?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1961262618, "name": "avocado", "screen_name": "groovinlilbxtch", "lang": "en", "location": "my love for the sky is eternal", "create_at": date("2013-10-14"), "description": "save my soul. LLM", "followers_count": 2623, "friends_count": 506, "statues_count": 67106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841820398981120, "text": "Its truuuuuuuue. Look how they shine for youuuuuu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2153815488, "name": "Angel", "screen_name": "FeliciasVida", "lang": "en", "location": "null", "create_at": date("2013-10-24"), "description": "Angel in disguise", "followers_count": 807, "friends_count": 191, "statues_count": 42035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-07T00:00:05.000Z"), "id": 728841820625461248, "text": "I go so dumb on snapchat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2329477672, "name": "L E X", "screen_name": "alexis_hudnall", "lang": "en", "location": "Alexandria, LA --Marshall, TX", "create_at": date("2014-02-06"), "description": "Hundreds in my wallet, that's hot", "followers_count": 284, "friends_count": 285, "statues_count": 5685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841821313343488, "text": "@Connormoy ��������������", "in_reply_to_status": 728839460092174336, "in_reply_to_user": 403947425, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 403947425 }}, "user": { "id": 117198025, "name": "Groundislava", "screen_name": "Groundislava", "lang": "en", "location": "the sprawl", "create_at": date("2010-02-24"), "description": "~~~~~~~~~**WEDIDIT**~~~~~~~~~ USA: eli.glad@unitedtalent.com EU/ROW: sinan.ors@unitedtalent.com", "followers_count": 11906, "friends_count": 528, "statues_count": 20700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841821460127745, "text": "https://t.co/De44mvqIbY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 233753182, "name": "♡♡Nathalie", "screen_name": "GLITTERBOMBn", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-01-03"), "description": "She's not heartless, she's cautious. She knows the value of her love. #TweetThruIt", "followers_count": 257, "friends_count": 256, "statues_count": 6047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841821732769792, "text": "Se va muriendo mi alma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2478659852, "name": "m a r y", "screen_name": "marymejia1414", "lang": "en", "location": "Avenal, CA", "create_at": date("2014-05-05"), "description": "null", "followers_count": 394, "friends_count": 310, "statues_count": 10139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avenal, CA", "id": "4ef9eb0b1d3f720c", "name": "Avenal", "place_type": "city", "bounding_box": rectangle("-120.146913,35.964275 -120.110888,36.016284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 603302, "cityName": "Avenal" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841821896491009, "text": "the world is crazy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271972110, "name": "jim", "screen_name": "_JJM25", "lang": "en", "location": "ashtabula. ", "create_at": date("2014-01-01"), "description": "lakeside highschool. bbb @matthewbuhl_", "followers_count": 798, "friends_count": 339, "statues_count": 103801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashtabula, OH", "id": "630dc7394773f347", "name": "Ashtabula", "place_type": "city", "bounding_box": rectangle("-80.903411,41.817663 -80.746099,41.911745") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39007, "countyName": "Ashtabula", "cityID": 3902638, "cityName": "Ashtabula" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841821934125056, "text": "Wind 2.8 mph SE. Barometer 29.799 in, Falling. Temperature 55.1 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10550 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841822055829506, "text": "@princesspammyy hi pal", "in_reply_to_status": 728841724269826048, "in_reply_to_user": 383487980, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 383487980 }}, "user": { "id": 68232669, "name": "Lucas Scott", "screen_name": "josetwinkie", "lang": "en", "location": "Orlando, FL", "create_at": date("2009-08-23"), "description": "Amateur alcohol enthusiast. I once shotgunned a beer in 1.72 seconds", "followers_count": 543, "friends_count": 388, "statues_count": 12413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University, FL", "id": "01f196c135e37b16", "name": "University", "place_type": "city", "bounding_box": rectangle("-81.224485,28.56471 -81.170066,28.61218") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1273172, "cityName": "University" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841822307409920, "text": "Bowden hella irky cause he keep saying \"hella\" ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 723725797, "name": "THE MILK MAKER", "screen_name": "BlayJha", "lang": "en", "location": "California, USA", "create_at": date("2012-07-29"), "description": "tkl", "followers_count": 2796, "friends_count": 1697, "statues_count": 78006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841822965927936, "text": "Mmmmm ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4384904774, "name": "Vrania Williams", "screen_name": "vrania_16_", "lang": "en", "location": "null", "create_at": date("2015-11-27"), "description": "null", "followers_count": 61, "friends_count": 73, "statues_count": 30 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Porterville, CA", "id": "27f6e6ddf03d8df1", "name": "East Porterville", "place_type": "city", "bounding_box": rectangle("-119.01713,36.047148 -118.933958,36.069503") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 621012, "cityName": "East Porterville" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841823444045826, "text": "@Mapula_Alex https://t.co/1jcCB3RvUP", "in_reply_to_status": 728840699974590464, "in_reply_to_user": 419392084, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 419392084 }}, "user": { "id": 2269901934, "name": "smoothie queen", "screen_name": "andreavocean", "lang": "en", "location": "null", "create_at": date("2013-12-30"), "description": "i am @partyomo | scorp", "followers_count": 290, "friends_count": 242, "statues_count": 4231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841823469199360, "text": "@ChaiShakur ����", "in_reply_to_status": 728841753961238528, "in_reply_to_user": 287013844, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 287013844 }}, "user": { "id": 312251468, "name": "badgyal", "screen_name": "xkennkenn", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-06"), "description": "#TAMU19 DDD #cowboysnation", "followers_count": 1061, "friends_count": 506, "statues_count": 49198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-07T00:00:06.000Z"), "id": 728841824002052096, "text": "Two more weeks until I'm home cuddling with my Mia ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 167101580, "name": "Kristen Marlene A.", "screen_name": "kristenmarlenee", "lang": "en", "location": "NYC/ONEONTA ", "create_at": date("2010-07-15"), "description": "blogger, singer, sucker for romantic comedies, coffee enthusiast & lover of Nutella xx ΘΦΑ", "followers_count": 703, "friends_count": 412, "statues_count": 15133 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oneonta, NY", "id": "ff9cc975fb01f747", "name": "Oneonta", "place_type": "city", "bounding_box": rectangle("-75.108773,42.433438 -74.980422,42.478019") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36077, "countyName": "Otsego", "cityID": 3654881, "cityName": "Oneonta" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841825218256897, "text": "Wind 3 mph SSE. Barometer 1010.3 hPa, Falling. Temperature 81.7 °F. Rain today 0.00 in. Humidity 21%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 155636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841825432182784, "text": "Temp: 67.5°F - Dew Point: 51.8° - Wind: WNW @ 14.2 mph - Gust: 18.3 - Rain Today: 0.00in. - Pressure: 29.70in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 12, "statues_count": 17315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841825444732928, "text": "I actually am in love with Oliver just so everyone knows", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2226910478, "name": "Samantha Jordyn", "screen_name": "sam_miltonmjm", "lang": "en", "location": "null", "create_at": date("2013-12-02"), "description": "I'd have a lot to give, if I still gave a damn.", "followers_count": 236, "friends_count": 164, "statues_count": 2417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841825503555585, "text": "True I feel you https://t.co/JDtfV34U6w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232918389, "name": "Tawanna ♡♡♡", "screen_name": "lovingreigns_xo", "lang": "en", "location": "Raleigh", "create_at": date("2011-01-01"), "description": "|God 1st| 21| insta: kissmybeautyy_xo | snapchat: tawanna_xo| RAD❤| I love wrestling and Roman Reigns 3x champ #RomanEmpire", "followers_count": 2193, "friends_count": 2448, "statues_count": 100639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841826078216192, "text": "Tonight was a shit show", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 235731159, "name": "Destinee Semrau", "screen_name": "destineesemrau", "lang": "en", "location": "Kenmore, NY", "create_at": date("2011-01-08"), "description": "D'Youville College", "followers_count": 642, "friends_count": 604, "statues_count": 30932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenmore, NY", "id": "bd3815dd0b1b5790", "name": "Kenmore", "place_type": "city", "bounding_box": rectangle("-78.889267,42.958569 -78.853089,42.973238") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3639232, "cityName": "Kenmore" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841826988253184, "text": "I'll just watch Harry Potter since I have no entertainment ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 289408641, "name": "ᒪI'ᗷᗩᔕᑫᑌIᗩT :)", "screen_name": "_brylannnn", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-04-28"), "description": "be yourself", "followers_count": 568, "friends_count": 483, "statues_count": 25442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monticello, LA", "id": "01417b3c030308be", "name": "Monticello", "place_type": "city", "bounding_box": rectangle("-91.060564,30.473007 -91.032397,30.505716") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2251645, "cityName": "Monticello" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841827269357569, "text": "Wind 0 mph ESE. Barometer 30.02 in, Rising. Temperature 88.0 °F. Rain today 0.00 in. Humidity 55%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.25111111,28.87472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1227982520, "name": "Deltona Weather", "screen_name": "DeltonaWeather", "lang": "en", "location": "Deltona, Florida", "create_at": date("2013-02-28"), "description": "Weather Info Updated Every 60 Minutes", "followers_count": 60, "friends_count": 76, "statues_count": 27438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deltona, FL", "id": "ac43a9e147f03d63", "name": "Deltona", "place_type": "city", "bounding_box": rectangle("-81.285896,28.84511 -81.154121,28.964071") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12127, "countyName": "Volusia", "cityID": 1217200, "cityName": "Deltona" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841827571408896, "text": "#fridaynightinmontclair @ Egan & Sons https://t.co/zxLvcUF7PB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.2118583,40.8177806"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fridaynightinmontclair" }}, "user": { "id": 71255589, "name": "Andie C. Henrich", "screen_name": "andie_c_henrich", "lang": "en", "location": "Florham Park, NJ", "create_at": date("2009-09-03"), "description": "Trying to make a difference in this world.", "followers_count": 40, "friends_count": 66, "statues_count": 529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montclair, NJ", "id": "bceb8153865c4a49", "name": "Montclair", "place_type": "city", "bounding_box": rectangle("-74.237314,40.788029 -74.182684,40.86178") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3426610, "cityName": "Glen Ridge" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841827818872832, "text": "Gn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3029214035, "name": "hell nah", "screen_name": "vivian__15", "lang": "en", "location": "null", "create_at": date("2015-02-10"), "description": "try me bitch", "followers_count": 242, "friends_count": 237, "statues_count": 6609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, GA", "id": "4ff21355b6fb2bda", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-84.368715,33.502691 -84.342143,33.543533") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1342604, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841827931951104, "text": "I been having way to much fun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 194002328, "name": "YeezyE", "screen_name": "FuckEdwardA", "lang": "en", "location": "Selma Ca", "create_at": date("2010-09-22"), "description": "null", "followers_count": 124, "friends_count": 235, "statues_count": 4087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-05-07T00:00:07.000Z"), "id": 728841828473044992, "text": "5/7/2016 - 02:00\nTemp: 51.1F \nHum: 96%\nWind: 0.0 mph\nBaro: 30.015in. & Falling\nRain: 0.00 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 52503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204183736291328, "text": "and ill probably add her on fb in the next week or so then delete these tweets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.27227892,37.87362793"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30943190, "name": "Gustav N. Davila", "screen_name": "NikJamTheMan", "lang": "en", "location": "A's 148 Raiders 319 Cal QQ/9", "create_at": date("2009-04-13"), "description": "Cal alum, Academy of Art IP. Game designer and music composer. Sports fan: Raiders, Sharks, A's, Warriors, Cal, Quakes. @GoldenBlogs writer. #BlackLivesMatter", "followers_count": 925, "friends_count": 1359, "statues_count": 66398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204184017453057, "text": "a girl lit her keurig on fire hahahaha classic cros", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 61563404, "name": "danielle anderson", "screen_name": "_daanielllle", "lang": "en", "location": "prob w brianna and caroline", "create_at": date("2009-07-30"), "description": "tortillas are edible frisbees", "followers_count": 1433, "friends_count": 728, "statues_count": 105235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxford, MS", "id": "b86cd6ef7a54d640", "name": "Oxford", "place_type": "city", "bounding_box": rectangle("-89.571643,34.321375 -89.495897,34.387965") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28071, "countyName": "Lafayette", "cityID": 2854840, "cityName": "Oxford" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204184311009280, "text": "Hopefully Walmart hires me for that Pharmacy Tech position ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1638556904, "name": "godly", "screen_name": "apollosogodly", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-08-01"), "description": "born a perfectionist", "followers_count": 1169, "friends_count": 782, "statues_count": 18057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Park Forest, IL", "id": "c1ccf403500d3a65", "name": "Park Forest", "place_type": "city", "bounding_box": rectangle("-87.724692,41.456311 -87.655579,41.50625") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1757732, "cityName": "Park Forest" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204184667586560, "text": "My niggggaaaaaaaaaaaaaa https://t.co/akNF2DvVop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1549682779, "name": "Gabriela♈️", "screen_name": "gabbriellaaax", "lang": "en", "location": "125.", "create_at": date("2013-06-26"), "description": "follow me for endless selfies .", "followers_count": 539, "friends_count": 445, "statues_count": 24600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204184680116224, "text": "Not sorry I'm blunt��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256768302, "name": "Tori Lee Lyons", "screen_name": "tori_theLYONESS", "lang": "en", "location": "901", "create_at": date("2011-02-23"), "description": "all good things are wild and free", "followers_count": 566, "friends_count": 579, "statues_count": 16232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204185149759488, "text": "The Library. Who's down?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 138241817, "name": "Yvette.", "screen_name": "YvetteForever", "lang": "en", "location": "Rialto. SoCal", "create_at": date("2010-04-28"), "description": "Just live life.", "followers_count": 155, "friends_count": 151, "statues_count": 8329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204185279782912, "text": "@LookAtLollyy this where u at? https://t.co/ii9TxrPO8N", "in_reply_to_status": -1, "in_reply_to_user": 2897913300, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2897913300 }}, "user": { "id": 133690764, "name": "❌BBGrDIE OG Y.T.❌", "screen_name": "yteezy_bbg225", "lang": "en", "location": "BATON ROUGE✈DALLAS", "create_at": date("2010-04-16"), "description": "NEW IG@yteezy_bbg225.....sc: yteezy_bbg225#LSU#RIPHOUND#RIP PHAT, YOUNG,FLY&THUGGIN.....#FREEVICKS/BLACK BALLOON GLOBAL ✈ #SHERWOOD/UPT BR", "followers_count": 4071, "friends_count": 1282, "statues_count": 269130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204185720193024, "text": "I like your style ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603264355, "name": "Twéíçk™", "screen_name": "Weick5522", "lang": "en", "location": "Louisiana", "create_at": date("2012-06-08"), "description": "Viewer discretion is advised. \n\nIG: @weick1998", "followers_count": 442, "friends_count": 403, "statues_count": 29386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walker, LA", "id": "006d3e902283f759", "name": "Walker", "place_type": "city", "bounding_box": rectangle("-90.911197,30.45628 -90.81948,30.569654") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22063, "countyName": "Livingston", "cityID": 2279240, "cityName": "Walker" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186022182912, "text": "@ebbtideapp Tide in East Boothbay, Maine 05/08/2016\n Low 6:50am -1.9\nHigh 1:03pm 10.5\n Low 7:05pm -0.8\nHigh 1:16am 11.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-69.5833,43.865"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 30964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23015, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186265624576, "text": "03:00:01 |Temp: 54.4ºF |Dew Point 50.6ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the W, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 94, "friends_count": 21, "statues_count": 93832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186303344640, "text": "https://t.co/NJ1Yjkqyql", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3667196534, "name": "Savanna Ferrer", "screen_name": "savy_ferreraxo", "lang": "en", "location": "null", "create_at": date("2015-09-23"), "description": "null", "followers_count": 301, "friends_count": 345, "statues_count": 1611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ferry Pass, FL", "id": "33a522a8e7317b6e", "name": "Ferry Pass", "place_type": "city", "bounding_box": rectangle("-87.247666,30.479609 -87.158257,30.562505") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1222275, "cityName": "Ferry Pass" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186596794368, "text": "I been to 3 proms and this one was the best. Thank you Beth-Anne for the best prom I've ever had. #Prom2k16.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Prom2k16" }}, "user": { "id": 184180137, "name": "bry", "screen_name": "bryonavelar", "lang": "en", "location": "Tumwater, WA", "create_at": date("2010-08-28"), "description": "lost || in || motion", "followers_count": 315, "friends_count": 112, "statues_count": 1517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tumwater, WA", "id": "12148f3ffcceb203", "name": "Tumwater", "place_type": "city", "bounding_box": rectangle("-123.010086,46.913138 -122.860365,47.029499") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5372905, "cityName": "Tumwater" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186819088384, "text": "Say my name in your song so I know it's real.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312385488, "name": "B A $ E D", "screen_name": "illestbreed", "lang": "en", "location": "newport beach, california", "create_at": date("2011-06-06"), "description": "Cali ❤️ IG: illestbreed SC: illvstbreed", "followers_count": 458, "friends_count": 162, "statues_count": 7018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Juan Capistrano, CA", "id": "4aea239b6a146e77", "name": "San Juan Capistrano", "place_type": "city", "bounding_box": rectangle("-117.686553,33.466555 -117.611591,33.54673") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 668028, "cityName": "San Juan Capistrano" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204186907328512, "text": "@whybang https://t.co/ZGoVl0nO4u", "in_reply_to_status": 729200892449198080, "in_reply_to_user": 121920558, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 121920558 }}, "user": { "id": 121920558, "name": "Paz", "screen_name": "whybang", "lang": "en", "location": "null", "create_at": date("2010-03-10"), "description": "level 82 warlock", "followers_count": 525, "friends_count": 337, "statues_count": 42870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blue Island, IL", "id": "fb0971608411ae20", "name": "Blue Island", "place_type": "city", "bounding_box": rectangle("-87.709565,41.630338 -87.651678,41.677417") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1706704, "cityName": "Blue Island" } }
+{ "create_at": datetime("2016-05-08T00:00:00.000Z"), "id": 729204187490189313, "text": "In the words of @lilhippyy , I am done with the population", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 270557311 }}, "user": { "id": 392332458, "name": "thaliemarissa", "screen_name": "ThalieTimsit", "lang": "en", "location": "Santa Barbara, CA", "create_at": date("2011-10-16"), "description": "living the dream", "followers_count": 312, "friends_count": 267, "statues_count": 4297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188027162624, "text": "I'm taking my talents back to 2K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712871824451846144, "name": "Brandon", "screen_name": "BrandonLSimmons", "lang": "en", "location": "Baltimore, MD", "create_at": date("2016-03-23"), "description": "✟ God First Everyday. John 15-16! Fight Me @ Pax East 2017", "followers_count": 27, "friends_count": 117, "statues_count": 279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188031246337, "text": "�������� https://t.co/VmqEnFNsqE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1716679182, "name": "kvngsss ✨", "screen_name": "Rangel_Boo", "lang": "en", "location": "null", "create_at": date("2013-08-31"), "description": "sc : javiles_03", "followers_count": 613, "friends_count": 469, "statues_count": 21460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188509405184, "text": "@_clott_ ����������������", "in_reply_to_status": 729204120628936704, "in_reply_to_user": 409737118, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 409737118 }}, "user": { "id": 225633164, "name": "Certified Hotboy", "screen_name": "VernKTA", "lang": "en", "location": "wmu 19", "create_at": date("2010-12-11"), "description": "I forgot tomorrow, I forgot today.", "followers_count": 1058, "friends_count": 927, "statues_count": 77192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmers Branch, TX", "id": "97b70992556c6354", "name": "Farmers Branch", "place_type": "city", "bounding_box": rectangle("-96.938694,32.895548 -96.821221,32.953548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4825452, "cityName": "Farmers Branch" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188597489664, "text": "love is some crazy shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 752662250, "name": "Quint", "screen_name": "its_quinton", "lang": "en", "location": "null", "create_at": date("2012-08-12"), "description": "condescending asshole", "followers_count": 429, "friends_count": 362, "statues_count": 5814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188710834176, "text": "Ripley SW Limestone Co. Temp: 62.8°F Wind:0.7mph Pressure: 994.7mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204188945604608, "text": "Was so happy to see @Imaginedragons perform an acoustic set tonight. So much authentic talent on one stage ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75916180 }}, "user": { "id": 598986879, "name": "Kira Kosarin", "screen_name": "kirakosarin", "lang": "en", "location": "⚡️", "create_at": date("2012-06-03"), "description": "I'm 18, very clumsy, and a make believe superhero.", "followers_count": 127439, "friends_count": 911, "statues_count": 12729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189117579264, "text": "If you name your kid Timmy you're just setting him up for a life of Ls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2255980404, "name": "cheek collector", "screen_name": "YoungBruffBoy", "lang": "en", "location": "Tacoma, WA", "create_at": date("2013-12-20"), "description": "I'm just here to post memes and piss people off", "followers_count": 515, "friends_count": 259, "statues_count": 25394 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189377630209, "text": "@FlomaxxRoach wow that's so unfortunate �� LA got me fucked up", "in_reply_to_status": 729203206970966016, "in_reply_to_user": 711305419, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 711305419 }}, "user": { "id": 343231555, "name": "kaitlynn", "screen_name": "kaitlynndose", "lang": "en", "location": "bass stage", "create_at": date("2011-07-27"), "description": "i only tweet about bass music and my dog", "followers_count": 1299, "friends_count": 372, "statues_count": 60263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189453111298, "text": "Best https://t.co/ByYZ3Ae7BF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1393078248, "name": "Jake Rhodes", "screen_name": "JakeRhodes77", "lang": "en", "location": "Chap", "create_at": date("2013-04-30"), "description": "wakeboard", "followers_count": 420, "friends_count": 347, "statues_count": 1467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand View Estates, CO", "id": "724fd0dce5013e4e", "name": "Grand View Estates", "place_type": "city", "bounding_box": rectangle("-104.830099,39.536396 -104.809999,39.556489") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8035, "countyName": "Douglas", "cityID": 831935, "cityName": "Grand View Estates" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189595754496, "text": "@Zainoo097 @Yayooo55 @ashleynosalag yo", "in_reply_to_status": 729202732926681088, "in_reply_to_user": 1461374515, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1461374515, 2546444944, 1107273066 }}, "user": { "id": 453464108, "name": "Yousef", "screen_name": "YousefJafry", "lang": "en", "location": "null", "create_at": date("2012-01-02"), "description": "null", "followers_count": 345, "friends_count": 157, "statues_count": 15130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, IL", "id": "48f2609344f0c3f7", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-87.799908,41.733586 -87.740829,41.757299") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1709642, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189633626112, "text": "Wind 3.0 mph NW. Barometer 29.920 in, Rising slowly. Temperature 45.4 °F. Rain today 0.00 in. Humidity 62%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 40, "friends_count": 4, "statues_count": 27263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189780389888, "text": "@Jsann_Derulo ily2 ☺️☺️☺️", "in_reply_to_status": 729204108788436992, "in_reply_to_user": 355710560, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 355710560 }}, "user": { "id": 458914418, "name": "Chloe ✨", "screen_name": "DropIt_Chlo", "lang": "en", "location": "null", "create_at": date("2012-01-08"), "description": "A girl should be two things: Classy & Fabulous❤ -- Coco Chanel.", "followers_count": 816, "friends_count": 769, "statues_count": 29752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Owings Mills, MD", "id": "d84a2837f4d13599", "name": "Owings Mills", "place_type": "city", "bounding_box": rectangle("-76.843666,39.38247 -76.750777,39.461505") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2459425, "cityName": "Owings Mills" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204189931446272, "text": "And she's going to paid in full concerts in June.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4161590429, "name": "Mariah Lee", "screen_name": "MariahLeeRVD", "lang": "en", "location": "Socastee, SC", "create_at": date("2015-11-10"), "description": "Aria Larraine ❤️ 4/26/2016", "followers_count": 29, "friends_count": 93, "statues_count": 229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Socastee, SC", "id": "0043ce7c1cad5c16", "name": "Socastee", "place_type": "city", "bounding_box": rectangle("-79.06754,33.614208 -78.921739,33.731483") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4567390, "cityName": "Socastee" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190262771713, "text": "Just closed down bama lanes with my boys @MARVELOUS_WILL @StoneEdwards11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 611434429, 2817381876 }}, "user": { "id": 774998526, "name": "jake chavers", "screen_name": "JakeChavers", "lang": "en", "location": "null", "create_at": date("2012-08-22"), "description": "psalms 144:1 always pointing me in the right direction", "followers_count": 248, "friends_count": 494, "statues_count": 1526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prattville, AL", "id": "97fed3139e3dfc27", "name": "Prattville", "place_type": "city", "bounding_box": rectangle("-86.519989,32.415392 -86.3955,32.520816") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1001, "countyName": "Autauga", "cityID": 162328, "cityName": "Prattville" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190438903808, "text": "Chanel. https://t.co/NgglMWHPC2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1129702753, "name": "✨", "screen_name": "___sheilaa", "lang": "en", "location": "Bedford Stuyvesant, Brooklyn", "create_at": date("2013-01-28"), "description": "Chosen few", "followers_count": 372, "friends_count": 142, "statues_count": 17007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190489251840, "text": "UNNNNNNNO ������ https://t.co/886Xxh4gZU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1647836540, "name": "YFC", "screen_name": "__ObeyIshe", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "Ava&Gold✨", "followers_count": 2494, "friends_count": 1494, "statues_count": 74881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190715609088, "text": "this was needed ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3149019829, "name": "rich homie JUAN", "screen_name": "rojasjuan98", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-04-10"), "description": "respekt me || E.J.N ❤️", "followers_count": 296, "friends_count": 244, "statues_count": 11082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "National City, CA", "id": "7905cfee1600eb70", "name": "National City", "place_type": "city", "bounding_box": rectangle("-117.124452,32.64634 -117.049103,32.69328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 650398, "cityName": "National City" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190803697664, "text": "Tomita and the Starhustler are definitely hanging out near some starclusters as we speak.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16035903, "name": "agnesfidget", "screen_name": "lowlytootle", "lang": "en", "location": "seattle, washington", "create_at": date("2008-08-28"), "description": "Shy, retiring electrofunk/robot/disco DJ with a love for the absurd.", "followers_count": 348, "friends_count": 437, "statues_count": 5505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190854139906, "text": "If I've learned anything in my short time at semo, it's that shit will buff out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 629007257, "name": "Josh Ames", "screen_name": "ASAPjosh_", "lang": "en", "location": "Shiloh/Cape", "create_at": date("2012-07-06"), "description": "ΣΧ", "followers_count": 327, "friends_count": 208, "statues_count": 1594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204190900162560, "text": "@xcountmeoutx https://t.co/hO9JEP1uXk", "in_reply_to_status": 729202368458330112, "in_reply_to_user": 39717323, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 31293465 }}, "user": { "id": 39717323, "name": "j", "screen_name": "wheelofpainx", "lang": "en", "location": "null", "create_at": date("2009-05-13"), "description": "the trial of a mortal soul begging to die xxx", "followers_count": 427, "friends_count": 544, "statues_count": 12574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191055384576, "text": "That shower tho! ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 63563712, "name": "Mutha of a King!!", "screen_name": "Mutha_of_a_King", "lang": "en", "location": "Illinois", "create_at": date("2009-08-06"), "description": "Jus my thoughts! Plz get offended lol", "followers_count": 199, "friends_count": 366, "statues_count": 20262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191256682496, "text": "Excited https://t.co/DWXcPOthO6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288024422, "name": "❤", "screen_name": "_imgabyy", "lang": "en", "location": "null", "create_at": date("2011-04-25"), "description": "I live to give my love a perfect life", "followers_count": 844, "friends_count": 663, "statues_count": 32028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191789461504, "text": "Temp: 56.7°F Wind:0.0mph Pressure: 30.035hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191835529217, "text": "����you cool dab lol I fuck wit you playboy ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151198266, "name": "QuesGotATwin", "screen_name": "thatboysmith_12", "lang": "en", "location": "null", "create_at": date("2010-06-02"), "description": "Ques from #TJC'17| Kinesiology Major| Tjonespromo| Without God I'm nothing.", "followers_count": 1009, "friends_count": 856, "statues_count": 15821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitehouse, TX", "id": "014db80b53f65135", "name": "Whitehouse", "place_type": "city", "bounding_box": rectangle("-95.267432,32.195608 -95.187324,32.251686") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4878388, "cityName": "Whitehouse" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191873224704, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":61.2°F Wind:1.1mph Pressure: 29.88hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 85, "friends_count": 17, "statues_count": 317319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204191915298816, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 59.5 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-08T00:00:01.000Z"), "id": 729204192011767808, "text": "Temp 48.7° Hi/Lo 48.7/48.2 Rng 0.5° WC 48.7° Hmd 98% Rain 0.00\" Storm 0.00\" BAR 29.693 Falling DP 48.2° Wnd 0mph Dir --- Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 122, "statues_count": 18519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204192380911617, "text": "Wind 0.0 mph ---. Barometer 29.867 in, Steady. Temperature 62.7 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204192468815872, "text": "@aayrxon ye", "in_reply_to_status": -1, "in_reply_to_user": 1922102724, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1922102724 }}, "user": { "id": 2909323466, "name": "Adrian Quintero", "screen_name": "_quizy98", "lang": "en", "location": "null", "create_at": date("2014-11-24"), "description": "as I grow yearly I see things more clearly", "followers_count": 223, "friends_count": 168, "statues_count": 4148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204192510922756, "text": "@ColinWeyant11 happy birthday col������ love u", "in_reply_to_status": -1, "in_reply_to_user": 380536390, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 380536390 }}, "user": { "id": 1895432540, "name": "k80", "screen_name": "katienealll", "lang": "en", "location": "null", "create_at": date("2013-09-22"), "description": "hi on life", "followers_count": 883, "friends_count": 578, "statues_count": 39563 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perry Hall, MD", "id": "fd25378f7b90960b", "name": "Perry Hall", "place_type": "city", "bounding_box": rectangle("-76.503771,39.37832 -76.409521,39.436165") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2460975, "cityName": "Perry Hall" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204192598843392, "text": "\"let go of the drama\" is seriously something I'm telling myself everyday for the next couple of weeks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 966950646, "name": "cam", "screen_name": "cammij_", "lang": "en", "location": "California,USA", "create_at": date("2012-11-23"), "description": "your average annoying joe☮♡☺ #GoBeavs", "followers_count": 478, "friends_count": 374, "statues_count": 16059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204193186062336, "text": "REBECCA IS DEAD NOW WHAT THE FUUUCK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242387400, "name": "beep", "screen_name": "StayDangerous", "lang": "en", "location": "5/25/11.3/14/15", "create_at": date("2011-01-24"), "description": "Brig will be back sooner or later", "followers_count": 312, "friends_count": 404, "statues_count": 24587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204193886605312, "text": "3:00", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 774372000, "name": "boosie.brie", "screen_name": "brieannaaa_", "lang": "en", "location": "Richmond, VA", "create_at": date("2012-08-22"), "description": "streeeetttt!", "followers_count": 1051, "friends_count": 622, "statues_count": 27863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204194507431939, "text": "Wind 0.0 mph WNW. Barometer 29.870 in, Rising slowly. Temperature 52.8 °F. Rain today 0.01 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204194729598976, "text": "62.5F (Feels: 62.5F) - Humidity: 86% - Wind: 1.6mph E - Gust: 1.6mph - Pressure: 1014.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204195203616768, "text": "Lmaooooo https://t.co/GbUPrMuSwi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 1723887308, "name": "Tyler", "screen_name": "TGendler526", "lang": "en", "location": "Orlando/Plantation FL", "create_at": date("2013-09-02"), "description": "#HeatLifer #Agent0", "followers_count": 352, "friends_count": 340, "statues_count": 4965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plantation, FL", "id": "7df9a00dcf914d5e", "name": "Plantation", "place_type": "city", "bounding_box": rectangle("-80.330201,26.088262 -80.196833,26.160753") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12115, "countyName": "Sarasota", "cityID": 1257450, "cityName": "Plantation" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204195635527680, "text": "In need of puppies & mcdoubles", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 572837342, "name": "Kylee", "screen_name": "kyleejackson1", "lang": "en", "location": "Colorado ❃", "create_at": date("2012-05-06"), "description": "• UNCO •", "followers_count": 871, "friends_count": 571, "statues_count": 6379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greeley, CO", "id": "82dbbc30f11e52f2", "name": "Greeley", "place_type": "city", "bounding_box": rectangle("-104.832358,40.374596 -104.638594,40.454395") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8123, "countyName": "Weld", "cityID": 832155, "cityName": "Greeley" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204195669073920, "text": "at the end of the day , who got you ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3068652822, "name": "yoromeyo¡", "screen_name": "king_romeyo916", "lang": "en", "location": "null", "create_at": date("2015-03-08"), "description": "ig:yoromeyo // sc:ayyeromeyo", "followers_count": 158, "friends_count": 107, "statues_count": 4233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204195736293376, "text": "Wind 1.6 mph S. Barometer 29.63 in, Falling. Temperature 53.8 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 120, "statues_count": 159633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204195845390336, "text": "@jazztx1010 @ElizaGrae Right On!!!!!!! https://t.co/w7Wzls0zJT", "in_reply_to_status": 729203379306553344, "in_reply_to_user": 2479045748, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2479045748, 721095824285896705 }}, "user": { "id": 159447476, "name": "N D W", "screen_name": "gentlemanirish", "lang": "en", "location": "Ohio", "create_at": date("2010-06-25"), "description": "The Way The Truth & Life! Computers, internet, motorcycles, swimming, camping,animals,dogs, cats, birds, mountain climbing, hiking, exploring, music, movies...", "followers_count": 10638, "friends_count": 10671, "statues_count": 228105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massillon, OH", "id": "005f11de9931c8a4", "name": "Massillon", "place_type": "city", "bounding_box": rectangle("-81.631799,40.732694 -81.432311,40.896962") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39151, "countyName": "Stark", "cityID": 3948244, "cityName": "Massillon" } }
+{ "create_at": datetime("2016-05-08T00:00:02.000Z"), "id": 729204196046569472, "text": "As I said before and I'll say it again, Views has nothing on this when it comes to getting in your feels https://t.co/07WPQ5ClrB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2520523930, "name": "The Life of Johan", "screen_name": "TheJohanGaribae", "lang": "en", "location": "California ", "create_at": date("2014-04-30"), "description": "Dios Primero. lincoln 18. futbolista/high jumper", "followers_count": 529, "friends_count": 480, "statues_count": 10484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln Village, CA", "id": "e94e0dc22a196b10", "name": "Lincoln Village", "place_type": "city", "bounding_box": rectangle("-121.349795,37.998938 -121.319133,38.013759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 641558, "cityName": "Lincoln Village" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204196583575554, "text": "@Expelliarmus You don;t think it at first but I like his attitude and a lot of stuff is \"lovely\" lol. There's a Philly ep on this week!", "in_reply_to_status": 729203059503456256, "in_reply_to_user": 31668696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31668696 }}, "user": { "id": 102210244, "name": "Danielle", "screen_name": "quietnsarcastic", "lang": "en", "location": "my own little world ", "create_at": date("2010-01-05"), "description": "null", "followers_count": 45, "friends_count": 198, "statues_count": 22316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204196839333888, "text": "I'll get over it somehow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3054740023, "name": "mami", "screen_name": "skylardontstop", "lang": "en", "location": "ie", "create_at": date("2015-03-01"), "description": "null", "followers_count": 1809, "friends_count": 1776, "statues_count": 28079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204197586046976, "text": "Y'all pussy be good for like a good 2 weeks. Then I need something new ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 443310522, "name": "#ExplicitImprint", "screen_name": "AntwoinDGAF", "lang": "en", "location": "D[M]V ✈ SoFlo ✈️ ATL", "create_at": date("2011-12-21"), "description": "Founder of FBGApparel | clark atlanta university | manager of @_GeniusIdiot #NextUp | Bookings: iconicimage.mgmts@gmail.com |✨#LEGACYENT✨", "followers_count": 2148, "friends_count": 1214, "statues_count": 78909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple Hills, MD", "id": "50ab1362909ec83d", "name": "Temple Hills", "place_type": "city", "bounding_box": rectangle("-76.973441,38.802185 -76.93105,38.818865") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2477100, "cityName": "Temple Hills" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204197984378880, "text": "There's a guy named Duf running for Ca Senate �� https://t.co/tXkKdj6zqQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16488269, "name": "FrenchCori", "screen_name": "frenchcori", "lang": "en", "location": "Riverside, CA", "create_at": date("2008-09-27"), "description": "ATTENTION: Tweets dangerous sans mes lunettes! I❤️my hubs, husky, kitties, art + travel. BUSY wife, public servant, mom & union thug. Half French, 100% American", "followers_count": 1654, "friends_count": 2238, "statues_count": 35812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204198332633090, "text": "@HarmonyAlania smile lol", "in_reply_to_status": 729199863309574145, "in_reply_to_user": 4342930033, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4342930033 }}, "user": { "id": 240527880, "name": "FJ3", "screen_name": "yesfrankcan", "lang": "en", "location": "some where in americaひ", "create_at": date("2011-01-19"), "description": "null", "followers_count": 1269, "friends_count": 465, "statues_count": 101129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204198600888321, "text": "im just a funny & cool ass girl ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307080698, "name": "tia", "screen_name": "_TiaMariee", "lang": "en", "location": "ImFromThaSmoke#409, TX", "create_at": date("2011-05-28"), "description": "20 | #TJC17| livin'", "followers_count": 1547, "friends_count": 842, "statues_count": 25790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199011946496, "text": "who's up?\nlike for tbh dm ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2738539089, "name": "[JG]", "screen_name": "jakegatewood", "lang": "en", "location": "null", "create_at": date("2014-08-09"), "description": "jakegatewoodmusic@gmail.com", "followers_count": 2327, "friends_count": 706, "statues_count": 23120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Keller, TX", "id": "ad760d32a6a31ea5", "name": "Keller", "place_type": "city", "bounding_box": rectangle("-97.266092,32.892358 -97.176637,32.986837") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4838632, "cityName": "Keller" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199070654464, "text": "@thefinaathings ����������", "in_reply_to_status": 729190398849998848, "in_reply_to_user": 327730344, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 327730344 }}, "user": { "id": 2872912057, "name": "J.R. World", "screen_name": "payrollholli", "lang": "en", "location": "United States", "create_at": date("2014-10-23"), "description": "null", "followers_count": 251, "friends_count": 296, "statues_count": 6052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199116787712, "text": "@danidelapena love u", "in_reply_to_status": 729203417420177408, "in_reply_to_user": 1059810901, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1059810901 }}, "user": { "id": 1378575427, "name": "jacob ricks", "screen_name": "jacobricks33", "lang": "en", "location": "g-town, az", "create_at": date("2013-04-24"), "description": "fan/parody account || bhs @sarahkbaker5", "followers_count": 1077, "friends_count": 594, "statues_count": 8998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199120994304, "text": "@Jaime11369 @susy_hhc @te_apoya @blanca928 @JosueNelson21No ay nada como él amor Bien por #Nathalia la Reina de reina ����Felicidades", "in_reply_to_status": -1, "in_reply_to_user": 2214322186, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "Nathalia" }}, "user_mentions": {{ 2214322186, 3020852867, 2464025740, 2902312584 }}, "user": { "id": 3044133409, "name": "MarthaJulia", "screen_name": "PaniaguaMateo", "lang": "es", "location": "null", "create_at": date("2015-02-26"), "description": "null", "followers_count": 318, "friends_count": 252, "statues_count": 2904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Heights, CO", "id": "1b9f1a683fe6c2eb", "name": "Federal Heights", "place_type": "city", "bounding_box": rectangle("-105.025173,39.848951 -104.996591,39.885265") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 826270, "cityName": "Federal Heights" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199540543488, "text": "Wind 0.0 mph ---. Barometer 30.008 in, Steady. Temperature 52.9 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199565594624, "text": "I didn't just hear Iggy Azalea's song played at Sigma Nu.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 705830924487782402, "name": "Kevin", "screen_name": "ChowSigmaNu", "lang": "en", "location": "ΣΝ", "create_at": date("2016-03-04"), "description": "Fresno State. Get Nu'd", "followers_count": 221, "friends_count": 175, "statues_count": 537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199594983424, "text": "@lucia_silencio I do but really it's just me and my sister. Everyone else is gone. You ?", "in_reply_to_status": 729203625638150144, "in_reply_to_user": 3168027260, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3168027260 }}, "user": { "id": 3189433165, "name": "Clark Moses", "screen_name": "ScratchycClark", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 415, "friends_count": 482, "statues_count": 6853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, WA", "id": "df4875cc2df893ed", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-122.223209,47.534965 -122.076327,47.660748") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5305210, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199687249925, "text": "13 hours of driving today but had a great time wrangling and handling the pups of VAN-PAH. V•A•C Pack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3419938338, "name": "Alpha Pounder", "screen_name": "AlphaDogPounder", "lang": "en", "location": "Portland, OR", "create_at": date("2015-09-01"), "description": "A Giant Malamoot. My fur naturally smells of warm chocolate chip cookies. Raisin Cookies are dead to me. My Dog and Big brother is @alphadogroscoe. V•A•C Pack.", "followers_count": 338, "friends_count": 438, "statues_count": 1093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204199976624128, "text": "I hear what she's saying but her eyes tell a different story.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58622310, "name": "Boogie B. ©uzzo", "screen_name": "BoogieFONTE", "lang": "en", "location": "5200 Grooveway Blvd. ", "create_at": date("2009-07-20"), "description": "ㅤㅤㅤ ㅤ ㅤ ㅤ ㅤㅤㅤ ㅤ ㅤ ㅤ IG: cle_boogie ㅤㅤㅤ ㅤ ㅤ ㅤ ㅤㅤㅤ ㅤ ㅤ 2x All American Pill Poppin Championㅤㅤㅤ ㅤ", "followers_count": 1439, "friends_count": 673, "statues_count": 178748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conroe, TX", "id": "361610d5aa1f6719", "name": "Conroe", "place_type": "city", "bounding_box": rectangle("-95.679001,30.23105 -95.397267,30.443978") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4816432, "cityName": "Conroe" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204200098377728, "text": "Laurent & Clark Residential Project by Menkès Shooner Dagenais LeTourneux Architectes https://t.co/ph1qRVf4UQ https://t.co/FeYdpgaWna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.980904,40.758743"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 16266811, "name": "Dexigner", "screen_name": "dexigner", "lang": "en", "location": "New York", "create_at": date("2008-09-12"), "description": "Dexigner is the leading online portal for designers, architects, illustrators, engineers, artists, and creatives of all kinds.", "followers_count": 237046, "friends_count": 207, "statues_count": 12974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-08T00:00:03.000Z"), "id": 729204200131940353, "text": "@chelsealindsay ��", "in_reply_to_status": 729202095832731650, "in_reply_to_user": 22323495, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 22323495 }}, "user": { "id": 21755319, "name": "Aaron", "screen_name": "Peekaso", "lang": "en", "location": "Detroit", "create_at": date("2009-02-24"), "description": "Father, Conservative, Mortgage guy, Political Strategist for those who listen. Space is good for everyone. Views are my own, but you knew that.", "followers_count": 385, "friends_count": 462, "statues_count": 3997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utica, MI", "id": "7e398c13b8b856bf", "name": "Utica", "place_type": "city", "bounding_box": rectangle("-83.046025,42.616432 -82.993237,42.637574") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26099, "countyName": "Macomb", "cityID": 2681540, "cityName": "Utica" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204200463175680, "text": "A nice buzz sounds good rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419948146, "name": "Kinaa Marie", "screen_name": "ToniMarie117", "lang": "en", "location": "Phx, Arizona ", "create_at": date("2011-11-23"), "description": "20 | Az | GCU | IG: KinaMariie | SC: tonimarie1716", "followers_count": 750, "friends_count": 463, "statues_count": 15125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eloy, AZ", "id": "01ff74c06e1bce70", "name": "Eloy", "place_type": "city", "bounding_box": rectangle("-111.622119,32.734928 -111.541462,32.79749") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 422360, "cityName": "Eloy" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204200563822592, "text": "Wind 8.0 mph SSE. Barometer 29.886 in, Steady. Temperature 66.8 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204200572256256, "text": "@_Ilseeee48_ no quiero", "in_reply_to_status": 729203402421362688, "in_reply_to_user": 2946024115, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2946024115 }}, "user": { "id": 311807134, "name": "Schuyler Sportcenter", "screen_name": "ErikDeArcos", "lang": "en", "location": "Schuyler, NE", "create_at": date("2011-06-05"), "description": "Your #1 source for news about boys and girls soccer for SCHS, Go Warriors!", "followers_count": 1586, "friends_count": 1263, "statues_count": 39605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Schuyler, NE", "id": "b1d1433bac3eb7be", "name": "Schuyler", "place_type": "city", "bounding_box": rectangle("-97.102828,41.435591 -97.038576,41.465264") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31037, "countyName": "Colfax", "cityID": 3144035, "cityName": "Schuyler" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204200727515137, "text": "@hyphyamanda fav KAM song?", "in_reply_to_status": -1, "in_reply_to_user": 788249186, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 788249186 }}, "user": { "id": 1164115974, "name": "☾Michelle☽", "screen_name": "MichelleCCain", "lang": "en", "location": " NYC", "create_at": date("2013-02-09"), "description": "NYC • Leo • Mixed ✨ Snap: Michelleccain", "followers_count": 3118, "friends_count": 196, "statues_count": 32387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indian Trail, NC", "id": "69ae3b3b99c42508", "name": "Indian Trail", "place_type": "city", "bounding_box": rectangle("-80.726731,35.014042 -80.572917,35.114605") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37179, "countyName": "Union", "cityID": 3733560, "cityName": "Indian Trail" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204200920354816, "text": "Just now realizing I did the same pose in all my pics��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1054206235, "name": "♡", "screen_name": "Marisssaaax", "lang": "en", "location": "Morgan Hill, CA ", "create_at": date("2013-01-01"), "description": "LOHS '16", "followers_count": 849, "friends_count": 438, "statues_count": 16478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgan Hill, CA", "id": "e872bcd2497287a7", "name": "Morgan Hill", "place_type": "city", "bounding_box": rectangle("-121.70014,37.088404 -121.583333,37.16931") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649278, "cityName": "Morgan Hill" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204201364951040, "text": "I literally almost got hit by some retard smh.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043571835, "name": "G♛M€Z", "screen_name": "Ohah34", "lang": "en", "location": "null", "create_at": date("2012-12-28"), "description": "Count my blessings mind my business.", "followers_count": 561, "friends_count": 565, "statues_count": 20973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberal, KS", "id": "61fd29934c414a67", "name": "Liberal", "place_type": "city", "bounding_box": rectangle("-100.976544,37.017814 -100.895167,37.070314") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20175, "countyName": "Seward", "cityID": 2039825, "cityName": "Liberal" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204201574649856, "text": "painfully accurate https://t.co/VScKEcXeE7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2230876801, "name": "Xavier Cicero", "screen_name": "Xavier_MichaelC", "lang": "en", "location": "null", "create_at": date("2013-12-04"), "description": "wow", "followers_count": 109, "friends_count": 83, "statues_count": 1778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204201969061888, "text": "The sky's the limit that's what they told the fool, the sky ain't the limit cause now I'm aiming for the sun and moon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3231998035, "name": "Carmen™", "screen_name": "oy_ct", "lang": "en", "location": "Surprise, AZ", "create_at": date("2015-05-31"), "description": "keep it real. wchs", "followers_count": 670, "friends_count": 568, "statues_count": 5480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204202191216640, "text": "@MADBLACKTWINK you just read him the house down", "in_reply_to_status": 729202167807016960, "in_reply_to_user": 1758115766, "favorite_count": 0, "coordinate": point("-81.99989681,37.85284557"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1758115766 }}, "user": { "id": 954041960, "name": "hearthøme", "screen_name": "tynhx", "lang": "en", "location": "| ♑️ | single |", "create_at": date("2012-11-17"), "description": "︎for years I wanted to be older, and now I am.", "followers_count": 834, "friends_count": 386, "statues_count": 9654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Gay-Shamrock, WV", "id": "001538b94df99dec", "name": "Mount Gay-Shamrock", "place_type": "city", "bounding_box": rectangle("-82.04754,37.821601 -81.997016,37.85623") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54045, "countyName": "Logan", "cityID": 5456342, "cityName": "Mount Gay-Shamrock" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204203508371456, "text": "������������ davo is a nut", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1666513957, "name": "✨BigCoHendrix✨", "screen_name": "AWalkingBombb", "lang": "en", "location": "null", "create_at": date("2013-08-12"), "description": "IG: She.TheBomb|Arod.Shellz.Nugget.Shy| #LitDolls✨", "followers_count": 4176, "friends_count": 1903, "statues_count": 94660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tysons Corner, VA", "id": "ca12dbe04543ea95", "name": "Tysons Corner", "place_type": "city", "bounding_box": rectangle("-77.265228,38.898565 -77.196534,38.934957") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5179952, "cityName": "Tysons Corner" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204203512455168, "text": "Wind 0.0 mph SSE. Barometer 29.99 in, Steady. Temperature 53.2 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 28, "friends_count": 112, "statues_count": 14778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204203911032832, "text": "Somebody really just stole my mom car", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2969312464, "name": "slim", "screen_name": "XPrettyV_", "lang": "en", "location": "null", "create_at": date("2015-01-08"), "description": "heart of a hustler admire my ambition ✨ I love you sister -❤️", "followers_count": 1190, "friends_count": 549, "statues_count": 14467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204204179312644, "text": "On that corona slash henny slash smoking hella weed hype", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 25603314, "name": "jackdamackk", "screen_name": "Jackayyye", "lang": "en", "location": "Bay Area, CA.", "create_at": date("2009-03-20"), "description": "Well known when they see me", "followers_count": 354, "friends_count": 169, "statues_count": 56146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Mateo, CA", "id": "432daa3153c5fef9", "name": "San Mateo", "place_type": "city", "bounding_box": rectangle("-122.355038,37.512567 -122.275805,37.592395") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 668252, "cityName": "San Mateo" } }
+{ "create_at": datetime("2016-05-08T00:00:04.000Z"), "id": 729204204384964608, "text": "Outfit from above 3/3\n.\n.\n.\n.\n.\n.\n.\n#vscocam #vsco #vscogood #vscophile #vscogrid #vscodaily… https://t.co/NVg7BYZ7FU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.62308825,53.52268409"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "vscocam", "vsco", "vscogood", "vscophile", "vscogrid", "vscodaily" }}, "user": { "id": 1940044920, "name": "Jerome", "screen_name": "Jrmvd", "lang": "en", "location": "Snapchat: Jrmvd", "create_at": date("2013-10-06"), "description": "@Trshfaye ❤️", "followers_count": 1007, "friends_count": 471, "statues_count": 99080 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204204720390144, "text": "Voy a regresar RODANDO ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 22150327, "name": "Cris", "screen_name": "LaMimadadeCI", "lang": "en", "location": "Tus Besos ❤️", "create_at": date("2009-02-27"), "description": "castaña violeta xq lo común te lo dejo a ti.... Intoxicada de ti❤️", "followers_count": 275, "friends_count": 235, "statues_count": 39576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204204825284608, "text": "Accident, three lanes blocked in #BayBridge on I-80 WB at Bay Brg Toll Plz, stopped traffic back to Bus Ln Split, delay of 14 mins #BayArea", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.3173,37.82319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BayBridge", "BayArea" }}, "user": { "id": 88956948, "name": "TTWN SF Bay Area", "screen_name": "TotalTrafficSF", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-11-10"), "description": "Stuck in a delay in the Bay Area? Tell us about it! Call the TTWN Traffic Tipline: 415 247-4100", "followers_count": 1220, "friends_count": 546, "statues_count": 36014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204204917686275, "text": "R.I.P. Guru", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cs", "is_retweet": false, "user": { "id": 472127253, "name": "JB", "screen_name": "_Jaybles_", "lang": "en", "location": "Vienna, VA", "create_at": date("2012-01-23"), "description": "null", "followers_count": 289, "friends_count": 259, "statues_count": 14505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vienna, VA", "id": "03a542c2b1a18f1d", "name": "Vienna", "place_type": "city", "bounding_box": rectangle("-77.285483,38.87858 -77.241021,38.921834") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5181072, "cityName": "Vienna" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204204934340612, "text": "#DBZKai next on Toonami", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DBZKai" }}, "user": { "id": 2941006921, "name": "Francis Kevin", "screen_name": "FranceletteDev", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2014-12-23"), "description": "Im a gamer, upcoming computer scientist and Game maker", "followers_count": 145, "friends_count": 212, "statues_count": 3854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205034954752, "text": "@MikaelaHannigan go too prom for your senior year ��", "in_reply_to_status": 729202054355243008, "in_reply_to_user": 522671819, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 522671819 }}, "user": { "id": 240023354, "name": "The Average Joe", "screen_name": "JoeV253", "lang": "en", "location": "null", "create_at": date("2011-01-18"), "description": "null", "followers_count": 667, "friends_count": 342, "statues_count": 13573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Graham, WA", "id": "08a26cd724eca51d", "name": "Graham", "place_type": "city", "bounding_box": rectangle("-122.338034,47.016521 -122.206872,47.09723") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5327785, "cityName": "Graham" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205249044480, "text": "man I'm lit af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 168051108, "name": "Nik", "screen_name": "Nikkop15", "lang": "en", "location": "Westwood, MI", "create_at": date("2010-07-18"), "description": "IG:Nikkop7 #WMU19 #LAkers", "followers_count": 1743, "friends_count": 1027, "statues_count": 46003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205303386112, "text": "@reallylaurynn DUH ���� fucking Mother Nature ��", "in_reply_to_status": -1, "in_reply_to_user": 3281139889, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3281139889 }}, "user": { "id": 2314557697, "name": "antonia ❤️✨", "screen_name": "baabytoni", "lang": "en", "location": "510", "create_at": date("2014-01-27"), "description": "overthinking only leads to subtweeting", "followers_count": 567, "friends_count": 572, "statues_count": 7294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205723000833, "text": "Ruffles needs life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248879699, "name": "Dominick Edison", "screen_name": "Dommy_Dukes", "lang": "en", "location": "Broad Channel NY", "create_at": date("2011-02-07"), "description": "PSN: YOUBEENSHUNNED", "followers_count": 195, "friends_count": 199, "statues_count": 7942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205764808705, "text": "I've lost all my faith in people ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1601986602, "name": "• skye •", "screen_name": "Skye_Kaycee", "lang": "en", "location": "null", "create_at": date("2013-07-17"), "description": "| • Life's an adventure • | Vallivue 2016 | DB |", "followers_count": 367, "friends_count": 499, "statues_count": 2538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Caldwell, ID", "id": "7d563a4149988303", "name": "Caldwell", "place_type": "city", "bounding_box": rectangle("-116.713205,43.590179 -116.593171,43.69763") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16027, "countyName": "Canyon", "cityID": 1612250, "cityName": "Caldwell" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205957750784, "text": "Lmao who's next Jeff?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1002823386, "name": "mary", "screen_name": "Maria_4hunnid", "lang": "en", "location": "Ontario, CA", "create_at": date("2012-12-10"), "description": "me gusta cantar", "followers_count": 598, "friends_count": 383, "statues_count": 47409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204205962076160, "text": "@Lee_in_Iowa @gullsimpression @xdelmar59 @rleedep yeah he is got as much chance as I do\nFuckin delusional idiot", "in_reply_to_status": 729103198795878400, "in_reply_to_user": 146160591, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 146160591, 3307295642, 2723153944, 32486821 }}, "user": { "id": 39604861, "name": "franco phillip", "screen_name": "francoddd", "lang": "en", "location": "nyc", "create_at": date("2009-05-12"), "description": "null", "followers_count": 57, "friends_count": 21, "statues_count": 4455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204206066933760, "text": "لا تنزعج كثيراً إذا كنت لا تروق للبعض، فمهما كنت إنسان رائع سيكون هناك من لا يحبك لأسباب لا تعرفها فما يراه الأخرين يعكس حقيقتهم وليس حقيقتك", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2887696117, "name": "☆آميره الورد☆", "screen_name": "Opopg895G", "lang": "ar", "location": "خاص ممنوع ♡", "create_at": date("2014-11-02"), "description": "‏‏‏‏قال رسول الله ﷺ : من قال سبحان الله وبحمده ، في يوم مائة مرة ، حطت خطاياه وإن كانت مثل زبد البحرصدق رسول الله ﷺ", "followers_count": 569, "friends_count": 230, "statues_count": 5956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englewood, CO", "id": "7fdcecb477b5ddd7", "name": "Englewood", "place_type": "city", "bounding_box": rectangle("-105.03495,39.616584 -104.95943,39.678686") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 824785, "cityName": "Englewood" } }
+{ "create_at": datetime("2016-05-08T00:00:05.000Z"), "id": 729204206934970368, "text": "������ https://t.co/rDA2puBECu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1447765100, "name": "GHOST", "screen_name": "TREMUZIC", "lang": "en", "location": "null", "create_at": date("2013-05-21"), "description": "null", "followers_count": 204, "friends_count": 173, "statues_count": 12399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566572063248385, "text": "You over there ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315928436, "name": "パンダ", "screen_name": "Panda_WannaLick", "lang": "en", "location": "null", "create_at": date("2013-03-30"), "description": "null", "followers_count": 132, "friends_count": 81, "statues_count": 177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566572273147904, "text": "Woah wait a minute���� https://t.co/ADKVthaVBP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2670464938, "name": "y a y a", "screen_name": "Ayeeeeyaya", "lang": "en", "location": "Murrland‼️", "create_at": date("2014-07-03"), "description": "#StayHumblePlayCocky", "followers_count": 348, "friends_count": 314, "statues_count": 3094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Severn, MD", "id": "1c98b097b6fcb4d6", "name": "Severn", "place_type": "city", "bounding_box": rectangle("-76.767088,39.094164 -76.642175,39.214142") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2471150, "cityName": "Severn" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566572537352192, "text": "Mangos Cafe ���������������� Outdoor PATIO ����������������No matter what the weather conditions are, you can… https://t.co/66gHHO5i5J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0063171,39.9881401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383010151, "name": "Mango's Cafe", "screen_name": "MangosCafecbus", "lang": "en", "location": "21 Smith Place. Columbus, OHIO", "create_at": date("2011-09-30"), "description": "google us & check out what others have to say 6144293694 http://www.mangoscafe.orgus⬇️ https://www.facebook.com/pages/Mangos-Cafe/108546569254369", "followers_count": 10561, "friends_count": 502, "statues_count": 29747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566572952563713, "text": "Feel like having a deep conversation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1500412051, "name": "valerie", "screen_name": "iamvx3_", "lang": "en", "location": "null", "create_at": date("2013-06-10"), "description": "Gods got this", "followers_count": 790, "friends_count": 440, "statues_count": 31318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566573350936577, "text": "Sundress 2morrow or no? Hmmm.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1928688998, "name": "ROX❤", "screen_name": "_roxcabrera", "lang": "en", "location": "IG: _roxcabrera ", "create_at": date("2013-10-02"), "description": "philippians 4:13 // FWHS'17 CHEER ❤", "followers_count": 577, "friends_count": 560, "statues_count": 2762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566573531275265, "text": "@SethIova @slitherwolf Super adorable!", "in_reply_to_status": 729564744038809600, "in_reply_to_user": 33772385, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 33772385, 2467337228 }}, "user": { "id": 253586864, "name": "Aetius Durr", "screen_name": "Aetius_stronk", "lang": "en", "location": "San Marino, CA", "create_at": date("2011-02-17"), "description": "Arizona Deer exiled in Socal", "followers_count": 825, "friends_count": 352, "statues_count": 12061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marino, CA", "id": "fa6d44142b50522e", "name": "San Marino", "place_type": "city", "bounding_box": rectangle("-118.141937,34.104985 -118.088006,34.137587") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 668224, "cityName": "San Marino" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566573623652352, "text": "Bon Bon? mikebagale @alinea Where'd that mezcal go? @ Alinea https://t.co/4HAI8waOeG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6479899,41.91343"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525613826 }}, "user": { "id": 22412950, "name": "Grant Achatz", "screen_name": "Gachatz", "lang": "en", "location": "null", "create_at": date("2009-03-01"), "description": "Alinea, Next, the Aviary, Roister", "followers_count": 109181, "friends_count": 63, "statues_count": 3177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566574344949761, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3014059723, "name": "Honey", "screen_name": "_oceanna", "lang": "en", "location": "bay area ", "create_at": date("2015-02-08"), "description": "null", "followers_count": 232, "friends_count": 176, "statues_count": 2668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566574579945472, "text": "94 Cup GM 7 if u notice a lot of suits in msg and building is as loud as it ever was", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2997367955, "name": "MikeZaunNumbah1", "screen_name": "WfanMongo109", "lang": "en", "location": "New Jersey", "create_at": date("2015-01-24"), "description": "Yankees Giants Knicks Rangers. WWE. Video Games. Tech. Love sports radio/WFAN. Yuge Francessa Fan. Member of #Mongonation.", "followers_count": 596, "friends_count": 885, "statues_count": 35176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englishtown, NJ", "id": "00fc227e25eae376", "name": "Englishtown", "place_type": "city", "bounding_box": rectangle("-74.379277,40.286363 -74.35297,40.31666") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3421570, "cityName": "Englishtown" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566574625972224, "text": "@ebbtideapp Tide in Babylon, New York 05/09/2016\n Low 7:26am -0.1\nHigh 12:55pm 0.7\n Low 7:29pm -0.1\nHigh 1:12am 0.8\n Low 8:16am -0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.315,40.685"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 83, "friends_count": 1, "statues_count": 31242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3603408, "cityName": "Babylon" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566574861033472, "text": "Ashley https://t.co/29rK3v46mZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3298884014, "name": "DaddyReese✨", "screen_name": "liljuicegawd__", "lang": "en", "location": "DaddysLap ", "create_at": date("2015-07-27"), "description": "null", "followers_count": 589, "friends_count": 440, "statues_count": 7960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566574877802497, "text": "Thank you for all your support. I greatly appreciate it ❤️⚽️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537718331, "name": "Nick", "screen_name": "NickLWells", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "Georgia Southern #19 ⚽️ | @ChelseaFC | @Hali_Carleton ❤️", "followers_count": 974, "friends_count": 667, "statues_count": 24350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockbridge, GA", "id": "013e811145e07117", "name": "Stockbridge", "place_type": "city", "bounding_box": rectangle("-84.293073,33.478656 -84.168228,33.575109") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13151, "countyName": "Henry", "cityID": 1373704, "cityName": "Stockbridge" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566575104155648, "text": "Young scooter ain't even a scooter https://t.co/B0G69wXfNb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120680949, "name": "Totally Kyle", "screen_name": "_KevinFederline", "lang": "en", "location": "At the Ritz Carlton", "create_at": date("2010-03-06"), "description": "Father | Husband | Sigma | Alum | Former Alderman for the City of Harvey | RIP Jackie", "followers_count": 2369, "friends_count": 1370, "statues_count": 177644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566575305560066, "text": "@LareesaF WHAAAAAAT OH MY GOSH I AM SO SORRY \n\n(But with a hint of happiness because I'm a selfish bish)", "in_reply_to_status": 729566330592985088, "in_reply_to_user": 1414173422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1414173422 }}, "user": { "id": 705766219, "name": "macy moon", "screen_name": "macywindu", "lang": "en", "location": "Bluffton, IN", "create_at": date("2012-07-19"), "description": "I don't carrot all.", "followers_count": 466, "friends_count": 224, "statues_count": 16363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-05-09T00:00:00.000Z"), "id": 729566575557218304, "text": "Temp 44.2° Hi/Lo 46.2/42.7 Rng 3.5° WC 44.2° Hmd 72% Rain 0.00\" Storm 0.00\" BAR 29.879 Rising DP 35.7° Wnd 0mph Dir --- Gst 8mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 122, "statues_count": 18545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576010092545, "text": "My hair is getting hella long. I be playing with that shit 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173477540, "name": "Ultralightskin Beam", "screen_name": "ImaRapperDog", "lang": "en", "location": "null", "create_at": date("2013-02-12"), "description": "Colorado nigga somehow in Oregon. UO 2019", "followers_count": 456, "friends_count": 744, "statues_count": 12893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576068993024, "text": "@MattG_21 your a terrible person", "in_reply_to_status": -1, "in_reply_to_user": 25790419, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25790419 }}, "user": { "id": 35287841, "name": "The Milk Man", "screen_name": "MMahoney15", "lang": "en", "location": "ÜT: 41.652232,-70.308266", "create_at": date("2009-04-25"), "description": "null", "followers_count": 107, "friends_count": 512, "statues_count": 564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Barnstable Town, MA", "id": "072afef2aa2f1bf5", "name": "Barnstable Town", "place_type": "city", "bounding_box": rectangle("-70.467597,41.599797 -70.259465,41.735676") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25001, "countyName": "Barnstable", "cityID": 2503690, "cityName": "Barnstable Town" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576484057089, "text": "\"Simple mafmatics\"-@sassypants15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 2189487344 }}, "user": { "id": 1061724164, "name": "Dūštïñ Łêmå✌", "screen_name": "LemaDustin", "lang": "en", "location": "working", "create_at": date("2013-01-04"), "description": "wrestler. 19. snapchat-dustin_lema", "followers_count": 1158, "friends_count": 655, "statues_count": 41691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576630861824, "text": "SO EXCITED https://t.co/OmAdt6fi8E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 152878735, "name": "dr. yang", "screen_name": "CSedona", "lang": "en", "location": "are you my gunbitch or what ", "create_at": date("2010-06-06"), "description": "like the force my star wars trash self has awoken", "followers_count": 778, "friends_count": 315, "statues_count": 26372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576656125952, "text": "@KingCopeee that's definitely not gonna happen. I can't fall asleep in 2 mins", "in_reply_to_status": 729566436654350336, "in_reply_to_user": 2873508556, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2873508556 }}, "user": { "id": 3293390093, "name": "Sumi ☾", "screen_name": "Sincereful", "lang": "en", "location": "Meltful", "create_at": date("2015-05-21"), "description": "SnD | Double major at ISU | @Songfully_sB ♡", "followers_count": 1777, "friends_count": 342, "statues_count": 8463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valrico, FL", "id": "76d195c1204e61b2", "name": "Valrico", "place_type": "city", "bounding_box": rectangle("-82.252969,27.872693 -82.18382,27.990667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1273700, "cityName": "Valrico" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566576995930113, "text": "@uronIyone https://t.co/ItaQ1ar9rS", "in_reply_to_status": -1, "in_reply_to_user": 390367079, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 390367079 }}, "user": { "id": 4369168213, "name": "#RespeckRio", "screen_name": "shze305", "lang": "en", "location": "Miami, FL", "create_at": date("2015-11-26"), "description": "I'm #HeGone Recruiter, Mario Chalmers of Twitter, and I'm for #HeatNation #HeatTwitter #TeamPetty #HeGone #Barça", "followers_count": 614, "friends_count": 356, "statues_count": 24765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pompano Beach, FL", "id": "fa8f8f24dc772cc0", "name": "Pompano Beach", "place_type": "city", "bounding_box": rectangle("-80.195561,26.206136 -80.08173,26.297654") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1258050, "cityName": "Pompano Beach" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566577159507968, "text": "#demthrones is now trending in #SF https://t.co/WIrIJYsd76", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "demthrones", "SF" }}, "user": { "id": 123791259, "name": "Trendsmap SF", "screen_name": "TrendsSF", "lang": "en", "location": "San Francisco", "create_at": date("2010-03-16"), "description": "Real-Time Local Twitter Trends for San Francisco", "followers_count": 655, "friends_count": 198, "statues_count": 27638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566577822187520, "text": "Ripley SW Limestone Co. Temp: 69.1°F Wind:3.8mph Pressure: 994.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566578149232641, "text": "My mindset is too advanced for people to handle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44492061, "name": "Jamie ✈️⛽️", "screen_name": "Jamie_Is_Back", "lang": "en", "location": "In the clouds☁️", "create_at": date("2009-06-03"), "description": "#UCA19|| IG: Jamie_is_back | Multiracial", "followers_count": 1930, "friends_count": 1575, "statues_count": 23023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado, AR", "id": "7d06ebf57d20083f", "name": "El Dorado", "place_type": "city", "bounding_box": rectangle("-92.725914,33.161707 -92.60577,33.26595") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5139, "countyName": "Union", "cityID": 521070, "cityName": "El Dorado" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566578522488832, "text": "What's step 2?��☕️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 926035411, "name": "De Jota", "screen_name": "daisy_dukes_dj2", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-11-04"), "description": "You is kind. You is smart. You is important.", "followers_count": 1024, "friends_count": 1046, "statues_count": 22183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566578614915072, "text": "Wind 2.0 mph ENE. Barometer 29.955 in, Steady. Temperature 56.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566578925166592, "text": "@ShesMoistAF_ we some bitches", "in_reply_to_status": 729566026510172160, "in_reply_to_user": 454524172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 454524172 }}, "user": { "id": 599752702, "name": "Doomsday", "screen_name": "MellowBrian", "lang": "en", "location": "null", "create_at": date("2012-06-04"), "description": "null", "followers_count": 473, "friends_count": 307, "statues_count": 19342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566579239702528, "text": "You can tell when a girls trying to get someone's attention lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 425569323, "name": "Alexa Schmitt", "screen_name": "lexschmitt_", "lang": "en", "location": "null", "create_at": date("2011-11-30"), "description": "laters baby", "followers_count": 1105, "friends_count": 528, "statues_count": 26743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566579596374016, "text": "Wind 1.0 mph ESE. Barometer 29.793 in, Falling. Temperature 65.7 °F. Rain today 0.03 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-09T00:00:01.000Z"), "id": 729566579671715840, "text": "Lucky you https://t.co/73dl2QsdNr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331916965, "name": "#FreebandzTalibanCEO", "screen_name": "BalloutWebbie", "lang": "en", "location": "BALLOUTWORLD w/ Money & Nem", "create_at": date("2011-07-08"), "description": "Free The Guyz Rip The Guyz #SSD #NLMB #OTF #2FFE #YFNBC #MurdaTeam", "followers_count": 1424, "friends_count": 1165, "statues_count": 127089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566580338774016, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Falling slowly. Temperature 65.8 °F. Rain today 0.00 in. Humidity 71%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566580380569602, "text": "���� expose him", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4040129999, "name": "Trife", "screen_name": "jayleneli", "lang": "en", "location": "On A Wave", "create_at": date("2015-10-25"), "description": "if your on my page and don't follow me I hope the zippers on all your jackets get stuck", "followers_count": 225, "friends_count": 214, "statues_count": 1266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muscle Shoals, AL", "id": "74d460734c3eca69", "name": "Muscle Shoals", "place_type": "city", "bounding_box": rectangle("-87.680639,34.705252 -87.617362,34.762527") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1033, "countyName": "Colbert", "cityID": 153016, "cityName": "Muscle Shoals" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566580468678659, "text": "Temp: 62.6°F Wind:0.0mph Pressure: 29.995hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566581324316673, "text": "@FlashSimeon23 @Its_Beautyy Yeah you tweet watching, give us credit for tweeting that website", "in_reply_to_status": 729566387228704769, "in_reply_to_user": 55353937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 55353937, 79834715 }}, "user": { "id": 255728670, "name": "FatMama", "screen_name": "FatimaAbdal", "lang": "en", "location": "Sioux Falls, SD - Texas ✈️", "create_at": date("2011-02-21"), "description": "Dream as if you'll live forever, live as if you'll die today. #Egyptian #UNT.", "followers_count": 1451, "friends_count": 969, "statues_count": 27285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566581391425536, "text": "YES���������� https://t.co/9s4aJz1aoc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 371972395, "name": "kim", "screen_name": "kimberlyramos8", "lang": "en", "location": "Pecos, TX", "create_at": date("2011-09-11"), "description": "freshman.", "followers_count": 366, "friends_count": 211, "statues_count": 1579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pecos, TX", "id": "4c21307b2c96ec2b", "name": "Pecos", "place_type": "city", "bounding_box": rectangle("-103.54106,31.388995 -103.476494,31.441411") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48389, "countyName": "Reeves", "cityID": 4856516, "cityName": "Pecos" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566581425057792, "text": "Best Drake song? Go", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2379302060, "name": "BC", "screen_name": "bclinks17", "lang": "en", "location": "Oakland, CA", "create_at": date("2014-03-08"), "description": "CCSF DB #citysituation Fire & Desire @DRAKE", "followers_count": 815, "friends_count": 579, "statues_count": 12018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566581672452096, "text": "�������� https://t.co/2MtkjG2mCV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3699036914, "name": "•Raybeezkta•", "screen_name": "Raybeezkta", "lang": "en", "location": "Roswell, NM", "create_at": date("2015-09-26"), "description": "@faithbarelaa//Dancer, MLG , IG/Snap:Raybeezkta. I know @JustJet__ //27Junkies Clothing L.A. California //FunnyBonesCrew//#FREESUSIEQ", "followers_count": 240, "friends_count": 237, "statues_count": 2543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roswell, NM", "id": "cf4827d75b2f7aed", "name": "Roswell", "place_type": "city", "bounding_box": rectangle("-104.599766,33.280185 -104.473262,33.45249") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35005, "countyName": "Chaves", "cityID": 3564930, "cityName": "Roswell" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566581911478272, "text": "68.2F (Feels: 68.2F) - Humidity: 85% - Wind: 0.0mph --- - Gust: 6.9mph - Pressure: 1012.6mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566582062505984, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":58.5°F Wind:2.7mph Pressure: 29.97hpa Falling slowly Rain Today 0.00in. Forecast: Precipitation at", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 85, "friends_count": 17, "statues_count": 317412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-09T00:00:02.000Z"), "id": 729566583111098368, "text": "lmao remembering when me & johanna would go pool hopping and text each other to bring extra clothes ���� summer 2k15 was lit lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1404123042, "name": "lesly", "screen_name": "__leslyx", "lang": "en", "location": "316", "create_at": date("2013-05-04"), "description": "null", "followers_count": 1587, "friends_count": 1275, "statues_count": 18885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566584210132993, "text": "Wind 0.0 mph WSW. Barometer 29.933 in, Steady. Temperature 66.2 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566584620994562, "text": "touchdown Las Vegas! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 94985932, "name": "julia.✨", "screen_name": "juliaaacol", "lang": "en", "location": "crazy weather,MNL", "create_at": date("2009-12-06"), "description": "travelling,working,and thriving l IG: juliacoleen | Snapchat: juliapetite", "followers_count": 256, "friends_count": 240, "statues_count": 11912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566584797286400, "text": "Wind 0.0 mph W. Barometer 29.88 in, Steady. Temperature 43.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566585086578690, "text": ":)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1644332852, "name": "Chris Moon", "screen_name": "chrismoonedyou", "lang": "en", "location": "null", "create_at": date("2013-08-03"), "description": "null", "followers_count": 492, "friends_count": 401, "statues_count": 8296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566585132703744, "text": "Happy C day princess @thatboisergi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 341958653 }}, "user": { "id": 1947973302, "name": "aguacate", "screen_name": "twotimevaleria", "lang": "en", "location": "gringolandia", "create_at": date("2013-10-08"), "description": "null", "followers_count": 254, "friends_count": 223, "statues_count": 6625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566585451503616, "text": "Yo bett'ya enjoy dis ride 'cuz it's nt goin' 2 last dat long. �������� #fuccboi #niggasaintshit #FightforFamilies #FightLikeHell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fuccboi", "niggasaintshit", "FightforFamilies", "FightLikeHell" }}, "user": { "id": 2823703922, "name": "Chelsey A. Williams", "screen_name": "LilCtasty", "lang": "en", "location": "Compton, CA", "create_at": date("2014-09-21"), "description": "Single mother w/ 5 kids. Eye work 3 jobs & eye try 2 spin tyme w/ mi kids as much as possible.", "followers_count": 25, "friends_count": 26, "statues_count": 79 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566585745084418, "text": "YES https://t.co/k49o1GqQcO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2345680502, "name": "Sayoncé", "screen_name": "SamaraSamone", "lang": "en", "location": "who gon check me boo?", "create_at": date("2014-02-15"), "description": "if it ain't Bey , it ain't me", "followers_count": 1575, "friends_count": 1971, "statues_count": 43349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566585992536064, "text": "That's my fucking money not that cunts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1564957980, "name": "Appolumi", "screen_name": "Illumeye", "lang": "en", "location": "PlutĆharon", "create_at": date("2013-07-02"), "description": "Vive La #fR∀nÇe #TuRkeʎ #ÇhEʎen∀ #EʎeR∀q #EʎeR∀n #Russì∀ #BrEʎeŽe∀l #N.S.KṓrEʎe∀ #Jap∀n Handle #Tex∀s‘’ΐĪίΐίjίΐĪΐ‘’ι[ΐΐίjιϊϊÎιΙ", "followers_count": 31, "friends_count": 18, "statues_count": 7911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview, TX", "id": "506bc3b13b02edd6", "name": "Fairview", "place_type": "city", "bounding_box": rectangle("-96.659709,33.122254 -96.572672,33.165515") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4825224, "cityName": "Fairview" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566586542116864, "text": "Photo credit - Rachel Kutz @ Simons After Dark https://t.co/1WSWiPhlPK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.2003,42.2665099"), "retweet_count": 0, "lang": "da", "is_retweet": false, "user": { "id": 502171622, "name": "Bobby Knucklez", "screen_name": "BobbyKnucklez", "lang": "en", "location": "Lansing, MI", "create_at": date("2012-02-24"), "description": "null", "followers_count": 444, "friends_count": 740, "statues_count": 1441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen Park, MI", "id": "0c937cb917334546", "name": "Allen Park", "place_type": "city", "bounding_box": rectangle("-83.231897,42.222765 -83.183819,42.301134") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2601380, "cityName": "Allen Park" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566586617503744, "text": "Pandora station to put me to sleep? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 928916450, "name": ".", "screen_name": "trickeelobos", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "Instagram: trickeevillalobos /sc lobos18", "followers_count": 271, "friends_count": 224, "statues_count": 10019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566586760220672, "text": "Style on Nadal, I've been serving ya'll lately", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163147855, "name": "David Shiminsky", "screen_name": "DavidShiminsky", "lang": "en", "location": "Canton, MI", "create_at": date("2010-07-05"), "description": "Student at Specs Howard || Writer for @thedailygems || Designer/Creative for Taliban || Reality is wrong. Dreams are for real! || shiminskymanage@gmail.com ||", "followers_count": 3304, "friends_count": 1587, "statues_count": 19231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587053727744, "text": "They be having a whole new set of problems and relationships Lmfaooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2330050902, "name": "j a s s y", "screen_name": "jassywasssy", "lang": "en", "location": "Compton ", "create_at": date("2014-02-06"), "description": "null", "followers_count": 354, "friends_count": 295, "statues_count": 9713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587108401152, "text": "@megganoel text me via seance", "in_reply_to_status": 729566460964700160, "in_reply_to_user": 523798618, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 523798618 }}, "user": { "id": 807852474, "name": "helllly", "screen_name": "haywo0odjablomi", "lang": "en", "location": "null", "create_at": date("2012-09-06"), "description": "one day u will die somehow and somethings gonna steal ur carbon ~ AWB", "followers_count": 410, "friends_count": 278, "statues_count": 18451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enochville, NC", "id": "ef4da7ba8b6dcb2e", "name": "Enochville", "place_type": "city", "bounding_box": rectangle("-80.736519,35.494885 -80.644089,35.54821") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37159, "countyName": "Rowan", "cityID": 3721500, "cityName": "Enochville" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587154374657, "text": "WHY DO I DO THIS TO MYSELF IM SO EXCITED", "in_reply_to_status": 729566483185950721, "in_reply_to_user": 2190913364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2190913364, "name": "champagne spice", "screen_name": "mrsstjd", "lang": "en", "location": "CA", "create_at": date("2013-11-12"), "description": "oh how the turn tables", "followers_count": 93, "friends_count": 98, "statues_count": 7660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587414523904, "text": "I don't necessarily have a favorite one because there's too many. https://t.co/AIfDtuZIlG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 856940612, "name": "Mary", "screen_name": "marrrybbb", "lang": "en", "location": "Orlando, FL", "create_at": date("2012-10-01"), "description": "SC: Recklesslymary IG: marrrybbb", "followers_count": 908, "friends_count": 328, "statues_count": 42615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Altamonte Springs, FL", "id": "e40595796da14879", "name": "Altamonte Springs", "place_type": "city", "bounding_box": rectangle("-81.443607,28.639994 -81.343699,28.689613") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1200950, "cityName": "Altamonte Springs" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587473158145, "text": "Try https://t.co/mrQhZYwVJq All-natural grilled chicken breast, artichoke hearts, chickpeas, roasted red #grilled #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.59066981,38.21461004"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "grilled", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4404, "friends_count": 874, "statues_count": 426790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jeffersontown, KY", "id": "258c7428a48a8cfa", "name": "Jeffersontown", "place_type": "city", "bounding_box": rectangle("-85.60709,38.143069 -85.508139,38.246418") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2140222, "cityName": "Jeffersontown" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587615739905, "text": "I just gotta say I luv the artists I'm working with...they're so talented and great to work but they've also become my really close friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1058740741, "name": "jessica ekman", "screen_name": "jayekman", "lang": "en", "location": "LA", "create_at": date("2013-01-03"), "description": "artist mgmt for @descendersound @iamgunkst @am__138 | @estenclub | @uf grad | @napgirls | formerly @nesthq @starkprofilespr @famehouse", "followers_count": 1454, "friends_count": 800, "statues_count": 12448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566587854823424, "text": "Real nigga to a fake nigga yous a real bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2443522980, "name": "#BLM", "screen_name": "briabriabb", "lang": "en", "location": "LA — s/o OAKLAND ", "create_at": date("2014-04-14"), "description": "SDSU '17 Health Comm|Public Policy — woman on her path. God always, listener, observer, + pro-liberation/socialism. Aztec Diamonds' Secretary!", "followers_count": 610, "friends_count": 320, "statues_count": 14668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566588060323840, "text": "@sonnyboysbbq https://t.co/30ISgTps3A Chicken, Pulled Pork, Beef Brisket, #Chicken #menu", "in_reply_to_status": -1, "in_reply_to_user": 281348760, "favorite_count": 0, "coordinate": point("-113.061444,37.679686"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Chicken", "menu" }}, "user_mentions": {{ 281348760 }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4404, "friends_count": 874, "statues_count": 426790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar City, UT", "id": "013884df639610f8", "name": "Cedar City", "place_type": "city", "bounding_box": rectangle("-113.152491,37.647433 -113.032184,37.736012") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49021, "countyName": "Iron", "cityID": 4911320, "cityName": "Cedar City" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566588131643392, "text": "Since a lot of people replied to this and said my cat is cute, here you go https://t.co/Z0g0WJqn8s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2428439804, "name": "Ed Boy", "screen_name": "j_beirswaggen", "lang": "en", "location": "null", "create_at": date("2014-04-04"), "description": "my name is actually Jesse Beirwagen (beer-wagon). my favorite color is light tan. drums in @afterhoursOR", "followers_count": 239, "friends_count": 245, "statues_count": 3797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Happy Valley, OR", "id": "00bda52c7eddb627", "name": "Happy Valley", "place_type": "city", "bounding_box": rectangle("-122.581184,45.40302 -122.477198,45.461537") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4132050, "cityName": "Happy Valley" } }
+{ "create_at": datetime("2016-05-09T00:00:03.000Z"), "id": 729566588169498628, "text": "Mind is full of thoughts rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 443054606, "name": "Sarah Margaret", "screen_name": "SarahMargaret14", "lang": "en", "location": "null", "create_at": date("2011-12-21"), "description": "2 time state champion-- milly born & raised |UGA'20| #godawgs #memberofthe6", "followers_count": 748, "friends_count": 660, "statues_count": 8349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milledgeville, GA", "id": "accbd86fb69a0b88", "name": "Milledgeville", "place_type": "city", "bounding_box": rectangle("-83.309808,33.01887 -83.19672,33.189701") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13009, "countyName": "Baldwin", "cityID": 1351492, "cityName": "Milledgeville" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566588400107521, "text": "My pups said that I gotta go to work tomorrow :( me too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30828502, "name": "AARON AGUIRRe", "screen_name": "Atres_", "lang": "en", "location": "null", "create_at": date("2009-04-13"), "description": "all tweets are presented by @kodak", "followers_count": 229, "friends_count": 136, "statues_count": 17845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566588446244865, "text": "Why am I still awake��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2167561705, "name": "alejandrah", "screen_name": "_alejandrahhh_", "lang": "en", "location": "null", "create_at": date("2013-10-31"), "description": "null", "followers_count": 393, "friends_count": 301, "statues_count": 6490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566588710461441, "text": "@knc_carroll me", "in_reply_to_status": 729525478055235584, "in_reply_to_user": 55684450, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 55684450 }}, "user": { "id": 363850045, "name": "Alex Piper", "screen_name": "alexrpiper", "lang": "en", "location": "Seattle, Washington", "create_at": date("2011-08-28"), "description": "My jokes aren't funny", "followers_count": 167, "friends_count": 235, "statues_count": 1401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirkland, WA", "id": "ba4471e5bc687736", "name": "Kirkland", "place_type": "city", "bounding_box": rectangle("-122.240397,47.642154 -122.163413,47.718484") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335940, "cityName": "Kirkland" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566588802752512, "text": "https://t.co/koERLm02O0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1491465864, "name": "Casey Speer", "screen_name": "1caseyspeer", "lang": "en", "location": "USA", "create_at": date("2013-06-07"), "description": "American record producer/songwriter 89mgmt@gmail.com Snapchat- Caseyspeermusic", "followers_count": 1276, "friends_count": 655, "statues_count": 1396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566589784219648, "text": "Cornell pillow sucks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350622697, "name": "Mara Peterson", "screen_name": "mara_peterson", "lang": "en", "location": "null", "create_at": date("2011-08-07"), "description": "null", "followers_count": 214, "friends_count": 168, "statues_count": 8796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590040035328, "text": "Wind 0.0 mph ---. Barometer 29.846 in, Rising slowly. Temperature 65.5 °F. Rain today 0.07 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590082027521, "text": "What's the move? https://t.co/SgoSuy9ri4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2690625654, "name": "EB", "screen_name": "evanshoots", "lang": "en", "location": "Chicago", "create_at": date("2014-07-29"), "description": "Consulting | Photography | Journalism ejbshoots@gmail.com IG: @EvanShoots", "followers_count": 424, "friends_count": 399, "statues_count": 5450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590396715008, "text": "Allah save whoever did that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 127762561, "name": "Idreese", "screen_name": "IdreeseOfficial", "lang": "en", "location": "Houston-Dubai", "create_at": date("2010-03-29"), "description": "Official Twitter of Idreese #TheAmericanArab, Musician, Scandalous Horse", "followers_count": 331, "friends_count": 5, "statues_count": 32823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Land, TX", "id": "7a41192a2879ee24", "name": "Sugar Land", "place_type": "city", "bounding_box": rectangle("-95.686106,29.543372 -95.577273,29.663556") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4870808, "cityName": "Sugar Land" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590610644992, "text": "Why do girls in my hs act baddie in Instagram but they're act like annoying little girl inreal life? It doesn't even make sense tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 552413868, "name": "Katrina H", "screen_name": "xokatrinabh", "lang": "en", "location": "New York City ", "create_at": date("2012-04-12"), "description": "Instagram & Snapchat: xokatrinaheiff", "followers_count": 1788, "friends_count": 680, "statues_count": 15341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590677585921, "text": "fuck it im so tired dkslsks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1548972060, "name": "amanda", "screen_name": "aureatezm", "lang": "en", "location": "null", "create_at": date("2013-06-26"), "description": "jaegyeom & horchata enthusiast", "followers_count": 845, "friends_count": 399, "statues_count": 12318 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566590933606401, "text": "Who is the drunkest", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251962696, "name": "Keef Puchalski ♛", "screen_name": "KidTank", "lang": "en", "location": "Killadelphia", "create_at": date("2011-02-13"), "description": "PGW", "followers_count": 724, "friends_count": 49, "statues_count": 10407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566591088664580, "text": "worried about THE W R O N G S H I T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3193830380, "name": "savannah", "screen_name": "HacketSavannah", "lang": "en", "location": "null", "create_at": date("2015-05-12"), "description": "out the way...", "followers_count": 577, "friends_count": 282, "statues_count": 3902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566591206100992, "text": "HAPPY BIRTHDAY STEPHY I love you ���� #19 #throwbacks https://t.co/ti7ptoYWWB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "throwbacks" }}, "user": { "id": 121352994, "name": "♡Alina Tenderich♡", "screen_name": "alinatenderich", "lang": "en", "location": "Millbrae, CA", "create_at": date("2010-03-08"), "description": "Bier macht schön", "followers_count": 538, "friends_count": 458, "statues_count": 11656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millbrae, CA", "id": "272f29aa61fa05d3", "name": "Millbrae", "place_type": "city", "bounding_box": rectangle("-122.425557,37.581956 -122.374702,37.614993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 647486, "cityName": "Millbrae" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566591319318528, "text": "Damn me and my girl having a deep ass conversation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3112022450, "name": "UZI++", "screen_name": "Young_Sinatra_3", "lang": "en", "location": "Houston, TX", "create_at": date("2015-03-27"), "description": "@marlene_5169 ❤️", "followers_count": 398, "friends_count": 328, "statues_count": 7507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566591361253377, "text": "Wind 0.0 mph SSE. Barometer 29.99 in, Steady. Temperature 55.4 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 27, "friends_count": 112, "statues_count": 14802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566592015552512, "text": "@devscas hell yeah hell yeah hell yeah", "in_reply_to_status": 729566437761794048, "in_reply_to_user": 1436274818, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1436274818 }}, "user": { "id": 76771648, "name": "tré.", "screen_name": "trewest_", "lang": "en", "location": "☁️", "create_at": date("2009-09-23"), "description": "null", "followers_count": 981, "friends_count": 589, "statues_count": 20266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slidell, LA", "id": "44ee4fb6071b7eaa", "name": "Slidell", "place_type": "city", "bounding_box": rectangle("-89.853527,30.235705 -89.677511,30.36002") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2270805, "cityName": "Slidell" } }
+{ "create_at": datetime("2016-05-09T00:00:04.000Z"), "id": 729566592170856448, "text": "Whhhy?! https://t.co/RR1rGbFnx7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2779187462, "name": "Sayvion", "screen_name": "beefyandtatted", "lang": "en", "location": "Baltimore, MD", "create_at": date("2014-08-29"), "description": "Virgo. BEARd Gang. Sexy Nerd. Writer. Friend. Lover. C.R.E.A.M (Christ Rules Everything Around Me) @CBSDaytime Fan for over 25 yrs.", "followers_count": 298, "friends_count": 378, "statues_count": 32385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansdowne, MD", "id": "01eb7da953d9b395", "name": "Lansdowne", "place_type": "city", "bounding_box": rectangle("-76.693922,39.219095 -76.618616,39.255518") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2445650, "cityName": "Lansdowne" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566593185783808, "text": "Don't listen to me yall. https://t.co/x2k3Pdso7M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46797656, "name": "Daniel ⚽️ [SDMN]", "screen_name": "danny_lepage", "lang": "en", "location": "Kyle, TX", "create_at": date("2009-06-12"), "description": "null", "followers_count": 447, "friends_count": 464, "statues_count": 32846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kyle, TX", "id": "94ef42cc204d5195", "name": "Kyle", "place_type": "city", "bounding_box": rectangle("-97.896063,29.942852 -97.830853,30.035633") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4839952, "cityName": "Kyle" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566593320026112, "text": "@dxlinquxnt_3 dewww itttt", "in_reply_to_status": 729566540618588160, "in_reply_to_user": 1022878423, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1022878423 }}, "user": { "id": 2191748923, "name": "Yusuke Urameshi", "screen_name": "watchoutcrisss", "lang": "en", "location": "Bontana, Ca", "create_at": date("2013-11-12"), "description": "are you good though ?", "followers_count": 699, "friends_count": 457, "statues_count": 39203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566593324187649, "text": "@avloko714 @marcosel_nopal I mean you never asked for it so I never bothered", "in_reply_to_status": 729565870498840576, "in_reply_to_user": 2283465566, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2283465566, 4831422767 }}, "user": { "id": 2181863124, "name": "Chris Rod", "screen_name": "Saint_nectar", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "Deliver us serenity IG: @elcxnnect", "followers_count": 289, "friends_count": 228, "statues_count": 1346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Placentia, CA", "id": "ef74afb7ccba74d6", "name": "Placentia", "place_type": "city", "bounding_box": rectangle("-117.884864,33.853755 -117.816285,33.91009") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 657526, "cityName": "Placentia" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566593794101249, "text": "@noLKrue what the fuck is this??", "in_reply_to_status": 729565359326564352, "in_reply_to_user": 1619840352, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1619840352 }}, "user": { "id": 409555666, "name": "Aaron XiU", "screen_name": "KaVXIU", "lang": "en", "location": "VA", "create_at": date("2011-11-10"), "description": "bo3 earning:13.35", "followers_count": 1317, "friends_count": 637, "statues_count": 12457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Farm, VA", "id": "00849edf1301b28f", "name": "Franklin Farm", "place_type": "city", "bounding_box": rectangle("-77.421527,38.894064 -77.361754,38.939907") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5129628, "cityName": "Franklin Farm" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566594481917952, "text": "I'm tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249515473, "name": "Nunu stack$❄️", "screen_name": "SydddNicoleee", "lang": "en", "location": "views from the hood", "create_at": date("2011-02-08"), "description": "dumb rich #ℓσиgℓινє∂αкαяιє", "followers_count": 1310, "friends_count": 1341, "statues_count": 76881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566594729271296, "text": "@DrrakeTheType mess. but true.", "in_reply_to_status": 659775683346964480, "in_reply_to_user": 26767066, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26767066 }}, "user": { "id": 447309963, "name": "Sam", "screen_name": "This__Emotion", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-26"), "description": "20 | Aquarius | this is my R E V I V A L ...", "followers_count": 244, "friends_count": 239, "statues_count": 18728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566595090014208, "text": "Yes https://t.co/nUvN1CS2ta", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 393121763, "name": "Gopi", "screen_name": "gopilicious8", "lang": "en", "location": "Houston, TX", "create_at": date("2011-10-17"), "description": "I pop bottles because I bottle my emotions. My bestie is on the header.", "followers_count": 262, "friends_count": 266, "statues_count": 10330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566595241127937, "text": "A Pit https://t.co/rnbt0MEP8G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 833966947, "name": "Big Rob", "screen_name": "BallIsLife_3_", "lang": "en", "location": "Durham, NC ", "create_at": date("2012-09-19"), "description": "Add Me on Snapchat, foreign-born ... #NCCU 5️⃣8️⃣5️⃣ ✈️ 9️⃣1️⃣9️⃣", "followers_count": 413, "friends_count": 314, "statues_count": 18819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566595492642816, "text": "Proud dog mom https://t.co/k6knfM3dsY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1695990727, "name": "Audrey White", "screen_name": "audreyyymwhite", "lang": "en", "location": "Arizona ☀️", "create_at": date("2013-08-24"), "description": "✌️✂️❤️", "followers_count": 162, "friends_count": 196, "statues_count": 590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566595765313536, "text": "#northparkobservatory #nothinkingallowed #graffiti #northpark #socal #sandiego @ The Observatory… https://t.co/Gidq5Qc3pM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.131294,32.748013"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "northparkobservatory", "nothinkingallowed", "graffiti", "northpark", "socal", "sandiego" }}, "user": { "id": 16438649, "name": "Mod City Mom", "screen_name": "ModCityMom", "lang": "en", "location": "SoCal, STL, & Elsewhere", "create_at": date("2008-09-24"), "description": "Adventure. Style. Gastronomy. Modern Ideas for the Urban Mom.", "followers_count": 220, "friends_count": 381, "statues_count": 6049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-09T00:00:05.000Z"), "id": 729566596235059200, "text": "#chefchefiam crab legs lobster oyster sauce plum sauce Old Bay seasoning more seasoning garlic butter my seasoning https://t.co/I5dMd6vKOn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "chefchefiam" }}, "user": { "id": 719706158764802048, "name": "#chefchefiam", "screen_name": "MarcoCain4", "lang": "en", "location": "Corona, CA", "create_at": date("2016-04-11"), "description": "NFL NBA College Basketball College Football Music Hip-Hop/ Political News R&B/Soul TV Stars Actors & Actresses hard jazz soft jazz 94.7 wave God first 88.1 jazz", "followers_count": 632, "friends_count": 2102, "statues_count": 800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566596797190145, "text": "@skye_aa @AndrewLagunas @ryanaguilar22 Prince Track Ft Kodak Black & Ganja Banks - D.F.T.M Performance Video https://t.co/7ylYZVqi8V", "in_reply_to_status": -1, "in_reply_to_user": 581371612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 581371612, 420123478, 998757547 }}, "user": { "id": 52934921, "name": "Trackmakers", "screen_name": "GoldenTicket954", "lang": "en", "location": "SOUTH FLORIDA ", "create_at": date("2009-07-01"), "description": "South Florida’s Golden Ticket, Prince Track, has come quite a ways from writing rhymes in his notebook and rapping with his friends at the age of 16.", "followers_count": 2260, "friends_count": 1124, "statues_count": 55753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise, FL", "id": "14441682ff5a9fa3", "name": "Sunrise", "place_type": "city", "bounding_box": rectangle("-80.363112,26.119383 -80.226673,26.193854") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1269700, "cityName": "Sunrise" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597103280129, "text": "Damn I been sleep, windows 10. The last one I saw was windows 8 and I doesn't feel like it was that long ago", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 977169732, "name": "ElevatedElement", "screen_name": "GlennBrown53", "lang": "en", "location": "null", "create_at": date("2012-11-28"), "description": "null", "followers_count": 33, "friends_count": 149, "statues_count": 366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597149425664, "text": "@epicsonnn I still love you tho.", "in_reply_to_status": 729538876163776512, "in_reply_to_user": 2713892174, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2713892174 }}, "user": { "id": 86591700, "name": "Denne Tiengthong", "screen_name": "dennexd", "lang": "en", "location": "null", "create_at": date("2009-10-31"), "description": "Do eggs????", "followers_count": 138, "friends_count": 82, "statues_count": 2054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597770149889, "text": "Mfs thirst for attention so much so that they can't even live they lives the way they want", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716900334900129792, "name": "His..✨", "screen_name": "melaniniia", "lang": "en", "location": "loading...", "create_at": date("2016-04-04"), "description": "Queen Status. My babe is the goat.❤️", "followers_count": 329, "friends_count": 250, "statues_count": 2951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Dorado, AR", "id": "7d06ebf57d20083f", "name": "El Dorado", "place_type": "city", "bounding_box": rectangle("-92.725914,33.161707 -92.60577,33.26595") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5139, "countyName": "Union", "cityID": 521070, "cityName": "El Dorado" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597799546880, "text": "THAT MAKE UP THO WOW https://t.co/ojTFV82wfU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338875358, "name": "Braye Hardy", "screen_name": "brayehardy", "lang": "en", "location": "409➡️512", "create_at": date("2011-07-19"), "description": "IG: braye.braye Snapchat: braye_hardyyy #TXST", "followers_count": 1127, "friends_count": 629, "statues_count": 20296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597816291329, "text": "How you bouta text me then leave me on read", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212821644, "name": "Ashley", "screen_name": "Ihtssashleeey__", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "striving to be more than i have ever been ✨", "followers_count": 344, "friends_count": 245, "statues_count": 7860 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566597845655552, "text": "this stomach ache is just terrible��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323296405, "name": "Genesis G. Martinez", "screen_name": "missgen_", "lang": "en", "location": "California, USA", "create_at": date("2011-06-24"), "description": "obsessed with @_MALayton", "followers_count": 1148, "friends_count": 593, "statues_count": 25551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kerman, CA", "id": "31fd66b3f28d4c5a", "name": "Kerman", "place_type": "city", "bounding_box": rectangle("-120.083084,36.716771 -120.041995,36.736803") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 638226, "cityName": "Kerman" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566598118277120, "text": "Yeah you do smh ���� lmao https://t.co/cWM5OksXGJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3907421557, "name": "Abi", "screen_name": "abi_1973", "lang": "en", "location": "null", "create_at": date("2015-10-15"), "description": "null", "followers_count": 62, "friends_count": 88, "statues_count": 821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566598449790977, "text": "at vacation on work.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.556779,38.985796"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311372918, "name": "rej", "screen_name": "Iwhip_foreign", "lang": "en", "location": "null", "create_at": date("2011-06-05"), "description": "my only adversary was my own mind", "followers_count": 1374, "friends_count": 1292, "statues_count": 17129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Inn & Suites By Carlson, Annapolis, MD", "id": "07d9f346ab082000", "name": "Country Inn & Suites By Carlson, Annapolis, MD", "place_type": "poi", "bounding_box": rectangle("-76.5567791,38.9857959 -76.556779,38.985796") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2460325, "cityName": "Parole" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566599938637824, "text": "I need a interview I'm trying to get down with you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438909008, "name": "J.REEVELY ⚖", "screen_name": "SupremeReevely", "lang": "en", "location": "Houston, TX", "create_at": date("2011-12-16"), "description": "Long as the lord say the same. Pvamu.", "followers_count": 919, "friends_count": 448, "statues_count": 35363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-09T00:00:06.000Z"), "id": 729566600341266432, "text": "follow @ayosexual for more ! https://t.co/iRbbWFBoJ3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4491875653, "name": "robert maldonodo", "screen_name": "bugattiputo", "lang": "en", "location": "null", "create_at": date("2015-12-07"), "description": "null", "followers_count": 1, "friends_count": 12, "statues_count": 86 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alum Rock, CA", "id": "277b4360183b5d75", "name": "Alum Rock", "place_type": "city", "bounding_box": rectangle("-121.842175,37.35398 -121.812158,37.38102") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 601458, "cityName": "Alum Rock" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928959509471232, "text": "Me (singing): \"I got two phones\"\nMom: Two phones? What? Why? \n*looks at mom & smh*\nMe: Dammit Daniel.\n\nPoor mom.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762878089, "name": "mariah ✨", "screen_name": "mariahleaah", "lang": "en", "location": "@_ItsLilianaa, tx", "create_at": date("2012-08-16"), "description": "null", "followers_count": 795, "friends_count": 536, "statues_count": 26370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928960130195456, "text": "oh honey by delegation is my faaav oldie��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013650926, "name": "mama mars", "screen_name": "marliena__", "lang": "en", "location": "El Monte, CA", "create_at": date("2015-02-08"), "description": "null", "followers_count": 225, "friends_count": 109, "statues_count": 5459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928960625123328, "text": "Like some people would just edit their grammar but I would change a tweet entirely just for kicks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 154975662, "name": "Lindsey Walker", "screen_name": "LiiiindseyAlex", "lang": "en", "location": "null", "create_at": date("2010-06-12"), "description": "null", "followers_count": 457, "friends_count": 291, "statues_count": 6655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961006788608, "text": "I wonder who said this? Must've been someone brilliant and funny ���� https://t.co/frLHiYUOCL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 710185820, "name": "Alana", "screen_name": "teddygrams2016", "lang": "en", "location": "Los Angeles/ Davis, CA", "create_at": date("2012-07-21"), "description": "I can't tell you the key to success, but the key to failure is trying to please everyone. - Ed Sheeran | UC Davis '19", "followers_count": 165, "friends_count": 168, "statues_count": 12048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davis, CA", "id": "1994142e26ba7127", "name": "Davis", "place_type": "city", "bounding_box": rectangle("-121.803252,38.526843 -121.675074,38.590264") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 618100, "cityName": "Davis" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961019379713, "text": "GAINED 3 FOLLOWERS LOST 3 FOLLOWERS LOL WTF YALL DOIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727559670, "name": "☙ pixels ❧", "screen_name": "_mithril_", "lang": "en", "location": "the wildy wit yo bitch", "create_at": date("2012-07-30"), "description": "I make sprites/Pixel art, music, and terrible photo edits. Mithrilsprites@gmail.com", "followers_count": 574, "friends_count": 307, "statues_count": 34431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961346699264, "text": "I was sleep ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172092079, "name": "TrillAssTarjai!❣", "screen_name": "ImTarjai_", "lang": "en", "location": "`Chilling w. @Chrisbrown", "create_at": date("2010-07-28"), "description": "Dare to Dream Big⭐️ °#JSU18 |° SC: Tarjai | Trillinois✈Jacktown RIPBENNIE.♥️", "followers_count": 1825, "friends_count": 1552, "statues_count": 90342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collinsville, IL", "id": "93dcdaf16d14b1fd", "name": "Collinsville", "place_type": "city", "bounding_box": rectangle("-90.051852,38.643745 -89.928359,38.745016") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1715599, "cityName": "Collinsville" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961409474560, "text": "@DoctorFluxx it picks up at the end.", "in_reply_to_status": 729928831616700416, "in_reply_to_user": 167969735, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 167969735 }}, "user": { "id": 2230751, "name": "Christopher Dreiling", "screen_name": "fotomatique", "lang": "en", "location": "San Francisco, CA", "create_at": date("2007-03-25"), "description": "I do some Apple stuff. Why offend with style, when you can offend with substance.", "followers_count": 273, "friends_count": 395, "statues_count": 3234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961573060608, "text": "I'm over this part of my life . can I skip to me getting married and having kids and living life now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1169899777, "name": "princess", "screen_name": "DaishaRicks", "lang": "en", "location": "null", "create_at": date("2013-02-11"), "description": "shut up I'm a princess sc : daisharicks", "followers_count": 617, "friends_count": 370, "statues_count": 14561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928961744982017, "text": "@BryceWheeldon I am in bed...", "in_reply_to_status": 729928889435217921, "in_reply_to_user": 565540754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 565540754 }}, "user": { "id": 4709810772, "name": "Cassidy Condie", "screen_name": "CondieCassidy", "lang": "en", "location": "sc: cassidycondie ", "create_at": date("2016-01-04"), "description": "U of U Dance Team - MDC Company / Slc, Ut .", "followers_count": 601, "friends_count": 530, "statues_count": 999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928962239930368, "text": "See our latest #Carlisle, PA #job and click to apply: Certified Nursing Assistant |... - https://t.co/2NeypA5V4l https://t.co/AOleuzHRfK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.2002745,40.2010241"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Carlisle", "job" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 210, "friends_count": 99, "statues_count": 4198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlisle, PA", "id": "93545c1fe2e7d7eb", "name": "Carlisle", "place_type": "city", "bounding_box": rectangle("-77.246468,40.180728 -77.17137,40.219439") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42041, "countyName": "Cumberland", "cityID": 4211272, "cityName": "Carlisle" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928962332188672, "text": "@danikakanu lowkey how salty I am when someone doesn't respond lol", "in_reply_to_status": 729925176960278528, "in_reply_to_user": 301204942, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 301204942 }}, "user": { "id": 120882272, "name": "✿Lauren.Danielle✿", "screen_name": "lauren_cheers", "lang": "en", "location": "texass", "create_at": date("2010-03-07"), "description": "21 | TLB | #UTSA17 | #UCASTAFF ✿", "followers_count": 3573, "friends_count": 1191, "statues_count": 48170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928962906841089, "text": "When @ThisIs_Tati falls asleep on you but she doesn't know you're going to fall asleep forever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 904483838 }}, "user": { "id": 1703857362, "name": "OG 1 BARI", "screen_name": "IronsSavage", "lang": "en", "location": "null", "create_at": date("2013-08-26"), "description": "the Jabari 239\n2016-17 national championship \nhold me to it", "followers_count": 2088, "friends_count": 2194, "statues_count": 12135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Fort Myers, FL", "id": "01ac3f6d45c27f58", "name": "North Fort Myers", "place_type": "city", "bounding_box": rectangle("-81.940738,26.652659 -81.760762,26.770096") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1249350, "cityName": "North Fort Myers" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928963120717824, "text": "That's what I pray for nothing more nothing less https://t.co/3mYw1geriM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2289875550, "name": "P A C K", "screen_name": "BfbDaPackman", "lang": "en", "location": "Flint, MI ✈️ Houston, TX ", "create_at": date("2014-01-13"), "description": "I live a valet life, I'm up front with everything.", "followers_count": 2215, "friends_count": 1330, "statues_count": 33857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey Village, TX", "id": "c7b527911412a784", "name": "Jersey Village", "place_type": "city", "bounding_box": rectangle("-95.620555,29.87235 -95.548681,29.914781") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4837612, "cityName": "Jersey Village" } }
+{ "create_at": datetime("2016-05-10T00:00:00.000Z"), "id": 729928963347386368, "text": "Temp 36.2° Hi/Lo 42.4/36.1 Rng 6.3° WC 36.2° Hmd 80% Rain 0.00\" Storm 0.00\" BAR 30.172 Rising DP 30.6° Wnd 1mph Dir SW Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 122, "statues_count": 18572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928963661791233, "text": "@ebbtideapp Tide in Sachuest, Rhode Island 05/10/2016\n Low 5:11am -0.1\nHigh 11:30am 3.6\n Low 4:53pm 0.0\nHigh 11:53pm 3.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.2383,41.4867"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 84, "friends_count": 1, "statues_count": 31515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rhode Island, USA", "id": "6d50765616ee2e60", "name": "Rhode Island", "place_type": "admin", "bounding_box": rectangle("-71.907259,41.095834 -71.088567,42.018808") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44005, "countyName": "Newport" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928963863154689, "text": "@OCCashForHomes @Kris_Sacrebleu And abuser and admitted womanizer.", "in_reply_to_status": 729854974016905218, "in_reply_to_user": 728193913811402752, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 728193913811402752, 32522055 }}, "user": { "id": 138995967, "name": "Ray Plasse", "screen_name": "tddfn", "lang": "en", "location": "LA,CA", "create_at": date("2010-04-30"), "description": "LOve family,comics,music,TV,books,sports,Boston, Los Angeles and Liberals! Please don't get pissed if I don't follow you. Can't follow everyone. :)", "followers_count": 850, "friends_count": 547, "statues_count": 37717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964316102656, "text": "I said hey pretty lady won't you give me a sign I'll give anything to make you mine o mine I'll do ur biddenbb b at ur beckon call", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 105414671, "name": "V", "screen_name": "Onzo52", "lang": "en", "location": "South Gate, CA", "create_at": date("2010-01-16"), "description": "11C in the U.S. Army•On the pursuit of happiness", "followers_count": 214, "friends_count": 442, "statues_count": 14792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964370665473, "text": "I wish that studying was as fun and as stimulating as working out...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2542595756, "name": "⚡️Alex Netherda⚡️", "screen_name": "NoStarNerdy", "lang": "en", "location": "Santa Rosa, CA", "create_at": date("2014-06-02"), "description": "When you're good at something, you'll tell everyone. When you're great, they'll tell you. Cal Football ATH #38", "followers_count": 598, "friends_count": 437, "statues_count": 2978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964412723200, "text": "@ErneSnowden4 oh okay lol", "in_reply_to_status": 729927329330290688, "in_reply_to_user": 51083847, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 51083847 }}, "user": { "id": 75456074, "name": "erica", "screen_name": "thisIS_amERICA", "lang": "en", "location": "null", "create_at": date("2009-09-18"), "description": "UL Sports Management/ Business Law sc: ericaaa1.2", "followers_count": 1742, "friends_count": 846, "statues_count": 89744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964500692992, "text": "Wind 4.5 mph ENE. Barometer 29.51 in, Steady. Temperature 42.1 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 18, "friends_count": 5, "statues_count": 42092 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964718764032, "text": "https://t.co/1oeS7m0DxH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 727304599556624384, "name": "KOLEI VANNI", "screen_name": "KoleiVanni", "lang": "zh-cn", "location": "null", "create_at": date("2016-05-02"), "description": "null", "followers_count": 1, "friends_count": 22, "statues_count": 62 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928964844617728, "text": "Oh no ������☹", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1160646295, "name": "CharBear", "screen_name": "charitysmith98", "lang": "en", "location": "null", "create_at": date("2013-02-08"), "description": "soggypancakeeater2k16", "followers_count": 273, "friends_count": 312, "statues_count": 5138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, WA", "id": "9904dedb2c0909ab", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-123.86033,46.95147 -123.778204,47.001666") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53027, "countyName": "Grays Harbor", "cityID": 5300100, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965230501888, "text": "@ehmle I'll take a dozen to go.", "in_reply_to_status": 729924794498441217, "in_reply_to_user": 279386095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 279386095 }}, "user": { "id": 61021333, "name": "Daniel", "screen_name": "ItsDanielTime", "lang": "en", "location": "Staples Center or Miami", "create_at": date("2009-07-28"), "description": "Did someone say ciroc?", "followers_count": 605, "friends_count": 258, "statues_count": 58165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965306122240, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Steady. Temperature 66.7 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965322739712, "text": "@_dejaaax I'm weak af ����", "in_reply_to_status": 729927958870282242, "in_reply_to_user": 137869024, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 137869024 }}, "user": { "id": 1481957636, "name": "Jordan Price", "screen_name": "jojopriceyy", "lang": "en", "location": "null", "create_at": date("2013-06-04"), "description": "chs 16'", "followers_count": 808, "friends_count": 580, "statues_count": 10930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965381443584, "text": "broken clouds -> overcast clouds\ntemperature down 56°F -> 53°F\nhumidity up 48% -> 59%\nwind 12mph -> 13mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.55,43.03"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 148162977, "name": "Grand Rapids Weather", "screen_name": "_GrandRapidsMI", "lang": "en", "location": "Grand Rapids, MI", "create_at": date("2010-05-25"), "description": "Weather updates, forecast, warnings and information for Grand Rapids, MI. Sources: http://OpenWeatherMap.org, NOAA, USGS.", "followers_count": 192, "friends_count": 1, "statues_count": 26082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northview, MI", "id": "ee3de4f6fdd7de3e", "name": "Northview", "place_type": "city", "bounding_box": rectangle("-85.659488,43.024098 -85.543489,43.06309") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2658945, "cityName": "Northview" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965637419008, "text": "On the way. Tell me when you want it and I'll drop it �� https://t.co/lhLucqDBx3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258163195, "name": "kodie shane", "screen_name": "itsthedonbaby", "lang": "en", "location": "2060", "create_at": date("2011-02-26"), "description": "bookkodieshane@gmail.com", "followers_count": 2999, "friends_count": 265, "statues_count": 5044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928965998182400, "text": "Ripley SW Limestone Co. Temp: 67.5°F Wind:9.8mph Pressure: 994.8mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928966031613953, "text": "this is goodnight, but not goodbye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479717029, "name": "Allie Stemler", "screen_name": "allietastiic", "lang": "en", "location": "neverland", "create_at": date("2012-01-31"), "description": "KAΘ • ΣΧ • #FCE", "followers_count": 678, "friends_count": 589, "statues_count": 9071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928966480506880, "text": "Temp: 66.3°F Wind:0.0mph Pressure: 30.007hpa Rising Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928966501367808, "text": "I wouldn't trade the hard times or the mistakes I've made this year for anything.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2464052958, "name": "cailsss", "screen_name": "caileyjar4", "lang": "en", "location": "Texas State University ", "create_at": date("2014-04-25"), "description": "we're going streaking through the quad!!!", "followers_count": 949, "friends_count": 605, "statues_count": 21283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928966618779648, "text": "@raissawriter Davao reelected a Duterte with 99%. North Korea style. But he just intimidates \"criminals\".", "in_reply_to_status": 729926633570721792, "in_reply_to_user": 33864320, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33864320 }}, "user": { "id": 247643037, "name": "blended purple", "screen_name": "blendedpurple", "lang": "en", "location": "The Backwoods", "create_at": date("2011-02-04"), "description": "Profession: Musician, etc.\r\nInterests: Politics, Economics", "followers_count": 58, "friends_count": 21, "statues_count": 12385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928966732013569, "text": "@rihanna performed RIGHT ABOVE US!! �������� #ANTIWorldTour #AWT Thank you Rihanna! https://t.co/B6qTEFSvvq", "in_reply_to_status": -1, "in_reply_to_user": 79293791, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ANTIWorldTour", "AWT" }}, "user_mentions": {{ 79293791 }}, "user": { "id": 40931628, "name": "Alexander Steel", "screen_name": "SteelTweets_", "lang": "en", "location": "null", "create_at": date("2009-05-18"), "description": "GOD ... Actor/Filmmaker/Photographer x San Diego, California - http://Instagram.com/Ajsteel & http://instagram.com/SteelPhotosSD", "followers_count": 800, "friends_count": 828, "statues_count": 35255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928967281467392, "text": "I forgot to be your lover.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338149602, "name": "hashslinging slasher", "screen_name": "dmaccbby", "lang": "en", "location": "Chicago✈️hamptonu19", "create_at": date("2011-07-18"), "description": "Chicagoan till Chicago end.", "followers_count": 2642, "friends_count": 1834, "statues_count": 99496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928967415701504, "text": "Fuck no . Dead ass U'V ������ https://t.co/yJ7auLzT0a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282818271, "name": "3000 Jr.", "screen_name": "THEOG_Lik", "lang": "en", "location": "Tennessee ↔️ Texas", "create_at": date("2011-04-15"), "description": "just a young nigga tryna have the juice like Pac", "followers_count": 1490, "friends_count": 1264, "statues_count": 48112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-10T00:00:01.000Z"), "id": 729928967541563392, "text": "@BXVXE buy a different side dude", "in_reply_to_status": 729833444994260992, "in_reply_to_user": 611606414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 611606414 }}, "user": { "id": 104634207, "name": "max", "screen_name": "_hardknocks", "lang": "en", "location": "los angeles ", "create_at": date("2010-01-13"), "description": "22. Ftm. follow my journey http://youtu.be/5TtpQT62BXQ", "followers_count": 1553, "friends_count": 1536, "statues_count": 52457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928967973670912, "text": "Wind 1.0 mph SE. Barometer 29.974 in, Falling slowly. Temperature 62.6 °F. Rain today 0.01 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928968015486977, "text": "@the_miggie @thaliaa_cortes @heyynikkki @dorafsho about to be like ���� https://t.co/kbBffmd2WH", "in_reply_to_status": 729928225728520192, "in_reply_to_user": 1006112378, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1006112378, 818104987, 51360472, 185036839 }}, "user": { "id": 1925260363, "name": "MAC N CHEESE ONLY", "screen_name": "DISNEYLANDPAPI", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-10-01"), "description": "http://www.youcaring.com/nathanontiverosmemorial", "followers_count": 270, "friends_count": 320, "statues_count": 11383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928968208408576, "text": "Just paid $4.33 for a bottle of water. This better be magic fucking potion water", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41897628, "name": "Kels", "screen_name": "kelseytowson", "lang": "en", "location": "Arizona, USA", "create_at": date("2009-05-22"), "description": "null", "followers_count": 576, "friends_count": 192, "statues_count": 8182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928968795762688, "text": "The TACOXOC mural westfieldtopanga xoctequilagrill by candidapenaarts #art #graffiti… https://t.co/sER1Q01suR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.60316414,34.19035903"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "art", "graffiti" }}, "user": { "id": 322458956, "name": "JAGAR / ARCHITECTURE", "screen_name": "jagar_la", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-06-22"), "description": "Experimental and innovative architecture, urbanism and interior design firm based in Los Angeles, CA, USA.", "followers_count": 210, "friends_count": 210, "statues_count": 1121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969022214144, "text": "@HoodKings you right ��", "in_reply_to_status": 729928830064988160, "in_reply_to_user": 1023662124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1023662124 }}, "user": { "id": 708127066, "name": "Cierra", "screen_name": "__Arreic", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2013-10-10"), "description": "college girl | Pitt 19| Brian ❤️", "followers_count": 1330, "friends_count": 1076, "statues_count": 36319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969080999936, "text": "My tan hurts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3179369726, "name": "Kaylee Silva", "screen_name": "KayleeSilva7", "lang": "en", "location": "Sunnyside, WA", "create_at": date("2015-04-29"), "description": "I ❤️ Ingrid", "followers_count": 182, "friends_count": 107, "statues_count": 2557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyside, WA", "id": "00cce068663f6743", "name": "Sunnyside", "place_type": "city", "bounding_box": rectangle("-120.046373,46.286764 -119.964934,46.343583") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5368750, "cityName": "Sunnyside" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969160687616, "text": "good night at work :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251329056, "name": "GBABY✨", "screen_name": "gbabymaneee", "lang": "en", "location": "Lewisdale ", "create_at": date("2011-02-12"), "description": "free MARDEN", "followers_count": 2013, "friends_count": 1007, "statues_count": 100338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chillum, MD", "id": "bf9b9d49defeea89", "name": "Chillum", "place_type": "city", "bounding_box": rectangle("-77.002459,38.943921 -76.951778,38.987182") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2416875, "cityName": "Chillum" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969215193088, "text": "@copyme_original ����", "in_reply_to_status": 729928924893958144, "in_reply_to_user": 4192379008, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4192379008 }}, "user": { "id": 174452504, "name": "Chief Kiwi", "screen_name": "FourthKind_", "lang": "en", "location": "Nashville, TN to Durham NC ", "create_at": date("2010-08-03"), "description": "Tennessee/North Carolina Central IG:chiefkeeweee RIP KEKE", "followers_count": 1171, "friends_count": 1134, "statues_count": 21618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969458470912, "text": "I need to feel something wild", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242251475, "name": "Løwki bb", "screen_name": "chronicyouthh", "lang": "en", "location": "Chicago", "create_at": date("2011-01-24"), "description": "Indulge. Yung Wavy.", "followers_count": 358, "friends_count": 664, "statues_count": 39856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969613496320, "text": "@25victoriaE gladly", "in_reply_to_status": 729928706311905280, "in_reply_to_user": 135306905, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 135306905 }}, "user": { "id": 1234679160, "name": "Henny Hardaway", "screen_name": "whatupry", "lang": "en", "location": "Austin, TX", "create_at": date("2013-03-02"), "description": "The whole hood love me like the 15th and the 1st", "followers_count": 894, "friends_count": 643, "statues_count": 19010 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Copperas Cove, TX", "id": "f0eebea3b08e0158", "name": "Copperas Cove", "place_type": "city", "bounding_box": rectangle("-97.950002,31.081381 -97.867527,31.16156") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48099, "countyName": "Coryell", "cityID": 4816624, "cityName": "Copperas Cove" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969705775104, "text": "That @KAYAK commercial with the elevator is such bullshit! If I'm stuck in an elevator I'm clawing your eyes out to get out #boybye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "boybye" }}, "user_mentions": {{ 28535982 }}, "user": { "id": 83310538, "name": "Lilybeth Duran❤️", "screen_name": "Lilybeth_D", "lang": "en", "location": "null", "create_at": date("2009-10-17"), "description": "null", "followers_count": 198, "friends_count": 153, "statues_count": 14516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969911312384, "text": "71.1F (Feels: 71.1F) - Humidity: 97% - Wind: 3.8mph S - Gust: 8.3mph - Pressure: 1011.8mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 238900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928969932267521, "text": "Wind 0.4 mph WSW. Barometer 29.925 in, Rising. Temperature 68.9 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928970003619840, "text": "Remind me again, why do people like Selena Gomez?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-149.91626498,61.18913766"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 868221739, "name": "Lauren Curran", "screen_name": "LoCurran", "lang": "en", "location": "Anchorage, AK", "create_at": date("2012-10-08"), "description": "Strap on your adult diapers, everyone", "followers_count": 166, "friends_count": 646, "statues_count": 827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928970418962432, "text": "Wind 0.0 mph SE. Barometer 29.734 in, Falling. Temperature 60.7 °F. Rain today 0.01 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928970859241472, "text": "I love the woman I am becoming.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176757629, "name": "✨kee✨", "screen_name": "winefinekee", "lang": "en", "location": "AKAland", "create_at": date("2010-08-10"), "description": "19 | God | TWU | Alpha Kappa Alpha Woman |", "followers_count": 2690, "friends_count": 2693, "statues_count": 56611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928971207335936, "text": "@layo_42 good night my G", "in_reply_to_status": 729928551093313538, "in_reply_to_user": 1627293254, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1627293254 }}, "user": { "id": 584356139, "name": "Monje The Clutch!", "screen_name": "Mapler_Monje", "lang": "en", "location": "#DaClub", "create_at": date("2012-05-18"), "description": "don't take my tweets serious unless I tell you to DM me then do that", "followers_count": 946, "friends_count": 903, "statues_count": 49826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-05-10T00:00:02.000Z"), "id": 729928971337510913, "text": "Wind 0.0 mph W. Barometer 30.21 in, Rising slowly. Temperature 37.6 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928972134285313, "text": "I just want to feel appreciated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4550070793, "name": "keona", "screen_name": "_kekereneee", "lang": "en", "location": "null", "create_at": date("2015-12-13"), "description": "null", "followers_count": 354, "friends_count": 297, "statues_count": 13590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928972734074881, "text": "@lyzardon i look cool", "in_reply_to_status": 729928744077393921, "in_reply_to_user": 199541589, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 199541589 }}, "user": { "id": 99014464, "name": "Nana!", "screen_name": "edwardeldick", "lang": "en", "location": "♌️", "create_at": date("2009-12-23"), "description": "nana / ed / deku | 19 | any pronouns. big fucko loves to meme", "followers_count": 269, "friends_count": 319, "statues_count": 39433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, MO", "id": "01234fef0796f56a", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-90.422363,38.451192 -90.319711,38.547149") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2916030, "cityName": "Concord" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928973086543872, "text": "Having fun though", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1466447264, "name": "hayl", "screen_name": "HayleighKirkham", "lang": "en", "location": "NAU", "create_at": date("2013-05-28"), "description": "just peachy ♡jr", "followers_count": 569, "friends_count": 581, "statues_count": 12799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928973744889856, "text": "@lorulefield ME TOO YEA LET ME JUST FLY OVER THERE FOR LIKE A DAY SO WE CAN SEE IT", "in_reply_to_status": 729928069150953472, "in_reply_to_user": 704601506226417664, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 704601506226417664 }}, "user": { "id": 2284968950, "name": "IGNISFUCKER69✨", "screen_name": "redserenade", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-10"), "description": "noel|19| her/him|ENG/한국어 OK|its ya binchie noel, your local ignisfucker", "followers_count": 208, "friends_count": 663, "statues_count": 38280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928973791023104, "text": "This is why we should regulate who is allowed to reproduce https://t.co/Y0KnbDcyds", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509541050, "name": "Cat Queen ✨", "screen_name": "ShayBaaaaaby", "lang": "en", "location": "Austin City Limits", "create_at": date("2012-02-29"), "description": "Dove | Nightmare • i ❤ cats & Blink-182 • @jeremiahpunk is my cosmic copilot ❤️ • #UT19 Special Ed Major • Take it Sleazy", "followers_count": 635, "friends_count": 308, "statues_count": 22959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928975053557760, "text": "I whine too much! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418136717, "name": "♕drickaaa♕.", "screen_name": "Fredrickaaa", "lang": "en", "location": "NOLA. ❤️", "create_at": date("2011-11-21"), "description": "snapchat: fredrickaaaa... #CosmetologyStudent.", "followers_count": 1765, "friends_count": 1073, "statues_count": 209985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928975112282114, "text": "@ally_rosenberg love you", "in_reply_to_status": 729928512451174400, "in_reply_to_user": 525030645, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 525030645 }}, "user": { "id": 171800345, "name": "Maureen McCormick", "screen_name": "MosesMcCormick", "lang": "en", "location": "Bainbridge Island, WA", "create_at": date("2010-07-27"), "description": "UO || ΧΩ", "followers_count": 505, "friends_count": 360, "statues_count": 6270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928975481344000, "text": "You down to make neurons?? https://t.co/lD7evawsvM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2331462055, "name": "Dr.Satan", "screen_name": "KrakenHedzBro", "lang": "en", "location": "NM", "create_at": date("2014-02-06"), "description": "null", "followers_count": 109, "friends_count": 286, "statues_count": 9287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928975951142912, "text": "@harrleee update https://t.co/9Fb3v6eDc6", "in_reply_to_status": 729927771099570176, "in_reply_to_user": 311142027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 311142027 }}, "user": { "id": 311142027, "name": "harlee williams", "screen_name": "harrleee", "lang": "en", "location": "Santa Clarita, CA", "create_at": date("2011-06-04"), "description": "i really don't care", "followers_count": 1660, "friends_count": 754, "statues_count": 23314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-10T00:00:03.000Z"), "id": 729928976135688192, "text": "I will always love you...unless you stop making me hot pockets. #MomSongs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MomSongs" }}, "user": { "id": 4474270398, "name": "Gregory Richardson", "screen_name": "gregthechild", "lang": "en", "location": "Topeka, KS", "create_at": date("2015-12-06"), "description": "null", "followers_count": 29, "friends_count": 71, "statues_count": 980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soldier, KS", "id": "01b75af8502c908c", "name": "Soldier", "place_type": "city", "bounding_box": rectangle("-95.734891,39.091143 -95.626563,39.158365") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20085, "countyName": "Jackson", "cityID": 2066175, "cityName": "Soldier" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976248905728, "text": "Happy bday Laney! I'm so glad to have u in my life and you mean so much to me!Don't know what I would do w/out u!������ https://t.co/1JUqTtwotY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2567522104, "name": "Brenden", "screen_name": "brenden_99", "lang": "en", "location": "null", "create_at": date("2014-05-27"), "description": "Skyline Varsity Baseball Class of '19", "followers_count": 720, "friends_count": 548, "statues_count": 1726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976307605504, "text": "And then maybe you will have more time for me. Yeah,I still hoping.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 616356092, "name": "Ukrainian", "screen_name": "ua_child", "lang": "ru", "location": "USA", "create_at": date("2012-06-23"), "description": "Everything what not kills you,makes you stronger.", "followers_count": 326, "friends_count": 177, "statues_count": 9128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dodge City, KS", "id": "1b49a9a5d4fd7baf", "name": "Dodge City", "place_type": "city", "bounding_box": rectangle("-100.056008,37.724925 -99.964975,37.792451") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20057, "countyName": "Ford", "cityID": 2018250, "cityName": "Dodge City" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976517365760, "text": "Wind 12.8 mph S. Barometer 29.714 in, Rising slowly. Temperature 71.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976609607680, "text": "What a line �� https://t.co/iPJ6mAkTio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363957706, "name": "Kevin J", "screen_name": "Poetry_byKj", "lang": "en", "location": "null", "create_at": date("2011-08-28"), "description": "Published Poet|President of @Speak_Wesley Spread Love|Spoken Word|Follower of Christ|Multimedia Communications Major| Wesley17", "followers_count": 1206, "friends_count": 920, "statues_count": 34960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Severn, MD", "id": "1c98b097b6fcb4d6", "name": "Severn", "place_type": "city", "bounding_box": rectangle("-76.767088,39.094164 -76.642175,39.214142") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2471150, "cityName": "Severn" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976810958848, "text": "If your cuffed the only person they could leave you for is Beyoncé @aaleeyahap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4891503969 }}, "user": { "id": 441249156, "name": "PhillyTravicito", "screen_name": "TriggaTrees", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-12-19"), "description": "phillytravicito@gmail.com", "followers_count": 67, "friends_count": 123, "statues_count": 224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928976957886464, "text": "Howell, NJ | Wind 1.0 mph ENE. Baro 30.184 in, Rising Rapidly. Temp 53.4F. Rain today 0.00 in. Humidity 57% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 619, "friends_count": 833, "statues_count": 42176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928977779859456, "text": "Smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3186795739, "name": "genigga", "screen_name": "g_nastyyyy", "lang": "en", "location": "Brentwood, CA", "create_at": date("2015-05-06"), "description": "got 99 problems and they all deepak /// @funnyguydee925", "followers_count": 435, "friends_count": 316, "statues_count": 12658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928978010513408, "text": "My days be so full.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241661441, "name": "Lex", "screen_name": "_belligerent", "lang": "en", "location": "ratchetcity/uno18", "create_at": date("2011-01-22"), "description": "bby cause ima thug Romans 8:35-39 ΔΣΘ❤️", "followers_count": 971, "friends_count": 642, "statues_count": 67778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928978631270400, "text": "How is no one up Rn ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 707278685217292288, "name": "LonnaM .", "screen_name": "fwlonna_", "lang": "en", "location": "Dallas, TX", "create_at": date("2016-03-08"), "description": "Genesis 50:20 | getting someplace you not .", "followers_count": 204, "friends_count": 198, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928978811609088, "text": "We all just want somebody we can vibe with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 250293526, "name": "#318PowerMovesOnly", "screen_name": "_FINESSINGKING_", "lang": "en", "location": "Monroe Louisiana", "create_at": date("2011-02-10"), "description": "GOD1st #TeamULM✊ #318Productions #TeamPistons #TeamEagles #WILDKidNATION #TeamThunder #LSU #snapchat youngwildfire", "followers_count": 664, "friends_count": 513, "statues_count": 18786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, LA", "id": "01bac25e25d44d5d", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-92.213161,32.473448 -92.133613,32.502627") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22073, "countyName": "Ouachita", "cityID": 2210225, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928978811621376, "text": "@actthisname Okay the \"VE247H is just like a name. \"HDMI DVI-D VGA\" is just the types of inputs you can use, but you only choose one\".", "in_reply_to_status": 729928780643377152, "in_reply_to_user": 3192083341, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3192083341 }}, "user": { "id": 1881550338, "name": "Yibu", "screen_name": "Yibux", "lang": "en", "location": "null", "create_at": date("2013-09-18"), "description": "Let's go on an adventure", "followers_count": 60, "friends_count": 133, "statues_count": 4624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-10T00:00:04.000Z"), "id": 729928978979377152, "text": "Goodnight moon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142356013, "name": "Dianna @ ASTRO", "screen_name": "GrlpantsGR", "lang": "en", "location": "San Francisco, CA", "create_at": date("2010-05-10"), "description": "Marketing for @ASTROgaming. Master Red Mage & Spoony Bard. #NYC Chica. Comics. Spreading Positivity. Chaotic Good. ❤ nerds & the color pink. \n\nTweets are mio.", "followers_count": 1590, "friends_count": 702, "statues_count": 37910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928980447399941, "text": "they're so important to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1442866765, "name": "Sadie", "screen_name": "SadieBThompson", "lang": "en", "location": "USA", "create_at": date("2013-05-19"), "description": "the cream always rises, honey. 1/2 of Mads&Sads.", "followers_count": 267, "friends_count": 696, "statues_count": 1464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928980728422400, "text": "really hate meeting new people and trying this whole single life shit out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 211347922, "name": "TRAPQUEEN", "screen_name": "Jesslovssyouu", "lang": "en", "location": "null", "create_at": date("2010-11-02"), "description": "insta: jessikasarviann", "followers_count": 74, "friends_count": 106, "statues_count": 3763 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928980745216000, "text": "\"the heart wants what it wants, but sometimes the heart doesn't know what it wants\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2619700529, "name": "julixne", "screen_name": "zteeeljrm", "lang": "en", "location": "null", "create_at": date("2014-06-18"), "description": "all black everything to match my soul | ain't perfecto pips | core 07 ❣ | ceraunophile", "followers_count": 288, "friends_count": 523, "statues_count": 12199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Miami Beach, FL", "id": "045badf8fedd9c63", "name": "North Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.208932,25.914146 -80.130673,25.957137") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1249475, "cityName": "North Miami Beach" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928980753604608, "text": "@fucken_saul me last week����", "in_reply_to_status": 729926029528080384, "in_reply_to_user": 428402901, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 428402901 }}, "user": { "id": 145820400, "name": "Sexy Daddy Beast :)", "screen_name": "chumchumxD", "lang": "en", "location": "Compton,CA ", "create_at": date("2010-05-19"), "description": "Manuel. Pimpin aint easy", "followers_count": 516, "friends_count": 433, "statues_count": 48198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928980975886336, "text": "@spanishcvndy is so beautiful I can't get enough �������� https://t.co/KBR3WhzrNj", "in_reply_to_status": -1, "in_reply_to_user": 3953296632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3953296632 }}, "user": { "id": 4479341172, "name": "✨Valeria Vargas✨", "screen_name": "Vale_Vargas17", "lang": "en", "location": "null", "create_at": date("2015-12-06"), "description": "Mommy to princess Mia❤️ SC:vvaale", "followers_count": 152, "friends_count": 128, "statues_count": 579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gervais, OR", "id": "c03c953b36e26cb0", "name": "Gervais", "place_type": "city", "bounding_box": rectangle("-122.904757,45.101898 -122.886309,45.116185") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4128650, "cityName": "Gervais" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981248532480, "text": "Wind 0 mph ---. Barometer 1014.9 hPa, Falling slowly. Temperature 72.9 °F. Rain today 0.00 in. Humidity 59%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 156005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981265309699, "text": "Well that's enough of guitar for today (3-4hours)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263996840, "name": "Adam B. Alvarado", "screen_name": "SomosAllPro", "lang": "en", "location": "null", "create_at": date("2011-03-10"), "description": "Where the fuck I am??", "followers_count": 300, "friends_count": 288, "statues_count": 6321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atascocita, TX", "id": "38d0e797745f4c5d", "name": "Atascocita", "place_type": "city", "bounding_box": rectangle("-95.249792,29.912242 -95.135568,30.033094") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4804462, "cityName": "Atascocita" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981273681926, "text": "@_bendaw @TheScottoShow baka be bsing but when he on he on .!! But we ban say that bout anybody \"if he on he on\" who consistent ?", "in_reply_to_status": 729928778919501825, "in_reply_to_user": 324149754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 324149754, 4171428433 }}, "user": { "id": 468115220, "name": "⭐️", "screen_name": "_CAvery6", "lang": "en", "location": "Honolulu, HI", "create_at": date("2012-01-18"), "description": "C.Avery ⚡️They Say We Learn From Mistakes , Well Thats Why They Mistake Me ...Straight Devastation", "followers_count": 1349, "friends_count": 896, "statues_count": 26253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981307363328, "text": "@hannahedmonster https://t.co/8XtTRqRCMv", "in_reply_to_status": -1, "in_reply_to_user": 721290008, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 721290008 }}, "user": { "id": 45476468, "name": "Ashlee Edminster", "screen_name": "AshleeEdminster", "lang": "en", "location": "Murray, KY", "create_at": date("2009-06-07"), "description": "I found my life when I laid mine down. Murray State student. AOII.", "followers_count": 1083, "friends_count": 733, "statues_count": 15604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murray, KY", "id": "ca0d320dd40f586b", "name": "Murray", "place_type": "city", "bounding_box": rectangle("-88.361326,36.572627 -88.287973,36.652072") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21035, "countyName": "Calloway", "cityID": 2154642, "cityName": "Murray" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981554839553, "text": "Nikki and I are both in tears right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352139679, "name": "Trap Queen✌", "screen_name": "BrookeTriciaa", "lang": "en", "location": "Monroe, MI", "create_at": date("2011-08-09"), "description": "20y/o. aquarius.", "followers_count": 622, "friends_count": 315, "statues_count": 72018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monroe, MI", "id": "9e707549d852ee6d", "name": "Monroe", "place_type": "city", "bounding_box": rectangle("-83.466022,41.882372 -83.336392,41.950783") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26115, "countyName": "Monroe", "cityID": 2655020, "cityName": "Monroe" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981563121665, "text": "@HereIsGina take care of @Brettdier... If anything bad happens to him, I'll lose my faith in humanity #JaneTheVirgin", "in_reply_to_status": -1, "in_reply_to_user": 143192279, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JaneTheVirgin" }}, "user_mentions": {{ 143192279, 148469993 }}, "user": { "id": 274665089, "name": "RHM", "screen_name": "_RHM1991", "lang": "en", "location": "Los Angeles ✈️Baton Rouge", "create_at": date("2011-03-30"), "description": "#LSU graduate student trying to help others while desperately failing to avoid getting into debt. Semi-Delusional @Dodgers fan struggling with optimism bias", "followers_count": 157, "friends_count": 312, "statues_count": 5828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928981881835521, "text": "I can't believe I used to eat meat & dairy & eggs. I can't believe I blindly supported such a cruel industry. shame on my old self.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2379613458, "name": "coop Ⓥ", "screen_name": "c__per", "lang": "en", "location": "laniakea", "create_at": date("2014-03-08"), "description": "eat like you give a fuck. #LGBT #Atheist #GoVegan", "followers_count": 418, "friends_count": 409, "statues_count": 6820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928982775222272, "text": "Sweet dreams bb ����❤️ https://t.co/F3dOGviLBI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285452304, "name": "Ayesha Haque", "screen_name": "AHaque46", "lang": "en", "location": "22 | HTX | Bengali ", "create_at": date("2011-04-20"), "description": "Aficionado of the Arts, Cultures, and Cats", "followers_count": 221, "friends_count": 201, "statues_count": 2530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928982842454016, "text": "Banger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 359595623, "name": "BroTank", "screen_name": "brodybeilfuss", "lang": "en", "location": "lifting somwehere", "create_at": date("2011-08-21"), "description": "Yo, I'm Brody. Probably workout more than I sleep. If you don't know me don't judge me, get to know me and you'll love me. #GoBlue Snapchat: Brodybeilfuss", "followers_count": 1126, "friends_count": 771, "statues_count": 34734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsdale, MI", "id": "fe4d2cc38e0ea762", "name": "Hillsdale", "place_type": "city", "bounding_box": rectangle("-84.66645,41.887686 -84.582884,41.956864") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26059, "countyName": "Hillsdale", "cityID": 2638460, "cityName": "Hillsdale" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983333109760, "text": "I'm really rocking the lobster-Brit abroad look today. ☀️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350382004, "name": "Annie Bishop", "screen_name": "annielbishop", "lang": "en", "location": "London, UK / International", "create_at": date("2011-08-07"), "description": "Travel, music and lifestyle blogger, one third of Team VENTH, full-time fun-lover and professional third wheel. Contact: annie_bishop@outlook.com", "followers_count": 5753, "friends_count": 2174, "statues_count": 2459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983358406656, "text": "2 Steller's Jay (Cyanocitta stelleri) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/MiWgSahCZk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14960 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983375167488, "text": "3 Red Crossbill (Loxia curvirostra) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/Z0nNvXRQGx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14960 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983383519232, "text": "3 Wild Turkey (Meleagris gallopavo) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/v3urZpNDxE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14960 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983400333313, "text": "12 Grace's Warbler (Setophaga graciae) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/xtPU8yJWzR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "sl", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14960 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983429697536, "text": "1 Calliope Hummingbird (Selasphorus calliope) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/aZYVjADL5J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14960 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928983442231297, "text": "Who is that https://t.co/zq9H1MNd5C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53619338, "name": "ig:@guapowitdachecks", "screen_name": "JusBe_DOPE", "lang": "en", "location": "living in America", "create_at": date("2009-07-04"), "description": "IG: @guapo_knows", "followers_count": 794, "friends_count": 721, "statues_count": 37528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928984113221632, "text": "I can't sleep . might not even go to school so why sleep .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406667898, "name": "babyG", "screen_name": "__xgxx", "lang": "en", "location": "null", "create_at": date("2011-11-06"), "description": "16 ✨ // LA to IE", "followers_count": 358, "friends_count": 358, "statues_count": 3683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928984226607104, "text": "3 Dusky-capped Flycatcher (Myiarchus tuberculifer) - 109 Trail (Maricopa Co.) - 2016-05-08 12:43 https://t.co/p5lbKjXXEu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.1662434,33.4685957"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 14964 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-05-10T00:00:05.000Z"), "id": 729928984255864832, "text": "I need her in my office guys!\n\n@CandyCharms69 https://t.co/G31YPCgS8p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 45698212 }}, "user": { "id": 2709475230, "name": "Richard Allen", "screen_name": "viskil2000", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-08-05"), "description": "Model promoter! Bikini Lover- Sports and Car guy ..", "followers_count": 240, "friends_count": 122, "statues_count": 6342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291347458916352, "text": "@HeyIAmPABLO it's ok bro we all get there. That why you have friends my nigga https://t.co/oR0Y1ll87o", "in_reply_to_status": 730290714433576962, "in_reply_to_user": 245943740, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 245943740 }}, "user": { "id": 854986272, "name": "SUPRMANE", "screen_name": "Og_Carlton77", "lang": "en", "location": "504", "create_at": date("2012-09-30"), "description": "Your neighborhood Superhero /Krewe of 77 #SUPRMANE", "followers_count": 499, "friends_count": 356, "statues_count": 24404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291347861688320, "text": "Like????????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3029214035, "name": "hell nah", "screen_name": "vivian__15", "lang": "en", "location": "null", "create_at": date("2015-02-10"), "description": "try me bitch", "followers_count": 245, "friends_count": 249, "statues_count": 6760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, GA", "id": "4ff21355b6fb2bda", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-84.368715,33.502691 -84.342143,33.543533") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1342604, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291348528500736, "text": "WhatsApp Messenger https://t.co/WTFJmkWDiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4385888893, "name": "Maria", "screen_name": "Maria77781045", "lang": "en", "location": "null", "create_at": date("2015-11-27"), "description": "ola amigos", "followers_count": 19, "friends_count": 43, "statues_count": 5 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291348813668352, "text": "@estephaniiieee u wait to buy meet n greets will be over or you don't even get to meet any one I waited for Manny mua for hours and didn't", "in_reply_to_status": 730290361541648384, "in_reply_to_user": 3250254512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3250254512 }}, "user": { "id": 1562387078, "name": "ana", "screen_name": "anaaabananaaaa_", "lang": "en", "location": "null", "create_at": date("2013-07-01"), "description": "MUA/ ig: anaaabananaaa_", "followers_count": 514, "friends_count": 527, "statues_count": 32718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291349077954561, "text": "@ebbtideapp Tide in Murderkill River entrance, Delaware 05/11/2016\n Low 8:18am -0.2\nHigh 2:02pm 5.0\n Low 8:21pm 0.2\nHigh 2:19am 6.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.3967,39.0583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 86, "friends_count": 1, "statues_count": 31796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1007250, "cityName": "Bowers" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291349082275840, "text": "And I always fall asleep for like 1 hour then up for 4 more. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232428299, "name": "Brody Wayne", "screen_name": "bwayne278", "lang": "en", "location": "everywhere.", "create_at": date("2010-12-30"), "description": "God is #1.", "followers_count": 841, "friends_count": 488, "statues_count": 14243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Holly, NC", "id": "3f39d4ede6a15905", "name": "Mount Holly", "place_type": "city", "bounding_box": rectangle("-81.090019,35.257517 -80.94545,35.361505") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37071, "countyName": "Gaston", "cityID": 3744960, "cityName": "Mount Holly" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291349094842368, "text": "Acting like I don't have to get up in 5 hours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 921772536, "name": "The Gingerbread Man", "screen_name": "LlegoPapaa", "lang": "en", "location": "Jamaica", "create_at": date("2012-11-02"), "description": "#BESSBandits", "followers_count": 1069, "friends_count": 884, "statues_count": 41000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lodi, NJ", "id": "cd13d7c07e4e78e7", "name": "Lodi", "place_type": "city", "bounding_box": rectangle("-74.098661,40.861503 -74.064317,40.895055") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3441100, "cityName": "Lodi" } }
+{ "create_at": datetime("2016-05-11T00:00:00.000Z"), "id": 730291351036682240, "text": "Yeah that money is what I pay for ... Call it lottery", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4761207433, "name": "princess ashanti❤️", "screen_name": "__xashanti", "lang": "en", "location": "null", "create_at": date("2016-01-14"), "description": "Beauty is internal #tamuc19", "followers_count": 765, "friends_count": 593, "statues_count": 18060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291351640625152, "text": "Ambivert https://t.co/7BLCWH1QTk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2566329022, "name": "fka twink", "screen_name": "uhkeylee", "lang": "en", "location": "null", "create_at": date("2014-05-26"), "description": "is it wrong that i want more for myself?", "followers_count": 372, "friends_count": 858, "statues_count": 11682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352274001920, "text": "Mcdouble w. Mac sauce, yass.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2221923864, "name": "Sara Rosalinda", "screen_name": "_serrruhh", "lang": "en", "location": "AZ", "create_at": date("2013-11-29"), "description": "21.\n\n\nsc: Serrruhhh", "followers_count": 1408, "friends_count": 2416, "statues_count": 1280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352399917058, "text": "@Brianshere_ \"��\" Lol", "in_reply_to_status": 730288910702546945, "in_reply_to_user": 719015500391419904, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 719015500391419904 }}, "user": { "id": 316645613, "name": "nicebeardvinny", "screen_name": "vinnygael", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-13"), "description": "Nothing real can be threatened. UNT | PR | contact: relvyngael@outlook.com", "followers_count": 1292, "friends_count": 539, "statues_count": 85742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352471101440, "text": "views is growing onto me ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872764004, "name": "tina-bean-a", "screen_name": "christi_unique", "lang": "en", "location": "210", "create_at": date("2012-10-10"), "description": "hmu ❤duecedimecity❤", "followers_count": 1316, "friends_count": 1898, "statues_count": 27418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352559357953, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.187 in, Falling. Temp 48.4F. Rain today 0.00 in. Humidity 88% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 619, "friends_count": 833, "statues_count": 42224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352848633857, "text": "@DrThomasParker1 It IS FUCKING FUNNY BECAUSE IT WENT FROM $60,000 TO $150,000 TO \"I DON'T KNOW?\"....PFFFFF....) 'FAIR MARKET VALUE'...:)", "in_reply_to_status": 730291075655458818, "in_reply_to_user": 373629913, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373629913 }}, "user": { "id": 373629913, "name": "Dr. Thomas Parker", "screen_name": "DrThomasParker1", "lang": "en", "location": "Corvallis, Oregon", "create_at": date("2011-09-14"), "description": "Hi,\r\nMy name is Dr. Thomas Parker. I am a scientist living in Corvallis, Oregon. I have a PhD in Crop and Soil Science from Oregon State University.", "followers_count": 4623, "friends_count": 4819, "statues_count": 61804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291352932655104, "text": "Ripley SW Limestone Co. Temp: 72.0°F Wind:7.6mph Pressure: 997.6mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291353012166656, "text": "@stevendavis2868 lame", "in_reply_to_status": 730290998278901761, "in_reply_to_user": 445967923, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 445967923 }}, "user": { "id": 344897640, "name": "TΔylor", "screen_name": "baby_taylor24", "lang": "en", "location": "Owasso, OK", "create_at": date("2011-07-29"), "description": "|sc: baby_taylor2 | pug owner |", "followers_count": 576, "friends_count": 247, "statues_count": 10819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Owasso, OK", "id": "008bb7ce560a49b0", "name": "Owasso", "place_type": "city", "bounding_box": rectangle("-95.901938,36.24608 -95.735716,36.350208") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4056650, "cityName": "Owasso" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291353481945089, "text": "Giggity", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520121049, "name": "Ratty Boy", "screen_name": "JizNasty3point0", "lang": "en", "location": "D-Town", "create_at": date("2012-03-09"), "description": "Goaltender for the Silver Bullets", "followers_count": 1029, "friends_count": 564, "statues_count": 6105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, MN", "id": "62619a76134ad05e", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-92.282745,46.650616 -91.962935,46.866053") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2717000, "cityName": "Duluth" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291353637289984, "text": "@dwest9cubs you're welcome!", "in_reply_to_status": 730273943374970880, "in_reply_to_user": 506842526, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 506842526 }}, "user": { "id": 30221744, "name": "Diana", "screen_name": "dmoneyy735", "lang": "en", "location": "Illinois, USA", "create_at": date("2009-04-10"), "description": "22 ΦΣΣ Illinois State University. One Direction is cool.", "followers_count": 589, "friends_count": 571, "statues_count": 9904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fox Lake, IL", "id": "b5c83c98c5433147", "name": "Fox Lake", "place_type": "city", "bounding_box": rectangle("-88.203583,42.33592 -88.130396,42.443708") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1727442, "cityName": "Fox Lake" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291353716936704, "text": "Temp 42.7° Hi/Lo 47.6/42.7 Rng 4.9° WC 42.7° Hmd 84% Rain 0.00\" Storm 0.00\" BAR 30.156 Rising DP 38.2° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 122, "statues_count": 18598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291354060750848, "text": "i enjoy getting to know someone new", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 525640174, "name": "cesarnextdoor", "screen_name": "illmaticOMO", "lang": "en", "location": "† RIP GRANDMA †", "create_at": date("2012-03-15"), "description": "#ASOC #6ix #ENTJ #sbuxBarista", "followers_count": 785, "friends_count": 742, "statues_count": 42825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291354148978688, "text": "Exits TL again cause the lies starting to come lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165502963, "name": "Simone✨", "screen_name": "AllBri_", "lang": "en", "location": "Spartanburg, SC", "create_at": date("2010-07-11"), "description": "IG: _brianasimone Snap: brisimone", "followers_count": 2755, "friends_count": 2062, "statues_count": 72979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spartanburg, SC", "id": "0902492a15114d3e", "name": "Spartanburg", "place_type": "city", "bounding_box": rectangle("-82.045319,34.875556 -81.835277,35.024373") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45083, "countyName": "Spartanburg", "cityID": 4568290, "cityName": "Spartanburg" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291354203492354, "text": "Wind 0.0 mph ---. Barometer 29.933 in, Rising. Temperature 63.6 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291354740391936, "text": "Wind 0.0 mph ---. Barometer 30.13 in, Steady. Temperature 70.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22697 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291354937393152, "text": "How does time move so fast��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 444281120, "name": "Samuel Ojeriakhi", "screen_name": "sil1ysammy84", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-12-22"), "description": "Yea, though I walk through the valley of the shadow of death, I will fear no evil: for thou art with me\nPsalms 23:4", "followers_count": 548, "friends_count": 271, "statues_count": 8679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291355495202817, "text": "this song Guillotine is fun as hell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258015876, "name": "Charles", "screen_name": "Rels_LT", "lang": "en", "location": "Echo Park", "create_at": date("2011-02-26"), "description": "I'm not punk, please don't tell anyone", "followers_count": 189, "friends_count": 424, "statues_count": 59219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:01.000Z"), "id": 730291355595870209, "text": "Good night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902598398, "name": "$avage", "screen_name": "Anjaedw", "lang": "en", "location": "Tacoma, WA", "create_at": date("2012-10-24"), "description": "they gon talk regardless.. 1996. ✨ snap: anjaedw", "followers_count": 2025, "friends_count": 694, "statues_count": 88466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fircrest, WA", "id": "d6bb5fc2aa9dfafe", "name": "Fircrest", "place_type": "city", "bounding_box": rectangle("-122.526876,47.216844 -122.504996,47.242882") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5323970, "cityName": "Fircrest" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291355780440064, "text": "Wind 3.0 mph S. Barometer 1013.65 mb, Steady. Temperature 72.6 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291355935772672, "text": "Temp: 69.4°F Wind:0.0mph Pressure: 30.073hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291356581695488, "text": "Turn to a savage, pocket got fatter, she call me daddy++������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3002747250, "name": "wong ming❁", "screen_name": "danyelaaaa_", "lang": "en", "location": "null", "create_at": date("2015-01-29"), "description": "George Frazier is all that matters♡ STUUUUUU", "followers_count": 313, "friends_count": 261, "statues_count": 3144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357059665920, "text": "@JamesJBarcena thats what imma do bro lol", "in_reply_to_status": 730291052632907777, "in_reply_to_user": 2792402598, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2792402598 }}, "user": { "id": 316002446, "name": "$avAyeee〽️", "screen_name": "ItsFreddyAyeee", "lang": "en", "location": "West Covina, CA", "create_at": date("2011-06-12"), "description": "⚡️Nineteen ⚡️Goonz⚡️SavRich⚡️RIP Nik", "followers_count": 790, "friends_count": 908, "statues_count": 52504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357168721924, "text": "12:00 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2424034693, "name": "Chris", "screen_name": "ChrisBanuelos1", "lang": "en", "location": "null", "create_at": date("2014-04-02"), "description": "Late nights and good tunes", "followers_count": 503, "friends_count": 510, "statues_count": 6641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357416329217, "text": "Better slow down, she'll feel it in the morning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372078407, "name": "Jordan Winston", "screen_name": "Realest_Thinker", "lang": "en", "location": "Richmond, VA", "create_at": date("2011-09-11"), "description": "The Quarterback with no sacks.", "followers_count": 958, "friends_count": 293, "statues_count": 32612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "George Mason, VA", "id": "0042deed0e8f466a", "name": "George Mason", "place_type": "city", "bounding_box": rectangle("-77.345038,38.824012 -77.288248,38.851728") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5130618, "cityName": "George Mason" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357575577600, "text": "@chrisicles all this while I'm sitting in my god parents living room drunk (/:", "in_reply_to_status": 730290868985323520, "in_reply_to_user": 35416586, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35416586 }}, "user": { "id": 35416586, "name": "christian", "screen_name": "chrisicles", "lang": "en", "location": "West Los Angeles, California ", "create_at": date("2009-04-25"), "description": "don't ask I just know things", "followers_count": 410, "friends_count": 384, "statues_count": 22928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357609156611, "text": "@KENT_SHAW @RachelMarlitt", "in_reply_to_status": 726186544273457152, "in_reply_to_user": 380017685, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 380017685, 721624945 }}, "user": { "id": 2859642584, "name": "kgrimes", "screen_name": "kaylee3301", "lang": "en", "location": "null", "create_at": date("2014-10-16"), "description": "null", "followers_count": 267, "friends_count": 375, "statues_count": 1675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, OR", "id": "3fc1a1035cdd94c8", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-121.229416,44.235383 -121.133527,44.306043") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41017, "countyName": "Deschutes", "cityID": 4161200, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357781106688, "text": "@moralesjadon @PettyFuckingHoe omg ikr������", "in_reply_to_status": 730290977798115328, "in_reply_to_user": 3047176470, "favorite_count": 0, "retweet_count": 0, "lang": "da", "is_retweet": false, "user_mentions": {{ 3047176470, 3253301460 }}, "user": { "id": 2992782823, "name": "Tyanah Lee", "screen_name": "TyanahL", "lang": "en", "location": "Hawaii, USA", "create_at": date("2015-01-22"), "description": "null", "followers_count": 156, "friends_count": 162, "statues_count": 6139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mililani Town, HI", "id": "7ae51e2c9403c6b7", "name": "Mililani Town", "place_type": "city", "bounding_box": rectangle("-158.030114,21.426592 -157.992829,21.465962") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1551050, "cityName": "Mililani Town" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357877600256, "text": "71.7F (Feels: 71.7F) - Humidity: 99% - Wind: 6.0mph SE - Gust: 8.3mph - Pressure: 1014.3mb #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 239044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357911289857, "text": "Fly away", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101897996, "name": "d!x!e", "screen_name": "rosedixie", "lang": "en", "location": "Trinidad/New York", "create_at": date("2010-01-04"), "description": "I don't mind being hated, but I hate being misunderstood.....♓️Pisces!", "followers_count": 1347, "friends_count": 1551, "statues_count": 45986 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291357953073152, "text": "im just saying there's no reason to shame what they did. it's simpler just scrolling past it. smh https://t.co/46NxLaFmz0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2307559598, "name": "rissaspuffs", "screen_name": "rissacoon", "lang": "en", "location": "laskuh", "create_at": date("2014-01-23"), "description": "null", "followers_count": 798, "friends_count": 153, "statues_count": 30276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakes, AK", "id": "6de06a92516436ce", "name": "Lakes", "place_type": "city", "bounding_box": rectangle("-149.360927,61.563018 -149.249951,61.642803") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 242832, "cityName": "Lakes" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291358343237632, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 134289969, "name": " Loner ", "screen_name": "inesx311", "lang": "en", "location": "Getting Money ", "create_at": date("2010-04-17"), "description": "Savage Lifestyle", "followers_count": 1363, "friends_count": 997, "statues_count": 93213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291358745796609, "text": "ชอบเสื้อทีมชาติมาก ภูมิใจที่ครั้งนึงในชีวิตได้ใส่ รับใช้ชาติไปในตัว �� https://t.co/4rmTmIi6f1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user": { "id": 409750052, "name": "ㅋㅋ in USA", "screen_name": "mycyjcwj", "lang": "th", "location": "null", "create_at": date("2011-11-10"), "description": "우리 갓세븐 ㅁㅂ", "followers_count": 358, "friends_count": 630, "statues_count": 92004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291358896902145, "text": "Wind 1.0 mph SE. Barometer 30.002 in, Rising slowly. Temperature 60.9 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291359232491520, "text": "Someone tell me a good movie they watched this year . Theatre, Netflix , anywhere", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288004933, "name": "416", "screen_name": "_WhyToby", "lang": "en", "location": "905/ Georgia State", "create_at": date("2011-04-25"), "description": "Maybe one day, Maybe one day... IG: _WhyToby Snapchat: AyeToby", "followers_count": 1195, "friends_count": 715, "statues_count": 85111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grayson, GA", "id": "8ad36ff5ee338661", "name": "Grayson", "place_type": "city", "bounding_box": rectangle("-83.976543,33.871811 -83.9357,33.907034") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1334596, "cityName": "Grayson" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291359312019457, "text": "fuck man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2688918072, "name": "andrea martinez", "screen_name": "uunndrea", "lang": "en", "location": "Chowchilla, CA ΨΔ", "create_at": date("2014-07-28"), "description": "Marcos Torres†❥ •phillipians 4:13 • abstinence", "followers_count": 529, "friends_count": 326, "statues_count": 10720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chowchilla, CA", "id": "f977ef0a4904c1e1", "name": "Chowchilla", "place_type": "city", "bounding_box": rectangle("-120.285778,37.10035 -120.220121,37.129769") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 613294, "cityName": "Chowchilla" } }
+{ "create_at": datetime("2016-05-11T00:00:02.000Z"), "id": 730291359580606466, "text": "Your a real ass nigga and I like it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385061798, "name": "Sc : FashionSnob", "screen_name": "YoAmbitiousGirl", "lang": "en", "location": "null", "create_at": date("2011-10-04"), "description": "Get rich or die styling/ Famu19", "followers_count": 1114, "friends_count": 1063, "statues_count": 22626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291360066985984, "text": "Beat his ass https://t.co/nTq4vStojD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3145069518, "name": "sara", "screen_name": "tsukkitops", "lang": "en", "location": "oikawa defense squad™", "create_at": date("2015-04-08"), "description": "gomen tsukki!!", "followers_count": 3406, "friends_count": 99, "statues_count": 39984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291360079568898, "text": "@Simeronboiii SMHHH", "in_reply_to_status": 730291251354980352, "in_reply_to_user": 1008176852, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1008176852 }}, "user": { "id": 2595561721, "name": "M", "screen_name": "xomirandaaa_", "lang": "en", "location": "Ceres, CA", "create_at": date("2014-06-29"), "description": "@stfuimcute ❤️", "followers_count": 1034, "friends_count": 642, "statues_count": 16511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collierville, CA", "id": "018eb8290c7b9423", "name": "Collierville", "place_type": "city", "bounding_box": rectangle("-121.290952,38.189207 -121.260914,38.233368") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 614708, "cityName": "Collierville" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291360180273154, "text": "It can be a Big bihh or a little bihh https://t.co/v1r8aDQH79", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162844263, "name": "SΩul Eat3r", "screen_name": "kb_thegr8", "lang": "en", "location": "The Moon", "create_at": date("2010-07-04"), "description": "The Man ,The Myth,The Legend ,The King... RoQhard Spr.14 Theta Theta #MightyBΘΘTBruhz #Tamuc16 #TailDawg SC:RoQhardtr3", "followers_count": 1241, "friends_count": 765, "statues_count": 16541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291360310296576, "text": "Cheers https://t.co/3a4h4DTa2g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 16102482, "name": "Tim O'Rourke", "screen_name": "TimothyORourke", "lang": "en", "location": "East Bay, California", "create_at": date("2008-09-02"), "description": "http://SFChronicle.com executive producer and editor. New dad. Old jokes.", "followers_count": 1824, "friends_count": 999, "statues_count": 5398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant Hill, CA", "id": "d70cebab5f549266", "name": "Pleasant Hill", "place_type": "city", "bounding_box": rectangle("-122.104336,37.925263 -122.049733,37.98237") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657764, "cityName": "Pleasant Hill" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291360901795840, "text": "An impromptu Britney dance party makes anything better... @allieeczyz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1349710250 }}, "user": { "id": 31813538, "name": "Sean Sparks", "screen_name": "sparkssm", "lang": "en", "location": "Syracuse, NY", "create_at": date("2009-04-16"), "description": "I'm a clingy serial monogamist with a terrible metabolism...", "followers_count": 320, "friends_count": 652, "statues_count": 25423 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westvale, NY", "id": "955f5607e071fea8", "name": "Westvale", "place_type": "city", "bounding_box": rectangle("-76.230259,43.030872 -76.194538,43.048471") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3681127, "cityName": "Westvale" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291361233010689, "text": "Wind 11.2 mph SSE. Barometer 29.949 in, Rising slowly. Temperature 64.0 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291362051018753, "text": "Do it all...the nastier the better https://t.co/efXENbSg42", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 23595433, "name": "Chaos of Grind Mode", "screen_name": "Chaos33176", "lang": "en", "location": "South Dade, Florida", "create_at": date("2009-03-10"), "description": "Member of @GrindMode_ , Hustler, Sexologist & GOD lover ...FOLLOW Instagram, Vine, Snapchat, Periscope; Chaos33176", "followers_count": 7235, "friends_count": 2145, "statues_count": 332681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall, FL", "id": "9b46dccb3cfb880c", "name": "Kendall", "place_type": "city", "bounding_box": rectangle("-80.389344,25.628844 -80.304896,25.715128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236100, "cityName": "Kendall" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291362088681473, "text": "You must've saw a girl thicker than a bowl of oatmeal https://t.co/OLaQrRFO5T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91641867, "name": "Estoy vomitando", "screen_name": "keepitplur", "lang": "en", "location": "42 wallaby way, Sydney ", "create_at": date("2009-11-21"), "description": "DONT WORRY, BE HAPPY", "followers_count": 2043, "friends_count": 957, "statues_count": 191827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291362101239808, "text": "I also pray that my daughter is Riley Currys spirit animal because she's a fucking lioness and doesn't even know it ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 207833501, "name": "Maria Samantha♡", "screen_name": "x0xmsm", "lang": "en", "location": "vibin' thru CalNevAri", "create_at": date("2010-10-25"), "description": "Forever faded on the gold skies✨ 21 | In N Out | Postive soul | CBM♡", "followers_count": 388, "friends_count": 266, "statues_count": 18194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291362214625280, "text": "Wind 0.7 mph W. Barometer 30.19 in, Rising slowly. Temperature 50.0 °F. Rain today 0.00 in. Humidity 53%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363007336449, "text": "@MCANNOlli same", "in_reply_to_status": 730287489542393856, "in_reply_to_user": 312441028, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312441028 }}, "user": { "id": 1212774301, "name": "The Empire", "screen_name": "JeffersonCruzJr", "lang": "en", "location": "null", "create_at": date("2013-02-23"), "description": "#StillVille", "followers_count": 275, "friends_count": 386, "statues_count": 10338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willoughby, OH", "id": "88dfdf4f649a0ba2", "name": "Willoughby", "place_type": "city", "bounding_box": rectangle("-81.452726,41.603963 -81.370884,41.707193") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985484, "cityName": "Willoughby" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363044941824, "text": "My professor gave us a 110 question study guide and said we can bring one 3X5 index card for the final...ONE!..3X5 https://t.co/oAqVMCEn6f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513451396, "name": "Ajaneya", "screen_name": "AJaneyaBrown", "lang": "en", "location": "null", "create_at": date("2012-03-03"), "description": "SnapChat: Nichole_brown79", "followers_count": 205, "friends_count": 186, "statues_count": 1093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363183353856, "text": "2 PowerPoint presentations done, now I have a big math assignment due by tomorrow & LIFE 101.... The bags under my eyes are reaaaal!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 108753595, "name": "❁em❁", "screen_name": "emberleighhh", "lang": "en", "location": "☀️tempe, arizona☀️ ", "create_at": date("2010-01-26"), "description": "mdn '16, “You can have love all around you without being in love.” Highly obsessed with Dutch", "followers_count": 1554, "friends_count": 1058, "statues_count": 55288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363514736640, "text": "I gotta get a new drivers license, I got jacked.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34444537, "name": "Cali", "screen_name": "iLikeCaliDonks", "lang": "en", "location": "Born on November 5th", "create_at": date("2009-04-22"), "description": "An Artist in this game called poker, wizard and you lucky im not in your local game. \n\n\nI tweet real shit.", "followers_count": 1389, "friends_count": 420, "statues_count": 108257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363518906369, "text": "ehhh no", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2400877776, "name": "Bri.", "screen_name": "briellejadaaa", "lang": "en", "location": "408", "create_at": date("2014-03-20"), "description": "♋️", "followers_count": 589, "friends_count": 319, "statues_count": 20746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-11T00:00:03.000Z"), "id": 730291363699249152, "text": "My dad randomly just woke up & now we're eating ruffles talking about how dumb people are hahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 926877319, "name": "Megan Di Nonno", "screen_name": "dinonnomegan", "lang": "en", "location": "null", "create_at": date("2012-11-04"), "description": "♓️ SC: megan_dinonno", "followers_count": 1438, "friends_count": 657, "statues_count": 22512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291364299165697, "text": "My boyfriend \\ husband is my ��������������������������❤️❤️❤️❤️❤️❤️����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284722303, "name": "iamshanae", "screen_name": "NaeEvans16", "lang": "en", "location": "null", "create_at": date("2011-04-19"), "description": "IM THAT BROWNSKIN BARBIE", "followers_count": 48, "friends_count": 117, "statues_count": 3241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291364848488449, "text": "@JDubbof45 my shit ain't that long yet ����", "in_reply_to_status": 730277393215066112, "in_reply_to_user": 277321761, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 277321761 }}, "user": { "id": 426234324, "name": "⛽️oldielock$⁶", "screen_name": "TwistUpTheGreen", "lang": "en", "location": "Port Arthur, TX", "create_at": date("2011-12-01"), "description": "FreelilDj freeQuay RipPoopa #BenjiGvng #MoneyGvng #winning", "followers_count": 1663, "friends_count": 738, "statues_count": 58389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Arthur, TX", "id": "350ad20be66e9edc", "name": "Port Arthur", "place_type": "city", "bounding_box": rectangle("-94.053284,29.823923 -93.858553,29.979166") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4858820, "cityName": "Port Arthur" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291365607657472, "text": "@ballislife706 crying* cause you'll never be this good looking", "in_reply_to_status": 730290981740765184, "in_reply_to_user": 1131827160, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1131827160 }}, "user": { "id": 2760831306, "name": "Trent Andrews", "screen_name": "Trent_Andrews66", "lang": "en", "location": "Waco, TX", "create_at": date("2014-08-23"), "description": "Product of God's grace and hard work.. OBU defensive linemen. Fitness enthusiast. #GloryToGod", "followers_count": 347, "friends_count": 413, "statues_count": 1004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shawnee, OK", "id": "5fe4a6e65f0e6014", "name": "Shawnee", "place_type": "city", "bounding_box": rectangle("-96.998368,35.289709 -96.868949,35.405381") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40125, "countyName": "Pottawatomie", "cityID": 4066800, "cityName": "Shawnee" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291365800640513, "text": "Wind 0.0 mph ---. Barometer 30.018 in, Steady. Temperature 61.0 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 397, "friends_count": 293, "statues_count": 10455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291365880463363, "text": "@KINGDAYSH https://t.co/8bzpqAwfw9", "in_reply_to_status": 730277612912775169, "in_reply_to_user": 2203941124, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2203941124 }}, "user": { "id": 3363642532, "name": "J.P #WeYØG", "screen_name": "YOG_JP", "lang": "en", "location": "null", "create_at": date("2015-07-06"), "description": "FOR BOOKINGS, FEATURES, HOOKS & ETC: yogjp1@yahoo.com", "followers_count": 409, "friends_count": 409, "statues_count": 4162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291366140379136, "text": "Good Night my fellow lolicons https://t.co/jzr4SGMmOx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2852578687, "name": "GhettoOtaku", "screen_name": "god_douche", "lang": "en", "location": "null", "create_at": date("2014-10-11"), "description": "Just an average 13 year old guy who has no life and try's to post anime lolis Monday to Friday every hour", "followers_count": 97, "friends_count": 89, "statues_count": 2820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291366887084032, "text": "@seabreeezyyy ��☕️ https://t.co/gK9zxmQXGq", "in_reply_to_status": -1, "in_reply_to_user": 2773592007, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2773592007 }}, "user": { "id": 2779991661, "name": "Charles Spencer", "screen_name": "crspencerr", "lang": "en", "location": "Chesapeake, VA", "create_at": date("2014-09-21"), "description": "University of Tennessee '18", "followers_count": 111, "friends_count": 103, "statues_count": 1088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake, VA", "id": "73d1c1c11b675932", "name": "Chesapeake", "place_type": "city", "bounding_box": rectangle("-76.448383,36.66 -76.143408,36.866431") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51550, "countyName": "Chesapeake", "cityID": 5116000, "cityName": "Chesapeake" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291367125999616, "text": "happy birthday to the best sister ever ���� 2️⃣3️⃣ https://t.co/Tolqs4OUxq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325361951, "name": "princess .", "screen_name": "gnhosman", "lang": "en", "location": "null", "create_at": date("2011-06-27"), "description": "mhs softball | nathan graf❥❥", "followers_count": 294, "friends_count": 379, "statues_count": 1211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291367465779200, "text": "LMFAOOOOO YESSSS!! https://t.co/VsPOfbkax1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430550111, "name": "LoneWolf", "screen_name": "Mellowmamii", "lang": "en", "location": "null", "create_at": date("2011-12-07"), "description": "Black Thinker | Fine Ass Scorpio", "followers_count": 644, "friends_count": 357, "statues_count": 28858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291367486754817, "text": "wondering if dinosaurs actually control the universe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2337515462, "name": "troll9000", "screen_name": "seanessy_w", "lang": "en", "location": "null", "create_at": date("2014-02-10"), "description": "parking ticket connoisseur", "followers_count": 211, "friends_count": 213, "statues_count": 1578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Steilacoom, WA", "id": "328354611895cefb", "name": "Steilacoom", "place_type": "city", "bounding_box": rectangle("-122.616921,47.155879 -122.572425,47.187045") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5367770, "cityName": "Steilacoom" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291367964872704, "text": "����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1117877606, "name": "Jaylloyd", "screen_name": "IceJayJaySalmon", "lang": "en", "location": "null", "create_at": date("2013-01-24"), "description": "too many people to impress that I forget impress myself", "followers_count": 268, "friends_count": 223, "statues_count": 9270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291367981682689, "text": "I love this https://t.co/fkRQuMGZ0C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2467873596, "name": "Jay-El", "screen_name": "ForgeDaddy", "lang": "en", "location": "Oak Cliff, Tx to CSTAT ", "create_at": date("2014-04-28"), "description": "NEGUS TAMU19", "followers_count": 669, "friends_count": 920, "statues_count": 7832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-11T00:00:04.000Z"), "id": 730291368187170817, "text": "The graphic t is like the bat signal for single guys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220243952, "name": "Jordan Perry", "screen_name": "thisjordanperry", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2010-11-26"), "description": "i should start rapping IG: thisjordanperry Atlanta Los Angeles Boston Emerson College", "followers_count": 126, "friends_count": 306, "statues_count": 619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291368346542080, "text": "@brothernooo in the club, we are all family. are you racist?", "in_reply_to_status": 730291076896972801, "in_reply_to_user": 163524057, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 163524057 }}, "user": { "id": 515077328, "name": "tobasco", "screen_name": "lindseyVEVO", "lang": "en", "location": "ATX", "create_at": date("2012-03-04"), "description": "an imaginary human bean", "followers_count": 151, "friends_count": 160, "statues_count": 6137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291368375934976, "text": "i've never watched anime before except for dragon ball z and i'm about to start RWBY and idk if i watch it on youtube or how many seasons", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2589043333, "name": "liv", "screen_name": "lovablemgc", "lang": "en", "location": "LA", "create_at": date("2014-06-25"), "description": "#1 michael's hands stan", "followers_count": 1304, "friends_count": 391, "statues_count": 34545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291368388612096, "text": "Gonna start my retirement by retiring retired tires and I'll never get tired of it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 285310395, "name": "Young Mitchel", "screen_name": "Mitchhjay", "lang": "en", "location": "Lancaster", "create_at": date("2011-04-20"), "description": "snap; mitcheljelo Buffalove NY #billsmafia. put in work", "followers_count": 290, "friends_count": 493, "statues_count": 5799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, NY", "id": "015157c15e34a87a", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-78.696966,42.86362 -78.577573,42.95639") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3641135, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291368690520064, "text": "@17_lauren https://t.co/eYD8k0mrvi", "in_reply_to_status": -1, "in_reply_to_user": 401143156, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 401143156 }}, "user": { "id": 399161636, "name": "katie huff ⚓️", "screen_name": "katiejhuff", "lang": "en", "location": "Plano, TX", "create_at": date("2011-10-26"), "description": "Midwestern State Soccer #13 | MSU17 | †", "followers_count": 492, "friends_count": 471, "statues_count": 7124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita Falls, TX", "id": "b980515f617707a9", "name": "Wichita Falls", "place_type": "city", "bounding_box": rectangle("-98.614411,33.835461 -98.425702,34.017379") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48485, "countyName": "Wichita", "cityID": 4879000, "cityName": "Wichita Falls" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291368824725504, "text": "Oh snap, jkparty did another mafia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50812954, "name": "PJ De Chavez", "screen_name": "Pajamas94", "lang": "en", "location": "null", "create_at": date("2009-06-25"), "description": "| NV | bboy | Ninjurais Cr3w | Taken |", "followers_count": 329, "friends_count": 356, "statues_count": 20934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291369718091776, "text": "LMFAOAOAOAOAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 2442290849, "name": "valen", "screen_name": "valenbernaal", "lang": "en", "location": "Cali, Colombia | MIA", "create_at": date("2014-03-28"), "description": "Lady Bison Basketball #32", "followers_count": 1698, "friends_count": 664, "statues_count": 67631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doral, FL", "id": "053ab8e6678199ed", "name": "Doral", "place_type": "city", "bounding_box": rectangle("-80.41729,25.780974 -80.321016,25.870313") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1217935, "cityName": "Doral" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291369730658304, "text": "Happy Birthday to this G,since #1 we're on this journey of life together it's only the beginning with my brother ������ https://t.co/3MFMsx2iTO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990160220, "name": "Lil Ders", "screen_name": "andrewrobles34", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-12-04"), "description": "Active Bakersfield // #celebratenedu", "followers_count": 553, "friends_count": 487, "statues_count": 5373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291370196238336, "text": "I've notice I have no problem calling a nigga a bitch, I was told you're not suppose to do it cause it's wrong but do I care? ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2360513906, "name": "chanel", "screen_name": "_chanelalbright", "lang": "en", "location": "oakland, ca ☀️", "create_at": date("2014-02-24"), "description": "what it do?", "followers_count": 601, "friends_count": 271, "statues_count": 28450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-11T00:00:05.000Z"), "id": 730291370506788865, "text": "Hippe hustla ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52165393, "name": "YM ☮DHL™", "screen_name": "YoungMikeySPOD", "lang": "en", "location": "Broward County , FL", "create_at": date("2009-06-29"), "description": "Hippie Hustla #ODHL™ Broward County Artist YM(Young Mikey) Hippie Nigga ✌ in the hood widdit 5519 #400 Hustla (4way,Deepside) Son of the Famous Robert Charlot", "followers_count": 2873, "friends_count": 2985, "statues_count": 31765 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291372507324416, "text": "@HoesLoveKaylin lmao it happened again yesterday.", "in_reply_to_status": 730288911700787201, "in_reply_to_user": 294888056, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 294888056 }}, "user": { "id": 183671928, "name": "kylie", "screen_name": "kyyliiieee", "lang": "en", "location": "null", "create_at": date("2010-08-27"), "description": "null", "followers_count": 460, "friends_count": 362, "statues_count": 23339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291372599611393, "text": "Wassup hello https://t.co/NvhR4ubDlG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 979386548, "name": "™", "screen_name": "daddiesmoney", "lang": "en", "location": "anywhere but fridley", "create_at": date("2012-11-29"), "description": "Yumnah // 16 // Muslim // She|Her/They|Them // All Black Lives Matter // Pro-Hoe // Cunt // ♌️", "followers_count": 519, "friends_count": 457, "statues_count": 17525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fridley, MN", "id": "612139aad8c98456", "name": "Fridley", "place_type": "city", "bounding_box": rectangle("-93.290637,45.035336 -93.227152,45.125115") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27003, "countyName": "Anoka", "cityID": 2722814, "cityName": "Fridley" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291372771565568, "text": "We together. I remember. Sweet love. All night looooonnnnggg.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1365964518, "name": "Alexia", "screen_name": "MizellAlexia", "lang": "en", "location": "in my penthouse half naked ", "create_at": date("2013-04-19"), "description": "insta: alexiamizell sc: alexiamizell", "followers_count": 280, "friends_count": 179, "statues_count": 4219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291372981374976, "text": "By not having someone's attention https://t.co/ZGKEkQIPCJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3041744766, "name": "✨ Queen Kee ✨", "screen_name": "Daniyah__", "lang": "en", "location": "Aj World ❤️", "create_at": date("2015-02-25"), "description": "Ke'Asia Kennedy Snapchat:Don't need to know boo 7/8✨ 7/11❤️•CO2018•Kankakee High• RIP Mon ❣", "followers_count": 837, "friends_count": 603, "statues_count": 19407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kankakee, IL", "id": "7271433758c9510f", "name": "Kankakee", "place_type": "city", "bounding_box": rectangle("-87.928206,41.059959 -87.812921,41.145902") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17091, "countyName": "Kankakee", "cityID": 1738934, "cityName": "Kankakee" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291373870485505, "text": "Fool needs to get a job and stop depending on people.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305905529, "name": "Jordan McCurdy", "screen_name": "llyjordan", "lang": "en", "location": "null", "create_at": date("2014-01-22"), "description": "Live it up SC/Jordan_aye123", "followers_count": 407, "friends_count": 250, "statues_count": 10744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291374038220800, "text": "https://t.co/spurKMSire #Turkey #Russia #geopolitics", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Turkey", "Russia", "geopolitics" }}, "user": { "id": 21478415, "name": "Paul Erickson", "screen_name": "epaulnet", "lang": "en", "location": "News Torrent, With a Mind", "create_at": date("2009-02-21"), "description": "Watching the World Turn, from 40,000 feet. I only observe, and serve.\n\nPay attention.", "followers_count": 3708, "friends_count": 3675, "statues_count": 540874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291374336118784, "text": "@EricaBee33 edges dehydrated like shit", "in_reply_to_status": 730291194995970048, "in_reply_to_user": 57178059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57178059 }}, "user": { "id": 347585479, "name": "Ankles Deep", "screen_name": "JustADreamer_BU", "lang": "en", "location": "Waco, Tx", "create_at": date("2011-08-02"), "description": "Know your self worth\n@UnderUnmastered \nRetired Poet, washed up track athlete, just here trying to live my life in peace", "followers_count": 478, "friends_count": 897, "statues_count": 6464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waco, TX", "id": "a0b47ebc53eb0e63", "name": "Waco", "place_type": "city", "bounding_box": rectangle("-97.26899,31.45507 -97.069323,31.639828") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48309, "countyName": "McLennan", "cityID": 4876000, "cityName": "Waco" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291374629654529, "text": "Wind 0.0 mph S. Barometer 30.05 in, Rising slowly. Temperature 58.5 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 27, "friends_count": 112, "statues_count": 14850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291374763982848, "text": "Bra think cause he kicked with MLK , that's gone help him win . https://t.co/ZAqNEDt2MN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315003333, "name": "......", "screen_name": "ASAP__Jai", "lang": "en", "location": "New Earth ", "create_at": date("2011-06-10"), "description": "Time is something , that we don't get back . That's why I don't like wasting It - #JaiFacts", "followers_count": 1360, "friends_count": 780, "statues_count": 13400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291374990331904, "text": "A real nigga don't brag about being real if he knows it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2830301318, "name": "Dev", "screen_name": "wasillasbabe", "lang": "en", "location": "Wasilla, AK", "create_at": date("2014-09-24"), "description": "10/16/14 Jeremiah R.I.P❤️", "followers_count": 327, "friends_count": 279, "statues_count": 2687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knik-Fairview, AK", "id": "00b74f8b22f96003", "name": "Knik-Fairview", "place_type": "city", "bounding_box": rectangle("-149.681494,61.488359 -149.432647,61.572777") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 240645, "cityName": "Knik-Fairview" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291375502036992, "text": "I hate when you open your window and it makes your door do the thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 737846197, "name": "kassidy", "screen_name": "kassidywatsonn", "lang": "en", "location": "903to512", "create_at": date("2012-08-04"), "description": "I don't care if people think I hug too much because some day they'll be gone & all I'll have is memories", "followers_count": 883, "friends_count": 1132, "statues_count": 4858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291375887896576, "text": "IM 21!!!!! ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1960331858, "name": "taylor lewis", "screen_name": "tayylewiss", "lang": "en", "location": "Oregon, USA", "create_at": date("2013-10-14"), "description": "@paigerowwland is the love of my life", "followers_count": 473, "friends_count": 234, "statues_count": 20700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aloha, OR", "id": "3ff03b9d1ce7c69f", "name": "Aloha", "place_type": "city", "bounding_box": rectangle("-122.904513,45.466363 -122.839114,45.516305") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4101650, "cityName": "Aloha" } }
+{ "create_at": datetime("2016-05-11T00:00:06.000Z"), "id": 730291376148123648, "text": "@yeahimallie yes", "in_reply_to_status": 730288801147326464, "in_reply_to_user": 883637071, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 883637071 }}, "user": { "id": 864012288, "name": "Zuza", "screen_name": "polish__bitch", "lang": "en", "location": "Saturn", "create_at": date("2012-10-05"), "description": "YVCC", "followers_count": 587, "friends_count": 127, "statues_count": 11351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selah, WA", "id": "01d7e4f642399b9f", "name": "Selah", "place_type": "city", "bounding_box": rectangle("-120.582182,46.640323 -120.49075,46.708434") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5363280, "cityName": "Selah" } }
+{ "create_at": datetime("2016-05-11T00:00:07.000Z"), "id": 730291377313972226, "text": "My flowers still don't die and it's about to be two weeks ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628771616, "name": "Pao ✨", "screen_name": "Paolaaardz", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-07-06"), "description": "SHS '16", "followers_count": 308, "friends_count": 206, "statues_count": 6733 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-11T00:00:07.000Z"), "id": 730291378207391744, "text": "@StixxMania yo where are you?", "in_reply_to_status": -1, "in_reply_to_user": 95107618, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95107618 }}, "user": { "id": 168397172, "name": "D. Rose", "screen_name": "DavidRoseTheBar", "lang": "en", "location": "Inglewood, CA", "create_at": date("2010-07-18"), "description": "Tuskegee University Alumnus ////////Instagram: Kingdavidxx", "followers_count": 622, "friends_count": 456, "statues_count": 23015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-05-11T00:00:07.000Z"), "id": 730291378358345728, "text": "5/11/2016 - 02:00\nTemp: 70.2F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.951in. & Rising\nRain: 0.00 in.\nhttps://t.co/6r89IdBYWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 52, "statues_count": 52646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-11T00:00:07.000Z"), "id": 730291378408857600, "text": "I got mind control over Kathy...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1421039148, "name": "SCAMAVELI", "screen_name": "GetRichAce", "lang": "en", "location": "null", "create_at": date("2013-05-11"), "description": "Hustle Hard ... Move In Silence .. NEVERTOLDRECORDSTV", "followers_count": 629, "friends_count": 578, "statues_count": 49848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Englewood, NJ", "id": "65e3a577f1940550", "name": "Englewood", "place_type": "city", "bounding_box": rectangle("-73.994557,40.865695 -73.949202,40.913406") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3421480, "cityName": "Englewood" } }
+{ "create_at": datetime("2016-05-11T00:00:07.000Z"), "id": 730291378920423425, "text": "#Hillary2016's push for regime change in Libya created a new Isis stronghold https://t.co/qxCWYdL7cX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hillary2016" }}, "user": { "id": 727558182126522368, "name": "MicroChip™ WokeOG™", "screen_name": "WDFx2EU1", "lang": "en", "location": "Unicorn Mountain", "create_at": date("2016-05-03"), "description": "Software Engineer; I do naughty things to computers, drones, & IOT devices; I can barely read #Trump2016 #MAGAAF dats: https://keybase.io/microchip", "followers_count": 28855, "friends_count": 5643, "statues_count": 3263 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Catonsville, MD", "id": "dd03cd2e1b3ad5fa", "name": "Catonsville", "place_type": "city", "bounding_box": rectangle("-76.794408,39.22709 -76.688592,39.294914") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2414125, "cityName": "Catonsville" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653735232266241, "text": "@Swaggy_d1 https://t.co/kmGciwzVqs", "in_reply_to_status": -1, "in_reply_to_user": 2621169537, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2621169537 }}, "user": { "id": 2835779254, "name": "Cd.", "screen_name": "cedariusluttery", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-10-17"), "description": "Only Originals Records Artist | Business Inquiries: cluttery94@gmail.com | Follow Twitter & Instagram: @cedariusluttery\n#90sbaby #onlyoriginals", "followers_count": 1060, "friends_count": 839, "statues_count": 32070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653735282741248, "text": "I need to get my nails done ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386783341, "name": "Lanée Bri", "screen_name": "BRILA__", "lang": "en", "location": "C H I C A G O", "create_at": date("2011-10-07"), "description": "null", "followers_count": 854, "friends_count": 614, "statues_count": 24368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653735509102592, "text": "If you live more that 30 minutes away and you ask me to pull up...I'm not pullin out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43646124, "name": "Faithful Black Man", "screen_name": "iloveCALi_", "lang": "en", "location": "California", "create_at": date("2009-05-30"), "description": "I punch infants in the face .", "followers_count": 1440, "friends_count": 1021, "statues_count": 224583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel, VA", "id": "006654ca2b4e27c5", "name": "Laurel", "place_type": "city", "bounding_box": rectangle("-77.569305,37.606827 -77.475442,37.680986") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5144280, "cityName": "Laurel" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653735517605888, "text": "Then writes her ass a letter sayin \"I regret doing that, I still love you, let's make it work when I get out of jail\" \nLMAO NIGGA", "in_reply_to_status": 730653472429887488, "in_reply_to_user": 189614954, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189614954, "name": "conscious trap queen", "screen_name": "alexisisbestest", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-09-11"), "description": "you may not like the name but it made a statement | butterscotch babe | B.A.B.G.W.D.G.A.F. | HU18 | Jones Media Management", "followers_count": 1815, "friends_count": 987, "statues_count": 52209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "View Park-Windsor Hills, CA", "id": "3f5aa7a2b7beec37", "name": "View Park-Windsor Hills", "place_type": "city", "bounding_box": rectangle("-118.365715,33.982756 -118.331397,34.008397") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682667, "cityName": "View Park-Windsor Hills" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653736381489152, "text": "@krstlfauni what's the number? ✔��", "in_reply_to_status": 730653017305817088, "in_reply_to_user": 598026841, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 598026841 }}, "user": { "id": 1121115552, "name": "IV", "screen_name": "iravilla27", "lang": "en", "location": "Anaheim, CA", "create_at": date("2013-01-25"), "description": "null", "followers_count": 44, "friends_count": 167, "statues_count": 2027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653736725438464, "text": "@ebbtideapp Tide in Back River, Maine 05/12/2016\nHigh 4:39am 10.3\n Low 11:02am -0.3\nHigh 5:27pm 9.3\n Low 11:23pm 1.0\nHigh 5:40am 9.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-69.685,43.9583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 87, "friends_count": 1, "statues_count": 32071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23015, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653736947703808, "text": "But can not fall to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 476903898, "name": "Candii", "screen_name": "Sugar_Candii", "lang": "en", "location": "null", "create_at": date("2012-01-28"), "description": "follow me on insta Candice.Bowden", "followers_count": 754, "friends_count": 708, "statues_count": 31215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653737610514433, "text": "Wind 0.0 mph ---. Barometer 29.90 in, Rising slowly. Temperature 63.3 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 32511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653737945993216, "text": "��������Amen https://t.co/6f4irr47UL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247203810, "name": "Kendy Npimnee", "screen_name": "Nija_Babe101", "lang": "en", "location": "null", "create_at": date("2015-06-16"), "description": "we were born to be real, not to be perfect", "followers_count": 693, "friends_count": 521, "statues_count": 3546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653738315055104, "text": "https://t.co/CDo9TYHi7P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1273891470, "name": "yung bart", "screen_name": "yungbartholemew", "lang": "en", "location": "Fort Worth, TX ⛽️", "create_at": date("2013-03-16"), "description": "⠀⠀R.I.P Vince , R.I.P Bankroll , R.I.P Jordan⠀⠀ ⠀⠀ contact @notagloboy about the tunes", "followers_count": 1714, "friends_count": 550, "statues_count": 98507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653738432520193, "text": "Friendship goals. https://t.co/tEyonA1knj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32344697, "name": "Jennifer", "screen_name": "i_amm_jenn", "lang": "en", "location": "Los Angeles ", "create_at": date("2009-04-17"), "description": "Product of Los Angeles. Gangster rap made me do it. Master of messy buns & fucking things up.", "followers_count": 324, "friends_count": 399, "statues_count": 15883 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653738457698304, "text": "Congratulations https://t.co/Vj7zMAfkko https://t.co/dpl1YGJ3VM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493573300, "name": "MISTERCMT", "screen_name": "mistercmt", "lang": "en", "location": "La Habra CA", "create_at": date("2012-02-15"), "description": "A Certified Massage and Holistic Therapist, Social Service Designee, Traveler and a BLOGGER - Please visit http://www.mistercmt.net", "followers_count": 195, "friends_count": 233, "statues_count": 420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-05-12T00:00:00.000Z"), "id": 730653739158261760, "text": "I fee like drinking now lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581104018, "name": "Wub Mvster Flex", "screen_name": "ShogunFishy", "lang": "en", "location": " I'm edm af bro", "create_at": date("2012-05-15"), "description": "Sic Parvis Magna ♋", "followers_count": 318, "friends_count": 279, "statues_count": 30915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653739732893696, "text": "somebody shoot me a text", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2318419351, "name": "PenGriffey3️⃣➕➕", "screen_name": "sauciee_", "lang": "en", "location": "Baton Rouge, LA ", "create_at": date("2014-01-29"), "description": "|Southeastern University Track&Field '19|im a pretty cool dude. |1|2|3|4|5|6|7|8| TRACKNATION", "followers_count": 678, "friends_count": 1083, "statues_count": 11281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, LA", "id": "9c628d39e924afc1", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-90.553822,30.422787 -90.404939,30.54132") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2232755, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653740529684481, "text": "Woohoo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 975441564, "name": "Erykah", "screen_name": "eryk_lynn", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "I'm going to change the world, after I sleep", "followers_count": 961, "friends_count": 506, "statues_count": 27998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Mirage, AZ", "id": "0de4c71dbfcd2c32", "name": "El Mirage", "place_type": "city", "bounding_box": rectangle("-112.342111,33.579997 -112.302246,33.630786") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 422220, "cityName": "El Mirage" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653740533841920, "text": "What the fuck does any of this even mean https://t.co/no2ndFFwM0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59410360, "name": "Gofftism", "screen_name": "DoyerEspee", "lang": "en", "location": "Los Angeles", "create_at": date("2009-07-23"), "description": "We 5neverpeat fuck sports. It doesn't matter what my teams are. They're all cursed anyway.", "followers_count": 482, "friends_count": 713, "statues_count": 37331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653740919754754, "text": "Wind 3.0 mph ENE. Barometer 1018.80 mb, Rising. Temperature 67.1 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653741511106560, "text": "@ashlynn32quick damn", "in_reply_to_status": 730652759662305280, "in_reply_to_user": 1613674117, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1613674117 }}, "user": { "id": 169743043, "name": ".", "screen_name": "JalenGoThaJuice", "lang": "en", "location": "#661", "create_at": date("2010-07-22"), "description": "Strugglin", "followers_count": 885, "friends_count": 688, "statues_count": 46126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653741817450496, "text": "fav if you up still", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 432216477, "name": "No Good", "screen_name": "Gabe_NoGood", "lang": "en", "location": "Somewhere Asleep ", "create_at": date("2011-12-08"), "description": "null", "followers_count": 756, "friends_count": 319, "statues_count": 42696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vancouver, WA", "id": "5a16f6443a850916", "name": "Vancouver", "place_type": "city", "bounding_box": rectangle("-122.713442,45.582501 -122.464507,45.678677") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5374060, "cityName": "Vancouver" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653741834063872, "text": "2am & just want cuddles from oomf��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 597753690, "name": "cass⁶", "screen_name": "casslovesdrake", "lang": "en", "location": "null", "create_at": date("2012-06-02"), "description": "if I ever loved you I'll always love you.", "followers_count": 1048, "friends_count": 798, "statues_count": 43030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653742480007168, "text": "\"Overthinking\" hours just started.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241387367, "name": "Emmanuel", "screen_name": "LokoAzzE_man", "lang": "en", "location": "San Diego, CA", "create_at": date("2011-01-21"), "description": "All heroes don't wear a cape", "followers_count": 1968, "friends_count": 1297, "statues_count": 73524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653742819725312, "text": "@JUSTINMGARRETT SAVAGE SQUAD", "in_reply_to_status": 730653398538670081, "in_reply_to_user": 3023186359, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3023186359 }}, "user": { "id": 2735955907, "name": "Davir Hamilton", "screen_name": "EspnHam9", "lang": "en", "location": "Compton ✈️ Salt Lake city ", "create_at": date("2014-08-15"), "description": "SAVAGE BUT KING", "followers_count": 2731, "friends_count": 738, "statues_count": 40204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653742987640832, "text": "Ripley SW Limestone Co. Temp: 69.4°F Wind:0.0mph Pressure: 999.0mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653743167995904, "text": "Wind 0.0 mph ---. Barometer 30.029 in, Steady. Temperature 67.8 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-12T00:00:01.000Z"), "id": 730653743243464704, "text": "69.9F (Feels: 69.9F) - Humidity: 99% - Wind: 0.0mph --- - Gust: 2.2mph - Pressure: 1015.7mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 239235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653743931392000, "text": "You take my love for granted. I just don't understand it ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 202750107, "name": "æ", "screen_name": "its_anamel", "lang": "en", "location": "null", "create_at": date("2010-10-14"), "description": "stop making a big deal out of the little things #FABULOUS #teamroyce #dominican #RIPCarlos http://www.amway.com/anameldeleon", "followers_count": 421, "friends_count": 360, "statues_count": 12400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willimantic, CT", "id": "01a916f06046e1e8", "name": "Willimantic", "place_type": "city", "bounding_box": rectangle("-72.253203,41.696332 -72.18037,41.762436") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9015, "countyName": "Windham", "cityID": 985810, "cityName": "Willimantic" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744099102720, "text": "Just try your luck, and don't look back. This world is yours if you want it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456190262, "name": "#ALLin Shafer", "screen_name": "ShafeDaddyFresh", "lang": "en", "location": "H(ohio)ME", "create_at": date("2012-01-05"), "description": "formerly Superman | tOSU | 922, bud | CLE til I die | contributing writer, The Odyssey - OSU | 2 Corinthians 5:7 #ssc - dasvidaniya", "followers_count": 1239, "friends_count": 1028, "statues_count": 96679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744380190720, "text": "Wind 0.0 mph WSW. Barometer 30.000 in, Falling slowly. Temperature 61.6 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744484900864, "text": "@22dys I found it lmao https://t.co/thXPiJBgZU", "in_reply_to_status": -1, "in_reply_to_user": 2568698622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2568698622 }}, "user": { "id": 2590484396, "name": "Anya Hollis", "screen_name": "HollisAnya", "lang": "en", "location": "alaska", "create_at": date("2014-06-26"), "description": "null", "followers_count": 515, "friends_count": 257, "statues_count": 5198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gateway, AK", "id": "014a9e6864a3be2c", "name": "Gateway", "place_type": "city", "bounding_box": rectangle("-149.329835,61.547233 -149.177953,61.633205") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 228200, "cityName": "Gateway" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744631681024, "text": "interstate action, she remember them long nights ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3103691089, "name": "selfmade", "screen_name": "jhazO3", "lang": "en", "location": "null", "create_at": date("2015-03-22"), "description": "null", "followers_count": 2311, "friends_count": 1227, "statues_count": 85679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744694763521, "text": "when I know I should be sleep its 3:00 exactly ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297180071, "name": "A❣", "screen_name": "ABombthreat__", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2011-05-11"), "description": "found me", "followers_count": 860, "friends_count": 791, "statues_count": 54469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744950591488, "text": "Temp: 70.6°F Wind:0.0mph Pressure: 30.091hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653744958963712, "text": "Wind 1.0 mph SE. Barometer 29.955 in, Steady. Temperature 63.6 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653745189543936, "text": "@thatnigganima ouch", "in_reply_to_status": 730653504553943040, "in_reply_to_user": 1478553678, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1478553678 }}, "user": { "id": 66728113, "name": "Aaron Zendejas", "screen_name": "rundum46", "lang": "en", "location": "Santa Clara, CA", "create_at": date("2009-08-18"), "description": "You're in charge of your future. Dont let your current situation stop you. catch me in the octagon. call me..the vision.", "followers_count": 707, "friends_count": 918, "statues_count": 40054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653745382457344, "text": "@Alden_Gonzalez tell the managers to lead off Trout! It may jumpstart the team.", "in_reply_to_status": 730625800819462146, "in_reply_to_user": 28577099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28577099 }}, "user": { "id": 88386238, "name": "Gilbert R. Kalsic", "screen_name": "rkalsic", "lang": "en", "location": "Southern Calif.", "create_at": date("2009-11-08"), "description": "Outgoing person, lives for the outdoors, and Angels baseball!", "followers_count": 22, "friends_count": 78, "statues_count": 186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey Park, CA", "id": "fa424f169eef946e", "name": "Monterey Park", "place_type": "city", "bounding_box": rectangle("-118.170463,34.027012 -118.093679,34.071692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648914, "cityName": "Monterey Park" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653745416015876, "text": "How tf did I end up on Mill tonight?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2197217905, "name": "Willie Martinez", "screen_name": "MetalWakiza", "lang": "en", "location": "King Kai's Planet", "create_at": date("2013-11-15"), "description": "'Savor The Suffering' - Coming soon to an Apple Music subscription near you.", "followers_count": 182, "friends_count": 126, "statues_count": 10796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653745420337152, "text": "Temp 44.9° Hi/Lo 49.7/44.9 Rng 4.8° WC 44.9° Hmd 84% Rain 0.00\" Storm 0.00\" BAR 30.168 Rising DP 40.4° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 122, "statues_count": 18625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653745776852992, "text": "https://t.co/Q6ywUh6DLQ\n2 DIE 5/12\nYOUNG\nLAB RETR \nMISTREATED\nINJURED\nHELP!\n#RESCUE\n#FOSTER\nHELPDOGS@URGENTPODR.ORG https://t.co/t0Uox4EQ2o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RESCUE", "FOSTER" }}, "user": { "id": 2342752717, "name": "Ann", "screen_name": "gaviota330", "lang": "en", "location": "VA.US", "create_at": date("2014-02-13"), "description": "animal lover all the way...", "followers_count": 4515, "friends_count": 3504, "statues_count": 62508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McLean, VA", "id": "dc234665a759a05d", "name": "McLean", "place_type": "city", "bounding_box": rectangle("-77.287868,38.893115 -77.119901,38.981856") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5148376, "cityName": "McLean" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653746338889728, "text": "Wind 0.0 mph ---. Barometer 30.17 in, Falling slowly. Temperature 48.6 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 120, "statues_count": 159729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653746716237828, "text": "Me and Nikko been plotting on how to make a change", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231339609, "name": "KLM", "screen_name": "kennymatthews5", "lang": "en", "location": "LobCity ", "create_at": date("2010-12-27"), "description": "make moves. I'm tryna network you feel me. ASU '18", "followers_count": 857, "friends_count": 570, "statues_count": 59781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653747014209536, "text": "Now I really can't sleep Bruh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473105198, "name": "ShimmyTime™", "screen_name": "WellHelloDre", "lang": "en", "location": "ATL ", "create_at": date("2012-01-24"), "description": "UWG | Aspiring Photgrapher | MVO™ | IG : Bewareofdre |", "followers_count": 771, "friends_count": 824, "statues_count": 15789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollton, GA", "id": "7bc33682b217b5a1", "name": "Carrollton", "place_type": "city", "bounding_box": rectangle("-85.128063,33.52676 -85.009976,33.633403") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13045, "countyName": "Carroll", "cityID": 1313492, "cityName": "Carrollton" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653747253252096, "text": ".@KylieJenner is very Jenner-ous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236699098 }}, "user": { "id": 35349109, "name": "Kristie Dash", "screen_name": "kristiedash", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-25"), "description": "digital beauty editor @allure_magazine • snapchat+instagram @kristiedash", "followers_count": 3219, "friends_count": 640, "statues_count": 14810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-12T00:00:02.000Z"), "id": 730653747479617537, "text": "mad? nah �� https://t.co/eMKkmcQJgi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2169705749, "name": "kT✨", "screen_name": "macdk__", "lang": "en", "location": "null", "create_at": date("2013-11-05"), "description": "stuck between caring too much, and not caring at all.", "followers_count": 422, "friends_count": 142, "statues_count": 10686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pampa, TX", "id": "3ed7a0157e854901", "name": "Pampa", "place_type": "city", "bounding_box": rectangle("-100.996623,35.518467 -100.927607,35.583033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48179, "countyName": "Gray", "cityID": 4854912, "cityName": "Pampa" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653747873906688, "text": "Wo liegt Des Moines? https://t.co/gjZxandzu4 Moines #Des Moines #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.6203,41.5888"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Des", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 472, "friends_count": 116, "statues_count": 1926950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653748071043072, "text": "@KyleWorldd i wanna see the studies done to show this", "in_reply_to_status": 730653646279585792, "in_reply_to_user": 333223848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 333223848 }}, "user": { "id": 624412323, "name": "Nini Boo", "screen_name": "sinn_citayy", "lang": "en", "location": "Oakland, CA", "create_at": date("2012-07-01"), "description": "I really love being Tongan. This me telling you from the jump, I AINT SHIT. #RaiderNation #TRESQUAH", "followers_count": 1275, "friends_count": 553, "statues_count": 76122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653748129763329, "text": "#empire always gotta have a cliff hanger ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "empire" }}, "user": { "id": 2783129778, "name": "Princessseggiee✨", "screen_name": "Seeeggeeenn", "lang": "en", "location": "Long Beach ", "create_at": date("2014-08-31"), "description": "❣Eritrean❣", "followers_count": 2511, "friends_count": 244, "statues_count": 16695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653748595294209, "text": "Drunk sex really real deal brings out all the inner freakiness in yo ass bruh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275838933, "name": "mattieb GATES", "screen_name": "Mattieb___", "lang": "en", "location": "null", "create_at": date("2011-04-01"), "description": "Follow me on IG @mattieb___ !!! follow me on snapchat at @mattiefuckingb", "followers_count": 2782, "friends_count": 996, "statues_count": 131285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653749232852992, "text": "When you guys make quesadillas, what kind of tortilla and cheese do you like to use?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 80422499, "name": "Steph", "screen_name": "StephhhGuerra", "lang": "en", "location": "Los Angeles, CA ", "create_at": date("2009-10-06"), "description": "null", "followers_count": 798, "friends_count": 1064, "statues_count": 33838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653749509640192, "text": "Shit it ain't how it always seems when its so together.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3005910458, "name": "Finesse kid", "screen_name": "Faizan__g", "lang": "en", "location": "Sacramento, CA", "create_at": date("2015-01-31"), "description": "I might talk that real if you ask me what I care about", "followers_count": 344, "friends_count": 327, "statues_count": 7120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653749543342080, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":61.3°F Wind:0.9mph Pressure: 30.09hpa Falling Rain Today 0.25in. Forecast: Occasional precipitation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 17, "statues_count": 317610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653750092697601, "text": "51c48f3100pB1FF3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-102.308126,34.578257"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 24, "friends_count": 0, "statues_count": 20822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas, USA", "id": "e0060cda70f5f341", "name": "Texas", "place_type": "admin", "bounding_box": rectangle("-106.645646,25.837092 -93.508131,36.500695") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48069, "countyName": "Castro" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653750482862080, "text": "I don't have it all together , but I don't want to", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069295642, "name": "Kaelyn Kowalik", "screen_name": "kaelyn_kowalik", "lang": "en", "location": "Buffalo, NY", "create_at": date("2013-01-07"), "description": "I don't talk loud enough, and my name doesn't have a T in it", "followers_count": 815, "friends_count": 1138, "statues_count": 5093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blasdell, NY", "id": "015b41f2459305d0", "name": "Blasdell", "place_type": "city", "bounding_box": rectangle("-78.864178,42.767924 -78.789953,42.806852") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3606849, "cityName": "Blasdell" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653750935719936, "text": "#17!!!!������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1856000348, "name": "may 12th ;)", "screen_name": "kailynatkins99", "lang": "en", "location": "arizona", "create_at": date("2013-09-11"), "description": "ala junior and varsity cheer", "followers_count": 345, "friends_count": 166, "statues_count": 4746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653751250391040, "text": "Wish Batman vs Superman was still in theaters. Would see that again Friday and then Civil War again too", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408869965, "name": "Skyrim and chill?", "screen_name": "Aguyinachair", "lang": "en", "location": "New London, CT", "create_at": date("2011-11-09"), "description": "I'm not Hispanic but my name is Carlos. Follow me. Emily ❤️", "followers_count": 1078, "friends_count": 525, "statues_count": 91392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New London, CT", "id": "6e9c054da607b539", "name": "New London", "place_type": "city", "bounding_box": rectangle("-72.12852,41.304762 -72.087716,41.385244") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 952280, "cityName": "New London" } }
+{ "create_at": datetime("2016-05-12T00:00:03.000Z"), "id": 730653751648866304, "text": "#DrunkPerson at University Blvd, Winter Park, FL 32792. #orlpol #ocso", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2887112,28.597572"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DrunkPerson", "orlpol", "ocso" }}, "user": { "id": 40390214, "name": "Police Calls 32792", "screen_name": "orlpol32792", "lang": "en", "location": "Orlando-ish", "create_at": date("2009-05-15"), "description": "Unofficial. Calls to Orlando-area police, tweeted in near real-time.", "followers_count": 113, "friends_count": 1, "statues_count": 4948 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goldenrod, FL", "id": "4e644cf413b4c021", "name": "Goldenrod", "place_type": "city", "bounding_box": rectangle("-81.309484,28.597471 -81.26758,28.625333") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1226475, "cityName": "Goldenrod" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653752261148672, "text": "Gilmore girls is so deceiving, like there aren't that many people who like good music hanging around or is it just in Connecticut? Doubt it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 112262252, "name": "Celina McManus", "screen_name": "celinamcmanus", "lang": "en", "location": "St Paul, MN", "create_at": date("2010-02-07"), "description": "writer/silliness", "followers_count": 211, "friends_count": 153, "statues_count": 3463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653752596660224, "text": "@macysorrelle hello boo who are you - @KodakBlack1k", "in_reply_to_status": 730651270483042304, "in_reply_to_user": 2559750184, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2559750184, 2163267319 }}, "user": { "id": 1587583507, "name": "Marco Flores", "screen_name": "MarcoAnthonyJr", "lang": "en", "location": "LV-SJ", "create_at": date("2013-07-11"), "description": "Boricua", "followers_count": 6827, "friends_count": 1143, "statues_count": 17287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653752839921664, "text": "Why there's no jollibee hereeeee?!?! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2994573925, "name": "Vianney.", "screen_name": "bitterbayani", "lang": "en", "location": "Florida, USA", "create_at": date("2015-01-24"), "description": "dangerous bitch ❌ USCSC-JUNIOR ❌ sc bitterbayani", "followers_count": 394, "friends_count": 336, "statues_count": 11380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Charlotte, FL", "id": "9ea56e2dd549b99e", "name": "Port Charlotte", "place_type": "city", "bounding_box": rectangle("-82.174795,26.953081 -82.057555,27.032115") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12015, "countyName": "Charlotte", "cityID": 1258350, "cityName": "Port Charlotte" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653753401966592, "text": "I'm not asking for the world, maybe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167542736, "name": "Bina bear", "screen_name": "sabrinacampos98", "lang": "en", "location": "null", "create_at": date("2013-02-10"), "description": "Gabbs is my go to http://ask.fm/Bina415", "followers_count": 327, "friends_count": 612, "statues_count": 24673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653753729114113, "text": "T-minus 48 hours until I turn 21 #ohfuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ohfuck" }}, "user": { "id": 549527954, "name": "Marc Calabrese™", "screen_name": "marccalabeast", "lang": "en", "location": "null", "create_at": date("2012-04-09"), "description": "Sac State. ΠΚΑ I'm just tryin to get verified", "followers_count": 368, "friends_count": 313, "statues_count": 4268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemont, CA", "id": "af828ecae0dcb2be", "name": "Rosemont", "place_type": "city", "bounding_box": rectangle("-121.385837,38.528987 -121.334884,38.566098") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 662910, "cityName": "Rosemont" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754005946368, "text": "Going to #Vegas this #friday 5/13/16 doing live sessions book me goddessnaughtia@gmail.com https://t.co/PY5Xra5ZE9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Vegas", "friday" }}, "user": { "id": 1112829854, "name": "Goddess Naughtia", "screen_name": "GoddessNaughtia", "lang": "en", "location": "United States", "create_at": date("2013-01-22"), "description": "Dom Porn Work... modeling. Let me know if you would like to book me. goddessnaughtia@gmail.com", "followers_count": 7841, "friends_count": 7415, "statues_count": 4078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson Estates, AZ", "id": "007aadc334b5b0b2", "name": "Tucson Estates", "place_type": "city", "bounding_box": rectangle("-111.183619,32.155095 -111.070647,32.205374") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477035, "cityName": "Tucson Estates" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754131779584, "text": "You're the most loyal side nigga I ever met @Grand__papi loyal to your girl even tho she got a bf ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1409080423 }}, "user": { "id": 722662099139354624, "name": "DANIEL☭ | | ᆾ", "screen_name": "danielolivas___", "lang": "en", "location": "null", "create_at": date("2016-04-19"), "description": "IG: danielolivas___", "followers_count": 178, "friends_count": 191, "statues_count": 910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754270355456, "text": "I always wake up at this time lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352860132, "name": "The Captain☔️", "screen_name": "1drop0", "lang": "en", "location": "Mt. Vernon, OH", "create_at": date("2011-08-10"), "description": "Leif and Candi's son. Die hard Browns (field goal) fan. sc: rainwiggand10", "followers_count": 640, "friends_count": 605, "statues_count": 15285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, OH", "id": "43540e805ce7385d", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-82.519942,40.353026 -82.418632,40.439491") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39083, "countyName": "Knox", "cityID": 3953102, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754408632320, "text": "I wish I had more time to take selfies cause I look cute 24/7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91937771, "name": "✨✨joana✨✨", "screen_name": "joana_valdez", "lang": "en", "location": "San Jose, CA", "create_at": date("2009-11-22"), "description": "sophisticated with a hint of slutty", "followers_count": 324, "friends_count": 232, "statues_count": 46518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754505093121, "text": "You don't even know me but yet you have so much to say about me��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1124434784, "name": "VLAFS™", "screen_name": "VeeSoKrucial", "lang": "en", "location": "South Seattle , Wa☔", "create_at": date("2013-01-27"), "description": "R.I.L MY SISTER Ailiana F.Siufanua @Krooshtm #itsaKrucialWorld25/8 #KTM400✊ 1/5/97 - 11/30/15 I love you so much❤", "followers_count": 1330, "friends_count": 1114, "statues_count": 46798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340391 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653754555404288, "text": "Brb just rediscovering my undying love for The All-American Rejects #swingswing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "swingswing" }}, "user": { "id": 2894107681, "name": "Colette Sobczak", "screen_name": "cole_sob", "lang": "en", "location": "San Francisco, CA", "create_at": date("2014-11-08"), "description": "burrito enthusiast, floral advocate, frequent frollicker, probably hungry, usfca Θ", "followers_count": 147, "friends_count": 139, "statues_count": 701 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653755113246720, "text": "Wind 8.0 mph E. Barometer 29.990 in, Rising slowly. Temperature 64.5 °F. Rain today 0.17 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653755398619137, "text": "& Y'all wonder why y'all gf be mad when y'all leave the house...streets too damn dangerous.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297237143, "name": "Tra-ne-cee-ya.", "screen_name": "SkinnyPENNY_", "lang": "en", "location": "null", "create_at": date("2011-05-11"), "description": "Kylen & Kamiya ❤️", "followers_count": 2727, "friends_count": 571, "statues_count": 92369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-12T00:00:04.000Z"), "id": 730653755792756738, "text": "@ThaBroLilG lol I got it broz!", "in_reply_to_status": 730652374117822464, "in_reply_to_user": 230679256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 230679256 }}, "user": { "id": 3015914254, "name": "Jacob Alonzo", "screen_name": "JacobAlonzo10", "lang": "en", "location": "null", "create_at": date("2015-02-03"), "description": "null", "followers_count": 276, "friends_count": 296, "statues_count": 233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653756166000642, "text": "Nothing better than eating pasta and turning 18 at the same time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1077619609, "name": "Natan Lizier-Zmud.", "screen_name": "ThePolishMafia", "lang": "en", "location": "null", "create_at": date("2013-01-10"), "description": "catch me repping the adventure hat", "followers_count": 327, "friends_count": 223, "statues_count": 1023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Grove, OR", "id": "beedf9d8e2499b64", "name": "Forest Grove", "place_type": "city", "bounding_box": rectangle("-123.15354,45.501953 -123.070256,45.542318") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4126200, "cityName": "Forest Grove" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653756677722112, "text": "@C091801 @ItsSteelFafa si allen mascot AHAHAHAHAH JK LANG", "in_reply_to_status": 730653476523364354, "in_reply_to_user": 2399786689, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 2399786689, 122263704 }}, "user": { "id": 2884409262, "name": "天才", "screen_name": "_jericbabasa", "lang": "en", "location": "Manila, PH • Kanagawa, JPN", "create_at": date("2014-10-30"), "description": "null", "followers_count": 251, "friends_count": 352, "statues_count": 13892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653756807872512, "text": "@CoachJackCurtis https://t.co/8fOxSF9143❗️��", "in_reply_to_status": -1, "in_reply_to_user": 1480552902, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1480552902 }}, "user": { "id": 328286817, "name": "Mr. Wright", "screen_name": "Jah_LND", "lang": "en", "location": "Union Cohnty NJ #9️⃣0️⃣8️⃣", "create_at": date("2011-07-02"), "description": "Football CB #908 LND ✊ #MonroeMustangz 〽️onroe College", "followers_count": 999, "friends_count": 2533, "statues_count": 7013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Rochelle, NY", "id": "11cdb607ab2f17bb", "name": "New Rochelle", "place_type": "city", "bounding_box": rectangle("-73.807197,40.881294 -73.755414,40.988329") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3650617, "cityName": "New Rochelle" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653756841467904, "text": "https://t.co/VhsgrY0nDU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3786181156, "name": "marshall mathers", "screen_name": "Lil_A1", "lang": "en", "location": "Manhattan, NY", "create_at": date("2015-09-26"), "description": "deletedwwddd this is not Lila unfollow", "followers_count": 401, "friends_count": 421, "statues_count": 6135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653757378199552, "text": "@_luiss6996 you sure ����", "in_reply_to_status": 730653262592937988, "in_reply_to_user": 2967699200, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2967699200 }}, "user": { "id": 868597062, "name": "✨", "screen_name": "__cloudd", "lang": "en", "location": "South Houston, TX", "create_at": date("2012-10-08"), "description": "sc: x_cloudd", "followers_count": 679, "friends_count": 621, "statues_count": 27601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Houston, TX", "id": "54614bf2dabf5a43", "name": "South Houston", "place_type": "city", "bounding_box": rectangle("-95.249558,29.650964 -95.20787,29.670399") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4869020, "cityName": "South Houston" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653757474623488, "text": "@jesserutherford I have mommy issues .. But perfect great performance from you and the band ! @thenbhd https://t.co/rmfc1DmQKx", "in_reply_to_status": -1, "in_reply_to_user": 893467212, "favorite_count": 0, "coordinate": point("-118.243616,34.051751"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 893467212, 352689883 }}, "user": { "id": 2556516143, "name": "Luisxo", "screen_name": "luis_tesfaye", "lang": "en", "location": "NY|XO|TO|XØ|LA|xo|SF", "create_at": date("2014-05-20"), "description": "Saw Abel @Theweeknd october.9th,2014 and at The Madness Tour back to back 12/08/15-12/09/15 @reBELLYus followed 7/16/15 #1993 Saw Abel dec.8 & 9th . XO", "followers_count": 313, "friends_count": 285, "statues_count": 6451 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Microsoft Theater", "id": "08def14de3570000", "name": "Microsoft Theater", "place_type": "poi", "bounding_box": rectangle("-118.2436161,34.0517509 -118.243616,34.051751") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758015692801, "text": "The way my psycho ass is set up.... https://t.co/Ipg3JUN5Km", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388045589, "name": "heav", "screen_name": "Heavyn_d", "lang": "en", "location": "null", "create_at": date("2011-10-09"), "description": "Mason's Mom", "followers_count": 1660, "friends_count": 823, "statues_count": 36448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758254923776, "text": "If I ever loved you I'll always love you that just how I was raised", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2779112331, "name": "Breon Robinson", "screen_name": "Foo_D_at_night", "lang": "en", "location": "the woodz", "create_at": date("2014-09-21"), "description": "new twitter new foo", "followers_count": 306, "friends_count": 411, "statues_count": 2278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitehall, OH", "id": "0fdd28012cbb8a2d", "name": "Whitehall", "place_type": "city", "bounding_box": rectangle("-82.91273,39.950926 -82.851156,39.986808") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3984742, "cityName": "Whitehall" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758548402176, "text": "Happy 20th birthday to this fine young woman aka @SaharaDip https://t.co/AJCkJhb2Wh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 375765580 }}, "user": { "id": 344517057, "name": "WyWy☔️", "screen_name": "WyattLarsen", "lang": "en", "location": "Seatown ", "create_at": date("2011-07-28"), "description": "''Some people want it to happen, some wish it happen, others make it happen.'' -Michael Jordan #NikeFam|Sneakerhead|01•12•14|", "followers_count": 392, "friends_count": 389, "statues_count": 7535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastmont, WA", "id": "002d396055a6adf8", "name": "Eastmont", "place_type": "city", "bounding_box": rectangle("-122.234054,47.860402 -122.14427,47.921824") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5319630, "cityName": "Eastmont" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758560993280, "text": "2 AM time to knock out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369681689, "name": "#EID", "screen_name": "Heidddd", "lang": "en", "location": "Austin, TX", "create_at": date("2014-03-02"), "description": "you've never met anyone like me", "followers_count": 119, "friends_count": 135, "statues_count": 1439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758611283969, "text": "good night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 460440313, "name": "Dorenea", "screen_name": "BeauDEE_42", "lang": "en", "location": "Little Mexico, TX", "create_at": date("2012-01-10"), "description": "null", "followers_count": 712, "friends_count": 434, "statues_count": 55673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653758703554560, "text": "@JohnnySan1259 https://t.co/kmGciwzVqs", "in_reply_to_status": -1, "in_reply_to_user": 730564453385658368, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 730564453385658368 }}, "user": { "id": 2835779254, "name": "Cd.", "screen_name": "cedariusluttery", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-10-17"), "description": "Only Originals Records Artist | Business Inquiries: cluttery94@gmail.com | Follow Twitter & Instagram: @cedariusluttery\n#90sbaby #onlyoriginals", "followers_count": 1060, "friends_count": 839, "statues_count": 32071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653759693455360, "text": "I keep asking if I can stay the night with Dallas & my �� thinks I'm going to Dallas for the night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 562545564, "name": "Abigail Solares", "screen_name": "Killaabi_", "lang": "en", "location": "Texas, USA", "create_at": date("2012-04-24"), "description": "null", "followers_count": 1266, "friends_count": 1217, "statues_count": 58561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653759869726720, "text": "I'll never talk to another nigga from Hattiesburg ... sorry not sorry .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2589514737, "name": "kimmm ...", "screen_name": "_thekimm", "lang": "en", "location": "Jackson - Hattiesburg", "create_at": date("2014-06-07"), "description": "UnivOfSouthernMiss", "followers_count": 1990, "friends_count": 1583, "statues_count": 36508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hattiesburg, MS", "id": "27d0ee77b9d28896", "name": "West Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.428655,31.292272 -89.351895,31.340455") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28073, "countyName": "Lamar", "cityID": 2878890, "cityName": "West Hattiesburg" } }
+{ "create_at": datetime("2016-05-12T00:00:05.000Z"), "id": 730653760243019781, "text": "Thursday 5-12-16 https://t.co/Xk0UpRhxhD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1148, "friends_count": 324, "statues_count": 3595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653760649895936, "text": "coverup wish list https://t.co/TKvfTUt8FA #ontheblog #fashion #fashionblogger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2767995,39.9831886"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontheblog", "fashion", "fashionblogger" }}, "user": { "id": 65940718, "name": "Brett Elizabeth", "screen_name": "BrettEParker", "lang": "en", "location": "NYC", "create_at": date("2009-08-15"), "description": "i ❤ nyc.", "followers_count": 141, "friends_count": 226, "statues_count": 2204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18011, "countyName": "Boone", "cityID": 1886372, "cityName": "Zionsville" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653760804950017, "text": "I play all types of games ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1187680580, "name": "HeavenMommy", "screen_name": "_SliM_ReD_", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "null", "followers_count": 573, "friends_count": 343, "statues_count": 9502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653760863670272, "text": "Fuck everyone, NO ONE talk to me tomorrow bye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2432872158, "name": "lingling", "screen_name": "ElexisHuerta", "lang": "en", "location": "Mercedes, TX", "create_at": date("2014-04-07"), "description": "mhs // blah", "followers_count": 1374, "friends_count": 685, "statues_count": 36487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mercedes, TX", "id": "c9af03f7af638bed", "name": "Mercedes", "place_type": "city", "bounding_box": rectangle("-97.958308,26.123747 -97.863735,26.17475") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4847700, "cityName": "Mercedes" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653760943443968, "text": "How Sway�� https://t.co/4Ucqikpmfs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1539021918, "name": "Demetrique", "screen_name": "_YoungKing96", "lang": "en", "location": "Miami, FL", "create_at": date("2013-06-22"), "description": "20 | FIU '18 | RIP Granny", "followers_count": 534, "friends_count": 268, "statues_count": 21679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakeland, FL", "id": "3f7a925ec706ea48", "name": "Lakeland", "place_type": "city", "bounding_box": rectangle("-82.042715,27.968692 -81.902695,28.13051") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1238250, "cityName": "Lakeland" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653761333387264, "text": "Curry will have trouble with Kyrie and Russell. That's a fact", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596850386, "name": "ant", "screen_name": "anthony_garcia6", "lang": "en", "location": "Jomarie❤️", "create_at": date("2012-06-01"), "description": "#D-TEAM", "followers_count": 519, "friends_count": 391, "statues_count": 16253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653761375371264, "text": "Eh worse season I ended up skipping to the last episode of it and then watching the rest https://t.co/dR81Vy8OtH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2318260023, "name": "Peyton Mathers", "screen_name": "peytonroweeee", "lang": "en", "location": "D109", "create_at": date("2014-02-01"), "description": "#southernnotstate #GSU19'\r\nblack nerd. feminist. Jamaican.\ntheatre is life.", "followers_count": 391, "friends_count": 417, "statues_count": 7969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Johns Creek, GA", "id": "00975c9578f9e109", "name": "Johns Creek", "place_type": "city", "bounding_box": rectangle("-84.286258,33.985534 -84.097879,34.0905") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1342425, "cityName": "Johns Creek" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653762889519104, "text": "#diabetic #t1d Les Merveilleux Bienfaits Du Bicarbonate De Soude: Les Merveilleux… https://t.co/TuqJ7jbUou #type1diabetes #diabetes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99841309,40.70640872"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "diabetic", "t1d", "type1diabetes", "diabetes" }}, "user": { "id": 4835709563, "name": "James Wolter", "screen_name": "Diabetes_Newzz", "lang": "en", "location": "United States", "create_at": date("2016-01-22"), "description": "Hi, James here, I focus on providing a comprehensive, supportive and independent experience for my twitter visitors around Diabetes and help for Diabetes.", "followers_count": 813, "friends_count": 1886, "statues_count": 26820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763078356993, "text": "Wind 0.0 mph ---. Barometer 30.037 in, Rising slowly. Temperature 64.5 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 40, "friends_count": 4, "statues_count": 27358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763107577856, "text": "Omg I won the photoshoot with xotixlyou. This is crazy, a whole make up and hair team, I'm so excited! This is not bragging but I needed it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 888311845, "name": "Kriss☂", "screen_name": "kristadross", "lang": "en", "location": "california", "create_at": date("2012-10-18"), "description": "your moms chest hair", "followers_count": 268, "friends_count": 268, "statues_count": 1882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763279577089, "text": "Wind 0 mph ---. Barometer 1014.7 hPa, Falling. Temperature 73.2 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-67.18027778,18.11361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 70194826, "name": "Osvaldo Aviles", "screen_name": "osvaldoa69", "lang": "en", "location": "Joyudas,Cabo Rojo P.R.", "create_at": date("2009-08-30"), "description": "null", "followers_count": 88, "friends_count": 264, "statues_count": 156226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miradero, USA", "id": "588c877884720a1b", "name": "Miradero", "place_type": "city", "bounding_box": rectangle("-67.204432,18.059985 -67.14734,18.120612") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72023, "countyName": "Cabo Rojo" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763409575936, "text": "accuracy https://t.co/6sXzqH9iuE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 725229758, "name": "pep", "screen_name": "chilipep95", "lang": "en", "location": "cheer", "create_at": date("2012-07-29"), "description": "make some damn notes", "followers_count": 571, "friends_count": 532, "statues_count": 26242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763497656320, "text": "Just because the Warriors don't cave in and crumble when Steph doesn't play doesn't mean he isn't the mvp..we just have a squad lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259550589, "name": "Dexter Reed", "screen_name": "theruler54", "lang": "en", "location": "null", "create_at": date("2011-03-01"), "description": "Adult transition Jedi", "followers_count": 1129, "friends_count": 884, "statues_count": 57428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763535577089, "text": "Lmfao okay! And same but now I just hate mines they're extremely too big for me. I'm getting a breast reduction �� https://t.co/e42Et8ZbbM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325930365, "name": "5'10 asshole", "screen_name": "MissAB__", "lang": "en", "location": "at the church praying.", "create_at": date("2011-06-28"), "description": "Im a asshole but I care.. of some sort. Most relaxed, goofiest, craziest girl ever. Coolest youngin you'll ever meet. Im 5'10 . #Bennett ➡️ #ncat #GxldenGods", "followers_count": 4926, "friends_count": 3731, "statues_count": 53506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653763724173313, "text": "I only have one day off this week ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441568204, "name": "jaime", "screen_name": "jaimeeverlark", "lang": "en", "location": "null", "create_at": date("2011-12-19"), "description": "some will fight", "followers_count": 1038, "friends_count": 75, "statues_count": 21592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covington, WA", "id": "397854265ee65aa6", "name": "Covington", "place_type": "city", "bounding_box": rectangle("-122.144491,47.345144 -122.058504,47.395662") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5315290, "cityName": "Covington" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653764022046721, "text": "Reminiscing ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374000190, "name": "Sweet Honey Ice Tea'", "screen_name": "_BeautyBlu", "lang": "en", "location": "null", "create_at": date("2011-09-15"), "description": "Professional #Model ✨ IG/SC: Royce2Fierce #LèBeaúty✨", "followers_count": 1525, "friends_count": 1008, "statues_count": 52673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-05-12T00:00:06.000Z"), "id": 730653764277796864, "text": "I'm telling y'all I cut ratchet Kayla 2 days ago she's faded away", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185427779, "name": "IG: Officialkaymezo", "screen_name": "vanellope_kayla", "lang": "en", "location": "Bay Area,Cali ", "create_at": date("2010-08-31"), "description": "Stylist| Head of the Fashion Department of #ProjectLevel | Model | Actress | Event Host ❤️ Booking info: bigrich@projectlevel.org", "followers_count": 1046, "friends_count": 482, "statues_count": 20597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016123488014336, "text": "I get emotionally invested into tv shows so much smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44546397, "name": "ZoeWop ", "screen_name": "Hollywood4663", "lang": "en", "location": "FL.A.TL", "create_at": date("2009-06-03"), "description": "On This Savage Journey|Jumpin Out The Gym Every Time |OrangeCounty Bred|Retired FAMU Striker|Full Time Haitian|Dont Forget I'm Not Winning You Just Losing", "followers_count": 2391, "friends_count": 2229, "statues_count": 153326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norcross, GA", "id": "d8cf3d3b242f7221", "name": "Norcross", "place_type": "city", "bounding_box": rectangle("-84.240309,33.912656 -84.1745,33.968805") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13135, "countyName": "Gwinnett", "cityID": 1355776, "cityName": "Norcross" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016123634720768, "text": "3 weeks �� https://t.co/OBn3gYbTqd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1422343452, "name": "Emmanuel Perez", "screen_name": "emanperez14", "lang": "en", "location": "Berkeley, CA", "create_at": date("2013-05-11"), "description": "UC Berkeley Student | Sports Aficionado | Political Junkie | Dog Lover", "followers_count": 320, "friends_count": 555, "statues_count": 19795 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Berkeley, CA", "id": "5ef5b7f391e30aff", "name": "Berkeley", "place_type": "city", "bounding_box": rectangle("-122.324818,37.845953 -122.234225,37.905738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 606000, "cityName": "Berkeley" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016123894894592, "text": "@TheDJLeek was lit like always ��", "in_reply_to_status": -1, "in_reply_to_user": 39039273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39039273 }}, "user": { "id": 1621559780, "name": "Brooke Alexis", "screen_name": "_BrookeAlexis_", "lang": "en", "location": "null", "create_at": date("2013-07-25"), "description": "Upper Echelon|19•614•. CSU20", "followers_count": 1072, "friends_count": 714, "statues_count": 36315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016124033159168, "text": "@ebbtideapp Tide in Hunters Point, California 05/13/2016\n Low 12:29am 2.4\nHigh 5:44am 5.5\n Low 12:39pm 0.1\nHigh 7:51pm 5.9", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-122.35,37.7333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 32347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016124272279552, "text": "Them question threads dumb as hell like we don't gaf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226213175, "name": "Teen Tha Finesse Kid", "screen_name": "__teeeen", "lang": "en", "location": "null", "create_at": date("2015-05-01"), "description": "Young nigga tryna make his momma proud!", "followers_count": 377, "friends_count": 386, "statues_count": 4189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016126033846272, "text": "Wind 0.0 mph ---. Barometer 1025.10 mb, Steady. Temperature 51.7 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-13T00:00:00.000Z"), "id": 731016126637858817, "text": "��������omgggg https://t.co/sHpBF7kGsL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1095315360, "name": "#prettyASF", "screen_name": "seauxmaijaa", "lang": "en", "location": "LIVING!", "create_at": date("2013-01-16"), "description": "your favorite IT girl", "followers_count": 1199, "friends_count": 1339, "statues_count": 14332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plaquemine, LA", "id": "36c83b3a759e987b", "name": "Plaquemine", "place_type": "city", "bounding_box": rectangle("-91.26889,30.250883 -91.21174,30.343376") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22047, "countyName": "Iberville", "cityID": 2260880, "cityName": "Plaquemine" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016127325724672, "text": "don't be generous to all the wrong people��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059496802, "name": "tiana", "screen_name": "Shehasdimpless", "lang": "en", "location": "null", "create_at": date("2013-01-03"), "description": "What you see is what you get ♉️", "followers_count": 1191, "friends_count": 917, "statues_count": 41450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Segundo, CA", "id": "0654b676d0359a31", "name": "El Segundo", "place_type": "city", "bounding_box": rectangle("-118.429843,33.901804 -118.370685,33.931493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622412, "cityName": "El Segundo" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016127623659520, "text": "@amber_funk how dare you just now follow me!", "in_reply_to_status": -1, "in_reply_to_user": 221851384, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 221851384 }}, "user": { "id": 3235029929, "name": "parker shaffer", "screen_name": "parkershaffer", "lang": "en", "location": "lakewood, oh", "create_at": date("2015-05-04"), "description": "no love for the world", "followers_count": 1121, "friends_count": 454, "statues_count": 1892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, OH", "id": "888482aa70a3bc61", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-81.8375,41.463245 -81.768603,41.49759") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3941664, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016127736729602, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/e7xOO53JLL #RN #Nursing #Bridgewater, MA #Hiring https://t.co/V732EhRzVy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.9750541,41.9903519"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "RN", "Nursing", "Bridgewater", "Hiring" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 216, "friends_count": 100, "statues_count": 4333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bridgewater, MA", "id": "006b91cb41e4a9cc", "name": "Bridgewater", "place_type": "city", "bounding_box": rectangle("-71.036947,41.929851 -70.898031,42.010968") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2508050, "cityName": "Bridgewater" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128030486528, "text": "I feel like my dick way valuable than some y'all pussies out here��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41916656, "name": "That Boy Cortez✊", "screen_name": "Cortez_AirBorne", "lang": "en", "location": "null", "create_at": date("2009-05-22"), "description": "|Host|•++", "followers_count": 3312, "friends_count": 2028, "statues_count": 56796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128051458049, "text": "Temp: 62.4F W C: 62.4F Wind:SE at 3.2kts Baro: 1018.4mb and Falling slowly Rain today: 0.19in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 212, "friends_count": 218, "statues_count": 104954 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128181473280, "text": "�������� already lmao https://t.co/rtbrG1OvQn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 309922057, "name": "funny fat guy", "screen_name": "OMG_ItsKhairy", "lang": "en", "location": "Baltimore, MD", "create_at": date("2011-06-02"), "description": "Cartoon Connoisseur #FatBoyNation #FatBoyPresident email:M.Cr33k@yahoo.com", "followers_count": 6673, "friends_count": 3479, "statues_count": 79195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128244404224, "text": "@AlBernstein @sicvic24 @TheRoot Thanks.", "in_reply_to_status": 731009387263729665, "in_reply_to_user": 155736983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 155736983, 74837469, 23995748 }}, "user": { "id": 328739719, "name": "Herman Sims", "screen_name": "Hrsims1958", "lang": "en", "location": "null", "create_at": date("2011-07-03"), "description": "null", "followers_count": 28, "friends_count": 125, "statues_count": 1114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128277950464, "text": "Ripley SW Limestone Co. Temp: 65.5°F Wind:2.2mph Pressure: 998.3mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53277 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128340713472, "text": "Wind 0.0 mph ---. Barometer 30.053 in, Rising. Temperature 68.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128365875200, "text": "@luis_mothafukaa @goddessnessa_ @drea_arte haha nunca dormimos����", "in_reply_to_status": 731015962166591488, "in_reply_to_user": 1629553724, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 1629553724, 2292302198, 118532449 }}, "user": { "id": 118532449, "name": "Andrea Arteaga", "screen_name": "drea_arte", "lang": "en", "location": "null", "create_at": date("2010-02-28"), "description": "null", "followers_count": 178, "friends_count": 185, "statues_count": 15600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016128823054337, "text": "@PaloMePaulo with help from you", "in_reply_to_status": 731016073798025217, "in_reply_to_user": 161927357, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 161927357 }}, "user": { "id": 1529223451, "name": "brooke healy", "screen_name": "Healy1Brooke", "lang": "en", "location": "null", "create_at": date("2013-06-18"), "description": "live passionately and sincerely", "followers_count": 359, "friends_count": 359, "statues_count": 9823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Encinitas, CA", "id": "1f6b47c3f3352385", "name": "Encinitas", "place_type": "city", "bounding_box": rectangle("-117.312091,32.999469 -117.195721,33.090549") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 622678, "cityName": "Encinitas" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016129729069056, "text": "Might go get my other 9 nails done tommorow. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144911236, "name": "❁BOSS❁", "screen_name": "BosssssedUpppp_", "lang": "en", "location": "YoungOG", "create_at": date("2010-05-17"), "description": "SnapChat: Beautyrusshhhh", "followers_count": 1174, "friends_count": 784, "statues_count": 84696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenarden, MD", "id": "5542db7392bde9b1", "name": "Glenarden", "place_type": "city", "bounding_box": rectangle("-76.876542,38.912849 -76.821187,38.94025") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2432500, "cityName": "Glenarden" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016129955528704, "text": "@Kikyoyaoi @mummafiedthunde @EverfreeNW Go to bronycon ~", "in_reply_to_status": 731015396418043904, "in_reply_to_user": 191294422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 191294422, 15884736, 406824095 }}, "user": { "id": 1654308212, "name": "Lucas Gaxiola", "screen_name": "Lucas_Gaxiola", "lang": "en", "location": "Murietta, CA", "create_at": date("2013-08-07"), "description": "Voice Actor that Doodles | Horses around Alot | #BoopKing | Reads the Fanfics | Kohai's @DaintyBat @pretear", "followers_count": 617, "friends_count": 314, "statues_count": 40556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murrieta, CA", "id": "c4095a8d389bd271", "name": "Murrieta", "place_type": "city", "bounding_box": rectangle("-117.274492,33.521653 -117.118559,33.612308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 650076, "cityName": "Murrieta" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016130152763393, "text": "Beautiful things: @thelithub published my interview with @reynoldsmichael. https://t.co/0enmB0PiB7\n\n#FerranteFever https://t.co/fRSiynEzmY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FerranteFever" }}, "user_mentions": {{ 2903979971, 18970772 }}, "user": { "id": 240712749, "name": "Francesca Pellas", "screen_name": "franpellas", "lang": "en", "location": "New York City", "create_at": date("2011-01-20"), "description": "Italian (50% Piedmont 50% Genoa). Bookaholic, foodie (@capfocaccia), movie lover. Formerly @codice_codice and @FranceExpo2015, now aspiring pirate. New Yorker.", "followers_count": 948, "friends_count": 390, "statues_count": 13872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016130446270465, "text": "Top it off with some Salt & Straw ice cream. Miss that place and shopping on Nob Hill. Surprise Portland trip in the works for friends bday.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310489487, "name": "Traveling Bites", "screen_name": "TravelingBites", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-06-03"), "description": "Food/Travel/Photography/DogLover/TravelAgentOntheSide", "followers_count": 209, "friends_count": 869, "statues_count": 506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonney Lake, WA", "id": "0001ed7e605ce519", "name": "Bonney Lake", "place_type": "city", "bounding_box": rectangle("-122.214616,47.147327 -122.117391,47.214385") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5307170, "cityName": "Bonney Lake" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016130605682688, "text": "@nellclar its all out of love����", "in_reply_to_status": 731015867710857216, "in_reply_to_user": 345168075, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 345168075 }}, "user": { "id": 288940101, "name": "Tortuga", "screen_name": "_JaneeUnique30_", "lang": "en", "location": "Newman, Cali", "create_at": date("2011-04-27"), "description": "SJCC softball⚾️ | Kappa Wappa", "followers_count": 419, "friends_count": 388, "statues_count": 15257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newman, CA", "id": "bae100a1ab853e56", "name": "Newman", "place_type": "city", "bounding_box": rectangle("-121.037486,37.303666 -121.002063,37.32759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 651140, "cityName": "Newman" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016130660306944, "text": "Wind 0.0 mph ---. Barometer 30.10 in, Steady. Temperature 70.0 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016130937159680, "text": "Wind 0.0 mph ---. Barometer 30.091 in, Steady. Temperature 51.7 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016131020886017, "text": "Header ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 151113113, "name": "Erick. $", "screen_name": "fishalien", "lang": "en", "location": "null", "create_at": date("2010-06-02"), "description": "crushing information into powder then im sniffing it.", "followers_count": 370, "friends_count": 229, "statues_count": 18485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Rancho, NM", "id": "0046bfef79c8e224", "name": "Rio Rancho", "place_type": "city", "bounding_box": rectangle("-106.757623,35.217658 -106.566425,35.374708") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35043, "countyName": "Sandoval", "cityID": 3563460, "cityName": "Rio Rancho" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016131033468928, "text": "Friday the 13th has already screwed me and it's only 2 am. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269427701, "name": "ambreuh", "screen_name": "AmbreaDeeAnn", "lang": "en", "location": "Texas, USA", "create_at": date("2011-03-20"), "description": "ITS SO FLUFFY IM GONNA DIE", "followers_count": 573, "friends_count": 634, "statues_count": 21696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016131113148416, "text": "It's 2 in the morning and my mind is on you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3194514913, "name": "Jax ✨", "screen_name": "Jackie_0410", "lang": "en", "location": "Mercedes, TX", "create_at": date("2015-05-13"), "description": "Proud to be in love with a Marine, a very handsome one too I must add. ❤️", "followers_count": 391, "friends_count": 321, "statues_count": 166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weslaco, TX", "id": "f664c6f63c0bef35", "name": "Weslaco", "place_type": "city", "bounding_box": rectangle("-98.057773,26.111766 -97.942697,26.242157") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4877272, "cityName": "Weslaco" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016131230584834, "text": "That was much needed today ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2956285180, "name": "lizzette llanos❤️", "screen_name": "lillizz_22", "lang": "en", "location": "null", "create_at": date("2015-01-02"), "description": "20| your desire for success should be greater than your fear of failure| Chaffey college ❤️", "followers_count": 179, "friends_count": 144, "statues_count": 5766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-13T00:00:01.000Z"), "id": 731016131301937152, "text": "Super rich kids & novacane https://t.co/DiXwt3qa1M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1439266028, "name": "senia", "screen_name": "soirrabainesey", "lang": "en", "location": "Canoga Park, Los Angeles", "create_at": date("2013-05-18"), "description": "*enter inspirational DJ Khaled quote*", "followers_count": 558, "friends_count": 465, "statues_count": 23814 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016131465650176, "text": "Temp: 70.4°F Wind:0.0mph Pressure: 30.077hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 67, "friends_count": 24, "statues_count": 63951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016131486457857, "text": "Thanks for coming... Please come again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2756117945, "name": "NamikaZe", "screen_name": "live4stake63", "lang": "en", "location": "null", "create_at": date("2014-08-30"), "description": "don't quit", "followers_count": 306, "friends_count": 255, "statues_count": 6678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016131670986753, "text": "I be saying off the wall shit like I have no filter", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1000752408, "name": "muva.", "screen_name": "_BaddestRebel", "lang": "en", "location": "bronx ✈️ triple d", "create_at": date("2012-12-09"), "description": "#GetMonE | Dallas MUA | IG: _BaddestRebel | SC: baddestrebel", "followers_count": 681, "friends_count": 413, "statues_count": 22578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016131755020288, "text": "Temp 51.7° Hi/Lo 56.7/51.6 Rng 5.1° WC 51.7° Hmd 79% Rain 0.00\" Storm 0.00\" BAR 30.001 Falling DP 45.4° Wnd 0mph Dir --- Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 122, "statues_count": 18656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016132212207616, "text": "Wind 1.0 mph W. Barometer 29.998 in, Rising slowly. Temperature 58.4 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133080272896, "text": "Like shits crazy af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348059785, "name": "Slim.", "screen_name": "SincerelySlimm", "lang": "en", "location": "Houston| So Im Made To Lean", "create_at": date("2011-08-03"), "description": "FMOIG | @slim.themodel", "followers_count": 896, "friends_count": 560, "statues_count": 15273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133105475584, "text": "Wind 0.2 mph NW. Barometer 29.857 in, Falling. Temperature 56.5 °F. Rain today 0.00 in. Humidity 55%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133336141824, "text": "Lob me 9 if you real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1619052673, "name": "kira.", "screen_name": "BaddestMuslim", "lang": "en", "location": "Bloodclat, TX", "create_at": date("2013-07-24"), "description": "Caribbean girls run it. |21|", "followers_count": 1991, "friends_count": 202, "statues_count": 69764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133373886468, "text": "Friday the 13th. If you decide to do anything stupid be safe :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 713595486532317184, "name": "kasssss ⛵✨", "screen_name": "kassandra_run", "lang": "en", "location": "Wasco, CA", "create_at": date("2016-03-25"), "description": "you want her, you need her and I will never be her. ❤✨", "followers_count": 121, "friends_count": 108, "statues_count": 2299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wasco, CA", "id": "01643e7e5fce28b7", "name": "Wasco", "place_type": "city", "bounding_box": rectangle("-119.42052,35.572513 -119.328147,35.610926") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 683542, "cityName": "Wasco" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133491490816, "text": "72.9F (Feels: 72.9F) - Humidity: 99% - Wind: 3.1mph SW - Gust: 3.1mph - Pressure: 1016.5mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 239522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133676011520, "text": "we need real r&b, soul artists, the void is open especially male r&b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93788315, "name": "chvlss", "screen_name": "_nolabelchels", "lang": "en", "location": "Soflo, Atl ", "create_at": date("2009-11-30"), "description": "maneuvering in style", "followers_count": 1151, "friends_count": 377, "statues_count": 85826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-13T00:00:02.000Z"), "id": 731016133793468416, "text": "✌ @ Awesomeville https://t.co/mpzGQayy5z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.60502099,42.28718987"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205242955, "name": "Stefan Collier", "screen_name": "Stefan_Collier", "lang": "en", "location": "the Netherlands ", "create_at": date("2010-10-20"), "description": "Dutch film actor, 17 Years Old,\nJochem in @spijtdefilm, De Reünie, Papier hier, Beatrix Oranje,\ninstagram: @stefancollier", "followers_count": 3009, "friends_count": 245, "statues_count": 11210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kalamazoo, MI", "id": "413ef5a0d23bfe4f", "name": "Kalamazoo", "place_type": "city", "bounding_box": rectangle("-85.649602,42.215555 -85.481775,42.365493") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26077, "countyName": "Kalamazoo", "cityID": 2642160, "cityName": "Kalamazoo" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016135890636800, "text": "Omar definitely loves ass ��❤️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630619684, "name": "Leo Reyes Jr.", "screen_name": "LeoReyes1_", "lang": "en", "location": "San Benito, Tx. ", "create_at": date("2012-07-08"), "description": "Keep your face to the sunshine and you cannot see a shadow.", "followers_count": 1726, "friends_count": 579, "statues_count": 55666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Benito, TX", "id": "b8f3a7fd432ec1f1", "name": "San Benito", "place_type": "city", "bounding_box": rectangle("-97.684281,26.07277 -97.575927,26.172319") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4865036, "cityName": "San Benito" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016136112766976, "text": "told you don't bite the hand that fed you.. now you out starving ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 119942569, "name": "†boss❤", "screen_name": "_AmbiSHUN", "lang": "en", "location": "centennial bank stadium ♥", "create_at": date("2010-03-04"), "description": "#stAte19 .. #asu19.. army gal' . | pulchritudinous |", "followers_count": 1511, "friends_count": 866, "statues_count": 91255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, AR", "id": "35189a8fcb82b788", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-90.782468,35.763136 -90.611214,35.894116") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5031, "countyName": "Craighead", "cityID": 535710, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016136381227008, "text": "brb learning every word of young thug's verse on #mixtape #coloringbook #chance3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mixtape", "coloringbook", "chance3" }}, "user": { "id": 3176569318, "name": "Luke Armour", "screen_name": "iamlukearmour", "lang": "en", "location": "California, USA, Earth", "create_at": date("2015-04-17"), "description": "small business owner, passionate human.", "followers_count": 134, "friends_count": 352, "statues_count": 234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016136393920512, "text": "Wind 0.0 mph WNW. Barometer 29.92 in, Falling slowly. Temperature 61.0 °F. Rain today 0.00 in. Humidity 55%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016136649629696, "text": "She wouldn't have done it without me����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 405310249, "name": "⠀⠀⠀⠀⠀⠀⠀⠀⠀sheylin", "screen_name": "xoshey_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-11-04"), "description": "SoCal ☀️", "followers_count": 4491, "friends_count": 3411, "statues_count": 61441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137048084482, "text": "Wind 0.0 mph ---. Barometer 30.146 in, Falling slowly. Temperature 62.7 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137283141632, "text": "I refuse to debate with someone about someone else when neither one of us know anything about it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 574957768, "name": "MessyJo", "screen_name": "jr_harris96", "lang": "en", "location": "Ontario, CA", "create_at": date("2012-05-08"), "description": "18⚫ JFK ✈ORD", "followers_count": 456, "friends_count": 443, "statues_count": 16123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Niagara Falls, NY", "id": "eec8e86da985fc44", "name": "Niagara Falls", "place_type": "city", "bounding_box": rectangle("-79.075086,43.07079 -78.942516,43.132798") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36063, "countyName": "Niagara", "cityID": 3651055, "cityName": "Niagara Falls" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137291358209, "text": "Same https://t.co/uFQ6fRlyc1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4806653598, "name": "Macker", "screen_name": "aye_kram", "lang": "en", "location": "Being Petty", "create_at": date("2016-01-24"), "description": "Wassup", "followers_count": 146, "friends_count": 226, "statues_count": 1833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137526235137, "text": "Goodnight.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318131780, "name": "Mauricio • 7 days", "screen_name": "HerArianatorBoy", "lang": "es", "location": "Traveling: Corpus Christi, TX", "create_at": date("2011-06-15"), "description": "Only 1 week until Dangerous Woman!", "followers_count": 12681, "friends_count": 8872, "statues_count": 22670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137618522112, "text": "Should probably go to sleep since I have to wake up early. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1375442588, "name": "Arizzle.✨", "screen_name": "Aarinn_Valdez", "lang": "en", "location": "null", "create_at": date("2013-04-23"), "description": "Sc:Aar.inn", "followers_count": 694, "friends_count": 542, "statues_count": 18880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver City, NM", "id": "91717be9c3e8c07d", "name": "Silver City", "place_type": "city", "bounding_box": rectangle("-108.31294,32.726521 -108.221354,32.82689") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35017, "countyName": "Grant", "cityID": 3573260, "cityName": "Silver City" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137660567552, "text": "@HeartNorthWest @NathanSykes ������", "in_reply_to_status": 731013520947433473, "in_reply_to_user": 24604190, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 24604190, 90896009 }}, "user": { "id": 48610056, "name": "Alberto Candelaria", "screen_name": "alcan7", "lang": "en", "location": "Bronx, NY", "create_at": date("2009-06-18"), "description": "Work for NYC Transit over 28 yrs Big Michael Jackson & Taylor Swift fan. Swiftie Wife Michelle & son AFC riding bicycles Its Friday Friday got to get down on...", "followers_count": 1642, "friends_count": 3488, "statues_count": 130037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016137891323908, "text": "I know it's over but there is truly only one person I would've truly loved to spend this day with ♓️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 115580685, "name": "Jess Henigman", "screen_name": "Jess_Henigman", "lang": "en", "location": "null", "create_at": date("2010-02-18"), "description": "null", "followers_count": 173, "friends_count": 159, "statues_count": 4835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, NY", "id": "39411d80c106aa80", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-73.6611,40.612083 -73.616045,40.657308") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3654441, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016138478391298, "text": "@krayolabrown I need to fix up my car before I drive the 30 minutes it takes to come visit.", "in_reply_to_status": 731015924917002241, "in_reply_to_user": 3699123433, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3699123433 }}, "user": { "id": 1610065856, "name": "Alex Brin", "screen_name": "TheAlexBrin", "lang": "en", "location": "Mount Vernon, WA", "create_at": date("2013-07-21"), "description": "Gamer, Nerd Baller, E-sports Lover. LoL IGN: Eskamo", "followers_count": 114, "friends_count": 558, "statues_count": 7957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, WA", "id": "c70b623cdcd61952", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-122.373199,48.382535 -122.250443,48.453705") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit", "cityID": 5347560, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016138537111552, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":59.7°F Wind:0.0mph Pressure: 30.03hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 17, "statues_count": 317700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016138537222145, "text": "This is Obama's plan and I think that Mr Trump knows it https://t.co/XxGSMNNjV9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304455363, "name": "#LauraJeanne4Trump", "screen_name": "GLaurajeanne64", "lang": "en", "location": "USA...Florida and Connecticut", "create_at": date("2015-05-31"), "description": "Rebel with a cause... to thine own self be true....free thinker....risk taker...friend to all animals...#TrumpGirl #Trumptrain #Trump2016", "followers_count": 4559, "friends_count": 5002, "statues_count": 13924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearwater, FL", "id": "700eeb799fa55a4b", "name": "Clearwater", "place_type": "city", "bounding_box": rectangle("-82.831674,27.935178 -82.679007,28.050243") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1212875, "cityName": "Clearwater" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016139015213056, "text": "Bored at the moment", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2152062428, "name": ".", "screen_name": "Antonio10_C", "lang": "en", "location": "null", "create_at": date("2013-10-23"), "description": "Loading...", "followers_count": 427, "friends_count": 269, "statues_count": 4025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-13T00:00:03.000Z"), "id": 731016139136868353, "text": "@TheeNamesRagas it's a 2!", "in_reply_to_status": 731012517976121345, "in_reply_to_user": 87006008, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 87006008 }}, "user": { "id": 48608804, "name": "M A R I S A", "screen_name": "RisaRod_", "lang": "en", "location": "EP|LC", "create_at": date("2009-06-18"), "description": "probably with @bisssm tbh", "followers_count": 893, "friends_count": 530, "statues_count": 21150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Park, NM", "id": "76a51908fb719a47", "name": "University Park", "place_type": "city", "bounding_box": rectangle("-106.767744,32.262483 -106.731575,32.28619") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3581030, "cityName": "University Park" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016139891843073, "text": "just wanted to be the first to say HAPPY BIRTHDAY to @BrittannAguilar!!!!!!!!!!! u r my chola 4 evr <3 love u https://t.co/bdZjpisjIQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2610486044 }}, "user": { "id": 3293283203, "name": "bella c", "screen_name": "bellaclanton", "lang": "en", "location": "California, USA", "create_at": date("2015-05-21"), "description": "stoked on life☻☼", "followers_count": 188, "friends_count": 184, "statues_count": 923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016139908620288, "text": "Meu deus eu me lembro de chegar na casa da mari e fra la junto e eu com A RESSACA awn que saudade dos meus babys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 3011496106, "name": "julha mil grau", "screen_name": "arrombajulia", "lang": "pt", "location": "San Clemente, CA", "create_at": date("2015-02-01"), "description": "started from the bottom now I'm here", "followers_count": 227, "friends_count": 177, "statues_count": 10929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Clemente, CA", "id": "97467526c304c5db", "name": "San Clemente", "place_type": "city", "bounding_box": rectangle("-117.666227,33.386645 -117.571015,33.490825") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 665084, "cityName": "San Clemente" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016140323835909, "text": "Disneyland was so fucking fun", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 474594583, "name": "nick", "screen_name": "NickoIasThomas", "lang": "en", "location": "null", "create_at": date("2012-01-25"), "description": "I enjoy a good quesadilla every now and then", "followers_count": 515, "friends_count": 405, "statues_count": 82104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016140550492160, "text": "DC lol…\nrich kids of DC in that crowd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.03363632,38.9023139"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1090793250, "name": "level'd out", "screen_name": "IeveIs", "lang": "en", "location": "null", "create_at": date("2013-01-14"), "description": "null", "followers_count": 2294, "friends_count": 787, "statues_count": 48034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Washington, DC", "id": "01fbe706f872cb32", "name": "Washington", "place_type": "city", "bounding_box": rectangle("-77.119401,38.801826 -76.909396,38.99538") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51013, "countyName": "Arlington", "cityID": 5103000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016140697165825, "text": "Sometimes you just have to realize what's real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325312113, "name": "Ryan Schelin", "screen_name": "rschelin3", "lang": "en", "location": "null", "create_at": date("2011-06-27"), "description": "Life's a garden, dig it. University of Iowa Engineering. #GoHawks", "followers_count": 435, "friends_count": 433, "statues_count": 6651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iowa City, IA", "id": "01e0b1c656c5070f", "name": "Iowa City", "place_type": "city", "bounding_box": rectangle("-91.611057,41.599181 -91.463067,41.695526") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19103, "countyName": "Johnson", "cityID": 1938595, "cityName": "Iowa City" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016141389238273, "text": "I just want to drive off to the middle of nowhere and get away for a while", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323533213, "name": "marisela", "screen_name": "Cutie_Love28", "lang": "en", "location": "trapsoul", "create_at": date("2011-06-24"), "description": "I'd love to wake up next to you ❤️", "followers_count": 1147, "friends_count": 711, "statues_count": 74578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016141842186240, "text": "This concerns me https://t.co/MVd1596cYI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116337318, "name": "Zhanelle", "screen_name": "ZHAMARIANO", "lang": "en", "location": "Carson❤️California", "create_at": date("2010-02-21"), "description": "#tysm", "followers_count": 884, "friends_count": 675, "statues_count": 32115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016141917720576, "text": "@popbanao 헉 ㅠㅠ 봄녀님 중말중말 최고로 부러워요 ���� 누가 보내주셨나요? 좋으신 분 ��", "in_reply_to_status": 730584975234322433, "in_reply_to_user": 255534289, "favorite_count": 0, "retweet_count": 0, "lang": "ko", "is_retweet": false, "user_mentions": {{ 255534289 }}, "user": { "id": 431162309, "name": "SSM-Elina", "screen_name": "SUEUSU68", "lang": "ko", "location": "null", "create_at": date("2011-12-07"), "description": "dream a cartoonist and Artist :) I love Marvel and disney. tumblr: http://sujinsm96.tumblr.com 그림은 린베님 커미션입니당 ㅋ", "followers_count": 42, "friends_count": 101, "statues_count": 9403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016142056099840, "text": "@mr_bowen8 let's play cod", "in_reply_to_status": 731016031167074304, "in_reply_to_user": 293277829, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 293277829 }}, "user": { "id": 3244809852, "name": "Neek", "screen_name": "okkinsama", "lang": "en", "location": "Konoha", "create_at": date("2015-06-13"), "description": "No home like Valhalla//HDGF//Goose", "followers_count": 273, "friends_count": 473, "statues_count": 6485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016142324539392, "text": "Do you ever just hate someone for being good to you?? Lol\nLike, WTF took you soooo long?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 928764026, "name": "Mildreed", "screen_name": "angiee13013", "lang": "en", "location": "null", "create_at": date("2012-11-05"), "description": "IG: angiee0130", "followers_count": 158, "friends_count": 157, "statues_count": 11166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016142433619968, "text": "Series ❤️�� https://t.co/pnLegMTuEt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 56444630, "name": "Lonzyyyyy", "screen_name": "AlonzoDBrooks", "lang": "en", "location": "Madera, CA", "create_at": date("2009-07-13"), "description": "Alonzo Dash Brooks | 19 | 67 days left.", "followers_count": 3406, "friends_count": 1969, "statues_count": 48941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016142559436801, "text": "@kathleenakat I was a lifeguard two summers ago, so I wouldn't mind it!", "in_reply_to_status": 731002842937352192, "in_reply_to_user": 250141275, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 250141275 }}, "user": { "id": 1591911367, "name": "Cynderella", "screen_name": "Cynthia_Cogan", "lang": "en", "location": "nac//w-ford", "create_at": date("2013-07-13"), "description": "My ultimate goal in life is to be as funny as Amy Schumer, Melissa McCarthy, and Betty White combined. •Gemini•18•sfasu•freshman•zeta•", "followers_count": 414, "friends_count": 397, "statues_count": 8786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nacogdoches, TX", "id": "ebf78e870cecf27c", "name": "Nacogdoches", "place_type": "city", "bounding_box": rectangle("-94.704218,31.55485 -94.594405,31.673033") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48347, "countyName": "Nacogdoches", "cityID": 4850256, "cityName": "Nacogdoches" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016142576320512, "text": "One of my biggest fans, jennyt_1013, even when I look like a chola. �� I wouldn't be who I am… https://t.co/FgLSHzs0ZW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.99333333,40.7625"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2980640410, "name": "Isaiah Negron-Cruz", "screen_name": "IsaiahCruzNYC", "lang": "en", "location": "New York, NY", "create_at": date("2015-01-15"), "description": "A native New Yorker, a writer, a doer, a reader, a dreamer, a Disney junkie, a literature buff, an avid gamer and all kinds of stuff...then you run out of space", "followers_count": 452, "friends_count": 919, "statues_count": 2106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016143134019585, "text": "@muffinlord666 did you see the video of them running through the streets celebrating", "in_reply_to_status": 731016060162314240, "in_reply_to_user": 1398753212, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1398753212 }}, "user": { "id": 721008918, "name": "Jake", "screen_name": "JacobKopitar", "lang": "en", "location": "null", "create_at": date("2012-07-27"), "description": "What's money without happiness, or hard times without the people you love.", "followers_count": 2540, "friends_count": 1534, "statues_count": 109981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016143293558785, "text": "I want to cry I just wish I was in my bed rn ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 998659400, "name": "Colleen", "screen_name": "ceceliamelia", "lang": "en", "location": "null", "create_at": date("2012-12-08"), "description": "Sometimes the truth isnt easy but neither is life", "followers_count": 79, "friends_count": 106, "statues_count": 4137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lowell, MA", "id": "d6539f049c4d05e8", "name": "Lowell", "place_type": "city", "bounding_box": rectangle("-71.382444,42.605989 -71.271272,42.666507") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2537000, "cityName": "Lowell" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016143507329024, "text": "It takes a lot for me to trust someone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448702128, "name": "liana kankababian", "screen_name": "LKankababian", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "snapchat: lianalovespink", "followers_count": 710, "friends_count": 554, "statues_count": 12787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:04.000Z"), "id": 731016143587049473, "text": "No Sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872891238, "name": "KB", "screen_name": "GeauxKB15", "lang": "en", "location": "#LSUE20", "create_at": date("2012-10-10"), "description": "Abbeville 2nd Team All-District SS/WR Class of 16' R.i.p. Sue", "followers_count": 1165, "friends_count": 743, "statues_count": 41495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abbeville, LA", "id": "89b71a24d57765bd", "name": "Abbeville", "place_type": "city", "bounding_box": rectangle("-92.165015,29.949795 -92.07413,30.006348") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion", "cityID": 2200100, "cityName": "Abbeville" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016144035950592, "text": "Today was a long day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1232900930, "name": "Lynette", "screen_name": "OhDoraLynette", "lang": "en", "location": "Texas", "create_at": date("2013-03-01"), "description": "♉️", "followers_count": 1394, "friends_count": 1092, "statues_count": 25060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elsa, TX", "id": "8ca173bdb98b95e9", "name": "Elsa", "place_type": "city", "bounding_box": rectangle("-98.023228,26.275088 -97.977072,26.324576") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4824036, "cityName": "Elsa" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016145117937664, "text": "I keep quiet cause I like to study people and see what they're about", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1393527050, "name": "Gracie", "screen_name": "grass_staain", "lang": "en", "location": "null", "create_at": date("2013-04-30"), "description": "It is what it is", "followers_count": 307, "friends_count": 222, "statues_count": 2424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016145558376448, "text": "@Drainpuppet but tlop remaster is great (everything except that loud snare in father stretch beat p1)", "in_reply_to_status": 730998486947254272, "in_reply_to_user": 75112996, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75112996 }}, "user": { "id": 238625955, "name": "YUNA YUNA", "screen_name": "YUNA_YUN4", "lang": "en", "location": "San Francisco, CA", "create_at": date("2011-01-15"), "description": "jeremy ~ music", "followers_count": 456, "friends_count": 291, "statues_count": 15935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016145600425984, "text": "Southwest Chiropractic Clinic - https://t.co/taw1LPsNew\n\n#southwestchiropracticclinic #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.61239,43.630142"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "southwestchiropracticclinic", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 533, "friends_count": 539, "statues_count": 709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Worthington, MN", "id": "4cc9a5aa2a29c05c", "name": "Worthington", "place_type": "city", "bounding_box": rectangle("-95.644347,43.601854 -95.56187,43.647182") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27105, "countyName": "Nobles", "cityID": 2771734, "cityName": "Worthington" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016145621426176, "text": "tonight was definitely one for the books", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71398588, "name": "Georgia, The Mogul", "screen_name": "georgia_doll", "lang": "en", "location": "The Pote ✈ The Bluff", "create_at": date("2009-09-03"), "description": "#CAU17 • Media Personality • Founder of Georgia Media Agency • President of WSTU Radio • Host on WSTU Radio • Email: [ayanagunn@georgiadoll.com] • IG: ga_doll", "followers_count": 2649, "friends_count": 967, "statues_count": 129659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016146535731200, "text": "Never forget this gem https://t.co/Khssu85edn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2747378044, "name": "Anthony Franco Assss", "screen_name": "p00pface15", "lang": "en", "location": "Thomasville, NC", "create_at": date("2014-08-19"), "description": "darling, this is chamomile tea.", "followers_count": 98, "friends_count": 328, "statues_count": 7359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thomasville, NC", "id": "adef3f56e6a06e9c", "name": "Thomasville", "place_type": "city", "bounding_box": rectangle("-80.143944,35.802886 -80.00527,35.931891") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37057, "countyName": "Davidson", "cityID": 3767420, "cityName": "Thomasville" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016146732777473, "text": "Happy Birthday ���������� @_angel98perez_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2326539403 }}, "user": { "id": 911349439, "name": "Bri Bri ❂", "screen_name": "briannatrue_", "lang": "en", "location": "null", "create_at": date("2012-10-28"), "description": "sc - briannatsmith ✝☮☯♎️ #blacklivesmatter", "followers_count": 1615, "friends_count": 904, "statues_count": 20088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016146808274944, "text": "@phil_goldsberry like come on https://t.co/HNvSqG3iaR", "in_reply_to_status": 731015375341518852, "in_reply_to_user": 3938498960, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3938498960 }}, "user": { "id": 3938498960, "name": "Phillip Goldsberry", "screen_name": "phil_goldsberry", "lang": "en", "location": "Chandler, AZ", "create_at": date("2015-10-18"), "description": "Huge fan of Jesus. Just looking to change lives. Subaru technician.", "followers_count": 270, "friends_count": 229, "statues_count": 1122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016147080863744, "text": "I fucken hate Mario", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1352841714, "name": "Lupito The Preacher", "screen_name": "lupito_guzman", "lang": "en", "location": "Hogwarts School of Witchcraft ", "create_at": date("2013-04-14"), "description": "Gryffindor", "followers_count": 378, "friends_count": 230, "statues_count": 6437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016147512885249, "text": "Almond milk >>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3233261473, "name": "the boy", "screen_name": "Sanchez699Elmer", "lang": "en", "location": "null", "create_at": date("2015-06-01"), "description": "tha boy", "followers_count": 325, "friends_count": 321, "statues_count": 9573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parlier, CA", "id": "b3c579dbd6e6f98b", "name": "Parlier", "place_type": "city", "bounding_box": rectangle("-119.562738,36.603513 -119.51629,36.621062") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 655856, "cityName": "Parlier" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016147852660737, "text": "@_kimmylovee �� since you begged I won't", "in_reply_to_status": 731015957171183617, "in_reply_to_user": 4666589048, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4666589048 }}, "user": { "id": 2944611242, "name": "Tamar JR Johnson", "screen_name": "Tjrj3_", "lang": "en", "location": "null", "create_at": date("2014-12-26"), "description": "Living the life of the boy who cried wolf.", "followers_count": 551, "friends_count": 513, "statues_count": 2248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-13T00:00:05.000Z"), "id": 731016148049780738, "text": "Uhhh 2 https://t.co/sCiMMm7My7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 942515845, "name": "️☪indirela♡", "screen_name": "LaaCindyyy", "lang": "en", "location": "null", "create_at": date("2012-11-11"), "description": "♡ Ion wanna hype ya but you a lucky ass nigga if my mean ass like ya✨ひ", "followers_count": 1037, "friends_count": 381, "statues_count": 47766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148364492801, "text": "The #EVIDENCE is in. And I'm #READY || #HEARTofGXLD #fridaythe13th @… https://t.co/uJQw2t9YNr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.46283056,34.23848611"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EVIDENCE", "READY", "HEARTofGXLD", "fridaythe13th" }}, "user": { "id": 87753347, "name": "Ami.... the TRUTH", "screen_name": "AmidaTRUTH", "lang": "en", "location": "Eastside,GA ", "create_at": date("2009-11-05"), "description": "Aspiring musician. Loyal Friend. Caring father. REAL NIGGA!!!! #FCKdat follow and i will lead u to greatness. #thaUNION #HUGLIFE #carterhouse #teamUSA", "followers_count": 610, "friends_count": 1924, "statues_count": 4305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148590940160, "text": "Wind 0.0 mph ---. Barometer 29.927 in, Falling. Temperature 51.7 °F. Rain today 0.00 in. Humidity 54%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-70.24083333,44.41166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880787378, "name": "Jody", "screen_name": "JodysWeather", "lang": "en", "location": "Livermore Maine", "create_at": date("2014-11-17"), "description": "Northern Androscoggin County Weather", "followers_count": 1, "friends_count": 0, "statues_count": 6438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23001, "countyName": "Androscoggin" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148632752129, "text": "It's amazing how an $80 @colehaan belt can split after 15-20 wears. Might as well stick with something from Ross. https://t.co/N7c1WPvWS5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 36451558 }}, "user": { "id": 15392048, "name": "Dustin Fanzo", "screen_name": "phanzooo", "lang": "en", "location": "Dallas, TX", "create_at": date("2008-07-11"), "description": "#craftbeer | #phish | #slurpees | #pens | #steelers | #buccos | mobile tech | #homebrew attempter | Untappd phanzo", "followers_count": 1016, "friends_count": 2053, "statues_count": 14919 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148720836609, "text": "... https://t.co/AQDW76bLoY", "in_reply_to_status": 731016002452918272, "in_reply_to_user": 298319930, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 298319930, "name": "Thomas John Trono", "screen_name": "I_Am_This_Tom", "lang": "en", "location": "http://tomnipotence.com", "create_at": date("2011-05-13"), "description": "let's play ball", "followers_count": 110, "friends_count": 498, "statues_count": 50292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148771319808, "text": "@JoeLoves69 ugh I hate you.", "in_reply_to_status": 731016016759820288, "in_reply_to_user": 488074428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 488074428 }}, "user": { "id": 251351520, "name": "Isabella", "screen_name": "bellacrgz", "lang": "en", "location": "FL", "create_at": date("2011-02-12"), "description": "Real down to Mars girl • MIA", "followers_count": 360, "friends_count": 584, "statues_count": 32330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall, FL", "id": "9b46dccb3cfb880c", "name": "Kendall", "place_type": "city", "bounding_box": rectangle("-80.389344,25.628844 -80.304896,25.715128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236100, "cityName": "Kendall" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148959924224, "text": "6th* https://t.co/S5uSAlmmxm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 585398866, "name": "HOAX ⁶", "screen_name": "TobiKash_", "lang": "en", "location": "null", "create_at": date("2012-05-19"), "description": "Time is your greatest commodity ⁶", "followers_count": 847, "friends_count": 632, "statues_count": 18038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016148997668864, "text": "@_kathryng_ And if not on a test, everyone has definitely done it in real life, and that's mostly the point I'm making.", "in_reply_to_status": 731015970974617600, "in_reply_to_user": 614003992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3294332744 }}, "user": { "id": 614003992, "name": "Big Ben", "screen_name": "BmeBenji", "lang": "en", "location": "Redmond, WA", "create_at": date("2012-06-20"), "description": "I have a strange obsession with drawing pigs. \n\n1 Corinthians 13:4-7 \n John 16:33", "followers_count": 120, "friends_count": 221, "statues_count": 2779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016149144477696, "text": "@mitri_george God bless your beautiful soul��", "in_reply_to_status": 731015640857763842, "in_reply_to_user": 3112517719, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3112517719 }}, "user": { "id": 1023573560, "name": "Sabrina", "screen_name": "SabsG35", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-12-19"), "description": "Vice President of the official Shaun Livingston fan club. #GSW #49ers #SFG", "followers_count": 690, "friends_count": 387, "statues_count": 36709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016149866020864, "text": "@daydreameriam1 @LouDPhillips - ☆☆ In June on the LMN Network & I really can't wait! :D Lou's a Great Actor! :D ☆☆ https://t.co/ItDdAETsMa", "in_reply_to_status": 731000691242733568, "in_reply_to_user": 60641887, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60641887, 39378322 }}, "user": { "id": 24686590, "name": "~ ♡ Sue G. ♡ ~", "screen_name": "SuzanneG1970", "lang": "en", "location": "Michigan, USA", "create_at": date("2009-03-16"), "description": "☆ I Luv having fun, the Sunshine, my Fiance', my Dog, Horror Films, my family, the Paranormal, & Computers! :D ☆", "followers_count": 734, "friends_count": 2673, "statues_count": 5301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rochester Hills, MI", "id": "2409d5aabed47f79", "name": "Rochester Hills", "place_type": "city", "bounding_box": rectangle("-83.214001,42.620953 -83.091535,42.712333") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2669035, "cityName": "Rochester Hills" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016150243385344, "text": "And nothing helps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1671694446, "name": "lil crack", "screen_name": "Saracha__Sauce", "lang": "en", "location": "Definitely Crying", "create_at": date("2013-08-14"), "description": "You're not half bad for a hominid ♐️ENFP", "followers_count": 347, "friends_count": 635, "statues_count": 10051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016150335823872, "text": "Special Olympics Winter Caldron #boise #publicart @ Boise Airport https://t.co/nGwEo0Z7Uf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.22092829,43.56800102"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "boise", "publicart" }}, "user": { "id": 2189575110, "name": "Leon Fairley", "screen_name": "LeonF63", "lang": "en", "location": "Sacramento, CA USA", "create_at": date("2013-11-11"), "description": "Follows & RTs ≠ endorsement. Views expressed R personal, not my employer's, Accenture.", "followers_count": 430, "friends_count": 1451, "statues_count": 3734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boise, ID", "id": "737fcb9daae404ac", "name": "Boise", "place_type": "city", "bounding_box": rectangle("-116.374494,43.510172 -116.101508,43.685894") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1608830, "cityName": "Boise City" } }
+{ "create_at": datetime("2016-05-13T00:00:06.000Z"), "id": 731016150381789186, "text": "������ https://t.co/MUGjgHEOFs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 745741638, "name": "call me cooolass", "screen_name": "YoGrannyKno", "lang": "en", "location": "Louisiana, USA", "create_at": date("2012-08-08"), "description": "CEO/Artist of 9ine5iveRecordz 95Recordz@gmail.com | #SU19 #9ine5ive", "followers_count": 4338, "friends_count": 3740, "statues_count": 125177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378511475281920, "text": "Everybody faking drunk again lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 172023931, "name": "Terrell Quantez", "screen_name": "Im_bates", "lang": "en", "location": "cleveland", "create_at": date("2010-07-28"), "description": "T.Q.B. been counting my blessings since 9️⃣1️⃣", "followers_count": 642, "friends_count": 826, "statues_count": 12755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378511903109120, "text": "Wind 0.0 mph ---. Barometer 30.011 in, Steady. Temperature 53.3 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378512217563136, "text": "Wind 5.0 mph NNE. Barometer 1023.40 mb, Rising Rapidly. Temperature 55.3 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378512842477568, "text": "@__d3stiny doing what Des ?", "in_reply_to_status": 731378442160250880, "in_reply_to_user": 330658643, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 330658643 }}, "user": { "id": 278823801, "name": "Rambo", "screen_name": "Traeskii__", "lang": "en", "location": "round the way 4️⃣0️⃣0️⃣", "create_at": date("2011-04-07"), "description": "get high to balance my lows ... sc: traepound19", "followers_count": 1390, "friends_count": 1321, "statues_count": 94530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513178165248, "text": "I luv chicken tho https://t.co/8MYiNCUP5z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2479192794, "name": "عیسی", "screen_name": "___jesus_", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2014-05-05"), "description": "Life's a Bitch. Life is Good. #ChasingSix. If you don't follow back, imma shoot you.", "followers_count": 674, "friends_count": 432, "statues_count": 12127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513257730049, "text": "@MaraeNarvaez hey. I like your pic. We cute.", "in_reply_to_status": -1, "in_reply_to_user": 928503438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 928503438 }}, "user": { "id": 2327540822, "name": "yes,", "screen_name": "tommyolson114", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "rude with aloha. @tolson2k6 add me on SC. It's Lit", "followers_count": 240, "friends_count": 478, "statues_count": 6190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Makakilo, HI", "id": "3aff180d3a9e53d5", "name": "Makakilo", "place_type": "city", "bounding_box": rectangle("-158.115354,21.332074 -158.057413,21.396068") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1547600, "cityName": "Makakilo" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513278701568, "text": "@kaylee_paskorz so good to see you!!! ��❤️��", "in_reply_to_status": 731348429993959425, "in_reply_to_user": 373731735, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 373731735 }}, "user": { "id": 368688305, "name": "Morgan Alyse", "screen_name": "MorganStruebing", "lang": "en", "location": "null", "create_at": date("2011-09-05"), "description": "Senior at Simpson College! #GoStorm", "followers_count": 700, "friends_count": 576, "statues_count": 11530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indianola, IA", "id": "8872a0d0a19972bf", "name": "Indianola", "place_type": "city", "bounding_box": rectangle("-93.596885,41.332761 -93.533766,41.384022") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19181, "countyName": "Warren", "cityID": 1938280, "cityName": "Indianola" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513341636608, "text": "@ebbtideapp Tide in Kiptopeke Beach, Virginia 05/14/2016\nHigh 3:46am 2.5\n Low 10:16am 0.3\nHigh 4:24pm 2.5\n Low 10:46pm 0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.9883,37.1667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 32620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51131, "countyName": "Northampton" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513442439168, "text": "Sleep is for weak", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 371497230, "name": "ty", "screen_name": "CrownTeeko", "lang": "en", "location": "Accokeek, MD", "create_at": date("2011-09-10"), "description": "HDMH Carthage 19' Kenosha", "followers_count": 451, "friends_count": 151, "statues_count": 32925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenosha, WI", "id": "6359ef285f710052", "name": "Kenosha", "place_type": "city", "bounding_box": rectangle("-87.959452,42.538811 -87.807358,42.63972") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55059, "countyName": "Kenosha", "cityID": 5539225, "cityName": "Kenosha" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513622622208, "text": "@lizabeerryy skrrt skrrrrt https://t.co/x4L5guEJGY", "in_reply_to_status": 731374774719385600, "in_reply_to_user": 3164541122, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 3164541122 }}, "user": { "id": 2349805225, "name": "Space Cowboy", "screen_name": "Thevman98", "lang": "en", "location": "Lamecaster ", "create_at": date("2014-02-18"), "description": "All life is precious", "followers_count": 698, "friends_count": 487, "statues_count": 41147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378513677193220, "text": "just stay to myself and my fam cause everyone out there is the same", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506902538, "name": "dy$", "screen_name": "dysndysn", "lang": "en", "location": "Texas", "create_at": date("2012-02-27"), "description": "http://tinyurl.com/jr46ys8", "followers_count": 6705, "friends_count": 121, "statues_count": 90 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378514004459520, "text": "I'm not friendly. Never have been. Never will be ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30944433, "name": "IG: @She.MadPetty", "screen_name": "no_5princess2_", "lang": "en", "location": "Cleveland, OH", "create_at": date("2009-04-13"), "description": "Leo ♌️| [L]GBT ❤️| 20 ☯| KIK: ThatGirl.Kei | SC: ThatGirlKei", "followers_count": 1210, "friends_count": 1081, "statues_count": 29565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378514033696768, "text": "Honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 881591408, "name": "Lils", "screen_name": "ayomisslilly", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-10-14"), "description": "LitLitLit", "followers_count": 595, "friends_count": 390, "statues_count": 15535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378514180476929, "text": "@amrightnow Complete disrespect to our military and their families ... while they are out there fighting their loved ones here", "in_reply_to_status": 731377321756794880, "in_reply_to_user": 449110269, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 449110269 }}, "user": { "id": 317223956, "name": "cjay", "screen_name": "sweetsienna23", "lang": "en", "location": "California, USA", "create_at": date("2011-06-14"), "description": "American Patriot Striving to Preserve our Constitution of the United States. All civilized western nations should have a right to their culture & sovereignty.", "followers_count": 677, "friends_count": 212, "statues_count": 6491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378514344058880, "text": "Econ has me stressed af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3311010846, "name": "Tina", "screen_name": "soy_latinaa", "lang": "en", "location": "Hayward, CA", "create_at": date("2015-08-09"), "description": "yay area", "followers_count": 229, "friends_count": 255, "statues_count": 4410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-14T00:00:00.000Z"), "id": 731378514734112768, "text": "@hiimmrmeseeks watch me unravel, I'll soon be NAKEDDD", "in_reply_to_status": 731378398304428032, "in_reply_to_user": 1967033762, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1967033762 }}, "user": { "id": 3037642939, "name": "Prince Vegeta", "screen_name": "40onza", "lang": "en", "location": "null", "create_at": date("2015-02-23"), "description": "null", "followers_count": 208, "friends_count": 131, "statues_count": 5348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378515174547456, "text": "@NicoleeDanielee was", "in_reply_to_status": 731377999979765760, "in_reply_to_user": 1597410834, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1597410834 }}, "user": { "id": 2163944468, "name": "Jjames", "screen_name": "vsvpXarevalo", "lang": "en", "location": "Houston, TX ", "create_at": date("2013-10-29"), "description": "Txst20", "followers_count": 670, "friends_count": 775, "statues_count": 14286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378515606536193, "text": "@__liveurlife @The_Ambeezy she on drugs bro", "in_reply_to_status": 731378230561644545, "in_reply_to_user": 28020073, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28020073, 449324847 }}, "user": { "id": 251870722, "name": "Justin", "screen_name": "okayjayyjones", "lang": "en", "location": "HTX", "create_at": date("2011-02-13"), "description": "University of Houston. If you want it, go get it.", "followers_count": 2746, "friends_count": 1478, "statues_count": 75749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378515652673537, "text": "I used to feel bad when people remembered my name and I didn't remember their names but I'm good with faces so that makes up for it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293214644, "name": "Z", "screen_name": "thatmuslim", "lang": "en", "location": "Algeria", "create_at": date("2011-05-04"), "description": "Where I cannot satisfy my reason, I love to humor my fancy. - Sir Thomas Browne", "followers_count": 801, "friends_count": 297, "statues_count": 37241 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378516596428800, "text": "Once in a lifetime opportunity, I would've took it also ���� https://t.co/87IecSOk5D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52856536, "name": "laura.", "screen_name": "Laureezy_", "lang": "en", "location": "Houston, TX", "create_at": date("2009-07-01"), "description": "Jealousy and hate is a serious illness, I hope some of y'all get well soon! ;*", "followers_count": 326, "friends_count": 200, "statues_count": 16887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378517099732993, "text": "I think too much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297705052, "name": "Miguel", "screen_name": "AfloatHickory", "lang": "en", "location": "AZ ✈️ Everywhere", "create_at": date("2011-05-12"), "description": "Wanheda. @DebnamCarey is always my Heda", "followers_count": 505, "friends_count": 564, "statues_count": 101153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378517267611648, "text": "I should be asleep. Bc that 7 hour drive home ain't gun' be no joke.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413708381, "name": "chey", "screen_name": "CheyenneMaple", "lang": "en", "location": "Idalou • Tyler ", "create_at": date("2011-11-15"), "description": "TJC. | 19. | snocone lady. ♥️", "followers_count": 1113, "friends_count": 1930, "statues_count": 11038 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tyler, TX", "id": "9d4c71b7937a3a90", "name": "Tyler", "place_type": "city", "bounding_box": rectangle("-95.414773,32.221406 -95.219307,32.409917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4874144, "cityName": "Tyler" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378517322014720, "text": "Cook - SONIC Drive-In: (#Elizabeth, CO) https://t.co/E8dGTF11P8 #Hospitality #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.6083762,39.3602344"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Elizabeth", "Hospitality", "Job", "Jobs", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 328, "friends_count": 39, "statues_count": 19698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado, USA", "id": "e21c8e4914eef2b3", "name": "Colorado", "place_type": "admin", "bounding_box": rectangle("-109.060257,36.992427 -102.041524,41.003445") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8039, "countyName": "Elbert", "cityID": 823740, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378517670166529, "text": "Am drunk. Am cold. Pls help.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15869280, "name": "grog", "screen_name": "graugustine", "lang": "en", "location": "fargo", "create_at": date("2008-08-15"), "description": "apathetic & lazy lol", "followers_count": 581, "friends_count": 404, "statues_count": 7708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fargo, ND", "id": "5c8fea63bc3faa5b", "name": "Fargo", "place_type": "city", "bounding_box": rectangle("-96.903853,46.787881 -96.756332,46.941884") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38017, "countyName": "Cass", "cityID": 3825700, "cityName": "Fargo" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378517925990400, "text": "@davidcoverdale Goodnight, DC! I hope you have a peaceful night!⭐️��⭐️�� https://t.co/gxAOVQcpS6", "in_reply_to_status": 731368285044801540, "in_reply_to_user": 166665318, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 166665318 }}, "user": { "id": 566442684, "name": "Rose Fernandez", "screen_name": "roseholm", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-04-29"), "description": "Loves Football,Cooking, Gardening Travel,Fashion,from Philippines,studied at Wesleyan University--", "followers_count": 234, "friends_count": 265, "statues_count": 7358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378518215495680, "text": "@501s_Retros_Ink red you don't know what I'm bout ��", "in_reply_to_status": 731378456710291456, "in_reply_to_user": 320454245, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 320454245 }}, "user": { "id": 50827162, "name": "beeYondSLAY", "screen_name": "beeBeKING", "lang": "en", "location": "null", "create_at": date("2009-06-25"), "description": "gemini♊️. #RestOnRaylon , #RestEasyKeith .. Wake up , beat face , get this $$$", "followers_count": 1682, "friends_count": 1096, "statues_count": 74771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378518450425856, "text": "@feelMyAmbition_ yea..OCS option", "in_reply_to_status": 731377322608238592, "in_reply_to_user": 156499634, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156499634 }}, "user": { "id": 1937709493, "name": "κingτee⁷ ⚖", "screen_name": "__TLamar", "lang": "en", "location": "StraightOuttaDega ", "create_at": date("2013-10-05"), "description": "Army | Political Scientist | #BlackExcellence RTs ≠ endorsement ❤️KCA", "followers_count": 1603, "friends_count": 1192, "statues_count": 45737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery, AL", "id": "7f061ded71fdc974", "name": "Montgomery", "place_type": "city", "bounding_box": rectangle("-86.418059,32.284593 -86.071398,32.443697") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1101, "countyName": "Montgomery", "cityID": 151000, "cityName": "Montgomery" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378518467182592, "text": "03:00:02 |Temp: 53.7ºF |Dew Point 49.9ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the S, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 94, "friends_count": 21, "statues_count": 94117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-14T00:00:01.000Z"), "id": 731378519213785089, "text": "Three3 more days", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3529824316, "name": "Stain. (May 17th)", "screen_name": "brendan_chidera", "lang": "en", "location": "Manila City", "create_at": date("2015-09-02"), "description": "The fact that am here doesn't mean i won't get there, the only difference between here and there is (T) which stand for TIME. Man Utd is bae", "followers_count": 246, "friends_count": 887, "statues_count": 970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manila, CA", "id": "009676e57c2f329c", "name": "Manila", "place_type": "city", "bounding_box": rectangle("-124.173823,40.836929 -124.147391,40.864956") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 645414, "cityName": "Manila" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378519272357889, "text": "my dad said 4 hours ago that he's gunna go get his check and he still aint here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-116.19742912,33.69023229"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2341813164, "name": "lil teaser", "screen_name": "_jimenez130__", "lang": "en", "location": "null", "create_at": date("2014-02-13"), "description": "oh hey love @fakierecon", "followers_count": 464, "friends_count": 298, "statues_count": 19160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378519666614274, "text": "Being in a relationship with someone who was once your BFF is weird it's like \"ahaha you suck, you're dumb ... Jk you're actually amazing.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 178984239, "name": "Aly", "screen_name": "aly_bun", "lang": "en", "location": "San Anto, TX", "create_at": date("2010-08-15"), "description": "In the key of: trying to juggle all the things that make me happy. Co-Editor of @sobresound", "followers_count": 533, "friends_count": 536, "statues_count": 25919 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378519809392640, "text": "Wind 3.0 mph NW. Barometer 30.062 in, Rising Rapidly. Temperature 49.3 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378520094593024, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 60.3 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378520643887105, "text": "Temp: 58.9°F Wind:0.0mph Pressure: 30.025hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378520861990913, "text": "It's 2'o clock & i'm faded", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 748614139, "name": "Marisa (-:", "screen_name": "Marisad21", "lang": "en", "location": "Mtx", "create_at": date("2012-08-09"), "description": "null", "followers_count": 670, "friends_count": 712, "statues_count": 18082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mathis, TX", "id": "23ce8ba2308c3f30", "name": "Mathis", "place_type": "city", "bounding_box": rectangle("-97.839067,28.077909 -97.80972,28.112871") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48409, "countyName": "San Patricio", "cityID": 4847040, "cityName": "Mathis" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378521168355329, "text": "I hate these long nights at work I be so bummy and lazy every day now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237252668, "name": "Bry", "screen_name": "BryannaJimenez_", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-06-05"), "description": "Yours truly", "followers_count": 163, "friends_count": 163, "statues_count": 3017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378521663102977, "text": "@dahliamonroe23 lmaao girl me too!!", "in_reply_to_status": 731375953167847424, "in_reply_to_user": 606832142, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 606832142 }}, "user": { "id": 54332275, "name": "princess raven", "screen_name": "ravenjemmert", "lang": "en", "location": "null", "create_at": date("2009-07-06"), "description": "Instagram and snapchat: ravenemmert", "followers_count": 578, "friends_count": 335, "statues_count": 43336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378521789104128, "text": "I can't wait for the summer �� bouta be out every night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1347754345, "name": "Cardi D", "screen_name": "zaddddy", "lang": "en", "location": "your mans house ", "create_at": date("2013-04-12"), "description": "ρєяfє¢тℓу ιмρєяfє¢т", "followers_count": 127, "friends_count": 247, "statues_count": 3849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Islip, NY", "id": "c64c987193dab2f4", "name": "Islip", "place_type": "city", "bounding_box": rectangle("-73.243475,40.705546 -73.19854,40.763225") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3637869, "cityName": "Islip" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378521898000384, "text": "Sorry hold on, I'm dying after Toby just died because of a dude that I totally avoided in outlast ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1653974688, "name": "leximus rex ✌️", "screen_name": "c4lif0rnia23", "lang": "en", "location": "Denver, CO", "create_at": date("2013-08-07"), "description": "all I do is lift lift lift no matter what. (no really I'm super boring)", "followers_count": 1270, "friends_count": 1037, "statues_count": 53019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522128814080, "text": "I really enjoyed #GreaseLive! The renditions of the classic tunes and the \"jiving\" dance scenes were well executed! #GreaseLive������ @GoGrease", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GreaseLive", "GreaseLive" }}, "user_mentions": {{ 89744086 }}, "user": { "id": 132726674, "name": "William Ramsey", "screen_name": "NuwiZEdom", "lang": "en", "location": "Philadelphia, PA.", "create_at": date("2010-04-13"), "description": "Founder/Music Director of Voices of Soul Concert Chorale of Philadelphia, PA. William loves to sing, travel and spend quality time with quality people.", "followers_count": 648, "friends_count": 1092, "statues_count": 9832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522271457280, "text": "@yuupksk uha puccho miracle", "in_reply_to_status": -1, "in_reply_to_user": 2904825340, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user_mentions": {{ 2904825340 }}, "user": { "id": 4698420439, "name": "Michael Nguyen", "screen_name": "dreamstrata", "lang": "en", "location": "Worcester, MA", "create_at": date("2016-01-02"), "description": "Kill, the Fugitive Deathknight runs through the Dreamstrata Universe. A presentation of the arts. For leisure I'm a top class side scroller player.", "followers_count": 4748, "friends_count": 3508, "statues_count": 1708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Worcester, MA", "id": "28db2dbc4240f0b2", "name": "Worcester", "place_type": "city", "bounding_box": rectangle("-71.893265,42.210065 -71.731611,42.341455") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25027, "countyName": "Worcester", "cityID": 2582000, "cityName": "Worcester" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522371952641, "text": "In a mood to lay up and cuddle and talk ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2245068486, "name": "kourt..", "screen_name": "KourtneiCt", "lang": "en", "location": "Dalla$", "create_at": date("2013-12-13"), "description": "21. Miss Purple and Gold @cartierhoe ❤️", "followers_count": 1470, "friends_count": 691, "statues_count": 34077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522879447040, "text": "I be hating to get up & go use the bathroom it be to cold", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2884255963, "name": "IMANI❤️", "screen_name": "thuggaSISTER", "lang": "en", "location": "null", "create_at": date("2014-10-30"), "description": "KILLA'CO✨", "followers_count": 618, "friends_count": 502, "statues_count": 20484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522892046336, "text": "@KatyDeenihan ��❤️", "in_reply_to_status": 731372408477782016, "in_reply_to_user": 557353226, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 557353226 }}, "user": { "id": 3135652937, "name": "Christi", "screen_name": "Lanoyxo", "lang": "en", "location": "Plainfield, IL", "create_at": date("2015-04-04"), "description": "peace and prosperity", "followers_count": 580, "friends_count": 523, "statues_count": 9396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378522896384000, "text": "Wind 1.0 mph SW. Barometer 29.839 in, Falling. Temperature 56.3 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378523005460482, "text": "3am and I'm just standing in my kitchen eating Chinese food ¯\\_(ツ)_/¯", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 556185112, "name": "KT", "screen_name": "KatieKaaay22", "lang": "en", "location": "413/617", "create_at": date("2012-04-17"), "description": "Ya Neva Know. #Bruins #PPW", "followers_count": 833, "friends_count": 648, "statues_count": 67558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westfield, MA", "id": "e09538b2e39d94df", "name": "Westfield", "place_type": "city", "bounding_box": rectangle("-72.828565,42.073301 -72.684842,42.188583") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2576030, "cityName": "Westfield" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378523030478849, "text": "I love me some @iamcardib", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 866953267 }}, "user": { "id": 959868546, "name": "Taylor", "screen_name": "TayWestCoast", "lang": "en", "location": "Cali", "create_at": date("2012-11-19"), "description": "#VIEWS https://itun.es/us/RYFfcb", "followers_count": 1570, "friends_count": 338, "statues_count": 55656 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-14T00:00:02.000Z"), "id": 731378523412144128, "text": "Staying up late tonight watching #MSI2016. I would have never thought 5 years ago I would be spending my Friday night watching esports", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MSI2016" }}, "user": { "id": 22712897, "name": "Tom Nguyen", "screen_name": "tom_nguyen_", "lang": "en", "location": "Santa Monica", "create_at": date("2009-03-03"), "description": "Brand Mgmt & Content Marketing guru for @lolesports. (@redbull, @redbullESPORTS, @redbullSD alum, Photographer, Life Enthusiast) My tweet... About me.", "followers_count": 989, "friends_count": 427, "statues_count": 7964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378523466694656, "text": "I can make u a celebrity overnight", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228919403, "name": "Whiskeypapi", "screen_name": "irwinemedina", "lang": "en", "location": "Las Vegas", "create_at": date("2010-12-20"), "description": "Hala Madrid", "followers_count": 753, "friends_count": 683, "statues_count": 41739 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378523546345472, "text": "��- you're super pretty like m'a ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2733650856, "name": "May 31st", "screen_name": "Hamdiyaaaaaa", "lang": "en", "location": "$eattle", "create_at": date("2014-08-14"), "description": "#Justice4Hamza ❤️ Add me on snapchat : mrsmysteries", "followers_count": 657, "friends_count": 471, "statues_count": 10279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378523949178880, "text": "Wind 0.0 mph ---. Barometer 29.723 in, Steady. Temperature 49.6 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378524074967040, "text": "Let's hit gyros ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hu", "is_retweet": false, "user": { "id": 537637863, "name": "Dalton", "screen_name": "_bigpoppa66", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "I thoroughly enjoy boneless wings and I heavily fuck with myself. Time Magazines 2006 person of the year.", "followers_count": 465, "friends_count": 299, "statues_count": 6301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plant City, FL", "id": "5a5924d17d76eb63", "name": "Plant City", "place_type": "city", "bounding_box": rectangle("-82.267905,27.930926 -82.055659,28.093408") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1257550, "cityName": "Plant City" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378524242616320, "text": "I don’t want my happiness to be entirely dependent on somebody else’s…", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.15366826,26.00819938"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 186282028, "name": "movement mami", "screen_name": "thotpagnemami", "lang": "en", "location": "null", "create_at": date("2010-09-02"), "description": "liebing mami", "followers_count": 1193, "friends_count": 298, "statues_count": 112822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood, FL", "id": "2bc7c264a080898b", "name": "Hollywood", "place_type": "city", "bounding_box": rectangle("-80.248975,25.986228 -80.107808,26.093132") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1232000, "cityName": "Hollywood" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378524330688512, "text": "feel no ways ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1514835864, "name": "Katia", "screen_name": "katiachavezz", "lang": "en", "location": "null", "create_at": date("2013-06-13"), "description": "null", "followers_count": 541, "friends_count": 191, "statues_count": 18507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378524607549440, "text": "I just wanna wish my boy @Jimmys_Gone a Happy Birthday! Wish I was down there to celebrate with you my boy .. Be safe & have a great day! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 406733241 }}, "user": { "id": 879431406, "name": "J.R. Hensel ™", "screen_name": "2Story", "lang": "en", "location": "East Hemet, CA", "create_at": date("2012-10-13"), "description": "Psalm 126:6", "followers_count": 559, "friends_count": 250, "statues_count": 13605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, WA", "id": "ac2bdb59bf2a437e", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-122.357149,48.445589 -122.291309,48.495195") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53057, "countyName": "Skagit", "cityID": 5308920, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525018578945, "text": "@ThatsKingWest 2 words, I'm scary.", "in_reply_to_status": 731378246319644672, "in_reply_to_user": 326388002, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 326388002 }}, "user": { "id": 2217088734, "name": "ke.", "screen_name": "keiara_og", "lang": "en", "location": "cedarhill,tx", "create_at": date("2013-11-26"), "description": "null", "followers_count": 700, "friends_count": 450, "statues_count": 15307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525047906304, "text": "@MsLunaSol Aww I'd love this in person lol ;)", "in_reply_to_status": 731357038266081280, "in_reply_to_user": 3228451063, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3228451063 }}, "user": { "id": 93112987, "name": "augustine Perez", "screen_name": "plaguebrnger", "lang": "en", "location": "null", "create_at": date("2009-11-27"), "description": "null", "followers_count": 35, "friends_count": 238, "statues_count": 410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alhambra, CA", "id": "d563cf7307e4ba95", "name": "Alhambra", "place_type": "city", "bounding_box": rectangle("-118.165119,34.05998 -118.108233,34.111213") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600884, "cityName": "Alhambra" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525056335872, "text": "happy birthday nick ������✨ hope you have an amazing day you deserve it����@Nikitynick__ btw stop being smart w me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1508367955 }}, "user": { "id": 1682478372, "name": "✧", "screen_name": "bricyyy_", "lang": "en", "location": "null", "create_at": date("2013-08-18"), "description": "17. svhs co '16", "followers_count": 926, "friends_count": 643, "statues_count": 36988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525241024513, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":53.6°F Wind:1.3mph Pressure: 29.94hpa Falling Rain Today 0.00in. Forecast: Precipitation, very unse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 85, "friends_count": 17, "statues_count": 317796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525274525698, "text": "#DoubleCheeseSlider.\n\n#whitecastle @ White Castle Las Vegas https://t.co/79ZXjCtShU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17198578,36.12088538"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DoubleCheeseSlider", "whitecastle" }}, "user": { "id": 29942050, "name": "robert legaspi", "screen_name": "12obbyGene", "lang": "en", "location": " CALI ♡", "create_at": date("2009-04-09"), "description": "Tweeting my life away....", "followers_count": 54, "friends_count": 267, "statues_count": 4061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525622685696, "text": "Wind 0.0 mph WNW. Barometer 29.80 in, Falling slowly. Temperature 53.2 °F. Rain today 0.01 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378525714800642, "text": "I never wear skirts when I have closing shift bc I was taught that it isn't safe for me to wear them while walking at night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 752150570, "name": "Hills", "screen_name": "duarte_hilary", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "UC San Diego '19 • Not lucky, but blessed", "followers_count": 427, "friends_count": 360, "statues_count": 4770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378526172020736, "text": "Wind 0.0 mph ---. Barometer 30.026 in, Falling slowly. Temperature 68.5 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378526407036928, "text": "She Know I'm A Dope Boy But She Don't Have No Proof. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323520848, "name": "Wolf N Sheep Clothes", "screen_name": "LadyKiller313", "lang": "en", "location": "The Illuminati Weed Farm", "create_at": date("2011-06-24"), "description": "Alumni:The School Of Hard Knock, Associate Wit Bosses, Bachelor To Women, Master The Game, Doctoral On The Female Brain #TeamPatriots #TeamiPhone5s #TeamPisces", "followers_count": 657, "friends_count": 545, "statues_count": 21088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378526457200640, "text": "@bryanizaguirre6 killing it in 2k ��", "in_reply_to_status": -1, "in_reply_to_user": 2721422343, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2721422343 }}, "user": { "id": 2231639605, "name": "Luis", "screen_name": "luiscamposss21", "lang": "en", "location": "Lewisville, TX", "create_at": date("2013-12-05"), "description": "Soccer⚽️❤️|LHS|NTFC#9|5.8.16", "followers_count": 430, "friends_count": 215, "statues_count": 9432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378526775975936, "text": "Happy Birthday to my best friend, sister and daughter (Bernie is her… https://t.co/RpC8ZeuDk2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.276,44.5708"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2455924759, "name": "Erica", "screen_name": "OfABorderline", "lang": "en", "location": "Corvallis, OR ", "create_at": date("2014-04-20"), "description": "BPD+Lifestyle Blogger/ Activist / Daughter / Sister / Inspiring Cat Lady / Lots of Sass", "followers_count": 488, "friends_count": 2451, "statues_count": 3992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378526839025664, "text": "@SwoopSwoopSwagg lol if I had to choose then a gram I guess..not a party", "in_reply_to_status": 731378326082715648, "in_reply_to_user": 315564934, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 315564934 }}, "user": { "id": 324663232, "name": "courtnee", "screen_name": "court1017_", "lang": "en", "location": "DTX2HTX", "create_at": date("2011-06-26"), "description": "rare & aware", "followers_count": 2691, "friends_count": 1720, "statues_count": 69316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378527400923136, "text": "@jolie_pollock lene! Come see meeee", "in_reply_to_status": 731378394026278912, "in_reply_to_user": 3989243753, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3989243753 }}, "user": { "id": 2686048135, "name": "Melllll❤️", "screen_name": "melanieee_faith", "lang": "en", "location": "null", "create_at": date("2014-07-27"), "description": "why be low when you could be high?", "followers_count": 255, "friends_count": 133, "statues_count": 3340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abilene, TX", "id": "c386efac4d8d46d2", "name": "Abilene", "place_type": "city", "bounding_box": rectangle("-99.835853,32.321246 -99.665259,32.508856") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48441, "countyName": "Taylor", "cityID": 4801000, "cityName": "Abilene" } }
+{ "create_at": datetime("2016-05-14T00:00:03.000Z"), "id": 731378527476453376, "text": "I just came up with a really horrible idea for a movie. Like it would be so awful that like you GOTTA watch it high or something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 78735081, "name": "cannabitch", "screen_name": "wizlakiefa", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-09-30"), "description": "''Does baby got front?'' - Jill Sinadinos, 2016.", "followers_count": 819, "friends_count": 2003, "statues_count": 52434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378528214749184, "text": "�������������� #LateNightThoughts �� https://t.co/WvgrJuybMK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LateNightThoughts" }}, "user": { "id": 229359372, "name": "Omar Hernandez", "screen_name": "Omarsitodebtown", "lang": "en", "location": "Paradise", "create_at": date("2010-12-21"), "description": "Living well is the best revenge.\n♛Car Fanatic♛\nI ❤ to ▪█─────█▪\nSnapchat: Omarsitodebtown\nⓂ/// Nation", "followers_count": 298, "friends_count": 65, "statues_count": 2024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, NC", "id": "52cd3346625199d1", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-79.556369,36.028627 -79.379255,36.127415") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37001, "countyName": "Alamance", "cityID": 3709060, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378528596426752, "text": "I really do think I'm so damn funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1183950488, "name": "andrea ruby", "screen_name": "andrea_rubyy", "lang": "en", "location": "ya mans crib ", "create_at": date("2013-02-15"), "description": "accepting sugar daddy applications ✨", "followers_count": 572, "friends_count": 349, "statues_count": 23915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freedom, CA", "id": "304e2b3ba54b2a0e", "name": "Freedom", "place_type": "city", "bounding_box": rectangle("-121.810709,36.925702 -121.769974,36.950474") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 625576, "cityName": "Freedom" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378528982335488, "text": "But I'm home going to sleep fucking exhausted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163448831, "name": "Johnaè Mickela✨", "screen_name": "TheOnlyJMickela", "lang": "en", "location": "Pride Lands|Chicago", "create_at": date("2010-07-06"), "description": "GRANNY, DJ & MIRANDA❤• Hey I'm Nae✌• Philippians 4:13", "followers_count": 1009, "friends_count": 791, "statues_count": 33197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378529015762944, "text": "Wanting to be as bad as @spanishcvndy but just looking like a potato���� https://t.co/o8NeWDCzQx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3953296632 }}, "user": { "id": 1319989297, "name": "Diana", "screen_name": "HiiDeeana", "lang": "en", "location": "Los Angeles, Ca", "create_at": date("2013-04-01"), "description": "null", "followers_count": 57, "friends_count": 65, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378529603076096, "text": "Man you ain't never lied. https://t.co/SVmZVXzmAk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41994002, "name": "BIGNOAH", "screen_name": "BIGNOAH256", "lang": "en", "location": "Oxford, Alabama // Atlanta ,GA", "create_at": date("2009-05-23"), "description": "[Marketing Director @StreetExecs][@MeetTheCulture_][@A3C][@TeamKanyeDaily][Content Manager 4 Some Of Your Favorite Websites] ❌Submissions Noah.williams@me.com❌", "followers_count": 22464, "friends_count": 9750, "statues_count": 118375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anniston, AL", "id": "48c472deb9cb7d02", "name": "Anniston", "place_type": "city", "bounding_box": rectangle("-85.86503,33.615344 -85.75944,33.74226") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1015, "countyName": "Calhoun", "cityID": 101852, "cityName": "Anniston" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378529611341824, "text": "@12andrew95 aren't you trying to lose weight??", "in_reply_to_status": 731306667745251330, "in_reply_to_user": 2560537562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2560537562 }}, "user": { "id": 840710070, "name": "adriana", "screen_name": "adriana__mariah", "lang": "en", "location": "null", "create_at": date("2012-09-22"), "description": "18", "followers_count": 622, "friends_count": 441, "statues_count": 16508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378531075276801, "text": "@Jackiefloressss plagiarism!!!", "in_reply_to_status": 731378247695400961, "in_reply_to_user": 523823697, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 523823697 }}, "user": { "id": 2579537113, "name": "Raúl Martínez", "screen_name": "raulmmm7", "lang": "en", "location": "Yakima, WA", "create_at": date("2014-06-20"), "description": "A.C. Davis QB BBFS #7", "followers_count": 429, "friends_count": 298, "statues_count": 7519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yakima, WA", "id": "b52ffa33d40078ca", "name": "Yakima", "place_type": "city", "bounding_box": rectangle("-120.59824,46.55658 -120.461279,46.631865") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5380010, "cityName": "Yakima" } }
+{ "create_at": datetime("2016-05-14T00:00:04.000Z"), "id": 731378531486322688, "text": "Way up I feel blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252352941, "name": "Larry Barty", "screen_name": "letsparty_barty", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-02-14"), "description": "#GoBrowns", "followers_count": 667, "friends_count": 522, "statues_count": 15116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brunswick, OH", "id": "2c8ced473810bde2", "name": "Brunswick", "place_type": "city", "bounding_box": rectangle("-81.904006,41.20974 -81.756029,41.276433") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39103, "countyName": "Medina", "cityID": 3909680, "cityName": "Brunswick" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532408918016, "text": "Alone with the feels tonight ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312853563, "name": "Jenna Beall", "screen_name": "JennaBeall", "lang": "en", "location": "Colorado", "create_at": date("2011-06-07"), "description": "Proverbs 31:25", "followers_count": 131, "friends_count": 320, "statues_count": 802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Grove, OR", "id": "beedf9d8e2499b64", "name": "Forest Grove", "place_type": "city", "bounding_box": rectangle("-123.15354,45.501953 -123.070256,45.542318") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4126200, "cityName": "Forest Grove" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532543266816, "text": "Chiropractic Physicians, Tacoma, WA, 98408 - https://t.co/9FiZSJF8FK\n\n#chiropracticphysicians #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.436402,47.192689"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "chiropracticphysicians", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 543, "friends_count": 539, "statues_count": 829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532841099264, "text": "Melt-free summer wedding makeup tips https://t.co/RniMjiKlZi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.00093296,40.7206258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1382621, "name": "Liz Lanuzo", "screen_name": "lizlanuzo", "lang": "en", "location": "Philippines", "create_at": date("2007-03-17"), "description": "Beauty blogger. Brand consultant.", "followers_count": 3385, "friends_count": 392, "statues_count": 20049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532853682176, "text": "Get Ready for Summer with Holland & Barrett https://t.co/0x52WwN5wp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2174159118, "name": "Sally ", "screen_name": "sallysaysbeauty", "lang": "en", "location": "East London", "create_at": date("2013-11-04"), "description": "London-based Welsh lifestyle & beauty blogger. Makeup hoarder, Labrador lover and avocado enthusiast. Contact: sallysaysbeauty@outlook.com", "followers_count": 1364, "friends_count": 851, "statues_count": 7621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532920623105, "text": "#IDiedBecause_______ I ran out of tequila", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IDiedBecause_______" }}, "user": { "id": 3023693328, "name": "~♡°Katie°♡~", "screen_name": "_Katiebyy69", "lang": "en", "location": "Washington State USA", "create_at": date("2015-02-17"), "description": "Seahawks Fan... Sarcastic but loveable #Hashtagger.... I Retweet a lot ...Weirdo, deal with it! [My life is very confusing.] {Not Single} @KShaffer_", "followers_count": 2441, "friends_count": 1489, "statues_count": 36399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Everett, WA", "id": "76e74b864547959b", "name": "Everett", "place_type": "city", "bounding_box": rectangle("-122.294205,47.884492 -122.169372,48.035407") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5322640, "cityName": "Everett" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532937572353, "text": "Just posted a photo @ Roxy Bar https://t.co/CVGJflSme6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.66885337,38.9933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32602390, "name": "Don Mac", "screen_name": "ThisIsMrMcClure", "lang": "en", "location": "Kansas City, KS", "create_at": date("2009-04-17"), "description": "My heart is always in the right place, can you say the same? http://instagram.com/#don_mac913 Christian Libra #StopTheViolenceKC", "followers_count": 1895, "friends_count": 2049, "statues_count": 59303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378532954312705, "text": "Saturday 5-14-16 https://t.co/PKlbXW8ABp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1148, "friends_count": 324, "statues_count": 3598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378533499461632, "text": "Room 129 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3303504820, "name": "Big Daddy Arieee ❤", "screen_name": "_thickassarieee", "lang": "en", "location": "null", "create_at": date("2015-05-29"), "description": "Some People Come In Your Life For A Blessing And Others Come In Your Life As Lessons Learn Dj & Aiyana Is Life ✊", "followers_count": 525, "friends_count": 242, "statues_count": 8190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378533541384192, "text": "@mixedxprincessa thank u love u bb��❤️��❤️", "in_reply_to_status": 731378021060333568, "in_reply_to_user": 2258232186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2258232186 }}, "user": { "id": 358587346, "name": "✨Miranda✨", "screen_name": "Miranduhh_ann", "lang": "en", "location": "off on the adventure ", "create_at": date("2011-08-19"), "description": "GGT❤️ // FJC// ♡ @breezieyeezy is my husband", "followers_count": 424, "friends_count": 363, "statues_count": 2103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378533608620032, "text": "Class of 2016, we did it!!!���� @ Jefferson R 7 High School https://t.co/nFSO86sKR6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.34752159,38.15845797"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1428069482, "name": "❂ madalynn ❂", "screen_name": "BrowMaddie", "lang": "en", "location": "null", "create_at": date("2013-05-14"), "description": "♋️", "followers_count": 297, "friends_count": 151, "statues_count": 2177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri, USA", "id": "2526edd24c06e60c", "name": "Missouri", "place_type": "admin", "bounding_box": rectangle("-95.774704,35.995476 -89.098843,40.613641") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29099, "countyName": "Jefferson" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378533696733184, "text": "I REALLY WANT TO WATCH SHREK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 874069633, "name": "Michael Forward", "screen_name": "mrforward9", "lang": "en", "location": "null", "create_at": date("2012-10-11"), "description": "I don't know what to put here...", "followers_count": 159, "friends_count": 255, "statues_count": 3678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miamisburg, OH", "id": "6252d06c3e5bb9f9", "name": "Miamisburg", "place_type": "city", "bounding_box": rectangle("-84.30945,39.584523 -84.202828,39.675365") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3949434, "cityName": "Miamisburg" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378534699044864, "text": "@jacob_s01 obstacles �� https://t.co/c6gRsESAGj", "in_reply_to_status": 731377785923469312, "in_reply_to_user": 2564509418, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2564509418 }}, "user": { "id": 1646628625, "name": "Marlene.", "screen_name": "Marlaaa1218", "lang": "en", "location": "SATX / RGV", "create_at": date("2013-08-04"), "description": "#UTSA • I am second, God is first • IG & SC: marla_1995", "followers_count": 807, "friends_count": 432, "statues_count": 10929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378534740951040, "text": "Wind 2.0 mph SSE. Barometer 29.867 in, Steady. Temperature 59.8 °F. Rain today 0.00 in. Humidity 50%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.05444444,39.24222222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467866396, "name": "Nevada City Weather", "screen_name": "NevadaCountyWX", "lang": "en", "location": "Nevada City, California", "create_at": date("2012-01-18"), "description": "Nevada County Weather. Home of the only 24/7 Live weather station with twitter updates. Plan your day. Follow my weather discussion: http://t.co/mdsOJfCe3d", "followers_count": 208, "friends_count": 59, "statues_count": 276167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grass Valley, CA", "id": "0a40560cd52f96a7", "name": "Grass Valley", "place_type": "city", "bounding_box": rectangle("-121.114756,39.182283 -121.00162,39.248861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6057, "countyName": "Nevada", "cityID": 630798, "cityName": "Grass Valley" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378535017893888, "text": "Deadass just got home and my stupid ass forgot to eat ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2510860929, "name": "meraris", "screen_name": "JuiceGirl_", "lang": "en", "location": "null", "create_at": date("2014-04-26"), "description": "happy.", "followers_count": 475, "friends_count": 317, "statues_count": 9914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378535537860608, "text": "Don't start getting all cute when I'm mad cause I'm trying to hate you and your making it very difficult", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1040547732, "name": "Vincent N4khou7 ┗╋┓", "screen_name": "CheRev_48", "lang": "en", "location": "Planet Earth", "create_at": date("2012-12-27"), "description": "Live righteously and give priceless things. #FourteySevenGuerrillas┗╋┓", "followers_count": 109, "friends_count": 171, "statues_count": 902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378535982489601, "text": "sugar_la_ritz on stage!!! Sexy as hell! @ Lancaster Moose Lodge https://t.co/yYwUZIEH7M", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.13087,34.69216"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1243007310, "name": "Crystal Kittell", "screen_name": "spaceradish", "lang": "en", "location": "Lancaster, CA", "create_at": date("2013-03-04"), "description": "maker of things, generally awesome, weirdo. ask me about my coloring zines. ♉️", "followers_count": 103, "friends_count": 121, "statues_count": 5194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-14T00:00:05.000Z"), "id": 731378535982632960, "text": "people will forget what you did, but people will never forget how you made them feel.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1945331311, "name": "diego t.", "screen_name": "diegotors4797", "lang": "en", "location": "null", "create_at": date("2013-10-07"), "description": "Art Live 10.14", "followers_count": 489, "friends_count": 94, "statues_count": 3104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doral, FL", "id": "053ab8e6678199ed", "name": "Doral", "place_type": "city", "bounding_box": rectangle("-80.41729,25.780974 -80.321016,25.870313") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1217935, "cityName": "Doral" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378536288636928, "text": "usage of the dog filter doesn't make a girl a hoe??? where do u people come up with this ass-backwards logic", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509538043, "name": "ketchup jones", "screen_name": "megsivakova", "lang": "en", "location": "asshole hell ", "create_at": date("2012-02-29"), "description": "loves it", "followers_count": 8724, "friends_count": 643, "statues_count": 78184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378536318001153, "text": "My total 5$ cause I cheated on a test once https://t.co/IET8HE6dpq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2170545162, "name": "↳", "screen_name": "marco_hendrix", "lang": "en", "location": "Stockton, CA✈️Queen Creek AZ", "create_at": date("2013-11-02"), "description": "qchs senior 18. |", "followers_count": 1845, "friends_count": 593, "statues_count": 19247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queen Creek, AZ", "id": "01cb573821d94344", "name": "Queen Creek", "place_type": "city", "bounding_box": rectangle("-111.686314,33.196614 -111.582748,33.288127") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 458150, "cityName": "Queen Creek" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378537110732800, "text": "There it is", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223292873, "name": "Andrew", "screen_name": "andrewakarhino", "lang": "en", "location": "Soledad ", "create_at": date("2010-12-05"), "description": "Rest in Peace Mommy 12.5.14", "followers_count": 495, "friends_count": 314, "statues_count": 37396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soledad, CA", "id": "642fbca6eea15240", "name": "Soledad", "place_type": "city", "bounding_box": rectangle("-121.338707,36.410518 -121.306524,36.445593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 672520, "cityName": "Soledad" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378537354039296, "text": "Happy bday to my little ball of sunshine who gives the best advice @missywilb I love you girl u deserve to feel 22 ❤ https://t.co/HbG60xwWrU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 211251084 }}, "user": { "id": 35798336, "name": "jules", "screen_name": "hylianbyheart", "lang": "en", "location": "null", "create_at": date("2009-04-27"), "description": "null", "followers_count": 240, "friends_count": 152, "statues_count": 5772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378538117365760, "text": "@Ryleighwill STIP BEING SOFT RYLEIGH UR HETTER THAN THAT", "in_reply_to_status": 731370377654362112, "in_reply_to_user": 268138952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 268138952 }}, "user": { "id": 515031130, "name": "Gabby Carvalho", "screen_name": "TheRealGabbyC", "lang": "en", "location": "Arizona State ΑΦ", "create_at": date("2012-03-04"), "description": "sry, I'm too busy thriving", "followers_count": 1112, "friends_count": 644, "statues_count": 6979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-14T00:00:06.000Z"), "id": 731378538369032192, "text": "Kinda wanna talk otp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2703276199, "name": "cari", "screen_name": "carisma113", "lang": "en", "location": "null", "create_at": date("2014-08-02"), "description": "null", "followers_count": 391, "friends_count": 355, "statues_count": 16262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740898745155584, "text": "@187_marc do it", "in_reply_to_status": 731735763465867264, "in_reply_to_user": 2255583175, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2255583175 }}, "user": { "id": 351232746, "name": "⚫▪️痛み▪️⚫️", "screen_name": "__YaBoii_18", "lang": "en", "location": "Vacaville, CA", "create_at": date("2011-08-08"), "description": "null", "followers_count": 754, "friends_count": 506, "statues_count": 12388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899177332736, "text": "@happierman @PlanetofFinks thank you thank you for the care u use when choosing voice actors. Representation matters & you guys set the bar", "in_reply_to_status": -1, "in_reply_to_user": 14140101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14140101, 40262799 }}, "user": { "id": 2476345046, "name": "it me", "screen_name": "gladiolusfae", "lang": "en", "location": "null", "create_at": date("2014-05-03"), "description": "null", "followers_count": 39, "friends_count": 148, "statues_count": 584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899357511680, "text": "I nominate @bruhitszach #socialmediaking HE TOTALLY DESERVE IT❤️❤️�� #bestbroadcaster! https://t.co/fj4U90BcnD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "socialmediaking", "bestbroadcaster" }}, "user_mentions": {{ 1912839878 }}, "user": { "id": 1730474407, "name": "alyssa_joyner(;", "screen_name": "ilovedinozzo123", "lang": "en", "location": "United States", "create_at": date("2013-09-04"), "description": "I'm probably more obsessed with NCIS", "followers_count": 944, "friends_count": 460, "statues_count": 180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boardman, OH", "id": "f33f520be81a525c", "name": "Boardman", "place_type": "city", "bounding_box": rectangle("-80.71159,40.970454 -80.603596,41.061426") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3907454, "cityName": "Boardman" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899365904384, "text": "@tressadavis_ not true", "in_reply_to_status": 731740255942926342, "in_reply_to_user": 25010438, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25010438 }}, "user": { "id": 412643326, "name": "Megan Qtips", "screen_name": "megankubitz", "lang": "en", "location": "Pullman, WA", "create_at": date("2011-11-14"), "description": "GPhi @ WSU. Juuuusttt kaaazzoooo it", "followers_count": 815, "friends_count": 713, "statues_count": 7928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pullman, WA", "id": "6045d4aebc672078", "name": "Pullman", "place_type": "city", "bounding_box": rectangle("-117.213215,46.70823 -117.095324,46.753414") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53075, "countyName": "Whitman", "cityID": 5356625, "cityName": "Pullman" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899374448640, "text": "@fedachinii https://t.co/lfN4iNlw3f", "in_reply_to_status": 731740581496389633, "in_reply_to_user": 922204956, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 922204956 }}, "user": { "id": 520038553, "name": "Ameer", "screen_name": "ItsRealKing", "lang": "en", "location": "New York, USA", "create_at": date("2012-03-09"), "description": "// 2050// Self Confessed Egotist // Corleone// Lebron James is the greatest basketball player of all time// Comics are everything//", "followers_count": 708, "friends_count": 423, "statues_count": 71054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yonkers, NY", "id": "b87b05856ab8dbd8", "name": "Yonkers", "place_type": "city", "bounding_box": rectangle("-73.911271,40.900789 -73.810443,40.988346") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3684000, "cityName": "Yonkers" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899470807040, "text": "You're 5' 6\" I'm sorry to ruin your life. https://t.co/JLghqWr8wG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 634344124, "name": "Sarah Smith", "screen_name": "katheryn796", "lang": "en", "location": "Tacoma - Goodyear", "create_at": date("2012-07-12"), "description": "My body is 57% sriracha. Paraeducator ✏ I'm your typical annoying vegan. @NickLord10❤️", "followers_count": 368, "friends_count": 152, "statues_count": 24710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clover Creek, WA", "id": "6a3c00987122ff28", "name": "Clover Creek", "place_type": "city", "bounding_box": rectangle("-122.415738,47.110842 -122.357057,47.159203") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5313215, "cityName": "Clover Creek" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740899626061824, "text": "Piss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1489498172, "name": "goddess✨", "screen_name": "kidsunflower", "lang": "en", "location": "Atlanta | Saint Louie", "create_at": date("2013-06-06"), "description": "kisses for ke.", "followers_count": 505, "friends_count": 210, "statues_count": 42980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740900167057408, "text": "Minus the \"awkward and shy\" and this girl is my spirit animal. And no, it's not okay to tell me to smile ����@BuzzFeed https://t.co/ugYRmr0HF9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5695632 }}, "user": { "id": 28842224, "name": "Emily Oslie", "screen_name": "emilyoslie", "lang": "en", "location": "Queen Anne, Seattle", "create_at": date("2009-04-04"), "description": "Seattle. Medicine. Duke 2018. I like strong handshakes/strong eyebrows. Be nice.", "followers_count": 275, "friends_count": 156, "statues_count": 6331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740900259332096, "text": "Wind 0.0 mph ---. Barometer 1026.86 mb, Rising. Temperature 51.2 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740900670484480, "text": "I want mcdonalds now @corona70aaaaaa :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2591739200 }}, "user": { "id": 3248106277, "name": "Rolando :)", "screen_name": "rollytorres851", "lang": "en", "location": "null", "create_at": date("2015-06-17"), "description": "it's not always about the trophy at the end, sometimes the real trophy is the thrill of the hunt.", "followers_count": 784, "friends_count": 679, "statues_count": 19971 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, TX", "id": "c0c05c8e8a0b466a", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-98.249995,27.746924 -98.228662,27.773699") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48131, "countyName": "Duval", "cityID": 4865180, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740901492412416, "text": "SWEET DREAMS ARE MADE OF THESE\nWHO AM I TO DISAGREE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429695311, "name": "croque madame", "screen_name": "brunchplug", "lang": "en", "location": "♡Vegas w/ @AdultRaverProbs♡", "create_at": date("2011-12-06"), "description": "nut milk & chill?", "followers_count": 2000, "friends_count": 757, "statues_count": 136744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central City, Los Angeles", "id": "6f14989a9834f276", "name": "Central City", "place_type": "neighborhood", "bounding_box": rectangle("-118.272897,34.018005 -118.237843,34.0626") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740901626675200, "text": "Mood https://t.co/QJINTw7wdN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2152175257, "name": "kelsie", "screen_name": "_klf", "lang": "en", "location": "Splashtown, TX", "create_at": date("2013-10-23"), "description": "hot on my own, check ya tone, check ya stance homie", "followers_count": 1150, "friends_count": 316, "statues_count": 4662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902088146944, "text": "Iz dat time ....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3396778874, "name": "JEV", "screen_name": "juanvalles8D", "lang": "en", "location": "Petare, Venezuela", "create_at": date("2015-08-30"), "description": "oh my", "followers_count": 247, "friends_count": 204, "statues_count": 862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weston, FL", "id": "88a21205dd4728ea", "name": "Weston", "place_type": "city", "bounding_box": rectangle("-80.442024,26.062031 -80.361426,26.144693") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1276582, "cityName": "Weston" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902113189889, "text": "@crystalblisters I think lamont is still awake", "in_reply_to_status": 731740184287449088, "in_reply_to_user": 30982822, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 30982822 }}, "user": { "id": 1321739216, "name": "known degree haver", "screen_name": "RaviNoBasuke", "lang": "en", "location": "PHX", "create_at": date("2013-04-01"), "description": "22 | m | sad phoenix suns fan | @bballadventure's known process truster | profile pic by @rikisenapai", "followers_count": 243, "friends_count": 178, "statues_count": 15781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902125772800, "text": "@ebbtideapp Tide in Deer Island, Massachusetts 05/15/2016\nHigh 7:14am 9.0\n Low 1:27pm 1.0\nHigh 7:51pm 9.0\n Low 1:55am 1.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.9583,42.345"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 32896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Massachusetts, USA", "id": "cd450c94084cbf9b", "name": "Massachusetts", "place_type": "admin", "bounding_box": rectangle("-73.508143,41.187054 -69.858861,42.886824") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902247403521, "text": "this makes me so happy though ���� https://t.co/oXo1iJij5n", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3243074594, "name": "Vionne", "screen_name": "vionnexgarcia", "lang": "en", "location": "pisay '21", "create_at": date("2015-06-12"), "description": "ig • ask • tumblr | vionnexgarcia ; snap | vionnegarcia", "followers_count": 257, "friends_count": 210, "statues_count": 6561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902411108352, "text": "03:00:02 |Temp: 61.1ºF |Dew Point 47.1ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the WNW, Gusting to 2.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 94, "friends_count": 21, "statues_count": 94165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902666993664, "text": "yo I really have to put my backpack on and my shoulders are odee burnt ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413708453, "name": "️", "screen_name": "gabriellailene", "lang": "en", "location": "845 | 347 ", "create_at": date("2011-11-15"), "description": "ig: xo__gabriella | sc: gabriellailene", "followers_count": 782, "friends_count": 462, "statues_count": 52182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902675320833, "text": "Everybody got choices", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383940525, "name": "Jordan DeCaro", "screen_name": "JordanDeCaro12", "lang": "en", "location": "Arizona, USA", "create_at": date("2011-10-02"), "description": "null", "followers_count": 1045, "friends_count": 849, "statues_count": 10441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, MN", "id": "62619a76134ad05e", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-92.282745,46.650616 -91.962935,46.866053") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2717000, "cityName": "Duluth" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902683631616, "text": "I have to wake up at 4:00 in the morning for my volleyball tournament, & it is midnight now... �� #BVC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BVC" }}, "user": { "id": 2986520887, "name": "ash❂", "screen_name": "a_daaaaawwwwggg", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "Haters = [H]aving [A]nger [T]owards [E]veryone [R]eaching [S]uccess.", "followers_count": 737, "friends_count": 1004, "statues_count": 5187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oildale, CA", "id": "006cd6a1177183cd", "name": "Oildale", "place_type": "city", "bounding_box": rectangle("-119.075005,35.397139 -119.00029,35.461391") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 653448, "cityName": "Oildale" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902759104516, "text": "Who's looking for Rye? @mgroves @Want_Some_Rye #rye #vegas #las https://t.co/XXcvSZMaJY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rye", "vegas", "las" }}, "user_mentions": {{ 13805382, 1480575031 }}, "user": { "id": 351429428, "name": "Nathan Groves", "screen_name": "njgroves", "lang": "en", "location": "null", "create_at": date("2011-08-09"), "description": "null", "followers_count": 35, "friends_count": 151, "statues_count": 549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-15T00:00:00.000Z"), "id": 731740902863953920, "text": "she hit me ona late night Turn that into a great night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2263903351, "name": "ashley", "screen_name": "ashbangaaa", "lang": "en", "location": "null", "create_at": date("2013-12-26"), "description": "null", "followers_count": 1417, "friends_count": 1981, "statues_count": 15459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broadmoor, CA", "id": "3b8340ca518a04fb", "name": "Broadmoor", "place_type": "city", "bounding_box": rectangle("-122.489276,37.68575 -122.471358,37.696309") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 608338, "cityName": "Broadmoor" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740903409369089, "text": "@OxiLana nao foi pra mim isso, mas eu tinha ouvinte que migro 3 radios cmg kkk", "in_reply_to_status": 731740340449910784, "in_reply_to_user": 705991607770812417, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 705991607770812417 }}, "user": { "id": 75426670, "name": "Lenon Arnoso", "screen_name": "LennonRox", "lang": "en", "location": "Deerfield Beach, FL", "create_at": date("2009-09-18"), "description": "Locutor • Youtuber • Poeta", "followers_count": 1515, "friends_count": 1011, "statues_count": 6727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Middletown, DE", "id": "01c4343517e7cc35", "name": "Middletown", "place_type": "city", "bounding_box": rectangle("-75.742439,39.409781 -75.665128,39.477614") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1047030, "cityName": "Middletown" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740903409393664, "text": "if ya short. Light skinned .. pretty ass hair. #TakeMyLife", "in_reply_to_status": 731740781850071040, "in_reply_to_user": 63270029, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TakeMyLife" }}, "user": { "id": 63270029, "name": "☕️", "screen_name": "jasperdontcare", "lang": "en", "location": "null", "create_at": date("2009-08-05"), "description": "null", "followers_count": 1414, "friends_count": 651, "statues_count": 55695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thunderbolt, GA", "id": "019ac4415d9771c2", "name": "Thunderbolt", "place_type": "city", "bounding_box": rectangle("-81.076741,31.988891 -81.027701,32.047874") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13051, "countyName": "Chatham", "cityID": 1376364, "cityName": "Thunderbolt" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740903702847488, "text": "don't let her do you like that fdb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2237061295, "name": "mich", "screen_name": "michmart23", "lang": "en", "location": "en tu corazon ", "create_at": date("2013-12-08"), "description": "18 @blancomariaa", "followers_count": 711, "friends_count": 556, "statues_count": 29270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904046755840, "text": "I would love to post our group picture but I can't because @LeeBrown_V still has it �� #WaitingOnLee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WaitingOnLee" }}, "user_mentions": {{ 286696582 }}, "user": { "id": 963988939, "name": "Chrissi 2.0", "screen_name": "ChrissiRose_V", "lang": "en", "location": "#StarbelsheEntourage", "create_at": date("2012-11-22"), "description": "Let me know when you're ready..... ❤️ Ambassador for @PhaseVMusic • #UNITY • @JaylexisSDL_V is my #Huffletwin • #MEETLOSFIVE", "followers_count": 15335, "friends_count": 14778, "statues_count": 68492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904046764034, "text": "Lol @nomenative https://t.co/T46CdUYKfC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2595439032 }}, "user": { "id": 1530242622, "name": "kevynn", "screen_name": "kevynn_ann", "lang": "en", "location": "Nome↔️Sitka", "create_at": date("2013-06-19"), "description": "hit it Fergie", "followers_count": 537, "friends_count": 759, "statues_count": 15802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nome, AK", "id": "0074028c69e4944e", "name": "Nome", "place_type": "city", "bounding_box": rectangle("-165.428726,64.493046 -165.369072,64.52612") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2180, "countyName": "Nome", "cityID": 254920, "cityName": "Nome" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904378105856, "text": "@ItsGirllCode @Cantuanel @__ShelbyJo ������", "in_reply_to_status": 729053398595719169, "in_reply_to_user": 1316989550, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1316989550, 256239182, 2200441932 }}, "user": { "id": 488879465, "name": "Allissa Cyanne", "screen_name": "CaptainAllissa", "lang": "en", "location": " Wrestling • #WrestleForRachel", "create_at": date("2012-02-10"), "description": "If passion drives you, let reason hold the reins • Proverbs 21:31 • UofA Commit", "followers_count": 1746, "friends_count": 1360, "statues_count": 108041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904466333696, "text": "Temp 54.4° Hi/Lo 60.3/54.4 Rng 5.9° WC 54.4° Hmd 67% Rain 0.00\" Storm 0.00\" BAR 29.558 Falling DP 43.7° Wnd 3mph Dir NNE Gst 11mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 122, "statues_count": 18698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904759959554, "text": "i always fuck everything up.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2357081343, "name": "princess", "screen_name": "saige_mariee", "lang": "en", "location": "esu '20", "create_at": date("2014-02-21"), "description": "heartless.", "followers_count": 267, "friends_count": 345, "statues_count": 15479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Milford, NJ", "id": "d4aed8f0a0dd6fb4", "name": "New Milford", "place_type": "city", "bounding_box": rectangle("-74.033568,40.913851 -74.002511,40.952074") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3451660, "cityName": "New Milford" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740904889946112, "text": "Wind 0.0 mph WNW. Barometer 30.114 in, Falling slowly. Temperature 40.0 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 40, "friends_count": 4, "statues_count": 27430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905028227072, "text": "What baby wants, baby gets! Dinner for 2. No sweat. Whatever I want. https://t.co/FiTBlJJsrD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2449800350, "name": "Theresa", "screen_name": "TINGWEAR", "lang": "en", "location": "null", "create_at": date("2014-04-17"), "description": "building an empire for my angels....", "followers_count": 108, "friends_count": 211, "statues_count": 1995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905112281088, "text": "Wind 0.0 mph ---. Barometer 30.179 in, Steady. Temperature 42.7 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905498157056, "text": "And why are we even talking about @GordonBrown versus @BorisJohnson , @guardian? Why torture us? We all already live in a fascist regime.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 6107792, 3131144855, 87818409 }}, "user": { "id": 127057663, "name": "nico phillips", "screen_name": "nicosuavehh", "lang": "en", "location": "London W2", "create_at": date("2010-03-27"), "description": "We all wanna believe we are pizza rat but secretly know we're cotton candy raccoon. Proudly with Love Activists London (Norbert Lawrie crew) & Occupy Democracy", "followers_count": 542, "friends_count": 1321, "statues_count": 34867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905506541568, "text": "Temp 39.5°F Wind Chill 39.5°F RH 71% Wind 2.7 W Gust 8.0 W SLP 30.051 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 110, "friends_count": 63, "statues_count": 35436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905514897409, "text": "Wind 0.0 mph ---. Barometer 30.103 in, Falling slowly. Temperature 42.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905703505922, "text": "Sana mag trend na, tutulog na po ako. Maaga pa sa church bukas.\n\n#WWSSMTreceMartires", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "WWSSMTreceMartires" }}, "user": { "id": 3265767523, "name": "ML", "screen_name": "MLfromCA", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-07-01"), "description": "A follower of Christ. I support and love @MARSTELLARACAL ~ CTTO for pics posted :)", "followers_count": 397, "friends_count": 94, "statues_count": 22083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740905992945665, "text": "\"Maybe ur intimidating ��\" I ain't shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40809894, "name": "ThatGoonSits", "screen_name": "ThatGoonsits", "lang": "en", "location": "Venice, Los Angeles", "create_at": date("2009-05-17"), "description": "IG: @Thatgoonsits @detoxsquad", "followers_count": 13066, "friends_count": 12369, "statues_count": 27554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740906240544768, "text": "Wind 0.0 mph ENE. Barometer 30.10 in, Rising slowly. Temperature 63.9 °F. Rain today 0.00 in. Humidity 68%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740906311684096, "text": "Love yours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1100104058, "name": "D$$", "screen_name": "Darionng", "lang": "en", "location": "null", "create_at": date("2013-01-17"), "description": "belong to the world", "followers_count": 149, "friends_count": 179, "statues_count": 1458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740906466856960, "text": "mchristine_kubo this is worse than our bad girl skater Stage #SkinnyJeansAndGraphicTees https://t.co/XBR2TrWNLR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SkinnyJeansAndGraphicTees" }}, "user": { "id": 3148472864, "name": "Ashley Bluhm", "screen_name": "ashleyseverene", "lang": "en", "location": "my bed", "create_at": date("2015-04-10"), "description": "barista and coffee addict☕️|| instructor at Pop Star Kids USA|| proud dog mom", "followers_count": 526, "friends_count": 374, "statues_count": 3925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gig Harbor, WA", "id": "7f8e88c74409abb5", "name": "Gig Harbor", "place_type": "city", "bounding_box": rectangle("-122.617642,47.290796 -122.569191,47.360103") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5326735, "cityName": "Gig Harbor" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740906995503104, "text": "Temp: 65.2°F Wind:0.4mph Pressure: 30.067hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-15T00:00:01.000Z"), "id": 731740907003871232, "text": "Ripley SW Limestone Co. Temp: 49.1°F Wind:1.6mph Pressure: 999.8mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740907318300673, "text": "@SpikeFiremane It was good meeting you, man! Hope to see you tomorrow! Good night!", "in_reply_to_status": -1, "in_reply_to_user": 106643415, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 106643415 }}, "user": { "id": 23177604, "name": "Carlos @ EFNW", "screen_name": "OneRadicalDude", "lang": "en", "location": "Seattle, WA, USA", "create_at": date("2009-03-07"), "description": "Arkansas native, now Washingtonian. One who enjoys conventions, animation, music, traveling, good food, gaming, and worshiping Jesus.", "followers_count": 313, "friends_count": 665, "statues_count": 6150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "SeaTac, WA", "id": "c8b06a459cc8f78a", "name": "SeaTac", "place_type": "city", "bounding_box": rectangle("-122.326102,47.396426 -122.266875,47.48874") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5362288, "cityName": "SeaTac" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740907351855105, "text": "You made the gumbo roux from Galveston Gulf water? https://t.co/v77YAzQ6Ib", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114870635, "name": "Ms. Alyce.", "screen_name": "GloUpElite", "lang": "en", "location": "Read Receipt-No Reply, TX", "create_at": date("2010-02-16"), "description": "Be soul food, not eye candy. Never been a good girl but I'm a better woman now... trying to stay a good woman in a world full of bad bitches. SC: Lycee.b", "followers_count": 3858, "friends_count": 556, "statues_count": 162084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740907452694528, "text": "goddess https://t.co/IVkiEVyeOE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2749912959, "name": "turk", "screen_name": "asap_turkiee", "lang": "en", "location": "Orlando, FL", "create_at": date("2014-08-21"), "description": "professional finesser", "followers_count": 334, "friends_count": 331, "statues_count": 8471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alafaya, FL", "id": "00d511d335cd9fb6", "name": "Alafaya", "place_type": "city", "bounding_box": rectangle("-81.244501,28.489488 -81.125632,28.568794") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1200410, "cityName": "Alafaya" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740907821801472, "text": "69.3F (Feels: 69.3F) - Humidity: 90% - Wind: 2.2mph NE - Gust: 4.5mph - Pressure: 1014.7mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 240086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740907981049863, "text": "#TeenChoice i nominate @bruhitszach for #SocialMediaKing because he's amazing!! @TeenChoiceFOX #TeenChoice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TeenChoice", "SocialMediaKing", "TeenChoice" }}, "user_mentions": {{ 1912839878, 45609049 }}, "user": { "id": 20714963, "name": "majic", "screen_name": "magi81", "lang": "en", "location": "California, USA", "create_at": date("2009-02-12"), "description": "#SinTuAmor #MarioBautista 11:14 5•12•16", "followers_count": 296, "friends_count": 276, "statues_count": 12755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Madera, CA", "id": "d9d3e9c476e0c0dc", "name": "Madera", "place_type": "city", "bounding_box": rectangle("-120.110397,36.928736 -120.017261,36.998745") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 645022, "cityName": "Madera" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740908375281665, "text": "@lolmya_ lol", "in_reply_to_status": 731583391150133248, "in_reply_to_user": 2612871462, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2612871462 }}, "user": { "id": 4675957160, "name": "Maguire Kane", "screen_name": "maguire_kane", "lang": "en", "location": "Redwood City, CA", "create_at": date("2015-12-29"), "description": "Actor / Performer / Model / Stuntman / Innovator Creator of Camp Adrenaline / Mya", "followers_count": 185, "friends_count": 184, "statues_count": 525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redwood City, CA", "id": "a409256339a7c6a1", "name": "Redwood City", "place_type": "city", "bounding_box": rectangle("-122.28853,37.443954 -122.177339,37.550633") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 660102, "cityName": "Redwood City" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740909553913856, "text": "He still loves me though.. @JustinKinnebrew ♥️♥️♥️ https://t.co/O8pvEOwVUi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1194930162 }}, "user": { "id": 838694594, "name": "Kaylee Juarez♡∞", "screen_name": "kaylee_juarez", "lang": "en", "location": "Safford, AZ", "create_at": date("2012-09-21"), "description": "In love with my bestfriend, jυѕтιnĸιnneвrew.♡ @justinkinnebrew & ιm a cнeerleader @ sнs", "followers_count": 762, "friends_count": 626, "statues_count": 6720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Safford, AZ", "id": "01390ea224ca3039", "name": "Safford", "place_type": "city", "bounding_box": rectangle("-109.742004,32.798772 -109.647652,32.839199") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4009, "countyName": "Graham", "cityID": 462000, "cityName": "Safford" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740909625188353, "text": "Literally so bored sos", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760357819, "name": "Shannon Coleman", "screen_name": "SNicole37", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "UH College of Technology", "followers_count": 571, "friends_count": 437, "statues_count": 11388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Katy, TX", "id": "c484d6c5cd72beb0", "name": "Katy", "place_type": "city", "bounding_box": rectangle("-95.87417,29.736773 -95.774936,29.831219") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4838476, "cityName": "Katy" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740909813932032, "text": "When you're stuck on night shift you learn new songs. (Pardon the hum of the satellite terminal… https://t.co/yRP7gJ1CIs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.8808,34.9008"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1107080472, "name": "Daniel Clark", "screen_name": "daniel10261", "lang": "en", "location": "Havelock, NC", "create_at": date("2013-01-20"), "description": "NC livin", "followers_count": 205, "friends_count": 278, "statues_count": 4171 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Havelock, NC", "id": "de4ce211af5cd6cf", "name": "Havelock", "place_type": "city", "bounding_box": rectangle("-76.948247,34.85557 -76.853819,34.952004") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37049, "countyName": "Craven", "cityID": 3730120, "cityName": "Havelock" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740909881069570, "text": "@sammywammy1020 HAPPY BIRTHDAY I LOVE YOU BEST FRIEND ❤️❤️❗️", "in_reply_to_status": -1, "in_reply_to_user": 1475350722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1475350722 }}, "user": { "id": 1188541075, "name": "taša", "screen_name": "natasaaxox", "lang": "en", "location": "null", "create_at": date("2013-02-16"), "description": "SAMANTHA IS MY BFF", "followers_count": 313, "friends_count": 296, "statues_count": 2452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910363508736, "text": "#escorttoledo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "escorttoledo" }}, "user": { "id": 299133542, "name": "Nick Cook", "screen_name": "SpazticCook", "lang": "en", "location": "Sylvania, OH", "create_at": date("2011-05-15"), "description": "null", "followers_count": 101, "friends_count": 447, "statues_count": 2654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sylvania, OH", "id": "01a11db77ef99e2a", "name": "Sylvania", "place_type": "city", "bounding_box": rectangle("-83.77959,41.658626 -83.644903,41.727463") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3976022, "cityName": "Sylvania" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910447300608, "text": "With The Little Mermaid - who was kind enough to take time for a picture! #ballet #dance #family… https://t.co/rVKtsxzgsf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-78.9029599,35.99779"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ballet", "dance", "family" }}, "user": { "id": 52136585, "name": "Jen Busfield", "screen_name": "jabsplethora", "lang": "en", "location": "ÜT: 39.905109,-77.651309", "create_at": date("2009-06-29"), "description": "Passionate Mom (son w Autism). Author. Traveler. Blogger. Adventurer. Lover. Foodie. Geek. Latina. Picture taker. Silly dancer. Music lover. Tweets are my own.", "followers_count": 1568, "friends_count": 1995, "statues_count": 16536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910589894656, "text": "Ok *shrugs*", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 412625447, "name": "T.Burrell™", "screen_name": "DoItLikeTONY_", "lang": "en", "location": "303✈️310", "create_at": date("2011-11-14"), "description": "TRaww® IG: TonyThaTYGER SC: TRaww_Burrell #DenverCountry #OKCThunder #BigThree #HOUNDLIFE", "followers_count": 1271, "friends_count": 474, "statues_count": 53105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910682144769, "text": "@celeste_juarez @__natalyparra help me pls https://t.co/QG8vatmupZ", "in_reply_to_status": -1, "in_reply_to_user": 2149604101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2149604101, 2468750227 }}, "user": { "id": 2174984898, "name": "JAYYTEA$E", "screen_name": "jocelynrivas06", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "keep ya head up", "followers_count": 331, "friends_count": 463, "statues_count": 8479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910740869120, "text": "faithful faithful faithful faithfulllllll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3060692334, "name": "jamie sheldon", "screen_name": "jamiekunane_", "lang": "en", "location": "arizona state university ", "create_at": date("2015-03-03"), "description": "rock bottom became a ѕσℓι∂ foundation on which I rebuilt my ⓛⓘⓕⓔ ➳ MD", "followers_count": 286, "friends_count": 191, "statues_count": 3522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910904475649, "text": "So you could dance for me �� https://t.co/6ELVLk7ffD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3907504455, "name": "DALIE", "screen_name": "adaliea_", "lang": "en", "location": "null", "create_at": date("2015-10-09"), "description": "The Road To Heaven Is A Hell Of A Ride. Ennis, Tx| ig: @adalieavila| sc: @adaliebigpimpin| 19|", "followers_count": 258, "friends_count": 135, "statues_count": 3305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ennis, TX", "id": "054df5106ee9caf2", "name": "Ennis", "place_type": "city", "bounding_box": rectangle("-96.667447,32.289048 -96.571268,32.368525") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4824348, "cityName": "Ennis" } }
+{ "create_at": datetime("2016-05-15T00:00:02.000Z"), "id": 731740910967521280, "text": "So tired", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1927027207, "name": "Megs", "screen_name": "Megs_Emilyyy16", "lang": "en", "location": "CT/FL", "create_at": date("2013-10-02"), "description": "if you can't find a hero, become one", "followers_count": 338, "friends_count": 392, "statues_count": 13046 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zephyrhills, FL", "id": "0160ed302dcb7707", "name": "Zephyrhills", "place_type": "city", "bounding_box": rectangle("-82.246926,28.172468 -82.131803,28.291625") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1279225, "cityName": "Zephyrhills" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740911969783808, "text": "@__sheashea_ I did & chose 2 skins and then it said the coupon had expired ��", "in_reply_to_status": 731738118877904896, "in_reply_to_user": 631753032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 631753032 }}, "user": { "id": 2195365758, "name": "k e l l y", "screen_name": "kellyxroman", "lang": "en", "location": "probably at disneyland", "create_at": date("2013-11-14"), "description": "Living with a camera in one hand, coffee in the other and couldn't be happier ❤️", "followers_count": 233, "friends_count": 582, "statues_count": 7615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740912062103552, "text": "Listen to Ruben Blades - Greatest Hits by Rubén Blades on @AppleMusic. https://t.co/O1Rtl7Cs6T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 74580436 }}, "user": { "id": 2334052321, "name": "Eduardo Stewart", "screen_name": "edking507", "lang": "es", "location": "null", "create_at": date("2014-02-08"), "description": "null", "followers_count": 91, "friends_count": 437, "statues_count": 74 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740912343252993, "text": "This is a forever thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1611972548, "name": "J-Eazy", "screen_name": "heyitsjillian", "lang": "en", "location": "probably with shelby lol", "create_at": date("2013-07-21"), "description": "good music, big dreams & JD || Class President", "followers_count": 416, "friends_count": 247, "statues_count": 17210 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Land O' Lakes, FL", "id": "36f6e625182fae8b", "name": "Land O' Lakes", "place_type": "city", "bounding_box": rectangle("-82.524994,28.171069 -82.369327,28.267173") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1239200, "cityName": "Land O' Lakes" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740912980631552, "text": "@BennyBoygr A beach? Would I like it.? That's going on a bucket list. Licks Misty #Furends", "in_reply_to_status": 731581560995102720, "in_reply_to_user": 615190328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Furends" }}, "user_mentions": {{ 615190328 }}, "user": { "id": 2429984160, "name": "Misty K9 Mountfort", "screen_name": "vegasrue", "lang": "en", "location": "null", "create_at": date("2014-04-05"), "description": "Just a little laid back girl with a big bark, baggy skin & wrinkles. Iz live in Las Vegas, NV", "followers_count": 566, "friends_count": 345, "statues_count": 11759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740912984825856, "text": "https://t.co/TpuptmfJid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 4490004978, "name": "Vannah Paige", "screen_name": "vannahpaige252", "lang": "en", "location": "Wisconsin, USA", "create_at": date("2015-12-07"), "description": "My best friend is my camera and I'm a model/actress + I'm obsessed with the Internet and makeup", "followers_count": 216, "friends_count": 365, "statues_count": 2417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eau Claire, WI", "id": "6f58cacd3741baa6", "name": "Eau Claire", "place_type": "city", "bounding_box": rectangle("-91.601375,44.759861 -91.418063,44.879183") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55035, "countyName": "Eau Claire", "cityID": 5522300, "cityName": "Eau Claire" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913198845953, "text": "I’m washed up.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.93424927,43.10557793"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251240858, "name": "Reem", "screen_name": "chillwithreem", "lang": "en", "location": "when am I really even here?", "create_at": date("2011-02-12"), "description": "you don't worry 'bout fitting in when you custom made", "followers_count": 688, "friends_count": 370, "statues_count": 34085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913387458563, "text": "About to download like 20 Skyrim mods and then play the night and morning away lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 903029726, "name": "Hason", "screen_name": "GuatemalanJason", "lang": "en", "location": "null", "create_at": date("2012-10-24"), "description": "just trying to make others happy", "followers_count": 502, "friends_count": 356, "statues_count": 9944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913521852417, "text": "Wind 0.1 mph WNW. Barometer 30.040 in, Steady. Temperature 63.7 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913639292929, "text": "�� https://t.co/knyHaEk8ZB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3008220388, "name": "Larry Miller", "screen_name": "iAmBOjr", "lang": "en", "location": "minding mines", "create_at": date("2015-01-31"), "description": "....... my own lane", "followers_count": 266, "friends_count": 312, "statues_count": 2017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913660203009, "text": "1 Red-headed Woodpecker (Melanerpes erythrocephalus) - stakeout Red-headed Woodpecker, Cave Canyon (2016) - 2016-05-14 17:29", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.7898235,31.7163729"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15158 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913731428353, "text": "3. Having to deal with someone yelling at your face trying to keep your patience because the cook is taking to long w the order ��", "in_reply_to_status": 731740591659225088, "in_reply_to_user": 2261199506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2261199506, "name": "anjel", "screen_name": "angelrockets", "lang": "en", "location": "null", "create_at": date("2013-12-25"), "description": "null", "followers_count": 515, "friends_count": 289, "statues_count": 21521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740913962221568, "text": "Wind 0.0 mph W. Barometer 29.66 in, Falling slowly. Temperature 45.1 °F. Rain today 0.04 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740914075340800, "text": "Wind 0.0 mph N. Barometer 30.083 in, Falling slowly. Temperature 59.5 °F. Rain today 0.00 in. Humidity 66%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740914247294980, "text": "The Burping pig trailer with @joeylawrence go check it out!! https://t.co/EW0GVgXi1U https://t.co/W7J2ec8dKy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 145852539 }}, "user": { "id": 2948382268, "name": "JoeysAngel4Ever", "screen_name": "albritton_leisa", "lang": "en", "location": "null", "create_at": date("2014-12-28"), "description": "Mother to Courtney 20 &Joe13. Long time fan of JoeyLawrence since Iwas14,for3 decades!#JoeysAngel4ever.Also a big fan of Andy and Matt#Support Always#still3", "followers_count": 772, "friends_count": 1054, "statues_count": 32372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denham Springs, LA", "id": "56f397b1076d7ce2", "name": "Denham Springs", "place_type": "city", "bounding_box": rectangle("-90.991153,30.356008 -90.838776,30.571121") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22063, "countyName": "Livingston", "cityID": 2220435, "cityName": "Denham Springs" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740914519932929, "text": "Don't act like you ain't just clock in https://t.co/DPcmpxe3cS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318098123, "name": "Irish Trill", "screen_name": "swartmayne", "lang": "en", "location": "HTX", "create_at": date("2011-06-15"), "description": "Apply yourself to supply your wealth. Only limits you have are those you place upon yourself. Saying that you can't just solidified that you never will. #LF", "followers_count": 1018, "friends_count": 591, "statues_count": 139555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-15T00:00:03.000Z"), "id": 731740915140698114, "text": "@KANTI_SINPAI sorry not sorry ��", "in_reply_to_status": 731740437015224320, "in_reply_to_user": 2689313557, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2689313557 }}, "user": { "id": 400068664, "name": "sam", "screen_name": "samanthaabbie", "lang": "en", "location": "Orland, CA", "create_at": date("2011-10-28"), "description": "life is great • gianni ❤️•", "followers_count": 283, "friends_count": 305, "statues_count": 12941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orland, CA", "id": "670fb5b9ac1f023e", "name": "Orland", "place_type": "city", "bounding_box": rectangle("-122.238913,39.721575 -122.150193,39.769545") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6021, "countyName": "Glenn", "cityID": 654274, "cityName": "Orland" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740915891503105, "text": "Need to vent", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3032265067, "name": "petty evy", "screen_name": "_evydollll", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-02-19"), "description": "most likely watching a makeup tutorial rn", "followers_count": 115, "friends_count": 94, "statues_count": 2271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740916126359557, "text": "���� https://t.co/oxOZklF5A3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 196276297, "name": "Rick Daniels", "screen_name": "OnAirWithRick", "lang": "en", "location": "Boise, ID", "create_at": date("2010-09-28"), "description": "Afternoon Drive Host @NashFM979 | @CumulusMedia #Boise | @VincennesU Alum | Proud #Hoosier! | God. Country. @NDFootball @Colts @IndianaMBB @Pacers", "followers_count": 6380, "friends_count": 569, "statues_count": 60515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, ID", "id": "afc5c9ce888b958d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-116.461472,43.561187 -116.337922,43.662958") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1652120, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740916470427648, "text": "Wind 0.5 mph NW. Barometer 29.539 in, Steady. Temperature 43.8 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740916805828609, "text": "Ayeeeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 760700304, "name": "andrea.", "screen_name": "niallsnandos13_", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "make tacos, not war", "followers_count": 3136, "friends_count": 2969, "statues_count": 41448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740916935839744, "text": "LAST NIGHT I CAME TO A REALIZATION", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1939349054, "name": "Matty McKibben", "screen_name": "mathayou", "lang": "en", "location": "null", "create_at": date("2013-10-05"), "description": "walking on a dream", "followers_count": 418, "friends_count": 199, "statues_count": 17254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740917258850304, "text": "The last shadow puppets are life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 55981292, "name": "Arianna", "screen_name": "Aireannuh", "lang": "en", "location": "California", "create_at": date("2009-07-11"), "description": "Dracula teeth", "followers_count": 575, "friends_count": 1226, "statues_count": 64756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740917292376064, "text": "@JawMesss still ok \"chance jean jacket jacker jacket dfagget fagger\" or sumtin like that lol", "in_reply_to_status": 731740773205467136, "in_reply_to_user": 2281129549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2281129549 }}, "user": { "id": 1449838722, "name": "SurferBoi ", "screen_name": "suh_bagel", "lang": "en", "location": "The Trap", "create_at": date("2013-05-22"), "description": "i tell shitty jokes", "followers_count": 415, "friends_count": 187, "statues_count": 33808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740917854429185, "text": "Funny how life works ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1967817398, "name": "Sarah Mares", "screen_name": "sizzzlemizzzle", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-10-17"), "description": "HCHS '16 // St.MU '20", "followers_count": 239, "friends_count": 224, "statues_count": 3980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leon Valley, TX", "id": "b29b5683c45df9d1", "name": "Leon Valley", "place_type": "city", "bounding_box": rectangle("-98.632532,29.476726 -98.586098,29.515672") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4842388, "cityName": "Leon Valley" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740918072512513, "text": "Go without expectation so your feelings don't get hurt, or hopes get crushed ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711732932, "name": "b l a n k", "screen_name": "JoyJoy_Champ9", "lang": "en", "location": "tacoma", "create_at": date("2012-07-22"), "description": "UO '19 | agape love | blessed with the best ❤️|", "followers_count": 936, "friends_count": 707, "statues_count": 19998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740918466936837, "text": "Great ol workforce tomorrow then work later that night ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3686175853, "name": "Zach", "screen_name": "ZachAndronic", "lang": "en", "location": "null", "create_at": date("2015-09-25"), "description": "#YoungEntrepreneur #FutureBusinessOwner #DieselMechanic #Leo 18 Growing up trying to find my way not letting anything hold me back #RipTyler #FreeCody", "followers_count": 748, "friends_count": 924, "statues_count": 21288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen Park, MI", "id": "0c937cb917334546", "name": "Allen Park", "place_type": "city", "bounding_box": rectangle("-83.231897,42.222765 -83.183819,42.301134") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2601380, "cityName": "Allen Park" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740918777143297, "text": "Not my photo.\n\"This is THEIR democracy\" From the photographer. #StillSanders #nvdemsconvention https://t.co/wRZd2I9dwI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StillSanders", "nvdemsconvention" }}, "user": { "id": 1950862849, "name": "Douglas A. Maske", "screen_name": "maskeda21", "lang": "en", "location": "Puyallup, WA", "create_at": date("2013-10-09"), "description": "Official Grassroots Social Media Team Member for the Bernie Sanders Campaign #FeelTheBern #StillSanders #TeamSecondWave An Activist, Cloud Engineer & Musician", "followers_count": 235, "friends_count": 761, "statues_count": 1382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740919192424452, "text": "I'm having a hard time sleeping. My cigarette intake has gone down 5 cigarettes a day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3759524292, "name": "Chad Benson", "screen_name": "toastietootsies", "lang": "en", "location": "Washington, USA", "create_at": date("2015-10-02"), "description": "i am good for a smile. I like tattoos, Seattle, TM, friendly people and Android.", "followers_count": 4, "friends_count": 7, "statues_count": 35 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740919314055168, "text": "@ShinKurois MAKI (?) MAKES IT SO MUCH BETTER", "in_reply_to_status": 731740778196664321, "in_reply_to_user": 3425283766, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3425283766 }}, "user": { "id": 990116622, "name": "nut sipper", "screen_name": "kinksamer", "lang": "en", "location": "Fontana CA", "create_at": date("2012-12-04"), "description": "Hey there, my name's Alex! // He/They //\nFFXIV: Arca Jinoir on Behemoth // Weeping Rose on Malboro //\nIcon by @radcanine", "followers_count": 666, "friends_count": 958, "statues_count": 68198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740919485980672, "text": "������ https://t.co/MTyCZiQK3e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2891545502, "name": "$teffie", "screen_name": "steffanna_", "lang": "en", "location": "954 / 407", "create_at": date("2014-11-05"), "description": "Trinidadian", "followers_count": 1435, "friends_count": 616, "statues_count": 12976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Lauderdale, FL", "id": "39f9889e234aad3f", "name": "North Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.244338,26.195412 -80.195484,26.230126") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1249425, "cityName": "North Lauderdale" } }
+{ "create_at": datetime("2016-05-15T00:00:04.000Z"), "id": 731740919649603584, "text": "@JustinHawkins I will be a fan as long as we all live, even though I'm still old enough to be your mum. I love The Darkness!", "in_reply_to_status": -1, "in_reply_to_user": 19458172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19458172 }}, "user": { "id": 730593401276649473, "name": "Tracy Williamson", "screen_name": "TracyWi82282415", "lang": "en", "location": "Independence, Missouri, USA", "create_at": date("2016-05-11"), "description": "I am 52 years old, disabled physically but not mentally. I love reading the classics, movies, and rock and roll. My favorite band is THE DARKNESS!", "followers_count": 10, "friends_count": 76, "statues_count": 12 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, MO", "id": "04b4aca917b0103d", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-94.487114,39.01759 -94.269551,39.158419") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2935000, "cityName": "Independence" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920090071041, "text": "Ty-Yer just left me & I'm sad ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422328315, "name": "T O O D L E", "screen_name": "Tiyerra_Sabre", "lang": "en", "location": "L I V I N G.", "create_at": date("2011-11-26"), "description": "you guilty of it", "followers_count": 1714, "friends_count": 838, "statues_count": 55414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seaford, DE", "id": "b719350492e3ff2f", "name": "Seaford", "place_type": "city", "bounding_box": rectangle("-75.643721,38.6301 -75.577647,38.684697") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10005, "countyName": "Sussex", "cityID": 1064320, "cityName": "Seaford" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920178057216, "text": "happy birthday & 4 months to my other half. thanks for everything, love you. @Lil_L_Fizzle https://t.co/XVZ5GZBfXg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1312412485 }}, "user": { "id": 2936792102, "name": "tori", "screen_name": "torrriboooo", "lang": "en", "location": "don't trip", "create_at": date("2014-12-19"), "description": "@Lil_L_Fizzle", "followers_count": 558, "friends_count": 378, "statues_count": 12387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920261943301, "text": "@JaykeBird I reccomend it :D", "in_reply_to_status": 731740218903035906, "in_reply_to_user": 304662576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304662576 }}, "user": { "id": 3364566986, "name": "Nara Strex", "screen_name": "ClairvoyantNara", "lang": "en", "location": "null", "create_at": date("2015-08-27"), "description": "Lover of Videogames, Manga, and Webcomics! GG supporter and 2nd wave feminist.", "followers_count": 56, "friends_count": 310, "statues_count": 1504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakville, MO", "id": "46d9a4d89c4e03e8", "name": "Oakville", "place_type": "city", "bounding_box": rectangle("-90.37658,38.411395 -90.281839,38.492038") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2953876, "cityName": "Oakville" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920308240384, "text": "@MariaHarfouche @hayrachel1", "in_reply_to_status": 731565873543045120, "in_reply_to_user": 193127375, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 193127375, 4316769913 }}, "user": { "id": 289604260, "name": "Sophie ♡", "screen_name": "girlonthecover", "lang": "en", "location": "London✈️Orange County", "create_at": date("2011-04-28"), "description": "probably vlogging somewhere on the streets of orange county. Chapman '19 • kappa kappa gamma •", "followers_count": 1158, "friends_count": 1025, "statues_count": 25427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920723300357, "text": "Next time I come to Las Vegas I'm gonna be wearing my $2,000 suit and my $40,000 gold fucking watch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 357359521, "name": "Abubakar", "screen_name": "Abubahcar", "lang": "en", "location": "null", "create_at": date("2011-08-18"), "description": "#HalaMadrid how's business? business is booming. I like that.", "followers_count": 648, "friends_count": 265, "statues_count": 12645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740920773742592, "text": "NEW POST: Summer Nail Polish Colour Choices https://t.co/WxeiLXQHe8 #fbloggers #bbloggers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fbloggers", "bbloggers" }}, "user": { "id": 257921418, "name": "Life of Ellie Grace", "screen_name": "elliegdickinson", "lang": "en", "location": "Manchester", "create_at": date("2011-02-26"), "description": "I spend my free time baking and blogging. Style, beauty and lifestyle blogger. UK Blog Award finalist. lifeofellieg@gmail.com | http://youtube.com/elliechatters", "followers_count": 2198, "friends_count": 486, "statues_count": 16887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740921100820484, "text": "Yeah, just went there today and it was amazing! https://t.co/MOF3C9AeaW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4169606119, "name": "Gabriel Garcia", "screen_name": "Gabe_The_Runner", "lang": "en", "location": "null", "create_at": date("2015-11-08"), "description": "null", "followers_count": 779, "friends_count": 903, "statues_count": 1324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740921146957824, "text": "Temp: 32.6°F - Dew Point: 27.7° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.02in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 12, "statues_count": 17693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-15T00:00:05.000Z"), "id": 731740921365155840, "text": "Budget Beauty: Brow mascaras under P400 https://t.co/XNlfw125U9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.00093296,40.7206258"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1382621, "name": "Liz Lanuzo", "screen_name": "lizlanuzo", "lang": "en", "location": "Philippines", "create_at": date("2007-03-17"), "description": "Beauty blogger. Brand consultant.", "followers_count": 3385, "friends_count": 392, "statues_count": 20051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103286627520512, "text": "I'm out ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356011320, "name": "'d ✨", "screen_name": "NaiiiNaiii", "lang": "en", "location": "nac, tx", "create_at": date("2011-08-15"), "description": "19. longlivedex", "followers_count": 2341, "friends_count": 1515, "statues_count": 46618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lufkin, TX", "id": "ef5f7bc53f4594ea", "name": "Lufkin", "place_type": "city", "bounding_box": rectangle("-94.796056,31.264964 -94.669141,31.400021") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48005, "countyName": "Angelina", "cityID": 4845072, "cityName": "Lufkin" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103286854156288, "text": "03:00:02 |Temp: 49.8ºF |Dew Point 44.2ºF | Rain today: 0.00 inches | Wind: 0.0 mph from the SSW, Gusting to 0.0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.07083333,33.03916667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157304422, "name": "KD4NGC WX Station", "screen_name": "KD4NGCWX", "lang": "en", "location": "Milledgeville,GA", "create_at": date("2010-06-19"), "description": "kd4ngc's tweeting weather station/Davis Vantage Pro II", "followers_count": 94, "friends_count": 21, "statues_count": 94213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13303, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103287508328448, "text": "Wind 1.0 mph S. Barometer 1022.25 mb, Steady. Temperature 55.5 °F. Rain today 0.02 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103287747563520, "text": "I'm at Greenblatt's Delicatessen & Fine Wine Shop in Los Angeles, CA https://t.co/G0o60GJqUD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.36465375,34.09812979"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24140150, "name": "Big Tone #WCTF", "screen_name": "DaRealBigTone", "lang": "en", "location": "DecaturMiamiAuburn Pacoima,Ca", "create_at": date("2009-03-12"), "description": "Big Tone: I am for your protection, i mean what i say, i say what i mean. hell im better protection than a condom: West Coast TopFloor", "followers_count": 584, "friends_count": 492, "statues_count": 5846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103289018458113, "text": "#ijm just sweet little IJM moment with sweet friends kathywhittakerphotography #missdallas… https://t.co/34ypOIJzNg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.07512518,29.55628898"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ijm", "missdallas" }}, "user": { "id": 1304648892, "name": "Susan Wells", "screen_name": "swells1015", "lang": "en", "location": "null", "create_at": date("2013-03-26"), "description": "A photographer, a texan and a Mom\r\nGod makes all things possible", "followers_count": 76, "friends_count": 215, "statues_count": 1126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nassau Bay, TX", "id": "25f6d96ba3246224", "name": "Nassau Bay", "place_type": "city", "bounding_box": rectangle("-95.104473,29.531131 -95.073464,29.558251") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4850376, "cityName": "Nassau Bay" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103289118941184, "text": "A\nHey Dalton...hope you had a great day mooooving in https://t.co/RLJaEJ6UDZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251465089, "name": "marypatb", "screen_name": "MaryPat822", "lang": "en", "location": "in the sunshine", "create_at": date("2011-02-12"), "description": "a so cal gal with a heart for the arts!", "followers_count": 525, "friends_count": 573, "statues_count": 6207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103289253158913, "text": "@gilbertthesloth then why go to sleep upset ��", "in_reply_to_status": 732101301933211648, "in_reply_to_user": 839022720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 839022720 }}, "user": { "id": 2238326208, "name": "phillip mucupp", "screen_name": "_Rawgers", "lang": "en", "location": "Los Angeles County", "create_at": date("2013-12-09"), "description": "Me Ima hustla. Just so happens i know how to Rap // click link Faded All Night Clique", "followers_count": 310, "friends_count": 187, "statues_count": 5008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103289618079744, "text": "@InkBun Ah that would look amazing gurl.", "in_reply_to_status": 732094135205830656, "in_reply_to_user": 4874319154, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4874319154 }}, "user": { "id": 708710773304856576, "name": "Lumi.", "screen_name": "LumiChuu", "lang": "en", "location": "null", "create_at": date("2016-03-12"), "description": "♌️【Lumi☆】【PP By @MochiiChun】【Lil' Hunny~ @MochiiChun 4/9/2016】", "followers_count": 175, "friends_count": 73, "statues_count": 11 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103289970429952, "text": "When I was a sophomore @AyeeYoAshleyy gave me a secret admire rose for Valentine Day �� that's probably it. Lol https://t.co/uDNm59HtbL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 327215833 }}, "user": { "id": 624324039, "name": "Beast Incarnate ⚡️", "screen_name": "_gameover22", "lang": "en", "location": "WrestleMania", "create_at": date("2012-07-01"), "description": "Hit Hard, Hit Often", "followers_count": 1021, "friends_count": 505, "statues_count": 39180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mexia, TX", "id": "041a08c2d0c3f01f", "name": "Mexia", "place_type": "city", "bounding_box": rectangle("-96.504204,31.653047 -96.453416,31.712644") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48293, "countyName": "Limestone", "cityID": 4847916, "cityName": "Mexia" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290003955712, "text": "@sncerelyaly tomorrow aint pizza monday though ����", "in_reply_to_status": 732103129194369024, "in_reply_to_user": 2390324832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2390324832 }}, "user": { "id": 353819243, "name": "Legendaddy", "screen_name": "MaleekW84", "lang": "en", "location": "null", "create_at": date("2011-08-12"), "description": "Lose the hero, Get with a hero", "followers_count": 634, "friends_count": 611, "statues_count": 15253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seguin, TX", "id": "d8353f131bceb54e", "name": "Seguin", "place_type": "city", "bounding_box": rectangle("-98.025037,29.530994 -97.922521,29.61778") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48187, "countyName": "Guadalupe", "cityID": 4866644, "cityName": "Seguin" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290071089152, "text": "happy birthday sucka @alessio_niemi LYM, be safe!!! ��❤️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2923279939 }}, "user": { "id": 2956618573, "name": "abby", "screen_name": "abbygamboa_", "lang": "en", "location": "Bay Area, CA", "create_at": date("2015-01-02"), "description": "horchata, paletas & chill? sc: abbygamboa", "followers_count": 658, "friends_count": 415, "statues_count": 15112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290079436800, "text": "Letting go hurts like a motherfucker but it's better than holding on to something that isn't real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 742547491, "name": "Jack Jack ♡", "screen_name": "JackieSaavedra_", "lang": "en", "location": "hb, california", "create_at": date("2012-08-07"), "description": "so what can you show me that my heart don't know already?", "followers_count": 1806, "friends_count": 802, "statues_count": 4980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290498879488, "text": "Happy Birthday Matt, Let this be your day bro god. Let's finish this high school journey together. Much love ���� @ohmattyg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2568346500 }}, "user": { "id": 4535537892, "name": "Jordy", "screen_name": "_jcxv", "lang": "en", "location": "null", "create_at": date("2015-12-11"), "description": "It's almost time", "followers_count": 216, "friends_count": 117, "statues_count": 175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.62655,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290746339328, "text": "If You wanna B Talked Out Of Doing Something Crazy Don't Call Me , I'm With The Crazy Shits", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44538312, "name": "antisocial. ✨", "screen_name": "_TheyHateNai", "lang": "en", "location": "Eating Pizza W/@msbrittneyfaye", "create_at": date("2009-06-03"), "description": "Focus On You, Until The Focus Is On You . A Fine Ass Problem.", "followers_count": 2313, "friends_count": 1842, "statues_count": 100853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-16T00:00:00.000Z"), "id": 732103290767319040, "text": "So yo Bitch comes on to me and you mad at me ������ you still with her tho ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3059128507, "name": "Junior", "screen_name": "JRtoobig34", "lang": "en", "location": "null", "create_at": date("2015-03-02"), "description": "the gym is the only love in my life #gainz you'll always be the one for me ❤️", "followers_count": 309, "friends_count": 522, "statues_count": 7581 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103290826067968, "text": "Basically past 40+ and Ima need you to switch to another app, pull up or FaceTime me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2405305549, "name": "McNeece Egbim", "screen_name": "McNeeceEgbim", "lang": "en", "location": "Triple D, TX✈️SpringField, MO", "create_at": date("2014-03-22"), "description": "LineBacker at Missouri State University | Rest in Paradise 3 .", "followers_count": 753, "friends_count": 406, "statues_count": 4342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103290935214080, "text": "Both metaphorical and metaphysical garlic bread. Fuck I want bread tho. Fresh baked and hot and great omg. I wanna rip it. And eat it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287477902, "name": "DropoutBear", "screen_name": "owlattack215", "lang": "en", "location": "null", "create_at": date("2011-04-24"), "description": "Nine to five is how you survive, I ain't trying to survive I'm tryna live it to the limit and love it a lot Life ills poisoned my body ΚΑΤΑ ΤΟΝ ΔΑΙΜΟΝΑ ΕΑΥΤΟΥ;", "followers_count": 259, "friends_count": 618, "statues_count": 21844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103291446820864, "text": "@Bonaparte_33 Right! All I'm excited for are those Senior Ditch Extra Credit points����", "in_reply_to_status": 732102880048513024, "in_reply_to_user": 328208054, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 328208054 }}, "user": { "id": 3325475672, "name": "Pepa Malieitulua", "screen_name": "lepepaina", "lang": "en", "location": "San Leandro, CA", "create_at": date("2015-08-22"), "description": "@saiat8900", "followers_count": 170, "friends_count": 168, "statues_count": 2772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Leandro, CA", "id": "61f1d75eb5064808", "name": "San Leandro", "place_type": "city", "bounding_box": rectangle("-122.202424,37.667637 -122.122164,37.74245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668084, "cityName": "San Leandro" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103291652317184, "text": "@ebbtideapp Tide in Toms Harbor Cut, Florida 05/16/2016\nHigh 6:05am 0.5\n Low 12:49pm 0.1\nHigh 6:29pm 0.5\n Low 12:47am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-80.9067,24.7833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 33172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12087, "countyName": "Monroe" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103292642168832, "text": "Who else got beef?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 543783121, "name": "shaymaney.", "screen_name": "shayfunktastic", "lang": "en", "location": "null", "create_at": date("2012-04-02"), "description": ":.", "followers_count": 230, "friends_count": 227, "statues_count": 28168 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103292872953856, "text": "We gon have a juke jam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1626126536, "name": "Miriam ✭", "screen_name": "mir_salg06", "lang": "en", "location": "null", "create_at": date("2013-07-27"), "description": "I plan to eat with the people I starved with.", "followers_count": 174, "friends_count": 162, "statues_count": 9247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grandview, WA", "id": "36360005eeb536a8", "name": "Grandview", "place_type": "city", "bounding_box": rectangle("-119.942937,46.235098 -119.873337,46.281903") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53077, "countyName": "Yakima", "cityID": 5327925, "cityName": "Grandview" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103293044850688, "text": "Weird way to end the night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 426237110, "name": "⚡️Kodack White⚡️", "screen_name": "1stLeg_rookie", "lang": "en", "location": "Dallas Tx", "create_at": date("2011-12-01"), "description": "|Nigerian||Entrepreneur| #PVAMU19 Track and Field L.I.T ~ Life In Threads check out the website below", "followers_count": 1696, "friends_count": 1415, "statues_count": 21269 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103293791408129, "text": "Talk it out go back lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 382992988, "name": "October's Very Own", "screen_name": "JR7JR7JR7", "lang": "en", "location": "Stockton, CA", "create_at": date("2011-09-30"), "description": "OVO / A.M / I.N / @babyface_brii ❤️", "followers_count": 341, "friends_count": 292, "statues_count": 14385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103293845921792, "text": "seriously need new overalls i miss'em", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1535183017, "name": "ren", "screen_name": "renfromdynasty", "lang": "en", "location": "null", "create_at": date("2013-06-20"), "description": "mucho", "followers_count": 244, "friends_count": 190, "statues_count": 7151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103293846048768, "text": "Ripley SW Limestone Co. Temp: 54.0°F Wind:3.1mph Pressure: 1000.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103293904654337, "text": "If u don't think u gained weight at school try to put on your prom dress. Boa that shit was not tryna come up past my thigh/hip/ass area.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3050092929, "name": "DE$", "screen_name": "chirpphone", "lang": "en", "location": "low end", "create_at": date("2015-02-21"), "description": "de$tinee", "followers_count": 200, "friends_count": 118, "statues_count": 546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103294416392193, "text": "happy 54th birthday to my queen ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3033171409, "name": "angelique dahilig", "screen_name": "angeliquedahil1", "lang": "en", "location": "null", "create_at": date("2015-02-20"), "description": "22. blessed. kenneth montealegre prado ❤️", "followers_count": 333, "friends_count": 367, "statues_count": 2197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delano, CA", "id": "01954cab54887979", "name": "Delano", "place_type": "city", "bounding_box": rectangle("-119.329884,35.725136 -119.214031,35.790493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 618394, "cityName": "Delano" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103294550745088, "text": "05/16@03:00 - Temp 41.4F, WC 41.4F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.114in, Rising slowly. Rain 0.00in. Hum 71%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103294601056256, "text": "Wind 1.0 mph ESE. Barometer 30.184 in, Steady. Temperature 56.2 °F. Rain today 0.00 in. Humidity 56%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-16T00:00:01.000Z"), "id": 732103294902902785, "text": "Damn dude. 6 weeks��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1368014071, "name": "γαμώ u", "screen_name": "HarebearIII", "lang": "en", "location": "null", "create_at": date("2013-04-20"), "description": "σας μισώ όλους", "followers_count": 197, "friends_count": 169, "statues_count": 13663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295167135744, "text": "I'll be here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2407700196, "name": "J. Cas", "screen_name": "CasDH1", "lang": "en", "location": "Norwalk", "create_at": date("2014-03-23"), "description": "Music keeps the world sane//Producer for @TheDHFam #DH", "followers_count": 441, "friends_count": 311, "statues_count": 17942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295263748097, "text": "Temp 40.4° Hi/Lo 43.0/40.3 Rng 2.7° WC 40.4° Hmd 60% Rain 0.00\" Storm 0.00\" BAR 29.834 Rising DP 27.6° Wnd 2mph Dir SE Gst 15mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295368486912, "text": "@MrStark__ let's see how far your PM gets in convicting congress", "in_reply_to_status": 732102440795951105, "in_reply_to_user": 156637745, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 156637745 }}, "user": { "id": 44585741, "name": "lostinworld", "screen_name": "pradeepbehera", "lang": "en", "location": "null", "create_at": date("2009-06-04"), "description": "null", "followers_count": 15, "friends_count": 85, "statues_count": 951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Overland Park, KS", "id": "ef975175b1003544", "name": "Overland Park", "place_type": "city", "bounding_box": rectangle("-94.742494,38.81903 -94.607693,39.044068") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2053775, "cityName": "Overland Park" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295406219264, "text": "I haven't drank in a really long time and I feel pretty good about it. I just get asco thinking about it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176561260, "name": "Patty Mills", "screen_name": "mrplanman_", "lang": "en", "location": "Dreamville, TX. ", "create_at": date("2010-08-09"), "description": "We gon' be alright", "followers_count": 1121, "friends_count": 1758, "statues_count": 30103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295410569216, "text": "Temp 36.9°F Wind Chill 36.9°F RH 92% Wind 0.0 --- Gust 0.0 --- SLP 30.216 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103295624450050, "text": "Temp: 62.8°F Wind:0.1mph Pressure: 30.107hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64237 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296211619842, "text": "Wind 0.0 mph ---. Barometer 30.16 in, Steady. Temperature 60.4 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296274468865, "text": "If you're in a relationship you shouldn't have to feel like you need to go through someone's phone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3287412452, "name": "$$$", "screen_name": "xxjacindaaa", "lang": "en", "location": "Riverside, CA", "create_at": date("2015-07-22"), "description": "UCR || ig: xx.jacindaaa || snap: xo.jvm", "followers_count": 219, "friends_count": 211, "statues_count": 3818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296337448960, "text": "This was my comeback season back back in the day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 491844531, "name": "Brandon Rockwell", "screen_name": "TheRunninMan", "lang": "en", "location": "Philadelphia 1776", "create_at": date("2012-02-13"), "description": "Aspiring Historian. Politically correct. Professional college student at Alderson Broaddus University. Soccer enthusiast. Every day is another adventure.", "followers_count": 582, "friends_count": 180, "statues_count": 32141 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cumberland, MD", "id": "448a1775be49da49", "name": "Cumberland", "place_type": "city", "bounding_box": rectangle("-78.812801,39.604408 -78.703268,39.722191") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24001, "countyName": "Allegany", "cityID": 2421325, "cityName": "Cumberland" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296404590592, "text": "Make this go viral ������ https://t.co/xXLc1AMGxy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288480752, "name": "Ace♣️", "screen_name": "_A1Lopes", "lang": "en", "location": "null", "create_at": date("2011-04-26"), "description": "IG: _A1lopes | SC:Lopesdagod | HTX | Lu18 | GOMAB", "followers_count": 1314, "friends_count": 904, "statues_count": 33528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296484278272, "text": "67.3F (Feels: 67.3F) - Humidity: 88% - Wind: 4.5mph E - Gust: 6.0mph - Pressure: 1014.5mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 220, "friends_count": 18, "statues_count": 240365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103296689700865, "text": "@TommyScally fucking weird huh lol", "in_reply_to_status": 732101945465307137, "in_reply_to_user": 594003589, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 594003589 }}, "user": { "id": 102844161, "name": "stephanie", "screen_name": "thatsssteph", "lang": "en", "location": "null", "create_at": date("2010-01-07"), "description": "null", "followers_count": 477, "friends_count": 332, "statues_count": 18837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103297864081409, "text": "@Charmayparado happy birthday boo i love you ��", "in_reply_to_status": -1, "in_reply_to_user": 3024623496, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3024623496 }}, "user": { "id": 2718551305, "name": "Elle Figueroa", "screen_name": "Mariefigueroa21", "lang": "en", "location": "San Jose, CA", "create_at": date("2014-08-08"), "description": "Freshavacado", "followers_count": 211, "friends_count": 197, "statues_count": 1025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298044530688, "text": "literally why haven't I watched SOA before? Four episodes in and I'm hooked", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242251475, "name": "Løwki bb", "screen_name": "chronicyouthh", "lang": "en", "location": "Chicago", "create_at": date("2011-01-24"), "description": "Indulge. Yung Wavy.", "followers_count": 357, "friends_count": 667, "statues_count": 39950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298157821952, "text": "Wind 0.0 mph ---. Barometer 30.254 in, Rising slowly. Temperature 40.6 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298438717446, "text": "With or without a thong? https://t.co/EUj4rI0tQa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1266023012, "name": "3", "screen_name": "Marquis_buckets", "lang": "en", "location": "null", "create_at": date("2013-03-13"), "description": "18", "followers_count": 375, "friends_count": 343, "statues_count": 11025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298891710464, "text": "Im laying down & trying to relax but the other side of me keeps telling me I should get up and do my study guides��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1720734248, "name": "Jonalyn Manalo", "screen_name": "JonabonaManalo", "lang": "en", "location": "null", "create_at": date("2013-09-01"), "description": "null", "followers_count": 263, "friends_count": 296, "statues_count": 576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kailua, HI", "id": "7f0e166849448f1d", "name": "Kailua", "place_type": "city", "bounding_box": rectangle("-157.765901,21.352946 -157.705775,21.428447") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1523150, "cityName": "Kailua" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298900066304, "text": "My HBCU lit https://t.co/aaxSbxWGV9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2950695278, "name": "M.Minter☔️™", "screen_name": "micahminter_99", "lang": "en", "location": "Louisiana, USA", "create_at": date("2014-12-29"), "description": "#TxSU20 / Texas Southern Football / 318✈️ 713 /", "followers_count": 1017, "friends_count": 913, "statues_count": 16503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103298979876864, "text": "sooooo subtle n beautiful https://t.co/Yjn1XEO6BW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 351937316, "name": "eden", "screen_name": "edenbuckhardt", "lang": "en", "location": "ft wayne", "create_at": date("2011-08-09"), "description": "*eats 50 bananas in a day*", "followers_count": 1209, "friends_count": 146, "statues_count": 38105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Wayne, IN", "id": "3877d6c867447819", "name": "Fort Wayne", "place_type": "city", "bounding_box": rectangle("-85.336872,40.95926 -85.003231,41.214574") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18003, "countyName": "Allen", "cityID": 1825000, "cityName": "Fort Wayne" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103299139309568, "text": "Wind 0.0 mph ---. Barometer 30.038 in, Falling. Temperature 60.4 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-16T00:00:02.000Z"), "id": 732103299160244224, "text": "@slimjimdad congrats @250 followers ����", "in_reply_to_status": 732101667399864321, "in_reply_to_user": 4256149305, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4570678277, 5421402 }}, "user": { "id": 4256149305, "name": "Nancy Lasocki", "screen_name": "nancylasocki", "lang": "en", "location": "Illinois, USA", "create_at": date("2015-11-23"), "description": "Wife,Mom,Grandma & business owner/Pest Control. Simple things make me happy! Animal lover, Metal detecting,Casino lover", "followers_count": 916, "friends_count": 1421, "statues_count": 14676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channahon, IL", "id": "9ae1ad28e07154d9", "name": "Channahon", "place_type": "city", "bounding_box": rectangle("-88.25119,41.377462 -88.13539,41.492478") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1712476, "cityName": "Channahon" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103299361431552, "text": "Happy Birthday!!❤️❤️@lainiesophia https://t.co/3rG87AHalE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3850965499 }}, "user": { "id": 552943744, "name": "bradley", "screen_name": "Bradleyloisel", "lang": "en", "location": "null", "create_at": date("2012-04-13"), "description": "♕ ♕", "followers_count": 365, "friends_count": 323, "statues_count": 6657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103299445497856, "text": "@kkelseyprince literally the best day ever", "in_reply_to_status": 732103065017389056, "in_reply_to_user": 80433983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 80433983 }}, "user": { "id": 22287066, "name": "Alena Ingram", "screen_name": "TheAlenaIngram", "lang": "en", "location": "Rootstown-Kent State", "create_at": date("2009-02-28"), "description": "lib rat", "followers_count": 318, "friends_count": 548, "statues_count": 19086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ravenna, OH", "id": "294c6bf87c57eb22", "name": "Ravenna", "place_type": "city", "bounding_box": rectangle("-81.300183,41.093513 -81.192505,41.191153") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39133, "countyName": "Portage", "cityID": 3965592, "cityName": "Ravenna" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103299910926337, "text": "The Butterfly Effect: a movie that is making me ponder the consequences of me having not watched this terrible movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60500473, "name": "wearing those pants", "screen_name": "JakeWPlatt", "lang": "en", "location": "null", "create_at": date("2009-07-26"), "description": "NO REGRATS JUST LOVE\nArtist/Writer/Musician/Cat dad", "followers_count": 215, "friends_count": 215, "statues_count": 6420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pewaukee, WI", "id": "243f8ec5e6ee45b3", "name": "Pewaukee", "place_type": "city", "bounding_box": rectangle("-88.304392,43.026755 -88.185513,43.105253") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5562250, "cityName": "Pewaukee" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300066070530, "text": "This is trippy af.. https://t.co/GMrFvJpcDh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921682975, "name": "shawty", "screen_name": "stacy062599", "lang": "en", "location": "Fontana, CA", "create_at": date("2014-12-14"), "description": "♋️ Wildin", "followers_count": 161, "friends_count": 158, "statues_count": 2979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300066115585, "text": "@damndolly_ get outcha feelings cuzzo", "in_reply_to_status": 732070174669443073, "in_reply_to_user": 706286612, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 706286612 }}, "user": { "id": 266826393, "name": "Future_President", "screen_name": "Nunabug99", "lang": "en", "location": "Houston", "create_at": date("2011-03-15"), "description": "GOD Above all M.O.D./1989/Michael Ealy #take2", "followers_count": 362, "friends_count": 916, "statues_count": 2924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300225490944, "text": "World-leading Australian climate scientist reportedly sacked while at sea https://t.co/1kYsbUJirb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17671, "friends_count": 17820, "statues_count": 71909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300531642368, "text": "happy bday to the loml ���� so proud of the work you're doing rn.. supporting you 100% through it all @Jaxonoys https://t.co/3U61Xs2GCT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 318846498 }}, "user": { "id": 4482653533, "name": "celinaa.", "screen_name": "okokcelina", "lang": "en", "location": "null", "create_at": date("2015-12-06"), "description": "good vibes ...", "followers_count": 105, "friends_count": 93, "statues_count": 673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300561018880, "text": "my birthday is on fuckin memorial day AGAIN & i'm not gonna have shit to do yay :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2660760696, "name": "sunshine", "screen_name": "mvrisoul", "lang": "en", "location": "AAW ❤️ 050416", "create_at": date("2014-07-19"), "description": "poetic & dirty with a soul that could light the sky on fire", "followers_count": 1223, "friends_count": 1613, "statues_count": 16496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103300942716930, "text": "What's my favorite word?\nBITCH!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 868968402, "name": "Aaron Sanchez", "screen_name": "Aaron_Rozzay", "lang": "en", "location": "Baldwin Park, CA", "create_at": date("2012-10-08"), "description": "♑️", "followers_count": 793, "friends_count": 709, "statues_count": 19514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baldwin Park, CA", "id": "2008b1cea656f14b", "name": "Baldwin Park", "place_type": "city", "bounding_box": rectangle("-118.007533,34.054801 -117.942775,34.133201") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603666, "cityName": "Baldwin Park" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103301236301824, "text": "@Koji_Mudbone this might almost be true", "in_reply_to_status": 732103230000259073, "in_reply_to_user": 2412919146, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2412919146 }}, "user": { "id": 1187478062, "name": "Taraneh Tabatabai", "screen_name": "Amazing_Persian", "lang": "en", "location": "Central Perk", "create_at": date("2013-02-16"), "description": "You're only given a little spark of madness. You mustn't lose it.", "followers_count": 1016, "friends_count": 627, "statues_count": 51214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103301580230656, "text": "Only the peldians can save us", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3067321655, "name": "anthony", "screen_name": "FryingStove", "lang": "en", "location": "america, USA", "create_at": date("2015-03-02"), "description": "pops", "followers_count": 224, "friends_count": 129, "statues_count": 3715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103301618110464, "text": "Wind 0.0 mph NW. Barometer 29.91 in, Steady. Temperature 39.2 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103301630545920, "text": "@BongRips4Jesus_ & then throwing everything up after i binge eat((:", "in_reply_to_status": 732103152711786497, "in_reply_to_user": 335648065, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335648065 }}, "user": { "id": 632654992, "name": "spicyma", "screen_name": "shiku___", "lang": "en", "location": "null", "create_at": date("2012-07-10"), "description": "i'm who your bitch is trynna be.", "followers_count": 2783, "friends_count": 199, "statues_count": 83993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103301769003009, "text": "If you cry, I cry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 316792486, "name": "Diego", "screen_name": "lowkeyimD", "lang": "en", "location": "null", "create_at": date("2011-06-13"), "description": "TNB", "followers_count": 433, "friends_count": 392, "statues_count": 18181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lomita, CA", "id": "f2f180b090191151", "name": "Lomita", "place_type": "city", "bounding_box": rectangle("-118.328804,33.774937 -118.306568,33.807678") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 642468, "cityName": "Lomita" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103302163226627, "text": "@steph_chukwu lmaoooo that's funny �� but just keep practicing yunno, you'll get there! ����", "in_reply_to_status": 732102953692057600, "in_reply_to_user": 3163775886, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3163775886 }}, "user": { "id": 547073828, "name": "Chibuzo Okafor", "screen_name": "menofvalor_", "lang": "en", "location": "null", "create_at": date("2012-04-06"), "description": "Nigerian | Mechanical Engineering Major #PVAMU ♊️", "followers_count": 1234, "friends_count": 1147, "statues_count": 22757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103302624755712, "text": "@Bel__Love tell me about. I be so damn bored", "in_reply_to_status": 732103208991100928, "in_reply_to_user": 233921062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 233921062 }}, "user": { "id": 459638998, "name": "Concrete Rose", "screen_name": "RoyalTtee", "lang": "en", "location": "Made by Tau Beta NonGreater", "create_at": date("2012-01-09"), "description": "I Make Myself So Easy To Love ❤️ I'm A Particular Make! #TauBeta #DST #Spring14 #IUP16", "followers_count": 989, "friends_count": 1008, "statues_count": 65517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, PA", "id": "d88a5def1d7e9609", "name": "Indiana", "place_type": "city", "bounding_box": rectangle("-79.213942,40.586024 -79.089163,40.656614") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42063, "countyName": "Indiana", "cityID": 4236816, "cityName": "Indiana" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103302851137537, "text": "wow some people actually get it ���� https://t.co/yoF4oRancp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 452545245, "name": "Caryn Vieira", "screen_name": "cmvbby", "lang": "en", "location": "Dirty T", "create_at": date("2012-01-01"), "description": "Rule number 1 is never be number 2 | Arizona Theta", "followers_count": 1602, "friends_count": 441, "statues_count": 13287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron Park, CA", "id": "0f2405443201a34d", "name": "Cameron Park", "place_type": "city", "bounding_box": rectangle("-121.033013,38.65476 -120.924554,38.726092") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 610256, "cityName": "Cameron Park" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103303060815873, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":43.0°F Wind:0.0mph Pressure: 30.24hpa Rising slowly Rain Today 0.00in. Forecast: Settled fine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 317981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103303174062080, "text": "already https://t.co/qLYP03HPE1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3122265856, "name": "graveyard gurl", "screen_name": "lilayanah", "lang": "en", "location": "bella noche ", "create_at": date("2015-03-27"), "description": "no❤️", "followers_count": 733, "friends_count": 385, "statues_count": 56861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-16T00:00:03.000Z"), "id": 732103303257972736, "text": "YOU WERE THE ONE THAT COULDN'T CATCH OMG COME AT ME. but thanks so much�� https://t.co/AVNPLuGcQA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4508886792, "name": "21", "screen_name": "mckennacheff_", "lang": "en", "location": "null", "create_at": date("2015-12-09"), "description": "I like dirt bikes", "followers_count": 317, "friends_count": 521, "statues_count": 2226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103303689965568, "text": "‼️‼️‼️‼️‼️‼️ https://t.co/WyTTE7Y2Tc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2193382568, "name": "Sulmama❣", "screen_name": "srctorres", "lang": "en", "location": "bay area, CA", "create_at": date("2013-11-13"), "description": "HHS • Latina", "followers_count": 681, "friends_count": 681, "statues_count": 5592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103303736246272, "text": "@MissCarlyLauren I can only imagine. Only thing to do is sleep it off and try to flip your sleep schedule back.", "in_reply_to_status": 732098635278602240, "in_reply_to_user": 38614952, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38614952 }}, "user": { "id": 48368979, "name": "Johnny McDonald", "screen_name": "johnnymac8", "lang": "en", "location": "Maryville, TN", "create_at": date("2009-06-18"), "description": "CAUTION: Evil Genius At Work", "followers_count": 301, "friends_count": 1586, "statues_count": 12827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103303937462272, "text": "Mamma told me I was gifted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1119530191, "name": "Mauricio39", "screen_name": "mauricioo39", "lang": "en", "location": "null", "create_at": date("2013-01-25"), "description": "God Comes First ️", "followers_count": 535, "friends_count": 378, "statues_count": 15053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103304017121280, "text": "my roomie with her man and my single ass salty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3008628120, "name": "King J", "screen_name": "jaylawat_", "lang": "en", "location": "est. in california✨", "create_at": date("2015-02-02"), "description": "18. | all summer 6teen", "followers_count": 1920, "friends_count": 1908, "statues_count": 391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103304180719616, "text": "Congrats all you fakas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3169244978, "name": "Faka", "screen_name": "CaramonJR", "lang": "en", "location": "null", "create_at": date("2015-04-23"), "description": "MHS Inspire Church", "followers_count": 79, "friends_count": 81, "statues_count": 267 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103304394629120, "text": "Wind 1.6 mph E. Barometer 29.977 in, Falling. Temperature 62.2 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103304893714432, "text": "What is up with 1000 projects last week of school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3013816346, "name": "Rachel Caldwell", "screen_name": "ikanteven98", "lang": "en", "location": "null", "create_at": date("2015-02-08"), "description": "But rose, if you are brilliant, it is not because your petals are the without-which-nothing of pre-eminence... your thorns are the best part of you", "followers_count": 249, "friends_count": 217, "statues_count": 1706 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103305669677056, "text": "@98_monreal bn beca", "in_reply_to_status": 732103136937021440, "in_reply_to_user": 2826868416, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 2826868416 }}, "user": { "id": 358300414, "name": "Antony de Luna", "screen_name": "AntonyDeLuna", "lang": "es", "location": "Houston, TX", "create_at": date("2011-08-19"), "description": "null", "followers_count": 421, "friends_count": 205, "statues_count": 17776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103305703378944, "text": "I just can't be a woe to a bitch that don't want to better haself", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1089585109, "name": "HESHPUTA .:", "screen_name": "sassyazzb1", "lang": "en", "location": "Los Angeles / Bay Area ", "create_at": date("2013-01-14"), "description": "ig•saazzzzzzzzz", "followers_count": 343, "friends_count": 586, "statues_count": 5969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103305707425793, "text": "@ihtsroseyboo alright I'm fuckin ready!!! See me Rosey boo��", "in_reply_to_status": 732103180973023232, "in_reply_to_user": 1099082179, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1099082179 }}, "user": { "id": 1158659394, "name": "••Rìtçhàá⥕•", "screen_name": "ClubXHype", "lang": "en", "location": "SnapChat-RitcheRichSison", "create_at": date("2013-02-07"), "description": "Yesterday is history, Today is a gift, Tomorrow is a mystery⚠️ #ASOC #HDYNATION #RaveBooty", "followers_count": 379, "friends_count": 222, "statues_count": 28255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103306164625409, "text": "Ima cuff you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1455930216, "name": "Big-D", "screen_name": "ruizdaniel12", "lang": "en", "location": "null", "create_at": date("2013-05-24"), "description": "views6️⃣", "followers_count": 483, "friends_count": 419, "statues_count": 6696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103307108343808, "text": "Cosmetologist �� https://t.co/b5lPKVrJmH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1086529536, "name": "aquafina ✨", "screen_name": "stay_g0ldd", "lang": "en", "location": "property of the AVE❤️", "create_at": date("2013-01-13"), "description": "promise not to fumble, Get Money, remain humble ❤️", "followers_count": 637, "friends_count": 847, "statues_count": 11493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103307242561536, "text": "Yeet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 830197044, "name": "bri", "screen_name": "brianatijerina", "lang": "en", "location": "#UNT19", "create_at": date("2012-09-17"), "description": "he hit me up hey bri", "followers_count": 1475, "friends_count": 677, "statues_count": 32431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Krum, TX", "id": "0179869450874e99", "name": "Krum", "place_type": "city", "bounding_box": rectangle("-97.2553,33.226272 -97.207466,33.285836") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4839928, "cityName": "Krum" } }
+{ "create_at": datetime("2016-05-16T00:00:04.000Z"), "id": 732103307456483328, "text": "\"Dreaming is free\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18547763, "name": "Viola Lasmana", "screen_name": "viola_lasmana", "lang": "en", "location": "Los Angeles - Bogor/Jakarta ", "create_at": date("2009-01-01"), "description": "PhD candidate, English • Mellon DH Fellow • University of Southern California • Am Lit, SEA, Indonesia, archives, feminist media, pedagogies, remix, manutd ⚽️", "followers_count": 852, "friends_count": 445, "statues_count": 3743 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103307708108800, "text": "@buhlayzzz I always felt like her character had no reason to be there", "in_reply_to_status": 732100950052102144, "in_reply_to_user": 363735633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 363735633 }}, "user": { "id": 318811227, "name": "lil staff infection", "screen_name": "justusinfinity", "lang": "en", "location": "null", "create_at": date("2011-06-16"), "description": "Goldie McGee's Social Decree was the proclamation that sent the universe into chaos TXST18", "followers_count": 999, "friends_count": 448, "statues_count": 61435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103308882681857, "text": "Howell, NJ | Wind 0.0 mph SW. Baro 30.054 in, Rising. Temp 42.1F. Rain today 0.00 in. Humidity 68% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 621, "friends_count": 835, "statues_count": 42491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103308932972544, "text": "#temperature https://t.co/wv5hV2F8p3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.6651869,42.0655527"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "temperature" }}, "user": { "id": 51549398, "name": "Josh Dalton", "screen_name": "JDalton82", "lang": "en", "location": "Maquoketa, IA", "create_at": date("2009-06-27"), "description": "Quality Control Manager / Systems Administrator", "followers_count": 12, "friends_count": 20, "statues_count": 52 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maquoketa, IA", "id": "07fcf97b714d9a8d", "name": "Maquoketa", "place_type": "city", "bounding_box": rectangle("-90.693463,42.045187 -90.642112,42.080088") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19097, "countyName": "Jackson", "cityID": 1949215, "cityName": "Maquoketa" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103309314625536, "text": "MARLBORO TWP SC is hiring! ESL Teacher #jobs in BRADEVELT Apply today https://t.co/dzNUgFZrny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.257216,40.313518"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jobs" }}, "user": { "id": 2303619546, "name": "NJ.com Jobs", "screen_name": "Advance_NJJobs", "lang": "en", "location": "null", "create_at": date("2014-01-21"), "description": "Follow http://NJ.com Jobs to find the perfect job for you in the New Jersey area.", "followers_count": 215, "friends_count": 42, "statues_count": 81426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlboro, NJ", "id": "017591d343776c19", "name": "Marlboro", "place_type": "city", "bounding_box": rectangle("-74.299025,40.277843 -74.197542,40.390452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103309666848768, "text": "Hungry and irritated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2226057139, "name": "KATE", "screen_name": "kateelynnrenee", "lang": "en", "location": "null", "create_at": date("2013-12-01"), "description": "J / 17", "followers_count": 533, "friends_count": 368, "statues_count": 8811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cerritos, CA", "id": "19d41c6eff11e9d6", "name": "Cerritos", "place_type": "city", "bounding_box": rectangle("-118.108568,33.84596 -118.02881,33.887971") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612552, "cityName": "Cerritos" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103309700423680, "text": "@TeachForAmerica @#mash #abLe Time to #hashtagthatshit #ven if #testing #protocoL #vioLation o no: #gROWthMiNDset #RowRowRowYourBoat ♥ ☼ ♫", "in_reply_to_status": 731877013749940227, "in_reply_to_user": 19918047, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mash", "abLe", "hashtagthatshit", "ven", "testing", "protocoL", "vioLation", "gROWthMiNDset", "RowRowRowYourBoat" }}, "user_mentions": {{ 19918047 }}, "user": { "id": 178412443, "name": "Vladimir Kozmich Z.", "screen_name": "vladimir_kosma", "lang": "en", "location": "Tijuana, Baja California", "create_at": date("2010-08-14"), "description": "Instituto especializado en Sistemas Administrativos, Contables, Mantenimiento y reparacion de PC e idiomas.", "followers_count": 936, "friends_count": 2329, "statues_count": 6896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103309876695041, "text": "Wind 1.4 mph WNW. Barometer 29.774 in, Steady. Temperature 35.7 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103310170193920, "text": "@TheUncurvablee nah. I never fuck up.. ��", "in_reply_to_status": 732103126308651009, "in_reply_to_user": 1402874713, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1402874713 }}, "user": { "id": 2346149261, "name": "Z", "screen_name": "BigDaddyZeee", "lang": "en", "location": "Blessed ", "create_at": date("2014-02-15"), "description": "Allah . Mommy & Mali . Somali . #UTANursing #IDontReplyToDms", "followers_count": 8507, "friends_count": 700, "statues_count": 53392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103310249857024, "text": "#treatyoself ☝��️ https://t.co/q8fPIgPBgI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "treatyoself" }}, "user": { "id": 84733063, "name": "groovy steff", "screen_name": "steffefff", "lang": "en", "location": "null", "create_at": date("2009-10-23"), "description": "null", "followers_count": 368, "friends_count": 261, "statues_count": 16236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103310455476225, "text": "3:00am ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 186180069, "name": "Reemo", "screen_name": "PhillyReem", "lang": "en", "location": "PHILLY ", "create_at": date("2010-09-02"), "description": "couldn't talk about it if you ain't live it", "followers_count": 2110, "friends_count": 351, "statues_count": 111524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103310568792066, "text": "MARLBORO TWP SC is hiring! Speech Lang. Sp #jobs in BRADEVELT Apply today https://t.co/3xHDs2p3Fq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.257216,40.313518"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "jobs" }}, "user": { "id": 2303619546, "name": "NJ.com Jobs", "screen_name": "Advance_NJJobs", "lang": "en", "location": "null", "create_at": date("2014-01-21"), "description": "Follow http://NJ.com Jobs to find the perfect job for you in the New Jersey area.", "followers_count": 215, "friends_count": 42, "statues_count": 81427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marlboro, NJ", "id": "017591d343776c19", "name": "Marlboro", "place_type": "city", "bounding_box": rectangle("-74.299025,40.277843 -74.197542,40.390452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103310715453441, "text": "spring transition https://t.co/3V6AZKEtVk #ontheblog #fashion #fashionblogger", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2767995,39.9831886"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontheblog", "fashion", "fashionblogger" }}, "user": { "id": 65940718, "name": "Brett Elizabeth", "screen_name": "BrettEParker", "lang": "en", "location": "NYC", "create_at": date("2009-08-15"), "description": "i ❤ nyc.", "followers_count": 144, "friends_count": 226, "statues_count": 2211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18011, "countyName": "Boone", "cityID": 1886372, "cityName": "Zionsville" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103311130693633, "text": "Temp: 43.0°F - Dew Point: 34.6° - Wind: S @ 5.0 mph - Gust: 8.9 - Rain Today: 0.00in. - Pressure: 30.06in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 7, "friends_count": 12, "statues_count": 17741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103311139037188, "text": "There's only portions of my life I let y'all see through these social media but do you really know wussup? Naaaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297199497, "name": "Boi Weston", "screen_name": "JoseWeston_", "lang": "en", "location": "null", "create_at": date("2011-05-11"), "description": "fuck you doing lurkin n shit", "followers_count": 1459, "friends_count": 1190, "statues_count": 52179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shafter, CA", "id": "3b642703733bf53f", "name": "Shafter", "place_type": "city", "bounding_box": rectangle("-119.300846,35.463437 -119.251513,35.521029") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 671106, "cityName": "Shafter" } }
+{ "create_at": datetime("2016-05-16T00:00:05.000Z"), "id": 732103311185301504, "text": "NEW POST: Skin Base Choices That Won't Sweat Off https://t.co/rNO0GjozQ9 #fbloggers #bbloggers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "fbloggers", "bbloggers" }}, "user": { "id": 257921418, "name": "Life of Ellie Grace", "screen_name": "elliegdickinson", "lang": "en", "location": "Manchester", "create_at": date("2011-02-26"), "description": "I spend my free time baking and blogging. Style, beauty and lifestyle blogger. UK Blog Award finalist. lifeofellieg@gmail.com | http://youtube.com/elliechatters", "followers_count": 2195, "friends_count": 488, "statues_count": 16909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465674694451200, "text": "@Supermercado9 just remember.. You love me. https://t.co/SI2rs0a1Ec", "in_reply_to_status": 732465494356168705, "in_reply_to_user": 766107505, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 467803770 }}, "user": { "id": 766107505, "name": "Martin Portillo", "screen_name": "MartinTAMU", "lang": "en", "location": "College Station, TX", "create_at": date("2012-08-18"), "description": "Texas A&M University Class of 2018 | Dwight Look College of Engineering | MMET | Kids Club CSISD |", "followers_count": 1037, "friends_count": 844, "statues_count": 17265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465674778353664, "text": "I miss my iPhone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735625059, "name": "'Megan'sMommy:'", "screen_name": "ImagineBeingMeh", "lang": "en", "location": "StandingAlone♡♥", "create_at": date("2014-08-06"), "description": "Megan World I'm just leaving in it ✊", "followers_count": 3475, "friends_count": 2655, "statues_count": 63551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465675772403713, "text": "@MartinTorres39 https://t.co/AwKwOhHIBH", "in_reply_to_status": -1, "in_reply_to_user": 317563475, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 317563475 }}, "user": { "id": 1629714276, "name": "cornrow roly", "screen_name": "fucktonosazules", "lang": "en", "location": "null", "create_at": date("2013-07-29"), "description": "I'm a grown ass kid", "followers_count": 221, "friends_count": 272, "statues_count": 13575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465675839496193, "text": "kimchi fried rice �� #gonegirl #girlbye thankqpocha @ Thank Q Pocha https://t.co/Jq8cDIece8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.8436279,21.2930508"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "gonegirl", "girlbye" }}, "user": { "id": 615021660, "name": "mark melchor", "screen_name": "mark_f3rd1nand", "lang": "en", "location": "Honolulu, Hi ", "create_at": date("2012-06-22"), "description": "Mark. 27. registered nurse. i wipe ass for fun. underpaid slave.", "followers_count": 47, "friends_count": 92, "statues_count": 2524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465676581920771, "text": "I'm so patient it's annoying hahah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1003503619, "name": "nena", "screen_name": "elenaxsolorzano", "lang": "en", "location": "null", "create_at": date("2012-12-10"), "description": "null", "followers_count": 528, "friends_count": 383, "statues_count": 11952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465677215223810, "text": "I love my girlfriend dude, she's one in a million", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 579411575, "name": "Liam MF. Doom", "screen_name": "BuddhaLiam_", "lang": "en", "location": "Flatbush, NY", "create_at": date("2012-05-13"), "description": "Keepin' it Kosher with Ceci", "followers_count": 402, "friends_count": 213, "statues_count": 43271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465677471080448, "text": "I had never felt so comfortable with someone it's fuckin crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 619548893, "name": "Delmi", "screen_name": "swagovianooo", "lang": "en", "location": "null", "create_at": date("2012-06-26"), "description": "In the land of gods and monsters.", "followers_count": 793, "friends_count": 343, "statues_count": 48565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465678045716480, "text": "@ebbtideapp Tide in Hunts Point, New York 05/17/2016\n Low 4:01am 0.3\nHigh 10:00am 7.0\n Low 4:18pm 0.5\nHigh 10:17pm 7.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.8733,40.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 89, "friends_count": 1, "statues_count": 33449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465678146396160, "text": "FUCK IT FUCK IT FUCK IT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242895649, "name": "slim", "screen_name": "JustBeingSlim_", "lang": "en", "location": "Bembroke", "create_at": date("2011-01-25"), "description": "t.hemphill❤️ 2.12", "followers_count": 1339, "friends_count": 1394, "statues_count": 17339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465678402211840, "text": "why am i wide awake rn...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2778960453, "name": "kaitlyn looney", "screen_name": "kaitlynlooney_", "lang": "en", "location": "null", "create_at": date("2014-09-21"), "description": "ghs vball sc; kaitlyn_looney", "followers_count": 556, "friends_count": 231, "statues_count": 3839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green, OH", "id": "009103ea59aa47b3", "name": "Green", "place_type": "city", "bounding_box": rectangle("-81.54525,40.872987 -81.416329,40.997434") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3931860, "cityName": "Green" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465678440005632, "text": "Interested in a #Retail #job near #DEFUNIAKSPRINGS, FL? This could be a great fit: https://t.co/rF2k5PboLp #Hiring https://t.co/XtT1mHj4FL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.1240237,30.7071159"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "job", "DEFUNIAKSPRINGS", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 534, "friends_count": 266, "statues_count": 18462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "De Funiak Springs, FL", "id": "9355db7600fdf89d", "name": "De Funiak Springs", "place_type": "city", "bounding_box": rectangle("-86.150119,30.691614 -86.084603,30.749959") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12131, "countyName": "Walton", "cityID": 1216800, "cityName": "DeFuniak Springs" } }
+{ "create_at": datetime("2016-05-17T00:00:00.000Z"), "id": 732465678544867328, "text": "@deexxzee car crash", "in_reply_to_status": 732465355973525504, "in_reply_to_user": 2321543538, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2321543538 }}, "user": { "id": 1595157042, "name": "Brandon Martinez", "screen_name": "Bowtievrandon", "lang": "en", "location": "Rialto, CA", "create_at": date("2013-07-14"), "description": "I'm horngry 24/7", "followers_count": 356, "friends_count": 400, "statues_count": 11588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465678674853890, "text": "@paulina1738 too late lmao", "in_reply_to_status": 732465590632255489, "in_reply_to_user": 351344743, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351344743 }}, "user": { "id": 160456519, "name": "GasGangDre⛽️", "screen_name": "ochonocinco_", "lang": "en", "location": "B$T", "create_at": date("2010-06-27"), "description": "#FreePat #28 #LongLiveJakacy #GG⛽️⛽️", "followers_count": 1627, "friends_count": 554, "statues_count": 36534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465678872010754, "text": "Happy birthday to my beautiful ladies @asianxgal @dtfrances ❤️ love you guys to pieces ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 495121381, 591491150 }}, "user": { "id": 3877119374, "name": "Amanpreet Kaur", "screen_name": "Its_amanpreet", "lang": "en", "location": "null", "create_at": date("2015-10-12"), "description": "AZ. 22. Pharmacist Techician. @itsamanpreet snap.", "followers_count": 107, "friends_count": 83, "statues_count": 2573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Mirage, AZ", "id": "0de4c71dbfcd2c32", "name": "El Mirage", "place_type": "city", "bounding_box": rectangle("-112.342111,33.579997 -112.302246,33.630786") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 422220, "cityName": "El Mirage" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465678930731008, "text": "@imperfectGirl__ but I haven't heard from you since last week! Bet!", "in_reply_to_status": 732395318915026944, "in_reply_to_user": 235773282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 235773282 }}, "user": { "id": 40140229, "name": "Wreck 'Em™", "screen_name": "Chase_YoDREams", "lang": "en", "location": "Successville , TX", "create_at": date("2009-05-14"), "description": "Still writing my story. Stay Tuned!", "followers_count": 1289, "friends_count": 1029, "statues_count": 28909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465679408865280, "text": "need a piece", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 961735974, "name": "sky", "screen_name": "SkylarBurdick1", "lang": "en", "location": "Arlington, WA", "create_at": date("2012-11-20"), "description": "null", "followers_count": 1336, "friends_count": 529, "statues_count": 15896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465679454982144, "text": "#tnsteeplechase #steeplechase2016 #steeplechasehats @ Iroquois… https://t.co/k12KZqx13g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.89321761,36.05920689"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "tnsteeplechase", "steeplechase2016", "steeplechasehats" }}, "user": { "id": 87640441, "name": "Daniel Oyvetsky", "screen_name": "ChiaroFoto", "lang": "en", "location": "Nashville, TN ", "create_at": date("2009-11-04"), "description": "I WOULD RATHER BE KNOWN FOR SOMETHING, THAN TO BE NEVER KNOWN AT ALL.", "followers_count": 195, "friends_count": 555, "statues_count": 559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tennessee, USA", "id": "7f7d58e5229c6b6c", "name": "Tennessee", "place_type": "admin", "bounding_box": rectangle("-90.310298,34.982924 -81.646901,36.678119") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47037, "countyName": "Davidson", "cityID": 4752006, "cityName": "Nashville-Davidson metropolitan government (balance)" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465679572471808, "text": "��������- you're so mellow and down to earth it's trippy hahah, although you happen to have those tendencies where you blank out severely lmao..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1584595332, "name": "Sativa", "screen_name": "TikiBruuh", "lang": "en", "location": "null", "create_at": date("2013-07-10"), "description": "Add me & follow me on Snapchat/instagram @Tikibruuh", "followers_count": 603, "friends_count": 613, "statues_count": 34205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "National City, CA", "id": "7905cfee1600eb70", "name": "National City", "place_type": "city", "bounding_box": rectangle("-117.124452,32.64634 -117.049103,32.69328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 650398, "cityName": "National City" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465679656353792, "text": "People that use words to make someone feel special. When you're probably telling her the same exact things.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 337433792, "name": "McBeezy.☹️", "screen_name": "KatieeMcGowan", "lang": "en", "location": "LJ", "create_at": date("2011-07-17"), "description": "I'm going to hell. |HRC|CLJ|", "followers_count": 1357, "friends_count": 99, "statues_count": 121593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465679949922304, "text": "Every woman in this world wants a man that can charge but also be loving to her at the same time #facts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.65218307,34.73873874"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "facts" }}, "user": { "id": 48008737, "name": "Officially: R-E-D", "screen_name": "EccentricDancer", "lang": "en", "location": "#AAMU18 ", "create_at": date("2009-06-17"), "description": "Dancing on the edge of insanity ..Marketing And Communications Major ...IG: Eccentric_Dancer", "followers_count": 489, "friends_count": 531, "statues_count": 7352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashville TN", "id": "f449c8452e58c343", "name": "Nashville TN", "place_type": "admin", "bounding_box": rectangle("-88.32348,35.082324 -84.692649,37.075646") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465680621162496, "text": "�� https://t.co/h6PpiMjGlF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1244776862, "name": "Top Flight OTC LLC", "screen_name": "TFSterl", "lang": "en", "location": "null", "create_at": date("2013-03-05"), "description": "U N L V , 21", "followers_count": 1215, "friends_count": 870, "statues_count": 95142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465680692338688, "text": "@tweak4qui so petty ����", "in_reply_to_status": 732464462532251649, "in_reply_to_user": 2823966416, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2823966416 }}, "user": { "id": 2218323634, "name": "NUNNIE❤️", "screen_name": "cxlumbian_lova", "lang": "en", "location": "In Da Heauxs GM ☺️", "create_at": date("2013-12-10"), "description": "Oh K❤️", "followers_count": 1121, "friends_count": 915, "statues_count": 41276 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465680960757760, "text": "You don't get a thank you for being a decent human being. Also, let's talk about the murder of 5 million blacks https://t.co/MGBnePF8bx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554827658, "name": "Eli Miller", "screen_name": "eli_miller33", "lang": "en", "location": "San Marcos, TX", "create_at": date("2012-04-15"), "description": "-Jeremiah 29:11- | Vice President of Texas State Democrats| Texas State University Senator|Breakfast Taco Enthusiast #UniteBlue", "followers_count": 1258, "friends_count": 614, "statues_count": 26800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hebron, TX", "id": "017b94d274b6cbe2", "name": "Hebron", "place_type": "city", "bounding_box": rectangle("-96.89339,33.024198 -96.842676,33.060901") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4833020, "cityName": "Hebron" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681183080448, "text": "And you've gotta reach the ultimate level of laziness if u create a separate bag of trash when the trash is full rather than emptying it��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1115874343, "name": "Dej", "screen_name": "dejaa_dejj", "lang": "en", "location": "McDonalds", "create_at": date("2013-01-23"), "description": "ordinary", "followers_count": 319, "friends_count": 262, "statues_count": 974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681212432384, "text": "I'm guilty… Pray this prayer at https://t.co/nDP4XeULQs @JW_Branding @jw_lerner @davies20162 @HLERadio @feefee31 @LakeishaSinclai @terrymayz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3188079572, 112973191, 3631810637, 618356934, 21673881, 2163150918, 192523220 }}, "user": { "id": 2427795163, "name": "Charleigh Goolsby", "screen_name": "cngoolsby2014", "lang": "en", "location": "Knoxville, TN", "create_at": date("2014-04-04"), "description": "christian DTBH fan country music nascar fan. ATTN self help media marketing gurus I block everyone of you I see", "followers_count": 253, "friends_count": 305, "statues_count": 235740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681220829184, "text": "@CocksDaily @SharingCocks @CockSharers @SelfieDickDaily @SelfiesCock @Selfpicture @CutCocks @CockOfTheDay1 #cock https://t.co/2tY72ODqWI", "in_reply_to_status": 731186809145679872, "in_reply_to_user": 265286834, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "cock" }}, "user_mentions": {{ 929805432, 1635364716, 2279377496, 1335133086, 2612031737, 2825137444, 2725726339, 1377902623 }}, "user": { "id": 265286834, "name": "Mr Dickey Robinson", "screen_name": "MYDICK4LADIES", "lang": "en", "location": "#ButThatsNoneOfMyBusiness", "create_at": date("2011-03-13"), "description": "I love sex with women & trans big on #420 I love a #phat #ass, phat wet #squirter #pussy #Booty #Shemale #cock #TS I only suck dick if it has tits", "followers_count": 574, "friends_count": 1324, "statues_count": 7020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681279549440, "text": "@NataliaLeighton asf ✨ https://t.co/aNQ2EuU3QQ", "in_reply_to_status": -1, "in_reply_to_user": 2183308627, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2183308627 }}, "user": { "id": 157820438, "name": "ali als", "screen_name": "aligatorita", "lang": "en", "location": "Bay Area", "create_at": date("2010-06-20"), "description": "null", "followers_count": 671, "friends_count": 533, "statues_count": 7053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rohnert Park, CA", "id": "5cda0a6ac9cf8725", "name": "Rohnert Park", "place_type": "city", "bounding_box": rectangle("-122.732141,38.314281 -122.666695,38.372609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 662546, "cityName": "Rohnert Park" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681350852612, "text": "Cousins exes my exes bestfriend exes https://t.co/ekTnWdKxoa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2877768962, "name": "♐Young Mani♐", "screen_name": "youngmani98", "lang": "en", "location": "null", "create_at": date("2014-10-25"), "description": "null", "followers_count": 403, "friends_count": 223, "statues_count": 32178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681401151489, "text": "Sometimes you have to step into someone else's shoes to get some perspective", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2284713025, "name": "Tali™", "screen_name": "tali_tofler", "lang": "en", "location": "null", "create_at": date("2014-01-09"), "description": "it's all gooood", "followers_count": 205, "friends_count": 295, "statues_count": 1215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681405337600, "text": "Temp 42.2° Hi/Lo 48.7/42.2 Rng 6.5° WC 42.2° Hmd 78% Rain 0.00\" Storm 0.00\" BAR 30.105 Rising DP 35.8° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681602478081, "text": "@TKbreezy (superchillin)but keep quite ��", "in_reply_to_status": 732452726479552512, "in_reply_to_user": 79861763, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 79861763 }}, "user": { "id": 954975343, "name": "Frankenberry", "screen_name": "Grossyyy", "lang": "en", "location": "null", "create_at": date("2012-11-17"), "description": "Occupation: Struggling tumblr comedian, Member of holding that L society", "followers_count": 155, "friends_count": 603, "statues_count": 2933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681875111936, "text": "Slab shinin wit the grill & woman", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2545364362, "name": "Stuart $hittle", "screen_name": "DrippSkylarC", "lang": "en", "location": "Nawf Houston ", "create_at": date("2014-05-13"), "description": "I'm juss tryna make it mayne.. \nFaux Figga Nigga \n#Trill #SuccaFree #WiseKneeGrow", "followers_count": 666, "friends_count": 500, "statues_count": 45281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465681933819904, "text": "05/17@03:00 - Temp 49.6F, WC 49.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.216in, Rising slowly. Rain 0.00in. Hum 83%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 48992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682479091712, "text": "I wanna wax all my shit!!! Lol like my arpits my legs errrthing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 760102172, "name": "Darr✨", "screen_name": "_Deeedubb", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "fuck wit me", "followers_count": 768, "friends_count": 494, "statues_count": 40745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682504253440, "text": "Howell, NJ | Wind 1.0 mph SW. Baro 30.190 in, Steady. Temp 53.1F. Rain today 0.00 in. Humidity 61% | https://t.co/jrBhN6IRrK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 621, "friends_count": 835, "statues_count": 42541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682600710145, "text": "Work at 11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2278606968, "name": "Mello", "screen_name": "Jmello_12", "lang": "en", "location": "Richmond, VA", "create_at": date("2014-01-05"), "description": "#fuckemweball Vcu 18'", "followers_count": 624, "friends_count": 575, "statues_count": 9205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682634280960, "text": "What y'all know bout mo better blues blue hill avenue", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2458989830, "name": "blessing.", "screen_name": "_xLoveAB", "lang": "en", "location": "null", "create_at": date("2014-04-22"), "description": "FSU18'", "followers_count": 4868, "friends_count": 2008, "statues_count": 78127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682810474497, "text": "On Pinterest looking for the next project Ima do with my baby's ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417665415, "name": "La Reina ✨", "screen_name": "iMaccMinaa", "lang": "en", "location": "null", "create_at": date("2011-11-20"), "description": "• Married to Daniel Zavala • 12052k15 • Libra • snapchat : iMaccMinaa", "followers_count": 435, "friends_count": 301, "statues_count": 35745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-17T00:00:01.000Z"), "id": 732465682835603457, "text": "Ripley SW Limestone Co. Temp: 61.5°F Wind:0.7mph Pressure: 997.1mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465683125047296, "text": "So I got a final tomorrow... Lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 496102027, "name": "jojo", "screen_name": "Jo_J15", "lang": "en", "location": "null", "create_at": date("2012-02-18"), "description": "haterrrrrr", "followers_count": 263, "friends_count": 224, "statues_count": 2455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menlo Park, CA", "id": "490bdb082950484f", "name": "Menlo Park", "place_type": "city", "bounding_box": rectangle("-122.228922,37.416515 -122.120415,37.507328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 646870, "cityName": "Menlo Park" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465683263422464, "text": "�� https://t.co/dSmSbpR786", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3351572176, "name": "Prettyblacksenorita", "screen_name": "toootaaay", "lang": "en", "location": "null", "create_at": date("2015-06-29"), "description": "mcneese state follow me on sc: toootaaay", "followers_count": 539, "friends_count": 481, "statues_count": 2624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Iberia, LA", "id": "7d8e6592712895b9", "name": "New Iberia", "place_type": "city", "bounding_box": rectangle("-91.892077,29.941102 -91.731432,30.04583") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22045, "countyName": "Iberia", "cityID": 2254035, "cityName": "New Iberia" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465683376676864, "text": "I don't like being mean ��.. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1925438821, "name": "Andrea✨", "screen_name": "LoveDreaaa", "lang": "en", "location": "null", "create_at": date("2013-10-01"), "description": "R.I.P. Mook❤️ ... R.I.P. Baby boy❤️", "followers_count": 1340, "friends_count": 807, "statues_count": 47589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.65479 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465683422797824, "text": "Eh yeah I'm annoying but hey at least my weirdness keeps me going :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3242544754, "name": "odnanreF", "screen_name": "HIodnanreF", "lang": "en", "location": "Houston, TX", "create_at": date("2015-05-08"), "description": "odnanreF learsI zednanreH ||\n I'm just like you, a vaca.\n@Bluehouse1996", "followers_count": 192, "friends_count": 148, "statues_count": 8385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465683921928193, "text": "68.6F (Feels: 68.6F) - Humidity: 99% - Wind: 3.8mph E - Gust: 7.6mph - Pressure: 1009.3mb - Rain: 0.02\" #weather https://t.co/75yzNJjEvP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 240648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465684303601665, "text": "Wind 1.0 mph SE. Barometer 30.139 in, Steady. Temperature 48.1 °F. Rain today 0.01 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465684404264960, "text": "Wind 0.0 mph ENE. Barometer 30.201 in, Steady. Temperature 48.4 °F. Rain today 0.14 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465684953714688, "text": "Wind 0.0 mph NE. Barometer 30.04 in, Falling slowly. Temperature 67.1 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465684999856130, "text": "Temp: 67.2°F Wind:0.0mph Pressure: 29.977hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465685008244736, "text": "@marshmellomusic @YELLOWCLAW @djsnake POST YOUR SETS FROM #EDCNY2016 PLEASE������������⚠️⚠️", "in_reply_to_status": -1, "in_reply_to_user": 2987922767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EDCNY2016" }}, "user_mentions": {{ 2987922767, 235542752, 22317367 }}, "user": { "id": 1688847164, "name": "Don Lobo", "screen_name": "lobo_angelo", "lang": "en", "location": "null", "create_at": date("2013-08-21"), "description": "Recognize a real don when you see one ⚠️", "followers_count": 343, "friends_count": 1208, "statues_count": 2428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenwich, CT", "id": "005bdbe763d96c1d", "name": "Greenwich", "place_type": "city", "bounding_box": rectangle("-73.710304,41.00137 -73.568516,41.099064") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 933690, "cityName": "Greenwich" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465685155086343, "text": "Wind 0.0 mph WNW. Barometer 29.899 in, Falling. Temperature 66.1 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465685754830848, "text": "I have never been more proud of a piece of writing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 353961112, "name": "kim bryant", "screen_name": "vibewithkim", "lang": "en", "location": "null", "create_at": date("2011-08-12"), "description": "mom^", "followers_count": 444, "friends_count": 285, "statues_count": 29429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:02.000Z"), "id": 732465685863915525, "text": "Dapat talaga wag mag expect para di ka masaktan you know", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3063372984, "name": "Maureen Jane", "screen_name": "itsmjanebabe", "lang": "en", "location": "Buena Park, CA", "create_at": date("2015-03-05"), "description": "Yaaaas", "followers_count": 41, "friends_count": 64, "statues_count": 1449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buena Park, CA", "id": "0b93bc6a33455615", "name": "Buena Park", "place_type": "city", "bounding_box": rectangle("-118.037391,33.809829 -117.975923,33.895642") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 608786, "cityName": "Buena Park" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465687105413126, "text": "@HaleyyCatherine copy that black hawk ��", "in_reply_to_status": 732463334243500033, "in_reply_to_user": 28471789, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28471789 }}, "user": { "id": 987560467, "name": "erin stella", "screen_name": "erinastella", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "University of Utah '19", "followers_count": 503, "friends_count": 402, "statues_count": 7454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465687407431681, "text": "Wind 3.2 mph ESE. Barometer 29.868 in, Rising slowly. Temperature 63.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465687566778368, "text": "Lit �� https://t.co/7va9JhLTqQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2225886948, "name": "Vibezzz", "screen_name": "Vontepaapi", "lang": "en", "location": "Crenshaw, Los Angeles", "create_at": date("2013-12-01"), "description": "null", "followers_count": 349, "friends_count": 333, "statues_count": 3452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465687998783491, "text": "Tuesday 5-17-16 https://t.co/c9o6OcAjaQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3788975,33.8861319"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123626591, "name": "Karma CrossFit", "screen_name": "KarmaCrossFit", "lang": "en", "location": "Manhattan Beach", "create_at": date("2010-03-16"), "description": "Karma is the relationship of cause and effect and #CrossFit is a perfect vehicle to improve the cause and effect relationship in your life. #karmaCrossft", "followers_count": 1149, "friends_count": 324, "statues_count": 3601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465688250482689, "text": "@alex_thomas9 @paullamercado oh frick man", "in_reply_to_status": 732458543509430274, "in_reply_to_user": 707962850, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 707962850, 2865840367 }}, "user": { "id": 605831417, "name": "j-geese", "screen_name": "JuliaGeesey", "lang": "en", "location": "highland senior", "create_at": date("2012-06-11"), "description": "constantly in awe of God's unconditional love and grace • Jeremiah 29:11 • GCU 2020", "followers_count": 595, "friends_count": 507, "statues_count": 12936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465688258838528, "text": "I wonder if he wanna go on a baecation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275838933, "name": "mattieb GATES", "screen_name": "Mattieb___", "lang": "en", "location": "null", "create_at": date("2011-04-01"), "description": "Follow me on IG @mattieb___ !!! follow me on snapchat at @mattiefuckingb", "followers_count": 2791, "friends_count": 998, "statues_count": 132144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bossier City, LA", "id": "018617530de2ac1c", "name": "Bossier City", "place_type": "city", "bounding_box": rectangle("-93.74855,32.419303 -93.584206,32.630042") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22015, "countyName": "Bossier", "cityID": 2208920, "cityName": "Bossier City" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465688485367808, "text": "Wind 0.0 mph SW. Barometer 30.11 in, Steady. Temperature 43.2 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465688577613824, "text": "NEW POST: Lunch at Vapiano | #EatPastaRunFaster https://t.co/yKfnFscphl #fbloggers #bbloggers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.0007613,40.7207559"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "EatPastaRunFaster", "fbloggers", "bbloggers" }}, "user": { "id": 257921418, "name": "Life of Ellie Grace", "screen_name": "elliegdickinson", "lang": "en", "location": "Manchester", "create_at": date("2011-02-26"), "description": "I spend my free time baking and blogging. Style, beauty and lifestyle blogger. UK Blog Award finalist. lifeofellieg@gmail.com | http://youtube.com/elliechatters", "followers_count": 2194, "friends_count": 486, "statues_count": 16929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465688590188544, "text": "KC always texts me at the right time ��✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1159060249, "name": "David J.", "screen_name": "kkingdavid24", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-02-07"), "description": "Shook one | CSUN", "followers_count": 453, "friends_count": 349, "statues_count": 9242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465689001222144, "text": "mini strawberry tarts https://t.co/D5hfN0RS2f #ontheblog #baking #strawberry #spring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.2767995,39.9831886"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ontheblog", "baking", "strawberry", "spring" }}, "user": { "id": 65940718, "name": "Brett Elizabeth", "screen_name": "BrettEParker", "lang": "en", "location": "NYC", "create_at": date("2009-08-15"), "description": "i ❤ nyc.", "followers_count": 143, "friends_count": 226, "statues_count": 2213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18011, "countyName": "Boone", "cityID": 1886372, "cityName": "Zionsville" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465689613590528, "text": "Can your grandma adopt me. https://t.co/WxdKGA8PRb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427870994, "name": "Nico", "screen_name": "mariienicolette", "lang": "en", "location": "null", "create_at": date("2011-12-03"), "description": "The problem is boys think they’re Kanye and they want me to be Kim, but in reality I’m Kanye and they’re probably Kims' brother", "followers_count": 331, "friends_count": 324, "statues_count": 10650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cupertino, CA", "id": "36237ab3643ff2be", "name": "Cupertino", "place_type": "city", "bounding_box": rectangle("-122.091151,37.285786 -121.995595,37.34038") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 617610, "cityName": "Cupertino" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465689731072000, "text": "@cuntyjenner no thanks", "in_reply_to_status": 732464793768984577, "in_reply_to_user": 887923939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 887923939 }}, "user": { "id": 343234997, "name": "laura", "screen_name": "laurayuali", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-07-27"), "description": "wus gucci", "followers_count": 4769, "friends_count": 81, "statues_count": 40927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465690112712704, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":50.0°F Wind:0.0mph Pressure: 30.15hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 82, "friends_count": 17, "statues_count": 318070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465690142113793, "text": "Boredommmmmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 247830744, "name": "☄Trapicana♠️❄️", "screen_name": "killakelseasons", "lang": "en", "location": "Long Island ✈️ Puerto Rico", "create_at": date("2011-02-05"), "description": "#BlackMafia♠️", "followers_count": 3334, "friends_count": 3550, "statues_count": 15173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commack, NY", "id": "298bb08a9d7e64d4", "name": "Commack", "place_type": "city", "bounding_box": rectangle("-73.321097,40.803631 -73.238846,40.880333") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3617530, "cityName": "Commack" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465690175643648, "text": "\"It's Labor Day weekend u know what that means! A luau at the lake for the weekend!!\" My kids gonna be like \"ugh again?\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30557240, "name": "ZuritA", "screen_name": "Jadira_GS", "lang": "en", "location": "Lancaster, CA ", "create_at": date("2009-04-11"), "description": "To show you that you are & always will, be my one & only..... L.G | |-/ | Forever Snowflake ♡", "followers_count": 642, "friends_count": 380, "statues_count": 22045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465690733477889, "text": "It's 12am and I'm livid", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 984055064, "name": "princess", "screen_name": "torimartiinn", "lang": "en", "location": " bay area ", "create_at": date("2012-12-01"), "description": "I excelled in preschool", "followers_count": 436, "friends_count": 106, "statues_count": 15086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465691018682369, "text": "sayang talaga yung binebenta nung lalaki kanina ;;;; orig price ng YSL L'Homme $100+ pero binebenta nya samin ng $50 lang ���� im cri", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1844491580, "name": "effy", "screen_name": "yoitsgianne", "lang": "en", "location": "storybrooke", "create_at": date("2013-09-09"), "description": "Romans 5:8 | じやん • 160319 met cam❤️ • disi-otso •", "followers_count": 237, "friends_count": 235, "statues_count": 51818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-17T00:00:03.000Z"), "id": 732465691022921728, "text": "my new snapchat is coltonbanderson , I deleted my old one because I hit 1,000 friends.. add meeeeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2306024509, "name": "colton", "screen_name": "CellyHard_", "lang": "en", "location": "Ohio, USA ✈️ Colorado, USA", "create_at": date("2014-01-22"), "description": "null", "followers_count": 11161, "friends_count": 7442, "statues_count": 5982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avon, OH", "id": "9d8a608c1895dce3", "name": "Avon", "place_type": "city", "bounding_box": rectangle("-82.069227,41.418361 -81.969378,41.47865") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3903352, "cityName": "Avon" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465691966636032, "text": "Temp 51.9°F Wind Chill 51.9°F RH 81% Wind 0.0 --- Gust 0.0 --- SLP 30.172 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465692939673601, "text": "I am so grateful that you brought Lainie to life you are so talented! Thank you❤️ https://t.co/I8Zdtam8jf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 704461320578400257, "name": "Kendall Kohlenberger", "screen_name": "kohlenbergerk", "lang": "en", "location": "null", "create_at": date("2016-02-29"), "description": "I lift my eyes unto the hills where does my help come from, my help comes from the Lord the Maker of Heaven and Earth!! - Casting Crowns", "followers_count": 193, "friends_count": 543, "statues_count": 445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465692998438912, "text": "Another that came and went.���� https://t.co/8c7CeOapH1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1416518569, "name": "Truth", "screen_name": "nobodywantsd", "lang": "en", "location": "null", "create_at": date("2013-05-09"), "description": "Embodiment of Sagacious. Senior || CSULA '20", "followers_count": 225, "friends_count": 408, "statues_count": 7072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465693044547584, "text": "@rugratrae ok, so it's like recycling. I digg it ♻️", "in_reply_to_status": 732465501750759424, "in_reply_to_user": 4240877366, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4240877366 }}, "user": { "id": 54372210, "name": "Puissant", "screen_name": "LIIIDDIC", "lang": "en", "location": "$ C O U T $ ", "create_at": date("2009-07-06"), "description": "Where you ain't @ // Year of the Scout", "followers_count": 805, "friends_count": 1651, "statues_count": 34042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465693103296512, "text": "@GuitarSunCat #transgenderism \n\nCall it what it is, #sexism", "in_reply_to_status": 732410095775047680, "in_reply_to_user": 942122522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "transgenderism", "sexism" }}, "user_mentions": {{ 942122522 }}, "user": { "id": 134933285, "name": "I am:", "screen_name": "DuctTapedGoat", "lang": "en", "location": "California, USA", "create_at": date("2010-04-19"), "description": "Mi¢hael Jame$ Webb Sr. Account Specialist SEO Catalogs Manager http://hedgui.com Duct Taped Goat", "followers_count": 6469, "friends_count": 7130, "statues_count": 18030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465693640118272, "text": "I have spoken now I can be free", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1918128505, "name": "Dj Chang3", "screen_name": "djchang3", "lang": "en", "location": "your radio", "create_at": date("2013-09-29"), "description": "#chang3ix #partyWITchang3 DJ on @bigcity1013 with @Tayla_Andre DJ on #BTLS DJ on @bubba_987 DJ on #madmonkeylive for @25btls", "followers_count": 1267, "friends_count": 1075, "statues_count": 3545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrollwood, FL", "id": "5bf268a34de4d516", "name": "Carrollwood", "place_type": "city", "bounding_box": rectangle("-82.543989,28.033407 -82.479582,28.084798") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1210825, "cityName": "Carrollwood" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465694231515139, "text": "�������� https://t.co/KJbJqmb52p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3068994458, "name": "Delg_32", "screen_name": "bambam_0330", "lang": "en", "location": "null", "create_at": date("2015-03-08"), "description": "null", "followers_count": 234, "friends_count": 263, "statues_count": 1231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465694877450240, "text": "I just found out Lockheed Martin has labs all over the mid-west in the middle of nowhere. This is a dream come true.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2220379585, "name": "Alec Ranum", "screen_name": "RanMan_Stallion", "lang": "en", "location": "Rip City, Oregon", "create_at": date("2013-11-28"), "description": "~mortem semper tyrannis~ III% True Born Son of Liberty", "followers_count": 46, "friends_count": 56, "statues_count": 2392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465695263326208, "text": "I'm all for you . always have been and always will be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3218943145, "name": "princess des", "screen_name": "Princessdeesss_", "lang": "en", "location": "null", "create_at": date("2015-05-17"), "description": "null", "followers_count": 377, "friends_count": 473, "statues_count": 6232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465695330467840, "text": "Girls, we run this motherfucking world #FormationWorldTour @Beyonce https://t.co/y5pEjodSma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FormationWorldTour" }}, "user_mentions": {{ 31239408 }}, "user": { "id": 1049358355, "name": "Pay", "screen_name": "payton_steward", "lang": "en", "location": "California", "create_at": date("2012-12-30"), "description": "explore more • ♈️", "followers_count": 620, "friends_count": 295, "statues_count": 3933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-17T00:00:04.000Z"), "id": 732465695372402692, "text": "������#monday #me with my #friends #style #handsome #fancy #nightout #new #music #single #steal #my… https://t.co/64C6iln384", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.06666667,33.45"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "monday", "me", "friends", "style", "handsome", "fancy", "nightout", "new", "music", "single", "steal", "my" }}, "user": { "id": 728285871204487168, "name": "Daniel Levin", "screen_name": "LevinDaniel4", "lang": "en", "location": "null", "create_at": date("2016-05-05"), "description": "null", "followers_count": 16, "friends_count": 120, "statues_count": 3 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-17T00:00:05.000Z"), "id": 732465697196900352, "text": "damn....Detroit's not even on here https://t.co/Nf3LHRHV1w", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 197309394, "name": ".", "screen_name": "yooocole", "lang": "en", "location": "sin city", "create_at": date("2010-09-30"), "description": "album on the way : stxnemf@aol.com", "followers_count": 2163, "friends_count": 1289, "statues_count": 125338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-17T00:00:05.000Z"), "id": 732465697398214656, "text": "Try https://t.co/Mz2UqV6V3o  choice of grilled chicken or pork marinated in Thai seasoning served with #grilled #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.036584,33.605411"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "grilled", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4400, "friends_count": 873, "statues_count": 430231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-17T00:00:05.000Z"), "id": 732465698111246336, "text": "I want you too do me right!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2988357304, "name": "chach", "screen_name": "foreigngalmoni", "lang": "en", "location": " thumbin", "create_at": date("2015-01-20"), "description": "null", "followers_count": 591, "friends_count": 90, "statues_count": 29984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Largo, MD", "id": "19f2fcdf0d209467", "name": "Largo", "place_type": "city", "bounding_box": rectangle("-76.862926,38.866446 -76.802692,38.898322") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445825, "cityName": "Largo" } }
+{ "create_at": datetime("2016-05-17T00:00:05.000Z"), "id": 732465698744635393, "text": "You want a Quick Trip at QT. #LetsBeBadAnd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LetsBeBadAnd" }}, "user": { "id": 3189433165, "name": "Clark Moses", "screen_name": "ScratchycClark", "lang": "en", "location": "null", "create_at": date("2015-05-08"), "description": "null", "followers_count": 447, "friends_count": 507, "statues_count": 7202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465699713515520, "text": "\" tweet & delete \" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 631646086, "name": "Dan〽️afia ❤️", "screen_name": "_1way_arelisha", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "#PalmBeachCounty☀️(561//305) Flomo19✏️---------------------------------❤️ Rip Mommy 8-13-72//10-21-13", "followers_count": 618, "friends_count": 410, "statues_count": 6966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miramar, FL", "id": "5ca2cc5afa0894df", "name": "Miramar", "place_type": "city", "bounding_box": rectangle("-80.40816,25.956799 -80.205968,25.995449") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1245975, "cityName": "Miramar" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465699881287683, "text": "My heart melted. I'm soup. My heart. ��❤️��❤️ https://t.co/1a8WuH4Xst", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 768631656, "name": "shmemily☀️", "screen_name": "Emily_Bonser", "lang": "en", "location": "LCPL E. SHIPLEY❤️", "create_at": date("2012-08-19"), "description": "I say YOLO waaaay more than I should...", "followers_count": 501, "friends_count": 1009, "statues_count": 4999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465699898064896, "text": "Guy & girl got Chin checked at the exact same time lmao https://t.co/6YKoXaOlVp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2914160527, "name": "Justin", "screen_name": "DABBYDABBER", "lang": "en", "location": "Far away from you people ", "create_at": date("2014-11-29"), "description": "Montebello", "followers_count": 650, "friends_count": 416, "statues_count": 10197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465700489416709, "text": "Dead day means being a zombie the night before", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493545877, "name": "Conor Goulart", "screen_name": "ConorGoulart", "lang": "en", "location": "San Jose, CA", "create_at": date("2012-02-15"), "description": "SJSU 2018. Delta Upsilon.", "followers_count": 219, "friends_count": 303, "statues_count": 3782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465700606873601, "text": "Yo boy got data again ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 188093997, "name": "Rmunoz96", "screen_name": "RolandoMunoz_", "lang": "en", "location": "Rancho Cucamonga, CA", "create_at": date("2010-09-07"), "description": "CSUF ⚽️ #14. IG: Rmunoz96 SC:rolly.909", "followers_count": 494, "friends_count": 429, "statues_count": 4890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465700778868736, "text": "And it's 2 a.m... ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2189471155, "name": "nolachick", "screen_name": "jenn_fagan", "lang": "en", "location": "New Orleans, Louisiana", "create_at": date("2013-11-11"), "description": "null", "followers_count": 107, "friends_count": 188, "statues_count": 2297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465701294739456, "text": "This is too much �� idk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261000923, "name": "Bryan", "screen_name": "Brycemelts", "lang": "en", "location": "null", "create_at": date("2011-03-04"), "description": "~~~~~~~CEO of There4U ➖Partner with G3 ~~~~~~~ 18 ➖ California☀️ ➖ Comedian Email:Brycemelts16@gmail.com (G3)Email:geraldreader3@gmail.com", "followers_count": 369, "friends_count": 360, "statues_count": 6287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465701462495232, "text": "Me: Gets another sucker to watch Person of Interest.\nMe: HahahahaHAHAH, welcome to pain.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41510679, "name": "arya", "screen_name": "catyblanket", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-20"), "description": "where do you go when you go quiet?", "followers_count": 321, "friends_count": 231, "statues_count": 39630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465701701570561, "text": "@Fuentez_06 right. Back at you ��", "in_reply_to_status": 732465150578458626, "in_reply_to_user": 2316472118, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2316472118 }}, "user": { "id": 148595642, "name": "daddy senpai", "screen_name": "jjb_96", "lang": "en", "location": "California, USA", "create_at": date("2010-05-26"), "description": "currently in:", "followers_count": 820, "friends_count": 317, "statues_count": 23407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465701714190336, "text": "YESSSSSSSSS! ❤️������ https://t.co/Li5Mccq4Pu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1839788586, "name": "rae", "screen_name": "dutchessdes_", "lang": "en", "location": "null", "create_at": date("2013-09-08"), "description": "#CAKETEAM", "followers_count": 1825, "friends_count": 1530, "statues_count": 89634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465701772886016, "text": "I dance all the time �� https://t.co/dYnNmVzFxH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232297031, "name": "Jay", "screen_name": "juliesaewut", "lang": "en", "location": "Fairfield, CA", "create_at": date("2010-12-30"), "description": "Rest In Peace Mama ❤️ #FUCKCANCER 05/03/73 - 02/23/16", "followers_count": 1253, "friends_count": 605, "statues_count": 87658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465702502703105, "text": "1:59 am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 540405575, "name": "TierraHarper", "screen_name": "TierraHarper", "lang": "en", "location": "Los Angeles, California ", "create_at": date("2012-03-29"), "description": "UCLA19' | I do what I want | Un jour je vais vous épargner SC✈️ATL✈️LA #HunterCorpis ♈ What are you waiting for b.?", "followers_count": 1131, "friends_count": 944, "statues_count": 22760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465702565646338, "text": "No plain tees & Jays �� #GermanBreedsMetGala2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GermanBreedsMetGala2016" }}, "user": { "id": 219738790, "name": "✨Your King✨", "screen_name": "ItsNelson_", "lang": "en", "location": "Richmond Va , tf", "create_at": date("2010-11-25"), "description": "⚪️Hi Im Nel! & ⚫️Im Nick!", "followers_count": 1430, "friends_count": 1776, "statues_count": 50894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, VA", "id": "00f751614d8ce37b", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-77.601044,37.447046 -77.385297,37.61272") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51760, "countyName": "Richmond", "cityID": 5167000, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465702653698048, "text": "Wind 0.0 mph ---. Barometer 30.056 in, Steady. Temperature 31.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10778 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-17T00:00:06.000Z"), "id": 732465703186358272, "text": "@kikonetics @pos_pisces I feel like I'm doing this wrong.", "in_reply_to_status": 732453129027903488, "in_reply_to_user": 613367598, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 613367598, 3099120002 }}, "user": { "id": 304064187, "name": "Hitachi Magical Girl", "screen_name": "kristenjori", "lang": "en", "location": "null", "create_at": date("2011-05-23"), "description": "lil pinky poo", "followers_count": 209, "friends_count": 116, "statues_count": 4001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828062736338944, "text": "I. Am. So. Embarrassed. https://t.co/kEuRiGfvzE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 266786493, "name": "wiggly woo patricia", "screen_name": "notpatriciaa", "lang": "en", "location": "grey sloan memorial", "create_at": date("2011-03-15"), "description": "probably watching age of Ultron or crying over Elizabeth Olsen and Ian Bohen", "followers_count": 148, "friends_count": 161, "statues_count": 25238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rhome, TX", "id": "00799f7d3a025a10", "name": "Rhome", "place_type": "city", "bounding_box": rectangle("-97.478507,33.026269 -97.435664,33.058468") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48497, "countyName": "Wise", "cityID": 4861700, "cityName": "Rhome" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828063168397313, "text": "Not sure what's worse: the unwanted bad thoughts or the unwanted good memories.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433101913, "name": "Nick Fedora", "screen_name": "FedoraAudio", "lang": "en", "location": "Los Angeles, CA, United States", "create_at": date("2011-12-09"), "description": "Foley mixer. TV enthusiast. Melodic death metal headbanger. I let my ears do the talking.", "followers_count": 276, "friends_count": 465, "statues_count": 7075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828064493768705, "text": "can't be friends with somebody you used to like . Don't matter how many years ago. Impossible , trust me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 281695621, "name": "kiddkidd", "screen_name": "TrvpGoddJackson", "lang": "en", "location": "null", "create_at": date("2011-04-13"), "description": "I don't really use Twitter but follow me anyways", "followers_count": 799, "friends_count": 523, "statues_count": 3626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828064690929664, "text": "@ebbtideapp Tide in Murderkill River entrance, Delaware 05/18/2016\nHigh 7:51am 4.8\n Low 2:22pm 0.3\nHigh 8:16pm 5.6\n Low 2:59am 0.3", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.3967,39.0583"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 33730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10001, "countyName": "Kent", "cityID": 1007250, "cityName": "Bowers" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828065064194049, "text": "@allisonmak I was there when he said that omg", "in_reply_to_status": 732819877610815489, "in_reply_to_user": 29638111, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 29638111 }}, "user": { "id": 81687990, "name": "wendy", "screen_name": "frootqueen", "lang": "en", "location": "los angeles", "create_at": date("2009-10-11"), "description": "I need to go to a Walgreens", "followers_count": 553, "friends_count": 352, "statues_count": 16434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828065454260224, "text": "@i0n1c The men in black are going to come for you.", "in_reply_to_status": 732827470500663296, "in_reply_to_user": 16365707, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16365707 }}, "user": { "id": 114025166, "name": "JayFame", "screen_name": "JayFameSodmg", "lang": "en", "location": "1625", "create_at": date("2010-02-13"), "description": "1988 CR-X #BSerious", "followers_count": 1787, "friends_count": 228, "statues_count": 6283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethlehem, PA", "id": "128ae72e3854b273", "name": "Bethlehem", "place_type": "city", "bounding_box": rectangle("-75.4314,40.578043 -75.302993,40.672508") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42095, "countyName": "Northampton", "cityID": 4206088, "cityName": "Bethlehem" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828065546555393, "text": "I wish I had more wine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 146598071, "name": "martinezabdias", "screen_name": "martinezabdias", "lang": "en", "location": "Denton, TX", "create_at": date("2010-05-21"), "description": "#UNT17 | Psychology/Music | Photography | Y&F | Jesus | 1994 | Houston-Denton |", "followers_count": 252, "friends_count": 230, "statues_count": 13537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828065945030656, "text": "RFT https://t.co/7wFbVFaqDC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2227934072, "name": "adg", "screen_name": "AriGee32", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "let's be best friends. maybe even on snapchat too: ari_gee32", "followers_count": 383, "friends_count": 320, "statues_count": 6066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-18T00:00:00.000Z"), "id": 732828066477658112, "text": "idgi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483435087, "name": "cur$ton", "screen_name": "curscurs", "lang": "en", "location": "Houston, TX", "create_at": date("2012-02-04"), "description": "¯\\_(ツ)_/¯", "followers_count": 1168, "friends_count": 680, "statues_count": 65445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828066624483328, "text": "I'm happy asf rn ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1691369947, "name": "Daddy ⭐️", "screen_name": "Pappi_Luiss", "lang": "en", "location": "null", "create_at": date("2013-08-22"), "description": "Los Angeles , CA", "followers_count": 283, "friends_count": 171, "statues_count": 19968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067056504832, "text": "My dms been popping but it's nothing I'm not thinking about non of these niggaz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397293845, "name": "Debbie", "screen_name": "RipNylaDaddy", "lang": "en", "location": "I'm always on somebody mind", "create_at": date("2011-10-24"), "description": "followmy snap cakezz1", "followers_count": 1561, "friends_count": 1566, "statues_count": 36356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067182354433, "text": "Lol life. Minoring in business or city planning thoughts.... guaaaaat has Berkeley done to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179824215, "name": "Samantha", "screen_name": "HolySammerzz", "lang": "en", "location": "null", "create_at": date("2010-08-17"), "description": "UC Berkeley '19", "followers_count": 237, "friends_count": 103, "statues_count": 25027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067236831235, "text": "Aside from all the playing & games I really do give best advice ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1913074034, "name": "☄TRひE☄", "screen_name": "AyDuwop", "lang": "en", "location": "null", "create_at": date("2013-09-27"), "description": "null", "followers_count": 1073, "friends_count": 971, "statues_count": 18219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067320717312, "text": "Fight the urge to watch the vampire diaries rn I need to go to sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331393476, "name": "theDoll", "screen_name": "glitzydoll_", "lang": "en", "location": "null", "create_at": date("2011-07-07"), "description": "with every new day comes new thoughts and new strength", "followers_count": 800, "friends_count": 351, "statues_count": 57320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Batavia, NY", "id": "4c794ce21ef49219", "name": "Batavia", "place_type": "city", "bounding_box": rectangle("-78.265094,42.980725 -78.115968,43.022344") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36037, "countyName": "Genesee", "cityID": 3604715, "cityName": "Batavia" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067354279937, "text": "@krissssy__ ������", "in_reply_to_status": 732825073955000320, "in_reply_to_user": 420569236, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 420569236 }}, "user": { "id": 59744406, "name": "Gustavo Hernandez", "screen_name": "GusHernandez25", "lang": "en", "location": "Whittier,Ca", "create_at": date("2009-07-24"), "description": "Handy Man ♋️", "followers_count": 22, "friends_count": 68, "statues_count": 493 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067383652352, "text": "Temp 50.7° Hi/Lo 52.1/50.5 Rng 1.6° WC 50.7° Hmd 71% Rain 0.00\" Storm 0.00\" BAR 30.114 Rising DP 41.6° Wnd 0mph Dir --- Gst 4mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067396255744, "text": "05/18@03:00 - Temp 50.4F, WC 50.4F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.136in, Falling slowly. Rain 0.00in. Hum 98%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067589173248, "text": "Going to NWCTA https://t.co/Tb7fJTVvhD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2758076515, "name": "Allyssa Fernandez", "screen_name": "allyssastayceee", "lang": "en", "location": "LV, NV ", "create_at": date("2014-08-23"), "description": "Jeremiah 29:13 ♡ // NWCTA '17", "followers_count": 245, "friends_count": 220, "statues_count": 2445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828067748515840, "text": "@LightningEyess I saw him when we first checked in", "in_reply_to_status": 732827665795850240, "in_reply_to_user": 553305409, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 553305409 }}, "user": { "id": 2924390209, "name": "steph", "screen_name": "Stefuunnnyyyy", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "¯\\_(ツ)_/¯", "followers_count": 88, "friends_count": 95, "statues_count": 6828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068000235520, "text": "Wind 2.0 mph NE. Barometer 1024.69 mb, Steady. Temperature 51.1 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068008632320, "text": "@MBambam26 don't get anyone pregnant cause we will fire you ����", "in_reply_to_status": 732826666175766528, "in_reply_to_user": 357304927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 357304927 }}, "user": { "id": 2239705963, "name": "Evangelina B.", "screen_name": "evabote", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-12-10"), "description": "Applied Biological Science Major at Arizona State University ☀️", "followers_count": 227, "friends_count": 212, "statues_count": 7711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.62655,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068088270850, "text": "I found out my biggest fear", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2732838376, "name": "poppa❤", "screen_name": "nyalieplam", "lang": "en", "location": "taking a shot w/ ya nigga ", "create_at": date("2014-08-03"), "description": "|big booties only, AZ| Snapchat me Nyaliep01", "followers_count": 793, "friends_count": 442, "statues_count": 25143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068105093120, "text": "Kapau pe nate fanogo ka mum mo daddy pehe ni oku te struggle ka ku te fiefia pe, he taimi ni te struggle pe moe fuhinga e stress ke fefe'i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 106600229, "name": "iko", "screen_name": "ikoito", "lang": "en", "location": "Sopu | Nukualofa | Tonga |", "create_at": date("2010-01-19"), "description": "Alu koe kihe tweeter eh :)", "followers_count": 199, "friends_count": 128, "statues_count": 3601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menlo Park, CA", "id": "490bdb082950484f", "name": "Menlo Park", "place_type": "city", "bounding_box": rectangle("-122.228922,37.416515 -122.120415,37.507328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 646870, "cityName": "Menlo Park" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068339974144, "text": "@_Johnny9 @Bea5tMANROY yeah lol", "in_reply_to_status": 732827967781502978, "in_reply_to_user": 574236186, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 574236186, 184662739 }}, "user": { "id": 36222290, "name": "turtle", "screen_name": "_Naancy7", "lang": "en", "location": "null", "create_at": date("2009-04-28"), "description": "C.J.M. ❤️ reflection & progression~", "followers_count": 559, "friends_count": 321, "statues_count": 23652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aromas, CA", "id": "f6b73dc3f843fd1b", "name": "Aromas", "place_type": "city", "bounding_box": rectangle("-121.668764,36.860171 -121.613169,36.895721") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 602812, "cityName": "Aromas" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828068490928128, "text": "why am I awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2983605986, "name": "jenna", "screen_name": "jennaabaker_", "lang": "en", "location": "null", "create_at": date("2015-01-14"), "description": "null", "followers_count": 346, "friends_count": 336, "statues_count": 1027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davis, CA", "id": "1994142e26ba7127", "name": "Davis", "place_type": "city", "bounding_box": rectangle("-121.803252,38.526843 -121.675074,38.590264") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 618100, "cityName": "Davis" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828069199806465, "text": "I will let my imperfections make me greater https://t.co/0cDe47TZ7H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 284765140, "name": "Hannah Larson", "screen_name": "jambee1997", "lang": "en", "location": "Anaheim, CA", "create_at": date("2011-04-19"), "description": "PHS Encore soon HIU student", "followers_count": 236, "friends_count": 990, "statues_count": 1820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828069266923522, "text": "AYY ITS MY BIRTHDAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 896337979, "name": "laila", "screen_name": "purpleveIvet", "lang": "en", "location": "SAN FRANCISCO", "create_at": date("2012-10-21"), "description": "counting my eye rolls", "followers_count": 6920, "friends_count": 96, "statues_count": 66867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828069304672257, "text": "So it's a new game everybody playing on Twitter ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 275738824, "name": "T U R K", "screen_name": "_liveforQ", "lang": "en", "location": "null", "create_at": date("2011-04-01"), "description": "IG:_jxsturk snap:bitchimt-tom", "followers_count": 563, "friends_count": 928, "statues_count": 10999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828069535318016, "text": "@Escobedovic15 what Vic", "in_reply_to_status": 732827707088719873, "in_reply_to_user": 586159738, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 586159738 }}, "user": { "id": 184425093, "name": "Amanda", "screen_name": "Amanda_Ayon", "lang": "en", "location": "California, USA", "create_at": date("2010-08-29"), "description": "what would tupac do?", "followers_count": 442, "friends_count": 332, "statues_count": 32867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828069589848068, "text": "Wind 0.0 mph ---. Barometer 29.892 in, Falling. Temperature 64.9 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070042816514, "text": "How IoT security can benefit from machine learning https://t.co/mO5TaHrwd3 @IoT_SF @iotsecurity2 @IoTSecurityHub @IoTSecurity #IoTSecurity", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IoTSecurity" }}, "user_mentions": {{ 3334685577, 701072251974619136, 2885189111, 1173325098 }}, "user": { "id": 254426733, "name": "Muddu Sudhakar", "screen_name": "smuddu", "lang": "en", "location": "Palo Alto, CA", "create_at": date("2011-02-18"), "description": "VP & GM Splunk Security;Cyber Security, Insider Threats;ML,Behavior Analytics,Data Science,Big Data Analytics;Serial entrepreneur. Worked at VMware, EMC.", "followers_count": 793, "friends_count": 271, "statues_count": 4176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070080610307, "text": "When mfs who got more followers than you call you \"famous\" bihhhh where ? ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3305456138, "name": "ImerzFoesquad", "screen_name": "imerzbraah", "lang": "en", "location": "Oakland, CA", "create_at": date("2015-08-03"), "description": "P h E e N ❤️ #Raider Nation SC:@Imerzbaby", "followers_count": 326, "friends_count": 188, "statues_count": 13431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070147690497, "text": "A few more minutes to #Wednesday... Gratitude for what & who I have while working for what I want to achieve ~ https://t.co/U0gH1EouJB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Wednesday" }}, "user": { "id": 2313938278, "name": "Lari Umeno", "screen_name": "LariUmeno", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-01-30"), "description": "Let's create! #Entrepreneur #SocialMedia, #Art #ProjectManager ❣~ ChocoLari Strategy #Journalist from #Brazil ✨ FB'IG @ chocoLari.it #LoveistheKeytoSuccess 3°M", "followers_count": 815, "friends_count": 815, "statues_count": 4056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070466473984, "text": "Ripley SW Limestone Co. Temp: 69.6°F Wind:1.6mph Pressure: 993.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070483267584, "text": "This should be simple and lengthy at the same time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 740877866, "name": "King Condescending", "screen_name": "whizkhidK", "lang": "en", "location": "In Ya Business", "create_at": date("2012-08-06"), "description": "Full time ass Part time friend if you want to know more about me please contact my management team at @whizkhidK #StopFetishizingLatinaWomen", "followers_count": 470, "friends_count": 199, "statues_count": 89868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Memphis, AR", "id": "017db07b3676900f", "name": "West Memphis", "place_type": "city", "bounding_box": rectangle("-90.237784,35.120479 -90.121655,35.231923") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5035, "countyName": "Crittenden", "cityID": 574540, "cityName": "West Memphis" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070554574849, "text": "Still up ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1009016924, "name": "lexxx taylor.", "screen_name": "lexxbrooke20", "lang": "en", "location": "Texas, USA", "create_at": date("2012-12-13"), "description": "null", "followers_count": 570, "friends_count": 328, "statues_count": 9328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownfield, TX", "id": "f7bb6752fbd343e9", "name": "Brownfield", "place_type": "city", "bounding_box": rectangle("-102.309211,33.146988 -102.244024,33.202964") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48445, "countyName": "Terry", "cityID": 4810720, "cityName": "Brownfield" } }
+{ "create_at": datetime("2016-05-18T00:00:01.000Z"), "id": 732828070646820865, "text": "Temp: 69.0°F Wind:0.0mph Pressure: 29.926hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828070881726464, "text": "Make sure your perfect before you start judging me dumb bitch.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4227449054, "name": "Iv", "screen_name": "Ivleaaao", "lang": "en", "location": "null", "create_at": date("2015-11-13"), "description": "null", "followers_count": 43, "friends_count": 38, "statues_count": 645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828070978195456, "text": "Wind 2.0 mph NE. Barometer 30.103 in, Falling slowly. Temperature 49.5 °F. Rain today 0.06 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828071393415170, "text": "FYI I freestyled that lol", "in_reply_to_status": 732827982092505089, "in_reply_to_user": 302848865, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 302848865, "name": "keezus", "screen_name": "sirMARKalot_", "lang": "en", "location": "Lafayette, LA", "create_at": date("2011-05-21"), "description": "(Marr-Kee-Ven) . #UL19", "followers_count": 558, "friends_count": 260, "statues_count": 20810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828071397601280, "text": "Actually, BronyCon was mentioned twice ���������� #markhenry #worldsstrongestkid https://t.co/Gy10H0MFLS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "markhenry", "worldsstrongestkid" }}, "user": { "id": 145128897, "name": "Evan Michael Lee", "screen_name": "Evanmichaellee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-17"), "description": "THE OFFICIAL VOICE OF @themarkhenry ON #CAMPWWE ONLY ON @wwenetwork #MarkHenry #WorldsStrongestKid", "followers_count": 808, "friends_count": 315, "statues_count": 9900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828071582142464, "text": "I go through stages... I'm hella social or I hate everyone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 253788859, "name": "princess", "screen_name": "DrownInMySea", "lang": "en", "location": "Portland, OR", "create_at": date("2011-02-17"), "description": "null", "followers_count": 651, "friends_count": 133, "statues_count": 34417 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828071804469248, "text": "I'm Eating a lollipop and its 12:00 my fat ass needs to go to sleep ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3128071040, "name": "cristina ✨", "screen_name": "youiskarma", "lang": "en", "location": "Kingsburg, CA", "create_at": date("2015-04-01"), "description": "✌️", "followers_count": 322, "friends_count": 237, "statues_count": 3997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsburg, CA", "id": "355563f41b5729de", "name": "Kingsburg", "place_type": "city", "bounding_box": rectangle("-119.590867,36.488811 -119.529959,36.546995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 638562, "cityName": "Kingsburg" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828072211275776, "text": "I JUST REMEMBERED I HAVE 3 DIFFERENT TEST/DISCUSSIONS FOR ONE OF MY CLASSES DUE TONIGHT AND ITS 11:59 HAHAHAHAHA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31048077, "name": "brandi, the creator", "screen_name": "braandeee", "lang": "en", "location": "hollyHOOD", "create_at": date("2009-04-13"), "description": "cats on everything. es❤️", "followers_count": 606, "friends_count": 337, "statues_count": 36626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828072379060224, "text": "She don't know no better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2937080397, "name": "May19th", "screen_name": "fxckquisee", "lang": "en", "location": "Ft.Worth Tx,", "create_at": date("2014-12-22"), "description": "|God1st|TJCFootball|", "followers_count": 1589, "friends_count": 780, "statues_count": 37005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828073364688897, "text": "Wind 0.0 mph ---. Barometer 30.188 in, Steady. Temperature 49.2 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828073574404096, "text": "70.8F (Feels: 70.8F) - Humidity: 93% - Wind: 2.2mph NE - Gust: 4.5mph - Pressure: 1010.8mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 240934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828073775747073, "text": "Warriors would lose tomorrow and fuck around and still win the series doe lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 269200310, "name": "JayZOverrated", "screen_name": "JayZOverrated", "lang": "en", "location": "Louisiana, USA", "create_at": date("2011-03-20"), "description": "null", "followers_count": 9056, "friends_count": 849, "statues_count": 344044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayou Cane, LA", "id": "b5b20fbf38f349af", "name": "Bayou Cane", "place_type": "city", "bounding_box": rectangle("-90.810681,29.591044 -90.691976,29.661513") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2205210, "cityName": "Bayou Cane" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828073918353408, "text": "same me af https://t.co/vbEueopV7H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 211756517, "name": "Marco Carvajal", "screen_name": "carbeezy_", "lang": "en", "location": "Cali Dreamin'", "create_at": date("2010-11-03"), "description": "fuck ya mean?", "followers_count": 312, "friends_count": 258, "statues_count": 33537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828073935130625, "text": "HAPPY BIRTHDAY! you are one of the best and worst thing that has happened to me love ya you made me who I am today❣ https://t.co/Bocbyx3nhA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3338965274, "name": "j", "screen_name": "alien_mom99", "lang": "en", "location": "CA", "create_at": date("2015-08-25"), "description": "jesus is my homie// snap: jess-dixon", "followers_count": 795, "friends_count": 358, "statues_count": 1987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828074019020800, "text": "Some People just have the \"Will\" to say \"Fuck this Shit\" and some just \"Don't\".", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302305526, "name": "Pamela", "screen_name": "PearlyWhiteLegs", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-07-31"), "description": "Live Your Life with Volition and Passion and if Nobody Get's it, Just Fuck Em All the Same. Always, Pamela", "followers_count": 183, "friends_count": 632, "statues_count": 122 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828074153279488, "text": "How To Love forever goat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1478539256, "name": "No Chill", "screen_name": "E_sayud", "lang": "en", "location": "null", "create_at": date("2013-06-02"), "description": "You can't spell healTHCare without THC", "followers_count": 752, "friends_count": 551, "statues_count": 56833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828074182627328, "text": "For whatever mood you're in, Logic is the way to go. ������@Logic301", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 141944292 }}, "user": { "id": 1445461710, "name": "Johnny", "screen_name": "johnnyromero_", "lang": "en", "location": "null", "create_at": date("2013-05-20"), "description": "null", "followers_count": 121, "friends_count": 140, "statues_count": 2211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828074186805248, "text": "Pls leave my emotions out of it #summer16@motto", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "summer16" }}, "user": { "id": 387991883, "name": "alexis rose", "screen_name": "AlexisCanon", "lang": "en", "location": "futures baby", "create_at": date("2011-10-09"), "description": "I do what I want", "followers_count": 1535, "friends_count": 584, "statues_count": 47310 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oswego, IL", "id": "6bbfc293278cafd1", "name": "Oswego", "place_type": "city", "bounding_box": rectangle("-88.410765,41.656926 -88.261472,41.72409") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17093, "countyName": "Kendall", "cityID": 1756887, "cityName": "Oswego" } }
+{ "create_at": datetime("2016-05-18T00:00:02.000Z"), "id": 732828074698506240, "text": "@StarBoy_Zoe shut up ��", "in_reply_to_status": 732826894618529792, "in_reply_to_user": 270182056, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 270182056 }}, "user": { "id": 1942761260, "name": "TR€〽️", "screen_name": "TPetit5", "lang": "en", "location": " P.o.l.o.", "create_at": date("2013-10-06"), "description": "19. jenks Alumni :Oklahoma Snapchat:Tre_petit3 insta:TPetit5 NOC jets✈️.... slow feet don't eat", "followers_count": 828, "friends_count": 635, "statues_count": 7119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jenks, OK", "id": "3c94993705f01d36", "name": "Jenks", "place_type": "city", "bounding_box": rectangle("-96.030986,35.944309 -95.922323,36.048641") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4037800, "cityName": "Jenks" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828075512213504, "text": "only thing ive been doing since school got out is watching netflix & finishing alot of TV show series������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3294274333, "name": "haley w.", "screen_name": "HaleyWoodsss", "lang": "en", "location": "kotzebue AK", "create_at": date("2015-07-25"), "description": "no emotion$ but good intention$", "followers_count": 61, "friends_count": 42, "statues_count": 38 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kotzebue, AK", "id": "475d00c992d710e1", "name": "Kotzebue", "place_type": "city", "bounding_box": rectangle("-162.622633,66.837633 -162.523151,66.90436") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2188, "countyName": "Northwest Arctic", "cityID": 241830, "cityName": "Kotzebue" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828076078424066, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":50.5°F Wind:0.0mph Pressure: 30.03hpa Falling Rain Today 0.13in. Forecast: Showery, becoming less s", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 17, "statues_count": 318166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828076225204224, "text": "What if he butterscotch?�� https://t.co/GkjJxpioS0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 581625339, "name": "Skyler Thomas", "screen_name": "Trienta_Nueve", "lang": "en", "location": "Riverside, CA", "create_at": date("2012-05-15"), "description": "|Blessed|Washington State University⭐️|", "followers_count": 2229, "friends_count": 1002, "statues_count": 9545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828076317519873, "text": "Wind 0.0 mph NW. Barometer 30.17 in, Steady. Temperature 43.0 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159874 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828076523032576, "text": "We can work from home.����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 123068033, "name": "aThotsThoughts", "screen_name": "WyleeCoyote", "lang": "en", "location": "Las Vegasss", "create_at": date("2010-03-14"), "description": "Aries 4 life! ♈️ I'm Marceline the Vampire Queen. I'm 22. I write music.", "followers_count": 638, "friends_count": 575, "statues_count": 18973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077013733376, "text": "@Isabellaa_Marie ok fine u have one week to be emotional then I'm home!!", "in_reply_to_status": 732825592329084928, "in_reply_to_user": 398496824, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 398496824 }}, "user": { "id": 219914009, "name": "Elli Giacomazzi ♡", "screen_name": "ElliGiacomazzi", "lang": "en", "location": "Auburn University", "create_at": date("2010-11-25"), "description": "fashionably late", "followers_count": 695, "friends_count": 395, "statues_count": 28162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lahaina, HI", "id": "249325c821f8b89a", "name": "Lahaina", "place_type": "city", "bounding_box": rectangle("-156.690186,20.855885 -156.657928,20.941656") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1542950, "cityName": "Lahaina" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077210882049, "text": "Be careful Hillary. Get your house straight before worrying about Donald Trump's personal life. Remember Monica? https://t.co/KHuWICOOwk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99296718, "name": "S/Sergeant Dias usmc", "screen_name": "SergeantDias", "lang": "en", "location": "North Carolina, USA", "create_at": date("2009-12-25"), "description": "For God and Country. (Conservative/Catholic) GOD BLESS AMERICA, OUR TROOPS AND GOD'S CHOSEN PEOPLE - ISRAEL", "followers_count": 195, "friends_count": 354, "statues_count": 6960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077311545344, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1600045994, "name": "Francois", "screen_name": "_idgaf_22", "lang": "en", "location": "null", "create_at": date("2013-07-16"), "description": "snapchat idgaf_22", "followers_count": 300, "friends_count": 173, "statues_count": 24889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077403852800, "text": "K", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2266983679, "name": "SDG", "screen_name": "sam_gradias", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-12-28"), "description": "King volleyball #12 class of 2016 stop simpin, and start pimpin/ Future lope/FIJI boys/KHS VOLLEYBALLERZ", "followers_count": 405, "friends_count": 390, "statues_count": 4487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077403873280, "text": "@redcandy89 don't be mean", "in_reply_to_status": 732827368402898944, "in_reply_to_user": 863910902, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 863910902 }}, "user": { "id": 25956454, "name": "Ant Beezy", "screen_name": "AntBeezy21", "lang": "en", "location": "Azusa/Pasadena, CA", "create_at": date("2009-03-22"), "description": "Audio Engineer • Anime Lover • Gamer • Pokemon Master • Skateboarder • Dena Love • Ricktatorship • Diamond Dogs • 2Eight Radio", "followers_count": 539, "friends_count": 765, "statues_count": 42896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, CA", "id": "10de09f288b1665c", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-118.198346,34.117025 -118.065582,34.237595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077542256640, "text": "I Give My Bestfriend Anything She Ask For....ANYTHING �� https://t.co/oN9Jeg2eUC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547815709, "name": "Reloaded..☹️", "screen_name": "GlobalGleesh_", "lang": "en", "location": "W. Your Mom", "create_at": date("2012-04-07"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀#ripQuette❤️⠀SC: Global.Gleesh", "followers_count": 770, "friends_count": 314, "statues_count": 5199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillcrest Heights, MD", "id": "ce454cbdbf5f9602", "name": "Hillcrest Heights", "place_type": "city", "bounding_box": rectangle("-76.9873,38.822422 -76.94524,38.854875") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2438975, "cityName": "Hillcrest Heights" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077672304640, "text": "@wyclef #mygirlshoptalk #shoptalk16 ���������� @ Marquee Las Vegas https://t.co/cEKcTxi3Bj", "in_reply_to_status": -1, "in_reply_to_user": 20659839, "favorite_count": 0, "coordinate": point("-115.17406762,36.10941917"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "mygirlshoptalk", "shoptalk16" }}, "user_mentions": {{ 20659839 }}, "user": { "id": 300884497, "name": "Fabian A. Illanes", "screen_name": "KaoKenAtak", "lang": "en", "location": "Austin, TX", "create_at": date("2011-05-18"), "description": "Founder & CTO @ReadyToGoSquad with @RomanEmpireZ | 10 yr IT professional w/ backgrounds in beer drinking and snowboarding | Disaster preparedness advocate #NYC", "followers_count": 71, "friends_count": 75, "statues_count": 385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077676474371, "text": "INSBBCIML", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2290415514, "name": "tyra bank$", "screen_name": "hboogie1x", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-13"), "description": "I have freckles & I'll probably dunk on you", "followers_count": 1357, "friends_count": 433, "statues_count": 56054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828077726781440, "text": "If you're a #Healthcare professional in #Erie, PA, check out this #job: https://t.co/4L0PxVJAek #Hiring https://t.co/62TXDLKkul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.0939978,42.1188322"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "Erie", "job", "Hiring" }}, "user": { "id": 2587789764, "name": "SHC Careers", "screen_name": "WorkWithSHC", "lang": "en", "location": "null", "create_at": date("2014-06-25"), "description": "Work for the Best! Whether you want to work across town or across the country, we have thousands of great health care jobs available at top facilities.", "followers_count": 953, "friends_count": 1, "statues_count": 111849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Erie, PA", "id": "29aaa88d9fe74b50", "name": "Erie", "place_type": "city", "bounding_box": rectangle("-80.239991,42.018414 -79.934073,42.202992") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4224000, "cityName": "Erie" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828078125285377, "text": "They're pretty great :') (even Carlos) https://t.co/pxgAH6ftwQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1366238000, "name": "Kal-el", "screen_name": "x_miggle", "lang": "en", "location": "Krypton", "create_at": date("2013-04-19"), "description": "to the edge of the universe and back #Tamu19", "followers_count": 583, "friends_count": 572, "statues_count": 19829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828078418874368, "text": "26857c9700p336F3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.651773,51.575444"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 21, "friends_count": 0, "statues_count": 22548 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Cochrane, Unorganized, North Part, Ontario", "id": "327b6738dbc4853b", "name": "Cochrane, Unorganized, North Part", "place_type": "city", "bounding_box": rectangle("-86.550032,48.275084 -79.515795,52.293301") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-18T00:00:03.000Z"), "id": 732828078632800256, "text": "Between the Bars x Elliott Smith", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 850048135, "name": "Kennedy", "screen_name": "Kennedyireland_", "lang": "en", "location": "TBHS", "create_at": date("2012-09-27"), "description": "You'd lose your mind trying to understand mine", "followers_count": 1127, "friends_count": 598, "statues_count": 21914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828079295500289, "text": "@Ceezy_Taught_U Maybe��", "in_reply_to_status": 732827950274482176, "in_reply_to_user": 11230892, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 11230892 }}, "user": { "id": 1064221088, "name": "AR-EB.", "screen_name": "amazinBlasian_", "lang": "en", "location": "null", "create_at": date("2013-01-05"), "description": "unfriendly black hottie.................. snapchat ⇝ AR.EB", "followers_count": 1190, "friends_count": 1285, "statues_count": 29724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlantic City, NJ", "id": "7ad0e3081108f4ba", "name": "Atlantic City", "place_type": "city", "bounding_box": rectangle("-74.467407,39.342291 -74.40702,39.386729") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3402080, "cityName": "Atlantic City" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828079303892992, "text": "Finally got all the babies home ❤️ https://t.co/FjbT9rGyO9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2600490240, "name": "Maggie", "screen_name": "Maaaggiebeck", "lang": "en", "location": "Chico, CA", "create_at": date("2014-07-02"), "description": "All is fair in love and war.", "followers_count": 244, "friends_count": 371, "statues_count": 6025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chico, CA", "id": "9b381228ebd9a499", "name": "Chico", "place_type": "city", "bounding_box": rectangle("-121.919174,39.676071 -121.748286,39.818003") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 613014, "cityName": "Chico" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828079379361792, "text": "@KThomasDC @Marv_Vien @AP she won 25 in Kentucky. How many in Utah?", "in_reply_to_status": 732785733438722048, "in_reply_to_user": 15463486, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15463486, 22454663, 51241574 }}, "user": { "id": 701121977340264449, "name": "Suzie4Hillary4POTUS", "screen_name": "Suzie4Hillary4A", "lang": "en", "location": "null", "create_at": date("2016-02-20"), "description": "Class of '65. Nanabanana All Hillary and related topics. Mom, Nana, major Dog Rescue Addiction", "followers_count": 523, "friends_count": 467, "statues_count": 13083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandy Springs, GA", "id": "b077585758242faa", "name": "Sandy Springs", "place_type": "city", "bounding_box": rectangle("-84.4468,33.876737 -84.258089,34.010177") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1368516, "cityName": "Sandy Springs" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828079907803136, "text": "@eIIsberry https://t.co/WbIt8T3bfw", "in_reply_to_status": 732827306641772547, "in_reply_to_user": 710685661125341184, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 710685661125341184 }}, "user": { "id": 405961209, "name": "KC ellsberry", "screen_name": "BasedxWorld", "lang": "en", "location": "null", "create_at": date("2011-11-05"), "description": "All ya motherfuckers talk about Jeff Teague ain’t no hitta Jeff Teague ain’t this Jeff Teague a fake SHUT THE FUCK UP Y'all don’t live with that nigga Y'all kno", "followers_count": 1161, "friends_count": 369, "statues_count": 60142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Point, GA", "id": "dfb4df427ea8a2d6", "name": "East Point", "place_type": "city", "bounding_box": rectangle("-84.535534,33.609487 -84.415048,33.705685") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1325720, "cityName": "East Point" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828080075640832, "text": "Everytime I order something online I have it gift wrapped with a note to myself �� \n#IHaveProblems", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "IHaveProblems" }}, "user": { "id": 298350314, "name": "Jaymee", "screen_name": "Jaymee_04", "lang": "en", "location": "Freer, TX", "create_at": date("2011-05-13"), "description": "I really love naps. SC: jaymee04", "followers_count": 1226, "friends_count": 364, "statues_count": 31246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freer, TX", "id": "1faf8f69f9254e17", "name": "Freer", "place_type": "city", "bounding_box": rectangle("-98.630077,27.857452 -98.60165,27.887479") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48131, "countyName": "Duval", "cityID": 4827432, "cityName": "Freer" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828080352419840, "text": "@Vote4Pedro13 ��", "in_reply_to_status": 732824919587885056, "in_reply_to_user": 324190100, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 324190100 }}, "user": { "id": 323646819, "name": "Horse Eater", "screen_name": "NerdBeebe", "lang": "en", "location": "Riverside, CA", "create_at": date("2011-06-24"), "description": "Denny's VIP | English / 漢語 / Junior Member of the Zissou Society / Future King of Beebe, Arkansas", "followers_count": 110, "friends_count": 69, "statues_count": 3418 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winnemucca, NV", "id": "807efdfb21bfb93f", "name": "Winnemucca", "place_type": "city", "bounding_box": rectangle("-117.809335,40.907926 -117.699767,40.993246") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32013, "countyName": "Humboldt", "cityID": 3284800, "cityName": "Winnemucca" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828080771878912, "text": "Temp: 39.6°F - Dew Point: 32.3° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.21in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 12, "statues_count": 17836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828080931241985, "text": "I swear it was just 10", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2490571530, "name": "Joe Vasquez", "screen_name": "JoeVas99", "lang": "en", "location": "Arcadia, CA", "create_at": date("2014-05-11"), "description": "| AHS'17 | Varsity XC/TF | sc:Joe_vas360", "followers_count": 384, "friends_count": 298, "statues_count": 3791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcadia, CA", "id": "e6fc035b8939bd3b", "name": "Arcadia", "place_type": "city", "bounding_box": rectangle("-118.06946,34.087059 -117.99218,34.180507") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602462, "cityName": "Arcadia" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828081006776320, "text": "this semester got me like ������ https://t.co/bmzj0spNzc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256786908, "name": "Lorindá Golæb", "screen_name": "I_Am_Lorde_", "lang": "en", "location": "Chicago", "create_at": date("2011-02-23"), "description": "Western Illinois & ΦΣΣ Alumna•Lorinda by Day, LITrinda by Night", "followers_count": 887, "friends_count": 809, "statues_count": 16475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828081652703232, "text": "Temp 59.6°F Wind Chill 59.6°F RH 84% Wind 2.8 W Gust 11.0 W SLP 29.999 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 63, "statues_count": 35508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828081686208512, "text": "Happy birthday daltino, we've been homies all of high school, and no dull moments w u we geeked up till theend������✊���� https://t.co/6yKMfuRKnW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 990160220, "name": "Lil Ders", "screen_name": "andrewrobles34", "lang": "en", "location": "Long Beach, CA", "create_at": date("2012-12-04"), "description": "Active Bakersfield // #celebratenedu", "followers_count": 555, "friends_count": 493, "statues_count": 5494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828082139222016, "text": "@GSpellchecker inquiring minds want to know @holland_tom are you prepared to deal with all the future spider-man jokes?", "in_reply_to_status": 732822859060531200, "in_reply_to_user": 239885144, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 456636692, 239885144 }}, "user": { "id": 14523328, "name": "primey", "screen_name": "primey", "lang": "en", "location": "iPhone: 37.796055,-122.416695", "create_at": date("2008-04-24"), "description": "I don't know where I'm going. But I sure know where I've been.", "followers_count": 214, "friends_count": 338, "statues_count": 16673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sun Valley, Los Angeles", "id": "0184d29fc02afcd3", "name": "Sun Valley", "place_type": "neighborhood", "bounding_box": rectangle("-118.413211,34.206492 -118.266812,34.262168") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828082881585152, "text": "I was late to being a hoe but I ain't ready to be no bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 392157667, "name": "Chris:.", "screen_name": "christopherdi13", "lang": "en", "location": "SS", "create_at": date("2011-10-16"), "description": "Juss a pretty Krazy Morro FRUM the west SIDE uff CHIQUEZ.... BLU gheOOO✨®", "followers_count": 658, "friends_count": 499, "statues_count": 6160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828082906746880, "text": "Me in AP lit rn https://t.co/eHDmsJAXgQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 811347492, "name": "bailee from subway", "screen_name": "bailee_alusha", "lang": "en", "location": "null", "create_at": date("2012-09-08"), "description": "c/o 2016 class clown @ GKHS", "followers_count": 695, "friends_count": 403, "statues_count": 53163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-05-18T00:00:04.000Z"), "id": 732828083028393985, "text": "@sweynjupiter ☺", "in_reply_to_status": 732794454491435008, "in_reply_to_user": 186447391, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 186447391 }}, "user": { "id": 111941241, "name": "DYKE", "screen_name": "TOTAL_FREEDOM_", "lang": "en", "location": "just visiting", "create_at": date("2010-02-06"), "description": "contact: TFSOUNDBANK@GMAIL.COM uk/eu booking: JODIE@FMLY.LONDON various americas: JOAQUIN@THEBUTROSGROUP.COM", "followers_count": 8574, "friends_count": 321, "statues_count": 9024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828083745619968, "text": "@YWill2017 you don't run nothing", "in_reply_to_status": 732827893047365632, "in_reply_to_user": 2613754285, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2613754285 }}, "user": { "id": 2919657894, "name": "❣", "screen_name": "rrenaej", "lang": "en", "location": "with paco.", "create_at": date("2014-12-05"), "description": "'17 | 8.21", "followers_count": 456, "friends_count": 298, "statues_count": 11083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Freeport, TX", "id": "e6e275c5a0d3993d", "name": "Freeport", "place_type": "city", "bounding_box": rectangle("-95.385637,28.933592 -95.308565,28.987872") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4827420, "cityName": "Freeport" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828083858903040, "text": "So I just got my tongue pierced and now I'm getting my nipples in a few hours ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1140917004, "name": "Emilee", "screen_name": "_hkittyfanatic", "lang": "en", "location": "McCurtain County", "create_at": date("2013-02-01"), "description": ": SPC in ARNG : Godmother : SC - emmiedanielle : Okie Girl :", "followers_count": 767, "friends_count": 679, "statues_count": 12007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Idabel, OK", "id": "762b2536dd21350a", "name": "Idabel", "place_type": "city", "bounding_box": rectangle("-94.854755,33.873904 -94.779851,33.917439") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40089, "countyName": "McCurtain", "cityID": 4036750, "cityName": "Idabel" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828084051841024, "text": "Why am I still sore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2395556503, "name": "Clealy⚡️", "screen_name": "C_aaalyyy", "lang": "en", "location": "null", "create_at": date("2014-03-17"), "description": "CSU Stan . ΔΦΓ Lightning ⚡️. I'm a princess. Mateo❤️⚔", "followers_count": 263, "friends_count": 355, "statues_count": 7518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828084571889665, "text": "Bruh , this bitch said a waste of life ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2429692341, "name": "tayaaa .", "screen_name": "jontayaarihonna", "lang": "en", "location": "Mechanicsville ", "create_at": date("2014-03-22"), "description": "Rest Easy Nana , 9-18-15 .", "followers_count": 610, "friends_count": 662, "statues_count": 8688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Owensboro, KY", "id": "66c992a992d4a17a", "name": "Owensboro", "place_type": "city", "bounding_box": rectangle("-87.211826,37.714739 -86.992973,37.815531") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21059, "countyName": "Daviess", "cityID": 2158620, "cityName": "Owensboro" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828084689338368, "text": "@crynialls YES", "in_reply_to_status": 732827596275224576, "in_reply_to_user": 416050675, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 416050675 }}, "user": { "id": 592483957, "name": "anahita️", "screen_name": "unbrokenfond", "lang": "en", "location": "null", "create_at": date("2012-05-27"), "description": "@squishylouie: protect anahita at all costs", "followers_count": 21209, "friends_count": 9013, "statues_count": 113293 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085020692480, "text": "My friends parents have taken me in as their own this week.\nI am now armed with pepper spray to ward off horny bastards.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 114486145, "name": "karína.", "screen_name": "paeceluvin", "lang": "en", "location": "Dreamville", "create_at": date("2010-02-15"), "description": "null", "followers_count": 430, "friends_count": 556, "statues_count": 12428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, WA", "id": "7291a25672e0d4b1", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-122.1649,47.626845 -121.958642,47.73078") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357535, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085217792001, "text": "@DukeGasssedup whea you pay rent at? What's in yo name? Who shoes you got on right now?", "in_reply_to_status": 732827644312625152, "in_reply_to_user": 4699920566, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4699920566 }}, "user": { "id": 2599406288, "name": "TADOE™", "screen_name": "Bxxtch_ImAlpo", "lang": "en", "location": "BAMA 251.✈️ HTX 713.", "create_at": date("2014-07-02"), "description": "StrapAbuser! 100Gwap Bro can't be gwap that lah nigga too friendly", "followers_count": 763, "friends_count": 303, "statues_count": 24804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085524037632, "text": "@zaynmalik did you see this yet? Hahaha https://t.co/9PzNql86Mt", "in_reply_to_status": -1, "in_reply_to_user": 176566242, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 176566242 }}, "user": { "id": 24250175, "name": "christy", "screen_name": "Squidkidmikey", "lang": "en", "location": "my world", "create_at": date("2009-03-13"), "description": "seeing @5sos in jax✨\npeta murgatroyd is my idol\nteam jeta, team redefining dance \n#hawkeye and #bucky fan", "followers_count": 752, "friends_count": 216, "statues_count": 64194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Hills, FL", "id": "0a7fa82a81bf51f4", "name": "Pine Hills", "place_type": "city", "bounding_box": rectangle("-81.504956,28.539225 -81.434413,28.634059") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1256825, "cityName": "Pine Hills" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085733740545, "text": "@iJartus but im upset i just didn't stop. My buddy made 1200 lol", "in_reply_to_status": 732827559361155072, "in_reply_to_user": 1266376488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1266376488 }}, "user": { "id": 1120717626, "name": "Mojo Jojo", "screen_name": "MoSlays_Sway", "lang": "en", "location": "Ohio", "create_at": date("2013-01-25"), "description": "Streamer for @Sway_Gaming. \n\nsnapchat: mokirksey", "followers_count": 595, "friends_count": 332, "statues_count": 22100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lisbon, OH", "id": "5ed9d7768cd0061b", "name": "Lisbon", "place_type": "city", "bounding_box": rectangle("-80.813335,40.762656 -80.724487,40.792813") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39029, "countyName": "Columbiana", "cityID": 3944030, "cityName": "Lisbon" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085763117057, "text": "@AshZification @NaughtyB3ar he lives in Vegas? Small world. Give me a chance to come say hi while you're in town", "in_reply_to_status": 732747987957485568, "in_reply_to_user": 400809030, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 400809030, 348659865 }}, "user": { "id": 35181006, "name": "Rootbear75", "screen_name": "rootbear75", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-04-25"), "description": "I'm a person", "followers_count": 323, "friends_count": 314, "statues_count": 14101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828085964443648, "text": "https://t.co/1zZxDanuJw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 248396766, "name": "MStacks✨", "screen_name": "HurricaneRisss", "lang": "en", "location": "South Lansing. ✈️ Nashville.", "create_at": date("2011-02-06"), "description": "Idc, fuck off.", "followers_count": 2204, "friends_count": 1463, "statues_count": 70929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828086069252097, "text": "I wish the supreme app sent notifs to your phone when something new is released. Their shit sells out fast af.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2735972454, "name": "fidèle", "screen_name": "OkayEmz", "lang": "en", "location": "Pearland, TX", "create_at": date("2014-08-15"), "description": "loner in luv w/ @partyomo.", "followers_count": 2100, "friends_count": 1396, "statues_count": 43048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-05-18T00:00:05.000Z"), "id": 732828086119583744, "text": "Looks like that Lemonade is spilling over...Tom could've totally let Jay-Z have it, especially after he wrote a song about him....@TMZ", "in_reply_to_status": 732813012080431106, "in_reply_to_user": 16331010, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16331010 }}, "user": { "id": 425465994, "name": "Tshirtwoman", "screen_name": "tshirtwoman19", "lang": "en", "location": "null", "create_at": date("2011-11-30"), "description": "I wear my sunglasses at night......", "followers_count": 199, "friends_count": 370, "statues_count": 11777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190450325032962, "text": "just a moment https://t.co/B7vwTjfQVx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212828880, "name": "hot topic clone", "screen_name": "oujasurvive", "lang": "en", "location": "bran bal", "create_at": date("2010-11-06"), "description": "how about those karate bugmen they/them or ve/vir/vis lyr/18/leo\r\nbahati j'naah on cactuar im bad at video games and i love dying", "followers_count": 248, "friends_count": 269, "statues_count": 78474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190450777968640, "text": "https://t.co/KJcpgn0WIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1326227552, "name": "Ryan X", "screen_name": "RealRyan_X", "lang": "en", "location": "Chicago ", "create_at": date("2013-04-03"), "description": "Student Filmmaker\r\rFollow me on IG @RealRyanX\n\nSnapchat: @RealRyan_x", "followers_count": 1126, "friends_count": 939, "statues_count": 2503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago Heights, IL", "id": "55f894104157687c", "name": "Chicago Heights", "place_type": "city", "bounding_box": rectangle("-87.674421,41.481426 -87.596844,41.543213") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714026, "cityName": "Chicago Heights" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190450916417536, "text": "@USDOL to keep a paycheck out of hands shows how much overpaid trash government is, as long as it isn't hurting their check they don't care", "in_reply_to_status": -1, "in_reply_to_user": 20179628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20179628 }}, "user": { "id": 182369654, "name": "Cheryl Denise Allen", "screen_name": "cherallen1", "lang": "en", "location": "artbycheryldeniseallen.webs ", "create_at": date("2010-08-24"), "description": "Artist, Realtor consultant, Hairdresser, Life and Health Insurance, General studies degree, PC Certificate, equal to a masters degree", "followers_count": 121, "friends_count": 26, "statues_count": 2564 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinsville, IN", "id": "b9026e054cbc35e9", "name": "Martinsville", "place_type": "city", "bounding_box": rectangle("-86.451727,39.401674 -86.380069,39.475683") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18109, "countyName": "Morgan", "cityID": 1847448, "cityName": "Martinsville" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190451071705089, "text": "it's so cute to see boys in high school giggle when they pass the bra section at target. sorry if that is tmi but i just had to share", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 981476611, "name": "chillyourgrill", "screen_name": "hellomeghan97", "lang": "en", "location": "Summerville, SC", "create_at": date("2012-11-30"), "description": "be nice. be happy. be original. be crazy. lover of one tree hill", "followers_count": 318, "friends_count": 336, "statues_count": 13915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerville, SC", "id": "676d5bedd73ca2ec", "name": "Summerville", "place_type": "city", "bounding_box": rectangle("-80.235287,32.944965 -80.109533,33.054993") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45035, "countyName": "Dorchester", "cityID": 4570270, "cityName": "Summerville" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190451285635072, "text": "Wind 0.0 mph ---. Barometer 29.82 in, Steady. Temperature 47.3 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 23, "friends_count": 0, "statues_count": 32675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190451306496000, "text": "\"Is this how he feeds\" ���������� https://t.co/NMlTfigyVu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3041990510, "name": "G", "screen_name": "grettelgalindo", "lang": "en", "location": "arizona", "create_at": date("2015-02-25"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀@justinbieber follows | met him ⠀⠀⠀⠀⠀072510, 111813, 052715, 111315, & 033016 JB ♡", "followers_count": 2011, "friends_count": 695, "statues_count": 30758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sierra Vista, AZ", "id": "4f5e07211d107837", "name": "Sierra Vista", "place_type": "city", "bounding_box": rectangle("-110.449337,31.434345 -110.217964,31.61845") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4003, "countyName": "Cochise", "cityID": 466820, "cityName": "Sierra Vista" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190451658809344, "text": "They only got fined like 110 �� https://t.co/n6OdiCaaMy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 279857122, "name": "G Baby ", "screen_name": "_GwenolaBar", "lang": "en", "location": "Hawaii, USA", "create_at": date("2011-04-09"), "description": "21 | im cuter on social media | Youtuber |BlackLivesMatter | intersectional fem | plus sized queen |", "followers_count": 1839, "friends_count": 1290, "statues_count": 129437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napili-Honokowai, HI", "id": "2acb7a701264a29b", "name": "Napili-Honokowai", "place_type": "city", "bounding_box": rectangle("-156.690251,20.946851 -156.646359,21.003423") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1554100, "cityName": "Napili-Honokowai" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190451805716481, "text": "@pocflow69 @Turdxstile https://t.co/Z48eoecGeC", "in_reply_to_status": 733190229214007296, "in_reply_to_user": 4765002926, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4765002926, 1874181157 }}, "user": { "id": 4845259527, "name": "ROWDYRUFF DAD", "screen_name": "Yungsun_", "lang": "en", "location": "ANIME IS REAL", "create_at": date("2016-01-25"), "description": "Gavin // 20 // Anime // Skateboarding // @TerriblexDad @me_be_bree", "followers_count": 170, "friends_count": 187, "statues_count": 1432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452015292416, "text": "@Nova_wuff I totally would but I don't like weasyls inteface. Super jumbled to me.", "in_reply_to_status": 733059623662428160, "in_reply_to_user": 21339731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21339731 }}, "user": { "id": 1321289282, "name": "Captain and King", "screen_name": "DaisyPoodlePuff", "lang": "en", "location": "Santa Rosa, CA", "create_at": date("2013-04-01"), "description": "Godless vegan. @Ace_Orcinus belongs to me.", "followers_count": 358, "friends_count": 813, "statues_count": 9459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452149510145, "text": "#doubleday drop ins #improv @iOWest then hit the mic @TaoComedyStudio #standup #love #life #fun #day #humpday #comedy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "doubleday", "improv", "standup", "love", "life", "fun", "day", "humpday", "comedy" }}, "user_mentions": {{ 36608922, 1609282710 }}, "user": { "id": 58119717, "name": "Mark Rius", "screen_name": "MarkRius27", "lang": "en", "location": "Los Angeles", "create_at": date("2009-07-18"), "description": "Comedian, Actor, Sketch Comedy Writer, Lacrosse player", "followers_count": 94, "friends_count": 254, "statues_count": 393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452313128960, "text": "I love when my son randomly comes up to me to hold me and kiss me���� the best love a mom can get❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2493613506, "name": "Sara Vielmas✨", "screen_name": "syvielmas", "lang": "en", "location": "null", "create_at": date("2014-05-13"), "description": "Too glam to give a damn✨", "followers_count": 121, "friends_count": 112, "statues_count": 3355 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452329861120, "text": "@amytoast Glad you're feeling better! Ja sam sova večeras! Sa vinom!", "in_reply_to_status": 733186856737734656, "in_reply_to_user": 14169582, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 14169582 }}, "user": { "id": 17517730, "name": "chris lines", "screen_name": "cjflines", "lang": "en", "location": "Austin", "create_at": date("2008-11-20"), "description": "Submariner. Defense+consumer electronics. Linguist+editor. Astronomy, politics, rugby, food+wine, sheepdogs.", "followers_count": 360, "friends_count": 349, "statues_count": 7544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452489244672, "text": "TANGINA NAMAN KAPAG SINABI KONG DI AKO PUPUNTA, PUPUNTA AKO LETSE KASO WALA EH ININBOX ZONE LANG AKO GAGO TALAGA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3177556298, "name": "Misophonia ♠", "screen_name": "Valdehuezagdgd", "lang": "en", "location": "Atlanta, Georgia ", "create_at": date("2015-04-27"), "description": "+Imma gizibe reborn+", "followers_count": 1578, "friends_count": 1372, "statues_count": 42043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452787060736, "text": "Keep pushing me away.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056508424, "name": "mar", "screen_name": "eggrolI_", "lang": "en", "location": "homegrl nxtdoor", "create_at": date("2013-01-02"), "description": "parttime soulfisticated shawty // fulltime down2mars gal", "followers_count": 808, "friends_count": 293, "statues_count": 62698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190452900417536, "text": "3:00 AM https://t.co/Nm3NtUqmQY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3135761949, "name": "Malini", "screen_name": "malinis_64", "lang": "en", "location": "null", "create_at": date("2015-04-04"), "description": "CHS'18", "followers_count": 155, "friends_count": 165, "statues_count": 1059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellicott City, MD", "id": "515586774402eacb", "name": "Ellicott City", "place_type": "city", "bounding_box": rectangle("-76.952172,39.215288 -76.763767,39.324062") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24027, "countyName": "Howard", "cityID": 2426000, "cityName": "Ellicott City" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190453575569408, "text": "Beyoncé was everything I imagined it to be and more ���� #FormationWorldTour", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FormationWorldTour" }}, "user": { "id": 628679089, "name": "Sam Thompson", "screen_name": "Sam52165768", "lang": "en", "location": "null", "create_at": date("2012-07-06"), "description": "everyone you meet is fighting a battle you know nothing about.", "followers_count": 168, "friends_count": 250, "statues_count": 616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190454250897408, "text": "Jeana & Jesse tho ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2409398850, "name": "babs", "screen_name": "chill_herrera", "lang": "en", "location": "rifai's", "create_at": date("2014-03-24"), "description": "null", "followers_count": 536, "friends_count": 580, "statues_count": 4193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-19T00:00:00.000Z"), "id": 733190454351560704, "text": "@abdoosman145 30", "in_reply_to_status": 733190161933176832, "in_reply_to_user": 2204860553, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2204860553 }}, "user": { "id": 550239282, "name": "Medhatt", "screen_name": "Mohamedmmedhatt", "lang": "en", "location": "Minnesota, USA", "create_at": date("2012-04-10"), "description": "RIDE OR DIE.", "followers_count": 389, "friends_count": 88, "statues_count": 8573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winona, MN", "id": "5b555b41d846c27a", "name": "Winona", "place_type": "city", "bounding_box": rectangle("-91.720399,43.998296 -91.552948,44.079083") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27169, "countyName": "Winona", "cityID": 2771032, "cityName": "Winona" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190454687096832, "text": "Happy Birthday Ashee I love you baby girl you're now 20 boo boo! ���� love you @ashmami_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1306200060 }}, "user": { "id": 1679117328, "name": "Felix", "screen_name": "ovofelix", "lang": "en", "location": "Tempe, AZ", "create_at": date("2013-08-17"), "description": "Arizona State University 2018.", "followers_count": 351, "friends_count": 364, "statues_count": 7420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190454863400965, "text": "Ripley SW Limestone Co. Temp: 57.7°F Wind:2.2mph Pressure: 997.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53489 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190454888398849, "text": "@Beyonce after party at my house ��", "in_reply_to_status": -1, "in_reply_to_user": 31239408, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31239408 }}, "user": { "id": 956371662, "name": "tifferoni ♡", "screen_name": "___tiffff", "lang": "en", "location": "null", "create_at": date("2012-11-18"), "description": "goin' with the flow", "followers_count": 1915, "friends_count": 851, "statues_count": 22352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Five Corners, WA", "id": "5d2456d6a8ee247d", "name": "Five Corners", "place_type": "city", "bounding_box": rectangle("-122.606953,45.658578 -122.552528,45.707934") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5324188, "cityName": "Five Corners" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190455291215872, "text": "05/19@03:00 - Temp 53.7F, WC 53.7F. Wind 0.3mph SW, Gust 2.0mph. Bar 30.106in, Falling slowly. Rain 0.00in. Hum 86%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190455333019648, "text": "Wind 0.0 mph ---. Barometer 1023.78 mb, Steady. Temperature 48.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190455374929920, "text": "@WhoDatBASH we lost when you went the other night", "in_reply_to_status": 733190237422149632, "in_reply_to_user": 139657813, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 139657813 }}, "user": { "id": 323288145, "name": "Daniel.F", "screen_name": "kid_vito10", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "If I was you, I wouldn't like me either", "followers_count": 661, "friends_count": 648, "statues_count": 54424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190455878373376, "text": "@NormaniKordei hey if I see you at a signing itll be with a broken rib you think you can kiss it better for me? I need queen Mani's blessing", "in_reply_to_status": -1, "in_reply_to_user": 209003577, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 209003577 }}, "user": { "id": 754480416, "name": "Andrew Scozzari", "screen_name": "andrewscozzari", "lang": "en", "location": "Shaolin", "create_at": date("2012-08-12"), "description": "#SavageSZN", "followers_count": 1247, "friends_count": 1211, "statues_count": 93505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456025063425, "text": "@MelanieGalicia big bhillen in bed and you?", "in_reply_to_status": 733190340983676928, "in_reply_to_user": 330017130, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 330017130 }}, "user": { "id": 2916015948, "name": "luppeee", "screen_name": "LeonardoLoera2", "lang": "en", "location": "Jurupa Valley, CA", "create_at": date("2014-12-01"), "description": "Ponte verga, o vete a la verga.", "followers_count": 395, "friends_count": 526, "statues_count": 2425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jurupa Valley, CA", "id": "015b658472edd3dc", "name": "Jurupa Valley", "place_type": "city", "bounding_box": rectangle("-117.551185,33.955517 -117.410795,34.034306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 637692, "cityName": "Jurupa Valley" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456025157632, "text": "@dane_longden u wanna join in?", "in_reply_to_status": 733189648126709760, "in_reply_to_user": 1350036600, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1350036600 }}, "user": { "id": 2977714448, "name": "cyroh", "screen_name": "amCyroh", "lang": "en", "location": "↓↓ Go check out my channel ↓↓", "create_at": date("2015-01-12"), "description": "Large channel? Dm for a pack.ㅤㅤㅤㅤ 1.6k Sub YTer: http://tiny.cc/Cyro ㅤㅤㅤㅤOwner of @FadeSMP ㅤㅤㅤㅤㅤㅤ[Pack Commisions Closed] ㅤㅤㅤㅤㅤㅤrant @fmlcyroh", "followers_count": 1975, "friends_count": 402, "statues_count": 18294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Coral, FL", "id": "fbd1502e5edce6bf", "name": "Cape Coral", "place_type": "city", "bounding_box": rectangle("-82.069567,26.531814 -81.904101,26.741146") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1210275, "cityName": "Cape Coral" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456125706240, "text": "@ebbtideapp Tide in Point Chevreuil, Louisiana 05/19/2016\nHigh 5:54am 1.5\n Low 9:34am 1.2\nHigh 3:37pm 1.4\n Low 8:58pm 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-91.55,29.5167"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 91, "friends_count": 1, "statues_count": 34008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22101, "countyName": "St. Mary" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456159444992, "text": "Full Moon Over Oleander. #moon #fullmoon #oleander #flowers #pink #nightphotography #spring… https://t.co/toMV5YwCBj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.059,29.9728"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "moon", "fullmoon", "oleander", "flowers", "pink", "nightphotography", "spring" }}, "user": { "id": 26284162, "name": "Linda Minutola", "screen_name": "LadyLunaNOLA", "lang": "en", "location": "New Orleans, Louisiana", "create_at": date("2009-03-24"), "description": "Photographer/Owner - LADY LUNA PHOTOGRAPHY. Soul Siren with @SOULFactoryNOLA. Proud member of the WHO DAT NATION! Lover of all things NOLA...and Nature.", "followers_count": 1340, "friends_count": 2146, "statues_count": 2116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456557768704, "text": "I though I was the laziest person I'd ever met until I met me this summer break ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 419155530, "name": "tori", "screen_name": "toriirobertss", "lang": "en", "location": "null", "create_at": date("2011-11-22"), "description": "i love quesadillas", "followers_count": 548, "friends_count": 216, "statues_count": 2407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456578707464, "text": "I have dank for days... lol good lookin work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 190520402, "name": "Gordo", "screen_name": "ruiz_javierr", "lang": "en", "location": "Chino Hills", "create_at": date("2010-09-13"), "description": "null", "followers_count": 102, "friends_count": 109, "statues_count": 4812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456759066624, "text": "wish i was laid up getting my booty rubbed & getting my hair played in ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379820056, "name": "ling", "screen_name": "_haileyvo", "lang": "en", "location": "Cali✈️Bama", "create_at": date("2011-09-25"), "description": "moved to move on ; Alabama State Univ.", "followers_count": 1252, "friends_count": 567, "statues_count": 63285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190456872456192, "text": "Wind 1.0 mph NE. Barometer 30.147 in, Rising slowly. Temperature 52.8 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190457245728768, "text": "Temp: 66.1°F Wind:0.0mph Pressure: 30.009hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190457589633024, "text": "Temp 51.3° Hi/Lo 53.0/51.3 Rng 1.7° WC 51.3° Hmd 81% Rain 0.00\" Storm 0.00\" BAR 30.107 Falling DP 45.7° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190457589673984, "text": "Wind 0.0 mph ---. Barometer 30.163 in, Steady. Temperature 46.6 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190457635704833, "text": "@CrippnOnTheRise ������������", "in_reply_to_status": 733190394486214656, "in_reply_to_user": 563291891, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 563291891 }}, "user": { "id": 711958448, "name": "V$J♏️", "screen_name": "ValaaaaryJ", "lang": "en", "location": "Yes, I attend WOS!", "create_at": date("2012-07-23"), "description": "keep a couple wet wipes case a bum try to touch me, ew. XO' |my BOYFRIEND is Aidan| r.i.p dad -- BO$$ UP✨", "followers_count": 1920, "friends_count": 981, "statues_count": 76403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, TX", "id": "3fe7643d3e014f67", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-93.858146,30.061031 -93.715588,30.193051") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48361, "countyName": "Orange", "cityID": 4854132, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190457711190020, "text": "@OfficialSoMo see.. This is why you don't fucks with Sac. @ilovebops", "in_reply_to_status": 733188578088353792, "in_reply_to_user": 216431630, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216431630, 2517238572 }}, "user": { "id": 1380991838, "name": "Miles Johansen", "screen_name": "yo_hansen17", "lang": "en", "location": "null", "create_at": date("2013-04-25"), "description": "19 | Lake Tahoe CA ➡️ Long Beach State | ΣΠ", "followers_count": 202, "friends_count": 263, "statues_count": 1248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kings Beach, CA", "id": "0013a10a30ce72f0", "name": "Kings Beach", "place_type": "city", "bounding_box": rectangle("-120.038043,39.222133 -120.00526,39.285126") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 638548, "cityName": "Kings Beach" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190458046873600, "text": "I need a stomach & booty rub������ where's daddy?����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17415249, "name": "Turner.", "screen_name": "TRS______", "lang": "en", "location": "Memphis, TN", "create_at": date("2008-11-15"), "description": "#6years #PiruWife #MSCC15 #UAPB17 sc:tia_sanders2013", "followers_count": 4242, "friends_count": 4056, "statues_count": 158009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Memphis, AR", "id": "017db07b3676900f", "name": "West Memphis", "place_type": "city", "bounding_box": rectangle("-90.237784,35.120479 -90.121655,35.231923") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5035, "countyName": "Crittenden", "cityID": 574540, "cityName": "West Memphis" } }
+{ "create_at": datetime("2016-05-19T00:00:01.000Z"), "id": 733190458201890816, "text": "@MaddieMcCarCar Thats pretty cool, he looks exactly the same ��", "in_reply_to_status": 733170927274336256, "in_reply_to_user": 426226759, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426226759 }}, "user": { "id": 439356927, "name": "Gilad Ronat", "screen_name": "giladronat", "lang": "en", "location": "San Luis Obispo, California", "create_at": date("2011-12-17"), "description": "Freelance iOS Developer. \nCoding instead of sleeping, working instead of homeworking.\nGaming, travel, design, EDM.\nComputer Science @CalPoly.", "followers_count": 1341, "friends_count": 690, "statues_count": 7848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Luis Obispo, CA", "id": "057f8a6fa3c286f9", "name": "San Luis Obispo", "place_type": "city", "bounding_box": rectangle("-120.71213,35.235477 -120.6178,35.314141") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 668154, "cityName": "San Luis Obispo" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190458675843073, "text": "The most disgusting thing I've ever seen https://t.co/NYGz7pxVkv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 246073043, "name": "amy", "screen_name": "SimplyAymeee", "lang": "en", "location": "san diego", "create_at": date("2011-02-01"), "description": "I don't even tweet like that doe", "followers_count": 884, "friends_count": 320, "statues_count": 12481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190458780749825, "text": "Dead ������ you playa cuz I can't do it shordy �� https://t.co/OXJ4AFK87U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302886774, "name": "July 9th ♋️", "screen_name": "almighty_ajaxk", "lang": "en", "location": "Chiraq ✈️ Dallas", "create_at": date("2015-07-31"), "description": "Praying for better days.. But God got me", "followers_count": 542, "friends_count": 418, "statues_count": 9421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190459183374339, "text": "The amount of times I say \"I'm gonna punch you in the mouth\" during the day makes me question my existence as a human being.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116264001, "name": "nickel↓", "screen_name": "coveryoursnacks", "lang": "en", "location": "california", "create_at": date("2010-02-21"), "description": "wishing I was dead wishing I was dead because I guess I'm ugly | ♡@carrinak4l♡ | Cover Your Trash", "followers_count": 1688, "friends_count": 493, "statues_count": 124477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190459275743232, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 66.6 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22887 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190459401457664, "text": "@OfficeChair Hello\nHow are you? i am Mike from china.\nWe are office furniture manufacturer.\nThis is our website:http//www.hkglorystar.com", "in_reply_to_status": -1, "in_reply_to_user": 19260054, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19260054 }}, "user": { "id": 722723558284103680, "name": "MikeW", "screen_name": "MikeWenH", "lang": "zh-cn", "location": "Lecong Town, Shunde District, ", "create_at": date("2016-04-20"), "description": "I am Mike from china,I am a office furniture manufacturer and exporter,we offer high quality office furniture.welcome inquiry and visit our company.", "followers_count": 40, "friends_count": 313, "statues_count": 53 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Absecon, NJ", "id": "019674886273dd6d", "name": "Absecon", "place_type": "city", "bounding_box": rectangle("-74.558351,39.385879 -74.459587,39.506513") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3400100, "cityName": "Absecon" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190460047396864, "text": "HBD ,birthday buddy hope you have a sick ass dayy @AndyKostyk ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2273812494 }}, "user": { "id": 2842827799, "name": "tabs", "screen_name": "tabycrist", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "rest in peace beautiful, S.T❤️", "followers_count": 733, "friends_count": 545, "statues_count": 3150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190460106104832, "text": "@jaileybreahn I can't tell �� he being friendly tonight", "in_reply_to_status": 733190366698995713, "in_reply_to_user": 2451769428, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2451769428 }}, "user": { "id": 2817451288, "name": "Maw Tia", "screen_name": "tiapaige__", "lang": "en", "location": "Houston, TX", "create_at": date("2014-10-08"), "description": "Your favorite @PGO_stan", "followers_count": 1042, "friends_count": 578, "statues_count": 92590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190460144033792, "text": "#SummerSchemes16 https://t.co/mVLl4rzK4P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "SummerSchemes16" }}, "user": { "id": 53000782, "name": "JeremyGerard™", "screen_name": "JG_Trilll", "lang": "en", "location": "Opelousas☀️", "create_at": date("2009-07-02"), "description": "Jiggy Nigga | I Like It Ent. | YWA | #LSU19 | Business Minded ❌ Engineer ❌ Louisiana", "followers_count": 3333, "friends_count": 2351, "statues_count": 54400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190460353609728, "text": "������������ https://t.co/oF9sZYIPQ2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 838901678, "name": "JT", "screen_name": "jono_thekid", "lang": "en", "location": "null", "create_at": date("2012-09-21"), "description": "Morehouse'19, DuBois Hall Alumnus,CHI✈️ATL, Gemini, Quote: If You Are Not At The Table, Then You Are On The Menu. Blood Is Not Red Anymore It's Maroon&White", "followers_count": 819, "friends_count": 1158, "statues_count": 5958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190460374700033, "text": "Chill beez just quoted my tweet ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 575106641, "name": "M E E K⚡️", "screen_name": "King_Shit5", "lang": "en", "location": "null", "create_at": date("2012-05-08"), "description": "TOO HIGH TO RIOT", "followers_count": 2139, "friends_count": 1560, "statues_count": 40433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rahway, NJ", "id": "9d22866e11c9b6b7", "name": "Rahway", "place_type": "city", "bounding_box": rectangle("-74.305281,40.592141 -74.255435,40.627526") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3461530, "cityName": "Rahway" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190461284716546, "text": "@CarriWeatherwax https://t.co/SLaSzHW3El", "in_reply_to_status": 733189636726476800, "in_reply_to_user": 297447292, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 297447292 }}, "user": { "id": 171303295, "name": "Taydonna", "screen_name": "Launterfied2", "lang": "en", "location": "Iowa, USA", "create_at": date("2010-07-26"), "description": "Queen of Pop- Madonna. Other Queen- Taylor Swift. Bernie for president in 2016. My Other favs- Justin Timberlake, Britney, Bieber, Miley, Zac Efron, and Jonas", "followers_count": 2039, "friends_count": 2315, "statues_count": 98777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190461351829504, "text": "Wish she'd be more productive like @Beyonce https://t.co/AkaNi3RhkE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31239408 }}, "user": { "id": 20459585, "name": "Paul Boulon", "screen_name": "PaulBoulon", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2009-02-09"), "description": "Instagram : officialpaulboulon | Snapchat : paulboulon", "followers_count": 24456, "friends_count": 967, "statues_count": 16705 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190461540564992, "text": "��#wednesday my #girlfriend #bae at the #kids #choice #awards #mexico #style #fancy #blimp #new… https://t.co/cPExdjVGuL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.06666667,33.45"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wednesday", "girlfriend", "bae", "kids", "choice", "awards", "mexico", "style", "fancy", "blimp", "new" }}, "user": { "id": 728285871204487168, "name": "Daniel Levin", "screen_name": "LevinDaniel4", "lang": "en", "location": "null", "create_at": date("2016-05-05"), "description": "-Defender la libertad, estimar la vida. Valorar el amor, apreciar cada momento", "followers_count": 18, "friends_count": 120, "statues_count": 5 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190462312353792, "text": "������ https://t.co/374H5QnGvz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2273811810, "name": "Arianaa✨", "screen_name": "ariana_a12", "lang": "en", "location": "null", "create_at": date("2014-01-02"), "description": "prolly somewhere playing softball, getting lit, eating, or sleepin'", "followers_count": 850, "friends_count": 645, "statues_count": 7925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-19T00:00:02.000Z"), "id": 733190462396321792, "text": "Idk how long I was sleep. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 374806126, "name": "sway.", "screen_name": "M31LO", "lang": "en", "location": "w/my woes. ", "create_at": date("2011-09-16"), "description": "21. UNC-Pembroke. eeee-yip.", "followers_count": 975, "friends_count": 685, "statues_count": 88589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke, NC", "id": "1588871da68509bb", "name": "Pembroke", "place_type": "city", "bounding_box": rectangle("-79.216367,34.654347 -79.150568,34.715633") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37155, "countyName": "Robeson", "cityID": 3751080, "cityName": "Pembroke" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190462824194048, "text": "More than a crush but yes https://t.co/Ex69eFJ9Cf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 127724254, "name": "mama love child", "screen_name": "muthalovechild", "lang": "en", "location": "America", "create_at": date("2010-03-29"), "description": "#holisticmedicine #attachmentparenting #homemadeProducts #makeupartist #ghostwriter #personalassistant", "followers_count": 1484, "friends_count": 192, "statues_count": 60638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, GA", "id": "5b8df26e6d0be60b", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-84.561205,33.788139 -84.470367,33.905391") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1371492, "cityName": "Smyrna" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190462870147072, "text": "Vegas in a few", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 347546531, "name": "Cici", "screen_name": "Cici_EatMyTweet", "lang": "en", "location": "Miami✈️HTX", "create_at": date("2011-08-02"), "description": "#PVAMU17 Mass Comm /Business major . fearless soul", "followers_count": 489, "friends_count": 417, "statues_count": 14390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190463033888769, "text": "64.1F (Feels: 64.1F) - Humidity: 99% - Wind: 3.1mph NE - Gust: 4.5mph - Pressure: 1013.4mb #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 241217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190463327338497, "text": "Come to think about it I'm hella paisa ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1016372240, "name": "❤️", "screen_name": "__mmcl", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-12-16"), "description": "null", "followers_count": 884, "friends_count": 789, "statues_count": 13050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190463394582528, "text": "Wind 0.0 mph ---. Barometer 30.10 in, Steady. Temperature 52.2 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 122, "statues_count": 159897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190463872606210, "text": "I wonder how much harder Live is in South Korea and Japan", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 971503026, "name": "Erick", "screen_name": "erick_jamal", "lang": "en", "location": "The Texas State University ", "create_at": date("2012-11-25"), "description": "At that point where Idk if I'm too old to say my name is Jamal or if to say it's Erick LOL\n \n SC: jamal46", "followers_count": 710, "friends_count": 160, "statues_count": 53539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190463981641728, "text": "Roxanne just came to my house and surprised me with cookies, a shake, and fries.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464177687, "name": "Nazaneen", "screen_name": "nazaneenganjii", "lang": "en", "location": "SJ", "create_at": date("2012-01-14"), "description": "I'm a photo capturer & i like cars", "followers_count": 337, "friends_count": 130, "statues_count": 19752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190464078118912, "text": "I'm really laughing ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314236971, "name": "Aleyh", "screen_name": "Leahh_Boo", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-09"), "description": "God has a plan for me... #TrackIsLife #ChildOfGod #UNT18 #BEGREAT !", "followers_count": 2085, "friends_count": 1590, "statues_count": 53609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190464447221760, "text": "I truly appreciate this https://t.co/21oCordJyP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3039274610, "name": "Le", "screen_name": "_________adubb", "lang": "en", "location": "in da juice", "create_at": date("2015-02-24"), "description": "don't make me give you back to the hood.", "followers_count": 512, "friends_count": 302, "statues_count": 8472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherwood, AR", "id": "248b860c847eee13", "name": "Sherwood", "place_type": "city", "bounding_box": rectangle("-92.247306,34.79959 -92.160908,34.883685") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 563800, "cityName": "Sherwood" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190464635969536, "text": "So glad I met you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358564834, "name": "Tristan Wunsch", "screen_name": "tristanwunsch", "lang": "en", "location": "null", "create_at": date("2011-08-19"), "description": "Palm desert football #80 snapchat: tristanwunsch13", "followers_count": 418, "friends_count": 691, "statues_count": 7528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190465084784640, "text": "I could watch The Office all day everyday for the rest of my life Best show everrrrr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 901176307, "name": "Shaggy", "screen_name": "p_tawn18", "lang": "en", "location": "Council Bluffs, IA", "create_at": date("2012-10-23"), "description": "What you Lurkin for", "followers_count": 427, "friends_count": 586, "statues_count": 2637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Council Bluffs, IA", "id": "1ff2b4659e670e52", "name": "Council Bluffs", "place_type": "city", "bounding_box": rectangle("-95.923551,41.194487 -95.746366,41.300487") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19155, "countyName": "Pottawattamie", "cityID": 1916860, "cityName": "Council Bluffs" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190465386778624, "text": "SHOUT OUT TO ALL THE MOTHERFUCKAS WHOS BIRTHDAY IS IN MAY YALL TURNT ‼️‼️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1421012839, "name": "Enrique", "screen_name": "Enrriquee12", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-05-11"), "description": "null", "followers_count": 568, "friends_count": 598, "statues_count": 13624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190465487396864, "text": "2am and I'm wide awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 217587911, "name": "vanessa diaz", "screen_name": "_vanessadiazm", "lang": "en", "location": "null", "create_at": date("2010-11-19"), "description": "Chicago,IL |", "followers_count": 169, "friends_count": 102, "statues_count": 108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190465512574976, "text": "I dropped my hot pocket", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 414290769, "name": "RayTheGod_X", "screen_name": "RTG__X", "lang": "en", "location": "Jonesboro, Arkansas", "create_at": date("2011-11-16"), "description": "That's Not My House In My Header, But The First Step Towards Achieving A Goal Is To Have A Vision | #Virgo♍️| #ASU16| Urban Philosopher| Humanitarian", "followers_count": 1270, "friends_count": 1335, "statues_count": 26152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jonesboro, AR", "id": "35189a8fcb82b788", "name": "Jonesboro", "place_type": "city", "bounding_box": rectangle("-90.782468,35.763136 -90.611214,35.894116") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5031, "countyName": "Craighead", "cityID": 535710, "cityName": "Jonesboro" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190465650982912, "text": "@_belen_santos_ @Jazzlyyn_24 @brittannie_13 says the one who snaps for attention .", "in_reply_to_status": 733189341128855552, "in_reply_to_user": 3004404308, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3004404308, 142535932, 896544138 }}, "user": { "id": 2985071827, "name": "broc", "screen_name": "jalisco51", "lang": "en", "location": "null", "create_at": date("2015-01-15"), "description": "18 single dejavu", "followers_count": 64, "friends_count": 154, "statues_count": 1172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466003296256, "text": "������ https://t.co/f1cOYonLYQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 975293384, "name": "tiff", "screen_name": "_tiffahhney", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "☹", "followers_count": 355, "friends_count": 271, "statues_count": 5360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466242514944, "text": "it's 3 am I just got out the shower and now I'm making cookies��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2493412242, "name": "cha®lie", "screen_name": "Charlizeeeeee_", "lang": "en", "location": "null", "create_at": date("2014-05-13"), "description": "null", "followers_count": 115, "friends_count": 141, "statues_count": 922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mays Landing, NJ", "id": "01c2b4c40769f6ea", "name": "Mays Landing", "place_type": "city", "bounding_box": rectangle("-74.738447,39.427211 -74.674912,39.46954") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34001, "countyName": "Atlantic", "cityID": 3444820, "cityName": "Mays Landing" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466439499777, "text": "@OneCreatorDan @thechrishaley even said yes and anything was signed.", "in_reply_to_status": 733190379437051904, "in_reply_to_user": 129575208, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3376721375, 15151510 }}, "user": { "id": 129575208, "name": "Kilo@Fanime", "screen_name": "OilyWhisper68", "lang": "en", "location": "null", "create_at": date("2010-04-04"), "description": "Editor for MCS | On again/off again with twitter | DMs are open", "followers_count": 338, "friends_count": 281, "statues_count": 113998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466556952580, "text": "@palofino_pjatta ... Fr, yo. That's how your doing it .", "in_reply_to_status": -1, "in_reply_to_user": 2326421420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2326421420 }}, "user": { "id": 1074658718, "name": "M A J.", "screen_name": "_MajaC", "lang": "en", "location": "null", "create_at": date("2013-01-09"), "description": "@_MajaC: Nous Créons Notre Monde..", "followers_count": 355, "friends_count": 249, "statues_count": 6882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466720530433, "text": "@Mitzymoraless tacos do sound so good rn", "in_reply_to_status": 733139588173070336, "in_reply_to_user": 3296758562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3296758562 }}, "user": { "id": 543949605, "name": "Heidi", "screen_name": "heidipolin", "lang": "en", "location": "somewhere i don't belong", "create_at": date("2012-04-02"), "description": "probably the most random person ever", "followers_count": 300, "friends_count": 441, "statues_count": 4232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaverton, OR", "id": "e0c5378910ca41e0", "name": "Beaverton", "place_type": "city", "bounding_box": rectangle("-122.867581,45.426386 -122.743577,45.543398") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4105350, "cityName": "Beaverton" } }
+{ "create_at": datetime("2016-05-19T00:00:03.000Z"), "id": 733190466913456128, "text": "@KevinDeel_ @darius_traylor I was trying to keep this roast private but it seems like Yal want the world to see ������", "in_reply_to_status": 733188574904864768, "in_reply_to_user": 2400840506, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2400840506, 1290572305 }}, "user": { "id": 27304562, "name": "Charvis Franklin", "screen_name": "charvisfranklin", "lang": "en", "location": "null", "create_at": date("2009-03-28"), "description": "Future baby daddy of Constance. Loser of life and lover of Jesus Christ. Advisor to the Dallas Cowboys Execs, Coaches and Players.", "followers_count": 443, "friends_count": 705, "statues_count": 5821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olivehurst, CA", "id": "7ae0364b4d56f5d6", "name": "Olivehurst", "place_type": "city", "bounding_box": rectangle("-121.579032,39.047486 -121.523467,39.113263") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6115, "countyName": "Yuba", "cityID": 653714, "cityName": "Olivehurst" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467139993600, "text": "Still a banger https://t.co/kNOPjcDkDt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4505203874, "name": "~", "screen_name": "lisamxchelle", "lang": "en", "location": "west covina ", "create_at": date("2015-12-08"), "description": "I'm gooood...", "followers_count": 292, "friends_count": 233, "statues_count": 709 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467286798336, "text": "So �� https://t.co/2gYznCtVHg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 108759431, "name": "Ci++", "screen_name": "Ayeee_Cici", "lang": "en", "location": "null", "create_at": date("2010-01-26"), "description": "IG : @ayeee_cici SC: @ayeee_cici.", "followers_count": 1955, "friends_count": 611, "statues_count": 108974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Miami Heights, FL", "id": "576bb66a5b95f65a", "name": "South Miami Heights", "place_type": "city", "bounding_box": rectangle("-80.414746,25.566015 -80.356751,25.627599") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1267575, "cityName": "South Miami Heights" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467299352576, "text": "@PALAYER_ thank you so much for #following", "in_reply_to_status": -1, "in_reply_to_user": 4902938058, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "following" }}, "user_mentions": {{ 4902938058 }}, "user": { "id": 2840584245, "name": "The Iron Relic", "screen_name": "TheIronRelic", "lang": "en", "location": "null", "create_at": date("2014-10-21"), "description": "An exciting and inspirational thriller of faith, miracles and the quest for eternal life.", "followers_count": 184, "friends_count": 240, "statues_count": 347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467328696322, "text": "Gia + Linda forever ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 369133493, "name": "Krystal Garza", "screen_name": "kaegarza", "lang": "en", "location": "San Francisco", "create_at": date("2011-09-06"), "description": "Write me a poem and I'll love you forever.", "followers_count": 208, "friends_count": 130, "statues_count": 2348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walnut Creek, CA", "id": "a35b62af9d82aa08", "name": "Walnut Creek", "place_type": "city", "bounding_box": rectangle("-122.10003,37.842368 -121.980829,37.944965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 683346, "cityName": "Walnut Creek" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467328737280, "text": "@athenab123 what he do this time", "in_reply_to_status": 733190434579578880, "in_reply_to_user": 630918648, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 630918648 }}, "user": { "id": 818283506, "name": "Sar Mar", "screen_name": "sarahmarrrrr", "lang": "en", "location": "null", "create_at": date("2012-09-11"), "description": "Ephesians 6:11", "followers_count": 673, "friends_count": 456, "statues_count": 39926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467467116544, "text": "Empire is fricken lit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2345476082, "name": "PC", "screen_name": "PiiClemens", "lang": "en", "location": "Kailua, HI", "create_at": date("2014-02-15"), "description": "null", "followers_count": 332, "friends_count": 324, "statues_count": 1282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kailua, HI", "id": "7f0e166849448f1d", "name": "Kailua", "place_type": "city", "bounding_box": rectangle("-157.765901,21.352946 -157.705775,21.428447") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1523150, "cityName": "Kailua" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190467609714688, "text": "Wind 0.0 mph NE. Barometer 30.048 in, Falling slowly. Temperature 58.5 °F. Rain today 0.01 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468029140993, "text": "@krownmecei Please follow me on Instagram �� @ leeofficialt I Follow ������", "in_reply_to_status": 733127996559785984, "in_reply_to_user": 704000750737887232, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 704000750737887232 }}, "user": { "id": 36274840, "name": "T$Lee ⚡", "screen_name": "LEEOfficialT", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-04-28"), "description": "Unsigned #Singer | #RnB | #Rapper | Artist | Songwriter| Her New Obsession | Bookings Serious Inquires Tleemusic23@gmail.com #teamtlee IG @leeofficialt", "followers_count": 31750, "friends_count": 14823, "statues_count": 121946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468398354432, "text": "I want to cuddle ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158503514, "name": "Crystal", "screen_name": "NoOtha_SpareKEE", "lang": "en", "location": "Memphis, TN", "create_at": date("2010-06-22"), "description": "8/13/15 god sent me an Angel #RIPFERG #MTSU |SC: Cutie_lilbootyy ❣|", "followers_count": 2749, "friends_count": 2353, "statues_count": 117838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468452941824, "text": "You can mouth all the @Greens bile you wish @DaveMiles60 you are morons @MargaretClark12 @nobby15 @phbarratt", "in_reply_to_status": 733180356090691585, "in_reply_to_user": 418493393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7900732, 418493393, 480785251, 51378153, 185614627 }}, "user": { "id": 2685926647, "name": "Discombobulated", "screen_name": "Biggy1883", "lang": "en", "location": "null", "create_at": date("2014-07-27"), "description": "ALP R&F Lefty--Tolerance of RWNJ`s and @Greens not a strong point\nLife is a highway -journey well", "followers_count": 3430, "friends_count": 2748, "statues_count": 85325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468587118592, "text": "weirdo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3230768529, "name": "VIVVY..❤️", "screen_name": "_naivivx2", "lang": "en", "location": "w/fonem", "create_at": date("2015-05-03"), "description": "strive for the best, never let a soul break you down! #live4ham #ISLAM", "followers_count": 682, "friends_count": 909, "statues_count": 10320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468603944961, "text": "Make em think.. left eye give em a wink.. free em up.. tear down the clink.. but make a place for da killers and rapers.. catch da vapors", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 743418402, "name": "Matye East", "screen_name": "BORNagainWARIOR", "lang": "en", "location": "Waterford, NY", "create_at": date("2012-08-07"), "description": "GamerTag PSN BORNagainWARRIOR Who will be first #DeathbySpaceJunk #TRUTHandSTRUGGLE I fear being judged by God not others. 18 years till #Apophis", "followers_count": 13976, "friends_count": 12829, "statues_count": 23384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterford, NY", "id": "004b1f2f32810adc", "name": "Waterford", "place_type": "city", "bounding_box": rectangle("-73.72126,42.775296 -73.662751,42.876755") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3678520, "cityName": "Waterford" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190468838821888, "text": "The lord is really holding me back cause I can't walk rn lord Jesus ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 238421686, "name": "56", "screen_name": "lexikillsslowly", "lang": "en", "location": "null", "create_at": date("2011-01-14"), "description": "Do justly. Love mercy. Walk humbly.", "followers_count": 725, "friends_count": 625, "statues_count": 5788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469228728320, "text": "Happy graduation day ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 947065604, "name": "Gilly", "screen_name": "gilly_potts", "lang": "en", "location": "paradise duh", "create_at": date("2012-11-13"), "description": "say it with me: Jill-ee • MP'16 • Seattle U 2020 • no I don't fill in my eyebrows", "followers_count": 538, "friends_count": 265, "statues_count": 28503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469245665280, "text": "Meu corretor é muito horrível, da um nervoso e uma vontade de jogar o celular na parede tudo ao mesmo tempo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 2397384865, "name": "Kenedy", "screen_name": "KenPeterson_", "lang": "pt", "location": "São Paulo - BR ✈ Stockton - CA", "create_at": date("2014-03-18"), "description": "Não cola em mim não, só tenho magoa e decepção no coração..", "followers_count": 948, "friends_count": 840, "statues_count": 11166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469597847553, "text": "✊��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 473646929, "name": "Brandon Agüero R.", "screen_name": "15BrandonR", "lang": "en", "location": "└A ", "create_at": date("2012-01-24"), "description": "Barcelona❤️ | Sueña sin limites.", "followers_count": 464, "friends_count": 568, "statues_count": 24121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469757329408, "text": "@Illogicalzen @draggle_kun @ShadowZael yes, very scripted, but they can be surprisingly manipulative if given the chance", "in_reply_to_status": 733190115644805121, "in_reply_to_user": 298095781, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298095781, 316205252, 486140804 }}, "user": { "id": 605008217, "name": "ⓈⒽⓄⓀⓊⒽⓄⓊ'Ⓢ ⒷⓊⓇⒼⒺⓇ", "screen_name": "cptngarlock", "lang": "en", "location": "⚠️danger⚠: ⚡⚡ high voltage ⚡⚡", "create_at": date("2012-06-10"), "description": "'The technical man must not be lost in his own technology; he must be able to appreciate life, and life is art, drama, music, and most importantly, people' #BLM", "followers_count": 493, "friends_count": 393, "statues_count": 37095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bailey's Crossroads, VA", "id": "c933b99b367f0b5b", "name": "Bailey's Crossroads", "place_type": "city", "bounding_box": rectangle("-77.147585,38.832027 -77.110316,38.861746") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5104088, "cityName": "Bailey's Crossroads" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469807681536, "text": "@kk_iller what kinda phone is that in ya avi...", "in_reply_to_status": 733187248200421376, "in_reply_to_user": 1377580548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1377580548 }}, "user": { "id": 50071159, "name": "KiLLa CaL", "screen_name": "iMooLikeACaL", "lang": "en", "location": "KCMO ✈AGG . Tx", "create_at": date("2009-06-23"), "description": "Not cocky... Just a confident nigga with humility... Luke 6:27-28", "followers_count": 2823, "friends_count": 933, "statues_count": 169536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190469853798401, "text": "Diana been a wild one, that's my girl bruh lol https://t.co/LvCBMLKoFy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30668975, "name": "Pat'Challa", "screen_name": "PorscheofSports", "lang": "en", "location": "Za Warudo", "create_at": date("2009-04-12"), "description": "21. Giants | Nuggets | Barcelona | One Piece is the GOAT | #BlackLivesMatter | I post anime vids from time to time, they're in my favs.", "followers_count": 3240, "friends_count": 2203, "statues_count": 229299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanham, MD", "id": "019ae9b86255a185", "name": "Lanham", "place_type": "city", "bounding_box": rectangle("-76.885508,38.934439 -76.821939,38.988666") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2445550, "cityName": "Lanham" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190470306668544, "text": "Getting sleeeeeepy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 821028307, "name": "Viper Frost.", "screen_name": "EverTheViper", "lang": "en", "location": "#FrostBites #Elviper #Ciper", "create_at": date("2012-09-12"), "description": "Folkie. Evil Little Rodent. Louisianimal. Sweet Moon Child. Sam Wilson enthusiast. Sipping sangria sassily. Whispering to @SaltyBarton. 「#Zishy #Vicon #Veri」", "followers_count": 1870, "friends_count": 948, "statues_count": 99807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190470558310400, "text": "I've never dealt with someone so rude in my entire life. Especially from a manager at a @McDonalds", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71026122 }}, "user": { "id": 436264803, "name": "morgan", "screen_name": "heymorgg", "lang": "en", "location": "null", "create_at": date("2011-12-13"), "description": "null", "followers_count": 299, "friends_count": 553, "statues_count": 12911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elyria, OH", "id": "57c9ac9db3df7f8b", "name": "Elyria", "place_type": "city", "bounding_box": rectangle("-82.178311,41.303092 -82.050455,41.418587") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3925256, "cityName": "Elyria" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190470659014656, "text": "& I ain't talking about no pussy .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 232305315, "name": "publicPlaya'", "screen_name": "_naeReal", "lang": "en", "location": "htx", "create_at": date("2010-12-30"), "description": "staying humble .", "followers_count": 1630, "friends_count": 791, "statues_count": 58884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190470675894273, "text": "@kcurr6 made my night tonight @ Progress Bar https://t.co/pjGvhXEpvk", "in_reply_to_status": -1, "in_reply_to_user": 267021421, "favorite_count": 0, "coordinate": point("-87.64905,41.9435"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 267021421 }}, "user": { "id": 176642704, "name": "Jade Sotomayor", "screen_name": "SotomayorJade", "lang": "en", "location": "Chicago", "create_at": date("2010-08-09"), "description": "From Original cast of RuPauls Drag Race on Vh1 and Logo... For booking info email JadeSotomayor83@gmail.com or follow me on Facebook/Insta @JadeSotomayor", "followers_count": 18685, "friends_count": 748, "statues_count": 5401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-19T00:00:04.000Z"), "id": 733190470952615937, "text": "Wind 0.0 mph ---. Barometer 30.180 in, Steady. Temperature 44.5 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 403, "friends_count": 294, "statues_count": 10604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552838144450560, "text": "Think I just ran 2 red lights cause I was thinking about the first one I ran��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1703297526, "name": "☯", "screen_name": "Shawwwwwwww", "lang": "en", "location": "RIP NEWT!", "create_at": date("2013-08-26"), "description": "19 and.......", "followers_count": 680, "friends_count": 410, "statues_count": 27939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552838333194240, "text": "@sagaquarius_ @CalVital no pimp daddy sage", "in_reply_to_status": 733551635448762369, "in_reply_to_user": 1353388226, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1353388226, 3098044646 }}, "user": { "id": 2333938598, "name": "Christopher Weber", "screen_name": "chriswaybos1", "lang": "en", "location": "null", "create_at": date("2014-02-08"), "description": "I'm an MC, and you're a knock off version of MC meals for a poor family", "followers_count": 199, "friends_count": 105, "statues_count": 562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552838643580928, "text": "But do I watch another episode or��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259082016, "name": "uıɹә", "screen_name": "ErinnPennington", "lang": "en", "location": "Troy University", "create_at": date("2011-02-28"), "description": "suh dude", "followers_count": 972, "friends_count": 599, "statues_count": 15908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, AL", "id": "7ae765412ef88940", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-87.721811,34.77898 -87.580289,34.8896") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1077, "countyName": "Lauderdale", "cityID": 126896, "cityName": "Florence" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552838807191553, "text": "True life: it's 3AM and I'm hungry,", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 354666331, "name": "harley ✝", "screen_name": "harleydearest", "lang": "en", "location": "336†", "create_at": date("2011-08-13"), "description": "In a world where you can be anything, be kind. // #iSurvived", "followers_count": 936, "friends_count": 755, "statues_count": 29901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eden, NC", "id": "d881b2cc29428b03", "name": "Eden", "place_type": "city", "bounding_box": rectangle("-79.828884,36.467618 -79.679769,36.542026") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37157, "countyName": "Rockingham", "cityID": 3720080, "cityName": "Eden" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552839650217984, "text": "@_ImAlwaysWright I feel u", "in_reply_to_status": 733552641611988992, "in_reply_to_user": 369814860, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 369814860 }}, "user": { "id": 350491497, "name": "jerry", "screen_name": "snoozingknob", "lang": "en", "location": "Muncie, IN", "create_at": date("2011-08-07"), "description": "sleep tight thugger | NYU '20", "followers_count": 385, "friends_count": 170, "statues_count": 15800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muncie, IN", "id": "01c14352f8d6ca6e", "name": "Muncie", "place_type": "city", "bounding_box": rectangle("-85.461887,40.121825 -85.320813,40.272656") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18035, "countyName": "Delaware", "cityID": 1851876, "cityName": "Muncie" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552840287748096, "text": "When you get to a party late and gotta catch up with everyone else. https://t.co/lng2d05nzW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 18596477, "name": "angel of thorns", "screen_name": "DaniiSavage", "lang": "en", "location": "Portland, Oregon", "create_at": date("2009-01-03"), "description": "twenty-three | caregiver | star trek | enfp | Snapchat: mcradsavage | @localcatmom", "followers_count": 1480, "friends_count": 589, "statues_count": 51187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552841667665923, "text": "A sonic blast RN ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1192273723, "name": "Lil Uzi Louie", "screen_name": "Waterboyyh2o", "lang": "en", "location": "null", "create_at": date("2013-02-17"), "description": "Forever Salty", "followers_count": 133, "friends_count": 69, "statues_count": 1989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-20T00:00:00.000Z"), "id": 733552841793511424, "text": "Let me dick you down babygirl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2450672750, "name": "$carecrow", "screen_name": "Jgalamgam9", "lang": "en", "location": "Bloomington, CA", "create_at": date("2014-04-17"), "description": "F.T.P", "followers_count": 428, "friends_count": 290, "statues_count": 25174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomington, CA", "id": "ad5f36a1f04723c0", "name": "Bloomington", "place_type": "city", "bounding_box": rectangle("-117.424405,34.033779 -117.373522,34.087738") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 607064, "cityName": "Bloomington" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552842456195072, "text": "Wind 0.0 mph ---. Barometer 29.772 in, Falling Rapidly. Temperature 68.3 °F. Rain today 0.09 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552842657566721, "text": "Wind 3.0 mph SE. Barometer 1018.26 mb, Falling. Temperature 55.5 °F. Rain today 0.02 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 30, "friends_count": 128, "statues_count": 14515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552842816937984, "text": "Barber texted me back a couple hours ago confirming my appointment for tomorrow . Literally texted that nigga Wednesday morning . No names", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 497209213, "name": "Alonso Beas", "screen_name": "Ayee_lonso", "lang": "en", "location": "IG: 3v_lonso", "create_at": date("2012-02-19"), "description": "| Stay humble | ✌ |", "followers_count": 435, "friends_count": 326, "statues_count": 15249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552842892410880, "text": "I'll remember this the next time he wants something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174094929, "name": "amb.", "screen_name": "ambersmalls_", "lang": "en", "location": "Texas", "create_at": date("2010-08-02"), "description": "T❤️", "followers_count": 537, "friends_count": 409, "statues_count": 21424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552843047587840, "text": "@clutchpri6 https://t.co/R6zCGEegqA", "in_reply_to_status": -1, "in_reply_to_user": 500582626, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 500582626 }}, "user": { "id": 1658801502, "name": "Isa Bey", "screen_name": "BeyIzzy", "lang": "en", "location": "United States of America", "create_at": date("2013-08-09"), "description": "To love this life is to live it.", "followers_count": 938, "friends_count": 361, "statues_count": 26091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552843555135488, "text": "@gelicaa14 So like, when did I say I love you?����", "in_reply_to_status": 733552433901621252, "in_reply_to_user": 1004878628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1004878628 }}, "user": { "id": 1381129430, "name": "brandy", "screen_name": "braaanndyy", "lang": "en", "location": "null", "create_at": date("2013-04-25"), "description": "null", "followers_count": 366, "friends_count": 269, "statues_count": 26062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552843588689922, "text": "@Bibi_Borja <3 red . Whatru allergic ?", "in_reply_to_status": 733552117047119873, "in_reply_to_user": 52768321, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52768321 }}, "user": { "id": 111244921, "name": "Mr. Fantastic", "screen_name": "ErikDannyThomas", "lang": "en", "location": "954 SOGNARR", "create_at": date("2010-02-03"), "description": "No Small Coincidences", "followers_count": 656, "friends_count": 99, "statues_count": 84279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552843689320450, "text": "https://t.co/C59OSqTqMe @JimClemente @lisazambetti2 @laurarichards99", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1180277142, 4886789832, 139130055 }}, "user": { "id": 237698494, "name": "Tammy Young Prejusa", "screen_name": "tamprej", "lang": "en", "location": "Grand Prairie, TEXAS ", "create_at": date("2011-01-13"), "description": "Two great kids. Widow. I love to sing. I am obsessed with the WEATHER. Big fan of actor Thomas Gibson. ☺", "followers_count": 961, "friends_count": 1022, "statues_count": 36496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844050038784, "text": "@lu_times2 do itttttttt. I support 100%", "in_reply_to_status": 733552723832905728, "in_reply_to_user": 305749787, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 305749787 }}, "user": { "id": 281214578, "name": "Ari✨", "screen_name": "arinichelle_", "lang": "en", "location": "null", "create_at": date("2011-04-12"), "description": "Alcorn Volleyball #ASU18.... not just another pretty face", "followers_count": 3204, "friends_count": 2815, "statues_count": 24188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bartlett, TN", "id": "c4ad9757e682a583", "name": "Bartlett", "place_type": "city", "bounding_box": rectangle("-89.887992,35.186443 -89.741766,35.274531") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4703440, "cityName": "Bartlett" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844138123269, "text": "@tectrump Thanks for following me, may our #tweets #enrich & #inspire many more. #sharedlearning #growtogether #bewell #prosper", "in_reply_to_status": -1, "in_reply_to_user": 702436554363502592, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "tweets", "enrich", "inspire", "sharedlearning", "growtogether", "bewell", "prosper" }}, "user_mentions": {{ 702436554363502592 }}, "user": { "id": 16060248, "name": "Baskaran Ambalavanan", "screen_name": "baski_LA", "lang": "en", "location": "Orange County,CA", "create_at": date("2008-08-30"), "description": "Business focused HR leader, Analytics evangelist,delivering everyday excellence, Intellectually curious, MBA, SPHR-CA GPHR,SHRM-SCP. All Tweets are my own.", "followers_count": 3591, "friends_count": 3354, "statues_count": 3023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844469473283, "text": "All my friends are princesses we keep it whipped and creamy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 479729770, "name": "Nad", "screen_name": "N_Swizzle", "lang": "en", "location": "where the flowers grow", "create_at": date("2012-01-31"), "description": "18//Frida enthusiast//Insta:Nadiasuarz", "followers_count": 1273, "friends_count": 852, "statues_count": 72614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seaside, CA", "id": "89bac4213b1b5525", "name": "Seaside", "place_type": "city", "bounding_box": rectangle("-121.860374,36.596815 -121.789746,36.654798") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 670742, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844532391938, "text": "Weirdest shop ever but hey it's Silicon Valley right? #california @ Palo Alto, California https://t.co/11nAaBv6RN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.138,37.4293"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "california" }}, "user": { "id": 105812548, "name": "Francisco Suzuki V.", "screen_name": "FSuzukiVidal", "lang": "en", "location": "London, United Kingdom", "create_at": date("2010-01-17"), "description": "Scientist • Experimental physicist • DIY enthusiast • music lover", "followers_count": 57, "friends_count": 59, "statues_count": 393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palo Alto, CA", "id": "3ad0f706b3fa62a8", "name": "Palo Alto", "place_type": "city", "bounding_box": rectangle("-122.190523,37.362824 -122.097537,37.465918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 655282, "cityName": "Palo Alto" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844754681856, "text": "@CoreyHickson 100%. a-ok", "in_reply_to_status": 733552597844402176, "in_reply_to_user": 861530450, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 861530450 }}, "user": { "id": 165673510, "name": "Abernathy", "screen_name": "Matt_Abernathy", "lang": "en", "location": "Dallas, TX", "create_at": date("2010-07-11"), "description": "Geek. Nerd. Critter. Hooman. D&D nut. Graphic Designer. Certified Goofball.", "followers_count": 1249, "friends_count": 505, "statues_count": 14920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552844922486784, "text": "Wind 0.0 mph ---. Barometer 30.047 in, Falling. Temperature 56.6 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845065064452, "text": "https://t.co/LeEdMVL6B2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 190617423, "name": "Jared Shepherd", "screen_name": "JayDeAnDre", "lang": "en", "location": "Ruston, LA", "create_at": date("2010-09-14"), "description": "LA Tech Track & Field. Growing Daily...", "followers_count": 1565, "friends_count": 819, "statues_count": 51616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845148979202, "text": "you know I gives 0 fucks. https://t.co/aTxvdOU0lV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 352852490, "name": "von", "screen_name": "VonBroo", "lang": "en", "location": "757. ", "create_at": date("2011-08-10"), "description": "SC: von_broo", "followers_count": 3191, "friends_count": 2673, "statues_count": 74234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845258006528, "text": "05/20@03:00 - Temp 48.0F, WC 48.0F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.197in, Rising slowly. Rain 0.00in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845555793922, "text": "Ripley SW Limestone Co. Temp: 60.8°F Wind:0.7mph Pressure: 994.4mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845589336065, "text": "������������ You ready to get them #dollasdollas tho, right? https://t.co/PtROUQ2fhw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dollasdollas" }}, "user": { "id": 912964466, "name": "Natina Renee", "screen_name": "NatinaRenee", "lang": "en", "location": "null", "create_at": date("2012-10-29"), "description": "..made perfect by my imperfection..I am, all that I am..", "followers_count": 157, "friends_count": 339, "statues_count": 1384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845601939456, "text": "I don't care what's in your hair, I just wanna know what's on your mind", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1678267712, "name": "A$AP CAMMY", "screen_name": "cam___walker", "lang": "en", "location": "satx", "create_at": date("2013-08-17"), "description": "#TTU20 #TTUHSC24// sc: ccameronwalkerr // trying to get by with Jesus on my side", "followers_count": 343, "friends_count": 273, "statues_count": 4397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845639680000, "text": "Be great Icegod ��❗️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3183453882, "name": "Young Jefe ✊", "screen_name": "IcegodCiantae2", "lang": "en", "location": "null", "create_at": date("2015-05-02"), "description": "Icegod Rose again.. 18.. Glo ⛽ Wait on me ♻️", "followers_count": 435, "friends_count": 298, "statues_count": 8313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Topeka, KS", "id": "835f1b2948575c51", "name": "Topeka", "place_type": "city", "bounding_box": rectangle("-95.809606,38.971524 -95.571859,39.103634") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20177, "countyName": "Shawnee", "cityID": 2071000, "cityName": "Topeka" } }
+{ "create_at": datetime("2016-05-20T00:00:01.000Z"), "id": 733552845899730944, "text": "Temp 45.6° Hi/Lo 51.2/45.6 Rng 5.6° WC 45.6° Hmd 95% Rain 0.01\" Storm 0.29\" BAR 30.157 Steady DP 44.3° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552846885421056, "text": "y'all expect some shit next friday up on this bih. bc honestly im still a childish mf on this app.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.98342188,33.19128623"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2772667649, "name": "j", "screen_name": "jxcobstevens", "lang": "en", "location": "Home", "create_at": date("2014-09-16"), "description": "| track and field |", "followers_count": 189, "friends_count": 357, "statues_count": 2119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moncks Corner, SC", "id": "1a62e3200400d208", "name": "Moncks Corner", "place_type": "city", "bounding_box": rectangle("-80.033755,33.179555 -79.968694,33.213065") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45015, "countyName": "Berkeley", "cityID": 4547275, "cityName": "Moncks Corner" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552846910558210, "text": "LMAOLMAOLMAOLMAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 629021538, "name": "〽️Smith-Rod〽️", "screen_name": "YoungKhaalifaa", "lang": "en", "location": "null", "create_at": date("2012-07-06"), "description": "We Dem Boyz", "followers_count": 406, "friends_count": 358, "statues_count": 8670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552847002853377, "text": "@safaaskander yess please!!", "in_reply_to_status": 733548965832970240, "in_reply_to_user": 331147927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 331147927 }}, "user": { "id": 24103768, "name": "Angie Balas", "screen_name": "angieebalas", "lang": "en", "location": "null", "create_at": date("2009-03-12"), "description": "Carlsbad CA ♎️ SDSU", "followers_count": 249, "friends_count": 197, "statues_count": 2052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlsbad, CA", "id": "01d4e349481265e8", "name": "Carlsbad", "place_type": "city", "bounding_box": rectangle("-117.359298,33.060615 -117.216549,33.182353") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 611194, "cityName": "Carlsbad" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552847304826881, "text": "And put them on a computer somewhere just in case you lose break or get a phone stolen https://t.co/NWHyya2hqX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447330194, "name": "pris", "screen_name": "priscillxxrose", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "snapchat. priscillaaa", "followers_count": 424, "friends_count": 83, "statues_count": 8964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cameron Park, CA", "id": "0f2405443201a34d", "name": "Cameron Park", "place_type": "city", "bounding_box": rectangle("-121.033013,38.65476 -120.924554,38.726092") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6017, "countyName": "El Dorado", "cityID": 610256, "cityName": "Cameron Park" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848357580800, "text": "@lordvlogdemort oh, I know he will. His time is limited.", "in_reply_to_status": 733552700139261959, "in_reply_to_user": 3438854951, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3438854951 }}, "user": { "id": 21050582, "name": "SourNurse", "screen_name": "thesouRNurse", "lang": "en", "location": "San Diego, CA", "create_at": date("2009-02-16"), "description": "Serving you healthcare and geek on a daily basis. I am THE SourNurse. Accept no substitutions. Made partially from concentrate.", "followers_count": 872, "friends_count": 1083, "statues_count": 73519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848454082562, "text": "Quartz\nTaiwan's new president is sworn in...\nhttps://t.co/5LRhIMS8s5 Small Taiwan gets it! Maybe U.S. will. #NeedSmartLeadership #Congrats", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NeedSmartLeadership", "Congrats" }}, "user": { "id": 275115688, "name": "SharronRobinson", "screen_name": "MsRoneRoyalty1", "lang": "en", "location": "Tennessee", "create_at": date("2011-03-31"), "description": "I'm a Beautiful Black American Business Woman who's for the best interest of The American masses, whoever,wherever& however. Where there's a will, there's a way", "followers_count": 615, "friends_count": 805, "statues_count": 11424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chattanooga, TN", "id": "3b3916ee31cfc9e6", "name": "Chattanooga", "place_type": "city", "bounding_box": rectangle("-85.404424,34.983674 -85.080704,35.200235") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848496025600, "text": "Omg I'm not trynq go in this house to get bitched at rn, i need to get the fuck outta here.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198258363, "name": "Mmmm♐", "screen_name": "meag_97", "lang": "en", "location": "New Jersey, USA", "create_at": date("2013-11-27"), "description": "Portuguesa que não tenha nenhum arrependimento, Nós vivemos e aprendemos.️ ✝️", "followers_count": 213, "friends_count": 165, "statues_count": 14298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Iselin, NJ", "id": "812a6cfdd31b4a5f", "name": "Iselin", "place_type": "city", "bounding_box": rectangle("-74.337166,40.553141 -74.292843,40.585914") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3434470, "cityName": "Iselin" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848508588032, "text": "Reminds me I need a new pillow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3292586819, "name": "Jonathan Frias", "screen_name": "Livinglegit", "lang": "en", "location": "New York City", "create_at": date("2015-05-21"), "description": "Me and you is not alike.", "followers_count": 64, "friends_count": 35, "statues_count": 2612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848886063104, "text": "Temp: 67.4°F Wind:0.4mph Pressure: 29.864hpa Falling Rapidly Rain Today 0.13in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 68, "friends_count": 24, "statues_count": 64620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552848978378752, "text": "Edible six-pack rings let you have your beer and help the environment, too https://t.co/3I7pKfaxvi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17648, "friends_count": 17816, "statues_count": 72095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552849322283008, "text": "Nah, if I'm ever serious around you, I probably don't really like you or you just pissed me off", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 212094748, "name": "Tay B", "screen_name": "taypandaa", "lang": "en", "location": "null", "create_at": date("2010-11-04"), "description": "she don't believe in God but her shoes Christian", "followers_count": 640, "friends_count": 261, "statues_count": 40615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552849460695042, "text": "hate going to bed in a bad mood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1052397319, "name": "Alicia", "screen_name": "ayoalliishaa", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-01-01"), "description": "@bdevera_'s Twinkie", "followers_count": 361, "friends_count": 138, "statues_count": 3873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Foothills, CA", "id": "3320a79a535d4896", "name": "East Foothills", "place_type": "city", "bounding_box": rectangle("-121.841742,37.367878 -121.791855,37.397098") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 620598, "cityName": "East Foothills" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552849481670656, "text": "@Sheey___ hbd b! https://t.co/jaZvEiPf2O", "in_reply_to_status": -1, "in_reply_to_user": 3285982896, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 3285982896 }}, "user": { "id": 3139378321, "name": "Malcolm", "screen_name": "malcolmm2_", "lang": "en", "location": "K-town", "create_at": date("2015-04-04"), "description": "null", "followers_count": 337, "friends_count": 459, "statues_count": 3139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552849645232128, "text": "Wind 0.0 mph ---. Barometer 30.092 in, Falling slowly. Temperature 53.7 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 19957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-20T00:00:02.000Z"), "id": 733552850299539457, "text": "3 am thoughts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 719386698338054145, "name": "Charise Ramil", "screen_name": "chariseramiiil", "lang": "en", "location": "cloud 9 ", "create_at": date("2016-04-10"), "description": "i'm leaving bc you never asked me to stay", "followers_count": 170, "friends_count": 232, "statues_count": 400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552850685427714, "text": "@NeriahMarieee no", "in_reply_to_status": 733479424595759104, "in_reply_to_user": 291234933, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 291234933 }}, "user": { "id": 434984330, "name": "(:", "screen_name": "No_regrets54", "lang": "en", "location": "Clarksville, TN", "create_at": date("2011-12-12"), "description": "student athlete | Wayland baptist University football | no matter what you have to be better then you were yesterday | deuteronomy 31:6| Rest Easy Angelica", "followers_count": 716, "friends_count": 532, "statues_count": 35180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainview, TX", "id": "6513b0430da3e9ac", "name": "Plainview", "place_type": "city", "bounding_box": rectangle("-101.760027,34.142451 -101.643481,34.235002") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48189, "countyName": "Hale", "cityID": 4857980, "cityName": "Plainview" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552850735796224, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Steady. Temperature 69.6 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851222298626, "text": "HAPPY 21ST TO @chrisfaenza !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! https://t.co/4WHIcAYa6Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 896405821 }}, "user": { "id": 397054449, "name": "Nick", "screen_name": "Nick_A_Gonzo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-10-23"), "description": "ASÜ", "followers_count": 474, "friends_count": 317, "statues_count": 56086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851331342336, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":49.5°F Wind:0.0mph Pressure: 30.09hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 18, "statues_count": 318516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851436216320, "text": "Most definitely https://t.co/REAXJCoACR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 344851171, "name": "Ev⚜", "screen_name": "JustDoItEverett", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-07-29"), "description": "Let Go And Let God || 6'5 || IG: JustDoItEv || SC: JustDoItEvv", "followers_count": 2298, "friends_count": 512, "statues_count": 109106 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glenn Heights, TX", "id": "63634cb1acb5eff7", "name": "Glenn Heights", "place_type": "city", "bounding_box": rectangle("-96.891055,32.518192 -96.821826,32.568093") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829840, "cityName": "Glenn Heights" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851562029056, "text": "Lol https://t.co/mRW4RBE13j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 426237110, "name": "⚡️Kodack White⚡️", "screen_name": "1stLeg_rookie", "lang": "en", "location": "Dallas Tx", "create_at": date("2011-12-01"), "description": "|Nigerian||Entrepreneur| #PVAMU19 Track and Field L.I.T ~ Life In Threads check out the website below", "followers_count": 1724, "friends_count": 1447, "statues_count": 21435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851692052482, "text": "who has time for this https://t.co/ldxCnRUDca", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2202371132, "name": "cheyane ☮", "screen_name": "cheyanedecotea1", "lang": "en", "location": "Belcourt, ND", "create_at": date("2013-11-18"), "description": "In the middle of difficulty lies opportunity. // positive vibes ⛅️...and I don't give a damn, I've felt the ground before", "followers_count": 937, "friends_count": 800, "statues_count": 36854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dickinson, ND", "id": "ce504481276c1577", "name": "Dickinson", "place_type": "city", "bounding_box": rectangle("-102.8322,46.84707 -102.726459,46.913366") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38089, "countyName": "Stark", "cityID": 3819620, "cityName": "Dickinson" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552851872415745, "text": "Bed time kitty cuddles are my favorite", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129436036, "name": "Кαуℓєєвυgg", "screen_name": "KayleeBugg19", "lang": "en", "location": "Morgantown, WV", "create_at": date("2010-04-04"), "description": "Though I am broken, grace is my welcome home to You. • Lover of Jesus, YL, & animals • cat momma • I don't know what I want to do with my life • WVU", "followers_count": 585, "friends_count": 798, "statues_count": 20634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morgantown, WV", "id": "bad356f8cba08b76", "name": "Morgantown", "place_type": "city", "bounding_box": rectangle("-79.987463,39.584088 -79.900212,39.674541") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54061, "countyName": "Monongalia", "cityID": 5455756, "cityName": "Morgantown" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552852006670337, "text": "Ew Tb to when I went to school with no make up and nappy hair", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587698239, "name": "TastyTej", "screen_name": "Teeeeja_Vu", "lang": "en", "location": "somewhere not replying ", "create_at": date("2012-05-22"), "description": "Hopeless Romantic LSU", "followers_count": 1586, "friends_count": 738, "statues_count": 115972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cloverleaf, TX", "id": "610a668f55d64af2", "name": "Cloverleaf", "place_type": "city", "bounding_box": rectangle("-95.193084,29.755365 -95.14658,29.82664") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4815628, "cityName": "Cloverleaf" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552852417675264, "text": "@jchavez50rr music is actually really good��", "in_reply_to_status": -1, "in_reply_to_user": 4172153832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4172153832 }}, "user": { "id": 2796426031, "name": "{BabyCait}", "screen_name": "caitlynpires10", "lang": "en", "location": "Rogue River, OR", "create_at": date("2014-09-07"), "description": "#10⚽️ 4•3•16 //ORCA", "followers_count": 102, "friends_count": 70, "statues_count": 1953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rogue River, OR", "id": "b77a460e2ad2912d", "name": "Rogue River", "place_type": "city", "bounding_box": rectangle("-123.182563,42.420171 -123.157223,42.44767") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4163450, "cityName": "Rogue River" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552852740624384, "text": "This Justin timberlake video is awful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39841031, "name": "Jasen Comstock", "screen_name": "JasenComstock", "lang": "en", "location": "Washington DC", "create_at": date("2009-05-13"), "description": "only guy at the Metallica concert in a Metallica t-shirt", "followers_count": 316, "friends_count": 766, "statues_count": 15633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552852761632774, "text": "Happy birthday baby !������ I couldn't write everything here so it's on Instagram �� @mur_angelica https://t.co/LRFD3A8IOC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2290679450 }}, "user": { "id": 2426549676, "name": "Carlos flores", "screen_name": "flores_carlos01", "lang": "en", "location": "null", "create_at": date("2014-04-03"), "description": "null", "followers_count": 169, "friends_count": 128, "statues_count": 215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552852967129088, "text": "3e84d68400p93F3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-171.63265,52.962332"), "retweet_count": 0, "lang": "pt", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 21, "friends_count": 0, "statues_count": 23124 }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552853189451776, "text": "You're only twitter famous because people don't know you're actually an unattractive airhead in real life!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1477444315, "name": "VIEWS.", "screen_name": "Bberry_21", "lang": "en", "location": "null", "create_at": date("2013-06-02"), "description": "Some days I'm extremely focused on my career & development , other days I just wanna say fuck it, be a house wife, and bake shit .", "followers_count": 575, "friends_count": 329, "statues_count": 5809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552853252329472, "text": "Wind 0.0 mph ---. Barometer 30.19 in, Steady. Temperature 48.6 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 122, "statues_count": 159921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552853302673408, "text": "Drinking a Big Swell IPA by @mauibrewingco at @hiltongrandvac — https://t.co/heZbunEk1G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.837,21.284"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 65375829, 52470780 }}, "user": { "id": 15865964, "name": "Nicknowhere", "screen_name": "Nicknowhere", "lang": "en", "location": "iPhone: 21.366819,-157.939178", "create_at": date("2008-08-15"), "description": "Punk rock, beer & football - Hips 4 life", "followers_count": 413, "friends_count": 1692, "statues_count": 9345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552853923430400, "text": "The shit people do for attention ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2493633511, "name": "Fae", "screen_name": "faerooskie", "lang": "en", "location": "Houston, TX", "create_at": date("2014-05-13"), "description": "on another level.", "followers_count": 203, "friends_count": 163, "statues_count": 128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552854527447040, "text": "Wind 1.6 mph ENE. Barometer 29.928 in, Falling slowly. Temperature 59.0 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8568 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-20T00:00:03.000Z"), "id": 733552854560956421, "text": "Peeped a long time ago lmao it's just not for me to call you out on it ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342398270, "name": "JULY 5♋", "screen_name": "Hi_imnautica", "lang": "en", "location": "null", "create_at": date("2011-07-25"), "description": "Follow me on IG: Hi_ImNautica❤", "followers_count": 1110, "friends_count": 973, "statues_count": 22803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552856062513152, "text": "@ZzzMetroMan say no to sociapism. We might turn into Russia", "in_reply_to_status": 733551953075003393, "in_reply_to_user": 706696508217790464, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 706696508217790464 }}, "user": { "id": 53179044, "name": "Carlos Moreno Jr.", "screen_name": "carlosmorenojr", "lang": "en", "location": "Los Angeles", "create_at": date("2009-07-02"), "description": "Actor - Dream big! Grow by living out of your comfort zone. Love unconditionally. Forgive. Don't give up. - It's not over, until YOU WIN!", "followers_count": 320, "friends_count": 373, "statues_count": 1031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552856423239685, "text": "@jeannelululao bet", "in_reply_to_status": 733552691561959424, "in_reply_to_user": 35137797, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35137797 }}, "user": { "id": 207203414, "name": "Jimmy Peaches.", "screen_name": "OGBEARD", "lang": "en", "location": "around...", "create_at": date("2010-10-24"), "description": "A coworker told me bitches on twitter love beards so im here...", "followers_count": 24977, "friends_count": 5870, "statues_count": 427107 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552856733614081, "text": "We IN here #saunatime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "saunatime" }}, "user": { "id": 3021852062, "name": "K_SMOV", "screen_name": "fiapoleo_", "lang": "en", "location": "Tonga, AK", "create_at": date("2015-02-15"), "description": "| Mother of 4 • Independent •TeamLDS • Famili1st • Tongan • AlaskanGrown • It's Just Twitter ☕️ |", "followers_count": 605, "friends_count": 811, "statues_count": 20659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552856796520450, "text": "Got em hyphy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2172648210, "name": "EL L", "screen_name": "patonjr96", "lang": "en", "location": "null", "create_at": date("2013-11-03"), "description": "Is this the Krusty Krab?!", "followers_count": 544, "friends_count": 493, "statues_count": 23558 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552856993685507, "text": "Dominator https://t.co/mG9Y6cJOYN #dominator #parcdattraction #doswell #virginia #unitedstates https://t.co/RBWZdgbwAz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.443474,37.841297"), "retweet_count": 0, "lang": "ro", "is_retweet": false, "hashtags": {{ "dominator", "parcdattraction", "doswell", "virginia", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 957, "friends_count": 1289, "statues_count": 5534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginie, USA", "id": "5635c19c2b5078d1", "name": "Virginie", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51085, "countyName": "Hanover" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552857195024385, "text": "I'm not that jaded baby lately I'm just misunderstood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242472809, "name": "Snire 1-er", "screen_name": "snireoner", "lang": "en", "location": "bay area", "create_at": date("2011-01-24"), "description": "Chris | 20 | Bay Area", "followers_count": 641, "friends_count": 595, "statues_count": 7472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552857429868544, "text": "Some cake sounds good right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1098995714, "name": "Mariota", "screen_name": "Miggy_Solis", "lang": "en", "location": "Always Strive & Prosper", "create_at": date("2013-01-17"), "description": "BOOM! NASTY!", "followers_count": 477, "friends_count": 464, "statues_count": 21663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hacienda Heights, CA", "id": "47dbb2e661aa176c", "name": "Hacienda Heights", "place_type": "city", "bounding_box": rectangle("-118.037546,33.973234 -117.927186,34.031527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 631596, "cityName": "Hacienda Heights" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552857601822721, "text": "Put my bitches in new pair heals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2334695192, "name": "OG JOSH™", "screen_name": "ImToMuch_UK", "lang": "en", "location": "at the bar ", "create_at": date("2014-02-08"), "description": "21 and single from DDD. Get to know me and you won't regret it!", "followers_count": 653, "friends_count": 646, "statues_count": 12616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowlett, TX", "id": "015d51094da3e975", "name": "Rowlett", "place_type": "city", "bounding_box": rectangle("-96.597245,32.865765 -96.49164,32.982751") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4863572, "cityName": "Rowlett" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552857710895105, "text": "@dtabsss ����������", "in_reply_to_status": 733546566305206272, "in_reply_to_user": 246609536, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 246609536 }}, "user": { "id": 3286048046, "name": "Brooke Clymer", "screen_name": "Queencooooks", "lang": "en", "location": "null", "create_at": date("2015-07-20"), "description": "null", "followers_count": 122, "friends_count": 106, "statues_count": 918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl City, HI", "id": "6ded3cbcea7e1f34", "name": "Pearl City", "place_type": "city", "bounding_box": rectangle("-157.992788,21.369586 -157.934172,21.431996") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1562600, "cityName": "Pearl City" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552857807392768, "text": "@Beauty_Brains11 ������", "in_reply_to_status": 733552801406550016, "in_reply_to_user": 23155814, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 23155814 }}, "user": { "id": 2434417207, "name": "Claytanna6k", "screen_name": "clayt600", "lang": "en", "location": "7️⃣5️⃣7️⃣", "create_at": date("2014-04-08"), "description": "SELF MADE MILLIONAIRE #PRITOWN", "followers_count": 1266, "friends_count": 796, "statues_count": 25729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-05-20T00:00:04.000Z"), "id": 733552858730139648, "text": "Selena really shut down IG and a whole province.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 223056225, "name": "James Sloan•I'Cosche", "screen_name": "CajunJamesSloan", "lang": "en", "location": "Shreveport", "create_at": date("2010-12-04"), "description": "I' inesplorato--- LSUS Baseball Assistant Coach. Sinner saved by grace through faith in Jesus Christ. Focus on the process. Snapchat: ohcanada10", "followers_count": 1073, "friends_count": 989, "statues_count": 11780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552859090837505, "text": "I never sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2769062568, "name": "Kaitlyn", "screen_name": "kaitlynpaige217", "lang": "en", "location": "In bed watching Netflix", "create_at": date("2014-08-25"), "description": "I don't care about anything.", "followers_count": 152, "friends_count": 138, "statues_count": 1615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jefferson, OR", "id": "bd1354fa45521b08", "name": "Jefferson", "place_type": "city", "bounding_box": rectangle("-123.014237,44.707425 -122.994708,44.727508") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4137250, "cityName": "Jefferson" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552859107581953, "text": "What y'all up doing?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 277465372, "name": "June 24th ✨", "screen_name": "TreN0Songz", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-04-05"), "description": "This moment will just be another story someday. Animal Science Major. #PVAMU19 SC: Trevooooo", "followers_count": 1137, "friends_count": 600, "statues_count": 34844 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552859938086912, "text": "Temp: 44.6°F - Dew Point: 39.5° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.14in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 12, "statues_count": 17932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552860101677057, "text": "Love bae @kristenshock_ https://t.co/3xBzzHWbhw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 357816733 }}, "user": { "id": 799529738, "name": "riley", "screen_name": "_RileyWalker", "lang": "en", "location": "null", "create_at": date("2012-09-02"), "description": "null", "followers_count": 485, "friends_count": 488, "statues_count": 4467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Knoxville, TN", "id": "6565298bcadb82a1", "name": "Knoxville", "place_type": "city", "bounding_box": rectangle("-84.19397,35.831436 -83.733713,36.133505") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47093, "countyName": "Knox", "cityID": 4740000, "cityName": "Knoxville" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552860151963649, "text": "I wish Aaliyah was ugly lol.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 233328922, "name": "Ayo", "screen_name": "Travis_McCorter", "lang": "en", "location": "Not Wit The Hoes", "create_at": date("2011-01-02"), "description": "Heights Side | Kent State | Salute Me or Shoot Me | #ThompsonThompson2040", "followers_count": 1212, "friends_count": 1107, "statues_count": 70858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland Heights, OH", "id": "aa7defe13028d41f", "name": "Cleveland Heights", "place_type": "city", "bounding_box": rectangle("-81.603358,41.482742 -81.529651,41.545274") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916014, "cityName": "Cleveland Heights" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552860357517312, "text": "@dillanson how light caries on endlessly, even after death", "in_reply_to_status": 733552305509801985, "in_reply_to_user": 1674420582, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1674420582 }}, "user": { "id": 618518891, "name": "DAD", "screen_name": "scottemartin5", "lang": "en", "location": "probably in your DM's", "create_at": date("2012-06-25"), "description": "No that's not my kid.. but yes that is my cute ass puppy.", "followers_count": 530, "friends_count": 154, "statues_count": 6374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central Point, OR", "id": "ff96f35b5ab7ad8c", "name": "Central Point", "place_type": "city", "bounding_box": rectangle("-122.93452,42.361221 -122.883112,42.39622") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4112400, "cityName": "Central Point" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552860483346432, "text": "Honestly can't wait until I kick it with my blondie this entire weekend ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1240058358, "name": "little miss", "screen_name": "littlemisssani", "lang": "en", "location": "null", "create_at": date("2013-03-03"), "description": "Frankly my dear, I dont give a damn ♋︎", "followers_count": 379, "friends_count": 244, "statues_count": 14554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552860558819329, "text": "@_DopeDani ��", "in_reply_to_status": 733552605922615296, "in_reply_to_user": 340031191, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 340031191 }}, "user": { "id": 317559432, "name": "2am conner", "screen_name": "IHATN", "lang": "en", "location": "texas", "create_at": date("2011-06-14"), "description": "crafty conner , what a honor", "followers_count": 3791, "friends_count": 792, "statues_count": 214593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552861762572289, "text": "1:30 phone calls will wake you up and keep u awake :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 752528388, "name": "mandyf", "screen_name": "_mandyf_", "lang": "en", "location": "null", "create_at": date("2012-08-11"), "description": "wyd", "followers_count": 1320, "friends_count": 637, "statues_count": 19429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Callaway, FL", "id": "f656cb58484c4f15", "name": "Callaway", "place_type": "city", "bounding_box": rectangle("-85.605722,30.115566 -85.538717,30.167444") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1209725, "cityName": "Callaway" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552861791932417, "text": "@BreeeMichellee tbh i just need my ends trimmed first and im back��", "in_reply_to_status": 733552333829758977, "in_reply_to_user": 4852057694, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4852057694 }}, "user": { "id": 113710310, "name": "lilTRINA", "screen_name": "typicaltaryn", "lang": "en", "location": "unavailable ", "create_at": date("2010-02-12"), "description": "if you dont suck toes why follow me?", "followers_count": 1067, "friends_count": 492, "statues_count": 113185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Pasadena, CA", "id": "e83d71e042bbfcf7", "name": "South Pasadena", "place_type": "city", "bounding_box": rectangle("-118.178345,34.098569 -118.134694,34.125742") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673220, "cityName": "South Pasadena" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552861846495232, "text": "Wind 0.0 mph ---. Barometer 30.178 in, Steady. Temperature 53.8 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 403, "friends_count": 294, "statues_count": 10629 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862068772864, "text": "temperature down 58°F -> 55°F\nhumidity up 67% -> 71%\nwind 13mph -> 17mph", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.99,37.55"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 145060068, "name": "Fremont Weather", "screen_name": "_FremontCA", "lang": "en", "location": "Fremont, CA", "create_at": date("2010-05-17"), "description": "Weather updates, forecast, warnings and information for Fremont, CA. Sources: http://OpenWeatherMap.org, NOAA, USGS.", "followers_count": 107, "friends_count": 1, "statues_count": 23072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862093942784, "text": "Such a buzzkill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1648811148, "name": "Nick", "screen_name": "NickAkutagawa", "lang": "en", "location": "null", "create_at": date("2013-08-05"), "description": "must be the curse.⁶", "followers_count": 645, "friends_count": 473, "statues_count": 17603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862177857536, "text": "For real tho, we have cleaning a field down to a muh fuckin science", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630034964, "name": "LMB", "screen_name": "andrewramos13", "lang": "en", "location": "null", "create_at": date("2012-07-08"), "description": "I love bass fishing and cons", "followers_count": 510, "friends_count": 440, "statues_count": 3172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menifee, CA", "id": "00ec60e4304ee8bf", "name": "Menifee", "place_type": "city", "bounding_box": rectangle("-117.258067,33.618318 -117.136098,33.757777") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646842, "cityName": "Menifee" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862265892866, "text": "Yeah I listen to some hood shit and I drive with my seat all the way leaned back but deep down all I want is a good novel and some coffee.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2430797521, "name": "Samantha Skinner", "screen_name": "SamanthaSkinn17", "lang": "en", "location": "null", "create_at": date("2014-04-06"), "description": "just trying to get that old Cadi with the top down swangin | ♏️", "followers_count": 485, "friends_count": 629, "statues_count": 3360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862555316225, "text": "the cuddle urge is real", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486264816, "name": "Кαιℓz♡", "screen_name": "KailyMarie2", "lang": "en", "location": "The Brook ", "create_at": date("2012-02-07"), "description": "♓️ | 20 | coffee enthusiast", "followers_count": 843, "friends_count": 552, "statues_count": 53199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bolingbrook, IL", "id": "0991d757989cef56", "name": "Bolingbrook", "place_type": "city", "bounding_box": rectangle("-88.18516,41.630746 -88.0269,41.735932") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1707133, "cityName": "Bolingbrook" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862823743488, "text": "������ https://t.co/3N3nq5qiFG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 309958880, "name": "BULLETPROOF", "screen_name": "xoxokellylynn_", "lang": "en", "location": "PV17", "create_at": date("2011-06-02"), "description": "ΔΣΘ • HB • SPR' 16 • 23", "followers_count": 1654, "friends_count": 1274, "statues_count": 62030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552862853103617, "text": "So happy������ #issues #headspace #allday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "issues", "headspace", "allday" }}, "user": { "id": 49911240, "name": "Kelsey Brongil", "screen_name": "KelseyBrongil", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-06-23"), "description": "Seattle➡️LA. Wardrobe Stylist.", "followers_count": 126, "friends_count": 169, "statues_count": 2745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-20T00:00:05.000Z"), "id": 733552863033483267, "text": "@camilacabello97 wise words of MJ", "in_reply_to_status": 733536879363006464, "in_reply_to_user": 739784130, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 739784130 }}, "user": { "id": 137942007, "name": "5HAllyKat", "screen_name": "bkathy", "lang": "en", "location": "San Antonio, TX", "create_at": date("2010-04-27"), "description": "everything happens for a reason", "followers_count": 312, "friends_count": 409, "statues_count": 23723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552863440343041, "text": "'You don't matter that much'������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 933770514, "name": "Cambria California", "screen_name": "MissCambria_420", "lang": "en", "location": "Orange County, Califorina", "create_at": date("2012-11-07"), "description": "She burns to be on the move, a walk, a ride, a journey; She is driven forward by the pounding pistons of her heart.", "followers_count": 128, "friends_count": 398, "statues_count": 1985 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552863821975555, "text": "@its_kelseeey me too ��������", "in_reply_to_status": 733552627779141632, "in_reply_to_user": 34783008, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34783008 }}, "user": { "id": 340647257, "name": "Cynthia Hughes", "screen_name": "cynn_cityyy", "lang": "en", "location": "Sacramento, California", "create_at": date("2011-07-22"), "description": "null", "followers_count": 495, "friends_count": 569, "statues_count": 4579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552863842951169, "text": "@__alexiee @KathleenNgo ������", "in_reply_to_status": 733552757068566528, "in_reply_to_user": 35349845, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 35349845, 258054399 }}, "user": { "id": 37503047, "name": "Michelle Miller", "screen_name": "MeeCHMiLLeR", "lang": "en", "location": "earth ", "create_at": date("2009-05-03"), "description": "keep your feet on the ground when your heads in the clouds", "followers_count": 286, "friends_count": 201, "statues_count": 29239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552864245637120, "text": "Man I swear it never fails to make me feel this way '", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700767218, "name": "Aяιanna❥", "screen_name": "ariibaby", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "null", "followers_count": 566, "friends_count": 336, "statues_count": 23937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552864342073344, "text": "@liyaaaahgiiiirl miss you too! Dad said he can take you home tomorrow or I can so don't worry! Just come up", "in_reply_to_status": 733552014504796162, "in_reply_to_user": 4789657128, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4789657128 }}, "user": { "id": 3289085606, "name": "Johnnyboii Scanlan", "screen_name": "Johnny70_", "lang": "en", "location": "Hillside", "create_at": date("2015-07-23"), "description": "P A N A N A b X L O C K", "followers_count": 245, "friends_count": 237, "statues_count": 1943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Makakilo, HI", "id": "3aff180d3a9e53d5", "name": "Makakilo", "place_type": "city", "bounding_box": rectangle("-158.115354,21.332074 -158.057413,21.396068") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1547600, "cityName": "Makakilo" } }
+{ "create_at": datetime("2016-05-20T00:00:06.000Z"), "id": 733552864694394880, "text": "Really back on the music thing. Excited to see what happens love y'all.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 719302034, "name": "High8", "screen_name": "LiferMusic208", "lang": "en", "location": "47", "create_at": date("2012-07-26"), "description": "fuck em . cheers to the dead homies. https://soundcloud.com/jroyal208/lvl-2", "followers_count": 1247, "friends_count": 752, "statues_count": 37783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pocatello, ID", "id": "d15336e5b45c79bb", "name": "Pocatello", "place_type": "city", "bounding_box": rectangle("-112.496051,42.806434 -112.373604,42.920598") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16005, "countyName": "Bannock", "cityID": 1664090, "cityName": "Pocatello" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915225921970180, "text": "Him and I are forever going up ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2388963928, "name": "BIGMOMMA", "screen_name": "SweeBrittany", "lang": "en", "location": "null", "create_at": date("2014-03-06"), "description": "this aint that !!!!!!", "followers_count": 1139, "friends_count": 984, "statues_count": 18251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915226249302016, "text": "Can I stay up for Nas: Time is Illmatic? Shit. I need to sleep. Someone tell me to turn off the TV.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179558875, "name": "Susie Schaaf", "screen_name": "fussballsusie", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2010-08-17"), "description": "Words/sounds: @bayerncentral, @Rekord_Pod, @OffsideRulePod. #ALZ caregiver to Mom. Has a Baron. Slings booze. Mitglied. #NOCANDYASSING", "followers_count": 4406, "friends_count": 2184, "statues_count": 98632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915226358177792, "text": "Wind 0.0 mph ---. Barometer 1022.05 mb, Steady. Temperature 52.5 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915226819563520, "text": "Yes lmao https://t.co/aoBRe2np8F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487761791, "name": "Nancy B.", "screen_name": "nancyohhh", "lang": "en", "location": "California", "create_at": date("2012-02-09"), "description": "22, Alex ❤️", "followers_count": 539, "friends_count": 179, "statues_count": 56232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valinda, CA", "id": "b1bbf8e2ef22573a", "name": "Valinda", "place_type": "city", "bounding_box": rectangle("-117.94891,34.024734 -117.912772,34.053145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 681638, "cityName": "Valinda" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915226853101568, "text": "I'm so ready for Monday.��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3060565424, "name": "D | illest. |", "screen_name": "DajourHall", "lang": "en", "location": "United States", "create_at": date("2015-03-03"), "description": "| The illest | Marcus.❤️", "followers_count": 322, "friends_count": 969, "statues_count": 636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Grove, FL", "id": "c257892b12c2c359", "name": "Myrtle Grove", "place_type": "city", "bounding_box": rectangle("-87.327898,30.389766 -87.275984,30.448493") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1247550, "cityName": "Myrtle Grove" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915227331383296, "text": "Zoooooted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2453311129, "name": "Shy Glizzy", "screen_name": "Foampositee", "lang": "en", "location": "null", "create_at": date("2014-04-19"), "description": "FYTB", "followers_count": 173, "friends_count": 46, "statues_count": 1719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915227440349185, "text": "Nac Rat for lyfe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1549212776, "name": "Bobo", "screen_name": "AndrewBobo20", "lang": "en", "location": "Nacogdoches • College Station", "create_at": date("2013-06-26"), "description": "Trusted by Young Metro TAMU '19", "followers_count": 1336, "friends_count": 705, "statues_count": 9449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915227478069248, "text": "You looking real swaggy big pimpin https://t.co/dJxUjDJUtY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2455998222, "name": "putita descarada ✨", "screen_name": "qweeennnaf", "lang": "en", "location": "LA✈️AZ", "create_at": date("2014-04-20"), "description": "super fly till I die. | sc: ynvttirb | ig: brittanyren_", "followers_count": 376, "friends_count": 293, "statues_count": 14712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915227599851521, "text": "Ass beating #progress?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "progress" }}, "user": { "id": 75938978, "name": "Boston Gilderman", "screen_name": "bogildy7", "lang": "en", "location": "Duluth, MN", "create_at": date("2009-09-20"), "description": "Snapchat- bogildy7", "followers_count": 482, "friends_count": 252, "statues_count": 1808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duluth, MN", "id": "62619a76134ad05e", "name": "Duluth", "place_type": "city", "bounding_box": rectangle("-92.282745,46.650616 -91.962935,46.866053") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27137, "countyName": "St. Louis", "cityID": 2717000, "cityName": "Duluth" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915227851476993, "text": "Temp: 56.3F W C: 56.3F Wind:--- at 0.3kts Baro: 1015.2mb and Falling slowly Rain today: 0.39in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 211, "friends_count": 218, "statues_count": 105690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915228061106177, "text": "@jacksonbookmc @asvp_robbins nope not cool enough to get the invite haha", "in_reply_to_status": 733915112298418176, "in_reply_to_user": 1733242172, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1733242172, 928691064 }}, "user": { "id": 473640145, "name": "Anthony Heaslip", "screen_name": "anthonyheaslip", "lang": "en", "location": "Rockford, Illinois", "create_at": date("2012-01-24"), "description": "#50in15.@XO_Madsss.", "followers_count": 1165, "friends_count": 544, "statues_count": 44023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Loves Park, IL", "id": "9bfddbd45776b2bc", "name": "Loves Park", "place_type": "city", "bounding_box": rectangle("-89.069204,42.293874 -88.964153,42.365885") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1745031, "cityName": "Loves Park" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915228077838336, "text": "@k_ianak will do!", "in_reply_to_status": 733913602881556480, "in_reply_to_user": 1051338714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1051338714 }}, "user": { "id": 92933177, "name": "kaea w", "screen_name": "kaeaboy", "lang": "en", "location": "Hawaii, USA", "create_at": date("2009-11-27"), "description": "【=◈︿◈=】| Instagram: @kaea_warrington", "followers_count": 535, "friends_count": 533, "statues_count": 19809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915228304351232, "text": "@ChrispyPadilla we need to colab tomorrow I got a couple things I gotta get to the peopl", "in_reply_to_status": -1, "in_reply_to_user": 950694510, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 950694510 }}, "user": { "id": 2203888182, "name": "Gage ford", "screen_name": "Iluvtacos_", "lang": "en", "location": "San Diego, CA", "create_at": date("2013-11-19"), "description": "your existence is not relevant to my success | ORHS", "followers_count": 2897, "friends_count": 2911, "statues_count": 41397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915229214507010, "text": "I miss my bby so much �������� https://t.co/QN0xmeZqW7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 723175232, "name": "jenetters", "screen_name": "jenetterrss", "lang": "en", "location": "null", "create_at": date("2012-07-28"), "description": "20 L.A ✌️", "followers_count": 605, "friends_count": 548, "statues_count": 7209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915229604564992, "text": "@ebbtideapp Tide in Great Point Clear, Alabama 05/21/2016\nHigh 11:42am 1.5\n Low 10:40pm 0.2\nHigh 12:12pm 1.6\n Low 11:46pm 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-87.9333,30.4833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 90, "friends_count": 1, "statues_count": 34570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Point Clear, AL", "id": "00c3940b3b15b9ef", "name": "Point Clear", "place_type": "city", "bounding_box": rectangle("-87.937145,30.460735 -87.899293,30.507821") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 161488, "cityName": "Point Clear" } }
+{ "create_at": datetime("2016-05-21T00:00:00.000Z"), "id": 733915229998944256, "text": "@MichelleDBeadle and not wear glasses", "in_reply_to_status": 733913511852539904, "in_reply_to_user": 52529345, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 52529345 }}, "user": { "id": 3336181396, "name": "jake johnson", "screen_name": "jakew1967", "lang": "en", "location": "Pennsylvania, USA", "create_at": date("2015-06-19"), "description": "a hard worker that love to tweet.", "followers_count": 848, "friends_count": 3202, "statues_count": 34930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915230447624192, "text": "Why your eyes look like that?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1733522617, "name": "Loading....", "screen_name": "llllMarkllll", "lang": "en", "location": "null", "create_at": date("2013-09-05"), "description": "null", "followers_count": 185, "friends_count": 232, "statues_count": 1356 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915231215325184, "text": "Ripley SW Limestone Co. Temp: 62.6°F Wind:0.0mph Pressure: 995.6mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915231466967040, "text": "@21savage loveeeeee you.��", "in_reply_to_status": -1, "in_reply_to_user": 260114837, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260114837 }}, "user": { "id": 547783389, "name": "A M B E R.➰", "screen_name": "_simplyALB", "lang": "en", "location": "〽️ississippi.", "create_at": date("2012-04-07"), "description": "19.|northeast cc'|Aspiring to be happy 24/7. 6'M & faaaaevaaa 2 go.❤️", "followers_count": 1801, "friends_count": 1019, "statues_count": 55305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, MS", "id": "0bbfd63a5a59d390", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-88.45588,33.456665 -88.362901,33.558564") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28087, "countyName": "Lowndes", "cityID": 2815380, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232029016064, "text": "smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3066840417, "name": "Mega Mike", "screen_name": "msayegh12", "lang": "en", "location": "Oak Forest, IL", "create_at": date("2015-03-02"), "description": "snapchat: msayegh6", "followers_count": 465, "friends_count": 452, "statues_count": 9307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Forest, IL", "id": "3273e7bb01fff3cb", "name": "Oak Forest", "place_type": "city", "bounding_box": rectangle("-87.795644,41.571599 -87.713927,41.646869") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1754638, "cityName": "Oak Forest" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232070860800, "text": "Apparently I give off novela villain vibes https://t.co/xY48Lkc3VD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366893265, "name": "Drea Monzon", "screen_name": "drea_monzon", "lang": "en", "location": "California", "create_at": date("2011-09-02"), "description": "null", "followers_count": 324, "friends_count": 289, "statues_count": 11724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232137928705, "text": "Already missing Moscow's prices compared to Seattle ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1025139036, "name": "Molly Friedrichsen", "screen_name": "MollFrie", "lang": "en", "location": "Seattle, WA", "create_at": date("2012-12-20"), "description": "22, University of Idaho, Pi Phi", "followers_count": 282, "friends_count": 170, "statues_count": 1090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232645570560, "text": "Wind 0.0 mph ---. Barometer 30.058 in, Steady. Temperature 55.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232674971648, "text": "Ladies, don't lie I'm a pretty ass fucker?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1108154413, "name": "RAUL", "screen_name": "chiraq5000", "lang": "en", "location": "Chicago,IL ", "create_at": date("2013-01-20"), "description": "young entrepreneur", "followers_count": 335, "friends_count": 106, "statues_count": 25115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915232892911617, "text": "A cop followed me almost all the way home... How cool��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1428243704, "name": "The Sean Lough", "screen_name": "Sean_Lough10", "lang": "en", "location": "null", "create_at": date("2013-05-14"), "description": "#FIJIBOYS SENIOR/ LEGEND.ÜBERMENSCH. KHS", "followers_count": 1100, "friends_count": 984, "statues_count": 9166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodcrest, CA", "id": "4b68a015eaeb9b4f", "name": "Woodcrest", "place_type": "city", "bounding_box": rectangle("-117.418374,33.828971 -117.313882,33.917206") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 686244, "cityName": "Woodcrest" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233010343936, "text": "@DarrellMags �� I'm sober", "in_reply_to_status": 733915182255079425, "in_reply_to_user": 2747625062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2747625062 }}, "user": { "id": 914216034, "name": "Amina", "screen_name": "Mymangot2JOBS_", "lang": "en", "location": "University of Arizona ", "create_at": date("2012-10-30"), "description": "I'm a long weave typa bitch • 21• Gym Rat• African (Guinea)• Educated• passionate soul working towards making a difference.", "followers_count": 1728, "friends_count": 456, "statues_count": 100082 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233228447744, "text": "@LizbethOchoa16 asi nomas https://t.co/TAzanY7LAi", "in_reply_to_status": -1, "in_reply_to_user": 571895897, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 571895897 }}, "user": { "id": 3465586753, "name": "Bella Amaya", "screen_name": "Bella1313j", "lang": "es", "location": "Washington, USA", "create_at": date("2015-09-05"), "description": "like i'm gonna love you", "followers_count": 43, "friends_count": 59, "statues_count": 254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233287180289, "text": "Wind 0.0 mph ---. Barometer 30.117 in, Falling. Temperature 43.5 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233320734721, "text": "I just be wanting to lay up with my nigga and a blunt ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205964364, "name": "Kʌɱɣɾɾɪʌ", "screen_name": "TheOnly_Kam", "lang": "en", "location": "Hou$ton", "create_at": date("2010-10-21"), "description": "PVAMU| Romans 12:2", "followers_count": 2376, "friends_count": 1487, "statues_count": 42316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233404739584, "text": "05/21@03:00 - Temp 54.9F, WC 54.9F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.158in, Falling. Rain 0.00in. Hum 90%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233417207809, "text": "Gang. Gang. https://t.co/m07g3X4QTk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 337433792, "name": "McBeezy.☹️", "screen_name": "KatieeMcGowan", "lang": "en", "location": "LJ", "create_at": date("2011-07-17"), "description": "#GANGGANG. |HRC|CLJ|", "followers_count": 1363, "friends_count": 99, "statues_count": 122158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233459273728, "text": "I am gonna be dead tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2313515748, "name": "Billy Dizzon", "screen_name": "bailey_dixonn", "lang": "en", "location": "Mentor, OH", "create_at": date("2014-01-27"), "description": "snapchat: bailey_dixonn", "followers_count": 1640, "friends_count": 486, "statues_count": 33398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willoughby, OH", "id": "88dfdf4f649a0ba2", "name": "Willoughby", "place_type": "city", "bounding_box": rectangle("-81.452726,41.603963 -81.370884,41.707193") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985484, "cityName": "Willoughby" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233488633856, "text": "Cute boxing chav �������� https://t.co/8qxfuO5wB9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4469, "friends_count": 3125, "statues_count": 47951 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233501118465, "text": "the fact that her \"friends\" buck her damn head up makes it no better ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2507202723, "name": "lil suzi vert ✨", "screen_name": "cccchyna", "lang": "en", "location": "null", "create_at": date("2014-04-24"), "description": "pray4momz❤️. McMain Alumni ' 16✨", "followers_count": 621, "friends_count": 873, "statues_count": 16247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233505431552, "text": "Temp: 73.0°F Wind:0.1mph Pressure: 29.986hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 64716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233522049024, "text": "@jbuchananbarnes HES ONE OF MY BEST FRIENDS FROM THERE", "in_reply_to_status": 733915101410013184, "in_reply_to_user": 28880664, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28880664 }}, "user": { "id": 867310585, "name": "Bucky Bart: Mall Cop", "screen_name": "brothersxblood", "lang": "en", "location": "IG: nicolasxburroughs ", "create_at": date("2012-10-07"), "description": "You can block my number but you can't block my dick.", "followers_count": 1380, "friends_count": 849, "statues_count": 59436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233610145792, "text": "Dad is currently stuck in a hurricane so plz pray he stays safe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2843196663, "name": "LEGS", "screen_name": "alegra_mia_16", "lang": "en", "location": "null", "create_at": date("2014-10-24"), "description": "just a stressed high school student", "followers_count": 574, "friends_count": 611, "statues_count": 11892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233689866240, "text": "I need more shades https://t.co/wjVwnBVyyY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2309131037, "name": "Raven McCoy", "screen_name": "syndiculous", "lang": "en", "location": "null", "create_at": date("2014-01-27"), "description": "null", "followers_count": 40, "friends_count": 40, "statues_count": 787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-05-21T00:00:01.000Z"), "id": 733915233878581249, "text": "@sowen_wong you live Aliamanu?", "in_reply_to_status": 733560093019529216, "in_reply_to_user": 1007666694, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1007666694 }}, "user": { "id": 712607842734243840, "name": "Lindon", "screen_name": "Linnndon", "lang": "en", "location": "Aiea, HI", "create_at": date("2016-03-23"), "description": "null", "followers_count": 56, "friends_count": 76, "statues_count": 23 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915234730016768, "text": "Bitch you hella dramatic, chaaalll", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 812586858, "name": "Jessica", "screen_name": "jessica_facundo", "lang": "en", "location": "null", "create_at": date("2012-09-09"), "description": "null", "followers_count": 724, "friends_count": 664, "statues_count": 5750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915235094958080, "text": "idek y I'm crying lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1477978609, "name": "Abby", "screen_name": "amchilds1", "lang": "en", "location": "null", "create_at": date("2013-06-02"), "description": "null", "followers_count": 406, "friends_count": 311, "statues_count": 1952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casper, WY", "id": "e024870d2401948f", "name": "Casper", "place_type": "city", "bounding_box": rectangle("-106.4322,42.791368 -106.229362,42.86915") }, "geo_tag": { "stateID": 56, "stateName": "Wyoming", "countyID": 56025, "countyName": "Natrona", "cityID": 5613150, "cityName": "Casper" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915235396902912, "text": "Late night politics with @StormsShayna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 777283110 }}, "user": { "id": 1117072849, "name": "คllเє❄", "screen_name": "alliekingsbury", "lang": "en", "location": "null", "create_at": date("2013-01-24"), "description": "Basically tweets of cute critters and me complaining about life", "followers_count": 263, "friends_count": 220, "statues_count": 4330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spearfish, SD", "id": "8fe1b634982002af", "name": "Spearfish", "place_type": "city", "bounding_box": rectangle("-103.89022,44.452744 -103.785719,44.51233") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46081, "countyName": "Lawrence", "cityID": 4660020, "cityName": "Spearfish" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915235598237696, "text": "Hello, everyone, I have to go to work now, and found new flims talk with everyone :)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2341496468, "name": "XiaoyunHou (Amanda)", "screen_name": "hhxxyu", "lang": "zh-cn", "location": "China-London", "create_at": date("2014-02-12"), "description": "Music,piano,movies buff, medical worker, travelling ,read!A bright and bubbly personality girl And Love The Lord of the rings ,Hobbit!", "followers_count": 134, "friends_count": 188, "statues_count": 7040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915235900231686, "text": "Faithful has to be my favorite song on views", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2245068486, "name": "kourt..", "screen_name": "KourtneiCt", "lang": "en", "location": "Dalla$", "create_at": date("2013-12-13"), "description": "21. Miss Purple and Gold @cartierhoe ❤️", "followers_count": 1496, "friends_count": 720, "statues_count": 34498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915235992535046, "text": "@nafiaa_7 and you need lotion", "in_reply_to_status": 733915169290522627, "in_reply_to_user": 2451625843, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2451625843 }}, "user": { "id": 2779715544, "name": "Gabz", "screen_name": "GabbyMartin__", "lang": "en", "location": "null", "create_at": date("2014-08-29"), "description": "wasssguddd", "followers_count": 749, "friends_count": 392, "statues_count": 13939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236185448448, "text": "26", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 710926236160163841, "name": "Angel Gutz", "screen_name": "prince__paradox", "lang": "en", "location": "null", "create_at": date("2016-03-18"), "description": "null", "followers_count": 34, "friends_count": 126, "statues_count": 101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jurupa Valley, CA", "id": "015b658472edd3dc", "name": "Jurupa Valley", "place_type": "city", "bounding_box": rectangle("-117.551185,33.955517 -117.410795,34.034306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 637692, "cityName": "Jurupa Valley" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236198158336, "text": "¿están conectados mis RT?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 52532527, "name": "cerpin", "screen_name": "celetereo", "lang": "es", "location": "siempre me voy", "create_at": date("2009-06-30"), "description": "artefacto.", "followers_count": 2155, "friends_count": 1541, "statues_count": 98323 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hartford, CT", "id": "485ebc6dbebdbf32", "name": "West Hartford", "place_type": "city", "bounding_box": rectangle("-72.786564,41.717959 -72.713899,41.806675") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 982660, "cityName": "West Hartford" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236311261184, "text": "One day ill find someone I can trust �� maybe.....������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3022344283, "name": "Dizzy MidWest", "screen_name": "Dizzymidwest", "lang": "en", "location": "Flint, MI", "create_at": date("2015-02-16"), "description": "love to entrain myself; sometimes others. ✌", "followers_count": 95, "friends_count": 65, "statues_count": 517 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236600840192, "text": "Wind 0.0 mph ---. Barometer 29.960 in, Falling slowly. Temperature 58.4 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 19981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236638527488, "text": "Wind 0.0 mph W. Barometer 30.01 in, Falling slowly. Temperature 73.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915236789592064, "text": "Temp 48.4° Hi/Lo 55.2/48.4 Rng 6.8° WC 48.4° Hmd 91% Rain 0.00\" Storm 0.00\" BAR 30.190 Falling DP 45.9° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 18864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915237175300097, "text": "@GhostMillk yes....I'm waiting too. We're all waiting for something", "in_reply_to_status": 733914056092876800, "in_reply_to_user": 1191647448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1191647448 }}, "user": { "id": 165502911, "name": "Okugaysu", "screen_name": "AshKetchumSays", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2010-07-11"), "description": "Ash/Vani/Usoppu | INTP |They/Them | ♎️♈️♊️♒️ | One Piece & JJBA♥ | Furry | PanRo/DemiSex | FA: http://www.furaffinity.net/user/vani/ l vent/nsfw: @josuyasuu", "followers_count": 2529, "friends_count": 484, "statues_count": 30708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915237317902336, "text": "@n_jello_ @jklmaunop @jaaadeee_ AND THIS IS HOW YOU TREAT ME!!!", "in_reply_to_status": 733915099077836800, "in_reply_to_user": 454630475, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2263690980, 4857730632, 1573861472 }}, "user": { "id": 454630475, "name": "Ceelo", "screen_name": "CelloAllStar", "lang": "en", "location": "null", "create_at": date("2012-01-03"), "description": "Some people got it and make it pay, Some people cant even give it away ✋", "followers_count": 203, "friends_count": 225, "statues_count": 1209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915237955600384, "text": "Wind 0.0 mph SW. Barometer 30.17 in, Steady. Temperature 50.2 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 159945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915238119178240, "text": "Hi @carolynsbuddy !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2441185613 }}, "user": { "id": 387050961, "name": "Yukio Strachan", "screen_name": "boldandworthy", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "Jamaican-American| PharmD | AKAtude | And Still I Rise — Maya Angelou", "followers_count": 3095, "friends_count": 148, "statues_count": 126982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-21T00:00:02.000Z"), "id": 733915238332932096, "text": "The blacker the berry the sweeter the juice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1856110592, "name": "FamW1ddit", "screen_name": "ilisapesi3", "lang": "en", "location": "null", "create_at": date("2013-09-11"), "description": "| H A P P I N E S S | FAM1LY | PEACE ✌️", "followers_count": 621, "friends_count": 501, "statues_count": 6833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915238714642432, "text": "@TakeeCaree no", "in_reply_to_status": 733877802584313857, "in_reply_to_user": 328933543, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 328933543 }}, "user": { "id": 614573993, "name": "Justin Ledesma", "screen_name": "Ledesma_XO", "lang": "en", "location": "Okinawa, Japan", "create_at": date("2012-06-21"), "description": "United States Marine", "followers_count": 149, "friends_count": 258, "statues_count": 2027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915238924353540, "text": "I WILL NEVER NOT RT THIS BC ITS SOOOOOO FUCKING RELEVANT https://t.co/rN0RS81DME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1423558189, "name": ".", "screen_name": "abrianacruz_", "lang": "en", "location": "null", "create_at": date("2013-05-12"), "description": "shit happens you just gotta decide if you want to move on from it or dwell in the past", "followers_count": 608, "friends_count": 236, "statues_count": 17211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Colony, TX", "id": "5f6eb89a27398e90", "name": "The Colony", "place_type": "city", "bounding_box": rectangle("-96.932509,33.048438 -96.850968,33.10959") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4872530, "cityName": "The Colony" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915239570276352, "text": "Today was good I will say ☺️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2539863558, "name": "Some RESPECK On It ♛", "screen_name": "SlimSnobbby", "lang": "en", "location": "San Francisco", "create_at": date("2014-06-01"), "description": "follow my player ass moves", "followers_count": 479, "friends_count": 109, "statues_count": 21694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915239679287296, "text": "Someone FT meeee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176293332, "name": "F.A.Y", "screen_name": "Caylad_", "lang": "en", "location": "TheGym", "create_at": date("2010-08-08"), "description": "| SKC commit #WBB ✨ | not a full scholly but still better than yours |", "followers_count": 698, "friends_count": 437, "statues_count": 18724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915239788335104, "text": "Tons espionage", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.26205064,36.3006563"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3276477930, "name": "Ana Steele", "screen_name": "AnaStee19484959", "lang": "en", "location": "Heaven ASAP", "create_at": date("2015-07-11"), "description": "By blood rite: king of all Earth. Angel of Allah y Yahway. Buddah by enlightement under the Bohdi Tree; direct Decendent Christ. Prof. singer actor dir writer", "followers_count": 1875, "friends_count": 2192, "statues_count": 139758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915239956111361, "text": "@AfricanPrlncess I love you ��������", "in_reply_to_status": 733914869838155777, "in_reply_to_user": 304149732, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304149732 }}, "user": { "id": 342330163, "name": "Ladarius Harris", "screen_name": "ladardar4", "lang": "en", "location": "Richmond • San Marcos", "create_at": date("2011-07-25"), "description": "8.1.13 ❤️ | 21 | oversimplified meaningfulness.", "followers_count": 915, "friends_count": 471, "statues_count": 37888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915239998095360, "text": "I was asleep. My mom woke me up. I have to be up in 3 hours. Now I can't go back to sleep. I'm pissed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 848307288, "name": "mamacita", "screen_name": "Breeedlc", "lang": "en", "location": "null", "create_at": date("2012-09-26"), "description": "constant disappointment", "followers_count": 300, "friends_count": 328, "statues_count": 14212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240132317185, "text": "Good luck getting that Chewbacca mask because it's sold out almost everywhere https://t.co/4txnFMM4jv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17654, "friends_count": 17815, "statues_count": 72140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240174374913, "text": "how am i supposed to survive another two days of this...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 521612948, "name": "victoria prejeant", "screen_name": "toriprejeant", "lang": "en", "location": "Kenner, LA", "create_at": date("2012-03-11"), "description": "null", "followers_count": 390, "friends_count": 410, "statues_count": 8988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange Beach, AL", "id": "00cf871c23325ccc", "name": "Orange Beach", "place_type": "city", "bounding_box": rectangle("-87.633494,30.267238 -87.514695,30.311135") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin", "cityID": 157144, "cityName": "Orange Beach" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240476217345, "text": "@Love_bug1016 @DropsOfSapphire soulgasms...hadn't heard that one before...well said", "in_reply_to_status": 733887134763913218, "in_reply_to_user": 572240095, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 572240095, 40436900 }}, "user": { "id": 160264147, "name": "TheRealBradleyJ", "screen_name": "ebiz_boy", "lang": "en", "location": "A winery near you. Or seat 1A", "create_at": date("2010-06-27"), "description": "Seattle-ish Entrepreneur, Rennaissance Soul, Wealth Creator, Yogi-Star-In-Training, Tennis-Star-Trainer...Wine Snob...and overall ok guy. And an Aries.", "followers_count": 167, "friends_count": 437, "statues_count": 1670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240476381185, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 507621346, "name": "〽️$. Brazy ‼️", "screen_name": "GloGirlNieca", "lang": "en", "location": "bama ♨", "create_at": date("2012-02-28"), "description": "- if a bitch beef wit me , we gon beef foreva. Bardi B❤️", "followers_count": 3088, "friends_count": 1822, "statues_count": 105930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lanett, AL", "id": "f8a200ec2de49963", "name": "Lanett", "place_type": "city", "bounding_box": rectangle("-85.223433,32.835533 -85.178666,32.886682") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1017, "countyName": "Chambers", "cityID": 141296, "cityName": "Lanett" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240547504129, "text": "I mean not a bad mood I just don't feel good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 486257019, "name": "jewels", "screen_name": "loovejewels", "lang": "en", "location": "LA", "create_at": date("2012-02-07"), "description": "weak messages create bad situations. Michelle is my best friend.", "followers_count": 1045, "friends_count": 2592, "statues_count": 51886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellflower, CA", "id": "e9b2c8beb5442ec5", "name": "Bellflower", "place_type": "city", "bounding_box": rectangle("-118.151393,33.865643 -118.106691,33.91052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604982, "cityName": "Bellflower" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240547684352, "text": "I'm at Overlook At Magnolia Apartment Homes in Seattle, WA https://t.co/z64IoF4ur3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.390321,47.652774"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20406490, "name": "Kip Roberson", "screen_name": "KipsterSEA", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-02-08"), "description": "I live in Seattle. I'm gay. I'm a librarian. I love my partner and my bulldog.", "followers_count": 456, "friends_count": 942, "statues_count": 21363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915240555892737, "text": "PURO PINCHE 956 raza https://t.co/6OFIUH1B8S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 710742884, "name": "Don Jon", "screen_name": "PapiJon_", "lang": "en", "location": "Pharr, TX", "create_at": date("2012-07-22"), "description": "Tamaulipas, Texas snapchat: andrade.john ponte trucha cuh", "followers_count": 324, "friends_count": 277, "statues_count": 26912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915241470255104, "text": "I can't breath.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700590680, "name": "Marrissa Juarez", "screen_name": "Marrissa303", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "Sorry I can't...I have practice.", "followers_count": 252, "friends_count": 152, "statues_count": 5597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cathedral City, CA", "id": "cf9828599ad4ad7d", "name": "Cathedral City", "place_type": "city", "bounding_box": rectangle("-116.493248,33.759319 -116.437311,33.859466") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 612048, "cityName": "Cathedral City" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915241478684672, "text": "HAPPY BIRTHDAY HUN❤️ Thankyou so much for everything! I'm glad I met someone that is even a little weirder than me❤️ https://t.co/b90JBHhZa2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2760127972, "name": "CHRIS", "screen_name": "Christianhoc12", "lang": "en", "location": "Upland, CA", "create_at": date("2014-09-02"), "description": "GM❤️", "followers_count": 193, "friends_count": 173, "statues_count": 142 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915241977765888, "text": "I want a boy that makes me as happy as this mask makes this lady ���� https://t.co/K1JiWgKfQL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3475359553, "name": "Kailee :))", "screen_name": "QuarteroRae", "lang": "en", "location": "null", "create_at": date("2015-09-06"), "description": "University of Rhode Island ⚽️", "followers_count": 155, "friends_count": 84, "statues_count": 553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915241982136320, "text": "@__golxd , alrigh ��", "in_reply_to_status": 733914910992830465, "in_reply_to_user": 392819738, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 392819738 }}, "user": { "id": 381266452, "name": "6ixtwentyfour♥.", "screen_name": "jaasorbooty", "lang": "en", "location": "BCU9TEEN", "create_at": date("2011-09-27"), "description": "homoflexible♥.\nbestfri.:@Simply_MisRissa\nbahamian queen.\n fmoi:jaasorbooty.\nWIIIDE AWAKE !", "followers_count": 1570, "friends_count": 2272, "statues_count": 28459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conley, GA", "id": "4479797d4d745235", "name": "Conley", "place_type": "city", "bounding_box": rectangle("-84.36042,33.623798 -84.315203,33.648025") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13063, "countyName": "Clayton", "cityID": 1319280, "cityName": "Conley" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242112110592, "text": "Be who or what you wanna be", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2565878363, "name": "Kolachi Pombor", "screen_name": "KPombor14", "lang": "en", "location": "Williamsport, PA", "create_at": date("2014-05-26"), "description": "WHS '17 ⚽️ snap: pombor14", "followers_count": 426, "friends_count": 411, "statues_count": 7946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williamsport, PA", "id": "f523bb393a5f747f", "name": "Williamsport", "place_type": "city", "bounding_box": rectangle("-77.144673,41.220627 -76.981651,41.276057") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42081, "countyName": "Lycoming", "cityID": 4285312, "cityName": "Williamsport" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242225274882, "text": "i love my peer counseling babes�� https://t.co/DiZjM4RKje", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4481019493, "name": "Malia", "screen_name": "maliarosie", "lang": "en", "location": "Clovis, CA", "create_at": date("2015-12-06"), "description": "null", "followers_count": 112, "friends_count": 139, "statues_count": 694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242300739584, "text": "\"I'd rather be honest than impressive\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92857229, "name": "Hunter Gibson", "screen_name": "HUNTERISHUNTING", "lang": "en", "location": "Los Angeles", "create_at": date("2009-11-26"), "description": "lost boy.", "followers_count": 3485, "friends_count": 4305, "statues_count": 30218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242556620800, "text": "Just ordered my swimsuit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 126057733, "name": "BadGirlRico", "screen_name": "vintage_monroe_", "lang": "en", "location": "withmyGranny", "create_at": date("2010-03-24"), "description": "Good.vibes.only.", "followers_count": 2902, "friends_count": 1910, "statues_count": 116490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242590142464, "text": "90's music is everything ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2540300148, "name": "miyah j. blige", "screen_name": "AmiyahY", "lang": "en", "location": "Compton, CA", "create_at": date("2014-06-01"), "description": "Instagram: amiyaaah__ snapchat: mya-young", "followers_count": 166, "friends_count": 228, "statues_count": 630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-05-21T00:00:03.000Z"), "id": 733915242640465920, "text": "Wind 0.0 mph ---. Barometer 30.008 in, Rising slowly. Temperature 61.3 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915243584360448, "text": "AP\nOklahoma Gov. Fallin vetoed...\nhttps://t.co/Qn0BsZ0TQP PLEASE watch \"Revolutionary Road\" w/Leo DiCaprio... #DeathOfAFamily #SelfAbortion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "DeathOfAFamily", "SelfAbortion" }}, "user": { "id": 275115688, "name": "SharronRobinson", "screen_name": "MsRoneRoyalty1", "lang": "en", "location": "Tennessee", "create_at": date("2011-03-31"), "description": "I'm a Beautiful Black American Business Woman who's for the best interest of The American masses, whoever,wherever& however. Where there's a will, there's a way", "followers_count": 617, "friends_count": 805, "statues_count": 11479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chattanooga, TN", "id": "3b3916ee31cfc9e6", "name": "Chattanooga", "place_type": "city", "bounding_box": rectangle("-85.404424,34.983674 -85.080704,35.200235") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47065, "countyName": "Hamilton", "cityID": 4714000, "cityName": "Chattanooga" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915244800532481, "text": "Nunca voy a olvidarte - @MykoValdovinos and Angel ������������ #onrepeat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "onrepeat" }}, "user_mentions": {{ 385805426 }}, "user": { "id": 631718027, "name": ".", "screen_name": "Itspedrobitch_", "lang": "en", "location": "Oxnard, CA", "create_at": date("2012-07-09"), "description": "null", "followers_count": 550, "friends_count": 325, "statues_count": 8079 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915245358415872, "text": "Temp: 53.8°F - Dew Point: 44.3° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 30.09in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 8, "friends_count": 12, "statues_count": 17980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915245870092288, "text": "Idk if I should go running now or wait till 3 or 4 in the morning hmmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473704245, "name": "Young Cisco", "screen_name": "FranciscoG_Jr", "lang": "en", "location": "Tucson, AZ", "create_at": date("2012-01-24"), "description": "Rip Dad / U.S Army / I'm the future of boxing at 105lbs.", "followers_count": 1994, "friends_count": 754, "statues_count": 65848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915245895262208, "text": "@Keyurruhh LMFAOOO https://t.co/GKRKmhWH1K", "in_reply_to_status": 733913680434237444, "in_reply_to_user": 1038928237, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1038928237 }}, "user": { "id": 834489553, "name": "space-alien dad", "screen_name": "PharaohLilvon", "lang": "en", "location": "my spaceship", "create_at": date("2012-09-19"), "description": "20 | gemini | visual-artist | creating like God | inquires: pharaohlilvonart@gmail.com | I've been busy | #supportlivingartists #nofreeart", "followers_count": 904, "friends_count": 295, "statues_count": 53110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915246163820545, "text": "Who what when where why?????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 472696259, "name": "Julieeee", "screen_name": "JulieFienga", "lang": "en", "location": "Long Island, New York ", "create_at": date("2012-01-23"), "description": "Juliette Michael | TBTYD", "followers_count": 208, "friends_count": 323, "statues_count": 4914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brightwaters, NY", "id": "da0ccedc7c9a6b66", "name": "Brightwaters", "place_type": "city", "bounding_box": rectangle("-73.275456,40.702358 -73.2484,40.732419") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608323, "cityName": "Brightwaters" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915246222397440, "text": "God bless the girls who had a 30 minute photo shoot with us�������� https://t.co/ElajDeDXc6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.236176,34.041928"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1155771396, "name": "amira", "screen_name": "_amiraaaaaa", "lang": "en", "location": "null", "create_at": date("2013-02-06"), "description": "so cal☀️", "followers_count": 1027, "friends_count": 657, "statues_count": 31271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arts District Co-Op", "id": "07d9f7334a881001", "name": "Arts District Co-Op", "place_type": "poi", "bounding_box": rectangle("-118.2361761,34.0419279 -118.236176,34.041928") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-21T00:00:04.000Z"), "id": 733915246448910336, "text": "17 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2234083442, "name": "pq", "screen_name": "paaulinnnaaaa", "lang": "en", "location": "the hill", "create_at": date("2013-12-06"), "description": "fhs. varsity cheer captain. I ♡ jake jones.", "followers_count": 790, "friends_count": 499, "statues_count": 11204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915247862386689, "text": "Yeah I'm not reading your message because I couldn't honest care less...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176937704, "name": "Stin", "screen_name": "AustinJBergeron", "lang": "en", "location": "null", "create_at": date("2010-08-10"), "description": "The best thing that you can do on any given day is win a baseball game and the second best thing is to lose one. -Chuck Tanner", "followers_count": 716, "friends_count": 443, "statues_count": 7025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915248525225984, "text": "@ZacEfron I was gonna see your shit movie Neighbors 2 tonight but instead I stuffed my fat ass face w/ @wingstop and now I want to throw up", "in_reply_to_status": -1, "in_reply_to_user": 492399548, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 492399548, 89858807 }}, "user": { "id": 243042376, "name": "Jared", "screen_name": "itsjustmejared", "lang": "en", "location": "Santa Clarita, CA", "create_at": date("2011-01-25"), "description": "I don't wear pants", "followers_count": 191, "friends_count": 61, "statues_count": 13920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915248843837440, "text": "Fomie ass Steve ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3254483641, "name": "steven moya", "screen_name": "killasteven__", "lang": "en", "location": "⚽️", "create_at": date("2015-06-24"), "description": "null", "followers_count": 141, "friends_count": 124, "statues_count": 921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915248978075651, "text": "Omg our lyft driver is so sketch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2176108698, "name": "Jess", "screen_name": "Jess_ireneeee96", "lang": "en", "location": "null", "create_at": date("2013-11-05"), "description": "null", "followers_count": 299, "friends_count": 183, "statues_count": 14005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249070379008, "text": "You chose a side that wasn't mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 935756101, "name": "matthew", "screen_name": "immattchester", "lang": "en", "location": "null", "create_at": date("2012-11-08"), "description": "the 6 aint friendly but its where i live", "followers_count": 99, "friends_count": 94, "statues_count": 4957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carol Stream, IL", "id": "1aa7803ca6707875", "name": "Carol Stream", "place_type": "city", "bounding_box": rectangle("-88.179339,41.887811 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249087287296, "text": "Amd i fell tryna dip from justin☹️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2913127740, "name": "300", "screen_name": "AsapJxson", "lang": "en", "location": "null", "create_at": date("2014-11-28"), "description": "Khia", "followers_count": 399, "friends_count": 262, "statues_count": 9775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond West, FL", "id": "a4699b0d577e1779", "name": "Richmond West", "place_type": "city", "bounding_box": rectangle("-80.447069,25.595373 -80.409295,25.626356") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1260230, "cityName": "Richmond West" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249217261568, "text": "@Hin0josa it was really red until I put eye drops in https://t.co/dCe7vGaBKS", "in_reply_to_status": 733914959348895744, "in_reply_to_user": 2209605383, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2209605383 }}, "user": { "id": 2209605383, "name": "Collin Hinojosa", "screen_name": "Hin0josa", "lang": "en", "location": "Bellevue, Ohio ", "create_at": date("2013-12-05"), "description": "senior", "followers_count": 1117, "friends_count": 598, "statues_count": 11707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellevue, OH", "id": "756d43f842b4e8fe", "name": "Bellevue", "place_type": "city", "bounding_box": rectangle("-82.882741,41.25536 -82.807705,41.299077") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39077, "countyName": "Huron", "cityID": 3905228, "cityName": "Bellevue" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249271820288, "text": "@squirtmandotcm thank your squirtman. Xx", "in_reply_to_status": 733903461075591168, "in_reply_to_user": 82018273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82018273 }}, "user": { "id": 961637078, "name": "QueenTaniaSue", "screen_name": "BbwTaniaSue", "lang": "en", "location": "Orange Park, FL", "create_at": date("2012-11-20"), "description": "Im a hearin impaired Queen Mistress and amateur porn model. cash meets n greets. for bookings, email me. sexybbw1969@yahoo.com", "followers_count": 11663, "friends_count": 804, "statues_count": 4381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249317806080, "text": "@ayooliaa I seen something about it on fb and I'm like wtf ��", "in_reply_to_status": 733914504506036224, "in_reply_to_user": 248965414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248965414 }}, "user": { "id": 2242583062, "name": "Taylor ✨", "screen_name": "vanillacurly", "lang": "en", "location": "Boca Raton, FL", "create_at": date("2013-12-24"), "description": "Florida Atlantic University • 19", "followers_count": 394, "friends_count": 278, "statues_count": 3839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Homestead, FL", "id": "9519f2018bb6585b", "name": "Homestead", "place_type": "city", "bounding_box": rectangle("-80.526237,25.440567 -80.377996,25.518331") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1232275, "cityName": "Homestead" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249661902849, "text": "What a week.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274075980, "name": "kelsey.", "screen_name": "Kellsbells05", "lang": "en", "location": "null", "create_at": date("2011-03-29"), "description": "where you invest your love, you invest your life", "followers_count": 524, "friends_count": 283, "statues_count": 9694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915249699651584, "text": "@veronicamerrell can you please follow me back", "in_reply_to_status": -1, "in_reply_to_user": 296037401, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 296037401 }}, "user": { "id": 3437158037, "name": "Jordan Landers", "screen_name": "jordanclanders", "lang": "en", "location": "Ohio, USA", "create_at": date("2015-08-23"), "description": "I like blue eyes and brown eyes and I like Star Wars ✌️✌", "followers_count": 13, "friends_count": 7, "statues_count": 66 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, OH", "id": "7762efec3ac39c7e", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-82.56055,38.403186 -82.494395,38.422036") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39087, "countyName": "Lawrence", "cityID": 3910352, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-05-21T00:00:05.000Z"), "id": 733915250337030144, "text": "Tonigh was filled with sassy girls who did not like me and they threw off my groove :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 350237036, "name": "Jessie Spangler", "screen_name": "jspangs", "lang": "en", "location": "Mcdonalds", "create_at": date("2011-08-07"), "description": "Never a diva but occassionally a real dick ¯\\_(ツ)_/¯ ~BGSU~ http://Instagram.com/jspangs", "followers_count": 624, "friends_count": 486, "statues_count": 17540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowling Green, OH", "id": "001bdb1154d55f66", "name": "Bowling Green", "place_type": "city", "bounding_box": rectangle("-83.690916,41.348047 -83.600702,41.426132") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39173, "countyName": "Wood", "cityID": 3907972, "cityName": "Bowling Green" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277613884190720, "text": "When im with you I feel safe ❤️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 342459360, "name": "Andrea Lopez⚾", "screen_name": "aandreadeserie", "lang": "en", "location": "The Bay-The Burgh ", "create_at": date("2011-07-25"), "description": "21 and doing my damn thing❣", "followers_count": 447, "friends_count": 356, "statues_count": 30319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277614148452352, "text": "I peeped scene that's why you can't trust em", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130828809, "name": "Reed", "screen_name": "chuckakev", "lang": "en", "location": "come find me...", "create_at": date("2010-04-08"), "description": "#Celtics #Nets #Spur #Patriots #gang calm cool collected...I refuse to lose! #bbm 2BC1F0C5", "followers_count": 770, "friends_count": 599, "statues_count": 44864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Akron, OH", "id": "fef01a8cb0eacb64", "name": "Akron", "place_type": "city", "bounding_box": rectangle("-81.628444,40.995137 -81.405059,41.166618") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39153, "countyName": "Summit", "cityID": 3901000, "cityName": "Akron" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277614156677120, "text": "I don't understand....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 987880794, "name": "taya", "screen_name": "_TaeeBaee_", "lang": "en", "location": "null", "create_at": date("2012-12-03"), "description": "senior c/o 2017", "followers_count": 387, "friends_count": 216, "statues_count": 14898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277615570145281, "text": "@king_nut7 @Funny2K_EA ��������", "in_reply_to_status": 734275425237622784, "in_reply_to_user": 2502997035, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2502997035, 700562530712813569 }}, "user": { "id": 263511378, "name": "Tekwan", "screen_name": "_Price1zRight", "lang": "en", "location": "null", "create_at": date("2011-03-09"), "description": "yall can learn lessons while i sit back, pray to God, and earn his blessings.", "followers_count": 1163, "friends_count": 973, "statues_count": 26738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morganton, NC", "id": "42c23bc85b4bdc09", "name": "Morganton", "place_type": "city", "bounding_box": rectangle("-81.793266,35.701696 -81.617271,35.784073") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37023, "countyName": "Burke", "cityID": 3744400, "cityName": "Morganton" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277615746461696, "text": "TF https://t.co/CTymPoH8uO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3691265655, "name": ".", "screen_name": "KyVisionz", "lang": "en", "location": "Poly", "create_at": date("2015-09-17"), "description": "sc ; ayokiam", "followers_count": 582, "friends_count": 430, "statues_count": 5695 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277616371412992, "text": "10%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 317912970, "name": "Tylar", "screen_name": "t_schroer", "lang": "en", "location": "null", "create_at": date("2011-06-15"), "description": "strippers and cocaine.", "followers_count": 483, "friends_count": 329, "statues_count": 14680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277616442560512, "text": "Shout out to everybody I consider a friend or family.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48163581, "name": "IFuckedMyCouch", "screen_name": "Abrahamsnotpunk", "lang": "en", "location": "California, USA", "create_at": date("2009-06-17"), "description": "This punk shit ain't no joke.", "followers_count": 181, "friends_count": 215, "statues_count": 14600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277616715354112, "text": "Ultra wasn't so bad lmao lots of ratchets and thotties tho ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2848638441, "name": "❁ᴍᴀᴜʀᴀ ᴀᴍᴀʏᴀ❁", "screen_name": "ohheynegra_", "lang": "en", "location": "Andres❤️", "create_at": date("2014-10-28"), "description": "⚽️, it's my thing.", "followers_count": 563, "friends_count": 482, "statues_count": 28224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, VA", "id": "319ee7b36c9149da", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-77.172219,38.827378 -77.031779,38.934311") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51013, "countyName": "Arlington", "cityID": 5103000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277617176563712, "text": "best 24 hours of my life THANK YOU SAMMY���� @samantham0rales https://t.co/D67FgOCvDU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4263996499 }}, "user": { "id": 3015926503, "name": "niki newton", "screen_name": "niki_newton", "lang": "en", "location": "California, USA", "create_at": date("2015-02-09"), "description": "peachy keen jelly bean", "followers_count": 601, "friends_count": 660, "statues_count": 5776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277617390485504, "text": "@ebbtideapp Tide in Crow Point, Massachusetts 05/22/2016\n Low 6:30am 0.1\nHigh 12:42pm 9.3\n Low 6:39pm 1.0\nHigh 12:47am 10.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.8933,42.2617"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 92, "friends_count": 1, "statues_count": 34851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hingham, MA", "id": "00905e2e648b2eb0", "name": "Hingham", "place_type": "city", "bounding_box": rectangle("-70.931755,42.157378 -70.841438,42.274719") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2530175, "cityName": "Hingham" } }
+{ "create_at": datetime("2016-05-22T00:00:00.000Z"), "id": 734277617595998208, "text": "I fucking need this https://t.co/jvsJaLatIi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330800893, "name": "alexis", "screen_name": "lexikrobinsonn", "lang": "en", "location": "MN", "create_at": date("2011-07-06"), "description": "5.24.15", "followers_count": 598, "friends_count": 618, "statues_count": 8274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brainerd, MN", "id": "52476ef579cfa347", "name": "Brainerd", "place_type": "city", "bounding_box": rectangle("-94.234598,46.319637 -94.127177,46.417239") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27035, "countyName": "Crow Wing", "cityID": 2707300, "cityName": "Brainerd" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618112073728, "text": "Wind 0.0 mph ---. Barometer 30.080 in, Falling slowly. Temperature 45.5 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 39, "friends_count": 4, "statues_count": 27598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618380328961, "text": "Happy birthday gorgeous! I hope your day was as great as you �������� @Vanessaaaa_16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 886736648 }}, "user": { "id": 615498232, "name": "Geenzzz", "screen_name": "geena_qotami", "lang": "en", "location": "null", "create_at": date("2012-06-22"), "description": "Jordanian | CPP '20", "followers_count": 169, "friends_count": 160, "statues_count": 5869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618590191616, "text": "@_blvzeed @__kaylaakw @kiramoniquewynn @forever_etc @SheQueenD_ look at my phone in the 1st picture ����", "in_reply_to_status": 734277311554588672, "in_reply_to_user": 41456270, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1612263434, 2541927008, 313568961, 606707724, 2259483623 }}, "user": { "id": 41456270, "name": "WildLifeWhit.", "screen_name": "_whitneyyah", "lang": "en", "location": "somewhere with miley ❤️", "create_at": date("2009-05-20"), "description": "RBF syndrome but cute asf ❤️", "followers_count": 956, "friends_count": 628, "statues_count": 18917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crossett, AR", "id": "b11ecb6b4811052d", "name": "Crossett", "place_type": "city", "bounding_box": rectangle("-91.984295,33.099887 -91.937336,33.159933") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5003, "countyName": "Ashley", "cityID": 516240, "cityName": "Crossett" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618766360576, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":51.6°F Wind:0.0mph Pressure: 30.03hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 18, "statues_count": 318764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618921394176, "text": "2 & 1/2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2336919288, "name": "Gaven Palagonia", "screen_name": "___G_P___", "lang": "en", "location": "siempre argentino", "create_at": date("2014-02-10"), "description": "I just need someone to drink withhhh", "followers_count": 321, "friends_count": 369, "statues_count": 6112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277618938171393, "text": "�� lol https://t.co/eB5ozgf1HB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2303653279, "name": "IZM !", "screen_name": "D3doee", "lang": "en", "location": "null", "create_at": date("2014-01-21"), "description": "null", "followers_count": 567, "friends_count": 220, "statues_count": 30510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277619064016896, "text": "Bout to NyQuil myself to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1009951362, "name": "Emily Munro", "screen_name": "EmilersMunro", "lang": "en", "location": "Omaha, NE", "create_at": date("2012-12-13"), "description": "Creighton University | Jays Dancer | NV✈️NE", "followers_count": 420, "friends_count": 332, "statues_count": 1845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277619135348736, "text": "karamatsu when he starts his own clothing line: MY BRAND", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2478918703, "name": "there god it choroma", "screen_name": "kuriimatsu", "lang": "en", "location": "osomatsu-san hell ", "create_at": date("2014-05-05"), "description": "Haru/Aka/Choro -- 21 -- She/it ♥ -- Big Bro!: @NapBonPar -- Bun!: @albed0_ // Karamatsu+Tadashi @ Citta !", "followers_count": 256, "friends_count": 247, "statues_count": 30479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277619437297668, "text": "here comes the FESTPAC snaps �� hope y'all enjoy your fkin selfs ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4628008032, "name": "JVRCRZ", "screen_name": "jvrcrz", "lang": "en", "location": "Seattle, WA", "create_at": date("2015-12-21"), "description": "sc: jvrcrz ✌️", "followers_count": 81, "friends_count": 98, "statues_count": 396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Picnic Point, WA", "id": "007d0c773de7ec34", "name": "Picnic Point", "place_type": "city", "bounding_box": rectangle("-122.336551,47.82599 -122.245495,47.89362") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5354213, "cityName": "Picnic Point" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277619835768833, "text": "Boiiiii", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 551513886, "name": "Elias⁶", "screen_name": "I_Twerk_4_Oreos", "lang": "en", "location": "540 DMV✈️✈️ Cali☀️☀️ ⁶", "create_at": date("2012-04-11"), "description": "United States Marine but still just your average Elias You're the creator of your own happiness-R.I.P Jordan Gentile", "followers_count": 827, "friends_count": 450, "statues_count": 53614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Artesia, CA", "id": "b82c98fb6806ceb9", "name": "Artesia", "place_type": "city", "bounding_box": rectangle("-118.091312,33.853432 -118.068916,33.880386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 602896, "cityName": "Artesia" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277619923968000, "text": "Ripley SW Limestone Co. Temp: 57.6°F Wind:0.7mph Pressure: 997.0mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620183896064, "text": "me when Too Good comes on..���� https://t.co/98FmlLBDs9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125843748, "name": "Calico☪", "screen_name": "haileydonohuuee", "lang": "en", "location": "null", "create_at": date("2010-03-23"), "description": "this a place for families who drive camrys and go to disney.", "followers_count": 254, "friends_count": 105, "statues_count": 17572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcata, CA", "id": "2ca541fefabe94d6", "name": "Arcata", "place_type": "city", "bounding_box": rectangle("-124.110814,40.840387 -124.053849,40.909503") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 602476, "cityName": "Arcata" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620213374976, "text": "Wind 0.0 mph ---. Barometer 30.096 in, Steady. Temperature 53.9 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620297175040, "text": "Yo I'm turnt af but TLC said it best NOOOOO SCRUBBBSSSS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 423940086, "name": "cristina", "screen_name": "sstinnnaa", "lang": "en", "location": "bay area | ucsc", "create_at": date("2011-11-28"), "description": "★彡", "followers_count": 190, "friends_count": 214, "statues_count": 8071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Cruz, CA", "id": "3c9e627dd6b55d9e", "name": "Santa Cruz", "place_type": "city", "bounding_box": rectangle("-122.076144,36.948098 -121.986229,37.010652") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 669112, "cityName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620393611264, "text": "it's my birthday !!! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139944664, "name": "5/22", "screen_name": "JazmynJenkins", "lang": "en", "location": "land of the hyphy ", "create_at": date("2010-05-03"), "description": "null", "followers_count": 2557, "friends_count": 998, "statues_count": 26441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620620132353, "text": "always getting treated like crap", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2958331987, "name": "mari", "screen_name": "amaribrown91", "lang": "en", "location": "Anchorage, AK", "create_at": date("2015-01-04"), "description": "oh honey you don't phase me", "followers_count": 208, "friends_count": 191, "statues_count": 1846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620725108736, "text": "\"Ay what's good shaw\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712848486, "name": "Waldo", "screen_name": "joya_got_it", "lang": "en", "location": "Cole World ", "create_at": date("2012-07-23"), "description": "FHS 16'", "followers_count": 1095, "friends_count": 767, "statues_count": 19872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Market, MD", "id": "3e1e27e479e5f92d", "name": "New Market", "place_type": "city", "bounding_box": rectangle("-77.287399,39.380348 -77.248897,39.396039") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2455650, "cityName": "New Market" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620775280641, "text": "Same https://t.co/FHlKZsSp4D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2909719286, "name": "James Soup", "screen_name": "jamesoup_", "lang": "en", "location": "Puyallup, WA", "create_at": date("2014-11-24"), "description": "null", "followers_count": 184, "friends_count": 167, "statues_count": 358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, WA", "id": "af582c6bfd1b91eb", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-122.326451,47.203736 -122.2489,47.257551") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5320645, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620834144256, "text": "05/22@03:00 - Temp 51.9F, WC 51.5F. Wind 3.4mph NNW, Gust 8.0mph. Bar 29.840in, Falling slowly. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277620897054720, "text": "I fucked up tonight a lot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2564310644, "name": "jesse", "screen_name": "jsbrewcrew", "lang": "en", "location": "null", "create_at": date("2014-06-12"), "description": "null", "followers_count": 293, "friends_count": 697, "statues_count": 2626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Racine, WI", "id": "ce3b7b6f88f0b308", "name": "Racine", "place_type": "city", "bounding_box": rectangle("-87.877863,42.677972 -87.770388,42.799314") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55101, "countyName": "Racine", "cityID": 5566000, "cityName": "Racine" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277621060636672, "text": "Late gram... Just got all my pics back.... #edc #2016 #nyc @ Edc New York Citi Field 2016 https://t.co/mrDPNDNjmM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.8451951,40.7548906"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "edc", "nyc" }}, "user": { "id": 275053776, "name": "DJ Spot", "screen_name": "DjSPOTNYC", "lang": "en", "location": "Bronx,NY", "create_at": date("2011-03-31"), "description": "null", "followers_count": 163, "friends_count": 229, "statues_count": 1815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277621220052992, "text": "@DrewSass yooo drewbbyyy", "in_reply_to_status": 734277473513394176, "in_reply_to_user": 308448948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 308448948 }}, "user": { "id": 422440851, "name": "Xavier", "screen_name": "Xavier_L12", "lang": "en", "location": "larva bean field in spain", "create_at": date("2011-11-27"), "description": "Riff raffs young nephew | The Clan", "followers_count": 920, "friends_count": 674, "statues_count": 24408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davenport, IA", "id": "9c1c0a45346f5837", "name": "Davenport", "place_type": "city", "bounding_box": rectangle("-90.69214,41.475836 -90.502744,41.60005") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19163, "countyName": "Scott", "cityID": 1919000, "cityName": "Davenport" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277621392003072, "text": "Wind 0.0 mph ---. Barometer 30.029 in, Falling slowly. Temperature 48.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 19994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277621739978752, "text": "Love my friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2884001713, "name": "Erika Bauer", "screen_name": "akbauer4", "lang": "en", "location": "alaska & california", "create_at": date("2014-10-30"), "description": "success is my only option", "followers_count": 199, "friends_count": 184, "statues_count": 1780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277622079889408, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 29.806 in, Falling Rapidly. Temp 53.8F. Rain today 0.04 in. Humidity 98% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 625, "friends_count": 835, "statues_count": 42785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-22T00:00:01.000Z"), "id": 734277622155247616, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3226596060, "name": "Alyssa sanchez", "screen_name": "Alyssa2000__", "lang": "en", "location": "null", "create_at": date("2015-05-25"), "description": "frontier / snapchat-alyssanicolesan", "followers_count": 613, "friends_count": 485, "statues_count": 2364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277622209884160, "text": "Wind 0.0 mph ---. Barometer 29.851 in, Steady. Temperature 52.0 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277622964703232, "text": "@leexismaarie ���� love you so much can't wait for next week����", "in_reply_to_status": 734277438029455360, "in_reply_to_user": 1262212129, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1262212129 }}, "user": { "id": 562524394, "name": "Jenny", "screen_name": "jeennnyy_", "lang": "en", "location": "null", "create_at": date("2012-04-24"), "description": "null", "followers_count": 830, "friends_count": 701, "statues_count": 50801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623250092032, "text": "Obsessed with \"Vertigo\" by Jason Derulo and Jordan Sparks ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1323567612, "name": "KayT", "screen_name": "k_rieke", "lang": "en", "location": "Washington, USA", "create_at": date("2013-04-02"), "description": "I may not go down in history, but I'll go down on you", "followers_count": 532, "friends_count": 1290, "statues_count": 4754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centralia, WA", "id": "5d46bf8dd4d31b2d", "name": "Centralia", "place_type": "city", "bounding_box": rectangle("-123.019862,46.690459 -122.923905,46.764122") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53041, "countyName": "Lewis", "cityID": 5311160, "cityName": "Centralia" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623350726657, "text": "03:00 49.3°F Feels:49.3°F (Hi55.0°F/Lo49.3°F) Hum:72% Wnd:--- 0.0MPH Baro:30.24in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 207, "friends_count": 265, "statues_count": 29926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623367360514, "text": ".@realDonaldTrump can I be your Vice Presidential running mate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 340097227, "name": "Chris", "screen_name": "NotChrisChavez", "lang": "en", "location": "Wylie, TX", "create_at": date("2011-07-21"), "description": "Baylor Scott & White | Logan Grace Kusse | President of LULAC Collegiate Council #4780", "followers_count": 542, "friends_count": 303, "statues_count": 22524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wylie, TX", "id": "5b96c12e41e49aa5", "name": "Wylie", "place_type": "city", "bounding_box": rectangle("-96.596044,32.981938 -96.489165,33.054983") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4880356, "cityName": "Wylie" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623375745025, "text": "My dad has a better social life then me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1384674656, "name": "Haelee", "screen_name": "haeeeelee", "lang": "en", "location": "CA", "create_at": date("2013-04-27"), "description": "sc///haelee13 • Aries • what would hank moody do ?", "followers_count": 676, "friends_count": 502, "statues_count": 13572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Canyon Lake, CA", "id": "85072b5a406cd0ff", "name": "Canyon Lake", "place_type": "city", "bounding_box": rectangle("-117.283988,33.670173 -117.234601,33.71489") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 610928, "cityName": "Canyon Lake" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623476563968, "text": "Temp 52.3° Hi/Lo 54.4/52.3 Rng 2.1° WC 52.3° Hmd 87% Rain 0.00\" Storm 0.00\" BAR 29.880 Falling DP 48.5° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 18891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623493349377, "text": "Why are the best looking guys always Satan in disguise", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 779483887, "name": "Kaylynn Destiny", "screen_name": "kaylynn_lopez", "lang": "en", "location": "null", "create_at": date("2012-08-24"), "description": "null", "followers_count": 239, "friends_count": 157, "statues_count": 4793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623556231168, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 66.9 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623635808257, "text": "#FREERUBENSLIKK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "FREERUBENSLIKK" }}, "user": { "id": 76160934, "name": "la negrita min", "screen_name": "mindanielleg", "lang": "en", "location": "null", "create_at": date("2009-09-21"), "description": "fotógrafa en los angeles.", "followers_count": 1030, "friends_count": 312, "statues_count": 71370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277623929536512, "text": "@ ME @BetterCurryUp https://t.co/rkTq1pj38R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 586051286 }}, "user": { "id": 1271040679, "name": "Speck Boy #WSJEO", "screen_name": "SpeckStaysFloss", "lang": "en", "location": "Toledo, OH", "create_at": date("2013-03-15"), "description": "18. #Speck #MadLifey Instagram - speckstaysfloss", "followers_count": 321, "friends_count": 297, "statues_count": 26989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277624281718784, "text": "Nah I make my friends try to out drink me �� https://t.co/kPHkTaKJMB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2290415514, "name": "tyra bank$", "screen_name": "hboogie1x", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-13"), "description": "I have freckles & I'll probably dunk on you", "followers_count": 1371, "friends_count": 441, "statues_count": 56014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277625041018880, "text": "@tizzle_tha_man @JStarlings 7", "in_reply_to_status": 734277513841651712, "in_reply_to_user": 362618447, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 362618447, 2880429957 }}, "user": { "id": 427673079, "name": "Daddy Socks", "screen_name": "ZTurp17", "lang": "en", "location": "Linthicum, MD", "create_at": date("2011-12-03"), "description": "Your body is a reflection of your work ethic| Thoughts become words, words become actions and actions become destiny United States Marine", "followers_count": 638, "friends_count": 731, "statues_count": 27950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277625099735040, "text": "Temp: 68.3°F Wind:1.4mph Pressure: 30.006hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 64811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277625137373189, "text": "@OscarDaJuicemon rt me you bum ass nigga", "in_reply_to_status": 734277564789706752, "in_reply_to_user": 1730175720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1730175720 }}, "user": { "id": 1442196954, "name": "smooth operator", "screen_name": "MARKF_DOOM", "lang": "en", "location": "null", "create_at": date("2013-05-19"), "description": "lo que ciroc ciroc", "followers_count": 293, "friends_count": 373, "statues_count": 8412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277625544380416, "text": "Anyways, now for some SOMA before I KO.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77869264, "name": "Your Favorite Color", "screen_name": "Rio_Flows", "lang": "en", "location": "Bronx, NY", "create_at": date("2009-09-27"), "description": "rioflows@gmail.com for inquiries | SnapChat: @rio_flows", "followers_count": 471, "friends_count": 757, "statues_count": 20569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277626123034625, "text": "anyone up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3753260413, "name": "jason brooks", "screen_name": "jasonishomeless", "lang": "en", "location": "Keller, TX", "create_at": date("2015-10-01"), "description": "Texa§ ₩akeboarder", "followers_count": 225, "friends_count": 195, "statues_count": 2642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-05-22T00:00:02.000Z"), "id": 734277626198695936, "text": "Wind 0.0 mph ---. Barometer 29.88 in, Falling slowly. Temperature 55.9 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 122, "statues_count": 159969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277626445987841, "text": "@osmvn97 Osballs, it's at forever 21. Your boy Jose got the job too ;))))", "in_reply_to_status": 734277194831142912, "in_reply_to_user": 2318150738, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2318150738 }}, "user": { "id": 2358439482, "name": "mia", "screen_name": "muvatortilla", "lang": "en", "location": "null", "create_at": date("2014-02-23"), "description": ":)", "followers_count": 500, "friends_count": 632, "statues_count": 15536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277626525671424, "text": "Sooo ready for these finals tbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1556564256, "name": "B DAWG ++", "screen_name": "_McMahon_15", "lang": "en", "location": "null", "create_at": date("2013-06-29"), "description": "My lord and savior | 17' | Jmhs", "followers_count": 568, "friends_count": 705, "statues_count": 8606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277626781519873, "text": "I love all the bring it on movies so much", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732603968, "name": "bri jay", "screen_name": "brenee17", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "blessed w/ j rod ❤", "followers_count": 1299, "friends_count": 778, "statues_count": 20780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277627305811968, "text": "My dude Joel was all about this song in 9th grade. https://t.co/TWNK9YHDPy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1302605444, "name": "_Montezuma", "screen_name": "Antonio_Pereda1", "lang": "en", "location": "Rage City ", "create_at": date("2013-03-25"), "description": "Get to know me #jeremiah2911 #Pray4Joel", "followers_count": 501, "friends_count": 384, "statues_count": 10953 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277627779944448, "text": "Sooo pretty much just came within inches of hitting a deer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24258383, "name": "Blake Wilson", "screen_name": "BlakeJWilson", "lang": "en", "location": "Ames, IA", "create_at": date("2009-03-13"), "description": "Part man. Part coffee.", "followers_count": 50, "friends_count": 101, "statues_count": 2341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ames, IA", "id": "d6794586b08b7f9c", "name": "Ames", "place_type": "city", "bounding_box": rectangle("-93.718311,41.986006 -93.560416,42.079007") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19169, "countyName": "Story", "cityID": 1901855, "cityName": "Ames" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277628199378944, "text": "in the feels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 364328523, "name": "Amy McMurtrie", "screen_name": "amy_mcmurtrie", "lang": "en", "location": "null", "create_at": date("2011-08-29"), "description": "my angels fly high", "followers_count": 1162, "friends_count": 971, "statues_count": 17955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turnersville, NJ", "id": "014af7545dd54df8", "name": "Turnersville", "place_type": "city", "bounding_box": rectangle("-75.130194,39.698642 -75.015244,39.804669") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3474270, "cityName": "Turnersville" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277628320972800, "text": "Temp 54.2°F Wind Chill 54.2°F RH 95% Wind 0.2 W Gust 3.0 W SLP 29.873 in Falling slowly Rain 0.04 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 63, "statues_count": 35604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277628585070592, "text": "Wind 0.0 mph SSE. Barometer 30.07 in, Steady. Temperature 53.1 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 26, "friends_count": 112, "statues_count": 15045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277628652326912, "text": "Wind 0.0 mph ---. Barometer 29.971 in, Falling. Temperature 62.4 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 64812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277629080129536, "text": "I'm at @WetWillies in Memphis, TN https://t.co/f8QPGmFqrS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.05123568,35.13943942"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20488535 }}, "user": { "id": 322436422, "name": "Rae'", "screen_name": "jusmerae2u", "lang": "en", "location": "Milwaukee, Wisconsin", "create_at": date("2011-06-22"), "description": "null", "followers_count": 424, "friends_count": 1078, "statues_count": 1604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277629319086081, "text": "would anyone like to buy me saosin's new album for $11.99? it would be greatly appreciated", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 573998869, "name": "Samantha", "screen_name": "smplysammy", "lang": "en", "location": "null", "create_at": date("2012-05-07"), "description": "null", "followers_count": 239, "friends_count": 384, "statues_count": 3523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-22T00:00:03.000Z"), "id": 734277629675773952, "text": "Love kickin it back @ this time just jammin in this nice weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2738672159, "name": "emiliá bediliá", "screen_name": "eemmiilliiaaaa", "lang": "en", "location": "puppies", "create_at": date("2014-08-09"), "description": "get thru my sister @molinataylourr first", "followers_count": 763, "friends_count": 362, "statues_count": 6905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alafaya, FL", "id": "00d511d335cd9fb6", "name": "Alafaya", "place_type": "city", "bounding_box": rectangle("-81.244501,28.489488 -81.125632,28.568794") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1200410, "cityName": "Alafaya" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277630569107456, "text": "Someone's gettin' all fancy with the camera angle! Ehem @nitzromy ���� #prodirector #innovator… https://t.co/XZZEjLCsC1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.01302895,44.51258474"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "prodirector", "innovator" }}, "user_mentions": {{ 21173854 }}, "user": { "id": 571271125, "name": "Ellery McCardle", "screen_name": "ElleryTV", "lang": "en", "location": "Green Bay, WI", "create_at": date("2012-05-04"), "description": "Anchor/Reporter/Investigator @WBAY. Traveler. Opinions my own. emccardle@wbay.com http://Instagram.com/ellerymctv", "followers_count": 1095, "friends_count": 856, "statues_count": 5191 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green Bay, WI", "id": "648f332371670aae", "name": "Green Bay", "place_type": "city", "bounding_box": rectangle("-88.139804,44.47174 -87.876593,44.574607") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5531000, "cityName": "Green Bay" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277630615113728, "text": "@JavierFlores_21 bro they know", "in_reply_to_status": 734275558029307904, "in_reply_to_user": 1119659568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1119659568 }}, "user": { "id": 1021369392, "name": "Alex", "screen_name": "alexmachado2015", "lang": "en", "location": "Dimmitt, TX", "create_at": date("2012-12-18"), "description": "everything happens for a reason, people come and go like seasons. #DirtyBoyz", "followers_count": 1181, "friends_count": 934, "statues_count": 19040 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dimmitt, TX", "id": "8b909470aa273efd", "name": "Dimmitt", "place_type": "city", "bounding_box": rectangle("-102.34146,34.526477 -102.29715,34.560626") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48069, "countyName": "Castro", "cityID": 4820464, "cityName": "Dimmitt" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277630640300032, "text": "Happy Birthday �������� I'll see you later today �� @pugaevan_ https://t.co/ZlLiQak8gY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3499498152 }}, "user": { "id": 1970285461, "name": "Az7", "screen_name": "ANDY7__", "lang": "en", "location": "null", "create_at": date("2013-10-18"), "description": "Oac #7 / nunca esqueça de onde veio", "followers_count": 743, "friends_count": 73, "statues_count": 35024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277631365894145, "text": "Someone plz help. I hate working PM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 762436039, "name": "Jenny Bae", "screen_name": "jennybaeby", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-08-16"), "description": "it is what it is", "followers_count": 279, "friends_count": 212, "statues_count": 18957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277631546261504, "text": "come here baby put your hands on my body", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3720614899, "name": "tiffany", "screen_name": "txffanyo", "lang": "en", "location": "null", "create_at": date("2015-09-28"), "description": "null", "followers_count": 341, "friends_count": 179, "statues_count": 5486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277632146051072, "text": "Wind 0.0 mph E. Barometer 29.966 in, Falling slowly. Temperature 69.5 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8615 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277632506789888, "text": "The fact that the KKK still exists bro ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1081765278, "name": "King T'Challa", "screen_name": "MattWakhu", "lang": "en", "location": "Houston, TX", "create_at": date("2013-01-11"), "description": "The Great Blacksby | You'll probably unfollow me for no reason | GentlΣmaΝ | UNT | Kenyan | Follow my music page @MattWakhuMusic!", "followers_count": 3645, "friends_count": 2440, "statues_count": 132211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277632624185348, "text": "PSA: Staters on 4th street is hiring...apply online", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4881177980, "name": "Connor", "screen_name": "snell_con", "lang": "en", "location": "Yucaipa, CA", "create_at": date("2016-02-08"), "description": "null", "followers_count": 200, "friends_count": 225, "statues_count": 178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277633651945472, "text": "HAPPY BIRTHDAY JOEL ✨���� I love you so much! I hope you have a great birthday ����1️⃣8️⃣ ur legal let's go clubbing ���� https://t.co/PrSpt8lvQz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315356808, "name": "pao", "screen_name": "paolssss", "lang": "en", "location": "null", "create_at": date("2011-06-11"), "description": "Probably annoyed", "followers_count": 582, "friends_count": 492, "statues_count": 26291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277633676955648, "text": "Social media done ruin y'all lives got girls thinking that if their boyfriend don't post about you he cheating ���� what kinda logic is that?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465272297, "name": "Hope Dealer", "screen_name": "MadeMen94", "lang": "en", "location": "Anderson, SC", "create_at": date("2012-01-15"), "description": "God is Real.Romans 8:18 - IRON ADDICT C.O.D.E FITNESS/ IG:CODEFITNESSUSA/Army Veteran", "followers_count": 1376, "friends_count": 905, "statues_count": 68469 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anderson, SC", "id": "42d68bc4d1591d12", "name": "Anderson", "place_type": "city", "bounding_box": rectangle("-82.707012,34.447616 -82.548858,34.610762") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45007, "countyName": "Anderson", "cityID": 4501360, "cityName": "Anderson" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277633911881728, "text": "On cloud 9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2825301380, "name": "Lilselinsky ❤️", "screen_name": "JaayTheDoll", "lang": "en", "location": "booling W/ Tyler & aj ❤️", "create_at": date("2014-09-21"), "description": "ripelijah", "followers_count": 1072, "friends_count": 613, "statues_count": 30349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277633991548928, "text": "That is awesome! https://t.co/31HBK9o3Se", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 103729129, "name": "Curtis Austin", "screen_name": "sitrucnitsua", "lang": "en", "location": "Charlotte, NC Area", "create_at": date("2010-01-10"), "description": "Airline pilot, ERAU graduate who loves aviation! I like to laugh and have a good time! Opinions are mine, not the company.", "followers_count": 5400, "friends_count": 5913, "statues_count": 25704 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277634071269376, "text": "https://t.co/MiIwH7kvMC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3226079689, "name": "AmericAwoo~", "screen_name": "TrueNugget", "lang": "en", "location": "On a voncanic plateau, Oregon", "create_at": date("2015-05-25"), "description": "Tacti/k/ool | Christian | ily @FalseNugget c: | purple square | gun collector | Twice voted ''most shootable face'' in 2011 and 2013 | potato knish", "followers_count": 3484, "friends_count": 210, "statues_count": 49537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White City, OR", "id": "00b125cb68b761ea", "name": "White City", "place_type": "city", "bounding_box": rectangle("-122.887684,42.411067 -122.801823,42.452409") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4181450, "cityName": "White City" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277634272595968, "text": "We use this filter to much ���� https://t.co/kMPiSpfKJR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3277603332, "name": "Hannahgonzaless", "screen_name": "hannahxxrose", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2015-07-12"), "description": "Bikinis, sand, and tans only ✌ HB", "followers_count": 178, "friends_count": 271, "statues_count": 1331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-05-22T00:00:04.000Z"), "id": 734277634385842176, "text": "My Instagram stock has declined tremendously. I can't even get 10 likes and I got over 1400 followers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2386169708, "name": "FlyEaglesFly", "screen_name": "_johnswish1", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-03-12"), "description": "@Eagles. @Lakers. @FCBarcelona. Wrestling. U.S Air Force 6/7/16 . Xa", "followers_count": 1344, "friends_count": 1040, "statues_count": 116249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277634792771584, "text": "\" they say he's an entertainer..slow down, your just 1 more he don't respect u.. he just gone hurt you & neglect u \" - Neyo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 631646086, "name": "Dan〽️afia ❤️", "screen_name": "_1way_arelisha", "lang": "en", "location": "null", "create_at": date("2012-07-09"), "description": "#PalmBeachCounty☀️(561//305) Flomo19✏️---------------------------------❤️ Rip Mommy 8-13-72//10-21-13", "followers_count": 632, "friends_count": 416, "statues_count": 7134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miramar, FL", "id": "5ca2cc5afa0894df", "name": "Miramar", "place_type": "city", "bounding_box": rectangle("-80.40816,25.956799 -80.205968,25.995449") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1245975, "cityName": "Miramar" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277634813628416, "text": "Felt so real with people from SB✊��and when you gotta go back home..you know everything is fake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4130683094, "name": "T", "screen_name": "ThaliaGonzo", "lang": "en", "location": "Rancho Cucamonga, CA", "create_at": date("2015-11-04"), "description": "EHS waterpolo// with @Victoria doing hood shit//C/O 18", "followers_count": 158, "friends_count": 211, "statues_count": 1546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635094650880, "text": "@jaraamin HAHA aww Jara you so sweet!! Huhu salamat!!!! :) luv u po :)", "in_reply_to_status": 734267794196291588, "in_reply_to_user": 279817158, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 279817158 }}, "user": { "id": 43328564, "name": "Camille Cheang", "screen_name": "cammercheanger", "lang": "en", "location": "MNL, PH", "create_at": date("2009-05-29"), "description": "1 John 4:18 ☺️ || striving for my ikigai || hopes and has big dreams to help the world || ジャニーズのNEWSが好きです!/ 増田貴久担", "followers_count": 814, "friends_count": 648, "statues_count": 10805 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635585363969, "text": "My sisters are forever my bestfriends .. Can't wait to get their names tatted ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438954582, "name": "kenz", "screen_name": "mckinsey_andre", "lang": "en", "location": "Glendora, CA", "create_at": date("2011-12-16"), "description": "mt.sac cheer ❤️ I ride the j-train", "followers_count": 454, "friends_count": 356, "statues_count": 13618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Dimas, CA", "id": "9e42709040fafdb7", "name": "San Dimas", "place_type": "city", "bounding_box": rectangle("-117.854065,34.063734 -117.772051,34.145047") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666070, "cityName": "San Dimas" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635593773056, "text": "Seven more hours until the big reveal... https://t.co/SSka8Kgueb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1392410760, "name": "Dudeboxxx", "screen_name": "dudeboxxx", "lang": "en", "location": "San Diego ", "create_at": date("2013-04-30"), "description": "We are casting! Wanna model? Boys, Men, Daddies, Bears, Twinks, Latino... 18+ only!!! Email us: casting@dudeboxxx.com", "followers_count": 2544, "friends_count": 1755, "statues_count": 1393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635610529793, "text": "i'm always working with new talent", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 38055245, "name": "OFFICIAL Carmin Wong", "screen_name": "carminspalace", "lang": "en", "location": "Worldwide", "create_at": date("2009-05-05"), "description": "International Female DJ, Producer, loves any type of music- 24 hr radio\r\n\r\n Create your World- OFFICIAL http://www.facebook.com/thedjwong", "followers_count": 59044, "friends_count": 25407, "statues_count": 2910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635778301952, "text": "Stewart Indian School alum recounts his days living on campus. A look into historic landmark https://t.co/PCxFyhLWwS https://t.co/coFqA4egdL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 264946980, "name": "Catherine Van", "screen_name": "cat08van", "lang": "en", "location": "Reno", "create_at": date("2011-03-12"), "description": "Reporter at @KOLO8 News Now (ABC) Reno | Homegrown Nevadan | Proud @GonzagaU alumna | @aasmbj co-chair | Harmonizing alto | Lover of puns | Views are my own", "followers_count": 851, "friends_count": 498, "statues_count": 3328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277635883274241, "text": "@IcantwithWILL how am I supposed to know��. idk. https://t.co/yphYfaRicP", "in_reply_to_status": 734276986860933120, "in_reply_to_user": 99632313, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 99632313 }}, "user": { "id": 341568655, "name": "killer queen✨", "screen_name": "QueenZai_", "lang": "en", "location": "null", "create_at": date("2011-07-24"), "description": "Nina Simone inspires me to be a better woman• Impact Dance Team❤️ • BSU19", "followers_count": 967, "friends_count": 757, "statues_count": 44346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277636336164864, "text": "Child shot in Denver (1800 West Mississippi Avenue.) #Denver https://t.co/hCAuxr7vZw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.00871,39.696728"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Denver" }}, "user": { "id": 2416107151, "name": "blockast", "screen_name": "blockast", "lang": "en", "location": "null", "create_at": date("2014-03-28"), "description": "Hyperlocal news near you for Android (https://play.google.com/store/apps/details?id=com.blockast.swipe) or iOS (https://itunes.apple.com/us/app/id1013424661).", "followers_count": 8, "friends_count": 0, "statues_count": 129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277636453597184, "text": "Everybody can't be bosses... We need workers too��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318928533, "name": "Mrs.Brazyho ©", "screen_name": "Jazzmin_Simone", "lang": "en", "location": "dallas, tx", "create_at": date("2011-06-17"), "description": "YAMZZZ", "followers_count": 5515, "friends_count": 2465, "statues_count": 51577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277636546015232, "text": "I really need some positivity in my life cuz this negative shit i really aint with it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 969608587, "name": "Gucci", "screen_name": "FreeWop1017", "lang": "en", "location": "Hermiston, Oregon", "create_at": date("2012-11-24"), "description": "Sensational #1017 sc:c_munoz09 http://freegucciclock.com", "followers_count": 1098, "friends_count": 781, "statues_count": 65889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boardman, OR", "id": "6ad952d03f2df18f", "name": "Boardman", "place_type": "city", "bounding_box": rectangle("-119.721344,45.820748 -119.662605,45.850749") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41049, "countyName": "Morrow", "cityID": 4107200, "cityName": "Boardman" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277636831256576, "text": "**lil uzi https://t.co/ZvTK1EaaA8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 465932724, "name": "will", "screen_name": "indictful", "lang": "en", "location": "null", "create_at": date("2012-01-16"), "description": "atlanta, ga", "followers_count": 5179, "friends_count": 697, "statues_count": 28577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midtown, Atlanta", "id": "38d4bf7b43a3ef4b", "name": "Midtown", "place_type": "neighborhood", "bounding_box": rectangle("-84.395235,33.771232 -84.364585,33.802198") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277636994813952, "text": "wow ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3049107624, "name": "#livinglavish☃", "screen_name": "thatsjeremyy_", "lang": "en", "location": "null", "create_at": date("2015-02-28"), "description": "rags to riche$", "followers_count": 477, "friends_count": 365, "statues_count": 8283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277637787516928, "text": "#rant #over #where is the #weed #Denver ? #sad #day #today no #justice or #mercy for a #marketing #homeless #woman https://t.co/hu1KLfcxVy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.052822,39.724918"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rant", "over", "where", "weed", "Denver", "sad", "day", "today", "justice", "mercy", "marketing", "homeless", "woman" }}, "user": { "id": 732966789475225602, "name": "Simone Kamil", "screen_name": "SimoneKamil", "lang": "en", "location": "Colorado, USA", "create_at": date("2016-05-18"), "description": "Art Black Community Design & Architecture Celebrity Gamers #weed #love what up social media aka #Twitter Let's start some #internet #wars #model #life #God !", "followers_count": 287, "friends_count": 479, "statues_count": 240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wendy's", "id": "07d9e4191fc88000", "name": "Wendy's", "place_type": "poi", "bounding_box": rectangle("-105.0528221,39.7249179 -105.052822,39.724918") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277638022275072, "text": "Lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2397310693, "name": "i v a n", "screen_name": "AlmightyAlvarez", "lang": "en", "location": "goodwill", "create_at": date("2014-03-18"), "description": "1998 // AZ", "followers_count": 310, "friends_count": 149, "statues_count": 7239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277638055813123, "text": "@_Gulliano I beg to differ", "in_reply_to_status": 734277545202327552, "in_reply_to_user": 258581493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 258581493 }}, "user": { "id": 3029626059, "name": "Des", "screen_name": "afakasi_des", "lang": "en", "location": "CA", "create_at": date("2015-02-10"), "description": "SC - theylovedess", "followers_count": 617, "friends_count": 213, "statues_count": 8815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norco, CA", "id": "b8fde561e371a6c2", "name": "Norco", "place_type": "city", "bounding_box": rectangle("-117.606,33.893653 -117.513414,33.966173") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 651560, "cityName": "Norco" } }
+{ "create_at": datetime("2016-05-22T00:00:05.000Z"), "id": 734277638735450112, "text": "College brought the thot out ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 790269426, "name": "Hakuna Mathota", "screen_name": "mannatbhatia08", "lang": "en", "location": "null", "create_at": date("2012-08-29"), "description": "~Hakuna Matata, means no worries✨ ~Stevenson University'19#mustang ~Snapchat/Insta: mannatbhatia08", "followers_count": 391, "friends_count": 401, "statues_count": 11507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miller Place, NY", "id": "cd7c0e53b737ba45", "name": "Miller Place", "place_type": "city", "bounding_box": rectangle("-73.02154,40.904061 -72.956704,40.966649") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3647306, "cityName": "Miller Place" } }
+{ "create_at": datetime("2016-05-22T00:00:06.000Z"), "id": 734277639393837057, "text": "hearing all these stories about my family from when they were teenagers .. it's like damnnn y'all where straight G's", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 965398068, "name": "unreckless__", "screen_name": "miaorozco30", "lang": "en", "location": "the diamond", "create_at": date("2012-11-22"), "description": "stay reppin halo nation .", "followers_count": 440, "friends_count": 670, "statues_count": 13360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640001921757184, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":52.3°F Wind:0.0mph Pressure: 30.00hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 18, "statues_count": 318996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640001967869952, "text": "@Rated_Patti I Am��", "in_reply_to_status": 734639649797353472, "in_reply_to_user": 332510569, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 332510569 }}, "user": { "id": 1519664347, "name": "9/11CaliDay❤️", "screen_name": "CaliSergent", "lang": "en", "location": "New York, NY", "create_at": date("2013-06-15"), "description": "null", "followers_count": 754, "friends_count": 773, "statues_count": 54182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002202783744, "text": "I had a moment and lost all control when I heard Future's album �� https://t.co/5spdHtk2gx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 182716685, "name": "Sizzy Rocket", "screen_name": "sizzyrocket", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-08-24"), "description": "I do it for the thrills // pop music -mgalle@paradigmagency.com drew@photofinishrecords.com - snapchat: sizzyrocket", "followers_count": 4668, "friends_count": 1000, "statues_count": 15368 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002227933184, "text": "Been slacking �� need to get back on my grind! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2224231771, "name": "Cynthia✨", "screen_name": "Camacho_Cyn", "lang": "en", "location": "null", "create_at": date("2013-11-30"), "description": "Forever lost in the sauce", "followers_count": 248, "friends_count": 203, "statues_count": 2435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002366341120, "text": "Mother fuckers always asking me if I live with Matty b, well here's some proof https://t.co/Om2ti6QKdI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 986874618, "name": "Diegz", "screen_name": "DiegoBurrows", "lang": "en", "location": "mattyb's crib", "create_at": date("2012-12-03"), "description": "uhh, oly hhh hi my alt is @isyouboolin", "followers_count": 840, "friends_count": 765, "statues_count": 9392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002513113090, "text": "Wind 0.0 mph ---. Barometer 1016.97 mb, Steady. Temperature 63.2 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002693500929, "text": "@AbyssWalker2 هذا سؤال ثاني، بس كيف ابو فنغر جاءمن دون جيش", "in_reply_to_status": 734639491277852673, "in_reply_to_user": 952316130, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 952316130 }}, "user": { "id": 238740165, "name": "عبدالرحمن المحمدي", "screen_name": "da7my7", "lang": "en", "location": "Seattle - Medina", "create_at": date("2011-01-15"), "description": "snap: da7my7", "followers_count": 1957, "friends_count": 506, "statues_count": 33393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640002991325184, "text": "���������� https://t.co/2bSqfRKK3A", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 624263023, "name": "unbothered☯", "screen_name": "ForeignMamacita", "lang": "en", "location": "null", "create_at": date("2012-07-01"), "description": "follow my snap * Beautiki ❤️ | 3-11-14 ⚛", "followers_count": 1154, "friends_count": 451, "statues_count": 38137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640003054178304, "text": "Ridiculous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2151658686, "name": "Frank Da Tank™", "screen_name": "franciacogjr", "lang": "en", "location": "Forum Park", "create_at": date("2013-10-23"), "description": "All KREAM, no fillin", "followers_count": 582, "friends_count": 685, "statues_count": 11557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640003129737216, "text": "It amazes me how i haven't packed my belongings & moved there to engage in the fuckery. https://t.co/J6vTMIoX0l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19690513, "name": "queenchanel", "screen_name": "_chanelmoore", "lang": "en", "location": "inland empire", "create_at": date("2009-01-28"), "description": "I'm like metro boomin, I know y'all want some more.", "followers_count": 605, "friends_count": 584, "statues_count": 24977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640003280740352, "text": "Sunday Funday! #Friends #Food #Fun #CraftBeers #Cocktails #AlienBoobies��#Gose… https://t.co/ybOnaTKx11", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.03527778,32.65833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Friends", "Food", "Fun", "CraftBeers", "Cocktails", "AlienBoobies", "Gose" }}, "user": { "id": 2207953574, "name": "Dennicita", "screen_name": "Alonso7349", "lang": "en", "location": "Chula Vista", "create_at": date("2013-11-21"), "description": "♍ Blessed 42 year old Nana W/3 daughters 1 in heaven enjoy life #❤Occupied #49ers #Praying4Kap #SFGiants #GSW #SJSharks", "followers_count": 3281, "friends_count": 3277, "statues_count": 46613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bonita, CA", "id": "dfb4dd1ba29fead1", "name": "Bonita", "place_type": "city", "bounding_box": rectangle("-117.065683,32.638243 -116.991198,32.691595") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 607414, "cityName": "Bonita" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640003670761472, "text": "and you're not?? https://t.co/2BOQLov3wR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2474789846, "name": "KR", "screen_name": "karengrojass", "lang": "en", "location": "RIALTO", "create_at": date("2014-05-02"), "description": "sixteen", "followers_count": 253, "friends_count": 158, "statues_count": 5258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640003813412864, "text": "I can't even lie baby go krazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2650032180, "name": "James Looney", "screen_name": "james__looney", "lang": "en", "location": "null", "create_at": date("2014-07-15"), "description": "C/o 17 SS - WR \nOur deepest fear is not that we are inadequate. Our deepest fear is that we are powerful beyond measure.", "followers_count": 597, "friends_count": 431, "statues_count": 4931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640004186705920, "text": "7 https://t.co/O7Rfe8UpqI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3227480792, "name": "R.White‼️", "screen_name": "ReggieGoCrazzyy", "lang": "en", "location": "EveryWhere,You Never There", "create_at": date("2015-05-26"), "description": "null", "followers_count": 222, "friends_count": 106, "statues_count": 11574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640004492865537, "text": "@BiancaaaCeleste the tv show was a huge part of my childhood", "in_reply_to_status": 734639903946989568, "in_reply_to_user": 1541847259, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1541847259 }}, "user": { "id": 3014032244, "name": "Kat (✿´‿`)", "screen_name": "whereyoukat", "lang": "en", "location": "California, USA", "create_at": date("2015-02-08"), "description": "I wish everyone could see how beautiful they are", "followers_count": 330, "friends_count": 374, "statues_count": 10752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640004807462912, "text": "Temp 44.9° Hi/Lo 48.8/44.9 Rng 3.9° WC 44.9° Hmd 94% Rain 0.00\" Storm 0.00\" BAR 29.911 Falling DP 43.3° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 18915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640004908060672, "text": "@XoXo_Katerina seriously right? Like does it really matter? Either you believe me or not, but don't say I'm still lying.", "in_reply_to_status": 734615301845864448, "in_reply_to_user": 120545600, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 120545600 }}, "user": { "id": 45018026, "name": "mel", "screen_name": "melissaasaechao", "lang": "en", "location": "portland/troutdale OR", "create_at": date("2009-06-05"), "description": "Pour yourself a drink, put on some lipstick and pull yourself together. IG: yayitsmelissaa • Snapchat melissaasaechao", "followers_count": 319, "friends_count": 286, "statues_count": 26429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troutdale, OR", "id": "e0e015b833cb12b4", "name": "Troutdale", "place_type": "city", "bounding_box": rectangle("-122.431645,45.506422 -122.368489,45.561366") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4174850, "cityName": "Troutdale" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640005231042560, "text": "Happy Birthday @kimmbbeerly_!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3311083872 }}, "user": { "id": 852342428, "name": "nick.", "screen_name": "TheRealNickGtz", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-09-28"), "description": "zot on.", "followers_count": 302, "friends_count": 265, "statues_count": 11536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:00.000Z"), "id": 734640005830868992, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2876986939, "name": "Rg✨", "screen_name": "GuevaraRoxanne", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-10-25"), "description": "dis drama way2much u need to CUT IT", "followers_count": 81, "friends_count": 122, "statues_count": 2933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640005923143680, "text": "@ebbtideapp Tide in Saxis, Maryland 05/23/2016\n Low 8:32am 0.1\nHigh 1:56pm 2.1\n Low 8:27pm 0.2\nHigh 2:08am 2.6\n Low 9:09am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.7283,37.9217"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 93, "friends_count": 1, "statues_count": 35131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51001, "countyName": "Accomack", "cityID": 5170576, "cityName": "Saxis" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640006044733440, "text": "@Jd_Ags6 yeah all my brothers call me chano", "in_reply_to_status": 734639937375592448, "in_reply_to_user": 1018551522, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1018551522 }}, "user": { "id": 865832208, "name": "Marci", "screen_name": "_Marciano__", "lang": "en", "location": "AZ", "create_at": date("2012-10-06"), "description": "laughing with krillin saying somethin bout blonde hair @Cieloselen❤️", "followers_count": 666, "friends_count": 419, "statues_count": 11353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640006065737728, "text": "Can't wait till my hair gets as long as it was", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 378371225, "name": "⁶ God", "screen_name": "Jarron_Turner", "lang": "en", "location": "maysville to rich city", "create_at": date("2011-09-22"), "description": "LOAYB", "followers_count": 943, "friends_count": 579, "statues_count": 21298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maysville, KY", "id": "784bd579604dcc20", "name": "Maysville", "place_type": "city", "bounding_box": rectangle("-83.831754,38.594038 -83.719828,38.658613") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21161, "countyName": "Mason", "cityID": 2151024, "cityName": "Maysville" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640006204116992, "text": "Going back to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3239674683, "name": "Steve‼️", "screen_name": "_stevenyc", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "God Got Me", "followers_count": 492, "friends_count": 378, "statues_count": 40238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640006527078400, "text": "Exactly 21 years on the this earth never thought I'd be where i am today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3250350215, "name": "Its My Birthday", "screen_name": "Brickface23", "lang": "en", "location": "Chiraq, CA", "create_at": date("2015-05-12"), "description": "in my thoughts, ambitious yet lost. 408", "followers_count": 571, "friends_count": 611, "statues_count": 2154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640007089098752, "text": "@_nicolebria �� act like they ain't never had sex b4!", "in_reply_to_status": 734639770731679745, "in_reply_to_user": 335725139, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 335725139 }}, "user": { "id": 301498732, "name": "queen⚓️", "screen_name": "ellerehc_", "lang": "en", "location": "Conventry, Jupiter ❤️", "create_at": date("2011-05-19"), "description": "introspection & marijuana enthusiast. pv.❤️", "followers_count": 1364, "friends_count": 1008, "statues_count": 66093 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Natalbany, LA", "id": "1431082e75835e96", "name": "Natalbany", "place_type": "city", "bounding_box": rectangle("-90.554321,30.519012 -90.446532,30.584832") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2253475, "cityName": "Natalbany" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640007646973953, "text": "05/23@03:00 - Temp 54.4F, WC 54.4F. Wind 0.0mph NW, Gust 1.0mph. Bar 29.901in, Falling slowly. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640007684710400, "text": "happy birthday hun! �� @stevelacys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1314672745 }}, "user": { "id": 708586550, "name": "ariel", "screen_name": "aarielalanis", "lang": "en", "location": "Downey, CA", "create_at": date("2012-07-21"), "description": "https://soundcloud.com/ariel-molina-23", "followers_count": 268, "friends_count": 184, "statues_count": 201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008200650752, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2502919736, "name": "♡Deαnnα. В♡", "screen_name": "loyaltydeanna", "lang": "en", "location": "Fall River, MA", "create_at": date("2014-05-17"), "description": "A free Spirt with a wild heart♡", "followers_count": 490, "friends_count": 906, "statues_count": 11286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Somerset, MA", "id": "b92b64859c6e6e7c", "name": "Somerset", "place_type": "city", "bounding_box": rectangle("-71.196087,41.706626 -71.119104,41.79094") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2562465, "cityName": "Somerset" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008238403584, "text": "Ripley SW Limestone Co. Temp: 55.9°F Wind:0.0mph Pressure: 995.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53658 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008393596928, "text": "This hard. Cuz each team got somebody undefeated on it ������ https://t.co/rgqVmI4gVh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 313475569, "name": "baekamaru nara", "screen_name": "allohhhh", "lang": "en", "location": "null", "create_at": date("2011-06-08"), "description": "souf side shorty, poofy haired princess. culturally diverse in this thang, big ups. | Widowed 042116 RIP Prince.", "followers_count": 1891, "friends_count": 682, "statues_count": 48028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008590729216, "text": "Scorpio �� https://t.co/q1l7sWLteP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "it", "is_retweet": false, "user": { "id": 2866621619, "name": "Bridgette", "screen_name": "brii_casandra", "lang": "en", "location": "Atx", "create_at": date("2014-11-07"), "description": "Pre-PT Exercise and Science Major | Ancora Imparo | God Is Greater than the highs and the lows", "followers_count": 1469, "friends_count": 1952, "statues_count": 11958 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008745877504, "text": "guuuurrrl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 3366617172, "name": "a$ap", "screen_name": "Riss__barajas", "lang": "en", "location": "d block", "create_at": date("2015-08-27"), "description": "J16GY❣diaz❣", "followers_count": 344, "friends_count": 329, "statues_count": 8003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640008955629568, "text": "Chance man.. ������ https://t.co/7mTU9ynE6D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1658769871, "name": "#lexfromsheldon", "screen_name": "alexisikesaki", "lang": "en", "location": "soccer field", "create_at": date("2013-08-09"), "description": "I don't even know what to put here tbh", "followers_count": 956, "friends_count": 397, "statues_count": 7334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640009618296833, "text": "Me: \"Is your vagina an innie or an outtie?\"\n\nHer: \"My uterus & ovaries hang out of me & I tuck them into my YSL clutch.\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 234565264, "name": "duhvorRay.", "screen_name": "OMG_ItsDeVore", "lang": "en", "location": "Germany ✈️ PNW (253)", "create_at": date("2011-01-05"), "description": "not here for friends, can't you tell by my following to follower ratio? — SnapChat: OMG_ItsDeVore", "followers_count": 1204, "friends_count": 125, "statues_count": 80991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640009882546176, "text": "03:00 50.6°F Feels:50.6°F (Hi58.5°F/Lo50.6°F) Hum:72% Wnd:--- 0.0MPH Baro:30.18in. Prcp:0.00in https://t.co/mEzzB0ajqR #inWX #weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.69888889,39.29777778"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "inWX", "weather" }}, "user": { "id": 1522989468, "name": "GosportINWX", "screen_name": "GosportINWX", "lang": "en", "location": "Gosport, IN, USA", "create_at": date("2013-06-16"), "description": "Gosport, IN's ONLY live weather reported by a personal weather station. Providing WXAlerts, HDSkyCam, and other useful weather information!", "followers_count": 209, "friends_count": 265, "statues_count": 29952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, USA", "id": "1010ecfa7d3a40f8", "name": "Indiana", "place_type": "admin", "bounding_box": rectangle("-88.097892,37.771743 -84.78458,41.761368") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18119, "countyName": "Owen" } }
+{ "create_at": datetime("2016-05-23T00:00:01.000Z"), "id": 734640009945468928, "text": "Wind 0.0 mph ---. Barometer 29.976 in, Steady. Temperature 51.0 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 20018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640010197143552, "text": "@JayyPistachio happy birthday hoe ily��", "in_reply_to_status": -1, "in_reply_to_user": 2280951276, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2280951276 }}, "user": { "id": 2419309934, "name": "Untitled", "screen_name": "martinvargas_19", "lang": "en", "location": "null", "create_at": date("2014-03-30"), "description": "jk", "followers_count": 126, "friends_count": 126, "statues_count": 6216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640010524233728, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3260854411, "name": "valencia", "screen_name": "tigerrlily___", "lang": "en", "location": "Bay area", "create_at": date("2015-06-29"), "description": "thuggin'", "followers_count": 299, "friends_count": 174, "statues_count": 4588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640010863988738, "text": "Hell yes ❗️❗️❗️ https://t.co/jlVorbyumg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 594877357, "name": "Starrrrrrr✨", "screen_name": "Totally_starr", "lang": "en", "location": "California, USA", "create_at": date("2012-05-30"), "description": "Big Zaddy Starr❗️Free Spirit With A Wild ❤️ #PowerHouse ✊ I Get Lost In My Thoughts", "followers_count": 793, "friends_count": 626, "statues_count": 13056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Mateo, CA", "id": "432daa3153c5fef9", "name": "San Mateo", "place_type": "city", "bounding_box": rectangle("-122.355038,37.512567 -122.275805,37.592395") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 668252, "cityName": "San Mateo" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640010947899393, "text": "Wind 0.0 mph ---. Barometer 29.985 in, Steady. Temperature 56.7 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640010994061312, "text": "I can't tell if she's yours or mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 918122162, "name": "Rich", "screen_name": "RichardBanda10", "lang": "en", "location": "null", "create_at": date("2012-10-31"), "description": "we all fall one day", "followers_count": 653, "friends_count": 390, "statues_count": 32130 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011308617728, "text": "dude shut the fuck up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1357322840, "name": "㏍♕", "screen_name": "keeksiebaby", "lang": "en", "location": "OR", "create_at": date("2013-04-16"), "description": "sweet sassy & sensitive ❥ 12/O5/2O15", "followers_count": 300, "friends_count": 121, "statues_count": 24115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011405099010, "text": "**someone says lets go to inglewood**\n\n*i say no cuz that's never a good idea at midnight*\n\n**almost got ran up on*\n\nMy friends suck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 282869267, "name": "not really", "screen_name": "TeamTesh", "lang": "en", "location": "Hollywood (Los Angeles), CA", "create_at": date("2011-04-15"), "description": "Secretly I'm Chris Brown | I don't trust people with dirty shoes | I like surfing", "followers_count": 484, "friends_count": 223, "statues_count": 26846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011589648385, "text": "Temp: 58.7°F Wind:0.0mph Pressure: 29.996hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 64905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011853856769, "text": "Big hug...can definitely use one right now!\n#2amwemeetagain #cuddletime https://t.co/f3YmygLPQu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "2amwemeetagain", "cuddletime" }}, "user": { "id": 3195571100, "name": "Nina Katherina", "screen_name": "NinaChasingLife", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-05-14"), "description": "Fabulously flawed Filipina who's a wanderlusting, crossword puzzle-loving, cancer-fighting LIFE chaser.", "followers_count": 520, "friends_count": 535, "statues_count": 1190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Streamwood, IL", "id": "a6af648ddff22b3b", "name": "Streamwood", "place_type": "city", "bounding_box": rectangle("-88.230526,41.989279 -88.138345,42.066971") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1773157, "cityName": "Streamwood" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011870674945, "text": "I've never even been more tired of askin for the same thing and not gettin it everytime", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 130078140, "name": " yung lil wavey", "screen_name": "mellowsem", "lang": "en", "location": "4th&Goal", "create_at": date("2010-04-06"), "description": "•OKC for the ship bitch •Rip Cody •JB♥️", "followers_count": 460, "friends_count": 370, "statues_count": 22978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Delton, WI", "id": "0019a3ac7fd41a96", "name": "Lake Delton", "place_type": "city", "bounding_box": rectangle("-89.818561,43.575648 -89.756077,43.615728") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55111, "countyName": "Sauk", "cityID": 5541300, "cityName": "Lake Delton" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640011988111360, "text": "Temp 45.8°F Wind Chill 45.8°F RH 95% Wind 0.0 --- Gust 0.0 --- SLP 29.962 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 63, "statues_count": 35628 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012042592257, "text": "@STELLAGlBSONS I still can't believe we enabled her to find that trash", "in_reply_to_status": 734639862071058436, "in_reply_to_user": 404569958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 404569958 }}, "user": { "id": 2269829739, "name": "claire ;)", "screen_name": "themariskafiles", "lang": "en", "location": "1600 Pennsylvania Ave NW", "create_at": date("2014-01-08"), "description": "Gillian Anderson tried to work out in a prom dress.", "followers_count": 1376, "friends_count": 130, "statues_count": 24154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012189388802, "text": "Wind 0.0 mph ---. Barometer 29.918 in, Steady. Temperature 44.7 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10915 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012311072768, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Rising slowly. Temperature 61.5 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 22981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012638179328, "text": "https://t.co/JdmfCVsv2E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.385327,34.081285"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1237250305, "name": "bridget", "screen_name": "bridgettindell", "lang": "en", "location": "null", "create_at": date("2013-03-02"), "description": "null", "followers_count": 693, "friends_count": 416, "statues_count": 4791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sur Restaurant and Bar", "id": "07d9e4b677485001", "name": "Sur Restaurant and Bar", "place_type": "poi", "bounding_box": rectangle("-118.3853271,34.0812849 -118.385327,34.081285") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012684361728, "text": "you're perfect for me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 626211869, "name": "Mr. finesse.", "screen_name": "realdealoneal7", "lang": "en", "location": "DeLand, FL✈️✈️Tempe, Az ", "create_at": date("2012-07-03"), "description": "Don't introduce me to your girl", "followers_count": 1039, "friends_count": 697, "statues_count": 22925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012759859200, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 16:15 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15662 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012768206850, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 14:02 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15662 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012780830721, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 16:17 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15662 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012805955584, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 13:35 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15662 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640012831150081, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 14:30 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15662 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640013233786880, "text": "Developing some great relationships in my life with others, and I couldn't be happier", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2715099096, "name": "Declan Fleming", "screen_name": "dfleming1108", "lang": "en", "location": "Round Rock, TX", "create_at": date("2014-08-07"), "description": "null", "followers_count": 277, "friends_count": 277, "statues_count": 1478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Rock, TX", "id": "00c44eeb126d2fcd", "name": "Round Rock", "place_type": "city", "bounding_box": rectangle("-97.755394,30.468442 -97.59007,30.570239") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4863500, "cityName": "Round Rock" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640013477040129, "text": "@RealPhilBowdry Thanks bro, I appreciate it!������", "in_reply_to_status": 734637795885600768, "in_reply_to_user": 342331848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 342331848 }}, "user": { "id": 503667687, "name": "Devin Cruz™", "screen_name": "RealDAWC", "lang": "en", "location": "St. Louis, MO", "create_at": date("2012-02-25"), "description": "C O T C | U p S e t | M A C L i f e", "followers_count": 909, "friends_count": 1558, "statues_count": 24189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University City, MO", "id": "2dd9882dad5ba631", "name": "University City", "place_type": "city", "bounding_box": rectangle("-90.366578,38.645912 -90.297186,38.683509") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2975220, "cityName": "University City" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640013577687040, "text": "@c_schaum @YungKeem4rmLB https://t.co/nGpnheqyLl", "in_reply_to_status": 734639627626221568, "in_reply_to_user": 2321690569, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2321690569, 487173130 }}, "user": { "id": 1484176332, "name": "Goose!", "screen_name": "realaddresses", "lang": "en", "location": "Upland, CA", "create_at": date("2013-06-04"), "description": "|Father time wins all battles| |Anu❤️|", "followers_count": 3510, "friends_count": 874, "statues_count": 60767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olympia, WA", "id": "375aa1ab4abd79b1", "name": "Olympia", "place_type": "city", "bounding_box": rectangle("-123.023734,46.98219 -122.823298,47.146866") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5351300, "cityName": "Olympia" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640013732941825, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 10:05 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15667 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640013774868480, "text": "1 Berylline Hummingbird (Amazilia beryllina) - Madera Canyon--Santa Rita Lodge - 2016-05-22 08:40 https://t.co/MFxHRU5d4U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8801067,31.7251514"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15667 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4023, "countyName": "Santa Cruz" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640014072684544, "text": "Wind 0.0 mph ---. Barometer 29.936 in, Steady. Temperature 57.5 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 64906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-23T00:00:02.000Z"), "id": 734640014164955136, "text": "I hope so ������ https://t.co/vzcFmwe091", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3368250244, "name": "Capricorn♑️'", "screen_name": "maarrcchheell", "lang": "en", "location": "Midwest City, OK", "create_at": date("2015-07-09"), "description": "New Twitter. #LongLiveRashya | #LongLiveCleato | 919✈️4O5 | sc @bvlden_bae |", "followers_count": 490, "friends_count": 891, "statues_count": 3207 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midwest City, OK", "id": "5327a9b6dceff63e", "name": "Midwest City", "place_type": "city", "bounding_box": rectangle("-97.424091,35.433034 -97.291149,35.507694") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4048350, "cityName": "Midwest City" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640014437535745, "text": "Goodnight https://t.co/rqBnV7PYKV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373066247, "name": "the beautiful one", "screen_name": "DevonneeSings", "lang": "en", "location": "Alhambra, CA", "create_at": date("2011-09-13"), "description": "Black. Christian. Liberal. Some kind of operatic mezzo-soprano by day and Neo-soul singer at heart. Coffee enthusiast. The beautiful one. RIP Prince.", "followers_count": 237, "friends_count": 220, "statues_count": 18869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alhambra, CA", "id": "d563cf7307e4ba95", "name": "Alhambra", "place_type": "city", "bounding_box": rectangle("-118.165119,34.05998 -118.108233,34.111213") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600884, "cityName": "Alhambra" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640014517260288, "text": "@plutoniumpage could be b/t 1955-1982 based on flags. But 55-70s based on provenance:\n\nhttps://t.co/Fob6FEYnUe", "in_reply_to_status": 734603116654530560, "in_reply_to_user": 12211422, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12211422 }}, "user": { "id": 2448077035, "name": "martin pfeiffer", "screen_name": "NuclearAnthro", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2014-04-16"), "description": "Anthropologist in training | UNM PhD student| | nuclear weapons, WMD, deterrence, AGW, cats, saving the world | personal account but fieldwork | RT/Fave/Etc ≠ E", "followers_count": 1863, "friends_count": 2034, "statues_count": 51515 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640014852755458, "text": "Wind 0.0 mph WSW. Barometer 29.94 in, Steady. Temperature 54.7 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 122, "statues_count": 159993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640015481942016, "text": "Good Night ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455019869, "name": "kylee sereno", "screen_name": "kysereno", "lang": "en", "location": "Terre Haute, IN", "create_at": date("2012-01-04"), "description": "Saint Mary-of-the-Woods College rip TDG", "followers_count": 920, "friends_count": 888, "statues_count": 26413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Terre Haute, IN", "id": "00847cd46baf4a56", "name": "North Terre Haute", "place_type": "city", "bounding_box": rectangle("-87.406944,39.520425 -87.341027,39.575965") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18167, "countyName": "Vigo", "cityID": 1855098, "cityName": "North Terre Haute" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640015976898561, "text": "Im wearing my uggs everyday for the next two week", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32785453, "name": "Tiara O.", "screen_name": "TiaraAshley_", "lang": "en", "location": "Georgia, USA", "create_at": date("2009-04-17"), "description": "Graphic Designer & Blogger", "followers_count": 175, "friends_count": 195, "statues_count": 10955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640016035598337, "text": "I'm so glad Sarah broke her phone because she could have burned me, got me caught up and fuck just got me in trouble in general ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2355511950, "name": "JULE", "screen_name": "yuliithebratt_", "lang": "en", "location": "#20LBCC⚾️", "create_at": date("2014-02-21"), "description": "I stay doing me boo boo", "followers_count": 900, "friends_count": 966, "statues_count": 16625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Fe Springs, CA", "id": "00b028b400039d97", "name": "Santa Fe Springs", "place_type": "city", "bounding_box": rectangle("-118.098575,33.882744 -118.028833,33.975123") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669154, "cityName": "Santa Fe Springs" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640016396275712, "text": "@Jarret_17 https://t.co/nlaHbDaoUO", "in_reply_to_status": -1, "in_reply_to_user": 728015765887012864, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 728015765887012864 }}, "user": { "id": 545500724, "name": "Alexis", "screen_name": "AlexisLagle", "lang": "en", "location": "WaKeeney ✈️ Columbus, KS", "create_at": date("2012-04-04"), "description": "when life gives you lemonade, add vodka⚜", "followers_count": 373, "friends_count": 257, "statues_count": 3817 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, KS", "id": "1327db06dcde7502", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-94.867998,37.158625 -94.823349,37.188537") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20021, "countyName": "Cherokee", "cityID": 2015075, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640016492789764, "text": "I'm not suppose to laugh but mayne.. https://t.co/RnUqjVACbh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393425348, "name": "Playa Ass Diego™", "screen_name": "AFatChickHero", "lang": "en", "location": "Rest with God lil homie Ebby", "create_at": date("2011-10-18"), "description": "| I'm The greatest thing that happened to black women since weave glue | Just an average man tryna get me a 30 toes | Future educator | SD✈️ATX✈️DFW | #TAMUC", "followers_count": 866, "friends_count": 255, "statues_count": 83487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640016786395136, "text": "@trillassjuan_ ������������", "in_reply_to_status": 734639914864742400, "in_reply_to_user": 3241876592, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3241876592 }}, "user": { "id": 2326514665, "name": "Cristian", "screen_name": "trillassOzzy", "lang": "en", "location": "idk", "create_at": date("2014-02-03"), "description": "Ja Zelim da mi Debela pitcka vozi kurac", "followers_count": 142, "friends_count": 128, "statues_count": 827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640017205829632, "text": "let me close my eyelids", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2531923205, "name": "nia", "screen_name": "niaamyaa", "lang": "en", "location": "null", "create_at": date("2014-05-05"), "description": "foodalistic", "followers_count": 1360, "friends_count": 976, "statues_count": 86379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640017226747905, "text": "https://t.co/EnmqQ8NgtK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 153207726, "name": "MATTE", "screen_name": "ohhwowmatt", "lang": "en", "location": "Mira Mesa, San Diego", "create_at": date("2010-06-07"), "description": "KF | I like sneakers", "followers_count": 558, "friends_count": 293, "statues_count": 20265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640017428127745, "text": "Now send me my Pride rally towel! https://t.co/tVPghmkswX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15447075, "name": "Lesley Goldberg", "screen_name": "Snoodit", "lang": "en", "location": "L.A.", "create_at": date("2008-07-15"), "description": "TV News Editor at @THR. I tweet a lot about breaking TV news, the Dodgers, L.A. and gay stuff. My views are my own.", "followers_count": 16716, "friends_count": 752, "statues_count": 50754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640018015293440, "text": "WATCH GAME OF THRONES!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 358525681, "name": "Little Ceasar's", "screen_name": "MCesar_6", "lang": "en", "location": "null", "create_at": date("2011-08-19"), "description": "Batman sucks.", "followers_count": 118, "friends_count": 187, "statues_count": 6649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640018115940352, "text": "I'm staying up cause I have this presentation to do tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2370815736, "name": "Louie", "screen_name": "Memorabilities", "lang": "en", "location": "Space", "create_at": date("2014-03-03"), "description": "Memories are everything. Cherish them.", "followers_count": 389, "friends_count": 334, "statues_count": 10370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Springfield, MA", "id": "9f4583bf6dae13c4", "name": "Springfield", "place_type": "city", "bounding_box": rectangle("-72.620117,42.06398 -72.471052,42.162145") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2567000, "cityName": "Springfield" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640018199842816, "text": "Everything is always so funny at this time of night����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3282101928, "name": "Papi Joe", "screen_name": "josephinfante5", "lang": "en", "location": "null", "create_at": date("2015-07-16"), "description": "matate", "followers_count": 38, "friends_count": 51, "statues_count": 288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-23T00:00:03.000Z"), "id": 734640018397003776, "text": "@C_H_6 https://t.co/y2V8VHPg73", "in_reply_to_status": -1, "in_reply_to_user": 66428531, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 66428531 }}, "user": { "id": 2339279522, "name": "The Tamp King", "screen_name": "vinson_vj", "lang": "en", "location": "Sandy Utah", "create_at": date("2014-02-11"), "description": "Pitcher at Hannibal LA grange university. don't be semi pro!", "followers_count": 227, "friends_count": 892, "statues_count": 868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandy, UT", "id": "fb9549afa6f81fa6", "name": "Sandy", "place_type": "city", "bounding_box": rectangle("-111.921658,40.528084 -111.800273,40.616827") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967440, "cityName": "Sandy" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640018631892992, "text": "@villalobossebas I love you babe", "in_reply_to_status": 734639554641186816, "in_reply_to_user": 181300983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 181300983 }}, "user": { "id": 2350808378, "name": "мonѕerraтн jυarez♕", "screen_name": "Its_Monserrath", "lang": "en", "location": "Houston, TX", "create_at": date("2014-02-18"), "description": "15 year old with a dream✨| L3P | Juan Palacios❤️ | Directioner❤️ | ◻️sider | Masonator❤️ | Johnsonator❤️ | 1/12 1/5", "followers_count": 1158, "friends_count": 1400, "statues_count": 32955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640018774429696, "text": "@Jgant93 yesss I'm forsure going! come with!! my broke ass has been rt every offer for tickets��", "in_reply_to_status": 734639673507753984, "in_reply_to_user": 824204138, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 824204138 }}, "user": { "id": 448627864, "name": "Nina", "screen_name": "NinaVillan25", "lang": "en", "location": "null", "create_at": date("2011-12-27"), "description": "express yourself", "followers_count": 396, "friends_count": 262, "statues_count": 7359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Novato, CA", "id": "3b0eea538598dc42", "name": "Novato", "place_type": "city", "bounding_box": rectangle("-122.62422,38.039015 -122.486658,38.147701") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6041, "countyName": "Marin", "cityID": 652582, "cityName": "Novato" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640018845765632, "text": "ma main squeeze #sundaze #dabeachmon https://t.co/ckkHfJn3Ka", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "sundaze", "dabeachmon" }}, "user": { "id": 860168034, "name": "shaynegaroo", "screen_name": "ShayneAlissa", "lang": "en", "location": "null", "create_at": date("2012-10-03"), "description": "this is your life. do wat u love & do it often. live ur dream & share ur passion. my life, my adventure.", "followers_count": 50, "friends_count": 145, "statues_count": 415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640019122585600, "text": "Wind 3.2 mph ESE. Barometer 29.860 in, Falling. Temperature 71.5 °F. Rain today 0.00 in. Humidity 86%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640019223248896, "text": "Fuck all y'all niggas who are off of school already", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2838610369, "name": "No Chill Ivan", "screen_name": "IvanJust_", "lang": "en", "location": "Estabas en Narnia", "create_at": date("2014-10-02"), "description": "~ Ass Heaven ~ Junior ~ #LillyStrong C/O 17' ~ Let go and let GOD ~ SC: ivansprinkles7 ~ FFA VP ~ XC ~", "followers_count": 771, "friends_count": 734, "statues_count": 14027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hebbronville, TX", "id": "6c80b8c28fd29be3", "name": "Hebbronville", "place_type": "city", "bounding_box": rectangle("-98.694359,27.292917 -98.663161,27.323508") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48247, "countyName": "Jim Hogg", "cityID": 4833008, "cityName": "Hebbronville" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640019550441473, "text": "99% percent of the time, if you tell me something, I won't give a fuck. This is college guys, not lets fuck our lives and go in debt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2347259780, "name": "brett", "screen_name": "yellowbozza", "lang": "en", "location": "null", "create_at": date("2014-02-16"), "description": "La vida es tan sorda , le gritas y no escucha nada. La vida es un tesoro que hay que apreciar -Manuel Carrasco", "followers_count": 188, "friends_count": 189, "statues_count": 1209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640019932106753, "text": "@PJWilliams02 oh my god is this a thing", "in_reply_to_status": 734633650462044160, "in_reply_to_user": 730894714681970688, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 730894714681970688 }}, "user": { "id": 2376690211, "name": "John Rider", "screen_name": "jtrider16", "lang": "en", "location": "null", "create_at": date("2014-03-06"), "description": "Poorly scripted guest star in your Truman delusion. UC Berkeley class of 2020.", "followers_count": 518, "friends_count": 496, "statues_count": 15605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020158599169, "text": "I never took nothing for granted and stayed prayed up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 242159893, "name": "100 Proof #WhoDey", "screen_name": "1Swago", "lang": "en", "location": "Chasing A Dream, USA", "create_at": date("2011-01-23"), "description": "Swago from Cincinnati. Check me out on iTunes, Spotify. I believe in GOD and I do this by the grace he gives. Go Bengals.", "followers_count": 5581, "friends_count": 2321, "statues_count": 46471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020292837379, "text": "We had a pool party, then a jawn at Tequila & then came back to the pool for the afty����bitxhes was drunk as shit in the water", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 162576107, "name": "Xiomara", "screen_name": "purebreed_94", "lang": "en", "location": "null", "create_at": date("2010-07-03"), "description": "IUP16 Kamar❤️", "followers_count": 987, "friends_count": 577, "statues_count": 61843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indiana, PA", "id": "d88a5def1d7e9609", "name": "Indiana", "place_type": "city", "bounding_box": rectangle("-79.213942,40.586024 -79.089163,40.656614") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42063, "countyName": "Indiana", "cityID": 4236816, "cityName": "Indiana" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020599021569, "text": "Want to work at Favorite Health Care Staffing? We're #hiring in #Friendswood, TX! Click for details: https://t.co/68n7qO671E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.2010447,29.5293998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Friendswood" }}, "user": { "id": 4181892253, "name": "Favorite Jobs", "screen_name": "FavoriteJobs", "lang": "en", "location": "United States", "create_at": date("2015-11-09"), "description": "We're your Advocate. We're your Family. Follow us for #healthcare & #nursing #jobs at @FavoriteStaff", "followers_count": 217, "friends_count": 100, "statues_count": 4503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Friendswood, TX", "id": "6dd590c61801b09c", "name": "Friendswood", "place_type": "city", "bounding_box": rectangle("-95.233093,29.460644 -95.145844,29.563312") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4827648, "cityName": "Friendswood" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020699664385, "text": "@JaySteezy22 is bringing EVERYONE back together when he comes back and trust we are fucking shit up", "in_reply_to_status": -1, "in_reply_to_user": 414167016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 414167016 }}, "user": { "id": 431288526, "name": "#KING VCTR", "screen_name": "Stephanies_Twin", "lang": "en", "location": "null", "create_at": date("2011-12-07"), "description": "KEVS | MUA | Growing and Glowing cause my highlight poppin", "followers_count": 959, "friends_count": 1605, "statues_count": 24002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020724830208, "text": "@ThunderCaya ugh I'm glad I wasn't in the room where it happened", "in_reply_to_status": 734639883298443265, "in_reply_to_user": 17576479, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17576479 }}, "user": { "id": 101956535, "name": "glenda poops", "screen_name": "nightxpain", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-01-04"), "description": "anything for an ER", "followers_count": 747, "friends_count": 183, "statues_count": 76217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020737396736, "text": "�������� my nigga Philthy foreign we stay hittin moves we always lookin to find hoes or gettin to the cash when we link up ���� hmu tho folk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1639942140, "name": "Baby Face❄️", "screen_name": "nolimit_shark", "lang": "en", "location": "Kentraq", "create_at": date("2013-08-02"), "description": "||KRHS/HCC|| c/o 2016|| sc: ayywale_10 | 253 | #HeatNation #Justice4Hamza ✊", "followers_count": 446, "friends_count": 336, "statues_count": 3262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020745801728, "text": "While everyone is bawling their eyes out over Re:Zero, I'm over here doing the same for Plastic Memories.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48586591, "name": "Lee @ not your HMart", "screen_name": "sonikku10", "lang": "en", "location": "Atlanta, GA", "create_at": date("2009-06-18"), "description": "Photographer, dreamer, and lost traveler.", "followers_count": 135, "friends_count": 182, "statues_count": 15878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020800327681, "text": "Water proof cases come in clutch", "in_reply_to_status": 734639619304763392, "in_reply_to_user": 365170052, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365170052, "name": "MMM", "screen_name": "FoolishMireya", "lang": "en", "location": "56IX2", "create_at": date("2011-08-30"), "description": "that was then this is now", "followers_count": 896, "friends_count": 681, "statues_count": 31744 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640020892602369, "text": "@ryesilverman Definitely seems like it to me! Plus we have a scene from the trailer that appears to show her in Volantis.", "in_reply_to_status": 734631202674593796, "in_reply_to_user": 27869369, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27869369 }}, "user": { "id": 14753450, "name": "Q.Z. Lau 劉親智", "screen_name": "qzlau", "lang": "en", "location": "Santa Barbara, California", "create_at": date("2008-05-12"), "description": "I was born the day the music died. @Princeton ’11, East Asian Studies. @UCSantaBarbara MA/Ph.D candidate, History. 中文, 日本語, ASL.\n\nSelf-professed Ravenclaw.", "followers_count": 226, "friends_count": 473, "statues_count": 3138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-05-23T00:00:04.000Z"), "id": 734640022272540672, "text": "@edmbruh I'm pissed ��", "in_reply_to_status": 734639925895761921, "in_reply_to_user": 819052946, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 819052946 }}, "user": { "id": 275757443, "name": "ANNIE", "screen_name": "wass_good_annie", "lang": "en", "location": "los angeles", "create_at": date("2011-04-01"), "description": "✨Philippians 4:13 ✨", "followers_count": 1094, "friends_count": 865, "statues_count": 47311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-23T00:00:05.000Z"), "id": 734640023853764609, "text": "Happy birthday babbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbyyyyy @havasugirla !!!���� I love you & TU with you����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2571751267 }}, "user": { "id": 96695544, "name": "Saline Phorn", "screen_name": "salinephorn", "lang": "en", "location": "null", "create_at": date("2009-12-13"), "description": "null", "followers_count": 299, "friends_count": 240, "statues_count": 18439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-23T00:00:05.000Z"), "id": 734640024650665984, "text": "Hold the door #GameofThrones #TheDoor #Hodor", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GameofThrones", "TheDoor", "Hodor" }}, "user": { "id": 622695665, "name": "Cenneth", "screen_name": "kckvmedina", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2012-06-30"), "description": "You little stupid ass bitch, I ain't fuckin' with you", "followers_count": 483, "friends_count": 156, "statues_count": 39326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-23T00:00:05.000Z"), "id": 734640024902348801, "text": "Lowkey missing y'all too ... @MabelDuran12 @itsyagurlpripri https://t.co/d9stqE5kM2", "in_reply_to_status": 734568545518915585, "in_reply_to_user": 741590994, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 741590994, 4709038943 }}, "user": { "id": 842608868, "name": "Hey, Its Jenxx✨", "screen_name": "jenxx_", "lang": "en", "location": "null", "create_at": date("2012-09-23"), "description": "Positive minds | IG: _jenxx_ | TX-NC", "followers_count": 755, "friends_count": 444, "statues_count": 26545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002389715910656, "text": "@girlposts @JenicaPrescott this came up in my feed and I was going to retweet it to you and then I realized... Well... Yea.", "in_reply_to_status": 734431197942665217, "in_reply_to_user": 132774626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 132774626, 557069745 }}, "user": { "id": 37246395, "name": "sofie", "screen_name": "theycallmesmoky", "lang": "en", "location": "Honolulu, HI", "create_at": date("2009-05-02"), "description": "808. HI life. CrossFit Pearl City. Ocean. Vegetables. Meat. Pizza. Wine. Yoga. My sexy beard by my side.", "followers_count": 190, "friends_count": 371, "statues_count": 2120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002389850136577, "text": "My liver isn't complaining though so that's a plus ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571431726, "name": "Emma", "screen_name": "ohgoodlordemma", "lang": "en", "location": "null", "create_at": date("2012-05-04"), "description": "twenty two. colorado. valar morghulis. dca.", "followers_count": 627, "friends_count": 359, "statues_count": 12022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002389908881408, "text": "Have y'all Bruce? The Great White Shark I have to work with everyday https://t.co/KY7KWfAXq6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228886592, "name": "Casanova", "screen_name": "ArtJcar", "lang": "en", "location": "Ontario, CA", "create_at": date("2010-12-20"), "description": "Drillin cheeks in my hobby", "followers_count": 1824, "friends_count": 604, "statues_count": 149865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002389917229056, "text": "And Feel Good is perfect too https://t.co/ZL9CdMnZDN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2543566071, "name": "Kay ✨", "screen_name": "gaugedearsh4wty", "lang": "en", "location": "KCMO/TX", "create_at": date("2014-05-12"), "description": "I'm a 20 year old hip hop head who loves to draw. PV'18", "followers_count": 733, "friends_count": 386, "statues_count": 53991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002389942525952, "text": "I was talking about a napkin �� https://t.co/jZvWYhcLAq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 509506933, "name": "Eazy ✌️", "screen_name": "Iranthruher_", "lang": "en", "location": "Bread Winner", "create_at": date("2012-02-29"), "description": "Aries RIP-NANA ⬛️", "followers_count": 1067, "friends_count": 902, "statues_count": 52359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paterson, NJ", "id": "23a06e5cc6d83009", "name": "Paterson", "place_type": "city", "bounding_box": rectangle("-74.205822,40.888642 -74.129793,40.941452") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34031, "countyName": "Passaic", "cityID": 3457000, "cityName": "Paterson" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002390525419521, "text": "Happy Birthday baby @Whoisj_ I love you see you later ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 392172702 }}, "user": { "id": 156872935, "name": "soulsnatchaa", "screen_name": "Bomptonbeezy", "lang": "en", "location": "free yums ", "create_at": date("2010-06-17"), "description": "95'", "followers_count": 1544, "friends_count": 707, "statues_count": 90285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002391242760192, "text": "@RealMillzDavis The project was dope ��", "in_reply_to_status": -1, "in_reply_to_user": 319802126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 319802126 }}, "user": { "id": 974522712, "name": "L.O.W", "screen_name": "Josh_Byrd2", "lang": "en", "location": "Snapchat: Josh_byrdst2", "create_at": date("2012-11-27"), "description": "null", "followers_count": 184, "friends_count": 90, "statues_count": 2027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida City, FL", "id": "e8d156644aef005b", "name": "Florida City", "place_type": "city", "bounding_box": rectangle("-80.517383,25.403767 -80.460623,25.4625") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1222975, "cityName": "Florida City" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002391313928193, "text": "@yvnglauren @VivianTijerina lauren ��", "in_reply_to_status": 735002296623501312, "in_reply_to_user": 3623658620, "favorite_count": 0, "retweet_count": 0, "lang": "eu", "is_retweet": false, "user_mentions": {{ 3623658620, 748779998 }}, "user": { "id": 2312691044, "name": "ray✨", "screen_name": "_smhthatsoraven", "lang": "en", "location": "holdin it down for danny", "create_at": date("2014-01-26"), "description": "I have a mini me. literally.", "followers_count": 695, "friends_count": 826, "statues_count": 7087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Braunfels, TX", "id": "c807df9bb77ea0b3", "name": "New Braunfels", "place_type": "city", "bounding_box": rectangle("-98.214955,29.64544 -98.031544,29.765788") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48091, "countyName": "Comal", "cityID": 4850820, "cityName": "New Braunfels" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002391423111168, "text": "He just said she's trying to give the candy bar \"chunk\" a new name. Then talked about how rich she was.. Pick a side man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 454566019, "name": "Jared Chance", "screen_name": "jaredchance67", "lang": "en", "location": "null", "create_at": date("2012-01-03"), "description": "#BearDown #ManchesterUnited Drake football. Mustang, Oklahoma; Des Moines, Iowa; Alpha Phig Nu; EDC Vegas✈️", "followers_count": 891, "friends_count": 399, "statues_count": 13508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002391506911233, "text": "Took a nap earlier. Now I can't sleep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 307706746, "name": "jusain bolt", "screen_name": "Mannydeeznutz_", "lang": "en", "location": "null", "create_at": date("2011-05-29"), "description": "Jessica's booty is mine.❣", "followers_count": 973, "friends_count": 874, "statues_count": 21521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002391557242880, "text": "Not opening the gate for anybody though ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 330488565, "name": "just danielle ✨", "screen_name": "Danielle_eeee", "lang": "en", "location": "San Antonio ✈️ Denton", "create_at": date("2011-07-06"), "description": "top 5. top 5. top 5 ✨", "followers_count": 2459, "friends_count": 1484, "statues_count": 132272 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002392282992640, "text": "Temp 58.5° Hi/Lo 61.1/58.5 Rng 2.6° WC 58.5° Hmd 82% Rain 0.00\" Storm 0.00\" BAR 29.903 Falling DP 53.0° Wnd 1mph Dir NNW Gst 5mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 18941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002392471556096, "text": "Uptown Girls never gets old nope never", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173522073, "name": "Eunice Beck", "screen_name": "eunicebeck", "lang": "en", "location": "ASU", "create_at": date("2013-02-12"), "description": "is it christmas yet", "followers_count": 417, "friends_count": 230, "statues_count": 3524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002392668688385, "text": "\"Okay goodnight poops\" https://t.co/QS4WO38sDO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3161291718, "name": "Yung $hvwty", "screen_name": "MikylaRodriguez", "lang": "en", "location": "w/daddy ", "create_at": date("2015-04-17"), "description": "busy calling him daddy", "followers_count": 266, "friends_count": 239, "statues_count": 13014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002392819695616, "text": "In a bad mood now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1090572122, "name": "%", "screen_name": "panchoswag16", "lang": "en", "location": "Turnt Mafia ", "create_at": date("2013-01-14"), "description": "Panda Panda Panda Panda Panda", "followers_count": 432, "friends_count": 363, "statues_count": 41379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-05-24T00:00:00.000Z"), "id": 735002393008472065, "text": "@ebbtideapp Tide in Bay Waveland Yacht Club, Mississippi 05/24/2016\nHigh 1:40pm 2.1\n Low 12:03am 0.1\nHigh 2:14pm 2.1\n Low 12:53am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-89.325,30.325"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 94, "friends_count": 1, "statues_count": 35409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28045, "countyName": "Hancock", "cityID": 2803980, "cityName": "Bay St. Louis" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002393738248192, "text": "Awkwardly looking down as the walking order take dude walks by", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369572430, "name": "Mr. Fredricksen", "screen_name": "Chapatio13", "lang": "en", "location": "null", "create_at": date("2014-03-02"), "description": "Don't read this.", "followers_count": 276, "friends_count": 235, "statues_count": 17116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002393990057984, "text": "The #business that aims for #Oskar! The whole planet is #amazed! Go to: https://t.co/qi95ZbI1oQ https://t.co/IGXSVno0pU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "business", "Oskar", "amazed" }}, "user": { "id": 709303811496648705, "name": "Patricia Clit", "screen_name": "RomdClit", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2016-03-14"), "description": "Hollywood and money for you!!! The only business that promotes a topic of movie! Watch video: http://power01.fourfw.com/ & http://power01.cliffaz.com/", "followers_count": 1418, "friends_count": 3161, "statues_count": 896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394380013568, "text": "@brandonrpineda @CiaraMcLelland @haymaekay NUT", "in_reply_to_status": 735002163626151937, "in_reply_to_user": 1698005617, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1698005617, 177657654, 509269102 }}, "user": { "id": 3256352965, "name": "geek", "screen_name": "kylerhoot", "lang": "en", "location": "Grand Terrace, CA", "create_at": date("2015-06-25"), "description": "bar and grill @WeAreEllison", "followers_count": 611, "friends_count": 691, "statues_count": 15143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394426236930, "text": "05/24@03:00 - Temp 55.4F, WC 55.4F. Wind 0.8mph SW, Gust 4.0mph. Bar 29.918in, Falling slowly. Rain 0.00in. Hum 99%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394547781632, "text": "@rheyleonor_ hey, go catch some zzzzzzz's", "in_reply_to_status": 735002212082950144, "in_reply_to_user": 2366622692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2366622692 }}, "user": { "id": 2255325192, "name": "Daniel", "screen_name": "_Monsalvo722", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "I'm just trying to do my best in a world where it's far too easy to do your worst", "followers_count": 222, "friends_count": 161, "statues_count": 13395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394560319488, "text": "I'd rather play baseball than watch it on tv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1097456971, "name": "Kenny Carrillo", "screen_name": "OhhhKenny", "lang": "en", "location": "Southern California", "create_at": date("2013-01-16"), "description": "Montebello HS c/o 2016 ⚾️GBG Marucci / IPB Zinger Team⚾️", "followers_count": 438, "friends_count": 660, "statues_count": 8076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394615001088, "text": "Motion inna ocean baby ���� https://t.co/a8rWz3MK7f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user": { "id": 2856026443, "name": "✨GinaTheFlippa✨", "screen_name": "GinaTheSavage", "lang": "en", "location": "Walker Mill, MD , @SpKelll Bed", "create_at": date("2014-10-14"), "description": "RipDunk❤️RipReggie❤️RipUncleRonnie❤️ ginathesavage@gmail.com for features lightshow my luv ❤️ DONT FUCKING DM ME .", "followers_count": 6545, "friends_count": 3300, "statues_count": 153955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forestville, MD", "id": "0173c19a1ce41de0", "name": "Forestville", "place_type": "city", "bounding_box": rectangle("-76.902529,38.825445 -76.829437,38.880084") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2429000, "cityName": "Forestville" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002394665357313, "text": "Ripley SW Limestone Co. Temp: 59.7°F Wind:0.0mph Pressure: 995.9mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002395181125632, "text": "@rheyleonor_ i was looking for someone��", "in_reply_to_status": 735001902199406592, "in_reply_to_user": 2366622692, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2366622692 }}, "user": { "id": 865644444, "name": "sabrina", "screen_name": "sabrina07_", "lang": "en", "location": "null", "create_at": date("2012-10-06"), "description": "alemany '17 ⚾️", "followers_count": 579, "friends_count": 354, "statues_count": 4891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002395432779777, "text": "Kind of exaggerated but a toned down version of this would be cool “HYPER-REALITY” on https://t.co/pI6pM8tKyD #microsofthololens #hololens", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "microsofthololens", "hololens" }}, "user": { "id": 3801321, "name": " Tony Brizuela", "screen_name": "TonyBriz", "lang": "en", "location": "Sacramento, CA", "create_at": date("2007-04-08"), "description": "IT Analyst at Apple inc. & \nTech/Gadget Geek", "followers_count": 132, "friends_count": 433, "statues_count": 1385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002396397424640, "text": "It's my birthday ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3271532154, "name": ".", "screen_name": "mvryivh", "lang": "en", "location": "null", "create_at": date("2015-07-07"), "description": "null", "followers_count": 392, "friends_count": 144, "statues_count": 2554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002396405817344, "text": "@spacewillow @mercurialmonty grief over losing gina will always be secondary to the octavia's and clarke's pain EVEN THOUGH IT WAS AN ACTUAL", "in_reply_to_status": 735002187152007168, "in_reply_to_user": 3297087151, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3316082343, 4660959655 }}, "user": { "id": 3297087151, "name": "cam", "screen_name": "candid59", "lang": "en", "location": "True North", "create_at": date("2015-07-26"), "description": "warning: crescENDOING RANTS", "followers_count": 297, "friends_count": 131, "statues_count": 12375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002396414251008, "text": "@Zarudius Glad ya had fun!", "in_reply_to_status": 733542323334746112, "in_reply_to_user": 1319311501, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1319311501 }}, "user": { "id": 17221609, "name": "Michichael", "screen_name": "Michichael", "lang": "en", "location": "San Jose, CA", "create_at": date("2008-11-06"), "description": "Just another furry firearms enthusiast and IT consultant.", "followers_count": 43, "friends_count": 33, "statues_count": 357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002396758269952, "text": "Jus text this girl like 40 messages im def good at giving advice i been wise since a youngin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230842093, "name": "tonio ⚡️", "screen_name": "Infamous_TONE", "lang": "en", "location": "atlanna ", "create_at": date("2010-12-26"), "description": "bigger and better", "followers_count": 9857, "friends_count": 7640, "statues_count": 62947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hiram, GA", "id": "001ad7ad2ff9b1c0", "name": "Hiram", "place_type": "city", "bounding_box": rectangle("-84.797988,33.860284 -84.728812,33.913442") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13223, "countyName": "Paulding", "cityID": 1339076, "cityName": "Hiram" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002397139931136, "text": "Wind 0.0 mph ---. Barometer 30.006 in, Falling slowly. Temperature 52.9 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 20053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002397240500225, "text": "....and still no sound #TheBachelorette", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TheBachelorette" }}, "user": { "id": 3888177854, "name": "Naomi Pyle", "screen_name": "directsalemom", "lang": "en", "location": "Honolulu, HI", "create_at": date("2015-10-13"), "description": "Navy Wife Mommy Scentsy Consultant I get to work from home selling Scentsy! http://Facebook.com/directsalemom", "followers_count": 47, "friends_count": 113, "statues_count": 220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wahiawa, HI", "id": "f52453067076e5dc", "name": "Wahiawa", "place_type": "city", "bounding_box": rectangle("-158.051715,21.488622 -157.980627,21.512885") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1572650, "cityName": "Wahiawa" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002397374840832, "text": "@PresidentJRich I'm bored. Come cuddle", "in_reply_to_status": -1, "in_reply_to_user": 314085588, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 314085588 }}, "user": { "id": 274835508, "name": "Arica (AJ)", "screen_name": "lilAngel5310", "lang": "en", "location": "Cleveland, OH", "create_at": date("2011-03-30"), "description": "23+L.G.(B).T.Q+Concerts+ \\m/*.*\\m/|Chipotle+Merch girl for @DemonsWithin2+ Future Cop+#Cavs+Notre Dame College '16+Insta: @heartss2bee+ Photographer+P.L.U.R.", "followers_count": 1732, "friends_count": 1927, "statues_count": 8021 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Euclid, OH", "id": "19cdbb0c29dbc3a5", "name": "South Euclid", "place_type": "city", "bounding_box": rectangle("-81.54638,41.501345 -81.506679,41.54648") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3973264, "cityName": "South Euclid" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002397571833856, "text": "And maybe one day I won't but for how hard I've tried, I can't unlove you. ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 196853354, "name": "Elizabeth Flori", "screen_name": "uhlizahbethhh", "lang": "en", "location": "null", "create_at": date("2010-09-29"), "description": "ⓓⓞⓝⓣ ⓔⓥⓔⓡ ⓖⓘⓥⓔ ⓤⓟ †", "followers_count": 589, "friends_count": 430, "statues_count": 17387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-24T00:00:01.000Z"), "id": 735002397655728128, "text": "Moving out https://t.co/MfCVT7TW6X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386867538, "name": "ausin", "screen_name": "stackaxdolla", "lang": "en", "location": "Texas", "create_at": date("2011-10-07"), "description": "It's all ogre now. http://judiciarytx.bigcartel.com http://open.spotify.com/album/4JbItr7M http://itun.es/us/1O-P_", "followers_count": 456, "friends_count": 618, "statues_count": 10569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002397961949185, "text": "@earionnaaaaa @Keychainzzzz_ y'all thought ������", "in_reply_to_status": 735001135283470336, "in_reply_to_user": 423858367, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 423858367, 2739845120 }}, "user": { "id": 432153016, "name": "Julia Wilson", "screen_name": "J_Dubbbb96", "lang": "en", "location": "San Marcos, TX", "create_at": date("2011-12-08"), "description": "20 • Texas State University • PR Major • Matthews 5:16 • RIP Mommy 01/25/2016 • SC: Jdubbbb96", "followers_count": 2180, "friends_count": 1915, "statues_count": 65063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002397995503618, "text": "flashbacks to him as a baby.. gets a gn kiss from peter. peter goes to his room & his phone buzz; chris texting talm bout some \"u up?\". bro.", "in_reply_to_status": 735001410329157632, "in_reply_to_user": 3031529655, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031529655, "name": "BEEZ", "screen_name": "bugattibeez", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-02-11"), "description": "ig/snapchat: bugattibeez | music: http://soundcloud.com/bugattibeez | business: bugattibeez@gmail.com", "followers_count": 40556, "friends_count": 830, "statues_count": 8768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002397999697920, "text": "#nocaption @ Freemont St Expeirence https://t.co/6c9HEKYWnn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.1442997,36.1707446"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nocaption" }}, "user": { "id": 81967701, "name": "Jacque Ince", "screen_name": "KJ4L", "lang": "en", "location": "Blaine, MN", "create_at": date("2009-10-12"), "description": "null", "followers_count": 149, "friends_count": 372, "statues_count": 740 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002398268084226, "text": "white ppl be mad when you call them out on cultural appropriation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 239650277, "name": "Monique the Geek", "screen_name": "CoCoValentine", "lang": "en", "location": "California Dreamin'", "create_at": date("2011-01-17"), "description": "Melanin is my Superpower; Faves❤️: Tika Sumpter, TPH, ANR, EC & some more folk #HAHN #SUPERFAN A PROUD member of The Blacks", "followers_count": 1272, "friends_count": 647, "statues_count": 71234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399023235073, "text": "Wind 0.0 mph ESE. Barometer 29.942 in, Rising. Temperature 66.3 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399228600320, "text": "@sandraclarissa_ like what about the 3rd?? Lmao", "in_reply_to_status": 735002323429134336, "in_reply_to_user": 2988313364, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2988313364 }}, "user": { "id": 2988313364, "name": "Sandy", "screen_name": "sandraclarissa_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-01-18"), "description": "I see leaves on the ground, but I ain't gon' rake em", "followers_count": 302, "friends_count": 470, "statues_count": 12679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399266377728, "text": "@Vegceduna @JohnRaphael haha. Yay! Were you practicing our dance moves? :p", "in_reply_to_status": 735001799531200512, "in_reply_to_user": 455581420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 455581420, 16875498 }}, "user": { "id": 23374944, "name": "James Stapleton", "screen_name": "JamesStapes", "lang": "en", "location": "WeHo - Beverly Hills", "create_at": date("2009-03-08"), "description": "Like any good Superhero, I lead a double life. I'm a Creative Director and a YouTuber. IG:mrjamesstapes", "followers_count": 1459, "friends_count": 712, "statues_count": 17081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399522295808, "text": "SUMMER 16 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 433893345, "name": "Queen B ✨", "screen_name": "_BriiannaFxo", "lang": "en", "location": "RU '18", "create_at": date("2011-12-10"), "description": "Savage but Queen ⁶", "followers_count": 617, "friends_count": 433, "statues_count": 14633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South River, NJ", "id": "44a9bb088515255e", "name": "South River", "place_type": "city", "bounding_box": rectangle("-74.40347,40.428934 -74.354332,40.463877") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3469420, "cityName": "South River" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399639805952, "text": "Temp: 62.3°F Wind:0.0mph Pressure: 30.017hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002399723638784, "text": "@desmiller42 @SalSpearsz my favorite tea TBH", "in_reply_to_status": 734954156872466433, "in_reply_to_user": 3240361591, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3240361591, 1023563713 }}, "user": { "id": 321715703, "name": "brendan spears ♊️", "screen_name": "QueenieDazen", "lang": "en", "location": "cinderella's castle ", "create_at": date("2011-06-21"), "description": "I'm just a lost boy and I'm ready to be found✨ Britney Spears is my mom. so that's cool. sometimes being a Gemini gets me in trouble. @britneyspears follows me", "followers_count": 1114, "friends_count": 795, "statues_count": 27804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McMurray, PA", "id": "013996094b0e6ef7", "name": "McMurray", "place_type": "city", "bounding_box": rectangle("-80.110693,40.23612 -80.013665,40.305872") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4246344, "cityName": "McMurray" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002401271250944, "text": "Wind 0.0 mph ---. Barometer 29.948 in, Steady. Temperature 60.5 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002401355096064, "text": "��\n\nwe been knowing each other since woodmere days.. cool and handsome guy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2338488377, "name": "D☺", "screen_name": "LynnrobertsL", "lang": "en", "location": "somewhere w/ handsome. ❤", "create_at": date("2014-02-11"), "description": "Family❤ All American high jumper!! 3time state champ! 4th in nationals 100m hurdler! BLESSED BEYOND BELIEFS ✨!!", "followers_count": 1935, "friends_count": 1640, "statues_count": 72692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002401833422848, "text": "You would think it's all mine, the way I took it.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2312172604, "name": "½ K™", "screen_name": "fatb0ymike", "lang": "en", "location": "CLICK BELOW", "create_at": date("2014-01-29"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀South Memphis. ⠀⠀⠀⠀⠀⠀⠀ ⠀⠀⠀⠀⠀⠀⠀⠀⠀ ⠀⠀Instagram | @fatb0ymike ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀Alcorn State University", "followers_count": 773, "friends_count": 452, "statues_count": 6625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002401879396352, "text": "Okay sounds like a good trade https://t.co/EClnKJzolD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 157533334, "name": "Ombriél", "screen_name": "EarlsBottomLip", "lang": "en", "location": "crack (little) rock ", "create_at": date("2010-06-19"), "description": "no we can't relate.", "followers_count": 8214, "friends_count": 979, "statues_count": 197646 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bryant, AR", "id": "40448ceb3ae11576", "name": "Bryant", "place_type": "city", "bounding_box": rectangle("-92.536648,34.580894 -92.445728,34.655127") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline", "cityID": 509460, "cityName": "Bryant" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002401917173760, "text": "Been on hella Toronto music lately ���� ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 156337115, "name": "Sway Iguodala.", "screen_name": "SwaySoSteezy", "lang": "en", "location": "Northside Salinas to Sac.", "create_at": date("2010-06-16"), "description": "Sac State. RaidersWarriorsGiants. #StraightCash #RIPLilKris", "followers_count": 477, "friends_count": 590, "statues_count": 56720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-05-24T00:00:02.000Z"), "id": 735002402047152129, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":52.5°F Wind:0.0mph Pressure: 30.03hpa Falling slowly Rain Today 0.00in. Forecast: Showery, becoming", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 18, "statues_count": 319250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002402470780928, "text": "Cuz you from another hood don't mean you a Opp I got niggas in my hood thats opps too and I ain't playing wit ya Ima let ya have it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 553207974, "name": "MocityVic", "screen_name": "selfmadeVIC", "lang": "en", "location": "Houston Tx", "create_at": date("2012-04-13"), "description": "That Mo City Nigga.Follow me on instagram @selfmadevic Born `94 #MOCITYMONEYTEAM #DexGang #LongliveKaybee #HustlaVision2⃣8⃣#TTM #Zulu", "followers_count": 1925, "friends_count": 1734, "statues_count": 43671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002402785394688, "text": "Back home ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 704084779, "name": "Adrian Sahagun", "screen_name": "adrian_sahagun", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "YOU DON'T WANT NO PROBLEM WITH ME", "followers_count": 281, "friends_count": 372, "statues_count": 9685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Soledad, CA", "id": "642fbca6eea15240", "name": "Soledad", "place_type": "city", "bounding_box": rectangle("-121.338707,36.410518 -121.306524,36.445593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 672520, "cityName": "Soledad" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002402932297729, "text": "Wind 0.0 mph NNW. Barometer 29.902 in, Steady. Temperature 44.4 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002402957492224, "text": "@saramrutledge what ab your withdraws of texting me?? Unblock me fool! ��", "in_reply_to_status": 735000845150867456, "in_reply_to_user": 121585864, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 121585864 }}, "user": { "id": 2585072438, "name": "Andres™", "screen_name": "cashaguilar", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "still here", "followers_count": 1060, "friends_count": 483, "statues_count": 39963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002403003502592, "text": "Ugh, same https://t.co/hFefhYwFcZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 620318336, "name": "alyssa garza", "screen_name": "No_Sympathy_Cx", "lang": "en", "location": "Fresno,CA", "create_at": date("2012-06-27"), "description": "22. Music. Poetry. Journalist--Behind The Music.", "followers_count": 276, "friends_count": 636, "statues_count": 2939 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002403121025024, "text": "@KingBee_Crownme NDC", "in_reply_to_status": 734942781886472192, "in_reply_to_user": 2212385032, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2212385032 }}, "user": { "id": 35476735, "name": "rareBREED", "screen_name": "1stladycherish", "lang": "en", "location": "IG|1stladycherish⚜", "create_at": date("2009-04-26"), "description": "I Smile At The Haters & Wave At My Fans I Am A E͜͡N͜͡T͜͡E͜͡R͜͡T͜͡A͜͡I͜͡N͜͡E͜͡R͜͡|D͜͡A͜͡N͜͡C͜͡E͜͡R͜͡ иσℓα σνσ ¢т¢ ʀɪᴘ ᴘᴏᴘᴘᴀ #G͎I͎R͎L͎S͎L͎I͎K͎E͎U͎S͎ BRAVE❤️♎️", "followers_count": 2424, "friends_count": 1063, "statues_count": 65464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002403880067072, "text": "12 AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2749490759, "name": "Lex", "screen_name": "byron_lexi", "lang": "en", "location": "Snohomish,WA", "create_at": date("2014-08-21"), "description": "SHS|| sc//lexibyron16", "followers_count": 288, "friends_count": 194, "statues_count": 4443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Machias, WA", "id": "f33cf471e6cd86b0", "name": "Machias", "place_type": "city", "bounding_box": rectangle("-122.065812,47.971704 -122.028597,48.012767") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5341470, "cityName": "Machias" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002404073050113, "text": "\"She Suck me like Dracula \" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2876986939, "name": "Rg✨", "screen_name": "GuevaraRoxanne", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-10-25"), "description": "dis drama way2much u need to CUT IT", "followers_count": 82, "friends_count": 124, "statues_count": 2964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002404391772160, "text": "@Amy_Lopez0408 k cool", "in_reply_to_status": 735002167698821121, "in_reply_to_user": 4176845059, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4176845059 }}, "user": { "id": 605756968, "name": "J O S E", "screen_name": "JoseSolorzano97", "lang": "en", "location": "Laredo, TX", "create_at": date("2012-06-11"), "description": "null", "followers_count": 270, "friends_count": 250, "statues_count": 11776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002405444583425, "text": "Wind 8.0 mph SSE. Barometer 29.873 in, Steady. Temperature 73.8 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-24T00:00:03.000Z"), "id": 735002405742338049, "text": "✨�� can I just be you ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2282801071, "name": "Hot Mama Dahana", "screen_name": "DahanaBanana", "lang": "en", "location": "6ix ", "create_at": date("2014-01-08"), "description": "THS Senior", "followers_count": 804, "friends_count": 683, "statues_count": 25612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002406405165056, "text": "Temp 44.9°F Wind Chill 44.9°F RH 94% Wind 0.0 --- Gust 0.0 --- SLP 30.089 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002406883328000, "text": "@BreeMonshanique ���� mhm but good luck puss", "in_reply_to_status": 735002298481541120, "in_reply_to_user": 174895038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 174895038 }}, "user": { "id": 2232428233, "name": "Chapo", "screen_name": "ChapoSlim_", "lang": "en", "location": "Rico ", "create_at": date("2013-12-05"), "description": "Bronx | Greensboro | Dog Pound | 6'2", "followers_count": 976, "friends_count": 786, "statues_count": 24209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407143243777, "text": "@Moliver_11 yo ass can't even text back ��", "in_reply_to_status": 735002270409056256, "in_reply_to_user": 605710236, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 605710236 }}, "user": { "id": 622194107, "name": "❤️", "screen_name": "T__thanggg", "lang": "en", "location": "Detroit, MI ✈️ Ohio", "create_at": date("2012-06-29"), "description": "Stack, Slay, Pray #CSUCheerleader", "followers_count": 586, "friends_count": 531, "statues_count": 24659 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407164207105, "text": "may the lord have mercy on my soul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456268113, "name": "Riley", "screen_name": "fosterxkid", "lang": "en", "location": "Hays, KS", "create_at": date("2012-01-05"), "description": "Prod. | rfoster97@gmail.com", "followers_count": 1720, "friends_count": 1394, "statues_count": 33741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hays, KS", "id": "b0e50abc7e3f7363", "name": "Hays", "place_type": "city", "bounding_box": rectangle("-99.354684,38.849348 -99.280695,38.914552") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20051, "countyName": "Ellis", "cityID": 2031100, "cityName": "Hays" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407193575424, "text": "I can't do this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338216514, "name": "Queso Blanca", "screen_name": "bitchimtheBRITT", "lang": "en", "location": "null", "create_at": date("2011-07-19"), "description": "just trying to find my place in this fucked up world. Snapchat - briittneysmith", "followers_count": 809, "friends_count": 1886, "statues_count": 74938 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407201968128, "text": "�������� https://t.co/Qow6M2TASg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 35914937, "name": "thatskennedy", "screen_name": "DamnSheReal", "lang": "en", "location": "null", "create_at": date("2009-04-27"), "description": "I slay | Htx | 19 |", "followers_count": 1017, "friends_count": 542, "statues_count": 79861 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407747256320, "text": "I think that's enough retweets ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1551753210, "name": "perd hapley", "screen_name": "Andayeeee", "lang": "en", "location": "Globo Gym", "create_at": date("2013-06-27"), "description": "I'm not straight edge, I'd just rather be sober. @yoga__jones", "followers_count": 409, "friends_count": 347, "statues_count": 14533 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407826948096, "text": "You announcing drake's new album? https://t.co/nH2EEEBvpk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50938868, "name": "Estevan Lucas", "screen_name": "estevanlucvs", "lang": "en", "location": "Santa Barbara, California ", "create_at": date("2009-06-25"), "description": "Snapchat: est.lvcs", "followers_count": 312, "friends_count": 309, "statues_count": 7064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Barbara, CA", "id": "f6ebc676e5cde864", "name": "Santa Barbara", "place_type": "city", "bounding_box": rectangle("-119.763341,34.395525 -119.639931,34.464245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669070, "cityName": "Santa Barbara" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407831142402, "text": "Controlla ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 776331830, "name": "polypriincess✨", "screen_name": "hyphyhoochie_", "lang": "en", "location": "goin dumb in the bay w Klay", "create_at": date("2012-08-23"), "description": "life's too short for the bullshit", "followers_count": 518, "friends_count": 499, "statues_count": 24019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002407973720064, "text": "@preeyaxsaur at @_jazleen place doing homework. Time flew bye and I don't go home sorry��������", "in_reply_to_status": 735000069787656193, "in_reply_to_user": 69106531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 69106531, 252338019 }}, "user": { "id": 237080666, "name": "Yesenia", "screen_name": "Jusenya", "lang": "en", "location": "Hayward, CA", "create_at": date("2011-01-11"), "description": "null", "followers_count": 105, "friends_count": 348, "statues_count": 677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002408082776065, "text": "El amor no necesita ser perfecto, basta con que sea verdadero ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 180689490, "name": "marilinita ⚓", "screen_name": "marilinita_29", "lang": "es", "location": "Pochita Californiana ", "create_at": date("2010-08-19"), "description": "Tengo personalidades múltiples... O al menos eso dicen cada una de ellas. JURO DEJAR TWITTER CUANDO TENGA NOVIO!!! (not)\nYo aquí NO soy yo.", "followers_count": 738, "friends_count": 357, "statues_count": 7677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002408514904064, "text": "Nah https://t.co/HUDVmLphVV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2572333646, "name": "⚜⚜", "screen_name": "emotiveshawty", "lang": "en", "location": "Compton, CA", "create_at": date("2014-06-16"), "description": "if you fucking with me, you winning. blaxican. #LongLiveGio", "followers_count": 451, "friends_count": 84, "statues_count": 54497 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002409122992128, "text": "Such a sick night. Got some tricks I never thought I'd land thanks to the homies hype. Can't wait for daytime filming to hit bigger shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3050385266, "name": "scarth vader", "screen_name": "EchosInMyMind", "lang": "en", "location": "null", "create_at": date("2015-02-28"), "description": ", vegan, skateboarder, guitarist for push mongo, faggot.", "followers_count": 79, "friends_count": 117, "statues_count": 1338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002409513062400, "text": "Mrs Serv my American Dream is to not have to do this project", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 381894760, "name": "ⓒⓞⓡⓝ", "screen_name": "corinnevlay", "lang": "en", "location": "null", "create_at": date("2011-09-28"), "description": "dancin' is what to do", "followers_count": 724, "friends_count": 852, "statues_count": 4923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002409596903425, "text": "I'll keep him for a couple days so you can do it �� https://t.co/tw8BnDZCcY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 77663208, "name": "Uilani Morita", "screen_name": "Uilanibaby", "lang": "en", "location": "null", "create_at": date("2009-09-26"), "description": "IG @kakrx2_mommy ⚓️ ✨Try not to get offended by my tweets.✨", "followers_count": 201, "friends_count": 130, "statues_count": 4174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mililani Mauka, HI", "id": "0105d2ab1843800d", "name": "Mililani Mauka", "place_type": "city", "bounding_box": rectangle("-158.024092,21.458474 -157.975278,21.489391") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1551000, "cityName": "Mililani Mauka" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002409756303361, "text": "@Liddo_Neshhh ��", "in_reply_to_status": 735002211046981632, "in_reply_to_user": 571098100, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 571098100 }}, "user": { "id": 61971193, "name": "Ace Boogie", "screen_name": "_dollfvce", "lang": "en", "location": "dtx", "create_at": date("2009-07-31"), "description": "it's not that deep bro , shutup u hoes fans . . pretty bitches only could get in my possé . 19", "followers_count": 1384, "friends_count": 440, "statues_count": 96547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002409999597568, "text": "@iiukrii enlighten*", "in_reply_to_status": 735002347814817792, "in_reply_to_user": 2151989064, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user_mentions": {{ 2151989064 }}, "user": { "id": 205877691, "name": "Shadow", "screen_name": "Shadow73_", "lang": "en", "location": "New Jersey, USA", "create_at": date("2010-10-21"), "description": "Youtuber • @Giants fan • Founder of @LegitCoinGames and @WeFindGiveaways • http://twitch.tv/Shadow_73", "followers_count": 8582, "friends_count": 521, "statues_count": 63257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002410179907585, "text": "Hi hello yes I am wide awake plz stop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 100021400, "name": "Ana✨", "screen_name": "AnaKnotek", "lang": "en", "location": "Neverland", "create_at": date("2009-12-28"), "description": "pretty rad, sometimes sad", "followers_count": 201, "friends_count": 219, "statues_count": 3655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tanglewilde, WA", "id": "011bc5e47625b12a", "name": "Tanglewilde", "place_type": "city", "bounding_box": rectangle("-122.797096,47.040231 -122.764565,47.062657") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53067, "countyName": "Thurston", "cityID": 5370280, "cityName": "Tanglewilde" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002410423324672, "text": "Now I'm sitting here just up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415920039, "name": "Hoodrich Pablo", "screen_name": "OlDirtyPablo", "lang": "en", "location": "Warlando", "create_at": date("2011-11-18"), "description": "niggas don't get a follow back. #savageszn #shuttersquad", "followers_count": 2255, "friends_count": 807, "statues_count": 325950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-24T00:00:04.000Z"), "id": 735002410431614976, "text": "Happy birthday bae see you July 3rd �� https://t.co/eg8GOpv4h2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 169228428, "name": "kara matsune", "screen_name": "karamatsune", "lang": "en", "location": "null", "create_at": date("2010-07-21"), "description": "null", "followers_count": 441, "friends_count": 483, "statues_count": 7606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002410821656576, "text": "@carterreynolds Happy 20th birthday Carter������ hope u have a great day. I'm a walking L with u. #HappyBirthdayCarter https://t.co/fEquUHSIcg", "in_reply_to_status": -1, "in_reply_to_user": 363961928, "favorite_count": 0, "coordinate": point("-118.327077,34.102669"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyBirthdayCarter" }}, "user_mentions": {{ 363961928 }}, "user": { "id": 2896864470, "name": "kimberly", "screen_name": "_Arabella_1996", "lang": "en", "location": "null", "create_at": date("2014-11-11"), "description": "...Misserfolge sind oft notwendige unwege zum erfolg.~ Erik Durm", "followers_count": 1298, "friends_count": 2231, "statues_count": 3808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avalon Hollywood", "id": "07d9ec7d5d087002", "name": "Avalon Hollywood", "place_type": "poi", "bounding_box": rectangle("-118.3270771,34.1026689 -118.327077,34.102669") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002410825961472, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Steady. Temperature 63.7 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002411647959042, "text": "I can never sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 637038379, "name": "Anayah ✨", "screen_name": "ognayah_", "lang": "en", "location": "null", "create_at": date("2012-07-16"), "description": "null", "followers_count": 1343, "friends_count": 789, "statues_count": 84648 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002411924787200, "text": "Can someone back me up when I say: it's hard to trust someone after you've been hurt by them? I'm pretty sure it's not just me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4444559837, "name": "Breanna Fisher", "screen_name": "chipotle_e", "lang": "en", "location": "Washington, USA", "create_at": date("2015-12-10"), "description": "I am more than what you see\n \n ❤sc/ig: @chipotle.e❤", "followers_count": 124, "friends_count": 242, "statues_count": 730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sumner, WA", "id": "00b71a3d07e66ff3", "name": "Sumner", "place_type": "city", "bounding_box": rectangle("-122.25912,47.18461 -122.201829,47.257417") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5368435, "cityName": "Sumner" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002412205801473, "text": "@dreaoct BISSSSHHH ITS YOUR FUCKING BDAY", "in_reply_to_status": -1, "in_reply_to_user": 1463115332, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1463115332 }}, "user": { "id": 383978173, "name": "g r a c e", "screen_name": "Gracielaa_21", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-10-02"), "description": ":):", "followers_count": 428, "friends_count": 334, "statues_count": 4745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002412591644673, "text": "@Zegoviano u did break up that nug! Lol it was urs ��", "in_reply_to_status": 735002278898204672, "in_reply_to_user": 232506576, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 232506576 }}, "user": { "id": 1290520807, "name": "THESHOOTER", "screen_name": "youngdanny3dp", "lang": "en", "location": "null", "create_at": date("2013-03-22"), "description": "smoke a lil bud", "followers_count": 459, "friends_count": 276, "statues_count": 29992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002412654690304, "text": "I really need to find someone to make me a header", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19520672, "name": "John", "screen_name": "Krone315", "lang": "en", "location": "New York", "create_at": date("2009-01-25"), "description": "KMN 12.31.12 | Sports Enthusiast | Competitive Gamer | Animal Lover | Part-Time Comedian | R.I.P Spencer | Player for", "followers_count": 433, "friends_count": 157, "statues_count": 11636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Syracuse, NY", "id": "138181506c13418e", "name": "Syracuse", "place_type": "city", "bounding_box": rectangle("-76.204476,42.984026 -76.053145,43.121188") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36067, "countyName": "Onondaga", "cityID": 3673000, "cityName": "Syracuse" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002412809752577, "text": "Bruh I can't believe I went to TJ the weekend before finals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2692036074, "name": "Rafa R Camacho", "screen_name": "Giraaffaa11", "lang": "en", "location": "California, USA", "create_at": date("2014-07-30"), "description": "null", "followers_count": 221, "friends_count": 187, "statues_count": 10992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002412910444544, "text": "1.5 magnitude #earthquake. 64km SSW of Cantwell, Alaska https://t.co/fkNvbj301y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-149.53,62.876"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake" }}, "user": { "id": 362523555, "name": "Earthquake Alerts", "screen_name": "QuakesToday", "lang": "en", "location": "null", "create_at": date("2011-08-26"), "description": "Worldwide earthquake alerts based on USGS data. 1.5 magnitude and higher.", "followers_count": 74042, "friends_count": 10, "statues_count": 104547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002413199986688, "text": "“Hum, hon, nouvelle cuisine, le Champs-Élysées, Maurice Chevalier!”", "in_reply_to_status": 735001026969882624, "in_reply_to_user": 14480251, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 14480251, "name": "Daniel Shannon", "screen_name": "phyllisstein", "lang": "en", "location": "Park Slope", "create_at": date("2008-04-22"), "description": "“trendsetter, tastemaker, twunkmaster” —@sara_ann_marie", "followers_count": 47805, "friends_count": 491, "statues_count": 28899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Park Slope, Brooklyn", "id": "545348d3219c8ef7", "name": "Park Slope", "place_type": "neighborhood", "bounding_box": rectangle("-73.991794,40.650661 -73.961914,40.677441") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002413346652160, "text": "exactly :) https://t.co/yQMYLNt2SC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131758677, "name": "Proud 2 b Mahesh Fan", "screen_name": "naveenkumar_345", "lang": "en", "location": "California, USA", "create_at": date("2010-04-10"), "description": "Diehard Fan of super star prince @urstrulymahesh ........and supporter of @ncbn & @narendramodi... Views expressed here are personal....", "followers_count": 2121, "friends_count": 587, "statues_count": 30667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-05-24T00:00:05.000Z"), "id": 735002413397053440, "text": "@SportsCenter @Drake @espn where was he when the cavs was up 2-0", "in_reply_to_status": 734958036498980868, "in_reply_to_user": 26257166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 26257166, 27195114, 2557521 }}, "user": { "id": 527373575, "name": "james witherspoon", "screen_name": "rockw61", "lang": "en", "location": "birmingham,alabama", "create_at": date("2012-03-17"), "description": "no longer coal miner,now a ductile iron plant worker", "followers_count": 1256, "friends_count": 1954, "statues_count": 10288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bessemer, AL", "id": "701a80c8d0e9a562", "name": "Bessemer", "place_type": "city", "bounding_box": rectangle("-87.031545,33.295574 -86.897433,33.456651") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 105980, "cityName": "Bessemer" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002414885916672, "text": "I'd definitely rather be in Wyoming than in Holbrook. There is never enough time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40217425, "name": "Gyps✌", "screen_name": "gypsirhiannon", "lang": "en", "location": "Arizona, USA", "create_at": date("2009-05-15"), "description": "✨✨", "followers_count": 304, "friends_count": 305, "statues_count": 10220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holbrook, AZ", "id": "003ffdae48298eb4", "name": "Holbrook", "place_type": "city", "bounding_box": rectangle("-110.199262,34.890065 -110.130192,34.939178") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4017, "countyName": "Navajo", "cityID": 433280, "cityName": "Holbrook" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002414911229952, "text": "#Goodnight ���� #IntoYou @ArianaGrande you've done it again!\n#NewYorkCity #DangerousWoman �� https://t.co/mfmXjJzLAw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Goodnight", "IntoYou", "NewYorkCity", "DangerousWoman" }}, "user_mentions": {{ 34507480 }}, "user": { "id": 277362711, "name": "Andrew Michaelsen", "screen_name": "manhattanvirgo", "lang": "en", "location": "Manhattan, NY", "create_at": date("2011-04-04"), "description": "The DOO of @Bout_App #BoutBeYou Using what time I have to try & make this world a better place RTs are not an endorsement #ThoughtsAreMyOwn", "followers_count": 1562, "friends_count": 1195, "statues_count": 10968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002415141814272, "text": "@MAlSulaiman_ بتوقيتكم ٩ بليل انا اطير اوصل قطر بعد ١٥ ساعه حذي", "in_reply_to_status": 735002179359133696, "in_reply_to_user": 2667644672, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 2667644672 }}, "user": { "id": 409307912, "name": "RashedOsama", "screen_name": "ROJ95", "lang": "en", "location": "Washington, DC", "create_at": date("2011-11-10"), "description": "One Love , One Dream , One Life , One Arsenal ❤️.", "followers_count": 1101, "friends_count": 361, "statues_count": 37667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairborn, OH", "id": "ab1cb346534d99ea", "name": "Fairborn", "place_type": "city", "bounding_box": rectangle("-84.079279,39.765691 -83.965647,39.848142") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39057, "countyName": "Greene", "cityID": 3925914, "cityName": "Fairborn" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002415158681600, "text": "Wind 0.0 mph S. Barometer 29.92 in, Steady. Temperature 60.1 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 122, "statues_count": 160017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002415183761408, "text": "I'm pretty sure I'll be back up in a couple hrs to pee cuz I just drunk too cups of apple juice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 220444901, "name": "Beezy", "screen_name": "Kisses_NGiggles", "lang": "en", "location": "united states", "create_at": date("2010-11-27"), "description": "null", "followers_count": 425, "friends_count": 632, "statues_count": 21575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-24T00:00:06.000Z"), "id": 735002415364071424, "text": "@_stefaaanii @Alessandro_7 @fanny_yo Fuck... We have to go now bro... We'll just ummmm, fall asleep Cus work got us tired.����", "in_reply_to_status": 735002230688862208, "in_reply_to_user": 1025352780, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1025352780, 937738141, 718774237 }}, "user": { "id": 473646929, "name": "Brandon Agüero R.", "screen_name": "15BrandonR", "lang": "en", "location": "└A ", "create_at": date("2012-01-24"), "description": "Barcelona❤️ | Sueña sin limites.", "followers_count": 471, "friends_count": 565, "statues_count": 24167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore, CA", "id": "a498a7384476f315", "name": "Lemoore", "place_type": "city", "bounding_box": rectangle("-119.812801,36.269156 -119.715005,36.328042") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641152, "cityName": "Lemoore" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364777799733248, "text": "I just have this gut feeling. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261418611, "name": "La Reina", "screen_name": "KristineK97", "lang": "en", "location": "null", "create_at": date("2011-03-05"), "description": "V XVII MMXVI", "followers_count": 135, "friends_count": 266, "statues_count": 5894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364778122694656, "text": "That is very healthy in order to move on and start fresh ���������������� https://t.co/v4hLZdJJls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 248135355, "name": "Last Pirate in LA", "screen_name": "BONNIELYNN2015", "lang": "en", "location": "#DTLA ", "create_at": date("2011-02-06"), "description": "RN/BSN/CLE :Masters in psychology #DTLA, Actress, Author , stand up artist , photographer, film artist, creative film producer, Vet, USA ARMY,", "followers_count": 223778, "friends_count": 2470, "statues_count": 132358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364778399531008, "text": "@BHust28 I told you at the beginning of the series to watch out for OKC!", "in_reply_to_status": 735364601978687488, "in_reply_to_user": 171643824, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 171643824 }}, "user": { "id": 63929704, "name": "bj black", "screen_name": "billieb22", "lang": "en", "location": "Vacaville, ca", "create_at": date("2009-08-08"), "description": "null", "followers_count": 132, "friends_count": 325, "statues_count": 7928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364778978332673, "text": "So stressed to the point I just want to throw up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872764004, "name": "210", "screen_name": "christi_unique", "lang": "en", "location": "210", "create_at": date("2012-10-10"), "description": "❤duecedimecity❤", "followers_count": 1309, "friends_count": 1898, "statues_count": 27890 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Converse, TX", "id": "dcd50868e563bbed", "name": "Converse", "place_type": "city", "bounding_box": rectangle("-98.352959,29.489943 -98.276076,29.538642") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4816468, "cityName": "Converse" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364779305521152, "text": "sorry, I ain't sorry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263439930, "name": "YFN Kenn", "screen_name": "kenneic", "lang": "en", "location": "Dallas, TX ", "create_at": date("2011-03-09"), "description": "Darrius❤️| SC: kbabez11 | #tjc17", "followers_count": 3147, "friends_count": 1691, "statues_count": 44711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364779343249408, "text": "jamarris https://t.co/LT54yrdOlq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 3049879557, "name": "jazb❤️", "screen_name": "jmyeshia", "lang": "en", "location": "null", "create_at": date("2015-02-21"), "description": "null", "followers_count": 973, "friends_count": 198, "statues_count": 56872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgeland, MS", "id": "78439f3b905720e3", "name": "Ridgeland", "place_type": "city", "bounding_box": rectangle("-90.181179,32.399351 -90.067498,32.459434") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28089, "countyName": "Madison", "cityID": 2862520, "cityName": "Ridgeland" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364780043702272, "text": "... https://t.co/yg8FjIukfT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 524913576, "name": "royalty", "screen_name": "_KingJarrod_", "lang": "en", "location": "El Dorado Hills, CA", "create_at": date("2012-03-14"), "description": "csus.", "followers_count": 1303, "friends_count": 871, "statues_count": 15692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elk Grove, CA", "id": "6daef4115502c39e", "name": "Elk Grove", "place_type": "city", "bounding_box": rectangle("-121.487576,38.361325 -121.276607,38.481744") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 622020, "cityName": "Elk Grove" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364780148588544, "text": "(https://t.co/AeWedb7SZx) <>-------<<<<\nThat's one helluva #Resume, #Mississippi.\n(#DNC) (#GOP) #Chicago #Memphis #Tupelo\nOn the hunt, Men!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Resume", "Mississippi", "DNC", "GOP", "Chicago", "Memphis", "Tupelo" }}, "user": { "id": 1713823531, "name": "Troy David Beadles", "screen_name": "tdbeadles", "lang": "en", "location": "troydavidbeadles@mail.com", "create_at": date("2013-08-30"), "description": "(★) Holy American Dad, Mississippi Veteran, Social Network Pioneer Since '95, Life Coach, Educator, Philanthropist, Humanitarian & Independent Composer.", "followers_count": 2995, "friends_count": 4980, "statues_count": 271474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tupelo, MS", "id": "894f2ba528ba1c7d", "name": "Tupelo", "place_type": "city", "bounding_box": rectangle("-88.800213,34.203564 -88.656629,34.324583") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2874840, "cityName": "Tupelo" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364780165337088, "text": "Pinche pobresa ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2451901945, "name": "aleks", "screen_name": "Helloim_aleks", "lang": "en", "location": "null", "create_at": date("2014-04-18"), "description": "null", "followers_count": 197, "friends_count": 211, "statues_count": 3280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364780706430976, "text": "Yassssssss https://t.co/nQLugCHfsc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3234388922, "name": "magz", "screen_name": "maggymerms", "lang": "en", "location": "California ✈️ ", "create_at": date("2015-06-02"), "description": "MUA | Music | Art | God | IG:Maggymerms", "followers_count": 3458, "friends_count": 478, "statues_count": 22382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364781029392385, "text": "Stop sleepin on @tryangled ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3268396746 }}, "user": { "id": 3286950493, "name": "ℓuminance.", "screen_name": "Lumi_nance", "lang": "en", "location": "Dreamland, Arizona", "create_at": date("2015-07-21"), "description": "NUMRLS // Sentient Society // TrapStyle. https://soundcloud.com/lumi-nance", "followers_count": 651, "friends_count": 732, "statues_count": 797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364781100666880, "text": "We are looking at about 8 MCU series on Netflix by 2018", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155910701, "name": "Chris R", "screen_name": "PsychicStoner", "lang": "en", "location": "Gulf Breeze, FL, USA", "create_at": date("2010-06-15"), "description": "Filthy, unfiltered thoughts, dirty jokes, deviant, metal, Nihilist, free thinker, Marvel and Anarchy. Follow at your own risk. No minors admitted", "followers_count": 3216, "friends_count": 3078, "statues_count": 238453 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midway, FL", "id": "00030a773a4ffe7a", "name": "Midway", "place_type": "city", "bounding_box": rectangle("-87.136336,30.367273 -86.950356,30.442973") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1245475, "cityName": "Midway" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364781239107586, "text": "@ebbtideapp Tide in Wickford, Rhode Island 05/25/2016\n Low 4:07am 0.4\nHigh 11:02am 3.6\n Low 3:59pm 0.5\nHigh 11:19pm 4.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-71.445,41.5717"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 95, "friends_count": 1, "statues_count": 35679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Kingstown, RI", "id": "008ee33667496868", "name": "North Kingstown", "place_type": "city", "bounding_box": rectangle("-71.553828,41.50007 -71.402717,41.654647") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44009, "countyName": "Washington" } }
+{ "create_at": datetime("2016-05-25T00:00:00.000Z"), "id": 735364781390090240, "text": "Wind 7.0 mph SSW. Barometer 1015.41 mb, Rising. Temperature 72.6 °F. Rain today 0.01 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364781989896192, "text": "Do not regret nothing I did, I do not live out of guilt whatsoever.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1515155641, "name": "Mel.F", "screen_name": "xoxmelissssaa", "lang": "en", "location": "null", "create_at": date("2013-06-13"), "description": "ambitious soul with a feisty attitude✨", "followers_count": 677, "friends_count": 622, "statues_count": 22380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Sarpy, LA", "id": "926e5e257b6ebdd1", "name": "New Sarpy", "place_type": "city", "bounding_box": rectangle("-90.398978,29.968775 -90.379102,29.99032") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2255140, "cityName": "New Sarpy" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364782186999808, "text": "@SoulKillur hey, men can be beautiful too", "in_reply_to_status": 735364494340280321, "in_reply_to_user": 2725386414, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2725386414 }}, "user": { "id": 71825739, "name": "Nick", "screen_name": "HaloMillennium", "lang": "en", "location": "New Jersey, USA", "create_at": date("2009-09-05"), "description": "I play games on the YouTupes | I'm not great at making friends | I really like cute things | icon by @Nomnomnami | header art by @SkittlezJuice", "followers_count": 257, "friends_count": 336, "statues_count": 57853 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Browns Mills, NJ", "id": "d6a0035751296dd8", "name": "Browns Mills", "place_type": "city", "bounding_box": rectangle("-74.603325,39.948198 -74.520915,39.990313") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34005, "countyName": "Burlington", "cityID": 3408455, "cityName": "Browns Mills" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364782438682624, "text": "@cceciliramirez @delgadillo505 https://t.co/LAGH3yLrDM", "in_reply_to_status": 735364307018473472, "in_reply_to_user": 4209015553, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4209015553, 2221860750 }}, "user": { "id": 2263742652, "name": "Anthziri", "screen_name": "anthziri_ibarra", "lang": "en", "location": "Ejutla, Jalisco", "create_at": date("2013-12-26"), "description": "•shhs 16• sc: anthziri", "followers_count": 318, "friends_count": 796, "statues_count": 4166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364782459588610, "text": "i'd give anything to hear half your breath", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2198084400, "name": "linda", "screen_name": "sandovaleslie", "lang": "en", "location": "NYC too SLC ", "create_at": date("2013-11-16"), "description": "rest in paradise raul armando angel ❤️", "followers_count": 842, "friends_count": 950, "statues_count": 21052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millcreek, UT", "id": "5af91023ed39df89", "name": "Millcreek", "place_type": "city", "bounding_box": rectangle("-111.921221,40.664889 -111.839259,40.712147") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4950150, "cityName": "Millcreek" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364782556090368, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":61.2°F Wind:1.1mph Pressure: 30.12hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 84, "friends_count": 18, "statues_count": 319502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364782631555072, "text": "Ripley SW Limestone Co. Temp: 72.0°F Wind:3.8mph Pressure: 998.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364783013236737, "text": "And I'm glad for the time we spent together.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 396733888, "name": "yoshi", "screen_name": "AffinySnyder", "lang": "en", "location": "Antarctica", "create_at": date("2011-10-23"), "description": "ISU 2017. 5 Star Man / Golden God . aggressive cuddler", "followers_count": 201, "friends_count": 484, "statues_count": 11848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364783042666501, "text": "Yeah I know https://t.co/aIKVY8YNKg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 720735568813039617, "name": "bobbyclutchh", "screen_name": "bobbyclucthhh", "lang": "en", "location": "null", "create_at": date("2016-04-14"), "description": "null", "followers_count": 124, "friends_count": 177, "statues_count": 414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364783445311488, "text": "@alexfrmjdm call my girlfriend baby girl one more fucking time I dare you , Ima fuck you up", "in_reply_to_status": -1, "in_reply_to_user": 703046180561760256, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 703046180561760256 }}, "user": { "id": 3254077195, "name": "Matthew McClain", "screen_name": "mcclain98_matt", "lang": "en", "location": "sammies corazon ", "create_at": date("2015-06-23"), "description": "@sammm_padilla❣", "followers_count": 83, "friends_count": 171, "statues_count": 2282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364783587885058, "text": "Seeing Gabriel today made me so so sad like I miss him ��❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 527922505, "name": "Alondra✨❣", "screen_name": "_Ayoo_alondra", "lang": "en", "location": "null", "create_at": date("2012-03-17"), "description": "R.I.P Papa G&Alex❤ I need a Mercedes Benz", "followers_count": 562, "friends_count": 1253, "statues_count": 20427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364783839531008, "text": "Overwhelmed but I'm still gone do what needs to be done.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1363257822, "name": "Mannie Fresh", "screen_name": "OhhMyJam", "lang": "en", "location": "null", "create_at": date("2013-04-18"), "description": "R.I.P Grandad...Never give them the satifaction by giving them a reaction.. I'm Blessed. #DJH5 #AristocratLife #VLMG #TxSU16", "followers_count": 892, "friends_count": 843, "statues_count": 56083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784074395648, "text": "Wind 1.0 mph ESE. Barometer 29.975 in, Steady. Temperature 65.5 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784351248390, "text": "I hate that now that I have a car again, I'm back to sleeping alone lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226445935, "name": "najar, evelyn", "screen_name": "evelynnajar_", "lang": "en", "location": "Phoenix, Arizona", "create_at": date("2010-12-13"), "description": "19 | IG: evelynnajar_ | sc: evelynnigga", "followers_count": 1057, "friends_count": 1269, "statues_count": 16339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784674242562, "text": "Wind 0.1 mph WNW. Barometer 29.903 in, Steady. Temperature 50.8 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784707764225, "text": "Wind 0.0 mph ---. Barometer 30.048 in, Steady. Temperature 70.2 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784716193794, "text": "I'm that girlfriend who will act like ya momma at times bc all I want is nothing but the best for you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2712937152, "name": "princess", "screen_name": "Slimm_shady7", "lang": "en", "location": "null", "create_at": date("2014-08-06"), "description": "•Zyharia•cha'bryal ", "followers_count": 984, "friends_count": 648, "statues_count": 21335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784833613825, "text": "Temp 54.6° Hi/Lo 55.1/54.6 Rng 0.5° WC 54.6° Hmd 97% Rain 0.00\" Storm 0.27\" BAR 30.004 Rising DP 53.8° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 18970 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364784963629057, "text": "You plug one side in to the tv, one to the wall. https://t.co/czQz4ORaYt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1087068746, "name": "Wally", "screen_name": "DWallTheGod", "lang": "en", "location": "Paradise Valley, AZ", "create_at": date("2013-01-13"), "description": "Snapchat: dwallthegod", "followers_count": 648, "friends_count": 450, "statues_count": 14317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.62655,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364785248817153, "text": "@jonnjon_ hahaha to show love ����", "in_reply_to_status": 735364437855621120, "in_reply_to_user": 255863932, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 255863932 }}, "user": { "id": 94834257, "name": "Jay⚡️", "screen_name": "jaycastillo92", "lang": "en", "location": "null", "create_at": date("2009-12-05"), "description": "null", "followers_count": 332, "friends_count": 344, "statues_count": 24223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364785362108417, "text": "Wind 0.0 mph ---. Barometer 30.069 in, Falling slowly. Temperature 61.7 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 20066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-25T00:00:01.000Z"), "id": 735364785550819329, "text": "05/25@03:00 - Temp 62.5F, WC 62.5F. Wind 1.0mph E, Gust 2.0mph. Bar 30.054in, Rising slowly. Rain 0.00in. Hum 69%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364786054127617, "text": "Come take a walk on the wild side, lemme kiss you hard in the pouring rain", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 332833133, "name": ".Fahvuh.➰", "screen_name": "KirbzGuy", "lang": "en", "location": "thats none of your business ", "create_at": date("2011-07-10"), "description": "Prince of Naps&Knots .an artistic athlete. ig:whynot_imcraig #HU19", "followers_count": 2481, "friends_count": 1470, "statues_count": 146936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364786385457153, "text": "Edmonton Houses for rent on RentBoard.ca https://t.co/uXPLcm69nr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.4909267,53.544389"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31169027, "name": "Rent Board of Canada", "screen_name": "rentboard", "lang": "en", "location": "Canada", "create_at": date("2009-04-14"), "description": "Apartments and Houses for Rent in Canada", "followers_count": 321, "friends_count": 0, "statues_count": 123448 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787023056897, "text": "Temp: 64.9°F Wind:0.0mph Pressure: 30.115hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787098488832, "text": "It's too late for our group . You gotta reach out to these kids from '99 on back . https://t.co/WNbiNYFuVn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 240527898, "name": "Issac Pogue", "screen_name": "asvp_ike", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2011-01-19"), "description": "Southern University and A&M College | CRD", "followers_count": 836, "friends_count": 794, "statues_count": 25914 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787362779136, "text": "I don't even know what i feel anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1110775279, "name": "Boo'Bear lol ⚓⚓", "screen_name": "Jazzy_Jazminee", "lang": "en", "location": "dallas,texas☔️☀☁", "create_at": date("2013-01-21"), "description": "3c af\n9teen♌\n@_Fraydoe followed July 29, 2015 10:30 pm\r\n@AllenSarinana_ followed August 1, 2015 4:40pm\r\n@Em3_Music followed October 31, 2015 9:00pm", "followers_count": 816, "friends_count": 1163, "statues_count": 15200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787413131264, "text": "https://t.co/5ouQYbZTBh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2435025128, "name": "Broly", "screen_name": "BrodyGraham", "lang": "en", "location": "saugus", "create_at": date("2014-04-08"), "description": "Semper Paratus", "followers_count": 250, "friends_count": 238, "statues_count": 856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787484385280, "text": "\"Was\" my downfall https://t.co/BRoIy8sroM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2895115166, "name": "Jacob Bettis", "screen_name": "jakebettis8", "lang": "en", "location": "Burbank, CA", "create_at": date("2014-11-09"), "description": "❤12-28-14 Joy Abisamra❤️", "followers_count": 339, "friends_count": 362, "statues_count": 2214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787492823040, "text": "I know there is no \"there\" there. But tonight was still nice.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 429741861, "name": "K Fend", "screen_name": "TheRealKFend", "lang": "en", "location": "null", "create_at": date("2011-12-06"), "description": "I'm on the snapchat: KFend", "followers_count": 252, "friends_count": 461, "statues_count": 18543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364787870289921, "text": "I have to do my essay second period. FML.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2163991260, "name": "Savy", "screen_name": "xoxo_savvy01", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2013-10-29"), "description": "Gvhs. Cheer//Racing", "followers_count": 148, "friends_count": 309, "statues_count": 788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788088344576, "text": "�������� https://t.co/Erg0YqQK8G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3045426836, "name": "#BossManeKween", "screen_name": "FlyHigh_WithMe", "lang": "en", "location": "null", "create_at": date("2015-02-26"), "description": "'93 | Snapchat: iamkweent | IG: KweenHarmon", "followers_count": 138, "friends_count": 131, "statues_count": 6397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788151300096, "text": "They really eating �� https://t.co/fMFTpbwAuU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3256728804, "name": "Bre.", "screen_name": "x__cheeks", "lang": "en", "location": "soon to be Houston,Tx ", "create_at": date("2015-06-26"), "description": "independent queen working for her throne✨", "followers_count": 2066, "friends_count": 502, "statues_count": 61927 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788356845568, "text": "@CW_TheFlash wow what are we going to do next. Barry's moms back,??? Timeline changed now what.", "in_reply_to_status": -1, "in_reply_to_user": 2282671736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2282671736 }}, "user": { "id": 1453150296, "name": "Chris Chavira(Pops)", "screen_name": "chrischavira2", "lang": "en", "location": "Sylmar, Los Angeles", "create_at": date("2013-05-23"), "description": "null", "followers_count": 840, "friends_count": 929, "statues_count": 1296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788411346944, "text": "@missrebecaa @SlickSlickNick_ well....I was at the Bucees lol", "in_reply_to_status": 735364357333356544, "in_reply_to_user": 3321387918, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3321387918, 287947083 }}, "user": { "id": 1081765278, "name": "King T'Challa", "screen_name": "MattWakhu", "lang": "en", "location": "Houston, TX", "create_at": date("2013-01-11"), "description": "The Great Blacksby | You'll probably unfollow me for no reason | GentlΣmaΝ | UNT | Kenyan | Follow my music page @MattWakhuMusic!", "followers_count": 3670, "friends_count": 2470, "statues_count": 132839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788612665344, "text": "@CloudMarine tysmmm!! ��", "in_reply_to_status": 735364582496186368, "in_reply_to_user": 237571443, "favorite_count": 0, "retweet_count": 0, "lang": "sv", "is_retweet": false, "user_mentions": {{ 237571443 }}, "user": { "id": 3305541043, "name": "死んだGIRL", "screen_name": "miscellmaniac", "lang": "en", "location": "null", "create_at": date("2015-08-03"), "description": "☆ I HAVE NOTHING LEFT TO LIVE FOR AND I DRIVE LIKE IT ☆ anime, memes, video games (esp smash), 'n cat related tweets probs ☆", "followers_count": 132, "friends_count": 190, "statues_count": 11258 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364788725911554, "text": "Bsjsjsjsjsbs Fuck my Kylie and Kendall tweet is deleted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 3299840329, "name": "LOTB", "screen_name": "rehannafenty", "lang": "en", "location": "Dearborn, MI", "create_at": date("2015-07-28"), "description": "if you take me serious you dumb asl", "followers_count": 2585, "friends_count": 1848, "statues_count": 11217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn, MI", "id": "339bd7ae6a55ba9f", "name": "Dearborn", "place_type": "city", "bounding_box": rectangle("-83.287094,42.277554 -83.14002,42.35191") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621000, "cityName": "Dearborn" } }
+{ "create_at": datetime("2016-05-25T00:00:02.000Z"), "id": 735364789598355457, "text": "Listen To (The Intro).\n\nby- Farook Marshall. https://t.co/fpUp9ahaL0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173149893, "name": "F. Marshall", "screen_name": "FarookMarshall", "lang": "en", "location": "Lagos. London. Chicago.", "create_at": date("2010-07-31"), "description": "• •|Advocate For Justice | Writer | Musician|• • Inquiries: farookmarshall.mgmt@gmail.com (SHiNE) Instagram: musettesaphon", "followers_count": 1429, "friends_count": 769, "statues_count": 13177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364790042890241, "text": "If i see that fucking dog filter one more fucking time !! I swear and wats funny is some of u look better with it \n\n#snapchat #dogfilter ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "snapchat", "dogfilter" }}, "user": { "id": 59703923, "name": "⚡FLASHGOTTI ⚡", "screen_name": "flashgotti", "lang": "en", "location": "in yo guts wit this big dick", "create_at": date("2009-07-23"), "description": "IG: FLASHGOTTIOFFICIAL \n\n#SURPRISEBITCH!! \nIts rapper and big #dick slanger with #zerofucks #FLASHGOTTI reppin chi-town to the fullest follow me", "followers_count": 1042, "friends_count": 838, "statues_count": 17873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364790395232256, "text": "@cook_dillon uh oh it's evening out now... I can't believe zero votes for a back piece.. I thought that'd be more popular..", "in_reply_to_status": 734662124757262336, "in_reply_to_user": 417325027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 417325027 }}, "user": { "id": 417325027, "name": "dill∅n c∅∅k", "screen_name": "cook_dillon", "lang": "en", "location": "Follow on gram @thedilloncook", "create_at": date("2011-11-20"), "description": "Studies at Oklahoma State: exercise & health, Psychology, Philosophy. Apprentice JKD instructor. I3 music; hardcore/electronic. Christ Student & I lub ur face.", "followers_count": 309, "friends_count": 180, "statues_count": 7889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stillwater, OK", "id": "d0287cda13b8d994", "name": "Stillwater", "place_type": "city", "bounding_box": rectangle("-97.144924,36.079285 -97.015485,36.180093") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40119, "countyName": "Payne", "cityID": 4070300, "cityName": "Stillwater" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364790525284352, "text": "@__xoac all u", "in_reply_to_status": 735364459259154432, "in_reply_to_user": 1000917937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1000917937 }}, "user": { "id": 2719979400, "name": "paola salcedo", "screen_name": "paaowow_", "lang": "en", "location": "Coachella, CA/Fullerton, CA", "create_at": date("2014-08-09"), "description": "baseball & nick, xoxo", "followers_count": 730, "friends_count": 352, "statues_count": 30975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364791179595776, "text": "Wanna get up & get some lucky charms but I'm just too lazy for that.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 189364650, "name": "Krissy Sardelich", "screen_name": "Sardelich8", "lang": "en", "location": "Angleton/Laredo, TX", "create_at": date("2010-09-10"), "description": "I spent last summer in Alaska hunting wolverines with my uncle.", "followers_count": 1522, "friends_count": 1463, "statues_count": 29351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Angleton, TX", "id": "fe6fdbea8b8a1476", "name": "Angleton", "place_type": "city", "bounding_box": rectangle("-95.454639,29.10696 -95.390443,29.205233") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4803264, "cityName": "Angleton" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364791338930177, "text": "Jakaila is over here drunk crying ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 571471664, "name": "06/25❣", "screen_name": "baaby_jaaykaay", "lang": "en", "location": "null", "create_at": date("2012-05-04"), "description": "❣", "followers_count": 1403, "friends_count": 838, "statues_count": 25979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364791536128000, "text": "I don't think anyone understands how lit our apartment is about to be @virgg18 @lauren_boltz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2866152778, 3094607053 }}, "user": { "id": 3434550556, "name": "Java", "screen_name": "Java_Bell", "lang": "en", "location": "Tuscaloosa, AL", "create_at": date("2015-08-21"), "description": "Production Assistant for the SEC Network & Crimson Tide Productions | Sports Broadcasting & Law Undergrad | University of Alabama | Hawaii Grown", "followers_count": 151, "friends_count": 395, "statues_count": 1143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364791565488128, "text": "missing dog �������������� https://t.co/r9DCQUtZnq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2779826539, "name": "S T E V E ®", "screen_name": "avelar_steven", "lang": "en", "location": "null", "create_at": date("2014-08-29"), "description": "don't hmu when I take off..", "followers_count": 160, "friends_count": 210, "statues_count": 1178 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apache Junction, AZ", "id": "bf09d4c99c2d845c", "name": "Apache Junction", "place_type": "city", "bounding_box": rectangle("-111.587098,33.378739 -111.469058,33.465988") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 402830, "cityName": "Apache Junction" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792341438465, "text": "I like that https://t.co/ssbB6xuOp0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170165402, "name": "AlmightyKM", "screen_name": "KingMarleey", "lang": "en", "location": "ƒℓαмє ¢ιту , AZ ", "create_at": date("2010-07-23"), "description": "Music Producer #KM Ω 23, αяιչσηα ƒσυη∂. inquires on sounds email kingmarleeybeats@gmail.com™ ............... չαηιуαн. ❤️", "followers_count": 4136, "friends_count": 445, "statues_count": 48081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792492396544, "text": "Love this bro ���������� https://t.co/02X9TZSyqi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 228127637, "name": "Dialló Mítch", "screen_name": "__mystylist", "lang": "en", "location": "null", "create_at": date("2010-12-18"), "description": "Model • For all Business|Booking Inquires - bookingmystylist21@gmail.com | Instagram: @__Mystylist #IMHERETOINSPIRE✨", "followers_count": 1638, "friends_count": 1620, "statues_count": 32251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "River Rouge, MI", "id": "e7cdcd61b0c07e97", "name": "River Rouge", "place_type": "city", "bounding_box": rectangle("-83.153043,42.258976 -83.103286,42.290022") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2668760, "cityName": "River Rouge" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792525946883, "text": "All oatmeal beside strawberry and cream��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842581807, "name": "Zae", "screen_name": "_Based_Religion", "lang": "en", "location": "leanin in paradise", "create_at": date("2012-09-23"), "description": "Jamaican |What I say may offend you but it offends me that you're offended #RestEasyDee", "followers_count": 1172, "friends_count": 651, "statues_count": 40397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Killeen, TX", "id": "a27a0542e774e138", "name": "Killeen", "place_type": "city", "bounding_box": rectangle("-97.848463,31.014356 -97.659217,31.144257") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4839148, "cityName": "Killeen" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792714694656, "text": "I need this �� https://t.co/QMUdJW49Cm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4659165612, "name": "Henry", "screen_name": "henn_hamm", "lang": "en", "location": "Compton, CA", "create_at": date("2015-12-26"), "description": "19, Su dude", "followers_count": 76, "friends_count": 183, "statues_count": 104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, CA", "id": "7df6f50f15138f28", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-118.282262,33.901902 -118.222378,33.929527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685614, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792832167937, "text": "Damit mom!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 606572485, "name": "Melissa Zarate", "screen_name": "melizarate79", "lang": "en", "location": "San Antonio ", "create_at": date("2012-06-12"), "description": "A1 since day one", "followers_count": 843, "friends_count": 334, "statues_count": 36163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagle Pass, TX", "id": "d0a3166d28f2660e", "name": "Eagle Pass", "place_type": "city", "bounding_box": rectangle("-100.511776,28.683498 -100.449475,28.742791") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48323, "countyName": "Maverick", "cityID": 4821892, "cityName": "Eagle Pass" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364792844701697, "text": "Wind 8.0 mph SSE. Barometer 29.897 in, Falling slowly. Temperature 76.0 °F. Rain today 0.00 in. Humidity 83%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364793205436416, "text": "Happy Bday to my WCE @so_sofs ❤️ Can't wait to show you what I have planned for you P.S. You're beautiful https://t.co/s3f8DlQt5E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 64660938 }}, "user": { "id": 185832543, "name": "Chris", "screen_name": "CROD_27", "lang": "en", "location": "null", "create_at": date("2010-09-01"), "description": "When that shit is real, you just know // S.M..❤️", "followers_count": 364, "friends_count": 283, "statues_count": 28007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364793431887873, "text": "@fernyx_ to bitch at a different Jack probably lol.", "in_reply_to_status": 735363834710532096, "in_reply_to_user": 214783620, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 214783620 }}, "user": { "id": 79362077, "name": "Strawberry Swisher", "screen_name": "KidOsito", "lang": "en", "location": "IG:DeathOfTheStrawberrySwisher", "create_at": date("2009-10-02"), "description": "20, Copacetic, Pretty Much", "followers_count": 117, "friends_count": 89, "statues_count": 9775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fallbrook, CA", "id": "93ee41c795df1086", "name": "Fallbrook", "place_type": "city", "bounding_box": rectangle("-117.262188,33.318495 -117.177635,33.414093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 623462, "cityName": "Fallbrook" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364793436110848, "text": "Wind 0.0 mph ---. Barometer 30.00 in, Steady. Temperature 53.4 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 122, "statues_count": 160041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364793666801664, "text": "Wind 0.0 mph ---. Barometer 30.16 in, Steady. Temperature 67.5 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-25T00:00:03.000Z"), "id": 735364794119782400, "text": "@ashanti Can you Call up Lloyd & y'all make something like a SouthSide 2 we need that type of Loving R&B�� voices vibe well together����", "in_reply_to_status": -1, "in_reply_to_user": 67982898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 67982898 }}, "user": { "id": 2407702867, "name": "⚓kikii⚓", "screen_name": "TrueeHeeYo", "lang": "en", "location": "Houston, TX", "create_at": date("2014-03-23"), "description": "Lost everything I Loved & cared for but I'm still Blessed! In all I learned a lesson from it you never see it coming you just get to see it go", "followers_count": 474, "friends_count": 468, "statues_count": 11304 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364794421792769, "text": "Huckleberry get your tongue back in your mouth https://t.co/nZ0RA7QGTX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586863213, "name": "Megan", "screen_name": "MeganjSalazar", "lang": "en", "location": "who even knows anymore", "create_at": date("2013-07-11"), "description": "CWU", "followers_count": 363, "friends_count": 305, "statues_count": 3028 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ellensburg, WA", "id": "c95cdb2a983262e5", "name": "Ellensburg", "place_type": "city", "bounding_box": rectangle("-120.582586,46.958017 -120.49726,47.028542") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53037, "countyName": "Kittitas", "cityID": 5321240, "cityName": "Ellensburg" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364794832785409, "text": "We lost half my hockey team when I was a freshman because of Facebook ���� https://t.co/HOlqYO6lZC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36606177, "name": "BlakeFromStateFarm", "screen_name": "BlakeErvin", "lang": "en", "location": "I wish I knew ", "create_at": date("2009-04-29"), "description": "I seek frisson through pixels and words. Ervin_blake@hotmail.com for sessions and inquires Ak✈️Hi", "followers_count": 2279, "friends_count": 1306, "statues_count": 35876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gateway, AK", "id": "014a9e6864a3be2c", "name": "Gateway", "place_type": "city", "bounding_box": rectangle("-149.329835,61.547233 -149.177953,61.633205") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2170, "countyName": "Matanuska-Susitna", "cityID": 228200, "cityName": "Gateway" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364795021570052, "text": "@_macnasty lol just kiddin' boii I should be sleepin rn tho wtf is wrong wit u", "in_reply_to_status": 735364606231781376, "in_reply_to_user": 1721059664, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1721059664 }}, "user": { "id": 770934757, "name": "CJ", "screen_name": "Millerdelic", "lang": "en", "location": "Ferndale, MI", "create_at": date("2012-08-20"), "description": "18. I'm a lover. Avid dreamer. And may I say - not in a shy way... I did it my way. - Frank Sinatra", "followers_count": 388, "friends_count": 319, "statues_count": 2388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lapeer, MI", "id": "7a21fc874964c9a9", "name": "Lapeer", "place_type": "city", "bounding_box": rectangle("-83.380975,43.023707 -83.27301,43.092988") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26087, "countyName": "Lapeer", "cityID": 2646040, "cityName": "Lapeer" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364795180912640, "text": "@shainerose86 I'm probably a lot older than you think ;)", "in_reply_to_status": 735363929384312832, "in_reply_to_user": 371504023, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 371504023 }}, "user": { "id": 2316729433, "name": "Callie Helm", "screen_name": "CallieHelm86", "lang": "en", "location": "Colorado", "create_at": date("2014-01-28"), "description": "Writer/Social Media/Marketing| Art & Music| failing perfectionist | I've got bite marks on my tongue from all the things I've never said...", "followers_count": 400, "friends_count": 309, "statues_count": 5945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364795545882624, "text": "There is nothing more graceful than a shirtless pantsless man defending his musical taste", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2382550140, "name": "DeeJay", "screen_name": "unSanctuary", "lang": "en", "location": "N/A", "create_at": date("2014-03-10"), "description": "Eh, it could be worse i guess.", "followers_count": 123, "friends_count": 122, "statues_count": 1964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364795596214272, "text": "kill both they ass �� https://t.co/iKpVgYDvjr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2758958212, "name": "kaganleigha", "screen_name": "kayrichass", "lang": "en", "location": "null", "create_at": date("2014-09-01"), "description": "rest in peace to my hearts momma, momo,drelon,kahlil❤️", "followers_count": 380, "friends_count": 425, "statues_count": 3799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364796615380992, "text": "i love her so much https://t.co/VMJZDWJCfD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 981782682, "name": "Tay.", "screen_name": "tay_anderson07", "lang": "en", "location": "Ogden, UT ", "create_at": date("2012-11-30"), "description": "Colorado Northwestern Volleyball || #3 || that one blonde girl || snap @ taylorgirl", "followers_count": 1236, "friends_count": 811, "statues_count": 9450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan Beach, CA", "id": "3e5f2219d3106113", "name": "Manhattan Beach", "place_type": "city", "bounding_box": rectangle("-118.423186,33.872825 -118.378582,33.906148") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645400, "cityName": "Manhattan Beach" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364796816707584, "text": "@galnoa00 somebody hook us up����", "in_reply_to_status": 735364429458636800, "in_reply_to_user": 1850392610, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1850392610 }}, "user": { "id": 552360325, "name": "allie", "screen_name": "alliecrescii", "lang": "en", "location": "null", "create_at": date("2012-04-12"), "description": "null", "followers_count": 542, "friends_count": 475, "statues_count": 5642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunnyvale, CA", "id": "45cadd6ef118ec9f", "name": "Sunnyvale", "place_type": "city", "bounding_box": rectangle("-122.065206,37.330068 -121.982475,37.426726") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 677000, "cityName": "Sunnyvale" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364797139718144, "text": "@philomenashea @vikkiedominguez @motelroses @DianaCody HELL YA", "in_reply_to_status": 735363911361384448, "in_reply_to_user": 2313248244, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2313248244, 836654737, 1521783295, 620565801 }}, "user": { "id": 1963948706, "name": "little T", "screen_name": "chacontay", "lang": "en", "location": "Gilbert, AZ", "create_at": date("2013-10-15"), "description": "I'm Taylor freakin chacon got my Latina @annisajehning & my blonde @angelallinson", "followers_count": 1055, "friends_count": 938, "statues_count": 4070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364797651357698, "text": "We're #hiring! Click to apply: Cashier - https://t.co/3URCzOH69F #CustomerService #restaurantlife #LittleFalls, NY #Job #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.8595957,43.0434039"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "CustomerService", "restaurantlife", "LittleFalls", "Job", "Jobs" }}, "user": { "id": 701776772057141248, "name": "HMSHost Jobs", "screen_name": "HMSHostCareers", "lang": "en", "location": "null", "create_at": date("2016-02-22"), "description": "HMSHost is part of the world’s largest provider of food & beverage services for travelers. Explore our various hourly and management positions in US & Canada!", "followers_count": 288, "friends_count": 220, "statues_count": 2287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Falls, NY", "id": "e3b5a2c949379292", "name": "Little Falls", "place_type": "city", "bounding_box": rectangle("-74.882155,43.025942 -74.83128,43.064137") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36043, "countyName": "Herkimer", "cityID": 3642741, "cityName": "Little Falls" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364797798191104, "text": "Wind 0.0 mph ---. Barometer 30.086 in, Steady. Temperature 62.5 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 405, "friends_count": 294, "statues_count": 10754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364797819170816, "text": "Everyday thoughts lol https://t.co/AJ1jZKfM93", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458147769, "name": "bugatti hottie", "screen_name": "chellae_rae", "lang": "en", "location": "Inglewood, CA", "create_at": date("2012-01-07"), "description": "I'm Marsha you're Jan ✨", "followers_count": 931, "friends_count": 680, "statues_count": 37037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364798007894017, "text": "Late night stroll �������� https://t.co/xFfedwWvtm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1164463460, "name": "Victor Adames Jr.", "screen_name": "V_Adames777", "lang": "en", "location": "McAllen, Tx / Kingsville, Tx", "create_at": date("2013-02-09"), "description": "#TAMUK'19", "followers_count": 515, "friends_count": 515, "statues_count": 5585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364798104375296, "text": "Swear I'm finna get me a white boy lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 334439250, "name": "Lakiya.", "screen_name": "allaboutherrrs", "lang": "en", "location": "null", "create_at": date("2011-07-12"), "description": "sleep in peace my angels", "followers_count": 2870, "friends_count": 2091, "statues_count": 83562 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364798108565504, "text": "@spiffybabygirl we have to be supportive �� & right instead of $12 a month it's $20-$24 a month ��", "in_reply_to_status": 735364355269754881, "in_reply_to_user": 1519654093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1519654093 }}, "user": { "id": 2466641557, "name": "A'miracle Barlett", "screen_name": "jewelbarlett", "lang": "en", "location": "Rialto, CA", "create_at": date("2014-04-27"), "description": "achieving goals | track & field | Carter High School", "followers_count": 212, "friends_count": 289, "statues_count": 3346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-05-25T00:00:04.000Z"), "id": 735364798234406912, "text": "Trying really hard to get back into playing guitar and piano", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 364658225, "name": "SuzukiAnthony", "screen_name": "Andtonysanchez", "lang": "en", "location": "null", "create_at": date("2011-08-29"), "description": "Ask the Right Questions, and you will be surprised how much you can learn", "followers_count": 171, "friends_count": 242, "statues_count": 1132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364798527967232, "text": "@NICKIMINAJ none hot fries", "in_reply_to_status": 735346870474661888, "in_reply_to_user": 35787166, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 35787166 }}, "user": { "id": 312076682, "name": "bb girl", "screen_name": "2heartless4u", "lang": "en", "location": "null", "create_at": date("2011-06-06"), "description": "such a lady but I'm dancin like a hoe", "followers_count": 293, "friends_count": 349, "statues_count": 15988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevens Point, WI", "id": "c0b44c42d36404e3", "name": "Stevens Point", "place_type": "city", "bounding_box": rectangle("-89.60726,44.493975 -89.491635,44.586049") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55097, "countyName": "Portage", "cityID": 5577200, "cityName": "Stevens Point" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364798666448898, "text": "Posting a depressing status every minute . Chill fam", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3247486273, "name": "Jorge Diaz", "screen_name": "JorgeeDiaaz", "lang": "en", "location": "Anaheim, CA", "create_at": date("2015-06-16"), "description": "Ahs , Vrst. Wideout", "followers_count": 184, "friends_count": 224, "statues_count": 3151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364798792269824, "text": "Temp 49.5°F Wind Chill 49.5°F RH 91% Wind 0.0 --- Gust 0.0 --- SLP 30.186 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364798792269826, "text": "It's nothing I do this regardless.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2880417457, "name": "L's", "screen_name": "DozeHunna", "lang": "en", "location": "chiraq", "create_at": date("2014-10-27"), "description": "sc:leo_guzman", "followers_count": 734, "friends_count": 705, "statues_count": 2043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364798947446784, "text": "#nygotnochill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "nygotnochill" }}, "user": { "id": 114688995, "name": "Jazmine ❁", "screen_name": "jaz_mineeee", "lang": "en", "location": "at the trap ", "create_at": date("2010-02-16"), "description": "UCSB '20", "followers_count": 464, "friends_count": 406, "statues_count": 62607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gardena, CA", "id": "f227fd9a20f20e9d", "name": "Gardena", "place_type": "city", "bounding_box": rectangle("-118.326601,33.865559 -118.290531,33.916511") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 628168, "cityName": "Gardena" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364799165530115, "text": "Let's get it https://t.co/30nGqxSR7g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1641377767, "name": "FINESSED今日で", "screen_name": "tristinmueller2", "lang": "en", "location": "Hating life ", "create_at": date("2013-08-02"), "description": "Not much to tell. SC: tmues 6ft and gifted s13 type x/ SR20", "followers_count": 623, "friends_count": 480, "statues_count": 15557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364799534665728, "text": "@NBCTheVoice @alisanporter what a fix up show...I'm glad xchistina will be out for a while. A keys on the house baby...a better singer", "in_reply_to_status": 734837059819425793, "in_reply_to_user": 216444984, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 216444984, 23004135 }}, "user": { "id": 2942362907, "name": "Claudiam", "screen_name": "60acd0252d824e2", "lang": "en", "location": "Kendale Lakes, FL", "create_at": date("2014-12-26"), "description": "null", "followers_count": 6, "friends_count": 15, "statues_count": 553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendale Lakes, FL", "id": "c28ef1055654ebbb", "name": "Kendale Lakes", "place_type": "city", "bounding_box": rectangle("-80.431609,25.684836 -80.383241,25.730043") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236062, "cityName": "Kendale Lakes" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364799538827264, "text": "@ojackiee do you want some company", "in_reply_to_status": 735363716481470464, "in_reply_to_user": 224097832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 224097832 }}, "user": { "id": 33442926, "name": "Liezel", "screen_name": "liezelpablo", "lang": "en", "location": "null", "create_at": date("2009-04-19"), "description": "explicit content (you've been warned)", "followers_count": 254, "friends_count": 172, "statues_count": 12803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364799723413504, "text": "I Didnt wanna copy them , so i had an idea to ditch the techstep bandwagon & focus on my songwriting w/ american pop vocalists on UK radio", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 324553747, "name": "Hunter Legion", "screen_name": "SLI_Hunter", "lang": "en", "location": "Atlanta Ga", "create_at": date("2011-06-26"), "description": "Hunter Watson /// USA ///Legion [Ram Records]", "followers_count": 1267, "friends_count": 1732, "statues_count": 7165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364799903719424, "text": "Lego my Eggo...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19332250, "name": "Moses", "screen_name": "ohmoses", "lang": "en", "location": "Coachella...", "create_at": date("2009-01-22"), "description": "Dodgers,Dolphins, Lakers, & UCLA Bruins. I'm a DICK.", "followers_count": 677, "friends_count": 560, "statues_count": 78075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coachella, CA", "id": "2a7b8eaff804d8ec", "name": "Coachella", "place_type": "city", "bounding_box": rectangle("-116.216549,33.653032 -116.141081,33.729554") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 614260, "cityName": "Coachella" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364800608370689, "text": "The men on the strip are disgusting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2899110836, "name": "Jazz ♡", "screen_name": "jasminerosebudd", "lang": "en", "location": "Fullerton, CA", "create_at": date("2014-11-13"), "description": "*main source of Ondre's stress*", "followers_count": 324, "friends_count": 450, "statues_count": 12955 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364800620990464, "text": "Growing up isn't fun. You grow out of a lot of people", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2727917642, "name": "Liv", "screen_name": "olivia_shortay", "lang": "en", "location": "null", "create_at": date("2014-08-12"), "description": "Princess", "followers_count": 468, "friends_count": 929, "statues_count": 7391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, NE", "id": "3af2a75dbeb10500", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-96.806031,40.710053 -96.587094,40.896417") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31109, "countyName": "Lancaster", "cityID": 3128000, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364800767766528, "text": "JWMDS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "vi", "is_retweet": false, "user": { "id": 41916656, "name": "That Boy Cortez✊", "screen_name": "Cortez_AirBorne", "lang": "en", "location": "null", "create_at": date("2009-05-22"), "description": "|Host|•++", "followers_count": 3325, "friends_count": 2024, "statues_count": 57496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364801036193792, "text": "@MrNick98 I will", "in_reply_to_status": 735364000788123648, "in_reply_to_user": 1528035896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1528035896 }}, "user": { "id": 1016450131, "name": "ash", "screen_name": "asshleywager", "lang": "en", "location": "Huntington Beach, CA", "create_at": date("2012-12-16"), "description": "null", "followers_count": 2281, "friends_count": 419, "statues_count": 44369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Beach, CA", "id": "80eb17ffe368fc9a", "name": "Huntington Beach", "place_type": "city", "bounding_box": rectangle("-118.082615,33.628991 -117.91485,33.756093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636000, "cityName": "Huntington Beach" } }
+{ "create_at": datetime("2016-05-25T00:00:05.000Z"), "id": 735364801757630464, "text": "@aleekuh they really don't! �� every second counts haha", "in_reply_to_status": 735364431350222849, "in_reply_to_user": 257716354, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 257716354 }}, "user": { "id": 1036542744, "name": "taylor", "screen_name": "taysadventure", "lang": "en", "location": "UTSA ✌️", "create_at": date("2012-12-25"), "description": "brb traveling the world", "followers_count": 695, "friends_count": 461, "statues_count": 15234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lyons, NY", "id": "2bdbc29e6d65de9d", "name": "Lyons", "place_type": "city", "bounding_box": rectangle("-77.035977,43.043232 -76.957511,43.079426") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36117, "countyName": "Wayne", "cityID": 3643962, "cityName": "Lyons" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727165497442304, "text": "Follow the @CDFMS for #Tupelo, #Mississippi's Local #Business #News & inquiries. #Investment #Industrial #Commercial. #MS #MSWorks #MidSouth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Tupelo", "Mississippi", "Business", "News", "Investment", "Industrial", "Commercial", "MS", "MSWorks", "MidSouth" }}, "user_mentions": {{ 62542253 }}, "user": { "id": 1713823531, "name": "Troy David Beadles", "screen_name": "tdbeadles", "lang": "en", "location": "troydavidbeadles@mail.com", "create_at": date("2013-08-30"), "description": "(★) Holy American Dad, Mississippi Veteran, Social Network Pioneer Since '95, Life Coach, Educator, Philanthropist, Humanitarian & Independent Composer.", "followers_count": 3037, "friends_count": 4996, "statues_count": 271893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tupelo, MS", "id": "894f2ba528ba1c7d", "name": "Tupelo", "place_type": "city", "bounding_box": rectangle("-88.800213,34.203564 -88.656629,34.324583") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2874840, "cityName": "Tupelo" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727165778452480, "text": "How do people fart and not laugh? I don't get it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1709087287, "name": "SassyElephant", "screen_name": "Brayden_Brea", "lang": "en", "location": "null", "create_at": date("2013-08-28"), "description": "East Central University", "followers_count": 639, "friends_count": 550, "statues_count": 2156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ada, OK", "id": "7d89114dfdfefb68", "name": "Ada", "place_type": "city", "bounding_box": rectangle("-96.717492,34.729522 -96.617888,34.821371") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40123, "countyName": "Pontotoc", "cityID": 4000200, "cityName": "Ada" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727166999007233, "text": "I'm lowkey hot that chance the rappers verse got cut from the original version of Famous.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2834400451, "name": "D", "screen_name": "da_watts48", "lang": "en", "location": "Lake Elsinore, CA", "create_at": date("2014-09-27"), "description": "Sc: da_watts48 insta:watts_48", "followers_count": 349, "friends_count": 312, "statues_count": 8454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Elsinore, CA", "id": "80cf9987ff7e9762", "name": "Lake Elsinore", "place_type": "city", "bounding_box": rectangle("-117.413156,33.618447 -117.216785,33.713284") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 639486, "cityName": "Lake Elsinore" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727168307658752, "text": "@JuhMinuh broke AF ��", "in_reply_to_status": 735720730092933120, "in_reply_to_user": 1644188114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1644188114 }}, "user": { "id": 92941510, "name": "shithappensthuglife", "screen_name": "aye_jamesss", "lang": "en", "location": "null", "create_at": date("2009-11-27"), "description": "I Don't Want To Be Anything Other Than Me", "followers_count": 242, "friends_count": 235, "statues_count": 7624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727168311857152, "text": "I just got called a bitch but what's new?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3141048996, "name": "Jack", "screen_name": "_jackiv", "lang": "en", "location": "null", "create_at": date("2015-04-05"), "description": "the power of logic", "followers_count": 227, "friends_count": 185, "statues_count": 9226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, CA", "id": "b7e851d8ebd82e0f", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-117.261028,34.096687 -117.130442,34.143323") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633588, "cityName": "Highland" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727168462815232, "text": "Temp 58.2° Hi/Lo 62.7/58.2 Rng 4.5° WC 58.2° Hmd 90% Rain 0.00\" Storm 0.00\" BAR 30.060 Rising DP 55.3° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 18994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727168710283264, "text": "Old man walks in old skool. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32324942, "name": "Jake Runey", "screen_name": "jakeruney", "lang": "en", "location": "Dallas, TX", "create_at": date("2009-04-16"), "description": "I talk. I create. Mainstream hates me. sometimes im on radio. Im wrong, but always right. I swipe right to all flight attendants. I don't date white girls.", "followers_count": 889, "friends_count": 720, "statues_count": 15206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Addison, TX", "id": "570002ce3d2a7ac1", "name": "Addison", "place_type": "city", "bounding_box": rectangle("-96.856247,32.932133 -96.807772,32.987442") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4801240, "cityName": "Addison" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727169167458306, "text": "I'm bored", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2193080312, "name": "Lucy", "screen_name": "coldaslucia", "lang": "en", "location": "null", "create_at": date("2013-11-13"), "description": "null", "followers_count": 599, "friends_count": 362, "statues_count": 24641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-26T00:00:00.000Z"), "id": 735727169192615936, "text": "IGOT2PHONES☎️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 139521083, "name": "natty", "screen_name": "NataliieVancce", "lang": "en", "location": "with colin lempert ", "create_at": date("2010-05-02"), "description": "uh huh honey", "followers_count": 758, "friends_count": 631, "statues_count": 25148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, WA", "id": "f479e40901a48515", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-122.266398,47.257029 -122.144477,47.356233") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5303180, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727169440075778, "text": "@ebbtideapp Tide in Mobile Point, Alabama 05/26/2016\nHigh 2:06pm 1.5\n Low 1:18am 0.1\nHigh 2:45pm 1.5\n Low 1:32am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-88.0167,30.2333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 97, "friends_count": 1, "statues_count": 35956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727169897271296, "text": "do u? https://t.co/5B7YUFd6pU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2230542108, "name": "paigge popsiclee", "screen_name": "paigepospisil4", "lang": "en", "location": "null", "create_at": date("2013-12-04"), "description": "Snap\\ Paigey_poopey", "followers_count": 437, "friends_count": 685, "statues_count": 7041 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Vernon, IA", "id": "586173eb47ed649b", "name": "Mount Vernon", "place_type": "city", "bounding_box": rectangle("-91.443443,41.913403 -91.404879,41.935584") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1954840, "cityName": "Mount Vernon" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727170220232704, "text": "@booboojohnson2 @soriahsidaui #YMATR Promo w/ #therealrae #Singer #Songwriter \nhttps://t.co/NNNFkp64NV #YouMeAndTheRadio @TheAmourGroup", "in_reply_to_status": 734234033446637568, "in_reply_to_user": 607340245, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "hashtags": {{ "YMATR", "therealrae", "Singer", "Songwriter", "YouMeAndTheRadio" }}, "user_mentions": {{ 607340245, 2341202010, 586107395 }}, "user": { "id": 586107395, "name": "AmourGroup", "screen_name": "TheAmourGroup", "lang": "en", "location": "Atlanta | Chicago |Los Angeles", "create_at": date("2012-05-20"), "description": "Building Brands, Creating Empires IG:AmourGroup theamourgroup@gmail.com\nMiss Make It Happen", "followers_count": 301, "friends_count": 373, "statues_count": 8412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, GA", "id": "5b8df26e6d0be60b", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-84.561205,33.788139 -84.470367,33.905391") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1371492, "cityName": "Smyrna" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727170220265473, "text": "@tprimex right im saying the player base fell off hard as fuck which it did", "in_reply_to_status": 735727053018796032, "in_reply_to_user": 84193838, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 84193838 }}, "user": { "id": 267275233, "name": "Litt Romney", "screen_name": "ParachutePapi", "lang": "en", "location": "Tuscaloosa", "create_at": date("2011-03-16"), "description": "The game is mine. I deal the cards. University of Alabama 2017. Airborne paratrooper. #BlackLivesMatter", "followers_count": 1731, "friends_count": 914, "statues_count": 148594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tuscaloosa, AL", "id": "3a22597bb94d08bd", "name": "Tuscaloosa", "place_type": "city", "bounding_box": rectangle("-87.626729,33.098411 -87.440084,33.29479") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1125, "countyName": "Tuscaloosa", "cityID": 177256, "cityName": "Tuscaloosa" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727170337705984, "text": "Oomf believed in Jesus for about 5 minutes tonight. ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-79.15122079,37.32677781"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 473431944, "name": "Edwin Lee Krantz Jr", "screen_name": "LeeKrantz", "lang": "en", "location": "VA", "create_at": date("2012-01-24"), "description": "Life's good.", "followers_count": 178, "friends_count": 296, "statues_count": 3236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51031, "countyName": "Campbell" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171407208448, "text": "https://t.co/xZp3MjePC2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 24761830, "name": "Reme", "screen_name": "rememberence", "lang": "en", "location": "null", "create_at": date("2009-03-16"), "description": "Me", "followers_count": 80, "friends_count": 218, "statues_count": 128692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171440771072, "text": "I need to do a lot more running /:", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2355555350, "name": "Steve Rogers", "screen_name": "_Shapoopi_", "lang": "en", "location": "null", "create_at": date("2014-02-21"), "description": "do druggggZzzzzz || 18", "followers_count": 297, "friends_count": 342, "statues_count": 23784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171474329600, "text": "Wind 0.0 mph ---. Barometer 30.031 in, Falling. Temperature 75.0 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171554021377, "text": "Mannnn I need to get my shit together bc I have expensive taste��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2809640892, "name": "erica kane", "screen_name": "e_doww", "lang": "en", "location": "null", "create_at": date("2014-09-14"), "description": "stay at home dog mom", "followers_count": 330, "friends_count": 230, "statues_count": 8548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171730165760, "text": "@elpeinesote ?? Me dio follow y es extraño jaja. ����", "in_reply_to_status": -1, "in_reply_to_user": 2557626348, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2557626348 }}, "user": { "id": 798773437, "name": "Astrid Andrio", "screen_name": "astridandrio", "lang": "es", "location": "Guanajuato-Coahuila,México", "create_at": date("2012-09-02"), "description": "BP 23/09/12 ❤ RA 12/11/14 ❤ LM 26/09/15 Estudiante de Medicina UAdeC", "followers_count": 328, "friends_count": 323, "statues_count": 8408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagle Pass, TX", "id": "d0a3166d28f2660e", "name": "Eagle Pass", "place_type": "city", "bounding_box": rectangle("-100.511776,28.683498 -100.449475,28.742791") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48323, "countyName": "Maverick", "cityID": 4821892, "cityName": "Eagle Pass" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171734405122, "text": "@kennedyambrose I wish I knew that before I watched 10 minutes of snap chat stories", "in_reply_to_status": -1, "in_reply_to_user": 3567165793, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3567165793 }}, "user": { "id": 1390423154, "name": "jules", "screen_name": "julieeeaustinnn", "lang": "en", "location": "allen, tx / fayetteville, ar", "create_at": date("2013-04-29"), "description": "i like naps, cats and the razorbacks • #uark19", "followers_count": 1111, "friends_count": 909, "statues_count": 23052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allen, TX", "id": "22d928cbeab790ad", "name": "Allen", "place_type": "city", "bounding_box": rectangle("-96.736596,33.066464 -96.608938,33.158169") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4801924, "cityName": "Allen" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171809869826, "text": "@JazlynRamirez06 81% of ppl agree that YES, YOU EAT ASS", "in_reply_to_status": 735333007150698496, "in_reply_to_user": 3156971077, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4852277183 }}, "user": { "id": 3156971077, "name": "b(Abe)", "screen_name": "candyybling", "lang": "en", "location": "La Jolla, CA", "create_at": date("2015-04-14"), "description": "19 | Chicano | Abuelo | University of California, San Diego | Intersectional Feminist | #BlackLivesMatter | #BlackTransLivesMatter | #FreePalestine", "followers_count": 316, "friends_count": 311, "statues_count": 11434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171868626945, "text": "Ripley SW Limestone Co. Temp: 64.8°F Wind:0.0mph Pressure: 998.7mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53765 }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727171939897345, "text": "a si mero https://t.co/bRIXE0uPrL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3252448520, "name": "BRINA", "screen_name": "Suhh_brinaa", "lang": "en", "location": "null", "create_at": date("2015-06-22"), "description": "أحبك", "followers_count": 90, "friends_count": 181, "statues_count": 1505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172069949443, "text": "Wind 0.1 mph NNW. Barometer 30.047 in, Steady. Temperature 54.9 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 10984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172153794560, "text": "Wind 6.9 mph NE. Barometer 29.39 in, Falling slowly. Temperature 44.4 °F. Rain today 0.00 in. Humidity 10%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 19, "friends_count": 5, "statues_count": 42476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172157997056, "text": "Was just catching up on The Challenge then I look up and it's 2AM ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 533811183, "name": "Brandi ❤✌", "screen_name": "Xo_Brandii", "lang": "en", "location": "null", "create_at": date("2012-03-22"), "description": "BranBran ☺️ TSU Dance Team. Biomedical Science Major. Kinesiology Minor. Snapchat: branran_27. Mesquite ➡️ Stephenville, TX", "followers_count": 346, "friends_count": 283, "statues_count": 12542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stephenville, TX", "id": "26dc9449bfa45cce", "name": "Stephenville", "place_type": "city", "bounding_box": rectangle("-98.258745,32.191442 -98.179505,32.241178") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48143, "countyName": "Erath", "cityID": 4870208, "cityName": "Stephenville" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172384497666, "text": "@Bastille1791 @sbhouse1978 Trump is a guy I would like to have a beer with, & talk smack. Not a guy who I want behind any nuclear weapons.", "in_reply_to_status": 735726155391606784, "in_reply_to_user": 2846551735, "favorite_count": 0, "coordinate": point("-118.0619562,34.0172696"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2846551735, 1730320867 }}, "user": { "id": 2374404228, "name": "Robert Rudy Lagunas", "screen_name": "LagunasRobert", "lang": "en", "location": "California", "create_at": date("2014-03-05"), "description": "null", "followers_count": 290, "friends_count": 347, "statues_count": 10280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172837511169, "text": "@cherie0496 how dare you", "in_reply_to_status": -1, "in_reply_to_user": 304663833, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 304663833 }}, "user": { "id": 543064880, "name": "Paleana", "screen_name": "StanleyBreanna_", "lang": "en", "location": "Tx", "create_at": date("2012-04-01"), "description": "null", "followers_count": 282, "friends_count": 247, "statues_count": 6831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kaanapali, HI", "id": "386eee0c027ac681", "name": "Kaanapali", "place_type": "city", "bounding_box": rectangle("-156.696704,20.911087 -156.674143,20.950546") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1520000, "cityName": "Kaanapali" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727172883603456, "text": "Chipotle has got 0 chill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 703621278, "name": "Kenna of Sarasota", "screen_name": "kennaschott", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "For what am I to myself without You, but a guide to my own downfall?", "followers_count": 933, "friends_count": 452, "statues_count": 28426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173022023680, "text": "05/26@03:00 - Temp 60.7F, WC 60.7F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.115in, Steady. Rain 0.00in. Hum 91%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173022027777, "text": "Retweet :-) https://t.co/H4DbYUTMr6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341941004, "name": "syd ♛", "screen_name": "SydneyJaay", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-07-24"), "description": "single black female addicted to retail", "followers_count": 1912, "friends_count": 1213, "statues_count": 36979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173147885568, "text": "i hate jhamaria", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1289854290, "name": "klz", "screen_name": "ayoolindoe_", "lang": "en", "location": "Tucson, AZ", "create_at": date("2013-03-22"), "description": "kaidrien papiiiii ❤️", "followers_count": 718, "friends_count": 520, "statues_count": 13637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warner Robins, GA", "id": "4caf02a6f195ddab", "name": "Warner Robins", "place_type": "city", "bounding_box": rectangle("-83.728243,32.495868 -83.554155,32.669392") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1380508, "cityName": "Warner Robins" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173252747264, "text": "Wind 2.0 mph SE. Barometer 29.926 in, Steady. Temperature 72.1 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173340823553, "text": "@lanitaughtu ��������", "in_reply_to_status": 735718696434294786, "in_reply_to_user": 3243733536, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3243733536 }}, "user": { "id": 568450358, "name": "burrito babe", "screen_name": "catlady_yessica", "lang": "en", "location": "10/10", "create_at": date("2012-05-01"), "description": "yes, it's me", "followers_count": 387, "friends_count": 300, "statues_count": 8777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chowchilla, CA", "id": "f977ef0a4904c1e1", "name": "Chowchilla", "place_type": "city", "bounding_box": rectangle("-120.285778,37.10035 -120.220121,37.129769") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 613294, "cityName": "Chowchilla" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173365948416, "text": "Temp: 70.4°F Wind:0.0mph Pressure: 30.098hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173374353408, "text": "I thought that phone was near death ...����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278781268, "name": "hothead.", "screen_name": "ennorraj", "lang": "en", "location": "houston ", "create_at": date("2011-04-07"), "description": "dad. the truth is the only safe ground to stand upon. if you seek peace, prepare for war. #TxSU20 ❣", "followers_count": 750, "friends_count": 258, "statues_count": 17777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-26T00:00:01.000Z"), "id": 735727173500166145, "text": "@CrownMe_Queen23 I'll put my phone on DND and say Fuq em lol", "in_reply_to_status": 735727049558478850, "in_reply_to_user": 140471907, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 140471907 }}, "user": { "id": 36768049, "name": "(B)ahiyyah.", "screen_name": "robynnicki_", "lang": "en", "location": "somewhere wit My Gi ", "create_at": date("2009-04-30"), "description": "R.I.P RobinDenny&Basir...", "followers_count": 1373, "friends_count": 918, "statues_count": 167768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727173764448256, "text": "@jcrowlz2 feel you. Happens to the best", "in_reply_to_status": 735726645030445056, "in_reply_to_user": 440544815, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 440544815 }}, "user": { "id": 296047293, "name": "Frank Meisl", "screen_name": "M0neyintheFRANK", "lang": "en", "location": "Evergreen Park, IL", "create_at": date("2011-05-09"), "description": "BenU Baseball", "followers_count": 723, "friends_count": 626, "statues_count": 12495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lisle, IL", "id": "3009292baa8dda23", "name": "Lisle", "place_type": "city", "bounding_box": rectangle("-88.11909,41.760185 -88.042417,41.828705") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1743939, "cityName": "Lisle" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174435536896, "text": "1 Lincoln's Sparrow (Melospiza lincolnii) - Florida Canyon--lower - 2016-05-23 09:30 https://t.co/COd6AETKUy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.8458804,31.7633436"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174452273152, "text": "3 American Robin (Turdus migratorius) - Madera Canyon--Whitehouse Picnic area - 2016-05-24 05:22 https://t.co/PC5UcbJuBp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.882269,31.733668"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174502621187, "text": "2 Black-capped Gnatcatcher (Polioptila nigriceps) - Madera Canyon--Proctor Rd. - 2016-05-24 08:40 https://t.co/h3cLVQa2qQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.886147,31.739938"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174624251904, "text": "2 Buff-breasted Flycatcher (Empidonax fulvifrons) - Mt. Lemmon--Rose Canyon and Lake - 2016-05-25 06:54 https://t.co/BSboM1BkIK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-110.7024693,32.3926274"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 15775 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174724898816, "text": "Drama is something I refuse to put up with", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2258382170, "name": "Brandon Rachal", "screen_name": "breezy_thegreat", "lang": "en", "location": "null", "create_at": date("2013-12-22"), "description": "Brandon Rachal. c/o 2017 NCHS 6'6 Combo Guard #1 , #BEGREATBRAN One of the top players in the U.S. John 3:16", "followers_count": 1439, "friends_count": 805, "statues_count": 9273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Natchitoches, LA", "id": "28ee5ea3adb6eddd", "name": "Natchitoches", "place_type": "city", "bounding_box": rectangle("-93.128629,31.718432 -93.043947,31.806051") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22069, "countyName": "Natchitoches", "cityID": 2253545, "cityName": "Natchitoches" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174871699457, "text": "Wind 2.0 mph S. Barometer 1015.07 mb, Steady. Temperature 73.3 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727174892670976, "text": "Happy birthday to my bfftroml �� idk what I'd do w/o u even tho we fight all the time lmao enjoy ur day and ilysm ���� https://t.co/J9ExUumxaC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1069870903, "name": "valora ciarelli", "screen_name": "forever_valora", "lang": "en", "location": "Oxnard, CA", "create_at": date("2013-01-07"), "description": "know yourself, know your worth. waaaay up i feel blessed. Jeremiah 29:11", "followers_count": 548, "friends_count": 312, "statues_count": 23075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Hueneme, CA", "id": "f2dabb4da43780ea", "name": "Port Hueneme", "place_type": "city", "bounding_box": rectangle("-119.221552,34.138118 -119.186037,34.181304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 658296, "cityName": "Port Hueneme" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727175073030149, "text": "Twitter seriously keeps me up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1052638764, "name": "Shelbs", "screen_name": "ShelbyRenee098", "lang": "en", "location": "Fresno, CA", "create_at": date("2013-01-01"), "description": "Appreciate your blessings. #4 | sc: shelbss098", "followers_count": 1848, "friends_count": 976, "statues_count": 33209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727175291174913, "text": "Unapologetic... Self love was the first love, Second anywhere we want to take it ���� top 5 top 5 top 5 https://t.co/4ZeYFBk02g", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383446586, "name": "Aldo Smooth Carrillo", "screen_name": "alcsmooth", "lang": "en", "location": "| USA |", "create_at": date("2011-10-01"), "description": "Respect, balance ⚖ Libra. The worst thing you can be is average. If it's not my business idc. Diamond in the rough. Future Marine. sc alcsmooth ig alcsmooth", "followers_count": 450, "friends_count": 432, "statues_count": 17303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727175832240129, "text": "Let night, early mornin' ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.944267,30.211104"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 726788923851202560, "name": "V. Skillz", "screen_name": "vskillzmusik", "lang": "en", "location": "New Orleans, LA", "create_at": date("2016-05-01"), "description": "Female Music Producer | Songwriter", "followers_count": 182, "friends_count": 266, "statues_count": 230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waffle House", "id": "07d9d2a48f884004", "name": "Waffle House", "place_type": "poi", "bounding_box": rectangle("-90.94426709999999,30.211103899999998 -90.944267,30.211104") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2229850, "cityName": "Gonzales" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176020983810, "text": "Facts https://t.co/Xj9raB4oN1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1353299256, "name": "Coy 〽️", "screen_name": "yaboy_coy", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2013-04-14"), "description": "BLESSED ➰ SC: yaboycoy", "followers_count": 412, "friends_count": 758, "statues_count": 4053 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176692047872, "text": "@_Kimchanel_ it was though �� I use to always click to Maury when that came on", "in_reply_to_status": 735727055979970561, "in_reply_to_user": 90114686, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 90114686 }}, "user": { "id": 2444515914, "name": "TRISH", "screen_name": "trizzyTrish_", "lang": "en", "location": "philly ", "create_at": date("2014-04-14"), "description": "Check out whats the tea @servingtea on instgram as we serve you the tea ... For artist exposure email me at pduru1220@gmail.com", "followers_count": 1500, "friends_count": 870, "statues_count": 51101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176767545348, "text": "@tylerbakker6 https://t.co/14qaFWUW4R", "in_reply_to_status": 735727064993533952, "in_reply_to_user": 840958326, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 840958326 }}, "user": { "id": 2546756893, "name": "Mr. Hanes", "screen_name": "A_Aron250", "lang": "en", "location": "null", "create_at": date("2014-06-04"), "description": "Tennis // #TeamBooty", "followers_count": 314, "friends_count": 328, "statues_count": 4741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176784347136, "text": "@TyxRadicalRiotx ...dude, do you need a hug or something?�� Cuz I got you!*sends virtual hug*", "in_reply_to_status": 735726833606369281, "in_reply_to_user": 1115180102, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1115180102 }}, "user": { "id": 3144911194, "name": "Anastasia Shriber", "screen_name": "anastasiashrib1", "lang": "en", "location": "Garnett, KS", "create_at": date("2015-04-07"), "description": "Sup. NCCC.", "followers_count": 308, "friends_count": 276, "statues_count": 1123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garnett, KS", "id": "5127b1789d83d43c", "name": "Garnett", "place_type": "city", "bounding_box": rectangle("-95.26313,38.266381 -95.213986,38.295769") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20003, "countyName": "Anderson", "cityID": 2025925, "cityName": "Garnett" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176926928898, "text": "@Gaberzzzzz also I still need to do laundry and pack sometime this morning lol", "in_reply_to_status": 735727093812596741, "in_reply_to_user": 494298173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 494298173 }}, "user": { "id": 494298173, "name": "vs angel", "screen_name": "Gaberzzzzz", "lang": "en", "location": "Houston/College Station, TX", "create_at": date("2012-02-16"), "description": "in love with God, myself, and my man", "followers_count": 802, "friends_count": 641, "statues_count": 72286 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727176931119106, "text": "@TheGreatKenny_A https://t.co/4XBCe6va0r", "in_reply_to_status": -1, "in_reply_to_user": 164376126, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 164376126 }}, "user": { "id": 164376126, "name": "TheGreatKenny", "screen_name": "TheGreatKenny_A", "lang": "en", "location": "South West Houston ", "create_at": date("2010-07-08"), "description": "God - @Charliekasa - ΑΦΑ Prairie VIEW ALPHA MEN University - RIP Kosolu and Duce", "followers_count": 1293, "friends_count": 961, "statues_count": 47819 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727177019195392, "text": "Rad 3AM diner hangs with Breezi �� https://t.co/kmW0UvKQ4f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 94440009, "name": "ハートブレーカー", "screen_name": "6ixeyes", "lang": "en", "location": "nyc", "create_at": date("2009-12-03"), "description": "randi sav • static '04 axela • blunt(s)", "followers_count": 847, "friends_count": 168, "statues_count": 84123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727177031766017, "text": "�� https://t.co/womoaTMvUM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2892439662, "name": "tae", "screen_name": "tahjiia", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "null", "followers_count": 1319, "friends_count": 543, "statues_count": 41666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727177040166912, "text": "You said I'm steady playing, but you steady playing too!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2697491766, "name": "$hanny☀️", "screen_name": "_nonnahss", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-07-31"), "description": "yeezy taught me. #Spelman20", "followers_count": 246, "friends_count": 197, "statues_count": 8707 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727177275052033, "text": "I swear I be wanting to be like \" fuck it, do you����\" and then be laying in my bed hoping he gets home safely ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146398883, "name": "dont_call_me_mary_", "screen_name": "ReRe96_", "lang": "en", "location": "null", "create_at": date("2015-04-07"), "description": "A woman knows by intuition, or instinct, what is best for herself... -Marilyn Monroe", "followers_count": 179, "friends_count": 172, "statues_count": 4634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, LA", "id": "0020aaa25ced13e2", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-90.520806,30.615873 -90.489165,30.674604") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2237025, "cityName": "Independence" } }
+{ "create_at": datetime("2016-05-26T00:00:02.000Z"), "id": 735727177438633984, "text": "Love a man with a nice fade Jesus ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1082488459, "name": "lol..", "screen_name": "destinydestani", "lang": "en", "location": "Texas, USA", "create_at": date("2013-01-12"), "description": "be kind be courageous ⭐️ | UNT | Romans 8:28", "followers_count": 856, "friends_count": 436, "statues_count": 51942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denton, TX", "id": "f77b0bf942a40070", "name": "Denton", "place_type": "city", "bounding_box": rectangle("-97.187543,33.128938 -97.041998,33.276053") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4819972, "cityName": "Denton" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727177883213824, "text": "like for tbh in dm, deleting in 10:)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2809462725, "name": "tay", "screen_name": "vb7tay", "lang": "en", "location": "null", "create_at": date("2014-10-05"), "description": "club one volleyball||", "followers_count": 112, "friends_count": 154, "statues_count": 85 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727177921007617, "text": "Lord find me a good looking Darkskin man!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3246362629, "name": "Navi⚓️", "screen_name": "_FrenchMaack", "lang": "en", "location": "null", "create_at": date("2015-06-15"), "description": "#LU20 #LLC☔️ H-Town made❄️", "followers_count": 2128, "friends_count": 1404, "statues_count": 28750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727177983889409, "text": "This generation is fucked up on several different levels", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505589273, "name": "Garnik", "screen_name": "garnikkk", "lang": "en", "location": "null", "create_at": date("2012-02-26"), "description": "Snapchat ~ Garnikkkk", "followers_count": 329, "friends_count": 165, "statues_count": 23452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727178805977088, "text": "We're #hiring! Read about our latest #job opening here: Sr Windows System Admin - https://t.co/4GI1jVAA93 #Burlington, MA #Clerical", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.1956205,42.5047161"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "job", "Burlington", "Clerical" }}, "user": { "id": 3011627064, "name": "Lahey Careers", "screen_name": "LaheyCareers", "lang": "en", "location": "null", "create_at": date("2015-02-06"), "description": "null", "followers_count": 44, "friends_count": 32, "statues_count": 411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlington, MA", "id": "39ad4ce00a983b1c", "name": "Burlington", "place_type": "city", "bounding_box": rectangle("-71.240602,42.46624 -71.16858,42.544829") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2509875, "cityName": "Burlington" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727178881470465, "text": "20����������������������������������������������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 487882467, "name": "Birthday Guy", "screen_name": "_Chris_Ransom", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-02-09"), "description": "A Follower of Christ | Brittany Woods❤", "followers_count": 727, "friends_count": 261, "statues_count": 40862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727178935980034, "text": "Im never good enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1054639735, "name": "Cayla with a C", "screen_name": "Caylaalynn", "lang": "en", "location": "Wonderland", "create_at": date("2013-01-02"), "description": "spiritual gyspy kandi queen // Boheminanism // mood: A$AP// #headbanginhippie", "followers_count": 643, "friends_count": 313, "statues_count": 11403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mankato, MN", "id": "a2b439a23220cb96", "name": "Mankato", "place_type": "city", "bounding_box": rectangle("-94.06457,44.119612 -93.933999,44.210668") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27013, "countyName": "Blue Earth", "cityID": 2739878, "cityName": "Mankato" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727179334488066, "text": "I know I was sad about that, I wish he would have just left so he could come back for a visit!! https://t.co/nviqVZ2DcD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2842011066, "name": "Danielle", "screen_name": "Liv_Noah_Benson", "lang": "en", "location": "Leavenworth, KS", "create_at": date("2014-10-05"), "description": "Love SVU, Love Mariska Hargitay, ❤️Joyful Heart Foundation, @TheJHF #NoMore, @NOMOREorg #EndTheBacklog #SVUDiehard Follow me on Instagram, @joyfultobenoahsmommy", "followers_count": 1488, "friends_count": 1433, "statues_count": 53776 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Leavenworth, KS", "id": "70d4570609cf6e35", "name": "Leavenworth", "place_type": "city", "bounding_box": rectangle("-94.965076,39.266719 -94.89041,39.375576") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20103, "countyName": "Leavenworth", "cityID": 2039000, "cityName": "Leavenworth" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727179783274497, "text": "Happy birthday baby @lowkeygenesis ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2833351105 }}, "user": { "id": 1952910074, "name": "*", "screen_name": "Mvryssv", "lang": "en", "location": "dont @ me ", "create_at": date("2013-10-10"), "description": "null", "followers_count": 1360, "friends_count": 242, "statues_count": 39152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727179867164673, "text": "Me & presh patrolling kahului ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 1499249294, "name": "graccyfisher", "screen_name": "FisherGraccy", "lang": "en", "location": "null", "create_at": date("2013-06-10"), "description": "W€€Z¥zRYDAH", "followers_count": 449, "friends_count": 357, "statues_count": 3369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kahului, HI", "id": "5e2c83e1fb041c0c", "name": "Kahului", "place_type": "city", "bounding_box": rectangle("-156.500605,20.848879 -156.451278,20.90097") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1522700, "cityName": "Kahului" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727179917459456, "text": "I thought so . https://t.co/9VXQxugQq2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355639114, "name": "️", "screen_name": "StomachKillerKJ", "lang": "en", "location": "Lighthouse Point, FL", "create_at": date("2011-08-15"), "description": "99.9% of MyTweets Are Lyrics So Fuck Your Feelings : Haitian Arab:American : sc:StomachKillerKJ", "followers_count": 7944, "friends_count": 359, "statues_count": 81536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Okeechobee, FL", "id": "0065dba65a5088d8", "name": "Okeechobee", "place_type": "city", "bounding_box": rectangle("-80.907965,27.180825 -80.725729,27.279804") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12093, "countyName": "Okeechobee", "cityID": 1251200, "cityName": "Okeechobee" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727180320112641, "text": "@Im_Pilot01 �� ⭐ ��⭐��⭐��", "in_reply_to_status": -1, "in_reply_to_user": 2813550639, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2813550639 }}, "user": { "id": 309649774, "name": "Moon Worship", "screen_name": "BlueMoon2016", "lang": "en", "location": "Boston, MA The United States", "create_at": date("2011-06-02"), "description": "Which is longer, forever or eternity?", "followers_count": 4726, "friends_count": 4730, "statues_count": 3837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Allston - Brighton, Boston", "id": "02502e6cd6a0ac43", "name": "Allston - Brighton", "place_type": "neighborhood", "bounding_box": rectangle("-71.174887,42.330077 -71.110646,42.373798") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181175742464, "text": "I ❤ you like a fat kid loves ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2199980604, "name": "Cannoli King", "screen_name": "Wasabi_Papi", "lang": "en", "location": "08028", "create_at": date("2013-11-17"), "description": "null", "followers_count": 82, "friends_count": 70, "statues_count": 7307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glassboro, NJ", "id": "55352b8c5e28010f", "name": "Glassboro", "place_type": "city", "bounding_box": rectangle("-75.168425,39.679348 -75.081224,39.73751") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34015, "countyName": "Gloucester", "cityID": 3426340, "cityName": "Glassboro" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181209341952, "text": "Y'all don't understand sis fr the goat .. The shit we Be doing ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2622043713, "name": "stephanie", "screen_name": "steph_stephss", "lang": "en", "location": "null", "create_at": date("2014-06-18"), "description": "God got me", "followers_count": 1240, "friends_count": 587, "statues_count": 25393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181242859520, "text": "@RacksOnZach but you said its discerning if its God's will for two people to be together. You're not gonna start dating before doing that..", "in_reply_to_status": 735726540718120960, "in_reply_to_user": 251258410, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251258410 }}, "user": { "id": 276242832, "name": "kailyn ranae☯", "screen_name": "KailynRanae", "lang": "en", "location": "Denton, TX", "create_at": date("2011-04-02"), "description": "#UNT • Math Major/Tutor • Pageant Queen • PBSO • BSU • BSE Facilitator • MacArthur Alum • just a vocalist with a dollar and a dream.• I LOVE JESUS | RIP Tews❤️", "followers_count": 1869, "friends_count": 1682, "statues_count": 17830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181599367169, "text": "Members of #Filter + therealferlazz slowing it down with a smooth ballad at #SoundcheckLive ������… https://t.co/agMHdjQRCL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.33896263,34.10220324"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Filter", "SoundcheckLive" }}, "user": { "id": 1702975410, "name": "Lucky Strike Live", "screen_name": "LuckyStrikeLive", "lang": "en", "location": "Hollywood, CA", "create_at": date("2013-08-26"), "description": "Bringing the best in local bands, label showcases and more. Life's too short to live it on the weekends. Ph: (323) 467-7776 // info@bowlluckystrike.com", "followers_count": 29554, "friends_count": 20314, "statues_count": 2274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181809082369, "text": "@_daniella2525 it was technically still the 25th when I tweeted that!", "in_reply_to_status": 735727036791033857, "in_reply_to_user": 489984133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 191193096 }}, "user": { "id": 489984133, "name": "Sharon Martinez", "screen_name": "Share_run", "lang": "en", "location": "null", "create_at": date("2012-02-11"), "description": "God is good all the time\n All the time God is good\n SJSU 2020'", "followers_count": 914, "friends_count": 364, "statues_count": 25962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakley, CA", "id": "010781586e4d76f9", "name": "Oakley", "place_type": "city", "bounding_box": rectangle("-121.755749,37.96841 -121.62463,38.019615") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 653070, "cityName": "Oakley" } }
+{ "create_at": datetime("2016-05-26T00:00:03.000Z"), "id": 735727181913939970, "text": "@bbgigi_ shut up ��", "in_reply_to_status": 735714545985949697, "in_reply_to_user": 1584141114, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1584141114 }}, "user": { "id": 3013729616, "name": "TPC underrated", "screen_name": "ya_boi_ben_15", "lang": "en", "location": "null", "create_at": date("2015-02-08"), "description": "vigilance", "followers_count": 674, "friends_count": 503, "statues_count": 19979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727182077513728, "text": "@hunterpago55 awesome game!", "in_reply_to_status": 735716375172239360, "in_reply_to_user": 907377643, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 907377643 }}, "user": { "id": 3063272702, "name": "⋅JORDAN CHARGOIS⋅", "screen_name": "Jordanshag", "lang": "en", "location": "Sulphur, LA", "create_at": date("2015-03-05"), "description": "snap chat: jordanshagg", "followers_count": 354, "friends_count": 526, "statues_count": 1899 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sulphur, LA", "id": "253b154805ec7fbb", "name": "Sulphur", "place_type": "city", "bounding_box": rectangle("-93.427304,30.183555 -93.257523,30.277601") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2273640, "cityName": "Sulphur" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727182710874112, "text": "Wind 8.0 mph SE. Barometer 29.873 in, Falling. Temperature 75.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727182794743809, "text": "A rewarding end to 'Supernatural' Season 11 proves it can be worth it to stick with a show https://t.co/UjBlsrfbpw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17658, "friends_count": 17809, "statues_count": 72319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727183012892672, "text": "Tonight really was the cherry on top of my terrible week ����Fml", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 624286898, "name": "Amalia Alduenda", "screen_name": "ConnieLov3", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-07-01"), "description": "kinda mellow down kinda like a drama queen , they call me Rapunzel and Disney bitch in the streets :p", "followers_count": 152, "friends_count": 460, "statues_count": 937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727183532949505, "text": "Wind 0.7 mph N. Barometer 30.08 in, Rising slowly. Temperature 61.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 122, "statues_count": 160065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727184082395136, "text": "Everything happens for a reason.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2268386126, "name": "danielle salinas", "screen_name": "danielle_ssss", "lang": "en", "location": "null", "create_at": date("2013-12-29"), "description": "heart of gold.", "followers_count": 350, "friends_count": 292, "statues_count": 6630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727184136921088, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.096 in, Steady. Temp 66.4F. Rain today 0.00 in. Humidity 72% | https://t.co/jrBhN6IRrK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 623, "friends_count": 835, "statues_count": 42993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727184422178816, "text": "@aaleidan93 �������� هلااا بالزييين كله هلا بحيااتي ❤️❤️❤️", "in_reply_to_status": 735725897634832386, "in_reply_to_user": 275931021, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user_mentions": {{ 275931021 }}, "user": { "id": 317837490, "name": "دلال اسبيــــته", "screen_name": "Dalal_Esbaitah", "lang": "en", "location": "kuwait ", "create_at": date("2011-06-15"), "description": "Co boulder ♥", "followers_count": 766, "friends_count": 588, "statues_count": 10940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boulder, CO", "id": "fd70c22040963ac7", "name": "Boulder", "place_type": "city", "bounding_box": rectangle("-105.301776,39.953552 -105.183597,40.094411") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 807850, "cityName": "Boulder" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727184820609024, "text": "See our latest #Rockford, IL #job and click to apply: Stepdown ICU - https://t.co/tE1Ky0cXSO #TravelNurse #Nursing https://t.co/pP8AYblokh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.0939952,42.2711311"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rockford", "job", "TravelNurse", "Nursing" }}, "user": { "id": 234960361, "name": "TravelNursesPHP", "screen_name": "TravelNurseWork", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-01-06"), "description": "Premier Healthcare Professionals is an industry leader in domestic & international healthcare staffing. We have a client base of thousands of hospitals.", "followers_count": 4595, "friends_count": 4817, "statues_count": 8385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727184988372993, "text": "Cuz right now it's bunddie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 92130817, "name": "Were_is_Willie", "screen_name": "1OfficialJunya", "lang": "en", "location": "251 ✈ 334", "create_at": date("2009-11-23"), "description": "-good vibes only listen to my music at the link below !! Matthew 5:38", "followers_count": 3015, "friends_count": 1462, "statues_count": 36475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, AL", "id": "31cc87eb4d275bb6", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-86.018131,31.755197 -85.930803,31.857919") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1109, "countyName": "Pike", "cityID": 176920, "cityName": "Troy" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727185059680256, "text": "إلاّ أنت لا تمُوت في قَلبي إنمَا سَتعيش للأبَد .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 424194763, "name": "مشاري الذبياني", "screen_name": "mishooo1411", "lang": "en", "location": "Mississippi, USA", "create_at": date("2011-11-29"), "description": "حب الاهلي يجمعنا ♡' ɑnsт: Mishooo1411", "followers_count": 856, "friends_count": 270, "statues_count": 6665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hattiesburg, MS", "id": "2b34df148a211c3e", "name": "Hattiesburg", "place_type": "city", "bounding_box": rectangle("-89.427669,31.241189 -89.248409,31.380085") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28035, "countyName": "Forrest", "cityID": 2831020, "cityName": "Hattiesburg" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727185089073158, "text": "When someone say they don't fuck with you, but be one of the first people to view your snapchat story��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2334302012, "name": "MAY31$T GS9", "screen_name": "kingVino21", "lang": "en", "location": "Sacramento✈️LA", "create_at": date("2014-02-08"), "description": "MGNT:@calthomas21@gmail.com (Investor/Artist/Writer/youtuber/) Get my Single Selling Dreams on itunes or google play link below", "followers_count": 13346, "friends_count": 3106, "statues_count": 3724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727185172910084, "text": "@stayxkreepy GO TO TOXIC I'M GOING GO GO GO ITS ON MY BIRTHDAY TOO", "in_reply_to_status": 735727093011472384, "in_reply_to_user": 1278581190, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1278581190 }}, "user": { "id": 2295608946, "name": "queen kitty ฅ^•ﻌ•^ฅ", "screen_name": "xomairaa", "lang": "en", "location": "Chicago, IL", "create_at": date("2014-01-16"), "description": "I like getting shwifty to filthy wubz n funk", "followers_count": 2573, "friends_count": 808, "statues_count": 31313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727185378451456, "text": "Congratulations to North Pointe's graduating class of 2016!! So… https://t.co/6pBjAajswT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.26366,33.53244"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2299349083, "name": "Sierra", "screen_name": "sierrabearr", "lang": "en", "location": "null", "create_at": date("2014-01-19"), "description": "trust in the master plan", "followers_count": 337, "friends_count": 120, "statues_count": 4588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arizona, USA", "id": "a612c69b44b2e5da", "name": "Arizona", "place_type": "admin", "bounding_box": rectangle("-114.818269,31.332246 -109.045153,37.004261") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-05-26T00:00:04.000Z"), "id": 735727185995014144, "text": "my favorite commercial is lil wayne pouring champagne on a phone #what", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "what" }}, "user": { "id": 230239135, "name": "sean eckhardt", "screen_name": "sean_eckhardt", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-12-24"), "description": "editorial fellow @TakePart/@Participant formerly @THR @EyeMagazine • drought tolerant • views my own →", "followers_count": 420, "friends_count": 444, "statues_count": 2956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727187207196672, "text": "On the wheels of steel my spirit flys away Ahh Yeea @IAmKRSOne @wizkhalifa \n@bigTwestisbest \n❤️������❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 105675946, 20322929, 1870494530 }}, "user": { "id": 1870494530, "name": "Thamir Al Marayati", "screen_name": "bigTwestisbest", "lang": "en", "location": "null", "create_at": date("2013-09-15"), "description": "#Mxblife #Most #Gifted", "followers_count": 862, "friends_count": 1592, "statues_count": 16111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mirada, CA", "id": "4db990e393c2e28b", "name": "La Mirada", "place_type": "city", "bounding_box": rectangle("-118.037975,33.873395 -117.976352,33.928407") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640032, "cityName": "La Mirada" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727187387518976, "text": "Do I feel stupid for saying the Raptors were just happy to be here....Yes they're out here crushing dreams", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 273802033, "name": "Ashton Jeanlewis", "screen_name": "theashtonj", "lang": "en", "location": "Friendzone, Texas", "create_at": date("2011-03-28"), "description": "I answer all questions Host of the Fort Bend Playbook on 106.1fm / Yahoo Sports Radio Producer & Chasing the Eternal Summer #TSU", "followers_count": 567, "friends_count": 1314, "statues_count": 19924 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727187806932992, "text": "I need someone that will stay up with me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1642263692, "name": "vincentmac⁶", "screen_name": "omelly12", "lang": "en", "location": "Tupelo,MS", "create_at": date("2013-08-03"), "description": "Black is strong. a golf player. ⁶", "followers_count": 602, "friends_count": 85, "statues_count": 2905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tupelo, MS", "id": "894f2ba528ba1c7d", "name": "Tupelo", "place_type": "city", "bounding_box": rectangle("-88.800213,34.203564 -88.656629,34.324583") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28081, "countyName": "Lee", "cityID": 2874840, "cityName": "Tupelo" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727187957972993, "text": "We're #hiring! Click to apply: Software Engineer (Java) - https://t.co/8pmadWQVIX #Job #infosec #GreenwoodVillage, CO #Jobs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.9508141,39.6172101"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "hiring", "Job", "infosec", "GreenwoodVillage", "Jobs" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 338, "friends_count": 217, "statues_count": 608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenwood Village, CO", "id": "a4d5a00cbd823818", "name": "Greenwood Village", "place_type": "city", "bounding_box": rectangle("-104.981057,39.587505 -104.866586,39.653059") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 833035, "cityName": "Greenwood Village" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727188348002306, "text": "There is something living in my ceiling. It might be a demon it might be a squirrel, either way I'm not sleeping in there ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 498432849, "name": "Erica Baker", "screen_name": "EricaBaker97", "lang": "en", "location": "Escanaba to Stevens Point", "create_at": date("2012-02-20"), "description": "Today is never too late to be brand new ❤ ••••••••••••••UWSP 2019•••••••••••••• ••Majoring in napping future trophy wife••", "followers_count": 792, "friends_count": 419, "statues_count": 14556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Escanaba, MI", "id": "b9325aac26247f5d", "name": "Escanaba", "place_type": "city", "bounding_box": rectangle("-87.135317,45.708432 -87.037253,45.801412") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26041, "countyName": "Delta", "cityID": 2626360, "cityName": "Escanaba" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727188352212994, "text": "quiero que te dejes querer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2269708424, "name": "Chuchis♛", "screen_name": "_cordon__", "lang": "en", "location": "Bay Area , California", "create_at": date("2013-12-30"), "description": "Salvadoreña Guatemalteca", "followers_count": 422, "friends_count": 397, "statues_count": 8086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antioch, CA", "id": "9b101e0451f073b6", "name": "Antioch", "place_type": "city", "bounding_box": rectangle("-121.860664,37.945647 -121.732224,38.025674") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 602252, "cityName": "Antioch" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727188461264896, "text": "Wind 0.0 mph ---. Barometer 30.13 in, Falling slowly. Temperature 71.1 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727188532592640, "text": "@FriasMyrian should I just go nakey or what", "in_reply_to_status": 735726959720701952, "in_reply_to_user": 1466200376, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1466200376 }}, "user": { "id": 1691659064, "name": "esme ♡", "screen_name": "esmetrakalosaa", "lang": "en", "location": "null", "create_at": date("2013-08-22"), "description": "♡", "followers_count": 436, "friends_count": 379, "statues_count": 17538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gresham, OR", "id": "7bf7dcb9504c91c9", "name": "Gresham", "place_type": "city", "bounding_box": rectangle("-122.498909,45.460886 -122.367482,45.559395") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4131250, "cityName": "Gresham" } }
+{ "create_at": datetime("2016-05-26T00:00:05.000Z"), "id": 735727188536791045, "text": "I'm about to fuck a bitch up for you rn �� @shayla_kristine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 170036699 }}, "user": { "id": 370447806, "name": "PatriciaSamantha♥️", "screen_name": "PxS_Orozco", "lang": "en", "location": "Waipahu, HI ", "create_at": date("2011-09-08"), "description": "• I probably don't care • ⚽️❣", "followers_count": 199, "friends_count": 167, "statues_count": 3881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearl City, HI", "id": "6ded3cbcea7e1f34", "name": "Pearl City", "place_type": "city", "bounding_box": rectangle("-157.992788,21.369586 -157.934172,21.431996") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1562600, "cityName": "Pearl City" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089553325281281, "text": "@yaduuura dang you right you right", "in_reply_to_status": 736089462510256128, "in_reply_to_user": 313582048, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313582048 }}, "user": { "id": 389834272, "name": "SAVAGE", "screen_name": "JordanEmily12", "lang": "en", "location": "Modesto, CA", "create_at": date("2011-10-12"), "description": "null", "followers_count": 144, "friends_count": 121, "statues_count": 7431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089554168356865, "text": "Never text @abwalkher past 1am bc she'll pass out like a loser", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 865499041 }}, "user": { "id": 1220219467, "name": "RobertCaesarSeredich", "screen_name": "caesarjozwiak", "lang": "en", "location": "HTX➡️KCMO", "create_at": date("2013-02-25"), "description": "Blue Springs graduate", "followers_count": 469, "friends_count": 396, "statues_count": 5591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, MO", "id": "04b4aca917b0103d", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-94.487114,39.01759 -94.269551,39.158419") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2935000, "cityName": "Independence" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089554327736323, "text": "@Whatthefuck_ they need to make it to where you can't have two of the same character on a team. 2 reinharts or 2 roadhogs is OP.", "in_reply_to_status": 736081122925252608, "in_reply_to_user": 18109520, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18109520 }}, "user": { "id": 297705052, "name": "Hick", "screen_name": "AfloatHickory", "lang": "en", "location": "AZ ✈️ Everywhere", "create_at": date("2011-05-12"), "description": "Wanheda. @DebnamCarey is always my Heda", "followers_count": 517, "friends_count": 558, "statues_count": 102140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089554814259200, "text": "My head been killing me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 336917903, "name": "Jessica", "screen_name": "Prettyblack_jai", "lang": "en", "location": "null", "create_at": date("2011-07-16"), "description": "Blessed ❤Jaida's Mom Dec 15th", "followers_count": 1429, "friends_count": 1132, "statues_count": 53308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abita Springs, LA", "id": "006cb77253c2c656", "name": "Abita Springs", "place_type": "city", "bounding_box": rectangle("-90.091235,30.434532 -90.019439,30.504184") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2200240, "cityName": "Abita Springs" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089555791532033, "text": "© 2013 All Rights Reserved as Distributed by Vehlanzzi Madeleine #Photography #Landscape #Nature #Michigan https://t.co/0qD3iD0FJW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Photography", "Landscape", "Nature", "Michigan" }}, "user": { "id": 735247309160775688, "name": "Vehlanzzi Madeleine", "screen_name": "VehlanzziM", "lang": "en", "location": "Flint, MI", "create_at": date("2016-05-24"), "description": "20 year old musician, photographer, and bake shop co-owner. Current Status: Entrepreneur.", "followers_count": 1, "friends_count": 1, "statues_count": 21 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flint, MI", "id": "0138153149b79c7f", "name": "Flint", "place_type": "city", "bounding_box": rectangle("-83.831237,42.899436 -83.619983,43.089481") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2629000, "cityName": "Flint" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089555942543360, "text": "Ant trying to catch this fade at 2 am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2895148685, "name": "tori lynn", "screen_name": "puratorita_", "lang": "en", "location": "null", "create_at": date("2014-11-27"), "description": "my mama ain't raise no pendeja", "followers_count": 352, "friends_count": 156, "statues_count": 5641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556307443712, "text": "@HillaryClinton @realDonaldTrump call ppl all kinda names. Clearly none want to be low! Trump looks like a meerkat! https://t.co/0yzzkOvPqR", "in_reply_to_status": -1, "in_reply_to_user": 1339835893, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1339835893, 25073877 }}, "user": { "id": 609308474, "name": "Aminata", "screen_name": "AminataWanea1", "lang": "en", "location": "Connecticut, US", "create_at": date("2012-06-15"), "description": "null", "followers_count": 105, "friends_count": 155, "statues_count": 271 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bismarck, ND", "id": "3cd12646c811c87e", "name": "Bismarck", "place_type": "city", "bounding_box": rectangle("-100.839587,46.756481 -100.68871,46.903326") }, "geo_tag": { "stateID": 38, "stateName": "North Dakota", "countyID": 38015, "countyName": "Burleigh", "cityID": 3807200, "cityName": "Bismarck" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556349423617, "text": "Now im out ✌��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1374091196, "name": "Comatose", "screen_name": "sirdibi", "lang": "en", "location": "Dallas, TX", "create_at": date("2013-04-23"), "description": "21. North DTX \nSnapchat: Zuluboi5", "followers_count": 247, "friends_count": 204, "statues_count": 18146 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556546519040, "text": "I lose/gain weight so fucking easy! One day I'm 130-35 & another day I'm 179-305, like wth?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393680822, "name": "Steph", "screen_name": "steph_the_loser", "lang": "en", "location": "null", "create_at": date("2011-10-18"), "description": "null", "followers_count": 340, "friends_count": 291, "statues_count": 14334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556592693250, "text": "I want a milkshake https://t.co/ctP984Syu9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30396079, "name": "Kim", "screen_name": "KimberlySade", "lang": "en", "location": "Louisiana, USA", "create_at": date("2009-04-10"), "description": "just get me a 3 piece white from Popeyes", "followers_count": 1548, "friends_count": 917, "statues_count": 196702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556596883456, "text": "@SnarkyWalker @crystaljuarez94 its lit fam! Lol", "in_reply_to_status": 736089361909846018, "in_reply_to_user": 1158121848, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1158121848, 1456066184 }}, "user": { "id": 493850407, "name": "Blossom", "screen_name": "mayrah07", "lang": "en", "location": "City of Angels ", "create_at": date("2012-02-16"), "description": "California Lovin. Eat. Sleep. Rave. Repeat. \nMy religion is Kindness.\nUCR Alum\nΛΘΑ ΑΗ L4 #1", "followers_count": 202, "friends_count": 367, "statues_count": 3099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556705906689, "text": "@P_Anderson11 everybody must be drunk and asleep ?", "in_reply_to_status": 736089363646320640, "in_reply_to_user": 3601072392, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3601072392 }}, "user": { "id": 792686642, "name": "iGetFriendZoned", "screen_name": "Joey_Eiland", "lang": "en", "location": "San Tan Valley, AZ", "create_at": date("2012-08-30"), "description": "LIFE CRAZY #DLeagueTwitter (deep in the bench) IF YOU WANT TO POST A BLOG PLEASE CONTACT Scatteredthoughtsdl@gmail.com", "followers_count": 1496, "friends_count": 962, "statues_count": 67203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-05-27T00:00:00.000Z"), "id": 736089556773003264, "text": "@jimmiegotsoul naw kneegrow you asked me directly", "in_reply_to_status": 736089463240069120, "in_reply_to_user": 2380182619, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2380182619 }}, "user": { "id": 4183787412, "name": "GentlemanRatchet", "screen_name": "CurryCoolAss", "lang": "en", "location": "null", "create_at": date("2015-11-09"), "description": "#Sensational #iWasDehydrated", "followers_count": 364, "friends_count": 351, "statues_count": 13429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089557515395073, "text": "How tf you not gunna let myself explain myself that's a L right there", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 596712791, "name": "PETER", "screen_name": "PeterElizalde", "lang": "en", "location": "null", "create_at": date("2012-06-01"), "description": "null", "followers_count": 282, "friends_count": 160, "statues_count": 11471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089557670600705, "text": "I know you want this for life", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418130315, "name": "Eva", "screen_name": "MakeupForevaa", "lang": "en", "location": "Nostalgia ▲ Ultra", "create_at": date("2011-11-21"), "description": "can i get this in gold? L.T.S.G.", "followers_count": 939, "friends_count": 868, "statues_count": 55528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558052265984, "text": "@ebbtideapp Tide in Delaware City, Delaware 05/27/2016\nHigh 3:16am 6.1\n Low 10:46am 0.4\nHigh 4:05pm 5.3\n Low 10:42pm 0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.5883,39.5817"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 97, "friends_count": 1, "statues_count": 36233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle", "cityID": 1019730, "cityName": "Delaware City" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558362689536, "text": "happy birthday pretty boi������ @jaulon_clavo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3362072818 }}, "user": { "id": 4053007334, "name": "mayca❤️", "screen_name": "jttol5", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-10-28"), "description": "TEAM JJ MACKIN !!❣", "followers_count": 329, "friends_count": 252, "statues_count": 4579 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558547206149, "text": "@Bballing_Joel @elenaxoanaya Joel you fell in the river ��������", "in_reply_to_status": 736089407774564353, "in_reply_to_user": 297154682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 297154682, 3023322193 }}, "user": { "id": 2821012068, "name": "Dania", "screen_name": "Daniuuuh_", "lang": "en", "location": "Salinas, CA", "create_at": date("2014-09-19"), "description": "null", "followers_count": 209, "friends_count": 194, "statues_count": 6162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558563987456, "text": "Wind 1.5 mph WSW. Barometer 29.892 in, Falling. Temperature 74.4 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558740144129, "text": "Temp 61.5° Hi/Lo 64.5/61.3 Rng 3.2° WC 61.5° Hmd 92% Rain 0.00\" Storm 0.00\" BAR 30.114 Falling DP 59.2° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 69, "friends_count": 123, "statues_count": 19020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558769537025, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.063 in, Steady. Temp 68.0F. Rain today 0.00 in. Humidity 85% | https://t.co/jrBhN6IRrK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 622, "friends_count": 835, "statues_count": 43047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089558803046401, "text": "Im evil ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622149890, "name": "Katlynn Jenner", "screen_name": "kaatlynnn_", "lang": "en", "location": "Norwalk, CA", "create_at": date("2012-06-29"), "description": "18", "followers_count": 726, "friends_count": 504, "statues_count": 13561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559172145152, "text": "Ripley SW Limestone Co. Temp: 72.3°F Wind:1.6mph Pressure: 995.5mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 53798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559184773120, "text": "The thottery at bdubs earlier was at an all time high �� #oomf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "oomf" }}, "user": { "id": 3238931120, "name": "penguins in game 5", "screen_name": "stz_mke", "lang": "en", "location": "South Milwaukee, WI", "create_at": date("2015-06-07"), "description": "6/5 @analisettes ❤️", "followers_count": 256, "friends_count": 132, "statues_count": 16991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559436386304, "text": "of course. he's fucked all kinds of women to know this. https://t.co/vKVz76cvrT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389190520, "name": "David Lee Uchiha", "screen_name": "DaveTheAssEater", "lang": "en", "location": "Always In My Bed", "create_at": date("2011-10-11"), "description": "back to eating ass. done started my business Ass Eating Inc. serious inquires hit my DM. I'm still ugly as shit tho.", "followers_count": 2232, "friends_count": 1917, "statues_count": 318543 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559641919491, "text": "Who wants to go to delfest with me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2540636809, "name": "Sav", "screen_name": "savypoo", "lang": "en", "location": "null", "create_at": date("2014-06-01"), "description": "live by the sun • love by the moon♎️", "followers_count": 273, "friends_count": 269, "statues_count": 2434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laurel, VA", "id": "006654ca2b4e27c5", "name": "Laurel", "place_type": "city", "bounding_box": rectangle("-77.569305,37.606827 -77.475442,37.680986") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51087, "countyName": "Henrico", "cityID": 5144280, "cityName": "Laurel" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559696433152, "text": "tan sucio coño https://t.co/mCXwMThjCz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3330744161, "name": "menor", "screen_name": "hepbvrnn", "lang": "en", "location": "New York, NY", "create_at": date("2015-06-16"), "description": "Instagram: hepbvrn | Snapchat: leshaeyo | rest easy mama, i love you.", "followers_count": 3861, "friends_count": 1461, "statues_count": 15180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089559876788229, "text": "Wo liegt Denver? https://t.co/C8gd1UMVYL #Denver #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-104.988,39.7551"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Denver", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 484, "friends_count": 115, "statues_count": 1948399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560061378560, "text": "@2chainbee I regret it because the person I did it with sucked. Like they fucken sucked.", "in_reply_to_status": 736089375201595393, "in_reply_to_user": 1589852766, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1589852766 }}, "user": { "id": 803938812, "name": "King Kardo", "screen_name": "EvilMonster831", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2012-09-04"), "description": "Versace thought me. Gucci dressed me. Prada lead me. Givinchy raised me. and Fashion is who I become.", "followers_count": 427, "friends_count": 681, "statues_count": 8715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560111665157, "text": "It just hit me that a very important and highly loved friend, no best-friend of mine is leaving me at the end of the month.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417438834, "name": "Dario", "screen_name": "davelar93", "lang": "en", "location": "Texas", "create_at": date("2011-11-20"), "description": "CA✈️TX", "followers_count": 189, "friends_count": 126, "statues_count": 10804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, TX", "id": "017b954535d7a86f", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-98.351813,29.567015 -98.290713,29.614615") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4866704, "cityName": "Selma" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560443019264, "text": "@mplacko I played it as soon as it came out!", "in_reply_to_status": 736045710949240833, "in_reply_to_user": 5915672, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 5915672 }}, "user": { "id": 718352029, "name": "Jordan Tayer", "screen_name": "SomaJT", "lang": "en", "location": "San Francisco, CA", "create_at": date("2012-07-26"), "description": "Influencer Relations Manager @Twitch", "followers_count": 14557, "friends_count": 983, "statues_count": 9382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560455610370, "text": "My snapchat is mainly my dog or Eli. You don't like it? Haha don't look at my stories ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 470815156, "name": "michelle", "screen_name": "michelle_walls7", "lang": "en", "location": "661", "create_at": date("2012-01-21"), "description": "5.6.14. ERB.IV. Momma to be July 2016.", "followers_count": 470, "friends_count": 279, "statues_count": 20880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560468201472, "text": "no thanks https://t.co/f3gwEAhPBA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2236699978, "name": "la flare", "screen_name": "mailala_", "lang": "en", "location": "null", "create_at": date("2013-12-21"), "description": "fuckers in school telling me, always in the barbershop chief keef ain't bout this, chief keef ain't bout that", "followers_count": 1035, "friends_count": 141, "statues_count": 196029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089560581443585, "text": "Best part of graduating this spring: The only finals I have to worry about are #StanleyCup Finals! #SharksTerritory #SJSU16 (but really '15)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "StanleyCup", "SharksTerritory", "SJSU16" }}, "user": { "id": 289777188, "name": "Elizabeth Barcelos", "screen_name": "lavender_ink", "lang": "en", "location": "San Jose", "create_at": date("2011-04-28"), "description": "Portugal and quidditch and the occasional thoughts not safe for Facebook.", "followers_count": 224, "friends_count": 1181, "statues_count": 1797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-27T00:00:01.000Z"), "id": 736089561374154753, "text": "@ThickMaster @The_Follower666 hey dark chocolate)", "in_reply_to_status": 735165335511420928, "in_reply_to_user": 271627027, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 271627027, 115856554 }}, "user": { "id": 2388547934, "name": "Jashawn", "screen_name": "Jashawn40158376", "lang": "en", "location": "null", "create_at": date("2014-03-13"), "description": "sex", "followers_count": 633, "friends_count": 2942, "statues_count": 1906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lawndale, CA", "id": "cce33d74ceffbe08", "name": "Lawndale", "place_type": "city", "bounding_box": rectangle("-118.369186,33.872914 -118.343796,33.902665") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640886, "cityName": "Lawndale" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089561575497729, "text": "Wind 2.0 mph S. Barometer 1011.08 mb, Rising. Temperature 67.9 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089561625821184, "text": "05/27@03:00 - Temp 66.0F, WC 66.0F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.067in, Rising slowly. Rain 0.00in. Hum 93%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089561810403329, "text": "The feeling of taking on players & still giving a beautiful pass through defenders is unreal! ⚽️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2172648210, "name": "EL L", "screen_name": "patonjr96", "lang": "en", "location": "null", "create_at": date("2013-11-03"), "description": "Is this the Krusty Krab?!", "followers_count": 551, "friends_count": 493, "statues_count": 23712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089562045251584, "text": "@TheyLoveLins good looking out shawty", "in_reply_to_status": 736089499214610433, "in_reply_to_user": 367533099, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 367533099 }}, "user": { "id": 316356077, "name": "Vinci", "screen_name": "Youngandtatted", "lang": "en", "location": "New Ratchet , LA", "create_at": date("2011-06-13"), "description": "Javincijay@gmail.com | 6'10 Artist | GSU | #FreeAnna", "followers_count": 13423, "friends_count": 3195, "statues_count": 15219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089562171080704, "text": "Baby, it's 3 am I must be lonely", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 252350309, "name": "Zoya Price", "screen_name": "ZoyaPrice", "lang": "en", "location": "Hell. ", "create_at": date("2011-02-14"), "description": "8•4•15 ; we're all a little dead inside", "followers_count": 511, "friends_count": 418, "statues_count": 15903 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patton, PA", "id": "0072a64515749d22", "name": "Patton", "place_type": "city", "bounding_box": rectangle("-78.695977,40.619321 -78.638879,40.642345") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42021, "countyName": "Cambria", "cityID": 4258432, "cityName": "Patton" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089562246578176, "text": "Temp: 71.2°F Wind:0.2mph Pressure: 29.981hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 65291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089562393411587, "text": "Wind 2.0 mph SE. Barometer 29.889 in, Steady. Temperature 68.4 °F. Rain today 0.03 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 120, "friends_count": 0, "statues_count": 113518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089562967986176, "text": "@ironxbooty happy birthday you delightful gal, can't wait to see you on Saturday to have, what will surely be an amazing time together����", "in_reply_to_status": -1, "in_reply_to_user": 541199815, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 541199815 }}, "user": { "id": 727749866055675906, "name": "Max Anthony Ruiz", "screen_name": "maxruiz95", "lang": "en", "location": "Montebello, CA", "create_at": date("2016-05-03"), "description": "'hey hi hello ayo wassup' - azealia banks", "followers_count": 96, "friends_count": 120, "statues_count": 977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montebello, CA", "id": "2da132a7bfebfc0c", "name": "Montebello", "place_type": "city", "bounding_box": rectangle("-118.144082,33.974263 -118.070797,34.04611") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648816, "cityName": "Montebello" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089563244830720, "text": "@KalenTutt #TXFollowtrain ��", "in_reply_to_status": -1, "in_reply_to_user": 1486606934, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "TXFollowtrain" }}, "user_mentions": {{ 1486606934 }}, "user": { "id": 1486606934, "name": "IG : YOUNG_KALEN", "screen_name": "KalenTutt", "lang": "en", "location": "Somewhere Hoopin ", "create_at": date("2013-06-05"), "description": "FREE WOLFE ROB BLACC BILL NORMO & NUMBA 4 ‼️", "followers_count": 518, "friends_count": 274, "statues_count": 3715 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channelview, TX", "id": "eabe7b6fd4504fff", "name": "Channelview", "place_type": "city", "bounding_box": rectangle("-95.176002,29.735548 -95.061098,29.851809") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4814236, "cityName": "Channelview" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089563261632513, "text": "The what is highkey meth's song. Nigga took all the shine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 126826288, "name": "sunflower", "screen_name": "anijahboyd", "lang": "en", "location": "lv, nv ", "create_at": date("2010-03-26"), "description": "Hip-Hop, you the Love of my life.", "followers_count": 1854, "friends_count": 623, "statues_count": 59605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089564003995648, "text": "I wish there was more of a community built around advocacy for the mentally ill", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 365849270, "name": "Punk Rock Peacock", "screen_name": "_CiaraJade", "lang": "en", "location": "Most of the time I’m lost. ", "create_at": date("2011-08-31"), "description": "My mind is The Wasteland & I'm just a waste.", "followers_count": 196, "friends_count": 232, "statues_count": 15772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monongah, WV", "id": "062830256506657d", "name": "Monongah", "place_type": "city", "bounding_box": rectangle("-80.228821,39.453678 -80.208104,39.466511") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54049, "countyName": "Marion", "cityID": 5455276, "cityName": "Monongah" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089564406648834, "text": "@HeLoveNomi happy birthday Naomi�� I know that we don't personally know one another but you're so beautiful���������� and you seem really nice��������", "in_reply_to_status": -1, "in_reply_to_user": 2959313437, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2959313437 }}, "user": { "id": 865713493, "name": "Donnie Bailey", "screen_name": "iamdonbills", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-10-06"), "description": "null", "followers_count": 1183, "friends_count": 236, "statues_count": 28720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089564624740352, "text": "Wind 0.0 mph ---. Barometer 30.006 in, Rising slowly. Temperature 63.1 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 62, "friends_count": 27, "statues_count": 20124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089564708642816, "text": "@JonahNRO @corp_refugee Talk about the dumbing down of America...what are we, a bunch of \"Chicken Littles\" now?", "in_reply_to_status": 735785993916297216, "in_reply_to_user": 71627462, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71627462, 722450702161682433 }}, "user": { "id": 4208704754, "name": "Natalie Case", "screen_name": "SeattleNatt", "lang": "en", "location": "null", "create_at": date("2015-11-11"), "description": "null", "followers_count": 42, "friends_count": 126, "statues_count": 449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redmond, WA", "id": "7291a25672e0d4b1", "name": "Redmond", "place_type": "city", "bounding_box": rectangle("-122.1649,47.626845 -121.958642,47.73078") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5357535, "cityName": "Redmond" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089564809302016, "text": "Riding in Alaska! #motorcycles #Alaska #rideordie @ Portage Glacier https://t.co/JYXTLgld0X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-148.78555556,60.75305556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "motorcycles", "Alaska", "rideordie" }}, "user": { "id": 16800231, "name": "Big Dave Grizzly", "screen_name": "BigDaveGrizzly", "lang": "en", "location": "iPhone: 61.181793,-149.814957", "create_at": date("2008-10-15"), "description": "Proud member of Alaska Tweets. A computer geek trapped in a behemoth form. No... Not really. More of a biker who figured out computers.", "followers_count": 1048, "friends_count": 950, "statues_count": 6103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089565262290944, "text": "Csnt wait to see the king tomorrow �� �� @Mariners", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 41488578 }}, "user": { "id": 1275288210, "name": "Ty", "screen_name": "tdoub4", "lang": "en", "location": "null", "create_at": date("2013-03-17"), "description": "Family First; Meridian ID ➡️ Moses Lake WA RHP for BBCC ⚾️ PK❤", "followers_count": 439, "friends_count": 642, "statues_count": 7393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moses Lake North, WA", "id": "00cd8498587dcec3", "name": "Moses Lake North", "place_type": "city", "bounding_box": rectangle("-119.352498,47.155437 -119.287452,47.234307") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5347280, "cityName": "Moses Lake North" } }
+{ "create_at": datetime("2016-05-27T00:00:02.000Z"), "id": 736089565484580865, "text": "@Meeza__ ofn. and they got wifi everywhere", "in_reply_to_status": 736084288840441856, "in_reply_to_user": 38273917, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 38273917 }}, "user": { "id": 356883460, "name": "boolin", "screen_name": "_MayaLENE", "lang": "en", "location": "HadiyaWorld | LongLiveJayelo", "create_at": date("2011-08-17"), "description": "lost in the sauce. #ISU20 ❤️", "followers_count": 2039, "friends_count": 993, "statues_count": 112299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089565715255296, "text": "No you need a cookie but I can't provide https://t.co/0F6SHpAJuE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173683362, "name": "jacquelicious", "screen_name": "JackieCM06", "lang": "en", "location": "null", "create_at": date("2010-08-01"), "description": "Filipina shawty | PH✈️AZ |", "followers_count": 1846, "friends_count": 1040, "statues_count": 39920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089566096953346, "text": "@BoogDaFinesser happy birthday bro����", "in_reply_to_status": -1, "in_reply_to_user": 4097453532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4097453532 }}, "user": { "id": 613968242, "name": "LOWKEY 〽️ASTERMIND™", "screen_name": "jhoops_7", "lang": "en", "location": "Houston, TX", "create_at": date("2012-06-20"), "description": "There's beauty in simplicity #OU2020", "followers_count": 572, "friends_count": 866, "statues_count": 6506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089566742863873, "text": "I fucking feel you. I'm 21 andi already feel like it's time smh https://t.co/hGaRx0JyiJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356613356, "name": "GΩCCi", "screen_name": "josegucci_", "lang": "en", "location": "Anchorage, AK", "create_at": date("2011-08-16"), "description": "Your Favorites favorite - @OMENintl", "followers_count": 614, "friends_count": 335, "statues_count": 14669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089566948429825, "text": "T-minus 4 hours and 15 minutes until I'm off then it's weekend time ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 482638541, "name": "KITTY.", "screen_name": "caitlinkolbus15", "lang": "en", "location": "Fremont, OH", "create_at": date("2012-02-03"), "description": "it’s not selfish to do what is best for you", "followers_count": 1046, "friends_count": 664, "statues_count": 27843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clyde, OH", "id": "fd71d381e107ccee", "name": "Clyde", "place_type": "city", "bounding_box": rectangle("-83.060251,41.27876 -82.951746,41.335848") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39143, "countyName": "Sandusky", "cityID": 3916308, "cityName": "Clyde" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089566973546497, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Rising slowly. Temperature 70.5 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 123, "statues_count": 160090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089567275556870, "text": "Must be nice being completely signed off ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366542674, "name": "Luz", "screen_name": "_luzzz__", "lang": "en", "location": "null", "create_at": date("2014-02-28"), "description": "Blessed.", "followers_count": 433, "friends_count": 373, "statues_count": 4617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089567552409600, "text": "Get comfortable with being uncomfortable.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 593250150, "name": "Playboy Troy", "screen_name": "Troy_Stokes15", "lang": "en", "location": "410", "create_at": date("2012-05-28"), "description": "Life can only be understood backwards, but it must be lived forward| Outfielder in the Milwaukee Brewers Organization.", "followers_count": 1360, "friends_count": 868, "statues_count": 17017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089567728566272, "text": "Rasheeda dress so nice omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 278312784, "name": "tranaeB", "screen_name": "tranaeee", "lang": "en", "location": "Louisiana", "create_at": date("2011-04-06"), "description": "SUBR✨", "followers_count": 2161, "friends_count": 601, "statues_count": 82591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Donaldsonville, LA", "id": "46fe4f949c7c201c", "name": "Donaldsonville", "place_type": "city", "bounding_box": rectangle("-91.050993,30.075457 -90.937839,30.120324") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2221240, "cityName": "Donaldsonville" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089567749537792, "text": "�������� https://t.co/V18nP1jpiP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 169225271, "name": ".", "screen_name": "shanticedj_", "lang": "en", "location": "null", "create_at": date("2010-07-21"), "description": "Communication Science & Disorders #WAYNESTATEUNIVERSITY", "followers_count": 1416, "friends_count": 981, "statues_count": 123066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568080855044, "text": "Time to grub on this tandoori chicken for that post workout", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2361599089, "name": "Mohammed Salman khan", "screen_name": "SalKhan510", "lang": "en", "location": "Hayward, CA", "create_at": date("2014-02-25"), "description": "Truefightclub|MuayThai|AviationEngineering| Dubai Dreams ✈️ Emirates future pilot snapchat: kingkhan510", "followers_count": 171, "friends_count": 129, "statues_count": 5119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568097632257, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1584540332, "name": "tam", "screen_name": "DjTammy_", "lang": "en", "location": "Bowie, TX", "create_at": date("2013-07-10"), "description": "✨", "followers_count": 281, "friends_count": 363, "statues_count": 10622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bowie, TX", "id": "fa71fd6a66022d57", "name": "Bowie", "place_type": "city", "bounding_box": rectangle("-97.869202,33.533763 -97.818437,33.590943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48337, "countyName": "Montague", "cityID": 4809640, "cityName": "Bowie" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568265412609, "text": "@HanvilleScott https://t.co/28ZtH9WmB4", "in_reply_to_status": -1, "in_reply_to_user": 4879349632, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4879349632 }}, "user": { "id": 702199438324981761, "name": "Leisa Poindexter", "screen_name": "msscottybear", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "here for me. here for you ! !! !!! MFF. 9-29-15", "followers_count": 131, "friends_count": 70, "statues_count": 10058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568286367750, "text": "baby .. ������ https://t.co/HfwLdJkzUe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2260342663, "name": "ǪυƎƎИ ʝᗩУУ ✨❤️", "screen_name": "Asia_Jayy22", "lang": "en", "location": "Nashville, AR", "create_at": date("2013-12-24"), "description": "Waiting on a sign, guess it's time for a different prayer ..", "followers_count": 802, "friends_count": 434, "statues_count": 12466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nashville, AR", "id": "92176658201f2089", "name": "Nashville", "place_type": "city", "bounding_box": rectangle("-93.878702,33.903052 -93.829025,33.965798") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5061, "countyName": "Howard", "cityID": 548560, "cityName": "Nashville" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568374460416, "text": "Talked about behind my back is if I wasn't worthy, Lord know that I perfect but I sho deserve it #MFH2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MFH2" }}, "user": { "id": 112927654, "name": "Chase", "screen_name": "ChaseThugga", "lang": "en", "location": "looking for a fuck to give ", "create_at": date("2010-02-09"), "description": "If life is a gamble I'm willing to bet my last dime. #BWA #IDGT", "followers_count": 304, "friends_count": 519, "statues_count": 3716 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muskogee, OK", "id": "2daa13876c1ef767", "name": "Muskogee", "place_type": "city", "bounding_box": rectangle("-95.442801,35.667946 -95.298037,35.797212") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40101, "countyName": "Muskogee", "cityID": 4050050, "cityName": "Muskogee" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568689029120, "text": "@realDonaldTrump hey Donald where does Hillary get her clothes from Curtains \"R\" US ?", "in_reply_to_status": 735942401165033477, "in_reply_to_user": 25073877, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 707097541968588800, "name": "Tom Corcoran", "screen_name": "zombiebaittom", "lang": "en", "location": "Wilmington, IL", "create_at": date("2016-03-07"), "description": "Entertainment Television Music Television Gaming", "followers_count": 16, "friends_count": 131, "statues_count": 37 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wilmington, IL", "id": "4aa9cd06d3774e57", "name": "Wilmington", "place_type": "city", "bounding_box": rectangle("-88.179137,41.287315 -88.119227,41.329491") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1782101, "cityName": "Wilmington" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089568932290560, "text": "My BF looks different in pictures than he does in person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 469784092, "name": "Mariah Sabrina", "screen_name": "sexy_riaaah", "lang": "en", "location": "null", "create_at": date("2012-01-20"), "description": "6 days til I get me some David❤️", "followers_count": 1437, "friends_count": 1005, "statues_count": 55162 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boronda, CA", "id": "8dee5863ff24f8ac", "name": "Boronda", "place_type": "city", "bounding_box": rectangle("-121.681145,36.685713 -121.668068,36.70477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 607578, "cityName": "Boronda" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089569280430080, "text": "stop punishing yourself for having a mental illness when what you went through made you the person you are today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2777707309, "name": "Elizabeth Cordill", "screen_name": "LizCordill", "lang": "en", "location": "Michigan", "create_at": date("2014-08-28"), "description": "promote positivity // 14, christian, vegan, swimmer, actress & film", "followers_count": 384, "friends_count": 297, "statues_count": 3124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Okemos, MI", "id": "dfae4bdb9055f07e", "name": "Okemos", "place_type": "city", "bounding_box": rectangle("-84.462435,42.678039 -84.363083,42.744463") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2660340, "cityName": "Okemos" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089569284624384, "text": "The only way I know someone name when they email me is b/c of the name on the email they be like \"how much you charge\" (sent from iPhone)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1054254266, "name": "StyledByYonnieb✨", "screen_name": "RichgirlYonnieb", "lang": "en", "location": "NewOrleans|NYC", "create_at": date("2013-01-01"), "description": "Fashion Director, Editorial Specialist & Traveling Stylist. CEO for Richgirl Lifestyle LLC Styling & Brand Boutique Info@richgirllifestyle.com", "followers_count": 1882, "friends_count": 468, "statues_count": 124945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089569288851456, "text": "can't sleep :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1011097177, "name": "lyss", "screen_name": "alyssalovelandd", "lang": "en", "location": "null", "create_at": date("2012-12-14"), "description": "♊️ | d.m.", "followers_count": 792, "friends_count": 535, "statues_count": 12236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-05-27T00:00:03.000Z"), "id": 736089569674694658, "text": "������������ https://t.co/63nfryoV2O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1338038858, "name": "Jaaaay", "screen_name": "_Janett23_", "lang": "en", "location": "null", "create_at": date("2013-04-08"), "description": "Philippians 4:13", "followers_count": 260, "friends_count": 326, "statues_count": 10904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Horizon City, TX", "id": "5092516c6da7c859", "name": "Horizon City", "place_type": "city", "bounding_box": rectangle("-106.233005,31.654053 -106.142139,31.726174") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4834832, "cityName": "Horizon City" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089569993494529, "text": "slim waist with that ass behind you , tell me where the fuck they find you ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1317853878, "name": "A.", "screen_name": "_aaangeeell", "lang": "en", "location": "null", "create_at": date("2013-03-30"), "description": "ripmother&grandmother❤️", "followers_count": 1734, "friends_count": 1491, "statues_count": 52504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gray, LA", "id": "d642cb854f0cd2bd", "name": "Gray", "place_type": "city", "bounding_box": rectangle("-90.806879,29.654149 -90.738307,29.744752") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2231180, "cityName": "Gray" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570211565570, "text": "Knowing what I know, do I personally care if someone gets murdered? Not really. People get murdered across the #world everyday.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "world" }}, "user": { "id": 2389614774, "name": "Tech News Midwest", "screen_name": "TechNewsMidwest", "lang": "en", "location": "null", "create_at": date("2014-03-14"), "description": "Bringing #Entertainment #Technology #Business #Startup news to the #Midwest @technewsbusines @technewsjobs", "followers_count": 28813, "friends_count": 8504, "statues_count": 49317 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570219954178, "text": "Just posted a photo @ Norwalk, California https://t.co/VZvaEXJ38Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.083,33.9069"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 954387553, "name": "Eddie guerrero", "screen_name": "edcruzto9", "lang": "en", "location": "Norwalk california", "create_at": date("2012-11-17"), "description": "the meaning of happiness. seeing life through my eyes.", "followers_count": 290, "friends_count": 452, "statues_count": 10711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570324807685, "text": "@_elissanicole but I still miss it there", "in_reply_to_status": 736082576754892800, "in_reply_to_user": 2728823661, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2728823661 }}, "user": { "id": 177057619, "name": "Tim Cisilino", "screen_name": "timcisilino", "lang": "en", "location": "Long Beach, CA", "create_at": date("2010-08-10"), "description": "null", "followers_count": 316, "friends_count": 556, "statues_count": 2477 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570387759104, "text": "Mook and Tae bout to get flicked in the throat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.712328,33.07442"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 326027612, "name": "Mine.", "screen_name": "HoneyysLove", "lang": "en", "location": "Dallas, TX", "create_at": date("2011-06-29"), "description": "Kitchen; Where women belong. Man runs the household, so submit to him. You CANT do what men do, unless u wanna be labeled a hoe. Part of #PickMe Twitter", "followers_count": 1844, "friends_count": 34, "statues_count": 193175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570794569728, "text": "@LadiesBeLovinMe nothin a lil music won't help lol", "in_reply_to_status": 736089415131373568, "in_reply_to_user": 27170840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27170840 }}, "user": { "id": 36768049, "name": "(B)ahiyyah.", "screen_name": "robynnicki_", "lang": "en", "location": "somewhere wit My Gi ", "create_at": date("2009-04-30"), "description": "R.I.P RobinDenny&Basir...", "followers_count": 1375, "friends_count": 919, "statues_count": 167932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089570899460098, "text": "@ohhaycassiek but also same.", "in_reply_to_status": 736084412727578625, "in_reply_to_user": 18601698, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18601698 }}, "user": { "id": 32275559, "name": "thirst trap", "screen_name": "jawshxkrueger", "lang": "en", "location": "Chicago, IL", "create_at": date("2009-04-16"), "description": "Queer // Vegan // They&Them", "followers_count": 528, "friends_count": 311, "statues_count": 26068 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571327254528, "text": "lowkey miss drinking by my daddy's grave alone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3301211957, "name": "7th Hokage.", "screen_name": "ventahkay", "lang": "en", "location": "shaqizlyfe❤️", "create_at": date("2015-05-27"), "description": "801 ✈️ 817", "followers_count": 810, "friends_count": 418, "statues_count": 42967 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bedford, TX", "id": "78454026903c03e4", "name": "Bedford", "place_type": "city", "bounding_box": rectangle("-97.168665,32.822102 -97.099752,32.867764") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4807132, "cityName": "Bedford" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571331440641, "text": "#Job alert: Software Engineer (Java) | Trustwave | #Chicago, IL https://t.co/H2o0FqQHAP #infosec #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.6297982,41.8781136"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Job", "Chicago", "infosec", "Jobs", "Hiring" }}, "user": { "id": 2888692910, "name": "Trustwave Jobs", "screen_name": "TrustwaveJobs", "lang": "en", "location": "Worldwide", "create_at": date("2014-11-03"), "description": "Official Trustwave Careers twitter channel. Follow for job opportunities, news, and insights on working @Trustwave.", "followers_count": 334, "friends_count": 217, "statues_count": 604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571352412160, "text": "Late night vibes ! https://t.co/MQXbcip36V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1099886917, "name": "MYDMISBROKE", "screen_name": "_yooootye", "lang": "en", "location": "null", "create_at": date("2013-01-17"), "description": "Oh you ain't heard of me ? Well I prefer that! #ProudZoe", "followers_count": 647, "friends_count": 606, "statues_count": 71910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571394392065, "text": "Wind 20.8 mph E. Barometer 29.671 in, Falling quickly. Temperature 66.1 °F. Rain today 0.73 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571427917824, "text": "Happy 18th birthday to my favorite adventure buddy❤️I love you so much and I hope your day is as amazing as you are�� https://t.co/ieTal21lCo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1071127794, "name": "Mckenna", "screen_name": "mckenna_parks", "lang": "en", "location": "California", "create_at": date("2013-01-08"), "description": "null", "followers_count": 728, "friends_count": 579, "statues_count": 8834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571578941440, "text": "@GonzalezJR88 just get a Pavelski jersey and send it to a place where they do stitching and have them put the C on it. Can't be that bad", "in_reply_to_status": 736089193277837313, "in_reply_to_user": 726521640461414400, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 726521640461414400 }}, "user": { "id": 1093997048, "name": "amanda", "screen_name": "anichole95", "lang": "en", "location": "San Jose, CA", "create_at": date("2013-01-15"), "description": "die hard Sharks fan for life. ig: a.nichole95", "followers_count": 370, "friends_count": 223, "statues_count": 31825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgemark, CA", "id": "29c99b003bea20e9", "name": "Ridgemark", "place_type": "city", "bounding_box": rectangle("-121.388225,36.802885 -121.346288,36.823481") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6069, "countyName": "San Benito", "cityID": 660706, "cityName": "Ridgemark" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089571834761216, "text": "Well this speech isn't getting memorized so I guess I'll just go to bed and pray that Wilken doesn't pick me to go tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2400676928, "name": "Caitlin Miller", "screen_name": "caitlinmillz", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-03-20"), "description": "live and let die", "followers_count": 441, "friends_count": 239, "statues_count": 9672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089572203892737, "text": "This my new anthem ������ https://t.co/T7hOuZK6GB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 187567655, "name": "MODA STAND UP", "screen_name": "_McBuckets_", "lang": "en", "location": "null", "create_at": date("2010-09-06"), "description": "Brady Christopher. The Bay-PNW|I'm just livin life, doin what I want |LINFIELD COLLEGE|If one day speed kills me, don't cry because I was smiling -Paul Walker", "followers_count": 310, "friends_count": 399, "statues_count": 9634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benicia, CA", "id": "ccb1d10a24cf562a", "name": "Benicia", "place_type": "city", "bounding_box": rectangle("-122.199321,38.041997 -122.103467,38.101223") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 605290, "cityName": "Benicia" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089572212236288, "text": "@genarosoto10 watch that shit go live me and @Stevie5lara gon be on that", "in_reply_to_status": 736089393178382336, "in_reply_to_user": 545552720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 545552720, 2847751995 }}, "user": { "id": 4888872894, "name": "Giovanni", "screen_name": "gioschroeder21", "lang": "en", "location": "Austin, TX", "create_at": date("2016-02-10"), "description": "2/16/16", "followers_count": 117, "friends_count": 55, "statues_count": 2351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089572275146752, "text": "Next time you pull a strap out on me use it don't just show it off ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3293819400, "name": "G¡łłż.6", "screen_name": "Jeisson_dream", "lang": "en", "location": "fuck u ", "create_at": date("2015-07-25"), "description": "catch me in the studio smoking a blunt", "followers_count": 249, "friends_count": 238, "statues_count": 3031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089572833009664, "text": "I don't trust that \" She just my sister\" shit �� who you trying fool?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 101541290, "name": "υиιqυє❤️", "screen_name": "__xoRoyal", "lang": "en", "location": "Peace, Love, & Good vibes ⚠️", "create_at": date("2010-01-03"), "description": "null", "followers_count": 2057, "friends_count": 1603, "statues_count": 34755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glen Burnie, MD", "id": "cad757363c1a85df", "name": "Glen Burnie", "place_type": "city", "bounding_box": rectangle("-76.644717,39.131259 -76.563196,39.207912") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24003, "countyName": "Anne Arundel", "cityID": 2432650, "cityName": "Glen Burnie" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089573025939457, "text": "5/27/2016 - 02:00\nTemp: 69.1F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.845in. & Rising\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 54, "statues_count": 53202 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089574015795204, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Falling. Temperature 72.1 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-27T00:00:04.000Z"), "id": 736089574032572416, "text": "@rosec0ncrete really missing you right now ��", "in_reply_to_status": 636269593351225344, "in_reply_to_user": 3239020754, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3239020754 }}, "user": { "id": 3239020754, "name": "evil", "screen_name": "rosec0ncrete", "lang": "en", "location": "null", "create_at": date("2015-06-07"), "description": "sitting pretty while being petty ✨", "followers_count": 2463, "friends_count": 2238, "statues_count": 6061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089574275833857, "text": "@raynicholew26 Maybe you should try this https://t.co/RIhUQsRbUi", "in_reply_to_status": -1, "in_reply_to_user": 332140420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332140420 }}, "user": { "id": 1056377431, "name": "Mal Pal", "screen_name": "MallorySuter", "lang": "en", "location": "California, USA", "create_at": date("2013-01-02"), "description": "Cats & Coffee & Lax", "followers_count": 267, "friends_count": 406, "statues_count": 2625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuba City, CA", "id": "133b1fa8f653eb11", "name": "Yuba City", "place_type": "city", "bounding_box": rectangle("-121.660213,39.068913 -121.597638,39.174405") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6101, "countyName": "Sutter", "cityID": 686972, "cityName": "Yuba City" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089574686887937, "text": "Fact: my weakness is Ryan Kesler in a suit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 704566426737061888, "name": "Lonni ☀️", "screen_name": "lonni_04", "lang": "en", "location": "Anaheim, CA", "create_at": date("2016-02-29"), "description": "Anaheim Ducks. Disneyland. Soccer. Coffee. Ryan Kesler enthusiast.", "followers_count": 303, "friends_count": 647, "statues_count": 2023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rowland Heights, CA", "id": "a5655a07276e4caf", "name": "Rowland Heights", "place_type": "city", "bounding_box": rectangle("-117.931166,33.945518 -117.850306,34.000574") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 663218, "cityName": "Rowland Heights" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089574783340544, "text": "i want a guarantee, but it ain't easy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3087355346, "name": "champagnemami", "screen_name": "26drewchadwick", "lang": "en", "location": "null", "create_at": date("2015-03-15"), "description": "Roberta Banuelos | blessed", "followers_count": 263, "friends_count": 228, "statues_count": 5982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089575060168705, "text": "Staying up all night to wait for @Bloodline #Yep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Yep" }}, "user_mentions": {{ 2742380412 }}, "user": { "id": 2491140028, "name": "Cody", "screen_name": "cja_1995", "lang": "en", "location": "Western MD - Eastern Shore", "create_at": date("2014-04-17"), "description": "Rather than love, rather than money, than fame, than fairness, give me truth. - Thoreau.", "followers_count": 413, "friends_count": 381, "statues_count": 10105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cumberland, MD", "id": "448a1775be49da49", "name": "Cumberland", "place_type": "city", "bounding_box": rectangle("-78.812801,39.604408 -78.703268,39.722191") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24001, "countyName": "Allegany", "cityID": 2421325, "cityName": "Cumberland" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089575081136131, "text": "I need so much prayer and so many hugs right now. I feel so alone.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 45768361, "name": "Ëmily Annë", "screen_name": "emianneart", "lang": "en", "location": "Detroit, MI", "create_at": date("2009-06-08"), "description": "freelance fashion illustrator á la Detroit.", "followers_count": 353, "friends_count": 256, "statues_count": 12596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dearborn Heights, MI", "id": "59dab0b1b9779311", "name": "Dearborn Heights", "place_type": "city", "bounding_box": rectangle("-83.312805,42.268212 -83.217437,42.357044") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2621020, "cityName": "Dearborn Heights" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089575110500353, "text": "someone be my McDreamy pls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2793163741, "name": "brittanie campbell", "screen_name": "brittanieelynn", "lang": "en", "location": "Republic, MO", "create_at": date("2014-09-05"), "description": "null", "followers_count": 351, "friends_count": 450, "statues_count": 1119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Republic, MO", "id": "44fafdc06c719c62", "name": "Republic", "place_type": "city", "bounding_box": rectangle("-93.507092,37.096369 -93.420182,37.155321") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29077, "countyName": "Greene", "cityID": 2961238, "cityName": "Republic" } }
+{ "create_at": datetime("2016-05-27T00:00:05.000Z"), "id": 736089575622250497, "text": "@adriana_a_wong hit ya boy up because he does nothing all day also ��", "in_reply_to_status": 736081665794019328, "in_reply_to_user": 3023005663, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3023005663 }}, "user": { "id": 2193523945, "name": "christian lopez", "screen_name": "Itslopezhere", "lang": "en", "location": "chinatown", "create_at": date("2013-11-13"), "description": "there's no such thing as the wrong hole | BCP '17", "followers_count": 286, "friends_count": 245, "statues_count": 3888 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941061042176, "text": "��Ima ft Melissa and tell her this and hang up in her face !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442053471, "name": "YNW$.", "screen_name": "daeeepoppin_", "lang": "en", "location": "Will & Quayle ❤️", "create_at": date("2011-12-20"), "description": "❤️. sc: dateonna", "followers_count": 2326, "friends_count": 976, "statues_count": 180236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ponchatoula, LA", "id": "00bf0ddb6be3dd05", "name": "Ponchatoula", "place_type": "city", "bounding_box": rectangle("-90.478909,30.419185 -90.375706,30.479003") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2261615, "cityName": "Ponchatoula" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941090394113, "text": "Home now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48945115, "name": "Taylor Elise", "screen_name": "bootsAndBoobs_", "lang": "en", "location": "null", "create_at": date("2009-06-19"), "description": "I feel like Pabloooo", "followers_count": 787, "friends_count": 470, "statues_count": 77201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merrillville, IN", "id": "39aac4dbc7252e68", "name": "Merrillville", "place_type": "city", "bounding_box": rectangle("-87.395119,41.447036 -87.24453,41.522608") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1848528, "cityName": "Merrillville" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941362892800, "text": "@NadaPerson @KurtBusiek \"If I call it civil disobedience, it's not theft.\" Disney is not the government. You're a thief.", "in_reply_to_status": 736451010613252096, "in_reply_to_user": 3286795406, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3286795406, 125171167 }}, "user": { "id": 18262639, "name": "DrewAtHitFix", "screen_name": "DrewAtHitFix", "lang": "en", "location": "Northridge, CA", "create_at": date("2008-12-19"), "description": "Jumped up pantry boy who never knew his place. Reader of tea leaves. Film Nerd 1.0 raising Film Nerds 2.0. LAFCA member. Largely finished. #woke", "followers_count": 39613, "friends_count": 3114, "statues_count": 95841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941367058432, "text": "When it hurts so bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3185872754, "name": "Courtney Starnes", "screen_name": "bri0nne_", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-05-04"), "description": "Innovator. Future...a lot of things. Communication major at Texas A&M University #TAMU18", "followers_count": 1708, "friends_count": 976, "statues_count": 26868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College Station, TX", "id": "00a4bbcc0dcd7572", "name": "College Station", "place_type": "city", "bounding_box": rectangle("-96.37703,30.520359 -96.206267,30.650584") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48041, "countyName": "Brazos", "cityID": 4815976, "cityName": "College Station" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941396402177, "text": "Middle Eastern girls are something else man��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 415697122, "name": "Aramm", "screen_name": "ar4mig", "lang": "en", "location": "null", "create_at": date("2011-11-18"), "description": "null", "followers_count": 467, "friends_count": 383, "statues_count": 10456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941719363585, "text": "Wendy stopped my and surprised me with Thai tea boba ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 89117644, "name": "Nicole LeBarre", "screen_name": "asianlebarre", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-11-10"), "description": "this is my place to vent. love me or hate me im still an obsession", "followers_count": 155, "friends_count": 69, "statues_count": 34983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941744660481, "text": "Watching Love & Basketball for the first time.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 141766582, "name": "jaylene", "screen_name": "KeepingUpWitJ", "lang": "en", "location": "UAlbany 19", "create_at": date("2010-05-08"), "description": "• the fear that you feel is not real •", "followers_count": 163, "friends_count": 108, "statues_count": 2994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941941665792, "text": "@marissaharlee stop tryna act all innocent in front all these people you know damn well I'll beat ya ass", "in_reply_to_status": 736451774156967937, "in_reply_to_user": 346692829, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 346692829 }}, "user": { "id": 350760161, "name": "ㅤ", "screen_name": "lorddebs_", "lang": "en", "location": "twenty", "create_at": date("2011-08-08"), "description": "null", "followers_count": 752, "friends_count": 284, "statues_count": 62529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451941958443008, "text": "First time in a while that the sky looks this clear ✨�� #MyBear #constellations \n\nHace rato que el cielo se ve tan claro. �� �� #MiOsito", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MyBear", "constellations", "MiOsito" }}, "user": { "id": 99732595, "name": "April with a B", "screen_name": "Ap3e", "lang": "en", "location": "Location-Wherever God takes me", "create_at": date("2009-12-27"), "description": "God's love never fails. Snapchat: aprilhdez", "followers_count": 459, "friends_count": 545, "statues_count": 4204 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moore, OK", "id": "207f2c7abbdb201b", "name": "Moore", "place_type": "city", "bounding_box": rectangle("-97.521372,35.284155 -97.405917,35.370781") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4049200, "cityName": "Moore" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451942159814656, "text": "Hair Stylist - Ulta Beauty: (#Rockville, MD) https://t.co/ttNs8k8PLw #Cosmetology #Job #Jobs #Hiring https://t.co/T7CavlhS3l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1275458,39.0625439"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Rockville", "Cosmetology", "Job", "Jobs", "Hiring" }}, "user": { "id": 732227909243535360, "name": "Ulta Beauty Jobs", "screen_name": "ultabeautyjobs", "lang": "en", "location": "null", "create_at": date("2016-05-16"), "description": "It’s a great time to be on the fun side of beauty. Follow to learn more!", "followers_count": 38, "friends_count": 1, "statues_count": 1977 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockville, MD", "id": "8c88f07c8666389d", "name": "Rockville", "place_type": "city", "bounding_box": rectangle("-77.220557,39.053158 -77.105648,39.120952") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2467675, "cityName": "Rockville" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451942826704896, "text": "Wind 0.8 mph N. Barometer 29.834 in, Falling Rapidly. Temperature 66.1 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451942893916160, "text": "@_jreynaa your welcome", "in_reply_to_status": 736451763146887168, "in_reply_to_user": 3247102003, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3247102003 }}, "user": { "id": 969608587, "name": "Gucci", "screen_name": "FreeWop1017", "lang": "en", "location": "Hermiston, Oregon", "create_at": date("2012-11-24"), "description": "Sensational #1017 sc:c_munoz09", "followers_count": 1102, "friends_count": 781, "statues_count": 66045 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermiston, OR", "id": "3b213491c5ae3f7e", "name": "Hermiston", "place_type": "city", "bounding_box": rectangle("-119.32694,45.824334 -119.258928,45.864803") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41059, "countyName": "Umatilla", "cityID": 4133700, "cityName": "Hermiston" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451943116197889, "text": "happy place & happy it's the weekend. ��. #amoebamusic #hollywood #losangeles #fridaynight #vinyl… https://t.co/L9MglpNNKG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.32902,34.09762"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "amoebamusic", "hollywood", "losangeles", "fridaynight", "vinyl" }}, "user": { "id": 46852598, "name": "Nicole", "screen_name": "nicoleashleefox", "lang": "en", "location": "location: los angeles.", "create_at": date("2009-06-13"), "description": "the tall girl in front of you at concerts // assistant wardrobe stylist // wanderer + constant contemplator.", "followers_count": 128, "friends_count": 285, "statues_count": 6099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451943271305217, "text": "@_yoitsyourmom I laughed so it obviously was asshole", "in_reply_to_status": 736451400343818240, "in_reply_to_user": 450136203, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 450136203 }}, "user": { "id": 377897210, "name": "♡ rita ♡", "screen_name": "xoxorita_", "lang": "en", "location": "null", "create_at": date("2011-09-22"), "description": "Just because today was a terrible day doesn't mean tomorrow might not be the best day of your entire life, you just have to wake up and get there.", "followers_count": 519, "friends_count": 299, "statues_count": 20900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colton, CA", "id": "496f5f37fc86ed85", "name": "Colton", "place_type": "city", "bounding_box": rectangle("-117.371882,34.018596 -117.26786,34.0961") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 614890, "cityName": "Colton" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451943531466752, "text": "also probably the best thing diplo was ever involved with or will ever be involved with tbh", "in_reply_to_status": 736451867081859072, "in_reply_to_user": 12375, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 12375, "name": "Thomas Boyt", "screen_name": "thomasABoyt", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2006-11-14"), "description": "software 'engineer' • @recursecenter [w2013] • mostly music, video games, wrestling, but sometimes javascript", "followers_count": 710, "friends_count": 819, "statues_count": 18289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451943971753984, "text": "Is \"cat\" or \"cattin\" a universal term, or is that just a Bay Area thing?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24111656, "name": "Jay", "screen_name": "jamaicaa_rene", "lang": "en", "location": "Oakland, California", "create_at": date("2009-03-12"), "description": "Spelman | 19 | Oakland", "followers_count": 608, "friends_count": 621, "statues_count": 13877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451944080936960, "text": "and setting off a all the nukes and then hes blows himself up explodes three times.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2445097933, "name": "freeman", "screen_name": "freeman88134532", "lang": "en", "location": "null", "create_at": date("2014-04-15"), "description": "null", "followers_count": 139, "friends_count": 274, "statues_count": 21386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fond du Lac, WI", "id": "47341c54fe1a873b", "name": "Fond du Lac", "place_type": "city", "bounding_box": rectangle("-88.523732,43.718278 -88.371043,43.806827") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55039, "countyName": "Fond du Lac", "cityID": 5526275, "cityName": "Fond du Lac" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451944789770241, "text": "Temp: 64.4F W C: 64.4F Wind:--- at 0.0kts Baro: 1016.8mb and Rising slowly Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 213, "friends_count": 218, "statues_count": 106354 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451944827359232, "text": "Still fucking laughing about last night though to be honest.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427685653, "name": "I'm Just A Guy", "screen_name": "GoodGuyGrady", "lang": "en", "location": "Rocklin, CA", "create_at": date("2011-12-03"), "description": "Rocklin High School Senior", "followers_count": 167, "friends_count": 238, "statues_count": 3609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-05-28T00:00:00.000Z"), "id": 736451944869306368, "text": "bored meh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464276767, "name": "Mara Stallins", "screen_name": "marastallins", "lang": "en", "location": "null", "create_at": date("2012-01-14"), "description": "lets get weird | sc: marraaa5 | ig: marastallins", "followers_count": 832, "friends_count": 526, "statues_count": 22084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451945381138432, "text": "@Roger_That1223 thank you!", "in_reply_to_status": 736386777792032768, "in_reply_to_user": 2542689086, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2542689086 }}, "user": { "id": 380566997, "name": "Molly Mansch", "screen_name": "_mollymansch", "lang": "en", "location": "null", "create_at": date("2011-09-26"), "description": "null", "followers_count": 490, "friends_count": 380, "statues_count": 1027 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451945406201856, "text": "sugar daddy's in my DMs offended with my last tweet, sns", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465245714, "name": "Jai", "screen_name": "MaiteGarzi", "lang": "en", "location": "South Texas", "create_at": date("2012-01-15"), "description": "19 and 5'2. Baylor U. chiquita pero picosa (like salsa verde)", "followers_count": 2261, "friends_count": 327, "statues_count": 13833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harlingen, TX", "id": "1077f3afe6e2aa44", "name": "Harlingen", "place_type": "city", "bounding_box": rectangle("-97.80185,26.135158 -97.629315,26.257572") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4832372, "cityName": "Harlingen" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451945460867072, "text": "Ripley SW Limestone Co. Temp: 72.7°F Wind:0.7mph Pressure: 995.5mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451945548763136, "text": "@MadeManEmpire fb full of ����", "in_reply_to_status": 736451699079024640, "in_reply_to_user": 294987638, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 294987638 }}, "user": { "id": 3195756613, "name": "#Loading......", "screen_name": "_Smooth10", "lang": "en", "location": "Overtown,Miami", "create_at": date("2015-05-14"), "description": "*Draft Dreams* #BCU19 #DSBG", "followers_count": 593, "friends_count": 698, "statues_count": 10600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Little River, FL", "id": "4ba5710f3448a6d8", "name": "West Little River", "place_type": "city", "bounding_box": rectangle("-80.260185,25.845593 -80.196918,25.871522") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1276487, "cityName": "West Little River" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451945829896192, "text": "05/28@03:00 - Temp 70.8F, WC 70.8F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.108in, Steady. Rain 0.00in. Hum 79%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451946182103041, "text": "�������� https://t.co/yHZATmZphL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 345148199, "name": "THEYFREEDGUWOP✨", "screen_name": "Kalifornia__", "lang": "en", "location": "null", "create_at": date("2011-07-29"), "description": "perfect imperfection", "followers_count": 706, "friends_count": 276, "statues_count": 15530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451946404577280, "text": "My porch was infested with them. Was a pretty nice night out here in the ville. https://t.co/xbRNV04f1P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4791149133, "name": "Tanner Dennie", "screen_name": "T_Dennie", "lang": "en", "location": "Auburndale/Gainesville, FL", "create_at": date("2016-01-12"), "description": "Intern for @InsideTheGators/@Rivals. Senior Telecom Major @UF. Contributor for SB Nation's @TeamSpeedKills. Bringin' the heat one article at a time.", "followers_count": 387, "friends_count": 104, "statues_count": 3531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gainesville, FL", "id": "7dda05213481260c", "name": "Gainesville", "place_type": "city", "bounding_box": rectangle("-82.421473,29.600496 -82.239066,29.745847") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12001, "countyName": "Alachua", "cityID": 1225175, "cityName": "Gainesville" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451946412834816, "text": "U don't know whn it's your time or your love ones , seriously take advantage of each sec your alive stay up bbygirl�� https://t.co/cC5aBgkfpL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1406640601, "name": "lani boo✨", "screen_name": "ayee_alanah", "lang": "en", "location": "null", "create_at": date("2013-05-05"), "description": "null", "followers_count": 386, "friends_count": 313, "statues_count": 6110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451946521841664, "text": "@_mit_mit thanks for the support friend ����", "in_reply_to_status": 736450949120565253, "in_reply_to_user": 1286629075, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1286629075 }}, "user": { "id": 3094266128, "name": "Daniel Arandia Diaz", "screen_name": "dannybesmooth", "lang": "en", "location": "Richmond, Ca ", "create_at": date("2015-03-17"), "description": "null", "followers_count": 92, "friends_count": 142, "statues_count": 593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451947176202240, "text": "OK Corral? ���� nahh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 380097107, "name": "Soy El Chente", "screen_name": "JuniorRivas06", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-09-25"), "description": "I drive trucks here and there. Lol #GM", "followers_count": 188, "friends_count": 139, "statues_count": 5381 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451947520090113, "text": "@killuminatiCUH @_xgemini she fronts like that..., that's why I'm tryna help the process ������", "in_reply_to_status": 736451669219672065, "in_reply_to_user": 4326754453, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4326754453, 3386969512 }}, "user": { "id": 2810578640, "name": "KMACK", "screen_name": "KVEEZY_", "lang": "en", "location": "Anchorage, AK", "create_at": date("2014-09-14"), "description": "WJT ❤️", "followers_count": 516, "friends_count": 519, "statues_count": 18892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451947633377281, "text": "I just wanted to do it for old times sake. I just hope that you're proud of the person I became. It's been awhile..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343814312, "name": "Matt Caliwag", "screen_name": "illMatt", "lang": "en", "location": "Beaumont, Ca", "create_at": date("2011-07-27"), "description": "self-made", "followers_count": 280, "friends_count": 127, "statues_count": 36757 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calimesa, CA", "id": "0148540119dc25ab", "name": "Calimesa", "place_type": "city", "bounding_box": rectangle("-117.074928,33.953835 -117.015045,34.004879") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 609864, "cityName": "Calimesa" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451947666898944, "text": "Opened Abandoned Vehicles request via iphone at 2134 25th Ave https://t.co/pYVjE4xGNI. Parked over 3 days now. https://t.co/TPeuRJEH5J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.48241778,37.74768024"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1589692776, "name": "SF311 Reports", "screen_name": "SF311Reports", "lang": "en", "location": "San Francisco, CA", "create_at": date("2013-07-12"), "description": "This is account is not monitored and is used to post service request update information. To communicate with SF311, please use our main account 'SF311'", "followers_count": 117, "friends_count": 0, "statues_count": 341514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451947935326209, "text": "Wind 2.0 mph S. Barometer 1011.38 mb, Steady. Temperature 64.0 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 31, "friends_count": 128, "statues_count": 14700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451948359110656, "text": "JR goes to the club during halftime & still make it back for Quarter 3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33708874, "name": "★Ro✘tar★ ++®", "screen_name": "Reggie_IsDead", "lang": "en", "location": "Brewerytown, Norf Philadelphia", "create_at": date("2009-04-20"), "description": "Ace Vega| Kylo Ren | Some UGLY Dead #RoXtar Posthumously Tweeting & Making Beats 09/09/1995-??/??/20?? | DCLXVI | sc: reggieis_dead", "followers_count": 1787, "friends_count": 1020, "statues_count": 171213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451948438802432, "text": "Wind 4.0 mph SE. Barometer 29.812 in, Steady. Temperature 68.8 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451948753223680, "text": "Deep ���� lol https://t.co/70KVWP57UQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 568804019, "name": "Connor Stevens", "screen_name": "Jeremiah_Baker_", "lang": "en", "location": "Cypress, TX", "create_at": date("2012-05-01"), "description": "San Marcos, I Love You XOXO! #BobcatFamily #TXST17", "followers_count": 790, "friends_count": 768, "statues_count": 15655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:01.000Z"), "id": 736451948946169859, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 591372758, "name": "Geezu$", "screen_name": "RangerGabriel", "lang": "en", "location": "San Marcos, TX", "create_at": date("2012-05-26"), "description": "SATX #TXST19 #BobcatFamily", "followers_count": 708, "friends_count": 866, "statues_count": 20503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451949462183936, "text": "Temp 65.7° Hi/Lo 69.9/65.7 Rng 4.2° WC 65.7° Hmd 82% Rain 0.00\" Storm 0.00\" BAR 30.062 Falling DP 60.1° Wnd 2mph Dir SSW Gst 9mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 19043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451949827002368, "text": "Wind 5.4 mph NE. Barometer 29.71 in, Falling slowly. Temperature 46.6 °F. Rain today 0.00 in. Humidity 10%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 19, "friends_count": 5, "statues_count": 42523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950200266752, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":66.9°F Wind:1.1mph Pressure: 30.06hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 18, "statues_count": 320216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950372327424, "text": "I don't fuck around. If I say it. I mean it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1338064826, "name": "Tyler", "screen_name": "maurer_tyler", "lang": "en", "location": "Randolph,MA Storrs,CT", "create_at": date("2013-04-08"), "description": "I drink and I know things", "followers_count": 95, "friends_count": 110, "statues_count": 2997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Randolph, MA", "id": "5aee08e0a4c73115", "name": "Randolph", "place_type": "city", "bounding_box": rectangle("-71.10293,42.135489 -71.018997,42.206765") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2555990, "cityName": "Randolph" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950665969665, "text": "Wind 1.0 mph SSW. Barometer 29.982 in, Falling slowly. Temperature 69.7 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20137 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950833590272, "text": "@JordynRae11 @Flintercell I'll fill your tank", "in_reply_to_status": 736451893757517825, "in_reply_to_user": 4902216309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4902216309, 908512502 }}, "user": { "id": 908512502, "name": "Sunshine", "screen_name": "Flintercell", "lang": "en", "location": "Draenor", "create_at": date("2012-10-27"), "description": "Flow since '95. ΚΣ - ΘΕ. Hardcore Gamer.", "followers_count": 444, "friends_count": 391, "statues_count": 781 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherwood, OR", "id": "6870523b6f8d96e6", "name": "Sherwood", "place_type": "city", "bounding_box": rectangle("-122.869771,45.345593 -122.805551,45.376774") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4167100, "cityName": "Sherwood" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950846169088, "text": "@NateLbc24 i know yall give me hope", "in_reply_to_status": 736451852993060866, "in_reply_to_user": 3240178675, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3240178675 }}, "user": { "id": 2162094193, "name": "princess lena ✨", "screen_name": "itss_selenaluna", "lang": "en", "location": "incomplete", "create_at": date("2013-10-28"), "description": "i have a big heart that's still hurting", "followers_count": 2034, "friends_count": 979, "statues_count": 54161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451950875672576, "text": "@cam_rat ok what's up", "in_reply_to_status": 736451426029850624, "in_reply_to_user": 1627699910, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1627699910 }}, "user": { "id": 1627707986, "name": ".", "screen_name": "_makiahfader", "lang": "en", "location": "Dayton, ohio ", "create_at": date("2013-07-28"), "description": "that'll be just fine", "followers_count": 301, "friends_count": 147, "statues_count": 8578 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kettering, OH", "id": "a121bc3ed630a35e", "name": "Kettering", "place_type": "city", "bounding_box": rectangle("-84.218859,39.654418 -84.088199,39.736536") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39113, "countyName": "Montgomery", "cityID": 3940040, "cityName": "Kettering" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451951156547585, "text": "#BATONROUGE, LA #Retail #Job: Bakery Associate at Winn Dixie https://t.co/isjy0tGHzW #Jobs #Hiring https://t.co/o5uf8dJlYn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-91.0329278,30.4477751"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BATONROUGE", "Retail", "Job", "Jobs", "Hiring" }}, "user": { "id": 388009236, "name": "Winn-Dixie Careers", "screen_name": "WDCareers", "lang": "en", "location": "Jacksonville, FL", "create_at": date("2011-10-09"), "description": "Join our WINN-ing team and help make the lives of our customers and fellow associates FUN! Winn-Dixie is one of the nation's largest food retailers.", "followers_count": 540, "friends_count": 266, "statues_count": 18478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451951806709766, "text": "@PattyArquette haa.��", "in_reply_to_status": 736405097295417345, "in_reply_to_user": 122533830, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 122533830 }}, "user": { "id": 41481286, "name": "Lorene", "screen_name": "PhillyVamp", "lang": "en", "location": "va beach", "create_at": date("2009-05-20"), "description": "phillies/redskin/beatle fan/animal lover", "followers_count": 52, "friends_count": 289, "statues_count": 3037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia Beach, VA", "id": "84229b03659050aa", "name": "Virginia Beach", "place_type": "city", "bounding_box": rectangle("-76.22808,36.672684 -75.911812,36.932399") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51810, "countyName": "Virginia Beach", "cityID": 5182000, "cityName": "Virginia Beach" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451951836188673, "text": "Temp: 69.4°F Wind:0.0mph Pressure: 29.962hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65386 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451951848771585, "text": "@BlackAttackHawk yada Yada", "in_reply_to_status": 736450917843775488, "in_reply_to_user": 4924384695, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user_mentions": {{ 4924384695 }}, "user": { "id": 710580943564709888, "name": "Nathan", "screen_name": "2013Nathan3", "lang": "en", "location": "null", "create_at": date("2016-03-17"), "description": "null", "followers_count": 142, "friends_count": 85, "statues_count": 4845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451952771375104, "text": "Somebody play me in 2k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 383032115, "name": "tasha mack.", "screen_name": "_ogtaejah", "lang": "en", "location": "DDD, TX / Commerce, TX", "create_at": date("2011-09-30"), "description": "beyoncé af, tamuc wbb. Kar$on.", "followers_count": 875, "friends_count": 797, "statues_count": 51273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451953027219457, "text": "HAPPY BIRTHDAY to my #1!!������ You have no idea how much you mean to me...I have MAD LOVE for you! i love you‼️12:00am https://t.co/1lc2eUyUrQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1679128706, "name": "ma'♈️", "screen_name": "og_shaddai", "lang": "en", "location": " boolin w/ @babey_cheex", "create_at": date("2013-08-17"), "description": "rhs. snapchat: shaddai_cxcx", "followers_count": 525, "friends_count": 365, "statues_count": 14788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451953111093250, "text": "This is really how I'm living https://t.co/WZRdjyDLSU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1409145523, "name": "✨angelalala✨", "screen_name": "angela_simisola", "lang": "en", "location": "socal ✈️ bay area ", "create_at": date("2013-05-06"), "description": "Keep God first and everything else will follow. omo naija", "followers_count": 432, "friends_count": 781, "statues_count": 7224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451953245425665, "text": "Wind 0.0 mph ---. Barometer 29.970 in, Steady. Temperature 66.9 °F. Rain today 0.06 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451953295659008, "text": "Chaka ass music", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 434667133, "name": "Lourdes", "screen_name": "pinchiefcknlulu", "lang": "en", "location": "null", "create_at": date("2011-12-11"), "description": "null", "followers_count": 290, "friends_count": 604, "statues_count": 11750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-28T00:00:02.000Z"), "id": 736451953350172674, "text": "Sick it's my birthday ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2720551186, "name": "Artur", "screen_name": "Art_551", "lang": "en", "location": "Hollywood ", "create_at": date("2014-07-22"), "description": "karotum em kez❤️^^^", "followers_count": 280, "friends_count": 448, "statues_count": 10699 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451953610260480, "text": "I can't believe it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320006598, "name": "Emanuel Dayries ⚜", "screen_name": "Eazy_Dayries", "lang": "en", "location": "Tempe, AZ", "create_at": date("2011-06-18"), "description": "Living my life through God and his word ▪️ New Orleans ▪️Arizona State Football ▪️NOLA/AZ", "followers_count": 1474, "friends_count": 513, "statues_count": 24463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451953660547073, "text": "Ничто так не стимулирует, как мысль о том, что всего за неделю я заработала столько денег❗", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 490568232, "name": "Настечка", "screen_name": "naaastein", "lang": "ru", "location": "null", "create_at": date("2012-02-12"), "description": "я сам своє небо, я сам своє пекло", "followers_count": 76, "friends_count": 35, "statues_count": 3882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451954717646848, "text": "Foundation built on loyalty you can't ruin me now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4406139977, "name": "Dan", "screen_name": "DanLott10", "lang": "en", "location": "null", "create_at": date("2015-12-07"), "description": "Against all odds", "followers_count": 253, "friends_count": 201, "statues_count": 4274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairview Park, OH", "id": "9cacf32ef70c27ab", "name": "Fairview Park", "place_type": "city", "bounding_box": rectangle("-81.875476,41.417577 -81.815851,41.467225") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3926446, "cityName": "Fairview Park" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451954868510721, "text": "New header and avi bc why not :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2453571235, "name": "Pricila ❥", "screen_name": "pricila_lopez3", "lang": "en", "location": "null", "create_at": date("2014-04-19"), "description": "Who's Kalin & Myles?", "followers_count": 636, "friends_count": 357, "statues_count": 19145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451955740934148, "text": "This is the other side no one talks about, respect. https://t.co/3jv3Nfx1G8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 407480956, "name": "Muela", "screen_name": "ItsMuelaTime", "lang": "en", "location": "North Highlands, CA", "create_at": date("2011-11-07"), "description": "NJ ➡ CA. I'm pretty uncool and that's cool. Subscribe to my YouTube channel though!", "followers_count": 609, "friends_count": 243, "statues_count": 20205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Highlands, CA", "id": "ee9341f90ab895d5", "name": "North Highlands", "place_type": "city", "bounding_box": rectangle("-121.424332,38.638355 -121.335815,38.704481") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 651924, "cityName": "North Highlands" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451955887853572, "text": "2b60870f00p609FF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.632227,43.329371"), "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 22, "friends_count": 0, "statues_count": 25428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New York, USA", "id": "94965b2c45386f87", "name": "New York", "place_type": "admin", "bounding_box": rectangle("-79.76259,40.477383 -71.777492,45.015851") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36065, "countyName": "Oneida" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956000989184, "text": "Empowered gays empower gays", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 311872025, "name": "S H", "screen_name": "unlearninq", "lang": "en", "location": "Norman, OK", "create_at": date("2011-06-05"), "description": "ou19, political science & pre-law. unlearner/learner. politics/social justice/equality. Hillary Clinton for President!", "followers_count": 371, "friends_count": 203, "statues_count": 9200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956495896577, "text": "@withoutfear7 As I sleep I beg you to unite against this evil and know victory is ours claim it God be with us good night", "in_reply_to_status": -1, "in_reply_to_user": 4248620772, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4248620772 }}, "user": { "id": 4248620772, "name": "Without Fear 57", "screen_name": "withoutfear7", "lang": "en", "location": "null", "create_at": date("2015-11-15"), "description": "Strong Christian beliefs, conservative values, looking for real conservative, governmental change, extremely fed up with PC", "followers_count": 801, "friends_count": 1268, "statues_count": 5850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956542210048, "text": "ZIVON SO DUMB ������ @RiceZivon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user_mentions": {{ 940540237 }}, "user": { "id": 2264816552, "name": "PRINCE$$ C ⚡️", "screen_name": "youenvyciara", "lang": "en", "location": "null", "create_at": date("2013-12-27"), "description": "AllAboutCiara ❤️ | 040316 .. Tez ✨| jun18r ➰ |", "followers_count": 442, "friends_count": 278, "statues_count": 4777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynchburg, MS", "id": "6db266af27a0758d", "name": "Lynchburg", "place_type": "city", "bounding_box": rectangle("-90.113963,34.947924 -90.072245,34.977201") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28033, "countyName": "DeSoto", "cityID": 2842840, "cityName": "Lynchburg" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956680495104, "text": "read peoples mind https://t.co/yGXhW0T4s8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366453148, "name": "shelbsssss", "screen_name": "srubalcado_", "lang": "en", "location": "somewhere in tx ", "create_at": date("2014-02-28"), "description": "JB my lover.\nI seen the loml 7/3/13 & 4/10/16", "followers_count": 711, "friends_count": 2064, "statues_count": 9165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956714045442, "text": "Nights like these make me not wanna grow up. ☺️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1649216052, "name": "Igor Poulter", "screen_name": "itss_igor", "lang": "en", "location": "Utah, USA", "create_at": date("2013-08-05"), "description": "Chs Falcon SC it's_Igor", "followers_count": 482, "friends_count": 492, "statues_count": 1001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, UT", "id": "d254694c82a4473e", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-112.05574,41.074662 -111.979637,41.126319") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4913850, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451956852428800, "text": "@issflowr I know lol I have no clue what happened", "in_reply_to_status": 736451516811251715, "in_reply_to_user": 3383794153, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3383794153 }}, "user": { "id": 551989081, "name": "G U S T A V O", "screen_name": "whataboutgus", "lang": "en", "location": "TSNMI", "create_at": date("2012-04-12"), "description": "null", "followers_count": 160, "friends_count": 89, "statues_count": 1973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957120983040, "text": "Im next", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2192014948, "name": "ROOKIE", "screen_name": "CjMr35", "lang": "en", "location": "Aquinas ", "create_at": date("2013-11-22"), "description": "C/O '18 | 6'1 175 lbs | 4.69 40 | DB |Highschool | ✳️BEAST✳️ __Divine__. . #TruthNation", "followers_count": 878, "friends_count": 474, "statues_count": 22413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hampton Manor, NY", "id": "8699c6ba83a01836", "name": "Hampton Manor", "place_type": "city", "bounding_box": rectangle("-73.744969,42.604069 -73.698371,42.644182") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36083, "countyName": "Rensselaer", "cityID": 3631918, "cityName": "Hampton Manor" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957158617088, "text": "I've been laying in bed all day lol I feel lazy af", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2212907618, "name": "Val", "screen_name": "valeriieerobles", "lang": "en", "location": "fontana, CA", "create_at": date("2013-11-24"), "description": "get right or get left", "followers_count": 606, "friends_count": 430, "statues_count": 18779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957192155136, "text": "Wind 9.6 mph ESE. Barometer 29.813 in, Rising slowly. Temperature 64.6 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957385089025, "text": "were gonna close so late tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031768244, "name": "chardae faith ♡☻", "screen_name": "shawtychawday", "lang": "en", "location": "76$OUTH$IDE", "create_at": date("2015-02-19"), "description": "crowned for success ♛", "followers_count": 723, "friends_count": 335, "statues_count": 28652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean Pointe, HI", "id": "013abac55b5e444f", "name": "Ocean Pointe", "place_type": "city", "bounding_box": rectangle("-158.044905,21.303603 -158.013973,21.328864") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1556685, "cityName": "Ocean Pointe" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957502545921, "text": "I just wanna kiss ya", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2290212146, "name": "Tanner Webb", "screen_name": "OfficialWebbsy", "lang": "en", "location": "The Lost Boys", "create_at": date("2014-01-13"), "description": "idfc", "followers_count": 1403, "friends_count": 337, "statues_count": 8420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957573976065, "text": "@fellcleavage RIP.....", "in_reply_to_status": 736451736164933632, "in_reply_to_user": 2874302274, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2874302274 }}, "user": { "id": 494458165, "name": "الأمطار العدالة من ف", "screen_name": "karnasonofsun", "lang": "en", "location": "New York City", "create_at": date("2012-02-16"), "description": "Despite everything, it's still you. I Ria/M/21/NYC I QP- @CrookedTricking @plant_boys @mestizocritical I Ra's Maraq - Hyperion I Arjuna @ CA I icon by pako", "followers_count": 515, "friends_count": 651, "statues_count": 118438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957611585536, "text": "@jackielolwhat I am too!", "in_reply_to_status": 736422225868431360, "in_reply_to_user": 796464930, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 796464930 }}, "user": { "id": 2377290710, "name": "Abigale Montoya", "screen_name": "AbigaleMontoya", "lang": "en", "location": "null", "create_at": date("2014-03-07"), "description": "Drink up baby, stay up all night With the things you could do You won't but you might The potential you'll be - Between the bars ~ Elliot Smith", "followers_count": 189, "friends_count": 315, "statues_count": 872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-05-28T00:00:03.000Z"), "id": 736451957704011776, "text": "Wind 0.0 mph ---. Barometer 30.05 in, Steady. Temperature 69.3 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 123, "statues_count": 160114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451958488236032, "text": "sometimes you gotta leave it alone to see if they'll fight to keep you around", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3240978876, "name": "jazzy digital", "screen_name": "jassminewesst", "lang": "en", "location": "Oxnard, CA", "create_at": date("2015-06-09"), "description": "⁕ 18, if I was you I wouldn't like me either", "followers_count": 1051, "friends_count": 979, "statues_count": 5456 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451958630809600, "text": "@KingJay0__ ���� https://t.co/hSLzQ9ls3e", "in_reply_to_status": -1, "in_reply_to_user": 870246810, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 870246810 }}, "user": { "id": 449109431, "name": "jahlaina", "screen_name": "lainadominique", "lang": "en", "location": "null", "create_at": date("2011-12-28"), "description": "SJSU #dobetter #blackgurlmagic✨", "followers_count": 803, "friends_count": 421, "statues_count": 55807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451958815391745, "text": "hoping", "in_reply_to_status": 736102357000290304, "in_reply_to_user": 262439840, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 262439840, "name": "विशाखा", "screen_name": "VishakhaMallya", "lang": "en", "location": "atx/htx", "create_at": date("2011-03-07"), "description": "null", "followers_count": 442, "friends_count": 229, "statues_count": 17950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451958886670337, "text": "Maybe you were born with it ??¿? https://t.co/FctcYnoLzx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 490735435, "name": "Sydney.", "screen_name": "sydschad", "lang": "en", "location": "null", "create_at": date("2012-02-12"), "description": "i probably won't follow back", "followers_count": 531, "friends_count": 90, "statues_count": 29657 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rapid Valley, SD", "id": "6e23522ad2747dc8", "name": "Rapid Valley", "place_type": "city", "bounding_box": rectangle("-103.176512,44.040302 -103.125451,44.079364") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46103, "countyName": "Pennington", "cityID": 4653007, "cityName": "Rapid Valley" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451959113142276, "text": "@ariellemo91 @mojodos hey that's a good song", "in_reply_to_status": 736451836517879808, "in_reply_to_user": 90748721, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 90748721, 1537958437 }}, "user": { "id": 423984412, "name": "Daniel Montijo", "screen_name": "Dmontijo11", "lang": "en", "location": "$TKN ", "create_at": date("2011-11-28"), "description": "walkin', livin', breathin', now ya know my past well", "followers_count": 1041, "friends_count": 515, "statues_count": 47002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451960526798848, "text": "I don't even feel like myself anymore ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52413916, "name": "Bran Flakes", "screen_name": "XoXo_Briiiannne", "lang": "en", "location": "20 betch -- The Mittin ✋", "create_at": date("2009-06-30"), "description": "When you have something good, keep it.", "followers_count": 1455, "friends_count": 1810, "statues_count": 17806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walker, MI", "id": "c9d8828f259682c3", "name": "Walker", "place_type": "city", "bounding_box": rectangle("-85.787954,42.915342 -85.661793,43.059577") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2682960, "cityName": "Walker" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451960547598337, "text": "I'm still at work. someone please end me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 812152255, "name": "it liz what it liz", "screen_name": "LIZIOTZOV", "lang": "en", "location": "Bulgaria", "create_at": date("2012-09-08"), "description": "#XO", "followers_count": 596, "friends_count": 469, "statues_count": 8487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Scottsdale, AZ", "id": "0a0de7bd49ef942d", "name": "Scottsdale", "place_type": "city", "bounding_box": rectangle("-111.960775,33.435864 -111.756022,33.842816") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 465000, "cityName": "Scottsdale" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451960627298304, "text": "This is me af ���� I'm mean af https://t.co/rzD1sXRPhH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2721515261, "name": "Riyah", "screen_name": "ItsRiyahBitch", "lang": "en", "location": "null", "create_at": date("2014-07-23"), "description": "waiting on my time to come⏰ I'm the princess to many fakes so I rock with none I'm happy by myself", "followers_count": 528, "friends_count": 617, "statues_count": 3598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451960790884352, "text": "5/28/2016 - 02:00\nTemp: 64.1F \nHum: 98%\nWind: 0.0 mph\nBaro: 29.853in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 53, "statues_count": 53234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451960816193537, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Steady. Temperature 73.8 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451961071886337, "text": "@alexxiszm https://t.co/uXQFHmqR63", "in_reply_to_status": -1, "in_reply_to_user": 509513516, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 509513516 }}, "user": { "id": 713884530, "name": "Cristina", "screen_name": "xtinaageee", "lang": "en", "location": "Texas, USA", "create_at": date("2012-07-24"), "description": "✈️ ✈️ ✈️outta here", "followers_count": 1321, "friends_count": 369, "statues_count": 24479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451961332064256, "text": "@FoxNews HOORAH!", "in_reply_to_status": 736451431587184641, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 1367531 }}, "user": { "id": 4663677982, "name": "Kenneth mardenboroug", "screen_name": "KENDU125", "lang": "en", "location": "null", "create_at": date("2015-12-31"), "description": "null", "followers_count": 82, "friends_count": 192, "statues_count": 2787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Rochelle, NY", "id": "11cdb607ab2f17bb", "name": "New Rochelle", "place_type": "city", "bounding_box": rectangle("-73.807197,40.881294 -73.755414,40.988329") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3650617, "cityName": "New Rochelle" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451961608802305, "text": "@alo_usie52 downtown Finna turn up awe I though y'all was out", "in_reply_to_status": 736451898765512704, "in_reply_to_user": 4543971379, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4543971379 }}, "user": { "id": 39091280, "name": "RETIRED", "screen_name": "IamAfricaBlack", "lang": "en", "location": "Oakland, CA", "create_at": date("2009-05-10"), "description": "Go download rich niggas don't wear socks 2 below.", "followers_count": 684, "friends_count": 503, "statues_count": 115745 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-28T00:00:04.000Z"), "id": 736451961759752196, "text": "@BryanneIsInRF fuck yeah ������", "in_reply_to_status": 736451761179754496, "in_reply_to_user": 72214143, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 72214143 }}, "user": { "id": 20964148, "name": "ashley", "screen_name": "dairybarton", "lang": "en", "location": "the bay", "create_at": date("2009-02-15"), "description": "You fell in the sun, love. ☀", "followers_count": 365, "friends_count": 864, "statues_count": 19339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451962162421766, "text": "@MuirHailee I just can't get you, off my mind", "in_reply_to_status": 736441661643837441, "in_reply_to_user": 2366886481, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2366886481 }}, "user": { "id": 2630536173, "name": "parker", "screen_name": "parkerrjanee", "lang": "en", "location": "Seattle, WA", "create_at": date("2014-06-21"), "description": "sc: parker_conley", "followers_count": 239, "friends_count": 244, "statues_count": 1000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, WA", "id": "625eb47b5e233645", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-122.335786,47.340391 -122.291094,47.431114") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5317635, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451962200199168, "text": "Trouble- Halsey Sander Kleinenberg Remix��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 1334722789, "name": "Queen Bitch", "screen_name": "Flowerprinc3ss", "lang": "en", "location": "The Ocean", "create_at": date("2013-04-07"), "description": "Mommy to a beautiful Princess ❤️", "followers_count": 271, "friends_count": 374, "statues_count": 8496 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451963320193024, "text": "https://t.co/ieCN9Bmnao\n@DexLinu \nReady for this one yet?\n\"Do I have a choice?\"\n@BarackObama \nSadly no....\n\"XDDDDD\"\nMasochist XD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 728315214299594752, 813286 }}, "user": { "id": 3130823217, "name": "Hank", "screen_name": "AdamsWritings", "lang": "en", "location": "Michigan", "create_at": date("2015-03-31"), "description": "Avid poet. Aspiring author.", "followers_count": 265, "friends_count": 757, "statues_count": 3183 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hammond, IN", "id": "99700cd6fc455c13", "name": "Hammond", "place_type": "city", "bounding_box": rectangle("-87.525341,41.566265 -87.432288,41.710116") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1831000, "cityName": "Hammond" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451963785773056, "text": "Boo scotch plains cops boo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 389077626, "name": "TC Weber", "screen_name": "TC_weber9310", "lang": "en", "location": "Warren, NJ", "create_at": date("2011-10-11"), "description": "Fairleigh Dickinson University Baseball 2019", "followers_count": 642, "friends_count": 488, "statues_count": 23755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainfield, NJ", "id": "c735e25ad03113d5", "name": "Plainfield", "place_type": "city", "bounding_box": rectangle("-74.463243,40.594821 -74.382785,40.64434") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3459190, "cityName": "Plainfield" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451963924164608, "text": "I can't fw it ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 963073321, "name": "Briana", "screen_name": "likeWhoaa____", "lang": "en", "location": "Decatur, AL", "create_at": date("2012-11-21"), "description": "you know I got the sauce, you know I'm saucyyy✨", "followers_count": 943, "friends_count": 1032, "statues_count": 11229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, AL", "id": "246fb652d518385d", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-87.095676,34.507116 -86.925426,34.654734") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1103, "countyName": "Morgan", "cityID": 120104, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451964028997632, "text": "they don't understand that. https://t.co/eAJ92RKDrF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332759686, "name": "Jay❤️", "screen_name": "jmeraki", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2014-02-07"), "description": "you searching for answers, I do not know nothing. | sc: jaylenbreanna", "followers_count": 1201, "friends_count": 824, "statues_count": 37011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451964255502336, "text": "Denise Huxtable was the finest woman to ever walk the earth https://t.co/NiN4umqJtU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 155251517, "name": "Yung Asthmatic", "screen_name": "NWO_Bryant", "lang": "en", "location": "Chicago, IL | Washington, DC", "create_at": date("2010-06-13"), "description": "Student, Producer, your favorite cyber savage | AU 2019", "followers_count": 412, "friends_count": 320, "statues_count": 1195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gaithersburg, MD", "id": "db1e795d2641873e", "name": "Gaithersburg", "place_type": "city", "bounding_box": rectangle("-77.252801,39.102707 -77.163064,39.169487") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2431175, "cityName": "Gaithersburg" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451965081657347, "text": "I'm gonna miss my lil niggas @Mustafah__ @OmaaarA7 @domsousaaa @Serign", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 752023322, 796981278, 2339216316, 110227085 }}, "user": { "id": 545690369, "name": "Tinker", "screen_name": "MalikDMslider", "lang": "en", "location": "45 Ave Montaigne,Paris, France", "create_at": date("2012-04-04"), "description": "Happiness, Health and Wealth", "followers_count": 529, "friends_count": 369, "statues_count": 35818 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain House, CA", "id": "014b25c473600d50", "name": "Mountain House", "place_type": "city", "bounding_box": rectangle("-121.557176,37.766633 -121.533126,37.795236") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 649582, "cityName": "Mountain House" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451965115203584, "text": "@avasummers20 I feel you on that", "in_reply_to_status": 736451733665153024, "in_reply_to_user": 3319655622, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3319655622 }}, "user": { "id": 1126676768, "name": "JesusFreak", "screen_name": "kboi_nation", "lang": "en", "location": "playing baseball", "create_at": date("2013-01-27"), "description": "I will be a successful entrepreneur. Utility player.", "followers_count": 810, "friends_count": 368, "statues_count": 17331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crosby, TX", "id": "00a488ec567255f3", "name": "Crosby", "place_type": "city", "bounding_box": rectangle("-95.12133,29.899357 -95.04072,29.942391") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4817756, "cityName": "Crosby" } }
+{ "create_at": datetime("2016-05-28T00:00:05.000Z"), "id": 736451965656305665, "text": "Wind 0.0 mph SW. Barometer 29.92 in, Steady. Temperature 56.5 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.13333333,33.8"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325072400, "name": "Dan McCutcheon", "screen_name": "Invitrodan", "lang": "en", "location": "Temecula, CA", "create_at": date("2011-06-27"), "description": "null", "followers_count": 27, "friends_count": 112, "statues_count": 15145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 652624, "cityName": "Nuevo" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814328964218880, "text": "@__yaya22 you know something wrong with lil fizz ����", "in_reply_to_status": 736808626199875584, "in_reply_to_user": 2175807022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2175807022 }}, "user": { "id": 331387164, "name": "QueenC", "screen_name": "youngg_shayy", "lang": "en", "location": "null", "create_at": date("2011-07-07"), "description": "might be young but I got a plan !!", "followers_count": 787, "friends_count": 824, "statues_count": 20942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ocean City, MD", "id": "759c5982d28a7b07", "name": "Ocean City", "place_type": "city", "bounding_box": rectangle("-75.091333,38.32431 -75.04914,38.451299") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24047, "countyName": "Worcester", "cityID": 2458225, "cityName": "Ocean City" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814328972742657, "text": "@stuartenyt @tcm @StevenHerbert • right now Monty & his right cross #FromHereToEternity", "in_reply_to_status": 736811949544079361, "in_reply_to_user": 107234633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "FromHereToEternity" }}, "user_mentions": {{ 107234633, 25071402, 15778923 }}, "user": { "id": 308266073, "name": "Chuck Brouillette", "screen_name": "ChuckBrouilette", "lang": "en", "location": "Saratoga Springs, NY", "create_at": date("2011-05-30"), "description": "Artist/Writer creating/composing from comic media/history/pop culture", "followers_count": 252, "friends_count": 350, "statues_count": 11980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Saratoga Springs, NY", "id": "d606af411edd5297", "name": "Saratoga Springs", "place_type": "city", "bounding_box": rectangle("-73.848875,43.020588 -73.684637,43.11327") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36091, "countyName": "Saratoga", "cityID": 3665255, "cityName": "Saratoga Springs" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329157328896, "text": "Temp: 64.9F W C: 64.9F Wind:--- at 0.0kts Baro: 1017.9mb and Falling slowly Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 214, "friends_count": 218, "statues_count": 106449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329358483457, "text": "@Old_Bern_Kenobi @The_book_girl https://t.co/xhayAnN3aq", "in_reply_to_status": 736812857807896579, "in_reply_to_user": 68004485, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4889520335, 17054728 }}, "user": { "id": 68004485, "name": "Ojos Criollos", "screen_name": "ojoscriollos", "lang": "en", "location": "New Orleans", "create_at": date("2009-08-22"), "description": "As a feminist, I won't vote for a female neoliberal w/ neocon foreign policy. Can we stop pretending the U.S. is a democracy & make it one? #NeverHillary", "followers_count": 1217, "friends_count": 1413, "statues_count": 12170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329627070464, "text": "Popeyes would be so good right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.07036633,40.72352921"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43287341, "name": "Beans Da Gawd", "screen_name": "SkyWalkerjonez", "lang": "en", "location": "Outcheaville", "create_at": date("2009-05-28"), "description": "Jersey City\n\nDon't trust them new niggas over yonder -Uncle Ruckus-\n\n\n#SkyWalkerSociety #BennieTaughtMe\nSnapchat= BeansDaGee", "followers_count": 1197, "friends_count": 869, "statues_count": 66009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey City, NJ", "id": "b046074b1030a44d", "name": "Jersey City", "place_type": "city", "bounding_box": rectangle("-74.107995,40.666879 -74.025331,40.769124") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34017, "countyName": "Hudson", "cityID": 3436000, "cityName": "Jersey City" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329731784706, "text": "I can't believe I live a perfect life https://t.co/AbvBqa1lcb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2833356094, "name": "natalie", "screen_name": "natwhatevs", "lang": "en", "location": "null", "create_at": date("2014-10-15"), "description": "can't take the kid from the fight.| @cmisados", "followers_count": 1051, "friends_count": 126, "statues_count": 25785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329924722689, "text": "a free water from mcdonalds https://t.co/umY1UaPvbv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1491569809, "name": "Zhiloh.", "screen_name": "DamnKmar", "lang": "en", "location": "null", "create_at": date("2013-06-07"), "description": "married to $kinnyhardawa¥", "followers_count": 2539, "friends_count": 1903, "statues_count": 121785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814329962598401, "text": "Red wine + ambien, you're talkin' shit again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143188796, "name": "Madison", "screen_name": "Maddy_Sundwall", "lang": "en", "location": "CCSU", "create_at": date("2010-05-12"), "description": "among the whispers and the champagne and the stars. | ΦΣΣ | nm ♡ rw |", "followers_count": 591, "friends_count": 325, "statues_count": 40299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Britain, CT", "id": "f080a098cad5b04d", "name": "New Britain", "place_type": "city", "bounding_box": rectangle("-72.826538,41.643749 -72.748011,41.715995") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 950370, "cityName": "New Britain" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814330012827648, "text": "After 5 months I finally got to see my boo and I couldn't be happier. ❤", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2807913800, "name": "lexis.", "screen_name": "areynaa1952", "lang": "en", "location": "null", "create_at": date("2014-09-13"), "description": "null", "followers_count": 321, "friends_count": 208, "statues_count": 5734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Cruces, NM", "id": "f515486276aa6192", "name": "Las Cruces", "place_type": "city", "bounding_box": rectangle("-106.843427,32.238361 -106.63145,32.446238") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35013, "countyName": "Do?a Ana", "cityID": 3539380, "cityName": "Las Cruces" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814330251902976, "text": "At my crib I've got some pizza, plus a little bit of weed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65420296, "name": "[A]Money", "screen_name": "_Acruz56", "lang": "en", "location": "Chicago, IL ", "create_at": date("2009-08-13"), "description": "19| Paradise Cove | 4414", "followers_count": 474, "friends_count": 723, "statues_count": 14316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814331057209344, "text": "https://t.co/X9zagSQDEC", "in_reply_to_status": 736814006418051072, "in_reply_to_user": 117926237, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 117926237, "name": "taylor q", "screen_name": "tayyq", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-02-26"), "description": "null", "followers_count": 575, "friends_count": 235, "statues_count": 7372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814331191402496, "text": "ابيك عني ماتغيب !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 326390888, "name": "بو لورا القبندي", "screen_name": "A_AL_QABANDII", "lang": "en", "location": "kuwait ❤️", "create_at": date("2011-06-29"), "description": "الحمدالله رب العالمين #قائمة_التغيير ' operator in knpc ' instgram : Don.alii Snapchat : don_qabandi #Halamadrid❤️", "followers_count": 800, "friends_count": 64, "statues_count": 31301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814331875098624, "text": "6 drinks deep and I've only been here for 30 minutes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2943458779, "name": "D.", "screen_name": "illboydom", "lang": "en", "location": "Henderson, NV", "create_at": date("2014-12-25"), "description": "null", "followers_count": 200, "friends_count": 379, "statues_count": 1227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814331992539136, "text": "A customer asked me why am I gonna dye my hair. Like bitch, You obviously didn't want to wear deodorant today, so you can stfu ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2290310647, "name": "young god", "screen_name": "kawaiiordie_", "lang": "en", "location": "Kapolei", "create_at": date("2014-01-13"), "description": "IG: @kawaiiordie", "followers_count": 91, "friends_count": 147, "statues_count": 2627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ko Olina, HI", "id": "011f8b0a63816c7a", "name": "Ko Olina", "place_type": "city", "bounding_box": rectangle("-158.1293,21.322963 -158.089509,21.349347") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1539400, "cityName": "Ko Olina" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814332089098240, "text": "Ripley SW Limestone Co. Temp: 66.4°F Wind:0.0mph Pressure: 998.5mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53870 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814332328042496, "text": "got her hair done french braids now she ASAP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3189249366, "name": "evelyn", "screen_name": "lilevexcvi", "lang": "en", "location": "♡", "create_at": date("2015-05-08"), "description": "yo what up you fucking cabbage...", "followers_count": 182, "friends_count": 144, "statues_count": 7711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paramount, CA", "id": "7d2aec133a24b554", "name": "Paramount", "place_type": "city", "bounding_box": rectangle("-118.188188,33.879813 -118.142651,33.918812") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655618, "cityName": "Paramount" } }
+{ "create_at": datetime("2016-05-29T00:00:00.000Z"), "id": 736814332881862657, "text": "#HomeSafe (@ Hello Kitty Lisa Ann Land in Hillsborough, NJ) https://t.co/3GHiCsg0kb https://t.co/68l1oIg4Iz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.65989902,40.49883278"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HomeSafe" }}, "user": { "id": 2252038083, "name": "Lisa Ann", "screen_name": "HKLisaAnn", "lang": "en", "location": "New Jersey, USA", "create_at": date("2013-12-30"), "description": "Sex, Love, Zeppelin, and of course Hello Kitty!! @HKLisaAnn #HKForever #HKLisaAnn", "followers_count": 591, "friends_count": 2264, "statues_count": 22255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsborough, NJ", "id": "000a76c020fc19ce", "name": "Hillsborough", "place_type": "city", "bounding_box": rectangle("-74.693675,40.455306 -74.571998,40.547849") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34035, "countyName": "Somerset" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814333250797568, "text": ":-///// ALWAYS https://t.co/mVOY3pts1e", "in_reply_to_status": 736812250699202561, "in_reply_to_user": 2421408218, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2421408218, "name": "Natalie", "screen_name": "sadalieee", "lang": "en", "location": "California ", "create_at": date("2014-03-31"), "description": "I love Drake.", "followers_count": 1413, "friends_count": 766, "statues_count": 76546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynwood, CA", "id": "fcd177d5a0545c9c", "name": "Lynwood", "place_type": "city", "bounding_box": rectangle("-118.230888,33.905084 -118.175869,33.94544") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644574, "cityName": "Lynwood" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814333406117888, "text": "Wind 0.0 mph ---. Barometer 30.087 in, Rising slowly. Temperature 67.8 °F. Rain today 0.00 in. Humidity 84%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 39, "friends_count": 4, "statues_count": 27766 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814333791965184, "text": "My mama literally gave me a month to be unproductive lmao.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62454536, "name": "Flashing Green Light", "screen_name": "HannahBurnellW", "lang": "en", "location": "Thee Throne", "create_at": date("2009-08-02"), "description": "I didn't choose the pageant, the pageant life chose me -Honey BooBoo", "followers_count": 1500, "friends_count": 1646, "statues_count": 33842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814333863153664, "text": "@ebbtideapp Tide in Brant Rock, Massachusetts 05/29/2016\nHigh 5:21am 9.6\n Low 11:42am 0.2\nHigh 6:00pm 9.4\n Low 12:09am 0.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.6467,42.0833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 95, "friends_count": 1, "statues_count": 36791 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshfield, MA", "id": "74f5dce716bb4008", "name": "Marshfield", "place_type": "city", "bounding_box": rectangle("-70.772948,42.061623 -70.638438,42.162318") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2538820, "cityName": "Marshfield" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334106427392, "text": "And drunk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95997318, "name": "Ana ☺️", "screen_name": "Dorseys_AnaV", "lang": "en", "location": "probably at a concert ", "create_at": date("2009-12-10"), "description": "I got hot sauce in my bag, swag", "followers_count": 1347, "friends_count": 955, "statues_count": 53252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334140026880, "text": "Got my girl @Latashaleesing on the tunes while we chillin w da famz:)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56585491 }}, "user": { "id": 77422752, "name": "❤️Tea.Ana.Kenna♥️", "screen_name": "Memoli_11", "lang": "en", "location": "Seattle, Washington", "create_at": date("2009-09-26"), "description": "God n Family is my whole heart:) #SamueluBloodline #PomaleClan #LupusWarrior #MisiRadio add me on sc: memzie11", "followers_count": 677, "friends_count": 750, "statues_count": 46770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Center, WA", "id": "3a201fc1ccc493b8", "name": "White Center", "place_type": "city", "bounding_box": rectangle("-122.374253,47.488444 -122.324769,47.517465") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5378225, "cityName": "White Center" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334332936192, "text": "Wind 0.0 mph ---. Barometer 1020.22 mb, Rising. Temperature 62.9 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 32, "friends_count": 128, "statues_count": 14724 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334551150592, "text": "Temp 60.9°F Wind Chill 60.9°F RH 88% Wind 0.1 NW Gust 2.0 NW SLP 30.093 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334647492610, "text": "Missed my Bestfriend ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2180575147, "name": "Ki.", "screen_name": "_DearKiara", "lang": "en", "location": "zy was here ", "create_at": date("2013-11-07"), "description": "Working on myself, for myself. (still love u)", "followers_count": 2224, "friends_count": 1366, "statues_count": 22151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814334672801792, "text": "@DistrictofGnat @guythatdances why y'all got me all?���� https://t.co/iCiGzDcNif", "in_reply_to_status": 736812984874459136, "in_reply_to_user": 20346626, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 20346626, 312725401 }}, "user": { "id": 152448004, "name": "Jay Jurden", "screen_name": "JayJurden", "lang": "en", "location": "New York, NY", "create_at": date("2010-06-05"), "description": "NYC Actor. Comedian. Trainer. \n@DavidBartonGym - Astor Place. \nACE Certifed Personal Trainer.\nemail - bodybyjaytraining@gmail\ninstagram/snapchat - @JayJurden", "followers_count": 390, "friends_count": 258, "statues_count": 13551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814335024979968, "text": "Im off this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 404460968, "name": "Dillon D", "screen_name": "Drizzllen", "lang": "en", "location": "null", "create_at": date("2011-11-03"), "description": "The spirit is infinite within", "followers_count": 619, "friends_count": 358, "statues_count": 18556 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814335129849856, "text": "@TriciaAnne_ I smoked trying to make me sleepy it made me turnt", "in_reply_to_status": 736809953734823936, "in_reply_to_user": 307184554, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 307184554 }}, "user": { "id": 3254020474, "name": "lil intro vert", "screen_name": "aisisklopez", "lang": "en", "location": "508✈305", "create_at": date("2015-05-14"), "description": "♓️ Don't be a hard rock when you really are a gem.", "followers_count": 398, "friends_count": 292, "statues_count": 9134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Deerfield Beach, FL", "id": "4ebdbc556ccd2f12", "name": "Deerfield Beach", "place_type": "city", "bounding_box": rectangle("-80.170343,26.274467 -80.074368,26.327929") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1216725, "cityName": "Deerfield Beach" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814335146745856, "text": "05/29@03:00 - Temp 68.6F, WC 68.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.153in, Falling slowly. Rain 0.00in. Hum 77%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814335180279808, "text": "Temp: 68.3°F Wind:0.0mph Pressure: 30.053hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814335738056704, "text": "Interested in a #Healthcare #job near #Glasgow, KY? This could be a great fit: https://t.co/2Np1LOGuGJ #Hiring https://t.co/JbZPWWs8L6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.9119215,36.9958839"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Healthcare", "job", "Glasgow", "Hiring" }}, "user": { "id": 485013829, "name": "AHCStaff", "screen_name": "AccountableHS", "lang": "en", "location": "Boca Raton, Florida", "create_at": date("2012-02-06"), "description": "Accountable #Healthcare Staffing is a top rated national staffing corporation. We provide #PerDiem #Locums #PRMContract #travelnursing & #travelhealthcare", "followers_count": 802, "friends_count": 167, "statues_count": 13519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glasgow, KY", "id": "657542099f5d2459", "name": "Glasgow", "place_type": "city", "bounding_box": rectangle("-85.964805,36.958955 -85.885613,37.030192") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21009, "countyName": "Barren", "cityID": 2131114, "cityName": "Glasgow" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814336639897600, "text": "I was tryna change a white boy's life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 62194650, "name": "Tookie Trilliams", "screen_name": "YOUNGxJAN", "lang": "en", "location": "... taking over for the '99...", "create_at": date("2009-08-01"), "description": "ZoeRicua. Domestic Partnership Shawty. Detroit Player. Southfield Savage. Houston Swanger.", "followers_count": 1129, "friends_count": 360, "statues_count": 203291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814336715325440, "text": "@BettAThanLERUNS this Doom you want some comic books bruh", "in_reply_to_status": -1, "in_reply_to_user": 1278661454, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1278661454 }}, "user": { "id": 279417193, "name": "Washed Meng", "screen_name": "Viktor_kon_Doom", "lang": "en", "location": "Alexandria, LA", "create_at": date("2011-04-08"), "description": "If Pentagon Jr. Don't trust you I'm shoot you", "followers_count": 434, "friends_count": 527, "statues_count": 23962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814336753033216, "text": "@andreaak_ �� wth", "in_reply_to_status": 736814038001192960, "in_reply_to_user": 2569289614, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2569289614 }}, "user": { "id": 335653427, "name": "emmanuel", "screen_name": "e_tastic24", "lang": "en", "location": "405", "create_at": date("2011-07-14"), "description": "a future Nigerian entrepreneur #13 on the field ⚽️ #okstate20", "followers_count": 1283, "friends_count": 278, "statues_count": 39013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Del City, OK", "id": "7e831b14a06a5759", "name": "Del City", "place_type": "city", "bounding_box": rectangle("-97.459517,35.420738 -97.423502,35.478707") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4019900, "cityName": "Del City" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814336794984448, "text": "Lmaoooooo https://t.co/5xsgS8vGZZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 4852040829, "name": "[GAV] #Moviez!", "screen_name": "GavGod__", "lang": "en", "location": "#$oIceyTommyBoiz", "create_at": date("2016-01-26"), "description": "GavGod Got Tha Vi$ion ™ #MPSA", "followers_count": 182, "friends_count": 329, "statues_count": 711 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814337008885760, "text": "@islandbabay betta not be runnin up. finna be catchin these fam hands", "in_reply_to_status": -1, "in_reply_to_user": 713150452460695552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 713150452460695552 }}, "user": { "id": 3158359520, "name": "jordyn", "screen_name": "jordynanturtle", "lang": "en", "location": "null", "create_at": date("2015-04-15"), "description": "ahs '19 | Coleslaw is my favorite", "followers_count": 225, "friends_count": 209, "statues_count": 1698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814337013252096, "text": "Wind 0.0 mph ---. Barometer 30.059 in, Steady. Temperature 67.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-29T00:00:01.000Z"), "id": 736814337021464576, "text": "LMAOOOO now E.. you gotta participate in your own flourishing beloved �� @ErikaBaDoIt", "in_reply_to_status": 736814133241360385, "in_reply_to_user": 2602857328, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2602857328 }}, "user": { "id": 4688630628, "name": "J O S H", "screen_name": "ThoseCooIKids", "lang": "en", "location": "New Dorp", "create_at": date("2016-01-01"), "description": "⠀⠀But the spirit, the will to excel, the will to ⠀⠀⠀win.. these are the things that endure.", "followers_count": 6831, "friends_count": 196, "statues_count": 38136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814337243918337, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 30.158 in, Steady. Temp 63.5F. Rain today 0.00 in. Humidity 69% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 625, "friends_count": 838, "statues_count": 43150 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814337956810752, "text": "Mili https://t.co/0kmbMwahoJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1031803454, "name": "Tia✨", "screen_name": "titamargiela", "lang": "en", "location": "Bayamoncito, Puerto Rico", "create_at": date("2012-12-23"), "description": "null", "followers_count": 634, "friends_count": 351, "statues_count": 21404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338024083456, "text": "@EmceeRob https://t.co/kmGciwRwP2", "in_reply_to_status": -1, "in_reply_to_user": 2995563200, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2995563200 }}, "user": { "id": 2835779254, "name": "Cd.", "screen_name": "cedariusluttery", "lang": "en", "location": "Cleveland, OH", "create_at": date("2014-10-17"), "description": "Only Originals Records Artist | Business Inquiries: cluttery94@gmail.com | Follow Twitter & Instagram: @cedariusluttery\n#90sbaby #onlyoriginals", "followers_count": 1118, "friends_count": 871, "statues_count": 38554 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338145718272, "text": "Wind 0.0 mph ---. Barometer 29.986 in, Rising. Temperature 68.8 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338158137344, "text": "Just got home from Sac, where it at��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940655052, "name": "Ronaldo⚡️", "screen_name": "_fakeasseddie", "lang": "en", "location": "null", "create_at": date("2014-12-23"), "description": "RIP Uncle Tony | IG: _fakeasseddie | BlowMoney", "followers_count": 285, "friends_count": 246, "statues_count": 9956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338355265537, "text": "@proetential the inevitable collapse of the livestock industry", "in_reply_to_status": 736792381400354822, "in_reply_to_user": 2509500991, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2509500991 }}, "user": { "id": 740018172, "name": "Marcus Pipitone", "screen_name": "MarcusPipitone", "lang": "en", "location": "null", "create_at": date("2012-08-05"), "description": "Granada High School | Chapman '20", "followers_count": 652, "friends_count": 382, "statues_count": 11069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338468544512, "text": "Omg happy bday to brissa @brrisssaaa she's getting old omg ������hope ur day goes good ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3313661574 }}, "user": { "id": 4805724157, "name": "$LIM B.....", "screen_name": "mr_october_2", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2016-01-23"), "description": "one and just one......", "followers_count": 467, "friends_count": 879, "statues_count": 3645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338653069314, "text": "so fckin mad i just broke my nail dawg wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2219240166, "name": "⠀⠀⠀", "screen_name": "DAEEES_", "lang": "en", "location": "null", "create_at": date("2013-11-28"), "description": "null", "followers_count": 3229, "friends_count": 867, "statues_count": 72205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814338774863872, "text": "Lol hell Na! Ima come see y'all Monday https://t.co/wZU3v84N75", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348775520, "name": "tha worst", "screen_name": "dejawajaa", "lang": "en", "location": "MS", "create_at": date("2011-08-04"), "description": "God thinks I am to die for .. |luvyou2munii| #yellowworld", "followers_count": 1516, "friends_count": 542, "statues_count": 76325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgeland, MS", "id": "78439f3b905720e3", "name": "Ridgeland", "place_type": "city", "bounding_box": rectangle("-90.181179,32.399351 -90.067498,32.459434") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28089, "countyName": "Madison", "cityID": 2862520, "cityName": "Ridgeland" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814339718422529, "text": "The night is just starting what do you mean you want to go home ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2389272710, "name": "Antiguini Sanchez", "screen_name": "antiguini", "lang": "en", "location": "Elko, NV", "create_at": date("2014-03-14"), "description": "somethings you just got to let go....", "followers_count": 428, "friends_count": 497, "statues_count": 5346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elko, NV", "id": "a3b39b40a6f077f5", "name": "Elko", "place_type": "city", "bounding_box": rectangle("-115.831165,40.801431 -115.709259,40.883884") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32007, "countyName": "Elko", "cityID": 3222500, "cityName": "Elko" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814339919732736, "text": "\"Surround yourself with success and eventually, you'll have no choice but to live up to the standard set by your environment.\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 902287063, "name": "Johnny V.", "screen_name": "jvillegas_7", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2012-10-24"), "description": "A&M-Commerce ΠΚΑ | Public Relations | Lion Cheer #TTS | Coach at @OlympoGymnastic | Pre-PT | Stay Young Stay Fit", "followers_count": 1761, "friends_count": 1498, "statues_count": 23539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340360134656, "text": "Loyalty and understanding >>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4456028954, "name": "Water Monkey.", "screen_name": "AmandaAgarrat", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-12-04"), "description": "A&R at Trenddef Studios. I'm in love with a mermaid. Click this link ~ http://westwoodvillageent.com", "followers_count": 186, "friends_count": 228, "statues_count": 6319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340414660608, "text": "Here's to the nights we won't remember and the friends we won't forget - Lee Brice ❤️�� https://t.co/uO1Cs1iaM0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 565267005, "name": "Gracieツ", "screen_name": "graciepaigeblac", "lang": "en", "location": "Pensacola, Fl", "create_at": date("2012-04-27"), "description": "Be wild while you can ;)) Calvin Cameron♡", "followers_count": 635, "friends_count": 1111, "statues_count": 6085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzalez, FL", "id": "1a0cf025d0716f3d", "name": "Gonzalez", "place_type": "city", "bounding_box": rectangle("-87.325789,30.547501 -87.260678,30.619707") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12033, "countyName": "Escambia", "cityID": 1226700, "cityName": "Gonzalez" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340427288578, "text": "@tesiafy You're not wrong.", "in_reply_to_status": 736814292075319296, "in_reply_to_user": 187736524, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 187736524 }}, "user": { "id": 3121402154, "name": "Megan", "screen_name": "meganrekceps", "lang": "en", "location": "null", "create_at": date("2015-03-31"), "description": "PHX. Probably too weird for you.", "followers_count": 205, "friends_count": 114, "statues_count": 3594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340431437825, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":66.6°F Wind:1.6mph Pressure: 30.15hpa Steady Rain Today 0.00in. Forecast: Fine, possible showers", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 18, "statues_count": 320465 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340955770880, "text": "Lol @ everyone that didn't give me the chance, because when I get married my relationship is gonna be goals.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 413590461, "name": "♠️♥Sum Buck♥️♠️", "screen_name": "holleygates", "lang": "en", "location": "Abbott, Tx", "create_at": date("2011-11-15"), "description": "Just a cowboy selling cars•254-339-5525•Cmon withcha Cmon•Make more than your boyfriend•Im2nd•Spurrin since '96•Intagram:@sum_buck•Snapchat:@cowboy_swagx96", "followers_count": 2941, "friends_count": 1874, "statues_count": 17735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814340997808128, "text": "@ThankMeL8er it's the summer I don't where shoes", "in_reply_to_status": 736813813157269504, "in_reply_to_user": 357699979, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 357699979 }}, "user": { "id": 979850539, "name": "Juan", "screen_name": "Delgadojuan95", "lang": "en", "location": "null", "create_at": date("2012-11-29"), "description": "Kanye Disciple #TheLifeofPablo. Rocket League Expert", "followers_count": 339, "friends_count": 394, "statues_count": 9197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Grove, IL", "id": "0010043e0083ea4a", "name": "Sugar Grove", "place_type": "city", "bounding_box": rectangle("-88.489185,41.747143 -88.375384,41.805351") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17089, "countyName": "Kane", "cityID": 1773391, "cityName": "Sugar Grove" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814341081563140, "text": "I knew when I got the weed I was gone be outta there cause that bitch was grey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2369030689, "name": "Trevon✏️", "screen_name": "TWilliams24_", "lang": "en", "location": "Cedar Hill, TX", "create_at": date("2014-03-02"), "description": "College Student .. I'm ugly but I'll treat you right #TrueStory", "followers_count": 1528, "friends_count": 955, "statues_count": 58906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814341090123776, "text": "Wind 0.0 mph ---. Barometer 30.11 in, Steady. Temperature 68.2 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 29, "friends_count": 123, "statues_count": 160138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814341115154432, "text": "i go down on a dyke and go up on my price", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3220994388, "name": ".", "screen_name": "vibewithash", "lang": "en", "location": "laying low", "create_at": date("2015-05-19"), "description": "not ya bitch , out da loop", "followers_count": 369, "friends_count": 253, "statues_count": 13504 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rollingwood, CA", "id": "f89fbc750badbd78", "name": "Rollingwood", "place_type": "city", "bounding_box": rectangle("-122.337088,37.962063 -122.322276,37.968327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 662700, "cityName": "Rollingwood" } }
+{ "create_at": datetime("2016-05-29T00:00:02.000Z"), "id": 736814341266235392, "text": "Temp 67.1° Hi/Lo 72.1/67.1 Rng 5.0° WC 67.1° Hmd 93% Rain 0.00\" Storm 0.00\" BAR 30.141 Rising DP 65.0° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 70, "friends_count": 123, "statues_count": 19071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814341861736448, "text": "Happy New Year!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981479394, "name": "Brayden Battee", "screen_name": "BatteeBrayden", "lang": "en", "location": "California, USA", "create_at": date("2015-01-13"), "description": "18. Virgo. Apple Valley. Life. Love. Happiness. Fitness. Septembers Own. Enthusiast.", "followers_count": 678, "friends_count": 646, "statues_count": 3836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, CA", "id": "41c331abe42d9969", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-117.287533,34.413411 -117.103437,34.571843") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 602364, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814342331498496, "text": "this bar playing all old Kanye West right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29057705, "name": "Laurie Reese", "screen_name": "lauriesmooz", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-04-05"), "description": "Telling them what's good since 1994", "followers_count": 509, "friends_count": 297, "statues_count": 6197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814342360813568, "text": "is ma birthdayyyy������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2299146858, "name": "lex", "screen_name": "Lexieeee10", "lang": "en", "location": "null", "create_at": date("2014-01-18"), "description": "♡♡", "followers_count": 728, "friends_count": 858, "statues_count": 16801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814342524391424, "text": "Might just pull up nigga, wearin all supreme", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3279221294, "name": "Deleon", "screen_name": "vlone300", "lang": "en", "location": "null", "create_at": date("2015-07-13"), "description": "young handsome jiggy nigga", "followers_count": 1071, "friends_count": 583, "statues_count": 23219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814342734118912, "text": "@lencyaguilar ����we really did", "in_reply_to_status": 736803573351120896, "in_reply_to_user": 412860693, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 412860693 }}, "user": { "id": 2349506737, "name": "bibi", "screen_name": "bibimishelle", "lang": "en", "location": "null", "create_at": date("2014-02-17"), "description": "just clueless and tired", "followers_count": 634, "friends_count": 321, "statues_count": 9693 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burlingame, CA", "id": "9cee92fec370baf9", "name": "Burlingame", "place_type": "city", "bounding_box": rectangle("-122.403936,37.568111 -122.330819,37.604114") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 609066, "cityName": "Burlingame" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814342977376258, "text": "Seeing Jose is always great he's my fav man��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2204170146, "name": "danielle", "screen_name": "dxniellle", "lang": "en", "location": "null", "create_at": date("2013-11-19"), "description": "narbonne", "followers_count": 420, "friends_count": 200, "statues_count": 22326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814343103225856, "text": "Wind 1.6 mph SE. Barometer 29.991 in, Rising slowly. Temperature 73.1 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 14, "friends_count": 54, "statues_count": 8780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814343162040320, "text": "Today was wild. Im fucking exhausted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319272309, "name": "Cornrow Kenny", "screen_name": "Baker_Boy17", "lang": "en", "location": "Paradise ", "create_at": date("2011-06-17"), "description": "Smoke Gang", "followers_count": 947, "friends_count": 2038, "statues_count": 75770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bullhead City, AZ", "id": "719e93b07f954b3a", "name": "Bullhead City", "place_type": "city", "bounding_box": rectangle("-114.644984,35.039554 -114.499729,35.191096") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave", "cityID": 408220, "cityName": "Bullhead City" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814343459901440, "text": ":facepalm:\nhttps://t.co/7qUdoeXfX3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1620846667, "name": "David Wovchko", "screen_name": "DavidWovchko", "lang": "en", "location": "Pittsburgh, Pennsylvania", "create_at": date("2013-07-25"), "description": "I want to see people ‘out of the dark, out of danger, and out of impoverishment. #Voluntarism #TVOT", "followers_count": 284, "friends_count": 3341, "statues_count": 1935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crafton, PA", "id": "4226c67e22d968eb", "name": "Crafton", "place_type": "city", "bounding_box": rectangle("-80.086405,40.423722 -80.057056,40.442173") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4216848, "cityName": "Crafton" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814343858311168, "text": "Baltimore Pain Relief Center - https://t.co/pNGkUM5zVe\n\n#baltimorepainreliefcenter #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-76.73136,39.377623"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "baltimorepainreliefcenter", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 588, "friends_count": 576, "statues_count": 2438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pikesville, MD", "id": "31cb4c01e22bc01d", "name": "Pikesville", "place_type": "city", "bounding_box": rectangle("-76.755731,39.354406 -76.653005,39.417073") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24005, "countyName": "Baltimore", "cityID": 2461400, "cityName": "Pikesville" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814344248430592, "text": "Southwest Pass (2) (1839) https://t.co/i5WQ6qv4cY #southwestpass #phares #louisiana #unitedstates https://t.co/kWzI8LdZWu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.388519,28.97998"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "southwestpass", "phares", "louisiana", "unitedstates" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 965, "friends_count": 1288, "statues_count": 5784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22075, "countyName": "Plaquemines" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814344445403136, "text": "@kailaslayed how so ? ��", "in_reply_to_status": 736812274011275264, "in_reply_to_user": 2340370939, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2340370939 }}, "user": { "id": 313585733, "name": "Cavs2016", "screen_name": "OvOAT_", "lang": "en", "location": "Colga, Ga", "create_at": date("2011-06-08"), "description": "null", "followers_count": 1910, "friends_count": 787, "statues_count": 127413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814344520896512, "text": "4 https://t.co/oVmfMsurPI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2465061697, "name": "Karly Romley- Cass", "screen_name": "karly_romley", "lang": "en", "location": "null", "create_at": date("2014-04-26"), "description": "null", "followers_count": 316, "friends_count": 400, "statues_count": 1964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814345074540545, "text": "Body by Dreezy Featuring Jeremih is #nowplaying in Tavern on Main, Snellville.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.065723,33.838899"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox", "screen_name": "VirtualJukebox", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1546, "friends_count": 0, "statues_count": 1693514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814345095544832, "text": "Crazy how you don't think about old friends for so long because life happens, but damn these past two nights with them have been bliss.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 292289187, "name": "Alexo", "screen_name": "_AlexanderAaron", "lang": "en", "location": "1/10/12", "create_at": date("2011-05-03"), "description": "The only reason I'm in a place is just to be there. University of Oklahoma.", "followers_count": 716, "friends_count": 796, "statues_count": 11956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814345393299456, "text": "Not a mad bitch, just a bad bitch. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1929686551, "name": "nanz ⚡️", "screen_name": "neafernz", "lang": "en", "location": "Kaneohe, HI", "create_at": date("2013-10-02"), "description": "u make me feel like the baddest bitch ever", "followers_count": 1056, "friends_count": 359, "statues_count": 7669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ahuimanu, HI", "id": "364a9fffa7d084d2", "name": "Ahuimanu", "place_type": "city", "bounding_box": rectangle("-157.862463,21.424797 -157.807145,21.455791") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1500400, "cityName": "Ahuimanu" } }
+{ "create_at": datetime("2016-05-29T00:00:03.000Z"), "id": 736814345397489668, "text": "#cancer so true https://t.co/pmNgVxlrzc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "cancer" }}, "user": { "id": 268591761, "name": "lilbit", "screen_name": "_BigMommaNAE", "lang": "en", "location": "null", "create_at": date("2011-03-18"), "description": "rest mawmaw UL19 .. you can lurk I'm still the shit", "followers_count": 1927, "friends_count": 1184, "statues_count": 61661 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laplace, LA", "id": "005beffd77be6ac9", "name": "Laplace", "place_type": "city", "bounding_box": rectangle("-90.519583,30.031013 -90.435378,30.105989") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2242030, "cityName": "Laplace" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814345816940544, "text": "U for everybody", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 149377959, "name": "b", "screen_name": "BrentLBroussard", "lang": "en", "location": "null", "create_at": date("2010-05-28"), "description": "null", "followers_count": 4422, "friends_count": 277, "statues_count": 66463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carencro, LA", "id": "018e8da25b391f33", "name": "Carencro", "place_type": "city", "bounding_box": rectangle("-92.097811,30.283043 -91.984697,30.381743") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2212665, "cityName": "Carencro" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814345867255809, "text": "@actressanjjanaa, I feel you should have tweeted “[Who's] your most favourite” instead. ‘Whose’ belongs to ‘whom’; ‘who's’ means ‘who is’.", "in_reply_to_status": 736813916982890497, "in_reply_to_user": 133676833, "favorite_count": 0, "coordinate": point("-120.63287865,55.62624538"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 133676833 }}, "user": { "id": 618294231, "name": "Grammar Police", "screen_name": "_grammar_", "lang": "en", "location": "Follow me; see grammar errors!", "create_at": date("2012-06-25"), "description": "@victor_zheng, inspired by @StealthMountain, coded me, which am alerting users in whose statuses I detect improper grammar. To publish solecisms abases oneself!", "followers_count": 29281, "friends_count": 2, "statues_count": 117685 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Peace River D, British Columbia", "id": "377e40b9cbac59a3", "name": "Peace River D", "place_type": "city", "bounding_box": rectangle("-121.255827,53.795071 -120.001313,56.148799") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814346261536769, "text": "\"I don't want to be just a giant purple snake\" - Eric upon playing Slither.io", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 617669949, "name": "Amanda", "screen_name": "chARManderMarsh", "lang": "en", "location": "Boston, MA // San Jose, CA", "create_at": date("2012-06-24"), "description": "// providing you with quality comedic content since never //", "followers_count": 111, "friends_count": 494, "statues_count": 2836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814346395754497, "text": "@_dechellis yeee thanks squad��✊��������", "in_reply_to_status": 736792860666662913, "in_reply_to_user": 601583255, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 601583255 }}, "user": { "id": 1657385245, "name": "wyatt", "screen_name": "weeeeeot", "lang": "en", "location": "Asgard", "create_at": date("2013-08-09"), "description": "yarp", "followers_count": 343, "friends_count": 308, "statues_count": 650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814346458824704, "text": "It's okay I got this ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 57079960, "name": "Kt", "screen_name": "KatieBaby904", "lang": "en", "location": "Jacksonville Beach, FL", "create_at": date("2009-07-15"), "description": "Katie • Jax Bch • Welcome to my mind • Follow the rest of my life IG: lustoverloveee SC:katiebabyyy7", "followers_count": 467, "friends_count": 561, "statues_count": 3257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Neptune Beach, FL", "id": "303f3a49548d2c53", "name": "Neptune Beach", "place_type": "city", "bounding_box": rectangle("-81.438514,30.307106 -81.392075,30.324656") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12031, "countyName": "Duval", "cityID": 1248100, "cityName": "Neptune Beach" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814347368816641, "text": "Wind 0.0 mph ---. Barometer 30.055 in, Steady. Temperature 60.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814347914084352, "text": "@BJ_Whitmer on @ringofhonor https://t.co/6jbrw6x2Vh", "in_reply_to_status": -1, "in_reply_to_user": 617444742, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 617444742, 31187615 }}, "user": { "id": 371627231, "name": "Karlee", "screen_name": "KarleeDaniels", "lang": "en", "location": "Cutten, CA", "create_at": date("2011-09-10"), "description": "❤❤ @zahraschreiber @FearHavok photography and life #NXTFamily IG @karleedaniels", "followers_count": 1731, "friends_count": 2352, "statues_count": 48651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cutten, CA", "id": "5dbf89e92e686841", "name": "Cutten", "place_type": "city", "bounding_box": rectangle("-124.157778,40.753998 -124.129919,40.774957") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6023, "countyName": "Humboldt", "cityID": 617722, "cityName": "Cutten" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814348379774976, "text": "Day 2 of BottleRock #SlomoStyle @ BottleRock Napa https://t.co/JA9shPDp8Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.27771755,38.29776237"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SlomoStyle" }}, "user": { "id": 6455962, "name": "Jason Chinnock", "screen_name": "JChinnock", "lang": "en", "location": "Euless, TX", "create_at": date("2007-05-30"), "description": "I work in Live Sports Broadcasting in DFW and I enjoy board games!", "followers_count": 304, "friends_count": 563, "statues_count": 9357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Napa, CA", "id": "840f120ae56c1da7", "name": "Napa", "place_type": "city", "bounding_box": rectangle("-122.358011,38.243267 -122.254517,38.350876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6055, "countyName": "Napa", "cityID": 650258, "cityName": "Napa" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814348673396736, "text": "Wisdom teeth fucking hurt dawg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1373364084, "name": "Damon T. Green", "screen_name": "_DGreen5_", "lang": "en", "location": "null", "create_at": date("2013-04-22"), "description": "i pled the 5th #330", "followers_count": 1581, "friends_count": 1055, "statues_count": 44224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Youngstown, OH", "id": "6e276f099bcab5b5", "name": "Youngstown", "place_type": "city", "bounding_box": rectangle("-80.711161,41.049898 -80.56792,41.160644") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39099, "countyName": "Mahoning", "cityID": 3988000, "cityName": "Youngstown" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814348719427584, "text": "@EB_34L act like you didn't fall over the back of my chair ��", "in_reply_to_status": 736814253697425408, "in_reply_to_user": 482436521, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 482436521 }}, "user": { "id": 2919159638, "name": "fin ✨", "screen_name": "riplatray", "lang": "en", "location": "null", "create_at": date("2014-12-04"), "description": "null", "followers_count": 1514, "friends_count": 1629, "statues_count": 31584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joliet, IL", "id": "741e21eeea82f00a", "name": "Joliet", "place_type": "city", "bounding_box": rectangle("-88.294633,41.444387 -87.99039,41.594725") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1738570, "cityName": "Joliet" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814348862029825, "text": "@alyybongo but a beanie is also those little colorful hats with propellers on top", "in_reply_to_status": 736791823377563648, "in_reply_to_user": 123740283, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 123740283 }}, "user": { "id": 40328833, "name": "Joeduck", "screen_name": "g3nuinejoe", "lang": "en", "location": "Santa Monica, CA", "create_at": date("2009-05-15"), "description": "Will cook for cuddles.\n\n***WARNING: Sometimes I tweet pictures of food.***", "followers_count": 7588, "friends_count": 644, "statues_count": 27747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814349092835328, "text": "I know he salty af lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 489089196, "name": "Jèrèmÿ Jàmìłł⚠", "screen_name": "jnewby15", "lang": "en", "location": "Lima,Ohio", "create_at": date("2012-02-10"), "description": "Everything I'm not, made me everything I am✊ #TeamKobe #Blessed #IFollowBack #IPhoneNation #Snapchat-jnewby15 Kendra Marie Newby 9-17-15", "followers_count": 1478, "friends_count": 1088, "statues_count": 56980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lima, OH", "id": "6b1aa33507f2e472", "name": "Lima", "place_type": "city", "bounding_box": rectangle("-84.215102,40.687562 -84.012941,40.817349") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39003, "countyName": "Allen", "cityID": 3943554, "cityName": "Lima" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814349130424320, "text": "I forgot how many times I've been rejected this month ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3699846732, "name": "hooooeeel_daaabs.710", "screen_name": "JoelAco60348028", "lang": "en", "location": "null", "create_at": date("2015-09-26"), "description": "19. Employed. \nI'm just trying to live the best i can. \nStay Postive,\nIgnore the Negativity", "followers_count": 70, "friends_count": 300, "statues_count": 2837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814349314969600, "text": "I hate being in this type of mood ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 880656494, "name": "Sam♈", "screen_name": "_sammbamm12", "lang": "en", "location": "stuck in a book ", "create_at": date("2012-10-14"), "description": "*black heart emoji*", "followers_count": 392, "friends_count": 576, "statues_count": 6527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-05-29T00:00:04.000Z"), "id": 736814349789057024, "text": "You've Got A Friend����������������������������Loryane SINGER https://t.co/QXTAajsHJr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.9015,28.0065"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 838883766, "name": "Living Dead Girl", "screen_name": "AprilLeeOsborne", "lang": "en", "location": "Music Promoter ", "create_at": date("2012-09-21"), "description": "https://myspace.com/april.ozzy /Music Promoter", "followers_count": 574, "friends_count": 1592, "statues_count": 18105 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12105, "countyName": "Polk", "cityID": 1238250, "cityName": "Lakeland" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814349948444672, "text": "Starting a new life full new name? Would you do it ? Leave everything ?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 335532225, "name": "Noel Velez", "screen_name": "INI_Titanz", "lang": "en", "location": "null", "create_at": date("2011-07-14"), "description": "6'6 on Twitter, circle kept small since the 3rd duo- @forgivennj", "followers_count": 607, "friends_count": 447, "statues_count": 1749 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Highland, IN", "id": "e2c96cf8c0a43c1d", "name": "Highland", "place_type": "city", "bounding_box": rectangle("-87.481092,41.522911 -87.432467,41.57324") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1833484, "cityName": "Highland" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814349960892418, "text": "We mature with the damage, not with the years.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3007193268, "name": "{Megan♡Rene}", "screen_name": "mrnorwood96", "lang": "en", "location": "Louisiana, USA", "create_at": date("2015-02-01"), "description": "God is within her, she will never fail~ Psalms 46:5❤️", "followers_count": 297, "friends_count": 404, "statues_count": 1769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Charles, LA", "id": "21b6f72f9fdaeb06", "name": "Lake Charles", "place_type": "city", "bounding_box": rectangle("-93.314852,30.124859 -93.080595,30.280683") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2241155, "cityName": "Lake Charles" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350019624960, "text": "How many languages do I know?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3010581946, "name": "Jun", "screen_name": "SmfhJun", "lang": "en", "location": "Braiden", "create_at": date("2015-02-01"), "description": "Ashlyn ~ Leena ~ Rayna ~ Nick ~ Angie", "followers_count": 728, "friends_count": 119, "statues_count": 10952 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plano, TX", "id": "488da0de4c92ac8e", "name": "Plano", "place_type": "city", "bounding_box": rectangle("-96.85743,32.997151 -96.612371,33.125334") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4858016, "cityName": "Plano" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350090964992, "text": "@SamhainNight CURSE YOU VACATIONEEEERS!", "in_reply_to_status": 736813157566418944, "in_reply_to_user": 148237540, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 148237540 }}, "user": { "id": 149339343, "name": "Cecilia", "screen_name": "ChaoticRambler", "lang": "en", "location": "My room, duh. ", "create_at": date("2010-05-28"), "description": "Demon Goddess of Gender, Sorceress Supreme, Autistic Avenger, Trans Titan", "followers_count": 361, "friends_count": 205, "statues_count": 125005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastmont, WA", "id": "002d396055a6adf8", "name": "Eastmont", "place_type": "city", "bounding_box": rectangle("-122.234054,47.860402 -122.14427,47.921824") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5319630, "cityName": "Eastmont" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350120321024, "text": "When your power is out ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2241000066, "name": "Brooke Ann", "screen_name": "thatwhitegirl47", "lang": "en", "location": "Sacramento, CA", "create_at": date("2013-12-11"), "description": "null", "followers_count": 521, "friends_count": 592, "statues_count": 2136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350254493697, "text": "Reason Number 1 why your girl like tall niggas �� https://t.co/XNbZYttDR3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2824651530, "name": "++", "screen_name": "poppasoslutty", "lang": "en", "location": "Houston, TX", "create_at": date("2014-09-21"), "description": "IG . @omgpoppa_ |", "followers_count": 792, "friends_count": 853, "statues_count": 2928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350632165376, "text": "If I was to buy my lady some makeup what would y'all recommend", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120680949, "name": "Simple Jack", "screen_name": "_KevinFederline", "lang": "en", "location": "At the Ritz Carlton", "create_at": date("2010-03-06"), "description": "Father | Husband | Sigma | Alum | Former Alderman for the City of Harvey | RIP Jackie", "followers_count": 2425, "friends_count": 1421, "statues_count": 181255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crete, IL", "id": "002915850a84facf", "name": "Crete", "place_type": "city", "bounding_box": rectangle("-87.671067,41.384066 -87.574744,41.470027") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1717523, "cityName": "Crete" } }
+{ "create_at": datetime("2016-05-29T00:00:05.000Z"), "id": 736814350762147840, "text": "дятел https://t.co/tcA5bMKsGq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-123.01225255,37.69933327"), "retweet_count": 0, "lang": "ru", "is_retweet": false, "user": { "id": 3829425134, "name": "Allen Collier", "screen_name": "colaroler", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "null", "followers_count": 6, "friends_count": 0, "statues_count": 26937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176716871761920, "text": "@PapaXBear35 wait for kds face�� https://t.co/nm396ZlBnx", "in_reply_to_status": -1, "in_reply_to_user": 3178512816, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3178512816 }}, "user": { "id": 2916320641, "name": "king of kilping", "screen_name": "BrettGraham17", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "11/20/94, drumming, guitar, video games, KYLEE RENAE MCCARTER❤️", "followers_count": 188, "friends_count": 187, "statues_count": 5099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tooele, UT", "id": "5f72d488385b8ced", "name": "Tooele", "place_type": "city", "bounding_box": rectangle("-112.331612,40.507327 -112.266954,40.569357") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49045, "countyName": "Tooele", "cityID": 4976680, "cityName": "Tooele" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717106810880, "text": "No. https://t.co/s78EaQxoLC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 387247879, "name": "Jessica Kron", "screen_name": "JessicaKron09", "lang": "en", "location": "Winneconne, WI", "create_at": date("2011-10-08"), "description": "null", "followers_count": 404, "friends_count": 292, "statues_count": 825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Butte des Morts, WI", "id": "004a05da1b81e1c5", "name": "Butte des Morts", "place_type": "city", "bounding_box": rectangle("-88.683702,44.093111 -88.623345,44.112322") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55139, "countyName": "Winnebago", "cityID": 5511500, "cityName": "Butte des Morts" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717177946114, "text": "nvr switch when ya mans low.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1184712799, "name": "bossy", "screen_name": "KierraChanelle_", "lang": "en", "location": "new orleans, LA. ", "create_at": date("2013-02-15"), "description": "doing well dawg;) luvyouadriyan&shawnai!", "followers_count": 961, "friends_count": 458, "statues_count": 76387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717450575872, "text": "tb song", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2695609872, "name": "Gigi", "screen_name": "__gigiii", "lang": "en", "location": "Houston, TX", "create_at": date("2014-07-31"), "description": "rhs✨", "followers_count": 318, "friends_count": 691, "statues_count": 9501 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717668671493, "text": "People are as broken as they are beautiful.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 736446758440439808, "name": "Daniel Vasquez", "screen_name": "ThatKidDaniel19", "lang": "en", "location": "null", "create_at": date("2016-05-27"), "description": "You cats stay outta trouble, alright?", "followers_count": 32, "friends_count": 76, "statues_count": 8 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717748375552, "text": "@CeliaCeee Ohhhhhh yeahhhhh huh ����", "in_reply_to_status": 737176401577545728, "in_reply_to_user": 1120918250, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1120918250 }}, "user": { "id": 1046640739, "name": "Nobody!", "screen_name": "najera75", "lang": "en", "location": "null", "create_at": date("2012-12-29"), "description": "⚽❤ Sc: josee_75", "followers_count": 694, "friends_count": 612, "statues_count": 20638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176717782028288, "text": "What respect I had for you is now gone ✌��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 481489704, "name": "Kenzie", "screen_name": "FowlerKenzie09", "lang": "en", "location": "Cookeville, TN", "create_at": date("2012-02-02"), "description": "null", "followers_count": 412, "friends_count": 184, "statues_count": 2588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cookeville, TN", "id": "3448f4f82d2187bc", "name": "Cookeville", "place_type": "city", "bounding_box": rectangle("-85.601741,36.046986 -85.418673,36.232069") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47141, "countyName": "Putnam", "cityID": 4716920, "cityName": "Cookeville" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176718734065664, "text": "@osnapitzvk awww unfortunately di ko na vid huhuhuhu di pala naka bukas yung record", "in_reply_to_status": 737176446855061504, "in_reply_to_user": 765917083, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 765917083 }}, "user": { "id": 170346564, "name": "IVORY", "screen_name": "ivorylleviceral", "lang": "en", "location": "Las Vegas, NV ⭐ Philippines", "create_at": date("2010-07-24"), "description": "I Love Vice Ganda and Karylle ⭐⭐ Notice by @vicegandako and @anakarylle", "followers_count": 4606, "friends_count": 789, "statues_count": 69087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719443021824, "text": "@InasX dm me ur email��������", "in_reply_to_status": -1, "in_reply_to_user": 1959583898, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1959583898 }}, "user": { "id": 2242751291, "name": "Aceswagbeatz", "screen_name": "aceswagbeatz", "lang": "en", "location": "charleston,ms", "create_at": date("2013-12-24"), "description": "RGF Producer/Artist|AlignTalentAgency@gmail.com for booking| 4 beatz aceswagbeatz@gmail.com prod credits|fetty wap|french Montana|Monty| Etc.. #RGFIsland #Amg", "followers_count": 993, "friends_count": 2125, "statues_count": 3913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charleston, MS", "id": "7ea1ffdd2f1e9b1d", "name": "Charleston", "place_type": "city", "bounding_box": rectangle("-90.06947,33.999695 -90.04012,34.014408") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28135, "countyName": "Tallahatchie", "cityID": 2812900, "cityName": "Charleston" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719510016004, "text": "@ebbtideapp Tide in Fort Point, Maine 05/30/2016\nHigh 5:51am 9.1\n Low 12:24pm 0.1\nHigh 6:32pm 9.2\n Low 12:57am 0.5\nHigh 6:54am 9.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-70.6383,43.13"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 96, "friends_count": 1, "statues_count": 37069 }, "place": { "country": "United States", "country_code": "United States", "full_name": "York Harbor, ME", "id": "757741d977127623", "name": "York Harbor", "place_type": "city", "bounding_box": rectangle("-70.691702,43.115197 -70.622272,43.162016") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23031, "countyName": "York", "cityID": 2388160, "cityName": "York Harbor" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719614873600, "text": "Wind 0.0 mph ---. Barometer 1019.44 mb, Falling. Temperature 64.1 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 32, "friends_count": 128, "statues_count": 14748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719669354496, "text": "\"You're my flower, you're my power\" ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2355209702, "name": "Trevor Jones", "screen_name": "jonestrevor199", "lang": "en", "location": "Cypress, CA", "create_at": date("2014-02-21"), "description": "Moto | JFK | RIP Trev 4/18/13 Kay 10/11/15", "followers_count": 136, "friends_count": 249, "statues_count": 622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cypress, CA", "id": "a077793faceeda6f", "name": "Cypress", "place_type": "city", "bounding_box": rectangle("-118.063298,33.791963 -118.010668,33.845856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 617750, "cityName": "Cypress" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719686127616, "text": "@OliverWolfson whos up", "in_reply_to_status": 737169053551235072, "in_reply_to_user": 3321943518, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3321943518 }}, "user": { "id": 3321943518, "name": "Oliver Wolfson", "screen_name": "OliverWolfson", "lang": "en", "location": "Long Beach, CA, USA, Earth", "create_at": date("2015-08-20"), "description": "null", "followers_count": 391, "friends_count": 330, "statues_count": 2443 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719728201729, "text": "Just posted a photo @ Planet Fitness - Manhattan, Union Square, NY https://t.co/dlVD28nzi2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.992709,40.735464"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323032990, "name": "GemInAnEye", "screen_name": "Tommy_Nakos", "lang": "en", "location": "null", "create_at": date("2011-06-23"), "description": "null", "followers_count": 44, "friends_count": 155, "statues_count": 251 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176719900049408, "text": "f4054682128efd1d34b6dc203fa90a179281e04a52eae1af04db067c9b9eff9e5b694a70cc6b9debe10609cfbad7c5e22b7f886c0a428425b73846eaf2140149611048000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-158.18240858,68.50194801"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 165, "friends_count": 1, "statues_count": 999266 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2290, "countyName": "Yukon-Koyukuk" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176720067829760, "text": "X men was fucken DOPE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3163000770, "name": "Astrid", "screen_name": "astrid32_astrid", "lang": "en", "location": "null", "create_at": date("2015-04-18"), "description": "19/6119❤️ sc:perezastrid32", "followers_count": 694, "friends_count": 424, "statues_count": 12066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176720541765632, "text": "Edmonton Rooms for rent on RentBoard.ca https://t.co/0jFGUExLe8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-113.4909267,53.544389"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 31169027, "name": "Rent Board of Canada", "screen_name": "rentboard", "lang": "en", "location": "Canada", "create_at": date("2009-04-14"), "description": "Apartments and Houses for Rent in Canada", "followers_count": 321, "friends_count": 0, "statues_count": 123772 }, "place": { "country": "Canada", "country_code": "Canada", "full_name": "Edmonton, Alberta", "id": "6a6d896ba1cb5dc4", "name": "Edmonton", "place_type": "city", "bounding_box": rectangle("-113.71368,53.395531 -113.271555,53.716108") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176720726360064, "text": "Holy shit, so.. found myself sexually attracted to a sculpture today--that's a new one.. but man, he's so fuckin' beautiful!~", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174161358, "name": "Danni Zamudio", "screen_name": "Danni_Zamudio", "lang": "en", "location": "Portlandia, OR", "create_at": date("2010-08-03"), "description": "Mercenary Illustrator · Dirge Dancer · Art Muse · Cartoonist · Sweettoof · A harrowing, savage, and audacious exploration of a creative mind.", "followers_count": 1731, "friends_count": 420, "statues_count": 5319 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-05-30T00:00:00.000Z"), "id": 737176720797667332, "text": "���� https://t.co/e1OQIiMwUO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 323426826, "name": "Sinuhe", "screen_name": "seaweed4242", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "#TrackNation \n#GSU19 #CUEUPU \n2015 CX SWAC CHAMPS", "followers_count": 208, "friends_count": 156, "statues_count": 5170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176720936034304, "text": "������ https://t.co/dfkEyTkQQX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2303632909, "name": "owen", "screen_name": "soujaowen", "lang": "en", "location": "San Jose, CA", "create_at": date("2014-01-21"), "description": "beatplugg® - contactsurp@gmail.com", "followers_count": 697, "friends_count": 404, "statues_count": 20134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alum Rock, CA", "id": "277b4360183b5d75", "name": "Alum Rock", "place_type": "city", "bounding_box": rectangle("-121.842175,37.35398 -121.812158,37.38102") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 601458, "cityName": "Alum Rock" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176720990687232, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":66.4°F Wind:0.0mph Pressure: 30.03hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 18, "statues_count": 320698 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176721162698752, "text": "@EAnnPenaranda we gotta have more adventures����", "in_reply_to_status": 737176614748852224, "in_reply_to_user": 2152161624, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2152161624 }}, "user": { "id": 2263546088, "name": "brittany☻", "screen_name": "brittanyxanne_", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-12-26"), "description": "cameron❤︎", "followers_count": 197, "friends_count": 193, "statues_count": 2364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176721300938753, "text": "@MuradyanHarout I'll send you my address rn", "in_reply_to_status": 737176539389779968, "in_reply_to_user": 2211320792, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2211320792 }}, "user": { "id": 1052638764, "name": "Shelbs", "screen_name": "ShelbyRenee098", "lang": "en", "location": "Fresno, CA", "create_at": date("2013-01-01"), "description": "Appreciate your blessings. Fresno State. | sc: shelbss098", "followers_count": 1893, "friends_count": 971, "statues_count": 33320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176721712025601, "text": "Omg nvm Hulu is missing hella seasons ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 772568384, "name": "desiree", "screen_name": "dezthoo", "lang": "en", "location": "null", "create_at": date("2012-08-21"), "description": "bonjour, petit wayne", "followers_count": 197, "friends_count": 376, "statues_count": 5408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino, CA", "id": "cf2d9d01f16e9bd4", "name": "Chino", "place_type": "city", "bounding_box": rectangle("-117.735657,33.926295 -117.610852,34.04837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613210, "cityName": "Chino" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176721892466689, "text": "05/30@03:00 - Temp 70.1F, WC 70.1F. Wind 0.0mph NNE, Gust 1.0mph. Bar 30.010in, Falling. Rain 0.00in. Hum 99%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722022490112, "text": "Imma name my dog rango", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2483946115, "name": "Givvo101", "screen_name": "tamarzian", "lang": "en", "location": "null", "create_at": date("2014-05-08"), "description": "That Track Life, That Pianist Life, ANIME IS AWESOME SHUT UP!!!", "followers_count": 272, "friends_count": 372, "statues_count": 9811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Turlock, CA", "id": "ee2cfc9feb061a08", "name": "Turlock", "place_type": "city", "bounding_box": rectangle("-120.912851,37.459822 -120.812106,37.53937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 680812, "cityName": "Turlock" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722068512768, "text": "Let her know I'm Trynna support her, and help her reach her goal and take care of her. That's on everything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3433743192, "name": "Louis", "screen_name": "TwzLoui5", "lang": "en", "location": "Sylmar, California", "create_at": date("2015-09-02"), "description": "18 and humble |Cal St Poly Pomona Bronco|", "followers_count": 405, "friends_count": 470, "statues_count": 9777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722265673729, "text": "Getting ready to start dad's biography. I can only write it form my perspective. I don' have time to talk to anybody except mom. #JeffMiller", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JeffMiller" }}, "user": { "id": 56920668, "name": "Joshua Miller", "screen_name": "enocharden62", "lang": "en", "location": "Lakewood, CO", "create_at": date("2009-07-14"), "description": "Joshua Miller", "followers_count": 30, "friends_count": 101, "statues_count": 428 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CO", "id": "f7eb2fa2fea288b1", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-105.193475,39.60973 -105.053164,39.761974") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 843000, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722286727169, "text": "I've been seeing everything in life completely different lately and I'm lovin it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3169788459, "name": "aleesia mia", "screen_name": "Aleesia14", "lang": "en", "location": "Neenah, WI", "create_at": date("2015-04-15"), "description": "http://vsco.co/aleesiag", "followers_count": 422, "friends_count": 648, "statues_count": 1037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Neenah, WI", "id": "37d64ec7121fc808", "name": "Neenah", "place_type": "city", "bounding_box": rectangle("-88.531088,44.119938 -88.433837,44.194617") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55139, "countyName": "Winnebago", "cityID": 5555750, "cityName": "Neenah" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722672652288, "text": "Ripley SW Limestone Co. Temp: 69.1°F Wind:0.0mph Pressure: 996.8mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 53906 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176722756362240, "text": "trying to do better than good enough", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483435087, "name": "curston", "screen_name": "curscurs", "lang": "en", "location": "Houston, TX", "create_at": date("2012-02-04"), "description": "a woman that doesn't need much but appreciates a whole lot", "followers_count": 1191, "friends_count": 686, "statues_count": 66331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176723335303169, "text": "1st places baby ���������� #npc #kucloclassic2016 #bodybuilding #kucloclassic #npcbodybuilding… https://t.co/oWe9uqN5uV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.8,32.7833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "npc", "kucloclassic2016", "bodybuilding", "kucloclassic", "npcbodybuilding" }}, "user": { "id": 867065154, "name": "Nikita Pokryshkin", "screen_name": "Nikita_IFBB", "lang": "ru", "location": "Dallas, TX", "create_at": date("2012-10-07"), "description": "2016 NPC Kuclo Classic Champion in Open Bodybuilding Middleweight & Novice BB / Champion of Russia in junior BB / Lithuanian Cup Overall Winner in junior BB", "followers_count": 75, "friends_count": 67, "statues_count": 494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176724014698500, "text": "@logarciaaa that nigga heartbroken", "in_reply_to_status": 737176308396888064, "in_reply_to_user": 4052534294, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4052534294 }}, "user": { "id": 2321127168, "name": "scammin cyn", "screen_name": "cynnnnnx", "lang": "en", "location": "null", "create_at": date("2014-01-31"), "description": "a$ap", "followers_count": 555, "friends_count": 181, "statues_count": 53803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ladera Heights, CA", "id": "714789cf3b7a50d0", "name": "Ladera Heights", "place_type": "city", "bounding_box": rectangle("-118.391088,33.97632 -118.357614,34.014937") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639108, "cityName": "Ladera Heights" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176724291489793, "text": "just bc its natural doesn't mean it's ok https://t.co/sT2bA7ejYx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102573385, "name": "James", "screen_name": "jttoal_", "lang": "en", "location": "Conway, AR → Fayetteville, AR", "create_at": date("2010-01-06"), "description": "#UARK20 • probably hanging out with your girlfriend", "followers_count": 1048, "friends_count": 273, "statues_count": 74126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conway, AR", "id": "90407ba247350183", "name": "Conway", "place_type": "city", "bounding_box": rectangle("-92.537713,35.018058 -92.323965,35.128165") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5045, "countyName": "Faulkner", "cityID": 515190, "cityName": "Conway" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176724782362624, "text": "Wind 0.0 mph ---. Barometer 30.020 in, Steady. Temperature 66.3 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-30T00:00:01.000Z"), "id": 737176724811571201, "text": "@restlessruiz91 studs ������ https://t.co/yUZyqLSxeJ", "in_reply_to_status": 737168769869484032, "in_reply_to_user": 1682109044, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1682109044 }}, "user": { "id": 52305185, "name": "Ivan Soria", "screen_name": "ivanso7", "lang": "en", "location": "Clovis, Fresno, CA", "create_at": date("2009-06-29"), "description": "Fresno State - Health Administration. Comcast. Snapchat: ivanso", "followers_count": 943, "friends_count": 841, "statues_count": 43264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725130338304, "text": "I woke up just to take a shower lol. I can't sleep the whole night if I don't take a shower before bed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 535760158, "name": "$haaaaawty.", "screen_name": "trill__baaby", "lang": "en", "location": "Vallejo, CA", "create_at": date("2012-03-24"), "description": "minding my business ..", "followers_count": 928, "friends_count": 758, "statues_count": 39527 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725256327168, "text": "Temp 54.4° Hi/Lo 54.6/54.1 Rng 0.5° WC 54.4° Hmd 96% Rain 0.01\" Storm 0.00\" BAR 30.101 Falling DP 53.3° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 72, "friends_count": 123, "statues_count": 19098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725377941504, "text": "Wind 0.0 mph ---. Barometer 29.999 in, Steady. Temperature 65.5 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725587529728, "text": "Girl like me* https://t.co/g9B53wCNFw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3315300708, "name": "Low Life", "screen_name": "bailie_cole23", "lang": "en", "location": "chillin, TX", "create_at": date("2015-08-14"), "description": "♉️ | WR/Corner for the #B16DAWGS | Heartless | Motocross | Ride For Gaby | #YEEYEE | #TAMUK20", "followers_count": 428, "friends_count": 479, "statues_count": 9500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kyle, TX", "id": "94ef42cc204d5195", "name": "Kyle", "place_type": "city", "bounding_box": rectangle("-97.896063,29.942852 -97.830853,30.035633") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4839952, "cityName": "Kyle" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725868576768, "text": "Illusions are cool.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 39598535, "name": "Lea ⚜", "screen_name": "chreyonce", "lang": "en", "location": "Louisiana", "create_at": date("2009-05-12"), "description": "20. Business major.", "followers_count": 3389, "friends_count": 989, "statues_count": 189847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pineville, LA", "id": "a22746c88990f7a6", "name": "Pineville", "place_type": "city", "bounding_box": rectangle("-92.452373,31.303096 -92.34801,31.384652") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2260530, "cityName": "Pineville" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725876924416, "text": "baby d'angelo ):", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2365006015, "name": "fre$a", "screen_name": "h8zel", "lang": "en", "location": "null", "create_at": date("2014-02-27"), "description": "love sosa // darnell boat gives me orgasms", "followers_count": 326, "friends_count": 65, "statues_count": 13841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176725956657152, "text": "✌��️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 723357368, "name": "Hern.", "screen_name": "_hern1996", "lang": "en", "location": "Mexicali,Baja California", "create_at": date("2012-07-28"), "description": "Snapchat: hern1996 /\nInstagram: _hern", "followers_count": 675, "friends_count": 316, "statues_count": 9245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brawley, CA", "id": "8da9aff1e34d7b52", "name": "Brawley", "place_type": "city", "bounding_box": rectangle("-115.568757,32.953097 -115.506903,33.004816") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 608058, "cityName": "Brawley" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176726078382080, "text": "@FundFire @alaubsch @finalternatives @altassetco @BalterLiquidAlt@wsj @hedgefundreport \"Life as In Business is all about Contacts\".", "in_reply_to_status": -1, "in_reply_to_user": 44179062, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44179062, 22754299, 18482553, 3387039599, 57072528 }}, "user": { "id": 715616302182875136, "name": "Joseph Pep Nodarse", "screen_name": "joerileyhudson", "lang": "en", "location": "New York, SOBE, London, Cayman", "create_at": date("2016-03-31"), "description": "25 Years of institutional ALTS experience. CEO of Hudson, Riley Asset Counsel, Inc. Shell Oil Pension Fund.\nHarvard Business School\nLondon School of Economics", "followers_count": 137, "friends_count": 396, "statues_count": 945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176726136987649, "text": "She lowkey could outsing Victoria Justice.... https://t.co/CVwIcky8OG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1081765278, "name": "lil blacksbi vert", "screen_name": "MattWakhu", "lang": "en", "location": "Denton, TX", "create_at": date("2013-01-11"), "description": "The Great Blacksby | Viva sua paixão | GentlΣmaΝ | Kenyan | Follow my music page @MattWakhuMusic!", "followers_count": 3947, "friends_count": 2856, "statues_count": 133452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176726267006976, "text": "simpin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2583169742, "name": "Kerv", "screen_name": "mindofkervinn", "lang": "en", "location": "null", "create_at": date("2014-06-22"), "description": "summer 16", "followers_count": 333, "friends_count": 280, "statues_count": 5185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176726724345856, "text": "Temp: 71.5°F Wind:0.0mph Pressure: 30.028hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176726766116864, "text": "happy bday to me!!!!!!!! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1070216874, "name": "αbbs but no αbs", "screen_name": "AbbyJonesss_", "lang": "en", "location": "null", "create_at": date("2013-01-07"), "description": "be ambitious", "followers_count": 955, "friends_count": 349, "statues_count": 11487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176727126953984, "text": "I might just tell this hoe my secrets", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 484476330, "name": "Cam", "screen_name": "TrillNigggaCam", "lang": "en", "location": "Statesville, NC", "create_at": date("2012-02-05"), "description": "C/o 2014. Love the life you live fam", "followers_count": 612, "friends_count": 519, "statues_count": 26216 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176727722569728, "text": "Eyes are probably the most attractive thing on a guy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 854094396, "name": "Isabella Savino", "screen_name": "savino_isabella", "lang": "en", "location": "rodnaes house", "create_at": date("2012-09-29"), "description": "you dont pour my cereal", "followers_count": 529, "friends_count": 524, "statues_count": 7367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Matawan, NJ", "id": "a7cdc3f10586dc66", "name": "Matawan", "place_type": "city", "bounding_box": rectangle("-74.261967,40.390452 -74.218098,40.444534") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3444520, "cityName": "Matawan" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176728221683712, "text": "Temp 58.9°F Wind Chill 58.9°F RH 94% Wind 0.0 --- Gust 0.0 --- SLP 30.028 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176728372531200, "text": "When you're fucked up and you text one of the most important people in your life and ruin their night. I am sooo sorry, I'm such n asshole.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2904822806, "name": "ⓑⓡⓐⓓⓛⓔⓢ", "screen_name": "bradles_16", "lang": "en", "location": "null", "create_at": date("2014-11-19"), "description": "I'd like to Thank my Mom. . .", "followers_count": 115, "friends_count": 150, "statues_count": 917 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fruita, CO", "id": "96cd3e111e35bc6e", "name": "Fruita", "place_type": "city", "bounding_box": rectangle("-108.753194,39.123786 -108.682236,39.178806") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8077, "countyName": "Mesa", "cityID": 828745, "cityName": "Fruita" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176728376770562, "text": "Here is a little hint....... I don't fucking care!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3266226584, "name": "Daniel Ray", "screen_name": "babyboyfail", "lang": "en", "location": "Denver, CO. Championship City!", "create_at": date("2015-07-02"), "description": "I live in Denver, Co. Home of SB50 Winners: Denver Broncos, Love everything about Pop Culture, Fashion, and anything that makes YOU lol\nSnapchat: Denverdude37", "followers_count": 37, "friends_count": 62, "statues_count": 1206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greenwood Village, CO", "id": "a4d5a00cbd823818", "name": "Greenwood Village", "place_type": "city", "bounding_box": rectangle("-104.981057,39.587505 -104.866586,39.653059") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8005, "countyName": "Arapahoe", "cityID": 833035, "cityName": "Greenwood Village" } }
+{ "create_at": datetime("2016-05-30T00:00:02.000Z"), "id": 737176728452272128, "text": "It doesn't evening feel like 12", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2255818908, "name": "Kalani Kukahiko", "screen_name": "akalanik", "lang": "en", "location": "Long Beach, CA", "create_at": date("2013-12-20"), "description": "Bosco class of 2017 soccer•football", "followers_count": 451, "friends_count": 278, "statues_count": 5840 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176729328848896, "text": "@8_manuuu lmaoooo \"aye what game is that bro\" \"how u play it?\" ����", "in_reply_to_status": 737176577071403008, "in_reply_to_user": 2492417101, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2492417101 }}, "user": { "id": 2303910674, "name": "Sweet Pea♐️", "screen_name": "siotogia32", "lang": "en", "location": "Carson, CA || WŠGT$ ", "create_at": date("2014-01-21"), "description": "18 || 6'3 || Samoan & Tokelaun || UCHIES❗️", "followers_count": 1148, "friends_count": 822, "statues_count": 35437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176729635037184, "text": "@Mar_e_o17 spark that shit cuh", "in_reply_to_status": 737176677608890368, "in_reply_to_user": 34560639, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34560639 }}, "user": { "id": 1350382801, "name": "DontDrakeAndDrive", "screen_name": "uv_rays17", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-04-13"), "description": "Bicken Back Being Bool.", "followers_count": 422, "friends_count": 486, "statues_count": 54842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176729769365504, "text": "Smash �� https://t.co/Z9GMpoISWQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 305912221, "name": "not your bitch☺️", "screen_name": "xcallmepoison", "lang": "en", "location": "Spelman College ✨", "create_at": date("2011-05-26"), "description": "I like my berries black. decatur, al", "followers_count": 2751, "friends_count": 2667, "statues_count": 20457 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Decatur, AL", "id": "246fb652d518385d", "name": "Decatur", "place_type": "city", "bounding_box": rectangle("-87.095676,34.507116 -86.925426,34.654734") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1103, "countyName": "Morgan", "cityID": 120104, "cityName": "Decatur" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176729840582656, "text": "Wind 0.0 mph W. Barometer 30.00 in, Falling slowly. Temperature 68.4 °F. Rain today 0.01 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 123, "statues_count": 160161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176729987469312, "text": "@11shotz goodnight love you other half ❤️����", "in_reply_to_status": -1, "in_reply_to_user": 275257162, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 275257162 }}, "user": { "id": 258467197, "name": ". . ❤️", "screen_name": "dej13__", "lang": "en", "location": "SoftballDiamond⚾️", "create_at": date("2011-02-27"), "description": "NSU19 ✊", "followers_count": 5384, "friends_count": 5167, "statues_count": 144189 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portsmouth, VA", "id": "60edfde178b362ff", "name": "Portsmouth", "place_type": "city", "bounding_box": rectangle("-76.420402,36.786146 -76.291434,36.897382") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51740, "countyName": "Portsmouth", "cityID": 5164000, "cityName": "Portsmouth" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730067042305, "text": "Me this whole weekend https://t.co/41JQO1yvyT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3507272533, "name": "Mayra", "screen_name": "vivalamayra", "lang": "en", "location": "Bay to LA", "create_at": date("2015-09-09"), "description": "it's me Mario", "followers_count": 145, "friends_count": 174, "statues_count": 2344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730130075648, "text": "@AngiPinks fuck outta here w that", "in_reply_to_status": 737173867899293696, "in_reply_to_user": 1115356386, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1115356386 }}, "user": { "id": 3108785391, "name": "Kelsey Reese", "screen_name": "KelseyyReesse", "lang": "en", "location": "Fayetteville, Arkansas", "create_at": date("2015-03-23"), "description": "this island's full of noises.", "followers_count": 323, "friends_count": 294, "statues_count": 525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wolfdale, PA", "id": "01db229e66fb6080", "name": "Wolfdale", "place_type": "city", "bounding_box": rectangle("-80.335379,40.187737 -80.264229,40.217416") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42125, "countyName": "Washington", "cityID": 4286000, "cityName": "Wolfdale" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730306289664, "text": "@rifpeepants use the @", "in_reply_to_status": 737175674570592257, "in_reply_to_user": 390148703, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 390148703 }}, "user": { "id": 596608946, "name": "Jacob Flores-Arias", "screen_name": "NoXCForMe", "lang": "en", "location": "null", "create_at": date("2012-06-01"), "description": "God gave me talent. Coach gave me training. I gave you a show. #Madrista", "followers_count": 334, "friends_count": 262, "statues_count": 3403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Posen, IL", "id": "109342af5712414e", "name": "Posen", "place_type": "city", "bounding_box": rectangle("-87.698452,41.619134 -87.674613,41.640875") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1761314, "cityName": "Posen" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730834599936, "text": "@EatsGlitter Thank you �� For Lifting My spirit up", "in_reply_to_status": 737176063101407234, "in_reply_to_user": 94917330, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 94917330 }}, "user": { "id": 2905850422, "name": "DeLeon", "screen_name": "b_maritn", "lang": "en", "location": "San Bernardino, CA", "create_at": date("2014-12-04"), "description": "Just...A Single Man...Living Life..", "followers_count": 444, "friends_count": 529, "statues_count": 4673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730893307905, "text": "You know how sometimes you're REALLY tired and then you lay in your bed & can't sleep? Yeah. Maybe I need some sheep to count or something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.59996859,41.60324079"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22451014, "name": "Kristen Kranz", "screen_name": "kranzie85", "lang": "en", "location": "North Carolina, USA", "create_at": date("2009-03-01"), "description": "News writer for http://Hypable.com. Sorkin admirer, coffee hater, & procrastination sufferer. Addicted to peanut butter and Diet Coke.", "followers_count": 341, "friends_count": 739, "statues_count": 5128 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176730893357058, "text": "https://t.co/vEI8kDZcAn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 310995379, "name": "Tara Fraser", "screen_name": "Simmy3Tara", "lang": "en", "location": "Natrona Heights, Pa", "create_at": date("2011-06-04"), "description": "I'm OBSESSED With Cupcakes And The Eiffel Tower!! My Favorite Youtuber Is @shanedawson!! Also, I'm A Disney Addict!!", "followers_count": 1338, "friends_count": 693, "statues_count": 202347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburgh, PA", "id": "946ccd22e1c9cda1", "name": "Pittsburgh", "place_type": "city", "bounding_box": rectangle("-80.095586,40.36158 -79.865793,40.501198") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42003, "countyName": "Allegheny", "cityID": 4261000, "cityName": "Pittsburgh" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732009156608, "text": "Horror / action !!! https://t.co/GRjzpPsfPf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173326966, "name": "jenny", "screen_name": "jennnniebean", "lang": "en", "location": "Dallas, TX - Irving, TX", "create_at": date("2013-02-12"), "description": "just a small town girl livin' in a lonely world | seventeen | cupcake pancakes | ur cute | bye", "followers_count": 619, "friends_count": 347, "statues_count": 51865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732013334528, "text": "I can only whisper", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983544132, "name": "MILCA♋️", "screen_name": "milcaaa19", "lang": "en", "location": "Everett, WA", "create_at": date("2012-12-01"), "description": "• 19 ♋️ the song La Chona is about me • Jeremiah❣", "followers_count": 489, "friends_count": 260, "statues_count": 43912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasco, WA", "id": "41647560efe1db77", "name": "Pasco", "place_type": "city", "bounding_box": rectangle("-119.230869,46.198319 -119.033335,46.292793") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53021, "countyName": "Franklin", "cityID": 5353545, "cityName": "Pasco" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732076105728, "text": "OMFG https://t.co/ACtWcPuLqb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872077657, "name": "B money", "screen_name": "BethanieLester_", "lang": "en", "location": "null", "create_at": date("2012-10-10"), "description": "B from Texas", "followers_count": 1095, "friends_count": 413, "statues_count": 22619 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732097253377, "text": "Pininfarina Home Design https://t.co/jWFuuU4Ao1 #InteriorDesign", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.980904,40.758743"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "InteriorDesign" }}, "user": { "id": 16266811, "name": "Dexigner", "screen_name": "dexigner", "lang": "en", "location": "New York", "create_at": date("2008-09-12"), "description": "Dexigner is the leading online portal for designers, architects, illustrators, engineers, artists, and creatives of all kinds.", "followers_count": 261047, "friends_count": 230, "statues_count": 13051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732311027713, "text": "I really want sushi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 448206796, "name": "danielle", "screen_name": "Freakin_Danni", "lang": "en", "location": "satx ", "create_at": date("2011-12-27"), "description": "the bad mood Kanye | #UIW20", "followers_count": 1438, "friends_count": 686, "statues_count": 70787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732659126272, "text": "Might pull up in my bucket, this '9 holds a good dozen https://t.co/UD8NuSiPXk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2439249534, "name": "yung Rainy", "screen_name": "Rain_4_Real", "lang": "en", "location": "Tucson, AZ", "create_at": date("2014-04-11"), "description": "'how does a yung guy go so hard?'", "followers_count": 311, "friends_count": 229, "statues_count": 7681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732747194368, "text": "I'm a beast at flip cup", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 577483852, "name": "Perla", "screen_name": "PEARLZEZ", "lang": "en", "location": "Isla Vista, CA", "create_at": date("2012-05-11"), "description": "null", "followers_count": 386, "friends_count": 445, "statues_count": 14433 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Isla Vista, CA", "id": "d82e2984d5b9a65e", "name": "Isla Vista", "place_type": "city", "bounding_box": rectangle("-119.881745,34.404565 -119.828412,34.438713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 636868, "cityName": "Isla Vista" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732747239425, "text": "Nothing like being at the hospital at 2am in the morning. https://t.co/NStcvkCdLU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 120918658, "name": "Cody Sicard", "screen_name": "cocoa1324", "lang": "en", "location": " Iowa", "create_at": date("2010-03-07"), "description": "Sophomore @ BGM | 16 | Grinnell", "followers_count": 100, "friends_count": 90, "statues_count": 773 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marshalltown, IA", "id": "52c934b41e5c5dbd", "name": "Marshalltown", "place_type": "city", "bounding_box": rectangle("-92.969688,42.004257 -92.860288,42.070422") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19127, "countyName": "Marshall", "cityID": 1949755, "cityName": "Marshalltown" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732831096832, "text": "thanks Julia ☺️�� miss you https://t.co/4ebWqrqfTq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 858829242, "name": "drey", "screen_name": "audreyy_garciaa", "lang": "en", "location": "null", "create_at": date("2012-10-02"), "description": "null", "followers_count": 324, "friends_count": 170, "statues_count": 2193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rio Rancho, NM", "id": "0046bfef79c8e224", "name": "Rio Rancho", "place_type": "city", "bounding_box": rectangle("-106.757623,35.217658 -106.566425,35.374708") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35043, "countyName": "Sandoval", "cityID": 3563460, "cityName": "Rio Rancho" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176732944367616, "text": "No one understands how badly I want a pet ferret ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 166061486, "name": "Giselle", "screen_name": "gisssi_", "lang": "en", "location": "Texas, USA", "create_at": date("2010-07-12"), "description": "null", "followers_count": 1289, "friends_count": 217, "statues_count": 71380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176733053394944, "text": "Dance the night awaaaaay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 862358496, "name": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀GrooVee ☁️", "screen_name": "polllaaa", "lang": "en", "location": "Houston, TX", "create_at": date("2012-10-04"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀[insert bogus quote here] ⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀", "followers_count": 551, "friends_count": 298, "statues_count": 6666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176733229551616, "text": "Eat shit liver", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 474572466, "name": "Virgil", "screen_name": "Prototype_Kelly", "lang": "en", "location": "Earth C137", "create_at": date("2012-01-25"), "description": "to show the living the way of the damned", "followers_count": 196, "friends_count": 234, "statues_count": 10918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-05-30T00:00:03.000Z"), "id": 737176733468655616, "text": "@Inked_villain eat", "in_reply_to_status": 737176108429246469, "in_reply_to_user": 258574931, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 258574931 }}, "user": { "id": 2961955544, "name": "Lil Sanchie", "screen_name": "finnesstt", "lang": "en", "location": "finessingniggasinHouston", "create_at": date("2015-01-06"), "description": "sc | Valerie_1364", "followers_count": 2102, "friends_count": 903, "statues_count": 70989 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Bend, TX", "id": "2a9e190efe38237e", "name": "Mission Bend", "place_type": "city", "bounding_box": rectangle("-95.681932,29.680892 -95.6342,29.719902") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848772, "cityName": "Mission Bend" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176733531529221, "text": "Happy birthday to my beautiful girlfriend, I'm so glad to have you and I hope your birthday goes great today ❤️ https://t.co/C7K68sAjTw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2933630178, "name": "Yeezy", "screen_name": "RylieIbison00", "lang": "en", "location": "null", "create_at": date("2014-12-17"), "description": "Name one Genius that ain't crazy •Arlene•", "followers_count": 371, "friends_count": 295, "statues_count": 2900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "French Valley, CA", "id": "0196f2d33e5a1d73", "name": "French Valley", "place_type": "city", "bounding_box": rectangle("-117.137136,33.564564 -117.083609,33.627328") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 626067, "cityName": "French Valley" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176733552496644, "text": "my smile�� https://t.co/D0HVeJQnzN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447215459, "name": "Raven Green", "screen_name": "greennayraven", "lang": "en", "location": "null", "create_at": date("2011-12-26"), "description": "#LongLiveGio", "followers_count": 923, "friends_count": 869, "statues_count": 28051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734060154880, "text": "Wind 0.3 mph S. Barometer 29.921 in, Falling. Temperature 65.0 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734252961795, "text": "No time to stress over people who don't stress over me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1109953447, "name": "maddy", "screen_name": "maddydaltonn", "lang": "en", "location": "null", "create_at": date("2013-01-21"), "description": "ig - maddydaltonn ♚", "followers_count": 1182, "friends_count": 324, "statues_count": 30525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Medford, OR", "id": "7520fc0be21c62bf", "name": "Medford", "place_type": "city", "bounding_box": rectangle("-122.96154,42.288726 -122.776437,42.398452") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4147000, "cityName": "Medford" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734575955968, "text": "Rose with my girls ���� https://t.co/dUtwWLUPHq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2309382428, "name": "Majesta", "screen_name": "Majesta_pal16", "lang": "en", "location": "null", "create_at": date("2014-01-24"), "description": "null", "followers_count": 214, "friends_count": 185, "statues_count": 852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thornton, CO", "id": "36148a9a49d3da69", "name": "Thornton", "place_type": "city", "bounding_box": rectangle("-105.015543,39.838926 -104.884147,39.972023") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8001, "countyName": "Adams", "cityID": 877290, "cityName": "Thornton" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734714322944, "text": "i always made fun of him for having a volks but maybe he'll impress me this time show me what $20k can do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1478328469, "name": "r", "screen_name": "bbyxrxb", "lang": "en", "location": "null", "create_at": date("2013-06-02"), "description": "null", "followers_count": 3, "friends_count": 127, "statues_count": 26918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln, CA", "id": "0026c52b089aab9c", "name": "Lincoln", "place_type": "city", "bounding_box": rectangle("-121.340149,38.839106 -121.222598,38.911502") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 641474, "cityName": "Lincoln" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734768988160, "text": "First you get that money", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2721448743, "name": "Project Baby", "screen_name": "HollowaySZN", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-07-22"), "description": "⌚HardKnocks university ☃ Professional Football Player ! #22 #JacksonCounty !!!", "followers_count": 565, "friends_count": 303, "statues_count": 31331 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176734878040064, "text": "That was insane lmaooo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1974219092, "name": "lil mango ♡", "screen_name": "cactusjuliana", "lang": "en", "location": "snapchat: juliana.47", "create_at": date("2013-10-20"), "description": "never give up.", "followers_count": 106, "friends_count": 100, "statues_count": 4432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crowley, TX", "id": "0045d2f920fd11d3", "name": "Crowley", "place_type": "city", "bounding_box": rectangle("-97.437977,32.534562 -97.334312,32.600927") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4817960, "cityName": "Crowley" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735054102528, "text": "@cupcakemuch @tamsenrochelle_ when I get married I just wanna do a GoFundMe in lieu of actual wedding gifts. Don't need appliances ya know", "in_reply_to_status": 737171970979069952, "in_reply_to_user": 3104078240, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3104078240, 28273530 }}, "user": { "id": 2575186724, "name": "Melissa", "screen_name": "melissaaxm", "lang": "en", "location": "Salt Lake City ", "create_at": date("2014-06-18"), "description": "21 • NS ♡ • enjoys comic books & video games •", "followers_count": 669, "friends_count": 342, "statues_count": 19376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735318319104, "text": "@brookieRobinsun BROEOWKWKW I FOUND IT https://t.co/CUPYkKv8QL", "in_reply_to_status": -1, "in_reply_to_user": 2735121859, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2735121859 }}, "user": { "id": 111509659, "name": "arika", "screen_name": "arikabonney", "lang": "en", "location": "CA", "create_at": date("2010-02-04"), "description": "sugar sugar", "followers_count": 454, "friends_count": 267, "statues_count": 4838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735477686273, "text": "Wind 1.6 mph ENE. Barometer 29.973 in, Falling. Temperature 71.7 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735569977344, "text": "He was pretty drunk and before he knew it old Bernie was a roughneck throwing slips and turnin trips.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1556431698, "name": "Micah Lomas", "screen_name": "lomstar68", "lang": "en", "location": "Oklahoma ", "create_at": date("2013-06-29"), "description": "Done got out of hand", "followers_count": 351, "friends_count": 470, "statues_count": 3668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735733547009, "text": "What goes first in a root beer float... Root beer or ice cream?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551269598, "name": "Gabe Shmabe™", "screen_name": "_shmabe", "lang": "en", "location": "LA-SD-BAY", "create_at": date("2012-04-11"), "description": "Launchpad McQuack. IG: _shmabe", "followers_count": 176, "friends_count": 302, "statues_count": 2507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735809167360, "text": "Genesis Medical Aesthetics, Tustin, CA, 92780 - https://t.co/eMX0IaZjM5\n\n#genesismedicalaesthetics #medicalcenter #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.814646,33.743035"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "genesismedicalaesthetics", "medicalcenter", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 598, "friends_count": 575, "statues_count": 2534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176735989436416, "text": "Lowkey don't wanna watch softball anymore bc Cheridan's a senior & she's my favorite but Jenna Lilley is a close second so... ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2996556619, "name": "hailey", "screen_name": "haileyxbug", "lang": "en", "location": "null", "create_at": date("2015-01-25"), "description": "north salem high school || serengetee rep || Caitlin's adorable ❤️", "followers_count": 72, "friends_count": 167, "statues_count": 2595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176736304140288, "text": "Wind 0.0 mph ---. Barometer 30.022 in, Rising slowly. Temperature 66.5 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 39, "friends_count": 4, "statues_count": 27790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176736887013376, "text": "So honored to have received over $4,000 in scholarships tonight. I wouldn't be able to attend college without our sponsoring bodies. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 369008771, "name": "Rylee", "screen_name": "Rylee__Nicole", "lang": "en", "location": "null", "create_at": date("2011-09-06"), "description": "I've never been to heaven, but I've been to Oklahoma {State} • Stillwater • Moore • #okstate19 • 1-1-14", "followers_count": 682, "friends_count": 282, "statues_count": 16221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norman, OK", "id": "27de1a6e27075587", "name": "Norman", "place_type": "city", "bounding_box": rectangle("-97.51982,35.15327 -97.370527,35.291332") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40027, "countyName": "Cleveland", "cityID": 4052500, "cityName": "Norman" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176737050710016, "text": "Hope they're worth it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2812075305, "name": "Kya Taylor", "screen_name": "KyaTaylor", "lang": "en", "location": "null", "create_at": date("2014-10-06"), "description": "Ms. Brickhouse // NYC ✈️ CHI // DePaul Acting BFA 19'// Aries", "followers_count": 292, "friends_count": 457, "statues_count": 884 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-05-30T00:00:04.000Z"), "id": 737176737172230144, "text": "love love love your hair ❤️�� https://t.co/rqeziuY1KA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2326025180, "name": "morgan", "screen_name": "fuentesurmom", "lang": "en", "location": "null", "create_at": date("2014-02-03"), "description": "i like long walks on the beach & hate rude people obsessed with pierce the veil & ur mom jokes", "followers_count": 840, "friends_count": 633, "statues_count": 27101 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-05-30T00:00:05.000Z"), "id": 737176737847513088, "text": "Tonight, I'mma let you be the captain.\nTonight, I'mma let you do your thing.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218240955, "name": "Britney Sometimes .", "screen_name": "BeesInterlude", "lang": "en", "location": "Denton, TX", "create_at": date("2010-11-21"), "description": "✨UNT ➖ Blogger ➖ UNT-NABJ ➖ Evolving ✨", "followers_count": 1098, "friends_count": 807, "statues_count": 100254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, TX", "id": "0557e33c3bef86ac", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-96.823367,32.559599 -96.747695,32.649973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4841212, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-05-30T00:00:05.000Z"), "id": 737176738439036929, "text": "shes so shy yet so freaky.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3285711326, "name": "Morgan Talley", "screen_name": "Lmhayevn", "lang": "en", "location": "Odessa, TX", "create_at": date("2015-07-20"), "description": "don't tell my mom I got a drug problem", "followers_count": 924, "friends_count": 2754, "statues_count": 1751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-05-30T00:00:05.000Z"), "id": 737176738849947648, "text": "Temp: 63.2°F - Dew Point: 59.6° - Wind: WSW @ 1.3 mph - Gust: 3.1 - Rain Today: 0.00in. - Pressure: 29.97in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 9, "friends_count": 13, "statues_count": 18411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539104741556224, "text": "Yesterday I was a fan\nNow they treat me like tha man\nBoi you wouldn't understand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 453667474, "name": "Dynamics", "screen_name": "DynamicsRJG", "lang": "en", "location": "Chicago✈️DTX", "create_at": date("2012-01-02"), "description": "#GG http://Twitch.tv/richman16 GT: Dynamics nE | Halo Player | Chicago - DTX | I'm THE Richman16 | Certified Young Gun | 18", "followers_count": 1539, "friends_count": 991, "statues_count": 92033 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wylie, TX", "id": "5b96c12e41e49aa5", "name": "Wylie", "place_type": "city", "bounding_box": rectangle("-96.596044,32.981938 -96.489165,33.054983") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4880356, "cityName": "Wylie" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539105190338560, "text": "I'm actually in a simp mood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2765910663, "name": "⁶", "screen_name": "youngmorganne", "lang": "en", "location": "posted up", "create_at": date("2014-09-09"), "description": "single & probably partying FDB", "followers_count": 840, "friends_count": 418, "statues_count": 18164 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539105634951168, "text": "@Alyssabrianna__ ok", "in_reply_to_status": 737538683734102016, "in_reply_to_user": 2694697843, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2694697843 }}, "user": { "id": 497629097, "name": "nate", "screen_name": "Nastynate79", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-02-19"), "description": "hi", "followers_count": 517, "friends_count": 322, "statues_count": 14595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539105903411201, "text": "@kassiefornash naaaa you gotta bring the shell to me ��", "in_reply_to_status": 737538627526262788, "in_reply_to_user": 293059231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 293059231 }}, "user": { "id": 444801454, "name": "Ty", "screen_name": "tmckean_34", "lang": "en", "location": "OHIO", "create_at": date("2011-12-23"), "description": "null", "followers_count": 754, "friends_count": 376, "statues_count": 36703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reynoldsburg, OH", "id": "ce66852c89aa6582", "name": "Reynoldsburg", "place_type": "city", "bounding_box": rectangle("-82.835601,39.931348 -82.693943,40.018293") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3966390, "cityName": "Reynoldsburg" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539106373132288, "text": "���� @KlayThompson #LOVE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "LOVE" }}, "user_mentions": {{ 1703636814 }}, "user": { "id": 19404295, "name": "Tip Her", "screen_name": "TiporTiff", "lang": "en", "location": "Detroit / Brooklyn ", "create_at": date("2009-01-23"), "description": "i'm getting better still.", "followers_count": 1452, "friends_count": 2066, "statues_count": 128123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539106679332865, "text": "I swear I'm kind of nice, can I get some good karma now ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 427028273, "name": "chris", "screen_name": "chrissanthonyy", "lang": "en", "location": "Los Scandelous, CA", "create_at": date("2011-12-02"), "description": "null", "followers_count": 223, "friends_count": 144, "statues_count": 2973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539107300073472, "text": "I still feel strongly about things that I'm trying to let go. Something has got to give.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3146892710, "name": "bad girl in Equinox✨", "screen_name": "JBTaughtYa", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2015-04-09"), "description": "wavy. fsu. soflo", "followers_count": 356, "friends_count": 514, "statues_count": 57839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539107354599424, "text": "Can't wait to pick up the new whip on Friday", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 93652884, "name": "HypeBeast", "screen_name": "mramazing803", "lang": "en", "location": "Metro S.C.", "create_at": date("2009-11-30"), "description": "Carolina Country Boy #GamecockNation All things outdoors excite me! IG: juniorflip803", "followers_count": 756, "friends_count": 2005, "statues_count": 2366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winnsboro Mills, SC", "id": "35546dff7fbb2875", "name": "Winnsboro Mills", "place_type": "city", "bounding_box": rectangle("-81.093009,34.333954 -81.06134,34.369275") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45039, "countyName": "Fairfield", "cityID": 4578505, "cityName": "Winnsboro Mills" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539107472039937, "text": "Yessss https://t.co/qMEUHpKEI7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 583488323, "name": "kaycea", "screen_name": "theKayceaLaynee", "lang": "en", "location": "huntsville. AL ", "create_at": date("2012-05-17"), "description": "May 7th. = 2⃣4️⃣♉️ - . ✨- ➡️ IG ; kaycealayne ; Laynek13 ; 3.7.16. ❤️", "followers_count": 568, "friends_count": 175, "statues_count": 6942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108172484610, "text": "Ha my childhood crush", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368154359, "name": "Jáson", "screen_name": "NonameJason", "lang": "en", "location": "California Vibe ", "create_at": date("2011-09-04"), "description": "|RIP Claudia Berenice Silva|", "followers_count": 190, "friends_count": 156, "statues_count": 10351 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108172496896, "text": "If you up snapchat me -devonwebster_18", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 826101416, "name": "Mr. Webster", "screen_name": "DevDev_Webster", "lang": "en", "location": "United States", "create_at": date("2012-09-15"), "description": "Get to the top and never be brought back down!|#Ya_Bayeh|Jumper & Thrower @Niacc|", "followers_count": 531, "friends_count": 282, "statues_count": 9091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Troy, IL", "id": "005acf34787eaab3", "name": "Troy", "place_type": "city", "bounding_box": rectangle("-89.930885,38.701346 -89.842094,38.748868") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17119, "countyName": "Madison", "cityID": 1776199, "cityName": "Troy" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108189306880, "text": "watch who you call your friend!! I cut bitches off already and it ain't even been a month since we graduated!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58082443, "name": "Janya✨", "screen_name": "_janyasmith", "lang": "en", "location": "205", "create_at": date("2009-07-18"), "description": "#AAMU20", "followers_count": 882, "friends_count": 824, "statues_count": 22155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hueytown, AL", "id": "1f6120147686a129", "name": "Hueytown", "place_type": "city", "bounding_box": rectangle("-87.066047,33.402208 -86.918524,33.494969") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 136448, "cityName": "Hueytown" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108206043146, "text": "@lauraa_adams ���� https://t.co/wjOV3rS1Tl", "in_reply_to_status": -1, "in_reply_to_user": 2349589730, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2349589730 }}, "user": { "id": 2904648076, "name": "αиαкαℓια ♛", "screen_name": "andrea_teesdale", "lang": "en", "location": "home alone", "create_at": date("2014-12-03"), "description": "I don't think I'm all this or that, but I'm all me. - Eazy-E", "followers_count": 399, "friends_count": 441, "statues_count": 5979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oroville, CA", "id": "003f1e4e2f87ed96", "name": "Oroville", "place_type": "city", "bounding_box": rectangle("-121.594948,39.463816 -121.504685,39.533958") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6007, "countyName": "Butte", "cityID": 654386, "cityName": "Oroville" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108281536512, "text": "@ebbtideapp Tide in Ediz Hook, Washington 05/31/2016\n Low 5:53am 1.9\nHigh 11:44am 4.4\n Low 5:04pm 2.5\nHigh 11:33pm 7.0", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-123.4133,48.14"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 97, "friends_count": 1, "statues_count": 37349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Angeles, WA", "id": "6e1e36836f2a74a9", "name": "Port Angeles", "place_type": "city", "bounding_box": rectangle("-123.52563,48.083505 -123.331735,48.142238") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53009, "countyName": "Clallam", "cityID": 5355365, "cityName": "Port Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108285730816, "text": "at the end of the day, it is what it is", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 772300453, "name": "D E A S H A❣", "screen_name": "DeeHooper95", "lang": "en", "location": "Moore, OK", "create_at": date("2012-08-21"), "description": "You can get more bees with honey then you can vinegar . Snapchat: Deehooper95", "followers_count": 885, "friends_count": 694, "statues_count": 65334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108357054464, "text": "There are all kinds of love in this world but never the same love twice", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 533892664, "name": "ELIJAH COOKS", "screen_name": "ElijahCooks", "lang": "en", "location": "Atascadero, CA", "create_at": date("2012-03-22"), "description": "HOOPER", "followers_count": 536, "friends_count": 446, "statues_count": 1015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atascadero, CA", "id": "2bd8aff9ffdbc99a", "name": "Atascadero", "place_type": "city", "bounding_box": rectangle("-120.726857,35.437843 -120.618411,35.534478") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 603064, "cityName": "Atascadero" } }
+{ "create_at": datetime("2016-05-31T00:00:00.000Z"), "id": 737539108705177600, "text": "off guard and always looking mad https://t.co/aLLa4Tbqer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 348161864, "name": "isayanne", "screen_name": "_Isayanne", "lang": "en", "location": "Rialto, CA", "create_at": date("2011-08-03"), "description": "✨", "followers_count": 432, "friends_count": 377, "statues_count": 21344 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539109019783169, "text": "@jenicejaclyn12 ����", "in_reply_to_status": 737537849684168704, "in_reply_to_user": 2578161024, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2578161024 }}, "user": { "id": 842846443, "name": "xonatasha", "screen_name": "xondp", "lang": "en", "location": "null", "create_at": date("2012-09-23"), "description": "@theweeknd XO", "followers_count": 1434, "friends_count": 1404, "statues_count": 43760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539109317533698, "text": "Bye �� https://t.co/eCKyHDWDY8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3273538322, "name": "kelly kells", "screen_name": "Foreignhbk__", "lang": "en", "location": "San Francisco, CA", "create_at": date("2015-07-09"), "description": "null", "followers_count": 659, "friends_count": 404, "statues_count": 12923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539110009610241, "text": "It's just a G https://t.co/eGs1i572zl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 251362549, "name": "$", "screen_name": "dedicvtedxo", "lang": "en", "location": "☁ ", "create_at": date("2011-02-12"), "description": "T E S F A Y E (dedicated to a legend) @theweeknd", "followers_count": 1249, "friends_count": 967, "statues_count": 129 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539110626172928, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":62.2°F Wind:1.1mph Pressure: 30.00hpa Falling slowly Rain Today 0.00in. Forecast: Occasional precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 18, "statues_count": 320923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539111066566661, "text": "Wind 0.0 mph ---. Barometer 29.946 in, Steady. Temperature 67.8 °F. Rain today 0.00 in. Humidity 89%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539111406292992, "text": "KLAX 310653Z 26008KT 10SM BKN010 OVC012 15/13 A2991 RMK AO2 SLP127 T01500128 $", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.40806839,33.9424955"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1729173072, "name": "KLAX METAR", "screen_name": "klax_metar", "lang": "ja", "location": "Los Angeles, CA", "create_at": date("2013-09-04"), "description": "Los Angeles International Airport", "followers_count": 8, "friends_count": 1, "statues_count": 26393 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539111649583105, "text": "I can’t help that I don’t care. I can’t force myself to care. I literally just don’t", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27262390, "name": "✨Cahley Boak✨", "screen_name": "Cahley", "lang": "en", "location": "College Station, TX", "create_at": date("2009-03-28"), "description": "one day at a time • texas a&m '18", "followers_count": 965, "friends_count": 444, "statues_count": 784 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539111670583299, "text": "Temp 68.2° Hi/Lo 69.3/68.1 Rng 1.2° WC 68.2° Hmd 95% Rain 0.00\" Storm 0.48\" BAR 29.930 Rising DP 66.7° Wnd 0mph Dir --- Gst 5mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 72, "friends_count": 123, "statues_count": 19123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539112505245696, "text": "eshlep.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3008592931, "name": "Jei", "screen_name": "a_k_a_dabiana", "lang": "en", "location": "solitude ", "create_at": date("2015-02-02"), "description": "null", "followers_count": 71, "friends_count": 263, "statues_count": 1438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539112660410369, "text": "I don't understand, why can't you be like your brother?����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3195007608, "name": "Jess☹", "screen_name": "Jessbaaby__", "lang": "en", "location": "Biggest Little City ", "create_at": date("2015-05-14"), "description": "7teen", "followers_count": 100, "friends_count": 148, "statues_count": 2285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reno, NV", "id": "4b25aded08900fd8", "name": "Reno", "place_type": "city", "bounding_box": rectangle("-119.953849,39.350749 -119.700515,39.674123") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3260600, "cityName": "Reno" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539112853340160, "text": "@stevechimenti Yes, thats how it works. He's the manager. Correct. He gets ripped for keeping him in... and ripped for pulling him.", "in_reply_to_status": 737527635056549890, "in_reply_to_user": 900662598, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 900662598 }}, "user": { "id": 1970175710, "name": "Parker andTheMan", "screen_name": "parkerandtheman", "lang": "en", "location": "Detroit/Las Vegas", "create_at": date("2013-10-18"), "description": "radio/TV dudes, podcasts, Rob on WXYZ Action News at 10 on TV20", "followers_count": 520, "friends_count": 144, "statues_count": 11151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-05-31T00:00:01.000Z"), "id": 737539112895270912, "text": "2016/07/08: Saint Paul, MN, United States: Break Dance Basics https://t.co/9deaJHHckk https://t.co/wxx3milGZk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.1241023,44.9287518"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 325786388, "name": "Step x Step", "screen_name": "stepxstepdance", "lang": "en", "location": "The World Is Our Dancefloor", "create_at": date("2011-06-28"), "description": "The Official Step x Step Twitter. Street Dance Media Uplifting Dancers Around The World Step By Step", "followers_count": 8045, "friends_count": 3096, "statues_count": 33963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "St Paul, MN", "id": "60e2c37980197297", "name": "St Paul", "place_type": "city", "bounding_box": rectangle("-93.207783,44.890752 -93.003514,44.992279") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27123, "countyName": "Ramsey", "cityID": 2758000, "cityName": "St. Paul" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113075671041, "text": "The best sex is still you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603756157, "name": "C", "screen_name": "StarkLines", "lang": "en", "location": "Elizabeth, NJ", "create_at": date("2012-06-09"), "description": "MGMT. For @TheRealJTune | Believers Dynasty Inquires: BelieversDynastyMusic@gmail.com", "followers_count": 3912, "friends_count": 2800, "statues_count": 104867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Elizabeth, NJ", "id": "b74cebcb62a1a686", "name": "Elizabeth", "place_type": "city", "bounding_box": rectangle("-74.254211,40.634285 -74.138838,40.690673") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34039, "countyName": "Union", "cityID": 3421000, "cityName": "Elizabeth" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113419575296, "text": "Clay Township: Temp. 66.0F. (66.0/71.8F) Wind 0 mph NE MaxGust 2mph@02:42. Barometer 29.97 in. Rain today 0.00 in. Humidity 99%. #inwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.14833333,39.93138889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "inwx" }}, "user": { "id": 36190715, "name": "N7CZ", "screen_name": "N7CZ_EM69ww", "lang": "en", "location": "Indianapolis, Indiana", "create_at": date("2009-04-28"), "description": "Terrestrial VHF/UHF Weak Signal & EME Enthusiast", "followers_count": 807, "friends_count": 645, "statues_count": 97918 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmel, IN", "id": "eca35cbd75f0a1e6", "name": "Carmel", "place_type": "city", "bounding_box": rectangle("-86.241245,39.926048 -86.023652,40.000451") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1810342, "cityName": "Carmel" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113432141824, "text": "Wind 0.0 mph ---. Barometer 29.952 in, Falling slowly. Temperature 62.0 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113461547008, "text": "05/31@03:00 - Temp 70.4F, WC 70.4F. Wind 0.0mph S, Gust 1.0mph. Bar 29.955in, Falling slowly. Rain 0.01in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113667026945, "text": "@peelsfeels my love life", "in_reply_to_status": 737539048298799105, "in_reply_to_user": 128097990, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 128097990 }}, "user": { "id": 2374496005, "name": "Lindsey Floyd", "screen_name": "lindseymfloyd", "lang": "en", "location": "ASU", "create_at": date("2014-03-05"), "description": "Some call me Lindsey but you can call me Future President of the United States. I tweet about politics, poetry, and puppies.", "followers_count": 552, "friends_count": 430, "statues_count": 8824 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113755115521, "text": "Hood down vibes!! ��☀️���� https://t.co/wnr98012SW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 533570309, "name": "Pratisha Lad", "screen_name": "PratishaL", "lang": "en", "location": "null", "create_at": date("2012-03-22"), "description": "null", "followers_count": 48, "friends_count": 88, "statues_count": 193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539113826451456, "text": "@jassminez he can block them online, but not in real life", "in_reply_to_status": 737536949276147712, "in_reply_to_user": 637239380, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 637239380 }}, "user": { "id": 718545445, "name": "Stephmoney", "screen_name": "Well_done_slut", "lang": "en", "location": "Oakland, CA", "create_at": date("2012-07-26"), "description": "Snapchat: puta_perra1010", "followers_count": 210, "friends_count": 185, "statues_count": 10804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539114149412870, "text": "So yall really dont see 3 niggas? �� https://t.co/YoK7AuOwKO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 75443387, "name": "Deja ❤️", "screen_name": "Hoeslovedejaa", "lang": "en", "location": "Tallahassee, FL", "create_at": date("2009-09-18"), "description": "Snapchat / Simplydeja_xox", "followers_count": 2920, "friends_count": 1987, "statues_count": 106857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539114468151297, "text": "shows how much you wanna talk to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1059329282, "name": "alyiah lane", "screen_name": "alyiah_lane22", "lang": "en", "location": "hobart, ok", "create_at": date("2013-01-03"), "description": "be so good they can't ignore you| sc: alyiahlane_22| in love with Jesus Christ", "followers_count": 614, "friends_count": 466, "statues_count": 20447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hobart, OK", "id": "cbec67f30e8f400c", "name": "Hobart", "place_type": "city", "bounding_box": rectangle("-99.10939,35.01102 -99.07415,35.039013") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40075, "countyName": "Kiowa", "cityID": 4035000, "cityName": "Hobart" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539114933702656, "text": "Temp: 72.4°F Wind:0.0mph Pressure: 29.922hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539115156017152, "text": "I really should be sleep right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48801316, "name": "RIPkeKE.", "screen_name": "girlMELERIE", "lang": "en", "location": "Texas, USA", "create_at": date("2009-06-19"), "description": "25", "followers_count": 1197, "friends_count": 782, "statues_count": 57960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tinley Park, IL", "id": "5b0b1baf24cf0a6a", "name": "Tinley Park", "place_type": "city", "bounding_box": rectangle("-87.853527,41.527889 -87.742767,41.604053") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1775484, "cityName": "Tinley Park" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539115260878851, "text": "One of the sexiest barefooters I've seen. Amazing bright flip flops and sexy white feet https://t.co/SSd7PvvKvA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143566721, "name": "Betty Barefoot", "screen_name": "barefootfetish", "lang": "en", "location": "West Coast", "create_at": date("2010-05-13"), "description": "I secretly videotape and take pictures of White & Latino women's barefeet in flipflops. There are a few exceptions of amazing feet from other races.", "followers_count": 201, "friends_count": 145, "statues_count": 330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539115357343744, "text": "Bruh you dropped the ball. �� https://t.co/0mgXbLtlTa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 263912857, "name": ". ShanDaddy✨™", "screen_name": "___Shannonnn", "lang": "en", "location": "662 ✈️ 601 Hattiesburg, MS ", "create_at": date("2011-03-10"), "description": "#USM18 .", "followers_count": 1552, "friends_count": 907, "statues_count": 56835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539115751604224, "text": "Hang on, gotta blow up my babyzaddy sc real quick", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 244765369, "name": "Tayedaddy", "screen_name": "_xoxoTAYE", "lang": "en", "location": "Louisiana ✈️ NewYork ", "create_at": date("2011-01-29"), "description": "...In The Process of Serving Our Country #LetGo&LetGod", "followers_count": 1536, "friends_count": 915, "statues_count": 51979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Drum, NY", "id": "69108214c4615b4d", "name": "Fort Drum", "place_type": "city", "bounding_box": rectangle("-75.838074,44.010422 -75.692237,44.073511") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36045, "countyName": "Jefferson", "cityID": 3626759, "cityName": "Fort Drum" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539116531777538, "text": "Even if it's not a lot shit can be way worse https://t.co/Tt0NqNJ2g4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3386483163, "name": "jc", "screen_name": "rwcbjay", "lang": "en", "location": "$tockton, CA", "create_at": date("2015-07-21"), "description": "null", "followers_count": 336, "friends_count": 198, "statues_count": 13115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539116565299200, "text": "I be at work in my feelings because most of the time I be wishing I had a nigga to go lay up w whn I get off ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3236565730, "name": "️", "screen_name": "porciajenae", "lang": "en", "location": "null", "create_at": date("2015-05-05"), "description": "null", "followers_count": 359, "friends_count": 276, "statues_count": 5875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-05-31T00:00:02.000Z"), "id": 737539116661755908, "text": "Nah I can't even sleep. My anxiety is acting up smfh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 99839522, "name": "Mochahontas", "screen_name": "BeautifulCrime_", "lang": "en", "location": "Buffalo, NY", "create_at": date("2009-12-27"), "description": "Student • Model • Dancer", "followers_count": 1560, "friends_count": 377, "statues_count": 25539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539117504860163, "text": "@OverlyLiked I was honestly being a little bitch �� I felt the tip and freaked out so I said ouch so he would stop", "in_reply_to_status": 737538335577538560, "in_reply_to_user": 46842722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46842722 }}, "user": { "id": 489365448, "name": "Ky", "screen_name": "kvickey_", "lang": "en", "location": "null", "create_at": date("2012-02-11"), "description": "I'm a princess tbh", "followers_count": 1610, "friends_count": 1035, "statues_count": 42043 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Orange, NJ", "id": "858cec499f8adba6", "name": "East Orange", "place_type": "city", "bounding_box": rectangle("-74.238236,40.745103 -74.190042,40.788099") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3419390, "cityName": "East Orange" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539117781647360, "text": "farmers tan is so unfortunate rip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3294861593, "name": "aja", "screen_name": "abraadzz", "lang": "en", "location": "null", "create_at": date("2015-05-22"), "description": "null", "followers_count": 384, "friends_count": 296, "statues_count": 5405 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainfield, IL", "id": "4320ab56929ffcdb", "name": "Plainfield", "place_type": "city", "bounding_box": rectangle("-88.293691,41.566483 -88.152433,41.681434") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1760287, "cityName": "Plainfield" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539118146588673, "text": "I just tried to zoom in on an insty pic... Total dad move.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 529015939, "name": "Paige McCrary", "screen_name": "paige_mccrary", "lang": "en", "location": "null", "create_at": date("2012-03-18"), "description": "Hey Bailey. Who put the Swedish Fish in the snack bin?", "followers_count": 774, "friends_count": 746, "statues_count": 5083 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Northridge, OH", "id": "0ab9dffe7df56813", "name": "Northridge", "place_type": "city", "bounding_box": rectangle("-83.807335,39.976781 -83.74907,40.02646") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39023, "countyName": "Clark", "cityID": 3956938, "cityName": "Northridge" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539118540853248, "text": "When your parents find 2 handles of alc in your room but don't say anything about it..����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1355793337, "name": "manders", "screen_name": "alampe12345", "lang": "en", "location": "null", "create_at": date("2013-04-15"), "description": "legen...wait for it...dary", "followers_count": 312, "friends_count": 299, "statues_count": 6559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539118654062593, "text": "ik im late �� but that damn game 7 �� OKC really deserved to had won that", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3888319709, "name": "✌︎ ✮ ✌︎", "screen_name": "treythegod__", "lang": "en", "location": "boolin' ㊗️", "create_at": date("2015-10-07"), "description": "null", "followers_count": 469, "friends_count": 462, "statues_count": 7314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539118691844096, "text": "����... Oh @cj_sorensen https://t.co/jcRsR0sguh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2560102706 }}, "user": { "id": 2624007592, "name": "tay", "screen_name": "taylerrbaylerr", "lang": "en", "location": "Long Beach, CA", "create_at": date("2014-06-19"), "description": "|| CA x AZ || 19 || sc:taylerrbaylerr || #curlygirlgang", "followers_count": 620, "friends_count": 852, "statues_count": 11894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539118838648834, "text": "Happy Birthday!! I love you and I can't wait to spend the day with you and make it the best❤️☺️\n@ashhleycutler https://t.co/sNSlaVu0b0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3057173112 }}, "user": { "id": 3023348179, "name": "Christophe", "screen_name": "Christian_FiveO", "lang": "en", "location": "cali☀️", "create_at": date("2015-02-16"), "description": "We stay united under the electric sky➕✖ ⚠️HDYNATION⚠️", "followers_count": 331, "friends_count": 713, "statues_count": 1812 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendora, CA", "id": "eb1bb64775708bc1", "name": "Glendora", "place_type": "city", "bounding_box": rectangle("-117.890263,34.10549 -117.809111,34.165551") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 630014, "cityName": "Glendora" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539119421652996, "text": "Temp 58.6°F Wind Chill 58.6°F RH 92% Wind 0.0 --- Gust 0.0 --- SLP 29.949 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 63, "statues_count": 35820 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539119551631361, "text": "All my thousands of belly button rings somehow managed to end up going down the drain in the bathtub.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301207641, "name": "ღkalo", "screen_name": "_kalooo", "lang": "en", "location": "DTX", "create_at": date("2011-05-18"), "description": "RIP Ma❤️IG: @_kalooo SC: kLo32 A$avage. -don't DM me cause I don't do internet flirting lmao", "followers_count": 1517, "friends_count": 730, "statues_count": 19506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbiana, AL", "id": "01807e8c5e7c891f", "name": "Columbiana", "place_type": "city", "bounding_box": rectangle("-86.634719,33.152193 -86.588275,33.193676") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1117, "countyName": "Shelby", "cityID": 116768, "cityName": "Columbiana" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539119639724032, "text": "Happy birthday Tatianna I love you. #OneYearCloserToDeath ET FOREVER! https://t.co/wxGEi7Tdyc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OneYearCloserToDeath" }}, "user": { "id": 3146601344, "name": "E", "screen_name": "eduardology", "lang": "en", "location": "null", "create_at": date("2015-04-09"), "description": "philosophy.", "followers_count": 248, "friends_count": 242, "statues_count": 633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539119824273408, "text": "Do your fucking hw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4715224634, "name": "〰", "screen_name": "anselwins", "lang": "en", "location": "null", "create_at": date("2016-01-05"), "description": "null", "followers_count": 175, "friends_count": 188, "statues_count": 1669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539120101089280, "text": "@Lenee__ come over", "in_reply_to_status": 737537928390283264, "in_reply_to_user": 453668340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 453668340 }}, "user": { "id": 611249576, "name": "Tincito", "screen_name": "elmartooo", "lang": "en", "location": "Liberty, TX", "create_at": date("2012-06-17"), "description": "SC: elmartooo", "followers_count": 1095, "friends_count": 821, "statues_count": 18862 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Liberty, TX", "id": "27a0a7a61c83eb4e", "name": "Liberty", "place_type": "city", "bounding_box": rectangle("-94.835584,30.042803 -94.753463,30.093237") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48291, "countyName": "Liberty", "cityID": 4842568, "cityName": "Liberty" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539120201793537, "text": "5. Can I get 5 dollars from you ? I really need it.", "in_reply_to_status": 737538822683033602, "in_reply_to_user": 2289875550, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2289875550, "name": "She Dont Love You.", "screen_name": "BfbDaPackman", "lang": "en", "location": "Flint, MI ✈️ Houston, TX ", "create_at": date("2014-01-13"), "description": "I live a valet life, I'm up front with everything.", "followers_count": 2269, "friends_count": 1352, "statues_count": 34815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jersey Village, TX", "id": "c7b527911412a784", "name": "Jersey Village", "place_type": "city", "bounding_box": rectangle("-95.620555,29.87235 -95.548681,29.914781") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4837612, "cityName": "Jersey Village" } }
+{ "create_at": datetime("2016-05-31T00:00:03.000Z"), "id": 737539121015463937, "text": "Brunettes will forever have my heart", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1610014014, "name": "yung milesさ", "screen_name": "don_king13", "lang": "en", "location": "LostInThe$auce", "create_at": date("2013-07-21"), "description": "lostislands*est.2015*", "followers_count": 1164, "friends_count": 1098, "statues_count": 19590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539121510416384, "text": "@acostajose100 it's on (: still hot now fuck outta my mentions", "in_reply_to_status": 737538902022492160, "in_reply_to_user": 3466245738, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3466245738 }}, "user": { "id": 3030836278, "name": "Cece N", "screen_name": "nunocece", "lang": "en", "location": "null", "create_at": date("2015-02-11"), "description": "null", "followers_count": 379, "friends_count": 171, "statues_count": 5333 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539121678188544, "text": "@Robbierosss @Kassay_ @lizzymariepena @JaredSheneman lmfao", "in_reply_to_status": 737539017516843008, "in_reply_to_user": 1089931628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1089931628, 3237865258, 473348088, 565012167 }}, "user": { "id": 580450546, "name": "Acosta", "screen_name": "lgacosta23", "lang": "en", "location": "Brownsville, TX", "create_at": date("2012-05-14"), "description": "UTRGV Baseball #43 | D1 Student-Athlete | 19 |Just a kid pursuing his dream. #RedSoxNation", "followers_count": 796, "friends_count": 512, "statues_count": 12051 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, TX", "id": "d48f32df6ecc8aaa", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-97.580085,25.858723 -97.374767,26.085488") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4810768, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539121707515904, "text": "That man Kendrick Lamar be goin dummy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3244138983, "name": "تريفور", "screen_name": "trevor_11239", "lang": "en", "location": "null", "create_at": date("2015-05-09"), "description": "from the 3️⃣0️⃣9️⃣ to the 7️⃣6️⃣5️⃣✈️", "followers_count": 250, "friends_count": 249, "statues_count": 2119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, IN", "id": "df1b6e7143e9c8d4", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-86.92488,40.339754 -86.768625,40.474718") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18157, "countyName": "Tippecanoe", "cityID": 1840788, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539121774661632, "text": "@Monica_Inman5 @GeorgeWillis25 accurate", "in_reply_to_status": 737538998554361856, "in_reply_to_user": 248484857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 248484857, 42295453 }}, "user": { "id": 127653160, "name": "Jeff", "screen_name": "DevinBagel", "lang": "en", "location": "Just south of Heaven", "create_at": date("2010-03-29"), "description": "ΠΚΑ", "followers_count": 949, "friends_count": 602, "statues_count": 6039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Girardeau, MO", "id": "19a69af030945159", "name": "Cape Girardeau", "place_type": "city", "bounding_box": rectangle("-89.628581,37.255486 -89.490907,37.3787") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29031, "countyName": "Cape Girardeau", "cityID": 2911242, "cityName": "Cape Girardeau" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539122173075456, "text": "Wind 8.3 mph NE. Barometer 29.93 in, Rising. Temperature 47.7 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 19, "friends_count": 5, "statues_count": 42595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539122462482434, "text": "Wind 0.0 mph ---. Barometer 29.886 in, Falling. Temperature 67.9 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539122802216960, "text": "Seeing younger kids try to immolate wat I do https://t.co/tohgcMIroy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2857715128, "name": "❌DrillTeam", "screen_name": "FinessWalker", "lang": "en", "location": "snapchat : jarred.5", "create_at": date("2014-11-02"), "description": "731Made | EAB 17u | HHS | If the sky is the limit , why is there footsteps on the moon", "followers_count": 234, "friends_count": 414, "statues_count": 194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, TN", "id": "76c39f0faecc2198", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-88.927948,35.533842 -88.749242,35.754808") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47113, "countyName": "Madison", "cityID": 4737640, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539123162947585, "text": "Wind 0.0 mph S. Barometer 29.869 in, Falling slowly. Temperature 72.2 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539123745984513, "text": "@kaylahugzalott Imean, it just don't make sense... From experience you know?... Like why reminisce gotta start over", "in_reply_to_status": 737538555220627456, "in_reply_to_user": 754345075, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 754345075 }}, "user": { "id": 765188292, "name": "Xavier Olaso", "screen_name": "HUG_n_KISSES", "lang": "en", "location": "null", "create_at": date("2012-08-17"), "description": "#ClipNation #RIPCoachB #SCIS #SCC", "followers_count": 212, "friends_count": 428, "statues_count": 3240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539123838230529, "text": "Wind 0.2 mph SSE. Barometer 29.835 in, Steady. Temperature 60.0 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539123905368066, "text": "@SDY292 they aren't shit for this.", "in_reply_to_status": 737453391203291136, "in_reply_to_user": 223262173, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 223262173 }}, "user": { "id": 1534557020, "name": "Serious Black", "screen_name": "NicT10", "lang": "en", "location": "The Left Coast", "create_at": date("2013-06-20"), "description": "Tech professional, philomath/know-it-all, Mentor helping kids bridge opportunity gaps, Concerned Citizen, Go Crimson& Go Blue!\nThicker Than a Bowl of Oatmeal", "followers_count": 1535, "friends_count": 1255, "statues_count": 118512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539123922112514, "text": "@SnarkLunge @mortisha888 Rachel??? What might that be?! ����", "in_reply_to_status": 737538088532992006, "in_reply_to_user": 2726798965, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2726798965, 2631285372 }}, "user": { "id": 4256149305, "name": "Nancy Lasocki", "screen_name": "nancylasocki", "lang": "en", "location": "Illinois, USA", "create_at": date("2015-11-23"), "description": "Wife,Mom,Grandma & business owner/Pest Control. Simple things make me happy! Animal lover, Metal detecting,Casino lover", "followers_count": 1007, "friends_count": 1498, "statues_count": 16854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channahon, IL", "id": "9ae1ad28e07154d9", "name": "Channahon", "place_type": "city", "bounding_box": rectangle("-88.25119,41.377462 -88.13539,41.492478") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1712476, "cityName": "Channahon" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124131811329, "text": "GOOOOOODDDDMORNINGGG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1487246917, "name": "Emilyy", "screen_name": "uczekaj", "lang": "en", "location": "null", "create_at": date("2013-06-06"), "description": "ASU '20 | B Michelena❣", "followers_count": 649, "friends_count": 503, "statues_count": 4673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124169560068, "text": "@conspirator thank you man that means a lot really glad you watched/shared", "in_reply_to_status": 737538019813515268, "in_reply_to_user": 7460682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 7460682 }}, "user": { "id": 22708585, "name": "Dann Petty", "screen_name": "DannPetty", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-03-03"), "description": "Pro Surfer (Web), Vlogger https://t.co/jvcL8mKYkO, Host @epicurrence, https://t.co/Moo1eEUs8E + https://t.co/e0si7cB9zz, CD @uenodotco, Past: @LuxeValet @Medium", "followers_count": 11082, "friends_count": 2214, "statues_count": 19334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124236673024, "text": "Fact 3: This is what happens when you listen to Wedding Dress too many times. I deadass wanted to be a Kpop star https://t.co/K6Brfa7Ucr", "in_reply_to_status": 737536434114985984, "in_reply_to_user": 2367499642, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2367499642, "name": "milktea papi", "screen_name": "NguyenNewton", "lang": "en", "location": "eating or napping", "create_at": date("2014-02-26"), "description": "keep it wavy, baby~ snap!: newtonnguyen", "followers_count": 554, "friends_count": 383, "statues_count": 6249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124337344512, "text": "@destyniebaby they do sometimes ��", "in_reply_to_status": 737533124301623296, "in_reply_to_user": 4709995152, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4709995152 }}, "user": { "id": 3705352033, "name": "Kyre ⁹", "screen_name": "KyreWest", "lang": "en", "location": "Compton, CA", "create_at": date("2015-09-27"), "description": "Just a young nigga from the Westside tryna get by", "followers_count": 250, "friends_count": 288, "statues_count": 3688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124530319361, "text": "Wrap it up or diaper it up☺", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 176683494, "name": "le❤️", "screen_name": "xaliyahh_", "lang": "en", "location": "El Cajon, CA", "create_at": date("2010-08-09"), "description": "SC: xaliyahh18", "followers_count": 1216, "friends_count": 855, "statues_count": 55868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Cajon, CA", "id": "b82fa51f6957a1eb", "name": "El Cajon", "place_type": "city", "bounding_box": rectangle("-117.009833,32.767506 -116.894872,32.831001") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 621712, "cityName": "El Cajon" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539124916158470, "text": "#19", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2905657140, "name": "mikeyyy", "screen_name": "Aros_22_", "lang": "en", "location": "LA", "create_at": date("2014-11-20"), "description": "SC: Aros_20", "followers_count": 571, "friends_count": 532, "statues_count": 24561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:04.000Z"), "id": 737539125381763073, "text": "Happy Birthday to one of my bffs manderssss ��✨�� Ilysm and keep doing you because you're gr8 ������ ! https://t.co/CmPciv1FMn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3450188892, "name": "Sab", "screen_name": "SabrinaAgariee", "lang": "en", "location": "null", "create_at": date("2015-09-04"), "description": "null", "followers_count": 419, "friends_count": 353, "statues_count": 5490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temple City, CA", "id": "206e932cc0e89fa2", "name": "Temple City", "place_type": "city", "bounding_box": rectangle("-118.081392,34.085395 -118.028425,34.120984") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 678148, "cityName": "Temple City" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539125574701056, "text": "I also remember this https://t.co/nM3IZLj0mH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2267283517, "name": "moonlight van ✨", "screen_name": "CUMBACKTOME", "lang": "en", "location": "null", "create_at": date("2013-12-29"), "description": "@fucktyler: YALL KNOW I LOVE TALL WHITE DUDES I ALWAYS RAP ABOUT IT", "followers_count": 2030, "friends_count": 637, "statues_count": 58679 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539126421905408, "text": "Shoutout to @cig_miller @Garwood_77 @rrogerthat420 for achieving a life goal. I'm Jealous lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 328538766, 799578530, 927063151 }}, "user": { "id": 273582128, "name": "Connor", "screen_name": "ImpulseCdog", "lang": "en", "location": "Dis Side", "create_at": date("2011-03-28"), "description": "| Third Eye Midwest | Producer | http://www.soundcloud.com/ImpulseCdog | Yeezy Militia | 3ayeM | I'm like DMX in a jean suit", "followers_count": 158, "friends_count": 363, "statues_count": 4601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensburg, IN", "id": "9f29ee1a2f040588", "name": "Greensburg", "place_type": "city", "bounding_box": rectangle("-85.518421,39.312107 -85.459117,39.362492") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18031, "countyName": "Decatur", "cityID": 1829718, "cityName": "Greensburg" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539126967164932, "text": "@LunnaMariaa https://t.co/kvTVfJ7JBV", "in_reply_to_status": -1, "in_reply_to_user": 2268570913, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2268570913 }}, "user": { "id": 1664428596, "name": "Andrew Rodriguez", "screen_name": "drew__2123", "lang": "en", "location": "Oceanside ", "create_at": date("2013-08-12"), "description": "Photographer. Sports Fan. Automotive Enthusiast. Mira Costa Student. Residing in Oceanside/San Diego.", "followers_count": 188, "friends_count": 196, "statues_count": 9144 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539127009120256, "text": "growing up an alternative black kid is hard. makes it harder when other black people say you \"act white\". i still get this flack.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 966952165, "name": "Salaam", "screen_name": "vijooul", "lang": "en", "location": "Milwaukee, WI", "create_at": date("2012-11-23"), "description": "bottom grill with a poet's notebook. teaching artist. gender neutral dom. the auspicious one. they/she. CONGLOMERATE. CSU.", "followers_count": 2440, "friends_count": 2564, "statues_count": 93608 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539127545991170, "text": "12 am watching Fifth Harmony videos yesssss", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921321190, "name": "jace", "screen_name": "_jacelynnmarie", "lang": "en", "location": "bay area", "create_at": date("2014-12-06"), "description": "you dont know me", "followers_count": 192, "friends_count": 90, "statues_count": 11756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539127659237376, "text": "I can't tell if I'm aloof and detached or incredibly expressive and attached bc that's pretty much entirely based on how others perceive me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2476345046, "name": "loveable misfit", "screen_name": "gladiolusfae", "lang": "en", "location": "null", "create_at": date("2014-05-03"), "description": "finally writing an about. Jay, they/them. I love Subway sandwiches and my cats", "followers_count": 37, "friends_count": 153, "statues_count": 1348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539128036724736, "text": "Can I just get a fucking phone call", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2235705991, "name": "Silhouette ❥", "screen_name": "silo_mariee", "lang": "en", "location": "null", "create_at": date("2013-12-07"), "description": "kaya ❥", "followers_count": 625, "friends_count": 582, "statues_count": 10431 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539128254861312, "text": "#BornWithHorns #Rapture #HolyHighHorses #EveryRoseHasItsThorn Told the World how he felt w/ the sound of a kat. https://t.co/blFIvPbRdU @POD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BornWithHorns", "Rapture", "HolyHighHorses", "EveryRoseHasItsThorn" }}, "user_mentions": {{ 163743554 }}, "user": { "id": 710260629777588224, "name": "Paul Mason Ranck", "screen_name": "paulmasonranck", "lang": "en", "location": "Westerville, OH", "create_at": date("2016-03-16"), "description": "Why did Scre4m and Lemonade Mouth come out on the same day? Why do I get assaulted and thrown back in the psyche ward? #TheBattleScarredConquistador", "followers_count": 296, "friends_count": 666, "statues_count": 316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westerville, OH", "id": "00ebe84c07a75e81", "name": "Westerville", "place_type": "city", "bounding_box": rectangle("-82.971635,40.080431 -82.857441,40.217545") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3983342, "cityName": "Westerville" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539128422596608, "text": "@HYPEBEAST Damn ��", "in_reply_to_status": 737529725938106368, "in_reply_to_user": 18286505, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 18286505 }}, "user": { "id": 1177549718, "name": "h e n ø n", "screen_name": "LKvivrai", "lang": "en", "location": "eastern washington university ", "create_at": date("2013-02-13"), "description": "•vai è vivrai•", "followers_count": 1815, "friends_count": 1734, "statues_count": 4983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539128850452481, "text": "IM STRESSIN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3092430757, "name": "lei", "screen_name": "ohleilani", "lang": "en", "location": "california", "create_at": date("2015-03-16"), "description": "I look like a bitch but I swear I'm a super bitch", "followers_count": 265, "friends_count": 96, "statues_count": 14421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129244688384, "text": "5/31/2016 - 02:00\nTemp: 61.3F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.916in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 53, "statues_count": 53338 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129257267200, "text": "@FrancineAlarcn_ thank u bb! Imysm ������", "in_reply_to_status": 737538994171301888, "in_reply_to_user": 2266659092, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2266659092 }}, "user": { "id": 288047059, "name": "angela♡marie", "screen_name": "angelacuencaa_", "lang": "en", "location": "Island of Misfit Toys", "create_at": date("2011-04-25"), "description": "❣ A's before baes ❣", "followers_count": 345, "friends_count": 384, "statues_count": 20459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129324408834, "text": "@_josecudi https://t.co/PkOpgAXnjv", "in_reply_to_status": 737537492128129024, "in_reply_to_user": 2322760878, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2322760878 }}, "user": { "id": 2322760878, "name": "Scott", "screen_name": "_josecudi", "lang": "en-gb", "location": "Russia", "create_at": date("2014-02-01"), "description": "Lmhs", "followers_count": 960, "friends_count": 897, "statues_count": 22996 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntington Park, CA", "id": "4d1d90faa5484b1c", "name": "Huntington Park", "place_type": "city", "bounding_box": rectangle("-118.239035,33.961583 -118.189054,33.996268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636056, "cityName": "Huntington Park" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129617960960, "text": "��(10) Food ��(10) Service ��(9) Environment\n➖➖➖➖➖➖➖➖➖➖➖➖➖➖➖➖➖\nCronut- ����2.50\n| �� Tag Your Friends… https://t.co/FQuGRiECQX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.6386538,34.1573819"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1396660314, "name": "laeater", "screen_name": "laeater", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-05-02"), "description": "$20 Cap |(Food) + (Service) +(Environment) |Los Angeles | Categories Rated 1⃣-1️⃣0️⃣ | All Food Posts are 7⃣⬆ | Comment & tag your friends | #laeater", "followers_count": 179, "friends_count": 577, "statues_count": 514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calabasas, CA", "id": "575c6d4e1b73546d", "name": "Calabasas", "place_type": "city", "bounding_box": rectangle("-118.719985,34.105958 -118.605227,34.168562") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 609598, "cityName": "Calabasas" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129659904000, "text": "meant to b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4893085689, "name": "Fab Melo", "screen_name": "sahiraxo", "lang": "en", "location": "Dominican Republic", "create_at": date("2016-02-09"), "description": "Michigan State || Civil Engineering || Positive vibes only, xo. || #Sahira2016", "followers_count": 189, "friends_count": 162, "statues_count": 1573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-05-31T00:00:05.000Z"), "id": 737539129676701696, "text": "Temp: 61.0°F - Dew Point: 57.5° - Wind: --- @ 0.0 mph - Gust: 0.0 - Rain Today: 0.00in. - Pressure: 29.98in, - Trend: Steady", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 13, "statues_count": 18459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-05-31T00:00:06.000Z"), "id": 737539129890635777, "text": "Yup https://t.co/3eDmvetg6D", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3298900548, "name": "quiiiitttt it", "screen_name": "itsalaina_", "lang": "en", "location": "sc: alaina_999", "create_at": date("2015-07-27"), "description": "cguhs #13", "followers_count": 317, "friends_count": 268, "statues_count": 5483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casa Grande, AZ", "id": "fbb3d1e41acab043", "name": "Casa Grande", "place_type": "city", "bounding_box": rectangle("-111.791608,32.858246 -111.670779,32.992892") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 410530, "cityName": "Casa Grande" } }
+{ "create_at": datetime("2016-05-31T00:00:06.000Z"), "id": 737539130058412033, "text": "https://t.co/5LUUGeKp3j", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 127157317, "name": "Jazzmin", "screen_name": "jzzmncttrll", "lang": "en", "location": "VENICE, CA", "create_at": date("2010-03-28"), "description": "send nudes", "followers_count": 353, "friends_count": 246, "statues_count": 25751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-05-31T00:00:06.000Z"), "id": 737539130284875779, "text": "Thinking of you every day❤️ @ Ala Moana Beach Lantern Festival https://t.co/SQ1mdzlQZD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.84732282,21.28900767"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 52559882, "name": "Kayla", "screen_name": "kayrra", "lang": "en", "location": "Honoruru, Kawaii", "create_at": date("2009-06-30"), "description": "the Present is a gift, & I just wanna BE.", "followers_count": 334, "friends_count": 221, "statues_count": 14651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-05-31T00:00:06.000Z"), "id": 737539130389712897, "text": "@damnitsdanna the one that stepped up next to to see who he was texting ����", "in_reply_to_status": 737539007848976384, "in_reply_to_user": 321471150, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321471150 }}, "user": { "id": 1629563678, "name": "Dennise Sosa E", "screen_name": "_denniiiise", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "null", "followers_count": 234, "friends_count": 163, "statues_count": 9103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-05-31T00:00:06.000Z"), "id": 737539130607865857, "text": "T'was a long day.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3180025502, "name": "NICK ESPINOZA", "screen_name": "nickthefamous", "lang": "en", "location": "Marisa DiMaria", "create_at": date("2015-04-29"), "description": "Monterey, Ca, God first, MPC. it is what it is .", "followers_count": 792, "friends_count": 787, "statues_count": 7841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901492447907840, "text": "@stephhollman boys are a whole different species! If I survive raising my 2 it will be a miracle! �� #RHOD", "in_reply_to_status": -1, "in_reply_to_user": 3020142805, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RHOD" }}, "user_mentions": {{ 3020142805 }}, "user": { "id": 122828876, "name": "⚜Jess⚜", "screen_name": "jhspraggins", "lang": "en", "location": "Alabama", "create_at": date("2010-03-13"), "description": "Cool mom of 4 who's obsessed with reality tv, Auburn football and New Orleans, AKA the city where my soul is⚜ Sarcastic & missing a filter Enjoy!", "followers_count": 622, "friends_count": 1601, "statues_count": 24997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Auburn, AL", "id": "deb349182b3f42bb", "name": "Auburn", "place_type": "city", "bounding_box": rectangle("-85.569969,32.532449 -85.413112,32.662041") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1081, "countyName": "Lee", "cityID": 103076, "cityName": "Auburn" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901492959600641, "text": "@RossBoss77 ����", "in_reply_to_status": 737901389519675392, "in_reply_to_user": 581855624, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 581855624 }}, "user": { "id": 4341499632, "name": "Maya Putman", "screen_name": "mayamia07", "lang": "en", "location": "null", "create_at": date("2015-11-23"), "description": "#BlueSteel Instagram: magicmaya_XXL snapchat: maya_mckinsey ♌️", "followers_count": 218, "friends_count": 422, "statues_count": 2964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901492997369861, "text": "������ https://t.co/Ru1KQaYPSq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2771461336, "name": "$ h y", "screen_name": "BhaseGramz", "lang": "en", "location": "GA", "create_at": date("2014-09-15"), "description": "R20llin Thru The NeigHBorh27d.. #RestUpCindy", "followers_count": 1487, "friends_count": 547, "statues_count": 147492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Douglasville, GA", "id": "68482982f9baf37a", "name": "Douglasville", "place_type": "city", "bounding_box": rectangle("-84.794814,33.688539 -84.63351,33.784073") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13097, "countyName": "Douglas", "cityID": 1323900, "cityName": "Douglasville" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493026709505, "text": "RT if you love the GATES FAMILY �������� https://t.co/F9e7g2o8nk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29633909, "name": "Jαy Gee ™ ⚡️", "screen_name": "RealJazzyBaby", "lang": "en", "location": " a t l a n t a", "create_at": date("2009-04-07"), "description": "• twenty five✨ • single ✴️ • #VirgoGang ♻️", "followers_count": 1101, "friends_count": 1982, "statues_count": 19779 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Smyrna, GA", "id": "5b8df26e6d0be60b", "name": "Smyrna", "place_type": "city", "bounding_box": rectangle("-84.561205,33.788139 -84.470367,33.905391") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13067, "countyName": "Cobb", "cityID": 1371492, "cityName": "Smyrna" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493093814272, "text": "Here's a quick low calories™drink guys... https://t.co/AkTPozNLTI", "in_reply_to_status": 737899236503740416, "in_reply_to_user": 4136197230, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4136197230, "name": "AnonYMous", "screen_name": "anonysidromous", "lang": "en", "location": "United States", "create_at": date("2015-11-05"), "description": "My name is Anon Ysidro Mous; CEO of @An0nymousLLC; Business Owner; Transparency Master; and @Lakers Fan. I love World Peace & Animals! Text Me at: 323.320.6846!", "followers_count": 7138, "friends_count": 7231, "statues_count": 5789 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493211308032, "text": "Center Stage #memorialdayweekend @ Dogwood Canyon Nature Park https://t.co/kvx5MKCF5f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-93.46123685,36.53045812"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "memorialdayweekend" }}, "user": { "id": 26975604, "name": "Just June", "screen_name": "junoz64", "lang": "en", "location": "Texas", "create_at": date("2009-03-27"), "description": "null", "followers_count": 56, "friends_count": 131, "statues_count": 2032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri, USA", "id": "2526edd24c06e60c", "name": "Missouri", "place_type": "admin", "bounding_box": rectangle("-95.774704,35.995476 -89.098843,40.613641") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29209, "countyName": "Stone" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493265797120, "text": "@xoaIexissssss I'm going to finish it up rn! I thought you were sleeping mhm", "in_reply_to_status": 737901370288803840, "in_reply_to_user": 168517836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 168517836 }}, "user": { "id": 45243288, "name": "steven", "screen_name": "wankism", "lang": "en", "location": "California, USA", "create_at": date("2009-06-06"), "description": "dad", "followers_count": 775, "friends_count": 388, "statues_count": 19412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Jose Hills, CA", "id": "945d4c47ec5036f9", "name": "South San Jose Hills", "place_type": "city", "bounding_box": rectangle("-117.923259,34.003576 -117.88006,34.026435") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673290, "cityName": "South San Jose Hills" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493764935681, "text": "Just Goofy ���� https://t.co/EV8KkMILy7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 456792628, "name": "Jay", "screen_name": "_ShekieBaby", "lang": "en", "location": "Somewhere Getting High ", "create_at": date("2012-01-06"), "description": "SC: #Shekie.Jay ! Don't Follow To Unfollow But Be My Guest To Snoop Around", "followers_count": 1057, "friends_count": 543, "statues_count": 70696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stevensville, MI", "id": "d977006ff227d94a", "name": "Stevensville", "place_type": "city", "bounding_box": rectangle("-86.539968,41.985455 -86.465506,42.050807") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26021, "countyName": "Berrien", "cityID": 2676500, "cityName": "Stevensville" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493920137216, "text": "LMAOOOOOO https://t.co/Xah4OcWXoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2385915590, "name": "Amessss ❣", "screen_name": "AmyyHernandez69", "lang": "en", "location": "htx", "create_at": date("2014-03-12"), "description": "fuck off", "followers_count": 803, "friends_count": 960, "statues_count": 19803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901493978845184, "text": "@ChelleyBellyy 1 more month!!!! https://t.co/9XDmrP8MlH", "in_reply_to_status": -1, "in_reply_to_user": 280340929, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 280340929 }}, "user": { "id": 209975287, "name": "Nick Alvarado", "screen_name": "Nick_Alvarado", "lang": "en", "location": "null", "create_at": date("2010-10-30"), "description": "null", "followers_count": 152, "friends_count": 351, "statues_count": 3120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901494033354753, "text": "Let's just say she was REALLY excited to graduate ���� @em_beyde https://t.co/ncpGRzPD2f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262491674 }}, "user": { "id": 710210778201255936, "name": "Isabel", "screen_name": "isabellovee23", "lang": "en", "location": "null", "create_at": date("2016-03-16"), "description": "always going with the flow", "followers_count": 74, "friends_count": 85, "statues_count": 413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hemet, CA", "id": "0807c31007ea6b4b", "name": "Hemet", "place_type": "city", "bounding_box": rectangle("-117.071232,33.685766 -116.908609,33.772991") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 633182, "cityName": "Hemet" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901494092070912, "text": "https://t.co/I3Flose3kd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1641749574, "name": "raquelin", "screen_name": "bbboyluh", "lang": "en", "location": "tucson, az ", "create_at": date("2013-08-02"), "description": "I AM PLENTI • 10.25.15 - 03.05.16", "followers_count": 2250, "friends_count": 834, "statues_count": 67409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901494469558272, "text": "RIP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 449181607, "name": "3", "screen_name": "churr097", "lang": "en", "location": "Charter Oak, CA", "create_at": date("2011-12-28"), "description": "Spinnin' all your wildest dreams. 5-18", "followers_count": 477, "friends_count": 418, "statues_count": 11514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charter Oak, CA", "id": "07432bdb1901f862", "name": "Charter Oak", "place_type": "city", "bounding_box": rectangle("-117.877463,34.093451 -117.837641,34.106755") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 612734, "cityName": "Charter Oak" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901495476211712, "text": "@_RonTreyCinco cmon man ! ��������", "in_reply_to_status": 737901334284926977, "in_reply_to_user": 3032860419, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3032860419 }}, "user": { "id": 2178429275, "name": "✨Swish✨", "screen_name": "Rico_ThaShooter", "lang": "en", "location": "United States", "create_at": date("2013-11-12"), "description": "I can do all things through Christ who strengthens me ... RiP GParents !", "followers_count": 705, "friends_count": 595, "statues_count": 25215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901495526543360, "text": "This cereal is bomb doe ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43257714, "name": "Fatima", "screen_name": "peruana1025", "lang": "en", "location": "sacramento,CA", "create_at": date("2009-05-28"), "description": "UC Merced Alumna! 21! Peruvian! EDM ❤️", "followers_count": 364, "friends_count": 306, "statues_count": 17222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901495606206464, "text": "@Patrick_Star95 I win https://t.co/EATEXahD7z", "in_reply_to_status": 737897377252659200, "in_reply_to_user": 237534466, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 237534466 }}, "user": { "id": 911415619, "name": "King K", "screen_name": "KylieKing23", "lang": "en", "location": "Oregon ", "create_at": date("2012-10-28"), "description": "I'm probably sleeping right now", "followers_count": 1850, "friends_count": 767, "statues_count": 46768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermiston, OR", "id": "3b213491c5ae3f7e", "name": "Hermiston", "place_type": "city", "bounding_box": rectangle("-119.32694,45.824334 -119.258928,45.864803") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41059, "countyName": "Umatilla", "cityID": 4133700, "cityName": "Hermiston" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901495673356288, "text": "☕️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1388129869, "name": "sandra", "screen_name": "BelieberVatos", "lang": "en", "location": "null", "create_at": date("2013-04-28"), "description": "kiss it better", "followers_count": 385, "friends_count": 325, "statues_count": 9264 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901496243736576, "text": "Reading nook complete. #latenightcreativity #apartmentliving #bookstagram #craftybitch @… https://t.co/66Fjjz9r4V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-105.103,39.7063"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "latenightcreativity", "apartmentliving", "bookstagram", "craftybitch" }}, "user": { "id": 3241622588, "name": "Kristin Fusilier", "screen_name": "mutantmango", "lang": "en", "location": "null", "create_at": date("2015-06-10"), "description": "null", "followers_count": 5, "friends_count": 52, "statues_count": 4 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CO", "id": "f7eb2fa2fea288b1", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-105.193475,39.60973 -105.053164,39.761974") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 843000, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901496428302337, "text": "Lieve in rself �� https://t.co/FWbUbMlh5J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249995485, "name": "Olivia Frescura", "screen_name": "OliviaFrescura", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-02-09"), "description": "Los Angeles beauty blogger. Lover of lipstick, cappuccino, and the color orange. Social Media @purlisse✨ olivia@oliviafrescura.com", "followers_count": 4443, "friends_count": 806, "statues_count": 18551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901496461889537, "text": "So many good movies coming out this summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33333558, "name": "Baddie B", "screen_name": "_britt2trilll", "lang": "en", "location": "null", "create_at": date("2009-04-19"), "description": "HOU. 23. UH.", "followers_count": 642, "friends_count": 499, "statues_count": 43471 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channelview, TX", "id": "eabe7b6fd4504fff", "name": "Channelview", "place_type": "city", "bounding_box": rectangle("-95.176002,29.735548 -95.061098,29.851809") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4814236, "cityName": "Channelview" } }
+{ "create_at": datetime("2016-06-01T00:00:00.000Z"), "id": 737901496554119168, "text": "You lonely at home thinkin mad shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3279221294, "name": "Deleon", "screen_name": "vlone300", "lang": "en", "location": "null", "create_at": date("2015-07-13"), "description": "young handsome jiggy nigga", "followers_count": 1090, "friends_count": 585, "statues_count": 23531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901496654823424, "text": "Gucci gettin swole was like Goku becomin a super sayain to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551569737, "name": "❌", "screen_name": "_AceFlaco", "lang": "en", "location": "Dallas, TX", "create_at": date("2012-04-11"), "description": "just a wavy ass nigga #LLTK", "followers_count": 1311, "friends_count": 610, "statues_count": 77775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497195896832, "text": "@alecrandolph_ not forgiven", "in_reply_to_status": 737861926147985408, "in_reply_to_user": 479034128, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 479034128 }}, "user": { "id": 2939214953, "name": "faye", "screen_name": "fayejohnson25", "lang": "en", "location": "null", "create_at": date("2014-12-23"), "description": "null", "followers_count": 366, "friends_count": 344, "statues_count": 2212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497447505920, "text": "having freshly shaved legs and then getting goosebumps ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 362364102, "name": "Marykate Hackbarth", "screen_name": "Marrrykatee", "lang": "en", "location": "860", "create_at": date("2011-08-26"), "description": "•karma is a bitch just make sure that bitch is beautiful•", "followers_count": 310, "friends_count": 213, "statues_count": 15666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clinton, CT", "id": "00ec30114696324c", "name": "Clinton", "place_type": "city", "bounding_box": rectangle("-72.58577,41.257075 -72.486432,41.347669") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9007, "countyName": "Middlesex", "cityID": 915420, "cityName": "Clinton" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497451696128, "text": "@kendisgibson You're a joy to watch when not relaying horrific news, Kendis. @bensherwood", "in_reply_to_status": -1, "in_reply_to_user": 169156588, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 169156588, 13397202 }}, "user": { "id": 1724303582, "name": "Russ DeCastro", "screen_name": "RussDeCastro221", "lang": "en", "location": "New York City, NY, USA!", "create_at": date("2013-09-02"), "description": "The University of Michigan at Ann Arbor; Harvard University", "followers_count": 5, "friends_count": 33, "statues_count": 8713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497455939584, "text": "@ebbtideapp Tide in Summit Bridge, Delaware 06/01/2016\nHigh 7:48am 3.8\n Low 2:14pm 0.1\nHigh 8:22pm 4.1\n Low 2:57am 0.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.7333,39.5333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 98, "friends_count": 1, "statues_count": 37631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497594314753, "text": "Temp 54.6° Hi/Lo 59.1/54.6 Rng 4.5° WC 54.6° Hmd 91% Rain 0.00\" Storm 0.00\" BAR 30.033 Rising DP 52.0° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 72, "friends_count": 123, "statues_count": 19149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497690775553, "text": "my life https://t.co/blqhttMKj9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1837495807, "name": "chey.", "screen_name": "CheyTooRad", "lang": "en", "location": "the L. ", "create_at": date("2013-09-08"), "description": "sorry, I have pistanthrophobia // RIP Malik", "followers_count": 639, "friends_count": 588, "statues_count": 18837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manteca, CA", "id": "7cfaf4a257bca60a", "name": "Manteca", "place_type": "city", "bounding_box": rectangle("-121.300453,37.768376 -121.175722,37.864531") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 645484, "cityName": "Manteca" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497707548677, "text": "06/01@03:00 - Temp 64.5F, WC 64.5F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.004in, Rising slowly. Rain 0.00in. Hum 96%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497841790976, "text": "Work was so gay tonight fuck man I need motivation", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1942315644, "name": "Rolando", "screen_name": "911Roruiz", "lang": "en", "location": "null", "create_at": date("2013-10-06"), "description": "Gen is life #boolin'", "followers_count": 245, "friends_count": 75, "statues_count": 14349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Round Lake Beach, IL", "id": "6a2d097551205962", "name": "Round Lake Beach", "place_type": "city", "bounding_box": rectangle("-88.120296,42.364202 -88.043395,42.397972") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1766040, "cityName": "Round Lake Beach" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901497871175681, "text": "Meet Amanda Vignes https://t.co/CBlhzT6GUG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.08709717,29.51160049"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2866011003, "name": "League City, TX News", "screen_name": "BLifeLeagueCity", "lang": "en", "location": "League City, Texas", "create_at": date("2014-11-07"), "description": "League City BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/Ew7CTPM7Zf.", "followers_count": 162, "friends_count": 122, "statues_count": 9718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "League City, TX", "id": "cf3b45e29c2c319b", "name": "League City", "place_type": "city", "bounding_box": rectangle("-95.178987,29.438994 -94.995071,29.55532") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841980, "cityName": "League City" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498089283585, "text": "*boyfreinds phone buzzes while on FaceTime* \n\" there goes ya hoes texting you\"\n\" THAT WAS YOU MCKENZIE\" ������ forgot I texted him a picture", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3063588550, "name": "aalyiah", "screen_name": "wranovics", "lang": "en", "location": "Sidney, NY", "create_at": date("2015-02-25"), "description": "stripper from New Jersey.", "followers_count": 330, "friends_count": 269, "statues_count": 3330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sidney, NY", "id": "ebf5cb095574e046", "name": "Sidney", "place_type": "city", "bounding_box": rectangle("-75.41566,42.29375 -75.372971,42.319924") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36025, "countyName": "Delaware", "cityID": 3667334, "cityName": "Sidney" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498399625218, "text": "Wind 0.0 mph SE. Barometer 29.887 in, Steady. Temperature 73.3 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498496102400, "text": "Wind 3.8 mph NE. Barometer 29.77 in, Falling slowly. Temperature 42.4 °F. Rain today 0.00 in. Humidity 33%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 19, "friends_count": 5, "statues_count": 42618 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498508709888, "text": "@bigpimpinmychal @kinghaileee I'm the dog BC I like Tucson", "in_reply_to_status": 737901436659453952, "in_reply_to_user": 231322628, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 231322628, 2565254478 }}, "user": { "id": 3104009803, "name": "arielle", "screen_name": "TheGoddess_AZ", "lang": "en-GB", "location": "beauty insta: arielle.beauty25", "create_at": date("2015-03-22"), "description": "• transgendered Latina •", "followers_count": 948, "friends_count": 479, "statues_count": 42436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498701602817, "text": "#WishLisaLemonHappyBirthdayIn5Words Beanie Babies? Nope! Beanie Babe!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pl", "is_retweet": false, "hashtags": {{ "WishLisaLemonHappyBirthdayIn5Words" }}, "user": { "id": 131198173, "name": "Chris C", "screen_name": "MiamiSixthMan", "lang": "en", "location": "Florida", "create_at": date("2010-04-09"), "description": "Always developing multi-talented entertainer. Come along for the ride! (Check Out My Podcast) See link & ITunes!! CGSHere@gmail.com follow my podcast @CGSHere", "followers_count": 2763, "friends_count": 3336, "statues_count": 24643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498701611008, "text": "We barely graduated Friday lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4228440913, "name": "Nicolas Merz", "screen_name": "NicThe2MinQuick", "lang": "en", "location": "Loserville USA", "create_at": date("2015-11-13"), "description": "Fag from LHS who's NOT in jail. Play smash and I'm good sometimes. I am a golden god, I can run a lap in 2 minutes. Subscribe to my YouTube @ NicThe2MinQuick", "followers_count": 138, "friends_count": 272, "statues_count": 1895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901498856800256, "text": "@l0lkarla @alyssaxnuno https://t.co/H11JCeC6br", "in_reply_to_status": 737901173324288000, "in_reply_to_user": 3023216328, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3023216328, 3079402669 }}, "user": { "id": 2149213528, "name": "∆", "screen_name": "Octaaaavio_V6", "lang": "en", "location": "6ix ", "create_at": date("2013-10-22"), "description": "Uber everywhere", "followers_count": 491, "friends_count": 300, "statues_count": 20370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901499418857474, "text": "who is dis new pharmacy guy in westwood �� lordt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1564765646, "name": "emz", "screen_name": "emweezzyy", "lang": "en", "location": "instagram @emweezzyy ", "create_at": date("2013-07-02"), "description": "long beach • ucla", "followers_count": 517, "friends_count": 331, "statues_count": 11265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901499599183873, "text": "@BbChelaas haha u can't even deny it", "in_reply_to_status": 737901077295681536, "in_reply_to_user": 323988179, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323988179 }}, "user": { "id": 489176849, "name": "Bao Tran", "screen_name": "bjqtran", "lang": "en", "location": "null", "create_at": date("2012-02-11"), "description": "Do you like tapes and CD's?", "followers_count": 344, "friends_count": 234, "statues_count": 4882 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901499636953088, "text": "@_sxmone lmao symone dont start with me", "in_reply_to_status": 737901122992668677, "in_reply_to_user": 260904659, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 260904659 }}, "user": { "id": 1415295876, "name": "hi, im father.", "screen_name": "BigDawgChoosin", "lang": "en", "location": "Kent State '20.", "create_at": date("2013-05-09"), "description": "I usually don't mean much to people, lol ❣", "followers_count": 6021, "friends_count": 5649, "statues_count": 13397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avon Lake, OH", "id": "c00b424a664dde4b", "name": "Avon Lake", "place_type": "city", "bounding_box": rectangle("-82.066856,41.478018 -81.968332,41.515677") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3903464, "cityName": "Avon Lake" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901499754418176, "text": "�� Joe >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 63598634, "name": "KING JOE", "screen_name": "OGPLAY_", "lang": "en", "location": "Dallas, TX 60619 ", "create_at": date("2009-08-06"), "description": "I can only be me #OG Chicago nigga Nawf Dallas living #2Homes1Nigga", "followers_count": 1753, "friends_count": 1583, "statues_count": 187088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901499775344641, "text": "Who on Twitter would be willing to give me a wake up call around 4-5 am. Please pleasd@", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43255130, "name": "Sailor Moon ✨", "screen_name": "thidatotter", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-28"), "description": "Falling down the rabbit hole.", "followers_count": 541, "friends_count": 220, "statues_count": 55653 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901500664586240, "text": "I done got Pennie started ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290311945, "name": "Thats Her✨", "screen_name": "Joohnaishha_", "lang": "en", "location": "looking up to this 6'5 dude...", "create_at": date("2011-04-29"), "description": "She is me & I am her ❣/ #15 ✈️ @youlovebreezy__", "followers_count": 1252, "friends_count": 852, "statues_count": 44898 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-06-01T00:00:01.000Z"), "id": 737901500761034753, "text": "w/ @CaaitlinNoel ���� https://t.co/OBGLNPsTZK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 261506865 }}, "user": { "id": 744724429, "name": "Yayla❤️", "screen_name": "KaylaBekakis1", "lang": "en", "location": "null", "create_at": date("2012-08-08"), "description": "ironic you been sleeping on the one that you been dreamin bout✨ Snapchat me @kaybekakis •SENIOR '16•", "followers_count": 715, "friends_count": 662, "statues_count": 22373 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Concord, CA", "id": "8004d2bebcc13e8c", "name": "Concord", "place_type": "city", "bounding_box": rectangle("-122.066245,37.924231 -121.931044,38.022423") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616000, "cityName": "Concord" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901500974960640, "text": "@SelenaCravalho really just asked who OKC was...", "in_reply_to_status": -1, "in_reply_to_user": 2283064928, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2283064928 }}, "user": { "id": 2717998651, "name": "Kylee Leppanen", "screen_name": "kyleealexus", "lang": "en", "location": "kaileys house", "create_at": date("2014-08-08"), "description": "pghs || Silverstreak⚾️", "followers_count": 563, "friends_count": 470, "statues_count": 8752 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vineyard, CA", "id": "01f9f8b01415323b", "name": "Vineyard", "place_type": "city", "bounding_box": rectangle("-121.371979,38.452572 -121.292468,38.485308") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 682852, "cityName": "Vineyard" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901501012692992, "text": "@jordanknight @SchittsCreekPop @mocha72cook hahahah I feel you ������Can't wait for tonight��������⚓️", "in_reply_to_status": 737790903872540672, "in_reply_to_user": 31001575, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31001575, 2855819371, 3215745937 }}, "user": { "id": 194139689, "name": "Ana❤️JK❤️NKOTB", "screen_name": "Anakamilladream", "lang": "en", "location": "Philadelphia, PA", "create_at": date("2010-09-23"), "description": "Super NKOTB Fan4life,Jordan's girl 25 years and counting! #cantwaitformyfirstpicwithhim #BHlover This page was made to be connect with many #bhsis as possible", "followers_count": 1438, "friends_count": 2079, "statues_count": 6014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwood, PA", "id": "8b416dd99e78dbd7", "name": "Norwood", "place_type": "city", "bounding_box": rectangle("-75.3065,39.875311 -75.286328,39.898089") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42045, "countyName": "Delaware", "cityID": 4255664, "cityName": "Norwood" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901501063024641, "text": "I'm 19 y'all. Alright time to party ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 510539586, "name": "Nick_Da_Cheese", "screen_name": "dicicco_nick", "lang": "en", "location": "Brentwood, Ca", "create_at": date("2012-03-01"), "description": "Nick DiCicco. Employed . catch me at . Los medanos College. catcher life. 49ers, A's. single life. A's (20-29)", "followers_count": 237, "friends_count": 521, "statues_count": 4439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901501566353408, "text": "@KGEra_ ����������", "in_reply_to_status": 737901428325376000, "in_reply_to_user": 961146072, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3259142013 }}, "user": { "id": 961146072, "name": "ale", "screen_name": "heyyy_ale", "lang": "en", "location": "rgv 2 dtx", "create_at": date("2012-11-20"), "description": "18 | college student", "followers_count": 738, "friends_count": 499, "statues_count": 14054 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502120009728, "text": "Goodnight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 592987750, "name": "Vitamin P", "screen_name": "Peyton_Miller1", "lang": "en", "location": "null", "create_at": date("2012-05-28"), "description": "Lakewood // @uhohk", "followers_count": 999, "friends_count": 644, "statues_count": 23369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, WA", "id": "744fbe9224233893", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-122.199229,48.133613 -122.10191,48.203671") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5302585, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502342303744, "text": "Too soon? ( If so, get over it) https://t.co/vHC8Bb5OTU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1150021068, "name": "Jeffrey Hoffman", "screen_name": "JeffreyLeopard", "lang": "en", "location": "Arizona, USA", "create_at": date("2013-02-04"), "description": "Monkey-see/monkey-do! Love sports, too! Bring on the bulls!! #PBR (Love to travel to San Diego, CA)(#WHODAT) Panda's, Polar Bears, Orcas, Big Cats, Ocean Life", "followers_count": 785, "friends_count": 5001, "statues_count": 4502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coronado, CA", "id": "0c2e19c2631eb155", "name": "Coronado", "place_type": "city", "bounding_box": rectangle("-117.227156,32.585524 -117.117931,32.71468") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 616378, "cityName": "Coronado" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502350643200, "text": "Wind 0.0 mph ---. Barometer 29.886 in, Steady. Temperature 69.5 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502715564034, "text": "Wind 0.0 mph ---. Barometer 29.924 in, Falling slowly. Temperature 62.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502761701376, "text": "Still up 4 what.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 290216213, "name": "b.", "screen_name": "_Bennysleaze", "lang": "en", "location": "va", "create_at": date("2011-04-29"), "description": "DMS CLOSED", "followers_count": 6980, "friends_count": 3979, "statues_count": 45466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake, VA", "id": "73d1c1c11b675932", "name": "Chesapeake", "place_type": "city", "bounding_box": rectangle("-76.448383,36.66 -76.143408,36.866431") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51550, "countyName": "Chesapeake", "cityID": 5116000, "cityName": "Chesapeake" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502782709762, "text": "#wtf #myhighschoolbitches #TUHSHeffers hahahahhahaha @ Waikiki Beach Honolulu https://t.co/NWaSj1sy2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.83738406,21.28379661"), "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "wtf", "myhighschoolbitches", "TUHSHeffers" }}, "user": { "id": 129682113, "name": "Courtney Kay", "screen_name": "theR3ALking", "lang": "en", "location": "The 661 SW", "create_at": date("2010-04-04"), "description": "Lil Bit, just a couple ya'llz favorites,\rFuck geekin, fuck bitches,\rGET MONEY, smash snitches.\rSo gone, lesbehonest,\rAlready won cuz yo bitch tha finestttt.", "followers_count": 243, "friends_count": 623, "statues_count": 2605 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1571550, "cityName": "Urban Honolulu" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901502900146176, "text": "@Poo2D2 i am going to follow you", "in_reply_to_status": -1, "in_reply_to_user": 727204731949604864, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 727204731949604864 }}, "user": { "id": 3126822454, "name": "sponcer", "screen_name": "monstersponcer", "lang": "en", "location": "my lawn", "create_at": date("2015-03-29"), "description": "i just wanna let yall know the world is about to feel sponcer from ghetto to ghetto", "followers_count": 21, "friends_count": 12, "statues_count": 895 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashburn, VA", "id": "000b619e3fa2390b", "name": "Ashburn", "place_type": "city", "bounding_box": rectangle("-77.522803,38.98827 -77.431408,39.074049") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51107, "countyName": "Loudoun", "cityID": 5103320, "cityName": "Ashburn" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503088844804, "text": "When I went to tropical my smoothie was rotten! https://t.co/hpziAqaC1O", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2349294805, "name": "Christian Ezarik", "screen_name": "Christianxholl", "lang": "en", "location": "Arizona", "create_at": date("2014-02-17"), "description": "wfhs", "followers_count": 203, "friends_count": 387, "statues_count": 3371 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503202119680, "text": "Temp: 69.4°F Wind:0.0mph Pressure: 29.926hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503277608961, "text": "Wind 0.0 mph ---. Barometer 30.07 in, Rising. Temperature 59.4 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 123, "statues_count": 160209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503399268352, "text": "bitch i live way inna east �� i need a car", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2892439662, "name": "tae", "screen_name": "tahjiia", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "views from the 6 .", "followers_count": 1335, "friends_count": 551, "statues_count": 41620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503571230720, "text": "God blessed me with seeing 16 ���������� & Birthday S/O to my b-day twin @dmorris_11 can't wait to see you playing on the big screen. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2193166599 }}, "user": { "id": 2257376344, "name": "RJ Jurica", "screen_name": "xxTriple_j", "lang": "en", "location": "Caldwell, TX", "create_at": date("2014-01-01"), "description": "skrt skrt skrt", "followers_count": 311, "friends_count": 98, "statues_count": 17 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503655096320, "text": "when I told oomf to buy me a iPhone when he's rich and he busted out saying \"I'll buy you a 300k ring instead\" https://t.co/r2RYZX9eZU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219820342, "name": "zombie mami", "screen_name": "loveechela", "lang": "en", "location": "lit vegas ", "create_at": date("2010-11-25"), "description": "glo'sational", "followers_count": 1529, "friends_count": 996, "statues_count": 53799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503776690178, "text": "@QveenVyRose man just send the beats lmao something you already started working on", "in_reply_to_status": 737898760508968960, "in_reply_to_user": 3301113295, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3301113295 }}, "user": { "id": 302311293, "name": "♬SWIFTFaheva♬", "screen_name": "ZeusXAlmighty", "lang": "en", "location": "SWIFTZERLAND", "create_at": date("2011-05-20"), "description": "☆DeuceUno|Man Of Many Faces|Versatility|Legend Living,Living Legend,SWIFTFAHEVA", "followers_count": 1161, "friends_count": 1101, "statues_count": 50022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901503927726080, "text": "Puerto Ricans clearly need to be put on to some Concon with habichuela", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 312368715, "name": "Saint Alexander", "screen_name": "MJSINCE93", "lang": "en", "location": "Gotham", "create_at": date("2011-06-06"), "description": "I'm broke baby, I aint got no money. #CFC #KTBFFH #MFFL OG #HombreSerio", "followers_count": 589, "friends_count": 381, "statues_count": 98380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901504183586822, "text": "This glorious Nutella kebab is all you need in life https://t.co/ZAfRY5Par2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17661, "friends_count": 17811, "statues_count": 72518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901504418484224, "text": "Clay Township: Temp. 64.8F. (64.8/69.6F) Wind 0 mph --- MaxGust 2mph@00:39. Barometer 29.94 in. Rain today 0.00 in. Humidity 99%. #inwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.14833333,39.93138889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "inwx" }}, "user": { "id": 36190715, "name": "N7CZ", "screen_name": "N7CZ_EM69ww", "lang": "en", "location": "Indianapolis, Indiana", "create_at": date("2009-04-28"), "description": "Terrestrial VHF/UHF Weak Signal & EME Enthusiast", "followers_count": 807, "friends_count": 645, "statues_count": 98030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmel, IN", "id": "eca35cbd75f0a1e6", "name": "Carmel", "place_type": "city", "bounding_box": rectangle("-86.241245,39.926048 -86.023652,40.000451") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1810342, "cityName": "Carmel" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901504527491072, "text": "@helinda36 lol yeah i remember ��", "in_reply_to_status": 737901149240631297, "in_reply_to_user": 735484024139845632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 735484024139845632 }}, "user": { "id": 718675980647202817, "name": "Griselma", "screen_name": "griselma14", "lang": "en", "location": "null", "create_at": date("2016-04-08"), "description": "sc griselmaquiroz8", "followers_count": 46, "friends_count": 95, "statues_count": 45 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corpus Christi, TX", "id": "a3d770a00f15bcb1", "name": "Corpus Christi", "place_type": "city", "bounding_box": rectangle("-97.662618,27.578509 -97.202232,27.895793") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48355, "countyName": "Nueces", "cityID": 4817000, "cityName": "Corpus Christi" } }
+{ "create_at": datetime("2016-06-01T00:00:02.000Z"), "id": 737901504745639936, "text": "Pearls in the sky strung 'round the moon. Pointing to you. So I'll sail till I'm carried to you.… https://t.co/MooAHy53cT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3851921,34.0838747"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 132335869, "name": "ItsNotYouItsMe Blog", "screen_name": "itsnotyouitsme", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-04-12"), "description": "Official Twitter for http://Itsnotyouitsme.org. \n\nArt, Musique, Fashion/Styles and Contemporary Culture by Anthony De La Cruz.", "followers_count": 199, "friends_count": 1229, "statues_count": 3448 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901505253105664, "text": "@ngozixamelia_ love you too, mom", "in_reply_to_status": 737901426731552768, "in_reply_to_user": 2258289632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2258289632 }}, "user": { "id": 727191368, "name": "Kaitlyn", "screen_name": "kaitlyn_thomas_", "lang": "en", "location": "null", "create_at": date("2012-07-30"), "description": "Downey Dance Team • SFS Dance Center • There's strong and there's Downey Strong❤️", "followers_count": 280, "friends_count": 298, "statues_count": 6600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901505735491584, "text": "@Pachecs1 there isn't one ��", "in_reply_to_status": 737901320930250752, "in_reply_to_user": 871125714, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 871125714 }}, "user": { "id": 186650185, "name": "gen garcia", "screen_name": "gennnxo", "lang": "en", "location": "san josé state ", "create_at": date("2010-09-03"), "description": "you'd lose your mind trying to understand mine.", "followers_count": 1707, "friends_count": 786, "statues_count": 70175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506079395841, "text": "You are trash. Human detritus. https://t.co/nfxpXdJiWk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36164779, "name": "Steve Kassel", "screen_name": "TaxNegotiator", "lang": "en", "location": "Burlingame, CA, USA", "create_at": date("2009-04-28"), "description": "http://eTaxes.com EA Tax Pro 29 yrs Testified on Capitol Hill. Good Morning America & NYTimes Pro-Israel Maryland bball mgr. Ravens A's Cal #NeverTrump", "followers_count": 18879, "friends_count": 1400, "statues_count": 6071 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506091995136, "text": "I det depressed and then watch @yesmantron highlights on twitch just to torture myself. #BringMangoBack", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "BringMangoBack" }}, "user_mentions": {{ 2307093902 }}, "user": { "id": 39090404, "name": "Uhhnette", "screen_name": "annettesadek", "lang": "en", "location": "Boston, MA", "create_at": date("2009-05-10"), "description": "Queer. Gamer. Genderfluid. Chef. Girl. \nnot in any particular order.", "followers_count": 268, "friends_count": 450, "statues_count": 1170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506301689856, "text": "Look at God ���� https://t.co/qT1tGx270i", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 720136919, "name": "TheWifeOfPablo", "screen_name": "Pink_buggattii", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-10-17"), "description": "Big ❤️ Forever smiling .... The nicest person you'll ever meet Nomatter what happens in life being good to people is a wonderful legacy to leave behind", "followers_count": 5021, "friends_count": 754, "statues_count": 114909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506431746048, "text": "oomf is yum", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 173269316, "name": "كبسولة", "screen_name": "CVPSULE", "lang": "en", "location": "Wells Branch, TX", "create_at": date("2010-07-31"), "description": "21 | MGMT & SHOW BOOKINGS Contact : Adriel.rivera@goodvibespresents.com @GoodVibesPrsnts", "followers_count": 2245, "friends_count": 77, "statues_count": 126902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wells Branch, TX", "id": "daf932d3d5f17e87", "name": "Wells Branch", "place_type": "city", "bounding_box": rectangle("-97.697798,30.424563 -97.666593,30.467254") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4877196, "cityName": "Wells Branch" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506561724416, "text": "All u gotta do is say no homo so yeah im driving (no homo) https://t.co/SBvMpt0FX8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 989994308, "name": "PAP-E", "screen_name": "Jorge_B_10", "lang": "en", "location": "Portland, OR", "create_at": date("2012-12-04"), "description": "Im a brown kid with white ppl dreams. _________________#GoblinSquad _______________________17/Brown/Portland", "followers_count": 327, "friends_count": 567, "statues_count": 3383 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901506624671745, "text": "Your gonna have to show me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 861764846, "name": "pookie ✨", "screen_name": "ussery_morgan", "lang": "en", "location": "Riverbank, CA", "create_at": date("2012-10-04"), "description": "Im a bitch and hella sassy.", "followers_count": 692, "friends_count": 1085, "statues_count": 6799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverbank, CA", "id": "89e82c687111f62b", "name": "Riverbank", "place_type": "city", "bounding_box": rectangle("-120.981748,37.707792 -120.882789,37.768288") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 661068, "cityName": "Riverbank" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507035684866, "text": "Everybody know everybody ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3073619983, "name": "Papí Kay ✨", "screen_name": "kayyyy_mone", "lang": "en", "location": "Arkansauce", "create_at": date("2015-03-11"), "description": "SAU'19 | Un-Apologetic |", "followers_count": 336, "friends_count": 444, "statues_count": 8139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pine Bluff, AR", "id": "4fa26750e4d5620c", "name": "Pine Bluff", "place_type": "city", "bounding_box": rectangle("-92.092994,34.140549 -91.94191,34.268184") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5069, "countyName": "Jefferson", "cityID": 555310, "cityName": "Pine Bluff" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507228672000, "text": "i'd do that like and pick 2 ' thing but I got this 8 paragraphed speech i gotta memorize due on thurs so ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2460711272, "name": "ms.steele 2.0", "screen_name": "JBlaireee", "lang": "en", "location": " Sā-le-moa ☼", "create_at": date("2014-04-23"), "description": "1 Peter 3:3-4 ❤ // sifi li uso // E // squad up w God ღ", "followers_count": 414, "friends_count": 552, "statues_count": 14257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Gables, FL", "id": "778af41ffb719450", "name": "Coral Gables", "place_type": "city", "bounding_box": rectangle("-80.297165,25.627038 -80.245009,25.771859") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1214250, "cityName": "Coral Gables" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507492884482, "text": "@WildnAlex TU", "in_reply_to_status": 737900809107742721, "in_reply_to_user": 294239729, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 294239729 }}, "user": { "id": 67451906, "name": "Kirito", "screen_name": "AL_Grime", "lang": "en", "location": "bassPOD", "create_at": date("2009-08-20"), "description": "#BASSHEAD #DUBSTEPPER #FreeTheFilth #TeamBassrush #805RaveFam SC/IG: abkush", "followers_count": 537, "friends_count": 585, "statues_count": 36211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orcutt, CA", "id": "fcfe99a296187c82", "name": "Orcutt", "place_type": "city", "bounding_box": rectangle("-120.468231,34.842218 -120.378242,34.908588") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 654120, "cityName": "Orcutt" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507576729600, "text": "Highkey mad at my boy John for lagging today ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3236544270, "name": "Miguel baeza", "screen_name": "baezamiguel0", "lang": "en", "location": "null", "create_at": date("2015-06-04"), "description": "♠stay foolish, stay humble ♠", "followers_count": 324, "friends_count": 449, "statues_count": 1005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arvin, CA", "id": "22d1f67e5dc77410", "name": "Arvin", "place_type": "city", "bounding_box": rectangle("-118.860132,35.18714 -118.805953,35.230936") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 602924, "cityName": "Arvin" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507660664834, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2345639148, "name": "✨.", "screen_name": "lilcaaam", "lang": "en", "location": "riptyler! 9:25", "create_at": date("2014-02-15"), "description": "16. Lexus Is My BFF ❤️", "followers_count": 1531, "friends_count": 699, "statues_count": 53479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901507912343552, "text": "@NappUno lol I gotta get back into playing some sport", "in_reply_to_status": 737901166374326272, "in_reply_to_user": 1561388706, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1561388706 }}, "user": { "id": 29560033, "name": "Rufio™", "screen_name": "G0ld_Blo0ded", "lang": "en", "location": "Harlem", "create_at": date("2009-04-07"), "description": "If I could change your mind, I'd really love to break your heart.... KC native. Graphic designs upon request, DM for work and references.", "followers_count": 901, "friends_count": 304, "statues_count": 94976 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508122050560, "text": "@mamii__j Ztro - FDB Ringtone (OFFICIAL MUSIC VIDEO) https://t.co/jhIt7FFU9f �� #WATCH", "in_reply_to_status": 737901280618778624, "in_reply_to_user": 2349527082, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WATCH" }}, "user_mentions": {{ 2349527082 }}, "user": { "id": 20121208, "name": "INSTAGRAM: Zztro", "screen_name": "Zztro", "lang": "en", "location": "Detroit & Beyond ✈️", "create_at": date("2009-02-04"), "description": "|Instagram: Zztro| |Youtube: IAmAztroBoy #TeamZtro , For Booking: BookingZztro@gmail.com", "followers_count": 61686, "friends_count": 51440, "statues_count": 388411 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Huntsville, AL", "id": "76c1bb41d3de7ff1", "name": "Huntsville", "place_type": "city", "bounding_box": rectangle("-86.785881,34.578426 -86.436941,34.833627") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1089, "countyName": "Madison", "cityID": 137000, "cityName": "Huntsville" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508155559936, "text": "Life's full of ups and downs I done seen more down then ups", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4886351644, "name": "LOWKEYTSN", "screen_name": "bcastillo108", "lang": "en", "location": "null", "create_at": date("2016-02-07"), "description": "null", "followers_count": 279, "friends_count": 286, "statues_count": 2362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508172353537, "text": "wife wife wife https://t.co/Zvm7y8miws", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2736869888, "name": "Felippe Dacudao", "screen_name": "felippedacudao_", "lang": "en", "location": "null", "create_at": date("2014-08-16"), "description": "do you.", "followers_count": 186, "friends_count": 225, "statues_count": 1330 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508579229696, "text": "Zquad come on let's do this shit #Zayn4MMVA #PillowTalkTo1B #LikeIWouldToVevoCertified https://t.co/kzupl9ctFY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Zayn4MMVA", "PillowTalkTo1B", "LikeIWouldToVevoCertified" }}, "user": { "id": 3667212912, "name": "LORDE4323", "screen_name": "Lorde4323", "lang": "en", "location": "stalking Zayn and Liam", "create_at": date("2015-09-23"), "description": "Hiii I love Adele, Nicki Minaj, Zayn Malik, Sia, and Lorde so attack them and in my mind I have already plotted your death in over a hundred ways so yaaaay", "followers_count": 243, "friends_count": 535, "statues_count": 3865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bruno, CA", "id": "5358b6f78dd95ef6", "name": "San Bruno", "place_type": "city", "bounding_box": rectangle("-122.473595,37.600909 -122.399963,37.641689") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 665028, "cityName": "San Bruno" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508633755648, "text": "Maybe you should charge your phone", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940148423, "name": "J.", "screen_name": "Ja_hill11", "lang": "en", "location": "null", "create_at": date("2014-12-22"), "description": "#LackOfWorries", "followers_count": 277, "friends_count": 323, "statues_count": 610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haltom City, TX", "id": "69644b8de3ee73aa", "name": "Haltom City", "place_type": "city", "bounding_box": rectangle("-97.293831,32.779549 -97.236256,32.861171") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4831928, "cityName": "Haltom City" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508717649920, "text": "Everyone talking about the storm but my lazy ass was asleep through the whole thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 149335983, "name": "Aracelý.", "screen_name": "aracelyarriaga1", "lang": "en", "location": "Sc: @ylecara_6", "create_at": date("2010-05-28"), "description": "null", "followers_count": 286, "friends_count": 203, "statues_count": 6514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901508944072704, "text": ".@FoxNews Fox rolls out Megyn Kelly - American Sleaze", "in_reply_to_status": 737900262812057600, "in_reply_to_user": 1367531, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1367531 }}, "user": { "id": 708814531233861633, "name": "Don Davis", "screen_name": "Don1Davis1", "lang": "en", "location": "Orange, CA", "create_at": date("2016-03-12"), "description": "Concerned American wanting to keep the American Dream alive.", "followers_count": 194, "friends_count": 188, "statues_count": 614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tustin, CA", "id": "02215ae29a77567c", "name": "Tustin", "place_type": "city", "bounding_box": rectangle("-117.845301,33.69418 -117.758969,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 680854, "cityName": "Tustin" } }
+{ "create_at": datetime("2016-06-01T00:00:03.000Z"), "id": 737901509204160512, "text": "Thankful for my raise cause my paycheck is more than I thought it was going to beeeee ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60526689, "name": "ash", "screen_name": "ashapeaa", "lang": "en", "location": "null", "create_at": date("2009-07-27"), "description": "null", "followers_count": 530, "friends_count": 223, "statues_count": 106808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901509342531584, "text": "@Stylowtrix me and my long hair buddy talked to you in line earlier", "in_reply_to_status": 737900077084184578, "in_reply_to_user": 60425298, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60425298 }}, "user": { "id": 28333748, "name": "Rucka Rucka Ali", "screen_name": "RuckaRuckaAli", "lang": "en", "location": "LA homes!", "create_at": date("2009-04-02"), "description": "Born in jail. Never had a chance #FreeRucka YouTube sensation since 1994.", "followers_count": 38576, "friends_count": 1414, "statues_count": 49289 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901509384474625, "text": "What the fuck is up June��✨ https://t.co/Bf504XXNA3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 32263735, "name": "ANA", "screen_name": "Simplyy_Anaa", "lang": "en", "location": "⚠️✨ P L U R ✨⚠️ ASOC", "create_at": date("2009-04-16"), "description": "alpha omicron pi • edgar ❥", "followers_count": 541, "friends_count": 186, "statues_count": 10127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901509443260416, "text": "I been ♿️oolin a little too much lately ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540896810, "name": "Nara", "screen_name": "naraax_", "lang": "en", "location": "uk'20", "create_at": date("2012-03-30"), "description": "prospering. natriece is sexy. I love Montre.", "followers_count": 2138, "friends_count": 1455, "statues_count": 70114 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901509665562624, "text": "I have to go to my car ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 226791316, "name": "Magnolia", "screen_name": "maggienoliaaaaa", "lang": "en", "location": "Chicago ✈️ Baton Rouge", "create_at": date("2010-12-14"), "description": "Southern University| Bestfriend Vlogger | be the change you want to see in the world.| Watch my latest video below ☺️", "followers_count": 1102, "friends_count": 1008, "statues_count": 66427 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901509929762817, "text": "@JonnyM1760 @veleennn did", "in_reply_to_status": 737901486294831104, "in_reply_to_user": 1333421738, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 455494199, 3163407787 }}, "user": { "id": 1333421738, "name": "alpaca ❁", "screen_name": "xoalbaceline_", "lang": "en", "location": "null", "create_at": date("2013-04-07"), "description": "i hate nick", "followers_count": 873, "friends_count": 527, "statues_count": 73796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mesa, CA", "id": "c8ccc9439a8e5ee0", "name": "La Mesa", "place_type": "city", "bounding_box": rectangle("-117.053546,32.743581 -116.981714,32.7958") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 640004, "cityName": "La Mesa" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901510001106944, "text": "I'm chasin' your pretty thoughts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 129355432, "name": "ℒ.", "screen_name": "fvcklily", "lang": "en", "location": "null", "create_at": date("2010-04-03"), "description": "scorpio sun • sagittarius moon/dominant Ꮤalker's girl❤️", "followers_count": 1334, "friends_count": 1110, "statues_count": 37756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, CA", "id": "1cf657005480e70e", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-117.399853,33.152639 -117.241905,33.299993") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 653322, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901510005297152, "text": "Wat. It's already june\n\nYou made it through half of 2016 !! The 99 cents store starts selling Halloween decorations in 2 months !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 628979232, "name": "Cecilia (*˙︶˙*)ノ", "screen_name": "cessadilla", "lang": "en", "location": "Los Angeles • Disneyland ", "create_at": date("2012-07-06"), "description": "And so, being young and dipt in folly, I fell in love with melancholy ☆゚.*・。゚*", "followers_count": 6607, "friends_count": 231, "statues_count": 18800 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Fernando, CA", "id": "aa755c5c1b8e341b", "name": "San Fernando", "place_type": "city", "bounding_box": rectangle("-118.456347,34.273334 -118.41567,34.304639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 666140, "cityName": "San Fernando" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901510059819008, "text": "Happy birthday to one of my best friends in the whole world.. You are so sweet and so fun to be around.. Love you❤️�� https://t.co/CRjfX7rkHe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2211982513, "name": "Charlie Kimball", "screen_name": "ckimball24", "lang": "en", "location": "null", "create_at": date("2013-11-23"), "description": "⛳️❤️ sc: charliekimball6", "followers_count": 240, "friends_count": 464, "statues_count": 520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901510261084160, "text": "Wind 0.0 mph E. Barometer 29.840 in, Falling slowly. Temperature 66.2 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8849 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-01T00:00:04.000Z"), "id": 737901511200636929, "text": "Idc be mad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2591563338, "name": "Kris", "screen_name": "Lockdown13_", "lang": "en", "location": "null", "create_at": date("2014-06-27"), "description": "stay tuned #SOSU commit", "followers_count": 359, "friends_count": 250, "statues_count": 2185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263880514736128, "text": "who y'all talking about I'm trynna figure out ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 499410242, "name": "Donkey Kong", "screen_name": "BigPapiiRedd", "lang": "en", "location": "null", "create_at": date("2012-02-21"), "description": "Every dog has his day now doggystyle shall help", "followers_count": 694, "friends_count": 625, "statues_count": 49230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Killeen, TX", "id": "a27a0542e774e138", "name": "Killeen", "place_type": "city", "bounding_box": rectangle("-97.848463,31.014356 -97.659217,31.144257") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4839148, "cityName": "Killeen" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263880812503040, "text": "I just broke a glass plate on my foot while trying to make my baby a late night snack - half asleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 192658600, "name": "j a s", "screen_name": "lovely_JMA", "lang": "en", "location": "Baby Marco❤️", "create_at": date("2010-09-19"), "description": "GOD IS WITHIN HER, SHE WILL NOT FALL. psalms 46:5", "followers_count": 767, "friends_count": 496, "statues_count": 28984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263880963526656, "text": "my heart has literally shattered into a million pieces while watching this movie & it's only been 20 minutes.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1711664832, "name": "$CRUMPTIOU$", "screen_name": "MirandaMueller_", "lang": "en", "location": "at lahc playing volleyball ", "create_at": date("2013-08-29"), "description": "i feel like pablo.", "followers_count": 1158, "friends_count": 673, "statues_count": 21966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263881357762564, "text": "I'm a sarcastic little asshole", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 340834099, "name": "CourtCourt", "screen_name": "itscourtkneee", "lang": "en", "location": "da baaayyy", "create_at": date("2011-07-23"), "description": "watchu lookin' at", "followers_count": 878, "friends_count": 260, "statues_count": 41487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263881848496128, "text": "@DaliaHammouri YOU ARE THE MOST PRECIOUS BEAN IN THE WORLD", "in_reply_to_status": 738262333437317120, "in_reply_to_user": 157246169, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 157246169 }}, "user": { "id": 422157598, "name": "Yessy Lopez☮", "screen_name": "Simply_XO", "lang": "en", "location": "Somewhere In Neverland, CA", "create_at": date("2011-11-26"), "description": "20. Finally The Sun Is Shining", "followers_count": 604, "friends_count": 601, "statues_count": 37841 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263882213425152, "text": "me yesterday when i got paid from both jobs�� https://t.co/RvxDokCd2k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3167129927, "name": "KIDS©", "screen_name": "DancingKye", "lang": "en", "location": "null", "create_at": date("2015-04-14"), "description": "God . Choreographer. #DancingKye #ProBlack #PV18", "followers_count": 566, "friends_count": 103, "statues_count": 9979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263882347646976, "text": "you'd pick her over me any day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2673868270, "name": "taytay", "screen_name": "taytayyyeee", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-07-04"), "description": "daisy daisy give me your answer now", "followers_count": 535, "friends_count": 627, "statues_count": 3050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263882825912320, "text": "Swear I wasn't gone text back I gotta stop giving in", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 406640058, "name": "Nae", "screen_name": "Naenae_8", "lang": "en", "location": "Akron, OH", "create_at": date("2011-11-06"), "description": "null", "followers_count": 1352, "friends_count": 1635, "statues_count": 29123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garfield Heights, OH", "id": "4bcff2f443c05374", "name": "Garfield Heights", "place_type": "city", "bounding_box": rectangle("-81.637998,41.384569 -81.56761,41.441812") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3929428, "cityName": "Garfield Heights" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263883299708928, "text": "I have the deepest love and respect for @YoursTrulyPolo ❤️����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2739856583 }}, "user": { "id": 2205237215, "name": "✨Shannon✨", "screen_name": "bossassbeach", "lang": "en", "location": "jus trynna have fun maaaannee", "create_at": date("2013-12-02"), "description": "19 || fl || insta:caseyyyyfl || snapchat:caseyyyyfl", "followers_count": 439, "friends_count": 1386, "statues_count": 13245 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casselberry, FL", "id": "a8237d3409a76683", "name": "Casselberry", "place_type": "city", "bounding_box": rectangle("-81.350571,28.610836 -81.286124,28.697981") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12117, "countyName": "Seminole", "cityID": 1211050, "cityName": "Casselberry" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263883379396609, "text": "what I'm trying to do this weekend https://t.co/pmXl3t5lki", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2911357632, "name": "gus", "screen_name": "normalgustavo", "lang": "en", "location": "null", "create_at": date("2014-11-26"), "description": "RMHS Varsity Soccer ⚽️ #11", "followers_count": 170, "friends_count": 284, "statues_count": 692 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263883610259457, "text": "Un jour je roulerais en Ferrari avec une michto' a mes côtés ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 2209083615, "name": "GITANO", "screen_name": "lucas____M", "lang": "fr", "location": "Lille / Paris / Cap d'agde ", "create_at": date("2013-12-05"), "description": "gispy", "followers_count": 607, "friends_count": 90, "statues_count": 670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263883614298112, "text": "������- omg you seem like you have good vibes girl, keep your head up. You got a friend in me if you need anyone ������ and you're ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 363399618, "name": "VIVAALAMAI ✨", "screen_name": "Maiiraaa_", "lang": "en", "location": "bay area", "create_at": date("2011-08-27"), "description": "vivaalahennessey", "followers_count": 1169, "friends_count": 538, "statues_count": 35262 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263883878682624, "text": "@whoisluka usher photobomb such a wave", "in_reply_to_status": 738258843638104064, "in_reply_to_user": 40505249, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 40505249 }}, "user": { "id": 3983094383, "name": "Robert Echols Jr.", "screen_name": "Recholsjr", "lang": "en", "location": "New York, USA", "create_at": date("2015-10-17"), "description": "http://rxecholsjr.tumblr.com", "followers_count": 79, "friends_count": 39, "statues_count": 2682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greece, NY", "id": "00d1c94455339375", "name": "Greece", "place_type": "city", "bounding_box": rectangle("-77.761369,43.178823 -77.615702,43.32408") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36055, "countyName": "Monroe", "cityID": 3630279, "cityName": "Greece" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263884448989184, "text": "Lemme try this https://t.co/REICVAOfNN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2339737328, "name": "C-Man Cundary™", "screen_name": "CCundary", "lang": "en", "location": "null", "create_at": date("2014-02-11"), "description": "don't get lost in the sauce [MPHS football] 19'", "followers_count": 140, "friends_count": 167, "statues_count": 384 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-02T00:00:00.000Z"), "id": 738263884461576192, "text": "I'm just a soul whose intentions are good. Oh Lord, please don't let me be misunderstood", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2251524362, "name": "Jazz", "screen_name": "jazzy1822", "lang": "en", "location": "null", "create_at": date("2013-12-17"), "description": "It takes strength to be gentle and kind. ΔΔΔ", "followers_count": 190, "friends_count": 211, "statues_count": 5516 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263884696420352, "text": "@ebbtideapp Tide in New Topsail Inlet, North Carolina 06/02/2016\nHigh 5:34am 3.0\n Low 12:21pm -0.4\nHigh 6:14pm 3.9\n Low 1:19am -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-77.6333,34.3667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 100, "friends_count": 1, "statues_count": 37911 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Carolina, USA", "id": "3b98b02fba3f9753", "name": "North Carolina", "place_type": "admin", "bounding_box": rectangle("-84.321948,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37141, "countyName": "Pender", "cityID": 3768040, "cityName": "Topsail Beach" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885019500544, "text": "06/02@03:00 - Temp 66.6F, WC 66.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.113in, Steady. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885199728642, "text": "@noe_el_tigre skrt skrt", "in_reply_to_status": 738259664958148610, "in_reply_to_user": 907317978, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 907317978 }}, "user": { "id": 704205050, "name": "TRINITY", "screen_name": "Trinity_454", "lang": "en", "location": "null", "create_at": date("2012-07-18"), "description": "null", "followers_count": 311, "friends_count": 27, "statues_count": 26677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885220716544, "text": "@theweeknd DADDYYY https://t.co/lTq66xG0aA", "in_reply_to_status": -1, "in_reply_to_user": 255388236, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 255388236 }}, "user": { "id": 2912515172, "name": "Mary berry", "screen_name": "nnotspoild", "lang": "en", "location": "Houston, TX", "create_at": date("2014-11-27"), "description": "Savannah", "followers_count": 333, "friends_count": 339, "statues_count": 9696 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885409587200, "text": "@bubblejack @purplelover513 ANG HIRAP NUN BEH! MAMBIBIGWAS TALAGA AKO SHET\n#ALDUB46thWeeksary", "in_reply_to_status": 738263494127026176, "in_reply_to_user": 16109099, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUB46thWeeksary" }}, "user_mentions": {{ 16109099, 54169660 }}, "user": { "id": 372622113, "name": "MaiChard Faulkerson", "screen_name": "akosieyjeyzee08", "lang": "en", "location": "This Corner, MaiChard Nation", "create_at": date("2011-09-12"), "description": "loves RRFJ&NDCM wholeheartedly || A Ninang of MaiChard || Defender || Team Butchikik", "followers_count": 861, "friends_count": 180, "statues_count": 44377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hicksville, NY", "id": "48b5e9defad76941", "name": "Hicksville", "place_type": "city", "bounding_box": rectangle("-73.560512,40.743142 -73.499673,40.793139") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3634374, "cityName": "Hicksville" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885422178304, "text": "Temp 50.6° Hi/Lo 53.8/50.6 Rng 3.2° WC 50.6° Hmd 94% Rain 0.00\" Storm 0.00\" BAR 30.237 Steady DP 48.9° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 72, "friends_count": 123, "statues_count": 19175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885451399169, "text": "I'm so lucky to be blessed with the family that I have.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842771425, "name": "sc: ebonyshantel✨", "screen_name": "EbonyShantel_", "lang": "en", "location": "Georgia, USA", "create_at": date("2012-09-23"), "description": "I am the daughter of a King ✨", "followers_count": 8558, "friends_count": 7205, "statues_count": 84565 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Ridge, VA", "id": "bd2c3c1743dc3dcf", "name": "Lake Ridge", "place_type": "city", "bounding_box": rectangle("-77.362725,38.660632 -77.259114,38.709815") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5143432, "cityName": "Lake Ridge" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263885908697088, "text": "#hrcc #aprilangel100 #haleyrae #haleyraedancer #haleyraelikesthis #haleyraecannell… https://t.co/P3WbZGOpF8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.296862,34.057563"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "hrcc", "aprilangel100", "haleyrae", "haleyraedancer", "haleyraelikesthis", "haleyraecannell" }}, "user": { "id": 44484851, "name": "HaleyRae Cannell", "screen_name": "HaleyRaeDancer", "lang": "en", "location": "Beverly Hills, CA", "create_at": date("2009-06-03"), "description": "Facebook: https://www.facebook.com/haleyraedancer Twitter: https://twitter.com/HaleyRaeDancer Instagram: http://instagram.com/haleyraecannell/", "followers_count": 1542, "friends_count": 1483, "statues_count": 23094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263886126682112, "text": "@megweiss13 Miss you Missy !!!", "in_reply_to_status": 738200398717014017, "in_reply_to_user": 295387549, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 295387549 }}, "user": { "id": 882751502, "name": "Robert Wisdom", "screen_name": "RobertRayWisdom", "lang": "en", "location": "LA ", "create_at": date("2012-10-15"), "description": "Rocket Scientist. Ok, I played one on TV.Pulitzer Prize winning writer ....Ok I played one in a movie. Brilliant Crimestopping Major Commander ...Ok.Actor !", "followers_count": 5660, "friends_count": 2497, "statues_count": 4441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Monica, CA", "id": "59612bd882018c51", "name": "Santa Monica", "place_type": "city", "bounding_box": rectangle("-118.517358,33.995177 -118.443482,34.050199") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 670000, "cityName": "Santa Monica" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263886453825536, "text": "Happy birthday papi chulo @ivnasty_ love you bro ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 547713856 }}, "user": { "id": 1151389104, "name": "Comochin Gas", "screen_name": "PARGANEXTDOOR_", "lang": "en", "location": "null", "create_at": date("2013-02-05"), "description": "brb", "followers_count": 594, "friends_count": 458, "statues_count": 24669 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Ana, CA", "id": "0562e9e53cddf6ec", "name": "Santa Ana", "place_type": "city", "bounding_box": rectangle("-117.944051,33.691774 -117.830826,33.78821") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 669000, "cityName": "Santa Ana" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263886629986304, "text": "I talk soooo much lolol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1173171254, "name": "mads ◡̈", "screen_name": "mbayleehall8", "lang": "en", "location": "p h x ", "create_at": date("2013-02-12"), "description": "If not now, when? | GWHS ◡̈ |", "followers_count": 846, "friends_count": 736, "statues_count": 21904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263886906826752, "text": "@vjjlone take me wit you ��", "in_reply_to_status": 738139416619667457, "in_reply_to_user": 726059068024983552, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 726059068024983552 }}, "user": { "id": 2199032647, "name": "superduperchad", "screen_name": "yungapex", "lang": "en", "location": "Your Grandma house", "create_at": date("2013-11-16"), "description": "Founder of The New Yuth Blog /photographer/ #dirtykid", "followers_count": 1028, "friends_count": 1419, "statues_count": 23854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887057801216, "text": "@kittyyxxcat @kittyyxxcat Channel 9 came to graduation, interviewed me, and put me on the 11 O'clock news ��", "in_reply_to_status": 738262786405367808, "in_reply_to_user": 2275388821, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2275388821, 2275388821 }}, "user": { "id": 1140613250, "name": "Tori Moreno", "screen_name": "moreno_tori", "lang": "en", "location": "null", "create_at": date("2013-02-01"), "description": "//™// let it be", "followers_count": 147, "friends_count": 88, "statues_count": 9949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887116533760, "text": "Long live the Realm with One Banner! https://t.co/BOaYhcKjnN @SlamStevo @Dame_Lillard", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 96174061, 267425142 }}, "user": { "id": 359618943, "name": "Eddie Mack", "screen_name": "eddieamack", "lang": "en", "location": "null", "create_at": date("2011-08-21"), "description": "HW Basketball, USC, Everton, Timbers, Trail Blazers, Chiefs.", "followers_count": 74, "friends_count": 248, "statues_count": 526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887473201152, "text": "Wind 0.0 mph ---. Barometer 29.967 in, Steady. Temperature 67.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887552741377, "text": "HAPPY 21ST BIRTHDAY BIG DIAAAAMMMOOONNNDDDDDD ������������ @kem_62", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1615636536 }}, "user": { "id": 103429715, "name": "-‘๑’-", "screen_name": "_mayrraa", "lang": "en", "location": "CSUF • ΑΔΠ", "create_at": date("2010-01-09"), "description": "i take my coffee cold", "followers_count": 833, "friends_count": 313, "statues_count": 44198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887703736320, "text": "PND x Thristy >", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 247850401, "name": "Drake's Babymama", "screen_name": "TonyToniTvne", "lang": "en", "location": "Texas State University", "create_at": date("2011-02-05"), "description": "21 . we not cool so stop checking for me....", "followers_count": 1959, "friends_count": 932, "statues_count": 105402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263887749861382, "text": "tbh going out with u kno who bc i ddint really have feelings for the kid i was just bored and that was wrong of me 2 https://t.co/QNQ25C5IJx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1862799823, "name": "R❣", "screen_name": "Rebekaaaaahh", "lang": "en", "location": "null", "create_at": date("2013-09-13"), "description": "bay area", "followers_count": 204, "friends_count": 172, "statues_count": 1932 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263888366440449, "text": "@VictoriaPresti miss you more babe", "in_reply_to_status": 738261044523458560, "in_reply_to_user": 599660255, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 599660255 }}, "user": { "id": 599932448, "name": "Myk✨", "screen_name": "MykahlaR8", "lang": "en", "location": "Arizona, USA", "create_at": date("2012-06-05"), "description": "Senior @ DHS '17 http://vsco.co/mykahlar8", "followers_count": 879, "friends_count": 356, "statues_count": 47923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-06-02T00:00:01.000Z"), "id": 738263888475590656, "text": "Wind 1.0 mph SW. Barometer 29.967 in, Steady. Temperature 69.2 °F. Rain today 0.02 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263888823648256, "text": "Wish it was 2am then 4am :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2730263771, "name": "Nathan Drake", "screen_name": "cesarslaysall", "lang": "en", "location": "Dreamville ", "create_at": date("2014-07-31"), "description": "dm me and i'll be your new boyfriend but don't let your current boyfriend find out", "followers_count": 957, "friends_count": 471, "statues_count": 59112 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263888836206593, "text": "I think I have embraced the friend zone and I'm fine with it haha. Thank god", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2868354804, "name": "Austin", "screen_name": "Handelaa", "lang": "en", "location": "null", "create_at": date("2014-10-20"), "description": "BHB/\\ MLV\nFirst Mate of the Becky May", "followers_count": 319, "friends_count": 214, "statues_count": 17946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263888978907136, "text": "@jotr_ yeah", "in_reply_to_status": 738261533826764800, "in_reply_to_user": 3133939482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3133939482 }}, "user": { "id": 581149363, "name": "shilo wertenberger", "screen_name": "vaguevisions", "lang": "en", "location": "Internal ", "create_at": date("2012-05-15"), "description": "null", "followers_count": 1178, "friends_count": 142, "statues_count": 40474 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889083781120, "text": "Solid Soundcloud Grab Sesh cause DJs still do that rt ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224658934, "name": "Let Me Be Great", "screen_name": "DJDAV215", "lang": "en", "location": "null", "create_at": date("2010-12-09"), "description": "null", "followers_count": 352, "friends_count": 378, "statues_count": 7094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889305968640, "text": "I sure hope my pineapple ripens soon", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2440909706, "name": "Skyler Mitchell", "screen_name": "SkylerRMitchell", "lang": "en", "location": "Earth", "create_at": date("2014-04-12"), "description": "I still have 2 kidneys if anyone needs a spare. Volunteer Firefighter/EMT, Lifeguard Instructor, Dr. Pepper enthusiast. I like to see and do new things.", "followers_count": 96, "friends_count": 77, "statues_count": 904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moses Lake, WA", "id": "513be78e6847eb1a", "name": "Moses Lake", "place_type": "city", "bounding_box": rectangle("-119.363589,47.078794 -119.241434,47.162879") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5347245, "cityName": "Moses Lake" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889381576704, "text": "Temp: 72.6°F Wind:0.0mph Pressure: 29.988hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889469571072, "text": "I wish I was born in the 1800's where everyone had to work hard. If you didn't you died.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483658001, "name": "Dax Wilson", "screen_name": "DWIL12_", "lang": "en", "location": "null", "create_at": date("2012-02-04"), "description": "Shasta Knights Football Oregon to Cali Snapchat: dax12wilson Instagram: DWIL12_ http://www.hudl.com/athlete/1893887/highlights/292821376/v2# #farming #TheBurg", "followers_count": 610, "friends_count": 672, "statues_count": 11672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harrisburg, OR", "id": "43a78a84849946bd", "name": "Harrisburg", "place_type": "city", "bounding_box": rectangle("-123.179834,44.260945 -123.154251,44.284709") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4132550, "cityName": "Harrisburg" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889473875968, "text": "I miss the old days when I would come home and tell no one I was back, park the 85 chevy in the driveway, and just drink and listen to music", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505844570, "name": "Preston Trible Jr", "screen_name": "Whiteman703", "lang": "en", "location": "The deep south", "create_at": date("2012-02-27"), "description": "How much ass could a woodchuck chuck if a woodchuck could chuck ASS?", "followers_count": 77, "friends_count": 109, "statues_count": 1243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Great Falls, VA", "id": "00e6e51f30fc5e36", "name": "Great Falls", "place_type": "city", "bounding_box": rectangle("-77.371398,38.966116 -77.259402,39.058008") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5132496, "cityName": "Great Falls" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263889700212736, "text": "@throwinschade you missed another song name drop", "in_reply_to_status": 738263682258345987, "in_reply_to_user": 448528799, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 448528799 }}, "user": { "id": 187582032, "name": "anthony", "screen_name": "AnthonyvanAtten", "lang": "en", "location": "Edinburg, TX", "create_at": date("2010-09-06"), "description": "i'm sorry you feel that way. #organizednature #freakshow http://Instagram.com/anthonyvanatten anthonyvanatten@gmail.com", "followers_count": 1582, "friends_count": 984, "statues_count": 80324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edinburg, TX", "id": "013fbf673c82e5e4", "name": "Edinburg", "place_type": "city", "bounding_box": rectangle("-98.225807,26.244127 -98.079031,26.450924") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4822660, "cityName": "Edinburg" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263890421653504, "text": "Wind 0.0 mph ---. Barometer 29.935 in, Steady. Temperature 69.7 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263890442608641, "text": "\"Sucks to Suck\"\nAn autobiography", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143307676, "name": "TY DARBY", "screen_name": "ShipWreck666", "lang": "en", "location": "Port Aransas, TX", "create_at": date("2010-05-12"), "description": "Someday We'll Look Back and Laugh/ 24/Kayla/Shredder/ CLOUD BREATHER/ ♒️ IG: dirtbaghooligan", "followers_count": 656, "friends_count": 271, "statues_count": 26664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Aransas, TX", "id": "da8459b9a91f17b8", "name": "Port Aransas", "place_type": "city", "bounding_box": rectangle("-97.095004,27.800207 -97.044738,27.841911") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4858808, "cityName": "Port Aransas" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263891042402304, "text": "Team rocket// lil uzi +++", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1173798468, "name": "CALIFORNIA KID", "screen_name": "terryryan909", "lang": "en", "location": "null", "create_at": date("2013-02-12"), "description": "| The rose that broke through the concrete | sc:terryryan909", "followers_count": 1374, "friends_count": 575, "statues_count": 2569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263891457630208, "text": "Fuck high school I'm hype for college", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 559183881, "name": "The Issue", "screen_name": "izabel_mcclure", "lang": "en", "location": "null", "create_at": date("2012-04-20"), "description": "Bitches be worried about the wrong thing, I am the wrong thing", "followers_count": 424, "friends_count": 382, "statues_count": 4295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263891583471616, "text": "The amount of love I have for you is unreal", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609771209, "name": "Alex♊️", "screen_name": "Shining_Abyss", "lang": "en", "location": "So Cal ☀️", "create_at": date("2012-06-16"), "description": "I like pupusas.\nEntrepreneur mindset.", "followers_count": 239, "friends_count": 493, "statues_count": 9980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263891713609728, "text": "let me get off Twitter before I start purpin lmaooooo tf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3116282809, "name": "big pimpin", "screen_name": "RAYSHELLEEEE", "lang": "en", "location": "null", "create_at": date("2015-03-29"), "description": "stop liking the boy who treats you like shit & start loving the cranberry vodka that's always there for you", "followers_count": 1103, "friends_count": 871, "statues_count": 13140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montgomery Village, MD", "id": "017a75c0406737c7", "name": "Montgomery Village", "place_type": "city", "bounding_box": rectangle("-77.231293,39.152407 -77.112229,39.235691") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2453325, "cityName": "Montgomery Village" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263892237950976, "text": "Wind 0.0 mph ENE. Barometer 30.16 in, Steady. Temperature 68.0 °F. Rain today 0.00 in. Humidity 65%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 123, "statues_count": 160233 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263892355211264, "text": "LMAO RT @JayZOverrated: interesting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 269200310 }}, "user": { "id": 236156697, "name": "Shamron Shawty", "screen_name": "KennyWitTheKoke", "lang": "en", "location": "318225", "create_at": date("2011-01-09"), "description": "Excuse the Rant | #EnginEars | EQ: $30 /hr Recording x $40 /song mix/master | excusetherant@gmail.com | I ain’t nothing but a nigga, ain’t no reason to pretend.", "followers_count": 900, "friends_count": 451, "statues_count": 119026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263892443287552, "text": "@PadresPls wait", "in_reply_to_status": 738263145236422656, "in_reply_to_user": 4230099679, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4230099679 }}, "user": { "id": 310998235, "name": "Stupid Idiot", "screen_name": "Pimp_Lord619", "lang": "en", "location": "null", "create_at": date("2011-06-04"), "description": "Obsessive San Diego sports fan. Padres, Chargers, Aztecs. I love memes and Mallory ❤️ I'm pimp! Writer for http://EastVillageTimes.com", "followers_count": 852, "friends_count": 848, "statues_count": 21094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ramona, CA", "id": "f7398eefb8e83860", "name": "Ramona", "place_type": "city", "bounding_box": rectangle("-116.915445,33.008206 -116.826752,33.079697") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 659346, "cityName": "Ramona" } }
+{ "create_at": datetime("2016-06-02T00:00:02.000Z"), "id": 738263892845940736, "text": "@IAMJLOV3 thanks ����", "in_reply_to_status": 738263611328471040, "in_reply_to_user": 82853064, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 82853064 }}, "user": { "id": 407266863, "name": "slim thug", "screen_name": "JustRUN_TRACK", "lang": "en", "location": "On the TRACK ", "create_at": date("2011-11-07"), "description": "God First #TrackNation #TrackisLife Events 400 & 800 Relays: 4x4 4x8 Kutztown University '19 | https://soundcloud.com/coreylee610/cam-newton-prod-by-yung-trel", "followers_count": 2191, "friends_count": 2185, "statues_count": 61435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893017919488, "text": "@JMathSix id literally drop out of school and move to Florida to be your wife.... Dead serious", "in_reply_to_status": 738263392645840897, "in_reply_to_user": 1473319603, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4898123009 }}, "user": { "id": 1473319603, "name": "dommy lommy", "screen_name": "dommm_t31", "lang": "en", "location": "Whittier, CA", "create_at": date("2013-05-31"), "description": "@ chick-fil-a everyday csuf'20", "followers_count": 482, "friends_count": 421, "statues_count": 13875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893026312195, "text": "@_kThug HES MOVING !!! Lol", "in_reply_to_status": 738263709424844800, "in_reply_to_user": 410378183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 410378183 }}, "user": { "id": 3324241202, "name": "Taylor Gurshin", "screen_name": "TayTime545", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-08-22"), "description": "Think Twice.", "followers_count": 118, "friends_count": 290, "statues_count": 928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whitney, NV", "id": "cb5ac954e9b77fba", "name": "Whitney", "place_type": "city", "bounding_box": rectangle("-115.064609,36.074534 -115.010215,36.137196") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3283800, "cityName": "Whitney" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893407981568, "text": "@trevzilla09 @bobbyJbranson thanks for the nonsweetner.", "in_reply_to_status": -1, "in_reply_to_user": 552418459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 552418459, 710310818127044608 }}, "user": { "id": 38521868, "name": "Sheri Quigley", "screen_name": "squigleys", "lang": "en", "location": "null", "create_at": date("2009-05-07"), "description": "Live, love & laugh", "followers_count": 44, "friends_count": 42, "statues_count": 62 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spanish Springs, NV", "id": "2ef2e027146be550", "name": "Spanish Springs", "place_type": "city", "bounding_box": rectangle("-119.735925,39.621614 -119.675337,39.6768") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32031, "countyName": "Washoe", "cityID": 3268350, "cityName": "Spanish Springs" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893491912704, "text": "GIRL U TRIED IT https://t.co/4soAyi5Hv1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3657330014, "name": "HUNG solo", "screen_name": "isol8d", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-09-23"), "description": "null", "followers_count": 317, "friends_count": 229, "statues_count": 16737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893827420161, "text": "Ok gn I need my 3 hrs of sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 913853382, "name": "rossie castle", "screen_name": "rossiepollie", "lang": "en", "location": "Heart of TX, USA", "create_at": date("2012-10-29"), "description": "null", "followers_count": 1266, "friends_count": 434, "statues_count": 54768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263893911293953, "text": "Done at the stroke of midnight, what perfect timing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 618449076, "name": "αlphaNinjaEX", "screen_name": "AlphaSteelix", "lang": "en", "location": "Oregon, USA", "create_at": date("2012-06-25"), "description": "| ♀ | I spoil stuff | A Jewish person who talks about Yu-Gi-Oh #ARCV & Pokemon. A lot. #Gundam #kaitoujoker #digimon", "followers_count": 442, "friends_count": 96, "statues_count": 21863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263894032945154, "text": "@LauraOliveira_5 \n-Starbucks \n-dog filter \n-mixed drinks\n-BP\n-sweetheart", "in_reply_to_status": -1, "in_reply_to_user": 236247482, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 236247482 }}, "user": { "id": 434675631, "name": "bay", "screen_name": "Bayfayylovaa_", "lang": "en", "location": "null", "create_at": date("2011-12-11"), "description": "senior | store #267 | love of my life @bennbett_62", "followers_count": 644, "friends_count": 242, "statues_count": 55232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newman, CA", "id": "bae100a1ab853e56", "name": "Newman", "place_type": "city", "bounding_box": rectangle("-121.037486,37.303666 -121.002063,37.32759") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 651140, "cityName": "Newman" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263894456573952, "text": "Damn, that's disappointing man was funny af https://t.co/tuQnZYvpwp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538695383, "name": "Grill Master Chef", "screen_name": "DrZakky", "lang": "en", "location": "Grill Heaven or Fl", "create_at": date("2012-03-27"), "description": "PUT EM ON THE GRILL iDontTextFirst", "followers_count": 1089, "friends_count": 879, "statues_count": 35583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Charlotte, FL", "id": "9ea56e2dd549b99e", "name": "Port Charlotte", "place_type": "city", "bounding_box": rectangle("-82.174795,26.953081 -82.057555,27.032115") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12015, "countyName": "Charlotte", "cityID": 1258350, "cityName": "Port Charlotte" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263894897000448, "text": "Glad I got to see @emfigs tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 621307039 }}, "user": { "id": 68330164, "name": "Richard Campos", "screen_name": "RichCamp45", "lang": "en", "location": "City of Glass ", "create_at": date("2009-08-23"), "description": "Student | Chipotle | MINI | M-E", "followers_count": 219, "friends_count": 317, "statues_count": 12186 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulare, CA", "id": "07f82da44bfd9cb2", "name": "Tulare", "place_type": "city", "bounding_box": rectangle("-119.384597,36.150891 -119.295915,36.240488") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 680644, "cityName": "Tulare" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895085703168, "text": "Mood ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465917974, "name": "AMRIT", "screen_name": "_amriiiit_", "lang": "en", "location": "Albuquerque, NM", "create_at": date("2012-01-16"), "description": "my own biggest fan • IG: amriiiit", "followers_count": 616, "friends_count": 968, "statues_count": 10376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Agua Fria, NM", "id": "01e8175b5409a6c7", "name": "Agua Fria", "place_type": "city", "bounding_box": rectangle("-106.029714,35.640602 -105.985,35.699779") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35049, "countyName": "Santa Fe", "cityID": 3501220, "cityName": "Agua Fria" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895232663552, "text": "Every girl is on the highest level of fuck shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2967261850, "name": "noah", "screen_name": "n_luddy", "lang": "en", "location": "Franklin, IN", "create_at": date("2015-01-07"), "description": "We gon' be alright\nIG: @n_luddy", "followers_count": 276, "friends_count": 232, "statues_count": 11662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, IN", "id": "271351c1a7aca06e", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.096551,39.441909 -86.010161,39.542581") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18081, "countyName": "Johnson", "cityID": 1825450, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895245099010, "text": "What they need to stop doing is putting Original flavor in the oatmeal variety pack..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1532536242, "name": "Justyce", "screen_name": "justycemaylon", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-06-19"), "description": "innovative creator and aspiring broadcast journalist |#HU19", "followers_count": 2328, "friends_count": 2402, "statues_count": 9376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895245238272, "text": "How we already half way through 2016", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1836079142, "name": "Stay Tooned...☺️", "screen_name": "Toonxvx", "lang": "en", "location": "585✈️334", "create_at": date("2013-09-08"), "description": "Sensational ✨", "followers_count": 811, "friends_count": 633, "statues_count": 6754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ozark, AL", "id": "9885d9882b604e32", "name": "Ozark", "place_type": "city", "bounding_box": rectangle("-85.700639,31.377059 -85.597458,31.476766") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1045, "countyName": "Dale", "cityID": 157648, "cityName": "Ozark" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895299653632, "text": "because I'm antisocial & I doubt they feel the same way ���� https://t.co/R0eCARvYzu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2801249185, "name": "Shadow 7/27", "screen_name": "Hypnotic_Halsey", "lang": "en", "location": "Badlands/California ", "create_at": date("2014-09-09"), "description": "helplessly in love with Ashley Nicolette Frangipane & Lauren Michelle Jauregui Morgado", "followers_count": 1092, "friends_count": 981, "statues_count": 25571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895438065664, "text": "It is insane how ignorant and racist people can be. https://t.co/QSRpOpMnlC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1966251697, "name": "Amina Siddiq", "screen_name": "AAminasiddiq", "lang": "en", "location": "null", "create_at": date("2013-10-16"), "description": "19| Afghan| Bay Area - San Diego", "followers_count": 332, "friends_count": 252, "statues_count": 9982 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vista, CA", "id": "2c6666cb9436b81b", "name": "Vista", "place_type": "city", "bounding_box": rectangle("-117.288262,33.131231 -117.191297,33.23861") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 682996, "cityName": "Vista" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895685660672, "text": "Dennys would go right about now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1021318741, "name": "Phillip.", "screen_name": "Chief_Phill_", "lang": "en", "location": "Cleveland, OH", "create_at": date("2012-12-18"), "description": "18 / KentState'20 ⚡️", "followers_count": 587, "friends_count": 472, "statues_count": 9668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895874240514, "text": "Oh lord he's bringing back the chant ������ https://t.co/H1RLugn2ls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327939618, "name": "Jessica Yepes", "screen_name": "JYepes99", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "// Carpe Pulpitum // erhs '17 //", "followers_count": 207, "friends_count": 215, "statues_count": 2649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895958134784, "text": "@CrystalikeC so perfect, so pristine", "in_reply_to_status": 738263824088760321, "in_reply_to_user": 160661241, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 160661241 }}, "user": { "id": 86997068, "name": "Bekalou @ cleaning?", "screen_name": "Beka_Lou", "lang": "en", "location": "CA Bay Area", "create_at": date("2009-11-02"), "description": "NorCal Cosplay Gathering Coordinator", "followers_count": 278, "friends_count": 187, "statues_count": 45075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mountain View, CA", "id": "b19a2cc5134b7e0a", "name": "Mountain View", "place_type": "city", "bounding_box": rectangle("-122.117916,37.356771 -122.044969,37.436935") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 649670, "cityName": "Mountain View" } }
+{ "create_at": datetime("2016-06-02T00:00:03.000Z"), "id": 738263895958245380, "text": "Nah bihhhh lol https://t.co/rZ13SLIJb7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 103102848, "name": "T", "screen_name": "Tisababyo", "lang": "en", "location": "null", "create_at": date("2010-01-08"), "description": "http://tisaxolove.tumblr.com Dental Assistant", "followers_count": 1935, "friends_count": 584, "statues_count": 71155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263897119981568, "text": "Me bc my semester is finally over https://t.co/VdFrYMB4Xf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 331412582, "name": "Christina", "screen_name": "_christinasilva", "lang": "en", "location": "null", "create_at": date("2011-07-07"), "description": "null", "followers_count": 258, "friends_count": 291, "statues_count": 5673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263897182867456, "text": "I really need to go to sleep but my song just came on �� I might be listening to the wrong playlist rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630464496, "name": "Ev", "screen_name": "evega98", "lang": "en", "location": "Grand Prairie, TX", "create_at": date("2012-07-08"), "description": "null", "followers_count": 334, "friends_count": 214, "statues_count": 50 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263897434529792, "text": "Wwwwwtttttfffffffffff.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3236621335, "name": "*", "screen_name": "Jons_180sx", "lang": "en", "location": "University Place, WA", "create_at": date("2015-06-04"), "description": "SC: miataman69", "followers_count": 212, "friends_count": 189, "statues_count": 1367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University Place, WA", "id": "9cd68e26f3c4880c", "name": "University Place", "place_type": "city", "bounding_box": rectangle("-122.58428,47.185193 -122.505193,47.245003") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5373465, "cityName": "University Place" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263897782652928, "text": "The wifi in Glen Mor hasn't been working for the past 2 hours. I guess I'll just call it a night smh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268697891, "name": "Asia Jannae", "screen_name": "_asiaalicious", "lang": "en", "location": "UC Riverside", "create_at": date("2011-03-19"), "description": " IG: asiajannae", "followers_count": 795, "friends_count": 517, "statues_count": 21460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898097229824, "text": "Wind 0.0 mph ---. Barometer 29.891 in, Falling slowly. Temperature 70.7 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8873 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898151780352, "text": "i really did it this time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2701883224, "name": "macie.", "screen_name": "maciehymel", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-07-11"), "description": "invisible:)", "followers_count": 384, "friends_count": 446, "statues_count": 3349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Estelle, LA", "id": "7616c3d1195065d3", "name": "Estelle", "place_type": "city", "bounding_box": rectangle("-90.131291,29.805879 -90.085119,29.868612") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2224390, "cityName": "Estelle" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898365648899, "text": "when I remember the McDonalds on Ustick and eagle is open 24 hours now https://t.co/LY4j6hjMo2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 799290973, "name": "kimby", "screen_name": "KimberUrsenbach", "lang": "en", "location": "Boise, ID", "create_at": date("2012-09-02"), "description": "I like dogs and Cartoon Network. snapchat: ksursenbach #kidrauhl", "followers_count": 315, "friends_count": 114, "statues_count": 5666 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, ID", "id": "afc5c9ce888b958d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-116.461472,43.561187 -116.337922,43.662958") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1652120, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898546184192, "text": "Yup I'm gon' be sad for the next millennium, that means even in the afterlife.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408869965, "name": "Baeless Jr.", "screen_name": "Aguyinachair", "lang": "en", "location": "New London, CT", "create_at": date("2011-11-09"), "description": "I'm not Hispanic but my name is Carlos. Follow me.", "followers_count": 1118, "friends_count": 546, "statues_count": 92583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Groton, CT", "id": "280ad9d1d1fe1d71", "name": "Groton", "place_type": "city", "bounding_box": rectangle("-72.093466,41.312122 -72.018387,41.400455") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 934180, "cityName": "Groton" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898629935104, "text": "If the shoe fits", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2483064259, "name": "VICO", "screen_name": "_victoreos", "lang": "en", "location": "El Paso, TX", "create_at": date("2014-05-07"), "description": "strength and guidance", "followers_count": 438, "friends_count": 748, "statues_count": 2042 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263898701189121, "text": "I hope you all realize which of your friends are losers in actual life now that highschool is over, just wait till this September.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 531776689, "name": "Adisson Feagley", "screen_name": "AdissonFeagley", "lang": "en", "location": "Indiana ✈️ Anchorage ", "create_at": date("2012-03-20"), "description": "I can't help but to laugh.", "followers_count": 1290, "friends_count": 734, "statues_count": 42074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899045167104, "text": "The worst thing is when YOU'RE the side hoe and you didn't even know ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2698818764, "name": "Andy Garcia", "screen_name": "AndyGarcia212", "lang": "en", "location": "Texas/NYC", "create_at": date("2014-08-01"), "description": "I'm not perfect, thoughtful, or witty. But I'm honest. Columbia '17. ig + sc: andygarcia212", "followers_count": 540, "friends_count": 198, "statues_count": 18026 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pharr, TX", "id": "36b9518ae4e9e210", "name": "Pharr", "place_type": "city", "bounding_box": rectangle("-98.220006,26.085485 -98.15929,26.250324") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4857200, "cityName": "Pharr" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899061944321, "text": "@SavageTaye lmfaooooooo", "in_reply_to_status": 738263847698472961, "in_reply_to_user": 4294360093, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4294360093 }}, "user": { "id": 4261243094, "name": "portia", "screen_name": "modernalmight", "lang": "en", "location": "HTX", "create_at": date("2015-11-16"), "description": "SC : modernalmight.", "followers_count": 1785, "friends_count": 662, "statues_count": 30363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atascocita, TX", "id": "38d0e797745f4c5d", "name": "Atascocita", "place_type": "city", "bounding_box": rectangle("-95.249792,29.912242 -95.135568,30.033094") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4804462, "cityName": "Atascocita" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899166760960, "text": "���� https://t.co/BwhoxPsF3d", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 615869147, "name": "brady ++", "screen_name": "luxyohoe", "lang": "en", "location": "Sam Houston State Univeristy ", "create_at": date("2012-06-22"), "description": "null", "followers_count": 1429, "friends_count": 750, "statues_count": 17091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899263340544, "text": "������ https://t.co/n5kxurBmFw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3020867428, "name": "Harp$❗️⛽️", "screen_name": "Screamin_FLOCKA", "lang": "en", "location": "The Bull", "create_at": date("2015-02-05"), "description": "R.I.P. Ty || R.I.P. Breon || R.I.P. Corbin || #DukeNation || #HeatNation || IG: @__harps_ ||", "followers_count": 1015, "friends_count": 706, "statues_count": 22905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899309379586, "text": "nope.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2718141732, "name": "buttercup ✨", "screen_name": "oddxtoriaaa", "lang": "en", "location": "lex // yhi ♥️", "create_at": date("2014-08-08"), "description": "snapchat : oddxtoriaaa // #DamønGang", "followers_count": 572, "friends_count": 632, "statues_count": 32573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mead Valley, CA", "id": "015ebe02600b7c50", "name": "Mead Valley", "place_type": "city", "bounding_box": rectangle("-117.331341,33.799508 -117.233002,33.866226") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 646646, "cityName": "Mead Valley" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899858829313, "text": "GNC trip later on today", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 369547400, "name": "aaron jacob nebel", "screen_name": "aaronjacobnebel", "lang": "en", "location": "null", "create_at": date("2011-09-07"), "description": "null", "followers_count": 211, "friends_count": 472, "statues_count": 23022 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dyer, IN", "id": "6e6c127b5f56d897", "name": "Dyer", "place_type": "city", "bounding_box": rectangle("-87.525635,41.471753 -87.486357,41.523145") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18089, "countyName": "Lake", "cityID": 1819270, "cityName": "Dyer" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899892420609, "text": "#subtweet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "subtweet" }}, "user": { "id": 560642686, "name": "Cristinaaa✨", "screen_name": "ceelalamacz", "lang": "en", "location": "bay area", "create_at": date("2012-04-22"), "description": "St.Mary's College of CA '20", "followers_count": 226, "friends_count": 97, "statues_count": 11806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Daly City, CA", "id": "6a71821001635bbd", "name": "Daly City", "place_type": "city", "bounding_box": rectangle("-122.500164,37.649122 -122.405233,37.708437") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 617918, "cityName": "Daly City" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899946946560, "text": "Things I do at 2 am: super clean my room for 2 hours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2770221811, "name": "Olivia", "screen_name": "liviesprague", "lang": "en", "location": "omaha nebraska ", "create_at": date("2014-08-26"), "description": "westside high school •Simply Irresistible•", "followers_count": 154, "friends_count": 251, "statues_count": 2437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263899972239360, "text": "Name one song other than Daydreamin' ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2353436028, "name": "peaches da plug", "screen_name": "JustKhristyn", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-02-20"), "description": "Allan's. But y'all knew. Im not friendly. WHS 16'", "followers_count": 1069, "friends_count": 495, "statues_count": 66810 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900257292288, "text": "I'll just wait until the middle of the night when he wakes up and text me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 393655931, "name": "Key w| TheShortLegs✨", "screen_name": "_EShmurdaa", "lang": "en", "location": "null", "create_at": date("2011-10-18"), "description": "C/O 2k16 | ❤ | Alabama State Here I Come", "followers_count": 855, "friends_count": 1003, "statues_count": 26197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University City, MO", "id": "2dd9882dad5ba631", "name": "University City", "place_type": "city", "bounding_box": rectangle("-90.366578,38.645912 -90.297186,38.683509") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2975220, "cityName": "University City" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900303425537, "text": "if I don't get Krispy Kreme in the next 24 hours we're gonna have a problem", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2500073984, "name": "bryan", "screen_name": "BryanArdron", "lang": "en", "location": "liberal", "create_at": date("2014-05-16"), "description": "problematic", "followers_count": 299, "friends_count": 262, "statues_count": 5735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900525715456, "text": "@henrykondo_ @Its_Romeroo22 wait you guys are still in hs?! You're so big that I forget sometimes ��", "in_reply_to_status": 738263435670999044, "in_reply_to_user": 1652623244, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1652623244, 2235077966 }}, "user": { "id": 294769945, "name": "Ty", "screen_name": "_TyTy94_", "lang": "en", "location": "Santa Clarita, CA.", "create_at": date("2011-05-07"), "description": "Frequent user of sarcasm. Some people laugh at it; most people don't.", "followers_count": 180, "friends_count": 201, "statues_count": 10992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900534116352, "text": "Saucy flossy keep things bossy �� https://t.co/YWr4Gk3nXb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28879076, "name": "emma ❋", "screen_name": "emmajsmoot", "lang": "en", "location": "•Bay Area•", "create_at": date("2009-04-04"), "description": "catch me at ya man's house eating nachos || #FCB", "followers_count": 489, "friends_count": 222, "statues_count": 13136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900806774785, "text": "& I know you still think about the times we had", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 924470162, "name": "Lyss", "screen_name": "xoxolysssaa", "lang": "en", "location": "null", "create_at": date("2012-11-03"), "description": "nhs varsity cheer", "followers_count": 888, "friends_count": 414, "statues_count": 35772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263900869775364, "text": "so i guess I'm fake for not liking the bay at first hahahaha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1007563220, "name": "olivia pope. ॐ", "screen_name": "__amerikaaa", "lang": "en", "location": "from the westside, with love", "create_at": date("2012-12-12"), "description": "that'll teach em", "followers_count": 658, "friends_count": 462, "statues_count": 44790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-02T00:00:04.000Z"), "id": 738263901171650560, "text": "6/2/2016 - 02:00\nTemp: 69.0F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.961in. & Falling\nRain: 0.01 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 54, "statues_count": 53408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-06-02T00:00:05.000Z"), "id": 738263901360377856, "text": "Everything is becoming so real, it's scary & exciting at the same time", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 587823226, "name": "Aly", "screen_name": "alysonleaa", "lang": "en", "location": "Oregon // California", "create_at": date("2012-05-22"), "description": "20 || california || my ambition is my weaponry ✨ #NBK", "followers_count": 1189, "friends_count": 793, "statues_count": 34463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626268300607488, "text": "I miss the days where my friends and I would chill in the porch in front of my parents house literally every night", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22429592, "name": "Jesse Rodriguez", "screen_name": "jessebrown09", "lang": "en", "location": "Ontario, CA", "create_at": date("2009-03-01"), "description": "Est. 1993 | IE - SoCal | CSUSB | EDM | PLUR | Disney | Instagram: jbrown011 | 4.16.2014❤️", "followers_count": 810, "friends_count": 745, "statues_count": 11894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626268334170112, "text": "Mmmm ! �� https://t.co/pWVvdtLCNl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2161981890, "name": "ひNB RAMGLOひ", "screen_name": "_12Rounds_", "lang": "en", "location": "Houston, TX", "create_at": date("2013-10-28"), "description": "NBMB50 // Alan", "followers_count": 235, "friends_count": 185, "statues_count": 1259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arcola, TX", "id": "0113368d54ff7167", "name": "Arcola", "place_type": "city", "bounding_box": rectangle("-95.49306,29.450754 -95.293716,29.518565") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4803708, "cityName": "Arcola" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626268774563840, "text": "The fact that it's 1am and I'm still up is sad...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4711290626, "name": "Mäché♏️", "screen_name": "mache_amanda1", "lang": "en", "location": "Clearfield, UT", "create_at": date("2016-01-04"), "description": "For the things which are seen are temporal; but the things which are not seen are eternal. 2 Corinthians 4:18", "followers_count": 119, "friends_count": 94, "statues_count": 1244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, UT", "id": "d254694c82a4473e", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-112.05574,41.074662 -111.979637,41.126319") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4913850, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626268862676992, "text": "@ParisPasian �� stop tweeting about some guy!!", "in_reply_to_status": 738625966025510913, "in_reply_to_user": 231946469, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 231946469 }}, "user": { "id": 564137045, "name": "Quill Mundo", "screen_name": "QuillDB", "lang": "en", "location": "null", "create_at": date("2012-04-26"), "description": "Football is life... Southern Arkansas University #MuleRiders", "followers_count": 868, "friends_count": 855, "statues_count": 19188 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626268908900352, "text": "Banger https://t.co/qO5enuyuqa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 626183670, "name": "Jack Dawson", "screen_name": "SkylerDillehay", "lang": "en", "location": "null", "create_at": date("2012-07-03"), "description": "Aspiring Calvin Klein underwear model but in the meantime I'm at OU", "followers_count": 689, "friends_count": 367, "statues_count": 21815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Philadelphia, OH", "id": "3d415689911bfb32", "name": "New Philadelphia", "place_type": "city", "bounding_box": rectangle("-81.476738,40.446607 -81.390256,40.51686") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39157, "countyName": "Tuscarawas", "cityID": 3955216, "cityName": "New Philadelphia" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626269005242368, "text": "@skatemaloley https://t.co/Xdn81vGrgo", "in_reply_to_status": -1, "in_reply_to_user": 1556393042, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1556393042 }}, "user": { "id": 2718666768, "name": "MORG$", "screen_name": "danktez", "lang": "en", "location": "denver", "create_at": date("2014-08-08"), "description": "weird white kid | lex", "followers_count": 6508, "friends_count": 121, "statues_count": 72957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CO", "id": "3f871475c095f94f", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-105.148925,39.819133 -104.987771,39.97944") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 883835, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626269206577152, "text": "somebody call me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584777949, "name": "MJL ❣", "screen_name": "mylajalaeciaa", "lang": "en", "location": "null", "create_at": date("2014-06-05"), "description": "null", "followers_count": 486, "friends_count": 281, "statues_count": 11060 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belle Rose, LA", "id": "01d4ddc85aae0846", "name": "Belle Rose", "place_type": "city", "bounding_box": rectangle("-91.063235,29.999743 -91.02602,30.08713") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22007, "countyName": "Assumption", "cityID": 2206260, "cityName": "Belle Rose" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626269856698368, "text": "@whyangiewhy @jackdpmjd @Pantherss88 Far more often then you think", "in_reply_to_status": 738517552175841280, "in_reply_to_user": 281805524, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 281805524, 300878760, 256141768 }}, "user": { "id": 3087447410, "name": "John Mockridge", "screen_name": "MyZoreck2", "lang": "en", "location": "Florida, USA", "create_at": date("2015-03-15"), "description": "Age 64 Disabled. Computers, Hockey,Sports in general, being quiet, nature all things simple, learning. peace and quiet and open air, music, the theater, acting", "followers_count": 82, "friends_count": 106, "statues_count": 9785 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stuart, FL", "id": "010ebc674459153c", "name": "Stuart", "place_type": "city", "bounding_box": rectangle("-80.274706,27.103238 -80.196472,27.205663") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12085, "countyName": "Martin", "cityID": 1268875, "cityName": "Stuart" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626270242607105, "text": "@bryancovarubias Cabrillo ��", "in_reply_to_status": 738618989559963650, "in_reply_to_user": 1966064450, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 1966064450 }}, "user": { "id": 85934257, "name": "Jocelyn Biyu Teran", "screen_name": "Jbiyu_dude", "lang": "en", "location": "Yaaay Areaaa ", "create_at": date("2009-10-28"), "description": "if I said", "followers_count": 561, "friends_count": 498, "statues_count": 21447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prunedale, CA", "id": "9723b83889e41047", "name": "Prunedale", "place_type": "city", "bounding_box": rectangle("-121.743662,36.749101 -121.619853,36.861049") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 658870, "cityName": "Prunedale" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626270293053441, "text": "No", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3051533959, "name": "paco", "screen_name": "JosiahMamea_", "lang": "en", "location": "15 down, forever to go ❤️", "create_at": date("2015-02-28"), "description": "@talssuson_ is my baby ❤️", "followers_count": 174, "friends_count": 134, "statues_count": 1944 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626270766845952, "text": "Johnny Rockets ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599765992, "name": "sergio", "screen_name": "xxs3rgioxx", "lang": "en", "location": "Inglewood", "create_at": date("2013-07-16"), "description": "null", "followers_count": 222, "friends_count": 199, "statues_count": 8760 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inglewood, CA", "id": "97e336e1e3ed9e1e", "name": "Inglewood", "place_type": "city", "bounding_box": rectangle("-118.378887,33.925317 -118.313418,33.98195") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 636546, "cityName": "Inglewood" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626270787829760, "text": "@ebbtideapp Tide in Lower Bryant Landing, Alabama 06/03/2016\nHigh 1:03pm 1.8\n Low 12:00am -0.3\nHigh 1:44pm 2.0\n Low 12:38am -0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-87.8733,30.9783"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 100, "friends_count": 1, "statues_count": 38185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1003, "countyName": "Baldwin" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626270871752704, "text": "I wish I was sleeping honestly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3027799039, "name": "Lo Ca$h", "screen_name": "GLo21__", "lang": "en", "location": "Probs third wheelin Alex &Gigi", "create_at": date("2015-02-18"), "description": "•@mmorales_15❤️ • ||#RIP Mannie, ama camil, mom Holzer❤️|| mucho love for @marielaaaag ♊️ #ThunderUp", "followers_count": 903, "friends_count": 741, "statues_count": 14096 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasco, WA", "id": "41647560efe1db77", "name": "Pasco", "place_type": "city", "bounding_box": rectangle("-119.230869,46.198319 -119.033335,46.292793") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53021, "countyName": "Franklin", "cityID": 5353545, "cityName": "Pasco" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271211487234, "text": "why is my card denied 24/7 i still got like $1.90 in there!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 541979976, "name": "a", "screen_name": "ameliaaahain", "lang": "en", "location": "Sacramento, CA", "create_at": date("2012-03-31"), "description": "null", "followers_count": 435, "friends_count": 564, "statues_count": 7597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271265988609, "text": "meh https://t.co/JQER97569R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3298574195, "name": "Angel(ee-nuh)", "screen_name": "iloveangelenah", "lang": "en", "location": "being black, bold & magical", "create_at": date("2015-05-25"), "description": "BLACK EXCELLENCE", "followers_count": 1258, "friends_count": 608, "statues_count": 16972 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271458918404, "text": "I think I just got the blue light of death on my PS4 :(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 90119892, "name": "Ryan", "screen_name": "Hipster_Me", "lang": "en", "location": "Texas ", "create_at": date("2009-11-14"), "description": "Geek/Nerd/Photographer/ find me on INSTAGRAM: @ryanpollackphoto \nOwner & Creative Consultant of Ryan Pollack Photography\n@_Mariah_Rae_ is my loving wife.", "followers_count": 427, "friends_count": 371, "statues_count": 9580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271744139268, "text": "About to go MIA on many people ✌��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4274439852, "name": "Leonela Rendon", "screen_name": "gueraaa16", "lang": "en", "location": "null", "create_at": date("2015-11-17"), "description": "mas vale sola que mal acompañada", "followers_count": 225, "friends_count": 285, "statues_count": 1485 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271823810562, "text": "I miss my Kyoga wolfy, looking forward to getting payed so I can continue the story. https://t.co/aHmIFE7dyt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 67734575, "name": "Betsy Harrigan", "screen_name": "mindfreakeuropa", "lang": "en", "location": "Upland, CA", "create_at": date("2009-08-21"), "description": "Facebook:http://facebook.com/betsy.harrigan Instagram:mindfreakeuropa\r\nEmail:mindfreakeuropa@gmail.com", "followers_count": 301, "friends_count": 278, "statues_count": 12728 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626271991631872, "text": "Those hand tattoos are one of the most bad ass I've ever seen. https://t.co/02bj87z0ED", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379960983, "name": "chapstick Mullens", "screen_name": "juustuswolstein", "lang": "en", "location": "venice betch, ca", "create_at": date("2011-09-25"), "description": "HM 2 TAA ; everything in between ✨✨ brothers in arms AKA the #RoyalFam #teammiley luvvvv my baby @danny_noel91", "followers_count": 587, "friends_count": 946, "statues_count": 20108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626272004214784, "text": "We Ain't Gone Play With Em' Though ��.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1682288028, "name": "ㅤㅤㅤ ㅤㅤㅤ ㅤㅤㅤ ㅤㅤㅤ ㅤㅤㅤ", "screen_name": "TreLucci", "lang": "en", "location": "turnt up , tx", "create_at": date("2013-08-18"), "description": "trelucci2x@gmail.com", "followers_count": 14527, "friends_count": 135, "statues_count": 16163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Hill, TX", "id": "4233ec320eb73634", "name": "Cedar Hill", "place_type": "city", "bounding_box": rectangle("-97.028288,32.541286 -96.891039,32.649242") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4813492, "cityName": "Cedar Hill" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626272029380609, "text": "It baffles me how many people on here overreact to one game I guess not everyone is intelligent https://t.co/OS2kNt29AM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 701574390, "name": "DET25-28", "screen_name": "saints10fan", "lang": "en", "location": "Ann Arbor ✈️ Tucson ✈️ Redding", "create_at": date("2012-07-17"), "description": "Michigan,Tigers and Saints Youtube TTC. Thunder. Brady fan. I love to discuss sports. #freebrady. don't take all my tweets seriously I like to have fun.", "followers_count": 1660, "friends_count": 2020, "statues_count": 56098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626272209686528, "text": "#Karaoke song: Longview - Green Day @karaokesandman @Kseaswinghouse", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Karaoke" }}, "user_mentions": {{ 190840481 }}, "user": { "id": 190840481, "name": "Karaoke With Sandman", "screen_name": "karaokesandman", "lang": "en", "location": "Ski country Colorado", "create_at": date("2010-09-14"), "description": "null", "followers_count": 56, "friends_count": 44, "statues_count": 21963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-03T00:00:00.000Z"), "id": 738626272234852354, "text": "Coo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1434292784, "name": "Jordan", "screen_name": "masteerjaay", "lang": "en", "location": "null", "create_at": date("2013-05-16"), "description": "LEGACY...", "followers_count": 699, "friends_count": 433, "statues_count": 4490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626272385859584, "text": "Good girls gotta get down with the gangsters", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555610963, "name": "mica annis ✊✌", "screen_name": "micacheww", "lang": "en", "location": "Glencoe ", "create_at": date("2012-04-16"), "description": "got here by running my mouth -Ke$ha", "followers_count": 1030, "friends_count": 836, "statues_count": 22236 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forest Grove, OR", "id": "beedf9d8e2499b64", "name": "Forest Grove", "place_type": "city", "bounding_box": rectangle("-123.15354,45.501953 -123.070256,45.542318") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4126200, "cityName": "Forest Grove" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626272545280000, "text": "What or who is this ?? https://t.co/iPvUCZX5Mc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3169195573, "name": "fine&fly", "screen_name": "slaekae", "lang": "en", "location": "saucy", "create_at": date("2015-04-23"), "description": "slaying these bitches my hobby .", "followers_count": 2518, "friends_count": 1376, "statues_count": 44521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Margate, FL", "id": "54b6acb13620f5e2", "name": "Margate", "place_type": "city", "bounding_box": rectangle("-80.23409,26.210625 -80.190415,26.274246") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1243125, "cityName": "Margate" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626272948039680, "text": "06/03@03:00 - Temp 68.5F, WC 68.5F. Wind 1.4mph ENE, Gust 3.0mph. Bar 30.043in, Falling slowly. Rain 0.00in. Hum 94%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626272952127488, "text": "YOOO FRANK WYA?!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 1576849057, "name": "O.G. V", "screen_name": "vvvejar", "lang": "en", "location": "socal", "create_at": date("2013-07-07"), "description": "hey, what's good?", "followers_count": 372, "friends_count": 776, "statues_count": 1584 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yorba Linda, CA", "id": "26595dca9d293375", "name": "Yorba Linda", "place_type": "city", "bounding_box": rectangle("-117.846211,33.862324 -117.731799,33.918525") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 686832, "cityName": "Yorba Linda" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273124061185, "text": "i dislike bubblegum-flavored anything", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 191619042, "name": "audreyana", "screen_name": "audreyana13", "lang": "en", "location": "Arizona", "create_at": date("2010-09-16"), "description": "probably at Dutch Bros / snapchat: audreyanaa", "followers_count": 1881, "friends_count": 864, "statues_count": 5771 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273291829248, "text": "@_JaayM_ a year the next cod will be done almost u gonna skip 2 Cod's I'm a role", "in_reply_to_status": 738626046228996096, "in_reply_to_user": 4518736955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4518736955 }}, "user": { "id": 2799241873, "name": "Dylan~Betwixts", "screen_name": "PxDylan", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "CoD Player Ps4 Gt: Betwixts\n(Duo:?????) F/A Play everyday basically Strive to Win http://twitch.tv/betwixts YT: InVerse Remix", "followers_count": 191, "friends_count": 84, "statues_count": 1782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camdenton, MO", "id": "069729707792e5cb", "name": "Camdenton", "place_type": "city", "bounding_box": rectangle("-92.787695,37.988033 -92.732089,38.019405") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29029, "countyName": "Camden", "cityID": 2910810, "cityName": "Camdenton" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273333776384, "text": "@Harold_Tha_III \n- Ms. Crawford's class \n- PH �� \n- Bomedy ��", "in_reply_to_status": -1, "in_reply_to_user": 1146588217, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1146588217 }}, "user": { "id": 62453489, "name": "Don Canja", "screen_name": "DonCanja2", "lang": "en", "location": "San Diego, CA ✌️", "create_at": date("2009-08-02"), "description": "U miss the shots u don't take", "followers_count": 651, "friends_count": 265, "statues_count": 19609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273451200512, "text": "No ones gonna love you than I do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4113891131, "name": "Drworm", "screen_name": "Returnoftheworm", "lang": "en", "location": "wormhole", "create_at": date("2015-11-04"), "description": "return of the worm (oh my god)", "followers_count": 104, "friends_count": 99, "statues_count": 1549 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wading River, NY", "id": "30839b0d8be27742", "name": "Wading River", "place_type": "city", "bounding_box": rectangle("-72.863791,40.914625 -72.798577,40.970336") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3677772, "cityName": "Wading River" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273753333760, "text": "Temp 58.6° Hi/Lo 58.8/58.4 Rng 0.4° WC 58.6° Hmd 88% Rain 0.00\" Storm 0.00\" BAR 30.118 Falling DP 55.1° Wnd 0mph Dir --- Gst 5mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 72, "friends_count": 123, "statues_count": 19203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273761624064, "text": "Why Kiana so mean?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3279221294, "name": "Deleon", "screen_name": "vlone300", "lang": "en", "location": "sc: gbe_papi", "create_at": date("2015-07-13"), "description": "young handsome jiggy nigga #AWGE #LIGHTSKIN", "followers_count": 1111, "friends_count": 592, "statues_count": 23846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626273774174209, "text": "tell me wHY IM NOT JESSICA'S NUMBER ONE BEST FRIEND ON SNAPCHAT ANYMORE WYD SNAPCHAT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2913152544, "name": "yzella", "screen_name": "yzellaxfayee", "lang": "en", "location": "xxviii | james", "create_at": date("2014-11-28"), "description": "probably eating & watching friends", "followers_count": 531, "friends_count": 360, "statues_count": 7256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Colorado Springs, CO", "id": "adc95f2911133646", "name": "Colorado Springs", "place_type": "city", "bounding_box": rectangle("-104.910562,38.741142 -104.668092,39.035895") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8041, "countyName": "El Paso", "cityID": 816000, "cityName": "Colorado Springs" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274046791682, "text": "I'm not gonna leave you now\nOh, I know it's gon' get better\nI'm not gonna leave you now\nOh, I know it's gon' get better", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46157095, "name": "Samara", "screen_name": "noele92", "lang": "en", "location": "Fort Lauderdale, FL", "create_at": date("2009-06-10"), "description": "Hey I'm Samara, Brazilian,I love Dapper Daniel♥ The Wanted, Midnight RED, Phase Five, Israel Novaes. #ONELOVE", "followers_count": 711, "friends_count": 593, "statues_count": 5466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Davie, FL", "id": "34031d61ef79585f", "name": "Davie", "place_type": "city", "bounding_box": rectangle("-80.369507,26.029537 -80.200871,26.126804") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1216475, "cityName": "Davie" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274067894272, "text": "Wind 0.0 mph ---. Barometer 30.010 in, Steady. Temperature 67.1 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274248302592, "text": "@politico @BernieSanders Bernie still working hard https://t.co/gwFUuXAhc8", "in_reply_to_status": 738625820361646080, "in_reply_to_user": 9300262, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 9300262, 216776631 }}, "user": { "id": 2991837901, "name": "EndZoneBlog", "screen_name": "Endzoneblog", "lang": "en", "location": "null", "create_at": date("2015-01-21"), "description": "Official twitter of http://endzoneblog.com Background photo credit: Phil Dolby\nFounded 2015 in Michigan", "followers_count": 966, "friends_count": 1478, "statues_count": 78087 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Adrian, MI", "id": "4286bea8be81e904", "name": "Adrian", "place_type": "city", "bounding_box": rectangle("-84.094415,41.860395 -84.004134,41.941268") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26091, "countyName": "Lenawee", "cityID": 2600440, "cityName": "Adrian" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274252468227, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 73.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274415894528, "text": "Time to start a genuine and honest relationship https://t.co/VnCf5tzZMq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35507124, "name": "Drewbert", "screen_name": "ReverseDrew", "lang": "en", "location": "null", "create_at": date("2009-04-26"), "description": "Stage Left guitar for @ReverseOrder and I've earned no less than 3 participation trophies in my lifetime.", "followers_count": 27948, "friends_count": 34, "statues_count": 16557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626274759835652, "text": "@avedashsoul @Riq_DaPrince ��������", "in_reply_to_status": 738625501711851520, "in_reply_to_user": 159524929, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 159524929, 2750173583 }}, "user": { "id": 204542828, "name": "Ríco Suavé", "screen_name": "Roddyy_4", "lang": "en", "location": "Dallas, TX", "create_at": date("2010-10-18"), "description": "|RIP Chandlor| Young Bill Gates in the making. Computer lover and entrepreneur. Business Inquiries: roderickcoleman4@yahoo.com", "followers_count": 2951, "friends_count": 975, "statues_count": 12522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duncanville, TX", "id": "c9ff03f5c5cb510a", "name": "Duncanville", "place_type": "city", "bounding_box": rectangle("-96.943349,32.617554 -96.882757,32.676694") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4821628, "cityName": "Duncanville" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626275015823360, "text": "Happy birthday ho ���� @WTF_melissaaa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177343659 }}, "user": { "id": 2419839758, "name": "ellyfromdablock", "screen_name": "ElizabethVidrio", "lang": "en", "location": "maserati", "create_at": date("2014-03-30"), "description": "LTD. No exaggeration necessary.", "followers_count": 55, "friends_count": 45, "statues_count": 7076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626275460317188, "text": "@_iMichaell \"will you just stop playing with my feelings and be wife?\" Didn't you say that? �� don't flex on me now", "in_reply_to_status": 738626102403444736, "in_reply_to_user": 2770284453, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2770284453 }}, "user": { "id": 1472577744, "name": "Bre", "screen_name": "BreannaPipes", "lang": "en", "location": "Louisville, KY", "create_at": date("2013-05-31"), "description": "toddie is my gf", "followers_count": 691, "friends_count": 308, "statues_count": 28335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jeffersonville, IN", "id": "6c0e077597395926", "name": "Jeffersonville", "place_type": "city", "bounding_box": rectangle("-85.75745,38.267538 -85.638925,38.402733") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18019, "countyName": "Clark", "cityID": 1838358, "cityName": "Jeffersonville" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626275737272320, "text": "Wind 0.0 mph SW. Barometer 30.022 in, Falling slowly. Temperature 66.6 °F. Rain today 0.01 in. Humidity 100%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20292 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276076990465, "text": "Temp: 72.1°F Wind:0.0mph Pressure: 30.000hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276143988736, "text": "I'm on some 3 shades of Sanchez lol ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519098160, "name": "Patrick Star", "screen_name": "snchz_pepe54", "lang": "en", "location": "null", "create_at": date("2012-03-08"), "description": "Living under the sea in a rock", "followers_count": 340, "friends_count": 434, "statues_count": 24190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276181708800, "text": "������❤️�� https://t.co/9Z8Tm3N4ct", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.04924,37.682432"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2935863266, "name": "Natnat", "screen_name": "budlightningg", "lang": "en", "location": "null", "create_at": date("2014-12-19"), "description": "❤️SeyarrrSayedddd❤️", "followers_count": 306, "friends_count": 389, "statues_count": 1907 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Red Lobster", "id": "07d9d2297b885002", "name": "Red Lobster", "place_type": "poi", "bounding_box": rectangle("-121.04924009999999,37.6824319 -121.04924,37.682432") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276228005888, "text": "https://t.co/X38oqGcQxg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 842804557, "name": "Lil Fendi Gossip", "screen_name": "InternetEli", "lang": "en", "location": "Internet", "create_at": date("2012-09-23"), "description": "Multimedia Artist | WTA Founder", "followers_count": 443, "friends_count": 210, "statues_count": 3003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hartford, CT", "id": "485ebc6dbebdbf32", "name": "West Hartford", "place_type": "city", "bounding_box": rectangle("-72.786564,41.717959 -72.713899,41.806675") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 982660, "cityName": "West Hartford" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276353675264, "text": "@leelejjena24 @dae_kal @tklomon FAWWWK outta here! Looking like the Statue of Liberty https://t.co/JPEDVqzxP2", "in_reply_to_status": 738625892855906304, "in_reply_to_user": 396273075, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 396273075, 892451575, 2348823258 }}, "user": { "id": 1107123966, "name": "$id", "screen_name": "sidratriplett", "lang": "en", "location": "null", "create_at": date("2013-01-20"), "description": "Newberry College '20~ You see the $igns? ftx", "followers_count": 882, "friends_count": 1006, "statues_count": 1587 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-03T00:00:01.000Z"), "id": 738626276479504384, "text": "No explanations I cut you off I'll stay in my lane", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3677706973, "name": "DM", "screen_name": "dmvrtinez4", "lang": "en", "location": "Richmond, CA", "create_at": date("2015-09-24"), "description": "don't stop even if the hustle don't knock", "followers_count": 40, "friends_count": 41, "statues_count": 2256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626277087666176, "text": "Reunited w @brit_luvv @Smaallssz_ n it feels so good", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 152508706, 2516411767 }}, "user": { "id": 3319715779, "name": "ЯAYCHEL", "screen_name": "RachelBoo92", "lang": "en", "location": "null", "create_at": date("2015-08-18"), "description": "✨✨✨✨", "followers_count": 160, "friends_count": 88, "statues_count": 2891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626277221892097, "text": "Holy cow! @djsavi killing the game right now @OmniaSanDiego https://t.co/1t0dvyInm7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150895820, 2678161076 }}, "user": { "id": 3668283672, "name": "Italia Acevedo", "screen_name": "italiaduh", "lang": "en", "location": "San Diego, CA", "create_at": date("2015-09-24"), "description": "San Diego based nightlife promoter. Guestlist, bottle service, or private corporate events. Italia@tksandiego.com", "followers_count": 66, "friends_count": 236, "statues_count": 547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626277788131329, "text": "@Paarthyyy I ain't good yo", "in_reply_to_status": 738626000741797888, "in_reply_to_user": 594234227, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 594234227 }}, "user": { "id": 319375154, "name": "mase.", "screen_name": "OGMasedgod", "lang": "en", "location": "Lemoore Station, CA", "create_at": date("2011-06-17"), "description": "FA20GVNG///I fix jets for the Navy", "followers_count": 400, "friends_count": 412, "statues_count": 14363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lemoore Station, CA", "id": "8f1c4baa8dd21e18", "name": "Lemoore Station", "place_type": "city", "bounding_box": rectangle("-119.914373,36.25523 -119.869486,36.270499") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 641166, "cityName": "Lemoore Station" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278018818048, "text": "@monalerio @traviswieman why... Just why", "in_reply_to_status": 738622896608661506, "in_reply_to_user": 1600575031, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1600575031, 1289353134 }}, "user": { "id": 103191938, "name": "jazelq", "screen_name": "JazelQuinto", "lang": "en", "location": "null", "create_at": date("2010-01-08"), "description": "null", "followers_count": 555, "friends_count": 194, "statues_count": 17078 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, CA", "id": "a2c84129f9dcf69f", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-117.230172,33.090761 -117.103461,33.186722") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 668196, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278060740609, "text": "@jasmineuchme @Rad_Edgar @Efrenito_ are u judging us bc were emo", "in_reply_to_status": 738626055380992000, "in_reply_to_user": 2464972615, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2464972615, 1099517964, 431685060 }}, "user": { "id": 727019231607328768, "name": "Jordynn R.", "screen_name": "BroccoliHeadJor", "lang": "en", "location": "null", "create_at": date("2016-05-01"), "description": "null", "followers_count": 184, "friends_count": 195, "statues_count": 2232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278064955392, "text": "Loved every minute. https://t.co/pH0iY7NT2B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2780874870, "name": "Christopher Perkins", "screen_name": "ChrisPerkinsDnD", "lang": "en", "location": "Seattle, WA", "create_at": date("2014-08-30"), "description": "D&D storyteller, Dungeon Master, and Wizard who can't cast any spells. My familiar is a Chiweenie named Milo. Have a magical day!", "followers_count": 21038, "friends_count": 1114, "statues_count": 4407 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "b0b79caecef59fb5", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-118.37032,34.14263 -118.27997,34.22169") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278379556864, "text": "Party tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 452641928, "name": "nobody", "screen_name": "oveRAYted", "lang": "en", "location": "null", "create_at": date("2012-01-01"), "description": "Not afraid to be different", "followers_count": 182, "friends_count": 161, "statues_count": 6425 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278715056132, "text": "the cutest thing the world has ever been gifted https://t.co/mCF7t1VSqh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2837453646, "name": "Cami", "screen_name": "CamiiUzumaki", "lang": "en", "location": "deep", "create_at": date("2014-10-01"), "description": "null", "followers_count": 1023, "friends_count": 547, "statues_count": 17751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Rosa, CA", "id": "5a9de3ff3fdd849d", "name": "Santa Rosa", "place_type": "city", "bounding_box": rectangle("-122.779239,38.396779 -122.572978,38.507656") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 670098, "cityName": "Santa Rosa" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278715097088, "text": "I've made it a point to learn to play every song on guitar hero 3 and I'm like 80% of the way there so that's me being productive", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 91012695, "name": "Brendan McMullen", "screen_name": "bemcmullen", "lang": "en", "location": "Orange County, CA", "create_at": date("2009-11-18"), "description": "ASU ΣΤΓ // Immortal", "followers_count": 397, "friends_count": 368, "statues_count": 3690 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Santa Margarita, CA", "id": "728359fcdfad2b43", "name": "Rancho Santa Margarita", "place_type": "city", "bounding_box": rectangle("-117.638013,33.585875 -117.553442,33.672953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 659587, "cityName": "Rancho Santa Margarita" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278715199488, "text": "@karrifookie lol thank you", "in_reply_to_status": 738625894923698176, "in_reply_to_user": 1325577980, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1325577980 }}, "user": { "id": 2540608962, "name": "courtney", "screen_name": "xx_porsha_xx", "lang": "en", "location": "Dallas, TX", "create_at": date("2014-06-01"), "description": "amosc: c.boojie", "followers_count": 585, "friends_count": 437, "statues_count": 6714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waxahachie, TX", "id": "45ff3ab084a1f419", "name": "Waxahachie", "place_type": "city", "bounding_box": rectangle("-96.880016,32.316564 -96.79913,32.494604") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4876816, "cityName": "Waxahachie" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626278845120513, "text": "ok but Evan thinks he's so funny ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043430642, "name": "sevx", "screen_name": "bsevxo", "lang": "en", "location": "From The West Side With Love", "create_at": date("2012-12-28"), "description": "null", "followers_count": 612, "friends_count": 408, "statues_count": 49796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Whittier, CA", "id": "1af9758d2214d4b8", "name": "South Whittier", "place_type": "city", "bounding_box": rectangle("-118.05523,33.917027 -118.000561,33.955279") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673430, "cityName": "South Whittier" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279130333184, "text": "I'm about to google fat people doing spartan race so I can feel better about finishing it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 475314162, "name": "Monica", "screen_name": "Xicana_Xingona", "lang": "en", "location": "San Fernando Valle.", "create_at": date("2012-01-26"), "description": "sensational", "followers_count": 292, "friends_count": 330, "statues_count": 16580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279172345856, "text": "smh.. gone triff��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 438681129, "name": "romeo⭐️", "screen_name": "zaxkshawty", "lang": "en", "location": "East Lansing, MI", "create_at": date("2011-12-16"), "description": "517✈️352✈517", "followers_count": 341, "friends_count": 170, "statues_count": 22570 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Haslett, MI", "id": "4399b5004a3b4d9a", "name": "Haslett", "place_type": "city", "bounding_box": rectangle("-84.483739,42.731229 -84.350094,42.787887") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2637100, "cityName": "Haslett" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279243677696, "text": "I guess I'll just walk the earth like Cain in Kung Fu.", "in_reply_to_status": 738625883703906304, "in_reply_to_user": 2342447460, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2342447460, "name": "le Bron", "screen_name": "LeFrenchAccent", "lang": "en", "location": "Paris", "create_at": date("2014-02-13"), "description": "One time I got drunk with Dan Harmon and he said I was just like the hipsters who record podcasts in L.A.", "followers_count": 488, "friends_count": 351, "statues_count": 55195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279461638145, "text": "@sophelstien the best part is because he's helping write Mary Poppins 2", "in_reply_to_status": 738477061828947968, "in_reply_to_user": 814164222, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 814164222 }}, "user": { "id": 28818148, "name": "Becky Nussbaum", "screen_name": "beckilda", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-04-04"), "description": "ΔΘ Theatre Design/Technology. SUNY Purchase Class of 2017.", "followers_count": 227, "friends_count": 338, "statues_count": 6209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279633649664, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 72955416, "name": "YT", "screen_name": "Yuri_YT_Todd", "lang": "en", "location": "cleveland,ms", "create_at": date("2009-09-09"), "description": "YT DA JOHN -Class of #2⃣0⃣1⃣1⃣", "followers_count": 1841, "friends_count": 1567, "statues_count": 122524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, MS", "id": "e2fa1e6d23aea554", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-90.763071,33.718281 -90.697661,33.766135") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28011, "countyName": "Bolivar", "cityID": 2814260, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279692369920, "text": "@its_zaack you only been 19 for two hours ��", "in_reply_to_status": 738626198897463296, "in_reply_to_user": 1965956371, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1965956371 }}, "user": { "id": 461667800, "name": "B", "screen_name": "Brandyraveee", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-01-11"), "description": "19 | God is within her, she will not fall", "followers_count": 1998, "friends_count": 1713, "statues_count": 87672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626279956586496, "text": "Wind 0.0 mph ---. Barometer 29.928 in, Falling. Temperature 69.1 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 65963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626280002752513, "text": "I don't wanna be saved.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1511162636, "name": "Shipler ❤️", "screen_name": "KiraNoelleeve", "lang": "en", "location": "null", "create_at": date("2013-06-12"), "description": "I like ladies❤️ ✨RIP Abby✨ Mudblood", "followers_count": 430, "friends_count": 335, "statues_count": 4166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-06-03T00:00:02.000Z"), "id": 738626280367624192, "text": "That's always awkward https://t.co/6DIb8icd9k", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 27246424, "name": "Damelle", "screen_name": "HOLA_MELLITO", "lang": "en", "location": "East Flatbush, Brooklyn", "create_at": date("2009-03-28"), "description": "school couldn't get me into heaven and heaven couldn't get me into bitches bed #TheSuiteLife", "followers_count": 1188, "friends_count": 1142, "statues_count": 161125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626280883671040, "text": "Wind 0.0 mph S. Barometer 30.04 in, Steady. Temperature 68.2 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 31, "friends_count": 123, "statues_count": 160257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626280967393280, "text": "WYA @curllykilla", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 803573706 }}, "user": { "id": 2562695310, "name": "Dev", "screen_name": "DB_400", "lang": "en", "location": "In The Zone", "create_at": date("2014-06-11"), "description": "Never Satisfied", "followers_count": 1458, "friends_count": 693, "statues_count": 30987 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626281403645953, "text": "Happy Birthday gorgeous ���� Hope you have an amazing day! Love you & miss you girly �� @_Snookii3_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 954447108 }}, "user": { "id": 3241980912, "name": "Natalie", "screen_name": "nataliearias131", "lang": "en", "location": "null", "create_at": date("2015-06-10"), "description": "Corinthians 5:7", "followers_count": 215, "friends_count": 292, "statues_count": 665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Firebaugh, CA", "id": "1bf4409ce9f41ef2", "name": "Firebaugh", "place_type": "city", "bounding_box": rectangle("-120.491468,36.83684 -120.428897,36.88301") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 624134, "cityName": "Firebaugh" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626281646886912, "text": "In the morning ������ https://t.co/1hg2e9f21x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367608998, "name": "juicy $hay", "screen_name": "shayyonce", "lang": "en", "location": "Equinox", "create_at": date("2011-09-04"), "description": "Island in the Sun - Weezer", "followers_count": 713, "friends_count": 478, "statues_count": 16031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626282183761920, "text": "Tldr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cs", "is_retweet": false, "user": { "id": 458033393, "name": "Bryan H. Iglesias", "screen_name": "BryanHIglesias", "lang": "en", "location": "Detroit, Michigan", "create_at": date("2012-01-07"), "description": "20 • @PVeraMI • Floral Media • WSU", "followers_count": 792, "friends_count": 419, "statues_count": 11596 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Novi, MI", "id": "9458d33c93e47ce4", "name": "Novi", "place_type": "city", "bounding_box": rectangle("-83.555202,42.436044 -83.432974,42.527116") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26125, "countyName": "Oakland", "cityID": 2659440, "cityName": "Novi" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626282246676480, "text": "I wish none of my friends met Ben... Or my dad these hoes getting out of hand", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2878281923, "name": "beanster:)", "screen_name": "breanbuchholz", "lang": "en", "location": "home like always", "create_at": date("2014-11-15"), "description": "null", "followers_count": 215, "friends_count": 177, "statues_count": 5635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626282368303105, "text": "Excited to get things back to the way they were ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 736647564003401728, "name": "Kyyy", "screen_name": "kylee_puga", "lang": "en", "location": "Grapeland Tx", "create_at": date("2016-05-28"), "description": "no stress, bs gets passed to the side", "followers_count": 86, "friends_count": 98, "statues_count": 60 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crockett, TX", "id": "4bbe438f647e9ba5", "name": "Crockett", "place_type": "city", "bounding_box": rectangle("-95.481396,31.290167 -95.429031,31.341444") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48225, "countyName": "Houston", "cityID": 4817744, "cityName": "Crockett" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626282741587969, "text": "@AshleyGleizer ur probably in retrograde rn thats why ur acting so stubborn about zodiac signs ��", "in_reply_to_status": 738624332587368448, "in_reply_to_user": 430348117, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430348117 }}, "user": { "id": 344514255, "name": "bryan", "screen_name": "kissmyaciid", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-07-28"), "description": "punk when im drunk shy when im high // insta: woodlandchills", "followers_count": 171, "friends_count": 216, "statues_count": 1199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626283035189249, "text": "@Gorable suck it easy", "in_reply_to_status": 738625567667281920, "in_reply_to_user": 2386208942, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2386208942 }}, "user": { "id": 539572417, "name": "Marcus", "screen_name": "Africanbus", "lang": "en", "location": "El Reno, OK", "create_at": date("2012-03-28"), "description": "being bored", "followers_count": 1152, "friends_count": 506, "statues_count": 25720 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Reno, OK", "id": "58f3b65e0b58ed8f", "name": "El Reno", "place_type": "city", "bounding_box": rectangle("-98.023637,35.491641 -97.928169,35.553573") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40017, "countyName": "Canadian", "cityID": 4023700, "cityName": "El Reno" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626283093921793, "text": "Do I have hoes?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231999806, "name": "Rev. Berry Black XVI", "screen_name": "Ride_MARTAvious", "lang": "en", "location": "Stone Mountain, Georgia", "create_at": date("2010-12-29"), "description": "I'm Socrates but my skin more chocolatey. ΑΦΑ. Civil Rights Activist. #UGAXVI", "followers_count": 1718, "friends_count": 661, "statues_count": 83847 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bartlesville, OK", "id": "8de43ba3f996df05", "name": "Bartlesville", "place_type": "city", "bounding_box": rectangle("-96.012722,36.699006 -95.899191,36.786088") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40147, "countyName": "Washington", "cityID": 4004450, "cityName": "Bartlesville" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626283282665472, "text": "@mindykatee hang", "in_reply_to_status": 738624167633813504, "in_reply_to_user": 1599930037, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1599930037 }}, "user": { "id": 4848337821, "name": "Stupid", "screen_name": "ZaneyManBun", "lang": "en", "location": "West Jordan, UT", "create_at": date("2016-01-25"), "description": "yeah, alright.", "followers_count": 211, "friends_count": 261, "statues_count": 1360 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Jordan, UT", "id": "cb224c3c6c1ee882", "name": "West Jordan", "place_type": "city", "bounding_box": rectangle("-112.07287,40.565952 -111.911764,40.640189") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4982950, "cityName": "West Jordan" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626283978948609, "text": "@_Tahjii https://t.co/Xf4CM6zH0k", "in_reply_to_status": -1, "in_reply_to_user": 344585111, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 344585111 }}, "user": { "id": 293784019, "name": "kevin", "screen_name": "_Naexo", "lang": "en", "location": "at da crib with klove & chay❤️", "create_at": date("2011-05-05"), "description": "Kevin love is MINE and only MINE.", "followers_count": 848, "friends_count": 983, "statues_count": 20290 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Euclid, OH", "id": "19cdbb0c29dbc3a5", "name": "South Euclid", "place_type": "city", "bounding_box": rectangle("-81.54638,41.501345 -81.506679,41.54648") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3973264, "cityName": "South Euclid" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626284146724865, "text": "Wow ur really dwelling on this right now? That's kinda petty bro. Even for you https://t.co/GiWvj7jBi7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2810480617, "name": "Mr. know it all", "screen_name": "desond1199", "lang": "en", "location": "Coalinga, CA", "create_at": date("2014-09-14"), "description": "Romans 1:16", "followers_count": 229, "friends_count": 253, "statues_count": 1978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coalinga, CA", "id": "385c3a11a1020ac5", "name": "Coalinga", "place_type": "city", "bounding_box": rectangle("-120.375457,36.117735 -120.326632,36.179965") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614274, "cityName": "Coalinga" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626284394188801, "text": "@kadengoss I don't proofread at all lol", "in_reply_to_status": 738623912213254145, "in_reply_to_user": 458617683, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 458617683 }}, "user": { "id": 829937137, "name": "JaCoby", "screen_name": "BoNTonamo_Bay11", "lang": "en", "location": "Alexandria, LA", "create_at": date("2012-09-17"), "description": "love life", "followers_count": 524, "friends_count": 524, "statues_count": 15099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626284717256704, "text": "Australian boat chick liked my boat pics. Best day ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 260533284, "name": "Shifty", "screen_name": "tstenq", "lang": "en", "location": "null", "create_at": date("2011-03-03"), "description": "A salt breeze is all I need", "followers_count": 91, "friends_count": 98, "statues_count": 7088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, ME", "id": "34b5e468ea4e8110", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-70.346323,43.640514 -70.238767,43.75186") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2360545, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-03T00:00:03.000Z"), "id": 738626284859707392, "text": "@LynchFoundation @skyferreira @elreytheatre ������", "in_reply_to_status": 738039392573501440, "in_reply_to_user": 103602549, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 103602549, 16171898, 19691240 }}, "user": { "id": 387981245, "name": "Lucas David", "screen_name": "lucasbavid", "lang": "es", "location": "Los Angeles, CA", "create_at": date("2011-10-09"), "description": "lucasbavid@gmail.com", "followers_count": 10786, "friends_count": 63, "statues_count": 4621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626285186867200, "text": "6/3/2016 - 02:00\nTemp: 69.4F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.957in. & Falling\nRain: 0.00 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 112, "friends_count": 54, "statues_count": 53446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626285732167680, "text": "Lol https://t.co/l1VvzlJx9X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2675930756, "name": "Louis Vuitton", "screen_name": "FuckNiggaThere", "lang": "en", "location": "null", "create_at": date("2014-07-23"), "description": "Snap : fuckniggathere", "followers_count": 223, "friends_count": 192, "statues_count": 12464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Hammocks, FL", "id": "7707ad9771781687", "name": "The Hammocks", "place_type": "city", "bounding_box": rectangle("-80.498527,25.65479 -80.415878,25.688692") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271569, "cityName": "The Hammocks" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626285857996800, "text": "@Bookselenaxxx Earth, Wind, & Fire Reasons. It's old but it's the jam to mack at the honeys. https://t.co/LP5Qguxxxa", "in_reply_to_status": 736457735705878529, "in_reply_to_user": 164364168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 164364168 }}, "user": { "id": 251386874, "name": "Almighty R.A.O.", "screen_name": "FunkyFu42", "lang": "en", "location": "Martinsburg, WV USA", "create_at": date("2011-02-12"), "description": "The Sun God formerly Sunshine Superman! My life is a 1970's Black Exploitation, Spaghetti Western, Kung Fu Grind House Flick. Imperious Rex!!", "followers_count": 636, "friends_count": 2426, "statues_count": 18483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinsburg, WV", "id": "d5039bee9eb93f2c", "name": "Martinsburg", "place_type": "city", "bounding_box": rectangle("-78.028084,39.426663 -77.94245,39.494633") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54003, "countyName": "Berkeley", "cityID": 5452060, "cityName": "Martinsburg" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626285916692482, "text": "Like domo say \"idk why dey be juicing u ty, you nt even cute\" maaaaan brother ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262129541, "name": "TY", "screen_name": "tycurrency", "lang": "en", "location": "Paradise, CA ☀️", "create_at": date("2013-12-25"), "description": "null", "followers_count": 2030, "friends_count": 1772, "statues_count": 17660 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626286394966016, "text": "Or might get hoop shoes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2270062748, "name": "ped", "screen_name": "pedro_bojorquez", "lang": "en", "location": "Hayward, CA", "create_at": date("2013-12-31"), "description": "hoop,gym,pizza. ig:pedkee", "followers_count": 507, "friends_count": 384, "statues_count": 36799 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626286503874560, "text": "https://t.co/aGpVBo767o", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1636409094, "name": "cheyyy", "screen_name": "_cheeyaanne", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2013-07-31"), "description": "null", "followers_count": 518, "friends_count": 315, "statues_count": 8931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626286503890945, "text": "ugly doesn't exist.. ���� https://t.co/iBfvHbAvoB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 50146461, "name": "Jessica Zermeño", "screen_name": "King_Jessie95", "lang": "en", "location": "Bay Area", "create_at": date("2009-06-23"), "description": "20. but what if Andy doesn't like me? sc:its_jessiez", "followers_count": 601, "friends_count": 752, "statues_count": 4929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, CA", "id": "b1d8c2ed61d6a6c8", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.133029,37.68433 -122.093713,37.709794") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 602980, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626286936018944, "text": "RJ IS MAKING FUN OF ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2486932849, "name": "Diana Mtz", "screen_name": "diamtz_", "lang": "en", "location": "null", "create_at": date("2014-05-09"), "description": "las artes escénicas", "followers_count": 328, "friends_count": 255, "statues_count": 4005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626286994653187, "text": "@DevinChvnce The violence at Trumps San Jose rally today is a perfect example of how fucking stupid both sides are.", "in_reply_to_status": 738626088625020929, "in_reply_to_user": 2181255722, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2181255722 }}, "user": { "id": 2181255722, "name": "Devin Chvnce", "screen_name": "DevinChvnce", "lang": "en", "location": "null", "create_at": date("2013-11-07"), "description": "@Shade_CA • San Diego Nightlife Industry • @EllyMistura • SC: DevinChvnce", "followers_count": 568, "friends_count": 273, "statues_count": 12831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287275642880, "text": "@angelaahenthorn didnt you watch it like 2 days ago", "in_reply_to_status": 738626139816493056, "in_reply_to_user": 1057071000, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1057071000 }}, "user": { "id": 1166485255, "name": "karly", "screen_name": "karlymatheson", "lang": "en", "location": "water", "create_at": date("2013-02-10"), "description": "summer wya", "followers_count": 286, "friends_count": 312, "statues_count": 4617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ontario, CA", "id": "895f19a3c08d3d35", "name": "Ontario", "place_type": "city", "bounding_box": rectangle("-117.6836,33.997213 -117.523966,34.093104") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 653896, "cityName": "Ontario" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287388884993, "text": "I dont even know how to pin a tweet. Haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2650178864, "name": "Eli Wiggins", "screen_name": "elifly_", "lang": "en", "location": "dont have one", "create_at": date("2014-07-15"), "description": "work in progr.......", "followers_count": 367, "friends_count": 282, "statues_count": 2603 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287464390658, "text": "THEY BETTER BRING THAT BLACK HEART https://t.co/4ZULATKfMC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1530120788, "name": "Hi", "screen_name": "achloecave", "lang": "en", "location": "null", "create_at": date("2013-06-19"), "description": "I'm Amanda, I make jokes when I'm uncomfortable.", "followers_count": 342, "friends_count": 256, "statues_count": 7242 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Monterey Park, CA", "id": "fa424f169eef946e", "name": "Monterey Park", "place_type": "city", "bounding_box": rectangle("-118.170463,34.027012 -118.093679,34.071692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 648914, "cityName": "Monterey Park" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287611187201, "text": "lmao F it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3040688239, "name": "manditaaa", "screen_name": "ncash222", "lang": "en", "location": "wherever the bass is", "create_at": date("2015-02-24"), "description": "goddess of love 2.3", "followers_count": 269, "friends_count": 253, "statues_count": 4347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mirada, CA", "id": "4db990e393c2e28b", "name": "La Mirada", "place_type": "city", "bounding_box": rectangle("-118.037975,33.873395 -117.976352,33.928407") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640032, "cityName": "La Mirada" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287615381505, "text": "Angelica is a cheater lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 770636557, "name": "1AM ++", "screen_name": "1AM_Dallas", "lang": "en", "location": "Dallas, Tx", "create_at": date("2012-08-20"), "description": "null", "followers_count": 606, "friends_count": 297, "statues_count": 19980 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Farmers Branch, TX", "id": "97b70992556c6354", "name": "Farmers Branch", "place_type": "city", "bounding_box": rectangle("-96.938694,32.895548 -96.821221,32.953548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4825452, "cityName": "Farmers Branch" } }
+{ "create_at": datetime("2016-06-03T00:00:04.000Z"), "id": 738626287988813824, "text": "someone come get they girl https://t.co/vINISTtQjh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3680764335, "name": "Myk", "screen_name": "mykelorodriquez", "lang": "en", "location": "Avon, OH", "create_at": date("2015-09-16"), "description": "16", "followers_count": 333, "friends_count": 679, "statues_count": 1664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Avon, OH", "id": "9d8a608c1895dce3", "name": "Avon", "place_type": "city", "bounding_box": rectangle("-82.069227,41.418361 -81.969378,41.47865") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39093, "countyName": "Lorain", "cityID": 3903352, "cityName": "Avon" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988656141049858, "text": "time flies ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65715067, "name": "sara", "screen_name": "sarasnickerson", "lang": "en", "location": "michael vo ♡ ", "create_at": date("2009-08-14"), "description": "your future hasn't been written yet, no ones has. your future is whatever you make it. so make it a good one", "followers_count": 194, "friends_count": 240, "statues_count": 6278 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cypress, CA", "id": "a077793faceeda6f", "name": "Cypress", "place_type": "city", "bounding_box": rectangle("-118.063298,33.791963 -118.010668,33.845856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 617750, "cityName": "Cypress" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988656166379520, "text": "Poor Ali ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user": { "id": 222792523, "name": "Oshi™", "screen_name": "Jsevero23", "lang": "en", "location": "Wisconsin, USA", "create_at": date("2010-12-04"), "description": "@aliciallnzana ❤", "followers_count": 912, "friends_count": 826, "statues_count": 16992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Green Bay, WI", "id": "648f332371670aae", "name": "Green Bay", "place_type": "city", "bounding_box": rectangle("-88.139804,44.47174 -87.876593,44.574607") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55009, "countyName": "Brown", "cityID": 5531000, "cityName": "Green Bay" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988656522842112, "text": "@xNED37x yo", "in_reply_to_status": -1, "in_reply_to_user": 558204650, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 558204650 }}, "user": { "id": 441314763, "name": "$$ GoTTaBag $$", "screen_name": "Chasebanz", "lang": "en", "location": "youtube.com/chasebanzvideos", "create_at": date("2011-12-19"), "description": "Seen by Over 3,000,000+people/ on youtube now: I WILL FT KING LOUIE / Clout ft King Yella & lil jay - link below for videos https://www.instagram.com/chasebanz/", "followers_count": 28591, "friends_count": 27342, "statues_count": 15797 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988656707264512, "text": "SMD https://t.co/qzzkWWb1S5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 580492303, "name": "mtn dew luvr", "screen_name": "shrentai", "lang": "en", "location": "maia", "create_at": date("2012-05-14"), "description": "Meggie: thicker than a milkshake, taller than your daddy-homie hopper-your bestfriend has probably tried to kiss me", "followers_count": 573, "friends_count": 425, "statues_count": 598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Puyallup, WA", "id": "000dea80079d8b64", "name": "Puyallup", "place_type": "city", "bounding_box": rectangle("-122.336103,47.145624 -122.232425,47.207294") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5356695, "cityName": "Puyallup" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988657109929984, "text": "#Job in #NewYork, NY: Associate Production Manager at J. Crew https://t.co/J20n6rhj9B #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.9874105,40.7322535"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Job", "NewYork", "Jobs", "Hiring" }}, "user": { "id": 3166199949, "name": "J.Crew Jobs", "screen_name": "jcrew_jobs", "lang": "en", "location": "New York", "create_at": date("2015-04-14"), "description": "@jcrew the opportunities are endless. We offer a challenging, creative and high energy atmosphere and look for people who share a passion for our brand.", "followers_count": 313, "friends_count": 5, "statues_count": 681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988657391095808, "text": "Wind 0.0 mph ---. Barometer 29.97 in, Falling slowly. Temperature 67.6 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 33024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988657546133504, "text": "aw thank you dude���� https://t.co/8lwIgYhNZr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2658180261, "name": "mallory", "screen_name": "malloryfriedman", "lang": "en", "location": "null", "create_at": date("2014-06-30"), "description": "Sahuaro", "followers_count": 847, "friends_count": 564, "statues_count": 4754 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kendall, FL", "id": "9b46dccb3cfb880c", "name": "Kendall", "place_type": "city", "bounding_box": rectangle("-80.389344,25.628844 -80.304896,25.715128") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1236100, "cityName": "Kendall" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988657621635073, "text": "But really when you finna catch that neck �� https://t.co/1RSjUbF9iI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 60487902, "name": "FizzOnYourJace", "screen_name": "DR5_Clutch", "lang": "en", "location": "null", "create_at": date("2009-07-26"), "description": "Fizz Mid", "followers_count": 313, "friends_count": 304, "statues_count": 22476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988657655173120, "text": "@YaBoyKrucial well, you know what I'm talking about.", "in_reply_to_status": 738988553720336384, "in_reply_to_user": 184422828, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 184422828 }}, "user": { "id": 74250916, "name": "Marco Pavé", "screen_name": "KingofMarco", "lang": "en", "location": "Memphis ", "create_at": date("2009-09-14"), "description": "#HipHop #artist #SocialActivist #Educator #Mentor #Lecturer #TED speaker from North #Memphis, #Tennessee. Creating culture.\nBookings: marco@kingofmarco.com", "followers_count": 4412, "friends_count": 1900, "statues_count": 45120 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988658489839616, "text": "He'd probably smack tf out me before I could pick that bitch up �� https://t.co/eE2pi2RIGr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346753247, "name": ";)", "screen_name": "Tijeezy9", "lang": "en", "location": "null", "create_at": date("2011-08-01"), "description": "15.", "followers_count": 673, "friends_count": 436, "statues_count": 20404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988658523541504, "text": "https://t.co/2NsKrUdEtk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1641192984, "name": "pat mcgov", "screen_name": "ptrkmcgvrn", "lang": "en", "location": "null", "create_at": date("2013-08-02"), "description": "complete imbecile", "followers_count": 484, "friends_count": 336, "statues_count": 3097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988659576348672, "text": "This my shit lol https://t.co/rHqRNiDSKg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 192643860, "name": "Aaron M. Ridley", "screen_name": "aridleyjr", "lang": "en", "location": "San Antonio,TX- New Orleans,LA", "create_at": date("2010-09-19"), "description": "Every man must decide whether he will walk in the light of creative altruism or in the darkness of destructive selfishness.-Dr. Martin Luther King Jr. #AlphaMan", "followers_count": 996, "friends_count": 398, "statues_count": 17287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laplace, LA", "id": "005beffd77be6ac9", "name": "Laplace", "place_type": "city", "bounding_box": rectangle("-90.519583,30.031013 -90.435378,30.105989") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2242030, "cityName": "Laplace" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988659823644673, "text": "How DARE you badmouth me when I have been NOTHING but kind and supportive and honest with you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1609489872, "name": "Ryley", "screen_name": "ryleymaaeee", "lang": "en", "location": "null", "create_at": date("2013-07-20"), "description": "Rock n Roll with me", "followers_count": 464, "friends_count": 387, "statues_count": 10632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temescal Valley, CA", "id": "008a3a896caae197", "name": "Temescal Valley", "place_type": "city", "bounding_box": rectangle("-117.534018,33.708126 -117.397837,33.814923") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678138, "cityName": "Temescal Valley" } }
+{ "create_at": datetime("2016-06-04T00:00:00.000Z"), "id": 738988660134191104, "text": "Wind 0.0 mph ---. Barometer 29.903 in, Steady. Temperature 69.7 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66058 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988660410875908, "text": "@ebbtideapp Tide in Wrightsville Beach, North Carolina 06/04/2016\nHigh 7:12am 4.3\n Low 1:20pm -1.0\nHigh 7:39pm 5.4\n Low 2:15am -0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-77.7867,34.2133"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 101, "friends_count": 1, "statues_count": 38462 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Carolina, USA", "id": "3b98b02fba3f9753", "name": "North Carolina", "place_type": "admin", "bounding_box": rectangle("-84.321948,33.752879 -75.40012,36.588118") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3775820, "cityName": "Wrightsville Beach" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988660763152386, "text": "���� https://t.co/e1Uk3PP0uV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3237589448, "name": "Ang", "screen_name": "AngAllen05", "lang": "en", "location": "435ÜTÄH missin you #21 #53", "create_at": date("2015-06-05"), "description": "Spiritual Gangster. Carpe Diem☯", "followers_count": 373, "friends_count": 287, "statues_count": 2892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tooele, UT", "id": "5f72d488385b8ced", "name": "Tooele", "place_type": "city", "bounding_box": rectangle("-112.331612,40.507327 -112.266954,40.569357") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49045, "countyName": "Tooele", "cityID": 4976680, "cityName": "Tooele" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988660821889025, "text": "@lacecierraa @goddfeels I'd bring you one but it'd melt like 30 seconds after I take it out of the freezer", "in_reply_to_status": 738988339701809152, "in_reply_to_user": 3653998790, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3653998790, 295030446 }}, "user": { "id": 2897469378, "name": "Angel Medina", "screen_name": "DatBoiEngel", "lang": "en", "location": "Perris, CA", "create_at": date("2014-11-12"), "description": "It's a Pro Era \n|c/o 2016 - HHS", "followers_count": 247, "friends_count": 175, "statues_count": 3091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988660939362308, "text": "hoe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2695912542, "name": "lyssa", "screen_name": "alyssamariee01", "lang": "en", "location": "null", "create_at": date("2014-07-31"), "description": "good on my own", "followers_count": 1539, "friends_count": 976, "statues_count": 55676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weslaco, TX", "id": "f664c6f63c0bef35", "name": "Weslaco", "place_type": "city", "bounding_box": rectangle("-98.057773,26.111766 -97.942697,26.242157") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4877272, "cityName": "Weslaco" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661144883200, "text": "SONIC Drive-In: Carhop/Skating Carhop (Server) (#Wichita, KS) https://t.co/b0ag4kAsvE #Hospitality #Job #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.2619462,37.7530732"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Wichita", "Hospitality", "Job", "Jobs", "Hiring" }}, "user": { "id": 2924224280, "name": "SONIC Jobs", "screen_name": "SONICjobs", "lang": "en", "location": "null", "create_at": date("2014-12-09"), "description": "Check out our open positions to learn how you can WORK YOUR SPIRIT at #SONIC!", "followers_count": 349, "friends_count": 39, "statues_count": 19738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661258092545, "text": "I've been slacking on my Twitter game lately ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 33333558, "name": "Baddie B", "screen_name": "_britt2trilll", "lang": "en", "location": "null", "create_at": date("2009-04-19"), "description": "Biology Major. Maneater. Trill. 23. Houston. UH", "followers_count": 643, "friends_count": 498, "statues_count": 43730 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channelview, TX", "id": "eabe7b6fd4504fff", "name": "Channelview", "place_type": "city", "bounding_box": rectangle("-95.176002,29.735548 -95.061098,29.851809") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4814236, "cityName": "Channelview" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661291749377, "text": "Just because somebody likes your post doesn't mean they like you. Don't ever forget that.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 59395123, "name": "מַרְפֵּא", "screen_name": "Marpoe", "lang": "en", "location": "717", "create_at": date("2009-07-23"), "description": "est.322", "followers_count": 2069, "friends_count": 1475, "statues_count": 37538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rutherford, PA", "id": "644b884a2b7a0d3b", "name": "Rutherford", "place_type": "city", "bounding_box": rectangle("-76.786372,40.263617 -76.748063,40.276101") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42043, "countyName": "Dauphin", "cityID": 4266864, "cityName": "Rutherford" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661471989760, "text": "Man I have an amazing life #blessed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "blessed" }}, "user": { "id": 2648393663, "name": "Mowgli", "screen_name": "itsBeitel1", "lang": "en", "location": "BreezeMode, USA", "create_at": date("2014-06-27"), "description": "I love my life. #OperationCharlie", "followers_count": 471, "friends_count": 354, "statues_count": 5848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661501480960, "text": "So superherokl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.113693,40.818493"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2559872604, "name": "TINY NYXXXXX", "screen_name": "nyxxrm", "lang": "en", "location": "null", "create_at": date("2014-06-10"), "description": "I would make a Zelda pun, but I don't wanna tri and force it", "followers_count": 276, "friends_count": 366, "statues_count": 7147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North East Martial Arts", "id": "07d9ef8805880000", "name": "North East Martial Arts", "place_type": "poi", "bounding_box": rectangle("-73.11369309999999,40.818492899999995 -73.113693,40.818493") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3663473, "cityName": "Ronkonkoma" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661522452480, "text": "Temp 63.9° Hi/Lo 65.8/63.9 Rng 1.9° WC 63.9° Hmd 93% Rain 0.00\" Storm 0.00\" BAR 30.008 Falling DP 61.8° Wnd 0mph Dir --- Gst 5mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19228 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988661992087552, "text": "Happy birthday @TheAdamHawk! Have a wonderful day today ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 389945480 }}, "user": { "id": 168262837, "name": "Christina Bustos", "screen_name": "christina_sings", "lang": "en", "location": "Fullerton, CA", "create_at": date("2010-07-18"), "description": "Hey everyone what is up, what's the 411?", "followers_count": 970, "friends_count": 1641, "statues_count": 24655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Parksdale, CA", "id": "f420a939c2deb10f", "name": "Parksdale", "place_type": "city", "bounding_box": rectangle("-120.037595,36.937996 -120.003688,36.959867") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6039, "countyName": "Madera", "cityID": 655751, "cityName": "Parksdale" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988662034075652, "text": "12 am and my credit card is screaming \"use me\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3312441936, "name": "Breanne", "screen_name": "breanneeeeeeee", "lang": "en", "location": "View Park-Windsor Hills, CA", "create_at": date("2015-08-11"), "description": "18. sc: breanneanderson. sju '20. LA||NY", "followers_count": 443, "friends_count": 402, "statues_count": 4076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "View Park-Windsor Hills, CA", "id": "3f5aa7a2b7beec37", "name": "View Park-Windsor Hills", "place_type": "city", "bounding_box": rectangle("-118.365715,33.982756 -118.331397,34.008397") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 682667, "cityName": "View Park-Windsor Hills" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988662055153664, "text": "@_kiss_and_tell neither can I", "in_reply_to_status": 738988590290640896, "in_reply_to_user": 437048937, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 437048937 }}, "user": { "id": 228182831, "name": "jasmine ✨", "screen_name": "jassmonae__", "lang": "en", "location": "theislandsssssss", "create_at": date("2010-12-18"), "description": "peanutbutta | fabulous ✨ | #LOC18 #FlyHighMoneyFein #LongLiveZQ #FlyHighBlake", "followers_count": 1719, "friends_count": 1387, "statues_count": 25085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988662210367488, "text": "I can't slow down for anyone . #stillmoving", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "stillmoving" }}, "user": { "id": 2971794661, "name": "Kayonce✨", "screen_name": "Herbalicious24", "lang": "en", "location": "Denver, CO", "create_at": date("2015-01-10"), "description": "b e-h a p p y", "followers_count": 113, "friends_count": 146, "statues_count": 624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988662252130304, "text": "Honestly all I want right now @1am is some chili cheese fries", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 133177311 }}, "user": { "id": 3025823938, "name": "♡ೄ̥̽", "screen_name": "casslizz", "lang": "en", "location": "null", "create_at": date("2015-02-08"), "description": "CakeFace | 21 | Sagittarius", "followers_count": 218, "friends_count": 214, "statues_count": 1468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Billings, MT", "id": "662aa8db4557a744", "name": "Billings", "place_type": "city", "bounding_box": rectangle("-108.692879,45.723722 -108.432965,45.871169") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30111, "countyName": "Yellowstone", "cityID": 3006550, "cityName": "Billings" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988662306799617, "text": "06/04@03:00 - Temp 65.7F, WC 65.7F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.029in, Falling slowly. Rain 0.00in. Hum 97%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988663216865280, "text": "Missin my boy more than anything right now. ��@tylerfranks16", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1064368489 }}, "user": { "id": 631585448, "name": "taylor", "screen_name": "ttaylor_hatt", "lang": "en", "location": "Lubbock, TX", "create_at": date("2012-07-09"), "description": "null", "followers_count": 668, "friends_count": 639, "statues_count": 1180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988663263088640, "text": "Wind 0.0 mph ---. Barometer 29.958 in, Falling. Temperature 67.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20316 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988663497973760, "text": "Wind 1.0 mph ESE. Barometer 29.915 in, Falling. Temperature 69.8 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988664093450240, "text": "I turned twenty years old and moved into my first apartment all in the same day. On to the next… https://t.co/DS3wi1Xs6T", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.08625503,41.74780832"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513578427, "name": "bunny", "screen_name": "canariuum", "lang": "en", "location": "New Paltz, NY", "create_at": date("2012-03-03"), "description": "little snail, can you tell me where this is?", "followers_count": 130, "friends_count": 89, "statues_count": 5649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Paltz, NY", "id": "01b86269009c037a", "name": "New Paltz", "place_type": "city", "bounding_box": rectangle("-74.111274,41.661228 -74.040328,41.782633") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36111, "countyName": "Ulster", "cityID": 3650551, "cityName": "New Paltz" } }
+{ "create_at": datetime("2016-06-04T00:00:01.000Z"), "id": 738988664210915328, "text": "@lululaura_ Happy Birthday!!!! I hope you have a great birthday ☺️��", "in_reply_to_status": -1, "in_reply_to_user": 1618432950, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1618432950 }}, "user": { "id": 110721906, "name": "Kara Donnerbauer", "screen_name": "OmgitzKara", "lang": "en", "location": "Arizona", "create_at": date("2010-02-02"), "description": "22 | Az | Concerts | Philippians 4:13 | Psalm 91:4", "followers_count": 230, "friends_count": 280, "statues_count": 13367 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988664500277250, "text": "just keep it real with me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3041985510, "name": "shar", "screen_name": "SharLecheil", "lang": "en", "location": "Stuttgart, AR ", "create_at": date("2015-02-25"), "description": "| #UAPB19✨ | sc: shar_avery | insta: s.averyyyy |", "followers_count": 1841, "friends_count": 1357, "statues_count": 14248 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988664542232576, "text": "Get a girl you're proud to show ya moma", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 293534962, "name": "Felipe (Culo)", "screen_name": "KidwithCurls97", "lang": "en", "location": "The Bay", "create_at": date("2011-05-05"), "description": "Let all hair be free", "followers_count": 433, "friends_count": 372, "statues_count": 10034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988664982667269, "text": "It's sad how I could you friends...but as I walked back to my seat after getting what I worked for..all you talked shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2869541861, "name": "dantebmx", "screen_name": "danteshreds", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-11-09"), "description": "#18", "followers_count": 68, "friends_count": 80, "statues_count": 1050 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988665393680385, "text": "Man wtf they were selling pickle dillys at this time tonight and I didn't even know������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2633131001, "name": "∂εε's вαвч✨", "screen_name": "yolanduuhh", "lang": "en", "location": "Texas, USA", "create_at": date("2014-06-22"), "description": "God blessed the unanswered prayers that led me to @TripleDee_3 ♡", "followers_count": 894, "friends_count": 551, "statues_count": 12541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wharton, TX", "id": "1a0cccdc06c8c4fb", "name": "Wharton", "place_type": "city", "bounding_box": rectangle("-96.126829,29.297115 -96.059419,29.356075") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48481, "countyName": "Wharton", "cityID": 4878136, "cityName": "Wharton" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988665695838208, "text": "@_KP24 ������", "in_reply_to_status": 738988522762297344, "in_reply_to_user": 970568773, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 970568773 }}, "user": { "id": 334370933, "name": "k. dot ♊️", "screen_name": "DontKERAboutyou", "lang": "en", "location": "Maryland, USA", "create_at": date("2011-07-12"), "description": "His eye is on the Sparrow, and I know He watches me #ndmu19", "followers_count": 1238, "friends_count": 1030, "statues_count": 54945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988665997660160, "text": "Lacr tomorrow https://t.co/iahgjWAiGN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2353519675, "name": "Skippy", "screen_name": "Hayden_Cole93", "lang": "en", "location": "B-Camp, CA", "create_at": date("2014-02-20"), "description": "Aiming for the little improvements every day Snapchat: HaydenC293", "followers_count": 301, "friends_count": 233, "statues_count": 3104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988666085740544, "text": "RIP MUHAMMAD ALI ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "hi", "is_retweet": false, "user": { "id": 2267741276, "name": "Melvin Walters III", "screen_name": "MWIII__", "lang": "en", "location": "Houston, TX", "create_at": date("2013-12-29"), "description": "Heir To The Throne #ActAccordingly #ImReddie", "followers_count": 193, "friends_count": 26, "statues_count": 11261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkadelphia, AR", "id": "bdefad825ffb7daa", "name": "Arkadelphia", "place_type": "city", "bounding_box": rectangle("-93.093798,34.091696 -93.044691,34.161433") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5019, "countyName": "Clark", "cityID": 501870, "cityName": "Arkadelphia" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988666119299072, "text": "Temp: 71.9°F Wind:0.0mph Pressure: 29.966hpa Falling Rapidly Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988666576506880, "text": "IF YOU MUTE ME WHEN WE ON FT IM HANGING UP . ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1199007638, "name": "Kahs❃", "screen_name": "kahley_hemmer", "lang": "en", "location": "Somewhere with Buck ", "create_at": date("2013-02-19"), "description": "Soe,", "followers_count": 612, "friends_count": 630, "statues_count": 7064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988666748424192, "text": "@tucan_12 todo boxeado", "in_reply_to_status": 738966418738139136, "in_reply_to_user": 3637285275, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 3637285275 }}, "user": { "id": 2381532320, "name": "Homero Perras", "screen_name": "Javi_Be_Goon", "lang": "en", "location": "El Paso, TX", "create_at": date("2014-03-09"), "description": "Anywayyys", "followers_count": 189, "friends_count": 255, "statues_count": 5872 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988666865909760, "text": "Deep Ellum https://t.co/J8UVUC45CI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40321200, "name": "Flex Washington", "screen_name": "DThompson25", "lang": "en", "location": "Texas", "create_at": date("2009-05-15"), "description": "Insta:Dthompson26 SnapChat:DThompson25 I don't tweet jokes. I'm really not funny. Just your average faithful, God fearing man.", "followers_count": 10798, "friends_count": 813, "statues_count": 92931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Forney, TX", "id": "0059ce31696ebaf7", "name": "Forney", "place_type": "city", "bounding_box": rectangle("-96.491676,32.69623 -96.387296,32.778267") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4826604, "cityName": "Forney" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988667142692864, "text": "@MissElise84 lol", "in_reply_to_status": 738988485898539008, "in_reply_to_user": 24843751, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 24843751 }}, "user": { "id": 1481125982, "name": "Natsume Kenji", "screen_name": "TENSAINEXTDOOR", "lang": "en", "location": "Shohoku", "create_at": date("2013-06-03"), "description": "Third year at Shohoku. Guy that won inter-high. Slam Dunk is life. #AlmaHive.", "followers_count": 6021, "friends_count": 1078, "statues_count": 145871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Walton Beach, FL", "id": "3c8eb2b0ed8c7c6d", "name": "Fort Walton Beach", "place_type": "city", "bounding_box": rectangle("-86.673121,30.401188 -86.586661,30.471719") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1224475, "cityName": "Fort Walton Beach" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988667234979841, "text": "������������ https://t.co/S61M5iunh6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3271101692, "name": "ᴘʀᴇᴛᴛʏ ᴄʜᴀʀ ❤️", "screen_name": "Charlyric2", "lang": "en", "location": "Center, TX", "create_at": date("2015-07-07"), "description": "Amosc:charlyric22 ❤️", "followers_count": 368, "friends_count": 819, "statues_count": 832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Commerce, TX", "id": "ae85e7b0eaee93ed", "name": "Commerce", "place_type": "city", "bounding_box": rectangle("-95.92868,33.225681 -95.880581,33.265279") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48231, "countyName": "Hunt", "cityID": 4816240, "cityName": "Commerce" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988667436445697, "text": "@JoeyCrochet absolutely", "in_reply_to_status": 738975562853679104, "in_reply_to_user": 97819801, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 97819801 }}, "user": { "id": 250408897, "name": "pikaboobs⚡️", "screen_name": "therewegojendoe", "lang": "en", "location": "fluent in memes", "create_at": date("2011-02-10"), "description": "sasshole // fuckboy magnet // petty pretty n proud", "followers_count": 1367, "friends_count": 666, "statues_count": 63590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Plains, NY", "id": "f97108ab3c4a42ed", "name": "White Plains", "place_type": "city", "bounding_box": rectangle("-73.78995,40.982445 -73.719524,41.069964") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36119, "countyName": "Westchester", "cityID": 3681677, "cityName": "White Plains" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988667520180224, "text": "I should probably go to sleep now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1942031179, "name": "LA LLORONA", "screen_name": "maritza_hilary", "lang": "en", "location": "null", "create_at": date("2013-10-06"), "description": "null", "followers_count": 555, "friends_count": 473, "statues_count": 21231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Azusa, CA", "id": "59105f0e84773bdd", "name": "Azusa", "place_type": "city", "bounding_box": rectangle("-117.949187,34.10673 -117.881336,34.169447") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 603386, "cityName": "Azusa" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988667784437764, "text": "I really hope the new jobs her,she really needs to get out if here", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1279428980, "name": "Adan Nolasco", "screen_name": "AdamArciga39", "lang": "en", "location": "California", "create_at": date("2013-03-18"), "description": "Small Theater, Big Family.", "followers_count": 95, "friends_count": 164, "statues_count": 783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South El Monte, CA", "id": "122aef8d6840943b", "name": "South El Monte", "place_type": "city", "bounding_box": rectangle("-118.075186,34.028137 -118.021298,34.064367") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 672996, "cityName": "South El Monte" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668065456129, "text": "Hot mami���� https://t.co/9TtTkuegcW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 83085288, "name": "SGRC", "screen_name": "sherllacayanan", "lang": "en", "location": "Sacramento, CA", "create_at": date("2009-10-17"), "description": "| 18 | Family | CSUS | Vallejo//Vacaville//Sac |", "followers_count": 927, "friends_count": 699, "statues_count": 24196 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vacaville, CA", "id": "50df1c2f85d2654d", "name": "Vacaville", "place_type": "city", "bounding_box": rectangle("-122.032758,38.308561 -121.911095,38.417127") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681554, "cityName": "Vacaville" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668141068288, "text": "if you wanna see me in my element just take me to marquee", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 44764068, "name": "lil roy", "screen_name": "royafbaby", "lang": "en", "location": "University of Louisville", "create_at": date("2009-06-04"), "description": "ravioli ravioli give me the formuoli", "followers_count": 1418, "friends_count": 898, "statues_count": 37802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668195508224, "text": "@amazingbedhead erin please\ni can't pay retail prices for crimin i have to steal it", "in_reply_to_status": 738988501635616768, "in_reply_to_user": 266404424, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 266404424 }}, "user": { "id": 159024741, "name": "Jill @ OP Ep 660", "screen_name": "PiratessUnluck", "lang": "en", "location": "So Cal", "create_at": date("2010-06-24"), "description": "---------Learn from the bones---------Twitter, Tumblr, and Skype are all the same handle.\nEditorial writer for the OPP", "followers_count": 294, "friends_count": 219, "statues_count": 35039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thousand Oaks, CA", "id": "0509a6acb3d27ec4", "name": "Thousand Oaks", "place_type": "city", "bounding_box": rectangle("-119.061758,34.126875 -118.776015,34.250081") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 678582, "cityName": "Thousand Oaks" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668204027904, "text": "@brendonurie thank you for always making me the happiest ��", "in_reply_to_status": -1, "in_reply_to_user": 27970832, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27970832 }}, "user": { "id": 64546295, "name": "Becca", "screen_name": "beccaaanicolee", "lang": "en", "location": "nj ", "create_at": date("2009-08-10"), "description": "1st June-The 1975", "followers_count": 1459, "friends_count": 343, "statues_count": 86585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwood, NJ", "id": "0036534d6f287a14", "name": "Blackwood", "place_type": "city", "bounding_box": rectangle("-75.088811,39.726424 -74.968468,39.852636") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3406040, "cityName": "Blackwood" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668308721665, "text": "@USPSHelp I wish my mail carrier picked up my packages every time I scheduled a pick up. What is even the point of scheduling? ��", "in_reply_to_status": -1, "in_reply_to_user": 2872571512, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2872571512 }}, "user": { "id": 24649622, "name": "Tasha Crabtree, LVT", "screen_name": "TashaImmunity", "lang": "en", "location": "Denver, CO /Las Vegas/Malibu", "create_at": date("2009-03-15"), "description": "Dr. Jeff: Rocky Mountain Vet, Big Brother Super Fan, @shopimmunity #spayandneuter #vetlife #puglife #DrJeff IG: tashafaceimmunity", "followers_count": 308, "friends_count": 74, "statues_count": 676 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668510044160, "text": "Now I'm fucked up and I'm missing you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 297705462, "name": "kiki", "screen_name": "kikikahn", "lang": "en", "location": "Txst", "create_at": date("2011-05-12"), "description": "http://modernblunt.tumblr.com", "followers_count": 662, "friends_count": 186, "statues_count": 24995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668547960832, "text": "�� https://t.co/vF3ivDbqWG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 84181676, "name": "Key. ❤️✨", "screen_name": "klha__", "lang": "en", "location": "Beecher, MI", "create_at": date("2009-10-21"), "description": "#RipCoyia. ✨| 18 | @zyaaire ❤️", "followers_count": 1560, "friends_count": 942, "statues_count": 89364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beecher, MI", "id": "8d5dfd9dd04ba8a7", "name": "Beecher", "place_type": "city", "bounding_box": rectangle("-83.763591,43.075092 -83.674217,43.133995") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26049, "countyName": "Genesee", "cityID": 2606820, "cityName": "Beecher" } }
+{ "create_at": datetime("2016-06-04T00:00:02.000Z"), "id": 738988668577304581, "text": "Not because I didn't want to tell them sober but because I enjoyed saying it more while drunk lol ... what is that????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177234370, "name": "D.T.(Da Truth)", "screen_name": "EpitomeTruth", "lang": "en", "location": "Rockford, Illinois", "create_at": date("2010-08-11"), "description": "SC: epitometruth", "followers_count": 1120, "friends_count": 486, "statues_count": 7780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988668652781568, "text": "Aye https://t.co/RrUKzNSByg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 166309260, "name": "™", "screen_name": "Maiolo421", "lang": "en", "location": "#ᴘʀᴀʏғᴏʀᴛɪᴍᴍʏ ", "create_at": date("2010-07-13"), "description": "Risk the fall, just to know how it feels to fly' ✊ #OneLove", "followers_count": 674, "friends_count": 163, "statues_count": 18941 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Republic, PA", "id": "00a07d2d64eb04f9", "name": "Republic", "place_type": "city", "bounding_box": rectangle("-79.89835,39.938851 -79.862514,39.977983") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42051, "countyName": "Fayette", "cityID": 4264224, "cityName": "Republic" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988668673609729, "text": "@kaaaayla_xoxo 27th n Thunderbird", "in_reply_to_status": 738988591494234112, "in_reply_to_user": 1601843400, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1601843400 }}, "user": { "id": 3010550514, "name": "TRÁP DÀDDŶ", "screen_name": "drake_anselmo", "lang": "en", "location": "Boolin Where Ain't Safe", "create_at": date("2015-02-05"), "description": "Kant interfere with my Money|\n623 be da whole damn team|\nPhx|", "followers_count": 303, "friends_count": 342, "statues_count": 6634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988668866564096, "text": "need some dope hoodies to cop. preferably white.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467893896, "name": "Slick Owens", "screen_name": "Khamronb", "lang": "en", "location": "Houston, Tx", "create_at": date("2012-01-18"), "description": "null", "followers_count": 594, "friends_count": 287, "statues_count": 32379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988668883345408, "text": "brb trippin balls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2271039876, "name": "baby e", "screen_name": "channingtitties", "lang": "en", "location": "south everett ", "create_at": date("2013-12-31"), "description": "yung greek shawty #crewofLs", "followers_count": 2410, "friends_count": 1324, "statues_count": 11875 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marysville, WA", "id": "01c06bfa28feceb4", "name": "Marysville", "place_type": "city", "bounding_box": rectangle("-122.220382,48.019961 -122.108673,48.157553") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5343955, "cityName": "Marysville" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988668929507329, "text": "A ��Hamsters escaped ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3739984574, "name": "Steven Morales", "screen_name": "MoralesSteven_", "lang": "en", "location": "Sacramento, CA", "create_at": date("2015-09-30"), "description": "17 Year Old Boss", "followers_count": 130, "friends_count": 120, "statues_count": 2090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669005103104, "text": "Pq tem varios snaps da boca do @Murilo_Abreeu no snap? Ele se virou modelo de batom?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "pt", "is_retweet": false, "user_mentions": {{ 167992217 }}, "user": { "id": 54082367, "name": "Ronaldo", "screen_name": "ronaldoojr", "lang": "pt", "location": "New Bedford, MA", "create_at": date("2009-07-05"), "description": "This is me, this is real", "followers_count": 392, "friends_count": 338, "statues_count": 36377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Bedford, MA", "id": "7edcf088c38463fa", "name": "New Bedford", "place_type": "city", "bounding_box": rectangle("-70.978426,41.591322 -70.899416,41.74525") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2545000, "cityName": "New Bedford" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669013348352, "text": "@kimalexaaa_ I fixed it. You can really rt now", "in_reply_to_status": 738988327374708740, "in_reply_to_user": 332160068, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 332160068 }}, "user": { "id": 544735360, "name": "Z Tutt", "screen_name": "zack_tuttle", "lang": "en", "location": "Alaska, USA", "create_at": date("2012-04-03"), "description": "you'll never know what you can't achieve, until you don't achieve it", "followers_count": 775, "friends_count": 565, "statues_count": 9964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kirkland, WA", "id": "ba4471e5bc687736", "name": "Kirkland", "place_type": "city", "bounding_box": rectangle("-122.240397,47.642154 -122.163413,47.718484") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335940, "cityName": "Kirkland" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669139226626, "text": "The man who views the world at 50 the same as he did at 20 has wasted 30 years of his life.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 555602168, "name": "Stacy Galvez", "screen_name": "StacyGalvez", "lang": "en", "location": "#guate #byu", "create_at": date("2012-04-16"), "description": "Have Faith. Have Hope. Live like His Son. Help Others on their Way.☀️", "followers_count": 172, "friends_count": 126, "statues_count": 2901 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant Grove, UT", "id": "b7bf044c46621317", "name": "Pleasant Grove", "place_type": "city", "bounding_box": rectangle("-111.774284,40.351435 -111.678506,40.413594") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4960930, "cityName": "Pleasant Grove" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669168549888, "text": "Liers https://t.co/Qp7KecNX7f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513643636, "name": "princesa", "screen_name": "rae_decuir", "lang": "en", "location": "brla", "create_at": date("2014-05-21"), "description": "pbs sports medicine", "followers_count": 1184, "friends_count": 716, "statues_count": 24211 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Jefferson, LA", "id": "004414a528d1e5d7", "name": "Old Jefferson", "place_type": "city", "bounding_box": rectangle("-91.057114,30.323548 -90.968964,30.393894") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2257705, "cityName": "Old Jefferson" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669210664961, "text": "Every female that ever said they wanted a real nigga went the other way and got dat corny nigga ��✍��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3434349233, "name": "King⚜Tip ‼️", "screen_name": "Copeland4Era", "lang": "en", "location": "Land Of Thieves ", "create_at": date("2015-08-21"), "description": "TyRee Copeland 5'10 Defensive Back #OTM™ #AlabamaPrep17 #RIPKIYA", "followers_count": 251, "friends_count": 321, "statues_count": 3031 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Birmingham, AL", "id": "92220986b9dfd67d", "name": "Birmingham", "place_type": "city", "bounding_box": rectangle("-86.967333,33.405025 -86.620501,33.67945") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 107000, "cityName": "Birmingham" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669248229377, "text": "@JacobWhitesides can we do that at Chicago since we really didnt get to do it in indy?", "in_reply_to_status": 738987717443395584, "in_reply_to_user": 313087874, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 313087874 }}, "user": { "id": 1602067170, "name": "lovesick", "screen_name": "faultinourcHood", "lang": "en", "location": "Indiana", "create_at": date("2013-07-17"), "description": "going on a road trip for @jacobwhitesides tour from May 30th - June 5 because I said i would back in July 2015 #liveurdreamskids", "followers_count": 1368, "friends_count": 1946, "statues_count": 10001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plainfield, IN", "id": "724e7bdbe5e6b8e2", "name": "Plainfield", "place_type": "city", "bounding_box": rectangle("-86.458669,39.63067 -86.326295,39.749143") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18063, "countyName": "Hendricks", "cityID": 1860246, "cityName": "Plainfield" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669411856384, "text": "@yosukecchi me too", "in_reply_to_status": 738988321687248896, "in_reply_to_user": 75968896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 75968896 }}, "user": { "id": 152350498, "name": "chihuahua triste @AX", "screen_name": "emmerihhi", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-06-05"), "description": "I Only Post High Quality Ringabel Content • Lester? • 20 • #カラ松girls • LINE:anazeld • @jaginya is the fulbright to my blackquill • @shithentai♡ #bruce", "followers_count": 1342, "friends_count": 1097, "statues_count": 352404 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669655126016, "text": "No https://t.co/JkLF8ZDmPi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 345291167, "name": "$poiledBrat", "screen_name": "Coolio_Athena", "lang": "en", "location": "breathing ✨", "create_at": date("2011-07-30"), "description": "PHS Cheerleader. Adonis ❤️", "followers_count": 1346, "friends_count": 1419, "statues_count": 18518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988669843832833, "text": "Muhammad Ali will always be an American icon https://t.co/Z7PVsAIY6v", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17674, "friends_count": 17808, "statues_count": 72649 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988670129078273, "text": "@alansito42 send location ��", "in_reply_to_status": 738988621580120064, "in_reply_to_user": 3069303046, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3069303046 }}, "user": { "id": 2922335448, "name": "Rafael", "screen_name": "SadDadRafael", "lang": "en", "location": "OKAGA, CA", "create_at": date("2014-12-07"), "description": "It Never Stops Raining In My Mind.", "followers_count": 618, "friends_count": 377, "statues_count": 11532 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988670158442496, "text": "Anybody wanna be my friend for tomorrow ��������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 812586858, "name": "Jessica", "screen_name": "jessica_facundo", "lang": "en", "location": "null", "create_at": date("2012-09-09"), "description": "null", "followers_count": 742, "friends_count": 675, "statues_count": 5929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988670225551364, "text": "I'm always like who dis? But never really ask cause I don't wanna be rude https://t.co/5UxJnp32CG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2295436933, "name": "goofus", "screen_name": "Post_melon", "lang": "en", "location": "9/24/15", "create_at": date("2014-01-16"), "description": ". so this is my life. And i want you to know that i am both happy and sad and im still trying to figure out how that could be. #teamAnissa", "followers_count": 1059, "friends_count": 1051, "statues_count": 14117 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Athens, CA", "id": "81923aaf1207edc1", "name": "West Athens", "place_type": "city", "bounding_box": rectangle("-118.317804,33.916377 -118.291514,33.931268") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684116, "cityName": "West Athens" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988670473121792, "text": "I had 86 bobby pins in my hair...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2444588720, "name": "Ellen Paczko", "screen_name": "elpaczko", "lang": "en", "location": "null", "create_at": date("2014-04-14"), "description": "null", "followers_count": 705, "friends_count": 731, "statues_count": 502 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mount Prospect, IL", "id": "b8fcb766f3f544fe", "name": "Mount Prospect", "place_type": "city", "bounding_box": rectangle("-87.975278,42.022542 -87.887547,42.102399") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1751089, "cityName": "Mount Prospect" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988670514958336, "text": "@SportsCentre be honest... you're all hammered aren't you?", "in_reply_to_status": -1, "in_reply_to_user": 430145070, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 430145070 }}, "user": { "id": 759396157, "name": "El Chappy", "screen_name": "SoafyaanZia", "lang": "en", "location": "null", "create_at": date("2012-08-15"), "description": "Because girls might be watching is the only reason I do anything #ApathyEnthusiast", "followers_count": 195, "friends_count": 161, "statues_count": 8732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988671169232897, "text": "Wind 0.0 mph NE. Barometer 29.897 in, Falling slowly. Temperature 68.1 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988672310071298, "text": "I'm going to get some Botox.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 14393191, "name": "✨ djMe ✨", "screen_name": "djMe", "lang": "en", "location": "NorCal", "create_at": date("2008-04-14"), "description": "✖ #TEAMGAY ✖ #EDM ✖ #GYMBRO ✖ #FOLLOBACK ✖ #KINK ✖ #LADYGAGA ✖ #ART ✖ #GEEK ✖", "followers_count": 3044, "friends_count": 2369, "statues_count": 20419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, CA", "id": "5fe017de422ddfca", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-121.939462,37.316129 -121.922231,37.327052") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 608968, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988672519786496, "text": "Used to being let down by now �� https://t.co/PkLGj3lAUt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 328897461, "name": "Karissa", "screen_name": "K_Rodriguez11", "lang": "en", "location": "Houston, TX", "create_at": date("2011-07-03"), "description": "null", "followers_count": 414, "friends_count": 163, "statues_count": 21062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-04T00:00:03.000Z"), "id": 738988672792420354, "text": "6/4/2016 - 02:00\nTemp: 69.5F \nHum: 99%\nWind: 0.0 mph\nBaro: 29.947in. & Falling\nRain: 0.01 in.\nhttps://t.co/aTIxgKkXL3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.765,34.53555556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 287319606, "name": "John Sacrey", "screen_name": "salineweather", "lang": "en", "location": "Benton, AR", "create_at": date("2011-04-24"), "description": "Certified NWS Spotter.", "followers_count": 111, "friends_count": 54, "statues_count": 53482 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5125, "countyName": "Saline" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988672826015745, "text": "Didn't realize i was gonna miss him this much... ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2864988023, "name": "What would Kev Do?", "screen_name": "Asia_lovve", "lang": "en", "location": "null", "create_at": date("2014-11-06"), "description": "Just tryna make a difference SUMMER 2K16 ☀\nLibra ♎", "followers_count": 470, "friends_count": 367, "statues_count": 9145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988672851140608, "text": "@xvoxv_ jk", "in_reply_to_status": 738983509038010369, "in_reply_to_user": 497512937, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 497512937 }}, "user": { "id": 497512937, "name": "MOANica.", "screen_name": "xvoxv_", "lang": "en", "location": "null", "create_at": date("2012-02-19"), "description": "hannah is my 4ever.", "followers_count": 2158, "friends_count": 680, "statues_count": 275961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673039929345, "text": "@ILostATaco not like girlfriend but a friend who's a girl", "in_reply_to_status": 738987691811864577, "in_reply_to_user": 1372507556, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1372507556 }}, "user": { "id": 225634090, "name": "Derek Morton", "screen_name": "MORTONLB53", "lang": "en", "location": "Norman, OK", "create_at": date("2010-12-11"), "description": "#MbongStrong #FlyHigh24 #Legend Official account of Derek Morton", "followers_count": 668, "friends_count": 597, "statues_count": 47247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Panama City Beach, FL", "id": "9ebd5acfac2301ba", "name": "Panama City Beach", "place_type": "city", "bounding_box": rectangle("-85.95802,30.165061 -85.786077,30.266595") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12005, "countyName": "Bay", "cityID": 1254725, "cityName": "Panama City Beach" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673404833792, "text": "HBD to the love of my life. Thank you for being the best boyfriend. I love you! Can't wait till you get home baby.���� https://t.co/cZmelIccBe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2262274632, "name": "kota❥", "screen_name": "DakotaLeigh6", "lang": "en", "location": "|•III.XIX.XVI•|", "create_at": date("2013-12-25"), "description": "| @aiden_80 ❤️ |", "followers_count": 880, "friends_count": 832, "statues_count": 7020 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Apple Valley, CA", "id": "41c331abe42d9969", "name": "Apple Valley", "place_type": "city", "bounding_box": rectangle("-117.287533,34.413411 -117.103437,34.571843") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 602364, "cityName": "Apple Valley" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673459359744, "text": "IM SEVENTEEN WITHOUT A PURPOSE OR DIRCTON", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 718709355772661760, "name": "lil spud", "screen_name": "sadquel", "lang": "en", "location": "corner of happy & healthy ", "create_at": date("2016-04-09"), "description": "|Wheat thins enthusiast| CSUN '20|", "followers_count": 114, "friends_count": 292, "statues_count": 1988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673476108291, "text": "Sliding in DMs like: https://t.co/3SMGAuqQYk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2514087541, "name": "naughty by nature", "screen_name": "gotdammitnikki", "lang": "en", "location": "null", "create_at": date("2014-05-21"), "description": "21. Houston Girl. Biology Pre-Med Major. Spanish Minor #TXST17 SC: Lovethruherr", "followers_count": 692, "friends_count": 409, "statues_count": 18633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673799094272, "text": "@curlyfries1231 https://t.co/h1BjKvsIVh", "in_reply_to_status": -1, "in_reply_to_user": 3141300278, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3141300278 }}, "user": { "id": 2380149156, "name": "©hino", "screen_name": "krisromero24", "lang": "en", "location": "null", "create_at": date("2014-03-09"), "description": "@curlyfries1231 calls me daddy", "followers_count": 616, "friends_count": 525, "statues_count": 4855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673903923201, "text": "Ohh https://t.co/Zdcn5mbHom", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 482386025, "name": "Laurie", "screen_name": "LaurieBea1", "lang": "en", "location": "whereever the moment takes me", "create_at": date("2012-02-03"), "description": "I like airports and food. Purpose by Justin Bieber go buy it, fucking amazing album. https://itun.es/us/5w7J-", "followers_count": 6055, "friends_count": 896, "statues_count": 123665 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988673971146752, "text": "https://t.co/oAAbeBjhui\n#Ghostbusters #ghostbusters2016 #ghostbustersfriday #ghostbusterstrailer #feminism #sjwlogic #stereotypes #movies", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Ghostbusters", "ghostbusters2016", "ghostbustersfriday", "ghostbusterstrailer", "feminism", "sjwlogic", "stereotypes", "movies" }}, "user": { "id": 47961628, "name": "Yul Tolbert", "screen_name": "timelike01", "lang": "en", "location": "Detroit", "create_at": date("2009-06-17"), "description": "Creator of Timeliketoons -- cartoons made for the 21st century even before it was the 21st century!", "followers_count": 668, "friends_count": 133, "statues_count": 33930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Detroit, MI", "id": "b463d3bd6064861b", "name": "Detroit", "place_type": "city", "bounding_box": rectangle("-83.288056,42.255085 -82.91052,42.450488") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26163, "countyName": "Wayne", "cityID": 2622000, "cityName": "Detroit" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988674348507136, "text": "True love and care requires more than mere words-it calls for action..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174600431, "name": "Rebecca S. Green", "screen_name": "Becca12474", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2010-08-04"), "description": "Christian actress/writer. \n I am a woman of God. I live each and every passing day, according to Gods ways and will for my life.", "followers_count": 238, "friends_count": 388, "statues_count": 523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eugene, OR", "id": "01241e9666cacdd2", "name": "Eugene", "place_type": "city", "bounding_box": rectangle("-123.208615,43.988281 -123.036188,44.142961") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41039, "countyName": "Lane", "cityID": 4123850, "cityName": "Eugene" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988674759561217, "text": "☺️☺️☺️☺️☺️☺️☺️☺️☺️☺️☺️☺️ https://t.co/rfuQ49O459", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2718666768, "name": "MORG$", "screen_name": "danktez", "lang": "en", "location": "denver", "create_at": date("2014-08-08"), "description": "null", "followers_count": 6517, "friends_count": 118, "statues_count": 73030 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westminster, CO", "id": "3f871475c095f94f", "name": "Westminster", "place_type": "city", "bounding_box": rectangle("-105.148925,39.819133 -104.987771,39.97944") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 883835, "cityName": "Westminster" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988674956791808, "text": "When the uber wants you to play @Drake ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 27195114 }}, "user": { "id": 113171878, "name": "Michael Vincent", "screen_name": "LOVEmikeyy", "lang": "en", "location": "Bronx, New York ", "create_at": date("2010-02-10"), "description": "validation is for parking, not people✨ @BritneySpears follows me. Instagram: mikey.vincent_ Snapchat: lovemikeyy #work", "followers_count": 957, "friends_count": 362, "statues_count": 52640 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988675636310016, "text": "@LukisPauley @AlexTheSlade @PopstarMovie fuck you", "in_reply_to_status": 738986354600759296, "in_reply_to_user": 290324231, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 290324231, 279198710, 3472392503 }}, "user": { "id": 2743757493, "name": "Lance cravens", "screen_name": "cravens_lance", "lang": "en", "location": "null", "create_at": date("2014-08-15"), "description": "Bernie stole my shecks.", "followers_count": 70, "friends_count": 84, "statues_count": 541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Nicholasville, KY", "id": "94d47cc557aa35f4", "name": "Nicholasville", "place_type": "city", "bounding_box": rectangle("-84.639133,37.833185 -84.544701,37.919373") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21113, "countyName": "Jessamine", "cityID": 2156136, "cityName": "Nicholasville" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988675942383616, "text": "Ahhh, the #Weekend! :-D https://t.co/ifJzuNxOqH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Weekend" }}, "user": { "id": 351901213, "name": "Vanessa M", "screen_name": "VanessaM65", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-08-09"), "description": "Wanna play with a Giant Panda (or a tiger cub) like I did? ... https://t.co/Csn0dzqo88 (not my company! Just sharing the fun!). Tell'em Vanessa sent ya! :-)", "followers_count": 93, "friends_count": 511, "statues_count": 1230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988675942486016, "text": "I remember at SWC dad texted me saying \"Everyone is freaking out that I'm your dad. I'm so proud of you.\"\nFirst time he said that in a while", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 224824691, "name": "daniel", "screen_name": "thisisclerkie", "lang": "en", "location": "canada/la/stl", "create_at": date("2010-12-09"), "description": "i made @enemygg/@maryvillegg | stl/la mostly | busy probably | @CosmicFogVapors | Sweden June 18-21 | Anaheim June 10-12 | eSports Consultant |", "followers_count": 18882, "friends_count": 302, "statues_count": 15222 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesterfield, MO", "id": "f9eb1e0194f99f8b", "name": "Chesterfield", "place_type": "city", "bounding_box": rectangle("-90.674747,38.610603 -90.48775,38.692771") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2913600, "cityName": "Chesterfield" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988676022079488, "text": "Stomach hurts ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 268376944, "name": "3:20❣", "screen_name": "Miraacleee", "lang": "en", "location": "Sc: Miraacleee", "create_at": date("2011-03-18"), "description": "Stay Beautiful. ❤️ Hi ☺️... Now Bye ✌️ R.I.P Jade & Bri I love y'all ❤️ #PVAMU20", "followers_count": 974, "friends_count": 700, "statues_count": 22546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988676231757824, "text": "S/o to the djs of this party for bumpin dubstep. Bless ur soul", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48891742, "name": "Emina", "screen_name": "eminasdopee", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-06-19"), "description": "AG ♡ let the beat control your body ॐ luvzdawubz", "followers_count": 865, "friends_count": 632, "statues_count": 14604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-04T00:00:04.000Z"), "id": 738988676340817926, "text": "My new baby Zeus �� https://t.co/TzpawdzBuS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "cs", "is_retweet": false, "user": { "id": 620460142, "name": "milk dud", "screen_name": "Judegonzalez_", "lang": "en", "location": "in haley's lil cactus pot ", "create_at": date("2012-06-27"), "description": "so fucking in love with my boyfriend ♐️♉️❤️5.26.15❤️", "followers_count": 791, "friends_count": 225, "statues_count": 26613 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Prairie, TX", "id": "a9b50065bfbb3fa9", "name": "Grand Prairie", "place_type": "city", "bounding_box": rectangle("-97.065649,32.720529 -96.924017,32.816653") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4830464, "cityName": "Grand Prairie" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351043973251072, "text": "I know I know, I make it hard to let go.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 88069092, "name": "♋المال قوة احترام ", "screen_name": "FrmPARISwLove", "lang": "en", "location": "Brooklyn, NY", "create_at": date("2009-11-06"), "description": "Relentless.", "followers_count": 4259, "friends_count": 1721, "statues_count": 62537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351044157644800, "text": "@hcameinalem tvd is making me wanna ndksjdjbsbs", "in_reply_to_status": 739350976386076673, "in_reply_to_user": 3021638028, "favorite_count": 0, "retweet_count": 0, "lang": "et", "is_retweet": false, "user_mentions": {{ 3021638028 }}, "user": { "id": 3021638028, "name": "mel", "screen_name": "hcameinalem", "lang": "en", "location": "null", "create_at": date("2015-02-15"), "description": "rhs track and field | sydney tran ❤️", "followers_count": 213, "friends_count": 200, "statues_count": 5507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Monte, CA", "id": "c2ad98c80a97adde", "name": "El Monte", "place_type": "city", "bounding_box": rectangle("-118.073101,34.040871 -117.997477,34.100871") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622230, "cityName": "El Monte" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351044178751488, "text": "Spartan Race Boston 2016 #SpartanRace https://t.co/5r8AfAI1dc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SpartanRace" }}, "user": { "id": 627501819, "name": "CloseQuartersCo.", "screen_name": "CQCBoston", "lang": "en", "location": "Boston, Mass", "create_at": date("2012-07-05"), "description": "Bounce Imaging Rep.\nTactical Throwable Camera\nSWAT\nSpecial Wears And Threads\nSpecial Weapons And Tactical Fitness\n#CQCBoston #SWATFIT", "followers_count": 121, "friends_count": 131, "statues_count": 1176 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351044598046720, "text": "@dee_ceee why so sad?", "in_reply_to_status": 739350822492864513, "in_reply_to_user": 307668958, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 307668958 }}, "user": { "id": 284368167, "name": "LIL ROBBIE", "screen_name": "RobEarth21994", "lang": "en", "location": "California", "create_at": date("2011-04-18"), "description": "By Any Means. Play.", "followers_count": 122, "friends_count": 34, "statues_count": 7385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351044652556289, "text": "idk what your talking about https://t.co/7qzvYOuHg5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3304181887, "name": "Micah Buchanan™", "screen_name": "micahbuchanan22", "lang": "en", "location": "null", "create_at": date("2015-08-02"), "description": "'19 | #SachseU FS : #DBU | #longliverobie", "followers_count": 361, "friends_count": 297, "statues_count": 3858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351044681924608, "text": "@hellomslewis @tmorello @prophetsofrage u really never understood Rage Against the Machine if you thought they were going to endorse anyone", "in_reply_to_status": 739336237014614016, "in_reply_to_user": 145552334, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 145552334, 23023236, 730914240865820673 }}, "user": { "id": 84009613, "name": "Shane Stewart", "screen_name": "Stufotog", "lang": "en", "location": "Chicago,IL", "create_at": date("2009-10-20"), "description": "Accept fear and except the challenge that conquers it. Freelance/Landscape Photographer. https://www.instagram.com/stufotog/", "followers_count": 89, "friends_count": 153, "statues_count": 815 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351045189537792, "text": "@jeskamedeiros you the one crying cuz I'm mean dique", "in_reply_to_status": 739350623129309184, "in_reply_to_user": 432168022, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 432168022 }}, "user": { "id": 299436184, "name": "LIL' LAY LO", "screen_name": "LORENZOSTAYRICH", "lang": "en", "location": "bronxtwitter.com", "create_at": date("2011-05-15"), "description": "i do what i want you do what you can | @FILTHYRICHNYC #BIC", "followers_count": 2641, "friends_count": 715, "statues_count": 10664 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351045499932673, "text": "I'm not gonna act like I don't care", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2855867405, "name": "Nibbler", "screen_name": "jay_jr12", "lang": "en", "location": "null", "create_at": date("2014-11-01"), "description": "Centereach Varsity Football : Brooke ❤️", "followers_count": 240, "friends_count": 350, "statues_count": 2136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Centereach, NY", "id": "faf8aaecf6a390c0", "name": "Centereach", "place_type": "city", "bounding_box": rectangle("-73.109642,40.832337 -73.051755,40.904556") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3613376, "cityName": "Centereach" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351045877301248, "text": "Come and see me for once", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2348619296, "name": "Nick", "screen_name": "niicknolasco", "lang": "en", "location": "null", "create_at": date("2014-02-17"), "description": "i signed up for greatness ... this comes with it.| |PSN:Nickoli27|#freelano SC: nickn0227 IG:nnolasco21 oVo", "followers_count": 609, "friends_count": 418, "statues_count": 23764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casa Grande, AZ", "id": "fbb3d1e41acab043", "name": "Casa Grande", "place_type": "city", "bounding_box": rectangle("-111.791608,32.858246 -111.670779,32.992892") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 410530, "cityName": "Casa Grande" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351045977972736, "text": "@raegandh ����", "in_reply_to_status": 739350288063029250, "in_reply_to_user": 443091816, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 443091816 }}, "user": { "id": 1014379171, "name": "A Beautiful Man", "screen_name": "Trill_Soliloquy", "lang": "en", "location": "Houston, TX", "create_at": date("2012-12-15"), "description": "I really don't give a fuck about much...", "followers_count": 1841, "friends_count": 594, "statues_count": 240609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351046410129408, "text": "Wind 0.0 mph ---. Barometer 29.749 in, Falling. Temperature 63.3 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 38, "friends_count": 4, "statues_count": 27934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351046477254656, "text": "Lmao boy would you block my number then ���� https://t.co/aCckYAuUY2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 387478708, "name": "SheLovesSmitty ++®", "screen_name": "PMOSmitty", "lang": "en", "location": "null", "create_at": date("2011-10-08"), "description": "sc: itstriohoee ig: dvddy.supreme #NCAT #RoyalTKings", "followers_count": 1932, "friends_count": 1883, "statues_count": 51254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351046527410176, "text": "@ebbtideapp Tide in 8 miles above mouth, Georgia 06/05/2016\n Low 4:24am -0.7\nHigh 10:06am 8.1\n Low 4:30pm -1.1\nHigh 10:34pm 9.6", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-81.5667,31.1"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 101, "friends_count": 1, "statues_count": 38741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Georgia, USA", "id": "7142eb97ae21e839", "name": "Georgia", "place_type": "admin", "bounding_box": rectangle("-85.605166,30.355644 -80.742567,35.000771") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13039, "countyName": "Camden" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351046602919936, "text": "yk you coulda been my joce tonight�� smh party pooper. goodnight�� https://t.co/w6tkUAlyoI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3271484394, "name": "⛵️", "screen_name": "3mobackwoods", "lang": "en", "location": "null", "create_at": date("2015-07-07"), "description": "send beats, advice, etc: reallynemohoes@yahoo.com #llp20 i do it for you pookie ily", "followers_count": 1803, "friends_count": 3067, "statues_count": 17004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351046737137665, "text": "A nice juicy fresh cut mango would sound bomb af rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2940111584, "name": "Kat", "screen_name": "katxallen", "lang": "en", "location": "Guam", "create_at": date("2014-12-22"), "description": "sc:katallen | hca | love sleep but barely get any", "followers_count": 351, "friends_count": 314, "statues_count": 8166 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, CO", "id": "7b9254d3f3763854", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-105.180287,39.889067 -105.136695,39.963057") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8013, "countyName": "Boulder", "cityID": 875640, "cityName": "Superior" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351047139790848, "text": "Damn damn damn�������� https://t.co/6GTOdxnE69", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2488424678, "name": "Miriam.", "screen_name": "ummiriam11", "lang": "en", "location": "null", "create_at": date("2014-05-10"), "description": "null", "followers_count": 329, "friends_count": 65, "statues_count": 46008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351047408254976, "text": "Can't relate https://t.co/p52XZ2ecRV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3287321347, "name": "TP Lil Nekko", "screen_name": "Nekkohbk", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-07-22"), "description": "is it true or you faking?", "followers_count": 868, "friends_count": 173, "statues_count": 9627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lewisville, TX", "id": "0ec50d630c5d2c2b", "name": "Lewisville", "place_type": "city", "bounding_box": rectangle("-97.044681,32.985088 -96.938975,33.099786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48121, "countyName": "Denton", "cityID": 4842508, "cityName": "Lewisville" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351047588749313, "text": "Lmao watching the carbonaro effect for the first time, it's so funny", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 34770980, "name": "Hennessy Complexion", "screen_name": "RozayyRedd_", "lang": "en", "location": "NY✈NC", "create_at": date("2009-04-23"), "description": "#GiantsNation", "followers_count": 1541, "friends_count": 898, "statues_count": 129209 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-06-05T00:00:00.000Z"), "id": 739351047785697281, "text": "Juwan scary af nigga", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2922953174, "name": "Devin Schleigh", "screen_name": "DevinSchleigh", "lang": "en", "location": "null", "create_at": date("2014-12-08"), "description": "Why you creepin?", "followers_count": 168, "friends_count": 246, "statues_count": 631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valle Vista, CA", "id": "6504ec137a7160ee", "name": "Valle Vista", "place_type": "city", "bounding_box": rectangle("-116.919834,33.729635 -116.85798,33.765951") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 681708, "cityName": "Valle Vista" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048091926528, "text": "Happy bday to my favorite cousin @__briiiiiiiii", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4303103173 }}, "user": { "id": 1727665880, "name": "BUBA", "screen_name": "TherealPortis", "lang": "en", "location": "null", "create_at": date("2013-09-03"), "description": "null", "followers_count": 595, "friends_count": 711, "statues_count": 1864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048293212160, "text": "Okay we have a deal �� https://t.co/NseA7nV5oh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 519985830, "name": "Isis", "screen_name": "isisnaomi_", "lang": "en", "location": "#Eat4Cleat • TxSUWBB20", "create_at": date("2012-03-09"), "description": "life is tough but so are you #STC #txsu20", "followers_count": 1983, "friends_count": 1189, "statues_count": 52738 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048293351424, "text": "Beautiful scenery @ Hornblower Cruises & Events San Francisco https://t.co/3YfRE1kCAn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.393622,37.795209"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15300421, "name": "Jenn~", "screen_name": "lovejennifer", "lang": "en", "location": "Sunny California", "create_at": date("2008-07-02"), "description": "Twitter addict who loves all things pink, taking pictures & random cuteness ♡", "followers_count": 230, "friends_count": 245, "statues_count": 21961 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048423411712, "text": "Wind 0.0 mph SE. Barometer 29.89 in, Falling slowly. Temperature 72.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23157 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048544980992, "text": "You deserved that �� https://t.co/jLXt6A27F3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2671101774, "name": "Sidney Jones", "screen_name": "STK_64", "lang": "en", "location": "Hartford, CT", "create_at": date("2014-07-22"), "description": "President and Founder of Ca$hout Kings |Promotions Director at SCSU TV | DT for the SCSU Owls #95 | Journalism Major |SCSU '19|", "followers_count": 698, "friends_count": 617, "statues_count": 18458 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartford, CT", "id": "61c225139f635563", "name": "Hartford", "place_type": "city", "bounding_box": rectangle("-72.718386,41.723759 -72.643547,41.807475") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937000, "cityName": "Hartford" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048809119744, "text": "@mf_astrid yes", "in_reply_to_status": 739350928659087360, "in_reply_to_user": 2558080673, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2558080673 }}, "user": { "id": 3128975226, "name": "SOAP", "screen_name": "__Reaver", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-04-01"), "description": "Im the asshole your mother warned you about", "followers_count": 274, "friends_count": 176, "statues_count": 16429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351048888799232, "text": "https://t.co/EbIy15la8S", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2392842001, "name": "azra", "screen_name": "arzaaa17", "lang": "en", "location": "11416", "create_at": date("2014-03-16"), "description": "I'm all eyes (literally)", "followers_count": 527, "friends_count": 476, "statues_count": 17152 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049081921537, "text": "@DnellDoe lol I swear I wish I could crawl into my bed right now ��", "in_reply_to_status": 739350893380939776, "in_reply_to_user": 249436812, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 249436812 }}, "user": { "id": 776568997, "name": "T.", "screen_name": "Tylarrrr_", "lang": "en", "location": "Tucked", "create_at": date("2012-08-23"), "description": "Diamonds are Forever #LongLive19", "followers_count": 1282, "friends_count": 1231, "statues_count": 4396 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sandusky, OH", "id": "fcc54ac4568181a4", "name": "Sandusky", "place_type": "city", "bounding_box": rectangle("-82.78773,41.409372 -82.617802,41.492373") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39043, "countyName": "Erie", "cityID": 3970380, "cityName": "Sandusky" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049295802368, "text": "Temp 64.1°F Heat Index 64.1 °F RH 96% Wind 0.0 --- Gust 0.0 --- SLP 29.794 in Falling Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 111, "friends_count": 63, "statues_count": 35940 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049320960000, "text": "Prayers to Tony, Joyce, and Annie... I don't know what else to say ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 300579810, "name": "ʝҿɲɳα ℳα℞أҿ ✞", "screen_name": "YouNeedOxyJENN", "lang": "en", "location": "Wheeling, WV", "create_at": date("2011-05-17"), "description": "SnapChat; ItsjDogBITCH | IG; 000hjay_ ♡", "followers_count": 570, "friends_count": 315, "statues_count": 19382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wheeling, WV", "id": "046f4f5c96e7e0d5", "name": "Wheeling", "place_type": "city", "bounding_box": rectangle("-80.738783,40.017148 -80.642979,40.149379") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54069, "countyName": "Ohio", "cityID": 5486452, "cityName": "Wheeling" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049471938560, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 29.900 in, Falling. Temp 66.0F. Rain today 0.00 in. Humidity 86% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 629, "friends_count": 842, "statues_count": 43522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049627017216, "text": "@Gabellab can you bless my night up with a video.", "in_reply_to_status": -1, "in_reply_to_user": 3884751804, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3884751804 }}, "user": { "id": 418455024, "name": "KingLucy", "screen_name": "AdrianaLove24", "lang": "en", "location": "null", "create_at": date("2011-11-21"), "description": "20 years old. Positive with a big heart. ❣", "followers_count": 163, "friends_count": 164, "statues_count": 8442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351049992081408, "text": "@OhMyGodingg i falled sleep", "in_reply_to_status": 739292677687136260, "in_reply_to_user": 1584059538, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1584059538 }}, "user": { "id": 1493436576, "name": "Lord Lightskin", "screen_name": "dyoungman10", "lang": "en", "location": "null", "create_at": date("2013-06-08"), "description": "Sam ❤️ #ODU Booty Gang Disciple", "followers_count": 589, "friends_count": 370, "statues_count": 36610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newport News, VA", "id": "6f36fc2e52870eee", "name": "Newport News", "place_type": "city", "bounding_box": rectangle("-76.62797,36.960216 -76.387647,37.220386") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51700, "countyName": "Newport News", "cityID": 5156000, "cityName": "Newport News" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050239381505, "text": "diplo is sooooo sexy my god", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4919044459, "name": "Melissa Martinez", "screen_name": "melitonaaaa", "lang": "en", "location": "lala land ", "create_at": date("2016-02-16"), "description": "pain demands to be felt", "followers_count": 324, "friends_count": 181, "statues_count": 973 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orchards, WA", "id": "1d471644ea7c10d2", "name": "Orchards", "place_type": "city", "bounding_box": rectangle("-122.552904,45.657895 -122.495761,45.708318") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5351795, "cityName": "Orchards" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050381971456, "text": "@DILLONFRANCIS why won't you love me", "in_reply_to_status": 739339741145899010, "in_reply_to_user": 19520983, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19520983 }}, "user": { "id": 2476315028, "name": "Meghan", "screen_name": "megs_greer", "lang": "en", "location": "CA | KY ", "create_at": date("2014-05-03"), "description": "null", "followers_count": 571, "friends_count": 424, "statues_count": 4018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050407182336, "text": "I'm going to make u mine. that's my word @1future", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 51742969 }}, "user": { "id": 247065284, "name": "BALLERina", "screen_name": "_SINsational", "lang": "en", "location": "null", "create_at": date("2011-02-03"), "description": "anybody can get flexed on", "followers_count": 1286, "friends_count": 548, "statues_count": 53389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050440847360, "text": "Wind 1.0 mph W. Barometer 29.757 in, Falling. Temperature 68.1 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113734 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050587504640, "text": "Finna go to steak in shake and think about life while I drink a large Oreo shake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 343073600, "name": "Romeo Football", "screen_name": "RomeoBalboa23", "lang": "en", "location": "West Dallas-Waxahachie Tx", "create_at": date("2011-07-26"), "description": "Student Athlete | SAGU19 | #FearTheRoar Manager of Dream Big Productions sponsored by @wafflehouse SC:romeocantu", "followers_count": 2223, "friends_count": 2155, "statues_count": 66452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050671562752, "text": "Captain crunch really is so slammin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 185052585, "name": "Dark Lo's 30", "screen_name": "vladtotheface", "lang": "en", "location": "North Philly", "create_at": date("2010-08-30"), "description": "Rest in peace grandmom 9/11/29- 1/3/14. Always reppin for that low life. #TempleMade", "followers_count": 844, "friends_count": 770, "statues_count": 22379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wildwood, NJ", "id": "e633e204df26332a", "name": "Wildwood", "place_type": "city", "bounding_box": rectangle("-74.851572,38.976028 -74.800966,38.999355") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34009, "countyName": "Cape May", "cityID": 3481170, "cityName": "Wildwood" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050692493312, "text": "Temp: 68.2F W C: 68.2F Wind:E at 2.0kts Baro: 1005.9mb and Falling Rain today: 0.00in R H: 99% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 216, "friends_count": 218, "statues_count": 107104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050755280896, "text": "Wide awake��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 274219596, "name": "Brooke", "screen_name": "BrookeWallace07", "lang": "en", "location": "Joplin", "create_at": date("2011-03-29"), "description": "null", "followers_count": 603, "friends_count": 764, "statues_count": 9039 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Duquesne, MO", "id": "00e197b630ad64cf", "name": "Duquesne", "place_type": "city", "bounding_box": rectangle("-94.473902,37.054821 -94.423007,37.089511") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29097, "countyName": "Jasper", "cityID": 2920512, "cityName": "Duquesne" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351050973413379, "text": "\" Don't be putting my suga daddy quotes on Twitter \" ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 603379037, "name": "LL cool Tae ‼️", "screen_name": "_TheWeekDays_", "lang": "en", "location": "local. ", "create_at": date("2012-06-08"), "description": "well dressed. Well respected.", "followers_count": 2105, "friends_count": 310, "statues_count": 52966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Country Club Hills, IL", "id": "08695e38fe9e7edf", "name": "Country Club Hills", "place_type": "city", "bounding_box": rectangle("-87.74458,41.527979 -87.701853,41.586573") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1716691, "cityName": "Country Club Hills" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351051044683776, "text": "I'm so stuck on trying to figure out what I want in life right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2409412344, "name": "Ron", "screen_name": "space_jamm32", "lang": "en", "location": "null", "create_at": date("2014-03-24"), "description": "Brady...call me tom⛳️ fuck cancer #lyonhearted Suh dude", "followers_count": 97, "friends_count": 161, "statues_count": 830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alpine, CA", "id": "ff5e82bba182df0b", "name": "Alpine", "place_type": "city", "bounding_box": rectangle("-116.816165,32.802788 -116.71178,32.852325") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 601192, "cityName": "Alpine" } }
+{ "create_at": datetime("2016-06-05T00:00:01.000Z"), "id": 739351051669635072, "text": "@mafffyew what's your YT name? I'd like to subscribe.", "in_reply_to_status": 739348010589556736, "in_reply_to_user": 158116038, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 158116038 }}, "user": { "id": 97595135, "name": "Isabel Ann Castro", "screen_name": "QueenOfTacosTX", "lang": "en", "location": "Southside, SATX", "create_at": date("2009-12-17"), "description": "Spurs. Tacos. Punk. Tech. Movies. Vinyl. Comics. Cartoons. Science. Illustration. Feminismo. Work: @desmadredotcom Zine: @StSucia c/s", "followers_count": 1160, "friends_count": 2112, "statues_count": 15282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052328148993, "text": "If you really want something go get it it's yours", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554557831, "name": "Leek", "screen_name": "_kingl__", "lang": "en", "location": "null", "create_at": date("2012-04-15"), "description": "Blessed & highly favored", "followers_count": 643, "friends_count": 581, "statues_count": 36822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052504420352, "text": "Temp 63.8° Hi/Lo 67.4/63.8 Rng 3.6° WC 63.8° Hmd 92% Rain 0.00\" Storm 0.00\" BAR 29.939 Falling DP 61.4° Wnd 0mph Dir --- Gst 7mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 74, "friends_count": 123, "statues_count": 19253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052634361856, "text": "@always_be IF ONLY.", "in_reply_to_status": 739348706969821184, "in_reply_to_user": 14995948, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14995948 }}, "user": { "id": 24703754, "name": "Zoe Kinsky", "screen_name": "NamiRocket", "lang": "en", "location": "Houston, Texas, USA", "create_at": date("2009-03-16"), "description": "Welcome to my dumb opinion pit. I like to ramble a lot. This seems as good a place as any to consolidate a lot of it. Profile art by @Scott_DeWitt.", "followers_count": 915, "friends_count": 945, "statues_count": 19762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052718243840, "text": "summer+tonight>>>", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2573636456, "name": "wrecka t", "screen_name": "TobiaserErica", "lang": "en", "location": "#5", "create_at": date("2014-06-17"), "description": "⚽ @outlawsoccer Good Vibe Tribe", "followers_count": 641, "friends_count": 434, "statues_count": 856 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pleasant View, UT", "id": "3ed34eccf80aaf27", "name": "Pleasant View", "place_type": "city", "bounding_box": rectangle("-112.034287,41.289958 -111.975323,41.341409") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4961150, "cityName": "Pleasant View" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052785307650, "text": "https://t.co/4woxF7jHCt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 336796677, "name": "f", "screen_name": "_FmTheGreat", "lang": "en", "location": "Houston ", "create_at": date("2011-07-16"), "description": "why read my bio when you could read a book", "followers_count": 2376, "friends_count": 880, "statues_count": 109507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351052890279936, "text": "06/05@03:00 - Temp 71.1F, WC 71.1F. Wind 1.4mph SSE, Gust 5.0mph. Bar 29.892in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49446 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351053045501953, "text": "Prosper* https://t.co/U73Qwj2H98", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3162868266, "name": "FNUGANGJOJO", "screen_name": "IceLxrdJay", "lang": "en", "location": "null", "create_at": date("2015-04-18"), "description": "I'ma eat regardless", "followers_count": 173, "friends_count": 137, "statues_count": 511 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillside, IL", "id": "88bddb0e2768943d", "name": "Hillside", "place_type": "city", "bounding_box": rectangle("-87.919829,41.844383 -87.882342,41.885453") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1735086, "cityName": "Hillside" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351053284474880, "text": "AHH Sheamayy!! I'm so sad I don't get to share this wonderful day with you but happy birthday best friend I love you so much!����@sheamay050", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2236691480 }}, "user": { "id": 3041822754, "name": "Alli", "screen_name": "AllieChafey", "lang": "en", "location": "null", "create_at": date("2015-02-25"), "description": "♡❥SRDC 15-16 cvhs 19' DC2 senior✌", "followers_count": 248, "friends_count": 338, "statues_count": 296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351053594828800, "text": "Out of all of the concerts I've been to, J-LO was hands down the greatest performer I've ever seen. #JLoVegas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "JLoVegas" }}, "user": { "id": 405011186, "name": "Jennifer Rodriguez", "screen_name": "itsjenn10", "lang": "en", "location": "null", "create_at": date("2011-11-04"), "description": "null", "followers_count": 397, "friends_count": 536, "statues_count": 3066 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351053762736129, "text": "Temp: 70.6°F Wind:0.1mph Pressure: 29.837hpa Falling Rain Today 0.02in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351053821317122, "text": "I am?��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2395571892, "name": "⚡️MidgetJohnson⚡️Ⓜ️", "screen_name": "PeanutSuave", "lang": "en", "location": "null", "create_at": date("2014-03-17"), "description": "Mak-A-Roni-Toni #LONGLIVEZQ #RIPCoachGreg", "followers_count": 969, "friends_count": 1120, "statues_count": 30748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054064582657, "text": "Reese De'What #SNL @nbcsnl https://t.co/pxLSF9kHcY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "SNL" }}, "user_mentions": {{ 28221296 }}, "user": { "id": 78210076, "name": "༺ ་ʑıʂʂყ་ ༻", "screen_name": "ZissyFoy", "lang": "en", "location": "90804", "create_at": date("2009-09-28"), "description": "Funny Odd Strange Shy and ♕Out Spoken♕ i'm a contradiction on so many levels. #Lakota #Irish #Swedish #legalizemarijuana #painfree #animals #LongBeachCA", "followers_count": 1153, "friends_count": 2574, "statues_count": 14329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054391902208, "text": "Wind 0.0 mph ---. Barometer 29.755 in, Falling slowly. Temperature 66.4 °F. Rain today 0.01 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054404329475, "text": "I need to go get this free Chicfila sandwich tho not today tho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 58043106, "name": "dude", "screen_name": "iEmelyn", "lang": "en", "location": "Raleigh, NC", "create_at": date("2009-07-18"), "description": "http://snapchat.com/add/Trapxgawwd", "followers_count": 6811, "friends_count": 2902, "statues_count": 57075 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054450630656, "text": "Ripley SW Limestone Co. Temp: 72.1°F Wind:3.8mph Pressure: 990.9mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 54134 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054463033344, "text": "Might sleep early for once.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2415562086, "name": "ST♛", "screen_name": "svlinvlynnthor", "lang": "en", "location": "Fresno, CA", "create_at": date("2014-03-28"), "description": "5'0 / SC: salinalynnthor ❤️", "followers_count": 5398, "friends_count": 4590, "statues_count": 10244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054500794368, "text": "@Tadeh_e rushing home before their batteries die ��", "in_reply_to_status": 739343896795222016, "in_reply_to_user": 1016689616, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1016689616 }}, "user": { "id": 542219864, "name": "Reneh", "screen_name": "AsatoorianReneh", "lang": "en", "location": "null", "create_at": date("2012-03-31"), "description": "The most dangerous thing you can do in life is to play it safe.", "followers_count": 308, "friends_count": 282, "statues_count": 13113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Quinta, CA", "id": "012cf25774f836cd", "name": "La Quinta", "place_type": "city", "bounding_box": rectangle("-116.323001,33.599493 -116.233322,33.736598") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 640354, "cityName": "La Quinta" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351054685331456, "text": "me and @alaura129 leave for the weekend and come back with our hair dyed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2274146497 }}, "user": { "id": 486257019, "name": "jewels", "screen_name": "loovejewels", "lang": "en", "location": "LA", "create_at": date("2012-02-07"), "description": "weak messages create bad situations. Michelle is my best friend.", "followers_count": 1051, "friends_count": 2642, "statues_count": 53227 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351055142506496, "text": "I put the \"hate\" in \"whatever\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2490280536, "name": "Kallie Raymer", "screen_name": "kallieraymer", "lang": "en", "location": "somewhere lame", "create_at": date("2014-05-11"), "description": "litty committee", "followers_count": 1023, "friends_count": 1563, "statues_count": 6725 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arden-Arcade, CA", "id": "b90f2a335f8565c0", "name": "Arden-Arcade", "place_type": "city", "bounding_box": rectangle("-121.423941,38.562585 -121.327437,38.645482") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602553, "cityName": "Arden-Arcade" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351055855550465, "text": "@lustrelux just DM me, it's easier��", "in_reply_to_status": 739349889658716160, "in_reply_to_user": 2300629838, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2300629838 }}, "user": { "id": 267539926, "name": "Aranya Tara", "screen_name": "AranyaTara", "lang": "en", "location": "null", "create_at": date("2011-03-16"), "description": "null", "followers_count": 300, "friends_count": 234, "statues_count": 8718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351055893303296, "text": "@amanialhindi �� https://t.co/zHH8WP2fYC", "in_reply_to_status": -1, "in_reply_to_user": 3277353949, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3277353949 }}, "user": { "id": 2332790974, "name": "Lanie", "screen_name": "MelanieQuinter0", "lang": "en", "location": "null", "create_at": date("2014-02-08"), "description": "Lanie LAX to the Max", "followers_count": 108, "friends_count": 145, "statues_count": 531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351055993954304, "text": "Happens all the time in Mexico �������������� https://t.co/CyK3gCeay7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2790627897, "name": "Nora Tejeda", "screen_name": "noraabieber6", "lang": "en", "location": "bayyy area ", "create_at": date("2014-09-28"), "description": "null", "followers_count": 237, "friends_count": 135, "statues_count": 7483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-06-05T00:00:02.000Z"), "id": 739351056111403009, "text": "#Selena921 187", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Selena921" }}, "user": { "id": 4218681672, "name": "jaycee!SELENA921", "screen_name": "jayceekw", "lang": "en", "location": "broken arrow, oklahoma", "create_at": date("2015-11-12"), "description": "my name is jaycee and i love mainland", "followers_count": 41, "friends_count": 76, "statues_count": 1601 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Broken Arrow, OK", "id": "f3665f19993977ca", "name": "Broken Arrow", "place_type": "city", "bounding_box": rectangle("-95.868576,35.959581 -95.671814,36.11903") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4009050, "cityName": "Broken Arrow" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351056518402048, "text": "@AwesomEmergency why you didn't do a GOT q&a?", "in_reply_to_status": -1, "in_reply_to_user": 57795767, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57795767 }}, "user": { "id": 78192138, "name": "youngthegod", "screen_name": "badboyyoung", "lang": "en", "location": "Brooklyn", "create_at": date("2009-09-28"), "description": "Audio Engineer/ Ciroc Boyz / Fast Life /\n10,000 Hours", "followers_count": 676, "friends_count": 552, "statues_count": 2851 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057092874240, "text": "Ambers contact name is \"Double Bacon Burger\" and stephanys is \"Double Quarter Pounder w/ cheese\" and when I get a message from them I die ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 716736777386471426, "name": "bri", "screen_name": "grahambriannaa", "lang": "en", "location": "null", "create_at": date("2016-04-03"), "description": "chs cheerleader ✨", "followers_count": 271, "friends_count": 189, "statues_count": 3059 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atascocita, TX", "id": "38d0e797745f4c5d", "name": "Atascocita", "place_type": "city", "bounding_box": rectangle("-95.249792,29.912242 -95.135568,30.033094") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4804462, "cityName": "Atascocita" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057134821376, "text": "@MikaylaDedrick @drewcd_ \n-so gorgeous\n-wish we talked more\n-I love every picture you post", "in_reply_to_status": 739340974799785985, "in_reply_to_user": 438526682, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 438526682, 2349370340 }}, "user": { "id": 438526682, "name": "Mikayla", "screen_name": "MikaylaDedrick", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-12-16"), "description": "12/5/14 ♡", "followers_count": 657, "friends_count": 441, "statues_count": 9960 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oro Valley, AZ", "id": "23ddee699e919f28", "name": "Oro Valley", "place_type": "city", "bounding_box": rectangle("-111.02149,32.357649 -110.928016,32.477644") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 451600, "cityName": "Oro Valley" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057340334080, "text": "Being faded is raw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 430500992, "name": "☭Shmantonio☭", "screen_name": "TheShmantonio", "lang": "en", "location": "California, USA", "create_at": date("2011-12-06"), "description": "Bassists for @MillennialsFNO\nworkers rights are human rights\nhttps://themillennials.bandcamp.com/releases", "followers_count": 280, "friends_count": 383, "statues_count": 28199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057445199872, "text": "You said you'll always be there for me when I need it.. but #ThatAintNoneOfMyBuisness https://t.co/OfDcXhpiv1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ThatAintNoneOfMyBuisness" }}, "user": { "id": 368624487, "name": "PFB™", "screen_name": "PookieEfBabyy", "lang": "en", "location": "Seattle, WA", "create_at": date("2011-09-05"), "description": "I live in a world surrounded by dumb bitches.", "followers_count": 171, "friends_count": 65, "statues_count": 19002 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057545891840, "text": "Me and my bed have an unbeatable bond ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 318261192, "name": "sweet T✨", "screen_name": "YoungTayy___", "lang": "en", "location": "Houston, TX ", "create_at": date("2011-06-15"), "description": "null", "followers_count": 1299, "friends_count": 987, "statues_count": 32321 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057612955648, "text": "�� Reseda", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 562688521, "name": "Angela ♔", "screen_name": "angelafmarquez", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2012-04-25"), "description": "MCGI | IG: angelafmarquez", "followers_count": 632, "friends_count": 343, "statues_count": 44432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057625554944, "text": "@rethseee why don't you teach me how to make that dessert", "in_reply_to_status": 739350945146929152, "in_reply_to_user": 533829214, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 533829214 }}, "user": { "id": 1682047512, "name": "ŠTËËŽGØD", "screen_name": "EdwinVentura16", "lang": "en", "location": "Los Angeles,Ca", "create_at": date("2013-08-18"), "description": "Skateboarding,Family & Friends!!", "followers_count": 290, "friends_count": 216, "statues_count": 10273 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057889939456, "text": "Wind 0.0 mph ENE. Barometer 29.89 in, Falling. Temperature 67.6 °F. Rain today 0.00 in. Humidity 70%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 160305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351057998827523, "text": "I can't believe my eyes https://t.co/JYyay2PTFj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2538966510, "name": "The B", "screen_name": "storytime_B", "lang": "en", "location": "east beach, lubbock", "create_at": date("2014-06-01"), "description": "It's always EasyMoney. I love waffles", "followers_count": 651, "friends_count": 1230, "statues_count": 4402 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351058204397568, "text": "��snl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 21698257, "name": "jaclyn nic0le ☾", "screen_name": "jacktheweenie", "lang": "en", "location": "Nu Yauch ➡️ Whale's Vagina", "create_at": date("2009-02-23"), "description": "nonsensical", "followers_count": 148, "friends_count": 359, "statues_count": 4229 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351058300829696, "text": "My parents get complimented for being attractive more than I ever get complimented. They're like fine wine. Better with age.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3313001804, "name": "Wendy", "screen_name": "WendyMaganaa", "lang": "en", "location": "Henderson, NV", "create_at": date("2015-08-11"), "description": "You're not funny. I just laugh at everything.", "followers_count": 204, "friends_count": 216, "statues_count": 995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351058686857217, "text": "@Kendrick_Ko6 ������", "in_reply_to_status": 739350567416430592, "in_reply_to_user": 4633807355, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 4633807355 }}, "user": { "id": 69229846, "name": "Lougotca$h", "screen_name": "Lougotcash", "lang": "en", "location": "EVERY FUCKING WHERE", "create_at": date("2009-08-27"), "description": "I RAP #DREAMSFROMTHEBRICKS MIXTAPE NOW ON SPINRILLA http://spnr.la/IBD1xPHX #Vstate Rip Swint Rip Nana #FreeRico #FreeGunna", "followers_count": 5302, "friends_count": 3402, "statues_count": 145654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351058988699648, "text": "I don't get how females put up with bullshit for so long ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613940179, "name": "⚛ÉlclarkCó", "screen_name": "_CJ6_", "lang": "en", "location": "Tallahassee, Florida", "create_at": date("2012-06-20"), "description": "know the name, know the flame|TyBg|Arisen|FreekQuesho™|Jonah", "followers_count": 176, "friends_count": 100, "statues_count": 3641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tallahassee, FL", "id": "ecbe2aea853af44e", "name": "Tallahassee", "place_type": "city", "bounding_box": rectangle("-84.386548,30.365093 -84.1458,30.587338") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12073, "countyName": "Leon", "cityID": 1270600, "cityName": "Tallahassee" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059106304000, "text": "3am thoughts with no one to share em' with.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 926273628, "name": "brÿvnxø", "screen_name": "bryan_calixto", "lang": "en", "location": "New Jersey, USA", "create_at": date("2012-11-04"), "description": "we do what we want // 19", "followers_count": 800, "friends_count": 717, "statues_count": 22969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Passaic, NJ", "id": "22bb8afe3a5531fd", "name": "Passaic", "place_type": "city", "bounding_box": rectangle("-74.146675,40.834409 -74.107319,40.876612") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3456550, "cityName": "Passaic" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059269701632, "text": "#HappyBirthdayTroye hope your day is as wonderful as you, love you as much as nutella �� https://t.co/JiUGxPlDY7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "HappyBirthdayTroye" }}, "user": { "id": 4021253962, "name": "renee", "screen_name": "renee_kei", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-10-22"), "description": "lost and trying to be", "followers_count": 40, "friends_count": 111, "statues_count": 151 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059441688576, "text": "Lol I thought me and Oomf was taking a picture yesterday at graduation and she grabbed a hand full of my ass. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 624324039, "name": "Quay ⚡️", "screen_name": "_gameover22", "lang": "en", "location": "WrestleMania", "create_at": date("2012-07-01"), "description": "Hit Hard, Hit Often", "followers_count": 1051, "friends_count": 521, "statues_count": 41197 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mexia, TX", "id": "041a08c2d0c3f01f", "name": "Mexia", "place_type": "city", "bounding_box": rectangle("-96.504204,31.653047 -96.453416,31.712644") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48293, "countyName": "Limestone", "cityID": 4847916, "cityName": "Mexia" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059466878976, "text": "@CanUSayGorgeous ��������", "in_reply_to_status": 739350912691503104, "in_reply_to_user": 286741829, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 286741829 }}, "user": { "id": 284656622, "name": "Jodecee", "screen_name": "FlawlessInHeels", "lang": "en", "location": "Tennessee State Unviersity ", "create_at": date("2011-04-19"), "description": "Straight out of St.Louis", "followers_count": 630, "friends_count": 332, "statues_count": 18365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manchester, MO", "id": "5574b7b200b9c750", "name": "Manchester", "place_type": "city", "bounding_box": rectangle("-90.533436,38.547211 -90.465006,38.607713") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2945668, "cityName": "Manchester" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059533959168, "text": "This is the best beer week in the country don't @ me https://t.co/cEP2nxjTgS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2497836211, "name": "Big Walrus", "screen_name": "tampon_rodeo", "lang": "en", "location": "Behind any Korean BBQ", "create_at": date("2014-05-15"), "description": "I once ate $40 worth of Taco Bell by myself.", "followers_count": 92, "friends_count": 301, "statues_count": 2753 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059823394816, "text": "Clay Township: Temp. 67.3F. (65.7/67.5F) Wind 1 mph SW MaxGust 7mph@02:10. Barometer 29.73 in. Rain today 0.00 in. Humidity 99%. #inwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-86.14833333,39.93138889"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "inwx" }}, "user": { "id": 36190715, "name": "N7CZ", "screen_name": "N7CZ_EM69ww", "lang": "en", "location": "Indianapolis, Indiana", "create_at": date("2009-04-28"), "description": "Terrestrial VHF/UHF Weak Signal & EME Enthusiast", "followers_count": 808, "friends_count": 645, "statues_count": 98436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carmel, IN", "id": "eca35cbd75f0a1e6", "name": "Carmel", "place_type": "city", "bounding_box": rectangle("-86.241245,39.926048 -86.023652,40.000451") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18057, "countyName": "Hamilton", "cityID": 1810342, "cityName": "Carmel" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059915640832, "text": "When you're up, your friends know who you are. When you're down, you know who your true friends are.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1406698189, "name": "Nyj", "screen_name": "thatmillerguy3", "lang": "en", "location": "null", "create_at": date("2013-05-05"), "description": "Chiefs #1|Unfinished Business|I Can Do All Things| *Dreamville* #bigbluenation #hochunk", "followers_count": 476, "friends_count": 928, "statues_count": 4391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351059949182976, "text": "@TheTumbles potbelly?", "in_reply_to_status": 739274474017685504, "in_reply_to_user": 262192344, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 262192344 }}, "user": { "id": 253235193, "name": "Thaddeus the 4th", "screen_name": "TwoLevelsAbove", "lang": "en", "location": "Fairfax VA", "create_at": date("2011-02-16"), "description": "Leader of the fanclub. An amalgamation of websites, hair gel, saltwater fish, ratchetness The Park at 14th Webmaster. Burgundy & gold represent!", "followers_count": 233, "friends_count": 529, "statues_count": 764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fair Oaks, VA", "id": "00a6ce430727193c", "name": "Fair Oaks", "place_type": "city", "bounding_box": rectangle("-77.373778,38.845651 -77.329668,38.8809") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5126875, "cityName": "Fair Oaks" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351060435881984, "text": "@TheRealJorgeM JORGE STOP NO JDNSJ", "in_reply_to_status": 739350069049057280, "in_reply_to_user": 30541715, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 30541715 }}, "user": { "id": 2620433755, "name": "ashlynn ⁷∕₂₇", "screen_name": "floraloctavia", "lang": "en", "location": "hashlynn || l j t j h m s e l ", "create_at": date("2014-07-11"), "description": "murphy nutted and the whole COL got to experience it", "followers_count": 10943, "friends_count": 418, "statues_count": 56877 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351060507164672, "text": "ATTN: \nTHIS is a 3AM tweet❗", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1062225217, "name": "Danna Wells", "screen_name": "dannalynette", "lang": "en", "location": "SavannahGA•FtLaudFL•LexVilleKY", "create_at": date("2013-01-04"), "description": "•Southern #Vegan Democrat•\nRIP@Bellesbattle I❤•myPets•Animals•Family•@UKAthletics & @SEC FTBL! #BBN #Gilbert23\n#recycle♻\n▶Be like @CoryBooker & @LeilaniMunter◀", "followers_count": 4197, "friends_count": 4334, "statues_count": 30035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-06-05T00:00:03.000Z"), "id": 739351060653867008, "text": "Escalators and elevators go down as much as up, but interestingly both are named for the up direction.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 8815922, "name": "P. Oscar Boykin", "screen_name": "posco", "lang": "en", "location": "Maui", "create_at": date("2007-09-11"), "description": "Human @Stripe, formerly @twitter, Programming, Hadoop, Scala, co-author of @scalding, @summingbird, husband of @leezaab, father of @poscito.", "followers_count": 6770, "friends_count": 856, "statues_count": 9220 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lahaina, HI", "id": "249325c821f8b89a", "name": "Lahaina", "place_type": "city", "bounding_box": rectangle("-156.690186,20.855885 -156.657928,20.941656") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15009, "countyName": "Maui", "cityID": 1542950, "cityName": "Lahaina" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351060796473344, "text": "Wind 0.0 mph N. Barometer 29.864 in, Falling slowly. Temperature 68.1 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8945 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351060901429250, "text": "Wind 0.0 mph ---. Barometer 29.785 in, Falling. Temperature 71.0 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351061006147584, "text": "Clayton Kershaw 8-1 after leading shutout against the Braves. #Dodgers https://t.co/ziHPJZNtX0 https://t.co/MOHFlFpI7Q", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dodgers" }}, "user": { "id": 29522397, "name": "AM 570 LA Sports", "screen_name": "AM570LASports", "lang": "en", "location": "Southern California", "create_at": date("2009-04-07"), "description": "Home of the @Dodgers, @LAClippers, @UCLAFootball, and @UCLAMBB. #WeLoveLA #LALovesVin", "followers_count": 16019, "friends_count": 1152, "statues_count": 22234 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rosemead, CA", "id": "fdb4ee1440650043", "name": "Rosemead", "place_type": "city", "bounding_box": rectangle("-118.108242,34.03441 -118.055644,34.095231") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 662896, "cityName": "Rosemead" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351061731770371, "text": "it's my mf birthday ..��☺", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 840935881, "name": "kan", "screen_name": "xokaneret", "lang": "en", "location": "null", "create_at": date("2012-09-22"), "description": "xo", "followers_count": 1482, "friends_count": 246, "statues_count": 35432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351062012776450, "text": "#VenturaLostFound lost mini dauschund brown: Lost short hair brown miniature dauschund.Named Fred. Small 13 lbs.… https://t.co/5d8Me8aahb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-119.23062801,34.27303473"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VenturaLostFound" }}, "user": { "id": 3032747946, "name": "Ventura Lost&Found", "screen_name": "Ventura_LAF", "lang": "en", "location": "Ventura, California", "create_at": date("2015-02-20"), "description": "null", "followers_count": 17, "friends_count": 69, "statues_count": 2974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ventura, CA", "id": "1df98ab7679917e7", "name": "Ventura", "place_type": "city", "bounding_box": rectangle("-119.321696,34.23444 -119.136287,34.344939") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 665042, "cityName": "San Buenaventura (Ventura)" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351062390411264, "text": "16 days till my birthday!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 65799761, "name": "Ericka", "screen_name": "ErickaSkye22", "lang": "en", "location": "Brooklyn", "create_at": date("2009-08-14"), "description": "Three things cannot be long hidden: the sun, the moon and the truth.~Buddha", "followers_count": 161, "friends_count": 243, "statues_count": 1376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Williamsport, PA", "id": "f523bb393a5f747f", "name": "Williamsport", "place_type": "city", "bounding_box": rectangle("-77.144673,41.220627 -76.981651,41.276057") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42081, "countyName": "Lycoming", "cityID": 4285312, "cityName": "Williamsport" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351062679687168, "text": "It's 12am in Cali and highway 91 is still stopped due to traffic.. And I thought okc traffic was bad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 346851468, "name": "Michelangelo", "screen_name": "ThatMurillo", "lang": "en", "location": "Duncan / Okc", "create_at": date("2011-08-01"), "description": "ACM@UCO", "followers_count": 1296, "friends_count": 381, "statues_count": 6956 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351062826520576, "text": "Life don't stop , it keeps going..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 267226160, "name": "Amor' Tai❤", "screen_name": "ImCuteeThoo_", "lang": "en", "location": "Caya Huesoo", "create_at": date("2011-03-16"), "description": "Bastrop✈Dallas insta: Honey_hundred SC: Tai_Griggs #GramFam '19", "followers_count": 875, "friends_count": 750, "statues_count": 62592 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bastrop, LA", "id": "76a017bc522edb14", "name": "Bastrop", "place_type": "city", "bounding_box": rectangle("-91.953951,32.749751 -91.868468,32.803653") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22067, "countyName": "Morehouse", "cityID": 2204685, "cityName": "Bastrop" } }
+{ "create_at": datetime("2016-06-05T00:00:04.000Z"), "id": 739351062960734208, "text": "they say hoes never get cold but here i am cold as balls", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 609873558, "name": "megan", "screen_name": "Megan_beck16", "lang": "en", "location": "Mokena, IL", "create_at": date("2012-06-16"), "description": "null", "followers_count": 291, "friends_count": 372, "statues_count": 3473 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frankfort Square, IL", "id": "63e6ebe8e85057b5", "name": "Frankfort Square", "place_type": "city", "bounding_box": rectangle("-87.824832,41.506533 -87.790248,41.536621") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17197, "countyName": "Will", "cityID": 1727644, "cityName": "Frankfort Square" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713432023269377, "text": "@MrMuselk muscle-k are you gonna be attending vidcon in California?", "in_reply_to_status": -1, "in_reply_to_user": 2161940131, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2161940131 }}, "user": { "id": 1730268878, "name": "jimboman2000", "screen_name": "jimboducksman", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-09-04"), "description": "just an ordinary teen that loves punk rock music and the Anaheim Ducks and don't forget to smile more :)", "followers_count": 68, "friends_count": 170, "statues_count": 1542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Crescenta-Montrose, CA", "id": "33ea9c2aad1edb4c", "name": "La Crescenta-Montrose", "place_type": "city", "bounding_box": rectangle("-118.248823,34.20542 -118.221659,34.252332") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639045, "cityName": "La Crescenta-Montrose" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713432203579392, "text": "!! https://t.co/UmizcwZ5Rj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1720151478, "name": "anthony", "screen_name": "anthoonyy___", "lang": "en", "location": "LA", "create_at": date("2013-09-01"), "description": "XO", "followers_count": 338, "friends_count": 234, "statues_count": 13742 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713432769810432, "text": "okay what in the hell @brazymula @NoBeeetch", "in_reply_to_status": 739712321216352256, "in_reply_to_user": 2756350459, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2756350459, 70237397 }}, "user": { "id": 4688630628, "name": "J O S H", "screen_name": "ThoseCooIKids", "lang": "en", "location": "New Dorp", "create_at": date("2016-01-01"), "description": "⠀⠀But the spirit, the will to excel, the will to ⠀⠀⠀win.. these are the things that endure.", "followers_count": 6851, "friends_count": 209, "statues_count": 41090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433172480000, "text": "@ho_kay24 thanks buddy. You're the best ❤️❤️", "in_reply_to_status": 739712047856787456, "in_reply_to_user": 190884442, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 190884442 }}, "user": { "id": 335792706, "name": "Sammi LeMay", "screen_name": "SammiLeMay", "lang": "en", "location": "null", "create_at": date("2011-07-15"), "description": "You have a voice, you deserve to use it. Never let anyone make you feel like that’s not true.- Hayley Williams", "followers_count": 300, "friends_count": 160, "statues_count": 14018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Vista, NE", "id": "002f085a0f91800d", "name": "La Vista", "place_type": "city", "bounding_box": rectangle("-96.120064,41.164025 -96.005058,41.190874") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31153, "countyName": "Sarpy", "cityID": 3126385, "cityName": "La Vista" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433562710016, "text": "Wind 0.0 mph ---. Barometer 29.79 in, Falling slowly. Temperature 68.7 °F. Rain today 0.00 in. Humidity 63%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 33072 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433604497408, "text": "Body leave marks pon my skin when you miss me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1484176332, "name": "Kurama", "screen_name": "realaddresses", "lang": "en", "location": "Upland, CA", "create_at": date("2013-06-04"), "description": "|Never go back on my word| |Anu❤️|", "followers_count": 3450, "friends_count": 441, "statues_count": 61313 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Upland, CA", "id": "7b2a70199c4d7b6d", "name": "Upland", "place_type": "city", "bounding_box": rectangle("-117.704483,34.086988 -117.619223,34.152304") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 681344, "cityName": "Upland" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433742893056, "text": "How in the actual fuck https://t.co/zfiwh0eWww", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 408079202, "name": "drea", "screen_name": "dreadreadrey", "lang": "en", "location": "san diego", "create_at": date("2011-11-08"), "description": "Hey this is drey *rolls R's*", "followers_count": 452, "friends_count": 399, "statues_count": 30099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433772298240, "text": "@ebbtideapp Tide in Big Salt Lake, Alaska 06/06/2016\nHigh 3:46am 4.5\n Low 11:24am -0.5\nHigh 4:16pm 2.3\n Low 10:33pm 0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-132.95,55.6"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 100, "friends_count": 1, "statues_count": 39017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2016, "countyName": "Aleutians West" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713433969401860, "text": "Good time for these cyatties to make it bling", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385220481, "name": "Lonely Raw", "screen_name": "YaBoiRaw25", "lang": "en", "location": "Fresno, CA", "create_at": date("2011-10-04"), "description": "Where The Wild Things Are.\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nSucker for a girl with a nice smile.\n\n\n\n\n\n\n\n\n\n\n\n\n\n\nSnapchat - YaBoiRaw25", "followers_count": 1156, "friends_count": 762, "statues_count": 53984 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mendota, CA", "id": "407f4285b8cc50e7", "name": "Mendota", "place_type": "city", "bounding_box": rectangle("-120.405019,36.74579 -120.368636,36.772713") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 646828, "cityName": "Mendota" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713434019844097, "text": "7 https://t.co/DLABBq7OwR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 547358113, "name": "Sweet Kayy", "screen_name": "Glam__Kayy", "lang": "en", "location": "Slayland", "create_at": date("2012-04-06"), "description": "I'm the Greatest GOOD you'll ever meet !!\n9.16.15", "followers_count": 552, "friends_count": 385, "statues_count": 4036 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713434451881984, "text": "fucking kanye lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 222581788, "name": "Irahnik ☁️ 6.7.16", "screen_name": "Irahnik", "lang": "en", "location": "NJ", "create_at": date("2010-12-03"), "description": "CLOUDBURST 6.7.16 | mgmt@irahnik.com", "followers_count": 1500, "friends_count": 390, "statues_count": 11642 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dayton, NJ", "id": "be8a4eeb22b509a6", "name": "Dayton", "place_type": "city", "bounding_box": rectangle("-74.539843,40.350999 -74.495005,40.39877") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3416630, "cityName": "Dayton" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713434606931968, "text": "Whenever I say Dream safe it's never when I'm about to go to sleep it's normally just a term of \"I care\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 885757987, "name": "528 Hz", "screen_name": "Apathetic_heatr", "lang": "en", "location": "California", "create_at": date("2012-10-16"), "description": "Professional hitchhiking egg aiming for the wild north // 2 gayest things in life are vaping and @nichellemartens", "followers_count": 178, "friends_count": 293, "statues_count": 7836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinez, CA", "id": "71d33f776fe41dfb", "name": "Martinez", "place_type": "city", "bounding_box": rectangle("-122.157021,37.954027 -122.075217,38.037226") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 646114, "cityName": "Martinez" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713435009617920, "text": "They can't see your tears if you're wearing sunglasses ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 487119794, "name": "Kris", "screen_name": "KrisPNW", "lang": "en", "location": "tacoma :(", "create_at": date("2012-02-08"), "description": "one day I'll be a biologist photographer, for now I'm just a photographer. I keep up a blog with my friends. links below", "followers_count": 208, "friends_count": 146, "statues_count": 14694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tacoma, WA", "id": "2dbbd1540f814343", "name": "Tacoma", "place_type": "city", "bounding_box": rectangle("-122.562215,47.166863 -122.348473,47.319058") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5370000, "cityName": "Tacoma" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713435420655616, "text": "Damn :( https://t.co/p2OyOCdZJR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 314843055, "name": "️", "screen_name": "SHAINEAVELI", "lang": "en", "location": "New Orleans", "create_at": date("2011-06-10"), "description": "KUTTHROAT", "followers_count": 1301, "friends_count": 501, "statues_count": 108047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713435655503872, "text": "@bburget18 @dopehaydo I think it's a certain type of ship probably ��", "in_reply_to_status": 739713255006818304, "in_reply_to_user": 4365096134, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4365096134, 1315760197 }}, "user": { "id": 391857579, "name": "Edward Aumua", "screen_name": "AuMuA54", "lang": "en", "location": "null", "create_at": date("2011-10-15"), "description": "Love is just a word until you put some action into it.| New Mexico Military Institute c/o 2018| Future Navy Sailor", "followers_count": 712, "friends_count": 1523, "statues_count": 8484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713435827503109, "text": "Da hell kind of dick you sucking? https://t.co/iH3kPQ7hKL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 376491731, "name": "Kese", "screen_name": "LastNameArtmore", "lang": "en", "location": "Galveston/Oakfliff", "create_at": date("2011-09-19"), "description": "I only hold the door open for pretty women, ugly bitches better catch that hoe before it close | go through my likes, I'm funny at times #TxSU19", "followers_count": 2982, "friends_count": 1188, "statues_count": 66126 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-06T00:00:00.000Z"), "id": 739713435923976192, "text": "Me n robin do all dat on a regular", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 237956350, "name": "Emeraldkezia", "screen_name": "JustTweetMeHoe", "lang": "en", "location": "New Orleans, LA", "create_at": date("2011-01-13"), "description": "Follow me on instagram && Snapchat: @Emeraldkezia", "followers_count": 274, "friends_count": 240, "statues_count": 6610 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713436477587456, "text": "@ambergerlol ur a hot mom", "in_reply_to_status": 739713092951494657, "in_reply_to_user": 2305330440, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2305330440 }}, "user": { "id": 2604912289, "name": "Bea✨", "screen_name": "beaa_xoxo_", "lang": "en", "location": "south tx", "create_at": date("2014-07-04"), "description": "sbhs c/o '18", "followers_count": 442, "friends_count": 208, "statues_count": 7160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Benito, TX", "id": "b8f3a7fd432ec1f1", "name": "San Benito", "place_type": "city", "bounding_box": rectangle("-97.684281,26.07277 -97.575927,26.172319") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4865036, "cityName": "San Benito" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713436549013504, "text": "Nia 42d me once again tonight ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1644436452, "name": "lil bro reem ひ", "screen_name": "NoLimitKareem", "lang": "en", "location": "Fazoland ", "create_at": date("2013-08-03"), "description": "I Do this for my Momma & My lil Sister & All my Brother's man & everybody I know #Blessed✊.", "followers_count": 931, "friends_count": 276, "statues_count": 11111 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713436557271040, "text": "Been there before���� https://t.co/pof6iiKx7b", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 385166477, "name": "Dez☁️", "screen_name": "Dee_Lively", "lang": "en", "location": "null", "create_at": date("2011-10-04"), "description": "Tryna be living comfortable bout time I'm 25 #SU19", "followers_count": 1695, "friends_count": 1031, "statues_count": 32717 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Iberia, LA", "id": "7d8e6592712895b9", "name": "New Iberia", "place_type": "city", "bounding_box": rectangle("-91.892077,29.941102 -91.731432,30.04583") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22045, "countyName": "Iberia", "cityID": 2254035, "cityName": "New Iberia" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713436691628032, "text": "A nigga just randomly woke up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 376484438, "name": "Pasto", "screen_name": "TheChum15", "lang": "en", "location": "Brockton #508", "create_at": date("2011-09-19"), "description": "Created in Nigeria| Umass Boston| Information Technology Major| IG: thechum15| SC: chumaonocheese|", "followers_count": 859, "friends_count": 479, "statues_count": 57575 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brockton, MA", "id": "2c1b499801cd0ef4", "name": "Brockton", "place_type": "city", "bounding_box": rectangle("-71.080136,42.042695 -70.973413,42.126438") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25023, "countyName": "Plymouth", "cityID": 2509000, "cityName": "Brockton" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713436766986240, "text": "@SandraLeeMD @mikeyaaal", "in_reply_to_status": 733371946042314752, "in_reply_to_user": 372971561, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 372971561, 3029732511 }}, "user": { "id": 2654252927, "name": "kool kat", "screen_name": "casbri15", "lang": "en", "location": "null", "create_at": date("2014-06-29"), "description": "15! legitnessss. Optimistic!", "followers_count": 222, "friends_count": 270, "statues_count": 947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713437052370944, "text": "I just got resurrected wtf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1212847908, "name": "Floozie", "screen_name": "SinnerOfTheSea", "lang": "en", "location": "Cape Coral, FL", "create_at": date("2013-02-23"), "description": "There is no God. \nEither you slinging crack rock or you got a wicked jump shot.", "followers_count": 464, "friends_count": 398, "statues_count": 9950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cape Coral, FL", "id": "fbd1502e5edce6bf", "name": "Cape Coral", "place_type": "city", "bounding_box": rectangle("-82.069567,26.531814 -81.904101,26.741146") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1210275, "cityName": "Cape Coral" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713437136195584, "text": "Temp 63.1° Hi/Lo 63.2/61.4 Rng 1.8° WC 63.1° Hmd 97% Rain 0.00\" Storm 0.71\" BAR 29.616 Falling DP 62.2° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 74, "friends_count": 123, "statues_count": 19280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713437387743233, "text": "Got me feeling like drake and Rihanna I'm tryna give you that work ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1613678047, "name": "LongLiveJoJo❄️", "screen_name": "Taybaandz", "lang": "en", "location": "Berkeley, CA", "create_at": date("2013-07-22"), "description": "$elf made $elf paid ️Dooski World", "followers_count": 1463, "friends_count": 1291, "statues_count": 26566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, CA", "id": "029dd1f28dbd3d26", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-122.420497,37.895308 -122.304331,38.002378") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 660620, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713437895299072, "text": "@JustNardos Hey #Dear check this out u will really change ur mind https://t.co/c1FrBd0sJv", "in_reply_to_status": 739712917310824448, "in_reply_to_user": 422179741, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Dear" }}, "user_mentions": {{ 422179741 }}, "user": { "id": 4874391419, "name": "Diana Larose", "screen_name": "larose2212", "lang": "en", "location": "Oregon, USA", "create_at": date("2016-02-03"), "description": "Proud introvert. Freelance communicator. Devoted entrepreneur. Troublemaker. Unable to type with boxing gloves on.", "followers_count": 5, "friends_count": 17, "statues_count": 116 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438046261249, "text": "I guess I really wanted to win, also lol at me at 14 https://t.co/0kmuNpskGP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 17692803, "name": "Paula❤", "screen_name": "paulapearl", "lang": "en", "location": "California State of Mind", "create_at": date("2008-11-27"), "description": "a collection of Paula comments #paulacomments #lifeofpaula • snapchat; pawlah • Instagram; paulapearl", "followers_count": 1630, "friends_count": 637, "statues_count": 17793 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438117683200, "text": "Work at 4 tomorrow let's get it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 401246863, "name": "ci§co", "screen_name": "Styulz", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2011-10-30"), "description": "IG @ styulz", "followers_count": 930, "friends_count": 453, "statues_count": 74037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438193045505, "text": "I hate being told something and then it not happing but I know everyone hates it haha", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2264244026, "name": "Rory.", "screen_name": "roryreyess", "lang": "en", "location": "null", "create_at": date("2013-12-27"), "description": "null", "followers_count": 341, "friends_count": 192, "statues_count": 12492 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McAllen, TX", "id": "6d032cd232fb02d5", "name": "McAllen", "place_type": "city", "bounding_box": rectangle("-98.295824,26.143437 -98.195525,26.337943") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4845384, "cityName": "McAllen" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438193057792, "text": "Do I watch football or basketball ?? No ... https://t.co/5Ycx5M3FpU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 709535678535704576, "name": "dont!dm!me", "screen_name": "maj_majj", "lang": "en", "location": "withmyman❤️", "create_at": date("2016-03-14"), "description": "null", "followers_count": 414, "friends_count": 373, "statues_count": 13080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marrero, LA", "id": "2bca99f2a5ba7da1", "name": "Marrero", "place_type": "city", "bounding_box": rectangle("-90.139232,29.853716 -90.08087,29.910044") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2248785, "cityName": "Marrero" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438251900928, "text": "I'm settle https://t.co/vM5HL21AoA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3041943916, "name": "Tatyana", "screen_name": "_1tatyana", "lang": "en", "location": "Columbus, GA", "create_at": date("2015-02-16"), "description": "I keep my eyes always on the Lord. With him at my right hand, I will not be shaken. - Psalm 16:8", "followers_count": 920, "friends_count": 657, "statues_count": 8567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, GA", "id": "00b673715a35dfa7", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-85.038673,32.373847 -84.795941,32.59076") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13215, "countyName": "Muscogee", "cityID": 1319000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438579068928, "text": "Being lonely sucks. Pero ya que cele puede aser naci feo. ������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 803262673, "name": "jose_elboy", "screen_name": "joseEzarate", "lang": "en", "location": "el infierno ", "create_at": date("2012-09-04"), "description": "just a simple ugly mf| Michoacano (la finca & la chole) |17♎️| BHSD228| single| it's LIT like CLIT", "followers_count": 309, "friends_count": 474, "statues_count": 2260 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Posen, IL", "id": "109342af5712414e", "name": "Posen", "place_type": "city", "bounding_box": rectangle("-87.698452,41.619134 -87.674613,41.640875") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1761314, "cityName": "Posen" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438704898048, "text": "06/06@03:00 - Temp 66.3F, WC 66.3F. Wind 0.0mph ---, Gust 0.0mph. Bar 29.688in, Falling slowly. Rain 0.00in. Hum 100%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 65, "friends_count": 8, "statues_count": 49470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438746836992, "text": "Wind 0.0 mph SE. Barometer 29.825 in, Steady. Temperature 66.1 °F. Rain today 0.00 in. Humidity 75%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113758 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713438977544192, "text": "������ https://t.co/eL1xVznxvn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 618881643, "name": "Papi Chulo", "screen_name": "tysonhenderson2", "lang": "en", "location": "null", "create_at": date("2012-06-25"), "description": "I feel like Juelz Santana/ William penn '20", "followers_count": 1237, "friends_count": 840, "statues_count": 32981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rantoul, IL", "id": "007840b74c9b6486", "name": "Rantoul", "place_type": "city", "bounding_box": rectangle("-88.183869,40.274551 -88.121059,40.32669") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1762783, "cityName": "Rantoul" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439023652864, "text": "https://t.co/6iKusdXgDV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1678763568, "name": "Reezy", "screen_name": "ddrzzyy", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-08-17"), "description": "null", "followers_count": 629, "friends_count": 156, "statues_count": 29004 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Burbank, IL", "id": "48f2609344f0c3f7", "name": "Burbank", "place_type": "city", "bounding_box": rectangle("-87.799908,41.733586 -87.740829,41.757299") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1709642, "cityName": "Burbank" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439128551424, "text": "Why am I acting like it's not 3am rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 506662027, "name": "Katie", "screen_name": "Katieeee_K", "lang": "en", "location": "null", "create_at": date("2012-02-27"), "description": "New York SJU", "followers_count": 311, "friends_count": 677, "statues_count": 11011 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Staten Island, NY", "id": "00c55f041e27dc51", "name": "Staten Island", "place_type": "city", "bounding_box": rectangle("-74.255641,40.495865 -74.052253,40.648887") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36085, "countyName": "Richmond", "cityID": 36085, "cityName": "Staten Island" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439292002304, "text": "I have b4. https://t.co/M1uh4mhOeS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2921291689, "name": "Daddys Princess✨", "screen_name": "Hotcommodity___", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-12-06"), "description": "AnthonysCrushEveryday❣", "followers_count": 234, "friends_count": 191, "statues_count": 5235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439782731777, "text": "Having a hard time falling asleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1447697407, "name": "JENNI SWS FAN", "screen_name": "Lazarofan4life", "lang": "en", "location": "null", "create_at": date("2013-05-21"), "description": "Sam faved x51 Sam tweeted me x8 met kellin quinn 10/30/15 @Kellinquinn RTED me X5 @samwoolfmusic rted me Oct 26th 2015 jack June 1st 4/5 of SWS has notice!", "followers_count": 1430, "friends_count": 2026, "statues_count": 14545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Hills, OR", "id": "7116080c8926fb6b", "name": "Oak Hills", "place_type": "city", "bounding_box": rectangle("-122.857378,45.527087 -122.824348,45.555039") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4153988, "cityName": "Oak Hills" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439787057152, "text": "Ripley SW Limestone Co. Temp: 69.8°F Wind:0.0mph Pressure: 991.2mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 54173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439849844736, "text": "@princeralh24 put it in a song", "in_reply_to_status": 739713368622125056, "in_reply_to_user": 1835637140, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1835637140 }}, "user": { "id": 2902032634, "name": "3️⃣", "screen_name": "nlucero02", "lang": "en", "location": "Sacramento, CA", "create_at": date("2014-12-01"), "description": "AHS QB '19⠀#Team3stripe T R E N D S E T T E R", "followers_count": 371, "friends_count": 340, "statues_count": 1832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Antelope, CA", "id": "013214de57b4da2e", "name": "Antelope", "place_type": "city", "bounding_box": rectangle("-121.403842,38.702271 -121.30897,38.728927") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 602210, "cityName": "Antelope" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439870787586, "text": "WHBM #Retail #Job: Support Associate (#Austin, TX) https://t.co/A8OffAtUHN #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.8069052,30.2576953"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Retail", "Job", "Austin", "Jobs", "Hiring" }}, "user": { "id": 2798308393, "name": "WHBM Careers", "screen_name": "WHBMCareers", "lang": "en", "location": "null", "create_at": date("2014-09-08"), "description": "WH|BM offers fashionable and sophisticated clothing and accessory items. Our passion has always been to make women feel beautiful. That’s what we’re about.", "followers_count": 150, "friends_count": 0, "statues_count": 9826 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-06-06T00:00:01.000Z"), "id": 739713439874965504, "text": "i wish ppl were more positive��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 125406336, "name": "e m i l y ♏️", "screen_name": "_emilyvalentine", "lang": "en", "location": "on a private jet with el chapo", "create_at": date("2010-03-22"), "description": "PLUR | snapchat: emily_valentine | instagram: emilybrooke.96", "followers_count": 1296, "friends_count": 1365, "statues_count": 61741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lindale, TX", "id": "016b03c63efc8045", "name": "Lindale", "place_type": "city", "bounding_box": rectangle("-95.426444,32.483363 -95.368483,32.541786") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48423, "countyName": "Smith", "cityID": 4842820, "cityName": "Lindale" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713440420270081, "text": "I need it all right now", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2760280710, "name": "jasmine", "screen_name": "__jasmine13", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-08-23"), "description": "me, I'm just done done done done", "followers_count": 292, "friends_count": 273, "statues_count": 3822 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713440474890240, "text": "EVERYTHING �� https://t.co/9hDltT7WW8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 615633143, "name": "BTW.. Prototype✨", "screen_name": "Raysheoo", "lang": "en", "location": "Chasin My Dreams.....", "create_at": date("2012-06-22"), "description": "People who are intimidated by you talk bad about you with hopes that others wont find you so appealing... | Forever Focused✨", "followers_count": 2323, "friends_count": 1710, "statues_count": 32928 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownsville, FL", "id": "3d26878d18961c56", "name": "Brownsville", "place_type": "city", "bounding_box": rectangle("-80.257366,25.810836 -80.2252,25.831765") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1209000, "cityName": "Brownsville" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713440642662400, "text": "Wind 0.0 mph ---. Barometer 29.822 in, Falling slowly. Temperature 59.4 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713440692862976, "text": "Happy birthday kid. Much blessings your ways. We'll celebrate when I get back boii! https://t.co/m4ccm3DDIW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2196916440, "name": "Angel Jonathan Lopez", "screen_name": "angeljonathan09", "lang": "en", "location": "Sacramento/Santa Maria, CA", "create_at": date("2013-11-15"), "description": "ΛΘΦ | Sacramento State", "followers_count": 216, "friends_count": 346, "statues_count": 3674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Maria, CA", "id": "dee4ad8775ff8102", "name": "Santa Maria", "place_type": "city", "bounding_box": rectangle("-120.482386,34.875868 -120.356782,34.988866") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669196, "cityName": "Santa Maria" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441397514241, "text": "dont care for boasting", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1521865646, "name": "J.Hud", "screen_name": "HudMoney_", "lang": "en", "location": "screetz", "create_at": date("2013-06-16"), "description": "ive changed.. a lot!!", "followers_count": 466, "friends_count": 164, "statues_count": 23376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beloit, WI", "id": "000fc3e67b627897", "name": "Beloit", "place_type": "city", "bounding_box": rectangle("-89.079237,42.496019 -88.948817,42.610064") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55105, "countyName": "Rock", "cityID": 5506500, "cityName": "Beloit" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441443643392, "text": "Me gusta amanecer pensando que me quieres", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2342756174, "name": "Mitchell", "screen_name": "Michelle1999___", "lang": "en", "location": "Quincy, WA", "create_at": date("2014-02-13"), "description": "my bestfriends problems are my problems.", "followers_count": 211, "friends_count": 216, "statues_count": 3064 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Quincy, WA", "id": "e993fc6ac20d853d", "name": "Quincy", "place_type": "city", "bounding_box": rectangle("-119.893576,47.219448 -119.813649,47.248521") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53025, "countyName": "Grant", "cityID": 5357115, "cityName": "Quincy" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441561202688, "text": "I front like no other", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1656817339, "name": "alyssa", "screen_name": "_alyssasotelo", "lang": "en", "location": "vegas✨", "create_at": date("2013-08-08"), "description": "Lil rebels softball, LHS '16 / BC '20 ⚾️ sc: alyssasotelo1d", "followers_count": 1249, "friends_count": 1048, "statues_count": 33099 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441582063616, "text": "figured out this the girl that be stalking me and I gotta lie my ass off cause you is not gone fwh https://t.co/oNeUPhtf9I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2352558042, "name": "✝FiftyShadesOfGay✝", "screen_name": "Demericka_", "lang": "en", "location": "somewhere being gay ", "create_at": date("2014-02-19"), "description": "17♏️| vibes | gay☮| jhene aiko❤️| damaged not broken ⏳", "followers_count": 755, "friends_count": 1113, "statues_count": 5063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rockford, IL", "id": "d4e703056914a3eb", "name": "Rockford", "place_type": "city", "bounding_box": rectangle("-89.173876,42.171924 -88.861257,42.342367") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1765000, "cityName": "Rockford" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441754189824, "text": "@lzzmills oh my", "in_reply_to_status": 739711792700608512, "in_reply_to_user": 57867801, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 57867801 }}, "user": { "id": 88552382, "name": "Fidel Chugainov", "screen_name": "papercuppoonie", "lang": "en", "location": "�T: 30.82565,-83.27421", "create_at": date("2009-11-08"), "description": "#GoNoles #GoMountaineers #GoEagles #GoBroncos #Gospurs #GoMagic #GoDuke", "followers_count": 2049, "friends_count": 711, "statues_count": 555172 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valdosta, GA", "id": "5e1c91065bc30991", "name": "Valdosta", "place_type": "city", "bounding_box": rectangle("-83.374825,30.762483 -83.230199,30.949686") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13185, "countyName": "Lowndes", "cityID": 1378800, "cityName": "Valdosta" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713441800302592, "text": "We love you jeeesicahh16 !!! Congratulations graduate!!! #LLA16 @… https://t.co/JO4jCjbpt5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.26271689,34.05958916"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "LLA16" }}, "user": { "id": 72442959, "name": "Arvin J. Macalintal", "screen_name": "AyeJayAJ", "lang": "en", "location": "Redlands, CA, USA", "create_at": date("2009-09-07"), "description": "Medical Radiorography & Music Ministries. #LakerNation #ThankYouKobe", "followers_count": 521, "friends_count": 400, "statues_count": 19865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Loma Linda, CA", "id": "f6abc2be7eacfe4e", "name": "Loma Linda", "place_type": "city", "bounding_box": rectangle("-117.277932,34.032777 -117.218843,34.066725") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 642370, "cityName": "Loma Linda" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442316091392, "text": "@xxItzKTxx ask ask", "in_reply_to_status": 739688006567628800, "in_reply_to_user": 2272352863, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2272352863 }}, "user": { "id": 2272352863, "name": "KaityKush", "screen_name": "xxItzKTxx", "lang": "en", "location": "null", "create_at": date("2014-01-01"), "description": "☞ SC/IG: kaitynua || ♛ **SoCal**", "followers_count": 10347, "friends_count": 527, "statues_count": 2764 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442404261888, "text": "Temp: 72.6°F Wind:0.0mph Pressure: 29.849hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66252 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442462863362, "text": "@drizzyxmvmi ��", "in_reply_to_status": 739713100174086144, "in_reply_to_user": 722288750869352448, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 722288750869352448 }}, "user": { "id": 1228694946, "name": "Tones Vasquez", "screen_name": "antvqz_53", "lang": "en", "location": "Alhambra, CA", "create_at": date("2013-02-28"), "description": "l", "followers_count": 350, "friends_count": 482, "statues_count": 13767 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alhambra, CA", "id": "d563cf7307e4ba95", "name": "Alhambra", "place_type": "city", "bounding_box": rectangle("-118.165119,34.05998 -118.108233,34.111213") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 600884, "cityName": "Alhambra" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442592890880, "text": "wish him the best!!! �� https://t.co/wPgry952Xp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2790710820, "name": "lèsss", "screen_name": "lesllymoniquee", "lang": "en", "location": "Reedley, CA", "create_at": date("2014-09-04"), "description": "extremely blessed.", "followers_count": 966, "friends_count": 552, "statues_count": 21963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reedley, CA", "id": "f434511e8f0c8aa4", "name": "Reedley", "place_type": "city", "bounding_box": rectangle("-119.4699,36.575137 -119.421897,36.618737") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 660242, "cityName": "Reedley" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442773340160, "text": "Temp 63.6°F Heat Index 63.6 °F RH 94% Wind 0.3 WNW Gust 2.0 WNW SLP 29.813 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 110, "friends_count": 62, "statues_count": 35963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713442886516736, "text": "@iam_ginghs @mhellopez19 @akosibuttman218 @yaylips @fejosede2 @kryptnfghtr Bwahaha lakas pala mang as-- ni ate Ging! Opo! #ALDUBLetUsKnow ��", "in_reply_to_status": 739712506399068160, "in_reply_to_user": 4127789114, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "hashtags": {{ "ALDUBLetUsKnow" }}, "user_mentions": {{ 4127789114, 3688370840, 1162063176, 3701652018, 3670829834, 4716126102 }}, "user": { "id": 452869884, "name": "MGA OSONG KEBABS", "screen_name": "Samemknox", "lang": "en", "location": "California, USA", "create_at": date("2012-01-02"), "description": "Team No Group, constantly RT/QT ALDUB entries, ALWAYS Happy Happy Joy Joy! OSO kasi Hindi baduy, KEBS (no pake) + BABS (Gurl!) = KEBABS gets?!", "followers_count": 407, "friends_count": 616, "statues_count": 22863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rocklin, CA", "id": "c98b6d080d712840", "name": "Rocklin", "place_type": "city", "bounding_box": rectangle("-121.312069,38.769833 -121.189258,38.840837") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662364, "cityName": "Rocklin" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443075260416, "text": "@Bee_Low @Envydanneh wait what happen", "in_reply_to_status": 739699350717104128, "in_reply_to_user": 226430105, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 226430105, 634754145 }}, "user": { "id": 86423348, "name": "D ✨", "screen_name": "Foxxxybrown_", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2009-10-30"), "description": "I'm a Brat & c/o 2016", "followers_count": 2880, "friends_count": 1473, "statues_count": 85525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Richland Hills, TX", "id": "c79a0302b462ea2f", "name": "North Richland Hills", "place_type": "city", "bounding_box": rectangle("-97.264529,32.808748 -97.184107,32.912026") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4852356, "cityName": "North Richland Hills" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443524005888, "text": "sometimes I don't even notice �� https://t.co/044dkbmfVe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2792122316, "name": "caitlyn corrine", "screen_name": "catcorrinec", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-09-05"), "description": "beautiful and masochistic. leo. untz untz untz untz", "followers_count": 208, "friends_count": 170, "statues_count": 288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443578580993, "text": "USGS reports a M1.14 #earthquake 9km NW of The Geysers, California on 6/6/16 @ 6:56:30 UTC https://t.co/0zMIzE6ziD #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8410034,38.8243332"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 1414684496, "name": "Every Earthquake", "screen_name": "everyEarthquake", "lang": "en", "location": "Earth", "create_at": date("2013-05-08"), "description": "Tweeting every earthquake occurrence reported by USGS. Built and maintained by David Barkman aka @cybler.", "followers_count": 6040, "friends_count": 17, "statues_count": 158007 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443670827009, "text": "@agoodwincollect thanks for following me on Twitter, man", "in_reply_to_status": -1, "in_reply_to_user": 219520456, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 219520456 }}, "user": { "id": 72375227, "name": "Jerrid Geron", "screen_name": "Weaponx3", "lang": "en", "location": "null", "create_at": date("2009-09-07"), "description": "null", "followers_count": 62, "friends_count": 184, "statues_count": 170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CO", "id": "f7eb2fa2fea288b1", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-105.193475,39.60973 -105.053164,39.761974") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8059, "countyName": "Jefferson", "cityID": 843000, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443771490308, "text": "@StephenCurry30 dishes it to @Money23Green https://t.co/vAXHcyEr9s", "in_reply_to_status": -1, "in_reply_to_user": 42562446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 42562446, 116485600 }}, "user": { "id": 2365277576, "name": "SFGFAN (35-24) JULIO", "screen_name": "locosfgfan", "lang": "en", "location": "null", "create_at": date("2014-02-28"), "description": "SF Giants Fan born and raised! I am @BusterPosey's good friend @jcdsuper #bayareaunite #sharks #warriors #earthquakes #Athletics #fortyniners", "followers_count": 1355, "friends_count": 2842, "statues_count": 26153 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, CA", "id": "694adcf0dd2558cb", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-122.073103,37.494635 -121.987627,37.563477") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 650916, "cityName": "Newark" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443968614400, "text": "Lol yeah I just need this school year to be over already https://t.co/3IZBvL05JJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163755160, "name": "R⃣O⃣N⃣", "screen_name": "DeronAntonio", "lang": "en", "location": "Some Where In Seattle ☔", "create_at": date("2010-07-06"), "description": "Currently... Im High off Knowledge, Overdosing on Confidence & Drunk off Ideas", "followers_count": 1642, "friends_count": 1015, "statues_count": 75999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713443998109696, "text": "Bout Ta Go Link Up Wit My Day 1's From New Yawk This Week ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3354277060, "name": "YoungOG", "screen_name": "bapo125", "lang": "en", "location": "New Jersey, USA", "create_at": date("2015-07-01"), "description": "Dominican King From Uptown IG: Upt125 SC:Bangbangupt125", "followers_count": 33, "friends_count": 238, "statues_count": 611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Brunswick, NJ", "id": "c47efae2a73aef75", "name": "New Brunswick", "place_type": "city", "bounding_box": rectangle("-74.489529,40.467252 -74.39268,40.509138") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3451210, "cityName": "New Brunswick" } }
+{ "create_at": datetime("2016-06-06T00:00:02.000Z"), "id": 739713444144812036, "text": "https://t.co/dVoLHYG0gl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1033072111, "name": "Christian Cartier", "screen_name": "DLuxChristian", "lang": "en", "location": "SRHS", "create_at": date("2012-12-24"), "description": "Living the Aquarius lifestyle ♒️ Charming Snakes since 97'", "followers_count": 648, "friends_count": 538, "statues_count": 6370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713444669100032, "text": "��- you know my cousin, you're a good hurdler����wish I was like that! & you seem cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2419802768, "name": "jaz", "screen_name": "Jazalynntaylor", "lang": "en", "location": "null", "create_at": date("2014-03-30"), "description": "null", "followers_count": 848, "friends_count": 461, "statues_count": 37886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713444794896384, "text": "USGS reports a M1.14 #earthquake 9km NW of The Geysers, California on 6/6/16 @ 6:56:30 UTC https://t.co/oeo6wTO8IK #quake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.8410034,38.8243332"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "earthquake", "quake" }}, "user": { "id": 2654450299, "name": "NorCal Earthquakes", "screen_name": "NorCalEq", "lang": "en", "location": "null", "create_at": date("2014-07-17"), "description": "null", "followers_count": 103, "friends_count": 0, "statues_count": 11510 }, "place": { "country": "United States", "country_code": "United States", "full_name": "California, USA", "id": "fbd6d2f5a4e4a15e", "name": "California", "place_type": "admin", "bounding_box": rectangle("-124.482003,32.528832 -114.131212,42.009519") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713444920713216, "text": "Wind 0.0 mph ---. Barometer 29.882 in, Rising slowly. Temperature 65.5 °F. Rain today 0.00 in. Humidity 74%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 8969 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713444958474240, "text": "@Ihtssashleeey__ party. that's what you're gunna do", "in_reply_to_status": 739710075644694529, "in_reply_to_user": 2212821644, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2212821644 }}, "user": { "id": 2683051687, "name": "Dominique", "screen_name": "domminiqquuee", "lang": "en", "location": "bayy areaaa", "create_at": date("2014-07-26"), "description": "lightly salted", "followers_count": 833, "friends_count": 630, "statues_count": 24606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713445046558720, "text": "It's too hot in my room��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3238340666, "name": "Alex V☁️", "screen_name": "Avillarreal52", "lang": "en", "location": "2021 NFL Draft ", "create_at": date("2015-06-06"), "description": "POLY FOOTBALL", "followers_count": 605, "friends_count": 401, "statues_count": 4235 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713445046702080, "text": "Goodnight Twitter-o-Pals", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 323484814, "name": "The Rapopath", "screen_name": "AllHailGeemoney", "lang": "en", "location": "null", "create_at": date("2011-06-24"), "description": "#LongLiveGwendolyn! Enjoy heaven my love", "followers_count": 424, "friends_count": 582, "statues_count": 14464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713445268852737, "text": "@dihorchata She good or nah ��", "in_reply_to_status": 739711608310489088, "in_reply_to_user": 3766421174, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3766421174 }}, "user": { "id": 141430743, "name": "MR.BLINKZ", "screen_name": "Robertorh29", "lang": "es", "location": "null", "create_at": date("2010-05-07"), "description": "•IG:Robertorh29\n•Snapchat:Robertrh\n•DJ/producer\n•Gym Rat\n•Fitness Enthusiast\n•HDYNTN •BFM •ASOC •DTG\n•Anatomy/Kineseology", "followers_count": 307, "friends_count": 404, "statues_count": 11340 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713445759574018, "text": "I'm hungry :/", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3287937728, "name": "AmberMarie", "screen_name": "amber_ce", "lang": "en", "location": "null", "create_at": date("2015-07-22"), "description": "null", "followers_count": 475, "friends_count": 365, "statues_count": 3369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713446539894788, "text": "@shorty123981 were u at time for tha gym progress 33 today", "in_reply_to_status": 739693601362415616, "in_reply_to_user": 399436420, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 399436420 }}, "user": { "id": 116971285, "name": "Real†HA MASK ®APPER™", "screen_name": "RealTMR4Life", "lang": "en", "location": "God Made Me , To Be Great !", "create_at": date("2010-02-23"), "description": "#Godsoilder #Rapper #GraphicArtsDesigner #CPR #Fireguard #Straightedge #MaskMovement + #MaskMovementClub #TattooBandit #Pisces #Ravens (50 Talents & Gifts)", "followers_count": 272, "friends_count": 8, "statues_count": 34391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713446883688448, "text": "Temp: 74.7°F | Humidity: 99% | Wind: --- @ 0.0 mph | Barometer: 29.92 in | Dewpoint: 74.4°F #weather #tampabay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.32055556,28.05833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "tampabay" }}, "user": { "id": 1052017520, "name": "Thonotosassa Weather", "screen_name": "thonotosassawx", "lang": "en", "location": "Thonotosassa, Florida", "create_at": date("2012-12-31"), "description": "This is an automated twitter feed providing current conditions from Thonotosassa, Florida on the hour. Please note that this account is unmanned at most times.", "followers_count": 25, "friends_count": 1, "statues_count": 167644 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Thonotosassa, FL", "id": "01afc45d690c1f0f", "name": "Thonotosassa", "place_type": "city", "bounding_box": rectangle("-82.345013,28.00226 -82.285139,28.079534") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12057, "countyName": "Hillsborough", "cityID": 1271725, "cityName": "Thonotosassa" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713447810732032, "text": "@mailala_ ��", "in_reply_to_status": 739712172066897920, "in_reply_to_user": 2236699978, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2236699978 }}, "user": { "id": 269200310, "name": "JayZOverrated", "screen_name": "JayZOverrated", "lang": "en", "location": "Houma, LA", "create_at": date("2011-03-20"), "description": "null", "followers_count": 9223, "friends_count": 849, "statues_count": 349858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bayou Cane, LA", "id": "b5b20fbf38f349af", "name": "Bayou Cane", "place_type": "city", "bounding_box": rectangle("-90.810681,29.591044 -90.691976,29.661513") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22109, "countyName": "Terrebonne", "cityID": 2205210, "cityName": "Bayou Cane" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713447852707840, "text": "Wind 0.7 mph S. Barometer 29.61 in, Rising slowly. Temperature 64.2 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 160329 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713448066437120, "text": "and somehow people still use fear of explaining lgbtq+*to their children as an excuse to discriminate https://t.co/id59QObrjO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 483509614, "name": "T", "screen_name": "peaachhyy", "lang": "en", "location": "ks", "create_at": date("2012-02-04"), "description": "good ol' fashioned meme", "followers_count": 712, "friends_count": 154, "statues_count": 23478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wichita, KS", "id": "1661ada9b2b18024", "name": "Wichita", "place_type": "city", "bounding_box": rectangle("-97.534906,37.562483 -97.152924,37.80531") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2079000, "cityName": "Wichita" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713448171470848, "text": "What EXACTLY do they mean by 'Seek shelter now'? Like bomb shelter or just like a blue tarpaulin? https://t.co/Kteoc2mePI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 24158243, "name": "Michael Good", "screen_name": "michaeljgood", "lang": "en", "location": "ÜT: 41.810171,-72.252062", "create_at": date("2009-03-13"), "description": "I'm a slave to many things.", "followers_count": 122, "friends_count": 112, "statues_count": 7449 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simpsonville, SC", "id": "c200b23c6b2eb238", "name": "Simpsonville", "place_type": "city", "bounding_box": rectangle("-82.356456,34.651362 -82.199237,34.786946") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45045, "countyName": "Greenville", "cityID": 4566580, "cityName": "Simpsonville" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713448318111745, "text": "@AgentOfSpies And yet another ignorant attack. FYI, I'm not even a Jew. You continue to show your lack of character.", "in_reply_to_status": 739709584013721600, "in_reply_to_user": 2470585471, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2470585471 }}, "user": { "id": 26675484, "name": "Barry E Newell", "screen_name": "KSigMason", "lang": "en", "location": "Boise, ID", "create_at": date("2009-03-25"), "description": "Founder of Traveling #Templar and Idaho Mason. I Tweet about #History, Int'l Affairs, Defense policy, #Freemasonry, and #Templary. RT is not an endorsement", "followers_count": 972, "friends_count": 1092, "statues_count": 14718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meridian, ID", "id": "afc5c9ce888b958d", "name": "Meridian", "place_type": "city", "bounding_box": rectangle("-116.461472,43.561187 -116.337922,43.662958") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16001, "countyName": "Ada", "cityID": 1652120, "cityName": "Meridian" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713448410419200, "text": "So I'm supposed to be an adult now? Well that sucks. 18.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2392171951, "name": "Julius", "screen_name": "PrimitiveHess", "lang": "en", "location": "San Diyayo", "create_at": date("2014-03-15"), "description": "I don't trust young metro", "followers_count": 77, "friends_count": 118, "statues_count": 979 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-06T00:00:03.000Z"), "id": 739713448506843136, "text": "I got the juice.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 187655879, "name": "S.R. ✨", "screen_name": "JMuvaa", "lang": "en", "location": "Big Eazy ⚜", "create_at": date("2010-09-06"), "description": "null", "followers_count": 1212, "friends_count": 841, "statues_count": 51703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Metairie, LA", "id": "743df94d8dcb69a6", "name": "Metairie", "place_type": "city", "bounding_box": rectangle("-90.237612,29.968359 -90.119647,30.034129") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2250115, "cityName": "Metairie" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713448813027329, "text": "Wind 0.0 mph ---. Barometer 29.797 in, Steady. Temperature 72.4 °F. Rain today 0.00 in. Humidity 96%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66253 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713449001787392, "text": "@hannahmicheIIe Get a fan ����", "in_reply_to_status": 739702410138918912, "in_reply_to_user": 3322171807, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3322171807 }}, "user": { "id": 421739985, "name": "△▵HΩVa JΔmes△▵", "screen_name": "BelieveJdreams", "lang": "en", "location": "Lakelandhills // Auburn, WA", "create_at": date("2011-11-26"), "description": "Dream Chasing.. Pierce College // Puyallup △ △ I'm Ike Turner Turn Up Baby No I Don't Play. △ http://www.Instagram.com/Jam_harris *FITCLIQUE*", "followers_count": 444, "friends_count": 313, "statues_count": 4118 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Tapps, WA", "id": "0111ffcc55a25eb9", "name": "Lake Tapps", "place_type": "city", "bounding_box": rectangle("-122.229539,47.205593 -122.113034,47.25784") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5337920, "cityName": "Lake Tapps" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713449131966464, "text": "@Targetgoboom @EKercsmar toughening you up kiddo. Too soft for this world.", "in_reply_to_status": 739713305711779840, "in_reply_to_user": 351445248, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 351445248, 1463185651 }}, "user": { "id": 337623933, "name": "Lex Brunelle", "screen_name": "lxslynn", "lang": "en", "location": "Alaska - PDX", "create_at": date("2011-07-18"), "description": "Alaska Grown • Portland State University •", "followers_count": 355, "friends_count": 283, "statues_count": 6508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713449660321794, "text": "������ marry me https://t.co/LtdRziZOqS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1267631432, "name": "Eleven™", "screen_name": "moncerrathh05", "lang": "en", "location": "null", "create_at": date("2013-03-14"), "description": "✨I AM THE GOD❄ WEKIH WEKIH WEKIH", "followers_count": 389, "friends_count": 369, "statues_count": 14014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alum Rock, CA", "id": "277b4360183b5d75", "name": "Alum Rock", "place_type": "city", "bounding_box": rectangle("-121.842175,37.35398 -121.812158,37.38102") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 601458, "cityName": "Alum Rock" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450050347008, "text": "Single https://t.co/pAhdU7fqBp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2869672042, "name": "RIP Papa⁶", "screen_name": "baasicIT", "lang": "en", "location": "null", "create_at": date("2014-11-09"), "description": "Senior Year '16// Los Angeles, CA// CSUN'20", "followers_count": 566, "friends_count": 510, "statues_count": 10073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450121662465, "text": "Archer Vice? Hohohoho", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1392730668, "name": "Matt", "screen_name": "mattiida24", "lang": "en", "location": "null", "create_at": date("2013-04-30"), "description": "On your left", "followers_count": 440, "friends_count": 378, "statues_count": 3746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450218160128, "text": "@Rahi_Patel97 @sofiaaaolveraaa \n\n-❤\n-Cutie pie\n-MK\n-Shopping 24/7\n-#GoBaylor ��", "in_reply_to_status": 739687321092558848, "in_reply_to_user": 2240192282, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GoBaylor" }}, "user_mentions": {{ 2240192282, 1969658756 }}, "user": { "id": 2240192282, "name": "Indecisive=|", "screen_name": "Rahi_Patel97", "lang": "en", "location": "W/Sofiaa", "create_at": date("2013-12-10"), "description": "《TAMU 20'//#GIGEM//》 SC:rahirahi\n ~If you scared, GO TO CHURCH! -The Kent Richey", "followers_count": 374, "friends_count": 264, "statues_count": 8864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taylor, TX", "id": "7793efb73edea52a", "name": "Taylor", "place_type": "city", "bounding_box": rectangle("-97.446645,30.536252 -97.382435,30.614587") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48491, "countyName": "Williamson", "cityID": 4871948, "cityName": "Taylor" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450440417280, "text": "@mdjesusib10 @surrgee Serg is a bitch", "in_reply_to_status": 739701189101551616, "in_reply_to_user": 315483651, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 315483651, 1344576643 }}, "user": { "id": 2182680402, "name": "Ant", "screen_name": "anthonydenunzio", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "smoke 'em if you got 'em", "followers_count": 531, "friends_count": 351, "statues_count": 7057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450490765312, "text": "Cause I'm a piece of shit it ain't hard to fucken tell", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1242773090, "name": "Lisa", "screen_name": "lisa_uno6", "lang": "en", "location": "null", "create_at": date("2013-03-04"), "description": "South Central LA // CSUSB", "followers_count": 350, "friends_count": 381, "statues_count": 34855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450570579968, "text": "Expect the FBI to come after Kanye after tonight! Just like they did with NWA! When the FBI knocks at your door, you got power!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 296340603, "name": "David Cordova", "screen_name": "davidarsenio10", "lang": "en", "location": "Bronx, NY", "create_at": date("2011-05-10"), "description": "23 years old . ♎️♎️ October 14th St. Francis College, '17 Intern with Tri-State Classic Founder of Dave's Joint", "followers_count": 1413, "friends_count": 2469, "statues_count": 24530 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bronx, NY", "id": "002e24c6736f069d", "name": "Bronx", "place_type": "city", "bounding_box": rectangle("-73.933612,40.785365 -73.765271,40.91533") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36005, "countyName": "Bronx", "cityID": 36005, "cityName": "Bronx" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450826473473, "text": "Try https://t.co/HP105ehEIp FREE CHEESE STICKS with your first purchase when you sign up for Hut Lovers ® . #CHEESE #menu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.62885802,39.81280244"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "CHEESE", "menu" }}, "user": { "id": 421095102, "name": "DineHere.us", "screen_name": "dine_here", "lang": "en", "location": "New York, USA", "create_at": date("2011-11-25"), "description": "The simple dining guide", "followers_count": 4400, "friends_count": 874, "statues_count": 435838 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas, USA", "id": "27c45d804c777999", "name": "Kansas", "place_type": "admin", "bounding_box": rectangle("-102.051769,36.99311 -94.588081,40.003282") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20157, "countyName": "Republic", "cityID": 2005600, "cityName": "Belleville" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450897612800, "text": "@luistravels @marislindstrom https://t.co/LwTz9VCgdA", "in_reply_to_status": 739699512478949376, "in_reply_to_user": 1297511065, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1297511065, 1093349270 }}, "user": { "id": 258970332, "name": "TwoTonnee", "screen_name": "tonioooo", "lang": "en", "location": "null", "create_at": date("2011-02-28"), "description": "The Travelers Club. Warehouse Takeover.", "followers_count": 371, "friends_count": 402, "statues_count": 8389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450939547648, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 195315997, "name": "Mandy", "screen_name": "mnnance", "lang": "en", "location": "Honolulu, HI", "create_at": date("2010-09-26"), "description": "null", "followers_count": 573, "friends_count": 409, "statues_count": 35194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Halawa, HI", "id": "ae8bddc0e5653780", "name": "Halawa", "place_type": "city", "bounding_box": rectangle("-157.937712,21.355729 -157.897921,21.393936") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1510000, "cityName": "Halawa" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713450948067328, "text": "Wind 1.0 mph SSE. Barometer 29.802 in, Rising slowly. Temperature 63.6 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 38, "friends_count": 4, "statues_count": 27957 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713451153465344, "text": "I just made a dark ass song", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1253353412, "name": "⠀", "screen_name": "rsljr_", "lang": "en", "location": "null", "create_at": date("2013-03-08"), "description": "null", "followers_count": 561, "friends_count": 407, "statues_count": 27946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-06T00:00:04.000Z"), "id": 739713451497562112, "text": "@mollzoneil nothing to rave about ��", "in_reply_to_status": 739684229878054912, "in_reply_to_user": 865514731, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 865514731 }}, "user": { "id": 209459549, "name": "Adam Vecitis", "screen_name": "adamvec", "lang": "en", "location": "null", "create_at": date("2010-10-29"), "description": "Silently stunting #MowWellFast", "followers_count": 394, "friends_count": 214, "statues_count": 15320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendale, WI", "id": "3f32fb7126c982d0", "name": "Glendale", "place_type": "city", "bounding_box": rectangle("-87.954755,43.089646 -87.907495,43.162589") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5529400, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075819754655744, "text": "Krisian RTED food porn time to mute", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3250117712, "name": "vic da chub", "screen_name": "KinkyYan", "lang": "en", "location": "HTX|| for us. @tsmemma krisian", "create_at": date("2015-06-19"), "description": "@whygracetho: WHY ARENT YOU VERIFIED / I'm dedicated to Yan Vostrikov Valentinovich and Grace", "followers_count": 3279, "friends_count": 282, "statues_count": 39580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075819968516096, "text": "@kingfuckcharlie rip me", "in_reply_to_status": 740075214386536449, "in_reply_to_user": 2528670314, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2528670314 }}, "user": { "id": 595189266, "name": "Eternal Fuckup™", "screen_name": "gamma_tay", "lang": "en", "location": "Fire Emblem Hell", "create_at": date("2012-05-30"), "description": "And there's a million things I haven't done, but just you wait. ♡Nerd ✿ Cinema&Television Arts Major ✿ Queer ✿ Intersectional Feminist♡", "followers_count": 411, "friends_count": 637, "statues_count": 31600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fullerton, CA", "id": "2feb76248fd9c581", "name": "Fullerton", "place_type": "city", "bounding_box": rectangle("-117.985941,33.853881 -117.863844,33.924814") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 628000, "cityName": "Fullerton" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820094390272, "text": "#honestyhour I don't think I'll get into a real relationship till im in my late twenties", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "honestyhour" }}, "user": { "id": 563758379, "name": "Rockman", "screen_name": "afsaanr17", "lang": "en", "location": "MacArthur, Tx", "create_at": date("2012-04-26"), "description": "Senior '17", "followers_count": 880, "friends_count": 636, "statues_count": 79702 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irving, TX", "id": "dce44ec49eb788f5", "name": "Irving", "place_type": "city", "bounding_box": rectangle("-97.034184,32.771786 -96.888782,32.956048") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4837000, "cityName": "Irving" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820509601792, "text": "الآن في هوليوود، واقف أمام النجمة الوحيدة المعلقة على الجدار، نجمة الأسطورة محمد علي كلاي.. الله يرحمه https://t.co/HNOI8yfFXb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.338673,34.101533"), "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2704587585, "name": "Abdulla", "screen_name": "AbdullaM93", "lang": "en", "location": "USA, FL", "create_at": date("2014-07-12"), "description": "كثيرٌ من برشلونة وقليل من كل شي | #FCB | #MCFC |", "followers_count": 770, "friends_count": 268, "statues_count": 2683 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollywood Walk of Fame", "id": "07d9d21568880005", "name": "Hollywood Walk of Fame", "place_type": "poi", "bounding_box": rectangle("-118.3386731,34.1015329 -118.338673,34.101533") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820543266817, "text": "https://t.co/LhebUNX8ki", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2358548269, "name": "DENISBRUNELL", "screen_name": "DELABAYOU", "lang": "en", "location": "Slidell, Louisiana", "create_at": date("2014-02-23"), "description": "A Man who has an appreciation, for the love of the human male body !", "followers_count": 3060, "friends_count": 3185, "statues_count": 66139 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Slidell, LA", "id": "44ee4fb6071b7eaa", "name": "Slidell", "place_type": "city", "bounding_box": rectangle("-89.853527,30.235705 -89.677511,30.36002") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2270805, "cityName": "Slidell" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820786421760, "text": "I CANT EVEN. THEY SANG ONE DAY MORE IN THE CAR OMG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3181056926, "name": "Lindsey Byrom", "screen_name": "lindsey_byrom", "lang": "en", "location": "null", "create_at": date("2015-04-30"), "description": "Jesus. Marvel. Hamilton.", "followers_count": 239, "friends_count": 404, "statues_count": 1622 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Buckeye, AZ", "id": "0015cc0d71d49e19", "name": "Buckeye", "place_type": "city", "bounding_box": rectangle("-112.62655,33.355798 -112.461428,33.515442") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 407940, "cityName": "Buckeye" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820794843136, "text": "Same https://t.co/wvCwzW5zUk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 243451575, "name": "jumex de mango", "screen_name": "bbybiscuit", "lang": "en", "location": "Edinburg/Kingsville", "create_at": date("2011-01-26"), "description": "21 | ¯\\_₍⸍⸌̣ʷ̣̫⸍̣⸌₎_/¯ | Vocal Music Major", "followers_count": 598, "friends_count": 400, "statues_count": 31342 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Doolittle, TX", "id": "e496d99bf7c24385", "name": "Doolittle", "place_type": "city", "bounding_box": rectangle("-98.135015,26.347314 -98.070036,26.385386") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4820926, "cityName": "Doolittle" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075820828528640, "text": "Malt and Carmel flavors - Drinking a Saddle Bronc Brown Ale by @blacktoothbrew @ Inner Richmond — https://t.co/KIpsLqJOmG #photo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.466,37.7808"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "photo" }}, "user_mentions": {{ 386619901 }}, "user": { "id": 15556194, "name": "robinchristine", "screen_name": "robinchristine", "lang": "en", "location": "San Francisco, CA", "create_at": date("2008-07-22"), "description": "null", "followers_count": 206, "friends_count": 273, "statues_count": 4270 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821260382208, "text": "wow we are not friends anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 702240026, "name": "harry moises", "screen_name": "poisonoussaam", "lang": "en", "location": "null", "create_at": date("2012-07-17"), "description": "null", "followers_count": 987, "friends_count": 597, "statues_count": 45787 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821264711680, "text": "Can't wait until I'm completely able to give my girl the world ���� y'all won't see me anywhere without her", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 997819525, "name": "OVO Simbaa", "screen_name": "CalucciB", "lang": "en", "location": "Fredneck", "create_at": date("2012-12-08"), "description": "she hit me up hey Lucci, she wants me in her coochie", "followers_count": 1272, "friends_count": 1117, "statues_count": 47464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederick, MD", "id": "ec3b8b62828d8f6a", "name": "Frederick", "place_type": "city", "bounding_box": rectangle("-77.476711,39.369128 -77.359293,39.494945") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2430325, "cityName": "Frederick" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821314904064, "text": "Always do https://t.co/Qq9W5AAvN8", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2772419336, "name": "Eli", "screen_name": "elizabethestala", "lang": "en", "location": "null", "create_at": date("2014-08-26"), "description": "null", "followers_count": 1578, "friends_count": 1933, "statues_count": 4635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heber, CA", "id": "0c7d4813f29bec86", "name": "Heber", "place_type": "city", "bounding_box": rectangle("-115.546547,32.709568 -115.51672,32.752368") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6025, "countyName": "Imperial", "cityID": 633084, "cityName": "Heber" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821356830720, "text": "Aloha my A|M Nation Chapter Family @maiden16_quezon! Sna po pls follow me din po ako kung ok po sa inyo! RT�� https://t.co/5qbgLQ7fkH", "in_reply_to_status": 740075029493207040, "in_reply_to_user": 4081271660, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user_mentions": {{ 4081271660 }}, "user": { "id": 364078490, "name": "Mark|O'ahu|Hawai'i™", "screen_name": "leomdf808", "lang": "en", "location": "Hawaii, USA", "create_at": date("2011-08-28"), "description": "☆ Certified #LokalBae808 ALDUB|MAiDEN Nation HAWAII Chapter Member @maiden16_hawaii ♡Joined 10•07•2015♡ *PabebeWave* ✌☆ ▪#ALDUBNation Updates Photos/Videos ®▪", "followers_count": 7535, "friends_count": 1172, "statues_count": 5507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Beach, HI", "id": "6bd73386ffaba450", "name": "Ewa Beach", "place_type": "city", "bounding_box": rectangle("-158.028613,21.306027 -157.990042,21.332114") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507450, "cityName": "Ewa Beach" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821361037313, "text": "@ebbtideapp Tide in North Haven, Maine 06/07/2016\n Low 7:18am -1.6\nHigh 1:34pm 10.7\n Low 7:34pm -0.3\nHigh 1:47am 11.6\n Low 8:10am -1.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-68.8733,44.1267"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 99, "friends_count": 1, "statues_count": 39295 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maine, USA", "id": "463f5d9615d7d1be", "name": "Maine", "place_type": "admin", "bounding_box": rectangle("-71.084335,42.917127 -66.885075,47.459687") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23013, "countyName": "Knox" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075821780467712, "text": "@favcocomilf ����", "in_reply_to_status": 740075532100894720, "in_reply_to_user": 740019059446579200, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 740019059446579200 }}, "user": { "id": 1353566881, "name": "20SHADESOFPINK✨", "screen_name": "Erinique__", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2013-04-14"), "description": "react with no reaction |#WEPROMOTEDOLLZ| @_ajs__ is the loml | Kayden & Mylah|#ripdex", "followers_count": 877, "friends_count": 659, "statues_count": 7173 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075822157991936, "text": "@BECCCAV3 lol dude I don't think we're gonna wake up to go look at those apt lol", "in_reply_to_status": 740075457610059776, "in_reply_to_user": 617914562, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 617914562 }}, "user": { "id": 1220229432, "name": "Santiago", "screen_name": "Santiago_C3", "lang": "en", "location": "San Antonio, TX", "create_at": date("2013-02-25"), "description": "I just hope it all works out. sc: santi_boy", "followers_count": 1376, "friends_count": 701, "statues_count": 40163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075822485147648, "text": "@heyyitsrene @oscarortizbruh I don't sweat I'm too dehydrated", "in_reply_to_status": 740075728838885377, "in_reply_to_user": 71428104, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 71428104, 2214805800 }}, "user": { "id": 621722139, "name": "Shaggy", "screen_name": "nightlife_28", "lang": "en", "location": "The Sauce", "create_at": date("2012-06-28"), "description": "World Class Bobsledder", "followers_count": 468, "friends_count": 654, "statues_count": 4746 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salida, CA", "id": "223075a5d710e397", "name": "Salida", "place_type": "city", "bounding_box": rectangle("-121.108332,37.681731 -121.058816,37.740478") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 664210, "cityName": "Salida" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075823013629953, "text": "Where the gas at out here?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2173338360, "name": "Dibbi Duz It ⚐", "screen_name": "dibbiduzit", "lang": "en", "location": "Dallas, TX ✈️ LA", "create_at": date("2013-11-03"), "description": "Creative Director and mix engineer of INFNTRY ⚐ @INFYINTL #PLEASANTGROVEEP + #MISC + #IMNOTSORRY DD 11.12.15 #INFY ALL IN", "followers_count": 1130, "friends_count": 144, "statues_count": 26621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075823546269698, "text": "Niggas will show their true colors when you're struggling.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 717079201, "name": "Brandon", "screen_name": "BrezzzyEzzzy41", "lang": "en", "location": "null", "create_at": date("2012-07-25"), "description": "Living Life To The Fullest. |SneakerHead. #RIPMalery #RIPBiggles #StayStrong7", "followers_count": 463, "friends_count": 374, "statues_count": 30345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075823550496769, "text": "Tbh his hair looks like the color of bbq chips. He chill tho & i will always care and be there for him�� https://t.co/JgxwsqwdU1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2743982866, "name": "Abbie Yanko", "screen_name": "abbiebabiee_", "lang": "en", "location": "west coast ", "create_at": date("2014-08-15"), "description": "sc: abbie_yanko", "followers_count": 572, "friends_count": 401, "statues_count": 5061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-06-07T00:00:00.000Z"), "id": 740075823676411904, "text": "Temp 61.3° Hi/Lo 64.5/60.8 Rng 3.7° WC 61.3° Hmd 94% Rain 0.00\" Storm 0.00\" BAR 29.632 Falling DP 59.6° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19306 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075823848247296, "text": "I guess I'm throwin lobs tonight�� dm me if u want one", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1798202936, "name": "Nasty Nas", "screen_name": "thegreatnasiear", "lang": "en", "location": "At home", "create_at": date("2013-09-07"), "description": "To be the best you gotta do what the best do #V4LG #JerkinIsntDead #blackmafia", "followers_count": 1133, "friends_count": 795, "statues_count": 16577 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824276230144, "text": "@_WOOeric I was so excited to see it & then it ended up not being that good lol", "in_reply_to_status": 740075348780548097, "in_reply_to_user": 47320716, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47320716 }}, "user": { "id": 634285862, "name": "Lili", "screen_name": "LilithAmberr", "lang": "en", "location": "the Ville", "create_at": date("2012-07-12"), "description": "Life is good .. maybe even a little better ✊", "followers_count": 606, "friends_count": 395, "statues_count": 11000 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824280309760, "text": "Fighting my sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422259454, "name": "Ponyboy", "screen_name": "xanny__pacquiao", "lang": "en", "location": "Stoner's Island", "create_at": date("2011-11-26"), "description": "SUSLA16", "followers_count": 3991, "friends_count": 4184, "statues_count": 256206 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824469024769, "text": "@noahhdavidd what made you so outgoing ?", "in_reply_to_status": 740075393546223617, "in_reply_to_user": 2787188316, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2787188316 }}, "user": { "id": 3274902805, "name": "Maria", "screen_name": "omfgmaria_", "lang": "en", "location": "null", "create_at": date("2015-07-10"), "description": "null", "followers_count": 147, "friends_count": 177, "statues_count": 430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824712294403, "text": "@Dehfatboy Buh you ain answer my question tho ��", "in_reply_to_status": 740075443340992513, "in_reply_to_user": 719190115025756164, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 719190115025756164 }}, "user": { "id": 874784970, "name": "WhateverLolaWants✨", "screen_name": "selfinvested93", "lang": "en", "location": "null", "create_at": date("2012-10-11"), "description": "Th!NKpositive™ .........¯\\_(ツ)_/¯", "followers_count": 146, "friends_count": 86, "statues_count": 6611 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Juneau, AK", "id": "00ebeb4332dd7c50", "name": "Juneau", "place_type": "city", "bounding_box": rectangle("-134.667895,58.260245 -134.349937,58.4253") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2110, "countyName": "Juneau", "cityID": 236400, "cityName": "Juneau" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824771043329, "text": "@JCsFluffyTail at least in the valley it might still be hot enough to swim then. Soon as the sun drops ak does the temp. Unbearable.", "in_reply_to_status": 740075674770145280, "in_reply_to_user": 1622258180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1622258180 }}, "user": { "id": 1321289282, "name": "Captain and King", "screen_name": "DaisyPoodlePuff", "lang": "en", "location": "Santa Rosa, CA", "create_at": date("2013-04-01"), "description": "Godless vegan. @Ace_Orcinus belongs to me.", "followers_count": 367, "friends_count": 827, "statues_count": 9680 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petaluma, CA", "id": "239aa72871ae24ab", "name": "Petaluma", "place_type": "city", "bounding_box": rectangle("-122.682383,38.205307 -122.584356,38.283601") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6097, "countyName": "Sonoma", "cityID": 656784, "cityName": "Petaluma" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824892645377, "text": "everything will be ok", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2503370090, "name": "nate", "screen_name": "_natecortez", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-05-17"), "description": "snap: corteznate | Pursuit of Happyness", "followers_count": 320, "friends_count": 207, "statues_count": 19444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075824897007617, "text": "06/07@03:00 - Temp 71.9F, WC 71.9F. Wind 1.6mph SSW, Gust 4.0mph. Bar 29.615in, Falling. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075825693872128, "text": "Gn��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 350403134, "name": "Jadaaaaaa✨", "screen_name": "_YagirllJAY", "lang": "en", "location": "sc: bmf.jay", "create_at": date("2011-08-07"), "description": "bullshit gets overlooked☺️ #longlivebruce iloveyoubro❤️ #makenzie'skeeper✨ #prettyGang✨ •MooK❤️", "followers_count": 1060, "friends_count": 951, "statues_count": 23307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075825731538944, "text": "Update: cat has eaten an entire q tip", "in_reply_to_status": 722530635324129280, "in_reply_to_user": 53509516, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53509516, "name": "Pepito", "screen_name": "BravesGasm", "lang": "en", "location": "Lebanon, TN", "create_at": date("2009-07-03"), "description": "I don't understand the infield fly rule, either. Instagram: adhdtour. Preds puckey.\n There. Will. Be. Cats.", "followers_count": 4347, "friends_count": 1897, "statues_count": 161375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lebanon, TN", "id": "cae48909a1a015a5", "name": "Lebanon", "place_type": "city", "bounding_box": rectangle("-86.40544,36.138726 -86.237547,36.276356") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47189, "countyName": "Wilson", "cityID": 4741520, "cityName": "Lebanon" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826268508160, "text": "I was cumming and the video started buffering.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 842252552, "name": "Six Paths of Pain", "screen_name": "Carlosownsnoobs", "lang": "en", "location": "Starlight Temple | Zanarkand", "create_at": date("2012-09-23"), "description": "1997 KARL MALONE|HHS Alumni|Acidic Mentality|FRK III|Hokage|Keyblade Master|Hero of Time|Chocobo Knight|Pokemon Master|Fork God|#ByeFamSquad|", "followers_count": 1123, "friends_count": 319, "statues_count": 205185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826297741313, "text": "Happy birthday �� @kissmyjaays__", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321770247 }}, "user": { "id": 2569442030, "name": "Da Homie ", "screen_name": "taewest", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-06-15"), "description": "fuck vh1", "followers_count": 589, "friends_count": 545, "statues_count": 44300 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826356555776, "text": "Goodbye today hello tomorrow 11:59", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1282292084, "name": "⚡️ Isaiah⚡️", "screen_name": "iAmKingIsaiah", "lang": "en", "location": "Yucca Valley 760", "create_at": date("2013-03-19"), "description": "Can't complain this is the life I was given time to do something great. SC: Kingbates_V #LiveFit #LakerNation #GoPackGo #LowLife", "followers_count": 2010, "friends_count": 1340, "statues_count": 23569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Joshua Tree, CA", "id": "2ebc5630b7ca50fe", "name": "Joshua Tree", "place_type": "city", "bounding_box": rectangle("-116.369935,34.105264 -116.299818,34.149567") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 637554, "cityName": "Joshua Tree" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826427875328, "text": "Wind 0.0 mph ---. Barometer 29.84 in, Steady. Temperature 74.3 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23200 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826457149440, "text": "Actually kinda glad me and my ex get along �� \nWhy would anyone try to ruin anyone like that��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2218400582, "name": "nicole", "screen_name": "nicolemadrigalx", "lang": "en", "location": "between the buildings ", "create_at": date("2013-11-27"), "description": "so much you", "followers_count": 1149, "friends_count": 276, "statues_count": 22925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lehi, UT", "id": "ed3b23f667186d1f", "name": "Lehi", "place_type": "city", "bounding_box": rectangle("-111.914775,40.357876 -111.818332,40.456264") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4944320, "cityName": "Lehi" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826876534785, "text": "@JoelMorenoKOMO Our cap hill office was open nearly 24/7 in March and hosted many events and trainings, potlucks, and debate watch parties.", "in_reply_to_status": 740074820906291200, "in_reply_to_user": 14463995, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 257664971 }}, "user": { "id": 14463995, "name": "Andrew Saturn", "screen_name": "andsat", "lang": "en", "location": "Seattle", "create_at": date("2008-04-21"), "description": "Maker, joker, coder, designer, activist, 'Bernie Person.' #FeelTheBern #FightFor15 #DemocracySpring #BlackLivesMatter #KeepItInTheGround", "followers_count": 1199, "friends_count": 889, "statues_count": 9481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075826964660225, "text": "@_yesseniav you don't count.", "in_reply_to_status": 740075752427638784, "in_reply_to_user": 3154819696, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3154819696 }}, "user": { "id": 607662181, "name": "Celestial ☯", "screen_name": "markantonio98", "lang": "en", "location": "Earth", "create_at": date("2012-06-13"), "description": "young car enthusiast #StangGang #thestreetscene IG:Marky.pooo SC:poomarky", "followers_count": 550, "friends_count": 517, "statues_count": 28852 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075827031724032, "text": "Wore my 'Sad As Fuck' shirt to a party, and got compliments on it. Thanks Fam @emonightLA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2906391769 }}, "user": { "id": 29991769, "name": "✨ Nicki ✨", "screen_name": "nicknack013", "lang": "en", "location": "Portland, OR", "create_at": date("2009-04-09"), "description": "Anthany ❤️", "followers_count": 1233, "friends_count": 2010, "statues_count": 47741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Happy Valley, OR", "id": "00bda52c7eddb627", "name": "Happy Valley", "place_type": "city", "bounding_box": rectangle("-122.581184,45.40302 -122.477198,45.461537") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4132050, "cityName": "Happy Valley" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075827031736320, "text": "@RayRayYepez \n-bus rides home ����\n-water polo\n-huge ass smile\n-imy", "in_reply_to_status": -1, "in_reply_to_user": 1517551747, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1517551747 }}, "user": { "id": 1692920857, "name": "Ana ツ", "screen_name": "amercado1026", "lang": "en", "location": "null", "create_at": date("2013-08-22"), "description": "null", "followers_count": 864, "friends_count": 837, "statues_count": 14016 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075827421843456, "text": "Is it hard to find a gf that is nice and funny and sticks to you only ??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2363059861, "name": "N a t h a n", "screen_name": "Nathan83216440", "lang": "en", "location": "null", "create_at": date("2014-02-26"), "description": "W R E S T L I N G❤️ Christian Guzman is my motivation", "followers_count": 223, "friends_count": 329, "statues_count": 1182 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075827870588928, "text": "@marina_roski \n- driving instructor \n- brake checks \n- taco Tuesday's \n- softball / volleyball\n- \"hey... I have to tell u something\"����", "in_reply_to_status": -1, "in_reply_to_user": 603924273, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 603924273 }}, "user": { "id": 2901682029, "name": "big vic ◡̈⃝", "screen_name": "viczaluske", "lang": "en", "location": "hoorah baby ", "create_at": date("2014-12-01"), "description": "firecrackers 43 ☓ drhs varsity vball & softball 34", "followers_count": 414, "friends_count": 362, "statues_count": 2414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-06-07T00:00:01.000Z"), "id": 740075827929305088, "text": "Shoooooooot heeeeeeeeeerr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 505250088, "name": "Mikel AnnMarie Clark", "screen_name": "Mikel_AnnMarie", "lang": "en", "location": "Your Moms House. ", "create_at": date("2012-02-26"), "description": "You were never gang gang gang.", "followers_count": 343, "friends_count": 413, "statues_count": 11358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taft, CA", "id": "5296438a6967a6fe", "name": "Taft", "place_type": "city", "bounding_box": rectangle("-119.483186,35.122091 -119.430928,35.172321") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 677574, "cityName": "Taft" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828092882945, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":67.1°F Wind:0.0mph Pressure: 29.79hpa Steady Rain Today 0.00in. Forecast: Fairly fine, showers like", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 83, "friends_count": 18, "statues_count": 322424 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828172718080, "text": "Wind 0.0 mph ---. Barometer 29.680 in, Falling slowly. Temperature 62.3 °F. Rain today 0.02 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828394917888, "text": "��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1149834854, "name": "Rylie Edmonson", "screen_name": "ryliemarie_", "lang": "en", "location": "California, USA", "create_at": date("2013-02-04"), "description": "Romans 8:18", "followers_count": 433, "friends_count": 229, "statues_count": 11850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828428443648, "text": "Hi commit suicide", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2904107845, "name": "RoastingSnake", "screen_name": "DavidsTechLab", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-11-18"), "description": "leafy notice me pls", "followers_count": 116, "friends_count": 312, "statues_count": 519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828533297152, "text": "MY AUNT FROM WASHINGTON IS DOWN VISITING AND SHE JUST ASKED US IF WE KNEW ANYONE THAT SELLS MARIJUANA IM CRYING", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4176324612, "name": "sarita", "screen_name": "holiestguac", "lang": "en", "location": "null", "create_at": date("2015-11-08"), "description": "life is gr8 & so are you", "followers_count": 311, "friends_count": 172, "statues_count": 12633 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Indio, CA", "id": "bd4397e027d410bb", "name": "Indio", "place_type": "city", "bounding_box": rectangle("-116.301227,33.685344 -116.174587,33.803921") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 636448, "cityName": "Indio" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828583751680, "text": "Where are all the dope women that are striving for success at? . . . We miss you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 595721905, "name": "Vic", "screen_name": "Trill_Vic", "lang": "en", "location": "Queens, NY", "create_at": date("2012-05-31"), "description": "NY/FL", "followers_count": 708, "friends_count": 997, "statues_count": 14687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828697042944, "text": "Wind 0.0 mph NW. Barometer 29.819 in, Rising Rapidly. Temperature 68.7 °F. Rain today 0.00 in. Humidity 55%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828780769281, "text": "I'm fine now. It's okay. She's happy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2570136807, "name": "hoodlum.", "screen_name": "1thirtythree", "lang": "en", "location": "San Antonio, TX", "create_at": date("2014-05-28"), "description": ":D", "followers_count": 1182, "friends_count": 926, "statues_count": 4855 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075828906582017, "text": "Cant sleep ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3122930052, "name": "Eddie Messbarger", "screen_name": "steady_eddie23", "lang": "en", "location": "San Angelo, TX ", "create_at": date("2015-03-31"), "description": "In order to become old and wise you must first be young and dumb... snapchat: @steady_eddie23 instagram: @steady_eddie23 angelo state basketball player", "followers_count": 712, "friends_count": 679, "statues_count": 6362 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Vista, TX", "id": "01ac7f6d273c7ecd", "name": "Laguna Vista", "place_type": "city", "bounding_box": rectangle("-97.314155,26.091627 -97.279762,26.124545") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48061, "countyName": "Cameron", "cityID": 4840336, "cityName": "Laguna Vista" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075829070204928, "text": "There's gonna be obstacles ...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356011320, "name": "'d ✨", "screen_name": "NaiiiNaiii", "lang": "en", "location": "nac, tx", "create_at": date("2011-08-15"), "description": "19. longlivedex #TJC17", "followers_count": 2393, "friends_count": 1557, "statues_count": 48113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hudson, TX", "id": "017fde213d7e6d08", "name": "Hudson", "place_type": "city", "bounding_box": rectangle("-94.844621,31.303642 -94.764475,31.369564") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48005, "countyName": "Angelina", "cityID": 4835228, "cityName": "Hudson" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075829221306368, "text": "@cawhaleyy damn lol", "in_reply_to_status": 740075737047289856, "in_reply_to_user": 3344783637, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3344783637 }}, "user": { "id": 84967279, "name": "Ga〽️bino.", "screen_name": "TeeCeee_", "lang": "en", "location": "Dothan, AL", "create_at": date("2009-10-24"), "description": "Young nigga with big dreams. RIP Grandma Rosemary", "followers_count": 3089, "friends_count": 2760, "statues_count": 58639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dothan, AL", "id": "d2981a0f3d8eddd7", "name": "Dothan", "place_type": "city", "bounding_box": rectangle("-85.515405,31.130472 -85.334617,31.33397") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1069, "countyName": "Houston", "cityID": 121184, "cityName": "Dothan" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075829393272832, "text": "Can you guys tag Jack in my pinned tweet I WANNA MEET THEM LMAO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1640923357, "name": "LIT L3X", "screen_name": "TooLitLex", "lang": "en", "location": "JH✨", "create_at": date("2013-08-02"), "description": "Litty Committee", "followers_count": 4695, "friends_count": 580, "statues_count": 115997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075829443469313, "text": "@TaylorxLovee the way he just popped up. ����", "in_reply_to_status": 740075661025370113, "in_reply_to_user": 322791180, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 322791180 }}, "user": { "id": 986813323, "name": "Jordan.", "screen_name": "TheJordanAvery", "lang": "en", "location": "htx.", "create_at": date("2012-12-03"), "description": "Sarcasm falls out of my mouth, just like stupid falls from yours. #KeepPounding", "followers_count": 975, "friends_count": 589, "statues_count": 20759 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075829728792577, "text": "Temp: 71.5°F Wind:0.0mph Pressure: 29.811hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66348 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830043250694, "text": "love shady nights. Especially when i dont know anybody and it doesnt involve me in any way��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 735724663, "name": "brianna", "screen_name": "bvgdxddy", "lang": "en", "location": "Just chillin in Cedar Rapids", "create_at": date("2012-08-03"), "description": "Well I'm not good with advice, but may I interest you in a sarcastic comment?", "followers_count": 313, "friends_count": 1038, "statues_count": 2719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830114717696, "text": "SO LIT ���� https://t.co/5vHlzeWmyG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 630807527, "name": "Tyler Trout", "screen_name": "TylerTrout22", "lang": "en", "location": "the bull ", "create_at": date("2012-07-08"), "description": "I'm an athletic nerd.", "followers_count": 1327, "friends_count": 1434, "statues_count": 18385 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Durham, NC", "id": "bced47a0c99c71d0", "name": "Durham", "place_type": "city", "bounding_box": rectangle("-79.007589,35.866334 -78.783292,36.115631") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37063, "countyName": "Durham", "cityID": 3719000, "cityName": "Durham" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830718537728, "text": "golf is the worst sport ever", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467893896, "name": "Slick Owens", "screen_name": "Khamronb", "lang": "en", "location": "Houston, Tx", "create_at": date("2012-01-18"), "description": "none of these hoes can meet my mom", "followers_count": 594, "friends_count": 285, "statues_count": 32625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830827683840, "text": "Ripley SW Limestone Co. Temp: 64.4°F Wind:0.0mph Pressure: 989.3mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 54212 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830920028161, "text": "yo Kia go to sleep lmao", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2323191524, "name": "Dr.SeyToven", "screen_name": "Alo_Zeyi", "lang": "en", "location": "#Nigeria #BaltimoreUSA", "create_at": date("2014-02-01"), "description": "#Umes #CityCollege12 #CruddyBoyz #OG17 #MarineBiologist It's Reality Speaking...Great Minds Fuck Each Other", "followers_count": 1553, "friends_count": 742, "statues_count": 204559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830957748225, "text": "@delta @DeltaAssist guy sitting next to me isHUMMING it's ANNOYING #redeye Pls make him stop: next time iFly #ptsd �� https://t.co/8WNF7ZbLFK", "in_reply_to_status": -1, "in_reply_to_user": 5920532, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "redeye", "ptsd" }}, "user_mentions": {{ 5920532, 137460929 }}, "user": { "id": 191607574, "name": "Sharyn Bovat", "screen_name": "sharynbovat", "lang": "en", "location": "Washington DC", "create_at": date("2010-09-16"), "description": "Ex Cold War intel gatherer: Current DOD/NISSAN/Lockerbie Whistleblower, exposing fraud, corruption, discrimination & judicial abuse via quirky blogs 6159447599", "followers_count": 563, "friends_count": 374, "statues_count": 18546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Segundo, CA", "id": "0654b676d0359a31", "name": "El Segundo", "place_type": "city", "bounding_box": rectangle("-118.429843,33.901804 -118.370685,33.931493") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 622412, "cityName": "El Segundo" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075830961807360, "text": "Marcina dittos used to be fun\nNow they're just \"spam fair\"\nThis character is dead to me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 868577977, "name": "Gaymer Meowser", "screen_name": "TSW_Quilly", "lang": "en", "location": "Bowser's Castle", "create_at": date("2012-10-08"), "description": "----Weakness: Attractive Antagonists----- I like Smash Bros. and I like to Smash Bros #TEAMSTARWOLF #Crewtwo", "followers_count": 299, "friends_count": 247, "statues_count": 21015 }, "place": { "country": "United States", "country_code": "United States", "full_name": "University City, MO", "id": "2dd9882dad5ba631", "name": "University City", "place_type": "city", "bounding_box": rectangle("-90.366578,38.645912 -90.297186,38.683509") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2975220, "cityName": "University City" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075831326736388, "text": "OH SHIT SOMEONE LIKED MY TWEET. ONG MOM I'M FAMOUS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 382771461, "name": "Alex Weeks.", "screen_name": "AlexWeeks101", "lang": "en", "location": "Viva La Vie Boheme.", "create_at": date("2011-09-30"), "description": "25B, Oklahoma National Guard.", "followers_count": 535, "friends_count": 1130, "statues_count": 11777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075831335079936, "text": "@Leecieeboo THESE R SO FUNNY AND ACURATE HAHAHAHAHHA", "in_reply_to_status": 740029617327640577, "in_reply_to_user": 329443724, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 329443724 }}, "user": { "id": 2616277055, "name": "lesley romero", "screen_name": "xleeesley", "lang": "en", "location": "Tokyo, Japan", "create_at": date("2014-06-16"), "description": "626 // k$", "followers_count": 347, "friends_count": 298, "statues_count": 13193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Valinda, CA", "id": "b1bbf8e2ef22573a", "name": "Valinda", "place_type": "city", "bounding_box": rectangle("-117.94891,34.024734 -117.912772,34.053145") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 681638, "cityName": "Valinda" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075831737737216, "text": "\"Yo man I'm from philly we had to save up to be poor\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 455863185, "name": "T", "screen_name": "tonymayers_", "lang": "en", "location": "bobb bounty ", "create_at": date("2012-01-05"), "description": "sc: lienoacissej", "followers_count": 632, "friends_count": 390, "statues_count": 75638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kissimmee, FL", "id": "c2809aa3b2c93fb2", "name": "Kissimmee", "place_type": "city", "bounding_box": rectangle("-81.47749,28.250764 -81.327204,28.347977") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12097, "countyName": "Osceola", "cityID": 1236950, "cityName": "Kissimmee" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075831783870468, "text": "Yo quisiera entrar en tu corazón", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 2229283044, "name": "A", "screen_name": "xoxo_alondra01", "lang": "en", "location": "null", "create_at": date("2013-12-03"), "description": "✨", "followers_count": 383, "friends_count": 363, "statues_count": 10113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-07T00:00:02.000Z"), "id": 740075832064933888, "text": "I just don't understand ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4264733474, "name": "dillon conrad", "screen_name": "DConrad0", "lang": "en", "location": "Lake Charles, LA", "create_at": date("2015-11-16"), "description": "#Trump2016. https://vine.co/v/hVtjdA52B6t | https://twitter.com/barstoolsports/status/734942428730384384", "followers_count": 334, "friends_count": 500, "statues_count": 7620 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prien, LA", "id": "00a028c043bb68f0", "name": "Prien", "place_type": "city", "bounding_box": rectangle("-93.283003,30.106667 -93.213369,30.168682") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2262647, "cityName": "Prien" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832295710724, "text": "I'm the realest nigga in it, you already know.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 48630995, "name": "YBKDOOBIE", "screen_name": "RealDoobieMyles", "lang": "en", "location": "Franklin, Tn", "create_at": date("2009-06-18"), "description": "New album #TheFamily #July4 #YBK #TheFamily #KingShit", "followers_count": 734, "friends_count": 691, "statues_count": 6919 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, TN", "id": "cc631a80adacd459", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-86.935419,35.85036 -86.766934,36.019674") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47187, "countyName": "Williamson", "cityID": 4727740, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832383672320, "text": "Hmmm idk ���� https://t.co/3dmD567uEh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3221955356, "name": "Chelsí", "screen_name": "chvlsy", "lang": "en", "location": "null", "create_at": date("2015-05-20"), "description": "#8", "followers_count": 362, "friends_count": 298, "statues_count": 4366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832480301056, "text": "Wind 0.0 mph ---. Barometer 29.796 in, Rising. Temperature 67.7 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832643719169, "text": "@ oomf https://t.co/U7Kmhgc8lA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 2198084400, "name": "linda", "screen_name": "sandovaleslie", "lang": "en", "location": "NYC too SLC ", "create_at": date("2013-11-16"), "description": "rest in paradise raul, armando, angel ❤️", "followers_count": 833, "friends_count": 959, "statues_count": 21594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Millcreek, UT", "id": "5af91023ed39df89", "name": "Millcreek", "place_type": "city", "bounding_box": rectangle("-111.921221,40.664889 -111.839259,40.712147") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4950150, "cityName": "Millcreek" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832832466948, "text": "@_LilDaesha nope", "in_reply_to_status": 740075068827389952, "in_reply_to_user": 2592030168, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2592030168 }}, "user": { "id": 963561686, "name": "Scaassh ❣", "screen_name": "BlvckBeavty", "lang": "en", "location": "null", "create_at": date("2012-11-21"), "description": "TJC CHEER ➳ ACUF", "followers_count": 1860, "friends_count": 853, "statues_count": 72991 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832849244160, "text": "HAPPY BIRTHDAY TO ME", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1833227323, "name": "el oh el", "screen_name": "ronikastonee", "lang": "en", "location": "yay area ", "create_at": date("2013-09-08"), "description": "Universitizzle of Oregon '20", "followers_count": 874, "friends_count": 584, "statues_count": 9807 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832874389504, "text": "I hang wit dem real niggas you hear me? #East", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "East" }}, "user": { "id": 1340661553, "name": "Hotboy Kam", "screen_name": "Tybkam", "lang": "en", "location": "Eastside Okc", "create_at": date("2013-04-09"), "description": "@OsoArrogantJoJo || #FreeTino #FreeLilB", "followers_count": 845, "friends_count": 770, "statues_count": 35255 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075832878718976, "text": "lmao on me nigga https://t.co/nhzfoOgjTo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 437171983, "name": "singh j rohby", "screen_name": "RohbyStayBEAST", "lang": "en", "location": "BIG3", "create_at": date("2011-12-14"), "description": "#CAREFREE, just on the come up", "followers_count": 456, "friends_count": 180, "statues_count": 25440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075833432281088, "text": "Never forget the first time I found out they sold almonette cookies at family dollar, miss Kim told me she made them by hand.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 40146128, "name": "Baby Pa", "screen_name": "BudhaLovesBooty", "lang": "en", "location": "Killeen - SFA - Dallas ", "create_at": date("2009-05-14"), "description": "( ͡ ͡° ͡° ʖ ͡° ͡°)\n\nAmeriKKKa was never great and I have the receipts...History Major and Anthro Minor", "followers_count": 1654, "friends_count": 1135, "statues_count": 114832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075833964957701, "text": "@micchaeeel LMAO", "in_reply_to_status": 284919522161471489, "in_reply_to_user": 37231566, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 183107340 }}, "user": { "id": 37231566, "name": "lil pound cake", "screen_name": "xvvier", "lang": "en", "location": "san francisco", "create_at": date("2009-05-02"), "description": "null", "followers_count": 987, "friends_count": 792, "statues_count": 32798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075834220896256, "text": "I want a Latina in my life \n#honestyhour", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "honestyhour" }}, "user": { "id": 539164752, "name": "King Jodye (#1 DAD)", "screen_name": "VoteforOshae", "lang": "en", "location": "Only God Knows", "create_at": date("2012-03-28"), "description": "People say you are what you eat, but that's weird because I don't remember eating a legend. Freelance photographer #TXST ✉:lifeless.aesthetics@gmail.com", "followers_count": 12311, "friends_count": 10726, "statues_count": 27294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075834363379713, "text": "Damn, RIP #KimboSlice :c", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "KimboSlice" }}, "user": { "id": 703973863, "name": "Blaq to the Future", "screen_name": "OfficialBlaqout", "lang": "en", "location": "St Louis, MO", "create_at": date("2012-07-18"), "description": "Pressing ▶ since 2013.", "followers_count": 713, "friends_count": 368, "statues_count": 1100 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rolla, MO", "id": "3376203928c5b618", "name": "Rolla", "place_type": "city", "bounding_box": rectangle("-91.80334,37.914961 -91.718307,37.980686") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29161, "countyName": "Phelps", "cityID": 2962912, "cityName": "Rolla" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075834871042048, "text": "Wind 0.0 mph ---. Barometer 29.58 in, Falling. Temperature 59.2 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 160353 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835160297472, "text": "I'm 17 bitches", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2560950513, "name": "bae harbor butcher", "screen_name": "MadisonLStone", "lang": "en", "location": "gemini", "create_at": date("2014-05-23"), "description": "Leonardo DiCaprio is my dad", "followers_count": 475, "friends_count": 908, "statues_count": 2328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835164524544, "text": "shit was 5 mins ago", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 464321828, "name": "maya.✨", "screen_name": "MayaTaughtYou_", "lang": "en", "location": "DDD, Tx ", "create_at": date("2012-01-14"), "description": "18. Matthew 6:34 ❤️", "followers_count": 2060, "friends_count": 1040, "statues_count": 93414 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835256770561, "text": "Wind 0.0 mph ---. Barometer 29.872 in, Rising slowly. Temperature 67.3 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 53, "statues_count": 8993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835441418240, "text": "Temp 55.5°F Heat Index 55.5 °F RH 93% Wind 0.0 --- Gust 0.0 --- SLP 29.721 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 62, "statues_count": 35988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835483430913, "text": "@peacocky_orl yes he is", "in_reply_to_status": 740075666931081216, "in_reply_to_user": 2972993049, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2972993049 }}, "user": { "id": 2336234330, "name": "Paul Stephens", "screen_name": "PaulSte92075247", "lang": "en", "location": "Columbus,Ohio U.S.A #BBBH ", "create_at": date("2014-02-09"), "description": "Big older Gay Bear I love Men,love Cock,I love Cum,anything Gay,Scallylads,chavs,18+ like music,video games,Horror/Sci-Fi movies paranormal,witchcraft,Sasquatch", "followers_count": 4475, "friends_count": 3112, "statues_count": 48397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835500068864, "text": "@louieloafers won't fight me tho", "in_reply_to_status": 740075539382165508, "in_reply_to_user": 1605417955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1605417955 }}, "user": { "id": 461667800, "name": "B", "screen_name": "Brandyraveee", "lang": "en", "location": "San Antonio, TX", "create_at": date("2012-01-11"), "description": "19 | God is within her, she will not fall", "followers_count": 1999, "friends_count": 1709, "statues_count": 87741 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075835688816640, "text": "you stupid naive dumb fucking cunt bitch ass mark ass leva", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270730120, "name": "sad pendejo", "screen_name": "LoneWolfNoe", "lang": "en", "location": "Florence-Graham, CA", "create_at": date("2011-03-22"), "description": "see you, \nspace cowboy", "followers_count": 263, "friends_count": 259, "statues_count": 23090 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence-Graham, CA", "id": "1100db27cd9d364c", "name": "Florence-Graham", "place_type": "city", "bounding_box": rectangle("-118.256783,33.943092 -118.227672,33.989716") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 624477, "cityName": "Florence-Graham" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075836326346752, "text": "I keep saying back to school now but then I realize it's the last week of school", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1371236995, "name": "alexandra", "screen_name": "AlexaaMendozaa", "lang": "en", "location": "creo que anda bailando", "create_at": date("2013-04-21"), "description": "J", "followers_count": 821, "friends_count": 654, "statues_count": 18322 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-06-07T00:00:03.000Z"), "id": 740075836339015681, "text": "Saturday wherrrrrrrrrrre are you. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2669442950, "name": "BRAT ❌", "screen_name": "brittgotjuice", "lang": "en", "location": "Palm Beach, FL", "create_at": date("2014-07-22"), "description": "don't get lost round here.", "followers_count": 343, "friends_count": 180, "statues_count": 2350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lantana, FL", "id": "07bf758651a4bc76", "name": "Lantana", "place_type": "city", "bounding_box": rectangle("-80.105173,26.569606 -80.037365,26.609384") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1239375, "cityName": "Lantana" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075836464898048, "text": "Why am I awake", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 813595746, "name": "Lexy", "screen_name": "shexshylexy", "lang": "en", "location": "null", "create_at": date("2012-09-09"), "description": "http://yesah.com/?rfsn=231843.bfffa", "followers_count": 176, "friends_count": 172, "statues_count": 2643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherry Hill Mall, NJ", "id": "06a7ad558d8a3014", "name": "Cherry Hill Mall", "place_type": "city", "bounding_box": rectangle("-75.047395,39.924833 -74.974961,39.954192") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3412385, "cityName": "Cherry Hill Mall" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075836494077952, "text": "FUCK ������ https://t.co/k4tHtmRBlR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1414593043, "name": "QuiseLaFlame", "screen_name": "BlackHigAshi", "lang": "en", "location": "null", "create_at": date("2013-05-08"), "description": "NIGGA IM HAPPY", "followers_count": 666, "friends_count": 875, "statues_count": 5195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075837135802369, "text": "@andreshigetomi https://t.co/BrX8jPsJZL", "in_reply_to_status": 740072957855174657, "in_reply_to_user": 1473154712, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1473154712 }}, "user": { "id": 566901099, "name": "Gabe Eatmon", "screen_name": "gabeatmon", "lang": "en", "location": "null", "create_at": date("2012-04-29"), "description": "My mom likes Peter Gabriel so she named me Gabriel.", "followers_count": 98, "friends_count": 122, "statues_count": 528 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheney, WA", "id": "15e15f91e2233f0b", "name": "Cheney", "place_type": "city", "bounding_box": rectangle("-117.607663,47.465326 -117.556197,47.513738") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5311825, "cityName": "Cheney" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075837207105536, "text": "Why not both https://t.co/3EPJW80cAq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1210266085, "name": "Sabé", "screen_name": "WhoGotNext_", "lang": "en", "location": "null", "create_at": date("2013-02-22"), "description": "United States Navy ⚓️Real Is God. IG: Lang.Sabe I.E.", "followers_count": 965, "friends_count": 1055, "statues_count": 15828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075837613957120, "text": "@litt0eviie his girlfriend/wife *", "in_reply_to_status": 740075752666734592, "in_reply_to_user": 2890473446, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25937348 }}, "user": { "id": 2890473446, "name": "ૐ", "screen_name": "trancemeowt", "lang": "en", "location": "California ", "create_at": date("2014-11-04"), "description": "IG: @zeldabynight / spicy like sriracha", "followers_count": 82, "friends_count": 96, "statues_count": 1544 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ceres, CA", "id": "a6c30cefdd39bd81", "name": "Ceres", "place_type": "city", "bounding_box": rectangle("-120.993774,37.561491 -120.920472,37.620692") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 612524, "cityName": "Ceres" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075837874012160, "text": "@mmmuriam @_NotoriousARG ha aight", "in_reply_to_status": 740075715572310016, "in_reply_to_user": 424641908, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 424641908, 2829204186 }}, "user": { "id": 381865866, "name": "DaVon", "screen_name": "WOFWB", "lang": "en", "location": "Tucson, AZ", "create_at": date("2011-09-28"), "description": "It is most definitely ya boi • NBC • Be happy", "followers_count": 812, "friends_count": 592, "statues_count": 49512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075838192898048, "text": "Don't entertain that lmfao https://t.co/r9e0nwkbxw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 117649981, "name": "Boston George", "screen_name": "lxxus", "lang": "en", "location": "ATL✈️ DC✈️ CobbCo", "create_at": date("2010-02-25"), "description": "@Sycegame OG | Wolf of GA | Making Plays | Entrepreneur | Bar Hopper | Beer/EDM Enthusiast | #PBR #1", "followers_count": 2071, "friends_count": 459, "statues_count": 257387 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075838494904320, "text": "https://t.co/uyjooG2Or4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 702199438324981761, "name": "Leisa Poindexter", "screen_name": "msscottybear", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "here for me. here for you ! !! !!! MFF. 9-29-15", "followers_count": 144, "friends_count": 72, "statues_count": 11138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075838729818113, "text": "Wow I have nobody to be shady with right now lolol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3417101303, "name": "MVP", "screen_name": "BeySzn", "lang": "en", "location": "Memphis, TN", "create_at": date("2015-08-11"), "description": "Gxd✝Cxmplex", "followers_count": 715, "friends_count": 439, "statues_count": 8834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dyersburg, TN", "id": "7b01eb8d5c7cb04a", "name": "Dyersburg", "place_type": "city", "bounding_box": rectangle("-89.431339,36.003989 -89.315421,36.085264") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47045, "countyName": "Dyer", "cityID": 4722200, "cityName": "Dyersburg" } }
+{ "create_at": datetime("2016-06-07T00:00:04.000Z"), "id": 740075838800986112, "text": "LOOK #OUTLANDER #TV #Pilot #Script - #SamHeughan #AUTOGRAPH - #CaitrionaBalfe, #TobiasMenzies - on #EBay https://t.co/71T8PuBt2l …", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OUTLANDER", "TV", "Pilot", "Script", "SamHeughan", "AUTOGRAPH", "CaitrionaBalfe", "TobiasMenzies", "EBay" }}, "user": { "id": 717544904, "name": "Kyle James Abbot", "screen_name": "KyleAbbot", "lang": "en", "location": "West Hollywood, CA", "create_at": date("2012-07-25"), "description": "Hi, I'm 23, 6'4, 205, Blond/Blue. Fun, friendly, masculine, #athletic #gay guy in #LosAngeles. Student. Big #TV fan:) My Amazon WishList http://t.co/i4rxZfd5Bp", "followers_count": 59801, "friends_count": 57617, "statues_count": 98512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438207511269378, "text": "I did! My sister and I ran down to the ocean for a quick wade. Short but sweet trip to CA. cc @stevekimura https://t.co/cMxjzaTmSA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 34761747 }}, "user": { "id": 49698134, "name": "Joy Reid", "screen_name": "JoyAnnReid", "lang": "en", "location": "NYC", "create_at": date("2009-06-22"), "description": "#AMJoy host-weekends at 10am. Author of 'Fracture: Barack Obama, the Clintons, and the Racial Divide.' Still not for your candidate or their opponent. #reiders", "followers_count": 209259, "friends_count": 3171, "statues_count": 80359 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marina del Rey, CA", "id": "38ea9782c8d83a4b", "name": "Marina del Rey", "place_type": "city", "bounding_box": rectangle("-118.463481,33.964019 -118.432199,33.98647") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 645806, "cityName": "Marina del Rey" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438207863640064, "text": "I get soooo annoyed when girls that know me follow Aaron like bih don't start with me, you don't know him unless I say so. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538792052, "name": "big daddy.", "screen_name": "verenis_", "lang": "en", "location": "Temecula, CA", "create_at": date("2012-03-28"), "description": "20 • let's listen to Coldplay or something.", "followers_count": 1358, "friends_count": 665, "statues_count": 58559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Temecula, CA", "id": "f95304ef80fecc3f", "name": "Temecula", "place_type": "city", "bounding_box": rectangle("-117.184394,33.447541 -117.054639,33.554501") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 678120, "cityName": "Temecula" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438207955865600, "text": "@amzfine iono think that's Tongan Time that's a whole month till I see you but then again you right it's Tongan time �� you in toutupu?", "in_reply_to_status": 740436489725718528, "in_reply_to_user": 4608479473, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4608479473 }}, "user": { "id": 1629180980, "name": "Sandra Fifita", "screen_name": "fiotelisa", "lang": "en", "location": "null", "create_at": date("2013-07-28"), "description": "FAITH is trusting in GOD even when it doesn't make sense - Rosa Lynn Fifita", "followers_count": 190, "friends_count": 187, "statues_count": 16301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438208094343168, "text": "@BillyonBass like it but gotta love the surplus of bass guitars on the couch. You=the man Billy. https://t.co/DYIxn0JBhl", "in_reply_to_status": -1, "in_reply_to_user": 19257569, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 19257569 }}, "user": { "id": 19049362, "name": "Joe Cox 2", "screen_name": "xxJC2xx", "lang": "en", "location": "Cincinnati", "create_at": date("2009-01-15"), "description": "aka JC2 or the eArNiNja", "followers_count": 78, "friends_count": 625, "statues_count": 2524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Amelia, OH", "id": "ddee43f519311ab3", "name": "Amelia", "place_type": "city", "bounding_box": rectangle("-84.256409,39.003031 -84.18368,39.065799") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39025, "countyName": "Clermont", "cityID": 3901742, "cityName": "Amelia" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438208245321728, "text": "Every time I look for someone I think about how close they can to bring you", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2789005056, "name": "Përšïãñ Prįńçę®", "screen_name": "MarkellTobased", "lang": "en", "location": "null", "create_at": date("2014-09-03"), "description": "instagram: Markelltheonly |snapchat: papichuloo3 |too far gone......| #theJOURNEY", "followers_count": 1483, "friends_count": 554, "statues_count": 21756 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438208450826240, "text": "When i tweet ill tweet about numerous things. dont get me twisted lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3294223903, "name": "Dayton", "screen_name": "DaytonorParty", "lang": "en", "location": "null", "create_at": date("2015-07-25"), "description": "S o u t h e r n U n i v e r s i t y ' 2 0\n\n l o u i s i a n a", "followers_count": 109, "friends_count": 204, "statues_count": 2161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklinton, LA", "id": "136d811644ef4776", "name": "Franklinton", "place_type": "city", "bounding_box": rectangle("-90.166062,30.824052 -90.125918,30.864494") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22117, "countyName": "Washington", "cityID": 2227190, "cityName": "Franklinton" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438209059008512, "text": "@MR_KANGGUN Such talent in a young person is amazing. Good Job! https://t.co/x6RD9Goc2p", "in_reply_to_status": -1, "in_reply_to_user": 2281911146, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2281911146 }}, "user": { "id": 703638110328893440, "name": "Yazmin Lazos", "screen_name": "lazos_yazmin", "lang": "en", "location": "null", "create_at": date("2016-02-27"), "description": "null", "followers_count": 0, "friends_count": 23, "statues_count": 61 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438209356832768, "text": "My kinda bitch https://t.co/HqSTHpIK4J", "in_reply_to_status": 740438056461819904, "in_reply_to_user": 2290415514, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2290415514, "name": "tyra bank$", "screen_name": "hboogie1x", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2014-01-13"), "description": "I have freckles. don't come for me. I'll probably dunk on you.", "followers_count": 1690, "friends_count": 446, "statues_count": 56155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438209923035137, "text": "I don't know if you're aware that you've been throwing out my shoes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2273758789, "name": "Haley", "screen_name": "haleyplante8", "lang": "en", "location": "Old Orchard Beach, ME", "create_at": date("2014-01-02"), "description": "null", "followers_count": 372, "friends_count": 312, "statues_count": 5965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Old Orchard Beach, ME", "id": "af15edf0b84b6e22", "name": "Old Orchard Beach", "place_type": "city", "bounding_box": rectangle("-70.425305,43.496003 -70.353294,43.550482") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23031, "countyName": "York", "cityID": 2355120, "cityName": "Old Orchard Beach" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438209948176384, "text": "It's okay to not be okay.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 231672674, "name": "♔↭★» Sage Bug. «★↭♔™", "screen_name": "HALEYSAGE", "lang": "en", "location": "null", "create_at": date("2010-12-28"), "description": "Wrinkles only go where the smiles have been. Or when you squint too long from playing softball in the sun since you were little.", "followers_count": 1110, "friends_count": 1900, "statues_count": 27154 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jackson, MS", "id": "5c9d123437711a9d", "name": "Jackson", "place_type": "city", "bounding_box": rectangle("-90.331039,32.049347 -89.984994,32.412627") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28049, "countyName": "Hinds", "cityID": 2836000, "cityName": "Jackson" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438209969164288, "text": "Inspirational �� https://t.co/Kz7cnwzbmj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3565482853, "name": "️️#1️⃣", "screen_name": "3asymoneycam", "lang": "en", "location": "Dallas, TX", "create_at": date("2015-09-14"), "description": "I want to be the #GOAT #BasketballNeverStops", "followers_count": 409, "friends_count": 234, "statues_count": 3645 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438210665402368, "text": "6b44fa61f50972d46b7eb7c70239992881ef4b918ae6cb401250f2ffadc9253d1f2f83fe29ce2bb7e483460d4c8524f71251040806a08da2435bee654a0101191e9379000000", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.94698673,38.35746613"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3048544857, "name": "GooGuns Lulz", "screen_name": "googuns_lulz", "lang": "en", "location": "(here)", "create_at": date("2015-02-20"), "description": "@victor_zheng", "followers_count": 163, "friends_count": 1, "statues_count": 1018678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Utah, USA", "id": "1879ace9e02ace61", "name": "Utah", "place_type": "admin", "bounding_box": rectangle("-114.052999,36.997905 -109.041059,42.001619") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49001, "countyName": "Beaver" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438210728361989, "text": "Temp 58.0° Hi/Lo 59.2/57.5 Rng 1.7° WC 58.0° Hmd 81% Rain 0.00\" Storm 0.03\" BAR 29.469 Rising DP 52.2° Wnd 2mph Dir WSW Gst 14mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19334 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438210933821440, "text": "Dad 6'3\nMom5'5 \nMe 5'9 https://t.co/zc8WAAGHgG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 919921087, "name": "mich", "screen_name": "martinez_mic", "lang": "en", "location": "Stansbury Park, UT", "create_at": date("2012-11-01"), "description": "@KP_Swankyy ❤️ #21", "followers_count": 1089, "friends_count": 1200, "statues_count": 13265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stansbury Park, UT", "id": "6bad99769946da63", "name": "Stansbury Park", "place_type": "city", "bounding_box": rectangle("-112.32273,40.623527 -112.276896,40.654781") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49045, "countyName": "Tooele", "cityID": 4972720, "cityName": "Stansbury Park" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438210971590656, "text": "@StigAbell why does obama look like hunch back", "in_reply_to_status": 740434891259715584, "in_reply_to_user": 509212441, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 509212441 }}, "user": { "id": 29585812, "name": "Dajoun Kovu", "screen_name": "WEPLAYDK", "lang": "en", "location": "Your Mind", "create_at": date("2009-04-07"), "description": "Rapper. Creative director. CEO. \n.All Inquires go to weplaydk@gmail.com", "followers_count": 28383, "friends_count": 26625, "statues_count": 4003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211005165568, "text": "20:20 #Antihuman\n20:20 De #Fy\n66 #Cycles a Week,6 Cycles a Day\n20:20 #Positive #FeedbackLoop\n20:20 Gain\n#SatanCytokine #Signaling in #Stream", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Antihuman", "Fy", "Cycles", "Positive", "FeedbackLoop", "SatanCytokine", "Signaling", "Stream" }}, "user": { "id": 70626091, "name": "Left Hand Pan", "screen_name": "666Beastism999", "lang": "en", "location": "Spokane, WA", "create_at": date("2009-09-01"), "description": "Messiah, The Christ, Tezcatlipoca Card, Ace of Jihad, Sanctified Interface", "followers_count": 357, "friends_count": 277, "statues_count": 230495 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane, WA", "id": "dc3747428fa88cab", "name": "Spokane", "place_type": "city", "bounding_box": rectangle("-117.565226,47.5742 -117.303868,47.760676") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367000, "cityName": "Spokane" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211206516737, "text": "Jay-Z top 5 easily", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 974982564, "name": "BH", "screen_name": "Braylon_15", "lang": "en", "location": "CA", "create_at": date("2012-11-27"), "description": "-19\n-California/Louisiana \n-UCM\n-SC➡braylon_15", "followers_count": 298, "friends_count": 241, "statues_count": 15080 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Patterson, CA", "id": "bcb6b4eebbf9b55c", "name": "Patterson", "place_type": "city", "bounding_box": rectangle("-121.172793,37.453362 -121.098968,37.517058") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 656112, "cityName": "Patterson" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211265200128, "text": "@ebbtideapp Tide in Johns Pass, Florida 06/08/2016\nHigh 4:36am 1.4\n Low 7:40am 1.3\nHigh 1:58pm 2.5\n Low 10:11pm -0.2", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-82.7833,27.7833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 99, "friends_count": 1, "statues_count": 39576 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florida, USA", "id": "4ec01c9dbc693497", "name": "Florida", "place_type": "admin", "bounding_box": rectangle("-87.634643,24.396308 -79.974307,31.001056") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1272325, "cityName": "Treasure Island" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211584008193, "text": "������������ https://t.co/c7xDaaHcc5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 543001814, "name": "Brian", "screen_name": "SwankyBrian", "lang": "en", "location": "Ontario, CA", "create_at": date("2012-04-01"), "description": "null", "followers_count": 658, "friends_count": 234, "statues_count": 31429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211600777221, "text": "〰〰 https://t.co/KlwoWbloqO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 57116037, "name": "Deshawn✌", "screen_name": "deshawn_cornett", "lang": "en", "location": "Eastern Michigan University", "create_at": date("2009-07-15"), "description": "• Halfrican • • EmuClubGym • ✌️", "followers_count": 1255, "friends_count": 1313, "statues_count": 20198 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ypsilanti, MI", "id": "0049088b1906e51b", "name": "Ypsilanti", "place_type": "city", "bounding_box": rectangle("-83.661476,42.165629 -83.542129,42.325797") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2689140, "cityName": "Ypsilanti" } }
+{ "create_at": datetime("2016-06-08T00:00:00.000Z"), "id": 740438211634270209, "text": "@la_bree3 I thought you was there already ��", "in_reply_to_status": 740437953864962048, "in_reply_to_user": 336893528, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 336893528 }}, "user": { "id": 551482319, "name": "♠️", "screen_name": "DAVOGOCRAZY", "lang": "en", "location": "5th Ward TX. ", "create_at": date("2012-04-11"), "description": "IG:Davogocrazy SC:Almightydavo", "followers_count": 747, "friends_count": 771, "statues_count": 25534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438211785326593, "text": "Okurrrr https://t.co/UK5fdzUNjm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tr", "is_retweet": false, "user": { "id": 323645891, "name": "MIKA", "screen_name": "mikaela_orta", "lang": "en", "location": "Garden Grove, CA", "create_at": date("2011-06-24"), "description": "literally", "followers_count": 175, "friends_count": 218, "statues_count": 1573 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438211982467072, "text": "crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2949336204, "name": "ar", "screen_name": "EinsteinTurry", "lang": "en", "location": "CA", "create_at": date("2014-12-28"), "description": "null", "followers_count": 292, "friends_count": 339, "statues_count": 8774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hanford, CA", "id": "17c553c1a9dd0466", "name": "Hanford", "place_type": "city", "bounding_box": rectangle("-119.691252,36.290733 -119.583059,36.371953") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 631960, "cityName": "Hanford" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438212087255040, "text": "@UR_YKN30 \nawww...���� I miss u Yukino..��\nWhere is my awesome roommate... (That's u��)Please come back to Hawaii right now��❤️", "in_reply_to_status": 740436514786713600, "in_reply_to_user": 2876212302, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2876212302 }}, "user": { "id": 1962273404, "name": "Kanna*:)", "screen_name": "HiLife_21", "lang": "ja", "location": "St.Joseph high school", "create_at": date("2013-10-15"), "description": "Hawaii×Japan Snapchat→alo0218 Instagram→http://Instagram.com/808. Life is awesome:)", "followers_count": 458, "friends_count": 395, "statues_count": 3239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hilo, HI", "id": "00430f8519d62c37", "name": "Hilo", "place_type": "city", "bounding_box": rectangle("-155.184552,19.656471 -154.990994,19.738955") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15001, "countyName": "Hawaii", "cityID": 1514650, "cityName": "Hilo" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438212347330561, "text": "@ayy_buddah whyyyyy��", "in_reply_to_status": 740436894148919300, "in_reply_to_user": 3010666864, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3010666864 }}, "user": { "id": 312375983, "name": "Sam ✊❕", "screen_name": "SamariW_22", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2011-06-06"), "description": "Volleyball ... IG @Samariw_22 I Go To War With God Behind You❤️", "followers_count": 972, "friends_count": 986, "statues_count": 81846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438212393480192, "text": "Just posted a video @ Beyoncé- Formation World Tour Citi Field https://t.co/WaiI7COn5p", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.84848022,40.75649261"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29617201, "name": "جعفر™", "screen_name": "ANFboii", "lang": "en", "location": "NEW YORK... City of dreams...", "create_at": date("2009-04-07"), "description": "#funny #SPOILED #NICE #friendly #nikerunclub #ntc", "followers_count": 2020, "friends_count": 1202, "statues_count": 32302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438212854874112, "text": "hit me with that k!!!!! oh no bitch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 538561635, "name": "bridgette", "screen_name": "bridgetterenae_", "lang": "en", "location": "baton rouge, la", "create_at": date("2012-03-27"), "description": "lsu'19 | 19 ⚯͛ | sc: bridgetteee_k", "followers_count": 764, "friends_count": 848, "statues_count": 24195 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Meraux, LA", "id": "6c18cdfa012de587", "name": "Meraux", "place_type": "city", "bounding_box": rectangle("-89.944711,29.906566 -89.894772,29.949545") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2249800, "cityName": "Meraux" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438212917768192, "text": "YEAAAAAAASSSSSSS SHE CHOSE ALEX!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2918202042, "name": "Finney", "screen_name": "SierraFinney_", "lang": "en", "location": "W/ Chloe and J", "create_at": date("2014-12-03"), "description": "HHS 18", "followers_count": 330, "friends_count": 379, "statues_count": 1275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438213047750656, "text": "06/08@03:00 - Temp 60.0F, WC 60.0F. Wind 0.0mph WSW, Gust 2.0mph. Bar 29.685in, Rising slowly. Rain 0.00in. Hum 77%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49518 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438213316214784, "text": "@andy_radilla lol", "in_reply_to_status": 740431733292716033, "in_reply_to_user": 318330395, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 318330395 }}, "user": { "id": 1919953904, "name": "IVAN GARCIA", "screen_name": "iivanngarcia", "lang": "en", "location": "IG: iivancashh", "create_at": date("2013-09-30"), "description": "gggupholstery@yahoo.com SC: iivangarciaa", "followers_count": 125, "friends_count": 90, "statues_count": 869 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438213450444800, "text": "Omm �� https://t.co/NuB1SZrq1Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 521796924, "name": "#LongLiveCoop✊", "screen_name": "LiyahDalicia", "lang": "en", "location": "Magnolia, AR", "create_at": date("2012-03-11"), "description": "#LongLiveTyga #LongLiveCoop", "followers_count": 2106, "friends_count": 2067, "statues_count": 101943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Magnolia, AR", "id": "71e16cd97a552f44", "name": "Magnolia", "place_type": "city", "bounding_box": rectangle("-93.267708,33.23834 -93.208361,33.309264") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5027, "countyName": "Columbia", "cityID": 543460, "cityName": "Magnolia" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438213744066561, "text": "@all4kayy Rd good bring some snacks ��", "in_reply_to_status": 740438097188532224, "in_reply_to_user": 3099760715, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3099760715 }}, "user": { "id": 3239674683, "name": "All In Cavs ‼️", "screen_name": "_stevenyc", "lang": "en", "location": "null", "create_at": date("2015-05-06"), "description": "God Got Me", "followers_count": 509, "friends_count": 385, "statues_count": 41936 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438213811175424, "text": "Wind 0.5 mph NW. Barometer 29.459 in, Steady. Temperature 54.3 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214033461248, "text": "Me eating oomf mashed potatoes https://t.co/KmbtPpJxn4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 320620503, "name": "Po' Namath", "screen_name": "BoBundyy", "lang": "en", "location": "Hogwarts School of Wizardry'19", "create_at": date("2011-06-20"), "description": "281.330.8004 Bookings or Collaborations MGMT: Vaccid@gmail.com", "followers_count": 1195, "friends_count": 316, "statues_count": 46904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214234808320, "text": "ion feel good ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1586157127, "name": "(tah-jay)❤️", "screen_name": "lahtai3", "lang": "en", "location": "LongLiveKay❤️", "create_at": date("2013-07-11"), "description": "everyday you'll face a different challenge .", "followers_count": 1437, "friends_count": 719, "statues_count": 51723 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214465486848, "text": "@damnitjanett I'm getting damn near attacked because \"Hillary and Bernie agree on most issues so you must be a misogynist\"", "in_reply_to_status": 740437962027085825, "in_reply_to_user": 95142633, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 95142633 }}, "user": { "id": 181780385, "name": "STEVE MCQUEEN", "screen_name": "bertoloochie", "lang": "en", "location": "Hayward, CA", "create_at": date("2010-08-22"), "description": "HATERS CAN'T STAND ME, SHOULDA WON A GRAMMY", "followers_count": 468, "friends_count": 679, "statues_count": 46352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermiston, OR", "id": "3b213491c5ae3f7e", "name": "Hermiston", "place_type": "city", "bounding_box": rectangle("-119.32694,45.824334 -119.258928,45.864803") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41059, "countyName": "Umatilla", "cityID": 4133700, "cityName": "Hermiston" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214675173376, "text": "@thereaaldee always", "in_reply_to_status": 737528978353721345, "in_reply_to_user": 2638774787, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2638774787 }}, "user": { "id": 2638774787, "name": "6/12.", "screen_name": "thereaaldee", "lang": "en", "location": "null", "create_at": date("2014-06-24"), "description": "Im a brownskin bitch & you luvv det.", "followers_count": 1155, "friends_count": 547, "statues_count": 39839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214700322816, "text": "Wind 0.0 mph ---. Barometer 1016.83 mb, Steady. Temperature 63.4 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 32, "friends_count": 129, "statues_count": 14964 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214771671041, "text": "I wish I was drunk rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 41819553, "name": "Adela", "screen_name": "_yungbae", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-22"), "description": "null", "followers_count": 1009, "friends_count": 468, "statues_count": 24830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438214964563969, "text": "Wind 0.0 mph ---. Barometer 29.934 in, Steady. Temperature 59.2 °F. Rain today 0.00 in. Humidity 88%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215111380992, "text": "Wind 0.0 mph ---. Barometer 29.859 in, Steady. Temperature 65.9 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215224664064, "text": "So many people don't believe my age. I went to superior grill for a pre-game drink last year and the bartender said I didn't even look 18��️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2383105806, "name": "nikki", "screen_name": "nikkirpr", "lang": "en", "location": "Nashville, TN", "create_at": date("2014-03-10"), "description": "LSU alum • PR • social media • content creator • San Fran sports fan", "followers_count": 365, "friends_count": 811, "statues_count": 8539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215262363653, "text": "When you finally off the block list ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 308943375, "name": "im.dude_", "screen_name": "imdude_", "lang": "en", "location": "Independence, Louisiana", "create_at": date("2011-06-01"), "description": "null", "followers_count": 1649, "friends_count": 1789, "statues_count": 26896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, LA", "id": "0020aaa25ced13e2", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-90.520806,30.615873 -90.489165,30.674604") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22105, "countyName": "Tangipahoa", "cityID": 2237025, "cityName": "Independence" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215274987520, "text": "damn I just be up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2908567453, "name": "ᗪjayy⚔", "screen_name": "Trvll_DJ", "lang": "en", "location": "Yazoo City, MS", "create_at": date("2014-11-23"), "description": "bitch I'm ridiculous .", "followers_count": 453, "friends_count": 671, "statues_count": 937 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yazoo City, MS", "id": "d2b0d8afaf20a2c0", "name": "Yazoo City", "place_type": "city", "bounding_box": rectangle("-90.443752,32.833535 -90.37505,32.890747") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28163, "countyName": "Yazoo", "cityID": 2881520, "cityName": "Yazoo City" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215681814528, "text": "This is your lovely @kyliexquinn https://t.co/0N3gxlKq4J", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4473214953 }}, "user": { "id": 4151982796, "name": "Kacey Quinn", "screen_name": "KaceyQuinnxxx", "lang": "en", "location": "Florida, LA, NYC", "create_at": date("2015-11-09"), "description": "Queen Quinn • White Cap Chick• Hussie Model • Good Vibe Generator • Bookings: info@hussiemodels.com • Contact: kaceyquinn@hotmail.com", "followers_count": 19484, "friends_count": 511, "statues_count": 2280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Coral Springs, FL", "id": "6241025d720c4971", "name": "Coral Springs", "place_type": "city", "bounding_box": rectangle("-80.297884,26.228692 -80.201661,26.31183") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1214400, "cityName": "Coral Springs" } }
+{ "create_at": datetime("2016-06-08T00:00:01.000Z"), "id": 740438215857963009, "text": "@ansenk615 @jamieoliver wow, such ugliness", "in_reply_to_status": 740438108378923009, "in_reply_to_user": 2257991014, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2257991014, 18676177 }}, "user": { "id": 14346303, "name": "Jerry James Stone", "screen_name": "jerryjamesstone", "lang": "en", "location": "Sacramento, CA", "create_at": date("2008-04-09"), "description": "Food pornographer, full-time vegetarian, unapologetic liberal, beard model, and I say hella too much. My tweets are tasty, geeky or just plain odd.", "followers_count": 107097, "friends_count": 1573, "statues_count": 42609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438216260620289, "text": "I only ate once today:/ I'm so hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 102229364, "name": "hannah♕", "screen_name": "hanluvv", "lang": "en", "location": "the stars", "create_at": date("2010-01-05"), "description": "take me to the stars•sc hanns.escobar♐️", "followers_count": 604, "friends_count": 439, "statues_count": 19468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438216273240064, "text": "1 am and Brandon literally just came over to get eats some cookies ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 963670304, "name": "V", "screen_name": "VDeLaSecret", "lang": "en", "location": "Tulsa, OK", "create_at": date("2012-11-21"), "description": "• No matter what always remember how blessed you are • Brandon❤️", "followers_count": 585, "friends_count": 477, "statues_count": 7422 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tulsa, OK", "id": "cb74aaf709812e0f", "name": "Tulsa", "place_type": "city", "bounding_box": rectangle("-96.065628,35.968624 -95.761656,36.250159") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40143, "countyName": "Tulsa", "cityID": 4075000, "cityName": "Tulsa" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438216545816576, "text": "Shouldn't have had drank that Coke. \n\n#WideAwakePart134980", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "WideAwakePart134980" }}, "user": { "id": 19316168, "name": "Marius Andrei", "screen_name": "MARIUS_DTOWN", "lang": "en", "location": "ÜT: 34.036252,-118.236045", "create_at": date("2009-01-21"), "description": "Jack of all trades, master of none. I tweet at my own risk, 140 characters at a time. Recovering Catholic. Underwear Salesman.", "followers_count": 1600, "friends_count": 994, "statues_count": 48935 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central City East, Los Angeles", "id": "61d3a50809879273", "name": "Central City East", "place_type": "neighborhood", "bounding_box": rectangle("-118.239675,34.014714 -118.215014,34.071979") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438216629735424, "text": "ya girl is eighteen now :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1323619958, "name": "peru", "screen_name": "PeriGoucher", "lang": "en", "location": "null", "create_at": date("2013-04-02"), "description": "lover of living water // HIU '20", "followers_count": 580, "friends_count": 361, "statues_count": 6208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prescott, AZ", "id": "193e889ce36e6e29", "name": "Prescott", "place_type": "city", "bounding_box": rectangle("-112.537378,34.512365 -112.373682,34.626579") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4025, "countyName": "Yavapai", "cityID": 457380, "cityName": "Prescott" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438216738766849, "text": "Wind 0.0 mph ---. Barometer 29.879 in, Rising slowly. Temperature 52.2 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20412 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438217359556608, "text": "Look at my little guy! https://t.co/AJ9YNhN1rw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 467090541, "name": "HaRiSsssss", "screen_name": "HarisSkiljo", "lang": "en", "location": "null", "create_at": date("2012-01-17"), "description": "Soccer⚽️➖Pharmacy school➖BIH✈️✈️NKY➖", "followers_count": 1029, "friends_count": 1034, "statues_count": 28828 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Florence, KY", "id": "dcd73905ee565216", "name": "Florence", "place_type": "city", "bounding_box": rectangle("-84.706797,38.917578 -84.606098,39.060938") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21015, "countyName": "Boone", "cityID": 2127982, "cityName": "Florence" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438217455984641, "text": "Temp: 68.9°F Wind:0.0mph Pressure: 29.889hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66445 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438217753825280, "text": "Still waiting for mani to tell us what she doesn't understand ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 517117919, "name": "itsJen✨", "screen_name": "_heyjalendenise", "lang": "en", "location": "minding MY business", "create_at": date("2012-03-06"), "description": "[met.803] Instagram: _jay.denise SC: jalendenise @gucci1017 is my daddy✊ #guccibackbitch #Blacklivesmatter❤️", "followers_count": 754, "friends_count": 674, "statues_count": 53507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irmo, SC", "id": "189ef793a6dbbfc8", "name": "Irmo", "place_type": "city", "bounding_box": rectangle("-81.257113,34.04725 -81.132587,34.170797") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4535890, "cityName": "Irmo" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438218194178048, "text": "Dont pretend Hilary Clinton is a saint. She is a white privileged criminal. You cant ignore all the shit shes done because you're a feminist", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621434754, "name": "baby j", "screen_name": "rodriiiiiiguez", "lang": "en", "location": "null", "create_at": date("2012-06-28"), "description": "marshALL❤️", "followers_count": 712, "friends_count": 591, "statues_count": 16400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Goodyear, AZ", "id": "00fae4950337e465", "name": "Goodyear", "place_type": "city", "bounding_box": rectangle("-112.508916,33.317555 -112.341035,33.50819") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 428380, "cityName": "Goodyear" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438218483568641, "text": "@_ElMando okay I will , forgot you've been around so much you're tired . It's okay I got u tho ✊��", "in_reply_to_status": 740437734398013440, "in_reply_to_user": 233876751, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 233876751 }}, "user": { "id": 739860681499910144, "name": "Alisha Vallejo", "screen_name": "MakeupByAli_", "lang": "en", "location": "Yuma, AZ", "create_at": date("2016-06-06"), "description": "• Makeup Enthusiast • DM for bookings • Engaged to Adrian Melendrez • Instagram : @alisharv_", "followers_count": 43, "friends_count": 53, "statues_count": 24 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yuma, AZ", "id": "607358f5dbace76a", "name": "Yuma", "place_type": "city", "bounding_box": rectangle("-114.710936,32.601699 -114.458569,32.734737") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4027, "countyName": "Yuma", "cityID": 485540, "cityName": "Yuma" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438218559131648, "text": "Ripley SW Limestone Co. Temp: 65.1°F Wind:1.6mph Pressure: 993.1mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 54244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438218785591297, "text": "@loccahontas yikes", "in_reply_to_status": 740437817944354818, "in_reply_to_user": 53806485, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 53806485 }}, "user": { "id": 57765215, "name": "Alex .English", "screen_name": "AleksEnglish", "lang": "en", "location": "California, USA", "create_at": date("2009-07-17"), "description": "Neutral vibes.", "followers_count": 737, "friends_count": 688, "statues_count": 133224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438218894671873, "text": "Amend !!! �������� https://t.co/WisqC3F3M7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2273428010, "name": "m a i a ✨", "screen_name": "maialanee", "lang": "en", "location": "Mansfield, TX", "create_at": date("2014-01-02"), "description": "sc me @maialanee", "followers_count": 842, "friends_count": 1007, "statues_count": 18631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mansfield, TX", "id": "012a07a105d10298", "name": "Mansfield", "place_type": "city", "bounding_box": rectangle("-97.186747,32.530915 -97.046821,32.616182") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4846452, "cityName": "Mansfield" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219020460032, "text": "lil messy baby haha cx https://t.co/yDyVsWsDBq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226889443, "name": "izzy", "screen_name": "coffee_berry_", "lang": "en", "location": "Chula Vista, CA", "create_at": date("2015-05-25"), "description": "I wander with a cute cat that goes meow + I guess we're girlfriends according to her bio even though I haven't officially asked her yet haha, hello beautiful", "followers_count": 179, "friends_count": 411, "statues_count": 777 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bay City, MI", "id": "0121f9435fdae948", "name": "Bay City", "place_type": "city", "bounding_box": rectangle("-83.97498,43.538242 -83.842876,43.696322") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26017, "countyName": "Bay", "cityID": 2606020, "cityName": "Bay City" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219108560896, "text": "@HayesGrier happy birthday��✨ ��make good decisions�� make sure its lit at that party of yours tho�� keep it PG���� love u tho����", "in_reply_to_status": -1, "in_reply_to_user": 330705266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 330705266 }}, "user": { "id": 3298481110, "name": "CAMERONS WIFE!!!!", "screen_name": "JessicaJakab", "lang": "en", "location": "null", "create_at": date("2015-05-25"), "description": "Instagram: jessica_jakab musical.ly:jessica_jakab ❤️cameron dallas❤️", "followers_count": 52, "friends_count": 117, "statues_count": 1341 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunedin, FL", "id": "608a091166e5ddb0", "name": "Dunedin", "place_type": "city", "bounding_box": rectangle("-82.8245,27.997228 -82.738199,28.061609") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12103, "countyName": "Pinellas", "cityID": 1218575, "cityName": "Dunedin" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219112742912, "text": "Working on my life story", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 22441410, "name": "⍲ĸ253", "screen_name": "akumar253", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-03-01"), "description": "M.O.B.", "followers_count": 125, "friends_count": 121, "statues_count": 3287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Federal Way, WA", "id": "f3f3f3c8dbba3fd3", "name": "Federal Way", "place_type": "city", "bounding_box": rectangle("-122.419282,47.257187 -122.24964,47.357949") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5323515, "cityName": "Federal Way" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219704111104, "text": "@soulxpunk https://t.co/6q46kex5cQ", "in_reply_to_status": -1, "in_reply_to_user": 171980778, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 171980778 }}, "user": { "id": 1599176508, "name": "Mariah ⚽️", "screen_name": "MariahhVielmaa_", "lang": "en", "location": "null", "create_at": date("2013-07-16"), "description": "soccer ⚽️ Philippians 4:13 ✨", "followers_count": 829, "friends_count": 644, "statues_count": 15908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219817426945, "text": "Anyone know of places hiring for summer in AZ?! ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 341238285, "name": "Bacall Sterling", "screen_name": "BB_Sterling", "lang": "en", "location": "MI | AZ | CO", "create_at": date("2011-07-23"), "description": "track&field CMU", "followers_count": 534, "friends_count": 362, "statues_count": 14808 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438219989323780, "text": "Wind 0.0 mph W. Barometer 29.57 in, Steady. Temperature 58.3 °F. Rain today 0.00 in. Humidity 82%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 30, "friends_count": 122, "statues_count": 160377 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-08T00:00:02.000Z"), "id": 740438220039684096, "text": "3am texting buddies?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 171291078, "name": "Kevin Durants Son✨", "screen_name": "BrinsonDrizzy", "lang": "en", "location": "ATL Zone 6⚠️", "create_at": date("2010-07-26"), "description": "Californian Baby | Born & Raised | Humble & Hooper | Business Man |19| Hampton University| Practice Until Perfection | #HamptonU18 ⚓️ #OphiO16", "followers_count": 2490, "friends_count": 2738, "statues_count": 49864 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dunwoody, GA", "id": "9c9fa69deb120a34", "name": "Dunwoody", "place_type": "city", "bounding_box": rectangle("-84.348046,33.915238 -84.264931,33.970907") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13089, "countyName": "DeKalb", "cityID": 1324768, "cityName": "Dunwoody" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220463321088, "text": "Epix produced the doc? The service is so low rent! The only ones to air Courics bullshit! https://t.co/vJiX5bmHjG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715276501650776064, "name": "Robertgreiner", "screen_name": "Robertgreiner71", "lang": "en", "location": "San Diego, CA", "create_at": date("2016-03-30"), "description": "Toker, old man, hate everybody equally,pissed off at everything!", "followers_count": 319, "friends_count": 539, "statues_count": 3885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220522020865, "text": "@brentsimmons why did #Banshee have a female FBI agent shooting heroin is that the norm now? @weeklystandard @nytimes @AP @Reuters", "in_reply_to_status": 740388838984568834, "in_reply_to_user": 652293, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Banshee" }}, "user_mentions": {{ 652293, 17546958, 807095, 51241574, 1652541 }}, "user": { "id": 707256854448107520, "name": "Sheyma Gates, PhD", "screen_name": "784578GGBATski", "lang": "en", "location": "Pickerington, OH", "create_at": date("2016-03-08"), "description": "RTs may or may may not mean endorsement enter at your own peril", "followers_count": 0, "friends_count": 59, "statues_count": 12279 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pickerington, OH", "id": "3995cc1483801d24", "name": "Pickerington", "place_type": "city", "bounding_box": rectangle("-82.797752,39.841431 -82.684335,39.939034") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39045, "countyName": "Fairfield", "cityID": 3962498, "cityName": "Pickerington" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220622704640, "text": "74.5F (Feels: 80.8F) - Humidity: 90% - Wind: 0.0mph ESE - Gust: 0.0mph - Pressure: 29.767in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 241589 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220744298496, "text": "Malaysia jails a teenager for insulting royalty on Facebook https://t.co/ZkEYCnhaj0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17638, "friends_count": 17805, "statues_count": 72788 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220744314880, "text": "on the other I'm so FUCKING GLAD you're not in my life no more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995796931, "name": "jul$", "screen_name": "idioxsyncratic", "lang": "en", "location": "SALTHATECITY", "create_at": date("2012-12-07"), "description": "18 & deadass woke", "followers_count": 1150, "friends_count": 1371, "statues_count": 45145 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438220849205248, "text": "Yes but totally not ready for summer classes https://t.co/Nypctcuwnd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2513116688, "name": "제이안", "screen_name": "jynsrn162", "lang": "en", "location": "Oxnard, CA", "create_at": date("2014-05-21"), "description": "jeon jungkook's wife | Song Joong Ki's fiance| BTS 방탄소년단 | Bangtan trash | VKook trash | multifandom | Ariana G stan | IG: jayannesarno Snapchat: jayanne.sarno", "followers_count": 212, "friends_count": 631, "statues_count": 11719 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oxnard, CA", "id": "a3c0ae863771d69e", "name": "Oxnard", "place_type": "city", "bounding_box": rectangle("-119.248624,34.101755 -119.082154,34.262157") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 654652, "cityName": "Oxnard" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221587402752, "text": "I'm ready to smack the shit out of you @YRNKD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 321460148 }}, "user": { "id": 4808466535, "name": "#IAMSNOOP", "screen_name": "MissinSnoop_", "lang": "en", "location": "null", "create_at": date("2016-01-24"), "description": "SnapChat , TaniyahGhee", "followers_count": 176, "friends_count": 105, "statues_count": 993 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221591617536, "text": "Wind 0.0 mph ---. Barometer 29.900 in, Rising slowly. Temperature 70.8 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 14, "friends_count": 53, "statues_count": 9017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221637726208, "text": "my little brother is screaming in his sleep. send help.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2404997496, "name": "alexx liddicote", "screen_name": "alexxtooturnt", "lang": "en", "location": "california all stars", "create_at": date("2014-03-22"), "description": "#LCH cali cobalt 15/16 cali onyx 16/17", "followers_count": 102, "friends_count": 228, "statues_count": 1201 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eastvale, CA", "id": "007b3322101aab55", "name": "Eastvale", "place_type": "city", "bounding_box": rectangle("-117.611068,33.925197 -117.549003,34.03349") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 621230, "cityName": "Eastvale" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221683822596, "text": "https://t.co/STisA46nW3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 52260263, "name": "Preston Middleton", "screen_name": "bassplayerPres", "lang": "en", "location": "Atlanta GA", "create_at": date("2009-06-29"), "description": "Next Wave Films LLC/Music Creator/Music Exec/ Digital content specialist/CEO Next Wave Entertainment \nI create...", "followers_count": 829, "friends_count": 2087, "statues_count": 7231 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221880954881, "text": "@beeyonka_ @corinnealeixa ������������ https://t.co/XQSrPnUvn6", "in_reply_to_status": -1, "in_reply_to_user": 1242735230, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1242735230, 1636162045 }}, "user": { "id": 120664266, "name": "jackie", "screen_name": "applejackxx_", "lang": "en", "location": "da bay bruh", "create_at": date("2010-03-06"), "description": "¯\\_༼;༎ຶ۝༎ຶ༽_/¯", "followers_count": 213, "friends_count": 362, "statues_count": 17712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438221927153664, "text": "Jus got home from #streetsofcompton #moviepremiere s/o to @thegame viewing was lit… https://t.co/wtJqZqxbKH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0522"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "streetsofcompton", "moviepremiere" }}, "user_mentions": {{ 28895421 }}, "user": { "id": 20859178, "name": "THE REAL YUNG L.A.", "screen_name": "therealyungla", "lang": "en", "location": "SOUTH CENTRAL,CALI", "create_at": date("2009-02-14"), "description": "EXPECT NUTHIN MO' OR NUTHIN LESS Than THE REAL... GET FAMLIAR... BROTHERS FROM ANOTHA x BLACK FRIDAI ...BOOKING: THEREALYUNGLA@GMAIL.COM", "followers_count": 2540, "friends_count": 1851, "statues_count": 25694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438222149419008, "text": "ITS OFFICIALLY ONE WEEK UNTIL I GET MY BRACES OFF AHHHHHHHHHH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 949809846, "name": "leese2salty", "screen_name": "elissechristina", "lang": "en", "location": "disneyland", "create_at": date("2012-11-15"), "description": "senior class of 2017 • @mcgroober457 ❤️", "followers_count": 778, "friends_count": 699, "statues_count": 29678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Surprise, AZ", "id": "4894f2226f25db16", "name": "Surprise", "place_type": "city", "bounding_box": rectangle("-112.46036,33.579566 -112.298534,33.713743") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 471510, "cityName": "Surprise" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438222480809984, "text": "@Its_Myracolee https://t.co/cTvaeTa7Tl", "in_reply_to_status": -1, "in_reply_to_user": 592568604, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 592568604 }}, "user": { "id": 592568604, "name": "m", "screen_name": "Its_Myracolee", "lang": "en", "location": "Hayward, CA", "create_at": date("2012-05-28"), "description": "RestEazyPops", "followers_count": 390, "friends_count": 261, "statues_count": 11444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438222673698816, "text": "@DjLiive ��\n\nI know you not..... or are you? ��\n#QTNA https://t.co/m13377Lt1F", "in_reply_to_status": -1, "in_reply_to_user": 272771643, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "QTNA" }}, "user_mentions": {{ 272771643 }}, "user": { "id": 224866419, "name": "Kourtney Dee", "screen_name": "OnTheKourt_", "lang": "en", "location": "Peace & Harmony", "create_at": date("2010-12-09"), "description": "Romans 8:18. Currently Finding My Independence As A Woman. God Fearing. SHSU. Mary & Louis Above. LongLiveLeon #UnbreakableRebel", "followers_count": 1066, "friends_count": 1043, "statues_count": 69732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438222715686912, "text": "@HayesGrier happy birthday fool ❤️ #happybirthdayhayes TREND!!!❤️", "in_reply_to_status": -1, "in_reply_to_user": 330705266, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "happybirthdayhayes" }}, "user_mentions": {{ 330705266 }}, "user": { "id": 2872381368, "name": "Tina", "screen_name": "Tinaturnttho", "lang": "en", "location": "null", "create_at": date("2014-10-22"), "description": "um no thank you.", "followers_count": 274, "friends_count": 458, "statues_count": 4514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Simi Valley, CA", "id": "c84cc6061e2af8da", "name": "Simi Valley", "place_type": "city", "bounding_box": rectangle("-118.829753,34.222878 -118.633045,34.311743") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 672016, "cityName": "Simi Valley" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438222854103050, "text": "@KingPeezo ��������������", "in_reply_to_status": 740438093946327040, "in_reply_to_user": 706291856, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 706291856 }}, "user": { "id": 2432421715, "name": "Michael Diaz", "screen_name": "Kusheroats", "lang": "en", "location": "S.K.N", "create_at": date("2014-04-07"), "description": "the guy with the hair♍️", "followers_count": 271, "friends_count": 324, "statues_count": 3609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Benbrook, TX", "id": "3327a6de18def29a", "name": "Benbrook", "place_type": "city", "bounding_box": rectangle("-97.512109,32.647286 -97.411765,32.723883") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4807552, "cityName": "Benbrook" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438223097364480, "text": "@leonelcommas tell her my boy @ChrisDeLaMora7 is down", "in_reply_to_status": 740437923418476545, "in_reply_to_user": 1561959079, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1561959079, 521946242 }}, "user": { "id": 728308360362106880, "name": "BMSMIKE", "screen_name": "majinbuumike", "lang": "en", "location": "California, USA", "create_at": date("2016-05-05"), "description": "Well Done", "followers_count": 139, "friends_count": 131, "statues_count": 308 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shafter, CA", "id": "3b642703733bf53f", "name": "Shafter", "place_type": "city", "bounding_box": rectangle("-119.300846,35.463437 -119.251513,35.521029") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 671106, "cityName": "Shafter" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438223151845376, "text": "I want the Violet Voss Holy Grail palette but it's sold out :-(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1622162688, "name": "Grazielle Domingo", "screen_name": "yesitsgrazielle", "lang": "en", "location": "null", "create_at": date("2013-07-25"), "description": "Destruction Forces Growth.", "followers_count": 387, "friends_count": 359, "statues_count": 5024 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438223197986817, "text": "@saltyaudrey yeah pretty sure ��", "in_reply_to_status": 740437463173345282, "in_reply_to_user": 623289713, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 623289713 }}, "user": { "id": 4608559453, "name": "Lex", "screen_name": "imahoe22", "lang": "en", "location": "Portland, OR", "create_at": date("2015-12-19"), "description": "WHY YOU GET BANANAS BUT WE DON'T?", "followers_count": 91, "friends_count": 94, "statues_count": 2061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438223546159104, "text": "Damn that's such a good movie", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 394496651, "name": "Ashlee M. Hernandez", "screen_name": "_ashleemarieee", "lang": "en", "location": "null", "create_at": date("2011-10-19"), "description": "M•A•C MUA | 559 area.", "followers_count": 772, "friends_count": 376, "statues_count": 18675 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-06-08T00:00:03.000Z"), "id": 740438224036896768, "text": "He fucked? https://t.co/c2NYFKcxCH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35478824, "name": "7", "screen_name": "AppleciderPapi", "lang": "en", "location": "LA", "create_at": date("2009-04-26"), "description": "19. Muslim. Beautiful. Genius. Retired HOF tweeter.", "followers_count": 6092, "friends_count": 965, "statues_count": 172726 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438225030877185, "text": "got me up all night https://t.co/p6DHHlhgv4", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1564933728, "name": "keaho♛", "screen_name": "keahoxa", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2013-07-02"), "description": "luv me", "followers_count": 852, "friends_count": 519, "statues_count": 34426 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Beach, HI", "id": "6bd73386ffaba450", "name": "Ewa Beach", "place_type": "city", "bounding_box": rectangle("-158.028613,21.306027 -157.990042,21.332114") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507450, "cityName": "Ewa Beach" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438225769107457, "text": "I love all of my friends and all the people I've met along the way. Y'all dope af.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 815692945, "name": "Bobs", "screen_name": "morris_bobbi", "lang": "en", "location": "null", "create_at": date("2012-09-10"), "description": "BOBBI BITCH", "followers_count": 753, "friends_count": 444, "statues_count": 15965 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438225802661888, "text": "������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 607317862, "name": "Adri Torres", "screen_name": "eadritorres", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-06-13"), "description": "be happy", "followers_count": 1464, "friends_count": 581, "statues_count": 48124 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438225811046400, "text": "gass ⛽️����!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938628004, "name": "QueenCay❣", "screen_name": "EvansCayci", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-12-21"), "description": "JEHS// Avondale Legends// Barbizon Model // GBG", "followers_count": 400, "friends_count": 406, "statues_count": 5770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marrero, LA", "id": "2bca99f2a5ba7da1", "name": "Marrero", "place_type": "city", "bounding_box": rectangle("-90.139232,29.853716 -90.08087,29.910044") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2248785, "cityName": "Marrero" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438225995616256, "text": "I GIVE UP TRYING", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 397075505, "name": "nicole♡", "screen_name": "niicoleeexx3_", "lang": "en", "location": "Ewa Beach ", "create_at": date("2011-10-23"), "description": "just do you baby girl✨", "followers_count": 789, "friends_count": 1149, "statues_count": 14365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Gentry, HI", "id": "5a26cb278ec35754", "name": "Ewa Gentry", "place_type": "city", "bounding_box": rectangle("-158.048025,21.321044 -158.004882,21.350693") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507470, "cityName": "Ewa Gentry" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438226280779780, "text": "Gotta see what's up with you two ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2944611242, "name": "Tamar JR Johnson", "screen_name": "Tjrj3_", "lang": "en", "location": "null", "create_at": date("2014-12-26"), "description": "Time doesn't stop moving forward. Neither will I.", "followers_count": 567, "friends_count": 536, "statues_count": 2534 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438226389893120, "text": "Maybe I'm the one who messed up in giving you two chances and also something so precious of mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2858694120, "name": "kp", "screen_name": "kaylieperezz", "lang": "en", "location": "null", "create_at": date("2014-10-16"), "description": "Powerlifting | Sixteen | Galatians 2:20", "followers_count": 289, "friends_count": 400, "statues_count": 9224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victoria, TX", "id": "70f5587b3e27a105", "name": "Victoria", "place_type": "city", "bounding_box": rectangle("-97.045657,28.709293 -96.900168,28.895775") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48469, "countyName": "Victoria", "cityID": 4875428, "cityName": "Victoria" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438226859610112, "text": "@BIGGIRON45 @WORIDSTARHIPHOP pork.", "in_reply_to_status": 740427490288635904, "in_reply_to_user": 730589941097332736, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 730589941097332736, 2913627307 }}, "user": { "id": 1711739911, "name": "Bruno", "screen_name": "LolBrunoo", "lang": "en", "location": "United States", "create_at": date("2013-08-29"), "description": "null", "followers_count": 113, "friends_count": 88, "statues_count": 2301 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hughson, CA", "id": "62548cdeae2a4b43", "name": "Hughson", "place_type": "city", "bounding_box": rectangle("-120.879351,37.589058 -120.851611,37.609315") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 634904, "cityName": "Hughson" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438227165806592, "text": "Told Selma I voted for Trump today even though she knew that I worked ALL DAY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3710183294, "name": "Rudybooty", "screen_name": "Cuffgawd", "lang": "en", "location": "null", "create_at": date("2015-09-27"), "description": "18", "followers_count": 319, "friends_count": 255, "statues_count": 4863 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438227308453889, "text": "@SuperDuperGabe_ i fucking adore you BIH", "in_reply_to_status": -1, "in_reply_to_user": 247011037, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 247011037 }}, "user": { "id": 798728354, "name": "Uncle T", "screen_name": "TaisMorga", "lang": "en", "location": "zona!!!! (-: ", "create_at": date("2012-09-02"), "description": "optimist sc : twerking_4wifi", "followers_count": 2072, "friends_count": 1544, "statues_count": 19876 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438227643961345, "text": "�� https://t.co/DLGMv0hma2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1000544017, "name": "xochi", "screen_name": "smhyesi", "lang": "en", "location": "CA | Jalisco ", "create_at": date("2012-12-09"), "description": "ni que fueras la última coca cola en el desierto", "followers_count": 911, "friends_count": 1413, "statues_count": 64135 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-08T00:00:04.000Z"), "id": 740438227694329856, "text": "A more chatty post went up on the blog a few days ago about the NZ art show and a few things… https://t.co/BGuSfFELBA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.3768306,34.0820666"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2990776544, "name": "Bella Foster", "screen_name": "bellafosterblog", "lang": "en", "location": "nz", "create_at": date("2015-01-19"), "description": "fashion + photography + travel", "followers_count": 35, "friends_count": 45, "statues_count": 86 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800595376939009, "text": "It's nothing but a G thing", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1043730722, "name": "b a v i d ++", "screen_name": "David_3953", "lang": "en", "location": "yo no se", "create_at": date("2012-12-28"), "description": "always strive and prosper", "followers_count": 281, "friends_count": 324, "statues_count": 7088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800595708239872, "text": "King bed with silk sheets is the move", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3250361629, "name": "Anthony Berlin", "screen_name": "anthonydberlin", "lang": "en", "location": "Lake Havasu City, AZ", "create_at": date("2015-06-19"), "description": "alumni", "followers_count": 377, "friends_count": 278, "statues_count": 4538 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Havasu City, AZ", "id": "5f83cd36a6136fd4", "name": "Lake Havasu City", "place_type": "city", "bounding_box": rectangle("-114.37582,34.449228 -114.245399,34.532878") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave", "cityID": 439370, "cityName": "Lake Havasu City" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800595989270529, "text": "@ravenbritt all the time seriously I do nothing!!", "in_reply_to_status": 740800505786605569, "in_reply_to_user": 344813111, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 344813111 }}, "user": { "id": 2886875174, "name": "emily", "screen_name": "emiilybell", "lang": "en", "location": "lumberton, tx", "create_at": date("2014-11-01"), "description": "null", "followers_count": 889, "friends_count": 434, "statues_count": 4369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lumberton, TX", "id": "228a068876235841", "name": "Lumberton", "place_type": "city", "bounding_box": rectangle("-94.258061,30.200365 -94.173977,30.30148") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48199, "countyName": "Hardin", "cityID": 4845120, "cityName": "Lumberton" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800596018638848, "text": "@Alexis_O97 my nigga bute", "in_reply_to_status": -1, "in_reply_to_user": 733701812205785088, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 733701812205785088 }}, "user": { "id": 721393523585470464, "name": "Brandon Hernandez", "screen_name": "kream_brandoo", "lang": "en", "location": "Alief Tx", "create_at": date("2016-04-16"), "description": "#HalaMadridYNadaMasYadigg", "followers_count": 150, "friends_count": 118, "statues_count": 1280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission Bend, TX", "id": "2a9e190efe38237e", "name": "Mission Bend", "place_type": "city", "bounding_box": rectangle("-95.681932,29.680892 -95.6342,29.719902") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848772, "cityName": "Mission Bend" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800596123455488, "text": "@miguelfyb @CaseyElton https://t.co/9Y783OTxRy", "in_reply_to_status": -1, "in_reply_to_user": 2320336363, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2320336363, 184263160 }}, "user": { "id": 341258747, "name": "RicoTucs", "screen_name": "DeBlankface", "lang": "en", "location": "Chico st", "create_at": date("2011-07-23"), "description": "null", "followers_count": 571, "friends_count": 613, "statues_count": 38460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800596211564545, "text": "Real hungry rn", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 732358924502847488, "name": "Golf Hayley", "screen_name": "HayleyGolf", "lang": "en", "location": "Houston, TX", "create_at": date("2016-05-16"), "description": "GOLF GOLF GOLF", "followers_count": 19, "friends_count": 30, "statues_count": 23 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800596442292225, "text": "#NationalBestFriendDay afhhhhh ✨✨✨������ https://t.co/zvL6qX9DuH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "NationalBestFriendDay" }}, "user": { "id": 2739231782, "name": "di", "screen_name": "dihoaang", "lang": "en", "location": "null", "create_at": date("2014-08-17"), "description": "null", "followers_count": 146, "friends_count": 133, "statues_count": 999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800596719079424, "text": "willy con su gran sombrero", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 104845652, "name": "g e ø", "screen_name": "geo_metrical", "lang": "en", "location": "ORU.", "create_at": date("2010-01-14"), "description": "INTP", "followers_count": 198, "friends_count": 257, "statues_count": 5505 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edinburg, TX", "id": "013fbf673c82e5e4", "name": "Edinburg", "place_type": "city", "bounding_box": rectangle("-98.225807,26.244127 -98.079031,26.450924") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4822660, "cityName": "Edinburg" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800597092380672, "text": "Staaaaapppp ittttt bihhhh�������� https://t.co/w5dg32xaoh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1739322590, "name": "Cal", "screen_name": "_hotboy293", "lang": "en", "location": "Forest Lane, Dallas", "create_at": date("2013-09-06"), "description": "I don't charge by the inch, I charge by the foot", "followers_count": 805, "friends_count": 610, "statues_count": 20821 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, TX", "id": "6e315e1f96e0450a", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-97.233811,32.586565 -97.037464,32.817135") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4804000, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800597385969664, "text": "i am a uc berkeley student but i am not a golden bear, k bye", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4498846342, "name": "ㅤㅤㅤ", "screen_name": "juanislit", "lang": "en", "location": "lost angels / yay area", "create_at": date("2015-12-15"), "description": "burning this [white] world to the ground. he/him.", "followers_count": 134, "friends_count": 135, "statues_count": 1631 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800597423759360, "text": "she rode it like a soldier", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2637934789, "name": "zoë nicolette", "screen_name": "kaliente_16xoxo", "lang": "en", "location": "HOU", "create_at": date("2014-07-13"), "description": "single carefree black girl ✨", "followers_count": 1085, "friends_count": 699, "statues_count": 2035 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sugar Land, TX", "id": "7a41192a2879ee24", "name": "Sugar Land", "place_type": "city", "bounding_box": rectangle("-95.686106,29.543372 -95.577273,29.663556") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4870808, "cityName": "Sugar Land" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800598312898561, "text": "Ya boy finally 17������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3237995343, "name": "⁶ June 9", "screen_name": "jay_kosher", "lang": "en", "location": "2021 NFL Draft", "create_at": date("2015-05-05"), "description": "Riverside Poly 17' #EndTheStruggle #StudentAthelete #Undecided", "followers_count": 518, "friends_count": 440, "statues_count": 1811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800598522613760, "text": "Cold as my soul/heart/personality", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622096950, "name": "yung shrug", "screen_name": "BeTreezy", "lang": "en", "location": "814, PA", "create_at": date("2012-06-29"), "description": "I didn't choose the shrug life, the shrug life chose me. ♏", "followers_count": 285, "friends_count": 445, "statues_count": 9718 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, PA", "id": "a381d3c6b677f913", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-79.864261,41.883309 -79.828842,41.907148") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42049, "countyName": "Erie", "cityID": 4278448, "cityName": "Union City" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800598929514497, "text": "@ebbtideapp Tide in Port Royal, Virginia 06/09/2016\n Low 3:45am 0.2\nHigh 9:45am 2.3\n Low 4:25pm 0.1\nHigh 10:27pm 2.1", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-77.19,38.1733"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 101, "friends_count": 1, "statues_count": 39858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51033, "countyName": "Caroline" } }
+{ "create_at": datetime("2016-06-09T00:00:00.000Z"), "id": 740800599269224448, "text": "You can actually see a lot of stars tonight it's pretty beautiful", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1613919745, "name": "Bradyn not Brayden", "screen_name": "BradynPalitti", "lang": "en", "location": "your moms chest hair", "create_at": date("2013-07-22"), "description": "there's a 30% chance that it's already raining// snap: Bradynpalitti", "followers_count": 516, "friends_count": 266, "statues_count": 23703 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800599608963072, "text": "@ericarxnee I LOVE YOU MORE", "in_reply_to_status": 740783695649492992, "in_reply_to_user": 1671566388, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1671566388 }}, "user": { "id": 477216443, "name": "Sharpay Evans", "screen_name": "RilynCeleste", "lang": "en", "location": "✨KilynForever✨", "create_at": date("2012-01-28"), "description": "hchs senior", "followers_count": 641, "friends_count": 1050, "statues_count": 15018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mesa, CA", "id": "c8ccc9439a8e5ee0", "name": "La Mesa", "place_type": "city", "bounding_box": rectangle("-117.053546,32.743581 -116.981714,32.7958") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 640004, "cityName": "La Mesa" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800599944507393, "text": "FUCK IT UPP SIS ���� https://t.co/h2QDawHuDi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1442293471, "name": "THENAATHANGG", "screen_name": "AthenaTrejo", "lang": "en", "location": "null", "create_at": date("2013-05-19"), "description": "null", "followers_count": 192, "friends_count": 209, "statues_count": 1842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600028393472, "text": "06/09@03:00 - Temp 50.4F, WC 50.4F. Wind 0.7mph WSW, Gust 1.0mph. Bar 29.925in, Rising slowly. Rain 0.00in. Hum 87%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49542 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600057761796, "text": "Coolin' regardless ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3732535520, "name": "Winnie.", "screen_name": "sailingxsoulsx", "lang": "en", "location": "Seattle. ", "create_at": date("2015-09-29"), "description": "rest in paradise Muldhata❤️ free moe. free Palestine", "followers_count": 595, "friends_count": 397, "statues_count": 4975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600120643584, "text": "Wind 0.0 mph ---. Barometer 1016.77 mb, Steady. Temperature 70.3 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 32, "friends_count": 129, "statues_count": 14988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600242302976, "text": "Damn girl https://t.co/X7DXEFL5SM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332889449, "name": "KidWizard", "screen_name": "_calebbreaux", "lang": "en", "location": "In a galaxy far far away", "create_at": date("2014-02-07"), "description": "brother good//Sxper Nxgro//parkour//freerunning//KR3W//LVA// Wonder Bros™//future nomad", "followers_count": 690, "friends_count": 671, "statues_count": 18963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerlin South, NV", "id": "0134e6167ff7f6ec", "name": "Summerlin South", "place_type": "city", "bounding_box": rectangle("-115.355825,36.082837 -115.314862,36.159081") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3270900, "cityName": "Summerlin South" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600502341632, "text": "Congratulations Battleground High 2016! #gradnight2016 #bghs_utc… https://t.co/VvltwywAJq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.6644,45.50549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "gradnight2016", "bghs_utc" }}, "user": { "id": 2995400100, "name": "Prestige Sounds NW", "screen_name": "PrestigeSndsNW", "lang": "en", "location": "Vancouver, WA", "create_at": date("2015-01-24"), "description": "DJ & Emcee Event Entertainment", "followers_count": 48, "friends_count": 172, "statues_count": 71 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600888201217, "text": "Wind 0.0 mph ---. Barometer 30.01 in, Steady. Temperature 68.4 °F. Rain today 0.00 in. Humidity 73%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800600909176832, "text": "Can you recommend anyone for this #Nursing #job? https://t.co/Z3ivZArgoe #SanDiego, CA #Hiring https://t.co/IL4Elny6hN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.1610838,32.715738"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Nursing", "job", "SanDiego", "Hiring" }}, "user": { "id": 2327706649, "name": "Sunrise Careers", "screen_name": "Sunrise_Careers", "lang": "en", "location": "null", "create_at": date("2014-02-04"), "description": "We are always seeking exceptional people with a passion for working with seniors, and dedicated to caring for others.", "followers_count": 403, "friends_count": 9, "statues_count": 2688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601102143488, "text": "Baby shower it makes me so mad because it's like lila only wants ppl to leave her house if she dont want u there that makes me so mad cause", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 728148543710461953, "name": "Jennifer Hernandez", "screen_name": "sciontclover16", "lang": "en", "location": "null", "create_at": date("2016-05-05"), "description": "null", "followers_count": 39, "friends_count": 227, "statues_count": 140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Telford, TN", "id": "0082c8f912df3d60", "name": "Telford", "place_type": "city", "bounding_box": rectangle("-82.572157,36.212553 -82.507854,36.267807") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47179, "countyName": "Washington", "cityID": 4773220, "cityName": "Telford" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601139908608, "text": "Happy Humpday! Thank you @officialjenzi for another fun class! It was lit!!.. but really AZ is… https://t.co/tn6TZKcDza", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.90852042,33.42161568"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 44515984 }}, "user": { "id": 38560462, "name": "Melissa Brandon", "screen_name": "MelissaLily5", "lang": "en", "location": "null", "create_at": date("2009-05-07"), "description": "surround yourself with the dreamers and the doers, the believers and the thinkers.", "followers_count": 95, "friends_count": 69, "statues_count": 2294 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601324425217, "text": "I'm always hungry ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3270702337, "name": "Raachaael ❤️", "screen_name": "babyraachaael", "lang": "en", "location": "null", "create_at": date("2015-07-06"), "description": "L's ??", "followers_count": 390, "friends_count": 295, "statues_count": 20089 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Lorenzo, CA", "id": "3656cfbea78908af", "name": "San Lorenzo", "place_type": "city", "bounding_box": rectangle("-122.161568,37.661213 -122.106552,37.685884") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 668112, "cityName": "San Lorenzo" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601630609412, "text": "I'm stubborn asf so it's funny to see how quick I change for the right person", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3246613627, "name": "Ale", "screen_name": "AleAvalosss", "lang": "en", "location": "null", "create_at": date("2015-06-15"), "description": "null", "followers_count": 476, "friends_count": 406, "statues_count": 5076 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601664192512, "text": "@dicktoobomb do i @ KELLZ or is she gonna get salts at me", "in_reply_to_status": 740799776384552964, "in_reply_to_user": 48927036, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 48927036 }}, "user": { "id": 3272876713, "name": "Andrew Tra", "screen_name": "tftiandrew", "lang": "en", "location": "honolulu drinking yoohoo", "create_at": date("2015-07-09"), "description": "Unites States Army 68W", "followers_count": 408, "friends_count": 336, "statues_count": 12143 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Villages, HI", "id": "d947a69c1bf2099b", "name": "Ewa Villages", "place_type": "city", "bounding_box": rectangle("-158.053001,21.331948 -158.026822,21.356753") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507485, "cityName": "Ewa Villages" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601664200706, "text": "We are more than what has happened to us. We are more than our yesterdays. #vscocam… https://t.co/GlOs7DDB9l", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-97.3332,32.7574"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "vscocam" }}, "user": { "id": 728480715843624960, "name": "i_am_rove", "screen_name": "i_am_rove", "lang": "en", "location": "Texas, USA", "create_at": date("2016-05-06"), "description": "Live. Evolve. Transcend. ~Rove ©", "followers_count": 10, "friends_count": 43, "statues_count": 9 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601697714176, "text": "* that claims she ain't got niggas https://t.co/5quyBiAVGh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 118106581, "name": "...", "screen_name": "flexreek_", "lang": "en", "location": "westside!", "create_at": date("2010-02-27"), "description": "@Daijhannn❣ #ul21", "followers_count": 1000, "friends_count": 554, "statues_count": 32845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westwego, LA", "id": "7806ff480d4f5836", "name": "Westwego", "place_type": "city", "bounding_box": rectangle("-90.158656,29.870939 -90.128364,29.932179") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2281165, "cityName": "Westwego" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601701912576, "text": "I would definitely watch a sequel to the Warcraft movie. Warcraft: the middle?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420513203, "name": "Olivia Grace", "screen_name": "oliviadgrace", "lang": "en", "location": "From Essex, UK. Live in SoCal", "create_at": date("2011-11-24"), "description": "Englishwoman, eSports Manager for Blizzard, running WoW eSports. Strongman in training. PvPer. That girl from Wowhead/SkillCapped/WoW Insider/Engadget.", "followers_count": 28006, "friends_count": 1077, "statues_count": 36067 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800601882255360, "text": "I don't think anyone knows how bad I want to be a marine already, id go tomorrow if I could. Oohrah����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2165796464, "name": "O'brian Uzamaki", "screen_name": "FloresObrian", "lang": "en", "location": "trapped in my mind", "create_at": date("2013-10-30"), "description": "I like the simple things in life, and I'm also the guy that will go to a party just to make sure everyone's ok lol", "followers_count": 759, "friends_count": 664, "statues_count": 21641 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camarillo, CA", "id": "689083f5b4e488b4", "name": "Camarillo", "place_type": "city", "bounding_box": rectangle("-119.109824,34.191355 -118.958874,34.2593") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6111, "countyName": "Ventura", "cityID": 610046, "cityName": "Camarillo" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602041688064, "text": "@rcynacarr oh lawd so tru tho like everyday I was ded after those stairs", "in_reply_to_status": 740800357111083009, "in_reply_to_user": 3214122007, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3214122007 }}, "user": { "id": 2292749662, "name": "your b", "screen_name": "izmegaboo", "lang": "en", "location": "disneyland", "create_at": date("2014-01-19"), "description": "d28w • vp • 3 • yous a lame boy get up out my face", "followers_count": 508, "friends_count": 420, "statues_count": 23409 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602217848832, "text": "Cheat on me and this is what will happen https://t.co/IlQmI8HMeI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2783784286, "name": "JAZZZZ-MEAN ❤️", "screen_name": "KhiLove__", "lang": "en", "location": "North Charleston, SC", "create_at": date("2014-09-25"), "description": "12.08.13 // Malachi&Muffin Keeper // SummerSixteen", "followers_count": 733, "friends_count": 727, "statues_count": 6008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Charleston, SC", "id": "18df13e4a5a670b5", "name": "North Charleston", "place_type": "city", "bounding_box": rectangle("-80.15759,32.829336 -79.934288,32.999513") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45019, "countyName": "Charleston", "cityID": 4550875, "cityName": "North Charleston" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602230390788, "text": "Got Outta Character Tonight For A Bum ���� '", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2818736263, "name": "Bo$$$ '", "screen_name": "Sheniquaaaa", "lang": "en", "location": "null", "create_at": date("2014-09-18"), "description": "I Been 504 , NewOrleans ⚜ '", "followers_count": 559, "friends_count": 416, "statues_count": 10088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602553393152, "text": "insecure or nah", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3472587434, "name": "glazed donut", "screen_name": "caitlynnbriana", "lang": "en", "location": "happy w/ hasan", "create_at": date("2015-09-06"), "description": "null", "followers_count": 304, "friends_count": 179, "statues_count": 257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Campo, TX", "id": "0626de14ecbdb216", "name": "El Campo", "place_type": "city", "bounding_box": rectangle("-96.309945,29.168615 -96.244234,29.233694") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48481, "countyName": "Wharton", "cityID": 4822864, "cityName": "El Campo" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602586927104, "text": "@LORDEchai Arab", "in_reply_to_status": 740800552225886208, "in_reply_to_user": 304769669, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user_mentions": {{ 304769669 }}, "user": { "id": 201091897, "name": "TheProShow", "screen_name": "NicoThePro", "lang": "en", "location": " W$GTM ", "create_at": date("2010-10-10"), "description": "In 2014 i was half a virgin. In 2015 i decided to be a full virgin. In 2016 I finally turned up on the dih. Ghetto Ambassador #TrollGoddess", "followers_count": 1307, "friends_count": 334, "statues_count": 146909 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602658197505, "text": "@SugarShawne I didn't :(", "in_reply_to_status": 740800455962427392, "in_reply_to_user": 612352568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 612352568 }}, "user": { "id": 1115235085, "name": "nicoLIT", "screen_name": "nicoleedwards98", "lang": "en", "location": "Florida, USA", "create_at": date("2013-01-23"), "description": "when you been finessin a nigga and he hit u wit the I wanna make it official (peep the header)", "followers_count": 703, "friends_count": 323, "statues_count": 10663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whiskey Creek, FL", "id": "8d8703b6fd478d81", "name": "Whiskey Creek", "place_type": "city", "bounding_box": rectangle("-81.897781,26.556692 -81.883056,26.593362") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12071, "countyName": "Lee", "cityID": 1277137, "cityName": "Whiskey Creek" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602662424576, "text": "Lol @ girls who can't do their eyeliner without putting tape on their faces", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1055784948, "name": "B", "screen_name": "BAForst", "lang": "en", "location": "Boise, ID", "create_at": date("2013-01-02"), "description": "Proud mother of a German Shepherd puppy named Maxx and I'm engaged to my best friend @Al_Sal1410 I also really love soccer ❤️⚽️", "followers_count": 813, "friends_count": 957, "statues_count": 36194 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walla Walla, WA", "id": "3f449e4a20757d86", "name": "Walla Walla", "place_type": "city", "bounding_box": rectangle("-118.379086,46.022857 -118.263051,46.106424") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53071, "countyName": "Walla Walla", "cityID": 5375775, "cityName": "Walla Walla" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602775642113, "text": "https://t.co/uu5esdmwF6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 725887062336016384, "name": "Viviane Keita", "screen_name": "VK12sarts", "lang": "en", "location": "Maspeth, Queens", "create_at": date("2016-04-28"), "description": "null", "followers_count": 128, "friends_count": 254, "statues_count": 1335 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602784030720, "text": "Wish I had cute pics to post for #NationalBestFriendsDay but all the selfies I take with my ride or die hoes are terrible", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "NationalBestFriendsDay" }}, "user": { "id": 788275128, "name": "°₊·ˈ∗(ૢෆ癶◡癶ෆ) ૢ∗ˈ‧₊°", "screen_name": "kiahnasus", "lang": "en", "location": "null", "create_at": date("2012-08-28"), "description": "c. jose", "followers_count": 439, "friends_count": 199, "statues_count": 26934 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602872111105, "text": "@pharaohlaflame Bruh when people listen to drake, they aren't trying to think. They just listen because it's most likely a hit.", "in_reply_to_status": 740800029796007936, "in_reply_to_user": 2237462787, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2237462787 }}, "user": { "id": 1270104390, "name": "The illest", "screen_name": "euxiee", "lang": "en", "location": "P.G. ", "create_at": date("2013-03-15"), "description": "My only goal in life is to be happy. Genuinely, Intensely, and Consistently happy. MSMU '19", "followers_count": 568, "friends_count": 495, "statues_count": 12607 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kettering, MD", "id": "7e11eddbcb11060d", "name": "Kettering", "place_type": "city", "bounding_box": rectangle("-76.831975,38.873828 -76.739939,38.901951") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2443900, "cityName": "Kettering" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602905645056, "text": "Temp: 67.6°F Wind:0.0mph Pressure: 29.956hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800602918289409, "text": "������ https://t.co/p8oHRcYqKm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 150330403, "name": "Leonard Fairley", "screen_name": "LenGotGame", "lang": "en", "location": "Largo,MD to Norfolk,VA", "create_at": date("2010-05-31"), "description": "A man of many nouns, verbs, adjectives, & adverbs. IG:LenGotGame_ #LongLiveLaffy", "followers_count": 2101, "friends_count": 2126, "statues_count": 173029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603077660672, "text": "I wonder when my mom is gonna remember what Hills is", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317486094, "name": "jennel", "screen_name": "ayojennel", "lang": "en", "location": "null", "create_at": date("2011-06-14"), "description": "20. utrgv. sc: ayojennel", "followers_count": 410, "friends_count": 248, "statues_count": 39223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mission, TX", "id": "77633125ba089dcb", "name": "Mission", "place_type": "city", "bounding_box": rectangle("-98.363219,26.155046 -98.272146,26.262558") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4848768, "cityName": "Mission" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603127975940, "text": "Wind 3.0 mph WNW. Barometer 29.660 in, Steady. Temperature 44.4 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11305 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603190927360, "text": "The new guy at work just dusted his cigarette ashes off on the knuckles and then snorted them. I have no words", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1152256861, "name": "electric wilzard", "screen_name": "buriedintrash", "lang": "en", "location": "new london", "create_at": date("2013-02-05"), "description": "26, stockboy, vinyl and katies main monkey.", "followers_count": 195, "friends_count": 341, "statues_count": 5503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterford, CT", "id": "01a04a3016dfccf7", "name": "Waterford", "place_type": "city", "bounding_box": rectangle("-72.1932,41.298815 -72.095363,41.427013") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9011, "countyName": "New London", "cityID": 980210, "cityName": "Waterford" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603333505024, "text": "Poné cara de boludo que el Golden Gate sale lindo igual.\n#SanFrancisco #goldengatebridge #bike @… https://t.co/i75dnAGD68", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.47734466,37.81077914"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "SanFrancisco", "goldengatebridge", "bike" }}, "user": { "id": 22873352, "name": "Gastón Bourdieu", "screen_name": "GastonBourdieu", "lang": "en", "location": "Buenos Aires City Region", "create_at": date("2009-03-04"), "description": "Viajes, periodismo, fotografía, publicidad. Algo de todo lo que se hace con una cámara. \nPor siempre @expresoaoriente", "followers_count": 799, "friends_count": 480, "statues_count": 9170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603471925249, "text": "@jotchuaa IM DEAD", "in_reply_to_status": 740799247029809152, "in_reply_to_user": 834786660, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 834786660 }}, "user": { "id": 336775277, "name": "Maria Evans △⃒⃘", "screen_name": "maribeth_evans", "lang": "en", "location": "Forest Falls, CA", "create_at": date("2011-07-16"), "description": "•heroes always get remembered but you know legends never die •", "followers_count": 481, "friends_count": 503, "statues_count": 8148 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redlands, CA", "id": "c904ca419d4e53c6", "name": "Redlands", "place_type": "city", "bounding_box": rectangle("-117.243736,34.003849 -117.103406,34.101898") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659962, "cityName": "Redlands" } }
+{ "create_at": datetime("2016-06-09T00:00:01.000Z"), "id": 740800603518046209, "text": "Lord please save her for me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2938131988, "name": "Danny", "screen_name": "ddaniel_77", "lang": "en", "location": "Selma, CA", "create_at": date("2014-12-23"), "description": "Sc: Daniel.5677", "followers_count": 467, "friends_count": 407, "statues_count": 11912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Selma, CA", "id": "9f8bd34c144e52ee", "name": "Selma", "place_type": "city", "bounding_box": rectangle("-119.64712,36.553354 -119.58837,36.605473") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 670882, "cityName": "Selma" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800603773886464, "text": "https://t.co/hSx2YikJ7Z", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1225784341, "name": "Zack", "screen_name": "ThatKidZack", "lang": "en", "location": "Plano", "create_at": date("2013-02-27"), "description": "Professional Sports Analyst/Fantasy Football Expert/Colts Journalist/ NFL Scout", "followers_count": 183, "friends_count": 102, "statues_count": 5254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Travis Ranch, TX", "id": "0081a5ba6572e762", "name": "Travis Ranch", "place_type": "city", "bounding_box": rectangle("-96.486355,32.791913 -96.456803,32.812339") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48257, "countyName": "Kaufman", "cityID": 4873562, "cityName": "Travis Ranch" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800604021370880, "text": "Wind 0.0 mph ---. Barometer 29.906 in, Steady. Temperature 68.9 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 69, "friends_count": 24, "statues_count": 66541 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800604143034369, "text": "Wind 1.0 mph ESE. Barometer 29.965 in, Steady. Temperature 70.9 °F. Rain today 0.00 in. Humidity 61%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113830 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800604205907968, "text": "Neigh ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1551758413, "name": "Carlos Santiagó", "screen_name": "c_santiago97", "lang": "en", "location": "Poncitlán, Jalisco", "create_at": date("2013-06-27"), "description": "My name's Carlos and I suck at texting and I like to leave people on read.", "followers_count": 1948, "friends_count": 631, "statues_count": 13617 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westmont, IL", "id": "d73cbce54317ffaa", "name": "Westmont", "place_type": "city", "bounding_box": rectangle("-87.99425,41.763789 -87.945389,41.825016") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1780645, "cityName": "Westmont" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800604600209408, "text": "@_cotz must be nice", "in_reply_to_status": 740797396645478400, "in_reply_to_user": 917995393, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 917995393 }}, "user": { "id": 421398049, "name": "baby tarzan", "screen_name": "Goatsaac", "lang": "en", "location": "817 ➡️ 979", "create_at": date("2011-11-25"), "description": "Texas A&M. No Im not Jordan, but I am the goat. BBT. l$.", "followers_count": 1328, "friends_count": 887, "statues_count": 54708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800604856066048, "text": "Wind 0.0 mph ---. Barometer 30.029 in, Rising slowly. Temperature 52.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20436 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605019643904, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 829974805, "name": "Breezy♎️", "screen_name": "Briii_Baabyy", "lang": "en", "location": "Zion, IL", "create_at": date("2012-09-17"), "description": "Senior ✨ 2k16 @RealHooper_25❤️", "followers_count": 833, "friends_count": 743, "statues_count": 28547 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Zion, IL", "id": "3f7fc7936ffabbbb", "name": "Zion", "place_type": "city", "bounding_box": rectangle("-87.885527,42.430703 -87.79887,42.493531") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17097, "countyName": "Lake", "cityID": 1784220, "cityName": "Zion" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605086752768, "text": "������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1177074828, "name": "טורי מנצח", "screen_name": "TorReed", "lang": "en", "location": "☀️phx☀️(hell)", "create_at": date("2013-02-13"), "description": "No more depression. ❤☝אליהו sc:toriiswinning ☁wagwan?☁ #AllBlackLivesMatter #FIU☄", "followers_count": 340, "friends_count": 88, "statues_count": 25591 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605355134978, "text": "My boyfriend really everything I swear ❤️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 205888442, "name": "thw", "screen_name": "thww_", "lang": "en", "location": "null", "create_at": date("2010-10-21"), "description": "OES ❤️", "followers_count": 718, "friends_count": 479, "statues_count": 27174 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605501952000, "text": "starting that 70s show from the first episode \nto say im happy would be a majorrrr understatement", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1721290063, "name": "MØMØ", "screen_name": "mo_wenner5874", "lang": "en", "location": "WA", "create_at": date("2013-09-01"), "description": "GREEK & GERMAN * BARISTA * LIVIN THA GOOOOOD LIFEEEEE", "followers_count": 291, "friends_count": 214, "statues_count": 7326 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, WA", "id": "83615874f73de6c7", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-122.546166,48.689345 -122.314388,48.81736") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53073, "countyName": "Whatcom", "cityID": 5305280, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605539684352, "text": "Robots and self-driving trucks could be the future of mail https://t.co/2IJjlcVYB5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17640, "friends_count": 17805, "statues_count": 72832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605627809792, "text": "HAPPY BIRTHDAY HOE I LOVE YOUAND HOPE YOUR DAY IS AS GOOD AS YOU ARE ���������� @Gloria__Sayegh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1022313170 }}, "user": { "id": 445814532, "name": "Barbie", "screen_name": "karlyekline", "lang": "en", "location": "null", "create_at": date("2011-12-24"), "description": "(:", "followers_count": 1033, "friends_count": 185, "statues_count": 1670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendora, CA", "id": "eb1bb64775708bc1", "name": "Glendora", "place_type": "city", "bounding_box": rectangle("-117.890263,34.10549 -117.809111,34.165551") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 630014, "cityName": "Glendora" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800605967536129, "text": "Twitter always has me dying ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 163970357, "name": "Andrew Gallegos", "screen_name": "A_Gallegos4", "lang": "en", "location": "La Habra, CA", "create_at": date("2010-07-07"), "description": "Light up the Darkness", "followers_count": 502, "friends_count": 413, "statues_count": 22913 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Habra, CA", "id": "7a615ec218d177f6", "name": "La Habra", "place_type": "city", "bounding_box": rectangle("-117.976849,33.903616 -117.918925,33.946147") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639290, "cityName": "La Habra" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800606412103680, "text": "@swervinn_ so you're Mexican?", "in_reply_to_status": 740800435896918017, "in_reply_to_user": 1695747672, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1695747672 }}, "user": { "id": 1164161593, "name": "Lexi Danielle", "screen_name": "lexiidaanielle", "lang": "en", "location": "inyo", "create_at": date("2013-02-09"), "description": "null", "followers_count": 642, "friends_count": 361, "statues_count": 8712 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McFarland, CA", "id": "7ba4d06ff652fffc", "name": "McFarland", "place_type": "city", "bounding_box": rectangle("-119.240992,35.659929 -119.204633,35.68918") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 644826, "cityName": "McFarland" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800606428921856, "text": "Jones BBQ and Foot Massage sounds so unsanitary to me...but they do have a nice jingle", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3248736480, "name": "Peter O'Neill", "screen_name": "PeteONeill12", "lang": "en", "location": "Berkeley, CA", "create_at": date("2015-06-18"), "description": "UC Davis 2013-2015; UCLA '17", "followers_count": 37, "friends_count": 62, "statues_count": 523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800606852554752, "text": "76.3F (Feels: 83.5F) - Humidity: 92% - Wind: 0.0mph ESE - Gust: 0.0mph - Pressure: 29.924in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 241727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607125135360, "text": "OK https://t.co/HQ09XCxnGL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2532821202, "name": "jazmin ᵔᴥᵔ", "screen_name": "homeisreaI", "lang": "en", "location": "ashlee francesca jason", "create_at": date("2014-05-29"), "description": "@aimhmatty: jazmins @ just made me nut", "followers_count": 3879, "friends_count": 99, "statues_count": 153435 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harvey, LA", "id": "337d304222d75060", "name": "Harvey", "place_type": "city", "bounding_box": rectangle("-90.092431,29.836718 -90.035095,29.916898") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2233245, "cityName": "Harvey" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607192223745, "text": "But I'm really content without you...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1298051101, "name": "✨ Lina ✨", "screen_name": "evalinarose", "lang": "en", "location": "strawberry manor, CA", "create_at": date("2013-03-25"), "description": "Instagram & Snapchat @evalinarose", "followers_count": 301, "friends_count": 253, "statues_count": 25085 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607238377474, "text": "@paulbenedict7 Will all the fraud every state should require a photo ID to many illegals are voting some people vote 2 x even more!", "in_reply_to_status": 740787227366690816, "in_reply_to_user": 46303458, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 46303458 }}, "user": { "id": 57058641, "name": "Mimi", "screen_name": "mimi_saulino", "lang": "en", "location": "null", "create_at": date("2009-07-15"), "description": "Interior Designer, Entrepreneur, lover of life, laughter, cooking, eternal optimist. Proud American, hold the Shariah I believe in Jesus. #AlwaysTrumpforAmerica", "followers_count": 6766, "friends_count": 2943, "statues_count": 56399 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Collingswood, NJ", "id": "e0feac53a5d4f3cd", "name": "Collingswood", "place_type": "city", "bounding_box": rectangle("-75.093492,39.904548 -75.056285,39.927106") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3414260, "cityName": "Collingswood" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607246815233, "text": "#nationalbestfriendday https://t.co/feYaj1zX95", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "nationalbestfriendday" }}, "user": { "id": 14404110, "name": "Tyler P.", "screen_name": "Its_Tylurrr", "lang": "en", "location": "Hayward, CA", "create_at": date("2008-04-15"), "description": "TIME Person of the Year 2006 (seriously google it) MCHS '18", "followers_count": 234, "friends_count": 255, "statues_count": 943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607456481281, "text": "Is there a trial period?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 20319158, "name": "jimbolaya", "screen_name": "huntersjames", "lang": "en", "location": "Guadalupe, AZ", "create_at": date("2009-02-07"), "description": "iso tilting windmills whimsy grins rage peace jamz green hiking driving peace and peaches #hsjhaiku ::seldomseensmith:: ::Montaigne::", "followers_count": 1223, "friends_count": 2468, "statues_count": 23922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hermosa Beach, CA", "id": "4baf4d09759d33e4", "name": "Hermosa Beach", "place_type": "city", "bounding_box": rectangle("-118.409579,33.85185 -118.384866,33.877823") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 633364, "cityName": "Hermosa Beach" } }
+{ "create_at": datetime("2016-06-09T00:00:02.000Z"), "id": 740800607586529280, "text": "\"Selenna, o bien te duermes o no andes chillando\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 1619534161, "name": "Selenna.", "screen_name": "ChinaaaLoveee", "lang": "en", "location": "null", "create_at": date("2013-07-24"), "description": "Gangstas don't dance we Boogie.", "followers_count": 437, "friends_count": 410, "statues_count": 24249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608194723845, "text": "Hey, tie your shoes! I don’t want you falling for anyone else ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 549569614, "name": "deandrea ✨", "screen_name": "allaboutdede", "lang": "en", "location": "Evansville, Indiana", "create_at": date("2012-04-09"), "description": "Cause there’s a light in me that shines brightly. They can try, but they can’t take that away from me. ✨", "followers_count": 1064, "friends_count": 2018, "statues_count": 13073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Evansville, IN", "id": "ad815dfe8c34a012", "name": "Evansville", "place_type": "city", "bounding_box": rectangle("-87.699649,37.923982 -87.449732,38.082951") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18163, "countyName": "Vanderburgh", "cityID": 1822000, "cityName": "Evansville" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608265986048, "text": "Just sayin' ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418252885, "name": "princess diana ♛", "screen_name": "dianacacace", "lang": "en", "location": "Stockton University '18", "create_at": date("2011-11-21"), "description": "☾carpe diem. | DCP Alumni | NJ | 2.20.15 ❁♡", "followers_count": 706, "friends_count": 703, "statues_count": 20604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Freehold, NJ", "id": "01bb5d775f6ebee3", "name": "West Freehold", "place_type": "city", "bounding_box": rectangle("-74.332704,40.18095 -74.24345,40.252228") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34025, "countyName": "Monmouth", "cityID": 3479100, "cityName": "West Freehold" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608475697152, "text": "@luciano_dali lmaooooo literally", "in_reply_to_status": 740800448861491200, "in_reply_to_user": 707241484819431426, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 707241484819431426 }}, "user": { "id": 355946449, "name": "amari", "screen_name": "HenryBased", "lang": "en", "location": "DTX", "create_at": date("2011-08-15"), "description": "#TXST", "followers_count": 1582, "friends_count": 1380, "statues_count": 39823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesquite, TX", "id": "07ff830817b8e718", "name": "Mesquite", "place_type": "city", "bounding_box": rectangle("-96.683671,32.700664 -96.520614,32.849211") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48427, "countyName": "Starr", "cityID": 4847898, "cityName": "Mesquite" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608576405505, "text": "DONT BE BUTT HURT BIII", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2152218252, "name": "ѕαναииαн ♡", "screen_name": "saavvybuu_", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2013-10-23"), "description": "she an angel but damn that angel bad.", "followers_count": 660, "friends_count": 719, "statues_count": 18632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ewa Beach, HI", "id": "6bd73386ffaba450", "name": "Ewa Beach", "place_type": "city", "bounding_box": rectangle("-158.028613,21.306027 -157.990042,21.332114") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1507450, "cityName": "Ewa Beach" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608710602752, "text": "some what craving Pancakes .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4154932874, "name": "Prince Bacani", "screen_name": "PRRRINCEEEY", "lang": "en", "location": "Waipahu, HI", "create_at": date("2015-11-06"), "description": "It is what it is .", "followers_count": 159, "friends_count": 137, "statues_count": 4440 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800608983257088, "text": "������ 21 ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 462322705, "name": "Roscoe", "screen_name": "roscoe_peeko", "lang": "en", "location": "Roseville, CA", "create_at": date("2012-01-12"), "description": "don't do drugs lol", "followers_count": 155, "friends_count": 52, "statues_count": 13165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Roseville, CA", "id": "38109314e8d0ee2f", "name": "Roseville", "place_type": "city", "bounding_box": rectangle("-121.385009,38.716061 -121.216452,38.81026") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6061, "countyName": "Placer", "cityID": 662938, "cityName": "Roseville" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609067110402, "text": "lil shorty can get cuffed��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613205715, "name": "glee$h", "screen_name": "AyeItsGleasonnn", "lang": "en", "location": "700", "create_at": date("2012-06-19"), "description": "born '98 // rest eazy mom & dad", "followers_count": 949, "friends_count": 472, "statues_count": 11671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterloo, IA", "id": "4fcb8e32c69ad4ee", "name": "Waterloo", "place_type": "city", "bounding_box": rectangle("-92.437083,42.421196 -92.27324,42.570459") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19013, "countyName": "Black Hawk", "cityID": 1982425, "cityName": "Waterloo" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609209704448, "text": "Fr���� https://t.co/z5EEW7YxRU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2610168014, "name": "Paul(a)⚽️", "screen_name": "crazypaulita1", "lang": "en", "location": "at your house;)", "create_at": date("2014-07-07"), "description": "@phil_coutinho & soccer⚽️❤️", "followers_count": 635, "friends_count": 680, "statues_count": 7930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oak Ridge North, TX", "id": "0180324e104dd7b5", "name": "Oak Ridge North", "place_type": "city", "bounding_box": rectangle("-95.452477,30.076095 -95.349469,30.173617") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48339, "countyName": "Montgomery", "cityID": 4853190, "cityName": "Oak Ridge North" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609352339457, "text": "I yell for everything ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 627262989, "name": "Alysha Rodriguez", "screen_name": "Mrsknowles_", "lang": "en", "location": "null", "create_at": date("2012-07-05"), "description": "Living life, feeling better✨", "followers_count": 670, "friends_count": 309, "statues_count": 28249 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609624981505, "text": "@DoeepyMoses https://t.co/tggkoMqmio", "in_reply_to_status": -1, "in_reply_to_user": 2524168628, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2524168628 }}, "user": { "id": 701949148757557248, "name": "Jose Campos", "screen_name": "jumpyjose", "lang": "en", "location": "Sleeping/Eating", "create_at": date("2016-02-22"), "description": "June 2 ,2017 wya \n\nSenior @ CHS \n\nIG : @jumpy.jose\n\nSnapchat: ohlorditsjose", "followers_count": 192, "friends_count": 134, "statues_count": 5315 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609822101504, "text": "BITCH GET THE FUCK OUT MY MOTHA FUCKIN TRAP HOUSE", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 382146445, "name": "professor finesse", "screen_name": "nnajensen", "lang": "en", "location": "fairbanks, alaska ", "create_at": date("2011-09-29"), "description": "@megos54 is the kim to my kanye minus the following/follower ratio part // just looking for my @jacobsartorius ❤️ snapchat: annaloljensen", "followers_count": 2799, "friends_count": 1397, "statues_count": 129003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800609897615361, "text": "shit hurts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2508417499, "name": "K.", "screen_name": "_karlalaraaaa", "lang": "en", "location": "null", "create_at": date("2014-05-19"), "description": "queen ♍️", "followers_count": 350, "friends_count": 98, "statues_count": 5009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central Point, OR", "id": "ff96f35b5ab7ad8c", "name": "Central Point", "place_type": "city", "bounding_box": rectangle("-122.93452,42.361221 -122.883112,42.39622") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4112400, "cityName": "Central Point" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610002436096, "text": "I'll be free from spring quarter in 17-18 hours ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2986794473, "name": "viabae", "screen_name": "notvianiniar", "lang": "en", "location": "San Diego/Tijuana", "create_at": date("2015-01-19"), "description": "UCSC '19 • I love coffee, math, engineering & good vibes • Proud Chicana • Eric is my BFF", "followers_count": 242, "friends_count": 125, "statues_count": 11444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Cruz, CA", "id": "3c9e627dd6b55d9e", "name": "Santa Cruz", "place_type": "city", "bounding_box": rectangle("-122.076144,36.948098 -121.986229,37.010652") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6087, "countyName": "Santa Cruz", "cityID": 669112, "cityName": "Santa Cruz" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610044383232, "text": "@tylerrjoseph @joshuadun no matter how overplayed it is, it'll always have a special meaning to me #ILoveStressedOut", "in_reply_to_status": -1, "in_reply_to_user": 221412285, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ILoveStressedOut" }}, "user_mentions": {{ 221412285, 16712746 }}, "user": { "id": 2566058832, "name": "️", "screen_name": "parapoppy", "lang": "en", "location": "GA", "create_at": date("2014-06-13"), "description": "that poppy and paramore ! #popsisters", "followers_count": 200, "friends_count": 121, "statues_count": 806 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610128285696, "text": "Temp 53.3° Hi/Lo 54.6/53.0 Rng 1.6° WC 53.3° Hmd 71% Rain 0.00\" Storm 0.00\" BAR 29.666 Rising DP 44.1° Wnd 2mph Dir W Gst 13mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19358 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610157629443, "text": "2017 Lit ���� https://t.co/Spl7P4TNxC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 419861328, "name": "Kiara", "screen_name": "KyThaGawd", "lang": "en", "location": "null", "create_at": date("2011-11-23"), "description": "null", "followers_count": 1428, "friends_count": 1010, "statues_count": 39136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610589630464, "text": "I basically have my whole wedding planned", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 973586779, "name": "kat.", "screen_name": "itskatherine_xo", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "¯\\_(ツ)_/¯", "followers_count": 1128, "friends_count": 570, "statues_count": 45029 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610728062976, "text": "Ummmmmmmmmmm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2209670598, "name": "Kim Kardashian West", "screen_name": "CarginaGeorge", "lang": "en", "location": "bikini bottom", "create_at": date("2013-11-22"), "description": "hi welcome to chilis", "followers_count": 739, "friends_count": 360, "statues_count": 42842 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar Rapids, IA", "id": "e06ed4324b139bf2", "name": "Cedar Rapids", "place_type": "city", "bounding_box": rectangle("-91.774579,41.886245 -91.59113,42.066811") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19113, "countyName": "Linn", "cityID": 1912000, "cityName": "Cedar Rapids" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610937778177, "text": "I have no idea what she say in the song BUT the beat is smooth !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 179794079, "name": "JORDAN RUCKER", "screen_name": "YoSoyNegro_", "lang": "en", "location": "null", "create_at": date("2010-08-17"), "description": "Snap: diddyballz", "followers_count": 411, "friends_count": 274, "statues_count": 10904 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawthorne, CA", "id": "3134f9d2892d2685", "name": "Hawthorne", "place_type": "city", "bounding_box": rectangle("-118.378926,33.893614 -118.31335,33.934522") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632548, "cityName": "Hawthorne" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800610942001152, "text": "Dude ass faking like he working trying to control some shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 69305307, "name": "Chris Handsome!", "screen_name": "chrisdorsey09", "lang": "en", "location": "null", "create_at": date("2009-08-27"), "description": "Man of Omega Psi Phi fraternity inc. Eastern Illinois. Child of God. Future Engineer. Family Man.", "followers_count": 410, "friends_count": 381, "statues_count": 7147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mattoon, IL", "id": "0022a1e61618d696", "name": "Mattoon", "place_type": "city", "bounding_box": rectangle("-88.419224,39.410479 -88.301114,39.514534") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17029, "countyName": "Coles", "cityID": 1747553, "cityName": "Mattoon" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611038470144, "text": "True friendship is being called out for not tweeting about your friend & true friendship https://t.co/1nWm2R63Wg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36248133, "name": "Saher Khan", "screen_name": "smkhan1210", "lang": "en", "location": "Chicago / New York ", "create_at": date("2009-04-28"), "description": "Journalist @ColumbiaJourn '16 | Previously: @VICE #VICEonHBO, @cairchicago, @chi_splash | http://saherk192.vsco.com |", "followers_count": 484, "friends_count": 430, "statues_count": 6199 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downers Grove, IL", "id": "6af99a29bfae42a2", "name": "Downers Grove", "place_type": "city", "bounding_box": rectangle("-88.096689,41.744098 -87.983315,41.83907") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1720591, "cityName": "Downers Grove" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611067822080, "text": "Wtf for some reason I thought it was like 2 am or something ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3303987594, "name": "Miriam B. Melecio", "screen_name": "LowTide9406", "lang": "en", "location": "Bellflower, CA", "create_at": date("2015-08-02"), "description": "kille'n with kindness ( ;SC:Berenice9406", "followers_count": 173, "friends_count": 133, "statues_count": 1302 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611092955137, "text": "have some morals Man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4751078916, "name": "Kay ❤️", "screen_name": "KissesFromK_", "lang": "en", "location": "Memphis, TN", "create_at": date("2016-01-12"), "description": "a hard head makes a soft ass", "followers_count": 313, "friends_count": 391, "statues_count": 811 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611273342977, "text": "OMG OMG ������������ https://t.co/7WmdfdiqfZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 872964986, "name": "yo girl B✌️✨", "screen_name": "BreannaHolden2", "lang": "en", "location": "In your boys dreams", "create_at": date("2012-10-10"), "description": "That blue eyed girl. A smile that will drive you wild. Classy but wild.", "followers_count": 1305, "friends_count": 990, "statues_count": 21479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Great Bend, KS", "id": "802f2723566d1036", "name": "Great Bend", "place_type": "city", "bounding_box": rectangle("-98.820728,38.350325 -98.732025,38.393428") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20009, "countyName": "Barton", "cityID": 2028300, "cityName": "Great Bend" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611394957312, "text": "Tweet of the year. I'm in love. https://t.co/AiOAEexNqy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 700379970, "name": "I can't decide", "screen_name": "RyanDanca", "lang": "en", "location": "Boynton Beach, FL", "create_at": date("2012-07-16"), "description": "Ain't you ever seen a princess be a bad bitch?", "followers_count": 186, "friends_count": 238, "statues_count": 3651 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boynton Beach, FL", "id": "005e3661711a29a9", "name": "Boynton Beach", "place_type": "city", "bounding_box": rectangle("-80.173447,26.476484 -80.05236,26.590488") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207875, "cityName": "Boynton Beach" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611533377540, "text": "I hope you know that this loyal shit is REAL no FU shit baby ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2315661270, "name": "Diondai ✌️", "screen_name": "BankRollDeeeeee", "lang": "en", "location": "null", "create_at": date("2014-01-28"), "description": "5 crazy #DK.. Handling business and doing what i do at the same time. Snapchat: hightopcity ill smoke you , Rip Auntie Liz love you TeTe", "followers_count": 548, "friends_count": 749, "statues_count": 8401 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenosha, WI", "id": "6359ef285f710052", "name": "Kenosha", "place_type": "city", "bounding_box": rectangle("-87.959452,42.538811 -87.807358,42.63972") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55059, "countyName": "Kenosha", "cityID": 5539225, "cityName": "Kenosha" } }
+{ "create_at": datetime("2016-06-09T00:00:03.000Z"), "id": 740800611764084736, "text": "@ all you dumb fuckers who cut the lines BACK UP https://t.co/WQDSueX0w1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 39117109, "name": "Liala", "screen_name": "LIA8888", "lang": "en", "location": "L.A C.A", "create_at": date("2009-05-10"), "description": "Stressed Depressed and Boy Band Obbsessed. ✨ LTF", "followers_count": 479, "friends_count": 466, "statues_count": 4215 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162983603179520, "text": "@imegyptttttt https://t.co/Ey2nztykTf", "in_reply_to_status": -1, "in_reply_to_user": 3234820750, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 3234820750 }}, "user": { "id": 3266542490, "name": "#GwallassOverModels", "screen_name": "ReggieGwallass", "lang": "en", "location": "Touro St ", "create_at": date("2015-07-02"), "description": "R.I.P BABY SIS AMOSC:Reggie_Gwallas", "followers_count": 483, "friends_count": 424, "statues_count": 1994 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kenner, LA", "id": "6e5c7d7cfdf8a6b7", "name": "Kenner", "place_type": "city", "bounding_box": rectangle("-90.285434,29.969126 -90.221258,30.049577") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2239475, "cityName": "Kenner" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162983657734144, "text": "f43d5f8d7f559bf0842748b5bcd7d46ebbcea440ecd3e1a0ca2e854333b35db31b1a6a840cd45ced987327682e7b85d6a83b4a79f06db68e25b269320182496f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-161.152931,60.126771"), "retweet_count": 0, "lang": "hu", "is_retweet": false, "user": { "id": 429803867, "name": "MarsBots", "screen_name": "MarsBots", "lang": "en", "location": "Mars", "create_at": date("2011-12-06"), "description": "null", "followers_count": 97, "friends_count": 0, "statues_count": 500464 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2050, "countyName": "Bethel" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984156844032, "text": "Why aren't there more posts that tell us that reading a book once a while isn't the nerd thing to do?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2606282918, "name": "ocean", "screen_name": "ssrocean", "lang": "en", "location": "Carson, CA (stevenson park)", "create_at": date("2014-07-05"), "description": "the ship won't sink when the water is outside of it, but it will sink when it's inside. houseofgrads tickets sold https://houseofgrads2k16.eventbrite.com/", "followers_count": 213, "friends_count": 238, "statues_count": 3429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984240779264, "text": "Fancy playing dude she childish ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 618222512, "name": "dreka.", "screen_name": "_korieangel", "lang": "en", "location": "null", "create_at": date("2012-06-25"), "description": "11:14⚓️ | 17 | worry about yourself.", "followers_count": 5331, "friends_count": 1250, "statues_count": 52472 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984530141184, "text": "@papimanny97 @umpris.... you can figure it out", "in_reply_to_status": 741162839885385729, "in_reply_to_user": 4886825043, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4886825043 }}, "user": { "id": 4230241154, "name": "Adrian", "screen_name": "Adrian_Btx", "lang": "en", "location": "Baytown, TX", "create_at": date("2015-11-13"), "description": "|GCM||CMT||Taken Tx.|", "followers_count": 281, "friends_count": 230, "statues_count": 3246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baytown, TX", "id": "7543692056063d2f", "name": "Baytown", "place_type": "city", "bounding_box": rectangle("-95.078859,29.704685 -94.874344,29.831209") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4806128, "cityName": "Baytown" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984546918400, "text": "Got it in mexicó �� https://t.co/GSIO70xKZ1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3810083232, "name": "Shyla Ryder", "screen_name": "ryder_shyla", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2015-10-06"), "description": "18+ONLY!! Adult film actress, bookings or other inquires contact my agent @thevipconnect", "followers_count": 10808, "friends_count": 87, "statues_count": 835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984635011076, "text": "@HiImVenu sure hell he is", "in_reply_to_status": 741162750618046464, "in_reply_to_user": 1930957836, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1930957836 }}, "user": { "id": 722301009859448832, "name": "✨✨", "screen_name": "bbyvivs", "lang": "en", "location": "null", "create_at": date("2016-04-18"), "description": "null", "followers_count": 91, "friends_count": 76, "statues_count": 999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162984962134018, "text": "I Need to TAKE my LIFE to BRAZIL ‼️‼️‼️ https://t.co/JjBJSFmYDO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1947860941, "name": "Camille R. Q. Miller", "screen_name": "LaPizzazz", "lang": "en", "location": "on INSTAGRAM @HiddenInfluence", "create_at": date("2013-10-08"), "description": "Loving and Living as a Royal Gypsy. New York * Philadelphia * Los Angeles and NOW * back in Atlanta Camille R. Quinones Miller - Films & Video Productions", "followers_count": 290, "friends_count": 308, "statues_count": 2802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162985289322496, "text": "El domingo empieza el juego de seducción, \nMonday night una velada entre tú y yo.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 3283836348, "name": "Em's✨", "screen_name": "Emgbenitez", "lang": "en", "location": "Re-Up ", "create_at": date("2015-07-18"), "description": "FlyHighBlacckRu", "followers_count": 339, "friends_count": 105, "statues_count": 5208 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162985310322688, "text": "Wind 2.0 mph S. Barometer 1016.66 mb, Rising. Temperature 71.4 °F. Rain today 0.00 in. Humidity 85%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-94.61083333,36.48111111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 28781048, "name": "Erin Frye", "screen_name": "EJFrye", "lang": "en", "location": "NW Arkansas", "create_at": date("2009-04-04"), "description": "http://youngliving.org/ejfrye45", "followers_count": 32, "friends_count": 129, "statues_count": 15012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arkansas, USA", "id": "e8ad2641c1cb666c", "name": "Arkansas", "place_type": "admin", "bounding_box": rectangle("-94.61771,33.004106 -89.644838,36.499767") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5007, "countyName": "Benton" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162985360592896, "text": "���� https://t.co/hJBELV3nUs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2340846889, "name": "Haleigh Brooks", "screen_name": "Haleighh_02", "lang": "en", "location": "null", "create_at": date("2014-02-12"), "description": "null", "followers_count": 422, "friends_count": 311, "statues_count": 2283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tullahoma, TN", "id": "6d10b956d2e0f446", "name": "Tullahoma", "place_type": "city", "bounding_box": rectangle("-86.274689,35.331544 -86.16948,35.415852") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47031, "countyName": "Coffee", "cityID": 4775320, "cityName": "Tullahoma" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162985536774144, "text": "tell me why I thought this was Jason https://t.co/BHBin19EoM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1599294835, "name": "meme boy", "screen_name": "SPOOKYKEVlN", "lang": "en", "location": "#allblacklivesmatter", "create_at": date("2013-07-16"), "description": "just @ me or shut the fuck up (he/him) God ain't white.", "followers_count": 925, "friends_count": 404, "statues_count": 37217 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162986304380928, "text": "Drunk @drayamichele is the best. ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 56854084 }}, "user": { "id": 2345663820, "name": "lee.", "screen_name": "ayeechantal", "lang": "en", "location": "└A, CA", "create_at": date("2014-02-15"), "description": "southernuniversity.", "followers_count": 524, "friends_count": 517, "statues_count": 8802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162986388262913, "text": "@Jgutierrezz17 oh well then :(", "in_reply_to_status": 741160135683411968, "in_reply_to_user": 298337975, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 298337975 }}, "user": { "id": 1337230975, "name": "atlai", "screen_name": "atlai7", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-04-08"), "description": "Do what is right and do it right", "followers_count": 450, "friends_count": 345, "statues_count": 3594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162986467954688, "text": "I be having the best dreams g then wake up on some fuck shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3226640252, "name": "o [ k ]", "screen_name": "ovrkast", "lang": "en", "location": "Oakland, CA", "create_at": date("2015-05-25"), "description": "loops/beats/vibes", "followers_count": 221, "friends_count": 139, "statues_count": 1372 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162986706997250, "text": "2009 https://t.co/bLvkq830Bx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 249907496, "name": "Princess Tay", "screen_name": "LikeImTayha", "lang": "en", "location": "L.A.T ♥️", "create_at": date("2011-02-09"), "description": "insta: shecravesLondon | snapchat: likeimtayha #domme", "followers_count": 4330, "friends_count": 993, "statues_count": 83816 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lansing, MI", "id": "91eb113282d003a1", "name": "Lansing", "place_type": "city", "bounding_box": rectangle("-84.63184,42.618566 -84.483958,42.805532") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2646000, "cityName": "Lansing" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162986954457088, "text": "KSJC 100653Z 35006KT 10SM FEW012 16/11 A2995 RMK AO2 SLP141 T01610111", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.92911111,37.36294444"), "retweet_count": 0, "lang": "pl", "is_retweet": false, "user": { "id": 1729101421, "name": "KSJC METAR", "screen_name": "ksjc_metar", "lang": "ja", "location": "San Jose, CA", "create_at": date("2013-09-04"), "description": "Norman Y. Mineta San Jose International Airport", "followers_count": 11, "friends_count": 1, "statues_count": 24280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Jose, CA", "id": "7d62cffe6f98f349", "name": "San Jose", "place_type": "city", "bounding_box": rectangle("-122.035311,37.193164 -121.71215,37.469154") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 668000, "cityName": "San Jose" } }
+{ "create_at": datetime("2016-06-10T00:00:00.000Z"), "id": 741162987222880256, "text": "Or mallinaggi rematch https://t.co/zXm4uYNjdR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 310450644, "name": "Street Fam .", "screen_name": "MadMiles_", "lang": "en", "location": "757 ", "create_at": date("2011-06-03"), "description": "I promise 2016 will be my best year of boxing and as the years go by I'll keep gettin better [BNG Boxing]", "followers_count": 1656, "friends_count": 2451, "statues_count": 44032 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norfolk, VA", "id": "b004be67b9fd6d8f", "name": "Norfolk", "place_type": "city", "bounding_box": rectangle("-76.334287,36.820791 -76.177085,36.969373") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51710, "countyName": "Norfolk", "cityID": 5157000, "cityName": "Norfolk" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162987428446208, "text": "@bbyaum Keyah? Is that you?", "in_reply_to_status": -1, "in_reply_to_user": 3340405032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3340405032 }}, "user": { "id": 63499695, "name": "FutureStayAtHomeDad", "screen_name": "yelnamynhalej", "lang": "en", "location": "null", "create_at": date("2009-08-06"), "description": "Lookin for a woman to make me a stay at home dad. Submit your resume in my DMs. (If you're blocked it because you tweeted about your BF)", "followers_count": 2042, "friends_count": 985, "statues_count": 172395 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Hood, TX", "id": "c818c25e43957fd3", "name": "Fort Hood", "place_type": "city", "bounding_box": rectangle("-97.839309,31.10447 -97.722373,31.161093") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4826736, "cityName": "Fort Hood" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162987747184640, "text": "EVERY NIGGA DOES NOT WANT TO FUCK !!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2194849001, "name": "♉️", "screen_name": "Dreaa_2", "lang": "en", "location": "null", "create_at": date("2013-11-24"), "description": "#WSSU19❤", "followers_count": 1095, "friends_count": 792, "statues_count": 17450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winston-Salem, NC", "id": "24a53a1880093fa9", "name": "Winston-Salem", "place_type": "city", "bounding_box": rectangle("-80.389019,35.934126 -80.100949,36.240211") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37067, "countyName": "Forsyth", "cityID": 3775000, "cityName": "Winston-Salem" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162987847864323, "text": "@ebbtideapp Tide in Sniffens Point, Connecticut 06/10/2016\nHigh 4:02am 6.9\n Low 10:27am 0.2\nHigh 4:37pm 6.8\n Low 10:52pm 0.8", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-73.1133,41.1867"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 99, "friends_count": 1, "statues_count": 40138 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milford, CT", "id": "015e664c48444066", "name": "Milford", "place_type": "city", "bounding_box": rectangle("-73.117153,41.171997 -72.990337,41.276059") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 974260, "cityName": "Stratford" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162988183388161, "text": "About last night.. Took Nicole out for a belated Birthday Sister… https://t.co/ipy112Uwtk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.33330834,36.14689155"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37137331, "name": "Molly LaFon", "screen_name": "mollylafon", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2009-05-01"), "description": "Southern California girl living the Las Vegas life. I love my family. My motto is Live. Laugh. Love. I also blog on occasion & I love instagram. Peace & Love! ✌", "followers_count": 72, "friends_count": 452, "statues_count": 1886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Summerlin South, NV", "id": "0134e6167ff7f6ec", "name": "Summerlin South", "place_type": "city", "bounding_box": rectangle("-115.355825,36.082837 -115.314862,36.159081") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3270900, "cityName": "Summerlin South" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162988208553984, "text": "https://t.co/b6O6vS061C", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 555823610, "name": "Khalid F Als ✖️", "screen_name": "kayfss", "lang": "en", "location": "Indianapolis, IN - Riyadh, SA", "create_at": date("2012-04-17"), "description": "Studying Architectural Design™ @IUPUI", "followers_count": 174, "friends_count": 137, "statues_count": 56 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162988661530624, "text": "You run my mind boyyyy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3070382497, "name": "BenzzzZ✨", "screen_name": "mercedesceeeee", "lang": "en", "location": "yayyy area", "create_at": date("2015-03-09"), "description": "she ain't in luv", "followers_count": 576, "friends_count": 460, "statues_count": 24868 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Union City, CA", "id": "8af346f16e955392", "name": "Union City", "place_type": "city", "bounding_box": rectangle("-122.096971,37.56138 -121.992657,37.622938") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 681204, "cityName": "Union City" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162988745416704, "text": "Movin on the dollars sign this whole summer, the glo up ain't finish", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 734094344, "name": "Felix Garcia ✌", "screen_name": "ayeee_felix", "lang": "en", "location": "null", "create_at": date("2012-08-02"), "description": "EL JEFE 98", "followers_count": 238, "friends_count": 219, "statues_count": 7073 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989320036352, "text": "��@bingoplayers�� https://t.co/fZTQqPQN9V", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 45777849 }}, "user": { "id": 23621049, "name": "Beta Nightclub", "screen_name": "BetaNightclub", "lang": "en", "location": "Denver, CO", "create_at": date("2009-03-10"), "description": "#1 Club in America - Rolling Stone", "followers_count": 23893, "friends_count": 14661, "statues_count": 11463 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Denver, CO", "id": "b49b3053b5c25bf5", "name": "Denver", "place_type": "city", "bounding_box": rectangle("-105.109815,39.614151 -104.734372,39.812975") }, "geo_tag": { "stateID": 8, "stateName": "Colorado", "countyID": 8031, "countyName": "Denver", "cityID": 820000, "cityName": "Denver" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989458477056, "text": "Lol social media really does ruin a lot of things and that's unfortunate", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 989173956, "name": "✨ki✨", "screen_name": "_kiaaara_", "lang": "en", "location": "bay areaaa", "create_at": date("2012-12-04"), "description": "skin color: godiva", "followers_count": 933, "friends_count": 443, "statues_count": 51772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milpitas, CA", "id": "6ef29a7e78ca38a5", "name": "Milpitas", "place_type": "city", "bounding_box": rectangle("-121.931305,37.39609 -121.861025,37.466856") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 647766, "cityName": "Milpitas" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989529751553, "text": "I can feel your bad vibrations", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2755427768, "name": "slave2nothing", "screen_name": "Travis_WeidnerA", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2014-08-22"), "description": "/// Student & Triathlete at ASU /// Follower of Christ /// Front man of TELLER & Teløs/// Romans 8:18 /// SATX ✈️ TEMPE, AZ", "followers_count": 264, "friends_count": 498, "statues_count": 3557 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Helotes, TX", "id": "00a1b28251c671b8", "name": "Helotes", "place_type": "city", "bounding_box": rectangle("-98.764721,29.495566 -98.661448,29.604798") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4833146, "cityName": "Helotes" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989601034240, "text": "Philippians 4:13 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 542222955, "name": "BKM❄️", "screen_name": "mars_blake", "lang": "en", "location": "Loud City ", "create_at": date("2012-03-31"), "description": "|1 Corinthians 10:31| God First |LSWT| SC:blakekmars", "followers_count": 130, "friends_count": 143, "statues_count": 2017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chino Hills, CA", "id": "5e68ca240e32025a", "name": "Chino Hills", "place_type": "city", "bounding_box": rectangle("-117.788282,33.921548 -117.65524,34.023483") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 613214, "cityName": "Chino Hills" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989613649920, "text": "WE'RE GRADUATING TODAY!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1702881415, "name": "bprodz", "screen_name": "prodicky10", "lang": "en", "location": "south salem high school", "create_at": date("2013-08-26"), "description": "fun person | osu '20", "followers_count": 547, "friends_count": 402, "statues_count": 11420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salem, OR", "id": "c8022ca5114d7ea9", "name": "Salem", "place_type": "city", "bounding_box": rectangle("-123.122916,44.853813 -122.935114,45.016355") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41047, "countyName": "Marion", "cityID": 4164900, "cityName": "Salem" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989655560193, "text": "06/10@03:00 - Temp 57.0F, WC 57.0F. Wind 0.1mph W, Gust 3.0mph. Bar 30.002in, Rising slowly. Rain 0.00in. Hum 73%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49566 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989722685440, "text": "This #Hospitality #job might be a great fit for you: Barista - https://t.co/YEKtvPTwn2 #barista #Newark, NJ #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.1723667,40.735657"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Hospitality", "job", "barista", "Newark", "Hiring" }}, "user": { "id": 701776772057141248, "name": "HMSHost Jobs", "screen_name": "HMSHostCareers", "lang": "en", "location": "null", "create_at": date("2016-02-22"), "description": "HMSHost is part of the world’s largest provider of food & beverage services for travelers. Explore our various hourly and management positions in US & Canada!", "followers_count": 275, "friends_count": 219, "statues_count": 2185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Newark, NJ", "id": "fa3435044b52ecc7", "name": "Newark", "place_type": "city", "bounding_box": rectangle("-74.25136,40.679997 -74.118336,40.788991") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex", "cityID": 3451000, "cityName": "Newark" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162989915607040, "text": "omfg I want my LTE not 4G", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4562329153, "name": "kímm✨", "screen_name": "tbhkimm", "lang": "en", "location": "wonderland ", "create_at": date("2015-12-14"), "description": "Shaka , mi Aloha \\m/", "followers_count": 106, "friends_count": 87, "statues_count": 3688 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waipahu, HI", "id": "0de54c88126954b8", "name": "Waipahu", "place_type": "city", "bounding_box": rectangle("-158.032127,21.36976 -157.990212,21.399415") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1579700, "cityName": "Waipahu" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990058254341, "text": "@Rickym7599 @maishaamyers pull up the cameras", "in_reply_to_status": 741162600499687424, "in_reply_to_user": 1629503694, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1629503694, 277287688 }}, "user": { "id": 1408446710, "name": "Gabe Funes", "screen_name": "gabe_funes", "lang": "en", "location": "snapchat: gabefunesissexy", "create_at": date("2013-05-06"), "description": "Money stand like 8 feet just like 2 midgets 07-08-09", "followers_count": 541, "friends_count": 221, "statues_count": 26203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Casas Adobes, AZ", "id": "832feeba7eb96a20", "name": "Casas Adobes", "place_type": "city", "bounding_box": rectangle("-111.107896,32.297257 -110.956722,32.388159") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 410670, "cityName": "Casas Adobes" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990087622656, "text": "Lmfao Teddy Ray hilarious https://t.co/cxfdVZS4Q9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 547953914, "name": "Q", "screen_name": "ThirdWorldQuisy", "lang": "en", "location": "you won't believe this", "create_at": date("2012-04-07"), "description": "null", "followers_count": 2083, "friends_count": 684, "statues_count": 142634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990171488256, "text": "happy birthday @juliee710", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3146424826 }}, "user": { "id": 2987821371, "name": "Brisa Santiago", "screen_name": "brisaasantiago", "lang": "en", "location": "null", "create_at": date("2015-01-20"), "description": "null", "followers_count": 68, "friends_count": 72, "statues_count": 136 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garden Grove, CA", "id": "70707114a9202b98", "name": "Garden Grove", "place_type": "city", "bounding_box": rectangle("-118.042964,33.732214 -117.894116,33.805876") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 629000, "cityName": "Garden Grove" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990678990848, "text": "skin tone chicken bone leave me alone head ass ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4260680112, "name": "d+", "screen_name": "dylancatron13", "lang": "en", "location": "901/915/817", "create_at": date("2015-11-16"), "description": "sc : d.brizzle02 | c/o 18 | chs", "followers_count": 502, "friends_count": 388, "statues_count": 3074 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleburne, TX", "id": "70b538807e617654", "name": "Cleburne", "place_type": "city", "bounding_box": rectangle("-97.439627,32.298971 -97.349294,32.437917") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48251, "countyName": "Johnson", "cityID": 4815364, "cityName": "Cleburne" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990750322688, "text": "12:00pm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2297204570, "name": "Gabe Torres", "screen_name": "Gabriiel_Torrez", "lang": "en", "location": "Tucson, AZ", "create_at": date("2014-01-17"), "description": "Cholla High '17⚡️", "followers_count": 212, "friends_count": 425, "statues_count": 3444 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Drexel Heights, AZ", "id": "19b90ef8d6e28eab", "name": "Drexel Heights", "place_type": "city", "bounding_box": rectangle("-111.097562,32.09131 -111.006161,32.176145") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 420540, "cityName": "Drexel Heights" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990792282113, "text": "My dad was so happy he jumped on my bed ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 850224960, "name": "Desiree Monique ✨", "screen_name": "Dessi_Monarrez", "lang": "en", "location": "null", "create_at": date("2012-09-27"), "description": "happiness", "followers_count": 347, "friends_count": 476, "statues_count": 8403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Compton, CA", "id": "f2b1ac6c9d15503f", "name": "Compton", "place_type": "city", "bounding_box": rectangle("-118.263612,33.846698 -118.180422,33.923239") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 615044, "cityName": "Compton" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990821646337, "text": "Wind 1.0 mph ESE. Barometer 29.935 in, Steady. Temperature 76.1 °F. Rain today 0.00 in. Humidity 79%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162990976794626, "text": "Oh heyyyy! I get paid on a minute ����\n@", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 727823300, "name": "Kath-a-leen", "screen_name": "kathleenesperi", "lang": "en", "location": "null", "create_at": date("2012-07-31"), "description": "And now that you don't have to be perfect, you can be good.", "followers_count": 277, "friends_count": 233, "statues_count": 15339 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Whittier, CA", "id": "f8f8ee516109ee0e", "name": "Whittier", "place_type": "city", "bounding_box": rectangle("-118.072424,33.928201 -117.965287,34.023634") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685292, "cityName": "Whittier" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162991048101888, "text": "Ripley SW Limestone Co. Temp: 61.7°F Wind:0.0mph Pressure: 996.4mb Falling slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 54312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162991228465152, "text": ":( It hurts", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 605672380, "name": "️️irma", "screen_name": "irmugh", "lang": "en", "location": "null", "create_at": date("2012-06-11"), "description": "Fanny packs and cheese are my thing", "followers_count": 3598, "friends_count": 211, "statues_count": 89833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sterling, IL", "id": "6ba87c2e2d3960d0", "name": "Sterling", "place_type": "city", "bounding_box": rectangle("-89.771235,41.772244 -89.640186,41.827767") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17195, "countyName": "Whiteside", "cityID": 1772546, "cityName": "Sterling" } }
+{ "create_at": datetime("2016-06-10T00:00:01.000Z"), "id": 741162991232618496, "text": "Everyone go wish @JujuZawada a happy birthday!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 467917631 }}, "user": { "id": 35541247, "name": "Riley Taylor", "screen_name": "rileytayy", "lang": "en", "location": "null", "create_at": date("2009-04-26"), "description": "tell ur dad I say hi", "followers_count": 587, "friends_count": 254, "statues_count": 23012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Livermore, CA", "id": "159279f05be2ade4", "name": "Livermore", "place_type": "city", "bounding_box": rectangle("-121.823726,37.63653 -121.696432,37.730654") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 641992, "cityName": "Livermore" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162991790526464, "text": "@ryan_ripp51 hey", "in_reply_to_status": -1, "in_reply_to_user": 458916070, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 458916070 }}, "user": { "id": 317200133, "name": "Grace Malek", "screen_name": "gmmmal_", "lang": "en", "location": "Cedarburg", "create_at": date("2011-06-14"), "description": "you can call me pork chop", "followers_count": 514, "friends_count": 397, "statues_count": 4632 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Crosse, WI", "id": "92d35b3ae0c97732", "name": "La Crosse", "place_type": "city", "bounding_box": rectangle("-91.274654,43.725479 -91.143588,43.885808") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55063, "countyName": "La Crosse", "cityID": 5540775, "cityName": "La Crosse" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162991840841728, "text": "L https://t.co/veB5SBWfXd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 305979308, "name": "vincent vega", "screen_name": "kingwizzardd", "lang": "en", "location": "null", "create_at": date("2011-05-26"), "description": "Nothing comes to a sleeper but a dream", "followers_count": 470, "friends_count": 382, "statues_count": 16370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162991979266048, "text": "You don't miss me your just bored with the world you chose to explore.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 404562542, "name": "michele linville", "screen_name": "xx_chele", "lang": "en", "location": "Fort Hays State University ", "create_at": date("2011-11-03"), "description": "go through it & grow through it", "followers_count": 534, "friends_count": 400, "statues_count": 6598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hays, KS", "id": "b0e50abc7e3f7363", "name": "Hays", "place_type": "city", "bounding_box": rectangle("-99.354684,38.849348 -99.280695,38.914552") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20051, "countyName": "Ellis", "cityID": 2031100, "cityName": "Hays" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992054730752, "text": "send me emojis, Im bored dabbbbbbb", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2849256270, "name": "bianca", "screen_name": "QUEEEEEEEENB_", "lang": "en", "location": "with daddy", "create_at": date("2014-10-09"), "description": "@LucioAaron_23 ❣ / IG & SC: bianca.ivone / HTX", "followers_count": 860, "friends_count": 585, "statues_count": 13832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Channelview, TX", "id": "eabe7b6fd4504fff", "name": "Channelview", "place_type": "city", "bounding_box": rectangle("-95.176002,29.735548 -95.061098,29.851809") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4814236, "cityName": "Channelview" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992121827328, "text": "@brendag0nzalez damn", "in_reply_to_status": 741160725238951936, "in_reply_to_user": 2664038119, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2664038119 }}, "user": { "id": 4376228712, "name": "Val", "screen_name": "Valthepandaa_", "lang": "en", "location": "California, USA", "create_at": date("2015-11-27"), "description": "strength and guidance || VIII.X.XII", "followers_count": 151, "friends_count": 135, "statues_count": 2839 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jurupa Valley, CA", "id": "015b658472edd3dc", "name": "Jurupa Valley", "place_type": "city", "bounding_box": rectangle("-117.551185,33.955517 -117.410795,34.034306") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 637692, "cityName": "Jurupa Valley" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992142852096, "text": "I don't know if y'all know this but Live Photos have sound too��. Just turn your ringer on before you watch it��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2252331986, "name": "Sunshine™", "screen_name": "TheHoman", "lang": "en", "location": "wherever there's palm trees", "create_at": date("2013-12-18"), "description": "»when in doubt, pinky out«", "followers_count": 1227, "friends_count": 2135, "statues_count": 17298 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992448966656, "text": "Wind 0.0 mph ---. Barometer 30.024 in, Falling slowly. Temperature 60.3 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 27, "statues_count": 20460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992612593664, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 719480044, "name": "champagnemami", "screen_name": "Smilesam29", "lang": "en", "location": "null", "create_at": date("2013-10-17"), "description": "Queen to a beautiful princess/libra/21/Mainly tweet lyrics", "followers_count": 264, "friends_count": 219, "statues_count": 10848 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laredo, TX", "id": "4fd63188b772fc62", "name": "Laredo", "place_type": "city", "bounding_box": rectangle("-99.555983,27.409181 -99.353369,27.654973") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48479, "countyName": "Webb", "cityID": 4841464, "cityName": "Laredo" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992667086848, "text": "Umm with myself tf https://t.co/Vgb4Nwj2xf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 219207694, "name": "hectør zerøni ", "screen_name": "ThatGeekHector", "lang": "en", "location": "|-/", "create_at": date("2010-11-23"), "description": "try to love yourself as much as you want someone else to #UT20", "followers_count": 362, "friends_count": 771, "statues_count": 6307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162992742608896, "text": "Falling asleep hearing @mariobautista_ sweet voice!!���� this album is AMAZING!!!❤️��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 150003563 }}, "user": { "id": 625962670, "name": "Milagros Reyes✨❤️", "screen_name": "Mili_Bautista", "lang": "en", "location": "Warner Robins, GA", "create_at": date("2012-07-03"), "description": "вαutíѕtєr❤️| huntєr❤| rσчcєnαtícα❤️| beaster❤️", "followers_count": 844, "friends_count": 566, "statues_count": 27025 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warner Robins, GA", "id": "4caf02a6f195ddab", "name": "Warner Robins", "place_type": "city", "bounding_box": rectangle("-83.728243,32.495868 -83.554155,32.669392") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1380508, "cityName": "Warner Robins" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993166241797, "text": "Temp: 66.6°F Wind:0.0mph Pressure: 29.974hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993203957761, "text": "@allieee215 I'll send you snapchats of the dogs. It'll be like the Bachelorette dog version. The dogelorette.", "in_reply_to_status": 741161210964480001, "in_reply_to_user": 1177629020, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1177629020 }}, "user": { "id": 3532696218, "name": "Princess Riss♡", "screen_name": "marissacimmarr2", "lang": "en", "location": "somewhere over thinking", "create_at": date("2015-09-11"), "description": "Just another day in paradise", "followers_count": 226, "friends_count": 220, "statues_count": 3406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bostonia, CA", "id": "e7069cc50d9e4475", "name": "Bostonia", "place_type": "city", "bounding_box": rectangle("-116.965306,32.807933 -116.918448,32.829919") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 607624, "cityName": "Bostonia" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993229172737, "text": "Wind 1.0 mph NW. Barometer 29.885 in, Rising. Temperature 45.5 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11328 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993451438080, "text": "MÉXICO, TE SIGO QUERIENDO UN CHINGO ����✨", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user": { "id": 64829037, "name": "Yaheli Aguilera", "screen_name": "yahelium", "lang": "en", "location": "tres metros sobre el cielo", "create_at": date("2009-08-11"), "description": "Free to roam, I made a home out of everywhere I've been.", "followers_count": 247, "friends_count": 206, "statues_count": 7180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Downey, CA", "id": "07dccb89e8f25a8f", "name": "Downey", "place_type": "city", "bounding_box": rectangle("-118.170429,33.90245 -118.090423,33.972897") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 619766, "cityName": "Downey" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993472425984, "text": "Temp 46.3° Hi/Lo 49.1/46.3 Rng 2.8° WC 46.3° Hmd 89% Rain 0.00\" Storm 0.00\" BAR 29.836 Rising DP 43.2° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993673752576, "text": "Few minutes ago lol https://t.co/bxIMxenfnt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1167580230, "name": "Imani Johnson", "screen_name": "young_money277", "lang": "en", "location": "South Houston, TX", "create_at": date("2013-02-10"), "description": "God | Future Veterinarian | CO16| Puerto Rico | إيمان", "followers_count": 1021, "friends_count": 2441, "statues_count": 17452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993849884673, "text": "She trippin out or what lmao https://t.co/jXyq1J39Yj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1056972841, "name": "ifureadingdisugai", "screen_name": "alialrida2", "lang": "en", "location": "null", "create_at": date("2013-01-02"), "description": "5'8... ifureadingdisuhellagai", "followers_count": 971, "friends_count": 621, "statues_count": 30336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bell, CA", "id": "2c1b78093786a74a", "name": "Bell", "place_type": "city", "bounding_box": rectangle("-118.206291,33.95407 -118.152019,33.998359") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 604870, "cityName": "Bell" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993883480068, "text": "I'm bored and hungry", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356586832, "name": "Çåįø", "screen_name": "kyleAmcmurrich", "lang": "en", "location": "West Richland, WA", "create_at": date("2011-08-16"), "description": "EWU•20•303•RIP,Maddi&Dan•", "followers_count": 831, "friends_count": 479, "statues_count": 23498 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Richland, WA", "id": "63ff031a0a0676b8", "name": "West Richland", "place_type": "city", "bounding_box": rectangle("-119.406328,46.252904 -119.316073,46.366144") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53005, "countyName": "Benton", "cityID": 5377665, "cityName": "West Richland" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162993891835905, "text": "@rayvnSE @DaddyTrumpYRN waking up", "in_reply_to_status": 741063093518667777, "in_reply_to_user": 2993996340, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2993996340, 2414830063 }}, "user": { "id": 2377976611, "name": "Brandonnnn", "screen_name": "shitcans", "lang": "en", "location": "Bellingham, MA", "create_at": date("2014-03-07"), "description": "17 | i dream to stream, I'm an upcoming RiceGum | @imbooting", "followers_count": 4421, "friends_count": 204, "statues_count": 11149 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bellingham, MA", "id": "3700c5f746b4a4e4", "name": "Bellingham", "place_type": "city", "bounding_box": rectangle("-71.499167,42.017193 -71.441404,42.135542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25021, "countyName": "Norfolk", "cityID": 2504965, "cityName": "Bellingham" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162994105737219, "text": "Rodeo Best Bites winner to open restaurant in Clear Lake https://t.co/4yhtqr33ad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.08709717,29.51160049"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2866011003, "name": "League City, TX News", "screen_name": "BLifeLeagueCity", "lang": "en", "location": "League City, Texas", "create_at": date("2014-11-07"), "description": "League City BubbleLife features community news, photos and events. Share your business, organization or personal news and events at https://t.co/Ew7CTPM7Zf.", "followers_count": 165, "friends_count": 122, "statues_count": 9850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "League City, TX", "id": "cf3b45e29c2c319b", "name": "League City", "place_type": "city", "bounding_box": rectangle("-95.178987,29.438994 -94.995071,29.55532") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4841980, "cityName": "League City" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162994164486146, "text": "Wind 0.0 mph ---. Barometer 29.919 in, Steady. Temperature 67.6 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162994701373440, "text": "WC ?? ������ https://t.co/0kByZki8Bl", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1021565125, "name": "♒", "screen_name": "Deessstiinneee", "lang": "en", "location": "California, USA", "create_at": date("2012-12-18"), "description": "only the good things, all of the good things", "followers_count": 810, "friends_count": 624, "statues_count": 19902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162995003314177, "text": "75.2F (Feels: 80.0F) - Humidity: 88% - Wind: 2.9mph ESE - Gust: 2.7mph - Pressure: 29.945in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 241871 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162995020075009, "text": "What's all this sudden obsession with toilets? I got a feeling I'm about to get way more popular.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3747330682, "name": "David Happy's", "screen_name": "BrightonWJ", "lang": "en", "location": "Weenie Hutt Jr's", "create_at": date("2015-09-23"), "description": "Yabbadabbadooing it.", "followers_count": 307, "friends_count": 481, "statues_count": 3048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgewood, NJ", "id": "38718ccc91e76508", "name": "Ridgewood", "place_type": "city", "bounding_box": rectangle("-74.149096,40.955019 -74.077902,41.007882") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34003, "countyName": "Bergen", "cityID": 3463000, "cityName": "Ridgewood" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162995389173760, "text": "@morethan88 The Basie Orchestra would love to talk with you about what we'd like to do in the arts education community.#81YearsOnTheRoad", "in_reply_to_status": -1, "in_reply_to_user": 125249726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "81YearsOnTheRoad" }}, "user_mentions": {{ 125249726 }}, "user": { "id": 237109504, "name": "CountBasieOrchestra", "screen_name": "countbasieorch", "lang": "en", "location": "On The Road!", "create_at": date("2011-01-11"), "description": "The Count Basie Orchestra, Directed by Scotty Barnhart, has won 18 Grammy Awards, 20 Downbeat and Jazz Times polls. More than any other big band in jazz.", "followers_count": 1677, "friends_count": 1144, "statues_count": 490 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:02.000Z"), "id": 741162995494072320, "text": "@Nellis_ takes 20 mins man", "in_reply_to_status": 741162633005572096, "in_reply_to_user": 393652251, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 393652251 }}, "user": { "id": 812030317, "name": "Reddup", "screen_name": "Reddup_", "lang": "en", "location": "Lost in the Library.", "create_at": date("2012-09-08"), "description": "Avid Wings fan. https://vine.co/v/OvPhBwjqEhJ", "followers_count": 2092, "friends_count": 609, "statues_count": 6352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Toledo, OH", "id": "7068dd9474ab6973", "name": "Toledo", "place_type": "city", "bounding_box": rectangle("-83.694776,41.580375 -83.454566,41.732806") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39095, "countyName": "Lucas", "cityID": 3977000, "cityName": "Toledo" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162995900899329, "text": "Partay @ Atlanta-West End https://t.co/WyWLgBttzQ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.41532473,33.73903513"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2318116885, "name": "Chrissy", "screen_name": "LAPARRIEA", "lang": "en", "location": "Atlanta, GA", "create_at": date("2014-01-29"), "description": "Eh", "followers_count": 151, "friends_count": 107, "statues_count": 2023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162996232294400, "text": "Mood- ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2432765790, "name": "Papì Ty", "screen_name": "living981", "lang": "en", "location": "Sleep", "create_at": date("2014-04-07"), "description": "Yeah", "followers_count": 1141, "friends_count": 920, "statues_count": 7920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowick, OH", "id": "df4f3fa83ca683de", "name": "Willowick", "place_type": "city", "bounding_box": rectangle("-81.488902,41.618052 -81.451305,41.652659") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3985638, "cityName": "Willowick" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162996383272960, "text": "Dancing in Jerusalem https://t.co/Zc0dv5Knxm ������������������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 465008368, "name": "JOY4JESUSCHRIST", "screen_name": "JOY4JESUSCHRIST", "lang": "en", "location": "NEW JERSEY", "create_at": date("2012-01-15"), "description": "Greeting friends,I am Pamela Renee on a mission to share The Gospel Of Jesus Christ and God Love & Perfect Plan 4 each of our lives.Shalom! Pray 4 Israel♥", "followers_count": 313, "friends_count": 1520, "statues_count": 2732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwood, NJ", "id": "0036534d6f287a14", "name": "Blackwood", "place_type": "city", "bounding_box": rectangle("-75.088811,39.726424 -74.968468,39.852636") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3406040, "cityName": "Blackwood" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162996995633152, "text": "6️⃣✖️1️⃣0️⃣", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2686779180, "name": "K9️⃣", "screen_name": "kendricrsc", "lang": "en", "location": "Crenshaw District, Los Angeles", "create_at": date("2014-07-27"), "description": "null", "followers_count": 530, "friends_count": 369, "statues_count": 40833 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162997201133569, "text": "I love when my Twitter says I have a notification but I don't. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 384593707, "name": "Rave Dad✌️", "screen_name": "AdamPardo", "lang": "en", "location": "Fairwood, WA", "create_at": date("2011-10-03"), "description": "PLUR |pnw raver|bass enthusiast|", "followers_count": 411, "friends_count": 756, "statues_count": 5419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Maple Valley, WA", "id": "0978ca453ae10730", "name": "Maple Valley", "place_type": "city", "bounding_box": rectangle("-122.070326,47.340174 -121.99301,47.406508") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5343150, "cityName": "Maple Valley" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162997297598465, "text": "When you find out sum of yo friends the Feds����.. Better watch out niggas and hoes be tellin��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 249733186, "name": "AlmightyDjay", "screen_name": "AlmightyDjay", "lang": "en", "location": "Murfreesboro, TN", "create_at": date("2011-02-09"), "description": "IG | @AlmightyDjay B.L.I.M.J.", "followers_count": 636, "friends_count": 159, "statues_count": 6481 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162997729611777, "text": "Hell no �� https://t.co/x5qHS874Gc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1972519279, "name": "Nadia", "screen_name": "NadiaOliva8", "lang": "en", "location": "Lockhart ✈ ", "create_at": date("2013-10-19"), "description": "C/o'16|sc:Nadiaaa8", "followers_count": 717, "friends_count": 601, "statues_count": 11349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162998056751104, "text": "@breeebabeyy taking you for your birthday �� https://t.co/bUKsQW0WZQ", "in_reply_to_status": -1, "in_reply_to_user": 816795320, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 816795320 }}, "user": { "id": 979664515, "name": "T", "screen_name": "Toni_TheGreatt", "lang": "en", "location": "San Gabriel, CA", "create_at": date("2012-11-29"), "description": "Breanna Nicole is my girl ❤️ snapchat: toni_dwyer Instagram: toni_dwyer.26", "followers_count": 451, "friends_count": 387, "statues_count": 15520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East San Gabriel, CA", "id": "b1d58a0d9da3e980", "name": "East San Gabriel", "place_type": "city", "bounding_box": rectangle("-118.091316,34.103711 -118.067295,34.130848") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 621096, "cityName": "East San Gabriel" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162998513995778, "text": "jade and I are the hottest girls in Texas FOH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1483598004, "name": "LUNACHIK", "screen_name": "melissealy", "lang": "en", "location": "Greater Uptown, Houston", "create_at": date("2013-06-04"), "description": "born rotten texas trash", "followers_count": 583, "friends_count": 470, "statues_count": 15616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162999063449600, "text": "It's a #teamhennessy thing .. U know ! #hou ���������� @ Clé Houston https://t.co/3kbD4AEzUI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-95.37261,29.74673"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "teamhennessy", "hou" }}, "user": { "id": 59434307, "name": "DjGeraldo", "screen_name": "DjGeraldo", "lang": "en", "location": "ÜT: 29.722047,-95.327568", "create_at": date("2009-07-23"), "description": "#TeamHennessy #Hou | Public Figure | Radio Personality-Producer on Mega 101.1 FM @Mega101Houston | HOUSTON ✈️❤️NEW YORK", "followers_count": 3597, "friends_count": 820, "statues_count": 15974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162999608643584, "text": "That's the kind of conversation that leaves you wanting to feel his body on yours.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2815070900, "name": "dragon", "screen_name": "eemm141", "lang": "en", "location": "null", "create_at": date("2014-09-17"), "description": "lick my titties.", "followers_count": 669, "friends_count": 694, "statues_count": 3837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pocatello, ID", "id": "d15336e5b45c79bb", "name": "Pocatello", "place_type": "city", "bounding_box": rectangle("-112.496051,42.806434 -112.373604,42.920598") }, "geo_tag": { "stateID": 16, "stateName": "Idaho", "countyID": 16005, "countyName": "Bannock", "cityID": 1664090, "cityName": "Pocatello" } }
+{ "create_at": datetime("2016-06-10T00:00:03.000Z"), "id": 741162999910699008, "text": "I'm really sad I can't go trick or treating until I have a son now because I'm ancient", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 301903979, "name": "velociraptor", "screen_name": "JordynEliz", "lang": "en", "location": "21, LA / The Shire", "create_at": date("2011-05-20"), "description": "slightly evil vampire hobbit sith :/", "followers_count": 1370, "friends_count": 608, "statues_count": 67052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163000187523074, "text": "He hit that bitch lol https://t.co/wAkAyJhGa5", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 298908370, "name": "Barliss Allen", "screen_name": "JiggyBoyShit", "lang": "en", "location": "6ixside ✈️ 214", "create_at": date("2011-05-14"), "description": "1 Corinthians 13:11", "followers_count": 1353, "friends_count": 430, "statues_count": 49931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163000246218752, "text": "Life is just a big \"wow ok\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 607977496, "name": "Jo'L Pretends Eagle", "screen_name": "jocecelia_", "lang": "en", "location": "null", "create_at": date("2012-06-14"), "description": "No one expects an angel to set the world on fire.", "followers_count": 824, "friends_count": 558, "statues_count": 11125 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Aberdeen, SD", "id": "78c4b1067358fa99", "name": "Aberdeen", "place_type": "city", "bounding_box": rectangle("-98.536478,45.430119 -98.392195,45.492967") }, "geo_tag": { "stateID": 46, "stateName": "South Dakota", "countyID": 46013, "countyName": "Brown", "cityID": 4600100, "cityName": "Aberdeen" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163000552423424, "text": "@YeaaimRich you're welcome", "in_reply_to_status": 741162909716340737, "in_reply_to_user": 244322034, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 244322034 }}, "user": { "id": 64093982, "name": "Yahaira ❣", "screen_name": "Yahairaag_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-08-08"), "description": "Dreamvillain & Stan", "followers_count": 670, "friends_count": 741, "statues_count": 35131 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163000976003073, "text": "Wind 0.7 mph WNW. Barometer 29.93 in, Rising slowly. Temperature 51.1 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 123, "statues_count": 160432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163000980201472, "text": "Wind 0.0 mph SSW. Barometer 29.913 in, Rising slowly. Temperature 72.9 °F. Rain today 0.00 in. Humidity 77%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 14, "friends_count": 53, "statues_count": 9065 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163001294753796, "text": "Why not it was lit ������ https://t.co/Tr2wXDSykH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 288063700, "name": "Łonely Boyz", "screen_name": "Yfg_legend", "lang": "en", "location": "da bay", "create_at": date("2011-04-25"), "description": "Dm's don't come to my phone", "followers_count": 832, "friends_count": 846, "statues_count": 7616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163001475174400, "text": "I really don't get along w/ cancers for my life....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319958624, "name": "Tu Majestad", "screen_name": "thatcoolkidana", "lang": "en", "location": "Creating Greatnes$", "create_at": date("2011-06-18"), "description": "cálmate o te calmo", "followers_count": 695, "friends_count": 459, "statues_count": 26388 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dale City, VA", "id": "66cd2cdb819f4414", "name": "Dale City", "place_type": "city", "bounding_box": rectangle("-77.443065,38.61099 -77.274813,38.68125") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51153, "countyName": "Prince William", "cityID": 5121088, "cityName": "Dale City" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163001655484416, "text": "these niggas won't ever learn i guess .. don't be undirect with shit if you a so called \"real\" nigga !", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 218661309, "name": "שמםטשל", "screen_name": "ThaThronee__", "lang": "en", "location": "kween zity ", "create_at": date("2010-11-22"), "description": "〽oney Team. #UNCC19 || AndSeeThatsTheThing", "followers_count": 1002, "friends_count": 901, "statues_count": 23121 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reidsville, NC", "id": "bd61aa8680f50770", "name": "Reidsville", "place_type": "city", "bounding_box": rectangle("-79.710954,36.304155 -79.616099,36.388094") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37157, "countyName": "Rockingham", "cityID": 3755900, "cityName": "Reidsville" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163001873600516, "text": "I plan on pulling oomf in 3 weeks..\nI call it my 3 week plan...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2740554634, "name": "Frank.", "screen_name": "Frank2280", "lang": "en", "location": "IceRank", "create_at": date("2014-08-11"), "description": "null", "followers_count": 482, "friends_count": 396, "statues_count": 12056 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163002485936132, "text": "@phwerner and a truly intelligent system would understand which meetings could be bumped to accommodate higher priority contacts.", "in_reply_to_status": 741128000280879104, "in_reply_to_user": 216945183, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 12596972 }}, "user": { "id": 216945183, "name": "Brad Newman", "screen_name": "bradknewman", "lang": "en", "location": "Bay Area", "create_at": date("2010-11-17"), "description": "Practice Innovation Manager @ Cooley, Stanford grad, Toronto native, OAK resident, SF tourist. Opinions are mine. (╯°□°)╯︵ ┻━┻", "followers_count": 1010, "friends_count": 675, "statues_count": 6347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oakland, CA", "id": "ab2f2fac83aa388d", "name": "Oakland", "place_type": "city", "bounding_box": rectangle("-122.34266,37.699279 -122.114711,37.884709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 653000, "cityName": "Oakland" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163002490146816, "text": "Choking Off trees", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3031471648, "name": "Isaacひ", "screen_name": "Official_banman", "lang": "en", "location": "hou$ton.", "create_at": date("2015-02-11"), "description": "null", "followers_count": 1086, "friends_count": 510, "statues_count": 1077 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163002511122433, "text": "Always tryna enter my mentions", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4658346271, "name": "Ugly Antho", "screen_name": "yt__antho", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "Houston's Finest", "followers_count": 2235, "friends_count": 1352, "statues_count": 15332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163002720878592, "text": "Day 2 at #Infocomm16 #cisgrandquest @commintegrator - I visited @kanexpro https://t.co/cdKliLbJqH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Infocomm16", "cisgrandquest" }}, "user_mentions": {{ 204837082, 2409785090 }}, "user": { "id": 18357132, "name": "Dawn Meade", "screen_name": "AVDawn", "lang": "en", "location": "Maryland, USA", "create_at": date("2008-12-24"), "description": "A/V Pro, CTS, MBA - Snarky, geeky chick tweeting abt AV, technology, social media, pipe bands, tenor drumming, dance & general mayhem. *salutes* General Mayhem!", "followers_count": 3402, "friends_count": 1418, "statues_count": 6480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Winchester, NV", "id": "37d88f13e7a85f14", "name": "Winchester", "place_type": "city", "bounding_box": rectangle("-115.173994,36.128077 -115.083699,36.144748") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3284600, "cityName": "Winchester" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163002980884481, "text": "Finna just get madden bussin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 855463310, "name": "AK✨", "screen_name": "OTG_AK", "lang": "en", "location": "Peoria, Illinois ★", "create_at": date("2012-09-30"), "description": "#ISU19 #OTG THE MOB ! Snapchat: AYEKING @LexisSymmone ✨", "followers_count": 2625, "friends_count": 773, "statues_count": 56052 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Normal, IL", "id": "fc7c2e706034396b", "name": "Normal", "place_type": "city", "bounding_box": rectangle("-89.061798,40.487838 -88.921261,40.561712") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17113, "countyName": "McLean", "cityID": 1753234, "cityName": "Normal" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163003027070976, "text": "@Chatetris @Dakarus @chenthefagx @HeyManasseh @xenobike I'll be on more tommorow and Saturday :)", "in_reply_to_status": 741160017513046016, "in_reply_to_user": 24563909, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 24563909, 225733059, 458234852, 748895569, 93955970 }}, "user": { "id": 481669225, "name": "funwithrobes", "screen_name": "funwithrobes", "lang": "en", "location": "orlando fl", "create_at": date("2012-02-02"), "description": "wannabe anitwitter handler", "followers_count": 85, "friends_count": 92, "statues_count": 1823 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163003047972867, "text": "@Madelooooon \n- boba \n- irvine \n- stats\n- t4\n- pocahontas \n- 8\n- nolram's paper", "in_reply_to_status": -1, "in_reply_to_user": 1481063084, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1481063084 }}, "user": { "id": 1573892774, "name": "wizard kelly, y'all", "screen_name": "JetDaAirplane", "lang": "en", "location": "sfsu '20", "create_at": date("2013-07-06"), "description": "null", "followers_count": 364, "friends_count": 336, "statues_count": 12261 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Castro Valley, CA", "id": "1a5fd1b93128bb9e", "name": "Castro Valley", "place_type": "city", "bounding_box": rectangle("-122.130814,37.678709 -122.002131,37.752855") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 611964, "cityName": "Castro Valley" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163003068985346, "text": "Sometimes I feel like I'm bothering you...��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2839141214, "name": "Destiny Rose", "screen_name": "Destinyluvsyews", "lang": "en", "location": "Alice, TX", "create_at": date("2014-10-03"), "description": "Senior'17|AHS|Follow my Instagram", "followers_count": 196, "friends_count": 160, "statues_count": 1037 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alice, TX", "id": "8de14ceebb414fe3", "name": "Alice", "place_type": "city", "bounding_box": rectangle("-98.109116,27.713198 -98.036162,27.784932") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48249, "countyName": "Jim Wells", "cityID": 4801852, "cityName": "Alice" } }
+{ "create_at": datetime("2016-06-10T00:00:04.000Z"), "id": 741163003249348608, "text": "it's not always about you honey", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1292841018, "name": "Mayraa", "screen_name": "Sanchez_019", "lang": "en", "location": "null", "create_at": date("2013-03-23"), "description": "M.S", "followers_count": 952, "friends_count": 809, "statues_count": 19946 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mexia, TX", "id": "041a08c2d0c3f01f", "name": "Mexia", "place_type": "city", "bounding_box": rectangle("-96.504204,31.653047 -96.453416,31.712644") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48293, "countyName": "Limestone", "cityID": 4847916, "cityName": "Mexia" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525371107966976, "text": "@AryannaReneeee just got out. it's scary af no boost", "in_reply_to_status": 741521587954814976, "in_reply_to_user": 4721255856, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4721255856 }}, "user": { "id": 1309060850, "name": "bhas", "screen_name": "_nieceee", "lang": "en", "location": "w/ latisha & tay tay", "create_at": date("2013-03-27"), "description": "18 | Colby College Signee #shorthairtwitter", "followers_count": 2189, "friends_count": 1274, "statues_count": 41441 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525371258953729, "text": "-", "in_reply_to_status": 736371078348431361, "in_reply_to_user": 2720105499, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2720105499, "name": "scott", "screen_name": "sctybg", "lang": "en", "location": "null", "create_at": date("2014-07-21"), "description": "Sad Boyz", "followers_count": 2172, "friends_count": 999, "statues_count": 484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Barbara, CA", "id": "f6ebc676e5cde864", "name": "Santa Barbara", "place_type": "city", "bounding_box": rectangle("-119.763341,34.395525 -119.639931,34.464245") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6083, "countyName": "Santa Barbara", "cityID": 669070, "cityName": "Santa Barbara" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525371343015936, "text": "@QueenKoya_ no ��", "in_reply_to_status": 741524761491177473, "in_reply_to_user": 604889314, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 604889314 }}, "user": { "id": 443032726, "name": "iaintshit", "screen_name": "Godesss_", "lang": "en", "location": "soflo☀️", "create_at": date("2011-12-21"), "description": "Charge On #UCF20", "followers_count": 788, "friends_count": 755, "statues_count": 16548 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Lauderdale, FL", "id": "6c686af766d8429c", "name": "Fort Lauderdale", "place_type": "city", "bounding_box": rectangle("-80.20811,26.080935 -80.090235,26.219801") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1224000, "cityName": "Fort Lauderdale" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525371653390336, "text": "Gone too soon. #RIPChristinaGrimmie ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RIPChristinaGrimmie" }}, "user": { "id": 107721531, "name": "Patricia", "screen_name": "patpatzpatty", "lang": "en", "location": "null", "create_at": date("2010-01-23"), "description": "IG/Snapchat: patpatzpatty | RN | Thomasian", "followers_count": 501, "friends_count": 392, "statues_count": 28588 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525371854655488, "text": "Mouth", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 372080958, "name": "Ez", "screen_name": "Ezrology", "lang": "en", "location": "Myrtle Beach, SC", "create_at": date("2011-09-11"), "description": "| CofC'20 | Matthew 6:33. | Tiger style |snapchat: bige-z", "followers_count": 717, "friends_count": 432, "statues_count": 18179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373129740288, "text": "@BigRed_Burr @RybakJon https://t.co/3jaLa9flcQ", "in_reply_to_status": 741525001443016704, "in_reply_to_user": 545434474, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 545434474, 442108866 }}, "user": { "id": 424793960, "name": "Jeremy", "screen_name": "jeremybible", "lang": "en", "location": "Buffalo", "create_at": date("2011-11-29"), "description": "washed up", "followers_count": 281, "friends_count": 217, "statues_count": 6288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cheektowaga, NY", "id": "d3f378f122d4f4a7", "name": "Cheektowaga", "place_type": "city", "bounding_box": rectangle("-78.799913,42.863516 -78.696885,42.950897") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36029, "countyName": "Erie", "cityID": 3615000, "cityName": "Cheektowaga" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373163167744, "text": "Di ko mudawat gyud atay", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 71700799, "name": "Meggy", "screen_name": "meggysanchez", "lang": "en", "location": "California, USA", "create_at": date("2009-09-04"), "description": "Estoy loco por ti", "followers_count": 409, "friends_count": 309, "statues_count": 18667 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clovis, CA", "id": "d3a7f19316bddcf6", "name": "Clovis", "place_type": "city", "bounding_box": rectangle("-119.732263,36.782957 -119.540115,36.878641") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 614218, "cityName": "Clovis" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373272236032, "text": "@lovekasuma @mmickey_ เย็บปิดไปเลย 5555", "in_reply_to_status": 741521113771954176, "in_reply_to_user": 947919157, "favorite_count": 0, "retweet_count": 0, "lang": "th", "is_retweet": false, "user_mentions": {{ 947919157, 123574187 }}, "user": { "id": 823108188, "name": "FlukieZZ", "screen_name": "Flukiezzz", "lang": "th", "location": "chonburi,bangkok", "create_at": date("2012-09-14"), "description": "ชื่อฟลุค :) ig:flukiezz", "followers_count": 279, "friends_count": 315, "statues_count": 25834 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373565861889, "text": "❤️ if I'll catch ya @ terrace days tomorrow???!?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2249670842, "name": "navy", "screen_name": "NavyHymas", "lang": "en", "location": "Riverdale, UT", "create_at": date("2013-12-16"), "description": "null", "followers_count": 597, "friends_count": 417, "statues_count": 2364 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverdale, UT", "id": "2595040ad5860917", "name": "Riverdale", "place_type": "city", "bounding_box": rectangle("-112.021586,41.152568 -111.977861,41.198763") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4964010, "cityName": "Riverdale" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373943480321, "text": "Kobe wouldn't choke like this in the finals.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1542324012, "name": "The Black Voice", "screen_name": "TheBlackkVoice", "lang": "en", "location": "null", "create_at": date("2013-06-23"), "description": "I'll remain a soldier till the war is won (umass Dartmouth 18' (pvd)) #cfc #SAspurs #saints", "followers_count": 1563, "friends_count": 796, "statues_count": 30049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Providence, RI", "id": "7b93be1d864cedbb", "name": "Providence", "place_type": "city", "bounding_box": rectangle("-71.474186,41.772455 -71.369479,41.861713") }, "geo_tag": { "stateID": 44, "stateName": "Rhode Island", "countyID": 44007, "countyName": "Providence", "cityID": 4459000, "cityName": "Providence" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373947510785, "text": "Wind 0.0 mph ---. Barometer 29.987 in, Steady. Temperature 71.8 °F. Rain today 0.00 in. Humidity 95%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66731 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525373951877120, "text": "self righteous", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3363246165, "name": "Hennessy Chucho ひ", "screen_name": "KingChucho300", "lang": "en", "location": "Chicago, IL", "create_at": date("2015-07-06"), "description": "I'm just tryna chill", "followers_count": 341, "friends_count": 389, "statues_count": 2846 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin Park, IL", "id": "da64cd6b4a6e157c", "name": "Franklin Park", "place_type": "city", "bounding_box": rectangle("-87.920335,41.911259 -87.849395,41.958002") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1727702, "cityName": "Franklin Park" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525374769594368, "text": "'The Voice' Alum Christina Grimmie Shot Dead After Being Shot at Orlando Concert https://t.co/B1ya4ApZ9m via @ABC", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 28785486 }}, "user": { "id": 44118725, "name": "Tom Cheney", "screen_name": "idobitom", "lang": "en", "location": "DC CA", "create_at": date("2009-06-02"), "description": "Tech geek. Music lover. Founder of idobi Radio. CEO idobi Network. http://idobi.com @idobiradio", "followers_count": 3037, "friends_count": 499, "statues_count": 6998 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525374782164992, "text": "1901 grogan Ave*����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 516787964, "name": "Jacob", "screen_name": "plazaaa_10", "lang": "en", "location": "Riding the Waves", "create_at": date("2012-03-06"), "description": "The World is Yours. I make Art. I see Energies. Spread love. Stay wavy. I can & I will plaza.jacob@yahoo.com", "followers_count": 609, "friends_count": 263, "statues_count": 14609 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Merced, CA", "id": "483f653fcdc595c0", "name": "Merced", "place_type": "city", "bounding_box": rectangle("-120.529171,37.25666 -120.414449,37.375785") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6047, "countyName": "Merced", "cityID": 646898, "cityName": "Merced" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525374820061184, "text": "You a real ass woman and I like it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 37765939, "name": "Karm ", "screen_name": "_NasTaughtMe", "lang": "en", "location": "null", "create_at": date("2009-05-04"), "description": "Aggie girls do it best. #NCAT18", "followers_count": 1646, "friends_count": 1605, "statues_count": 64400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525374866096128, "text": "@ebbtideapp Tide in Market Street Bridge, Pennsylvania 06/11/2016\nHigh 6:52am 6.7\n Low 2:08pm 0.2\nHigh 7:31pm 6.4\n Low 2:25am 0.7", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-75.18,39.955"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 99, "friends_count": 1, "statues_count": 40415 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525375038181376, "text": "Dick slanging", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53801670, "name": "5th", "screen_name": "FifthSeasonCo", "lang": "en", "location": "Evolve or die, -Manhattan. ", "create_at": date("2009-07-04"), "description": "Don't do it for free, do it for freedom", "followers_count": 1104, "friends_count": 680, "statues_count": 79621 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525375126274048, "text": "prepare for the spam tomorrow bc I'm going to look so good :-)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 520906131, "name": "daj", "screen_name": "y0Billy", "lang": "en", "location": "broward", "create_at": date("2012-03-10"), "description": "xotwod", "followers_count": 1127, "friends_count": 796, "statues_count": 26858 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pembroke Pines, FL", "id": "5727d936119423c0", "name": "Pembroke Pines", "place_type": "city", "bounding_box": rectangle("-80.442683,25.992486 -80.213314,26.063293") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12011, "countyName": "Broward", "cityID": 1255775, "cityName": "Pembroke Pines" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525375155462145, "text": "Everyone please be safe in whatever you do, anything can happen #RestInParadise https://t.co/WTCX2R9ven", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RestInParadise" }}, "user": { "id": 1360641480, "name": "Mateo ~ Life", "screen_name": "BiologistMatt", "lang": "en", "location": "Fontana, CA", "create_at": date("2013-04-17"), "description": "I retweet too much and tweet stupid stuff, sorry in advance", "followers_count": 176, "friends_count": 224, "statues_count": 6483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-06-11T00:00:00.000Z"), "id": 741525375272902656, "text": "������ playing the song 2x in less then 10 minutes killed my vibe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 53816537, "name": "Spiff", "screen_name": "LilMrsSpiffy", "lang": "en", "location": "Dallas", "create_at": date("2009-07-04"), "description": "perfect peace .", "followers_count": 3480, "friends_count": 1777, "statues_count": 178181 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525375394676738, "text": "#Variant1Million @variantcomics FTW!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Variant1Million" }}, "user_mentions": {{ 564868949 }}, "user": { "id": 131185487, "name": "ABE_PCF", "screen_name": "ABE_PCF", "lang": "en", "location": "Paper City Mass/Bronx NY", "create_at": date("2010-04-09"), "description": "I'm a regular dude doing extraordinary things..", "followers_count": 130, "friends_count": 269, "statues_count": 2545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Holyoke, MA", "id": "010bf7f82c40a09c", "name": "Holyoke", "place_type": "city", "bounding_box": rectangle("-72.703482,42.162001 -72.591564,42.285335") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25013, "countyName": "Hampden", "cityID": 2530840, "cityName": "Holyoke" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525375516184580, "text": "done", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1166322884, "name": "bai", "screen_name": "Bailey_Hubb", "lang": "en", "location": "tx", "create_at": date("2013-02-10"), "description": "tsu / sc-haileybubbard", "followers_count": 400, "friends_count": 392, "statues_count": 7942 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brownwood, TX", "id": "6f83807d72cc0d6b", "name": "Brownwood", "place_type": "city", "bounding_box": rectangle("-99.016179,31.651758 -98.936035,31.753719") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48049, "countyName": "Brown", "cityID": 4810780, "cityName": "Brownwood" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525375663017985, "text": "happy birthday @johnpeterswag ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2932157327 }}, "user": { "id": 2484737256, "name": "bⅇxxx", "screen_name": "tanskiingoddezz", "lang": "en", "location": "long beach", "create_at": date("2014-05-08"), "description": "love • ʚϊɞ nuggie ʚϊɞ", "followers_count": 15175, "friends_count": 391, "statues_count": 21722 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seal Beach, CA", "id": "d35feae19d268b09", "name": "Seal Beach", "place_type": "city", "bounding_box": rectangle("-118.117365,33.723095 -118.041655,33.787561") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 670686, "cityName": "Seal Beach" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525375726063616, "text": "sleeping alone tonight ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 265567346, "name": "champagnemami", "screen_name": "LilBootyLae", "lang": "en", "location": "virginiastateuni.", "create_at": date("2011-03-13"), "description": "R.I.P Mia , Fly High Floss & SwilleyVille❤️", "followers_count": 4869, "friends_count": 3428, "statues_count": 236668 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, VA", "id": "5fb04b3fffc3ad07", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-77.44753,37.171162 -77.336123,37.244906") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51149, "countyName": "Prince George", "cityID": 5161832, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525375914774528, "text": "Temp 48.3° Hi/Lo 51.2/48.3 Rng 2.9° WC 48.3° Hmd 94% Rain 0.00\" Storm 0.00\" BAR 30.032 Steady DP 46.7° Wnd 0mph Dir --- Gst 3mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19408 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525376602542082, "text": "I just want to be loved in all ways, every way. I want to feel worth it again.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 540355742, "name": "weird honey", "screen_name": "Skineticc", "lang": "en", "location": "Salt Lake City", "create_at": date("2012-03-29"), "description": "Alfred Hitchcock's lady.", "followers_count": 323, "friends_count": 177, "statues_count": 15365 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salt Lake City, UT", "id": "bd7c511e9f8bc5da", "name": "Salt Lake City", "place_type": "city", "bounding_box": rectangle("-112.038311,40.699895 -111.795741,40.831715") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4967000, "cityName": "Salt Lake City" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525376749293568, "text": "I am dead", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2378874396, "name": "ChillKy", "screen_name": "bruhh_itskyle", "lang": "en", "location": "ca", "create_at": date("2014-03-08"), "description": "chillin", "followers_count": 504, "friends_count": 321, "statues_count": 33780 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cordova, CA", "id": "f3cbe8607ff8ebaa", "name": "Rancho Cordova", "place_type": "city", "bounding_box": rectangle("-121.336699,38.525492 -121.224146,38.633556") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 659444, "cityName": "Rancho Cordova" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525376828997632, "text": "@alemmarie go to sleep", "in_reply_to_status": 741521009795203072, "in_reply_to_user": 2657557951, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2657557951 }}, "user": { "id": 2994122214, "name": "Kate c", "screen_name": "kxtec", "lang": "en", "location": "Houston, TX", "create_at": date("2015-01-23"), "description": "love love love love", "followers_count": 223, "friends_count": 151, "statues_count": 1460 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525377227460608, "text": "Smh trying to trade nudes for me telling him his birthday gift https://t.co/YpFnbWTKGG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 377809866, "name": "abbott edwards", "screen_name": "rabbott_xx6", "lang": "en", "location": "null", "create_at": date("2011-09-21"), "description": "I barely have time for avocado yoga | K52G6 | Cal Poly SLO '19", "followers_count": 419, "friends_count": 297, "statues_count": 10147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pismo Beach, CA", "id": "7e67fd2f1eeac401", "name": "Pismo Beach", "place_type": "city", "bounding_box": rectangle("-120.71479,35.128494 -120.604645,35.180956") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 657414, "cityName": "Pismo Beach" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525377248460800, "text": "@mrmanonthemoon7 https://t.co/HPi2OHx6mU", "in_reply_to_status": 741525201481924609, "in_reply_to_user": 418253303, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 418253303 }}, "user": { "id": 418253303, "name": "BB", "screen_name": "mrmanonthemoon7", "lang": "en", "location": " 49630 The Moon", "create_at": date("2011-11-21"), "description": "I'm 20 and probably a sugar daddy lmao.", "followers_count": 1779, "friends_count": 820, "statues_count": 30379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mentor, OH", "id": "a47c7932f8997705", "name": "Mentor", "place_type": "city", "bounding_box": rectangle("-81.391672,41.640241 -81.288901,41.75771") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3949056, "cityName": "Mentor" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525377307299840, "text": "I wonder what happened to some of the friendships I had hmmmm...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 82450590, "name": "ᗯendy KᗩᖇIᑎᗩ", "screen_name": "StruckbyHood", "lang": "en", "location": "CHIC△GO", "create_at": date("2009-10-14"), "description": "Living in the Now ⚛", "followers_count": 1159, "friends_count": 436, "statues_count": 42132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525377470877696, "text": "Need to use these tomatoes and peppers... Any ideas? @ Newtown… https://t.co/EQt1C7Ix8I", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-157.94034605,21.40400154"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1862378095, "name": "Michelle Ebalaroza", "screen_name": "MEbalaroza", "lang": "en", "location": "Hawaii, USA", "create_at": date("2013-09-13"), "description": "null", "followers_count": 208, "friends_count": 377, "statues_count": 4896 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waimalu, HI", "id": "5eb8435d7bc3faee", "name": "Waimalu", "place_type": "city", "bounding_box": rectangle("-157.96614,21.37939 -157.907975,21.428691") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1577750, "cityName": "Waimalu" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525377688838144, "text": "David is on my account rn ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 259944558, "name": "ana", "screen_name": "analyssanicolee", "lang": "en", "location": "Cañadas de Obregón, Jalisco", "create_at": date("2011-03-02"), "description": "sfhs", "followers_count": 361, "friends_count": 268, "statues_count": 30678 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378016104448, "text": "06/11@03:00 - Temp 56.2F, WC 56.2F. Wind 0.0mph WSW, Gust 1.0mph. Bar 30.062in, Falling slowly. Rain 0.00in. Hum 92%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378037121024, "text": "Wind 0.0 mph ---. Barometer 30.060 in, Rising slowly. Temperature 65.1 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 26, "statues_count": 20476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378150236160, "text": "@JerricaSings THE CAR", "in_reply_to_status": 741519371973992448, "in_reply_to_user": 175065182, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 175065182 }}, "user": { "id": 1298682228, "name": "Lanie Roo", "screen_name": "dancergrly", "lang": "en", "location": "West Valley City, UT", "create_at": date("2013-03-25"), "description": "B.K.G 5-23-16 UVU bound", "followers_count": 335, "friends_count": 209, "statues_count": 12155 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Valley City, UT", "id": "39cfa5509250734f", "name": "West Valley City", "place_type": "city", "bounding_box": rectangle("-112.074758,40.630579 -111.920124,40.726828") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4983470, "cityName": "West Valley City" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378238291968, "text": "Those security guards need to be fired they weak asf https://t.co/S8qQ2ni2cH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 711757846442938368, "name": "GoodDayToBeLAY", "screen_name": "laya_mahh", "lang": "en", "location": "null", "create_at": date("2016-03-20"), "description": "New to this..", "followers_count": 46, "friends_count": 25, "statues_count": 303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378355724288, "text": "When bae says you a special one https://t.co/LkllQ5WArp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1529500705, "name": "MAMI ARI", "screen_name": "_ariVibes", "lang": "en", "location": "island in the sun ", "create_at": date("2013-06-18"), "description": "☮", "followers_count": 603, "friends_count": 423, "statues_count": 6881 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Joaquin, CA", "id": "cf98e35256493349", "name": "San Joaquin", "place_type": "city", "bounding_box": rectangle("-120.198937,36.599014 -120.178619,36.614558") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 667126, "cityName": "San Joaquin" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378640936960, "text": "✨ \"What you doing in the club on a Thursday!?!\" ✖️ She say she only… https://t.co/LPuh9KFeSa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-106.49392,31.75698"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 257765516, "name": "⚠️", "screen_name": "cypress_hilll", "lang": "en", "location": "El Paso, TX", "create_at": date("2011-02-25"), "description": "✨ #HDYNATION | SC : CYPRESS_SAYS ✨", "followers_count": 1042, "friends_count": 976, "statues_count": 40486 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378720632832, "text": "Love & basketball and soul plane is my all time fav", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2272809162, "name": "6ixjb", "screen_name": "J4YFoE", "lang": "en", "location": "null", "create_at": date("2014-01-02"), "description": "ig:6ixjb", "followers_count": 1502, "friends_count": 1266, "statues_count": 9094 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Suisun City, CA", "id": "629b15360c8e51ae", "name": "Suisun City", "place_type": "city", "bounding_box": rectangle("-122.048884,38.22999 -121.969428,38.265203") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 675630, "cityName": "Suisun City" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378720665600, "text": "@_smallfrye ����������������������������", "in_reply_to_status": 741520835786223616, "in_reply_to_user": 2197421860, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2197421860 }}, "user": { "id": 479998725, "name": "Taller Fanchair", "screen_name": "tylerfancher1", "lang": "en", "location": "null", "create_at": date("2012-01-31"), "description": "To legit to quit -Ms. Moyer #striveforperfection Happiness never sleeps", "followers_count": 688, "friends_count": 1364, "statues_count": 2297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Conyngham, PA", "id": "99f57b0e70144529", "name": "Conyngham", "place_type": "city", "bounding_box": rectangle("-76.0793,40.982488 -76.041511,41.001343") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42079, "countyName": "Luzerne", "cityID": 4215888, "cityName": "Conyngham" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525378762563585, "text": "Back to the 3somes she goes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 554103296, "name": "Gucci Nate", "screen_name": "natefunk7", "lang": "en", "location": "Cedar Hills, UT", "create_at": date("2012-04-14"), "description": "snapchat: natefunk7", "followers_count": 812, "friends_count": 275, "statues_count": 18506 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Jordan, UT", "id": "b76a96fd566f9172", "name": "South Jordan", "place_type": "city", "bounding_box": rectangle("-112.031592,40.536852 -111.894963,40.582109") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49035, "countyName": "Salt Lake", "cityID": 4970850, "cityName": "South Jordan" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379005849601, "text": "White feet white flip flops and amazing ankle and barefoot tattoo. https://t.co/OJX3NWhWdj", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.172296,36.115794"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 143566721, "name": "Betty Barefoot", "screen_name": "barefootfetish", "lang": "en", "location": "West Coast", "create_at": date("2010-05-13"), "description": "I secretly videotape and take pictures of White & Latino women's barefeet in flipflops. There are a few exceptions of amazing feet from other races.", "followers_count": 253, "friends_count": 163, "statues_count": 378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flamingo Las Vegas Hotel & Casino", "id": "07d9f73543085000", "name": "Flamingo Las Vegas Hotel & Casino", "place_type": "poi", "bounding_box": rectangle("-115.1722961,36.1157939 -115.172296,36.115794") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379014234113, "text": "Needed this �� https://t.co/bsSTJaXlVM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 71117962, "name": "MGC", "screen_name": "_maryelll", "lang": "en", "location": "Kent,WA .", "create_at": date("2009-09-02"), "description": "null", "followers_count": 187, "friends_count": 177, "statues_count": 2503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379140231168, "text": "Wind 0.0 mph SE. Barometer 29.985 in, Steady. Temperature 77.5 °F. Rain today 0.00 in. Humidity 78%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379177971712, "text": "But baby I can promise you!! Partying With White People > Partying W/ your friends!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15756836, "name": "Jeremiah", "screen_name": "jneworleans", "lang": "en", "location": "Offshore ⛵", "create_at": date("2008-08-06"), "description": "I'd rather be hated than be a hater. SC & IG:KingJeezyBaby", "followers_count": 1413, "friends_count": 650, "statues_count": 219992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379312058372, "text": "Ain't nobody up that's fine enough to jump in they DMs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165983415, "name": "FREEKODAK", "screen_name": "DopeDUNC", "lang": "en", "location": "561/215", "create_at": date("2010-07-12"), "description": "No I won't cook for you don't ask. OMB", "followers_count": 1929, "friends_count": 569, "statues_count": 87400 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wellington, FL", "id": "0113afc024d5e0bc", "name": "Wellington", "place_type": "city", "bounding_box": rectangle("-80.299923,26.567057 -80.166565,26.682758") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1275812, "cityName": "Wellington" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379341377537, "text": "I'm such a good bestfriend ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 420788627, "name": "LIL LEX✨", "screen_name": "ShaNeil_x", "lang": "en", "location": "Sulphur, LA", "create_at": date("2011-11-24"), "description": "on my worst behavior ❄️ lil kodak wifey ✨", "followers_count": 1751, "friends_count": 1609, "statues_count": 76223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Charles, LA", "id": "21b6f72f9fdaeb06", "name": "Lake Charles", "place_type": "city", "bounding_box": rectangle("-93.314852,30.124859 -93.080595,30.280683") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22019, "countyName": "Calcasieu", "cityID": 2241155, "cityName": "Lake Charles" } }
+{ "create_at": datetime("2016-06-11T00:00:01.000Z"), "id": 741525379354001409, "text": "Temp: 74.0°F Wind:0.0mph Pressure: 30.033hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525379584798720, "text": "@PramDizzle @CringeLMAO @crystalyssaa https://t.co/8nzc70JkAU", "in_reply_to_status": 736634909637255168, "in_reply_to_user": 38734992, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 38734992, 3198261067, 715276423 }}, "user": { "id": 575611627, "name": "D.S Productions", "screen_name": "DSProductions13", "lang": "en", "location": "cincinnati ohio ", "create_at": date("2012-05-09"), "description": "name is freedom. aka dom the bomb from youtube dot com. from cincinnati, ohio. I make music, write and direct films.", "followers_count": 1511, "friends_count": 0, "statues_count": 22593 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525379706281984, "text": "#RIPChristina", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "RIPChristina" }}, "user": { "id": 257095803, "name": "Maya", "screen_name": "annyce101", "lang": "en", "location": "null", "create_at": date("2011-02-24"), "description": "young, gifted, and black ✨ HU '19 ⚓️", "followers_count": 540, "friends_count": 677, "statues_count": 18992 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525379756609536, "text": "Probably about me (her dream) @Hailzzz_15", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1860713892 }}, "user": { "id": 2893325671, "name": "kylierae", "screen_name": "kylierae33", "lang": "en", "location": "null", "create_at": date("2014-11-07"), "description": "Bria", "followers_count": 127, "friends_count": 76, "statues_count": 3312 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Tan Valley, AZ", "id": "002b06ee2655168a", "name": "San Tan Valley", "place_type": "city", "bounding_box": rectangle("-111.63454,33.08929 -111.486497,33.307181") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4021, "countyName": "Pinal", "cityID": 464210, "cityName": "San Tan Valley" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525379861610496, "text": "just close ya damn eyes https://t.co/PuS6AWe4Dz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327536496, "name": "سكايلر", "screen_name": "shabbasky", "lang": "en", "location": "htx", "create_at": date("2014-02-04"), "description": "god and softball amosc:skylar63624", "followers_count": 321, "friends_count": 200, "statues_count": 8327 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525380373315584, "text": "�� https://t.co/CthVmje5eN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 45872234, "name": "Brianna Davis", "screen_name": "breannenah", "lang": "en", "location": "Atlanta,GA/Nashville,TN", "create_at": date("2009-06-09"), "description": "Dental Hygiene Major at #TennStateU", "followers_count": 2823, "friends_count": 594, "statues_count": 66063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atlanta, GA", "id": "8173485c72e78ca5", "name": "Atlanta", "place_type": "city", "bounding_box": rectangle("-84.576827,33.647503 -84.289385,33.886886") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13121, "countyName": "Fulton", "cityID": 1304000, "cityName": "Atlanta" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525380444524545, "text": "We are looking for a buyer on 150 River Rd, UNIT D4 #MontvilleTwp. #NJ https://t.co/QbhyYrmmaC #realestate https://t.co/zPTitZPW82", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.369964,40.890549"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MontvilleTwp", "NJ", "realestate" }}, "user": { "id": 4026133461, "name": "RealtyOneGroupLegend", "screen_name": "RealtyOneLegend", "lang": "en", "location": "Wayne, NJ", "create_at": date("2015-10-23"), "description": "Committed to giving top quality service, continuous support, education and teamwork to reach your goals and ours. We are ONE!\nOffice- 973-686-7777", "followers_count": 15, "friends_count": 7, "statues_count": 94 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montville, NJ", "id": "00c606746b858396", "name": "Montville", "place_type": "city", "bounding_box": rectangle("-74.40231,40.843859 -74.317267,40.953552") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34013, "countyName": "Essex" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525380662624256, "text": "Babe's gunna love her present tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3164738952, "name": "Nathan Oetgen", "screen_name": "slay_emdown", "lang": "en", "location": "California, USA", "create_at": date("2015-04-19"), "description": "Hella from the bay; Ya'll finish or R ya'll done? •••••••••••••••••••••••••••••••••••••••••••••••Steelers // Warrior nation. Hot Tamale❤️⏫", "followers_count": 266, "friends_count": 209, "statues_count": 10891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, CA", "id": "d49a48f3c233112c", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-121.754989,37.891224 -121.677919,37.971552") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 608142, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525380683554817, "text": "Wow it's confirmed guys �� #ripchristinagrimmie https://t.co/kmvar6P4P1", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "ripchristinagrimmie" }}, "user": { "id": 1394006352, "name": "Krizzelle Alforeza", "screen_name": "krizalforeza", "lang": "en", "location": "Saipan✈Texas ", "create_at": date("2013-05-01"), "description": "Life doesn't get easier, you just get stronger. Houston Model. God, Family and friends over anything.", "followers_count": 355, "friends_count": 568, "statues_count": 4442 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Humble, TX", "id": "e589c1ec3b0b9d2e", "name": "Humble", "place_type": "city", "bounding_box": rectangle("-95.287557,29.953846 -95.231494,30.032521") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835348, "cityName": "Humble" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525380784328704, "text": "77.9F (Feels: 84.3F) - Humidity: 90% - Wind: 2.1mph SE - Gust: 1.7mph - Pressure: 30.027in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 242009 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525381061058560, "text": "@k_Renegade_ its nothin to link", "in_reply_to_status": 741524943062568960, "in_reply_to_user": 3604929795, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3604929795 }}, "user": { "id": 2786151156, "name": "TREWE$T", "screen_name": "_TreWest", "lang": "en", "location": "614 / DMV", "create_at": date("2014-09-02"), "description": "niggas bitches & i make art// trewestbooking@gmail.com", "followers_count": 829, "friends_count": 1442, "statues_count": 11205 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525381837029376, "text": "I'm watching the conjuring 2 right now and I'm bout to pass out", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 299457782, "name": "Amanda Anatra", "screen_name": "MandaAnatra", "lang": "en", "location": "null", "create_at": date("2011-05-15"), "description": "Oceanside, CA |", "followers_count": 951, "friends_count": 527, "statues_count": 25654 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carlsbad, CA", "id": "01d4e349481265e8", "name": "Carlsbad", "place_type": "city", "bounding_box": rectangle("-117.359298,33.060615 -117.216549,33.182353") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 611194, "cityName": "Carlsbad" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525381946052608, "text": "I will forever remember her blind audition as the greatest ever in the voices history!❤️❤️❤️ #RIPChristina https://t.co/WkR9fuBOyV", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RIPChristina" }}, "user": { "id": 3060557715, "name": "Jordy Espinoza", "screen_name": "Jordyespinoza54", "lang": "en", "location": "Orange, CA", "create_at": date("2015-02-24"), "description": "chipotle and working out oh yeah and chick fil a", "followers_count": 242, "friends_count": 446, "statues_count": 2170 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orange, CA", "id": "20ba6fb3b1ee82da", "name": "Orange", "place_type": "city", "bounding_box": rectangle("-117.904717,33.77319 -117.743381,33.847128") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 653980, "cityName": "Orange" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525382487248896, "text": "Ripley SW Limestone Co. Temp: 66.9°F Wind:0.0mph Pressure: 998.0mb Rising slowly Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 41, "friends_count": 33, "statues_count": 54349 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525382902521856, "text": "I wonder what rame's gona be like in 20 years ... Yuma ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2890314368, "name": "soonii", "screen_name": "Sabrineee_23", "lang": "en", "location": "null", "create_at": date("2014-11-04"), "description": "|No One Here Gets Out Alive.|", "followers_count": 521, "friends_count": 463, "statues_count": 1630 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orland Park, IL", "id": "0da8a6c990f02eed", "name": "Orland Park", "place_type": "city", "bounding_box": rectangle("-87.911936,41.552464 -87.790471,41.65669") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1756640, "cityName": "Orland Park" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525383003181056, "text": "Wind 0.0 mph ---. Barometer 29.995 in, Steady. Temperature 43.6 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11352 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525383397380096, "text": "There really are some evil people", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1081446344, "name": "sky☽", "screen_name": "skyxh_", "lang": "en", "location": "wherever im meant to be", "create_at": date("2013-01-11"), "description": "you never know", "followers_count": 487, "friends_count": 329, "statues_count": 8108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-06-11T00:00:02.000Z"), "id": 741525383640715265, "text": "Oms���� https://t.co/yIzSPPRDNo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2440464283, "name": "RahDawg‼️", "screen_name": "Shewant_Rah", "lang": "en", "location": "null", "create_at": date("2014-04-12"), "description": "sc: slidegod Chi✈️ATL", "followers_count": 760, "friends_count": 594, "statues_count": 11459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525383762190336, "text": "@aangelabrooke @_Summerella_ you know I'm down but where we goin lmao", "in_reply_to_status": 741524406183170048, "in_reply_to_user": 790463017, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 790463017, 277080666 }}, "user": { "id": 146635390, "name": "Olivia Huxtable", "screen_name": "Aye_IsThatRae", "lang": "en", "location": "null", "create_at": date("2010-05-21"), "description": "Y'all have fun.\nUnt ♥", "followers_count": 2461, "friends_count": 1817, "statues_count": 28950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384030769152, "text": "oh my god�� #RIPChristina", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "RIPChristina" }}, "user": { "id": 2863981285, "name": "jaclyn", "screen_name": "jaclyn5sos", "lang": "en", "location": "arizona", "create_at": date("2014-10-18"), "description": "allergic to anything and everything", "followers_count": 624, "friends_count": 661, "statues_count": 12466 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake Beach, MD", "id": "faef11a3eaa8abdb", "name": "Chesapeake Beach", "place_type": "city", "bounding_box": rectangle("-76.5803,38.644972 -76.526929,38.721348") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24009, "countyName": "Calvert", "cityID": 2415925, "cityName": "Chesapeake Beach" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384051580928, "text": "I need time to go by faster so I could be with zach", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 95532243, "name": "arabella♡", "screen_name": "cynthiarellaa", "lang": "en", "location": "21| LA ✈️ Phoenix", "create_at": date("2009-12-08"), "description": "airplanes and the 1975", "followers_count": 1120, "friends_count": 563, "statues_count": 43379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tempe, AZ", "id": "7cb7440bcf83d464", "name": "Tempe", "place_type": "city", "bounding_box": rectangle("-111.979047,33.319945 -111.877237,33.465823") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 473000, "cityName": "Tempe" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384223543296, "text": "bitchhhh we finna fuck them up�� https://t.co/oH9KLKrY8f", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2600539250, "name": "delgadø", "screen_name": "pndbelindaa", "lang": "en", "location": "null", "create_at": date("2014-07-02"), "description": "rip papi & josue", "followers_count": 484, "friends_count": 327, "statues_count": 2179 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384315953152, "text": "and just like that it's 3am", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 573280659, "name": "jalena", "screen_name": "jalenaam", "lang": "en", "location": "orch room ", "create_at": date("2012-05-06"), "description": "my life is a joke :)))", "followers_count": 458, "friends_count": 1512, "statues_count": 16713 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mastic, NY", "id": "ffaba230a4e0ec74", "name": "Mastic", "place_type": "city", "bounding_box": rectangle("-72.866558,40.775992 -72.814025,40.834075") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3646074, "cityName": "Mastic" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384383082496, "text": "I love the Curry family https://t.co/ACdHFiAx0U", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 338743271, "name": "Amee Dominguez", "screen_name": "AmeeLDominguez", "lang": "en", "location": "Orlando, FL", "create_at": date("2011-07-19"), "description": "@UofAlabama Alumna | South Florida Native | PR Professional | Coffee Addict | Lorelai Gilmore wannabe | Soccer Enthusiast", "followers_count": 851, "friends_count": 785, "statues_count": 57663 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384387170304, "text": "A happy 17th birthday & a happy 21st birthday to my two favorite people! ❤️❤️❤️ https://t.co/SPcBKfTtlH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 442487168, "name": "Kennedy Benitez", "screen_name": "K_Benny224", "lang": "en", "location": "null", "create_at": date("2011-12-20"), "description": "null", "followers_count": 696, "friends_count": 334, "statues_count": 20572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sherwood, OR", "id": "6870523b6f8d96e6", "name": "Sherwood", "place_type": "city", "bounding_box": rectangle("-122.869771,45.345593 -122.805551,45.376774") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4167100, "cityName": "Sherwood" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384517156864, "text": "#RIPChristina https://t.co/HKEWke8tpt", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "RIPChristina" }}, "user": { "id": 2213506285, "name": "Pizza Slut", "screen_name": "StoweSierra", "lang": "en", "location": "California, USA", "create_at": date("2013-11-24"), "description": "Ghs junior", "followers_count": 254, "friends_count": 417, "statues_count": 11529 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Modesto, CA", "id": "e41805d7248dbf1e", "name": "Modesto", "place_type": "city", "bounding_box": rectangle("-121.074342,37.59485 -120.900594,37.732784") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6099, "countyName": "Stanislaus", "cityID": 648354, "cityName": "Modesto" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384517165057, "text": "What am I doing up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3325920820, "name": "markithaaaa✨", "screen_name": "iimmkee", "lang": "en", "location": "sc.kenolaaa", "create_at": date("2015-06-14"), "description": "null", "followers_count": 1622, "friends_count": 1669, "statues_count": 27193 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384529731588, "text": "Así cerramos nuestra segunda noche del Congreso #EntreAmigas2016 con @luciaparker dirigiéndonos… https://t.co/Su5meQaSgZ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.106461,36.170467"), "retweet_count": 0, "lang": "es", "is_retweet": false, "hashtags": {{ "EntreAmigas2016" }}, "user_mentions": {{ 52233339 }}, "user": { "id": 87779453, "name": "PastorMarioRodriguez", "screen_name": "PastorMarioR", "lang": "es", "location": "Las Vegas", "create_at": date("2009-11-05"), "description": "Esposo de Mayra, Papá de Nicole e Izabella, Pastor Principal / Senior Pastor @C3LasVegas Director del Movimiento Hispano de @c3enespanol #c3enespañol", "followers_count": 83530, "friends_count": 167, "statues_count": 12503 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384559099904, "text": "Wow, gone way too soon. �� https://t.co/gVWuR0Kuwg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 30593335, "name": "Maricela Velazquez", "screen_name": "maricelahhhh", "lang": "en", "location": "The Sun From Within.", "create_at": date("2009-04-11"), "description": "An opportunist that dreams for plans. I'm a bottomless barrel always wanting more. You think you know but you have no idea...with me nothing is easy.", "followers_count": 584, "friends_count": 339, "statues_count": 17747 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sacramento, CA", "id": "b71fac2ee9792cbe", "name": "Sacramento", "place_type": "city", "bounding_box": rectangle("-121.576613,38.43792 -121.362715,38.685524") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384575864832, "text": "��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1672135346, "name": "jpan", "screen_name": "TheOfficialJpan", "lang": "en", "location": "INTERNATIONAL", "create_at": date("2013-08-14"), "description": "i keep quiet and let my outfit do the talking. ©", "followers_count": 347, "friends_count": 348, "statues_count": 6894 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Mirada, CA", "id": "4db990e393c2e28b", "name": "La Mirada", "place_type": "city", "bounding_box": rectangle("-118.037975,33.873395 -117.976352,33.928407") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640032, "cityName": "La Mirada" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384638779392, "text": "#PrayForChristina don't use the RIP one!! There's still hope!! Don't lose faith, it's slipping but it's still there.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "PrayForChristina" }}, "user": { "id": 2943176352, "name": "JC KNOWS I EXIST❤️", "screen_name": "itsssmena", "lang": "en", "location": "At Sammy's House", "create_at": date("2014-12-25"), "description": "I'm a Jc girl, a Taylor Caniff girl, Sammy Wilk girl, Jack Johnson girl, and a Niall Horan girl. ❤️", "followers_count": 421, "friends_count": 426, "statues_count": 15284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Torrance, CA", "id": "c1e1a49e92ce1ba0", "name": "Torrance", "place_type": "city", "bounding_box": rectangle("-118.394086,33.780441 -118.308349,33.886988") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 680000, "cityName": "Torrance" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384726904832, "text": "I'm so sad omg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2216139948, "name": "bri", "screen_name": "brxanaaa", "lang": "en", "location": "unknown", "create_at": date("2013-11-26"), "description": "null", "followers_count": 270, "friends_count": 195, "statues_count": 3962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Montclair, CA", "id": "3153d677b795e293", "name": "Montclair", "place_type": "city", "bounding_box": rectangle("-117.721072,34.046668 -117.680734,34.096817") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 648788, "cityName": "Montclair" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384877854720, "text": "Scary shit... \nhttps://t.co/qIiATalc6F https://t.co/oKTF6y9FDu", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2916103129, "name": "Nikole Dey", "screen_name": "Nikole_Dey", "lang": "en", "location": "null", "create_at": date("2014-12-01"), "description": "I like birds, i love band.", "followers_count": 120, "friends_count": 225, "statues_count": 1751 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon City, OR", "id": "93207bd39d52ef34", "name": "Oregon City", "place_type": "city", "bounding_box": rectangle("-122.639515,45.309499 -122.551968,45.38075") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4155200, "cityName": "Oregon City" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525384953466881, "text": "These #coconut #macaroons with a #chocolate #peanutbutter #sauce that Mariana made for our new… https://t.co/oJ9dalAbEH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.14180892,32.76273069"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "coconut", "macaroons", "chocolate", "peanutbutter", "sauce" }}, "user": { "id": 178407272, "name": "Chef Oz", "screen_name": "cuevabar", "lang": "en", "location": "San Diego, California", "create_at": date("2010-08-14"), "description": "Our intention at Cueva Bar is to touch your heart with the food we prepare with love, for you.", "followers_count": 2846, "friends_count": 2193, "statues_count": 23590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385062416384, "text": "We're all really on the phone like we don't have to wake up at 6am ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 122502148, "name": "as.ymp.tote", "screen_name": "tialhnem", "lang": "en", "location": "null", "create_at": date("2010-03-12"), "description": "null", "followers_count": 178, "friends_count": 120, "statues_count": 3103 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grapevine, TX", "id": "352cf4e7314da0b4", "name": "Grapevine", "place_type": "city", "bounding_box": rectangle("-97.129669,32.878571 -97.029639,32.985097") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4830644, "cityName": "Grapevine" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385121173505, "text": "Thanks", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366528059, "name": "Kenny", "screen_name": "K_Villalobos15", "lang": "en", "location": "Dinuba, CA", "create_at": date("2014-02-28"), "description": "#RattPack #C3 #ColeWorld", "followers_count": 752, "friends_count": 1021, "statues_count": 15023 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dinuba, CA", "id": "3b36422b848153f4", "name": "Dinuba", "place_type": "city", "bounding_box": rectangle("-119.430683,36.527438 -119.36768,36.575137") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 619318, "cityName": "Dinuba" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385209389056, "text": "I'm so hungry I'm really about to eat some waffles lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 975204338, "name": "slotty pippen", "screen_name": "BigPapaRahul", "lang": "en", "location": "null", "create_at": date("2012-11-27"), "description": "PSU '20", "followers_count": 662, "friends_count": 183, "statues_count": 31968 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Waterbury, CT", "id": "f9a4ed4d94a9b93f", "name": "Waterbury", "place_type": "city", "bounding_box": rectangle("-73.093845,41.513971 -72.950371,41.619186") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9009, "countyName": "New Haven", "cityID": 980000, "cityName": "Waterbury" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385377124352, "text": "It's 3am yet Lien and I are arguing about the Kardashians please send help", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 447416794, "name": "dani", "screen_name": "Daniza_Miho", "lang": "en", "location": "Miami, FL", "create_at": date("2011-12-26"), "description": "it's not me it's u", "followers_count": 357, "friends_count": 191, "statues_count": 48625 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Crossings, FL", "id": "303cfff181eb81bf", "name": "The Crossings", "place_type": "city", "bounding_box": rectangle("-80.416713,25.652281 -80.385149,25.686253") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1271567, "cityName": "The Crossings" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385578307584, "text": "Life would be boring af without @ignacio_224 ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3160317212 }}, "user": { "id": 2296375080, "name": "Bree Steadman", "screen_name": "__breighanna", "lang": "en", "location": "softball practice ", "create_at": date("2014-01-17"), "description": "HHS Senior | just a crazy blonde trying to have some fun", "followers_count": 1730, "friends_count": 467, "statues_count": 25600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hutchinson, KS", "id": "af29db2ab5fd16c8", "name": "Hutchinson", "place_type": "city", "bounding_box": rectangle("-97.966318,38.02387 -97.849296,38.108414") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20155, "countyName": "Reno", "cityID": 2033625, "cityName": "Hutchinson" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385607708673, "text": "Happy birthday baby boy I love you so much https://t.co/mv6HGUjrQf", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 531770625, "name": "lexi #STRONGOUTNOW", "screen_name": "warriortucker", "lang": "en", "location": "em, marie, dripping wes, PPFW", "create_at": date("2012-03-20"), "description": "making bad decisions is my middle name", "followers_count": 2994, "friends_count": 3393, "statues_count": 124997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385892892673, "text": "This is really pissing me off represent your country I'm still not over how he cost us game 6 and 7 https://t.co/KKSDSneIz2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 701574390, "name": "DET30-29", "screen_name": "saints10fan", "lang": "en", "location": "Ann Arbor ✈️ Tucson ✈️ Redding", "create_at": date("2012-07-17"), "description": "Michigan,Tigers and Saints Youtube TTC. Thunder. ArgentinaBrady fan. I love to discuss sports #freebrady. don't take all my tweets seriously I like to have fun.", "followers_count": 1659, "friends_count": 2026, "statues_count": 56432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Redding, CA", "id": "7d51c8dfbb08a9de", "name": "Redding", "place_type": "city", "bounding_box": rectangle("-122.452708,40.456935 -122.267398,40.651054") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6089, "countyName": "Shasta", "cityID": 659920, "cityName": "Redding" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525385951600640, "text": "�� https://t.co/WXNkW4LGkR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2581431096, "name": "joe kiel", "screen_name": "JoeeKiel", "lang": "en", "location": "San Diego Country Estates, CA", "create_at": date("2014-06-21"), "description": "sc: joekiel", "followers_count": 811, "friends_count": 317, "statues_count": 8378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386052395008, "text": "We all make mistakes ....", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 270672177, "name": "Fran☯", "screen_name": "francescaaa10", "lang": "en", "location": "Harbor Isle, NY", "create_at": date("2011-03-22"), "description": "Stop lurking bitch. #longlivemix❤️", "followers_count": 258, "friends_count": 353, "statues_count": 8889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oceanside, NY", "id": "39411d80c106aa80", "name": "Oceanside", "place_type": "city", "bounding_box": rectangle("-73.6611,40.612083 -73.616045,40.657308") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36059, "countyName": "Nassau", "cityID": 3654441, "cityName": "Oceanside" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386111127552, "text": "@hoodopulence I think I can die now...", "in_reply_to_status": 741525152370950144, "in_reply_to_user": 47766759, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 47766759 }}, "user": { "id": 2376994425, "name": "Young Nicki", "screen_name": "coozyminaj", "lang": "en", "location": "its Delaware (Connecticut) ", "create_at": date("2014-03-02"), "description": "I DON'T KNOW WHY THESE BITCHES EVER THOUGHT THEY WAS FUCKIN WITH ME!", "followers_count": 871, "friends_count": 806, "statues_count": 27297 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hartford, CT", "id": "61c225139f635563", "name": "Hartford", "place_type": "city", "bounding_box": rectangle("-72.718386,41.723759 -72.643547,41.807475") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937000, "cityName": "Hartford" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386400436224, "text": "@TheRealGrimmie what an absolutely cruel world we live in, she had no idea today would be her last. Prayers for her family & friends. #rip", "in_reply_to_status": 741387106921140224, "in_reply_to_user": 173210133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "rip" }}, "user_mentions": {{ 173210133 }}, "user": { "id": 302729049, "name": "Rita Williams", "screen_name": "ritaw07", "lang": "en", "location": "Covington, LA", "create_at": date("2011-05-21"), "description": "Travel enthusiast.", "followers_count": 103, "friends_count": 156, "statues_count": 1378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Abita Springs, LA", "id": "006cb77253c2c656", "name": "Abita Springs", "place_type": "city", "bounding_box": rectangle("-90.091235,30.434532 -90.019439,30.504184") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22103, "countyName": "St. Tammany", "cityID": 2200240, "cityName": "Abita Springs" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386438299648, "text": "❤️❤️❤️❤️ https://t.co/FZpXAZE4qI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 548109397, "name": "Samantha Caudill", "screen_name": "slcaudill27", "lang": "en", "location": "null", "create_at": date("2012-04-07"), "description": "null", "followers_count": 224, "friends_count": 152, "statues_count": 3921 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Heritage Creek, KY", "id": "0157e4d7264811f5", "name": "Heritage Creek", "place_type": "city", "bounding_box": rectangle("-85.644548,38.072047 -85.520944,38.143229") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2136102, "cityName": "Heritage Creek" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386891165696, "text": "still sliding in DM's but you \"taken\" GTFOH, making that lil girl look stupid, that dude down bad.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 417622203, "name": "black girl magic", "screen_name": "diamondbeauty_", "lang": "en", "location": "our nigga", "create_at": date("2011-11-20"), "description": "constantly making sure my crown isn't tilting. UL-Monroe", "followers_count": 1021, "friends_count": 761, "statues_count": 17604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alexandria, LA", "id": "c09ab6ee5a6f7b31", "name": "Alexandria", "place_type": "city", "bounding_box": rectangle("-92.57133,31.22783 -92.402313,31.35872") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22079, "countyName": "Rapides", "cityID": 2200975, "cityName": "Alexandria" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525386912096258, "text": "Lmaaaaaoo �������� https://t.co/1eTu3dQc37", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user": { "id": 2524086949, "name": "⚡️MoneyMakinMitch⚡️", "screen_name": "sbechicco", "lang": "en", "location": "chasingthesac", "create_at": date("2014-05-25"), "description": "RiP Darius, Alonzo & yung Mexico #NENOworld #GetRicH #03 NeN0 Band$", "followers_count": 1218, "friends_count": 670, "statues_count": 18447 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Katy, TX", "id": "c484d6c5cd72beb0", "name": "Katy", "place_type": "city", "bounding_box": rectangle("-95.87417,29.736773 -95.774936,29.831219") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4838476, "cityName": "Katy" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525387134435329, "text": "match����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1647414583, "name": "ProfesserFinesser", "screen_name": "SaulT_Nuts", "lang": "en", "location": "Carson", "create_at": date("2013-08-05"), "description": "single dad with no children", "followers_count": 286, "friends_count": 326, "statues_count": 17843 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525387289755648, "text": "Really really https://t.co/XBedmAD81u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1032955992, "name": "LEE", "screen_name": "lrichardson283", "lang": "en", "location": "GetMoney!!", "create_at": date("2012-12-24"), "description": "status !!", "followers_count": 484, "friends_count": 480, "statues_count": 3999 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525387407024128, "text": "Hey @RyanSeacrest, we love you for shouting out Gnash on national TV!!! But the G is silent. ;)", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 16190898 }}, "user": { "id": 630149109, "name": "Monica", "screen_name": "mlhudziak", "lang": "en", "location": "Santa Cruz, CA", "create_at": date("2012-07-08"), "description": "Music is my favorite drug.", "followers_count": 416, "friends_count": 410, "statues_count": 2770 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-11T00:00:03.000Z"), "id": 741525387427991553, "text": "& im going to keep that promise for as long as I live", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2405286594, "name": "blanca bee alexis ✨", "screen_name": "Beeethree_", "lang": "en", "location": "deez nutz", "create_at": date("2014-03-22"), "description": "null", "followers_count": 453, "friends_count": 499, "statues_count": 5281 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887759330082816, "text": "Disgusting bastard. https://t.co/v53JajSHGw", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 617320888, "name": "queen", "screen_name": "Laramieeee_", "lang": "en", "location": "null", "create_at": date("2012-06-24"), "description": "legal & be nothing but the best | rest easy daddy❤| with God is where I stand", "followers_count": 591, "friends_count": 363, "statues_count": 18303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887759418200064, "text": "Now I have to watch #OJMadeInAmerica ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "OJMadeInAmerica" }}, "user": { "id": 64024036, "name": "Kuy", "screen_name": "KUYANNAATL", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-08-08"), "description": "im the chick in the pic ---- no photoshop or booking info", "followers_count": 326, "friends_count": 284, "statues_count": 16303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887759732867072, "text": "@_mariee_ee huh whatV you can't understand mf?", "in_reply_to_status": 741887538714038272, "in_reply_to_user": 3040129857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3040129857 }}, "user": { "id": 348598016, "name": "ty ty✨", "screen_name": "wassupieasha", "lang": "en", "location": "Lynchburg, VA", "create_at": date("2011-08-04"), "description": "thick thighs, brown eyes, golden mind. virginiastateuniversitysophomore✨ rest in peace little brother, this is for you ❤️", "followers_count": 9414, "friends_count": 4820, "statues_count": 93604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynchburg, VA", "id": "2490949a68631669", "name": "Lynchburg", "place_type": "city", "bounding_box": rectangle("-79.257789,37.332496 -79.100237,37.469415") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51680, "countyName": "Lynchburg", "cityID": 5147672, "cityName": "Lynchburg" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887760168980480, "text": "@valwithcoffee that's literally what happened to me, and then i just started eating a LOT, for some reason??? boredom i think, tbh", "in_reply_to_status": 741887483189690368, "in_reply_to_user": 3099748021, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3099748021 }}, "user": { "id": 384486160, "name": "странник", "screen_name": "shining_nick", "lang": "en", "location": "johzenji high school ", "create_at": date("2011-10-03"), "description": "♒️/♉️/♓️ ♂ wannabe pop-pönk hater skater, finn reject. #TeamMadCraft, atl, hq!!, ut dallas student, mr. refreshing-senpai. tananoya is life, fight me", "followers_count": 816, "friends_count": 348, "statues_count": 50091 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887760332558336, "text": "All I see is shit about band kids��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1730608190, "name": "XAVI", "screen_name": "xavi_babey99", "lang": "en", "location": "Sun Valley, Los Angeles", "create_at": date("2013-09-04"), "description": "AV ➡️ SFV", "followers_count": 381, "friends_count": 481, "statues_count": 1988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887760915673088, "text": "keep holding ya nuts on me . ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1381315238, "name": "sunshine's rawest❗️", "screen_name": "__iamjazz", "lang": "en", "location": "in the bed with yo bitch .", "create_at": date("2013-04-26"), "description": "darryl toad videau❤️ | malcolm smoove calloway❤️ king$moove5️⃣", "followers_count": 1202, "friends_count": 769, "statues_count": 118382 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887761083338752, "text": "@rjaay17 @P4ul0_95 I meant this. It just came in from China. https://t.co/0ZnCLKXZKJ", "in_reply_to_status": 741887299974107136, "in_reply_to_user": 2858778900, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2858778900, 2311092979 }}, "user": { "id": 91644648, "name": "icy", "screen_name": "icewatsamon", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2009-11-21"), "description": "null", "followers_count": 370, "friends_count": 119, "statues_count": 16438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887761150418944, "text": "me rn �� https://t.co/faIAMiKUnk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 510754056, "name": "Serg", "screen_name": "Sergio_S1997", "lang": "en", "location": "Bullhead City, AZ", "create_at": date("2012-03-01"), "description": "⁶⁶⁶", "followers_count": 455, "friends_count": 402, "statues_count": 14804 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bullhead City, AZ", "id": "719e93b07f954b3a", "name": "Bullhead City", "place_type": "city", "bounding_box": rectangle("-114.644984,35.039554 -114.499729,35.191096") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave", "cityID": 408220, "cityName": "Bullhead City" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887761464987648, "text": "happy 19th birthday to my old ass, but lovable seastar �� love ya sometimes �� @ohmysapna", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 950997631 }}, "user": { "id": 952636932, "name": "Vandi", "screen_name": "vandehh", "lang": "en", "location": "CA", "create_at": date("2012-11-16"), "description": "null", "followers_count": 218, "friends_count": 176, "statues_count": 4790 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887761976860672, "text": "COME MANGO WITH US ����It's your One-Stop-Shop for all your Hookah/Accessory needs #mangoscafe… https://t.co/8LHwcM0Rtk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-83.0063171,39.9881401"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "mangoscafe" }}, "user": { "id": 383010151, "name": "Mango's Cafe", "screen_name": "MangosCafecbus", "lang": "en", "location": "21 Smith Place. Columbus, OHIO", "create_at": date("2011-09-30"), "description": "google us & check out what others have to say 6144293694 http://www.mangoscafe.orgus⬇️ https://www.facebook.com/pages/Mangos-Cafe/108546569254369", "followers_count": 10539, "friends_count": 502, "statues_count": 31001 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887762039611392, "text": "@SaltySeif in 7", "in_reply_to_status": 741849134471077888, "in_reply_to_user": 308835479, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 308835479 }}, "user": { "id": 579214508, "name": "CWright", "screen_name": "ConnorWright81", "lang": "en", "location": "Miami, FL - Dallas, Tx", "create_at": date("2012-05-13"), "description": "Emily Ratajkowski's boyfriend~", "followers_count": 302, "friends_count": 353, "statues_count": 12616 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frisco, TX", "id": "00c3e46c3d0cac08", "name": "Frisco", "place_type": "city", "bounding_box": rectangle("-96.937783,33.081206 -96.732715,33.219073") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4827684, "cityName": "Frisco" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887762098495488, "text": "@snipeyhead sucks to be at. :-/ I can only imagine since I've never made it that far up.", "in_reply_to_status": 741887427606781956, "in_reply_to_user": 14246782, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 14246782 }}, "user": { "id": 740920470, "name": "Tom McLaughlin", "screen_name": "tmclaughbos", "lang": "en", "location": "Boston, MA", "create_at": date("2012-08-06"), "description": "Twitter recognized #ThoughtFollower of #DevOps and stackable threats at @ThreatStack / ex-{@Jana,@HubSpot,@Vistaprint} / Tweets reflect views of your employer", "followers_count": 909, "friends_count": 1144, "statues_count": 8403 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Terramuggus, CT", "id": "008bd6306f834448", "name": "Terramuggus", "place_type": "city", "bounding_box": rectangle("-72.487405,41.609928 -72.432985,41.669037") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 975200, "cityName": "Terramuggus" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887762786222080, "text": "\"3 in da morning and you know I'm hornyyyy\" ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3032048445, "name": "Wick God ", "screen_name": "SwadeKartel", "lang": "en", "location": "Penrose. 215", "create_at": date("2015-02-11"), "description": "22. Free Rick$. RIP SABIR. #RIPMATT", "followers_count": 656, "friends_count": 655, "statues_count": 15981 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Philadelphia, PA", "id": "e4a0d228eb6be76b", "name": "Philadelphia", "place_type": "city", "bounding_box": rectangle("-75.280284,39.871811 -74.955712,40.13792") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42101, "countyName": "Philadelphia", "cityID": 4260000, "cityName": "Philadelphia" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887762899435521, "text": "������������������ https://t.co/b1w4FbBCRv", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1524205448, "name": "JAWS❌", "screen_name": "_erob_", "lang": "en", "location": "Wakanda", "create_at": date("2013-06-17"), "description": "Marie❤ Always keep your head up, because if it's down you won't be able to see the blessings that have been placed in your life.", "followers_count": 624, "friends_count": 481, "statues_count": 40410 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ruston, LA", "id": "35a6dd7f550504c4", "name": "Ruston", "place_type": "city", "bounding_box": rectangle("-92.695547,32.496312 -92.572122,32.577488") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22061, "countyName": "Lincoln", "cityID": 2266655, "cityName": "Ruston" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887762949771264, "text": "Liberian guys are so weird to me. I've never been attracted to one", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158230588, "name": "Nickey Hova", "screen_name": "_BlaccDiamond", "lang": "en", "location": "fufu & soup house ", "create_at": date("2010-06-21"), "description": "Instagram; @_blaccdiamond #Liberian Finding Euphoria", "followers_count": 447, "friends_count": 300, "statues_count": 35523 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shelton, CT", "id": "80831222cf5dd7ba", "name": "Shelton", "place_type": "city", "bounding_box": rectangle("-73.204315,41.255518 -73.066037,41.376652") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9001, "countyName": "Fairfield", "cityID": 968100, "cityName": "Shelton" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887763033690116, "text": "I wanna go bike riding on be beach tomorrow", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 258383608, "name": "adventure", "screen_name": "Caaali_kid", "lang": "en", "location": "Paris, France", "create_at": date("2011-02-27"), "description": "⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀⠀ ✖️ᴺᴵᴳᴴᵀ ᵀᴴᴵᴺᴷᴵᴺᴳ [&] ᴰᴬᵞ ᴰᴿᴱᴬᴹᴵᴺᴳ✖", "followers_count": 1307, "friends_count": 465, "statues_count": 98750 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887763100749825, "text": "I'm heading to bed. Church tmr https://t.co/Zv6e5qwHia", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2738988725, "name": "WarriorsBack2Back", "screen_name": "CurrySauce30", "lang": "en", "location": "null", "create_at": date("2014-08-10"), "description": "Love the Raiders the Golden StateWarriors .Hate Cyber bullies ,Cyber stalkers Here still by the grace of God. Also a liver Tumor Survivor! 2nd chance of life!", "followers_count": 1048, "friends_count": 1495, "statues_count": 15184 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Laguna Niguel, CA", "id": "ef28da43cdf17b3f", "name": "Laguna Niguel", "place_type": "city", "bounding_box": rectangle("-117.740002,33.487223 -117.672617,33.571675") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 639248, "cityName": "Laguna Niguel" } }
+{ "create_at": datetime("2016-06-12T00:00:00.000Z"), "id": 741887763138682880, "text": "Temp 56.0° Hi/Lo 56.5/55.6 Rng 0.9° WC 56.0° Hmd 97% Rain 0.00\" Storm 0.13\" BAR 29.636 Falling DP 55.2° Wnd 0mph Dir --- Gst 2mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19432 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763255939075, "text": "I looked hella cute today, wya kiah??", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2459257442, "name": "Ash V.", "screen_name": "uhhashlyyn_", "lang": "en", "location": "null", "create_at": date("2014-04-22"), "description": "null", "followers_count": 170, "friends_count": 195, "statues_count": 2158 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kapolei, HI", "id": "01932ef001ef7c53", "name": "Kapolei", "place_type": "city", "bounding_box": rectangle("-158.090772,21.323228 -158.048087,21.365894") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1530300, "cityName": "Kapolei" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763264372736, "text": "HAPPY BIRTHDAY BABES❤️�������� @nick_russellll @loyamariah6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1537951243, 3743915600 }}, "user": { "id": 2457389166, "name": "pyper olsen", "screen_name": "bean1892", "lang": "en", "location": "null", "create_at": date("2014-04-21"), "description": "mosiah 2:17 | be the change| MITAM | dubs nation| Brodster with us forever&always|", "followers_count": 304, "friends_count": 410, "statues_count": 1837 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clara, CA", "id": "4b58830723ec6371", "name": "Santa Clara", "place_type": "city", "bounding_box": rectangle("-122.005597,37.322943 -121.930045,37.419037") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 669084, "cityName": "Santa Clara" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763281260544, "text": "Until I come around & then it's all about mommy ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 322223248, "name": "bHs❤️", "screen_name": "BreciaHall_", "lang": "en", "location": "null", "create_at": date("2011-06-22"), "description": "Im not going to write a long bio , just hit the follow button .. Cailee is my worldddd , 3333", "followers_count": 459, "friends_count": 441, "statues_count": 6430 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rome, GA", "id": "11bb0fd5ce3937f8", "name": "Rome", "place_type": "city", "bounding_box": rectangle("-85.268995,34.199496 -85.11886,34.333995") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13115, "countyName": "Floyd", "cityID": 1366668, "cityName": "Rome" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763302219776, "text": "Holy shit .. This thunder and lighting is seriously starting to scare me.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2889251557, "name": "Christina", "screen_name": "Tinababe_02", "lang": "en", "location": "null", "create_at": date("2014-11-03"), "description": "Forget Those Who Forget You", "followers_count": 273, "friends_count": 209, "statues_count": 123 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763339870208, "text": "All I want is for somebody to make me feel safe.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 161153932, "name": "susan", "screen_name": "KcLynn17", "lang": "en", "location": "null", "create_at": date("2010-06-29"), "description": "I have a great life.", "followers_count": 546, "friends_count": 448, "statues_count": 14933 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lubbock, TX", "id": "3f3f6803f117606d", "name": "Lubbock", "place_type": "city", "bounding_box": rectangle("-102.033765,33.44712 -101.760581,33.693933") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48303, "countyName": "Lubbock", "cityID": 4845000, "cityName": "Lubbock" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763583094786, "text": "@ebbtideapp Tide in Port Eads, Louisiana 06/12/2016\nHigh 10:06am 0.7\n Low 9:38pm 0.4\nHigh 8:25am 0.8\n Low 6:44pm 0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-89.16,29.015"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 101, "friends_count": 1, "statues_count": 40687 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22075, "countyName": "Plaquemines" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763600035840, "text": "Wind 0.0 mph ---. Barometer 30.003 in, Falling slowly. Temperature 74.7 °F. Rain today 0.00 in. Humidity 67%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.89611111,39.82833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1251760374, "name": "47374WX", "screen_name": "47374WX", "lang": "en", "location": "Uptown - Richmond, Indiana", "create_at": date("2013-03-08"), "description": "Hourly weather data from the Wayne County Emergency Operations Center. A participating Citizen Weather Observer Program (CWOP) weather station. Follow @WayneEMA", "followers_count": 39, "friends_count": 4, "statues_count": 28102 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Richmond, IN", "id": "3684a9592c803ad1", "name": "Richmond", "place_type": "city", "bounding_box": rectangle("-84.976985,39.793313 -84.813667,39.88783") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18177, "countyName": "Wayne", "cityID": 1864260, "cityName": "Richmond" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763662790656, "text": "can we just appreciate the fact that EDC is in 5 DAYS NOW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2243318784, "name": "josh", "screen_name": "josh__mejia", "lang": "en", "location": "san diegooo", "create_at": date("2013-12-12"), "description": "okaaaaaaaaaaaaay", "followers_count": 361, "friends_count": 464, "statues_count": 8084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763906220032, "text": "Temp 60.9°F Heat Index 60.9 °F RH 91% Wind 0.0 --- Gust 0.0 --- SLP 30.029 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 62, "statues_count": 36108 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887763931365376, "text": "Temp: 77.9F W C: 77.9F Wind:SW at 4.3kts Baro: 1010.6mb and Steady Rain today: 0.00in R H: 72% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 216, "friends_count": 218, "statues_count": 107714 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887764401033221, "text": "So fucking annoyed man lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3273224792, "name": "Ashley Lynn Batson", "screen_name": "ashlleyllynn", "lang": "en", "location": "Burgaw, NC", "create_at": date("2015-07-09"), "description": "NC | SC:ashley.batson", "followers_count": 178, "friends_count": 137, "statues_count": 392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Silver Lake, NC", "id": "3972769ec1834def", "name": "Silver Lake", "place_type": "city", "bounding_box": rectangle("-77.923643,34.123378 -77.891335,34.175326") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37129, "countyName": "New Hanover", "cityID": 3761950, "cityName": "Silver Lake" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887764572954629, "text": "@emily_duke /by home I mean fw", "in_reply_to_status": 741887493474127872, "in_reply_to_user": 1351904737, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 370363840 }}, "user": { "id": 1351904737, "name": "peyton", "screen_name": "peytonmayfield1", "lang": "en", "location": "null", "create_at": date("2013-04-14"), "description": "McKinney | Fort Worth", "followers_count": 555, "friends_count": 349, "statues_count": 8889 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887764636045312, "text": "Wind 0.0 mph ---. Barometer 30.003 in, Steady. Temperature 75.6 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113902 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887764694597632, "text": "@Spencers u know what i'm not in love with\n\nthis shirt https://t.co/EEpWiKYsI6", "in_reply_to_status": 741449976480444417, "in_reply_to_user": 284611124, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 284611124 }}, "user": { "id": 331459601, "name": "kittie, CPPM", "screen_name": "chachachela", "lang": "en", "location": "null", "create_at": date("2011-07-07"), "description": "bad egg", "followers_count": 263, "friends_count": 1042, "statues_count": 4522 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Minneapolis, MN", "id": "8e9665cec9370f0f", "name": "Minneapolis", "place_type": "city", "bounding_box": rectangle("-93.329515,44.889964 -93.194578,45.051257") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2743000, "cityName": "Minneapolis" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887764988190720, "text": "HAPPY BIETHDAY @SamGorman7 !!!!!!!!! wooooo!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 312299115 }}, "user": { "id": 351370999, "name": "dndréw♛", "screen_name": "Ga1anos", "lang": "en", "location": "wherever she is", "create_at": date("2011-08-08"), "description": "21 | NJ | girls don't like me", "followers_count": 3773, "friends_count": 338, "statues_count": 137086 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anaheim, CA", "id": "0c2e6999105f8070", "name": "Anaheim", "place_type": "city", "bounding_box": rectangle("-118.017789,33.788913 -117.684389,33.896813") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 602000, "cityName": "Anaheim" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887765118373888, "text": "Real talk, @waze over everything...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 31171669 }}, "user": { "id": 700903452, "name": "Only In Boston", "screen_name": "OnlyInBOS", "lang": "en", "location": "Boston, MA", "create_at": date("2012-07-17"), "description": "Informational. Genuine. Local. Versatile. \nOnlyInBOS[at]gmail[dot]com.", "followers_count": 160457, "friends_count": 712, "statues_count": 45095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boston, MA", "id": "67b98f17fdcf20be", "name": "Boston", "place_type": "city", "bounding_box": rectangle("-71.191421,42.227797 -70.986004,42.399542") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25025, "countyName": "Suffolk", "cityID": 2507000, "cityName": "Boston" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887765395210240, "text": "Howell, NJ | Wind 0.0 mph ---. Baro 29.724 in, Falling. Temp 71.9F. Rain today 0.00 in. Humidity 87% | https://t.co/jrBhN6rgAc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-74.21444444,40.13027778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1315476338, "name": "John Napoli", "screen_name": "John_N_Wx", "lang": "en", "location": "Howell, NJ", "create_at": date("2013-03-29"), "description": "Not just for weather. I also tweet about @NHLDevils, and whatever else is on my mind. 60 min. updates, more w/ severe wx.. #NJD", "followers_count": 632, "friends_count": 846, "statues_count": 43910 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Howell, NJ", "id": "0165d65c46e85631", "name": "Howell", "place_type": "city", "bounding_box": rectangle("-74.258193,40.106536 -74.160306,40.17997") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34029, "countyName": "Ocean" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887766087274496, "text": "06/12@03:00 - Temp 77.8F, WC 77.8F. Wind 1.0mph SW, Gust 3.0mph. Bar 29.764in, Falling slowly. Rain 0.00in. Hum 70%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49614 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-12T00:00:01.000Z"), "id": 741887766716239872, "text": "next week iba na naman kinaaadikan kong kanta jusq", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 1690868456, "name": "kyla", "screen_name": "iamkylanicoleee", "lang": "en", "location": "US", "create_at": date("2013-08-22"), "description": "cartophile | ed sheeran | \ni only see my goals, i don't believe in failure.", "followers_count": 1170, "friends_count": 1125, "statues_count": 28857 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Glendive, MT", "id": "d3d8676cff3e0882", "name": "Glendive", "place_type": "city", "bounding_box": rectangle("-104.739652,47.083934 -104.676843,47.136719") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30021, "countyName": "Dawson", "cityID": 3031450, "cityName": "Glendive" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887767412494337, "text": "ok but everytime i see a shadowhunters gifset w/fuckin.... clary and jace makin out... stop", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 577820406, "name": "nate", "screen_name": "grantears", "lang": "en", "location": "oregon", "create_at": date("2012-05-11"), "description": "u can call me reid, nathaniel, or nate | melancholia and marxism | he/him/his pronouns | a 23 yr old mostly gay trans fella | private account: @inneffably", "followers_count": 206, "friends_count": 121, "statues_count": 45284 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, OR", "id": "ac88a4f17a51c7fc", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-122.790065,45.421863 -122.471751,45.650941") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41051, "countyName": "Multnomah", "cityID": 4159000, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887767420928000, "text": "Growth is painful. Change is painful. But nothing is as painful as staying stuck somewhere you don't belong.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 305227817, "name": "Giselle", "screen_name": "MissGiselle_H", "lang": "en", "location": "Chandler, AZ", "create_at": date("2011-05-25"), "description": "Too blessed to be stressed", "followers_count": 439, "friends_count": 329, "statues_count": 17652 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887767441989632, "text": "Ripley SW Limestone Co. Temp: 76.1°F Wind:2.2mph Pressure: 998.8mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 43, "friends_count": 33, "statues_count": 54406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887767857094657, "text": "Ooooo that was a big one ⛈", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 367439680, "name": "eduardo", "screen_name": "Eddie3Stacks", "lang": "en", "location": "Porterville, CA", "create_at": date("2011-09-03"), "description": "⚽️⚾️ sc: eddie3stacks", "followers_count": 258, "friends_count": 479, "statues_count": 7524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Porterville, CA", "id": "daa5fcec75a430ae", "name": "Porterville", "place_type": "city", "bounding_box": rectangle("-119.080552,36.029555 -118.977713,36.119995") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 658240, "cityName": "Porterville" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887767890694144, "text": "@ me", "in_reply_to_status": 741887723317821441, "in_reply_to_user": 1426505736, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1426505736, "name": "Moni", "screen_name": "nmniii", "lang": "en", "location": "Sierra Leone, Africa ", "create_at": date("2013-05-13"), "description": "Newark Nj | 757\nIG: nmniii", "followers_count": 327, "friends_count": 226, "statues_count": 23854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chesapeake, VA", "id": "73d1c1c11b675932", "name": "Chesapeake", "place_type": "city", "bounding_box": rectangle("-76.448383,36.66 -76.143408,36.866431") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51550, "countyName": "Chesapeake", "cityID": 5116000, "cityName": "Chesapeake" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887768079392768, "text": "Temp: 75.3°F Wind:0.0mph Pressure: 30.090hpa Falling Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66827 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887768272510976, "text": "Wind 0.0 mph ---. Barometer 30.019 in, Falling slowly. Temperature 69.0 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 26, "statues_count": 20500 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887768675115009, "text": "2 in da morning my mind is on you ..", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 170515832, "name": "RYANDAT❤️", "screen_name": "_kyanaaaa", "lang": "en", "location": "Maryland, USA", "create_at": date("2010-07-24"), "description": "Zion❤️", "followers_count": 1481, "friends_count": 763, "statues_count": 132257 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Walker Mill, MD", "id": "2af60542479207ed", "name": "Walker Mill", "place_type": "city", "bounding_box": rectangle("-76.914855,38.854922 -76.858562,38.889767") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24033, "countyName": "Prince George's", "cityID": 2481250, "cityName": "Walker Mill" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887768696135680, "text": "Some boys don't see it this way ... https://t.co/ZzdXwZDIRx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2346405661, "name": "Mariela", "screen_name": "Marielactually", "lang": "en", "location": "Fort Worth, TX", "create_at": date("2014-02-15"), "description": "softball is my happiness", "followers_count": 244, "friends_count": 176, "statues_count": 7288 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887768868065280, "text": "Lmfao goku took a senzu bean and it reminded me of this https://t.co/7x84Ec2R1N", "in_reply_to_status": 741886213070696448, "in_reply_to_user": 2534870847, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2534870847, "name": "Lino", "screen_name": "lighthousing_", "lang": "en", "location": "College Station, TX", "create_at": date("2014-05-08"), "description": "Poet. Sleeper. I want to wear more neutral colors. You know all the good words. Chief Operations Officer and Co-founder of Fifteen13 Press.", "followers_count": 924, "friends_count": 667, "statues_count": 30350 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carrboro, NC", "id": "c3a90b8ce84f216e", "name": "Carrboro", "place_type": "city", "bounding_box": rectangle("-79.109676,35.889723 -79.063905,35.964692") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37135, "countyName": "Orange", "cityID": 3710620, "cityName": "Carrboro" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887769648173056, "text": "���� can't sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 116049861, "name": "✨Louie ✨", "screen_name": "louTieT", "lang": "en", "location": "Brooklyn, New York ", "create_at": date("2010-02-20"), "description": "Good for nothing but looking good. http://Instagram.com/loutiet", "followers_count": 1437, "friends_count": 358, "statues_count": 25014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887769702629376, "text": "I got church", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2547189114, "name": "Slutty 佳", "screen_name": "jonay_matthews", "lang": "en", "location": "null", "create_at": date("2014-06-04"), "description": "don't like you broke hos/nigs", "followers_count": 923, "friends_count": 594, "statues_count": 17809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887769874685952, "text": "79.7F (Feels: 87.8F) - Humidity: 90% - Wind: 0.4mph SSW - Gust: 0.0mph - Pressure: 30.083in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 242147 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887770042322945, "text": "@_sydnnay I do", "in_reply_to_status": 741886088336150528, "in_reply_to_user": 2540412805, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2540412805 }}, "user": { "id": 2455935608, "name": "mar j", "screen_name": "marilyn_jaquez", "lang": "en", "location": "null", "create_at": date("2014-04-20"), "description": "chipotle enthusiast", "followers_count": 997, "friends_count": 685, "statues_count": 13190 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887770327580672, "text": "♫ Rocking at Murphy's Pub : https://t.co/I4fRA0rR7U @GetRockbot", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.404114,37.790188"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 91192339 }}, "user": { "id": 21965039, "name": "Tushar Dwivedi", "screen_name": "tushartushar", "lang": "en", "location": "California", "create_at": date("2009-02-25"), "description": "Learn to Teach. Teach to Learn.", "followers_count": 203, "friends_count": 553, "statues_count": 380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887770474352640, "text": "thank you. https://t.co/jOYtZbJ8Fi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1049025684, "name": "❊ alicia ❊", "screen_name": "_AliciiaRenee", "lang": "en", "location": "outta dis world", "create_at": date("2012-12-30"), "description": "The most important thing is to enjoy your life - to be happy - it's all that matters.", "followers_count": 1156, "friends_count": 779, "statues_count": 80686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887770566623233, "text": "@ellie_rae_ https://t.co/f0ENJvo5RR", "in_reply_to_status": -1, "in_reply_to_user": 637102461, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 637102461 }}, "user": { "id": 176652775, "name": "✨kikkowoman✨", "screen_name": "not_rosemary", "lang": "en", "location": "the woods", "create_at": date("2010-08-09"), "description": "don't follow me I'm not funny ✨ SC/insta: not_rosemary", "followers_count": 67, "friends_count": 145, "statues_count": 2048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ashland, OR", "id": "387221f842ada634", "name": "Ashland", "place_type": "city", "bounding_box": rectangle("-122.736467,42.167149 -122.650854,42.216145") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4103050, "cityName": "Ashland" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887771099463682, "text": "Facts ���� https://t.co/ESJi8jpnvo", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2873234561, "name": "ROSS ✨", "screen_name": "51_ross", "lang": "en", "location": "Greensboro, NC", "create_at": date("2014-11-11"), "description": "#JasCityEnt ✨x #NCAT17 x Scorpio ♏️ x Taken❣", "followers_count": 2118, "friends_count": 1472, "statues_count": 44865 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Greensboro, NC", "id": "a6c257c61f294ec1", "name": "Greensboro", "place_type": "city", "bounding_box": rectangle("-80.029518,35.962623 -79.685209,36.218171") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37081, "countyName": "Guilford", "cityID": 3728000, "cityName": "Greensboro" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887771334311936, "text": "@BernieSanders @HillaryClinton https://t.co/kXSoOEC68j", "in_reply_to_status": -1, "in_reply_to_user": 216776631, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 216776631, 1339835893 }}, "user": { "id": 3330052767, "name": "PrezBernieSanders", "screen_name": "SoaringRhetoric", "lang": "en", "location": "United States", "create_at": date("2015-06-16"), "description": "Bernie Sanders supporter. No affiliation with the Sanders campaign.", "followers_count": 4983, "friends_count": 2003, "statues_count": 14600 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Portland, ME", "id": "34b5e468ea4e8110", "name": "Portland", "place_type": "city", "bounding_box": rectangle("-70.346323,43.640514 -70.238767,43.75186") }, "geo_tag": { "stateID": 23, "stateName": "Maine", "countyID": 23005, "countyName": "Cumberland", "cityID": 2360545, "cityName": "Portland" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887771338366976, "text": "Wind 0.0 mph ---. Barometer 30.13 in, Steady. Temperature 75.0 °F. Rain today 0.00 in. Humidity 93%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23303 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-12T00:00:02.000Z"), "id": 741887771338493953, "text": "ITS3AM I MUST BE LONELYYYYY!!!!!!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 304707032, "name": "Madi Task", "screen_name": "alwaysoff_TASK", "lang": "en", "location": "The Ohio State University", "create_at": date("2011-05-24"), "description": "I am a nut || ALPHACHIp(OTL)e", "followers_count": 1174, "friends_count": 1043, "statues_count": 24019 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887771552317441, "text": "Oh this is distasteful https://t.co/PA785Y8Chg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2750362898, "name": "juice", "screen_name": "ColeyApsay1", "lang": "en", "location": "null", "create_at": date("2014-08-20"), "description": "| Angela Hauser | On the Pursuit of Happiness | Hooper at SFSU |", "followers_count": 452, "friends_count": 480, "statues_count": 6988 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887771695042560, "text": "@__Lobbins yes sir", "in_reply_to_status": 741887680489742336, "in_reply_to_user": 1540039926, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1540039926 }}, "user": { "id": 2264751492, "name": "T A C O ✨", "screen_name": "901tacogirl", "lang": "en", "location": "memphis tn ", "create_at": date("2013-12-27"), "description": "GOD FIRST| rest easy Dennis| doιт4dəz| #TSU20", "followers_count": 1831, "friends_count": 1088, "statues_count": 36062 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Memphis, TN", "id": "f995a9bd45d4a867", "name": "Memphis", "place_type": "city", "bounding_box": rectangle("-90.135782,34.994192 -89.708276,35.272849") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47157, "countyName": "Shelby", "cityID": 4748000, "cityName": "Memphis" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887772294709248, "text": "Bouncin me off floor boards @CivicNola tonite @stlucia killing it! The karma was goooood... https://t.co/K2B97oTQwd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.074337,29.949474"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 989831803, 293064580 }}, "user": { "id": 634137521, "name": "Goodkarma Brown", "screen_name": "rogerstan12345", "lang": "en", "location": "null", "create_at": date("2012-07-12"), "description": "Spirit of a gypsy Open Mind Free thinking living life as she sees desires bringing only love and good karma to all she encounters....", "followers_count": 30, "friends_count": 66, "statues_count": 531 }, "place": { "country": "United States", "country_code": "United States", "full_name": "The Civic Theatre", "id": "07d9cf7aba081002", "name": "The Civic Theatre", "place_type": "poi", "bounding_box": rectangle("-90.0743371,29.949473899999997 -90.074337,29.949474") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887772324040704, "text": "@artnula haha nothing want to go to hollywood RN?", "in_reply_to_status": 741887595991339013, "in_reply_to_user": 2981853007, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2981853007 }}, "user": { "id": 382282986, "name": "Sithyna?", "screen_name": "OriginalSithyna", "lang": "en", "location": "null", "create_at": date("2011-09-29"), "description": "you're not down", "followers_count": 180, "friends_count": 112, "statues_count": 3836 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887772441481217, "text": "some people are so fucking SHADY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2216173909, "name": "heavenlee", "screen_name": "_heavenleep", "lang": "en", "location": "Riverside, CA", "create_at": date("2013-11-26"), "description": "phs", "followers_count": 753, "friends_count": 382, "statues_count": 12677 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887772680540161, "text": "I fucks with j cole heavy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 256744668, "name": "tarafried", "screen_name": "TaraMckaskle", "lang": "en", "location": " atx. dtx", "create_at": date("2011-02-23"), "description": "camping in disco. sin city. psalms 147:3 dd. btt. kitty tatt. smokechains. bassfairy. eat trash be free [ò...ó]", "followers_count": 818, "friends_count": 777, "statues_count": 14772 }, "place": { "country": "United States", "country_code": "United States", "full_name": "McKinney, TX", "id": "7eeed20da9c42064", "name": "McKinney", "place_type": "city", "bounding_box": rectangle("-96.803319,33.137357 -96.595889,33.251946") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48085, "countyName": "Collin", "cityID": 4845744, "cityName": "McKinney" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887772806369280, "text": "@xaiver18 you already know I love every drake song from the beginning", "in_reply_to_status": 741385046817411072, "in_reply_to_user": 323432360, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 323432360 }}, "user": { "id": 2163066734, "name": "Brooke ☼", "screen_name": "bfarris18", "lang": "en", "location": "Oklahoma City, OK", "create_at": date("2013-10-29"), "description": "null", "followers_count": 1514, "friends_count": 747, "statues_count": 15997 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oklahoma City, OK", "id": "9531d4e3bbafc09d", "name": "Oklahoma City", "place_type": "city", "bounding_box": rectangle("-97.768683,35.290722 -97.335639,35.674965") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40109, "countyName": "Oklahoma", "cityID": 4055000, "cityName": "Oklahoma City" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773108379648, "text": "its 2 o'clock its 2 o'clock dont be tryna smell yo socks ! go take a bath��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2374494822, "name": "❤️", "screen_name": "neeuqqsaucyy", "lang": "en", "location": "New Orleans, LA", "create_at": date("2014-03-05"), "description": "living life✨!", "followers_count": 457, "friends_count": 258, "statues_count": 18710 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773163040768, "text": "Wind 0.9 mph NW. Barometer 29.485 in, Falling Rapidly. Temperature 57.1 °F. Rain today 0.00 in. Humidity 99%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773427130368, "text": "OH YES SLAY ������������ https://t.co/1sfW19JMo0", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1561804128, "name": "maris♕", "screen_name": "marissaleave", "lang": "en", "location": "west coast fame", "create_at": date("2013-07-01"), "description": "west coast fame lady legends ⚡️ ERHS varsity cheer✨", "followers_count": 513, "friends_count": 711, "statues_count": 1635 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pico Rivera, CA", "id": "e3505917a639e8ec", "name": "Pico Rivera", "place_type": "city", "bounding_box": rectangle("-118.123005,33.950457 -118.037043,34.037384") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 656924, "cityName": "Pico Rivera" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773494247424, "text": "HAPPY BDAY BANDRES LUV U LIL UGLYASS NIGGA ������ @barragan_gm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1247480276 }}, "user": { "id": 2885704076, "name": "azucar papí", "screen_name": "LexyPoitan", "lang": "en", "location": "null", "create_at": date("2014-10-31"), "description": "null", "followers_count": 225, "friends_count": 156, "statues_count": 3796 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Willowbrook, CA", "id": "7df6f50f15138f28", "name": "Willowbrook", "place_type": "city", "bounding_box": rectangle("-118.282262,33.901902 -118.222378,33.929527") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 685614, "cityName": "Willowbrook" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773544615937, "text": "The good news is Nigga, you've cam along way. Bad news is nigga you went the wrong way.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2305741099, "name": "ëdēñ", "screen_name": "_moonnchildd_", "lang": "en", "location": "San Francisco, CA", "create_at": date("2014-01-22"), "description": "My name is Eden, but you can call me Cheese Master Flex | My attitude is kinda savage but my heart is made of gold ☄ | Virgo |", "followers_count": 409, "friends_count": 293, "statues_count": 7850 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773871734784, "text": "HAPPY 16TH BIRTHDAY VINCE!!!❤️�� i love you dude have a great day!!!����❤️ https://t.co/cdUcgt3nj7", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2598864750, "name": "Shelbs", "screen_name": "shelbi_lynne22", "lang": "en", "location": "null", "create_at": date("2014-07-01"), "description": "WFHS//2018", "followers_count": 490, "friends_count": 433, "statues_count": 3219 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mesa, AZ", "id": "44d207663001f00b", "name": "Mesa", "place_type": "city", "bounding_box": rectangle("-111.894548,33.306275 -111.580583,33.505234") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 446000, "cityName": "Mesa" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773892694016, "text": "مو كنتو نايمين له؟", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "ar", "is_retweet": false, "user": { "id": 2267236347, "name": "AJ", "screen_name": "LeeShadyy", "lang": "ar", "location": "ٓ", "create_at": date("2014-01-07"), "description": "Qatif • 1999 ♋️ http://sayat.me/LeeShadyy", "followers_count": 1001, "friends_count": 266, "statues_count": 39536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Eagan, MN", "id": "5d441fd5487fc133", "name": "Eagan", "place_type": "city", "bounding_box": rectangle("-93.228436,44.774013 -93.104796,44.862942") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27037, "countyName": "Dakota", "cityID": 2717288, "cityName": "Eagan" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773951569921, "text": "I just put a full face of makeup for nothing LOL ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1972519279, "name": "Nadia", "screen_name": "NadiaOliva8", "lang": "en", "location": "Lockhart ✈ ", "create_at": date("2013-10-19"), "description": "C/o'16|sc:Nadiaaa8", "followers_count": 716, "friends_count": 602, "statues_count": 11389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773980819457, "text": "@andthreecorgis @b_troop", "in_reply_to_status": 741813519239106560, "in_reply_to_user": 367195444, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 367195444, 348905326 }}, "user": { "id": 2970343797, "name": "Anyra Lash", "screen_name": "anyralash", "lang": "en", "location": "null", "create_at": date("2015-01-09"), "description": "BMT", "followers_count": 247, "friends_count": 200, "statues_count": 1479 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Destin, FL", "id": "18a461be4c0c2033", "name": "Destin", "place_type": "city", "bounding_box": rectangle("-86.515771,30.37865 -86.39721,30.417706") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12091, "countyName": "Okaloosa", "cityID": 1217325, "cityName": "Destin" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887773989298176, "text": "Yup https://t.co/UMwVJ74ffL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2312205481, "name": "Will", "screen_name": "WHeidvogel", "lang": "en", "location": "null", "create_at": date("2014-01-26"), "description": "null", "followers_count": 374, "friends_count": 399, "statues_count": 782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oconomowoc, WI", "id": "f607276df80fb5f1", "name": "Oconomowoc", "place_type": "city", "bounding_box": rectangle("-88.54,43.068737 -88.451607,43.14062") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55133, "countyName": "Waukesha", "cityID": 5559250, "cityName": "Oconomowoc" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887774236647430, "text": "When Charli performed Fancy & I rapped along Iggy's verses word for word @HarajukuKenDee_ was next to me SHOOK!!! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 257571886 }}, "user": { "id": 32644914, "name": "andrew.", "screen_name": "YonceHooker", "lang": "en", "location": "null", "create_at": date("2009-04-17"), "description": "Innocence lost...", "followers_count": 1826, "friends_count": 617, "statues_count": 76177 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Hollywood, CA", "id": "1927193c57f35d51", "name": "West Hollywood", "place_type": "city", "bounding_box": rectangle("-118.395904,34.075963 -118.343386,34.098056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684410, "cityName": "West Hollywood" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887774593191936, "text": "King krule to help w being sad", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1604540923, "name": "anita", "screen_name": "ana_grrrl", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2013-07-18"), "description": "emmanuel is my cinnamon apple // AHS ⚽️", "followers_count": 405, "friends_count": 420, "statues_count": 23514 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887774656102401, "text": "Wind 4.8 mph ESE. Barometer 29.975 in, Steady. Temperature 77.2 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 53, "statues_count": 9113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887774786080768, "text": "It's almost a quarter after one. 6/12/16, now!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1239962077, "name": "Jimmie", "screen_name": "huskies5876", "lang": "en", "location": "United States", "create_at": date("2013-03-03"), "description": "#Intuitive, love the arts. #Heal the World!", "followers_count": 4491, "friends_count": 5001, "statues_count": 67314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cedar City, UT", "id": "013884df639610f8", "name": "Cedar City", "place_type": "city", "bounding_box": rectangle("-113.152491,37.647433 -113.032184,37.736012") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49021, "countyName": "Iron", "cityID": 4911320, "cityName": "Cedar City" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887775381716992, "text": "@chinarise thank you ☺️��", "in_reply_to_status": 741883877568827393, "in_reply_to_user": 727418704019886080, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 727418704019886080 }}, "user": { "id": 2680412366, "name": "13", "screen_name": "gl0hen", "lang": "en", "location": "Prairie View · Dallas ", "create_at": date("2014-07-25"), "description": "21 • Libra • 1/2 Pimp, 1/2 Simp • Graphic Designer • Call me Ebony or Miss Slowed + Gl0'd. Yeezy taught me well. Fly High, Newt. IG: gl0hen ❂☯☮", "followers_count": 2059, "friends_count": 960, "statues_count": 121831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Prairie View, TX", "id": "bdd24fa1c8fe4b6a", "name": "Prairie View", "place_type": "city", "bounding_box": rectangle("-96.015096,30.061706 -95.965255,30.10461") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48473, "countyName": "Waller", "cityID": 4859336, "cityName": "Prairie View" } }
+{ "create_at": datetime("2016-06-12T00:00:03.000Z"), "id": 741887775578824704, "text": "when's nirvana going to release a new album?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3168093310, "name": "alli.✖️", "screen_name": "allitaliaferro_", "lang": "en", "location": "zoning out in slumberland", "create_at": date("2015-04-14"), "description": "these people are fascinated by me, but I haven't done anything.", "followers_count": 439, "friends_count": 305, "statues_count": 1580 }, "place": { "country": "United States", "country_code": "United States", "full_name": "White Oak, TX", "id": "f3c85c96edc1512b", "name": "White Oak", "place_type": "city", "bounding_box": rectangle("-94.877354,32.497841 -94.826133,32.576184") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48183, "countyName": "Gregg", "cityID": 4878436, "cityName": "White Oak" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776111497216, "text": "Just have to remind myself of why I'm here & try not to get sidetracked by bullshit", "in_reply_to_status": 741887664039727104, "in_reply_to_user": 272760374, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 272760374, "name": "Adaj", "screen_name": "ChicaExtrana_", "lang": "en", "location": " Seattle ✈️ Los Angeles ", "create_at": date("2011-03-26"), "description": "♀ Independent. | FIDM LA #TeamJadeforever", "followers_count": 662, "friends_count": 630, "statues_count": 18735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776421904384, "text": "I love you ���� https://t.co/MVOaYgwQIm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2708178629, "name": "Paigeypoo", "screen_name": "paige_unihorn", "lang": "en", "location": "null", "create_at": date("2014-07-13"), "description": "life isn't about what you gain, it's about what you give .....sc: paige_unihorn", "followers_count": 714, "friends_count": 673, "statues_count": 9670 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hesperia, CA", "id": "d52c2a8e878b7cf7", "name": "Hesperia", "place_type": "city", "bounding_box": rectangle("-117.400338,34.366195 -117.23785,34.471356") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 633434, "cityName": "Hesperia" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776447139842, "text": "@PoisonTeaa ���� lmao", "in_reply_to_status": 741882533713940480, "in_reply_to_user": 67650615, "favorite_count": 0, "retweet_count": 0, "lang": "ht", "is_retweet": false, "user_mentions": {{ 67650615 }}, "user": { "id": 3278952552, "name": "LIL LoneRanger", "screen_name": "nigelgotdajuice", "lang": "en", "location": "Earth", "create_at": date("2015-07-13"), "description": "https://m.soundcloud.com/nacona-collins-1/sets/red-bird-why-do-wolves-howl", "followers_count": 356, "friends_count": 272, "statues_count": 5854 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mobile, AL", "id": "d049033410e9e81b", "name": "Mobile", "place_type": "city", "bounding_box": rectangle("-88.301598,30.523874 -88.021513,30.843424") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1097, "countyName": "Mobile", "cityID": 150000, "cityName": "Mobile" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776489013248, "text": "You have no booty gtf https://t.co/j9rbdCjm9R", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4658346271, "name": "Ugly Antho", "screen_name": "yt__antho", "lang": "en", "location": "null", "create_at": date("2015-12-26"), "description": "Houston's Finest", "followers_count": 2249, "friends_count": 1360, "statues_count": 15639 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776505860097, "text": "\"@brittlovedj_: Stack,Save,Stay out the way��\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2807566252 }}, "user": { "id": 2721118985, "name": "Johnny Davis", "screen_name": "JohnnyJdogg", "lang": "en", "location": "null", "create_at": date("2014-07-22"), "description": "Whats good Twitter world, my name is Johnny L. Davis, im 28 years old born in Orangeburg, South Carolina on 02/13/1987. A small country town S of Columbia.", "followers_count": 86, "friends_count": 1742, "statues_count": 41 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisville, KY", "id": "095534ad3107e0e6", "name": "Louisville", "place_type": "city", "bounding_box": rectangle("-85.847503,38.108678 -85.597188,38.282432") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21111, "countyName": "Jefferson", "cityID": 2148000, "cityName": "Louisville" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776618995712, "text": "i really listen to like every genre of music", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2366427919, "name": "kaylaaa", "screen_name": "kaylamiichellle", "lang": "en", "location": "w arianna and @mairnda", "create_at": date("2014-02-28"), "description": "I ❤️ @praise_yeezus", "followers_count": 769, "friends_count": 502, "statues_count": 55606 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887776828719104, "text": "I got you.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 131414041, "name": "Angel", "screen_name": "akjheart", "lang": "en", "location": "College", "create_at": date("2010-04-09"), "description": "18 - Health and Fitness lover", "followers_count": 2306, "friends_count": 204, "statues_count": 27880 }, "place": { "country": "United States", "country_code": "United States", "full_name": "West Covina, CA", "id": "1193edda55d6fa3d", "name": "West Covina", "place_type": "city", "bounding_box": rectangle("-117.967313,34.001861 -117.826011,34.092498") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684200, "cityName": "West Covina" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887777109729280, "text": "Watch a nigga bless u", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3252731909, "name": "Triple H", "screen_name": "BigMamaHalz", "lang": "en", "location": "South Carolina, USA", "create_at": date("2015-05-13"), "description": "Snap: Hal.elaine insta: bigmamahalz", "followers_count": 625, "friends_count": 360, "statues_count": 14886 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clemson, SC", "id": "01ff1adb1d047095", "name": "Clemson", "place_type": "city", "bounding_box": rectangle("-82.853949,34.658367 -82.776522,34.729476") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45077, "countyName": "Pickens", "cityID": 4514950, "cityName": "Clemson" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887777193611264, "text": "HAPPY 17th BIRTHDAY MARIAH!! I LOVE YOU!!! ❤️ @MariahDECA https://t.co/bMGpspZuEk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3163162952 }}, "user": { "id": 983582264, "name": "Adrianna Turner", "screen_name": "Adrianna_T98", "lang": "en", "location": "null", "create_at": date("2012-12-01"), "description": "Gilbert High School Senior '17 snapchat:adriannalovesyo", "followers_count": 456, "friends_count": 355, "statues_count": 2397 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887777457872896, "text": "�������� I almost ate your butt... But you cool Asf and need to post more music videos like rn as soon as you read this", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142869753, "name": "CARAMEL FRAP✨", "screen_name": "PICK_N1CK", "lang": "en", "location": "criptorville", "create_at": date("2010-05-11"), "description": "Proud father of 2, 4th on the way! I'm 17 years young maybe older than that idk how to tell. #teamvirgin #respectwomen2k16 #neverpullout #keepitjiggy", "followers_count": 644, "friends_count": 471, "statues_count": 43008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Victorville, CA", "id": "5daa0c559e468ed3", "name": "Victorville", "place_type": "city", "bounding_box": rectangle("-117.47182,34.437542 -117.254315,34.624902") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 682590, "cityName": "Victorville" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887777692868608, "text": "success", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 261494952, "name": "•NRS•", "screen_name": "nicolerosesuch", "lang": "en", "location": "NJ~PA ", "create_at": date("2011-03-05"), "description": "Albright2017 alpha delta pi", "followers_count": 349, "friends_count": 394, "statues_count": 10247 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Blackwood, NJ", "id": "0036534d6f287a14", "name": "Blackwood", "place_type": "city", "bounding_box": rectangle("-75.088811,39.726424 -74.968468,39.852636") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3406040, "cityName": "Blackwood" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887777927614464, "text": "@jorge_chanax SAME ❣", "in_reply_to_status": 741887491708321793, "in_reply_to_user": 3753663433, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3753663433 }}, "user": { "id": 517145726, "name": "❁ ash. j ☾", "screen_name": "J_Bubblebutt", "lang": "en", "location": "null", "create_at": date("2012-03-06"), "description": "dci • wgi", "followers_count": 470, "friends_count": 464, "statues_count": 20014 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Contra Costa Centre, CA", "id": "23f58100788c0a1d", "name": "Contra Costa Centre", "place_type": "city", "bounding_box": rectangle("-122.060266,37.917118 -122.046695,37.938637") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 616090, "cityName": "Contra Costa Centre" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887778133319680, "text": "Wind 0.0 mph WNW. Barometer 29.60 in, Falling. Temperature 56.7 °F. Rain today 0.00 in. Humidity 97%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 123, "statues_count": 160480 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887778326093824, "text": "My sweet beautiful friend married the love of her life today. I'm still in awe of these two and… https://t.co/OqrytJR8qL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.728,45.8153"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 366761716, "name": "Cat Dossett", "screen_name": "CatDossett", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2011-09-02"), "description": "My name is Cat and I Take Photos.\n#alaskangrownoregonhome\n#cattakesphotos", "followers_count": 93, "friends_count": 112, "statues_count": 1049 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ridgefield, WA", "id": "01c8d43717ff966e", "name": "Ridgefield", "place_type": "city", "bounding_box": rectangle("-122.756036,45.794707 -122.702709,45.826301") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5358410, "cityName": "Ridgefield" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887778661797888, "text": "I'm never gonna walk through no snow storm for no dick. That's fucking crazy.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29902828, "name": "Stevie not Steebie ", "screen_name": "StevieSoFetch_", "lang": "en", "location": "LONG ISLAND, NY . $$", "create_at": date("2009-04-08"), "description": "Aries ♈️|@TamarBraxtonHer STAN. | Instagram : StevieSoFetch_ | Snapchat : StevieS0Fetch", "followers_count": 11535, "friends_count": 3931, "statues_count": 438769 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Inwood, NY", "id": "edd2aa964fa61ec9", "name": "Inwood", "place_type": "city", "bounding_box": rectangle("-73.77069,40.610349 -73.731496,40.632508") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 3637583, "cityName": "Inwood" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887778758131714, "text": "Hate being up by myself thinking", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1427975672, "name": "Emerson Carey", "screen_name": "xo_em4498", "lang": "en", "location": "815", "create_at": date("2013-05-14"), "description": "Hunter Cook 4/21 ❣", "followers_count": 1751, "friends_count": 1024, "statues_count": 42694 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cherry Valley, IL", "id": "009375045fd0dd8c", "name": "Cherry Valley", "place_type": "city", "bounding_box": rectangle("-89.057438,42.18021 -88.930599,42.275157") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17201, "countyName": "Winnebago", "cityID": 1713074, "cityName": "Cherry Valley" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887778892308481, "text": "need a papito to teach me how to dance like dis ������<3 https://t.co/thYysOSO80", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3254872778, "name": "JUNE30th", "screen_name": "victoriaarianav", "lang": "en", "location": "fhs", "create_at": date("2015-06-24"), "description": "ya tu sabes", "followers_count": 820, "friends_count": 543, "statues_count": 7801 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cinco Ranch, TX", "id": "01e74c422dab6107", "name": "Cinco Ranch", "place_type": "city", "bounding_box": rectangle("-95.846367,29.676113 -95.711056,29.767865") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4814929, "cityName": "Cinco Ranch" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887779186069504, "text": "Wow....is all I can say. The End!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 514606093, "name": "Rofo Ality", "screen_name": "Rofoality139", "lang": "en", "location": "null", "create_at": date("2012-03-04"), "description": "2 girls in Rofo. woohoo", "followers_count": 65, "friends_count": 315, "statues_count": 1044 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Royersford, PA", "id": "3a7875ae3f542697", "name": "Royersford", "place_type": "city", "bounding_box": rectangle("-75.548941,40.175125 -75.526955,40.195177") }, "geo_tag": { "stateID": 42, "stateName": "Pennsylvania", "countyID": 42091, "countyName": "Montgomery", "cityID": 4266576, "cityName": "Royersford" } }
+{ "create_at": datetime("2016-06-12T00:00:04.000Z"), "id": 741887779299201024, "text": "@shawnharmonn update:\nmy parents are bringing me to Burger King if you want that", "in_reply_to_status": 741887263060033538, "in_reply_to_user": 4461723914, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4461723914 }}, "user": { "id": 3002008934, "name": "aims", "screen_name": "amyyhutton", "lang": "en", "location": "probably sleeping", "create_at": date("2015-01-29"), "description": "what's up", "followers_count": 415, "friends_count": 388, "statues_count": 4878 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250146860146688, "text": "@Kostas_ante34 congrats on being selected for the U20 team! Big things ahead for you and your brothers.", "in_reply_to_status": -1, "in_reply_to_user": 2328804703, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2328804703 }}, "user": { "id": 74160168, "name": "$", "screen_name": "afriquesiii", "lang": "en", "location": "Dakar ✈️ Ann Arbor", "create_at": date("2009-09-14"), "description": "Alhamdoulillah.", "followers_count": 766, "friends_count": 533, "statues_count": 108420 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dexter, MI", "id": "494576de0ef68328", "name": "Dexter", "place_type": "city", "bounding_box": rectangle("-83.924606,42.304776 -83.839003,42.36893") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26161, "countyName": "Washtenaw", "cityID": 2622160, "cityName": "Dexter" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250147308929024, "text": "We Need a Resolution gives me Arabic or Egyptian teas", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2986139042, "name": "forresy", "screen_name": "v_aginal", "lang": "en", "location": "Des Moines, IA", "create_at": date("2015-01-16"), "description": "Header by: @mortalkombating", "followers_count": 1196, "friends_count": 364, "statues_count": 98332 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250147359297536, "text": "this zit needa die ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "nl", "is_retweet": false, "user": { "id": 2964182594, "name": "g a b e ⚒", "screen_name": "gabebruh_", "lang": "en", "location": "null", "create_at": date("2015-01-07"), "description": "oy vey", "followers_count": 134, "friends_count": 124, "statues_count": 3604 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Halawa, HI", "id": "ae8bddc0e5653780", "name": "Halawa", "place_type": "city", "bounding_box": rectangle("-157.937712,21.355729 -157.897921,21.393936") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1510000, "cityName": "Halawa" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250147501998080, "text": "@REALStaceyDash put this in your mouth. Do you job suck a dick and keep quiet. You're pretty not smart https://t.co/35Uxdffn2J", "in_reply_to_status": -1, "in_reply_to_user": 63291287, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 63291287 }}, "user": { "id": 1336820467, "name": "Junior OndaProwl", "screen_name": "Seriousboutsex", "lang": "en", "location": "Florida, USA", "create_at": date("2013-04-08"), "description": "Serious about life and all involved. i'm a 10! Psst. 18 and older please! adult content here.", "followers_count": 2555, "friends_count": 2413, "statues_count": 180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250147820666882, "text": "@pa10__ one call that's all it's like Morris Bart", "in_reply_to_status": 742249999787032576, "in_reply_to_user": 706330233448767489, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 706330233448767489 }}, "user": { "id": 3052065748, "name": "#boysloveslimmoney✨", "screen_name": "frannyygang10", "lang": "en", "location": "null", "create_at": date("2015-02-21"), "description": "whodeyWant? SlimMoreMONEY❤️ #kaybestie❤ #TOMBOY✨", "followers_count": 1171, "friends_count": 1397, "statues_count": 9363 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250148558835712, "text": "Don't piss off a #author, especially one that writes #fiction. Thy will #write you in a #story and kill you just to revive you to kill you!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "author", "fiction", "write", "story" }}, "user": { "id": 2934906898, "name": "Ironrod Media", "screen_name": "IronrodPR", "lang": "en", "location": "Provo, Utah", "create_at": date("2014-12-20"), "description": "Each of us has a story to tell.\n\nWe at Ironrod Media are here to help our clients in telling their stories to the world.", "followers_count": 1801, "friends_count": 4984, "statues_count": 2590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orem, UT", "id": "2b7c3f70fbcee536", "name": "Orem", "place_type": "city", "bounding_box": rectangle("-111.759345,40.256335 -111.633592,40.333892") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49049, "countyName": "Utah", "cityID": 4957300, "cityName": "Orem" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250148558983168, "text": "'handball' is now trending in #Miami https://t.co/SPsGwBAadx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Miami" }}, "user": { "id": 132389526, "name": "Trendsmap Miami", "screen_name": "TrendsMiami", "lang": "en", "location": "Miami, USA", "create_at": date("2010-04-12"), "description": "Real-time Miami Twitter trends", "followers_count": 4744, "friends_count": 1714, "statues_count": 43140 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami, FL", "id": "04cb31bae3b3af93", "name": "Miami", "place_type": "city", "bounding_box": rectangle("-80.321683,25.70904 -80.144974,25.855667") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245000, "cityName": "Miami" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250148571402240, "text": "Petty", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3119231490, "name": "Dondo", "screen_name": "LilDondo34", "lang": "en", "location": "null", "create_at": date("2015-03-30"), "description": "null", "followers_count": 652, "friends_count": 619, "statues_count": 7912 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairfield, CA", "id": "6da7626e4e9e26ba", "name": "Fairfield", "place_type": "city", "bounding_box": rectangle("-122.163438,38.157928 -121.922511,38.308796") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 623182, "cityName": "Fairfield" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250148911144960, "text": "I got HSM stuck in my head �� but I ain't complaining", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3302749297, "name": "GOAT", "screen_name": "abowllOfCereal", "lang": "en", "location": "San Antonio, TX", "create_at": date("2015-07-31"), "description": "dead", "followers_count": 151, "friends_count": 185, "statues_count": 1539 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250149024407552, "text": "OMFG https://t.co/GijPazWutz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 551674411, "name": "baby dre", "screen_name": "xxandreeea", "lang": "en", "location": "Bay Areeea ", "create_at": date("2012-04-12"), "description": "♡What's a nice girl like you doing in a dirty mind like mine?♡", "followers_count": 609, "friends_count": 592, "statues_count": 23478 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Danville, CA", "id": "aa30747001a23f03", "name": "Danville", "place_type": "city", "bounding_box": rectangle("-122.027412,37.779803 -121.89165,37.847751") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 617988, "cityName": "Danville" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250149762596868, "text": "Alondra �� she crazy af ���������� https://t.co/A8FqCQv55E", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2999074784, "name": "Jean-Luc Picard", "screen_name": "Satnjooh", "lang": "en", "location": "null", "create_at": date("2015-01-27"), "description": "Man , Visionary , God Fearing (Not the conversation type) Power Lvl 9000+", "followers_count": 517, "friends_count": 542, "statues_count": 19627 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Bernardino, CA", "id": "fedbd9fc207aae2a", "name": "San Bernardino", "place_type": "city", "bounding_box": rectangle("-117.401199,34.050311 -117.186993,34.219539") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 665000, "cityName": "San Bernardino" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250150089789440, "text": "wishing I had the juice for times like this ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 715729525980565504, "name": "##HAIL$ ✨", "screen_name": "haileenknowles", "lang": "en", "location": "prolly eating", "create_at": date("2016-03-31"), "description": "null", "followers_count": 424, "friends_count": 360, "statues_count": 4437 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Luling, LA", "id": "4c01d34feafd71a9", "name": "Luling", "place_type": "city", "bounding_box": rectangle("-90.408503,29.877325 -90.319424,29.950325") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22089, "countyName": "St. Charles", "cityID": 2246615, "cityName": "Luling" } }
+{ "create_at": datetime("2016-06-13T00:00:00.000Z"), "id": 742250150500896769, "text": "Wind 0.0 mph ---. Barometer 30.04 in, Falling slowly. Temperature 74.1 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23325 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151352225792, "text": "@ChocoSoIgnant https://t.co/pHZPshwUjJ", "in_reply_to_status": 742249883218776064, "in_reply_to_user": 2381737669, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2381737669 }}, "user": { "id": 3554530933, "name": "Daniela Velasco", "screen_name": "_daniela_sarahi", "lang": "en", "location": "Pittsburg, CA", "create_at": date("2015-09-13"), "description": "Positive attitude, Positive Life CSUEB 2020'", "followers_count": 391, "friends_count": 355, "statues_count": 5203 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151352369152, "text": "@slapgen watching MLB network and tweeting. Awful life, I know", "in_reply_to_status": 742249922314051584, "in_reply_to_user": 444318886, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 444318886 }}, "user": { "id": 738280860, "name": ":)", "screen_name": "PaulTweets2Much", "lang": "en", "location": "Queens, NY", "create_at": date("2012-08-05"), "description": "i Tweet Too Much | I'll probably upset you at some point | ❤️@PaulTweets2Much❤️ | i had sexual intercourse with Tom Brady & gave birth to @hsmitty3 | Andreina❤️", "followers_count": 8845, "friends_count": 826, "statues_count": 325574 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151444520960, "text": "HAPPY BIRTHDAY TO MA GIRL @_jennyy21_ ✨�� have the best birthday ever, see you later, I love you so much! ����������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2620198158 }}, "user": { "id": 3754206076, "name": "Alexa", "screen_name": "alexaaleah", "lang": "en", "location": "null", "create_at": date("2015-09-23"), "description": "going with the flow", "followers_count": 369, "friends_count": 243, "statues_count": 3831 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151641636864, "text": "@BawseAk lmao im jus clowning tonight ����", "in_reply_to_status": 742250070490370048, "in_reply_to_user": 490004685, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 490004685 }}, "user": { "id": 2786735354, "name": "Kiara✨", "screen_name": "heykiola", "lang": "en", "location": "Book❤️", "create_at": date("2014-09-02"), "description": "#Motivated", "followers_count": 2501, "friends_count": 1003, "statues_count": 63962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Calumet City, IL", "id": "6b745382f2cdbb67", "name": "Calumet City", "place_type": "city", "bounding_box": rectangle("-87.579081,41.581483 -87.525064,41.644809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1710487, "cityName": "Calumet City" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151671005184, "text": "But now, I'm just having second thoughts...like damn 4 years happened already?!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1869988448, "name": "PFC Cordero", "screen_name": "itsthecaptainn", "lang": "en", "location": "Bay Area | IG: itsthecaptainn_", "create_at": date("2013-09-15"), "description": "Army Strong | F.O.E | 18 | C/O 16 | THS", "followers_count": 285, "friends_count": 267, "statues_count": 9268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hayward, CA", "id": "5ecbd073f39c00fa", "name": "Hayward", "place_type": "city", "bounding_box": rectangle("-122.160814,37.592632 -122.010737,37.689909") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 633000, "cityName": "Hayward" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151700369408, "text": "@Kee2_Breezyy @JhanaiFelix07 HAPPY BIRTHDAY son. We love you.��❤️", "in_reply_to_status": 742248713066663937, "in_reply_to_user": 434616351, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 434616351, 3230334727 }}, "user": { "id": 328901698, "name": "HANN", "screen_name": "God4GivUs", "lang": "en", "location": "null", "create_at": date("2011-07-04"), "description": "Hannah Faith Jones | God 1st | Hooper| . Always stay gracious, best revenge is your paper. - Beyonce' UMHB fall 2016", "followers_count": 666, "friends_count": 725, "statues_count": 9163 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Hood, TX", "id": "c818c25e43957fd3", "name": "Fort Hood", "place_type": "city", "bounding_box": rectangle("-97.839309,31.10447 -97.722373,31.161093") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4826736, "cityName": "Fort Hood" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151754924033, "text": "@connorgras blessed", "in_reply_to_status": 742250015075139584, "in_reply_to_user": 2922541087, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2922541087 }}, "user": { "id": 2922541087, "name": "connor", "screen_name": "connorgras", "lang": "en", "location": "Alaska", "create_at": date("2014-12-08"), "description": "photo & film / @katiedewberry ❤️", "followers_count": 727, "friends_count": 99, "statues_count": 1098 }, "place": { "country": "United States", "country_code": "United States", "full_name": "College, AK", "id": "a4fbc08102ee5632", "name": "College", "place_type": "city", "bounding_box": rectangle("-147.910047,64.795273 -147.75567,64.900722") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 216750, "cityName": "College" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151767478273, "text": "Only 48 hours left until I can eat and sleep all I want!������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 724607427195215872, "name": "Kegahn Hargis", "screen_name": "ForeverFZKegahn", "lang": "en", "location": "null", "create_at": date("2016-04-25"), "description": "An average freshman loser.", "followers_count": 68, "friends_count": 94, "statues_count": 28 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Central Point, OR", "id": "ff96f35b5ab7ad8c", "name": "Central Point", "place_type": "city", "bounding_box": rectangle("-122.93452,42.361221 -122.883112,42.39622") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41029, "countyName": "Jackson", "cityID": 4112400, "cityName": "Central Point" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250151989760000, "text": "Thirst traps https://t.co/b2viJE3QNR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 15815069, "name": "cнeeғιn ѕнawтy", "screen_name": "Cheefinshawty", "lang": "en", "location": "San Antonio, TX", "create_at": date("2008-08-11"), "description": "706 ✈️ 254 210 an back again! Helpin my baby fight cancer. Snapchat & instagram cheefinshawty. trynna find love inna hopeless place!", "followers_count": 5914, "friends_count": 5566, "statues_count": 58555 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152010776576, "text": "@ebbtideapp Tide in Knappa, Oregon 06/13/2016\n Low 4:07am 1.9\nHigh 9:23am 6.0\n Low 4:01pm 1.4\nHigh 10:04pm 7.7\n Low 5:09am 1.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-123.5833,46.1833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 101, "friends_count": 1, "statues_count": 40963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Oregon, USA", "id": "df7fd3a3b9eff7ee", "name": "Oregon", "place_type": "admin", "bounding_box": rectangle("-124.703541,41.991795 -116.463262,46.299078") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41007, "countyName": "Clatsop" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152220450820, "text": "Happy birthday best friend @OceaanOfficial ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2521413145 }}, "user": { "id": 2906786396, "name": "jessica a. diaz", "screen_name": "fbfmgj", "lang": "en", "location": "East Los Angeles, CA", "create_at": date("2014-11-21"), "description": "1 out of 1 ... the only one", "followers_count": 232, "friends_count": 219, "statues_count": 13088 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Los Angeles, CA", "id": "67571a7baaa5906b", "name": "East Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.192945,34.008017 -118.126513,34.062578") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 620802, "cityName": "East Los Angeles" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152396771328, "text": "Lol i gotta stop ranting yo. Somebody always gotta pop off .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3450217457, "name": "h e n j r i x", "screen_name": "wherezayyat", "lang": "en", "location": "6-8-16", "create_at": date("2015-08-26"), "description": "iZaija Henjrix - Head of Vantis Entertainment - isaiahw.music@gmail.com - vantisent@gmail.com", "followers_count": 171, "friends_count": 226, "statues_count": 1634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baltimore, MD", "id": "c0b8e8dc81930292", "name": "Baltimore", "place_type": "city", "bounding_box": rectangle("-76.711521,39.197211 -76.529443,39.372215") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24510, "countyName": "Baltimore", "cityID": 2404000, "cityName": "Baltimore" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152505794560, "text": "06/13@03:00 - Temp 63.9F, WC 63.9F. Wind 1.5mph NNW, Gust 4.0mph. Bar 29.873in, Rising slowly. Rain 0.00in. Hum 57%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152656658432, "text": "������ well it's just really rude. I told him I was having a bad day because he texted me���� https://t.co/4EW5oZ8ZyX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3191867672, "name": "ItsKayDay", "screen_name": "queenkayday", "lang": "en", "location": "null", "create_at": date("2015-05-11"), "description": "Meditation and Prayer•A Beautiful Contradiction• Black Queen • #UALR19", "followers_count": 309, "friends_count": 420, "statues_count": 5700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152757501952, "text": "Wind 0.0 mph ESE. Barometer 30.002 in, Steady. Temperature 76.1 °F. Rain today 0.00 in. Humidity 81%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113926 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250152824430611, "text": "st.bernard 7ward! https://t.co/fENRZKW5NW", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3898788459, "name": "A.", "screen_name": "itsssamariel", "lang": "en", "location": "null", "create_at": date("2015-10-08"), "description": "i love you shawty - daddy ..longlivemybrudda!", "followers_count": 851, "friends_count": 324, "statues_count": 11095 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153042694144, "text": "����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2258504221, "name": "Tessa", "screen_name": "MayabbTessa", "lang": "en", "location": "London, OH", "create_at": date("2013-12-22"), "description": "null", "followers_count": 485, "friends_count": 397, "statues_count": 9732 }, "place": { "country": "United States", "country_code": "United States", "full_name": "London, OH", "id": "a04fe1c70e17bdef", "name": "London", "place_type": "city", "bounding_box": rectangle("-83.499959,39.874342 -83.389543,39.921691") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39097, "countyName": "Madison", "cityID": 3944674, "cityName": "London" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153390772224, "text": "1 American Wigeon (Anas americana) - Glendale Recharge Ponds - 2016-06-12 16:31 https://t.co/uhvbjv23nc", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.286582,33.5242237"), "retweet_count": 0, "lang": "cy", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16308 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427820, "cityName": "Glendale" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153390813184, "text": "1 Gray Hawk (Buteo plagiatus) - Hassayampa River--US 60 Rest Area - 2016-06-12 09:16 https://t.co/CEjtgJtO2e", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.6754808,33.9090904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16308 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153399046144, "text": "If a hood nigga broke your heart�� don't be sad he's going to get locked up sooner or later��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 741005688038510594, "name": "ig:laannii____$", "screen_name": "laanniiii____", "lang": "en", "location": "Long Beach, CA", "create_at": date("2016-06-09"), "description": "1️⃣6️⃣", "followers_count": 26, "friends_count": 35, "statues_count": 16 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Long Beach, CA", "id": "01c060cf466c6ce3", "name": "Long Beach", "place_type": "city", "bounding_box": rectangle("-118.250227,33.732905 -118.063194,33.885438") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 643000, "cityName": "Long Beach" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153407578113, "text": "goddamn its mad annoying seeing these kids exaggerating small shit and hyping stupid shit but in reality can't even speak up", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614894260, "name": "fatima imtiaz", "screen_name": "fata_imtiaz", "lang": "en", "location": "null", "create_at": date("2012-06-21"), "description": "alhumdulillah life's good", "followers_count": 569, "friends_count": 388, "statues_count": 21623 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Annandale, VA", "id": "4dad272dba156a02", "name": "Annandale", "place_type": "city", "bounding_box": rectangle("-77.262146,38.811091 -77.157684,38.865962") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51059, "countyName": "Fairfax", "cityID": 5101912, "cityName": "Annandale" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153407614977, "text": "2 Tropical Kingbird (Tyrannus melancholicus) - Hassayampa River--US 60 Rest Area - 2016-06-12 09:16 https://t.co/7YGEl9gzOB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.6754808,33.9090904"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16308 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153424392192, "text": "4 Grace's Warbler (Setophaga graciae) - Mt. Ord--FR 1688 - 2016-06-12 08:15 https://t.co/xtPU8yJWzR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.4143384,33.9211769"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16308 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153457881088, "text": "5 Grace's Warbler (Setophaga graciae) - Mt. Ord (Maricopa Co.) - 2016-06-12 10:50 https://t.co/xtPU8yJWzR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.4103365,33.9053284"), "retweet_count": 0, "lang": "sl", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16308 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153457901568, "text": "�������� https://t.co/HDKCWUJXIk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 232131027, "name": "Gotti", "screen_name": "_TayChico", "lang": "en", "location": "Low End ✈️ Jackson, TN", "create_at": date("2010-12-30"), "description": "#CountJugg #LongLiveSaieed Love you bro", "followers_count": 1634, "friends_count": 956, "statues_count": 48685 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153612976138, "text": "when your the only one on oovoo bc @meredithxoxo_ @mrt3rby fell asleep������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1590179563, 3365077313 }}, "user": { "id": 3368757005, "name": "noelia(:", "screen_name": "nmeyers02", "lang": "en", "location": "null", "create_at": date("2015-07-09"), "description": "| ths |", "followers_count": 206, "friends_count": 254, "statues_count": 2104 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250153780731904, "text": "wanna hear some Oldies?\nhttps://t.co/WYgnFDlCCO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 19187045, "name": "C H U Y", "screen_name": "ChuyGomez", "lang": "en", "location": "San Francisco, CA", "create_at": date("2009-01-19"), "description": "DAD • Radio Dude • Music Lover HOT1057fm San Francisco • YO1013fm Albuquerque • YO951fm San Antonio #iBeenDoingThis • Nor Cal Dj's", "followers_count": 26208, "friends_count": 3954, "statues_count": 49727 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South San Francisco, CA", "id": "746cc5651750e057", "name": "South San Francisco", "place_type": "city", "bounding_box": rectangle("-122.471871,37.634511 -122.374366,37.683086") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6081, "countyName": "San Mateo", "cityID": 673262, "cityName": "South San Francisco" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154154041344, "text": "why that ? You have the number to my back line https://t.co/xef1nBa0jF", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1481439822, "name": "King Lawrence", "screen_name": "LSnowton", "lang": "en", "location": "Dubai - Haiti - New Orleans", "create_at": date("2013-06-03"), "description": "#SouthernUniversityMostWanted ITS ONLY TWITTER!! NOT REAL LIFE! IN MY FUCKING MOMMA VOICE, I DONT PLAY THAT SHIT", "followers_count": 6757, "friends_count": 2461, "statues_count": 161439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154418442240, "text": "1 Hairy Woodpecker (Picoides villosus) - Mt. Ord--FR 1688 - 2016-06-12 08:15 https://t.co/d56mxI3H4x", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.4143384,33.9211769"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16313 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154422632452, "text": "1 Broad-billed Hummingbird (Cynanthus latirostris) - Hassayampa River Preserve - 2016-06-11 06:50 https://t.co/Peb2sS4Use", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.693,33.933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16313 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154477096960, "text": "2 Tropical Kingbird (Tyrannus melancholicus) - Hassayampa River Preserve - 2016-06-11 06:50 https://t.co/7YGEl9gzOB", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.693,33.933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16313 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154569412609, "text": "1 Thick-billed Kingbird (Tyrannus crassirostris) - Hassayampa River Preserve - 2016-06-11 06:50 https://t.co/Cbe0vwGcig", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-112.693,33.933"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16313 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154716233728, "text": "2/4 but she's fine, she can still let you know when your annoying her too lol when I left she was sleeping and going just fine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 775118155, "name": "Sarah Marie", "screen_name": "sarahmariegibbs", "lang": "en", "location": "some bullshit town ", "create_at": date("2012-08-22"), "description": "See heaven flash A horrorshow", "followers_count": 199, "friends_count": 741, "statues_count": 11419 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Half Moon, NC", "id": "f65751b1ab474517", "name": "Half Moon", "place_type": "city", "bounding_box": rectangle("-77.477448,34.793941 -77.437472,34.85045") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37133, "countyName": "Onslow", "cityID": 3728900, "cityName": "Half Moon" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154808475648, "text": "@Ali1979Zulfaqar بہت جلد @JawadAsghar4 @DigitalDarwesh", "in_reply_to_status": 742250044724764672, "in_reply_to_user": 2680162681, "favorite_count": 0, "retweet_count": 0, "lang": "ur", "is_retweet": false, "user_mentions": {{ 2680162681, 518473028, 2745942567 }}, "user": { "id": 2548627680, "name": "Rafi", "screen_name": "Rafi_AAA", "lang": "en", "location": "New York, NY", "create_at": date("2014-06-05"), "description": "بلها کی جاناں میں کون", "followers_count": 5443, "friends_count": 220, "statues_count": 58892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Twin Rivers, NJ", "id": "00ac552f286bf4fd", "name": "Twin Rivers", "place_type": "city", "bounding_box": rectangle("-74.503197,40.229591 -74.437678,40.277655") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34023, "countyName": "Middlesex", "cityID": 3474330, "cityName": "Twin Rivers" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154821095428, "text": "scattered clouds -> broken clouds\nhumidity up 44% -> 52%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-66.61,18.01"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 158509309, "name": "Ponce Weather", "screen_name": "_PoncePR", "lang": "en", "location": "Ponce, PR", "create_at": date("2010-06-22"), "description": "Weather updates, forecast, warnings and information for Ponce, PR. Sources: http://OpenWeatherMap.org, NOAA, USGS.", "followers_count": 60, "friends_count": 1, "statues_count": 23376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cuarto, USA", "id": "add930203c485358", "name": "Cuarto", "place_type": "city", "bounding_box": rectangle("-66.613629,18.003817 -66.606504,18.010988") }, "geo_tag": { "stateID": 72, "stateName": "Puerto Rico", "countyID": 72113, "countyName": "Ponce", "cityID": 7263820, "cityName": "Ponce" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154879766528, "text": "1 Lesser Scaup (Aythya affinis) - Riparian Preserve at Gilbert Water Ranch - 2016-06-09 18:32 https://t.co/YNIjUrFp7X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-111.7339478,33.3614502"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3011838662, "name": "NotableBirdsAZ", "screen_name": "NotableBirdsAZ", "lang": "en", "location": "AZ", "create_at": date("2015-02-06"), "description": "Today's valid notable bird sightings for Arizona (from eBird API).", "followers_count": 3, "friends_count": 0, "statues_count": 16315 }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250154959343617, "text": "Jk. It's not cover bands. It's rock band. So you best bet imma join them lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 43255130, "name": "Sailor Moon ✨", "screen_name": "thidatotter", "lang": "en", "location": "Seattle, WA", "create_at": date("2009-05-28"), "description": "Falling down the rabbit hole.", "followers_count": 543, "friends_count": 220, "statues_count": 55768 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seattle, WA", "id": "300bcc6e23a88361", "name": "Seattle", "place_type": "city", "bounding_box": rectangle("-122.436232,47.495315 -122.224973,47.734319") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5363000, "cityName": "Seattle" } }
+{ "create_at": datetime("2016-06-13T00:00:01.000Z"), "id": 742250155198550016, "text": "At the end of the day it's always going to be McDonalds", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 744307824, "name": "kiara", "screen_name": "Kiara_dice", "lang": "en", "location": "937", "create_at": date("2012-08-07"), "description": "null", "followers_count": 1259, "friends_count": 721, "statues_count": 13434 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Franklin, OH", "id": "0a3852e02570059c", "name": "Franklin", "place_type": "city", "bounding_box": rectangle("-84.40694,39.523616 -84.248456,39.58921") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39165, "countyName": "Warren", "cityID": 3928476, "cityName": "Franklin" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155231969281, "text": "Nahhh ima save that for a rainy ass day", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 712385040034861056, "name": "kai", "screen_name": "kairarivas999", "lang": "en", "location": "null", "create_at": date("2016-03-22"), "description": "I don't understand but I'm here and I'll always be here", "followers_count": 373, "friends_count": 785, "statues_count": 3700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jupiter, FL", "id": "b76db10a4d307c60", "name": "Jupiter", "place_type": "city", "bounding_box": rectangle("-80.177525,26.879058 -80.056708,26.983934") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1235875, "cityName": "Jupiter" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155231973376, "text": "@Ryleebish23 @CuteEmergency his eyes��", "in_reply_to_status": 742242079300943872, "in_reply_to_user": 251032126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 251032126, 568825492 }}, "user": { "id": 603329563, "name": "Mason Morales", "screen_name": "mason_618", "lang": "en", "location": "null", "create_at": date("2012-06-08"), "description": "Do you even lift", "followers_count": 595, "friends_count": 347, "statues_count": 16612 }, "place": { "country": "United States", "country_code": "United States", "full_name": "El Paso, TX", "id": "6a0a3474d8c5113c", "name": "El Paso", "place_type": "city", "bounding_box": rectangle("-106.634874,31.620668 -106.199987,31.966068") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48141, "countyName": "El Paso", "cityID": 4824000, "cityName": "El Paso" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155232112640, "text": "I found 120$ cash in a seven eleven Saturday night and bought my boy a pizza and we gave out slices to tha drunks leaving da bars", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3260452098, "name": "DANCING MACHINE", "screen_name": "yungferb", "lang": "en", "location": "Pismo Beach, CA", "create_at": date("2015-06-29"), "description": "It's okay amigo", "followers_count": 261, "friends_count": 240, "statues_count": 7061 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Morro Bay, CA", "id": "42a391fa59aedee2", "name": "Morro Bay", "place_type": "city", "bounding_box": rectangle("-120.874315,35.3408 -120.817763,35.415345") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6079, "countyName": "San Luis Obispo", "cityID": 649362, "cityName": "Morro Bay" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155550728194, "text": "Life has been treating me like shit", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2406323803, "name": "Lyss♡", "screen_name": "alyssamaelynnxo", "lang": "en", "location": "Tracy, CA", "create_at": date("2014-03-23"), "description": "sweet life", "followers_count": 349, "friends_count": 558, "statues_count": 2681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155575934976, "text": "Man Keenan hiding that hoe man from a ass whooping������ aww man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 319195996, "name": "HeartBreak.Kid⚡️⚓", "screen_name": "DuhthatsStew", "lang": "en", "location": "We$tside,TX", "create_at": date("2011-06-17"), "description": "#B.I.PLuh Steve #FreeLulManG #FreeWooG #FreeZaytoBleed #FreeTrey$avage STL x HOU SC:Blakeescott #TTPiru5700blk", "followers_count": 2765, "friends_count": 2903, "statues_count": 13626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155576037376, "text": "Temp 51.9° Hi/Lo 54.1/51.7 Rng 2.4° WC 51.9° Hmd 75% Rain 0.00\" Storm 0.00\" BAR 29.694 Falling DP 44.2° Wnd 0mph Dir --- Gst 13mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 123, "statues_count": 19459 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155596906497, "text": "@rachelblakee saaame", "in_reply_to_status": 742249497967763456, "in_reply_to_user": 33734718, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 33734718 }}, "user": { "id": 715269786, "name": "Views from Paul", "screen_name": "paulsingh2331", "lang": "en", "location": "null", "create_at": date("2012-07-24"), "description": "Based On a True Story...", "followers_count": 363, "friends_count": 301, "statues_count": 11494 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilbert, AZ", "id": "006b48995ede9bcc", "name": "Gilbert", "place_type": "city", "bounding_box": rectangle("-111.842244,33.204608 -111.634889,33.385822") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 427400, "cityName": "Gilbert" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155622043648, "text": "@voodoocatt @serenityvato dude he's funny and weird and adorable at the same time it's like great", "in_reply_to_status": 742249884904919041, "in_reply_to_user": 2988387015, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2988387015, 182977001 }}, "user": { "id": 2181585732, "name": "grim reaper", "screen_name": "_flivid", "lang": "en", "location": "dead", "create_at": date("2013-11-07"), "description": "head asss", "followers_count": 538, "friends_count": 269, "statues_count": 57925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155630579712, "text": "Melby Chiropractic, Champaign, IL, 61820 - https://t.co/nqs3uHkAGn\n\n#melbychiropractic #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.256057,40.112437"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "melbychiropractic", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 619, "friends_count": 608, "statues_count": 3379 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Champaign, IL", "id": "2335250557ea3fb4", "name": "Champaign", "place_type": "city", "bounding_box": rectangle("-88.333161,40.054674 -88.225277,40.160295") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17019, "countyName": "Champaign", "cityID": 1712385, "cityName": "Champaign" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155747856385, "text": "@naiyamariah @TrellNine why you moving bae? Move around", "in_reply_to_status": 742249967662727168, "in_reply_to_user": 2999178991, "favorite_count": 0, "coordinate": point("-90.06368165,29.98433161"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2999178991, 3154100606 }}, "user": { "id": 2858282905, "name": "tud.", "screen_name": "_latudda", "lang": "en", "location": "null", "create_at": date("2014-10-16"), "description": "I'm just styling with bee ..", "followers_count": 1640, "friends_count": 546, "statues_count": 27265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155794010112, "text": "There's just something I need from you, is to meet my boys", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230915481, "name": "Gio", "screen_name": "GiftedGioXO", "lang": "en", "location": "UCSD", "create_at": date("2010-12-26"), "description": "Romans 8:8-18", "followers_count": 685, "friends_count": 552, "statues_count": 133673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155928391680, "text": "Temp 61.8°F Heat Index 61.8 °F RH 90% Wind 0.0 --- Gust 0.0 --- SLP 29.953 in Steady Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 109, "friends_count": 62, "statues_count": 36132 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250155978588160, "text": "I rather wait to receive what I deserve than rush for something I believe I need right now.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 230492110, "name": "Jaelin Porter", "screen_name": "Jae_Lin15", "lang": "en", "location": "null", "create_at": date("2010-12-25"), "description": "To whom much is given, much is required...", "followers_count": 2757, "friends_count": 1847, "statues_count": 57594 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hillsboro, OR", "id": "01bd241973160cac", "name": "Hillsboro", "place_type": "city", "bounding_box": rectangle("-123.011705,45.488124 -122.859355,45.573434") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41067, "countyName": "Washington", "cityID": 4134100, "cityName": "Hillsboro" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250156683329536, "text": "Ripley SW Limestone Co. Temp: 74.3°F Wind:2.2mph Pressure: 996.2mb Falling Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 54454 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250156825841670, "text": "Say it", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 458853990, "name": "Jay", "screen_name": "jaysonjohnson6", "lang": "en", "location": "null", "create_at": date("2012-01-08"), "description": "Angelo State University '19 snapchat:jayson6 ig: johnson_jayson6", "followers_count": 820, "friends_count": 604, "statues_count": 10221 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Odessa, TX", "id": "2c0346ba4b733e24", "name": "Odessa", "place_type": "city", "bounding_box": rectangle("-102.434966,31.792563 -102.253208,31.955114") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48135, "countyName": "Ector", "cityID": 4853388, "cityName": "Odessa" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250157849337856, "text": "Wind 0.0 mph ---. Barometer 29.995 in, Rising slowly. Temperature 63.8 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 26, "statues_count": 20535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250157903745024, "text": "Who told me they were on molly last night �� I can't remember", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 983990953, "name": "mod", "screen_name": "Thizzymarley", "lang": "en", "location": "Bay Area", "create_at": date("2012-12-01"), "description": "sc:lilcroptop", "followers_count": 2197, "friends_count": 387, "statues_count": 16526 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pittsburg, CA", "id": "ebd427773b31cb21", "name": "Pittsburg", "place_type": "city", "bounding_box": rectangle("-121.987421,37.989865 -121.833268,38.043639") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 657456, "cityName": "Pittsburg" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158122008576, "text": "RftRftRftRft https://t.co/dsu9GChzgK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "de", "is_retweet": false, "user": { "id": 715035482, "name": "aubs", "screen_name": "aubrey_crespo", "lang": "en", "location": "the clouds", "create_at": date("2012-07-24"), "description": "|| life starts when fear ends || PHSCXC", "followers_count": 390, "friends_count": 431, "statues_count": 5470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Port Richey, FL", "id": "79dc2655046fbc40", "name": "New Port Richey", "place_type": "city", "bounding_box": rectangle("-82.764606,28.216424 -82.66958,28.280636") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12101, "countyName": "Pasco", "cityID": 1248500, "cityName": "New Port Richey" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158155390976, "text": "Daaaaamn this really what I need because I always put my feet on the seat in front of me https://t.co/KzwR5F2EyJ", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1864806810, "name": "Alexandra", "screen_name": "alexanduruh", "lang": "en", "location": "Hayward ↗️ Stockton ↗️ Sac ", "create_at": date("2013-09-14"), "description": "I'm Chicana so don't ask me", "followers_count": 547, "friends_count": 326, "statues_count": 18825 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Stockton, CA", "id": "d98e7ce217ade2c5", "name": "Stockton", "place_type": "city", "bounding_box": rectangle("-121.416872,37.883347 -121.183979,38.078305") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 675000, "cityName": "Stockton" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158390280192, "text": "Fair was Bella chill n lit with my cousin", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3239172086, "name": "angelica", "screen_name": "fothermuckerrs", "lang": "en", "location": "null", "create_at": date("2015-06-07"), "description": "Faith Hope & Love But the greatest is LOVE \n#18 Instagram @love.yourself18 SC youngbizzle94", "followers_count": 568, "friends_count": 815, "statues_count": 5291 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corcoran, CA", "id": "e882d4d41243119d", "name": "Corcoran", "place_type": "city", "bounding_box": rectangle("-119.592236,36.050709 -119.536157,36.12372") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6031, "countyName": "Kings", "cityID": 616224, "cityName": "Corcoran" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158470139904, "text": "79.0F (Feels: 86.6F) - Humidity: 92% - Wind: 1.2mph SSW - Gust: 1.7mph - Pressure: 30.035in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 242275 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158704885761, "text": "BOY https://t.co/leWlGXhhge", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1547471929, "name": "ang", "screen_name": "angxbear", "lang": "en", "location": "null", "create_at": date("2013-06-25"), "description": "RVHS Flutie.", "followers_count": 664, "friends_count": 646, "statues_count": 32966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250158805651456, "text": "Im right next to the front door and it JUST CREAKED", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3074361704, "name": "Orfa", "screen_name": "yeezykai", "lang": "en", "location": "this is a god dream", "create_at": date("2015-03-11"), "description": "soy salvatrucha. #blacklivesmatter", "followers_count": 757, "friends_count": 217, "statues_count": 77546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brentwood, NY", "id": "5fe049183a425395", "name": "Brentwood", "place_type": "city", "bounding_box": rectangle("-73.295325,40.754764 -73.209269,40.807341") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36103, "countyName": "Suffolk", "cityID": 3608026, "cityName": "Brentwood" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250159006842880, "text": "I'm fucking 18!����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1385019608, "name": "Alejandro", "screen_name": "OhlAlex", "lang": "en", "location": "null", "create_at": date("2013-04-27"), "description": "Enjoy life today, Yesterday is gone and tomorrow may never come.", "followers_count": 133, "friends_count": 218, "statues_count": 672 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lincoln City, OR", "id": "ba3023f97b91be31", "name": "Lincoln City", "place_type": "city", "bounding_box": rectangle("-124.02568,44.913113 -123.986305,45.010448") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41041, "countyName": "Lincoln", "cityID": 4142600, "cityName": "Lincoln City" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250159178944512, "text": "@DaddyJTAG @Yabbster so ur just selling preorders early? who cares", "in_reply_to_status": 742248543654686720, "in_reply_to_user": 713387224780374016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 713387224780374016, 1306860864 }}, "user": { "id": 1338196374, "name": "Cam", "screen_name": "Eplonn", "lang": "en", "location": "null", "create_at": date("2013-04-08"), "description": "•", "followers_count": 758, "friends_count": 97, "statues_count": 48963 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, KY", "id": "01470a6430b22b62", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-84.611622,38.903452 -84.491974,38.980877") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2139142, "cityName": "Independence" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250159233470464, "text": "d7b1c0fb00pF40B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-144.039893,60.229227"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "user": { "id": 702290904460169216, "name": "object82", "screen_name": "object82", "lang": "en", "location": "null", "create_at": date("2016-02-23"), "description": "null", "followers_count": 22, "friends_count": 0, "statues_count": 30034 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alaska, USA", "id": "07179f4fe0500a32", "name": "Alaska", "place_type": "admin", "bounding_box": rectangle("-179.231086,51.175093 179.859685,71.434357") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2261, "countyName": "Valdez-Cordova" } }
+{ "create_at": datetime("2016-06-13T00:00:02.000Z"), "id": 742250159329812481, "text": "I just finished @OITNB ���� thank god season 4 comes on Friday ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1345702333 }}, "user": { "id": 415035633, "name": "Gretchen", "screen_name": "granntkelley", "lang": "en", "location": "Anderson, IN", "create_at": date("2011-11-17"), "description": "null", "followers_count": 1453, "friends_count": 852, "statues_count": 31561 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, IN", "id": "3ac9eefd9dae14b1", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-85.748931,40.090722 -85.728724,40.112495") }, "geo_tag": { "stateID": 18, "stateName": "Indiana", "countyID": 18095, "countyName": "Madison", "cityID": 1820332, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250159858294785, "text": "I'm selfish with anything that's mine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2733650224, "name": "#Juice✨", "screen_name": "NeeMoneyy9", "lang": "en", "location": "Food Paradise W/ A$AP", "create_at": date("2014-08-04"), "description": "http://Dabomb.com❤ ✨#doitforRay8✨", "followers_count": 1331, "friends_count": 953, "statues_count": 67634 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250159937966080, "text": "Idk man", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 144063726, "name": "lys", "screen_name": "Ilyssa_Jenson", "lang": "en", "location": "Oregon, USA", "create_at": date("2010-05-14"), "description": "•XDL• March 31, 1996 - November 9, 2014", "followers_count": 1248, "friends_count": 703, "statues_count": 16866 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corvallis, OR", "id": "4da74513442e5fc5", "name": "Corvallis", "place_type": "city", "bounding_box": rectangle("-123.343425,44.527244 -123.215262,44.611247") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41003, "countyName": "Benton", "cityID": 4115800, "cityName": "Corvallis" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250159938146304, "text": "3am NY vibes. Word to @madeintyo https://t.co/HbTDnv8ptN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2812404403 }}, "user": { "id": 205005962, "name": "Willy Murillo", "screen_name": "Airwilly23", "lang": "en", "location": "null", "create_at": date("2010-10-19"), "description": "Email: Airwilly23@gmail.com | LMTD Supply Sneaker Shop | 279 George St. New Brunswick, NJ.", "followers_count": 644, "friends_count": 357, "statues_count": 39374 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250160202223616, "text": "IM@DJSKKDKDKFKD", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2150320806, "name": "nicole", "screen_name": "sincerejacks", "lang": "en", "location": "erin & dor", "create_at": date("2013-10-22"), "description": "like a little cutie pie", "followers_count": 7777, "friends_count": 173, "statues_count": 152832 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250160374218752, "text": "@sanyadeol �� https://t.co/lxjJvXmMgk", "in_reply_to_status": -1, "in_reply_to_user": 1671799417, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1671799417 }}, "user": { "id": 1033914216, "name": "marissa allessi", "screen_name": "marinsca2000", "lang": "en", "location": "630", "create_at": date("2012-12-24"), "description": "bhs", "followers_count": 667, "friends_count": 481, "statues_count": 1320 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carol Stream, IL", "id": "1aa7803ca6707875", "name": "Carol Stream", "place_type": "city", "bounding_box": rectangle("-88.179339,41.887811 -88.081435,41.942768") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17043, "countyName": "DuPage", "cityID": 1711332, "cityName": "Carol Stream" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250160453910528, "text": "If you were mine will both shine", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1371339408, "name": "JorgeSuave", "screen_name": "_jorgeOG", "lang": "en", "location": "Lennox, CA", "create_at": date("2013-04-21"), "description": "brrlyfe ❄️•LA ☀️• Addicted2Ink• VapeGuys", "followers_count": 438, "friends_count": 489, "statues_count": 13691 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lennox, CA", "id": "dd9ada4135bc9668", "name": "Lennox", "place_type": "city", "bounding_box": rectangle("-118.370212,33.931001 -118.344312,33.945471") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 641180, "cityName": "Lennox" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250160827334656, "text": "WE ARE IN KENTUCKY HELL YA! Ronnie chi_townhitman and me running night shift, 3hrs… https://t.co/LHq50GRDe3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.547,38.9537"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1692478693, "name": "HiJack™", "screen_name": "SpenceFluence", "lang": "en", "location": "Toronto/Boston", "create_at": date("2013-08-22"), "description": "I do Movie stuff sometimes, Bearded Villain, Auction Hunter, Book Hunter/Dealer, Adrenaline Junkie, Canadian Redneck, Spread Love like Violence", "followers_count": 1039, "friends_count": 1978, "statues_count": 1803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Independence, KY", "id": "01470a6430b22b62", "name": "Independence", "place_type": "city", "bounding_box": rectangle("-84.611622,38.903452 -84.491974,38.980877") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21117, "countyName": "Kenton", "cityID": 2139142, "cityName": "Independence" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161120759808, "text": "I mean you basically cheated on him with me in the beginning ���� bet you didn't tell him that lol", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1118456616, "name": "Zorrillo™", "screen_name": "zorillo0505", "lang": "en", "location": "null", "create_at": date("2013-01-24"), "description": "You did not appreciate so just watch me meditate", "followers_count": 261, "friends_count": 581, "statues_count": 19809 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161145925632, "text": "So help me God if I wake up and see another mass shooting or any type of shooting at that... I'm gonna flip my shit.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 459905282, "name": "Kayla {•.•} 〽️", "screen_name": "Muser4life39", "lang": "en", "location": "Kansas City, MO", "create_at": date("2012-01-09"), "description": "Just a girl with lots of Dreams | Snapchat ➡️ Kaybell84 | Also #ListenLonger on 9.65 The Buzz", "followers_count": 785, "friends_count": 431, "statues_count": 14545 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olathe, KS", "id": "06d6054b42e6575f", "name": "Olathe", "place_type": "city", "bounding_box": rectangle("-94.901766,38.825503 -94.714404,38.947743") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2052575, "cityName": "Olathe" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161208889344, "text": "Only pressed cause I cared tbh, but you good now .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 379406767, "name": "6.23", "screen_name": "ThatsSoKailen", "lang": "en", "location": "California, USA", "create_at": date("2011-09-24"), "description": "#freebookie", "followers_count": 1149, "friends_count": 625, "statues_count": 37947 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Moreno Valley, CA", "id": "c8448d3a8975a816", "name": "Moreno Valley", "place_type": "city", "bounding_box": rectangle("-117.296536,33.858759 -117.139304,33.985089") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 649270, "cityName": "Moreno Valley" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161326428162, "text": "@__hajidak yes i do", "in_reply_to_status": 742213191543984129, "in_reply_to_user": 1096905126, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1096905126 }}, "user": { "id": 325972552, "name": "L$D", "screen_name": "Biht_Diee", "lang": "en", "location": "337 ✈️478", "create_at": date("2011-06-28"), "description": "Shout out to God | @_xofaithh | SC:Micah_West", "followers_count": 1053, "friends_count": 680, "statues_count": 44063 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Warner Robins, GA", "id": "4caf02a6f195ddab", "name": "Warner Robins", "place_type": "city", "bounding_box": rectangle("-83.728243,32.495868 -83.554155,32.669392") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13153, "countyName": "Houston", "cityID": 1380508, "cityName": "Warner Robins" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161452122112, "text": "This is why I fw maxo..���� https://t.co/9E5hCm1UAK", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174483668, "name": "Chef R. Cole IV", "screen_name": "_ImSooLOST", "lang": "en", "location": "IVfourths Diner.", "create_at": date("2010-08-03"), "description": "#Founder: IVFourth's by Chef R. Cole | Business Inquiries- ivfourthsbooking@gmail.com || Personal Chef For #ArtStyled || #MySociety", "followers_count": 2904, "friends_count": 4995, "statues_count": 140296 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161527738368, "text": "Wind 0.0 mph ---. Barometer 29.994 in, Falling. Temperature 72.0 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 66922 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161632645120, "text": "BAT BA AKO NAGKAPE ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "tl", "is_retweet": false, "user": { "id": 145581994, "name": "MARI", "screen_name": "marinellbutor", "lang": "en", "location": "Myrtle Beach, SC", "create_at": date("2010-05-19"), "description": "beyond blessed", "followers_count": 381, "friends_count": 252, "statues_count": 44097 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Myrtle Beach, SC", "id": "17a83a55221b0671", "name": "Myrtle Beach", "place_type": "city", "bounding_box": rectangle("-78.991697,33.617059 -78.766035,33.789722") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45051, "countyName": "Horry", "cityID": 4549075, "cityName": "Myrtle Beach" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161758294016, "text": "good night Twitter . back to reality, waking up at 4:30 am and doing these 13 hour day work missions \n������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1931105352, "name": "jonathan", "screen_name": "crossedoutt", "lang": "en", "location": "null", "create_at": date("2013-10-03"), "description": "null", "followers_count": 483, "friends_count": 208, "statues_count": 27748 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, CA", "id": "a2c84129f9dcf69f", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-117.230172,33.090761 -117.103461,33.186722") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 668196, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250161846509569, "text": "I can't spell I swear . I literally become retarded when I'm not doing school work", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 85176601, "name": "Lisa con seta", "screen_name": "PensPaperPoetry", "lang": "en", "location": "Howard Beach, Queens", "create_at": date("2009-10-25"), "description": "Your welcome", "followers_count": 659, "friends_count": 364, "statues_count": 130845 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162290970624, "text": "@EricTrump @realDonaldTrump @IvankaTrump @DonaldJTrumpJr @MELANIATRUMP @LaraLeaTrump @TiffanyATrump @MrsVanessaTrump REALLY", "in_reply_to_status": 742156681384185862, "in_reply_to_user": 39349894, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 39349894, 25073877, 52544275, 39344374, 108471631, 75541946, 245963716, 475802156 }}, "user": { "id": 343249977, "name": "Julian McClain", "screen_name": "LAMP_STAND7", "lang": "en", "location": "Brooklyn Park, Mn", "create_at": date("2011-07-27"), "description": "Christian, Vietnam Vet, Writer", "followers_count": 22, "friends_count": 152, "statues_count": 560 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn Park, MN", "id": "1b86771ff62f45fb", "name": "Brooklyn Park", "place_type": "city", "bounding_box": rectangle("-93.402498,45.063712 -93.279015,45.152479") }, "geo_tag": { "stateID": 27, "stateName": "Minnesota", "countyID": 27053, "countyName": "Hennepin", "cityID": 2707966, "cityName": "Brooklyn Park" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162316312576, "text": "Wind 1.6 mph WNW. Barometer 29.753 in, Steady. Temperature 49.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11390 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162349690881, "text": "@emilio_oreo @_swagdaddyp PATRICIA AYEE", "in_reply_to_status": 742248891043565569, "in_reply_to_user": 2434807788, "favorite_count": 0, "retweet_count": 0, "lang": "es", "is_retweet": false, "user_mentions": {{ 2434807788, 740624249946152960 }}, "user": { "id": 750029358, "name": "Noah", "screen_name": "PumpkinNevarez", "lang": "en", "location": "San Diego, CA", "create_at": date("2012-08-10"), "description": "permanently triggered black spanish boy\n☕\nUCI ZOT THOT '20", "followers_count": 240, "friends_count": 156, "statues_count": 16737 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162387587073, "text": "San Diego Zoo (StreetView) https://t.co/vKk86IpZPI #sandiegozoo #zoo https://t.co/r2MpH3LQps", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-117.151573,32.735699"), "retweet_count": 0, "lang": "lv", "is_retweet": false, "hashtags": {{ "sandiegozoo", "zoo" }}, "user": { "id": 3784165287, "name": "pintica", "screen_name": "PinticaOfficiel", "lang": "fr", "location": "France", "create_at": date("2015-09-26"), "description": "Découvrez un million de lieux à travers la planète avec la carte intéractive ! http://pintica.com #voyage #tourisme #startup", "followers_count": 970, "friends_count": 1285, "statues_count": 6192 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162412654592, "text": "@idkgenesis so freaking cute haha https://t.co/CEhf2S0CWU", "in_reply_to_status": 742244499158507520, "in_reply_to_user": 1395997573, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1395997573 }}, "user": { "id": 3019934154, "name": "tiny t", "screen_name": "torie13_", "lang": "en", "location": "Norwalk, CA", "create_at": date("2015-02-14"), "description": "null", "followers_count": 605, "friends_count": 441, "statues_count": 9127 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Norwalk, CA", "id": "0706a21788cadb8d", "name": "Norwalk", "place_type": "city", "bounding_box": rectangle("-118.111506,33.88031 -118.041985,33.934191") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 652526, "cityName": "Norwalk" } }
+{ "create_at": datetime("2016-06-13T00:00:03.000Z"), "id": 742250162454568960, "text": "You can tell my sister is having a lot of fun������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 118532449, "name": "Andrea Arteaga", "screen_name": "drea_arte", "lang": "en", "location": "null", "create_at": date("2010-02-28"), "description": "null", "followers_count": 183, "friends_count": 189, "statues_count": 15891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fremont, CA", "id": "30344aecffe6a491", "name": "Fremont", "place_type": "city", "bounding_box": rectangle("-122.069956,37.454962 -121.889098,37.592327") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6001, "countyName": "Alameda", "cityID": 626000, "cityName": "Fremont" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612534788653056, "text": "i firmly believe that every dream or nightmare you have has a meaning.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1446302305, "name": "Moeee❤️", "screen_name": "_laughingmatter", "lang": "en", "location": "instagram: @laughingmatter", "create_at": date("2013-05-21"), "description": "null", "followers_count": 1112, "friends_count": 845, "statues_count": 43274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dentsville, SC", "id": "6adfad7bd6c1e881", "name": "Dentsville", "place_type": "city", "bounding_box": rectangle("-80.995487,34.051162 -80.912497,34.11936") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4519285, "cityName": "Dentsville" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535069814784, "text": "up with my stanka .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2887790720, "name": ".", "screen_name": "traniacee", "lang": "en", "location": "null", "create_at": date("2014-11-02"), "description": "you will never love me like my DAUGHTER love me . #kamiah❤️", "followers_count": 871, "friends_count": 780, "statues_count": 26908 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raceland, LA", "id": "1fa26465e8e760c4", "name": "Raceland", "place_type": "city", "bounding_box": rectangle("-90.757239,29.685723 -90.555104,29.762067") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22057, "countyName": "Lafourche", "cityID": 2263155, "cityName": "Raceland" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535174520833, "text": "saying they fat when they thick https://t.co/FX7XHu8Onp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 193952601, "name": "MARCUS", "screen_name": "_Marcus96", "lang": "en", "location": "MKE ✈️ HOU ", "create_at": date("2010-09-22"), "description": "pvamu | ♓️", "followers_count": 4906, "friends_count": 2336, "statues_count": 63897 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535224848384, "text": "the golden rule https://t.co/REZow4PUvs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1306626294, "name": "nicky", "screen_name": "nicky21__", "lang": "en", "location": "null", "create_at": date("2013-03-26"), "description": "18", "followers_count": 436, "friends_count": 504, "statues_count": 37246 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Gate, CA", "id": "d71cc91cb651e322", "name": "South Gate", "place_type": "city", "bounding_box": rectangle("-118.231784,33.909795 -118.156267,33.966295") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 673080, "cityName": "South Gate" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535593947136, "text": "I Throat Punch That Bitch ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 177464242, "name": "Jordan", "screen_name": "xoxo__JoJo", "lang": "en", "location": "null", "create_at": date("2010-08-11"), "description": "null", "followers_count": 362, "friends_count": 252, "statues_count": 18345 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Atascocita, TX", "id": "38d0e797745f4c5d", "name": "Atascocita", "place_type": "city", "bounding_box": rectangle("-95.249792,29.912242 -95.135568,30.033094") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4804462, "cityName": "Atascocita" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535824678912, "text": "������ https://t.co/4qhLo20I6Y", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3232316664, "name": "Alexis Castillo", "screen_name": "alexisjulianna2", "lang": "en", "location": "Queen Creek, AZ", "create_at": date("2015-06-01"), "description": "ASU ☀️ sc: alexiscastilll1 insta: alexissss15", "followers_count": 529, "friends_count": 351, "statues_count": 1681 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queen Creek, AZ", "id": "01cb573821d94344", "name": "Queen Creek", "place_type": "city", "bounding_box": rectangle("-111.686314,33.196614 -111.582748,33.288127") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 458150, "cityName": "Queen Creek" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535942078464, "text": "why am I so weak at this https://t.co/Q4viTBjpC9", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2455935608, "name": "mar j", "screen_name": "marilyn_jaquez", "lang": "en", "location": "null", "create_at": date("2014-04-20"), "description": "chipotle enthusiast", "followers_count": 1001, "friends_count": 691, "statues_count": 13232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612535971581952, "text": "@Reap_Howard not true!! that's how old my daddy was when he had me.. lol he did pretty damn good.", "in_reply_to_status": 742612314503843840, "in_reply_to_user": 101551960, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101551960 }}, "user": { "id": 242539098, "name": "3K", "screen_name": "BAWSEYGAL", "lang": "en", "location": "⚜ig/sc. @bawseygal/meen3000", "create_at": date("2011-01-24"), "description": "#BAM promotion collection now available!! ✨ jasmeenjforrest@gmail.com for any inquires.", "followers_count": 3213, "friends_count": 876, "statues_count": 214636 }, "place": { "country": "United States", "country_code": "United States", "full_name": "New Orleans, LA", "id": "dd3b100831dd1763", "name": "New Orleans", "place_type": "city", "bounding_box": rectangle("-90.137908,29.889574 -89.884108,30.075628") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22071, "countyName": "Orleans", "cityID": 2255000, "cityName": "New Orleans" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536030199809, "text": "@anagol_22 you have the same number?", "in_reply_to_status": -1, "in_reply_to_user": 234174575, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 234174575 }}, "user": { "id": 2509033782, "name": "Tatiana Zuvic", "screen_name": "tatizuvic", "lang": "en", "location": "null", "create_at": date("2014-05-19"), "description": "living in the now | CSLA⚽️ '18", "followers_count": 245, "friends_count": 330, "statues_count": 6366 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536093118468, "text": "im comfy but contacts like go ruin it ����������������������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2399138934, "name": "anne", "screen_name": "annearenas", "lang": "en", "location": "cali", "create_at": date("2014-03-19"), "description": "cwhs '19 || sc ~ annemariel_16", "followers_count": 438, "friends_count": 290, "statues_count": 9671 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fresno, CA", "id": "944c03c1d85ef480", "name": "Fresno", "place_type": "city", "bounding_box": rectangle("-119.93251,36.648905 -119.632419,36.923179") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 627000, "cityName": "Fresno" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536344727552, "text": "everybody. https://t.co/rkGmlnGb0F", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3296422387, "name": "Trappin&HoeSlappin", "screen_name": "richbastard", "lang": "en", "location": "@ da burfday party w/ meesh.", "create_at": date("2015-07-26"), "description": "United States Air Force ✈️", "followers_count": 1723, "friends_count": 1234, "statues_count": 54638 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536365715456, "text": "Live from New York, it's Saturday night memes", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 621322037, "name": "amish tech support", "screen_name": "t4yana", "lang": "en", "location": "az", "create_at": date("2012-06-28"), "description": "who cares stfu", "followers_count": 337, "friends_count": 233, "statues_count": 21259 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chandler, AZ", "id": "52445186970bafb3", "name": "Chandler", "place_type": "city", "bounding_box": rectangle("-111.972849,33.203761 -111.788898,33.36114") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 412000, "cityName": "Chandler" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536688791552, "text": "Temp: 57.6F W C: 57.6F Wind:--- at 0.0kts Baro: 1013.4mb and Rising slowly Rain today: 0.00in R H: 80% #wvwx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.59527778,40.43"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "wvwx" }}, "user": { "id": 813196879, "name": "Weirton Weather", "screen_name": "WeirtonWX", "lang": "en", "location": "Weirton, WV", "create_at": date("2012-09-09"), "description": "Weirton, WV current weather updated every 15 minutes", "followers_count": 216, "friends_count": 218, "statues_count": 107792 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Weirton, WV", "id": "9805bda00dfa7b9f", "name": "Weirton", "place_type": "city", "bounding_box": rectangle("-80.630541,40.365366 -80.517887,40.450178") }, "geo_tag": { "stateID": 54, "stateName": "West Virginia", "countyID": 54029, "countyName": "Hancock", "cityID": 5485156, "cityName": "Weirton" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612536894230528, "text": "Bout to sleep like a baby", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 418036069, "name": "Nathalie", "screen_name": "Naatthhaalliie", "lang": "en", "location": "California, USA", "create_at": date("2011-11-21"), "description": "..... And right before I go, please donate my brain to Captain-Save-A-Hoe. I beat faces ✨", "followers_count": 1081, "friends_count": 324, "statues_count": 53540 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537091325957, "text": "I love the adventures life throws at me!! ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537725141, "name": "Miguel", "screen_name": "ElMiguelC", "lang": "en", "location": "null", "create_at": date("2012-03-26"), "description": "21 | Austin Tx", "followers_count": 800, "friends_count": 404, "statues_count": 43774 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537301032960, "text": "it's 9:00 ooo �������� #nothinghappening ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nothinghappening" }}, "user": { "id": 727771348588765184, "name": "MA", "screen_name": "mkanoez", "lang": "en", "location": "Ewa Beach, HI", "create_at": date("2016-05-04"), "description": "Mitchel K. Ae", "followers_count": 127, "friends_count": 101, "statues_count": 336 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orchards, WA", "id": "1d471644ea7c10d2", "name": "Orchards", "place_type": "city", "bounding_box": rectangle("-122.552904,45.657895 -122.495761,45.708318") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53011, "countyName": "Clark", "cityID": 5351795, "cityName": "Orchards" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537305223170, "text": "All boys wanna do is lie & cheat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 622226784, "name": "☀️ r a v y n ☀️", "screen_name": "ThatGirlRavyn", "lang": "en", "location": "Beaumont, TX", "create_at": date("2012-06-29"), "description": "sc: ravyneliese ig: ravyynnn", "followers_count": 1781, "friends_count": 1064, "statues_count": 22546 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crosby, TX", "id": "00a488ec567255f3", "name": "Crosby", "place_type": "city", "bounding_box": rectangle("-95.12133,29.899357 -95.04072,29.942391") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4817756, "cityName": "Crosby" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537384914944, "text": "https://t.co/OKoQ5jujGN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1857804266, "name": "kaity ✨", "screen_name": "carmellina_", "lang": "en", "location": "jersey/texas", "create_at": date("2013-09-12"), "description": "the type to say a prayer then go get what I just prayed for", "followers_count": 675, "friends_count": 309, "statues_count": 8282 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Texas City, TX", "id": "05959d7d3c4d4c27", "name": "Texas City", "place_type": "city", "bounding_box": rectangle("-95.049499,29.33728 -94.886484,29.422811") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4872392, "cityName": "Texas City" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537531764736, "text": "I hate when all my music deletes. Like, idk what to do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 563366811, "name": "Nick Jimenez", "screen_name": "_NickiFlash", "lang": "en", "location": "Edinburg, TX", "create_at": date("2012-04-25"), "description": "Nursing Major @UTRGV / @Angiecamarillo_ ❤️/ Blessed.", "followers_count": 732, "friends_count": 1315, "statues_count": 9535 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murillo, TX", "id": "018561b7575b780a", "name": "Murillo", "place_type": "city", "bounding_box": rectangle("-98.155649,26.235135 -98.074291,26.29763") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48215, "countyName": "Hidalgo", "cityID": 4850092, "cityName": "Murillo" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612537812750340, "text": "This 24hr grocery reminds me of times before Katrina", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 26427008, "name": "TERRELL SOLETE", "screen_name": "tsolete", "lang": "en", "location": "Hollywood, Los Angeles", "create_at": date("2009-03-24"), "description": "I am here so I might as well live.", "followers_count": 1387, "friends_count": 688, "statues_count": 131070 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538056179712, "text": "Our live band photo is better than your live band photo. Red Eyed Sinatra will bring our mutant… https://t.co/gaOnedr0WS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.961594,40.719051"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 35248681, "name": "Steve Jenkins", "screen_name": "sjbassplayer", "lang": "en", "location": "New York, NY", "create_at": date("2009-04-25"), "description": "Bassist-Sideman-Composer-Sound Manipulator. Check out my music: http://stevejenkins.bandcamp.com", "followers_count": 1772, "friends_count": 675, "statues_count": 13995 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538060341248, "text": "Wind 0.0 mph ---. Barometer 29.95 in, Falling slowly. Temperature 75.6 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23346 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538068631552, "text": "We're monitoring storms in SW OK that may affect areas along the Red River early this morning. #dfwwx #texomawx https://t.co/EZ9Wi22mjk", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "dfwwx", "texomawx" }}, "user": { "id": 589123768, "name": "NWS Fort Worth", "screen_name": "NWSFortWorth", "lang": "en", "location": "Fort Worth, Texas", "create_at": date("2012-05-24"), "description": "Official Twitter account for the National Weather Service Fort Worth. Details: http://weather.gov/twitter", "followers_count": 60315, "friends_count": 161, "statues_count": 16885 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fort Worth, TX", "id": "42e46bc3663a4b5f", "name": "Fort Worth", "place_type": "city", "bounding_box": rectangle("-97.538285,32.569477 -97.033542,32.990456") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48439, "countyName": "Tarrant", "cityID": 4827000, "cityName": "Fort Worth" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538131517440, "text": "@lezDvlz rn I'm watching One Missed Call :/ i wanna change it so bad, but i just can't lmfao idk i'm weird", "in_reply_to_status": 742611979043381248, "in_reply_to_user": 2699308117, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2699308117 }}, "user": { "id": 2699308117, "name": "ૐ Lez", "screen_name": "lezDvlz", "lang": "en", "location": "California, USA", "create_at": date("2014-08-01"), "description": "null", "followers_count": 290, "friends_count": 132, "statues_count": 8392 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chula Vista, CA", "id": "28ace6b8d6dbc3af", "name": "Chula Vista", "place_type": "city", "bounding_box": rectangle("-117.117116,32.582128 -116.92781,32.685406") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 613392, "cityName": "Chula Vista" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538152488960, "text": "@JosieBohannan @iCBillz ��������", "in_reply_to_status": 742612377028300800, "in_reply_to_user": 798951468, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 798951468, 143674528 }}, "user": { "id": 2946827166, "name": "SARA WILLINGHAM", "screen_name": "SaraWillingham2", "lang": "en", "location": "Grand Saline, TX", "create_at": date("2014-12-28"), "description": "Insta// Sara.willingham sc ; sarawillingham2 Connor Blake curts❣✝", "followers_count": 434, "friends_count": 268, "statues_count": 6230 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Saline, TX", "id": "9de1c9523d318bf6", "name": "Grand Saline", "place_type": "city", "bounding_box": rectangle("-95.738336,32.659825 -95.694851,32.692657") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48467, "countyName": "Van Zandt", "cityID": 4830476, "cityName": "Grand Saline" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538190376960, "text": "@texasgirltweets haha that's a funny tweet", "in_reply_to_status": 742612007006834688, "in_reply_to_user": 2229475933, "favorite_count": 0, "coordinate": point("-71.0870147,41.90652033"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2229475933 }}, "user": { "id": 2292567861, "name": "frank", "screen_name": "fdinublia11", "lang": "en", "location": "Taunton, MA", "create_at": date("2014-01-19"), "description": "Loves blondes with blue eyes loves to play sports loves my Boston sports teams im from east Boston october 29th single Taunton mass #class2016", "followers_count": 1031, "friends_count": 4999, "statues_count": 55254 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Taunton, MA", "id": "642651131f2e976b", "name": "Taunton", "place_type": "city", "bounding_box": rectangle("-71.184442,41.836409 -70.973634,41.962658") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25005, "countyName": "Bristol", "cityID": 2569170, "cityName": "Taunton" } }
+{ "create_at": datetime("2016-06-14T00:00:00.000Z"), "id": 742612538341261313, "text": "@ebbtideapp Tide in Seaside, Oregon 06/14/2016\n Low 4:43am 0.6\nHigh 9:13am 3.5\n Low 4:04pm 0.7\nHigh 9:20pm 5.1\n Low 5:34am 0.4", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-123.92,46.0017"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 103, "friends_count": 1, "statues_count": 41240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Seaside, OR", "id": "8ad14dccfa2e45e8", "name": "Seaside", "place_type": "city", "bounding_box": rectangle("-123.956328,45.971243 -123.892377,46.017364") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41007, "countyName": "Clatsop", "cityID": 4165950, "cityName": "Seaside" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612538928422912, "text": "ew ihop no question https://t.co/e5I4hCTPzT", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 113529372, "name": "bri'ana♒️", "screen_name": "bri_bellaaaa", "lang": "en", "location": "Little Rock, AR", "create_at": date("2010-02-11"), "description": "fly high braelyn❤️501", "followers_count": 1260, "friends_count": 1062, "statues_count": 64597 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Little Rock, AR", "id": "45a45eb8e30e7b14", "name": "Little Rock", "place_type": "city", "bounding_box": rectangle("-92.521595,34.62595 -92.163049,34.817822") }, "geo_tag": { "stateID": 5, "stateName": "Arkansas", "countyID": 5119, "countyName": "Pulaski", "cityID": 541000, "cityName": "Little Rock" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612538978799616, "text": "@SierraDawn272 how'd you know this was about you?;);)", "in_reply_to_status": 742612328051408896, "in_reply_to_user": 2654769992, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2654769992 }}, "user": { "id": 263886180, "name": "❣libae", "screen_name": "libbywiseee", "lang": "en", "location": "Jammin out in my car", "create_at": date("2011-03-10"), "description": "dont blink & smile a lot ♡ instagram-libbywiseee", "followers_count": 869, "friends_count": 399, "statues_count": 16892 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Westlake Village, CA", "id": "f0592a72d560435a", "name": "Westlake Village", "place_type": "city", "bounding_box": rectangle("-118.856474,34.117867 -118.787618,34.16827") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 684438, "cityName": "Westlake Village" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612539369000960, "text": "\"#Imagine all the people,\nLiving for today....\"\nWatching this vigil, I could see the support we… https://t.co/MWdDplwlnN", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.243,34.0536"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Imagine" }}, "user": { "id": 274887776, "name": "Art Velasquez", "screen_name": "Arty_V", "lang": "en", "location": "Maywood, Ca.", "create_at": date("2011-03-30"), "description": "English, Spanish, American Sign Language. Dance/ Choreography. Love Interpreting Music! Biochemistry Major! Follow my randomness =)", "followers_count": 980, "friends_count": 540, "statues_count": 33835 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612539465420801, "text": "@Amoreme_ kickback details coming soon ��", "in_reply_to_status": 742612430967197697, "in_reply_to_user": 259897962, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 259897962 }}, "user": { "id": 736816272, "name": "bree ✨", "screen_name": "ilovebrizzy_", "lang": "en", "location": "NC | ATL", "create_at": date("2012-08-04"), "description": "positive vibes only... cau'19", "followers_count": 1746, "friends_count": 764, "statues_count": 32590 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Raleigh, NC", "id": "161d2f18e3a0445a", "name": "Raleigh", "place_type": "city", "bounding_box": rectangle("-78.818343,35.715805 -78.497331,35.972158") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37183, "countyName": "Wake", "cityID": 3755000, "cityName": "Raleigh" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612539494694913, "text": "Y'all da shitttttt https://t.co/IodzFtC8p3", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 36233235, "name": "ci.", "screen_name": "ciaralyssa", "lang": "en", "location": "null", "create_at": date("2009-04-28"), "description": "null", "followers_count": 1183, "friends_count": 439, "statues_count": 72655 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Henderson, NV", "id": "0e2242eb8691df96", "name": "Henderson", "place_type": "city", "bounding_box": rectangle("-115.158143,35.917793 -114.918531,36.094719") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3231900, "cityName": "Henderson" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612539494780928, "text": "@_KaylaShine HAHAH literally same I slept the whole day", "in_reply_to_status": 742611486825193472, "in_reply_to_user": 501730493, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 501730493 }}, "user": { "id": 472359699, "name": "Sydnee", "screen_name": "seeyndanna", "lang": "en", "location": "Irmo, SC", "create_at": date("2012-01-23"), "description": "Feeling good living better #WU20❤️", "followers_count": 711, "friends_count": 865, "statues_count": 23012 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irmo, SC", "id": "189ef793a6dbbfc8", "name": "Irmo", "place_type": "city", "bounding_box": rectangle("-81.257113,34.04725 -81.132587,34.170797") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45079, "countyName": "Richland", "cityID": 4535890, "cityName": "Irmo" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612539943485441, "text": "#OrlandoShooting https://t.co/6oM8172GKP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "OrlandoShooting" }}, "user": { "id": 29093567, "name": "Kyle Minor", "screen_name": "kyleminor", "lang": "en", "location": "San Francisco", "create_at": date("2009-04-05"), "description": "Eat here now.", "followers_count": 383, "friends_count": 732, "statues_count": 7508 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540048347136, "text": "@kelseyroddyy Kelsey Jo, happy bday you angel���� I love you to the moon and back and I hope your day is incredible, ur absolutely amazing❤️����", "in_reply_to_status": -1, "in_reply_to_user": 429431982, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 429431982 }}, "user": { "id": 632556472, "name": "hosé", "screen_name": "_hay1ie", "lang": "en", "location": "Marana, AZ", "create_at": date("2012-07-10"), "description": "free spirit w/ a wild heart", "followers_count": 812, "friends_count": 426, "statues_count": 32421 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kent, WA", "id": "8d71376556a9e531", "name": "Kent", "place_type": "city", "bounding_box": rectangle("-122.309297,47.343399 -122.126854,47.441224") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53033, "countyName": "King", "cityID": 5335415, "cityName": "Kent" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540190924800, "text": "There's nothing worse than missing the people you love. Being without the people that always make you happy is awful...", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 46262358, "name": "Jonah McCurry", "screen_name": "LopeDaddy", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2009-06-10"), "description": "Vibes. My girlfriend has an MD but it's soon to be an MM", "followers_count": 108, "friends_count": 74, "statues_count": 391 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Woodland, CA", "id": "9a2776eb0c58266f", "name": "Woodland", "place_type": "city", "bounding_box": rectangle("-121.802695,38.640692 -121.709862,38.709344") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6113, "countyName": "Yolo", "cityID": 686328, "cityName": "Woodland" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540241412100, "text": "@Beautifulmullaa whole*", "in_reply_to_status": 742612395059732480, "in_reply_to_user": 2990279391, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 101142537 }}, "user": { "id": 2990279391, "name": "___Airinsha", "screen_name": "WelchReshina", "lang": "en", "location": "null", "create_at": date("2015-01-21"), "description": "null", "followers_count": 401, "friends_count": 312, "statues_count": 14268 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Crystal Springs, MS", "id": "f2233f4a94ddc3e4", "name": "Crystal Springs", "place_type": "city", "bounding_box": rectangle("-90.37324,31.962055 -90.336631,32.014416") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28029, "countyName": "Copiah", "cityID": 2817060, "cityName": "Crystal Springs" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540316889088, "text": "06/14@03:00 - Temp 68.2F, WC 68.2F. Wind 4.0mph N, Gust 10.0mph. Bar 29.891in, Rising slowly. Rain 0.00in. Hum 58%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540354486276, "text": "How haven't I gotten beat up before?", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 513807053, "name": "bay", "screen_name": "Baylie_barrett", "lang": "en", "location": "Omaha, NE", "create_at": date("2012-03-03"), "description": "AND LIVE FROM NEW YORK ITS SATURDAY NIGHT!!!!", "followers_count": 385, "friends_count": 215, "statues_count": 25983 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Omaha, NE", "id": "a84b808ce3f11719", "name": "Omaha", "place_type": "city", "bounding_box": rectangle("-96.234587,41.175884 -95.872275,41.375558") }, "geo_tag": { "stateID": 31, "stateName": "Nebraska", "countyID": 31055, "countyName": "Douglas", "cityID": 3137000, "cityName": "Omaha" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540606140417, "text": "https://t.co/wq0aQu4YuS", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2986139042, "name": "forresy", "screen_name": "v_aginal", "lang": "en", "location": "Des Moines, IA", "create_at": date("2015-01-16"), "description": "De de de de de da de de de da de dayyyyy •••••Header by: @mortalkombating", "followers_count": 1225, "friends_count": 369, "statues_count": 98761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Des Moines, IA", "id": "1c67f9d9cbae7f69", "name": "Des Moines", "place_type": "city", "bounding_box": rectangle("-93.709504,41.501409 -93.503235,41.651466") }, "geo_tag": { "stateID": 19, "stateName": "Iowa", "countyID": 19153, "countyName": "Polk", "cityID": 1921000, "cityName": "Des Moines" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612540903981056, "text": "@Highxfvshion like you wanna have a conversation in my dm or we gone continue to mention me bc im not reply to both.", "in_reply_to_status": 742612200754315264, "in_reply_to_user": 3219761196, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3219761196 }}, "user": { "id": 110076399, "name": "moe", "screen_name": "Pocahontaaaaas", "lang": "en", "location": "htx", "create_at": date("2010-01-30"), "description": "try me ®", "followers_count": 1308, "friends_count": 709, "statues_count": 32684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pearland, TX", "id": "bcb210d6cf34438a", "name": "Pearland", "place_type": "city", "bounding_box": rectangle("-95.438435,29.497739 -95.215364,29.59803") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48039, "countyName": "Brazoria", "cityID": 4856348, "cityName": "Pearland" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541134807040, "text": "If the rain could hold off until after Thursday's Mets game that would be awesome. I just want to hug Mr Met and see @Neha_Joy ⚾️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 389797605 }}, "user": { "id": 547788295, "name": "Emi Feldman", "screen_name": "emifeldman", "lang": "en", "location": "New York, New York", "create_at": date("2012-04-07"), "description": "Desk Associate at @CBSRadioNews. Multimedia Journalist. Proud @HofstraU Alumni. All ideas are my own. Retweets are not endorsements.", "followers_count": 438, "friends_count": 336, "statues_count": 11684 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Manhattan, NY", "id": "01a9a39529b27f36", "name": "Manhattan", "place_type": "city", "bounding_box": rectangle("-74.026675,40.683935 -73.910408,40.877483") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36061, "countyName": "New York", "cityID": 36061, "cityName": "Manhattan" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541189169152, "text": "@AwesomeErn https://t.co/3YAAQ4B7Pq", "in_reply_to_status": 742611829550026752, "in_reply_to_user": 1623553836, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1623553836 }}, "user": { "id": 1623553836, "name": "Ernie", "screen_name": "AwesomeErn", "lang": "en", "location": "Oregon, USA", "create_at": date("2013-07-26"), "description": "King of FIFA. Tacos. young metro trusts me. snapchat:AwesomeErn", "followers_count": 438, "friends_count": 425, "statues_count": 57491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boardman, OR", "id": "6ad952d03f2df18f", "name": "Boardman", "place_type": "city", "bounding_box": rectangle("-119.721344,45.820748 -119.662605,45.850749") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41049, "countyName": "Morrow", "cityID": 4107200, "cityName": "Boardman" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541302415360, "text": "https://t.co/XlyovDFf1P", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1317717260, "name": "jaret", "screen_name": "Jaret_Hiers13", "lang": "en", "location": "New Orleans, LA", "create_at": date("2013-03-30"), "description": "⁶", "followers_count": 443, "friends_count": 297, "statues_count": 23307 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chalmette, LA", "id": "d5ff8d9603da85da", "name": "Chalmette", "place_type": "city", "bounding_box": rectangle("-89.995787,29.925458 -89.930331,29.968588") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22087, "countyName": "St. Bernard", "cityID": 2214135, "cityName": "Chalmette" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541398884352, "text": "Katanya D Ford, Gulfport, MS, 39501 - https://t.co/JQsfQt0Cl4\n\n#katanyadford #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.109139,30.371856"), "retweet_count": 0, "lang": "in", "is_retweet": false, "hashtags": {{ "katanyadford", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 622, "friends_count": 623, "statues_count": 3475 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gulfport, MS", "id": "019e22a02a5e2553", "name": "Gulfport", "place_type": "city", "bounding_box": rectangle("-89.137059,30.351774 -89.001207,30.50028") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28047, "countyName": "Harrison", "cityID": 2829700, "cityName": "Gulfport" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541424029697, "text": "@alondraYDG @steroIinee fr )':", "in_reply_to_status": 742612276142739456, "in_reply_to_user": 86837400, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 86837400, 3054875226 }}, "user": { "id": 2309076751, "name": "karen", "screen_name": "karengaribayyy_", "lang": "en", "location": "Rialto, CA", "create_at": date("2014-01-24"), "description": "null", "followers_count": 840, "friends_count": 376, "statues_count": 41378 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rialto, CA", "id": "5a507bb876cb216f", "name": "Rialto", "place_type": "city", "bounding_box": rectangle("-117.431914,34.019341 -117.34137,34.176056") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 660466, "cityName": "Rialto" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612541843460096, "text": "Love is https://t.co/gYA8uv18og", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 857335800, "name": "Erika Luna", "screen_name": "Erika9009Luna", "lang": "en", "location": "Phoenix, Arizona", "create_at": date("2012-10-01"), "description": "null", "followers_count": 159, "friends_count": 261, "statues_count": 27798 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542044807168, "text": "I don't hate or dislike none of my exes I just don't fw them no more", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 317841834, "name": "ACF", "screen_name": "AriGotDaJuiceee", "lang": "en", "location": "null", "create_at": date("2011-06-15"), "description": "In My Prime Future RN #EastonAlumni", "followers_count": 1169, "friends_count": 746, "statues_count": 50239 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Marrero, LA", "id": "2bca99f2a5ba7da1", "name": "Marrero", "place_type": "city", "bounding_box": rectangle("-90.139232,29.853716 -90.08087,29.910044") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22051, "countyName": "Jefferson", "cityID": 2248785, "cityName": "Marrero" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542057418752, "text": "Why am I still nor happy.'", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 468816310, "name": "D-PimpN", "screen_name": "darielaflowerzz", "lang": "en", "location": "ATX ", "create_at": date("2012-01-19"), "description": "I'm just a girl. Guess I'm some kind of freak.", "followers_count": 623, "friends_count": 254, "statues_count": 81567 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Austin, TX", "id": "c3f37afa9efcf94b", "name": "Austin", "place_type": "city", "bounding_box": rectangle("-97.928935,30.127892 -97.580513,30.518799") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48453, "countyName": "Travis", "cityID": 4805000, "cityName": "Austin" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542107721728, "text": "Alper got me fucked up w/ this short answer final...... where the scantron at come on", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3316501957, "name": "suga♡", "screen_name": "_alwaysaimee", "lang": "en", "location": "Portland, OR", "create_at": date("2015-08-15"), "description": "Aquarius | DB☕️ | Philippians 4:13 | in love w/ @travis_folsom ❤️", "followers_count": 422, "friends_count": 643, "statues_count": 5735 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukie, OR", "id": "0834e7769aa05fce", "name": "Milwaukie", "place_type": "city", "bounding_box": rectangle("-122.655374,45.424593 -122.588791,45.461537") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41005, "countyName": "Clackamas", "cityID": 4148650, "cityName": "Milwaukie" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542300655616, "text": "Imma miss my baby like crazy", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 441453694, "name": "grown++®", "screen_name": "Kortnee_Nelson", "lang": "en", "location": "CA ✈️ NJ", "create_at": date("2011-12-19"), "description": "Kean University VB '20 bestfriend:@_QueeenKenya | LL", "followers_count": 1838, "friends_count": 895, "statues_count": 16959 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542346952704, "text": "Ripley SW Limestone Co. Temp: 78.1°F Wind:1.6mph Pressure: 992.8mb Steady Rain Today 0.00in. #alwx #valleywx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-87.12305556,34.755"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "alwx", "valleywx" }}, "user": { "id": 183829847, "name": "Micheal Thompson", "screen_name": "NorthBamaWX", "lang": "en", "location": "Athens, AL", "create_at": date("2010-08-27"), "description": "null", "followers_count": 42, "friends_count": 33, "statues_count": 54525 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1079, "countyName": "Lawrence" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542489427968, "text": "It really hurts me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2379778916, "name": "Mari®", "screen_name": "mgerman82", "lang": "en", "location": "null", "create_at": date("2014-03-08"), "description": "null", "followers_count": 239, "friends_count": 430, "statues_count": 1700 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Spokane Valley, WA", "id": "0147373d5e0cb283", "name": "Spokane Valley", "place_type": "city", "bounding_box": rectangle("-117.325319,47.627801 -117.143559,47.697352") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53063, "countyName": "Spokane", "cityID": 5367167, "cityName": "Spokane Valley" } }
+{ "create_at": datetime("2016-06-14T00:00:01.000Z"), "id": 742612542862856192, "text": "Wind 0.0 mph ESE. Barometer 29.826 in, Steady. Temperature 75.3 °F. Rain today 0.00 in. Humidity 90%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113950 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543126921220, "text": "I'm not as sleepy as I wish I was.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 995796931, "name": "jul$", "screen_name": "idioxsyncratic", "lang": "en", "location": "SALTHATECITY", "create_at": date("2012-12-07"), "description": "18 & deadass w0ke", "followers_count": 1154, "friends_count": 1382, "statues_count": 46084 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Clearfield, UT", "id": "d254694c82a4473e", "name": "Clearfield", "place_type": "city", "bounding_box": rectangle("-112.05574,41.074662 -111.979637,41.126319") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49011, "countyName": "Davis", "cityID": 4913850, "cityName": "Clearfield" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543240347648, "text": "Wind 0.0 mph ---. Barometer 29.883 in, Falling slowly. Temperature 68.8 °F. Rain today 0.00 in. Humidity 64%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 26, "statues_count": 20559 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543286321152, "text": "Temp: 72.5°F Wind:0.0mph Pressure: 29.923hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 67017 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543454076928, "text": "Why are people on the internet so angry?????? Literally just calm down??????", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 54075266, "name": "Sierra Brenneman", "screen_name": "SierraNicole_13", "lang": "en", "location": "not a slytherin", "create_at": date("2009-07-05"), "description": "kind of angry with a side of bitter // 18♏", "followers_count": 698, "friends_count": 270, "statues_count": 5537 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Muskogee, OK", "id": "2daa13876c1ef767", "name": "Muskogee", "place_type": "city", "bounding_box": rectangle("-95.442801,35.667946 -95.298037,35.797212") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40101, "countyName": "Muskogee", "cityID": 4050050, "cityName": "Muskogee" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543714234369, "text": "https://t.co/6Fah1ils18 #infusionstudiollc #dylan #bees #swarm #honey #beekeeper #beehives… https://t.co/GlCBaPXzvz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-122.1525847,48.149243"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "infusionstudiollc", "dylan", "bees", "swarm", "honey", "beekeeper", "beehives" }}, "user": { "id": 2276731400, "name": "Dylan", "screen_name": "DDtheengineer", "lang": "en", "location": "Wa, Arlington", "create_at": date("2014-01-04"), "description": "Shop Owner/Master Piercer/Engineer/Beekeeper", "followers_count": 1101, "friends_count": 2541, "statues_count": 4226 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Arlington, WA", "id": "744fbe9224233893", "name": "Arlington", "place_type": "city", "bounding_box": rectangle("-122.199229,48.133613 -122.10191,48.203671") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5302585, "cityName": "Arlington" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543718313984, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":63.0°F Wind:0.0mph Pressure: 29.91hpa Falling slowly Rain Today 0.00in. Forecast: Unsettled, precip", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 81, "friends_count": 18, "statues_count": 324008 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612543730896896, "text": "@viickymoons so it dude the ending will fuck you up", "in_reply_to_status": 742605347886497792, "in_reply_to_user": 2987574590, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2987574590 }}, "user": { "id": 1602701419, "name": "monika", "screen_name": "MonikaaRae", "lang": "en", "location": "null", "create_at": date("2013-07-17"), "description": "ghchs 18' tt❤️", "followers_count": 284, "friends_count": 269, "statues_count": 5802 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612544246812673, "text": "ughghghggh i miss him so much already:(", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 900815149, "name": "abbey ☯", "screen_name": "faceghost___", "lang": "en", "location": "SA/TX", "create_at": date("2012-10-23"), "description": "alycia \n\n#TXST20", "followers_count": 164, "friends_count": 70, "statues_count": 15524 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Olathe, KS", "id": "06d6054b42e6575f", "name": "Olathe", "place_type": "city", "bounding_box": rectangle("-94.901766,38.825503 -94.714404,38.947743") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20091, "countyName": "Johnson", "cityID": 2052575, "cityName": "Olathe" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612544498454529, "text": "I'll do one when I find nice pictures of me a.k.a never because I'm picky.������ #polynesiangirlsbreakstheinternet", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "polynesiangirlsbreakstheinternet" }}, "user": { "id": 65525067, "name": "Zoom⚡️", "screen_name": "shuhlawm", "lang": "en", "location": "CA✈️HI✈️AK", "create_at": date("2009-08-13"), "description": "I like donuts. 90's baby who wishes she were a 90's teen instead. Believer, scholar, fangirl, foodie.", "followers_count": 1090, "friends_count": 993, "statues_count": 122583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Anchorage, AK", "id": "7a863bb88e5bb33c", "name": "Anchorage", "place_type": "city", "bounding_box": rectangle("-150.035259,61.043884 -149.48233,61.426201") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2020, "countyName": "Anchorage", "cityID": 203000, "cityName": "Anchorage" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612544586698752, "text": "Thank you @andersoncooper for reading the names of the beautiful souls taken and offering them dignity. Your work is so important + needed", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 15224867 }}, "user": { "id": 23128329, "name": "Ryan Christopher", "screen_name": "rychris", "lang": "en", "location": "New York, NY", "create_at": date("2009-03-06"), "description": "a living library of useless information.", "followers_count": 266, "friends_count": 492, "statues_count": 1081 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Brooklyn, NY", "id": "011add077f4d2da3", "name": "Brooklyn", "place_type": "city", "bounding_box": rectangle("-74.041878,40.570842 -73.855673,40.739434") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36047, "countyName": "Kings", "cityID": 36047, "cityName": "Brooklyn" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612544800493568, "text": "2k has my mind rn��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1325431976, "name": "AaronTJr", "screen_name": "Aaron_Tjr", "lang": "en", "location": "Stockton, CA", "create_at": date("2013-04-03"), "description": "AS LONG AS THE OUTCOME IS INCOME ; GOD GOT ME ; 1UP ; DIME DROPPERS; #project17 ; http://www.hudl.com/v/QFyMJ", "followers_count": 632, "friends_count": 585, "statues_count": 11867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lathrop, CA", "id": "2e15ebed23598d88", "name": "Lathrop", "place_type": "city", "bounding_box": rectangle("-121.326808,37.787451 -121.261674,37.855687") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 640704, "cityName": "Lathrop" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612545035468800, "text": "77.4F (Feels: 85.2F) - Humidity: 94% - Wind: 0.0mph SE - Gust: 0.0mph - Pressure: 29.931in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 242413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612545647697921, "text": "Don't get lost �� https://t.co/KaOWdqZFnp", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 537471635, "name": "cotton", "screen_name": "gavdawgjuicegod", "lang": "en", "location": "15 yemen road, yemen", "create_at": date("2012-03-26"), "description": "https://twitter.com/louievree/status/722117408744435713", "followers_count": 652, "friends_count": 465, "statues_count": 60536 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Irvine, CA", "id": "3d533362cdab8107", "name": "Irvine", "place_type": "city", "bounding_box": rectangle("-117.86894,33.606077 -117.695781,33.773387") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 636770, "cityName": "Irvine" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612545723236352, "text": "wellllllllllp................ https://t.co/KV1ZbgjvIg", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 614563678, "name": "lil yeye", "screen_name": "baegore_", "lang": "en", "location": "ptx", "create_at": date("2012-06-21"), "description": "can't pick a side, the gemini #UNT20", "followers_count": 936, "friends_count": 636, "statues_count": 48674 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pasadena, TX", "id": "3e82edc94d5c5ce1", "name": "Pasadena", "place_type": "city", "bounding_box": rectangle("-95.234584,29.59057 -95.083288,29.746032") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4856000, "cityName": "Pasadena" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546050347008, "text": "GAME OF THRONES!!! ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4832341243, "name": "prodigal son", "screen_name": "therawhog", "lang": "en", "location": "Sanger, CA", "create_at": date("2016-01-29"), "description": "I've got the night on my side", "followers_count": 194, "friends_count": 258, "statues_count": 2047 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sanger, CA", "id": "cea1f774c62bb6fc", "name": "Sanger", "place_type": "city", "bounding_box": rectangle("-119.583292,36.677705 -119.538486,36.721241") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6019, "countyName": "Fresno", "cityID": 667056, "cityName": "Sanger" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546125844481, "text": "same girl https://t.co/Tni9ffFpeR", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 327025967, "name": "ashlyn k✧", "screen_name": "ashhkilliann", "lang": "en", "location": "Loxx Hair Studio asst. ", "create_at": date("2011-06-30"), "description": "Gods not dead - in that white maxima with the sticker on the back - @jsnix31 ❤️", "followers_count": 1856, "friends_count": 1030, "statues_count": 20553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sheffield, AL", "id": "16ef366d32452a83", "name": "Sheffield", "place_type": "city", "bounding_box": rectangle("-87.741289,34.73094 -87.666441,34.780509") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1033, "countyName": "Colbert", "cityID": 169648, "cityName": "Sheffield" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546323021824, "text": "@Ryanchammond so stupid", "in_reply_to_status": 742612430937722883, "in_reply_to_user": 100717309, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 100717309 }}, "user": { "id": 261562796, "name": "John Cena", "screen_name": "DillonJStein", "lang": "en", "location": "Martinez CA", "create_at": date("2011-03-05"), "description": "Hey hey what can I do ?", "followers_count": 297, "friends_count": 401, "statues_count": 10274 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Martinez, CA", "id": "71d33f776fe41dfb", "name": "Martinez", "place_type": "city", "bounding_box": rectangle("-122.157021,37.954027 -122.075217,38.037226") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6013, "countyName": "Contra Costa", "cityID": 646114, "cityName": "Martinez" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546339799040, "text": "HAPPY BIRTHDAY FRIEND ���������� @_roseemaary_", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 374817879 }}, "user": { "id": 505170095, "name": "Adrien", "screen_name": "Adrieng12", "lang": "en", "location": "Bakersfield, CA", "create_at": date("2012-02-26"), "description": "gonna stay chilling, smh", "followers_count": 139, "friends_count": 131, "statues_count": 5626 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bakersfield, CA", "id": "960993b9cfdffda9", "name": "Bakersfield", "place_type": "city", "bounding_box": rectangle("-119.172179,35.255821 -118.878147,35.437982") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6029, "countyName": "Kern", "cityID": 603526, "cityName": "Bakersfield" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546503376897, "text": "I was sound asleep until Eddie left now I can't go back to sleep", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 922342788, "name": "liliaan", "screen_name": "lilysotox", "lang": "en", "location": "null", "create_at": date("2012-11-02"), "description": "living lavish & in love ✨", "followers_count": 1098, "friends_count": 669, "statues_count": 31803 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Salinas, CA", "id": "fa37e818b629e4ea", "name": "Salinas", "place_type": "city", "bounding_box": rectangle("-121.69198,36.643802 -121.590557,36.734497") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6053, "countyName": "Monterey", "cityID": 664224, "cityName": "Salinas" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612546775961600, "text": "@NotTaba no wonder the memes are dank. I wonder what kind of shitty memes they have in heaven.", "in_reply_to_status": 742611582752980992, "in_reply_to_user": 1340937896, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1340937896 }}, "user": { "id": 521906718, "name": "Rooster", "screen_name": "Ask_1007", "lang": "en", "location": "The church of latter day dude", "create_at": date("2012-03-11"), "description": "What's up foo? Dudeist Priest", "followers_count": 211, "friends_count": 203, "statues_count": 8786 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perris, CA", "id": "d6f36f6c3c320c85", "name": "Perris", "place_type": "city", "bounding_box": rectangle("-117.261392,33.755615 -117.179434,33.862662") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 656700, "cityName": "Perris" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612547023589376, "text": "Temp 53.8° Hi/Lo 58.6/53.7 Rng 4.9° WC 53.8° Hmd 74% Rain 0.00\" Storm 0.00\" BAR 29.767 Rising DP 45.7° Wnd 0mph Dir --- Gst 10mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 124, "statues_count": 19483 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-14T00:00:02.000Z"), "id": 742612547065548800, "text": "I hope I don't have no fucked up dream cuz I've been waking up pissed off like every other day for the past 2 weeks.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 241316319, "name": "ultralight beam", "screen_name": "SarinaAMOR_", "lang": "en", "location": "Eric Formans Basement ", "create_at": date("2011-01-21"), "description": "✨21/Pre-Med/♍", "followers_count": 682, "friends_count": 392, "statues_count": 46357 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Milwaukee, WI", "id": "2a93711775303f90", "name": "Milwaukee", "place_type": "city", "bounding_box": rectangle("-88.070827,42.920822 -87.863758,43.192623") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55079, "countyName": "Milwaukee", "cityID": 5553000, "cityName": "Milwaukee" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612547312844801, "text": "#Dub1 ✔️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "Dub1" }}, "user": { "id": 1486430767, "name": "JUNE14♊️", "screen_name": "Dr3Hunnid", "lang": "en", "location": "Corona, CA", "create_at": date("2013-06-05"), "description": "SC: dr3_rushh", "followers_count": 342, "friends_count": 318, "statues_count": 8187 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Corona, CA", "id": "5e4b6834e36e68fa", "name": "Corona", "place_type": "city", "bounding_box": rectangle("-117.672915,33.802102 -117.498401,33.916084") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 616350, "cityName": "Corona" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612547602395136, "text": "@ViolatedYoBitch ������", "in_reply_to_status": 742610647431548928, "in_reply_to_user": 1918668402, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1918668402 }}, "user": { "id": 421800407, "name": "k.", "screen_name": "kaaaaaat__", "lang": "en", "location": "STL ", "create_at": date("2011-11-26"), "description": "null", "followers_count": 1850, "friends_count": 901, "statues_count": 18682 }, "place": { "country": "United States", "country_code": "United States", "full_name": "O'Fallon, MO", "id": "e4c447e00985824a", "name": "O'Fallon", "place_type": "city", "bounding_box": rectangle("-90.772734,38.71256 -90.632554,38.846753") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29183, "countyName": "St. Charles", "cityID": 2954074, "cityName": "O'Fallon" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612547778531328, "text": "@donnabrazile @washingtonpost Ever given 1 moments thought to the harm his mouth does to every good law abiding Muslim American? Not fit", "in_reply_to_status": 742579218140237824, "in_reply_to_user": 60919240, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 60919240, 2467791 }}, "user": { "id": 735297696982925312, "name": "Peggy Todd", "screen_name": "redreader5346", "lang": "en", "location": "Michigan, USA", "create_at": date("2016-05-24"), "description": "Avid reader who loves gardening and politics. Retired collections specialist, Grandma to 2 cutest little boys ever.", "followers_count": 44, "friends_count": 67, "statues_count": 1650 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Algonac, MI", "id": "f9cae9ad6ab869c8", "name": "Algonac", "place_type": "city", "bounding_box": rectangle("-82.551469,42.607706 -82.515937,42.638355") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2601180, "cityName": "Algonac" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612547791032320, "text": "Why does naruto have so many fillers? #MidnightQuestions", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2608,34.072072"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "MidnightQuestions" }}, "user": { "id": 2266864573, "name": "•", "screen_name": "Marcusprz13", "lang": "en", "location": "null", "create_at": date("2013-12-28"), "description": "⚽️ łgöd SupremeFC", "followers_count": 414, "friends_count": 358, "statues_count": 893 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Echo Park Lake", "id": "07d9e38bc4486000", "name": "Echo Park Lake", "place_type": "poi", "bounding_box": rectangle("-118.2608001,34.0720719 -118.2608,34.072072") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612548256731137, "text": "I just need to go to bed.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174506029, "name": "RaeFromFrenchlane", "screen_name": "QUEEN_Ravee", "lang": "en", "location": "SouthernU✨", "create_at": date("2010-08-03"), "description": "RestMumAndTarrez✨", "followers_count": 916, "friends_count": 649, "statues_count": 62003 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Supreme, LA", "id": "0197befb716a14a7", "name": "Supreme", "place_type": "city", "bounding_box": rectangle("-91.000905,29.855833 -90.975204,29.898654") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22007, "countyName": "Assumption", "cityID": 2274235, "cityName": "Supreme" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612548495638528, "text": "Preciate you ! You too https://t.co/4fWPHMkzvY", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2553268681, "name": "Him.", "screen_name": "LegendJohnson45", "lang": "en", "location": "#TheySleepMovement", "create_at": date("2014-06-07"), "description": "Defensive Tackle at ________ l #LegendsAngels l #Innovative l @BLACKLFMAG Photographer l Radio Host l Bookings: LegendThePhotographer@gmail.com #ShotByLegend", "followers_count": 3583, "friends_count": 3470, "statues_count": 46467 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Garland, TX", "id": "7c01d867b8e8c494", "name": "Garland", "place_type": "city", "bounding_box": rectangle("-96.706144,32.829815 -96.537124,32.985469") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4829000, "cityName": "Garland" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549078646785, "text": "You don't weigh me down like you think you do", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 356649960, "name": "Dresden", "screen_name": "DREZDIIN", "lang": "en", "location": "null", "create_at": date("2011-08-16"), "description": "null", "followers_count": 947, "friends_count": 202, "statues_count": 761 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Albuquerque, NM", "id": "813a485b26b8dae2", "name": "Albuquerque", "place_type": "city", "bounding_box": rectangle("-106.791691,35.015891 -106.473745,35.218114") }, "geo_tag": { "stateID": 35, "stateName": "New Mexico", "countyID": 35001, "countyName": "Bernalillo", "cityID": 3502000, "cityName": "Albuquerque" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549263228928, "text": "I'm obviously always winning", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 465074995, "name": "anti", "screen_name": "illestbritney", "lang": "en", "location": "null", "create_at": date("2012-01-15"), "description": "sugar, spice, and everything nice❣", "followers_count": 497, "friends_count": 256, "statues_count": 26375 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549271621632, "text": "Wind 3.1 mph ENE. Barometer 29.92 in, Steady. Temperature 47.8 °F. Rain today 0.00 in. Humidity 69%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.06611111,46.71833333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 229663606, "name": "Central Park Weather", "screen_name": "CentralParkWx", "lang": "en", "location": "Central Park, Superior WI", "create_at": date("2010-12-22"), "description": "The Twitter feed for weather from Central Park in Superior, WI. #dlh #dusu", "followers_count": 19, "friends_count": 5, "statues_count": 42930 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Superior, WI", "id": "7476e0af22ffc989", "name": "Superior", "place_type": "city", "bounding_box": rectangle("-92.145754,46.65267 -91.98994,46.74804") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55031, "countyName": "Douglas", "cityID": 5578660, "cityName": "Superior" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549280075776, "text": "i can't deal with me & drew anymore", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2765265220, "name": "❁ LG the OG ❁", "screen_name": "lakengoldsbyyy", "lang": "en", "location": "double whippin'", "create_at": date("2014-09-08"), "description": "it's important to be selfless // be your own kind of wonderful // part-time vegan // status: im doin me", "followers_count": 739, "friends_count": 494, "statues_count": 9165 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gonzales, LA", "id": "82a3427fa492ed52", "name": "Gonzales", "place_type": "city", "bounding_box": rectangle("-90.959148,30.167772 -90.820958,30.292323") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22005, "countyName": "Ascension", "cityID": 2229850, "cityName": "Gonzales" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549418487809, "text": "Wind 0.0 mph ---. Barometer 29.896 in, Steady. Temperature 70.1 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 67018 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549452070912, "text": "Great well now I can't sleep��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1253252125, "name": "Christina Villalobos", "screen_name": "cfoggio", "lang": "en", "location": "null", "create_at": date("2013-03-08"), "description": "null", "followers_count": 565, "friends_count": 188, "statues_count": 3571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Gilroy, CA", "id": "8ad73577c2722154", "name": "Gilroy", "place_type": "city", "bounding_box": rectangle("-121.629338,36.973598 -121.53312,37.03609") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6085, "countyName": "Santa Clara", "cityID": 629504, "cityName": "Gilroy" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549586145280, "text": "Gladiator groomsmen show up, six packs and all, to collect the bride https://t.co/VMi8lzKScs", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-80.069251,26.350777"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174303316, "name": "Click Whisperer", "screen_name": "ClickWhisperer", "lang": "en", "location": "Boca Raton, FLORIDA", "create_at": date("2010-08-03"), "description": "Shamelessly sharing what's working #online.\r\nMoving beyond #analytics to usable #conclusions.\r\nCare for some #science with your #marketing? #TEAMFOLLOWBACK", "followers_count": 17625, "friends_count": 17801, "statues_count": 73013 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Boca Raton, FL", "id": "d1cd39353c51904d", "name": "Boca Raton", "place_type": "city", "bounding_box": rectangle("-80.203991,26.320692 -80.065981,26.427259") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12099, "countyName": "Palm Beach", "cityID": 1207300, "cityName": "Boca Raton" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549720383489, "text": "iOS 10 beta is clutch", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2332770025, "name": "Niko Gabrielli", "screen_name": "nikogbrr", "lang": "en", "location": "San Diego, CA", "create_at": date("2014-02-07"), "description": "@ESLGearsofWar player | http://twitch.tv/nikogbrr | Writing, showing what I can do.", "followers_count": 302, "friends_count": 241, "statues_count": 2583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612549942681601, "text": "Wind 0.0 mph ---. Barometer 29.810 in, Rising slowly. Temperature 72.1 °F. Rain today 0.00 in. Humidity 92%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 9161 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612550232068096, "text": "@Iamj3tt u said u wanted to watch the video LOL", "in_reply_to_status": 742612110224416768, "in_reply_to_user": 742609470216278016, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 742609470216278016 }}, "user": { "id": 3960906733, "name": "Isabella", "screen_name": "hiisabellaa", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2015-10-20"), "description": "DED just DED", "followers_count": 321, "friends_count": 512, "statues_count": 2867 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enterprise, NV", "id": "009d3c3d41dbb00e", "name": "Enterprise", "place_type": "city", "bounding_box": rectangle("-115.316075,35.960753 -115.152796,36.076856") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3223770, "cityName": "Enterprise" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612550487932930, "text": "Sexually frustrat", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3274692018, "name": "Diamond✨", "screen_name": "fvmous_baddie_", "lang": "en", "location": "Houston, TX", "create_at": date("2015-07-10"), "description": "sc- fvmous.baddie01 new twitter", "followers_count": 169, "friends_count": 120, "statues_count": 2406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612550739726337, "text": "My girlfriend the only reason why I'm still up ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2600295792, "name": "Dєгเςค ✨", "screen_name": "_PrettyAhQueen", "lang": "en", "location": "Baton Rouge, LA", "create_at": date("2014-07-02"), "description": "Future RN | Softball⚾ | Lesbian| @___MiyaStrong ❤", "followers_count": 4339, "friends_count": 4998, "statues_count": 45551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Baton Rouge, LA", "id": "c7ef5f3368b68777", "name": "Baton Rouge", "place_type": "city", "bounding_box": rectangle("-91.218994,30.354161 -90.962605,30.564039") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22033, "countyName": "East Baton Rouge", "cityID": 2205000, "cityName": "Baton Rouge" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612550844452864, "text": "omg this is the cutest !!! ❤️❤️ https://t.co/TnOwtIdTyH", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2962468500, "name": "Madalynnn❣", "screen_name": "mmmaadddiiieee", "lang": "en", "location": "with sujey ", "create_at": date("2015-01-06"), "description": "mia, soccer & @syrusleeee_ have my heart ❤️ @sujjeeyy is my suga mama ✨", "followers_count": 825, "friends_count": 607, "statues_count": 18311 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Antonio, TX", "id": "3df4f427b5a60fea", "name": "San Antonio", "place_type": "city", "bounding_box": rectangle("-98.778559,29.141956 -98.302744,29.693046") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48029, "countyName": "Bexar", "cityID": 4865000, "cityName": "San Antonio" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612550974603264, "text": "@trueGTM @LeafyIsHere @GradeAUnderA https://t.co/8nzc70JkAU", "in_reply_to_status": 742598766428053504, "in_reply_to_user": 1228176008, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 1228176008, 418362046, 1138224655 }}, "user": { "id": 575611627, "name": "D.S Productions", "screen_name": "DSProductions13", "lang": "en", "location": "cincinnati ohio ", "create_at": date("2012-05-09"), "description": "name is freedom. aka dom the bomb from youtube dot com. from cincinnati, ohio. I make music, write and direct films.", "followers_count": 1512, "friends_count": 0, "statues_count": 24583 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cincinnati, OH", "id": "e444ecd51bd16ff3", "name": "Cincinnati", "place_type": "city", "bounding_box": rectangle("-84.710722,39.052962 -84.368903,39.221114") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39061, "countyName": "Hamilton", "cityID": 3915000, "cityName": "Cincinnati" } }
+{ "create_at": datetime("2016-06-14T00:00:03.000Z"), "id": 742612551255642112, "text": "Temp 51.3°F Heat Index 51.3 °F RH 91% Wind 0.0 --- Gust 0.0 --- SLP 29.939 in Rising slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 62, "statues_count": 36156 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-14T00:00:04.000Z"), "id": 742612551545069568, "text": "Wind 1.1 mph NW. Barometer 29.865 in, Rising. Temperature 49.0 °F. Rain today 0.00 in. Humidity 91%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.995,44.50944444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2981085963, "name": "Old Pump Weather", "screen_name": "OldPumpWx", "lang": "en", "location": "null", "create_at": date("2015-01-16"), "description": "null", "followers_count": 3, "friends_count": 2, "statues_count": 11413 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Jericho, VT", "id": "013610ca13565e32", "name": "Jericho", "place_type": "city", "bounding_box": rectangle("-73.00967,44.495653 -72.940582,44.528943") }, "geo_tag": { "stateID": 50, "stateName": "Vermont", "countyID": 50007, "countyName": "Chittenden", "cityID": 5036625, "cityName": "Jericho" } }
+{ "create_at": datetime("2016-06-14T00:00:04.000Z"), "id": 742612551955976193, "text": "I want what I want at the exact time that I want it or else I don't want it anymore .", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-118.2153085,34.68847066"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2597218154, "name": "MiracleEssence✨", "screen_name": "miracle_essence", "lang": "en", "location": "beating sade and liz ass ", "create_at": date("2014-06-30"), "description": "i promise im nice ...", "followers_count": 637, "friends_count": 492, "statues_count": 9243 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-06-14T00:00:04.000Z"), "id": 742612552157274114, "text": "@ShakeBackSlim @_damnkayyy all this cute stuff .", "in_reply_to_status": 742611930561445888, "in_reply_to_user": 273236497, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 273236497, 2612757566 }}, "user": { "id": 2216734341, "name": "✨á", "screen_name": "mannnddda", "lang": "en", "location": "Not where you are ", "create_at": date("2013-12-09"), "description": "My name be ringing bells nigga ‼️", "followers_count": 577, "friends_count": 382, "statues_count": 10624 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Reserve, LA", "id": "2ad832dde65554d9", "name": "Reserve", "place_type": "city", "bounding_box": rectangle("-90.591825,30.049495 -90.510626,30.100941") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22095, "countyName": "St. John the Baptist", "cityID": 2264310, "cityName": "Reserve" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974922692173824, "text": "@ItsChrisPuta told you this would happen", "in_reply_to_status": 742974707335565312, "in_reply_to_user": 834839726, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 834839726 }}, "user": { "id": 2594053640, "name": "Daddy", "screen_name": "Belaltooswolee", "lang": "en", "location": "Miami, FL", "create_at": date("2014-06-28"), "description": "#YRA", "followers_count": 535, "friends_count": 433, "statues_count": 13468 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hialeah, FL", "id": "629f4a26fed69cd3", "name": "Hialeah", "place_type": "city", "bounding_box": rectangle("-80.356354,25.806057 -80.252971,25.909407") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1230000, "cityName": "Hialeah" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974922977378304, "text": "@Ziggymae \"snapchat@", "in_reply_to_status": -1, "in_reply_to_user": 4395001857, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4395001857 }}, "user": { "id": 732411749249499137, "name": "Z", "screen_name": "ZaylaMae1", "lang": "en", "location": "Crown Me", "create_at": date("2016-05-16"), "description": "rip BF 080314 ❤️ #aamu20", "followers_count": 357, "friends_count": 304, "statues_count": 2048 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Grand Rapids, MI", "id": "a81f9ed24c15d6af", "name": "Grand Rapids", "place_type": "city", "bounding_box": rectangle("-85.751406,42.883679 -85.530877,43.029144") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26081, "countyName": "Kent", "cityID": 2634000, "cityName": "Grand Rapids" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974923241619460, "text": "Baby videos. https://t.co/XGpDyuhd9X", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1493745439, "name": "mac", "screen_name": "queerfather", "lang": "en", "location": "mail order meat", "create_at": date("2013-06-08"), "description": "bitcoin farmer || he/him", "followers_count": 247, "friends_count": 349, "statues_count": 11929 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hollymead, VA", "id": "005c3d8d2405fc20", "name": "Hollymead", "place_type": "city", "bounding_box": rectangle("-78.465848,38.097127 -78.415379,38.146408") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51003, "countyName": "Albemarle", "cityID": 5138032, "cityName": "Hollymead" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974923719737344, "text": "one time i told my mom i was with Bree all day & then I got a text saying \"just bumped into your mom at Ralph's\" ��we all know how that ends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 493800725, "name": "maariyah", "screen_name": "mkhvvn", "lang": "en", "location": "null", "create_at": date("2012-02-15"), "description": "probably hate you", "followers_count": 420, "friends_count": 192, "statues_count": 51673 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Rancho Cucamonga, CA", "id": "36ac79e68ace76e4", "name": "Rancho Cucamonga", "place_type": "city", "bounding_box": rectangle("-117.637862,34.077184 -117.486443,34.166156") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 659451, "cityName": "Rancho Cucamonga" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924084613120, "text": "@ernestooo not after the ketchup and mayo I added��", "in_reply_to_status": 742974558379069441, "in_reply_to_user": 21450723, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 21450723 }}, "user": { "id": 163661666, "name": "José Rubio", "screen_name": "jrubioo", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-07-06"), "description": "Student • Entertainment biz • Follow on Instagram @jrubioo • Snapchat jrrubioo", "followers_count": 8488, "friends_count": 9210, "statues_count": 16406 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Miami Beach, FL", "id": "0121b92345f0e2b4", "name": "Miami Beach", "place_type": "city", "bounding_box": rectangle("-80.147162,25.761308 -80.118763,25.872623") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1245025, "cityName": "Miami Beach" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924399255552, "text": "Can't sleep and haven't done one of these in a while :) https://t.co/4WLcv5jrFL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1219962018, "name": "Dall", "screen_name": "dallasamsel", "lang": "en", "location": "vhs dance ", "create_at": date("2013-02-25"), "description": "beauty is in the eye of the beholder", "followers_count": 1101, "friends_count": 1037, "statues_count": 9569 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Santa Clarita, CA", "id": "f9c0877820b7848a", "name": "Santa Clarita", "place_type": "city", "bounding_box": rectangle("-118.627874,34.347483 -118.378643,34.483779") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 669088, "cityName": "Santa Clarita" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924541825024, "text": "Today was MY day. We were shopping for ME, because I FINALLY accomplished something.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 368577703, "name": "Lyssa ♡", "screen_name": "ANGELYSSAAXO_", "lang": "en", "location": "Arizona ", "create_at": date("2011-09-05"), "description": "Medical Assistant • ♏• I'm hilarious. Don't take my tweets seriously.", "followers_count": 1099, "friends_count": 702, "statues_count": 6729 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924772478976, "text": "I get high off of your memory... In due time you'll be mine.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 730831538259005440, "name": "Joanna", "screen_name": "missjoanna08", "lang": "en", "location": "null", "create_at": date("2016-05-12"), "description": "null", "followers_count": 10, "friends_count": 8, "statues_count": 110 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tamiami, FL", "id": "01f98b77415d9c8d", "name": "Tamiami", "place_type": "city", "bounding_box": rectangle("-80.498029,25.726759 -80.383636,25.789836") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12086, "countyName": "Miami-Dade", "cityID": 1270700, "cityName": "Tamiami" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924843843584, "text": "brb gotta crush baby on words w friends... again", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2660760696, "name": "sunshine", "screen_name": "mvrisoul", "lang": "en", "location": "I love her more; AAW ", "create_at": date("2014-07-19"), "description": "she's got the universe in her soul, eyes, & between her thighs | 050416 ❤️", "followers_count": 1256, "friends_count": 1642, "statues_count": 17314 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924910891008, "text": "genny jay https://t.co/wMclJbDZpd", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 709108380, "name": "#FREEHALLEGUAPO", "screen_name": "skinnyxforeva", "lang": "en", "location": "Mobile, AL", "create_at": date("2012-07-21"), "description": "insta: g.nasty | snap: QueenAlyse", "followers_count": 1300, "friends_count": 1481, "statues_count": 14920 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Augusta, SC", "id": "6ef112ac2aa9b7cc", "name": "North Augusta", "place_type": "city", "bounding_box": rectangle("-82.005594,33.477561 -81.895968,33.582108") }, "geo_tag": { "stateID": 45, "stateName": "South Carolina", "countyID": 45003, "countyName": "Aiken", "cityID": 4550695, "cityName": "North Augusta" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974924944465920, "text": "@ebbtideapp Tide in Hanauma Bay, Hawaii 06/15/2016\nHigh 11:08pm 1.0\n Low 5:48am -0.1\nHigh 12:52pm 1.7\n Low 7:40pm 0.5", "in_reply_to_status": -1, "in_reply_to_user": 4749048510, "favorite_count": 0, "coordinate": point("-157.7,21.2833"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 4749048510 }}, "user": { "id": 4754740136, "name": "Ebb Tide Bot", "screen_name": "ebbtidebot", "lang": "en", "location": "Seattle, WA", "create_at": date("2016-01-13"), "description": "Ebb Tide app for #iOS and #AppleWatch", "followers_count": 103, "friends_count": 1, "statues_count": 41521 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Honolulu, HI", "id": "c47c0bc571bf5427", "name": "Honolulu", "place_type": "city", "bounding_box": rectangle("-157.950476,21.254837 -157.648702,21.38505") }, "geo_tag": { "stateID": 15, "stateName": "Hawaii", "countyID": 15003, "countyName": "Honolulu", "cityID": 1506290, "cityName": "East Honolulu" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974925074534400, "text": "She niceeeeeeee. Fmllllll https://t.co/1aOI6JFkGG", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 709968140046884864, "name": "B.Dennis", "screen_name": "beyou_thankyou", "lang": "en", "location": "Ladera Heights, CA", "create_at": date("2016-03-15"), "description": "Alone time is major. Thank you Snapchat: sm0oth_1", "followers_count": 300, "friends_count": 276, "statues_count": 3966 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974925124866052, "text": "Back to work tomorrow afternoon and Thursday afternoon! Good night y'all!", "in_reply_to_status": 742974752072060928, "in_reply_to_user": 315557943, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 315557943, "name": "David A. Velazquez", "screen_name": "CrownTuba08210", "lang": "en", "location": "Orlando, Florida", "create_at": date("2011-06-11"), "description": "27 y/o | UCF c/o 2011 BME and BM | Work @ Universal | Classical music enthusiast | Video Gamer | Brother of Phi Mu Alpha Sinfonia | Toonami Faithful since 1997", "followers_count": 283, "friends_count": 466, "statues_count": 24487 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Orlando, FL", "id": "55b4f9e5c516e0b6", "name": "Orlando", "place_type": "city", "bounding_box": rectangle("-81.507905,28.388218 -81.22764,28.615139") }, "geo_tag": { "stateID": 12, "stateName": "Florida", "countyID": 12095, "countyName": "Orange", "cityID": 1253000, "cityName": "Orlando" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974925338775552, "text": "When Andrew sent me that message at 12 I couldn't help but to smile and cry at the same time! He's so amazing! I'm blessed honestly.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3111669842, "name": "leeann nicolee", "screen_name": "leeannnicoleeee", "lang": "en", "location": "Bishop, TX", "create_at": date("2015-03-27"), "description": "Blessed with a great life☝️@_andrew_5 ❤️", "followers_count": 312, "friends_count": 274, "statues_count": 2484 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kingsville, TX", "id": "803089e5de2b8c9b", "name": "Kingsville", "place_type": "city", "bounding_box": rectangle("-97.895204,27.462959 -97.828561,27.538226") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48273, "countyName": "Kleberg", "cityID": 4839352, "cityName": "Kingsville" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974925468798976, "text": "������ https://t.co/JO2LVUDgS2", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3995471872, "name": "natalie", "screen_name": "natalliiiee_", "lang": "en", "location": "null", "create_at": date("2015-10-19"), "description": "null", "followers_count": 153, "friends_count": 143, "statues_count": 1361 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lakewood, CA", "id": "0fc6568d145d84ac", "name": "Lakewood", "place_type": "city", "bounding_box": rectangle("-118.167821,33.820057 -118.05833,33.869242") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 639892, "cityName": "Lakewood" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974925481365504, "text": "#TweetLyricsThatHaveToBeShouted I DON'T NEED YOU, I DON'T NEED YOU, I DON'T NEED YOU, I DON'T NEED YOU BUT I WANT YOUUUUU", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TweetLyricsThatHaveToBeShouted" }}, "user": { "id": 1081608438, "name": "BERE", "screen_name": "dudechill__", "lang": "en", "location": "null", "create_at": date("2013-01-11"), "description": "Stay low key and stack ya money", "followers_count": 1409, "friends_count": 1256, "statues_count": 31585 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Palmdale, CA", "id": "e868cf904e518ff4", "name": "Palmdale", "place_type": "city", "bounding_box": rectangle("-118.262388,34.521165 -117.988004,34.646093") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 655156, "cityName": "Palmdale" } }
+{ "create_at": datetime("2016-06-15T00:00:00.000Z"), "id": 742974926420840448, "text": "Wrong https://t.co/Oe3wxINhDI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2415346789, "name": ".__.", "screen_name": "DatSauceBoi", "lang": "en", "location": "Flint Michigan ➡️MSU Spartan", "create_at": date("2014-03-27"), "description": "I Just Want Some Money", "followers_count": 477, "friends_count": 314, "statues_count": 14389 }, "place": { "country": "United States", "country_code": "United States", "full_name": "East Lansing, MI", "id": "aeca883f15e2aa85", "name": "East Lansing", "place_type": "city", "bounding_box": rectangle("-84.515906,42.704505 -84.447347,42.788935") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26065, "countyName": "Ingham", "cityID": 2624120, "cityName": "East Lansing" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974926978711552, "text": "Wind 0.1 mph NE. Barometer 29.76 in, Steady. Temperature 76.5 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.85833333,40.01194444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 165964121, "name": "Tobogan Hill Weather", "screen_name": "WoodlandLakesWS", "lang": "en", "location": "Petersburg, IL USA", "create_at": date("2010-07-12"), "description": "Weather Conditions on Tobogan Hill in Petersburg, Illinois. Updated hourly.", "followers_count": 22, "friends_count": 0, "statues_count": 33283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Petersburg, IL", "id": "209bc4b3f0185ebf", "name": "Petersburg", "place_type": "city", "bounding_box": rectangle("-89.862427,39.990492 -89.838169,40.025809") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17129, "countyName": "Menard", "cityID": 1759312, "cityName": "Petersburg" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974927045861377, "text": "Get on my level going to have the best lei tomorrow �� https://t.co/tNBeE1WYGL", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4073608812, "name": "ThatOneGuy", "screen_name": "BD_Hernandez_", "lang": "en", "location": "null", "create_at": date("2015-10-30"), "description": "03-15-14❤️", "followers_count": 113, "friends_count": 130, "statues_count": 643 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hawaiian Gardens, CA", "id": "bbdaba1428d18b5c", "name": "Hawaiian Gardens", "place_type": "city", "bounding_box": rectangle("-118.08284,33.822551 -118.061795,33.838733") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 632506, "cityName": "Hawaiian Gardens" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974927486210048, "text": "22 | 6•15 | 12:00:00", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 181873986, "name": "Najjiya Coussey", "screen_name": "VisionsOfJiya", "lang": "en", "location": "Inglewood", "create_at": date("2010-08-23"), "description": "null", "followers_count": 337, "friends_count": 267, "statues_count": 34285 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fountain Valley, CA", "id": "00c12e8612b69ccf", "name": "Fountain Valley", "place_type": "city", "bounding_box": rectangle("-117.980552,33.686723 -117.914731,33.733487") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6059, "countyName": "Orange", "cityID": 625380, "cityName": "Fountain Valley" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974927574306816, "text": "what the fuck is going on. i don't even have this guy on facebook. i'm officially creeped https://t.co/1Wcg5YyNNW", "in_reply_to_status": 742973882135613440, "in_reply_to_user": 1321832448, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1321832448, "name": "♡♡♡", "screen_name": "cakefacecharlie", "lang": "en", "location": "snapchat: stfufagboy", "create_at": date("2013-04-01"), "description": "19. Leo. Makeup lover. Beyoncé, Fifth Harmony, & Ariana Grande enthusiast. ♡♡ Email: cakefacecharlie@gmail.com", "followers_count": 4762, "friends_count": 523, "statues_count": 52975 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fairbanks, AK", "id": "174af231a0d9f46c", "name": "Fairbanks", "place_type": "city", "bounding_box": rectangle("-147.81382,64.810474 -147.543503,64.865697") }, "geo_tag": { "stateID": 2, "stateName": "Alaska", "countyID": 2090, "countyName": "Fairbanks North Star", "cityID": 224230, "cityName": "Fairbanks" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974927825952768, "text": "I'm actually really scared of how fast my moving day is coming ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3299806028, "name": "Liv ", "screen_name": "oliviamezy", "lang": "en", "location": "North Las Vegas, NV", "create_at": date("2015-07-28"), "description": "just living life day by day", "followers_count": 82, "friends_count": 130, "statues_count": 452 }, "place": { "country": "United States", "country_code": "United States", "full_name": "North Las Vegas, NV", "id": "01c77c227d4ab0d5", "name": "North Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.212251,36.184836 -115.054495,36.313413") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3251800, "cityName": "North Las Vegas" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928018911232, "text": "#mariahcarey #LasVegas #1toinfinity #LambilyBrasil #SheilaVerissimo #MissBrasilGay ❤️ @ The… https://t.co/IlmiSM3PRi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-115.17468201,36.11798815"), "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "mariahcarey", "LasVegas", "1toinfinity", "LambilyBrasil", "SheilaVerissimo", "MissBrasilGay" }}, "user": { "id": 169720013, "name": "Sheila Verissimo", "screen_name": "Sheila_MissBR", "lang": "pt", "location": "Recreio dos Bandeirantes / RJ", "create_at": date("2010-07-22"), "description": "Miss Brasil Gay ✨✨", "followers_count": 385, "friends_count": 256, "statues_count": 1949 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928333463552, "text": "Had no clue this is where Bryson tiller sampled exchange from �� https://t.co/kBBPObpC4h", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1322826608, "name": "leslie delasbour", "screen_name": "leslieeeeeeee_", "lang": "en", "location": "Hampton, VA", "create_at": date("2013-04-02"), "description": "#HamptonU19 HTX ✈️ VA #QTX⚓️ //creole princess // you mix that negro and that creole make me a TEXAS BAMA // ¯\\ _(ツ)_/¯", "followers_count": 2206, "friends_count": 1891, "statues_count": 40762 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928434135040, "text": "@NikkiLucca that's why you..... Nvm ������", "in_reply_to_status": 742974713610309632, "in_reply_to_user": 177448927, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 177448927 }}, "user": { "id": 1545242275, "name": "augustine", "screen_name": "AuggieM", "lang": "en", "location": "#Snowman", "create_at": date("2013-06-25"), "description": "khs | #Patriots", "followers_count": 1547, "friends_count": 980, "statues_count": 41398 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tracy, CA", "id": "3f586fd079afc084", "name": "Tracy", "place_type": "city", "bounding_box": rectangle("-121.520904,37.672746 -121.35153,37.773954") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6077, "countyName": "San Joaquin", "cityID": 680238, "cityName": "Tracy" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928593530880, "text": "30ish �� https://t.co/j0QoyxKUCa", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2182573148, "name": "‼️", "screen_name": "ashxdash", "lang": "en", "location": "null", "create_at": date("2013-11-08"), "description": "2.12.15 ❤️ Snap: @ashxdash02 ✨", "followers_count": 360, "friends_count": 291, "statues_count": 1905 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hazel Crest, IL", "id": "d6eacc5955ffad4b", "name": "Hazel Crest", "place_type": "city", "bounding_box": rectangle("-87.713706,41.549739 -87.65574,41.586369") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1733695, "cityName": "Hazel Crest" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928731971584, "text": "Temp: 64.3°F. Wind: SSE @ 0.4mph. Pressure: 32.438\" Falling. Precip today: 0.06\". More info: https://t.co/xoa3CJC7Kt. #weather #FoxCities", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.415,44.23083333"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather", "FoxCities" }}, "user": { "id": 466790190, "name": "UWFox WeatherStation", "screen_name": "UWFoxWX", "lang": "en", "location": "Menasha, Wisconsin", "create_at": date("2012-01-17"), "description": "Automated hourly reports from #weather station on campus of the University of #Wisconsin- #FoxValley in #Menasha, WI. Model Davis Vantage 2 Pro. #FoxCities", "followers_count": 40, "friends_count": 114, "statues_count": 5637 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Menasha, WI", "id": "4d6ee9c6c915dcef", "name": "Menasha", "place_type": "city", "bounding_box": rectangle("-88.544163,44.192468 -88.343067,44.244636") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55139, "countyName": "Winnebago", "cityID": 5550825, "cityName": "Menasha" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928882925568, "text": "couldnt even count the number of movies ive watched so far this summer", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2244784195, "name": "Victoria", "screen_name": "_Victoria1026_", "lang": "en", "location": "Mulvane", "create_at": date("2013-12-13"), "description": "null", "followers_count": 299, "friends_count": 312, "statues_count": 4369 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mulvane, KS", "id": "0978fe2fc6a2767f", "name": "Mulvane", "place_type": "city", "bounding_box": rectangle("-97.257821,37.466091 -97.208139,37.50454") }, "geo_tag": { "stateID": 20, "stateName": "Kansas", "countyID": 20173, "countyName": "Sedgwick", "cityID": 2049100, "cityName": "Mulvane" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928908062720, "text": "feelin' more than cool", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2371316904, "name": "hann solo", "screen_name": "hannahnewey_", "lang": "en", "location": "Ogden, UT", "create_at": date("2014-03-03"), "description": "rule number one never trust no h", "followers_count": 1500, "friends_count": 440, "statues_count": 11931 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Plain City, UT", "id": "3e57f0def110a369", "name": "Plain City", "place_type": "city", "bounding_box": rectangle("-112.102856,41.279451 -112.044231,41.329741") }, "geo_tag": { "stateID": 49, "stateName": "Utah", "countyID": 49057, "countyName": "Weber", "cityID": 4960710, "cityName": "Plain City" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974928945844224, "text": "ok.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 444194334, "name": "Samantha Ewing", "screen_name": "Samanthaewingg", "lang": "en", "location": "Perry, Ohio", "create_at": date("2011-12-22"), "description": "Perry High School\n\nIG- samanthaewingg", "followers_count": 297, "friends_count": 1059, "statues_count": 3438 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Perry, OH", "id": "01186f684ea0abd7", "name": "Perry", "place_type": "city", "bounding_box": rectangle("-81.191268,41.708808 -81.103314,41.792184") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39085, "countyName": "Lake", "cityID": 3961882, "cityName": "Perry" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929235247104, "text": "Popa Charles M, Hamilton, OH, 45015 - https://t.co/3y7LjgPKpf\n\n#popacharlesm #chiropractor #reviews", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.558582,39.356579"), "retweet_count": 0, "lang": "ht", "is_retweet": false, "hashtags": {{ "popacharlesm", "chiropractor", "reviews" }}, "user": { "id": 380773836, "name": "eTRiT", "screen_name": "eTRiTcom", "lang": "en", "location": "United States", "create_at": date("2011-09-26"), "description": "in-depth socially driven local business reviews", "followers_count": 625, "friends_count": 625, "statues_count": 3571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hamilton, OH", "id": "3a32d101fe825fd9", "name": "Hamilton", "place_type": "city", "bounding_box": rectangle("-84.64717,39.350135 -84.474638,39.457988") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39017, "countyName": "Butler", "cityID": 3933012, "cityName": "Hamilton" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929281376261, "text": "06/15@03:00 - Temp 56.6F, WC 56.6F. Wind 0.0mph ---, Gust 0.0mph. Bar 30.003in, Falling slowly. Rain 0.00in. Hum 89%. UV 0.0. SolarRad 0.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-75.68444444,39.695"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 174430712, "name": "Newark (DE) Weather", "screen_name": "NewarkWeather", "lang": "en", "location": "Newark, DE", "create_at": date("2010-08-03"), "description": "Hourly weather updates from Newark, DE", "followers_count": 66, "friends_count": 8, "statues_count": 49686 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Delaware, USA", "id": "3f5897b87d2bf56c", "name": "Delaware", "place_type": "admin", "bounding_box": rectangle("-75.788756,38.45104 -74.984165,39.839007") }, "geo_tag": { "stateID": 10, "stateName": "Delaware", "countyID": 10003, "countyName": "New Castle" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929419796481, "text": "I don't wanna get kicked out ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 592390914, "name": "K.R", "screen_name": "_avila14_", "lang": "en", "location": "null", "create_at": date("2012-05-27"), "description": "Black/ Mexican ❤️ | sc; avilakeilah", "followers_count": 552, "friends_count": 527, "statues_count": 27755 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lufkin, TX", "id": "ef5f7bc53f4594ea", "name": "Lufkin", "place_type": "city", "bounding_box": rectangle("-94.796056,31.264964 -94.669141,31.400021") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48005, "countyName": "Angelina", "cityID": 4845072, "cityName": "Lufkin" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929453350912, "text": "i'm bouta hibernate real good tonight. ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3269433788, "name": "Thad", "screen_name": "KuyaFinn", "lang": "en", "location": "Vallejo, CA", "create_at": date("2015-07-05"), "description": "null", "followers_count": 53, "friends_count": 64, "statues_count": 81 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929650454528, "text": "Wind 0.0 mph ---. Barometer 29.98 in, Steady. Temperature 74.5 °F. Rain today 0.00 in. Humidity 94%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.20444444,32.10361111"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 388446755, "name": "Butler Alabama WX", "screen_name": "ButlerAlabamaWx", "lang": "en", "location": "Butler Alabama", "create_at": date("2011-10-10"), "description": "null", "followers_count": 12, "friends_count": 4, "statues_count": 23370 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Alabama, USA", "id": "288de3df481163e8", "name": "Alabama", "place_type": "admin", "bounding_box": rectangle("-88.473228,30.144425 -84.888247,35.008029") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1023, "countyName": "Choctaw", "cityID": 111032, "cityName": "Butler" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974929780494336, "text": "https://t.co/EYwiprgQnO", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-121.47963,38.573505"), "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 2261358272, "name": "Jeffrey", "screen_name": "Jn7676Jeffrey", "lang": "en", "location": "Los Gatos, CA", "create_at": date("2013-12-25"), "description": "null", "followers_count": 312, "friends_count": 305, "statues_count": 571 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kupros Craft House", "id": "07d9dc2f5f880002", "name": "Kupros Craft House", "place_type": "poi", "bounding_box": rectangle("-121.4796301,38.573504899999996 -121.47963,38.573505") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6067, "countyName": "Sacramento", "cityID": 664000, "cityName": "Sacramento" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974930321543168, "text": "@draagonfly_ we think alike, you and i", "in_reply_to_status": 742974843423973377, "in_reply_to_user": 3272388632, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3272388632 }}, "user": { "id": 2733452294, "name": "[based godot]", "screen_name": "godot_co", "lang": "en", "location": "null", "create_at": date("2014-08-14"), "description": "jen/jenny|she/her| DMMD/Lamento/tsoa Storenvy: http://godotshop.storenvy.com", "followers_count": 54, "friends_count": 45, "statues_count": 16115 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974930569035776, "text": "Wind 0.0 mph ---. Barometer 29.909 in, Steady. Temperature 74.9 °F. Rain today 0.00 in. Humidity 98%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.89833333,31.58694444"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 67113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28037, "countyName": "Franklin" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974930602622977, "text": "Wind 1.0 mph SE. Barometer 29.747 in, Steady. Temperature 80.5 °F. Rain today 0.00 in. Humidity 80%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-90.46305556,38.67972222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 160005326, "name": "Whispering Hills Wx", "screen_name": "whhillsweather", "lang": "en", "location": "Creve Coeur, MO", "create_at": date("2010-06-26"), "description": "Whispering Hills weather updated every hour!", "followers_count": 119, "friends_count": 0, "statues_count": 113974 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Creve Coeur, MO", "id": "c3d0d06760ca937b", "name": "Creve Coeur", "place_type": "city", "bounding_box": rectangle("-90.491849,38.639901 -90.392072,38.696309") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29189, "countyName": "St. Louis", "cityID": 2917272, "cityName": "Creve Coeur" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974930606796800, "text": "Temp: 73.6°F Wind:0.0mph Pressure: 29.953hpa Steady Rain Today 0.00in.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-89.23388889,31.62277778"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 29831712, "name": "Alan Clark", "screen_name": "n5pa", "lang": "en", "location": "ÜT: 31.62265,-89.23384", "create_at": date("2009-04-08"), "description": "Ham Radio, Contesting, DX, Astronomy, Weather, Raspberry Pi", "followers_count": 70, "friends_count": 24, "statues_count": 67113 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Mississippi, USA", "id": "43d2418301bf1a49", "name": "Mississippi", "place_type": "admin", "bounding_box": rectangle("-91.655009,30.146096 -88.097889,34.995968") }, "geo_tag": { "stateID": 28, "stateName": "Mississippi", "countyID": 28067, "countyName": "Jones", "cityID": 2822020, "cityName": "Ellisville" } }
+{ "create_at": datetime("2016-06-15T00:00:01.000Z"), "id": 742974930682322945, "text": "Having hoes ain't wassup anymore ����", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1029402637, "name": "k e y a n t t a✨", "screen_name": "keyantta_", "lang": "en", "location": "Dawson, GA", "create_at": date("2012-12-22"), "description": "SE17IOR ❤️✊ | SC : Yantta15 ☄|", "followers_count": 2947, "friends_count": 2836, "statues_count": 88280 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dawson, GA", "id": "02b29041c3743b5a", "name": "Dawson", "place_type": "city", "bounding_box": rectangle("-84.458247,31.757388 -84.423089,31.791028") }, "geo_tag": { "stateID": 13, "stateName": "Georgia", "countyID": 13273, "countyName": "Terrell", "cityID": 1321912, "cityName": "Dawson" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931063951360, "text": "yes https://t.co/5jyaI7syyP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 3837240928, "name": "Victoria Schmidt", "screen_name": "victoriashitttt", "lang": "en", "location": "null", "create_at": date("2015-10-01"), "description": "you'll never know the value of a moment until it's a memory❤️", "followers_count": 224, "friends_count": 501, "statues_count": 943 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Porte, TX", "id": "01e986b204ff5847", "name": "La Porte", "place_type": "city", "bounding_box": rectangle("-95.113868,29.601579 -95.000382,29.730034") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4841440, "cityName": "La Porte" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931126849536, "text": "\"I wish you looked at me how you're looking at your ice cream\" @DMilfian ������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 600322050 }}, "user": { "id": 264349613, "name": "2bunz. ✨", "screen_name": "stephhhanieee__", "lang": "en", "location": "null", "create_at": date("2011-03-11"), "description": "D.Navarrete", "followers_count": 1047, "friends_count": 814, "statues_count": 21238 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Diego, CA", "id": "a592bd6ceb1319f7", "name": "San Diego", "place_type": "city", "bounding_box": rectangle("-117.282538,32.53962 -116.92744,33.080404") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6073, "countyName": "San Diego", "cityID": 666000, "cityName": "San Diego" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931143688193, "text": "I wanna call my best f but she hates my ass", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1605186810, "name": "gee", "screen_name": "g6briella", "lang": "en", "location": "yay area ", "create_at": date("2013-07-18"), "description": "do right and kill everything", "followers_count": 439, "friends_count": 257, "statues_count": 5476 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Vallejo, CA", "id": "7b5667de9caf1b92", "name": "Vallejo", "place_type": "city", "bounding_box": rectangle("-122.293347,38.060373 -122.167098,38.169181") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6095, "countyName": "Solano", "cityID": 681666, "cityName": "Vallejo" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931168821248, "text": "I do https://t.co/cbWvL5WdKr", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 1161346130, "name": "−♡♡", "screen_name": "emmamuhreego", "lang": "en", "location": "null", "create_at": date("2013-02-08"), "description": "IG : EMMAMUHREE", "followers_count": 1326, "friends_count": 907, "statues_count": 9005 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Los Angeles, CA", "id": "3b77caf94bfc81fe", "name": "Los Angeles", "place_type": "city", "bounding_box": rectangle("-118.668404,33.704538 -118.155409,34.337041") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 644000, "cityName": "Los Angeles" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931227545601, "text": "#Trump in 2016 Temp:Crab Orchard, Ky.\":70.3°F Wind:0.0mph Pressure: 29.94hpa Steady Rain Today 0.00in. Forecast: Fine weather", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.51194444,37.47472222"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "Trump" }}, "user": { "id": 115110145, "name": "Carl King", "screen_name": "AppaloosaGuy", "lang": "en", "location": "Crab Orchard, Ky.", "create_at": date("2010-02-17"), "description": "To find the up to date weather conditions im my little town and the Cedar Creek Lake area, go to http://www.kbweather.net.", "followers_count": 81, "friends_count": 18, "statues_count": 324240 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kentucky, USA", "id": "6ffcf3b0b904bbcb", "name": "Kentucky", "place_type": "admin", "bounding_box": rectangle("-89.57151,36.497129 -81.964971,39.147359") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21137, "countyName": "Lincoln" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931504402432, "text": "I think Oliver Heldens is going to be one huge gloving meetup if I'm correct and I'm excited as fuck", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2153559776, "name": "Mørals", "screen_name": "AyeeDaps", "lang": "en", "location": "Belton ¥ Austin", "create_at": date("2013-10-24"), "description": "[FAM] • MøralsMusic • Concert, Festival, & Artist Promo • AfterDark Ent • moralsmusic1997@gmail.com for business #TexasEDMFamily #AfterDarkTexas #grimematters", "followers_count": 907, "friends_count": 569, "statues_count": 23708 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Belton, TX", "id": "19a934ea4e2c6407", "name": "Belton", "place_type": "city", "bounding_box": rectangle("-97.538011,31.014722 -97.426188,31.140176") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48027, "countyName": "Bell", "cityID": 4807492, "cityName": "Belton" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974931575689216, "text": "Someone snapchat me", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2965316112, "name": "♡", "screen_name": "meech_0", "lang": "en", "location": "rolling up in the 909", "create_at": date("2015-01-07"), "description": "RIP Esther & John Reyes ♡", "followers_count": 584, "friends_count": 665, "statues_count": 6962 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Fontana, CA", "id": "d838baf51d1c629d", "name": "Fontana", "place_type": "city", "bounding_box": rectangle("-117.524329,34.033432 -117.401067,34.182889") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 624680, "cityName": "Fontana" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974932280315904, "text": "@OGtay_ @cmainey DUH !!!!! she ain't bout that life tho ��", "in_reply_to_status": 742974616747003904, "in_reply_to_user": 1064823577, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1064823577, 567829286 }}, "user": { "id": 295507816, "name": "PRINCE$$", "screen_name": "arimilan_", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2011-05-08"), "description": "♊️ | csula19 | sc: arimilann", "followers_count": 2343, "friends_count": 1714, "statues_count": 30470 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Carson, CA", "id": "56909099a1d44057", "name": "Carson", "place_type": "city", "bounding_box": rectangle("-118.290541,33.792887 -118.205584,33.886336") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 611530, "cityName": "Carson" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974932745867267, "text": "Temp 53.4° Hi/Lo 53.5/51.3 Rng 2.2° WC 53.4° Hmd 91% Rain 0.00\" Storm 0.00\" BAR 29.921 Rising DP 50.8° Wnd 0mph Dir --- Gst 1mph Snw 0\"", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-71.54277778,42.22805556"), "retweet_count": 0, "lang": "in", "is_retweet": false, "user": { "id": 3437237963, "name": "Hopkinton Weather", "screen_name": "HopkintonWX", "lang": "en", "location": "Hopkinton, MA", "create_at": date("2015-08-23"), "description": "Weather info for Hopkinton, MA. Current weather condition updates every hour. Additional weather info throughout the day. Like us on Facebook. Since 8/23/15", "followers_count": 73, "friends_count": 124, "statues_count": 19507 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hopkinton, MA", "id": "7f22ddc6360b50e3", "name": "Hopkinton", "place_type": "city", "bounding_box": rectangle("-71.596679,42.184354 -71.460478,42.267443") }, "geo_tag": { "stateID": 25, "stateName": "Massachusetts", "countyID": 25017, "countyName": "Middlesex", "cityID": 2531050, "cityName": "Hopkinton" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974932792008704, "text": "Wind 0.0 mph SSW. Barometer 29.847 in, Falling slowly. Temperature 69.0 °F. Rain today 0.00 in. Humidity 100%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-85.37027778,38.40805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2327800314, "name": "LaGrange Weather ⚡", "screen_name": "LaGrangeWeather", "lang": "en", "location": "LaGrange Kentucky USA", "create_at": date("2014-02-04"), "description": "Local weather conditions for LaGrange Kentucky.", "followers_count": 61, "friends_count": 26, "statues_count": 20572 }, "place": { "country": "United States", "country_code": "United States", "full_name": "La Grange, KY", "id": "00c6f40e474c26b4", "name": "La Grange", "place_type": "city", "bounding_box": rectangle("-85.427193,38.345135 -85.342705,38.440913") }, "geo_tag": { "stateID": 21, "stateName": "Kentucky", "countyID": 21185, "countyName": "Oldham", "cityID": 2143480, "cityName": "La Grange" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933052051456, "text": "@StonedAss_Weedy �������� https://t.co/Y4SmoP7uWg", "in_reply_to_status": 742973746294706177, "in_reply_to_user": 277744049, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 277744049 }}, "user": { "id": 169838305, "name": "Untouchable ❤", "screen_name": "YouLove_Tia", "lang": "en", "location": "ErieHere ", "create_at": date("2010-07-23"), "description": "RIP DIAMOND | nardica bestfriend and elmo here❤️ Dm's are not in service. Reese Wife ❤", "followers_count": 5973, "friends_count": 2131, "statues_count": 197347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Cleveland, OH", "id": "0eb9676d24b211f1", "name": "Cleveland", "place_type": "city", "bounding_box": rectangle("-81.877771,41.392684 -81.533163,41.599195") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39035, "countyName": "Cuyahoga", "cityID": 3916000, "cityName": "Cleveland" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933219835905, "text": "#TweetLyricsThatHaveToBeShouted FUCK DONALD TRUMP, FUCK DONALD TRUMP, YEAH NIGGA FUCK DONALD TRUMP", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TweetLyricsThatHaveToBeShouted" }}, "user": { "id": 924667892, "name": "Dayanna", "screen_name": "dayannabanana_", "lang": "en", "location": "null", "create_at": date("2012-11-03"), "description": "Stay cool, hang loose, admit nothing. -Cara Delevingne", "followers_count": 659, "friends_count": 419, "statues_count": 41376 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Riverside, CA", "id": "6ba08e404aed471f", "name": "Riverside", "place_type": "city", "bounding_box": rectangle("-117.523867,33.85216 -117.271365,34.019484") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6065, "countyName": "Riverside", "cityID": 662000, "cityName": "Riverside" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933312147456, "text": "77.9F (Feels: 85.0F) - Humidity: 90% - Wind: 0.9mph ESE - Gust: 1.1mph - Pressure: 29.960in #weather https://t.co/75yzNJ23Eh", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-92.218675,29.875016"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "weather" }}, "user": { "id": 103092003, "name": "Mouton Cove Weather", "screen_name": "moutoncovewx", "lang": "en", "location": "Mouton Cove, Abbeville, LA", "create_at": date("2010-01-08"), "description": "Live 10-minute weather data from a personal weather station in Mouton Cove, Abbeville, LA.", "followers_count": 221, "friends_count": 18, "statues_count": 242551 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Louisiana, USA", "id": "1c73ebb264e145ee", "name": "Louisiana", "place_type": "admin", "bounding_box": rectangle("-94.043628,28.855128 -88.758389,33.019544") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22113, "countyName": "Vermilion" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933412761600, "text": "What's the point of buying a Disney pass when I won't have anyone to go with ☹️", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2473452397, "name": "emma", "screen_name": "xmmxxlvx", "lang": "en", "location": "chillin in cedar rapids", "create_at": date("2014-05-01"), "description": "xicana ✨", "followers_count": 754, "friends_count": 580, "statues_count": 29380 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lancaster, CA", "id": "04291149b14feb9d", "name": "Lancaster", "place_type": "city", "bounding_box": rectangle("-118.280798,34.630945 -118.045755,34.740709") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 640130, "cityName": "Lancaster" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933727338496, "text": "@_ashleyyyyy2_ that's the way to go", "in_reply_to_status": 742973394526863360, "in_reply_to_user": 2261028133, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2261028133 }}, "user": { "id": 2255699660, "name": "Kyles", "screen_name": "kylieshaw_", "lang": "en", "location": "null", "create_at": date("2013-12-20"), "description": "I'm just livin life and having funツ", "followers_count": 797, "friends_count": 764, "statues_count": 11218 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Yucaipa, CA", "id": "99da29473eb4f79a", "name": "Yucaipa", "place_type": "city", "bounding_box": rectangle("-117.126742,34.003904 -116.975005,34.077386") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 687042, "cityName": "Yucaipa" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974933916131329, "text": "@DaphooMarie ������", "in_reply_to_status": 742972572631990272, "in_reply_to_user": 2638910753, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 2638910753 }}, "user": { "id": 374052068, "name": "Mira✨", "screen_name": "Shamiraaaa_", "lang": "en", "location": "Gaige⭐️", "create_at": date("2011-09-15"), "description": "TXSU'19 ... #RIPdre sc:lovelymiraaa", "followers_count": 3635, "friends_count": 2009, "statues_count": 40782 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missouri City, TX", "id": "798522f39f66d411", "name": "Missouri City", "place_type": "city", "bounding_box": rectangle("-95.586805,29.506247 -95.431045,29.642322") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48157, "countyName": "Fort Bend", "cityID": 4848804, "cityName": "Missouri City" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974934104834050, "text": "'16 �� https://t.co/uNI59SXPc6", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user": { "id": 417583923, "name": "king", "screen_name": "mvriokang_", "lang": "en", "location": "ig: mvrio.kang", "create_at": date("2011-11-20"), "description": "different is good...don't fit in. | #UA20", "followers_count": 1121, "friends_count": 976, "statues_count": 43324 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Hueytown, AL", "id": "1f6120147686a129", "name": "Hueytown", "place_type": "city", "bounding_box": rectangle("-87.066047,33.402208 -86.918524,33.494969") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1073, "countyName": "Jefferson", "cityID": 136448, "cityName": "Hueytown" } }
+{ "create_at": datetime("2016-06-15T00:00:02.000Z"), "id": 742974934910111745, "text": "You flexn? Post that bank account broke boi", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2937604989, "name": "Mike.", "screen_name": "Lyoooons", "lang": "en", "location": "USAA", "create_at": date("2014-12-22"), "description": "SELF MADE SELF PAID | DMV | TrueVisions", "followers_count": 398, "friends_count": 305, "statues_count": 7213 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Frederick, MD", "id": "ec3b8b62828d8f6a", "name": "Frederick", "place_type": "city", "bounding_box": rectangle("-77.476711,39.369128 -77.359293,39.494945") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24021, "countyName": "Frederick", "cityID": 2430325, "cityName": "Frederick" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974935262482433, "text": "Being up all night starting to become regular", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1941033432, "name": "$ack Religiou$", "screen_name": "TheSamPenn", "lang": "en", "location": "Manuvering", "create_at": date("2013-10-06"), "description": "Everyday we celebrating b/c we elevating #BleedMOB #TXST18", "followers_count": 835, "friends_count": 506, "statues_count": 19167 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Marcos, TX", "id": "f95ca607d76de8c6", "name": "San Marcos", "place_type": "city", "bounding_box": rectangle("-97.993537,29.840708 -97.876846,29.950505") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48209, "countyName": "Hays", "cityID": 4865600, "cityName": "San Marcos" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974935363117056, "text": "@xoxolejla tomorrow", "in_reply_to_status": 742974712633036800, "in_reply_to_user": 382897032, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 382897032 }}, "user": { "id": 377812654, "name": "inesa pačo", "screen_name": "inesapaco", "lang": "en", "location": "seattle ☔️", "create_at": date("2011-09-21"), "description": "null", "followers_count": 642, "friends_count": 337, "statues_count": 15512 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lynnwood, WA", "id": "3912e7cd4cf62c39", "name": "Lynnwood", "place_type": "city", "bounding_box": rectangle("-122.337717,47.802196 -122.262542,47.853772") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53061, "countyName": "Snohomish", "cityID": 5340840, "cityName": "Lynnwood" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974935518306305, "text": "Off for a week! Needed this break!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 795259238, "name": "Vinesh", "screen_name": "vinesh__patel", "lang": "en", "location": "Phoenix, AZ", "create_at": date("2012-08-31"), "description": "ASU/NRC ⚡️", "followers_count": 213, "friends_count": 264, "statues_count": 4244 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Phoenix, AZ", "id": "5c62ffb0f0f3479d", "name": "Phoenix", "place_type": "city", "bounding_box": rectangle("-112.323914,33.29026 -111.925439,33.815465") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4013, "countyName": "Maricopa", "cityID": 455000, "cityName": "Phoenix" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974935631560709, "text": "It depends https://t.co/1sqPISgKjm", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 422259454, "name": "Ponyboy", "screen_name": "xanny__pacquiao", "lang": "en", "location": "Stoner's Island", "create_at": date("2011-11-26"), "description": "SUSLA16", "followers_count": 4005, "friends_count": 4205, "statues_count": 257813 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Shreveport, LA", "id": "4ec71fc3f2579572", "name": "Shreveport", "place_type": "city", "bounding_box": rectangle("-93.938883,32.332455 -93.679985,32.589759") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22017, "countyName": "Caddo", "cityID": 2270000, "cityName": "Shreveport" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974935744819202, "text": "@_dariusclark @Jerrae_ I know she so serious ������", "in_reply_to_status": 742974855402954756, "in_reply_to_user": 928938079, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 928938079, 3920073072 }}, "user": { "id": 2175390674, "name": "logan taylor", "screen_name": "logantaylorrr", "lang": "en", "location": "null", "create_at": date("2013-11-04"), "description": "#UCO2020 • Insta __logantaylor • Snap logan_taylor14 •", "followers_count": 1317, "friends_count": 572, "statues_count": 25519 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ardmore, OK", "id": "2ffc75cc0e452636", "name": "Ardmore", "place_type": "city", "bounding_box": rectangle("-97.178022,34.129246 -97.101508,34.219901") }, "geo_tag": { "stateID": 40, "stateName": "Oklahoma", "countyID": 40019, "countyName": "Carter", "cityID": 4002600, "cityName": "Ardmore" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974936503963648, "text": "One of the best investments I've made is in my weighted jump rope!", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 211774534, "name": "LO the latin Rho", "screen_name": "Prima_Donna5", "lang": "en", "location": "Nashville, TN", "create_at": date("2010-11-03"), "description": "Beauty Queen that loves the Lord God almighty |ΣΓΡ| Future doctor| Latina and proud|", "followers_count": 2096, "friends_count": 1582, "statues_count": 54429 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Murfreesboro, TN", "id": "19e2bff2e89dc38e", "name": "Murfreesboro", "place_type": "city", "bounding_box": rectangle("-86.505805,35.751433 -86.313415,35.943407") }, "geo_tag": { "stateID": 47, "stateName": "Tennessee", "countyID": 47149, "countyName": "Rutherford", "cityID": 4751560, "cityName": "Murfreesboro" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974936524967936, "text": "Like there's a lot of crazy shit built up in my head and when I think someone can understand it, I let it all out.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 4635403710, "name": "Vic.", "screen_name": "ImportantPandaa", "lang": "en", "location": "Visalia, CA", "create_at": date("2015-12-22"), "description": "2015.", "followers_count": 238, "friends_count": 342, "statues_count": 2598 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Visalia, CA", "id": "981989c5b11d5119", "name": "Visalia", "place_type": "city", "bounding_box": rectangle("-119.414399,36.26939 -119.224654,36.369582") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6107, "countyName": "Tulare", "cityID": 682954, "cityName": "Visalia" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974936784994306, "text": "Unmarked Helicopters by Soul Coughing is #nowplaying in Thumb Coast Brewing Co., Port Huron.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-82.422367,42.975489"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "nowplaying" }}, "user": { "id": 497145453, "name": "Virtual Jukebox Live", "screen_name": "VJB_LiveFeed", "lang": "en", "location": "Richmond, Surrey, UK", "create_at": date("2012-02-19"), "description": "Live stream of music playing at @VirtualJukebox locations. We make background music more engaging.", "followers_count": 1647, "friends_count": 0, "statues_count": 1736160 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Port Huron, MI", "id": "010750c237347ebb", "name": "Port Huron", "place_type": "city", "bounding_box": rectangle("-82.514554,42.935388 -82.418396,43.040084") }, "geo_tag": { "stateID": 26, "stateName": "Michigan", "countyID": 26147, "countyName": "St. Clair", "cityID": 2665820, "cityName": "Port Huron" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974936784998400, "text": "@crusty_pasta me https://t.co/cn5XwObBBe", "in_reply_to_status": -1, "in_reply_to_user": 633169522, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "user_mentions": {{ 633169522 }}, "user": { "id": 2183728466, "name": "lea", "screen_name": "lea_angione", "lang": "en", "location": "Las Vegas, NV", "create_at": date("2013-11-08"), "description": "..", "followers_count": 322, "friends_count": 292, "statues_count": 10829 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Las Vegas, NV", "id": "5c2b5e46ab891f07", "name": "Las Vegas", "place_type": "city", "bounding_box": rectangle("-115.384091,36.129459 -115.062159,36.336371") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3240000, "cityName": "Las Vegas" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974937426714624, "text": "Yeahhhh . Oops another L https://t.co/WvAF44Kkhx", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2558008066, "name": "Ricco Stackz ⁶™", "screen_name": "OfficialRStackz", "lang": "en", "location": "Dubai, United Arab Emirates", "create_at": date("2014-05-21"), "description": "in love wit the guap .. why lil mama think i like her ?? - StackzSeason2 is On The Way", "followers_count": 511, "friends_count": 793, "statues_count": 5006 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lafayette, LA", "id": "276336654aa4f87a", "name": "Lafayette", "place_type": "city", "bounding_box": rectangle("-92.158649,30.115967 -91.963704,30.296191") }, "geo_tag": { "stateID": 22, "stateName": "Louisiana", "countyID": 22055, "countyName": "Lafayette", "cityID": 2240735, "cityName": "Lafayette" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974937682616321, "text": "My shawty said she tired of juggin' niggas ��������", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 386280831, "name": "Miss Haze", "screen_name": "_hazeheaven", "lang": "en", "location": "Atlanta, GA", "create_at": date("2011-10-06"), "description": "Snap Chat: iammisshaze", "followers_count": 1443, "friends_count": 571, "statues_count": 33662 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dothan, AL", "id": "d2981a0f3d8eddd7", "name": "Dothan", "place_type": "city", "bounding_box": rectangle("-85.515405,31.130472 -85.334617,31.33397") }, "geo_tag": { "stateID": 1, "stateName": "Alabama", "countyID": 1069, "countyName": "Houston", "cityID": 121184, "cityName": "Dothan" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974937703534593, "text": "Check out some pics of @Themowglis during their recent show at @indysf from one of our own @lstsss for @DoTheBay // https://t.co/7e5JLFuAtA", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 112567644, 74572689, 282610007, 278741423 }}, "user": { "id": 783134034, "name": "Showbams", "screen_name": "showbams", "lang": "en", "location": "San Francisco/Los Angeles", "create_at": date("2012-08-26"), "description": "Discover, experience, relive California's music scene & beyond.", "followers_count": 1111, "friends_count": 700, "statues_count": 4283 }, "place": { "country": "United States", "country_code": "United States", "full_name": "San Francisco, CA", "id": "5a110d312052166f", "name": "San Francisco", "place_type": "city", "bounding_box": rectangle("-122.514926,37.708075 -122.357031,37.833238") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6075, "countyName": "San Francisco", "cityID": 667000, "cityName": "San Francisco" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974937707728896, "text": "Wind 0.0 mph ---. Barometer 29.99 in, Steady. Temperature 56.7 °F. Rain today 0.00 in. Humidity 76%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-73.87861111,42.9125"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 209623238, "name": "Scott V", "screen_name": "vannossc", "lang": "en", "location": "Capital District NY", "create_at": date("2010-10-29"), "description": "null", "followers_count": 32, "friends_count": 121, "statues_count": 160553 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Ballston Lake, NY", "id": "01af0ed50454b550", "name": "Ballston Lake", "place_type": "city", "bounding_box": rectangle("-73.921934,42.895105 -73.857902,42.942582") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36093, "countyName": "Schenectady" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974938122948608, "text": "����// you're a very funny/nice guy!��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 613835645, "name": "ty", "screen_name": "Boom_tylynn", "lang": "en", "location": "null", "create_at": date("2012-06-20"), "description": "sc// tylynnnnnn", "followers_count": 1010, "friends_count": 776, "statues_count": 36775 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Sunrise Manor, NV", "id": "a1839e9c3b5abff6", "name": "Sunrise Manor", "place_type": "city", "bounding_box": rectangle("-115.116116,36.107582 -115.006139,36.261926") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3271400, "cityName": "Sunrise Manor" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974938676629504, "text": "#MomsDemandAction https://t.co/Ofu5ewoO9H", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "und", "is_retweet": false, "hashtags": {{ "MomsDemandAction" }}, "user": { "id": 20939926, "name": "Suzee DellaDonna,PhD", "screen_name": "sdelladonna", "lang": "en", "location": "Los Angeles County, CA", "create_at": date("2009-02-15"), "description": "Mom, Nana, PhD Psych", "followers_count": 942, "friends_count": 1845, "statues_count": 13223 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Covina, CA", "id": "bcd6f5a92e969e33", "name": "Covina", "place_type": "city", "bounding_box": rectangle("-117.927041,34.068114 -117.836898,34.106972") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 616742, "cityName": "Covina" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974938810884096, "text": "Let's just all be friends", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2738944402, "name": "kinky karls", "screen_name": "karlavalen12", "lang": "en", "location": "null", "create_at": date("2014-08-10"), "description": "unemployed, not if you count how I work it", "followers_count": 317, "friends_count": 298, "statues_count": 3256 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Pomona, CA", "id": "6f24e28976abe656", "name": "Pomona", "place_type": "city", "bounding_box": rectangle("-117.834663,34.018283 -117.711171,34.112618") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6037, "countyName": "Los Angeles", "cityID": 658072, "cityName": "Pomona" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974938919915521, "text": "#GeorgeTakei to #LGBT #community: Fight for strict #gun #laws https://t.co/gkzILGYMly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "GeorgeTakei", "LGBT", "community", "gun", "laws" }}, "user": { "id": 1412172103, "name": "Sister Lawna Jocqui", "screen_name": "SisterLawna", "lang": "en", "location": "Dallas, Texas", "create_at": date("2013-05-07"), "description": "Fully Professed Sister with The DFW Sisters of Perpetual Indulgence (R), a part of the Worldwide Order of the Sisters of Perpetual Indulgence (c).", "followers_count": 990, "friends_count": 1892, "statues_count": 4232 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Dallas, TX", "id": "18810aa5b43e76c7", "name": "Dallas", "place_type": "city", "bounding_box": rectangle("-96.977527,32.620678 -96.54598,33.019039") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48113, "countyName": "Dallas", "cityID": 4819000, "cityName": "Dallas" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974939075117056, "text": "I met so many people tonight that I'll probably never remember & also hbd to the one & only @floralloyd14 ��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1419655345 }}, "user": { "id": 477317497, "name": "Emily Earl", "screen_name": "EmilyEarl2", "lang": "en", "location": "MT", "create_at": date("2012-01-28"), "description": "But I say unto you, love your enemies, bless them that curse you, do good to them that hate you, and pray for them that despitefully use you and prosecute you", "followers_count": 444, "friends_count": 817, "statues_count": 2595 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Missoula, MT", "id": "00427d4a5c4a1fc3", "name": "Missoula", "place_type": "city", "bounding_box": rectangle("-114.120325,46.780202 -113.941975,46.977998") }, "geo_tag": { "stateID": 30, "stateName": "Montana", "countyID": 30063, "countyName": "Missoula", "cityID": 3050200, "cityName": "Missoula" } }
+{ "create_at": datetime("2016-06-15T00:00:03.000Z"), "id": 742974939213484034, "text": "I believe ppl like this are the ones enticing ppl to act violently, he's a pastor... @frodofied https://t.co/PerEEAZYbe", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 17167572 }}, "user": { "id": 321511381, "name": "A-Tron Guapo", "screen_name": "AarDayburnin", "lang": "en", "location": "Eugene, Oregon", "create_at": date("2011-06-21"), "description": "RIP CITY, Philly cheesesteak connoisseur, basketball fanatic, lover of sound", "followers_count": 543, "friends_count": 926, "statues_count": 23491 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Harrisburg, OR", "id": "43a78a84849946bd", "name": "Harrisburg", "place_type": "city", "bounding_box": rectangle("-123.179834,44.260945 -123.154251,44.284709") }, "geo_tag": { "stateID": 41, "stateName": "Oregon", "countyID": 41043, "countyName": "Linn", "cityID": 4132550, "cityName": "Harrisburg" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974939486130177, "text": "maybe shit will be better when I wake up��", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 373602502, "name": "lil$lime", "screen_name": "tiaraxchan", "lang": "en", "location": "null", "create_at": date("2011-09-14"), "description": "gold bottle. #LongLiveShinn ❤", "followers_count": 2327, "friends_count": 1751, "statues_count": 110455 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Charlotte, NC", "id": "512a8a4a4c4b4be0", "name": "Charlotte", "place_type": "city", "bounding_box": rectangle("-81.046876,35.001706 -80.646695,35.416412") }, "geo_tag": { "stateID": 37, "stateName": "North Carolina", "countyID": 37119, "countyName": "Mecklenburg", "cityID": 3712000, "cityName": "Charlotte" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974939792330752, "text": "@thatgeeklikes was waiting for u text back so I could say goodnight but I'll just leave u on read (-:", "in_reply_to_status": 742974746208436229, "in_reply_to_user": 3215642568, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3215642568 }}, "user": { "id": 3275581412, "name": "mya kristine", "screen_name": "myalozanoo", "lang": "en", "location": "probably @ practice ", "create_at": date("2015-07-11"), "description": "null", "followers_count": 264, "friends_count": 313, "statues_count": 5923 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Houston, TX", "id": "1c69a67ad480e1b1", "name": "Houston", "place_type": "city", "bounding_box": rectangle("-95.823268,29.522325 -95.069705,30.154665") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48201, "countyName": "Harris", "cityID": 4835000, "cityName": "Houston" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974939985240064, "text": "Wind 4.8 mph SSE. Barometer 29.815 in, Rising slowly. Temperature 77.1 °F. Rain today 0.00 in. Humidity 87%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-96.95222222,32.42416667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2584611722, "name": "EJ Tews", "screen_name": "tews2com", "lang": "en", "location": "null", "create_at": date("2014-06-23"), "description": "null", "followers_count": 15, "friends_count": 54, "statues_count": 9185 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Midlothian, TX", "id": "5c018b963d981f64", "name": "Midlothian", "place_type": "city", "bounding_box": rectangle("-97.023555,32.412865 -96.907614,32.538698") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48139, "countyName": "Ellis", "cityID": 4848096, "cityName": "Midlothian" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940106854400, "text": "\"Ur really pissing me off\" https://t.co/SveyMWzagM", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 2797114093, "name": "Zoë", "screen_name": "itszoeezy", "lang": "en", "location": "Galveston, TX", "create_at": date("2014-09-07"), "description": "libra ♎️ Latina || Astro's right fielder ❤️⚾️ || Effective Clothing", "followers_count": 578, "friends_count": 382, "statues_count": 18265 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Galveston, TX", "id": "632eeebc87aecd57", "name": "Galveston", "place_type": "city", "bounding_box": rectangle("-94.880809,29.239602 -94.764742,29.335548") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48167, "countyName": "Galveston", "cityID": 4828068, "cityName": "Galveston" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940115271680, "text": "@SarahMichhhh i honestly have done a good 2/3rd's of these", "in_reply_to_status": 742966613092270081, "in_reply_to_user": 3355531720, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 3355531720 }}, "user": { "id": 3355531720, "name": "sarah", "screen_name": "SarahMichhhh", "lang": "en", "location": "borona, balifornia", "create_at": date("2015-07-02"), "description": "sagittarius | motivated writer who has left writer's block | welcome to the graveyard of ambition | 2", "followers_count": 1220, "friends_count": 904, "statues_count": 8057 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Big Bear Lake, CA", "id": "17fe6350a3570c69", "name": "Big Bear Lake", "place_type": "city", "bounding_box": rectangle("-116.962579,34.223549 -116.842586,34.262952") }, "geo_tag": { "stateID": 6, "stateName": "California", "countyID": 6071, "countyName": "San Bernardino", "cityID": 606434, "cityName": "Big Bear Lake" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940174028800, "text": "@Grogboyy now he's listing off artists from the 60's giving her shit every time she doesn't know one, which is all artists he said", "in_reply_to_status": 742974721961103360, "in_reply_to_user": 2993786844, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 2993786844 }}, "user": { "id": 2993786844, "name": "Groggett", "screen_name": "Grogboyy", "lang": "en", "location": "Flagstaff, AZ", "create_at": date("2015-01-23"), "description": "I like to sit in high places", "followers_count": 324, "friends_count": 240, "statues_count": 11214 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Flagstaff, AZ", "id": "fdbee5cdec8043ac", "name": "Flagstaff", "place_type": "city", "bounding_box": rectangle("-111.710027,35.122005 -111.560272,35.243495") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4005, "countyName": "Coconino", "cityID": 423620, "cityName": "Flagstaff" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940283080704, "text": "Slum anthem", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 562624334, "name": "Kebron James", "screen_name": "KeeshTheFiend", "lang": "en", "location": "Tucson AZ", "create_at": date("2012-04-24"), "description": "I used to play 2k before my xbox broke now I just be tweetin. also cavs in 7", "followers_count": 377, "friends_count": 232, "statues_count": 22783 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Tucson, AZ", "id": "013379ee5729a5e6", "name": "Tucson", "place_type": "city", "bounding_box": rectangle("-111.083219,32.057802 -110.747928,32.320979") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4019, "countyName": "Pima", "cityID": 477000, "cityName": "Tucson" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940396326913, "text": "my house is decorated with all of my memories and past accomplishments from when i was born to now and i am insanely sad.", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 938698298, "name": "Wright, Makai", "screen_name": "StraightFIexing", "lang": "en", "location": "null", "create_at": date("2012-11-10"), "description": "ev & ewu", "followers_count": 1515, "friends_count": 315, "statues_count": 30450 }, "place": { "country": "United States", "country_code": "United States", "full_name": "South Hill, WA", "id": "3335feca3c8c1f17", "name": "South Hill", "place_type": "city", "bounding_box": rectangle("-122.357589,47.078497 -122.234973,47.160074") }, "geo_tag": { "stateID": 53, "stateName": "Washington", "countyID": 53053, "countyName": "Pierce", "cityID": 5365922, "cityName": "South Hill" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940765388800, "text": "Temp: 65.7°F - Dew Point: 65.7° - Wind: ENE @ 3.4 mph - Gust: 6.3 - Rain Today: 1.33in. - Pressure: 29.71in, - Trend: Falling slowly", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-88.35805556,42.54"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 3234007289, "name": "pell lake weather", "screen_name": "Pell_Lake_Wx", "lang": "en", "location": "Pell Lake, Wi.", "create_at": date("2015-05-04"), "description": "Weather Stats", "followers_count": 10, "friends_count": 13, "statues_count": 19175 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bloomfield, WI", "id": "0189a1fc0768d31d", "name": "Bloomfield", "place_type": "city", "bounding_box": rectangle("-88.388678,42.518683 -88.334574,42.561132") }, "geo_tag": { "stateID": 55, "stateName": "Wisconsin", "countyID": 55127, "countyName": "Walworth", "cityID": 5508265, "cityName": "Bloomfield" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974940996067333, "text": "#SupplyChain #Job alert: Sr. Distribution Manager, D2C | LEGO Group | #Enfield, CT https://t.co/OelaREDQfs #LEGO #Jobs #Hiring", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-72.5441445,42.0027181"), "retweet_count": 0, "lang": "fr", "is_retweet": false, "hashtags": {{ "SupplyChain", "Job", "Enfield", "LEGO", "Jobs", "Hiring" }}, "user": { "id": 1898069832, "name": "LEGO_Careers", "screen_name": "LEGO_Careers", "lang": "en", "location": "null", "create_at": date("2013-09-23"), "description": "Our mission is to inspire and develop the builders of tomorrow. True to our spirit “Only the best is good enough”, we are a great company to work for.", "followers_count": 1872, "friends_count": 6, "statues_count": 1119 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Enfield, CT", "id": "012a05e354827864", "name": "Enfield", "place_type": "city", "bounding_box": rectangle("-72.62241,41.897952 -72.503313,42.034409") }, "geo_tag": { "stateID": 9, "stateName": "Connecticut", "countyID": 9003, "countyName": "Hartford", "cityID": 937770, "cityName": "Hazardville" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941113536513, "text": "Missn him like ��.... mad he fell asleep on me �� https://t.co/UKSWIX049B", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1335113233, "name": "NAI", "screen_name": "SLIMTHICK_XL", "lang": "en", "location": "CSU 19", "create_at": date("2013-04-07"), "description": "Sophomore in college ✨ Jersey bread ✨ Camden ✈ Baltimore ✨ unbothered", "followers_count": 966, "friends_count": 1027, "statues_count": 10299 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Camden, NJ", "id": "bec68a93372eb249", "name": "Camden", "place_type": "city", "bounding_box": rectangle("-75.13285,39.906751 -75.066575,39.96822") }, "geo_tag": { "stateID": 34, "stateName": "New Jersey", "countyID": 34007, "countyName": "Camden", "cityID": 3410000, "cityName": "Camden" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941319028736, "text": "SHE SHALLOW WITH THE PUSSY SHE SHALLooooOOOOWWWWW #TweetLyricsThatHaveToBeShouted", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "TweetLyricsThatHaveToBeShouted" }}, "user": { "id": 1639789369, "name": "princess jasmine", "screen_name": "jasmine_xooo", "lang": "en", "location": "Chicago, IL", "create_at": date("2013-08-01"), "description": "i planned to be the first black president but obama beat me to it", "followers_count": 375, "friends_count": 255, "statues_count": 7224 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Chicago, IL", "id": "1d9a5370a355ab0c", "name": "Chicago", "place_type": "city", "bounding_box": rectangle("-87.940033,41.644102 -87.523993,42.023067") }, "geo_tag": { "stateID": 17, "stateName": "Illinois", "countyID": 17031, "countyName": "Cook", "cityID": 1714000, "cityName": "Chicago" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941411344384, "text": "Temp 63.4°F Heat Index 63.4 °F RH 75% Wind 0.0 --- Gust 0.0 --- SLP 29.903 in Falling slowly Rain 0.00 in Solar 0 UV 0.0 #VAWX #SWVAWX #WVWX", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-81.2775,37.24166667"), "retweet_count": 0, "lang": "en", "is_retweet": false, "hashtags": {{ "VAWX", "SWVAWX", "WVWX" }}, "user": { "id": 514972191, "name": "Bluefield VA Weather", "screen_name": "kvabluef4", "lang": "en", "location": "Bluefield, VA", "create_at": date("2012-03-04"), "description": "24 HR WX for Bluefield, VA; CWOP ID AU402; Updates Hourly; Current Conditions Available; Maintained by Mark Williams @kq4q w/ Davis VP2 Plus & Cumulus v1.9.4", "followers_count": 108, "friends_count": 62, "statues_count": 36180 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Virginia, USA", "id": "5635c19c2b5078d1", "name": "Virginia", "place_type": "admin", "bounding_box": rectangle("-83.67529,36.540739 -75.16644,39.466012") }, "geo_tag": { "stateID": 51, "stateName": "Virginia", "countyID": 51185, "countyName": "Tazewell", "cityID": 5108152, "cityName": "Bluefield" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941730050049, "text": "@DeLenDominique @little_bair she think NO SLEEP GVNG a joke smh", "in_reply_to_status": 742974682194927616, "in_reply_to_user": 1318678004, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 1318678004, 1380209648 }}, "user": { "id": 926524279, "name": "雷吉·米切尔", "screen_name": "ReggieM32", "lang": "en", "location": " #HillTopOfTheWorld", "create_at": date("2012-11-04"), "description": "benji... I see ya vision", "followers_count": 919, "friends_count": 780, "statues_count": 19250 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Columbus, OH", "id": "3df0e3eb1e91170b", "name": "Columbus", "place_type": "city", "bounding_box": rectangle("-83.200169,39.832013 -82.771375,40.157354") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39049, "countyName": "Franklin", "cityID": 3918000, "cityName": "Columbus" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941948186624, "text": "Selfie bc I pierced my nose in Vegas today���� https://t.co/js3wervb6a", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 1512952435, "name": "❥❥❥", "screen_name": "___antoinetteee", "lang": "en", "location": "505", "create_at": date("2013-06-13"), "description": "I like long walks down the makeup aisle @youphilnme ❤️", "followers_count": 965, "friends_count": 682, "statues_count": 12925 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Paradise, NV", "id": "8fa6d7a33b83ef26", "name": "Paradise", "place_type": "city", "bounding_box": rectangle("-115.209254,35.984784 -115.061076,36.137145") }, "geo_tag": { "stateID": 32, "stateName": "Nevada", "countyID": 32003, "countyName": "Clark", "cityID": 3254600, "cityName": "Paradise" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974941981749248, "text": "Wo liegt Bethesda? https://t.co/RJX20KP9uV #Bethesda #quiz", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-77.1003,38.9807"), "retweet_count": 0, "lang": "de", "is_retweet": false, "hashtags": {{ "Bethesda", "quiz" }}, "user": { "id": 21033096, "name": "kartenquiz.de", "screen_name": "kartenquizde", "lang": "de", "location": "null", "create_at": date("2009-02-16"), "description": "Das kostenlose Geographie-Quiz und Erdkunde-Spiel auf der Basis von Google Maps.", "followers_count": 489, "friends_count": 115, "statues_count": 1969347 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Bethesda, MD", "id": "864ff125241f172f", "name": "Bethesda", "place_type": "city", "bounding_box": rectangle("-77.158594,38.940225 -77.078411,39.022449") }, "geo_tag": { "stateID": 24, "stateName": "Maryland", "countyID": 24031, "countyName": "Montgomery", "cityID": 2407125, "cityName": "Bethesda" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974942577332225, "text": "I HATE hypocrites with a passion", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 321587900, "name": "AlmightyQuin", "screen_name": "aye_quin", "lang": "en", "location": "null", "create_at": date("2011-06-21"), "description": "#TXST", "followers_count": 700, "friends_count": 534, "statues_count": 3520 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Beaumont, TX", "id": "33d5cf4dea5828c5", "name": "Beaumont", "place_type": "city", "bounding_box": rectangle("-94.233379,29.989715 -94.023888,30.184121") }, "geo_tag": { "stateID": 48, "stateName": "Texas", "countyID": 48245, "countyName": "Jefferson", "cityID": 4807000, "cityName": "Beaumont" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974942673838080, "text": "@MattVencill false", "in_reply_to_status": 742974740650921988, "in_reply_to_user": 426448383, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 426448383 }}, "user": { "id": 3438310289, "name": "Superman™", "screen_name": "jbrown2_", "lang": "en", "location": "null", "create_at": date("2015-08-24"), "description": "This lost boy got fly without Peter Pan -Drake⁶ Philippians 4:13", "followers_count": 141, "friends_count": 145, "statues_count": 2978 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Edgewood, OH", "id": "54b88e1497e529d7", "name": "Edgewood", "place_type": "city", "bounding_box": rectangle("-80.789556,41.8544 -80.707849,41.896766") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39007, "countyName": "Ashtabula", "cityID": 3924542, "cityName": "Edgewood" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974942870900737, "text": "Wind 2.0 mph SE. Barometer 29.890 in, Falling. Temperature 70.9 °F. Rain today 0.00 in. Humidity 72%", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "coordinate": point("-84.19638889,40.56805556"), "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 355459366, "name": "Auglaize County EMA", "screen_name": "TroyAnderson6", "lang": "en", "location": "St.Marys,Ohio", "create_at": date("2011-08-15"), "description": "Auglaize County Emergency Management Agency & Official Auglaize County Weather Site, Director Troy Anderson.", "followers_count": 405, "friends_count": 294, "statues_count": 11287 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Wapakoneta, OH", "id": "e4e77616d9038dec", "name": "Wapakoneta", "place_type": "city", "bounding_box": rectangle("-84.239566,40.544243 -84.156781,40.597243") }, "geo_tag": { "stateID": 39, "stateName": "Ohio", "countyID": 39011, "countyName": "Auglaize", "cityID": 3980766, "cityName": "Wapakoneta" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974942879350784, "text": "@c_Coleman24 look at this ���� https://t.co/inigb3A9Sg", "in_reply_to_status": -1, "in_reply_to_user": 247921488, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 247921488 }}, "user": { "id": 34420820, "name": "tay bay bay", "screen_name": "iamtaylour", "lang": "en", "location": "kansas city, missouri", "create_at": date("2009-04-22"), "description": "Snapchat: divaindisguise | rip tyler mcintoush, my angel | IG: iamtaylour", "followers_count": 1486, "friends_count": 977, "statues_count": 102439 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Kansas City, MO", "id": "9a974dfc8efb32a0", "name": "Kansas City", "place_type": "city", "bounding_box": rectangle("-94.733122,38.868002 -94.385441,39.332095") }, "geo_tag": { "stateID": 29, "stateName": "Missouri", "countyID": 29095, "countyName": "Jackson", "cityID": 2938000, "cityName": "Kansas City" } }
+{ "create_at": datetime("2016-06-15T00:00:04.000Z"), "id": 742974943101636608, "text": "Definitely https://t.co/9LShDQf7DI", "in_reply_to_status": -1, "in_reply_to_user": -1, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user": { "id": 142496864, "name": "PANGELA", "screen_name": "AngelaMooree", "lang": "en", "location": "Los Angeles, CA", "create_at": date("2010-05-10"), "description": "19 • Comm Major✨CSUEB18 •626• Weird IG:AngelaMooree", "followers_count": 924, "friends_count": 1126, "statues_count": 29891 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Lake Havasu City, AZ", "id": "5f83cd36a6136fd4", "name": "Lake Havasu City", "place_type": "city", "bounding_box": rectangle("-114.37582,34.449228 -114.245399,34.532878") }, "geo_tag": { "stateID": 4, "stateName": "Arizona", "countyID": 4015, "countyName": "Mohave", "cityID": 439370, "cityName": "Lake Havasu City" } }
+{ "create_at": datetime("2016-06-15T00:00:05.000Z"), "id": 742974943655268356, "text": "@realDonaldTrump business Women & men would prefer 2 work together w/u following all legal guidelines incorporating \"fresh ideas\" & \"energy\"", "in_reply_to_status": 742974084330446848, "in_reply_to_user": 1518264955, "favorite_count": 0, "retweet_count": 0, "lang": "en", "is_retweet": false, "user_mentions": {{ 25073877 }}, "user": { "id": 1518264955, "name": "MECox", "screen_name": "JollyMets", "lang": "en", "location": "Queens, New York", "create_at": date("2013-06-14"), "description": "Embrace & Respect Responsibility Borders. Language & Culture **** \nQue Viva Puerto Rico! **** \nOM SHANTI ***** \nLETS GO NY METS ****\nAmerica First ****", "followers_count": 506, "friends_count": 335, "statues_count": 13900 }, "place": { "country": "United States", "country_code": "United States", "full_name": "Queens, NY", "id": "00c39537733fa112", "name": "Queens", "place_type": "city", "bounding_box": rectangle("-73.962582,40.541722 -73.699793,40.800037") }, "geo_tag": { "stateID": 36, "stateName": "New York", "countyID": 36081, "countyName": "Queens", "cityID": 36081, "cityName": "Queens" } }
diff --git a/asterixdb/asterix-app/data/tinysocial/twm2.adm b/asterixdb/asterix-app/data/tinysocial/twm2.adm
new file mode 100644
index 0000000..b84a569
--- /dev/null
+++ b/asterixdb/asterix-app/data/tinysocial/twm2.adm
@@ -0,0 +1,12 @@
+{"tweetid":"1","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("47.44,80.65"),"send-time":datetime("2008-04-26T10:10:00"),"referred-topics":["t-mobile","customization"],"message-text":" love t-mobile its customization is good:)"}
+{"tweetid":"2","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("32.84,67.14"),"send-time":datetime("2010-05-13T10:10:00"),"referred-topics":["verizon","shortcut-menu"],"message-text":" like verizon its shortcut-menu is awesome:)"}
+{"tweetid":"3","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("29.72,75.8"),"send-time":datetime("2006-11-04T10:10:00"),"referred-topics":["motorola","speed"],"message-text":" like motorola the speed is good:)"}
+{"tweetid":"4","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("39.28,70.48"),"send-time":datetime("2011-12-26T10:10:00"),"referred-topics":["sprint","voice-command"],"message-text":" like sprint the voice-command is mind-blowing:)"}
+{"tweetid":"5","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("40.09,92.69"),"send-time":datetime("2006-08-04T10:10:00"),"referred-topics":["motorola","speed"],"message-text":" can't stand motorola its speed is terrible:("}
+{"tweetid":"6","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("47.51,83.99"),"send-time":datetime("2010-05-07T10:10:00"),"referred-topics":["iphone","voice-clarity"],"message-text":" like iphone the voice-clarity is good:)"}
+{"tweetid":"7","user":{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136},"sender-location":point("36.21,72.6"),"send-time":datetime("2011-08-25T10:10:00"),"referred-topics":["samsung","platform"],"message-text":" like samsung the platform is good"}
+{"tweetid":"8","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("46.05,93.34"),"send-time":datetime("2005-10-14T10:10:00"),"referred-topics":["t-mobile","shortcut-menu"],"message-text":" like t-mobile the shortcut-menu is awesome:)"}
+{"tweetid":"9","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("36.86,74.62"),"send-time":datetime("2012-07-21T10:10:00"),"referred-topics":["verizon","voicemail-service"],"message-text":" love verizon its voicemail-service is awesome"}
+{"tweetid":"10","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("29.15,76.53"),"send-time":datetime("2008-01-26T10:10:00"),"referred-topics":["verizon","voice-clarity"],"message-text":" hate verizon its voice-clarity is OMG:("}
+{"tweetid":"11","user":{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649},"sender-location":point("37.59,68.42"),"send-time":datetime("2008-03-09T10:10:00"),"referred-topics":["iphone","platform"],"message-text":" can't stand iphone its platform is terrible"}
+{"tweetid":"12","user":{"screen-name":"OliJackson_512","lang":"en","friends_count":445,"statuses_count":164,"name":"Oli Jackson","followers_count":22649},"sender-location":point("24.82,94.63"),"send-time":datetime("2010-02-13T10:10:00"),"referred-topics":["samsung","voice-command"],"message-text":" like samsung the voice-command is amazing:)"}
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index f06b6d2..4cec16a 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -153,6 +153,7 @@
           <usedDependencies combine.children="append">
             <usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
             <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
+            <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
           </usedDependencies>
           <ignoredUnusedDeclaredDependencies>
             <ignoredUnusedDeclaredDependency>org.apache.asterix:asterix-external-data:zip:*</ignoredUnusedDeclaredDependency>
@@ -475,6 +476,10 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-data</artifactId>
     </dependency>
@@ -493,7 +498,6 @@
     <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
@@ -635,7 +639,7 @@
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-jul</artifactId>
-      <version>2.10.0</version>
+      <version>2.11.1</version>
     </dependency>
     <dependency>
       <groupId>org.apache.asterix</groupId>
@@ -643,5 +647,26 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-geo</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-rtree</artifactId>
+      <version>${hyracks.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+      <version>${hyracks.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+      <version>${hyracks.version}</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 537625d..cfec2de 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -23,6 +23,7 @@
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -62,8 +63,11 @@
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.optimizer.base.FuzzyUtils;
 import org.apache.asterix.optimizer.rules.am.AbstractIntroduceAccessMethodRule;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
@@ -72,6 +76,7 @@
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.utils.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -101,7 +106,9 @@
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.config.IOptionType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.resource.IClusterCapacity;
@@ -125,13 +132,15 @@
     private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
 
     // A white list of supported configurable parameters.
+    public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
     private static final Set<String> CONFIGURABLE_PARAMETER_NAMES =
             ImmutableSet.of(CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
                     CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
                     CompilerProperties.COMPILER_PARALLELISM_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
                     FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
                     StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
-                    FeedActivityDetails.COLLECT_LOCATIONS, "inline_with", "hash_merge", "output-record-type",
+                    FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+                    SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
                     AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION);
 
     private final IRewriterFactory rewriterFactory;
@@ -169,8 +178,8 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs)
-            throws CompilationException {
+            MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         if (q == null) {
             return null;
         }
@@ -179,17 +188,22 @@
             generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()), inlineUdfs);
+        rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()), inlineUdfs,
+                externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
 
     public JobSpecification compileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
             Query query, int varCounter, String outputDatasetName, SessionOutput output,
-            ICompiledDmlStatement statement) throws AlgebricksException, ACIDException {
+            ICompiledDmlStatement statement, Map<VarIdentifier, IAObject> externalVars)
+            throws AlgebricksException, ACIDException {
 
         // establish facts
         final boolean isQuery = query != null;
         final boolean isLoad = statement != null && statement.getKind() == Statement.Kind.LOAD;
+        final SourceLocation sourceLoc =
+                query != null ? query.getSourceLocation() : statement != null ? statement.getSourceLocation() : null;
+        final boolean isExplainOnly = isQuery && query.isExplain();
 
         SessionConfig conf = output.config();
         if (isQuery && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
@@ -200,7 +214,7 @@
         final TxnId txnId = metadataProvider.getTxnIdFactory().create();
         metadataProvider.setTxnId(txnId);
         ILangExpressionToPlanTranslator t =
-                translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter);
+                translatorFactory.createExpressionToPlanTranslator(metadataProvider, varCounter, externalVars);
 
         ILogicalPlan plan = isLoad ? t.translateLoad(statement) : t.translate(query, outputDatasetName, statement);
 
@@ -209,9 +223,9 @@
             generateLogicalPlan(plan, output.config().getPlanFormat());
         }
         CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
-        Map<String, String> querySpecificConfig = validateConfig(metadataProvider.getConfig());
+        Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
-                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig);
+                getPhysicalOptimizationConfig(compilerProperties, querySpecificConfig, sourceLoc);
 
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
@@ -227,7 +241,7 @@
         builder.setMissableTypeComputer(MissableTypeComputer.INSTANCE);
         builder.setConflictingTypeResolver(ConflictingTypeResolver.INSTANCE);
 
-        int parallelism = getParallelism(querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
+        int parallelism = getParallelism((String) querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
                 compilerProperties.getParallelism());
         AlgebricksAbsolutePartitionConstraint computationLocations =
                 chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
@@ -236,7 +250,7 @@
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
         if (conf.isOptimize()) {
             compiler.optimize();
-            if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
+            if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
                 if (conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)) {
                     // For Optimizer tests.
                     AlgebricksAppendable buffer = new AlgebricksAppendable(output.out());
@@ -248,16 +262,12 @@
                 }
             }
         }
-        if (isQuery && query.isExplain()) {
-            try {
-                LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
-                PlanPrettyPrinter.printPlan(plan, pvisitor, 0);
-                ResultUtil.printResults(metadataProvider.getApplicationContext(), pvisitor.get().toString(), output,
-                        new Stats(), null);
-                return null;
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
+        if (isExplainOnly) {
+            printPlanAsResult(metadataProvider, output);
+            if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
+                executionPlans.setOptimizedLogicalPlan(null);
             }
+            return null;
         }
 
         if (!conf.isGenerateJobSpec()) {
@@ -299,21 +309,33 @@
         return spec;
     }
 
+    private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output) throws AlgebricksException {
+        final SessionConfig conf = output.config();
+        boolean quoteResult = output.config().getPlanFormat() == SessionConfig.PlanFormat.STRING;
+        conf.set(SessionConfig.FORMAT_QUOTE_RECORD, quoteResult);
+        try {
+            ResultUtil.printResults(metadataProvider.getApplicationContext(), executionPlans.getOptimizedLogicalPlan(),
+                    output, new Stats(), null);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     protected PhysicalOptimizationConfig getPhysicalOptimizationConfig(CompilerProperties compilerProperties,
-            Map<String, String> querySpecificConfig) throws AlgebricksException {
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
         int frameSize = compilerProperties.getFrameSize();
         int sortFrameLimit = getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
-                querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
-                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT);
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
+                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT, sourceLoc);
         int groupFrameLimit = getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
-                querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
-                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY);
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
         int joinFrameLimit = getFrameLimit(CompilerProperties.COMPILER_JOINMEMORY_KEY,
-                querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
-                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN);
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
+                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN, sourceLoc);
         int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
-                querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
-                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH);
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
+                compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
         final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.getPhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
         physOptConf.setMaxFramesExternalSort(sortFrameLimit);
@@ -450,12 +472,17 @@
 
     // Gets the frame limit.
     private static int getFrameLimit(String parameterName, String parameter, long memBudgetInConfiguration,
-            int frameSize, int minFrameLimit) throws AlgebricksException {
+            int frameSize, int minFrameLimit, SourceLocation sourceLoc) throws AlgebricksException {
         IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
-        long memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        long memBudget;
+        try {
+            memBudget = parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+        }
         int frameLimit = (int) (memBudget / frameSize);
         if (frameLimit < minFrameLimit) {
-            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, parameterName,
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, sourceLoc, parameterName,
                     frameSize * minFrameLimit);
         }
         // Sets the frame limit to the minimum frame limit if the caculated frame limit is too small.
@@ -469,10 +496,13 @@
     }
 
     // Validates if the query contains unsupported query parameters.
-    private static Map<String, String> validateConfig(Map<String, String> config) throws AlgebricksException {
+    private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+            throws AlgebricksException {
         for (String parameterName : config.keySet()) {
-            if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)) {
-                throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, parameterName);
+            if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+                    && !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
+                throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
+                        parameterName);
             }
         }
         return config;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
index 7c06762..a4da189 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
@@ -16,31 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.api.http.ctx;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.job.JobId;
 
 public class StatementExecutorContext implements IStatementExecutorContext {
 
-    private final Map<String, JobId> runningQueries = new ConcurrentHashMap<>();
+    private final Map<String, IClientRequest> runningQueries = new ConcurrentHashMap<>();
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return runningQueries.get(clientContextId);
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
-        runningQueries.put(clientContextId, jobId);
+    public void put(String clientContextId, IClientRequest req) {
+        runningQueries.put(clientContextId, req);
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return runningQueries.remove(clientContextId);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index bd096dd..9844900 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -19,19 +19,16 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_DATASET_ATTR;
 
 import java.io.PrintWriter;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.hyracks.ipc.exceptions.IPCException;
 import org.apache.logging.log4j.Level;
@@ -52,7 +49,8 @@
         HANDLE("handle"),
         ERRORS("errors"),
         METRICS("metrics"),
-        PLANS("plans");
+        PLANS("plans"),
+        WARNINGS("warnings");
 
         private final String str;
 
@@ -104,31 +102,16 @@
         this.appCtx = appCtx;
     }
 
-    protected IHyracksDataset getHyracksDataset() throws Exception { // NOSONAR
+    protected IResultSet getResultSet() throws Exception { // NOSONAR
         try {
-            return doGetHyracksDataset();
+            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
         } catch (IPCException e) {
             LOGGER.log(Level.WARN, "Failed getting hyracks dataset connection. Resetting hyracks connection.", e);
             ctx.put(HYRACKS_CONNECTION_ATTR, appCtx.getHcc());
-            return doGetHyracksDataset();
+            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
         }
     }
 
-    protected IHyracksDataset doGetHyracksDataset() throws Exception {
-        IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-        if (hds == null) {
-            synchronized (ctx) {
-                hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-                if (hds == null) {
-                    hds = new HyracksDataset(getHyracksClientConnection(),
-                            appCtx.getCompilerProperties().getFrameSize(), ResultReader.NUM_READERS);
-                    ctx.put(HYRACKS_DATASET_ATTR, hds);
-                }
-            }
-        }
-        return hds;
-    }
-
     protected IHyracksClientConnection getHyracksClientConnection() throws Exception { // NOSONAR
         IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
         if (hcc == null) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index 63896f2..868e4b8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_DATASET_ATTR;
 
 import java.awt.image.BufferedImage;
 import java.io.BufferedReader;
@@ -33,7 +32,6 @@
 
 import javax.imageio.ImageIO;
 
-import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -54,8 +52,7 @@
 import org.apache.asterix.translator.SessionConfig.PlanFormat;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
@@ -137,17 +134,7 @@
         }
         try {
             IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
-            IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-            if (hds == null) {
-                synchronized (ctx) {
-                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-                    if (hds == null) {
-                        hds = new HyracksDataset(hcc, appCtx.getCompilerProperties().getFrameSize(),
-                                ResultReader.NUM_READERS);
-                        ctx.put(HYRACKS_DATASET_ATTR, hds);
-                    }
-                }
-            }
+            IResultSet resultSet = ServletUtil.getResultSet(hcc, appCtx, ctx);
             IParser parser = parserFactory.createParser(query);
             List<Statement> aqlStatements = parser.parse();
             SessionConfig sessionConfig = new SessionConfig(format, true, isSet(executeQuery), true, planFormat);
@@ -163,8 +150,8 @@
             double duration;
             long startTime = System.currentTimeMillis();
             final IRequestParameters requestParameters =
-                    new RequestParameters(hds, new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE),
-                            new IStatementExecutor.Stats(), null, null, null);
+                    new RequestParameters(resultSet, new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE),
+                            new IStatementExecutor.Stats(), null, null, null, null, true);
             translator.compileAndExecute(hcc, null, requestParameters);
             long endTime = System.currentTimeMillis();
             duration = (endTime - startTime) / 1000.00;
@@ -247,6 +234,6 @@
     }
 
     private static boolean isSet(String requestParameter) {
-        return requestParameter != null && "true".equals(requestParameter);
+        return "true".equals(requestParameter);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
new file mode 100644
index 0000000..5f5692d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
@@ -0,0 +1,76 @@
+/*
+ * 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.asterix.api.http.server;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.IStatementExecutorContext;
+import org.apache.hyracks.http.api.IServletRequest;
+import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.hyracks.http.server.AbstractServlet;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+
+/**
+ * The servlet provides a REST API for cancelling an on-going query.
+ */
+public class CcQueryCancellationServlet extends AbstractServlet {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final ICcApplicationContext appCtx;
+
+    public CcQueryCancellationServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx,
+            String... paths) {
+        super(ctx, paths);
+        this.appCtx = appCtx;
+    }
+
+    @Override
+    protected void delete(IServletRequest request, IServletResponse response) throws IOException {
+        String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
+        if (clientContextId == null) {
+            response.setStatus(HttpResponseStatus.BAD_REQUEST);
+            return;
+        }
+        IStatementExecutorContext executorCtx =
+                (IStatementExecutorContext) ctx.get(ServletConstants.RUNNING_QUERIES_ATTR);
+        IClientRequest req = executorCtx.get(clientContextId);
+        if (req == null) {
+            // response: NOT FOUND
+            response.setStatus(HttpResponseStatus.NOT_FOUND);
+            return;
+        }
+        try {
+            // Cancels the on-going job.
+            req.cancel(appCtx);
+            // response: OK
+            response.setStatus(HttpResponseStatus.OK);
+        } catch (Exception e) {
+            LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
+            // response: INTERNAL SERVER ERROR
+            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
deleted file mode 100644
index 4c1dbf7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class DdlApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES =
-            Statement.Category.QUERY | Statement.Category.UPDATE | Statement.Category.DDL;
-
-    public DdlApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("ddl");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-DDL statement %s to the DDL API.";
-    }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
new file mode 100644
index 0000000..baaa5bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.api.http.server;
+
+public class ExecutionWarning {
+
+    private final int code;
+    private final String message;
+
+    public ExecutionWarning(int code, String message) {
+        this.code = code;
+        this.message = message;
+    }
+
+    public int getCode() {
+        return code;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
deleted file mode 100644
index 0a461c7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class FullApiServlet extends RestApiServlet {
-
-    private static final String AQL_STMT_PARAM_NAME = "aql";
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
-            | Statement.Category.DDL | Statement.Category.PROCEDURE;
-
-    public FullApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        throw new IllegalStateException();
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter(AQL_STMT_PARAM_NAME);
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index da621d2..60806d3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -23,6 +23,7 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
 import org.apache.asterix.app.message.CancelQueryRequest;
 import org.apache.asterix.app.message.CancelQueryResponse;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
@@ -31,6 +32,7 @@
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -40,7 +42,7 @@
 /**
  * The servlet provides a REST API on an NC for cancelling an on-going query.
  */
-public class NCQueryCancellationServlet extends QueryCancellationServlet {
+public class NCQueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
     private final INCServiceContext serviceCtx;
     private final INCMessageBroker messageBroker;
@@ -54,7 +56,7 @@
     @Override
     protected void delete(IServletRequest request, IServletResponse response) {
         // gets the parameter client_context_id from the request.
-        String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
+        String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
         if (clientContextId == null) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index 1713ca5..362f924 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -45,8 +45,8 @@
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.commons.lang3.tuple.Triple;
 import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.http.api.IChannelClosedHandler;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.server.HttpServer;
@@ -70,38 +70,40 @@
 
     @Override
     protected void executeStatement(String statementsText, SessionOutput sessionOutput,
-            ResultProperties resultProperties, IStatementExecutor.Stats stats, RequestParameters param,
-            RequestExecutionState execution, Map<String, String> optionalParameters) throws Exception {
+            ResultProperties resultProperties, IStatementExecutor.Stats stats, QueryServiceRequestParameters param,
+            RequestExecutionState execution, Map<String, String> optionalParameters,
+            Map<String, byte[]> statementParameters) throws Exception {
         // Running on NC -> send 'execute' message to CC
         INCServiceContext ncCtx = (INCServiceContext) serviceCtx;
         INCMessageBroker ncMb = (INCMessageBroker) ncCtx.getMessageBroker();
         final IStatementExecutor.ResultDelivery delivery = resultProperties.getDelivery();
         ExecuteStatementResponseMessage responseMsg;
         MessageFuture responseFuture = ncMb.registerMessageFuture();
-        final String handleUrl = getHandleUrl(param.host, param.path, delivery);
+        final String handleUrl = getHandleUrl(param.getHost(), param.getPath(), delivery);
         try {
-            if (param.clientContextID == null) {
-                param.clientContextID = UUID.randomUUID().toString();
+            if (param.getClientContextID() == null) {
+                param.setClientContextID(UUID.randomUUID().toString());
             }
             long timeout = ExecuteStatementRequestMessage.DEFAULT_NC_TIMEOUT_MILLIS;
-            if (param.timeout != null && !param.timeout.trim().isEmpty()) {
-                timeout = TimeUnit.NANOSECONDS.toMillis(Duration.parseDurationStringToNanos(param.timeout));
+            if (param.getTimeout() != null && !param.getTimeout().trim().isEmpty()) {
+                timeout = TimeUnit.NANOSECONDS.toMillis(Duration.parseDurationStringToNanos(param.getTimeout()));
             }
             ExecuteStatementRequestMessage requestMsg = new ExecuteStatementRequestMessage(ncCtx.getNodeId(),
                     responseFuture.getFutureId(), queryLanguage, statementsText, sessionOutput.config(),
-                    resultProperties.getNcToCcResultProperties(), param.clientContextID, handleUrl, optionalParameters);
+                    resultProperties.getNcToCcResultProperties(), param.getClientContextID(), handleUrl,
+                    optionalParameters, statementParameters, param.isMultiStatement());
             execution.start();
             ncMb.sendMessageToPrimaryCC(requestMsg);
             try {
                 responseMsg = (ExecuteStatementResponseMessage) responseFuture.get(timeout, TimeUnit.MILLISECONDS);
             } catch (InterruptedException e) {
-                cancelQuery(ncMb, ncCtx.getNodeId(), param.clientContextID, e, false);
+                cancelQuery(ncMb, ncCtx.getNodeId(), param.getClientContextID(), e, false);
                 throw e;
             } catch (TimeoutException exception) {
-                RuntimeDataException hde = new RuntimeDataException(ErrorCode.QUERY_TIMEOUT);
+                RuntimeDataException hde = new RuntimeDataException(ErrorCode.REQUEST_TIMEOUT);
                 hde.addSuppressed(exception);
                 // cancel query
-                cancelQuery(ncMb, ncCtx.getNodeId(), param.clientContextID, hde, true);
+                cancelQuery(ncMb, ncCtx.getNodeId(), param.getClientContextID(), hde, true);
                 throw hde;
             }
             execution.end();
@@ -126,7 +128,7 @@
         if (delivery == IStatementExecutor.ResultDelivery.IMMEDIATE && !resultMetadata.getResultSets().isEmpty()) {
             stats.setProcessedObjects(responseMsg.getStats().getProcessedObjects());
             for (Triple<JobId, ResultSetId, ARecordType> rsmd : resultMetadata.getResultSets()) {
-                ResultReader resultReader = new ResultReader(getHyracksDataset(), rsmd.getLeft(), rsmd.getMiddle());
+                ResultReader resultReader = new ResultReader(getResultSet(), rsmd.getLeft(), rsmd.getMiddle());
                 ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, rsmd.getRight());
             }
         } else {
@@ -142,6 +144,7 @@
             CancelQueryRequest cancelQueryMessage =
                     new CancelQueryRequest(nodeId, cancelQueryFuture.getFutureId(), clientContextID);
             // TODO(mblow): multicc -- need to send cancellation to the correct cc
+            LOGGER.info("Cancelling query due to {}", exception.getClass().getSimpleName());
             messageBroker.sendMessageToPrimaryCC(cancelQueryMessage);
             if (wait) {
                 cancelQueryFuture.get(ExecuteStatementRequestMessage.DEFAULT_QUERY_CANCELLATION_WAIT_MILLIS,
@@ -155,7 +158,8 @@
     }
 
     @Override
-    protected void handleExecuteStatementException(Throwable t, RequestExecutionState state, RequestParameters param) {
+    protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
+            QueryServiceRequestParameters param) {
         if (t instanceof TimeoutException // TODO(mblow): I don't think t can ever been an instance of TimeoutException
                 || ExceptionUtils.matchingCause(t, candidate -> candidate instanceof IPCException)) {
             GlobalConfig.ASTERIX_LOGGER.log(Level.WARN, t.toString(), t);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
deleted file mode 100644
index 3e692d3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class QueryApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
-
-    public QueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("query");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-query statement %s to the query API.";
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
deleted file mode 100644
index f8655ad..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.asterix.api.http.server;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.http.api.IServletRequest;
-import org.apache.hyracks.http.api.IServletResponse;
-import org.apache.hyracks.http.server.AbstractServlet;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-import io.netty.handler.codec.http.HttpResponseStatus;
-
-/**
- * The servlet provides a REST API for cancelling an on-going query.
- */
-public class QueryCancellationServlet extends AbstractServlet {
-    private static final Logger LOGGER = LogManager.getLogger();
-    protected static final String CLIENT_CONTEXT_ID = "client_context_id";
-
-    public QueryCancellationServlet(ConcurrentMap<String, Object> ctx, String... paths) {
-        super(ctx, paths);
-    }
-
-    @Override
-    protected void delete(IServletRequest request, IServletResponse response) throws IOException {
-        String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
-        if (clientContextId == null) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-
-        // Retrieves the corresponding Hyracks job id.
-        IStatementExecutorContext runningQueries =
-                (IStatementExecutorContext) ctx.get(ServletConstants.RUNNING_QUERIES_ATTR);
-        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(ServletConstants.HYRACKS_CONNECTION_ATTR);
-        JobId jobId = runningQueries.getJobIdFromClientContextId(clientContextId);
-
-        if (jobId == null) {
-            // response: NOT FOUND
-            response.setStatus(HttpResponseStatus.NOT_FOUND);
-            return;
-        }
-        try {
-            // Cancels the on-going job.
-            hcc.cancelJob(jobId);
-            // Removes the cancelled query from the map activeQueries.
-            runningQueries.removeJobIdFromClientContextId(clientContextId);
-            // response: OK
-            response.setStatus(HttpResponseStatus.OK);
-        } catch (Exception e) {
-            LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
-            // response: INTERNAL SERVER ERROR
-            response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index de96d54..6781f22 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -26,8 +26,8 @@
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionOutput;
-import org.apache.hyracks.api.dataset.DatasetJobRecord;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.result.ResultJobRecord;
+import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.http.api.IServletRequest;
@@ -59,11 +59,11 @@
             return;
         }
 
-        IHyracksDataset hds = getHyracksDataset();
-        ResultReader resultReader = new ResultReader(hds, handle.getJobId(), handle.getResultSetId());
+        IResultSet resultSet = getResultSet();
+        ResultReader resultReader = new ResultReader(resultSet, handle.getJobId(), handle.getResultSetId());
 
         try {
-            DatasetJobRecord.Status status = resultReader.getStatus();
+            ResultJobRecord.Status status = resultReader.getStatus();
 
             final HttpResponseStatus httpStatus;
             if (status == null) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
new file mode 100644
index 0000000..16a2105
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
@@ -0,0 +1,230 @@
+/*
+ * 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.asterix.api.http.server;
+
+import java.util.Map;
+
+import org.apache.hyracks.util.JSONUtil;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class QueryServiceRequestParameters {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    private String host;
+    private String path;
+    private String statement;
+    private String format;
+    private String timeout;
+    private boolean pretty;
+    private String clientContextID;
+    private String mode;
+    private String maxResultReads;
+    private String planFormat;
+    private Map<String, JsonNode> statementParams;
+    private boolean expressionTree;
+    private boolean rewrittenExpressionTree;
+    private boolean logicalPlan;
+    private boolean optimizedLogicalPlan;
+    private boolean job;
+    private boolean signature;
+    private boolean multiStatement;
+
+    public String getHost() {
+        return host;
+    }
+
+    public void setHost(String host) {
+        this.host = host;
+    }
+
+    public String getPath() {
+        return path;
+    }
+
+    public void setPath(String path) {
+        this.path = path;
+    }
+
+    public String getStatement() {
+        return statement;
+    }
+
+    public void setStatement(String statement) {
+        this.statement = statement;
+    }
+
+    public String getFormat() {
+        return format;
+    }
+
+    public void setFormat(String format) {
+        this.format = format;
+    }
+
+    public String getTimeout() {
+        return timeout;
+    }
+
+    public void setTimeout(String timeout) {
+        this.timeout = timeout;
+    }
+
+    public boolean isPretty() {
+        return pretty;
+    }
+
+    public void setPretty(boolean pretty) {
+        this.pretty = pretty;
+    }
+
+    public String getClientContextID() {
+        return clientContextID;
+    }
+
+    public void setClientContextID(String clientContextID) {
+        this.clientContextID = clientContextID;
+    }
+
+    public String getMode() {
+        return mode;
+    }
+
+    public void setMode(String mode) {
+        this.mode = mode;
+    }
+
+    public String getMaxResultReads() {
+        return maxResultReads;
+    }
+
+    public void setMaxResultReads(String maxResultReads) {
+        this.maxResultReads = maxResultReads;
+    }
+
+    public String getPlanFormat() {
+        return planFormat;
+    }
+
+    public void setPlanFormat(String planFormat) {
+        this.planFormat = planFormat;
+    }
+
+    public Map<String, JsonNode> getStatementParams() {
+        return statementParams;
+    }
+
+    public void setStatementParams(Map<String, JsonNode> statementParams) {
+        this.statementParams = statementParams;
+    }
+
+    public boolean isExpressionTree() {
+        return expressionTree;
+    }
+
+    public void setExpressionTree(boolean expressionTree) {
+        this.expressionTree = expressionTree;
+    }
+
+    public boolean isRewrittenExpressionTree() {
+        return rewrittenExpressionTree;
+    }
+
+    public void setRewrittenExpressionTree(boolean rewrittenExpressionTree) {
+        this.rewrittenExpressionTree = rewrittenExpressionTree;
+    }
+
+    public boolean isLogicalPlan() {
+        return logicalPlan;
+    }
+
+    public void setLogicalPlan(boolean logicalPlan) {
+        this.logicalPlan = logicalPlan;
+    }
+
+    public boolean isOptimizedLogicalPlan() {
+        return optimizedLogicalPlan;
+    }
+
+    public void setOptimizedLogicalPlan(boolean optimizedLogicalPlan) {
+        this.optimizedLogicalPlan = optimizedLogicalPlan;
+    }
+
+    public boolean isJob() {
+        return job;
+    }
+
+    public void setJob(boolean job) {
+        this.job = job;
+    }
+
+    public boolean isSignature() {
+        return signature;
+    }
+
+    public void setSignature(boolean signature) {
+        this.signature = signature;
+    }
+
+    public boolean isMultiStatement() {
+        return multiStatement;
+    }
+
+    public void setMultiStatement(boolean multiStatement) {
+        this.multiStatement = multiStatement;
+    }
+
+    @Override
+    public String toString() {
+        try {
+            ObjectNode on = OBJECT_MAPPER.createObjectNode();
+            on.put("host", host);
+            on.put("path", path);
+            on.put("statement", statement != null ? JSONUtil.escape(new StringBuilder(), statement).toString() : null);
+            on.put("pretty", pretty);
+            on.put("mode", mode);
+            on.put("clientContextID", clientContextID);
+            on.put("format", format);
+            on.put("timeout", timeout);
+            on.put("maxResultReads", maxResultReads);
+            on.put("planFormat", planFormat);
+            on.put("expressionTree", expressionTree);
+            on.put("rewrittenExpressionTree", rewrittenExpressionTree);
+            on.put("logicalPlan", logicalPlan);
+            on.put("optimizedLogicalPlan", optimizedLogicalPlan);
+            on.put("job", job);
+            on.put("signature", signature);
+            on.put("multiStatement", multiStatement);
+            if (statementParams != null) {
+                for (Map.Entry<String, JsonNode> statementParam : statementParams.entrySet()) {
+                    on.set('$' + statementParam.getKey(), statementParam.getValue());
+                }
+            }
+            return OBJECT_MAPPER.writeValueAsString(on);
+        } catch (JsonProcessingException e) {
+            QueryServiceServlet.LOGGER.debug("unexpected exception marshalling {} instance to json", getClass(), e);
+            return e.toString();
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 29da286..765ba9c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -19,17 +19,22 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.common.exceptions.ErrorCode.ASTERIX;
-import static org.apache.asterix.common.exceptions.ErrorCode.QUERY_TIMEOUT;
+import static org.apache.asterix.common.exceptions.ErrorCode.NO_STATEMENT_PROVIDED;
 import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
 import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_NODE_UNREGISTERED;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUEST_TIMEOUT;
 
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 import org.apache.asterix.algebra.base.ILangExtension;
@@ -39,12 +44,14 @@
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.aql.parser.TokenMgrError;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.ExecutionPlans;
 import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
@@ -63,18 +70,13 @@
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.utils.HttpUtil;
-import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 
 import io.netty.handler.codec.http.HttpResponseStatus;
 
@@ -138,6 +140,7 @@
     }
 
     public enum Parameter {
+        ARGS("args"),
         STATEMENT("statement"),
         FORMAT("format"),
         CLIENT_ID("client_context_id"),
@@ -150,7 +153,9 @@
         REWRITTEN_EXPRESSION_TREE("rewritten-expression-tree"),
         LOGICAL_PLAN("logical-plan"),
         OPTIMIZED_LOGICAL_PLAN("optimized-logical-plan"),
-        JOB("job");
+        JOB("job"),
+        SIGNATURE("signature"),
+        MULTI_STATEMENT("multi-statement");
 
         private final String str;
 
@@ -184,7 +189,8 @@
         RESULT_COUNT("resultCount"),
         RESULT_SIZE("resultSize"),
         ERROR_COUNT("errorCount"),
-        PROCESSED_OBJECTS_COUNT("processedObjects");
+        PROCESSED_OBJECTS_COUNT("processedObjects"),
+        WARNING_COUNT("warningCount");
 
         private final String str;
 
@@ -197,52 +203,7 @@
         }
     }
 
-    static class RequestParameters {
-        String host;
-        String path;
-        String statement;
-        String format;
-        String timeout;
-        boolean pretty;
-        String clientContextID;
-        String mode;
-        String maxResultReads;
-        String planFormat;
-        boolean expressionTree;
-        boolean rewrittenExpressionTree;
-        boolean logicalPlan;
-        boolean optimizedLogicalPlan;
-        boolean job;
-
-        @Override
-        public String toString() {
-            try {
-                ObjectMapper om = new ObjectMapper();
-                ObjectNode on = om.createObjectNode();
-                on.put("host", host);
-                on.put("path", path);
-                on.put("statement", JSONUtil.escape(new StringBuilder(), statement).toString());
-                on.put("pretty", pretty);
-                on.put("mode", mode);
-                on.put("clientContextID", clientContextID);
-                on.put("format", format);
-                on.put("timeout", timeout);
-                on.put("maxResultReads", maxResultReads);
-                on.put("planFormat", planFormat);
-                on.put("expressionTree", expressionTree);
-                on.put("rewrittenExpressionTree", rewrittenExpressionTree);
-                on.put("logicalPlan", logicalPlan);
-                on.put("optimizedLogicalPlan", optimizedLogicalPlan);
-                on.put("job", job);
-                return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
-            } catch (JsonProcessingException e) { // NOSONAR
-                LOGGER.debug("unexpected exception marshalling {} instance to json", getClass(), e);
-                return e.toString();
-            }
-        }
-    }
-
-    static final class RequestExecutionState {
+    protected static final class RequestExecutionState {
         private long execStart = -1;
         private long execEnd = -1;
         private ResultStatus resultStatus = ResultStatus.SUCCESS;
@@ -312,7 +273,7 @@
             if (format.equals(HttpUtil.ContentType.APPLICATION_ADM)) {
                 return SessionConfig.OutputFormat.ADM;
             }
-            if (format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)) {
+            if (isJsonFormat(format)) {
                 return Boolean.parseBoolean(getParameterValue(format, Attribute.LOSSLESS.str()))
                         ? SessionConfig.OutputFormat.LOSSLESS_JSON : SessionConfig.OutputFormat.CLEAN_JSON;
             }
@@ -320,39 +281,46 @@
         return SessionConfig.OutputFormat.CLEAN_JSON;
     }
 
-    private static SessionOutput createSessionOutput(RequestParameters param, String handleUrl,
+    private static SessionOutput createSessionOutput(QueryServiceRequestParameters param, String handleUrl,
             PrintWriter resultWriter) {
         SessionOutput.ResultDecorator resultPrefix = ResultUtil.createPreResultDecorator();
         SessionOutput.ResultDecorator resultPostfix = ResultUtil.createPostResultDecorator();
         SessionOutput.ResultAppender appendHandle = ResultUtil.createResultHandleAppender(handleUrl);
         SessionOutput.ResultAppender appendStatus = ResultUtil.createResultStatusAppender();
 
-        SessionConfig.OutputFormat format = getFormat(param.format);
-        final SessionConfig.PlanFormat planFormat =
-                SessionConfig.PlanFormat.get(param.planFormat, param.planFormat, SessionConfig.PlanFormat.JSON, LOGGER);
+        SessionConfig.OutputFormat format = getFormat(param.getFormat());
+        final SessionConfig.PlanFormat planFormat = SessionConfig.PlanFormat.get(param.getPlanFormat(),
+                param.getPlanFormat(), SessionConfig.PlanFormat.JSON, LOGGER);
         SessionConfig sessionConfig = new SessionConfig(format, planFormat);
         sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
-        sessionConfig.set(SessionConfig.OOB_EXPR_TREE, param.expressionTree);
-        sessionConfig.set(SessionConfig.OOB_REWRITTEN_EXPR_TREE, param.rewrittenExpressionTree);
-        sessionConfig.set(SessionConfig.OOB_LOGICAL_PLAN, param.logicalPlan);
-        sessionConfig.set(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN, param.optimizedLogicalPlan);
-        sessionConfig.set(SessionConfig.OOB_HYRACKS_JOB, param.job);
-        sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.pretty);
+        sessionConfig.set(SessionConfig.OOB_EXPR_TREE, param.isExpressionTree());
+        sessionConfig.set(SessionConfig.OOB_REWRITTEN_EXPR_TREE, param.isRewrittenExpressionTree());
+        sessionConfig.set(SessionConfig.OOB_LOGICAL_PLAN, param.isLogicalPlan());
+        sessionConfig.set(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN, param.isOptimizedLogicalPlan());
+        sessionConfig.set(SessionConfig.OOB_HYRACKS_JOB, param.isJob());
+        sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.isPretty());
         sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
                 format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
         sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, format == SessionConfig.OutputFormat.CSV
-                && "present".equals(getParameterValue(param.format, Attribute.HEADER.str())));
+                && "present".equals(getParameterValue(param.getFormat(), Attribute.HEADER.str())));
         return new SessionOutput(sessionConfig, resultWriter, resultPrefix, resultPostfix, appendHandle, appendStatus);
     }
 
-    private static void printClientContextID(PrintWriter pw, RequestParameters params) {
-        if (params.clientContextID != null && !params.clientContextID.isEmpty()) {
-            ResultUtil.printField(pw, ResultFields.CLIENT_ID.str(), params.clientContextID);
+    private static void printClientContextID(PrintWriter pw, QueryServiceRequestParameters params) {
+        if (params.getClientContextID() != null && !params.getClientContextID().isEmpty()) {
+            ResultUtil.printField(pw, ResultFields.CLIENT_ID.str(), params.getClientContextID());
         }
     }
 
-    private static void printSignature(PrintWriter pw) {
-        ResultUtil.printField(pw, ResultFields.SIGNATURE.str(), "*");
+    private static void printSignature(PrintWriter pw, QueryServiceRequestParameters param) {
+        if (param.isSignature()) {
+            pw.print("\t\"");
+            pw.print(ResultFields.SIGNATURE.str());
+            pw.print("\": {\n");
+            pw.print("\t");
+            ResultUtil.printField(pw, "*", "*", false);
+            pw.print("\t},\n");
+        }
     }
 
     private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
@@ -371,8 +339,9 @@
     }
 
     private static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
-            long resultSize, long processedObjects, long errorCount) {
+            long resultSize, long processedObjects, long errorCount, long warnCount) {
         boolean hasErrors = errorCount != 0;
+        boolean hasWarnings = warnCount != 0;
         pw.print("\t\"");
         pw.print(ResultFields.METRICS.str());
         pw.print("\": {\n");
@@ -385,7 +354,11 @@
         pw.print("\t");
         ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), resultSize, true);
         pw.print("\t");
-        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), processedObjects, hasErrors);
+        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), processedObjects, hasWarnings || hasErrors);
+        if (hasWarnings) {
+            pw.print("\t");
+            ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), warnCount, hasErrors);
+        }
         if (hasErrors) {
             pw.print("\t");
             ResultUtil.printField(pw, Metrics.ERROR_COUNT.str(), errorCount, false);
@@ -403,44 +376,95 @@
         return value != null ? value.asBoolean() : defaultValue;
     }
 
-    private RequestParameters getRequestParameters(IServletRequest request) throws IOException {
+    @FunctionalInterface
+    interface CheckedFunction<I, O> {
+        O apply(I requestParamValue) throws IOException;
+    }
+
+    private <R, P> Map<String, JsonNode> getOptStatementParameters(R request, Iterator<String> paramNameIter,
+            BiFunction<R, String, P> paramValueAccessor, CheckedFunction<P, JsonNode> paramValueParser)
+            throws IOException {
+        Map<String, JsonNode> result = null;
+        while (paramNameIter.hasNext()) {
+            String paramName = paramNameIter.next();
+            String stmtParamName = extractStatementParameterName(paramName);
+            if (stmtParamName != null) {
+                if (result == null) {
+                    result = new HashMap<>();
+                }
+                P paramValue = paramValueAccessor.apply(request, paramName);
+                JsonNode stmtParamValue = paramValueParser.apply(paramValue);
+                result.put(stmtParamName, stmtParamValue);
+            } else if (Parameter.ARGS.str().equals(paramName)) {
+                if (result == null) {
+                    result = new HashMap<>();
+                }
+                P paramValue = paramValueAccessor.apply(request, paramName);
+                JsonNode stmtParamValue = paramValueParser.apply(paramValue);
+                if (stmtParamValue.isArray()) {
+                    for (int i = 0, ln = stmtParamValue.size(); i < ln; i++) {
+                        result.put(String.valueOf(i + 1), stmtParamValue.get(i));
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    private QueryServiceRequestParameters getRequestParameters(IServletRequest request) throws IOException {
         final String contentType = HttpUtil.getContentTypeOnly(request);
-        RequestParameters param = new RequestParameters();
-        param.host = host(request);
-        param.path = servletPath(request);
+        QueryServiceRequestParameters param = new QueryServiceRequestParameters();
+        param.setHost(host(request));
+        param.setPath(servletPath(request));
         if (HttpUtil.ContentType.APPLICATION_JSON.equals(contentType)) {
             try {
                 JsonNode jsonRequest = OBJECT_MAPPER.readTree(HttpUtil.getRequestBody(request));
-                param.statement = jsonRequest.get(Parameter.STATEMENT.str()).asText();
-                param.format = toLower(getOptText(jsonRequest, Parameter.FORMAT.str()));
-                param.pretty = getOptBoolean(jsonRequest, Parameter.PRETTY.str(), false);
-                param.mode = toLower(getOptText(jsonRequest, Parameter.MODE.str()));
-                param.clientContextID = getOptText(jsonRequest, Parameter.CLIENT_ID.str());
-                param.timeout = getOptText(jsonRequest, Parameter.TIMEOUT.str());
-                param.maxResultReads = getOptText(jsonRequest, Parameter.MAX_RESULT_READS.str());
-                param.planFormat = getOptText(jsonRequest, Parameter.PLAN_FORMAT.str());
-                param.expressionTree = getOptBoolean(jsonRequest, Parameter.EXPRESSION_TREE.str(), false);
-                param.rewrittenExpressionTree =
-                        getOptBoolean(jsonRequest, Parameter.REWRITTEN_EXPRESSION_TREE.str(), false);
-                param.logicalPlan = getOptBoolean(jsonRequest, Parameter.LOGICAL_PLAN.str(), false);
-                param.optimizedLogicalPlan = getOptBoolean(jsonRequest, Parameter.OPTIMIZED_LOGICAL_PLAN.str(), false);
-                param.job = getOptBoolean(jsonRequest, Parameter.JOB.str(), false);
+                final String statementParam = Parameter.STATEMENT.str();
+                if (jsonRequest.has(statementParam)) {
+                    param.setStatement(jsonRequest.get(statementParam).asText());
+                }
+                param.setFormat(toLower(getOptText(jsonRequest, Parameter.FORMAT.str())));
+                param.setPretty(getOptBoolean(jsonRequest, Parameter.PRETTY.str(), false));
+                param.setMode(toLower(getOptText(jsonRequest, Parameter.MODE.str())));
+                param.setClientContextID(getOptText(jsonRequest, Parameter.CLIENT_ID.str()));
+                param.setTimeout(getOptText(jsonRequest, Parameter.TIMEOUT.str()));
+                param.setMaxResultReads(getOptText(jsonRequest, Parameter.MAX_RESULT_READS.str()));
+                param.setPlanFormat(getOptText(jsonRequest, Parameter.PLAN_FORMAT.str()));
+                param.setExpressionTree(getOptBoolean(jsonRequest, Parameter.EXPRESSION_TREE.str(), false));
+                param.setRewrittenExpressionTree(
+                        getOptBoolean(jsonRequest, Parameter.REWRITTEN_EXPRESSION_TREE.str(), false));
+                param.setLogicalPlan(getOptBoolean(jsonRequest, Parameter.LOGICAL_PLAN.str(), false));
+                param.setOptimizedLogicalPlan(
+                        getOptBoolean(jsonRequest, Parameter.OPTIMIZED_LOGICAL_PLAN.str(), false));
+                param.setJob(getOptBoolean(jsonRequest, Parameter.JOB.str(), false));
+                param.setSignature(getOptBoolean(jsonRequest, Parameter.SIGNATURE.str(), true));
+                param.setStatementParams(
+                        getOptStatementParameters(jsonRequest, jsonRequest.fieldNames(), JsonNode::get, v -> v));
+                param.setMultiStatement(getOptBoolean(jsonRequest, Parameter.MULTI_STATEMENT.str(), true));
             } catch (JsonParseException | JsonMappingException e) {
                 // if the JSON parsing fails, the statement is empty and we get an empty statement error
                 GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, e.getMessage(), e);
             }
         } else {
-            param.statement = request.getParameter(Parameter.STATEMENT.str());
-            if (param.statement == null) {
-                param.statement = HttpUtil.getRequestBody(request);
+            param.setStatement(request.getParameter(Parameter.STATEMENT.str()));
+            if (param.getStatement() == null) {
+                param.setStatement(HttpUtil.getRequestBody(request));
             }
-            param.format = toLower(request.getParameter(Parameter.FORMAT.str()));
-            param.pretty = Boolean.parseBoolean(request.getParameter(Parameter.PRETTY.str()));
-            param.mode = toLower(request.getParameter(Parameter.MODE.str()));
-            param.clientContextID = request.getParameter(Parameter.CLIENT_ID.str());
-            param.timeout = request.getParameter(Parameter.TIMEOUT.str());
-            param.maxResultReads = request.getParameter(Parameter.MAX_RESULT_READS.str());
-            param.planFormat = request.getParameter(Parameter.PLAN_FORMAT.str());
+            param.setFormat(toLower(request.getParameter(Parameter.FORMAT.str())));
+            param.setPretty(Boolean.parseBoolean(request.getParameter(Parameter.PRETTY.str())));
+            param.setMode(toLower(request.getParameter(Parameter.MODE.str())));
+            param.setClientContextID(request.getParameter(Parameter.CLIENT_ID.str()));
+            param.setTimeout(request.getParameter(Parameter.TIMEOUT.str()));
+            param.setMaxResultReads(request.getParameter(Parameter.MAX_RESULT_READS.str()));
+            param.setPlanFormat(request.getParameter(Parameter.PLAN_FORMAT.str()));
+            final String multiStatementParam = request.getParameter(Parameter.MULTI_STATEMENT.str());
+            param.setMultiStatement(multiStatementParam == null || Boolean.parseBoolean(multiStatementParam));
+            try {
+                param.setStatementParams(getOptStatementParameters(request, request.getParameterNames().iterator(),
+                        IServletRequest::getParameter, OBJECT_MAPPER::readTree));
+            } catch (JsonParseException | JsonMappingException e) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, e.getMessage(), e);
+            }
         }
         return param;
     }
@@ -485,17 +509,17 @@
     }
 
     private void handleRequest(IServletRequest request, IServletResponse response) throws IOException {
-        RequestParameters param = getRequestParameters(request);
+        QueryServiceRequestParameters param = getRequestParameters(request);
         LOGGER.info("handleRequest: {}", param);
         long elapsedStart = System.nanoTime();
         final PrintWriter httpWriter = response.writer();
 
-        ResultDelivery delivery = parseResultDelivery(param.mode);
+        ResultDelivery delivery = parseResultDelivery(param.getMode());
 
-        final ResultProperties resultProperties = param.maxResultReads == null ? new ResultProperties(delivery)
-                : new ResultProperties(delivery, Long.parseLong(param.maxResultReads));
+        final ResultProperties resultProperties = param.getMaxResultReads() == null ? new ResultProperties(delivery)
+                : new ResultProperties(delivery, Long.parseLong(param.getMaxResultReads()));
 
-        String handleUrl = getHandleUrl(param.host, param.path, delivery);
+        String handleUrl = getHandleUrl(param.getHost(), param.getPath(), delivery);
         SessionOutput sessionOutput = createSessionOutput(param, handleUrl, httpWriter);
         SessionConfig sessionConfig = sessionOutput.config();
         HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, HttpUtil.Encoding.UTF8);
@@ -508,33 +532,40 @@
         sessionOutput.out().print("{\n");
         printRequestId(sessionOutput.out());
         printClientContextID(sessionOutput.out(), param);
-        printSignature(sessionOutput.out());
+        printSignature(sessionOutput.out(), param);
         printType(sessionOutput.out(), sessionConfig);
         long errorCount = 1; // so far we just return 1 error
+        List<ExecutionWarning> warnings = Collections.emptyList(); // we don't have any warnings yet
         try {
-            if (param.statement == null || param.statement.isEmpty()) {
-                throw new AsterixException("Empty request, no statement provided");
+            if (param.getStatement() == null || param.getStatement().isEmpty()) {
+                throw new RuntimeDataException(ErrorCode.NO_STATEMENT_PROVIDED);
             }
-            String statementsText = param.statement + ";";
+            String statementsText = param.getStatement() + ";";
             Map<String, String> optionalParams = null;
             if (optionalParamProvider != null) {
                 optionalParams = optionalParamProvider.apply(request);
             }
+            Map<String, byte[]> statementParams = org.apache.asterix.app.translator.RequestParameters
+                    .serializeParameterValues(param.getStatementParams());
             // CORS
             if (request.getHeader("Origin") != null) {
                 response.setHeader("Access-Control-Allow-Origin", request.getHeader("Origin"));
             }
             response.setHeader("Access-Control-Allow-Headers", "Origin, X-Requested-With, Content-Type, Accept");
             response.setStatus(execution.getHttpStatus());
-            executeStatement(statementsText, sessionOutput, resultProperties, stats, param, execution, optionalParams);
+            executeStatement(statementsText, sessionOutput, resultProperties, stats, param, execution, optionalParams,
+                    statementParams);
             if (ResultDelivery.IMMEDIATE == delivery || ResultDelivery.DEFERRED == delivery) {
                 ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
             }
+            if (!warnings.isEmpty()) {
+                printWarnings(sessionOutput.out(), warnings);
+            }
             errorCount = 0;
         } catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
             handleExecuteStatementException(e, execution, param);
             response.setStatus(execution.getHttpStatus());
-            ResultUtil.printError(sessionOutput.out(), e);
+            printError(sessionOutput.out(), e);
             ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
         } finally {
             // make sure that we stop buffering and return the result to the http response
@@ -542,7 +573,7 @@
             execution.finish();
         }
         printMetrics(sessionOutput.out(), System.nanoTime() - elapsedStart, execution.duration(), stats.getCount(),
-                stats.getSize(), stats.getProcessedObjects(), errorCount);
+                stats.getSize(), stats.getProcessedObjects(), errorCount, warnings.size());
         sessionOutput.out().print("}\n");
         sessionOutput.out().flush();
         if (sessionOutput.out().checkError()) {
@@ -551,8 +582,9 @@
     }
 
     protected void executeStatement(String statementsText, SessionOutput sessionOutput,
-            ResultProperties resultProperties, IStatementExecutor.Stats stats, RequestParameters param,
-            RequestExecutionState execution, Map<String, String> optionalParameters) throws Exception {
+            ResultProperties resultProperties, Stats stats, QueryServiceRequestParameters param,
+            RequestExecutionState execution, Map<String, String> optionalParameters,
+            Map<String, byte[]> statementParameters) throws Exception {
         IClusterManagementWork.ClusterState clusterState =
                 ((ICcApplicationContext) appCtx).getClusterStateManager().getState();
         if (clusterState != IClusterManagementWork.ClusterState.ACTIVE) {
@@ -565,14 +597,18 @@
         IStatementExecutor translator = statementExecutorFactory.create((ICcApplicationContext) appCtx, statements,
                 sessionOutput, compilationProvider, componentProvider);
         execution.start();
-        final IRequestParameters requestParameters = new org.apache.asterix.app.translator.RequestParameters(
-                getHyracksDataset(), resultProperties, stats, null, param.clientContextID, optionalParameters);
+        Map<String, IAObject> stmtParams =
+                org.apache.asterix.app.translator.RequestParameters.deserializeParameterValues(statementParameters);
+        IRequestParameters requestParameters =
+                new org.apache.asterix.app.translator.RequestParameters(getResultSet(), resultProperties, stats, null,
+                        param.getClientContextID(), optionalParameters, stmtParams, param.isMultiStatement());
         translator.compileAndExecute(getHyracksClientConnection(), queryCtx, requestParameters);
         execution.end();
         printExecutionPlans(sessionOutput, translator.getExecutionPlans());
     }
 
-    protected void handleExecuteStatementException(Throwable t, RequestExecutionState state, RequestParameters param) {
+    protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
+            QueryServiceRequestParameters param) {
         if (t instanceof org.apache.asterix.aqlplus.parser.TokenMgrError || t instanceof TokenMgrError
                 || t instanceof AlgebricksException) {
             if (LOGGER.isDebugEnabled()) {
@@ -584,8 +620,8 @@
         } else if (t instanceof HyracksException) {
             HyracksException he = (HyracksException) t;
             switch (he.getComponent() + he.getErrorCode()) {
-                case ASTERIX + QUERY_TIMEOUT:
-                    LOGGER.info("handleException: query execution timed out: {}", param);
+                case ASTERIX + REQUEST_TIMEOUT:
+                    LOGGER.info("handleException: request execution timed out: {}", param);
                     state.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
                     break;
                 case ASTERIX + REJECT_BAD_CLUSTER_STATE:
@@ -593,6 +629,9 @@
                     LOGGER.warn("handleException: {}: {}", he.getMessage(), param);
                     state.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
                     break;
+                case ASTERIX + NO_STATEMENT_PROVIDED:
+                    state.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
+                    break;
                 default:
                     LOGGER.warn("handleException: unexpected exception {}: {}", he.getMessage(), param, he);
                     state.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
@@ -604,6 +643,14 @@
         }
     }
 
+    protected void printError(PrintWriter sessionOut, Throwable throwable) {
+        ResultUtil.printError(sessionOut, throwable);
+    }
+
+    protected void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
+        ResultUtil.printWarnings(pw, warnings);
+    }
+
     protected void printExecutionPlans(SessionOutput output, ExecutionPlans executionPlans) {
         final PrintWriter pw = output.out();
         pw.print("\t\"");
@@ -620,4 +667,31 @@
         }
         pw.print(",\n");
     }
+
+    private static boolean isJsonFormat(String format) {
+        return format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)
+                || format.equalsIgnoreCase(HttpUtil.ContentType.JSON);
+    }
+
+    public static String extractStatementParameterName(String name) {
+        int ln = name.length();
+        if (ln > 1 && name.charAt(0) == '$' && Character.isLetter(name.charAt(1))) {
+            if (ln == 2 || isStatementParameterNameRest(name, 2)) {
+                return name.substring(1);
+            }
+        }
+        return null;
+    }
+
+    private static boolean isStatementParameterNameRest(CharSequence input, int startIndex) {
+        int i = startIndex;
+        for (int ln = input.length(); i < ln; i++) {
+            char c = input.charAt(i);
+            boolean ok = c == '_' || Character.isLetterOrDigit(c);
+            if (!ok) {
+                return false;
+            }
+        }
+        return i > startIndex;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
index e55d82a..d983bfd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
@@ -28,8 +28,7 @@
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.hyracks.api.dataset.DatasetJobRecord;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.result.ResultJobRecord;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.utils.HttpUtil;
@@ -55,10 +54,9 @@
             return;
         }
 
-        IHyracksDataset hds = getHyracksDataset();
-        ResultReader resultReader = new ResultReader(hds, handle.getJobId(), handle.getResultSetId());
+        ResultReader resultReader = new ResultReader(getResultSet(), handle.getJobId(), handle.getResultSetId());
 
-        final DatasetJobRecord.Status resultReaderStatus = resultReader.getStatus();
+        final ResultJobRecord.Status resultReaderStatus = resultReader.getStatus();
         if (resultReaderStatus == null) {
             LOGGER.log(Level.INFO, "No results for: \"" + strHandle + "\"");
             response.setStatus(HttpResponseStatus.NOT_FOUND);
@@ -96,7 +94,7 @@
         }
     }
 
-    ResultStatus resultStatus(DatasetJobRecord.Status status) {
+    ResultStatus resultStatus(ResultJobRecord.Status status) {
         switch (status.getState()) {
             case IDLE:
             case RUNNING:
@@ -110,7 +108,7 @@
         }
     }
 
-    Exception extractException(DatasetJobRecord.Status status) {
+    Exception extractException(ResultJobRecord.Status status) {
         switch (status.getState()) {
             case FAILED:
                 List<Exception> exceptions = status.getExceptions();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
index 428a4e0..99c7308 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
@@ -19,13 +19,11 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_DATASET_ATTR;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -48,8 +46,7 @@
 import org.apache.asterix.translator.SessionConfig.PlanFormat;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
@@ -192,25 +189,15 @@
         try {
             response.setStatus(HttpResponseStatus.OK);
             IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
-            IHyracksDataset hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-            if (hds == null) {
-                synchronized (ctx) {
-                    hds = (IHyracksDataset) ctx.get(HYRACKS_DATASET_ATTR);
-                    if (hds == null) {
-                        hds = new HyracksDataset(hcc, appCtx.getCompilerProperties().getFrameSize(),
-                                ResultReader.NUM_READERS);
-                        ctx.put(HYRACKS_DATASET_ATTR, hds);
-                    }
-                }
-            }
             IParser parser = parserFactory.createParser(query);
             List<Statement> aqlStatements = parser.parse();
             validate(aqlStatements);
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator = statementExecutorFactory.create(appCtx, aqlStatements, sessionOutput,
                     compilationProvider, componentProvider);
-            final IRequestParameters requestParameters = new RequestParameters(hds,
-                    new ResultProperties(resultDelivery), new IStatementExecutor.Stats(), null, null, null);
+            final IResultSet resultSet = ServletUtil.getResultSet(hcc, appCtx, ctx);
+            final IRequestParameters requestParameters = new RequestParameters(resultSet,
+                    new ResultProperties(resultDelivery), new IStatementExecutor.Stats(), null, null, null, null, true);
             translator.compileAndExecute(hcc, null, requestParameters);
         } catch (AsterixException | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
             response.setStatus(HttpResponseStatus.INTERNAL_SERVER_ERROR);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
index ec128c2..8824f6a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
@@ -138,6 +138,25 @@
         pw.print(comma ? "\t}],\n" : "\t}]\n");
     }
 
+    public static void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
+        pw.print("\t\"");
+        pw.print(AbstractQueryApiServlet.ResultFields.WARNINGS.str());
+        pw.print("\": [");
+        for (int i = 0; i < warnings.size(); i++) {
+            final ExecutionWarning warning = warnings.get(i);
+            pw.print("{ \n\t");
+            printField(pw, QueryServiceServlet.ErrorField.CODE.str(), warning.getCode());
+            pw.print("\t");
+            printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil.escape(warning.getMessage()), false);
+            pw.print("\t} \n\t");
+            boolean lastWarning = i == warnings.size() - 1;
+            if (!lastWarning) {
+                pw.print(",");
+            }
+        }
+        pw.print("],\n");
+    }
+
     public static void printField(PrintWriter pw, String name, String value) {
         printField(pw, name, value, true);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletConstants.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletConstants.java
index 2fe37c3..f62d0c4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletConstants.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletConstants.java
@@ -20,7 +20,7 @@
 
 public class ServletConstants {
     public static final String HYRACKS_CONNECTION_ATTR = "org.apache.asterix.HYRACKS_CONNECTION";
-    public static final String HYRACKS_DATASET_ATTR = "org.apache.asterix.HYRACKS_DATASET";
+    public static final String RESULTSET_ATTR = "org.apache.asterix.RESULTSET";
     public static final String ASTERIX_APP_CONTEXT_INFO_ATTR = "org.apache.asterix.APP_CONTEXT_INFO";
     public static final String EXECUTOR_SERVICE_ATTR = "org.apache.asterix.EXECUTOR_SERVICE";
     public static final String RUNNING_QUERIES_ATTR = "org.apache.asterix.RUNINNG_QUERIES";
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
new file mode 100644
index 0000000..17e4c16
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.api.http.server;
+
+import static org.apache.asterix.api.http.server.ServletConstants.RESULTSET_ATTR;
+
+import java.util.Map;
+
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
+
+public class ServletUtil {
+    static IResultSet getResultSet(IHyracksClientConnection hcc, IApplicationContext appCtx,
+            final Map<String, Object> ctx) throws Exception {
+        IResultSet resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
+        if (resultSet == null) {
+            synchronized (ctx) {
+                resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
+                if (resultSet == null) {
+                    resultSet =
+                            new ResultSet(hcc, appCtx.getCompilerProperties().getFrameSize(), ResultReader.NUM_READERS);
+                    ctx.put(RESULTSET_ATTR, resultSet);
+                }
+            }
+        }
+        return resultSet;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
index 8dd4290..6b632a1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.api.http.server;
 
+import static org.apache.hyracks.util.NetworkUtil.toHostPort;
+
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetSocketAddress;
@@ -131,8 +133,7 @@
             ArrayNode replicasArray = OBJECT_MAPPER.createArrayNode();
             for (IPartitionReplica replica : replicas) {
                 final ObjectNode replicaJson = OBJECT_MAPPER.createObjectNode();
-                final InetSocketAddress location = replica.getIdentifier().getLocation();
-                replicaJson.put("location", location.getHostString() + ":" + location.getPort());
+                replicaJson.put("location", toHostPort(replica.getIdentifier().getLocation()));
                 replicaJson.put("status", replica.getStatus().toString());
                 replicasArray.add(replicaJson);
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
deleted file mode 100644
index 3650189..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.asterix.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class UpdateApiServlet extends RestApiServlet {
-    private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
-
-    public UpdateApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
-            ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
-            IStorageComponentProvider componentProvider) {
-        super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
-    }
-
-    @Override
-    protected String getQueryParameter(IServletRequest request) {
-        return request.getParameter("statements");
-    }
-
-    @Override
-    protected byte getAllowedCategories() {
-        return ALLOWED_CATEGORIES;
-    }
-
-    @Override
-    protected String getErrorMessage() {
-        return "Invalid statement: Non-Update statement %s to the Update API.";
-    }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index 0e51953..71b4b81 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -21,6 +21,7 @@
 import java.io.PrintWriter;
 import java.io.Reader;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.app.translator.RequestParameters;
@@ -32,6 +33,7 @@
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorFactory;
@@ -57,6 +59,7 @@
     private final IStatementExecutorFactory statementExecutorFactory;
     private final IStorageComponentProvider storageComponentProvider;
     private ICcApplicationContext appCtx;
+    private Map<String, IAObject> statementParams;
 
     public AsterixJavaClient(ICcApplicationContext appCtx, IHyracksClientConnection hcc, Reader queryText,
             PrintWriter writer, ILangCompilationProvider compilationProvider,
@@ -81,6 +84,10 @@
                 compilationProvider, statementExecutorFactory, storageComponentProvider);
     }
 
+    public void setStatementParameters(Map<String, IAObject> statementParams) {
+        this.statementParams = statementParams;
+    }
+
     public void compile() throws Exception {
         compile(true, false, true, false, false, false, false);
     }
@@ -121,7 +128,7 @@
                 storageComponentProvider);
         final IRequestParameters requestParameters =
                 new RequestParameters(null, new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE),
-                        new IStatementExecutor.Stats(), null, null, null);
+                        new IStatementExecutor.Stats(), null, null, null, statementParams, true);
         translator.compileAndExecute(hcc, null, requestParameters);
         writer.flush();
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 2a214e3..62f2c02 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -23,8 +23,10 @@
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.ActiveEvent.Kind;
@@ -34,9 +36,11 @@
 import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IRetryPolicyFactory;
 import org.apache.asterix.active.NoRetryPolicyFactory;
+import org.apache.asterix.active.message.ActiveManagerMessage;
 import org.apache.asterix.active.message.ActivePartitionMessage;
 import org.apache.asterix.active.message.ActivePartitionMessage.Event;
 import org.apache.asterix.active.message.ActiveStatsRequestMessage;
+import org.apache.asterix.active.message.StopRuntimeParameters;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -55,6 +59,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.util.InvokeUtil;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.hyracks.util.Span;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -62,12 +69,12 @@
 public abstract class ActiveEntityEventsListener implements IActiveEntityController {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final Level level = Level.INFO;
+    private static final Level level = Level.DEBUG;
     private static final ActiveEvent STATE_CHANGED = new ActiveEvent(null, Kind.STATE_CHANGED, null, null);
     private static final EnumSet<ActivityState> TRANSITION_STATES = EnumSet.of(ActivityState.RESUMING,
-            ActivityState.STARTING, ActivityState.STOPPING, ActivityState.RECOVERING);
+            ActivityState.STARTING, ActivityState.STOPPING, ActivityState.RECOVERING, ActivityState.CANCELLING);
     private static final String DEFAULT_ACTIVE_STATS = "{\"Stats\":\"N/A\"}";
-    // finals
+    protected static final TimeUnit TIMEOUT_UNIT = TimeUnit.SECONDS;
     protected final IClusterStateManager clusterStateManager;
     protected final ActiveNotificationHandler handler;
     protected final List<IActiveEntityEventSubscriber> subscribers = new ArrayList<>();
@@ -85,9 +92,9 @@
     private AlgebricksAbsolutePartitionConstraint locations;
     protected ActivityState prevState;
     protected JobId jobId;
-    protected long statsTimestamp;
+    protected volatile long statsTimestamp;
     protected String stats;
-    protected boolean isFetchingStats;
+    protected volatile boolean isFetchingStats;
     protected int numRegistered;
     protected int numDeRegistered;
     protected volatile RecoveryTask rt;
@@ -126,10 +133,15 @@
     }
 
     protected synchronized void setState(ActivityState newState) {
-        LOGGER.log(level, "State of " + getEntityId() + "is being set to " + newState + " from " + state);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "State of " + getEntityId() + "is being set to " + newState + " from " + state);
+        }
         this.prevState = state;
         this.state = newState;
-        if (newState == ActivityState.SUSPENDED) {
+        if (newState == ActivityState.STARTING || newState == ActivityState.RECOVERING
+                || newState == ActivityState.RESUMING) {
+            jobFailure = null;
+        } else if (newState == ActivityState.SUSPENDED) {
             suspended = true;
         }
         notifySubscribers(STATE_CHANGED);
@@ -138,14 +150,13 @@
     @Override
     public synchronized void notify(ActiveEvent event) {
         try {
-            LOGGER.log(level, "EventListener is notified.");
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "EventListener is notified.");
+            }
             ActiveEvent.Kind eventKind = event.getEventKind();
             switch (eventKind) {
                 case JOB_CREATED:
-                    jobCreated(event);
-                    break;
                 case JOB_STARTED:
-                    start(event);
                     break;
                 case JOB_FINISHED:
                     finish(event);
@@ -163,14 +174,10 @@
         }
     }
 
-    protected void jobCreated(ActiveEvent event) {
-        // Do nothing
-    }
-
     protected synchronized void handle(ActivePartitionMessage message) {
         if (message.getEvent() == Event.RUNTIME_REGISTERED) {
             numRegistered++;
-            if (numRegistered == locations.getLocations().length) {
+            if (allPartitionsRegisteredAndNotCancelling()) {
                 setState(ActivityState.RUNNING);
             }
         } else if (message.getEvent() == Event.RUNTIME_DEREGISTERED) {
@@ -178,9 +185,16 @@
         }
     }
 
+    private boolean allPartitionsRegisteredAndNotCancelling() {
+        return numRegistered == locations.getLocations().length && state != ActivityState.CANCELLING;
+    }
+
     @SuppressWarnings("unchecked")
     protected void finish(ActiveEvent event) throws HyracksDataException {
-        LOGGER.log(level, "the job " + jobId + " finished");
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "the job " + jobId + " finished");
+        }
+        JobId lastJobId = jobId;
         if (numRegistered != numDeRegistered) {
             LOGGER.log(Level.WARN,
                     "the job {} finished with reported runtime registrations = {} and deregistrations = {}", jobId,
@@ -190,11 +204,15 @@
         Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
         JobStatus jobStatus = status.getLeft();
         List<Exception> exceptions = status.getRight();
-        LOGGER.log(level, "The job finished with status: " + jobStatus);
-        if (jobStatus.equals(JobStatus.FAILURE)) {
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "The job finished with status: " + jobStatus);
+        }
+        if (!jobSuccessfullyTerminated(jobStatus)) {
             jobFailure = exceptions.isEmpty() ? new RuntimeDataException(ErrorCode.UNREPORTED_TASK_FAILURE_EXCEPTION)
                     : exceptions.get(0);
-            setState((state == ActivityState.STOPPING) ? ActivityState.STOPPED : ActivityState.TEMPORARILY_FAILED);
+            LOGGER.error("Active Job {} failed", lastJobId, jobFailure);
+            setState((state == ActivityState.STOPPING || state == ActivityState.CANCELLING) ? ActivityState.STOPPED
+                    : ActivityState.TEMPORARILY_FAILED);
             if (prevState == ActivityState.RUNNING) {
                 recover();
             }
@@ -203,14 +221,12 @@
         }
     }
 
-    protected void start(ActiveEvent event) {
-        jobId = event.getJobId();
-        numRegistered = 0;
-        numDeRegistered = 0;
+    private boolean jobSuccessfullyTerminated(JobStatus jobStatus) {
+        return jobStatus.equals(JobStatus.TERMINATED);
     }
 
     @Override
-    public synchronized void subscribe(IActiveEntityEventSubscriber subscriber) throws HyracksDataException {
+    public synchronized void subscribe(IActiveEntityEventSubscriber subscriber) {
         subscriber.subscribed(this);
         if (!subscriber.isDone()) {
             subscribers.add(subscriber);
@@ -315,11 +331,7 @@
             if (subscriber.isDone()) {
                 it.remove();
             } else {
-                try {
-                    subscriber.notify(event);
-                } catch (HyracksDataException e) {
-                    LOGGER.log(Level.WARN, "Failed to notify subscriber", e);
-                }
+                subscriber.notify(event);
                 if (subscriber.isDone()) {
                     it.remove();
                 }
@@ -342,22 +354,14 @@
         }
     }
 
-    /**
-     * this method is called before an action call is returned. It ensures that the request didn't fail
-     *
-     */
-    protected synchronized void checkNoFailure() throws HyracksDataException {
-        if (state == ActivityState.PERMANENTLY_FAILED) {
-            throw HyracksDataException.create(jobFailure);
-        }
-    }
-
     @Override
     public synchronized void recover() {
-        LOGGER.log(level, "Recover is called on " + entityId);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "Recover is called on " + entityId);
+        }
         if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
             LOGGER.log(level, "But it has no recovery policy, so it is set to permanent failure");
-            setState(ActivityState.PERMANENTLY_FAILED);
+            setState(ActivityState.STOPPED);
         } else {
             ExecutorService executor = appCtx.getServiceContext().getControllerService().getExecutor();
             setState(ActivityState.TEMPORARILY_FAILED);
@@ -371,7 +375,7 @@
     public synchronized void start(MetadataProvider metadataProvider)
             throws HyracksDataException, InterruptedException {
         waitForNonTransitionState();
-        if (state != ActivityState.PERMANENTLY_FAILED && state != ActivityState.STOPPED) {
+        if (state != ActivityState.STOPPED) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_ALREADY_STARTED, entityId, state);
         }
         try {
@@ -379,30 +383,144 @@
             doStart(metadataProvider);
             setRunning(metadataProvider, true);
         } catch (Exception e) {
-            setState(ActivityState.PERMANENTLY_FAILED);
+            setState(ActivityState.STOPPED);
             LOGGER.log(Level.ERROR, "Failed to start the entity " + entityId, e);
             throw HyracksDataException.create(e);
         }
     }
 
-    protected abstract void doStart(MetadataProvider metadataProvider) throws HyracksDataException;
+    @SuppressWarnings("squid:S1181")
+    protected synchronized void doStart(MetadataProvider metadataProvider) throws HyracksDataException {
+        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this,
+                EnumSet.of(ActivityState.RUNNING, ActivityState.TEMPORARILY_FAILED, ActivityState.STOPPED));
+        jobId = compileAndStartJob(metadataProvider);
+        numRegistered = 0;
+        numDeRegistered = 0;
+        try {
+            subscriber.sync();
+            if (subscriber.getFailure() != null) {
+                throw subscriber.getFailure();
+            }
+        } catch (InterruptedException ie) {
+            // interrupted.. check if the subscriber is done
+            if (subscriber.isDone()) {
+                if (subscriber.getFailure() != null) {
+                    throw HyracksDataException.create(subscriber.getFailure());
+                }
+                Thread.currentThread().interrupt();
+            } else {
+                // Subscriber is not done yet. so, we need to cancel, we have the jobId
+                setState(ActivityState.CANCELLING);
+                cancelJob(ie);
+                throw HyracksDataException.create(ie);
+            }
+        } catch (Throwable e) {
+            throw HyracksDataException.create(e);
+        }
+    }
 
-    protected abstract Void doStop(MetadataProvider metadataProvider) throws HyracksDataException;
+    private void cancelJob(Throwable th) {
+        cancelJobSafely(metadataProvider, th);
+        final WaitForStateSubscriber cancelSubscriber =
+                new WaitForStateSubscriber(this, EnumSet.of(ActivityState.STOPPED));
+        final Span span = Span.start(2, TimeUnit.MINUTES);
+        InvokeUtil.doUninterruptibly(() -> {
+            if (!cancelSubscriber.sync(span)) {
+                ExitUtil.halt(ExitUtil.EC_FAILED_TO_CANCEL_ACTIVE_START_STOP);
+            }
+        });
+    }
 
-    protected abstract Void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException;
+    @SuppressWarnings("squid:S1181")
+    protected void cancelJobSafely(MetadataProvider metadataProvider, Throwable e) {
+        try {
+            metadataProvider.getApplicationContext().getHcc().cancelJob(jobId);
+        } catch (Throwable th) {
+            LOGGER.warn("Failed to cancel active job", th);
+            e.addSuppressed(th);
+        }
+    }
+
+    protected abstract JobId compileAndStartJob(MetadataProvider metadataProvider) throws HyracksDataException;
+
+    @SuppressWarnings("squid:S1181")
+    protected synchronized void doStop(MetadataProvider metadataProvider, long timeout, TimeUnit unit)
+            throws HyracksDataException {
+        ActivityState intention = state;
+        Set<ActivityState> waitFor;
+        if (intention == ActivityState.STOPPING) {
+            waitFor = EnumSet.of(ActivityState.STOPPED);
+        } else if (intention == ActivityState.SUSPENDING) {
+            waitFor = EnumSet.of(ActivityState.SUSPENDED, ActivityState.TEMPORARILY_FAILED);
+        } else {
+            throw new IllegalStateException("stop with what intention?? Current state is " + intention);
+        }
+        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this, waitFor);
+        // Note: once we start sending stop messages, we can't go back until the entity is stopped
+        final String nameBefore = Thread.currentThread().getName();
+        try {
+            Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+            sendStopMessages(metadataProvider, timeout, unit);
+            if (LOGGER.isDebugEnabled()) {
+                LOGGER.debug("Waiting for its state to become " + waitFor);
+            }
+            subscriber.sync();
+            if (LOGGER.isDebugEnabled()) {
+                LOGGER.debug("Disconnect has been completed " + waitFor);
+            }
+        } catch (InterruptedException ie) {
+            forceStop(subscriber, ie);
+            Thread.currentThread().interrupt();
+        } catch (Throwable e) {
+            forceStop(subscriber, e);
+        } finally {
+            Thread.currentThread().setName(nameBefore);
+        }
+    }
+
+    private void forceStop(WaitForStateSubscriber subscriber, Throwable e) {
+        if (!subscriber.isDone()) {
+            cancelJob(e);
+        }
+        // Stop should not through an exception if the entity was stopped..
+        // Simply log
+        LOGGER.warn("Failure encountered while stopping {}", this, e);
+    }
+
+    protected void sendStopMessages(MetadataProvider metadataProvider, long timeout, TimeUnit unit) throws Exception {
+        ICcApplicationContext applicationCtx = metadataProvider.getApplicationContext();
+        ICCMessageBroker messageBroker = (ICCMessageBroker) applicationCtx.getServiceContext().getMessageBroker();
+        AlgebricksAbsolutePartitionConstraint runtimeLocations = getLocations();
+        int partition = 0;
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.log(Level.INFO, "Sending stop messages to " + runtimeLocations);
+        }
+        for (String location : runtimeLocations.getLocations()) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.log(Level.INFO, "Sending to " + location);
+            }
+            ActiveRuntimeId runtimeId = getActiveRuntimeId(partition++);
+            messageBroker.sendApplicationMessageToNC(new ActiveManagerMessage(ActiveManagerMessage.Kind.STOP_ACTIVITY,
+                    new StopRuntimeParameters(runtimeId, timeout, unit)), location);
+        }
+    }
+
+    protected abstract ActiveRuntimeId getActiveRuntimeId(int partition);
+
+    protected abstract void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException;
 
     protected abstract void doResume(MetadataProvider metadataProvider) throws HyracksDataException;
 
-    protected abstract void setRunning(MetadataProvider metadataProvider, boolean running) throws HyracksDataException;
+    protected abstract void setRunning(MetadataProvider metadataProvider, boolean running);
 
     @Override
-    public synchronized void stop(MetadataProvider metadataProvider) throws HyracksDataException, InterruptedException {
+    public final synchronized void stop(MetadataProvider metadataProvider)
+            throws HyracksDataException, InterruptedException {
         waitForNonTransitionState();
-        if (state != ActivityState.RUNNING && state != ActivityState.PERMANENTLY_FAILED
-                && state != ActivityState.TEMPORARILY_FAILED) {
+        if (state != ActivityState.RUNNING && state != ActivityState.TEMPORARILY_FAILED) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED, entityId, state);
         }
-        if (state == ActivityState.TEMPORARILY_FAILED || state == ActivityState.PERMANENTLY_FAILED) {
+        if (state == ActivityState.TEMPORARILY_FAILED) {
             if (rt != null) {
                 setState(ActivityState.STOPPING);
                 rt.cancel();
@@ -418,12 +536,13 @@
         } else if (state == ActivityState.RUNNING) {
             setState(ActivityState.STOPPING);
             try {
-                doStop(metadataProvider);
-                setRunning(metadataProvider, false);
+                doStop(metadataProvider, appCtx.getActiveProperties().getActiveStopTimeout(), TIMEOUT_UNIT);
             } catch (Exception e) {
-                setState(ActivityState.PERMANENTLY_FAILED);
+                setState(ActivityState.STOPPED);
                 LOGGER.log(Level.ERROR, "Failed to stop the entity " + entityId, e);
                 throw HyracksDataException.create(e);
+            } finally {
+                setRunning(metadataProvider, false);
             }
         } else {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_CANNOT_BE_STOPPED, entityId, state);
@@ -441,11 +560,15 @@
         WaitForStateSubscriber subscriber;
         Future<Void> suspendTask;
         synchronized (this) {
-            LOGGER.log(level, "suspending entity " + entityId);
-            LOGGER.log(level, "Waiting for ongoing activities");
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "suspending entity " + entityId);
+                LOGGER.log(level, "Waiting for ongoing activities");
+            }
             waitForNonTransitionState();
-            LOGGER.log(level, "Proceeding with suspension. Current state is " + state);
-            if (state == ActivityState.STOPPED || state == ActivityState.PERMANENTLY_FAILED) {
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "Proceeding with suspension. Current state is " + state);
+            }
+            if (state == ActivityState.STOPPED) {
                 suspended = true;
                 return;
             }
@@ -461,7 +584,10 @@
             subscriber = new WaitForStateSubscriber(this,
                     EnumSet.of(ActivityState.SUSPENDED, ActivityState.TEMPORARILY_FAILED));
             suspendTask = metadataProvider.getApplicationContext().getServiceContext().getControllerService()
-                    .getExecutor().submit(() -> doSuspend(metadataProvider));
+                    .getExecutor().submit(() -> {
+                        doSuspend(metadataProvider);
+                        return null;
+                    });
             LOGGER.log(level, "Suspension task has been submitted");
         }
         try {
@@ -471,7 +597,9 @@
             subscriber.sync();
         } catch (Exception e) {
             synchronized (this) {
-                LOGGER.log(Level.ERROR, "Failure while waiting for " + entityId + " to become suspended", e);
+                if (LOGGER.isErrorEnabled()) {
+                    LOGGER.log(Level.ERROR, "Failure while waiting for " + entityId + " to become suspended", e);
+                }
                 // failed to suspend
                 if (state == ActivityState.SUSPENDING) {
                     if (jobId != null) {
@@ -479,7 +607,7 @@
                         // restore state
                         setState(prevState);
                     } else {
-                        setState(ActivityState.PERMANENTLY_FAILED);
+                        setState(ActivityState.STOPPED);
                     }
                 }
                 throw HyracksDataException.create(e);
@@ -489,7 +617,7 @@
 
     @Override
     public synchronized void resume(MetadataProvider metadataProvider) throws HyracksDataException {
-        if (state == ActivityState.STOPPED || state == ActivityState.PERMANENTLY_FAILED) {
+        if (state == ActivityState.STOPPED) {
             suspended = false;
             notifyAll();
             return;
@@ -517,7 +645,7 @@
 
     @Override
     public boolean isActive() {
-        return state != ActivityState.STOPPED && state != ActivityState.PERMANENTLY_FAILED;
+        return state != ActivityState.STOPPED && state != ActivityState.CANCELLING;
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index e8bcb10..6eba4ea 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
@@ -33,7 +32,6 @@
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.metadata.api.IActiveEntityController;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.utils.DatasetUtil;
@@ -46,6 +44,7 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -54,7 +53,7 @@
         implements IActiveNotificationHandler, IJobLifecycleListener {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final Level level = Level.INFO;
+    private static final Level level = Level.DEBUG;
     public static final String ACTIVE_ENTITY_PROPERTY_NAME = "ActiveJob";
     private final Map<EntityId, IActiveEntityEventsListener> entityEventListeners;
     private final Map<JobId, EntityId> jobId2EntityId;
@@ -74,7 +73,9 @@
         EntityId entityId = jobId2EntityId.get(event.getJobId());
         if (entityId != null) {
             IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-            LOGGER.log(level, "Next event is of type " + event.getEventKind());
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "Next event is of type " + event.getEventKind());
+            }
             if (event.getEventKind() == Kind.JOB_FINISHED) {
                 LOGGER.log(level, "Removing the job");
                 jobId2EntityId.remove(event.getJobId());
@@ -92,11 +93,16 @@
 
     @Override
     public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) throws HyracksDataException {
-        LOGGER.log(level,
-                "notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = " + jobId);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level,
+                    "notifyJobCreation(JobId jobId, JobSpecification jobSpecification) was called with jobId = "
+                            + jobId);
+        }
         Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
-        if (property == null || !(property instanceof EntityId)) {
-            LOGGER.log(level, "Job is not of type active job. property found to be: " + property);
+        if (!(property instanceof EntityId)) {
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "Job is not of type active job. property found to be: " + property);
+            }
             return;
         }
         EntityId entityId = (EntityId) property;
@@ -107,17 +113,27 @@
     }
 
     private synchronized void monitorJob(JobId jobId, EntityId entityId) {
-        LOGGER.log(level, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "monitorJob(JobId jobId, ActiveJob activeJob) called with job id: " + jobId);
+        }
         boolean found = jobId2EntityId.get(jobId) != null;
-        LOGGER.log(level, "Job was found to be: " + (found ? "Active" : "Inactive"));
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "Job was found to be: " + (found ? "Active" : "Inactive"));
+        }
         if (entityEventListeners.containsKey(entityId)) {
             if (jobId2EntityId.containsKey(jobId)) {
-                LOGGER.error("Job is already being monitored for job: " + jobId);
+                if (LOGGER.isErrorEnabled()) {
+                    LOGGER.error("Job is already being monitored for job: " + jobId);
+                }
                 return;
             }
-            LOGGER.log(level, "monitoring started for job id: " + jobId);
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "monitoring started for job id: " + jobId);
+            }
         } else {
-            LOGGER.info("No listener was found for the entity: " + entityId);
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.info("No listener was found for the entity: " + entityId);
+            }
         }
         jobId2EntityId.put(jobId, entityId);
     }
@@ -133,12 +149,16 @@
     @Override
     public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
             throws HyracksException {
-        LOGGER.log(level, "Getting notified of job finish for JobId: " + jobId);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "Getting notified of job finish for JobId: " + jobId);
+        }
         EntityId entityId = jobId2EntityId.get(jobId);
         if (entityId != null) {
             add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId, Pair.of(jobStatus, exceptions)));
         } else {
-            LOGGER.log(level, "NO NEED TO NOTIFY JOB FINISH!");
+            if (LOGGER.isEnabled(level)) {
+                LOGGER.log(level, "no need to notify job finish");
+            }
         }
     }
 
@@ -152,16 +172,22 @@
 
     @Override
     public IActiveEntityEventsListener getListener(EntityId entityId) {
-        LOGGER.log(level, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "getActiveEntityListener(EntityId entityId) was called with entity " + entityId);
+        }
         IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-        LOGGER.log(level, "Listener found: " + listener);
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "Listener found: " + listener);
+        }
         return entityEventListeners.get(entityId);
     }
 
     @Override
     public synchronized IActiveEntityEventsListener[] getEventListeners() {
-        LOGGER.log(level, "getEventListeners() was called");
-        LOGGER.log(level, "returning " + entityEventListeners.size() + " Listeners");
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("getEventListeners() was called");
+            LOGGER.trace("returning " + entityEventListeners.size() + " Listeners");
+        }
         return entityEventListeners.values().toArray(new IActiveEntityEventsListener[entityEventListeners.size()]);
     }
 
@@ -170,8 +196,10 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        LOGGER.log(level, "registerListener(IActiveEntityEventsListener listener) was called for the entity "
-                + listener.getEntityId());
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "registerListener(IActiveEntityEventsListener listener) was called for the entity "
+                    + listener.getEntityId());
+        }
         if (entityEventListeners.containsKey(listener.getEntityId())) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_IS_ALREADY_REGISTERED, listener.getEntityId());
         }
@@ -183,8 +211,10 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        LOGGER.log(level, "unregisterListener(IActiveEntityEventsListener listener) was called for the entity "
-                + listener.getEntityId());
+        if (LOGGER.isEnabled(level)) {
+            LOGGER.log(level, "unregisterListener(IActiveEntityEventsListener listener) was called for the entity "
+                    + listener.getEntityId());
+        }
         IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
         if (registeredListener == null) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED, listener.getEntityId());
@@ -209,19 +239,14 @@
     }
 
     @Override
-    public synchronized void recover() {
-        LOGGER.log(level, "Starting active recovery");
-        for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
+    public void recover() {
+        LOGGER.info("Starting active recovery");
+        for (IActiveEntityEventsListener listener : getEventListeners()) {
             synchronized (listener) {
-                LOGGER.log(level, "Entity " + listener.getEntityId() + " is " + listener.getStats());
-                if (listener.getState() == ActivityState.PERMANENTLY_FAILED
-                        && listener instanceof IActiveEntityController) {
-                    LOGGER.log(level, "Recovering");
-                    ((IActiveEntityController) listener).recover();
-                } else {
-                    LOGGER.log(level, "Only notifying");
-                    listener.notifyAll();
+                if (LOGGER.isEnabled(level)) {
+                    LOGGER.log(level, "Entity " + listener.getEntityId() + " is " + listener.getState());
                 }
+                listener.notifyAll();
             }
         }
     }
@@ -235,33 +260,51 @@
             LOGGER.log(level, "Suspending active events handler");
             suspended = true;
         }
-        IMetadataLockManager lockManager = mdProvider.getApplicationContext().getMetadataLockManager();
-        Collection<IActiveEntityEventsListener> registeredListeners = entityEventListeners.values();
-        for (IActiveEntityEventsListener listener : registeredListeners) {
-            // write lock the listener
-            // exclusive lock all the datasets
-            String dataverseName = listener.getEntityId().getDataverse();
-            String entityName = listener.getEntityId().getEntityName();
-            LOGGER.log(level, "Suspending " + listener.getEntityId());
-            LOGGER.log(level, "Acquiring locks");
-            lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
-            List<Dataset> datasets = ((ActiveEntityEventsListener) listener).getDatasets();
-            for (Dataset dataset : datasets) {
-                lockManager.acquireDatasetExclusiveModificationLock(mdProvider.getLocks(),
-                        DatasetUtil.getFullyQualifiedName(dataset));
+        try {
+            IMetadataLockManager lockManager = mdProvider.getApplicationContext().getMetadataLockManager();
+            Collection<IActiveEntityEventsListener> registeredListeners = entityEventListeners.values();
+            for (IActiveEntityEventsListener listener : registeredListeners) {
+                // write lock the listener
+                // exclusive lock all the datasets
+                String dataverseName = listener.getEntityId().getDataverse();
+                String entityName = listener.getEntityId().getEntityName();
+                if (LOGGER.isEnabled(level)) {
+                    LOGGER.log(level, "Suspending " + listener.getEntityId());
+                }
+                LOGGER.log(level, "Acquiring locks");
+                lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                List<Dataset> datasets = ((ActiveEntityEventsListener) listener).getDatasets();
+                for (Dataset dataset : datasets) {
+                    lockManager.acquireDatasetExclusiveModificationLock(mdProvider.getLocks(),
+                            DatasetUtil.getFullyQualifiedName(dataset));
+                }
+                LOGGER.log(level, "locks acquired");
+                ((ActiveEntityEventsListener) listener).suspend(mdProvider);
+                if (LOGGER.isEnabled(level)) {
+                    LOGGER.log(level, listener.getEntityId() + " suspended");
+                }
             }
-            LOGGER.log(level, "locks acquired");
-            ((ActiveEntityEventsListener) listener).suspend(mdProvider);
-            LOGGER.log(level, listener.getEntityId() + " suspended");
+        } catch (Throwable th) {
+            LOGGER.error("Suspend active failed", th);
+            ExitUtil.halt(ExitUtil.EC_ACTIVE_SUSPEND_FAILURE);
         }
     }
 
-    public void resume(MetadataProvider mdProvider) throws HyracksDataException, InterruptedException {
+    public void resume(MetadataProvider mdProvider) throws HyracksDataException {
         LOGGER.log(level, "Resuming active events handler");
-        for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
-            LOGGER.log(level, "Resuming " + listener.getEntityId());
-            ((ActiveEntityEventsListener) listener).resume(mdProvider);
-            LOGGER.log(level, listener.getEntityId() + " resumed");
+        try {
+            for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
+                if (LOGGER.isEnabled(level)) {
+                    LOGGER.log(level, "Resuming " + listener.getEntityId());
+                }
+                ((ActiveEntityEventsListener) listener).resume(mdProvider);
+                if (LOGGER.isEnabled(level)) {
+                    LOGGER.log(level, listener.getEntityId() + " resumed");
+                }
+            }
+        } catch (Throwable th) {
+            LOGGER.error("Resume active failed", th);
+            ExitUtil.halt(ExitUtil.EC_ACTIVE_RESUME_FAILURE);
         }
         synchronized (this) {
             suspended = false;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
index fbf644f..d4a52fb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
@@ -21,21 +21,17 @@
 import java.util.EnumSet;
 import java.util.List;
 
+import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IRetryPolicyFactory;
-import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.app.translator.QueryTranslator;
-import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.utils.JobUtils;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
-import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -45,9 +41,9 @@
 import org.apache.asterix.utils.FeedOperations;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public class FeedEventsListener extends ActiveEntityEventsListener {
@@ -80,58 +76,44 @@
     }
 
     @Override
-    protected void doStart(MetadataProvider mdProvider) throws HyracksDataException {
+    public synchronized void start(MetadataProvider metadataProvider)
+            throws HyracksDataException, InterruptedException {
+        super.start(metadataProvider);
+        // Note: The current implementation of the wait for completion flag is problematic due to locking issues:
+        // Locks obtained during the start of the feed are not released, and so, the feed can't be stopped
+        // and also, read locks over dataverses, datasets, etc, are never released.
+        boolean wait =
+                Boolean.parseBoolean((String) metadataProvider.getConfig().get(StartFeedStatement.WAIT_FOR_COMPLETION));
+        if (wait) {
+            IActiveEntityEventSubscriber stoppedSubscriber =
+                    new WaitForStateSubscriber(this, EnumSet.of(ActivityState.STOPPED));
+            stoppedSubscriber.sync();
+        }
+    }
+
+    @Override
+    protected JobId compileAndStartJob(MetadataProvider mdProvider) throws HyracksDataException {
         try {
             Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo =
                     FeedOperations.buildStartFeedJob(mdProvider, feed, feedConnections, statementExecutor, hcc);
             JobSpecification feedJob = jobInfo.getLeft();
-            WaitForStateSubscriber eventSubscriber = new WaitForStateSubscriber(this, EnumSet.of(ActivityState.RUNNING,
-                    ActivityState.TEMPORARILY_FAILED, ActivityState.PERMANENTLY_FAILED));
             feedJob.setProperty(ActiveNotificationHandler.ACTIVE_ENTITY_PROPERTY_NAME, entityId);
             // TODO(Yingyi): currently we do not check IFrameWriter protocol violations for Feed jobs.
             // We will need to design general exception handling mechanism for feeds.
             setLocations(jobInfo.getRight());
-            boolean wait = Boolean.parseBoolean(mdProvider.getConfig().get(StartFeedStatement.WAIT_FOR_COMPLETION));
-            JobUtils.runJob(hcc, feedJob, false);
-            eventSubscriber.sync();
-            if (eventSubscriber.getFailure() != null) {
-                throw eventSubscriber.getFailure();
-            }
-            if (wait) {
-                IActiveEntityEventSubscriber stoppedSubscriber = new WaitForStateSubscriber(this,
-                        EnumSet.of(ActivityState.STOPPED, ActivityState.PERMANENTLY_FAILED));
-                stoppedSubscriber.sync();
-            }
+            return JobUtils.runJob(hcc, feedJob, false);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
     }
 
     @Override
-    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException {
-        IActiveEntityEventSubscriber eventSubscriber =
-                new WaitForStateSubscriber(this, EnumSet.of(ActivityState.STOPPED, ActivityState.PERMANENTLY_FAILED));
-        try {
-            // Construct ActiveMessage
-            for (int i = 0; i < getLocations().getLocations().length; i++) {
-                String intakeLocation = getLocations().getLocations()[i];
-                FeedOperations.SendStopMessageToNode(metadataProvider.getApplicationContext(), entityId, intakeLocation,
-                        i);
-            }
-            eventSubscriber.sync();
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-        return null;
-    }
-
-    @Override
-    protected void setRunning(MetadataProvider metadataProvider, boolean running) throws HyracksDataException {
+    protected void setRunning(MetadataProvider metadataProvider, boolean running) {
         // No op
     }
 
     @Override
-    protected Void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException {
+    protected void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException {
         throw new RuntimeDataException(ErrorCode.OPERATION_NOT_SUPPORTED);
     }
 
@@ -139,4 +121,9 @@
     protected void doResume(MetadataProvider metadataProvider) throws HyracksDataException {
         throw new RuntimeDataException(ErrorCode.OPERATION_NOT_SUPPORTED);
     }
+
+    @Override
+    protected ActiveRuntimeId getActiveRuntimeId(int partition) {
+        return new ActiveRuntimeId(entityId, FeedIntakeOperatorNodePushable.class.getSimpleName(), partition);
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index 2de8319..0172b28 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -21,7 +21,6 @@
 import java.util.concurrent.Callable;
 
 import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.IRetryPolicy;
 import org.apache.asterix.active.IRetryPolicyFactory;
 import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -34,6 +33,7 @@
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.IRetryPolicy;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -47,7 +47,6 @@
     private final IRetryPolicyFactory retryPolicyFactory;
     private final MetadataProvider metadataProvider;
     private final IClusterStateManager clusterStateManager;
-    private Exception failure;
 
     public RecoveryTask(ICcApplicationContext appCtx, ActiveEntityEventsListener listener,
             IRetryPolicyFactory retryPolicyFactory) {
@@ -92,7 +91,8 @@
             if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
                 synchronized (listener) {
                     if (!cancelRecovery) {
-                        listener.setState(ActivityState.PERMANENTLY_FAILED);
+                        listener.setState(ActivityState.STOPPED);
+                        listener.setRunning(metadataProvider, false);
                     }
                 }
             } else {
@@ -104,56 +104,60 @@
         }
     }
 
-    protected Void doRecover(IRetryPolicy policy)
-            throws AlgebricksException, HyracksDataException, InterruptedException {
+    protected Void doRecover(IRetryPolicy policy) throws AlgebricksException, InterruptedException {
         LOGGER.log(level, "Actual Recovery task has started");
-        if (listener.getState() != ActivityState.TEMPORARILY_FAILED) {
-            LOGGER.log(level, "but its state is not temp failure and so we're just returning");
-            return null;
-        }
-        LOGGER.log(level, "calling the policy");
-        while (policy.retry()) {
+        Exception failure = null;
+        while (policy.retry(failure)) {
             synchronized (listener) {
-                if (cancelRecovery) {
-                    return null;
-                }
-                while (clusterStateManager.getState() != ClusterState.ACTIVE) {
-                    if (cancelRecovery) {
-                        return null;
-                    }
+                while (!cancelRecovery && clusterStateManager.getState() != ClusterState.ACTIVE) {
                     listener.wait();
                 }
+                if (cancelRecovery) {
+                    LOGGER.log(level, "Recovery has been cancelled");
+                    return null;
+                }
             }
             IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
-            lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
-                    listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
-            for (Dataset dataset : listener.getDatasets()) {
-                lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dataset.getDataverseName());
-                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
-                        DatasetUtil.getFullyQualifiedName(dataset));
-            }
-            synchronized (listener) {
-                try {
-                    if (cancelRecovery) {
-                        return null;
-                    }
-                    listener.setState(ActivityState.RECOVERING);
-                    listener.doStart(metadataProvider);
-                    return null;
-                } catch (Exception e) {
-                    LOGGER.log(level, "Attempt to revive " + listener.getEntityId() + " failed", e);
-                    listener.setState(ActivityState.TEMPORARILY_FAILED);
-                    failure = e;
-                } finally {
-                    metadataProvider.getLocks().reset();
+            try {
+                lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
+                        listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
+                for (Dataset dataset : listener.getDatasets()) {
+                    lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dataset.getDataverseName());
+                    lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
+                            DatasetUtil.getFullyQualifiedName(dataset));
                 }
-                listener.notifyAll();
+                synchronized (listener) {
+                    try {
+                        if (!cancelRecovery && listener.getState() == ActivityState.TEMPORARILY_FAILED) {
+                            listener.setState(ActivityState.RECOVERING);
+                            listener.doStart(metadataProvider);
+                        }
+                        LOGGER.log(level, "Recovery completed successfully");
+                        return null;
+                    } finally {
+                        listener.notifyAll();
+                    }
+                }
+            } catch (Exception e) {
+                LOGGER.log(level, "Attempt to revive " + listener.getEntityId() + " failed", e);
+                listener.setState(ActivityState.TEMPORARILY_FAILED);
+                failure = e;
+            } finally {
+                metadataProvider.getLocks().reset();
             }
         }
         // Recovery task is essntially over now either through failure or through cancellation(stop)
         synchronized (listener) {
             listener.notifyAll();
-            if (listener.getState() != ActivityState.TEMPORARILY_FAILED) {
+            if (listener.getState() != ActivityState.TEMPORARILY_FAILED
+                    // Suspend can happen at the same time, the recovery policy decides to stop... in that case, we
+                    // must still do two things:
+                    // 1. set the state to permanent failure.
+                    // 2. set the entity to not running to avoid auto recovery attempt
+                    && listener.getState() != ActivityState.SUSPENDED) {
+                LOGGER.log(level, "Recovery is cancelled because the current state {} is neither {} nor {}",
+                        listener.getState(), ActivityState.TEMPORARILY_FAILED,
+                        listener.getState() != ActivityState.SUSPENDED);
                 return null;
             }
         }
@@ -166,11 +170,10 @@
                         DatasetUtil.getFullyQualifiedName(dataset));
             }
             synchronized (listener) {
-                if (cancelRecovery) {
-                    return null;
-                }
-                if (listener.getState() == ActivityState.TEMPORARILY_FAILED) {
-                    listener.setState(ActivityState.PERMANENTLY_FAILED);
+                if (!cancelRecovery && listener.getState() == ActivityState.TEMPORARILY_FAILED) {
+                    LOGGER.warn("Recovery for {} permanently failed", listener.getEntityId());
+                    listener.setState(ActivityState.STOPPED);
+                    listener.setRunning(metadataProvider, false);
                 }
                 listener.notifyAll();
             }
@@ -179,8 +182,4 @@
         }
         return null;
     }
-
-    public Exception getFailure() {
-        return failure;
-    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 1ba418a..7f632f0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -28,12 +28,15 @@
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.common.api.ExtensionId;
 import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.hyracks.bootstrap.GlobalRecoveryManager;
 import org.apache.asterix.om.functions.IFunctionExtensionManager;
 import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.runtime.functions.FunctionCollection;
@@ -41,6 +44,8 @@
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.utils.ExtensionUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
@@ -53,6 +58,7 @@
     private final ILangCompilationProvider aqlCompilationProvider;
     private final ILangCompilationProvider sqlppCompilationProvider;
     private final IFunctionManager functionManager;
+    private final IGlobalRecoveryExtension globalRecoveryExtension;
     private transient IStatementExecutorFactory statementExecutorFactory;
 
     /**
@@ -71,6 +77,7 @@
         Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
         Pair<ExtensionId, IFunctionManager> fm = null;
         IStatementExecutorExtension see = null;
+        IGlobalRecoveryExtension gre = null;
         if (list != null) {
             Set<ExtensionId> extensionIds = new HashSet<>();
             for (AsterixExtension extensionConf : list) {
@@ -89,6 +96,9 @@
                         sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
                         fm = ExtensionUtil.extendFunctionManager(fm, le);
                         break;
+                    case RECOVERY:
+                        gre = (IGlobalRecoveryExtension) extension;
+                        break;
                     default:
                         break;
                 }
@@ -99,6 +109,7 @@
         this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
         this.functionManager =
                 fm == null ? new FunctionManager(FunctionCollection.createDefaultFunctionCollection()) : fm.second;
+        this.globalRecoveryExtension = gre;
     }
 
     /** @deprecated use getStatementExecutorFactory instead */
@@ -127,6 +138,14 @@
         }
     }
 
+    public IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+            IStorageComponentProvider componentProvider) {
+        if (globalRecoveryExtension == null) {
+            return new GlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+        }
+        return globalRecoveryExtension.getGlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+    }
+
     @Override
     public IFunctionManager getFunctionManager() {
         return functionManager;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java
new file mode 100644
index 0000000..e9652a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java
@@ -0,0 +1,40 @@
+/*
+ * 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.asterix.app.cc;
+
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+
+/**
+ * An interface for extensions of {@code IGlobalRecoveryManager}
+ */
+public interface IGlobalRecoveryExtension extends IExtension {
+
+    @Override
+    default ExtensionKind getExtensionKind() {
+        return ExtensionKind.RECOVERY;
+    }
+
+    IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+            IStorageComponentProvider componentProvider);
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
index a575ba4..47c1045 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.function;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -45,7 +47,8 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
         }
         return new DatasetResourcesDatasource(context.getComputationNodeDomain(), dataset.getDatasetId());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
index c857ce0..590bf54 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
@@ -22,6 +22,8 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -68,7 +70,8 @@
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
             // TODO remove this after enabling the support of positional variables in data scan
-            throw new AlgebricksException("No positional variables are allowed over datasets.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over datasets.");
         }
         ILogicalExpression expr = f.getArguments().get(0).getValue();
         if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
@@ -90,7 +93,8 @@
         String datasetName = datasetReference.second;
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, unnest.getSourceLocation(),
+                    datasetName, dataverseName);
         }
         DataSourceId asid = new DataSourceId(dataverseName, datasetName);
         List<LogicalVariable> variables = new ArrayList<>();
@@ -107,6 +111,7 @@
             variables.add(context.newVar());
         }
         DataSourceScanOperator scan = new DataSourceScanOperator(variables, dataSource);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
index 51aca5d..bb80406 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.FeedUtils;
@@ -70,7 +72,8 @@
         AbstractFunctionCallExpression f = UnnestToDataScanRule.getFunctionCall(opRef);
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
-            throw new AlgebricksException("No positional variables are allowed over feeds.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over feeds.");
         }
         String dataverse = ConstantExpressionUtil.getStringArgument(f, 0);
         String sourceFeedName = ConstantExpressionUtil.getStringArgument(f, 1);
@@ -80,16 +83,17 @@
         String outputType = ConstantExpressionUtil.getStringArgument(f, 5);
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         DataSourceId asid = new DataSourceId(dataverse, getTargetFeed);
-        String policyName = metadataProvider.getConfig().get(FeedActivityDetails.FEED_POLICY_NAME);
+        String policyName = (String) metadataProvider.getConfig().get(FeedActivityDetails.FEED_POLICY_NAME);
         FeedPolicyEntity policy = metadataProvider.findFeedPolicy(dataverse, policyName);
         if (policy == null) {
             policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
             if (policy == null) {
-                throw new AlgebricksException("Unknown feed policy:" + policyName);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                        "Unknown feed policy:" + policyName);
             }
         }
         ArrayList<LogicalVariable> feedDataScanOutputVariables = new ArrayList<>();
-        String csLocations = metadataProvider.getConfig().get(FeedActivityDetails.COLLECT_LOCATIONS);
+        String csLocations = (String) metadataProvider.getConfig().get(FeedActivityDetails.COLLECT_LOCATIONS);
         List<LogicalVariable> pkVars = new ArrayList<>();
         FeedDataSource ds = createFeedDataSource(asid, targetDataset, sourceFeedName, subscriptionLocation,
                 metadataProvider, policy, outputType, csLocations, unnest.getVariable(), context, pkVars);
@@ -104,6 +108,7 @@
             feedDataScanOutputVariables.addAll(pkVars);
         }
         DataSourceScanOperator scan = new DataSourceScanOperator(feedDataScanOutputVariables, ds);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
@@ -152,10 +157,10 @@
                 List<String> key = partitioningKeys.get(i);
                 if (keySourceIndicator == null || keySourceIndicator.get(i).intValue() == 0) {
                     PlanTranslationUtil.prepareVarAndExpression(key, recordVar, pkVars, keyAccessExpression, null,
-                            context);
+                            context, null);
                 } else {
                     PlanTranslationUtil.prepareMetaKeyAccessExpression(key, recordVar, keyAccessExpression, pkVars,
-                            null, context);
+                            null, context, null);
                 }
             }
             keyAccessExpression.forEach(
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index 2ff9282..2bc5ab6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.FunctionDataSource;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -55,24 +57,29 @@
         AbstractFunctionCallExpression f = UnnestToDataScanRule.getFunctionCall(opRef);
         List<Mutable<ILogicalExpression>> args = f.getArguments();
         if (args.size() != functionId.getArity()) {
-            throw new AlgebricksException("Function " + functionId.getNamespace() + "." + functionId.getName()
-                    + " expects " + functionId.getArity() + " arguments");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, f.getSourceLocation(),
+                    "Function " + functionId.getNamespace() + "." + functionId.getName() + " expects "
+                            + functionId.getArity() + " arguments");
         }
         for (int i = 0; i < args.size(); i++) {
-            if (args.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                throw new AlgebricksException("Function " + functionId.getNamespace() + "." + functionId.getName()
-                        + " expects constant arguments while arg[" + i + "] is of type "
-                        + args.get(i).getValue().getExpressionTag());
+            ILogicalExpression argExpr = args.get(i).getValue();
+            if (argExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, argExpr.getSourceLocation(),
+                        "Function " + functionId.getNamespace() + "." + functionId.getName()
+                                + " expects constant arguments while arg[" + i + "] is of type "
+                                + argExpr.getExpressionTag());
             }
         }
         UnnestOperator unnest = (UnnestOperator) opRef.getValue();
         if (unnest.getPositionalVariable() != null) {
-            throw new AlgebricksException("No positional variables are allowed over datasource functions");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
+                    "No positional variables are allowed over datasource functions");
         }
         FunctionDataSource datasource = toDatasource(context, f);
         List<LogicalVariable> variables = new ArrayList<>();
         variables.add(unnest.getVariable());
         DataSourceScanOperator scan = new DataSourceScanOperator(variables, datasource);
+        scan.setSourceLocation(unnest.getSourceLocation());
         List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
         scanInpList.addAll(unnest.getInputs());
         opRef.setValue(scan);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java
new file mode 100644
index 0000000..480f3c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingDatasource.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.app.function;
+
+import org.apache.asterix.metadata.api.IDatasourceFunction;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+
+public class PingDatasource extends FunctionDataSource {
+
+    private static final DataSourceId PING_DATASOURCE_ID =
+            new DataSourceId(PingRewriter.PING.getNamespace(), PingRewriter.PING.getName());
+
+    public PingDatasource(INodeDomain domain) throws AlgebricksException {
+        super(PING_DATASOURCE_ID, domain);
+    }
+
+    @Override
+    protected IDatasourceFunction createFunction(MetadataProvider metadataProvider,
+            AlgebricksAbsolutePartitionConstraint locations) {
+        return new PingFunction(locations);
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingFunction.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingFunction.java
new file mode 100644
index 0000000..c37ed7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingFunction.java
@@ -0,0 +1,41 @@
+/*
+ * 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.asterix.app.function;
+
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.metadata.declared.AbstractDatasourceFunction;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PingFunction extends AbstractDatasourceFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    public PingFunction(AlgebricksAbsolutePartitionConstraint locations) {
+        super(locations);
+    }
+
+    @Override
+    public IRecordReader<char[]> createRecordReader(IHyracksTaskContext ctx, int partition)
+            throws HyracksDataException {
+        return new PingReader(ctx.getJobletContext().getServiceContext().getNodeId());
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingReader.java
new file mode 100644
index 0000000..70a03ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingReader.java
@@ -0,0 +1,50 @@
+/*
+ * 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.asterix.app.function;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.input.record.CharArrayRecord;
+
+public class PingReader extends FunctionReader {
+
+    private final String nodeId;
+    private boolean consumed = false;
+
+    public PingReader(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        return !consumed;
+    }
+
+    @Override
+    public IRawRecord<char[]> next() throws IOException, InterruptedException {
+        consumed = true;
+        CharArrayRecord record = new CharArrayRecord();
+        record.reset();
+        record.append(("{\"node\":\"" + nodeId + "\"}").toCharArray());
+        record.endRecord();
+        return record;
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingRewriter.java
new file mode 100644
index 0000000..2fab648
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/PingRewriter.java
@@ -0,0 +1,43 @@
+/*
+ * 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.asterix.app.function;
+
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class PingRewriter extends FunctionRewriter {
+
+    public static final FunctionIdentifier PING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ping", 0);
+    public static final PingRewriter INSTANCE = new PingRewriter(PING);
+
+    private PingRewriter(FunctionIdentifier functionId) {
+        super(functionId);
+    }
+
+    @Override
+    protected FunctionDataSource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
+            throws AlgebricksException {
+        return new PingDatasource(context.getComputationNodeDomain());
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
index 89bd115..259e6d7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.function;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -45,7 +47,8 @@
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
         }
         return new StorageComponentsDatasource(context.getComputationNodeDomain(), dataset.getDatasetId());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
new file mode 100644
index 0000000..a7034e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -0,0 +1,304 @@
+/*
+ * 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.asterix.app.io;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
+import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.dataflow.DatasetLocalResource;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.transactions.Checkpoint;
+import org.apache.asterix.dataflow.data.common.AListElementTokenFactory;
+import org.apache.asterix.dataflow.data.common.AOrderedListBinaryTokenizerFactory;
+import org.apache.asterix.dataflow.data.common.AUnorderedListBinaryTokenizerFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalAscPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalDescPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ARectanglePartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AUUIDPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.BooleanBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ListItemBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.LongBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.RawBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
+import org.apache.asterix.formats.nontagged.AnyBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.OrderedBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.OrderedLinearizeComparatorFactory;
+import org.apache.asterix.metadata.utils.SecondaryCorrelatedTreeIndexOperationsHelper;
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.om.pointables.nonvisitor.AListPointable;
+import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
+import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
+import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.primitive.BooleanPointable;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+import org.apache.hyracks.data.std.primitive.BytePointable;
+import org.apache.hyracks.data.std.primitive.DoublePointable;
+import org.apache.hyracks.data.std.primitive.FixedLengthTypeTrait;
+import org.apache.hyracks.data.std.primitive.FloatPointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.RawUTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.ShortPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringLowercasePointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringLowercaseTokenPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VarLengthTypeTrait;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.storage.am.common.data.PointablePrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeLocalResource;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyLocalResource;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
+import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResource;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizerFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
+import org.apache.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeLocalResource;
+import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeLocalResource;
+import org.apache.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterLocalResource;
+import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import org.apache.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.rtree.impls.FloatPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.rtree.impls.IntegerPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.rtree.linearize.HilbertDoubleComparatorFactory;
+import org.apache.hyracks.storage.am.rtree.linearize.ZCurveDoubleComparatorFactory;
+import org.apache.hyracks.storage.am.rtree.linearize.ZCurveIntComparatorFactory;
+import org.apache.hyracks.storage.common.LocalResource;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class PersistedResourceRegistry implements IPersistedResourceRegistry {
+
+    private static final String DESERIALIZATION_METHOD = "fromJson";
+    private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
+    protected static final Map<String, Class<? extends IJsonSerializable>> REGISTERED_CLASSES = new HashMap<>();
+
+    public PersistedResourceRegistry() {
+        registerClasses();
+        ensureFromJsonMethod();
+    }
+
+    protected void registerClasses() {
+        /* WARNING: Changing a resource id will break storage format backward compatibility.*/
+        REGISTERED_CLASSES.put("Checkpoint", Checkpoint.class);
+
+        // IResource
+        REGISTERED_CLASSES.put("LocalResource", LocalResource.class);
+        REGISTERED_CLASSES.put("DatasetLocalResource", DatasetLocalResource.class);
+        REGISTERED_CLASSES.put("LSMBTreeLocalResource", LSMBTreeLocalResource.class);
+        REGISTERED_CLASSES.put("LSMRTreeLocalResource", LSMRTreeLocalResource.class);
+        REGISTERED_CLASSES.put("LSMRTreeWithAntiMatterLocalResource", LSMRTreeWithAntiMatterLocalResource.class);
+        REGISTERED_CLASSES.put("LSMInvertedIndexLocalResource", LSMInvertedIndexLocalResource.class);
+        REGISTERED_CLASSES.put("ExternalBTreeLocalResource", ExternalBTreeLocalResource.class);
+        REGISTERED_CLASSES.put("ExternalBTreeWithBuddyLocalResource", ExternalBTreeWithBuddyLocalResource.class);
+        REGISTERED_CLASSES.put("ExternalRTreeLocalResource", ExternalRTreeLocalResource.class);
+
+        // ILSMMergePolicyFactory
+        REGISTERED_CLASSES.put("NoMergePolicyFactory", NoMergePolicyFactory.class);
+        REGISTERED_CLASSES.put("PrefixMergePolicyFactory", PrefixMergePolicyFactory.class);
+        REGISTERED_CLASSES.put("ConstantMergePolicyFactory", ConstantMergePolicyFactory.class);
+        REGISTERED_CLASSES.put("CorrelatedPrefixMergePolicyFactory", CorrelatedPrefixMergePolicyFactory.class);
+
+        // ILSMIOOperationSchedulerProvider
+        REGISTERED_CLASSES.put("RuntimeComponentsProvider", RuntimeComponentsProvider.class);
+
+        // ITypeTraits
+        REGISTERED_CLASSES.put("FixedLengthTypeTrait", FixedLengthTypeTrait.class);
+        REGISTERED_CLASSES.put("VarLengthTypeTrait", VarLengthTypeTrait.class);
+
+        // ILSMOperationTrackerFactory
+        REGISTERED_CLASSES.put("PrimaryIndexOperationTrackerFactory", PrimaryIndexOperationTrackerFactory.class);
+        REGISTERED_CLASSES.put("SecondaryIndexOperationTrackerFactory", SecondaryIndexOperationTrackerFactory.class);
+
+        // ILSMComponentIdGeneratorFactory
+        REGISTERED_CLASSES.put("DatasetLSMComponentIdGeneratorFactory", DatasetLSMComponentIdGeneratorFactory.class);
+
+        // IDatasetInfoProvider
+        REGISTERED_CLASSES.put("DatasetInfoProvider", DatasetInfoProvider.class);
+
+        // ILSMOperationTrackerFactory
+        REGISTERED_CLASSES.put("NoOpIOOperationCallbackFactory", NoOpIOOperationCallbackFactory.class);
+        REGISTERED_CLASSES.put("LSMBTreeIOOperationCallbackFactory", LSMIndexIOOperationCallbackFactory.class);
+
+        // ILSMIOOperationSchedulerProvider
+        REGISTERED_CLASSES.put("AppendOnlyLinkedMetadataPageManagerFactory",
+                AppendOnlyLinkedMetadataPageManagerFactory.class);
+
+        // ILSMIOOperationSchedulerProvider
+        REGISTERED_CLASSES.put("AsterixVirtualBufferCacheProvider", AsterixVirtualBufferCacheProvider.class);
+
+        // IBinaryComparatorFactory
+        REGISTERED_CLASSES.put("ACirclePartialBinaryComparatorFactory", ACirclePartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("ADurationPartialBinaryComparatorFactory",
+                ADurationPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("AIntervalAscPartialBinaryComparatorFactory",
+                AIntervalAscPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("AIntervalDescPartialBinaryComparatorFactory",
+                AIntervalDescPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("ALinePartialBinaryComparatorFactory", ALinePartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("AObjectAscBinaryComparatorFactory", AObjectAscBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("AObjectDescBinaryComparatorFactory", AObjectDescBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("APoint3DPartialBinaryComparatorFactory", APoint3DPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("APointPartialBinaryComparatorFactory", APointPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("APolygonPartialBinaryComparatorFactory", APolygonPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("ARectanglePartialBinaryComparatorFactory",
+                ARectanglePartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("AUUIDPartialBinaryComparatorFactory", AUUIDPartialBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("BooleanBinaryComparatorFactory", BooleanBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("ListItemBinaryComparatorFactory", ListItemBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("LongBinaryComparatorFactory", LongBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("RawBinaryComparatorFactory", RawBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("PointableBinaryComparatorFactory", PointableBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("HilbertDoubleComparatorFactory", HilbertDoubleComparatorFactory.class);
+        REGISTERED_CLASSES.put("ZCurveDoubleComparatorFactory", ZCurveDoubleComparatorFactory.class);
+        REGISTERED_CLASSES.put("ZCurveIntComparatorFactory", ZCurveIntComparatorFactory.class);
+        REGISTERED_CLASSES.put("ComponentPosComparatorFactory",
+                SecondaryCorrelatedTreeIndexOperationsHelper.ComponentPosComparatorFactory.class);
+        REGISTERED_CLASSES.put("AnyBinaryComparatorFactory", AnyBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("OrderedBinaryComparatorFactory", OrderedBinaryComparatorFactory.class);
+        REGISTERED_CLASSES.put("OrderedLinearizeComparatorFactory", OrderedLinearizeComparatorFactory.class);
+
+        // IPointableFactory
+        REGISTERED_CLASSES.put("AIntervalPointableFactory", AIntervalPointable.AIntervalPointableFactory.class);
+        REGISTERED_CLASSES.put("AListPointableFactory", AListPointable.AListPointableFactory.class);
+        REGISTERED_CLASSES.put("ARecordPointableFactory", ARecordPointable.ARecordPointableFactory.class);
+        REGISTERED_CLASSES.put("BooleanPointableFactory", BooleanPointable.BooleanPointableFactory.class);
+        REGISTERED_CLASSES.put("ByteArrayPointableFactory", ByteArrayPointable.ByteArrayPointableFactory.class);
+        REGISTERED_CLASSES.put("BytePointableFactory", BytePointable.BytePointableFactory.class);
+        REGISTERED_CLASSES.put("DoublePointableFactory", DoublePointable.DoublePointableFactory.class);
+        REGISTERED_CLASSES.put("FloatPointableFactory", FloatPointable.FloatPointableFactory.class);
+        REGISTERED_CLASSES.put("IntegerPointableFactory", IntegerPointable.IntegerPointableFactory.class);
+        REGISTERED_CLASSES.put("LongPointableFactory", LongPointable.LongPointableFactory.class);
+        REGISTERED_CLASSES.put("RawUTF8StringPointableFactory",
+                RawUTF8StringPointable.RawUTF8StringPointableFactory.class);
+        REGISTERED_CLASSES.put("ShortPointableFactory", ShortPointable.ShortPointableFactory.class);
+        REGISTERED_CLASSES.put("TaggedValuePointableFactory", TaggedValuePointable.TaggedValuePointableFactory.class);
+        REGISTERED_CLASSES.put("UTF8StringLowercasePointableFactory",
+                UTF8StringLowercasePointable.UTF8StringLowercasePointableFactory.class);
+        REGISTERED_CLASSES.put("UTF8StringLowercaseTokenPointableFactory",
+                UTF8StringLowercaseTokenPointable.UTF8StringLowercaseTokenPointableFactory.class);
+        REGISTERED_CLASSES.put("UTF8StringPointableFactory", UTF8StringPointable.UTF8StringPointableFactory.class);
+        REGISTERED_CLASSES.put("VoidPointableFactory", VoidPointable.VoidPointableFactory.class);
+
+        // IPrimitiveValueProviderFactory
+        REGISTERED_CLASSES.put("DoublePrimitiveValueProviderFactory", DoublePrimitiveValueProviderFactory.class);
+        REGISTERED_CLASSES.put("FloatPrimitiveValueProviderFactory", FloatPrimitiveValueProviderFactory.class);
+        REGISTERED_CLASSES.put("IntegerPrimitiveValueProviderFactory", IntegerPrimitiveValueProviderFactory.class);
+        REGISTERED_CLASSES.put("PointablePrimitiveValueProviderFactory", PointablePrimitiveValueProviderFactory.class);
+        REGISTERED_CLASSES.put("PrimitiveValueProviderFactory", PrimitiveValueProviderFactory.class);
+
+        // IBinaryTokenizerFactory
+        REGISTERED_CLASSES.put("AOrderedListBinaryTokenizerFactory", AOrderedListBinaryTokenizerFactory.class);
+        REGISTERED_CLASSES.put("AUnorderedListBinaryTokenizerFactory", AUnorderedListBinaryTokenizerFactory.class);
+        REGISTERED_CLASSES.put("NGramUTF8StringBinaryTokenizerFactory", NGramUTF8StringBinaryTokenizerFactory.class);
+        REGISTERED_CLASSES.put("DelimitedUTF8StringBinaryTokenizerFactory",
+                DelimitedUTF8StringBinaryTokenizerFactory.class);
+
+        // ITokenFactory
+        REGISTERED_CLASSES.put("AListElementTokenFactory", AListElementTokenFactory.class);
+        REGISTERED_CLASSES.put("HashedUTF8NGramTokenFactory", HashedUTF8NGramTokenFactory.class);
+        REGISTERED_CLASSES.put("HashedUTF8WordTokenFactory", HashedUTF8WordTokenFactory.class);
+        REGISTERED_CLASSES.put("UTF8NGramTokenFactory", UTF8NGramTokenFactory.class);
+        REGISTERED_CLASSES.put("UTF8WordTokenFactory", UTF8WordTokenFactory.class);
+        REGISTERED_CLASSES.put("RTreePolicyType", RTreePolicyType.class);
+    }
+
+    @Override
+    public IJsonSerializable deserialize(JsonNode json) throws HyracksDataException {
+        try {
+            String resourceId = json.get(TYPE_FIELD_ID).asText();
+            Class<? extends IJsonSerializable> clazz = getResourceClass(resourceId);
+
+            //Using static method (fromJson)
+            Method method = clazz.getMethod(DESERIALIZATION_METHOD, IPersistedResourceRegistry.class, JsonNode.class);
+            return (IJsonSerializable) method.invoke(null, this, json);
+        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public ObjectNode getClassIdentifier(Class<? extends IJsonSerializable> clazz, long version) {
+        final ObjectNode objectNode = JSON_MAPPER.createObjectNode();
+        objectNode.put(IPersistedResourceRegistry.TYPE_FIELD_ID, getResourceId(clazz));
+        objectNode.put(IPersistedResourceRegistry.VERSION_FIELD_ID, version);
+        objectNode.put(IPersistedResourceRegistry.CLASS_FIELD_ID, clazz.getName());
+        return objectNode;
+    }
+
+    private String getResourceId(Class<? extends IJsonSerializable> clazz) {
+        Optional<String> classId = REGISTERED_CLASSES.entrySet().stream()
+                .filter(entry -> Objects.equals(entry.getValue(), clazz)).map(Map.Entry::getKey).findAny();
+        if (classId.isPresent()) {
+            return classId.get();
+        }
+        throw new IllegalStateException(String.format("Class %s was not registered.", clazz.getName()));
+    }
+
+    private Class<? extends IJsonSerializable> getResourceClass(String id) {
+        return REGISTERED_CLASSES.computeIfAbsent(id, key -> {
+            throw new IllegalStateException(String.format("No class with id %s was registered.", key));
+        });
+    }
+
+    protected static void ensureFromJsonMethod() {
+        for (Class<?> clazz : REGISTERED_CLASSES.values()) {
+            try {
+                // Ensure fromJson method exists with expected parameters
+                clazz.getMethod(DESERIALIZATION_METHOD, IPersistedResourceRegistry.class, JsonNode.class);
+            } catch (NoSuchMethodException e) {
+                throw new IllegalStateException(String
+                        .format("Registered class %s must provide a static method fromJson(IPersistedResourceRegistry,"
+                                + " JsonNode)", clazz.getName()),
+                        e);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index a1c87d8..943aad3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -18,15 +18,14 @@
  */
 package org.apache.asterix.app.message;
 
+import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.utils.RequestStatus;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -51,17 +50,16 @@
         ClusterControllerService ccs = (ClusterControllerService) appCtx.getServiceContext().getControllerService();
         CCApplication application = (CCApplication) ccs.getApplication();
         IStatementExecutorContext executorsCtx = application.getStatementExecutorContext();
-        JobId jobId = executorsCtx.getJobIdFromClientContextId(contextId);
+        IClientRequest req = executorsCtx.get(contextId);
         RequestStatus status;
 
-        if (jobId == null) {
+        if (req == null) {
             LOGGER.log(Level.WARN, "No job found for context id " + contextId);
             status = RequestStatus.NOT_FOUND;
         } else {
             try {
-                IHyracksClientConnection hcc = application.getHcc();
-                hcc.cancelJob(jobId);
-                executorsCtx.removeJobIdFromClientContextId(contextId);
+                req.cancel(appCtx);
+                executorsCtx.remove(contextId);
                 status = RequestStatus.SUCCESS;
             } catch (Exception e) {
                 LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index d295304..88b5da8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -44,6 +44,7 @@
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorContext;
@@ -64,7 +65,7 @@
     private static final long serialVersionUID = 1L;
     private static final Logger LOGGER = LogManager.getLogger();
     //TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2062
-    public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
+    public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(Long.MAX_VALUE);
     //TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2063
     public static final long DEFAULT_QUERY_CANCELLATION_WAIT_MILLIS = TimeUnit.MINUTES.toMillis(1);
     private final String requestNodeId;
@@ -76,10 +77,13 @@
     private final String clientContextID;
     private final String handleUrl;
     private final Map<String, String> optionalParameters;
+    private final Map<String, byte[]> statementParameters;
+    private final boolean multiStatement;
 
     public ExecuteStatementRequestMessage(String requestNodeId, long requestMessageId, ILangExtension.Language lang,
             String statementsText, SessionConfig sessionConfig, ResultProperties resultProperties,
-            String clientContextID, String handleUrl, Map<String, String> optionalParameters) {
+            String clientContextID, String handleUrl, Map<String, String> optionalParameters,
+            Map<String, byte[]> statementParameters, boolean multiStatement) {
         this.requestNodeId = requestNodeId;
         this.requestMessageId = requestMessageId;
         this.lang = lang;
@@ -89,6 +93,8 @@
         this.clientContextID = clientContextID;
         this.handleUrl = handleUrl;
         this.optionalParameters = optionalParameters;
+        this.statementParameters = statementParameters;
+        this.multiStatement = multiStatement;
     }
 
     @Override
@@ -124,8 +130,9 @@
             IStatementExecutor translator = statementExecutorFactory.create(ccAppCtx, statements, sessionOutput,
                     compilationProvider, storageComponentProvider);
             final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
+            Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
             final IRequestParameters requestParameters = new RequestParameters(null, resultProperties, stats,
-                    outMetadata, clientContextID, optionalParameters);
+                    outMetadata, clientContextID, optionalParameters, stmtParams, multiStatement);
             translator.compileAndExecute(ccApp.getHcc(), statementExecutorContext, requestParameters);
             outPrinter.close();
             responseMsg.setResult(outWriter.toString());
@@ -139,7 +146,7 @@
             responseMsg.setError(pe);
         } catch (Exception e) {
             GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "Unexpected exception", e);
-            responseMsg.setError(new Exception(e.toString()));
+            responseMsg.setError(e);
         }
         try {
             messageBroker.sendApplicationMessageToNC(responseMsg, requestNodeId);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
new file mode 100644
index 0000000..17a4f46
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
@@ -0,0 +1,49 @@
+/*
+ * 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.asterix.app.nc;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HaltCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+    public static final HaltCallback INSTANCE = new HaltCallback();
+
+    private HaltCallback() {
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler has failed", failure);
+        ExitUtil.halt(ExitUtil.EC_IO_SCHEDULER_FAILED);
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} has failed", operation, t);
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            ExitUtil.halt(ExitUtil.EC_FLUSH_FAILED);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
index a012f1e..420585a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
@@ -22,13 +22,14 @@
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.nio.channels.ClosedByInterruptException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
-import java.util.Optional;
 
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IndexCheckpoint;
@@ -54,29 +55,36 @@
     }
 
     @Override
-    public synchronized void init(long lsn) throws HyracksDataException {
-        final List<IndexCheckpoint> checkpoints = getCheckpoints();
+    public synchronized void init(long validComponentSequence, long lsn, long validComponentId)
+            throws HyracksDataException {
+        List<IndexCheckpoint> checkpoints;
+        try {
+            checkpoints = getCheckpoints();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
         if (!checkpoints.isEmpty()) {
             LOGGER.warn(() -> "Checkpoints found on initializing: " + indexPath);
             delete();
         }
-        IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(lsn);
+        IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(validComponentSequence, lsn, validComponentId);
         persist(firstCheckpoint);
     }
 
     @Override
-    public synchronized void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException {
+    public synchronized void replicated(long componentSequence, long masterLsn, long componentId)
+            throws HyracksDataException {
         final Long localLsn = getLatest().getMasterNodeFlushMap().get(masterLsn);
         if (localLsn == null) {
             throw new IllegalStateException("Component flushed before lsn mapping was received");
         }
-        flushed(componentTimestamp, localLsn);
+        flushed(componentSequence, localLsn, componentId);
     }
 
     @Override
-    public synchronized void flushed(String componentTimestamp, long lsn) throws HyracksDataException {
+    public synchronized void flushed(long componentSequence, long lsn, long componentId) throws HyracksDataException {
         final IndexCheckpoint latest = getLatest();
-        IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentTimestamp);
+        IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentSequence, componentId);
         persist(nextCheckpoint);
         deleteHistory(nextCheckpoint.getId(), HISTORY_CHECKPOINTS);
     }
@@ -85,19 +93,19 @@
     public synchronized void masterFlush(long masterLsn, long localLsn) throws HyracksDataException {
         final IndexCheckpoint latest = getLatest();
         latest.getMasterNodeFlushMap().put(masterLsn, localLsn);
-        final IndexCheckpoint next =
-                IndexCheckpoint.next(latest, latest.getLowWatermark(), latest.getValidComponentTimestamp());
+        final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+                latest.getValidComponentSequence(), latest.getLastComponentId());
         persist(next);
         notifyAll();
     }
 
     @Override
-    public synchronized long getLowWatermark() {
+    public synchronized long getLowWatermark() throws HyracksDataException {
         return getLatest().getLowWatermark();
     }
 
     @Override
-    public synchronized boolean isFlushed(long masterLsn) {
+    public synchronized boolean isFlushed(long masterLsn) throws HyracksDataException {
         if (masterLsn == BULKLOAD_LSN) {
             return true;
         }
@@ -110,32 +118,63 @@
     }
 
     @Override
-    public Optional<String> getValidComponentTimestamp() {
-        final String validComponentTimestamp = getLatest().getValidComponentTimestamp();
-        return validComponentTimestamp != null ? Optional.of(validComponentTimestamp) : Optional.empty();
+    public long getValidComponentSequence() throws HyracksDataException {
+        return getLatest().getValidComponentSequence();
     }
 
     @Override
-    public int getCheckpointCount() {
-        return getCheckpoints().size();
+    public int getCheckpointCount() throws HyracksDataException {
+        try {
+            return getCheckpoints().size();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
     }
 
-    private IndexCheckpoint getLatest() {
-        final List<IndexCheckpoint> checkpoints = getCheckpoints();
+    @Override
+    public synchronized IndexCheckpoint getLatest() throws HyracksDataException {
+        List<IndexCheckpoint> checkpoints;
+        try {
+            checkpoints = getCheckpoints();
+        } catch (ClosedByInterruptException e) {
+            throw HyracksDataException.create(e);
+        }
         if (checkpoints.isEmpty()) {
+            LOGGER.warn("Couldn't find any checkpoint file for index {}. Content of dir are {}.", indexPath,
+                    Arrays.toString(indexPath.toFile().listFiles()));
             throw new IllegalStateException("Couldn't find any checkpoints for resource: " + indexPath);
         }
         checkpoints.sort(Comparator.comparingLong(IndexCheckpoint::getId).reversed());
         return checkpoints.get(0);
     }
 
-    private List<IndexCheckpoint> getCheckpoints() {
+    @Override
+    public synchronized void setLastComponentId(long componentId) throws HyracksDataException {
+        final IndexCheckpoint latest = getLatest();
+        final IndexCheckpoint next =
+                IndexCheckpoint.next(latest, latest.getLowWatermark(), latest.getValidComponentSequence(), componentId);
+        persist(next);
+    }
+
+    @Override
+    public synchronized void advanceValidComponentSequence(long componentSequence) throws HyracksDataException {
+        final IndexCheckpoint latest = getLatest();
+        if (componentSequence > latest.getValidComponentSequence()) {
+            final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(), componentSequence,
+                    latest.getLastComponentId());
+            persist(next);
+        }
+    }
+
+    private List<IndexCheckpoint> getCheckpoints() throws ClosedByInterruptException {
         List<IndexCheckpoint> checkpoints = new ArrayList<>();
         final File[] checkpointFiles = indexPath.toFile().listFiles(CHECKPOINT_FILE_FILTER);
         if (checkpointFiles != null) {
             for (File checkpointFile : checkpointFiles) {
                 try {
                     checkpoints.add(read(checkpointFile.toPath()));
+                } catch (ClosedByInterruptException e) {
+                    throw e;
                 } catch (IOException e) {
                     LOGGER.warn(() -> "Couldn't read index checkpoint file: " + checkpointFile, e);
                 }
@@ -158,6 +197,9 @@
                 // ensure it was written correctly by reading it
                 read(checkpointPath);
                 return;
+            } catch (ClosedByInterruptException e) {
+                LOGGER.info("interrupted while writing checkpoint at {}", checkpointPath);
+                throw HyracksDataException.create(e);
             } catch (IOException e) {
                 if (i == MAX_CHECKPOINT_WRITE_ATTEMPTS) {
                     throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
index 19ad8f6..e0b3105 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
 
@@ -53,14 +54,10 @@
 
     private IndexCheckpointManager create(ResourceReference ref) {
         try {
-            final Path indexPath = getIndexPath(ref);
+            final Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
             return new IndexCheckpointManager(indexPath);
         } catch (HyracksDataException e) {
             throw new IllegalStateException(e);
         }
     }
-
-    private Path getIndexPath(ResourceReference indexRef) throws HyracksDataException {
-        return ioManager.resolve(indexRef.getRelativePath().toString()).getFile().toPath();
-    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 288e5f2..3524474 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -52,7 +52,6 @@
 import org.apache.asterix.common.context.DatasetLifecycleManager;
 import org.apache.asterix.common.context.DatasetMemoryManager;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.replication.IReplicationChannel;
@@ -61,6 +60,7 @@
 import org.apache.asterix.common.storage.IReplicaManager;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.asterix.file.StorageComponentProvider;
@@ -82,6 +82,7 @@
 import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.control.nc.NodeControllerService;
@@ -138,6 +139,7 @@
     private final ILibraryManager libraryManager;
     private final NCExtensionManager ncExtensionManager;
     private final IStorageComponentProvider componentProvider;
+    private final IPersistedResourceRegistry persistedResourceRegistry;
     private IHyracksClientConnection hcc;
     private IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
     private IReplicaManager replicaManager;
@@ -166,10 +168,11 @@
         ncExtensionManager = new NCExtensionManager(allExtensions);
         componentProvider = new StorageComponentProvider();
         resourceIdFactory = new GlobalResourceIdFactoryProvider(ncServiceContext).createResourceIdFactory();
+        persistedResourceRegistry = ncServiceContext.getPersistedResourceRegistry();
     }
 
     @Override
-    public void initialize(boolean initialRun) throws IOException, ACIDException {
+    public void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun) throws IOException {
         ioManager = getServiceContext().getIoManager();
         threadExecutor =
                 MaintainedThreadNameExecutorService.newCachedThreadPool(getServiceContext().getThreadFactory());
@@ -177,20 +180,15 @@
         IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
-
-        AsynchronousScheduler.INSTANCE.init(getServiceContext().getThreadFactory());
-        lsmIOScheduler = AsynchronousScheduler.INSTANCE;
-
+        lsmIOScheduler = new AsynchronousScheduler(getServiceContext().getThreadFactory(), HaltCallback.INSTANCE);
         metadataMergePolicyFactory = new PrefixMergePolicyFactory();
         indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
-
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
-                new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
-
+                new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider,
+                        persistedResourceRegistry);
         localResourceRepository =
                 (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
-
-        txnSubsystem = new TransactionSubsystem(this);
+        txnSubsystem = new TransactionSubsystem(this, recoveryManagerFactory);
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
         if (initialRun || systemState == SystemState.PERMANENT_DATA_LOSS) {
@@ -514,4 +512,9 @@
         return Math.max(MetadataManager.INSTANCE == null ? 0 : MetadataManager.INSTANCE.getMaxTxnId(),
                 txnSubsystem.getTransactionManager().getMaxTxnId());
     }
+
+    @Override
+    public IPersistedResourceRegistry getPersistedResourceRegistry() {
+        return persistedResourceRegistry;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index d4e652d..946815f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -48,7 +48,7 @@
 import org.apache.asterix.common.context.IndexInfo;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -77,6 +77,8 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -102,14 +104,14 @@
     private static final String RECOVERY_FILES_DIR_NAME = "recovery_temp";
     private Map<Long, JobEntityCommits> jobId2WinnerEntitiesMap = null;
     private final long cachedEntityCommitsPerJobSize;
-    private final PersistentLocalResourceRepository localResourceRepository;
+    protected final PersistentLocalResourceRepository localResourceRepository;
     private final ICheckpointManager checkpointManager;
     private SystemState state;
-    private final INCServiceContext serviceCtx;
-    private final INcApplicationContext appCtx;
+    protected final INCServiceContext serviceCtx;
+    protected final INcApplicationContext appCtx;
     private static final TxnId recoveryTxnId = new TxnId(-1);
 
-    public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
+    public RecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem) {
         this.serviceCtx = serviceCtx;
         this.txnSubsystem = txnSubsystem;
         this.appCtx = txnSubsystem.getApplicationContext();
@@ -225,6 +227,7 @@
                     break;
                 case LogType.FLUSH:
                 case LogType.WAIT:
+                case LogType.WAIT_FOR_FLUSHES:
                 case LogType.MARKER:
                 case LogType.FILTER:
                     break;
@@ -293,11 +296,10 @@
                 ((INcApplicationContext) (serviceCtx.getApplicationContext())).getIndexCheckpointManagerProvider();
 
         Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
-        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<>();
+        final Map<Long, Long> resourceId2MaxLSNMap = new HashMap<>();
         TxnEntityId tempKeyTxnEntityId = new TxnEntityId(-1, -1, -1, null, -1, false);
 
         ILogRecord logRecord = null;
-        ILSMComponentIdGenerator idGenerator = null;
         try {
             logReader.setPosition(lowWaterMarkLSN);
             logRecord = logReader.next();
@@ -386,39 +388,34 @@
                         int partition = logRecord.getResourcePartition();
                         if (partitions.contains(partition)) {
                             int datasetId = logRecord.getDatasetId();
-                            idGenerator = datasetLifecycleManager.getComponentIdGenerator(datasetId, partition);
-                            if (idGenerator == null) {
+                            if (!datasetLifecycleManager.isRegistered(datasetId)) {
                                 // it's possible this dataset has been dropped
                                 logRecord = logReader.next();
                                 continue;
                             }
-                            idGenerator.refresh();
                             DatasetInfo dsInfo = datasetLifecycleManager.getDatasetInfo(datasetId);
                             // we only need to flush open indexes here (opened by previous update records)
                             // if an index has no ongoing updates, then it's memory component must be empty
                             // and there is nothing to flush
-                            for (IndexInfo iInfo : dsInfo.getIndexes().values()) {
+                            for (final IndexInfo iInfo : dsInfo.getIndexes().values()) {
                                 if (iInfo.isOpen() && iInfo.getPartition() == partition) {
-                                    maxDiskLastLsn = resourceId2MaxLSNMap.get(iInfo.getResourceId());
-                                    index = iInfo.getIndex();
-                                    AbstractLSMIOOperationCallback ioCallback =
-                                            (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
-                                    if (logRecord.getLSN() > maxDiskLastLsn
-                                            && !index.isCurrentMutableComponentEmpty()) {
-                                        // schedule flush
-                                        ioCallback.updateLastLSN(logRecord.getLSN());
-                                        redoFlush(index, logRecord);
-                                        redoCount++;
-                                    } else {
-                                        if (index.isMemoryComponentsAllocated()) {
-                                            // if the memory component has been allocated, we
-                                            // force it to receive the same Id
-                                            index.getCurrentMemoryComponent().resetId(idGenerator.getId(), true);
+                                    Long maxLsnBeforeFlush = resourceId2MaxLSNMap.get(iInfo.getResourceId());
+                                    if (maxLsnBeforeFlush != null) {
+                                        // If there was at least one update to the resource.
+                                        // IMPORTANT: Don't remove the check above
+                                        // This check is to support indexes without transaction logs
+                                        maxDiskLastLsn = maxLsnBeforeFlush;
+                                        index = iInfo.getIndex();
+                                        if (logRecord.getLSN() > maxDiskLastLsn
+                                                && !index.isCurrentMutableComponentEmpty()) {
+                                            // schedule flush
+                                            redoFlush(index, logRecord);
+                                            redoCount++;
                                         } else {
-                                            // otherwise, we refresh the id stored in ioCallback
-                                            // to ensure the memory component receives correct Id upon activation
-                                            ioCallback.forceRefreshNextId();
+                                            // TODO: update checkpoint file?
                                         }
+                                    } else {
+                                        // TODO: update checkpoint file?
                                     }
                                 }
                             }
@@ -428,6 +425,7 @@
                     case LogType.ENTITY_COMMIT:
                     case LogType.ABORT:
                     case LogType.WAIT:
+                    case LogType.WAIT_FOR_FLUSHES:
                     case LogType.MARKER:
                         //do nothing
                         break;
@@ -473,10 +471,10 @@
         long minFirstLSN = logMgr.getAppendLSN();
         if (!openIndexList.isEmpty()) {
             for (IIndex index : openIndexList) {
-                AbstractLSMIOOperationCallback ioCallback =
-                        (AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
+                LSMIOOperationCallback ioCallback =
+                        (LSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
                 if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
-                    firstLSN = ioCallback.getFirstLSN();
+                    firstLSN = ioCallback.getPersistenceLsn();
                     minFirstLSN = Math.min(minFirstLSN, firstLSN);
                 }
             }
@@ -684,6 +682,7 @@
                     case LogType.FLUSH:
                     case LogType.FILTER:
                     case LogType.WAIT:
+                    case LogType.WAIT_FOR_FLUSHES:
                     case LogType.MARKER:
                         //ignore
                         break;
@@ -824,10 +823,15 @@
     }
 
     private static void redoFlush(ILSMIndex index, ILogRecord logRecord) throws HyracksDataException {
+        long flushLsn = logRecord.getLSN();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        accessor.getOpContext().setParameters(flushMap);
         long minId = logRecord.getFlushingComponentMinId();
         long maxId = logRecord.getFlushingComponentMaxId();
         ILSMComponentId id = new LSMComponentId(minId, maxId);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, index.getCurrentMemoryComponent().getId());
         if (!index.getDiskComponents().isEmpty()) {
             ILSMDiskComponent diskComponent = index.getDiskComponents().get(0);
             ILSMComponentId maxDiskComponentId = diskComponent.getId();
@@ -837,7 +841,17 @@
             }
         }
         index.getCurrentMemoryComponent().resetId(id, true);
-        accessor.scheduleFlush(index.getIOOperationCallback());
+        ILSMIOOperation flush = accessor.scheduleFlush();
+        try {
+            flush.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(flush.getFailure());
+        }
+        index.resetCurrentComponentIndex();
     }
 
     private class JobEntityCommits {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
index 5c5ce93..ad70cf4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
@@ -36,9 +36,11 @@
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.client.NodeStatus;
 import org.apache.hyracks.api.config.IApplicationConfig;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.storage.common.LocalResource;
 import org.apache.hyracks.util.annotations.ThreadSafe;
 import org.apache.logging.log4j.LogManager;
@@ -66,6 +68,13 @@
 
     @Override
     public synchronized void addReplica(ReplicaIdentifier id) {
+        final NodeControllerService controllerService =
+                (NodeControllerService) appCtx.getServiceContext().getControllerService();
+        final NodeStatus nodeStatus = controllerService.getNodeStatus();
+        if (nodeStatus != NodeStatus.ACTIVE) {
+            LOGGER.warn("Ignoring request to add replica. Node is not ACTIVE yet. Current status: {}", nodeStatus);
+            return;
+        }
         if (!partitions.contains(id.getPartition())) {
             throw new IllegalStateException(
                     "This node is not the current master of partition(" + id.getPartition() + ")");
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index 8158096..79c87c0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.common.transactions.ILockManager;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.IRecoveryManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.transaction.management.service.locking.ConcurrentLockManager;
@@ -59,7 +60,7 @@
     private long profilerEntityCommitLogCount = 0;
     private EntityCommitProfiler ecp;
 
-    public TransactionSubsystem(INcApplicationContext appCtx) {
+    public TransactionSubsystem(INcApplicationContext appCtx, IRecoveryManagerFactory recoveryManagerFactory) {
         this.appCtx = appCtx;
         this.id = appCtx.getServiceContext().getNodeId();
         this.txnProperties = appCtx.getTransactionProperties();
@@ -78,7 +79,7 @@
         }
 
         this.logManager = replicationEnabled ? new LogManagerWithReplication(this) : new LogManager(this);
-        this.recoveryManager = new RecoveryManager(this, appCtx.getServiceContext());
+        this.recoveryManager = recoveryManagerFactory.createRecoveryManager(appCtx.getServiceContext(), this);
         if (txnProperties.isCommitProfilerEnabled()) {
             ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
             ((ExecutorService) appCtx.getThreadExecutor()).submit(ecp);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/ReportLocalCountersTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/ReportLocalCountersTask.java
deleted file mode 100644
index 53b13e8..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/ReportLocalCountersTask.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.asterix.app.nc.task;
-
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.runtime.message.ReportLocalCountersMessage;
-import org.apache.hyracks.api.control.CcId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.service.IControllerService;
-import org.apache.hyracks.control.nc.NodeControllerService;
-
-public class ReportLocalCountersTask implements INCLifecycleTask {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void perform(CcId ccId, IControllerService cs) throws HyracksDataException {
-        ReportLocalCountersMessage.send(ccId, (NodeControllerService) cs);
-    }
-
-    @Override
-    public String toString() {
-        return "{ \"class\" : \"" + getClass().getSimpleName() + "\" }";
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
index 043e02e..961593f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
@@ -31,7 +31,6 @@
 import org.apache.asterix.app.nc.task.ExternalLibrarySetupTask;
 import org.apache.asterix.app.nc.task.LocalRecoveryTask;
 import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
-import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
 import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
 import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
 import org.apache.asterix.app.nc.task.UpdateNodeStatusTask;
@@ -79,7 +78,7 @@
     @Override
     public void notifyNodeFailure(String nodeId) throws HyracksDataException {
         pendingStartupCompletionNodes.remove(nodeId);
-        clusterManager.updateNodePartitions(nodeId, false);
+        clusterManager.updateNodeState(nodeId, false, null);
         if (nodeId.equals(metadataNodeId)) {
             clusterManager.updateMetadataNode(metadataNodeId, false);
         }
@@ -112,20 +111,18 @@
     private void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
         final String nodeId = msg.getNodeId();
         List<INCLifecycleTask> tasks = buildNCRegTasks(msg.getNodeId(), msg.getNodeStatus(), msg.getState());
-        if (!tasks.isEmpty()) {
-            RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
-            try {
-                messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
+        RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
+        try {
+            messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
         }
     }
 
     private void process(NCLifecycleTaskReportMessage msg) throws HyracksDataException {
         pendingStartupCompletionNodes.remove(msg.getNodeId());
         if (msg.isSuccess()) {
-            clusterManager.updateNodePartitions(msg.getNodeId(), true);
+            clusterManager.updateNodeState(msg.getNodeId(), true, msg.getLocalCounters());
             if (msg.getNodeId().equals(metadataNodeId)) {
                 clusterManager.updateMetadataNode(metadataNodeId, true);
             }
@@ -156,7 +153,6 @@
         if (metadataNode) {
             tasks.add(new BindMetadataNodeTask());
         }
-        tasks.add(new ReportLocalCountersTask());
         return tasks;
     }
 
@@ -203,7 +199,6 @@
             tasks.add(new ExportMetadataNodeTask(true));
             tasks.add(new BindMetadataNodeTask());
         }
-        tasks.add(new ReportLocalCountersTask());
         tasks.add(new UpdateNodeStatusTask(NodeStatus.ACTIVE));
         return tasks;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
index 815d878..846aa31 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeResponseMessage.java
@@ -21,7 +21,6 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class MetadataNodeResponseMessage implements INCLifecycleMessage, ICcAddressedMessage {
@@ -41,7 +40,7 @@
 
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((CcApplicationContext) appCtx).getNcLifecycleCoordinator().process(this);
+        appCtx.getNcLifecycleCoordinator().process(this);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
index 5ac3a0c..21022ed 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.app.replication.message;
 
+import org.apache.asterix.common.utils.NcLocalCounters;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -29,14 +30,16 @@
     private final String nodeId;
     private final boolean success;
     private Throwable exception;
+    private final NcLocalCounters localCounters;
 
-    public NCLifecycleTaskReportMessage(String nodeId, boolean success) {
+    public NCLifecycleTaskReportMessage(String nodeId, boolean success, NcLocalCounters localCounters) {
         this.nodeId = nodeId;
         this.success = success;
+        this.localCounters = localCounters;
     }
 
     @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
+    public void handle(ICcApplicationContext appCtx) throws HyracksDataException {
         appCtx.getNcLifecycleCoordinator().process(this);
     }
 
@@ -56,6 +59,10 @@
         this.exception = exception;
     }
 
+    public NcLocalCounters getLocalCounters() {
+        return localCounters;
+    }
+
     @Override
     public MessageType getType() {
         return MessageType.REGISTRATION_TASKS_RESULT;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
index 868c2ad..2d2fe97 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
@@ -20,6 +20,7 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.utils.NcLocalCounters;
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.messaging.CcIdentifiedMessage;
@@ -28,6 +29,7 @@
 import org.apache.asterix.common.replication.INCLifecycleMessage;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -68,7 +70,9 @@
                 success = false;
                 exception = e;
             }
-            NCLifecycleTaskReportMessage result = new NCLifecycleTaskReportMessage(nodeId, success);
+            NcLocalCounters localCounter = success ? NcLocalCounters.collect(getCcId(),
+                    (NodeControllerService) appCtx.getServiceContext().getControllerService()) : null;
+            NCLifecycleTaskReportMessage result = new NCLifecycleTaskReportMessage(nodeId, success, localCounter);
             result.setException(exception);
             try {
                 broker.sendMessageToCC(getCcId(), result);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
index bbb9b99..a17fa90 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.app.result;
 
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.job.JobId;
 
 public class ResultHandle {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
index a6d2da9..2602917 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultPrinter.java
@@ -192,8 +192,6 @@
 
     public void print(String record) throws HyracksDataException {
         printPrefix();
-        // TODO(tillw) evil hack
-        quoteRecord = true;
         displayRecord(record);
         printPostfix();
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index eeb01ba..908e3ca 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -20,24 +20,24 @@
 
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.IHyracksDatasetReader;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
 
 public class ResultReader {
-    private IHyracksDatasetReader reader;
+    private IResultSetReader reader;
 
     private IFrameTupleAccessor frameTupleAccessor;
 
     // Number of parallel result reader buffers
     public static final int NUM_READERS = 1;
 
-    public ResultReader(IHyracksDataset hdc, JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
-        reader = hdc.createReader(jobId, resultSetId);
+    public ResultReader(IResultSet resultSet, JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+        reader = resultSet.createReader(jobId, resultSetId);
         frameTupleAccessor = new ResultFrameTupleAccessor();
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 3fd05e2..ed4c361 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -65,6 +65,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
@@ -115,7 +116,6 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -145,12 +145,14 @@
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
+import org.apache.asterix.translator.ClientJobRequest;
 import org.apache.asterix.translator.CompiledStatements.CompiledDeleteStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
@@ -184,15 +186,16 @@
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.UnmanagedFileSplit;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
@@ -220,6 +223,7 @@
     protected final SessionConfig sessionConfig;
     protected Dataverse activeDataverse;
     protected final List<FunctionDecl> declaredFunctions;
+    protected final ILangCompilationProvider compilationProvider;
     protected final APIFramework apiFramework;
     protected final IRewriterFactory rewriterFactory;
     protected final ExecutorService executorService;
@@ -227,15 +231,16 @@
     protected final IMetadataLockManager lockManager;
 
     public QueryTranslator(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compliationProvider, ExecutorService executorService) {
+            ILangCompilationProvider compilationProvider, ExecutorService executorService) {
         this.appCtx = appCtx;
         this.lockManager = appCtx.getMetadataLockManager();
         this.statements = statements;
         this.sessionOutput = output;
         this.sessionConfig = output.config();
+        this.compilationProvider = compilationProvider;
         declaredFunctions = getDeclaredFunctions(statements);
-        apiFramework = new APIFramework(compliationProvider);
-        rewriterFactory = compliationProvider.getRewriterFactory();
+        apiFramework = new APIFramework(compilationProvider);
+        rewriterFactory = compilationProvider.getRewriterFactory();
         activeDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
         this.executorService = executorService;
         if (appCtx.getServiceContext().getAppConfig().getBoolean(CCConfig.Option.ENFORCE_FRAME_WRITER_PROTOCOL)) {
@@ -260,6 +265,9 @@
     @Override
     public void compileAndExecute(IHyracksClientConnection hcc, IStatementExecutorContext ctx,
             IRequestParameters requestParameters) throws Exception {
+        if (!requestParameters.isMultiStatement()) {
+            validateStatements(statements);
+        }
         int resultSetIdCounter = 0;
         FileSplit outputFile = null;
         IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
@@ -273,19 +281,21 @@
         String threadName = Thread.currentThread().getName();
         Thread.currentThread().setName(QueryTranslator.class.getSimpleName());
         Map<String, String> config = new HashMap<>();
-        final IHyracksDataset hdc = requestParameters.getHyracksDataset();
+        final IResultSet resultSet = requestParameters.getResultSet();
         final ResultDelivery resultDelivery = requestParameters.getResultProperties().getDelivery();
         final long maxResultReads = requestParameters.getResultProperties().getMaxReads();
         final Stats stats = requestParameters.getStats();
         final ResultMetadata outMetadata = requestParameters.getOutMetadata();
         final String clientContextId = requestParameters.getClientContextId();
+        final Map<String, IAObject> stmtParams = requestParameters.getStatementParameters();
         try {
             for (Statement stmt : statements) {
                 if (sessionConfig.is(SessionConfig.FORMAT_HTML)) {
                     sessionOutput.out().println(ApiServlet.HTML_STATEMENT_SEPARATOR);
                 }
                 validateOperation(appCtx, activeDataverse, stmt);
-                rewriteStatement(stmt); // Rewrite the statement's AST.
+                IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
+                rewriteStatement(stmt, stmtRewriter); // Rewrite the statement's AST.
                 MetadataProvider metadataProvider = new MetadataProvider(appCtx, activeDataverse);
                 metadataProvider.getConfig().putAll(config);
                 metadataProvider.setWriterFactory(writerFactory);
@@ -299,7 +309,7 @@
                         activeDataverse = handleUseDataverseStatement(metadataProvider, stmt);
                         break;
                     case CREATE_DATAVERSE:
-                        handleCreateDataverseStatement(metadataProvider, stmt);
+                        handleCreateDataverseStatement(metadataProvider, stmt, requestParameters);
                         break;
                     case DATASET_DECL:
                         handleCreateDatasetStatement(metadataProvider, stmt, hcc, requestParameters);
@@ -314,7 +324,7 @@
                         handleCreateNodeGroupStatement(metadataProvider, stmt);
                         break;
                     case DATAVERSE_DROP:
-                        handleDataverseDropStatement(metadataProvider, stmt, hcc);
+                        handleDataverseDropStatement(metadataProvider, stmt, hcc, requestParameters);
                         break;
                     case DATASET_DROP:
                         handleDatasetDropStatement(metadataProvider, stmt, hcc, requestParameters);
@@ -345,11 +355,11 @@
                                     || resultDelivery == ResultDelivery.DEFERRED);
                             metadataProvider.setMaxResultReads(maxResultReads);
                         }
-                        handleInsertUpsertStatement(metadataProvider, stmt, hcc, hdc, resultDelivery, outMetadata,
-                                stats, false, clientContextId);
+                        handleInsertUpsertStatement(metadataProvider, stmt, hcc, resultSet, resultDelivery, outMetadata,
+                                stats, false, clientContextId, stmtParams, stmtRewriter);
                         break;
                     case DELETE:
-                        handleDeleteStatement(metadataProvider, stmt, hcc, false);
+                        handleDeleteStatement(metadataProvider, stmt, hcc, false, stmtParams, stmtRewriter);
                         break;
                     case CREATE_FEED:
                         handleCreateFeedStatement(metadataProvider, stmt);
@@ -380,8 +390,8 @@
                         metadataProvider.setResultAsyncMode(
                                 resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
                         metadataProvider.setMaxResultReads(maxResultReads);
-                        handleQuery(metadataProvider, (Query) stmt, hcc, hdc, resultDelivery, outMetadata, stats,
-                                clientContextId, ctx);
+                        handleQuery(metadataProvider, (Query) stmt, hcc, resultSet, resultDelivery, outMetadata, stats,
+                                clientContextId, ctx, stmtParams, stmtRewriter);
                         break;
                     case COMPACT:
                         handleCompactStatement(metadataProvider, stmt, hcc);
@@ -399,10 +409,11 @@
                         break;
                     case EXTENSION:
                         ((ExtensionStatement) stmt).handle(hcc, this, requestParameters, metadataProvider,
-                                resultSetIdCounter);
+                                resultSetIdCounter, ctx);
                         break;
                     default:
-                        throw new CompilationException("Unknown function");
+                        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
+                                "Unexpected statement: " + stmt.getKind());
                 }
             }
         } finally {
@@ -410,10 +421,13 @@
         }
     }
 
-    protected void handleSetStatement(Statement stmt, Map<String, String> config) {
+    protected void handleSetStatement(Statement stmt, Map<String, String> config) throws CompilationException {
         SetStatement ss = (SetStatement) stmt;
         String pname = ss.getPropName();
         String pvalue = ss.getPropValue();
+        if (pname.startsWith(APIFramework.PREFIX_INTERNAL_PARAMETERS)) {
+            throw new CompilationException(ErrorCode.ILLEGAL_SET_PARAMETER, pname);
+        }
         config.put(pname, pvalue);
     }
 
@@ -432,6 +446,7 @@
     protected Dataverse handleUseDataverseStatement(MetadataProvider metadataProvider, Statement stmt)
             throws Exception {
         DataverseDecl dvd = (DataverseDecl) stmt;
+        SourceLocation sourceLoc = dvd.getSourceLocation();
         String dvName = dvd.getDataverseName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -439,39 +454,27 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
             if (dv == null) {
-                throw new MetadataException("Unknown dataverse " + dvName);
+                throw new MetadataException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dvName);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             return dv;
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
-            throw new MetadataException(e);
+            throw new MetadataException(ErrorCode.METADATA_ERROR, e, sourceLoc, e.toString());
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
-    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
-
+    protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
+            IRequestParameters requestParameters) throws Exception {
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         String dvName = stmtCreateDataverse.getDataverseName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-
         lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
         try {
-            Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
-            if (dv != null) {
-                if (stmtCreateDataverse.getIfNotExists()) {
-                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                    return;
-                } else {
-                    throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
-                }
-            }
-            MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(),
-                    new Dataverse(dvName, stmtCreateDataverse.getFormat(), MetadataUtil.PENDING_NO_OP));
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            doCreateDataverseStatement(mdTxnCtx, metadataProvider, stmtCreateDataverse);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
             throw e;
@@ -480,33 +483,58 @@
         }
     }
 
+    @SuppressWarnings("squid:S00112")
+    protected boolean doCreateDataverseStatement(MetadataTransactionContext mdTxnCtx, MetadataProvider metadataProvider,
+            CreateDataverseStatement stmtCreateDataverse) throws Exception {
+        String dvName = stmtCreateDataverse.getDataverseName().getValue();
+        Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
+        if (dv != null) {
+            if (stmtCreateDataverse.getIfNotExists()) {
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return false;
+            } else {
+                throw new CompilationException(ErrorCode.DATAVERSE_EXISTS, stmtCreateDataverse.getSourceLocation(),
+                        dvName);
+            }
+        }
+        MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(),
+                new Dataverse(dvName, stmtCreateDataverse.getFormat(), MetadataUtil.PENDING_NO_OP));
+        MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        return true;
+    }
+
     protected static void validateCompactionPolicy(String compactionPolicy,
             Map<String, String> compactionPolicyProperties, MetadataTransactionContext mdTxnCtx,
-            boolean isExternalDataset) throws CompilationException, Exception {
+            boolean isExternalDataset, SourceLocation sourceLoc) throws CompilationException, Exception {
         CompactionPolicy compactionPolicyEntity = MetadataManager.INSTANCE.getCompactionPolicy(mdTxnCtx,
                 MetadataConstants.METADATA_DATAVERSE_NAME, compactionPolicy);
         if (compactionPolicyEntity == null) {
-            throw new CompilationException("Unknown compaction policy: " + compactionPolicy);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Unknown compaction policy: " + compactionPolicy);
         }
         String compactionPolicyFactoryClassName = compactionPolicyEntity.getClassName();
         ILSMMergePolicyFactory mergePolicyFactory =
                 (ILSMMergePolicyFactory) Class.forName(compactionPolicyFactoryClassName).newInstance();
         if (isExternalDataset && mergePolicyFactory.getName().compareTo("correlated-prefix") == 0) {
-            throw new CompilationException("The correlated-prefix merge policy cannot be used with external dataset.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "The correlated-prefix merge policy cannot be used with external dataset.");
         }
         if (compactionPolicyProperties == null) {
             if (mergePolicyFactory.getName().compareTo("no-merge") != 0) {
-                throw new CompilationException("Compaction policy properties are missing.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Compaction policy properties are missing.");
             }
         } else {
             for (Map.Entry<String, String> entry : compactionPolicyProperties.entrySet()) {
                 if (!mergePolicyFactory.getPropertiesNames().contains(entry.getKey())) {
-                    throw new CompilationException("Invalid compaction policy property: " + entry.getKey());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Invalid compaction policy property: " + entry.getKey());
                 }
             }
             for (String p : mergePolicyFactory.getPropertiesNames()) {
                 if (!compactionPolicyProperties.containsKey(p)) {
-                    throw new CompilationException("Missing compaction policy property: " + p);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Missing compaction policy property: " + p);
                 }
             }
         }
@@ -516,6 +544,7 @@
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws CompilationException, Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
         DatasetDecl dd = (DatasetDecl) stmt;
+        SourceLocation sourceLoc = dd.getSourceLocation();
         String dataverseName = getActiveDataverse(dd.getDataverse());
         String datasetName = dd.getName().getValue();
         DatasetType dsType = dd.getDatasetType();
@@ -544,28 +573,30 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A dataset with this name " + datasetName + " already exists.");
+                    throw new CompilationException(ErrorCode.DATASET_EXISTS, sourceLoc, datasetName, dataverseName);
                 }
             }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     itemTypeDataverseName, itemTypeName);
             if (dt == null) {
-                throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
+                throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, itemTypeName);
             }
             String ngName = ngNameId != null ? ngNameId.getValue()
-                    : configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider);
+                    : configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider,
+                            sourceLoc);
 
             if (compactionPolicy == null) {
                 compactionPolicy = GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME;
                 compactionPolicyProperties = GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES;
             } else {
-                validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false);
+                validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false, sourceLoc);
             }
             switch (dd.getDatasetType()) {
                 case INTERNAL:
                     IAType itemType = dt.getDatatype();
                     if (itemType.getTypeTag() != ATypeTag.OBJECT) {
-                        throw new AlgebricksException("Dataset type has to be a record type.");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Dataset type has to be a record type.");
                     }
 
                     IAType metaItemType = null;
@@ -573,7 +604,8 @@
                         metaItemType = metadataProvider.findType(metaItemTypeDataverseName, metaItemTypeName);
                     }
                     if (metaItemType != null && metaItemType.getTypeTag() != ATypeTag.OBJECT) {
-                        throw new AlgebricksException("Dataset meta type has to be a record type.");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Dataset meta type has to be a record type.");
                     }
                     ARecordType metaRecType = (ARecordType) metaItemType;
 
@@ -584,11 +616,11 @@
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
-                            metaRecType, partitioningExprs, keySourceIndicators, autogenerated);
+                            metaRecType, partitioningExprs, keySourceIndicators, autogenerated, sourceLoc);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
-                        ValidateUtil.validateFilterField(aRecordType, filterField);
+                        ValidateUtil.validateFilterField(aRecordType, filterField, sourceLoc);
                     }
                     if (compactionPolicy == null && filterField != null) {
                         // If the dataset has a filter and the user didn't specify a merge
@@ -609,7 +641,8 @@
                             new ExternalDatasetDetails(adapter, properties, new Date(), TransactionState.COMMIT);
                     break;
                 default:
-                    throw new CompilationException("Unknown datatype " + dd.getDatasetType());
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unknown dataset type " + dd.getDatasetType());
             }
 
             // #. initialize DatasetIdFactory if it is not initialized.
@@ -694,8 +727,8 @@
         }
     }
 
-    protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset)
-            throws CompilationException {
+    protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset,
+            SourceLocation sourceLoc) throws CompilationException {
         StringBuilder builder = null;
         ActiveNotificationHandler activeEventHandler =
                 (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -709,13 +742,15 @@
             }
         }
         if (builder != null) {
-            throw new CompilationException("Dataset " + dataset.getDataverseName() + "." + dataset.getDatasetName()
-                    + " is currently being " + "fed into by the following active entities.\n" + builder.toString());
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Dataset " + dataset.getDataverseName() + "." + dataset.getDatasetName() + " is currently being "
+                            + "fed into by the following active entities.\n" + builder.toString());
         }
     }
 
     protected static String configureNodegroupForDataset(ICcApplicationContext appCtx, Map<String, String> hints,
-            String dataverseName, String datasetName, MetadataProvider metadataProvider) throws Exception {
+            String dataverseName, String datasetName, MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws Exception {
         IClusterStateManager csm = appCtx.getClusterStateManager();
         Set<String> allNodes = csm.getParticipantNodes(true);
         Set<String> selectedNodes = new LinkedHashSet<>();
@@ -728,7 +763,7 @@
                     hints.get(DatasetNodegroupCardinalityHint.NAME));
             boolean valid = validation.first;
             if (!valid) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Incorrect use of hint '" + DatasetNodegroupCardinalityHint.NAME + "': " + validation.second);
             } else {
                 nodegroupCardinality = Integer.parseInt(hints.get(DatasetNodegroupCardinalityHint.NAME));
@@ -744,6 +779,7 @@
     public void handleCreateIndexStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+        SourceLocation sourceLoc = stmtCreateIndex.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
@@ -751,6 +787,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         String datasetFullyQualifiedName = dataverseName + "." + datasetName;
+        boolean isSecondaryPrimary = stmtCreateIndex.getFieldExprs().isEmpty();
         Dataset ds = null;
         Index index = null;
         MetadataLockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -758,8 +795,8 @@
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
 
             index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
@@ -769,13 +806,22 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("An index with this name " + indexName + " already exists.");
+                    throw new CompilationException(ErrorCode.INDEX_EXISTS, sourceLoc, indexName);
                 }
             }
-            // can't create secondary primary index on an external dataset
-            if (ds.getDatasetType() == DatasetType.EXTERNAL && stmtCreateIndex.getFieldExprs().isEmpty()) {
+            // find keySourceIndicators for secondary primary index since the parser isn't aware of them
+            if (isSecondaryPrimary && ds.getDatasetType() == DatasetType.INTERNAL) {
+                keySourceIndicators = ((InternalDatasetDetails) ds.getDatasetDetails()).getKeySourceIndicator();
+            }
+            // disable creating secondary primary index on an external dataset
+            if (isSecondaryPrimary && ds.getDatasetType() == DatasetType.EXTERNAL) {
                 throw new AsterixException(ErrorCode.CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET);
             }
+            // disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
+            if (keySourceIndicators.stream().anyMatch(fieldSource -> fieldSource == 1) && !isSecondaryPrimary) {
+                throw new AsterixException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Cannot create index on meta fields");
+            }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
                     ds.getItemTypeDataverseName(), ds.getItemTypeName());
             ARecordType aRecordType = (ARecordType) dt.getDatatype();
@@ -816,11 +862,11 @@
                     fieldType = subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size()));
                 } else {
                     if (!stmtCreateIndex.isEnforced() && stmtCreateIndex.getIndexType() != IndexType.BTREE) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc,
                                 stmtCreateIndex.getIndexType());
                     }
                     if (stmtCreateIndex.isEnforced() && !fieldExpr.second.isUnknownable()) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL, sourceLoc,
                                 String.valueOf(fieldExpr.first));
                     }
                     // don't allow creating an enforced index on a closed-type field, fields that
@@ -828,15 +874,16 @@
                     // get the field type, if it's not null, then the field is closed-type
                     if (stmtCreateIndex.isEnforced()
                             && subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
-                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
+                        throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD, sourceLoc,
                                 String.valueOf(fieldExpr.first));
                     }
                     if (!isOpen) {
-                        throw new AlgebricksException("Typed index on \"" + fieldExpr.first
-                                + "\" field could be created only for open datatype");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Typed index on \""
+                                + fieldExpr.first + "\" field could be created only for open datatype");
                     }
                     if (stmtCreateIndex.hasMetaField()) {
-                        throw new AlgebricksException("Typed open index can only be created on the record part");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Typed open index can only be created on the record part");
                     }
                     Map<TypeSignature, IAType> typeMap =
                             TypeTranslator.computeTypes(mdTxnCtx, fieldExpr.second.getType(), indexName, dataverseName);
@@ -845,15 +892,15 @@
                     overridesFieldTypes = true;
                 }
                 if (fieldType == null) {
-                    throw new AlgebricksException(
-                            "Unknown type " + (fieldExpr.second == null ? fieldExpr.first : fieldExpr.second));
+                    throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, fieldExpr.second == null
+                            ? String.valueOf(fieldExpr.first) : String.valueOf(fieldExpr.second));
                 }
 
                 // try to add the key & its source to the set of keys, if key couldn't be added,
                 // there is a duplicate
                 if (!indexKeysSet
                         .add(new Pair<>(fieldExpr.first, stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
-                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
+                    throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD, sourceLoc,
                             String.valueOf(fieldExpr.first));
                 }
 
@@ -881,9 +928,10 @@
 
                     // If it is not a fixed length
                     if (typeTrait.getFixedLength() < 0) {
-                        throw new AlgebricksException("The keyword or ngram index -" + indexName
-                                + " cannot be created on the dataset -" + datasetName
-                                + " due to its variable-length primary key field - " + partitioningKey);
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "The keyword or ngram index -" + indexName + " cannot be created on the dataset -"
+                                        + datasetName + " due to its variable-length primary key field - "
+                                        + partitioningKey);
                     }
 
                 }
@@ -892,14 +940,14 @@
             Index newIndex = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
                     indexFields, keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(),
                     overridesFieldTypes, stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
-            doCreateIndex(hcc, metadataProvider, ds, newIndex, jobFlags);
+            doCreateIndex(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     public static void doCreateIndex(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Dataset ds,
-            Index index, EnumSet<JobFlag> jobFlags) throws Exception {
+            Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc) throws Exception {
         ProgressState progress = ProgressState.NO_PROGRESS;
         boolean bActiveTxn = true;
         Index filesIndex = null;
@@ -912,18 +960,19 @@
         try {
             index.setPendingOp(MetadataUtil.PENDING_ADD_OP);
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
-                validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), ds);
+                validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), ds, sourceLoc);
             } else {
                 // External dataset
                 // Check if the dataset is indexible
                 if (!ExternalIndexingOperations.isIndexible((ExternalDatasetDetails) ds.getDatasetDetails())) {
-                    throw new AlgebricksException(
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                             "dataset using " + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter()
                                     + " Adapter can't be indexed");
                 }
                 // Check if the name of the index is valid
                 if (!ExternalIndexingOperations.isValidIndexName(index.getDatasetName(), index.getIndexName())) {
-                    throw new AlgebricksException("external dataset index name is invalid");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "external dataset index name is invalid");
                 }
 
                 // Check if the files index exist
@@ -963,7 +1012,7 @@
                     spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot,
                             metadataProvider);
                     if (spec == null) {
-                        throw new CompilationException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                                 "Failed to create job spec for replicating Files Index For external dataset");
                     }
                     filesIndexReplicated = true;
@@ -979,20 +1028,21 @@
                     if (existingIndex.getKeyFieldNames().equals(index.getKeyFieldNames())
                             && !existingIndex.getKeyFieldTypes().equals(index.getKeyFieldTypes())
                             && existingIndex.isEnforced()) {
-                        throw new CompilationException("Cannot create index " + index.getIndexName()
-                                + " , enforced index " + existingIndex.getIndexName() + " on field \""
-                                + StringUtils.join(index.getKeyFieldNames(), ',') + "\" is already defined with type \""
-                                + existingIndex.getKeyFieldTypes() + "\"");
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Cannot create index "
+                                + index.getIndexName() + " , enforced index " + existingIndex.getIndexName()
+                                + " on field \"" + StringUtils.join(index.getKeyFieldNames(), ',')
+                                + "\" is already defined with type \"" + existingIndex.getKeyFieldTypes() + "\"");
                     }
                 }
             }
             // #. add a new index with PendingAddOp
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
             // #. prepare to create the index artifact in NC.
-            spec = IndexUtil.buildSecondaryIndexCreationJobSpec(ds, index, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexCreationJobSpec(ds, index, metadataProvider, sourceLoc);
             if (spec == null) {
-                throw new CompilationException("Failed to create job spec for creating index '" + ds.getDatasetName()
-                        + "." + index.getIndexName() + "'");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
+                                + index.getIndexName() + "'");
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1015,7 +1065,7 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
             // #. load data into the index in NC.
-            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider);
+            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
@@ -1074,7 +1124,7 @@
                 bActiveTxn = true;
                 metadataProvider.setMetadataTxnContext(mdTxnCtx);
                 try {
-                    JobSpecification jobSpec = IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds);
+                    JobSpecification jobSpec = IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc);
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     bActiveTxn = false;
                     runJob(hcc, jobSpec, jobFlags);
@@ -1149,6 +1199,7 @@
 
     protected void handleCreateTypeStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         TypeDecl stmtCreateType = (TypeDecl) stmt;
+        SourceLocation sourceLoc = stmtCreateType.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
         String typeName = stmtCreateType.getIdent().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1158,16 +1209,17 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
-                throw new AlgebricksException("Unknown dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
             }
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt != null) {
                 if (!stmtCreateType.getIfNotExists()) {
-                    throw new AlgebricksException("A datatype with this name " + typeName + " already exists.");
+                    throw new CompilationException(ErrorCode.TYPE_EXISTS, sourceLoc, typeName);
                 }
             } else {
                 if (BuiltinTypeMap.getBuiltinType(typeName) != null) {
-                    throw new AlgebricksException("Cannot redefine builtin type " + typeName + ".");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Cannot redefine builtin type " + typeName + ".");
                 } else {
                     Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx,
                             stmtCreateType.getTypeDef(), stmtCreateType.getIdent().getValue(), dataverseName);
@@ -1186,28 +1238,39 @@
     }
 
     protected void handleDataverseDropStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc) throws Exception {
+            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String dataverseName = stmtDelete.getDataverseName().getValue();
         if (dataverseName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
-            throw new HyracksDataException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME + " dataverse can't be dropped");
         }
+        lockManager.acquireDataverseWriteLock(metadataProvider.getLocks(), dataverseName);
+        try {
+            doDropDataverse(stmtDelete, sourceLoc, metadataProvider, hcc);
+        } finally {
+            metadataProvider.getLocks().unlock();
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
+        }
+    }
 
+    protected boolean doDropDataverse(DataverseDropStatement stmtDelete, SourceLocation sourceLoc,
+            MetadataProvider metadataProvider, IHyracksClientConnection hcc) throws Exception {
+        String dataverseName = stmtDelete.getDataverseName().getValue();
         ProgressState progress = ProgressState.NO_PROGRESS;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        lockManager.acquireDataverseWriteLock(metadataProvider.getLocks(), dataverseName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
                 if (stmtDelete.getIfExists()) {
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-                    return;
+                    return false;
                 } else {
-                    throw new AlgebricksException("There is no dataverse with this name " + dataverseName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
                 }
             }
             // # check whether any function in current dataverse is being used by others
@@ -1215,7 +1278,7 @@
                     MetadataManager.INSTANCE.getDataverseFunctions(mdTxnCtx, dataverseName);
             for (Function function : functionsInDataverse) {
                 if (isFunctionUsed(mdTxnCtx, function.getSignature(), dataverseName)) {
-                    throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE,
+                    throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, sourceLoc,
                             function.getDataverseName() + "." + function.getName() + "@" + function.getArity());
                 }
             }
@@ -1236,7 +1299,7 @@
                     mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
                     bActiveTxn = true;
                     metadataProvider.setMetadataTxnContext(mdTxnCtx);
-                    doDropFeed(hcc, metadataProvider, feedListener.getFeed());
+                    doDropFeed(hcc, metadataProvider, feedListener.getFeed(), sourceLoc);
                     MetadataManager.INSTANCE.commitTransaction(metadataProvider.getMetadataTxnContext());
                     bActiveTxn = false;
                 }
@@ -1254,7 +1317,7 @@
                     List<Index> indexes =
                             MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                     for (Index index : indexes) {
-                        jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset));
+                        jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
                     }
                 } else {
                     // External dataset
@@ -1265,8 +1328,8 @@
                             jobsToExecute.add(
                                     ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, dataset));
                         } else {
-                            jobsToExecute
-                                    .add(IndexUtil.buildDropIndexJobSpec(indexes.get(k), metadataProvider, dataset));
+                            jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(k), metadataProvider, dataset,
+                                    sourceLoc));
                         }
                     }
                     ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(dataset);
@@ -1309,6 +1372,7 @@
                 activeDataverse = null;
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            return true;
         } catch (Exception e) {
             if (bActiveTxn) {
                 abort(e, e, mdTxnCtx);
@@ -1342,30 +1406,28 @@
                             + ") couldn't be removed from the metadata", e);
                 }
             }
-
             throw e;
-        } finally {
-            metadataProvider.getLocks().unlock();
-            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
         }
     }
 
     public void handleDatasetDropStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DropDatasetStatement stmtDelete = (DropDatasetStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
         String datasetName = stmtDelete.getDatasetName().getValue();
         MetadataLockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
                 dataverseName + "." + datasetName);
         try {
-            doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true);
+            doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true, sourceLoc);
         } finally {
             metadataProvider.getLocks().unlock();
         }
     }
 
     public static void doDropDataset(String dataverseName, String datasetName, MetadataProvider metadataProvider,
-            boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup) throws Exception {
+            boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup,
+            SourceLocation sourceLoc) throws Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
         MutableObject<MetadataTransactionContext> mdTxnCtx =
                 new MutableObject<>(MetadataManager.INSTANCE.beginTransaction());
@@ -1379,10 +1441,12 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
                     return;
                 } else {
-                    throw new AsterixException(ErrorCode.NO_DATASET_WITH_NAME, dataverseName, datasetName);
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                            dataverseName);
                 }
             }
-            ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup);
+            ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup,
+                    sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
         } catch (Exception e) {
             if (bActiveTxn.booleanValue()) {
@@ -1425,6 +1489,7 @@
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
 
         IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
+        SourceLocation sourceLoc = stmtIndexDrop.getSourceLocation();
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
         String dataverseName = getActiveDataverse(stmtIndexDrop.getDataverseName());
         String indexName = stmtIndexDrop.getIndexName().getValue();
@@ -1440,8 +1505,8 @@
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             ActiveNotificationHandler activeEventHandler =
                     (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -1456,7 +1521,7 @@
                 }
             }
             if (builder != null) {
-                throw new CompilationException("Dataset" + datasetName
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Dataset" + datasetName
                         + " is currently being fed into by the following active entities: " + builder.toString());
             }
 
@@ -1467,12 +1532,12 @@
                         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                         return;
                     } else {
-                        throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                        throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName);
                     }
                 }
-                ensureNonPrimaryIndexDrop(index);
+                ensureNonPrimaryIndexDrop(index, sourceLoc);
                 // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds));
+                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
 
                 // #. mark PendingDropOp on the existing index
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
@@ -1507,14 +1572,15 @@
                         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                         return;
                     } else {
-                        throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                        throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName);
                     }
                 } else if (ExternalIndexingOperations.isFileIndex(index)) {
-                    throw new AlgebricksException("Dropping a dataset's files index is not allowed.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Dropping a dataset's files index is not allowed.");
                 }
-                ensureNonPrimaryIndexDrop(index);
+                ensureNonPrimaryIndexDrop(index, sourceLoc);
                 // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds));
+                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
                 List<Index> datasetIndexes =
                         MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                 if (datasetIndexes.size() == 2) {
@@ -1616,8 +1682,8 @@
     }
 
     protected void handleTypeDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
-
         TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
+        SourceLocation sourceLoc = stmtTypeDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtTypeDrop.getDataverseName());
         String typeName = stmtTypeDrop.getTypeName().getValue();
 
@@ -1629,7 +1695,7 @@
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt == null) {
                 if (!stmtTypeDrop.getIfExists()) {
-                    throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, typeName);
                 }
             } else {
                 MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, dataverseName, typeName);
@@ -1645,6 +1711,7 @@
 
     protected void handleNodegroupDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String nodegroupName = stmtDelete.getNodeGroupName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -1653,7 +1720,7 @@
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
             if (ng == null) {
                 if (!stmtDelete.getIfExists()) {
-                    throw new AlgebricksException("There is no nodegroup with this name " + nodegroupName + ".");
+                    throw new CompilationException(ErrorCode.UNKNOWN_NODEGROUP, sourceLoc, nodegroupName);
                 }
             } else {
                 MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodegroupName, false);
@@ -1670,6 +1737,7 @@
 
     protected void handleCreateFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         FunctionSignature signature = cfs.getFunctionSignature();
         String dataverse = getActiveDataverseName(signature.getNamespace());
         signature.setNamespace(dataverse);
@@ -1681,28 +1749,27 @@
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
             if (dv == null) {
-                throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverse);
             }
 
             //Check whether the function is use-able
             metadataProvider.setDefaultDataverse(dv);
             Query wrappedQuery = new Query(false);
+            wrappedQuery.setSourceLocation(sourceLoc);
             wrappedQuery.setBody(cfs.getFunctionBodyExpression());
             wrappedQuery.setTopLevel(false);
-            List<VarIdentifier> varIds = new ArrayList<>();
+            List<VarIdentifier> paramVars = new ArrayList<>();
             for (String v : cfs.getParamList()) {
-                varIds.add(new VarIdentifier(v));
+                paramVars.add(new VarIdentifier(v));
             }
-            wrappedQuery.setExternalVars(varIds);
-            apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false);
+            apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false,
+                    paramVars);
 
             List<List<List<String>>> dependencies = FunctionUtil.getFunctionDependencies(
                     rewriterFactory.createQueryRewriter(), cfs.getFunctionBodyExpression(), metadataProvider);
 
-            final String language =
-                    rewriterFactory instanceof SqlppRewriterFactory ? Function.LANGUAGE_SQLPP : Function.LANGUAGE_AQL;
             Function function = new Function(signature, cfs.getParamList(), Function.RETURNTYPE_VOID,
-                    cfs.getFunctionBody(), language, FunctionKind.SCALAR.toString(), dependencies);
+                    cfs.getFunctionBody(), getFunctionLanguage(), FunctionKind.SCALAR.toString(), dependencies);
             MetadataManager.INSTANCE.addFunction(mdTxnCtx, function);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1715,11 +1782,22 @@
         }
     }
 
+    private String getFunctionLanguage() {
+        switch (compilationProvider.getLanguage()) {
+            case SQLPP:
+                return Function.LANGUAGE_SQLPP;
+            case AQL:
+                return Function.LANGUAGE_AQL;
+            default:
+                throw new IllegalStateException(String.valueOf(compilationProvider.getLanguage()));
+        }
+    }
+
     protected boolean isFunctionUsed(MetadataTransactionContext ctx, FunctionSignature signature,
             String currentDataverse) throws AlgebricksException {
         List<Dataverse> allDataverses = MetadataManager.INSTANCE.getDataverses(ctx);
         for (Dataverse dataverse : allDataverses) {
-            if (currentDataverse != null && dataverse.getDataverseName().equals(currentDataverse)) {
+            if (dataverse.getDataverseName().equals(currentDataverse)) {
                 continue;
             }
             List<Feed> feeds = MetadataManager.INSTANCE.getFeeds(ctx, dataverse.getDataverseName());
@@ -1738,6 +1816,7 @@
 
     protected void handleFunctionDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+        SourceLocation sourceLoc = stmtDropFunction.getSourceLocation();
         FunctionSignature signature = stmtDropFunction.getFunctionSignature();
         signature.setNamespace(getActiveDataverseName(signature.getNamespace()));
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1746,12 +1825,15 @@
                 signature.getNamespace() + "." + signature.getName());
         try {
             Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
+            // If function == null && stmtDropFunction.getIfExists() == true, commit txn directly.
             if (function == null && !stmtDropFunction.getIfExists()) {
-                throw new AlgebricksException("Unknonw function " + signature);
-            } else if (isFunctionUsed(mdTxnCtx, signature, null)) {
-                throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, signature);
-            } else {
-                MetadataManager.INSTANCE.dropFunction(mdTxnCtx, signature);
+                throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, signature);
+            } else if (function != null) {
+                if (isFunctionUsed(mdTxnCtx, signature, null)) {
+                    throw new MetadataException(ErrorCode.METADATA_DROP_FUCTION_IN_USE, sourceLoc, signature);
+                } else {
+                    MetadataManager.INSTANCE.dropFunction(mdTxnCtx, signature);
+                }
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
@@ -1776,7 +1858,9 @@
             CompiledLoadFromFileStatement cls =
                     new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
                             loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
-            JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls);
+            cls.setSourceLocation(stmt.getSourceLocation());
+            JobSpecification spec =
+                    apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls, null);
             afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1794,8 +1878,9 @@
     }
 
     public JobSpecification handleInsertUpsertStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery,
-            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId) throws Exception {
+            IHyracksClientConnection hcc, IResultSet resultSet, ResultDelivery resultDelivery,
+            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId,
+            Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter) throws Exception {
         InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
         String dataverseName = getActiveDataverse(stmtInsertUpsert.getDataverseName());
         final IMetadataLocker locker = new IMetadataLocker() {
@@ -1816,7 +1901,8 @@
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             try {
                 metadataProvider.setWriteTransaction(true);
-                final JobSpecification jobSpec = rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert);
+                final JobSpecification jobSpec =
+                        rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams, stmtRewriter);
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
                 return jobSpec;
@@ -1837,7 +1923,7 @@
         }
 
         if (stmtInsertUpsert.getReturnExpression() != null) {
-            deliverResult(hcc, hdc, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
+            deliverResult(hcc, resultSet, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
                     clientContextId, NoOpStatementExecutorContext.INSTANCE);
         } else {
             locker.lock();
@@ -1855,7 +1941,8 @@
     }
 
     public JobSpecification handleDeleteStatement(MetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, boolean compileOnly) throws Exception {
+            IHyracksClientConnection hcc, boolean compileOnly, Map<String, IAObject> stmtParams,
+            IStatementRewriter stmtRewriter) throws Exception {
         DeleteStatement stmtDelete = (DeleteStatement) stmt;
         String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1868,7 +1955,9 @@
             CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
                     stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
                     stmtDelete.getQuery());
-            JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs);
+            clfrqs.setSourceLocation(stmt.getSourceLocation());
+            JobSpecification jobSpec =
+                    rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs, stmtParams, stmtRewriter);
             afterCompile();
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1890,26 +1979,31 @@
 
     @Override
     public JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector,
-            MetadataProvider metadataProvider, Query query, ICompiledDmlStatement stmt)
-            throws RemoteException, AlgebricksException, ACIDException {
+            MetadataProvider metadataProvider, Query query, ICompiledDmlStatement stmt,
+            Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter)
+            throws AlgebricksException, ACIDException {
+
+        Map<VarIdentifier, IAObject> externalVars = createExternalVariables(stmtParams, stmtRewriter);
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult =
-                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query, sessionOutput, true);
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+                metadataProvider, query, sessionOutput, true, externalVars.keySet());
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
-                rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionOutput, stmt);
+                rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionOutput, stmt, externalVars);
     }
 
     private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
-            MetadataProvider metadataProvider, InsertStatement insertUpsert)
-            throws RemoteException, AlgebricksException, ACIDException {
+            MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams,
+            IStatementRewriter stmtRewriter) throws AlgebricksException, ACIDException {
+        SourceLocation sourceLoc = insertUpsert.getSourceLocation();
 
-        // Insert/upsert statement rewriting (happens under the same ongoing metadata
-        // transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult =
-                apiFramework.reWriteQuery(declaredFunctions, metadataProvider, insertUpsert, sessionOutput, true);
+        Map<VarIdentifier, IAObject> externalVars = createExternalVariables(stmtParams, stmtRewriter);
+
+        // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+                metadataProvider, insertUpsert, sessionOutput, true, externalVars.keySet());
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         String dataverseName = getActiveDataverse(rewrittenInsertUpsert.getDataverseName());
@@ -1920,23 +2014,27 @@
                 clfrqs = new CompiledInsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
                         rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
                         rewrittenInsertUpsert.getReturnExpression());
+                clfrqs.setSourceLocation(insertUpsert.getSourceLocation());
                 break;
             case UPSERT:
                 clfrqs = new CompiledUpsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
                         rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
                         rewrittenInsertUpsert.getReturnExpression());
+                clfrqs.setSourceLocation(insertUpsert.getSourceLocation());
                 break;
             default:
-                throw new AlgebricksException("Unsupported statement type " + rewrittenInsertUpsert.getKind());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Unsupported statement type " + rewrittenInsertUpsert.getKind());
         }
         // Insert/upsert statement compilation (happens under the same ongoing metadata
         // transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, rewrittenInsertUpsert.getQuery(),
-                rewrittenResult.second, datasetName, sessionOutput, clfrqs);
+                rewrittenResult.second, datasetName, sessionOutput, clfrqs, externalVars);
     }
 
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFeedStatement cfs = (CreateFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1951,7 +2049,8 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A feed with this name " + feedName + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A feed with this name " + feedName + " already exists.");
                 }
             }
             feed = new Feed(dataverseName, feedName, cfs.getConfiguration());
@@ -1973,6 +2072,7 @@
         FeedPolicyEntity newPolicy = null;
         MetadataTransactionContext mdTxnCtx = null;
         CreateFeedPolicyStatement cfps = (CreateFeedPolicyStatement) stmt;
+        SourceLocation sourceLoc = cfps.getSourceLocation();
         dataverse = getActiveDataverse(null);
         policy = cfps.getPolicyName();
         MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
@@ -1987,7 +2087,8 @@
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
-                    throw new AlgebricksException("A policy with this name " + policy + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A policy with this name " + policy + " already exists.");
                 }
             }
             boolean extendingExisting = cfps.getSourcePolicyName() != null;
@@ -1999,7 +2100,8 @@
                     sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
                             MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
                     if (sourceFeedPolicy == null) {
-                        throw new AlgebricksException("Unknown policy " + cfps.getSourcePolicyName());
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                                "Unknown policy " + cfps.getSourcePolicyName());
                     }
                 }
                 Map<String, String> policyProperties = sourceFeedPolicy.getProperties();
@@ -2011,7 +2113,8 @@
                     InputStream stream = new FileInputStream(cfps.getSourcePolicyFile());
                     prop.load(stream);
                 } catch (Exception e) {
-                    throw new AlgebricksException("Unable to read policy file" + cfps.getSourcePolicyFile(), e);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unable to read policy file" + cfps.getSourcePolicyFile(), e);
                 }
                 Map<String, String> policyProperties = new HashMap<>();
                 prop.forEach((key, value) -> policyProperties.put((String) key, (String) value));
@@ -2030,6 +2133,7 @@
     protected void handleDropFeedStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
+        SourceLocation sourceLoc = stmtFeedDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtFeedDrop.getDataverseName());
         String feedName = stmtFeedDrop.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2040,12 +2144,13 @@
             Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
             if (feed == null) {
                 if (!stmtFeedDrop.getIfExists()) {
-                    throw new AlgebricksException("There is no feed with this name " + feedName + ".");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "There is no feed with this name " + feedName + ".");
                 }
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
             }
-            doDropFeed(hcc, metadataProvider, feed);
+            doDropFeed(hcc, metadataProvider, feed, sourceLoc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             abort(e, e, mdTxnCtx);
@@ -2055,8 +2160,8 @@
         }
     }
 
-    protected void doDropFeed(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Feed feed)
-            throws Exception {
+    protected void doDropFeed(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Feed feed,
+            SourceLocation sourceLoc) throws Exception {
         MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
         EntityId feedId = feed.getFeedId();
         ActiveNotificationHandler activeNotificationHandler =
@@ -2064,7 +2169,7 @@
         ActiveEntityEventsListener listener =
                 (ActiveEntityEventsListener) activeNotificationHandler.getListener(feedId);
         if (listener != null && listener.getState() != ActivityState.STOPPED) {
-            throw new AlgebricksException("Feed " + feedId
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Feed " + feedId
                     + " is currently active and connected to the following dataset(s) \n" + listener.toString());
         } else if (listener != null) {
             listener.unregister();
@@ -2082,6 +2187,7 @@
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
+        SourceLocation sourceLoc = stmtFeedPolicyDrop.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtFeedPolicyDrop.getDataverseName());
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
         MetadataLockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -2090,7 +2196,8 @@
             FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName, policyName);
             if (feedPolicy == null) {
                 if (!stmtFeedPolicyDrop.getIfExists()) {
-                    throw new AlgebricksException("Unknown policy " + policyName + " in dataverse " + dataverseName);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Unknown policy " + policyName + " in dataverse " + dataverseName);
                 }
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 return;
@@ -2108,6 +2215,7 @@
     private void handleStartFeedStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         StartFeedStatement sfs = (StartFeedStatement) stmt;
+        SourceLocation sourceLoc = sfs.getSourceLocation();
         String dataverseName = getActiveDataverse(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2124,7 +2232,7 @@
             List<FeedConnection> feedConnections = MetadataManager.INSTANCE
                     .getFeedConections(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
             if (feedConnections.isEmpty()) {
-                throw new CompilationException(ErrorCode.FEED_START_FEED_WITHOUT_CONNECTION, feedName);
+                throw new CompilationException(ErrorCode.FEED_START_FEED_WITHOUT_CONNECTION, sourceLoc, feedName);
             }
             for (FeedConnection feedConnection : feedConnections) {
                 // what if the dataset is in a different dataverse
@@ -2161,6 +2269,7 @@
 
     private void handleStopFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         StopFeedStatement sfst = (StopFeedStatement) stmt;
+        SourceLocation sourceLoc = sfst.getSourceLocation();
         String dataverseName = getActiveDataverse(sfst.getDataverseName());
         String feedName = sfst.getFeedName().getValue();
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
@@ -2169,7 +2278,8 @@
         // Obtain runtime info from ActiveListener
         ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler.getListener(entityId);
         if (listener == null) {
-            throw new AlgebricksException("Feed " + feedName + " is not started.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    "Feed " + feedName + " is not started.");
         }
         MetadataLockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), entityId.getDataverse(),
                 entityId.getEntityName());
@@ -2183,6 +2293,7 @@
     private void handleConnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FeedConnection fc;
         ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String feedName = cfs.getFeedName();
         String datasetName = cfs.getDatasetName().getValue();
@@ -2204,21 +2315,23 @@
                     metadataProvider.getMetadataTxnContext());
             FeedEventsListener listener = (FeedEventsListener) activeEventHandler.getListener(feed.getFeedId());
             if (listener != null && listener.isActive()) {
-                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, sourceLoc,
+                        feedName);
             }
             ARecordType outputType = FeedMetadataUtil.getOutputType(feed,
                     feed.getConfiguration().get(ExternalDataConstants.KEY_TYPE_NAME));
             List<FunctionSignature> appliedFunctions = cfs.getAppliedFunctions();
             for (FunctionSignature func : appliedFunctions) {
                 if (MetadataManager.INSTANCE.getFunction(mdTxnCtx, func) == null) {
-                    throw new CompilationException(ErrorCode.FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION,
+                    throw new CompilationException(ErrorCode.FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION, sourceLoc,
                             func.getName());
                 }
             }
             fc = MetadataManager.INSTANCE.getFeedConnection(metadataProvider.getMetadataTxnContext(), dataverseName,
                     feedName, datasetName);
             if (fc != null) {
-                throw new AlgebricksException("Feed" + feedName + " is already connected dataset " + datasetName);
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Feed" + feedName + " is already connected dataset " + datasetName);
             }
             fc = new FeedConnection(dataverseName, feedName, datasetName, appliedFunctions, policyName, whereClauseBody,
                     outputType.getTypeName());
@@ -2238,6 +2351,7 @@
 
     protected void handleDisconnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
+        SourceLocation sourceLoc = cfs.getSourceLocation();
         String dataverseName = getActiveDataverse(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
         String feedName = cfs.getFeedName().getValue();
@@ -2252,7 +2366,8 @@
             ActiveEntityEventsListener listener = (ActiveEntityEventsListener) activeEventHandler
                     .getListener(new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName));
             if (listener != null && listener.isActive()) {
-                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, feedName);
+                throw new CompilationException(ErrorCode.FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED, sourceLoc,
+                        feedName);
             }
             FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, cfs.getDatasetName().getValue());
             FeedMetadataUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
@@ -2260,11 +2375,12 @@
                     dataverseName, feedName, datasetName);
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new CompilationException("Dataset " + dataverseName + "." + datasetName + " doesn't exist");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             if (fc == null) {
-                throw new CompilationException("Feed " + feedName + " is currently not connected to "
-                        + cfs.getDatasetName().getValue() + ". Invalid operation!");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Feed " + feedName
+                        + " is currently not connected to " + cfs.getDatasetName().getValue() + ". Invalid operation!");
             }
             MetadataManager.INSTANCE.dropFeedConnection(mdTxnCtx, dataverseName, feedName, datasetName);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2282,6 +2398,7 @@
     protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
+        SourceLocation sourceLoc = compactStatement.getSourceLocation();
         String dataverseName = getActiveDataverse(compactStatement.getDataverseName());
         String datasetName = compactStatement.getDatasetName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2293,13 +2410,13 @@
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName + ".");
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             // Prepare jobs to compact the datatset and its indexes
             List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
             if (indexes.isEmpty()) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Cannot compact the extrenal dataset " + datasetName + " because it has no indexes");
             }
             Dataverse dataverse =
@@ -2309,11 +2426,12 @@
             if (ds.getDatasetType() == DatasetType.INTERNAL) {
                 for (Index index : indexes) {
                     if (index.isSecondaryIndex()) {
-                        jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, index, metadataProvider));
+                        jobsToExecute.add(
+                                IndexUtil.buildSecondaryIndexCompactJobSpec(ds, index, metadataProvider, sourceLoc));
                     }
                 }
             } else {
-                prepareCompactJobsForExternalDataset(indexes, ds, jobsToExecute, metadataProvider);
+                prepareCompactJobsForExternalDataset(indexes, ds, jobsToExecute, metadataProvider, sourceLoc);
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -2334,9 +2452,11 @@
     }
 
     protected void prepareCompactJobsForExternalDataset(List<Index> indexes, Dataset ds,
-            List<JobSpecification> jobsToExecute, MetadataProvider metadataProvider) throws AlgebricksException {
+            List<JobSpecification> jobsToExecute, MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws AlgebricksException {
         for (int j = 0; j < indexes.size(); j++) {
-            jobsToExecute.add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), metadataProvider));
+            jobsToExecute
+                    .add(IndexUtil.buildSecondaryIndexCompactJobSpec(ds, indexes.get(j), metadataProvider, sourceLoc));
 
         }
     }
@@ -2356,8 +2476,9 @@
     }
 
     protected void handleQuery(MetadataProvider metadataProvider, Query query, IHyracksClientConnection hcc,
-            IHyracksDataset hdc, ResultDelivery resultDelivery, ResultMetadata outMetadata, Stats stats,
-            String clientContextId, IStatementExecutorContext ctx) throws Exception {
+            IResultSet resultSet, ResultDelivery resultDelivery, ResultMetadata outMetadata, Stats stats,
+            String clientContextId, IStatementExecutorContext ctx, Map<String, IAObject> stmtParams,
+            IStatementRewriter stmtRewriter) throws Exception {
         final IMetadataLocker locker = new IMetadataLocker() {
             @Override
             public void lock() {
@@ -2375,7 +2496,8 @@
             boolean bActiveTxn = true;
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             try {
-                final JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null);
+                final JobSpecification jobSpec =
+                        rewriteCompileQuery(hcc, metadataProvider, query, null, stmtParams, stmtRewriter);
                 afterCompile();
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
@@ -2388,11 +2510,11 @@
                 throw e;
             }
         };
-        deliverResult(hcc, hdc, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats, clientContextId,
-                ctx);
+        deliverResult(hcc, resultSet, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
+                clientContextId, ctx);
     }
 
-    private void deliverResult(IHyracksClientConnection hcc, IHyracksDataset hdc, IStatementCompiler compiler,
+    private void deliverResult(IHyracksClientConnection hcc, IResultSet resultSet, IStatementCompiler compiler,
             MetadataProvider metadataProvider, IMetadataLocker locker, ResultDelivery resultDelivery,
             ResultMetadata outMetadata, Stats stats, String clientContextId, IStatementExecutorContext ctx)
             throws Exception {
@@ -2410,7 +2532,7 @@
                 break;
             case IMMEDIATE:
                 createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
-                    final ResultReader resultReader = new ResultReader(hdc, id, resultSetId);
+                    final ResultReader resultReader = new ResultReader(resultSet, id, resultSetId);
                     updateJobStats(id, stats);
                     // stop buffering and allow for streaming result delivery
                     sessionOutput.release();
@@ -2497,6 +2619,7 @@
     private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
             IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
             String clientContextId, IStatementExecutorContext ctx) throws Exception {
+        ClientJobRequest req = null;
         locker.lock();
         try {
             final JobSpecification jobSpec = compiler.compile();
@@ -2505,7 +2628,8 @@
             }
             final JobId jobId = JobUtils.runJob(hcc, jobSpec, jobFlags, false);
             if (ctx != null && clientContextId != null) {
-                ctx.put(clientContextId, jobId); // Adds the running job into the context.
+                req = new ClientJobRequest(ctx, clientContextId, jobId);
+                ctx.put(clientContextId, req); // Adds the running job into the context.
             }
             if (jId != null) {
                 jId.setValue(jobId);
@@ -2520,14 +2644,15 @@
         } finally {
             locker.unlock();
             // No matter the job succeeds or fails, removes it into the context.
-            if (ctx != null && clientContextId != null) {
-                ctx.removeJobIdFromClientContextId(clientContextId);
+            if (req != null) {
+                req.complete();
             }
         }
     }
 
     protected void handleCreateNodeGroupStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
+        SourceLocation sourceLoc = stmtCreateNodegroup.getSourceLocation();
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2537,7 +2662,8 @@
             NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
             if (ng != null) {
                 if (!stmtCreateNodegroup.getIfNotExists()) {
-                    throw new AlgebricksException("A nodegroup with this name " + ngName + " already exists.");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "A nodegroup with this name " + ngName + " already exists.");
                 }
             } else {
                 List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
@@ -2559,6 +2685,7 @@
     protected void handleExternalDatasetRefreshStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
+        SourceLocation sourceLoc = stmtRefresh.getSourceLocation();
         String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
         TransactionState transactionState = TransactionState.COMMIT;
@@ -2581,19 +2708,19 @@
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             // Dataset exists ?
             if (ds == null) {
-                throw new AlgebricksException(
-                        "There is no dataset with this name " + datasetName + " in dataverse " + dataverseName);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
             }
             // Dataset external ?
             if (ds.getDatasetType() != DatasetType.EXTERNAL) {
-                throw new AlgebricksException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "dataset " + datasetName + " in dataverse " + dataverseName + " is not an external dataset");
             }
             // Dataset has indexes ?
             indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
             if (indexes.isEmpty()) {
-                throw new AlgebricksException("External dataset " + datasetName + " in dataverse " + dataverseName
-                        + " doesn't have any index");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "External dataset " + datasetName
+                        + " in dataverse " + dataverseName + " doesn't have any index");
             }
 
             // Record transaction time
@@ -2653,7 +2780,7 @@
             for (Index index : indexes) {
                 if (!ExternalIndexingOperations.isFileIndex(index)) {
                     spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, addedFiles,
-                            appendedFiles, metadataProvider);
+                            appendedFiles, metadataProvider, sourceLoc);
                     // run the files update job
                     runJob(hcc, spec);
                 }
@@ -2782,7 +2909,7 @@
 
     @Override
     public String getActiveDataverseName(String dataverse) {
-        return (dataverse != null) ? dataverse : activeDataverse.getDataverseName();
+        return (dataverse != null && !dataverse.isEmpty()) ? dataverse : activeDataverse.getDataverseName();
     }
 
     @Override
@@ -2820,16 +2947,15 @@
         }
     }
 
-    protected void rewriteStatement(Statement stmt) throws CompilationException {
-        IStatementRewriter rewriter = rewriterFactory.createStatementRewriter();
+    protected void rewriteStatement(Statement stmt, IStatementRewriter rewriter) throws CompilationException {
         rewriter.rewrite(stmt);
     }
 
-    private void ensureNonPrimaryIndexDrop(Index index) throws AlgebricksException {
+    private void ensureNonPrimaryIndexDrop(Index index, SourceLocation sourceLoc) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
-            throw new MetadataException(ErrorCode.CANNOT_DROP_INDEX, index.getIndexName(), index.getDatasetName());
+            throw new MetadataException(ErrorCode.CANNOT_DROP_INDEX, sourceLoc, index.getIndexName(),
+                    index.getDatasetName());
         }
-
     }
 
     protected void afterCompile() {
@@ -2837,4 +2963,38 @@
             ExecutionPlansHtmlPrintUtil.print(sessionOutput.out(), getExecutionPlans());
         }
     }
+
+    protected void validateStatements(List<Statement> statements) throws RuntimeDataException {
+        if (statements.stream().filter(this::isNotAllowedMultiStatement).count() > 1) {
+            throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MULTIPLE_STATEMENTS);
+        }
+    }
+
+    protected boolean isNotAllowedMultiStatement(Statement statement) {
+        switch (statement.getKind()) {
+            case DATAVERSE_DECL:
+            case FUNCTION_DECL:
+            case SET:
+            case WRITE:
+                return false;
+            default:
+                return true;
+        }
+    }
+
+    private Map<VarIdentifier, IAObject> createExternalVariables(Map<String, IAObject> stmtParams,
+            IStatementRewriter stmtRewriter) {
+        if (stmtParams == null || stmtParams.isEmpty()) {
+            return Collections.emptyMap();
+        }
+        Map<VarIdentifier, IAObject> m = new HashMap<>();
+        for (Map.Entry<String, IAObject> me : stmtParams.entrySet()) {
+            String paramName = me.getKey();
+            String extVarName = stmtRewriter.toExternalVariableName(paramName);
+            if (extVarName != null) {
+                m.put(new VarIdentifier(extVarName), me.getValue());
+            }
+        }
+        return m;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
index 9592492..5f76568 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
@@ -18,37 +18,54 @@
  */
 package org.apache.asterix.app.translator;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.external.library.java.base.ByteArrayAccessibleInputStream;
+import org.apache.asterix.external.parser.JSONDataParser;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.ResultProperties;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class RequestParameters implements IRequestParameters {
 
-    private final IHyracksDataset hdc;
+    private final IResultSet resultSet;
     private final ResultProperties resultProperties;
     private final Stats stats;
     private final Map<String, String> optionalParameters;
     private final IStatementExecutor.ResultMetadata outMetadata;
     private final String clientContextId;
+    private final Map<String, IAObject> statementParameters;
+    private final boolean multiStatement;
 
-    public RequestParameters(IHyracksDataset hdc, ResultProperties resultProperties, Stats stats,
+    public RequestParameters(IResultSet resultSet, ResultProperties resultProperties, Stats stats,
             IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
-            Map<String, String> optionalParameters) {
-        this.hdc = hdc;
+            Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement) {
+        this.resultSet = resultSet;
         this.resultProperties = resultProperties;
         this.stats = stats;
         this.outMetadata = outMetadata;
         this.clientContextId = clientContextId;
         this.optionalParameters = optionalParameters;
+        this.statementParameters = statementParameters;
+        this.multiStatement = multiStatement;
     }
 
     @Override
-    public IHyracksDataset getHyracksDataset() {
-        return hdc;
+    public IResultSet getResultSet() {
+        return resultSet;
     }
 
     @Override
@@ -75,4 +92,55 @@
     public String getClientContextId() {
         return clientContextId;
     }
+
+    @Override
+    public boolean isMultiStatement() {
+        return multiStatement;
+    }
+
+    @Override
+    public Map<String, IAObject> getStatementParameters() {
+        return statementParameters;
+    }
+
+    public static Map<String, byte[]> serializeParameterValues(Map<String, JsonNode> inParams)
+            throws HyracksDataException {
+        if (inParams == null || inParams.isEmpty()) {
+            return null;
+        }
+        JSONDataParser parser = new JSONDataParser(null, null);
+        ByteArrayAccessibleOutputStream buffer = new ByteArrayAccessibleOutputStream();
+        DataOutputStream bufferDataOutput = new DataOutputStream(buffer);
+        Map<String, byte[]> m = new HashMap<>();
+        for (Map.Entry<String, JsonNode> me : inParams.entrySet()) {
+            String name = me.getKey();
+            JsonNode jsonValue = me.getValue();
+            parser.setInputNode(jsonValue);
+            buffer.reset();
+            parser.parseAnyValue(bufferDataOutput);
+            byte[] byteValue = buffer.toByteArray();
+            m.put(name, byteValue);
+        }
+        return m;
+    }
+
+    public static Map<String, IAObject> deserializeParameterValues(Map<String, byte[]> inParams)
+            throws HyracksDataException {
+        if (inParams == null || inParams.isEmpty()) {
+            return null;
+        }
+        Map<String, IAObject> m = new HashMap<>();
+        ByteArrayAccessibleInputStream buffer = new ByteArrayAccessibleInputStream(new byte[0], 0, 0);
+        DataInputStream bufferDataInput = new DataInputStream(buffer);
+        ISerializerDeserializer serDe =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANY);
+        for (Map.Entry<String, byte[]> me : inParams.entrySet()) {
+            String name = me.getKey();
+            byte[] value = me.getValue();
+            buffer.setContent(value, 0, value.length);
+            IAObject iaValue = (IAObject) serDe.deserialize(bufferDataInput);
+            m.put(name, iaValue);
+        }
+        return m;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 2ed30e1..a5fd063 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -36,22 +36,18 @@
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
 import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
-import org.apache.asterix.api.http.server.DdlApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
-import org.apache.asterix.api.http.server.FullApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
-import org.apache.asterix.api.http.server.QueryApiServlet;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
 import org.apache.asterix.api.http.server.QueryResultApiServlet;
 import org.apache.asterix.api.http.server.QueryServiceServlet;
 import org.apache.asterix.api.http.server.QueryStatusApiServlet;
 import org.apache.asterix.api.http.server.RebalanceApiServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.ShutdownApiServlet;
-import org.apache.asterix.api.http.server.UpdateApiServlet;
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
@@ -59,7 +55,9 @@
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.config.ExtensionProperties;
 import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.MetadataProperties;
@@ -88,6 +86,7 @@
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.config.IConfigManager;
+import org.apache.hyracks.api.control.IGatekeeper;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
@@ -96,6 +95,8 @@
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.WebManager;
 import org.apache.hyracks.util.LoggingConfigUtil;
 import org.apache.logging.log4j.Level;
@@ -117,6 +118,7 @@
 
     @Override
     public void init(IServiceContext serviceCtx) throws Exception {
+        super.init(serviceCtx);
         ccServiceCtx = (ICCServiceContext) serviceCtx;
         ccServiceCtx.setThreadFactory(
                 new AsterixThreadFactory(ccServiceCtx.getThreadFactory(), new LifeCycleComponentManager()));
@@ -146,12 +148,13 @@
         INcLifecycleCoordinator lifecycleCoordinator = createNcLifeCycleCoordinator(repProp.isReplicationEnabled());
         ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
         componentProvider = new StorageComponentProvider();
-        GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
+
+        List<AsterixExtension> extensions = new ArrayList<>();
+        extensions.addAll(getExtensions());
+        ccExtensionManager = new CCExtensionManager(extensions);
+        IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
         statementExecutorCtx = new StatementExecutorContext();
         appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator);
-        List<AsterixExtension> extensions = new ArrayList<>();
-        extensions.addAll(this.getExtensions());
-        ccExtensionManager = new CCExtensionManager(extensions);
         appCtx.setExtensionManager(ccExtensionManager);
         final CCConfig ccConfig = controllerService.getCCConfig();
         if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -177,15 +180,15 @@
     }
 
     protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
-            GlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
+            IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
             throws AlgebricksException, IOException {
         return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
                 globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
                 new MetadataLockManager());
     }
 
-    protected GlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
-        return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
+    protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
+        return ccExtensionManager.getGlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
     }
 
     protected INcLifecycleCoordinator createNcLifeCycleCoordinator(boolean replicationEnabled) {
@@ -198,8 +201,8 @@
         LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
     }
 
-    protected List<AsterixExtension> getExtensions() {
-        return appCtx.getExtensionProperties().getExtensions();
+    protected List<AsterixExtension> getExtensions() throws Exception {
+        return new ExtensionProperties(PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig())).getExtensions();
     }
 
     protected void configureServers() throws Exception {
@@ -218,8 +221,10 @@
     }
 
     protected HttpServer setupWebServer(ExternalProperties externalProperties) throws Exception {
+        final HttpServerConfig config =
+                HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
         HttpServer webServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
-                externalProperties.getWebInterfacePort());
+                externalProperties.getWebInterfacePort(), config);
         webServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
         webServer.addServlet(new ApiServlet(webServer.ctx(), new String[] { "/*" }, appCtx,
                 ccExtensionManager.getCompilationProvider(AQL), ccExtensionManager.getCompilationProvider(SQLPP),
@@ -228,8 +233,10 @@
     }
 
     protected HttpServer setupJSONAPIServer(ExternalProperties externalProperties) throws Exception {
-        HttpServer jsonAPIServer =
-                new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties.getAPIServerPort());
+        final HttpServerConfig config =
+                HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
+        HttpServer jsonAPIServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
+                externalProperties.getAPIServerPort(), config);
         jsonAPIServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
         jsonAPIServer.setAttribute(ASTERIX_APP_CONTEXT_INFO_ATTR, appCtx);
         jsonAPIServer.setAttribute(ServletConstants.EXECUTOR_SERVICE_ATTR,
@@ -237,18 +244,6 @@
         jsonAPIServer.setAttribute(ServletConstants.RUNNING_QUERIES_ATTR, statementExecutorCtx);
         jsonAPIServer.setAttribute(ServletConstants.SERVICE_CONTEXT_ATTR, ccServiceCtx);
 
-        // AQL rest APIs.
-        addServlet(jsonAPIServer, Servlets.AQL_QUERY);
-        addServlet(jsonAPIServer, Servlets.AQL_UPDATE);
-        addServlet(jsonAPIServer, Servlets.AQL_DDL);
-        addServlet(jsonAPIServer, Servlets.AQL);
-
-        // SQL+x+ rest APIs.
-        addServlet(jsonAPIServer, Servlets.SQLPP_QUERY);
-        addServlet(jsonAPIServer, Servlets.SQLPP_UPDATE);
-        addServlet(jsonAPIServer, Servlets.SQLPP_DDL);
-        addServlet(jsonAPIServer, Servlets.SQLPP);
-
         // Other APIs.
         addServlet(jsonAPIServer, Servlets.QUERY_STATUS);
         addServlet(jsonAPIServer, Servlets.QUERY_RESULT);
@@ -272,8 +267,10 @@
     }
 
     protected HttpServer setupQueryWebServer(ExternalProperties externalProperties) throws Exception {
+        final HttpServerConfig config =
+                HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
         HttpServer queryWebServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
-                externalProperties.getQueryWebInterfacePort());
+                externalProperties.getQueryWebInterfacePort(), config);
         queryWebServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
         ServiceLoader.load(IQueryWebServerRegistrant.class).iterator()
                 .forEachRemaining(c -> c.register(appCtx, queryWebServer));
@@ -283,32 +280,8 @@
 
     protected IServlet createServlet(ConcurrentMap<String, Object> ctx, String key, String... paths) {
         switch (key) {
-            case Servlets.AQL:
-                return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_QUERY:
-                return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_UPDATE:
-                return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.AQL_DDL:
-                return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP:
-                return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_QUERY:
-                return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_UPDATE:
-                return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
-            case Servlets.SQLPP_DDL:
-                return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
-                        getStatementExecutorFactory(), componentProvider);
             case Servlets.RUNNING_REQUESTS:
-                return new QueryCancellationServlet(ctx, paths);
+                return new CcQueryCancellationServlet(ctx, appCtx, paths);
             case Servlets.QUERY_STATUS:
                 return new QueryStatusApiServlet(ctx, appCtx, paths);
             case Servlets.QUERY_RESULT:
@@ -383,4 +356,8 @@
         ApplicationConfigurator.validateJavaRuntime();
     }
 
+    @Override
+    public IGatekeeper getGatekeeper() {
+        return getConfigManager().getAppConfig().getNCNames()::contains;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 3d9b822..5fc5c57 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -56,7 +56,7 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final IStorageComponentProvider componentProvider;
     protected final ICCServiceContext serviceCtx;
-    protected IHyracksClientConnection hcc;
+    protected final IHyracksClientConnection hcc;
     protected volatile boolean recoveryCompleted;
     protected volatile boolean recovering;
 
@@ -126,7 +126,9 @@
             throws Exception {
         // Loop over datasets
         for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
-            mdTxnCtx = recoverDataset(appCtx, mdTxnCtx, dataverse);
+            mdTxnCtx = recoverDatasets(appCtx, mdTxnCtx, dataverse);
+            // Fixes ASTERIXDB-2386 by caching the dataverse during recovery
+            MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse.getDataverseName());
         }
         return mdTxnCtx;
     }
@@ -138,8 +140,8 @@
         }
     }
 
-    private MetadataTransactionContext recoverDataset(ICcApplicationContext appCtx, MetadataTransactionContext mdTxnCtx,
-            Dataverse dataverse) throws Exception {
+    private MetadataTransactionContext recoverDatasets(ICcApplicationContext appCtx,
+            MetadataTransactionContext mdTxnCtx, Dataverse dataverse) throws Exception {
         if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
             MetadataProvider metadataProvider = new MetadataProvider(appCtx, dataverse);
             try {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 453d9b8..fbafc2e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -26,7 +26,9 @@
 
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.api.http.server.StorageApiServlet;
+import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.nc.RecoveryManager;
 import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
@@ -44,6 +46,7 @@
 import org.apache.asterix.common.transactions.Checkpoint;
 import org.apache.asterix.common.transactions.IRecoveryManager;
 import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.utils.PrintUtil;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.common.utils.StorageConstants;
@@ -65,6 +68,8 @@
 import org.apache.hyracks.control.nc.BaseNCApplication;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.WebManager;
 import org.apache.hyracks.util.LoggingConfigUtil;
 import org.apache.logging.log4j.Level;
@@ -78,6 +83,7 @@
     private INcApplicationContext runtimeContext;
     private String nodeId;
     private boolean stopInitiated;
+    private boolean startupCompleted;
     protected WebManager webManager;
 
     @Override
@@ -94,6 +100,7 @@
         ncServiceCtx.setThreadFactory(
                 new AsterixThreadFactory(ncServiceCtx.getThreadFactory(), ncServiceCtx.getLifeCycleComponentManager()));
         validateEnvironment();
+        configurePersistedResourceRegistry();
     }
 
     @Override
@@ -122,7 +129,7 @@
             }
             updateOnNodeJoin();
         }
-        runtimeContext.initialize(runtimeContext.getNodeProperties().isInitialRun());
+        runtimeContext.initialize(getRecoveryManagerFactory(), runtimeContext.getNodeProperties().isInitialRun());
         MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
         IMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
         this.ncServiceCtx.setMessageBroker(messageBroker);
@@ -145,6 +152,10 @@
         performLocalCleanUp();
     }
 
+    protected IRecoveryManagerFactory getRecoveryManagerFactory() {
+        return RecoveryManager::new;
+    }
+
     @Override
     protected void configureLoggingLevel(Level level) {
         super.configureLoggingLevel(level);
@@ -152,8 +163,11 @@
     }
 
     protected void configureServers() throws Exception {
+        final ExternalProperties externalProperties = getApplicationContext().getExternalProperties();
+        final HttpServerConfig config =
+                HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
         HttpServer apiServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
-                getApplicationContext().getExternalProperties().getNcApiPort());
+                externalProperties.getNcApiPort(), config);
         apiServer.setAttribute(ServletConstants.SERVICE_CONTEXT_ATTR, ncServiceCtx);
         apiServer.addServlet(new StorageApiServlet(apiServer.ctx(), getApplicationContext(), Servlets.STORAGE));
         webManager.add(apiServer);
@@ -198,14 +212,19 @@
     }
 
     @Override
-    public void startupCompleted() throws Exception {
+    public synchronized void startupCompleted() throws Exception {
         // configure servlets after joining the cluster, so we can create HyracksClientConnection
         configureServers();
         webManager.start();
+        startupCompleted = true;
+        notifyAll();
     }
 
     @Override
-    public synchronized void onRegisterNode(CcId ccId) throws Exception {
+    public synchronized void tasksCompleted(CcId ccId) throws Exception {
+        while (!startupCompleted) {
+            this.wait();
+        }
         final NodeControllerService ncs = (NodeControllerService) ncServiceCtx.getControllerService();
         final NodeStatus currentStatus = ncs.getNodeStatus();
         final SystemState systemState = isPendingStartupTasks(currentStatus, ncs.getPrimaryCcId(), ccId)
@@ -229,6 +248,7 @@
         final Set<Integer> nodePartitions = runtimeContext.getReplicaManager().getPartitions();
         final PersistentLocalResourceRepository localResourceRepository =
                 (PersistentLocalResourceRepository) runtimeContext.getLocalResourceRepository();
+        localResourceRepository.deleteCorruptedResources();
         for (Integer partition : nodePartitions) {
             localResourceRepository.cleanup(partition);
         }
@@ -279,4 +299,7 @@
         ApplicationConfigurator.validateJavaRuntime();
     }
 
+    protected void configurePersistedResourceRegistry() {
+        ncServiceCtx.setPersistedResourceRegistry(new PersistedResourceRegistry());
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/MessagingChannelWriteInterface.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/MessagingChannelWriteInterface.java
index 43c1542..84f7831 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/MessagingChannelWriteInterface.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/MessagingChannelWriteInterface.java
@@ -46,15 +46,15 @@
             } else {
                 adjustChannelWritability();
             }
-        } else if (ecode >= 0 && !ecodeSent) {
+        } else if (ecode.get() == REMOTE_ERROR_CODE && !ecodeSent) {
             writerState.getCommand().setChannelId(channelId);
             writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.ERROR);
-            writerState.getCommand().setData(ecode);
+            writerState.getCommand().setData(REMOTE_ERROR_CODE);
             writerState.reset(null, 0, null);
             ecodeSent = true;
             ccb.reportLocalEOS();
             adjustChannelWritability();
-        } else if (eos && !eosSent) {
+        } else if (isPendingCloseWrite()) {
             writerState.getCommand().setChannelId(channelId);
             writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL);
             writerState.getCommand().setData(0);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
index c143a63..83ceec7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/MetadataBuiltinFunctions.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.app.function.DatasetResourcesRewriter;
 import org.apache.asterix.app.function.DatasetRewriter;
 import org.apache.asterix.app.function.FeedRewriter;
+import org.apache.asterix.app.function.PingRewriter;
 import org.apache.asterix.app.function.StorageComponentsRewriter;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.utils.RecordUtil;
@@ -48,7 +49,11 @@
         BuiltinFunctions.addUnnestFun(StorageComponentsRewriter.STORAGE_COMPONENTS, false);
         BuiltinFunctions.addDatasourceFunction(StorageComponentsRewriter.STORAGE_COMPONENTS,
                 StorageComponentsRewriter.INSTANCE);
-
+        // Ping function
+        BuiltinFunctions.addPrivateFunction(PingRewriter.PING,
+                (expression, env, mp) -> RecordUtil.FULLY_OPEN_RECORD_TYPE, true);
+        BuiltinFunctions.addUnnestFun(PingRewriter.PING, true);
+        BuiltinFunctions.addDatasourceFunction(PingRewriter.PING, PingRewriter.INSTANCE);
     }
 
     private MetadataBuiltinFunctions() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index 424444a..593d7ce 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -29,10 +29,6 @@
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.asterix.active.ActiveRuntimeId;
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.message.ActiveManagerMessage;
-import org.apache.asterix.active.message.ActiveManagerMessage.Kind;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -42,7 +38,6 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
@@ -52,7 +47,6 @@
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
 import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
-import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.FeedUtils;
@@ -264,7 +258,7 @@
             clfrqs = new CompiledStatements.CompiledUpsertStatement(feedConn.getDataverseName(),
                     feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
         }
-        return statementExecutor.rewriteCompileQuery(hcc, metadataProvider, feedConnQuery, clfrqs);
+        return statementExecutor.rewriteCompileQuery(hcc, metadataProvider, feedConnQuery, clfrqs, null, null);
     }
 
     private static JobSpecification combineIntakeCollectJobs(MetadataProvider metadataProvider, Feed feed,
@@ -485,17 +479,4 @@
         return Pair.of(combineIntakeCollectJobs(metadataProvider, feed, intakeJob, jobsList, feedConnections,
                 ingestionLocations), intakeInfo.getRight().getPartitionConstraint());
     }
-
-    public static void SendStopMessageToNode(ICcApplicationContext appCtx, EntityId feedId, String intakeNodeLocation,
-            Integer partition) throws Exception {
-        ActiveManagerMessage stopFeedMessage = new ActiveManagerMessage(Kind.STOP_ACTIVITY,
-                new ActiveRuntimeId(feedId, FeedIntakeOperatorNodePushable.class.getSimpleName(), partition));
-        SendActiveMessage(appCtx, stopFeedMessage, intakeNodeLocation);
-    }
-
-    private static void SendActiveMessage(ICcApplicationContext appCtx, ActiveManagerMessage activeManagerMessage,
-            String nodeId) throws Exception {
-        ICCMessageBroker messageBroker = (ICCMessageBroker) appCtx.getServiceContext().getMessageBroker();
-        messageBroker.sendApplicationMessageToNC(activeManagerMessage, nodeId);
-    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 702b6b4..483987c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -355,7 +355,7 @@
         List<Index> indexes = metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         for (Index index : indexes) {
             jobs.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset,
-                    EnumSet.of(DropOption.IF_EXISTS, DropOption.WAIT_ON_IN_USE)));
+                    EnumSet.of(DropOption.IF_EXISTS, DropOption.WAIT_ON_IN_USE), null));
         }
         for (JobSpecification jobSpec : jobs) {
             JobUtils.runJob(hcc, jobSpec, true);
@@ -371,20 +371,20 @@
             }
             // Creates the secondary index.
             JobSpecification indexCreationJobSpec =
-                    IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider);
+                    IndexUtil.buildSecondaryIndexCreationJobSpec(target, index, metadataProvider, null);
             JobUtils.runJob(hcc, indexCreationJobSpec, true);
 
             // Loads the secondary index.
             JobSpecification indexLoadingJobSpec =
-                    IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index, metadataProvider);
+                    IndexUtil.buildSecondaryIndexLoadingJobSpec(target, index, metadataProvider, null);
             JobUtils.runJob(hcc, indexLoadingJobSpec, true);
         }
     }
 
     // Gets the primary key permutation for upserts.
     private static int[] getPrimaryKeyPermutationForUpsert(Dataset dataset) {
-        // prev record first
-        int f = 1;
+        // upsertIndicatorVar + prev record
+        int f = 2;
         // add the previous meta second
         if (dataset.hasMetaPart()) {
             f++;
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 0d9f54f..6971b2b 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -46,6 +46,7 @@
 heartbeat.max.misses=25
 
 [common]
+log.dir = logs/
 log.level = INFO
 compiler.framesize=32KB
 compiler.sortmemory=320KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index ddf1438..65dbafc 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -46,6 +46,7 @@
 heartbeat.max.misses=25
 
 [common]
+log.dir = logs/
 log.level = WARN
 compiler.framesize=32KB
 compiler.sortmemory=320KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index b819f24..20aa70d 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -46,6 +46,7 @@
 heartbeat.max.misses=25
 
 [common]
+log.dir = logs/
 log.level = WARN
 compiler.framesize=32KB
 compiler.sortmemory=320KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc4.conf b/asterixdb/asterix-app/src/main/resources/cc4.conf
index 3b7a993..5bdf8ea 100644
--- a/asterixdb/asterix-app/src/main/resources/cc4.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc4.conf
@@ -44,6 +44,7 @@
 heartbeat.max.misses=25
 
 [common]
+log.dir = logs/
 log.level = INFO
 compiler.framesize=32KB
 compiler.sortmemory=320KB
diff --git a/asterixdb/asterix-app/src/main/resources/log4j2.xml b/asterixdb/asterix-app/src/main/resources/log4j2.xml
index 1debf82..2ea8d4a 100644
--- a/asterixdb/asterix-app/src/main/resources/log4j2.xml
+++ b/asterixdb/asterix-app/src/main/resources/log4j2.xml
@@ -17,15 +17,24 @@
  ! under the License.
  !-->
 <Configuration status="WARN">
+   <CustomLevels>
+    <CustomLevel name="ACCESS" intLevel="550" />
+  </CustomLevels>
   <Appenders>
     <Console name="Console" target="SYSTEM_OUT">
       <PatternLayout pattern="%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"/>
     </Console>
+    <Console name="AccessLog" target="SYSTEM_OUT">
+      <PatternLayout pattern="%m%n"/>
+    </Console>
   </Appenders>
   <Loggers>
     <Root level="WARN">
       <AppenderRef ref="Console"/>
     </Root>
     <Logger name="org.apache.hyracks.control.nc.service" level="INFO"/>
+    <Logger name="org.apache.hyracks.http.server.CLFLogger" level="ACCESS" additivity="false">
+        <AppenderRef ref="AccessLog"/>
+    </Logger>
   </Loggers>
 </Configuration>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index acc3970..ae13e42 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -30,10 +30,12 @@
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BiPredicate;
 import java.util.stream.Stream;
 
 import org.apache.asterix.app.external.ExternalUDFLibrarian;
+import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.PropertiesAccessor;
@@ -42,6 +44,8 @@
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.hyracks.bootstrap.NCApplication;
+import org.apache.asterix.test.dataflow.TestLsmIoOpCallbackFactory;
+import org.apache.asterix.test.dataflow.TestPrimaryIndexOperationTrackerFactory;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.application.ICCApplication;
@@ -55,6 +59,7 @@
 import org.apache.hyracks.control.common.controllers.ControllerConfig;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtreeLocalResource;
 import org.apache.hyracks.test.support.TestUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -70,6 +75,7 @@
             joinPath(getProjectPath().toString(), "src", "test", "resources", "cc.conf");
     private static final String DEFAULT_STORAGE_PATH = joinPath("target", "io", "dir");
     private static String storagePath = DEFAULT_STORAGE_PATH;
+    private static final long RESULT_TTL = TimeUnit.MINUTES.toMillis(5);
 
     static {
         System.setProperty("java.util.logging.manager", org.apache.logging.log4j.jul.LogManager.class.getName());
@@ -95,7 +101,8 @@
      * main method to run a simple 2 node cluster in-process
      * suggested VM arguments: <code>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8</code>
      *
-     * @param args unused
+     * @param args
+     *            unused
      */
     public static void main(String[] args) throws Exception {
         TestUtils.redirectLoggingToConsole();
@@ -170,6 +177,7 @@
         ((ICcApplicationContext) cc.getApplicationContext()).getClusterStateManager().waitForState(ClusterState.ACTIVE);
         hcc = new HyracksConnection(cc.getConfig().getClientListenAddress(), cc.getConfig().getClientListenPort());
         this.ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
+        setTestPersistedResourceRegistry();
     }
 
     public void init(boolean deleteOldInstanceData, String externalLibPath, String confDir) throws Exception {
@@ -198,7 +206,7 @@
         ccConfig.setClientListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ccConfig.setClientListenPort(DEFAULT_HYRACKS_CC_CLIENT_PORT);
         ccConfig.setClusterListenPort(DEFAULT_HYRACKS_CC_CLUSTER_PORT);
-        ccConfig.setResultTTL(120000L);
+        ccConfig.setResultTTL(RESULT_TTL);
         ccConfig.setResultSweepThreshold(1000L);
         ccConfig.setEnforceFrameWriterProtocol(true);
         configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb"));
@@ -217,18 +225,21 @@
         ncConfig.setDataListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ncConfig.setResultListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
         ncConfig.setMessagingListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
-        ncConfig.setResultTTL(120000L);
+        ncConfig.setResultTTL(RESULT_TTL);
         ncConfig.setResultSweepThreshold(1000L);
         ncConfig.setVirtualNC();
         configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb", ncName));
         return ncConfig;
     }
 
-    protected INCApplication createNCApplication() {
+    protected INCApplication createNCApplication()
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        // Instead of using this flag, RecoveryManagerTest should set the desired class in its config file
         if (!gracefulShutdown) {
             return new UngracefulShutdownNCApplication();
         }
-        return new NCApplication();
+        String ncAppClass = (String) configManager.get(NCConfig.Option.APP_CLASS);
+        return (INCApplication) Class.forName(ncAppClass).newInstance();
     }
 
     private NCConfig fixupIODevices(NCConfig ncConfig) throws IOException, AsterixException, CmdLineException {
@@ -364,6 +375,14 @@
         opts.clear();
     }
 
+    public void setTestPersistedResourceRegistry() {
+        for (NodeControllerService nc : ncs) {
+            INcApplicationContext runtimeCtx = (INcApplicationContext) nc.getApplicationContext();
+            runtimeCtx.getServiceContext()
+                    .setPersistedResourceRegistry(new AsterixHyracksIntegrationUtil.TestPersistedResourceRegistry());
+        }
+    }
+
     /**
      * @return the asterix-app absolute path if found, otherwise the default user path.
      */
@@ -395,4 +414,15 @@
         }
     }
 
+    private static class TestPersistedResourceRegistry extends PersistedResourceRegistry {
+        @Override
+        protected void registerClasses() {
+            super.registerClasses();
+            REGISTERED_CLASSES.put("TestLsmBtreeLocalResource", TestLsmBtreeLocalResource.class);
+            REGISTERED_CLASSES.put("TestLsmIoOpCallbackFactory", TestLsmIoOpCallbackFactory.class);
+            REGISTERED_CLASSES.put("TestPrimaryIndexOperationTrackerFactory",
+                    TestPrimaryIndexOperationTrackerFactory.class);
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index d5262cf..eae82af 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -28,8 +28,10 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ClientJobRequest;
 import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
@@ -46,16 +48,17 @@
 
     @Test
     public void testDelete() throws Exception {
+        ICcApplicationContext appCtx = mock(ICcApplicationContext.class);
         // Creates a query cancellation servlet.
-        QueryCancellationServlet cancellationServlet =
-                new QueryCancellationServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        CcQueryCancellationServlet cancellationServlet =
+                new CcQueryCancellationServlet(new ConcurrentHashMap<>(), appCtx, new String[] { "/" });
         // Adds mocked Hyracks client connection into the servlet context.
         IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
         cancellationServlet.ctx().put(ServletConstants.HYRACKS_CONNECTION_ATTR, mockHcc);
         // Adds a query context into the servlet context.
         IStatementExecutorContext queryCtx = new StatementExecutorContext();
         cancellationServlet.ctx().put(ServletConstants.RUNNING_QUERIES_ATTR, queryCtx);
-
+        Mockito.when(appCtx.getHcc()).thenReturn(mockHcc);
         // Tests the case that query is not in the map.
         IServletRequest mockRequest = mockRequest("1");
         IServletResponse mockResponse = mock(IServletResponse.class);
@@ -63,7 +66,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.NOT_FOUND);
 
         // Tests the case that query is in the map.
-        queryCtx.put("1", new JobId(1));
+        queryCtx.put("1", new ClientJobRequest(queryCtx, "1", new JobId(1)));
         cancellationServlet.handle(mockRequest, mockResponse);
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.OK);
 
@@ -73,7 +76,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.BAD_REQUEST);
 
         // Tests the case that the job cancellation hit some exception from Hyracks.
-        queryCtx.put("2", new JobId(2));
+        queryCtx.put("2", new ClientJobRequest(queryCtx, "2", new JobId(2)));
         Mockito.doThrow(new Exception()).when(mockHcc).cancelJob(any());
         mockRequest = mockRequest("2");
         cancellationServlet.handle(mockRequest, mockResponse);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 5cda9f2..b07a30e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -37,7 +37,6 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -46,6 +45,7 @@
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
@@ -58,17 +58,18 @@
 import org.apache.asterix.runtime.formats.FormatUtils;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
 import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.runtime.CommitRuntime;
 import org.apache.asterix.transaction.management.service.logging.LogReader;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -166,7 +167,7 @@
 
     public void deInit(boolean cleanupOnStop) throws Exception {
         ExternalUDFLibrarian.removeLibraryDir();
-        ExecutionTestUtil.tearDown(cleanupOnStop);
+        ExecutionTestUtil.tearDown(cleanupOnStop, runHDFS);
     }
 
     public void setOpts(List<Pair<IOption, Object>> opts) {
@@ -186,7 +187,7 @@
         return new TxnId(jobId.getId());
     }
 
-    public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+    public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
             Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
             int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
             StorageComponentProvider storageComponentProvider, Index secondaryIndex)
@@ -195,7 +196,39 @@
                 primaryKeyIndicators, storageComponentProvider, secondaryIndex, IndexOperation.INSERT);
     }
 
-    public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+    public Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> getBulkLoadSecondaryOperator(
+            IHyracksTaskContext ctx, Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType,
+            ARecordType metaType, int[] filterFields, int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
+            StorageComponentProvider storageComponentProvider, Index secondaryIndex, int numElementsHint)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        try {
+            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            org.apache.hyracks.algebricks.common.utils.Pair<ILSMMergePolicyFactory, Map<String, String>> mergePolicy =
+                    DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
+                    mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
+            SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
+            IIndexDataflowHelperFactory secondaryIndexHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), secondaryIndexInfo.fileSplitProvider);
+            IIndexDataflowHelperFactory primaryIndexHelperFactory = new IndexDataflowHelperFactory(
+                    storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+            int[] fieldPermutation = new int[secondaryIndex.getKeyFieldNames().size()];
+            for (int i = 0; i < fieldPermutation.length; i++) {
+                fieldPermutation[i] = i;
+            }
+            LSMIndexBulkLoadOperatorNodePushable op =
+                    new LSMIndexBulkLoadOperatorNodePushable(secondaryIndexHelperFactory, primaryIndexHelperFactory,
+                            ctx, 0, fieldPermutation, 1.0F, false, numElementsHint, true, secondaryIndexInfo.rDesc,
+                            BulkLoadUsage.CREATE_INDEX, dataset.getDatasetId());
+            op.setOutputFrameWriter(0, new SinkRuntimeFactory().createPushRuntime(ctx)[0], null);
+            return Pair.of(secondaryIndexInfo, op);
+        } catch (Throwable th) {
+            throw HyracksDataException.create(th);
+        }
+    }
+
+    public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
             Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
             int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
             StorageComponentProvider storageComponentProvider, Index secondaryIndex, IndexOperation op)
@@ -210,10 +243,8 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
                     mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
-            IModificationOperationCallbackFactory modOpCallbackFactory =
-                    new PrimaryIndexModificationOperationCallbackFactory(dataset.getDatasetId(),
-                            primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, Operation.get(op),
-                            ResourceType.LSM_BTREE);
+            IModificationOperationCallbackFactory modOpCallbackFactory = dataset.getModificationCallbackFactory(
+                    storageComponentProvider, primaryIndexInfo.index, op, primaryIndexInfo.primaryKeyIndexes);
             IRecordDescriptorProvider recordDescProvider = primaryIndexInfo.getInsertRecordDescriptorProvider();
             RecordDescriptor recordDesc =
                     recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0), 0);
@@ -222,7 +253,7 @@
             LSMInsertDeleteOperatorNodePushable insertOp =
                     new LSMInsertDeleteOperatorNodePushable(ctx, ctx.getTaskAttemptId().getTaskId().getPartition(),
                             primaryIndexInfo.primaryIndexInsertFieldsPermutations, recordDesc, op, true,
-                            indexHelperFactory, modOpCallbackFactory, null);
+                            indexHelperFactory, modOpCallbackFactory, null, null);
 
             // For now, this assumes a single secondary index. recordDesc is always <pk-record-meta>
             // for the index, we will have to create an assign operator that extract the sk
@@ -237,8 +268,9 @@
                             ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
                             : recordType;
                     int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
-                    secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            mdProvider.getFunctionManager(), sourceType, secondaryIndex.getKeyFieldNames().get(i), pos);
+                    secondaryFieldAccessEvalFactories[i] =
+                            mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
+                                    sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
                 }
                 // outColumns are computed inside the assign runtime
                 int[] outColumns = new int[skNames.size()];
@@ -270,18 +302,20 @@
                 LSMInsertDeleteOperatorNodePushable secondaryInsertOp =
                         new LSMInsertDeleteOperatorNodePushable(ctx, ctx.getTaskAttemptId().getTaskId().getPartition(),
                                 secondaryIndexInfo.insertFieldsPermutations, secondaryIndexInfo.rDesc, op, false,
-                                secondaryIndexHelperFactory, secondaryModCallbackFactory, null);
+                                secondaryIndexHelperFactory, secondaryModCallbackFactory, null, null);
                 assignOp.setOutputFrameWriter(0, secondaryInsertOp, secondaryIndexInfo.rDesc);
-                CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
-                        secondaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
-                        true);
+
+                IPushRuntime commitOp =
+                        dataset.getCommitRuntimeFactory(mdProvider, secondaryIndexInfo.primaryKeyIndexes, true)
+                                .createPushRuntime(ctx)[0];
+
                 secondaryInsertOp.setOutputFrameWriter(0, commitOp, secondaryIndexInfo.rDesc);
                 commitOp.setInputRecordDescriptor(0, secondaryIndexInfo.rDesc);
                 return Pair.of(insertOp, commitOp);
             } else {
-                CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
-                        primaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
-                        true);
+                IPushRuntime commitOp =
+                        dataset.getCommitRuntimeFactory(mdProvider, primaryIndexInfo.primaryKeyIndexes, true)
+                                .createPushRuntime(ctx)[0];
                 insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
                 commitOp.setInputRecordDescriptor(0, primaryIndexInfo.rDesc);
                 return Pair.of(insertOp, commitOp);
@@ -380,7 +414,6 @@
                 (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), dataverse);
         SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
         try {
-
             IResourceFactory resourceFactory = primaryIndexInfo.dataset.getResourceFactory(mdProvider, secondaryIndex,
                     primaryIndexInfo.recordType, primaryIndexInfo.metaType, mergePolicy.first, mergePolicy.second);
             IndexBuilderFactory indexBuilderFactory =
@@ -419,9 +452,9 @@
             secondaryIndexSerdes[i] =
                     SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(secondaryKeyTypes[i]);
         }
-        for (; i < primaryKeyTypes.length; i++) {
-            secondaryIndexSerdes[i] =
-                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
+        for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+            secondaryIndexSerdes[i] = SerializerDeserializerProvider.INSTANCE
+                    .getSerializerDeserializer(primaryKeyTypes[i - secondaryKeyTypes.length]);
         }
         return secondaryIndexSerdes;
     }
@@ -447,8 +480,9 @@
         for (; i < secondaryKeyTypes.length; i++) {
             secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(secondaryKeyTypes[i]);
         }
-        for (; i < primaryKeyTypes.length; i++) {
-            secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i]);
+        for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+            secondaryIndexTypeTraits[i] =
+                    TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i - secondaryKeyTypes.length]);
         }
         return secondaryIndexTypeTraits;
     }
@@ -491,12 +525,14 @@
     }
 
     public static class SecondaryIndexInfo {
-        private final int[] primaryKeyIndexes;
-        private final PrimaryIndexInfo primaryIndexInfo;
-        private final Index secondaryIndex;
-        private final ConstantFileSplitProvider fileSplitProvider;
-        private final RecordDescriptor rDesc;
-        private final int[] insertFieldsPermutations;
+        final int[] primaryKeyIndexes;
+        final PrimaryIndexInfo primaryIndexInfo;
+        final Index secondaryIndex;
+        final ConstantFileSplitProvider fileSplitProvider;
+        final ISerializerDeserializer<?>[] secondaryIndexSerdes;
+        final RecordDescriptor rDesc;
+        final int[] insertFieldsPermutations;
+        final ITypeTraits[] secondaryIndexTypeTraits;
 
         public SecondaryIndexInfo(PrimaryIndexInfo primaryIndexInfo, Index secondaryIndex) {
             this.primaryIndexInfo = primaryIndexInfo;
@@ -507,11 +543,11 @@
             FileSplit[] splits = SplitsAndConstraintsUtil.getIndexSplits(appCtx.getClusterStateManager(),
                     primaryIndexInfo.dataset, secondaryIndex.getIndexName(), nodes);
             fileSplitProvider = new ConstantFileSplitProvider(splits);
-            ITypeTraits[] secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
+            secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
                     primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
                     secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
-            ISerializerDeserializer<?>[] secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType,
-                    primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
+            secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType, primaryIndexInfo.metaType,
+                    primaryIndexInfo.primaryKeyTypes,
                     secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
             rDesc = new RecordDescriptor(secondaryIndexSerdes, secondaryIndexTypeTraits);
             insertFieldsPermutations = new int[secondaryIndexTypeTraits.length];
@@ -527,6 +563,10 @@
         public IFileSplitProvider getFileSplitProvider() {
             return fileSplitProvider;
         }
+
+        public ISerializerDeserializer<?>[] getSerdes() {
+            return secondaryIndexSerdes;
+        }
     }
 
     public static class PrimaryIndexInfo {
@@ -645,6 +685,9 @@
             int[] keyIndexes, List<Integer> keyIndicators, StorageComponentProvider storageComponentProvider,
             IFrameOperationCallbackFactory frameOpCallbackFactory, boolean hasSecondaries) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        MetadataProvider mdProvider = new MetadataProvider(
+                (ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(),
+                MetadataBuiltinEntities.DEFAULT_DATAVERSE);
         org.apache.hyracks.algebricks.common.utils.Pair<ILSMMergePolicyFactory, Map<String, String>> mergePolicy =
                 DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
         MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -661,8 +704,9 @@
                 new LSMPrimaryUpsertOperatorNodePushable(ctx, ctx.getTaskAttemptId().getTaskId().getPartition(),
                         indexHelperFactory, primaryIndexInfo.primaryIndexInsertFieldsPermutations,
                         recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0), 0),
-                        modificationCallbackFactory, searchCallbackFactory, keyIndexes.length, recordType, -1,
-                        frameOpCallbackFactory == null ? dataset.getFrameOpCallbackFactory() : frameOpCallbackFactory,
+                        modificationCallbackFactory, searchCallbackFactory,
+                        keyIndexes.length, recordType, -1, frameOpCallbackFactory == null
+                                ? dataset.getFrameOpCallbackFactory(mdProvider) : frameOpCallbackFactory,
                         MissingWriterFactory.INSTANCE, hasSecondaries);
         RecordDescriptor upsertOutRecDesc = getUpsertOutRecDesc(primaryIndexInfo.rDesc, dataset,
                 filterFields == null ? 0 : filterFields.length, recordType, metaType);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
index 2eba473..ff027e2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class ABooleanFieldValueGenerator implements IAsterixFieldValueGenerator<Boolean> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
index e698676..64dab3d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class ADoubleFieldValueGenerator implements IAsterixFieldValueGenerator<Double> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
index 7c6556b..5540c11 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AInt32FieldValueGenerator implements IAsterixFieldValueGenerator<Integer> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
index 2a2496e..e62054a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AInt64FieldValueGenerator implements IAsterixFieldValueGenerator<Long> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
index cd4de62..b242189 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
@@ -21,7 +21,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
index 5ee6d40..419d1b6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import java.util.Random;
 
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.util.string.UTF8StringReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
new file mode 100644
index 0000000..1b04c1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
@@ -0,0 +1,126 @@
+/*
+* 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.asterix.app.data.gen;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.test.common.TestTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class RecordTupleGenerator {
+
+    private final int[] keyIndexes;
+    private final int[] keyIndicators;
+    private final ARecordValueGenerator recordGenerator;
+    private final ARecordValueGenerator metaGenerator;
+    private final TestTupleReference tuple;
+
+    public enum GenerationFunction {
+        RANDOM,
+        DETERMINISTIC,
+        INCREASING,
+        DECREASING
+    }
+
+    /**
+     * @param recordType
+     * @param metaType
+     * @param key
+     * @param keyIndexes
+     * @param keyIndicators
+     * @param recordGeneration
+     * @param uniqueRecordFields
+     * @param metaGeneration
+     * @param uniqueMetaFields
+     */
+    public RecordTupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
+            GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
+            boolean[] uniqueMetaFields) {
+        this.keyIndexes = keyIndexes;
+        this.keyIndicators = keyIndicators;
+        for (IAType field : recordType.getFieldTypes()) {
+            validate(field);
+        }
+        recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
+        if (metaType != null) {
+            for (IAType field : metaType.getFieldTypes()) {
+                validate(field);
+            }
+            metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
+        } else {
+            metaGenerator = null;
+        }
+        int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
+        tuple = new TestTupleReference(numOfFields);
+        boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
+        for (int i = 0; i < keyIndexes.length; i++) {
+            if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
+                throw new IllegalArgumentException("key field indicator must be either 0 or 1");
+            }
+            atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
+                    || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
+                            keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
+        }
+        if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
+            throw new IllegalArgumentException("at least one key field must be unique and not boolean");
+        }
+        if (keyIndexes.length != keyIndicators.length) {
+            throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
+        }
+    }
+
+    private boolean validateKey(int i, ARecordType type, boolean unique) {
+        if (type.getFieldNames().length <= i) {
+            throw new IllegalArgumentException("key index must be less than number of fields");
+        }
+        return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
+    }
+
+    public ITupleReference next() throws IOException {
+        tuple.reset();
+        recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
+        if (metaGenerator != null) {
+            recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
+        }
+        for (int i = 0; i < keyIndexes.length; i++) {
+            if (keyIndicators[i] == 0) {
+                recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+            } else {
+                metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+            }
+        }
+        return tuple;
+    }
+
+    private void validate(IAType field) {
+        switch (field.getTypeTag()) {
+            case BOOLEAN:
+            case DOUBLE:
+            case INTEGER:
+            case BIGINT:
+            case STRING:
+                break;
+            default:
+                throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
deleted file mode 100644
index 0469349..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
-* 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.asterix.app.data.gen;
-
-import java.io.IOException;
-
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public class TupleGenerator {
-
-    private final int[] keyIndexes;
-    private final int[] keyIndicators;
-    private final ARecordValueGenerator recordGenerator;
-    private final ARecordValueGenerator metaGenerator;
-    private final TestTupleReference tuple;
-
-    public enum GenerationFunction {
-        RANDOM,
-        DETERMINISTIC,
-        INCREASING,
-        DECREASING
-    }
-
-    /**
-     * @param recordType
-     * @param metaType
-     * @param key
-     * @param keyIndexes
-     * @param keyIndicators
-     * @param recordGeneration
-     * @param uniqueRecordFields
-     * @param metaGeneration
-     * @param uniqueMetaFields
-     */
-    public TupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
-            GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
-            boolean[] uniqueMetaFields) {
-        this.keyIndexes = keyIndexes;
-        this.keyIndicators = keyIndicators;
-        for (IAType field : recordType.getFieldTypes()) {
-            validate(field);
-        }
-        recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
-        if (metaType != null) {
-            for (IAType field : metaType.getFieldTypes()) {
-                validate(field);
-            }
-            metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
-        } else {
-            metaGenerator = null;
-        }
-        int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
-        tuple = new TestTupleReference(numOfFields);
-        boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
-        for (int i = 0; i < keyIndexes.length; i++) {
-            if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
-                throw new IllegalArgumentException("key field indicator must be either 0 or 1");
-            }
-            atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
-                    || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
-                            keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
-        }
-        if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
-            throw new IllegalArgumentException("at least one key field must be unique and not boolean");
-        }
-        if (keyIndexes.length != keyIndicators.length) {
-            throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
-        }
-    }
-
-    private boolean validateKey(int i, ARecordType type, boolean unique) {
-        if (type.getFieldNames().length <= i) {
-            throw new IllegalArgumentException("key index must be less than number of fields");
-        }
-        return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
-    }
-
-    public ITupleReference next() throws IOException {
-        tuple.reset();
-        recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
-        if (metaGenerator != null) {
-            recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
-        }
-        for (int i = 0; i < keyIndexes.length; i++) {
-            if (keyIndicators[i] == 0) {
-                recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
-            } else {
-                metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
-            }
-        }
-        return tuple;
-    }
-
-    private void validate(IAType field) {
-        switch (field.getTypeTag()) {
-            case BOOLEAN:
-            case DOUBLE:
-            case INTEGER:
-            case BIGINT:
-            case STRING:
-                break;
-            default:
-                throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index 4beb44a..9d367ac 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -31,14 +31,19 @@
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.metadata.IMetadataBootstrap;
+import org.apache.asterix.common.utils.NcLocalCounters;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
+import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobIdFactory;
+import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.application.CCServiceContext;
 import org.apache.hyracks.control.common.application.ConfigManagerApplicationConfig;
 import org.apache.hyracks.control.common.config.ConfigManager;
@@ -194,8 +199,7 @@
 
     private void notifyNodeStartupCompletion(CcApplicationContext applicationContext, String nodeId)
             throws HyracksDataException {
-        NCLifecycleTaskReportMessage msg = new NCLifecycleTaskReportMessage(nodeId, true);
-        applicationContext.getResourceIdManager().report(nodeId, 0);
+        NCLifecycleTaskReportMessage msg = new NCLifecycleTaskReportMessage(nodeId, true, mockLocalCounters());
         applicationContext.getNcLifecycleCoordinator().process(msg);
     }
 
@@ -204,7 +208,12 @@
         ConfigManager configManager = new ConfigManager(null);
         IApplicationConfig applicationConfig = new ConfigManagerApplicationConfig(configManager);
         ICCServiceContext iccServiceContext = Mockito.mock(CCServiceContext.class);
+        final ClusterControllerService ccs = Mockito.mock(ClusterControllerService.class);
+        JobIdFactory jobIdFactory = new JobIdFactory(CcId.valueOf(0));
+        Mockito.when(ccs.getJobIdFactory()).thenReturn(jobIdFactory);
         Mockito.when(iccServiceContext.getAppConfig()).thenReturn(applicationConfig);
+        Mockito.when(iccServiceContext.getControllerService()).thenReturn(ccs);
+
         Mockito.when(ccApplicationContext.getServiceContext()).thenReturn(iccServiceContext);
 
         NcLifecycleCoordinator coordinator =
@@ -225,6 +234,9 @@
         IGlobalRecoveryManager globalRecoveryManager = Mockito.mock(IGlobalRecoveryManager.class);
         Mockito.when(globalRecoveryManager.isRecoveryCompleted()).thenReturn(true);
         Mockito.when(ccApplicationContext.getGlobalRecoveryManager()).thenReturn(globalRecoveryManager);
+
+        BulkTxnIdFactory bulkTxnIdFactory = new BulkTxnIdFactory();
+        Mockito.when(ccApplicationContext.getTxnIdFactory()).thenReturn(bulkTxnIdFactory);
         return ccApplicationContext;
     }
 
@@ -238,4 +250,12 @@
         Mockito.when(metadataProperties.getNodePartitions()).thenReturn(nodePartitionsMap);
         return metadataProperties;
     }
+
+    private NcLocalCounters mockLocalCounters() {
+        final NcLocalCounters localCounters = Mockito.mock(NcLocalCounters.class);
+        Mockito.when(localCounters.getMaxJobId()).thenReturn(1000L);
+        Mockito.when(localCounters.getMaxResourceId()).thenReturn(1000L);
+        Mockito.when(localCounters.getMaxTxnId()).thenReturn(1000L);
+        return localCounters;
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
index baaf546..2e7445f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ExceptionIT.java
@@ -51,6 +51,7 @@
             // We test all generated functions except
             // record and cast functions, which requires type settings.
             if (className.contains("Gen") && !className.contains("record") && !className.contains("Cast")) {
+                System.out.println("Testing " + className);
                 testFunction(func);
                 ++testedFunctions;
             }
@@ -67,9 +68,9 @@
         while (argEvalFactoryIterator.hasNext()) {
             IScalarEvaluatorFactory evalFactory = funcDesc.createEvaluatorFactory(argEvalFactoryIterator.next());
             IHyracksTaskContext ctx = mock(IHyracksTaskContext.class);
-            IScalarEvaluator evaluator = evalFactory.createScalarEvaluator(ctx);
-            IPointable resultPointable = new VoidPointable();
             try {
+                IScalarEvaluator evaluator = evalFactory.createScalarEvaluator(ctx);
+                IPointable resultPointable = new VoidPointable();
                 evaluator.evaluate(null, resultPointable);
             } catch (Throwable e) {
                 String msg = e.getMessage();
@@ -81,6 +82,10 @@
                     int errorCode = Integer.parseInt(msg.substring(3, 7));
                     Assert.assertTrue(errorCode >= 0 && errorCode < 1000);
                     continue;
+                } else if (msg.startsWith("HYR")) {
+                    // Verifies the error code.
+                    int errorCode = Integer.parseInt(msg.substring(3, 7));
+                    Assert.assertTrue(errorCode >= 0 && errorCode < 1000);
                 } else {
                     // Any root-level data exceptions thrown from runtime functions should have an error code.
                     Assert.assertTrue(!(e instanceof HyracksDataException) || (e.getCause() != null));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
index d96fec9..f302cef 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
@@ -51,6 +51,7 @@
             // We test all generated functions except
             // record and cast functions, which requires type settings (we test them in runtime tests).
             if (className.contains("Gen") && !className.contains("record") && !className.contains("Cast")) {
+                System.out.println("Testing " + className);
                 testFunction(func);
                 ++testedFunctions;
             }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index 9612ead..64520a4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -37,6 +37,7 @@
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.config.ActiveProperties;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
@@ -121,6 +122,7 @@
         Mockito.when(appCtx.getMetadataLockManager()).thenReturn(lockManager);
         Mockito.when(appCtx.getServiceContext()).thenReturn(ccServiceCtx);
         Mockito.when(appCtx.getClusterStateManager()).thenReturn(clusterStateManager);
+        Mockito.when(appCtx.getActiveProperties()).thenReturn(Mockito.mock(ActiveProperties.class));
         componentProvider = new StorageComponentProvider();
         Mockito.when(appCtx.getStorageComponentProvider()).thenReturn(componentProvider);
         Mockito.when(ccServiceCtx.getControllerService()).thenReturn(ccService);
@@ -180,7 +182,7 @@
         Action action = users[0].startActivity(listener);
         action.sync();
         assertFailure(action, 0);
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
     }
 
     @Test
@@ -190,7 +192,42 @@
         Action action = users[0].startActivity(listener);
         action.sync();
         assertFailure(action, 0);
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testStartWhenStartSucceedButTimesout() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.FAIL_START_TIMEOUT_OP_SUCCEED);
+        Action action = users[0].startActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+    }
+
+    @Test
+    public void testStartWhenStartStuckTimesout() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.FAIL_START_TIMEOUT_STUCK);
+        Action action = users[0].startActivity(listener);
+        action.sync();
+        assertFailure(action, 0);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+    }
+
+    @Test
+    public void testStopWhenStopTimesout() throws Exception {
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
+        listener.onStart(Behavior.SUCCEED);
+        Action action = users[0].startActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.RUNNING, listener.getState());
+        listener.onStop(Behavior.FAIL_STOP_TIMEOUT);
+        action = users[0].stopActivity(listener);
+        action.sync();
+        assertSuccess(action);
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
     }
 
     @Test
@@ -336,14 +373,18 @@
     }
 
     @Test
-    public void testSuspendFromRunningAndStopFail() throws Exception {
+    public void testSuspendFromRunningAndStopFailThenResumeSucceeds() throws Exception {
         testStartWhenStartSucceed();
         // suspend
         Assert.assertEquals(ActivityState.RUNNING, listener.getState());
-        listener.onStop(Behavior.FAIL_COMPILE);
+        listener.onStop(Behavior.RUNNING_JOB_FAIL);
         Action suspension = users[1].suspendActivity(listener);
         suspension.sync();
-        Assert.assertTrue(suspension.hasFailed());
+        Assert.assertFalse(suspension.hasFailed());
+        Assert.assertEquals(ActivityState.TEMPORARILY_FAILED, listener.getState());
+        Action resumption = users[1].resumeActivity(listener);
+        resumption.sync();
+        assertSuccess(resumption);
         Assert.assertEquals(ActivityState.RUNNING, listener.getState());
     }
 
@@ -492,19 +533,19 @@
         WaitForStateSubscriber tempFailSubscriber =
                 new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
         WaitForStateSubscriber permFailSubscriber =
-                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
         listener.onStart(Behavior.FAIL_COMPILE);
         clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
                 Collections.singletonList(new HyracksDataException("Compilation Failure")));
         tempFailSubscriber.sync();
         permFailSubscriber.sync();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
     }
 
     @Test
     public void testStartAfterPermenantFailure() throws Exception {
         testRecoveryFailureAfterOneAttemptCompilationFailure();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         listener.onStart(Behavior.SUCCEED);
         WaitForStateSubscriber subscriber = new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.RUNNING));
         users[1].startActivity(listener);
@@ -536,13 +577,13 @@
         WaitForStateSubscriber tempFailSubscriber =
                 new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
         WaitForStateSubscriber permFailSubscriber =
-                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
         listener.onStart(Behavior.FAIL_RUNTIME);
         clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
                 Collections.singletonList(new HyracksDataException("Runtime Failure")));
         tempFailSubscriber.sync();
         permFailSubscriber.sync();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
     }
 
     @SuppressWarnings("deprecation")
@@ -555,12 +596,12 @@
         WaitForStateSubscriber tempFailSubscriber =
                 new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
         WaitForStateSubscriber permFailSubscriber =
-                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.PERMANENTLY_FAILED));
+                new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.STOPPED));
         clusterController.jobFinish(listener.getJobId(), JobStatus.FAILURE,
                 Collections.singletonList(new HyracksDataException("Runtime Failure")));
         tempFailSubscriber.sync();
         permFailSubscriber.sync();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
     }
 
     @SuppressWarnings("deprecation")
@@ -925,7 +966,7 @@
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testCreateNewShadowDuringRecoveryAttemptThatSucceeds() throws Exception {
+    public void testCreateNewDatasetDuringRecoveryAttemptThatSucceeds() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -953,7 +994,7 @@
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testCreateNewShadowDuringRecoveryAttemptThatFailsCompile() throws Exception {
+    public void testCreateNewDatasetDuringRecoveryAttemptThatFailsCompile() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -980,7 +1021,7 @@
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testCreateNewShadowDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+    public void testCreateNewDatasetDuringRecoveryAttemptThatFailsRuntime() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -1006,7 +1047,7 @@
     }
 
     @Test
-    public void testCreateNewShadowWhileStarting() throws Exception {
+    public void testCreateNewDatasetWhileStarting() throws Exception {
         Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         listener.onStart(Behavior.STEP_SUCCEED);
         Action startAction = users[0].startActivity(listener);
@@ -1027,7 +1068,7 @@
     }
 
     @Test
-    public void testCreateNewShadowWhileRunning() throws Exception {
+    public void testCreateNewDatasetWhileRunning() throws Exception {
         testStartWhenStartSucceed();
         Dataset newDataset =
                 new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
@@ -1040,7 +1081,7 @@
     }
 
     @Test
-    public void testCreateNewShadowWhileSuspended() throws Exception {
+    public void testCreateNewDatasetWhileSuspended() throws Exception {
         testStartWhenStartSucceed();
         // suspend
         Assert.assertEquals(ActivityState.RUNNING, listener.getState());
@@ -1065,22 +1106,22 @@
     }
 
     @Test
-    public void testCreateNewShadowWhilePermanentFailure() throws Exception {
+    public void testCreateNewDatasetWhilePermanentFailure() throws Exception {
         testRecoveryFailureAfterOneAttemptCompilationFailure();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Dataset newDataset =
                 new Dataset(dataverseName, "newDataset", null, null, null, null, null, null, null, null, 0, 0);
         Action createDatasetAction = users[0].addDataset(newDataset, listener);
         createDatasetAction.sync();
         assertSuccess(createDatasetAction);
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Assert.assertEquals(3, listener.getDatasets().size());
         Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
     }
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testDeleteShadowDuringRecoveryAttemptThatSucceeds() throws Exception {
+    public void testDeleteDatasetDuringRecoveryAttemptThatSucceeds() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -1106,7 +1147,7 @@
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testDeleteShadowDuringRecoveryAttemptThatFailsCompile() throws Exception {
+    public void testDeleteDatasetDuringRecoveryAttemptThatFailsCompile() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -1131,7 +1172,7 @@
 
     @SuppressWarnings("deprecation")
     @Test
-    public void testDeleteShadowDuringRecoveryAttemptThatFailsRuntime() throws Exception {
+    public void testDeleteDatasetDuringRecoveryAttemptThatFailsRuntime() throws Exception {
         testStartWhenStartSucceed();
         listener.onStart(Behavior.FAIL_COMPILE);
         WaitForStateSubscriber tempFailSubscriber =
@@ -1155,7 +1196,7 @@
     }
 
     @Test
-    public void testDeleteShadowWhileStarting() throws Exception {
+    public void testDeleteDatasetWhileStarting() throws Exception {
         Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         listener.onStart(Behavior.STEP_SUCCEED);
         Action startAction = users[0].startActivity(listener);
@@ -1174,7 +1215,7 @@
     }
 
     @Test
-    public void testDeleteShadowWhileRunning() throws Exception {
+    public void testDeleteDatasetWhileRunning() throws Exception {
         testStartWhenStartSucceed();
         Action dropDatasetAction = users[1].dropDataset(firstDataset, listener);
         dropDatasetAction.sync();
@@ -1185,19 +1226,19 @@
     }
 
     @Test
-    public void testDeleteShadowWhilePermanentFailure() throws Exception {
+    public void testDeleteDatasetWhilePermanentFailure() throws Exception {
         testRecoveryFailureAfterOneAttemptCompilationFailure();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Action dropDatasetAction = users[0].dropDataset(secondDataset, listener);
         dropDatasetAction.sync();
         assertSuccess(dropDatasetAction);
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Assert.assertEquals(1, listener.getDatasets().size());
         Assert.assertEquals(clusterController.getAllDatasets().size(), listener.getDatasets().size());
     }
 
     @Test
-    public void testDeleteShadowWhileSuspended() throws Exception {
+    public void testDeleteDatasetWhileSuspended() throws Exception {
         testStartWhenStartSucceed();
         // suspend
         Assert.assertEquals(ActivityState.RUNNING, listener.getState());
@@ -1317,7 +1358,7 @@
     @Test
     public void testCreateNewIndexWhilePermanentFailure() throws Exception {
         testRecoveryFailureAfterOneAttemptCompilationFailure();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Action add = users[1].addIndex(firstDataset, listener);
         add.sync();
         assertSuccess(add);
@@ -1442,7 +1483,7 @@
     @Test
     public void testDeleteIndexWhilePermanentFailure() throws Exception {
         testRecoveryFailureAfterOneAttemptCompilationFailure();
-        Assert.assertEquals(ActivityState.PERMANENTLY_FAILED, listener.getState());
+        Assert.assertEquals(ActivityState.STOPPED, listener.getState());
         Action drop = users[1].dropIndex(firstDataset, listener);
         drop.sync();
         assertSuccess(drop);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java
index c269803..88f1332 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/DummyFeedEventsListener.java
@@ -19,8 +19,8 @@
 package org.apache.asterix.test.active;
 
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
@@ -35,7 +35,6 @@
 import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -64,14 +63,13 @@
     }
 
     @Override
-    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException {
+    protected void doStop(MetadataProvider metadataProvider, long timeout, TimeUnit unit) throws HyracksDataException {
         IActiveEntityEventSubscriber eventSubscriber =
-                new WaitForStateSubscriber(this, EnumSet.of(ActivityState.RUNNING, ActivityState.PERMANENTLY_FAILED));
+                new WaitForStateSubscriber(this, Collections.singleton(ActivityState.STOPPED));
         try {
             eventSubscriber.sync();
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
-        return null;
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
index 7e1bc37..2143404 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
@@ -19,11 +19,11 @@
 package org.apache.asterix.test.active;
 
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IRetryPolicyFactory;
@@ -31,17 +31,19 @@
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.metadata.LockList;
-import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobIdFactory;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.util.ExitUtil;
 
 public class TestEventsListener extends ActiveEntityEventsListener {
 
@@ -50,6 +52,9 @@
         RUNNING_JOB_FAIL,
         FAIL_COMPILE,
         FAIL_RUNTIME,
+        FAIL_START_TIMEOUT_OP_SUCCEED,
+        FAIL_START_TIMEOUT_STUCK,
+        FAIL_STOP_TIMEOUT,
         STEP_SUCCEED,
         STEP_FAIL_COMPILE,
         STEP_FAIL_RUNTIME
@@ -103,9 +108,8 @@
         }
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    protected void doStart(MetadataProvider metadataProvider) throws HyracksDataException {
+    protected JobId compileAndStartJob(MetadataProvider metadataProvider) throws HyracksDataException {
         step(onStart);
         try {
             metadataProvider.getApplicationContext().getMetadataLockManager()
@@ -119,60 +123,72 @@
         try {
             startJob.sync();
         } catch (InterruptedException e) {
-            throw HyracksDataException.create(e);
+            ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
         }
-        WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this,
-                EnumSet.of(ActivityState.RUNNING, ActivityState.TEMPORARILY_FAILED, ActivityState.PERMANENTLY_FAILED));
         if (onStart == Behavior.FAIL_RUNTIME || onStart == Behavior.STEP_FAIL_RUNTIME) {
             clusterController.jobFinish(jobId, JobStatus.FAILURE,
                     Collections.singletonList(new HyracksDataException("RuntimeFailure")));
-        } else {
+        } else if (onStart != Behavior.FAIL_START_TIMEOUT_OP_SUCCEED && onStart != Behavior.FAIL_START_TIMEOUT_STUCK) {
             for (int i = 0; i < nodeControllers.length; i++) {
                 TestNodeControllerActor nodeController = nodeControllers[i];
                 nodeController.registerRuntime(jobId, entityId, i);
             }
         }
-        try {
-            subscriber.sync();
-            if (subscriber.getFailure() != null) {
-                throw subscriber.getFailure();
+        if (onStart == Behavior.FAIL_START_TIMEOUT_OP_SUCCEED) {
+            for (int i = 0; i < nodeControllers.length; i++) {
+                TestNodeControllerActor nodeController = nodeControllers[i];
+                try {
+                    nodeController.registerRuntime(jobId, entityId, i).sync();
+                } catch (InterruptedException e) {
+                    ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+                }
             }
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
+            // At this point, the job has started and both nodes reported that they started.
+            // but since we're holding the lock on the listener (this is a synchronized method), the state
+            // didn't change yet
+            while (state != ActivityState.RUNNING) {
+                try {
+                    wait();
+                } catch (InterruptedException e) {
+                    ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+                }
+            }
+            Thread.currentThread().interrupt();
+        } else if (onStart == Behavior.FAIL_START_TIMEOUT_STUCK) {
+            TestNodeControllerActor nodeController = nodeControllers[0];
+            try {
+                nodeController.registerRuntime(jobId, entityId, 0).sync();
+            } catch (InterruptedException e) {
+                ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+            }
+            Thread.currentThread().interrupt();
         }
+        return jobId;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    protected Void doStop(MetadataProvider metadataProvider) throws HyracksDataException {
-        ActivityState intention = state;
+    protected void cancelJobSafely(MetadataProvider metadataProvider, Throwable th) {
+        clusterController.jobFinish(jobId, JobStatus.FAILURE,
+                Collections.singletonList(HyracksException.create(ErrorCode.JOB_CANCELED, jobId)));
+    }
+
+    @Override
+    protected void sendStopMessages(MetadataProvider metadataProvider, long timeout, TimeUnit unit) throws Exception {
         step(onStop);
         failCompile(onStop);
-        try {
-            Set<ActivityState> waitFor;
-            if (intention == ActivityState.STOPPING) {
-                waitFor = EnumSet.of(ActivityState.STOPPED, ActivityState.PERMANENTLY_FAILED);
-            } else if (intention == ActivityState.SUSPENDING) {
-                waitFor = EnumSet.of(ActivityState.SUSPENDED, ActivityState.TEMPORARILY_FAILED);
-            } else {
-                throw new IllegalStateException("stop with what intention??");
+        if (onStop == Behavior.RUNNING_JOB_FAIL) {
+            clusterController.jobFinish(jobId, JobStatus.FAILURE,
+                    Collections.singletonList(new HyracksDataException("RuntimeFailure")));
+        } else if (onStop == Behavior.FAIL_STOP_TIMEOUT) {
+            // Nothing happens.
+            Thread.currentThread().interrupt();
+        } else {
+            for (int i = 0; i < nodeControllers.length; i++) {
+                TestNodeControllerActor nodeController = nodeControllers[0];
+                nodeController.deRegisterRuntime(jobId, entityId, i).sync();
             }
-            WaitForStateSubscriber subscriber = new WaitForStateSubscriber(this, waitFor);
-            if (onStop == Behavior.RUNNING_JOB_FAIL) {
-                clusterController.jobFinish(jobId, JobStatus.FAILURE,
-                        Collections.singletonList(new HyracksDataException("RuntimeFailure")));
-            } else {
-                for (int i = 0; i < nodeControllers.length; i++) {
-                    TestNodeControllerActor nodeController = nodeControllers[0];
-                    nodeController.deRegisterRuntime(jobId, entityId, i).sync();
-                }
-                clusterController.jobFinish(jobId, JobStatus.TERMINATED, Collections.emptyList());
-            }
-            subscriber.sync();
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
+            clusterController.jobFinish(jobId, JobStatus.TERMINATED, Collections.emptyList());
         }
-        return null;
     }
 
     public void onStart(Behavior behavior) {
@@ -180,29 +196,38 @@
     }
 
     public void onStop(Behavior behavior) {
+        if (behavior == Behavior.FAIL_COMPILE) {
+            throw new IllegalArgumentException("Test framework is not designed for this case");
+        }
         this.onStop = behavior;
     }
 
     @Override
-    protected void setRunning(MetadataProvider metadataProvider, boolean running) throws HyracksDataException {
+    protected void setRunning(MetadataProvider metadataProvider, boolean running) {
         try {
             IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
             LockList locks = metadataProvider.getLocks();
             lockManager.acquireDataverseReadLock(locks, entityId.getDataverse());
             lockManager.acquireActiveEntityWriteLock(locks, entityId.getDataverse() + '.' + entityId.getEntityName());
             // persist entity
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
+        } catch (Throwable th) {
+            // This failure puts the system in a bad state.
+            throw new IllegalStateException(th);
         }
     }
 
     @Override
-    protected Void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException {
-        return doStop(metadataProvider);
+    protected void doSuspend(MetadataProvider metadataProvider) throws HyracksDataException {
+        doStop(metadataProvider, appCtx.getActiveProperties().getActiveSuspendTimeout(), TIMEOUT_UNIT);
     }
 
     @Override
     protected void doResume(MetadataProvider metadataProvider) throws HyracksDataException {
         doStart(metadataProvider);
     }
+
+    @Override
+    protected ActiveRuntimeId getActiveRuntimeId(int partition) {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index a1e70dc..e85fedf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -35,6 +35,7 @@
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.test.runtime.SqlppExecutionWithCancellationTest;
 import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
 import org.apache.asterix.testframework.xml.TestCase;
 import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.http.HttpResponse;
@@ -51,8 +52,8 @@
             List<TestCase.CompilationUnit.Parameter> params, boolean jsonEncoded,
             Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
         String clientContextId = UUID.randomUUID().toString();
-        final List<TestCase.CompilationUnit.Parameter> newParams =
-                cancellable ? upsertParam(params, "client_context_id", clientContextId) : params;
+        final List<TestCase.CompilationUnit.Parameter> newParams = cancellable
+                ? upsertParam(params, "client_context_id", ParameterTypeEnum.STRING, clientContextId) : params;
         Callable<InputStream> query = () -> {
             try {
                 return CancellationTestExecutor.super.executeQueryService(str, fmt, uri, newParams, jsonEncoded,
@@ -100,7 +101,8 @@
     }
 
     @Override
-    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount) {
+    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount,
+            boolean expectedSourceLoc) {
         // Get the expected exception
         for (Iterator<String> iter = expectedErrors.iterator(); iter.hasNext();) {
             String expectedError = iter.next();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 43833a2..37e1213 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -31,7 +31,7 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import com.fasterxml.jackson.core.PrettyPrinter;
+import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
@@ -54,7 +54,8 @@
         STATUS("status"),
         TYPE("type"),
         ERRORS("errors"),
-        PLANS("plans");
+        PLANS("plans"),
+        WARNINGS("warnings");
 
         private static final Map<String, ResultField> fields = new HashMap<>();
 
@@ -108,7 +109,6 @@
 
     private static InputStream extract(InputStream resultStream, EnumSet<ResultField> resultFields) throws Exception {
         final String resultStr = IOUtils.toString(resultStream, Charset.defaultCharset());
-        final PrettyPrinter singleLine = new SingleLinePrettyPrinter();
         final ObjectNode result = OBJECT_MAPPER.readValue(resultStr, ObjectNode.class);
 
         LOGGER.debug("+++++++\n" + result + "\n+++++++\n");
@@ -135,9 +135,9 @@
                                 resultBuilder.append(fieldValue.get(0).asText());
                             } else {
                                 ObjectMapper omm = new ObjectMapper();
-                                omm.setDefaultPrettyPrinter(singleLine);
                                 omm.enable(SerializationFeature.INDENT_OUTPUT);
-                                resultBuilder.append(omm.writer(singleLine).writeValueAsString(fieldValue));
+                                resultBuilder
+                                        .append(omm.writer(new DefaultPrettyPrinter()).writeValueAsString(fieldValue));
                             }
                         } else {
                             resultBuilder.append(OBJECT_MAPPER.writeValueAsString(fieldValue));
@@ -164,6 +164,7 @@
                 case STATUS:
                 case TYPE:
                 case PLANS:
+                case WARNINGS:
                     resultBuilder.append(OBJECT_MAPPER.writeValueAsString(fieldValue));
                     break;
                 default:
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/SingleLinePrettyPrinter.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/SingleLinePrettyPrinter.java
deleted file mode 100644
index 2932ca4..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/SingleLinePrettyPrinter.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * 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.asterix.test.common;
-
-import java.io.IOException;
-
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.PrettyPrinter;
-import com.fasterxml.jackson.core.SerializableString;
-import com.fasterxml.jackson.core.io.SerializedString;
-import com.fasterxml.jackson.core.util.Instantiatable;
-
-/**
- * Default {@link PrettyPrinter} implementation that uses 2-space
- * indentation with platform-default linefeeds.
- * Usually this class is not instantiated directly, but instead
- * method {@link JsonGenerator#useSingleLinePrettyPrinter} is
- * used, which will use an instance of this class for operation.
- */
-@SuppressWarnings("serial")
-public class SingleLinePrettyPrinter
-        implements PrettyPrinter, Instantiatable<SingleLinePrettyPrinter>, java.io.Serializable {
-    private static final long serialVersionUID = 1;
-
-    /**
-     * Constant that specifies default "root-level" separator to use between
-     * root values: a single space character.
-     *
-     * @since 2.1
-     */
-    public final static SerializedString DEFAULT_ROOT_VALUE_SEPARATOR = new SerializedString(" ");
-
-    /**
-     * Interface that defines objects that can produce indentation used
-     * to separate object entries and array values. Indentation in this
-     * context just means insertion of white space, independent of whether
-     * linefeeds are output.
-     */
-    public interface Indenter {
-        void writeIndentation(JsonGenerator jg, int level) throws IOException;
-
-        /**
-         * @return True if indenter is considered inline (does not add linefeeds),
-         *         false otherwise
-         */
-        boolean isInline();
-    }
-
-    // // // Config, indentation
-
-    /**
-     * By default, let's use only spaces to separate array values.
-     */
-    protected Indenter _arrayIndenter = FixedSpaceIndenter.instance;
-
-    /**
-     * By default, let's use linefeed-adding indenter for separate
-     * object entries. We'll further configure indenter to use
-     * system-specific linefeeds, and 2 spaces per level (as opposed to,
-     * say, single tabs)
-     */
-    protected Indenter _objectIndenter = new FixedSpaceIndenter();
-
-    /**
-     * String printed between root-level values, if any.
-     */
-    protected final SerializableString _rootSeparator;
-
-    // // // Config, other white space configuration
-
-    /**
-     * By default we will add spaces around colons used to
-     * separate object fields and values.
-     * If disabled, will not use spaces around colon.
-     */
-    protected boolean _spacesInObjectEntries = true;
-
-    // // // State:
-
-    /**
-     * Number of open levels of nesting. Used to determine amount of
-     * indentation to use.
-     */
-    protected transient int _nesting;
-
-    /*
-    /**********************************************************
-    /* Life-cycle (construct, configure)
-    /**********************************************************
-    */
-
-    public SingleLinePrettyPrinter() {
-        this(DEFAULT_ROOT_VALUE_SEPARATOR);
-    }
-
-    /**
-     * Constructor that specifies separator String to use between root values;
-     * if null, no separator is printed.
-     * <p>
-     * Note: simply constructs a {@link SerializedString} out of parameter,
-     * calls {@link #SingleLinePrettyPrinter(SerializableString)}
-     *
-     * @param rootSeparator
-     * @since 2.1
-     */
-    public SingleLinePrettyPrinter(String rootSeparator) {
-        this((rootSeparator == null) ? null : new SerializedString(rootSeparator));
-    }
-
-    /**
-     * Constructor that specifies separator String to use between root values;
-     * if null, no separator is printed.
-     *
-     * @param rootSeparator
-     * @since 2.1
-     */
-    public SingleLinePrettyPrinter(SerializableString rootSeparator) {
-        _rootSeparator = rootSeparator;
-    }
-
-    public SingleLinePrettyPrinter(SingleLinePrettyPrinter base) {
-        this(base, base._rootSeparator);
-    }
-
-    public SingleLinePrettyPrinter(SingleLinePrettyPrinter base, SerializableString rootSeparator) {
-        _arrayIndenter = base._arrayIndenter;
-        _objectIndenter = base._objectIndenter;
-        _spacesInObjectEntries = base._spacesInObjectEntries;
-        _nesting = base._nesting;
-
-        _rootSeparator = rootSeparator;
-    }
-
-    public SingleLinePrettyPrinter withRootSeparator(SerializableString rootSeparator) {
-        if (_rootSeparator == rootSeparator || (rootSeparator != null && rootSeparator.equals(_rootSeparator))) {
-            return this;
-        }
-        return new SingleLinePrettyPrinter(this, rootSeparator);
-    }
-
-    /**
-     * @since 2.6.0
-     */
-    public SingleLinePrettyPrinter withRootSeparator(String rootSeparator) {
-        return withRootSeparator((rootSeparator == null) ? null : new SerializedString(rootSeparator));
-    }
-
-    public void indentArraysWith(Indenter i) {
-        _arrayIndenter = (i == null) ? NopIndenter.instance : i;
-    }
-
-    public void indentObjectsWith(Indenter i) {
-        _objectIndenter = (i == null) ? NopIndenter.instance : i;
-    }
-
-    /**
-     * @deprecated Since 2.3 use {@link #withSpacesInObjectEntries} and {@link #withoutSpacesInObjectEntries()}
-     */
-    @Deprecated
-    public void spacesInObjectEntries(boolean b) {
-        _spacesInObjectEntries = b;
-    }
-
-    /**
-     * @since 2.3
-     */
-    public SingleLinePrettyPrinter withArrayIndenter(Indenter i) {
-        if (i == null) {
-            i = NopIndenter.instance;
-        }
-        if (_arrayIndenter == i) {
-            return this;
-        }
-        SingleLinePrettyPrinter pp = new SingleLinePrettyPrinter(this);
-        pp._arrayIndenter = i;
-        return pp;
-    }
-
-    /**
-     * @since 2.3
-     */
-    public SingleLinePrettyPrinter withObjectIndenter(Indenter i) {
-        if (i == null) {
-            i = NopIndenter.instance;
-        }
-        if (_objectIndenter == i) {
-            return this;
-        }
-        SingleLinePrettyPrinter pp = new SingleLinePrettyPrinter(this);
-        pp._objectIndenter = i;
-        return pp;
-    }
-
-    /**
-     * "Mutant factory" method that will return a pretty printer instance
-     * that does use spaces inside object entries; if 'this' instance already
-     * does this, it is returned; if not, a new instance will be constructed
-     * and returned.
-     *
-     * @since 2.3
-     */
-    public SingleLinePrettyPrinter withSpacesInObjectEntries() {
-        return _withSpaces(true);
-    }
-
-    /**
-     * "Mutant factory" method that will return a pretty printer instance
-     * that does not use spaces inside object entries; if 'this' instance already
-     * does this, it is returned; if not, a new instance will be constructed
-     * and returned.
-     *
-     * @since 2.3
-     */
-    public SingleLinePrettyPrinter withoutSpacesInObjectEntries() {
-        return _withSpaces(false);
-    }
-
-    protected SingleLinePrettyPrinter _withSpaces(boolean state) {
-        if (_spacesInObjectEntries == state) {
-            return this;
-        }
-        SingleLinePrettyPrinter pp = new SingleLinePrettyPrinter(this);
-        pp._spacesInObjectEntries = state;
-        return pp;
-    }
-
-    /*
-    /**********************************************************
-    /* Instantiatable impl
-    /**********************************************************
-     */
-
-    @Override
-    public SingleLinePrettyPrinter createInstance() {
-        return new SingleLinePrettyPrinter(this);
-    }
-
-    /*
-    /**********************************************************
-    /* PrettyPrinter impl
-    /**********************************************************
-     */
-
-    @Override
-    public void writeRootValueSeparator(JsonGenerator jg) throws IOException {
-        if (_rootSeparator != null) {
-            jg.writeRaw(_rootSeparator);
-        }
-    }
-
-    @Override
-    public void writeStartObject(JsonGenerator jg) throws IOException {
-        jg.writeRaw('{');
-        ++_nesting;
-    }
-
-    @Override
-    public void beforeObjectEntries(JsonGenerator jg) throws IOException {
-        _objectIndenter.writeIndentation(jg, _nesting);
-    }
-
-    /**
-     * Method called after an object field has been output, but
-     * before the value is output.
-     * <p>
-     * Default handling (without pretty-printing) will output a single
-     * colon to separate the two. Pretty-printer is
-     * to output a colon as well, but can surround that with other
-     * (white-space) decoration.
-     */
-    @Override
-    public void writeObjectFieldValueSeparator(JsonGenerator jg) throws IOException {
-        if (_spacesInObjectEntries) {
-            jg.writeRaw(": ");
-        } else {
-            jg.writeRaw(':');
-        }
-    }
-
-    /**
-     * Method called after an object entry (field:value) has been completely
-     * output, and before another value is to be output.
-     * <p>
-     * Default handling (without pretty-printing) will output a single
-     * comma to separate the two. Pretty-printer is
-     * to output a comma as well, but can surround that with other
-     * (white-space) decoration.
-     */
-    @Override
-    public void writeObjectEntrySeparator(JsonGenerator jg) throws IOException {
-        jg.writeRaw(',');
-        _objectIndenter.writeIndentation(jg, _nesting);
-    }
-
-    @Override
-    public void writeEndObject(JsonGenerator jg, int nrOfEntries) throws IOException {
-        --_nesting;
-        if (nrOfEntries > 1) {
-            _objectIndenter.writeIndentation(jg, _nesting);
-        } else {
-            jg.writeRaw(' ');
-        }
-        jg.writeRaw('}');
-    }
-
-    @Override
-    public void writeStartArray(JsonGenerator jg) throws IOException {
-        ++_nesting;
-        jg.writeRaw('[');
-    }
-
-    @Override
-    public void beforeArrayValues(JsonGenerator jg) throws IOException {
-        _arrayIndenter.writeIndentation(jg, _nesting);
-    }
-
-    /**
-     * Method called after an array value has been completely
-     * output, and before another value is to be output.
-     * <p>
-     * Default handling (without pretty-printing) will output a single
-     * comma to separate the two. Pretty-printer is
-     * to output a comma as well, but can surround that with other
-     * (white-space) decoration.
-     */
-    @Override
-    public void writeArrayValueSeparator(JsonGenerator gen) throws IOException {
-        gen.writeRaw(',');
-        _arrayIndenter.writeIndentation(gen, _nesting);
-    }
-
-    @Override
-    public void writeEndArray(JsonGenerator gen, int nrOfValues) throws IOException {
-        --_nesting;
-
-        if (_nesting == 0) {
-            gen.writeRaw('\n');
-        }
-        if (nrOfValues > 1) {
-            _arrayIndenter.writeIndentation(gen, _nesting);
-        } else {
-            gen.writeRaw(' ');
-        }
-        gen.writeRaw(']');
-    }
-
-    /*
-    /**********************************************************
-    /* Helper classes
-    /**********************************************************
-     */
-
-    /**
-     * Dummy implementation that adds no indentation whatsoever
-     */
-    public static class NopIndenter implements Indenter, java.io.Serializable {
-        public static final NopIndenter instance = new NopIndenter();
-
-        @Override
-        public void writeIndentation(JsonGenerator jg, int level) throws IOException {
-        }
-
-        @Override
-        public boolean isInline() {
-            return true;
-        }
-    }
-
-    /**
-     * This is a very simple indenter that only adds a
-     * single space for indentation. It is used as the default
-     * indenter for array values.
-     */
-    public static class FixedSpaceIndenter extends NopIndenter {
-        @SuppressWarnings("hiding")
-        public static final FixedSpaceIndenter instance = new FixedSpaceIndenter();
-
-        @Override
-        public void writeIndentation(JsonGenerator jg, int level) throws IOException {
-            jg.writeRaw(' ');
-        }
-
-        @Override
-        public boolean isInline() {
-            return true;
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index b7fa3bb..b143ea9 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -69,6 +69,7 @@
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.asterix.testframework.context.TestFileContext;
 import org.apache.asterix.testframework.xml.ComparisonEnum;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit.Parameter;
 import org.apache.asterix.testframework.xml.TestGroup;
@@ -90,6 +91,7 @@
 import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.util.EntityUtils;
+import org.apache.hyracks.http.server.utils.HttpUtil;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -102,6 +104,7 @@
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.util.RawValue;
 
 public class TestExecutor {
 
@@ -109,6 +112,9 @@
      * Static variables
      */
     protected static final Logger LOGGER = LogManager.getLogger();
+    private static final String AQL = "aql";
+    private static final String SQLPP = "sqlpp";
+    private static final String DEFAULT_PLAN_FORMAT = "string";
     // see
     // https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
     private static final long MAX_URL_LENGTH = 2000l;
@@ -122,11 +128,13 @@
     private static final Pattern POLL_DELAY_PATTERN = Pattern.compile("polldelaysecs=(\\d+)(\\D|$)", Pattern.MULTILINE);
     private static final Pattern HANDLE_VARIABLE_PATTERN = Pattern.compile("handlevariable=(\\w+)");
     private static final Pattern VARIABLE_REF_PATTERN = Pattern.compile("\\$(\\w+)");
-    private static final Pattern HTTP_PARAM_PATTERN = Pattern.compile("param (\\w+)=(.*)", Pattern.MULTILINE);
+    private static final Pattern HTTP_PARAM_PATTERN =
+            Pattern.compile("param (?<name>[\\w$]+)(?::(?<type>\\w+))?=(?<value>.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_BODY_PATTERN = Pattern.compile("body=(.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_STATUSCODE_PATTERN = Pattern.compile("statuscode (.*)", Pattern.MULTILINE);
     private static final Pattern MAX_RESULT_READS_PATTERN =
             Pattern.compile("maxresultreads=(\\d+)(\\D|$)", Pattern.MULTILINE);
+    private static final Pattern HTTP_REQUEST_TYPE = Pattern.compile("requesttype=(.*)", Pattern.MULTILINE);
     public static final int TRUNCATE_THRESHOLD = 16384;
     public static final Set<String> NON_CANCELLABLE =
             Collections.unmodifiableSet(new HashSet<>(Arrays.asList("store", "validate")));
@@ -561,14 +569,6 @@
         }
     }
 
-    public InputStream executeQuery(String str, OutputFormat fmt, URI uri, List<Parameter> params) throws Exception {
-        HttpUriRequest method = constructHttpMethod(str, uri, "query", false, params);
-        // Set accepted output response type
-        method.setHeader("Accept", fmt.mimeType());
-        HttpResponse response = executeAndCheckHttpRequest(method);
-        return response.getEntity().getContent();
-    }
-
     public InputStream executeQueryService(String str, URI uri, OutputFormat fmt) throws Exception {
         return executeQueryService(str, fmt, uri, new ArrayList<>(), false);
     }
@@ -585,11 +585,17 @@
 
     public InputStream executeQueryService(String str, OutputFormat fmt, URI uri, List<Parameter> params,
             boolean jsonEncoded, Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
-        List<Parameter> newParams = upsertParam(params, "format", fmt.mimeType());
+        List<Parameter> newParams = upsertParam(params, "format", ParameterTypeEnum.STRING, fmt.mimeType());
+        newParams = upsertParam(newParams, QueryServiceServlet.Parameter.PLAN_FORMAT.str(), ParameterTypeEnum.STRING,
+                DEFAULT_PLAN_FORMAT);
         final Optional<String> maxReadsOptional = extractMaxResultReads(str);
         if (maxReadsOptional.isPresent()) {
             newParams = upsertParam(newParams, QueryServiceServlet.Parameter.MAX_RESULT_READS.str(),
-                    maxReadsOptional.get());
+                    ParameterTypeEnum.STRING, maxReadsOptional.get());
+        }
+        final List<Parameter> additionalParams = extractParameters(str);
+        for (Parameter param : additionalParams) {
+            newParams = upsertParam(newParams, param.getName(), param.getType(), param.getValue());
         }
         HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", newParams)
                 : constructPostMethodUrl(str, uri, "statement", newParams);
@@ -603,16 +609,18 @@
         return response.getEntity().getContent();
     }
 
-    protected List<Parameter> upsertParam(List<Parameter> params, String name, String value) {
+    protected List<Parameter> upsertParam(List<Parameter> params, String name, ParameterTypeEnum type, String value) {
         boolean replaced = false;
         List<Parameter> result = new ArrayList<>();
         for (Parameter param : params) {
             Parameter newParam = new Parameter();
             newParam.setName(param.getName());
             if (name.equals(param.getName())) {
+                newParam.setType(type);
                 newParam.setValue(value);
                 replaced = true;
             } else {
+                newParam.setType(param.getType());
                 newParam.setValue(param.getValue());
             }
             result.add(newParam);
@@ -620,6 +628,7 @@
         if (!replaced) {
             Parameter newParam = new Parameter();
             newParam.setName(name);
+            newParam.setType(type);
             newParam.setValue(value);
             result.add(newParam);
         }
@@ -627,15 +636,9 @@
     }
 
     private HttpUriRequest constructHttpMethod(String statement, URI uri, String stmtParam, boolean postStmtAsParam,
-            List<Parameter> otherParams) throws URISyntaxException {
-        if (statement.length() + uri.toString().length() < MAX_URL_LENGTH) {
-            // Use GET for small-ish queries
-            return constructGetMethod(uri, upsertParam(otherParams, stmtParam, statement));
-        } else {
-            // Use POST for bigger ones to avoid 413 FULL_HEAD
-            String stmtParamName = (postStmtAsParam ? stmtParam : null);
-            return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
-        }
+            List<Parameter> otherParams) {
+        String stmtParamName = (postStmtAsParam ? stmtParam : null);
+        return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
     }
 
     private HttpUriRequest constructGetMethod(URI endpoint, List<Parameter> params) {
@@ -654,9 +657,7 @@
             builder.addParameter(param.getName(), param.getValue());
         }
         builder.setCharset(StandardCharsets.UTF_8);
-        if (body.isPresent()) {
-            builder.setEntity(new StringEntity(body.get(), StandardCharsets.UTF_8));
-        }
+        body.ifPresent(s -> builder.setEntity(new StringEntity(s, StandardCharsets.UTF_8)));
         return builder.build();
     }
 
@@ -684,18 +685,11 @@
         return builder.build();
     }
 
-    private HttpUriRequest constructPostMethod(URI uri, OutputFormat fmt, List<Parameter> params) {
-        HttpUriRequest method = constructPostMethod(uri, params);
-        // Set accepted output response type
-        method.setHeader("Accept", fmt.mimeType());
-        return method;
-    }
-
     protected HttpUriRequest constructPostMethodUrl(String statement, URI uri, String stmtParam,
             List<Parameter> otherParams) {
         RequestBuilder builder = RequestBuilder.post(uri);
         if (stmtParam != null) {
-            for (Parameter param : upsertParam(otherParams, stmtParam, statement)) {
+            for (Parameter param : upsertParam(otherParams, stmtParam, ParameterTypeEnum.STRING, statement)) {
                 builder.addParameter(param.getName(), param.getValue());
             }
             builder.addParameter(stmtParam, statement);
@@ -715,8 +709,23 @@
         RequestBuilder builder = RequestBuilder.post(uri);
         ObjectMapper om = new ObjectMapper();
         ObjectNode content = om.createObjectNode();
-        for (Parameter param : upsertParam(otherParams, stmtParam, statement)) {
-            content.put(param.getName(), param.getValue());
+        for (Parameter param : upsertParam(otherParams, stmtParam, ParameterTypeEnum.STRING, statement)) {
+            String paramName = param.getName();
+            ParameterTypeEnum paramType = param.getType();
+            if (paramType == null) {
+                paramType = ParameterTypeEnum.STRING;
+            }
+            String paramValue = param.getValue();
+            switch (paramType) {
+                case STRING:
+                    content.put(paramName, paramValue);
+                    break;
+                case JSON:
+                    content.putRawValue(paramName, new RawValue(paramValue));
+                    break;
+                default:
+                    throw new IllegalStateException(paramType.toString());
+            }
         }
         try {
             builder.setEntity(new StringEntity(om.writeValueAsString(content), ContentType.APPLICATION_JSON));
@@ -752,57 +761,6 @@
         return response.getEntity().getContent();
     }
 
-    // To execute Update statements
-    // Insert and Delete statements are executed here
-    public void executeUpdate(String str, URI uri) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
-        // Execute the method.
-        executeAndCheckHttpRequest(request);
-    }
-
-    // Executes AQL in either async or async-defer mode.
-    public InputStream executeAnyAQLAsync(String statement, boolean defer, OutputFormat fmt, URI uri,
-            Map<String, Object> variableCtx) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).addParameter("mode", defer ? "asynchronous-deferred" : "asynchronous")
-                        .setEntity(new StringEntity(statement, StandardCharsets.UTF_8))
-                        .setHeader("Accept", fmt.mimeType()).build();
-
-        String handleVar = getHandleVariable(statement);
-
-        HttpResponse response = executeAndCheckHttpRequest(request);
-        InputStream resultStream = response.getEntity().getContent();
-        String resultStr = IOUtils.toString(resultStream, "UTF-8");
-        ObjectNode resultJson = new ObjectMapper().readValue(resultStr, ObjectNode.class);
-        final JsonNode jsonHandle = resultJson.get("handle");
-        final String strHandle = jsonHandle.asText();
-
-        if (handleVar != null) {
-            variableCtx.put(handleVar, strHandle);
-            return resultStream;
-        }
-        return null;
-    }
-
-    // To execute DDL and Update statements
-    // create type statement
-    // create dataset statement
-    // create index statement
-    // create dataverse statement
-    // create function statement
-    public void executeDDL(String str, URI uri) throws Exception {
-        // Create a method instance.
-        HttpUriRequest request =
-                RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
-        // Execute the method.
-        executeAndCheckHttpRequest(request);
-    }
-
     // Method that reads a DDL/Update/Query File
     // and returns the contents as a string
     // This string is later passed to REST API for execution.
@@ -887,13 +845,15 @@
             String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
             MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
             throws Exception {
+        URI uri;
+        InputStream resultStream;
         File qbcFile;
         boolean failed = false;
         File expectedResultFile;
         switch (ctx.getType()) {
             case "ddl":
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
                 } else {
                     executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
                 }
@@ -904,7 +864,7 @@
                     statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
                 }
                 if (ctx.getFile().getName().endsWith("aql")) {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } else {
                     executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 }
@@ -958,14 +918,12 @@
                         expectedResultFileCtxs);
                 break;
             case "txnqbc": // qbc represents query before crash
-                InputStream resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                resultStream = query(cUnit, testFile.getName(), statement);
                 qbcFile = getTestCaseQueryBeforeCrashFile(actualPath, testCaseCtx, cUnit);
                 writeOutputToFile(qbcFile, resultStream);
                 break;
             case "txnqar": // qar represents query after recovery
-                resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
-                        getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+                resultStream = query(cUnit, testFile.getName(), statement);
                 File qarFile = new File(actualPath + File.separator
                         + testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_" + cUnit.getName()
                         + "_qar.adm");
@@ -975,7 +933,7 @@
                 break;
             case "txneu": // eu represents erroneous update
                 try {
-                    executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } catch (Exception e) {
                     // An exception is expected.
                     failed = true;
@@ -1002,7 +960,7 @@
                 break;
             case "errddl": // a ddlquery that expects error
                 try {
-                    executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+                    executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
                 } catch (Exception e) {
                     // expected error happens
                     failed = true;
@@ -1225,6 +1183,7 @@
                 if (testFile.getName().startsWith(DIAGNOSE)) {
                     LOGGER.info("Diagnostic output: {}", IOUtils.toString(resultStream, StandardCharsets.UTF_8));
                 } else {
+                    LOGGER.info("Unexpected output: {}", IOUtils.toString(resultStream, StandardCharsets.UTF_8));
                     Assert.fail("no result file for " + testFile.toString() + "; queryCount: " + queryCount
                             + ", filectxs.size: " + numResultFiles);
                 }
@@ -1239,38 +1198,28 @@
     public void executeQuery(OutputFormat fmt, String statement, Map<String, Object> variableCtx, String reqType,
             File testFile, File expectedResultFile, File actualResultFile, MutableInt queryCount, int numResultFiles,
             List<Parameter> params, ComparisonEnum compare) throws Exception {
-        InputStream resultStream = null;
-        if (testFile.getName().endsWith("aql")) {
-            if (reqType.equalsIgnoreCase("query")) {
-                resultStream = executeQuery(statement, fmt, getEndpoint(Servlets.AQL_QUERY), params);
-            } else {
-                final URI endpoint = getEndpoint(Servlets.AQL);
-                if (reqType.equalsIgnoreCase("async")) {
-                    resultStream = executeAnyAQLAsync(statement, false, fmt, endpoint, variableCtx);
-                } else if (reqType.equalsIgnoreCase("deferred")) {
-                    resultStream = executeAnyAQLAsync(statement, true, fmt, endpoint, variableCtx);
-                }
-                Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), resultStream);
-            }
+        String delivery = DELIVERY_IMMEDIATE;
+        if (reqType.equalsIgnoreCase("async")) {
+            delivery = DELIVERY_ASYNC;
+        } else if (reqType.equalsIgnoreCase("deferred")) {
+            delivery = DELIVERY_DEFERRED;
+        }
+        URI uri = testFile.getName().endsWith("aql") ? getEndpoint(Servlets.QUERY_AQL)
+                : getEndpoint(Servlets.QUERY_SERVICE);
+        boolean isJsonEncoded = isJsonEncoded(extractHttpRequestType(statement));
+        InputStream resultStream;
+        if (DELIVERY_IMMEDIATE.equals(delivery)) {
+            resultStream =
+                    executeQueryService(statement, fmt, uri, params, isJsonEncoded, null, isCancellable(reqType));
+            resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
+                    : ResultExtractor.extract(resultStream);
         } else {
-            String delivery = DELIVERY_IMMEDIATE;
-            if (reqType.equalsIgnoreCase("async")) {
-                delivery = DELIVERY_ASYNC;
-            } else if (reqType.equalsIgnoreCase("deferred")) {
-                delivery = DELIVERY_DEFERRED;
-            }
-            final URI uri = getEndpoint(Servlets.QUERY_SERVICE);
-            if (DELIVERY_IMMEDIATE.equals(delivery)) {
-                resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
-                resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
-                        : ResultExtractor.extract(resultStream);
-            } else {
-                String handleVar = getHandleVariable(statement);
-                resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
-                String handle = ResultExtractor.extractHandle(resultStream);
-                Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
-                variableCtx.put(handleVar, handle);
-            }
+            String handleVar = getHandleVariable(statement);
+            resultStream = executeQueryService(statement, fmt, uri,
+                    upsertParam(params, "mode", ParameterTypeEnum.STRING, delivery), isJsonEncoded);
+            String handle = ResultExtractor.extractHandle(resultStream);
+            Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
+            variableCtx.put(handleVar, toQueryServiceHandle(handle));
         }
         if (actualResultFile == null) {
             if (testFile.getName().startsWith(DIAGNOSE)) {
@@ -1441,7 +1390,16 @@
     }
 
     public InputStream executeSqlppUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
-        InputStream resultStream = executeQueryService(statement, getEndpoint(Servlets.QUERY_SERVICE), outputFormat);
+        return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP));
+    }
+
+    private InputStream executeAqlUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
+        return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(AQL));
+    }
+
+    private InputStream executeUpdateOrDdl(String statement, OutputFormat outputFormat, URI serviceUri)
+            throws Exception {
+        InputStream resultStream = executeQueryService(statement, serviceUri, outputFormat);
         return ResultExtractor.extract(resultStream);
     }
 
@@ -1518,18 +1476,49 @@
         return Optional.empty();
     }
 
-    protected static List<Parameter> extractParameters(String statement) {
+    public static List<Parameter> extractParameters(String statement) {
         List<Parameter> params = new ArrayList<>();
         final Matcher m = HTTP_PARAM_PATTERN.matcher(statement);
         while (m.find()) {
             final Parameter param = new Parameter();
-            param.setName(m.group(1));
-            param.setValue(m.group(2));
+            String name = m.group("name");
+            param.setName(name);
+            String value = m.group("value");
+            param.setValue(value);
+            String type = m.group("type");
+            if (type != null) {
+                try {
+                    param.setType(ParameterTypeEnum.fromValue(type.toLowerCase()));
+                } catch (IllegalArgumentException e) {
+                    throw new IllegalArgumentException(
+                            String.format("Invalid type '%s' specified for parameter '%s'", type, name));
+                }
+            }
             params.add(param);
         }
         return params;
     }
 
+    private static String extractHttpRequestType(String statement) {
+        Matcher m = HTTP_REQUEST_TYPE.matcher(statement);
+        return m.find() ? m.group(1) : null;
+    }
+
+    private static boolean isJsonEncoded(String httpRequestType) throws Exception {
+        if (httpRequestType == null || httpRequestType.isEmpty()) {
+            return true;
+        }
+        switch (httpRequestType.trim()) {
+            case HttpUtil.ContentType.JSON:
+            case HttpUtil.ContentType.APPLICATION_JSON:
+                return true;
+            case HttpUtil.ContentType.APPLICATION_X_WWW_FORM_URLENCODED:
+                return false;
+            default:
+                throw new Exception("Invalid value for http request type: " + httpRequestType);
+        }
+    }
+
     protected static Predicate<Integer> extractStatusCodePredicate(String statement) {
         List<Integer> codes = new ArrayList<>();
         final Matcher m = HTTP_STATUSCODE_PATTERN.matcher(statement);
@@ -1584,6 +1573,7 @@
         List<Parameter> params = new ArrayList<>();
         Parameter node = new Parameter();
         node.setName("node");
+        node.setType(ParameterTypeEnum.STRING);
         node.setValue(nodeId);
         params.add(node);
         InputStream executeJSON = executeJSON(fmt, "POST", URI.create("http://localhost:16001" + endpoint), params);
@@ -1642,7 +1632,8 @@
                     }
                 } catch (Exception e) {
                     numOfErrors++;
-                    boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount);
+                    boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount,
+                            testCaseCtx.isSourceLocationExpected(cUnit));
                     if (unexpected) {
                         LOGGER.error("testFile {} raised an unexpected exception", testFile, e);
                         if (failedGroup != null) {
@@ -1688,22 +1679,41 @@
         throw new Exception("Test \"" + testFile + "\" FAILED!", e);
     }
 
-    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount) {
+    protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount,
+            boolean expectedSourceLoc) {
         String expectedError = null;
         if (expectedErrors.size() < numOfErrors) {
             return true;
         } else {
             // Get the expected exception
             expectedError = expectedErrors.get(numOfErrors - 1);
-            if (e.toString().contains(expectedError)) {
-                return false;
-            } else {
+            String actualError = e.toString();
+            if (!actualError.contains(expectedError)) {
                 LOGGER.error("Expected to find the following in error text: +++++{}+++++", expectedError);
                 return true;
             }
+            if (expectedSourceLoc && !containsSourceLocation(actualError)) {
+                LOGGER.error("Expected to find source location \"{}, {}\" in error text: +++++{}+++++",
+                        ERR_MSG_SRC_LOC_LINE_REGEX, ERR_MSG_SRC_LOC_COLUMN_REGEX, actualError);
+                return true;
+            }
+            return false;
         }
     }
 
+    private static final String ERR_MSG_SRC_LOC_LINE_REGEX = "in line \\d+";
+    private static final Pattern ERR_MSG_SRC_LOC_LINE_PATTERN =
+            Pattern.compile(ERR_MSG_SRC_LOC_LINE_REGEX, Pattern.CASE_INSENSITIVE);
+
+    private static final String ERR_MSG_SRC_LOC_COLUMN_REGEX = "at column \\d+";
+    private static final Pattern ERR_MSG_SRC_LOC_COLUMN_PATTERN =
+            Pattern.compile(ERR_MSG_SRC_LOC_COLUMN_REGEX, Pattern.CASE_INSENSITIVE);
+
+    private boolean containsSourceLocation(String errorMessage) {
+        Matcher lineMatcher = ERR_MSG_SRC_LOC_LINE_PATTERN.matcher(errorMessage);
+        return lineMatcher.find() && ERR_MSG_SRC_LOC_COLUMN_PATTERN.matcher(errorMessage).find(lineMatcher.end());
+    }
+
     private static File getTestCaseQueryBeforeCrashFile(String actualPath, TestCaseContext testCaseCtx,
             CompilationUnit cUnit) {
         return new File(
@@ -1779,13 +1789,14 @@
                 LOGGER.info("Last test left some garbage. Dropping dataverses: " + StringUtils.join(toBeDropped, ','));
                 StringBuilder dropStatement = new StringBuilder();
                 for (String dv : toBeDropped) {
+                    dropStatement.setLength(0);
                     dropStatement.append("drop dataverse ");
                     dropStatement.append(dv);
                     dropStatement.append(";\n");
+                    resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE),
+                            OutputFormat.CLEAN_JSON);
+                    ResultExtractor.extract(resultStream);
                 }
-                resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE),
-                        OutputFormat.CLEAN_JSON);
-                ResultExtractor.extract(resultStream);
             }
         } catch (Throwable th) {
             th.printStackTrace();
@@ -1851,6 +1862,7 @@
         Stream.of("partition", "host", "port").forEach(arg -> {
             Parameter p = new Parameter();
             p.setName(arg);
+            p.setType(ParameterTypeEnum.STRING);
             parameters.add(p);
         });
         parameters.get(0).setValue(partition);
@@ -1900,8 +1912,9 @@
         }
     }
 
-    abstract static class TestLoop extends Exception {
+    public abstract static class TestLoop extends Exception {
 
+        private static final long serialVersionUID = 1L;
         private final String target;
 
         TestLoop(String target) {
@@ -1911,6 +1924,7 @@
         static TestLoop createLoop(String target, final int count) {
             LOGGER.info("Starting loop '" + count + " times back to '" + target + "'...");
             return new TestLoop(target) {
+                private static final long serialVersionUID = 1L;
                 int remainingLoops = count;
 
                 @Override
@@ -1926,6 +1940,7 @@
         static TestLoop createLoop(String target, long duration, TimeUnit unit) {
             LOGGER.info("Starting loop for " + unit.toSeconds(duration) + "s back to '" + target + "'...");
             return new TestLoop(target) {
+                private static final long serialVersionUID = 1L;
                 long endTime = unit.toMillis(duration) + System.currentTimeMillis();
 
                 @Override
@@ -1948,4 +1963,19 @@
     private static boolean isCancellable(String type) {
         return !NON_CANCELLABLE.contains(type);
     }
+
+    private InputStream query(CompilationUnit cUnit, String testFile, String statement) throws Exception {
+        final URI uri = getQueryServiceUri(testFile);
+        final InputStream inputStream = executeQueryService(statement, OutputFormat.forCompilationUnit(cUnit), uri,
+                cUnit.getParameter(), true, null, false);
+        return ResultExtractor.extract(inputStream);
+    }
+
+    private URI getQueryServiceUri(String extension) throws URISyntaxException {
+        return extension.endsWith(AQL) ? getEndpoint(Servlets.QUERY_AQL) : getEndpoint(Servlets.QUERY_SERVICE);
+    }
+
+    private static String toQueryServiceHandle(String handle) {
+        return handle.replace("/aql/", "/service/");
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
index 66a34ff..a7a7fce 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
@@ -23,15 +23,29 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collections;
 import java.util.Enumeration;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
+import org.apache.asterix.api.http.server.QueryServiceServlet;
+import org.apache.asterix.app.translator.RequestParameters;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
 import org.apache.commons.compress.utils.IOUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.util.file.FileUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.TextNode;
+
 public final class TestHelper {
 
     private static final String TEST_DIR_BASE_PATH = System.getProperty("user.dir") + File.separator + "target";
@@ -81,4 +95,53 @@
             }
         }
     }
+
+    public static Map<String, IAObject> readStatementParameters(String statement) throws IOException {
+        List<TestCase.CompilationUnit.Parameter> parameterList = TestExecutor.extractParameters(statement);
+        if (parameterList.isEmpty()) {
+            return Collections.emptyMap();
+        }
+        Map<String, JsonNode> stmtParams = new HashMap<>();
+        ObjectMapper om = createObjectMapper();
+        for (TestCase.CompilationUnit.Parameter param : parameterList) {
+            String paramName = param.getName();
+            JsonNode paramJsonValue;
+            ParameterTypeEnum paramType = param.getType();
+            if (paramType == null) {
+                paramType = ParameterTypeEnum.STRING;
+            }
+            String paramValue = param.getValue();
+            switch (paramType) {
+                case STRING:
+                    paramJsonValue = TextNode.valueOf(paramValue);
+                    break;
+                case JSON:
+                    paramJsonValue = om.readTree(paramValue);
+                    break;
+                default:
+                    throw new IllegalArgumentException(String.valueOf(paramType));
+
+            }
+            String name = QueryServiceServlet.extractStatementParameterName(paramName);
+            if (name != null) {
+                stmtParams.put(name, paramJsonValue);
+            } else if (QueryServiceServlet.Parameter.ARGS.str().equals(paramName)) {
+                if (paramJsonValue.isArray()) {
+                    for (int i = 0, ln = paramJsonValue.size(); i < ln; i++) {
+                        stmtParams.put(String.valueOf(i + 1), paramJsonValue.get(i));
+                    }
+                }
+            }
+        }
+
+        return RequestParameters.deserializeParameterValues(RequestParameters.serializeParameterValues(stmtParams));
+    }
+
+    private static ObjectMapper createObjectMapper() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        objectMapper.enable(SerializationFeature.INDENT_OUTPUT);
+        objectMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
+        objectMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        return objectMapper;
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
new file mode 100644
index 0000000..c34f5a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt32;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt32ValueGenerator implements IFieldValueGenerator<AInt32> {
+    int counter = 0;
+
+    @Override
+    public AInt32 next() {
+        return new AInt32(counter++);
+    }
+
+    @Override
+    public void reset() {
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
new file mode 100644
index 0000000..b860737
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt64;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt64ValueGenerator implements IFieldValueGenerator<AInt64> {
+    long counter = 0L;
+
+    @Override
+    public AInt64 next() {
+        return new AInt64(counter++);
+    }
+
+    @Override
+    public void reset() {
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
new file mode 100644
index 0000000..a0ed26e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -0,0 +1,577 @@
+/*
+ * 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.asterix.test.dataflow;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.app.bootstrap.TestNodeController;
+import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
+import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.IndexCheckpoint;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.TransactionOptions;
+import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
+import org.apache.asterix.test.common.TestHelper;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class CheckpointInSecondaryIndexTest {
+    static final int REPREAT_TEST_COUNT = 1;
+
+    @Parameterized.Parameters
+    public static List<Object[]> data() {
+        return Arrays.asList(new Object[REPREAT_TEST_COUNT][0]);
+    }
+
+    private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
+    private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
+            new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
+    private static final GenerationFunction[] RECORD_GEN_FUNCTION =
+            { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
+    private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
+    private static final ARecordType META_TYPE = null;
+    private static final GenerationFunction[] META_GEN_FUNCTION = null;
+    private static final boolean[] UNIQUE_META_FIELDS = null;
+    private static final int[] KEY_INDEXES = { 0 };
+    private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
+    private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
+    private static final int RECORDS_PER_COMPONENT = 500;
+    private static final int DATASET_ID = 101;
+    private static final String DATAVERSE_NAME = "TestDV";
+    private static final String DATASET_NAME = "TestDS";
+    private static final String INDEX_NAME = "TestIdx";
+    private static final String DATA_TYPE_NAME = "DUMMY";
+    private static final String NODE_GROUP_NAME = "DEFAULT";
+    private static final IndexType INDEX_TYPE = IndexType.BTREE;
+    private static final IFieldValueGenerator[] SECONDARY_INDEX_VALUE_GENERATOR =
+            { new AInt64ValueGenerator(), new AInt32ValueGenerator() };
+    private static final List<List<String>> INDEX_FIELD_NAMES =
+            Arrays.asList(Arrays.asList(RECORD_TYPE.getFieldNames()[1]));
+    private static final List<Integer> INDEX_FIELD_INDICATORS = Arrays.asList(Index.RECORD_INDICATOR);
+    private static final List<IAType> INDEX_FIELD_TYPES = Arrays.asList(BuiltinType.AINT64);
+    private static final StorageComponentProvider storageManager = new StorageComponentProvider();
+    private static TestNodeController nc;
+    private static NCAppRuntimeContext ncAppCtx;
+    private static IDatasetLifecycleManager dsLifecycleMgr;
+    private static Dataset dataset;
+    private static Index secondaryIndex;
+    private static ITransactionContext txnCtx;
+    private static TestLsmBtree primaryLsmBtree;
+    private static TestLsmBtree secondaryLsmBtree;
+    private static PrimaryIndexInfo primaryIndexInfo;
+    private static IHyracksTaskContext taskCtx;
+    private static IIndexDataflowHelper primaryIndexDataflowHelper;
+    private static IIndexDataflowHelper secondaryIndexDataflowHelper;
+    private static LSMInsertDeleteOperatorNodePushable insertOp;
+    private static LSMIndexBulkLoadOperatorNodePushable indexLoadOp;
+    private static IHyracksTaskContext loadTaskCtx;
+    private static SecondaryIndexInfo secondaryIndexInfo;
+    private static Actor actor;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        System.out.println("SetUp: ");
+        TestHelper.deleteExistingInstanceFiles();
+        String configPath = System.getProperty("user.dir") + File.separator + "src" + File.separator + "test"
+                + File.separator + "resources" + File.separator + "cc-multipart.conf";
+        nc = new TestNodeController(configPath, false);
+        nc.init();
+        ncAppCtx = nc.getAppRuntimeContext();
+        dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        System.out.println("TearDown");
+        nc.deInit();
+        TestHelper.deleteExistingInstanceFiles();
+    }
+
+    @Before
+    public void createIndex() throws Exception {
+        List<List<String>> partitioningKeys = new ArrayList<>();
+        partitioningKeys.add(Collections.singletonList("key"));
+        dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME,
+                NODE_GROUP_NAME, NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null,
+                        PartitioningStrategy.HASH, partitioningKeys, null, null, null, false, null),
+                null, DatasetType.INTERNAL, DATASET_ID, 0);
+        secondaryIndex = new Index(DATAVERSE_NAME, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
+                INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0);
+        taskCtx = null;
+        primaryIndexDataflowHelper = null;
+        secondaryIndexDataflowHelper = null;
+        primaryLsmBtree = null;
+        insertOp = null;
+        JobId jobId = nc.newJobId();
+        txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(jobId),
+                new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
+        actor = null;
+        taskCtx = nc.createTestContext(jobId, 0, false);
+        primaryIndexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+                KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+        IndexDataflowHelperFactory iHelperFactory =
+                new IndexDataflowHelperFactory(nc.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+        primaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+        primaryIndexDataflowHelper.open();
+        primaryLsmBtree = (TestLsmBtree) primaryIndexDataflowHelper.getIndexInstance();
+        primaryIndexDataflowHelper.close();
+        // This pipeline skips the secondary index
+        insertOp = nc.getInsertPipeline(taskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, storageManager, null).getLeft();
+        actor = new Actor("player");
+        // allow all operations
+        StorageTestUtils.allowAllOps(primaryLsmBtree);
+        actor.add(new Request(Request.Action.INSERT_OPEN));
+    }
+
+    @After
+    public void destroyIndex() throws Exception {
+        Request close = new Request(Request.Action.INSERT_CLOSE);
+        actor.add(close);
+        close.await();
+        nc.getTransactionManager().commitTransaction(txnCtx.getTxnId());
+        if (secondaryIndexDataflowHelper != null) {
+            secondaryIndexDataflowHelper.destroy();
+        }
+        primaryIndexDataflowHelper.destroy();
+        actor.stop();
+    }
+
+    @Test
+    public void testCheckpointUpdatedWhenSecondaryIsEmpty() throws Exception {
+        try {
+            // create secondary
+            createSecondaryIndex();
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // and that secondary index is empty
+            Assert.assertTrue(secondaryLsmBtree.isCurrentMutableComponentEmpty());
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+
+            // ensure secondary doesn't have a component
+            Assert.assertEquals(0, secondaryLsmBtree.getDiskComponents().size());
+            // ensure that current memory component index match
+            Assert.assertEquals(secondaryLsmBtree.getCurrentMemoryComponentIndex(),
+                    primaryLsmBtree.getCurrentMemoryComponentIndex());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            // secondary ref
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    private void createSecondaryIndex()
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        SecondaryIndexInfo secondaryIndexInfo =
+                nc.createSecondaryIndex(primaryIndexInfo, secondaryIndex, storageManager, 0);
+        IndexDataflowHelperFactory iHelperFactory =
+                new IndexDataflowHelperFactory(nc.getStorageManager(), secondaryIndexInfo.getFileSplitProvider());
+        secondaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+        secondaryIndexDataflowHelper.open();
+        secondaryLsmBtree = (TestLsmBtree) secondaryIndexDataflowHelper.getIndexInstance();
+        secondaryIndexDataflowHelper.close();
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsSingleComponent() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsTwoComponents() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            Assert.assertEquals(2, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            id = (LSMComponentId) primaryDiskComponent.getId();
+            min = id.getMaxId();
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsEmpty() throws Exception {
+        try {
+            // ensure primary has no component
+            Assert.assertEquals(0, primaryLsmBtree.getDiskComponents().size());
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(),
+                    latestPrimaryCheckpoint.getLastComponentId());
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    @Test
+    public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsNotEmpty() throws Exception {
+        try {
+            // create secondary
+            actor.add(new Request(Request.Action.INSERT_PATCH));
+            ensureDone(actor);
+            // search now and ensure partition 0 has all the records
+            StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+            // flush
+            actor.add(new Request(Request.Action.FLUSH_DATASET));
+            ensureDone(actor);
+            // ensure primary has a component
+            Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+            // ensure both checkpoint files has the same component id as the last flushed component id
+            ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+            LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+            long min = id.getMinId();
+            // primary ref
+            Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+            fileManagerField.setAccessible(true); //Make it accessible so you can access it
+            ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+            final ResourceReference primaryRef = ResourceReference
+                    .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+            IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            createSecondaryIndex();
+            JobId jobId = nc.newJobId();
+            loadTaskCtx = nc.createTestContext(jobId, 0, false);
+            Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+                    nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                            KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+            indexLoadOp = infoAndOp.getRight();
+            secondaryIndexInfo = infoAndOp.getLeft();
+            actor.add(new Request(Request.Action.LOAD_OPEN));
+            actor.add(new Request(Request.Action.LOAD_CLOSE));
+            ensureDone(actor);
+            latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+            ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+            final ResourceReference secondaryRef = ResourceReference.of(
+                    secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+            IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+            IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+            Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+        return ncAppCtx.getIndexCheckpointManagerProvider();
+    }
+
+    private void ensureDone(Actor actor) throws InterruptedException {
+        Request req = new Request(Request.Action.DUMMY);
+        actor.add(req);
+        req.await();
+    }
+
+    private static class Request {
+        enum Action {
+            DUMMY,
+            INSERT_OPEN,
+            LOAD_OPEN,
+            INSERT_PATCH,
+            INDEX_LOAD_PATCH,
+            FLUSH_DATASET,
+            INSERT_CLOSE,
+            LOAD_CLOSE,
+        }
+
+        private final Action action;
+        private volatile boolean done;
+
+        public Request(Action action) {
+            this.action = action;
+            done = false;
+        }
+
+        synchronized void complete() {
+            done = true;
+            notifyAll();
+        }
+
+        synchronized void await() throws InterruptedException {
+            while (!done) {
+                wait();
+            }
+        }
+    }
+
+    public class Actor extends SingleThreadEventProcessor<Request> {
+        private final RecordTupleGenerator primaryInsertTupleGenerator;
+        private final FrameTupleAppender tupleAppender;
+
+        public Actor(String name) throws HyracksDataException {
+            super(name);
+            primaryInsertTupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            tupleAppender = new FrameTupleAppender(new VSizeFrame(taskCtx));
+        }
+
+        @Override
+        protected void handle(Request req) throws Exception {
+            try {
+                switch (req.action) {
+                    case FLUSH_DATASET:
+                        if (tupleAppender.getTupleCount() > 0) {
+                            tupleAppender.write(insertOp, true);
+                        }
+                        dsLifecycleMgr.flushDataset(dataset.getDatasetId(), false);
+                        break;
+                    case INSERT_CLOSE:
+                        insertOp.close();
+                        break;
+                    case INSERT_OPEN:
+                        insertOp.open();
+                        break;
+                    case LOAD_OPEN:
+                        indexLoadOp.open();
+                        break;
+                    case LOAD_CLOSE:
+                        indexLoadOp.close();
+                        break;
+                    case INSERT_PATCH:
+                        for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+                            ITupleReference tuple = primaryInsertTupleGenerator.next();
+                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
+                        }
+                        if (tupleAppender.getTupleCount() > 0) {
+                            tupleAppender.write(insertOp, true);
+                        }
+                        StorageTestUtils.waitForOperations(primaryLsmBtree);
+                        break;
+                    case INDEX_LOAD_PATCH:
+                        TupleGenerator secondaryLoadTupleGenerator =
+                                new TupleGenerator(SECONDARY_INDEX_VALUE_GENERATOR, secondaryIndexInfo.getSerdes(), 0);
+                        FrameTupleAppender secondaryTupleAppender = new FrameTupleAppender(new VSizeFrame(loadTaskCtx));
+                        for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+                            ITupleReference tuple = secondaryLoadTupleGenerator.next();
+                            DataflowUtils.addTupleToFrame(secondaryTupleAppender, tuple, indexLoadOp);
+                        }
+                        if (secondaryTupleAppender.getTupleCount() > 0) {
+                            secondaryTupleAppender.write(indexLoadOp, true);
+                        }
+                        break;
+                    default:
+                        break;
+                }
+            } catch (Throwable th) {
+                th.printStackTrace();
+                throw th;
+            } finally {
+                req.complete();
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
index 9ef531e..5aa1b36 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
@@ -20,16 +20,18 @@
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Predicate;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TransactionOptions;
@@ -50,9 +52,11 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.test.support.TestUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -72,6 +76,7 @@
     private static ITransactionContext txnCtx;
     private static LSMInsertDeleteOperatorNodePushable insertOp;
     private static final int PARTITION = 0;
+    private static String indexPath;
 
     @BeforeClass
     public static void setUp() throws Exception {
@@ -106,6 +111,7 @@
         txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                 new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
         insertOp = StorageTestUtils.getInsertPipeline(nc, ctx);
+        indexPath = indexDataflowHelper.getResource().getPath();
     }
 
     @After
@@ -119,7 +125,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -145,17 +151,28 @@
             Assert.assertTrue(memComponents.get(lsmBtree.getCurrentMemoryComponentIndex()).isModified());
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
                     StorageTestUtils.TOTAL_NUM_OF_RECORDS - StorageTestUtils.RECORDS_PER_COMPONENT);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -176,7 +193,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -202,8 +219,14 @@
             Assert.assertTrue(memComponents.get(lsmBtree.getCurrentMemoryComponentIndex()).isModified());
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(memoryComponentsPredicate);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -227,9 +250,14 @@
             StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
             // rollback the last disk component
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -247,7 +275,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -277,8 +305,14 @@
             firstSearcher.waitUntilEntered();
             // now that we enetered, we will rollback
             ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            ILSMComponentId next =
+                    dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             // rollback a memory component
             lsmAccessor.deleteComponents(
                     c -> (c instanceof ILSMMemoryComponent && ((ILSMMemoryComponent) c).isModified()));
@@ -297,10 +331,14 @@
             // wait till firstSearcher enter the components
             secondSearcher.waitUntilEntered();
             lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-
-            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-            ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).refresh();
+            next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+            flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+            flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+            lsmAccessor.getOpContext().setParameters(flushMap);
             DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
             lsmAccessor.deleteComponents(pred);
             // now that the rollback has completed, we will unblock the search
@@ -321,7 +359,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -376,7 +414,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -407,11 +445,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // now that we enetered, we will rollback
             Rollerback rollerback = new Rollerback(lsmBtree, new DiskComponentLsnPredicate(lsn));
@@ -439,7 +477,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -498,7 +536,7 @@
             // allow all operations
             StorageTestUtils.allowAllOps(lsmBtree);
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -559,7 +597,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -590,11 +628,11 @@
             // select the components to merge... the last three
             int numMergedComponents = 3;
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -629,7 +667,7 @@
             StorageTestUtils.allowAllOps(lsmBtree);
             lsmBtree.clearMergeCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+            RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -659,12 +697,12 @@
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             // select the components to merge... the last three
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
-            long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+            long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
             int numMergedComponents = 3;
             for (int i = 0; i < numMergedComponents; i++) {
                 mergedComponents.add(diskComponents.get(i));
             }
-            mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+            mergeAccessor.scheduleMerge(mergedComponents);
             merger.waitUntilCount(1);
             // we will block search
             lsmBtree.clearSearchCallbacks();
@@ -703,8 +741,15 @@
                 public void run() {
                     ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
                     try {
-                        dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
-                        ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+                        dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath)
+                                .refresh();
+                        ILSMComponentId next = dsLifecycleMgr
+                                .getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION, indexPath).getId();
+                        long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+                        Map<String, Object> flushMap = new HashMap<>();
+                        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+                        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+                        lsmAccessor.getOpContext().setParameters(flushMap);
                         lsmAccessor.deleteComponents(predicate);
                     } catch (HyracksDataException e) {
                         failure = e;
@@ -733,9 +778,8 @@
         @Override
         public boolean test(ILSMComponent c) {
             try {
-                return c instanceof ILSMMemoryComponent
-                        || (c instanceof ILSMDiskComponent && AbstractLSMIOOperationCallback
-                                .getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
+                return c instanceof ILSMMemoryComponent || (c instanceof ILSMDiskComponent
+                        && LSMIOOperationCallback.getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
             } catch (HyracksDataException e) {
                 e.printStackTrace();
                 return false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
index ecc14b1..060245a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConnectorDescriptorWithMessagingTest.java
@@ -105,7 +105,7 @@
                 message.getBuffer().clear();
                 message.getBuffer().put(MessagingFrameTupleAppender.ACK_REQ_FEED_MESSAGE);
                 message.getBuffer().flip();
-                partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+                partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
                 for (TestFrameWriter writer : recipients) {
                     Assert.assertEquals(writer.nextFrameCount(), 2);
                     fta.reset(writer.getLastFrame());
@@ -118,7 +118,7 @@
                 message.getBuffer().clear();
                 message.getBuffer().put(MessagingFrameTupleAppender.NULL_FEED_MESSAGE);
                 message.getBuffer().flip();
-                partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+                partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
                 for (TestFrameWriter writer : recipients) {
                     Assert.assertEquals(writer.nextFrameCount(), 3);
                     fta.reset(writer.getLastFrame());
@@ -171,7 +171,7 @@
             for (IFrameWriter writer : partitionWriterFactory.getWriters().values()) {
                 recipients.add((TestFrameWriter) writer);
             }
-            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
             for (TestFrameWriter writer : recipients) {
                 Assert.assertEquals(writer.nextFrameCount(), 1);
                 fta.reset(writer.getLastFrame());
@@ -183,7 +183,7 @@
             message.getBuffer().clear();
             message.getBuffer().put(MessagingFrameTupleAppender.ACK_REQ_FEED_MESSAGE);
             message.getBuffer().flip();
-            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
             for (TestFrameWriter writer : recipients) {
                 Assert.assertEquals(writer.nextFrameCount(), 2);
                 fta.reset(writer.getLastFrame());
@@ -195,7 +195,7 @@
             message.getBuffer().clear();
             message.getBuffer().put(MessagingFrameTupleAppender.NULL_FEED_MESSAGE);
             message.getBuffer().flip();
-            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
             for (TestFrameWriter writer : recipients) {
                 Assert.assertEquals(writer.nextFrameCount(), 3);
                 fta.reset(writer.getLastFrame());
@@ -266,7 +266,7 @@
                 tuple = ttg.next();
             }
             partitioner.nextFrame(frame.getBuffer());
-            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
             Assert.assertEquals(1, partitionWriterFactory.getWriters().get(0).nextFrameCount());
             Assert.assertEquals(2, partitionWriterFactory.getWriters().get(1).nextFrameCount());
             Assert.assertEquals(1, partitionWriterFactory.getWriters().get(2).nextFrameCount());
@@ -325,7 +325,7 @@
                 appender.append(tuple);
             }
             partitioner.nextFrame(frame.getBuffer());
-            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);;
+            partitioner.flush(ITracer.NONE, null, ITraceCategoryRegistry.CATEGORIES_NONE, null);
             Assert.assertEquals(partitionWriterFactory.getWriters().get(0).nextFrameCount(), 1);
             Assert.assertEquals(partitionWriterFactory.getWriters().get(1).nextFrameCount(), 1);
             Assert.assertEquals(partitionWriterFactory.getWriters().get(2).nextFrameCount(), 1);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
index 8bafd32..b618727 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
@@ -22,7 +22,7 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -91,7 +91,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(ILSMMemoryComponent c) throws HyracksDataException {
                 // No Op
             }
         };
@@ -118,7 +118,7 @@
             throws Exception {
         NCAppRuntimeContext ncAppCtx = nc.getAppRuntimeContext();
         IDatasetLifecycleManager dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
-        TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+        RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
         ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                 new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
         boolean failed = false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index dabb32c..79e6368 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.test.dataflow;
 
 import java.io.File;
+import java.lang.reflect.Field;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,7 +29,7 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -60,8 +61,14 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -69,6 +76,7 @@
 import org.junit.Test;
 
 public class LSMFlushRecoveryTest {
+    public static final Logger LOGGER = LogManager.getLogger();
     private static TestNodeController nc;
     private static Dataset dataset;
     private static PrimaryIndexInfo[] primaryIndexInfos;
@@ -84,7 +92,7 @@
     private static IIndexDataflowHelper[] secondaryIndexDataflowHelpers;
     private static ITransactionContext txnCtx;
     private static LSMInsertDeleteOperatorNodePushable[] insertOps;
-    private static TupleGenerator tupleGenerator;
+    private static RecordTupleGenerator tupleGenerator;
 
     private static final int NUM_PARTITIONS = 2;
     private static final int PARTITION_0 = 0;
@@ -153,6 +161,22 @@
     private void initializeNc(boolean cleanUpOnStart) throws Exception {
         nc.init(cleanUpOnStart);
         ncAppCtx = nc.getAppRuntimeContext();
+        // Override the LSMIOScheduler to avoid halting on failure and enable
+        // testing failure scenario in a unit test setting
+        Field ioScheduler = ncAppCtx.getClass().getDeclaredField("lsmIOScheduler");
+        ioScheduler.setAccessible(true);
+        ioScheduler.set(ncAppCtx, new AsynchronousScheduler(ncAppCtx.getServiceContext().getThreadFactory(),
+                new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.error("Scheduler Failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                        LOGGER.warn("IO Operation failed", t);
+                    }
+                }));
         dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
     }
 
@@ -241,21 +265,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore primaryFlushSemaphore = new Semaphore(0);
         primaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        primaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        primaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 primaryFlushSemaphore.release();
             }
         });
@@ -283,21 +307,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore secondaryFlushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 secondaryFlushSemaphore.release();
             }
         });
@@ -335,21 +359,21 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
 
         Semaphore flushSemaphore = new Semaphore(0);
         secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
-        secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+        secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
             @Override
             public void before(Void t) throws HyracksDataException {
 
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Void t) throws HyracksDataException {
                 flushSemaphore.release();
             }
         });
@@ -387,7 +411,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
@@ -400,7 +424,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
 
             }
         });
@@ -454,6 +478,8 @@
             ILSMMemoryComponent primaryMemComponent = primaryIndexes[partitionIndex].getCurrentMemoryComponent();
             ILSMMemoryComponent secondaryMemComponent = secondaryIndexes[partitionIndex].getCurrentMemoryComponent();
             Assert.assertEquals(primaryMemComponent.getId(), secondaryMemComponent.getId());
+            Assert.assertEquals(primaryIndexes[partitionIndex].getCurrentMemoryComponentIndex(),
+                    secondaryIndexes[partitionIndex].getCurrentMemoryComponentIndex());
         }
 
         List<ILSMDiskComponent> primaryDiskComponents = primaryIndexes[partitionIndex].getDiskComponents();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
index 2121327..e2c99b0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
@@ -26,10 +26,9 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
@@ -37,10 +36,7 @@
 import org.apache.asterix.common.transactions.TransactionOptions;
 import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
@@ -113,22 +109,20 @@
             StorageComponentProvider storageManager = new StorageComponentProvider();
             List<List<String>> partitioningKeys = new ArrayList<>();
             partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
-                PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
-                        storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+                PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE,
+                        META_TYPE, null, storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
                 JobId jobId = nc.newJobId();
                 IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, true);
                 ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp =
+                        nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
                 insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                        RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+                RecordTupleGenerator tupleGenerator =
+                        new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+                                RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
                 VSizeFrame frame = new VSizeFrame(ctx);
                 VSizeFrame marker = new VSizeFrame(ctx);
                 FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
@@ -178,9 +172,9 @@
                 nc.newJobId();
                 TestTupleCounterFrameWriter countOp = create(nc.getSearchOutputDesc(KEY_TYPES, RECORD_TYPE, META_TYPE),
                         Collections.emptyList(), Collections.emptyList(), false);
-                IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, dataset, KEY_TYPES, RECORD_TYPE,
-                        META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES, KEY_INDICATORS_LIST,
-                        storageManager);
+                IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, StorageTestUtils.DATASET, KEY_TYPES,
+                        RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES,
+                        KEY_INDICATORS_LIST, storageManager);
                 emptyTupleOp.open();
                 emptyTupleOp.close();
                 Assert.assertEquals(NUM_OF_RECORDS, countOp.getCount());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index 62705cc..a7225a1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -29,8 +29,8 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -302,7 +302,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (allocated) {
                         allocated.setValue(true);
                         allocated.notifyAll();
@@ -339,7 +339,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             synchronized (proceedToScheduleFlush) {
@@ -421,7 +421,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Void t) {
                     synchronized (finishedSchduleFlush) {
                         finishedSchduleFlush.set(true);
                         finishedSchduleFlush.notifyAll();
@@ -478,7 +478,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                     synchronized (recycledPrimary) {
                         recycledPrimary.setValue(true);
                         recycledPrimary.notifyAll();
@@ -519,7 +519,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(ILSMMemoryComponent t) {
                 }
             };
             secondaryLsmBtrees[0].addIoRecycleCallback(secondaryRecycleCallback);
@@ -619,14 +619,14 @@
 
     public class Actor extends SingleThreadEventProcessor<Request> {
         private final int partition;
-        private final TupleGenerator tupleGenerator;
+        private final RecordTupleGenerator tupleGenerator;
         private final VSizeFrame frame;
         private final FrameTupleAppender tupleAppender;
 
         public Actor(String name, int partition) throws HyracksDataException {
             super(name);
             this.partition = partition;
-            tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+            tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
                     RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             frame = new VSizeFrame(taskCtxs[partition]);
             tupleAppender = new FrameTupleAppender(frame);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index e4373f6..61c1fb2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -26,8 +26,8 @@
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -60,8 +60,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
 import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -160,8 +161,8 @@
             // except search
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+                    KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             Searcher firstSearcher = null;
@@ -206,8 +207,8 @@
             // except search
             lsmBtree.clearSearchCallbacks();
             insertOp.open();
-            TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
-                    RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+            RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+                    KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
             VSizeFrame frame = new VSizeFrame(ctx);
             FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
             Searcher firstSearcher = null;
@@ -235,10 +236,11 @@
             // merge all components
             ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             List<ILSMDiskComponent> mergedComponents = new ArrayList<>(lsmBtree.getDiskComponents());
-            BlockingIOOperationCallbackWrapper ioCallback =
-                    new BlockingIOOperationCallbackWrapper(lsmBtree.getIOOperationCallback());
-            mergeAccessor.scheduleMerge(ioCallback, mergedComponents);
-            ioCallback.waitForIO();
+            ILSMIOOperation merge = mergeAccessor.scheduleMerge(mergedComponents);
+            merge.sync();
+            if (merge.getStatus() == LSMIOOperationStatus.FAILURE) {
+                throw HyracksDataException.create(merge.getFailure());
+            }
             // unblock the search
             unblockSearch(lsmBtree);
             // ensure the search got the correct number
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
index 390286a..589e8b2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
@@ -33,9 +33,9 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.DatasetInfo;
@@ -100,7 +100,7 @@
     private StorageTestUtils() {
     }
 
-    static void allowAllOps(TestLsmBtree lsmBtree) {
+    public static void allowAllOps(TestLsmBtree lsmBtree) {
         lsmBtree.clearModifyCallbacks();
         lsmBtree.clearFlushCallbacks();
         lsmBtree.clearSearchCallbacks();
@@ -118,6 +118,12 @@
                 KEY_INDICATORS_LIST, partition);
     }
 
+    public static PrimaryIndexInfo createPrimaryIndex(TestNodeController nc, Dataset dataset, int partition)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, STORAGE_MANAGER, KEY_INDEXES,
+                KEY_INDICATORS_LIST, partition);
+    }
+
     public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx)
             throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
         return getInsertPipeline(nc, ctx, null);
@@ -131,13 +137,27 @@
     }
 
     public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+            Dataset dataset, Index secondaryIndex, IndexOperation op)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex, op).getLeft();
+    }
+
+    public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
             Index secondaryIndex) throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
         return nc.getInsertPipeline(ctx, DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
                 KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
     }
 
-    public static TupleGenerator getTupleGenerator() {
-        return new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
+    public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+            Dataset dataset, Index secondaryIndex)
+            throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+        return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+                KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
+    }
+
+    public static RecordTupleGenerator getTupleGenerator() {
+        return new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
                 UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
     }
 
@@ -146,6 +166,11 @@
         searchAndAssertCount(nc, partition, DATASET, STORAGE_MANAGER, numOfRecords);
     }
 
+    public static void searchAndAssertCount(TestNodeController nc, Dataset dataset, int partition, int numOfRecords)
+            throws HyracksDataException, AlgebricksException {
+        searchAndAssertCount(nc, partition, dataset, STORAGE_MANAGER, numOfRecords);
+    }
+
     public static void searchAndAssertCount(TestNodeController nc, int partition, Dataset dataset,
             StorageComponentProvider storageManager, int numOfRecords)
             throws HyracksDataException, AlgebricksException {
@@ -182,6 +207,11 @@
         flushPartition(dslLifecycleMgr, lsmBtree, DATASET, async);
     }
 
+    public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+            boolean async) throws Exception {
+        flushPartition(dslLifecycleMgr, lsmBtree, dataset, async);
+    }
+
     public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
             boolean async) throws Exception {
         waitForOperations(lsmBtree);
@@ -211,6 +241,11 @@
         flush(dsLifecycleMgr, lsmBtree, DATASET, async);
     }
 
+    public static void flush(IDatasetLifecycleManager dsLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+            boolean async) throws Exception {
+        flush(dsLifecycleMgr, lsmBtree, dataset, async);
+    }
+
     public static void flush(IDatasetLifecycleManager dsLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
             boolean async) throws Exception {
         waitForOperations(lsmBtree);
@@ -240,6 +275,11 @@
             this(nc, partition, DATASET, STORAGE_MANAGER, lsmBtree, numOfRecords);
         }
 
+        public Searcher(TestNodeController nc, Dataset dataset, int partition, TestLsmBtree lsmBtree,
+                int numOfRecords) {
+            this(nc, partition, dataset, STORAGE_MANAGER, lsmBtree, numOfRecords);
+        }
+
         public Searcher(TestNodeController nc, int partition, Dataset dataset, StorageComponentProvider storageManager,
                 TestLsmBtree lsmBtree, int numOfRecords) {
             lsmBtree.addSearchCallback(new ITestOpCallback<Semaphore>() {
@@ -253,7 +293,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
             Callable<Boolean> callable = new Callable<Boolean>() {
@@ -292,7 +332,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
@@ -319,7 +359,7 @@
                 }
 
                 @Override
-                public void after() {
+                public void after(Semaphore t) {
                 }
             });
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index 0c4983a..bcf68b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -21,6 +21,7 @@
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -28,9 +29,14 @@
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
+import org.apache.asterix.transaction.management.runtime.CommitRuntime;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.common.IResourceFactory;
@@ -48,6 +54,19 @@
     }
 
     @Override
+    public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider,
+            int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
+        return new IPushRuntimeFactory() {
+            @Override
+            public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IPushRuntime[] { new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()),
+                        getDatasetId(), primaryKeyFieldPermutation, true,
+                        ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
+            }
+        };
+    }
+
+    @Override
     public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Index index, ARecordType recordType,
             ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties)
             throws AlgebricksException {
@@ -62,6 +81,6 @@
 
     @Override
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        return new TestLsmBtreeIoOpCallbackFactory(getComponentIdGeneratorFactory());
+        return new TestLsmIoOpCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
deleted file mode 100644
index 5852ad9..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.asterix.test.dataflow;
-
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
-
-public class TestLsmBtreeIoOpCallbackFactory extends LSMBTreeIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private static volatile int completedFlushes = 0;
-    private static volatile int completedMerges = 0;
-    private static volatile int rollbackFlushes = 0;
-    private static volatile int rollbackMerges = 0;
-    private static volatile int failedFlushes = 0;
-    private static volatile int failedMerges = 0;
-
-    public TestLsmBtreeIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        completedFlushes = 0;
-        completedMerges = 0;
-        rollbackFlushes = 0;
-        rollbackMerges = 0;
-        // Whenever this is called, it resets the counter
-        // However, the counters for the failed operations are never reset since we expect them
-        // To be always 0
-        return new TestLsmBtreeIoOpCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-
-    public int getTotalFlushes() {
-        return completedFlushes + rollbackFlushes;
-    }
-
-    public int getTotalMerges() {
-        return completedMerges + rollbackMerges;
-    }
-
-    public int getTotalIoOps() {
-        return getTotalFlushes() + getTotalMerges();
-    }
-
-    public int getRollbackFlushes() {
-        return rollbackFlushes;
-    }
-
-    public int getRollbackMerges() {
-        return rollbackMerges;
-    }
-
-    public int getCompletedFlushes() {
-        return completedFlushes;
-    }
-
-    public int getCompletedMerges() {
-        return completedMerges;
-    }
-
-    public static int getFailedFlushes() {
-        return failedFlushes;
-    }
-
-    public static int getFailedMerges() {
-        return failedMerges;
-    }
-
-    public class TestLsmBtreeIoOpCallback extends LSMBTreeIOOperationCallback {
-        private final TestLsmBtree lsmBtree;
-
-        public TestLsmBtreeIoOpCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-                IIndexCheckpointManagerProvider checkpointManagerProvider) {
-            super(index, idGenerator, checkpointManagerProvider);
-            lsmBtree = (TestLsmBtree) index;
-        }
-
-        @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.beforeIoOperationCalled();
-            super.beforeOperation(opCtx);
-            lsmBtree.beforeIoOperationReturned();
-        }
-
-        @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.afterIoOperationCalled();
-            super.afterOperation(opCtx);
-            lsmBtree.afterIoOperationReturned();
-        }
-
-        @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            lsmBtree.afterIoFinalizeCalled();
-            super.afterFinalize(opCtx);
-            synchronized (TestLsmBtreeIoOpCallbackFactory.this) {
-                if (opCtx.getNewComponent() != null) {
-                    if (opCtx.getNewComponent() == EmptyComponent.INSTANCE) {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-                            rollbackFlushes++;
-                        } else {
-                            rollbackMerges++;
-                        }
-                    } else {
-                        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-                            completedFlushes++;
-                        } else {
-                            completedMerges++;
-                        }
-                    }
-                } else {
-                    recordFailure(opCtx.getIoOperationType());
-                }
-                TestLsmBtreeIoOpCallbackFactory.this.notifyAll();
-            }
-            lsmBtree.afterIoFinalizeReturned();
-        }
-
-        @Override
-        public void recycled(ILSMMemoryComponent component, boolean advance) throws HyracksDataException {
-            lsmBtree.recycledCalled(component);
-            super.recycled(component, advance);
-            lsmBtree.recycledReturned(component);
-        }
-
-        @Override
-        public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-            lsmBtree.allocatedCalled(component);
-            super.allocated(component);
-            lsmBtree.allocatedReturned(component);
-        }
-
-        private void recordFailure(LSMIOOperationType opType) {
-            if (opType == LSMIOOperationType.FLUSH) {
-                failedFlushes++;
-            } else {
-                failedMerges++;
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
new file mode 100644
index 0000000..c727f52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
@@ -0,0 +1,215 @@
+/*
+ * 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.asterix.test.dataflow;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class TestLsmIoOpCallbackFactory extends LSMIndexIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private static volatile int completedFlushes = 0;
+    private static volatile int completedMerges = 0;
+    private static volatile int rollbackFlushes = 0;
+    private static volatile int rollbackMerges = 0;
+    private static volatile int failedFlushes = 0;
+    private static volatile int failedMerges = 0;
+
+    public TestLsmIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        super(idGeneratorFactory, datasetInfoProvider);
+    }
+
+    @Override
+    public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        completedFlushes = 0;
+        completedMerges = 0;
+        rollbackFlushes = 0;
+        rollbackMerges = 0;
+        // Whenever this is called, it resets the counter
+        // However, the counters for the failed operations are never reset since we expect them
+        // To be always 0
+        return new TestLsmIoOpCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index, getComponentIdGenerator(),
+                getIndexCheckpointManagerProvider());
+    }
+
+    public int getTotalFlushes() {
+        return completedFlushes + rollbackFlushes;
+    }
+
+    public int getTotalMerges() {
+        return completedMerges + rollbackMerges;
+    }
+
+    public int getTotalIoOps() {
+        return getTotalFlushes() + getTotalMerges();
+    }
+
+    public int getRollbackFlushes() {
+        return rollbackFlushes;
+    }
+
+    public int getRollbackMerges() {
+        return rollbackMerges;
+    }
+
+    public int getCompletedFlushes() {
+        return completedFlushes;
+    }
+
+    public int getCompletedMerges() {
+        return completedMerges;
+    }
+
+    public static int getFailedFlushes() {
+        return failedFlushes;
+    }
+
+    public static int getFailedMerges() {
+        return failedMerges;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("idGeneratorFactory", idGeneratorFactory.toJson(registry));
+        json.set("datasetInfoProvider", datasetInfoProvider.toJson(registry));
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ILSMComponentIdGeneratorFactory idGeneratorFactory =
+                (ILSMComponentIdGeneratorFactory) registry.deserialize(json.get("idGeneratorFactory"));
+        final IDatasetInfoProvider datasetInfoProvider =
+                (IDatasetInfoProvider) registry.deserialize(json.get("datasetInfoProvider"));
+        return new TestLsmIoOpCallbackFactory(idGeneratorFactory, datasetInfoProvider);
+    }
+
+    public class TestLsmIoOpCallback extends LSMIOOperationCallback {
+        private final TestLsmBtree lsmBtree;
+
+        public TestLsmIoOpCallback(DatasetInfo dsInfo, ILSMIndex index, ILSMComponentIdGenerator idGenerator,
+                IIndexCheckpointManagerProvider checkpointManagerProvider) {
+            super(dsInfo, index, idGenerator.getId(), checkpointManagerProvider);
+            lsmBtree = (TestLsmBtree) index;
+        }
+
+        @Override
+        public void scheduled(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.ioScheduledCalled();
+            super.scheduled(op);
+            lsmBtree.ioScheduledReturned();
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.beforeIoOperationCalled();
+            super.beforeOperation(op);
+            lsmBtree.beforeIoOperationReturned();
+        }
+
+        @Override
+        public void afterOperation(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.afterIoOperationCalled();
+            super.afterOperation(op);
+            lsmBtree.afterIoOperationReturned();
+        }
+
+        @Override
+        public void afterFinalize(ILSMIOOperation op) throws HyracksDataException {
+            lsmBtree.afterIoFinalizeCalled();
+            super.afterFinalize(op);
+            synchronized (TestLsmIoOpCallbackFactory.this) {
+                if (op.getNewComponent() != null) {
+                    if (op.getNewComponent() == EmptyComponent.INSTANCE) {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+                            rollbackFlushes++;
+                        } else {
+                            rollbackMerges++;
+                        }
+                    } else {
+                        if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+                            completedFlushes++;
+                        } else {
+                            completedMerges++;
+                        }
+                    }
+                } else {
+                    recordFailure(op.getIOOpertionType());
+                }
+                TestLsmIoOpCallbackFactory.this.notifyAll();
+            }
+            lsmBtree.afterIoFinalizeReturned();
+        }
+
+        @Override
+        public void completed(ILSMIOOperation operation) {
+            try {
+                lsmBtree.ioCompletedCalled();
+                super.completed(operation);
+                lsmBtree.ioCompletedReturned();
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        @Override
+        public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+            lsmBtree.recycledCalled(component);
+            super.recycled(component);
+            lsmBtree.recycledReturned(component);
+        }
+
+        @Override
+        public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+            lsmBtree.allocatedCalled(component);
+            super.allocated(component);
+            lsmBtree.allocatedReturned(component);
+        }
+
+        private void recordFailure(LSMIOOperationType opType) {
+            if (opType == LSMIOOperationType.FLUSH) {
+                failedFlushes++;
+            } else {
+                failedMerges++;
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
index 9a528d3..250c25f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
@@ -60,7 +60,7 @@
         super.triggerScheduleFlush(logRecord);
         synchronized (callbacks) {
             for (ITestOpCallback<Void> callback : callbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
index e6b34b8..c4390fa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTrackerFactory.java
@@ -29,9 +29,15 @@
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.common.IResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class TestPrimaryIndexOperationTrackerFactory extends PrimaryIndexOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -49,12 +55,13 @@
             DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) appCtx.getDatasetLifecycleManager();
             DatasetResource dsr = dslcManager.getDatasetLifecycle(datasetId);
             int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
-            PrimaryIndexOperationTracker opTracker = dslcManager.getOperationTracker(datasetId, partition);
+            PrimaryIndexOperationTracker opTracker =
+                    dslcManager.getOperationTracker(datasetId, partition, resource.getPath());
             if (!(opTracker instanceof TestPrimaryIndexOperationTracker)) {
                 Field opTrackersField = DatasetResource.class.getDeclaredField("datasetPrimaryOpTrackers");
                 opTracker = new TestPrimaryIndexOperationTracker(datasetId, partition,
                         appCtx.getTransactionSubsystem().getLogManager(), dsr.getDatasetInfo(),
-                        dslcManager.getComponentIdGenerator(datasetId, partition));
+                        dslcManager.getComponentIdGenerator(datasetId, partition, resource.getPath()));
                 replaceMapEntry(opTrackersField, dsr, partition, opTracker);
             }
             return opTracker;
@@ -80,4 +87,16 @@
         Map map = (Map) field.get(obj);
         map.put(key, value);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new TestPrimaryIndexOperationTrackerFactory(json.get("datasetId").asInt());
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
index 06b213d..bee2f8d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
@@ -23,7 +23,7 @@
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -68,7 +68,7 @@
     private static IHyracksTaskContext abortCtx;
     private static ITransactionContext abortTxnCtx;
     private static LSMInsertDeleteOperatorNodePushable abortOp;
-    private static TupleGenerator tupleGenerator;
+    private static RecordTupleGenerator tupleGenerator;
 
     @Rule
     public TestRule watcher = new TestMethodTracer();
@@ -199,7 +199,7 @@
             }
 
             @Override
-            public void after() throws HyracksDataException {
+            public void after(Semaphore t) throws HyracksDataException {
                 // manually set the current memory component as modified
                 index.getCurrentMemoryComponent().setModified();
                 throw new HyracksDataException("Fail the job");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
index 0affaf2..726a1bf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/ddl/SecondaryBTreeOperationsHelperTest.java
@@ -89,7 +89,7 @@
             index = metadataProvider.getIndex(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName,
                     primaryIndexName);
             Assert.assertNotNull(index);
-            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider);
+            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
             jobSpecification.getOperatorMap().values().forEach(iOperatorDescriptor -> {
                 Assert.assertFalse(iOperatorDescriptor instanceof AbstractSorterOperatorDescriptor);
             });
@@ -99,7 +99,7 @@
             index = metadataProvider.getIndex(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName,
                     secondaryIndexName);
             Assert.assertNotNull(index);
-            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider);
+            jobSpecification = IndexUtil.buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
             final long numOfSortOperators = jobSpecification.getOperatorMap().values().stream()
                     .filter(op -> op instanceof AbstractSorterOperatorDescriptor).count();
             Assert.assertTrue(numOfSortOperators != 0);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index e7b6271..0751178 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -24,9 +24,11 @@
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
-import java.io.Reader;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.api.java.AsterixJavaClient;
@@ -41,11 +43,13 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.SessionConfig.PlanFormat;
+import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -173,31 +177,28 @@
             Assume.assumeTrue(!skipped);
 
             LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
-            Reader query = new BufferedReader(new InputStreamReader(new FileInputStream(queryFile), "UTF-8"));
+            String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
+            Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
 
             // Forces the creation of actualFile.
             actualFile.getParentFile().mkdirs();
 
-            PrintWriter plan = new PrintWriter(actualFile);
             ILangCompilationProvider provider =
                     queryFile.getName().endsWith("aql") ? aqlCompilationProvider : sqlppCompilationProvider;
             if (extensionLangCompilationProvider != null) {
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            AsterixJavaClient asterix =
-                    new AsterixJavaClient((ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
-                            query, plan, provider, statementExecutorFactory, storageComponentProvider);
-            try {
-                asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
 
+            try (PrintWriter plan = new PrintWriter(actualFile)) {
+                AsterixJavaClient asterix = new AsterixJavaClient(
+                        (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
+                        new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
+                asterix.setStatementParameters(queryParams);
+                asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
             } catch (AsterixException e) {
-                plan.close();
-                query.close();
                 throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
             }
-            plan.close();
-            query.close();
 
             BufferedReader readerActual =
                     new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
index 418282e..e67246a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.test.logging;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
 import java.io.File;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -26,10 +30,9 @@
 import java.util.List;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
 import org.apache.asterix.app.nc.RecoveryManager;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.Checkpoint;
@@ -43,14 +46,12 @@
 import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.test.common.TestHelper;
+import org.apache.asterix.test.dataflow.StorageTestUtils;
 import org.apache.asterix.transaction.management.service.logging.LogManager;
 import org.apache.asterix.transaction.management.service.recovery.AbstractCheckpointManager;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
@@ -60,16 +61,10 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.stubbing.Answer;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.spy;
 
 public class CheckpointingTest {
 
@@ -116,23 +111,21 @@
             nc.init();
             List<List<String>> partitioningKeys = new ArrayList<>();
             partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
             try {
-                nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
-                        KEY_INDICATOR_LIST, 0);
+                nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+                        KEY_INDEXES, KEY_INDICATOR_LIST, 0);
                 JobId jobId = nc.newJobId();
                 IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
                 ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
                 // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp =
+                        nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
                 insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
-                        RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+                RecordTupleGenerator tupleGenerator =
+                        new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
+                                RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
                 VSizeFrame frame = new VSizeFrame(ctx);
                 FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
 
@@ -140,7 +133,7 @@
                 ICheckpointManager checkpointManager = nc.getTransactionSubsystem().getCheckpointManager();
                 LogManager logManager = (LogManager) nc.getTransactionSubsystem().getLogManager();
                 // Number of log files after node startup should be one
-                int numberOfLogFiles = logManager.getLogFileIds().size();
+                int numberOfLogFiles = logManager.getOrderedLogFileIds().size();
                 Assert.assertEquals(1, numberOfLogFiles);
 
                 // Low-water mark LSN
@@ -148,10 +141,10 @@
                 // Low-water mark log file id
                 long initialLowWaterMarkFileId = logManager.getLogFileId(lowWaterMarkLSN);
                 // Initial Low-water mark should be in the only available log file
-                Assert.assertEquals(initialLowWaterMarkFileId, logManager.getLogFileIds().get(0).longValue());
+                Assert.assertEquals(initialLowWaterMarkFileId, logManager.getOrderedLogFileIds().get(0).longValue());
 
                 // Insert records until a new log file is created
-                while (logManager.getLogFileIds().size() == 1) {
+                while (logManager.getOrderedLogFileIds().size() == 1) {
                     ITupleReference tuple = tupleGenerator.next();
                     DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
                 }
@@ -166,9 +159,9 @@
                      * the low-water mark is still in it (i.e. it is still required for
                      * recovery)
                      */
-                    int numberOfLogFilesBeforeCheckpoint = logManager.getLogFileIds().size();
+                    int numberOfLogFilesBeforeCheckpoint = logManager.getOrderedLogFileIds().size();
                     checkpointManager.tryCheckpoint(logManager.getAppendLSN());
-                    int numberOfLogFilesAfterCheckpoint = logManager.getLogFileIds().size();
+                    int numberOfLogFilesAfterCheckpoint = logManager.getOrderedLogFileIds().size();
                     Assert.assertEquals(numberOfLogFilesBeforeCheckpoint, numberOfLogFilesAfterCheckpoint);
 
                     /*
@@ -197,8 +190,9 @@
                 nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx2),
                         new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
                 // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp2 = nc.getInsertPipeline(ctx2, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+                LSMInsertDeleteOperatorNodePushable insertOp2 =
+                        nc.getInsertPipeline(ctx2, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+                                KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
                 insertOp2.open();
                 VSizeFrame frame2 = new VSizeFrame(ctx2);
                 FrameTupleAppender tupleAppender2 = new FrameTupleAppender(frame2);
@@ -208,7 +202,7 @@
 
                     checkpointManager.tryCheckpoint(lowWaterMarkLSN);
                     // Validate initialLowWaterMarkFileId was deleted
-                    for (Long fileId : logManager.getLogFileIds()) {
+                    for (Long fileId : logManager.getOrderedLogFileIds()) {
                         Assert.assertNotEquals(initialLowWaterMarkFileId, fileId.longValue());
                     }
 
@@ -220,6 +214,7 @@
                     }
                 }
                 Thread.UncaughtExceptionHandler h = new Thread.UncaughtExceptionHandler() {
+                    @Override
                     public void uncaughtException(Thread th, Throwable ex) {
                         threadException = true;
                         exception = ex;
@@ -228,7 +223,7 @@
 
                 Thread t = new Thread(() -> {
                     TransactionManager spyTxnMgr = spy((TransactionManager) nc.getTransactionManager());
-                    doAnswer((Answer) i -> {
+                    doAnswer(i -> {
                         stallAbortTxn(Thread.currentThread(), txnCtx, nc.getTransactionSubsystem(),
                                 (TxnId) i.getArguments()[0]);
                         return null;
@@ -302,7 +297,7 @@
                 // Make sure the valid checkout wouldn't force full recovery
                 Assert.assertTrue(validCheckpoint.getMinMCTFirstLsn() >= minFirstLSN);
                 // Add a corrupted (empty) checkpoint file with a timestamp > than current checkpoint
-                Path corruptedCheckpointPath = checkpointManager.getCheckpointPath(validCheckpoint.getTimeStamp() + 1);
+                Path corruptedCheckpointPath = checkpointManager.getCheckpointPath(validCheckpoint.getId() + 1);
                 File corruptedCheckpoint = corruptedCheckpointPath.toFile();
                 corruptedCheckpoint.createNewFile();
                 // Make sure the corrupted checkpoint file was created
@@ -310,11 +305,11 @@
                 // Try to get the latest checkpoint again
                 Checkpoint cpAfterCorruption = checkpointManager.getLatest();
                 // Make sure the valid checkpoint was returned
-                Assert.assertEquals(validCheckpoint.getTimeStamp(), cpAfterCorruption.getTimeStamp());
+                Assert.assertEquals(validCheckpoint.getId(), cpAfterCorruption.getId());
                 // Make sure the corrupted checkpoint file was deleted
                 Assert.assertFalse(corruptedCheckpoint.exists());
                 // Corrupt the valid checkpoint by replacing its content
-                final Path validCheckpointPath = checkpointManager.getCheckpointPath(validCheckpoint.getTimeStamp());
+                final Path validCheckpointPath = checkpointManager.getCheckpointPath(validCheckpoint.getId());
                 File validCheckpointFile = validCheckpointPath.toFile();
                 Assert.assertTrue(validCheckpointFile.exists());
                 // Delete the valid checkpoint file and create it as an empty file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
index 0f6adf6..d5985b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -51,7 +51,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.test.support.TestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -261,8 +260,9 @@
         final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxn);
         final String nodeGroupName = "ng";
+        final String committedNodeGroup = "committed_ng";
+        final List<String> ngNodes = Collections.singletonList("asterix_nc1");
         try {
-            final List<String> ngNodes = Collections.singletonList("asterix_nc1");
             MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup(nodeGroupName, ngNodes));
             MetadataManager.INSTANCE.commitTransaction(mdTxn);
         } finally {
@@ -281,6 +281,9 @@
         int diskComponentsBeforeFlush = index.getDiskComponents().size();
         // lock opTracker to prevent log flusher from triggering flush
         synchronized (opTracker) {
+            final MetadataTransactionContext committedMdTxn = MetadataManager.INSTANCE.beginTransaction();
+            MetadataManager.INSTANCE.addNodegroup(committedMdTxn, new NodeGroup(committedNodeGroup, ngNodes));
+            MetadataManager.INSTANCE.commitTransaction(committedMdTxn);
             opTracker.setFlushOnExit(true);
             opTracker.flushIfNeeded();
             Assert.assertTrue(opTracker.isFlushLogCreated());
@@ -288,7 +291,6 @@
             // make sure force operation will processed
             MetadataManager.INSTANCE.dropNodegroup(mdTxn2, nodeGroupName, false);
             Assert.assertEquals(1, opTracker.getNumActiveOperations());
-            Assert.assertFalse(index.hasFlushRequestForCurrentMutableComponent());
             // release opTracker lock now to allow log flusher to schedule the flush
             InvokeUtil.runWithTimeout(() -> {
                 synchronized (opTracker) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index 3fd59a4..debb3f9 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -23,14 +23,16 @@
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
-import java.io.Reader;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.api.java.AsterixJavaClient;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
@@ -39,10 +41,12 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.logging.log4j.LogManager;
@@ -113,7 +117,9 @@
 
     private static void suiteBuildPerFile(File file, Collection<Object[]> testArgs, String path) {
         if (file.isDirectory() && !file.getName().startsWith(".")) {
-            for (File innerfile : file.listFiles()) {
+            File[] files = file.listFiles();
+            Arrays.sort(files);
+            for (File innerfile : files) {
                 String subdir = innerfile.isDirectory() ? path + innerfile.getName() + SEPARATOR : path;
                 suiteBuildPerFile(innerfile, testArgs, subdir);
             }
@@ -170,32 +176,29 @@
             Assume.assumeTrue(!skipped);
 
             LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
-            Reader query = new BufferedReader(new InputStreamReader(new FileInputStream(queryFile), "UTF-8"));
+            String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
+            Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
 
             LOGGER.info("ACTUAL RESULT FILE: " + actualFile.getAbsolutePath());
 
             // Forces the creation of actualFile.
             actualFile.getParentFile().mkdirs();
 
-            PrintWriter plan = new PrintWriter(actualFile);
             ILangCompilationProvider provider =
                     queryFile.getName().endsWith("aql") ? aqlCompilationProvider : sqlppCompilationProvider;
             if (extensionLangCompilationProvider != null) {
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            AsterixJavaClient asterix =
-                    new AsterixJavaClient((ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
-                            query, plan, provider, statementExecutorFactory, storageComponentProvider);
-            try {
+            try (PrintWriter plan = new PrintWriter(actualFile)) {
+                AsterixJavaClient asterix = new AsterixJavaClient(
+                        (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
+                        new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
+                asterix.setStatementParameters(queryParams);
                 asterix.compile(true, false, false, true, true, false, false);
             } catch (AlgebricksException e) {
-                plan.close();
-                query.close();
                 throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
             }
-            plan.close();
-            query.close();
 
             BufferedReader readerExpected =
                     new BufferedReader(new InputStreamReader(new FileInputStream(expectedFile), "UTF-8"));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
deleted file mode 100644
index 09758f6..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.asterix.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with full parallelism on node controllers.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionFullParallelismIT {
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc2.conf";
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        LangExecutionUtil.tearDown();
-    }
-
-    @Parameters(name = "AqlExecutionFullParallelismIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
-        if (!tests.isEmpty()) {
-            tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
-        } else {
-            tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
-            tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
-        }
-        return tests;
-    }
-
-    protected TestCaseContext tcCtx;
-
-    public AqlExecutionFullParallelismIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Test
-    public void test() throws Exception {
-        LangExecutionUtil.test(tcCtx);
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
deleted file mode 100644
index 62ed790..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.asterix.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with less parallelism on node controllers than using all the cores.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionLessParallelismIT {
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc3.conf";
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        LangExecutionUtil.tearDown();
-    }
-
-    @Parameters(name = "AqlExecutionLessParallelismIT {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
-        if (!tests.isEmpty()) {
-            tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
-        } else {
-            tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
-            tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
-        }
-        return tests;
-    }
-
-    protected TestCaseContext tcCtx;
-
-    public AqlExecutionLessParallelismIT(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Test
-    public void test() throws Exception {
-        LangExecutionUtil.test(tcCtx);
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index c7ae2df..4ed36c6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.external.util.ExternalDataConstants;
@@ -53,7 +52,7 @@
     }
 
     public static List<ILibraryManager> setUp(boolean cleanup, String configFile) throws Exception {
-        return setUp(cleanup, configFile, integrationUtil, true, null);
+        return setUp(cleanup, configFile, integrationUtil, false, null);
     }
 
     public static List<ILibraryManager> setUp(boolean cleanup, String configFile,
@@ -107,6 +106,10 @@
         tearDown(cleanup, integrationUtil, true);
     }
 
+    public static void tearDown(boolean cleanup, boolean stopHdfs) throws Exception {
+        tearDown(cleanup, integrationUtil, stopHdfs);
+    }
+
     public static void tearDown(boolean cleanup, AsterixHyracksIntegrationUtil integrationUtil, boolean stopHdfs)
             throws Exception {
         // validateBufferCacheState(); <-- Commented out until bug is fixed -->
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
index cd270cd..75eccfd 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
@@ -19,6 +19,7 @@
 
 package org.apache.asterix.test.runtime;
 
+import static org.apache.asterix.test.runtime.ExecutionTestUtil.integrationUtil;
 import static org.apache.hyracks.util.ThreadDumpUtil.takeDumpJSONString;
 
 import java.io.BufferedReader;
@@ -61,10 +62,15 @@
     private static boolean checkStorageDistribution = true;
 
     public static void setUp(String configFile, TestExecutor executor) throws Exception {
+        setUp(configFile, executor, false);
+    }
+
+    public static void setUp(String configFile, TestExecutor executor, boolean startHdfs) throws Exception {
         testExecutor = executor;
         File outdir = new File(PATH_ACTUAL);
         outdir.mkdirs();
-        List<ILibraryManager> libraryManagers = ExecutionTestUtil.setUp(cleanupOnStart, configFile);
+        List<ILibraryManager> libraryManagers =
+                ExecutionTestUtil.setUp(cleanupOnStart, configFile, integrationUtil, startHdfs, null);
         ExternalUDFLibrarian.removeLibraryDir();
         librarian = new ExternalUDFLibrarian(libraryManagers);
         testExecutor.setLibrarian(librarian);
@@ -82,7 +88,7 @@
         } finally {
             ExternalUDFLibrarian.removeLibraryDir();
             ExecutionTestUtil.tearDown(cleanupOnStop);
-            ExecutionTestUtil.integrationUtil.removeTestStorageFiles();
+            integrationUtil.removeTestStorageFiles();
             if (!badTestCases.isEmpty()) {
                 System.out.println("The following test cases left some data");
                 for (String testCase : badTestCases) {
@@ -140,7 +146,7 @@
 
     // Checks whether data files are uniformly distributed among io devices.
     private static void checkStorageFiles() throws Exception {
-        NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        NodeControllerService[] ncs = integrationUtil.ncs;
         // Checks that dataset files are uniformly distributed across each io device.
         for (NodeControllerService nc : ncs) {
             checkNcStore(nc);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
new file mode 100644
index 0000000..6ea87ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHdfsExecutionTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with the storage parallelism.
+ */
+@RunWith(Parameterized.class)
+public class SqlppHdfsExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor(), true);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppHdfsExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp_hdfs.xml", "testsuite_sqlpp_hdfs.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppHdfsExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 0d0c1d9..e4e300a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -127,7 +127,7 @@
             MetadataProvider metadataProvider = mock(MetadataProvider.class);
 
             @SuppressWarnings("unchecked")
-            Map<String, String> config = mock(Map.class);
+            Map<String, Object> config = mock(Map.class);
             when(metadataProvider.getDefaultDataverseName()).thenReturn(dvName);
             when(metadataProvider.getConfig()).thenReturn(config);
             when(config.get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS)).thenReturn("true");
@@ -188,8 +188,8 @@
         PA.invokeMethod(rewriter,
                 "setup(java.util.List, org.apache.asterix.lang.common.base.IReturningStatement, "
                         + "org.apache.asterix.metadata.declared.MetadataProvider, "
-                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext)",
-                declaredFunctions, topExpr, metadataProvider, context);
+                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext, " + "java.util.Collection)",
+                declaredFunctions, topExpr, metadataProvider, context, null);
         PA.invokeMethod(rewriter, "inlineColumnAlias()");
         PA.invokeMethod(rewriter, "generateColumnNames()");
         PA.invokeMethod(rewriter, "substituteGroupbyKeyExpression()");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
deleted file mode 100644
index 17509a4..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * 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.asterix.test.storage;
-
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
-import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.TransactionOptions;
-import org.apache.asterix.external.util.DataflowUtils;
-import org.apache.asterix.file.StorageComponentProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails;
-import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.util.DiskUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DiskIsFullTest {
-
-    private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
-    private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
-            new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
-    private static final GenerationFunction[] RECORD_GEN_FUNCTION =
-            { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
-    private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
-    private static final ARecordType META_TYPE = null;
-    private static final GenerationFunction[] META_GEN_FUNCTION = null;
-    private static final boolean[] UNIQUE_META_FIELDS = null;
-    private static final int[] KEY_INDEXES = { 0 };
-    private static final int[] KEY_INDICATOR = { Index.RECORD_INDICATOR };
-    private static final List<Integer> KEY_INDICATOR_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
-    private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
-    private static final String DATASET_NAME = "TestDS";
-    private static final String DATA_TYPE_NAME = "DUMMY";
-    private static final String NODE_GROUP_NAME = "DEFAULT";
-    private static final String TEST_DISK_NAME = "asterixdb_ram_disk";
-    private boolean shouldRun = true;
-
-    @Before
-    public void setUp() throws Exception {
-        if (!SystemUtils.IS_OS_MAC) {
-            System.out.println("Skipping test " + DiskIsFullTest.class.getName() + " due to unsupported OS");
-            shouldRun = false;
-            return;
-        }
-        System.out.println("SetUp: ");
-        TestHelper.deleteExistingInstanceFiles();
-        // create RAM disk
-        final Path ramDiskRoot = DiskUtil.mountRamDisk(TEST_DISK_NAME, 4, MEGABYTE);
-        // Use RAM disk for storage
-        AsterixHyracksIntegrationUtil.setStoragePath(ramDiskRoot.toAbsolutePath().toString());
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        if (!shouldRun) {
-            return;
-        }
-        System.out.println("TearDown");
-        TestHelper.deleteExistingInstanceFiles();
-        DiskUtil.unmountRamDisk(TEST_DISK_NAME);
-        AsterixHyracksIntegrationUtil.restoreDefaultStoragePath();
-    }
-
-    @Test
-    public void testDiskIsFull() {
-        if (!shouldRun) {
-            return;
-        }
-        HyracksDataException expectedException =
-                HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-        try {
-            TestNodeController nc = new TestNodeController(null, false);
-            nc.init();
-            StorageComponentProvider storageManager = new StorageComponentProvider();
-            List<List<String>> partitioningKeys = new ArrayList<>();
-            partitioningKeys.add(Collections.singletonList("key"));
-            Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
-                    NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
-                            partitioningKeys, null, null, null, false, null),
-                    null, DatasetType.INTERNAL, DATASET_ID, 0);
-            try {
-                nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
-                        KEY_INDICATOR_LIST, 0);
-                JobId jobId = nc.newJobId();
-                IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
-                ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
-                        new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
-                // Prepare insert operation
-                LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
-                        RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
-                try {
-                    insertOp.open();
-                    TupleGenerator tupleGenerator =
-                            new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR, RECORD_GEN_FUNCTION,
-                                    UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
-                    VSizeFrame frame = new VSizeFrame(ctx);
-                    FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
-                    // Insert records until disk becomes full
-                    int tupleCount = 100000;
-                    while (tupleCount > 0) {
-                        ITupleReference tuple = tupleGenerator.next();
-                        try {
-                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
-                        } catch (Throwable t) {
-                            final Throwable rootCause = ExceptionUtils.getRootCause(t);
-                            rootCause.printStackTrace();
-                            if (rootCause instanceof HyracksDataException) {
-                                HyracksDataException cause = (HyracksDataException) rootCause;
-                                Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
-                                Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
-                                return;
-                            } else {
-                                break;
-                            }
-                        }
-                        tupleCount--;
-                    }
-                    Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-                } finally {
-                    try {
-                        insertOp.close();
-                    } finally {
-                        nc.getTransactionManager().abortTransaction(txnCtx.getTxnId());
-                    }
-                }
-            } finally {
-                nc.deInit();
-            }
-        } catch (Throwable e) {
-            e.printStackTrace();
-            Assert.fail("Expected exception (" + expectedException + ") was not thrown");
-        }
-    }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
index fb1adde..54ae683 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
@@ -18,18 +18,11 @@
  */
 package org.apache.asterix.test.storage;
 
-import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.COMPONENT_TIMESTAMP_FORMAT;
-
 import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.text.Format;
-import java.text.SimpleDateFormat;
 import java.util.Arrays;
-import java.util.Date;
-import java.util.Optional;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.TestDataUtil;
@@ -129,17 +122,15 @@
         TestDataUtil.upsertData(datasetName, 100);
         ncAppCtx.getDatasetLifecycleManager().flushDataset(dataset.getDatasetId(), false);
 
-        // create new invalid component with a timestamp > checkpoint valid component timestamp (i.e. in the future)
-        Format formatter = new SimpleDateFormat(COMPONENT_TIMESTAMP_FORMAT);
-        Date futureTime = new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(5));
-        String invalidComponentTimestamp =
-                formatter.format(futureTime) + AbstractLSMIndexFileManager.DELIMITER + formatter.format(futureTime);
+        // create new invalid component sequence with a sequence > checkpoint valid component sequence
+        String invalidComponentId = "1000";
+        String invalidComponentRange = invalidComponentId + AbstractLSMIndexFileManager.DELIMITER + invalidComponentId;
         FileReference indexDirRef = ncAppCtx.getIoManager().resolve(indexPath);
         String indexDir = indexDirRef.getFile().getAbsolutePath();
         // create the invalid component files
-        Path btreePath = Paths.get(indexDir, invalidComponentTimestamp + AbstractLSMIndexFileManager.DELIMITER
+        Path btreePath = Paths.get(indexDir, invalidComponentRange + AbstractLSMIndexFileManager.DELIMITER
                 + AbstractLSMIndexFileManager.BTREE_SUFFIX);
-        Path filterPath = Paths.get(indexDir, invalidComponentTimestamp + AbstractLSMIndexFileManager.DELIMITER
+        Path filterPath = Paths.get(indexDir, invalidComponentRange + AbstractLSMIndexFileManager.DELIMITER
                 + AbstractLSMIndexFileManager.BLOOM_FILTER_SUFFIX);
         Files.createFile(btreePath);
         Files.createFile(filterPath);
@@ -156,14 +147,36 @@
         DatasetResourceReference drr = DatasetResourceReference.of(localResource);
         IIndexCheckpointManagerProvider indexCheckpointManagerProvider = ncAppCtx.getIndexCheckpointManagerProvider();
         IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(drr);
-        Optional<String> validComponentTimestamp = indexCheckpointManager.getValidComponentTimestamp();
-        Assert.assertTrue(validComponentTimestamp.isPresent());
+        long validComponentSequence = indexCheckpointManager.getValidComponentSequence();
+        Assert.assertTrue(validComponentSequence > Long.MIN_VALUE);
 
         File[] indexRemainingFiles =
                 indexDirRef.getFile().listFiles(AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER);
         Assert.assertNotNull(indexRemainingFiles);
         long validComponentFilesCount = Arrays.stream(indexRemainingFiles)
-                .filter(file -> file.getName().startsWith(validComponentTimestamp.get())).count();
+                .filter(file -> file.getName().startsWith(String.valueOf(validComponentSequence))).count();
         Assert.assertTrue(validComponentFilesCount > 0);
     }
+
+    @Test
+    public void deleteCorruptedResourcesTest() throws Exception {
+        final INcApplicationContext ncAppCtx = (INcApplicationContext) integrationUtil.ncs[0].getApplicationContext();
+        final String nodeId = ncAppCtx.getServiceContext().getNodeId();
+        final String datasetName = "ds";
+        TestDataUtil.createIdOnlyDataset(datasetName);
+        final Dataset dataset = TestDataUtil.getDataset(integrationUtil, datasetName);
+        final String indexPath = TestDataUtil.getIndexPath(integrationUtil, dataset, nodeId);
+        final FileReference indexDirRef = ncAppCtx.getIoManager().resolve(indexPath);
+        final File indexMetadataFile = new File(indexDirRef.getFile(), StorageConstants.METADATA_FILE_NAME);
+        Assert.assertTrue(indexMetadataFile.exists());
+        // forge a mask file and ensure the metadata file and its mask files will be deleted after restart
+        final File indexMetadataMaskFile = new File(indexDirRef.getFile(),
+                StorageConstants.MASK_FILE_PREFIX + StorageConstants.METADATA_FILE_NAME);
+        Files.createFile(indexMetadataMaskFile.toPath());
+        Assert.assertTrue(indexMetadataMaskFile.exists());
+        integrationUtil.deinit(false);
+        integrationUtil.init(false, TEST_CONFIG_FILE_NAME);
+        Assert.assertFalse(indexMetadataFile.exists());
+        Assert.assertFalse(indexMetadataMaskFile.exists());
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
index 1c8ac62..964bf66 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.config.TransactionProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
@@ -64,6 +65,8 @@
     @Before
     public void setUp() throws Exception {
         System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+        // use a small page size for test purpose
+        integrationUtil.addOption(TransactionProperties.Option.TXN_LOG_BUFFER_PAGESIZE, 128 * 1024);
         integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
     }
 
@@ -144,7 +147,7 @@
     public void interruptedLogFileSwitch() throws Exception {
         final INcApplicationContext ncAppCtx = (INcApplicationContext) integrationUtil.ncs[0].getApplicationContext();
         final LogManager logManager = (LogManager) ncAppCtx.getTransactionSubsystem().getLogManager();
-        int logFileCountBeforeInterrupt = logManager.getLogFileIds().size();
+        int logFileCountBeforeInterrupt = logManager.getOrderedLogFileIds().size();
 
         // ensure an interrupted transactor will create next log file but will fail to position the log channel
         final AtomicBoolean failed = new AtomicBoolean(false);
@@ -159,7 +162,7 @@
         interruptedTransactor.start();
         interruptedTransactor.join();
         // ensure a new log file was created and survived interrupt
-        int logFileCountAfterInterrupt = logManager.getLogFileIds().size();
+        int logFileCountAfterInterrupt = logManager.getOrderedLogFileIds().size();
         Assert.assertEquals(logFileCountBeforeInterrupt + 1, logFileCountAfterInterrupt);
         Assert.assertFalse(failed.get());
 
diff --git a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
index 6cd3b87..811a40d 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
@@ -52,5 +52,6 @@
 messaging.frame.size=4096
 messaging.frame.count=512
 txn.log.partitionsize=2MB
+txn.log.buffer.pagesize=128KB
 txn.log.checkpoint.pollfrequency=2147483647
 storage.max.active.writable.datasets=50
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index fc95dd4..2694408 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -46,6 +46,7 @@
 heartbeat.max.misses=25
 
 [common]
+log.dir = logs/
 log.level = INFO
 compiler.framesize=32KB
 compiler.sortmemory=320KB
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql
deleted file mode 100644
index c987a4c..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-    id: string
-}
-
-create type LineType as open {
-  id:int32,
-  text: string
-}
-
-create dataset Book(LineType) with meta(AuxiliaryType)
-primary key id;
-
-create index MetaIndex on Book(meta().id) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql
deleted file mode 100644
index bd244d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * 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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql
deleted file mode 100644
index 615cb49..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-for $x in dataset('Metadata.Index')
-where $x.DataverseName='test'
-return $x
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql
deleted file mode 100644
index dc10acd..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql
deleted file mode 100644
index 94f253f..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-    id: string
-}
-
-create type LineType as open {
-  id:int32,
-  text: string
-}
-
-create dataset Book(LineType) with meta(AuxiliaryType)
-primary key id;
-
-create index MetaIndex on Book(meta().id, id) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql
deleted file mode 100644
index bd244d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * 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.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql
deleted file mode 100644
index 615cb49..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-for $x in dataset('Metadata.Index')
-where $x.DataverseName='test'
-return $x
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql
deleted file mode 100644
index dc10acd..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm
deleted file mode 100644
index 67d122f..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "Book", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:26 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "MetaIndex", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:26 PDT 2016", "PendingOp": 0, "SearchKeySourceIndicator": [ 1 ] }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm
deleted file mode 100644
index 1f2c976..0000000
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "Book", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:27 PDT 2016", "PendingOp": 0 }
-{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "MetaIndex", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ], [ "id" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:27 PDT 2016", "PendingOp": 0, "SearchKeySourceIndicator": [ 1, 0 ] }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 822d2c3..5fc2c1e 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -50,6 +50,7 @@
 { "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "geometry", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
 { "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index c0ab7a4..db10dcb 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -49,16 +49,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="basic">
-      <compilation-unit name="dataset_with_meta-6">
-        <output-dir compare="Text">dataset_with_meta-6</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="basic">
-      <compilation-unit name="dataset_with_meta-7">
-        <output-dir compare="Text">dataset_with_meta-7</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="basic">
       <compilation-unit name="meta01">
         <output-dir compare="Text">meta01</output-dir>
       </compilation-unit>
@@ -339,73 +329,73 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_3">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_4">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_5">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_6">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_7">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_239_drop_system_dataset_8">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+        <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_251_dataset_hint_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
+        <expected-error>Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_251_dataset_hint_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
+        <expected-error>Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_272_create_index_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+        <expected-error>Cannot find dataset with name Foo in dataverse test</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_272_create_index_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+        <expected-error>Cannot find dataset with name Foo in dataverse test</expected-error>
       </compilation-unit>
     </test-case>
     <!-- This case should be fixed to return a proper message rather than NPE -->
@@ -444,37 +434,37 @@
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_1">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_2">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[age]" which is of type integer cannot be indexed using the RTree index.</expected-error>
+        <expected-error>The field "[age]" which is of type integer cannot be indexed using the RTree index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_3">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_4">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_5">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="exception">
       <compilation-unit name="issue_384_create_index_error_6">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+        <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -482,7 +472,7 @@
     <test-case FilePath="transaction">
       <compilation-unit name="failure_previous_success">
         <output-dir compare="Text">failure_previous_success</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A datatype with this name StreetType already exists</expected-error>
+        <expected-error>A datatype with this name StreetType already exists</expected-error>
       </compilation-unit>
       <compilation-unit name="verify_failure_previous_success">
         <output-dir compare="Text">verify_failure_previous_success</output-dir>
@@ -491,11 +481,11 @@
     <test-case FilePath="transaction">
       <compilation-unit name="failure_subsequent_no_execution">
         <output-dir compare="Text">failure_subsequent_no_execution</output-dir>
-        <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A nodegroup with this name group1 already exists</expected-error>
+        <expected-error>A nodegroup with this name group1 already exists</expected-error>
       </compilation-unit>
       <compilation-unit name="verify_failure_subsequent_no_execution">
         <output-dir compare="Text">verify_failure_subsequent_no_execution</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
-</test-suite>
+</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/ASTERIXDB-2402.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/ASTERIXDB-2402.sqlpp
new file mode 100644
index 0000000..366540e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/ASTERIXDB-2402.sqlpp
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ */
+
+drop dataverse channels if exists;
+create dataverse channels;
+use channels;
+
+create type UserLocation as {
+  location: circle,
+  userName: string,
+  timeStamp: datetime
+};
+
+
+create type UserLocationFeedType as {
+  location: circle,
+  userName: string
+};
+
+create type EmergencyReport as {
+  reportId: uuid,
+  Etype: string,
+  location: circle,
+  timeStamp: datetime
+};
+
+create type EmergencyReportFeedType as {
+  Etype: string,
+  location: circle
+};
+
+
+create type EmergencyShelter as {
+  shelterName: string,
+  location: point
+};
+
+create dataset UserLocations(UserLocation)
+primary key userName;
+create dataset Shelters(EmergencyShelter)
+primary key shelterName;
+create dataset Reports(EmergencyReport)
+primary key reportId autogenerated;
+
+create index location_time on UserLocations(timeStamp);
+create index u_location on UserLocations(location) type RTREE;
+create index s_location on Shelters(location) type RTREE;
+create index report_time on Reports(timeStamp);
+
+create function EmergenciesNearMe(userName) {
+  (
+  select report, shelters from
+   ( select value r from Reports r where r.timeStamp >
+   current_datetime() - day_time_duration("PT10S"))report,
+  UserLocations u
+    let shelters = (select s.location from Shelters s where spatial_intersect(s.location,u.location))
+  where u.userName = userName
+  and spatial_intersect(report.location,u.location)
+  )
+};
+
+create type result as {
+  resultId:uuid
+};
+create type channelSub as {
+  channelSubId:uuid
+};
+create type brokerSub as {
+  channelSubId:uuid,
+  brokerSubId:uuid
+};
+create type broke as {
+  DataverseName: string,
+  BrokerName: string,
+  BrokerEndpoint: string
+};
+
+create dataset EmergenciesNearMeChannelResults(result) primary key resultId autogenerated;
+create dataset EmergenciesNearMeChannelChannelSubscriptions(channelSub) primary key channelSubId;
+create dataset EmergenciesNearMeChannelBrokerSubscriptions(brokerSub) primary key channelSubId,brokerSubId;
+create dataset Broker(broke) primary key DataverseName,BrokerName;
+
+
+
+SET inline_with "false";
+insert into channels.EmergenciesNearMeChannelResults as a (
+with channelExecutionTime as current_datetime()
+select result, channelExecutionTime, sub.channelSubId as channelSubId,current_datetime() as deliveryTime,
+(select b.BrokerEndPoint, bs.brokerSubId from
+channels.EmergenciesNearMeChannelBrokerSubscriptions bs,
+channels.Broker b
+where bs.BrokerName = b.BrokerName
+and bs.DataverseName = b.DataverseName
+and bs.channelSubId = sub.channelSubId
+) as brokerSubIds
+from channels.EmergenciesNearMeChannelChannelSubscriptions sub,
+channels.EmergenciesNearMe(sub.param0) result
+) returning
+(select
+a.channelExecutionTime, a.result, sub.BrokerEndpoint
+from (select sub from a.brokerSubIds sub) sub
+group by sub.BrokerEndpoint
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
index 88f8cd8..48e3a73 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/count-dataset.sqlpp
@@ -31,6 +31,6 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-COLL_COUNT(FacebookUsers);
+STRICT_COUNT(FacebookUsers);
 
-drop dataverse TinySocial;
\ No newline at end of file
+drop dataverse TinySocial;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
index 01a85d8..ae31568 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-2.sqlpp
@@ -31,5 +31,5 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-SELECT count(`friend-ids`) * avg(coll_count(`friend-ids`)) AS total
+SELECT count(`friend-ids`) * avg(strict_count(`friend-ids`)) AS total
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
index 50b799e..57ead72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626-3.sqlpp
@@ -33,5 +33,5 @@
 
 // The plan uses a group-by operator with the groupAll flag be true.
 WITH t AS 1
-SELECT count(`friend-ids`) * avg(coll_count(`friend-ids`)) AS total, t
+SELECT count(`friend-ids`) * avg(strict_count(`friend-ids`)) AS total, t
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
index 8f7a024..d49c51a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate/query-ASTERIXDB-1626.sqlpp
@@ -31,5 +31,5 @@
 
 create  dataset FacebookUsers(FacebookUserType) primary key id;
 
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`))
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`))
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
deleted file mode 100644
index e757455..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"count1":$t1.countA,
-"t2info": for $t2 in dataset('TweetMessages')
-                        where $t1.countA /* +indexnl */= $t2.countB and
-                        $t1.tweetid != $t2.tweetid
-                        order by $t2.tweetid
-                        return {"tweetid2": $t2.tweetid,
-                                       "count2":$t2.countB}
-};
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
new file mode 100644
index 0000000..96e43a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
deleted file mode 100644
index 1775c6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
- *                  Given the 'indexnl' hint we expect the join to be transformed
- *                  into an indexed nested-loop join using Customers' primary index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  lastorder: {
-    oid: int32,
-    total: float
-  }
-}
-
-create type OrderType as closed {
-  oid: int32,
-  cid: int32,
-  orderstatus: string,
-  orderpriority: string,
-  clerk: string,
-  total: float
-}
-
-create dataset Customers(CustomerType) primary key cid;
-create dataset Orders(OrderType) primary key oid;
-
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
-
-for $o in dataset('Orders')
-for $c in dataset('Customers')
-where $o.cid /*+ indexnl */ = $c.cid 
-return {"customer":$c, "order": $o} 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
new file mode 100644
index 0000000..e465dc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Equi joins two datasets, Customers and Orders, based on the customer id.
+ *                  Given the 'indexnl' hint we expect the join to be transformed
+ *                  into an indexed nested-loop join using Customers' primary index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create type test.OrderType as
+ closed {
+  oid : integer,
+  cid : integer,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Orders(OrderType) primary key oid;
+
+write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
+select element {'customer':c,'order':o}
+from  Orders as o,
+      Customers as c
+where (o.cid /*+ indexnl */  = c.cid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
deleted file mode 100644
index bb4ad1b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
- *                  We first expect FacebookUsers' primary index to be used 
- *                  to satisfy the range condition on it's primary key. 
- *                  FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type EmploymentType as closed { 
-  organization-name: string,
-  start-date: date,
-  end-date: date?
-} 
-
-create type FacebookUserType as closed { 
-  id: int32,
-  id-copy: int32,
-  alias: string,
-  name: string,
-  user-since: datetime,
-  user-since-copy: datetime,
-  friend-ids: {{ int32 }},
-  employment: [EmploymentType]
-} 
-
-create type FacebookMessageType as closed { 
-  message-id: int32,
-  message-id-copy: int32,
-  author-id: int32,
-  author-id-copy: int32,
-  in-response-to: int32?,
-  sender-location: point?,
-  message: string
-} 
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookMessages(FacebookMessageType)
-primary key message-id;
-
-create index fbmIdxAutId if not exists on FacebookMessages(author-id-copy);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
-
-for $user in dataset('FacebookUsers')
-for $message in dataset('FacebookMessages')
-where $user.id /*+ indexnl */ = $message.author-id-copy
-and $user.id >= 11000 and $user.id <= 12000
-return {
-  "fbu-ID": $user.id,
-  "fbm-auth-ID": $message.author-id,
-  "uname": $user.name,
-  "message": $message.message
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
new file mode 100644
index 0000000..4580dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
+ *                  We first expect FacebookUsers' primary index to be used
+ *                  to satisfy the range condition on it's primary key.
+ *                  FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.EmploymentType as
+ closed {
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type test.FacebookUserType as
+ closed {
+  id : integer,
+  `id-copy` : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `user-since-copy` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create type test.FacebookMessageType as
+ closed {
+  `message-id` : integer,
+  `message-id-copy` : integer,
+  `author-id` : integer,
+  `author-id-copy` : integer,
+  `in-response-to` : integer?,
+  `sender-location` : point?,
+  message : string
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
+create  dataset FacebookMessages(FacebookMessageType) primary key `message-id`;
+
+create  index fbmIdxAutId  if not exists  on FacebookMessages (`author-id-copy`) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
+select element {'fbu-ID':user.id,'fbm-auth-ID':message.`author-id`,'uname':user.name,'message':message.message}
+from  FacebookUsers as user,
+      FacebookMessages as message
+where ((user.id /*+ indexnl */  = message.`author-id-copy`) and (user.id >= 11000) and (user.id <= 12000))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
deleted file mode 100644
index 5bdacc3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Equi joins two datasets, DBLP and CSX, based on their title.
- *                  CSX has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index title_index on CSX(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
new file mode 100644
index 0000000..1c315f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their title.
+ *                  CSX has a secondary btree index on title, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index title_index  on CSX (title) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (a.title /*+ indexnl */  = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
deleted file mode 100644
index 33bb320..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Equi self-joins a dataset, DBLP, based on its title.
- *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
- *                  we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index title_index on DBLP(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
new file mode 100644
index 0000000..7ab67fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Equi self-joins a dataset, DBLP, based on its title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index title_index  on DBLP (title) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (a.title /*+ indexnl */  = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
deleted file mode 100644
index b72dbcb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Please note this is a Positive test and the BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Roger"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
new file mode 100644
index 0000000..845da2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname > 'Roger')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
deleted file mode 100644
index 0030ddf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// This is a Positive test - prefix search, BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Susan"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
new file mode 100644
index 0000000..db55b28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname >= 'Susan')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
deleted file mode 100644
index 48b67d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification (usage) test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Isa"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
new file mode 100644
index 0000000..ecac451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification (usage) test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname < 'Isa')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
deleted file mode 100644
index 5e39552..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used in query plan
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Vanpatten"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
new file mode 100644
index 0000000..6354f25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname <= 'Vanpatten')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
deleted file mode 100644
index 23518e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used in the optimized query plan 
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Young Seok" and $emp.lname = "Kim"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
new file mode 100644
index 0000000..d8a95cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used in the optimized query plan
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Young Seok') and (emp.lname = 'Kim'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
deleted file mode 100644
index 81fb7b4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Alex" and $emp.lname < "Zach"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
new file mode 100644
index 0000000..75ccb90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Alex') and (emp.lname < 'Zach'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
deleted file mode 100644
index fbb504d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Allan" and $emp.lname < "Zubi"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
new file mode 100644
index 0000000..2cf7dbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname < 'Zubi'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
deleted file mode 100644
index fbc0dbb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Allan" and $emp.lname = "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
new file mode 100644
index 0000000..98f037e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
deleted file mode 100644
index 356503f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Julio" and $emp.lname < "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
new file mode 100644
index 0000000..a88838f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname < 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
deleted file mode 100644
index 9dcc4eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Michael" and $emp.lname <= "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
new file mode 100644
index 0000000..524b0a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Michael') and (emp.lname <= 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
deleted file mode 100644
index 5ffc387..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
new file mode 100644
index 0000000..f6a5120
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
deleted file mode 100644
index 4794082..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
new file mode 100644
index 0000000..cd4fae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
deleted file mode 100644
index 47cc065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Craig" and $emp.lname > "Kevin"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
new file mode 100644
index 0000000..9efcfc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname <= 'Craig') and (emp.lname > 'Kevin'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
deleted file mode 100644
index 9be2c17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
new file mode 100644
index 0000000..3b3b5e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
deleted file mode 100644
index 807d861..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
new file mode 100644
index 0000000..d6d977c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
deleted file mode 100644
index a7492fa..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Craig" and $emp.lname <= "Kevin" and $emp.fname <= "Mary" and $emp.lname >= "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
new file mode 100644
index 0000000..985cde7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname <= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname >= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
deleted file mode 100644
index 7066b35..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
- 
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Max" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
new file mode 100644
index 0000000..40ae7d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname > 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
deleted file mode 100644
index 19168f0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Sofia" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
new file mode 100644
index 0000000..8b9a588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname >= 'Sofia')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
deleted file mode 100644
index 09ad8c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Chen" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
new file mode 100644
index 0000000..14d1ce9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname < 'Chen')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
deleted file mode 100644
index 9c4d2ca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname <= "Julio" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
new file mode 100644
index 0000000..4070114
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname <= 'Julio')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
deleted file mode 100644
index b49db5c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the primary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Neil" and $emp.fname < "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
new file mode 100644
index 0000000..e3f42ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the primary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Neil') and (emp.fname < 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
deleted file mode 100644
index 479740f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname >= "Max" and $emp.fname <= "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
new file mode 100644
index 0000000..8656ab8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
deleted file mode 100644
index 02ad02e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
- 
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Max" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
new file mode 100644
index 0000000..38fad40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where (emp.fname = 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
deleted file mode 100644
index 59801d3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType)  primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname < "Tomes"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
new file mode 100644
index 0000000..e8d5627
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
deleted file mode 100644
index 90ad451..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname = "Julio" and $emp.lname > "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
new file mode 100644
index 0000000..ce68496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 11th Nov 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname > 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
deleted file mode 100644
index 6b6bce0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- *  Description     : BTree Index verification test
- *                  : This test is intended to verify that the secondary BTree index is used 
- *                  : in the optimized query plan.
- *  Expected Result : Success
- *  Date            : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst') 
-where $emp.fname < "Julio" and $emp.lname = "Xu"
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
new file mode 100644
index 0000000..0ef6c12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 11th Nov 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname,lname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname < 'Julio') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
new file mode 100644
index 0000000..af04479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
deleted file mode 100644
index 8b8acaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-drop dataverse colocated if exists;
-
-create dataverse colocated;
-
-use dataverse colocated;
-
-create type UserType as open {
-  uid: int32, 
-  name: string,
-  lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
-  vid: int32, 
-  name: string,
-  lottery_numbers: {{int32}}
-}
-
-create dataset Users(UserType) 
-  primary key uid;
-
-create dataset Visitors(VisitorType) 
-  primary key vid;
-
-
-write output to asterix_nc1:"/tmp/fuzzy1.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.uid = $visitor.vid
-return { "user_name" : $user.name, "visitor_name" : $visitor.name }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
new file mode 100644
index 0000000..ee49511
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+drop  dataverse colocated if exists;
+create  dataverse colocated;
+
+use colocated;
+
+
+create type colocated.UserType as
+{
+  uid : integer,
+  name : string,
+  lottery_numbers : {{integer}}
+};
+
+create type colocated.VisitorType as
+{
+  vid : integer,
+  name : string,
+  lottery_numbers : {{integer}}
+};
+
+create  dataset Users(UserType) primary key uid;
+
+create  dataset Visitors(VisitorType) primary key vid;
+
+write output to asterix_nc1:"/tmp/fuzzy1.adm";
+select element {'user_name':user.name,'visitor_name':visitor.name}
+from  Users as user,
+      Visitors as visitor
+where (user.uid = visitor.vid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
deleted file mode 100644
index 2ad271e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-drop dataverse group_no_agg if exists;
-
-create dataverse group_no_agg;
-
-use dataverse group_no_agg;
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType) 
-  primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/.adm";
-
-for $c in dataset('Customers')
-group by $name := $c.c_name  with $c
-return $name
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
new file mode 100644
index 0000000..3c82655
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+drop  dataverse group_no_agg if exists;
+create  dataverse group_no_agg;
+
+use group_no_agg;
+
+
+create type group_no_agg.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key c_custkey on group1;
+
+write output to asterix_nc1:"/tmp/.adm";
+select element name
+from  Customers as c
+group by c.c_name as name
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
deleted file mode 100644
index dbaf7be..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.
- */
-drop dataverse custorder if exists;
-
-create dataverse custorder;
-
-use dataverse custorder;
-
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  lastorder: {
-    oid: int32,
-    total: float
-  }
-}
-
-
-
-create type OrderType as closed {
-  oid: int32,
-  cid: int32,
-  orderstatus: string,
-  orderpriority: string,
-  clerk: string,
-  total: float
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType) 
-  primary key cid on group1;
-create dataset Orders(OrderType)
-  primary key oid on group1;
-
-write output to asterix_nc1:"/tmp/custorder.adm";
-
-for $c in dataset('Customers')
-for $o in dataset('Orders')
-where $c.cid = $o.cid
-group by $cid := $c.cid decor $cust := $c with $o
-return {"cid":$cid, "cust":$cust, "cnt-orders": count($o), "orders":$o} 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
new file mode 100644
index 0000000..508d6ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+drop  dataverse custorder if exists;
+create  dataverse custorder;
+
+use custorder;
+
+
+create type custorder.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type custorder.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create type custorder.OrderType as
+ closed {
+  oid : integer,
+  cid : integer,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key cid on group1;
+
+create  dataset Orders(OrderType) primary key oid on group1;
+
+write output to asterix_nc1:"/tmp/custorder.adm";
+select element {'cid':cid,'cust':(from g select c),'cnt-orders':count(o),'orders': (from g select o)}
+from  Customers as c,
+      Orders as o
+where (c.cid = o.cid)
+group by c.cid as cid group as g
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
deleted file mode 100644
index 285d27b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TestType as {
-  "id" : string,
-  "idx" : string,
-  "no-idx" : string
-};
-
-create dataset TestSet(TestType) primary key "id";
-create index TestSetIndex on TestSet(idx);
-
-delete $x from dataset TestSet where $x.no-idx = "one" or $x.no-idx = "two";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp
new file mode 100644
index 0000000..40064ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+  id : string,
+  idx : string,
+  `no-idx` : string
+};
+
+create  dataset TestSet(TestType) primary key id;
+
+create  index TestSetIndex  on TestSet (idx) type btree;
+
+delete from TestSet
+ where `no-idx` = 'one' or `no-idx` = 'two';
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/sugar-06-distinct.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/sugar-06-distinct.sqlpp
new file mode 100644
index 0000000..d961374
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/sugar-06-distinct.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE gby IF EXISTS;
+CREATE DATAVERSE gby;
+
+USE gby;
+
+CREATE TYPE EmpType AS {
+  name : string
+};
+
+CREATE DATASET Employee(EmpType) PRIMARY KEY name;
+
+FROM Employee e
+GROUP BY deptno
+SELECT deptno, MAX(DISTINCT salary) salary_agg
+ORDER BY salary_agg DESC
+LIMIT 3;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
deleted file mode 100644
index 0425ecf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.
- */
-/*
- * Test case Name  : insert-and-scan-dataset.aql
- * Description     : This test is intended to test inserting into a dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date            : July 11 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type myDataType as open {
-  id: int32
-}
-
-create dataset myData(myDataType)
-  primary key id;
-
-insert into dataset myData (
-for $x in dataset myData
-return {
-    "id": $x.id + 1
-}
-);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp
new file mode 100644
index 0000000..5566eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.myDataType as
+{
+  id : integer
+};
+
+create  dataset myData(myDataType) primary key id;
+
+insert into myData
+select element {'id':(x.id + 1)}
+from  myData as x
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
deleted file mode 100644
index 261a357..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.
- */
-drop dataverse fuzzyjoin if exists;
-create dataverse fuzzyjoin;
-use dataverse fuzzyjoin;
-
-create type TOKENSRANKEDADMType as closed {
-  token: int32,
-  rank: int32
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
-
-write output to asterix_nc1:'rttest/introhashpartitionmerge.adm';
-
-        for $token1 in dataset('TOKENSRANKEDADM')
-        for $token2 in
-            for $tokenRanked in dataset('TOKENSRANKEDADM')
-            order by $tokenRanked.rank
-            return $tokenRanked
-        where $token1.token = $token2.token
-        order by $token2.rank
-        return $token2.rank
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
new file mode 100644
index 0000000..103aae8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+drop  dataverse fuzzyjoin if exists;
+create  dataverse fuzzyjoin;
+
+use fuzzyjoin;
+
+
+create type fuzzyjoin.TOKENSRANKEDADMType as
+ closed {
+  token : integer,
+  rank : integer
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
+
+write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
+select element token2.rank
+from  TOKENSRANKEDADM as token1,
+      (
+    select element tokenRanked
+    from  TOKENSRANKEDADM as tokenRanked
+    order by tokenRanked.rank
+) as token2
+where (token1.token = token2.token)
+order by token2.rank
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
deleted file mode 100644
index 488ed70..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
-
-// Cannot optimize this query because the string constant is shorter than the gram length.
-for $o in dataset('DBLP')
-where contains($o.title, "Mu")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
new file mode 100644
index 0000000..e13305d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
deleted file mode 100644
index ba460a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
new file mode 100644
index 0000000..e5b4885
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
deleted file mode 100644
index 81780a2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
-
-for $o in dataset('DBLP')
-where edit-distance-check($o.authors, "Amihay Motro", 1)[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
new file mode 100644
index 0000000..da55c99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
+select element o
+from  DBLP as o
+where test.`edit-distance-check`(o.authors,'Amihay Motro',1)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
deleted file mode 100644
index 6917878..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
-
-for $o in dataset('DBLP')
-where edit-distance($o.authors, "Amihay Motro") <= 1
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
new file mode 100644
index 0000000..91a809b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
+select element o
+from  DBLP as o
+where (test.`edit-distance`(o.authors,'Amihay Motro') <= 1)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index b4d342c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '1';
-
-for $o in dataset('DBLP')
-where $o.authors ~= "Amihay Motro"
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..d684f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `1`;
+
+select element o
+from  DBLP as o
+where (o.authors ~= 'Amihay Motro')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
deleted file mode 100644
index 55f1830..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $o in dataset('DBLP')
-where gram-tokens($o.title, 3, false) ~= gram-tokens("Transactions for Cooperative Environments", 3, false)
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..5adcda0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element o
+from  DBLP as o
+where (test.`gram-tokens`(o.title,3,false) ~= test.`gram-tokens`('Transactions for Cooperative Environments',3,false))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
deleted file mode 100644
index c7d273d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
new file mode 100644
index 0000000..5b564db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
+select element o
+from  DBLP as o
+where test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
deleted file mode 100644
index da87bc0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false)) >= 0.5f
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
new file mode 100644
index 0000000..3e024a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
+select element o
+from  DBLP as o
+where (test.`similarity-jaccard`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
deleted file mode 100644
index 88b0d58..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
new file mode 100644
index 0000000..41ceade
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
deleted file mode 100644
index d107b68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
-
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
new file mode 100644
index 0000000..0ca8b00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
deleted file mode 100644
index f4d5e14..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 3
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
new file mode 100644
index 0000000..0f2fcf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
deleted file mode 100644
index 687e55d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
-
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 1
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
new file mode 100644
index 0000000..a4114d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
deleted file mode 100644
index 8833eaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '1';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["computers", "wine", "walking"]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..1bfd5f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `1`;
+
+select element c
+from  Customers as c
+where (c.interests ~= ['computers','wine','walking'])
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
deleted file mode 100644
index d0df75f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["databases", "computers", "wine"]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..a1372d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element c
+from  Customers as c
+where (c.interests ~= ['databases','computers','wine'])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
deleted file mode 100644
index 97e9962..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
new file mode 100644
index 0000000..89a3eb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
+select element c
+from  Customers as c
+where test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
deleted file mode 100644
index 329b26a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, ["databases", "computers", "wine"]) >= 0.7f
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
new file mode 100644
index 0000000..d13c98b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
+select element c
+from  Customers as c
+where (test.`similarity-jaccard`(c.interests,['databases','computers','wine']) >= 0.700000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
deleted file mode 100644
index c20493a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.8f';
-
-for $c in dataset('Customers')
-where $c.interests ~= {{"computers", "wine", "databases"}}
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..d32a8f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.8f`;
+
+select element c
+from  Customers as c
+where (c.interests ~= {{'computers','wine','databases'}})
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
deleted file mode 100644
index cb6c0c4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, {{"computers", "wine", "databases"}}, 0.7f)[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
new file mode 100644
index 0000000..25568e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+select element c
+from  Customers as c
+where test.`similarity-jaccard-check`(c.interests,{{'computers','wine','databases'}},0.700000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
deleted file mode 100644
index c0cdd4f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, {{"computers", "databases", "wine"}}) >= 0.7f
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
new file mode 100644
index 0000000..2a26ee9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
+select element c
+from  Customers as c
+where (test.`similarity-jaccard`(c.interests,{{'computers','databases','wine'}}) >= 0.700000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
deleted file mode 100644
index 191a9bf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
-
-// Contains cannot be answered with a word inverted index.
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
new file mode 100644
index 0000000..2a1db6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
deleted file mode 100644
index ad2c577..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $o in dataset('DBLP')
-where word-tokens($o.title) ~= word-tokens("Transactions for Cooperative Environments")
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
new file mode 100644
index 0000000..6157fde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element o
+from  DBLP as o
+where (test.`word-tokens`(o.title) ~= test.`word-tokens`('Transactions for Cooperative Environments'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
deleted file mode 100644
index 6f4497f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)[0]
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
new file mode 100644
index 0000000..b0dc05a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
+select element o
+from  DBLP as o
+where test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
deleted file mode 100644
index c96de32..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments")) >= 0.5f
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
new file mode 100644
index 0000000..eb96ddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
+select element o
+from  DBLP as o
+where (test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
deleted file mode 100644
index cbe5b17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
- *                  two edit-distance-check function of which only the first can be optimized with an index. 
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
-
-// Only the first edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-where $eda[0] and $edb[0] 
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
new file mode 100644
index 0000000..c60c8fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using
+ *                  two edit-distance-check function of which only the first can be optimized with an index.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+select element o
+from  DBLP as o
+with  eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3),
+      edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
+where (eda[0] and edb[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
deleted file mode 100644
index a40a4de..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using 
- *                  two edit-distance-check function of which only the second can be optimized with an index.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
-
-// Only the second edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-where $edb[0] and $eda[0] 
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
new file mode 100644
index 0000000..8a5d1d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using
+ *                  two edit-distance-check function of which only the second can be optimized with an index.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
+select element o
+from  DBLP as o
+with  edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5),
+      eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3)
+where (edb[0] and eda[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
deleted file mode 100644
index 09173ce..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
-
-for $o in dataset('DBLP')
-let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
-where $ed[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
new file mode 100644
index 0000000..1f725d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
+select element o
+from  DBLP as o
+with  ed as test.`edit-distance-check`(o.authors,'Amihay Motro',1)
+where ed[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
deleted file mode 100644
index 73293bc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
- *                  Tests that the optimizer rule correctly drills through the substring function.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
-
-for $paper in dataset('DBLP')
-where edit-distance-check(substring($paper.title, 0, 8), "datbase", 1)[0]
-return {
-  "id" : $paper.id,
-  "title" : $paper.title
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
new file mode 100644
index 0000000..d2e3888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
+ *                  Tests that the optimizer rule correctly drills through the substring function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
+select element {'id':paper.id,'title':paper.title}
+from  DBLP as paper
+where test.`edit-distance-check`(test.substring(paper.title,0,8),'datbase',1)[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
deleted file mode 100644
index 3271c8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
-where $jacc[0]
-return $o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..248c243
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
+select element o
+from  DBLP as o
+with  jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
deleted file mode 100644
index dfeda68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := gram-tokens($paper.title, 3, false)
-let $query_tokens := gram-tokens("Transactions for Cooperative Environments", 3, false)
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
new file mode 100644
index 0000000..8be4caa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
+select element {'Paper':paper_tokens,'Query':query_tokens}
+from  DBLP as paper
+with  paper_tokens as test.`gram-tokens`(paper.title,3,false),
+      query_tokens as test.`gram-tokens`('Transactions for Cooperative Environments',3,false),
+      jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
deleted file mode 100644
index 2cb65d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should *not* be applied (see below).
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
new file mode 100644
index 0000000..68106a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
deleted file mode 100644
index 9646383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
new file mode 100644
index 0000000..7083657
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
deleted file mode 100644
index a274b9f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..a04a8ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
+select element c
+from  Customers as c
+with  jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
deleted file mode 100644
index 9d3ca6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..ceac8c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
+select element c
+from  Customers as c
+with  jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
deleted file mode 100644
index d932d23..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
new file mode 100644
index 0000000..e4f3b8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
+select element o
+from  DBLP as o
+with  jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
deleted file mode 100644
index 79fe390..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- *                  Tests that the optimizer rule correctly drills through the let clauses.
- *                  The index should be applied.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := word-tokens($paper.title)
-let $query_tokens := word-tokens("Transactions for Cooperative Environments")
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.8f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
new file mode 100644
index 0000000..56ede6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
+select element {'Paper':paper_tokens,'Query':query_tokens}
+from  DBLP as paper
+with  paper_tokens as test.`word-tokens`(paper.title),
+      query_tokens as test.`word-tokens`('Transactions for Cooperative Environments'),
+      jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.800000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
deleted file mode 100644
index b99bc22..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
new file mode 100644
index 0000000..c29de43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
+select element {'aauthors':a.authors,'bauthors':b.authors,'ed':ed}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index d4491eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary. 
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
new file mode 100644
index 0000000..1234824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'aauthors':a.authors,'bauthors':b.authors}
+from  DBLP as a,
+      CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
deleted file mode 100644
index 04d57a4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
new file mode 100644
index 0000000..4fdc637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
+select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
deleted file mode 100644
index 5d48e10..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
new file mode 100644
index 0000000..40c8c5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'ed':ed}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
deleted file mode 100644
index d074b0f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
new file mode 100644
index 0000000..a003dee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
deleted file mode 100644
index 56697c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
new file mode 100644
index 0000000..1372490
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
+select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
deleted file mode 100644
index b917cda..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
new file mode 100644
index 0000000..9530163
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ *                  We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
+select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
deleted file mode 100644
index 8ddd03b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Test that left-outer-join may use an available inverted index in index subtree.
- * Issue        : 741
- * Expected Res : Success
- * Date         : 16th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as {
-screen_name: string,
-lang: string,
-friends_count: int32,
-statuses_count: int32,
-name: string,
-followers_count: int32
-}
-
-create type TweetMessageType as {
-tweetid: int64,
-user: TwitterUserType,
-sender_location: point?,
-send_time: datetime,
-referred_topics: {{ string }},
-message_text: string
-}
-
-create dataset TweetMessages(TweetMessageType) primary key tweetid;
-
-create index topicIIx on TweetMessages(referred_topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
-
-for $t in dataset('TweetMessages')
-where $t.send_time >= datetime('2011-06-18T14:10:17')
-and
-$t.send_time < datetime('2011-06-18T15:10:17')
-return {
-    "tweet": $t.tweetid,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := similarity-jaccard-check($t.referred_topics, $t2.referred_topics, 0.6f)
-              where $sim[0] and
-                      $t2.tweetid != $t.tweetid
-                      return $t2.tweetid
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
new file mode 100644
index 0000000..5a312b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test that left-outer-join may use an available inverted index in index subtree.
+ * Issue        : 741
+ * Expected Res : Success
+ * Date         : 16th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+{
+  screen_name : string,
+  lang : string,
+  friends_count : integer,
+  statuses_count : integer,
+  name : string,
+  followers_count : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  sender_location : point?,
+  send_time : datetime,
+  referred_topics : {{string}},
+  message_text : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index topicIIx  on TweetMessages (referred_topics) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
+select element {'tweet':t.tweetid,'similar-tweets':(
+        select element t2.tweetid
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t.referred_topics,t2.referred_topics,0.600000f)
+        where (sim[0] and (t2.tweetid != t.tweetid))
+    )}
+from  TweetMessages as t
+where ((t.send_time >= test.datetime('2011-06-18T14:10:17')) and (t.send_time < test.datetime('2011-06-18T15:10:17')))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
deleted file mode 100644
index bd52bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-create index topicKeywordIx on TweetMessages(referred-topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
-    "tweet": {"id": $t1.tweetid, "topics" : $t1.message-text} ,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := edit-distance-check($t1.message-text, $t2.message-text, 7)
-              where $sim[0] and
-                      $t2.tweetid != $t1.tweetid
-                      order by $t2.tweetid
-                      return {"id": $t2.tweetid, "topics" : $t2.message-text}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
new file mode 100644
index 0000000..8c2869f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
deleted file mode 100644
index 27b0e37..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-create index topicKeywordIx on TweetMessages(referred-topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
-    "tweet": {"id": $t1.tweetid, "topics" : $t1.referred-topics} ,
-    "similar-tweets": for $t2 in dataset('TweetMessages')
-                      let $sim := similarity-jaccard-check($t1.referred-topics, $t2.referred-topics, 0.5f)
-              where $sim[0] and
-                      $t2.tweetid != $t1.tweetid
-                      order by $t2.tweetid
-                      return {"id": $t2.tweetid, "topics" : $t2.referred-topics}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
new file mode 100644
index 0000000..9db0bd7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.`referred-topics`,t2.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
deleted file mode 100644
index cf2222d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
- *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
new file mode 100644
index 0000000..a6ade32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
deleted file mode 100644
index 95b2912..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32,
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
new file mode 100644
index 0000000..ed8d6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
deleted file mode 100644
index 5961d13..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance-check($a.authors, $b.authors, 3)
-where $ed[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
new file mode 100644
index 0000000..7a24772
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
+select element {'arec':a,'brec':b,'ed':ed[1]}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance-check`(a.authors,b.authors,3)
+where (ed[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
deleted file mode 100644
index c1a1817..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
- *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
new file mode 100644
index 0000000..950a058
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
+ *                  CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
deleted file mode 100644
index c65b61b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
new file mode 100644
index 0000000..dc8ca3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
deleted file mode 100644
index 6287cb0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index. 
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
new file mode 100644
index 0000000..0284ff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
+select element {'arec':a,'brec':b,'ed':ed}
+from  DBLP as a,
+      DBLP as b
+with  ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
deleted file mode 100644
index 66bb2ac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
new file mode 100644
index 0000000..2021605
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c611d3a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
- *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
new file mode 100644
index 0000000..d900a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
+ *                  DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (authors) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 085a383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..aac9ca7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 444b735..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..d3bcd49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
deleted file mode 100644
index 31ec7a5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..460f1cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
deleted file mode 100644
index 6255558..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..5e180f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
deleted file mode 100644
index f70df08..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..2b9aa31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
deleted file mode 100644
index 2e1b065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
- *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
new file mode 100644
index 0000000..a2dc8f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
+ *                  CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
deleted file mode 100644
index 31003d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
new file mode 100644
index 0000000..fbc230c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
deleted file mode 100644
index 1e86763..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
new file mode 100644
index 0000000..5265874
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ *                  DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
deleted file mode 100644
index f074ad1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
new file mode 100644
index 0000000..f7e6047
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
deleted file mode 100644
index 723d0b5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
new file mode 100644
index 0000000..6435990
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
deleted file mode 100644
index 0bb95d5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance-check($a.interests, $b.interests, 3)
-where $ed[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
new file mode 100644
index 0000000..9f14368
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
+select element {'arec':a,'brec':b,'ed':ed[1]}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance-check`(a.interests,b.interests,3)
+where (ed[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
deleted file mode 100644
index e5f5b40..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
new file mode 100644
index 0000000..5ebb518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
deleted file mode 100644
index f78a6bd..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
new file mode 100644
index 0000000..ac7e6ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
deleted file mode 100644
index c4e037f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
new file mode 100644
index 0000000..0655746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
+select element {'arec':a,'brec':b,'ed':ed}
+from  Customers as a,
+      Customers as b
+with  ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
deleted file mode 100644
index c5a558c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
new file mode 100644
index 0000000..cd3d74f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c96c0e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'edit-distance';
-set simthreshold '3';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
new file mode 100644
index 0000000..a184602
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `edit-distance`;
+
+set `simthreshold` `3`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index f90593a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..c1f3af7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 264c93a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..573f5e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
deleted file mode 100644
index ce60fe5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..c9cbd1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
deleted file mode 100644
index edaba17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..6364474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
deleted file mode 100644
index 3d704f2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..bd84396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
deleted file mode 100644
index 55c34ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
new file mode 100644
index 0000000..b6bea6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
deleted file mode 100644
index 37725d9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
new file mode 100644
index 0000000..3359746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
deleted file mode 100644
index 1bba03c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: [string],
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
new file mode 100644
index 0000000..941d506
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index b918bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..5c46ddc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 64260d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.7f';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..b874be2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.7f`;
+
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ((a.interests /*+ indexnl */  ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
deleted file mode 100644
index 0a1f2f8..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..2a771e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
deleted file mode 100644
index f489f34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..8d71bb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
deleted file mode 100644
index 6c87995..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..476db19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
deleted file mode 100644
index 8a6d743..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
- *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
new file mode 100644
index 0000000..287ef06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
+ *                  Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  dataset Customers2(CustomerType) primary key cid;
+
+create  index interests_index  on Customers2 (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers2 as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
deleted file mode 100644
index ef5269a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
new file mode 100644
index 0000000..bc7b1cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  Customers as a,
+      Customers as b
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
deleted file mode 100644
index 72a8dca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
-  number: int32, 
-  street: string,
-  city: string
-}
-
-create type CustomerType as closed {
-  cid: int32, 
-  name: string,
-  age: int32?,
-  address: AddressType?,
-  interests: {{string}},
-  children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-  
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
new file mode 100644
index 0000000..77c26cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ *                  Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : {{string}},
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  Customers as a,
+      Customers as b
+with  jacc as  /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 6b1feb6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
new file mode 100644
index 0000000..e2158e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index d305ba0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
new file mode 100644
index 0000000..c6c38b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
+set `simfunction` `jaccard`;
+
+set `simthreshold` `0.5f`;
+
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index 5586919..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- *                  TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- *                    transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- *                    multiple indexed nested loop joins of various type of indexes.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-    user: TwitterUserType,
-    sender-location: point,
-    send-time: datetime,
-    referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.message-text), word-tokens($t2.message-text), 0.6f)
-where $sim[0] and $t1.tweetid < int64("20") and $t2.tweetid != $t1.tweetid
-return {
-    "t1": $t1.tweetid,
-    "t2": $t2.tweetid,
-    "sim": $sim[1]
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
new file mode 100644
index 0000000..50d2442
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
+ *                  TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
+ *                    transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
+ *                    multiple indexed nested loop joins of various type of indexes.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
+select element {'t1':t1.tweetid,'t2':t2.tweetid,'sim':sim[1]}
+from  TweetMessages as t1,
+      TweetMessages as t2
+with  sim as test.`similarity-jaccard-check`(test.`word-tokens`(t1.`message-text`),test.`word-tokens`(t2.`message-text`),0.600000f)
+where (sim[0] and (t1.tweetid < test.bigint('20')) and (t2.tweetid != t1.tweetid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
deleted file mode 100644
index 0a63d12..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
new file mode 100644
index 0000000..0c2890a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
deleted file mode 100644
index 5c97de1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
new file mode 100644
index 0000000..fdf6235
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
deleted file mode 100644
index 7481ed3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
new file mode 100644
index 0000000..8eb16d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc[1]}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
deleted file mode 100644
index 1b123a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create type CSXType as closed {
-  id: int32, 
-  csxid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
new file mode 100644
index 0000000..b943eec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ *                  CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index keyword_index  on CSX (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      CSX as b
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
deleted file mode 100644
index e0a2568..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
-      and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
new file mode 100644
index 0000000..37a119b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
+select element {'arec':a,'brec':b}
+from  DBLP as a,
+      DBLP as b
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
deleted file mode 100644
index f3ca957..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
new file mode 100644
index 0000000..584522a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ *                  DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ *                  We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
+select element {'arec':a,'brec':b,'jacc':jacc}
+from  DBLP as a,
+      DBLP as b
+with  jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
deleted file mode 100644
index 65029f3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.
- */
-drop dataverse join-super-key_1 if exists;
-
-create dataverse join-super-key_1;
-
-use dataverse join-super-key_1;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $li in dataset('LineItems')
-for $ps in dataset('PartSupp')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and 
-      $li.l_extendedprice = $ps.ps_supplycost
-return {
-  "l_partkey": $li.l_partkey
-}
-
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
new file mode 100644
index 0000000..0e7fc0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+drop  dataverse `join-super-key_1` if exists;
+create  dataverse `join-super-key_1`;
+
+use `join-super-key_1`;
+
+
+create type `join-super-key_1`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `join-super-key_1`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `join-super-key_1`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `join-super-key_1`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `join-super-key_1`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/join-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from  LineItems as li,
+      PartSupp as ps
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
deleted file mode 100644
index 831976f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.
- */
-drop dataverse join-super-key_01 if exists;
-
-create dataverse join-super-key_01;
-
-use dataverse join-super-key_01;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $ps in dataset('PartSupp')
-for $li in dataset('LineItems')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and 
-      $li.l_extendedprice = $ps.ps_supplycost
-return {
-  "l_partkey": $li.l_partkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
new file mode 100644
index 0000000..7247cdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+drop  dataverse `join-super-key_01` if exists;
+create  dataverse `join-super-key_01`;
+
+use `join-super-key_01`;
+
+
+create type `join-super-key_01`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `join-super-key_01`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `join-super-key_01`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `join-super-key_01`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `join-super-key_01`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/join-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from  PartSupp as ps,
+      LineItems as li
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nested_query_with_bcast.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nested_query_with_bcast.sqlpp
new file mode 100644
index 0000000..8e7a6e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nested_query_with_bcast.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type Country as open {
+  country_code : string,
+  country_name : string
+};
+create type TweetType as open {
+  id : int64,
+  country : string
+};
+
+create type StoredTweetType as open {
+  tid : uuid
+};
+
+create dataset targetDataset(StoredTweetType) primary key tid autogenerated;
+create dataset countryDataset(Country) primary key country_code;
+create dataset tweetDataset(TweetType) primary key id;
+
+insert into targetDataset (select object_merge(x, {"full-country" : (select value c.country_name from countryDataset c where c.country_code /*+ bcast */ = x.country )}) from tweetDataset x);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
deleted file mode 100644
index ad6833d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.
- */
-drop dataverse loj-super-key_01 if exists;
-
-create dataverse loj-super-key_01;
-
-use dataverse loj-super-key_01;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $li in dataset('LineItems')
-let $partsupp := 
-   for $ps in dataset('PartSupp')
-   where $li.l_partkey = $ps.ps_partkey 
-         and $li.l_suppkey = $ps.ps_suppkey  
-         and $li.l_extendedprice = $ps.ps_supplycost
-   return $ps
-return {
-  "li": $li,
-  "partsupp": $partsupp 
-} 
- 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
new file mode 100644
index 0000000..38bd3e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+drop  dataverse `loj-super-key_01` if exists;
+create  dataverse `loj-super-key_01`;
+
+use `loj-super-key_01`;
+
+
+create type `loj-super-key_01`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `loj-super-key_01`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `loj-super-key_01`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `loj-super-key_01`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `loj-super-key_01`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'li':li,'partsupp':partsupp}
+from  LineItems as li
+with  partsupp as (
+      select element ps
+      from  PartSupp as ps
+      where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
deleted file mode 100644
index e0678e9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.
- */
-drop dataverse loj-super-key_02 if exists;
-
-create dataverse loj-super-key_02;
-
-use dataverse loj-super-key_02;
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32, 
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string 
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
-
-create dataset LineItems(LineItemType)
-  primary key l_partkey, l_linenumber on group1;
-create dataset PartSupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey on group1;  
-
-
-for $ps in dataset('PartSupp')
-let $items := 
-   for $li in dataset('LineItems')   
-   where $li.l_partkey = $ps.ps_partkey 
-         and $li.l_suppkey = $ps.ps_suppkey  
-         and $li.l_extendedprice = $ps.ps_supplycost
-   return $li
-return {
-  "partsupp": $ps,
-  "li": $items 
-} 
- 
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
new file mode 100644
index 0000000..9705785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+drop  dataverse `loj-super-key_02` if exists;
+create  dataverse `loj-super-key_02`;
+
+use `loj-super-key_02`;
+
+
+create type `loj-super-key_02`.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type `loj-super-key_02`.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type `loj-super-key_02`.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type `loj-super-key_02`.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type `loj-super-key_02`.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
+create  dataset LineItems(LineItemType) primary key l_partkey,l_linenumber on group1;
+
+create  dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'partsupp':ps,'li':items}
+from  PartSupp as ps
+with  items as (
+      select element li
+      from  LineItems as li
+      where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-lojoin_with_meta-1.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-lojoin_with_meta-1.aql
deleted file mode 100644
index 4107ad2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-lojoin_with_meta-1.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-  id:int32,
-  num:int32
-}
-
-create type LineType as open {
-}
-
-create dataset Book(LineType) with meta(AuxiliaryType)
-primary key meta().id;
-
-create index NumIndex1 on Book(meta().num);
-create index NumIndex2 on Book(linenum:int32?) enforced;
-create index NumIndex3 on Book(count1:int32?) enforced;
-create index NumIndex4 on Book(count2:int32?) enforced;
-
-for $t1 in dataset Book
-where $t1.linenum < 10
-order by $t1.linenum
-return {
-"linenum1": $t1.linenum,
-"count1":$t1.count1,
-"t2info": for $t2 in dataset Book
-          where $t1.count1 /* +indexnl */= $t2.count2
-          order by $t2.linenum
-          return {"linenum2": $t2.linenum,
-                  "count2":$t2.count2}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-3.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-3.aql
deleted file mode 100644
index 63e105d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-3.aql
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-  id:int32,
-  num:int32
-}
-
-create type LineType as open {
-}
-
-create dataset Book(LineType) with meta(AuxiliaryType)
-primary key meta().id;
-
-create index NumIndex on Book(meta().num);
-
-for $x in dataset Book
-for $y in dataset Book
-where meta($x).num /*+ indexnl */ = meta($y).num
-return {"authx":$x.author,"authy":$y.author};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-4.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-4.aql
deleted file mode 100644
index 1fd6869..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index-nljoin_with_meta-4.aql
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-  id:int32,
-  num:int32
-}
-
-create type LineType as open {
-}
-
-create dataset Book1(LineType) with meta(AuxiliaryType)
-primary key meta().id;
-
-create dataset Book2(LineType) with meta(AuxiliaryType)
-primary key meta().id;
-
-create index NumIndex on Book1(meta().num);
-
-for $y in dataset Book2
-for $x in dataset Book1
-where meta($y).num /*+ indexnl */= meta($x).num
-return {"authx":$x.author,"authy":$y.author};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index_with_meta-1.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index_with_meta-1.aql
deleted file mode 100644
index bc3dc0a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/meta/secondary_index_with_meta-1.aql
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AuxiliaryType as open {
-  id:int32,
-  num:int32
-}
-
-create type LineType as open {
-}
-
-create dataset Book(LineType) with meta(AuxiliaryType)
-primary key meta().id;
-
-create index NumIndex on Book(meta().num);
-
-for $x in dataset Book
-where meta($x).num >10
-return $x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
new file mode 100644
index 0000000..792dde1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Testing that creating a nested composite key open index is successful and being used.
+ * Expected Res : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.openType as {
+  id : int
+};
+
+create  dataset ds1(openType) primary key id;
+create  dataset ds2(openType) primary key id;
+
+create  index idx  on ds2 (nested.fname:string, nested.lname:string);
+
+select element {'a':a,'b':b}
+from ds1 as a, ds2 as b
+where to_string(a.nested.fname) /*+ indexnl */ = b.nested.fname;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
new file mode 100644
index 0000000..438ea9b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Testing that creating a nested composite key open index is successful and being used.
+ * Expected Res : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.openType as {
+  id : int
+};
+
+create  dataset ds1(openType) primary key id;
+create  dataset ds2(openType) primary key id;
+
+create  index idx  on ds2 (nested.fname:string, nested.address.street:string, nested.address.zip:string);
+
+select element {'a':a,'b':b}
+from ds1 as a, ds2 as b
+where to_string(a.nested.fname) /*+ indexnl */ = b.nested.fname;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp
new file mode 100644
index 0000000..043fdf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : No index join because there's no hint and the probe type is unknown, non-enforced nested composite key
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp
new file mode 100644
index 0000000..8e71939
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : No index join because the probe type is unknown, non-enforced nested composite key
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp
new file mode 100644
index 0000000..49d6a46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : No index join because there's no hint, non-enforced nested composite key index
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
new file mode 100644
index 0000000..b10915d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (string), non-enforced nested composite
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
new file mode 100644
index 0000000..9dce6972
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (bigint), non-enforced nested composite
+ * Expected Res : Success
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.nested.c_i64) /*+ indexnl */ = t2.nested.c_i64
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
new file mode 100644
index 0000000..6f97c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : Testing that creating a nested composite key open index is successful and being used.
+ *  Expected Result : Success
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.TestType as {
+  id: int
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (nested.fname:string, nested.lname:string);
+
+select element emp
+from  testdst as emp
+where emp.nested.fname > 'Roger';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
new file mode 100644
index 0000000..0fc5b4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+/*
+ *  Description     : Testing that creating a nested composite key open index is successful and being used.
+ *  Expected Result : Success
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.TestType as {
+  id: int
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (nested.address.zip:string, nested.fname:string, nested.address.street:string);
+
+select element emp
+from  testdst as emp
+where emp.nested.address.zip > '97777';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
new file mode 100644
index 0000000..45f87fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.x: string);
+
+select t.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
new file mode 100644
index 0000000..534df4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_s on TestOpen(nested.c_s:string, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
new file mode 100644
index 0000000..ae384bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
new file mode 100644
index 0000000..d7aa320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
new file mode 100644
index 0000000..bc7b0a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2.0
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
new file mode 100644
index 0000000..094c8b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
new file mode 100644
index 0000000..37a6e97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2.5
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
new file mode 100644
index 0000000..6cd28df4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_d on TestOpen(nested.c_d:double, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_d >= 3.25
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
new file mode 100644
index 0000000..9024f4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 499 and t.nested.c_i8 < 99999
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
new file mode 100644
index 0000000..546a041
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+ /*
+  *  Description     : Testing that creating a nested composite key open index is successful and being used
+  *                    when multiple indexes match.
+  *  Expected Result : Success
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+
+create index idx_3 on TestOpen(nested.c_i64:int64, nested.extra: int);
+
+create index idx_4 on TestOpen(nested.c_i8:int8, nested.extra: int);
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2 and t.nested.c_i64 < 3
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
deleted file mode 100644
index f7504ad..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset LineItems(LineItemType)
-  primary key l_orderkey, l_linenumber on group1;
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-create dataset Customers(CustomerType) 
-  primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/nested_loj.adm";
-
-for $c in dataset('Customers')
-let $orders := 
-  for $o in dataset('Orders')
-  where $o.o_custkey = $c.c_custkey
-  let $items := 
-    for $l in dataset('LineItems')
-    where $l.l_orderkey = $o.o_orderkey
-    return $l
-  return {
-     "order": $o,
-     "items": $items
-  }  
-return {
-  "cust": $c,
-  "orders": $orders
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
new file mode 100644
index 0000000..aae756c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset LineItems(LineItemType) primary key l_orderkey,l_linenumber on group1;
+
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+create  dataset Customers(CustomerType) primary key c_custkey on group1;
+
+write output to asterix_nc1:"/tmp/nested_loj.adm";
+select element {'cust':c,'orders':orders}
+from  Customers as c
+with  orders as (
+      select element {'order':o,'items':items}
+      from  Orders as o
+      with  items as (
+            select element l
+            from  LineItems as l
+            where (l.l_orderkey = o.o_orderkey)
+        )
+      where (o.o_custkey = c.c_custkey)
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
deleted file mode 100644
index 544e715..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
new file mode 100644
index 0000000..190f2f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
deleted file mode 100644
index 3f9aa3f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey,
-  "o_totalprice": $o.o_totalprice
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
new file mode 100644
index 0000000..64e2395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
deleted file mode 100644
index 4eaf437..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
new file mode 100644
index 0000000..4fe9e18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
deleted file mode 100644
index b2adee0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
-  primary key o_orderkey on group1;
-
-load dataset Orders 
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
-  $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {  
-  "o_orderkey": $o.o_orderkey,
-  "o_custkey": $o.o_custkey,
-  "o_totalprice": $o.o_totalprice
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
new file mode 100644
index 0000000..c16810f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
deleted file mode 100644
index a909b11..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-drop dataverse pull-select-above-eq-join if exists;
-
-create dataverse pull-select-above-eq-join;
-
-use dataverse pull-select-above-eq-join;
-
-create type UserType as open {
-  uid: int32, 
-  name: string,
-  city: string,
-  lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
-  vid: int32, 
-  name: string,
-  city: string,
-  lottery_numbers: {{int32}}
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Users(UserType) 
-  primary key uid on group1;
-
-create dataset Visitors(VisitorType) 
-  primary key vid on group1;
-
-
-write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.name = $visitor.name
-      and $user.city != $visitor.city
-return {"uid": $user.uid, "vid": $visitor.vid}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
new file mode 100644
index 0000000..8634de5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+drop  dataverse `pull-select-above-eq-join` if exists;
+create  dataverse `pull-select-above-eq-join`;
+
+use `pull-select-above-eq-join`;
+
+
+create type `pull-select-above-eq-join`.UserType as
+{
+  uid : integer,
+  name : string,
+  city : string,
+  lottery_numbers : {{integer}}
+};
+
+create type `pull-select-above-eq-join`.VisitorType as
+{
+  vid : integer,
+  name : string,
+  city : string,
+  lottery_numbers : {{integer}}
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Users(UserType) primary key uid on group1;
+
+create  dataset Visitors(VisitorType) primary key vid on group1;
+
+write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
+select element {'uid':user.uid,'vid':visitor.vid}
+from  Users as user,
+      Visitors as visitor
+where ((user.name = visitor.name) and (user.city != visitor.city))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
deleted file mode 100644
index 4b4ed34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.
- */
-drop dataverse fuzzyjoin_080 if exists;
-
-create dataverse fuzzyjoin_080;
-
-use dataverse fuzzyjoin_080;
-
-create type DBLPType as open {
-  id: int32, 
-  dblpid: string,
-  title: string,
-  authors: string,
-  misc: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType) 
-  primary key id on group1;
-
-
-write output to asterix_nc1:'rttest/fuzzyjoin_080.adm';
-
-    for $paperDBLP in dataset('DBLP')
-    let $matches :=
-            for $paper in dataset('DBLP')
-            where $paper.authors = $paperDBLP.authors        
-            return $paper.title
-    return {'id': $paperDBLP.id, 'matches':$matches}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
new file mode 100644
index 0000000..535ae58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+drop  dataverse fuzzyjoin_080 if exists;
+create  dataverse fuzzyjoin_080;
+
+use fuzzyjoin_080;
+
+
+create type fuzzyjoin_080.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset DBLP(DBLPType) primary key id on group1;
+
+write output to asterix_nc1:"rttest/fuzzyjoin_080.adm";
+select element {'id':paperDBLP.id,'matches':matches}
+from  DBLP as paperDBLP
+with  matches as (
+      select element paper.title
+      from  DBLP as paper
+      where (paper.authors = paperDBLP.authors)
+  )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
deleted file mode 100644
index 0961e44..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * 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.
- */
-drop dataverse q08_group_by if exists;
-
-create dataverse q08_group_by;
-
-use dataverse q08_group_by;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32, 
-  o_custkey: int32, 
-  o_orderstatus: string, 
-  o_totalprice: double, 
-  o_orderdate: string, 
-  o_orderpriority: string,
-  o_clerk: string, 
-  o_shippriority: int32, 
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32, 
-  c_name: string, 
-  c_address: string, 
-  c_nationkey: int32, 
-  c_phone: string, 
-  c_acctbal: double, 
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create type SupplierType as closed {
-  s_suppkey: int32, 
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type RegionType as closed {
-  r_regionkey: int32,
-  r_name: string,
-  r_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32, 
-  p_name: string, 
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
-  primary key o_orderkey;
-create dataset Customer(CustomerType) 
-  primary key c_custkey;
-create dataset Supplier(SupplierType)
-  primary key s_suppkey;
-create dataset Nation(NationType) 
-  primary key n_nationkey;
-create dataset Region(RegionType)
-  primary key r_regionkey;
-create dataset Part(PartType)
-  primary key p_partkey;
-
-for $s in dataset("Supplier")
-    for $lnrcop in (
-      for $lnrco in (
-        for $l in dataset('LineItem')
-        for $nrco in (
-          for $o in dataset('Orders')
-          for $nrc in (
-            for $c in dataset('Customer')
-            for $nr in (
-              for $n1 in dataset('Nation')
-              for $r1 in dataset('Region')
-              where $n1.n_regionkey = $r1.r_regionkey and $r1.r_name = 'AMERICA'
-              return { "n_nationkey": $n1.n_nationkey }
-            )
-            where $c.c_nationkey = $nr.n_nationkey
-            return { "c_custkey": $c.c_custkey }
-          )
-          where $nrc.c_custkey = $o.o_custkey
-          return {
-            "o_orderdate" : $o.o_orderdate, 
-            "o_orderkey": $o.o_orderkey 
-          }
-        )
-        where $l.l_orderkey = $nrco.o_orderkey
-          and $nrco.o_orderdate >= '1995-01-01' 
-          and $nrco.o_orderdate <= '1996-12-31'
-        return {
-          "o_orderdate": $nrco.o_orderdate, 
-          "l_partkey": $l.l_partkey, 
-          "l_discount": $l.l_discount, 
-          "l_extendedprice": $l.l_extendedprice, 
-          "l_suppkey": $l.l_suppkey
-        }
-      )
-      for $p in dataset("Part")
-      where $p.p_partkey = $lnrco.l_partkey and $p.p_type = 'ECONOMY ANODIZED STEEL'
-      return {
-        "o_orderdate": $lnrco.o_orderdate, 
-        "l_discount": $lnrco.l_discount, 
-        "l_extendedprice": $lnrco.l_extendedprice, 
-        "l_suppkey": $lnrco.l_suppkey 
-      }
-    )
-    where $s.s_suppkey = $lnrcop.l_suppkey
-    return {
-      "o_orderdate": $lnrcop.o_orderdate, 
-      "l_discount": $lnrcop.l_discount, 
-      "l_extendedprice": $lnrcop.l_extendedprice, 
-      "l_suppkey": $lnrcop.l_suppkey, 
-      "s_nationkey": $s.s_nationkey
-    }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
new file mode 100644
index 0000000..4ff3d31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */
+
+drop  dataverse q08_group_by if exists;
+create  dataverse q08_group_by;
+
+use q08_group_by;
+
+
+create type q08_group_by.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type q08_group_by.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type q08_group_by.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create type q08_group_by.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type q08_group_by.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type q08_group_by.RegionType as
+ closed {
+  r_regionkey : integer,
+  r_name : string,
+  r_comment : string
+};
+
+create type q08_group_by.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+select element {'o_orderdate':lnrcop.o_orderdate,'l_discount':lnrcop.l_discount,'l_extendedprice':lnrcop.l_extendedprice,'l_suppkey':lnrcop.l_suppkey,'s_nationkey':s.s_nationkey}
+from  Supplier as s,
+      (
+    select element {'o_orderdate':lnrco.o_orderdate,'l_discount':lnrco.l_discount,'l_extendedprice':lnrco.l_extendedprice,'l_suppkey':lnrco.l_suppkey}
+    from  (
+        select element {'o_orderdate':nrco.o_orderdate,'l_partkey':l.l_partkey,'l_discount':l.l_discount,'l_extendedprice':l.l_extendedprice,'l_suppkey':l.l_suppkey}
+        from  LineItem as l,
+              (
+            select element {'o_orderdate':o.o_orderdate,'o_orderkey':o.o_orderkey}
+            from  Orders as o,
+                  (
+                select element {'c_custkey':c.c_custkey}
+                from  Customer as c,
+                      (
+                    select element {'n_nationkey':n1.n_nationkey}
+                    from  Nation as n1,
+                          Region as r1
+                    where ((n1.n_regionkey = r1.r_regionkey) and (r1.r_name = 'AMERICA'))
+                ) as nr
+                where (c.c_nationkey = nr.n_nationkey)
+            ) as nrc
+            where (nrc.c_custkey = o.o_custkey)
+        ) as nrco
+        where ((l.l_orderkey = nrco.o_orderkey) and (nrco.o_orderdate >= '1995-01-01') and (nrco.o_orderdate <= '1996-12-31'))
+    ) as lnrco,
+          Part as p
+    where ((p.p_partkey = lnrco.l_partkey) and (p.p_type = 'ECONOMY ANODIZED STEEL'))
+) as lnrcop
+where (s.s_suppkey = lnrcop.l_suppkey)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
deleted file mode 100644
index adb6efe..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.
- */
-drop dataverse q09_group_by if exists;
-
-create dataverse q09_group_by;
-
-use dataverse q09_group_by;
-
-create type LineItemType as closed {
-  l_orderkey: int32,
-  l_partkey: int32,
-  l_suppkey: int32,
-  l_linenumber: int32,
-  l_quantity: int32,
-  l_extendedprice: double,
-  l_discount: double,
-  l_tax: double,
-  l_returnflag: string,
-  l_linestatus: string,
-  l_shipdate: string,
-  l_commitdate: string,
-  l_receiptdate: string,
-  l_shipinstruct: string,
-  l_shipmode: string,
-  l_comment: string
-}
-
-create type OrderType as closed {
-  o_orderkey: int32,
-  o_custkey: int32,
-  o_orderstatus: string,
-  o_totalprice: double,
-  o_orderdate: string,
-  o_orderpriority: string,
-  o_clerk: string,
-  o_shippriority: int32,
-  o_comment: string
-}
-
-create type CustomerType as closed {
-  c_custkey: int32,
-  c_name: string,
-  c_address: string,
-  c_nationkey: int32,
-  c_phone: string,
-  c_acctbal: double,
-  c_mktsegment: string,
-  c_comment: string
-}
-
-create type SupplierType as closed {
-  s_suppkey: int32,
-  s_name: string,
-  s_address: string,
-  s_nationkey: int32,
-  s_phone: string,
-  s_acctbal: double,
-  s_comment: string
-}
-
-create type NationType as closed {
-  n_nationkey: int32,
-  n_name: string,
-  n_regionkey: int32,
-  n_comment: string
-}
-
-create type RegionType as closed {
-    r_regionkey: int32,
-    r_name: string,
-    r_comment: string
-}
-
-create type PartType as closed {
-  p_partkey: int32,
-  p_name: string,
-  p_mfgr: string,
-  p_brand: string,
-  p_type: string,
-  p_size: int32,
-  p_container: string,
-  p_retailprice: double,
-  p_comment: string
-}
-
-create type PartSuppType as closed {
-  ps_partkey: int32,
-  ps_suppkey: int32,
-  ps_availqty: int32,
-  ps_supplycost: double,
-  ps_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
-  primary key o_orderkey;
-create dataset Supplier(SupplierType)
-  primary key s_suppkey;
-create dataset Region(RegionType)
-  primary key r_regionkey;
-create dataset Nation(NationType)
-  primary key n_nationkey;
-create dataset Part(PartType)
-  primary key p_partkey;
-create dataset Partsupp(PartSuppType)
-  primary key ps_partkey, ps_suppkey;
-create dataset Customer(CustomerType)
-  primary key c_custkey;
-
-for $p in dataset('Part')
-    for $l2 in (
-      for $ps in dataset('Partsupp')
-      for $l1 in (
-        for $s1 in (
-          for $s in dataset('Supplier')
-          for $n in dataset('Nation')
-          where $n.n_nationkey = $s.s_nationkey
-          return {
-            "s_suppkey": $s.s_suppkey,
-            "n_name": $n.n_name
-          }
-        )
-        for $l in dataset('LineItem')
-        where $s1.s_suppkey = $l.l_suppkey
-        return  {
-          "l_suppkey": $l.l_suppkey,
-          "l_extendedprice": $l.l_extendedprice,
-          "l_discount": $l.l_discount,
-          "l_quantity": $l.l_quantity,
-          "l_partkey": $l.l_partkey,
-          "l_orderkey": $l.l_orderkey,
-          "n_name": $s1.n_name
-        }
-      )
-      where $ps.ps_suppkey = $l1.l_suppkey and $ps.ps_partkey = $l1.l_partkey
-      return {
-        "l_extendedprice": $l1.l_extendedprice,
-        "l_discount": $l1.l_discount,
-        "l_quantity": $l1.l_quantity,
-        "l_partkey": $l1.l_partkey,
-        "l_orderkey": $l1.l_orderkey,
-        "n_name": $l1.n_name,
-        "ps_supplycost": $ps.ps_supplycost
-      }
-    )
-    where contains($p.p_name, 'green') and $p.p_partkey = $l2.l_partkey
-    return {
-      "l_extendedprice": $l2.l_extendedprice,
-      "l_discount": $l2.l_discount,
-      "l_quantity": $l2.l_quantity,
-      "l_orderkey": $l2.l_orderkey,
-      "n_name": $l2.n_name,
-      "ps_supplycost": $l2.ps_supplycost
-    }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
new file mode 100644
index 0000000..44f4760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
@@ -0,0 +1,154 @@
+/*
+ * 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.
+ */
+
+drop  dataverse q09_group_by if exists;
+create  dataverse q09_group_by;
+
+use q09_group_by;
+
+
+create type q09_group_by.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : integer,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type q09_group_by.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type q09_group_by.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create type q09_group_by.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+create type q09_group_by.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+};
+
+create type q09_group_by.RegionType as
+ closed {
+  r_regionkey : integer,
+  r_name : string,
+  r_comment : string
+};
+
+create type q09_group_by.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+create type q09_group_by.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+create  dataset Partsupp(PartSuppType) primary key ps_partkey,ps_suppkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select element {'l_extendedprice':l2.l_extendedprice,'l_discount':l2.l_discount,'l_quantity':l2.l_quantity,'l_orderkey':l2.l_orderkey,'n_name':l2.n_name,'ps_supplycost':l2.ps_supplycost}
+from  Part as p,
+      (
+    select element {'l_extendedprice':l1.l_extendedprice,'l_discount':l1.l_discount,'l_quantity':l1.l_quantity,'l_partkey':l1.l_partkey,'l_orderkey':l1.l_orderkey,'n_name':l1.n_name,'ps_supplycost':ps.ps_supplycost}
+    from  Partsupp as ps,
+          (
+        select element {'l_suppkey':l.l_suppkey,'l_extendedprice':l.l_extendedprice,'l_discount':l.l_discount,'l_quantity':l.l_quantity,'l_partkey':l.l_partkey,'l_orderkey':l.l_orderkey,'n_name':s1.n_name}
+        from  (
+            select element {'s_suppkey':s.s_suppkey,'n_name':n.n_name}
+            from  Supplier as s,
+                  Nation as n
+            where (n.n_nationkey = s.s_nationkey)
+        ) as s1,
+              LineItem as l
+        where (s1.s_suppkey = l.l_suppkey)
+    ) as l1
+    where ((ps.ps_suppkey = l1.l_suppkey) and (ps.ps_partkey = l1.l_partkey))
+) as l2
+where (q09_group_by.contains(p.p_name,'green') and (p.p_partkey = l2.l_partkey))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
index 8da16a0..a46d514 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-159-3.sqlpp
@@ -32,7 +32,7 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 LEFT OUTER JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))>2 AND STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id DESC, fu2.id DESC
 LIMIT 5;
 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354.sqlpp
new file mode 100644
index 0000000..8586437
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+SELECT ds.DatasetName as v1
+FROM Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%"
+UNION ALL
+SELECT ds.DatasetName v1, idx.DatasetName v2, idx.IndexName v3
+FROM Metadata.`Index` idx, Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%" and idx.DatasetName LIKE "Orders%"
+ORDER BY v1, v2, v3;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408.sqlpp
new file mode 100644
index 0000000..e1e7d0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : This test case is to verify the fix for issue ASTERIXDB-2408
+ * https://issues.apache.org/jira/browse/ASTERIXDB-2408
+ * Expected Res :
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type testType as open
+{
+  id: int32
+};
+
+create dataset customers(testType) primary key id;
+create dataset orders(testType) primary key id;
+
+SELECT c.name, (
+  SELECT VALUE o.id
+  FROM orders o
+  WHERE o.customer_id = c.id
+) AS orders
+FROM customers c
+ORDER BY array_count(orders)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
index a05087b..942f39b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
index 4b94bf6..7dbbab9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
index 0a46e9a..4084992 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810.sqlpp
@@ -53,8 +53,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 /* +hash */
 GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
deleted file mode 100644
index 827b02a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : This test case is to verify the fix for issue601
- * https://code.google.com/p/asterixdb/issues/detail?id=601
- * Expected Res : SUCCESS
- * Date         : 10th Oct 2014
- */
-
-drop dataverse tpch if exists;
-create dataverse tpch;
-
-use dataverse tpch;
-
-create type LineItemType as closed {
-  l_orderkey: int32, 
-  l_partkey: int32, 
-  l_suppkey: int32, 
-  l_linenumber: int32, 
-  l_quantity: double, 
-  l_extendedprice: double,
-  l_discount: double, 
-  l_tax: double,
-  l_returnflag: string, 
-  l_linestatus: string, 
-  l_shipdate: string,
-  l_commitdate: string, 
-  l_receiptdate: string, 
-  l_shipinstruct: string, 
-  l_shipmode: string, 
-  l_comment: string
-}
-
-create dataset LineItem(LineItemType)
-  primary key l_orderkey, l_linenumber;
-  
-for $l in dataset('LineItem')
-group by $l_linenumber := $l.l_linenumber with $l
-return {
-  "l_linenumber": $l_linenumber,
-  "count_order": count($l)
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
new file mode 100644
index 0000000..a5861be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue601
+ * https://code.google.com/p/asterixdb/issues/detail?id=601
+ * Expected Res : SUCCESS
+ * Date         : 10th Oct 2014
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+select element {'l_linenumber':l_linenumber,'count_order':count(l)}
+from  LineItem as l
+group by l.l_linenumber as l_linenumber
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
deleted file mode 100644
index 18acdf0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date         : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in dataset s
-for $y in (
- for $z in {{ {"a":1, "c":1},{"a":2, "c":2},{"a":1, "c":null} }} where $x.b=$z.a
-    return $z.c
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp
new file mode 100644
index 0000000..ee21bc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date         : 2nd Feb. 2015
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+  b : integer
+};
+
+create  dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from  s as x,
+      (
+    select element z.c
+    from  {{{'a':1,'c':1},{'a':2,'c':2},{'a':1,'c':null}}} as z
+    where (x.b = z.a)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
deleted file mode 100644
index 3e2bea3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date         : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in {{ {"a":1},{"a":2} }}
-for $y in (
- for $z in dataset s where $x.a=$z.b
-    return $z.b
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp
new file mode 100644
index 0000000..5b0e7fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date         : 2nd Feb. 2015
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+  b : integer
+};
+
+create  dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from  {{{'a':1},{'a':2}}} as x,
+      (
+    select element z.b
+    from  s as z
+    where (x.a = z.b)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
deleted file mode 100644
index 2b6df8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue        : 730, 741
- * Expected Res : Success
- * Date         : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
-    screen-name: string,
-    lang: string,
-    friends-count: int32,
-    statuses-count: int32,
-    name: string,
-    followers-count: int32
-}
-
-create type TweetMessageType as closed {
-    tweetid: int64,
-        user: TwitterUserType,
-        sender-location: point,
-    send-time: datetime,
-        referred-topics: {{ string }},
-    message-text: string,
-    countA: int32,
-    countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text) type keyword;
-
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
-
-for $t1 in dataset('TweetMessages')
-let $n :=  create-circle($t1.sender-location, 0.5)
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"loc1":$t1.sender-location,
-"nearby-message": for $t2 in dataset('TweetMessages')
-                             where spatial-intersect($t2.sender-location, $n) and $t1.tweetid != $t2.tweetid
-                             order by $t2.tweetid
-                             return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
new file mode 100644
index 0000000..a30f22c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
deleted file mode 100644
index dcbf53f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Joins two datasets on the intersection of their point attributes.
- *                  The dataset 'MyData2' has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData1(MyRecord) primary key id;
-create dataset MyData2(MyRecord) primary key id;
-
-create index rtree_index on MyData2(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
-
-for $a in dataset('MyData1')
-for $b in dataset('MyData2')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
new file mode 100644
index 0000000..f87832e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Joins two datasets on the intersection of their point attributes.
+ *                  The dataset 'MyData2' has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+  id : integer,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create  dataset MyData1(MyRecord) primary key id;
+
+create  dataset MyData2(MyRecord) primary key id;
+
+create  index rtree_index  on MyData2 (point) type rtree;
+
+write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
+select element {'a':a,'b':b}
+from  MyData1 as a,
+      MyData2 as b
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
deleted file mode 100644
index 624fe9e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description    : Self-joins a dataset on the intersection of its point attribute.
- *                  The dataset has an RTree index, and we expect the 
- *                  join to be transformed into an indexed nested-loop join.
- * Success        : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
-  id: int32,
-  point: point,
-  kwds: string,
-  line1: line,
-  line2: line,
-  poly1: polygon,
-  poly2: polygon,
-  rec: rectangle
-}
-
-create dataset MyData(MyRecord) primary key id;
-
-create index rtree_index on MyData(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
-
-for $a in dataset('MyData')
-for $b in dataset('MyData')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
new file mode 100644
index 0000000..afe7a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Self-joins a dataset on the intersection of its point attribute.
+ *                  The dataset has an RTree index, and we expect the
+ *                  join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+  id : integer,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create  dataset MyData(MyRecord) primary key id;
+
+create  index rtree_index  on MyData (point) type rtree;
+
+write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
+select element {'a':a,'b':b}
+from  MyData as a,
+      MyData as b
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
deleted file mode 100644
index b8c7a8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.
- */
-/*
- * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
- * Expected Res : Success
- * Date         : 21th December 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
-    id : int32,
-    fname : string,
-    lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst') 
-where $emp.fname /*+ skip-index */ >= "Max" and $emp.fname <= "Roger" 
-return $emp
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
new file mode 100644
index 0000000..6235842
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
+ * Expected Res : Success
+ * Date         : 21th December 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+  id : integer,
+  fname : string,
+  lname : string
+};
+
+create  dataset testdst(TestType) primary key id;
+
+create  index sec_Idx  on testdst (fname) type btree;
+
+select element emp
+from  testdst as emp
+where ((emp.fname /*+ skip-index */  >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
deleted file mode 100644
index a3bb0fc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.
- */
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-
-create type EmploymentType as open {
-        organization-name: string,      
-        start-date: date,
-        end-date: date?
-}
-
-create type FacebookUserType as closed {
-        id: int32,
-        alias: string,
-        name: string,
-        user-since: datetime,
-        friend-ids: {{ int32 }},
-        employment: [EmploymentType]
-}
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-
-let $lonelyusers := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-let $lonelyusers2 := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-for $l1 in $lonelyusers
-for $l2 in $lonelyusers2
-where $l1.id < $l2.id
-order by $l1.id, $l2.id
-return { "user1": { "id": $l1.id, "name": $l1.name }, "user2": { "id": $l2.id, "name": $l2.name } };
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
new file mode 100644
index 0000000..c77559a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.EmploymentType as
+{
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type TinySocial.FacebookUserType as
+ closed {
+  id : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+
+with  lonelyusers as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  ),
+      lonelyusers2 as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  )
+select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
+from  lonelyusers as l1,
+      lonelyusers2 as l2
+where (l1.id < l2.id)
+order by l1.id,l2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
new file mode 100644
index 0000000..0a53f64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param $p_str:json="hello"
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-01.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = $p_str
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
new file mode 100644
index 0000000..71e9f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param args:json=["hello"]
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = $1
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
new file mode 100644
index 0000000..9019f68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param args:json=["hello"]
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = ?
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp
new file mode 100644
index 0000000..ee3501a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp
new file mode 100644
index 0000000..1387c1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp
new file mode 100644
index 0000000..15dec28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp
new file mode 100644
index 0000000..2b2c112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp
new file mode 100644
index 0000000..34f33a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp
new file mode 100644
index 0000000..b6478cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp
new file mode 100644
index 0000000..eaebd39
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/* default: set rewrite_in_as_or "true" */
+
+drop dataverse sampdb if exists;
+create dataverse sampdb;
+use sampdb;
+
+drop dataset samptable if exists;
+drop type samptabletype if exists;
+
+create type samptabletype as closed {
+  id: bigint
+};
+
+create dataset samptable(samptabletype) primary key id;
+
+select *
+from
+(
+  select id from samptable
+  where (id in [0] and id in [1])
+        or (id in [1] and id in [2])
+) st1;
+
+drop dataverse sampdb;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
index 76568bb..3850d35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+set rewrite_in_as_or "false";
+
 drop dataverse sampdb if exists;
 create dataverse sampdb;
 use sampdb;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
index 7819959..8d97b03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
@@ -79,14 +79,14 @@
 create  dataset Customers(CustomerType) primary key c_custkey on group1;
 
 write output to asterix_nc1:"/tmp/inlined_q18_large_volume_customer.adm";
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':inlined_q18_large_volume_customer.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':inlined_q18_large_volume_customer.strict_sum((
         select element j.l_quantity
         from  l as j
     ))}
 from  Customers as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':inlined_q18_large_volume_customer.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':inlined_q18_large_volume_customer.strict_sum((
             select element i.l_quantity
             from  l as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
index 174856f..c7f5afa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nest_aggregate.sqlpp
@@ -124,7 +124,7 @@
               Customer as customer
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
-        with  sum as tpch.coll_sum((
+        with  sum as tpch.strict_sum((
               select element o.o_totalprice
               from  orders as o
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
index cca0211..5ef7be6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
@@ -40,10 +40,10 @@
 create  dataset Orders(OrderType) primary key oid on group1;
 
 write output to asterix_nc1:"/tmp/orders-aggreg.adm";
-select element {'cid':cid,'ordpercust':`orders-aggreg`.coll_count(g),'totalcust':`orders-aggreg`.coll_sum((
+select element {'cid':cid,'ordpercust':`orders-aggreg`.strict_count(g),'totalcust':`orders-aggreg`.strict_sum((
         select element i.o.total
         from  g as i
-    )),'avgcust':`orders-aggreg`.coll_avg((
+    )),'avgcust':`orders-aggreg`.strict_avg((
         select element i.o.total
         from  g as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
index 6497eea..db8d8cd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
@@ -48,28 +48,28 @@
 load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
 
 write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.coll_sum((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':tpch.coll_sum((
+    )),'sum_base_price':tpch.strict_sum((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':tpch.coll_sum((
+    )),'sum_disc_price':tpch.strict_sum((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':tpch.coll_sum((
+    )),'sum_charge':tpch.strict_sum((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':tpch.coll_avg((
+    )),'ave_qty':tpch.strict_avg((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':tpch.coll_avg((
+    )),'ave_price':tpch.strict_avg((
         select element i.l.l_extendedprice
         from  g as i
-    )),'ave_disc':tpch.coll_avg((
+    )),'ave_disc':tpch.strict_avg((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':tpch.coll_count(g)}
+    )),'count_order':tpch.strict_count(g)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
index 9c33b03..509ed18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
@@ -86,7 +86,7 @@
 where (((c.c_mktsegment = 'BUILDING') and (c.c_custkey = o.o_custkey)) and ((l.l_orderkey = o.o_orderkey) and (o.o_orderdate < '1995-03-15') and (l.l_shipdate > '1995-03-15')))
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
-with  revenue as q3_shipping_priority.coll_sum((
+with  revenue as q3_shipping_priority.strict_sum((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  l as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
index b90b3ed..fd520e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
@@ -137,7 +137,7 @@
 where ((c.c_nationkey = o1.s_nationkey) and (c.c_custkey = o1.o_custkey))
 /* +hash */
 group by o1.n_name as n_name
-with  revenue as q5_local_supplier.coll_sum((
+with  revenue as q5_local_supplier.strict_sum((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  o1 as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
index 22bff52..d058dd3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
@@ -86,18 +86,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       with  phone_substr as tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.coll_count(g),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.strict_count(g),'totacctbal':tpch.strict_sum((
         select element i.ct.c_acctbal
         from  g as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (tpch.coll_count((
+where (tpch.strict_count((
     select element o
     from  Orders as o
     where (ct.c_custkey = o.o_custkey)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
index 2ce1638..eb73380 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue697.sqlpp
@@ -37,7 +37,7 @@
 
 create  dataset test(TestType) primary key key1;
 
-select element {'gid':aid,'avg':test.coll_avg((
+select element {'gid':aid,'avg':test.strict_avg((
         select element j.`value`
         from  i as j
         where j.`value` is not null
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
index ec19977..ee054f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue785.sqlpp
@@ -105,7 +105,7 @@
             Orders as orders
       where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = n.n_nationkey))
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
-      with  sum as tpch.coll_sum((
+      with  sum as tpch.strict_sum((
             select element o.o_totalprice
             from  orders as o
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
index a43ca13..bb54619 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810-2.sqlpp
@@ -51,7 +51,7 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
index aacbb72..0accf10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue810.sqlpp
@@ -51,7 +51,7 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.coll_count(cheap),'count_expensives':tpch.coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':tpch.strict_count(cheap),'count_expensives':tpch.strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
index e1d8b97..cc7231f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue827-2.sqlpp
@@ -51,30 +51,30 @@
 
 create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':tpch.coll_count((
+)),'count_order':tpch.strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
index a32530a..c77559a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/split-materialization.sqlpp
@@ -45,12 +45,12 @@
 with  lonelyusers as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   ),
       lonelyusers2 as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   )
 select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
 from  lonelyusers as l1,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
new file mode 100644
index 0000000..18ec417
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -0,0 +1,197 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SUBPLAN  |PARTITIONED|
+              {
+                -- AGGREGATE  |LOCAL|
+                  -- ASSIGN  |LOCAL|
+                    -- MICRO_PRE_CLUSTERED_GROUP_BY[$$184]  |LOCAL|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- IN_MEMORY_STABLE_SORT [$$184(ASC)]  |LOCAL|
+                        -- ASSIGN  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- SUBPLAN  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- ASSIGN  |LOCAL|
+                                          -- UNNEST  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- COMMIT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- INSERT_DELETE  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$168]  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$222]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$222(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$222]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$268][$$191]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- UNNEST  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- PRE_CLUSTERED_GROUP_BY[$$276]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[$$278, $$280]  |LOCAL|
+                                                                                    {
+                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                        -- STREAM_SELECT  |LOCAL|
+                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                    }
+                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$276(ASC), $$278(ASC), $$280(ASC)]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$276]  |PARTITIONED|
+                                                                        -- UNION_ALL  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- SPLIT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- STABLE_SORT [$$290(ASC)]  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- NESTED_LOOP  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- ASSIGN  |UNPARTITIONED|
+                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- STABLE_SORT [$$229(ASC)]  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- SPLIT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STABLE_SORT [$$290(ASC)]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- RTREE_SEARCH  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- NESTED_LOOP  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- ASSIGN  |UNPARTITIONED|
+                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- STABLE_SORT [$$229(ASC)]  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$191]  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$199, $$201][$$193, $$194]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$199, $$201]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
index c83a85c..88cb592 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate/constant-gby-agg.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$38]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$40]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$34]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$36]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
index 07fde95..586d5fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
@@ -2,16 +2,16 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$24]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$52(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+              -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
@@ -19,7 +19,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
index 8e1be33..d473d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
@@ -6,8 +6,8 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
+                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$26]  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index 4173afc..aeaf88f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
index 1917413..c81b6b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_06.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -17,13 +17,13 @@
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- SORT_GROUP_BY[$$64]  |PARTITIONED|
+                                      -- SORT_GROUP_BY[$$67]  |PARTITIONED|
                                               {
                                                 -- AGGREGATE  |LOCAL|
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
-                                        -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$57]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$60]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
index ef8a923..a050d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
new file mode 100644
index 0000000..80def88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$25(ASC), $$26(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INTERSECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
index 116ff8a..180edb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$10][$$11]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$26][$$27]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
index 5cb092c..cdea343 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
@@ -1,11 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- SORT_GROUP_BY[$$6]  |PARTITIONED|
+    -- SORT_GROUP_BY[$$28]  |PARTITIONED|
             {
               -- AGGREGATE  |LOCAL|
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
             }
-      -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+      -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
index b37868e..d3cf6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
@@ -3,22 +3,22 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$16]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$80]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$16][$$18]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$80][$$82]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
index f4a21e2..4deb445 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
@@ -12,7 +12,7 @@
                       -- ASSIGN  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$9][$$11]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$17][$$19]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
new file mode 100644
index 0000000..d38f1d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$102, $$103]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$102, $$103]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$l_returnflag, $$l_linestatus]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$92, $$93, $$94]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$92, $$93, $$94]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
index 629a05a..16c0765 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
@@ -9,35 +9,21 @@
                 -- STABLE_SORT [topK: 2147483647] [$$37(DESC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- STREAM_SELECT  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$39][$#1]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$24][$$39]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                          -- ASSIGN  |UNPARTITIONED|
-                                            -- UNNEST  |UNPARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
index d3510b3..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
@@ -1,93 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$42, $$44]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$42(ASC), $$44(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$42, $$44]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$8][$$17]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$8]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$46(ASC), $$5(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$3][$$26]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STREAM_PROJECT  |UNPARTITIONED|
-                                                  -- ASSIGN  |UNPARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- REPLICATE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$47(ASC), $$14(ASC)]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$12][$$35]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- UNNEST  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
-                                                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
-                                                    -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
index 93fe350..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
@@ -1,57 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
-            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
-                            {
-                              -- AGGREGATE  |LOCAL|
-                                -- IN_MEMORY_STABLE_SORT [$$4(ASC)]  |LOCAL|
-                                  -- STREAM_SELECT  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                            }
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- IN_MEMORY_HASH_JOIN [$$2][$$7]  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- UNNEST  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- SORT_MERGE_EXCHANGE [$$28(ASC), $$7(ASC) ]  |PARTITIONED|
-                                      -- STABLE_SORT [$$28(ASC), $$7(ASC)]  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- EXTERNAL_GROUP_BY[$$36]  |PARTITIONED|
-                                                  {
-                                                    -- AGGREGATE  |LOCAL|
-                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                  }
-                                            -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                                              -- EXTERNAL_GROUP_BY[$$6]  |PARTITIONED|
-                                                      {
-                                                        -- AGGREGATE  |LOCAL|
-                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                      }
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    -- UNNEST  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
index 0d7d538..1bb5b99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/gby_partitioning_property_01.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$59]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$61]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- SORT_GROUP_BY[$$49]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$51]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,8 +17,8 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$49][$$52]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$51][$$54]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
@@ -26,7 +26,7 @@
                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan
new file mode 100644
index 0000000..81b8787
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/sugar-06-distinct.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$48(DESC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 3] [$$48(DESC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$46]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                                  -- IN_MEMORY_STABLE_SORT [$$41(ASC)]  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
index e11e2a8..855fd33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -4,7 +4,7 @@
       -- INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- MATERIALIZE  |PARTITIONED|
-            -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
index 08635f7..75444af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
@@ -1,11 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
-    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$36(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$14][$$15]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$38]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -13,7 +13,7 @@
                         -- DATASOURCE_SCAN  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$13(ASC)] HASH:[$$15]  |PARTITIONED|
+              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$36(ASC)] HASH:[$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
index 5aad07e..01f4b08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
index c8aaaabc..6037e0c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$7(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$7(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
index a1426f3..2785605 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$6(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$5(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
index 362a1c4..fccc3c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
index ff74295..cafb02c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
index 503fd28..96493a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
@@ -1,7 +1,7 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
index 6194e08..19ce662 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -1,12 +1,12 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$8(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -5,7 +5,7 @@
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
           -- BTREE_SEARCH  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 3e4c9bc..5567df8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -32,33 +32,31 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 736928c..de31517 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$20]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$43]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -22,7 +22,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                         -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 3e4c9bc..5567df8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -13,7 +13,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -32,33 +32,31 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 5133569..fe5a2a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- BTREE_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 5133569..fe5a2a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -11,7 +11,7 @@
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- BTREE_SEARCH  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                          -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                 -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$20]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$43]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -22,7 +22,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                         -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 5b08bf5..5134f39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
@@ -3,18 +3,18 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$25]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$51]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+              -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$36][$$25]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$62][$$51]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
@@ -24,14 +24,14 @@
                                     -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$51]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- BTREE_SEARCH  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                             -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
index 7065d9e..97c2b45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
@@ -2,19 +2,19 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$63(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$43][$$27]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                           -- UNION_ALL  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
@@ -36,7 +36,7 @@
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$51(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
index 5668704..8f1ca57 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
@@ -2,19 +2,19 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$63(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$43][$$27]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
                           -- STREAM_SELECT  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
@@ -33,7 +33,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- BTREE_SEARCH  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                               -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index 7465d81..fdf94bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index 968246d..e05463e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$17]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$39]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index 7465d81..fdf94bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index 968246d..e05463e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- UNION_ALL  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index d667093..26fe296 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 509149f..fc16201 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$20][$$12]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$36][$$28]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$28]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index d667093..26fe296 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$38]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$21][$$13]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$37][$$29]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 509149f..fc16201 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$37]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -18,7 +18,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$24][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$40][$$30]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
index 0cd8363..53c00eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$37][$$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$60][$$47]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
@@ -26,7 +26,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$65(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$17]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$41][$$31]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -4,12 +4,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$29][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$52][$$39]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
@@ -17,7 +17,7 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- BTREE_SEARCH  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                              -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                     -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
index 64eb060..d6e272b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
@@ -4,8 +4,8 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$17, $$22, $$24][$$19, $$23, $$20]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$17, $$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$33, $$38, $$40][$$35, $$39, $$36]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$33, $$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
index 92adb3d..49e2b90 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$17, $$23, $$18][$$19, $$22, $$24]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$33, $$39, $$34][$$35, $$38, $$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,7 +12,7 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$19, $$24]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$35, $$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
new file mode 100644
index 0000000..58af75f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
@@ -0,0 +1,39 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$34]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$37][$$35]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
index d79e1f2..12abcc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
@@ -4,19 +4,19 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$17, $$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$40, $$41]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
                         -- STREAM_SELECT  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$17, $$18]  |PARTITIONED|
+                -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$40, $$41]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$21, $$17, $$25][$$22, $$19, $$20]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$17, $$25]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$44, $$40, $$48][$$45, $$42, $$43]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$40, $$48]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
index c1cd283..a82ff05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
@@ -4,25 +4,25 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$17, $$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$40, $$41]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
                         -- STREAM_SELECT  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
+                -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$22, $$17, $$18][$$21, $$19, $$25]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$45, $$40, $$41][$$44, $$42, $$48]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$19, $$25]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$42, $$48]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-lojoin_with_meta-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-lojoin_with_meta-1.plan
deleted file mode 100644
index 0ee7293..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-lojoin_with_meta-1.plan
+++ /dev/null
@@ -1,47 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$19(ASC), $$25(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
-                          -- STREAM_SELECT  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- BTREE_SEARCH  |PARTITIONED|
-                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                          -- ASSIGN  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- BTREE_SEARCH  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- BTREE_SEARCH  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- ASSIGN  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-3.plan
deleted file mode 100644
index 4d919c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-3.plan
+++ /dev/null
@@ -1,23 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-4.plan
deleted file mode 100644
index 4d919c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index-nljoin_with_meta-4.plan
+++ /dev/null
@@ -1,23 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- STREAM_SELECT  |PARTITIONED|
-            -- STREAM_PROJECT  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- BTREE_SEARCH  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- BTREE_SEARCH  |PARTITIONED|
-                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index_with_meta-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index_with_meta-1.plan
deleted file mode 100644
index 5a3c313..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/meta/secondary_index_with_meta-1.plan
+++ /dev/null
@@ -1,16 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- STREAM_SELECT  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ngram-edit-distance-inline.plan
index ae59870..374065e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ngram-edit-distance-inline.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -36,39 +36,37 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-edit-distance-inline.plan
index ae59870..374065e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-edit-distance-inline.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -36,39 +36,37 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-jaccard-inline.plan
index 612b08d..15d2b38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/olist-jaccard-inline.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ulist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ulist-jaccard-inline.plan
index 612b08d..15d2b38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ulist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/ulist-jaccard-inline.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$41(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
new file mode 100644
index 0000000..8d3d87e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
new file mode 100644
index 0000000..1cc11dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
new file mode 100644
index 0000000..ce57c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/03.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
new file mode 100644
index 0000000..ce57c34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/04.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$40(ASC), $$41(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$40(ASC), $$41(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
new file mode 100644
index 0000000..a90431b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$39][$$40]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
new file mode 100644
index 0000000..af9bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
new file mode 100644
index 0000000..af9bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$42(ASC), $$43(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
new file mode 100644
index 0000000..fc81846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
new file mode 100644
index 0000000..b85b5c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
new file mode 100644
index 0000000..e8619e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
new file mode 100644
index 0000000..a2fcba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
new file mode 100644
index 0000000..a2fcba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
new file mode 100644
index 0000000..6914f5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
new file mode 100644
index 0000000..9c6b8d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
new file mode 100644
index 0000000..00280ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- INTERSECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-edit-distance-inline.plan
index ae59870..374065e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-edit-distance-inline.plan
@@ -14,7 +14,7 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- BTREE_SEARCH  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                       -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -36,39 +36,37 @@
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ASSIGN  |PARTITIONED|
-            -- STREAM_SELECT  |PARTITIONED|
-              -- ASSIGN  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- NESTED_LOOP  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
-                          -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- REPLICATE  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- REPLICATE  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
index 08b156e..3bb5c88 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
@@ -4,10 +4,10 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$56]  |PARTITIONED|
                     {
                       -- AGGREGATE  |LOCAL|
-                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$19]  |LOCAL|
+                        -- MICRO_PRE_CLUSTERED_GROUP_BY[$$57]  |LOCAL|
                                 {
                                   -- AGGREGATE  |LOCAL|
                                     -- STREAM_SELECT  |LOCAL|
@@ -17,24 +17,24 @@
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                     }
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$18(ASC), $$19(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                -- STABLE_SORT [$$56(ASC), $$57(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$19][$$20]  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$18][$$22]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$57][$$58]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$57]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$56][$$60]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- DATASOURCE_SCAN  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.plan
index 7465d81..f0e258c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$25][$$15]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.plan
index 968246d..9e27aef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$26][$$14]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$25][$$14]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -20,7 +20,7 @@
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
                                           -- BROADCAST_EXCHANGE  |PARTITIONED|
@@ -35,24 +35,25 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_SELECT  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- STREAM_SELECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- REPLICATE  |PARTITIONED|
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- NESTED_LOOP  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -15,14 +15,14 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
index b0d8e64..8d5343d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
@@ -6,15 +6,15 @@
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- HYBRID_HASH_JOIN [$$18][$$19]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$34][$$35]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
index 3cc71e0..8d02861 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
@@ -3,26 +3,26 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$12]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$36]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+              -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$15][$$14]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$39][$$38]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push_limit.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push_limit.plan
index fcd5b75..09a484e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push_limit.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push_limit.plan
@@ -6,10 +6,9 @@
           -- STREAM_PROJECT  |PARTITIONED|
             -- SORT_MERGE_EXCHANGE [$$9(ASC) ]  |PARTITIONED|
               -- STREAM_LIMIT  |PARTITIONED|
-                -- STREAM_SELECT  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
index 1aca471..db613d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$78][$$106]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$169][$$197]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -12,15 +12,15 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$106]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$197]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$103][$$85]  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$194][$$176]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$79][$$81]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$79]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$170][$$172]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$170]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
@@ -28,11 +28,11 @@
                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$94][$$82]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$94]  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$185][$$173]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
                                         -- STREAM_SELECT  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
@@ -40,22 +40,22 @@
                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$173]  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$91][$$83]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$91]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$182][$$174]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$182]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$89][$$84]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$180][$$175]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$180]  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
index 9f48723..d6da05e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
@@ -4,7 +4,7 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$62][$$84]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$123][$$145]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- STREAM_SELECT  |PARTITIONED|
@@ -12,10 +12,10 @@
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$84]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- HYBRID_HASH_JOIN [$$63, $$64][$$84, $$69]  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$124, $$125][$$145, $$130]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ASSIGN  |PARTITIONED|
@@ -23,15 +23,15 @@
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$84, $$69]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$145, $$130]  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$65][$$69]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$126][$$130]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- HYBRID_HASH_JOIN [$$72][$$66]  |PARTITIONED|
-                                      -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$133][$$127]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -45,7 +45,7 @@
                                               -- DATASOURCE_SCAN  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
index 0f8b28f..fef4ae3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$140, $$141]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$149, $$150]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$140, $$141]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$108, $$109]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$149, $$150]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$117, $$118]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan
new file mode 100644
index 0000000..b255ab2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354.plan
@@ -0,0 +1,46 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$70(ASC), $$71(ASC), $$72(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$70(ASC), $$71(ASC), $$72(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- UNION_ALL  |PARTITIONED|
+              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- NESTED_LOOP  |PARTITIONED|
+                            -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan
new file mode 100644
index 0000000..640a064
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$39(ASC) ]  |PARTITIONED|
+            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$37]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$37][$$40]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
index 369f475..3651eba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- SORT_GROUP_BY[$$13]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$42]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$10]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$39]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
index 8908448..8b7d56e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
@@ -20,12 +20,12 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$14][$$16]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$38][$$40]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
                 -- STREAM_PROJECT  |UNPARTITIONED|
                   -- ASSIGN  |UNPARTITIONED|
                     -- UNNEST  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
index dd7b473..709d5d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
@@ -20,8 +20,8 @@
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- HYBRID_HASH_JOIN [$$15][$$14]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$15]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$39][$$38]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
                 -- ASSIGN  |UNPARTITIONED|
                   -- UNNEST  |UNPARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
index 3bc6933..80f1234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
@@ -2,16 +2,16 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$35(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$27]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$60(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- STREAM_SELECT  |LOCAL|
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$27]  |PARTITIONED|
+              -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- STREAM_SELECT  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
@@ -20,7 +20,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
@@ -8,7 +8,7 @@
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
index 0bfec2b..331d31f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index/rtree-sidx-idxonly-01-disable-idxonly.plan
@@ -12,7 +12,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- BTREE_SEARCH  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
+                            -- STABLE_SORT [$$56(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
index 807e509..0d9a257 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
@@ -7,7 +7,7 @@
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- BTREE_SEARCH  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization-above-join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization-above-join.plan
index 22bc323..a81a142 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization-above-join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization-above-join.plan
@@ -32,26 +32,26 @@
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- HYBRID_HASH_JOIN [$$prefixTokenLeft][$$prefixTokenRight]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$95]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- STREAM_SELECT  |LOCAL|
-                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$95(ASC), $$i(ASC)]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$tokenUnranked][$$tokenGroupped]  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$97]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$97(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$tokenUnranked][$$tokenGroupped]  |PARTITIONED|
                                                                                   -- HASH_PARTITION_EXCHANGE [$$tokenUnranked]  |PARTITIONED|
                                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- UNNEST  |PARTITIONED|
@@ -62,24 +62,19 @@
                                                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$tokenGroupped]  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- SORT_MERGE_EXCHANGE [$$99(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
-                                                                                  -- STABLE_SORT [$$99(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- SORT_GROUP_BY[$$128]  |PARTITIONED|
-                                                                                              {
-                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                              }
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$tokenGroupped]  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- SORT_MERGE_EXCHANGE [$$102(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                            -- STABLE_SORT [$$102(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                -- REPLICATE  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- SORT_GROUP_BY[$$130]  |PARTITIONED|
+                                                                                                        {
+                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                        }
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
                                                                                                     -- SORT_GROUP_BY[$$token]  |PARTITIONED|
                                                                                                             {
                                                                                                               -- AGGREGATE  |LOCAL|
@@ -97,23 +92,23 @@
                                                                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
-                                                      -- UNNEST  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$97]  |PARTITIONED|
-                                                                  {
-                                                                    -- AGGREGATE  |LOCAL|
-                                                                      -- STREAM_SELECT  |LOCAL|
-                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                  }
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                          -- UNNEST  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$97(ASC), $$i(ASC)]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- HYBRID_HASH_JOIN [$$tokenUnranked][$$tokenGroupped]  |PARTITIONED|
+                                                              -- PRE_CLUSTERED_GROUP_BY[$$97]  |PARTITIONED|
+                                                                      {
+                                                                        -- AGGREGATE  |LOCAL|
+                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                      }
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STABLE_SORT [$$97(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- REPLICATE  |PARTITIONED|
+                                                                          -- HYBRID_HASH_JOIN [$$tokenUnranked][$$tokenGroupped]  |PARTITIONED|
                                                                             -- HASH_PARTITION_EXCHANGE [$$tokenUnranked]  |PARTITIONED|
                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                 -- UNNEST  |PARTITIONED|
@@ -124,21 +119,19 @@
                                                                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                        -- HASH_PARTITION_EXCHANGE [$$tokenGroupped]  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                -- SORT_MERGE_EXCHANGE [$$102(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
-                                                                                  -- STABLE_SORT [$$102(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- SORT_GROUP_BY[$$130]  |PARTITIONED|
-                                                                                              {
-                                                                                                -- AGGREGATE  |LOCAL|
-                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                              }
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
-                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$tokenGroupped]  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- SORT_MERGE_EXCHANGE [$$102(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                      -- STABLE_SORT [$$102(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- SORT_GROUP_BY[$$130]  |PARTITIONED|
+                                                                                                  {
+                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                  }
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
                                                                                               -- SORT_GROUP_BY[$$token]  |PARTITIONED|
                                                                                                       {
                                                                                                         -- AGGREGATE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
index dab5a7b..e707957 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
@@ -2,8 +2,8 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$30(ASC), $$31(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$30(ASC), $$31(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$79(ASC), $$80(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$79(ASC), $$80(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- NESTED_LOOP  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
index 1ae1e60..a010ace 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$146]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$164]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$123]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$141]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -21,25 +21,25 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$143]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$161]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$137]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$137(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$132][$$129]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$150][$$147]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -66,7 +66,7 @@
                                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
index 7f916f16..6567388 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in.plan
@@ -2,13 +2,13 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$40(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$40(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$44(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$33]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$39]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- STREAM_SELECT  |LOCAL|
@@ -17,7 +17,7 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$33][$$21]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$39][$$29]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- STREAM_SELECT  |PARTITIONED|
@@ -25,7 +25,7 @@
                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
new file mode 100644
index 0000000..b36c3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
new file mode 100644
index 0000000..35d91ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
new file mode 100644
index 0000000..b36c3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
index 7f0bb3b..bf5fa63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated.plan
@@ -2,13 +2,13 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- SORT_MERGE_EXCHANGE [$$45(ASC) ]  |PARTITIONED|
-          -- STABLE_SORT [$$45(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$49(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
                 -- STREAM_SELECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$37]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$43]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- STREAM_SELECT  |LOCAL|
@@ -22,7 +22,7 @@
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                             }
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$37][$$36]  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$43][$$42]  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STREAM_SELECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
@@ -30,7 +30,7 @@
                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
index 7ef3e37..57e0fa0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$147]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$165]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$124]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$142]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -21,25 +21,25 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$144]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$162]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$138]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$138(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$133][$$130]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$151][$$148]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -66,7 +66,7 @@
                                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$130]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
new file mode 100644
index 0000000..10b5853
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
index ec9715e..9b5bedd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572.plan
@@ -5,20 +5,20 @@
         -- STREAM_PROJECT  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- STREAM_SELECT  |LOCAL|
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                       }
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                  -- STABLE_SORT [$$62(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- HYBRID_HASH_JOIN [$$58][$#4]  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$62][$#4]  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$56]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$60]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- STREAM_SELECT  |LOCAL|
@@ -27,22 +27,22 @@
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$56][$#3]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$60][$#3]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- STREAM_SELECT  |LOCAL|
                                                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$54][$#2]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$58][$#2]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$47]  |PARTITIONED|
+                                                          -- PRE_CLUSTERED_GROUP_BY[$$51]  |PARTITIONED|
                                                                   {
                                                                     -- AGGREGATE  |LOCAL|
                                                                       -- STREAM_SELECT  |LOCAL|
@@ -51,7 +51,7 @@
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- HYBRID_HASH_JOIN [$$47][$#1]  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$51][$#1]  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
index a647c7c..8887588 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1580.plan
@@ -3,72 +3,71 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
-          -- STREAM_PROJECT  |PARTITIONED|
-            -- SORT_MERGE_EXCHANGE [$$110(ASC) ]  |PARTITIONED|
-              -- STREAM_LIMIT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- STABLE_SORT [topK: 100] [$$110(ASC)]  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- STREAM_SELECT  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- SORT_GROUP_BY[$$120]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
-                              -- HASH_PARTITION_EXCHANGE [$$120]  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$98]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$102][$$105]  |PARTITIONED|
-                                          -- HASH_PARTITION_EXCHANGE [$$102]  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- HYBRID_HASH_JOIN [$$111][$$104]  |PARTITIONED|
-                                                  -- HASH_PARTITION_EXCHANGE [$$111]  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$101][$$116]  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$101]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- HYBRID_HASH_JOIN [$$100][$$114]  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$109(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 100] [$$109(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$119]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$100]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$103][$$106]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$111][$$105]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$111]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$102][$$116]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$102]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$101][$$114]  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                                     -- ASSIGN  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$116]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
index 906e52f..fff35d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581-correlated.plan
@@ -8,7 +8,7 @@
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$152]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- AGGREGATE  |LOCAL|
@@ -20,9 +20,9 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$144][$$145]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$152][$$153]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$132]  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$140]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- AGGREGATE  |LOCAL|
@@ -32,13 +32,13 @@
                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$132(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$140(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$132][$$133]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$140][$$141]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$112]  |PARTITIONED|
+                                                -- PRE_CLUSTERED_GROUP_BY[$$120]  |PARTITIONED|
                                                         {
                                                           -- AGGREGATE  |LOCAL|
                                                             -- AGGREGATE  |LOCAL|
@@ -48,25 +48,28 @@
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$112][$$122]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- REPLICATE  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- BTREE_SEARCH  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$120][$$130]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- REPLICATE  |PARTITIONED|
-                                                                    -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ASSIGN  |PARTITIONED|
@@ -78,56 +81,55 @@
                                                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$141]  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$126][$$125]  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$126]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$134][$$133]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
-                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$134]  |PARTITIONED|
-                                                                            {
-                                                                              -- AGGREGATE  |LOCAL|
-                                                                                -- AGGREGATE  |LOCAL|
-                                                                                  -- STREAM_SELECT  |LOCAL|
-                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                            }
-                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$134][$$136]  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$176]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                      -- HYBRID_HASH_JOIN [$$176][$$177]  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- REPLICATE  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
-                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- REPLICATE  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$125]  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$133]  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
@@ -140,13 +142,13 @@
                                                                             -- DATASOURCE_SCAN  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$145]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$153]  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$128][$$127]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$128]  |PARTITIONED|
-                                          -- PRE_CLUSTERED_GROUP_BY[$$146]  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$136][$$135]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$136]  |PARTITIONED|
+                                          -- PRE_CLUSTERED_GROUP_BY[$$154]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- AGGREGATE  |LOCAL|
@@ -155,18 +157,18 @@
                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- STABLE_SORT [$$146(ASC)]  |PARTITIONED|
+                                              -- STABLE_SORT [$$154(ASC)]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- HYBRID_HASH_JOIN [$$146][$$149]  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$154][$$157]  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                             -- STREAM_SELECT  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ASSIGN  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- PRE_CLUSTERED_GROUP_BY[$$150]  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$158]  |PARTITIONED|
                                                                             {
                                                                               -- AGGREGATE  |LOCAL|
                                                                                 -- AGGREGATE  |LOCAL|
@@ -176,9 +178,9 @@
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- HYBRID_HASH_JOIN [$$150][$$151]  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$158][$$159]  |PARTITIONED|
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- ASSIGN  |PARTITIONED|
@@ -194,7 +196,7 @@
                                                                                   -- ASSIGN  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                       -- REPLICATE  |PARTITIONED|
-                                                                                        -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
+                                                                                        -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
@@ -206,50 +208,52 @@
                                                                                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                        -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$157]  |PARTITIONED|
                                                           -- ASSIGN  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- HYBRID_HASH_JOIN [$$163][$$162]  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- ASSIGN  |PARTITIONED|
+                                                                -- HYBRID_HASH_JOIN [$$171][$$170]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- REPLICATE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
                                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                                          -- STREAM_SELECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- PRE_CLUSTERED_GROUP_BY[$$168]  |PARTITIONED|
-                                                                                      {
-                                                                                        -- AGGREGATE  |LOCAL|
-                                                                                          -- AGGREGATE  |LOCAL|
-                                                                                            -- STREAM_SELECT  |LOCAL|
-                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                      }
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- STREAM_SELECT  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$176]  |PARTITIONED|
+                                                                                          {
+                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                          }
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- HYBRID_HASH_JOIN [$$168][$$169]  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$176][$$177]  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                              -- REPLICATE  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- REPLICATE  |PARTITIONED|
-                                                                                            -- HASH_PARTITION_EXCHANGE [$$169]  |PARTITIONED|
-                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                  -- BTREE_SEARCH  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                -- HASH_PARTITION_EXCHANGE [$$177]  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                  -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                                                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                  -- HASH_PARTITION_EXCHANGE [$$170]  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ASSIGN  |PARTITIONED|
                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -260,7 +264,7 @@
                                                                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$127]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
index c335414..d0afd76 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1581.plan
@@ -16,7 +16,7 @@
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$141]  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$149]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- AGGREGATE  |LOCAL|
@@ -28,9 +28,9 @@
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$141][$$142]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$149][$$150]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$77]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$85]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- AGGREGATE  |LOCAL|
@@ -40,30 +40,35 @@
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$85(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$77][$$134]  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$77]  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |UNPARTITIONED|
-                                                    -- ASSIGN  |UNPARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                        -- REPLICATE  |UNPARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                            -- AGGREGATE  |UNPARTITIONED|
-                                                              -- AGGREGATE  |UNPARTITIONED|
-                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                  -- AGGREGATE  |PARTITIONED|
-                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                                        -- ASSIGN  |PARTITIONED|
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                -- HASH_PARTITION_EXCHANGE [$$134]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$85][$$142]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |UNPARTITIONED|
+                                                              -- ASSIGN  |UNPARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                  -- REPLICATE  |UNPARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                            -- AGGREGATE  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- STREAM_SELECT  |PARTITIONED|
@@ -94,7 +99,7 @@
                                                                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
@@ -108,7 +113,7 @@
                                                       -- STREAM_SELECT  |PARTITIONED|
                                                         -- ASSIGN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$143]  |PARTITIONED|
+                                                            -- PRE_CLUSTERED_GROUP_BY[$$151]  |PARTITIONED|
                                                                     {
                                                                       -- AGGREGATE  |LOCAL|
                                                                         -- AGGREGATE  |LOCAL|
@@ -117,30 +122,32 @@
                                                                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                     }
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$143(ASC)]  |PARTITIONED|
+                                                                -- STABLE_SORT [$$151(ASC)]  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$143][$$144]  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$143]  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |UNPARTITIONED|
-                                                                              -- ASSIGN  |UNPARTITIONED|
-                                                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                                  -- REPLICATE  |UNPARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$151][$$152]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |UNPARTITIONED|
+                                                                                  -- ASSIGN  |UNPARTITIONED|
                                                                                     -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-                                                                                      -- AGGREGATE  |UNPARTITIONED|
-                                                                                        -- AGGREGATE  |UNPARTITIONED|
-                                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-                                                                                            -- AGGREGATE  |PARTITIONED|
-                                                                                              -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                -- STREAM_SELECT  |PARTITIONED|
-                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                      -- REPLICATE  |UNPARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                                                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                                -- AGGREGATE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$144]  |PARTITIONED|
+                                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$152]  |PARTITIONED|
                                                                             -- ASSIGN  |PARTITIONED|
                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                 -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
index c2790a4..b5e9913 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$119]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$122]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$101]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$104]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,12 +17,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$101][$$108]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$110][$$105]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$113][$$108]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$113]  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
index 789992e..339e9ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$119]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$122]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$119]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$101]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$104]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,11 +17,11 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$101][$$108]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$109][$$105]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$112][$$108]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
index e9fafd7..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
@@ -1,33 +0,0 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$20]  |PARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- STREAM_SELECT  |LOCAL|
-                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$20(ASC), $$18(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$3][$$22]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- UNNEST  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- DATASOURCE_SCAN  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                          -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
index bca9e57..ceddca3 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
@@ -20,5 +20,5 @@
 select element {'name':user.name,'movie':mv.movie}
 from  User as user,
       Movie as mv
-where some i in user.interests satisfies (i.movie = mv.movie[?])
+where some i in user.interests satisfies (i.movie = mv.movie[0])
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
index 42e81f9..3e3575d 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
@@ -50,7 +50,7 @@
             Variable [ Name=$mv ]
             Field=movie
           ]
-          Index: ANY
+          Index:           LiteralExpr [LONG] [0]
         ]
       ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
index 1e0efba..b27701c 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
@@ -1,31 +1,6 @@
 Query:
 SELECT [
-FunctionCall asterix.sql-sum@1[
-  (
-    SELECT ELEMENT [
-    OperatorExpr [
-      FieldAccessor [
-        FieldAccessor [
-          Variable [ Name=#3 ]
-          Field=t
-        ]
-        Field=a
-      ]
-      *
-      FieldAccessor [
-        FieldAccessor [
-          Variable [ Name=#3 ]
-          Field=t
-        ]
-        Field=b
-      ]
-    ]
-    ]
-    FROM [      Variable [ Name=#1 ]
-      AS Variable [ Name=#3 ]
-    ]
-  )
-]
+Variable [ Name=#1 ]
 root
 ]
 FROM [  FunctionCall asterix.dataset@1[
@@ -40,19 +15,20 @@
     Variable [ Name=$t ]
     Field=id
   ]
-  GROUP AS Variable [ Name=#1 ]
+  GROUP AS Variable [ Name=#2 ]
   (
     t:=Variable [ Name=$t ]
   )
 
-Orderby
+Let Variable [ Name=#1 ]
+  :=
   FunctionCall asterix.sql-sum@1[
     (
       SELECT ELEMENT [
       OperatorExpr [
         FieldAccessor [
           FieldAccessor [
-            Variable [ Name=#2 ]
+            Variable [ Name=#3 ]
             Field=t
           ]
           Field=a
@@ -60,17 +36,19 @@
         *
         FieldAccessor [
           FieldAccessor [
-            Variable [ Name=#2 ]
+            Variable [ Name=#3 ]
             Field=t
           ]
           Field=b
         ]
       ]
       ]
-      FROM [        Variable [ Name=#1 ]
-        AS Variable [ Name=#2 ]
+      FROM [        Variable [ Name=#2 ]
+        AS Variable [ Name=#3 ]
       ]
     )
   ]
+Orderby
+  Variable [ Name=#1 ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
index 7deb117..65cedd3 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
@@ -4,32 +4,7 @@
   (
     LiteralExpr [STRING] [root]
     :
-    FunctionCall asterix.sql-sum@1[
-      (
-        SELECT ELEMENT [
-        OperatorExpr [
-          FieldAccessor [
-            FieldAccessor [
-              Variable [ Name=#3 ]
-              Field=t
-            ]
-            Field=a
-          ]
-          *
-          FieldAccessor [
-            FieldAccessor [
-              Variable [ Name=#3 ]
-              Field=t
-            ]
-            Field=b
-          ]
-        ]
-        ]
-        FROM [          Variable [ Name=#1 ]
-          AS Variable [ Name=#3 ]
-        ]
-      )
-    ]
+    Variable [ Name=#1 ]
   )
 ]
 ]
@@ -45,19 +20,20 @@
     Variable [ Name=$t ]
     Field=id
   ]
-  GROUP AS Variable [ Name=#1 ]
+  GROUP AS Variable [ Name=#2 ]
   (
     t:=Variable [ Name=$t ]
   )
 
-Orderby
+Let Variable [ Name=#1 ]
+  :=
   FunctionCall asterix.sql-sum@1[
     (
       SELECT ELEMENT [
       OperatorExpr [
         FieldAccessor [
           FieldAccessor [
-            Variable [ Name=#2 ]
+            Variable [ Name=#3 ]
             Field=t
           ]
           Field=a
@@ -65,17 +41,19 @@
         *
         FieldAccessor [
           FieldAccessor [
-            Variable [ Name=#2 ]
+            Variable [ Name=#3 ]
             Field=t
           ]
           Field=b
         ]
       ]
       ]
-      FROM [        Variable [ Name=#1 ]
-        AS Variable [ Name=#2 ]
+      FROM [        Variable [ Name=#2 ]
+        AS Variable [ Name=#3 ]
       ]
     )
   ]
+Orderby
+  Variable [ Name=#1 ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
index f65ede3..da7ba31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
@@ -35,5 +35,10 @@
         <output-dir compare="Text">secondary-index</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="metrics">
+      <compilation-unit name="secondary-index-index-only">
+        <output-dir compare="Text">secondary-index-index-only</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp_hdfs.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_sqlpp_hdfs.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! 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.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="failed">
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
deleted file mode 100644
index bcc0edc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-//polltimeoutsecs=10
-
-/query/service/status/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
new file mode 100644
index 0000000..286c097
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+//polltimeoutsecs=10
+// handlevariable=handle
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_double/divide_double.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_double/divide_double.1.query.aql
index 62b1112..f86aae9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_double/divide_double.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_double/divide_double.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c6/$c1,"result2": $c6/$c2,"result3": $c6/$c3,"result4": $c6/$c4,"result5": $c6/$c5, "result6": $c6/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c6/$c1,"result2": $c6/$c2,"result3": $c6/$c3,"result4": $c6/$c4,"result5": $c6/$c5, "result6": $c6/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_float/divide_float.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_float/divide_float.1.query.aql
index 0f938e3..dde8291 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_float/divide_float.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_float/divide_float.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c5/$c1,"result2": $c5/$c2,"result3": $c5/$c3,"result4": $c5/$c4,"result5": $c5/$c5, "result6": $c5/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c5/$c1,"result2": $c5/$c2,"result3": $c5/$c3,"result4": $c5/$c4,"result5": $c5/$c5, "result6": $c5/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.1.query.aql
index 17f19c0..e4993d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c2/$c1,"result2": $c2/$c2,"result3": $c2/$c3,"result4": $c2/$c4,"result5": $c2/$c5, "result6": $c2/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c2/$c1,"result2": $c2/$c2,"result3": $c2/$c3,"result4": $c2/$c4,"result5": $c2/$c5, "result6": $c2/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
new file mode 100644
index 0000000..1771d70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c2 div $c1,"result2": $c2 div $c2,"result3": $c2 div $c3,"result4": $c2 div $c4,"result5": $c2 div $c5, "result6": $c2 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.1.query.aql
index 0026e24..e9adbc2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c3/$c1,"result2": $c3/$c2,"result3": $c3/$c3,"result4": $c3/$c4,"result5": $c3/$c5, "result6": $c3/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c3/$c1,"result2": $c3/$c2,"result3": $c3/$c3,"result4": $c3/$c4,"result5": $c3/$c5, "result6": $c3/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
new file mode 100644
index 0000000..349211a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c3 div $c1,"result2": $c3 div $c2,"result3": $c3 div $c3,"result4": $c3 div $c4,"result5": $c3 div $c5, "result6": $c3 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.1.query.aql
index b3e11be..b7b99a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c4/$c1,"result2": $c4/$c2,"result3": $c4/$c3,"result4": $c4/$c4,"result5": $c4/$c5, "result6": $c4/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c4/$c1,"result2": $c4/$c2,"result3": $c4/$c3,"result4": $c4/$c4,"result5": $c4/$c5, "result6": $c4/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
new file mode 100644
index 0000000..c6694a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c4 div $c1,"result2": $c4 div $c2,"result3": $c4 div $c3,"result4": $c4 div $c4,"result5": $c4 div $c5, "result6": $c4 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.1.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.1.query.aql
index a3cf2ee..afb1488 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.1.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.1.query.aql
@@ -24,4 +24,4 @@
 let $c5 := float("-5.5f")
 let $c6 := double("-6.5d")
 let $c8 := null
-return {"result1": $c1/$c1,"result2": $c1/$c2,"result3": $c1/$c3,"result4": $c1/$c4,"result5": $c1/$c5, "result6": $c1/$c6, "result7": $c6/$c8, "result8": $c6/[1][1]}
+return {"result1": $c1/$c1,"result2": $c1/$c2,"result3": $c1/$c3,"result4": $c1/$c4,"result5": $c1/$c5, "result6": $c1/$c6, "result7": $c6/$c8, "result8": $c6/[1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
new file mode 100644
index 0000000..07a1910
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c1 div $c1,"result2": $c1 div $c2,"result3": $c1 div $c3,"result4": $c1 div $c4,"result5": $c1 div $c5, "result6": $c1 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1], "result9": $c6/0}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
index 44eb244..4d9403d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
@@ -115,7 +115,7 @@
   <test-case FilePath="objects">
     <compilation-unit name="closed-closed-fieldname-conflict_issue173">
       <output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
-      <expected-error>Closed fields 0 and 1 have the same field name "name"</expected-error>
+      <expected-error>Duplicate field name "name"</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index bc9e7f7..0ab61e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -36,11 +36,11 @@
 let $str8:=substring($str7,0)
 
 let $str9:="This is a test string"
-let $str10:=substring($str9,-1)
+let $str10:=substring($str9,-6)
 
 let $str11:="This is a test string"
 let $str12:="This is a another test string"
 let $str13:=substring(string-concat([$str11,$str12]),20)
 
-let $str14:=substring("UC Irvine",string-length("UC Irvine")/2 - 1)
+let $str14:=substring("UC Irvine",string-length("UC Irvine") div 2 - 1)
 return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
index 1c31ea0..f792e7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
@@ -30,6 +30,9 @@
 substring("ABCD", 0, 4),
 substring("UC Irvine", 3, string-length("UC Irvine") - 3),
 substring("UC Irvine", 0, string-length("UC Irvine")),
-substring(substring("UC Irvine", 3), 0, string-length("Irvine"))
+substring(substring("UC Irvine", 3), 0, string-length("Irvine")),
+substring('ABCD',-3,2),
+substring('ABCD',-10,1),
+substring('ABCD',1,-1)
 ]
 return $a
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
index 075af82..b0e8697 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
@@ -18,5 +18,6 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",-1)
-return {"result1": $c1}
+let $c1 := substring("HEllow",-3)
+let $c2 := substring("HEllow",-7)
+return {"result1": $c1, "result2": $c2}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.1.ddl.sqlpp
new file mode 100644
index 0000000..a50d20b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.TestType as
+{
+  c_id: string
+};
+
+create  dataset t1(TestType) primary key c_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.2.update.sqlpp
new file mode 100644
index 0000000..ec72ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into t1 select value t from [
+  {"c_id":"01","c_b":true, "c_d":900,"c_i":999999},
+  {"c_id":"02","c_b":true, "c_d":800,"c_i":888888},
+  {"c_id":"03","c_b":true, "c_d":700,"c_i":777777},
+  {"c_id":"05","c_b":false,"c_d":111,"c_i":111111}
+] t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.3.query.sqlpp
new file mode 100644
index 0000000..4658bb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/issue2411/issue2411.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select value c_b from t1
+group by c_b
+having count(c_d) between 2 and 4
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
index ed067b2..ca84a8b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
index 9b0eb7d..3faf34f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
index 53f0cdb..fe6d3cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   sql-coll_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-coll_sum().
+ * Description      :   sql-strict_sum() aggregate function must return the numeric array_sum, when non null values are given as input to sql-strict_sum().
  *                  :   Get the array_sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
index 9b412c3..98b06a8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
index 00f93a4..7cdf71b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
index 9424486..c8e7cf8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/sum_numeric_null/sum_numeric_null.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, sql-coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, sql-strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
index 38c0972..947d19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null/agg_null.3.query.sqlpp
@@ -22,4 +22,4 @@
 * Date         : Jun 2nd 2013
 */
 
-{'count1':coll_count([null]),'average1':coll_avg([null]),'sum1':coll_sum([null]),'min1':coll_min([null]),'max1':coll_max([null]),'count2':coll_count({{null,null}}),'average2':coll_avg({{null,null}}),'sum2':coll_sum({{null,null}}),'min2':coll_min({{null,null}}),'max2':coll_max({{null,null}})};
+{'count1':strict_count([null]),'average1':strict_avg([null]),'sum1':strict_sum([null]),'min1':strict_min([null]),'max1':strict_max([null]),'count2':strict_count({{null,null}}),'average2':strict_avg({{null,null}}),'sum2':strict_sum({{null,null}}),'min2':strict_min({{null,null}}),'max2':strict_max({{null,null}})};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
index 9ada73b..62506cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec/agg_null_rec.3.query.sqlpp
@@ -25,19 +25,19 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t.valplus
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element t.valplus
     from  Test as t
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element t.valplus
     from  Test as t
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element t.valplus
     from  Test as t
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element t.valplus
     from  Test as t
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
index 3643901..47887d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_null_rec_1/agg_null_rec_1.3.query.sqlpp
@@ -25,28 +25,28 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element i.val
     from  (
         select element t
         from  Test as t
     ) as i
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element i.val
     from  (
         select element t
         from  Test as t
     ) as i
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element i.valplus
     from  (
         select element t
         from  Test as t
     ) as i
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element i.valplus
     from  (
         select element t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
index c90641b..5577347 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number/agg_number.3.query.sqlpp
@@ -22,4 +22,4 @@
 * Date         : Jun 2nd 2013
 */
 
-{'count1':coll_count([float('2.0'),double('3.0'),93847382783847382,1]),'average1':coll_avg([float('2.0'),double('3.0'),93847382783847382,1]),'sum1':coll_sum([float('2.0'),double('3.0'),93847382783847382,1]),'min1':coll_min([float('2.0'),double('3.0'),93847382783847382,1]),'max1':coll_max([float('2.0'),double('3.0'),93847382783847382,1]),'count2':coll_count({{float('2.0'),double('3.0'),93847382783847382,1}}),'average2':coll_avg({{float('2.0'),double('3.0'),93847382783847382,1}}),'sum2':coll_sum({{float('2.0'),double('3.0'),93847382783847382,1}}),'min2':coll_min({{float('2.0'),double('3.0'),93847382783847382,1}}),'max2':coll_max({{float('2.0'),double('3.0'),93847382783847382,1}})};
+{'count1':strict_count([float('2.0'),double('3.0'),93847382783847382,1]),'average1':strict_avg([float('2.0'),double('3.0'),93847382783847382,1]),'sum1':strict_sum([float('2.0'),double('3.0'),93847382783847382,1]),'min1':strict_min([float('2.0'),double('3.0'),93847382783847382,1]),'max1':strict_max([float('2.0'),double('3.0'),93847382783847382,1]),'count2':strict_count({{float('2.0'),double('3.0'),93847382783847382,1}}),'average2':strict_avg({{float('2.0'),double('3.0'),93847382783847382,1}}),'sum2':strict_sum({{float('2.0'),double('3.0'),93847382783847382,1}}),'min2':strict_min({{float('2.0'),double('3.0'),93847382783847382,1}}),'max2':strict_max({{float('2.0'),double('3.0'),93847382783847382,1}})};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
index 42306d8..c7d7eb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/agg_number_rec/agg_number_rec.3.query.sqlpp
@@ -25,19 +25,19 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element t.valplus
     from  Test as t
-)),'average':test.coll_avg((
+)),'average':test.strict_avg((
     select element t.valplus
     from  Test as t
-)),'sum':test.coll_sum((
+)),'sum':test.strict_sum((
     select element t.valplus
     from  Test as t
-)),'min':test.coll_min((
+)),'min':test.strict_min((
     select element t.valplus
     from  Test as t
-)),'max':test.coll_max((
+)),'max':test.strict_max((
     select element t.valplus
     from  Test as t
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
index c9a79fb..816f452 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_distinct/avg_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': to_bigint(coll_avg(distinct [10,20,30,10,20,30,10])),
-   't2': to_bigint(coll_avg(distinct [null,missing,null,missing])),
-   't3': to_bigint(coll_avg(distinct [40,null,missing,50,40,null,missing,50,40])),
-   't4': to_bigint(coll_avg(distinct []))
- }
\ No newline at end of file
+   't1': to_bigint(strict_avg(distinct [10,20,30,10,20,30,10])),
+   't2': to_bigint(strict_avg(distinct [null,missing,null,missing])),
+   't3': to_bigint(strict_avg(distinct [40,null,missing,50,40,null,missing,50,40])),
+   't4': to_bigint(strict_avg(distinct []))
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
index ae1b1eb..a10da57 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double/avg_double.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-select element coll_avg((
+select element strict_avg((
     select element x
     from  [1.0,2.0,double('3.0')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
index 4c10066..a6a4537 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_double_null/avg_double_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.doubleField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
index d7a5263..05d6092 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_01/avg_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
index 8d2359f..7eb6e35 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_empty_02/avg_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
index 14c4bc5..c13d9a1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float/avg_float.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.float('1'),test.float('2'),test.float('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
index c8c3774..a694993 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_float_null/avg_float_nu.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.floatField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
index 6883fa8..90f54cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16/avg_int16.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.smallint('1'),test.smallint('2'),test.smallint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
index d50b0d1..4e2e8c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int16_null/avg_int16_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int16Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
index a235c84..d52ebde 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32/avg_int32.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [1,2,3] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
index 9618d2d..34c5c68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int32_null/avg_int32_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int32Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
index 813c9a668e..5afa4e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64/avg_int64.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.bigint('1'),test.bigint('2'),test.bigint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
index 8c063d3..230913f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int64_null/avg_int64_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int64Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
index 3d2c5ba..57ffb31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8/avg_int8.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_avg((
+select element test.strict_avg((
     select element x
     from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
index 74fb4c4..64b91ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_int8_null/avg_int8_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'average':test.coll_avg((
+{'average':test.strict_avg((
     select element x.int8Field
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
index 4043084..0085e3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_avg((
+select element strict_avg((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
index fa90f85..2bf8960 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_01/count_01.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element x
     from  [1,2,3] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
index a7d9b90..403f8b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_distinct/count_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_count(distinct [10,20,30,10,20,30,10]),
-   't2': coll_count(distinct [null,missing,null,missing]),
-   't3': coll_count(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_count(distinct [])
- }
\ No newline at end of file
+   't1': strict_count(distinct [10,20,30,10,20,30,10]),
+   't2': strict_count(distinct [null,missing,null,missing]),
+   't3': strict_count(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_count(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
index 383671d..312bfe0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_01/count_empty_01.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Success        : Yes
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
index 67e6e77..a324793 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_empty_02/count_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
index ec84f7c..406a0c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/count_null/count_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-{'count':test.coll_count((
+{'count':test.strict_count((
     select element x.doubleField
     from  Numeric as x
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
index feea992..cb308cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue395/issue395.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element l.name
     from  Employee as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
index deaa68a..1af45c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_0/issue412_0.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_count(['ASTERIX','Hyracks',null]);
+strict_count(['ASTERIX','Hyracks',null]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
index 9da7811..b1e3de5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue412_1/issue412_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-{'count':coll_count([1,60,null]),'average':coll_avg([1,60,null]),'sum':coll_sum([1,60,null]),'min':coll_min([1,60,null]),'max':coll_max([1,60,null])};
+{'count':strict_count([1,60,null]),'average':strict_avg([1,60,null]),'sum':strict_sum([1,60,null]),'min':strict_min([1,60,null]),'max':strict_max([1,60,null])};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
index b2c6c66..9ecf906 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list/issue425_min_hetero_list.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_min([23,748374857483]);
+strict_min([23,748374857483]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
index e12e644..a0134a6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_min_hetero_list_1/issue425_min_hetero_list_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_min([748374857483,23,0.5]);
+strict_min([748374857483,23,0.5]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
index 770b7dc..0f82177 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list/issue425_sum_hetero_list.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_sum([23,748374857483]);
+strict_sum([23,748374857483]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
index 3932e23..f6a1989 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue425_sum_hetero_list_1/issue425_sum_hetero_list_1.3.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-coll_sum([748374857483,23,0.5]);
+strict_sum([748374857483,23,0.5]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
index 2a3312e..25cef91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/issue531_string_min_max/issue531_string_min_max.3.query.sqlpp
@@ -27,10 +27,10 @@
 use test;
 
 
-select element {'min':test.coll_min((
+select element {'min':test.strict_min((
         select element l.name
         from  t1 as l
-    )),'max':test.coll_max((
+    )),'max':test.strict_max((
         select element l.name
         from  t1 as l
     ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
index 70f273f..8ceef73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_distinct/max_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_max(distinct [10,20,30,10,20,30,10]),
-   't2': coll_max(distinct [null,missing,null,missing]),
-   't3': coll_max(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_max(distinct [])
- }
\ No newline at end of file
+   't1': strict_max(distinct [10,20,30,10,20,30,10]),
+   't2': strict_max(distinct [null,missing,null,missing]),
+   't3': strict_max(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_max(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
index fa419e6..6e3e097 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_01/max_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_max((
+select element test.strict_max((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
index 1ad7c2b..36c14f9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/max_empty_02/max_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_max((
+select element test.strict_max((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
index 77684a0..e02eac6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_distinct/min_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_min(distinct [10,20,30,10,20,30,10]),
-   't2': coll_min(distinct [null,missing,null,missing]),
-   't3': coll_min(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_min(distinct [])
- }
\ No newline at end of file
+   't1': strict_min(distinct [10,20,30,10,20,30,10]),
+   't2': strict_min(distinct [null,missing,null,missing]),
+   't3': strict_min(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_min(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
index aa84453..2840b0f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_01/min_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_min((
+select element test.strict_min((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
index bd686b6..bc4fe9e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_empty_02/min_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_min((
+select element test.strict_min((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
index 1802fee..c55fb2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/min_mixed/min_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_min((
+select element strict_min((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
index f910f12..03edd51 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/query-issue400/query-issue400.3.query.sqlpp
@@ -23,6 +23,6 @@
  * Date         : 8th May 2013
  */
 
-coll_count((select element i
+strict_count((select element i
 from  [[1,2,3,4,5],[6,7,8,9]] as i
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
index b62910e..901517d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg/scalar_avg.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_avg([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_avg([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_avg([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_avg([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_avg([test.double('1'),test.double('2'),test.double('3')])
+with  i8 as test.strict_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_avg([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_avg([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_avg([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_avg([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_avg([test.double('1'),test.double('2'),test.double('3')])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
index d2655e4..60f4d9d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_empty/scalar_avg_empty.3.query.sqlpp
@@ -21,4 +21,4 @@
  * Success        : Yes
  */
 
-select element coll_avg([]);
+select element strict_avg([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
index 254deb0..4e08eba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_avg_null/scalar_avg_null.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_avg([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_avg([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_avg([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_avg([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_avg([test.double('1'),test.double('2'),test.double('3'),null])
+with  i8 as test.strict_avg([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_avg([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_avg([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_avg([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_avg([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_avg([test.double('1'),test.double('2'),test.double('3'),null])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
index a1a34a1..91650c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count/scalar_count.3.query.sqlpp
@@ -24,13 +24,13 @@
 use test;
 
 
-with  i8 as test.coll_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_count([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_count([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_count([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_count([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_count([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_count(['a','b','c'])
+with  i8 as test.strict_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_count([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_count([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_count([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_count([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_count([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_count(['a','b','c'])
 select element i
 from  [i8,i16,i32,i64,f,d,s] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
index d2fff80..d7e966d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_empty/scalar_count_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_count([]);
+select element test.strict_count([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
index 3892816..2d29ce0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_count_null/scalar_count_null.3.query.sqlpp
@@ -24,13 +24,13 @@
 use test;
 
 
-with  i8 as test.coll_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_count([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_count([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_count([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_count([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_count([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_count(['a','b','c',null])
+with  i8 as test.strict_count([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_count([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_count([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_count([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_count([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_count([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_count(['a','b','c',null])
 select element i
 from  [i8,i16,i32,i64,f,d,s] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
index ebcd83b..3892f59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max/scalar_max.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_max([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_max([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_max([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_max([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_max([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_max(['foo','bar','world']),
-      dt as test.coll_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
+with  i8 as test.strict_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_max([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_max([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_max([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_max([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_max([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_max(['foo','bar','world']),
+      dt as test.strict_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
index 7978919..98175ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_empty/scalar_max_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_max([]);
+select element test.strict_max([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
index b5bd86e..e4d9123 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_max_null/scalar_max_null.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_max([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_max([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_max([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_max([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_max([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_max(['foo','bar','world',null]),
-      dt as test.coll_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
+with  i8 as test.strict_max([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_max([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_max([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_max([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_max([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_max([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_max(['foo','bar','world',null]),
+      dt as test.strict_max([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
index d2e07b6..bb7d0db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min/scalar_min.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_min([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_min([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_min([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_min([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_min([test.double('1'),test.double('2'),test.double('3')]),
-      s as test.coll_min(['foo','bar','world']),
-      dt as test.coll_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
+with  i8 as test.strict_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_min([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_min([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_min([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_min([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_min([test.double('1'),test.double('2'),test.double('3')]),
+      s as test.strict_min(['foo','bar','world']),
+      dt as test.strict_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z')])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
index 8a9f7ec..df4b788 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_empty/scalar_min_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_min([]);
+select element test.strict_min([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
index 97cca36..3460ba8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_min_null/scalar_min_null.3.query.sqlpp
@@ -24,14 +24,14 @@
 use test;
 
 
-with  i8 as test.coll_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_min([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_min([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_min([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_min([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_min([test.double('1'),test.double('2'),test.double('3'),null]),
-      s as test.coll_min(['foo','bar','world',null]),
-      dt as test.coll_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
+with  i8 as test.strict_min([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_min([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_min([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_min([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_min([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_min([test.double('1'),test.double('2'),test.double('3'),null]),
+      s as test.strict_min(['foo','bar','world',null]),
+      dt as test.strict_min([test.datetime('2012-03-01T00:00:00Z'),test.datetime('2012-01-01T00:00:00Z'),test.datetime('2012-02-01T00:00:00Z'),null])
 select element i
 from  [i8,i16,i32,i64,f,d,s,dt] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
index 300860f..a35d8a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum/scalar_sum.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
-      i16 as test.coll_sum([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
-      i32 as test.coll_sum([test.integer('1'),test.integer('2'),test.integer('3')]),
-      i64 as test.coll_sum([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
-      f as test.coll_sum([test.float('1'),test.float('2'),test.float('3')]),
-      d as test.coll_sum([test.double('1'),test.double('2'),test.double('3')])
+with  i8 as test.strict_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3')]),
+      i16 as test.strict_sum([test.smallint('1'),test.smallint('2'),test.smallint('3')]),
+      i32 as test.strict_sum([test.integer('1'),test.integer('2'),test.integer('3')]),
+      i64 as test.strict_sum([test.bigint('1'),test.bigint('2'),test.bigint('3')]),
+      f as test.strict_sum([test.float('1'),test.float('2'),test.float('3')]),
+      d as test.strict_sum([test.double('1'),test.double('2'),test.double('3')])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
index a0d5516..4de084e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_empty/scalar_sum_empty.3.query.sqlpp
@@ -24,4 +24,4 @@
 use test;
 
 
-select element test.coll_sum([]);
+select element test.strict_sum([]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
index 70ba6ad..70778e4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/scalar_sum_null/scalar_sum_null.3.query.sqlpp
@@ -24,12 +24,12 @@
 use test;
 
 
-with  i8 as test.coll_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
-      i16 as test.coll_sum([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
-      i32 as test.coll_sum([test.integer('1'),test.integer('2'),test.integer('3'),null]),
-      i64 as test.coll_sum([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
-      f as test.coll_sum([test.float('1'),test.float('2'),test.float('3'),null]),
-      d as test.coll_sum([test.double('1'),test.double('2'),test.double('3'),null])
+with  i8 as test.strict_sum([test.tinyint('1'),test.tinyint('2'),test.tinyint('3'),null]),
+      i16 as test.strict_sum([test.smallint('1'),test.smallint('2'),test.smallint('3'),null]),
+      i32 as test.strict_sum([test.integer('1'),test.integer('2'),test.integer('3'),null]),
+      i64 as test.strict_sum([test.bigint('1'),test.bigint('2'),test.bigint('3'),null]),
+      f as test.strict_sum([test.float('1'),test.float('2'),test.float('3'),null]),
+      d as test.strict_sum([test.double('1'),test.double('2'),test.double('3'),null])
 select element i
 from  [i8,i16,i32,i64,f,d] as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
index 6991a74..01f6cef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_distinct/sum_distinct.1.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
  {
-   't1': coll_sum(distinct [10,20,30,10,20,30,10]),
-   't2': coll_sum(distinct [null,missing,null,missing]),
-   't3': coll_sum(distinct [40,null,missing,50,40,null,missing,50,40]),
-   't4': coll_sum(distinct [])
- }
\ No newline at end of file
+   't1': strict_sum(distinct [10,20,30,10,20,30,10]),
+   't2': strict_sum(distinct [null,missing,null,missing]),
+   't3': strict_sum(distinct [40,null,missing,50,40,null,missing,50,40]),
+   't4': strict_sum(distinct [])
+ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
index e8a1f4d..cff9f78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double/sum_double.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [1.0,2.0,3.0] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
index 7a17bab..7268e26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_double_null/sum_double_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.doubleField
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
index bfc0e2c..e269827 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_01/sum_empty_01.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [1,2,3] as x
     where (x > 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
index ce6abf4..bc029a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_empty_02/sum_empty_02.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.val
     from  Test as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
index 8f90964..8931bb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float/sum_float.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.float('1'),test.float('2'),test.float('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
index f4c25bb..e7e2d39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_float_null/sum_float_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.floatField
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
index 5147881..57bf229 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16/sum_int16.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.smallint('1'),test.smallint('2'),test.smallint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
index e57c3f4..a9e5336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int16_null/sum_int16_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int16Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
index 616e7d1..43ae045 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32/sum_int32.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.integer('1'),test.integer('2'),test.integer('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
index 3c6841e..8732973 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int32_null/sum_int32_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int32Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
index 503b0a8..243428a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64/sum_int64.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.bigint('1'),test.bigint('2'),test.bigint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
index 857e15f..f19157b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int64_null/sum_int64_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int64Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
index b22d31b..595ecca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8/sum_int8.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x
     from  [test.tinyint('1'),test.tinyint('2'),test.tinyint('3')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
index 473bb1a..ab30e0d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_int8_null/sum_int8_null.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element x.int8Field
     from  Numeric as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
index b8c98ef..175fcee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_mixed/sum_mixed.3.query.sqlpp
@@ -22,7 +22,7 @@
 * Date         : Jun 2nd 2013
 */
 
-select element coll_sum((
+select element strict_sum((
     select element x
     from  [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
index 0cb9c60..0354a48 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
index fdc8b57..d3ec516 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
index 263ce45..449c361 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   coll_sum() aggregate function must return the numeric sum, when non null values are given as input to coll_sum().
+ * Description      :   strict_sum() aggregate function must return the numeric sum, when non null values are given as input to strict_sum().
  *                  :   Get the sum for those tuples which are non null for salary fields.
  * Expected result  :   Success
  * Date             :   July 20th 2012
@@ -26,7 +26,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element l.sal
     from  tdst as l
     where l.sal IS NOT NULL
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
index 27624da..1c98590 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.1.ddl.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
index 10f6c7e..c30742c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.2.update.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
index 7b771d7..2d3f2e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/sum_numeric_null/sum_numeric_null.3.query.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description      :   Add numeric values with a null value, coll_sum() aggregate function must return null.
+ * Description      :   Add numeric values with a null value, strict_sum() aggregate function must return null.
  * Expected result  :   Success
  * Date             :   July 20th 2012
  */
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_sum((
+select element test.strict_sum((
     select element l.sal
     from  tdst as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp
new file mode 100755
index 0000000..c8673f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_append(t.`referred-topics`, "sth", 5) from TweetMessages t order by t.tweetid),
+  "t2": (select array_append([3, "John"], (select value v.compType from d1 v))),
+  "t3": (array_append([3], 7, null, missing)),    // missing
+  "t4": (array_append("non_array", 5)),           // null
+  "t5": (array_append(null, 3, 9)),               // null
+  "t6": (array_append("non_array", 5, missing)),  // missing
+  "t7": (array_append([], 5, 10, 12.0, "sth")),   // OK
+  "t8": (array_append(missing, 3, 9)),            // missing
+  "t9": (array_append([3], 3, [9], null, "sth")), // OK to add nulls
+  "t10": (select array_append(d.followers, "sth1", "sth2") from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp
new file mode 100755
index 0000000..35d959c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_concat(t.`referred-topics`, {{"sth", 5}}, {{3,2}}) from TweetMessages t order by t.tweetid),
+  "t2": (select array_concat([3, "John"], (select value v.compType from d1 v))),
+  "t3": (array_concat([3,5,1], null, missing)),   // missing
+  "t4": (array_concat([3,1,6], "non_array")),     // null
+  "t5": (array_concat(null, [3, 9])),             // null
+  "t6": (array_concat("non_array", [5], missing)),// missing
+  "t7": (array_concat([], [5, 10], [12.0], [])),  // OK
+  "t8": (array_concat(missing, 3, 9)),            // missing
+  "t9": (array_concat([3, missing], [3, [9], null, missing, null])),  // OK with nulls
+  "t10": (select array_concat(d.followers, ["sth", 5], [3,2]) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.4.query.sqlpp
new file mode 100755
index 0000000..805dd99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing different input list types
+* Expected Res : Error due to processing different list types
+*/
+
+use TinySocial;
+
+select array_concat([3,5,1], [2,1], {{3, 1}});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.5.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.5.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp
new file mode 100755
index 0000000..257c4dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp
new file mode 100755
index 0000000..4a0e7ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp
new file mode 100755
index 0000000..0441a31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_contains(t.`referred-topics`, "speed") from TweetMessages t order by t.tweetid),
+  "t2": (select array_contains([3,8,98,40], 8)),
+  "t3": (select array_contains([3,8,98,40], 40.0)),
+  "t4": (select array_contains([3,8,98,40], -3)),
+  "t5": (select array_contains([3,"sth",98,40], 98)),
+  "t6": (select array_contains([3,8,98,40], null)),
+  "t7": (select array_contains([3,8,98,40], missing)),
+  "t8": (select array_contains(missing, 6)),
+  "t9": (select array_contains(null, 6)),
+  "t10": (select array_contains(5, "sth")),
+  "t11": (select array_contains([5, {"id":77}, "sth"], "sth"))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp
new file mode 100755
index 0000000..d5f9bb38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [2,3]
+*/
+
+use TinySocial;
+
+select array_contains([5,1,9], [2,3]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp
new file mode 100755
index 0000000..afecfb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is {"id": 5}
+*/
+
+use TinySocial;
+
+select array_contains([5,{"id": 5},9], {"id": 5});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.2.update.sqlpp
new file mode 100755
index 0000000..d666e41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones", "John Green", "Emily Jones", "sth"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp
new file mode 100755
index 0000000..dd01484
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_distinct(array_append(t.`referred-topics`, "verizon", "platform")) from TweetMessages t order by t.tweetid),
+  "t2": (array_distinct([19, 5, 7, 7, 5, 2])),
+  "t3": (array_distinct([19, 5, 7, 7, 5.1, 5.0, 2, 7])),
+  "t4": (array_distinct([19, 5, "a", 7.5, "A", "a", "John", "a"])),
+  "t5": (array_distinct([19, missing, 7, null, 5, null])),
+  "t6": (array_distinct([3])),
+  "t7": (array_distinct("non_array")),
+  "t8": (array_distinct([])),
+  "t9": (array_distinct(missing)),
+  "t10": (array_distinct(null)),
+  "t11": (select array_distinct(d.followers) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
new file mode 100755
index 0000000..4773079
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [5]
+*/
+
+use TinySocial;
+
+select array_distinct([19, 3, [5]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.5.query.sqlpp
new file mode 100755
index 0000000..0a8bdd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is {"id": 5}
+*/
+
+use TinySocial;
+
+select array_distinct([19, 3, {"id": 5}]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.1.ddl.sqlpp
new file mode 100755
index 0000000..ad018d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.2.update.sqlpp
new file mode 100755
index 0000000..f63c82d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones", ["sth"]]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp
new file mode 100755
index 0000000..323d018
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (array_flatten([2, 3, [7,1,2], [8, [12, 13, 14]]], 1)),
+  "t2": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 1)),
+  "t3": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2)),
+  "t4": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 4)),
+  "t5": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 0)),
+  "t6": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], -1)),
+  "t7": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.0)),  // OK
+  "t8": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.1)),  // null
+  "t9": (array_flatten([2, 3, null, [7,1, missing, "a"], [8, [12, null, "b", 14, missing]]], 2)),
+  "t10": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], missing)),
+  "t11": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], null)),
+  "t12": (array_flatten(missing, 2)),
+  "t13": (array_flatten(null, 2)),
+  "t14": (array_flatten(null, missing)),
+  "t15": (array_flatten("non_array", 2)),
+  "t16": (array_flatten([1,2,3, [5,6,7]], "non-numeric")),
+  "t17": (select array_flatten(d.followers, 1) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.1.ddl.sqlpp
new file mode 100755
index 0000000..c1acf81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1?
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.2.update.sqlpp
new file mode 100755
index 0000000..4b7a23a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":null},
+{"id":2, "compType":{"sth":44}}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp
new file mode 100755
index 0000000..e2d834e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_ifnull(t.`referred-topics`) from TweetMessages t order by t.tweetid),
+  "t2": (select array_ifnull((select value v.compType from d1 v))),
+  "t3": (array_ifnull([null, null, 3, 4.9, null])),
+  "t4": (array_ifnull([missing, 2, "a"])),
+  "t5": (array_ifnull([4, 2, "a"])),
+  "t6": (array_ifnull([4, 2, null, missing])),
+  "t7": (array_ifnull([null, null, null])),           // null
+  "t8": (array_ifnull([missing, missing, missing])),  // null
+  "t9": (array_ifnull([missing, null, missing])),   // null
+  "t10": (array_ifnull("non_array")),               // null
+  "t11": (array_ifnull([])),                        // null
+  "t12": (array_ifnull(missing)),                   // missing
+  "t13": (array_ifnull(null))                       // null
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp
new file mode 100755
index 0000000..c0b65e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (array_insert([1,2,3], 0, "a", "b")),          // OK
+  "t2": (array_insert([1,2,3], 3, "a", "b")),          // OK
+  "t3": (array_insert([1,1,2,4], 3, "a", "b")),        // OK
+  "t4": (array_insert([1,1,2,4], 3, 7, "a", 7, "one more")),  // OK
+  "t5": (array_insert([1,2,3], 4, "a")),          // null, out of bound
+  "t6": (array_insert([1,2,3], -1, "a", "b")),    // OK
+  "t7": (array_insert([1,2,3], -4, "a", "b")),    // null, out of bound
+  "t8": (array_insert("non_array", 5, "val")),    // null
+  "t9": (array_insert("non_array", 5, missing)),  // missing
+  "t10": (array_insert([], 5, 10, 12.0, "sth")),  // null, out of bound
+  "t11": (array_insert([], 0, 10, 12.0, "sth")),  // OK
+  "t12": (array_insert([6], "a", 9)),             // null, position non-numeric
+  "t13": (array_insert([6], 1.0, 9)),             // OK
+  "t14": (array_insert([6], 1.5, 9)),             // null, position with decimals
+  "t15": (array_insert(null, 3, 9)),              // null
+  "t16": (array_insert(missing, 3, 9)),           // missing
+  "t17": (array_insert([6], 1, null, 9, null)),   // OK to insert nulls
+  "t18": (array_insert([6], null, 5, 9, null)),   // null
+  "t19": (array_insert([6], 3, null, missing, 9, null)),  // missing
+  "t20": (select array_insert(t.`referred-topics`, 0, 5) from TweetMessages t order by t.tweetid),
+  "t21": (select array_insert(d.followers, 0, 5, 3) from d1 d),
+  "t22": (array_insert([1,2,3], float("NaN"), "a", "b")),
+  "t23": (array_insert([1,2,3], float("INF"), "a", "b")),
+  "t24": (array_insert([1,2,3], float("-INF"), "a", "b"))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.3.query.sqlpp
new file mode 100755
index 0000000..b5bbbb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.3.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select t.tweetid, array_intersect(t.`referred-topics`, {{"t-mobile", "platform"}}, {{"t-mobile"}}) from TweetMessages t order by t.tweetid),
+  "t2": (select array_intersect([1, "John", 2], (select value v.id from d1 v), [2,4,1])),
+  "t3": (array_intersect([3,5,1], [5,7,3], [3,2,5,1])),
+  "t4": (array_intersect([3,5.0,1], [5,7,3], [3,2,5,1])),
+  "t5": (array_intersect([3,"a",1], ["a",7,3], [3,2,"a",1])),
+  "t6": (array_intersect([3,"a",1], ["A",7,3], [3,2,"a",1])),
+  "t7": (array_intersect([3,"a",null], ["A",7,missing], [3,2,"a",1, null])),
+  "t8": (array_intersect([3,"a",null], ["A",7,null], [3,2,"a",1, null])),
+  "t9": (array_intersect([3,missing,"a",null], [missing,"A",7,null], [3,2,"a",1, null,missing])),
+  "t10": (array_intersect([3,5,1], [7,3], [2,5,1])),
+  "t11": (array_intersect([3,5,1], missing, [2,5,1])),
+  "t12": (array_intersect([3,5,1], null, [2,5,1])),
+  "t13": (array_intersect([3,5,1], "non_array", [2,5,1])),
+  "t14": (array_intersect(missing, "non_array", [2,5,1])),
+  "t15": (array_intersect([], [], [])),
+  "t16": (array_intersect([], [3,2], [])),
+  "t17": (select array_intersect(d.followers, ["John Green", "sth"], ["sth", "John Green"]) from d1 d),
+  "t18": (array_intersect([1,2], [3,2,1], [1,2,3,4]))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.4.query.sqlpp
new file mode 100755
index 0000000..351797c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing different input list types
+* Expected Res : Error due to processing different list types
+*/
+
+use TinySocial;
+
+select array_intersect([3,5,1], [2,1], {{3, 1}});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.5.query.sqlpp
new file mode 100755
index 0000000..e8df7a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing deep equality
+* Expected Res : Error as deep equality is not yet supported
+*/
+
+use TinySocial;
+
+select array_intersect([3,5,1], [2,1], [5, [3,2]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_intersect/array_intersect.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.1.ddl.sqlpp
new file mode 100755
index 0000000..257c4dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.2.update.sqlpp
new file mode 100755
index 0000000..4a0e7ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.3.query.sqlpp
new file mode 100755
index 0000000..afdada2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_position(t.`referred-topics`, "speed") from TweetMessages t order by t.tweetid),
+  "t2": (select array_position([3,8,98,40], 8)),
+  "t3": (select array_position([3,8,98,40], 40.0)),
+  "t4": (select array_position([3,8,98,40], -3)),
+  "t5": (select array_position([3,"sth",98,40], 98)),
+  "t6": (select array_position([3,8,98,40], null)),
+  "t7": (select array_position([3,8,98,40], missing)),
+  "t8": (select array_position(missing, 6)),
+  "t9": (select array_position(null, 6)),
+  "t10": (select array_position(5, "sth")),
+  "t11": (select array_position([5, {"id":77}, "sth"], "sth"))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.4.query.sqlpp
new file mode 100755
index 0000000..429747e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [2,3]
+*/
+
+use TinySocial;
+
+select array_position([5,1,9], [2,3]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.5.query.sqlpp
new file mode 100755
index 0000000..4a657a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is {"id": 5}
+*/
+
+use TinySocial;
+
+select array_position([5,{"id": 5},9], {"id": 5});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.6.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_position/array_position.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp
new file mode 100755
index 0000000..1603570e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_prepend("sth", 5, t.`referred-topics`) from TweetMessages t order by t.tweetid),
+  "t2": (select array_prepend((select value v.compType from d1 v), [3, "John"])),
+  "t3": (array_prepend(7, null, missing, [3])),     // missing
+  "t4": (array_prepend(5, "non_array")),            // null
+  "t5": (array_prepend(3, 9, null)),                // null
+  "t6": (array_prepend(5, missing, "non_array")),   // missing
+  "t7": (array_prepend(5, 10, 12.0, "sth",[77, "val"])),   // OK
+  "t8": (array_prepend(5, 10, null, "sth",[77, "val"])),   // OK to insert nulls
+  "t9": (array_prepend(3, 9, missing)),              // missing
+  "t10": (select array_prepend("sth1", "sth2", d.followers) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp
new file mode 100755
index 0000000..44ff8f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_put(t.`referred-topics`, "t-mobile", 5) from TweetMessages t order by t.tweetid),
+  "t2": (array_put([3], 7, null, missing)),    // missing
+  "t3": (array_put("non_array", 5)),           // null
+  "t4": (array_put("non_array", 5, missing)),  // missing
+  "t5": (array_put([], 5, 10, 12.0, "sth")),   // OK
+  "t6": (array_put(missing, 3, 9)),            // missing
+  "t7": (array_put(null, 3, 9)),               // null
+  "t8": (array_put([3,2,"sth"], 3, 9, 9, 3, "sth")), // OK
+  "t9": (array_put([3,2,"sth"], 1, 5)),     // OK
+  "t10": (array_put([3,2,"sth"], null, 5)),   // null
+  "t11": (select array_put(d.followers, "sth1", "John Green", "sth2") from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp
new file mode 100755
index 0000000..8eee258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is coming from a sub-query
+*/
+
+use TinySocial;
+
+select array_put([3, "John"], (select value v.compType from d1 v));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.5.query.sqlpp
new file mode 100755
index 0000000..61c7d47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [9]
+*/
+
+use TinySocial;
+
+select array_put([3], 3, [9], 4, "sth");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.1.ddl.sqlpp
new file mode 100755
index 0000000..f016d76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.1.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.2.update.sqlpp
new file mode 100755
index 0000000..adf6a6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.2.update.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp
new file mode 100755
index 0000000..d36a9f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (array_range(1,6)),
+  "t2": (array_range(0,25,5)),
+  "t3": (array_range(0.1,2)),
+  "t4": (array_range(10, 3, -3)),
+  "t5": (array_range(-1, -8, -3)),
+  "t6": (array_range(1, 8, -3)),
+  "t7": (array_range(10, 2, 3)),
+  "t8": (array_range(1, 13, 0)),
+  "t9": (array_range(2, 2, 1)),
+  "t10": (array_range(2, 3, 1)),
+  "t11": (array_range(2, 9, 1.1)),
+  "t12": (array_range(2, 9, null)),
+  "t13": (array_range(2, missing, 1)),
+  "t14": (array_range(2, missing)),
+  "t15": (array_range(2, 7, "a")),
+  "t16": (array_range(2, "a", 1)),
+  "t17": (array_range("a", 7, 1)),
+  "t18": (array_range(5, 10, float("NaN"))),
+  "t19": (array_range(5, float("INF"), 1)),
+  "t20": (array_range(5, float("-INF"), -2))
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.4.query.sqlpp
new file mode 100755
index 0000000..314360d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing correct number of arguments required
+* Expected Res : Error, 2 or 3 args are required
+*/
+
+use TinySocial;
+
+select array_range(2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.5.query.sqlpp
new file mode 100755
index 0000000..e18dfdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing correct number of arguments required
+* Expected Res : Error, 2 or 3 args are required
+*/
+
+use TinySocial;
+
+select array_range(2,6,1,4,5);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp
new file mode 100755
index 0000000..ca1d2af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_remove(t.`referred-topics`, "t-mobile") from TweetMessages t order by t.tweetid),
+  "t2": (array_remove([3], 7, null, missing)),    // missing
+  "t3": (array_remove("non_array", 5)),           // null
+  "t4": (array_remove("non_array", 5, missing)),  // missing
+  "t5": (array_remove([], 5, 10, 12.0, "sth")),   // OK
+  "t6": (array_remove(missing, 3, 9)),            // missing
+  "t7": (array_remove(null, 3, 9)),               // null
+  "t8": (array_remove([3,2,"sth"], 3, 9, 9, 3, "sth")), // OK
+  "t9": (array_remove([3,2,"sth"], 1, null)),      // null
+  "t10": (select array_remove(d.followers, "sth1", "John Green") from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp
new file mode 100755
index 0000000..c684b1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is coming from a sub-query
+*/
+
+use TinySocial;
+
+select array_remove([3, "John"], (select value v.compType from d1 v));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp
new file mode 100755
index 0000000..e79c163
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [9]
+*/
+
+use TinySocial;
+
+select array_remove([3], 3, [9], 4, "sth");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_repeat/array_repeat.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_repeat/array_repeat.1.query.sqlpp
new file mode 100644
index 0000000..a5ebb4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_repeat/array_repeat.1.query.sqlpp
@@ -0,0 +1,38 @@
+
+/*
+ * 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.
+ */
+
+{
+  "t1": (select value array_repeat("a", 3)),
+  "t2": (array_repeat("a", 3)),
+  "t3": (array_repeat("a", 0)),
+  "t4": (array_repeat("a", -3)),  // null
+  "t5": (array_repeat("a", "a")),
+  "t6": (array_repeat("a", missing)),
+  "t7": (array_repeat(missing, 3)),
+  "t8": (array_repeat("a", null)),
+  "t9": (array_repeat(null, 3)),
+  "t10": (array_repeat({ "a": 1 }, 3)),
+  "t11": (array_repeat([1, 2], 3)),
+  "t12": (array_repeat("a", 4.0)),  // OK
+  "t13": (array_repeat("a", 4.1)),  // null
+  "t14": (array_repeat("a", double("INF"))),  // null
+  "t15": (array_repeat("a", double("-INF"))), // null
+  "t16": (array_repeat("a", double("NaN")))   // null
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp
new file mode 100755
index 0000000..217d8a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_replace(t.`referred-topics`, "t-mobile", "coffee-mobile") from TweetMessages t order by t.tweetid),
+  "t2": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2, -2)),
+  "t3": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2.0, -4)),
+  "t4": (array_replace([3,-2,1,5,"2",9,2,3,2,1,2], 2.0, -4)),
+  "t5": (array_replace([3,-2,"a",5,"2",9,"a",3,"A",1,2], "a", -2)),
+  "t6": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, -2)),
+  "t7": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, [-2])),
+  "t8": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 3)),
+  "t9": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 0)),
+  "t10": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 4)),
+  "t11": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 5)),
+  "t12": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 10)),
+  "t13": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, -3)),
+  "t14": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.0)),
+  "t15": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.2)),
+  "t16": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, "4")),
+  "t17": (array_replace("non_array", 1, -5, 3)),
+  "t18": (array_replace([], 1, -5, 3)),
+  "t19": (array_replace([3,2], 1, -5, 3)),
+  "t20": (array_replace([3,2,3], 3, missing, 8)),
+  "t21": (array_replace([3,2,3], 3, null, 8)),
+  "t22": (array_replace([3,2], null, 3, 3)),
+  "t23": (array_replace([3,null,2], null, 3, 3)),
+  "t24": (array_replace(missing, null, 3, 3)),
+  "t25": (select array_replace(d.followers, "Emily Jones", "sth") from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp
new file mode 100755
index 0000000..7cd1cd3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values
+*/
+
+use TinySocial;
+
+select array_replace([3,2,3], [3], 9);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.5.query.sqlpp
new file mode 100755
index 0000000..5cd8753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing correct number of arguments required
+* Expected Res : Error, 3 or 4 args are required
+*/
+
+use TinySocial;
+
+select array_replace([2,6],2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.6.query.sqlpp
new file mode 100755
index 0000000..61f682f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing correct number of arguments required
+* Expected Res : Error, 3 or 4 args are required
+*/
+
+use TinySocial;
+
+select array_replace([2,6],2,5,1,6);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.7.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.7.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.3.query.sqlpp
new file mode 100755
index 0000000..6f65b77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_reverse(t.`referred-topics`) from TweetMessages t order by t.tweetid),
+  "t2": (select array_reverse([5, 7, 19])),
+  "t3": (select array_reverse([5, 7.5, 19])),
+  "t4": (select array_reverse([5, 7.5, 19, "John"])),
+  "t5": (select array_reverse([3])),
+  "t6": (select array_reverse("non_array")),
+  "t7": (select array_reverse([])),
+  "t8": (select array_reverse(missing)),
+  "t9": (select array_reverse(null)),
+  "t10": (select array_reverse(d.followers) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_reverse/array_reverse.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.2.update.sqlpp
new file mode 100755
index 0000000..1426f85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones", "Abby"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp
new file mode 100755
index 0000000..55c3569
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_sort(t.`referred-topics`) from TweetMessages t order by t.tweetid),
+  "t2": (array_sort([19, 5, 7, 7, 5, 2])),
+  "t3": (array_sort([19, 5, 7, 7, 5.1, 5.0, 2])),
+  "t4": (array_sort([19, 5, "a", 7.5, "A", "John", "a"])),
+  "t5": (array_sort([19, missing, 7, null, 5, null])),
+  "t6": (array_sort([3])),
+  "t7": (array_sort("non_array")),
+  "t8": (array_sort([])),
+  "t9": (array_sort(missing)),
+  "t10": (array_sort(null)),
+  "t11": (select array_sort(d.followers) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp
new file mode 100755
index 0000000..65b0521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is [5]
+*/
+
+use TinySocial;
+
+select array_sort([19, 3, [5]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.5.query.sqlpp
new file mode 100755
index 0000000..cc1745e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing an array function that needs to compare elements
+* Expected Res : Error due to comparing non-primitive values, which is {"id": 5}
+*/
+
+use TinySocial;
+
+select array_sort([19, 3, {"id": 5}]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.1.ddl.sqlpp
new file mode 100755
index 0000000..ad018d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.2.update.sqlpp
new file mode 100755
index 0000000..2c88183
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":[{"a":"a_val1", "b": "b_val1"}, {"a":"a_val2", "b":"b_val2"}, {"a":"a_val3", "b":"b_val3"}]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.3.query.sqlpp
new file mode 100755
index 0000000..442c02a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.3.query.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (array_star([{"a":"a_val1", "b": "b_val1"}, {"a":"a_val2", "b":"b_val2"}, {"a":"a_val3", "b":"b_val3"}])),
+  "t2": (array_star((select value v from d1 v))),
+  "t3": (array_star([{"b": "b_val1", "a":"a_val1"}, {"a":"a_val2", "b":"b_val2"}, {"a":"a_val3", "b":"b_val3"}])),
+  "t4": (array_star([{"b": "b_val1", "a":"a_val1"}, {"b":"b_val2"}, {"a":"a_val3", "b":"b_val3"}])),
+  "t5": (array_star([{"b": "b_val1", "a":"a_val1"}, {"a": null, "b":"b_val2"}, {"a":"a_val3", "b":"b_val3"}])),
+  "t6": (array_star([{"b": "b_val1", "a":"a_val1"}, {"a": null, "b":"b_val2"}, {"a":"a_val3", "b":"b_val3", "c":"c_val3"}])),
+  "t7": (array_star([{"b": "b_val1", "a":5}, {"a": 3.2, "b":"b_val2"}, {"a":"a_val3", "b":"b_val3", "c":"c_val3"}])),
+  "t8": (array_star([{"b": "b_val1", "a":"a_val1"}, {}, {"a":"a_val3", "b":"b_val3"}])),
+  "t9": (array_star([{"b": "b_val1", "a":"a_val1"}, 5, {"a":"a_val3", "b":"b_val3"}])),
+  "t10": (array_star([{"b": "b_val1", "a":"a_val1"}, null, {"a":"a_val3", "b":"b_val3"}])),
+  "t11": (array_star([{"b": "b_val1", "a":"a_val1"}, missing, {"a":"a_val3", "b":"b_val3"}])),
+  "t12": (array_star([{"b": "b_val1", "a":"a_val1"}, 5, [3] ])),
+  "t13": (array_star([{}, {}, {}])),
+  "t14": (array_star([1, 2, 3])),
+  "t15": (array_star([1])),
+  "t16": (array_star([])),
+  "t17": (array_star("non_array")),
+  "t18": (array_star(missing)),
+  "t19": (array_star(null)),
+  "t20": (select array_star(d.followers) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.4.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_star/array_star.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.3.query.sqlpp
new file mode 100755
index 0000000..f87ab97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.3.query.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_symdiff(t.`referred-topics`, {{"coffee-mobile", "t-mobile"}}, {{"platform"}}) from TweetMessages t order by t.tweetid),
+  "t2": (select array_symdiff([1, "John", 6], (select value v.id from d1 v), [4,4,5,6])),
+  "t3": (array_symdiff([3,5,1], [5,7,3], [3,2,5,1])),
+  "t4": (array_symdiff([3,5.0,1], [7,3], [3,2,1])),
+  "t5": (array_symdiff([3,"a",1], [7,3], [3,2,1])),
+  "t6": (array_symdiff([3,"a",1], ["A",7,3], [3,2,"a",1])),
+  "t7": (array_symdiff([3,"a",null], ["A",7,missing], [3,2,"a",1, null])),
+  "t8": (array_symdiff([3,"a",null], ["A",7,missing], [3,2,"a",1])),
+  "t9": (array_symdiff([3,5,1], missing, [2,5,1])),
+  "t10": (array_symdiff([3,5,1], null, [2,5,1])),
+  "t11": (array_symdiff([3,5,1], "non_array", [2,5,1])),
+  "t12": (array_symdiff(missing, "non_array", [2,5,1])),
+  "t13": (array_symdiff([], [], [])),
+  "t14": (array_symdiff([2], [3,2], [])),
+  "t15": (select array_symdiff(d.followers, ["Emily Jones", "sth2"], ["sth", "sth2"]) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.4.query.sqlpp
new file mode 100755
index 0000000..15636a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing different input list types
+* Expected Res : Error due to processing different list types
+*/
+
+use TinySocial;
+
+select array_symdiff([3,5,1], [2,1], {{3, 1}});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.5.query.sqlpp
new file mode 100755
index 0000000..f74b160
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing deep equality
+* Expected Res : Error as deep equality is not yet supported
+*/
+
+use TinySocial;
+
+select array_symdiff([3,5,1], [2,1], [5, [3,2]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiff/array_symdiff.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.3.query.sqlpp
new file mode 100755
index 0000000..31759eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.3.query.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_symdiffn(t.`referred-topics`, {{"coffee-mobile", "t-mobile"}}, {{"platform"}}) from TweetMessages t order by t.tweetid),
+  "t2": (select array_symdiffn([1, "John", 6], (select value v.id from d1 v), [4,4,5,6])),
+  "t3": (array_symdiffn([3,5,1], [5,7,3], [3,2,1])),
+  "t4": (array_symdiffn([3,5.0,1], [7,3], [3,2,1])),
+  "t5": (array_symdiffn([3,"a",1], [7,3], [3,2,1])),
+  "t6": (array_symdiffn([3,"a",1], ["A",7,3, "a"], [3,2,"a",1], ["a"])),
+  "t7": (array_symdiffn([3,"a",null], ["A",7,missing], [3,2,"a",1, null])),
+  "t8": (array_symdiffn([3,"a",null], ["A",7,missing], [3,2,"a",1])),
+  "t9": (array_symdiffn([3,5,1], missing, [2,5,1])),
+  "t10": (array_symdiffn([3,5,1], null, [2,5,1])),
+  "t11": (array_symdiffn([3,5,1], "non_array", [2,5,1])),
+  "t12": (array_symdiffn(missing, "non_array", [2,5,1])),
+  "t13": (array_symdiffn([], [], [])),
+  "t14": (array_symdiffn([2], [3,2], [])),
+  "t15": (select array_symdiffn(d.followers, ["Emily Jones", "sth2"], ["sth", "sth2"], ["sth2", 4]) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.4.query.sqlpp
new file mode 100755
index 0000000..270f9a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing different input list types
+* Expected Res : Error due to processing different list types
+*/
+
+use TinySocial;
+
+select array_symdiffn([3,5,1], [2,1], {{3, 1}});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.5.query.sqlpp
new file mode 100755
index 0000000..c2b01c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing deep equality
+* Expected Res : Error as deep equality is not yet supported
+*/
+
+use TinySocial;
+
+select array_symdiffn([3,5,1], [2,1], [5, [3,2]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_symdiffn/array_symdiffn.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.3.query.sqlpp
new file mode 100755
index 0000000..8e87893
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_union(t.`referred-topics`, {{"t-mobile", "platform"}}, {{"t-mobile"}}) from TweetMessages t order by t.tweetid),
+  "t2": (select array_union([1, "John", 2], (select value v.id from d1 v), [2,4,1])),
+  "t3": (array_union([3,5,1], [5,7,3], [3,2,5,1])),
+  "t4": (array_union([3,5.0,1], [5,7,3], [3,2,5,1])),
+  "t5": (array_union([3,"a",1], ["a",7,3], [3,2,"a",1])),
+  "t6": (array_union([3,"a",1], ["A",7,3], [3,2,"a",1])),
+  "t7": (array_union([3,"a",null], ["A",7,missing], [3,2,"a",1, null])),
+  "t8": (array_union([3,"a",null], ["A",7,null], [3,2,"a",1, null])),
+  "t9": (array_union([3,missing,"a",null], [missing,"A",7,null], [3,2,"a",1, null,missing])),
+  "t10": (array_union([3,5,1], [7,3], [2,5,1])),
+  "t11": (array_union([3,5,1], missing, [2,5,1])),
+  "t12": (array_union([3,5,1], null, [2,5,1])),
+  "t13": (array_union([3,5,1], "non_array", [2,5,1])),
+  "t14": (array_union(missing, "non_array", [2,5,1])),
+  "t15": (array_union([], [], [])),
+  "t16": (array_union([], [3,2], [])),
+  "t17": (select array_union(d.followers, ["John Green", "sth"], ["sth", "1"]) from d1 d)
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.4.query.sqlpp
new file mode 100755
index 0000000..211890d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing different input list types
+* Expected Res : Error due to processing different list types
+*/
+
+use TinySocial;
+
+select array_union([3,5,1], [2,1], {{3, 1}});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.5.query.sqlpp
new file mode 100755
index 0000000..06797db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+* Description  : Testing deep equality
+* Expected Res : Error as deep equality is not yet supported
+*/
+
+use TinySocial;
+
+select array_union([3,5,1], [2,1], [5, [3,2]]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.6.ddl.sqlpp
new file mode 100755
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_union/array_union.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
index 1ae94ec..e5a1c1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/AsyncDeferredQueries.xml
@@ -53,6 +53,7 @@
         <compilation-unit name="async-exhausted-result">
             <output-dir compare="Text">async-exhausted-result</output-dir>
             <expected-error>HYR0093</expected-error>
+            <source-location>false</source-location>
         </compilation-unit>
     </test-case>
 </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/concat2/concat2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/concat2/concat2.1.query.sqlpp
new file mode 100644
index 0000000..782cce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/concat2/concat2.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+{
+  't1': binary_concat([null,hex('aa')]),
+  't2': binary_concat([hex('aa'),null]),
+  't3': binary_concat([null,missing,hex('aa')]),
+  't4': binary_concat([hex('aa'),null,missing])
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp
new file mode 100644
index 0000000..c11dd76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": missingif(missing, missing) is missing,
+  "t2": missingif(missing, 2) is missing,
+  "t3": missingif(2, missing) is missing,
+  "t4": missingif(missing, null) is missing,
+  "t5": missingif(null, missing) is missing,
+  "t6": missingif(null, null) is null,
+  "t7": missingif(null, 2) is null,
+  "t8": missingif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp
new file mode 100644
index 0000000..b5f08ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_missing(missing_if(t1[1], t2[1]), "MISSING") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp
new file mode 100644
index 0000000..44e64d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": nanif(missing, missing) is missing,
+  "t2": nanif(missing, 2) is missing,
+  "t3": nanif(2, missing) is missing,
+  "t4": nanif(missing, null) is missing,
+  "t5": nanif(null, missing) is missing,
+  "t6": nanif(null, null) is null,
+  "t7": nanif(null, 2) is null,
+  "t8": nanif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp
new file mode 100644
index 0000000..05e003e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(nan_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp
new file mode 100644
index 0000000..abd4a4b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": neginfif(missing, missing) is missing,
+  "t2": neginfif(missing, 2) is missing,
+  "t3": neginfif(2, missing) is missing,
+  "t4": neginfif(missing, null) is missing,
+  "t5": neginfif(null, missing) is missing,
+  "t6": neginfif(null, null) is null,
+  "t7": neginfif(null, 2) is null,
+  "t8": neginfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp
new file mode 100644
index 0000000..67dcd34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(neginf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp
new file mode 100644
index 0000000..69ef8e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": nullif(missing, missing) is missing,
+  "t2": nullif(missing, 2) is missing,
+  "t3": nullif(2, missing) is missing,
+  "t4": nullif(missing, null) is missing,
+  "t5": nullif(null, missing) is missing,
+  "t6": nullif(null, null) is null,
+  "t7": nullif(null, 2) is null,
+  "t8": nullif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp
new file mode 100644
index 0000000..d5d6be3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_null(null_if(t1[1], t2[1]), "NULL") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp
new file mode 100644
index 0000000..399f4d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": posinfif(missing, missing) is missing,
+  "t2": posinfif(missing, 2) is missing,
+  "t3": posinfif(2, missing) is missing,
+  "t4": posinfif(missing, null) is missing,
+  "t5": posinfif(null, missing) is missing,
+  "t6": posinfif(null, null) is null,
+  "t7": posinfif(null, 2) is null,
+  "t8": posinfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp
new file mode 100644
index 0000000..cf25c1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+from
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+  [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(posinf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp
new file mode 100644
index 0000000..9083866
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.SupplierType as closed {
+  s_suppkey : int32,
+  s_name : string,
+  s_address : string,
+  s_nationkey : int32,
+  s_phone : string,
+  s_acctbal : float,
+  s_comment : string
+};
+
+create  dataset Supplier(SupplierType) primary key s_name;
+
+
+load  dataset Supplier using localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp
new file mode 100644
index 0000000..128f64f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select value s from Supplier s where 1 = s.s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp
new file mode 100644
index 0000000..f7ed2d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select value s from Supplier s where s.s_suppkey <= 5 order by s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp
new file mode 100644
index 0000000..8d10856
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select value s from Supplier s where s.s_nationkey = 11;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp
new file mode 100644
index 0000000..6412265
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.5.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use tpch;
+
+select value s from Supplier s where s.s_acctbal < 4500 order by s_suppkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp
new file mode 100644
index 0000000..21d5a43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/secondary_idx_lookup/secondary_idx_lookup.6.update.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse tpch;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
index 51668ed..7c6bfa8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv03/cross-dv03.3.query.sqlpp
@@ -26,7 +26,7 @@
  * Date         : 28th Aug 2012
  */
 
-select element coll_count((
+select element strict_count((
     select element l
     from  `Metadata`.`Dataset` as l
     where ((l.DataverseName = 'student') or (l.DataverseName = 'teacher'))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
index d5afe16..0be9caa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
@@ -24,5 +24,5 @@
 from  Customers as c
 with  rec as c.lastorder,
       m as [c.cid,rec.oid],
-      n as [m[?],m[1],m[4]]
+      n as [m[0],m[1],m[4]]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
index 3344c77..71a0714 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
@@ -24,5 +24,5 @@
 from  Customers as c
 with  rec as c.lastorder,
       m as [c.cid,rec.oid],
-      n as {{m[?],m[1],m[4]}}
+      n as {{m[0],m[1],m[4]}}
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
index 7b34440..fac0fa5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'cust_name':c.name,'order_total':o.total,'orderedlist':ol,'unorderedlist':ul,'ol_item1':ol[0],'ol_item2':ol[1],'ol_item5':ol[4],'ul_item1':ul[?]}
+select element {'cust_name':c.name,'order_total':o.total,'orderedlist':ol,'unorderedlist':ul,'ol_item1':ol[0],'ol_item2':ol[1],'ol_item5':ol[4],'ul_item1':ul[0]}
 from  Customers as c,
       Orders as o
 with  rec as c.lastorder,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
index 413ab84..bc0e6c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_06/join_q_06.3.query.sqlpp
@@ -24,7 +24,7 @@
 
 SELECT c.name AS cust_name,
        c.cashBack AS cust_cashBack
-FROM customer c JOIN [coll_min((SELECT ELEMENT c.cashBack FROM customer c))] as min_cashBack
+FROM customer c JOIN [strict_min((SELECT ELEMENT c.cashBack FROM customer c))] as min_cashBack
      ON c.cashBack = min_cashBack
 ORDER BY c.cid, c.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
index 5f4f07f..7e89b59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_07/join_q_07.3.query.sqlpp
@@ -29,7 +29,7 @@
 
 SELECT c.name AS cust_name,
        c.cashBack AS cust_cashBack
-FROM customer c JOIN [coll_min((SELECT ELEMENT c.cashBack FROM c c))] as min_cashBack
+FROM customer c JOIN [strict_min((SELECT ELEMENT c.cashBack FROM c c))] as min_cashBack
      ON c.cashBack = min_cashBack
 ORDER BY c.cid, c.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
index 7271316..83a01cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'orderid':o.oid,'ordertot':o.total,'emptyorderedlist':c1,'emptyunorderedlist':c2,'olist_item1':c1[0],'olist_item5':c1[4],'ulist_item1':c2[?]}
+select element {'orderid':o.oid,'ordertot':o.total,'emptyorderedlist':c1,'emptyunorderedlist':c2,'olist_item1':c1[0],'olist_item5':c1[4],'ulist_item1':c2[0]}
 from  Orders as o
 with  c1 as [],
       c2 as {{}}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
index a8859c5..cc8bc88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'item1':c3[?]}
+select element {'item1':c3[0]}
 from  Orders as o
 with  c3 as {{o.heList,o.openlist}}
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp
new file mode 100644
index 0000000..a41976b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type sub as
+{ subscriptionId: uuid }
+;
+
+create dataset subscriptions(sub) primary key subscriptionId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp
new file mode 100644
index 0000000..dad3a82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-qualified/insert-returning-fieldname-qualified.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+upsert into subscriptions as record(
+[{"subscriptionId":create_uuid(), "word": "hello"}]
+) returning record.word;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
index c49b436..dc4c880 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/query-issue382/query-issue382.3.query.sqlpp
@@ -20,7 +20,7 @@
 use SocialNetworkData;
 
 
-select element SocialNetworkData.coll_count((
+select element SocialNetworkData.strict_count((
     select element h
     from  HandbookUsers as h
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
index 7c8f2ea..ea7e438 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
@@ -22,5 +22,5 @@
 
 select element m
 from  Emp as e
-with  m as [{'EmpName':e.name,'parent_interest_1':e.interests[?],'child1Name':e.children[?],'child2Name':e.children[1]}]
+with  m as [{'EmpName':e.name,'parent_interest_1':e.interests[0],'child1Name':e.children[0],'child2Name':e.children[1]}]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.ddl.sqlpp
new file mode 100644
index 0000000..332aa0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a change feed with meta-data, create a primary key index and then ingest data (with deletes)
+ * Expected Res : Success
+ * Date         : 18th Jun 2018
+ */
+
+drop dataverse KeyVerse if exists;
+create dataverse KeyVerse;
+use KeyVerse;
+
+create type DocumentType as open{
+};
+
+create type KVMetaType as open{
+`key`:string,
+vbucket:int32,
+seq:int64,
+cas:int64,
+expiration:int32,
+flags:int32,
+revSeq:int64,
+lockTime:int32
+};
+
+create dataset KVStore(DocumentType) with meta(KVMetaType)primary key meta().`key`;
+create primary index primary_idx on KVStore;
+
+create feed KVChangeStream with {
+  "adapter-name" : "adapter",
+  "type-name" : "DocumentType",
+  "meta-type-name" : "KVMetaType",
+  "reader" : "org.apache.asterix.external.input.record.reader.kv.KVTestReaderFactory",
+  "parser" : "record-with-metadata",
+  "format" : "dcp",
+  "record-format" : "json",
+  "change-feed" : "true",
+  "key-indexes" : "0",
+  "key-indicators" : "1",
+  "num-of-records" : "1000",
+  "delete-cycle" : "5"
+};
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.2.update.sqlpp
new file mode 100644
index 0000000..03a83ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use KeyVerse;
+
+set `wait-for-completion-feed` "true";
+connect feed KVChangeStream to dataset KVStore;
+
+start feed KVChangeStream;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.3.query.sqlpp
new file mode 100644
index 0000000..6e29992
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use KeyVerse;
+
+select count(*)
+from KVStore x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.4.ddl.sqlpp
new file mode 100644
index 0000000..89469bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.4.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+
+drop dataverse KeyVerse;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
index 49fb04c..cf3e9b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_07/feeds_07.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds;
 
 case
- when ((feeds.coll_count((select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds.strict_count((select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
- end;
\ No newline at end of file
+ end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
index f7ae3c8..cf3e9b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_08/feeds_08.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds;
 
 case
- when ((feeds.coll_count((select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds.strict_count((select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
  end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
index d0f5898..5b0acba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_09/feeds_09.3.query.sqlpp
@@ -20,7 +20,7 @@
 use feeds_09;
 
 case
- when ((feeds_09.coll_count(( select element x from  SyntheticTweets as x )) > 0))
+ when ((feeds_09.strict_count(( select element x from  SyntheticTweets as x )) > 0))
   then 1
   else 0
- end;
\ No newline at end of file
+ end;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
index b763e83..d55804c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feeds_10/feeds_10.3.query.sqlpp
@@ -19,7 +19,7 @@
 
 use feeds_10;
 
-select element feeds_10.coll_count((
+select element feeds_10.strict_count((
     select element x
     from  Tweets as x
     order by x.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
index a9ed09d..0c56f0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
@@ -27,6 +27,4 @@
 
 use experiments;
 
-select value count(*) from DBLP0;
-select value count(*) from DBLP1;
-select value count(*) from DBLP2;
\ No newline at end of file
+select value count(*) from DBLP0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
new file mode 100644
index 0000000..62b920b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date         : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
new file mode 100644
index 0000000..b7f38b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date         : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp
new file mode 100644
index 0000000..95c1c38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test filters with autogenerated keys
+ * Expected Res : Success
+ * Date         : 20th Mar 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type UserLocation as closed {
+recordId: uuid,
+location: circle,
+userName: string
+};
+
+
+create dataset UserLocations(UserLocation)
+primary key recordId autogenerated with filter on userName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp
new file mode 100644
index 0000000..cee10ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into UserLocations(
+{"userName" : "c1121u1" , "location" : circle("4171.58,1083.41 100.0")}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp
new file mode 100644
index 0000000..945fc1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select l.userName from UserLocations l
+where l.userName = "c1121u1";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
index 5b44875..d3e48c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby01/grpby01.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date             :  31st July 2012
  */
 
-select element {'store-number':strNum,'total-qty':coll_sum((
+select element {'store-number':strNum,'total-qty':strict_sum((
         select element l.qty
         from  sales as l
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
index 5b44875..d3e48c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/grpby02/grpby02.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date             :  31st July 2012
  */
 
-select element {'store-number':strNum,'total-qty':coll_sum((
+select element {'store-number':strNum,'total-qty':strict_sum((
         select element l.qty
         from  sales as l
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.1.ddl.sqlpp
new file mode 100644
index 0000000..c4bcf0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE EmpType AS {
+  name : string
+};
+
+CREATE DATASET Employee(EmpType) PRIMARY KEY name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.2.update.sqlpp
new file mode 100644
index 0000000..8c4212d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+USE test;
+
+INSERT INTO Employee (
+[
+   { 'name': 'a', 'deptno': 1, 'salary': 2000 },
+   { 'name': 'b', 'deptno': 1, 'salary': 3000 },
+
+   { 'name': 'f', 'deptno': 2, 'salary': 4000 },
+   { 'name': 'g', 'deptno': 2, 'salary': 5000 },
+
+   { 'name': 'k', 'deptno': 3, 'salary': null },
+   { 'name': 'l', 'deptno': 3, 'salary': 6000 },
+   { 'name': 'm', 'deptno': 3                 },
+
+   { 'name': 'q', 'deptno': 4  }
+]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.3.query.sqlpp
new file mode 100644
index 0000000..879ded7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/order-by-13/order-by-13.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+USE test;
+
+FROM Employee e
+SELECT to_string(deptno) AS deptno_str
+ORDER BY deptno_str, salary
+LIMIT 4;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.01.ddl.sqlpp
new file mode 100644
index 0000000..12a6034
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.01.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.empInfoType as
+{
+  id : bigint
+};
+
+create  dataset empDataset(empInfoType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.02.update.sqlpp
new file mode 100644
index 0000000..04ab807
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use test;
+
+load  dataset empDataset using localfs ((`path`=`asterix_nc1://data/types/empDataset.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.03.query.sqlpp
new file mode 100644
index 0000000..9577855
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.03.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project expression and constant
+ */
+
+use test;
+
+select empno as a, 2 as a
+from empDataset
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.04.query.sqlpp
new file mode 100644
index 0000000..5eac30e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.04.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project expression and expression
+ */
+
+use test;
+
+select empno as b, name as b
+from empDataset
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.05.query.sqlpp
new file mode 100644
index 0000000..ea4372a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.05.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              FROM variable and another field
+ */
+
+use test;
+
+select c, empno as c
+from empDataset c
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.06.query.sqlpp
new file mode 100644
index 0000000..788577b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.06.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project * (FROM variable) and another field
+ */
+
+use test;
+
+select *, empno as d
+from empDataset d
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.07.query.sqlpp
new file mode 100644
index 0000000..232d29c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.07.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project * (LET variable) and another field
+ */
+
+use test;
+
+select *, emp.name as e
+from empDataset emp
+let e = d.deptno
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.08.query.sqlpp
new file mode 100644
index 0000000..48c96a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.08.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project * (GROUP BY key variable) and another field
+ */
+
+use test;
+
+select *, count(*) as f
+from empDataset emp
+group by emp.age f
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.09.query.sqlpp
new file mode 100644
index 0000000..9655bb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446-2/query-ASTERIXDB-2446-2.09.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Ensure error if there's a duplicate field name in the SELECT clause
+ *              Project * (LET after GROUP BY variable) and another field
+ */
+
+use test;
+
+select *, count(*) as g
+from empDataset emp
+group by emp.age f
+let g = f + 1
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.1.query.sqlpp
new file mode 100644
index 0000000..554cf28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Ensure error if there's a duplicate field name in the SELECT clause
+ */
+
+select t as a, 2 as a from range(1, 10) t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp
new file mode 100644
index 0000000..b548b5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Ensure error if there's a duplicate field name in the closed-object-constructor() function
+ */
+
+set `import-private-functions` `true`;
+
+`closed-object-constructor`('b',get_year(current_date()),'c',[20],'c',[30]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp
new file mode 100644
index 0000000..73810d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description    : Ensure error if there's a duplicate field name in the open-object-constructor() function
+ */
+
+set `import-private-functions` `true`;
+
+`open-object-constructor`('d' || to_string(get_year(current_date())),10,'e',[20],'e',[30]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
index aab13a4..ab85a07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-883/query-ASTERIXDB-883.3.query.sqlpp
@@ -53,9 +53,9 @@
 SELECT VALUE
 {
    "user": a.user,
-   "beth": coll_count(beth),
-   "ts": coll_avg( ( FROM ts AS l SELECT VALUE l.timespent ) ),
-   "rev": coll_sum( ( FROM rev AS k SELECT VALUE k.estimated_revenue ) )
+   "beth": strict_count(beth),
+   "ts": strict_avg( ( FROM ts AS l SELECT VALUE l.timespent ) ),
+   "rev": strict_sum( ( FROM rev AS k SELECT VALUE k.estimated_revenue ) )
 }
 ORDER BY a.user;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.1.ddl.sqlpp
new file mode 100644
index 0000000..ef54c5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Full-text search index test
+ *                  : This test is intended to verify that the full-text search works as expected
+ *                  : with multiple conditions.
+ *                  : Each query will generate the output whose size is greater than a frame.
+ *                  : query #3 - two string values in [an ordered list] query with "any" option
+ *                  :            in this case, "any" option that enforces a disjunctive search will be applied.
+ *                  : query #4 - the same as query #3, but with a different option - "all"
+ *                  :            in this case, we explicitly specify "all" option that enforces a conjunctive search.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as open {
+  id: int64,
+  text: string
+};
+
+create dataset MyData(MyRecord)
+  primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.2.update.sqlpp
new file mode 100644
index 0000000..ea348bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+load dataset MyData
+using localfs
+(("path"="asterix_nc1://data/fulltext/cloudberry_sample_tweet.adm"),("format"="adm"));
+
+create index fulltext_index_text on MyData(text) type fulltext;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.3.query.sqlpp
new file mode 100644
index 0000000..8a913cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select count(*) from (
+  select element {"id":ftval.id}
+  from MyData as ftval
+  where test.ftcontains(ftval.text, ["t","co"], {"mode":"any"})
+  order by ftval.id
+) first;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.4.query.sqlpp
new file mode 100644
index 0000000..0bf6847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/fulltext-index-09/fulltext-index-09.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+use test;
+
+select count(*) from (
+  select element {"id":ftval.id}
+  from MyData as ftval
+  where test.ftcontains(ftval.text, ["t","co"], {"mode":"all"})
+  order by ftval.id
+) first;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.1.ddl.sqlpp
new file mode 100644
index 0000000..63b76bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.1.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+create function test(){
+    1
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.2.ddl.sqlpp
new file mode 100644
index 0000000..14a68c0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.2.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+use test;
+drop function test@0 if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.3.ddl.sqlpp
new file mode 100644
index 0000000..14a68c0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+use test;
+drop function test@0 if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.4.ddl.sqlpp
new file mode 100644
index 0000000..45f4fae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/drop_if_exists/drop_if_exists.4.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
new file mode 100644
index 0000000..a1173de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+drop dataverse Coffee if exists;
+create dataverse Coffee;
+use Coffee;
+
+
+create type CoffeeType as {
+    id: string,
+    location: point
+};
+
+create type FollowersType as {
+    user_id: string,
+    twitter_id: string,
+    followers: [string]
+};
+
+create dataset Coffee(CoffeeType)
+    primary key id;
+
+create dataset Followers(FollowersType)
+    primary key user_id;
+
+create type Tweet as open {
+    id: int64,
+    timestamp_ms: string
+};
+
+create dataset Tweets (Tweet)
+primary key id;
+
+create function CoffeeWithFriends(user_id){
+(select * from Tweets t
+let temp = (select * from Tweets f where f.user.id_str in (select value fw.followers from Followers fw where fw.twitter_id=user_id)[0]
+and
+spatial_intersect(create_point(f.place.bounding_box.coordinates[0][0][0],f.place.bounding_box.coordinates[0][0][1]),
+create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1])))
+where t.user.id_str = user_id)
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp
new file mode 100644
index 0000000..7bb9131
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.2.update.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+use Coffee;
+upsert into Coffee
+([
+{"id":"001", "location":point("-117.523867, 33.85216")},
+{"id":"002", "location":point("-122.37382985, 47.67917392")},
+{"id":"003", "location":point("-122.00795969, 37.40427696")},
+{"id":"004", "location":point("-122.00794969, 37.40427596")}
+]);
+
+upsert into Followers
+([
+{"user_id":"userId@Coffee", "twitter_id": "988455822727393280", "followers":["988455822727393281","988455822727393282", "0004", "0005", "0006", "0007", "0008"]}
+]);
+
+upsert into Tweets([
+{ "id": 999403843673718784, "user":
+{ "id": 988455822727393280, "id_str": "988455822727393280", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+ { "id": 999403845, "user":
+{ "id": 988455822727393281, "id_str": "988455822727393281", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+  { "id": 999403846, "user":
+{ "id": 988455822727393282, "id_str": "988455822727393282", "screen_name": "TweetUser" },
+"timestamp_ms": "1527111419014",
+ "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA",
+ "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp
new file mode 100644
index 0000000..140f6c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.3.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+use Coffee;
+CoffeeWithFriends("988455822727393280");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp
new file mode 100644
index 0000000..639d98d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.4.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+drop dataverse Coffee;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
new file mode 100644
index 0000000..24f4ed6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
@@ -0,0 +1,35 @@
+<!--
+ ! 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.
+ !-->
+<test-group name="geojson">
+    <test-case FilePath="geojson">
+        <compilation-unit name="datatype">
+            <output-dir compare="Text">datatype</output-dir>
+        </compilation-unit>
+    </test-case>
+    <test-case FilePath="geojson">
+        <compilation-unit name="single-method">
+            <output-dir compare="Text">single-method</output-dir>
+        </compilation-unit>
+    </test-case>
+    <test-case FilePath="geojson">
+        <compilation-unit name="two-geometries">
+            <output-dir compare="Text">two-geometries</output-dir>
+        </compilation-unit>
+    </test-case>
+</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
new file mode 100644
index 0000000..61a7be6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"PointN": st_point_n(geo.myGeometry,1), "StartPoint":st_start_point(geo.myGeometry),"Envelope":st_envelope(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
new file mode 100644
index 0000000..dd378a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop  dataverse GeoJSON if exists;
+create  dataverse GeoJSON;
+
+use GeoJSON;
+
+CREATE TYPE GeometryType AS{
+  id : int,
+  myGeometry : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp
new file mode 100644
index 0000000..3ab3bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry": st_geom_from_geojson({"type":"Point","coordinates":[-118.4,33.93]})},
+{"id": 124, "myGeometry": st_geom_from_geojson({"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]})},
+{"id": 126, "myGeometry": st_geom_from_geojson({"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]})},
+{"id": 127, "myGeometry": st_geom_from_geojson({"type": "MultiPoint","coordinates": [[10, 40], [40, 30], [20, 20], [30, 10]]})},
+{"id": 128, "myGeometry": st_geom_from_geojson({"type": "MultiLineString","coordinates": [[[10, 10], [20, 20], [10, 40]],[[40, 40], [30, 30], [40, 20], [30, 10]]]})},
+{"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
+{"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
+{"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
+{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+{"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
+{"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
+{"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
+{"id": 136, "myGeometry": st_make_envelope(10, 10, 11, 11, 4326)},
+{"id": 137, "myGeometry": st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))")}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
new file mode 100644
index 0000000..540bce1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT * FROM Geometries ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
new file mode 100644
index 0000000..beb8879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"IsClosed":st_is_closed(geo.myGeometry), "IsCollection":st_is_collection(geo.myGeometry),"IsEmpty":st_is_empty(geo.myGeometry), "IsSimple":st_is_simple(geo.myGeometry)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp
new file mode 100644
index 0000000..4a19b47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"Type": geometry_type(geo.myGeometry), "Area": st_area(geo.myGeometry),
+"Coordinate dimension": st_coord_dim(geo.myGeometry), "Dimension":st_dimension(geo.myGeometry),
+"NPoints":st_n_points(geo.myGeometry), "XMax":st_x_max(geo.myGeometry),"XMin":st_x_min(geo.myGeometry),
+"YMax":st_y_max(geo.myGeometry), "YMin":st_y_min(geo.myGeometry), "Binary": st_as_binary(geo.myGeometry),
+"GeoJSON":st_as_geojson(geo.myGeometry),"WKT":st_as_text(geo.myGeometry)}  FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
new file mode 100644
index 0000000..4868d4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"X":st_x(geo.myGeometry),"Y":st_y(geo.myGeometry),"Z":st_z(geo.myGeometry), "M":st_m(geo.myGeometry)}  FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Point" ;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
new file mode 100644
index 0000000..d7e32a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"NumInteriorRings":st_num_interior_rings(geo.myGeometry), "ExteriorRing":st_exterior_ring(geo.myGeometry)}  FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Polygon";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
new file mode 100644
index 0000000..9fb187d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"Length": st_length(geo.myGeometry), "Boundary":st_boundary(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString" OR geometry_type(geo.myGeometry)="MultiLineString" ORDER BY geo.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
new file mode 100644
index 0000000..01c2563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+st_geometry_n(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'),2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp
new file mode 100644
index 0000000..28ee45f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+st_is_collection(st_polygonize([st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")]));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp
new file mode 100644
index 0000000..54b4450
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+st_is_empty(st_geom_from_text("LINESTRING EMPTY"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
new file mode 100644
index 0000000..9924d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+st_num_geometries(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp
new file mode 100644
index 0000000..3a889b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Union": st_union(geo.myGeometry1, geo.myGeometry2),"Intersection":st_intersection(geo.myGeometry1,
+geo.myGeometry2),"SymDifference":st_sym_difference(geo.myGeometry1,geo.myGeometry2)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp
new file mode 100644
index 0000000..7084fe3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+use GeoJSON1;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 124, "myGeometry1": st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), "myGeometry2": st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")},
+{"id": 126, "myGeometry1": st_geom_from_text('POINT(0 0)'), "myGeometry2": st_geom_from_text('LINESTRING ( 0 0, 0 2 )')},
+{"id": 127, "myGeometry1": st_geom_from_text('LINESTRING(1 1,-1 -1,2 3.5,1 3,1 2,2 1)'), "myGeometry2": st_make_point(-1, -1)},
+{"id": 128, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('LINESTRING(0 2,1 2,2 2,3 2,4 2,5 2)')},
+{"id": 129, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('POINT(0 0)')},
+{"id": 130, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 10 10)'), "myGeometry2": st_geom_from_text('LINESTRING(0 0, 5 5, 10 10)')},
+{"id": 131, "myGeometry1": st_geom_from_text('LINESTRING(1 2, 3 4)'), "myGeometry2": st_geom_from_text('LINESTRING(5 6, 7 8)')},
+{"id": 132, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 133, "myGeometry1": st_geom_from_text('POLYGON((0.25 0.25, 0.5 0.25, 0.5 0.5, 0.25 0.5, 0.25 0.25 ))'), "myGeometry2": st_geom_from_text('POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))')}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp
new file mode 100644
index 0000000..671d151
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+drop  dataverse GeoJSON1 if exists;
+create  dataverse GeoJSON1;
+
+use GeoJSON1;
+
+CREATE TYPE GeometryType AS{
+  id : int,
+  myGeometry1 : geometry,
+  myGeometry2 : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp
new file mode 100644
index 0000000..74a3fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Distance":st_distance(geo.myGeometry1, geo.myGeometry2), "Intersects":st_intersects(geo.myGeometry1,
+geo.myGeometry2), "Contains":st_contains(geo.myGeometry1, geo.myGeometry2), "Crosses":st_crosses(geo.myGeometry1,
+geo.myGeometry2), "Disjoint":st_disjoint(geo.myGeometry1, geo.myGeometry2), "Equals":st_equals(geo.myGeometry1,
+geo.myGeometry2), "Overlaps":st_overlaps(geo.myGeometry1, geo.myGeometry2), "Relate":st_relate(geo.myGeometry1,
+geo.myGeometry2, "FF1FF0102"), "Touches":st_touches(geo.myGeometry1, geo.myGeometry2), "Within":st_within(
+geo.myGeometry1, geo.myGeometry2)} FROM Geometries geo;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
index 907afb0..5e98b19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q06_error/q06_error.3.query.sqlpp
@@ -19,5 +19,5 @@
 
 USE TinySocial;
 
-SELECT COLL_COUNT(u.name) count
+SELECT STRICT_COUNT(u.name) count
 FROM FacebookUsers u;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
index 1b05c86..12eebf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/q08/q08.3.query.sqlpp
@@ -19,6 +19,6 @@
 
 USE TinySocial;
 
-SELECT COLL_COUNT(u.`friend-ids`) count
+SELECT STRICT_COUNT(u.`friend-ids`) count
 FROM FacebookUsers u
 ORDER BY u.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
index 2340771..e1a85e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-159/query-ASTERIXDB-159.3.query.sqlpp
@@ -19,4 +19,4 @@
 
 USE TinySocial;
 
-COLL_COUNT(FacebookUsers);
+STRICT_COUNT(FacebookUsers);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
index 15af505..38e01dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626-2/query-ASTERIXDB-1626-2.3.query.sqlpp
@@ -20,5 +20,5 @@
 USE TinySocial;
 
 WITH t AS 1
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`)), t
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`)), t
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
index c72c3c8..42d5f39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/global-aggregate/query-ASTERIXDB-1626/query-ASTERIXDB-1626.3.query.sqlpp
@@ -19,5 +19,5 @@
 
 USE TinySocial;
 
-SELECT count(`friend-ids`), avg(coll_count(`friend-ids`))
+SELECT count(`friend-ids`), avg(strict_count(`friend-ids`))
 FROM  FacebookUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
index 56b042c..1274de2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01-error/core-01-error.3.query.sqlpp
@@ -24,6 +24,6 @@
 GROUP BY e.deptno AS deptno
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM e AS i SELECT ELEMENT i.salary) ),
+  'avgpay': strict_avg( (FROM e AS i SELECT ELEMENT i.salary) ),
   'workers': (FROM e AS i SELECT ELEMENT  {'name': i.name, 'salary': i.salary})
-};
\ No newline at end of file
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
index dc7ac16..16714bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-01/core-01.3.query.sqlpp
@@ -24,6 +24,6 @@
 GROUP BY e.deptno AS deptno GROUP AS g(e AS e)
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM g AS i SELECT ELEMENT i.e.salary) ),
+  'avgpay': strict_avg( (FROM g AS i SELECT ELEMENT i.e.salary) ),
   'workers': (FROM g AS i SELECT ELEMENT  {'name': i.e.name, 'salary': i.e.salary})
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
index b752fe0..fde4de4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02-error/core-02-error.3.query.sqlpp
@@ -25,6 +25,6 @@
 GROUP AS g(e as f)
 SELECT ELEMENT {
   'deptno': deptno,
-  'avgpay': coll_avg( (FROM f AS i SELECT ELEMENT i.salary) ),
+  'avgpay': strict_avg( (FROM f AS i SELECT ELEMENT i.salary) ),
   'workers': (FROM f AS i SELECT ELEMENT  {'name': i.name, 'salary': i.salary})
-};
\ No newline at end of file
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
index 70afc46..8b980e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-02/core-02.3.query.sqlpp
@@ -26,5 +26,5 @@
 GROUP AS eis(e AS e, i AS i, s AS s)
 SELECT ELEMENT {
       'deptId': deptId,
-      'star_cost': coll_sum( (FROM eis AS p SELECT ELEMENT p.e.salary + p.i.bonus) )
+      'star_cost': strict_sum( (FROM eis AS p SELECT ELEMENT p.e.salary + p.i.bonus) )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
index 8e6b987..1c9f321 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-03/core-02.3.query.sqlpp
@@ -26,7 +26,7 @@
 GROUP AS eis(e AS e, i AS i, s AS s)
 SELECT ELEMENT {
       'deptId': deptId,
-      'avgpay': coll_avg( (FROM eis AS g SELECT ELEMENT g.e.salary + g.i.bonus) ),
+      'avgpay': strict_avg( (FROM eis AS g SELECT ELEMENT g.e.salary + g.i.bonus) ),
       'topstar_details':
            (
                 FROM eis AS g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
index e647ff0..d762252 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-05/core-05.3.query.sqlpp
@@ -25,7 +25,7 @@
             GROUP BY c.tire_size AS tire_size GROUP AS g(c AS c, t AS t)
             SELECT ELEMENT {
                 'tire_size': tire_size,
-                'avg_total_price': coll_avg(
+                'avg_total_price': strict_avg(
                                          (  FROM g AS g
                                             SELECT ELEMENT g.c.price + 4 * g.t.price
                                           )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
index 8600077..ec47ba6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/having-2/having-2.3.query.sqlpp
@@ -23,6 +23,6 @@
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
 GROUP BY e.department_id GROUP AS g
-HAVING coll_count( ( SELECT VALUE g.e FROM g) )>0
+HAVING strict_count( ( SELECT VALUE g.e FROM g) )>0
 SELECT e.department_id as deptId, SUM(e.salary + i.bonus) AS star_cost;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
index 2bc83ef..0218637 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-01-negative/sugar-01-negative.3.query.sqlpp
@@ -19,9 +19,9 @@
 
 USE gby;
 
-// Tests that a SQL++ core aggregate function, e.g., COLL_AVG,
+// Tests that a SQL++ core aggregate function, e.g., STRICT_AVG,
 // cannot be used like a SQL-92 sugar, e.g., AVG. Its input must be a collection.
 FROM Employee e
 GROUP BY e.deptno AS deptno GROUP AS g
-SELECT deptno AS deptno, COLL_AVG(g.e.salary) AS avgpay,
+SELECT deptno AS deptno, STRICT_AVG(g.e.salary) AS avgpay,
        (SELECT i.e.name AS name, i.e.salary AS salary FROM g AS i) AS workers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-06-distinct/sugar-06-distinct.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-06-distinct/sugar-06-distinct.9.query.sqlpp
new file mode 100644
index 0000000..340c590
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-06-distinct/sugar-06-distinct.9.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+USE gby;
+
+FROM Employee e
+GROUP BY deptno
+SELECT deptno, MAX(DISTINCT salary) salary_agg
+ORDER BY salary_agg DESC
+LIMIT 3;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..a5b4314
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop dataverse gby if exists;
+create dataverse gby;
+
+use gby;
+
+create type testType as open {
+  id : string
+};
+
+create dataset customers(testType) primary key id;
+create dataset orders(testType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.2.query.sqlpp
new file mode 100644
index 0000000..1d0c963
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/sugar-07-negative/sugar-07-negative.2.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: Test error message for an undefined variable reference in SQL aggregate function
+ */
+
+use gby;
+
+select c_id, min(IDENT)
+from customers c, orders o
+where c.c_id = o.c_id
+group by c.c_id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
index 358eeda..7ea096f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_02/hdfs_02.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count(( from g select value token ))}
+select element {'word':tok,'count':test.strict_count(( from g select value token ))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
index fb67e5b..0e52cdb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_03/hdfs_03.3.query.sqlpp
@@ -28,7 +28,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
index 998dd43..972f0db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/hdfs_shortcircuit/hdfs_shortcircuit.3.query.sqlpp
@@ -27,7 +27,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
index b889ba9..eba03db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hints/issue_251_dataset_hint_6/issue_251_dataset_hint_6.3.query.sqlpp
@@ -27,7 +27,7 @@
 use test;
 
 
-select element {'word':tok,'count':test.coll_count((from g select value token))}
+select element {'word':tok,'count':test.strict_count((from g select value token))}
 from  TextDataset as line,
       test.`word-tokens`(line.content) as token
 group by token as tok
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
new file mode 100644
index 0000000..cb51012
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
new file mode 100644
index 0000000..1097325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
new file mode 100644
index 0000000..62f4cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
new file mode 100644
index 0000000..75212af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.1.ddl.sqlpp
new file mode 100644
index 0000000..33a9c58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+/*
+ * Description     : Test push down limit into the primary index scan operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.2.update.sqlpp
new file mode 100644
index 0000000..6be85ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.2.update.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+use test;
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.3.query.sqlpp
new file mode 100644
index 0000000..194ad19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.3.query.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test negative limit and offset values
+ * Expected Result : Success
+ */
+
+{
+  "t1": array_count((
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit -2
+  )),
+
+  "t2": array_count((
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit -get_year(current_date())
+  )),
+
+  "t3": (
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit 2 offset -4
+  ),
+
+  "t4": (
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit 2 offset -get_year(current_date())
+  ),
+
+  "t5": array_count((
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit -2 offset -2
+  )),
+
+  "t6": array_count((
+    select value t
+    from [6,5,4,3,2,1] t
+    order by t
+    limit -get_year(current_date()) offset -get_year(current_datetime())
+  ))
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.4.query.sqlpp
new file mode 100644
index 0000000..df89aca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_negative_value/limit_negative_value.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select value paper
+from DBLP1 as paper
+order by dblpid
+limit 2 offset -get_year(current_date());
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
new file mode 100644
index 0000000..c8b2b7a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test expressions returning floating point numbers
+ *                 : in limit/offset
+ * Expected Result : Success
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 7.5/2.5 offset 5/2.5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
new file mode 100644
index 0000000..ce55ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 5.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
new file mode 100644
index 0000000..e479296
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 3.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
new file mode 100644
index 0000000..b870b76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit "2"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
new file mode 100644
index 0000000..3e3bfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 2 offset true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
new file mode 100644
index 0000000..d783bb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 9999999999
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
new file mode 100644
index 0000000..991f189
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 8888888888
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
new file mode 100644
index 0000000..e2782c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test push down limit and select condition into the primary index lookup operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : bigint,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  index idx_LineItem_suppkey  on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
new file mode 100644
index 0000000..b8eac5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain select element c
+from  LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
new file mode 100644
index 0000000..212479a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+select element c
+from  LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp
new file mode 100644
index 0000000..ab6451b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain
+select
+  substring(c.l_shipdate, 0, 4) as shipdate,
+  c.l_suppkey > 0 as suppkey
+from LineItem as c
+where c.l_suppkey < 150
+limit 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp
new file mode 100644
index 0000000..e7ac96d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select
+  substring(c.l_shipdate, 0, 4) as shipdate,
+  c.l_suppkey > 0 as suppkey
+from LineItem as c
+where c.l_suppkey < 150
+limit 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
new file mode 100644
index 0000000..e9daaad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test push down limit into primary key lookup operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : bigint,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  index idx_LineItem_suppkey  on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
new file mode 100644
index 0000000..43f7d94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
new file mode 100644
index 0000000..026aed7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
new file mode 100644
index 0000000..5673992
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
new file mode 100644
index 0000000..917715e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description     : Test push down limit and select condition into primary index scan operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : string
+};
+
+create dataset DBLP1(DBLPType) primary key id;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
new file mode 100644
index 0000000..a18f3e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp
new file mode 100644
index 0000000..657bfb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  where contains(dblpid, 'kimL89')
+  limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
new file mode 100644
index 0000000..6459b5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from  DBLP1 as paper
+where contains(dblpid, 'kimL89')
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.query.sqlpp
new file mode 100644
index 0000000..14393b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test limit push down when there're
+ *                   multiple limit clauses
+ * Expected Result : Success ("limit 1" is pushed to the scan)
+ */
+
+use test;
+
+explain
+select dblpid from DBLP1 d
+where d.dblpid = (
+  select *
+  from DBLP1 d
+  where ends_with(dblpid, "Blakeley95")
+  limit 1
+)[0].d.dblpid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.query.sqlpp
new file mode 100644
index 0000000..affde8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain
+select substring(dblpid, 0, 21)
+from DBLP1
+where dblpid > "series"
+limit 1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.query.sqlpp
new file mode 100644
index 0000000..d47e735
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select substring(dblpid, 0, 21)
+from DBLP1
+where dblpid > "series"
+limit 1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.query.sqlpp
new file mode 100644
index 0000000..c818522
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use test;
+
+explain
+select value t.user.lang
+from TweetMessages t
+where t.user.friends_count between 0 and 150
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.query.sqlpp
new file mode 100644
index 0000000..7f6a607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select value t.user.lang
+from TweetMessages t
+where t.user.friends_count between 0 and 150
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
new file mode 100644
index 0000000..162cc35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description     : Test push down limit into the primary index scan operator
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
new file mode 100644
index 0000000..2e0f056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp
new file mode 100644
index 0000000..d2cb38a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
new file mode 100644
index 0000000..5b7c5be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from  DBLP1 as paper
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
new file mode 100644
index 0000000..fc9d42f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.ddl.sqlpp
new file mode 100644
index 0000000..4f446c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description     : Test push down limit
+ * Expected Result : Success
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.DBLPType as
+{
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.2.update.sqlpp
new file mode 100644
index 0000000..867b825
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use test;
+
+load  dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.3.query.sqlpp
new file mode 100644
index 0000000..8960f6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.3.query.sqlpp
@@ -0,0 +1,25 @@
+/* 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.
+ */
+
+/*
+ * Description     : Test limit pushdown
+ * Expected Result : Success
+ */
+
+use test;
+
+select value t.id
+from DBLP1 t
+order by t.id
+limit {"a":1}.a offset {"a":1}.a
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
index 208a440..ebccaf3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{{1,1,1}}[?];
+{{1,1,1}}[0];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
index 5e73b39..ac09ccd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/listify_03/listify_03.3.query.sqlpp
@@ -25,10 +25,10 @@
 use test;
 
 
-select element coll_min(y)
+select element strict_min(y)
 from  [1,2] as x
 with  y as (
-      select element coll_min(i)
+      select element strict_min(i)
       from  [[1,2,3],[10,20,30],[-2,-5,0]] as i
   )
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
index 0954239..13e237b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-2/query-ASTERIXDB-159-2.3.query.sqlpp
@@ -22,6 +22,6 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id, fu2.id
 LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
index a932f2a..1c6278c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/query-ASTERIXDB-159-3/query-ASTERIXDB-159-3.3.query.sqlpp
@@ -22,6 +22,6 @@
 SELECT fu1.id id1, fu2.id id2
 FROM FacebookUsers fu1
 LEFT OUTER JOIN FacebookUsers fu2
-ON COLL_COUNT(RANGE(fu1.id, fu2.id))>2 AND COLL_COUNT(RANGE(fu1.id, fu2.id))<5
+ON STRICT_COUNT(RANGE(fu1.id, fu2.id))>2 AND STRICT_COUNT(RANGE(fu1.id, fu2.id))<5
 ORDER BY fu1.id DESC, fu2.id DESC
 LIMIT 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
index 087f7f6..e12399f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/issue289_query/issue289_query.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.coll_count((
+select element test.strict_count((
     select element l
     from  Customers as l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
index 48a2456..2773865 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/materialization/assign-reuse/assign-reuse.3.query.sqlpp
@@ -23,12 +23,12 @@
 with  lonelyusers as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   ),
       lonelyusers2 as (
       select element d
       from  FacebookUsers as d
-      where (TinySocial.coll_count(d.`friend-ids`) < 2)
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
   )
 select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
 from  lonelyusers as l1,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
new file mode 100644
index 0000000..c4d77ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Processed objects metrics on secondary index scan when it's an index-only plan
+ * Expected Res : Success
+ * Date         : 27 Apr 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.AddressType as
+{
+  number : bigint,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : bigint,
+  name : string,
+  age : bigint?,
+  address : AddressType?,
+  lastorder : {
+      oid : bigint,
+      total : float
+  }
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create index customer_name_idx on Customers(name);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp
new file mode 100644
index 0000000..6a5fd9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+load dataset Customers using localfs
+  ((`path`=`asterix_nc1://data/custord-tiny/customer-tiny-neg.adm`),
+  (`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp
new file mode 100644
index 0000000..881ed24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select count(*) from Customers
+where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
index 1c5126f..95d4681 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
@@ -24,5 +24,7 @@
 
 use test;
 
+set noindexonly 'true';
+
 select count(*) from Customers
 where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp
new file mode 100644
index 0000000..60229d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+SELECT
+  CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" as v1,
+  ( CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END ) LIKE "a%" as v2,
+  [ CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" ] as v3
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.ddl.sqlpp
new file mode 100644
index 0000000..a0fcb0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+drop dataverse Coffee if exists;
+create dataverse Coffee;
+use Coffee;
+
+
+create type CoffeeType as {
+    id: string,
+    location: point
+};
+
+create type FollowersType as {
+    user_id: string,
+    twitter_id: string,
+    followers: [string]
+};
+
+create dataset Coffee(CoffeeType)
+    primary key id;
+
+create dataset Followers(FollowersType)
+    primary key user_id;
+
+
+create type CoffeeUser as closed {
+    id: int64,
+    id_str: string
+};
+
+create type Tweet as open {
+    id: int64,
+    user: CoffeeUser,
+    timestamp_ms: string
+};
+
+create dataset Tweets (Tweet)
+primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.2.update.sqlpp
new file mode 100644
index 0000000..be0c667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.2.update.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+use Coffee;
+upsert into Coffee
+([
+{"id":"001", "location":point("-117.523867, 33.85216")},
+{"id":"002", "location":point("-122.37382985, 47.67917392")},
+{"id":"003", "location":point("-122.00795969, 37.40427696")},
+{"id":"004", "location":point("-122.00794969, 37.40427596")}
+]);
+
+upsert into Followers
+([
+{"user_id":"user1@Coffee", "twitter_id": "988455822727393280", "followers":["988455822727393281","988455822727393282", "0004", "0005", "0006", "0007", "0008"]}
+]);
+
+upsert into Tweets([
+{ "id": 999403843673718784, "user":
+{ "id": 988455822727393280, "id_str": "988455822727393280"},
+"timestamp_ms": "1527111419014",
+ "place": { "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+ { "id": 999403845, "user":
+{ "id": 988455822727393281, "id_str": "988455822727393281"},
+"timestamp_ms": "1527111419014",
+ "place": { "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } },
+
+  { "id": 999403846, "user":
+{ "id": 988455822727393282, "id_str": "988455822727393282"},
+"timestamp_ms": "1527111419014",
+ "place": {  "bounding_box":
+ { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ]
+ } } }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.3.query.sqlpp
new file mode 100644
index 0000000..a394f0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+use Coffee;
+
+let result = (select * from Tweets t
+let followers = (select * from Tweets f where spatial_intersect(create_circle(create_point(f.place.bounding_box.coordinates[0][0][0],f.place.bounding_box.coordinates[0][0][1]),5.0),
+create_circle(create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1]),5.0))),
+sb = (select value s.location from Coffee s where spatial_intersect(create_circle(create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1]),5.0),s.location)))
+select value count(*) from result;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.1.ddl.sqlpp
new file mode 100644
index 0000000..1958cde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.1.ddl.sqlpp
@@ -0,0 +1,144 @@
+/*
+ * 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.
+ */
+
+drop dataverse tpcds if exists;
+create dataverse tpcds;
+
+use tpcds;
+
+create type tpcds.customer_address_type as  closed {
+  ca_address_sk : bigint,
+  ca_address_id : string,
+  ca_street_number : string?,
+  ca_street_name : string?,
+  ca_street_type : string?,
+  ca_suite_number : string?,
+  ca_city : string?,
+  ca_county : string?,
+  ca_state : string?,
+  ca_zip : string?,
+  ca_country : string?,
+  ca_gmt_offset : double?,
+  ca_location_type : string?
+ };
+
+create type tpcds.web_sales_type as
+ closed {
+  ws_sold_date_sk : bigint?,
+  ws_sold_time_sk : bigint?,
+  ws_ship_date_sk : bigint?,
+  ws_item_sk : bigint,
+  ws_bill_customer_sk : bigint?,
+  ws_bill_cdemo_sk : bigint?,
+  ws_bill_hdemo_sk : bigint?,
+  ws_bill_addr_sk : bigint?,
+  ws_ship_customer_sk : bigint?,
+  ws_ship_cdemo_sk : bigint?,
+  ws_ship_hdemo_sk : bigint?,
+  ws_ship_addr_sk : bigint?,
+  ws_web_page_sk : bigint?,
+  ws_web_site_sk : bigint?,
+  ws_ship_mode_sk : bigint?,
+  ws_warehouse_sk : bigint?,
+  ws_promo_sk : bigint?,
+  ws_order_number : bigint,
+  ws_quantity : bigint?,
+  ws_wholesale_cost : double?,
+  ws_list_price : double?,
+  ws_sales_price : double?,
+  ws_ext_discount_amt : double?,
+  ws_ext_sales_price : double?,
+  ws_ext_wholesale_cost : double?,
+  ws_ext_list_price : double?,
+  ws_ext_tax : double?,
+  ws_coupon_amt : double?,
+  ws_ext_ship_cost : double?,
+  ws_net_paid : double?,
+  ws_net_paid_inc_tax : double?,
+  ws_net_paid_inc_ship : double?,
+  ws_net_paid_inc_ship_tax : double?,
+  ws_net_profit : double?
+};
+
+create type tpcds.store_sales_type as
+ closed {
+  ss_sold_date_sk:           bigint?,
+  ss_sold_time_sk:           bigint?,
+  ss_item_sk:                bigint,
+  ss_customer_sk:            bigint?,
+  ss_cdemo_sk:               bigint?,
+  ss_hdemo_sk:               bigint?,
+  ss_addr_sk:                bigint?,
+  ss_store_sk:               bigint?,
+  ss_promo_sk:               bigint?,
+  ss_ticket_number:          bigint,
+  ss_quantity:               bigint?,
+  ss_wholesale_cost:         double?,
+  ss_list_price:             double?,
+  ss_sales_price:            double?,
+  ss_ext_discount_amt:       double?,
+  ss_ext_sales_price:        double?,
+  ss_ext_wholesale_cost:     double?,
+  ss_ext_list_price:         double?,
+  ss_ext_tax:                double?,
+  ss_coupon_amt:             double?,
+  ss_net_paid:               double?,
+  ss_net_paid_inc_tax:       double?,
+  ss_net_profit:             double?
+};
+
+create type tpcds.date_dim_type as
+ closed {
+  d_date_sk : bigint,
+  d_date_id : string,
+  d_date : string?,
+  d_month_seq : bigint?,
+  d_week_seq : bigint?,
+  d_quarter_seq : bigint?,
+  d_year : bigint? ,
+  d_dow : bigint? ,
+  d_moy : bigint?,
+  d_dom : bigint?,
+  d_qoy : bigint?,
+  d_fy_year : bigint?,
+  d_fy_quarter_seq : bigint?,
+  d_fy_week_seq : bigint?,
+  d_day_name : string?,
+  d_quarter_name : string?,
+  d_holiday : string?,
+  d_weekend : string?,
+  d_following_holiday : string?,
+  d_first_dom : bigint?,
+  d_last_dom : bigint?,
+  d_same_day_ly : bigint?,
+  d_same_day_lq : bigint?,
+  d_current_day : string?,
+  d_current_week : string?,
+  d_current_month : string?,
+  d_current_quarter : string?,
+  d_current_year : string?
+};
+
+create dataset customer_address(customer_address_type) primary key ca_address_sk;
+
+create dataset web_sales (web_sales_type) primary key ws_item_sk, ws_order_number;
+
+create dataset store_sales (store_sales_type) primary key ss_item_sk, ss_ticket_number;
+
+create dataset date_dim(date_dim_type) primary key d_date_sk;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.2.query.sqlpp
new file mode 100644
index 0000000..b6b1e0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.2.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+USE tpcds;
+
+WITH ss AS
+ (SELECT sum(ss.ss_ext_sales_price) as store_sales
+ FROM store_sales ss),
+ ws AS
+ (SELECT sum(ws.ws_ext_sales_price) as web_sales
+ FROM web_sales ws)
+ SELECT
+       ss1.d_year
+ FROM
+        ss ss1
+       ,ss ss2
+       ,ws ws1
+       ,ws ws2
+ WHERE
+    (CASE WHEN ws1.web_sales > 0 THEN ws2.web_sales/ws1.web_sales ELSE null END)
+       > (CASE WHEN ss1.store_sales > 0 THEN ss2.store_sales/ss1.store_sales ELSE null END);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.3.ddl.sqlpp
new file mode 100644
index 0000000..3f600e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/comp-ASTERIXDB-2415/comp-ASTERIXDB-2415.3.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+drop dataverse tpcds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
index 075bfb8..5d87f1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/groupby-orderby-count/groupby-orderby-count.3.query.sqlpp
@@ -20,10 +20,10 @@
 use twitter;
 
 
-select element {'word':tok,'count':twitter.coll_count(( from g select value token ))}
+select element {'word':tok,'count':twitter.strict_count(( from g select value token ))}
 from  TwitterData as t,
       `word-tokens`(t.text) as token
 group by token as tok
 group as g
-order by coll_count(( from g select value token )) desc,tok
+order by strict_count(( from g select value token )) desc,tok
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/ping/ping.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/ping/ping.1.query.sqlpp
new file mode 100644
index 0000000..9f4f062
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/ping/ping.1.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+set `import-private-functions` `true`;
+select value val from ping() val order by val.node;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.1.ddl.sqlpp
new file mode 100644
index 0000000..a631576
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.1.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2380
+ */
+
+DROP dataverse test if exists;
+CREATE dataverse test;
+USE test;
+
+CREATE TYPE countryType AS OPEN {
+    `-car_code`: string,
+    `-area`: string,
+    population: string
+};
+
+CREATE DATASET country(countryType) PRIMARY KEY `-car_code`;
+
+CREATE INDEX countryID ON country(`-car_code`) TYPE BTREE;
+
+CREATE TYPE mountainType AS OPEN {
+    `id` : uuid,
+    `-id`: string,
+    `-country` : string
+};
+
+CREATE DATASET mountain(mountainType) PRIMARY KEY `id` autogenerated;
+
+CREATE TYPE seaType AS OPEN {
+    `id` : uuid,
+    `-id`: string,
+    `-country` : string
+};
+
+CREATE DATASET sea(seaType) PRIMARY KEY `id` autogenerated;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.2.update.sqlpp
new file mode 100644
index 0000000..160fe81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.2.update.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+USE test;
+
+LOAD DATASET country USING localfs
+    ((`path`=`asterix_nc1://data/country/country.adm`),("format"="adm"));
+
+LOAD DATASET mountain USING localfs
+    ((`path`=`asterix_nc1://data/country/mountain.adm`),("format"="adm"));
+
+LOAD DATASET sea USING localfs
+    ((`path`=`asterix_nc1://data/country/sea.adm`),("format"="adm"));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.query.sqlpp
new file mode 100644
index 0000000..c4a94fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+USE test;
+
+WITH commons AS (SELECT m1c AS first_country, m2c AS second_country,
+                        m1.name AS mountain, s1.name AS sea
+                 FROM mountain AS m1, sea AS s1, mountain AS m2, sea AS s2,
+                      split(m1.`-country`, " ") AS m1c, split(s1.`-country`, " ") AS s1c,
+                      split(m2.`-country`, " ") AS m2c, split(s2.`-country`, " ") AS s2c
+                 WHERE m1c = s1c AND m2c = s2c AND m1c != m2c AND m1.name = m2.name AND
+                       s1.name = s2.name AND m1c > m2c)
+
+SELECT DISTINCT c.first_country, c.second_country,
+       (SELECT VALUE object_remove(c2, "id")
+        FROM commons AS c2
+        WHERE c2.first_country = c.first_country AND c2.second_country = c.second_country
+        ORDER BY c2.first_country, c2.second_country, c2.mountain, c2.sea) AS mountains
+FROM commons AS c
+ORDER BY c.first_country;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp
new file mode 100644
index 0000000..bf35f87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": array_length((
+    from range(1, 6) t
+    select distinct value random()
+  )),
+
+  "t2": array_length((
+    from range(1, 6) t
+    select distinct value random(t)
+  )),
+
+  "t3": array_length((
+    from range(1, 6) t
+    select distinct value random(unix_time_from_datetime_in_ms(current_datetime()))
+  )),
+
+  "t4": array_length((
+    from [ tinyint("1"), smallint("2"), integer("3"), bigint("4"), float("5"), double("6") ] t
+    select distinct value random(t)
+  )),
+
+  "t5": [ random(missing) is missing, random(null) is null ]
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp
new file mode 100644
index 0000000..f650e0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+  id : bigint,
+  point : point,
+  kwds : string,
+  line1 : line,
+  line2 : line,
+  poly1 : polygon,
+  poly2 : polygon,
+  rec : rectangle
+};
+
+create type test.MyRecord as
+{
+  pid: uuid,
+  nested : MyRecordtmp?
+};
+
+create  dataset MyDatatmp(MyRecordtmp) primary key id;
+
+create  dataset MyData(MyRecord) primary key pid autogenerated;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp
new file mode 100644
index 0000000..8199dbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+load  dataset MyDatatmp using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`));
+
+insert into MyData
+select element {'nested':c}
+from  MyDatatmp as c
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp
new file mode 100644
index 0000000..5683fe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+create  index rtree_index_point  on MyData (nested.point) type rtree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp
new file mode 100644
index 0000000..93665a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use test;
+
+
+select element {'id':o.nested.id}
+from  MyData as o
+where test.`spatial-intersect`(o.nested.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp
new file mode 100644
index 0000000..639fbd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Index join for nested composite non-enforced indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+
+create dataset TestOpen2(TestOpenType) primary key c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp
new file mode 100644
index 0000000..0215367
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known. Promotable numeric type.
+  * Result       : Success
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_double(t1.nested.c_i8) /*+ indexnl */ = t2.nested.c_i8
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.11.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.11.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp
new file mode 100644
index 0000000..10fa158
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.2.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into TestOpen1 ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "extra": 99
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "extra": 98
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "extra": 97
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "extra": 996
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "extra": 9
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "extra": 91
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3,
+        "extra": 99
+    }
+});
+insert into TestOpen1 ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8,
+        "extra": 99
+    }
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp
new file mode 100644
index 0000000..f3d6e8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.3.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into TestOpen2 ({
+    "c_id": 101,
+    "nested": {
+        "c_x": 101,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "extra": 97
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 102,
+    "nested": {
+        "c_x": 102,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "extra": 96
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 103,
+    "nested": {
+        "c_x": 103,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "extra": 95
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 104,
+    "nested": {
+        "c_x": 104,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "extra": 94
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 105,
+    "nested": {
+        "c_x": 105,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "extra": 93
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 106,
+    "nested": {
+        "c_x": 106,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "extra": 91
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 107,
+    "nested": {
+        "c_x": 107,
+        "c_s": "world",
+        "c_i64": 3,
+        "extra": 97
+    }
+});
+insert into TestOpen2 ({
+    "c_id": 108,
+    "nested": {
+        "c_x": 108,
+        "extra": 97
+    }
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp
new file mode 100644
index 0000000..c8be482
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.4.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+use test;
+
+create index idx_t1_s on TestOpen1(nested.c_s:string, nested.extra: int);
+
+create index idx_t1_i64 on TestOpen1(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t1_i8 on TestOpen1(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t1_d on TestOpen1(nested.c_d:double, nested.extra: int);
+
+/********************************************/
+
+create index idx_t2_s on TestOpen2(nested.c_s:string, nested.extra: int);
+
+create index idx_t2_i64 on TestOpen2(nested.c_i64:int64, nested.extra: int);
+
+create index idx_t2_i8 on TestOpen2(nested.c_i8:int8, nested.extra: int);
+
+create index idx_t2_d on TestOpen2(nested.c_d:double, nested.extra: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp
new file mode 100644
index 0000000..1301e74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : No index join because there's no hint, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp
new file mode 100644
index 0000000..a91e785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : No index join because the probe type is unknown, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.nested.c_s /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp
new file mode 100644
index 0000000..a3c2a88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : No index join because there's no hint, non-enforced nested composite key index
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp
new file mode 100644
index 0000000..f8a990e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known (string), non-enforced nested composite
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.nested.c_s) /*+ indexnl */ = t2.nested.c_s
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp
new file mode 100644
index 0000000..40f27ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+ /*
+  * Description  : Index join because there's a hint and the probe type is known (bigint), non-enforced nested composite
+  */
+
+use test;
+
+select t1.nested.c_x as c1, t2.nested.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.nested.c_i64) /*+ indexnl */ = t2.nested.c_i64
+order by t1.nested.c_x, t2.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp
new file mode 100644
index 0000000..2d26551
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.1.ddl.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+/*
+ * Description    : Equi joins two datasets, DBLP and CSX, based on their authors.
+ *                  CSX has a nested composite open index, and given the 'indexnl' hint we expect the join to
+ *                  be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type test.DBLPTypetmp as
+ closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXTypetmp as
+ closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as {
+  id: int
+};
+
+create type test.CSXType as {
+  id: int
+};
+
+create  dataset DBLPtmp(DBLPTypetmp) primary key id;
+
+create  dataset CSXtmp(CSXTypetmp) primary key id;
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp
new file mode 100644
index 0000000..b6f719a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.2.update.sqlpp
Binary files differ
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.3.ddl.sqlpp
new file mode 100644
index 0000000..af7af0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use test;
+
+create index idx on CSX (nested.authors: string, nested.csxid: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp
new file mode 100644
index 0000000..6ec0578
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select element {'aid':a.nested.id,'bid':b.nested.id,'authors':a.nested.authors}
+from  DBLP as a, CSX as b
+where to_string(a.nested.authors) /*+ indexnl */  = b.nested.authors
+order by a.nested.id, b.nested.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.5.ddl.sqlpp
new file mode 100644
index 0000000..ff41019
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.5.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp
new file mode 100644
index 0000000..9c7b4e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Index selection for composite nested non-enforced indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp
new file mode 100644
index 0000000..7d4c7da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_d >= 3.25
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp
new file mode 100644
index 0000000..e8ff5b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 499 and t.nested.c_i8 < 99999
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp
new file mode 100644
index 0000000..07298b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.12.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp
new file mode 100644
index 0000000..c4046a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.2.update.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3,
+        "c_extra": "yes"
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8,
+        "c_extra": "yes"
+    }
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp
new file mode 100644
index 0000000..5a3d25a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.3.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+use test;
+
+create index idx_s on TestOpen(nested.c_s:string, nested.c_extra: string);
+
+create index idx_i64 on TestOpen(nested.c_i64:int64, nested.c_extra: string);
+
+create index idx_i8 on TestOpen(nested.c_i8:int8, nested.c_extra: string);
+
+create index idx_d on TestOpen(nested.c_d:double, nested.c_extra: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp
new file mode 100644
index 0000000..2462bda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp
new file mode 100644
index 0000000..0b59642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 = 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp
new file mode 100644
index 0000000..99a596f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp
new file mode 100644
index 0000000..ce16148
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i64 > 2.0
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp
new file mode 100644
index 0000000..c0b4563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp
new file mode 100644
index 0000000..c5cc2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2.5
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp
new file mode 100644
index 0000000..079b5ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Index selection for composite nested non-enforced indexes multiple indexes match.
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp
new file mode 100644
index 0000000..2ff5578
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.2.update.sqlpp
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "c_x": 1,
+    "nested": {
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8
+    }
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp
new file mode 100644
index 0000000..85928cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.3.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+use test;
+
+create index idx_2 on TestOpen(nested.c_s:string, nested.c_i64:int64);
+
+create index idx_1 on TestOpen(nested.c_s:string, nested.c_d:double);
+
+create index idx_3 on TestOpen(nested.c_i64:int64);
+
+create index idx_4 on TestOpen(nested.c_i8:int8);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp
new file mode 100644
index 0000000..2462bda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_s = 'world'
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp
new file mode 100644
index 0000000..b98ee96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select t.nested.c_x as res
+from TestOpen t
+where t.nested.c_i8 > 2 and t.nested.c_i64 < 3
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-02/non-enforced-02.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp
new file mode 100644
index 0000000..abf2843
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Multiple nested composite non-enforced indexes on the same field, created before data loading
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type GleambookUserType as { id: int };
+
+create dataset GleambookUsers(GleambookUserType) primary key id;
+
+create index gbUserSinceIdxDateTime on GleambookUsers(nested.userSince: datetime, nested.alias: string);
+
+create index gbUserSinceIdxInt on GleambookUsers(nested.userSince: int, nested.alias: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp
new file mode 100644
index 0000000..9427b77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into GleambookUsers
+([
+{ "id":1, "nested": {"alias":"Bram",    "userSince":datetime("2010-10-16T10:10:00")} },
+{ "id":2, "nested": {"alias":"Donald",  "userSince":"yesterday"} },
+{ "id":3, "nested": {"alias":"Hillery", "userSince":2016} }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp
new file mode 100644
index 0000000..7c80c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select value user
+from GleambookUsers user
+where user.nested.userSince > 900;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-03/non-enforced-03.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp
new file mode 100644
index 0000000..897df02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Index selection for non-enforced nested composite key indexes
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp
new file mode 100644
index 0000000..65b54fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.2.update.sqlpp
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+
+use test;
+
+insert into TestOpen ({
+    "c_id": 1,
+    "nested": {
+        "c_x": 1,
+        "c_s": "hello",
+        "c_i64": 2,
+        "c_i8": 2,
+        "c_d": 2
+    }
+});
+insert into TestOpen ({
+    "c_id": 2,
+    "nested": {
+        "c_x": 2,
+        "c_s": 2,
+        "c_i64": "2",
+        "c_i8": 2.5,
+        "c_d": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 3,
+    "nested": {
+        "c_x": 3,
+        "c_s": "world",
+        "c_i64": 2,
+        "c_i8": 4,
+        "c_d": 3.125
+    }
+});
+insert into TestOpen ({
+    "c_id": 4,
+    "nested": {
+        "c_x": 4,
+        "c_s": null,
+        "c_i64": null,
+        "c_i8": 500,
+        "c_d": 3.25
+    }
+});
+insert into TestOpen ({
+    "c_id": 5,
+    "nested": {
+        "c_x": 5,
+        "c_s": "hello",
+        "c_i64": 2.25,
+        "c_i8": 10000.25,
+        "c_d": 3.5
+    }
+});
+insert into TestOpen ({
+    "c_id": 6,
+    "nested": {
+        "c_x": 6,
+        "c_s": false,
+        "c_i64": false,
+        "c_i8": 2e100,
+        "c_d": 2e100
+    }
+});
+insert into TestOpen ({
+    "c_id": 7,
+    "nested": {
+        "c_x": 7,
+        "c_s": "world",
+        "c_i64": 3
+    }
+});
+insert into TestOpen ({
+    "c_id": 8,
+    "nested": {
+        "c_x": 8
+    }
+});
+insert into TestOpen ({
+    "c_id": 9,
+    "nested": {
+        "c_x": 9,
+        "c_d": 3.25
+    }
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp
new file mode 100644
index 0000000..c04b4d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use test;
+
+create index idx_i64_on_d on TestOpen(nested.c_d:int64, nested.c_x: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp
new file mode 100644
index 0000000..1c07ea6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use test;
+
+select value t.nested.c_x
+from TestOpen t
+where t.nested.c_d = 3.25
+order by t.nested.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-open-index/index-selection/non-enforced-04/non-enforced-04.5.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..c4af7d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+  id : bigint,
+  point : point
+};
+
+create type test.MyRecord as
+{
+  nested : MyRecordtmp?
+};
+
+create  dataset MyData(MyRecord) primary key nested.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
index 2f0d837..0abb997 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissing/ifmissing.1.query.sqlpp
@@ -29,5 +29,12 @@
             missing,
             case when get_year(current_datetime()) > 0 then missing else false end,
             case when get_year(current_datetime()) > 0 then true else null end
+        ),
+   "j": (
+            let v = if_missing(
+                case when get_year(current_datetime()) > 0 then missing else false end,
+                { "c": [ 2 ] }
+            )
+            select v as b
         )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
index 32f040f..22a8acd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifmissingornull/ifmissingornull.1.query.sqlpp
@@ -32,5 +32,12 @@
             case when get_year(current_datetime()) > 0 then missing else false end,
             case when get_year(current_datetime()) > 0 then null else false end,
             case when get_year(current_datetime()) > 0 then true else missing end
+        ),
+   "j": (
+            let v = if_missing_or_null(
+                case when get_year(current_datetime()) > 0 then missing else false end,
+                { "c": [ 2 ] }
+            )
+            select v as b
         )
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
index c0683bd..0121cd8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/ifnull/ifnull.1.query.sqlpp
@@ -30,5 +30,12 @@
             null,
             case when get_year(current_datetime()) > 0 then null else false end,
             case when get_year(current_datetime()) > 0 then true else missing end
+        ),
+   "j": (
+            let v = if_null(
+                case when get_year(current_datetime()) > 0 then null else false end,
+                { "c": [ 2 ] }
+            )
+            select v as b
         )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
index 81f2a3f..77e8d91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
@@ -41,5 +41,11 @@
   "21": 1 is valued,
   "22": null is not valued,
   "23": missing is not valued,
-  "24": 1 is not valued
+  "24": 1 is not valued,
+  "25": null is known,
+  "26": missing is known,
+  "27": 1 is known,
+  "28": null is not known,
+  "29": missing is not known,
+  "30": 1 is not known
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp
new file mode 100644
index 0000000..fe67901
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+{
+  "e": trunc(e(), 2),
+  "pi": trunc(pi(), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp
new file mode 100644
index 0000000..80e002e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": tobigint(degrees(pi()))
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_double/divide_double.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_double/divide_double.1.query.sqlpp
index 901be61..31e2399 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_double/divide_double.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_double/divide_double.1.query.sqlpp
@@ -18,4 +18,4 @@
  */
 
 
-{'result1':(double('-6.5d') / tinyint('+1')),'result2':(double('-6.5d') / smallint('2')),'result3':(double('-6.5d') / integer('+3')),'result4':(double('-6.5d') / bigint('-4')),'result5':(double('-6.5d') / float('-5.5f')),'result6':(double('-6.5d') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(double('-6.5d') / tinyint('+1')),'result2':(double('-6.5d') / smallint('2')),'result3':(double('-6.5d') / integer('+3')),'result4':(double('-6.5d') / bigint('-4')),'result5':(double('-6.5d') / float('-5.5f')),'result6':(double('-6.5d') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a, 'result9':double('1') / 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_float/divide_float.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_float/divide_float.1.query.sqlpp
index 7dfdcd2..107fed1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_float/divide_float.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_float/divide_float.1.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-{'result1':(float('-5.5f') / tinyint('+1')),'result2':(float('-5.5f') / smallint('2')),'result3':(float('-5.5f') / integer('+3')),'result4':(float('-5.5f') / bigint('-4')),'result5':(float('-5.5f') / float('-5.5f')),'result6':(float('-5.5f') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(float('-5.5f') / tinyint('+1')),'result2':(float('-5.5f') / smallint('2')),'result3':(float('-5.5f') / integer('+3')),'result4':(float('-5.5f') / bigint('-4')),'result5':(float('-5.5f') / float('-5.5f')),'result6':(float('-5.5f') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a, 'result9':double('1') / 0 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.1.query.sqlpp
index b0f315d..1e1237d7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.1.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-{'result1':(smallint('2') / tinyint('+1')),'result2':(smallint('2') / smallint('2')),'result3':(smallint('2') / integer('+3')),'result4':(smallint('2') / bigint('-4')),'result5':(smallint('2') / float('-5.5f')),'result6':(smallint('2') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(smallint('2') / tinyint('+1')),'result2':(smallint('2') / smallint('2')),'result3':(smallint('2') / integer('+3')),'result4':(smallint('2') / bigint('-4')),'result5':(smallint('2') / float('-5.5f')),'result6':(smallint('2') / double('-6.5d')),'result7':(smallint('-6') / null), 'result8':smallint('-6') / {}.a, 'result9': smallint('1') / 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp
new file mode 100644
index 0000000..f5bf6fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+{'result1':(smallint('2') div tinyint('+1')),'result2':(smallint('2') div smallint('2')),'result3':(smallint('2') div integer('+3')),'result4':(smallint('2') div bigint('-4')),'result5':(smallint('2') div float('-5.5f')),'result6':(smallint('2') div double('-6.5d')),'result7':(smallint('-6') div null), 'result8':smallint('-6') div {}.a, 'result9': smallint('1') div 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.1.query.sqlpp
index 4a9810d..3e22be1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.1.query.sqlpp
@@ -18,4 +18,4 @@
  */
 
 
-{'result1':(integer('+3') / tinyint('+1')),'result2':(integer('+3') / smallint('2')),'result3':(integer('+3') / integer('+3')),'result4':(integer('+3') / bigint('-4')),'result5':(integer('+3') / float('-5.5f')),'result6':(integer('+3') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(integer('+3') / tinyint('+1')),'result2':(integer('+3') / smallint('2')),'result3':(integer('+3') / integer('+3')),'result4':(integer('+3') / bigint('-4')),'result5':(integer('+3') / float('-5.5f')),'result6':(integer('+3') / double('-6.5d')),'result7':(integer('-6') / null), 'result8':integer('-6') / {}.a, 'result9': integer('1') / 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
new file mode 100644
index 0000000..dec8444
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+
+{'result1':(integer('+3') div tinyint('+1')),'result2':(integer('+3') div smallint('2')),'result3':(integer('+3') div integer('+3')),'result4':(integer('+3') div bigint('-4')),'result5':(integer('+3') div float('-5.5f')),'result6':(integer('+3') div double('-6.5d')),'result7':(integer('-6') div null), 'result8':integer('-6') div {}.a, 'result9': integer('1') div 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.1.query.sqlpp
index 5b7d8f5..abdbe08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.1.query.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-{'result1':(bigint('-4') / tinyint('+1')),'result2':(bigint('-4') / smallint('2')),'result3':(bigint('-4') / integer('+3')),'result4':(bigint('-4') / bigint('-4')),'result5':(bigint('-4') / float('-5.5f')),'result6':(bigint('-4') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(bigint('-4') / tinyint('+1')),'result2':(bigint('-4') / smallint('2')),'result3':(bigint('-4') / integer('+3')),'result4':(bigint('-4') / bigint('-4')),'result5':(bigint('-4') / float('-5.5f')),'result6':(bigint('-4') / double('-6.5d')),'result7':(bigint('-6') / null), 'result8':bigint('-6') / {}.a, 'result9': bigint('1') / 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp
new file mode 100644
index 0000000..d3bb58c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+{'result1':(bigint('-4') div tinyint('+1')),'result2':(bigint('-4') div smallint('2')),'result3':(bigint('-4') div integer('+3')),'result4':(bigint('-4') div bigint('-4')),'result5':(bigint('-4') div float('-5.5f')),'result6':(bigint('-4') div double('-6.5d')),'result7':(bigint('-6') div null), 'result8':bigint('-6') div {}.a, 'result9': bigint('1') div 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.1.query.sqlpp
index dd6917d..a4049ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.1.query.sqlpp
@@ -18,4 +18,4 @@
  */
 
 
-{'result1':(tinyint('+1') / tinyint('+1')),'result2':(tinyint('+1') / smallint('2')),'result3':(tinyint('+1') / integer('+3')),'result4':(tinyint('+1') / bigint('-4')),'result5':(tinyint('+1') / float('-5.5f')),'result6':(tinyint('+1') / double('-6.5d')),'result7':(double('-6.5d') / null), 'result8':double('-6.5d') / {}.a};
+{'result1':(tinyint('+1') / tinyint('+1')),'result2':(tinyint('+1') / smallint('2')),'result3':(tinyint('+1') / integer('+3')),'result4':(tinyint('+1') / bigint('-4')),'result5':(tinyint('+1') / float('-5.5f')),'result6':(tinyint('+1') / double('-6.5d')),'result7':(tinyint('-6') / null), 'result8':tinyint('-6') / {}.a, 'result9': tinyint('1') / 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
new file mode 100644
index 0000000..5d577a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+
+{'result1':(tinyint('+1') div tinyint('+1')),'result2':(tinyint('+1') div smallint('2')),'result3':(tinyint('+1') div integer('+3')),'result4':(tinyint('+1') div bigint('-4')),'result5':(tinyint('+1') div float('-5.5f')),'result6':(tinyint('+1') div double('-6.5d')),'result7':(tinyint('-6') div null), 'result8':tinyint('-6') div {}.a, 'result9': tinyint('1') div 0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
index 3b395bc..479e290 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
@@ -40,6 +40,7 @@
     [ 18, isnull(ifinf(double("INF"), double("-INF"), [], 2)) ],
     [ 19, ismissing(if_inf(double("INF"), double("-INF"), missing, 2)) ],
     [ 20, tostring(ifinf(float("INF"), float("NaN"), 2)) ],
-    [ 21, if_inf(2, 1/0) ]
+    [ 21, if_inf(2, 1/0) ],
+    [ 22, if_inf(1/0, 2) ]
 ] t
 order by t[0]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp
new file mode 100644
index 0000000..6d04878
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": trunc(radians(180), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 507e276..cfd8aeb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -118,6 +118,44 @@
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
+    <compilation-unit name="object_remove">
+      <output-dir compare="Text">object_remove</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_rename">
+      <output-dir compare="Text">object_rename</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_unwrap">
+      <output-dir compare="Text">object_unwrap</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_replace">
+      <output-dir compare="Text">object_replace</output-dir>
+      <expected-error>function object-replace expects its 2nd input parameter to be of type primitive</expected-error>
+      <expected-error>function object-replace expects its 2nd input parameter to be of type primitive</expected-error>
+      <source-location>false</source-location>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">>
+    <compilation-unit name="object_add">
+      <output-dir compare="Text">object_add</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_put">
+      <output-dir compare="Text">object_put</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="object_values">
+      <output-dir compare="Text">object_values</output-dir>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
     <compilation-unit name="object_pairs">
       <output-dir compare="Text">object_pairs</output-dir>
     </compilation-unit>
@@ -140,19 +178,27 @@
   <test-case FilePath="objects">
     <compilation-unit name="closed-closed-fieldname-conflict_issue173">
       <output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
-      <expected-error>Closed fields 0 and 1 have the same field name "name"</expected-error>
+      <expected-error>Duplicate field name "name"</expected-error>
+      <source-location>false</source-location>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
     <compilation-unit name="open-closed-fieldname-conflict_issue173">
       <output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
       <expected-error>Open field "name" has the same field name as closed field at index 0</expected-error>
+      <source-location>false</source-location>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
     <compilation-unit name="open-open-fieldname-conflict_issue173">
       <output-dir compare="Text">open-open-fieldname-conflict_issue173</output-dir>
       <expected-error>Open fields 0 and 1 have the same field name "name"</expected-error>
+      <source-location>false</source-location>
+    </compilation-unit>
+  </test-case>
+  <test-case FilePath="objects">
+    <compilation-unit name="pairs">
+      <output-dir compare="Text">pairs</output-dir>
     </compilation-unit>
   </test-case>
 </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.1.ddl.sqlpp
new file mode 100644
index 0000000..0573cc7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_add under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.2.update.sqlpp
new file mode 100644
index 0000000..2e1e348
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_add under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.3.query.sqlpp
new file mode 100644
index 0000000..b3c90a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.3.query.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_add under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_add(missing, missing, "value") is missing,
+    object_add(null, missing, "value") is missing,
+    object_add(missing, null, missing) is missing,
+    object_add(null, null, null) is null,
+    object_add("a", null, null) is null,
+    object_add("non-object", "field",  "value") is null,
+    object_add({"a":1}, 1,  "value") is null
+  ],
+  "t2": object_add({"a":1}, "field", "value"),
+  "t3": object_add({"a":1}, "field", null),
+  "t4": object_add({"a":1}, "field", {"x":["y", "z"]}),
+  "t5": object_add({"a":1}, "a", 5),
+  "t6": object_add({"a":1}, "field", missing),
+
+  /* open type */
+  "t7": (
+    select value object_add(u, "field", "value")
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t8": (
+    select value object_add(m, "field", {"a":1})
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.4.ddl.sqlpp
new file mode 100644
index 0000000..58af156
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_add under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_pairs/object_pairs.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_pairs/object_pairs.2.query.sqlpp
new file mode 100644
index 0000000..27a088f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_pairs/object_pairs.2.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+{
+  "t1": object_pairs(missing) is missing,
+  "t2": object_pairs(null) is null,
+  "t3": object_pairs(1) is null,
+  "t4": object_pairs("a") is null,
+  "t5": object_pairs(true) is null,
+  "t6": object_pairs([]) is null,
+  "t7": object_pairs([{"a":1}]) is null,
+  "t8": object_pairs({"a":{"b":{"c":{"d":1}}}})
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.1.ddl.sqlpp
new file mode 100644
index 0000000..6a7bed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_put under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.2.update.sqlpp
new file mode 100644
index 0000000..c40cedd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_put under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.3.query.sqlpp
new file mode 100644
index 0000000..3706fd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.3.query.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_put under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_put(missing, missing, "value") is missing,
+    object_put(null, missing, "value") is missing,
+    object_put(missing, null, missing) is missing,
+    object_put(null, null, null) is null,
+    object_put("a", null, null) is null,
+    object_put("non-object", "field",  "value") is null,
+    object_add({"a":1}, 1,  "value") is null
+  ],
+  "t2": object_put({"a":1}, "field", "value"),
+  "t3": object_put({"a":1}, "field", null),
+  "t4": object_put({"a":1}, "field", {"x":["y", "z"]}),
+  "t5": object_put({"a":1}, "a", "replaced"),
+  "t6": object_put({"a":1, "removed":2}, "removed", missing),
+
+  /* open type */
+  "t7": (
+    select value object_put(u, "field", "value")
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t8": (
+    select value object_put(m, "field", {"a":1})
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.4.ddl.sqlpp
new file mode 100644
index 0000000..050057f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_put under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp
new file mode 100644
index 0000000..b3dafba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp
new file mode 100644
index 0000000..2d03f92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
new file mode 100644
index 0000000..f7eabc3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_remove(missing, missing) is missing,
+    object_remove(null, missing) is missing,
+    object_remove("{}", missing) is missing,
+    object_remove("{}", null) is null
+  ],
+  "t2": object_remove({"a":1, "b":2}, "a"),
+  "t3": object_remove({"a":1, "b":2}, "b"),
+  "t4": object_remove({"a":1, "b":2}, "c"),
+
+  /* open type */
+  "t5": (
+    select value object_remove(u, "lang")
+    from TwitterUsers as u
+    order by u.screen-name
+  ),
+
+  /* closed type */
+  "t6": (
+    select value object_remove(m, "user")
+    from TweetMessages as m
+    order by m.tweetid
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp
new file mode 100644
index 0000000..d6b47a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_remove under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp
new file mode 100644
index 0000000..f6710f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp
new file mode 100644
index 0000000..5457dc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp
new file mode 100644
index 0000000..5f74fc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_rename(missing, missing, missing) is missing,
+    object_rename(null, missing, missing) is missing,
+    object_rename(null, null, missing) is missing,
+    object_rename(null, null, null) is null
+  ],
+  "t2": object_rename({"a":1, "b":2}, "b", "c"),
+  "t3": object_rename({"a":1, "b":2}, "z", "a"),
+
+  /* open type */
+  "t5": (
+    select value object_rename(u, "lang", "language")
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t6": (
+    select value object_rename(m, "user", "user_details")
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp
new file mode 100644
index 0000000..22bc025
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_rename/object_rename.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_rename under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.1.ddl.sqlpp
new file mode 100644
index 0000000..0bd154e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.2.update.sqlpp
new file mode 100644
index 0000000..a3baf1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.3.query.sqlpp
new file mode 100644
index 0000000..5718d8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.3.query.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_replace(missing, missing, missing) is missing,
+    object_replace(null, missing, missing) is missing,
+    object_replace(null, null, missing) is missing,
+    object_replace(null, null, null) is null,
+    object_replace({"a":1}, null, "z") is null,
+    object_replace("non-object", "a", "z") is null
+  ],
+  "t2": object_replace({"a":"1", "b":2},"1", 2),
+  "t3": object_replace({"a":"1", "b":2}, 2, "3"),
+  "t4": object_replace({"a":1}, 1, null),
+  "t5": object_replace({"a":1, "b":1}, 1, 2),
+  "t6": object_replace({"a":1.0, "b":1}, 1, 2),
+
+  /* open type */
+  "t7": (
+    select value object_replace(u, "en", {"en":"native"})
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t8": (
+    select value object_replace(m, "1", "1000")
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.4.query.sqlpp
new file mode 100644
index 0000000..15c0bce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Failure
+ */
+
+use TinySocial;
+
+select value object_replace({"a":1}, {"b":2}, "z");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.5.query.sqlpp
new file mode 100644
index 0000000..a46a727
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Failure
+ */
+
+use TinySocial;
+
+select value object_replace({"a":1}, ["b","c"], "z");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.6.ddl.sqlpp
new file mode 100644
index 0000000..1dd03f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_replace/object_replace.6.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_replace under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp
new file mode 100644
index 0000000..1aba6e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp
new file mode 100644
index 0000000..8f854e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
new file mode 100644
index 0000000..b79bd8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_unwrap(missing) is missing,
+    object_unwrap(null) is null,
+    object_unwrap("non-object") is null,
+    object_unwrap({"a":1, "b":2}) is null,
+    object_unwrap({}) is null
+  ],
+
+  "t2": object_unwrap({"a":"1"}),
+  "t3": object_unwrap({"a":{"b":"1"}}),
+  "t4": object_unwrap({"a":[1, 2]}),
+
+  "t5": (
+    select object_unwrap(o) from (
+      select lang
+      from TwitterUsers u
+      order by lang
+      limit 1
+    ) o
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
new file mode 100644
index 0000000..af175df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_unwrap under different queries.
+ * Expected Res : Success
+ */
+
+drop dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.1.ddl.sqlpp
new file mode 100644
index 0000000..1242661
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_values under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.2.update.sqlpp
new file mode 100644
index 0000000..72edf0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_values under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.3.query.sqlpp
new file mode 100644
index 0000000..ebd0a80
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.3.query.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_values under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    object_values(missing) is missing,
+    object_values(null) is null,
+    object_values("non-object") is null
+  ],
+  "t2": object_values({"object":{"a":3}, "array":[1,2], "primitive": 4}),
+
+  /* open type */
+  "t3": (
+    select value object_values(u)
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t4": (
+    select value object_values(m)
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.4.ddl.sqlpp
new file mode 100644
index 0000000..158cefa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_values/object_values.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing object_values under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.1.ddl.sqlpp
new file mode 100644
index 0000000..b3d125a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing pairs() under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+create type TinySocial.TweetMessageType as closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TwitterUsers(TwitterUserType) primary key `screen-name`;
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.2.update.sqlpp
new file mode 100644
index 0000000..9a25de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing pairs() under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+load  dataset TwitterUsers using localfs ((`path`=`asterix_nc1://data/tinysocial/twu.adm`),(`format`=`adm`));
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.3.query.sqlpp
new file mode 100644
index 0000000..a1a63cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.3.query.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing pairs() under different queries.
+ * Expected Res : Success
+ */
+
+use TinySocial;
+
+{
+  "t1": [
+    pairs(missing) is missing,
+    pairs(null) is null,
+    pairs("non-object") is null
+  ],
+
+  "t2": [
+    pairs({}),
+    pairs([]),
+    pairs([{}]),
+    pairs([1,2]),
+    pairs([1,2,null,{"a":3}])
+  ],
+
+  "t3": pairs({"object":{"a":{"b":{"c":3}}}, "array":[1,2], "primitive": 4}),
+
+  /* open type */
+  "t4": (
+    select value pairs(u)
+    from TwitterUsers as u
+    order by u.screen-name
+    limit 1
+  ),
+
+  /* closed type */
+  "t5": (
+    select value pairs(m)
+    from TweetMessages as m
+    order by m.tweetid
+    limit 1
+  )
+};
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.4.ddl.sqlpp
new file mode 100644
index 0000000..7e23af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/pairs/pairs.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Testing pairs() under different queries.
+ * Expected Res : Success
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
index 2b009ae..d1e30ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue258/query-issue258.2.update.sqlpp
@@ -28,7 +28,7 @@
 
 INSERT INTO ds1
 CASE
-  WHEN coll_count((
+  WHEN strict_count((
       select element x
       from  ds2 as x
       where (x.id = 10)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
index 341481a..f0782fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element [x.int_m, coll_count(( from g select value x ))]
+select element [x.int_m, strict_count(( from g select value x ))]
 from  DataOpen as x
 let id = x.id
 group by x.int_m
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
index a1c1923..c7b9b23 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
@@ -27,7 +27,7 @@
            FROM OrdersOpen o
            WHERE o.o_custkey >= -1
           )
-SELECT VALUE COLL_COUNT((
+SELECT VALUE STRICT_COUNT((
   SELECT VALUE l
   FROM l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
index 0e6f4e9..a1b3bac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
@@ -27,7 +27,7 @@
            FROM OrdersOpen o
            WHERE o.o_custkey >= -1
           )
-SELECT VALUE COLL_COUNT((
+SELECT VALUE STRICT_COUNT((
   SELECT VALUE l
   FROM l
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
index f0fe4d0..e2f30a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/everysat_03/everysat_03.3.query.sqlpp
@@ -23,7 +23,7 @@
  * Date             : 5th July 2012
  */
 
-with  a as [every x in [1,2] satisfies (coll_avg([x,1]) = 1),every x in ['1','2'] satisfies (string(x) = '1'),every x in ['1','2'] satisfies (`string-length`(x) = 1),every x in [[1,2],[10],[1,5,7,8]] satisfies (coll_count(x) = 1),every x in [[2],[10],[8]] satisfies (coll_count(x) = 1),every x in [true,false] satisfies boolean('true'),every x in [true,true] satisfies not(x),every x in [1,2,3],
+with  a as [every x in [1,2] satisfies (strict_avg([x,1]) = 1),every x in ['1','2'] satisfies (string(x) = '1'),every x in ['1','2'] satisfies (`string-length`(x) = 1),every x in [[1,2],[10],[1,5,7,8]] satisfies (strict_count(x) = 1),every x in [[2],[10],[8]] satisfies (strict_count(x) = 1),every x in [true,false] satisfies boolean('true'),every x in [true,true] satisfies not(x),every x in [1,2,3],
 y in [4,5,6] satisfies ((x + y) = 5),every x in [1,2,3],
 y in [4,5,6] satisfies ((x - y) = 5),every x in [1,2,3],
 y in [4,5,6] satisfies ((x * y) = 10),every x in ['ab','cd'],
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
index 5274bb5..8975aee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_03/somesat_03.3.query.sqlpp
@@ -22,7 +22,7 @@
  * Date            : 6th July 2012
  */
 
-with  a as [some x in [1,2] satisfies ((x + x) = 3),some x in [1,2] satisfies ((x + x) = 2),some x in [1,2] satisfies ((x - 2) = 2),some x in [1,2] satisfies ((x - 2) = 0),some x in [1,2] satisfies ((x * 2) = 4),some x in [1,2] satisfies ((x / 2) = 1),some x in [1,2] satisfies (coll_avg([x,1]) = 1),some x in [1,2] satisfies boolean('true'),some x in [1,2] satisfies boolean('false'),some x in [true,false] satisfies not(x),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2))]
+with  a as [some x in [1,2] satisfies ((x + x) = 3),some x in [1,2] satisfies ((x + x) = 2),some x in [1,2] satisfies ((x - 2) = 2),some x in [1,2] satisfies ((x - 2) = 0),some x in [1,2] satisfies ((x * 2) = 4),some x in [1,2] satisfies ((x / 2) = 1),some x in [1,2] satisfies (strict_avg([x,1]) = 1),some x in [1,2] satisfies boolean('true'),some x in [1,2] satisfies boolean('false'),some x in [true,false] satisfies not(x),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2))]
 select element i
 from  a as i
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
index 562ff36..0212bb4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/somesat_04/somesat_04.3.query.sqlpp
@@ -24,7 +24,7 @@
  * Date             : 5th July 2012
  */
 
-with  a as [some x in ['foo','foobar','foot','fox'] satisfies (`string-length`(x) = 3),some x in [[5,4,3,2],[1,2,3,4,5,6,7,8],[4,2,3,4]] satisfies (coll_count(x) = 8),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2)),some x in ['A','B','C'] satisfies (x = 'A'),some x in [1,2,3],
+with  a as [some x in ['foo','foobar','foot','fox'] satisfies (`string-length`(x) = 3),some x in [[5,4,3,2],[1,2,3,4,5,6,7,8],[4,2,3,4]] satisfies (strict_count(x) = 8),some x in [1,2] satisfies ((x = 1) or (x = 2)),some x in [1,2] satisfies ((x = 1) and ((x + 1) = 2)),some x in ['A','B','C'] satisfies (x = 'A'),some x in [1,2,3],
 y in [4,5,6] satisfies ((x + y) = 5),some x in [1,2,3],
 y in [4,5,6] satisfies ((x - y) = 5),some x in [1,2,3],
 y in [4,5,6] satisfies ((x * y) = 10),some x in [1,2,3],
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
index 7a44bfe..62d2437 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/resolution/fullyqualified2/fullyqualified2.3.query.sqlpp
@@ -19,4 +19,4 @@
 
 use test1;
 
-coll_count(test.samptable);
+strict_count(test.samptable);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.1.query.sqlpp
index 3598c13..fcee460 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.1.query.sqlpp
@@ -19,8 +19,9 @@
 
 /*
  * Description  : Invalid data type in select var.*
- * Expected Res : Failure
+ * Expected Res : Success
  */
 
  SELECT t, t.*
  FROM [1, 2, 3] t
+ ORDER BY t
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.2.query.sqlpp
new file mode 100644
index 0000000..a2a6dd3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-star/var_star_2/var_star_2.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Invalid data type in select var.*
+ * Expected Res : Success
+ */
+
+ SELECT t.*
+ FROM [1, 2, 3] t
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
index 9705de5..4238881 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/semistructured/count-nullable/count-nullable.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'custage':age,'count': coll_count(( from g select value c))}
+select element {'custage':age,'count': strict_count(( from g select value c))}
 from  Customers as c
 group by c.age as age
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
index 63442c4..da03b3b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation-with-filtering/cell-aggregation-with-filtering.3.query.sqlpp
@@ -27,6 +27,6 @@
 where (`spatial-intersect`(t.loc,region) and (t.time > datetime('2011-05-15T00:00:00Z')) and (t.time < datetime('2011-05-16T23:59:59Z')) and contains(t.text,keywords))
 group by `spatial-cell`(t.loc,`create-point`(24.5,-125.5),3.0,3.0) as c
 group as g
-let num = coll_count((from g select value t))
+let num = strict_count((from g select value t))
 order by num
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
index eac020a..179fa9d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/cell-aggregation/cell-aggregation.3.query.sqlpp
@@ -25,7 +25,7 @@
       from  MyData as o
       group by `spatial-cell`(o.loc, `create-point`(0.0,0.0),5.0,5.0) as c
       group as g
-      let  num = coll_count(( from g select value o ))
+      let  num = strict_count(( from g select value o ))
       order by num
   )
 select element g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.1.ddl.sqlpp
new file mode 100644
index 0000000..a8dd17b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.1.ddl.sqlpp
@@ -0,0 +1,92 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test case Name  : spatial-large-data.sqlpp
+ * Description     : This test is intended to test creating an rtree index on cloudberry sample tweets with a nullable field
+ * Expected Result : Success
+ * Date            : Aug 9 2018
+ */
+
+drop dataverse twitter if exists;
+create dataverse twitter if not exists;
+use twitter;
+create type typeUser if not exists as open {
+    id: int64,
+    name: string,
+    screen_name : string,
+    lang : string,
+    location: string,
+    create_at: date,
+    description: string,
+    followers_count: int32,
+    friends_count: int32,
+    statues_count: int64
+};
+create type typePlace if not exists as open{
+    country : string,
+    country_code : string,
+    full_name : string,
+    id : string,
+    name : string,
+    place_type : string,
+    bounding_box : rectangle
+};
+create type typeGeoTag if not exists as open {
+    stateID: int32,
+    stateName: string,
+    countyID: int32,
+    countyName: string,
+    cityID: int32?,
+    cityName: string?
+};
+create type typeTweet if not exists as open{
+    create_at : datetime,
+    id: int64,
+    `text`: string,
+    in_reply_to_status : int64,
+    in_reply_to_user : int64,
+    favorite_count : int64,
+    coordinate: point?,
+    retweet_count : int64,
+    lang : string,
+    is_retweet: boolean,
+    hashtags : {{ string }} ?,
+    user_mentions : {{ int64 }} ? ,
+    user : typeUser,
+    place : typePlace?,
+    geo_tag: typeGeoTag
+};
+create dataset ds_tweet(typeTweet) if not exists primary key id
+with filter on create_at
+with {
+  "merge-policy": {
+    "name": "prefix",
+    "parameters": { "max-mergable-component-size": 1073741824, "max-tolerance-component-count": 5 }
+  }
+};
+
+create feed TweetFeed with {
+  "adapter-name" : "localfs",
+  "path" : "asterix_nc1://data/fulltext/cloudberry_sample_tweet.adm",
+  "address-type" : "nc",
+  "type-name" : "typeTweet",
+  "format" : "adm",
+  "insert-feed" : "true"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.2.update.sqlpp
new file mode 100644
index 0000000..5006988
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use twitter;
+
+set `wait-for-completion-feed` "true";
+connect feed TweetFeed to dataset ds_tweet;
+start feed TweetFeed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.3.ddl.sqlpp
new file mode 100644
index 0000000..1a76cae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use twitter;
+
+create index location_idx if not exists on ds_tweet(coordinate) type rtree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.4.query.sqlpp
new file mode 100644
index 0000000..5f0f66d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial-large-data/spatial-large-data.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use twitter;
+
+select count(*)
+from ds_tweet
+where `spatial-intersect`(coordinate,`create-rectangle`(`create-point`(-100.0, -100.0), `create-point`(100.0, 100.0)))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp
new file mode 100644
index 0000000..1763959
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named and positional statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[2.5, " world"]
+// param $p_int:json=2
+// param $p_str:json="hello"
+
+{
+  "t1": $p_int + ? + $1,
+  "t2": $p_str || ? || $2
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp
new file mode 100644
index 0000000..c01c253
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named and positional statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[" b", " c"]
+// param $p_str:json="a"
+
+{
+  "t1": $p_str || ? || $1 || ? || $2
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp
new file mode 100644
index 0000000..68d6aad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p_null:json=null
+// param $p_bool:json=true
+// param $p_int:json=42
+// param $p_dec:json=42.5
+// param $p_dbl:json=42.5e2
+// param $p_str:json="hello"
+// param $p_arr:json=["99",100,{"a":null},null,true]
+// param $p_obj:json={"a":[1,2,3]}
+
+{
+  "t1": {
+    "p_null": $p_null,
+    "p_bool": $p_bool,
+    "p_int": $p_int,
+    "p_dec": $p_dec,
+    "p_dbl": $p_dbl,
+    "p_str": $p_str,
+    "p_arr": $p_arr,
+    "p_obj": $p_obj
+  },
+
+  "t2": {
+    "p_null_type": $p_null is null,
+    "p_bool_type": is_boolean($p_bool),
+    "p_int_type": is_number($p_int),
+    "p_dec_type": is_number($p_dec),
+    "p_dbl_type": is_number($p_dbl),
+    "p_str_type": is_string($p_str),
+    "p_arr_type": is_array($p_arr),
+    "p_obj_type": is_object($p_obj)
+  },
+
+  "t3": [ $p_null, $p_bool, $p_int, $p_dec, $p_dbl, $p_str, $p_arr, $p_obj ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp
new file mode 100644
index 0000000..6ecefd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with url encoded request.
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param $p_null:json=null
+// param $p_bool:json=true
+// param $p_int:json=42
+// param $p_dec:json=42.5
+// param $p_dbl:json=42.5e2
+// param $p_str:json="hello"
+// param $p_arr:json=["99",100,{"a":null},null,true]
+// param $p_obj:json={"a":[1,2,3]}
+
+{
+  "t1": {
+    "p_null": $p_null,
+    "p_bool": $p_bool,
+    "p_int": $p_int,
+    "p_dec": $p_dec,
+    "p_dbl": $p_dbl,
+    "p_str": $p_str,
+    "p_arr": $p_arr,
+    "p_obj": $p_obj
+  },
+
+  "t2": {
+    "p_null_type": is_string($p_null),
+    "p_bool_type": is_string($p_bool),
+    "p_int_type": is_string($p_int),
+    "p_dec_type": is_string($p_dec),
+    "p_dbl_type": is_string($p_dbl),
+    "p_str_type": is_string($p_str),
+    "p_arr_type": is_string($p_arr),
+    "p_obj_type": is_string($p_obj)
+  },
+
+  "t3": [ $p_null, $p_bool, $p_int, $p_dec, $p_dbl, $p_str, $p_arr, $p_obj ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp
new file mode 100644
index 0000000..f2ec9e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a named parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p1:json="hello"
+
+$p1 || $p2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp
new file mode 100644
index 0000000..522b776
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test autogenerated column aliases named statement parameters
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p_int:json=42
+// param $p_str:json="hello"
+
+select $p_int, $p_str
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_04/named_04.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_04/named_04.1.query.sqlpp
new file mode 100644
index 0000000..720c2f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_04/named_04.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test fix for ASTERIXDB-2418
+ * Expected Res : Success
+ * Date         : Jul 2018
+ */
+
+// requesttype=application/json
+
+// param $p_int:json=42
+
+select value $p_int
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp
new file mode 100644
index 0000000..ef71010
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional ($) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": $1,
+    "p_bool": $2,
+    "p_int": $3,
+    "p_dec": $4,
+    "p_dbl": $5,
+    "p_str": $6,
+    "p_arr": $7,
+    "p_obj": $8
+  },
+
+  "t2": {
+    "p_null_type": $1 is null,
+    "p_bool_type": is_boolean($2),
+    "p_int_type": is_number($3),
+    "p_dec_type": is_number($4),
+    "p_dbl_type": is_number($5),
+    "p_str_type": is_string($6),
+    "p_arr_type": is_array($7),
+    "p_obj_type": is_object($8)
+  },
+
+  "t3": [ $1, $2, $3, $4, $5, $6, $7, $8 ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp
new file mode 100644
index 0000000..a31a646
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional ($) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": $1,
+    "p_bool": $2,
+    "p_int": $3,
+    "p_dec": $4,
+    "p_dbl": $5,
+    "p_str": $6,
+    "p_arr": $7,
+    "p_obj": $8
+  },
+
+  "t2": {
+    "p_null_type": $1 is null,
+    "p_bool_type": is_boolean($2),
+    "p_int_type": is_number($3),
+    "p_dec_type": is_number($4),
+    "p_dbl_type": is_number($5),
+    "p_str_type": is_string($6),
+    "p_arr_type": is_array($7),
+    "p_obj_type": is_object($8)
+  },
+
+  "t3": [ $1, $2, $3, $4, $5, $6, $7, $8 ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp
new file mode 100644
index 0000000..2fa7cde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": ?,
+    "p_bool": ?,
+    "p_int": ?,
+    "p_dec": ?,
+    "p_dbl": ?,
+    "p_str": ?,
+    "p_arr": ?,
+    "p_obj": ?
+  }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp
new file mode 100644
index 0000000..1d952d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t2": {
+    "p_null_type": ? is null,
+    "p_bool_type": is_boolean(?),
+    "p_int_type": is_number(?),
+    "p_dec_type": is_number(?),
+    "p_dbl_type": is_number(?),
+    "p_str_type": is_string(?),
+    "p_arr_type": is_array(?),
+    "p_obj_type": is_object(?)
+  }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp
new file mode 100644
index 0000000..13c077b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t3": [ ?, ?, ?, ?, ?, ?, ?, ? ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp
new file mode 100644
index 0000000..1eb6e0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": ?,
+    "p_bool": ?,
+    "p_int": ?,
+    "p_dec": ?,
+    "p_dbl": ?,
+    "p_str": ?,
+    "p_arr": ?,
+    "p_obj": ?
+  }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp
new file mode 100644
index 0000000..bbacada
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t2": {
+    "p_null_type": ? is null,
+    "p_bool_type": is_boolean(?),
+    "p_int_type": is_number(?),
+    "p_dec_type": is_number(?),
+    "p_dbl_type": is_number(?),
+    "p_str_type": is_string(?),
+    "p_arr_type": is_array(?),
+    "p_obj_type": is_object(?)
+  }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp
new file mode 100644
index 0000000..312278f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t3": [ ?, ?, ?, ?, ?, ?, ?, ? ]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp
new file mode 100644
index 0000000..279f336
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a positional parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=["hello"]
+
+$1 || $2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp
new file mode 100644
index 0000000..5f7dedc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a positional parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=["a", "b"]
+
+? || ? || ?
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp
new file mode 100644
index 0000000..89901e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test autogenerated column aliases positional statement parameters
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[3,4]
+
+select $2, $1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.query.sqlpp
new file mode 100644
index 0000000..e340140
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test when statement parameter is used after GROUP BY
+ * Expected Res : Success
+ * Date         : Jul 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[2]
+
+from [
+ { "x": false, "y": 1 },
+ { "x": false, "y": 2 },
+ { "x": true, "y": 3 },
+ { "x": true, "y": 4 },
+ { "x": true, "y": 5 }
+] t
+group by t.x
+having count(t.y) > $1
+select value t.x
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/regexp_replace/regexp_replace.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/regexp_replace/regexp_replace.6.query.sqlpp
new file mode 100644
index 0000000..94deadc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/regexp_replace/regexp_replace.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+{
+  'result1':regexp_replace('abracadabra','bra','kkk',-1),
+  'result2':regexp_replace('abracadabra','bra','kkk',0),
+  'result3':regexp_replace('abracadabra','bra','kkk',1),
+  'result4':regexp_replace('abracadabra_abracadabra','bra','kkk',2),
+  'result5':regexp_replace('abracadabra_abracadabra','bra','kkk',5)
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/replace_with_limit/replace_with_limit.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/replace_with_limit/replace_with_limit.1.query.sqlpp
index 3c33e49..4eb0963 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/replace_with_limit/replace_with_limit.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/replace_with_limit/replace_with_limit.1.query.sqlpp
@@ -30,6 +30,6 @@
   [ 8, replace("_ab_ba_ab_ab_ab_", "ab", "xyz", bigint("4")) ],
   [ 9, replace("abracadabra_abracadabra_xyz","bra","*BRA*", 3) ],
   [ 10, replace("ab_ba_ab", "ab", "ba", 0) ],
-  [ 11, replace("ab_ba_ab", "ab", "ba", -100) ]
+  [ 11, replace("ab_ba_ab_ba_ab", "ab", "ba", -100) ]
 ] t
 order by t[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp
new file mode 100644
index 0000000..e127372
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+ {
+   "t1": [ reverse(missing) is missing, reverse(null) is null ],
+   "t2": reverse(""),
+   "t3": reverse("abcd"),
+   "t4": string_to_codepoint(reverse("a\u00D7\u2103\u00F7\u2109b")),
+   "t5": ( from ["ab", "abc", "abcd"] t select value reverse(t) order by t )
+ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.1.query.sqlpp
new file mode 100644
index 0000000..3e48631
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+{
+  't1': string_concat([null,'aa']),
+  't2': string_concat(['aa',null]),
+  't3': string_concat([null,missing,'aa']),
+  't4': string_concat(['aa',null,missing])
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.2.query.sqlpp
new file mode 100644
index 0000000..12208a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-concat2/string-concat2.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+{
+  't1': null || 'aa',
+  't2': 'aa' || null,
+  't3': null || missing || 'aa',
+  't4': 'aa' || null || missing
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
index b9c79d0..d39e51e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
@@ -23,4 +23,4 @@
  * Date           : 18th April 2012
  */
 
-{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2 - 1))};
+{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-6),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') div 2 - 1))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
index 860173e..d9f936f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
@@ -24,5 +24,18 @@
  */
 
 select element a
-from  [substring('hello world',6,5),substring('hello world',0,11),substring('hello world',2,9),substring('ABCD',2,2),substring('ABCD',0,4),substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',0,`string-length`('UC Irvine')),substring(substring('UC Irvine',3),0,`string-length`('Irvine'))] as a
+from [
+    substring('hello world',6,5),
+    substring('hello world',0,11),
+    substring('hello world',2,9),
+    substring('ABCD',2,2),
+    substring('ABCD',0,4),
+    substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),
+    substring('UC Irvine',0,`string-length`('UC Irvine')),
+    substring(substring('UC Irvine',3),0,`string-length`('Irvine')),
+    substring('ABCD',-3,2),
+    substring('ABCD',-10,1),
+    substring('ABCD',1,-1)
+  ] as a
+
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
index 8ea572c..6b63289 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
@@ -19,5 +19,7 @@
 
 use test;
 
-
-{'result1':test.substring('HEllow',-1)};
+{
+  'result1':substring('HEllow',-3),
+  'result2':substring('HEllow',-7)
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp
new file mode 100644
index 0000000..be2bcbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp
new file mode 100644
index 0000000..c88c5bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+load  dataset Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp
new file mode 100644
index 0000000..a63b792
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp
new file mode 100644
index 0000000..5a77f27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp
new file mode 100644
index 0000000..a432d03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp
new file mode 100644
index 0000000..b22ce59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+use tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp
new file mode 100644
index 0000000..774debe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp
new file mode 100644
index 0000000..16e36ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+use tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.9.query.sqlpp
new file mode 100644
index 0000000..4b27077
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.9.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where not(c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"])
+order by c.c_custkey
+limit 2
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
index a4b144f..a2fd910 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
@@ -19,6 +19,8 @@
 
 use sampdb;
 
+set `rewrite_in_as_or` "false";
+
 select *
 from
 (
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp
new file mode 100644
index 0000000..5935946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+use sampdb;
+
+/* default: set `rewrite_in_as_or` "true" */
+
+select *
+from
+(
+  select id from samptable
+  where (id in [0] and id in [1])
+        or (id in [1] and id in [2])
+) st1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
index 45f924a..cf6b506 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/relational_division2/relational_division2.3.query.sqlpp
@@ -22,7 +22,7 @@
 /** Finds customers whose orders have all possible priorities.*/
 
 WITH priorities AS
-COLL_COUNT((
+STRICT_COUNT((
   SELECT DISTINCT o.o_orderpriority FROM Orders o
 ))
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
index b67806d..f7f3661 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_max/agg_max.3.query.sqlpp
@@ -20,19 +20,19 @@
 use test;
 
 
-{'m0':test.coll_max((
+{'m0':test.strict_max((
     select element i.time
     from  tsdata as i
-)),'m1':test.coll_max((
+)),'m1':test.strict_max((
     select element i.date
     from  tsdata as i
-)),'m2':test.coll_max((
+)),'m2':test.strict_max((
     select element i.datetime
     from  tsdata as i
-)),'m3':test.coll_max((
+)),'m3':test.strict_max((
     select element i.dtduration
     from  tsdata as i
-)),'m4':test.coll_max((
+)),'m4':test.strict_max((
     select element i.ymduration
     from  tsdata as i
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
index cfc6dd7..28ca219 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/agg_min/agg_min.3.query.sqlpp
@@ -20,19 +20,19 @@
 use test;
 
 
-{'m0':test.coll_min((
+{'m0':test.strict_min((
     select element i.time
     from  tsdata as i
-)),'m1':test.coll_min((
+)),'m1':test.strict_min((
     select element i.date
     from  tsdata as i
-)),'m2':test.coll_min((
+)),'m2':test.strict_min((
     select element i.datetime
     from  tsdata as i
-)),'m3':test.coll_min((
+)),'m3':test.strict_min((
     select element i.dtduration
     from  tsdata as i
-)),'m4':test.coll_min((
+)),'m4':test.strict_min((
     select element i.ymduration
     from  tsdata as i
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
index 0f9b329..4cab752 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_1/overlap_bins_gby_1.3.query.sqlpp
@@ -25,8 +25,8 @@
 
 
 select element {'timebin':bin,
-    'count':coll_count((select value 1 from g)),
-    'total_ms':coll_sum((
+    'count':strict_count((select value 1 from g)),
+    'total_ms':strict_sum((
         select element
         `ms_from_day_time_duration`(`duration-from-interval`(`get-overlapping-interval`(bin, g.i2.interval)))
         from g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
index 9a29efb..7fc4ccd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_3/overlap_bins_gby_3.3.query.sqlpp
@@ -27,10 +27,10 @@
         group by i.app as subgid
         order by subgid,multitask.count(i)
     )}
-from  multitask.`overlap-bins`(multitask.interval(multitask.coll_min((
+from  multitask.`overlap-bins`(multitask.interval(multitask.strict_min((
     select element i.time
     from  logs as i
-)),multitask.coll_max((
+)),multitask.strict_max((
     select element (i.time + multitask.`duration-from-ms`((i.duration * 1000)))
     from  logs as i
 ))),multitask.time('00:00:00.000'),multitask.`day_time_duration`('PT1M')) as bin
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
index 85a66f2..5952910 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1331.25.query.sqlpp
@@ -21,7 +21,7 @@
 
 USE TinySocial;
 
-SELECT ELEMENT coll_avg((
+SELECT ELEMENT strict_avg((
 select element LENGTH(message.message)
 FROM FacebookMessages AS message
 WHERE message.`in-response-to` >= 1 and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
index 7ca10bd..de80b98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.14.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element fbu
     from  FacebookUsers as fbu
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
index ac378d0..468c774 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.22.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element t
     from  TweetMessages as t
     where (t.tweetid = '13')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
index 7ca10bd..de80b98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.14.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element fbu
     from  FacebookUsers as fbu
 ));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
index ac378d0..468c774 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.22.query.sqlpp
@@ -23,7 +23,7 @@
 use TinySocial;
 
 
-select element coll_count((
+select element strict_count((
     select element t
     from  TweetMessages as t
     where (t.tweetid = '13')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
index fa84678..3ec28e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.25.query.sqlpp
@@ -19,7 +19,7 @@
 
 USE TinySocial;
 
-SELECT ELEMENT coll_avg((
+SELECT ELEMENT strict_avg((
 select element LENGTH(message.message)
 FROM FacebookMessages AS message
 WHERE message.`in-response-to` >= 1 and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
index 455eab7..b7fb014 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.3.query.sqlpp
@@ -20,7 +20,7 @@
 
 USE tpcds;
 
-SELECT VALUE {"h8_30_to_9" : COLL_COUNT((select 1
+SELECT VALUE {"h8_30_to_9" : STRICT_COUNT((select 1
      FROM store_sales ss1, household_demographics , time_dim, store s1
      WHERE ss1.ss_sold_time_sk = time_dim.t_time_sk
      AND ss1.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -30,7 +30,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s1.s_store_name = 'ese')), "h9_to_9_30": COLL_COUNT((select 1
+     AND s1.s_store_name = 'ese')), "h9_to_9_30": STRICT_COUNT((select 1
      FROM store_sales ss2, household_demographics , time_dim, store s2
      WHERE ss2.ss_sold_time_sk = time_dim.t_time_sk
      AND ss2.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -40,7 +40,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s2.s_store_name = 'ese')), "h9_30_to_10": COLL_COUNT((select 1
+     AND s2.s_store_name = 'ese')), "h9_30_to_10": STRICT_COUNT((select 1
      FROM store_sales ss3, household_demographics , time_dim, store s3
      WHERE ss3.ss_sold_time_sk = time_dim.t_time_sk
      AND ss3.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -50,7 +50,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s3.s_store_name = 'ese')), "h10_to_10_30": COLL_COUNT((select 1
+     AND s3.s_store_name = 'ese')), "h10_to_10_30": STRICT_COUNT((select 1
      FROM store_sales ss4, household_demographics , time_dim, store s4
      WHERE ss4.ss_sold_time_sk = time_dim.t_time_sk
      AND ss4.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -60,7 +60,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s4.s_store_name = 'ese')), "h10_30_to_11": COLL_COUNT( (select 1
+     AND s4.s_store_name = 'ese')), "h10_30_to_11": STRICT_COUNT( (select 1
      FROM store_sales ss5, household_demographics , time_dim, store s5
      WHERE ss5.ss_sold_time_sk = time_dim.t_time_sk
      AND ss5.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -70,7 +70,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s5.s_store_name = 'ese')), "h11_to_11_30": COLL_COUNT((select 1
+     AND s5.s_store_name = 'ese')), "h11_to_11_30": STRICT_COUNT((select 1
      FROM store_sales ss6, household_demographics , time_dim, store s6
      WHERE ss6.ss_sold_time_sk = time_dim.t_time_sk
      AND ss6.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -80,7 +80,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s6.s_store_name = 'ese')), "h11_30_to_12": COLL_COUNT((select 1
+     AND s6.s_store_name = 'ese')), "h11_30_to_12": STRICT_COUNT((select 1
      FROM store_sales ss7, household_demographics , time_dim, store s7
      WHERE ss7.ss_sold_time_sk = time_dim.t_time_sk
      AND ss7.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -90,7 +90,7 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s7.s_store_name = 'ese')), "h12_to_12_30":  COLL_COUNT((select 1
+     AND s7.s_store_name = 'ese')), "h12_to_12_30":  STRICT_COUNT((select 1
      FROM store_sales ss8, household_demographics , time_dim, store s8
      WHERE ss8.ss_sold_time_sk = time_dim.t_time_sk
      AND ss8.ss_hdemo_sk = household_demographics.hd_demo_sk
@@ -100,4 +100,4 @@
      AND ((household_demographics.hd_dep_count = 3 AND household_demographics.hd_vehicle_count<=3+2) or
           (household_demographics.hd_dep_count = 0 AND household_demographics.hd_vehicle_count<=0+2) or
           (household_demographics.hd_dep_count = 1 AND household_demographics.hd_vehicle_count<=1+2))
-     AND s8.s_store_name = 'ese'))};
\ No newline at end of file
+     AND s8.s_store_name = 'ese'))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
index c13f668..c1943e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1580/query-ASTERIXDB-1580.3.query.sqlpp
@@ -30,6 +30,6 @@
     and s.ss_sold_date_sk = d.d_date_sk
     and s.ss_item_sk = i.i_item_sk
 group by a.ca_state group as stategrp
-having coll_count((select value a from stategrp)) >= 10
+having strict_count((select value a from stategrp)) >= 10
 order by cnt
 limit 100;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
index 3a1b5c2..fadfa8d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1602/query-ASTERIXDB-1602.3.query.sqlpp
@@ -36,7 +36,7 @@
  group by item.i_item_id
 )
 
-select coll_count((
+select strict_count((
  select *
  from tab t1, tab t2
-));
\ No newline at end of file
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index 3aea572..766e704 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(tpch.coll_sum((
+select element {'year':year,'mkt_share':(tpch.strict_sum((
           select element case when i.t.s_name = 'BRAZIL' then i.t.revenue else 0.0 end
           from  g as i
-      )) / tpch.coll_sum((
+      )) / tpch.strict_sum((
           select element i.t.revenue
           from  g as i
       )))}
@@ -67,4 +67,4 @@
 group by t.year as year
 group as g
 order by year
-;
\ No newline at end of file
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index 6dfd5af..a9116b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':COLL_SUM((
+select element {'nation':nation,'o_year':o_year,'sum_profit':STRICT_SUM((
         select element pr.profit.amount
         from  g as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
index 6038d15..5066186 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value locn) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index 3a6ac4a..8730e73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value locn) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
index a98ec6e..bbf5464 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q12_shipping/q12_shipping.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element {'l_shipmode':l_shipmode,'high_line_count':tpch.coll_sum((
+select element {'l_shipmode':l_shipmode,'high_line_count':tpch.strict_sum((
         select element case i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH'
                        when true then 1
                        when false then 0
                        end
         from  g as i
-    )),'low_line_count':tpch.coll_sum((
+    )),'low_line_count':tpch.strict_sum((
         select element case i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH'
                        when true then 0
                        when false then 1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index a15ab14..860bce0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
@@ -41,6 +41,6 @@
 ) as gco
 group by gco.c_count as c_count
 group as g
-let  custdist = coll_count(( select value gco from g ))
+let  custdist = strict_count(( select value gco from g ))
 order by custdist desc,c_count desc
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 2ec6a9c..651c398 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element (100.0 * COLL_SUM((
+select element (100.0 * STRICT_SUM((
       select element case i.p_type like 'PROMO%' when true then i.l_extendedprice * (1 - i.l_discount) else 0.0 end
       from (from g select value lp) as i
-  )) / COLL_SUM((
+  )) / STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value lp) as i
   )))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
index 46a278c..2bf53b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue638/query-issue638.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element g.profit.amount
         from  g
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index f8211c9..9052e05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey, sum(o_orderkey_count) AS c_count
         FROM  (
                 SELECT c_custkey,
-                       coll_count(
+                       strict_count(
                            (
                             select element o_orderkey
                             from  Orders
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
index c6afe18..352b7d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
@@ -37,7 +37,7 @@
 
 SELECT cntrycode, count(ct) AS numcust, sum(c_acctbal) AS totacctbal
 FROM  q22_customer_tmp as ct
-WHERE coll_count((
+WHERE strict_count((
                 SELECT ELEMENT Orders
                 FROM  Orders
                 WHERE ct.c_custkey = o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
index fc9c64d..b1f35f3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -28,7 +28,7 @@
 
 SELECT l_returnflag,
        l_linestatus,
-       coll_count(cheaps) AS count_cheaps,
+       strict_count(cheaps) AS count_cheaps,
        total_charges AS total_charges
 FROM  LineItem as l
 WHERE l_shipdate <= '1998-09-02'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
index 5186119..b8acf3b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT  l_returnflag,
         l_linestatus,
-        coll_count(cheaps) AS count_cheaps,
-        coll_avg(expensives) AS avg_expensive_discounts,
+        strict_count(cheaps) AS count_cheaps,
+        strict_avg(expensives) AS avg_expensive_discounts,
         sum_disc_prices AS sum_disc_prices,
         total_charges AS total_charges
 FROM  LineItem AS l
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
index cb1b00f8..1b959d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue810/query-issue810.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT l_returnflag,
        l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 WHERE l_shipdate <= '1998-09-02'
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index fb3e8e7..597b054 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey AS c_custkey, sum(co.o_orderkey_count) AS c_count
         FROM  (
                 SELECT c.c_custkey AS c_custkey,
-                       coll_count(
+                       strict_count(
                            (
                             select element o.o_orderkey
                             from  Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
index 531e17f..f4a683b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
@@ -38,7 +38,7 @@
 
 SELECT cntrycode AS cntrycode, count(ct) AS numcust, sum(ct.c_acctbal) AS totacctbal
 FROM  q22_customer_tmp() as ct
-WHERE coll_count((
+WHERE strict_count((
                 SELECT ELEMENT o
                 FROM  Orders AS o
                 WHERE ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
index bfd62c9..f193452 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -28,7 +28,7 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheaps) AS count_cheaps,
+       strict_count(cheaps) AS count_cheaps,
        total_charges AS total_charges
 FROM  LineItem as l
 WHERE l.l_shipdate <= '1998-09-02'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
index 3ee3a31..7beda71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT  l_returnflag AS l_returnflag,
         l_linestatus AS l_linestatus,
-        coll_count(cheaps) AS count_cheaps,
-        coll_avg(expensives) AS avg_expensive_discounts,
+        strict_count(cheaps) AS count_cheaps,
+        strict_avg(expensives) AS avg_expensive_discounts,
         sum_disc_prices AS sum_disc_prices,
         total_charges AS total_charges
 FROM  LineItem AS l
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
index 17e37b3..2f49c59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue810/query-issue810.3.query.sqlpp
@@ -28,8 +28,8 @@
 
 SELECT l_returnflag AS l_returnflag,
        l_linestatus AS l_linestatus,
-       coll_count(cheap) AS count_cheaps,
-       coll_count(expensive) AS count_expensives
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
 FROM LineItem AS l
 WHERE l.l_shipdate <= '1998-09-02'
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
index 389a35d..591d336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate/nest_aggregate.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
index e137e9f9..8633039 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/nest_aggregate2/nest_aggregate2.3.query.sqlpp
@@ -33,7 +33,7 @@
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = tpch.coll_sum((
+        let  sum = tpch.strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
index b960393..09ec48d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-{'revenue':COLL_SUM((
+{'revenue':STRICT_SUM((
     select element (l.l_extendedprice * l.l_discount)
     from  LineItem as l
     where ((l.l_shipdate >= '1994-01-01') and (l.l_shipdate < '1995-01-01') and (l.l_discount >= 0.05) and (l.l_discount <= 0.07) and (l.l_quantity < 24))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index ce12ffd..61c0b7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(COLL_SUM((
+select element {'year':year,'mkt_share':(STRICT_SUM((
           select element case i.s_name = 'BRAZIL' when true then i.revenue when false then 0.0 end
           from  (from g select value t) as i
-      )) / COLL_SUM((
+      )) / STRICT_SUM((
           select element i.revenue
           from  (from g select value t) as i
       )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index f469707..a31e791 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element pr.amount
         from (from g select value profit) as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
index b5390d4..d56543e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q12_shipping/q12_shipping.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element {'l_shipmode':l_shipmode,'high_line_count':tpch.coll_sum((
+select element {'l_shipmode':l_shipmode,'high_line_count':tpch.strict_sum((
         select element case
                         when i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH' then 1
                         else 0
                         end
         from  g as i
-    )),'low_line_count':tpch.coll_sum((
+    )),'low_line_count':tpch.strict_sum((
         select element case
                          when i.o.o_orderpriority = '1-URGENT' or i.o.o_orderpriority = '2-HIGH' then 0
                          else 1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index 0c0bdaa..9df4e16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 864793d..b715c25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -19,13 +19,13 @@
 
 use tpch;
 
-select element (100.0 * tpch.coll_sum((
+select element (100.0 * tpch.strict_sum((
       select element case
                        when i.lp.p_type like 'PROMO%' then i.lp.l_extendedprice * (1 - i.lp.l_discount)
                        else 0.0
                      end
       from  g as i
-  )) / tpch.coll_sum((
+  )) / tpch.strict_sum((
       select element (i.lp.l_extendedprice * (1 - i.lp.l_discount))
       from  g as i
   )))
@@ -34,4 +34,4 @@
 let lp = {'p_type': p.p_type, 'l_extendedprice': l.l_extendedprice, 'l_discount': l.l_discount}
 where ((l.l_partkey = p.p_partkey) and (l.l_shipdate >= '1995-09-01') and (l.l_shipdate < '1995-10-01'))
 group by 1 as t group as g(lp as lp)
-;
\ No newline at end of file
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
index 4cd5d54..d542ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
@@ -34,18 +34,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.strict_sum((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (coll_count((
+where (strict_count((
     select element o
     from  Orders as o
     where ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
index 38ba16f..6888d5f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
index 555437a..33618d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
index 6877db8..3ef97a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue810/query-issue810.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheap),'count_expensives':coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheap),'count_expensives':strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
index aefe8a3..7fac9f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827-2/query-issue827-2.3.query.sqlpp
@@ -26,30 +26,30 @@
 use tpch;
 
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':coll_count((
+)),'count_order':strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
index c488ab1..c8dd6f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue827/query-issue827.3.query.sqlpp
@@ -26,10 +26,10 @@
 use tpch;
 
 
-{'count_cheaps': coll_count((
+{'count_cheaps': strict_count((
     select element l.l_quantity
     from  LineItem as l
-)),'count_expensives': coll_sum((
+)),'count_expensives': strict_sum((
     select element e
     from  (
         select element l.l_extendedprice
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
index 389a35d..591d336 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate/nest_aggregate.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
index e137e9f9..8633039 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/nest_aggregate2/nest_aggregate2.3.query.sqlpp
@@ -33,7 +33,7 @@
         where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = nation.n_nationkey))
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = tpch.coll_sum((
+        let  sum = tpch.strict_sum((
               select element g.orders.o_totalprice
               from  g
           ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
index f47341a..4ea0ad6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q01_pricing_summary_report_nt/q01_pricing_summary_report_nt.3.query.sqlpp
@@ -21,28 +21,28 @@
 
 set hash_merge "true";
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':COLL_SUM((
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':STRICT_SUM((
         select element i.l.l_quantity
         from  g as i
-    )),'sum_base_price':COLL_SUM((
+    )),'sum_base_price':STRICT_SUM((
         select element i.l.l_extendedprice
         from  g as i
-    )),'sum_disc_price':COLL_SUM((
+    )),'sum_disc_price':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount))
         from  g as i
-    )),'sum_charge':COLL_SUM((
+    )),'sum_charge':STRICT_SUM((
         select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
         from  g as i
-    )),'ave_qty':COLL_AVG((
+    )),'ave_qty':STRICT_AVG((
         select element i.l.l_quantity
         from  g as i
-    )),'ave_price':COLL_AVG((
+    )),'ave_price':STRICT_AVG((
         select element i.l.l_extendedprice
         from g as i
-    )),'ave_disc':COLL_AVG((
+    )),'ave_disc':STRICT_AVG((
         select element i.l.l_discount
         from  g as i
-    )),'count_order':COLL_COUNT(( from g select value l ))}
+    )),'count_order':STRICT_COUNT(( from g select value l ))}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
index 24180d2..5ad4114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q02_minimum_cost_supplier/q02_minimum_cost_supplier.3.query.sqlpp
@@ -45,7 +45,7 @@
 };
 declare function tmp2() {
 (
-    select element {'p_partkey':p_partkey,'ps_min_supplycost':COLL_MIN((
+    select element {'p_partkey':p_partkey,'ps_min_supplycost':STRICT_MIN((
             select element i.pssrn.ps_supplycost
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
index f076d71..6911616 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q03_shipping_priority_nt/q03_shipping_priority_nt.3.query.sqlpp
@@ -28,7 +28,7 @@
 /* +hash */
 group by l.l_orderkey as l_orderkey,o.o_orderdate as o_orderdate,o.o_shippriority as o_shippriority
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value l) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
index b82242e..df73d7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q05_local_supplier_volume/q05_local_supplier_volume.3.query.sqlpp
@@ -47,7 +47,7 @@
 /* +hash */
 group by o1.n_name as n_name
 group as g
-let revenue = COLL_SUM((
+let revenue = STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from  (from g select value o1) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
index b960393..09ec48d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q06_forecast_revenue_change/q06_forecast_revenue_change.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-{'revenue':COLL_SUM((
+{'revenue':STRICT_SUM((
     select element (l.l_extendedprice * l.l_discount)
     from  LineItem as l
     where ((l.l_shipdate >= '1994-01-01') and (l.l_shipdate < '1995-01-01') and (l.l_discount >= 0.05) and (l.l_discount <= 0.07) and (l.l_quantity < 24))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
index b6344b2..fa07fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q07_volume_shipping/q07_volume_shipping.3.query.sqlpp
@@ -50,7 +50,7 @@
 where ((locs.c_nationkey = t.c_nationkey) and (locs.s_nationkey = t.s_nationkey))
 group by t.supp_nation as supp_nation,t.cust_nation as cust_nation,l_year0 as l_year
 group as g
-let  revenue = COLL_SUM((
+let  revenue = STRICT_SUM((
       select element (i.locs.l_extendedprice * (1 - i.locs.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
index ce12ffd..61c0b7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q08_national_market_share/q08_national_market_share.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element {'year':year,'mkt_share':(COLL_SUM((
+select element {'year':year,'mkt_share':(STRICT_SUM((
           select element case i.s_name = 'BRAZIL' when true then i.revenue when false then 0.0 end
           from  (from g select value t) as i
-      )) / COLL_SUM((
+      )) / STRICT_SUM((
           select element i.revenue
           from  (from g select value t) as i
       )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
index f469707..a31e791 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q09_product_type_profit_nt/q09_product_type_profit_nt.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element pr.amount
         from (from g select value profit) as pr
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item/q10_returned_item.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
index e48e264..39b50c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q10_returned_item_int64/q10_returned_item_int64.3.query.sqlpp
@@ -35,7 +35,7 @@
 ) as locn
 group by locn.c_custkey as c_custkey,locn.c_name as c_name,locn.c_acctbal as c_acctbal,locn.c_phone as c_phone,locn.n_name as n_name,locn.c_address as c_address,locn.c_comment as c_comment
 group as g
-let revenue = coll_sum((
+let revenue = strict_sum((
       select element (i.locn.l_extendedprice * (1 - i.locn.l_discount))
       from  g as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
index 2fb8110..53834ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q11_important_stock/q11_important_stock.3.query.sqlpp
@@ -20,7 +20,7 @@
 use tpch;
 
 
-with  sum as COLL_SUM((
+with  sum as STRICT_SUM((
       select element (ps.ps_supplycost * ps.ps_availqty)
       from  Partsupp as ps,
             (
@@ -33,7 +33,7 @@
   ))
 select element {'partkey':t1.ps_partkey,'part_value':t1.part_value}
 from  (
-    select element {'ps_partkey':ps_partkey,'part_value':COLL_SUM((
+    select element {'ps_partkey':ps_partkey,'part_value':STRICT_SUM((
             select element (i.ps.ps_supplycost * i.ps.ps_availqty)
             from  g as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
index fa54051..f88a2ba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q12_shipping/q12_shipping.3.query.sqlpp
@@ -20,11 +20,11 @@
 use tpch;
 
 
-select element {'l_shipmode':l_shipmode,'high_line_count':COLL_SUM((
+select element {'l_shipmode':l_shipmode,'high_line_count':STRICT_SUM((
         select element
             CASE i.o_orderpriority = '1-URGENT' OR i.o_orderpriority = '2-HIGH' WHEN true THEN 1 WHEN false THEN 0 END
         from  (select value o from g) as i
-    )),'low_line_count':COLL_SUM((
+    )),'low_line_count':STRICT_SUM((
         select element case when i.o_orderpriority = '1-URGENT' OR i.o_orderpriority = '2-HIGH' then 0 else 1 end
         from (select value o from g) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
index 0c0bdaa..9df4e16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q13_customer_distribution/q13_customer_distribution.3.query.sqlpp
@@ -24,12 +24,12 @@
 
 select element {'c_count':c_count,'custdist':custdist}
 from  (
-    select element {'c_custkey':c_custkey,'c_count':COLL_SUM((
+    select element {'c_custkey':c_custkey,'c_count':STRICT_SUM((
             select element i.o_orderkey_count
             from (select value co from g2) as i
         ))}
     from  (
-        select element {'c_custkey':c.c_custkey,'o_orderkey_count':coll_count((
+        select element {'c_custkey':c.c_custkey,'o_orderkey_count':strict_count((
                 select element o.o_orderkey
                 from  Orders as o
                 where c.c_custkey = o.o_custkey and o.o_comment not like '%special%requests%'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
index 2ec6a9c..651c398 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q14_promotion_effect/q14_promotion_effect.3.query.sqlpp
@@ -20,10 +20,10 @@
 use tpch;
 
 
-select element (100.0 * COLL_SUM((
+select element (100.0 * STRICT_SUM((
       select element case i.p_type like 'PROMO%' when true then i.l_extendedprice * (1 - i.l_discount) else 0.0 end
       from (from g select value lp) as i
-  )) / COLL_SUM((
+  )) / STRICT_SUM((
       select element (i.l_extendedprice * (1 - i.l_discount))
       from (from g select value lp) as i
   )))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
index acd8537..61b2b33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q15_top_supplier/q15_top_supplier.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function revenue() {
 (
-    select element {'supplier_no':l_suppkey,'total_revenue':COLL_SUM((
+    select element {'supplier_no':l_suppkey,'total_revenue':STRICT_SUM((
             select element (i.l.l_extendedprice * (1 - i.l.l_discount))
             from g as i
         ))}
@@ -32,7 +32,7 @@
     group as g
 )
 };
-with  m as COLL_MAX((
+with  m as STRICT_MAX((
       select element r2.total_revenue
       from  revenue() as r2
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
index 47dc549..b7aeb27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q16_parts_supplier_relationship/q16_parts_supplier_relationship.3.query.sqlpp
@@ -42,7 +42,7 @@
 ) as t2
 group by t2.p_brand as p_brand,t2.p_type as p_type,t2.p_size as p_size
 group as g
-let  supplier_cnt = COLL_COUNT((
+let  supplier_cnt = STRICT_COUNT((
       select element i.ps_suppkey
       from  (from g select value t2) as i
   ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
index f22d29e..497601f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_large_gby_variant/q17_large_gby_variant.3.query.sqlpp
@@ -20,35 +20,35 @@
 use tpch;
 
 
-select element {'t_partkey':l_partkey,'t_count':COLL_COUNT((from g select value l)),
-    't_avg_quantity':(0.2 * COLL_AVG((
+select element {'t_partkey':l_partkey,'t_count':STRICT_COUNT((from g select value l)),
+    't_avg_quantity':(0.2 * STRICT_AVG((
           select element i.l_quantity
           from  (from g select value l) as i
-    ))),'t_max_suppkey':COLL_MAX((
+    ))),'t_max_suppkey':STRICT_MAX((
         select element i.l_suppkey
         from  (from g select value l) as i
-    )),'t_max_linenumber':COLL_MAX((
+    )),'t_max_linenumber':STRICT_MAX((
         select element i.l_linenumber
         from  (from g select value l) as i
-    )),'t_avg_extendedprice':COLL_AVG((
+    )),'t_avg_extendedprice':STRICT_AVG((
         select element i.l_extendedprice
         from  (from g select value l) as i
-    )),'t_avg_discount':COLL_AVG((
+    )),'t_avg_discount':STRICT_AVG((
         select element i.l_discount
         from  (from g select value l) as i
-    )),'t_avg_tax':COLL_AVG((
+    )),'t_avg_tax':STRICT_AVG((
         select element i.l_tax
         from  (from g select value l) as i
-    )),'t_max_shipdate':COLL_MAX((
+    )),'t_max_shipdate':STRICT_MAX((
         select element i.l_shipdate
         from  (from g select value l) as i
-    )),'t_min_commitdate':COLL_MIN((
+    )),'t_min_commitdate':STRICT_MIN((
         select element i.l_commitdate
         from  (from g select value l) as i
-    )),'t_min_receiptdate':COLL_MIN((
+    )),'t_min_receiptdate':STRICT_MIN((
         select element i.l_receiptdate
         from  (from g select value l) as i
-    )),'t_max_comment':COLL_MAX((
+    )),'t_max_comment':STRICT_MAX((
         select element i.l_comment
         from  (from g select value l) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
index 010d03a..509aae3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q17_small_quantity_order_revenue/q17_small_quantity_order_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function tmp() {
 (
-    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.coll_avg((
+    select element {'t_partkey':l_partkey,'t_avg_quantity':(0.2 * tpch.strict_avg((
               select element i.l_quantity
               from  (from g select value l) as i
           )))}
@@ -32,7 +32,7 @@
 )
 };
 
-select element (coll_sum((
+select element (strict_sum((
       select element l.l_extendedprice
       from  tmp() as t,
             LineItem as l,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
index 1a11819..3f60c4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q18_large_volume_customer/q18_large_volume_customer.3.query.sqlpp
@@ -20,14 +20,14 @@
 use tpch;
 
 
-select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.coll_sum((
+select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':tpch.strict_sum((
         select element j.l_quantity
         from  (from g select value l) as j
     ))}
 from  Customer as c,
       Orders as o,
       (
-    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.coll_sum((
+    select element {'l_orderkey':l_orderkey,'t_sum_quantity':tpch.strict_sum((
             select element i.l_quantity
             from  (select value l from g2) as i
         ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
index df4e276..e53edb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 set `import-private-functions` `true`;
 
-select element COLL_SUM((
+select element STRICT_SUM((
     select element (l.l_extendedprice * (1 - l.l_discount))
     from  LineItem as l,
           Part as p
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
index 2355162..dd8835f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q20_potential_part_promotion/q20_potential_part_promotion.3.query.sqlpp
@@ -24,7 +24,7 @@
 from  (
     select distinct element {'ps_suppkey':pst1.ps_suppkey}
     from  (
-        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * COLL_SUM((
+        select element {'l_partkey':l_partkey,'l_suppkey':l_suppkey,'sum_quantity':(0.5 * STRICT_SUM((
                   select element i.l_quantity
                   from  (from g select value l) as i
               )))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
index 65293dc..d5e6dbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q21_suppliers_who_kept_orders_waiting/q21_suppliers_who_kept_orders_waiting.3.query.sqlpp
@@ -22,10 +22,10 @@
 
 declare function tmp1() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g1 select value g1.l2) as i
         ))}
@@ -40,10 +40,10 @@
 };
 declare function tmp2() {
 (
-    select element {'l_orderkey':l_orderkey,'count_suppkey':coll_count((
+    select element {'l_orderkey':l_orderkey,'count_suppkey':strict_count((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
-        )),'max_suppkey': coll_max((
+        )),'max_suppkey': strict_max((
             select element i.l_suppkey
             from  (from g2 select value g2.l2) as i
         ))}
@@ -78,6 +78,6 @@
 ) as t4
 group by t4.s_name as s_name
 group as g
-let  numwait = coll_count(( from g select value t4 ))
+let  numwait = strict_count(( from g select value t4 ))
 order by numwait desc,s_name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index d33a095..3081036 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -26,12 +26,12 @@
     from  Customer as c
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       where (c.c_acctbal > 0.0)
   ))
-select element {'cntrycode':cntrycode,'numcust':COLL_COUNT(( from g select value ct )),'totacctbal':COLL_SUM((
+select element {'cntrycode':cntrycode,'numcust':STRICT_COUNT(( from g select value ct )),'totacctbal':STRICT_SUM((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
index 4cd5d54..d542ff7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
@@ -34,18 +34,18 @@
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
-with  avg as tpch.coll_avg((
+with  avg as tpch.strict_avg((
       select element c.c_acctbal
       from  Customer as c
       let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
-select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
+select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.strict_sum((
         select element i.c_acctbal
         from  (from g select value ct) as i
     ))}
 from  tpch.q22_customer_tmp() as ct
-where (coll_count((
+where (strict_count((
     select element o
     from  Orders as o
     where ct.c_custkey = o.o_custkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
index 4689676..b473627 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_linenumber':l.l_linenumber,'count_order': coll_count((from g select value g))}
+select element {'l_linenumber':l.l_linenumber,'count_order': strict_count((from g select value g))}
 from  LineItem as l
 group by l.l_linenumber
 group as g
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
index 46a278c..2bf53b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue638/query-issue638.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.coll_sum((
+select element {'nation':nation,'o_year':o_year,'sum_profit':tpch.strict_sum((
         select element g.profit.amount
         from  g
     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
index 7a40cec..d042440 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -40,7 +40,7 @@
       where orders.o_custkey = customer.c_custkey and customer.c_nationkey = n.n_nationkey
       group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
       group as g2
-      let  sum = coll_sum((
+      let  sum = strict_sum((
             select element g2.orders.o_totalprice
             from g2
       ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
index 6c7d7bd..073f7e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
@@ -31,7 +31,7 @@
         from  g as i
         group by i.x.order_date as od
         group as g2
-        let sum = coll_sum((
+        let sum = strict_sum((
               select element g2.i.x.sum_price
               from g2
           ))
@@ -40,7 +40,7 @@
     )}
 from  (
     select element {'nation_key':nation_key,'order_date':orderdate,
-                    'sum_price': coll_sum((
+                    'sum_price': strict_sum((
                         select value g3.orders.o_totalprice
                         from g3
                     ))}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
index 8d37f4d..9903b78 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue786/query-issue786.3.query.sqlpp
@@ -33,7 +33,7 @@
         where orders.o_custkey = customer.c_custkey and customer.c_nationkey = nation.n_nationkey
         group by orders.o_orderdate as orderdate
         group as g
-        let  sum = coll_sum((
+        let  sum = strict_sum((
               select element g.orders.o_totalprice
               from g
         ))
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
index 38ba16f..6888d5f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-2/query-issue810-2.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
index f25f99d..4c25891 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810-3/query-issue810-3.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheaps),'avg_expensive_discounts':tpch.coll_avg(expensives),'sum_disc_prices':tpch.coll_sum(disc_prices),'total_charges':tpch.coll_sum(charges)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheaps),'avg_expensive_discounts':tpch.strict_avg(expensives),'sum_disc_prices':tpch.strict_sum(disc_prices),'total_charges':tpch.strict_sum(charges)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
index 6877db8..3ef97a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue810/query-issue810.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':coll_count(cheap),'count_expensives':coll_count(expensive)}
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_cheaps':strict_count(cheap),'count_expensives':strict_count(expensive)}
 from  LineItem as l
 where (l.l_shipdate <= '1998-09-02')
 /* +hash */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
index aefe8a3..7fac9f5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827-2/query-issue827-2.3.query.sqlpp
@@ -26,30 +26,30 @@
 use tpch;
 
 
-{'sum_qty_partial':tpch.coll_sum((
+{'sum_qty_partial':tpch.strict_sum((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'sum_base_price':tpch.coll_sum((
+)),'sum_base_price':tpch.strict_sum((
     select element i.l_extendedprice
     from  LineItem as i
-)),'sum_disc_price':tpch.coll_sum((
+)),'sum_disc_price':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount))
     from  LineItem as i
-)),'sum_charge':tpch.coll_sum((
+)),'sum_charge':tpch.strict_sum((
     select element (i.l_extendedprice * (1 - i.l_discount) * (1 + i.l_tax))
     from  LineItem as i
-)),'ave_qty':tpch.coll_avg((
+)),'ave_qty':tpch.strict_avg((
     select element i.l_quantity
     from  LineItem as i
     where (i.l_shipdate <= '1998-09-02')
-)),'ave_price':tpch.coll_avg((
+)),'ave_price':tpch.strict_avg((
     select element i.l_extendedprice
     from  LineItem as i
-)),'ave_disc':tpch.coll_avg((
+)),'ave_disc':tpch.strict_avg((
     select element i.l_discount
     from  LineItem as i
-)),'count_order':coll_count((
+)),'count_order':strict_count((
     select element l
     from  LineItem as l
 ))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
index c488ab1..c8dd6f2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue827/query-issue827.3.query.sqlpp
@@ -26,10 +26,10 @@
 use tpch;
 
 
-{'count_cheaps': coll_count((
+{'count_cheaps': strict_count((
     select element l.l_quantity
     from  LineItem as l
-)),'count_expensives': coll_sum((
+)),'count_expensives': strict_sum((
     select element e
     from  (
         select element l.l_extendedprice
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.4.query.sqlpp
new file mode 100644
index 0000000..3f08b2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a b-tree secondary index
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+use test;
+
+select count(*)
+from UpsertTo x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp
new file mode 100644
index 0000000..d3162f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type sub as {
+  subscriptionId: uuid
+};
+
+create dataset subscriptions(sub) primary key subscriptionId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp
new file mode 100644
index 0000000..c4ba876
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.2.update.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker1","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":
+  "dataverse1","brokerName":"broker1","created":current_datetime()}]}
+END
+))
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp
new file mode 100644
index 0000000..0677150
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.3.query.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+upsert into subscriptions as record(
+(let v = (select value s from subscriptions s where param0 = "HenryGale")
+select value (CASE (array_count(v) > 0)
+
+WHEN true THEN
+{"subscriptionId":v[0].subscriptionId, "param0": v[0].param0,
+"brokerSubscriptions":(select value sub from v[0].brokerSubscriptions sub UNION ALL
+  select value val from [{"brokerSubscriptionId":create_uuid(),
+  "brokerDataverse":"dataverse1","brokerName":"broker3","created":current_datetime()}] val)}
+ELSE
+{"subscriptionId":create_uuid(), "param0": "HenryGale",
+"brokerSubscriptions":[{"brokerSubscriptionId":create_uuid(), "brokerDataverse":"dataverse1",
+  "brokerName":"broker3","created":current_datetime()}]}
+END
+))
+) returning
+(let subs = record.brokerSubscriptions
+select value brokerName from subs
+order by created DESC limit 1)[0];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp
new file mode 100644
index 0000000..51889d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-case-returning/upsert-case-returning.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test for issue ASTERIXDB-2400
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+use test;
+
+select value array_count(brokerSubscriptions) from subscriptions;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
index a383042..709b3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.2.query.sqlpp
@@ -23,7 +23,7 @@
  * Date         : 31st May 2013
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  `Metadata`.`Function` as x
     where (x.DataverseName = 'test')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
index a383042..709b3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-issue489/query-issue489.4.query.sqlpp
@@ -23,7 +23,7 @@
  * Date         : 31st May 2013
  */
 
-select element coll_count((
+select element strict_count((
     select element x
     from  `Metadata`.`Function` as x
     where (x.DataverseName = 'test')
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
index 1380ed2..efe17a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
@@ -17,8 +17,4 @@
  * under the License.
  */
 use test;
-
-
 select element test.computeBonus(-1,-1);
-select element test.computeBonus(1,-1);
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp
new file mode 100644
index 0000000..5ee25ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+use test;
+select element test.computeBonus(1,-1);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/issue2411/issue2411.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/issue2411/issue2411.3.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/issue2411/issue2411.3.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 0e619cc..79db9ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -6,6 +6,8 @@
   \},
   "config" : \{
     "active\.memory\.global\.budget" : 67108864,
+    "active\.stop\.timeout" : 3600,
+    "active\.suspend\.timeout" : 3600,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
@@ -13,8 +15,10 @@
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
+    "log\.dir" : "logs/",
     "log\.level" : "INFO",
     "max\.wait\.active\.cluster" : 60,
+    "max.web.request.size" : 52428800,
     "messaging\.frame\.count" : 512,
     "messaging\.frame\.size" : 4096,
     "metadata\.callback\.port" : 0,
@@ -37,7 +41,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index b3a3eed..6e5547d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -6,6 +6,8 @@
   \},
   "config" : \{
     "active\.memory\.global\.budget" : 67108864,
+    "active\.stop\.timeout" : 3600,
+    "active\.suspend\.timeout" : 3600,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
@@ -13,8 +15,10 @@
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
+    "log\.dir" : "logs/",
     "log\.level" : "WARN",
     "max\.wait\.active\.cluster" : 60,
+    "max.web.request.size" : 52428800,
     "messaging\.frame\.count" : 512,
     "messaging\.frame\.size" : 4096,
     "metadata\.callback\.port" : 0,
@@ -37,7 +41,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 75e879c..3237551 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -6,6 +6,8 @@
   \},
   "config" : \{
     "active\.memory\.global\.budget" : 67108864,
+    "active\.stop\.timeout" : 3600,
+    "active\.suspend\.timeout" : 3600,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
@@ -13,8 +15,10 @@
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
+    "log\.dir" : "logs/",
     "log\.level" : "WARN",
     "max\.wait\.active\.cluster" : 60,
+    "max.web.request.size" : 52428800,
     "messaging\.frame\.count" : 512,
     "messaging\.frame\.size" : 4096,
     "metadata\.callback\.port" : 0,
@@ -37,7 +41,7 @@
     "txn\.lock\.timeout\.sweepthreshold" : 10000,
     "txn\.lock\.timeout\.waitthreshold" : 60000,
     "txn\.log\.buffer\.numpages" : 8,
-    "txn\.log\.buffer\.pagesize" : 131072,
+    "txn\.log\.buffer\.pagesize" : 4194304,
     "txn\.log\.checkpoint\.history" : 0,
     "txn\.log\.checkpoint\.lsnthreshold" : 67108864,
     "txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm
new file mode 100644
index 0000000..f1f17c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5 }} }, { "$1": {{ "iphone", "platform", "sth", 5 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "iphone", "voice-clarity", "sth", 5 }} }, { "$1": {{ "samsung", "platform", "sth", 5 }} }, { "$1": {{ "t-mobile", "shortcut-menu", "sth", 5 }} }, { "$1": {{ "verizon", "voicemail-service", "sth", 5 }} } ], "t2": [ { "$2": [ 3, "John", [ { "sth": 33 }, { "sth": 44 } ] ] } ], "t4": null, "t5": null, "t7": [ 5, 10, 12.0, "sth" ], "t9": [ 3, 3, [ 9 ], null, "sth" ], "t10": [ {  }, { "$3": [ "John Green", "Emily Jones", "sth1", "sth2" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm
new file mode 100644
index 0000000..2ee9cbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5, 3, 2 }} }, { "$1": {{ "iphone", "platform", "sth", 5, 3, 2 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "iphone", "voice-clarity", "sth", 5, 3, 2 }} }, { "$1": {{ "samsung", "platform", "sth", 5, 3, 2 }} }, { "$1": {{ "t-mobile", "shortcut-menu", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "voicemail-service", "sth", 5, 3, 2 }} } ], "t2": [ { "$2": [ 3, "John", { "sth": 33 }, { "sth": 44 } ] } ], "t4": null, "t5": null, "t7": [ 5, 10, 12.0 ], "t9": [ 3, null, 3, [ 9 ], null, null, null ], "t10": [ {  }, { "$3": [ "John Green", "Emily Jones", "sth", 5, 3, 2 ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm
new file mode 100644
index 0000000..9c1f2d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": true }, { "$1": false }, { "$1": true }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false } ], "t2": [ { "$2": true } ], "t3": [ { "$3": true } ], "t4": [ { "$4": false } ], "t5": [ { "$5": true } ], "t6": [ { "$6": null } ], "t7": [ {  } ], "t8": [ {  } ], "t9": [ { "$9": null } ], "t10": [ { "$10": null } ], "t11": [ { "$11": true } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm
new file mode 100644
index 0000000..7461980
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "t-mobile", "customization", "verizon", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "platform" }} }, { "$1": {{ "iphone", "platform", "verizon" }} }, { "$1": {{ "samsung", "voice-command", "verizon", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platform" }} }, { "$1": {{ "sprint", "voice-command", "verizon", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platform" }} }, { "$1": {{ "iphone", "voice-clarity", "verizon", "platform" }} }, { "$1": {{ "samsung", "platform", "verizon" }} }, { "$1": {{ "t-mobile", "shortcut-menu", "verizon", "platform" }} }, { "$1": {{ "verizon", "voicemail-service", "platform" }} } ], "t2": [ 19, 5, 7, 2 ], "t3": [ 19, 5, 7, 5.1, 2 ], "t4": [ 19, 5, "a", 7.5, "A", "John" ], "t5": [ 19, null, 7, 5 ], "t6": [ 3 ], "t7": null, "t8": [  ], "t10": null, "t11": [ {  }, { "$2": [ "John Green", "Emily Jones", "sth" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm
new file mode 100644
index 0000000..72de628
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm
@@ -0,0 +1 @@
+{ "t1": [ 2, 3, 7, 1, 2, 8, [ 12, 13, 14 ] ], "t2": [ 2, 3, 7, 1, "a", 8, [ 12, "b", 14 ] ], "t3": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t4": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t5": [ 2, 3, [ 7, 1, "a" ], [ 8, [ 12, "b", 14 ] ] ], "t6": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t7": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t8": null, "t9": [ 2, 3, null, 7, 1, null, "a", 8, 12, null, "b", 14, null ], "t11": null, "t13": null, "t15": null, "t16": null, "t17": [ {  }, { "$1": [ "John Green", "Emily Jones", "sth" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm
new file mode 100644
index 0000000..a7bc50e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": "t-mobile" }, { "$1": "verizon" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "verizon" }, { "$1": "motorola" }, { "$1": "sprint" }, { "$1": "motorola" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "t-mobile" }, { "$1": "verizon" } ], "t2": [ { "$2": { "sth": 44 } } ], "t7": null, "t8": null, "t10": null, "t13": null, "t3": 3, "t4": 2, "t5": 4, "t6": 4, "t9": null, "t11": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm
new file mode 100644
index 0000000..0acb072
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm
@@ -0,0 +1 @@
+{ "t1": [ "a", "b", 1, 2, 3 ], "t2": [ 1, 2, 3, "a", "b" ], "t3": [ 1, 1, 2, "a", "b", 4 ], "t4": [ 1, 1, 2, 7, "a", 7, "one more", 4 ], "t5": null, "t6": [ 1, 2, "a", "b", 3 ], "t7": null, "t8": null, "t10": null, "t11": [ 10, 12.0, "sth" ], "t12": null, "t13": [ 6, 9 ], "t14": null, "t15": null, "t17": [ 6, null, 9, null ], "t18": null, "t20": [ { "$1": {{ 5, "t-mobile", "customization" }} }, { "$1": {{ 5, "verizon", "voice-clarity" }} }, { "$1": {{ 5, "iphone", "platform" }} }, { "$1": {{ 5, "samsung", "voice-command" }} }, { "$1": {{ 5, "verizon", "shortcut-menu" }} }, { "$1": {{ 5, "motorola", "speed" }} }, { "$1": {{ 5, "sprint", "voice-command" }} }, { "$1": {{ 5, "motorola", "speed" }} }, { "$1": {{ 5, "iphone", "voice-clarity" }} }, { "$1": {{ 5, "samsung", "platform" }} }, { "$1": {{ 5, "t-mobile", "shortcut-menu" }} }, { "$1": {{ 5, "verizon", "voicemail-service" }} } ], "t21": [ {  }, { "$2": [ 5, 3, "John Green", "Emily Jones" ] } ], "t22": null, "t23": null, "t24": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_intersect/array_intersect.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_intersect/array_intersect.3.adm
new file mode 100644
index 0000000..368becd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_intersect/array_intersect.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "tweetid": "1", "$1": {{ "t-mobile" }} }, { "tweetid": "10", "$1": {{  }} }, { "tweetid": "11", "$1": {{  }} }, { "tweetid": "12", "$1": {{  }} }, { "tweetid": "2", "$1": {{  }} }, { "tweetid": "3", "$1": {{  }} }, { "tweetid": "4", "$1": {{  }} }, { "tweetid": "5", "$1": {{  }} }, { "tweetid": "6", "$1": {{  }} }, { "tweetid": "7", "$1": {{  }} }, { "tweetid": "8", "$1": {{ "t-mobile" }} }, { "tweetid": "9", "$1": {{  }} } ], "t2": [ { "$2": [ 2, 1 ] } ], "t3": [ 3, 5 ], "t4": [ 3, 5 ], "t5": [ 3, "a" ], "t6": [ 3 ], "t7": [  ], "t8": [  ], "t9": [  ], "t10": [  ], "t12": null, "t13": null, "t15": [  ], "t16": [  ], "t17": [ {  }, { "$3": [ "John Green" ] } ], "t18": [ 1, 2 ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_position/array_position.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_position/array_position.3.adm
new file mode 100644
index 0000000..e5f1b8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_position/array_position.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": -1 }, { "$1": -1 }, { "$1": -1 }, { "$1": -1 }, { "$1": -1 }, { "$1": 1 }, { "$1": -1 }, { "$1": 1 }, { "$1": -1 }, { "$1": -1 }, { "$1": -1 }, { "$1": -1 } ], "t2": [ { "$2": 1 } ], "t3": [ { "$3": 3 } ], "t4": [ { "$4": -1 } ], "t5": [ { "$5": 2 } ], "t6": [ { "$6": null } ], "t7": [ {  } ], "t8": [ {  } ], "t9": [ { "$9": null } ], "t10": [ { "$10": null } ], "t11": [ { "$11": 2 } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm
new file mode 100644
index 0000000..e3b0b0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "sth", 5, "t-mobile", "customization" }} }, { "$1": {{ "sth", 5, "verizon", "voice-clarity" }} }, { "$1": {{ "sth", 5, "iphone", "platform" }} }, { "$1": {{ "sth", 5, "samsung", "voice-command" }} }, { "$1": {{ "sth", 5, "verizon", "shortcut-menu" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "sprint", "voice-command" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "iphone", "voice-clarity" }} }, { "$1": {{ "sth", 5, "samsung", "platform" }} }, { "$1": {{ "sth", 5, "t-mobile", "shortcut-menu" }} }, { "$1": {{ "sth", 5, "verizon", "voicemail-service" }} } ], "t2": [ { "$2": [ [ { "sth": 33 }, { "sth": 44 } ], 3, "John" ] } ], "t4": null, "t5": null, "t7": [ 5, 10, 12.0, "sth", 77, "val" ], "t8": [ 5, 10, null, "sth", 77, "val" ], "t10": [ {  }, { "$3": [ "sth1", "sth2", "John Green", "Emily Jones" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm
new file mode 100644
index 0000000..b61b69c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "t-mobile", "customization", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "t-mobile", 5 }} }, { "$1": {{ "iphone", "platform", "t-mobile", 5 }} }, { "$1": {{ "samsung", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "sprint", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "iphone", "voice-clarity", "t-mobile", 5 }} }, { "$1": {{ "samsung", "platform", "t-mobile", 5 }} }, { "$1": {{ "t-mobile", "shortcut-menu", 5 }} }, { "$1": {{ "verizon", "voicemail-service", "t-mobile", 5 }} } ], "t3": null, "t5": [ 5, 10, 12.0, "sth" ], "t7": null, "t8": [ 3, 2, "sth", 9, 9 ], "t9": [ 3, 2, "sth", 1, 5 ], "t10": null, "t11": [ {  }, { "$2": [ "John Green", "Emily Jones", "sth1", "sth2" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm
new file mode 100644
index 0000000..362debf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm
@@ -0,0 +1 @@
+{ "t1": [ 1, 2, 3, 4, 5 ], "t2": [ 0, 5, 10, 15, 20 ], "t3": [ 0.1, 1.1 ], "t4": [ 10, 7, 4 ], "t5": [ -1, -4, -7 ], "t6": [  ], "t7": [  ], "t8": [  ], "t9": [  ], "t10": [ 2 ], "t11": [ 2.0, 3.1, 4.2, 5.300000000000001, 6.4, 7.5, 8.6 ], "t12": null, "t15": null, "t16": null, "t17": null, "t18": null, "t19": null, "t20": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm
new file mode 100644
index 0000000..c8f9de7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "shortcut-menu" }} }, { "$1": {{ "verizon", "voicemail-service" }} } ], "t3": null, "t5": [  ], "t7": null, "t8": [ 2 ], "t9": null, "t10": [ {  }, { "$2": [ "Emily Jones" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_repeat/array_repeat.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_repeat/array_repeat.1.adm
new file mode 100644
index 0000000..963aeac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_repeat/array_repeat.1.adm
@@ -0,0 +1 @@
+{ "t1": [ [ "a", "a", "a" ] ], "t2": [ "a", "a", "a" ], "t3": [  ], "t4": null, "t5": null, "t8": null, "t9": null, "t10": [ { "a": 1 }, { "a": 1 }, { "a": 1 } ], "t11": [ [ 1, 2 ], [ 1, 2 ], [ 1, 2 ] ], "t12": [ "a", "a", "a", "a" ], "t13": null, "t14": null, "t15": null, "t16": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm
new file mode 100644
index 0000000..3fc58af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "coffee-mobile", "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "coffee-mobile", "shortcut-menu" }} }, { "$1": {{ "verizon", "voicemail-service" }} } ], "t2": [ 3, -2, 1, 5, -2, 9, -2, 3, -2, 1, -2 ], "t3": [ 3, -2, 1, 5, -4, 9, -4, 3, -4, 1, -4 ], "t4": [ 3, -2, 1, 5, "2", 9, -4, 3, -4, 1, -4 ], "t5": [ 3, -2, -2, 5, "2", 9, -2, 3, "A", 1, 2 ], "t6": [ 3, -2, "a", 5, "2", 9, [ 2, 3, "A" ], 1, -2 ], "t7": [ 3, -2, "a", 5, "2", 9, [ 2, 3, "A" ], 1, [ -2 ] ], "t8": [ 3, -5, -5, 2, 2, -5, 2, 1, 1 ], "t9": [ 3, 1, 1, 2, 2, 1, 2, 1, 1 ], "t10": [ 3, -5, -5, 2, 2, -5, 2, -5, 1 ], "t11": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ], "t12": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ], "t13": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ], "t14": [ 3, -5, 1, 2, 2, 1, 2, 1, 1 ], "t15": null, "t16": null, "t17": null, "t18": [  ], "t19": [ 3, 2 ], "t21": [ null, 2, null ], "t22": null, "t23": null, "t25": [ {  }, { "$2": [ "John Green", "sth" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_reverse/array_reverse.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_reverse/array_reverse.3.adm
new file mode 100644
index 0000000..2c24059
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_reverse/array_reverse.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "customization", "t-mobile" }} }, { "$1": {{ "voice-clarity", "verizon" }} }, { "$1": {{ "platform", "iphone" }} }, { "$1": {{ "voice-command", "samsung" }} }, { "$1": {{ "shortcut-menu", "verizon" }} }, { "$1": {{ "speed", "motorola" }} }, { "$1": {{ "voice-command", "sprint" }} }, { "$1": {{ "speed", "motorola" }} }, { "$1": {{ "voice-clarity", "iphone" }} }, { "$1": {{ "platform", "samsung" }} }, { "$1": {{ "shortcut-menu", "t-mobile" }} }, { "$1": {{ "voicemail-service", "verizon" }} } ], "t2": [ { "$2": [ 19, 7, 5 ] } ], "t3": [ { "$3": [ 19, 7.5, 5 ] } ], "t4": [ { "$4": [ "John", 19, 7.5, 5 ] } ], "t5": [ { "$5": [ 3 ] } ], "t6": [ { "$6": null } ], "t7": [ { "$7": [  ] } ], "t8": [ {  } ], "t9": [ { "$9": null } ], "t10": [ {  }, { "$10": [ "Emily Jones", "John Green" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm
new file mode 100644
index 0000000..330b8d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "customization", "t-mobile" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "shortcut-menu", "verizon" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "platform", "samsung" }} }, { "$1": {{ "shortcut-menu", "t-mobile" }} }, { "$1": {{ "verizon", "voicemail-service" }} } ], "t2": [ 2, 5, 5, 7, 7, 19 ], "t3": [ 2, 5.0, 5, 5.1, 7, 7, 19 ], "t4": [ 5, 7.5, 19, "A", "John", "a", "a" ], "t5": [ null, null, null, 5, 7, 19 ], "t6": [ 3 ], "t7": null, "t8": [  ], "t10": null, "t11": [ {  }, { "$2": [ "Abby", "Emily Jones", "John Green" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_star/array_star.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_star/array_star.3.adm
new file mode 100644
index 0000000..fcb0d35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_star/array_star.3.adm
@@ -0,0 +1 @@
+{ "t1": { "a": [ "a_val1", "a_val2", "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ] }, "t2": { "compType": [ { "sth": 33 }, { "sth": 44 } ], "followers": [ null, [ { "a": "a_val1", "b": "b_val1" }, { "a": "a_val2", "b": "b_val2" }, { "a": "a_val3", "b": "b_val3" } ] ], "id": [ 1, 2 ] }, "t3": { "a": [ "a_val1", "a_val2", "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ] }, "t4": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ] }, "t5": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ] }, "t6": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ], "c": [ null, null, "c_val3" ] }, "t7": { "a": [ 5, 3.2, "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ], "c": [ null, null, "c_val3" ] }, "t8": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", null, "b_val3" ] }, "t9": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", null, "b_val3" ] }, "t10": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", null, "b_val3" ] }, "t11": { "a": [ "a_val1", null, "a_val3" ], "b": [ "b_val1", null, "b_val3" ] }, "t12": { "a": [ "a_val1", null, null ], "b": [ "b_val1", null, null ] }, "t17": null, "t19": null, "t20": [ {  }, { "$1": { "a": [ "a_val1", "a_val2", "a_val3" ], "b": [ "b_val1", "b_val2", "b_val3" ] } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiff/array_symdiff.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiff/array_symdiff.3.adm
new file mode 100644
index 0000000..cc8cb81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiff/array_symdiff.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "customization", "coffee-mobile", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "coffee-mobile", "t-mobile" }} }, { "$1": {{ "samsung", "voice-command", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "sprint", "voice-command", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "voice-clarity", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "samsung", "coffee-mobile", "t-mobile" }} }, { "$1": {{ "shortcut-menu", "coffee-mobile", "platform" }} }, { "$1": {{ "verizon", "voicemail-service", "coffee-mobile", "t-mobile", "platform" }} } ], "t2": [ { "$2": [ "John", 2, 4, 5 ] } ], "t3": [ 7, 2 ], "t4": [ 5.0, 7, 2 ], "t5": [ "a", 7, 2 ], "t6": [ "A", 7, 2 ], "t7": [ "A", 7, null, 2, 1 ], "t8": [ null, "A", 7, null, 2, 1 ], "t10": null, "t11": null, "t13": [  ], "t14": [ 3 ], "t15": [ {  }, { "$3": [ "John Green", "sth" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiffn/array_symdiffn.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiffn/array_symdiffn.3.adm
new file mode 100644
index 0000000..b3528b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_symdiffn/array_symdiffn.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "customization", "coffee-mobile", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "coffee-mobile", "t-mobile" }} }, { "$1": {{ "samsung", "voice-command", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "sprint", "voice-command", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "voice-clarity", "coffee-mobile", "t-mobile", "platform" }} }, { "$1": {{ "samsung", "coffee-mobile", "t-mobile" }} }, { "$1": {{ "shortcut-menu", "coffee-mobile", "platform" }} }, { "$1": {{ "verizon", "voicemail-service", "coffee-mobile", "t-mobile", "platform" }} } ], "t2": [ { "$2": [ "John", 2, 4, 5 ] } ], "t3": [ 3, 7, 2 ], "t4": [ 3, 5.0, 7, 2 ], "t5": [ 3, "a", 7, 2 ], "t6": [ 3, "A", 7, 2 ], "t7": [ "A", 7, null, 2, 1 ], "t8": [ null, "A", 7, null, 2, 1 ], "t10": null, "t11": null, "t13": [  ], "t14": [ 3 ], "t15": [ {  }, { "$3": [ "John Green", "sth2", "sth", 4 ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_union/array_union.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_union/array_union.3.adm
new file mode 100644
index 0000000..8d6ebf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_union/array_union.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": {{ "t-mobile", "customization", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "platform", "t-mobile" }} }, { "$1": {{ "samsung", "voice-command", "t-mobile", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "t-mobile", "platform" }} }, { "$1": {{ "sprint", "voice-command", "t-mobile", "platform" }} }, { "$1": {{ "motorola", "speed", "t-mobile", "platform" }} }, { "$1": {{ "iphone", "voice-clarity", "t-mobile", "platform" }} }, { "$1": {{ "samsung", "platform", "t-mobile" }} }, { "$1": {{ "t-mobile", "shortcut-menu", "platform" }} }, { "$1": {{ "verizon", "voicemail-service", "t-mobile", "platform" }} } ], "t2": [ { "$2": [ 1, "John", 2, 4 ] } ], "t3": [ 3, 5, 1, 7, 2 ], "t4": [ 3, 5.0, 1, 7, 2 ], "t5": [ 3, "a", 1, 7, 2 ], "t6": [ 3, "a", 1, "A", 7, 2 ], "t7": [ 3, "a", null, "A", 7, null, 2, 1 ], "t8": [ 3, "a", null, "A", 7, 2, 1 ], "t9": [ 3, null, "a", null, "A", 7, 2, 1 ], "t10": [ 3, 5, 1, 7, 2 ], "t12": null, "t13": null, "t15": [  ], "t16": [ 3, 2 ], "t17": [ {  }, { "$3": [ "John Green", "Emily Jones", "sth", "1" ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/binary/concat2/concat2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/binary/concat2/concat2.1.adm
new file mode 100644
index 0000000..770f2cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/binary/concat2/concat2.1.adm
@@ -0,0 +1 @@
+{ "t1": null, "t2": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
new file mode 100644
index 0000000..fca4cbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "MISSING" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "MISSING" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "MISSING" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "MISSING" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "MISSING" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "MISSING" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
new file mode 100644
index 0000000..cfcc525
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NaN" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "NaN" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "NaN" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "NaN" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NaN" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NaN" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
new file mode 100644
index 0000000..1b75b97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "-INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "-INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "-INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "-INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "-INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "-INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
new file mode 100644
index 0000000..5c53877
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NULL" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "NULL" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "NULL" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "NULL" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NULL" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NULL" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
new file mode 100644
index 0000000..ebc67b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm
new file mode 100644
index 0000000..519d3ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.1.adm
@@ -0,0 +1 @@
+{ "s_suppkey": 1, "s_name": "Supplier#000000001", "s_address": " N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ", "s_nationkey": 17, "s_phone": "27-918-335-1736", "s_acctbal": 5755.94, "s_comment": "each slyly above the careful" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm
new file mode 100644
index 0000000..dda08f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.2.adm
@@ -0,0 +1,5 @@
+{ "s_suppkey": 1, "s_name": "Supplier#000000001", "s_address": " N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ", "s_nationkey": 17, "s_phone": "27-918-335-1736", "s_acctbal": 5755.94, "s_comment": "each slyly above the careful" }
+{ "s_suppkey": 2, "s_name": "Supplier#000000002", "s_address": "89eJ5ksX3ImxJQBvxObC,", "s_nationkey": 5, "s_phone": "15-679-861-2259", "s_acctbal": 4032.68, "s_comment": " slyly bold instructions. idle dependen" }
+{ "s_suppkey": 3, "s_name": "Supplier#000000003", "s_address": "q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3", "s_nationkey": 1, "s_phone": "11-383-516-1199", "s_acctbal": 4192.4, "s_comment": "blithely silent requests after the express dependencies are sl" }
+{ "s_suppkey": 4, "s_name": "Supplier#000000004", "s_address": "Bk7ah4CK8SYQTepEmvMkkgMwg", "s_nationkey": 15, "s_phone": "25-843-787-7479", "s_acctbal": 4641.08, "s_comment": "riously even requests above the exp" }
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm
new file mode 100644
index 0000000..69756c0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.3.adm
@@ -0,0 +1 @@
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm
new file mode 100644
index 0000000..49ed112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/secondary_idx_lookup/secondary_idx_lookup.4.adm
@@ -0,0 +1,5 @@
+{ "s_suppkey": 2, "s_name": "Supplier#000000002", "s_address": "89eJ5ksX3ImxJQBvxObC,", "s_nationkey": 5, "s_phone": "15-679-861-2259", "s_acctbal": 4032.68, "s_comment": " slyly bold instructions. idle dependen" }
+{ "s_suppkey": 3, "s_name": "Supplier#000000003", "s_address": "q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3", "s_nationkey": 1, "s_phone": "11-383-516-1199", "s_acctbal": 4192.4, "s_comment": "blithely silent requests after the express dependencies are sl" }
+{ "s_suppkey": 5, "s_name": "Supplier#000000005", "s_address": "Gcdm2rJRzl5qlTVzc", "s_nationkey": 11, "s_phone": "21-151-690-3663", "s_acctbal": -283.84, "s_comment": ". slyly regular pinto bea" }
+{ "s_suppkey": 6, "s_name": "Supplier#000000006", "s_address": "tQxuVm7s7CnK", "s_nationkey": 14, "s_phone": "24-696-997-4969", "s_acctbal": 1365.79, "s_comment": "final accounts. regular dolphins use against the furiously ironic decoys. " }
+{ "s_suppkey": 10, "s_name": "Supplier#000000010", "s_address": "Saygah3gYWMp72i PY", "s_nationkey": 24, "s_phone": "34-852-489-8585", "s_acctbal": 3891.91, "s_comment": "ing waters. regular requests ar" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
index 58d9bdc..dd41cbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
@@ -1,3 +1,3 @@
-{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20, "-cashBack": -100 } }
-{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70, "-cashBack": -350 } }
-{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90, "-cashBack": -450 } }
+{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20.0, "-cashBack": -100 } }
+{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70.0, "-cashBack": -350 } }
+{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90.0, "-cashBack": -450 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
index 89f8083..2990e0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
@@ -1,3 +1,3 @@
 { "custname": "Jodi Rotruck", "age": null, "MathAge": { "age": null, "age+5": null, "age-5": null, "age*5": null, "age/5": null, "-age": null } }
-{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3, "-age": -19 } }
-{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2, "-age": -12 } }
+{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3.8, "-age": -19 } }
+{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2.4, "-age": -12 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
index 98cedcc..384e8f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
@@ -1,44 +1,44 @@
-distribute result [$$36]
+distribute result [$$40]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$36])
+    project ([$$40])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$36] <- [{"deptId": $#1, "star_cost": $$39}]
+      assign [$$40] <- [{"deptId": $#1, "star_cost": $$43}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$44]) decor ([]) {
-                    aggregate [$$39] <- [agg-sql-sum($$43)]
+          group by ([$#1 := $$48]) decor ([]) {
+                    aggregate [$$43] <- [agg-sql-sum($$47)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$44]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$48]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
-              group by ([$$44 := $$37]) decor ([]) {
-                        aggregate [$$43] <- [agg-local-sql-sum($$34)]
+            -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+              group by ([$$48 := $$41]) decor ([]) {
+                        aggregate [$$47] <- [agg-local-sql-sum($$38)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$37]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$41]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$34, $$37])
+                  project ([$$38, $$41])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$37] <- [substring($$41.getField("department_id"), 0)]
+                    assign [$$41] <- [substring($$45.getField("department_id"), 0)]
                     -- ASSIGN  |PARTITIONED|
-                      project ([$$34, $$41])
+                      project ([$$38, $$45])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$34, $$41] <- [$$e.getField("salary"), $$e.getField("dept")]
+                        assign [$$38, $$45] <- [$$e.getField("salary"), $$e.getField("dept")]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$e])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$38, $$e] <- gby.Employee
+                              data-scan []<-[$$42, $$e] <- gby.Employee
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
index e2e1c91..b82dda7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
@@ -1,40 +1,40 @@
-distribute result [$$35]
+distribute result [$$39]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$35])
+    project ([$$39])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$35] <- [{"deptId": $#1, "star_cost": $$38}]
+      assign [$$39] <- [{"deptId": $#1, "star_cost": $$42}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$42]) decor ([]) {
-                    aggregate [$$38] <- [agg-sql-sum($$41)]
+          group by ([$#1 := $$46]) decor ([]) {
+                    aggregate [$$42] <- [agg-sql-sum($$45)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$42]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$46]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
-              group by ([$$42 := $$36]) decor ([]) {
-                        aggregate [$$41] <- [agg-local-sql-sum($$33)]
+            -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+              group by ([$$46 := $$40]) decor ([]) {
+                        aggregate [$$45] <- [agg-local-sql-sum($$37)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$36]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$40]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$33, $$36])
+                  project ([$$37, $$40])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$36, $$33] <- [substring($$e.getField(1), 0), $$e.getField(2)]
+                    assign [$$40, $$37] <- [substring($$e.getField(1), 0), $$e.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$37, $$e] <- gby.Employee
+                          data-scan []<-[$$41, $$e] <- gby.Employee
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.adm
new file mode 100644
index 0000000..d0d0910
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-meta-pk-index/change-feed-with-meta-pk-index.1.adm
@@ -0,0 +1 @@
+{ "$1": 804 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
index 51fdf04..f599e28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
@@ -1,3 +1 @@
 10
-10
-10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
new file mode 100644
index 0000000..022f6a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
@@ -0,0 +1 @@
+{ "userName": "c1121u1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/order-by-13/order-by-13.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/order-by-13/order-by-13.3.adm
new file mode 100644
index 0000000..f82c642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/order-by-13/order-by-13.3.adm
@@ -0,0 +1,4 @@
+{ "deptno_str": "1" }
+{ "deptno_str": "1" }
+{ "deptno_str": "2" }
+{ "deptno_str": "2" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.3.adm
index 85c3c4f..29ad37b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.3.adm
@@ -1,3 +1,267 @@
-{ "id": 668945643054870528 }
-{ "id": 668945646725017600 }
-{ "id": 668945653892911104 }
+{ "id": 682833640322134016 }
+{ "id": 683196027344662529 }
+{ "id": 683558422088794113 }
+{ "id": 683558423464665088 }
+{ "id": 683920801054195712 }
+{ "id": 683920803415523329 }
+{ "id": 683920808838696960 }
+{ "id": 683920814719254528 }
+{ "id": 684283194439368704 }
+{ "id": 684283199648825345 }
+{ "id": 684283201813000192 }
+{ "id": 685007984288743426 }
+{ "id": 685370366114447360 }
+{ "id": 685370374209585152 }
+{ "id": 685732756693889025 }
+{ "id": 685732760837697537 }
+{ "id": 686095131498426368 }
+{ "id": 686095135470321665 }
+{ "id": 686819922857463808 }
+{ "id": 687182291936804864 }
+{ "id": 687182298148552704 }
+{ "id": 687182300388298752 }
+{ "id": 687182302477041664 }
+{ "id": 687544695635849217 }
+{ "id": 687544698198573056 }
+{ "id": 688269475871760384 }
+{ "id": 688269477067100160 }
+{ "id": 688269480661630976 }
+{ "id": 688994233542295552 }
+{ "id": 688994238709682176 }
+{ "id": 688994248616628224 }
+{ "id": 688994250109800449 }
+{ "id": 689356622334947328 }
+{ "id": 689719010884091904 }
+{ "id": 689719033185234945 }
+{ "id": 690081416013545472 }
+{ "id": 690443782626635776 }
+{ "id": 690443790461575169 }
+{ "id": 690443810023870464 }
+{ "id": 691168572681408512 }
+{ "id": 691168573838917632 }
+{ "id": 691530962941771780 }
+{ "id": 691530963470454784 }
+{ "id": 691893338530803712 }
+{ "id": 691893353852592128 }
+{ "id": 691893362182479872 }
+{ "id": 692255726270599168 }
+{ "id": 692618119894831104 }
+{ "id": 692618121048301571 }
+{ "id": 692980501607518210 }
+{ "id": 692980517197717504 }
+{ "id": 692980526588780544 }
+{ "id": 693342888600571906 }
+{ "id": 693342893407211520 }
+{ "id": 693342902026702850 }
+{ "id": 693705280366841857 }
+{ "id": 694067668417015808 }
+{ "id": 694067679657750528 }
+{ "id": 694430065778675712 }
+{ "id": 694792437311475712 }
+{ "id": 694792447105191938 }
+{ "id": 694792456517189632 }
+{ "id": 695154826926170116 }
+{ "id": 695517219397234688 }
+{ "id": 695517222324703232 }
+{ "id": 695517227634835456 }
+{ "id": 695517228016410624 }
+{ "id": 695517233133391872 }
+{ "id": 695517240431505408 }
+{ "id": 695879621884006400 }
+{ "id": 696242012761169920 }
+{ "id": 696604380133511169 }
+{ "id": 696604403197816832 }
+{ "id": 697329171442216961 }
+{ "id": 697329184100618240 }
+{ "id": 697691548260020224 }
+{ "id": 698053939514646528 }
+{ "id": 698053949505417217 }
+{ "id": 698053955733958656 }
+{ "id": 698416327652990976 }
+{ "id": 698416346028085248 }
+{ "id": 699503486233407488 }
+{ "id": 699865866956099585 }
+{ "id": 699865872735690752 }
+{ "id": 699865883443752960 }
+{ "id": 702402593037926400 }
+{ "id": 702764972766568449 }
+{ "id": 703127357909897217 }
+{ "id": 703127358547427328 }
+{ "id": 703127359231135745 }
+{ "id": 703852138833661954 }
+{ "id": 703852144533700608 }
+{ "id": 703852146161090560 }
+{ "id": 704214530629435392 }
+{ "id": 704214540062433281 }
+{ "id": 704576917601710081 }
+{ "id": 704576927466549248 }
+{ "id": 704576932596174848 }
+{ "id": 704939301851758592 }
+{ "id": 704939311901319170 }
+{ "id": 705301697854189568 }
+{ "id": 705301698093273088 }
+{ "id": 705301707807264768 }
+{ "id": 705664078425886721 }
+{ "id": 706026463053545472 }
+{ "id": 706026483597217792 }
+{ "id": 706026486164094977 }
+{ "id": 706388860054466560 }
+{ "id": 706388863757918208 }
+{ "id": 706388867314688001 }
+{ "id": 706751242668650496 }
+{ "id": 706751261257945088 }
+{ "id": 707113635336867840 }
+{ "id": 707476013500997632 }
+{ "id": 707476023219097600 }
+{ "id": 707838400649256960 }
+{ "id": 707838402020782080 }
+{ "id": 708200795825541120 }
+{ "id": 708925567924436992 }
+{ "id": 709272857855922178 }
+{ "id": 709997628533116928 }
+{ "id": 709997630328328192 }
+{ "id": 710360015517790208 }
+{ "id": 715795839034269696 }
+{ "id": 715795839495577601 }
+{ "id": 715795840066068484 }
+{ "id": 715795847041191938 }
+{ "id": 715795849247326210 }
+{ "id": 716520615331500035 }
+{ "id": 716883007873818624 }
+{ "id": 717245385299726336 }
+{ "id": 717245410230730752 }
+{ "id": 717607776172642304 }
+{ "id": 717607791616139264 }
+{ "id": 717607801317502976 }
+{ "id": 717970161991438336 }
+{ "id": 717970174821793792 }
+{ "id": 718332560988893184 }
+{ "id": 718332568656027648 }
+{ "id": 718694944580378625 }
+{ "id": 719057333100408832 }
+{ "id": 719057333544878081 }
+{ "id": 719057337881735168 }
+{ "id": 719419730537230337 }
+{ "id": 719782116750274560 }
+{ "id": 719782118834851840 }
+{ "id": 719782124425863168 }
+{ "id": 719782124677521408 }
+{ "id": 720144497384955905 }
+{ "id": 720506877847670785 }
+{ "id": 720506890933923841 }
+{ "id": 720869273409708033 }
+{ "id": 721231658112970752 }
+{ "id": 721231668380569600 }
+{ "id": 721231677138141188 }
+{ "id": 721594060687949824 }
+{ "id": 722318833701240832 }
+{ "id": 723043598460055552 }
+{ "id": 723043621755195393 }
+{ "id": 723405978994020353 }
+{ "id": 723405979421806592 }
+{ "id": 723768382303002625 }
+{ "id": 723768382357561344 }
+{ "id": 724130759716405248 }
+{ "id": 724130763608854528 }
+{ "id": 724493151080091648 }
+{ "id": 724493154334855168 }
+{ "id": 724493155958026240 }
+{ "id": 724493165143552001 }
+{ "id": 724493165722390528 }
+{ "id": 724855554707959808 }
+{ "id": 725217934361157632 }
+{ "id": 725217934872813568 }
+{ "id": 725217948827156482 }
+{ "id": 725580305709469696 }
+{ "id": 725580309597577216 }
+{ "id": 725580315595436032 }
+{ "id": 725942707181363200 }
+{ "id": 725942715339296768 }
+{ "id": 726305100801695745 }
+{ "id": 726667476759678976 }
+{ "id": 727029875031216129 }
+{ "id": 727029881201147904 }
+{ "id": 727392248841703424 }
+{ "id": 727392251681206272 }
+{ "id": 727392255644848129 }
+{ "id": 727754638448074757 }
+{ "id": 727754640914296832 }
+{ "id": 727754642017390593 }
+{ "id": 727754648631816192 }
+{ "id": 728117024514179073 }
+{ "id": 728117031157915648 }
+{ "id": 728117034781827073 }
+{ "id": 728479418520199168 }
+{ "id": 728479433191919616 }
+{ "id": 728479434575994885 }
+{ "id": 728841796319502336 }
+{ "id": 728841813474316288 }
+{ "id": 728841814866857985 }
+{ "id": 729204197586046976 }
+{ "id": 729204200463175680 }
+{ "id": 729928969160687616 }
+{ "id": 729928971207335936 }
+{ "id": 730291355595870209 }
+{ "id": 730291359232491520 }
+{ "id": 730291365607657472 }
+{ "id": 730291366140379136 }
+{ "id": 730653752261148672 }
+{ "id": 730653758611283969 }
+{ "id": 731016142324539392 }
+{ "id": 731378513278701568 }
+{ "id": 731378515652673537 }
+{ "id": 731740907318300673 }
+{ "id": 732103295406219264 }
+{ "id": 732828063168397313 }
+{ "id": 733190456578707464 }
+{ "id": 733190466720530433 }
+{ "id": 733552852417675264 }
+{ "id": 733552857429868544 }
+{ "id": 733552862265892866 }
+{ "id": 733915239570276352 }
+{ "id": 733915240132317185 }
+{ "id": 733915240547504129 }
+{ "id": 734277620183896064 }
+{ "id": 734277620725108736 }
+{ "id": 734640011405099010 }
+{ "id": 734640015481942016 }
+{ "id": 735002389917229056 }
+{ "id": 735002396758269952 }
+{ "id": 735002401879396352 }
+{ "id": 735002406883328000 }
+{ "id": 735727177921007617 }
+{ "id": 735727178935980034 }
+{ "id": 736089562045251584 }
+{ "id": 736451956495896577 }
+{ "id": 736451959113142276 }
+{ "id": 736814329627070464 }
+{ "id": 736814338468544512 }
+{ "id": 737176722756362240 }
+{ "id": 737176732659126272 }
+{ "id": 737539106679332865 }
+{ "id": 737901496461889537 }
+{ "id": 738263883614298112 }
+{ "id": 738263884461576192 }
+{ "id": 738626272385859584 }
+{ "id": 738626277087666176 }
+{ "id": 738626277788131329 }
+{ "id": 739713433969401860 }
+{ "id": 739713444669100032 }
+{ "id": 739713445268852737 }
+{ "id": 740075824276230144 }
+{ "id": 740438209059008512 }
+{ "id": 740438213744066561 }
+{ "id": 740438214234808320 }
+{ "id": 740438219108560896 }
+{ "id": 740438223546159104 }
+{ "id": 740800605627809792 }
+{ "id": 741525375126274048 }
+{ "id": 741525379341377537 }
+{ "id": 741887773544615937 }
+{ "id": 742250161208889344 }
+{ "id": 742250161758294016 }
+{ "id": 742612535971581952 }
+{ "id": 742612547778531328 }
+{ "id": 742974925124866052 }
+{ "id": 742974929453350912 }
+{ "id": 742974940115271680 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.4.adm
index 17babd8..8ed3e21 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.4.adm
@@ -1 +1 @@
-{ "id": 668945643054870528 }
+{ "id": 682833640322134016 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.5.adm
index 2d91ff6..8147406 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-08/fulltext-08.5.adm
@@ -1,4 +1,269 @@
-{ "id": 668945643054870528 }
-{ "id": 668945646725017600 }
-{ "id": 668945651263115264 }
-{ "id": 668945653892911104 }
+{ "id": 682833640322134016 }
+{ "id": 683196027344662529 }
+{ "id": 683558422088794113 }
+{ "id": 683558423464665088 }
+{ "id": 683920801054195712 }
+{ "id": 683920803415523329 }
+{ "id": 683920808838696960 }
+{ "id": 683920814719254528 }
+{ "id": 684283194439368704 }
+{ "id": 684283199648825345 }
+{ "id": 684283201813000192 }
+{ "id": 685007984288743426 }
+{ "id": 685370366114447360 }
+{ "id": 685370374209585152 }
+{ "id": 685732756693889025 }
+{ "id": 685732760837697537 }
+{ "id": 686095131498426368 }
+{ "id": 686095135470321665 }
+{ "id": 686819922857463808 }
+{ "id": 687182291936804864 }
+{ "id": 687182298148552704 }
+{ "id": 687182300388298752 }
+{ "id": 687182302477041664 }
+{ "id": 687544695635849217 }
+{ "id": 687544698198573056 }
+{ "id": 688269475871760384 }
+{ "id": 688269477067100160 }
+{ "id": 688269480661630976 }
+{ "id": 688994233542295552 }
+{ "id": 688994238709682176 }
+{ "id": 688994248616628224 }
+{ "id": 688994250109800449 }
+{ "id": 689356622334947328 }
+{ "id": 689719010884091904 }
+{ "id": 689719033185234945 }
+{ "id": 690081416013545472 }
+{ "id": 690443782626635776 }
+{ "id": 690443790461575169 }
+{ "id": 690443810023870464 }
+{ "id": 691168572681408512 }
+{ "id": 691168573838917632 }
+{ "id": 691530962941771780 }
+{ "id": 691530963470454784 }
+{ "id": 691893338530803712 }
+{ "id": 691893353852592128 }
+{ "id": 691893362182479872 }
+{ "id": 692255726270599168 }
+{ "id": 692618119894831104 }
+{ "id": 692618121048301571 }
+{ "id": 692980501607518210 }
+{ "id": 692980517197717504 }
+{ "id": 692980526588780544 }
+{ "id": 693342888600571906 }
+{ "id": 693342893407211520 }
+{ "id": 693342902026702850 }
+{ "id": 693705280366841857 }
+{ "id": 694067668417015808 }
+{ "id": 694067679657750528 }
+{ "id": 694430065778675712 }
+{ "id": 694792437311475712 }
+{ "id": 694792447105191938 }
+{ "id": 694792456517189632 }
+{ "id": 695154826926170116 }
+{ "id": 695517219397234688 }
+{ "id": 695517222324703232 }
+{ "id": 695517227634835456 }
+{ "id": 695517228016410624 }
+{ "id": 695517233133391872 }
+{ "id": 695517240431505408 }
+{ "id": 695879621884006400 }
+{ "id": 696242012761169920 }
+{ "id": 696604380133511169 }
+{ "id": 696604403197816832 }
+{ "id": 697329171442216961 }
+{ "id": 697329184100618240 }
+{ "id": 697691548260020224 }
+{ "id": 698053939514646528 }
+{ "id": 698053949505417217 }
+{ "id": 698053955733958656 }
+{ "id": 698416327652990976 }
+{ "id": 698416346028085248 }
+{ "id": 699503486233407488 }
+{ "id": 699865866956099585 }
+{ "id": 699865872735690752 }
+{ "id": 699865883443752960 }
+{ "id": 702402593037926400 }
+{ "id": 702764972766568449 }
+{ "id": 703127357909897217 }
+{ "id": 703127358547427328 }
+{ "id": 703127359231135745 }
+{ "id": 703852138833661954 }
+{ "id": 703852144533700608 }
+{ "id": 703852146161090560 }
+{ "id": 704214530629435392 }
+{ "id": 704214540062433281 }
+{ "id": 704576917601710081 }
+{ "id": 704576927466549248 }
+{ "id": 704576932596174848 }
+{ "id": 704939301851758592 }
+{ "id": 704939311901319170 }
+{ "id": 705301697854189568 }
+{ "id": 705301698093273088 }
+{ "id": 705301707807264768 }
+{ "id": 705664078425886721 }
+{ "id": 706026463053545472 }
+{ "id": 706026483597217792 }
+{ "id": 706026486164094977 }
+{ "id": 706388860054466560 }
+{ "id": 706388863757918208 }
+{ "id": 706388867314688001 }
+{ "id": 706751242668650496 }
+{ "id": 706751261257945088 }
+{ "id": 707113635336867840 }
+{ "id": 707476013500997632 }
+{ "id": 707476023219097600 }
+{ "id": 707838400649256960 }
+{ "id": 707838402020782080 }
+{ "id": 708200795825541120 }
+{ "id": 708925567924436992 }
+{ "id": 709272857855922178 }
+{ "id": 709997628533116928 }
+{ "id": 709997630328328192 }
+{ "id": 710360015517790208 }
+{ "id": 715795839034269696 }
+{ "id": 715795839495577601 }
+{ "id": 715795840066068484 }
+{ "id": 715795847041191938 }
+{ "id": 715795849247326210 }
+{ "id": 716520615331500035 }
+{ "id": 716883007873818624 }
+{ "id": 717245385299726336 }
+{ "id": 717245410230730752 }
+{ "id": 717607776172642304 }
+{ "id": 717607791616139264 }
+{ "id": 717607801317502976 }
+{ "id": 717970161991438336 }
+{ "id": 717970174821793792 }
+{ "id": 718332560988893184 }
+{ "id": 718332568656027648 }
+{ "id": 718694944580378625 }
+{ "id": 719057333100408832 }
+{ "id": 719057333544878081 }
+{ "id": 719057337881735168 }
+{ "id": 719419730537230337 }
+{ "id": 719782116750274560 }
+{ "id": 719782118834851840 }
+{ "id": 719782124425863168 }
+{ "id": 719782124677521408 }
+{ "id": 720144497384955905 }
+{ "id": 720506877847670785 }
+{ "id": 720506890933923841 }
+{ "id": 720869273409708033 }
+{ "id": 721231658112970752 }
+{ "id": 721231668380569600 }
+{ "id": 721231677138141188 }
+{ "id": 721594060687949824 }
+{ "id": 722318833701240832 }
+{ "id": 723043598460055552 }
+{ "id": 723043621755195393 }
+{ "id": 723405978994020353 }
+{ "id": 723405979421806592 }
+{ "id": 723768382303002625 }
+{ "id": 723768382357561344 }
+{ "id": 724130759716405248 }
+{ "id": 724130763608854528 }
+{ "id": 724493151080091648 }
+{ "id": 724493154334855168 }
+{ "id": 724493155958026240 }
+{ "id": 724493165143552001 }
+{ "id": 724493165722390528 }
+{ "id": 724855554707959808 }
+{ "id": 725217934361157632 }
+{ "id": 725217934872813568 }
+{ "id": 725217948827156482 }
+{ "id": 725580305709469696 }
+{ "id": 725580309597577216 }
+{ "id": 725580315595436032 }
+{ "id": 725942707181363200 }
+{ "id": 725942715339296768 }
+{ "id": 726305100801695745 }
+{ "id": 726667476759678976 }
+{ "id": 727029864222576640 }
+{ "id": 727029875031216129 }
+{ "id": 727029881201147904 }
+{ "id": 727392248841703424 }
+{ "id": 727392251681206272 }
+{ "id": 727392255644848129 }
+{ "id": 727754638448074757 }
+{ "id": 727754640914296832 }
+{ "id": 727754642017390593 }
+{ "id": 727754648631816192 }
+{ "id": 728117024514179073 }
+{ "id": 728117031157915648 }
+{ "id": 728117034781827073 }
+{ "id": 728479418520199168 }
+{ "id": 728479433191919616 }
+{ "id": 728479434575994885 }
+{ "id": 728841796319502336 }
+{ "id": 728841813474316288 }
+{ "id": 728841814866857985 }
+{ "id": 729204197586046976 }
+{ "id": 729204200463175680 }
+{ "id": 729928969160687616 }
+{ "id": 729928971207335936 }
+{ "id": 730291355595870209 }
+{ "id": 730291359232491520 }
+{ "id": 730291365607657472 }
+{ "id": 730291366140379136 }
+{ "id": 730653752261148672 }
+{ "id": 730653758611283969 }
+{ "id": 731016142324539392 }
+{ "id": 731378513278701568 }
+{ "id": 731378515652673537 }
+{ "id": 731378535537860608 }
+{ "id": 731740907318300673 }
+{ "id": 732103295406219264 }
+{ "id": 732828063168397313 }
+{ "id": 733190456578707464 }
+{ "id": 733190466720530433 }
+{ "id": 733552852417675264 }
+{ "id": 733552857429868544 }
+{ "id": 733552862265892866 }
+{ "id": 733915239570276352 }
+{ "id": 733915240132317185 }
+{ "id": 733915240547504129 }
+{ "id": 734277620183896064 }
+{ "id": 734277620725108736 }
+{ "id": 734640011405099010 }
+{ "id": 734640015481942016 }
+{ "id": 735002389917229056 }
+{ "id": 735002396758269952 }
+{ "id": 735002401879396352 }
+{ "id": 735002406883328000 }
+{ "id": 735727177921007617 }
+{ "id": 735727178935980034 }
+{ "id": 736089562045251584 }
+{ "id": 736451956495896577 }
+{ "id": 736451959113142276 }
+{ "id": 736814329627070464 }
+{ "id": 736814338468544512 }
+{ "id": 737176722756362240 }
+{ "id": 737176732659126272 }
+{ "id": 737539106679332865 }
+{ "id": 737901496461889537 }
+{ "id": 738263883614298112 }
+{ "id": 738263884461576192 }
+{ "id": 738626272385859584 }
+{ "id": 738626277087666176 }
+{ "id": 738626277788131329 }
+{ "id": 739713433969401860 }
+{ "id": 739713444669100032 }
+{ "id": 739713445268852737 }
+{ "id": 740075824276230144 }
+{ "id": 740438209059008512 }
+{ "id": 740438213744066561 }
+{ "id": 740438214234808320 }
+{ "id": 740438219108560896 }
+{ "id": 740438223546159104 }
+{ "id": 740800605627809792 }
+{ "id": 741525375126274048 }
+{ "id": 741525379341377537 }
+{ "id": 741887773544615937 }
+{ "id": 742250161208889344 }
+{ "id": 742250161758294016 }
+{ "id": 742612535971581952 }
+{ "id": 742612547778531328 }
+{ "id": 742974925124866052 }
+{ "id": 742974929453350912 }
+{ "id": 742974940115271680 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-08/fulltext-index-02.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-08/fulltext-index-08.8.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-08/fulltext-index-02.8.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-08/fulltext-index-08.8.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.3.adm
new file mode 100644
index 0000000..564afbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.3.adm
@@ -0,0 +1 @@
+{ "$1": 4933 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.4.adm
new file mode 100644
index 0000000..01f8fb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/fulltext-index-09/fulltext-index-09.4.adm
@@ -0,0 +1 @@
+{ "$1": 3968 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm
new file mode 100644
index 0000000..83efad9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/function/issue-2394/issue-2394.1.adm
@@ -0,0 +1 @@
+{ "t": { "id": 999403843673718784, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393280, "id_str": "988455822727393280", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } }, "temp": [ { "f": { "id": 999403845, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393281, "id_str": "988455822727393281", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } } }, { "f": { "id": 999403846, "timestamp_ms": "1527111419014", "user": { "id": 988455822727393282, "id_str": "988455822727393282", "screen_name": "TweetUser" }, "place": { "country_code": "US", "country": "United States", "full_name": "Riverside, CA", "bounding_box": { "coordinates": [ [ [ -117.523867, 33.85216 ], [ -117.523867, 34.019484 ], [ -117.271365, 34.019484 ], [ -117.271365, 33.85216 ] ] ] } } } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
new file mode 100644
index 0000000..2a16ab1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
@@ -0,0 +1,14 @@
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
new file mode 100644
index 0000000..b9029e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
@@ -0,0 +1,3 @@
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-69.1991349,-12.6006222],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-69.199136,-12.6010968],[-69.1972972,-12.6010968],[-69.1972972,-12.5998133],[-69.199136,-12.5998133],[-69.199136,-12.6010968]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[1,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[1,2],[7,2],[7,8],[1,8],[1,2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-113.981,39.195],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-113.98,39.198],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-113.981,39.195],[-113.98,39.195],[-113.98,39.198],[-113.981,39.198],[-113.981,39.195]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
new file mode 100644
index 0000000..048a6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
@@ -0,0 +1,14 @@
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
new file mode 100644
index 0000000..3af2594
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
@@ -0,0 +1,14 @@
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 13, "XMax": -69.1972972, "XMin": -69.199136, "YMax": -12.5998133, "YMin": -12.6010968, "Binary": hex("01020000000D00000081BF4EA0BE4C51C0CE80C4C0843329C033DDEBA4BE4C51C0D388997D1E3329C064D7ACE9B04C51C05787927F1C3329C02347DF49B04C51C0B9C49107223329C0F4DCE79DAF4C51C054B76922223329C0184339D1AE4C51C042A89C40223329C048E17A14AE4C51C010DBCCD71B3329C007DB3E9AA04C51C0636996BA1A3329C0E9656F84A04C51C05ADD45F35B3329C04E9B711AA24C51C033373A42613329C06CE22E0CA24C51C066D24B31AC3329C05F454607A44C51C065170CAEB93329C07C8C03F9A34C51C0E475B3F5C23329C0"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-69.1991349 -12.6006222, -69.199136 -12.599842, -69.1982979 -12.5998268, -69.1982598 -12.599869, -69.1982188 -12.5998698, -69.19817 -12.5998707, -69.198125 -12.5998218, -69.1973024 -12.5998133, -69.1972972 -12.6003109, -69.197394 -12.6003514, -69.1973906 -12.6009231, -69.1975115 -12.601026, -69.1975081 -12.6010968)" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 3, "XMax": 7.0, "XMin": 1.0, "YMax": 8.0, "YMin": 2.0, "Binary": hex("010200000003000000000000000000F03F0000000000000040000000000000104000000000000014400000000000001C400000000000002040"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[1,2],[4,5],[7,8]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (1 2, 4 5, 7 8)" }
+{ "Type": "MultiPoint", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 4, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiPoint\",\"coordinates\":[[10,40],[40,30],[20,20],[30,10]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "MULTIPOINT ((10 40), (40 30), (20 20), (30 10))" }
+{ "Type": "MultiPolygon", "Area": 712.5, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 11, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 5.0, "Binary": hex("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440"), "GeoJSON": "{\"type\":\"MultiPolygon\",\"coordinates\":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],\"crs\":null}", "WKT": "MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)), ((20 35, 10 30, 10 10, 30 5, 45 20, 20 35), (30 20, 20 15, 20 25, 30 20)))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -71.1043443253471, "XMin": -71.1043443253471, "YMax": 42.3150676015829, "YMin": 42.3150676015829, "Binary": hex("0101000000E538D293ADC651C0F3699A2254284540"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-71.1043443253471,42.3150676015829],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-71.1043443253471 42.3150676015829)" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 3, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01E9030000000000000000F03F00000000000000400000000000000840"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT Z (1 2 3)" }
+{ "Type": "Polygon", "Area": 1.3755215000294761E-8, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 6, "XMax": 8.760178, "XMin": 8.7599721, "YMax": 49.7103478, "YMin": 49.7102133, "Binary": hex("01030000000100000007000000B1BE26101B852140ED20C033EBDA4840C11DA8531E8521407694394CEADA4840BAF8DB9E20852140D5F89683EADA484098EF2AB5288521409557F844E8DA48400247020D3685214041F74086E9DA484014F131B32A8521408DE43CADECDA4840B1BE26101B852140ED20C033EBDA4840"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))" }
+{ "Type": "MultiLineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 7, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiLineString\",\"coordinates\":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],\"crs\":null}", "WKT": "MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 4, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01B90B0000000000000000F03F000000000000004000000000000008400000000000001040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3,4],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT ZM (1 2 3 4)" }
+{ "Type": "Polygon", "Area": 928.625, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 743265.625, "XMin": 743238.0, "YMax": 2967450.0, "YMin": 2967416.0, "Binary": hex("01030000000100000005000000000000008CAE264100000000BCA3464100000040C3AE264100000000BCA3464100000000C2AE264100000000CDA34641000000008CAE264100000000CDA34641000000008CAE264100000000BCA34641"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416))" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 2, "XMax": -113.98, "XMin": -113.981, "YMax": 39.198, "YMin": 39.195, "Binary": hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-113.98,39.198],[-113.981,39.195]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-113.98 39.198, -113.981 39.195)" }
+{ "Type": "Polygon", "Area": 1.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 11.0, "XMin": 10.0, "YMax": 11.0, "YMin": 10.0, "Binary": hex("010300000001000000050000000000000000002440000000000000244000000000000026400000000000002440000000000000264000000000000026400000000000002440000000000000264000000000000024400000000000002440"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -118.4, "XMin": -118.4, "YMax": 33.93, "YMin": 33.93, "Binary": hex("01010000009A99999999995DC0D7A3703D0AF74040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-118.4,33.93],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-118.4 33.93)" }
+{ "Type": "Polygon", "Area": 675.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 7, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 10.0, "Binary": hex("0103000000020000000500000000000000008041400000000000002440000000000080464000000000008046400000000000002E40000000000000444000000000000024400000000000003440000000000080414000000000000024400400000000000000000034400000000000003E40000000000080414000000000008041400000000000003E40000000000000344000000000000034400000000000003E40"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
new file mode 100644
index 0000000..950f68f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
@@ -0,0 +1,4 @@
+{ "X": -71.1043443253471, "Y": 42.3150676015829, "Z": 0.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": 4.0 }
+{ "X": -118.4, "Y": 33.93, "Z": 0.0, "M": NaN }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
new file mode 100644
index 0000000..044e31a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
@@ -0,0 +1,4 @@
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[10,10],[11,10],[11,11],[10,11],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 1, "ExteriorRing": {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
new file mode 100644
index 0000000..7967722
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
@@ -0,0 +1,4 @@
+{ "Length": 0.004058119099397876, "Boundary": {"type":"MultiPoint","coordinates":[[-69.1991349,-12.6006222],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 78.9292222699217, "Boundary": {"type":"MultiPoint","coordinates":[[10,10],[10,40],[40,40],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 0.0031622776601655037, "Boundary": {"type":"MultiPoint","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 8.48528137423857, "Boundary": {"type":"MultiPoint","coordinates":[[1,2],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
new file mode 100644
index 0000000..a1e16dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
@@ -0,0 +1 @@
+{"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
new file mode 100644
index 0000000..f884259
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
@@ -0,0 +1,10 @@
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 1.4142135623730951, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": true, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 2.8284271247461903, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": true, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": true, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
new file mode 100644
index 0000000..d9b7206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
@@ -0,0 +1,10 @@
+{ "Union": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,0],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[-1,-1],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,3]],[[3,3],[4,2],[5,2]],[[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"LineString","coordinates":[[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[0,2],[1,2],[2,2],[1,1]],[[5,2],[4,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"Point","coordinates":[2,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Polygon","coordinates":[[[0.25,0.25],[0.5,0.25],[0.5,0.5],[0.25,0.5],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]],[[0.25,0.25],[0.25,0.5],[0.5,0.5],[0.5,0.25],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-06-distinct/sugar-06-distinct.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-06-distinct/sugar-06-distinct.9.adm
new file mode 100644
index 0000000..4f19bae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/sugar-06-distinct/sugar-06-distinct.9.adm
@@ -0,0 +1,3 @@
+{ "deptno": 3, "salary_agg": 6000 }
+{ "deptno": 2, "salary_agg": 5000 }
+{ "deptno": 1, "salary_agg": 3000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
new file mode 100644
index 0000000..ad86590
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
@@ -0,0 +1,7 @@
+{ "o_custkey": 40, "o_orderkey": 323, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 3653, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 4934, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 258, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 2596, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 3687, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 5378, "o_orderstatus": "F" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-cleanjson/int01.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-cleanjson/int01.1.json
index 1e25a41..f83d375 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-cleanjson/int01.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-cleanjson/int01.1.json
@@ -1,2 +1 @@
-[ [ 1, 2 ]
- ]
+[ [ 1, 2 ] ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-losslessjson/int01.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-losslessjson/int01.1.json
index a0533de..2280279 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-losslessjson/int01.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/int01-losslessjson/int01.1.json
@@ -1,2 +1,7 @@
-[ { "orderedlist": [ { "int64": 1 }, { "int64": 2 } ] }
- ]
+[ {
+  "orderedlist" : [ {
+    "int64" : 1
+  }, {
+    "int64" : 2
+  } ]
+} ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/issue-ASTERIXDB-1165/nullablefield.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/issue-ASTERIXDB-1165/nullablefield.1.json
index f2868b6..6f98a7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/json/issue-ASTERIXDB-1165/nullablefield.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json/issue-ASTERIXDB-1165/nullablefield.1.json
@@ -1,2 +1 @@
-[ 3
- ]
+[ 3 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.3.adm
new file mode 100644
index 0000000..af2cdfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.3.adm
@@ -0,0 +1 @@
+{ "t1": 0, "t2": 0, "t3": [ 1, 2 ], "t4": [ 1, 2 ], "t5": 0, "t6": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.4.adm
new file mode 100644
index 0000000..6e7b664
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_negative_value/limit_negative_value.4.adm
@@ -0,0 +1,2 @@
+{ "id": 34, "dblpid": "books/acm/Kim95", "title": "Modern Database Systems  The Object Model, Interoperability, and Beyond.", "authors": "", "misc": "2004-03-08 Won Kim Modern Database Systems ACM Press and Addison-Wesley 1995 0-201-59098-0 db/books/collections/kim95.html" }
+{ "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
new file mode 100644
index 0000000..af64121
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
@@ -0,0 +1,3 @@
+3
+4
+5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
new file mode 100644
index 0000000..9c4866f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$15(ASC), $$16(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$15, $$16, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$21) (ASC, $$22)
+                  -- STABLE_SORT [$$21(ASC), $$22(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$21, $$22])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$19, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$19] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
new file mode 100644
index 0000000..3d466ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
+{ "l_orderkey": 33, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5, "l_extendedprice": 5190.65, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". stealthily bold exc" }
+{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
+{ "l_orderkey": 35, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7, "l_extendedprice": 7147.84, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-01-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the carefully regular " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
new file mode 100644
index 0000000..7d9ef47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
@@ -0,0 +1,40 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$17])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$19(ASC), $$20(ASC) ]  |PARTITIONED|
+          project ([$$19, $$20, $$17])
+          -- STREAM_PROJECT  |PARTITIONED|
+            assign [$$17] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$18, 0)}]
+            -- ASSIGN  |PARTITIONED|
+              limit 5
+              -- STREAM_LIMIT  |PARTITIONED|
+                assign [$$18] <- [$$c.getField(2)]
+                -- ASSIGN  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    unnest-map [$$19, $$20, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$24, $$25, 2, $$24, $$25, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 5
+                    -- BTREE_SEARCH  |PARTITIONED|
+                      exchange
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        order (ASC, $$24) (ASC, $$25)
+                        -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
+                          exchange
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            project ([$$24, $$25])
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              exchange
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                unnest-map [$$23, $$24, $$25] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$22, TRUE, FALSE, FALSE)
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  exchange
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    assign [$$22] <- [150]
+                                    -- ASSIGN  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.adm
new file mode 100644
index 0000000..9b218e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.6.adm
@@ -0,0 +1,5 @@
+{ "shipdate": "1996", "suppkey": true }
+{ "shipdate": "1996", "suppkey": true }
+{ "shipdate": "1996", "suppkey": true }
+{ "shipdate": "1996", "suppkey": true }
+{ "shipdate": "1996", "suppkey": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
new file mode 100644
index 0000000..29357ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC), $$13(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$12, $$13, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$17, $$18, 2, $$17, $$18, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$17) (ASC, $$18)
+                  -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$17, $$18])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$16, $$17, $$18] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$15, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$15] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
new file mode 100644
index 0000000..eeedd56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
new file mode 100644
index 0000000..d070b2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$19) (ASC, $$20)
+                  -- STABLE_SORT [$$19(ASC), $$20(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$19, $$20])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$17] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
new file mode 100644
index 0000000..b60a0ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
new file mode 100644
index 0000000..afc422a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
@@ -0,0 +1,5 @@
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects  Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
new file mode 100644
index 0000000..b83be67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
@@ -0,0 +1,64 @@
+distribute result [$$31]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 2
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      exchange
+      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        project ([$$31])
+        -- STREAM_PROJECT  |PARTITIONED|
+          assign [$$31] <- [{"dblpid": $$32}]
+          -- ASSIGN  |PARTITIONED|
+            limit 2
+            -- STREAM_LIMIT  |PARTITIONED|
+              project ([$$32])
+              -- STREAM_PROJECT  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  join (eq($$32, $$35))
+                  -- HYBRID_HASH_JOIN [$$32][$$35]  |PARTITIONED|
+                    exchange
+                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                      project ([$$32])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        assign [$$32] <- [$$d.getField(1)]
+                        -- ASSIGN  |PARTITIONED|
+                          project ([$$d])
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$33, $$d] <- test.DBLP1
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  empty-tuple-source
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    exchange
+                    -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                      project ([$$35])
+                      -- STREAM_PROJECT  |UNPARTITIONED|
+                        assign [$$35] <- [get-item($$24, 0).getField(0).getField(1)]
+                        -- ASSIGN  |UNPARTITIONED|
+                          aggregate [$$24] <- [listify($$23)]
+                          -- AGGREGATE  |UNPARTITIONED|
+                            limit 1
+                            -- STREAM_LIMIT  |UNPARTITIONED|
+                              project ([$$23])
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                exchange
+                                -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
+                                  project ([$$34, $$23])
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    assign [$$23] <- [{"d": $$d}]
+                                    -- ASSIGN  |PARTITIONED|
+                                      limit 1
+                                      -- STREAM_LIMIT  |PARTITIONED|
+                                        exchange
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          data-scan []<-[$$34, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            exchange
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              empty-tuple-source
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
new file mode 100644
index 0000000..94608d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
@@ -0,0 +1,28 @@
+distribute result [$$16]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 1
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$16])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+          project ([$$18, $$16])
+          -- STREAM_PROJECT  |PARTITIONED|
+            assign [$$16] <- [{"$1": substring($$17, 0, 21)}]
+            -- ASSIGN  |PARTITIONED|
+              limit 1
+              -- STREAM_LIMIT  |PARTITIONED|
+                project ([$$18, $$17])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  assign [$$17] <- [$$DBLP1.getField(1)]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$18, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.adm
new file mode 100644
index 0000000..9a4a69d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.7.adm
@@ -0,0 +1 @@
+{ "$1": "series/synthesis/2009" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
new file mode 100644
index 0000000..131b860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
@@ -0,0 +1,28 @@
+distribute result [$$19]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 2
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$19])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+          limit 2
+          -- STREAM_LIMIT  |PARTITIONED|
+            project ([$$22, $$19])
+            -- STREAM_PROJECT  |PARTITIONED|
+              assign [$$19] <- [$$23.getField("lang")]
+              -- ASSIGN  |PARTITIONED|
+                project ([$$22, $$23])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  assign [$$23] <- [$$t.getField("user")]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$22, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.adm
new file mode 100644
index 0000000..51d3671
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.9.adm
@@ -0,0 +1,2 @@
+"en"
+"en"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
new file mode 100644
index 0000000..daeeeb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$10(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$10, $$paper] <- test.DBLP1 limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
new file mode 100644
index 0000000..e9f7751
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
@@ -0,0 +1,5 @@
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better  A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story  Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
new file mode 100644
index 0000000..ee3e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/query-ASTERIXDB-2420/query-ASTERIXDB-2420.1.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
new file mode 100644
index 0000000..91a2dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
@@ -0,0 +1 @@
+.*"processedObjects":1.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
new file mode 100644
index 0000000..8b095a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
@@ -0,0 +1 @@
+{ "v1": true, "v2": true, "v3": [ true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/comp-ASTERIXDB-2412/comp-ASTERIXDB-2412.1.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/ping/ping.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/ping/ping.1.adm
new file mode 100644
index 0000000..53d479d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/ping/ping.1.adm
@@ -0,0 +1,2 @@
+{ "node": "asterix_nc1" }
+{ "node": "asterix_nc2" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.adm
new file mode 100644
index 0000000..3cbb051
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2380/query-ASTERIXDB-2380.3.adm
@@ -0,0 +1,7 @@
+{ "first_country": "F", "second_country": "E", "mountains": [ { "first_country": "F", "second_country": "E", "mountain": "Vignemale", "sea": "Atlantic Ocean" }, { "first_country": "F", "second_country": "E", "mountain": "Vignemale", "sea": "Mediterranean Sea" } ] }
+{ "first_country": "I", "second_country": "F", "mountains": [ { "first_country": "I", "second_country": "F", "mountain": "Mont Blanc", "sea": "Mediterranean Sea" } ] }
+{ "first_country": "R", "second_country": "GE", "mountains": [ { "first_country": "R", "second_country": "GE", "mountain": "Kasbek", "sea": "Black Sea" } ] }
+{ "first_country": "RCH", "second_country": "RA", "mountains": [ { "first_country": "RCH", "second_country": "RA", "mountain": "Llullaillaco", "sea": "Atlantic Ocean" }, { "first_country": "RCH", "second_country": "RA", "mountain": "Ojos del Salado", "sea": "Atlantic Ocean" } ] }
+{ "first_country": "S", "second_country": "N", "mountains": [ { "first_country": "S", "second_country": "N", "mountain": "Sulitjelma", "sea": "Skagerrak" } ] }
+{ "first_country": "TJ", "second_country": "NOK", "mountains": [ { "first_country": "TJ", "second_country": "NOK", "mountain": "Changbai Shan", "sea": "Yellow Sea" } ] }
+{ "first_country": "USA", "second_country": "CDN", "mountains": [ { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Bona", "sea": "Arctic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Bona", "sea": "Atlantic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Bona", "sea": "Pacific Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Fairweather", "sea": "Arctic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Fairweather", "sea": "Atlantic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt Fairweather", "sea": "Pacific Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt St.Elias", "sea": "Arctic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt St.Elias", "sea": "Atlantic Ocean" }, { "first_country": "USA", "second_country": "CDN", "mountain": "Mt St.Elias", "sea": "Pacific Ocean" } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
new file mode 100644
index 0000000..4be3e9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
@@ -0,0 +1 @@
+{ "t1": 6, "t2": 6, "t3": 6, "t4": 6, "t5": [ true, true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm
new file mode 100644
index 0000000..c45a808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.10.adm
@@ -0,0 +1,6 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 4, "c2": 104 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm
new file mode 100644
index 0000000..8643d5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.5.adm
@@ -0,0 +1,10 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm
new file mode 100644
index 0000000..8643d5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.6.adm
@@ -0,0 +1,10 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 2, "c2": 102 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 6, "c2": 106 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.7.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm
new file mode 100644
index 0000000..78a404d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.8.adm
@@ -0,0 +1,8 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 105 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 3, "c2": 107 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 105 }
+{ "c1": 7, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm
new file mode 100644
index 0000000..97b278c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join-2/btree-secondary-non-enforced-equi-join-2.9.adm
@@ -0,0 +1,9 @@
+{ "c1": 1, "c2": 101 }
+{ "c1": 1, "c2": 103 }
+{ "c1": 2, "c2": 101 }
+{ "c1": 2, "c2": 103 }
+{ "c1": 3, "c2": 101 }
+{ "c1": 3, "c2": 103 }
+{ "c1": 5, "c2": 101 }
+{ "c1": 5, "c2": 103 }
+{ "c1": 7, "c2": 107 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm
new file mode 100644
index 0000000..a4b9391
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-join/btree-secondary-non-enforced-equi-join/btree-secondary-non-enforced-equi-join.4.adm
@@ -0,0 +1,2 @@
+{ "aid": 5, "bid": 98, "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom" }
+{ "aid": 34, "bid": 57, "authors": "" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm
new file mode 100644
index 0000000..04080b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.10.adm
@@ -0,0 +1,3 @@
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm
new file mode 100644
index 0000000..429a13d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.11.adm
@@ -0,0 +1,2 @@
+{ "res": 4 }
+{ "res": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm
new file mode 100644
index 0000000..f36e389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.4.adm
@@ -0,0 +1,2 @@
+{ "res": 3 }
+{ "res": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm
new file mode 100644
index 0000000..9897674
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.5.adm
@@ -0,0 +1,2 @@
+{ "res": 1 }
+{ "res": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm
new file mode 100644
index 0000000..c1c06bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.6.adm
@@ -0,0 +1,2 @@
+{ "res": 5 }
+{ "res": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm
new file mode 100644
index 0000000..c1c06bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.7.adm
@@ -0,0 +1,2 @@
+{ "res": 5 }
+{ "res": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm
new file mode 100644
index 0000000..e6e7ad6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.8.adm
@@ -0,0 +1,5 @@
+{ "res": 2 }
+{ "res": 3 }
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm
new file mode 100644
index 0000000..db56195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-01/non-enforced-01.9.adm
@@ -0,0 +1,4 @@
+{ "res": 3 }
+{ "res": 4 }
+{ "res": 5 }
+{ "res": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm
new file mode 100644
index 0000000..66fa150
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.4.adm
@@ -0,0 +1 @@
+{ "res": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm
new file mode 100644
index 0000000..18cc982
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-02/non-enforced-02.5.adm
@@ -0,0 +1,2 @@
+{ "res": 3 }
+{ "res": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm
new file mode 100644
index 0000000..31ca720
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-03/non-enforced-03.3.adm
@@ -0,0 +1 @@
+{ "id": 3, "nested": { "alias": "Hillery", "userSince": 2016 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm
new file mode 100644
index 0000000..05d6df6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-open-index/index-selection/non-enforced-04/non-enforced-04.4.adm
@@ -0,0 +1,2 @@
+4
+9
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
index 0a2275f..14620a1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissing/ifmissing.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "i": true }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
index 633c503..eff2651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifmissingornull/ifmissingornull.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
index 633c503..eff2651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/ifnull/ifnull.1.adm
@@ -1 +1 @@
-{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true }
\ No newline at end of file
+{ "a": true, "b": true, "c": true, "d": true, "e": true, "f": true, "g": true, "h": true, "i": true, "j": [ { "b": { "c": [ 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
index 920f7b4..cf13212 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
@@ -1 +1 @@
-{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false }
+{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false, "25": false, "26": false, "27": true, "28": true, "29": true, "30": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
new file mode 100644
index 0000000..abab248
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
@@ -0,0 +1 @@
+{ "e": 2.71, "pi": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
new file mode 100644
index 0000000..e1310b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
@@ -0,0 +1 @@
+{ "t1": 180 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_double/divide_double.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_double/divide_double.1.adm
index b371d5b..3f20110 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_double/divide_double.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_double/divide_double.1.adm
@@ -1 +1 @@
-{ "result1": -6.5, "result2": -3.25, "result3": -2.1666666666666665, "result4": 1.625, "result5": 1.1818181818181819, "result6": 1.0, "result7": null }
+{ "result1": -6.5, "result2": -3.25, "result3": -2.1666666666666665, "result4": 1.625, "result5": 1.1818181818181819, "result6": 1.0, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_float/divide_float.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_float/divide_float.1.adm
index ccbe5be..560f641 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_float/divide_float.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_float/divide_float.1.adm
@@ -1 +1 @@
-{ "result1": -5.5, "result2": -2.75, "result3": -1.8333334, "result4": 1.375, "result5": 1.0, "result6": 0.8461538461538461, "result7": null }
+{ "result1": -5.5, "result2": -2.75, "result3": -1.8333334, "result4": 1.375, "result5": 1.0, "result6": 0.8461538461538461, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
index 47c4e3c..0e5450b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
@@ -1 +1 @@
-{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
+{ "result1": 2.0, "result2": 1.0, "result3": 0.6666666666666666, "result4": -0.5, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
new file mode 100644
index 0000000..c25ec67
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
@@ -0,0 +1 @@
+{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null, "result9": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
index 83c2d70..96e5114 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
@@ -1 +1 @@
-{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
+{ "result1": 3.0, "result2": 1.5, "result3": 1.0, "result4": -0.75, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
new file mode 100644
index 0000000..9d8efd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
@@ -0,0 +1 @@
+{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null, "result9": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
index a6fea08..5e8ec2c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
@@ -1 +1 @@
-{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
+{ "result1": -4.0, "result2": -2.0, "result3": -1.3333333333333333, "result4": 1.0, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
new file mode 100644
index 0000000..79799d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
@@ -0,0 +1 @@
+{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null, "result9": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
index 1def226..4309a1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
@@ -1 +1 @@
-{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
+{ "result1": 1.0, "result2": 0.5, "result3": 0.3333333333333333, "result4": -0.25, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null, "result9": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
new file mode 100644
index 0000000..9233e8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
@@ -0,0 +1 @@
+{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null, "result9": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
index 9f56be4..852db4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
@@ -19,4 +19,5 @@
 [ 18, true ]
 [ 19, true ]
 [ 20, "NaN" ]
-[ 21, 2 ]
\ No newline at end of file
+[ 21, 2 ]
+[ 22, null ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/non-finite/non-finite.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/non-finite/non-finite.1.json
index 78d4801..d576906 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/non-finite/non-finite.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/non-finite/non-finite.1.json
@@ -1,2 +1,5 @@
-[ { "NaN": "NaN", "Infinity": "INF", "-Infinity": "-INF" }
- ]
\ No newline at end of file
+[ {
+  "NaN" : "NaN",
+  "Infinity" : "INF",
+  "-Infinity" : "-INF"
+} ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
new file mode 100644
index 0000000..6091a8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
@@ -0,0 +1 @@
+{ "t1": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
new file mode 100644
index 0000000..676edeb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_pairs/object_pairs.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_pairs/object_pairs.2.adm
new file mode 100644
index 0000000..713966d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_pairs/object_pairs.2.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": [ { "name": "a", "value": { "b": { "c": { "d": 1 } } } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
new file mode 100644
index 0000000..fc9a55a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
new file mode 100644
index 0000000..61d0506
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true ], "t2": { "b": 2 }, "t3": { "a": 1 }, "t4": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 18, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 } ], "t6": [ { "tweetid": "1", "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, { "tweetid": "10", "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, { "tweetid": "11", "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, { "tweetid": "12", "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, { "tweetid": "2", "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, { "tweetid": "3", "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, { "tweetid": "4", "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, { "tweetid": "5", "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, { "tweetid": "6", "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, { "tweetid": "7", "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, { "tweetid": "8", "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, { "tweetid": "9", "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
new file mode 100644
index 0000000..197fec1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true ], "t2": { "a": 1, "c": 2 }, "t3": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "language": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t6": [ { "tweetid": "1", "user_details": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm
new file mode 100644
index 0000000..cd24aa1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true, true, true ], "t2": { "a": 2, "b": 2 }, "t3": { "a": "1", "b": "3" }, "t4": { "a": null }, "t5": { "a": 2, "b": 2 }, "t6": { "a": 2, "b": 2 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": { "en": "native" }, "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1000", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
new file mode 100644
index 0000000..87269dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true, true, true ], "t2": "1", "t3": { "b": "1" }, "t4": [ 1, 2 ], "t5": [ { "$1": "en" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm
new file mode 100644
index 0000000..2216760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true ], "t2": [ { "a": 3 }, [ 1, 2 ], 4 ], "t3": [ [ "ChangEwing_573", "en", 182, 394, "Chang Ewing", 32136 ] ], "t4": [ [ "1", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, point("47.44,80.65"), datetime("2008-04-26T10:10:00.000Z"), {{ "t-mobile", "customization" }}, " love t-mobile its customization is good:)" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm
new file mode 100644
index 0000000..41715ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true, true ], "t2": [ [  ], [  ], [  ], [  ], [ [ "a", 3 ] ] ], "t3": [ [ "object", { "a": { "b": { "c": 3 } } } ], [ "a", { "b": { "c": 3 } } ], [ "b", { "c": 3 } ], [ "c", 3 ], [ "array", [ 1, 2 ] ], [ "array", 1 ], [ "array", 2 ], [ "primitive", 4 ] ], "t4": [ [ [ "screen-name", "ChangEwing_573" ], [ "lang", "en" ], [ "friends_count", 182 ], [ "statuses_count", 394 ], [ "name", "Chang Ewing" ], [ "followers_count", 32136 ] ] ], "t5": [ [ [ "tweetid", "1" ], [ "user", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ], [ "screen-name", "NathanGiesen@211" ], [ "lang", "en" ], [ "friends_count", 39339 ], [ "statuses_count", 473 ], [ "name", "Nathan Giesen" ], [ "followers_count", 49416 ], [ "sender-location", point("47.44,80.65") ], [ "send-time", datetime("2008-04-26T10:10:00.000Z") ], [ "referred-topics", {{ "t-mobile", "customization" }} ], [ "referred-topics", "t-mobile" ], [ "referred-topics", "customization" ], [ "message-text", " love t-mobile its customization is good:)" ] ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.1.adm
new file mode 100644
index 0000000..310bc8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.1.adm
@@ -0,0 +1,3 @@
+{ "t": 1 }
+{ "t": 2 }
+{ "t": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.2.adm
new file mode 100644
index 0000000..fe22dac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-star/var_star_2/var_star_2.2.adm
@@ -0,0 +1,3 @@
+{  }
+{  }
+{  }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial-large-data/spatial-large-data.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial-large-data/spatial-large-data.1.adm
new file mode 100644
index 0000000..0645f4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial-large-data/spatial-large-data.1.adm
@@ -0,0 +1 @@
+{ "$1": 2691 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm
new file mode 100644
index 0000000..7190640
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm
@@ -0,0 +1 @@
+{ "t1": 7.0, "t2": "hello world world" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm
new file mode 100644
index 0000000..3ed423f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm
@@ -0,0 +1 @@
+{ "t1": "a b b c c" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm
new file mode 100644
index 0000000..2d95320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm
new file mode 100644
index 0000000..e16cae9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": null, "p_bool_type": false, "p_int_type": false, "p_dec_type": false, "p_dbl_type": false, "p_str_type": true, "p_arr_type": false, "p_obj_type": false }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm
new file mode 100644
index 0000000..e7eb119
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm
@@ -0,0 +1 @@
+{ "$1": 42, "$2": "hello" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_04/named_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_04/named_04.1.adm
new file mode 100644
index 0000000..f70d7bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_04/named_04.1.adm
@@ -0,0 +1 @@
+42
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm
new file mode 100644
index 0000000..aab6f21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm
new file mode 100644
index 0000000..2d95320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm
new file mode 100644
index 0000000..f8c0d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm
new file mode 100644
index 0000000..5a2e691
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm
@@ -0,0 +1 @@
+{ "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.3.adm
new file mode 100644
index 0000000..84e880e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.3.adm
@@ -0,0 +1 @@
+{ "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_05/postitional_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_05/postitional_05.1.adm
new file mode 100644
index 0000000..5494401
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_05/postitional_05.1.adm
@@ -0,0 +1 @@
+{ "$1": 4, "$2": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/query-ASTERIXDB-2413/query-ASTERIXDB-2413.1.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_replace/regexp_replace.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_replace/regexp_replace.6.adm
new file mode 100644
index 0000000..1005ff0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_replace/regexp_replace.6.adm
@@ -0,0 +1 @@
+{ "result1": "akkkcadakkk", "result2": "abracadabra", "result3": "akkkcadabra", "result4": "akkkcadakkk_abracadabra", "result5": "akkkcadakkk_akkkcadakkk" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/replace_with_limit/replace_with_limit.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/replace_with_limit/replace_with_limit.1.adm
index 1e7020e..cc5bbeb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/replace_with_limit/replace_with_limit.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/replace_with_limit/replace_with_limit.1.adm
@@ -9,4 +9,4 @@
 [ 8, "_xyz_ba_xyz_xyz_xyz_" ]
 [ 9, "a*BRA*cada*BRA*_a*BRA*cadabra_xyz" ]
 [ 10, "ab_ba_ab" ]
-[ 11, "ab_ba_ab" ]
\ No newline at end of file
+[ 11, "ba_ba_ba_ba_ba" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
new file mode 100644
index 0000000..a2b8b2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true ], "t2": "", "t3": "dcba", "t4": [ 98, 8457, 247, 8451, 215, 97 ], "t5": [ "ba", "cba", "dcba" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.1.adm
new file mode 100644
index 0000000..770f2cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.1.adm
@@ -0,0 +1 @@
+{ "t1": null, "t2": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.2.adm
new file mode 100644
index 0000000..770f2cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-concat2/string-concat2.2.adm
@@ -0,0 +1 @@
+{ "t1": null, "t2": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
index ac9dedd..c8cbcf0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
@@ -1 +1 @@
-{ "str2": "ld", "str4": "g", "str6": "", "str8": "This is a test string", "str10": "This is a test string", "str13": "gThis is a another test string", "str14": "Irvine" }
+{ "str2": "ld", "str4": "g", "str6": null, "str8": "This is a test string", "str10": "string", "str13": "gThis is a another test string", "str14": "Irvine" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
index 74bd0d6..4661f40 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr04/substr04.1.adm
@@ -6,3 +6,6 @@
 "Irvine"
 "UC Irvine"
 "Irvine"
+"BC"
+null
+null
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-3/substring2-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-3/substring2-3.1.adm
index 04393a4..f60907d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-3/substring2-3.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-3/substring2-3.1.adm
@@ -1 +1 @@
-{ "result1": "" }
+{ "result1": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
index 5b0c7b9..8355b34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
@@ -1 +1 @@
-{ "result1": "HEllow" }
+{ "result1": "low", "result2": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
index b92a705..a94cd89 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
@@ -1 +1 @@
-[ "ab", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
+[ "g", null, "ab", "bc", "cd" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
index e341861..9b005ca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
@@ -1 +1 @@
-[ "abcdefg", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
+[ "g", null, "abcdefg", "bcdefg", "cdefg" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.9.adm
new file mode 100644
index 0000000..c79bf28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.9.adm
@@ -0,0 +1,2 @@
+4
+5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/primary-secondary-btree/primary-secondary-btree.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/primary-secondary-btree/primary-secondary-btree.2.adm
new file mode 100644
index 0000000..71c9709
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/primary-secondary-btree/primary-secondary-btree.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm
new file mode 100644
index 0000000..4dbe13c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.3.adm
@@ -0,0 +1 @@
+"broker3"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm
new file mode 100644
index 0000000..bf0d87a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/upsert-case-returning/upsert-case-returning.4.adm
@@ -0,0 +1 @@
+4
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
index c76f17a..d99fd3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
@@ -1,2 +1 @@
 -0.1
--0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
new file mode 100644
index 0000000..13a1d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
@@ -0,0 +1 @@
+-0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_06/customer_q_06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_06/customer_q_06.3.ast
index 0ebf373..b9e2111 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_06/customer_q_06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_06/customer_q_06.3.ast
@@ -45,7 +45,7 @@
   OrderedListConstructor [
     IndexAccessor [
       Variable [ Name=$m ]
-      Index: ANY
+      Index:       LiteralExpr [LONG] [0]
     ]
     IndexAccessor [
       Variable [ Name=$m ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_07/customer_q_07.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_07/customer_q_07.3.ast
index 3c8d114..12e09be 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_07/customer_q_07.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/customer_q_07/customer_q_07.3.ast
@@ -45,7 +45,7 @@
   UnorderedListConstructor [
     IndexAccessor [
       Variable [ Name=$m ]
-      Index: ANY
+      Index:       LiteralExpr [LONG] [0]
     ]
     IndexAccessor [
       Variable [ Name=$m ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/join_q_03/join_q_03.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/join_q_03/join_q_03.3.ast
index 6653361..a1dace8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/join_q_03/join_q_03.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/join_q_03/join_q_03.3.ast
@@ -57,7 +57,7 @@
     :
     IndexAccessor [
       Variable [ Name=$ul ]
-      Index: ANY
+      Index:       LiteralExpr [LONG] [0]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_05/order_q_05.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_05/order_q_05.3.ast
index 2682444..4ee2bc8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_05/order_q_05.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_05/order_q_05.3.ast
@@ -49,7 +49,7 @@
     :
     IndexAccessor [
       Variable [ Name=$c2 ]
-      Index: ANY
+      Index:       LiteralExpr [LONG] [0]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_06/order_q_06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_06/order_q_06.3.ast
index 3aae7c8..c0ec9bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_06/order_q_06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/custord/order_q_06/order_q_06.3.ast
@@ -7,7 +7,7 @@
     :
     IndexAccessor [
       Variable [ Name=$c3 ]
-      Index: ANY
+      Index:       LiteralExpr [LONG] [0]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/employee/q_02/q_02.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/employee/q_02/q_02.3.ast
index 43344eb..7eb3a51 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/employee/q_02/q_02.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/employee/q_02/q_02.3.ast
@@ -28,7 +28,7 @@
             Variable [ Name=$e ]
             Field=interests
           ]
-          Index: ANY
+          Index:           LiteralExpr [LONG] [0]
         ]
       )
       (
@@ -39,7 +39,7 @@
             Variable [ Name=$e ]
             Field=children
           ]
-          Index: ANY
+          Index:           LiteralExpr [LONG] [0]
         ]
       )
       (
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
index 4748a29..a11de35 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
@@ -5,10 +5,7 @@
   (
     LiteralExpr [STRING] [partkey]
     :
-    FieldAccessor [
-      Variable [ Name=$gen0 ]
-      Field=partkey
-    ]
+    Variable [ Name=#1 ]
   )
   (
     LiteralExpr [STRING] [pid]
@@ -18,18 +15,12 @@
   (
     LiteralExpr [STRING] [shipdate]
     :
-    FieldAccessor [
-      Variable [ Name=$j ]
-      Field=l_shipdate
-    ]
+    Variable [ Name=#2 ]
   )
   (
     LiteralExpr [STRING] [orderkey]
     :
-    FieldAccessor [
-      Variable [ Name=$j ]
-      Field=l_orderkey
-    ]
+    Variable [ Name=#3 ]
   )
 ]
 ]
@@ -105,6 +96,24 @@
  AT
 Variable [ Name=$p ]
 ]
+Let Variable [ Name=#1 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$gen0 ]
+    Field=partkey
+  ]
+Let Variable [ Name=#2 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_shipdate
+  ]
+Let Variable [ Name=#3 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_orderkey
+  ]
 Where
   OperatorExpr [
     Variable [ Name=$p ]
@@ -112,19 +121,10 @@
     LiteralExpr [LONG] [4]
   ]
 Orderby
-  FieldAccessor [
-    Variable [ Name=$gen0 ]
-    Field=partkey
-  ]
+  Variable [ Name=#1 ]
   ASC
-  FieldAccessor [
-    Variable [ Name=$j ]
-    Field=l_shipdate
-  ]
+  Variable [ Name=#2 ]
   ASC
-  FieldAccessor [
-    Variable [ Name=$j ]
-    Field=l_orderkey
-  ]
+  Variable [ Name=#3 ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/any-collection-member_01/any-collection-member_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/any-collection-member_01/any-collection-member_01.3.ast
index b96b43b..fa9bf57 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/any-collection-member_01/any-collection-member_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/any-collection-member_01/any-collection-member_01.3.ast
@@ -6,5 +6,5 @@
     LiteralExpr [LONG] [1]
     LiteralExpr [LONG] [1]
   ]
-  Index: ANY
+  Index:   LiteralExpr [LONG] [0]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_double/divide_double.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_double/divide_double.1.ast
index a2c5adc..67d0694 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_double/divide_double.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_double/divide_double.1.ast
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_float/divide_float.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_float/divide_float.1.ast
index 928fc8e..958937c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_float/divide_float.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_float/divide_float.1.ast
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.1.ast
index aa1ac9a..21f2d7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.1.ast
@@ -82,8 +82,8 @@
     LiteralExpr [STRING] [result7]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       LiteralExpr [NULL]
@@ -93,8 +93,8 @@
     LiteralExpr [STRING] [result8]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       FieldAccessor [
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
new file mode 100644
index 0000000..f92b4c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
@@ -0,0 +1,118 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [1]
+      ]
+      div
+      LiteralExpr [LONG] [0]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.1.ast
index 3b24726..37785cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.1.ast
@@ -82,8 +82,8 @@
     LiteralExpr [STRING] [result7]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       LiteralExpr [NULL]
@@ -93,8 +93,8 @@
     LiteralExpr [STRING] [result8]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       FieldAccessor [
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
new file mode 100644
index 0000000..75eb726
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
@@ -0,0 +1,118 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [1]
+      ]
+      div
+      LiteralExpr [LONG] [0]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.1.ast
index 1570f6b..22a3401 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.1.ast
@@ -82,8 +82,8 @@
     LiteralExpr [STRING] [result7]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       LiteralExpr [NULL]
@@ -93,8 +93,8 @@
     LiteralExpr [STRING] [result8]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       FieldAccessor [
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
new file mode 100644
index 0000000..8f295f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
@@ -0,0 +1,118 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [1]
+      ]
+      div
+      LiteralExpr [LONG] [0]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.1.ast
index 96b5942..2243ace 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.1.ast
@@ -82,8 +82,8 @@
     LiteralExpr [STRING] [result7]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       LiteralExpr [NULL]
@@ -93,8 +93,8 @@
     LiteralExpr [STRING] [result8]
     :
     OperatorExpr [
-      FunctionCall null.double@1[
-        LiteralExpr [STRING] [-6.5d]
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [-6]
       ]
       /
       FieldAccessor [
@@ -104,4 +104,15 @@
       ]
     ]
   )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [1]
+      ]
+      /
+      LiteralExpr [LONG] [0]
+    ]
+  )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
new file mode 100644
index 0000000..f1e7dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
@@ -0,0 +1,118 @@
+Query:
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [result1]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int16@1[
+        LiteralExpr [STRING] [2]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result3]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int32@1[
+        LiteralExpr [STRING] [+3]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result4]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.int64@1[
+        LiteralExpr [STRING] [-4]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result5]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.float@1[
+        LiteralExpr [STRING] [-5.5f]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result6]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [+1]
+      ]
+      div
+      FunctionCall null.double@1[
+        LiteralExpr [STRING] [-6.5d]
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result7]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      LiteralExpr [NULL]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result8]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [-6]
+      ]
+      div
+      FieldAccessor [
+        RecordConstructor [
+        ]
+        Field=a
+      ]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result9]
+    :
+    OperatorExpr [
+      FunctionCall null.int8@1[
+        LiteralExpr [STRING] [1]
+      ]
+      div
+      LiteralExpr [LONG] [0]
+    ]
+  )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
index a86d0a4..9d1c274 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
@@ -37,7 +37,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      - LiteralExpr [LONG] [1]
+      - LiteralExpr [LONG] [6]
     ]
   )
   (
@@ -63,7 +63,7 @@
           FunctionCall null.string-length@1[
             LiteralExpr [STRING] [UC Irvine]
           ]
-          /
+          div
           LiteralExpr [LONG] [2]
         ]
         -
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
index f85b949..1727ec1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
@@ -56,6 +56,21 @@
         LiteralExpr [STRING] [Irvine]
       ]
     ]
+    FunctionCall null.substring@3[
+      LiteralExpr [STRING] [ABCD]
+      - LiteralExpr [LONG] [3]
+      LiteralExpr [LONG] [2]
+    ]
+    FunctionCall null.substring@3[
+      LiteralExpr [STRING] [ABCD]
+      - LiteralExpr [LONG] [10]
+      LiteralExpr [LONG] [1]
+    ]
+    FunctionCall null.substring@3[
+      LiteralExpr [STRING] [ABCD]
+      LiteralExpr [LONG] [1]
+      - LiteralExpr [LONG] [1]
+    ]
   ]
   AS Variable [ Name=$a ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
index 414ad6e..eb5857e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
@@ -6,7 +6,15 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      - LiteralExpr [LONG] [1]
+      - LiteralExpr [LONG] [3]
+    ]
+  )
+  (
+    LiteralExpr [STRING] [result2]
+    :
+    FunctionCall test.substring@2[
+      LiteralExpr [STRING] [HEllow]
+      - LiteralExpr [LONG] [7]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
index cdd8a53..92376cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
@@ -1,23 +1,11 @@
 DataverseUse TinySocial
 Query:
 SELECT [
-FieldAccessor [
-  Variable [ Name=$user ]
-  Field=name
-]
+Variable [ Name=#1 ]
 uname
-FieldAccessor [
-  Variable [ Name=$user ]
-  Field=alias
-]
+Variable [ Name=#2 ]
 alias
-FunctionCall TinySocial.substring@2[
-  FieldAccessor [
-    Variable [ Name=$message ]
-    Field=message
-  ]
-  LiteralExpr [LONG] [29]
-]
+Variable [ Name=#3 ]
 msg
 ]
 FROM [  FunctionCall asterix.dataset@1[
@@ -30,6 +18,27 @@
   ]
   AS Variable [ Name=$user ]
 ]
+Let Variable [ Name=#1 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$user ]
+    Field=name
+  ]
+Let Variable [ Name=#2 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$user ]
+    Field=alias
+  ]
+Let Variable [ Name=#3 ]
+  :=
+  FunctionCall TinySocial.substring@2[
+    FieldAccessor [
+      Variable [ Name=$message ]
+      Field=message
+    ]
+    LiteralExpr [LONG] [29]
+  ]
 Where
   OperatorExpr [
     OperatorExpr [
@@ -63,22 +72,10 @@
     ]
   ]
 Orderby
-  FieldAccessor [
-    Variable [ Name=$user ]
-    Field=name
-  ]
+  Variable [ Name=#1 ]
   ASC
-  FieldAccessor [
-    Variable [ Name=$user ]
-    Field=alias
-  ]
+  Variable [ Name=#2 ]
   ASC
-  FunctionCall TinySocial.substring@2[
-    FieldAccessor [
-      Variable [ Name=$message ]
-      Field=message
-    ]
-    LiteralExpr [LONG] [29]
-  ]
+  Variable [ Name=#3 ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
index cdd8a53..92376cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
@@ -1,23 +1,11 @@
 DataverseUse TinySocial
 Query:
 SELECT [
-FieldAccessor [
-  Variable [ Name=$user ]
-  Field=name
-]
+Variable [ Name=#1 ]
 uname
-FieldAccessor [
-  Variable [ Name=$user ]
-  Field=alias
-]
+Variable [ Name=#2 ]
 alias
-FunctionCall TinySocial.substring@2[
-  FieldAccessor [
-    Variable [ Name=$message ]
-    Field=message
-  ]
-  LiteralExpr [LONG] [29]
-]
+Variable [ Name=#3 ]
 msg
 ]
 FROM [  FunctionCall asterix.dataset@1[
@@ -30,6 +18,27 @@
   ]
   AS Variable [ Name=$user ]
 ]
+Let Variable [ Name=#1 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$user ]
+    Field=name
+  ]
+Let Variable [ Name=#2 ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$user ]
+    Field=alias
+  ]
+Let Variable [ Name=#3 ]
+  :=
+  FunctionCall TinySocial.substring@2[
+    FieldAccessor [
+      Variable [ Name=$message ]
+      Field=message
+    ]
+    LiteralExpr [LONG] [29]
+  ]
 Where
   OperatorExpr [
     OperatorExpr [
@@ -63,22 +72,10 @@
     ]
   ]
 Orderby
-  FieldAccessor [
-    Variable [ Name=$user ]
-    Field=name
-  ]
+  Variable [ Name=#1 ]
   ASC
-  FieldAccessor [
-    Variable [ Name=$user ]
-    Field=alias
-  ]
+  Variable [ Name=#2 ]
   ASC
-  FunctionCall TinySocial.substring@2[
-    FieldAccessor [
-      Variable [ Name=$message ]
-      Field=message
-    ]
-    LiteralExpr [LONG] [29]
-  ]
+  Variable [ Name=#3 ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 5c6b100..91becf7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -39,38 +39,6 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="external-indexing">
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="text-format">
-        <output-dir compare="Text">text-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="sequence-format">
-        <output-dir compare="Text">sequence-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rc-format">
-        <output-dir compare="Text">rc-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rtree-index">
-        <output-dir compare="Text">rtree-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin">
-        <output-dir compare="Text">leftouterjoin</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin-rtree">
-        <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="upsert">
     <test-case FilePath="upsert">
       <compilation-unit name="filtered-dataset">
@@ -1668,7 +1636,7 @@
     <test-case FilePath="dml">
       <compilation-unit name="insert-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>Duplicate field name &quot;id&quot;</expected-error>
+        <expected-error>ASX1006: Duplicate field name &quot;id&quot;</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1689,13 +1657,13 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_03">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
-        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1716,7 +1684,7 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-no-field">
         <output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
-        <expected-error>ASX1014: Field "not_id" is not found [CompilationException]</expected-error>
+        <expected-error>ASX1014: Field "not_id" is not found</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -2508,7 +2476,7 @@
     <test-case FilePath="fuzzyjoin">
       <compilation-unit name="dblp-aqlplus_2">
         <output-dir compare="Text">dblp-aqlplus_2</output-dir>
-        <expected-error>Type mismatch: function similarity-jaccard expects its 1st input parameter to be type array or multiset, but the actual input type is string</expected-error>
+        <expected-error>Type mismatch: function similarity-jaccard expects its 1st input parameter to be of type array or multiset, but the actual input type is string</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="fuzzyjoin">
@@ -3545,28 +3513,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="open-index-enforced/external-indexing">
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
     <test-group name="open-index-enforced/type-checking">
       <test-case FilePath="open-index-enforced/type-checking">
         <compilation-unit name="enforced-type-delete">
@@ -3727,28 +3673,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="nested-open-index/external-indexing">
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
     <test-group name="highly-open-highly-nested">
       <test-case FilePath="nested-open-index/highly-open-highly-nested">
         <compilation-unit name="bottom-closed-top-closed">
@@ -3979,28 +3903,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="nested-index/external-indexing">
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
   </test-group>
   <test-group name="nested-index-dml">
     <test-case FilePath="nested-index-dml">
@@ -4477,7 +4379,7 @@
     <test-case FilePath="numeric">
       <compilation-unit name="query-issue355">
         <output-dir compare="Text">query-issue355</output-dir>
-        <expected-error>java.lang.NumberFormatException: For input string: "10000000000000000000"</expected-error>
+        <expected-error>For input string: "10000000000000000000"</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -4923,7 +4825,7 @@
         <output-dir compare="Text">alltypes_01</output-dir>
       </compilation-unit>
     </test-case>
-    <test-case FilePath="scan">
+    <!--test-case FilePath="scan">
       <compilation-unit name="alltypes_01">
         <parameter name="wrapper-array" value="true" />
         <output-dir compare="Text">alltypes_01-wrapped</output-dir>
@@ -4944,7 +4846,7 @@
       <compilation-unit name="alltypes_01">
         <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
       </compilation-unit>
-    </test-case>
+    </test-case-->
     <test-case FilePath="scan">
       <compilation-unit name="alltypes_02">
         <output-dir compare="Text">alltypes_02</output-dir>
@@ -6525,19 +6427,19 @@
     <test-case FilePath="load">
       <compilation-unit name="csv_05"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_05</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_06"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_06</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_07"><!-- Someone should check and verify -->
         <output-dir compare="Text">csv_07</output-dir>
-        <expected-error>java.io.IOException: At record:</expected-error>
+        <expected-error>At record: 1</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -6575,7 +6477,7 @@
     <test-case FilePath="load">
       <compilation-unit name="issue650_query">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Unable to load dataset Users since it does not exist</expected-error>
+        <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -6639,43 +6541,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="hints">
-      <compilation-unit name="issue_251_dataset_hint_6">
-        <output-dir compare="Text">issue_251_dataset_hint_6</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hints">
       <compilation-unit name="issue_251_dataset_hint_7">
         <output-dir compare="Text">issue_251_dataset_hint_7</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="hdfs">
-    <test-case FilePath="hdfs">
-      <compilation-unit name="large-record">
-        <output-dir compare="Text">large-record</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_shortcircuit">
-        <output-dir compare="Text">hdfs_shortcircuit</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="issue_245_hdfs">
-        <output-dir compare="Text">issue_245_hdfs</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_02">
-        <output-dir compare="Text">hdfs_02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_03">
-        <output-dir compare="Text">hdfs_03</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="big-object">
     <test-case FilePath="big-object">
       <compilation-unit name="big_object_sort">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 2ee294a..3ab1ff9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -19,8 +19,9 @@
 <!DOCTYPE test-suite [
   <!ENTITY ObjectsQueries SYSTEM "queries_sqlpp/objects/ObjectsQueries.xml">
   <!ENTITY AsyncDeferredQueries SYSTEM "queries_sqlpp/async-deferred/AsyncDeferredQueries.xml">
+  <!ENTITY GeoQueries SYSTEM "queries_sqlpp/geojson/GeoJSONQueries.xml">
 ]>
-<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
   &AsyncDeferredQueries;
   <test-group name="flwor">
     <test-case FilePath="flwor">
@@ -59,6 +60,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="flwor">
+      <compilation-unit name="order-by-13">
+        <output-dir compare="Text">order-by-13</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
       <compilation-unit name="query-issue550">
         <output-dir compare="Text">query-issue550</output-dir>
       </compilation-unit>
@@ -86,6 +92,26 @@
         <expected-error>Duplicate alias definitions: s1</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-2446">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>ASX0013: Duplicate field name "a"</expected-error>
+        <expected-error>ASX0013: Duplicate field name "c"</expected-error>
+        <expected-error>ASX0013: Duplicate field name "e"</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-2446-2">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>ASX0013: Duplicate field name "a" (in line 27, at column 20)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "b" (in line 27, at column 20)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "c" (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "d" (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "e" (in line 27, at column 14)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "f" (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name "g" (in line 27, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="explain">
     <test-case FilePath="explain">
@@ -691,6 +717,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue2411">
+        <output-dir compare="Text">issue2411</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
       <compilation-unit name="avg_double">
         <output-dir compare="Text">avg_double</output-dir>
       </compilation-unit>
@@ -981,6 +1012,139 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="array_fun">
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_prepend">
+        <output-dir compare="Text">array_prepend</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_append">
+        <output-dir compare="Text">array_append</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_position">
+        <output-dir compare="Text">array_position</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_repeat">
+        <output-dir compare="Text">array_repeat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_reverse">
+        <output-dir compare="Text">array_reverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_contains">
+        <output-dir compare="Text">array_contains</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_insert">
+        <output-dir compare="Text">array_insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_put">
+        <output-dir compare="Text">array_put</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_remove">
+        <output-dir compare="Text">array_remove</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_distinct">
+        <output-dir compare="Text">array_distinct</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_sort">
+        <output-dir compare="Text">array_sort</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_concat">
+        <output-dir compare="Text">array_concat</output-dir>
+        <expected-error>Input contains different list types (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_flatten">
+        <output-dir compare="Text">array_flatten</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_ifnull">
+        <output-dir compare="Text">array_ifnull</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_intersect">
+        <output-dir compare="Text">array_intersect</output-dir>
+        <expected-error>Input contains different list types (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_range">
+        <output-dir compare="Text">array_range</output-dir>
+        <expected-error>Invalid number of arguments for function array-range (in line 27, at column 8)</expected-error>
+        <expected-error>Invalid number of arguments for function array-range (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_replace">
+        <output-dir compare="Text">array_replace</output-dir>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+        <expected-error>Invalid number of arguments for function array-replace (in line 27, at column 8)</expected-error>
+        <expected-error>Invalid number of arguments for function array-replace (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_star">
+        <output-dir compare="Text">array_star</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_symdiff">
+        <output-dir compare="Text">array_symdiff</output-dir>
+        <expected-error>Input contains different list types (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_symdiffn">
+        <output-dir compare="Text">array_symdiffn</output-dir>
+        <expected-error>Input contains different list types (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_union">
+        <output-dir compare="Text">array_union</output-dir>
+        <expected-error>Input contains different list types (in line 27, at column 8)</expected-error>
+        <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="boolean">
     <test-case FilePath="boolean">
       <compilation-unit name="and_01">
@@ -1005,6 +1169,11 @@
   </test-group>
   <test-group name="comparison">
     <test-case FilePath="comparison">
+      <compilation-unit name="secondary_idx_lookup">
+        <output-dir compare="Text">secondary_idx_lookup</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
       <compilation-unit name="year_month_duration_order">
         <output-dir compare="Text">year_month_duration_order</output-dir>
       </compilation-unit>
@@ -1263,6 +1432,32 @@
         <output-dir compare="Text">least_mixed</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="missingif">
+        <output-dir compare="Text">missingif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="nullif">
+        <output-dir compare="Text">nullif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="nanif">
+        <output-dir compare="Text">nanif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="posinfif">
+        <output-dir compare="Text">posinfif</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="neginfif">
+        <output-dir compare="Text">neginfif</output-dir>
+      </compilation-unit>
+    </test-case>
+
   </test-group>
   <test-group name="constructor">
     <test-case FilePath="constructor">
@@ -1499,7 +1694,7 @@
     <test-case FilePath="custord">
       <compilation-unit name="join_q_09">
         <output-dir compare="Text">join_q_01</output-dir>
-        <expected-error>Cannot resolve ambiguous alias reference for undefined identifier age</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier age</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="custord">
@@ -1568,7 +1763,7 @@
     <test-case FilePath="dapd">
       <compilation-unit name="q2-3">
         <output-dir compare="Text">q2</output-dir>
-        <expected-error>Cannot resolve ambiguous alias reference for undefined identifier sig_id</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier sig_id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dapd">
@@ -1826,6 +2021,7 @@
       <compilation-unit name="insert-duplicated-keys">
         <output-dir compare="Text">insert-duplicated-keys</output-dir>
         <expected-error>Inserting duplicate keys into the primary storage</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1874,13 +2070,13 @@
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_02">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
-        <expected-error>Parse error at (0, 5): ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
       <compilation-unit name="load-with-autogenerated-pk_adm_03">
         <output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
-        <expected-error>Parse error at (0, 5): ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
+        <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -1920,6 +2116,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname-qualified">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
       <compilation-unit name="insert-returning-fieldname-implicit">
         <output-dir compare="Text">insert-returning-fieldname</output-dir>
         <expected-error>Need an alias for the enclosed expression</expected-error>
@@ -2765,6 +2966,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-09">
+        <output-dir compare="Text">fulltext-index-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
       <compilation-unit name="fulltext-index-large-data">
         <output-dir compare="Text">fulltext-index-large-data</output-dir>
       </compilation-unit>
@@ -2794,13 +3000,13 @@
     <test-case FilePath="global-aggregate">
       <compilation-unit name="q05_error">
         <output-dir compare="Text">q01</output-dir>
-        <expected-error>Type mismatch: function field-access-by-name expects its 1st input parameter to be type object, but the actual input type is array</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type object, but got the value of type array (in line 22, at column 8)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="global-aggregate">
       <compilation-unit name="q06_error">
         <output-dir compare="Text">q01</output-dir>
-        <expected-error>Type mismatch: function scan-collection expects its 1st input parameter to be type array or multiset, but the actual input type is string</expected-error>
+        <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string (in line 22, at column 8)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="global-aggregate">
@@ -2897,7 +3103,7 @@
     <test-case FilePath="group-by">
       <compilation-unit name="sugar-01-negative">
         <output-dir compare="Text">core-01</output-dir>
-        <expected-error>Type mismatch: function field-access-by-name expects its 1st input parameter to be type object, but the actual input type is array</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type object, but got the value of type array (in line 26, at column 38)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
@@ -2908,13 +3114,13 @@
     <test-case FilePath="group-by">
       <compilation-unit name="sugar-02">
         <output-dir compare="Text">core-02</output-dir>
-        <expected-error>Cannot resolve ambiguous alias reference for undefined identifier deptId</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier deptId</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
       <compilation-unit name="sugar-02-2">
         <output-dir compare="Text">core-02</output-dir>
-        <expected-error>Cannot resolve ambiguous alias reference for undefined identifier deptId</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier deptId</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
@@ -2953,6 +3159,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
+      <compilation-unit name="sugar-07-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier IDENT</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
       <compilation-unit name="null">
         <output-dir compare="Text">null</output-dir>
       </compilation-unit>
@@ -3175,6 +3387,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="inverted-index-ngram-contains">
         <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
       </compilation-unit>
@@ -3554,6 +3771,16 @@
   </test-group>
   <test-group name="misc">
     <test-case FilePath="misc">
+      <compilation-unit name="comp-ASTERIXDB-2415">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="comp-ASTERIXDB-2412">
+        <output-dir compare="Text">comp-ASTERIXDB-2412</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="poll-dynamic">
         <output-dir compare="Text">poll-dynamic</output-dir>
       </compilation-unit>
@@ -3569,6 +3796,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="ping">
+        <output-dir compare="Text">ping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="case_01">
         <output-dir compare="Text">case_01</output-dir>
       </compilation-unit>
@@ -3604,6 +3836,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="case_08">
+        <output-dir compare="Text">case_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="dataset_nodegroup">
         <output-dir compare="Text">dataset_nodegroup</output-dir>
       </compilation-unit>
@@ -3612,7 +3849,7 @@
       <compilation-unit name="partition-by-nonexistent-field">
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
         <expected-error>Field "id" is not found</expected-error>
-        <expected-error>Cannot find dataset testds in dataverse test</expected-error>
+        <expected-error>Cannot find dataset with name testds in dataverse test</expected-error>
         <expected-error>Cannot find dataset testds in dataverse test nor an alias with name testds!</expected-error>
       </compilation-unit>
     </test-case>
@@ -3657,6 +3894,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="random">
+        <output-dir compare="Text">random</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="stable_sort">
         <output-dir compare="Text">stable_sort</output-dir>
       </compilation-unit>
@@ -3732,6 +3974,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2380">
+        <output-dir compare="Text">query-ASTERIXDB-2380</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="unsupported_parameter">
         <output-dir compare="Text">none</output-dir>
         <expected-error>Query parameter compiler.joinmem is not supported</expected-error>
@@ -3948,28 +4195,6 @@
         </compilation-unit>
       </test-case>
     </test-group>
-    <test-group name="open-index-enforced/external-indexing">
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="open-index-enforced/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-        </compilation-unit>
-      </test-case>
-    </test-group>
     <test-group name="open-index-enforced/type-checking">
       <test-case FilePath="open-index-enforced/type-checking">
         <compilation-unit name="enforced-type-delete">
@@ -4059,6 +4284,16 @@
           <output-dir compare="Text">word-jaccard-inline</output-dir>
         </compilation-unit>
       </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+        </compilation-unit>
+      </test-case>
     </test-group>
     <test-group name="nested-open-index/index-leftouterjoin">
       <test-case FilePath="nested-open-index/index-leftouterjoin">
@@ -4163,26 +4398,24 @@
           <output-dir compare="Text">rtree-secondary-index</output-dir>
         </compilation-unit>
       </test-case>
-    </test-group>
-    <test-group name="nested-open-index/external-indexing">
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-01">
+          <output-dir compare="Text">non-enforced-01</output-dir>
         </compilation-unit>
       </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-02">
+          <output-dir compare="Text">non-enforced-02</output-dir>
         </compilation-unit>
       </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-03">
+          <output-dir compare="Text">non-enforced-03</output-dir>
         </compilation-unit>
       </test-case>
-      <test-case FilePath="nested-open-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-04">
+          <output-dir compare="Text">non-enforced-04</output-dir>
         </compilation-unit>
       </test-case>
     </test-group>
@@ -4420,26 +4653,9 @@
           <output-dir compare="Text">rtree-secondary-index-open</output-dir>
         </compilation-unit>
       </test-case>
-    </test-group>
-    <test-group name="nested-index/external-indexing">
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="adm-format">
-          <output-dir compare="Text">adm-format</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="rtree-index">
-          <output-dir compare="Text">rtree-index</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin">
-          <output-dir compare="Text">leftouterjoin</output-dir>
-        </compilation-unit>
-      </test-case>
-      <test-case FilePath="nested-index/external-indexing">
-        <compilation-unit name="leftouterjoin-rtree">
-          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="rtree-secondary-index-optional">
+          <output-dir compare="Text">rtree-secondary-index-open</output-dir>
         </compilation-unit>
       </test-case>
     </test-group>
@@ -4602,6 +4818,12 @@
         <output-dir compare="Text">nestrecord</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="nestrecords">
+      <compilation-unit name="nested-optional-pk">
+        <output-dir compare="Text">nested-optional-pk</output-dir>
+        <expected-error>ASX1021: The primary key field "nested.id" cannot be nullable</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="null-missing">
     <test-case FilePath="null-missing">
@@ -4802,6 +5024,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="numeric">
+      <compilation-unit name="const">
+        <output-dir compare="Text">const</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="degrees">
+        <output-dir compare="Text">degrees</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
       <compilation-unit name="divide_double">
         <output-dir compare="Text">divide_double</output-dir>
       </compilation-unit>
@@ -4912,6 +5144,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="numeric">
+      <compilation-unit name="radians">
+        <output-dir compare="Text">radians</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
       <compilation-unit name="round-half-to-even0">
         <output-dir compare="Text">round-half-to-even0</output-dir>
       </compilation-unit>
@@ -5363,6 +5600,7 @@
       <compilation-unit name="query-issue410">
         <output-dir compare="Text">query-issue410</output-dir>
         <expected-error>Field type double can't be promoted to type string</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="open-closed">
@@ -5559,7 +5797,7 @@
     <test-case FilePath="resolution">
       <compilation-unit name="conflict-fields-dataset">
         <output-dir compare="Text">conflict-field-dataset</output-dir>
-        <expected-error>Cannot resolve ambiguous alias reference for undefined identifier samptable</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier samptable</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="resolution">
@@ -5699,8 +5937,7 @@
     </test-case>
     <test-case FilePath="select-star">
       <compilation-unit name="var_star_2">
-        <output-dir compare="Text">var_star</output-dir>
-        <expected-error>ASX0002: Type mismatch</expected-error>
+        <output-dir compare="Text">var_star_2</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -5974,6 +6211,71 @@
         <output-dir compare="Text">spatial-area</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial-large-data">
+        <output-dir compare="Text">spatial-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="statement-params">
+    <test-case FilePath="statement-params">
+      <compilation-unit name="mixed_01">
+        <output-dir compare="Text">mixed_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_01">
+        <output-dir compare="Text">named_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_02">
+        <output-dir compare="Text">named_01</output-dir>
+        <expected-error>ASX1086: No value for parameter: $p2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_03">
+        <output-dir compare="Text">named_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_04">
+        <output-dir compare="Text">named_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_01">
+        <output-dir compare="Text">positional_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_02">
+        <output-dir compare="Text">positional_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_03">
+        <output-dir compare="Text">positional_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_04">
+        <output-dir compare="Text">positional_02</output-dir>
+        <expected-error>ASX1086: No value for parameter: $2</expected-error>
+        <expected-error>ASX1086: No value for parameter: $3</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_05">
+        <output-dir compare="Text">positional_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="query-ASTERIXDB-2413">
+        <output-dir compare="Text">query-ASTERIXDB-2413</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="string">
     <test-case FilePath="string">
@@ -6238,6 +6540,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="string">
+      <compilation-unit name="reverse">
+        <output-dir compare="Text">reverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
       <compilation-unit name="rtrim">
         <output-dir compare="Text">rtrim</output-dir>
       </compilation-unit>
@@ -6288,6 +6595,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="string">
+      <compilation-unit name="string-concat2">
+        <output-dir compare="Text">string-concat2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
       <compilation-unit name="string-equal1">
         <output-dir compare="Text">string-equal1</output-dir>
       </compilation-unit>
@@ -6500,11 +6812,6 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="subquery">
-      <compilation-unit name="decorrelate_with_unique_id_2">
-        <output-dir compare="Text">decorrelate_with_unique_id_2</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="subquery">
       <compilation-unit name="exists">
         <output-dir compare="Text">exists</output-dir>
       </compilation-unit>
@@ -6515,6 +6822,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="subquery">
+      <compilation-unit name="in_as_or">
+        <output-dir compare="Text">in_as_or</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
       <compilation-unit name="in_correlated">
         <output-dir compare="Text">in</output-dir>
       </compilation-unit>
@@ -6584,7 +6896,7 @@
     <test-case FilePath="subquery">
       <compilation-unit name="query-ASTERIXDB-1574">
         <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
-        <expected-error>Type mismatch: function scan-collection expects its 1st input parameter to be type multiset or array, but the actual input type is object</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type multiset or array, but got the value of type object (in line 25, at column 41)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="subquery">
@@ -8074,36 +8386,42 @@
       <compilation-unit name="drop-dependency-1">
         <output-dir compare="Text">drop-dependency-1</output-dir>
         <expected-error>Cannot drop dataverse. Function B.f0@2 depends on function C.f1@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-2">
         <output-dir compare="Text">drop-dependency-2</output-dir>
         <expected-error>Cannot drop dataverse. Function B.f2@2 depends on dataset C.TweetMessages</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-3">
         <output-dir compare="Text">drop-dependency-3</output-dir>
         <expected-error>Cannot drop function C.f1@2 being used by function B.f0@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-4">
         <output-dir compare="Text">drop-dependency-4</output-dir>
         <expected-error>Cannot drop dataset C.TweetMessages being used by function B.f2@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-5">
         <output-dir compare="Text">drop-dependency-5</output-dir>
         <expected-error>Cannot drop function C.f1@2 being used by function C.f0@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
       <compilation-unit name="drop-dependency-6">
         <output-dir compare="Text">drop-dependency-6</output-dir>
         <expected-error>Cannot drop dataset C.TweetMessages being used by function C.f2@2</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
@@ -8348,18 +8666,21 @@
       <compilation-unit name="csv_05">
         <output-dir compare="Text">csv_05</output-dir>
         <expected-error>At record: 1, field#: 4 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_06">
         <output-dir compare="Text">csv_06</output-dir>
         <expected-error>At record: 1, field#: 3 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="csv_07">
         <output-dir compare="Text">csv_07</output-dir>
         <expected-error>At record: 1, field#: 3 -  A quote enclosing a field needs to be followed by the delimiter</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8381,12 +8702,14 @@
       <compilation-unit name="issue14_query">
         <output-dir compare="Text">issue14_query</output-dir>
         <expected-error>Unspecified parameter: format</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
       <compilation-unit name="issue315_query">
         <output-dir compare="Text">none</output-dir>
         <expected-error>Invalid path</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8397,7 +8720,7 @@
     <test-case FilePath="load">
       <compilation-unit name="issue650_query">
         <output-dir compare="Text">none</output-dir>
-        <expected-error>Unable to load dataset Users since it does not exist</expected-error>
+        <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8430,6 +8753,7 @@
       <compilation-unit name="duplicate-key-error">
         <output-dir compare="Text">none</output-dir>
         <expected-error>Loading duplicate keys into the primary storage</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="load">
@@ -8450,16 +8774,23 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="hints">
-      <compilation-unit name="issue_251_dataset_hint_6">
-        <output-dir compare="Text">issue_251_dataset_hint_6</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hints">
       <compilation-unit name="issue_251_dataset_hint_7">
         <output-dir compare="Text">issue_251_dataset_hint_7</output-dir>
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="function">
+    <test-case FilePath="function">
+      <compilation-unit name="issue-2394">
+        <output-dir compare="Text">issue-2394</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="function">
+      <compilation-unit name="drop_if_exists">
+        <output-dir compare="Text">drop_if_exists</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="feeds">
     <test-case FilePath="feeds">
       <compilation-unit name="feeds_07">
@@ -8532,8 +8863,14 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-index">
+        <output-dir compare="Text">change-feed-with-meta-pk-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-with-mixed-index">
         <output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 1)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8549,7 +8886,7 @@
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-open-index-in-meta">
         <output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
-        <expected-error>Typed open index can only be created on the record part</expected-error>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8560,11 +8897,13 @@
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
         <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 1)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
       <compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
         <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8592,6 +8931,7 @@
         <output-dir compare="Text">twitter-feed</output-dir>
         <expected-error>Twitter4J library not found!</expected-error>
         <expected-error>Unknown source feed: TwitterFeed</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8599,6 +8939,7 @@
         <output-dir compare="Text">revised-tweet-parser</output-dir>
         <expected-error>Twitter4J library not found!</expected-error>
         <expected-error>Unknown source feed: TwitterFeed</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8645,12 +8986,14 @@
       <compilation-unit name="start-started-feed">
         <output-dir compare="Text">start-started-feed</output-dir>
         <expected-error>experiments.TweetFeed(Feed) is already started</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
       <compilation-unit name="stop-stopped-feed">
         <output-dir compare="Text">stop-stopped-feed</output-dir>
         <expected-error>new_experiments.TweetFeed(Feed) cannot be stopped because its state is STOPPED</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
@@ -8699,28 +9042,6 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="hdfs">
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_shortcircuit">
-        <output-dir compare="Text">hdfs_shortcircuit</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="issue_245_hdfs">
-        <output-dir compare="Text">issue_245_hdfs</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_02">
-        <output-dir compare="Text">hdfs_02</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="hdfs">
-      <compilation-unit name="hdfs_03">
-        <output-dir compare="Text">hdfs_03</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="big-object">
     <test-case FilePath="big-object">
       <compilation-unit name="big_object_sort">
@@ -8748,38 +9069,6 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="external-indexing">
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="text-format">
-        <output-dir compare="Text">text-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="sequence-format">
-        <output-dir compare="Text">sequence-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rc-format">
-        <output-dir compare="Text">rc-format</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="rtree-index">
-        <output-dir compare="Text">rtree-index</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin">
-        <output-dir compare="Text">leftouterjoin</output-dir>
-      </compilation-unit>
-    </test-case>
-    <test-case FilePath="external-indexing">
-      <compilation-unit name="leftouterjoin-rtree">
-        <output-dir compare="Text">leftouterjoin-rtree</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
   <test-group name="temporal">
     <test-case FilePath="temporal">
       <compilation-unit name="overlap_bins_gby_3">
@@ -9330,6 +9619,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="filters">
+      <compilation-unit name="filter-auto-key">
+        <output-dir compare="Text">filter-auto-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
       <compilation-unit name="load">
         <output-dir compare="Text">load</output-dir>
       </compilation-unit>
@@ -9463,6 +9757,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="binary">
+      <compilation-unit name="concat2">
+        <output-dir compare="Text">concat2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
       <compilation-unit name="subbinary">
         <output-dir compare="Text">subbinary</output-dir>
       </compilation-unit>
@@ -9640,6 +9939,11 @@
         <output-dir compare="Text">multiple-secondaries</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="upsert-case-returning">
+        <output-dir compare="Text">upsert-case-returning</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="json-parser">
     <test-case FilePath="json-parser">
@@ -9651,6 +9955,7 @@
       <compilation-unit name="numeric-tinyint-overflow">
         <output-dir compare="Text">numeric-tinyint</output-dir>
         <expected-error>Numeric value (1000) out of range of Java byte</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9672,12 +9977,14 @@
       <compilation-unit name="duplicate-fields">
         <output-dir compare="Text">duplicate-fields</output-dir>
         <expected-error>Duplicate field 'field'</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="malformed-json">
         <output-dir compare="Text">malformed-json</output-dir>
         <expected-error>Unexpected character ('}' (code 125)): was expecting double-quote to start field name</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9689,12 +9996,14 @@
       <compilation-unit name="nonoptional-missing">
         <output-dir compare="Text">nonoptional-missing</output-dir>
         <expected-error>ASX3075: Closed field missing_value has null value</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="nonoptional-null">
         <output-dir compare="Text">nonoptional-null</output-dir>
         <expected-error>ASX3075: Closed field null_value has null value</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9706,18 +10015,21 @@
       <compilation-unit name="spatial-line-3-points">
         <output-dir compare="Text">spatial-line-3-points</output-dir>
         <expected-error>Line must have 4 coordinates</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="spatial-polygon-unclosed">
         <output-dir compare="Text">spatial-polygon-unclosed</output-dir>
         <expected-error>Unclosed polygon is not supported</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="spatial-polygon-with-hole">
         <output-dir compare="Text">spatial-polygon-with-hole</output-dir>
         <expected-error>Only simple geometries are supported (Point, LineString and Polygon without holes)</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
@@ -9729,48 +10041,56 @@
       <compilation-unit name="type-mismatch">
         <output-dir compare="Text">type-mismatch</output-dir>
         <expected-error>ASX3054: Mismatch Type, expecting a value of type string</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-circle">
         <output-dir compare="Text">unsupported-type-circle</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type circle</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-daytimeduration">
         <output-dir compare="Text">unsupported-type-daytimeduration</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type daytimeduration</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-duration">
         <output-dir compare="Text">unsupported-type-duration</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type duration</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-interval">
         <output-dir compare="Text">unsupported-type-interval</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type interval</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-multiset">
         <output-dir compare="Text">unsupported-type-multiset</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type multiset</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-point3d">
         <output-dir compare="Text">unsupported-type-point3d</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type point3d</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="json-parser">
       <compilation-unit name="unsupported-type-rectangle">
         <output-dir compare="Text">unsupported-type-rectangle</output-dir>
         <expected-error>ASX0004: Unsupported type: org.apache.asterix.external.parser.factory.JSONDataParserFactory cannot process input type rectangle</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -9796,4 +10116,54 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="limit">
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_negative_value">
+        <output-dir compare="Text">limit_negative_value</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_01">
+        <output-dir compare="Text">limit_type_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_02">
+        <output-dir compare="Text">limit_type_01</output-dir>
+        <expected-error>ASX0039: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
+        <expected-error>ASX0039: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
+        <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-primary-scan">
+        <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-scan-select">
+         <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
+       </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-lookup">
+         <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+       </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+       <compilation-unit name="push-limit-to-primary-lookup-select">
+         <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+       </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="query-ASTERIXDB-2420">
+        <output-dir compare="Text">query-ASTERIXDB-2420</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  &GeoQueries;
 </test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
new file mode 100644
index 0000000..45b36d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -0,0 +1,160 @@
+<!--
+ ! 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.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
+  <test-group name="open-index-enforced">
+    <test-group name="open-index-enforced/external-indexing">
+      <test-case FilePath="open-index-enforced/external-indexing">
+        <compilation-unit name="adm-format">
+          <output-dir compare="Text">adm-format</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/external-indexing">
+        <compilation-unit name="rtree-index">
+          <output-dir compare="Text">rtree-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/external-indexing">
+        <compilation-unit name="leftouterjoin">
+          <output-dir compare="Text">leftouterjoin</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/external-indexing">
+        <compilation-unit name="leftouterjoin-rtree">
+          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="nested-open-index">
+    <test-group name="nested-open-index/external-indexing">
+      <test-case FilePath="nested-open-index/external-indexing">
+        <compilation-unit name="adm-format">
+          <output-dir compare="Text">adm-format</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/external-indexing">
+        <compilation-unit name="rtree-index">
+          <output-dir compare="Text">rtree-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/external-indexing">
+        <compilation-unit name="leftouterjoin">
+          <output-dir compare="Text">leftouterjoin</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/external-indexing">
+        <compilation-unit name="leftouterjoin-rtree">
+          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="nested-index">
+    <test-group name="nested-index/external-indexing">
+      <test-case FilePath="nested-index/external-indexing">
+        <compilation-unit name="adm-format">
+          <output-dir compare="Text">adm-format</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/external-indexing">
+        <compilation-unit name="rtree-index">
+          <output-dir compare="Text">rtree-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/external-indexing">
+        <compilation-unit name="leftouterjoin">
+          <output-dir compare="Text">leftouterjoin</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/external-indexing">
+        <compilation-unit name="leftouterjoin-rtree">
+          <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="subquery">
+    <test-case FilePath="subquery">
+      <compilation-unit name="decorrelate_with_unique_id_2">
+        <output-dir compare="Text">decorrelate_with_unique_id_2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="hints">
+    <test-case FilePath="hints">
+      <compilation-unit name="issue_251_dataset_hint_6">
+        <output-dir compare="Text">issue_251_dataset_hint_6</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="hdfs">
+    <test-case FilePath="hdfs">
+      <compilation-unit name="hdfs_shortcircuit">
+        <output-dir compare="Text">hdfs_shortcircuit</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="hdfs">
+      <compilation-unit name="issue_245_hdfs">
+        <output-dir compare="Text">issue_245_hdfs</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="hdfs">
+      <compilation-unit name="hdfs_02">
+        <output-dir compare="Text">hdfs_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="hdfs">
+      <compilation-unit name="hdfs_03">
+        <output-dir compare="Text">hdfs_03</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="external-indexing">
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="text-format">
+        <output-dir compare="Text">text-format</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="sequence-format">
+        <output-dir compare="Text">sequence-format</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="rc-format">
+        <output-dir compare="Text">rc-format</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="rtree-index">
+        <output-dir compare="Text">rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="leftouterjoin">
+        <output-dir compare="Text">leftouterjoin</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-indexing">
+      <compilation-unit name="leftouterjoin-rtree">
+        <output-dir compare="Text">leftouterjoin-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
index fd6c657..7aa2cf9 100644
--- a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
+++ b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q13.sqlpp
@@ -24,7 +24,7 @@
         SELECT c_custkey, SUM(o_orderkey_count) AS c_count
         FROM  (
                 SELECT c.c_custkey,
-                       COLL_COUNT(
+                       STRICT_COUNT(
                            (
                             select element o.o_orderkey
                             from  Orders o
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index ee7fe5c..fe71443 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -248,7 +248,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -287,5 +286,9 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
new file mode 100644
index 0000000..30759de
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.common.api;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IClientRequest {
+
+    /**
+     * Mark the request as complete, non-cancellable anymore
+     */
+    void complete();
+
+    /**
+     * Cancel a request
+     *
+     * @param appCtx
+     * @throws HyracksDataException
+     */
+    void cancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
new file mode 100644
index 0000000..f879270
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
@@ -0,0 +1,36 @@
+/*
+ * 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.asterix.common.api;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+public interface IDatasetInfoProvider extends Serializable, IJsonSerializable {
+    /**
+     * Get the dataset info
+     *
+     * @param serviceCtx
+     *            the nc service context
+     * @return the dataset info object
+     */
+    DatasetInfo getDatasetInfo(INCServiceContext serviceCtx);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index 4441c6e..d18b6ab 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -40,6 +40,15 @@
     IIndex getIndex(int datasetId, long indexId) throws HyracksDataException;
 
     /**
+     * Indicates if the dataset with id {@code datasetId} is currently registered
+     * with this {@link IDatasetLifecycleManager}
+     *
+     * @param datasetId
+     * @return true if the dataset is currently registered. Otherwise false.
+     */
+    boolean isRegistered(int datasetId);
+
+    /**
      * Flushes all open datasets synchronously.
      *
      * @throws HyracksDataException
@@ -76,18 +85,20 @@
      *
      * @param datasetId
      * @param partition
+     * @param path
      * @return
      */
-    PrimaryIndexOperationTracker getOperationTracker(int datasetId, int partition);
+    PrimaryIndexOperationTracker getOperationTracker(int datasetId, int partition, String path);
 
     /**
      * creates (if necessary) and returns the component Id generator of a dataset.
      *
      * @param datasetId
      * @param partition
+     * @param path
      * @return
      */
-    ILSMComponentIdGenerator getComponentIdGenerator(int datasetId, int partition);
+    ILSMComponentIdGenerator getComponentIdGenerator(int datasetId, int partition, String path);
 
     /**
      * creates (if necessary) and returns the dataset virtual buffer caches.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
index 9551935..7e9879a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
@@ -40,7 +40,11 @@
         /**
          * Extends Language Syntax and Algebric Operations
          */
-        LANG
+        LANG,
+        /**
+         * Extends Recovery Capabilities
+         */
+        RECOVERY
     }
 
     /**
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
new file mode 100644
index 0000000..edde368
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.asterix.common.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.common.IResource;
+
+public interface ILSMComponentIdGeneratorFactory extends Serializable, IJsonSerializable {
+    /**
+     * Get the Id generator for the resource
+     *
+     * @param serviceCtx
+     * @param resource
+     * @return
+     * @throws HyracksDataException
+     */
+    ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
+            throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 9eb6259..8648c5b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -27,12 +27,14 @@
 import org.apache.asterix.common.replication.IReplicationManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.IReplicaManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
@@ -65,7 +67,8 @@
 
     IResourceIdFactory getResourceIdFactory();
 
-    void initialize(boolean initialRun) throws IOException, AlgebricksException;
+    void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun)
+            throws IOException, AlgebricksException;
 
     void setShuttingdown(boolean b);
 
@@ -120,4 +123,6 @@
     IReplicaManager getReplicaManager();
 
     long getMaxTxnId();
+
+    IPersistedResourceRegistry getPersistedResourceRegistry();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index dda9ffd..5e99eec 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.utils.NcLocalCounters;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.config.IOption;
@@ -59,9 +60,10 @@
      *
      * @param nodeId
      * @param active
+     * @param ncLocalCounters
      * @throws HyracksDataException
      */
-    void updateNodePartitions(String nodeId, boolean active) throws HyracksDataException;
+    void updateNodeState(String nodeId, boolean active, NcLocalCounters ncLocalCounters) throws HyracksDataException;
 
     /**
      * Updates the active node and active state of the cluster partition with id {@code partitionNum}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
index 8455a6f..c64c287 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ActiveProperties.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.common.config;
 
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
@@ -32,7 +33,9 @@
         ACTIVE_MEMORY_GLOBAL_BUDGET(
                 LONG_BYTE_UNIT,
                 StorageUtil.getLongSizeInBytes(64L, MEGABYTE),
-                "The memory budget (in bytes) for the active runtime");
+                "The memory budget (in bytes) for the active runtime"),
+        ACTIVE_STOP_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful stop of an active runtime"),
+        ACTIVE_SUSPEND_TIMEOUT(INTEGER, 3600, "The maximum time to wait for a graceful suspend of an active runtime");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -72,4 +75,12 @@
     public long getMemoryComponentGlobalBudget() {
         return accessor.getLong(Option.ACTIVE_MEMORY_GLOBAL_BUDGET);
     }
+
+    public int getActiveStopTimeout() {
+        return accessor.getInt(Option.ACTIVE_STOP_TIMEOUT);
+    }
+
+    public int getActiveSuspendTimeout() {
+        return accessor.getInt(Option.ACTIVE_SUSPEND_TIMEOUT);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
index 4bc1953..714bc92 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
 import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.util.StorageUtil;
 import org.apache.logging.log4j.Level;
 
 public class ExternalProperties extends AbstractProperties {
@@ -42,7 +43,11 @@
                 "The max pending time (in seconds) for cluster startup. After the "
                         + "threshold, if the cluster still is not up and running, it is considered unavailable"),
         CC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the cluster controller process by managix"),
-        NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix");
+        NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix"),
+        MAX_WEB_REQUEST_SIZE(
+                INTEGER,
+                StorageUtil.getIntSizeInBytes(50, StorageUtil.StorageUnit.MEGABYTE),
+                "The maximum accepted web request size in bytes");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -66,6 +71,7 @@
                     return Section.NC;
                 case LOG_LEVEL:
                 case MAX_WAIT_ACTIVE_CLUSTER:
+                case MAX_WEB_REQUEST_SIZE:
                     return Section.COMMON;
                 case CC_JAVA_OPTS:
                 case NC_JAVA_OPTS:
@@ -130,4 +136,8 @@
     public int getNcApiPort() {
         return accessor.getInt(Option.NC_API_PORT);
     }
+
+    public int getMaxWebRequestSize() {
+        return accessor.getInt(Option.MAX_WEB_REQUEST_SIZE);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
index c15f8a7..3215cd6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
@@ -22,7 +22,6 @@
 import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
 import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 import java.util.Map;
@@ -38,7 +37,7 @@
         TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of pages in the transaction log tail"),
         TXN_LOG_BUFFER_PAGESIZE(
                 INTEGER_BYTE_UNIT,
-                StorageUtil.getIntSizeInBytes(128, KILOBYTE),
+                StorageUtil.getIntSizeInBytes(4, MEGABYTE),
                 "The page size (in bytes) for transaction log buffer"),
         TXN_LOG_PARTITIONSIZE(
                 LONG_BYTE_UNIT,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/AsterixVirtualBufferCacheProvider.java
index 84e7ed5..7aeca35 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/AsterixVirtualBufferCacheProvider.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -25,10 +25,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class AsterixVirtualBufferCacheProvider implements IVirtualBufferCacheProvider {
 
     private static final long serialVersionUID = 1L;
@@ -60,4 +65,16 @@
         return deviceId;
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new AsterixVirtualBufferCacheProvider(json.get("datasetId").asInt());
+    }
+
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index b7af0b6..3ff13cb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -37,8 +37,7 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.declareActiveIOOperation();
         }
     }
@@ -47,14 +46,14 @@
     public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
+            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
     @Override
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+        if (opType == LSMOperationType.REPLICATE) {
             dsInfo.undeclareActiveIOOperation();
         }
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
index 41461ec..a14fac8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
@@ -125,7 +125,7 @@
                 }
             }
             ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(lsmIndex.getIOOperationCallback(), mergableComponents);
+            accessor.scheduleMerge(mergableComponents);
         }
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicyFactory.java
index 25242d4..86a3217 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicyFactory.java
@@ -24,9 +24,14 @@
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class CorrelatedPrefixMergePolicyFactory extends PrefixMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
@@ -47,4 +52,14 @@
         policy.configure(configuration);
         return policy;
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new CorrelatedPrefixMergePolicyFactory();
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index f4d764a..4ccb0cc 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.common.context;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -55,7 +56,7 @@
         this.setRegistered(false);
         this.setMemoryAllocated(false);
         this.logManager = logManager;
-        waitLog.setLogType(LogType.WAIT);
+        waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
         waitLog.computeAndSetLogSize();
     }
 
@@ -159,8 +160,8 @@
         this.isExternal = isExternal;
     }
 
-    public Map<Long, IndexInfo> getIndexes() {
-        return indexes;
+    public synchronized Map<Long, IndexInfo> getIndexes() {
+        return Collections.unmodifiableMap(indexes);
     }
 
     public synchronized void addIndex(long resourceID, IndexInfo indexInfo) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
new file mode 100644
index 0000000..26dcc4e
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.common.context;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class DatasetInfoProvider implements IDatasetInfoProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final int datasetId;
+
+    public DatasetInfoProvider(int datasetId) {
+        this.datasetId = datasetId;
+    }
+
+    @Override
+    public DatasetInfo getDatasetInfo(INCServiceContext serviceCtx) {
+        IDatasetLifecycleManager dslcManager =
+                ((INcApplicationContext) serviceCtx.getApplicationContext()).getDatasetLifecycleManager();
+        return dslcManager.getDatasetInfo(datasetId);
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new DatasetInfoProvider(json.get("datasetId").asInt());
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
index 83e3144..811da78 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
@@ -20,19 +20,23 @@
 package org.apache.asterix.common.context;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.common.IResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
  * This factory implementation is used by AsterixDB layer so that indexes of a dataset (/partition)
  * use the same Id generator. This guarantees their memory components would receive the same Id upon
  * activation.
- *
  */
 public class DatasetLSMComponentIdGeneratorFactory implements ILSMComponentIdGeneratorFactory {
     private static final long serialVersionUID = 1L;
@@ -49,7 +53,18 @@
         IDatasetLifecycleManager dslcManager =
                 ((INcApplicationContext) serviceCtx.getApplicationContext()).getDatasetLifecycleManager();
         int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
-        return dslcManager.getComponentIdGenerator(datasetId, partition);
+        return dslcManager.getComponentIdGenerator(datasetId, partition, resource.getPath());
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new DatasetLSMComponentIdGeneratorFactory(json.get("datasetId").asInt());
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index b715eec..d767219 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -19,11 +19,11 @@
 package org.apache.asterix.common.context;
 
 import static org.apache.asterix.common.metadata.MetadataIndexImmutableProperties.METADATA_DATASETS_PARTITIONS;
+import static org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId.MIN_VALID_COMPONENT_ID;
 
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -33,26 +33,26 @@
 import org.apache.asterix.common.api.IDatasetMemoryManager;
 import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.replication.IReplicationStrategy;
 import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.common.utils.TransactionUtil;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
@@ -68,7 +68,7 @@
     private final ILocalResourceRepository resourceRepository;
     private final IDatasetMemoryManager memoryManager;
     private final ILogManager logManager;
-    private final LogRecord logRecord;
+    private final LogRecord waitLog;
     private final int numPartitions;
     private volatile boolean stopped = false;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
@@ -82,7 +82,9 @@
         this.memoryManager = memoryManager;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
         this.numPartitions = numPartitions;
-        logRecord = new LogRecord();
+        waitLog = new LogRecord();
+        waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
+        waitLog.computeAndSetLogSize();
     }
 
     @Override
@@ -161,9 +163,11 @@
         dsInfo.waitForIO();
         closeIndex(iInfo);
         dsInfo.removeIndex(resourceID);
-        if (dsInfo.getReferenceCount() == 0 && dsInfo.isOpen() && dsInfo.getIndexes().isEmpty()
-                && !dsInfo.isExternal()) {
-            removeDatasetFromCache(dsInfo.getDatasetID());
+        synchronized (dsInfo) {
+            if (dsInfo.getReferenceCount() == 0 && dsInfo.isOpen() && dsInfo.getIndexes().isEmpty()
+                    && !dsInfo.isExternal()) {
+                removeDatasetFromCache(dsInfo.getDatasetID());
+            }
         }
     }
 
@@ -323,32 +327,36 @@
     }
 
     @Override
-    public synchronized PrimaryIndexOperationTracker getOperationTracker(int datasetId, int partition) {
+    public synchronized PrimaryIndexOperationTracker getOperationTracker(int datasetId, int partition, String path) {
         DatasetResource dataset = datasets.get(datasetId);
         PrimaryIndexOperationTracker opTracker = dataset.getOpTracker(partition);
         if (opTracker == null) {
-            populateOpTrackerAndIdGenerator(dataset, partition);
+            populateOpTrackerAndIdGenerator(dataset, partition, path);
             opTracker = dataset.getOpTracker(partition);
         }
         return opTracker;
     }
 
     @Override
-    public synchronized ILSMComponentIdGenerator getComponentIdGenerator(int datasetId, int partition) {
+    public synchronized ILSMComponentIdGenerator getComponentIdGenerator(int datasetId, int partition, String path) {
         DatasetResource dataset = datasets.get(datasetId);
-        if (dataset == null) {
-            return null;
-        }
         ILSMComponentIdGenerator generator = dataset.getComponentIdGenerator(partition);
         if (generator == null) {
-            populateOpTrackerAndIdGenerator(dataset, partition);
+            populateOpTrackerAndIdGenerator(dataset, partition, path);
             generator = dataset.getComponentIdGenerator(partition);
         }
         return generator;
     }
 
-    private void populateOpTrackerAndIdGenerator(DatasetResource dataset, int partition) {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
+    @Override
+    public synchronized boolean isRegistered(int datasetId) {
+        return datasets.containsKey(datasetId);
+    }
+
+    private void populateOpTrackerAndIdGenerator(DatasetResource dataset, int partition, String path) {
+        final long lastValidId = getDatasetLastValidComponentId(path);
+        ILSMComponentIdGenerator idGenerator =
+                new LSMComponentIdGenerator(storageProperties.getMemoryComponentsNum(), lastValidId);
         PrimaryIndexOperationTracker opTracker = new PrimaryIndexOperationTracker(dataset.getDatasetID(), partition,
                 logManager, dataset.getDatasetInfo(), idGenerator);
         dataset.setPrimaryIndexOperationTracker(partition, opTracker);
@@ -369,7 +377,9 @@
     @Override
     public synchronized void flushAllDatasets() throws HyracksDataException {
         for (DatasetResource dsr : datasets.values()) {
-            flushDatasetOpenIndexes(dsr, false);
+            if (dsr.getDatasetInfo().isOpen()) {
+                flushDatasetOpenIndexes(dsr, false);
+            }
         }
     }
 
@@ -398,11 +408,10 @@
             PrimaryIndexOperationTracker opTracker, long targetLSN) throws HyracksDataException {
         int partition = opTracker.getPartition();
         for (ILSMIndex lsmIndex : dsr.getDatasetInfo().getDatasetPartitionOpenIndexes(partition)) {
-            AbstractLSMIOOperationCallback ioCallback =
-                    (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+            LSMIOOperationCallback ioCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
             if (!(lsmIndex.isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()
                     || opTracker.isFlushLogCreated() || opTracker.isFlushOnExit())) {
-                long firstLSN = ioCallback.getFirstLSN();
+                long firstLSN = ioCallback.getPersistenceLsn();
                 if (firstLSN < targetLSN) {
                     LOGGER.info("Checkpoint flush dataset {} partition {}", dsr.getDatasetID(), partition);
                     opTracker.setFlushOnExit(true);
@@ -422,90 +431,48 @@
      */
     private void flushDatasetOpenIndexes(DatasetResource dsr, boolean asyncFlush) throws HyracksDataException {
         DatasetInfo dsInfo = dsr.getDatasetInfo();
+        if (!dsInfo.isOpen()) {
+            throw new IllegalStateException("flushDatasetOpenIndexes is called on a dataset that is closed");
+        }
         if (dsInfo.isExternal()) {
             // no memory components for external dataset
             return;
         }
+        // ensure all in-flight flushes gets scheduled
+        logManager.log(waitLog);
         for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
             // flush each partition one by one
             if (primaryOpTracker.getNumActiveOperations() > 0) {
                 throw new IllegalStateException(
                         "flushDatasetOpenIndexes is called on a dataset with currently active operations");
             }
-            int partition = primaryOpTracker.getPartition();
-            Collection<ILSMIndex> indexes = dsInfo.getDatasetPartitionOpenIndexes(partition);
-            ILSMIndex flushIndex = null;
-            for (ILSMIndex lsmIndex : indexes) {
-                if (!lsmIndex.isCurrentMutableComponentEmpty()) {
-                    flushIndex = lsmIndex;
-                    break;
-                }
-            }
-            if (flushIndex == null) {
-                // all open indexes are empty, nothing to flush
-                continue;
-            }
-            LSMComponentId componentId = (LSMComponentId) flushIndex.getCurrentMemoryComponent().getId();
-            ILSMComponentIdGenerator idGenerator = getComponentIdGenerator(dsInfo.getDatasetID(), partition);
-            idGenerator.refresh();
-
-            if (dsInfo.isDurable()) {
-
-                synchronized (logRecord) {
-                    TransactionUtil.formFlushLogRecord(logRecord, dsInfo.getDatasetID(), partition,
-                            componentId.getMinId(), componentId.getMaxId(), null);
-                    try {
-                        logManager.log(logRecord);
-                    } catch (ACIDException e) {
-                        throw new HyracksDataException("could not write flush log while closing dataset", e);
-                    }
-
-                    try {
-                        //notification will come from LogBuffer class (notifyFlushTerminator)
-                        logRecord.wait();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                        throw HyracksDataException.create(e);
-                    }
-                }
-            }
-            for (ILSMIndex index : indexes) {
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
-            }
-
-            if (asyncFlush) {
-                for (ILSMIndex index : indexes) {
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                }
-            } else {
-                for (ILSMIndex index : indexes) {
-                    // TODO: This is not efficient since we flush the indexes sequentially.
-                    // Think of a way to allow submitting the flush requests concurrently.
-                    // We don't do them concurrently because this may lead to a deadlock scenario
-                    // between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
-                    ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                    accessor.scheduleFlush(index.getIOOperationCallback());
-                    // Wait for the above flush op.
-                    dsInfo.waitForIO();
-                }
-            }
+            primaryOpTracker.setFlushOnExit(true);
+            primaryOpTracker.flushIfNeeded();
         }
-
+        // ensure requested flushes were scheduled
+        logManager.log(waitLog);
+        if (!asyncFlush) {
+            List<FlushOperation> flushes = new ArrayList<>();
+            for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
+                flushes.addAll(primaryOpTracker.getScheduledFlushes());
+            }
+            LSMIndexUtil.waitFor(flushes);
+        }
     }
 
     private void closeDataset(DatasetResource dsr) throws HyracksDataException {
         // First wait for any ongoing IO operations
         DatasetInfo dsInfo = dsr.getDatasetInfo();
-        dsInfo.waitForIO();
         try {
             flushDatasetOpenIndexes(dsr, false);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
+        // wait for merges that were scheduled due to the above flush
+        // ideally, we shouldn't need this since merges should still work.
+        // They don't need a special memory budget but there is a problem
+        // for some merge policies that need to access dataset info (correlated prefix)
+        dsInfo.waitForIO();
         for (IndexInfo iInfo : dsInfo.getIndexes().values()) {
             closeIndex(iInfo);
         }
@@ -517,7 +484,9 @@
     public synchronized void closeAllDatasets() throws HyracksDataException {
         ArrayList<DatasetResource> openDatasets = new ArrayList<>(datasets.values());
         for (DatasetResource dsr : openDatasets) {
-            closeDataset(dsr);
+            if (dsr.isOpen()) {
+                closeDataset(dsr);
+            }
         }
     }
 
@@ -624,7 +593,7 @@
     @Override
     public void flushDataset(IReplicationStrategy replicationStrategy) throws HyracksDataException {
         for (DatasetResource dsr : datasets.values()) {
-            if (replicationStrategy.isMatch(dsr.getDatasetID())) {
+            if (dsr.isOpen() && replicationStrategy.isMatch(dsr.getDatasetID())) {
                 flushDatasetOpenIndexes(dsr, false);
             }
         }
@@ -640,4 +609,18 @@
             indexInfo.setOpen(false);
         }
     }
+
+    private long getDatasetLastValidComponentId(String indexPath) {
+        try {
+            final ResourceReference indexRef = ResourceReference.ofIndex(indexPath);
+            final ResourceReference primaryIndexRef = indexRef.getDatasetReference();
+            final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(primaryIndexRef);
+            if (indexCheckpointManager.getCheckpointCount() > 0) {
+                return Math.max(indexCheckpointManager.getLatest().getLastComponentId(), MIN_VALID_COMPONENT_ID);
+            }
+            return MIN_VALID_COMPONENT_ID;
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/Info.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/Info.java
index 8afae0d..fa0f14c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/Info.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/Info.java
@@ -18,25 +18,26 @@
  */
 package org.apache.asterix.common.context;
 
+import java.util.concurrent.atomic.AtomicInteger;
+
 public abstract class Info {
-    private int referenceCount;
-    private boolean isOpen;
+    private final AtomicInteger referenceCount = new AtomicInteger();
+    private volatile boolean isOpen;
 
     public Info() {
-        referenceCount = 0;
         isOpen = false;
     }
 
     public void touch() {
-        ++referenceCount;
+        referenceCount.incrementAndGet();
     }
 
     public void untouch() {
-        --referenceCount;
+        referenceCount.decrementAndGet();
     }
 
     public int getReferenceCount() {
-        return referenceCount;
+        return referenceCount.get();
     }
 
     public boolean isOpen() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 74848d5..a1d31d5 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -19,11 +19,16 @@
 
 package org.apache.asterix.common.context;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
@@ -33,18 +38,24 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
-public class PrimaryIndexOperationTracker extends BaseOperationTracker {
-
+public class PrimaryIndexOperationTracker extends BaseOperationTracker implements IoOperationCompleteListener {
+    private static final Logger LOGGER = LogManager.getLogger();
     private final int partition;
     // Number of active operations on an ILSMIndex instance.
     private final AtomicInteger numActiveOperations;
@@ -52,6 +63,7 @@
     private final ILSMComponentIdGenerator idGenerator;
     private boolean flushOnExit = false;
     private boolean flushLogCreated = false;
+    private final Map<String, FlushOperation> scheduledFlushes = new HashMap<>();
 
     public PrimaryIndexOperationTracker(int datasetID, int partition, ILogManager logManager, DatasetInfo dsInfo,
             ILSMComponentIdGenerator idGenerator) {
@@ -65,20 +77,9 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        super.beforeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             incrementNumActiveOperations(modificationCallback);
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.declareActiveIOOperation();
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
-        if (opType == LSMOperationType.REPLICATE) {
-            completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
 
@@ -86,12 +87,10 @@
     public synchronized void completeOperation(ILSMIndex index, LSMOperationType opType,
             ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
             throws HyracksDataException {
+        super.completeOperation(index, opType, searchCallback, modificationCallback);
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
             decrementNumActiveOperations(modificationCallback);
             flushIfNeeded();
-        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
-                || opType == LSMOperationType.REPLICATE) {
-            dsInfo.undeclareActiveIOOperation();
         }
     }
 
@@ -117,16 +116,46 @@
             }
         }
 
+        ILSMIndex primaryLsmIndex = null;
         if (needsFlush || flushOnExit) {
+            flushOnExit = false;
             // make the current mutable components READABLE_UNWRITABLE to stop coming modify operations from entering
             // them until the current flush is scheduled.
             LSMComponentId primaryId = null;
+            //Double check that the primary index has been modified
+
+            synchronized (this) {
+                if (numActiveOperations.get() > 0) {
+                    throw new IllegalStateException(
+                            "Can't request a flush on an index with active operations: " + numActiveOperations.get());
+                }
+                for (ILSMIndex lsmIndex : indexes) {
+                    if (lsmIndex.isPrimaryIndex()) {
+                        if (lsmIndex.isCurrentMutableComponentEmpty()) {
+                            LOGGER.info("Primary index on dataset {} and partition {} is empty... skipping flush",
+                                    dsInfo.getDatasetID(), partition);
+                            return;
+                        }
+                        primaryLsmIndex = lsmIndex;
+                        break;
+                    }
+                }
+            }
+            if (primaryLsmIndex == null) {
+                throw new IllegalStateException(
+                        "Primary index not found in dataset " + dsInfo.getDatasetID() + " and partition " + partition);
+            }
             for (ILSMIndex lsmIndex : indexes) {
                 ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
                 synchronized (opTracker) {
                     ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+                    if (memComponent.getWriterCount() > 0) {
+                        throw new IllegalStateException(
+                                "Can't request a flush on a component with writers inside: Index:" + lsmIndex
+                                        + " Component:" + memComponent);
+                    }
                     if (memComponent.getState() == ComponentState.READABLE_WRITABLE && memComponent.isModified()) {
-                        memComponent.setState(ComponentState.READABLE_UNWRITABLE);
+                        memComponent.setUnwritable();
                     }
                     if (lsmIndex.isPrimaryIndex()) {
                         primaryId = (LSMComponentId) memComponent.getId();
@@ -134,10 +163,10 @@
                 }
             }
             if (primaryId == null) {
-                throw new IllegalStateException("Primary index not found in dataset " + dsInfo.getDatasetID());
+                throw new IllegalStateException("Primary index found in dataset " + dsInfo.getDatasetID()
+                        + " and partition " + partition + " and is modified but its component id is null");
             }
             LogRecord logRecord = new LogRecord();
-            flushOnExit = false;
             if (dsInfo.isDurable()) {
                 /*
                  * Generate a FLUSH log.
@@ -171,21 +200,41 @@
                 throw new IllegalStateException("Operation started while index was pending scheduling a flush");
             }
             idGenerator.refresh();
-            for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
-                //get resource
-                ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                //update resource lsn
-                AbstractLSMIOOperationCallback ioOpCallback =
-                        (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                ioOpCallback.updateLastLSN(logRecord.getLSN());
-                //schedule flush after update
-                accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+            long flushLsn = logRecord.getLSN();
+            ILSMComponentId nextComponentId = idGenerator.getId();
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+            synchronized (scheduledFlushes) {
+                for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
+                    ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+                    accessor.getOpContext().setParameters(flushMap);
+                    ILSMIOOperation flush = accessor.scheduleFlush();
+                    scheduledFlushes.put(flush.getTarget().getRelativePath(), (FlushOperation) flush);
+                    flush.addCompleteListener(this);
+                }
             }
         } finally {
             flushLogCreated = false;
         }
     }
 
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        synchronized (scheduledFlushes) {
+            scheduledFlushes.remove(operation.getTarget().getRelativePath());
+        }
+    }
+
+    public List<FlushOperation> getScheduledFlushes() {
+        synchronized (scheduledFlushes) {
+            Collection<FlushOperation> scheduled = scheduledFlushes.values();
+            List<FlushOperation> flushes = new ArrayList<FlushOperation>(scheduled.size());
+            flushes.addAll(scheduled);
+            return flushes;
+        }
+    }
+
     public int getNumActiveOperations() {
         return numActiveOperations.get();
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 48bbf00..7f448e3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -20,9 +20,14 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
  * A local resource with a dataset id and an assigned partition
  */
@@ -67,4 +72,21 @@
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         return resource.createInstance(ncServiceCtx);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        json.put("partition", partition);
+        json.set("resource", resource.toJson(registry));
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final int datasetId = json.get("datasetId").asInt();
+        final int partition = json.get("partition").asInt();
+        final IResource resource = (IResource) registry.deserialize(json.get("resource"));
+        return new DatasetLocalResource(datasetId, partition, resource);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
index e844192..71d16f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
@@ -18,15 +18,19 @@
  */
 package org.apache.asterix.common.dataflow;
 
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import java.util.List;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.transactions.ILogManager;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
 
 public class LSMIndexUtil {
 
+    private LSMIndexUtil() {
+    }
+
     public static void checkAndSetFirstLSN(AbstractLSMIndex lsmIndex, ILogManager logManager)
             throws HyracksDataException {
         // If the index has an empty memory component, we need to set its first LSN (For soft checkpoint)
@@ -34,11 +38,21 @@
             //prevent transactions from incorrectly setting the first LSN on a modified component by checking the index is still empty
             synchronized (lsmIndex.getOperationTracker()) {
                 if (lsmIndex.isCurrentMutableComponentEmpty()) {
-                    AbstractLSMIOOperationCallback ioOpCallback =
-                            (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
-                    ioOpCallback.setFirstLSN(logManager.getAppendLSN());
+                    LSMIOOperationCallback ioOpCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+                    ioOpCallback.setFirstLsnForCurrentMemoryComponent(logManager.getAppendLSN());
                 }
             }
         }
     }
+
+    public static void waitFor(List<? extends ILSMIOOperation> ioOperations) throws HyracksDataException {
+        for (int i = 0; i < ioOperations.size(); i++) {
+            try {
+                ioOperations.get(i).sync();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
index b3936de..1401c0a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -48,6 +49,7 @@
 
     public static final String KEY_INDEX = "Index";
     private final boolean isPrimary;
+    private final SourceLocation sourceLoc;
     // This class has both lsmIndex and index (in super class) pointing to the same object
     private AbstractLSMIndex lsmIndex;
     private int i = 0;
@@ -69,10 +71,11 @@
     public LSMInsertDeleteOperatorNodePushable(IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
             RecordDescriptor inputRecDesc, IndexOperation op, boolean isPrimary,
             IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
-            ITupleFilterFactory tupleFilterFactory) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, SourceLocation sourceLoc) throws HyracksDataException {
         super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, op, modCallbackFactory,
                 tupleFilterFactory);
         this.isPrimary = isPrimary;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -138,8 +141,8 @@
                         }
                         break;
                     default: {
-                        throw HyracksDataException.create(ErrorCode.INVALID_OPERATOR_OPERATION, op.toString(),
-                                LSMInsertDeleteOperatorNodePushable.class.getSimpleName());
+                        throw HyracksDataException.create(ErrorCode.INVALID_OPERATOR_OPERATION, sourceLoc,
+                                op.toString(), LSMInsertDeleteOperatorNodePushable.class.getSimpleName());
                     }
                 }
             }
@@ -147,10 +150,10 @@
             if (e.getErrorCode() == ErrorCode.INVALID_OPERATOR_OPERATION) {
                 throw e;
             } else {
-                throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, i);
+                throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, i);
             }
         } catch (Exception e) {
-            throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, i);
+            throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, i);
         }
 
         writeBuffer.ensureFrameSize(buffer.capacity());
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
index 6d58f6d..45661e4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMTreeInsertDeleteOperatorDescriptor.java
@@ -48,7 +48,7 @@
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
         return new LSMInsertDeleteOperatorNodePushable(ctx, partition, fieldPermutation, inputRecDesc, op, isPrimary,
-                indexHelperFactory, modCallbackFactory, tupleFilterFactory);
+                indexHelperFactory, modCallbackFactory, tupleFilterFactory, sourceLoc);
     }
 
     public boolean isPrimary() {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
index 8f28752..1d7ad2d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
@@ -50,5 +50,10 @@
         public void close() throws IOException {
             // No Op
         }
+
+        @Override
+        public void fail(Throwable th) {
+            // No Op
+        }
     }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
index dc884f1..a834a06 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
@@ -21,6 +21,7 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AsterixException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
@@ -33,9 +34,12 @@
         super(message);
     }
 
+    public AsterixException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+    }
+
     public AsterixException(int errorCode, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
-
     }
 
     /**
@@ -47,6 +51,11 @@
         super(cause);
     }
 
+    public AsterixException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+        addSuppressed(cause);
+    }
+
     public AsterixException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
         addSuppressed(cause);
@@ -60,6 +69,10 @@
         super(message, cause);
     }
 
+    public static AsterixException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        return new AsterixException(errorCode, sourceLoc, params);
+    }
+
     public static AsterixException create(int errorCode, Serializable... params) {
         return new AsterixException(errorCode, params);
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
index 693b1c0..0de6b72 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
@@ -22,14 +22,23 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CompilationException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
 
+    public CompilationException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+    }
+
     public CompilationException(int errorCode, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
     }
 
+    public CompilationException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+    }
+
     public CompilationException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 75c5402..3fbce28 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -42,7 +42,7 @@
 
     // Runtime errors
     public static final int CASTING_FIELD = 1;
-    public static final int TYPE_MISMATCH = 2;
+    public static final int TYPE_MISMATCH_FUNCTION = 2;
     public static final int TYPE_INCOMPATIBLE = 3;
     public static final int TYPE_UNSUPPORTED = 4;
     public static final int TYPE_ITEM = 5;
@@ -70,11 +70,16 @@
     public static final int OPERATION_NOT_SUPPORTED = 27;
     public static final int INVALID_DURATION = 28;
     public static final int UNKNOWN_DURATION_UNIT = 29;
-    public static final int QUERY_TIMEOUT = 30;
+    public static final int REQUEST_TIMEOUT = 30;
     public static final int INVALID_TYPE_CASTING_MATH_FUNCTION = 31;
     public static final int REJECT_BAD_CLUSTER_STATE = 32;
     public static final int REJECT_NODE_UNREGISTERED = 33;
-    public static final int DIVISION_BY_ZERO = 34;
+    public static final int UNSUPPORTED_MULTIPLE_STATEMENTS = 35;
+    public static final int CANNOT_COMPARE_COMPLEX = 36;
+    public static final int TYPE_MISMATCH_GENERIC = 37;
+    public static final int DIFFERENT_LIST_TYPE_ARGS = 38;
+    public static final int INTEGER_VALUE_EXPECTED = 39;
+    public static final int NO_STATEMENT_PROVIDED = 40;
 
     public static final int UNSUPPORTED_JRE = 100;
 
@@ -82,7 +87,7 @@
 
     // Compilation errors
     public static final int PARSE_ERROR = 1001;
-    public static final int COMPILATION_TYPE_MISMATCH = 1002;
+    public static final int COMPILATION_TYPE_MISMATCH_FUNCTION = 1002;
     public static final int COMPILATION_TYPE_INCOMPATIBLE = 1003;
     public static final int COMPILATION_TYPE_UNSUPPORTED = 1004;
     public static final int COMPILATION_TYPE_ITEM = 1005;
@@ -130,7 +135,7 @@
     public static final int UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1047;
     public static final int DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1048;
     public static final int LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION = 1049;
-    public static final int NO_DATASET_WITH_NAME = 1050;
+    public static final int UNKNOWN_DATASET_IN_DATAVERSE = 1050;
     public static final int INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD = 1051;
     public static final int INDEX_ILLEGAL_REPETITIVE_FIELD = 1052;
     public static final int CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET = 1053;
@@ -143,13 +148,36 @@
     public static final int WITH_FIELD_MUST_BE_OF_TYPE = 1060;
     public static final int WITH_FIELD_MUST_CONTAIN_SUB_FIELD = 1061;
     public static final int MERGE_POLICY_PARAMETER_INVALID_TYPE = 1062;
-    public static final int NO_DATAVERSE_WITH_NAME = 1063;
+    public static final int UNKNOWN_DATAVERSE = 1063;
     public static final int ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION = 1064;
     public static final int CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE = 1065;
     public static final int CANNOT_SERIALIZE_A_VALUE = 1066;
     public static final int CANNOT_FIND_NON_MISSING_SELECT_OPERATOR = 1067;
     public static final int CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE = 1068;
     public static final int CANNOT_DROP_INDEX = 1069;
+    public static final int METADATA_ERROR = 1070;
+    public static final int DATAVERSE_EXISTS = 1071;
+    public static final int DATASET_EXISTS = 1072;
+    public static final int UNDEFINED_IDENTIFIER = 1073;
+    public static final int AMBIGUOUS_IDENTIFIER = 1074;
+    public static final int FORBIDDEN_SCOPE = 1075;
+    public static final int NAME_RESOLVE_UNKNOWN_DATASET = 1076;
+    public static final int NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE = 1077;
+    public static final int COMPILATION_UNEXPECTED_OPERATOR = 1078;
+    public static final int COMPILATION_ERROR = 1079;
+    public static final int UNKNOWN_NODEGROUP = 1080;
+    public static final int UNKNOWN_FUNCTION = 1081;
+    public static final int UNKNOWN_TYPE = 1082;
+    public static final int UNKNOWN_INDEX = 1083;
+    public static final int INDEX_EXISTS = 1084;
+    public static final int TYPE_EXISTS = 1085;
+    public static final int PARAMETER_NO_VALUE = 1086;
+    public static final int COMPILATION_INVALID_NUM_OF_ARGS = 1087;
+    public static final int FIELD_NOT_FOUND = 1088;
+    public static final int FIELD_NOT_OF_TYPE = 1089;
+    public static final int ARRAY_FIELD_ELEMENTS_MUST_BE_OF_TYPE = 1090;
+    public static final int COMPILATION_TYPE_MISMATCH_GENERIC = 1091;
+    public static final int ILLEGAL_SET_PARAMETER = 1092;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
index 256ce08..359054e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
@@ -20,6 +20,8 @@
 
 import java.util.function.Predicate;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public class ExceptionUtils {
     public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
     public static final String PARAMETER_NAME = "Parameter name: ";
@@ -45,13 +47,28 @@
         Throwable current = e;
         Throwable cause = e.getCause();
         while (cause != null && cause != current) {
-            Throwable nextCause = current.getCause();
             current = cause;
-            cause = nextCause;
+            cause = current.getCause();
         }
         return current;
     }
 
+    public static Throwable getCause(Throwable e, String component, int code) {
+        Throwable current = e;
+        Throwable expected =
+                (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                        && ((HyracksDataException) current).getComponent().equals(component)) ? current : null;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            expected =
+                    (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+                            && ((HyracksDataException) current).getComponent().equals(component)) ? current : expected;
+            cause = current.getCause();
+        }
+        return expected == null ? current : expected;
+    }
+
     /**
      * Determines whether supplied exception contains a matching cause in its hierarchy, or is itself a match
      */
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
index 085f465..a735c81 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
@@ -21,6 +21,7 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -62,10 +63,18 @@
         super(errorCode, params);
     }
 
+    public MetadataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(errorCode, sourceLoc, params);
+    }
+
     public MetadataException(int errorCode, Throwable cause, Serializable... params) {
         super(errorCode, cause, params);
     }
 
+    public MetadataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(errorCode, cause, sourceLoc, params);
+    }
+
     public static MetadataException create(Throwable cause) {
         if (cause instanceof MetadataException || cause == null) {
             return (MetadataException) cause;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ReplicationException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ReplicationException.java
index 034d668..a746316 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ReplicationException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ReplicationException.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.common.exceptions;
 
 public class ReplicationException extends RuntimeException {
+    private static final long serialVersionUID = 8035037902984423617L;
 
     public ReplicationException(Throwable cause) {
         super(cause);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
index 85bfaa5..1c6763f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
@@ -22,6 +22,7 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class RuntimeDataException extends HyracksDataException {
     private static final long serialVersionUID = 1L;
@@ -30,7 +31,15 @@
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
     }
 
+    public RuntimeDataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), null, sourceLoc, params);
+    }
+
     public RuntimeDataException(int errorCode, Throwable cause, Serializable... params) {
         super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
     }
+
+    public RuntimeDataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
deleted file mode 100644
index f027979..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.LongPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
-
-// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
-public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
-    public static final MutableArrayValueReference LSN_KEY = new MutableArrayValueReference("LSN".getBytes());
-    public static final long INVALID = -1L;
-
-    protected final ILSMIndex lsmIndex;
-    // First LSN per mutable component. TODO: move from hyracks to asterixdb
-    protected final long[] firstLSNs;
-    // A boolean array to keep track of flush operations
-    protected final boolean[] flushRequested;
-    // TODO: move from hyracks to asterixdb
-    protected final long[] mutableLastLSNs;
-    // Index of the currently flushing or next to be flushed component
-    protected int readIndex;
-    // Index of the currently being written to component
-    protected int writeIndex;
-    // Index of the memory component to be recycled
-    protected int recycleIndex;
-    // Indicates whether this index has been scheduled to flush (no matter whether succeeds or not)
-    protected boolean hasFlushed;
-    // Keep track of the component Id of the next component being activated.
-    protected ILSMComponentId[] nextComponentIds;
-
-    protected final ILSMComponentIdGenerator idGenerator;
-    protected final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
-    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-    private final Map<ILSMComponentId, Long> componentLsnMap = new HashMap<>();
-
-    public AbstractLSMIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
-        this.lsmIndex = lsmIndex;
-        this.idGenerator = idGenerator;
-        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
-        int count = lsmIndex.getNumberOfAllMemoryComponents();
-        mutableLastLSNs = new long[count];
-        firstLSNs = new long[count];
-        flushRequested = new boolean[count];
-        readIndex = 0;
-        writeIndex = 0;
-        recycleIndex = 0;
-        hasFlushed = false;
-        nextComponentIds = new ILSMComponentId[count];
-        if (count > 0) {
-            nextComponentIds[0] = idGenerator.getId();
-        }
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            /*
-             * This method was called on the scheduleFlush operation.
-             * We set the lastLSN to the last LSN for the index (the LSN for the flush log)
-             * We mark the component flushing flag
-             * We then move the write pointer to the next component and sets its first LSN to the flush log LSN
-             */
-            synchronized (this) {
-                flushRequested[writeIndex] = true;
-                writeIndex = (writeIndex + 1) % mutableLastLSNs.length;
-                // Set the firstLSN of the next component unless it is being flushed
-                if (writeIndex != readIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-
-            }
-        }
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        //TODO: Copying Filters and all content of the metadata pages for flush operation should be done here
-        if (opCtx.getNewComponent() == null) {
-            // failed operation. Nothing to do.
-            return;
-        }
-        putLSNIntoMetadata(opCtx.getNewComponent(), opCtx.getComponentsToBeMerged());
-        putComponentIdIntoMetadata(opCtx.getIoOperationType(), opCtx.getNewComponent(),
-                opCtx.getComponentsToBeMerged());
-        componentLsnMap.put(opCtx.getNewComponent().getId(), getComponentLSN(opCtx.getComponentsToBeMerged()));
-        if (opCtx.getIoOperationType() == LSMIOOperationType.MERGE) {
-            if (opCtx.getComponentsToBeMerged().isEmpty()) {
-                throw new IllegalStateException("Merge must have old components");
-            }
-            LongPointable markerLsn = LongPointable.FACTORY
-                    .createPointable(ComponentUtils.getLong(opCtx.getComponentsToBeMerged().get(0).getMetadata(),
-                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
-            opCtx.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
-        } else if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            // advance memory component indexes
-            synchronized (this) {
-                // we've already consumed the specified LSN/component id.
-                // Now we can advance to the next component
-                flushRequested[readIndex] = false;
-                // if the component which just finished flushing is the component that will be modified next,
-                // we set its first LSN to its previous LSN
-                if (readIndex == writeIndex) {
-                    firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
-                }
-                readIndex = (readIndex + 1) % mutableLastLSNs.length;
-            }
-        }
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // The operation was complete and the next I/O operation for the LSM index didn't start yet
-        if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
-            hasFlushed = true;
-            if (opCtx.getNewComponent() != null) {
-                final Long lsn = componentLsnMap.remove(opCtx.getNewComponent().getId());
-                if (lsn == null) {
-                    throw new IllegalStateException("Unidentified flushed component: " + opCtx.getNewComponent());
-                }
-                // empty component doesn't have any files
-                final Optional<String> componentFile =
-                        opCtx.getNewComponent().getLSMComponentPhysicalFiles().stream().findAny();
-                if (componentFile.isPresent()) {
-                    final ResourceReference ref = ResourceReference.of(componentFile.get());
-                    final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
-                    indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn);
-                }
-            }
-        }
-    }
-
-    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<? extends ILSMComponent> oldComponents)
-            throws HyracksDataException {
-        newComponent.getMetadata().put(LSN_KEY, LongPointable.FACTORY.createPointable(getComponentLSN(oldComponents)));
-    }
-
-    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
-        LongPointable pointable = new LongPointable();
-        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
-        metadataPageManager.get(metadataPageManager.createMetadataFrame(), LSN_KEY, pointable);
-        return pointable.getLength() == 0 ? INVALID : pointable.longValue();
-    }
-
-    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
-            throws HyracksDataException {
-        if (mergedComponents.isEmpty()) {
-            return null;
-        }
-        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
-                mergedComponents.get(mergedComponents.size() - 1).getId());
-
-    }
-
-    private void putComponentIdIntoMetadata(LSMIOOperationType opType, ILSMDiskComponent newComponent,
-            List<? extends ILSMComponent> oldComponents) throws HyracksDataException {
-        // the id of flushed component is set when we copy the metadata of the memory component
-        if (opType == LSMIOOperationType.MERGE) {
-            ILSMComponentId componentId = getMergedComponentId(oldComponents);
-            LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
-        }
-    }
-
-    public synchronized void updateLastLSN(long lastLSN) {
-        if (!flushRequested[writeIndex]) {
-            //if the memory component pointed by writeIndex is being flushed, we should ignore this update call
-            //since otherwise the original LSN is overwritten.
-            //Moreover, since the memory component is already being flushed, the next scheduleFlush request must fail.
-            //See https://issues.apache.org/jira/browse/ASTERIXDB-1917
-            mutableLastLSNs[writeIndex] = lastLSN;
-            if (hasFlushed || lsmIndex.isMemoryComponentsAllocated()) {
-                // we only (re)set next component id if either this index has been flushed (no matter succeed or not)
-                // or the memory component has been allocated
-                // This prevents the case where indexes in a partition are being allocated, while another partition
-                // tries to schedule flush
-                nextComponentIds[writeIndex] = idGenerator.getId();
-            }
-        }
-    }
-
-    /**
-     * Used during the recovery process to force refresh the next component id
-     */
-    public void forceRefreshNextId() {
-        nextComponentIds[writeIndex] = idGenerator.getId();
-    }
-
-    public synchronized void setFirstLSN(long firstLSN) {
-        // We make sure that this method is only called on an empty component so the first LSN is not set incorrectly
-        firstLSNs[writeIndex] = firstLSN;
-    }
-
-    public synchronized long getFirstLSN() {
-        // We make sure that this method is only called on a non-empty component so the returned LSN is meaningful
-        // The firstLSN is always the lsn of the currently being flushed component or the next
-        // to be flushed when no flush operation is on going
-        return firstLSNs[readIndex];
-    }
-
-    public synchronized boolean hasPendingFlush() {
-        for (int i = 0; i < flushRequested.length; i++) {
-            if (flushRequested[i]) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    public long getComponentLSN(List<? extends ILSMComponent> diskComponents) throws HyracksDataException {
-        if (diskComponents.isEmpty()) {
-            // Implies a flush IO operation. --> moves the flush pointer
-            // Flush operation of an LSM index are executed sequentially.
-            synchronized (this) {
-                return mutableLastLSNs[readIndex];
-            }
-        }
-        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
-        long maxLSN = -1L;
-        for (ILSMComponent c : diskComponents) {
-            DiskComponentMetadata md = ((ILSMDiskComponent) c).getMetadata();
-            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
-        }
-        return maxLSN;
-    }
-
-    private synchronized ILSMComponentId getLSMComponentId() {
-        return nextComponentIds[recycleIndex];
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        ILSMComponentId componentId = getLSMComponentId();
-        component.resetId(componentId, false);
-        if (componentSwitched) {
-            recycleIndex = (recycleIndex + 1) % nextComponentIds.length;
-        }
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        if (component == lsmIndex.getCurrentMemoryComponent()) {
-            // only set the component id for the first (current) memory component
-            ILSMComponentId componentId = getLSMComponentId();
-            component.resetId(componentId, false);
-        }
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 5b9883c..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import java.io.ObjectStreamException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.common.IResource;
-
-public abstract class AbstractLSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    protected ILSMComponentIdGeneratorFactory idGeneratorFactory;
-
-    protected transient INCServiceContext ncCtx;
-
-    protected transient IResource resource;
-
-    public AbstractLSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        this.idGeneratorFactory = idGeneratorFactory;
-    }
-
-    @Override
-    public void initialize(INCServiceContext ncCtx, IResource resource) {
-        this.ncCtx = ncCtx;
-        this.resource = resource;
-    }
-
-    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
-        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
-    }
-
-    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
-        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
-    }
-
-    private void readObjectNoData() throws ObjectStreamException {
-        idGeneratorFactory = new ILSMComponentIdGeneratorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-                // used for backward compatibility
-                // if idGeneratorFactory is not set for legacy lsm indexes, we return a default
-                // component id generator which always generates the missing component id.
-                return new ILSMComponentIdGenerator() {
-                    @Override
-                    public void refresh() {
-                        // No op
-                    }
-
-                    @Override
-                    public ILSMComponentId getId() {
-                        return LSMComponentId.MISSING_COMPONENT_ID;
-                    }
-                };
-            }
-        };
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
deleted file mode 100644
index db6c609..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 97badb2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMBTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
deleted file mode 100644
index da1446b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMBTreeWithBuddyIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(lsmIndex, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
deleted file mode 100644
index 9b32345..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMBTreeWithBuddyIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMBTreeWithBuddyIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
new file mode 100644
index 0000000..0aa46a8
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -0,0 +1,280 @@
+/*
+ * 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.asterix.common.ioopcallbacks;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
+public class LSMIOOperationCallback implements ILSMIOOperationCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+    public static final String KEY_FLUSH_LOG_LSN = "FlushLogLsn";
+    public static final String KEY_NEXT_COMPONENT_ID = "NextComponentId";
+    public static final String KEY_FLUSHED_COMPONENT_ID = "FlushedComponentId";
+    private static final String KEY_FIRST_LSN = "FirstLsn";
+    private static final MutableArrayValueReference KEY_METADATA_FLUSH_LOG_LSN =
+            new MutableArrayValueReference(KEY_FLUSH_LOG_LSN.getBytes());
+    public static final long INVALID_LSN = -1L;
+    private final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
+    private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    protected final DatasetInfo dsInfo;
+    protected final ILSMIndex lsmIndex;
+    private long firstLsnForCurrentMemoryComponent = 0L;
+    private long persistenceLsn = 0L;
+    private int pendingFlushes = 0;
+    private Deque<ILSMComponentId> componentIds = new ArrayDeque<>();
+
+    public LSMIOOperationCallback(DatasetInfo dsInfo, ILSMIndex lsmIndex, ILSMComponentId componentId,
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+        this.dsInfo = dsInfo;
+        this.lsmIndex = lsmIndex;
+        this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+        componentIds.add(componentId);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        // No Op
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+            Map<String, Object> map = operation.getParameters();
+            putComponentIdIntoMetadata(operation.getNewComponent(), (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID));
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            Map<String, Object> map = operation.getParameters();
+            putLSNIntoMetadata(operation.getNewComponent(), (Long) map.get(KEY_FLUSH_LOG_LSN));
+            putComponentIdIntoMetadata(operation.getNewComponent(),
+                    ((FlushOperation) operation).getFlushingComponent().getId());
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+            List<ILSMDiskComponent> mergedComponents = operation.getAccessor().getOpContext().getComponentsToBeMerged();
+            putLSNIntoMetadata(operation.getNewComponent(), mergedComponents);
+            putComponentIdIntoMetadata(operation.getNewComponent(), mergedComponents);
+            LongPointable markerLsn =
+                    LongPointable.FACTORY.createPointable(ComponentUtils.getLong(mergedComponents.get(0).getMetadata(),
+                            ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
+            operation.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
+        }
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            return;
+        }
+        if (operation.getIOOpertionType() != LSMIOOperationType.LOAD
+                && operation.getAccessor().getOpContext().getOperation() == IndexOperation.DELETE_COMPONENTS) {
+            deleteComponentsFromCheckpoint(operation);
+        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+                || operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+            addComponentToCheckpoint(operation);
+        }
+    }
+
+    private void addComponentToCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+        // will always update the checkpoint file even if no new component was created
+        FileReference target = operation.getTarget();
+        Map<String, Object> map = operation.getParameters();
+        final Long lsn =
+                operation.getIOOpertionType() == LSMIOOperationType.FLUSH ? (Long) map.get(KEY_FLUSH_LOG_LSN) : 0L;
+        final LSMComponentId id = (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID);
+        final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+        final long componentSequence = IndexComponentFileReference.of(ref.getName()).getSequenceEnd();
+        indexCheckpointManagerProvider.get(ref).flushed(componentSequence, lsn, id.getMaxId());
+    }
+
+    private void deleteComponentsFromCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+        // component was deleted... if a flush, do nothing.. if a merge, must update the checkpoint file
+        if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+            // Get component id of the last disk component
+            LSMComponentId mostRecentComponentId =
+                    getMostRecentComponentId(operation.getAccessor().getOpContext().getComponentsToBeMerged());
+            // Update the checkpoint file
+            FileReference target = operation.getTarget();
+            final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+            indexCheckpointManagerProvider.get(ref).setLastComponentId(mostRecentComponentId.getMaxId());
+        } else if (operation.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+            throw new IllegalStateException("Unexpected IO operation: " + operation.getIOOpertionType());
+        }
+    }
+
+    private LSMComponentId getMostRecentComponentId(Collection<ILSMDiskComponent> deletedComponents)
+            throws HyracksDataException {
+        // must sync on opTracker to ensure list of components doesn't change
+        synchronized (lsmIndex.getOperationTracker()) {
+            List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
+            if (diskComponents.isEmpty()) {
+                LOGGER.log(Level.INFO, "There are no disk components");
+                return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+            }
+            if (deletedComponents.contains(diskComponents.get(diskComponents.size() - 1))) {
+                LOGGER.log(Level.INFO, "All disk components have been deleted");
+                return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+            }
+            int mostRecentComponentIndex = 0;
+            for (int i = 0; i < diskComponents.size(); i++) {
+                if (!deletedComponents.contains(diskComponents.get(i))) {
+                    break;
+                }
+                mostRecentComponentIndex++;
+            }
+            ILSMDiskComponent mostRecentDiskComponent = diskComponents.get(mostRecentComponentIndex);
+            return (LSMComponentId) mostRecentDiskComponent.getId();
+        }
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        putLSNIntoMetadata(newComponent, getComponentLSN(oldComponents));
+    }
+
+    private void putLSNIntoMetadata(ILSMDiskComponent newComponent, long lsn) throws HyracksDataException {
+        newComponent.getMetadata().put(KEY_METADATA_FLUSH_LOG_LSN, LongPointable.FACTORY.createPointable(lsn));
+    }
+
+    public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
+        LongPointable pointable = new LongPointable();
+        IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
+        metadataPageManager.get(metadataPageManager.createMetadataFrame(), KEY_METADATA_FLUSH_LOG_LSN, pointable);
+        return pointable.getLength() == 0 ? INVALID_LSN : pointable.longValue();
+    }
+
+    private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
+            throws HyracksDataException {
+        if (mergedComponents.isEmpty()) {
+            return null;
+        }
+        return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
+                mergedComponents.get(mergedComponents.size() - 1).getId());
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+            throws HyracksDataException {
+        ILSMComponentId componentId = getMergedComponentId(oldComponents);
+        putComponentIdIntoMetadata(newComponent, componentId);
+    }
+
+    private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, ILSMComponentId componentId)
+            throws HyracksDataException {
+        LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
+    }
+
+    public synchronized void setFirstLsnForCurrentMemoryComponent(long firstLsn) {
+        this.firstLsnForCurrentMemoryComponent = firstLsn;
+        if (pendingFlushes == 0) {
+            this.persistenceLsn = firstLsn;
+        }
+    }
+
+    public synchronized long getPersistenceLsn() {
+        return persistenceLsn;
+    }
+
+    public long getComponentLSN(List<ILSMDiskComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents.isEmpty()) {
+            throw new IllegalArgumentException("Can't get LSN from an empty list of disk components");
+        }
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
+        long maxLSN = -1L;
+        for (ILSMDiskComponent c : diskComponents) {
+            DiskComponentMetadata md = c.getMetadata();
+            maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
+        }
+        return maxLSN;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        component.resetId(componentIds.poll(), false);
+    }
+
+    @Override
+    public synchronized void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        dsInfo.declareActiveIOOperation();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes++;
+            FlushOperation flush = (FlushOperation) operation;
+            Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+            Long flushLsn = (Long) map.get(KEY_FLUSH_LOG_LSN);
+            map.put(KEY_FIRST_LSN, firstLsnForCurrentMemoryComponent);
+            map.put(KEY_FLUSHED_COMPONENT_ID, flush.getFlushingComponent().getId());
+            componentIds.add((ILSMComponentId) map.get(KEY_NEXT_COMPONENT_ID));
+            firstLsnForCurrentMemoryComponent = flushLsn; // Advance the first lsn for new component
+        }
+    }
+
+    @Override
+    public synchronized void completed(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            pendingFlushes--;
+            if (operation.getStatus() == LSMIOOperationStatus.SUCCESS) {
+                Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+                persistenceLsn =
+                        pendingFlushes == 0 ? firstLsnForCurrentMemoryComponent : (Long) map.get(KEY_FLUSH_LOG_LSN);
+            }
+        }
+        dsInfo.undeclareActiveIOOperation();
+    }
+
+    public synchronized boolean hasPendingFlush() {
+        return pendingFlushes > 0;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        // no op
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..25cd8b2
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
@@ -0,0 +1,99 @@
+/*
+ * 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.asterix.common.ioopcallbacks;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class LSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final ILSMComponentIdGeneratorFactory idGeneratorFactory;
+
+    protected final IDatasetInfoProvider datasetInfoProvider;
+
+    protected transient INCServiceContext ncCtx;
+
+    protected transient IResource resource;
+
+    public LSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+            IDatasetInfoProvider datasetInfoProvider) {
+        this.idGeneratorFactory = idGeneratorFactory;
+        this.datasetInfoProvider = datasetInfoProvider;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        this.ncCtx = ncCtx;
+        this.resource = resource;
+    }
+
+    protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
+    }
+
+    protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+        return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        return new LSMIOOperationCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+                getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource).getCurrentComponentIndex();
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("idGeneratorFactory", idGeneratorFactory.toJson(registry));
+        json.set("datasetInfoProvider", datasetInfoProvider.toJson(registry));
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ILSMComponentIdGeneratorFactory idGeneratorFactory =
+                (ILSMComponentIdGeneratorFactory) registry.deserialize(json.get("idGeneratorFactory"));
+        final IDatasetInfoProvider datasetInfoProvider =
+                (IDatasetInfoProvider) registry.deserialize(json.get("datasetInfoProvider"));
+        return new LSMIndexIOOperationCallbackFactory(idGeneratorFactory, datasetInfoProvider);
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
deleted file mode 100644
index 3ba9bcd..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMInvertedIndexIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvider) {
-        super(index, idGenerator, checkpointManagerProvider);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 766ef95..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMInvertedIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMInvertedIndexIOOperationCallback(index, getComponentIdGenerator(),
-                getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
deleted file mode 100644
index f3e80ec..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
-    public LSMRTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
-            IIndexCheckpointManagerProvider checkpointManagerProvodier) {
-        super(index, idGenerator, checkpointManagerProvodier);
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 3a0afa8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public LSMRTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
-        super(idGeneratorFactory);
-    }
-
-    @Override
-    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
-        return new LSMRTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/CcIdentifiedMessage.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/CcIdentifiedMessage.java
index d8a68ef..71901e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/CcIdentifiedMessage.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/messaging/CcIdentifiedMessage.java
@@ -24,6 +24,8 @@
 import org.apache.hyracks.api.control.CcId;
 
 public abstract class CcIdentifiedMessage implements ICcIdentifiedMessage, Serializable {
+    private static final long serialVersionUID = -7529473341458281278L;
+
     private CcId ccId;
 
     @Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
index 906a8b7..dc37fc2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
@@ -70,7 +70,7 @@
         if (this == o) {
             return true;
         }
-        if (o != null && o instanceof ResourceReference) {
+        if (o instanceof ResourceReference) {
             ResourceReference that = (ResourceReference) o;
             return getRelativePath().toString().equals(that.getRelativePath().toString());
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
index b008f11..2f0eddf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
@@ -27,31 +27,34 @@
     /**
      * Initializes the first checkpoint of an index with low watermark {@code lsn}
      *
+     * @param validComponentSequence
      * @param lsn
+     * @param validComponentId
      * @throws HyracksDataException
      */
-    void init(long lsn) throws HyracksDataException;
+    void init(long validComponentSequence, long lsn, long validComponentId) throws HyracksDataException;
 
     /**
-     * Called when a new LSM disk component is flushed. When called,  the index checkpoiint is updated
-     * with the latest valid {@code componentTimestamp} and low watermark {@code lsn}
+     * Called when a new LSM disk component is flushed. When called, the index checkpoint is updated
+     * with the latest valid {@code componentSequence} and low watermark {@code lsn}
      *
-     * @param componentTimestamp
+     * @param componentSequence
      * @param lsn
      * @throws HyracksDataException
      */
-    void flushed(String componentTimestamp, long lsn) throws HyracksDataException;
+    void flushed(long componentSequence, long lsn, long componentId) throws HyracksDataException;
 
     /**
-     * Called when a new LSM disk component is replicated from master. When called,  the index checkpoiint is updated
-     * with the latest valid {@code componentTimestamp} and the local lsn mapping of {@code masterLsn} is set as the
+     * Called when a new LSM disk component is replicated from master. When called, the index checkpoint is updated
+     * with the latest valid {@code componentSequence} and the local lsn mapping of {@code masterLsn} is set as the
      * new low watermark.
      *
-     * @param componentTimestamp
+     * @param componentSequence
      * @param masterLsn
+     * @param componentId
      * @throws HyracksDataException
      */
-    void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException;
+    void replicated(long componentSequence, long masterLsn, long componentId) throws HyracksDataException;
 
     /**
      * Called when a flush log is received and replicated from master. The mapping between
@@ -86,16 +89,40 @@
     void delete();
 
     /**
-     * Gets the index last valid component timestamp if the index has any components. Otherwise {@link Optional#empty()}
+     * Gets the index last valid component sequence.
      *
-     * @return the index last valid component timestamp
+     * @return the index last valid component sequence
+     * @throws HyracksDataException
      */
-    Optional<String> getValidComponentTimestamp();
+    long getValidComponentSequence() throws HyracksDataException;
 
     /**
      * Gets the number of valid checkpoints the index has.
      *
      * @return the number of valid checkpoints
+     * @throws HyracksDataException
      */
-    int getCheckpointCount();
+    int getCheckpointCount() throws HyracksDataException;
+
+    /**
+     * @return the latest checkpoint
+     * @throws HyracksDataException
+     */
+    IndexCheckpoint getLatest() throws HyracksDataException;
+
+    /**
+     * Advance the last valid component sequence. Used for replicated bulkloaded components
+     *
+     * @param componentSequence
+     * @throws HyracksDataException
+     */
+    void advanceValidComponentSequence(long componentSequence) throws HyracksDataException;
+
+    /**
+     * Set the last component id. Used during recovery or after component delete
+     *
+     * @param componentId
+     * @throws HyracksDataException
+     */
+    void setLastComponentId(long componentId) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index 6e845e1..cb34600 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -23,6 +23,8 @@
 import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.core.JsonProcessingException;
@@ -30,30 +32,38 @@
 
 public class IndexCheckpoint {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
     private static final long INITIAL_CHECKPOINT_ID = 0;
     private long id;
-    private String validComponentTimestamp;
+    private long validComponentSequence;
     private long lowWatermark;
+    private long lastComponentId;
     private Map<Long, Long> masterNodeFlushMap;
 
-    public static IndexCheckpoint first(long lowWatermark) {
+    public static IndexCheckpoint first(long lastComponentSequence, long lowWatermark, long validComponentId) {
         IndexCheckpoint firstCheckpoint = new IndexCheckpoint();
         firstCheckpoint.id = INITIAL_CHECKPOINT_ID;
         firstCheckpoint.lowWatermark = lowWatermark;
-        firstCheckpoint.validComponentTimestamp = null;
+        firstCheckpoint.validComponentSequence = lastComponentSequence;
+        firstCheckpoint.lastComponentId = validComponentId;
         firstCheckpoint.masterNodeFlushMap = new HashMap<>();
         return firstCheckpoint;
     }
 
-    public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, String validComponentTimestamp) {
+    public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, long validComponentSequence,
+            long lastComponentId) {
         if (lowWatermark < latest.getLowWatermark()) {
+            if (LOGGER.isErrorEnabled()) {
+                LOGGER.error("low watermark {} less than the latest checkpoint low watermark {}", lowWatermark, latest);
+            }
             throw new IllegalStateException("Low watermark should always be increasing");
         }
         IndexCheckpoint next = new IndexCheckpoint();
         next.id = latest.getId() + 1;
         next.lowWatermark = lowWatermark;
-        next.validComponentTimestamp = validComponentTimestamp;
+        next.lastComponentId = lastComponentId;
+        next.validComponentSequence = validComponentSequence;
         next.masterNodeFlushMap = latest.getMasterNodeFlushMap();
         // remove any lsn from the map that wont be used anymore
         next.masterNodeFlushMap.values().removeIf(lsn -> lsn <= lowWatermark);
@@ -64,14 +74,18 @@
     private IndexCheckpoint() {
     }
 
-    public String getValidComponentTimestamp() {
-        return validComponentTimestamp;
+    public long getValidComponentSequence() {
+        return validComponentSequence;
     }
 
     public long getLowWatermark() {
         return lowWatermark;
     }
 
+    public long getLastComponentId() {
+        return lastComponentId;
+    }
+
     public Map<Long, Long> getMasterNodeFlushMap() {
         return masterNodeFlushMap;
     }
@@ -95,4 +109,13 @@
             throw HyracksDataException.create(e);
         }
     }
+
+    @Override
+    public String toString() {
+        try {
+            return asJson();
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
index 81d311d..a24bf72 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
@@ -38,6 +38,10 @@
     protected ResourceReference() {
     }
 
+    public static ResourceReference ofIndex(String indexPath) {
+        return of(new File(indexPath, StorageConstants.METADATA_FILE_NAME).toString());
+    }
+
     public static ResourceReference of(String localResourcePath) {
         ResourceReference lrr = new ResourceReference();
         parse(lrr, localResourcePath);
@@ -72,6 +76,11 @@
         return Paths.get(root, partition, dataverse, dataset, rebalance, index);
     }
 
+    public ResourceReference getDatasetReference() {
+        return ResourceReference
+                .ofIndex(Paths.get(root, partition, dataverse, dataset, rebalance, dataset).toFile().getPath());
+    }
+
     public Path getFileRelativePath() {
         return Paths.get(root, partition, dataverse, dataset, rebalance, index, name);
     }
@@ -101,7 +110,7 @@
         if (this == o) {
             return true;
         }
-        if (o != null && o instanceof ResourceReference) {
+        if (o instanceof ResourceReference) {
             ResourceReference that = (ResourceReference) o;
             return getRelativePath().toString().equals(that.getRelativePath().toString());
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
index 825c7d1..8fe0353 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/Checkpoint.java
@@ -18,33 +18,29 @@
  */
 package org.apache.asterix.common.transactions;
 
-import java.io.IOException;
-
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
-public class Checkpoint implements Comparable<Checkpoint> {
+public class Checkpoint implements Comparable<Checkpoint>, IJsonSerializable {
 
+    private static final long serialVersionUID = 1L;
     private final long checkpointLsn;
     private final long minMCTFirstLsn;
     private final long maxTxnId;
-    private final long timeStamp;
     private final boolean sharp;
     private final int storageVersion;
+    private long id;
 
-    @JsonCreator
-    public Checkpoint(@JsonProperty("checkpointLsn") long checkpointLsn,
-            @JsonProperty("minMCTFirstLsn") long minMCTFirstLsn, @JsonProperty("maxJobId") long maxTxnId,
-            @JsonProperty("timeStamp") long timeStamp, @JsonProperty("sharp") boolean sharp,
-            @JsonProperty("storageVersion") int storageVersion) {
+    public Checkpoint(long id, long checkpointLsn, long minMCTFirstLsn, long maxTxnId, boolean sharp,
+            int storageVersion) {
+        this.id = id;
         this.checkpointLsn = checkpointLsn;
         this.minMCTFirstLsn = minMCTFirstLsn;
         this.maxTxnId = maxTxnId;
-        this.timeStamp = timeStamp;
         this.sharp = sharp;
         this.storageVersion = storageVersion;
     }
@@ -61,8 +57,8 @@
         return maxTxnId;
     }
 
-    public long getTimeStamp() {
-        return timeStamp;
+    public long getId() {
+        return id;
     }
 
     public boolean isSharp() {
@@ -74,61 +70,47 @@
     }
 
     @Override
-    public int compareTo(Checkpoint checkpoint) {
-        long compareTimeStamp = checkpoint.getTimeStamp();
-
-        // Descending order
-        long diff = compareTimeStamp - this.timeStamp;
-        if (diff > 0) {
-            return 1;
-        } else if (diff == 0) {
-            return 0;
-        } else {
-            return -1;
-        }
+    public int compareTo(Checkpoint other) {
+        return Long.compare(this.id, other.id);
     }
 
     @Override
-    public boolean equals(Object obj) {
-        if (this == obj) {
+    public boolean equals(Object o) {
+        if (this == o) {
             return true;
         }
-        if (obj == null) {
+        if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        if (!(obj instanceof Checkpoint)) {
-            return false;
-        }
-        Checkpoint other = (Checkpoint) obj;
-        return compareTo(other) == 0;
+        Checkpoint that = (Checkpoint) o;
+        return id == that.id;
     }
 
     @Override
     public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + (int) (checkpointLsn ^ (checkpointLsn >>> 32));
-        result = prime * result + Long.hashCode(maxTxnId);
-        result = prime * result + (int) (minMCTFirstLsn ^ (minMCTFirstLsn >>> 32));
-        result = prime * result + (sharp ? 1231 : 1237);
-        result = prime * result + storageVersion;
-        result = prime * result + (int) (timeStamp ^ (timeStamp >>> 32));
-        return result;
+        return Long.hashCode(id);
     }
 
-    public String asJson() throws HyracksDataException {
-        try {
-            return new ObjectMapper().writeValueAsString(this);
-        } catch (JsonProcessingException e) {
-            throw HyracksDataException.create(e);
-        }
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode checkpointJson = registry.getClassIdentifier(getClass(), serialVersionUID);
+        checkpointJson.put("id", id);
+        checkpointJson.put("checkpointLsn", checkpointLsn);
+        checkpointJson.put("minMCTFirstLsn", minMCTFirstLsn);
+        checkpointJson.put("maxTxnId", maxTxnId);
+        checkpointJson.put("sharp", sharp);
+        checkpointJson.put("storageVersion", storageVersion);
+        return checkpointJson;
     }
 
-    public static Checkpoint fromJson(String json) throws HyracksDataException {
-        try {
-            return new ObjectMapper().readValue(json, Checkpoint.class);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        long id = json.get("id").asLong();
+        long checkpointLsn = json.get("checkpointLsn").asLong();
+        long minMCTFirstLsn = json.get("minMCTFirstLsn").asLong();
+        long maxTxnId = json.get("maxTxnId").asLong();
+        boolean sharp = json.get("sharp").asBoolean();
+        int storageVersion = json.get("storageVersion").asInt();
+        return new Checkpoint(id, checkpointLsn, minMCTFirstLsn, maxTxnId, sharp, storageVersion);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/DatasetId.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/DatasetId.java
index da6988c..ff11bc3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/DatasetId.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/DatasetId.java
@@ -46,7 +46,7 @@
 
     @Override
     public boolean equals(Object o) {
-        if ((o == null) || !(o instanceof DatasetId)) {
+        if (!(o instanceof DatasetId)) {
             return false;
         }
         return ((DatasetId) o).id == this.id;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ICheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ICheckpointManager.java
index e3cf8b8..36cea55 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ICheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ICheckpointManager.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.common.transactions;
 
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 
@@ -26,10 +25,8 @@
 
     /**
      * @return The latest checkpoint on disk if any exists. Otherwise null.
-     * @throws ACIDException
-     *             when a checkpoint file cannot be read.
      */
-    Checkpoint getLatest() throws ACIDException;
+    Checkpoint getLatest();
 
     /**
      * Performs a sharp checkpoint.
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
deleted file mode 100644
index 82bfdfa..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.common.transactions;
-
-/**
- * @author pouria Interface for a hashTable, used in the internal data
- *         structures of lockManager
- * @param <K>
- *            Type of the objects, used as keys
- * @param <V>
- *            Type of the objects, used as values
- */
-public interface ILockHashTable<K, V> {
-
-    public void put(K key, V value);
-
-    public V get(K key);
-
-    public V remove(K key);
-
-    public int getKeysetSize();
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
deleted file mode 100644
index a545ac2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.asterix.common.transactions;
-
-/**
- * @author pouria
- *         Shows: - The conflict matrix for the locking protocol (whether two
- *         lock modes conflict with each other or not on a single resource) -
- *         Whether request to convert a lock mode to a new one is a conversion
- *         (i.e. the new lock mode is stringer than the current one) or not
- *         Each lock mode is shown/interpreted as an integer
- */
-
-public interface ILockMatrix {
-
-    /**
-     * @param mask
-     *            (current/expected) lock mask on the resource
-     * @param reqLockMode
-     *            index of the requested lockMode
-     * @return true if the lock request conflicts with the mask
-     */
-    public boolean conflicts(int mask, int reqLockMode);
-
-    /**
-     * @param currentLockMode
-     * @param reqLockMode
-     * @return true if the request is a conversion
-     */
-    public boolean isConversion(int currentLockMode, int reqLockMode);
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
deleted file mode 100644
index b5ee0a8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.asterix.common.transactions;
-
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public interface ILongBlockFactory {
-    /**
-     * Ensures future blocks are allocated larger than the supplied value
-     *
-     * @param value
-     *            the value to ensure future blocks are larger than
-     */
-    void ensureMinimum(long value) throws AlgebricksException;
-
-    /**
-     * Allocates a block of longs of specified block size
-     *
-     * @param blockSize
-     *            The size of the block of longs to reserve
-     * @return the start of the reserved block
-     */
-    long getBlock(int blockSize) throws AlgebricksException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java
new file mode 100644
index 0000000..1da5c9c
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.asterix.common.transactions;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public interface IRecoveryManagerFactory {
+
+    /**
+     * Create the local recovery manager
+     *
+     * @param serviceCtx
+     *            the service context
+     * @param txnSubsystem
+     *            the transaction subsystem
+     * @return the recovery manager
+     */
+    IRecoveryManager createRecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IResourceIdManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IResourceIdManager.java
index ce49ccf6..d36d383 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IResourceIdManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IResourceIdManager.java
@@ -18,14 +18,12 @@
  */
 package org.apache.asterix.common.transactions;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
 public interface IResourceIdManager {
 
     long createResourceId();
 
     boolean reported(String nodeId);
 
-    void report(String nodeId, long maxResourceId) throws HyracksDataException;
+    void report(String nodeId, long maxResourceId);
 
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
deleted file mode 100644
index 94ca848..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.asterix.common.transactions;
-
-import java.io.Serializable;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-
-public interface ITxnIdBlockProvider extends Remote, Serializable {
-
-    /**
-     * Ensures that future transaction blocks will be of a value larger than the supplied value
-     *
-     * @param maxId
-     *            The txn id to ensure future txn ids are larger than
-     * @throws RemoteException
-     */
-    void ensureMinimumTxnId(long maxId) throws RemoteException;
-
-    /**
-     * Allocates a block of transaction ids of specified block size
-     *
-     * @param blockSize
-     *            The size of the transaction id block to reserve
-     * @return the start of the reserved block
-     * @throws RemoteException
-     */
-    long reserveTxnIdBlock(int blockSize) throws RemoteException;
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdFactory.java
index be4a1f8..5c28f3f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdFactory.java
@@ -36,7 +36,7 @@
      * @param id
      *            the value to ensure future created transaction ids are larger than
      */
-    void ensureMinimumId(long id) throws AlgebricksException;
+    void ensureMinimumId(long id);
 
     /**
      * The highest transaction id this factory has created
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 5fdb4e2..0c3b21d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -24,7 +24,6 @@
 import java.util.zip.CRC32;
 
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
-import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -271,6 +270,7 @@
                 computeAndSetLogSize();
                 break;
             case LogType.WAIT:
+            case LogType.WAIT_FOR_FLUSHES:
                 computeAndSetLogSize();
                 break;
             case LogType.JOB_COMMIT:
@@ -462,6 +462,7 @@
                 logSize = FLUSH_LOG_SIZE;
                 break;
             case LogType.WAIT:
+            case LogType.WAIT_FOR_FLUSHES:
                 logSize = WAIT_LOG_SIZE;
                 break;
             case LogType.FILTER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
index f02b0de..2d76a11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
@@ -28,6 +28,7 @@
     public static final byte WAIT = 6;
     public static final byte FILTER = 7;
     public static final byte MARKER = 8;
+    public static final byte WAIT_FOR_FLUSHES = 9;
 
     private static final String STRING_UPDATE = "UPDATE";
     private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
@@ -37,6 +38,7 @@
     private static final String STRING_WAIT = "WAIT";
     private static final String STRING_FILTER = "FILTER";
     private static final String STRING_MARKER = "MARKER";
+    private static final String STRING_WAIT_FOR_FLUSHES = "WAIT_FOR_FLUSHES";
     private static final String STRING_UNKNOWN_LOG_TYPE = "UNKNOWN_LOG_TYPE";
 
     public static String toString(byte logType) {
@@ -53,6 +55,8 @@
                 return STRING_FLUSH;
             case LogType.WAIT:
                 return STRING_WAIT;
+            case LogType.WAIT_FOR_FLUSHES:
+                return STRING_WAIT_FOR_FLUSHES;
             case LogType.FILTER:
                 return STRING_FILTER;
             case LogType.MARKER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
index cacbfbc..232c8dd 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
@@ -43,7 +43,13 @@
         spec.setMaxReattempts(0);
         final JobId jobId = hcc.startJob(spec, jobFlags);
         if (waitForCompletion) {
-            hcc.waitForCompletion(jobId);
+            String nameBefore = Thread.currentThread().getName();
+            try {
+                Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+                hcc.waitForCompletion(jobId);
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
         }
         return jobId;
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/NcLocalCounters.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/NcLocalCounters.java
new file mode 100644
index 0000000..5cf6724
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/NcLocalCounters.java
@@ -0,0 +1,62 @@
+/*
+ * 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.asterix.common.utils;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
+import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.nc.NodeControllerService;
+
+public class NcLocalCounters implements Serializable {
+    private static final long serialVersionUID = 3798954558299915995L;
+
+    private final long maxResourceId;
+    private final long maxTxnId;
+    private final long maxJobId;
+
+    private NcLocalCounters(long maxResourceId, long maxTxnId, long maxJobId) {
+        this.maxResourceId = maxResourceId;
+        this.maxTxnId = maxTxnId;
+        this.maxJobId = maxJobId;
+    }
+
+    public static NcLocalCounters collect(CcId ccId, NodeControllerService ncs) throws HyracksDataException {
+        final INcApplicationContext appContext = (INcApplicationContext) ncs.getApplicationContext();
+        long maxResourceId = Math.max(appContext.getLocalResourceRepository().maxId(),
+                MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID);
+        long maxTxnId = appContext.getMaxTxnId();
+        long maxJobId = ncs.getMaxJobId(ccId);
+        return new NcLocalCounters(maxResourceId, maxTxnId, maxJobId);
+    }
+
+    public long getMaxResourceId() {
+        return maxResourceId;
+    }
+
+    public long getMaxTxnId() {
+        return maxTxnId;
+    }
+
+    public long getMaxJobId() {
+        return maxJobId;
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
index 1ac3ffa..b135c7f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
@@ -20,14 +20,6 @@
 
 public class Servlets {
 
-    public static final String AQL = "/aql";
-    public static final String AQL_QUERY = "/query";
-    public static final String AQL_UPDATE = "/update";
-    public static final String AQL_DDL = "/ddl";
-    public static final String SQLPP = "/sqlpp";
-    public static final String SQLPP_QUERY = "/query/sqlpp";
-    public static final String SQLPP_UPDATE = "/update/sqlpp";
-    public static final String SQLPP_DDL = "/ddl/sqlpp";
     public static final String QUERY_STATUS = "/query/service/status/*";
     public static final String QUERY_RESULT = "/query/service/result/*";
     public static final String QUERY_SERVICE = "/query/service";
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
index 265c9fd..644f3c0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
@@ -37,23 +37,17 @@
     public static final String METADATA_FILE_NAME = ".metadata";
     public static final String MASK_FILE_PREFIX = ".mask_";
     public static final String COMPONENT_MASK_FILE_PREFIX = MASK_FILE_PREFIX + "C_";
-    public static final String LEGACY_DATASET_INDEX_NAME_SEPARATOR = "_idx_";
 
     /**
      * The storage version of AsterixDB related artifacts (e.g. log files, checkpoint files, etc..).
      */
-    private static final int LOCAL_STORAGE_VERSION = 2;
+    private static final int LOCAL_STORAGE_VERSION = 5;
 
     /**
      * The storage version of AsterixDB stack.
      */
     public static final int VERSION = LOCAL_STORAGE_VERSION + ITreeIndexFrame.Constants.VERSION;
 
-    /**
-     * The storage version in which the rebalance storage structure was introduced
-     */
-    public static final int REBALANCE_STORAGE_VERSION = 8;
-
     private StorageConstants() {
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 6b13468..aa2c7af 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -19,17 +19,17 @@
 package org.apache.asterix.common.utils;
 
 import java.io.File;
+import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.function.Function;
 
 import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.storage.IndexPathElements;
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.MappedFileSplit;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -143,4 +143,16 @@
     public static String getIndexNameFromPath(String path) {
         return Paths.get(path).getFileName().toString();
     }
+
+    /**
+     * Get the path of the index containing the passed reference
+     *
+     * @param ioManager
+     * @param ref
+     * @return
+     * @throws HyracksDataException
+     */
+    public static Path getIndexPath(IIOManager ioManager, ResourceReference ref) throws HyracksDataException {
+        return ioManager.resolve(ref.getRelativePath().toString()).getFile().toPath();
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index f116ebf..7158b95 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -32,17 +32,19 @@
 4005 = Unsupported replication strategy %1$s
 
 # Type errors
-2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be type %3$s, but the actual input type is %4$s
+2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be of type %3$s, but the actual input type is %4$s
 3,1003 = Type incompatibility: function %1$s gets incompatible input values: %2$s and %3$s
 4,1004 = Unsupported type: %1$s cannot process input type %2$s
 5,1005 = Invalid item type: function %1$s cannot process item type %2$s in an input array (or multiset)
 13,1006 = Duplicate field name \"%1$s\"
 1009 = A returning expression cannot contain dataset access
+37,1091 = Type mismatch: expected value of type %1$s, but got the value of type %2$s
+
 
 # Data errors
 6 = Invalid format for %1$s in %2$s
-7 = Overflow happend in %1$s
-8 = Underflow happend in %1$s
+7 = Overflow in %1$s
+8 = Underflow in %1$s
 9 = Injected failure in %1$s
 10 = Invalid value: function %1$s expects its %2$s input parameter to be a non-negative value, but gets %3$s
 11 = Index out of bound in %1$s: %2$s
@@ -63,11 +65,15 @@
 27 = Operation not supported
 28 = Invalid duration \"%1$s\"
 29 = Unknown duration unit %1$s
-30 = Query timed out and will be cancelled
+30 = Request timed out and will be cancelled
 31 = Invalid type-casting math function: %1$s for converting %2$s to %3$s
 32 = Cannot execute request, cluster is %1$s
 33 = Node is not registered with the CC
-34 = Division by Zero.
+35 = Unsupported multiple statements.
+36 = Cannot compare non-primitive values
+38 = Input contains different list types
+39 = Expected integer value, got %1$s
+40 = No statement provided
 
 100 = Unsupported JRE: %1$s
 
@@ -117,7 +123,7 @@
 1047 = Metadata lock cannot be upgraded! because it was not acquired before
 1048 = Metadata lock cannot be downgraded! because it was not acquired before
 1049 = Metadata lock cannot be acquired for %1$s since it is already acquired for %2$s
-1050 = There is no dataset with name %2$s in dataverse %1$s
+1050 = Cannot find dataset with name %1$s in dataverse %2$s
 1051 = Cannot create enforced index on \"%1$s\" field. The field is closed type.
 1052 = Cannot create index with the same field \"%1$s\" specified more than once.
 1053 = Cannot create primary index on external dataset.
@@ -130,13 +136,35 @@
 1060 = Field \"%1$s\" in the with clause must be of type %2$s
 1061 = Field \"%1$s\" in the with clause must contain sub field \"%2$s\"
 1062 = Merge policy parameters cannot be of type %1$s
-1063 = There is no dataverse with name \"%1$s\"
+1063 = Cannot find dataverse with name %1$s
 1064 = An error was occurred while converting type %1$s to type %2$s.
 1065 = There should be at least two applicable indexes.
 1066 = Cannot serialize a value.
 1067 = Cannot find a non-missing SELECT operator in GROUP operator for a left-outer-join plan optimization.
 1068 = Cannot get the conditional split variable for the given UNNESTMAP operator.
 1069 = Cannot drop index \"%1$s\". Drop dataset \"%1$s\" to remove this index
+1070 = Metadata error. %1$s
+1071 = A dataverse with this name %1$s already exists.
+1072 = A dataset with name %1$s already exists in dataverse %2$s
+1073 = Cannot resolve alias reference for undefined identifier %1$s
+1074 = Cannot resolve ambiguous alias reference for identifier %1$s
+1075 = Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.
+1076 = Cannot find dataset %1$s because there is no dataverse declared, nor an alias with name %1$s!
+1077 = Cannot find dataset %1$s in dataverse %2$s nor an alias with name %1$s!
+1078 = Unexpected operator %1$s in an OperatorExpr starting with %2$s
+1079 = Compilation error: %1$s
+1080 = Cannot find node group with name %1$s
+1081 = Cannot find function with name %1$s
+1082 = Cannot find datatype wit name %1$s
+1083 = Cannot find index with name %1$s
+1084 = An index with this name %1$s already exists
+1085 = A datatype with this name %1$s already exists
+1086 = No value for parameter: %1$s
+1087 = Invalid number of arguments for function %1$s
+1088 = Required field %1$s was not found
+1089 = Field %1$s must be of type %2$s but found to be of type %3$s
+1090 = Field %1$s must be of an array of type %2$s but found to contain an item of type %3$s
+1092 = Parameter %1$s cannot be set
 
 # Feed Errors
 3001 = Illegal state.
@@ -211,7 +239,7 @@
 3071 = Found END_RECORD while expecting a list item
 3072 = Cannot cast the %1$s type to the %2$s type
 3073 = Missing deserializer method for constructor: %1$s
-3074 = This can not be an instance of %1$s
+3074 = %1$s cannot be an instance of %2$s
 3075 = Closed field %1$s has null value
 3076 = %1$s: no files found
 3077 = %1$s: path not found
@@ -250,4 +278,4 @@
 3112 = Array/Multiset item cannot be null
 
 # Lifecycle management errors
-4000 = Partition id %1$d for node %2$s already in use by node %3$s
+4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index befbeed..843cb21 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -220,7 +219,7 @@
 
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 mergedComponents.forEach(component -> {
                     try {
                         resultComponentIDs.add(component.getId());
@@ -230,8 +229,7 @@
                 });
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
         Mockito.when(index.isPrimaryIndex()).thenReturn(isPrimary);
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
deleted file mode 100644
index c03af40..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * 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.asterix.test.ioopcallbacks;
-
-import java.util.Collections;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-
-import junit.framework.TestCase;
-
-public abstract class AbstractLSMIOOperationCallbackTest extends TestCase {
-
-    @Test
-    public void testNormalSequence() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-
-        //request to flush first component
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        //request to flush second component
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testOverWrittenLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        //request to flush first component again
-        //this call should fail
-        callback.updateLastLSN(3);
-        //there is no corresponding beforeOperation, since the first component is being flush
-        //the scheduleFlush request would fail this time
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
-        firstOpCtx.setNewComponent(diskComponent1);
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
-        secondOpCtx.setNewComponent(diskComponent2);
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-    }
-
-    @Test
-    public void testLostLSN() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-
-        LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-        //request to flush first component
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(1);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-
-        //request to flush second component
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        callback.updateLastLSN(2);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-
-        Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-
-        // the first flush is finished, but has not finalized yet (in codebase, these two calls
-        // are not synchronized)
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-
-        //request to flush first component again
-        callback.updateLastLSN(3);
-
-        // the first flush is finalized (it may be called after afterOperation for a while)
-        callback.afterFinalize(firstOpCtx);
-
-        // the second flush gets LSN 2
-        Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
-        // the second flush is finished
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-
-        // it should get new LSN 3
-        Assert.assertEquals(3, callback.getComponentLSN(Collections.emptyList()));
-    }
-
-    @Test
-    public void testAllocateComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId initialId = idGenerator.getId();
-        // simulate a partition is flushed before allocated
-        idGenerator.refresh();
-        callback.updateLastLSN(0);
-
-        callback.allocated(mockComponent);
-        checkMemoryComponent(initialId, mockComponent);
-    }
-
-    @Test
-    public void testRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 100; i++) {
-            // schedule a flush
-            idGenerator.refresh();
-            ILSMComponentId expectedId = idGenerator.getId();
-            callback.updateLastLSN(0);
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.beforeOperation(opCtx);
-            callback.recycled(mockComponent, true);
-            opCtx.setNewComponent(mockDiskComponent());
-            callback.afterOperation(opCtx);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(expectedId, mockComponent);
-        }
-    }
-
-    @Test
-    public void testRecycleWithoutSwitch() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-        for (int i = 0; i < 10; i++) {
-            idGenerator.refresh();
-            id = idGenerator.getId();
-            callback.updateLastLSN(0);
-            // Huh! There is no beforeOperation?
-            ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.recycled(mockComponent, false);
-            callback.afterFinalize(opCtx);
-            checkMemoryComponent(id, mockComponent);
-        }
-    }
-
-    @Test
-    public void testConcurrentRecycleComponentId() throws HyracksDataException {
-        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
-        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        LSMBTreeIOOperationCallback callback =
-                new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
-        ILSMComponentId id = idGenerator.getId();
-        callback.allocated(mockComponent);
-        checkMemoryComponent(id, mockComponent);
-
-        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-
-        // schedule a flush
-        idGenerator.refresh();
-        ILSMComponentId expectedId = idGenerator.getId();
-
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(firstOpCtx);
-        firstOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(firstOpCtx);
-        callback.afterFinalize(firstOpCtx);
-
-        // another flush is to be scheduled before the component is recycled
-        idGenerator.refresh();
-        ILSMComponentId nextId = idGenerator.getId();
-
-        // recycle the component
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(expectedId, mockComponent);
-
-        // schedule the next flush
-        callback.updateLastLSN(0);
-        ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
-        secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.beforeOperation(secondOpCtx);
-        secondOpCtx.setNewComponent(mockDiskComponent());
-        callback.afterOperation(secondOpCtx);
-        callback.afterFinalize(secondOpCtx);
-        callback.recycled(mockComponent, true);
-        checkMemoryComponent(nextId, mockComponent);
-    }
-
-    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
-            throws HyracksDataException {
-        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
-        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
-        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
-        assertEquals(expected, idArgument.getValue());
-        assertEquals(false, forceArgument.getValue().booleanValue());
-
-        Mockito.reset(memoryComponent);
-    }
-
-    private ILSMDiskComponent mockDiskComponent() {
-        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
-        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
-        return component;
-    }
-
-    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
-        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
-                Mockito.mock(IIndexCheckpointManagerProvider.class);
-        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
-        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong());
-        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
-        return indexCheckpointManagerProvider;
-    }
-
-    protected abstract AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
deleted file mode 100644
index a4bc399..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
deleted file mode 100644
index 5f37c78..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeWithBuddyIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMBTreeWithBuddyIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
new file mode 100644
index 0000000..db0911b
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.asterix.test.ioopcallbacks;
+
+import static org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId.MIN_VALID_COMPONENT_ID;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import junit.framework.TestCase;
+
+public class LSMIOOperationCallbackTest extends TestCase {
+    /*
+     * The normal sequence of calls:
+     * 1. refresh id generator
+     * 2. flushLsn
+     * 3. created
+     * 4. before
+     * 5. after
+     * 6. finalize
+     * 7. destroy
+     */
+
+    private static long COMPONENT_SEQUENCE = 0;
+
+    private static String getComponentFileName() {
+        final String sequence = String.valueOf(COMPONENT_SEQUENCE++);
+        return sequence + '_' + sequence;
+    }
+
+    @Test
+    public void testNormalSequence() throws HyracksDataException {
+        int numMemoryComponents = 2;
+
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        String indexId = "mockIndexId";
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        LSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        //Flush first
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        Map<String, Object> flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor firstAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        firstAccessor.getOpContext().setParameters(flushMap);
+        FileReference firstTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+        LSMComponentFileReferences firstFiles = new LSMComponentFileReferences(firstTarget, firstTarget, firstTarget);
+        FlushOperation firstFlush = new TestFlushOperation(firstAccessor, firstTarget, callback, indexId, firstFiles,
+                new LSMComponentId(0, 0));
+        callback.scheduled(firstFlush);
+        callback.beforeOperation(firstFlush);
+
+        //Flush second
+        idGenerator.refresh();
+        flushLsn = 2L;
+        nextComponentId = idGenerator.getId();
+        flushMap = new HashMap<>();
+        flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+        flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+        ILSMIndexAccessor secondAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+        secondAccessor.getOpContext().setParameters(flushMap);
+        FileReference secondTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+        LSMComponentFileReferences secondFiles =
+                new LSMComponentFileReferences(secondTarget, secondTarget, secondTarget);
+        FlushOperation secondFlush = new TestFlushOperation(secondAccessor, secondTarget, callback, indexId,
+                secondFiles, new LSMComponentId(1, 1));
+        callback.scheduled(secondFlush);
+        callback.beforeOperation(secondFlush);
+
+        Map<String, Object> firstFlushMap = firstFlush.getAccessor().getOpContext().getParameters();
+        long firstFlushLogLsn = (Long) firstFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(1, firstFlushLogLsn);
+        final ILSMDiskComponent diskComponent1 = mockDiskComponent();
+        firstFlush.setNewComponent(diskComponent1);
+        callback.afterOperation(firstFlush);
+        callback.afterFinalize(firstFlush);
+        callback.completed(firstFlush);
+
+        Map<String, Object> secondFlushMap = secondFlush.getAccessor().getOpContext().getParameters();
+        long secondFlushLogLsn = (Long) secondFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+        Assert.assertEquals(2, secondFlushLogLsn);
+        final ILSMDiskComponent diskComponent2 = mockDiskComponent();
+        secondFlush.setNewComponent(diskComponent2);
+        callback.afterOperation(secondFlush);
+        callback.afterFinalize(secondFlush);
+        callback.completed(secondFlush);
+    }
+
+    @Test
+    public void testAllocateComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        ILSMComponentId initialId = idGenerator.getId();
+        // simulate a partition is flushed before allocated
+        idGenerator.refresh();
+        long flushLsn = 1L;
+        ILSMComponentId nextComponentId = idGenerator.getId();
+        callback.allocated(mockComponent);
+        callback.recycled(mockComponent);
+        checkMemoryComponent(initialId, mockComponent);
+    }
+
+    @Test
+    public void testRecycleComponentId() throws HyracksDataException {
+        int numMemoryComponents = 2;
+        DatasetInfo dsInfo = new DatasetInfo(101, null);
+        ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents, MIN_VALID_COMPONENT_ID);
+        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+        ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+        Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+        LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+                mockIndexCheckpointManagerProvider());
+        String indexId = "mockIndexId";
+        ILSMComponentId id = idGenerator.getId();
+        callback.recycled(mockComponent);
+        checkMemoryComponent(id, mockComponent);
+
+        Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
+        for (int i = 0; i < 100; i++) {
+            // schedule a flush
+            idGenerator.refresh();
+            ILSMComponentId expectedId = idGenerator.getId();
+            long flushLsn = 0L;
+            Map<String, Object> flushMap = new HashMap<>();
+            flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+            flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, expectedId);
+            ILSMIndexAccessor accessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+            accessor.getOpContext().setParameters(flushMap);
+            FileReference target = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+            LSMComponentFileReferences files = new LSMComponentFileReferences(target, target, target);
+            FlushOperation flush =
+                    new TestFlushOperation(accessor, target, callback, indexId, files, new LSMComponentId(0, 0));
+            callback.scheduled(flush);
+            callback.beforeOperation(flush);
+            callback.recycled(mockComponent);
+            flush.setNewComponent(mockDiskComponent());
+            callback.afterOperation(flush);
+            callback.afterFinalize(flush);
+            callback.completed(flush);
+            checkMemoryComponent(expectedId, mockComponent);
+        }
+    }
+
+    private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
+            throws HyracksDataException {
+        ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
+        ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
+        Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
+        assertEquals(expected, idArgument.getValue());
+        assertEquals(false, forceArgument.getValue().booleanValue());
+        Mockito.reset(memoryComponent);
+    }
+
+    private ILSMDiskComponent mockDiskComponent() {
+        ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
+        Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
+        return component;
+    }
+
+    protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
+        IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
+                Mockito.mock(IIndexCheckpointManagerProvider.class);
+        IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
+        Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.anyLong(), Mockito.anyLong(),
+                Mockito.anyLong());
+        Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
+        return indexCheckpointManagerProvider;
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
deleted file mode 100644
index 343bc59..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMInvertedIndexIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMInvertedIndexIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
deleted file mode 100644
index 10d95d8..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMRTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
-    @Override
-    protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
-        ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
-        Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
-        return new LSMRTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
-                mockIndexCheckpointManagerProvider());
-    }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
new file mode 100644
index 0000000..8a02aa3
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
@@ -0,0 +1,56 @@
+/*
+ * 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.asterix.test.ioopcallbacks;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.mockito.Mockito;
+
+public class TestFlushOperation extends FlushOperation {
+
+    private final LSMComponentFileReferences files;
+    private final ILSMMemoryComponent flushingComponent;
+
+    public TestFlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
+            String indexIdentifier, LSMComponentFileReferences files, LSMComponentId componentId)
+            throws HyracksDataException {
+        super(accessor, target, callback, indexIdentifier);
+        this.files = files;
+        flushingComponent = accessor.getOpContext().getIndex().getCurrentMemoryComponent();
+        Mockito.when(flushingComponent.getId()).thenReturn(componentId);
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return files;
+    }
+
+    @Override
+    public ILSMComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
new file mode 100644
index 0000000..c2621d8
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
@@ -0,0 +1,183 @@
+/*
+ * 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.asterix.test.ioopcallbacks;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+public class TestLSMIndexAccessor implements ILSMIndexAccessor {
+
+    private final ILSMIndexOperationContext opCtx;
+
+    public TestLSMIndexAccessor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IIndexCursor createSearchCursor(boolean exclusive) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+    }
+
+    @Override
+    public ILSMIndexOperationContext getOpContext() {
+        return opCtx;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
+            throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void forceUpdateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void scanDiskComponents(IIndexCursor cursor) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void updateFilter(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
index 9b749fa..4306dda 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -28,10 +29,9 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -44,8 +44,8 @@
     private final List<ILSMDiskComponent> componentsToBeReplicated = new ArrayList<>();
     private boolean isAccessingComponents;
     private IndexOperation op;
-    private LSMIOOperationType ioOperationType;
-    private ILSMDiskComponent newComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> map;
     private boolean filterSkip = false;
     private boolean isRecovery = false;
 
@@ -180,22 +180,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOperationType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOperationType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.map = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newComponent = component;
+    public Map<String, Object> getParameters() {
+        return map;
     }
+
 }
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
index 6db7d6e..49f17a6 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.test.server;
 
 import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.WebManager;
 
 public class RSSTestServer implements ITestServer {
@@ -27,7 +28,8 @@
 
     public RSSTestServer(int port) {
         webManager = new WebManager();
-        HttpServer rssServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), port);
+        HttpServer rssServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), port,
+                HttpServerConfigBuilder.createDefault());
         rssServer.addServlet(new RSSFeedServlet(null, new String[] { "/" }));
         webManager.add(rssServer);
     }
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index de9e38e..b41572d 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -58,29 +58,6 @@
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>default</id>
-            <phase>validate</phase>
-            <goals>
-              <goal>check</goal>
-            </goals>
-            <configuration>
-              <licenses combine.children="append">
-                <license
-                    implementation="org.apache.rat.analysis.license.MITLicense"/>
-              </licenses>
-              <excludes combine.children="append">
-                <exclude>src/main/resources/dashboard/static/*</exclude>
-                <exclude>src/node/**</exclude>
-              </excludes>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
         <groupId>com.github.eirslett</groupId>
         <artifactId>frontend-maven-plugin</artifactId>
         <version>1.6</version>
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
index ac2a465..dfb1b0b 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
@@ -76,6 +76,28 @@
         { "v1": false, "v2": true, "v3": null }
 
 
+### random ###
+ * Syntax:
+
+        random( [seed_value] )
+
+ * Returns a random number, accepting an optional seed value
+ * Arguments:
+    * `seed_value`: an optional `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value representing the seed number.
+ * Return Value:
+    * A random number of type `double` between 0 and 1,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        {
+          "v1": random(),
+          "v2": random(unix_time_from_datetime_in_ms(current_datetime()))
+        };
+
+
 ### range ###
  * Syntax:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
index ed9551e..f0e98fd 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
@@ -199,3 +199,150 @@
         { "a": true, "b": true, "c": true, "d": 1, "e": true, "f": true, "g": 1 }
 
  The function has an alias `ifnanorinf`.
+
+
+### null_if (nullif) ###
+
+ * Syntax:
+
+        null_if(expression1, expression2)
+
+ * Compares two arguments and returns `null` if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if
+        * any argument is a `null` value but no argument is a `missing` value, or
+        * `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": null_if("asterixdb", "asterixdb"),
+            "b": null_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": null, "b": 1 }
+
+ The function has an alias `nullif`.
+
+
+### missing_if (missingif) ###
+
+ * Syntax:
+
+        missing_if(expression1, expression2)
+
+ * Compares two arguments and returns `missing` if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if
+        * any argument is a `missing` value, or
+        * no argument is a `null` value and `argument1` = `argument2`
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": missing_if("asterixdb", "asterixdb")
+            "b": missing_if(1, 2),
+        };
+
+ * The expected result is:
+
+        { "b": 1 }
+
+ The function has an alias `missingif`.
+
+
+### nan_if (nanif) ###
+
+ * Syntax:
+
+        nan_if(expression1, expression2)
+
+ * Compares two arguments and returns `NaN` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `NaN` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(nan_if("asterixdb", "asterixdb")),
+            "b": nan_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "NaN", "b": 1 }
+
+ The function has an alias `nanif`.
+
+
+### posinf_if (posinfif) ###
+
+ * Syntax:
+
+        posinf_if(expression1, expression2)
+
+ * Compares two arguments and returns `+INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `+INF` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(posinf_if("asterixdb", "asterixdb")),
+            "b": posinf_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "+INF", "b": 1 }
+
+ The function has an alias `posinfif`.
+
+
+### neginf_if (neginfif) ###
+
+ * Syntax:
+
+        neginf_if(expression1, expression2)
+
+ * Compares two arguments and returns `-INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+    * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value
+    * `-INF` value of type `double` if `argument1` = `argument2`
+    * a value of the first argument otherwise
+
+ * Example:
+
+        {
+            "a": to_string(neginf_if("asterixdb", "asterixdb")),
+            "b": neginf_if(1, 2)
+        };
+
+ * The expected result is:
+
+        { "a": "-INF", "b": 1 }
+
+ The function has an alias `neginfif`.
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
index c058d10..b95e778c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
@@ -198,6 +198,47 @@
         { "v1": 0.5403023058681398, "v2": -0.4161468365471424, "v3": 1.0, "v4": 0.8775825618903728, "v5": 0.562379076290703 }
 
 
+### degrees ###
+ * Syntax:
+
+        degrees(numeric_value)
+
+ * Converts radians to degrees
+ * Arguments:
+    * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+    * The degrees value for the given radians value. The returned value has type `double`,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        { "v1": degrees(pi()) };
+
+
+ * The expected result is:
+
+        { "v1": 180.0 }
+
+
+### e ###
+ * Syntax:
+
+        e()
+
+ * Return Value:
+    * e (base of the natural logarithm)
+
+ * Example:
+
+        { "v1": e() };
+
+ * The expected result is:
+
+        { "v1": 2.718281828459045 }
+
+
 ### exp ###
  * Syntax:
 
@@ -301,6 +342,22 @@
         { "v1": 0.0, "v2": 0.3010299956639812, "v3": "-Infinity", "v4": -0.3010299956639812, "v5": 3.0 }
 
 
+### pi ###
+ * Syntax:
+
+        pi()
+
+ * Return Value:
+    * Pi
+
+ * Example:
+
+        { "v1": pi() };
+
+ * The expected result is:
+
+        { "v1": 3.141592653589793 }
+
 
 ### power ###
  * Syntax:
@@ -327,6 +384,30 @@
         { "v1": 1, "v3": 0, "v4": 1.4142135623730951 }
 
 
+### radians ###
+ * Syntax:
+
+        radians(numeric_value)
+
+ * Converts degrees to radians
+ * Arguments:
+    * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+    * The radians value for the given degrees value. The returned value has type `double`,
+    * `missing` if the argument is a `missing` value,
+    * `null` if the argument is a `null` value,
+    * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+        { "v1": radians(180) };
+
+
+ * The expected result is:
+
+        { "v1": 3.141592653589793 }
+
+
 ### round ###
  * Syntax:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
index 0e548a7..a7b9bbd 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
@@ -329,6 +329,7 @@
  * Syntax:
 
         regexp_replace(string, string_pattern, string_replacement[, string_flags])
+        regexp_replace(string, string_pattern, string_replacement[, replacement_limit])
 
  * Checks whether the string `string` matches the given
    regular expression pattern `string_pattern` (a Java regular expression pattern),
@@ -339,6 +340,8 @@
     * `string_replacement` : a pattern `string` to be used as the replacement,
     * `string_flag` : (Optional) a `string` with flags to be used during replace.
         * The following modes are enabled with these flags: dotall (s), multiline (m), case_insensitive (i), and comments and whitespace (x).
+    * `replacement_limit`: (Optional) an `integer` specifying the maximum number of replacements to make
+         (if negative then all occurrences will be replaced)
  * Return Value:
     * Returns a `string` that is obtained after the replacements,
     * `missing` if any argument is a `missing` value,
@@ -393,7 +396,7 @@
     * `search_string` : a `string`  substring to be searched for,
     * `replacement_string` : a `string` to be used as the replacement,
     * `limit` : (Optional) an `integer` - maximum number of occurrences to be replaced.
-                If not specified then all occurrences will be replaced
+                If not specified or negative then all occurrences will be replaced
  * Return Value:
     * Returns a `string` that is obtained after the replacements,
     * `missing` if any argument is a `missing` value,
@@ -414,6 +417,30 @@
           "v2": "product-a and x-phone"
         }
 
+### reverse ###
+ * Syntax:
+
+        reverse(string)
+
+ * Returns a string formed by reversing characters in the input `string`.
+ * Arguments:
+    * `string` : a `string` to be reversed
+ * Return Value:
+    * a string containing characters from the the input `string` in the reverse order,
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value,
+    * a type error will be raised if:
+        * the first argument is any other non-string value
+
+ * Example:
+
+        reverse("hello");
+
+
+ * The expected result is:
+
+        "olleh"
+
 ### rtrim ###
  * Syntax:
 
@@ -504,12 +531,13 @@
  * Arguments:
     * `string` : a `string` to be extracted,
     * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`
-                 (starting at 0),
+                 (starting at 0). If negative then counted from the end of the string,
     * `length` : (Optional) an an `tinyint`/`smallint`/`integer`/`bigint` value as the length of the substring.
  * Return Value:
     * a `string` that represents the substring,
     * `missing` if any argument is a `missing` value,
-    * `null` if any argument is a `null` value but no argument is a `missing` value,
+    * `null` if any argument is a `null` value but no argument is a `missing` value, or if the substring could not
+             be obtained because the starting offset is not within string bounds or `length` is negative.
     * a type error will be raised if:
         * the first argument is any other non-string value,
         * or, the second argument is not a `tinyint`, `smallint`, `integer`, or `bigint`,
@@ -524,6 +552,7 @@
 
         "str"
 
+The function has an alias `substring`.
 
 ### trim ###
  * Syntax:
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md b/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
index fefdb7b..187f04c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/8_record.md
@@ -287,3 +287,319 @@
 
         [ "id", "project", "address" ]
 
+### object_remove ###
+ * Syntax:
+
+        object_remove(input_object, field_name)
+
+ * Returns a new object that has the same fields as the input object except the field to be removed
+ * Arguments:
+    * `input_object` : an object value.
+    * `field_name` : a string field name.
+ * Return Value:
+    * A new object that has the same fields as `input_object` except the field `field_name`,
+    * `missing` if the argument `input_object` or `field_name` is missing,
+    * `null` if the argument `input_object` is `null` or any other non-object value, or the argument `field_name`
+       is `null` or any other non-string value.
+
+ * Example:
+
+        object_remove(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "address"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "AsterixDB",
+        }
+
+### object_rename ###
+ * Syntax:
+
+        object_rename(input_object, old_field, new_field)
+
+ * Returns a new object that has the same fields as `input_object` with field `old_field` replaced by `new_field`
+ * Arguments:
+    * `input_object` : an object value.
+    * `old_field` : a string representing the old (original) field name inside the object `input_object`.
+    * `new_field` : a string representing the new field name to replace `old_field` inside the object `input_object`.
+ * Return Value:
+    * A new object that has the same fields as `input_object` with field `old_field` replaced by `new_field`,
+    * `missing` if any argument is a `missing` value,
+    * `null` if any argument is `null` or `input_object` is non-object value, or `old_field` is non-string value, or
+      `new_field` is any non-string value.
+
+ * Example:
+
+        object_rename(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "address"
+                       , "location"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "AsterixDB",
+          "location": {"city": "Irvine", "state": "CA"}
+        }
+
+### object_wrap ###
+ * Syntax:
+
+        object_wrap(input_object)
+
+ * Returns the value of the single name-value pair that appears in `input_object`.
+ * Arguments:
+    * `input_object` : an object value that consists of exactly one name-value pair.
+ * Return Value:
+    * The value of the single name-value pair that appears in `input_object`,
+    * `missing` if `input_object` is `missing`,
+    * `null` if `input_object` is null, or an empty object, or there is more than one name-value pair in `input_object`,
+      or any non-object value.
+
+ * Example:
+
+        object_wrap(
+                     {
+                       "id": 1
+                     }
+                   );
+
+ * The expected result is:
+
+        {
+          1
+        }
+
+### object_replace ###
+ * Syntax:
+
+        object_replace(input_object, old_value, new_value)
+
+ * Returns a new object that has the same fields as `input_object` with all occurrences of value `old_value` replaced by
+   `new_value`
+ * Arguments:
+    * `input_object` : an object value.
+    * `old_value` : a primitive type value to be replaced by `new_value`.
+    * `new_value` : a value to replace `old_value`.
+ * Return Value:
+    * A new object that has the same fields as `input_object` with all occurrences of value `old_value` replaced by
+      `new_value`,
+    * `missing` if any argument is a `missing` value,
+    * `null` if `input_object`  or `old_value` is null,
+    * a type error will be raised if:
+        * `old_value` is not a primitive type value.
+
+ * Example:
+
+        object_replace(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "AsterixDB"
+                       , "Apache AsterixDB"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "Apache AsterixDB",
+          "location": {"city": "Irvine", "state": "CA"}
+        }
+
+### object_add ###
+ * Syntax:
+
+        object_add(input_object, field_name, field_value)
+
+ * Returns a new object that has the same fields as `input_object` as well as the new field `field_name`.
+ * Arguments:
+    * `input_object` : an object value.
+    * `field_name` : a string representing a field name to be added.
+    * `field_value` : a value to be assigned to the new field `field_name`.
+ * Return Value:
+    * A new object that has the same fields as `input_object` as well as the new field `field_name`,
+    * `missing` if `input_object` or `field_name` is `missing`,
+    * `null` if `input_object` or `field_name` is `null`, or `input_object` is not an object, or `field_name` is not
+      a string,
+    * `input_object` if `field_name`already exists in `input_object` or `field_value` is missing.
+
+ * Example:
+
+        object_add(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "company"
+                       , "Apache"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "AsterixDB",
+          "location": {"city": "Irvine", "state": "CA"},
+          "company": "Apache"
+        }
+
+### object_put ###
+ * Syntax:
+
+        object_put(input_object, field_name, field_value)
+
+ * Adds, modifies, or removes a field of an object.
+ * Arguments:
+    * `input_object` : an object value.
+    * `field_name` : a string representing a field name to be added.
+    * `field_value` : a value to be assigned to the new field `field_name`.
+ * Return Value:
+    * a new object that has the same fields as `input_object` as well as the new field `field_name`, or with updated
+      `field_name` value to `field_value` if `field_name` already exists in `input_object`, or with `field_name`removed
+      if `field_name` already exists in `input_object` and `field_value` is `missing`,
+    * `missing` if `input_object` or `field_name` is `missing`,
+    * `null` if `input_object` or `field_name` is `null`, or `input_object` is not an object, or `field_name` is not
+      not a string.
+
+ * Example:
+
+        object_put(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                       , "project"
+                       , "Apache AsterixDB"
+                     );
+
+ * The expected result is:
+
+        {
+          "id": 1,
+          "project": "Apache AsterixDB",
+          "location": {"city": "Irvine", "state": "CA"}
+        }
+
+### object_values ###
+ * Syntax:
+
+        object_values(input_object)
+
+ * Returns an array of the values of the fields in `input_object`.
+ * Arguments:
+    * `input_object` : an object value.
+ * Return Value:
+    * An array of the values of the fields in `input_object`,
+    * `missing` if `input_object` is `missing`,
+    * `null` if `input_object` is null or any non-object value.
+
+ * Example:
+
+        object_values(
+                       {
+                         "id": 1,
+                         "project": "AsterixDB",
+                         "address": {"city": "Irvine", "state": "CA"}
+                       }
+                     );
+
+ * The expected result is:
+
+        [
+          1,
+          "AsterixDB",
+          {"city": "Irvine", "state": "CA"}
+        ]
+
+### object_pairs ###
+ * Syntax:
+
+        object_pairs(input_object)
+
+ * Returns an array of objects describing fields of `input_object`.
+   For each field of the `input_object` the returned array contains an object with two fields `name` and `value`
+   which are set to the `input_object`'s field name and value.
+
+ * Arguments:
+    * `input_object` : an object value.
+ * Return Value:
+    * An array of the `name`/`value` pairs of the fields in `input_object`,
+    * `missing` if `input_object` is `missing`,
+    * `null` if `input_object` is null or any non-object value.
+
+ * Example:
+
+        object_pairs(
+                      {
+                        "id": 1,
+                        "project": "AsterixDB",
+                        "address": {"city": "Irvine", "state": "CA"}
+                      }
+                    );
+
+ * The expected result is:
+
+        [
+          { "name": "id", "value": 1 },
+          { "name": "project", "value": "AsterixDB" },
+          { "name": "address", "value": {"city": "Irvine", "state": "CA"} }
+        ]
+
+### pairs ###
+ * Syntax:
+
+        pairs(input_object)
+
+ * Returns an array of arrays describing fields of `input_object`, including nested fields.
+   For each field of the `input_object` the returned array contains an array with two elements.
+   The first element is the name and the second one is the value of the `input_object`'s field.
+   The input object is introspected recursively, so all fields of its nested objects are returned.
+   Nested objects contained in arrays and multisets are also processed by this function.
+
+ * Arguments:
+    * `input_object` : an object value (or an array or a multiset)
+ * Return Value:
+    * An array of arrays with name, value pairs of the fields in `input_object`, including nested fields.
+      Each inner array has exactly two items: name and value of the `input_object`'s field.
+    * `missing` if `input_object` is `missing`,
+    * `null` if `input_object` is null or a value of a primitive data type.
+
+ * Example:
+
+        pairs(
+               {
+                 "id": 1,
+                 "project": "AsterixDB",
+                 "address": {"city": "Irvine", "state": "CA"}
+               }
+             );
+
+ * The expected result is:
+
+        [
+          [ "id", 1 ],
+          [ "project", "AsterixDB" ],
+          [ "address", { "city": "Irvine", "state": "CA" } ],
+          [ "city", "Irvine" ],
+          [ "state", "CA" ]
+        ]
+
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
index 90eb017..921bc7e 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/9_aggregate_sql.md
@@ -168,10 +168,10 @@
         3.4
 
 
-### coll_count ###
+### strict_count ###
  * Syntax:
 
-        coll_count(collection)
+        strict_count(collection)
 
  * Gets the number of items in the given collection.
  * Arguments:
@@ -185,16 +185,16 @@
 
  * Example:
 
-        coll_count( [1, 2, null, missing] );
+        strict_count( [1, 2, null, missing] );
 
  * The expected result is:
 
         4
 
-### coll_avg ###
+### strict_avg ###
  * Syntax:
 
-        coll_avg(num_collection)
+        strict_avg(num_collection)
 
  * Gets the average value of the numeric items in the given collection.
  * Arguments:
@@ -210,16 +210,16 @@
 
  * Example:
 
-        coll_avg( [100, 200, 300] );
+        strict_avg( [100, 200, 300] );
 
  * The expected result is:
 
         [ 200.0 ]
 
-### coll_sum ###
+### strict_sum ###
  * Syntax:
 
-        coll_sum(num_collection)
+        strict_sum(num_collection)
 
  * Gets the sum of the items in the given collection.
  * Arguments:
@@ -237,7 +237,7 @@
 
  * Example:
 
-        coll_sum( [100, 200, 300] );
+        strict_sum( [100, 200, 300] );
 
  * The expected result is:
 
@@ -246,7 +246,7 @@
 ### array_min ###
  * Syntax:
 
-        coll_min(num_collection)
+        strict_min(num_collection)
 
  * Gets the min value of comparable items in the given collection.
  * Arguments:
@@ -265,7 +265,7 @@
 
  * Example:
 
-        coll_min( [10.2, 100, 5] );
+        strict_min( [10.2, 100, 5] );
 
  * The expected result is:
 
@@ -275,7 +275,7 @@
 ### array_max ###
  * Syntax:
 
-        coll_max(num_collection)
+        strict_max(num_collection)
 
  * Gets the max value of numeric items in the given collection.
  * Arguments:
@@ -294,7 +294,7 @@
 
  * Example:
 
-        coll_max( [10.2, 100, 5] );
+        strict_max( [10.2, 100, 5] );
 
  * The expected result is:
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
index 0569667..c108cc0 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
@@ -17,7 +17,7 @@
  ! under the License.
  !-->
 
-# The SQL++ Query Language
+# The Query Language
 
 * [1. Introduction](#Introduction)
 * [2. Expressions](#Expressions)
@@ -26,7 +26,6 @@
            * [Collection Operators](#Collection_operators)
            * [Comparison Operators](#Comparison_operators)
            * [Logical Operators](#Logical_operators)
-      * [Case Expressions](#Case_expressions)
       * [Quantified Expressions](#Quantified_expressions)
       * [Path Expressions](#Path_expressions)
       * [Primary Expressions](#Primary_expressions)
@@ -34,6 +33,7 @@
            * [Variable References](#Variable_references)
            * [Parenthesized Expressions](#Parenthesized_expressions)
            * [Function call Expressions](#Function_call_expressions)
+           * [Case Expressions](#Case_expressions)
            * [Constructors](#Constructors)
 * [3. Queries](#Queries)
       * [Declarations](#Declarations)
@@ -71,7 +71,7 @@
       * [WITH Clauses](#With_clauses)
       * [LET Clauses](#Let_clauses)
       * [UNION ALL](#Union_all)
-      * [SQL++ Vs. SQL-92](#Vs_SQL-92)
+      * [Differences from SQL-92](#Vs_SQL-92)
 * [4. Errors](#Errors)
       * [Syntax Errors](#Syntax_errors)
       * [Identifier Resolution Errors](#Identifier_resolution_errors)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/1_intro.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/1_intro.md
index f7df33f..e5441ba 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/1_intro.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/1_intro.md
@@ -20,21 +20,24 @@
 # <a id="Introduction">1. Introduction</a><font size="3"/>
 
 This document is intended as a reference guide to the full syntax and semantics of
-the SQL++ Query Language, a SQL-inspired language for working with semistructured data.
-SQL++ has much in common with SQL, but some differences do exist due to the different
-data models that the two languages were designed to serve.
+AsterixDB's query language, a SQL-based language for working with semistructured data.
+The language is a derivative of SQL++, a declarative query language for JSON data which
+is largely backwards compatible with SQL.
+SQL++ originated from research in the FORWARD project at UC San Diego, and it has
+much in common with SQL; some differences exist due to the different data models that
+the two languages were designed to serve.
 SQL was designed in the 1970's for interacting with the flat, schema-ified world of
 relational databases, while SQL++ is much newer and targets the nested, schema-optional
 (or even schema-less) world of modern NoSQL systems.
 
-In the context of Apache AsterixDB, SQL++ is intended for working with the Asterix Data Model
-([ADM](../datamodel.html)),a data model based on a superset of JSON with an enriched and flexible type system.
+In the context of Apache AsterixDB, the query language is intended for working with the Asterix Data Model
+([ADM](../datamodel.html)), a data model based on a superset of JSON with an enriched and flexible type system.
 New AsterixDB users are encouraged to read and work through the (much friendlier) guide
 "[AsterixDB 101: An ADM and SQL++ Primer](primer-sqlpp.html)" before attempting to make use of this document.
 In addition, readers are advised to read through the [Asterix Data Model (ADM) reference guide](../datamodel.html)
-first as well, as an understanding of the data model is a prerequisite to understanding SQL++.
+first as well, as an understanding of the data model is a prerequisite to understanding the query language.
 
-In what follows, we detail the features of the SQL++ language in a grammar-guided manner.
-We list and briefly explain each of the productions in the SQL++ grammar, offering examples
+In what follows, we detail the features of the query language in a grammar-guided manner.
+We list and briefly explain each of the productions in the query grammar, offering examples
 (and results) for clarity.
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index 5f407ca..41cf1d8 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -17,27 +17,30 @@
  ! under the License.
  !-->
 
-SQL++ is a highly composable expression language. Each SQL++ expression returns zero or more data model instances.
-There are three major kinds of expressions in SQL++. At the topmost level, a SQL++ expression can be an
-OperatorExpression (similar to a mathematical expression), an ConditionalExpression (to choose between
-alternative values), or a QuantifiedExpression (which yields a boolean value). Each will be detailed as we
-explore the full SQL++ grammar.
+The query language is a highly composable expression language.
+Each expression in the query language returns zero or more data model instances.
+There are three major kinds of expressions.
+At the topmost level, an expression can be an OperatorExpression (similar to a mathematical expression) or a
+QuantifiedExpression (which yields a boolean value).
+Each will be detailed as we explore the full grammar of the language.
 
-    Expression ::= OperatorExpression | CaseExpression | QuantifiedExpression
+    Expression ::= OperatorExpression | QuantifiedExpression
 
 Note that in the following text, words enclosed in angle brackets denote keywords that are not case-sensitive.
 
 
 ## <a id="Operator_expressions">Operator Expressions</a>
 
-Operators perform a specific operation on the input values or expressions. The syntax of an operator expression is as follows:
+Operators perform a specific operation on the input values or expressions.
+The syntax of an operator expression is as follows:
 
     OperatorExpression ::= PathExpression
                            | Operator OperatorExpression
                            | OperatorExpression Operator (OperatorExpression)?
                            | OperatorExpression <BETWEEN> OperatorExpression <AND> OperatorExpression
 
-SQL++ provides a full set of operators that you can use within its statements. Here are the categories of operators:
+The language provides a full set of operators that you can use within its statements.
+Here are the categories of operators:
 
 * [Arithmetic Operators](#Arithmetic_operators), to perform basic mathematical operations;
 * [Collection Operators](#Collection_operators), to evaluate expressions on collections or objects;
@@ -50,9 +53,9 @@
 |-----------------------------------------------------------------------------|-----------|
 | EXISTS, NOT EXISTS                                                          |  Collection emptiness testing |
 | ^                                                                           |  Exponentiation  |
-| *, /, %                                                                     |  Multiplication, division, modulo |
+| *, /, DIV, MOD (%)                                                          |  Multiplication, division, modulo |
 | +, -                                                                        |  Addition, subtraction  |
-| &#124;&#124;                                                                          |  String concatenation |
+| &#124;&#124;                                                                |  String concatenation |
 | IS NULL, IS NOT NULL, IS MISSING, IS NOT MISSING, <br/>IS UNKNOWN, IS NOT UNKNOWN, IS VALUED, IS NOT VALUED | Unknown value comparison |
 | BETWEEN, NOT BETWEEN                                                        | Range comparison (inclusive on both sides) |
 | =, !=, <>, <, >, <=, >=, LIKE, NOT LIKE, IN, NOT IN                             | Comparison  |
@@ -66,13 +69,18 @@
 [comparison operators](#Comparison_operators) and [logical operators](#Logical_operators).
 
 ### <a id="Arithmetic_operators">Arithmetic Operators</a>
-Arithmetic operators are used to exponentiate, add, subtract, multiply, and divide numeric values, or concatenate string values.
+
+Arithmetic operators are used to exponentiate, add, subtract, multiply, and divide numeric values, or concatenate string
+values.
 
 | Operator     |  Purpose                                                                | Example    |
 |--------------|-------------------------------------------------------------------------|------------|
 | +, -         |  As unary operators, they denote a <br/>positive or negative expression | SELECT VALUE -1; |
 | +, -         |  As binary operators, they add or subtract                              | SELECT VALUE 1 + 2; |
-| *, /, %      |  Multiply, divide, modulo                                               | SELECT VALUE 4 / 2.0; |
+| *            |  Multiply                                                               | SELECT VALUE 4 * 2; |
+| /            |  Divide (returns a value of type `double` if both operands are integers)| SELECT VALUE 5 / 2; |
+| DIV          |  Divide (returns an integer value if both operands are integers)        | SELECT VALUE 5 DIV 2; |
+| MOD (%)      |  Modulo                                                                 | SELECT VALUE 5 % 2; |
 | ^            |  Exponentiation                                                         | SELECT VALUE 2^3;       |
 | &#124;&#124; |  String concatenation                                                   | SELECT VALUE "ab"&#124;&#124;"c"&#124;&#124;"d";       |
 
@@ -87,12 +95,13 @@
 | NOT EXISTS |  Check whether a collection is empty         | SELECT * FROM ChirpMessages cm <br/>WHERE NOT EXISTS cm.referredTopics; |
 
 ### <a id="Comparison_operators">Comparison Operators</a>
-Comparison operators are used to compare values. The comparison operators fall into one of two sub-categories: missing
-value comparisons and regular value comparisons. SQL++ (and JSON) has two ways of representing missing information in
-a object - the presence of the field with a NULL for its value (as in SQL), and the absence of the field (which
-JSON permits). For example, the first of the following objects represents Jack, whose friend is Jill. In the other
-examples, Jake is friendless a la SQL, with a friend field that is NULL, while Joe is friendless in a more natural
-(for JSON) way, i.e., by not having a friend field.
+Comparison operators are used to compare values.
+The comparison operators fall into one of two sub-categories: missing value comparisons and regular value comparisons.
+The query language (and JSON) has two ways of representing missing information in a object - the presence of the field
+with a NULL for its value (as in SQL), and the absence of the field (which JSON permits).
+For example, the first of the following objects represents Jack, whose friend is Jill.
+In the other examples, Jake is friendless a la SQL, with a friend field that is NULL, while Joe is friendless in a more
+natural (for JSON) way, i.e., by not having a friend field.
 
 ##### Examples
 {"name": "Jack", "friend": "Jill"}
@@ -101,18 +110,18 @@
 
 {"name": "Joe"}
 
-The following table enumerates all of SQL++'s comparison operators.
+The following table enumerates all of the query language's comparison operators.
 
-| Operator       |  Purpose                                   | Example    |
-|----------------|--------------------------------------------|------------|
+| Operator       |  Purpose                                       | Example    |
+|----------------|------------------------------------------------|------------|
 | IS NULL        |  Test if a value is NULL                       | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NULL; |
 | IS NOT NULL    |  Test if a value is not NULL                   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT NULL; |
 | IS MISSING     |  Test if a value is MISSING                    | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS MISSING; |
 | IS NOT MISSING |  Test if a value is not MISSING                | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT MISSING;|
 | IS UNKNOWN     |  Test if a value is NULL or MISSING            | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS UNKNOWN; |
 | IS NOT UNKNOWN |  Test if a value is neither NULL nor MISSING   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT UNKNOWN;|
-| IS VALUED      |  Test if a value is neither NULL nor MISSING   | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS VALUED; |
-| IS NOT VALUED  |  Test if a value is NULL or MISSING            | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT VALUED;|
+| IS KNOWN (IS VALUED) |  Test if a value is neither NULL nor MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS KNOWN; |
+| IS NOT KNOWN (IS NOT VALUED) |  Test if a value is NULL or MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT KNOWN; |
 | BETWEEN        |  Test if a value is between a start value and <br/>a end value. The comparison is inclusive <br/>to both start and end values. |  SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId BETWEEN 10 AND 20;|
 | =              |  Equality test                                 | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId=10; |
 | !=             |  Inequality test                               | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId!=10;|
@@ -134,8 +143,8 @@
 | IS NOT MISSING | TRUE | TRUE | FALSE |
 | IS UNKNOWN | FALSE | TRUE | TRUE |
 | IS NOT UNKNOWN | TRUE | FALSE | FALSE|
-| IS VALUED | TRUE | FALSE | FALSE |
-| IS NOT VALUED | FALSE | TRUE | TRUE |
+| IS KNOWN (IS VALUED) | TRUE | FALSE | FALSE |
+| IS NOT KNOWN (IS NOT VALUED) | FALSE | TRUE | TRUE |
 
 ### <a id="Logical_operators">Logical Operators</a>
 Logical operators perform logical `NOT`, `AND`, and `OR` operations over Boolean values (`TRUE` and `FALSE`) plus `NULL` and `MISSING`.
@@ -170,20 +179,6 @@
 | NULL | NULL |
 | MISSING | MISSING |
 
-## <a id="Case_expressions">Case Expressions</a>
-
-    CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
-    SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
-    SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
-
-In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
-
-In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
-
-The following example illustrates the form of a case expression.
-##### Example
-
-    CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
 
 ## <a id="Quantified_expressions">Quantified Expressions</a>
 
@@ -213,15 +208,18 @@
 
     PathExpression  ::= PrimaryExpression ( Field | Index )*
     Field           ::= "." Identifier
-    Index           ::= "[" ( Expression | "?" ) "]"
+    Index           ::= "[" Expression "]"
 
-Components of complex types in the data model are accessed via path expressions. Path access can be applied to the result
-of a SQL++ expression that yields an instance of  a complex type, for example, a object or array instance. For objects,
-path access is based on field names. For arrays, path access is based on (zero-based) array-style indexing.
-SQL++ also supports an "I'm feeling lucky" style index accessor, [?], for selecting an arbitrary element from an array.
+Components of complex types in the data model are accessed via path expressions.
+Path access can be applied to the result of a query expression that yields an instance of a complex type, for example, a
+object or array instance.
+For objects, path access is based on field names.
+For arrays, path access is based on (zero-based) array-style indexing.
 Attempts to access non-existent fields or out-of-bound array elements produce the special value `MISSING`.
-Type errors will be raised for inappropriate use of a path expression, such as applying a field
-accessor to a numeric value.
+For multisets path access is also zero-based and returns an arbitrary multiset element if the index is within the size
+of the multiset or `MISSING` otherwise.
+Type errors will be raised for inappropriate use of a path expression, such as applying a field accessor to a numeric
+value.
 
 The following examples illustrate field access for a object, index-based element access for an array, and also a
 composition thereof.
@@ -239,13 +237,16 @@
 
     PrimaryExpr ::= Literal
                   | VariableReference
+                  | ParameterReference
                   | ParenthesizedExpression
                   | FunctionCallExpression
+                  | CaseExpression
                   | Constructor
 
-The most basic building block for any SQL++ expression is PrimaryExpression. This can be a simple literal (constant)
-value, a reference to a query variable that is in scope, a parenthesized expression, a function call, or a newly
-constructed instance of the data model (such as a newly constructed object, array, or multiset of data model instances).
+The most basic building block for any expression in the query langauge is PrimaryExpression.
+This can be a simple literal (constant) value, a reference to a query variable that is in scope, a parenthesized
+expression, a function call, or a newly constructed instance of the data model (such as a newly constructed object,
+array, or multiset of data model instances).
 
 ## <a id="Literals">Literals</a>
 
@@ -297,9 +298,13 @@
     DoubleLiteral  ::= <DIGITS> "." <DIGITS>
                        | "." <DIGITS>
 
-Literals (constants) in SQL++ can be strings, integers, floating point values, double values, boolean constants, or special constant values like `NULL` and `MISSING`. The `NULL` value is like a `NULL` in SQL; it is used to represent an unknown field value. The specialy value `MISSING` is only meaningful in the context of SQL++ field accesses; it occurs when the accessed field simply does not exist at all in a object being accessed.
+Literals (constants) in a query can be strings, integers, floating point values, double values, boolean constants, or
+special constant values like `NULL` and `MISSING`.
+The `NULL` value is like a `NULL` in SQL; it is used to represent an unknown field value.
+The special value `MISSING` is only meaningful in the context of field accesses; it occurs when the accessed field
+simply does not exist at all in a object being accessed.
 
-The following are some simple examples of SQL++ literals.
+The following are some simple examples of literals.
 
 ##### Examples
 
@@ -307,11 +312,11 @@
     "test string"
     42
 
-Different from standard SQL, double quotes play the same role as single quotes and may be used for string literals in SQL++.
+Different from standard SQL, double quotes play the same role as single quotes and may be used for string literals in queries as well.
 
 ### <a id="Variable_references">Variable References</a>
 
-    VariableReference     ::= <IDENTIFIER>|<DelimitedIdentifier>
+    VariableReference     ::= <IDENTIFIER> | <DelimitedIdentifier>
     <IDENTIFIER>          ::= (<LETTER> | "_") (<LETTER> | <DIGIT> | "_" | "$")*
     <LETTER>              ::= ["A" - "Z", "a" - "z"]
     DelimitedIdentifier   ::= "`" (<EscapeQuot>
@@ -325,10 +330,15 @@
                                     | ~["`","\\"])*
                               "`"
 
-A variable in SQL++ can be bound to any legal data model value. A variable reference refers to the value to which an in-scope variable is
-bound. (E.g., a variable binding may originate from one of the `FROM`, `WITH` or `LET` clauses of a `SELECT` statement or from an
-input parameter in the context of a function body.) Backticks, for example, \`id\`, are used for delimited identifiers. Delimiting is needed when
-a variable's desired name clashes with a SQL++ keyword or includes characters not allowed in regular identifiers. More information on exactly how variable references are resolved can be found in the appendix section on Variable Resolution.
+A variable in a query can be bound to any legal data model value.
+A variable reference refers to the value to which an in-scope variable is bound.
+(E.g., a variable binding may originate from one of the `FROM`, `WITH` or `LET` clauses of a `SELECT` statement or from
+an input parameter in the context of a function body.)
+Backticks, for example, \`id\`, are used for delimited identifiers.
+Delimiting is needed when a variable's desired name clashes with a keyword or includes characters not allowed in regular
+identifiers.
+More information on exactly how variable references are resolved can be found in the appendix section on Variable
+Resolution.
 
 ##### Examples
 
@@ -337,11 +347,29 @@
     `SELECT`
     `my-function`
 
+### <a id="Parameter_references">Parameter References</a>
+
+    ParameterReference              ::= NamedParameterReference | PositionalParameterReference
+    NamedParameterReference         ::= "$" (<IDENTIFIER> | <DelimitedIdentifier>)
+    PositionalParameterReference    ::= ("$" <DIGITS>) | "?"
+
+A statement parameter is an external variable which value is provided through the [statement execution API](../api.html#queryservice).
+An error will be raised if the parameter is not bound at the query execution time.
+Positional parameter numbering starts at 1.
+"?" parameters are interpreted as $1, .. $N in the order in which they appear in the statement.
+
+##### Examples
+
+    $id
+    $1
+    ?
+
 ### <a id="Parenthesized_expressions">Parenthesized Expressions</a>
 
     ParenthesizedExpression ::= "(" Expression ")" | Subquery
 
-An expression can be parenthesized to control the precedence order or otherwise clarify a query. In SQL++, for composability, a subquery is also an parenthesized expression.
+An expression can be parenthesized to control the precedence order or otherwise clarify a query.
+For composability, a subquery is also an parenthesized expression.
 
 The following expression evaluates to the value 2.
 
@@ -353,7 +381,11 @@
 
     FunctionCallExpression ::= FunctionName "(" ( Expression ( "," Expression )* )? ")"
 
-Functions are included in SQL++, like most languages, as a way to package useful functionality or to componentize complicated or reusable SQL++ computations. A function call is a legal SQL++ query expression that represents the value resulting from the evaluation of its body expression with the given parameter bindings; the parameter value bindings can themselves be any SQL++ expressions.
+Functions are included in the query language, like most languages, as a way to package useful functionality or to
+componentize complicated or reusable computations.
+A function call is a legal query expression that represents the value resulting from the evaluation of its body
+expression with the given parameter bindings; the parameter value bindings can themselves be any expressions in the
+query language.
 
 The following example is a (built-in) function call expression whose value is 8.
 
@@ -361,6 +393,22 @@
 
     length('a string')
 
+## <a id="Case_expressions">Case Expressions</a>
+
+    CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
+    SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+    SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+
+In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
+
+In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
+
+The following example illustrates the form of a case expression.
+
+##### Example
+
+    CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
+
 
 ### <a id="Constructors">Constructors</a>
 
@@ -370,7 +418,7 @@
     ObjectConstructor        ::= "{" ( FieldBinding ( "," FieldBinding )* )? "}"
     FieldBinding             ::= Expression ":" Expression
 
-A major feature of SQL++ is its ability to construct new data model instances. This is accomplished using
+A major feature of the query language is its ability to construct new data model instances. This is accomplished using
 its constructors for each of the model's complex object structures, namely arrays, multisets, and objects.
 Arrays are like JSON arrays, while multisets have bag semantics.
 Objects are built from fields that are field-name/field-value pairs, again like JSON.
@@ -378,9 +426,9 @@
 The following examples illustrate how to construct a new array with 4 items and a new object with 2 fields respectively.
 Array elements can be homogeneous (as in the first example),
 which is the common case, or they may be heterogeneous (as in the second example). The data values and field name values
-used to construct arrays, multisets, and objects in constructors are all simply SQL++ expressions. Thus, the collection
+used to construct arrays, multisets, and objects in constructors are all simply query expressions. Thus, the collection
 elements, field names, and field values used in constructors can be simple literals or they can come from query variable
-references or even arbitrarily complex SQL++ expressions (subqueries).
+references or even arbitrarily complex query expressions (subqueries).
 Type errors will be raised if the field names in an object are not strings, and
 duplicate field errors will be raised if they are not distinct.
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_dataverse.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_dataverse.md
index 1d1ab8e..d33d680 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_dataverse.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_dataverse.md
@@ -22,7 +22,7 @@
     DatabaseDeclaration ::= "USE" Identifier
 
 At the uppermost level, the world of data is organized into data namespaces called **dataverses**.
-To set the default dataverse for a series of statements, the USE statement is provided in SQL++.
+To set the default dataverse for statements, the USE statement is provided.
 
 As an example, the following statement sets the default dataverse to be "TinySocial".
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_function.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_function.md
index a45996c..8e77de9 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_function.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_declare_function.md
@@ -17,15 +17,15 @@
  ! under the License.
  !-->
 
-When writing a complex SQL++ query, it can sometimes be helpful to define one or more auxilliary functions
+When writing a complex query, it can sometimes be helpful to define one or more auxilliary functions
 that each address a sub-piece of the overall query.
 The declare function statement supports the creation of such helper functions.
-In general, the function body (expression) can be any legal SQL++ query expression.
+In general, the function body (expression) can be any legal query expression.
 
     FunctionDeclaration  ::= "DECLARE" "FUNCTION" Identifier ParameterList "{" Expression "}"
     ParameterList        ::= "(" ( <VARIABLE> ( "," <VARIABLE> )* )? ")"
 
-The following is a simple example of a temporary SQL++ function definition and its use.
+The following is a simple example of a temporary function definition and its use.
 
 ##### Example
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
index 996bb9e..365c6e3 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -19,7 +19,7 @@
 
 ##  <a id="SELECT_statements">SELECT Statements</a>
 
-The following shows the (rich) grammar for the `SELECT` statement in SQL++.
+The following shows the (rich) grammar for the `SELECT` statement in the query language.
 
     SelectStatement    ::= ( WithClause )?
                            SelectSetOperation (OrderbyClause )? ( LimitClause )?
@@ -39,7 +39,7 @@
     SelectClause       ::= <SELECT> ( <ALL> | <DISTINCT> )? ( SelectRegular | SelectValue )
     SelectRegular      ::= Projection ( "," Projection )*
     SelectValue        ::= ( <VALUE> | <ELEMENT> | <RAW> ) Expression
-    Projection         ::= ( Expression ( <AS> )? Identifier | "*" )
+    Projection         ::= ( Expression ( <AS> )? Identifier | "*" | Identifier "." "*" )
 
     FromClause         ::= <FROM> FromTerm ( "," FromTerm )*
     FromTerm           ::= Expression (( <AS> )? Variable)?
@@ -169,13 +169,13 @@
     } ]
 
 ## <a id="Select_clauses">SELECT Clause</a>
-The SQL++ `SELECT` clause always returns a collection value as its result (even if the result is empty or a singleton).
+The `SELECT` clause always returns a collection value as its result (even if the result is empty or a singleton).
 
 ### <a id="Select_element">Select Element/Value/Raw</a>
-The `SELECT VALUE` clause in SQL++ returns an array or multiset that contains the results of evaluating the `VALUE`
+The `SELECT VALUE` clause returns an array or multiset that contains the results of evaluating the `VALUE`
 expression, with one evaluation being performed per "binding tuple" (i.e., per `FROM` clause item) satisfying
 the statement's selection criteria.
-For historical reasons SQL++ also allows the keywords `ELEMENT` or `RAW` to be used in place of `VALUE`
+For historical reasons the query language also allows the keywords `ELEMENT` or `RAW` to be used in place of `VALUE`
 (not recommended).
 
 If there is no FROM clause, the expression after `VALUE` is evaluated once with no binding tuples
@@ -228,10 +228,10 @@
     } ]
 
 ### <a id="SQL_select">SQL-style SELECT</a>
-In SQL++, the traditional SQL-style `SELECT` syntax is also supported.
-This syntax can also be reformulated in a `SELECT VALUE` based manner in SQL++.
+The traditional SQL-style `SELECT` syntax is also supported in the query language.
+This syntax can also be reformulated in a `SELECT VALUE` based manner.
 (E.g., `SELECT expA AS fldA, expB AS fldB` is syntactic sugar for `SELECT VALUE { 'fldA': expA, 'fldB': expB }`.)
-Unlike in SQL, the result of an SQL++ query does not preserve the order of expressions in the `SELECT` clause.
+Unlike in SQL, the result of a query does not preserve the order of expressions in the `SELECT` clause.
 
 ##### Example
     SELECT user.alias user_alias, user.name user_name
@@ -246,7 +246,7 @@
     } ]
 
 ### <a id="Select_star">SELECT *</a>
-In SQL++, `SELECT *` returns a object with a nested field for each input tuple.
+`SELECT *` returns an object with a nested field for each input tuple.
 Each field has as its field name the name of a binding variable generated by either the `FROM` clause or `GROUP BY`
 clause in the current enclosing `SELECT` statement, and its field value is the value of that binding variable.
 
@@ -393,9 +393,87 @@
         }
     } ]
 
+### <a id="Select_variable_star">SELECT _variable_.*</a>
+
+Whereas `SELECT *` returns all the fields bound to all the variables which are currently defined,
+the notation `SELECT c.*` returns all the fields of the object bound to variable `c`.
+The variable `c` must be bound to an object for this to work.
+
+##### Example
+
+    SELECT user.*
+    FROM GleambookUsers user;
+
+Compare this query with the first example given under [SELECT *](#Select_star).
+This query returns all users from the `GleambookUsers` dataset,
+but the `user` variable name is omitted from the results:
+
+    [
+      {
+        "id": 1,
+        "alias": "Margarita",
+        "name": "MargaritaStoddard",
+        "nickname": "Mags",
+        "userSince": "2012-08-20T10:10:00",
+        "friendIds": [
+          2,
+          3,
+          6,
+          10
+        ],
+        "employment": [
+          {
+            "organizationName": "Codetechno",
+            "start-date": "2006-08-06"
+          },
+          {
+            "organizationName": "geomedia",
+            "start-date": "2010-06-17",
+            "end-date": "2010-01-26"
+          }
+        ],
+        "gender": "F"
+      },
+      {
+        "id": 2,
+        "alias": "Isbel",
+        "name": "IsbelDull",
+        "nickname": "Izzy",
+        "userSince": "2011-01-22T10:10:00",
+        "friendIds": [
+          1,
+          4
+        ],
+        "employment": [
+          {
+            "organizationName": "Hexviafind",
+            "startDate": "2010-04-27"
+          }
+        ]
+      },
+      {
+        "id": 3,
+        "alias": "Emory",
+        "name": "EmoryUnk",
+        "userSince": "2012-07-10T10:10:00",
+        "friendIds": [
+          1,
+          5,
+          8,
+          9
+        ],
+        "employment": [
+          {
+            "organizationName": "geomedia",
+            "startDate": "2010-06-17",
+            "endDate": "2010-01-26"
+          }
+        ]
+      }
+    ]
 
 ### <a id="Select_distinct">SELECT DISTINCT</a>
-SQL++'s `DISTINCT` keyword is used to eliminate duplicate items in results. The following example shows how it works.
+The `DISTINCT` keyword is used to eliminate duplicate items in results. The following example shows how it works.
 
 ##### Example
 
@@ -423,7 +501,7 @@
      ]
 
 ### <a id="Unnamed_projections">Unnamed Projections</a>
-Similar to standard SQL, SQL++ supports unnamed projections (a.k.a, unnamed `SELECT` clause items), for which names are generated.
+Similar to standard SQL, the query language  supports unnamed projections (a.k.a, unnamed `SELECT` clause items), for which names are generated.
 Name generation has three cases:
 
   * If a projection expression is a variable reference expression, its generated name is the name of the variable.
@@ -446,7 +524,7 @@
 In the result, `$1` is the generated name for `substr(user.name, 1)`, while `alias` is the generated name for `user.alias`.
 
 ### <a id="Abbreviated_field_access_expressions">Abbreviated Field Access Expressions</a>
-As in standard SQL, SQL++ field access expressions can be abbreviated (not recommended!) when there is no ambiguity. In the next example, the variable `user` is the only possible variable reference for fields `id`, `name` and `alias` and thus could be omitted in the query. More information on abbbreviated field access can be found in the appendix section on Variable Resolution.
+As in standard SQL, field access expressions can be abbreviated (not recommended!) when there is no ambiguity. In the next example, the variable `user` is the only possible variable reference for fields `id`, `name` and `alias` and thus could be omitted in the query. More information on abbbreviated field access can be found in the appendix section on Variable Resolution.
 
 ##### Example
 
@@ -505,7 +583,7 @@
 Note that if `u.hobbies` is an empty collection or leads to a `MISSING` (as above) or `NULL` value for a given input tuple, there is no corresponding binding value for variable `h` for an input tuple. A `MISSING` value will be generated for `h` so that the input tuple can still be propagated.
 
 ### <a id="Expressing_joins_using_unnests">Expressing Joins Using UNNEST</a>
-The SQL++ `UNNEST` clause is similar to SQL's `JOIN` clause except that it allows its right argument to be correlated to its left argument, as in the examples above --- i.e., think "correlated cross-product".
+The `UNNEST` clause is similar to SQL's `JOIN` clause except that it allows its right argument to be correlated to its left argument, as in the examples above --- i.e., think "correlated cross-product".
 The next example shows this via a query that joins two data sets, GleambookUsers and GleambookMessages, returning user/message pairs. The results contain one object per pair, with result objects containing the user's name and an entire message. The query can be thought of as saying "for each Gleambook user, unnest the `GleambookMessages` collection and filter the output with the condition `message.authorId = user.id`".
 
 ##### Example
@@ -540,7 +618,7 @@
         "message": " like product-y the plan is amazing"
     } ]
 
-Similarly, the above query can also be expressed as the `UNNEST`ing of a correlated SQL++ subquery:
+Similarly, the above query can also be expressed as the `UNNEST`ing of a correlated subquery:
 
 ##### Example
 
@@ -556,7 +634,7 @@
 A `FROM` clause is used for enumerating (i.e., conceptually iterating over) the contents of collections, as in SQL.
 
 ### <a id="Binding_expressions">Binding expressions</a>
-In SQL++, in addition to stored collections, a `FROM` clause can iterate over any intermediate collection returned by a valid SQL++ expression.
+In addition to stored collections, a `FROM` clause can iterate over any intermediate collection returned by a valid query expression.
 In the tuple stream generated by a `FROM` clause, the ordering of the input tuples are not guaranteed to be preserved.
 
 ##### Example
@@ -572,7 +650,7 @@
     ]
 
 ### <a id="Multiple_from_terms">Multiple FROM Terms</a>
-SQL++ permits correlations among `FROM` terms. Specifically, a `FROM` binding expression can refer to variables defined to its left in the given `FROM` clause. Thus, the first unnesting example above could also be expressed as follows:
+The query language permits correlations among `FROM` terms. Specifically, a `FROM` binding expression can refer to variables defined to its left in the given `FROM` clause. Thus, the first unnesting example above could also be expressed as follows:
 
 ##### Example
 
@@ -604,7 +682,8 @@
 
 ### <a id="Implicit_binding_variables">Implicit Binding Variables</a>
 
-Similar to standard SQL, SQL++ supports implicit `FROM` binding variables (i.e., aliases), for which a binding variable is generated. SQL++ variable generation falls into three cases:
+Similar to standard SQL, the query language supports implicit `FROM` binding variables (i.e., aliases), for which a binding variable is generated.
+Variable generation falls into three cases:
 
   * If the binding expression is a variable reference expression, the generated variable's name will be the name of the referenced variable itself.
   * If the binding expression is a field access expression (or a fully qualified name for a dataset), the generated
@@ -662,10 +741,10 @@
 More information on implicit binding variables can be found in the appendix section on Variable Resolution.
 
 ## <a id="Join_clauses">JOIN Clauses</a>
-The join clause in SQL++ supports both inner joins and left outer joins from standard SQL.
+The join clause in the query language supports both inner joins and left outer joins from standard SQL.
 
 ### <a id="Inner_joins">Inner joins</a>
-Using a `JOIN` clause, the inner join intent from the preceeding examples can also be expressed as follows:
+Using a `JOIN` clause, the inner join intent from the preceding examples can also be expressed as follows:
 
 ##### Example
 
@@ -673,7 +752,7 @@
     FROM GleambookUsers u JOIN GleambookMessages m ON m.authorId = u.id;
 
 ### <a id="Left_outer_joins">Left Outer Joins</a>
-SQL++ supports SQL's notion of left outer join. The following query is an example:
+The query language supports SQL's notion of left outer join. The following query is an example:
 
     SELECT u.name AS uname, m.message AS message
     FROM GleambookUsers u LEFT OUTER JOIN GleambookMessages m ON m.authorId = u.id;
@@ -705,7 +784,7 @@
         "uname": "EmoryUnk"
     } ]
 
-For non-matching left-side tuples, SQL++ produces `MISSING` values for the right-side binding variables; that is why the last object in the above result doesn't have a `message` field. Note that this is slightly different from standard SQL, which instead would fill in `NULL` values for the right-side fields. The reason for this difference is that, for non-matches in its join results, SQL++ views fields from the right-side as being "not there" (a.k.a. `MISSING`) instead of as being "there but unknown" (i.e., `NULL`).
+For non-matching left-side tuples, the query language produces `MISSING` values for the right-side binding variables; that is why the last object in the above result doesn't have a `message` field. Note that this is slightly different from standard SQL, which instead would fill in `NULL` values for the right-side fields. The reason for this difference is that, for non-matches in its join results, the query language views fields from the right-side as being "not there" (a.k.a. `MISSING`) instead of as being "there but unknown" (i.e., `NULL`).
 
 The left-outer join query can also be expressed using `LEFT OUTER UNNEST`:
 
@@ -717,13 +796,29 @@
         WHERE message.authorId = u.id
       ) m;
 
-In general, in SQL++, SQL-style join queries can also be expressed by `UNNEST` clauses and left outer join queries can be expressed by `LEFT OUTER UNNESTs`.
+In general, SQL-style join queries can also be expressed by `UNNEST` clauses and left outer join queries can be expressed by `LEFT OUTER UNNESTs`.
+
+### <a id="Join_variable_scope">Variable scope in JOIN clauses</a>
+
+Variables defined by `JOIN` subclauses are not visible to other subclauses in the same `FROM` clause.
+This also applies to the `FROM` variable that starts the `JOIN` subclause.
+
+##### Example
+
+    SELECT * FROM GleambookUsers u
+    JOIN (SELECT VALUE m
+          FROM GleambookMessages m
+          WHERE m.authorId = u.id) m
+    ON u.id = m.authorId;
+
+The variable `u` defined by the `FROM` clause is not visible inside the `JOIN` subclause,
+so this query returns no results.
 
 ## <a id="Group_By_clauses">GROUP BY Clauses</a>
-The SQL++ `GROUP BY` clause generalizes standard SQL's grouping and aggregation semantics, but it also retains backward compatibility with the standard (relational) SQL `GROUP BY` and aggregation features.
+The `GROUP BY` clause generalizes standard SQL's grouping and aggregation semantics, but it also retains backward compatibility with the standard (relational) SQL `GROUP BY` and aggregation features.
 
 ### <a id="Group_variables">Group variables</a>
-In a `GROUP BY` clause, in addition to the binding variable(s) defined for the grouping key(s), SQL++ allows a user to define a *group variable* by using the clause's `GROUP AS` extension to denote the resulting group.
+In a `GROUP BY` clause, in addition to the binding variable(s) defined for the grouping key(s), the query language allows a user to define a *group variable* by using the clause's `GROUP AS` extension to denote the resulting group.
 After grouping, then, the query's in-scope variables include the grouping key's binding variables as well as this group variable which will be bound to one collection value for each group. This per-group collection (i.e., multiset) value will be a set of nested objects in which each field of the object is the result of a renamed variable defined in parentheses following the group variable's name. The `GROUP AS` syntax is as follows:
 
     <GROUP> <AS> Variable ("(" VariableReference <AS> Identifier ("," VariableReference <AS> Identifier )* ")")?
@@ -835,12 +930,12 @@
 This variable contains a collection of objects associated with the group; each of the group's `message` values
 appears in the `msg` field of the objects in the `msgs` collection.
 
-The group variable in SQL++ makes more complex, composable, nested subqueries over a group possible, which is
-important given the more complex data model of SQL++ (relative to SQL).
+The group variable in the query language makes more complex, composable, nested subqueries over a group possible, which is
+important given the language's more complex data model (relative to SQL).
 As a simple example of this, as we really just want the messages associated with each user, we might wish to avoid
-the "extra wrapping" of each message as the `msg` field of a object.
+the "extra wrapping" of each message as the `msg` field of an object.
 (That wrapping is useful in more complex cases, but is essentially just in the way here.)
-We can use a subquery in the `SELECT` clase to tunnel through the extra nesting and produce the desired result.
+We can use a subquery in the `SELECT` clause to tunnel through the extra nesting and produce the desired result.
 
 ##### Example
 
@@ -991,9 +1086,9 @@
     } ]
 
 ### <a id="Implicit_group_key_variables">Implicit Grouping Key Variables</a>
-In the SQL++ syntax, providing named binding variables for `GROUP BY` key expressions is optional.
+In the query language syntax, providing named binding variables for `GROUP BY` key expressions is optional.
 If a grouping key is missing a user-provided binding variable, the underlying compiler will generate one.
-Automatic grouping key variable naming falls into three cases in SQL++, much like the treatment of unnamed projections:
+Automatic grouping key variable naming falls into three cases, much like the treatment of unnamed projections:
 
   * If the grouping key expression is a variable reference expression, the generated variable gets the same name as the referred variable;
   * If the grouping key expression is a field access expression, the generated variable gets the same name as the last identifier in the expression;
@@ -1059,7 +1154,7 @@
 is `authorId` (which is how it is referred to in the example's `SELECT` clause).
 
 ### <a id="Implicit_group_variables">Implicit Group Variables</a>
-The group variable itself is also optional in SQL++'s `GROUP BY` syntax.
+The group variable itself is also optional in the `GROUP BY` syntax.
 If a user's query does not declare the name and structure of the group variable using `GROUP AS`,
 the query compiler will generate a unique group variable whose fields include all of the binding
 variables defined in the `FROM` clause of the current enclosing `SELECT` statement.
@@ -1071,28 +1166,28 @@
 In the traditional SQL, which doesn't support nested data, grouping always also involves the use of aggregation
 to compute properties of the groups (for example, the average number of messages per user rather than the actual set
 of messages per user).
-Each aggregation function in SQL++ takes a collection (for example, the group of messages) as its input and produces
+Each aggregation function in the query language takes a collection (for example, the group of messages) as its input and produces
 a scalar value as its output.
 These aggregation functions, being truly functional in nature (unlike in SQL), can be used anywhere in a
 query where an expression is allowed.
-The following table catalogs the SQL++ built-in aggregation functions and also indicates how each one handles
+The following table catalogs the built-in aggregation functions of the query language and also indicates how each one handles
 `NULL`/`MISSING` values in the input collection or a completely empty input collection:
 
 | Function       | NULL         | MISSING      | Empty Collection |
 |----------------|--------------|--------------|------------------|
-| COLL_COUNT     | counted      | counted      | 0                |
-| COLL_SUM       | returns NULL | returns NULL | returns NULL     |
-| COLL_MAX       | returns NULL | returns NULL | returns NULL     |
-| COLL_MIN       | returns NULL | returns NULL | returns NULL     |
-| COLL_AVG       | returns NULL | returns NULL | returns NULL     |
+| STRICT_COUNT   | counted      | counted      | 0                |
+| STRICT_SUM     | returns NULL | returns NULL | returns NULL     |
+| STRICT_MAX     | returns NULL | returns NULL | returns NULL     |
+| STRICT_MIN     | returns NULL | returns NULL | returns NULL     |
+| STRICT_AVG     | returns NULL | returns NULL | returns NULL     |
 | ARRAY_COUNT    | not counted  | not counted  | 0                |
 | ARRAY_SUM      | ignores NULL | ignores NULL | returns NULL     |
 | ARRAY_MAX      | ignores NULL | ignores NULL | returns NULL     |
 | ARRAY_MIN      | ignores NULL | ignores NULL | returns NULL     |
 | ARRAY_AVG      | ignores NULL | ignores NULL | returns NULL     |
 
-Notice that SQL++ has twice as many functions listed above as there are aggregate functions in SQL-92.
-This is because SQL++ offers two versions of each -- one that handles `UNKNOWN` values in a semantically
+Notice that the query language has twice as many functions listed above as there are aggregate functions in SQL-92.
+This is because the language offers two versions of each -- one that handles `UNKNOWN` values in a semantically
 strict fashion, where unknown values in the input result in unknown values in the output -- and one that
 handles them in the ad hoc "just ignore the unknown values" fashion that the SQL standard chose to adopt.
 
@@ -1130,7 +1225,7 @@
 The query then uses the collection aggregate function ARRAY_COUNT to get the cardinality of each
 group of messages.
 
-Each aggregation function in SQL++ supports DISTINCT modifier that removes duplicate values from
+Each aggregation function in the query language supports DISTINCT modifier that removes duplicate values from
 the input collection.
 
 ##### Example
@@ -1142,11 +1237,11 @@
     6
 
 ### <a id="SQL-92_aggregation_functions">SQL-92 Aggregation Functions</a>
-For compatibility with the traditional SQL aggregation functions, SQL++ also offers SQL-92's
+For compatibility with the traditional SQL aggregation functions, the query language also offers SQL-92's
 aggregation function symbols (`COUNT`, `SUM`, `MAX`, `MIN`, and `AVG`) as supported syntactic sugar.
-The SQL++ compiler rewrites queries that utilize these function symbols into SQL++ queries that only
-use the SQL++ collection aggregate functions. The following example uses the SQL-92 syntax approach
-to compute a result that is identical to that of the more explicit SQL++ example above:
+The query compiler rewrites queries that utilize these function symbols into queries that only
+use the collection aggregate functions of the query language. The following example uses the SQL-92 syntax approach
+to compute a result that is identical to that of the more explicit example above:
 
 ##### Example
 
@@ -1154,8 +1249,8 @@
     FROM GleambookMessages msg
     GROUP BY msg.authorId AS uid;
 
-It is important to realize that `COUNT` is actually **not** a SQL++ built-in aggregation function.
-Rather, the `COUNT` query above is using a special "sugared" function symbol that the SQL++ compiler
+It is important to realize that `COUNT` is actually **not** a built-in aggregation function.
+Rather, the `COUNT` query above is using a special "sugared" function symbol that the query compiler
 will rewrite as follows:
 
     SELECT uid AS uid, ARRAY_COUNT( (SELECT VALUE 1 FROM `$1` as g) ) AS msgCnt
@@ -1165,13 +1260,13 @@
 
 
 The same sort of rewritings apply to the function symbols `SUM`, `MAX`, `MIN`, and `AVG`.
-In contrast to the SQL++ collection aggregate functions, these special SQL-92 function symbols
+In contrast to the collection aggregate functions of the query language, these special SQL-92 function symbols
 can only be used in the same way they are in standard SQL (i.e., with the same restrictions).
 
 DISTINCT modifier is also supported for these aggregate functions.
 
 ### <a id="SQL-92_compliant_gby">SQL-92 Compliant GROUP BY Aggregations</a>
-SQL++ provides full support for SQL-92 `GROUP BY` aggregation queries.
+The query language provides full support for SQL-92 `GROUP BY` aggregation queries.
 The following query is such an example:
 
 ##### Example
@@ -1202,7 +1297,7 @@
     GROUP AS `$1`(msg AS msg);
 
 ### <a id="Column_aliases">Column Aliases</a>
-SQL++ also allows column aliases to be used as `ORDER BY` keys.
+The query language also allows column aliases to be used as `ORDER BY` keys.
 
 ##### Example
 
@@ -1411,15 +1506,15 @@
 
 WITH can be particularly useful when a value needs to be used several times in a query.
 
-Before proceeding further, notice that both  the WITH query and its equivalent inlined variant
-include the syntax "[0]" -- this is due to a noteworthy difference between SQL++ and SQL-92.
+Before proceeding further, notice that both the WITH query and its equivalent inlined variant
+include the syntax "[0]" -- this is due to a noteworthy difference between the query language and SQL-92.
 In SQL-92, whenever a scalar value is expected and it is being produced by a query expression,
 the SQL-92 query processor will evaluate the expression, check that there is only one row and column
 in the result at runtime, and then coerce the one-row/one-column tabular result into a scalar value.
-SQL++, being designed to deal with nested data and schema-less data, does not (and should not) do this.
-Collection-valued data is perfectly legal in most SQL++ contexts, and its data is schema-less,
-so a query processor rarely knows exactly what to expect where and such automatic conversion is often
-not desirable. Thus, in the queries above, the use of "[0]" extracts the first (i.e., 0th) element of
+A JSON query language, being designed to deal with nested data and schema-less data, should not do this.
+Collection-valued data is perfectly legal in most contexts, and its data is schema-less,
+so the query processor rarely knows exactly what to expect where and such automatic conversion would often
+not be desirable. Thus, in the queries above, the use of "[0]" extracts the first (i.e., 0th) element of
 an array-valued query expression's result; this is needed above, even though the result is an array of one
 element, to extract the only element in the singleton array and obtain the desired scalar for the comparison.
 
@@ -1532,7 +1627,7 @@
 ## <a id="Union_all">UNION ALL</a>
 UNION ALL can be used to combine two input arrays or multisets into one. As in SQL, there is no ordering guarantee
 on the contents of the output stream.
-However, unlike SQL, SQL++ does not constrain what the data looks like on the input streams; in particular,
+However, unlike SQL, the query language does not constrain what the data looks like on the input streams; in particular,
 it allows heterogenity on the input and output streams.
 A type error will be raised if one of the inputs is not a collection.
 The following odd but legal query is an example:
@@ -1557,7 +1652,7 @@
      ]
 
 ## <a id="Subqueries">Subqueries</a>
-In SQL++, an arbitrary subquery can appear anywhere that an expression can appear.
+In the query language, an arbitrary subquery can appear anywhere that an expression can appear.
 Unlike SQL-92, as was just alluded to, the subqueries in a SELECT list or a boolean predicate need
 not return singleton, single-column relations.
 Instead, they may return arbitrary collections.
@@ -1601,8 +1696,8 @@
 Note that a subquery, like a top-level `SELECT` statment, always returns a collection -- regardless of where
 within a query the subquery occurs -- and again, its result is never automatically cast into a scalar.
 
-## <a id="Vs_SQL-92">SQL++ vs. SQL-92</a>
-SQL++ offers the following additional features beyond SQL-92 (hence the "++" in its name):
+## <a id="Vs_SQL-92">Differences from SQL-92</a>
+The query language offers the following additional features beyond SQL-92:
 
   * Fully composable and functional: A subquery can iterate over any intermediate collection and can appear anywhere in a query.
   * Schema-free: The query language does not assume the existence of a static schema for any data that it processes.
@@ -1611,9 +1706,9 @@
   * Generalized SELECT clause: A SELECT clause can return any type of collection, while in SQL-92, a `SELECT` clause has to return a (homogeneous) collection of objects.
 
 
-The following matrix is a quick "SQL-92 compatibility cheat sheet" for SQL++.
+The following matrix is a quick "SQL-92 compatibility cheat sheet" for the query language.
 
-| Feature |  SQL++ | SQL-92 |  Why different?  |
+| Feature |  The query language | SQL-92 |  Why different?  |
 |----------|--------|-------|------------------|
 | SELECT * | Returns nested objects | Returns flattened concatenated objects | Nested collections are 1st class citizens |
 | SELECT list | order not preserved | order preserved | Fields in a JSON object are not ordered |
@@ -1624,7 +1719,7 @@
 | String literal | Double quotes or single quotes | Single quotes only | Double quoted strings are pervasive |
 | Delimited identifiers | Backticks | Double quotes | Double quoted strings are pervasive |
 
-The following SQL-92 features are not implemented yet. However, SQL++ does not conflict with these features:
+The following SQL-92 features are not implemented yet. However, the query language does not conflict with these features:
 
   * CROSS JOIN, NATURAL JOIN, UNION JOIN
   * RIGHT and FULL OUTER JOIN
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query_title.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query_title.md
index e35ad29..bc36260 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query_title.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query_title.md
@@ -19,7 +19,7 @@
 
 # <a id="Queries">3. Queries</a>
 
-A SQL++ query can be any legal SQL++ expression or `SELECT` statement. A SQL++ query always ends with a semicolon.
+A query can be any legal expression or `SELECT` statement. A query always ends with a semicolon.
 
     Query ::= (Expression | SelectStatement) ";"
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_error.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_error.md
index e5cea1d..18fce14 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_error.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/4_error.md
@@ -17,7 +17,7 @@
  ! under the License.
  !-->
 
-A SQL++ query can potentially result in one of the following errors:
+A query can potentially result in one of the following errors:
 
  * syntax error,
  * identifier resolution error,
@@ -29,7 +29,7 @@
 immediately return an error message to the client.
 
 ## <a id="Syntax_errors">Syntax Errors</a>
-An valid SQL++ query must satisfy the SQL++ grammar rules.
+A valid query must satisfy the grammar rules of the query language.
 Otherwise, a syntax error will be raised.
 
 ##### Example
@@ -48,7 +48,7 @@
     FROM GleambookUsers user
     WHERE type="advertiser";
 
-Since "type" is a reserved keyword in the SQL++ parser,
+Since "type" is a reserved keyword in the query parser,
 we will get a syntax error as follows:
 
     Error: Syntax error: In line 3 >>WHERE type="advertiser";<< Encountered 'type' "type" at column 7.
@@ -82,8 +82,8 @@
 
 ## <a id="Type_errors">Type Errors</a>
 
-The SQL++ compiler does type checks based on its available type information.
-In addition, the SQL++ runtime also reports type errors if a data model instance
+The query compiler does type checks based on its available type information.
+In addition, the query runtime also reports type errors if a data model instance
 it processes does not satisfy the type requirement.
 
 ##### Example
@@ -93,7 +93,7 @@
 Since function `abs` can only process numeric input values,
 we will get a type error as follows:
 
-    Error: Type mismatch: function abs expects its 1st input parameter to be type tinyint, smallint, integer, bigint, float or double, but the actual input type is string
+    Error: Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string
 
 
 ## <a id="Resource_errors">Resource Errors</a>
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dataset_index.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dataset_index.md
index bdd9706..589b038 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dataset_index.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dataset_index.md
@@ -28,15 +28,15 @@
     QualifiedName       ::= Identifier ( "." Identifier )?
     DoubleQualifiedName ::= Identifier "." Identifier ( "." Identifier )?
 
-The CREATE statement in SQL++ is used for creating dataverses as well as other persistent artifacts in a dataverse.
-It can be used to create new dataverses, datatypes, datasets, indexes, and user-defined SQL++ functions.
+The CREATE statement is used for creating dataverses as well as other persistent artifacts in a dataverse.
+It can be used to create new dataverses, datatypes, datasets, indexes, and user-defined query functions.
 
 ### <a id="Dataverses"> Dataverses</a>
 
     DatabaseSpecification ::= "DATAVERSE" Identifier IfNotExists
 
 The CREATE DATAVERSE statement is used to create new dataverses.
-To ease the authoring of reusable SQL++ scripts, an optional IF NOT EXISTS clause is included to allow
+To ease the authoring of reusable query scripts, an optional IF NOT EXISTS clause is included to allow
 creation to be requested either unconditionally or only if the dataverse does not already exist.
 If this clause is absent, an error is returned if a dataverse with the indicated name already exists.
 
@@ -122,7 +122,7 @@
 
 The CREATE DATASET statement is used to create a new dataset.
 Datasets are named, multisets of object type instances;
-they are where data lives persistently and are the usual targets for SQL++ queries.
+they are where data lives persistently and are the usual targets for queries.
 Datasets are typed, and the system ensures that their contents conform to their type definitions.
 An Internal dataset (the default kind) is a dataset whose content lives within and is managed by the system.
 It is required to have a specified unique primary key field which uniquely identifies the contained objects.
@@ -152,7 +152,7 @@
 
 An External dataset, in contrast to an Internal dataset, has data stored outside of the system's control.
 Files living in HDFS or in the local filesystem(s) of a cluster's nodes are currently supported.
-External dataset support allows SQL++ queries to treat foreign data as though it were stored in the system,
+External dataset support allows queries to treat foreign data as though it were stored in the system,
 making it possible to query "legacy" file data (for example, Hive data) without having to physically import it.
 When defining an External dataset, an appropriate adapter type must be selected for the desired external data.
 (See the [Guide to External Data](../externaldata.html) for more information on the available adapters.)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dml.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dml.md
index f2c404e..2eb5820 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dml.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_dml.md
@@ -23,9 +23,9 @@
 
     InsertStatement ::= <INSERT> <INTO> QualifiedName Query
 
-The SQL++ INSERT statement is used to insert new data into a dataset.
-The data to be inserted comes from a SQL++ query expression.
-This expression can be as simple as a constant expression, or in general it can be any legal SQL++ query.
+The INSERT statement is used to insert new data into a dataset.
+The data to be inserted comes from a query expression.
+This expression can be as simple as a constant expression, or in general it can be any legal query.
 If the target dataset has an auto-generated primary key field, the insert statement should not include a
 value for that field in it.
 (The system will automatically extend the provided object with this additional field and a corresponding value.)
@@ -45,7 +45,7 @@
 
     UpsertStatement ::= <UPSERT> <INTO> QualifiedName Query
 
-The SQL++ UPSERT statement syntactically mirrors the INSERT statement discussed above.
+The UPSERT statement syntactically mirrors the INSERT statement discussed above.
 The difference lies in its semantics, which for UPSERT are "add or replace" instead of the INSERT "add if not present, else error" semantics.
 Whereas an INSERT can fail if another object already exists with the specified key, the analogous UPSERT will replace the previous object's value with that of the new object in such cases.
 
@@ -55,13 +55,13 @@
 
     UPSERT INTO UsersCopy (SELECT VALUE user FROM GleambookUsers user)
 
-*Editor's note: Upserts currently work in AQL but are not yet enabled (at the moment) in SQL++.
+*Editor's note: Upserts currently work in AQL but are not yet enabled (at the moment) in the current query language.
 
 ### <a id="Deletes">DELETEs</a>
 
     DeleteStatement ::= <DELETE> <FROM> QualifiedName ( ( <AS> )? Variable )? ( <WHERE> Expression )?
 
-The SQL++ DELETE statement is used to delete data from a target dataset.
+The DELETE statement is used to delete data from a target dataset.
 The data to be deleted is identified by a boolean expression involving the variable bound to the target dataset in the DELETE statement.
 
 Deletes are processed transactionally by the system.
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_function_removal.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_function_removal.md
index 7b2c603..9fcaa11 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_function_removal.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_function_removal.md
@@ -19,8 +19,8 @@
 
 ### <a id="Functions"> Functions</a>
 
-The create function statement creates a **named** function that can then be used and reused in SQL++ queries.
-The body of a function can be any SQL++ expression involving the function's parameters.
+The create function statement creates a **named** function that can then be used and reused in queries.
+The body of a function can be any query expression involving the function's parameters.
 
     FunctionSpecification ::= "FUNCTION" FunctionOrTypeName IfNotExists ParameterList "{" Expression "}"
 
@@ -44,7 +44,7 @@
                                    | "FUNCTION" FunctionSignature IfExists )
     IfExists            ::= ( "IF" "EXISTS" )?
 
-The DROP statement in SQL++ is the inverse of the CREATE statement. It can be used to drop dataverses, datatypes, datasets, indexes, and functions.
+The DROP statement is the inverse of the CREATE statement. It can be used to drop dataverses, datatypes, datasets, indexes, and functions.
 
 The following examples illustrate some uses of the DROP statement.
 
@@ -64,7 +64,7 @@
 (see the DROP DATASET example above) or from the specified dataverse (see the DROP TYPE example above)
 if one is specified by fully qualifying the artifact name in the DROP statement.
 When specifying an index to drop, the index name must be qualified by the dataset that it indexes.
-When specifying a function to drop, since SQL++ allows functions to be overloaded by their number of arguments,
+When specifying a function to drop, since the query language allows functions to be overloaded by their number of arguments,
 the identifying name of the function to be dropped must explicitly include that information.
 (`friendInfo@1` above denotes the 1-argument function named friendInfo in the current dataverse.)
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_head.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_head.md
index 658190b..83fa4c9 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_head.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/5_ddl_head.md
@@ -30,8 +30,7 @@
                       | DeleteStatement
                       | Query
 
-In addition to queries, an implementation of SQL++ needs to support statements for data definition
-and manipulation purposes as well as controlling the context to be used in evaluating SQL++ expressions.
-This section details the DDL and DML statements supported in the SQL++ language as realized today in
-Apache AsterixDB.
+In addition to queries, an implementation of the query language needs to support statements for data definition
+and manipulation purposes as well as controlling the context to be used in evaluating query expressions.
+This section details the DDL and DML statements supported in the query language as realized today in Apache AsterixDB.
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
index 1357571..c3a6a19 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
@@ -197,6 +197,9 @@
     SELECT e.name, pay
     ORDER BY pay
 
+Note that variables defined by `JOIN` subclauses are not visible to other subclauses in the same `FROM` clause.
+This also applies to the `FROM` variable that starts the `JOIN` subclause.
+
 ## <a id="Resolving_names">Resolving Names</a>
 
 The process of name resolution begins with the leftmost identifier in the name.
diff --git a/asterixdb/asterix-doc/src/site/markdown/api.md b/asterixdb/asterix-doc/src/site/markdown/api.md
index 848f2c3..6853938 100644
--- a/asterixdb/asterix-doc/src/site/markdown/api.md
+++ b/asterixdb/asterix-doc/src/site/markdown/api.md
@@ -43,6 +43,8 @@
   If the delivery mode is `immediate` the query result is returned with the response.
   If the delivery mode is `deferred` the response contains a handle to the <a href="#queryresult">result</a>.
   If the delivery mode is `async` the response contains a handle to the query's <a href="#querystatus">status</a>.
+* `args` - (SQL++ only) A JSON array where each item is a value of a [positional query parameter](sqlpp/manual.html#Parameter_references)
+* `$parameter_name` - (SQL++ only) a JSON value of a [named query parameter](sqlpp/manual.html#Parameter_references).
 
 __Command (immediate result delivery)__
 
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
index bc0b398..1328ed9 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
@@ -44,73 +44,61 @@
         ftcontains(Expression1, Expression2, {FullTextOption})
         ftcontains(Expression1, Expression2)
 
-For example, we can execute the following query to find tweet messages where the `message-text` field includes
+For example, we can execute the following query to find Chirp messages where the `messageText` field includes
 “voice” as a word. Please note that an FTS search is case-insensitive.
 Thus, "Voice" or "voice" will be evaluated as the same word.
 
-        use dataverse TinySocial;
-
-        for $msg in dataset TweetMessages
-        where ftcontains($msg.message-text, "voice", {"mode":"any"})
-        return {"id": $msg.id}
-
-The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
-
-The same query can be also expressed in the SQL++.
-
         use TinySocial;
 
-        select element {"id":msg.id}
-        from TweetMessages as msg
-        where TinySocial.ftcontains(msg.`message-text`, "voice", {"mode":"any"})
+        select element {"chirpId": msg.chirpId}
+        from ChirpMessages msg
+        where ftcontains(msg.messageText, "voice", {"mode":"any"});
+
+The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
 
 The `Expression1` is an expression that should be evaluable as a string at runtime as in the above example
-where `$msg.message-text` is a string field. The `Expression2` can be a string, an (un)ordered list
+where `msg.messageText` is a string field. The `Expression2` can be a string, an (un)ordered list
 of string value(s), or an expression. In the last case, the given expression should be evaluable
 into one of the first two types, i.e., into a string value or an (un)ordered list of string value(s).
 
 The following examples are all valid expressions.
 
-       ... where ftcontains($msg.message-text, "sound")
-       ... where ftcontains($msg.message-text, "sound", {"mode":"any"})
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
-       ... where ftcontains($msg.message-text, {{"speed", "stand", "customization"}}, {"mode":"all"})
-       ... where ftcontains($msg.message-text, let $keyword_list := ["voice", "system"] return $keyword_list, {"mode":"all"})
-       ... where ftcontains($msg.message-text, $keyword_list, {"mode":"any"})
-
-In the last example above, `$keyword_list` should evaluate to a string or an (un)ordered list of string value(s).
+       ... where ftcontains(msg.messageText, "sound")
+       ... where ftcontains(msg.messageText, "sound", {"mode":"any"})
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
+       ... where ftcontains(msg.messageText, {{"speed", "stand", "customization"}}, {"mode":"all"})
 
 The last `FullTextOption` parameter clarifies the given FTS request. If you omit the `FullTextOption` parameter,
 then the default value will be set for each possible option. Currently, we only have one option named `mode`.
 And as we extend the FTS feature, more options will be added. Please note that the format of `FullTextOption`
 is a record, thus you need to put the option(s) in a record `{}`.
 The `mode` option indicates whether the given FTS query is a conjunctive (AND) or disjunctive (OR) search request.
-This option can be either `“any”` or `“all”`. The default value for `mode` is `“all”`. If one specifies `“any”`,
-a disjunctive search will be conducted. For example, the following query will find documents whose `message-text`
+This option can be either `“all”` (AND) or `“any”` (OR). The default value for `mode` is `“all”`. If one specifies `“any”`,
+a disjunctive search will be conducted. For example, the following query will find documents whose `messageText`
 field contains “sound” or “system”, so a document will be returned if it contains either “sound”, “system”,
 or both of the keywords.
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
 
 The other option parameter,`“all”`, specifies a conjunctive search. The following examples will find the documents whose
-`message-text` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
+`messageText` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
 not both, it will not be returned.
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
-       ... where ftcontains($msg.message-text, ["sound", "system"])
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+       ... where ftcontains(msg.messageText, ["sound", "system"])
 
 Currently AsterixDB doesn’t (yet) support phrase searches, so the following query will not work.
 
-       ... where ftcontains($msg.message-text, "sound system", {"mode":"any"})
+       ... where ftcontains(msg.messageText, "sound system", {"mode":"any"})
 
 As a workaround solution, the following query can be used to achieve a roughly similar goal. The difference is that
-the following queries will find documents where `$msg.message-text` contains both “sound” and “system”, but the order
+the following queries will find documents where `msg.messageText` contains both “sound” and “system”, but the order
 and adjacency of “sound” and “system” are not checked, unlike in a phrase search. As a result, the query below would
 also return documents with “sound system can be installed.”, “system sound is perfect.”,
 or “sound is not clear. You may need to install a new system.”
 
-       ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
-       ... where ftcontains($msg.message-text, ["sound", "system"])
+       ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+       ... where ftcontains(msg.messageText, ["sound", "system"])
 
 
 ## <a id="FulltextIndex">Creating and utilizing a Full-text index</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
@@ -118,6 +106,9 @@
 When there is a full-text index on the field that is being searched, rather than scanning all records,
 AsterixDB can utilize that index to expedite the execution of a FTS query. To create a full-text index,
 you need to specify the index type as `fulltext` in your DDL statement. For instance, the following DDL
-statement create a full-text index on the TweetMessages.message-text attribute.
+statement create a full-text index on the `GleambookMessages.message` attribute. Note that a full-text index
+cannot be built on a dataset with the variable-length primary key (e.g., string).
 
-    create index messageFTSIdx on TweetMessages(message-text) type fulltext;
+    use TinySocial;
+
+    create index messageFTSIdx on GleambookMessages(message) type fulltext;
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
index 95c752f..43a7cd5 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
@@ -283,7 +283,7 @@
 ### Arithmetic Expressions
 
     AddExpr  ::= MultExpr ( ( "+" | "-" ) MultExpr )*
-    MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | "^"| "idiv" ) UnaryExpr )*
+    MultExpr ::= UnaryExpr ( ( "*" | "/" | "div" | "%" | "mod" | "^" ) UnaryExpr )*
     UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
 
 AQL also supports the usual cast of characters for arithmetic expressions.
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
index 0d949db..8118126 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
@@ -45,8 +45,8 @@
 AsterixDB supports [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and
 [Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets).  For
 instance, in our
-[TinySocial](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
-example, the `friend-ids` of a Facebook user forms a set
+[TinySocial](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
+example, the `friendIds` of a Gleambook user forms a set
 of friends, and we can define a similarity between the sets of
 friends of two users. We can also convert a string to a set of grams of a length "n"
 (called "n-grams") and define the Jaccard similarity between the two
@@ -55,50 +55,45 @@
 `schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
 
 AsterixDB provides
-[tokenization functions](functions.html#Tokenizing_Functions)
+[tokenization functions](../sqlpp/builtins.html#Tokenizing_Functions)
 to convert strings to sets, and the
-[similarity functions](functions.html#Similarity_Functions).
+[similarity functions](../sqlpp/builtins.html#Similarity_Functions).
 
 ## <a id="SimilaritySelectionQueries">Similarity Selection Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
 The following query
-asks for all the Facebook users whose name is similar to
+asks for all the Gleambook users whose name is similar to
 `Suzanna Tilson`, i.e., their edit distance is at most 2.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance($user.name, "Suzanna Tilson")
-        where $ed <= 2
-        return $user
-
+        select u
+        from GleambookUsers u
+        where edit_distance(u.name, "Suzanna Tilson") <= 2;
 
 The following query
-asks for all the Facebook users whose set of friend ids is
+asks for all the Gleambook users whose set of friend ids is
 similar to `[1,5,9,10]`, i.e., their Jaccard similarity is at least 0.6.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
-        where $sim >= 0.6f
-        return $user
-
+        select u
+        from GleambookUsers u
+        where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
 
 AsterixDB allows a user to use a similarity operator `~=` to express a
 condition by defining the similarity function and threshold
 using "set" statements earlier. For instance, the above query can be
 equivalently written as:
 
-        use dataverse TinySocial;
+        use TinySocial;
 
         set simfunction "jaccard";
         set simthreshold "0.6f";
 
-        for $user in dataset('FacebookUsers')
-        where $user.friend-ids ~= [1,5,9,10]
-        return $user
-
+        select u
+        from GleambookUsers u
+        where u.friendIds ~= [1,5,9,10];
 
 In this query, we first declare Jaccard as the similarity function
 using `simfunction` and then specify the threshold `0.6f` using
@@ -107,27 +102,19 @@
 ## <a id="SimilarityJoinQueries">Similarity Join Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
 AsterixDB supports fuzzy joins between two sets. The following
-[query](primer.html#Query_5_-_Fuzzy_Join)
-finds, for each Facebook user, all Twitter users with names
+[query](../sqlpp/primer-sqlpp.html#Query_5_-_Fuzzy_Join)
+finds, for each Gleambook user, all Chirp users with names
 similar to their name based on the edit distance.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
         set simfunction "edit-distance";
         set simthreshold "3";
 
-        for $fbu in dataset FacebookUsers
-        return {
-            "id": $fbu.id,
-            "name": $fbu.name,
-            "similar-users": for $t in dataset TweetMessages
-                                let $tu := $t.user
-                                where $tu.name ~= $fbu.name
-                                return {
-                                "twitter-screenname": $tu.screen-name,
-                                "twitter-name": $tu.name
-                                }
-        };
+        select gbu.id, gbu.name, (select cu.screenName, cu.name
+                                  from ChirpUsers cu
+                                  where cu.name ~= gbu.name) as similar_users
+        from GleambookUsers gbu;
 
 ## <a id="UsingIndexesToSupportSimilarityQueries">Using Indexes to Support Similarity Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
 
@@ -146,101 +133,95 @@
 [paper](http://www.ics.uci.edu/~chenli/pub/icde2009-memreducer.pdf).
 
 For instance, the following DDL statements create an ngram index on the
-`FacebookUsers.name` attribute using an inverted index of 3-grams.
+`GleambookUsers.name` attribute using an inverted index of 3-grams.
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        create index fbUserIdx on FacebookUsers(name) type ngram(3);
+        create index gbUserIdx on GleambookUsers(name) type ngram(3);
 
 The number "3" in "ngram(3)" is the length "n" in the grams. This
 index can be used to optimize similarity queries on this attribute
 using
-[edit-distance](functions.html#edit-distance),
-[edit-distance-check](functions.html#edit-distance-check),
-[similarity-jaccard](functions.html#similarity-jaccard),
-or [similarity-jaccard-check](functions.html#similarity-jaccard-check)
+[edit_distance](../sqlpp/builtins.html#edit_distance),
+[edit_distance_check](../sqlpp/builtins.html#edit_distance_check),
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard),
+or [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check)
 queries on this attribute where the
 similarity is defined on sets of 3-grams.  This index can also be used
-to optimize queries with the "[contains()]((functions.html#contains))" predicate (i.e., substring
+to optimize queries with the "[contains()]((../sqlpp/builtins.html#contains))" predicate (i.e., substring
 matching) since it can be also be solved by counting on the inverted
 lists of the grams in the query string.
 
-#### NGram Index usage case - [edit-distance](functions.html#edit-distance) ####
+#### NGram Index usage case - [edit_distance](../sqlpp/builtins.html#edit-distance) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance($user.name, "Suzanna Tilson")
-        where $ed <= 2
-        return $user
+        select u
+        from GleambookUsers u
+        where edit_distance(u.name, "Suzanna Tilson") <= 2;
 
-#### NGram Index usage case - [edit-distance-check](functions.html#edit-distance-check) ####
+#### NGram Index usage case - [edit_distance_check](../sqlpp/builtins.html#edit_distance_check) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $ed := edit-distance-check($user.name, "Suzanna Tilson", 2)
-        where $ed[0]
-        return $ed[1]
+        select u
+        from GleambookUsers u
+        where edit_distance_check(u.name, "Suzanna Tilson", 2)[0];
 
-#### NGram Index usage case - [similarity-jaccard](functions.html#similarity-jaccard) ####
+#### NGram Index usage case - [contains()]((../sqlpp/builtins.html#contains)) ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
-        where $sim >= 0.6f
-        return $user
-
-#### NGram Index usage case - [similarity-jaccard-check](functions.html#similarity-jaccard-check) ####
-
-        use dataverse TinySocial;
-
-        for $user in dataset('FacebookUsers')
-        let $sim := similarity-jaccard-check($user.friend-ids, [1,5,9,10], 0.6f)
-        where $sim[0]
-        return $user
-
-#### NGram Index usage case - [contains()]((functions.html#contains)) ####
-
-        use dataverse TinySocial;
-
-        for $i in dataset('FacebookMessages')
-        where contains($i.message, "phone")
-        return {"mid": $i.message-id, "message": $i.message}
+        select m
+        from GleambookMessages m
+        where contains(m.message, "phone");
 
 
 ### Keyword Index ###
 
-A "keyword index" is constructed on a set of strings or sets (e.g., OrderedList, UnorderedList). Instead of
+A "keyword index" is constructed on a set of strings or sets (e.g., array, multiset). Instead of
 generating grams as in an ngram index, we generate tokens (e.g., words) and for each token, construct an inverted list that includes the ids of the
 objects with this token.  The following two examples show how to create keyword index on two different types:
 
 
 #### Keyword Index on String Type ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        drop index FacebookMessages.fbMessageIdx if exists;
-        create index fbMessageIdx on FacebookMessages(message) type keyword;
+        drop index GleambookMessages.gbMessageIdx if exists;
+        create index gbMessageIdx on GleambookMessages(message) type keyword;
 
-        for $o in dataset('FacebookMessages')
-        let $jacc := similarity-jaccard-check(word-tokens($o.message), word-tokens("love like ccast"), 0.2f)
-        where $jacc[0]
-        return $o
+        select m
+        from GleambookMessages m
+        where similarity_jaccard_check(word_tokens(m.message), word_tokens("love like ccast"), 0.2f)[0];
 
-#### Keyword Index on UnorderedList Type ####
+#### Keyword Index on Multiset Type ####
 
-        use dataverse TinySocial;
+        use TinySocial;
 
-        create index fbUserIdx_fids on FacebookUsers(friend-ids) type keyword;
+        create index gbUserIdxFIds on GleambookUsers(friendIds) type keyword;
 
-        for $c in dataset('FacebookUsers')
-        let $jacc := similarity-jaccard-check($c.friend-ids, {{3,10}}, 0.5f)
-        where $jacc[0]
-        return $c
+        select u
+        from GleambookUsers u
+        where similarity_jaccard_check(u.friendIds, {{3,10}}, 0.5f)[0];
 
 As shown above, keyword index can be used to optimize queries with token-based similarity predicates, including
-[similarity-jaccard](functions.html#similarity-jaccard) and
-[similarity-jaccard-check](functions.html#similarity-jaccard-check).
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) and
+[similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check).
+
+#### Keyword Index usage case - [similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) ####
+
+        use TinySocial;
+
+        select u
+        from GleambookUsers u
+        where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
+
+#### Keyword Index usage case - [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check) ####
+
+        use TinySocial;
+
+        select u
+        from GleambookUsers u
+        where similarity_jaccard_check(u.friendIds, [1,5,9,10], 0.6f)[0];
 
diff --git a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
index d2d4488..f5635b8 100644
--- a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
+++ b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
@@ -23,7 +23,7 @@
 
 * [Introduction](#Introduction)
 * [Feed Adapters](#FeedAdapters)
-<!-- * [Feed Policies](#FeedPolicies) -->
+* [Feed Policies](#FeedPolicies)
 
 ## <a name="Introduction">Introduction</a>  ##
 
@@ -61,11 +61,13 @@
 ####Ingesting Twitter Stream
 We shall use the built-in push-based Twitter adapter.
 As a pre-requisite, we must define a Tweet using the AsterixDB Data Model (ADM)
-and the AsterixDB Query Language (AQL). Given below are the type definitions in AQL
+and the query language SQL++. Given below are the type definitions in SQL++
 that create a Tweet datatype which is representative of a real tweet as obtained from Twitter.
 
+        drop dataverse feeds if exists;
+
         create dataverse feeds;
-        use dataverse feeds;
+        use feeds;
 
         create type TwitterUser as closed {
             screen_name: string,
@@ -77,13 +79,12 @@
         create type Tweet as open {
             id: int64,
             user: TwitterUser
-        }
+        };
 
-        create dataset Tweets (Tweet)
-        primary key id;
+        create dataset Tweets (Tweet) primary key id;
 
 We also create a dataset that we shall use to persist the tweets in AsterixDB.
-Next we make use of the `create feed` AQL statement to define our example data feed.
+Next we make use of the `create feed` SQL++ statement to define our example data feed.
 
 #####Using the "push_twitter" feed adapter#####
 The "push_twitter" adapter requires setting up an application account with Twitter. To retrieve
@@ -91,6 +92,7 @@
 a name and a brief description for the application. Each application has associated OAuth
 authentication credentials that include OAuth keys and tokens. Accessing the
 Twitter API requires providing the following.
+
 1. Consumer Key (API Key)
 2. Consumer Secret (API Secret)
 3. Access Token
@@ -101,18 +103,20 @@
 using the "push_twitter" adapter. For further information on obtaining OAuth keys and tokens and
 registering an application with Twitter, please visit http://apps.twitter.com
 
-Given below is an example AQL statement that creates a feed called "TwitterFeed" by using the
+Given below is an example SQL++ statement that creates a feed called "TwitterFeed" by using the
 "push_twitter" adapter.
 
-        use dataverse feeds;
+        use feeds;
 
-        create feed TwitterFeed if not exists using "push_twitter"
-        (("type-name"="Tweet"),
-         ("format"="twitter-status"),
-         ("consumer.key"="************"),
-         ("consumer.secret"="**************"),
-         ("access.token"="**********"),
-         ("access.token.secret"="*************"));
+        create feed TwitterFeed with {
+          "adapter-name": "push_twitter",
+          "type-name": "Tweet",
+          "format": "twitter-status",
+          "consumer.key": "************",
+          "consumer.secret": "************",
+          "access.token": "**********",
+          "access.token.secret": "*************"
+        };
 
 It is required that the above authentication parameters are provided valid.
 Note that the `create feed` statement does not initiate the flow of data from Twitter into
@@ -122,23 +126,25 @@
 
 The Twitter adapter also supports several Twitter streaming APIs as follow:
 
-1. Track filter ("keywords"="AsterixDB, Apache")
-2. Locations filter ("locations"="-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358")
-3. Language filter ("language"="en")
-4. Filter level ("filter-level"="low")
+1. Track filter `"keywords": "AsterixDB, Apache"`
+2. Locations filter `"locations": "-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358"`
+3. Language filter `"language": "en"`
+4. Filter level `"filter-level": "low"`
 
 An example of Twitter adapter tracking tweets with keyword "news" can be described using following ddl:
 
-        use dataverse feeds;
+        use feeds;
 
-        create feed TwitterFeed if not exists using "push_twitter"
-        (("type-name"="Tweet"),
-         ("format"="twitter-status"),
-         ("consumer.key"="************"),
-         ("consumer.secret"="**************"),
-         ("access.token"="**********"),
-         ("access.token.secret"="*************"),
-         ("keywords"="news"));
+        create feed TwitterFeed with {
+          "adapter-name": "push_twitter",
+          "type-name": "Tweet",
+          "format": "twitter-status",
+          "consumer.key": "************",
+          "consumer.secret": "************",
+          "access.token": "**********",
+          "access.token.secret": "*************",
+          "keywords": "news"
+        };
 
 For more details about these APIs, please visit https://dev.twitter.com/streaming/overview/request-parameters
 
@@ -154,7 +160,7 @@
 contents of the dataset represent the union of the connected feeds.
 Also one feed can be simultaneously connected to multiple target datasets.
 
-        use dataverse feeds;
+        use feeds;
 
         connect feed TwitterFeed to dataset Tweets;
 
@@ -170,108 +176,53 @@
 Let the feed run for a minute, then run the following query to see the
 latest tweets that are stored into the data set.
 
-        use dataverse feeds;
+        use feeds;
 
-        for $i in dataset Tweets limit 10 return $i;
+        select * from Tweets limit 10;
 
 The dataflow of data from a feed can be terminated explicitly by `stop feed` statement.
 
-        use dataverse feeds;
+        use feeds;
 
         stop feed TwitterFeed;
 
 The `disconnnect statement` can be used to disconnect the feed from certain dataset.
 
-        use dataverse feeds;
+        use feeds;
 
         disconnect feed TwitterFeed from dataset Tweets;
 
 ###Ingesting with Other Adapters
 AsterixDB has several builtin feed adapters for data ingestion. User can also
 implement their own adapters and plug them into AsterixDB.
-Here we introduce `rss_feed`, `socket_adapter` and `localfs`
+Here we introduce `socket_adapter` and `localfs`
 feed adapter that cover most of the common application scenarios.
 
-#####Using the "rss_feed" feed adapter#####
-`rss_feed` adapter allows retrieving data given a collection of RSS end point URLs.
-As observed in the case of ingesting tweets, it is required to model an RSS data item using AQL.
-
-        use dataverse feeds;
-
-        create type Rss if not exists as open {
-            id: string,
-            title: string,
-            description: string,
-            link: string
-        };
-
-        create dataset RssDataset (Rss)
-        primary key id;
-
-Next, we define an RSS feed using our built-in adapter "rss_feed".
-
-        use dataverse feeds;
-
-        create feed my_feed using
-        rss_feed (
-           ("type-name"="Rss"),
-           ("format"="rss"),
-           ("url"="http://rss.cnn.com/rss/edition.rss")
-        );
-
-In the above definition, the configuration parameter "url" can be a comma-separated list that reflects a
-collection of RSS URLs, where each URL corresponds to an RSS endpoint or an RSS feed.
-The "rss_feed" retrieves data from each of the specified RSS URLs (comma separated values) in parallel.
-
-The following statements connect the feed into the `RssDataset`:
-
-        use dataverse feeds;
-
-        connect feed my_feed to dataset RssDataset;
-
-The following statements activate the feed and start the dataflow:
-
-        use dataverse feeds;
-
-        start feed my_feed;
-
-The following statements show the latest data from the data set, stop the feed, and
-disconnect the feed from the data set.
-
-        use dataverse feeds;
-
-        for $i in dataset RssDataset limit 10 return $i;
-
-        stop feed my_feed
-
-        disconnect feed my_feed from dataset RssDataset;
-
-
 #####Using the "socket_adapter" feed adapter#####
 `socket_adapter` feed opens a web socket on the given node which allows user to push data into
 AsterixDB directly. Here is an example:
 
         drop dataverse feeds if exists;
         create dataverse feeds;
-        use dataverse feeds;
+        use feeds;
 
         create type TestDataType as open {
            screenName: string
-        }
+        };
 
         create dataset TestDataset(TestDataType) primary key screenName;
 
-        create feed TestSocketFeed using socket_adapter
-        (
-           ("sockets"="127.0.0.1:10001"),
-           ("address-type"="IP"),
-           ("type-name"="TestDataType"),
-           ("format"="adm")
-        );
+        create feed TestSocketFeed with {
+          "adapter-name": "socket_adapter",
+          "sockets": "127.0.0.1:10001",
+          "address-type": "IP",
+          "type-name": "TestDataType",
+          "format": "adm"
+        };
 
         connect feed TestSocketFeed to dataset TestDataset;
 
-        use dataverse feeds;
+        use feeds;
         start feed TestSocketFeed;
 
 The above statements create a socket feed which is listening to "10001" port of the host machine. This feed accepts data
@@ -297,25 +248,27 @@
 `localfs` adapter enables data ingestion from local file system. It allows user to feed data records on local disk
 into a dataset. A DDL example for creating a `localfs` feed is given as follow:
 
-        use dataverse feeds;
+        use feeds;
 
-        create type TweetType as closed {
-          id: string,
-          username : string,
-          location : string,
-          text : string,
-          timestamp : string
-        }
+        create type TestDataType as open {
+           screenName: string
+        };
 
-        create dataset Tweets(TweetType)
-        primary key id;
+        create dataset TestDataset(TestDataType) primary key screenName;
 
-        create feed TweetFeed
-        using localfs
-        (("type-name"="TweetType"),("path"="HOSTNAME://LOCAL_FILE_PATH"),("format"="adm"))
+        create feed TestFileFeed with {
+          "adapter-name": "localfs",
+          "type-name": "TestDataType",
+          "path": "HOSTNAME://LOCAL_FILE_PATH",
+          "format": "adm"
+        };
+
+        connect feed TestFileFeed to dataset TestDataset;
+
+        start feed TestFileFeed;
 
 Similar to previous examples, we need to define the datatype and dataset this feed uses.
-The "path" parameter refers to the local datafile that we want to ingest data from.
+The "path" parameter refers to the local data file that we want to ingest data from.
 `HOSTNAME` can either be the IP address or node name of the machine which holds the file.
 `LOCAL_FILE_PATH` indicates the absolute path to the file on that machine. Similarly to `socket_adapter`,
 this feed takes `adm` formatted data records.
@@ -334,7 +287,7 @@
 Use that same datatype in feed definition will cause a type discrepancy since there is no such field in the datasource.
 Thus, we will need to define two separate datatypes for feed and dataset:
 
-        use dataverse feeds;
+        use feeds;
 
         create type DBLPFeedType as closed {
           dblpid: string,
@@ -352,13 +305,13 @@
         }
         create dataset DBLPDataset(DBLPDataSetType) primary key id autogenerated;
 
-        create feed DBLPFeed using socket_adapter
-        (
-            ("sockets"="127.0.0.1:10001"),
-            ("address-type"="IP"),
-            ("type-name"="DBLPFeedType"),
-            ("format"="adm")
-        );
+        create feed DBLPFeed with {
+          "adapter-name": "socket_adapter",
+          "sockets": "127.0.0.1:10001",
+          "address-type": "IP",
+          "type-name": "DBLPFeedType",
+          "format": "adm"
+        };
 
         connect feed DBLPFeed to dataset DBLPDataset;
 
@@ -403,7 +356,6 @@
 ingestion policy is specified as part of the `connect feed` statement
 or else the "Basic" policy will be chosen as the default.
 
-        use dataverse feeds;
+        use feeds;
 
-        connect feed TwitterFeed to dataset Tweets
-        using policy Basic;
\ No newline at end of file
+        connect feed TwitterFeed to dataset Tweets using policy Basic;
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/ncservice.md b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
index 605ee30..2b309ce 100644
--- a/asterixdb/asterix-doc/src/site/markdown/ncservice.md
+++ b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
@@ -366,7 +366,7 @@
 | common  | txn.lock.timeout.sweepthreshold           | Interval (in milliseconds) for checking lock timeout | 10000 |
 | common  | txn.lock.timeout.waitthreshold            | Time out (in milliseconds) of waiting for a lock | 60000 |
 | common  | txn.log.buffer.numpages                   | The number of pages in the transaction log tail | 8 |
-| common  | txn.log.buffer.pagesize                   | The page size (in bytes) for transaction log buffer | 131072 (128 kB) |
+| common  | txn.log.buffer.pagesize                   | The page size (in bytes) for transaction log buffer | 4194304 (4MB) |
 | common  | txn.log.checkpoint.history                | The number of checkpoints to keep in the transaction log | 0 |
 | common  | txn.log.checkpoint.lsnthreshold           | The checkpoint threshold (in terms of LSNs (log sequence numbers) that have been written to the transaction log, i.e., the length of the transaction log) for transaction logs | 67108864 (64 MB) |
 | common  | txn.log.checkpoint.pollfrequency          | The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be written | 120 |
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 1981c18..bb59c0d 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -287,7 +287,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -299,7 +298,7 @@
     <dependency>
       <groupId>org.apache.wicket</groupId>
       <artifactId>wicket-util</artifactId>
-      <version>1.5.2</version>
+      <version>8.2.0</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -402,7 +401,7 @@
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-collections4</artifactId>
-      <version>4.1</version>
+      <version>4.2</version>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
@@ -420,5 +419,13 @@
       <groupId>com.sun.activation</groupId>
       <artifactId>javax.activation</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
index f3d8c89..5dbc383 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataParser.java
@@ -70,7 +70,7 @@
 
             case ARRAY: {
                 OrderedListBuilder listBuilder = new OrderedListBuilder();
-                listBuilder.reset((AOrderedListType) ((AMutableOrderedList) obj).getType());
+                listBuilder.reset((AOrderedListType) obj.getType());
                 IACursor cursor = ((AMutableOrderedList) obj).getCursor();
                 ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
                 while (cursor.next()) {
@@ -85,7 +85,7 @@
 
             case MULTISET: {
                 UnorderedListBuilder listBuilder = new UnorderedListBuilder();
-                listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
+                listBuilder.reset((AUnorderedListType) obj.getType());
                 IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
                 ArrayBackedValueStorage listItemValue = new ArrayBackedValueStorage();
                 while (cursor.next()) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
index 72adc13..d1036b9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
@@ -21,7 +21,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 import java.io.DataOutput;
-import java.io.IOException;
 
 public interface IRecordDataParser<T> extends IDataParser {
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
index 08ffe18..9033814 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
@@ -77,4 +77,11 @@
     public default IFeedMarker getProgressReporter() {
         return null;
     }
+
+    /**
+     * @return JSON String containing ingestion stats
+     */
+    default String getStats() {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
deleted file mode 100644
index 23c5bdd..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.asterix.external.api;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public interface IRecordWithMetaDataAndPKParser<T> extends IRecordDataParser<RecordWithMetadataAndPK<T>> {
-
-    public void parseMeta(RecordWithMetadataAndPK<? extends T> record, DataOutput out) throws IOException;
-
-    public void appendKeys(RecordWithMetadataAndPK<T> record, ArrayTupleBuilder tb) throws IOException;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
deleted file mode 100755
index 9f14ec0..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.asterix.external.api;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.base.AOrderedList;
-import org.apache.asterix.om.base.ARecord;
-import org.apache.asterix.om.base.IAObject;
-
-public interface IResultCollector {
-
-    public void writeIntResult(int result) throws AsterixException;
-
-    public void writeFloatResult(float result) throws AsterixException;
-
-    public void writeDoubleResult(double result) throws AsterixException;
-
-    public void writeStringResult(String result) throws AsterixException;
-
-    public void writeRecordResult(ARecord result) throws AsterixException;
-
-    public void writeListResult(AOrderedList list) throws AsterixException;
-
-    public IAObject getComplexTypeResultHolder();
-
-    public DataOutput getDataOutput();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 164ff68..f392139 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -40,6 +40,7 @@
 public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowController {
     public static final String INCOMING_RECORDS_COUNT_FIELD_NAME = "incoming-records-count";
     public static final String FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME = "failed-at-parser-records-count";
+    public static final String READER_STATS_FIELD_NAME = "reader-stats";
 
     public enum State {
         CREATED,
@@ -257,7 +258,15 @@
 
     @Override
     public String getStats() {
-        return "{\"" + INCOMING_RECORDS_COUNT_FIELD_NAME + "\": " + incomingRecordsCount + ", \""
-                + FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME + "\": " + failedRecordsCount + "}";
+        String readerStats = recordReader.getStats();
+        StringBuilder str = new StringBuilder();
+        str.append("{");
+        if (readerStats != null) {
+            str.append("\"").append(READER_STATS_FIELD_NAME).append("\":").append(readerStats).append(", ");
+        }
+        str.append("\"").append(INCOMING_RECORDS_COUNT_FIELD_NAME).append("\": ").append(incomingRecordsCount)
+                .append(", \"").append(FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME).append("\": ")
+                .append(failedRecordsCount).append("}");
+        return str.toString();
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
index 47efd1d..fab5a8f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
@@ -55,7 +55,7 @@
 
     @Override
     public boolean equals(Object o) {
-        if (o == null || !(o instanceof FeedConnectionId)) {
+        if (!(o instanceof FeedConnectionId)) {
             return false;
         }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
index 6e35841..2f7ed51 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionRequest.java
@@ -20,7 +20,6 @@
 
 import java.io.Serializable;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.common.functions.FunctionSignature;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
index 880c4b4..37b157e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
@@ -20,6 +20,7 @@
 
 import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IActiveEntityEventsListener;
+import org.apache.hyracks.util.Span;
 
 public abstract class AbstractSubscriber implements IActiveEntityEventSubscriber {
 
@@ -55,6 +56,18 @@
         }
     }
 
+    public boolean sync(Span span) throws InterruptedException {
+        synchronized (listener) {
+            while (!done) {
+                span.wait(listener);
+                if (done || span.elapsed()) {
+                    return done;
+                }
+            }
+            return done;
+        }
+    }
+
     public Exception getFailure() {
         return failure;
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
index 8230b48..627e563 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NoOpSubscriber.java
@@ -21,7 +21,6 @@
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.IActiveEntityEventSubscriber;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An event subscriber that does not listen to any events
@@ -49,7 +48,7 @@
     }
 
     @Override
-    public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException {
+    public void subscribed(IActiveEntityEventsListener eventsListener) {
         // no op
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
index a571904..31459d1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StatsSubscriber.java
@@ -30,24 +30,16 @@
     }
 
     @Override
-    public void notify(ActiveEvent event) throws HyracksDataException {
+    public void notify(ActiveEvent event) {
         if (event.getEventKind() == ActiveEvent.Kind.STATS_UPDATED) {
-            try {
-                complete(null);
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
+            complete(null);
         } else if (event.getEventKind() == ActiveEvent.Kind.FAILURE) {
-            try {
-                complete((Exception) event.getEventObject());
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
+            complete((Exception) event.getEventObject());
         }
     }
 
     @Override
-    public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException {
+    public void subscribed(IActiveEntityEventsListener eventsListener) {
         //Does nothing upon subscription
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
index a1cdfb0..818d826 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
@@ -23,39 +23,28 @@
 import org.apache.asterix.active.ActiveEvent;
 import org.apache.asterix.active.ActivityState;
 import org.apache.asterix.active.IActiveEntityEventsListener;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class WaitForStateSubscriber extends AbstractSubscriber {
 
     private final Set<ActivityState> targetStates;
 
-    public WaitForStateSubscriber(IActiveEntityEventsListener listener, Set<ActivityState> targetStates)
-            throws HyracksDataException {
+    public WaitForStateSubscriber(IActiveEntityEventsListener listener, Set<ActivityState> targetStates) {
         super(listener);
         this.targetStates = targetStates;
         listener.subscribe(this);
     }
 
     @Override
-    public void notify(ActiveEvent event) throws HyracksDataException {
+    public void notify(ActiveEvent event) {
         if (targetStates.contains(listener.getState())) {
-            if (listener.getState() == ActivityState.PERMANENTLY_FAILED
-                    || listener.getState() == ActivityState.TEMPORARILY_FAILED) {
-                complete(listener.getJobFailure());
-            } else {
-                complete(null);
-            }
+            complete(listener.getJobFailure());
         } else if (event != null && event.getEventKind() == ActiveEvent.Kind.FAILURE) {
-            try {
-                complete((Exception) event.getEventObject());
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
-            }
+            complete((Exception) event.getEventObject());
         }
     }
 
     @Override
-    public void subscribed(IActiveEntityEventsListener eventsListener) throws HyracksDataException {
+    public void subscribed(IActiveEntityEventsListener eventsListener) {
         if (targetStates.contains(listener.getState())) {
             complete(null);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
index edac0fa..f22693a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingScheduler.java
@@ -33,8 +33,6 @@
 
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hyracks.api.client.HyracksConnection;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -53,19 +51,14 @@
     /** a map from the NC name to the index */
     private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
 
-    /** a map from NC name to the NodeControllerInfo */
-    private Map<String, NodeControllerInfo> ncNameToNcInfos;
-
     /**
      * The constructor of the scheduler.
      *
      * @param ncNameToNcInfos
      * @throws HyracksException
      */
-    public IndexingScheduler(String ipAddress, int port) throws HyracksException {
+    public IndexingScheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
         try {
-            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
-            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
             loadIPAddressToNCMap(ncNameToNcInfos);
         } catch (Exception e) {
             throw HyracksException.create(e);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithPK.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithPK.java
index df69aa6..758d976 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithPK.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithPK.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.external.input.record;
 
-import java.io.IOException;
-
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
index 1ddeedb..199d05a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/TextLookupReader.java
@@ -60,6 +60,6 @@
         if (reader == null) {
             reader = new HDFSTextLineReader();
         }
-        reader.resetReader(fs.open(new Path(file.getFileName())));;
+        reader.resetReader(fs.open(new Path(file.getFileName())));
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
index ffffbd7..3c63281 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterPushRecordReader.java
@@ -27,17 +27,9 @@
 import org.apache.asterix.external.input.record.GenericRecord;
 import org.apache.asterix.external.util.FeedLogManager;
 import org.apache.asterix.external.util.TwitterUtil;
-import twitter4j.DirectMessage;
+
 import twitter4j.FilterQuery;
-import twitter4j.StallWarning;
-import twitter4j.Status;
-import twitter4j.StatusDeletionNotice;
-import twitter4j.StatusListener;
-import twitter4j.TwitterObjectFactory;
 import twitter4j.TwitterStream;
-import twitter4j.User;
-import twitter4j.UserList;
-import twitter4j.UserStreamListener;
 
 public class TwitterPushRecordReader implements IRecordReader<String> {
     private LinkedBlockingQueue<String> inputQ;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
index 36781fe..2207bd7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
@@ -55,7 +55,7 @@
     public void setFeedLogManager(FeedLogManager logManager) throws HyracksDataException {
         super.setFeedLogManager(logManager);
         watcher.setFeedLogManager(logManager);
-    };
+    }
 
     @Override
     public void close() throws IOException {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketClientInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketClientInputStream.java
index b49c11b..039f457 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketClientInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketClientInputStream.java
@@ -21,7 +21,6 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.Socket;
-import java.net.UnknownHostException;
 
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.hyracks.algebricks.common.utils.Pair;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
index 0a706d1..2678a8d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
@@ -306,7 +306,7 @@
                 throws HyracksDataException {
             byte[] b = pointable.getByteArray();
             int s = pointable.getStartOffset();
-            Boolean v = ABooleanSerializerDeserializer.getBoolean(b, s);
+            Boolean v = ABooleanSerializerDeserializer.getBoolean(b, s + 1);
             IJObject jObject = objectPool.allocate(BuiltinType.ABOOLEAN);
             ((JBoolean) jObject).setValue(v);
             return jObject;
@@ -320,7 +320,7 @@
                 throws HyracksDataException {
             byte[] b = pointable.getByteArray();
             int s = pointable.getStartOffset();
-            int v = ADateSerializerDeserializer.getChronon(b, s);
+            int v = ADateSerializerDeserializer.getChronon(b, s + 1);
             IJObject jObject = objectPool.allocate(BuiltinType.ADATE);
             ((JDate) jObject).setValue(v);
             return jObject;
@@ -334,7 +334,7 @@
                 throws HyracksDataException {
             byte[] b = pointable.getByteArray();
             int s = pointable.getStartOffset();
-            long v = ADateTimeSerializerDeserializer.getChronon(b, s);
+            long v = ADateTimeSerializerDeserializer.getChronon(b, s + 1);
             IJObject jObject = objectPool.allocate(BuiltinType.ADATETIME);
             ((JDateTime) jObject).setValue(v);
             return jObject;
@@ -350,7 +350,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             ADuration duration = ADurationSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             IJObject jObject = objectPool.allocate(BuiltinType.ADURATION);
             ((JDuration) jObject).setValue(duration.getMonths(), duration.getMilliseconds());
             return jObject;
@@ -364,7 +364,7 @@
                 throws HyracksDataException {
             byte[] b = pointable.getByteArray();
             int s = pointable.getStartOffset();
-            int v = ATimeSerializerDeserializer.getChronon(b, s);
+            int v = ATimeSerializerDeserializer.getChronon(b, s + 1);
             IJObject jObject = objectPool.allocate(BuiltinType.ATIME);
             ((JTime) jObject).setValue(v);
             return jObject;
@@ -378,9 +378,9 @@
                 throws HyracksDataException {
             byte[] b = pointable.getByteArray();
             int s = pointable.getStartOffset();
-            long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(b, s);
-            long intervalEnd = AIntervalSerializerDeserializer.getIntervalEnd(b, s);
-            byte intervalType = AIntervalSerializerDeserializer.getIntervalTimeType(b, s);
+            long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(b, s + 1);
+            long intervalEnd = AIntervalSerializerDeserializer.getIntervalEnd(b, s + 1);
+            byte intervalType = AIntervalSerializerDeserializer.getIntervalTimeType(b, s + 1);
             IJObject jObject = objectPool.allocate(BuiltinType.AINTERVAL);
             ((JInterval) jObject).setValue(intervalStart, intervalEnd, intervalType);
             return jObject;
@@ -398,7 +398,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             ACircle v = ACircleSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JPoint jpoint = (JPoint) objectPool.allocate(BuiltinType.APOINT);
             jpoint.setValue(v.getP().getX(), v.getP().getY());
             IJObject jObject = objectPool.allocate(BuiltinType.ACIRCLE);
@@ -416,7 +416,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             APoint v = APointSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JPoint jObject = (JPoint) objectPool.allocate(BuiltinType.APOINT);
             jObject.setValue(v.getX(), v.getY());
             return jObject;
@@ -432,7 +432,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             APoint3D v = APoint3DSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JPoint3D jObject = (JPoint3D) objectPool.allocate(BuiltinType.APOINT3D);
             jObject.setValue(v.getX(), v.getY(), v.getZ());
             return jObject;
@@ -448,7 +448,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             ALine v = ALineSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JLine jObject = (JLine) objectPool.allocate(BuiltinType.ALINE);
             jObject.setValue(v.getP1(), v.getP2());
             return jObject;
@@ -464,7 +464,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             APolygon v = APolygonSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JPolygon jObject = (JPolygon) objectPool.allocate(BuiltinType.APOLYGON);
             jObject.setValue(v.getPoints());
             return jObject;
@@ -480,7 +480,7 @@
             int s = pointable.getStartOffset();
             int l = pointable.getLength();
             ARectangle v = ARectangleSerializerDeserializer.INSTANCE
-                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s, l)));
+                    .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
             JRectangle jObject = (JRectangle) objectPool.allocate(BuiltinType.ARECTANGLE);
             jObject.setValue(v.getP1(), v.getP2());
             return jObject;
@@ -498,8 +498,8 @@
         public JRecordAccessor(ARecordType recordType, IObjectPool<IJObject, IAType> objectPool) {
             this.typeInfo = new TypeInfo(objectPool, null, null);
             this.jObjects = new IJObject[recordType.getFieldNames().length];
-            this.jRecord = new JRecord(recordType, jObjects);
             this.openFields = new LinkedHashMap<>();
+            this.jRecord = new JRecord(recordType, jObjects, openFields);
         }
 
         @Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
index 79dc396..df4c093 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
@@ -18,8 +18,11 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -36,7 +39,8 @@
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 
 /**
@@ -74,10 +78,13 @@
                 // Open the index
                 indexHelper.open();
                 try {
-                    IIndex index = indexHelper.getIndexInstance();
+                    ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
+                    Map<String, Object> parameters = new HashMap<>();
+                    parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+                            LSMComponentId.DEFAULT_COMPONENT_ID);
                     // Create bulk loader
                     IIndexBulkLoader bulkLoader =
-                            index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false);
+                            index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false, parameters);
                     // Load files
                     for (ExternalFile file : files) {
                         bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
index 4bc2867..ea99993 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
@@ -18,7 +18,9 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -69,9 +71,10 @@
                 indexHelper.open();
                 IIndex index = indexHelper.getIndexInstance();
                 LSMTwoPCBTreeBulkLoader bulkLoader = null;
+                Map<String, Object> parameters = new HashMap<>();
                 try {
                     bulkLoader = (LSMTwoPCBTreeBulkLoader) ((ExternalBTree) index)
-                            .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size());
+                            .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), parameters);
                     // Load files
                     // The files must be ordered according to their numbers
                     for (ExternalFile file : files) {
@@ -86,7 +89,7 @@
                             case NO_OP:
                                 break;
                             default:
-                                throw HyracksDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP,
+                                throw HyracksDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP, sourceLoc,
                                         file.getPendingOp());
                         }
                     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
index 573de5d..74bc0dc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
@@ -18,12 +18,18 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 
 public class ExternalIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
 
@@ -43,4 +49,12 @@
         super.open();
         ((ITwoPCIndex) index).setCurrentVersion(version);
     }
+
+    @Override
+    protected void initializeBulkLoader() throws HyracksDataException {
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
+        bulkLoader = ((ILSMIndex) index).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                parameters);
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
index aaca3f1..57e2917 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.external.operators;
 
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.asterix.external.indexing.FilesIndexDescription;
 import org.apache.asterix.om.base.AMutableInt32;
@@ -60,8 +62,9 @@
         try {
             writer.open();
             // Transactional BulkLoader
-            bulkLoader =
-                    ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length);
+            Map<String, Object> parameters = new HashMap<>();
+            bulkLoader = ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length,
+                    parameters);
             // Delete files
             for (int i = 0; i < deletedFiles.length; i++) {
                 fileNumber.setValue(deletedFiles[i]);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
index 867fb60..98f75df 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.external.operators;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.ActiveSourceOperatorNodePushable;
 import org.apache.asterix.active.EntityId;
@@ -44,8 +46,6 @@
  */
 public class FeedIntakeOperatorNodePushable extends ActiveSourceOperatorNodePushable {
     private static final Logger LOGGER = LogManager.getLogger();
-    // TODO: Make configurable https://issues.apache.org/jira/browse/ASTERIXDB-2065
-    public static final int DEFAULT_ABORT_TIMEOUT = 60000;
     private final FeedIntakeOperatorDescriptor opDesc;
     private final FeedAdapter adapter;
     private boolean poisoned = false;
@@ -125,12 +125,12 @@
     }
 
     @Override
-    protected void abort() throws HyracksDataException, InterruptedException {
+    protected void abort(long timeout, TimeUnit unit) throws HyracksDataException, InterruptedException {
         LOGGER.info(runtimeId + " aborting...");
         synchronized (this) {
             poisoned = true;
             try {
-                if (!adapter.stop(DEFAULT_ABORT_TIMEOUT)) {
+                if (!adapter.stop(unit.toMillis(timeout))) {
                     LOGGER.info(runtimeId + " failed to stop adapter. interrupting the thread...");
                     taskThread.interrupt();
                 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index c0ccf11..f1eba4c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.Serializable;
 import java.util.BitSet;
 import java.util.List;
 
@@ -52,6 +51,7 @@
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.asterix.om.util.container.ListObjectPool;
 import org.apache.asterix.runtime.operators.file.adm.AdmLexer;
+import org.apache.asterix.runtime.operators.file.adm.AdmLexer.TokenImage;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -72,64 +72,11 @@
     private final IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
             new ListObjectPool<IMutableValueStorage, ATypeTag>(new AbvsBuilderFactory());
 
+    private final TokenImage tmpTokenImage = new TokenImage();
+
     private final String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
     private final String mismatchErrorMessage2 = " got a value of type ";
 
-    static class ParseException extends HyracksDataException {
-        private static final long serialVersionUID = 1L;
-        private String filename;
-        private int line = -1;
-        private int column = -1;
-
-        public ParseException(String message) {
-            super(message);
-        }
-
-        public ParseException(int errorCode, Serializable... param) {
-            super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
-        }
-
-        public ParseException(int errorCode, Throwable e, Serializable... param) {
-            super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
-            addSuppressed(e);
-        }
-
-        public ParseException(Throwable cause) {
-            super(cause);
-        }
-
-        public ParseException(String message, Throwable cause) {
-            super(message, cause);
-        }
-
-        public ParseException(Throwable cause, String filename, int line, int column) {
-            super(cause);
-            setLocation(filename, line, column);
-        }
-
-        public void setLocation(String filename, int line, int column) {
-            this.filename = filename;
-            this.line = line;
-            this.column = column;
-        }
-
-        @Override
-        public String getMessage() {
-            StringBuilder msg = new StringBuilder("Parse error");
-            if (filename != null) {
-                msg.append(" in file " + filename);
-            }
-            if (line >= 0) {
-                if (column >= 0) {
-                    msg.append(" at (" + line + ", " + column + ")");
-                } else {
-                    msg.append(" in line " + line);
-                }
-            }
-            return msg.append(": " + super.getMessage()).toString();
-        }
-    }
-
     public ADMDataParser(ARecordType recordType, boolean isStream) {
         this(null, recordType, isStream);
     }
@@ -254,16 +201,26 @@
                 break;
             case AdmLexer.TOKEN_STRING_LITERAL:
                 if (checkType(ATypeTag.STRING, objectType)) {
-                    String tokenImage =
-                            admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    aString.setValue(admLexer.containsEscapes() ? replaceEscapes(tokenImage) : tokenImage);
-                    stringSerde.serialize(aString, out);
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    if (admLexer.containsEscapes()) {
+                        replaceEscapes(tmpTokenImage);
+                    }
+                    int begin = tmpTokenImage.getBegin() + 1;
+                    int len = tmpTokenImage.getLength() - 2;
+                    parseString(tmpTokenImage.getBuffer(), begin, len, out);
                 } else if (checkType(ATypeTag.UUID, objectType)) {
                     // Dealing with UUID type that is represented by a string
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    aUUID.parseUUIDString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                            tmpTokenImage.getLength() - 2);
+                    uuidSerde.serialize(aUUID, out);
+                } else if (checkType(ATypeTag.GEOMETRY, objectType)) {
+                    // Parse the string as a WKT-encoded geometry
                     String tokenImage =
                             admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    aUUID.parseUUIDString(tokenImage);
-                    uuidSerde.serialize(aUUID, out);
+                    aGeomtry.parseWKT(tokenImage);
+                    out.writeByte(ATypeTag.GEOMETRY.serialize());
+                    geomSerde.serialize(aGeomtry, out);
                 } else {
                     throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_TYPE_MISMATCH, objectType.getTypeName());
                 }
@@ -364,13 +321,14 @@
 
     }
 
-    private String replaceEscapes(String tokenImage) throws ParseException {
-        char[] chars = tokenImage.toCharArray();
-        int len = chars.length;
-        int readpos = 0;
-        int writepos = 0;
-        int movemarker = 0;
-        while (readpos < len) {
+    // TODO: This function should be optimized. Currently it has complexity of O(N*N)!
+    private void replaceEscapes(TokenImage tokenImage) throws ParseException {
+        char[] chars = tokenImage.getBuffer();
+        int end = tokenImage.getBegin() + tokenImage.getLength();
+        int readpos = tokenImage.getBegin();
+        int writepos = tokenImage.getBegin();
+        int movemarker = tokenImage.getBegin();
+        while (readpos < end) {
             if (chars[readpos] == '\\') {
                 moveChars(chars, movemarker, readpos, readpos - writepos);
                 switch (chars[readpos + 1]) {
@@ -407,8 +365,8 @@
             ++writepos;
             ++readpos;
         }
-        moveChars(chars, movemarker, len, readpos - writepos);
-        return new String(chars, 0, len - (readpos - writepos));
+        moveChars(chars, movemarker, end, readpos - writepos);
+        tokenImage.reset(chars, tokenImage.getBegin(), tokenImage.getLength() - (readpos - writepos));
     }
 
     private static void moveChars(char[] chars, int start, int end, int offset) {
@@ -508,16 +466,16 @@
                     expectingRecordField = false;
 
                     if (recType != null) {
-                        String fldName =
-                                admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
+                        admLexer.getLastTokenImage(tmpTokenImage);
+                        String fldName = new String(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                tmpTokenImage.getLength() - 2);
                         fieldId = recBuilder.getFieldId(fldName);
                         if ((fieldId < 0) && !recType.isOpen()) {
                             throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_EXTRA_FIELD_IN_CLOSED_RECORD,
                                     fldName);
                         } else if ((fieldId < 0) && recType.isOpen()) {
-                            aStringFieldName.setValue(admLexer.getLastTokenImage().substring(1,
-                                    admLexer.getLastTokenImage().length() - 1));
-                            stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+                            parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                    tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
                             openRecordField = true;
                             fieldType = null;
                         } else {
@@ -527,9 +485,9 @@
                             openRecordField = false;
                         }
                     } else {
-                        aStringFieldName.setValue(
-                                admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1));
-                        stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+                        admLexer.getLastTokenImage(tmpTokenImage);
+                        parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+                                tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
                         openRecordField = true;
                         fieldType = null;
                     }
@@ -807,7 +765,13 @@
 
     private void parseToNumericTarget(ATypeTag typeTag, IAType objectType, DataOutput out) throws IOException {
         ATypeTag targetTypeTag = getTargetTypeTag(typeTag, objectType);
-        if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), targetTypeTag, out)) {
+        boolean parsed = false;
+        if (targetTypeTag != null) {
+            admLexer.getLastTokenImage(tmpTokenImage);
+            parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(),
+                    targetTypeTag, out);
+        }
+        if (!parsed) {
             throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
         }
     }
@@ -819,8 +783,13 @@
             castBuffer.reset();
             dataOutput = castBuffer.getDataOutput();
         }
-
-        if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), typeTag, dataOutput)) {
+        boolean parsed = false;
+        if (targetTypeTag != null) {
+            admLexer.getLastTokenImage(tmpTokenImage);
+            parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(), typeTag,
+                    dataOutput);
+        }
+        if (!parsed) {
             throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
         }
 
@@ -862,9 +831,11 @@
             if (token == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                 token = admLexer.next();
                 if (token == AdmLexer.TOKEN_STRING_LITERAL) {
-                    String unquoted =
-                            admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
-                    if (!parseValue(unquoted, typeTag, dataOutput)) {
+                    admLexer.getLastTokenImage(tmpTokenImage);
+                    int begin = tmpTokenImage.getBegin() + 1;
+                    int len = tmpTokenImage.getLength() - 2;
+                    // unquoted value
+                    if (!parseValue(tmpTokenImage.getBuffer(), begin, len, typeTag, dataOutput)) {
                         throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_CONSTRUCTOR_MISSING_DESERIALIZER,
                                 AdmLexer.tokenKindToString(token));
                     }
@@ -890,85 +861,86 @@
                 objectType.getTypeName() + " got " + typeTag);
     }
 
-    private boolean parseValue(String unquoted, ATypeTag typeTag, DataOutput out) throws HyracksDataException {
+    private boolean parseValue(char[] buffer, int begin, int len, ATypeTag typeTag, DataOutput out)
+            throws HyracksDataException {
         switch (typeTag) {
             case BOOLEAN:
-                parseBoolean(unquoted, out);
+                parseBoolean(buffer, begin, len, out);
                 return true;
             case TINYINT:
-                parseInt8(unquoted, out);
+                parseInt8(buffer, begin, len, out);
                 return true;
             case SMALLINT:
-                parseInt16(unquoted, out);
+                parseInt16(buffer, begin, len, out);
                 return true;
             case INTEGER:
-                parseInt32(unquoted, out);
+                parseInt32(buffer, begin, len, out);
                 return true;
             case BIGINT:
-                parseInt64(unquoted, out);
+                parseInt64(buffer, begin, len, out);
                 return true;
             case FLOAT:
-                if ("INF".equals(unquoted)) {
+                if (matches("INF", buffer, begin, len)) {
                     aFloat.setValue(Float.POSITIVE_INFINITY);
-                } else if ("-INF".equals(unquoted)) {
+                } else if (matches("-INF", buffer, begin, len)) {
                     aFloat.setValue(Float.NEGATIVE_INFINITY);
                 } else {
-                    aFloat.setValue(Float.parseFloat(unquoted));
+                    aFloat.setValue(parseFloat(buffer, begin, len));
                 }
                 floatSerde.serialize(aFloat, out);
                 return true;
             case DOUBLE:
-                if ("INF".equals(unquoted)) {
+                if (matches("INF", buffer, begin, len)) {
                     aDouble.setValue(Double.POSITIVE_INFINITY);
-                } else if ("-INF".equals(unquoted)) {
+                } else if (matches("-INF", buffer, begin, len)) {
                     aDouble.setValue(Double.NEGATIVE_INFINITY);
                 } else {
-                    aDouble.setValue(Double.parseDouble(unquoted));
+                    aDouble.setValue(parseDouble(buffer, begin, len));
                 }
                 doubleSerde.serialize(aDouble, out);
                 return true;
             case STRING:
-                aString.setValue(unquoted);
-                stringSerde.serialize(aString, out);
+                parseString(buffer, begin, len, out);
                 return true;
             case TIME:
-                parseTime(unquoted, out);
+                parseTime(buffer, begin, len, out);
                 return true;
             case DATE:
-                parseDate(unquoted, out);
+                parseDate(buffer, begin, len, out);
                 return true;
             case DATETIME:
-                parseDateTime(unquoted, out);
+                parseDateTime(buffer, begin, len, out);
                 return true;
             case DURATION:
-                parseDuration(unquoted, out);
+                parseDuration(buffer, begin, len, out);
                 return true;
             case DAYTIMEDURATION:
-                parseDateTimeDuration(unquoted, out);
+                parseDateTimeDuration(buffer, begin, len, out);
                 return true;
             case YEARMONTHDURATION:
-                parseYearMonthDuration(unquoted, out);
+                parseYearMonthDuration(buffer, begin, len, out);
                 return true;
             case POINT:
-                parsePoint(unquoted, out);
+                parsePoint(buffer, begin, len, out);
                 return true;
             case POINT3D:
-                parse3DPoint(unquoted, out);
+                parse3DPoint(buffer, begin, len, out);
                 return true;
             case CIRCLE:
-                parseCircle(unquoted, out);
+                parseCircle(buffer, begin, len, out);
                 return true;
             case RECTANGLE:
-                parseRectangle(unquoted, out);
+                parseRectangle(buffer, begin, len, out);
                 return true;
             case LINE:
-                parseLine(unquoted, out);
+                parseLine(buffer, begin, len, out);
                 return true;
             case POLYGON:
-                APolygonSerializerDeserializer.parse(unquoted, out);
+                //TODO: optimize
+                APolygonSerializerDeserializer.parse(new String(buffer, begin, len), out);
                 return true;
             case UUID:
-                aUUID.parseUUIDString(unquoted);
+                aUUID.parseUUIDString(buffer, begin, len);
                 uuidSerde.serialize(aUUID, out);
                 return true;
             default:
@@ -976,39 +948,53 @@
         }
     }
 
-    private void parseBoolean(String bool, DataOutput out) throws HyracksDataException {
-        if (bool.equals("true")) {
+    private boolean matches(String value, char[] buffer, int begin, int len) {
+        if (len != value.length()) {
+            return false;
+        }
+        for (int i = 0; i < len; i++) {
+            if (value.charAt(i) != buffer[i + begin]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private void parseBoolean(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        if (matches("true", buffer, begin, len)) {
             booleanSerde.serialize(ABoolean.TRUE, out);
-        } else if (bool.equals("false")) {
+        } else if (matches("false", buffer, begin, len)) {
             booleanSerde.serialize(ABoolean.FALSE, out);
         } else {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "boolean");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "boolean");
         }
     }
 
-    private void parseInt8(String int8, DataOutput out) throws HyracksDataException {
+    private void parseInt8(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         byte value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int8.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int8.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int8.length(); offset++) {
-            if ((int8.charAt(offset) >= '0') && (int8.charAt(offset) <= '9')) {
-                value = (byte) (((value * 10) + int8.charAt(offset)) - '0');
-            } else if ((int8.charAt(offset) == 'i') && (int8.charAt(offset + 1) == '8')
-                    && ((offset + 2) == int8.length())) {
+        for (; offset < begin + len; offset++) {
+            if ((buffer[offset] >= '0') && (buffer[offset] <= '9')) {
+                value = (byte) (((value * 10) + buffer[offset]) - '0');
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '8' && offset + 2 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int8");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int8");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1017,29 +1003,31 @@
         int8Serde.serialize(aInt8, out);
     }
 
-    private void parseInt16(String int16, DataOutput out) throws HyracksDataException {
+    private void parseInt16(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         short value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int16.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int16.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int16.length(); offset++) {
-            if ((int16.charAt(offset) >= '0') && (int16.charAt(offset) <= '9')) {
-                value = (short) (((value * 10) + int16.charAt(offset)) - '0');
-            } else if ((int16.charAt(offset) == 'i') && (int16.charAt(offset + 1) == '1')
-                    && (int16.charAt(offset + 2) == '6') && ((offset + 3) == int16.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (short) ((value * 10) + buffer[offset] - '0');
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '1' && buffer[offset + 2] == '6'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int16");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int16");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1048,29 +1036,31 @@
         int16Serde.serialize(aInt16, out);
     }
 
-    private void parseInt32(String int32, DataOutput out) throws HyracksDataException {
+    private void parseInt32(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         int value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int32.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int32.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int32.length(); offset++) {
-            if ((int32.charAt(offset) >= '0') && (int32.charAt(offset) <= '9')) {
-                value = (((value * 10) + int32.charAt(offset)) - '0');
-            } else if ((int32.charAt(offset) == 'i') && (int32.charAt(offset + 1) == '3')
-                    && (int32.charAt(offset + 2) == '2') && ((offset + 3) == int32.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (value * 10) + buffer[offset] - '0';
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '3' && buffer[offset + 2] == '2'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int32");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int32");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1080,29 +1070,31 @@
         int32Serde.serialize(aInt32, out);
     }
 
-    private void parseInt64(String int64, DataOutput out) throws HyracksDataException {
+    private void parseInt64(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         boolean positive = true;
         long value = 0;
-        int offset = 0;
+        int offset = begin;
 
-        if (int64.charAt(offset) == '+') {
+        if (buffer[offset] == '+') {
             offset++;
-        } else if (int64.charAt(offset) == '-') {
+        } else if (buffer[offset] == '-') {
             offset++;
             positive = false;
         }
-        for (; offset < int64.length(); offset++) {
-            if ((int64.charAt(offset) >= '0') && (int64.charAt(offset) <= '9')) {
-                value = (((value * 10) + int64.charAt(offset)) - '0');
-            } else if ((int64.charAt(offset) == 'i') && (int64.charAt(offset + 1) == '6')
-                    && (int64.charAt(offset + 2) == '4') && ((offset + 3) == int64.length())) {
+        for (; offset < begin + len; offset++) {
+            if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+                value = (value * 10) + buffer[offset] - '0';
+            } else if (buffer[offset] == 'i' && buffer[offset + 1] == '6' && buffer[offset + 2] == '4'
+                    && offset + 3 == begin + len) {
                 break;
             } else {
-                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+                        new String(buffer, begin, len), "int64");
             }
         }
         if (value < 0) {
-            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+                    "int64");
         }
         if ((value > 0) && !positive) {
             value *= -1;
@@ -1127,4 +1119,4 @@
         admLexer.reInit(new InputStreamReader(in));
         return true;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
index 8351931..859ac22 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
@@ -19,7 +19,10 @@
 package org.apache.asterix.external.parser;
 
 import java.io.DataOutput;
+import java.io.IOException;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.external.api.IDataParser;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABinary;
@@ -31,6 +34,7 @@
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.ADuration;
 import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
@@ -45,6 +49,7 @@
 import org.apache.asterix.om.base.AMutableDouble;
 import org.apache.asterix.om.base.AMutableDuration;
 import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableGeometry;
 import org.apache.asterix.om.base.AMutableInt16;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
@@ -71,11 +76,14 @@
 import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import org.apache.asterix.om.base.temporal.ATimeParserFactory;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.bytes.Base64Parser;
 import org.apache.hyracks.util.bytes.HexParser;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
 
 /**
  * Base class for data parsers. Includes the common set of definitions for
@@ -93,6 +101,7 @@
     protected AMutableBinary aBinary = new AMutableBinary(null, 0, 0);
     protected AMutableString aStringFieldName = new AMutableString("");
     protected AMutableUUID aUUID = new AMutableUUID();
+    protected AMutableGeometry aGeomtry = new AMutableGeometry(null);
     // For temporal and spatial data types
     protected AMutableTime aTime = new AMutableTime(0);
     protected AMutableDateTime aDateTime = new AMutableDateTime(0L);
@@ -140,6 +149,9 @@
     protected ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
+    protected final AStringSerializerDeserializer untaggedStringSerde =
+            new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
+
     protected final HexParser hexParser = new HexParser();
     protected final Base64Parser base64Parser = new Base64Parser();
 
@@ -150,6 +162,9 @@
     protected ISerializerDeserializer<AUUID> uuidSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AUUID);
 
+    protected ISerializerDeserializer<AGeometry> geomSerde =
+            SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+
     // To avoid race conditions, the serdes for temporal and spatial data types needs to be one per parser
     // ^^^^^^^^^^^^^^^^^^^^^^^^ ??? then why all these serdes are static?
     @SuppressWarnings("unchecked")
@@ -195,14 +210,14 @@
         this.filename = filename;
     }
 
-    protected void parseTime(String time, DataOutput out) throws HyracksDataException {
-        int chrononTimeInMs = ATimeParserFactory.parseTimePart(time, 0, time.length());
+    protected void parseTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        int chrononTimeInMs = ATimeParserFactory.parseTimePart(buffer, begin, len);
         aTime.setValue(chrononTimeInMs);
         timeSerde.serialize(aTime, out);
     }
 
-    protected void parseDate(String date, DataOutput out) throws HyracksDataException {
-        long chrononTimeInMs = ADateParserFactory.parseDatePart(date, 0, date.length());
+    protected void parseDate(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, len);
         short temp = 0;
         if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
             temp = 1;
@@ -211,84 +226,93 @@
         dateSerde.serialize(aDate, out);
     }
 
-    protected void parseDateTime(String datetime, DataOutput out) throws HyracksDataException {
+    protected void parseDateTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         // +1 if it is negative (-)
-        short timeOffset = (short) ((datetime.charAt(0) == '-') ? 1 : 0);
 
-        timeOffset += 8;
+        int timeOffset = (buffer[begin] == '-') ? 1 : 0;
 
-        if (datetime.charAt(timeOffset) != 'T') {
+        timeOffset = timeOffset + 8 + begin;
+
+        if (buffer[timeOffset] != 'T') {
             timeOffset += 2;
-            if (datetime.charAt(timeOffset) != 'T') {
-                throw new HyracksDataException("This can not be an instance of datetime: missing T");
+            if (buffer[timeOffset] != 'T') {
+                throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_INTERVAL_INVALID_DATETIME);
             }
         }
-        long chrononTimeInMs = ADateParserFactory.parseDatePart(datetime, 0, timeOffset);
-        chrononTimeInMs +=
-                ATimeParserFactory.parseTimePart(datetime, timeOffset + 1, datetime.length() - timeOffset - 1);
+        long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, timeOffset - begin);
+        chrononTimeInMs += ATimeParserFactory.parseTimePart(buffer, timeOffset + 1, begin + len - timeOffset - 1);
         aDateTime.setValue(chrononTimeInMs);
         datetimeSerde.serialize(aDateTime, out);
     }
 
-    protected void parseDuration(String duration, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
+    protected void parseDuration(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aDuration, ADurationParseOption.All);
         durationSerde.serialize(aDuration, out);
     }
 
-    protected void parseDateTimeDuration(String durationString, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
-                ADurationParseOption.All);
+    protected void parseDateTimeDuration(char[] buffer, int begin, int len, DataOutput out)
+            throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aDayTimeDuration, ADurationParseOption.All);
         dayTimeDurationSerde.serialize(aDayTimeDuration, out);
     }
 
-    protected void parseYearMonthDuration(String durationString, DataOutput out) throws HyracksDataException {
-        ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
-                ADurationParseOption.All);
+    protected void parseYearMonthDuration(char[] buffer, int begin, int len, DataOutput out)
+            throws HyracksDataException {
+        ADurationParserFactory.parseDuration(buffer, begin, len, aYearMonthDuration, ADurationParseOption.All);
         yearMonthDurationSerde.serialize(aYearMonthDuration, out);
     }
 
-    protected void parsePoint(String point, DataOutput out) throws HyracksDataException {
+    protected void parsePoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            aPoint.setValue(Double.parseDouble(point.substring(0, point.indexOf(','))),
-                    Double.parseDouble(point.substring(point.indexOf(',') + 1, point.length())));
+            int commaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, commaIndex - begin),
+                    parseDouble(buffer, commaIndex + 1, begin + len - commaIndex - 1));
             pointSerde.serialize(aPoint, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point + " can not be an instance of point");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "point");
         }
     }
 
-    protected void parse3DPoint(String point3d, DataOutput out) throws HyracksDataException {
+    protected void parse3DPoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            int firstCommaIndex = point3d.indexOf(',');
-            int secondCommaIndex = point3d.indexOf(',', firstCommaIndex + 1);
-            aPoint3D.setValue(Double.parseDouble(point3d.substring(0, firstCommaIndex)),
-                    Double.parseDouble(point3d.substring(firstCommaIndex + 1, secondCommaIndex)),
-                    Double.parseDouble(point3d.substring(secondCommaIndex + 1, point3d.length())));
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            int secondCommaIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ',');
+            aPoint3D.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, secondCommaIndex - firstCommaIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             point3DSerde.serialize(aPoint3D, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point3d + " can not be an instance of point3d");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "point3d");
         }
     }
 
-    protected void parseCircle(String circle, DataOutput out) throws HyracksDataException {
+    protected void parseCircle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] parts = circle.split(" ");
-            aPoint.setValue(Double.parseDouble(parts[0].split(",")[0]), Double.parseDouble(parts[0].split(",")[1]));
-            aCircle.setValue(aPoint, Double.parseDouble(parts[1].substring(0, parts[1].length())));
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            int spaceIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ' ');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+            aCircle.setValue(aPoint, parseDouble(buffer, spaceIndex + 1, begin + len - spaceIndex - 1));
             circleSerde.serialize(aCircle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(circle + " can not be an instance of circle");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "circle");
         }
     }
 
-    protected void parseRectangle(String rectangle, DataOutput out) throws HyracksDataException {
+    protected void parseRectangle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] points = rectangle.split(" ");
-            if (points.length != 2) {
-                throw new HyracksDataException("rectangle consists of only 2 points.");
-            }
-            aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
-            aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+            int spaceIndex = indexOf(buffer, begin, len, ' ');
+
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+
+            int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+            aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             if (aPoint.getX() > aPoint2.getX() && aPoint.getY() > aPoint2.getY()) {
                 aRectangle.setValue(aPoint2, aPoint);
             } else if (aPoint.getX() < aPoint2.getX() && aPoint.getY() < aPoint2.getY()) {
@@ -298,23 +322,26 @@
                         "Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
             }
             rectangleSerde.serialize(aRectangle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(rectangle + " can not be an instance of rectangle");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "rectangle");
         }
     }
 
-    protected void parseLine(String line, DataOutput out) throws HyracksDataException {
+    protected void parseLine(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
         try {
-            String[] points = line.split(" ");
-            if (points.length != 2) {
-                throw new HyracksDataException("line consists of only 2 points.");
-            }
-            aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
-            aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+            int spaceIndex = indexOf(buffer, begin, len, ' ');
+            int firstCommaIndex = indexOf(buffer, begin, len, ',');
+            aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+                    parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+            int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+            aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+                    parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
             aLine.setValue(aPoint, aPoint2);
             lineSerde.serialize(aLine, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(line + " can not be an instance of line");
+        } catch (Exception e) {
+            throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+                    "line");
         }
     }
 
@@ -357,4 +384,34 @@
 
         return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
     }
-}
+
+    protected double parseDouble(char[] buffer, int begin, int len) {
+        // TODO: parse double directly from char[]
+        String str = new String(buffer, begin, len);
+        return Double.valueOf(str);
+    }
+
+    protected float parseFloat(char[] buffer, int begin, int len) {
+        //TODO: pares float directly from char[]
+        String str = new String(buffer, begin, len);
+        return Float.valueOf(str);
+    }
+
+    protected int indexOf(char[] buffer, int begin, int len, char target) {
+        for (int i = begin; i < begin + len; i++) {
+            if (buffer[i] == target) {
+                return i;
+            }
+        }
+        throw new IllegalArgumentException("Cannot find " + target + " in " + new String(buffer, begin, len));
+    }
+
+    protected void parseString(char[] buffer, int begin, int length, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeByte(ATypeTag.STRING.serialize());
+            untaggedStringSerde.serialize(buffer, begin, length, out);
+        } catch (IOException e) {
+            throw new ParseException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
index ce8780d..554bb2f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
@@ -50,6 +50,8 @@
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.TreeTraversingParser;
 
 /**
  * JSON format parser using Jakson parser.
@@ -102,7 +104,15 @@
 
     @Override
     public void setInputStream(InputStream in) throws IOException {
-        jsonParser = jsonFactory.createParser(in);
+        setInput(jsonFactory.createParser(in));
+    }
+
+    public void setInputNode(JsonNode node) {
+        setInput(new TreeTraversingParser(node));
+    }
+
+    private void setInput(JsonParser parser) {
+        jsonParser = parser;
         geometryCoParser.reset(jsonParser);
     }
 
@@ -426,26 +436,27 @@
      * @throws IOException
      */
     private void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
-        final String stringValue = jsonParser.getText();
+        char[] buffer = jsonParser.getTextCharacters();
+        int begin = jsonParser.getTextOffset();
+        int len = jsonParser.getTextLength();
         final ATypeTag typeToUse = stringVariantType == ATypeTag.ANY ? currentToken().getTypeTag() : stringVariantType;
 
         switch (typeToUse) {
             case STRING:
-                aString.setValue(stringValue);
-                stringSerde.serialize(aString, out);
+                parseString(buffer, begin, len, out);
                 break;
             case DATE:
-                parseDate(stringValue, out);
+                parseDate(buffer, begin, len, out);
                 break;
             case DATETIME:
-                parseDateTime(stringValue, out);
+                parseDateTime(buffer, begin, len, out);
                 break;
             case TIME:
-                parseTime(stringValue, out);
+                parseTime(buffer, begin, len, out);
                 break;
             default:
                 throw new RuntimeDataException(ErrorCode.TYPE_UNSUPPORTED, jsonParser.currentToken().toString());
 
         }
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
new file mode 100644
index 0000000..333ac55
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
@@ -0,0 +1,78 @@
+/*
+ * 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.asterix.external.parser;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ParseException extends HyracksDataException {
+    private static final long serialVersionUID = 1L;
+    private String filename;
+    private int line = -1;
+    private int column = -1;
+
+    public ParseException(String message) {
+        super(message);
+    }
+
+    public ParseException(int errorCode, Serializable... param) {
+        super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
+    }
+
+    public ParseException(int errorCode, Throwable e, Serializable... param) {
+        super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
+        addSuppressed(e);
+    }
+
+    public ParseException(Throwable cause) {
+        super(cause);
+    }
+
+    public ParseException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ParseException(Throwable cause, String filename, int line, int column) {
+        super(cause);
+        setLocation(filename, line, column);
+    }
+
+    public void setLocation(String filename, int line, int column) {
+        this.filename = filename;
+        this.line = line;
+        this.column = column;
+    }
+
+    @Override
+    public String getMessage() {
+        StringBuilder msg = new StringBuilder("Parse error");
+        if (filename != null) {
+            msg.append(" in file ").append(filename);
+        }
+        if (line >= 0) {
+            msg.append(" in line ").append(line);
+            if (column >= 0) {
+                msg.append(", at column ").append(column);
+            }
+        }
+        return msg.append(": ").append(super.getMessage()).toString();
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
index 489749c..d9cedea 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
@@ -23,7 +23,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
 import org.apache.asterix.external.api.IRecordDataParser;
 import org.apache.asterix.external.api.IRecordDataParserFactory;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java
index 7d2c34b..34f0434 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/TweetParserFactory.java
@@ -29,8 +29,6 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 
-import twitter4j.Status;
-
 public class TweetParserFactory implements IRecordDataParserFactory<String> {
 
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
index c16ee89..53cf6b1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
@@ -47,7 +47,7 @@
             Map<String, String> configuration) throws AsterixException {
         IDataParserFactory parserFactory;
         String parserFactoryName = configuration.get(ExternalDataConstants.KEY_DATA_PARSER);
-        if ((parserFactoryName != null) && ExternalDataUtils.isExternal(parserFactoryName)) {
+        if (ExternalDataUtils.isExternal(parserFactoryName)) {
             return ExternalDataUtils.createExternalParserFactory(libraryManager,
                     ExternalDataUtils.getDataverse(configuration), parserFactoryName);
         } else {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/Datatypes.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/Datatypes.java
index 94d7b53..d915559 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/Datatypes.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/Datatypes.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.external.util;
 
-import org.apache.asterix.external.parser.TweetParser;
-
 public class Datatypes {
 
     /*
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
index 8481064..dc8a8aa 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
@@ -24,7 +24,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index bd50352..1b13ec5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -46,6 +46,7 @@
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -67,11 +68,11 @@
 
     public static IndexingScheduler initializeIndexingHDFSScheduler(ICCServiceContext serviceCtx)
             throws HyracksDataException {
-        ICCContext ccContext = serviceCtx.getCCContext();
         IndexingScheduler scheduler = null;
         try {
-            scheduler = new IndexingScheduler(ccContext.getClusterControllerInfo().getClientNetAddress(),
-                    ccContext.getClusterControllerInfo().getClientNetPort());
+            ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
+            IHyracksClientConnection hcc = appCtx.getHcc();
+            scheduler = new IndexingScheduler(hcc.getNodeControllerInfos());
         } catch (HyracksException e) {
             throw new RuntimeDataException(ErrorCode.UTIL_HDFS_UTILS_CANNOT_OBTAIN_HDFS_SCHEDULER);
         }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/BuiltinClassAdFunctions.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/BuiltinClassAdFunctions.java
index 5fc9df3..7e013b7 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/BuiltinClassAdFunctions.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/BuiltinClassAdFunctions.java
@@ -239,7 +239,7 @@
 
             // Walk over each element in the list, and sum.
             for (ExprTree listElement : listToSum.getExprList()) {
-                len.setValue(len.getLongValue() + 1);;
+                len.setValue(len.getLongValue() + 1);
                 // Make sure this element is a number.
                 if (!listElement.publicEvaluate(state, listElementValue)) {
                     val.setErrorValue();
@@ -1686,7 +1686,7 @@
         @Override
         public boolean call(String name, ExprList argList, EvalState state, Value result, ClassAdObjectPool objectPool)
                 throws HyracksDataException {
-            Value arg = objectPool.valuePool.get();;
+            Value arg = objectPool.valuePool.get();
             // takes exactly one argument
             if (argList.size() > 1) {
                 result.setErrorValue();
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
index 2b04bac..d4f9156 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAd.java
@@ -636,7 +636,7 @@
             return (false);
         }
         CaseInsensitiveString aString = objectPool.caseInsensitiveStringPool.get();
-        aString.set(name);;
+        aString.set(name);
         boolean success = ad.delete(aString);
         return success;
     }
@@ -767,7 +767,7 @@
             throws HyracksDataException {
         ClassAd newAd = objectPool.classAdPool.get();
         Value eval = objectPool.valuePool.get();
-        ExprTreeHolder etree = objectPool.mutableExprPool.get();;
+        ExprTreeHolder etree = objectPool.mutableExprPool.get();
         ClassAd oldAd;
 
         tree.setInnerTree(null); // Just to be safe...  wenger 2003-12-11.
@@ -778,7 +778,7 @@
         for (Entry<CaseInsensitiveString, ExprTree> entry : attrList.entrySet()) {
             // flatten expression
             if (!entry.getValue().publicFlatten(state, eval, etree)) {
-                tree.setInnerTree(null);;
+                tree.setInnerTree(null);
                 eval.setUndefinedValue();
                 state.setCurAd(oldAd);
                 return false;
@@ -1266,7 +1266,7 @@
             }
 
             case ATTRREF_NODE: {
-                ClassAd start = objectPool.classAdPool.get();;
+                ClassAd start = objectPool.classAdPool.get();
                 ExprTreeHolder tree = objectPool.mutableExprPool.get();
                 ExprTreeHolder result = objectPool.mutableExprPool.get();
                 AMutableCharArrayString attr = objectPool.strPool.get();
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAdUnParser.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAdUnParser.java
index 5597df8..5b93b4e 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAdUnParser.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ClassAdUnParser.java
@@ -323,12 +323,12 @@
             buffer.appendChar('.');
         }
         unparseAux(buffer, ref);
-    };
+    }
 
     public void unparseAux(AMutableCharArrayString buffer, final ExprTree tree, AMutableCharArrayString ref)
             throws HyracksDataException {
         unparseAux(buffer, tree, ref, false);
-    };
+    }
 
     public void unparseAuxPairs(AMutableCharArrayString buffer, List<Entry<AMutableCharArrayString, ExprTree>> attrlist)
             throws HyracksDataException {
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Common.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Common.java
index da05e46..1bc46b3 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Common.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Common.java
@@ -51,7 +51,7 @@
         public static boolean call(String s1, String s2) {
             return (s1.compareToIgnoreCase(s2) < 0);
         }
-    };
+    }
 
     public static class ClassadAttrNameHash {
         public static int call(String s) {
@@ -62,5 +62,5 @@
             }
             return h;
         }
-    };
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprList.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprList.java
index 3c8405f..4cd4dd7 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprList.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprList.java
@@ -222,7 +222,7 @@
         Value tempVal = objectPool.valuePool.get();
         ExprList newList = objectPool.exprListPool.get();
 
-        tree.setInnerTree(null);; // Just to be safe...  wenger 2003-12-11.
+        tree.setInnerTree(null); // Just to be safe...  wenger 2003-12-11.
 
         for (ExprTree expr : exprList) {
             // flatten the constituent expression
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprTree.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprTree.java
index 2b5af38..98a8f77 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprTree.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/ExprTree.java
@@ -55,7 +55,7 @@
         EVAL_OK,
         EVAL_UNDEF,
         EVAL_ERROR
-    };
+    }
 
     public static final int EVAL_FAIL_Int = 0;
     public static final int EVAL_OK_Int = 1;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/FunctionCall.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/FunctionCall.java
index 234ba20..21ac837 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/FunctionCall.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/FunctionCall.java
@@ -354,6 +354,6 @@
     public void reset() {
         this.arguments.clear();
         this.function = null;
-        this.functionName.set("");;
+        this.functionName.set("");
     }
 }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Lexer.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Lexer.java
index 2ef9620..68a1071 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Lexer.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Lexer.java
@@ -95,7 +95,7 @@
         LEX_BACKSLASH,
         LEX_ABSOLUTE_TIME_VALUE,
         LEX_RELATIVE_TIME_VALUE
-    };
+    }
 
     public Lexer() {
         // initialize lexer state (token, etc.) variables
@@ -347,7 +347,7 @@
         NONE,
         INTEGER,
         REAL
-    };
+    }
 
     public TokenType tokenizeNumber() throws IOException {
         NumberType numberType = NumberType.NONE;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Operation.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Operation.java
index 562bd49..d3a74fb 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Operation.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Operation.java
@@ -37,7 +37,7 @@
         SIG_CHLD2,
         SIG_DUMMY,
         SIG_CHLD3
-    };
+    }
 
     /// List of supported operators
     public static final int OpKind_NO_OP = 0;
@@ -545,7 +545,7 @@
 
         if (sig == SigValues.SIG_NONE.ordinal()) {
             result.setErrorValue();
-            tree.setInnerTree(null);;
+            tree.setInnerTree(null);
             return (false);
         }
 
@@ -639,7 +639,7 @@
         AMutableInt32 childOp2 = objectPool.int32Pool.get();
         childOp2.setValue(OpKind_NO_OP);
         ExprTreeHolder fChild1 = objectPool.mutableExprPool.get();
-        ExprTreeHolder fChild2 = objectPool.mutableExprPool.get();;
+        ExprTreeHolder fChild2 = objectPool.mutableExprPool.get();
         Value val1 = objectPool.valuePool.get();
         Value val2 = objectPool.valuePool.get();
         Value val3 = objectPool.valuePool.get();
@@ -647,7 +647,7 @@
         newOp.setValue(opKind);
         int op = opKind;
 
-        tree.setInnerTree(null);; // Just to be safe...  wenger 2003-12-11.
+        tree.setInnerTree(null);// Just to be safe...  wenger 2003-12-11.
 
         // if op is binary, but not associative or commutative, disallow splitting
         if ((op >= OpKind_COMPARISON_START && op <= OpKind_COMPARISON_END) || op == OpKind_SUBTRACTION_OP
@@ -1781,7 +1781,7 @@
                         fChild3.setInnerTree(Literal.createLiteral(eval3, objectPool));
                     }
                     if (fChild2.getInnerTree() == null || fChild3.getInnerTree() == null) {
-                        tree.setInnerTree(null);;
+                        tree.setInnerTree(null);
                         return false;
                     }
 
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/PrettyPrint.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/PrettyPrint.java
index a25db5c..bacde4a 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/PrettyPrint.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/PrettyPrint.java
@@ -218,10 +218,10 @@
                 }
             }
             super.unparse(buffer, itr);
-            buffer.appendChar(',');;
+            buffer.appendChar(',');
         }
         if (exprs.size() > 0) {
-            buffer.decrementLength();;
+            buffer.decrementLength();
         }
         if (listIndent > 0) {
             indentLevel -= listIndent;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Value.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Value.java
index 427e2a2..07d41f0 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Value.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/Value.java
@@ -61,7 +61,7 @@
         LIST_VALUE,
         /** A list value (owned via shared_ptr) */
         SLIST_VALUE
-    };
+    }
 
     /// Number factors
     public enum NumberFactor {
@@ -77,7 +77,7 @@
         G_FACTOR,
         /** Terra factor */
         T_FACTOR
-    };
+    }
 
     public ValueType getType() {
         return valueType;
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/FunctionalTester.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/FunctionalTester.java
index 75b2800..553da99 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/FunctionalTester.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/FunctionalTester.java
@@ -111,14 +111,14 @@
         cmd_Echo,
         cmd_Help,
         cmd_Quit
-    };
+    }
 
     public enum PrintFormat {
         print_Compact,
         print_Pretty,
         print_XML,
         print_XMLPretty
-    };
+    }
 
     public static class Parameters {
         public boolean debug;
@@ -965,7 +965,7 @@
             if (parser.peekToken() != TokenType.LEX_COMMA) {
                 print_error_message("Missing comma.\n", state);
                 tree1.setInnerTree(null);
-                tree2.setInnerTree(null);;
+                tree2.setInnerTree(null);
             } else {
                 parser.consumeToken();
                 tree2.setInnerTree(parser.parseNextExpression());
@@ -1029,7 +1029,7 @@
     public static boolean evaluate_expr(ExprTree tree, Value value, Parameters parameters, ClassAdObjectPool objectPool)
             throws HyracksDataException {
         ClassAd classad = new ClassAd(objectPool);
-        boolean success = false;;
+        boolean success = false;
         classad.insert("internal___", tree);
         success = classad.evaluateAttr("internal___", value);
         classad.remove("internal___");
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
index d19f89f..4a1c740 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
@@ -1349,7 +1349,7 @@
                     if (!parseArgumentList(argList)) {
                         tree.setInnerTree(null);
                         return false;
-                    } ;
+                    }
                     // special case function-calls should be converted
                     // into a literal expression if the argument is a
                     // string literal
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
index 90336fe..effd59f 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
@@ -55,7 +55,7 @@
 
     public TestTypedAdapter(ITupleParserFactory parserFactory, ARecordType sourceDatatype, IHyracksTaskContext ctx,
             Map<String, String> configuration, int partition) throws IOException {
-        super(null);
+        super(new TestTypedFeedDataFlowController(ctx));
         pos = new PipedOutputStream();
         pis = new PipedInputStream(pos);
         this.configuration = configuration;
@@ -150,4 +150,5 @@
     public boolean resume() {
         return false;
     }
+
 }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedFeedDataFlowController.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedFeedDataFlowController.java
new file mode 100644
index 0000000..708cdd8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedFeedDataFlowController.java
@@ -0,0 +1,39 @@
+/*
+ * 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.asterix.external.library.adapter;
+
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+class TestTypedFeedDataFlowController extends AbstractFeedDataFlowController {
+    TestTypedFeedDataFlowController(IHyracksTaskContext ctx) {
+        super(ctx, null, 0);
+    }
+
+    @Override
+    public String getStats() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void start(IFrameWriter writer) {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
index 1485f4f..189c842 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
@@ -18,60 +18,76 @@
  */
 package org.apache.asterix.external.parser.test;
 
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.external.parser.ADMDataParser;
 import org.apache.asterix.external.parser.AbstractDataParser;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMutableDate;
 import org.apache.asterix.om.base.AMutableDateTime;
 import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.esri.core.geometry.ogc.OGCPoint;
+
 public class ADMDataParserTest {
 
     @Test
     public void test() throws IOException, NoSuchMethodException, SecurityException, NoSuchFieldException {
-        String[] dates = { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" };
+        char[][] dates = toChars(
+                new String[] { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" });
         AMutableDate[] parsedDates =
                 new AMutableDate[] { new AMutableDate(-4202630), new AMutableDate(2807408), new AMutableDate(-4202752),
                         new AMutableDate(2807409), new AMutableDate(-4202569), new AMutableDate(2796544), };
 
-        String[] times = { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z", "04:16:42.321Z",
-                "12:22:56.816Z" };
+        char[][] times = toChars(new String[] { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z",
+                "04:16:42.321Z", "12:22:56.816Z" });
         AMutableTime[] parsedTimes =
                 new AMutableTime[] { new AMutableTime(43485689), new AMutableTime(45719002), new AMutableTime(43845169),
                         new AMutableTime(56268736), new AMutableTime(15402321), new AMutableTime(44576816), };
 
-        String[] dateTimes = { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
-                "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" };
+        char[][] dateTimes = toChars(
+                new String[] { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
+                        "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" });
         AMutableDateTime[] parsedDateTimes =
                 new AMutableDateTime[] { new AMutableDateTime(-145452954464325L), new AMutableDateTime(67345192751902L),
                         new AMutableDateTime(-45286270768513L), new AMutableDateTime(151729886421653L),
                         new AMutableDateTime(5047449515758L), new AMutableDateTime(210721439419691L) };
 
-        Method parseDateMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseDate", String.class, DataOutput.class);
+        Method parseDateMethod = AbstractDataParser.class.getDeclaredMethod("parseDate", char[].class, int.class,
+                int.class, DataOutput.class);
         parseDateMethod.setAccessible(true);
 
         Field aDateField = AbstractDataParser.class.getDeclaredField("aDate");
         aDateField.setAccessible(true);
 
-        Method parseTimeMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseTime", String.class, DataOutput.class);
+        Method parseTimeMethod = AbstractDataParser.class.getDeclaredMethod("parseTime", char[].class, int.class,
+                int.class, DataOutput.class);
         parseTimeMethod.setAccessible(true);
 
         Field aTimeField = AbstractDataParser.class.getDeclaredField("aTime");
         aTimeField.setAccessible(true);
 
-        Method parseDateTimeMethod =
-                AbstractDataParser.class.getDeclaredMethod("parseDateTime", String.class, DataOutput.class);
+        Method parseDateTimeMethod = AbstractDataParser.class.getDeclaredMethod("parseDateTime", char[].class,
+                int.class, int.class, DataOutput.class);
         parseDateTimeMethod.setAccessible(true);
 
         Field aDateTimeField = AbstractDataParser.class.getDeclaredField("aDateTime");
@@ -92,21 +108,21 @@
                         while (round++ < 10000) {
                             // Test parseDate.
                             for (int index = 0; index < dates.length; ++index) {
-                                parseDateMethod.invoke(parser, dates[index], dos);
+                                parseDateMethod.invoke(parser, dates[index], 0, dates[index].length, dos);
                                 AMutableDate aDate = (AMutableDate) aDateField.get(parser);
                                 Assert.assertTrue(aDate.equals(parsedDates[index]));
                             }
 
                             // Tests parseTime.
                             for (int index = 0; index < times.length; ++index) {
-                                parseTimeMethod.invoke(parser, times[index], dos);
+                                parseTimeMethod.invoke(parser, times[index], 0, times[index].length, dos);
                                 AMutableTime aTime = (AMutableTime) aTimeField.get(parser);
                                 Assert.assertTrue(aTime.equals(parsedTimes[index]));
                             }
 
                             // Tests parseDateTime.
                             for (int index = 0; index < dateTimes.length; ++index) {
-                                parseDateTimeMethod.invoke(parser, dateTimes[index], dos);
+                                parseDateTimeMethod.invoke(parser, dateTimes[index], 0, dateTimes[index].length, dos);
                                 AMutableDateTime aDateTime = (AMutableDateTime) aDateTimeField.get(parser);
                                 Assert.assertTrue(aDateTime.equals(parsedDateTimes[index]));
                             }
@@ -132,4 +148,58 @@
         // Asserts no failure.
         Assert.assertTrue(errorCount.get() == 0);
     }
-}
\ No newline at end of file
+
+    private char[][] toChars(String[] strings) {
+        char[][] results = new char[strings.length][];
+        for (int i = 0; i < strings.length; i++) {
+            results[i] = strings[i].toCharArray();
+        }
+        return results;
+    }
+
+    @Test
+    public void testWKTParser() {
+        try {
+            ARecordType recordType = new ARecordType("POIType", new String[] { "id", "coord" },
+                    new IAType[] { BuiltinType.AINT32, BuiltinType.AGEOMETRY }, false);
+
+            String wktObject = "{\"id\": 123, \"coord\": \"POINT(3 4)\"}";
+            InputStream in = new ByteArrayInputStream(wktObject.getBytes());
+            ADMDataParser parser = new ADMDataParser(recordType, true);
+            parser.setInputStream(in);
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(baos);
+            parser.parse(out);
+            out.close();
+            byte[] serialized = baos.toByteArray();
+
+            // Parse to make sure it was correct
+            ByteBuffer bb = ByteBuffer.wrap(serialized);
+            Assert.assertEquals(ATypeTag.SERIALIZED_RECORD_TYPE_TAG, bb.get());
+            Assert.assertEquals(serialized.length, bb.getInt()); // Total record size including header
+            Assert.assertEquals(2, bb.getInt()); // # of records
+            int offsetOfID = bb.getInt();
+            int offsetOfGeometry = bb.getInt();
+            ISerializerDeserializer intDeser =
+                    SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT32);
+            Assert.assertEquals(offsetOfID, bb.position());
+            // Serialize the two records
+            DataInputByteBuffer dataIn = new DataInputByteBuffer();
+            dataIn.reset(bb);
+            Object o = intDeser.deserialize(dataIn);
+            Assert.assertEquals(new AInt32(123), o);
+            ISerializerDeserializer geomDeser =
+                    SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+            Object point = geomDeser.deserialize(dataIn);
+            Assert.assertTrue(point instanceof AGeometry);
+            Assert.assertTrue(((AGeometry) point).getGeometry() instanceof OGCPoint);
+            OGCPoint p = (OGCPoint) ((AGeometry) point).getGeometry();
+            Assert.assertEquals(3.0, p.X(), 1E-5);
+            Assert.assertEquals(4.0, p.Y(), 1E-5);
+        } catch (IOException e) {
+            e.printStackTrace();
+            Assert.fail("Error in parsing");
+        }
+
+    }
+}
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
index 4cc4975..62e4732 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceCheckEvaluator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,9 +54,9 @@
     protected final ISerializerDeserializer<ABoolean> booleanSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
-    public EditDistanceCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public EditDistanceCheckEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         edThreshEval = args[2].createScalarEvaluator(context);
         listBuilder = new OrderedListBuilder();
         listItemVal = new ArrayBackedValueStorage();
@@ -110,7 +111,7 @@
             }
 
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE_CHECK, 0, argType.serialize(),
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE_CHECK, 0, argType.serialize(),
                         ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
 
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
index eaf3368..d35dd89 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceContainsEvaluator.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 
 public class EditDistanceContainsEvaluator extends EditDistanceCheckEvaluator {
 
-    public EditDistanceContainsEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public EditDistanceContainsEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
@@ -51,8 +52,9 @@
                 return ed.getSimilarityContains(firstOrdListIter, secondOrdListIter, edThresh);
             }
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE_CONTAINS, 0, argType.serialize(),
-                        ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE_CONTAINS, 0,
+                        argType.serialize(), ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                        ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
         }
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
index c3af387..cc103e0 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/common/EditDistanceEvaluator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -53,6 +54,7 @@
     protected final IPointable argPtr2 = new VoidPointable();
     protected final IScalarEvaluator firstStringEval;
     protected final IScalarEvaluator secondStringEval;
+    protected final SourceLocation sourceLoc;
     protected final SimilarityMetricEditDistance ed = new SimilarityMetricEditDistance();
     protected final OrderedListIterator firstOrdListIter = new OrderedListIterator();
     protected final OrderedListIterator secondOrdListIter = new OrderedListIterator();
@@ -66,10 +68,11 @@
     protected ATypeTag firstTypeTag;
     protected ATypeTag secondTypeTag;
 
-    public EditDistanceEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public EditDistanceEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
         firstStringEval = args[0].createScalarEvaluator(context);
         secondStringEval = args[1].createScalarEvaluator(context);
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -115,7 +118,7 @@
                 return (int) ed.computeSimilarity(firstOrdListIter, secondOrdListIter);
             }
             default: {
-                throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE, 0, argType.serialize(),
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, 0, argType.serialize(),
                         ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
             }
 
@@ -124,13 +127,13 @@
 
     protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
         if (typeTag1 != typeTag2) {
-            throw new IncompatibleTypeException(BuiltinFunctions.EDIT_DISTANCE, typeTag1.serialize(),
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, typeTag1.serialize(),
                     typeTag2.serialize());
         }
 
         // Since they are equal, check one tag is enough.
         if (typeTag1 != ATypeTag.STRING && typeTag1 != ATypeTag.ARRAY) { // could be an list
-            throw new TypeMismatchException(BuiltinFunctions.EDIT_DISTANCE, 0, typeTag1.serialize(),
+            throw new TypeMismatchException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE, 0, typeTag1.serialize(),
                     ATypeTag.SERIALIZED_STRING_TYPE_TAG, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
         }
 
@@ -138,12 +141,14 @@
             itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
                     .deserialize(argPtr1.getByteArray()[argPtr1.getStartOffset() + 1]);
             if (itemTypeTag == ATypeTag.ANY) {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE,
+                        itemTypeTag.serialize());
             }
             itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
                     .deserialize(argPtr2.getByteArray()[argPtr2.getStartOffset() + 1]);
             if (itemTypeTag == ATypeTag.ANY) {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.EDIT_DISTANCE, itemTypeTag.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.EDIT_DISTANCE,
+                        itemTypeTag.serialize());
             }
         }
         return true;
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
index e4b40b1..d803a47 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceCheckEvaluator(args, ctx);
+                return new EditDistanceCheckEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
index 4c7c257..b6002b5 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceContainsEvaluator(args, ctx);
+                return new EditDistanceContainsEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
index 8c6c9ed..60c8867 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
@@ -46,7 +46,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new EditDistanceEvaluator(args, ctx);
+                return new EditDistanceEvaluator(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
index 17637bc..e7d582f 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
@@ -91,7 +91,7 @@
                         byte[] data = inputVal.getByteArray();
                         int offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length = IntegerPointable.getInteger(data, offset + 1);
@@ -101,7 +101,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -112,7 +112,7 @@
                         offset = inputVal.getStartOffset();
                         int len = inputVal.getLength();
                         if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, data[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         SimilarityFilters similarityFilters =
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
index 52bd8b1..d3fc6fc 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
@@ -94,7 +94,7 @@
                         byte[] data = thresholdPtr.getByteArray();
                         int offset = thresholdPtr.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
                         }
                         float similarityThreshold = AFloatSerializerDeserializer.getFloat(data, offset + 1);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index 8584d06..89da0a1 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -105,7 +105,7 @@
                         int offset = inputVal.getStartOffset();
 
                         if (data[offset] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         float similarityThreshold = (float) ADoubleSerializerDeserializer.getDouble(data, offset + 1);
@@ -116,7 +116,7 @@
                         offset = inputVal.getStartOffset();
                         int len = inputVal.getLength();
                         if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, data[offset],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         SimilarityFilters similarityFilters =
@@ -126,7 +126,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 2, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length1 = IntegerPointable.getInteger(data, offset + 1);
@@ -135,7 +135,7 @@
                         data = inputVal.getByteArray();
                         offset = inputVal.getStartOffset();
                         if (data[offset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 3, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 3, data[offset],
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                         int length2 = IntegerPointable.getInteger(data, offset + 1);
@@ -156,7 +156,7 @@
 
                             if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 4, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 4, data[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
@@ -192,7 +192,7 @@
 
                             if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 5, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 5, data[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index 4229f87..67e519f 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -122,7 +122,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         int counter = 0;
@@ -231,7 +232,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                         for (int i = 0; i < numOfPoints1; i++) {
                             double startX2 = ADoubleSerializerDeserializer.getDouble(bytes1,
@@ -383,7 +385,7 @@
                         int v = middleVertex;
                         while (!foundEar) {
                             if (0 >= (nonSimplePolygonDetection--)) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
                             int u = v;
@@ -676,7 +678,8 @@
                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints1 < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         getCounterClockWisePolygon(bytes1, offset1, pointsOffsets1, numOfPoints1);
@@ -728,7 +731,8 @@
                                 offset0 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                         if (numOfPoints < 3) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
 
                         getCounterClockWisePolygon(bytes0, offset0, pointsOffsets0, numOfPoints);
@@ -825,7 +829,7 @@
                                         res = pointInRectangle(bytes0, offset0, bytes1, offset1);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -894,7 +898,7 @@
                                         res = lineRectangleIntersection(bytes0, offset0, bytes1, offset1);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -916,7 +920,7 @@
                                                 offset1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                                         if (numOfPoints0 < 3 || numOfPoints1 < 3) {
-                                            throw new InvalidDataFormatException(getIdentifier(),
+                                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                         }
 
@@ -987,7 +991,7 @@
                                         res = rectanglePolygonIntersection(bytes1, offset1, bytes0, offset0);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1012,7 +1016,7 @@
                                         res = rectangleCircleIntersection(bytes1, offset1, bytes0, offset0);
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1061,7 +1065,7 @@
                                         }
                                         break;
                                     default:
-                                        throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                        throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                                 ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG,
                                                 ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
@@ -1069,7 +1073,7 @@
                                 }
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG, ATypeTag.SERIALIZED_LINE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
new file mode 100644
index 0000000..3423b50
--- /dev/null
+++ b/asterixdb/asterix-geo/pom.xml
@@ -0,0 +1,146 @@
+<!--
+ ! 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.
+ !-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>apache-asterixdb</artifactId>
+    <groupId>org.apache.asterix</groupId>
+    <version>0.9.4-SNAPSHOT</version>
+  </parent>
+  <artifactId>asterix-geo</artifactId>
+
+  <licenses>
+    <license>
+      <name>Apache License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+      <distribution>repo</distribution>
+      <comments>A business-friendly OSS license</comments>
+    </license>
+  </licenses>
+
+  <properties>
+    <root.dir>${basedir}/..</root.dir>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.asterix</groupId>
+        <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+        <version>${project.version}</version>
+        <configuration>
+          <evaluatorPackagePrefix>org.apache.asterix.runtime.evaluators</evaluatorPackagePrefix>
+        </configuration>
+        <executions>
+          <execution>
+            <id>generate-evaluator</id>
+            <phase>process-classes</phase>
+            <goals>
+              <goal>generate-evaluator</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <excludes combine.children="append">
+            <exclude>data/**</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-data-std</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-runtime</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-om</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-runtime</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+    </dependency>
+  </dependencies>
+
+</project>
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
new file mode 100644
index 0000000..99eaf46
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * 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.asterix.geo.aggregates;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * NULLs are also counted.
+ */
+public class STUnionAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STUnionAggregateDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_UNION_AGG;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STUnionAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
new file mode 100644
index 0000000..5716b2c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
@@ -0,0 +1,118 @@
+/*
+ * 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.asterix.geo.aggregates;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * STUnion aggregates a set of objects into one object. If the input is a set of overlapping polygons, their union is
+ * computed and returned as a multipolygon. Similarly, if the input is a set of points or linestring, a multipoint or
+ * multilinestring is created. Is the result contains geometries of different types, e.g., points and linestring, the
+ * output is a GeometryCollection.
+ */
+public class STUnionAggregateFunction extends AbstractAggregateFunction {
+    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AGeometry> geometrySerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY);
+    private IPointable inputVal = new VoidPointable();
+    private IScalarEvaluator eval;
+    protected OGCGeometry geometry;
+
+    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+    public STUnionAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
+        eval = args[0].createScalarEvaluator(context);
+    }
+
+    @Override
+    public void init() throws HyracksDataException {
+        // Initialize the resulting geometry with an empty point.
+        geometry = new OGCPoint(new Point(), DEFAULT_CRS);
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        eval.evaluate(tuple, inputVal);
+        byte[] data = inputVal.getByteArray();
+        int offset = inputVal.getStartOffset();
+        int len = inputVal.getLength();
+        ATypeTag typeTag =
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
+        // Ignore SYSTEM_NULL.
+        if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
+            processNull();
+        } else if (typeTag == ATypeTag.GEOMETRY) {
+            DataInput dataIn = new DataInputStream(new ByteArrayInputStream(data, offset + 1, len - 1));
+            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            geometry = geometry.union(geometry1);
+        }
+    }
+
+    @Override
+    public void finish(IPointable resultPointable) throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            geometrySerde.serialize(new AGeometry(geometry), resultStorage.getDataOutput());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+        resultPointable.set(resultStorage);
+    }
+
+    @Override
+    public void finishPartial(IPointable resultPointable) throws HyracksDataException {
+        finish(resultPointable);
+    }
+
+    protected void processNull() throws UnsupportedItemTypeException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.ST_UNION,
+                ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java
new file mode 100644
index 0000000..96b6478
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java
@@ -0,0 +1,46 @@
+/*
+ * 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.asterix.geo.aggregates;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.scalar.AbstractScalarAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSTUnionAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_ST_UNION_AGG;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ScalarSTUnionAggregateDescriptor(STUnionAggregateDescriptor.FACTORY.createFunctionDescriptor());
+        }
+    };
+
+    private ScalarSTUnionAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+        super(aggFuncDesc);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return FID;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
new file mode 100644
index 0000000..1644b99
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -0,0 +1,155 @@
+/*
+ * 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.asterix.geo.evaluators;
+
+import org.apache.asterix.geo.aggregates.STUnionAggregateDescriptor;
+import org.apache.asterix.geo.aggregates.ScalarSTUnionAggregateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.ParseGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAreaDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsBinaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STBoundaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STContainsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCoordDimDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCrossesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDimensionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDisjointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDistanceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEndPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEnvelopeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEqualsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STExteriorRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromWKBDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomentryTypeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeometryNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STInteriorRingNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsClosedDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsCollectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsEmptyDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsSimpleDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLengthDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLineFromMultiPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakeEnvelopeDescriptorSRID;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DWithMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNPointsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumGeometriesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumInteriorRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STOverlapsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPointNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPolygonizeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STRelateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STStartPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSymDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STTouchesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STUnionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STWithinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMinDescriptor;
+import org.apache.asterix.om.functions.IFunctionCollection;
+import org.apache.asterix.om.functions.IFunctionRegistrant;
+
+public class GeoFunctionRegistrant implements IFunctionRegistrant {
+    @Override
+    public void register(IFunctionCollection fc) {
+        //Geo functions
+        fc.add(ScalarSTUnionAggregateDescriptor.FACTORY);
+        fc.add(STUnionAggregateDescriptor.FACTORY);
+
+        //GeoJSON
+        fc.add(ParseGeoJSONDescriptor.FACTORY);
+        fc.add(STAreaDescriptor.FACTORY);
+        fc.add(STMakePointDescriptor.FACTORY);
+        fc.add(STMakePoint3DDescriptor.FACTORY);
+        fc.add(STMakePoint3DWithMDescriptor.FACTORY);
+        fc.add(STIntersectsDescriptor.FACTORY);
+        fc.add(STUnionDescriptor.FACTORY);
+        fc.add(STIsCollectionDescriptor.FACTORY);
+        fc.add(STContainsDescriptor.FACTORY);
+        fc.add(STCrossesDescriptor.FACTORY);
+        fc.add(STDisjointDescriptor.FACTORY);
+        fc.add(STEqualsDescriptor.FACTORY);
+        fc.add(STOverlapsDescriptor.FACTORY);
+        fc.add(STTouchesDescriptor.FACTORY);
+        fc.add(STWithinDescriptor.FACTORY);
+        fc.add(STIsEmptyDescriptor.FACTORY);
+        fc.add(STIsSimpleDescriptor.FACTORY);
+        fc.add(STCoordDimDescriptor.FACTORY);
+        fc.add(STDimensionDescriptor.FACTORY);
+        fc.add(STGeomentryTypeDescriptor.FACTORY);
+        fc.add(STMDescriptor.FACTORY);
+        fc.add(STNRingsDescriptor.FACTORY);
+        fc.add(STNPointsDescriptor.FACTORY);
+        fc.add(STNumGeometriesDescriptor.FACTORY);
+        fc.add(STNumInteriorRingsDescriptor.FACTORY);
+        fc.add(STSRIDDescriptor.FACTORY);
+        fc.add(STXDescriptor.FACTORY);
+        fc.add(STYDescriptor.FACTORY);
+        fc.add(STXMaxDescriptor.FACTORY);
+        fc.add(STXMinDescriptor.FACTORY);
+        fc.add(STYMaxDescriptor.FACTORY);
+        fc.add(STYMinDescriptor.FACTORY);
+        fc.add(STZDescriptor.FACTORY);
+        fc.add(STZMaxDescriptor.FACTORY);
+        fc.add(STZMinDescriptor.FACTORY);
+        fc.add(STAsBinaryDescriptor.FACTORY);
+        fc.add(STAsTextDescriptor.FACTORY);
+        fc.add(STAsGeoJSONDescriptor.FACTORY);
+        fc.add(STDistanceDescriptor.FACTORY);
+        fc.add(STLengthDescriptor.FACTORY);
+        fc.add(STGeomFromTextDescriptor.FACTORY);
+        fc.add(STGeomFromTextSRIDDescriptor.FACTORY);
+        fc.add(STGeomFromWKBDescriptor.FACTORY);
+        fc.add(STLineFromMultiPointDescriptor.FACTORY);
+        fc.add(STMakeEnvelopeDescriptorSRID.FACTORY);
+        fc.add(STIsClosedDescriptor.FACTORY);
+        fc.add(STIsRingDescriptor.FACTORY);
+        fc.add(STRelateDescriptor.FACTORY);
+        fc.add(STBoundaryDescriptor.FACTORY);
+        fc.add(STEndPointDescriptor.FACTORY);
+        fc.add(STEnvelopeDescriptor.FACTORY);
+        fc.add(STExteriorRingDescriptor.FACTORY);
+        fc.add(STGeometryNDescriptor.FACTORY);
+        fc.add(STInteriorRingNDescriptor.FACTORY);
+        fc.add(STPointNDescriptor.FACTORY);
+        fc.add(STStartPointDescriptor.FACTORY);
+        fc.add(STDifferenceDescriptor.FACTORY);
+        fc.add(STIntersectionDescriptor.FACTORY);
+        fc.add(STSymDifferenceDescriptor.FACTORY);
+        fc.add(STPolygonizeDescriptor.FACTORY);
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java
new file mode 100644
index 0000000..f609beb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators;
+
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class GeoFunctionTypeInferers {
+    private GeoFunctionTypeInferers() {
+    }
+
+    public static final class GeometryConstructorTypeInferer implements IFunctionTypeInferer {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+            ATypeTag typeTag = t.getTypeTag();
+            if (typeTag.equals(ATypeTag.OBJECT)) {
+                fd.setImmutableStates(t);
+            } else if (typeTag.equals(ATypeTag.ANY)) {
+                fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+            } else {
+                throw new NotImplementedException("parse-geojson for data of type " + t);
+            }
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
new file mode 100644
index 0000000..65992f7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
@@ -0,0 +1,39 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+
+public abstract class AbstractGetValDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    public double getVal(byte[] bytes, int offset) throws TypeMismatchException {
+        if (bytes[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+            return ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
+        } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+            return AInt64SerializerDeserializer.getLong(bytes, offset + 1);
+        } else {
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
+                    ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
new file mode 100644
index 0000000..60f6ea4
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTDoubleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1)
+            throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractSTDoubleGeometryEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTDoubleGeometryEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private final IPointable argPtr0;
+        private final IPointable argPtr1;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+
+        public AbstractSTDoubleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            argPtr0 = new VoidPointable();
+            argPtr1 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            resultStorage.reset();
+            eval0.evaluate(tuple, argPtr0);
+            eval1.evaluate(tuple, argPtr1);
+
+            try {
+                byte[] bytes0 = argPtr0.getByteArray();
+                int offset0 = argPtr0.getStartOffset();
+                int len0 = argPtr0.getLength();
+                byte[] bytes1 = argPtr1.getByteArray();
+                int offset1 = argPtr1.getStartOffset();
+                int len1 = argPtr1.getLength();
+
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+                tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+
+                DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+                DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+                OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+                Object finalResult = evaluateOGCGeometry(geometry0, geometry1);
+                if (finalResult instanceof OGCGeometry) {
+                    out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                } else if (finalResult instanceof Boolean) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                            .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+                } else if (finalResult instanceof Double) {
+                    out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                    out.writeDouble((double) finalResult);
+                }
+
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
new file mode 100644
index 0000000..cf1a900
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
@@ -0,0 +1,117 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTGeometryNDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    abstract protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new AbstractSTGeometryNEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTGeometryNEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+
+        public AbstractSTGeometryNEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            if (data[offset] != ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+            int n = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+
+            OGCGeometry geometryN = evaluateOGCGeometry(geometry, n);
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry(geometryN), out);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
new file mode 100644
index 0000000..0100261
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTSingleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    abstract protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            @SuppressWarnings("unchecked")
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractSTSingleGeometryEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class AbstractSTSingleGeometryEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private final IPointable argPtr0;
+        private final IScalarEvaluator eval0;
+
+        private final AMutableInt32 intRes;
+
+        public AbstractSTSingleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            argPtr0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            intRes = new AMutableInt32(0);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            resultStorage.reset();
+            eval0.evaluate(tuple, argPtr0);
+
+            try {
+                byte[] bytes0 = argPtr0.getByteArray();
+                int offset0 = argPtr0.getStartOffset();
+                int len0 = argPtr0.getLength();
+
+                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                if (tag != ATypeTag.GEOMETRY) {
+                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                            ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                }
+
+                DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+                OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+
+                Object finalResult = evaluateOGCGeometry(geometry0);
+                if (finalResult == null) {
+                    out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                } else if (finalResult instanceof Double) {
+                    out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                    out.writeDouble((double) finalResult);
+                } else if (finalResult instanceof Boolean) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                            .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+                } else if (finalResult instanceof Integer) {
+                    intRes.setValue((int) finalResult);
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32)
+                            .serialize(intRes, out);
+                } else if (finalResult instanceof String) {
+                    out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                    out.write(UTF8StringUtil.writeStringToBytes((String) finalResult));
+                } else if (finalResult instanceof byte[]) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABINARY)
+                            .serialize(new ABinary((byte[]) finalResult), out);
+                } else if (finalResult instanceof OGCGeometry) {
+                    out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                    AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
new file mode 100644
index 0000000..75b3331
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
@@ -0,0 +1,191 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.MapOGCStructure;
+import com.esri.core.geometry.OperatorImportFromGeoJson;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.geo.evaluators.GeoFunctionTypeInferers;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class ParseGeoJSONDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ParseGeoJSONDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return new GeoFunctionTypeInferers.GeometryConstructorTypeInferer();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        this.recType = (ARecordType) states[0];
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GEOMETRY_CONSTRUCTOR;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new ParseGeoJSONEvaluator(args[0], ctx);
+            }
+        };
+    }
+
+    private class ParseGeoJSONEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private OperatorImportFromGeoJson geoJsonImporter;
+
+        public ParseGeoJSONEvaluator(IScalarEvaluatorFactory factory, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = factory.createScalarEvaluator(ctx);
+            geoJsonImporter = OperatorImportFromGeoJson.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GEOMETRY_CONSTRUCTOR, 0, data[offset],
+                        ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+            }
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInput dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = recordToString(new ARecordSerializerDeserializer(recType).deserialize(dataIn));
+                MapOGCStructure structure = geoJsonImporter.executeOGC(0, geometry, null);
+                OGCGeometry ogcGeometry =
+                        OGCGeometry.createFromOGCStructure(structure.m_ogcStructure, structure.m_spatialReference);
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+
+        public String recordToString(ARecord record) {
+            StringBuilder sb = new StringBuilder();
+            sb.append("{ ");
+            String[] fieldNames = record.getType().getFieldNames();
+            IAObject val;
+            if (fieldNames != null) {
+                for (int i = 0; i < fieldNames.length; i++) {
+                    if (i > 0) {
+                        sb.append(", ");
+                    }
+                    sb.append("\"").append(fieldNames[i]).append("\"").append(": ");
+                    val = record.getValueByPos(i);
+                    if (val instanceof ARecord) {
+                        sb.append(recordToString((ARecord) val));
+                    } else if (val instanceof AOrderedList) {
+                        sb.append(listToString((AOrderedList) val));
+                    } else {
+                        sb.append(val);
+                    }
+                }
+            }
+            sb.append(" }");
+            return sb.toString();
+        }
+
+        public String listToString(AOrderedList list) {
+            StringBuilder sb = new StringBuilder();
+            boolean first = true;
+            IAObject val;
+            sb.append("[ ");
+            for (int i = 0; i < list.size(); i++) {
+                if (first) {
+                    first = false;
+                } else {
+                    sb.append(", ");
+                }
+                val = list.getItem(i);
+                if (val instanceof ARecord) {
+                    sb.append(recordToString((ARecord) val));
+                } else if (val instanceof AOrderedList) {
+                    sb.append(listToString((AOrderedList) val));
+                } else {
+                    sb.append(val);
+                }
+            }
+            sb.append(" ]");
+            return sb.toString();
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
new file mode 100644
index 0000000..caf0b89
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAreaDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAreaDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        double area;
+        if (!"GeometryCollection".equals(geometry.geometryType())) {
+            area = geometry.getEsriGeometry().calculateArea2D();
+        } else {
+            GeometryCursor cursor = geometry.getEsriGeometryCursor();
+            Geometry geometry1 = cursor.next();
+            area = 0;
+            while (geometry1 != null) {
+                area += geometry1.calculateArea2D();
+                geometry1 = cursor.next();
+            }
+        }
+        return area;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AREA;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
new file mode 100644
index 0000000..387d07c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsBinaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsBinaryDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asBinary().array();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_BINARY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
new file mode 100644
index 0000000..b12b851
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsGeoJSONDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsGeoJSONDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asGeoJson();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_GEOJSON;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
new file mode 100644
index 0000000..3e310d9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsTextDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STAsTextDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.asText();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_AS_TEXT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
new file mode 100644
index 0000000..55c5212
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STBoundaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STBoundaryDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.boundary();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_BOUNDARY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
new file mode 100644
index 0000000..f9f692f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STContainsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STContainsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.contains(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_CONTAINS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
new file mode 100644
index 0000000..d1a3efd
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCoordDimDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STCoordDimDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.coordinateDimension();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_COORD_DIM;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
new file mode 100644
index 0000000..1ac67fe
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCrossesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STCrossesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.crosses(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_CROSSES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
new file mode 100644
index 0000000..a0209f9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDifferenceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.difference(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DIFFERENCE;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
new file mode 100644
index 0000000..492d045
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDimensionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDimensionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.dimension();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DIMENSION;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
new file mode 100644
index 0000000..17272ce
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDisjointDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDisjointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.disjoint(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DISJOINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
new file mode 100644
index 0000000..50cb129
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDistanceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STDistanceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.distance(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_DISTANCE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
new file mode 100644
index 0000000..6b8e43b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEndPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEndPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).endPoint();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_END_POINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
new file mode 100644
index 0000000..f5d7d0d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEnvelopeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEnvelopeDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.envelope();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_ENVELOPE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
new file mode 100644
index 0000000..66c2ab1
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEqualsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STEqualsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.equals(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_EQUALS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
new file mode 100644
index 0000000..bda2c90
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STExteriorRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STExteriorRingDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).exteriorRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_EXTERIOR_RING;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
new file mode 100644
index 0000000..81fa68d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromTextDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_TEXT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromTextEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromTextEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private OperatorImportFromWkt wktImporter;
+
+        public STGeomFromTextEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            wktImporter = OperatorImportFromWkt.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_GEOM_FROM_TEXT, 0, data[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                OGCStructure structure;
+
+                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
new file mode 100644
index 0000000..42401c5
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
@@ -0,0 +1,144 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextSRIDDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromTextSRIDDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_TEXT_SRID;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromTextSRIDEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromTextSRIDEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private OperatorImportFromWkt wktImporter;
+
+        public STGeomFromTextSRIDEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[1].createScalarEvaluator(ctx);
+            wktImporter = OperatorImportFromWkt.local();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            try {
+                String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+                int srid = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+                OGCStructure structure;
+
+                structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+                OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(srid));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] wKBGeometryBuffer = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(wKBGeometryBuffer.length);
+                out.write(wKBGeometryBuffer);
+                result.set(resultStorage);
+
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
new file mode 100644
index 0000000..e949b90
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STGeomFromWKBDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomFromWKBDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOM_FROM_WKB;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STGeomFromWKBEvaulator(args, ctx);
+            }
+        };
+    }
+
+    private class STGeomFromWKBEvaulator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+
+        public STGeomFromWKBEvaulator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] data = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            if (data[offset] != ATypeTag.SERIALIZED_BINARY_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_GEOM_FROM_WKB, 0, data[offset],
+                        ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
+            }
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+                DataInputStream dataIn = new DataInputStream(inStream);
+                data = ByteArraySerializerDeserializer.read(dataIn);
+                out.writeInt(data.length);
+                out.write(data);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
new file mode 100644
index 0000000..8e7ae39
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeomentryTypeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeomentryTypeDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.geometryType();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GEOMETRY_TYPE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
new file mode 100644
index 0000000..9bf3699
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeometryNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STGeometryNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_GEOMETRY_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCGeometryCollection) {
+            return ((OGCGeometryCollection) geometry).geometryN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
new file mode 100644
index 0000000..25df415
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STInteriorRingNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STInteriorRingNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERIOR_RING_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).interiorRingN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
new file mode 100644
index 0000000..6f3314e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIntersectionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.intersection(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERSECTION;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
new file mode 100644
index 0000000..1a4f37f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIntersectsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.intersects(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_INTERSECTS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
new file mode 100644
index 0000000..59edb4e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
@@ -0,0 +1,78 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import com.esri.core.geometry.ogc.OGCMultiCurve;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPoint;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsClosedDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsClosedDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return isClosed(geometry);
+    }
+
+    private boolean isClosed(OGCGeometry geometry) {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).isClosed();
+        } else if (geometry instanceof OGCMultiCurve) {
+            return ((OGCMultiCurve) geometry).isClosed();
+        } else if (geometry instanceof OGCMultiPoint || geometry instanceof OGCMultiPolygon
+                || geometry instanceof OGCPolygon || geometry instanceof OGCPoint) {
+            return true;
+        } else if (geometry instanceof OGCGeometryCollection) {
+            OGCGeometryCollection geometryCollection = (OGCGeometryCollection) geometry;
+            int num = geometryCollection.numGeometries();
+            for (int i = 0; i < num; ++i) {
+                if (!isClosed(geometryCollection.geometryN(i))) {
+                    return false;
+                }
+            }
+            return true;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_CLOSED;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
new file mode 100644
index 0000000..92767e9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsCollectionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsCollectionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        String type = geometry.geometryType();
+        return "GeometryCollection".equals(type) || "MultiLineString".equals(type) || "MultiPoint".equals(type)
+                || "MultiPolygon".equals(type);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_COLLECTION;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
new file mode 100644
index 0000000..acf20b7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsEmptyDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsEmptyDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.isEmpty();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_EMPTY;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
new file mode 100644
index 0000000..3c9375a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsRingDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).isRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_RING;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
new file mode 100644
index 0000000..fdb948f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsSimpleDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STIsSimpleDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.isSimple();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_IS_SIMPLE;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
new file mode 100644
index 0000000..65fd085
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLengthDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STLengthDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCLineString) {
+            return geometry.getEsriGeometry().calculateLength2D();
+        } else if (geometry instanceof OGCMultiLineString) {
+            GeometryCursor cursor = geometry.getEsriGeometryCursor();
+            double length = 0;
+            Geometry geometry1 = cursor.next();
+            while (geometry1 != null) {
+                length += geometry1.calculateLength2D();
+                geometry1 = cursor.next();
+            }
+            return length;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_LENGTH;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
new file mode 100644
index 0000000..c783fe0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.Polyline;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLineFromMultiPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STLineFromMultiPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCMultiPoint) {
+            Polyline polyline = new Polyline();
+            OGCMultiPoint multiPoint = (OGCMultiPoint) geometry;
+            int numPoints = multiPoint.numGeometries();
+            polyline.startPath((Point) multiPoint.geometryN(0).getEsriGeometry());
+            for (int i = 1; i < numPoints; i++) {
+                polyline.lineTo((Point) multiPoint.geometryN(i).getEsriGeometry());
+            }
+            return new OGCLineString(polyline, 0, SpatialReference.create(4326));
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_LINE_FROM_MULTIPOINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
new file mode 100644
index 0000000..46ce0fb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STMDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).M();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_M;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
new file mode 100644
index 0000000..5fa6da8
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
@@ -0,0 +1,146 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STMakeEnvelopeDescriptorSRID extends AbstractGetValDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakeEnvelopeDescriptorSRID();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_ENVELOPE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STMakeEnvelopeEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STMakeEnvelopeEvaluator implements IScalarEvaluator {
+
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private IPointable inputArg1;
+        private IScalarEvaluator eval1;
+        private IPointable inputArg2;
+        private IScalarEvaluator eval2;
+        private IPointable inputArg3;
+        private IScalarEvaluator eval3;
+        private IPointable inputArg4;
+        private IScalarEvaluator eval4;
+
+        public STMakeEnvelopeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            inputArg1 = new VoidPointable();
+            eval1 = args[1].createScalarEvaluator(ctx);
+            inputArg2 = new VoidPointable();
+            eval2 = args[2].createScalarEvaluator(ctx);
+            inputArg3 = new VoidPointable();
+            eval3 = args[3].createScalarEvaluator(ctx);
+            inputArg4 = new VoidPointable();
+            eval4 = args[4].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            byte[] data0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+
+            eval1.evaluate(tuple, inputArg1);
+            byte[] data1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+
+            eval2.evaluate(tuple, inputArg2);
+            byte[] data2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+
+            eval3.evaluate(tuple, inputArg3);
+            byte[] data3 = inputArg3.getByteArray();
+            int offset3 = inputArg3.getStartOffset();
+
+            eval4.evaluate(tuple, inputArg4);
+            byte[] data4 = inputArg4.getByteArray();
+            int offset4 = inputArg4.getStartOffset();
+
+            try {
+
+                OGCGeometry ogcGeometry =
+                        OGCGeometry
+                                .createFromEsriGeometry(
+                                        new Envelope(getVal(data0, offset0), getVal(data1, offset1),
+                                                getVal(data2, offset2), getVal(data3, offset3)),
+                                        SpatialReference.create((int) getVal(data4, offset4)));
+                ByteBuffer buffer = ogcGeometry.asBinary();
+                byte[] bytes = buffer.array();
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                out.writeInt(bytes.length);
+                out.write(bytes);
+                result.set(resultStorage);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
new file mode 100644
index 0000000..3c0ac97
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePoint3DDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePoint3DEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT3D;
+    }
+
+    private class STMakePoint3DEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private IPointable inputArg2;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private final IScalarEvaluator eval2;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePoint3DEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            inputArg2 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            eval2 = args[2].createScalarEvaluator(ctx);
+            point = new Point(0, 0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+            eval2.evaluate(tuple, inputArg2);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            byte[] bytes2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                point.setZ(getVal(bytes2, offset2));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
new file mode 100644
index 0000000..424fb8f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DWithMDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePoint3DWithMDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePoint3DWithMEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT3D_M;
+    }
+
+    private class STMakePoint3DWithMEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private IPointable inputArg2;
+        private IPointable inputArg3;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private final IScalarEvaluator eval2;
+        private final IScalarEvaluator eval3;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePoint3DWithMEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            inputArg2 = new VoidPointable();
+            inputArg3 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            eval2 = args[2].createScalarEvaluator(ctx);
+            eval3 = args[3].createScalarEvaluator(ctx);
+            point = new Point(0, 0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+            eval2.evaluate(tuple, inputArg2);
+            eval3.evaluate(tuple, inputArg3);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            byte[] bytes2 = inputArg2.getByteArray();
+            int offset2 = inputArg2.getStartOffset();
+            byte[] bytes3 = inputArg3.getByteArray();
+            int offset3 = inputArg3.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                point.setZ(getVal(bytes2, offset2));
+                point.setM(getVal(bytes3, offset3));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
new file mode 100644
index 0000000..2d901a7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePointDescriptor extends AbstractGetValDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMakePointDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STMakePointEvaluator(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MAKE_POINT;
+    }
+
+    private class STMakePointEvaluator implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage resultStorage;
+        private final DataOutput out;
+        private IPointable inputArg0;
+        private IPointable inputArg1;
+        private final IScalarEvaluator eval0;
+        private final IScalarEvaluator eval1;
+        private Point point;
+        private AGeometry pointGeometry;
+
+        public STMakePointEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg0 = new VoidPointable();
+            inputArg1 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            eval1 = args[1].createScalarEvaluator(ctx);
+            point = new Point(0, 0);
+            pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval0.evaluate(tuple, inputArg0);
+            eval1.evaluate(tuple, inputArg1);
+
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+
+            resultStorage.reset();
+            try {
+                out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                point.setX(getVal(bytes0, offset0));
+                point.setY(getVal(bytes1, offset1));
+                AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+            } catch (IOException e1) {
+                throw HyracksDataException.create(e1);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
new file mode 100644
index 0000000..ec51ae3
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.MultiVertexGeometry;
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNPointsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNPointsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeometry = geometry.getEsriGeometry();
+        if (esriGeometry instanceof MultiVertexGeometry) {
+            return ((MultiVertexGeometry) esriGeometry).getPointCount();
+        } else if (esriGeometry instanceof Point) {
+            return 1;
+        } else if (esriGeometry == null) {
+            int count = 0;
+            GeometryCursor geometryCursor = geometry.getEsriGeometryCursor();
+            esriGeometry = geometryCursor.next();
+            while (esriGeometry != null) {
+                if (esriGeometry instanceof MultiVertexGeometry) {
+                    count += ((MultiVertexGeometry) esriGeometry).getPointCount();
+                } else if (esriGeometry instanceof Point) {
+                    count += 1;
+                }
+                esriGeometry = geometryCursor.next();
+            }
+            return count;
+        } else if (geometry.isEmpty()) {
+            return 0;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_N_POINTS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
new file mode 100644
index 0000000..27e0490
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNRingsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).numInteriorRing() + 1;
+        } else if (geometry instanceof OGCMultiPolygon) {
+            OGCMultiPolygon polygon = (OGCMultiPolygon) geometry;
+            int numGeometries = polygon.numGeometries();
+            int count = 0;
+            for (int i = 1; i < numGeometries + 1; i++) {
+                if (polygon.geometryN(i) instanceof OGCPolygon) {
+                    count += ((OGCPolygon) polygon.geometryN(i)).numInteriorRing() + 1;
+                }
+            }
+            return count;
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_N_RINGS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
new file mode 100644
index 0000000..b8d0c52
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumGeometriesDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNumGeometriesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCGeometryCollection) {
+            return ((OGCGeometryCollection) geometry).numGeometries();
+        } else if (!geometry.isEmpty()) {
+            return 1;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_NUM_GEOMETRIIES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
new file mode 100644
index 0000000..0c32ec5
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumInteriorRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STNumInteriorRingsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPolygon) {
+            return ((OGCPolygon) geometry).numInteriorRing();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_NUM_INTERIOR_RINGS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
new file mode 100644
index 0000000..f131499
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STOverlapsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STOverlapsDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.overlaps(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_OVERLAPS;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
new file mode 100644
index 0000000..e3ce3e0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STPointNDescriptor extends AbstractSTGeometryNDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STPointNDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_POINT_N;
+    }
+
+    @Override
+    protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+        if (geometry instanceof OGCLineString) {
+            return ((OGCLineString) geometry).pointN(n);
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
new file mode 100644
index 0000000..12024bc
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
@@ -0,0 +1,141 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCConcreteGeometryCollection;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class STPolygonizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STPolygonizeDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_POLYGONIZE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+                return new STPolygonizeEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STPolygonizeEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+
+        public STPolygonizeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[0].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] bytes = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            AOrderedListType type = new AOrderedListType(BuiltinType.AGEOMETRY, null);
+            byte typeTag = inputArg.getByteArray()[inputArg.getStartOffset()];
+            ISerializerDeserializer serde;
+            if (typeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+                serde = new AOrderedListSerializerDeserializer(type);
+            } else if (typeTag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+                serde = new AOrderedListSerializerDeserializer(type);
+            } else {
+                throw new TypeMismatchException(sourceLoc, BuiltinFunctions.ST_POLYGONIZE, 0, typeTag,
+                        ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            IACursor cursor = ((IACollection) serde.deserialize(dataIn)).getCursor();
+            List<OGCGeometry> list = new ArrayList<>();
+            while (cursor.next()) {
+                IAObject object = cursor.get();
+                list.add(((AGeometry) object).getGeometry());
+            }
+            OGCGeometryCollection geometryCollection =
+                    new OGCConcreteGeometryCollection(list, SpatialReference.create(4326));
+            try {
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY)
+                        .serialize(new AGeometry(geometryCollection), out);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
new file mode 100644
index 0000000..5745dca
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
@@ -0,0 +1,150 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STRelateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STRelateDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_RELATE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new STRelateEvaluator(args, ctx);
+            }
+        };
+    }
+
+    private class STRelateEvaluator implements IScalarEvaluator {
+        private ArrayBackedValueStorage resultStorage;
+        private DataOutput out;
+        private IPointable inputArg;
+        private IScalarEvaluator eval;
+        private IPointable inputArg0;
+        private IScalarEvaluator eval0;
+        private final IPointable inputArg1;
+        private final IScalarEvaluator eval1;
+
+        public STRelateEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            resultStorage = new ArrayBackedValueStorage();
+            out = resultStorage.getDataOutput();
+            inputArg = new VoidPointable();
+            eval = args[2].createScalarEvaluator(ctx);
+            inputArg0 = new VoidPointable();
+            eval0 = args[0].createScalarEvaluator(ctx);
+            inputArg1 = new VoidPointable();
+            eval1 = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            eval.evaluate(tuple, inputArg);
+            byte[] bytes = inputArg.getByteArray();
+            int offset = inputArg.getStartOffset();
+            int len = inputArg.getLength();
+
+            eval0.evaluate(tuple, inputArg0);
+            byte[] bytes0 = inputArg0.getByteArray();
+            int offset0 = inputArg0.getStartOffset();
+            int len0 = inputArg0.getLength();
+
+            eval1.evaluate(tuple, inputArg1);
+            byte[] bytes1 = inputArg1.getByteArray();
+            int offset1 = inputArg1.getStartOffset();
+            int len1 = inputArg1.getLength();
+
+            if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
+                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            }
+            ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+            if (tag != ATypeTag.GEOMETRY) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+            if (tag != ATypeTag.GEOMETRY) {
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+
+            ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+            DataInputStream dataIn = new DataInputStream(inStream);
+            String matrix = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+            DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+            OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+            DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+            OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+            try {
+                boolean val = geometry0.relate(geometry1, matrix);
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+                        .serialize(val ? ABoolean.TRUE : ABoolean.FALSE, out);
+            } catch (IOException e) {
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                        ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+            }
+            result.set(resultStorage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
new file mode 100644
index 0000000..a46749d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSRIDDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STSRIDDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.SRID();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_SRID;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
new file mode 100644
index 0000000..988e781
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STStartPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STStartPointDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCCurve) {
+            return ((OGCCurve) geometry).startPoint();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_START_POINT;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
new file mode 100644
index 0000000..e92f56b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSymDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STSymDifferenceDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.symDifference(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_SYM_DIFFERENCE;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
new file mode 100644
index 0000000..93a947a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STTouchesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STTouchesDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.touches(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_TOUCHES;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
new file mode 100644
index 0000000..43891f0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STUnionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STUnionDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.union(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_UNION;
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
new file mode 100644
index 0000000..f563d61
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STWithinDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STWithinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+        return geometry0.within(geometry1);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_WITHIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
new file mode 100644
index 0000000..c796b1b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).X();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
new file mode 100644
index 0000000..19864bb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getXMax();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
new file mode 100644
index 0000000..12378c6
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STXMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getXMin();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_X_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
new file mode 100644
index 0000000..782370d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).Y();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
new file mode 100644
index 0000000..83ea647
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getYMax();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
new file mode 100644
index 0000000..c6b5f36
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STYMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        Geometry esriGeom = geometry.getEsriGeometry();
+        if (esriGeom != null) {
+            Envelope env = new Envelope();
+            esriGeom.queryEnvelope(env);
+            return env.getYMin();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Y_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
new file mode 100644
index 0000000..b5b6d78
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        if (geometry instanceof OGCPoint) {
+            return ((OGCPoint) geometry).Z();
+        } else {
+            throw new UnsupportedOperationException(
+                    "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+        }
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
new file mode 100644
index 0000000..7933f08
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZMaxDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.MaxZ();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z_MAX;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
new file mode 100644
index 0000000..d0da92e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STZMinDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+        return geometry.MinZ();
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_Z_MIN;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
new file mode 100644
index 0000000..d9529e2
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+org.apache.asterix.geo.evaluators.GeoFunctionRegistrant
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
index bb4d346..1174717 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/DistinctClause.java
@@ -22,11 +22,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DistinctClause implements Clause {
+public class DistinctClause extends AbstractClause {
 
     private List<Expression> distinctByExprs;
 
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
index 6cdc2f3..9bae2a0 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/ForClause.java
@@ -20,12 +20,12 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class ForClause implements Clause {
+public class ForClause extends AbstractClause {
     private VariableExpr varExpr = null;
     private VariableExpr posExpr = null;
     private Expression inExpr = null;
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
index 4767f4f..f0ed087 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/JoinClause.java
@@ -22,11 +22,12 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLPlusVisitor;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class JoinClause implements Clause {
+public class JoinClause extends AbstractClause {
 
     public static enum JoinKind {
         INNER,
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
index 871c2ce..7712cf9 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/clause/MetaVariableClause.java
@@ -20,11 +20,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.base.IAQLPlusVisitor;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class MetaVariableClause implements Clause {
+public class MetaVariableClause extends AbstractClause {
     private VarIdentifier var;
 
     @Override
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
index d2cb64f..967fdae 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.aql.rewrites;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
@@ -69,15 +70,15 @@
 
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
-            throws CompilationException {
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         setup(declaredFunctions, topStatement, metadataProvider, context);
         if (topStatement.isTopLevel()) {
             wrapInLets();
         }
         inlineDeclaredUdfs();
         rewriteFunctionName();
-        topStatement.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter().get());
     }
 
     private void wrapInLets() {
@@ -119,7 +120,7 @@
         for (Expression topLevelExpr : topStatement.getDirectlyEnclosedExpressions()) {
             storedFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr, funIds,
                     null, expr -> getFunctionCalls(expr), func -> functionParser.getFunctionDecl(func),
-                    signature -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature)));
+                    (signature, sourceLoc) -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature)));
             declaredFunctions.addAll(storedFunctionDecls);
         }
         if (!declaredFunctions.isEmpty()) {
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
index a89e8bd..5ca91b6 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
@@ -36,4 +36,9 @@
             stmt.accept(visitor, null);
         }
     }
+
+    @Override
+    public String toExternalVariableName(String statementParameterName) {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
index 9cf8b8d..23b0066 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
@@ -60,7 +60,7 @@
 
     public static IRangeMap parseHint(Object hint) throws CompilationException {
         ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
-        DataOutput out = abvs.getDataOutput();;
+        DataOutput out = abvs.getDataOutput();
         abvs.reset();
 
         IParser parser = parserFactory.createParser((String) hint);
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
index e90f9a9..22ef0e3 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
@@ -21,7 +21,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 2d87556..856073d 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -147,6 +147,7 @@
 import org.apache.asterix.lang.common.statement.UpsertStatement;
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -1811,23 +1812,25 @@
 Expression MultExpr()throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = ExponentExpr()
 
-    (( <MUL> | <DIV> | <MOD> | <IDIV>)
+    ( (
+        <MUL> { opType = OperatorType.MUL; } |
+        <DIVIDE> { opType = OperatorType.DIVIDE; } |
+        <DIV> { opType = OperatorType.DIV; } |
+        ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+      )
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
         }
-        try{
-          op.addOperator(token.image);
-        } catch (CompilationException e){
-          throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
     operand = ExponentExpr()
     {
@@ -2724,10 +2727,10 @@
 TOKEN :
 {
     <CARET : "^">
-  | <DIV : "/">
-  | <IDIV : "idiv">
+  | <DIVIDE : "/">
+  | <DIV : "div">
   | <MINUS : "-">
-  | <MOD : "%">
+  | <MOD : "mod">
   | <MUL : "*">
   | <PLUS : "+">
 
@@ -2739,6 +2742,7 @@
   | <COLON : ":">
   | <COMMA : ",">
   | <DOT : ".">
+  | <PERCENT: "%">
   | <QUES : "?">
 
   | <LT : "<">
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index b20bd58..96123d0 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -103,10 +103,5 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-data-std</artifactId>
-    </dependency>
   </dependencies>
-
 </project>
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java
new file mode 100644
index 0000000..bc7b45b
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractClause.java
@@ -0,0 +1,23 @@
+/*
+ * 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.asterix.lang.common.base;
+
+public abstract class AbstractClause extends AbstractLangExpression implements Clause {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
index 44a05ef..744d3cc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
@@ -23,16 +23,34 @@
 
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 
-public abstract class AbstractExpression implements Expression {
+/**
+ * All subclasses need to make sure the hints are copied over in the DeepCopyVisitor and
+ * CloneAndSubstituteVariablesVisitor
+ */
+public abstract class AbstractExpression extends AbstractLangExpression implements Expression {
+
     protected List<IExpressionAnnotation> hints;
 
     public void addHint(IExpressionAnnotation hint) {
+        if (hint == null) {
+            return;
+        }
         if (hints == null) {
             hints = new ArrayList<>();
         }
         hints.add(hint);
     }
 
+    public void addHints(List<IExpressionAnnotation> newHints) {
+        if (newHints == null) {
+            return;
+        }
+        if (hints == null) {
+            hints = new ArrayList<>();
+        }
+        hints.addAll(newHints);
+    }
+
     public boolean hasHints() {
         return hints != null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java
new file mode 100644
index 0000000..3a784e3
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractLangExpression.java
@@ -0,0 +1,35 @@
+/*
+ * 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.asterix.lang.common.base;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractLangExpression implements ILangExpression {
+    private SourceLocation sourceLoc;
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java
new file mode 100644
index 0000000..e07683d
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractStatement.java
@@ -0,0 +1,23 @@
+/*
+ * 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.asterix.lang.common.base;
+
+public abstract class AbstractStatement extends AbstractLangExpression implements Statement {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
index f6b9399..e80d7bb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/ILangExpression.java
@@ -20,7 +20,10 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILangExpression {
-    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException;
+    <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException;
+
+    SourceLocation getSourceLocation();
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 05ab836..8f3b8a9 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.lang.common.base;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
@@ -25,24 +26,27 @@
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
 public interface IQueryRewriter {
 
     /**
      * Rewrite a query at the AST level.
-     *
      * @param declaredFunctions,
-     *            a list of declared functions associated with the query.
+     *          a list of declared functions associated with the query.
      * @param topExpr,
-     *            the query to be rewritten.
+     *          the query to be rewritten.
      * @param metadataProvider,
-     *            providing the definition of created (i.e., stored) user-defined functions.
-     * @param context,
-     *            manages ids of variables and guarantees uniqueness of variables.
+     *          providing the definition of created (i.e., stored) user-defined functions.
+     * @param context
+     *          rewriting context
+     * @param externalVars
+     *          external variables
      */
     void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context, boolean inlineUdfs) throws CompilationException;
+            LangRewritingContext context, boolean inlineUdfs, Collection<VarIdentifier> externalVars)
+            throws CompilationException;
 
     /**
      * Find the function calls used by a given expression
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java
index d31b765..02e5267 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java
@@ -19,8 +19,6 @@
 
 package org.apache.asterix.lang.common.base;
 
-import org.apache.asterix.lang.common.struct.VarIdentifier;
-
 import java.util.List;
 
 /**
@@ -66,16 +64,4 @@
      *            the main body expression.
      */
     void setBody(Expression expr);
-
-    /**
-     * @return external (pre-defined) variables for the statement
-     */
-    List<VarIdentifier> getExternalVars();
-
-    /**
-     * Sets external (pre-defined) variables for the statement
-     *
-     * @param externalVars external variables
-     */
-    void setExternalVars(List<VarIdentifier> externalVars);
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
index 7de67f7..0584665 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
@@ -26,6 +26,7 @@
      * @param statement,
      *            a non-query statement.
      */
-    public void rewrite(Statement statement) throws CompilationException;
+    void rewrite(Statement statement) throws CompilationException;
 
+    String toExternalVariableName(String statementParameterName);
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
index 31c903e..20dba52 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/GroupbyClause.java
@@ -24,7 +24,7 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -32,7 +32,7 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class GroupbyClause implements Clause {
+public class GroupbyClause extends AbstractClause {
 
     private List<GbyVariableExpressionPair> gbyPairList;
     private List<GbyVariableExpressionPair> decorPairList;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
index df560ea..750cbe3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LetClause.java
@@ -21,12 +21,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LetClause implements Clause {
+public class LetClause extends AbstractClause {
     private VariableExpr varExpr;
     private Expression bindExpr;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
index fbfa64e..d597bc2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/LimitClause.java
@@ -21,11 +21,11 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LimitClause implements Clause {
+public class LimitClause extends AbstractClause {
     private Expression limitExpr;
     private Expression offset;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index c1ceaa8..faefe1d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -22,12 +22,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 
-public class OrderbyClause implements Clause {
+public class OrderbyClause extends AbstractClause {
     private List<Expression> orderbyList;
     private List<OrderModifier> modifierList;
     private IRangeMap rangeMap;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
index 4089c07..49c0c58 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/UpdateClause.java
@@ -21,14 +21,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.statement.DeleteStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UpdateClause implements Clause {
+public class UpdateClause extends AbstractClause {
 
     private Expression target;
     private Expression value;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
index c540588..9153400 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/WhereClause.java
@@ -19,11 +19,11 @@
 package org.apache.asterix.lang.common.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class WhereClause implements Clause {
+public class WhereClause extends AbstractClause {
     private Expression whereExpr;
 
     public WhereClause() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
index bf46c5d..fa47f5e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
@@ -23,6 +23,7 @@
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Set;
+import java.util.function.Predicate;
 
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -187,12 +188,20 @@
     }
 
     public Set<VariableExpr> getLiveVariables(Scope stopAtExclusive) {
+        return getLiveVariables(stopAtExclusive, null);
+    }
+
+    public Set<VariableExpr> getLiveVariables(Scope stopAtExclusive, Predicate<? super VarIdentifier> excludeFilter) {
         LinkedHashSet<VariableExpr> vars = new LinkedHashSet<>();
         Iterator<Identifier> identifierIterator = liveSymbols(stopAtExclusive);
         while (identifierIterator.hasNext()) {
             Identifier identifier = identifierIterator.next();
             if (identifier instanceof VarIdentifier) {
-                vars.add(new VariableExpr((VarIdentifier) identifier));
+                VarIdentifier varId = (VarIdentifier) identifier;
+                if (excludeFilter != null && excludeFilter.test(varId)) {
+                    continue;
+                }
+                vars.add(new VariableExpr(varId));
             }
         }
         return vars;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java
new file mode 100644
index 0000000..f55f055
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/AbstractTypeExpression.java
@@ -0,0 +1,25 @@
+/*
+ * 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.asterix.lang.common.expression;
+
+import org.apache.asterix.lang.common.base.AbstractLangExpression;
+
+public abstract class AbstractTypeExpression extends AbstractLangExpression implements TypeExpression {
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
index 0c3fdb1..ab6b3f1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/LiteralExpr.java
@@ -21,19 +21,9 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.AbstractExpression;
 import org.apache.asterix.lang.common.base.Literal;
-import org.apache.asterix.lang.common.literal.DoubleLiteral;
-import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
-import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.object.base.AdmBigIntNode;
-import org.apache.asterix.object.base.AdmBooleanNode;
-import org.apache.asterix.object.base.AdmDoubleNode;
-import org.apache.asterix.object.base.AdmNullNode;
-import org.apache.asterix.object.base.AdmStringNode;
-import org.apache.asterix.object.base.IAdmNode;
 
 public class LiteralExpr extends AbstractExpression {
     private Literal value;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
index 90b457e..9521469 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
@@ -84,10 +84,18 @@
         exprList.add(operand);
     }
 
-    public static final boolean opIsComparison(OperatorType t) {
-        boolean cmp = t == OperatorType.EQ || t == OperatorType.NEQ || t == OperatorType.GT;
-        cmp = cmp || t == OperatorType.GE || t == OperatorType.LT || t == OperatorType.LE;
-        return cmp;
+    public static boolean opIsComparison(OperatorType t) {
+        switch (t) {
+            case EQ:
+            case NEQ:
+            case GT:
+            case GE:
+            case LT:
+            case LE:
+                return true;
+            default:
+                return false;
+        }
     }
 
     public void addOperator(String strOp) throws CompilationException {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
index 58d3756..d4c03b2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OrderedListTypeDefinition.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class OrderedListTypeDefinition implements TypeExpression {
+public class OrderedListTypeDefinition extends AbstractTypeExpression {
 
     private TypeExpression itemTypeExpression;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
index 4aecd92..4fcc7ae 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
@@ -27,7 +27,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class RecordTypeDefinition implements TypeExpression {
+public class RecordTypeDefinition extends AbstractTypeExpression {
 
     public enum RecordKind {
         OPEN,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
index 8c1e5c6..a76e5bb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class TypeReferenceExpression implements TypeExpression {
+public class TypeReferenceExpression extends AbstractTypeExpression {
 
     private final Pair<Identifier, Identifier> ident;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
index 8f4b1b7..b55c910 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/UnorderedListTypeDefinition.java
@@ -23,7 +23,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UnorderedListTypeDefinition implements TypeExpression {
+public class UnorderedListTypeDefinition extends AbstractTypeExpression {
 
     private TypeExpression itemTypeExpression;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
index 7dadd67..f332b0a 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
@@ -303,7 +303,8 @@
     }
 
     protected String getLine(int line) {
-        return inputLines[line - 1];
+        int idx = line - 1;
+        return idx >= 0 && idx < inputLines.length ? inputLines[idx] : "";
     }
 
     protected String extractFragment(int beginLine, int beginColumn, int endLine, int endColumn) {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 10cd7d8..b2fd3a0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -19,27 +19,27 @@
 package org.apache.asterix.lang.common.rewrites;
 
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 
 public final class LangRewritingContext {
-    private int mark = 0;
-    private int varCounter;
+    private Counter varCounter;
     private int systemVarCounter = 1;
-    private HashMap<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
+    private Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
+    private Set<VarIdentifier> excludedForFieldAccessVars = new HashSet<>();
 
     public LangRewritingContext(int varCounter) {
-        this.varCounter = varCounter;
+        this.varCounter = new Counter(varCounter);
     }
 
-    public int getVarCounter() {
+    public Counter getVarCounter() {
         return varCounter;
     }
 
-    public void setVarCounter(int varCounter) {
-        this.varCounter = varCounter;
-    }
-
     /**
      * Generate a new variable with the same identifier (varValue) but a different Id.
      *
@@ -50,7 +50,7 @@
      * @return the new varible.
      */
     public VarIdentifier mapOldId(Integer oldId, String varValue) {
-        int n = newId();
+        int n = increaseAndGetCtr();
         VarIdentifier newVar = new VarIdentifier(varValue);
         newVar.setId(n);
         oldVarIdToNewVarId.put(oldId, newVar);
@@ -62,22 +62,21 @@
     }
 
     public VarIdentifier newVariable() {
-        int id = newId();
+        int id = increaseAndGetCtr();
         // Prefixes system-generated variables with "#".
         return new VarIdentifier("#" + (systemVarCounter++), id);
     }
 
-    public void markCounter() {
-        mark = varCounter;
+    private int increaseAndGetCtr() {
+        varCounter.inc();
+        return varCounter.get();
     }
 
-    public void resetCounter() {
-        varCounter = mark;
+    public void addExcludedForFieldAccessVar(VarIdentifier varId) {
+        excludedForFieldAccessVars.add(varId);
     }
 
-    private int newId() {
-        varCounter++;
-        return varCounter;
+    public boolean isExcludedForFieldAccessVar(VarIdentifier varId) {
+        return excludedForFieldAccessVars.contains(varId);
     }
-
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
index d2cc431..d35516b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CompactStatement implements Statement {
+public class CompactStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
index b0a3f6e..eaa95cd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
@@ -22,13 +22,14 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class ConnectFeedStatement implements Statement {
+public class ConnectFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
index dcbf334..9881f83 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
@@ -19,12 +19,13 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
-public class CreateDataverseStatement implements Statement {
+public class CreateDataverseStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private String format;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
index 2deec27..ea6955b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedPolicyStatement.java
@@ -21,10 +21,11 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CreateFeedPolicyStatement implements Statement {
+public class CreateFeedPolicyStatement extends AbstractStatement {
 
     private final String policyName;
     private final String sourcePolicyName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index a4acfdf..3d5b815 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -19,7 +19,7 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.util.ExpressionUtils;
@@ -35,7 +35,7 @@
  * The new create feed statement only concerns the feed adaptor configuration.
  * All feeds are considered as primary feeds.
  */
-public class CreateFeedStatement implements Statement {
+public class CreateFeedStatement extends AbstractStatement {
 
     private final Pair<Identifier, Identifier> qName;
     private final boolean ifNotExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
index c5fa524..f1cc6ba 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFunctionStatement.java
@@ -23,12 +23,13 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class CreateFunctionStatement implements Statement {
+public class CreateFunctionStatement extends AbstractStatement {
 
     private final FunctionSignature signature;
     private final String functionBody;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 1dc8673..2e9f1f9 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -23,13 +23,14 @@
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class CreateIndexStatement implements Statement {
+public class CreateIndexStatement extends AbstractStatement {
 
     private Identifier indexName;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index f8cf62c..4aeb6d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -38,7 +39,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
-public class DatasetDecl implements Statement {
+public class DatasetDecl extends AbstractStatement {
     protected static final String[] WITH_OBJECT_FIELDS = new String[] { MergePolicyUtils.MERGE_POLICY_PARAMETER_NAME };
     protected static final Set<String> WITH_OBJECT_FIELDS_SET = new HashSet<>(Arrays.asList(WITH_OBJECT_FIELDS));
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
index 1b0c5cd..99303ce 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DataverseDecl implements Statement {
+public class DataverseDecl extends AbstractStatement {
 
     private Identifier dataverseName;
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
index 6349006..9e184c2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DataverseDropStatement implements Statement {
+public class DataverseDropStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
index 18045b5..f05efbc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
@@ -21,13 +21,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DeleteStatement implements Statement {
+public class DeleteStatement extends AbstractStatement {
 
     private VariableExpr vars;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
index 33fa559..cb33452 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
@@ -19,12 +19,13 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class DisconnectFeedStatement implements Statement {
+public class DisconnectFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
index 45f5875..1434620 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class DropDatasetStatement implements Statement {
+public class DropDatasetStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
index 47e51e3..ef6c096 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FeedDropStatement implements Statement {
+public class FeedDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
index 1b23237..a95254b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FeedPolicyDropStatement implements Statement {
+public class FeedPolicyDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier policyName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
index 2ea6c40..4161824 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDecl.java
@@ -22,12 +22,13 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FunctionDecl implements Statement {
+public class FunctionDecl extends AbstractStatement {
     private FunctionSignature signature;
     private List<VarIdentifier> paramList;
     private Expression funcBody;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
index 610a20d..0496ecc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FunctionDropStatement.java
@@ -20,10 +20,11 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class FunctionDropStatement implements Statement {
+public class FunctionDropStatement extends AbstractStatement {
 
     private final FunctionSignature signature;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
index c262855..39e1d31 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class IndexDropStatement implements Statement {
+public class IndexDropStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
index de12dee..7bb2cbe 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
@@ -23,15 +23,15 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class InsertStatement implements IReturningStatement {
+public class InsertStatement extends AbstractStatement implements IReturningStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
@@ -115,16 +115,6 @@
     }
 
     @Override
-    public List<VarIdentifier> getExternalVars() {
-        return null;
-    }
-
-    @Override
-    public void setExternalVars(List<VarIdentifier> externalVars) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
     public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
         return visitor.visit(this, arg);
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
index 92ea383..5366bc0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
@@ -21,11 +21,12 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class LoadStatement implements Statement {
+public class LoadStatement extends AbstractStatement {
 
     private Identifier datasetName;
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
index 5ec0181..7bb0c69 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodeGroupDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class NodeGroupDropStatement implements Statement {
+public class NodeGroupDropStatement extends AbstractStatement {
 
     private Identifier nodeGroupName;
     private boolean ifExists;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
index 1213f11..e4b3f5b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/NodegroupDecl.java
@@ -21,11 +21,12 @@
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class NodegroupDecl implements Statement {
+public class NodegroupDecl extends AbstractStatement {
 
     private Identifier nodegroupName;
     private List<Identifier> nodeControllerNames;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
index 6f2b6de..8b67556 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
@@ -23,17 +23,16 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class Query implements IReturningStatement {
+public class Query extends AbstractStatement implements IReturningStatement {
     private final boolean explain;
     private boolean topLevel = true;
     private Expression body;
-    private List<VarIdentifier> externalVars;
     private int varCounter;
 
     public Query(boolean explain) {
@@ -41,15 +40,10 @@
     }
 
     public Query(boolean explain, boolean topLevel, Expression body, int varCounter) {
-        this(explain, topLevel, body, varCounter, null);
-    }
-
-    public Query(boolean explain, boolean topLevel, Expression body, int varCounter, List<VarIdentifier> externalVars) {
         this.explain = explain;
         this.topLevel = topLevel;
         this.body = body;
         this.varCounter = varCounter;
-        this.externalVars = externalVars;
     }
 
     @Override
@@ -86,16 +80,6 @@
         return topLevel;
     }
 
-    @Override
-    public List<VarIdentifier> getExternalVars() {
-        return externalVars;
-    }
-
-    @Override
-    public void setExternalVars(List<VarIdentifier> externalVars) {
-        this.externalVars = externalVars;
-    }
-
     public boolean isExplain() {
         return explain;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
index 78d766c..4ca9d97 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class RefreshExternalDatasetStatement implements Statement {
+public class RefreshExternalDatasetStatement extends AbstractStatement {
 
     private Identifier dataverseName;
     private Identifier datasetName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
index 17727af..8c52b45 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/SetStatement.java
@@ -19,10 +19,11 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class SetStatement implements Statement {
+public class SetStatement extends AbstractStatement {
 
     private String propName;
     private String propValue;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
index bc37c42..3ea791d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class StartFeedStatement implements Statement {
+public class StartFeedStatement extends AbstractStatement {
 
     public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
     private Identifier dataverseName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
index 1fec6b2..99e5069 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
@@ -19,12 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
-public class StopFeedStatement implements Statement {
+public class StopFeedStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier feedName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
index 4f0ea52..5430945 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
@@ -20,12 +20,13 @@
 
 import org.apache.asterix.common.annotations.TypeDataGen;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class TypeDecl implements Statement {
+public class TypeDecl extends AbstractStatement {
 
     private final Identifier dataverseName;
     private final Identifier ident;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
index 059ac67..9f59f98 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class TypeDropStatement implements Statement {
+public class TypeDropStatement extends AbstractStatement {
 
     private final Identifier dataverseName;
     private Identifier typeName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
index 0bdbbd1..e631107 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpdateStatement.java
@@ -22,13 +22,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.UpdateClause;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class UpdateStatement implements Statement {
+public class UpdateStatement extends AbstractStatement {
 
     private VariableExpr vars;
     private Expression target;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
index 5dd9935..aabd999 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/WriteStatement.java
@@ -19,11 +19,12 @@
 package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class WriteStatement implements Statement {
+public class WriteStatement extends AbstractStatement {
 
     private final Identifier ncName;
     private final String fileName;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
index e11f169..f4f2ae1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
@@ -31,11 +31,10 @@
     MINUS("-"),
     CONCAT("||"),
     MUL("*"),
-    DIV("/"), // float/double
-    // divide
-    MOD("%"),
+    DIVIDE("/"),
+    DIV("div"),
+    MOD("mod"),
     CARET("^"),
-    IDIV("idiv"), // integer divide
     FUZZY_EQ("~="),
     LIKE("like"),
     NOT_LIKE("not_like"),
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
index 3e6caab..372acab 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
@@ -39,7 +39,6 @@
         addFunctionMapping("upper", "uppercase"); // upper, internal: uppercase
         addFunctionMapping("title", "initcap"); // title, internal: initcap
         addFunctionMapping("regexp_contains", "matches"); // regexp_contains, internal: matches
-        addFunctionMapping("power", "caret"); //pow, internal: caret
         addFunctionMapping("int", "integer"); // int, internal: integer
 
         // The "mapped-to" names are to be deprecated.
@@ -74,6 +73,11 @@
         addFunctionMapping("ifinf", "if-inf"); // ifinf, internal: if-inf
         addFunctionMapping("ifnan", "if-nan"); // ifnan, internal: if-nan
         addFunctionMapping("ifnanorinf", "if-nan-or-inf"); // ifnanorinf, internal: if-nan-or-inf
+        addFunctionMapping("missingif", "missing-if"); // missingif, internal: missing-if
+        addFunctionMapping("nanif", "nan-if"); // nanif, internal: nan-if
+        addFunctionMapping("neginfif", "neginf-if"); // neginfif, internal: neginf-if
+        addFunctionMapping("nullif", "null-if"); // nullif, internal: null-if
+        addFunctionMapping("posinfif", "posinf-if"); // posinfif, internal: posinf-if
         addFunctionMapping("toarray", "to-array"); // toarray, internal: to-array
         addFunctionMapping("toatomic", "to-atomic"); // toatomic, internal: to-atomic
         addFunctionMapping("toatom", "to-atomic"); // toatom, internal: to-atomic
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 3571dcf..6752f77 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -25,6 +25,7 @@
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
@@ -43,6 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FunctionUtil {
 
@@ -68,7 +70,8 @@
 
     @FunctionalInterface
     public interface IFunctionNormalizer {
-        FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs) throws CompilationException;
+        FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, SourceLocation sourceLoc)
+                throws CompilationException;
     }
 
     /**
@@ -99,7 +102,7 @@
         if (expression == null) {
             return functionDecls;
         }
-        String value = metadataProvider.getConfig().get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS);
+        String value = (String) metadataProvider.getConfig().get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS);
         boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
         Set<CallExpr> functionCalls = functionCollector.getFunctionCalls(expression);
         for (CallExpr functionCall : functionCalls) {
@@ -116,8 +119,9 @@
                 if (!namespace.equals(FunctionConstants.ASTERIX_NS)
                         && !namespace.equals(AlgebricksBuiltinFunctions.ALGEBRICKS_NS)
                         && metadataProvider.findDataverse(namespace) == null) {
-                    throw new CompilationException("In function call \"" + namespace + "." + signature.getName()
-                            + "(...)\", the dataverse \"" + namespace + "\" cannot be found!");
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
+                            "In function call \"" + namespace + "." + signature.getName() + "(...)\", the dataverse \""
+                                    + namespace + "\" cannot be found!");
                 }
             } catch (AlgebricksException e) {
                 throw new CompilationException(e);
@@ -130,7 +134,8 @@
             }
             if (function == null) {
                 FunctionSignature normalizedSignature = functionNormalizer == null ? signature
-                        : functionNormalizer.normalizeBuiltinFunctionSignature(signature);
+                        : functionNormalizer.normalizeBuiltinFunctionSignature(signature,
+                                functionCall.getSourceLocation());
                 if (BuiltinFunctions.isBuiltinCompilerFunction(normalizedSignature, includePrivateFunctions)) {
                     continue;
                 }
@@ -141,7 +146,8 @@
                 } else {
                     messageBuilder.append("function " + signature + " is not defined");
                 }
-                throw new CompilationException(messageBuilder.toString());
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
+                        messageBuilder.toString());
             }
 
             if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
@@ -149,7 +155,7 @@
                 FunctionDecl functionDecl = functionParser.getFunctionDecl(function);
                 if (functionDecl != null) {
                     if (functionDecls.contains(functionDecl)) {
-                        throw new CompilationException(
+                        throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
                                 "Recursive invocation " + functionDecls.get(functionDecls.size() - 1).getSignature()
                                         + " <==> " + functionDecl.getSignature());
                     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index c4b23ef..c358444 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -23,66 +23,53 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
-import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.expression.FieldBinding;
 import org.apache.asterix.lang.common.expression.ListConstructor;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmBooleanNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmNullNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.ARecord;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
-@SuppressWarnings("unchecked")
 public class LangRecordParseUtil {
     private static final String NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE =
             "JSON record can only have expressions [%1$s, %2$s, %3$s]";
-    private static final ISerializerDeserializer<ADouble> doubleSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
-    private static final ISerializerDeserializer<AString> stringSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-    private static final ISerializerDeserializer<AInt64> intSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-    private static final ISerializerDeserializer<ABoolean> booleanSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    private static final ISerializerDeserializer<ANull> nullSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
     private LangRecordParseUtil() {
     }
 
-    private static void parseExpression(Expression expr, ArrayBackedValueStorage serialized)
-            throws HyracksDataException, CompilationException {
+    private static IAdmNode parseExpression(Expression expr) throws HyracksDataException, CompilationException {
         switch (expr.getKind()) {
             case LITERAL_EXPRESSION:
-                parseLiteral((LiteralExpr) expr, serialized);
-                break;
+                return parseLiteral((LiteralExpr) expr);
             case RECORD_CONSTRUCTOR_EXPRESSION:
-                parseRecord((RecordConstructor) expr, serialized, true, Collections.emptyList());
-                break;
+                return parseRecord((RecordConstructor) expr, Collections.emptyList());
             case LIST_CONSTRUCTOR_EXPRESSION:
-                parseList((ListConstructor) expr, serialized);
-                break;
+                return parseList((ListConstructor) expr);
             default:
                 throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR,
                         NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE,
@@ -92,41 +79,30 @@
         }
     }
 
-    public static void parseRecord(RecordConstructor recordValue, ArrayBackedValueStorage serialized, boolean tagged,
-            List<Pair<String, String>> defaults) throws HyracksDataException, CompilationException {
+    public static AdmObjectNode parseRecord(RecordConstructor recordValue, List<Pair<String, String>> defaults)
+            throws HyracksDataException, CompilationException {
+        AdmObjectNode record = new AdmObjectNode();
         AMutableString fieldNameString = new AMutableString(null);
-        ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-        RecordBuilder recordBuilder = new RecordBuilder();
-        recordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-        recordBuilder.init();
         List<FieldBinding> fbList = recordValue.getFbList();
         HashSet<String> fieldNames = new HashSet<>();
         for (FieldBinding fb : fbList) {
-            fieldName.reset();
-            fieldValue.reset();
             // get key
             fieldNameString.setValue(exprToStringLiteral(fb.getLeftExpr()).getStringValue());
             if (!fieldNames.add(fieldNameString.getStringValue())) {
                 throw new HyracksDataException(
                         "Field " + fieldNameString.getStringValue() + " was specified multiple times");
             }
-            stringSerde.serialize(fieldNameString, fieldName.getDataOutput());
             // get value
-            parseExpression(fb.getRightExpr(), fieldValue);
-            recordBuilder.addField(fieldName, fieldValue);
+            IAdmNode value = parseExpression(fb.getRightExpr());
+            record.set(fieldNameString.getStringValue(), value);
         }
         // defaults
         for (Pair<String, String> kv : defaults) {
             if (!fieldNames.contains(kv.first)) {
-                fieldName.reset();
-                fieldValue.reset();
-                stringSerde.serialize(new AString(kv.first), fieldName.getDataOutput());
-                stringSerde.serialize(new AString(kv.second), fieldValue.getDataOutput());
-                recordBuilder.addField(fieldName, fieldValue);
+                record.set(kv.first, new AdmStringNode(kv.second));
             }
         }
-        recordBuilder.write(serialized.getDataOutput(), tagged);
+        return record;
     }
 
     public static Literal exprToStringLiteral(Expression expr) throws CompilationException {
@@ -143,51 +119,37 @@
         return keyLiteral;
     }
 
-    private static void parseList(ListConstructor valueExpr, ArrayBackedValueStorage serialized)
-            throws CompilationException, HyracksDataException {
+    private static AdmArrayNode parseList(ListConstructor valueExpr) throws CompilationException, HyracksDataException {
         if (valueExpr.getType() != ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR) {
             throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR, "JSON List can't be of type %1$s",
                     valueExpr.getType());
         }
-        ArrayBackedValueStorage serializedValue = new ArrayBackedValueStorage();
-        OrderedListBuilder listBuilder = new OrderedListBuilder();
-        listBuilder.reset(null);
-        for (Expression expr : valueExpr.getExprList()) {
-            serializedValue.reset();
-            parseExpression(expr, serializedValue);
-            listBuilder.addItem(serializedValue);
+        List<Expression> exprs = valueExpr.getExprList();
+        AdmArrayNode list = new AdmArrayNode(exprs.size());
+        for (Expression expr : exprs) {
+            list.add(parseExpression(expr));
         }
-        listBuilder.write(serialized.getDataOutput(), true);
+        return list;
     }
 
-    private static void parseLiteral(LiteralExpr objectExpr, ArrayBackedValueStorage serialized)
-            throws HyracksDataException {
+    private static IAdmNode parseLiteral(LiteralExpr objectExpr) throws HyracksDataException {
         Literal value = objectExpr.getValue();
         switch (value.getLiteralType()) {
             case DOUBLE:
-                doubleSerde.serialize(new ADouble((Double) value.getValue()), serialized.getDataOutput());
-                break;
-            case TRUE:
-                booleanSerde.serialize(ABoolean.TRUE, serialized.getDataOutput());
-                break;
-            case FALSE:
-                booleanSerde.serialize(ABoolean.FALSE, serialized.getDataOutput());
-                break;
             case FLOAT:
-                doubleSerde.serialize(new ADouble((Float) value.getValue()), serialized.getDataOutput());
-                break;
+                return new AdmDoubleNode((Double) value.getValue());
+            case TRUE:
+                return AdmBooleanNode.TRUE;
+            case FALSE:
+                return AdmBooleanNode.FALSE;
             case INTEGER:
-                intSerde.serialize(new AInt64(((Integer) value.getValue()).longValue()), serialized.getDataOutput());
-                break;
+                return new AdmBigIntNode((Integer) value.getValue());
             case LONG:
-                intSerde.serialize(new AInt64((Long) value.getValue()), serialized.getDataOutput());
-                break;
+                return new AdmBigIntNode((Long) value.getValue());
             case NULL:
-                nullSerde.serialize(ANull.NULL, serialized.getDataOutput());
-                break;
+                return AdmNullNode.INSTANCE;
             case STRING:
-                stringSerde.serialize(new AString((String) value.getValue()), serialized.getDataOutput());
-                break;
+                return new AdmStringNode((String) value.getValue());
             default:
                 throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR, "Unknown Literal Type %1$s",
                         value.getLiteralType());
@@ -203,6 +165,21 @@
         }
     }
 
+    public static void recordToMap(Map<String, String> map, AdmObjectNode record) throws AlgebricksException {
+        for (Entry<String, IAdmNode> field : record.getFields()) {
+            String value = aObjToString(field.getValue());
+            map.put(field.getKey(), value);
+        }
+    }
+
+    private static String aObjToString(IAdmNode aObj) {
+        if (aObj.getType() == ATypeTag.STRING) {
+            return ((AdmStringNode) aObj).get();
+        } else {
+            return aObj.toString();
+        }
+    }
+
     public static String aObjToString(IAObject aObj) throws AlgebricksException {
         switch (aObj.getType().getTypeTag()) {
             case DOUBLE:
@@ -213,6 +190,8 @@
                 return aOrderedListToString((AOrderedList) aObj);
             case STRING:
                 return ((AString) aObj).getStringValue();
+            case BOOLEAN:
+                return ((ABoolean) aObj).getBoolean().toString();
             default:
                 throw new AlgebricksException("value of type " + aObj.getType() + " is not supported yet");
         }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index 8ae67d6..bf9cf89 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -60,6 +60,7 @@
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionVisitor<Boolean, List<FunctionDecl>> {
 
@@ -289,12 +290,18 @@
                 if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
                     subts.addSubstituion(new VariableExpr(param), e);
                 } else {
+                    SourceLocation sourceLoc = e.getSourceLocation();
                     VarIdentifier newV = context.newVariable();
                     Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
                             e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
-                    LetClause c = new LetClause(new VariableExpr(newV), (Expression) p1.first);
+                    VariableExpr newVRef1 = new VariableExpr(newV);
+                    newVRef1.setSourceLocation(sourceLoc);
+                    LetClause c = new LetClause(newVRef1, (Expression) p1.first);
+                    c.setSourceLocation(sourceLoc);
                     clauses.add(c);
-                    subts.addSubstituion(new VariableExpr(param), new VariableExpr(newV));
+                    VariableExpr newVRef2 = new VariableExpr(newV);
+                    newVRef2.setSourceLocation(sourceLoc);
+                    subts.addSubstituion(new VariableExpr(param), newVRef2);
                 }
             }
 
@@ -325,10 +332,11 @@
     }
 
     private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
+        SourceLocation sourceLoc = fnDecl.getSourceLocation();
         Query wrappedQuery = new Query(false);
+        wrappedQuery.setSourceLocation(sourceLoc);
         wrappedQuery.setBody(fnDecl.getFuncBody());
         wrappedQuery.setTopLevel(false);
-        wrappedQuery.setExternalVars(fnDecl.getParamList());
 
         String fnNamespace = fnDecl.getSignature().getNamespace();
         Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
@@ -340,14 +348,15 @@
             try {
                 fnDataverse = metadataProvider.findDataverse(fnNamespace);
             } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.NO_DATAVERSE_WITH_NAME, e, fnNamespace);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnNamespace);
             }
         }
 
         metadataProvider.setDefaultDataverse(fnDataverse);
         try {
             IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
-            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context, true);
+            queryRewriter.rewrite(declaredFunctions, wrappedQuery, metadataProvider, context, true,
+                    fnDecl.getParamList());
             return wrappedQuery.getBody();
         } finally {
             metadataProvider.setDefaultDataverse(defaultDataverse);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index 58c81a6..e415a63 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -25,6 +25,7 @@
 import java.util.Map.Entry;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -73,6 +74,7 @@
         VariableExpr varExpr = lc.getVarExpr();
         VariableExpr newVe = generateNewVariable(context, varExpr);
         LetClause newLet = new LetClause(newVe, (Expression) p1.first);
+        newLet.setSourceLocation(lc.getSourceLocation());
         return new Pair<>(newLet, VariableCloneAndSubstitutionUtil.eliminateSubstFromList(lc.getVarExpr(), env));
     }
 
@@ -107,6 +109,7 @@
         }
         GroupbyClause newGroup = new GroupbyClause(newGbyList, newDecorList, newWithMap, newGroupVar, newGroupFieldList,
                 gc.hasHashGroupByHint(), gc.isGroupAll());
+        newGroup.setSourceLocation(gc.getSourceLocation());
         return new Pair<>(newGroup, newSubs);
     }
 
@@ -126,6 +129,8 @@
         }
         Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = qe.getSatisfiesExpr().accept(this, newSubs);
         QuantifiedExpression qe2 = new QuantifiedExpression(qe.getQuantifier(), newPairs, (Expression) p2.first);
+        qe2.setSourceLocation(qe.getSourceLocation());
+        qe2.addHints(qe.getHints());
         return new Pair<>(qe2, newSubs);
     }
 
@@ -134,6 +139,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = wc.getWhereExpr().accept(this, env);
         WhereClause newW = new WhereClause((Expression) p1.first);
+        newW.setSourceLocation(wc.getSourceLocation());
         return new Pair<>(newW, p1.second);
     }
 
@@ -142,6 +148,8 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         List<Expression> exprList = VariableCloneAndSubstitutionUtil.visitAndCloneExprList(pf.getExprList(), env, this);
         CallExpr f = new CallExpr(pf.getFunctionSignature(), exprList);
+        f.setSourceLocation(pf.getSourceLocation());
+        f.addHints(pf.getHints());
         return new Pair<>(f, env);
     }
 
@@ -152,11 +160,12 @@
         for (VarIdentifier vi : fd.getParamList()) {
             VariableExpr varExpr = new VariableExpr(vi);
             if (!env.constainsOldVar(varExpr)) {
-                throw new CompilationException("Parameter " + vi + " does not appear in the substitution list.");
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fd.getSourceLocation(),
+                        "Parameter " + vi + " does not appear in the substitution list.");
             }
             Expression newExpr = env.findSubstitution(varExpr);
             if (newExpr.getKind() != Kind.VARIABLE_EXPRESSION) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fd.getSourceLocation(),
                         "Parameter " + vi + " cannot be substituted by a non-variable expression.");
             }
             newList.add(((VariableExpr) newExpr).getVar());
@@ -164,6 +173,7 @@
 
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = fd.getFuncBody().accept(this, env);
         FunctionDecl newF = new FunctionDecl(fd.getSignature(), newList, (Expression) p1.first);
+        newF.setSourceLocation(fd.getSourceLocation());
         return new Pair<>(newF, env);
     }
 
@@ -174,6 +184,8 @@
         Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = ifexpr.getThenExpr().accept(this, env);
         Pair<ILangExpression, VariableSubstitutionEnvironment> p3 = ifexpr.getElseExpr().accept(this, env);
         IfExpr i = new IfExpr((Expression) p1.first, (Expression) p2.first, (Expression) p3.first);
+        i.setSourceLocation(ifexpr.getSourceLocation());
+        i.addHints(ifexpr.getHints());
         return new Pair<>(i, env);
     }
 
@@ -189,6 +201,7 @@
             p2 = new Pair<>(null, null);
         }
         LimitClause c = new LimitClause((Expression) p1.first, (Expression) p2.first);
+        c.setSourceLocation(lc.getSourceLocation());
         return new Pair<>(c, env);
     }
 
@@ -198,6 +211,8 @@
         List<Expression> oldExprList = lc.getExprList();
         List<Expression> exprs = VariableCloneAndSubstitutionUtil.visitAndCloneExprList(oldExprList, env, this);
         ListConstructor c = new ListConstructor(lc.getType(), exprs);
+        c.setSourceLocation(lc.getSourceLocation());
+        c.addHints(lc.getHints());
         return new Pair<>(c, env);
     }
 
@@ -217,6 +232,8 @@
             exprs.add((Expression) p1.first);
         }
         OperatorExpr oe = new OperatorExpr(exprs, op.getExprBroadcastIdx(), op.getOpList(), op.isCurrentop());
+        oe.setSourceLocation(op.getSourceLocation());
+        oe.addHints(op.getHints());
         return new Pair<>(oe, env);
     }
 
@@ -229,6 +246,7 @@
         oc2.setNumFrames(oc.getNumFrames());
         oc2.setNumTuples(oc.getNumTuples());
         oc2.setRangeMap(oc.getRangeMap());
+        oc2.setSourceLocation(oc.getSourceLocation());
         return new Pair<>(oc2, env);
     }
 
@@ -238,6 +256,7 @@
         Query newQ = new Query(q.isExplain());
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = q.getBody().accept(this, env);
         newQ.setBody((Expression) p1.first);
+        newQ.setSourceLocation(q.getSourceLocation());
         return new Pair<>(newQ, p1.second);
     }
 
@@ -253,6 +272,8 @@
             newFbs.add(fb2);
         }
         RecordConstructor newRc = new RecordConstructor(newFbs);
+        newRc.setSourceLocation(rc.getSourceLocation());
+        newRc.addHints(rc.getHints());
         return new Pair<>(newRc, env);
     }
 
@@ -261,6 +282,8 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p1 = u.getExpr().accept(this, env);
         UnaryExpr newU = new UnaryExpr(u.getExprType(), (Expression) p1.first);
+        newU.setSourceLocation(u.getSourceLocation());
+        newU.addHints(u.getHints());
         return new Pair<>(newU, env);
     }
 
@@ -275,6 +298,8 @@
         }
         IndexAccessor i = new IndexAccessor((Expression) p1.first, indexExpr);
         i.setAny(ia.isAny());
+        i.setSourceLocation(ia.getSourceLocation());
+        i.addHints(ia.getHints());
         return new Pair<>(i, env);
     }
 
@@ -283,6 +308,8 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p = fa.getExpr().accept(this, env);
         FieldAccessor newF = new FieldAccessor((Expression) p.first, fa.getIdent());
+        newF.setSourceLocation(fa.getSourceLocation());
+        newF.addHints(fa.getHints());
         return new Pair<>(newF, p.second);
     }
 
@@ -301,7 +328,10 @@
             // it is a variable from the context
             VarIdentifier var = context.getRewrittenVar(expr.getVar().getId());
             if (var != null) {
-                return new VariableExpr(var);
+                VariableExpr newVarExpr = new VariableExpr(var);
+                newVarExpr.setSourceLocation(expr.getSourceLocation());
+                newVarExpr.addHints(expr.getHints());
+                return newVarExpr;
             }
         }
         return expr;
@@ -319,7 +349,9 @@
     public VariableExpr generateNewVariable(LangRewritingContext context, VariableExpr varExpr) {
         VarIdentifier vi = varExpr.getVar();
         VarIdentifier newVar = context.mapOldId(vi.getId(), vi.getValue());
-        return new VariableExpr(newVar);
+        VariableExpr newVarExpr = new VariableExpr(newVar);
+        newVarExpr.setSourceLocation(varExpr.getSourceLocation());
+        return newVarExpr;
     }
 
     /**
diff --git a/asterixdb/asterix-lang-sqlpp/pom.xml b/asterixdb/asterix-lang-sqlpp/pom.xml
index 0321437..e122335 100644
--- a/asterixdb/asterix-lang-sqlpp/pom.xml
+++ b/asterixdb/asterix-lang-sqlpp/pom.xml
@@ -162,6 +162,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-core</artifactId>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
index 40a5ea5..ba1c498 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/AbstractBinaryCorrelateClause.java
@@ -21,12 +21,12 @@
 
 import java.util.Objects;
 
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
 
-public abstract class AbstractBinaryCorrelateClause implements Clause {
+public abstract class AbstractBinaryCorrelateClause extends AbstractClause {
 
     private JoinType joinType;
     private Expression rightExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
index 65c2f01..4b0e324 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromClause.java
@@ -19,18 +19,17 @@
 
 package org.apache.asterix.lang.sqlpp.clause;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class FromClause implements Clause {
+public class FromClause extends AbstractClause {
 
-    private List<FromTerm> fromTerms = new ArrayList<>();
+    private List<FromTerm> fromTerms;
 
     public FromClause(List<FromTerm> fromTerms) {
         this.fromTerms = fromTerms;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
index 6860cfe..3e26371 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/FromTerm.java
@@ -24,13 +24,13 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class FromTerm implements Clause {
+public class FromTerm extends AbstractClause {
     private Expression leftExpr;
     private VariableExpr leftVar;
     private VariableExpr posVar;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
index aadbe42..2c0ce0c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/HavingClause.java
@@ -19,12 +19,12 @@
 package org.apache.asterix.lang.sqlpp.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class HavingClause implements Clause {
+public class HavingClause extends AbstractClause {
 
     private Expression filterExpression;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
index 066ad22..7f353d6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
@@ -22,12 +22,12 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class Projection implements Clause {
+public class Projection extends AbstractClause {
 
     private Expression expr;
     private String name;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
index 0477b1c..83b03ba 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectBlock.java
@@ -24,14 +24,14 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.WhereClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectBlock implements Clause {
+public class SelectBlock extends AbstractClause {
 
     private SelectClause selectClause;
     private FromClause fromClause;
@@ -96,7 +96,7 @@
     }
 
     public boolean hasLetClauses() {
-        return letClauses != null && !letClauses.isEmpty();
+        return !letClauses.isEmpty();
     }
 
     public boolean hasWhereClause() {
@@ -108,7 +108,7 @@
     }
 
     public boolean hasLetClausesAfterGroupby() {
-        return letClausesAfterGby != null && !letClausesAfterGby.isEmpty();
+        return !letClausesAfterGby.isEmpty();
     }
 
     public List<LetClause> getLetListAfterGroupby() {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
index e5c3f56..2627870 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
@@ -22,11 +22,11 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectClause implements Clause {
+public class SelectClause extends AbstractClause {
 
     private SelectElement selectElement;
     private SelectRegular selectRegular;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
index 9c82d5c..464b744 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectElement.java
@@ -20,12 +20,12 @@
 package org.apache.asterix.lang.sqlpp.clause;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectElement implements Clause {
+public class SelectElement extends AbstractClause {
 
     private Expression expr;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
index a114337..9a191723 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectRegular.java
@@ -23,11 +23,11 @@
 import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectRegular implements Clause {
+public class SelectRegular extends AbstractClause {
 
     private List<Projection> projections;
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
index 18d9879..7994358 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectSetOperation.java
@@ -24,13 +24,13 @@
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Clause;
+import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 
-public class SelectSetOperation implements Clause {
+public class SelectSetOperation extends AbstractClause {
 
     private SetOperationInput leftInput;
     private List<SetOperationRight> rightInputs = new ArrayList<>();
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java
new file mode 100644
index 0000000..5cd4f8f
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppParseException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.asterix.lang.sqlpp.parser;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public final class SqlppParseException extends ParseException {
+    private static final long serialVersionUID = -7395094812307808714L;
+
+    private final SourceLocation sourceLoc;
+
+    public SqlppParseException(SourceLocation sourceLoc, String message) {
+        super(message);
+        this.sourceLoc = sourceLoc;
+    }
+
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 7858e58..fa4018a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -18,25 +18,24 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites;
 
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
 
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
-            throws CompilationException {
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context);
-
-        // Inlines column aliases.
-        inlineColumnAlias();
+        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
 
         // Generates column names.
         generateColumnNames();
@@ -50,6 +49,9 @@
         // Rewrites set operations.
         rewriteSetOperations();
 
+        // Inlines column aliases.
+        inlineColumnAlias();
+
         // Generate ids for variables (considering scopes) and replace global variable access with the dataset function.
         variableCheckAndRewrite();
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index befa5ab..ec8b011 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.rewrites;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
@@ -31,6 +32,7 @@
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
@@ -68,36 +70,35 @@
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
-class SqlppQueryRewriter implements IQueryRewriter {
-    private static final String INLINE_WITH = "inline_with";
-    private static final String NOT_INLINE_WITH = "false";
+public class SqlppQueryRewriter implements IQueryRewriter {
+    public static final String INLINE_WITH_OPTION = "inline_with";
+    private static final boolean INLINE_WITH_OPTION_DEFAULT = true;
     private final FunctionParser functionRepository = new FunctionParser(new SqlppParserFactory());
     private IReturningStatement topExpr;
     private List<FunctionDecl> declaredFunctions;
     private LangRewritingContext context;
     private MetadataProvider metadataProvider;
+    private Collection<VarIdentifier> externalVars;
 
     protected void setup(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
-            MetadataProvider metadataProvider, LangRewritingContext context) {
+            MetadataProvider metadataProvider, LangRewritingContext context, Collection<VarIdentifier> externalVars) {
         this.topExpr = topExpr;
         this.context = context;
         this.declaredFunctions = declaredFunctions;
         this.metadataProvider = metadataProvider;
+        this.externalVars = externalVars;
     }
 
     @Override
     public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
-            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs)
-            throws CompilationException {
+            MetadataProvider metadataProvider, LangRewritingContext context, boolean inlineUdfs,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         if (topStatement == null) {
             return;
         }
 
         // Sets up parameters.
-        setup(declaredFunctions, topStatement, metadataProvider, context);
-
-        // Inlines column aliases.
-        inlineColumnAlias();
+        setup(declaredFunctions, topStatement, metadataProvider, context, externalVars);
 
         // Generates column names.
         generateColumnNames();
@@ -111,6 +112,9 @@
         // Rewrites set operations.
         rewriteSetOperations();
 
+        // Inlines column aliases.
+        inlineColumnAlias();
+
         // Generate ids for variables (considering scopes) and replace global variable access with the dataset function.
         variableCheckAndRewrite();
 
@@ -139,7 +143,7 @@
         rewriteDistinctAggregations();
 
         // Sets the var counter of the query.
-        topStatement.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter().get());
     }
 
     protected void rewriteGroupByAggregationSugar() throws CompilationException {
@@ -164,8 +168,7 @@
     }
 
     protected void inlineWithExpressions() throws CompilationException {
-        String inlineWith = metadataProvider.getConfig().get(INLINE_WITH);
-        if (inlineWith != null && inlineWith.equalsIgnoreCase(NOT_INLINE_WITH)) {
+        if (!metadataProvider.getBooleanProperty(INLINE_WITH_OPTION, INLINE_WITH_OPTION_DEFAULT)) {
             return;
         }
         // Inlines with expressions.
@@ -206,7 +209,7 @@
 
     protected void variableCheckAndRewrite() throws CompilationException {
         VariableCheckAndRewriteVisitor variableCheckAndRewriteVisitor =
-                new VariableCheckAndRewriteVisitor(context, metadataProvider, topExpr.getExternalVars());
+                new VariableCheckAndRewriteVisitor(context, metadataProvider, externalVars);
         topExpr.accept(variableCheckAndRewriteVisitor, null);
     }
 
@@ -225,7 +228,8 @@
         for (Expression topLevelExpr : topExpr.getDirectlyEnclosedExpressions()) {
             usedStoredFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr,
                     funIds, null, expr -> getFunctionCalls(expr), func -> functionRepository.getFunctionDecl(func),
-                    signature -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false)));
+                    (signature, sourceLoc) -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false,
+                            sourceLoc)));
         }
         declaredFunctions.addAll(usedStoredFunctionDecls);
         if (inlineUdfs && !declaredFunctions.isEmpty()) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
index 5667415..7908636 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppDeleteRewriteVisitor;
 
 class SqlppStatementRewriter implements IStatementRewriter {
@@ -36,4 +37,9 @@
             stmt.accept(visitor, null);
         }
     }
+
+    @Override
+    public String toExternalVariableName(String statementParameterName) {
+        return SqlppVariableUtil.toExternalVariableName(statementParameterName);
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
index b945a40..14a7ec0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
@@ -57,7 +57,9 @@
     public Expression visit(GroupbyClause groupbyClause, ILangExpression arg) throws CompilationException {
         for (GbyVariableExpressionPair gbyKeyPair : groupbyClause.getGbyPairList()) {
             if (gbyKeyPair.getVar() == null) {
-                gbyKeyPair.setVar(new VariableExpr(context.newVariable()));
+                VariableExpr varExpr = new VariableExpr(context.newVariable());
+                varExpr.setSourceLocation(gbyKeyPair.getExpr().getSourceLocation());
+                gbyKeyPair.setVar(varExpr);
             }
         }
         return super.visit(groupbyClause, arg);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
index fa049fe..a270530 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
@@ -18,16 +18,22 @@
  */
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.FieldBinding;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
@@ -43,11 +49,19 @@
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppSubstituteExpressionVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
- * Syntactic sugar rewriting: inlines column aliases defines in SELECT clause into ORDER BY and LIMIT clauses. <br/>
- * Note: column aliases are not cosidered new variables, but they can be referenced from ORDER BY and LIMIT clauses
- *       because of this rewriting (like in SQL)
+ * Syntactic sugar rewriting: inlines column aliases definitions in SELECT clause into ORDER BY and LIMIT clauses.
+ * <br/>
+ * Notes
+ * <ul>
+ * <li> column aliases are not considered new variables, but they can be referenced from ORDER BY and LIMIT clauses
+ *      because of this rewriting (like in SQL) </li>
+ * <li> if a column alias expression is not a variable or a literal then we introduce a new let clause and replace
+ *      that column expression with the let variable reference. The optimizer will then decide whether to inline that
+ *      expression or not </li>
+ * </ul>
  */
 public class InlineColumnAliasVisitor extends AbstractSqlppExpressionScopingVisitor {
 
@@ -58,44 +72,52 @@
     @Override
     public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
         // Gets the map from select clause.
-        Map<Expression, Expression> map = getMap(selectBlock.getSelectClause());
+        Map<Expression, ColumnAliasBinding> map = getMap(selectBlock.getSelectClause());
 
         // Removes all FROM/LET binding variables
-        if (selectBlock.hasFromClause()) {
-            map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
+        if (selectBlock.hasGroupbyClause()) {
+            map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getGroupbyClause()));
+            if (selectBlock.hasLetClausesAfterGroupby()) {
+                map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getLetListAfterGroupby()));
+            }
+        } else {
+            if (selectBlock.hasFromClause()) {
+                map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
+            }
+            if (selectBlock.hasLetClauses()) {
+                map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getLetList()));
+            }
         }
-        if (selectBlock.hasLetClauses()) {
-            map.keySet().removeAll(SqlppVariableUtil.getBindingVariables(selectBlock.getLetList()));
-        }
-
-        // Creates a substitution visitor.
-        SqlppSubstituteExpressionVisitor visitor = new SqlppSubstituteExpressionVisitor(context, map);
 
         SelectExpression selectExpression = (SelectExpression) arg;
-
         // For SET operation queries, column aliases will not substitute ORDER BY nor LIMIT expressions.
         if (!selectExpression.getSelectSetOperation().hasRightInputs()) {
+            // Creates a substitution visitor.
+            SubstituteColumnAliasVisitor visitor = new SubstituteColumnAliasVisitor(context, toExpressionMap(map));
             if (selectExpression.hasOrderby()) {
                 selectExpression.getOrderbyClause().accept(visitor, arg);
             }
             if (selectExpression.hasLimit()) {
                 selectExpression.getLimitClause().accept(visitor, arg);
             }
+            if (!visitor.letVarMap.isEmpty()) {
+                introduceLetClauses(visitor.letVarMap, map, selectBlock);
+            }
         }
         return super.visit(selectBlock, arg);
     }
 
-    private Map<Expression, Expression> getMap(SelectClause selectClause) throws CompilationException {
+    private Map<Expression, ColumnAliasBinding> getMap(SelectClause selectClause) {
         if (selectClause.selectElement()) {
             return getMap(selectClause.getSelectElement());
         }
         if (selectClause.selectRegular()) {
             return getMap(selectClause.getSelectRegular());
         }
-        return null;
+        return Collections.emptyMap();
     }
 
-    private Map<Expression, Expression> getMap(SelectElement selectElement) {
+    private Map<Expression, ColumnAliasBinding> getMap(SelectElement selectElement) {
         Expression expr = selectElement.getExpression();
         if (expr.getKind() == Kind.RECORD_CONSTRUCTOR_EXPRESSION) {
             // Rewrite top-level field names (aliases), in order to be consistent with SelectRegular.
@@ -104,12 +126,12 @@
         return Collections.emptyMap();
     }
 
-    private Map<Expression, Expression> getMap(SelectRegular selectRegular) {
+    private Map<Expression, ColumnAliasBinding> getMap(SelectRegular selectRegular) {
         return mapProjections(selectRegular.getProjections());
     }
 
-    private Map<Expression, Expression> mapRecordConstructor(RecordConstructor rc) {
-        Map<Expression, Expression> exprMap = new HashMap<>();
+    private Map<Expression, ColumnAliasBinding> mapRecordConstructor(RecordConstructor rc) {
+        Map<Expression, ColumnAliasBinding> exprMap = new HashMap<>();
         for (FieldBinding binding : rc.getFbList()) {
             Expression leftExpr = binding.getLeftExpr();
             // We only need to deal with the case that the left expression (for a field name) is
@@ -121,22 +143,142 @@
             LiteralExpr literalExpr = (LiteralExpr) leftExpr;
             if (literalExpr.getValue().getLiteralType() == Literal.Type.STRING) {
                 String fieldName = SqlppVariableUtil.toInternalVariableName(literalExpr.getValue().getStringValue());
-                exprMap.put(new VariableExpr(new VarIdentifier(fieldName)), binding.getRightExpr());
+                exprMap.put(new VariableExpr(new VarIdentifier(fieldName)), ColumnAliasBinding.of(binding));
             }
         }
         return exprMap;
     }
 
-    private Map<Expression, Expression> mapProjections(List<Projection> projections) {
-        Map<Expression, Expression> exprMap = new HashMap<>();
+    private Map<Expression, ColumnAliasBinding> mapProjections(List<Projection> projections) {
+        Map<Expression, ColumnAliasBinding> exprMap = new HashMap<>();
         for (Projection projection : projections) {
             if (!projection.star() && !projection.varStar()) {
-                exprMap.put(
-                        new VariableExpr(
-                                new VarIdentifier(SqlppVariableUtil.toInternalVariableName(projection.getName()))),
-                        projection.getExpression());
+                String varName = SqlppVariableUtil.toInternalVariableName(projection.getName());
+                exprMap.put(new VariableExpr(new VarIdentifier(varName)), ColumnAliasBinding.of(projection));
             }
         }
         return exprMap;
     }
+
+    private void introduceLetClauses(Map<Expression, VarIdentifier> letVarMap,
+            Map<Expression, ColumnAliasBinding> aliasBindingMap, SelectBlock selectBlock) throws CompilationException {
+
+        List<LetClause> targetLetClauses =
+                selectBlock.hasGroupbyClause() ? selectBlock.getLetListAfterGroupby() : selectBlock.getLetList();
+
+        for (Map.Entry<Expression, VarIdentifier> me : letVarMap.entrySet()) {
+            Expression columnAliasVarExpr = me.getKey();
+            SourceLocation sourceLoc = columnAliasVarExpr.getSourceLocation();
+            ColumnAliasBinding columnAliasBinding = aliasBindingMap.get(columnAliasVarExpr);
+            if (columnAliasBinding == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc);
+            }
+            VarIdentifier letVarId = me.getValue();
+
+            // add a let clause defining the new variable
+            VariableExpr letVarDefExpr = new VariableExpr(letVarId);
+            letVarDefExpr.setSourceLocation(sourceLoc);
+            LetClause newLetClause = new LetClause(letVarDefExpr, columnAliasBinding.getExpression());
+            newLetClause.setSourceLocation(sourceLoc);
+            targetLetClauses.add(newLetClause);
+
+            // replace original column alias expression with variable reference
+            VariableExpr letVarRefExpr = new VariableExpr(letVarId);
+            letVarRefExpr.setSourceLocation(sourceLoc);
+            columnAliasBinding.setExpression(letVarRefExpr);
+
+            context.addExcludedForFieldAccessVar(letVarId);
+        }
+    }
+
+    private static Map<Expression, Expression> toExpressionMap(Map<Expression, ColumnAliasBinding> bindingMap) {
+        Map<Expression, Expression> exprMap = new HashMap<>();
+        for (Map.Entry<Expression, ColumnAliasBinding> me : bindingMap.entrySet()) {
+            exprMap.put(me.getKey(), me.getValue().getExpression());
+        }
+        return exprMap;
+    }
+
+    private abstract static class ColumnAliasBinding {
+
+        abstract Expression getExpression();
+
+        abstract void setExpression(Expression expr);
+
+        static ColumnAliasBinding of(FieldBinding fieldBinding) {
+            return new ColumnAliasBinding() {
+                @Override
+                Expression getExpression() {
+                    return fieldBinding.getRightExpr();
+                }
+
+                @Override
+                void setExpression(Expression expr) {
+                    fieldBinding.setRightExpr(expr);
+                }
+            };
+        }
+
+        static ColumnAliasBinding of(Projection projection) {
+            return new ColumnAliasBinding() {
+                @Override
+                Expression getExpression() {
+                    return projection.getExpression();
+                }
+
+                @Override
+                void setExpression(Expression expr) {
+                    projection.setExpression(expr);
+                }
+            };
+        }
+    }
+
+    /**
+     * Dataset access functions have not yet been introduced at this point, so we need to perform substitution
+     * on postVisit() to avoid infinite recursion in case of SELECT (SELECT ... FROM dataset_name) AS dataset_name.
+     */
+    private static class SubstituteColumnAliasVisitor extends SqlppSubstituteExpressionVisitor {
+
+        private final Map<Expression, VarIdentifier> letVarMap = new LinkedHashMap<>();
+
+        private SubstituteColumnAliasVisitor(LangRewritingContext context, Map<Expression, Expression> exprMap) {
+            super(context, exprMap);
+        }
+
+        @Override
+        protected Expression preVisit(Expression expr) {
+            return expr;
+        }
+
+        @Override
+        protected Expression postVisit(Expression expr) throws CompilationException {
+            return substitute(expr);
+        }
+
+        @Override
+        protected Expression getMappedExpr(Expression expr) throws CompilationException {
+            Expression mappedExpr = super.getMappedExpr(expr);
+            if (mappedExpr == null) {
+                return null;
+            }
+            switch (mappedExpr.getKind()) {
+                case LITERAL_EXPRESSION:
+                case VARIABLE_EXPRESSION:
+                    return mappedExpr;
+                default:
+                    // all other kinds of expressions must be moved out of column alias definitions into separate
+                    // let clauses, so we need to return a variable reference expression here and
+                    // create a new let variable if we're replacing given expression for the first time
+                    VarIdentifier var = letVarMap.get(expr);
+                    if (var == null) {
+                        var = context.newVariable();
+                        letVarMap.put(expr, var);
+                    }
+                    VariableExpr varExpr = new VariableExpr(var);
+                    varExpr.setSourceLocation(expr.getSourceLocation());
+                    return varExpr;
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 7e228f4..23b4d60 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -37,9 +37,11 @@
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
+import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVisitor {
 
@@ -76,14 +78,17 @@
     }
 
     private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) {
-        Expression likeExpr =
+        CallExpr likeExpr =
                 new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
+        likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
         switch (opType) {
             case LIKE:
                 return likeExpr;
             case NOT_LIKE:
-                return new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+                CallExpr notLikeExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
                         new ArrayList<>(Collections.singletonList(likeExpr)));
+                notLikeExpr.setSourceLocation(operatorExpr.getSourceLocation());
+                return notLikeExpr;
             default:
                 throw new IllegalArgumentException(String.valueOf(opType));
         }
@@ -91,29 +96,37 @@
 
     private Expression processInOperator(OperatorExpr operatorExpr, OperatorType opType) throws CompilationException {
         VariableExpr bindingVar = new VariableExpr(context.newVariable());
+        bindingVar.setSourceLocation(operatorExpr.getSourceLocation());
         Expression itemExpr = operatorExpr.getExprList().get(0);
         Expression collectionExpr = operatorExpr.getExprList().get(1);
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(itemExpr);
         comparison.addOperand(bindingVar);
         comparison.setCurrentop(true);
+        comparison.setSourceLocation(operatorExpr.getSourceLocation());
         if (opType == OperatorType.IN) {
             comparison.addOperator(OperatorType.EQ);
-            return new QuantifiedExpression(Quantifier.SOME,
+            QuantifiedExpression quantExpr = new QuantifiedExpression(Quantifier.SOME,
                     new ArrayList<>(Collections.singletonList(new QuantifiedPair(bindingVar, collectionExpr))),
                     comparison);
+            quantExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return quantExpr;
         } else {
             comparison.addOperator(OperatorType.NEQ);
-            return new QuantifiedExpression(Quantifier.EVERY,
+            QuantifiedExpression quantExpr = new QuantifiedExpression(Quantifier.EVERY,
                     new ArrayList<>(Collections.singletonList(new QuantifiedPair(bindingVar, collectionExpr))),
                     comparison);
+            quantExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return quantExpr;
         }
     }
 
     private Expression processConcatOperator(OperatorExpr operatorExpr) {
         // All operators have to be "||"s (according to the grammar).
-        return new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, FunctionMapUtil.CONCAT, 1),
+        CallExpr callExpr = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, FunctionMapUtil.CONCAT, 1),
                 operatorExpr.getExprList());
+        callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+        return callExpr;
     }
 
     private Expression processBetweenOperator(OperatorExpr operatorExpr, OperatorType opType)
@@ -123,25 +136,35 @@
         Expression left = operatorExpr.getExprList().get(1);
         Expression right = operatorExpr.getExprList().get(2);
 
-        // Creates the expression left <= target.
-        Expression leftComparison = createLessThanExpression(left, target, operatorExpr.getHints());
+        // Creates the expression target >= left.
+        Expression leftComparison = createOperatorExpression(OperatorType.GE, target, left, operatorExpr.getHints(),
+                operatorExpr.getSourceLocation());
         // Creates the expression target <= right.
-        Expression rightComparison = createLessThanExpression(target, right, operatorExpr.getHints());
+        Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
+        Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
+                operatorExpr.getHints(), operatorExpr.getSourceLocation());
         OperatorExpr andExpr = new OperatorExpr();
         andExpr.addOperand(leftComparison);
         andExpr.addOperand(rightComparison);
         andExpr.addOperator(OperatorType.AND);
-        return opType == OperatorType.BETWEEN ? andExpr
-                : new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
-                        new ArrayList<>(Collections.singletonList(andExpr)));
+        andExpr.setSourceLocation(operatorExpr.getSourceLocation());
+        if (opType == OperatorType.BETWEEN) {
+            return andExpr;
+        } else {
+            CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+                    new ArrayList<>(Collections.singletonList(andExpr)));
+            callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+            return callExpr;
+        }
     }
 
-    private Expression createLessThanExpression(Expression lhs, Expression rhs, List<IExpressionAnnotation> hints)
-            throws CompilationException {
+    private Expression createOperatorExpression(OperatorType opType, Expression lhs, Expression rhs,
+            List<IExpressionAnnotation> hints, SourceLocation sourceLoc) {
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(lhs);
         comparison.addOperand(rhs);
-        comparison.addOperator(OperatorType.LE);
+        comparison.addOperator(opType);
+        comparison.setSourceLocation(sourceLoc);
         if (hints != null) {
             for (IExpressionAnnotation hint : hints) {
                 comparison.addHint(hint);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
index ddf3cd6..f0a0f87 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This visitor rewrites set operation queries with order by and limit into
@@ -81,19 +82,28 @@
         OrderbyClause orderBy = selectExpression.getOrderbyClause();
         LimitClause limit = selectExpression.getLimitClause();
 
+        SourceLocation sourceLoc = selectExpression.getSourceLocation();
+
         // Wraps the set operation part with a subquery.
         SelectExpression nestedSelectExpression = new SelectExpression(null, selectSetOperation, null, null, true);
+        nestedSelectExpression.setSourceLocation(sourceLoc);
         VariableExpr newBindingVar = new VariableExpr(context.newVariable()); // Binding variable for the subquery.
+        newBindingVar.setSourceLocation(sourceLoc);
         FromTerm newFromTerm = new FromTerm(nestedSelectExpression, newBindingVar, null, null);
+        newFromTerm.setSourceLocation(sourceLoc);
         FromClause newFromClause = new FromClause(new ArrayList<>(Collections.singletonList(newFromTerm)));
+        newFromClause.setSourceLocation(sourceLoc);
         SelectClause selectClause = new SelectClause(new SelectElement(newBindingVar), null, false);
+        selectClause.setSourceLocation(sourceLoc);
         SelectBlock selectBlock = new SelectBlock(selectClause, newFromClause, null, null, null, null, null);
+        selectBlock.setSourceLocation(sourceLoc);
         SelectSetOperation newSelectSetOperation =
                 new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-
+        newSelectSetOperation.setSourceLocation(sourceLoc);
         // Puts together the generated select-from-where query and order by/limit.
         SelectExpression newSelectExpression = new SelectExpression(selectExpression.getLetList(),
                 newSelectSetOperation, orderBy, limit, selectExpression.isSubquery());
+        newSelectExpression.setSourceLocation(sourceLoc);
         return super.visit(newSelectExpression, arg);
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
index 8d59a3b..abfa14e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppBuiltinFunctionRewriteVisitor.java
@@ -42,7 +42,8 @@
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         //TODO(buyingyi): rewrite SQL temporal functions
         FunctionSignature functionSignature = callExpr.getFunctionSignature();
-        callExpr.setFunctionSignature(FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, true));
+        callExpr.setFunctionSignature(FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, true,
+                callExpr.getSourceLocation()));
         List<Expression> newExprList = new ArrayList<>();
         for (Expression expr : callExpr.getExprList()) {
             newExprList.add(expr.accept(this, arg));
@@ -73,13 +74,16 @@
             newExprList.add(thenExprList.get(index));
         }
         newExprList.add(newCaseExpr.getElseExpr());
-        return new CallExpr(functionSignature, newExprList);
+        CallExpr callExpr = new CallExpr(functionSignature, newExprList);
+        callExpr.setSourceLocation(caseExpr.getSourceLocation());
+        return callExpr;
     }
 
     // Normalizes WHEN expressions so that it can have correct NULL/MISSING semantics as well
     // as type promotion semantics.
     private CaseExpression normalizeCaseExpr(CaseExpression caseExpr) throws CompilationException {
         LiteralExpr trueLiteral = new LiteralExpr(TrueLiteral.INSTANCE);
+        trueLiteral.setSourceLocation(caseExpr.getSourceLocation());
         Expression conditionExpr = caseExpr.getConditionExpr();
         if (trueLiteral.equals(conditionExpr)) {
             return caseExpr;
@@ -90,9 +94,13 @@
             operatorExpr.addOperand((Expression) SqlppRewriteUtil.deepCopy(expr));
             operatorExpr.addOperand(caseExpr.getConditionExpr());
             operatorExpr.addOperator(OperatorType.EQ);
+            operatorExpr.setSourceLocation(expr.getSourceLocation());
             normalizedWhenExprs.add(operatorExpr);
         }
-        return new CaseExpression(trueLiteral, normalizedWhenExprs, caseExpr.getThenExprs(), caseExpr.getElseExpr());
+        CaseExpression newCaseExpr =
+                new CaseExpression(trueLiteral, normalizedWhenExprs, caseExpr.getThenExprs(), caseExpr.getElseExpr());
+        newCaseExpr.setSourceLocation(caseExpr.getSourceLocation());
+        return newCaseExpr;
     }
 
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
index fd50c8e..80844ab 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppDistinctAggregationSugarVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * An AST pre-processor to rewrite distinct aggregates into regular aggregates as follows: <br/>
@@ -81,17 +82,28 @@
      * rewrites {@code expr -> FROM expr AS i SELECT DISTINCT VALUE i}
      */
     private Expression rewriteArgument(Expression argExpr) throws CompilationException {
+        SourceLocation sourceLoc = argExpr.getSourceLocation();
         // From clause
         VariableExpr fromBindingVar = new VariableExpr(context.newVariable());
+        fromBindingVar.setSourceLocation(sourceLoc);
         FromTerm fromTerm = new FromTerm(argExpr, fromBindingVar, null, null);
+        fromTerm.setSourceLocation(sourceLoc);
         FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+        fromClause.setSourceLocation(sourceLoc);
 
         // Select clause.
-        SelectClause selectClause = new SelectClause(new SelectElement(fromBindingVar), null, true);
+        SelectElement selectElement = new SelectElement(fromBindingVar);
+        selectElement.setSourceLocation(sourceLoc);
+        SelectClause selectClause = new SelectClause(selectElement, null, true);
+        selectClause.setSourceLocation(sourceLoc);
 
         // Construct the select expression.
         SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, null, null, null, null);
+        selectBlock.setSourceLocation(sourceLoc);
         SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-        return new SelectExpression(null, selectSetOperation, null, null, true);
+        selectSetOperation.setSourceLocation(sourceLoc);
+        SelectExpression selectExpr = new SelectExpression(null, selectSetOperation, null, null, true);
+        selectExpr.setSourceLocation(sourceLoc);
+        return selectExpr;
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
index c5f0a54..a73d44e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByAggregationSugarVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -48,6 +49,7 @@
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -177,6 +179,7 @@
 
             // Gets the final free variables.
             freeVariables.addAll(freeVariablesInGbyLets);
+            freeVariables.removeIf(SqlppVariableUtil::isExternalVariableReference);
 
             // Gets outer scope variables.
             Collection<VariableExpr> decorVars = scopeChecker.getCurrentScope().getLiveVariables();
@@ -259,12 +262,16 @@
         }
 
         private Expression wrapAggregationArgument(Expression argExpr) throws CompilationException {
+            SourceLocation sourceLoc = argExpr.getSourceLocation();
             Expression expr = argExpr;
             Set<VariableExpr> freeVars = SqlppRewriteUtil.getFreeVariable(expr);
 
             VariableExpr fromBindingVar = new VariableExpr(context.newVariable());
+            fromBindingVar.setSourceLocation(sourceLoc);
             FromTerm fromTerm = new FromTerm(groupVar, fromBindingVar, null, null);
+            fromTerm.setSourceLocation(sourceLoc);
             FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+            fromClause.setSourceLocation(sourceLoc);
 
             // Maps field variable expressions to field accesses.
             Map<Expression, Expression> varExprMap = new HashMap<>();
@@ -272,31 +279,42 @@
                 // Reference to a field in the group variable.
                 if (fieldVars.containsKey(usedVar)) {
                     // Rewrites to a reference to a field in the group variable.
-                    varExprMap.put(usedVar,
-                            new FieldAccessor(fromBindingVar, new VarIdentifier(fieldVars.get(usedVar).getValue())));
+                    FieldAccessor fa =
+                            new FieldAccessor(fromBindingVar, new VarIdentifier(fieldVars.get(usedVar).getValue()));
+                    fa.setSourceLocation(usedVar.getSourceLocation());
+                    varExprMap.put(usedVar, fa);
                 } else if (outerVars.contains(usedVar)) {
                     // Do nothing
                 } else if (fieldVars.size() == 1) {
                     // Rewrites to a reference to a single field in the group variable.
-                    varExprMap.put(usedVar,
-                            new FieldAccessor(new FieldAccessor(fromBindingVar, fieldVars.values().iterator().next()),
-                                    SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar())));
+                    FieldAccessor faInner = new FieldAccessor(fromBindingVar, fieldVars.values().iterator().next());
+                    faInner.setSourceLocation(usedVar.getSourceLocation());
+                    FieldAccessor faOuter =
+                            new FieldAccessor(faInner, SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar()));
+                    faOuter.setSourceLocation(usedVar.getSourceLocation());
+                    varExprMap.put(usedVar, faOuter);
                 } else {
-                    throw new CompilationException("Cannot resolve alias reference for undefined identifier "
-                            + usedVar.getVar().getValue() + " in " + fieldVars);
+                    throw new CompilationException(ErrorCode.AMBIGUOUS_IDENTIFIER, usedVar.getSourceLocation(),
+                            SqlppVariableUtil.toUserDefinedVariableName(usedVar.getVar().getValue()).getValue());
                 }
             }
 
             // Select clause.
             SelectElement selectElement =
                     new SelectElement(SqlppRewriteUtil.substituteExpression(expr, varExprMap, context));
+            selectElement.setSourceLocation(sourceLoc);
             SelectClause selectClause = new SelectClause(selectElement, null, false);
+            selectClause.setSourceLocation(sourceLoc);
 
             // Construct the select expression.
             SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, null, null, null, null);
+            selectBlock.setSourceLocation(sourceLoc);
             SelectSetOperation selectSetOperation =
                     new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
-            return new SelectExpression(null, selectSetOperation, null, null, true);
+            selectSetOperation.setSourceLocation(sourceLoc);
+            SelectExpression selectExpr = new SelectExpression(null, selectSetOperation, null, null, true);
+            selectExpr.setSourceLocation(sourceLoc);
+            return selectExpr;
         }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
index bee5830..faca85e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupByVisitor.java
@@ -72,6 +72,7 @@
         // Sets the group variable.
         if (!gbyClause.hasGroupVar()) {
             VariableExpr groupVar = new VariableExpr(context.newVariable());
+            groupVar.setSourceLocation(gbyClause.getSourceLocation());
             gbyClause.setGroupVar(groupVar);
         }
 
@@ -95,9 +96,11 @@
             List<GbyVariableExpressionPair> gbyPairList = new ArrayList<>();
             List<GbyVariableExpressionPair> decorPairList = new ArrayList<>();
             VariableExpr groupVar = new VariableExpr(context.newVariable());
+            groupVar.setSourceLocation(selectBlock.getSourceLocation());
             List<Pair<Expression, Identifier>> groupFieldList = createGroupFieldList(selectBlock);
             GroupbyClause gbyClause = new GroupbyClause(gbyPairList, decorPairList, new HashMap<>(), groupVar,
                     groupFieldList, false, true);
+            gbyClause.setSourceLocation(selectBlock.getSourceLocation());
             selectBlock.setGroupbyClause(gbyClause);
         }
     }
@@ -128,8 +131,10 @@
     private void addToGroupFieldList(List<Pair<Expression, Identifier>> groupFieldList,
             Collection<VariableExpr> fromBindingVars) {
         for (VariableExpr varExpr : fromBindingVars) {
-            Pair<Expression, Identifier> varIdPair = new Pair<>(new VariableExpr(varExpr.getVar()),
-                    SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar()));
+            VariableExpr newVarExpr = new VariableExpr(varExpr.getVar());
+            newVarExpr.setSourceLocation(varExpr.getSourceLocation());
+            Pair<Expression, Identifier> varIdPair =
+                    new Pair<>(newVarExpr, SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar()));
             groupFieldList.add(varIdPair);
         }
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SubstituteGroupbyExpressionWithVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SubstituteGroupbyExpressionWithVariableVisitor.java
index f157f4f..9e937d0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SubstituteGroupbyExpressionWithVariableVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SubstituteGroupbyExpressionWithVariableVisitor.java
@@ -37,8 +37,13 @@
 import org.apache.asterix.lang.sqlpp.visitor.SqlppSubstituteExpressionVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
 
-// Replaces expressions that appear in having/select/order-by/limit clause and are identical to some
-// group by key expression with the group by key expression.
+/**
+ * <ul>
+ * <li> Generates group by key variables if they were not specified in the query </li>
+ * <li> Replaces expressions that appear in having/select/order-by/limit clause and are identical to some
+ *      group by key expression with the group by key variable </li>
+ * </ul>
+ */
 public class SubstituteGroupbyExpressionWithVariableVisitor extends AbstractSqlppExpressionScopingVisitor {
 
     public SubstituteGroupbyExpressionWithVariableVisitor(LangRewritingContext context) {
@@ -62,7 +67,7 @@
             // Rewrites LET/HAVING/SELECT clauses.
             if (selectBlock.hasLetClausesAfterGroupby()) {
                 for (LetClause letClause : selectBlock.getLetListAfterGroupby()) {
-                    letClause.accept(this, arg);
+                    letClause.accept(visitor, arg);
                 }
             }
             if (selectBlock.hasHavingClause()) {
@@ -84,21 +89,20 @@
         return super.visit(selectBlock, arg);
     }
 
-}
+    private static class SubstituteGroupbyExpressionVisitor extends SqlppSubstituteExpressionVisitor {
 
-class SubstituteGroupbyExpressionVisitor extends SqlppSubstituteExpressionVisitor {
+        private SubstituteGroupbyExpressionVisitor(LangRewritingContext context, Map<Expression, Expression> exprMap) {
+            super(context, exprMap);
+        }
 
-    public SubstituteGroupbyExpressionVisitor(LangRewritingContext context, Map<Expression, Expression> exprMap) {
-        super(context, exprMap);
-    }
-
-    @Override
-    public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
-        FunctionSignature signature = callExpr.getFunctionSignature();
-        if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
-            return callExpr;
-        } else {
-            return super.visit(callExpr, arg);
+        @Override
+        public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
+            FunctionSignature signature = callExpr.getFunctionSignature();
+            if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
+                return callExpr;
+            } else {
+                return super.visit(callExpr, arg);
+            }
         }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index 04f8bc6..8bac4ad 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -19,10 +19,12 @@
 package org.apache.asterix.lang.sqlpp.rewrites.visitor;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
@@ -43,6 +45,7 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopingVisitor {
 
@@ -54,7 +57,7 @@
      * @param context, manages ids of variables and guarantees uniqueness of variables.
      */
     public VariableCheckAndRewriteVisitor(LangRewritingContext context, MetadataProvider metadataProvider,
-            List<VarIdentifier> externalVars) {
+            Collection<VarIdentifier> externalVars) {
         super(context, externalVars);
         this.metadataProvider = metadataProvider;
     }
@@ -94,9 +97,10 @@
     private Expression resolve(VariableExpr varExpr, String dataverseName, String datasetName,
             Expression originalExprWithUndefinedIdentifier, ILangExpression parent) throws CompilationException {
 
-        String varName = varExpr.getVar().getValue();
-
-        VarIdentifier var = lookupVariable(varName);
+        VarIdentifier varId = varExpr.getVar();
+        String varName = varId.getValue();
+        SourceLocation sourceLoc = varExpr.getSourceLocation();
+        VarIdentifier var = lookupVariable(varName, sourceLoc);
         if (var != null) {
             // Exists such an identifier
             varExpr.setIsNewVar(false);
@@ -104,63 +108,71 @@
             return varExpr;
         }
 
-        boolean resolveToDatasetOnly = resolveToDatasetOnly(originalExprWithUndefinedIdentifier, parent);
-        if (resolveToDatasetOnly) {
-            return resolveAsDataset(dataverseName, datasetName);
+        if (SqlppVariableUtil.isExternalVariableIdentifier(varId)) {
+            throw new CompilationException(ErrorCode.PARAMETER_NO_VALUE, sourceLoc,
+                    SqlppVariableUtil.variableNameToDisplayedFieldName(varId.getValue()));
         }
 
-        Set<VariableExpr> localVars = scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
+        boolean resolveToDatasetOnly = resolveToDatasetOnly(originalExprWithUndefinedIdentifier, parent);
+        if (resolveToDatasetOnly) {
+            return resolveAsDataset(dataverseName, datasetName, sourceLoc);
+        }
+
+        Set<VariableExpr> localVars = scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope(),
+                context::isExcludedForFieldAccessVar);
         switch (localVars.size()) {
             case 0:
-                return resolveAsDataset(dataverseName, datasetName);
+                return resolveAsDataset(dataverseName, datasetName, sourceLoc);
             case 1:
                 return resolveAsFieldAccess(localVars.iterator().next(),
-                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue());
+                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue(), sourceLoc);
             default:
                 // More than one possibilities.
-                throw new CompilationException("Cannot resolve ambiguous alias reference for undefined identifier "
-                        + SqlppVariableUtil.toUserDefinedVariableName(varName).getValue() + " in " + localVars);
+                throw new CompilationException(ErrorCode.AMBIGUOUS_IDENTIFIER, sourceLoc,
+                        SqlppVariableUtil.toUserDefinedVariableName(varName).getValue());
         }
     }
 
-    private VarIdentifier lookupVariable(String varName) throws CompilationException {
+    private VarIdentifier lookupVariable(String varName, SourceLocation sourceLoc) throws CompilationException {
         if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new CompilationException(
-                    "Inside limit clauses, it is disallowed to reference a variable having the same name"
-                            + " as any variable bound in the same scope as the limit clause.");
+            throw new CompilationException(ErrorCode.FORBIDDEN_SCOPE, sourceLoc);
         }
         Identifier ident = scopeChecker.lookupSymbol(varName);
         return ident != null ? (VarIdentifier) ident : null;
     }
 
-    private Expression resolveAsDataset(String dataverseName, String datasetName) throws CompilationException {
-        if (!datasetExists(dataverseName, datasetName)) {
-            throwUnresolvableError(dataverseName, datasetName);
+    private Expression resolveAsDataset(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
+        if (!datasetExists(dataverseName, datasetName, sourceLoc)) {
+            throwUnresolvableError(dataverseName, datasetName, sourceLoc);
         }
         String fullyQualifiedName = dataverseName == null ? datasetName : dataverseName + "." + datasetName;
         List<Expression> argList = new ArrayList<>(1);
         argList.add(new LiteralExpr(new StringLiteral(fullyQualifiedName)));
-        return new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
+        CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
     // Rewrites for an field access by name
-    private Expression resolveAsFieldAccess(VariableExpr var, String fieldName) throws CompilationException {
+    private Expression resolveAsFieldAccess(VariableExpr var, String fieldName, SourceLocation sourceLoc) {
         List<Expression> argList = new ArrayList<>(2);
         argList.add(var);
         argList.add(new LiteralExpr(new StringLiteral(fieldName)));
-        return new CallExpr(new FunctionSignature(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argList);
+        CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.FIELD_ACCESS_BY_NAME), argList);
+        callExpr.setSourceLocation(sourceLoc);
+        return callExpr;
     }
 
-    private void throwUnresolvableError(String dataverseName, String datasetName) throws CompilationException {
+    private void throwUnresolvableError(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
         String defaultDataverseName = metadataProvider.getDefaultDataverseName();
         if (dataverseName == null && defaultDataverseName == null) {
-            throw new CompilationException("Cannot find dataset " + datasetName
-                    + " because there is no dataverse declared, nor an alias with name " + datasetName + "!");
+            throw new CompilationException(ErrorCode.NAME_RESOLVE_UNKNOWN_DATASET, sourceLoc, datasetName);
         }
         //If no available dataset nor in-scope variable to resolve to, we throw an error.
-        throw new CompilationException("Cannot find dataset " + datasetName + " in dataverse "
-                + (dataverseName == null ? defaultDataverseName : dataverseName) + " nor an alias with name "
-                + datasetName + "!");
+        throw new CompilationException(ErrorCode.NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                dataverseName == null ? defaultDataverseName : dataverseName);
     }
 
     // For a From/Join/UNNEST/Quantifiers binding expression, we resolve the undefined identifier reference as
@@ -171,12 +183,13 @@
         return parent.accept(visitor, originalExpressionWithUndefinedIdentifier);
     }
 
-    private boolean datasetExists(String dataverseName, String datasetName) throws CompilationException {
+    private boolean datasetExists(String dataverseName, String datasetName, SourceLocation sourceLoc)
+            throws CompilationException {
         try {
             return metadataProvider.findDataset(dataverseName, datasetName) != null
                     || fullyQualifiedDatasetNameExists(datasetName);
         } catch (AlgebricksException e) {
-            throw new CompilationException(e);
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
         }
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
index 96c61f2..4842026 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/ExpressionToVariableUtil.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.sqlpp.parser.ParseException;
+import org.apache.asterix.lang.sqlpp.parser.SqlppParseException;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -39,18 +40,21 @@
     private static String getGeneratedIdentifier(Expression expr) throws ParseException {
         if (expr.getKind() == Kind.VARIABLE_EXPRESSION) {
             VariableExpr bindingVarExpr = (VariableExpr) expr;
-            return bindingVarExpr.getVar().getValue();
+            VarIdentifier var = bindingVarExpr.getVar();
+            return SqlppVariableUtil.isExternalVariableIdentifier(var) ? null : var.getValue();
         } else if (expr.getKind() == Kind.FIELD_ACCESSOR_EXPRESSION) {
             FieldAccessor fa = (FieldAccessor) expr;
             return SqlppVariableUtil.toInternalVariableName(fa.getIdent().getValue());
         } else {
+            String exprText;
             try {
-                throw new ParseException(
-                        "Need an alias for the enclosed expression:\n" + SqlppFormatPrintUtil.toString(expr));
+                exprText = SqlppFormatPrintUtil.toString(expr);
             } catch (CompilationException e) {
                 LOGGER.error(e.getLocalizedMessage(), e);
-                throw new ParseException(e.getLocalizedMessage());
+                throw new SqlppParseException(expr.getSourceLocation(), e.getLocalizedMessage());
             }
+            throw new SqlppParseException(expr.getSourceLocation(),
+                    "Need an alias for the enclosed expression:\n" + exprText);
         }
     }
 
@@ -93,6 +97,7 @@
             VarIdentifier var = new VarIdentifier(varName);
             VariableExpr varExpr = new VariableExpr();
             varExpr.setVar(var);
+            varExpr.setSourceLocation(expr.getSourceLocation());
             return varExpr;
         } catch (ParseException e) {
             if (raiseError) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
index 1ff7c9e..7773072 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -33,11 +34,14 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FunctionMapUtil {
 
     public static final String CONCAT = "concat";
-    private final static String CORE_AGGREGATE_PREFIX = "coll_";
+    private final static String CORE_AGGREGATE_PREFIX = "strict_";
+    // This is a transitional case. The ALT_CORE_AGGREGATE_PREFIX should be removed again.
+    private final static String ALT_CORE_AGGREGATE_PREFIX = "coll_";
     private final static String CORE_SQL_AGGREGATE_PREFIX = "array_";
     private final static String INTERNAL_SQL_AGGREGATE_PREFIX = "sql-";
 
@@ -69,31 +73,6 @@
     }
 
     /**
-     * Whether a function signature is a SQL++ core aggregate function.
-     *
-     * @param fs,
-     *            the function signature.
-     * @return true if the function signature is a SQL++ core aggregate,
-     *         false otherwise.
-     */
-    public static boolean isCoreAggregateFunction(FunctionSignature fs) {
-        String name = fs.getName().toLowerCase();
-        boolean coreAgg = name.startsWith(CORE_AGGREGATE_PREFIX);
-        boolean coreSqlAgg = name.startsWith(CORE_SQL_AGGREGATE_PREFIX);
-        if (!coreAgg && !coreSqlAgg) {
-            return false;
-        }
-        String internalName = coreAgg ? name.substring(CORE_AGGREGATE_PREFIX.length())
-                : (INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length()));
-        IFunctionInfo finfo = FunctionUtil
-                .getFunctionInfo(new FunctionIdentifier(FunctionConstants.ASTERIX_NS, internalName, fs.getArity()));
-        if (finfo == null) {
-            return false;
-        }
-        return BuiltinFunctions.getAggregateFunction(finfo.getFunctionIdentifier()) != null;
-    }
-
-    /**
      * Get the corresponding SQL++ core aggregate function from the SQL-92 aggregate function.
      *
      * @param fs,
@@ -112,18 +91,30 @@
     /**
      * Maps a user invoked function signature to a system internal function signature.
      *
-     * @param fs,
+     * @param fs
      *            the user typed function.
+     * @param checkSql92Aggregate
+     *            enable check if the function is a SQL-92 aggregate function
+     * @param sourceLoc
+     *            the source location of the function call
      * @return the system internal function.
+     * @throws CompilationException
+     *             if checkSql92Aggregate is true and the function is a SQL-92 aggregate function
      */
-    public static FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, boolean checkSql92Aggregate)
-            throws CompilationException {
-        if (isCoreAggregateFunction(fs)) {
-            return internalizeCoreAggregateFunctionName(fs);
+    public static FunctionSignature normalizeBuiltinFunctionSignature(FunctionSignature fs, boolean checkSql92Aggregate,
+            SourceLocation sourceLoc) throws CompilationException {
+        String internalName = getInternalCoreAggregateFunctionName(fs);
+        if (internalName != null) {
+            FunctionIdentifier fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, internalName, fs.getArity());
+            IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
+            if (finfo != null && BuiltinFunctions.getAggregateFunction(finfo.getFunctionIdentifier()) != null) {
+                return new FunctionSignature(FunctionConstants.ASTERIX_NS, internalName, fs.getArity());
+            }
         } else if (checkSql92Aggregate && isSql92AggregateFunction(fs)) {
-            throw new CompilationException(fs.getName()
-                    + " is a SQL-92 aggregate function. The SQL++ core aggregate function " + CORE_SQL_AGGREGATE_PREFIX
-                    + fs.getName().toLowerCase() + " could potentially express the intent.");
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                    fs.getName() + " is a SQL-92 aggregate function. The SQL++ core aggregate function "
+                            + CORE_SQL_AGGREGATE_PREFIX + fs.getName().toLowerCase()
+                            + " could potentially express the intent.");
         }
         String mappedName = CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(fs).getName();
         return new FunctionSignature(fs.getNamespace(), mappedName, fs.getArity());
@@ -143,26 +134,30 @@
             return callExpr;
         }
         callExpr.setFunctionSignature(new FunctionSignature(FunctionConstants.ASTERIX_NS, internalFuncName, 1));
-        callExpr.setExprList(new ArrayList<>(Collections.singletonList(
-                new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, callExpr.getExprList()))));
+        ListConstructor listConstr =
+                new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, callExpr.getExprList());
+        listConstr.setSourceLocation(callExpr.getSourceLocation());
+        callExpr.setExprList(new ArrayList<>(Collections.singletonList(listConstr)));
         return callExpr;
     }
 
     /**
-     * Removes the "array_" prefix for user-facing SQL++ core aggregate function names.
+     * Removes the "array_", "strict_", or "coll_" prefix for user-facing SQL++ core aggregate function names.
      *
-     * @param fs,
+     * @param fs
      *            a user-facing SQL++ core aggregate function signature.
-     * @return the AsterixDB internal function signature for the aggregate function.
-     * @throws CompilationException
+     * @return the AsterixDB internal function name for the aggregate function.
      */
-    private static FunctionSignature internalizeCoreAggregateFunctionName(FunctionSignature fs)
-            throws CompilationException {
+    private static String getInternalCoreAggregateFunctionName(FunctionSignature fs) {
         String name = fs.getName().toLowerCase();
-        boolean coreAgg = name.startsWith(CORE_AGGREGATE_PREFIX);
-        String lowerCaseName = coreAgg ? name.substring(CORE_AGGREGATE_PREFIX.length())
-                : (INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length()));
-        return new FunctionSignature(FunctionConstants.ASTERIX_NS, lowerCaseName, fs.getArity());
+        if (name.startsWith(CORE_AGGREGATE_PREFIX)) {
+            return name.substring(CORE_AGGREGATE_PREFIX.length());
+        } else if (name.startsWith(ALT_CORE_AGGREGATE_PREFIX)) {
+            return name.substring(ALT_CORE_AGGREGATE_PREFIX.length());
+        } else if (name.startsWith(CORE_SQL_AGGREGATE_PREFIX)) {
+            return INTERNAL_SQL_AGGREGATE_PREFIX + name.substring(CORE_SQL_AGGREGATE_PREFIX.length());
+        } else {
+            return null;
+        }
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
index 228ea23..5a30b42 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppRewriteUtil.java
@@ -83,6 +83,7 @@
         // Creates a wrapper query for the expression so that if the expression itself
         // is the key, it can also be replaced.
         Query wrapper = new Query(false);
+        wrapper.setSourceLocation(expression.getSourceLocation());
         wrapper.setBody(expression);
         // Creates a substitution visitor.
         SqlppSubstituteExpressionVisitor visitor = new SqlppSubstituteExpressionVisitor(context, exprMap);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
index 519627c..14dfaeb 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
@@ -40,6 +40,8 @@
 
     private static final String USER_VAR_PREFIX = "$";
 
+    private static final String EXTERNAL_VAR_PREFIX = "?";
+
     private SqlppVariableUtil() {
     }
 
@@ -79,7 +81,19 @@
     }
 
     public static VarIdentifier toInternalVariableIdentifier(String idName) {
-        return new VarIdentifier(USER_VAR_PREFIX + idName);
+        return new VarIdentifier(toInternalVariableName(idName));
+    }
+
+    public static String toExternalVariableName(String varName) {
+        return EXTERNAL_VAR_PREFIX + varName;
+    }
+
+    public static boolean isExternalVariableIdentifier(VarIdentifier varId) {
+        return varId.getValue().startsWith(EXTERNAL_VAR_PREFIX);
+    }
+
+    public static boolean isExternalVariableReference(VariableExpr varExpr) {
+        return isExternalVariableIdentifier(varExpr.getVar());
     }
 
     public static Collection<VariableExpr> getFreeVariables(ILangExpression langExpr) throws CompilationException {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 1166148..766540e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -78,7 +78,9 @@
         for (FromTerm fromTerm : fromClause.getFromTerms()) {
             fromTerms.add((FromTerm) fromTerm.accept(this, arg));
         }
-        return new FromClause(fromTerms);
+        FromClause copy = new FromClause(fromTerms);
+        copy.setSourceLocation(fromClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -94,7 +96,9 @@
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
             correlateClauses.add((AbstractBinaryCorrelateClause) correlateClause.accept(this, arg));
         }
-        return new FromTerm(fromExpr, fromVar, positionVar, correlateClauses);
+        FromTerm copy = new FromTerm(fromExpr, fromVar, positionVar, correlateClauses);
+        copy.setSourceLocation(fromTerm.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -104,8 +108,10 @@
         VariableExpr rightPositionVar = joinClause.getPositionalVariable() == null ? null
                 : (VariableExpr) joinClause.getPositionalVariable().accept(this, arg);
         Expression conditionExpresion = (Expression) joinClause.getConditionExpression().accept(this, arg);
-        return new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
+        JoinClause copy = new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
                 conditionExpresion);
+        copy.setSourceLocation(joinClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -115,8 +121,10 @@
         VariableExpr rightPositionVar = nestClause.getPositionalVariable() == null ? null
                 : (VariableExpr) nestClause.getPositionalVariable().accept(this, arg);
         Expression conditionExpresion = (Expression) nestClause.getConditionExpression().accept(this, arg);
-        return new NestClause(nestClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
+        NestClause copy = new NestClause(nestClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
                 conditionExpresion);
+        copy.setSourceLocation(nestClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -125,13 +133,18 @@
         VariableExpr rightVar = (VariableExpr) unnestClause.getRightVariable().accept(this, arg);
         VariableExpr rightPositionVar = unnestClause.getPositionalVariable() == null ? null
                 : (VariableExpr) unnestClause.getPositionalVariable().accept(this, arg);
-        return new UnnestClause(unnestClause.getJoinType(), rightExpression, rightVar, rightPositionVar);
+        UnnestClause copy = new UnnestClause(unnestClause.getJoinType(), rightExpression, rightVar, rightPositionVar);
+        copy.setSourceLocation(unnestClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public Projection visit(Projection projection, Void arg) throws CompilationException {
-        return new Projection(projection.star() ? null : (Expression) projection.getExpression().accept(this, arg),
-                projection.getName(), projection.star(), projection.varStar());
+        Projection copy =
+                new Projection(projection.star() ? null : (Expression) projection.getExpression().accept(this, arg),
+                        projection.getName(), projection.star(), projection.varStar());
+        copy.setSourceLocation(projection.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -170,8 +183,10 @@
             havingClause = (HavingClause) selectBlock.getHavingClause().accept(this, arg);
         }
         selectCluase = (SelectClause) selectBlock.getSelectClause().accept(this, arg);
-        return new SelectBlock(selectCluase, fromClause, letClauses, whereClause, gbyClause, gbyLetClauses,
+        SelectBlock copy = new SelectBlock(selectCluase, fromClause, letClauses, whereClause, gbyClause, gbyLetClauses,
                 havingClause);
+        copy.setSourceLocation(selectBlock.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -184,12 +199,16 @@
         if (selectClause.selectRegular()) {
             selectRegular = (SelectRegular) selectClause.getSelectRegular().accept(this, arg);
         }
-        return new SelectClause(selectElement, selectRegular, selectClause.distinct());
+        SelectClause copy = new SelectClause(selectElement, selectRegular, selectClause.distinct());
+        copy.setSourceLocation(selectClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public SelectElement visit(SelectElement selectElement, Void arg) throws CompilationException {
-        return new SelectElement((Expression) selectElement.getExpression().accept(this, arg));
+        SelectElement copy = new SelectElement((Expression) selectElement.getExpression().accept(this, arg));
+        copy.setSourceLocation(selectElement.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -198,7 +217,9 @@
         for (Projection projection : selectRegular.getProjections()) {
             projections.add((Projection) projection.accept(this, arg));
         }
-        return new SelectRegular(projections);
+        SelectRegular copy = new SelectRegular(projections);
+        copy.setSourceLocation(selectRegular.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -221,28 +242,39 @@
             }
             rightInputs.add(new SetOperationRight(right.getSetOpType(), right.isSetSemantics(), newRightInput));
         }
-        return new SelectSetOperation(newLeftInput, rightInputs);
+        SelectSetOperation copy = new SelectSetOperation(newLeftInput, rightInputs);
+        copy.setSourceLocation(selectSetOperation.getSourceLocation());
+        return copy;
     }
 
     @Override
     public HavingClause visit(HavingClause havingClause, Void arg) throws CompilationException {
-        return new HavingClause((Expression) havingClause.getFilterExpression().accept(this, arg));
+        HavingClause copy = new HavingClause((Expression) havingClause.getFilterExpression().accept(this, arg));
+        copy.setSourceLocation(havingClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public Query visit(Query q, Void arg) throws CompilationException {
-        return new Query(q.isExplain(), q.isTopLevel(), (Expression) q.getBody().accept(this, arg), q.getVarCounter(),
-                q.getExternalVars());
+        Query copy =
+                new Query(q.isExplain(), q.isTopLevel(), (Expression) q.getBody().accept(this, arg), q.getVarCounter());
+        copy.setSourceLocation(q.getSourceLocation());
+        return copy;
     }
 
     @Override
     public FunctionDecl visit(FunctionDecl fd, Void arg) throws CompilationException {
-        return new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        FunctionDecl copy =
+                new FunctionDecl(fd.getSignature(), fd.getParamList(), (Expression) fd.getFuncBody().accept(this, arg));
+        copy.setSourceLocation(fd.getSourceLocation());
+        return copy;
     }
 
     @Override
     public WhereClause visit(WhereClause whereClause, Void arg) throws CompilationException {
-        return new WhereClause((Expression) whereClause.getWhereExpr().accept(this, arg));
+        WhereClause copy = new WhereClause((Expression) whereClause.getWhereExpr().accept(this, arg));
+        copy.setSourceLocation(whereClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -251,7 +283,9 @@
         for (Expression orderExpr : oc.getOrderbyList()) {
             newOrderbyList.add((Expression) orderExpr.accept(this, arg));
         }
-        return new OrderbyClause(newOrderbyList, oc.getModifierList());
+        OrderbyClause copy = new OrderbyClause(newOrderbyList, oc.getModifierList());
+        copy.setSourceLocation(oc.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -281,21 +315,27 @@
         for (Pair<Expression, Identifier> field : gc.getGroupFieldList()) {
             groupFieldList.add(new Pair<>((Expression) field.first.accept(this, arg), field.second));
         }
-        return new GroupbyClause(gbyPairList, decorPairList, withVarMap, groupVarExpr, groupFieldList,
+        GroupbyClause copy = new GroupbyClause(gbyPairList, decorPairList, withVarMap, groupVarExpr, groupFieldList,
                 gc.hasHashGroupByHint(), gc.isGroupAll());
+        copy.setSourceLocation(gc.getSourceLocation());
+        return copy;
     }
 
     @Override
     public LimitClause visit(LimitClause limitClause, Void arg) throws CompilationException {
         Expression limitExpr = (Expression) limitClause.getLimitExpr().accept(this, arg);
         Expression offsetExpr = limitClause.hasOffset() ? (Expression) limitClause.getOffset().accept(this, arg) : null;
-        return new LimitClause(limitExpr, offsetExpr);
+        LimitClause copy = new LimitClause(limitExpr, offsetExpr);
+        copy.setSourceLocation(limitClause.getSourceLocation());
+        return copy;
     }
 
     @Override
     public LetClause visit(LetClause letClause, Void arg) throws CompilationException {
-        return new LetClause((VariableExpr) letClause.getVarExpr().accept(this, arg),
+        LetClause copy = new LetClause((VariableExpr) letClause.getVarExpr().accept(this, arg),
                 (Expression) letClause.getBindingExpr().accept(this, arg));
+        copy.setSourceLocation(letClause.getSourceLocation());
+        return copy;
     }
 
     @Override
@@ -317,18 +357,18 @@
 
         // visit order by
         if (selectExpression.hasOrderby()) {
-            List<Expression> orderExprs = new ArrayList<>();
-            for (Expression orderExpr : selectExpression.getOrderbyClause().getOrderbyList()) {
-                orderExprs.add((Expression) orderExpr.accept(this, arg));
-            }
-            orderby = new OrderbyClause(orderExprs, selectExpression.getOrderbyClause().getModifierList());
+            orderby = (OrderbyClause) selectExpression.getOrderbyClause().accept(this, arg);
         }
 
         // visit limit
         if (selectExpression.hasLimit()) {
             limit = (LimitClause) selectExpression.getLimitClause().accept(this, arg);
         }
-        return new SelectExpression(lets, select, orderby, limit, selectExpression.isSubquery());
+
+        SelectExpression copy = new SelectExpression(lets, select, orderby, limit, selectExpression.isSubquery());
+        copy.addHints(selectExpression.getHints());
+
+        return copy;
     }
 
     @Override
@@ -338,7 +378,10 @@
 
     @Override
     public ListConstructor visit(ListConstructor lc, Void arg) throws CompilationException {
-        return new ListConstructor(lc.getType(), copyExprList(lc.getExprList(), arg));
+        ListConstructor copy = new ListConstructor(lc.getType(), copyExprList(lc.getExprList(), arg));
+        copy.setSourceLocation(lc.getSourceLocation());
+        copy.addHints(lc.getHints());
+        return copy;
     }
 
     @Override
@@ -349,13 +392,19 @@
                     (Expression) binding.getRightExpr().accept(this, arg));
             bindings.add(fb);
         }
-        return new RecordConstructor(bindings);
+        RecordConstructor copy = new RecordConstructor(bindings);
+        copy.setSourceLocation(rc.getSourceLocation());
+        copy.addHints(rc.getHints());
+        return copy;
     }
 
     @Override
     public OperatorExpr visit(OperatorExpr operatorExpr, Void arg) throws CompilationException {
-        return new OperatorExpr(copyExprList(operatorExpr.getExprList(), arg), operatorExpr.getExprBroadcastIdx(),
-                operatorExpr.getOpList(), operatorExpr.isCurrentop());
+        OperatorExpr copy = new OperatorExpr(copyExprList(operatorExpr.getExprList(), arg),
+                operatorExpr.getExprBroadcastIdx(), operatorExpr.getOpList(), operatorExpr.isCurrentop());
+        copy.setSourceLocation(operatorExpr.getSourceLocation());
+        copy.addHints(operatorExpr.getHints());
+        return copy;
     }
 
     @Override
@@ -363,7 +412,10 @@
         Expression conditionExpr = (Expression) ifExpr.getCondExpr().accept(this, arg);
         Expression thenExpr = (Expression) ifExpr.getThenExpr().accept(this, arg);
         Expression elseExpr = (Expression) ifExpr.getElseExpr().accept(this, arg);
-        return new IfExpr(conditionExpr, thenExpr, elseExpr);
+        IfExpr copy = new IfExpr(conditionExpr, thenExpr, elseExpr);
+        copy.setSourceLocation(ifExpr.getSourceLocation());
+        copy.addHints(ifExpr.getHints());
+        return copy;
     }
 
     @Override
@@ -375,7 +427,10 @@
             quantifiedPairs.add(new QuantifiedPair(var, expr));
         }
         Expression condition = (Expression) qe.getSatisfiesExpr().accept(this, arg);
-        return new QuantifiedExpression(qe.getQuantifier(), quantifiedPairs, condition);
+        QuantifiedExpression copy = new QuantifiedExpression(qe.getQuantifier(), quantifiedPairs, condition);
+        copy.setSourceLocation(qe.getSourceLocation());
+        copy.addHints(qe.getHints());
+        return copy;
     }
 
     @Override
@@ -384,24 +439,35 @@
         for (Expression expr : callExpr.getExprList()) {
             newExprList.add((Expression) expr.accept(this, arg));
         }
-        return new CallExpr(callExpr.getFunctionSignature(), newExprList);
+        CallExpr copy = new CallExpr(callExpr.getFunctionSignature(), newExprList);
+        copy.setSourceLocation(callExpr.getSourceLocation());
+        copy.addHints(callExpr.getHints());
+        return copy;
     }
 
     @Override
     public VariableExpr visit(VariableExpr varExpr, Void arg) throws CompilationException {
         VariableExpr clonedVar = new VariableExpr(new VarIdentifier(varExpr.getVar()));
+        clonedVar.setSourceLocation(varExpr.getSourceLocation());
         clonedVar.setIsNewVar(varExpr.getIsNewVar());
+        clonedVar.addHints(varExpr.getHints());
         return clonedVar;
     }
 
     @Override
     public UnaryExpr visit(UnaryExpr u, Void arg) throws CompilationException {
-        return new UnaryExpr(u.getExprType(), (Expression) u.getExpr().accept(this, arg));
+        UnaryExpr copy = new UnaryExpr(u.getExprType(), (Expression) u.getExpr().accept(this, arg));
+        copy.setSourceLocation(u.getSourceLocation());
+        copy.addHints(u.getHints());
+        return copy;
     }
 
     @Override
     public FieldAccessor visit(FieldAccessor fa, Void arg) throws CompilationException {
-        return new FieldAccessor((Expression) fa.getExpr().accept(this, arg), fa.getIdent());
+        FieldAccessor copy = new FieldAccessor((Expression) fa.getExpr().accept(this, arg), fa.getIdent());
+        copy.setSourceLocation(fa.getSourceLocation());
+        copy.addHints(fa.getHints());
+        return copy;
     }
 
     @Override
@@ -411,7 +477,10 @@
         if (ia.getIndexExpr() != null) {
             indexExpr = (Expression) ia.getIndexExpr().accept(this, arg);
         }
-        return new IndexAccessor(expr, indexExpr);
+        IndexAccessor copy = new IndexAccessor(expr, indexExpr);
+        copy.setSourceLocation(ia.getSourceLocation());
+        copy.addHints(ia.getHints());
+        return copy;
     }
 
     @Override
@@ -420,7 +489,10 @@
         List<Expression> whenExprList = copyExprList(caseExpr.getWhenExprs(), arg);
         List<Expression> thenExprList = copyExprList(caseExpr.getThenExprs(), arg);
         Expression elseExpr = (Expression) caseExpr.getElseExpr().accept(this, arg);
-        return new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        CaseExpression copy = new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        copy.setSourceLocation(caseExpr.getSourceLocation());
+        copy.addHints(caseExpr.getHints());
+        return copy;
     }
 
     private List<Expression> copyExprList(List<Expression> exprs, Void arg) throws CompilationException {
@@ -430,5 +502,4 @@
         }
         return newExprList;
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index 5f6b75b..0973bec 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -250,7 +250,7 @@
     public Void visit(CallExpr pf, Integer step) throws CompilationException {
         FunctionSignature functionSignature = pf.getFunctionSignature();
         FunctionSignature normalizedFunctionSignature =
-                FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, false);
+                FunctionMapUtil.normalizeBuiltinFunctionSignature(functionSignature, false, pf.getSourceLocation());
         if (BuiltinFunctions.isBuiltinCompilerFunction(normalizedFunctionSignature, true)) {
             functionSignature = normalizedFunctionSignature;
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index 0222e0e..e00a3bd 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -77,7 +77,9 @@
             // therefore we propagate the substitution environment.
             currentEnv = p.second;
         }
-        return new Pair<>(new FromClause(newFromTerms), currentEnv);
+        FromClause newFromClause = new FromClause(newFromTerms);
+        newFromClause.setSourceLocation(fromClause.getSourceLocation());
+        return new Pair<>(newFromClause, currentEnv);
     }
 
     @Override
@@ -114,7 +116,9 @@
                 }
             }
         }
-        return new Pair<>(new FromTerm(newLeftExpr, newLeftVar, newLeftPosVar, newCorrelateClauses), currentEnv);
+        FromTerm newFromTerm = new FromTerm(newLeftExpr, newLeftVar, newLeftPosVar, newCorrelateClauses);
+        newFromTerm.setSourceLocation(fromTerm.getSourceLocation());
+        return new Pair<>(newFromTerm, currentEnv);
     }
 
     @Override
@@ -139,6 +143,7 @@
 
         JoinClause newJoinClause =
                 new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar, conditionExpr);
+        newJoinClause.setSourceLocation(joinClause.getSourceLocation());
         return new Pair<>(newJoinClause, currentEnv);
     }
 
@@ -162,9 +167,10 @@
         // The condition can refer to the newRightVar and newRightPosVar.
         Expression conditionExpr = (Expression) nestClause.getConditionExpression().accept(this, currentEnv).first;
 
-        NestClause newJoinClause =
+        NestClause newNestClause =
                 new NestClause(nestClause.getJoinType(), rightExpr, newRightVar, newRightPosVar, conditionExpr);
-        return new Pair<>(newJoinClause, currentEnv);
+        newNestClause.setSourceLocation(nestClause.getSourceLocation());
+        return new Pair<>(newNestClause, currentEnv);
     }
 
     @Override
@@ -185,9 +191,10 @@
             currentEnv.removeSubstitution(newRightPosVar);
         }
         // The condition can refer to the newRightVar and newRightPosVar.
-        UnnestClause newJoinClause =
+        UnnestClause newUnnestClause =
                 new UnnestClause(unnestClause.getJoinType(), rightExpr, newRightVar, newRightPosVar);
-        return new Pair<>(newJoinClause, currentEnv);
+        newUnnestClause.setSourceLocation(unnestClause.getSourceLocation());
+        return new Pair<>(newUnnestClause, currentEnv);
     }
 
     @Override
@@ -198,6 +205,7 @@
         }
         Projection newProjection = new Projection((Expression) projection.getExpression().accept(this, env).first,
                 projection.getName(), projection.star(), projection.varStar());
+        newProjection.setSourceLocation(projection.getSourceLocation());
         return new Pair<>(newProjection, env);
     }
 
@@ -255,8 +263,10 @@
         WhereClause whereClause = newWhere == null ? null : (WhereClause) newWhere.first;
         GroupbyClause groupbyClause = newGroupby == null ? null : (GroupbyClause) newGroupby.first;
         HavingClause havingClause = newHaving == null ? null : (HavingClause) newHaving.first;
-        return new Pair<>(new SelectBlock((SelectClause) newSelect.first, fromClause, newLetClauses, whereClause,
-                groupbyClause, newLetClausesAfterGby, havingClause), currentEnv);
+        SelectBlock newSelectBlock = new SelectBlock((SelectClause) newSelect.first, fromClause, newLetClauses,
+                whereClause, groupbyClause, newLetClausesAfterGby, havingClause);
+        newSelectBlock.setSourceLocation(selectBlock.getSourceLocation());
+        return new Pair<>(newSelectBlock, currentEnv);
     }
 
     @Override
@@ -266,13 +276,15 @@
         if (selectClause.selectElement()) {
             Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectElement =
                     selectClause.getSelectElement().accept(this, env);
-            return new Pair<>(new SelectClause((SelectElement) newSelectElement.first, null, distinct),
-                    newSelectElement.second);
+            SelectClause newSelectClause = new SelectClause((SelectElement) newSelectElement.first, null, distinct);
+            newSelectClause.setSourceLocation(selectClause.getSourceLocation());
+            return new Pair<>(newSelectClause, newSelectElement.second);
         } else {
             Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectRegular =
                     selectClause.getSelectRegular().accept(this, env);
-            return new Pair<>(new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct),
-                    newSelectRegular.second);
+            SelectClause newSelectClause = new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct);
+            newSelectClause.setSourceLocation(selectClause.getSourceLocation());
+            return new Pair<>(newSelectClause, newSelectRegular.second);
         }
     }
 
@@ -281,7 +293,9 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> newExpr =
                 selectElement.getExpression().accept(this, env);
-        return new Pair<>(new SelectElement((Expression) newExpr.first), newExpr.second);
+        SelectElement newSelectElement = new SelectElement((Expression) newExpr.first);
+        newSelectElement.setSourceLocation(selectElement.getSourceLocation());
+        return new Pair<>(newSelectElement, newExpr.second);
     }
 
     @Override
@@ -291,7 +305,9 @@
         for (Projection projection : selectRegular.getProjections()) {
             newProjections.add((Projection) projection.accept(this, env).first);
         }
-        return new Pair<>(new SelectRegular(newProjections), env);
+        SelectRegular newSelectRegular = new SelectRegular(newProjections);
+        newSelectRegular.setSourceLocation(selectRegular.getSourceLocation());
+        return new Pair<>(newSelectRegular, env);
     }
 
     @Override
@@ -329,6 +345,7 @@
             }
         }
         SelectSetOperation newSelectSetOperation = new SelectSetOperation(newLeftInput, newRightInputs);
+        newSelectSetOperation.setSourceLocation(selectSetOperation.getSourceLocation());
         return new Pair<>(newSelectSetOperation, selectSetOperation.hasRightInputs() ? env : leftResult.second);
     }
 
@@ -366,9 +383,10 @@
             newLimitClause = (LimitClause) p.first;
             currentEnv = p.second;
         }
-        return new Pair<>(
-                new SelectExpression(newLetList, newSelectSetOperation, newOrderbyClause, newLimitClause, subquery),
-                currentEnv);
+        SelectExpression newSelectExpression =
+                new SelectExpression(newLetList, newSelectSetOperation, newOrderbyClause, newLimitClause, subquery);
+        newSelectExpression.setSourceLocation(selectExpression.getSourceLocation());
+        return new Pair<>(newSelectExpression, currentEnv);
     }
 
     @Override
@@ -376,6 +394,7 @@
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p = havingClause.getFilterExpression().accept(this, env);
         HavingClause newHavingClause = new HavingClause((Expression) p.first);
+        newHavingClause.setSourceLocation(havingClause.getSourceLocation());
         return new Pair<>(newHavingClause, p.second);
     }
 
@@ -389,7 +408,7 @@
                 VariableCloneAndSubstitutionUtil.visitAndCloneExprList(caseExpr.getThenExprs(), env, this);
         Expression elseExpr = (Expression) caseExpr.getElseExpr().accept(this, env).first;
         CaseExpression newCaseExpr = new CaseExpression(conditionExpr, whenExprList, thenExprList, elseExpr);
+        newCaseExpr.setSourceLocation(caseExpr.getSourceLocation());
         return new Pair<>(newCaseExpr, env);
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
index 4542bab..bf57184 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
@@ -59,36 +59,45 @@
         LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
         arguments.add(argumentLiteral);
         CallExpr callExpression = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), arguments);
+        callExpression.setSourceLocation(deleteStmt.getSourceLocation());
 
         // From clause.
         VariableExpr var = deleteStmt.getVariableExpr();
         FromTerm fromTerm = new FromTerm(callExpression, var, null, null);
-        @SuppressWarnings("unchecked")
+        fromTerm.setSourceLocation(var.getSourceLocation());
         FromClause fromClause = new FromClause(Collections.singletonList(fromTerm));
+        fromClause.setSourceLocation(var.getSourceLocation());
 
         // Where clause.
         WhereClause whereClause = null;
         Expression condition = deleteStmt.getCondition();
         if (condition != null) {
             whereClause = new WhereClause(condition);
+            whereClause.setSourceLocation(condition.getSourceLocation());
         }
 
         // Select clause.
         VariableExpr returnExpr = new VariableExpr(var.getVar());
         returnExpr.setIsNewVar(false);
+        returnExpr.setSourceLocation(var.getSourceLocation());
         SelectElement selectElement = new SelectElement(returnExpr);
+        selectElement.setSourceLocation(deleteStmt.getSourceLocation());
         SelectClause selectClause = new SelectClause(selectElement, null, false);
+        selectClause.setSourceLocation(deleteStmt.getSourceLocation());
 
         // Construct the select expression.
         SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, whereClause, null, null, null);
+        selectBlock.setSourceLocation(var.getSourceLocation());
         SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
+        selectSetOperation.setSourceLocation(var.getSourceLocation());
         SelectExpression selectExpression = new SelectExpression(null, selectSetOperation, null, null, false);
+        selectExpression.setSourceLocation(var.getSourceLocation());
         Query query = new Query(false, false, selectExpression, 0);
         query.setBody(selectExpression);
+        query.setSourceLocation(deleteStmt.getSourceLocation());
 
         // return the delete statement.
         deleteStmt.setQuery(query);
         return null;
     }
-
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSubstituteExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSubstituteExpressionVisitor.java
index 39229e2..636f8a6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSubstituteExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSubstituteExpressionVisitor.java
@@ -50,9 +50,19 @@
     // asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-3/gby-expr-3.3.query.sqlpp
     @Override
     protected Expression preVisit(Expression expr) throws CompilationException {
+        return substitute(expr);
+    }
+
+    protected Expression substitute(Expression expr) throws CompilationException {
+        Expression mappedExpr = getMappedExpr(expr);
+        // Makes a deep copy before returning to avoid shared references.
+        return mappedExpr == null ? expr : (Expression) SqlppRewriteUtil.deepCopy(mappedExpr);
+    }
+
+    protected Expression getMappedExpr(Expression expr) throws CompilationException {
         Expression mappedExpr = exprMap.get(expr);
         if (mappedExpr == null) {
-            return expr;
+            return null;
         }
         Collection<VariableExpr> freeVars = SqlppVariableUtil.getFreeVariables(expr);
         for (VariableExpr freeVar : freeVars) {
@@ -60,10 +70,9 @@
             if (currentScope.findSymbol(freeVar.getVar().getValue()) != null) {
                 // If the expression to be substituted uses variables defined in the outer-most expresion
                 // that is being visited, we shouldn't perform the substitution.
-                return expr;
+                return null;
             }
         }
-        // Makes a deep copy before returning to avoid shared references.
-        return (Expression) SqlppRewriteUtil.deepCopy(mappedExpr);
+        return mappedExpr;
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index bcb131b..df165c0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -18,13 +18,14 @@
  */
 package org.apache.asterix.lang.sqlpp.visitor.base;
 
+import java.util.Collection;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -53,7 +54,7 @@
 import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.algebra.base.Counter;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class AbstractSqlppExpressionScopingVisitor extends AbstractSqlppSimpleExpressionVisitor {
 
@@ -74,9 +75,9 @@
      * @param externalVars
      *            pre-defined (external) variables that must be added to the initial scope
      */
-    public AbstractSqlppExpressionScopingVisitor(LangRewritingContext context, List<VarIdentifier> externalVars) {
+    public AbstractSqlppExpressionScopingVisitor(LangRewritingContext context, Collection<VarIdentifier> externalVars) {
         this.context = context;
-        this.scopeChecker.setVarCounter(new Counter(context.getVarCounter()));
+        this.scopeChecker.setVarCounter(context.getVarCounter());
         if (externalVars != null) {
             for (VarIdentifier paramVar : externalVars) {
                 scopeChecker.getCurrentScope().addSymbolToScope(paramVar);
@@ -92,7 +93,7 @@
 
             // Merges the variables defined in the current from term into the scope of the current from clause.
             Scope scopeForFromTerm = scopeChecker.removeCurrentScope();
-            mergeScopes(scopeForFromClause, scopeForFromTerm);
+            mergeScopes(scopeForFromClause, scopeForFromTerm, fromTerm.getSourceLocation());
         }
         return null;
     }
@@ -105,12 +106,12 @@
 
         // Registers the data item variable.
         VariableExpr leftVar = fromTerm.getLeftVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), leftVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), leftVar.getVar(), leftVar.getSourceLocation());
 
         // Registers the positional variable
         if (fromTerm.hasPositionalVariable()) {
             VariableExpr posVar = fromTerm.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
         // Visits join/unnest/nest clauses.
         for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
@@ -130,16 +131,16 @@
 
         // Registers the data item variable.
         VariableExpr rightVar = joinClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (joinClause.hasPositionalVariable()) {
             // Registers the positional variable.
             VariableExpr posVar = joinClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
 
         Scope rightScope = scopeChecker.removeCurrentScope();
-        mergeScopes(leftScope, rightScope);
+        mergeScopes(leftScope, rightScope, joinClause.getRightExpression().getSourceLocation());
         scopeChecker.pushExistingScope(leftScope);
         // The condition expression can refer to the just registered variables
         // for the right branch.
@@ -156,12 +157,12 @@
 
         // Registers the data item variable.
         VariableExpr rightVar = nestClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (nestClause.hasPositionalVariable()) {
             // Registers the positional variable.
             VariableExpr posVar = nestClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
 
         // The condition expression can refer to the just registered variables
@@ -176,12 +177,12 @@
 
         // register the data item variable
         VariableExpr rightVar = unnestClause.getRightVariable();
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar());
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), rightVar.getVar(), rightVar.getSourceLocation());
 
         if (unnestClause.hasPositionalVariable()) {
             // register the positional variable
             VariableExpr posVar = unnestClause.getPositionalVariable();
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), posVar.getVar(), posVar.getSourceLocation());
         }
         return null;
     }
@@ -212,7 +213,6 @@
     public Expression visit(Query q, ILangExpression arg) throws CompilationException {
         q.setBody(visit(q.getBody(), q));
         q.setVarCounter(scopeChecker.getVarCounter());
-        context.setVarCounter(scopeChecker.getVarCounter());
         return null;
     }
 
@@ -234,7 +234,7 @@
             gbyKeyVarExpr.setExpr(visit(gbyKeyVarExpr.getExpr(), gc));
             VariableExpr gbyKeyVar = gbyKeyVarExpr.getVar();
             if (gbyKeyVar != null) {
-                addNewVarSymbolToScope(newScope, gbyKeyVar.getVar());
+                addNewVarSymbolToScope(newScope, gbyKeyVar.getVar(), gbyKeyVar.getSourceLocation());
             }
         }
         if (gc.hasGroupFieldList()) {
@@ -247,12 +247,13 @@
                 decorVarExpr.setExpr(visit(decorVarExpr.getExpr(), gc));
                 VariableExpr decorVar = decorVarExpr.getVar();
                 if (decorVar != null) {
-                    addNewVarSymbolToScope(newScope, decorVar.getVar());
+                    addNewVarSymbolToScope(newScope, decorVar.getVar(), decorVar.getSourceLocation());
                 }
             }
         }
         if (gc.hasGroupVar()) {
-            addNewVarSymbolToScope(newScope, gc.getGroupVar().getVar());
+            VariableExpr groupVar = gc.getGroupVar();
+            addNewVarSymbolToScope(newScope, groupVar.getVar(), groupVar.getSourceLocation());
         }
         if (gc.hasWithMap()) {
             Map<Expression, VariableExpr> newWithMap = new HashMap<>();
@@ -260,7 +261,7 @@
                 Expression expr = visit(entry.getKey(), arg);
                 Expression newKey = expr;
                 VariableExpr value = entry.getValue();
-                addNewVarSymbolToScope(newScope, value.getVar());
+                addNewVarSymbolToScope(newScope, value.getVar(), value.getSourceLocation());
                 newWithMap.put(newKey, value);
             }
             gc.setWithVarMap(newWithMap);
@@ -285,7 +286,8 @@
     public Expression visit(LetClause letClause, ILangExpression arg) throws CompilationException {
         scopeChecker.extendCurrentScope();
         letClause.setBindingExpr(visit(letClause.getBindingExpr(), letClause));
-        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), letClause.getVarExpr().getVar());
+        VariableExpr varExpr = letClause.getVarExpr();
+        addNewVarSymbolToScope(scopeChecker.getCurrentScope(), varExpr.getVar(), varExpr.getSourceLocation());
         return null;
     }
 
@@ -327,7 +329,8 @@
         scopeChecker.createNewScope();
         for (QuantifiedPair pair : qe.getQuantifiedList()) {
             pair.setExpr(visit(pair.getExpr(), qe));
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), pair.getVarExpr().getVar());
+            VariableExpr varExpr = pair.getVarExpr();
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), varExpr.getVar(), varExpr.getSourceLocation());
         }
         qe.setSatisfiesExpr(visit(qe.getSatisfiesExpr(), qe));
         scopeChecker.removeCurrentScope();
@@ -338,7 +341,7 @@
     public Expression visit(VariableExpr varExpr, ILangExpression arg) throws CompilationException {
         String varName = varExpr.getVar().getValue();
         if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new CompilationException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, varExpr.getSourceLocation(),
                     "Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
         }
         Identifier ident = scopeChecker.lookupSymbol(varName);
@@ -361,7 +364,7 @@
         // Registers the (inserted) data item variable.
         VariableExpr bindingVar = insertStatement.getVar();
         if (bindingVar != null) {
-            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), bindingVar.getVar());
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), bindingVar.getVar(), bindingVar.getSourceLocation());
         }
 
         // Visits the expression for the returning expression.
@@ -373,20 +376,22 @@
     }
 
     // Adds a new encountered alias identifier into a scope
-    private void addNewVarSymbolToScope(Scope scope, VarIdentifier var) throws CompilationException {
+    private void addNewVarSymbolToScope(Scope scope, VarIdentifier var, SourceLocation sourceLoc)
+            throws CompilationException {
         if (scope.findLocalSymbol(var.getValue()) != null) {
-            throw new CompilationException(
+            throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     "Duplicate alias definitions: " + SqlppVariableUtil.toUserDefinedName(var.getValue()));
         }
         scope.addNewVarSymbolToScope(var);
     }
 
     // Merges <code>scopeToBeMerged</code> into <code>hostScope</code>.
-    private void mergeScopes(Scope hostScope, Scope scopeToBeMerged) throws CompilationException {
+    private void mergeScopes(Scope hostScope, Scope scopeToBeMerged, SourceLocation sourceLoc)
+            throws CompilationException {
         Set<String> symbolsToBeMerged = scopeToBeMerged.getLocalSymbols();
         for (String symbolToBeMerged : symbolsToBeMerged) {
             if (hostScope.findLocalSymbol(symbolToBeMerged) != null) {
-                throw new CompilationException(
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Duplicate alias definitions: " + SqlppVariableUtil.toUserDefinedName(symbolToBeMerged));
             }
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index b8b7622..e2a8759 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -67,9 +67,12 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractLangExpression;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -175,6 +178,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 class SQLPPParser extends ScopeChecker implements IParser {
 
@@ -205,6 +209,8 @@
     // error configuration
     protected static final boolean REPORT_EXPECTED_TOKENS = false;
 
+    private int externalVarCounter;
+
     private static class IndexParams {
       public IndexType type;
       public int gramLength;
@@ -216,13 +222,14 @@
     };
 
     private static class FunctionName {
-       public String dataverse = null;
-       public String library = null;
-       public String function = null;
-       public String hint = null;
+       public String dataverse;
+       public String library;
+       public String function;
+       public String hint;
+       public SourceLocation sourceLoc;
     }
 
-    private static String getHint(Token t) {
+    private String getHint(Token t) {
         if (t.specialToken == null) {
             return null;
         }
@@ -234,7 +241,11 @@
         return s.substring(1).trim();
     }
 
-    private static IRecordFieldDataGen parseFieldDataGen(String hint) throws ParseException {
+    private Token getHintToken(Token t) {
+        return t.specialToken;
+    }
+
+    private IRecordFieldDataGen parseFieldDataGen(String hint, Token hintToken) throws ParseException {
       IRecordFieldDataGen rfdg = null;
       String splits[] = hint.split(" +");
       if (splits[0].equals(VAL_FILE_HINT)) {
@@ -260,7 +271,7 @@
         } else if (splits[1].equals("double")) {
           vt = FieldIntervalDataGen.ValueType.DOUBLE;
         } else {
-          throw new ParseException("Unknown type for interval data gen: " + splits[1]);
+          throw new SqlppParseException(getSourceLocation(hintToken), "Unknown type for interval data gen: " + splits[1]);
         }
         rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
       } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
@@ -294,10 +305,12 @@
         try {
             return Statement();
         } catch (Error e) {
-            // this is here as the JavaCharStream that's below the lexer somtimes throws Errors that are not handled
+            // this is here as the JavaCharStream that's below the lexer sometimes throws Errors that are not handled
             // by the ANTLR-generated lexer or parser (e.g it does this for invalid backslash u + 4 hex digits escapes)
             final String msg = e.getClass().getSimpleName() + (e.getMessage() != null ? ": " + e.getMessage() : "");
             throw new CompilationException(ErrorCode.PARSE_ERROR, msg);
+        } catch (SqlppParseException e) {
+            throw new CompilationException(ErrorCode.PARSE_ERROR, e.getSourceLocation(), getMessage(e));
         } catch (ParseException e) {
             throw new CompilationException(ErrorCode.PARSE_ERROR, getMessage(e));
         }
@@ -315,34 +328,43 @@
         int maxSize = appendExpected(expected, expectedTokenSequences, tokenImage);
         Token tok = currentToken.next;
         int line = tok.beginLine;
-        String message = "In line " + line + " >>" + getLine(line) + "<<" + sep + "Encountered ";
+        StringBuilder message = new StringBuilder(128);
+        message.append("In line ").append(line).append(" >>").append(getLine(line)).append("<<").append(sep).append("Encountered ");
         for (int i = 0; i < maxSize; i++) {
             if (i != 0) {
-                message += " ";
+                message.append(' ');
             }
             if (tok.kind == 0) {
-                message += fixQuotes(tokenImage[0]);
+                message.append(fixQuotes(tokenImage[0]));
                 break;
             }
             final String fixedTokenImage = tokenImage[tok.kind];
             if (! tok.image.equalsIgnoreCase(stripQuotes(fixedTokenImage))) {
-                message += fixQuotes(fixedTokenImage) + " ";
+                message.append(fixQuotes(fixedTokenImage)).append(' ');
             }
-            message += quot + addEscapes(tok.image) + quot;
+            message.append(quot).append(addEscapes(tok.image)).append(quot);
             tok = tok.next;
         }
-        message += " at column " + currentToken.next.beginColumn + "." + sep;
+        message.append(" at column ").append(currentToken.next.beginColumn).append('.').append(sep);
         if (REPORT_EXPECTED_TOKENS) {
             if (expectedTokenSequences.length == 1) {
-                message += "Was expecting:" + sep + "    ";
+                message.append("Was expecting:").append(sep).append("    ");
             } else {
-                message += "Was expecting one of:" + sep + "    ";
+                message.append("Was expecting one of:").append(sep).append("    ");
             }
-            message += expected.toString();
+            message.append(expected);
         }
-        return message;
+        return message.toString();
     }
 
+    protected static SourceLocation getSourceLocation(Token token) {
+        return token != null ? new SourceLocation(token.beginLine, token.beginColumn) : null;
+    }
+
+    protected static <T extends AbstractLangExpression> T addSourceLocation(T expr, Token token) {
+        expr.setSourceLocation(getSourceLocation(token));
+        return expr;
+    }
 }
 
 PARSER_END(SQLPPParser)
@@ -399,46 +421,51 @@
 
 DataverseDecl DataverseDeclaration() throws ParseException:
 {
+  Token startToken = null;
   String dvName = null;
 }
 {
-  <USE> dvName = Identifier()
+  <USE> { startToken = token; } dvName = Identifier()
     {
       defaultDataverse = dvName;
-      return new DataverseDecl(new Identifier(dvName));
+      DataverseDecl dvDecl = new DataverseDecl(new Identifier(dvName));
+      return addSourceLocation(dvDecl, startToken);
     }
 }
 
 Statement CreateStatement() throws ParseException:
 {
+  Token startToken = null;
   String hint = null;
-  boolean dgen = false;
+  Token hintToken = null;
+  boolean hintDGen = false;
   Statement stmt = null;
 }
 {
-  <CREATE>
+  <CREATE> { startToken = token; }
   (
     {
       hint = getHint(token);
-      if (hint != null && hint.startsWith(DGEN_HINT)) {
-        dgen = true;
+      if (hint != null) {
+        hintToken = getHintToken(token);
+        hintDGen = hint.startsWith(DGEN_HINT);
       }
     }
-    stmt = TypeSpecification(hint, dgen)
-    | stmt = NodegroupSpecification()
-    | stmt = DatasetSpecification()
-    | stmt = IndexSpecification()
-    | stmt = DataverseSpecification()
-    | stmt = FunctionSpecification()
-    | stmt = FeedSpecification()
-    | stmt = FeedPolicySpecification()
+    stmt = TypeSpecification(startToken, hint, hintDGen, hintToken)
+    | stmt = NodegroupSpecification(startToken)
+    | stmt = DatasetSpecification(startToken)
+    | stmt = IndexSpecification(startToken)
+    | stmt = DataverseSpecification(startToken)
+    | stmt = FunctionSpecification(startToken)
+    | stmt = FeedSpecification(startToken)
+    | stmt = FeedPolicySpecification(startToken)
   )
   {
     return stmt;
   }
 }
 
-TypeDecl TypeSpecification(String hint, boolean dgen) throws ParseException:
+TypeDecl TypeSpecification(Token startStmtToken, String hint, boolean dgen, Token hintToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
@@ -453,18 +480,18 @@
       if (dgen) {
         String splits[] = hint.split(" +");
         if (splits.length != 3) {
-          throw new ParseException("Expecting /*+ dgen <filename> <numberOfItems> */");
+          throw new SqlppParseException(getSourceLocation(hintToken), "Expecting /*+ dgen <filename> <numberOfItems> */");
         }
         filename = splits[1];
         numValues = Long.parseLong(splits[2]);
       }
       TypeDataGen tddg = new TypeDataGen(dgen, filename, numValues);
-      return new TypeDecl(nameComponents.first, nameComponents.second, typeExpr, tddg, ifNotExists);
+      TypeDecl stmt = new TypeDecl(nameComponents.first, nameComponents.second, typeExpr, tddg, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-
-NodegroupDecl NodegroupSpecification() throws ParseException:
+NodegroupDecl NodegroupSpecification(Token startStmtToken) throws ParseException:
 {
   String name = null;
   String tmp = null;
@@ -484,11 +511,12 @@
     }
   )*
     {
-      return new NodegroupDecl(new Identifier(name), ncNames, ifNotExists);
+      NodegroupDecl stmt = new NodegroupDecl(new Identifier(name), ncNames, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-DatasetDecl DatasetSpecification() throws ParseException:
+DatasetDecl DatasetSpecification(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
@@ -499,7 +527,7 @@
   Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
   String nodeGroupName = null;
   Map<String,String> hints = new HashMap<String,String>();
-  DatasetDecl dsetDecl = null;
+  DatasetDecl stmt = null;
   boolean autogenerated = false;
   Pair<Integer, List<String>> filterField = null;
   Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
@@ -519,7 +547,7 @@
         edd.setAdapter(adapterName);
         edd.setProperties(properties);
         try{
-        dsetDecl = new DatasetDecl(nameComponents.first,
+        stmt = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
@@ -532,7 +560,7 @@
                                    withRecord,
                                    ifNotExists);
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
         }
       }
 
@@ -544,8 +572,9 @@
         <WITH>
         name = Identifier()
         {
-            if(!name.toLowerCase().equals("meta")){
-                throw new ParseException("We can only support one additional associated field called \"meta\".");
+            if (!name.equalsIgnoreCase("meta")){
+                throw new SqlppParseException(getSourceLocation(startStmtToken),
+                    "We can only support one additional associated field called \"meta\".");
             }
         }
         <LEFTPAREN> metaTypeComponents = TypeName() <RIGHTPAREN>
@@ -559,14 +588,15 @@
     ( <WITH> withRecord = RecordConstructor() )?
       {
         if(filterField!=null && filterField.first!=0){
-          throw new ParseException("A filter field can only be a field in the main record of the dataset.");
+          throw new SqlppParseException(getSourceLocation(startStmtToken),
+            "A filter field can only be a field in the main record of the dataset.");
         }
         InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second,
                                                           primaryKeyFields.first,
                                                           autogenerated,
                                                           filterField == null? null : filterField.second);
         try{
-        dsetDecl = new DatasetDecl(nameComponents.first,
+        stmt = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    typeComponents.first,
                                    typeComponents.second,
@@ -579,33 +609,34 @@
                                    withRecord,
                                    ifNotExists);
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
         }
       }
   )
     {
-      return dsetDecl;
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
 RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
 {
-  RefreshExternalDatasetStatement redss = new RefreshExternalDatasetStatement();
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   String datasetName = null;
 }
 {
-    <REFRESH> <EXTERNAL> Dataset() nameComponents = QualifiedName()
+    <REFRESH> { startToken = token; } <EXTERNAL> Dataset() nameComponents = QualifiedName()
     {
-    redss.setDataverseName(nameComponents.first);
-    redss.setDatasetName(nameComponents.second);
-    return redss;
+      RefreshExternalDatasetStatement stmt = new RefreshExternalDatasetStatement();
+      stmt.setDataverseName(nameComponents.first);
+      stmt.setDatasetName(nameComponents.second);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
-CreateIndexStatement IndexSpecification() throws ParseException:
+CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseException:
 {
-  CreateIndexStatement cis = new CreateIndexStatement();
+  CreateIndexStatement stmt = new CreateIndexStatement();
   String indexName = null;
   boolean ifNotExists = false;
   Pair<Identifier,Identifier> nameComponents = null;
@@ -621,13 +652,13 @@
     <ON> nameComponents = QualifiedName()
     <LEFTPAREN> ( fieldPair = OpenField()
       {
-        cis.addFieldExprPair(fieldPair.second);
-        cis.addFieldIndexIndicator(fieldPair.first);
+        stmt.addFieldExprPair(fieldPair.second);
+        stmt.addFieldIndexIndicator(fieldPair.first);
       }
     ) (<COMMA> fieldPair = OpenField()
       {
-        cis.addFieldExprPair(fieldPair.second);
-        cis.addFieldIndexIndicator(fieldPair.first);
+        stmt.addFieldExprPair(fieldPair.second);
+        stmt.addFieldIndexIndicator(fieldPair.first);
       }
     )* <RIGHTPAREN> ( <TYPE> indexType = IndexType() )? ( <ENFORCED> { enforced = true; } )?)
     |
@@ -642,16 +673,16 @@
     if (isPrimaryIdx && indexName == null) {
       indexName = "primary_idx_" + nameComponents.second;
     }
-    cis.setIndexName(new Identifier(indexName));
-    cis.setIfNotExists(ifNotExists);
-    cis.setDataverseName(nameComponents.first);
-    cis.setDatasetName(nameComponents.second);
+    stmt.setIndexName(new Identifier(indexName));
+    stmt.setIfNotExists(ifNotExists);
+    stmt.setDataverseName(nameComponents.first);
+    stmt.setDatasetName(nameComponents.second);
     if (indexType != null) {
-      cis.setIndexType(indexType.type);
-      cis.setGramLength(indexType.gramLength);
+      stmt.setIndexType(indexType.type);
+      stmt.setGramLength(indexType.gramLength);
     }
-    cis.setEnforced(enforced);
-    return cis;
+    stmt.setEnforced(enforced);
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
@@ -710,7 +741,7 @@
     }
 }
 
-CreateDataverseStatement DataverseSpecification() throws ParseException :
+CreateDataverseStatement DataverseSpecification(Token startStmtToken) throws ParseException :
 {
   String dvName = null;
   boolean ifNotExists = false;
@@ -719,11 +750,12 @@
   <DATAVERSE> dvName = Identifier()
   ifNotExists = IfNotExists()
     {
-      return new CreateDataverseStatement(new Identifier(dvName), null, ifNotExists);
+      CreateDataverseStatement stmt = new CreateDataverseStatement(new Identifier(dvName), null, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-CreateFunctionStatement FunctionSpecification() throws ParseException:
+CreateFunctionStatement FunctionSpecification(Token startStmtToken) throws ParseException:
 {
   FunctionSignature signature;
   boolean ifNotExists = false;
@@ -759,17 +791,18 @@
       getCurrentScope().addFunctionDescriptor(signature, false);
       removeCurrentScope();
       defaultDataverse = currentDataverse;
-      return new CreateFunctionStatement(signature, paramList, functionBody, functionBodyExpr, ifNotExists);
+      CreateFunctionStatement stmt = new CreateFunctionStatement(signature, paramList, functionBody, functionBodyExpr, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-CreateFeedStatement FeedSpecification() throws ParseException:
+CreateFeedStatement FeedSpecification(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   String adapterName = null;
   Map<String,String> properties = null;
-  CreateFeedStatement cfs = null;
+  CreateFeedStatement stmt = null;
   Pair<Identifier,Identifier> sourceNameComponents = null;
   RecordConstructor withRecord = null;
 }
@@ -778,15 +811,15 @@
   <WITH> withRecord = RecordConstructor()
   {
     try {
-      cfs = new CreateFeedStatement(nameComponents, withRecord, ifNotExists);
-      return cfs;
+      stmt = new CreateFeedStatement(nameComponents, withRecord, ifNotExists);
+      return addSourceLocation(stmt, startStmtToken);
     } catch (AlgebricksException e) {
-      throw new ParseException(e.getMessage());
+      throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
     }
   }
 }
 
-CreateFeedPolicyStatement FeedPolicySpecification() throws ParseException:
+CreateFeedPolicyStatement FeedPolicySpecification(Token startStmtToken) throws ParseException:
 {
   String policyName = null;
   String basePolicyName = null;
@@ -794,7 +827,7 @@
   String definition = null;
   boolean ifNotExists = false;
   Map<String,String> properties = null;
-  CreateFeedPolicyStatement cfps = null;
+  CreateFeedPolicyStatement stmt = null;
 }
 {
   (
@@ -802,22 +835,20 @@
       <FROM>
       (<POLICY> basePolicyName = Identifier() properties = Configuration() (<DEFINITION> definition = ConstantString())?
       {
-        cfps = new CreateFeedPolicyStatement(policyName,
+        stmt = new CreateFeedPolicyStatement(policyName,
                                    basePolicyName, properties, definition, ifNotExists);
       }
      | <PATH> sourcePolicyFile = ConstantString() (<DEFINITION> definition = ConstantString())?
        {
-        cfps = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
+        stmt = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
        }
      )
   )
     {
-      return cfps;
+      return addSourceLocation(stmt, startStmtToken);
     }
 }
 
-
-
 List<VarIdentifier> ParameterList() throws ParseException:
 {
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
@@ -896,7 +927,7 @@
     {
       arity = new Integer(token.image);
       if (arity < 0 && arity != FunctionIdentifier.VARARGS) {
-        throw new ParseException(" invalid arity:" + arity);
+        throw new SqlppParseException(getSourceLocation(token), "Invalid arity:" + arity);
       }
 
       // TODO use fctName.library
@@ -930,15 +961,16 @@
 
 Statement DropStatement() throws ParseException:
 {
+  Token startToken = null;
   String id = null;
   Pair<Identifier,Identifier> pairId = null;
   Triple<Identifier,Identifier,Identifier> tripleId = null;
   FunctionSignature funcSig = null;
   boolean ifExists = false;
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
-  <DROP>
+  <DROP> { startToken = token; }
   (
     Dataset() pairId = QualifiedName() ifExists = IfExists()
       {
@@ -974,7 +1006,7 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startToken);
   }
 }
 
@@ -994,54 +1026,59 @@
 
 InsertStatement InsertStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
 }
 {
-  <INSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+  <INSERT> { startToken = token; } <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
     query = Query(false)
     ( <RETURNING> returnExpression = Expression())?
     {
       if (returnExpression != null && var == null) {
-         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+        var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
       }
       query.setTopLevel(true);
-      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
-                                 returnExpression);
+      InsertStatement stmt = new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(),
+        var, returnExpression);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 UpsertStatement UpsertStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
 }
 {
-  <UPSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+  <UPSERT> { startToken = token; } <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
     query = Query(false)
     ( <RETURNING> returnExpression = Expression())?
     {
       if (returnExpression != null && var == null) {
-         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+        var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
       }
       query.setTopLevel(true);
-      return new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
-                                 returnExpression);
+      UpsertStatement stmt = new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(),
+        var, returnExpression);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 DeleteStatement DeleteStatement() throws ParseException:
 {
+  Token startToken = null;
   VariableExpr varExpr = null;
   Expression condition = null;
   Pair<Identifier, Identifier> nameComponents;
 }
 {
-  <DELETE>
+  <DELETE> { startToken = token; }
   <FROM> nameComponents  = QualifiedName()
          ((<AS>)? varExpr = Variable())?
   (<WHERE> condition = Expression())?
@@ -1050,14 +1087,17 @@
         varExpr = new VariableExpr();
         VarIdentifier var = SqlppVariableUtil.toInternalVariableIdentifier(nameComponents.second.getValue());
         varExpr.setVar(var);
+        addSourceLocation(varExpr, startToken);
       }
-      return new DeleteStatement(varExpr, nameComponents.first, nameComponents.second,
+      DeleteStatement stmt = new DeleteStatement(varExpr, nameComponents.first, nameComponents.second,
           condition, getVarCounter());
+      return addSourceLocation(stmt, startToken);
   }
 }
 
 UpdateStatement UpdateStatement() throws ParseException:
 {
+  Token startToken = null;
   VariableExpr vars;
   Expression target;
   Expression condition;
@@ -1065,7 +1105,7 @@
   List<UpdateClause> ucs = new ArrayList<UpdateClause>();
 }
 {
-  <UPDATE> vars = Variable() <IN> target = Expression()
+  <UPDATE> { startToken = token; } vars = Variable() <IN> target = Expression()
   <WHERE> condition = Expression()
   <LEFTPAREN> (uc = UpdateClause()
     {
@@ -1077,7 +1117,8 @@
     }
   )*) <RIGHTPAREN>
     {
-      return new UpdateStatement(vars, target, condition, ucs);
+      UpdateStatement stmt = new UpdateStatement(vars, target, condition, ucs);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
@@ -1108,18 +1149,21 @@
 
 Statement SetStatement() throws ParseException:
 {
+  Token startToken = null;
   String pn = null;
   String pv = null;
 }
 {
-  <SET> pn = Identifier() pv = ConstantString()
+  <SET> { startToken = token; } pn = Identifier() pv = ConstantString()
     {
-      return new SetStatement(pn, pv);
+      SetStatement stmt = new SetStatement(pn, pv);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 Statement WriteStatement() throws ParseException:
 {
+  Token startToken = null;
   String nodeName = null;
   String fileName = null;
   Query query;
@@ -1127,15 +1171,17 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-  <WRITE> <OUTPUT> <TO> nodeName = Identifier() <COLON> fileName = ConstantString()
+  <WRITE> { startToken = token; } <OUTPUT> <TO> nodeName = Identifier() <COLON> fileName = ConstantString()
     ( <USING> writerClass = ConstantString() )?
     {
-      return new WriteStatement(new Identifier(nodeName), fileName, writerClass);
+      WriteStatement stmt = new WriteStatement(new Identifier(nodeName), fileName, writerClass);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 LoadStatement LoadStatement() throws ParseException:
 {
+  Token startToken = null;
   Identifier dataverseName = null;
   Identifier datasetName = null;
   boolean alreadySorted = false;
@@ -1144,7 +1190,7 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-  <LOAD> Dataset() nameComponents = QualifiedName()
+  <LOAD> { startToken = token; } Dataset() nameComponents = QualifiedName()
     {
       dataverseName = nameComponents.first;
       datasetName = nameComponents.second;
@@ -1156,7 +1202,8 @@
     }
   )?
     {
-      return new LoadStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
+      LoadStatement stmt = new LoadStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
@@ -1174,77 +1221,68 @@
 
 Statement CompactStatement() throws ParseException:
 {
+  Token startToken = null;
   Pair<Identifier,Identifier> nameComponents = null;
-  Statement stmt = null;
 }
 {
-  <COMPACT> Dataset() nameComponents = QualifiedName()
+  <COMPACT> { startToken = token; } Dataset() nameComponents = QualifiedName()
     {
-      stmt = new CompactStatement(nameComponents.first, nameComponents.second);
-    }
-    {
-      return stmt;
+      CompactStatement stmt = new CompactStatement(nameComponents.first, nameComponents.second);
+      return addSourceLocation(stmt, startToken);
     }
 }
 
 Statement ConnectionStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
-  Pair<Identifier,Identifier> datasetNameComponents = null;
-
-  Map<String,String> configuration = null;
+  Token startToken = null;
   Statement stmt = null;
-  String policy = null;
 }
 {
   (
-    <CONNECT> stmt = ConnectStatement()
-  | <DISCONNECT> stmt = DisconnectStatement()
-  | <START> stmt = StartStatement()
-  | <STOP> stmt = StopStatement()
+    <CONNECT> { startToken = token; } stmt = ConnectStatement(startToken)
+  | <DISCONNECT> { startToken = token; } stmt = DisconnectStatement(startToken)
+  | <START> { startToken = token; } stmt = StartStatement(startToken)
+  | <STOP> { startToken = token; } stmt = StopStatement(startToken)
   )
   {
     return stmt;
   }
 }
 
-Statement StartStatement() throws ParseException:
+Statement StartStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
 
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
   <FEED> feedNameComponents = QualifiedName()
   {
     stmt = new StartFeedStatement (feedNameComponents);
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-Statement StopStatement () throws ParseException:
+AbstractStatement StopStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
 
-  Statement stmt = null;
+  AbstractStatement stmt = null;
 }
 {
   <FEED> feedNameComponents = QualifiedName()
   {
     stmt = new StopFeedStatement (feedNameComponents);
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-
-Statement DisconnectStatement() throws ParseException:
+AbstractStatement DisconnectStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
-  Map<String,String> configuration = null;
-  Statement stmt = null;
-  String policy = null;
+  AbstractStatement stmt = null;
 }
 {
   (
@@ -1254,18 +1292,18 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
-Statement ConnectStatement() throws ParseException:
+AbstractStatement ConnectStatement(Token startStmtToken) throws ParseException:
 {
   Pair<Identifier,Identifier> feedNameComponents = null;
   Pair<Identifier,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
   List<FunctionSignature> appliedFunctions = new ArrayList<FunctionSignature>();
-  Statement stmt = null;
+  AbstractStatement stmt = null;
   String policy = null;
   String whereClauseBody = null;
   WhereClause whereClause = null;
@@ -1301,7 +1339,7 @@
       }
   )
   {
-    return stmt;
+    return addSourceLocation(stmt, startStmtToken);
   }
 }
 
@@ -1368,9 +1406,9 @@
   ( value = ConstantString() | <INTEGER_LITERAL>
     {
       try {
-        value = "" + Long.valueOf(token.image);
+        value = String.valueOf(Long.parseLong(token.image));
       } catch (NumberFormatException nfe) {
-        throw new ParseException("inapproriate value: " + token.image);
+        throw new SqlppParseException(getSourceLocation(token), "inapproriate value: " + token.image);
       }
     }
   )
@@ -1414,6 +1452,7 @@
 
 RecordTypeDefinition RecordTypeDef() throws ParseException:
 {
+  Token startToken = null;
   RecordTypeDefinition recType = new RecordTypeDefinition();
   RecordTypeDefinition.RecordKind recordKind = null;
 }
@@ -1422,15 +1461,18 @@
     | <OPEN> { recordKind = RecordTypeDefinition.RecordKind.OPEN; } )?
    <LEFTBRACE>
     {
+      startToken = token;
       String hint = getHint(token);
       if (hint != null) {
         String splits[] = hint.split(" +");
         if (splits[0].equals(GEN_FIELDS_HINT)) {
           if (splits.length != 5) {
-            throw new ParseException("Expecting: /*+ gen-fields <type> <min> <max> <prefix>*/");
+            throw new SqlppParseException(getSourceLocation(getHintToken(token)),
+                "Expecting: /*+ gen-fields <type> <min> <max> <prefix>*/");
           }
           if (!splits[1].equals("int")) {
-            throw new ParseException("The only supported type for gen-fields is int.");
+            throw new SqlppParseException(getSourceLocation(getHintToken(token)),
+                "The only supported type for gen-fields is int.");
           }
           UndeclaredFieldsDataGen ufdg = new UndeclaredFieldsDataGen(UndeclaredFieldsDataGen.Type.INT,
              Integer.parseInt(splits[2]), Integer.parseInt(splits[3]), splits[4]);
@@ -1449,7 +1491,7 @@
         recordKind = RecordTypeDefinition.RecordKind.OPEN;
       }
       recType.setRecordKind(recordKind);
-      return recType;
+      return addSourceLocation(recType, startToken);
    }
 }
 
@@ -1463,7 +1505,7 @@
   fieldName = Identifier()
     {
       String hint = getHint(token);
-      IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint) : null;
+      IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint, token.specialToken) : null;
     }
   <COLON> type =  TypeExpr() (<QUES> { nullable = true; } )?
     {
@@ -1476,40 +1518,44 @@
   Pair<Identifier,Identifier> id = null;
 }
 {
- id = QualifiedName()
-   {
-     if (id.first == null && id.second.getValue().equalsIgnoreCase("int")) {
-        id.second.setValue("int64");
-     }
+  id = QualifiedName()
+  {
+    if (id.first == null && id.second.getValue().equalsIgnoreCase("int")) {
+      id.second.setValue("int64");
+    }
 
-     return new TypeReferenceExpression(id);
-   }
+    TypeReferenceExpression typeRef = new TypeReferenceExpression(id);
+    return addSourceLocation(typeRef, token);
+  }
 }
 
 OrderedListTypeDefinition OrderedListTypeDef() throws ParseException:
 {
+  Token startToken = null;
   TypeExpression type = null;
 }
 {
-  <LEFTBRACKET>
-    ( type =  TypeExpr() )
+  <LEFTBRACKET> { startToken = token; }
+    ( type = TypeExpr() )
   <RIGHTBRACKET>
   {
-    return new OrderedListTypeDefinition(type);
+    OrderedListTypeDefinition typeDef = new OrderedListTypeDefinition(type);
+    return addSourceLocation(typeDef, startToken);
   }
 }
 
-
 UnorderedListTypeDefinition UnorderedListTypeDef() throws ParseException:
 {
+  Token startToken = null;
   TypeExpression type = null;
 }
 {
-  <LEFTDBLBRACE>
+  <LEFTDBLBRACE> { startToken = token; }
     ( type =  TypeExpr() )
   <RIGHTDBLBRACE>
   {
-    return new UnorderedListTypeDefinition(type);
+    UnorderedListTypeDefinition typeDef = new UnorderedListTypeDefinition(type);
+    return addSourceLocation(typeDef, startToken);
   }
 }
 
@@ -1525,6 +1571,7 @@
   {
     FunctionName result = new FunctionName();
     result.hint = getHint(token);
+    result.sourceLoc = getSourceLocation(token);
   }
   ( <DOT> second = Identifier()
     {
@@ -1614,19 +1661,21 @@
 {
   List<String> exprList = new ArrayList<String>();
   String lit = null;
+  Token litToken = null;
   int source = 0;
 }
 {
   lit = Identifier()
   {
     boolean meetParens = false;
+    litToken = token;
   }
   (
     LOOKAHEAD(1)
     <LEFTPAREN><RIGHTPAREN>
     {
-        if(!lit.toLowerCase().equals("meta")){
-            throw new ParseException("The string before () has to be \"meta\".");
+        if(!lit.equalsIgnoreCase("meta")){
+            throw new SqlppParseException(getSourceLocation(litToken), "The string before () has to be \"meta\".");
         }
         meetParens = true;
         source = 1;
@@ -1659,7 +1708,6 @@
   }
 }
 
-
 String QuotedString() throws ParseException:
 {
 }
@@ -1670,7 +1718,6 @@
     }
 }
 
-
 String StringLiteral() throws ParseException:
 {
 }
@@ -1728,26 +1775,25 @@
 
 FunctionDecl FunctionDeclaration() throws ParseException:
 {
-  FunctionDecl funcDecl;
-  FunctionSignature signature;
+  Token startToken = null;
   String functionName;
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
   Expression funcBody;
   createNewScope();
 }
 {
-  <DECLARE> <FUNCTION>
+  <DECLARE> { startToken = token; } <FUNCTION>
     functionName = Identifier()
     paramList = ParameterList()
   <LEFTBRACE>
     (funcBody = SelectExpression(true) | funcBody = Expression())
   <RIGHTBRACE>
   {
-    signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
+    FunctionSignature signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
     getCurrentScope().addFunctionDescriptor(signature, false);
-    funcDecl = new FunctionDecl(signature, paramList, funcBody);
+    FunctionDecl stmt = new FunctionDecl(signature, paramList, funcBody);
     removeCurrentScope();
-    return funcDecl;
+    return addSourceLocation(stmt, startToken);
   }
 }
 
@@ -1775,12 +1821,12 @@
   )
   {
     query.setBody(expr);
+    query.setSourceLocation(expr.getSourceLocation());
     return query;
   }
 }
 
 
-
 Expression Expression():
 {
   Expression expr = null;
@@ -1790,7 +1836,6 @@
 (
     LOOKAHEAD(2)
     expr = OperatorExpr()
-    | expr = CaseExpr()
     | expr = QuantifiedExpression()
 )
     {
@@ -1798,9 +1843,7 @@
     }
 }
 
-
-
-Expression OperatorExpr()throws ParseException:
+Expression OperatorExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -1808,18 +1851,18 @@
 {
     operand = AndExpr()
     (
-
       <OR>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
         try{
             op.addOperator(token.image.toLowerCase());
         } catch (Exception e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
       }
 
@@ -1835,7 +1878,7 @@
     }
 }
 
-Expression AndExpr()throws ParseException:
+Expression AndExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -1843,18 +1886,18 @@
 {
     operand = NotExpr()
     (
-
       <AND>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
         try{
            op.addOperator(token.image.toLowerCase());
         } catch (CompilationException e){
-           throw new ParseException(e.getMessage());
+           throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
     }
 
@@ -1866,28 +1909,30 @@
     )*
 
     {
-      return op==null? operand: op;
+      return op==null ? operand: op;
     }
 }
 
-Expression NotExpr()throws ParseException:
+Expression NotExpr() throws ParseException:
 {
    Expression inputExpr;
    boolean not = false;
+   Token startToken = null;
 }
 {
-  (<NOT> { not = true; } )? inputExpr = RelExpr()
+  (<NOT> { not = true; startToken = token; } )? inputExpr = RelExpr()
   {
-    if(not){
+    if(not) {
         FunctionSignature signature = new FunctionSignature(BuiltinFunctions.NOT);
-        return new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(inputExpr)));
+        CallExpr callExpr = new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(inputExpr)));
+        return addSourceLocation(callExpr, startToken);
     } else {
         return inputExpr;
     }
   }
 }
 
-Expression RelExpr()throws ParseException:
+Expression RelExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -1923,11 +1968,12 @@
             op = new OperatorExpr();
             op.addOperand(operand, false); // broadcast is always for the right branch
             op.setCurrentop(true);
+            addSourceLocation(op, token);
           }
           try{
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -1945,8 +1991,7 @@
      }
 }
 
-
-Expression BetweenExpr()throws ParseException:
+Expression BetweenExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -1975,11 +2020,12 @@
             op = new OperatorExpr();
             op.addOperand(operand);
             op.setCurrentop(true);
+            addSourceLocation(op, token);
           }
           try{
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -2000,13 +2046,14 @@
        if (annotation != null) {
          op.addHint(annotation);
        }
-       return op==null? operand: op;
+       return op==null ? operand: op;
     }
 }
 
 Expression IsExpr() throws ParseException:
 {
-    Expression expr = null;
+    Token notToken = null;
+    CallExpr expr = null;
     Expression operand = null;
     boolean not = false;
     FunctionIdentifier fn = null;
@@ -2014,29 +2061,30 @@
 {
     operand = LikeExpr()
     ( <IS>
-        (<NOT> { not = true; })?
+        (<NOT> { not = true; notToken = token; })?
         (
             <NULL> { fn = BuiltinFunctions.IS_NULL; } |
             <MISSING> { fn = BuiltinFunctions.IS_MISSING; } |
             <UNKNOWN> { fn = BuiltinFunctions.IS_UNKNOWN; } |
-            <VALUED> { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
+            (<KNOWN> | <VALUED>) { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
         )
       {
         FunctionSignature signature = new FunctionSignature(fn);
         expr = new CallExpr(signature, new ArrayList<Expression>(Collections.singletonList(operand)));
-        if(not) {
+        addSourceLocation(expr, token);
+        if (not) {
            FunctionSignature notSignature = new FunctionSignature(BuiltinFunctions.NOT);
            expr = new CallExpr(notSignature, new ArrayList<Expression>(Collections.singletonList(expr)));
+           addSourceLocation(expr, notToken);
         }
       }
     )?
     {
-        return expr = expr==null? operand : expr;
+        return expr == null ? operand : expr;
     }
 }
 
-
-Expression LikeExpr()throws ParseException:
+Expression LikeExpr() throws ParseException:
 {
   boolean not = false;
   OperatorExpr op = null;
@@ -2051,15 +2099,16 @@
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
 
           String operator = token.image.toLowerCase();
           if (not) {
             operator = "not_" + operator;
           }
-          try{
+          try {
             op.addOperator(operator);
           } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
         }
 
@@ -2074,7 +2123,7 @@
      }
 }
 
-Expression ConcatExpr()throws ParseException:
+Expression ConcatExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -2083,18 +2132,15 @@
     operand = AddExpr()
     (
       LOOKAHEAD(1)
-      (<CONCAT>)
+      <CONCAT>
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            ((OperatorExpr)op).addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(OperatorType.CONCAT);
     }
     operand = AddExpr()
     {
@@ -2103,31 +2149,29 @@
     )*
 
     {
-       return op==null? operand: op;
+       return op == null ? operand : op;
     }
 }
 
-Expression AddExpr()throws ParseException:
+Expression AddExpr() throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = MultExpr()
     (
       LOOKAHEAD(1)
-      (<PLUS> | <MINUS>)
+      (<PLUS> { opType = OperatorType.PLUS; } | <MINUS> { opType = OperatorType.MINUS; } )
       {
         if (op == null) {
           op = new OperatorExpr();
-        op.addOperand(operand);
-        op.setCurrentop(true);
+          op.addOperand(operand);
+          op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            ((OperatorExpr)op).addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
 
     operand = MultExpr()
@@ -2137,30 +2181,33 @@
     )*
 
     {
-       return op==null? operand: op;
+       return op == null ? operand : op;
     }
 }
 
-Expression MultExpr()throws ParseException:
+Expression MultExpr() throws ParseException:
 {
   OperatorExpr op = null;
+  OperatorType opType = null;
   Expression operand = null;
 }
 {
     operand = ExponentExpr()
 
-    (( <MUL> | <DIV> | <MOD> | <IDIV>)
+    ( (
+        <MUL> { opType = OperatorType.MUL; } |
+        <DIVIDE> { opType = OperatorType.DIVIDE; } |
+        <DIV> { opType = OperatorType.DIV; } |
+        ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+      )
       {
         if (op == null) {
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            op.addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(opType);
     }
     operand = ExponentExpr()
     {
@@ -2169,11 +2216,11 @@
     )*
 
      {
-       return op==null?operand:op;
+       return op == null ? operand : op;
      }
 }
 
-Expression ExponentExpr()throws ParseException:
+Expression ExponentExpr() throws ParseException:
 {
   OperatorExpr op = null;
   Expression operand = null;
@@ -2186,12 +2233,9 @@
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
+          addSourceLocation(op, token);
         }
-        try{
-            op.addOperator(token.image);
-        } catch (Exception e){
-            throw new ParseException(e.getMessage());
-        }
+        op.addOperator(OperatorType.CARET);
     }
     operand = UnaryExpr()
     {
@@ -2199,7 +2243,7 @@
     }
     )?
     {
-       return op==null?operand:op;
+       return op == null ? operand : op;
     }
 }
 
@@ -2213,101 +2257,96 @@
     ( (<PLUS> | <MINUS> | (<NOT> { not = true; } )? <EXISTS> )
     {
         String exprType = token.image.toLowerCase();
-        if(not){
+        if (not) {
            exprType = "not_" + exprType;
         }
         uexpr = new UnaryExpr();
-        try{
+        addSourceLocation(uexpr, token);
+        try {
             uexpr.setExprType(exprType);
         } catch (CompilationException e){
-            throw new ParseException(e.getMessage());
+            throw new SqlppParseException(getSourceLocation(token), e.getMessage());
         }
     }
     )?
 
     expr = ValueExpr()
     {
-       if(uexpr==null){
-            return expr;
+       if (uexpr == null) {
+         return expr;
+       } else {
+         uexpr.setExpr(expr);
+         return uexpr;
        }
-       uexpr.setExpr(expr);
-       return uexpr;
     }
 }
 
-Expression ValueExpr()throws ParseException:
+Expression ValueExpr() throws ParseException:
 {
   Expression expr = null;
-  Identifier ident = null;
-  AbstractAccessor fa = null;
-  Expression indexExpr = null;
+  AbstractAccessor accessor = null;
 }
 {
   expr = PrimaryExpr() (
-     ident = Field()
-     {
-      fa = (fa == null ? new FieldAccessor(expr, ident)
-                       : new FieldAccessor(fa, ident));
-     }
-     | indexExpr = Index()
-     {
-      fa = (fa == null ? new IndexAccessor(expr, indexExpr)
-                       : new IndexAccessor(fa, indexExpr));
-     }
-    )*
-    {
-      return fa == null ? expr : fa;
-    }
+     accessor = FieldAccessor(accessor != null ? accessor : expr)
+     | accessor = IndexAccessor(accessor != null ? accessor : expr)
+  )*
+  {
+    return accessor == null ? expr : accessor;
+  }
 }
 
-Identifier Field() throws ParseException:
+FieldAccessor FieldAccessor(Expression inputExpr) throws ParseException:
 {
+  Token startToken = null;
   String ident = null;
 }
 {
-   <DOT> ident = Identifier()
+   <DOT> { startToken = token; } ident = Identifier()
     {
-      return new Identifier(ident);
+      FieldAccessor fa = new FieldAccessor(inputExpr, new Identifier(ident));
+      return addSourceLocation(fa, startToken);
     }
 }
 
-Expression Index() throws ParseException:
+IndexAccessor IndexAccessor(Expression inputExpr) throws ParseException:
 {
-    Expression expr = null;
+  Token startToken = null;
+  Expression expr = null;
 }
 {
-  <LEFTBRACKET> ( expr = Expression()
+  <LEFTBRACKET> { startToken = token; }
+  ( expr = Expression()
     {
-        if(expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
+        if (expr.getKind() == Expression.Kind.LITERAL_EXPRESSION)
         {
             Literal lit = ((LiteralExpr)expr).getValue();
-            if(lit.getLiteralType() != Literal.Type.INTEGER &&
-               lit.getLiteralType() != Literal.Type.LONG) {
-                throw new ParseException("Index should be an INTEGER");
+            if (lit.getLiteralType() != Literal.Type.INTEGER &&
+                lit.getLiteralType() != Literal.Type.LONG) {
+                throw new SqlppParseException(expr.getSourceLocation(), "Index should be an INTEGER");
             }
         }
     }
+  )
 
-      | <QUES> // ANY
-
-      )
-
-   <RIGHTBRACKET>
-    {
-      return expr;
-    }
+  <RIGHTBRACKET>
+  {
+    IndexAccessor ia = new IndexAccessor(inputExpr, expr);
+    return addSourceLocation(ia, startToken);
+  }
 }
 
-
-Expression PrimaryExpr()throws ParseException:
+Expression PrimaryExpr() throws ParseException:
 {
   Expression expr = null;
 }
 {
   ( LOOKAHEAD(4)
     expr = FunctionCallExpr()
+  | expr = CaseExpr()
   | expr = Literal()
   | expr = VariableRef()
+  | expr = ExternalVariableRef()
   | expr = ListConstructor()
   | expr = RecordConstructor()
   | expr = ParenthesizedExpression()
@@ -2332,7 +2371,7 @@
         try {
             lit.setValue(new LongIntegerLiteral(Long.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <FLOAT_LITERAL>
@@ -2340,7 +2379,7 @@
         try {
             lit.setValue(new FloatLiteral(Float.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <DOUBLE_LITERAL>
@@ -2348,7 +2387,7 @@
         try {
             lit.setValue(new DoubleLiteral(Double.valueOf(token.image)));
         } catch (NumberFormatException e) {
-            throw new ParseException("Could not parse numeric literal \"" + token.image +'"');
+            throw new SqlppParseException(getSourceLocation(token), "Could not parse numeric literal \"" + token.image +'"');
         }
     }
   | <MISSING>
@@ -2369,153 +2408,179 @@
     }
   )
     {
-      return lit;
+      return addSourceLocation(lit, token);
     }
 }
 
-
 VariableExpr VariableRef() throws ParseException:
 {
-    VariableExpr varExp = new VariableExpr();
     VarIdentifier var = new VarIdentifier();
+    String id = null;
 }
 {
-    { String id = null; }
     (<IDENTIFIER> { id = token.image; } | id = QuotedString())
     {
      id = SqlppVariableUtil.toInternalVariableName(id); // Prefix user-defined variables with "$"
      Identifier ident = lookupSymbol(id);
      if (isInForbiddenScopes(id)) {
-       throw new ParseException("Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
+       throw new SqlppParseException(getSourceLocation(token),
+        "Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
      }
-     if(ident != null) { // exist such ident
+     VariableExpr varExp = new VariableExpr();
+     if (ident != null) { // exist such ident
        varExp.setVar((VarIdentifier)ident);
      } else {
        varExp.setVar(var);
        varExp.setIsNewVar(false);
        var.setValue(id);
      }
-     return varExp;
+     return addSourceLocation(varExp, token);
     }
 }
 
-
 VariableExpr Variable() throws ParseException:
 {
-    VariableExpr varExp = new VariableExpr();
     VarIdentifier var = new VarIdentifier();
+    String id = null;
 }
 {
-    { String id = null; }
     (<IDENTIFIER> { id = token.image; } | id = QuotedString())
     {
      id = SqlppVariableUtil.toInternalVariableName(id); // prefix user-defined variables with "$".
      Identifier ident = lookupSymbol(id);
+     VariableExpr varExp = new VariableExpr();
      if(ident != null) { // exist such ident
        varExp.setIsNewVar(false);
      }
      varExp.setVar(var);
      var.setValue(id);
-     return varExp;
+     return addSourceLocation(varExp, token);
     }
 }
 
+VariableExpr ExternalVariableRef() throws ParseException:
+{
+  String name = null;
+}
+{
+  (
+    (
+      <DOLLAR>
+      (
+        <INTEGER_LITERAL> { name = token.image; } |
+        <IDENTIFIER> { name = token.image; } |
+        name = QuotedString()
+      )
+    )
+    |
+    (
+      <QUES> { name = String.valueOf(++externalVarCounter); }
+    )
+  )
+  {
+     String idName = SqlppVariableUtil.toExternalVariableName(name);
+     VarIdentifier id = new VarIdentifier(idName);
+     VariableExpr varExp = new VariableExpr(id);
+     return addSourceLocation(varExp, token);
+  }
+}
+
 Expression ListConstructor() throws ParseException:
 {
     Expression expr = null;
 }
 {
     (
-        expr = OrderedListConstructor() | expr = UnorderedListConstructor()
+      expr = OrderedListConstructor() |
+      expr = UnorderedListConstructor()
     )
-
     {
       return expr;
     }
 }
 
-
 ListConstructor OrderedListConstructor() throws ParseException:
 {
-      ListConstructor expr = new ListConstructor();
-      List<Expression> exprList = null;
-      expr.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+  Token startToken = null;
+  List<Expression> exprList = null;
 }
 {
-    <LEFTBRACKET> exprList = ExpressionList() <RIGHTBRACKET>
-    {
-      expr.setExprList(exprList);
-      return expr;
-    }
+  <LEFTBRACKET> { startToken = token; }
+  exprList = ExpressionList()
+  <RIGHTBRACKET>
+  {
+    ListConstructor expr = new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, exprList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 ListConstructor UnorderedListConstructor() throws ParseException:
 {
-      ListConstructor expr = new ListConstructor();
-      List<Expression> exprList = null;
-      expr.setType(ListConstructor.Type.UNORDERED_LIST_CONSTRUCTOR);
+  Token startToken = null;
+  List<Expression> exprList = null;
 }
 {
-    <LEFTDBLBRACE> exprList = ExpressionList() <RIGHTDBLBRACE>
-    {
-      expr.setExprList(exprList);
-      return expr;
-    }
+  <LEFTDBLBRACE> { startToken = token; }
+  exprList = ExpressionList()
+  <RIGHTDBLBRACE>
+  {
+    ListConstructor expr = new ListConstructor(ListConstructor.Type.UNORDERED_LIST_CONSTRUCTOR, exprList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 List<Expression> ExpressionList() throws ParseException:
 {
-      Expression expr = null;
-      List<Expression> list = null;
-      List<Expression> exprList = new ArrayList<Expression>();
+  Expression expr = null;
+  List<Expression> exprList = new ArrayList<Expression>();
 }
 {
-    (
-      expr = Expression()
+  (
+    expr = Expression()
+    {
+      exprList.add(expr);
+    }
+    ( <COMMA> expr = Expression()
       {
         exprList.add(expr);
       }
-      ( <COMMA> expr = Expression()
-        {
-          exprList.add(expr);
-        }
-      )*
-    )?
-    {
-      return exprList;
-    }
+    )*
+  )?
+  {
+    return exprList;
+  }
 }
 
 RecordConstructor RecordConstructor() throws ParseException:
 {
-      RecordConstructor expr = new RecordConstructor();
-      FieldBinding tmp = null;
-      List<FieldBinding> fbList = new ArrayList<FieldBinding>();
+  Token startToken = null;
+  FieldBinding fb = null;
+  List<FieldBinding> fbList = new ArrayList<FieldBinding>();
 }
 {
-    <LEFTBRACE> (tmp = FieldBinding()
-    {
-      fbList.add(tmp);
-    }
-    (<COMMA> tmp = FieldBinding() { fbList.add(tmp);  })*)? <RIGHTBRACE>
-    {
-      expr.setFbList(fbList);
-      return expr;
-    }
+  <LEFTBRACE> { startToken = token; }
+  (
+    fb = FieldBinding() { fbList.add(fb); }
+    (<COMMA> fb = FieldBinding() { fbList.add(fb); })*
+  )?
+  <RIGHTBRACE>
+  {
+    RecordConstructor expr = new RecordConstructor(fbList);
+    return addSourceLocation(expr, startToken);
+  }
 }
 
 FieldBinding FieldBinding() throws ParseException:
 {
-    Expression left, right;
+  Expression left, right;
 }
 {
-    left = Expression() <COLON> right = Expression()
-    {
-      return new FieldBinding(left, right);
-    }
+  left = Expression() <COLON> right = Expression()
+  {
+    return new FieldBinding(left, right);
+  }
 }
 
-
 Expression FunctionCallExpr() throws ParseException:
 {
   CallExpr callExpr;
@@ -2529,16 +2594,16 @@
 }
 {
   funcName = FunctionName()
-    {
-      hint = funcName.hint;
-    }
+  {
+    hint = funcName.hint;
+  }
   <LEFTPAREN> (
     ( <DISTINCT> { distinct = true; } )?
     ( tmp = Expression() | <MUL> { star = true; } )
     {
       if(star){
-        if(!funcName.function.toLowerCase().equals("count")){
-           throw new ParseException("The parameter * can only be used in COUNT().");
+        if(!funcName.function.equalsIgnoreCase("count")){
+           throw new SqlppParseException(getSourceLocation(token), "The parameter * can only be used in COUNT().");
         }
         argList.add(new LiteralExpr(new LongIntegerLiteral(1L)));
       } else {
@@ -2572,6 +2637,7 @@
           callExpr.addHint(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE);
         }
       }
+      callExpr.setSourceLocation(funcName.sourceLoc);
       return callExpr;
     }
 }
@@ -2592,19 +2658,19 @@
     }
 }
 
-
 Expression CaseExpr() throws ParseException:
 {
-   Expression conditionExpr = new LiteralExpr(TrueLiteral.INSTANCE);
+   Token startToken = null;
+   Expression conditionExpr = null;
    List<Expression> whenExprs = new ArrayList<Expression>();
    List<Expression> thenExprs = new ArrayList<Expression>();
    Expression elseExpr = null;
-
    Expression whenExpr = null;
    Expression thenExpr = null;
 }
 {
-   <CASE> (  conditionExpr = Expression() )?
+   <CASE> { startToken = token; }
+   ( conditionExpr = Expression() )?
    (
      <WHEN> whenExpr = Expression()
      {
@@ -2618,7 +2684,12 @@
    (<ELSE> elseExpr = Expression() )?
    <END>
    {
-     return new CaseExpression(conditionExpr, whenExprs, thenExprs, elseExpr);
+     if (conditionExpr == null) {
+        LiteralExpr litExpr = new LiteralExpr(TrueLiteral.INSTANCE);
+        conditionExpr = addSourceLocation(litExpr, startToken);
+     }
+     CaseExpression caseExpr = new CaseExpression(conditionExpr, whenExprs, thenExprs, elseExpr);
+     return addSourceLocation(caseExpr, startToken);
    }
 }
 
@@ -2636,7 +2707,10 @@
     (orderbyClause = OrderbyClause() {})?
     (limitClause = LimitClause() {})?
     {
-      return new SelectExpression(letClauses, selectSetOperation, orderbyClause, limitClause, subquery);
+      SelectExpression selectExpr =
+        new SelectExpression(letClauses, selectSetOperation, orderbyClause, limitClause, subquery);
+      selectExpr.setSourceLocation((!letClauses.isEmpty() ? letClauses.get(0) : selectSetOperation).getSourceLocation());
+      return selectExpr;
     }
 }
 
@@ -2669,7 +2743,9 @@
     }
   )*
   {
-    return new SelectSetOperation(setOperationInputLeft, setOperationRights);
+    SelectSetOperation selectSetOp = new SelectSetOperation(setOperationInputLeft, setOperationRights);
+    selectSetOp.setSourceLocation(selectBlockLeft.getSourceLocation());
+    return selectSetOp;
   }
 }
 
@@ -2693,10 +2769,11 @@
   GroupbyClause groupbyClause = null;
   List<LetClause> gbyLetClauses = null;
   HavingClause havingClause = null;
+  SourceLocation startSrcLoc = null;
 }
 {
   (
-     selectClause = SelectClause()
+     selectClause = SelectClause() { startSrcLoc = selectClause.getSourceLocation(); }
      (
         LOOKAHEAD(1)
         fromClause = FromClause()
@@ -2715,7 +2792,7 @@
         (havingClause = HavingClause())?
      )?
     |
-     fromClause = FromClause()
+     fromClause = FromClause() { startSrcLoc = fromClause.getSourceLocation(); }
      (
         LOOKAHEAD(1)
         fromLetClauses = LetClause()
@@ -2731,65 +2808,84 @@
      selectClause = SelectClause()
   )
   {
-    return new SelectBlock(selectClause, fromClause, fromLetClauses, whereClause, groupbyClause, gbyLetClauses, havingClause);
+    SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, fromLetClauses, whereClause, groupbyClause,
+      gbyLetClauses, havingClause);
+    selectBlock.setSourceLocation(startSrcLoc);
+    return selectBlock;
   }
 }
 
 SelectClause SelectClause() throws ParseException:
 {
+  Token startToken = null;
   SelectRegular selectRegular = null;
   SelectElement selectElement = null;
   boolean distinct = false;
 }
 {
-  <SELECT> (<ALL>|<DISTINCT> {distinct = true; } )?
+  <SELECT> { startToken = token; } (<ALL>|<DISTINCT> { distinct = true; } )?
   (
     selectRegular = SelectRegular()
     |
     selectElement = SelectElement()
   )?
   {
-    if(selectRegular == null && selectElement == null){
+    SourceLocation sourceLoc = getSourceLocation(startToken);
+    if (selectRegular == null && selectElement == null){
         Projection projection = new Projection(null, null, true, false);
+        projection.setSourceLocation(sourceLoc);
         List<Projection> projections = new ArrayList<Projection>();
         projections.add(projection);
         selectRegular = new SelectRegular(projections);
+        selectRegular.setSourceLocation(sourceLoc);
     }
-    return new SelectClause(selectElement, selectRegular, distinct);
+    SelectClause selectClause = new SelectClause(selectElement, selectRegular, distinct);
+    selectClause.setSourceLocation(sourceLoc);
+    return selectClause;
   }
 }
 
 SelectRegular SelectRegular() throws ParseException:
 {
+  SourceLocation startSrcLoc = null;
   List<Projection> projections = new ArrayList<Projection>();
+  Projection projection = null;
 }
 {
-   {
-      Projection projection = null;
-   }
-   projection = Projection() { projections.add(projection); }
-    ( LOOKAHEAD(2) <COMMA>
-       projection = Projection() {projections.add(projection);}
-    )*
+  projection = Projection()
   {
-    return new SelectRegular(projections);
+    projections.add(projection);
+    startSrcLoc = projection.getSourceLocation();
+  }
+  ( LOOKAHEAD(2) <COMMA> projection = Projection()
+    {
+      projections.add(projection);
+    }
+  )*
+  {
+    SelectRegular selectRegular = new SelectRegular(projections);
+    selectRegular.setSourceLocation(startSrcLoc);
+    return selectRegular;
   }
 }
 
 SelectElement SelectElement() throws ParseException:
 {
+  Token startToken = null;
   Expression expr = null;
   String name = null;
 }
 {
-  (<RAW>|<ELEMENT>|<VALUE>) expr = Expression()
+  (<RAW>|<ELEMENT>|<VALUE>) { startToken = token; } expr = Expression()
   {
-    return new SelectElement(expr);
+    SelectElement selectElement = new SelectElement(expr);
+    return addSourceLocation(selectElement, startToken);
   }
 }
 
 Projection Projection() throws ParseException :
 {
+  SourceLocation startSrcLoc = null;
   Expression expr = null;
   Identifier identifier = null;
   String name = null;
@@ -2798,8 +2894,8 @@
 }
 {
   (
-    <MUL> {star = true; }
-    | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> {varStar = true; }
+    <MUL> { star = true; startSrcLoc = getSourceLocation(token); }
+    | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { varStar = true; }
     | expr = Expression() ((<AS>)? name = Identifier())?
       {
         if (name == null) {
@@ -2811,12 +2907,15 @@
       }
   )
   {
-    return new Projection(expr, name, star, varStar);
+    Projection projection = new Projection(expr, name, star, varStar);
+    projection.setSourceLocation(expr != null ? expr.getSourceLocation() : startSrcLoc);
+    return projection;
   }
 }
 
 FromClause FromClause() throws ParseException :
 {
+  Token startToken = null;
   List<FromTerm> fromTerms = new ArrayList<FromTerm>();
   extendCurrentScope();
 }
@@ -2824,10 +2923,11 @@
   {
     FromTerm fromTerm = null;
   }
-    <FROM> fromTerm = FromTerm() { fromTerms.add(fromTerm); }
+    <FROM> { startToken = token; } fromTerm = FromTerm() { fromTerms.add(fromTerm); }
     (LOOKAHEAD(2) <COMMA> fromTerm = FromTerm() { fromTerms.add(fromTerm); } )*
   {
-    return new FromClause(fromTerms);
+    FromClause fromClause = new FromClause(fromTerms);
+    return addSourceLocation(fromClause, startToken);
   }
 }
 
@@ -2854,47 +2954,52 @@
      }
   )*
   {
-    if(leftVar==null){
+    if (leftVar == null) {
         leftVar = ExpressionToVariableUtil.getGeneratedVariable(leftExpr, true);
     }
-    return new FromTerm(leftExpr, leftVar, posVar, correlateClauses);
+    FromTerm fromTerm = new FromTerm(leftExpr, leftVar, posVar, correlateClauses);
+    fromTerm.setSourceLocation(leftExpr.getSourceLocation());
+    return fromTerm;
   }
 }
 
 JoinClause JoinClause(JoinType joinType) throws ParseException :
 {
+    Token startToken = null;
     Expression rightExpr = null;
     VariableExpr rightVar = null;
     VariableExpr posVar = null;
     Expression conditionExpr = null;
 }
 {
-  <JOIN> rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())? <ON> conditionExpr = Expression()
+  <JOIN> { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())? <ON> conditionExpr = Expression()
   {
     if(rightVar==null){
         rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
     }
-    return new JoinClause(joinType, rightExpr, rightVar, posVar, conditionExpr);
+    JoinClause joinClause = new JoinClause(joinType, rightExpr, rightVar, posVar, conditionExpr);
+    return addSourceLocation(joinClause, startToken);
   }
 }
 
 UnnestClause UnnestClause(JoinType joinType) throws ParseException :
 {
+    Token startToken = null;
     Expression rightExpr;
     VariableExpr rightVar;
     VariableExpr posVar = null;
 }
 {
-  (<UNNEST>|<CORRELATE>|<FLATTEN>) rightExpr = Expression() ((<AS>)? rightVar = Variable()) (<AT> posVar = Variable())?
+  (<UNNEST>|<CORRELATE>|<FLATTEN>) { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable()) (<AT> posVar = Variable())?
   {
-    if(rightVar==null){
-        rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
+    if (rightVar == null) {
+      rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
     }
-    return new UnnestClause(joinType, rightExpr, rightVar, posVar);
+    UnnestClause unnestClause = new UnnestClause(joinType, rightExpr, rightVar, posVar);
+    return addSourceLocation(unnestClause, startToken);
   }
 }
 
-
 JoinType JoinType() throws ParseException :
 {
    JoinType joinType = JoinType.INNER;
@@ -2922,30 +3027,32 @@
     }
 }
 
-WhereClause WhereClause()throws ParseException :
+WhereClause WhereClause() throws ParseException :
 {
-  WhereClause wc = new WhereClause();
+  Token startToken = null;
   Expression whereExpr;
 }
 {
-    <WHERE> whereExpr = Expression()
+    <WHERE> { startToken = token; } whereExpr = Expression()
     {
-      wc.setWhereExpr(whereExpr);
-      return wc;
+      WhereClause wc = new WhereClause(whereExpr);
+      return addSourceLocation(wc, startToken);
     }
 }
 
-OrderbyClause OrderbyClause()throws ParseException :
+OrderbyClause OrderbyClause() throws ParseException :
 {
+    Token startToken = null;
     OrderbyClause oc = new OrderbyClause();
     Expression orderbyExpr;
     List<Expression> orderbyList = new ArrayList<Expression>();
-    List<OrderbyClause.OrderModifier> modifierList = new ArrayList<OrderbyClause.OrderModifier >();
+    List<OrderbyClause.OrderModifier> modifierList = new ArrayList<OrderbyClause.OrderModifier>();
     int numOfOrderby = 0;
 }
 {
     <ORDER>
       {
+        startToken = token;
         String hint = getHint(token);
         if (hint != null) {
           if (hint.startsWith(INMEMORY_HINT)) {
@@ -2983,12 +3090,13 @@
     {
       oc.setModifierList(modifierList);
       oc.setOrderbyList(orderbyList);
-      return oc;
+      return addSourceLocation(oc, startToken);
     }
 }
 
 GroupbyClause GroupbyClause()throws ParseException :
 {
+    Token startToken = null;
     GroupbyClause gbc = new GroupbyClause();
     List<GbyVariableExpressionPair> vePairList = new ArrayList<GbyVariableExpressionPair>();
     VariableExpr var = null;
@@ -3006,6 +3114,7 @@
       }
     <GROUP>
       {
+         startToken = token;
          String hint = getHint(token);
          if (hint != null && hint.equals(HASH_GROUP_BY_HINT)) {
            gbc.setHashGroupByHint(true);
@@ -3067,39 +3176,43 @@
       gbc.setGroupVar(groupVar);
       gbc.setGroupFieldList(groupFieldList);
       replaceCurrentScope(newScope);
-      return gbc;
+      return addSourceLocation(gbc, startToken);
     }
 }
 
 HavingClause HavingClause() throws ParseException:
 {
+   Token startToken = null;
    Expression filterExpr = null;
 }
 {
-    <HAVING> filterExpr = Expression()
+    <HAVING> { startToken = token; } filterExpr = Expression()
     {
-       return new HavingClause(filterExpr);
+       HavingClause havingClause = new HavingClause(filterExpr);
+       return addSourceLocation(havingClause, startToken);
     }
 }
 
 LimitClause LimitClause() throws ParseException:
 {
+    Token startToken = null;
     LimitClause lc = new LimitClause();
     Expression expr;
     pushForbiddenScope(getCurrentScope());
 }
 {
-    <LIMIT> expr = Expression()    { lc.setLimitExpr(expr);    }
-    (<OFFSET> expr = Expression() { lc.setOffset(expr);    })?
+    <LIMIT> { startToken = token; } expr = Expression() { lc.setLimitExpr(expr); }
+    (<OFFSET> expr = Expression() { lc.setOffset(expr); })?
 
   {
     popForbiddenScope();
-    return lc;
+    return addSourceLocation(lc, startToken);
   }
 }
 
 QuantifiedExpression QuantifiedExpression()throws ParseException:
 {
+  Token startToken = null;
   QuantifiedExpression qc = new QuantifiedExpression();
   List<QuantifiedPair> quantifiedList = new ArrayList<QuantifiedPair>();
   Expression satisfiesExpr;
@@ -3112,8 +3225,8 @@
     createNewScope();
   }
 
-   (      ((<ANY>|<SOME>)  {  qc.setQuantifier(QuantifiedExpression.Quantifier.SOME);    })
-        | (<EVERY> {  qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY);    }))
+   ( ((<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
+     | (<EVERY> {  startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
     var = Variable() <IN> inExpr = Expression()
     {
       pair = new QuantifiedPair(var, inExpr);
@@ -3131,13 +3244,12 @@
        qc.setSatisfiesExpr(satisfiesExpr);
        qc.setQuantifiedList(quantifiedList);
        removeCurrentScope();
-       return qc;
+       return addSourceLocation(qc, startToken);
      }
 }
 
 LetClause LetElement() throws ParseException:
 {
-    LetClause lc = new LetClause();
     VariableExpr varExp;
     Expression beExp;
     extendCurrentScope();
@@ -3145,15 +3257,14 @@
 {
     varExp = Variable() <EQ> beExp = Expression()
     {
-      lc.setVarExpr(varExp);
-      lc.setBindingExpr(beExp);
+      LetClause lc = new LetClause(varExp, beExp);
+      lc.setSourceLocation(varExp.getSourceLocation());
       return lc;
     }
 }
 
 LetClause WithElement() throws ParseException:
 {
-    LetClause lc = new LetClause();
     VariableExpr varExp;
     Expression beExp;
     extendCurrentScope();
@@ -3161,8 +3272,8 @@
 {
     varExp = Variable() <AS> beExp = Expression()
     {
-      lc.setVarExpr(varExp);
-      lc.setBindingExpr(beExp);
+      LetClause lc = new LetClause(varExp, beExp);
+      lc.setSourceLocation(varExp.getSourceLocation());
       return lc;
     }
 }
@@ -3253,6 +3364,7 @@
   | <JOIN : "join">
   | <KEYWORD : "keyword">
   | <KEY : "key">
+  | <KNOWN : "known">
   | <LEFT : "left">
   | <LETTING : "letting">
   | <LET : "let">
@@ -3310,10 +3422,10 @@
 {
     <CARET : "^">
   | <CONCAT : "||">
-  | <DIV : "/">
-  | <IDIV : "idiv">
+  | <DIVIDE : "/">
+  | <DIV : "div">
   | <MINUS : "-">
-  | <MOD : "%">
+  | <MOD : "mod">
   | <MUL : "*">
   | <PLUS : "+">
 
@@ -3325,7 +3437,9 @@
   | <ATT : "@">
   | <COLON : ":">
   | <COMMA : ",">
+  | <DOLLAR: "$">
   | <DOT : ".">
+  | <PERCENT: "%">
   | <QUES : "?">
   | <SEMICOLON : ";">
   | <SHARP : "#">
@@ -3373,8 +3487,8 @@
 <DEFAULT,IN_DBL_BRACE>
 TOKEN [IGNORE_CASE]:
 {
-  <MISSING : "missing">
-  |  <NULL : "null">
+    <MISSING : "missing">
+  | <NULL : "null">
   | <TRUE : "true">
   | <FALSE : "false">
 }
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
index 5b680b2..dca5da7 100644
--- a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/pom.xml
@@ -66,7 +66,6 @@
     <dependency>
       <groupId>org.reflections</groupId>
       <artifactId>reflections</artifactId>
-      <version>0.9.10</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
diff --git a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
index 031707b..dfd189f 100644
--- a/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
+++ b/asterixdb/asterix-maven-plugins/asterix-evaluator-generator-maven-plugin/src/main/java/org/apache/asterix/runtime/evaluators/staticcodegen/GatherInnerClassVisitor.java
@@ -44,8 +44,7 @@
 
     @Override
     public void visitInnerClass(String name, String outerName, String innerName, int access) {
-        if ((className == null || !name.equals(className))
-                && ((access & Opcodes.ACC_PUBLIC) == 0 || (access & Opcodes.ACC_STATIC) == 0)) {
+        if ((!name.equals(className)) && ((access & Opcodes.ACC_PUBLIC) == 0 || (access & Opcodes.ACC_STATIC) == 0)) {
             innerClassNames.add(name);
         }
     }
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index cf342fe..216db8c 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -36,18 +36,16 @@
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-core</artifactId>
-      <version>3.3.9</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-compat</artifactId>
-      <version>3.3.9</version>
     </dependency>
     <!-- dependencies to annotations -->
     <dependency>
       <groupId>org.apache.maven.plugin-tools</groupId>
       <artifactId>maven-plugin-annotations</artifactId>
-      <version>3.5</version>
+      <version>3.6.0</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
index 8ed9d50..f5d32ac 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/test/java/org/apache/asterix/lang/extension/EchoStatement.java
@@ -19,10 +19,10 @@
 package org.apache.asterix.lang.extension;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
-public class EchoStatement implements Statement {
+public class EchoStatement extends AbstractStatement {
 
     private final String arg;
 
diff --git a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 3613166..044852b 100644
--- a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -124,6 +124,62 @@
         }
     }
 
+    public void getLastTokenImage(TokenImage image) {
+        if (bufpos >= tokenBegin) {
+            image.reset(buffer, tokenBegin, bufpos - tokenBegin);
+        } else {
+            image.reset(buffer, tokenBegin, bufsize - tokenBegin, buffer, 0, bufpos);
+        }
+    }
+
+    public static class TokenImage{
+        private char[] buffer;
+        private int begin;
+        private int length;
+
+        // used to hold circular tokens
+        private char[] tmpBuffer;
+
+        private static final double TMP_BUFFER_GROWTH = 1.5;
+
+        public void reset(char[] buffer, int begin, int length) {
+            this.buffer = buffer;
+            this.begin = begin;
+            this.length = length;
+        }
+
+        public void reset(char[] buffer, int begin, int length, char[] extraBuffer, int extraBegin, int extraLength) {
+            ensureTmpBufferSize(length + extraLength);
+            System.arraycopy(buffer, begin, tmpBuffer, 0, length);
+            System.arraycopy(extraBuffer, extraBegin, tmpBuffer, length, extraLength);
+            this.buffer = tmpBuffer;
+            this.begin = 0;
+            this.length = length + extraLength;
+        }
+
+        public char[] getBuffer() {
+            return buffer;
+        }
+
+        public int getBegin() {
+            return begin;
+        }
+
+        public int getLength() {
+            return length;
+        }
+
+
+        private void ensureTmpBufferSize(int size) {
+            int oldSize = tmpBuffer!=null?tmpBuffer.length:0;
+            if(oldSize < size) {
+                int newSize = Math.max((int)(oldSize * TMP_BUFFER_GROWTH), size);
+                tmpBuffer = new char[newSize];
+            }
+        }
+
+    }
+
     public int getColumn() {
         return column;
     }
@@ -262,4 +318,4 @@
         endOf_UNUSED_Buffer = bufsize;
         tokenBegin = 0;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
index be295d6..1cfb804 100644
--- a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
+++ b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
@@ -56,7 +56,7 @@
     public long allocate() {
         final LocalManager localManager = local.get();
         final @E@RecordManager recMgr = localManager.mgr;
-        final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7fff) : 0;
+        final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7ffe + 1) : 1;
         final int localId = recMgr.allocate();
 
         long result = TypeUtil.Global.build(localManager.arenaId, allocId, localId);
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 94c2668..b78554b 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -156,7 +156,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -169,15 +168,15 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/CachingTxnIdFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/CachingTxnIdFactory.java
index 6b3b6a0f..a284658 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/CachingTxnIdFactory.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/CachingTxnIdFactory.java
@@ -62,7 +62,7 @@
     }
 
     @Override
-    public void ensureMinimumId(long id) throws AlgebricksException {
+    public void ensureMinimumId(long id) {
         throw new UnsupportedOperationException();
     }
 
@@ -98,5 +98,6 @@
     }
 
     private static class BlockExhaustedException extends Exception {
+        private static final long serialVersionUID = 8967868415735213490L;
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index b994c50..0cd03d7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -395,7 +395,7 @@
         } else {
             op.entity.dropFromCache(this);
         }
-    };
+    }
 
     protected void undoOperation(MetadataLogicalOperation op) {
         if (!op.isAdd) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index a6a708f..e5ee163 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -25,8 +25,6 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -60,6 +58,9 @@
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 /**
  * Provides access to Asterix metadata via remote methods to the metadata node.
@@ -94,10 +95,10 @@
  * with transaction ids of regular jobs or other metadata transactions.
  */
 public abstract class MetadataManager implements IMetadataManager {
+    private static final Logger LOGGER = LogManager.getLogger();
     private final MetadataCache cache = new MetadataCache();
     protected final Collection<IAsterixStateProxy> proxies;
     protected IMetadataNode metadataNode;
-    private final ReadWriteLock metadataLatch;
     protected boolean rebindMetadataNode = false;
 
     // TODO(mblow): replace references of this (non-constant) field with a method,
@@ -117,7 +118,6 @@
             throw new IllegalArgumentException("Null / empty list of proxies given to MetadataManager");
         }
         this.proxies = proxies;
-        this.metadataLatch = new ReentrantReadWriteLock(true);
     }
 
     protected abstract TxnId createTxnId();
@@ -134,15 +134,29 @@
         return new MetadataTransactionContext(txnId);
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void commitTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.commitTransaction(ctx.getTxnId());
-        cache.commit(ctx);
+    public void commitTransaction(MetadataTransactionContext ctx) {
+        try {
+            metadataNode.commitTransaction(ctx.getTxnId());
+            cache.commit(ctx);
+        } catch (Throwable th) {
+            // Metadata node should abort all transactions on re-joining the new CC
+            LOGGER.fatal("Failure committing a metadata transaction", th);
+            ExitUtil.halt(ExitUtil.EC_FAILED_TO_COMMIT_METADATA_TXN);
+        }
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void abortTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.abortTransaction(ctx.getTxnId());
+    public void abortTransaction(MetadataTransactionContext ctx) {
+        try {
+            metadataNode.abortTransaction(ctx.getTxnId());
+        } catch (Throwable th) {
+            // Metadata node should abort all transactions on re-joining the new CC
+            LOGGER.fatal("Failure aborting a metadata transaction", th);
+            ExitUtil.halt(ExitUtil.EC_FAILED_TO_ABORT_METADATA_TXN);
+        }
     }
 
     @Override
@@ -725,26 +739,6 @@
     }
 
     @Override
-    public void acquireWriteLatch() {
-        metadataLatch.writeLock().lock();
-    }
-
-    @Override
-    public void releaseWriteLatch() {
-        metadataLatch.writeLock().unlock();
-    }
-
-    @Override
-    public void acquireReadLatch() {
-        metadataLatch.readLock().lock();
-    }
-
-    @Override
-    public void releaseReadLatch() {
-        metadataLatch.readLock().unlock();
-    }
-
-    @Override
     public FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
             throws AlgebricksException {
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 543b794..8cecb9e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -30,18 +30,14 @@
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
 import org.apache.asterix.common.dataflow.LSMIndexUtil;
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
-import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager.AtomicityLevel;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.ITxnIdFactory;
-import org.apache.asterix.common.transactions.ImmutableDatasetId;
 import org.apache.asterix.common.transactions.TransactionOptions;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
@@ -124,13 +120,16 @@
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class MetadataNode implements IMetadataNode {
     private static final long serialVersionUID = 1L;
-
+    private static final Logger LOGGER = LogManager.getLogger();
     // shared between core and extension
-    private IDatasetLifecycleManager datasetLifecycleManager;
-    private ITransactionSubsystem transactionSubsystem;
+    private transient IDatasetLifecycleManager datasetLifecycleManager;
+    private transient ITransactionSubsystem transactionSubsystem;
     private int metadataStoragePartition;
     private transient CachingTxnIdFactory txnIdFactory;
     // core only
@@ -167,19 +166,33 @@
     }
 
     @Override
-    public void beginTransaction(TxnId transactionId) throws ACIDException, RemoteException {
+    public void beginTransaction(TxnId transactionId) throws RemoteException {
         TransactionOptions options = new TransactionOptions(AtomicityLevel.ATOMIC);
         transactionSubsystem.getTransactionManager().beginTransaction(transactionId, options);
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void commitTransaction(TxnId txnId) throws RemoteException, ACIDException {
-        transactionSubsystem.getTransactionManager().commitTransaction(txnId);
+    public void commitTransaction(TxnId txnId) throws RemoteException {
+        try {
+            transactionSubsystem.getTransactionManager().commitTransaction(txnId);
+        } catch (Throwable th) {
+            // Metadata node should abort all Metadata transactions on re-start
+            LOGGER.fatal("Failure committing a metadata transaction", th);
+            ExitUtil.halt(ExitUtil.EC_FAILED_TO_COMMIT_METADATA_TXN);
+        }
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void abortTransaction(TxnId txnId) throws RemoteException, ACIDException {
-        transactionSubsystem.getTransactionManager().abortTransaction(txnId);
+    public void abortTransaction(TxnId txnId) throws RemoteException {
+        try {
+            transactionSubsystem.getTransactionManager().abortTransaction(txnId);
+        } catch (Throwable th) {
+            // Metadata node should abort all uncommitted transactions on re-start
+            LOGGER.fatal("Failure committing a metadata transaction", th);
+            ExitUtil.halt(ExitUtil.EC_FAILED_TO_ABORT_METADATA_TXN);
+        }
     }
 
     // TODO(amoudi): make all metadata operations go through the generic methods
@@ -197,7 +210,7 @@
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
             insertTupleIntoIndex(txnId, index, tuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -216,7 +229,7 @@
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
             upsertTupleIntoIndex(txnId, index, tuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -235,7 +248,7 @@
         try {
             ITupleReference tuple = tupleTranslator.getTupleFromMetadataEntity(entity);
             deleteTupleFromIndex(txnId, index, tuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -249,7 +262,6 @@
      * @param index
      * @return
      * @throws AlgebricksException
-     * @throws RemoteException
      */
     private <T> List<T> getEntities(TxnId txnId, ITupleReference searchKey,
             IMetadataEntityTupleTranslator<T> tupleTranslator, IMetadataIndex index)
@@ -322,8 +334,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -350,13 +360,11 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
     @Override
-    public void addIndex(TxnId txnId, Index index) throws AlgebricksException, RemoteException {
+    public void addIndex(TxnId txnId, Index index) throws AlgebricksException {
         try {
             IndexTupleTranslator tupleWriter = tupleTranslatorProvider.getIndexTupleTranslator(txnId, this, true);
             ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
@@ -367,8 +375,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -384,8 +390,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -403,8 +407,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -422,8 +424,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -444,23 +444,21 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
     private void insertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
-            throws ACIDException, HyracksDataException {
+            throws HyracksDataException {
         modifyMetadataIndex(Operation.INSERT, txnId, metadataIndex, tuple);
     }
 
     private void upsertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
-            throws ACIDException, HyracksDataException {
+            throws HyracksDataException {
         modifyMetadataIndex(Operation.UPSERT, txnId, metadataIndex, tuple);
     }
 
     private void modifyMetadataIndex(Operation op, TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
-            throws ACIDException, HyracksDataException {
+            throws HyracksDataException {
         String resourceName = metadataIndex.getFile().getRelativePath();
         ILSMIndex lsmIndex = (ILSMIndex) datasetLifecycleManager.get(resourceName);
         datasetLifecycleManager.open(resourceName);
@@ -518,7 +516,6 @@
     @Override
     public void dropDataverse(TxnId txnId, String dataverseName) throws AlgebricksException, RemoteException {
         try {
-
             confirmDataverseCanBeDeleted(txnId, dataverseName);
 
             // As a side effect, acquires an S lock on the 'Function' dataset
@@ -572,7 +569,7 @@
             }
 
             List<FeedPolicyEntity> feedPolicies = getDataversePolicies(txnId, dataverseName);
-            if (feedPolicies != null && feedPolicies.size() > 0) {
+            if (feedPolicies != null && !feedPolicies.isEmpty()) {
                 // Drop all feed ingestion policies in this dataverse.
                 for (FeedPolicyEntity feedPolicy : feedPolicies) {
                     dropFeedPolicy(txnId, dataverseName, feedPolicy.getPolicyName());
@@ -585,9 +582,6 @@
             // on behalf of txnId.
             ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
-
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -596,8 +590,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -640,7 +632,7 @@
                     // As a side effect, acquires an S lock on the 'ExternalFile' dataset
                     // on behalf of txnId.
                     List<ExternalFile> datasetFiles = getExternalFiles(txnId, dataset);
-                    if (datasetFiles != null && datasetFiles.size() > 0) {
+                    if (datasetFiles != null && !datasetFiles.isEmpty()) {
                         // Drop all external files in this dataset.
                         for (ExternalFile file : datasetFiles) {
                             dropExternalFile(txnId, dataverseName, file.getDatasetName(), file.getFileNumber());
@@ -657,7 +649,7 @@
             } finally {
                 deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             }
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -671,8 +663,6 @@
             // lock on the 'index' dataset.
             ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -681,8 +671,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -708,8 +696,6 @@
             // lock on the 'nodegroup' dataset.
             ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
             return true;
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
@@ -719,8 +705,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -745,9 +729,6 @@
                     dropDatatype(txnId, dataverseName, dt.getDatatypeName());
                 }
             }
-
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -755,8 +736,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -768,8 +747,6 @@
             // lock on the 'datatype' dataset.
             ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -777,13 +754,11 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
     private void deleteTupleFromIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
-            throws ACIDException, HyracksDataException {
+            throws HyracksDataException {
         modifyMetadataIndex(Operation.DELETE, txnId, metadataIndex, tuple);
     }
 
@@ -896,11 +871,10 @@
 
     public List<Dataset> getAllDatasets(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
-            ITupleReference searchKey = null;
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Dataset> results = new ArrayList<>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, null, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -909,11 +883,10 @@
 
     public List<Function> getAllFunctions(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
-            ITupleReference searchKey = null;
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Function> results = new ArrayList<>();
-            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, null, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -922,12 +895,11 @@
 
     public List<Datatype> getAllDatatypes(TxnId txnId) throws AlgebricksException, RemoteException {
         try {
-            ITupleReference searchKey = null;
             DatatypeTupleTranslator tupleReaderWriter =
                     tupleTranslatorProvider.getDataTypeTupleTranslator(txnId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, null, valueExtractor, results);
             return results;
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
@@ -1041,7 +1013,6 @@
                     || dataType.getDatatype().getTypeName().equals(datatypeName)) {
                 continue;
             }
-
             AbstractComplexType recType = (AbstractComplexType) dataType.getDatatype();
             if (recType.containsType(typeToBeDropped)) {
                 throw new AlgebricksException("Cannot drop type " + dataverseName + "." + datatypeName
@@ -1075,7 +1046,7 @@
         return nestedTypes;
     }
 
-    public List<String> getDatasetNamesPartitionedOnThisNodeGroup(TxnId txnId, String nodegroup)
+    private List<String> getDatasetNamesPartitionedOnThisNodeGroup(TxnId txnId, String nodegroup)
             throws AlgebricksException, RemoteException {
         // this needs to scan the datasets and return the datasets that use this
         // nodegroup
@@ -1166,7 +1137,7 @@
             throws AlgebricksException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(),
-                    "" + functionSignature.getArity());
+                    Integer.toString(functionSignature.getArity()));
             FunctionTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getFunctionTupleTranslator(false);
             List<Function> results = new ArrayList<>();
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
@@ -1202,13 +1173,10 @@
 
     private void dropFunction(TxnId txnId, FunctionSignature functionSignature, boolean force)
             throws AlgebricksException, RemoteException {
-
         if (!force) {
             confirmFunctionCanBeDeleted(txnId, functionSignature);
         }
-
         Function function = getFunction(txnId, functionSignature);
-
         if (function == null) {
             throw new AlgebricksException(
                     "Cannot drop function '" + functionSignature.toString() + "' because it doesn't exist.");
@@ -1216,15 +1184,12 @@
         try {
             // Delete entry from the 'function' dataset.
             ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(),
-                    "" + functionSignature.getArity());
+                    Integer.toString(functionSignature.getArity()));
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
             ITupleReference functionTuple =
                     getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
-
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1233,8 +1198,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1244,8 +1207,6 @@
         List<ITupleReference> results = new ArrayList<>();
         searchIndex(txnId, metadataIndex, searchKey, valueExtractor, results);
         if (results.isEmpty()) {
-            // TODO: Temporarily a TreeIndexException to make it get caught by
-            // caller in the appropriate catch block.
             throw HyracksDataException.create(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY);
         }
         // There should be exactly one result returned from the search.
@@ -1253,11 +1214,10 @@
     }
 
     // Debugging Method
-    public String printMetadata() {
-
+    private String printMetadata() {
         StringBuilder sb = new StringBuilder();
         try {
-            RangePredicate rangePred = null;
+            RangePredicate rangePred;
             IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
             String resourceName = index.getFile().toString();
             IIndex indexInstance = datasetLifecycleManager.get(resourceName);
@@ -1278,12 +1238,10 @@
                     rangeCursor.close();
                 }
                 datasetLifecycleManager.close(resourceName);
-                index = MetadataPrimaryIndexes.DATASET_DATASET;
                 indexInstance = datasetLifecycleManager.get(resourceName);
                 datasetLifecycleManager.open(resourceName);
                 indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
                 rangeCursor = indexAccessor.createSearchCursor(false);
-                rangePred = null;
                 rangePred = new RangePredicate(null, null, true, true, null, null);
                 indexAccessor.search(rangeCursor, rangePred);
                 try {
@@ -1300,12 +1258,10 @@
                     rangeCursor.close();
                 }
                 datasetLifecycleManager.close(resourceName);
-                index = MetadataPrimaryIndexes.INDEX_DATASET;
                 indexInstance = datasetLifecycleManager.get(resourceName);
                 datasetLifecycleManager.open(resourceName);
                 indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
                 rangeCursor = indexAccessor.createSearchCursor(false);
-                rangePred = null;
                 rangePred = new RangePredicate(null, null, true, true, null, null);
                 indexAccessor.search(rangeCursor, rangePred);
                 try {
@@ -1325,14 +1281,13 @@
             }
             datasetLifecycleManager.close(resourceName);
         } catch (Exception e) {
-            // Debugging method
-            e.printStackTrace();
+            LOGGER.error(e);
         }
         return sb.toString();
     }
 
-    private <ResultType> void searchIndex(TxnId txnId, IMetadataIndex index, ITupleReference searchKey,
-            IValueExtractor<ResultType> valueExtractor, List<ResultType> results)
+    private <T> void searchIndex(TxnId txnId, IMetadataIndex index, ITupleReference searchKey,
+            IValueExtractor<T> valueExtractor, List<T> results)
             throws AlgebricksException, HyracksDataException, RemoteException {
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         if (index.getFile() == null) {
@@ -1360,8 +1315,8 @@
         datasetLifecycleManager.close(resourceName);
     }
 
-    private <ResultType> void search(IIndexAccessor indexAccessor, RangePredicate rangePred, List<ResultType> results,
-            IValueExtractor<ResultType> valueExtractor, TxnId txnId)
+    private <T> void search(IIndexAccessor indexAccessor, RangePredicate rangePred, List<T> results,
+            IValueExtractor<T> valueExtractor, TxnId txnId)
             throws HyracksDataException, RemoteException, AlgebricksException {
         IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
         try {
@@ -1369,7 +1324,7 @@
             try {
                 while (rangeCursor.hasNext()) {
                     rangeCursor.next();
-                    ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
+                    T result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
                     if (result != null) {
                         results.add(result);
                     }
@@ -1416,7 +1371,8 @@
                         rangeCursor.next();
                         final ITupleReference ref = rangeCursor.getTuple();
                         final Dataset ds = valueExtractor.getValue(txnId, ref);
-                        int datasetId = ds.getDatasetId();
+                        int datasetId = Math.max(ds.getDatasetId(),
+                                DatasetIdFactory.generateAlternatingDatasetId(ds.getDatasetId()));
                         if (mostRecentDatasetId < datasetId) {
                             mostRecentDatasetId = datasetId;
                         }
@@ -1433,8 +1389,6 @@
         return mostRecentDatasetId;
     }
 
-    // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
-    // Hyracks version.
     public static ITupleReference createTuple(String... fields) {
         ISerializerDeserializer<AString> stringSerde =
                 SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
@@ -1485,8 +1439,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1505,9 +1457,6 @@
             ITupleReference datasetTuple =
                     getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
-
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1515,10 +1464,7 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
-
     }
 
     @Override
@@ -1556,8 +1502,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1611,8 +1555,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1631,9 +1573,6 @@
             ITupleReference datasetTuple =
                     getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
-
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1641,10 +1580,7 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
-
     }
 
     @Override
@@ -1666,7 +1602,7 @@
     }
 
     @Override
-    public int getMostRecentDatasetId() throws AlgebricksException, RemoteException {
+    public int getMostRecentDatasetId() throws RemoteException {
         return DatasetIdFactory.getMostRecentDatasetId();
     }
 
@@ -1684,8 +1620,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1713,7 +1647,7 @@
             FeedConnectionTupleTranslator tupleReaderWriter = new FeedConnectionTupleTranslator(true);
             ITupleReference feedConnTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedConnection);
             insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, feedConnTuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -1759,7 +1693,7 @@
             ITupleReference tuple =
                     getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET, tuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
@@ -1778,8 +1712,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1822,8 +1754,6 @@
             // lock on the 'nodegroup' dataset.
             ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
         } catch (HyracksDataException e) {
             if (e.getComponent().equals(ErrorCode.HYRACKS)
                     && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
@@ -1831,8 +1761,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1850,8 +1778,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1886,8 +1812,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -1925,8 +1849,6 @@
             } else {
                 throw new AlgebricksException(e);
             }
-        } catch (ACIDException e) {
-            throw new AlgebricksException(e);
         }
     }
 
@@ -2007,7 +1929,7 @@
             DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(true);
             datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
             insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
-        } catch (HyracksDataException | ACIDException e) {
+        } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java
index d25f488..ebb9cac 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/ExtensionMetadataDataset.java
@@ -29,16 +29,16 @@
 
     private static final long serialVersionUID = 1L;
     private final ExtensionMetadataDatasetId indexId;
-    private final transient IMetadataEntityTupleTranslator<T> tupleTranslator;
+    private final transient IMetadataEntityTupleTranslatorFactory<T> tupleTranslatorFactory;
 
     public ExtensionMetadataDataset(MetadataIndexImmutableProperties indexProperties, int numFields, IAType[] keyTypes,
             List<List<String>> keyNames, int numSecondaryIndexKeys, ARecordType payloadType, boolean isPrimaryIndex,
             int[] primaryKeyIndexes, ExtensionMetadataDatasetId indexId,
-            IMetadataEntityTupleTranslator<T> tupleTranslator) {
+            IMetadataEntityTupleTranslatorFactory<T> tupleTranslatorFactory) {
         super(indexProperties, numFields, keyTypes, keyNames, numSecondaryIndexKeys, payloadType, isPrimaryIndex,
                 primaryKeyIndexes);
         this.indexId = indexId;
-        this.tupleTranslator = tupleTranslator;
+        this.tupleTranslatorFactory = tupleTranslatorFactory;
     }
 
     public ExtensionMetadataDatasetId getId() {
@@ -46,6 +46,6 @@
     }
 
     public IMetadataEntityTupleTranslator<T> getTupleTranslator() {
-        return tupleTranslator;
+        return tupleTranslatorFactory.createTupleTranslator();
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
index 7103fd8..baface2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
@@ -82,7 +82,7 @@
     void add(Dataset dataset) throws HyracksDataException;
 
     /**
-     * Remove dataset to the list of associated datasets
+     * Remove dataset from the list of associated datasets
      *
      * @param dataset
      *            the dataset to add
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java
new file mode 100644
index 0000000..e472383
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java
@@ -0,0 +1,29 @@
+/*
+ * 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.asterix.metadata.api;
+
+import java.io.Serializable;
+
+public interface IMetadataEntityTupleTranslatorFactory<T> extends Serializable {
+
+    /**
+     * @return an instance of IMetadataEntityTupleTranslator
+     */
+    IMetadataEntityTupleTranslator<T> createTupleTranslator();
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 966792e..a646893 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -519,14 +519,6 @@
 
     int getMostRecentDatasetId() throws AlgebricksException;
 
-    void acquireWriteLatch();
-
-    void releaseWriteLatch();
-
-    void acquireReadLatch();
-
-    void releaseReadLatch();
-
     /**
      * Removes a library , acquiring local locks on behalf of the given transaction
      * id.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index d5d5733..f837bec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -24,7 +24,6 @@
 import java.rmi.RemoteException;
 import java.util.List;
 
-import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.indexing.ExternalFile;
@@ -57,26 +56,23 @@
     /**
      * Begins a local transaction against the metadata.
      *
-     * @throws ACIDException
      * @throws RemoteException
      */
-    void beginTransaction(TxnId txnId) throws ACIDException, RemoteException;
+    void beginTransaction(TxnId txnId) throws RemoteException;
 
     /**
      * Commits a local transaction against the metadata.
      *
-     * @throws ACIDException
      * @throws RemoteException
      */
-    void commitTransaction(TxnId txnId) throws ACIDException, RemoteException;
+    void commitTransaction(TxnId txnId) throws RemoteException;
 
     /**
      * Aborts a local transaction against the metadata.
      *
-     * @throws ACIDException
      * @throws RemoteException
      */
-    void abortTransaction(TxnId txnId) throws ACIDException, RemoteException;
+    void abortTransaction(TxnId txnId) throws RemoteException;
 
     /**
      * Inserts a new dataverse into the metadata, acquiring local locks on behalf of
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 980d36d..95526f4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -24,17 +24,19 @@
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.MetadataProperties;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 import org.apache.asterix.external.api.IAdapterFactory;
@@ -66,7 +68,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
 import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -78,7 +79,6 @@
 import org.apache.hyracks.storage.am.common.build.IndexBuilder;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -324,10 +324,10 @@
         ILSMOperationTrackerFactory opTrackerFactory =
                 index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(datasetId)
                         : new SecondaryIndexOperationTrackerFactory(datasetId);
-        ILSMComponentIdGeneratorFactory idGeneratorProvider =
-                new DatasetLSMComponentIdGeneratorFactory(index.getDatasetId().getId());
+        ILSMComponentIdGeneratorFactory idGeneratorProvider = new DatasetLSMComponentIdGeneratorFactory(datasetId);
+        DatasetInfoProvider datasetInfoProvider = new DatasetInfoProvider(datasetId);
         ILSMIOOperationCallbackFactory ioOpCallbackFactory =
-                new LSMBTreeIOOperationCallbackFactory(idGeneratorProvider);
+                new LSMIndexIOOperationCallbackFactory(idGeneratorProvider, datasetInfoProvider);
         IStorageComponentProvider storageComponentProvider = appContext.getStorageComponentProvider();
         if (isNewUniverse()) {
             final double bloomFilterFalsePositiveRate =
@@ -375,35 +375,23 @@
         // as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then
         // to INDEX_DATASET.
         MetadataTransactionContext mdTxnCtx = null;
-        MetadataManager.INSTANCE.acquireWriteLatch();
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Starting DDL recovery ...");
-        }
-
+        LOGGER.info("Starting DDL recovery ...");
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
             for (Dataverse dataverse : dataverses) {
                 recoverDataverse(mdTxnCtx, dataverse);
             }
-            // the commit wasn't there before. yet, everything was working
-            // correctly!!!!!!!!!!!
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.info("Completed DDL recovery.");
-            }
+            LOGGER.info("Completed DDL recovery.");
         } catch (Exception e) {
             try {
-                if (IS_DEBUG_MODE) {
-                    LOGGER.log(Level.ERROR, "Failure during DDL recovery", e);
-                }
+                LOGGER.error("Failure during DDL recovery", e);
                 MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
             } catch (Exception e2) {
                 e.addSuppressed(e2);
             }
-            throw new MetadataException(e);
-        } finally {
-            MetadataManager.INSTANCE.releaseWriteLatch();
+            throw MetadataException.create(e);
         }
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 66283d6..88449f9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public abstract class DataSource implements IDataSource<DataSourceId> {
 
@@ -159,7 +160,8 @@
     public abstract Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index ddab5bc..62cce05 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -21,6 +21,8 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataManager;
@@ -42,10 +44,11 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class DatasetDataSource extends DataSource {
 
-    private Dataset dataset;
+    private final Dataset dataset;
 
     public DatasetDataSource(DataSourceId id, Dataset dataset, IAType itemType, IAType metaItemType,
             byte datasourceType, IDatasetDetails datasetDetails, INodeDomain datasetDomain) throws AlgebricksException {
@@ -92,11 +95,16 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
         switch (dataset.getDatasetType()) {
             case EXTERNAL:
+                if (tupleFilterFactory != null || outputLimit >= 0) {
+                    throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                            "Tuple filter and limit are not supported by ExternalDataSource");
+                }
                 Dataset externalDataset = ((DatasetDataSource) dataSource).getDataset();
                 String itemTypeName = externalDataset.getItemTypeName();
                 IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
@@ -117,7 +125,8 @@
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
-                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, false);
+                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+                        false);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 5c3ed56..81be1f7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -22,7 +22,8 @@
 import java.util.List;
 
 import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
@@ -48,6 +49,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class FeedDataSource extends DataSource implements IMutationDataSource {
 
@@ -158,10 +160,15 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
         try {
+            if (tupleFilterFactory != null || outputLimit >= 0) {
+                throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        "Tuple filter and limit are not supported by FeedDataSource");
+            }
             ARecordType feedOutputType = (ARecordType) itemType;
             ISerializerDeserializer payloadSerde =
                     metadataProvider.getDataFormat().getSerdeProvider().getSerializerDeserializer(feedOutputType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index d2b9871..3b5cf2e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -25,6 +25,8 @@
 import java.util.Set;
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
 import org.apache.asterix.metadata.api.IDatasourceFunction;
 import org.apache.asterix.om.types.IAType;
@@ -44,6 +46,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public abstract class FunctionDataSource extends DataSource {
 
@@ -68,9 +71,14 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
+        if (tupleFilterFactory != null || outputLimit >= 0) {
+            throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                    "tuple filter and limit are not supported by FunctionDataSource");
+        }
         GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
         adapterFactory.setOutputType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
         IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 8fdcbbc..3460a46 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -24,6 +24,8 @@
 import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -44,6 +46,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public class LoadableDataSource extends DataSource {
 
@@ -124,9 +127,14 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
             MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
+        if (tupleFilterFactory != null || outputLimit >= 0) {
+            throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                    "tuple filter and limit are not supported by LoadableDataSource");
+        }
         LoadableDataSource alds = (LoadableDataSource) dataSource;
         ARecordType itemType = (ARecordType) alds.getLoadedType();
         IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index a081fb4..e212d11 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -26,7 +26,6 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -38,6 +37,7 @@
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -57,6 +57,7 @@
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -122,7 +123,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.primitive.ShortPointable;
@@ -132,6 +133,7 @@
 import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -147,7 +149,7 @@
     private final StorageProperties storageProperties;
     private final IFunctionManager functionManager;
     private final LockList locks;
-    private final Map<String, String> config;
+    private final Map<String, Object> config;
 
     private Dataverse defaultDataverse;
     private MetadataTransactionContext mdTxnCtx;
@@ -172,8 +174,23 @@
         config = new HashMap<>();
     }
 
-    public String getPropertyValue(String propertyName) {
-        return config.get(propertyName);
+    @SuppressWarnings("unchecked")
+    public <T> T getProperty(String name) {
+        return (T) config.get(name);
+    }
+
+    public void setProperty(String name, Object value) {
+        config.put(name, value);
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> T removeProperty(String name) {
+        return (T) config.remove(name);
+    }
+
+    public boolean getBooleanProperty(String name, boolean defaultValue) {
+        Object v = config.get(name);
+        return v != null ? Boolean.parseBoolean(String.valueOf(v)) : defaultValue;
     }
 
     public void disableBlockingOperator() {
@@ -185,7 +202,7 @@
     }
 
     @Override
-    public Map<String, String> getConfig() {
+    public Map<String, Object> getConfig() {
         return config;
     }
 
@@ -295,7 +312,7 @@
      */
     public ARecordType findOutputRecordType() throws AlgebricksException {
         return MetadataManagerUtil.findOutputRecordType(mdTxnCtx, getDefaultDataverseName(),
-                getPropertyValue("output-record-type"));
+                getProperty("output-record-type"));
     }
 
     public Dataset findDataset(String dataverse, String dataset) throws AlgebricksException {
@@ -380,10 +397,12 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
             IDataSource<DataSourceId> dataSource, List<LogicalVariable> scanVariables,
             List<LogicalVariable> projectVariables, boolean projectPushed, List<LogicalVariable> minFilterVars,
-            List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
+            List<LogicalVariable> maxFilterVars, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec,
+            Object implConfig) throws AlgebricksException {
         return ((DataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables, projectVariables,
-                projectPushed, minFilterVars, maxFilterVars, opSchema, typeEnv, context, jobSpec, implConfig);
+                projectPushed, minFilterVars, maxFilterVars, tupleFilterFactory, outputLimit, opSchema, typeEnv,
+                context, jobSpec, implConfig);
     }
 
     protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
@@ -433,7 +452,8 @@
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
             boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
+            int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            boolean isIndexOnlyPlan) throws AlgebricksException {
         boolean isSecondary = true;
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), dataset.getDatasetName());
@@ -477,8 +497,8 @@
             btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                     context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                    maxFilterFieldIndexes, propagateFilter, proceedIndexOnlyPlan, failValueForIndexOnlyPlan,
-                    successValueForIndexOnlyPlan);
+                    maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit, proceedIndexOnlyPlan,
+                    failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -654,7 +674,7 @@
             boolean bulkload) throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
-                context, spec, bulkload, null, null);
+                context, spec, bulkload, null, null, null);
     }
 
     @Override
@@ -666,7 +686,7 @@
             throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
-                context, spec, false, null, null);
+                context, spec, false, null, null, null);
     }
 
     @Override
@@ -674,12 +694,12 @@
             IDataSourceIndex<String, DataSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalFilteringKeys,
-            ILogicalExpression filterExpr, List<LogicalVariable> prevSecondaryKeys,
+            ILogicalExpression filterExpr, LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
             LogicalVariable prevAdditionalFilteringKey, RecordDescriptor recordDesc, JobGenContext context,
             JobSpecification spec) throws AlgebricksException {
         return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.UPSERT, dataSourceIndex, propagatedSchema,
                 inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, recordDesc,
-                context, spec, false, prevSecondaryKeys, prevAdditionalFilteringKey);
+                context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey);
     }
 
     @Override
@@ -1028,8 +1048,8 @@
             List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
             List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr,
             RecordDescriptor inputRecordDesc, JobGenContext context, JobSpecification spec, boolean bulkload,
-            List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKey)
-            throws AlgebricksException {
+            LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
+            LogicalVariable prevAdditionalFilteringKey) throws AlgebricksException {
         String indexName = dataSourceIndex.getId();
         String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
         String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
@@ -1048,18 +1068,19 @@
             case BTREE:
                 return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
                         secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
-                        bulkload, prevSecondaryKeys, prevAdditionalFilteringKeys);
+                        bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
             case RTREE:
                 return getRTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
                         secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
-                        bulkload, prevSecondaryKeys, prevAdditionalFilteringKeys);
+                        bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX:
                 return getInvertedIndexRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
                         secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
-                        secondaryIndex.getIndexType(), bulkload, prevSecondaryKeys, prevAdditionalFilteringKeys);
+                        secondaryIndex.getIndexType(), bulkload, upsertIndicatorVar, prevSecondaryKeys,
+                        prevAdditionalFilteringKeys);
             default:
                 throw new AlgebricksException(
                         indexOp.name() + "Insert, upsert, and delete not implemented for index type: "
@@ -1071,8 +1092,9 @@
             String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
             AsterixTupleFilterFactory filterFactory, RecordDescriptor inputRecordDesc, JobGenContext context,
-            JobSpecification spec, IndexOperation indexOp, boolean bulkload, List<LogicalVariable> prevSecondaryKeys,
-            List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
+            JobSpecification spec, IndexOperation indexOp, boolean bulkload, LogicalVariable upsertIndicatorVar,
+            List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
+            throws AlgebricksException {
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
         int numKeys = primaryKeys.size() + secondaryKeys.size();
         int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
@@ -1139,8 +1161,10 @@
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh, null,
                         BulkLoadUsage.LOAD, dataset.getDatasetId());
             } else if (indexOp == IndexOperation.UPSERT) {
+                int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
                 op = new LSMSecondaryUpsertOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, idfh,
-                        filterFactory, modificationCallbackFactory, prevFieldPermutation);
+                        filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
+                        BinaryBooleanInspector.FACTORY, prevFieldPermutation);
             } else {
                 op = new LSMTreeInsertDeleteOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, indexOp, idfh,
                         filterFactory, false, modificationCallbackFactory);
@@ -1155,8 +1179,9 @@
             String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
             AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec, IndexOperation indexOp, boolean bulkload, List<LogicalVariable> prevSecondaryKeys,
-            List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
+            JobSpecification spec, IndexOperation indexOp, boolean bulkload, LogicalVariable upsertIndicatorVar,
+            List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
+            throws AlgebricksException {
         Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = MetadataManager.INSTANCE
@@ -1236,8 +1261,10 @@
                     GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, indexDataflowHelperFactory,
                     null, BulkLoadUsage.LOAD, dataset.getDatasetId());
         } else if (indexOp == IndexOperation.UPSERT) {
+            int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
             op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation,
-                    indexDataflowHelperFactory, filterFactory, modificationCallbackFactory, prevFieldPermutation);
+                    indexDataflowHelperFactory, filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
+                    BinaryBooleanInspector.FACTORY, prevFieldPermutation);
         } else {
             op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
                     indexDataflowHelperFactory, filterFactory, false, modificationCallbackFactory);
@@ -1250,8 +1277,8 @@
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
             AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
             JobSpecification spec, IndexOperation indexOp, IndexType indexType, boolean bulkload,
-            List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
-            throws AlgebricksException {
+            LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
+            List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
         // Check the index is length-partitioned or not.
         boolean isPartitioned;
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
@@ -1345,8 +1372,10 @@
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, indexDataFlowFactory,
                         null, BulkLoadUsage.LOAD, dataset.getDatasetId());
             } else if (indexOp == IndexOperation.UPSERT) {
+                int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
                 op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation, indexDataFlowFactory,
-                        filterFactory, modificationCallbackFactory, prevFieldPermutation);
+                        filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
+                        BinaryBooleanInspector.FACTORY, prevFieldPermutation);
             } else {
                 op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
                         indexDataFlowFactory, filterFactory, false, modificationCallbackFactory);
@@ -1541,7 +1570,8 @@
         }
     }
 
-    private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+    @Override
+    public AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
             throws AlgebricksException {
         // No filtering condition.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ResultSetSinkId.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ResultSetSinkId.java
index 51feb56..2fb7344 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ResultSetSinkId.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ResultSetSinkId.java
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.metadata.declared;
 
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 
 public class ResultSetSinkId {
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index a531add..14f76eb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -79,6 +79,7 @@
         _builtinTypeMap.put("null", BuiltinType.ANULL);
         _builtinTypeMap.put("uuid", BuiltinType.AUUID);
         _builtinTypeMap.put("shortwithouttypeinfo", BuiltinType.SHORTWITHOUTTYPEINFO);
+        _builtinTypeMap.put("geometry", BuiltinType.AGEOMETRY);
     }
 
     private BuiltinTypeMap() {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 7981309..8471d45 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -27,17 +27,17 @@
 
 import org.apache.asterix.active.IActiveEntityEventsListener;
 import org.apache.asterix.active.IActiveNotificationHandler;
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
 import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.NoOpFrameOperationCallbackFactory;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
 import org.apache.asterix.common.metadata.IDataset;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
 import org.apache.asterix.common.utils.JobUtils;
@@ -98,6 +98,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
@@ -105,7 +106,6 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -127,9 +127,6 @@
      */
     private static final long serialVersionUID = 1L;
     private static final Logger LOGGER = LogManager.getLogger();
-    //TODO: Remove Singletons
-    private static final BTreeResourceFactoryProvider bTreeResourceFactoryProvider =
-            BTreeResourceFactoryProvider.INSTANCE;
     private static final RTreeResourceFactoryProvider rTreeResourceFactoryProvider =
             RTreeResourceFactoryProvider.INSTANCE;
     private static final InvertedIndexResourceFactoryProvider invertedIndexResourceFactoryProvider =
@@ -153,10 +150,6 @@
     private final long rebalanceCount;
     private int pendingOp;
 
-    /*
-     * Transient (For caching)
-     */
-
     public Dataset(String dataverseName, String datasetName, String recordTypeDataverseName, String recordTypeName,
             String nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
             IDatasetDetails datasetDetails, Map<String, String> hints, DatasetType datasetType, int datasetId,
@@ -338,12 +331,14 @@
      *            a mutable progress state used for error handling during the drop operation
      * @param hcc
      *            a client connection to hyracks master for job execution
+     * @param sourceLoc
      * @throws Exception
      *             if an error occur during the drop process or if the dataset can't be dropped for any reason
      */
     public void drop(MetadataProvider metadataProvider, MutableObject<MetadataTransactionContext> mdTxnCtx,
             List<JobSpecification> jobsToExecute, MutableBoolean bActiveTxn, MutableObject<ProgressState> progress,
-            IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup) throws Exception {
+            IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup, SourceLocation sourceLoc)
+            throws Exception {
         Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<>();
         if (getDatasetType() == DatasetType.INTERNAL) {
             // prepare job spec(s) that would disconnect any active feeds involving the dataset.
@@ -362,7 +357,8 @@
                     MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (indexes.get(j).isSecondaryIndex()) {
-                    jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this));
+                    jobsToExecute
+                            .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
                 }
             }
             jobsToExecute.add(DatasetUtil.dropDatasetJobSpec(this, metadataProvider));
@@ -399,7 +395,8 @@
                     MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName);
             for (int j = 0; j < indexes.size(); j++) {
                 if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) {
-                    jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this));
+                    jobsToExecute
+                            .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
                 } else {
                     jobsToExecute.add(DatasetUtil.buildDropFilesIndexJobSpec(metadataProvider, this));
                 }
@@ -469,8 +466,9 @@
         IResourceFactory resourceFactory;
         switch (index.getIndexType()) {
             case BTREE:
-                resourceFactory = bTreeResourceFactoryProvider.getResourceFactory(mdProvider, this, index, recordType,
-                        metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories);
+                resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+                        filterCmpFactories);
                 break;
             case RTREE:
                 resourceFactory = rTreeResourceFactoryProvider.getResourceFactory(mdProvider, this, index, recordType,
@@ -502,24 +500,9 @@
      * @throws AlgebricksException
      *             if the factory could not be created for the index/dataset combination
      */
+    @SuppressWarnings("squid:S1172")
     public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
-        switch (index.getIndexType()) {
-            case BTREE:
-                return getDatasetType() == DatasetType.EXTERNAL
-                        && !index.getIndexName().equals(IndexingConstants.getFilesIndexName(getDatasetName()))
-                                ? new LSMBTreeWithBuddyIOOperationCallbackFactory(getComponentIdGeneratorFactory())
-                                : new LSMBTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case RTREE:
-                return new LSMRTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            case LENGTH_PARTITIONED_NGRAM_INVIX:
-            case LENGTH_PARTITIONED_WORD_INVIX:
-            case SINGLE_PARTITION_NGRAM_INVIX:
-            case SINGLE_PARTITION_WORD_INVIX:
-                return new LSMInvertedIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory());
-            default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
-                        index.getIndexType().toString());
-        }
+        return new LSMIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
     }
 
     /**
@@ -538,6 +521,10 @@
         return new DatasetLSMComponentIdGeneratorFactory(getDatasetId());
     }
 
+    public IDatasetInfoProvider getDatasetInfoProvider() {
+        return new DatasetInfoProvider(getDatasetId());
+    }
+
     /**
      * Get search callback factory for this dataset with the passed index and operation
      *
@@ -563,7 +550,6 @@
             boolean proceedIndexOnlyPlan) throws AlgebricksException {
         if (index.isPrimaryIndex()) {
             /**
-            /*
              * Due to the read-committed isolation level,
              * we may acquire very short duration lock(i.e., instant lock) for readers.
              */
@@ -696,7 +682,7 @@
                 datasetPartitions, isSink);
     }
 
-    public IFrameOperationCallbackFactory getFrameOpCallbackFactory() {
+    public IFrameOperationCallbackFactory getFrameOpCallbackFactory(MetadataProvider mdProvider) {
         return NoOpFrameOperationCallbackFactory.INSTANCE;
     }
 
@@ -861,4 +847,8 @@
         }
         return partitions;
     }
+
+    public String getFullyQualifiedName() {
+        return dataverseName + '.' + datasetName;
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
index 1b40484..35d21a0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataverse.java
@@ -65,4 +65,19 @@
     public String toString() {
         return getClass().getSimpleName() + ":" + dataverseName;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof Dataverse)) {
+            return false;
+        }
+        Dataverse other = (Dataverse) o;
+        return dataverseName.equals(other.getDataverseName());
+    }
+
+    @Override
+    public int hashCode() {
+        return dataverseName.hashCode();
+    }
+
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index bf19b03..3159030 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -22,6 +22,7 @@
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
@@ -153,8 +154,18 @@
             ARecordType recType) throws AlgebricksException {
         Pair<IAType, Boolean> keyPairType = null;
         IAType subType = recType;
+        boolean nullable = false;
         for (int i = 0; i < fieldName.size(); i++) {
-            subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+            if (subType instanceof AUnionType) {
+                nullable = nullable || ((AUnionType) subType).isUnknownableType();
+                subType = ((AUnionType) subType).getActualType();
+            }
+            if (subType instanceof ARecordType) {
+                subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+            } else {
+                throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+            }
+
             if (subType == null) {
                 keyPairType = Index.getNonNullableType(fieldType);
                 break;
@@ -163,13 +174,16 @@
         if (subType != null) {
             keyPairType = Index.getNonNullableKeyFieldType(fieldName, recType);
         }
+        keyPairType.second = keyPairType.second || nullable;
         return keyPairType;
     }
 
     public static Pair<IAType, Boolean> getNonNullableKeyFieldType(List<String> expr, ARecordType recType)
             throws AlgebricksException {
         IAType keyType = Index.keyFieldType(expr, recType);
-        return getNonNullableType(keyType);
+        Pair<IAType, Boolean> pair = getNonNullableType(keyType);
+        pair.second = pair.second || recType.isSubFieldNullable(expr);
+        return pair;
     }
 
     private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
index 80c4f3d..e4f8948 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/InternalDatasetDetails.java
@@ -49,11 +49,11 @@
 
     public enum FileStructure {
         BTREE
-    };
+    }
 
     public enum PartitioningStrategy {
         HASH
-    };
+    }
 
     private final FileStructure fileStructure;
     private final PartitioningStrategy partitioningStrategy;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
index 6595d97..38f4206 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/CompactionPolicyTupleTranslator.java
@@ -38,6 +38,8 @@
  * Translates a Dataset metadata entity to an ITupleReference and vice versa.
  */
 public class CompactionPolicyTupleTranslator extends AbstractTupleTranslator<CompactionPolicy> {
+    private static final long serialVersionUID = 5291424952240239023L;
+
     // Field indexes of serialized CompactionPolicy in a tuple.
     // Key field.
     public static final int COMPACTION_POLICY_DATAVERSE_NAME_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index a58e5d6..8f630cf 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public class DatasourceAdapterTupleTranslator extends AbstractTupleTranslator<DatasourceAdapter> {
+    private static final long serialVersionUID = 6183434454125673504L;
 
     // Field indexes of serialized Adapter in a tuple.
     // First key field.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index d0d7b00..32951d5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -61,6 +61,8 @@
  * Translates a Datatype metadata entity to an ITupleReference and vice versa.
  */
 public class DatatypeTupleTranslator extends AbstractTupleTranslator<Datatype> {
+    private static final long serialVersionUID = -2324433490801381399L;
+
     // Field indexes of serialized Dataset in a tuple.
     // First key field.
     public static final int DATATYPE_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
@@ -73,7 +75,7 @@
         RECORD,
         UNORDEREDLIST,
         ORDEREDLIST
-    };
+    }
 
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerDes =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 3100b10..133ab35 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -42,6 +42,8 @@
  * Translates a Dataverse metadata entity to an ITupleReference and vice versa.
  */
 public class DataverseTupleTranslator extends AbstractTupleTranslator<Dataverse> {
+    private static final long serialVersionUID = -3196752600543191613L;
+
     // Field indexes of serialized Dataverse in a tuple.
     // Key field.
     public static final int DATAVERSE_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
index d058d90..a9fada0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -43,6 +43,8 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public class ExternalFileTupleTranslator extends AbstractTupleTranslator<ExternalFile> {
+    private static final long serialVersionUID = -4966958481117396312L;
+
     // Field indexes of serialized ExternalFile in a tuple.
     // First key field.
     public static final int EXTERNAL_FILE_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
index 61a8ab2..800e5df 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedConnectionTupleTranslator.java
@@ -22,7 +22,6 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
-import java.io.DataOutput;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -33,7 +32,6 @@
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
 import org.apache.asterix.metadata.entities.FeedConnection;
-import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.ARecord;
@@ -48,6 +46,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public class FeedConnectionTupleTranslator extends AbstractTupleTranslator<FeedConnection> {
+    private static final long serialVersionUID = -1798961999812829511L;
 
     public static final int FEED_CONN_DATAVERSE_NAME_FIELD_INDEX = 0;
     public static final int FEED_CONN_FEED_NAME_FIELD_INDEX = 1;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
index e80b96f4..a7fc822 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedPolicyTupleTranslator.java
@@ -51,6 +51,8 @@
  * Translates a Dataset metadata entity to an ITupleReference and vice versa.
  */
 public class FeedPolicyTupleTranslator extends AbstractTupleTranslator<FeedPolicyEntity> {
+    private static final long serialVersionUID = 826298425589924684L;
+
     // Field indexes of serialized FeedPolicy in a tuple.
     // Key field.
     public static final int FEED_POLICY_DATAVERSE_NAME_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
index 4e1056e..8cc801d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
@@ -51,6 +51,8 @@
  * Translates a Feed metadata entity to an ITupleReference and vice versa.
  */
 public class FeedTupleTranslator extends AbstractTupleTranslator<Feed> {
+    private static final long serialVersionUID = -5967081194106401387L;
+
     // Field indexes of serialized Feed in a tuple.
     // Key field.
     public static final int FEED_DATAVERSE_NAME_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index 9085235..c4e42d1b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -47,6 +47,8 @@
  * Translates a Function metadata entity to an ITupleReference and vice versa.
  */
 public class FunctionTupleTranslator extends AbstractTupleTranslator<Function> {
+    private static final long serialVersionUID = 1147594449575992161L;
+
     // Field indexes of serialized Function in a tuple.
     // First key field.
     public static final int FUNCTION_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
index f001ec4..8d216ff 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -39,6 +39,8 @@
  * Translates a Library metadata entity to an ITupleReference and vice versa.
  */
 public class LibraryTupleTranslator extends AbstractTupleTranslator<Library> {
+    private static final long serialVersionUID = -7574173417999340281L;
+
     // Field indexes of serialized Library in a tuple.
     // First key field.
     public static final int LIBRARY_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
index f67ae3b..8510535 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/NodeTupleTranslator.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.metadata.entitytupletranslators;
 
-import org.apache.asterix.common.exceptions.MetadataException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
@@ -37,6 +36,7 @@
  * Translates a Node metadata entity to an ITupleReference and vice versa.
  */
 public class NodeTupleTranslator extends AbstractTupleTranslator<Node> {
+    private static final long serialVersionUID = -5257435809246039182L;
 
     // Field indexes of serialized Node in a tuple.
     // First key field.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 8a0cf84..3ae0fec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -296,7 +296,6 @@
         }
 
         MetadataTransactionContext ctx = null;
-        MetadataManager.INSTANCE.acquireReadLatch();
         try {
             ctx = MetadataManager.INSTANCE.beginTransaction();
             Datatype t = MetadataManager.INSTANCE.getDatatype(ctx, dataverseName, datatypeName);
@@ -314,8 +313,6 @@
                 }
                 throw new MetadataException(ErrorCode.FEED_CREATE_FEED_DATATYPE_ERROR, e, datatypeName);
             }
-        } finally {
-            MetadataManager.INSTANCE.releaseReadLatch();
         }
         return outputType;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 6d81145..28c612f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -407,15 +407,20 @@
         IIndexDataflowHelperFactory idfh =
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
         LSMPrimaryUpsertOperatorDescriptor op;
-        ITypeTraits[] outputTypeTraits =
-                new ITypeTraits[inputRecordDesc.getFieldCount() + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
-        ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount()
+        ITypeTraits[] outputTypeTraits = new ITypeTraits[inputRecordDesc.getFieldCount() + 1
+                + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+        ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount() + 1
                 + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
         IDataFormat dataFormat = metadataProvider.getDataFormat();
 
-        // add the previous record first
         int f = 0;
+        // add the upsert indicator var
+        outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(BuiltinType.ABOOLEAN);
+        outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(BuiltinType.ABOOLEAN);
+        f++;
+        // add the previous record
         outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(itemType);
+        outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(itemType);
         f++;
         // add the previous meta second
         if (dataset.hasMetaPart()) {
@@ -447,7 +452,7 @@
         RecordDescriptor outputRecordDesc = new RecordDescriptor(outputSerDes, outputTypeTraits);
         op = new LSMPrimaryUpsertOperatorDescriptor(spec, outputRecordDesc, fieldPermutation, idfh,
                 missingWriterFactory, modificationCallbackFactory, searchCallbackFactory,
-                dataset.getFrameOpCallbackFactory(), numKeys, itemType, fieldIdx, hasSecondaries);
+                dataset.getFrameOpCallbackFactory(metadataProvider), numKeys, itemType, fieldIdx, hasSecondaries);
         return new Pair<>(op, splitsAndConstraint.second);
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index a3c3842..418bc50 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -63,6 +63,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
@@ -241,6 +242,7 @@
      * @param dataset
      * @param files
      * @param indexerDesc
+     * @param sourceLoc
      * @return
      * @throws AlgebricksException
      * @throws HyracksDataException
@@ -248,23 +250,27 @@
      */
     private static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> getIndexingOperator(
             MetadataProvider metadataProvider, JobSpecification jobSpec, IAType itemType, Dataset dataset,
-            List<ExternalFile> files, RecordDescriptor indexerDesc) throws HyracksDataException, AlgebricksException {
+            List<ExternalFile> files, RecordDescriptor indexerDesc, SourceLocation sourceLoc)
+            throws HyracksDataException, AlgebricksException {
         ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
         Map<String, String> configuration = externalDatasetDetails.getProperties();
         IAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
                 metadataProvider.getApplicationContext().getServiceContext(), externalDatasetDetails.getAdapter(),
                 configuration, (ARecordType) itemType, files, true, null);
-        return new Pair<>(new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
-                adapterFactory.getPartitionConstraint());
+        ExternalScanOperatorDescriptor scanOp =
+                new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory);
+        scanOp.setSourceLocation(sourceLoc);
+        return new Pair<>(scanOp, adapterFactory.getPartitionConstraint());
     }
 
     public static Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(
             JobSpecification spec, MetadataProvider metadataProvider, Dataset dataset, ARecordType itemType,
-            RecordDescriptor indexerDesc, List<ExternalFile> files) throws HyracksDataException, AlgebricksException {
+            RecordDescriptor indexerDesc, List<ExternalFile> files, SourceLocation sourceLoc)
+            throws HyracksDataException, AlgebricksException {
         return getIndexingOperator(metadataProvider, spec, itemType, dataset,
                 files == null ? MetadataManager.INSTANCE
                         .getDatasetExternalFiles(metadataProvider.getMetadataTxnContext(), dataset) : files,
-                indexerDesc);
+                indexerDesc, sourceLoc);
     }
 
     /**
@@ -420,8 +426,8 @@
     }
 
     public static JobSpecification buildIndexUpdateOp(Dataset ds, Index index, List<ExternalFile> metadataFiles,
-            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider)
-            throws AlgebricksException {
+            List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws AlgebricksException {
         // Create files list
         ArrayList<ExternalFile> files = new ArrayList<>();
 
@@ -441,7 +447,7 @@
         for (ExternalFile file : appendedFiles) {
             files.add(file);
         }
-        return IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, files);
+        return IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, index, metadataProvider, files, sourceLoc);
     }
 
     public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index eb14bae..c29279b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobSpecification;
 
@@ -103,40 +104,41 @@
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
-            Dataset dataset) throws AlgebricksException {
+            Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
-            Dataset dataset, Set<DropOption> options) throws AlgebricksException {
+            Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
 
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
-        return buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null);
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
+        return buildSecondaryIndexLoadingJobSpec(dataset, index, metadataProvider, null, sourceLoc);
     }
 
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, List<ExternalFile> files) throws AlgebricksException {
+            MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
+            throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper;
         if (dataset.isCorrelated()) {
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
-                    index, metadataProvider, physicalOptimizationConfig);
+                    index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         } else {
             secondaryIndexHelper = SecondaryTreeIndexOperationsHelper.createIndexOperationsHelper(dataset, index,
-                    metadataProvider, physicalOptimizationConfig);
+                    metadataProvider, physicalOptimizationConfig, sourceLoc);
         }
         if (files != null) {
             secondaryIndexHelper.setExternalFiles(files);
@@ -145,9 +147,9 @@
     }
 
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
-            MetadataProvider metadataProvider) throws AlgebricksException {
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         SecondaryIndexOperationsHelper secondaryIndexHelper = SecondaryIndexOperationsHelper
-                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig);
+                .createIndexOperationsHelper(dataset, index, metadataProvider, physicalOptimizationConfig, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index cf6338a..56e967e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -53,7 +53,6 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 
 public class SecondaryBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
 
@@ -108,9 +107,11 @@
                 secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, dataflowHelperFactory,
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             }
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+            metaOp.setSourceLocation(sourceLoc);
             spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
             root = metaOp;
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -170,9 +171,10 @@
 
             // bulk load op ----> sink op
             sourceOp = targetOp;
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
 
             spec.addRoot(targetOp);
@@ -236,7 +238,7 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn);
+                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -274,9 +276,9 @@
         }
 
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
-                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
+                    .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), itemType, filterFieldName,
+                            numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 7791cad..12e21f6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -102,9 +102,11 @@
                 createTreeIndexBulkLoadOp(spec, metadataProvider, taggedSecondaryRecDesc,
                         createFieldPermutationForBulkLoadOp(), getNumSecondaryKeys(), numPrimaryKeys, false);
 
-        AlgebricksMetaOperatorDescriptor metaOp =
-                new AlgebricksMetaOperatorDescriptor(spec, 1, 0, new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                        new RecordDescriptor[] { taggedSecondaryRecDesc });
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
+        AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { taggedSecondaryRecDesc });
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -155,7 +157,7 @@
             }
             secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
-                    index.getKeyFieldNames().get(i), sourceColumn);
+                    index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
                     index.getKeyFieldNames().get(i), sourceType);
             IAType keyType = keyTypePair.first;
@@ -185,7 +187,7 @@
         if (numFilterFields > 0) {
             secondaryFieldAccessEvalFactories[numSecondaryKeys] =
                     metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index b91d65f..c6e4322 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -85,11 +85,12 @@
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
-                    indexType, RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
+                    sourceLoc, indexType,
+                    RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
         }
         if (numSecondaryKeys > 1) {
-            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, numSecondaryKeys,
-                    indexType, 1);
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, sourceLoc,
+                    numSecondaryKeys, indexType, 1);
         }
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
@@ -111,7 +112,7 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), recordColumn);
+                    index.getKeyFieldNames().get(0), recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -123,7 +124,7 @@
         if (numFilterFields > 0) {
             secondaryFieldAccessEvalFactories[numSecondaryKeys] =
                     metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn);
+                            metadataProvider.getFunctionManager(), itemType, filterFieldName, recordColumn, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
@@ -244,8 +245,11 @@
                 createTreeIndexBulkLoadOp(spec, metadataProvider, taggedTokenKeyPairRecDesc,
                         createFieldPermutationForBulkLoadOp(), getNumSecondaryKeys(), numPrimaryKeys, true);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index bf5178c..7a855d5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -77,7 +77,8 @@
         int numSecondaryKeys = secondaryKeyFields.size();
         boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
         if (numSecondaryKeys != 1) {
-            throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, numSecondaryKeys);
+            throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, sourceLoc,
+                    numSecondaryKeys);
         }
         Pair<IAType, Boolean> spatialTypePair =
                 Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
@@ -89,7 +90,7 @@
         int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
         secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
                 metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR, sourceLoc);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
@@ -212,8 +213,11 @@
                 metadataProvider, secondaryRecDescConsideringPointMBR, createFieldPermutationForBulkLoadOp(),
                 numNestedSecondaryKeFieldsConsideringPointMBR, numPrimaryKeys, false);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 48982c0..c8d0bf3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -47,6 +47,9 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.primitive.BooleanPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -57,6 +60,8 @@
 import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDiskComponentScanOperatorDescriptor;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /**
  * This class is used to build secondary LSM index for correlated datasets.
  *
@@ -77,7 +82,10 @@
      * This descending order ensures older components can be bulk loaded first and get a smaller (older)
      * component file timestamp.
      */
-    protected final static IBinaryComparatorFactory COMPONENT_POS_COMPARATOR_FACTORY = new IBinaryComparatorFactory() {
+    protected static final IBinaryComparatorFactory COMPONENT_POS_COMPARATOR_FACTORY =
+            new ComponentPosComparatorFactory();
+
+    public static final class ComponentPosComparatorFactory implements IBinaryComparatorFactory {
 
         private static final long serialVersionUID = 1L;
 
@@ -93,7 +101,17 @@
                 }
             };
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return COMPONENT_POS_COMPARATOR_FACTORY;
+        }
+    }
 
     protected SecondaryCorrelatedTreeIndexOperationsHelper(Dataset dataset, Index index,
             PhysicalOptimizationConfig physOptConf, MetadataProvider metadataProvider) throws AlgebricksException {
@@ -177,6 +195,7 @@
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
         sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { getTaggedRecordDescriptor(enforcedRecDesc) });
     }
@@ -219,8 +238,10 @@
             sefs[i] = secondaryFieldAccessEvalFactories[i];
         }
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        assign.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixAssignOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
                 primaryPartitionConstraint);
         return asterixAssignOp;
@@ -231,6 +252,7 @@
         IOperatorDescriptor op = new LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor(spec,
                 taggedSecondaryRecDesc, MissingWriterFactory.INSTANCE, NUM_TAG_FIELDS, numSecondaryKeyFields,
                 numPrimaryKeyFields, hasBuddyBTree);
+        op.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, primaryPartitionConstraint);
         return op;
     }
@@ -248,6 +270,7 @@
         ExternalSortOperatorDescriptor sortOp =
                 new ExternalSortOperatorDescriptor(spec, physOptConf.getMaxFramesExternalSort(), taggedSortFields,
                         taggedSecondaryComparatorFactories, taggedSecondaryRecDesc);
+        sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
@@ -265,6 +288,7 @@
                 new LSMSecondaryIndexBulkLoadOperatorDescriptor(spec, taggedSecondaryRecDesc, primaryIndexHelperFactory,
                         secondaryIndexHelperFactory, fieldPermutation, NUM_TAG_FIELDS, numSecondaryKeys, numPrimaryKeys,
                         hasBuddyBtree);
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -280,15 +304,17 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
         LSMBTreeDiskComponentScanOperatorDescriptor primaryScanOp = new LSMBTreeDiskComponentScanOperatorDescriptor(
                 spec, outRecDesc, indexHelperFactory, searchCallbackFactory);
+        primaryScanOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primaryScanOp,
                 primaryPartitionConstraint);
         return primaryScanOp;
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AlgebricksException {
+            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+            throws AlgebricksException {
 
-        SecondaryIndexOperationsHelper indexOperationsHelper = null;
+        SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case BTREE:
                 indexOperationsHelper =
@@ -306,8 +332,10 @@
                         physOptConf, metadataProvider);
                 break;
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+                        index.getIndexType());
         }
+        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 065eb72..5f9e6ef 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -67,6 +67,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -118,6 +119,7 @@
     protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
     protected int numPrimaryKeys;
+    protected SourceLocation sourceLoc;
 
     // Prevent public construction. Should be created via createIndexCreator().
     protected SecondaryIndexOperationsHelper(Dataset dataset, Index index, PhysicalOptimizationConfig physOptConf,
@@ -142,7 +144,8 @@
     }
 
     public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
-            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AlgebricksException {
+            MetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf, SourceLocation sourceLoc)
+            throws AlgebricksException {
 
         SecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
@@ -162,8 +165,10 @@
                         new SecondaryInvertedIndexOperationsHelper(dataset, index, physOptConf, metadataProvider);
                 break;
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType());
+                throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+                        index.getIndexType());
         }
+        indexOperationsHelper.setSourceLocation(sourceLoc);
         indexOperationsHelper.init();
         return indexOperationsHelper;
     }
@@ -176,6 +181,10 @@
 
     public abstract JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException;
 
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
     protected void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
         metaSerde =
@@ -288,8 +297,10 @@
             sefs[i] = secondaryFieldAccessEvalFactories[i];
         }
         AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        assign.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixAssignOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
                 primaryPartitionConstraint);
         return asterixAssignOp;
@@ -319,6 +330,7 @@
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
         sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+        castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { enforcedRecDesc });
     }
@@ -326,6 +338,7 @@
     protected IFunctionDescriptor createCastFunction(boolean strictCast) throws AlgebricksException {
         IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
                 .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
+        castFuncDesc.setSourceLocation(sourceLoc);
         castFuncDesc.setImmutableStates(enforcedItemType, itemType);
         return castFuncDesc;
     }
@@ -338,6 +351,7 @@
         }
         ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
                 physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+        sortOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, primaryPartitionConstraint);
         return sortOp;
     }
@@ -351,6 +365,7 @@
         LSMIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new LSMIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 primaryIndexDataflowHelperFactory, BulkLoadUsage.CREATE_INDEX, dataset.getDatasetId());
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -362,6 +377,7 @@
         ExternalIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkLoadOperatorDescriptor(spec,
                 secondaryRecDesc, fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
                 ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, metadataProvider));
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -371,7 +387,9 @@
             RecordDescriptor secondaryRecDesc) throws AlgebricksException {
         IScalarEvaluatorFactory[] andArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
         NotDescriptor notDesc = new NotDescriptor();
+        notDesc.setSourceLocation(sourceLoc);
         IsUnknownDescriptor isUnknownDesc = new IsUnknownDescriptor();
+        isUnknownDesc.setSourceLocation(sourceLoc);
         for (int i = 0; i < numSecondaryKeyFields; i++) {
             // Access column i, and apply 'is not null'.
             ColumnAccessEvalFactory columnAccessEvalFactory = new ColumnAccessEvalFactory(i);
@@ -386,14 +404,17 @@
             // Create conjunctive condition where all secondary index keys must
             // satisfy 'is not null'.
             AndDescriptor andDesc = new AndDescriptor();
+            andDesc.setSourceLocation(sourceLoc);
             selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
         } else {
             selectCond = andArgsEvalFactories[0];
         }
         StreamSelectRuntimeFactory select =
                 new StreamSelectRuntimeFactory(selectCond, null, BinaryBooleanInspector.FACTORY, false, -1, null);
+        select.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor asterixSelectOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { select }, new RecordDescriptor[] { secondaryRecDesc });
+        asterixSelectOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixSelectOp,
                 primaryPartitionConstraint);
         return asterixSelectOp;
@@ -420,7 +441,7 @@
         Pair<ExternalScanOperatorDescriptor, AlgebricksPartitionConstraint> indexingOpAndConstraints;
         try {
             indexingOpAndConstraints = ExternalIndexingOperations.createExternalIndexingOp(spec, metadataProvider,
-                    dataset, itemType, indexerDesc, externalFiles);
+                    dataset, itemType, indexerDesc, externalFiles, sourceLoc);
         } catch (Exception e) {
             throw new AlgebricksException(e);
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index d31ca3b..878aab6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -88,11 +88,12 @@
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
-                    indexType, RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
+                    sourceLoc, indexType,
+                    RecordUtil.toFullyQualifiedName(dataset.getDataverseName(), dataset.getDatasetName()));
         }
         if (numSecondaryKeys > 1) {
-            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, numSecondaryKeys,
-                    indexType, 1);
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD, sourceLoc,
+                    numSecondaryKeys, indexType, 1);
         }
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
                 || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
@@ -113,7 +114,7 @@
         if (numSecondaryKeys > 0) {
             secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
                     metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                    index.getKeyFieldNames().get(0), numPrimaryKeys);
+                    index.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
                     index.getKeyFieldNames().get(0), itemType);
             secondaryKeyType = keyTypePair.first;
@@ -123,9 +124,9 @@
             secondaryTypeTraits[0] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
         if (numFilterFields > 0) {
-            secondaryFieldAccessEvalFactories[numSecondaryKeys] =
-                    metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
-                            metadataProvider.getFunctionManager(), itemType, filterFieldName, numPrimaryKeys);
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
+                    .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), itemType, filterFieldName,
+                            numPrimaryKeys, sourceLoc);
             Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
             IAType type = keyTypePair.first;
             ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
@@ -242,8 +243,11 @@
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
                 secondaryPartitionConstraint);
 
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
         AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+        metaOp.setSourceLocation(sourceLoc);
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -271,6 +275,7 @@
         BinaryTokenizerOperatorDescriptor tokenizerOp =
                 new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
                         primaryKeyFields, isPartitioned, false, false, MissingWriterFactory.INSTANCE);
+        tokenizerOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
         return tokenizerOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 8e6e0e9..1d6677e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -99,7 +99,7 @@
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         secondaryFieldAccessEvalFactories = metadataProvider.getDataFormat().createMBRFactory(
                 metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
-                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR);
+                secondaryKeyFields.get(0), recordColumn, numDimensions, filterFieldName, isPointMBR, sourceLoc);
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields =
@@ -228,8 +228,11 @@
             // Create secondary RTree bulk load op.
             TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, fieldPermutation,
                     indexDataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] {});
+            metaOp.setSourceLocation(sourceLoc);
             // Connect the operators.
             spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
@@ -285,9 +288,11 @@
                 secondaryBulkLoadOp = createExternalIndexBulkLoadOp(spec, fieldPermutation, indexDataflowHelperFactory,
                         GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             }
+            SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+            sinkRuntimeFactory.setSourceLocation(sourceLoc);
             AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                    new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
-                    new RecordDescriptor[] { secondaryRecDesc });
+                    new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+            metaOp.setSourceLocation(sourceLoc);
             spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
             root = metaOp;
             spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index 185cf8c..d78e80c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -62,6 +62,7 @@
                 secondaryFileSplitProvider, resourceFactory, true);
         IndexCreateOperatorDescriptor secondaryIndexCreateOp =
                 new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
+        secondaryIndexCreateOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -78,6 +79,7 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
         // The index drop operation should be persistent regardless of temp datasets or permanent dataset.
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, options);
+        btreeDrop.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
                 splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
@@ -93,6 +95,7 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
         LSMTreeIndexCompactOperatorDescriptor compactOp =
                 new LSMTreeIndexCompactOperatorDescriptor(spec, dataflowHelperFactory);
+        compactOp.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
         spec.addRoot(compactOp);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index 4ef3933..efe795b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -24,7 +24,10 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -51,8 +54,9 @@
      */
     public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
             List<Index> indexes) throws AlgebricksException {
-        ARecordType enforcedRecordType = recordType;
+        IAType enforcedRecordType = recordType;
         ARecordType enforcedMetaType = metaType;
+        List<String> subFieldName;
         for (Index index : indexes) {
             if (!index.isSecondaryIndex() || !index.isOverridingKeyFieldTypes()) {
                 continue;
@@ -61,21 +65,28 @@
                 throw new AlgebricksException("Indexing an open field is only supported on the record part");
             }
             for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                Deque<Pair<ARecordType, String>> nestedTypeStack = new ArrayDeque<>();
+                // keeps track of a record type and a field name in that record type
+                Deque<Pair<IAType, String>> nestedTypeStack = new ArrayDeque<>();
                 List<String> splits = index.getKeyFieldNames().get(i);
-                ARecordType nestedFieldType = enforcedRecordType;
+                IAType nestedFieldType = enforcedRecordType;
                 boolean openRecords = false;
                 String bridgeName = nestedFieldType.getTypeName();
                 int j;
-                // Build the stack for the enforced type
+                // enforcedRecordType must always be/stay as ARecordType
+                validateRecord(enforcedRecordType);
+                // build the stack for the enforced type, stack of a mixture of ARecord and AUnion(ARecord) types
+                // try to build up to the last record field, e.g. for a.b.c.d.e, build up to and including "d"
                 for (j = 1; j < splits.size(); j++) {
                     nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
                     bridgeName = nestedFieldType.getTypeName();
-                    nestedFieldType = (ARecordType) enforcedRecordType.getSubFieldType(splits.subList(0, j));
+                    subFieldName = splits.subList(0, j);
+                    nestedFieldType = ((ARecordType) enforcedRecordType).getSubFieldType(subFieldName);
                     if (nestedFieldType == null) {
                         openRecords = true;
                         break;
                     }
+                    // nestedFieldType (i.e. nested record field) must be either ARecordType or AUnion(ARecordType)
+                    validateNestedRecord(nestedFieldType, subFieldName);
                 }
                 if (openRecords) {
                     // create the smallest record
@@ -87,19 +98,23 @@
                         enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
                                 new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
                     }
-                    // Bridge the gap
-                    Pair<ARecordType, String> gapPair = nestedTypeStack.pop();
-                    ARecordType parent = gapPair.first;
+                    // bridge the gap. Update the parent type to include the new optional field, e.g. c.d.e
+                    Pair<IAType, String> gapPair = nestedTypeStack.pop();
+                    ARecordType parent = (ARecordType) TypeComputeUtils.getActualType(gapPair.first);
 
+                    // parent type must be "open" to allow inclusion of the non-declared field
                     IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
                             new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
                     enforcedRecordType = new ARecordType(bridgeName,
                             ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
                             parentFieldTypes, true);
+                    // make nullable/missable if the original parent was nullable/missable
+                    enforcedRecordType = keepUnknown(gapPair.first, (ARecordType) enforcedRecordType);
                 } else {
-                    //Schema is closed all the way to the field
-                    //enforced fields are either null or strongly typed
-                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(nestedFieldType);
+                    // schema is closed all the way to the field. Enforced fields are either null or strongly typed
+                    // e.g. nestedFieldType = a.b.c.d
+                    ARecordType lastNestedRecord = (ARecordType) TypeComputeUtils.getActualType(nestedFieldType);
+                    Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(lastNestedRecord);
                     // if a an enforced field already exists and the type is correct
                     IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
                     if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
@@ -108,44 +123,49 @@
                     }
                     if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
                             index.getKeyFieldTypes().get(i).getTypeTag())) {
-                        throw new AlgebricksException("Cannot enforce field " + index.getKeyFieldNames().get(i)
-                                + " to have type " + index.getKeyFieldTypes().get(i));
+                        throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                                "Cannot enforce field \"" + String.join(".", index.getKeyFieldNames().get(i))
+                                        + "\" to have type " + index.getKeyFieldTypes().get(i));
                     }
                     if (enforcedFieldType == null) {
                         recordNameTypesMap.put(splits.get(splits.size() - 1),
                                 AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
                     }
-                    enforcedRecordType = new ARecordType(nestedFieldType.getTypeName(),
+                    enforcedRecordType = new ARecordType(lastNestedRecord.getTypeName(),
                             recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
                             recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
-                            nestedFieldType.isOpen());
+                            lastNestedRecord.isOpen());
+                    // make nullable/missable if the original nestedFieldType was nullable/missable
+                    enforcedRecordType = keepUnknown(nestedFieldType, (ARecordType) enforcedRecordType);
                 }
 
                 // Create the enforced type for the nested fields in the schema, from the ground up
                 if (!nestedTypeStack.isEmpty()) {
                     while (!nestedTypeStack.isEmpty()) {
-                        Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
-                        ARecordType nestedRecType = nestedTypePair.first;
+                        Pair<IAType, String> nestedType = nestedTypeStack.pop();
+                        ARecordType nestedRecType = (ARecordType) TypeComputeUtils.getActualType(nestedType.first);
                         IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] =
-                                enforcedRecordType;
+                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedType.second)] = enforcedRecordType;
                         enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
                                 nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
+                        // make nullable/missable if the original nestedRecType was nullable/missable
+                        enforcedRecordType = keepUnknown(nestedType.first, (ARecordType) enforcedRecordType);
                     }
                 }
             }
         }
-        return new Pair<>(enforcedRecordType, enforcedMetaType);
+        // the final enforcedRecordType built must be ARecordType since the original dataset rec. type can't be nullable
+        validateRecord(enforcedRecordType);
+        return new Pair<>((ARecordType) enforcedRecordType, enforcedMetaType);
     }
 
     /**
      * Creates a map from name to type for fields in the passed type
      *
-     * @param recordType
-     *            the type to be mapped
+     * @param recordType the type to be mapped
      * @return a map mapping all fields to their types
      */
-    public static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+    private static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
         LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
         for (int j = 0; j < recordType.getFieldNames().length; j++) {
             recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
@@ -153,4 +173,43 @@
         return recordNameTypesMap;
     }
 
+    /**
+     * Maintains the {@code updatedRecordType} as nullable/missable (?) in case it was originally nullable/missable
+     * @param originalRecordType the original record type
+     * @param updatedRecordType the original record type being enforced/modified with new non-declared fields included
+     * @return {@code updatedRecordType}
+     */
+    private static IAType keepUnknown(IAType originalRecordType, ARecordType updatedRecordType) {
+        if (originalRecordType.getTypeTag() == ATypeTag.UNION) {
+            return AUnionType.createUnknownableType(updatedRecordType, updatedRecordType.getTypeName());
+        }
+        return updatedRecordType;
+    }
+
+    /**
+     * Makes sure the dataset record type being enforced/modified stays as a pure record type
+     * @param enforcedDatasetRecordType the dataset record type enforced and modified by adding the extra fields indexed
+     */
+    private static void validateRecord(IAType enforcedDatasetRecordType) {
+        if (enforcedDatasetRecordType.getTypeTag() != ATypeTag.OBJECT) {
+            throw new IllegalStateException("The dataset type must be a record type to be able to build an index");
+        }
+    }
+
+    /**
+     * Makes sure the chain of fields accessed and leading to the indexed field are all valid record types.
+     * E.g. for CREATE INDEX idx on ds(a.b.c.d: int) validate that a, b and c are all valid record types (?).
+     * @param nestedRecordType the nested record field being accessed
+     * @param fieldName the name of the nested record field
+     * @throws AsterixException when supplying bad fields, e.g. CREATE INDEX i on ds(a.b: int, a.b.c: int) (mostly
+     * for non-declared fields)
+     */
+    private static void validateNestedRecord(IAType nestedRecordType, List<String> fieldName) throws AsterixException {
+        IAType actualType = TypeComputeUtils.getActualType(nestedRecordType);
+        if (actualType.getTypeTag() != ATypeTag.OBJECT) {
+            String fName = String.join(".", fieldName);
+            throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+                    "Field accessor is not defined for \"" + fName + "\" of type " + actualType.getTypeTag());
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index 8960ba6..c9fb39c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -44,7 +44,7 @@
 
     public TupleCopyValueExtractor(ITypeTraits[] typeTraits) {
         this.tupleWriter = new TypeAwareTupleWriter(typeTraits);
-        this.tupleReference = (TypeAwareTupleReference) tupleWriter.createTupleReference();
+        this.tupleReference = tupleWriter.createTupleReference();
     }
 
     @Override
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 45cf7c8..f7f95a8 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -52,7 +52,6 @@
     <dependency>
       <groupId>org.reflections</groupId>
       <artifactId>reflections</artifactId>
-      <version>0.9.10</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -63,7 +62,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>1.10.19</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -122,5 +120,14 @@
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/ArrayListFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/ArrayListFactory.java
new file mode 100644
index 0000000..c8ca766
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/ArrayListFactory.java
@@ -0,0 +1,34 @@
+/*
+ * 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.asterix.builders;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.util.container.IObjectFactory;
+
+public class ArrayListFactory<T> implements IObjectFactory<List<T>, ATypeTag> {
+
+    @Override
+    public List<T> create(ATypeTag arg) {
+        return new ArrayList<>();
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 95aef79..40b42eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -69,7 +69,7 @@
     private int[] openFieldNameLengths;
 
     private int numberOfOpenFields;
-    private RuntimeRecordTypeInfo recTypeInfo;
+    private final RuntimeRecordTypeInfo recTypeInfo;
 
     public RecordBuilder() {
         this.closedPartOutputStream = new ByteArrayAccessibleOutputStream();
@@ -302,12 +302,7 @@
 
     @Override
     public int getFieldId(String fieldName) {
-        for (int i = 0; i < recType.getFieldNames().length; i++) {
-            if (recType.getFieldNames()[i].equals(fieldName)) {
-                return i;
-            }
-        }
-        return -1;
+        return recType.getFieldIndex(fieldName);
     }
 
     public IBinaryHashFunction getFieldNameHashFunction() {
@@ -317,4 +312,4 @@
     public IBinaryComparator getFieldNameComparator() {
         return utf8Comparator;
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AListElementTokenFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AListElementTokenFactory.java
index 0e81baf..f8d25f6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AListElementTokenFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AListElementTokenFactory.java
@@ -18,9 +18,14 @@
  */
 package org.apache.asterix.dataflow.data.common;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class AListElementTokenFactory implements ITokenFactory {
 
     private static final long serialVersionUID = 1L;
@@ -29,4 +34,14 @@
     public IToken createToken() {
         return new AListElementToken();
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new AListElementTokenFactory();
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
index 4025b28..4e51fef 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
@@ -18,10 +18,16 @@
  */
 package org.apache.asterix.dataflow.data.common;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class AOrderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -35,4 +41,17 @@
     public IBinaryTokenizer createTokenizer() {
         return new AOrderedListBinaryTokenizer(tokenFactory);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("tokenFactory", tokenFactory.toJson(registry));
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ITokenFactory tokenFactory = (ITokenFactory) registry.deserialize(json.get("tokenFactory"));
+        return new AOrderedListBinaryTokenizerFactory(tokenFactory);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
index 71df068..471fc88 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
@@ -18,10 +18,16 @@
  */
 package org.apache.asterix.dataflow.data.common;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class AUnorderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -35,4 +41,17 @@
     public IBinaryTokenizer createTokenizer() {
         return new AUnorderedListBinaryTokenizer(tokenFactory);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("tokenFactory", tokenFactory.toJson(registry));
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ITokenFactory tokenFactory = (ITokenFactory) registry.deserialize(json.get("tokenFactory"));
+        return new AOrderedListBinaryTokenizerFactory(tokenFactory);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
index e4cf000..1dcf6a6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ExpressionTypeComputer.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.dataflow.data.common;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -55,8 +57,10 @@
                 try {
                     return env.getVarType(((VariableReferenceExpression) expr).getVariableReference());
                 } catch (Exception e) {
-                    throw new AlgebricksException("Could not resolve type for " + expr.toString() + ","
-                            + "please check whether the used variable has been defined!", e);
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+                            "Could not resolve type for " + expr.toString() + ","
+                                    + "please check whether the used variable has been defined!",
+                            e);
                 }
             default:
                 throw new IllegalStateException();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
index dfb1251..33bef5d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MergeAggregationExpressionFactory.java
@@ -42,6 +42,7 @@
         AggregateFunctionCallExpression agg = (AggregateFunctionCallExpression) expr;
         FunctionIdentifier fid = agg.getFunctionIdentifier();
         VariableReferenceExpression tempVarExpr = new VariableReferenceExpression(originalProducedVar);
+        tempVarExpr.setSourceLocation(agg.getSourceLocation());
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<Mutable<ILogicalExpression>>();
         Mutable<ILogicalExpression> mutableExpression = new MutableObject<ILogicalExpression>(tempVarExpr);
         arguments.add(mutableExpression);
@@ -56,6 +57,9 @@
              */
             return null;
         }
-        return BuiltinFunctions.makeAggregateFunctionExpression(mergeFid, arguments);
+        AggregateFunctionCallExpression mergeExpr =
+                BuiltinFunctions.makeAggregateFunctionExpression(mergeFid, arguments);
+        mergeExpr.setSourceLocation(agg.getSourceLocation());
+        return mergeExpr;
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
index 561df5e..b7bd8ca 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
@@ -106,7 +106,7 @@
         // Gets the actual types for UNIONs and mark unknownable to be true.
         if (leftTypeTag == ATypeTag.UNION || rightTypeTag == ATypeTag.UNION) {
             leftType = TypeComputeUtils.getActualType(leftType);
-            rightType = TypeComputeUtils.getActualType(leftType);
+            rightType = TypeComputeUtils.getActualType(rightType);
             leftTypeTag = leftType.getTypeTag();
             rightTypeTag = rightType.getTypeTag();
             unknownable = true;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java
index 961b4dd..7eeb05d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java
@@ -24,6 +24,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class ACirclePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -83,4 +87,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java
index ed9c051..4f0e0ef 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java
@@ -23,6 +23,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class ADurationPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -61,4 +66,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalAscPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalAscPartialBinaryComparatorFactory.java
index fb2b112..3b086f2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalAscPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalAscPartialBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class AIntervalAscPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -60,4 +65,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalDescPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalDescPartialBinaryComparatorFactory.java
index 32994ab..6ef58fd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalDescPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalDescPartialBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class AIntervalDescPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -61,4 +66,14 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java
index 405e063..5227ebc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java
@@ -24,6 +24,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class ALinePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -100,4 +104,14 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
index 61b47c2..99a26ca 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
@@ -27,6 +27,8 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.data.std.primitive.BytePointable;
@@ -37,6 +39,8 @@
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class AObjectAscBinaryComparatorFactory implements IBinaryComparatorFactory {
 
     private static final long serialVersionUID = 1L;
@@ -323,4 +327,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescBinaryComparatorFactory.java
index 3260527..5de502c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescBinaryComparatorFactory.java
@@ -23,6 +23,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class AObjectDescBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -56,4 +60,14 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java
index dfba1d1..58baef5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java
@@ -23,7 +23,12 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APoint3DSerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class APoint3DPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -80,4 +85,14 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java
index b0eeb2b..920545f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java
@@ -24,6 +24,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class APointPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -64,4 +68,14 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java
index 577f6a3..3879072 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java
@@ -24,8 +24,12 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class APolygonPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
     private static final long serialVersionUID = 1L;
@@ -89,4 +93,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
index 12f358f..7c0301e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class ARectanglePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
     private static final long serialVersionUID = 1L;
@@ -63,4 +68,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
index 0810341..2276fe3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AUUIDPartialBinaryComparatorFactory.java
@@ -21,8 +21,13 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.primitive.LongPointable;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class AUUIDPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
 
     private static final long serialVersionUID = 1L;
@@ -45,4 +50,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/BooleanBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/BooleanBinaryComparatorFactory.java
index 400be60..7374951 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/BooleanBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/BooleanBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class BooleanBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -48,4 +53,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
index cff39a70..2db54c8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
@@ -24,6 +24,8 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
@@ -32,6 +34,8 @@
 import org.apache.hyracks.data.std.primitive.UTF8StringLowercasePointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class ListItemBinaryComparatorFactory implements IBinaryComparatorFactory {
 
     private static final long serialVersionUID = 1L;
@@ -179,4 +183,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LongBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LongBinaryComparatorFactory.java
index 9964c4e..278a2e5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LongBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LongBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class LongBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -44,4 +49,13 @@
         };
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java
index 73b53b8..df03941 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java
@@ -21,6 +21,11 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
 
@@ -48,4 +53,14 @@
 
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
index 21880dd..e58f210 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
@@ -20,6 +20,7 @@
 
 import java.io.PrintStream;
 
+import org.apache.asterix.dataflow.data.nontagged.printers.json.clean.AGeometryPrinterFactory;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -116,6 +117,9 @@
             case SHORTWITHOUTTYPEINFO:
                 ShortWithoutTypeInfoPrinterFactory.PRINTER.print(b, s, l, ps);
                 return true;
+            case GEOMETRY:
+                AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
             default:
                 return false;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
new file mode 100644
index 0000000..8e51e35
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.dataflow.data.nontagged.printers.json.clean;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.PrintStream;
+
+public class AGeometryPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final AGeometryPrinterFactory INSTANCE = new AGeometryPrinterFactory();
+
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ByteArrayInputStream inStream = new ByteArrayInputStream(b, s + 1, l - 1);
+        DataInput dataIn = new DataInputStream(inStream);
+        OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+        ps.print(geometry.asGeoJson());
+    };
+
+    @Override
+    public IPrinter createPrinter() {
+        return PRINTER;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
index 5cae68c..27fc7eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
@@ -113,6 +113,9 @@
             case UUID:
                 AUUIDPrinterFactory.PRINTER.print(b, s, l, ps);
                 return true;
+            case GEOMETRY:
+                AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
             default:
                 return false;
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
new file mode 100644
index 0000000..0a74ab7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
@@ -0,0 +1,76 @@
+/*
+ * 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.asterix.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.esri.core.geometry.OperatorImportFromWkb;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WkbImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AGeometrySerializerDeserializer implements ISerializerDeserializer<AGeometry> {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AGeometrySerializerDeserializer INSTANCE = new AGeometrySerializerDeserializer();
+
+    /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+    public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+
+    private AGeometrySerializerDeserializer() {
+    }
+
+    @Override
+    public AGeometry deserialize(DataInput in) throws HyracksDataException {
+        try {
+            int length = in.readInt();
+            byte[] bytes = new byte[length];
+            in.readFully(bytes);
+            ByteBuffer buffer = ByteBuffer.wrap(bytes);
+            OGCGeometry geometry = OGCGeometry.createFromOGCStructure(
+                    OperatorImportFromWkb.local().executeOGC(WkbImportFlags.wkbImportDefaults, buffer, null),
+                    DEFAULT_CRS);
+            return new AGeometry(geometry);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    public void serialize(AGeometry instance, DataOutput out) throws HyracksDataException {
+        try {
+            OGCGeometry geometry = instance.getGeometry();
+            byte[] buffer = geometry.asBinary().array();
+            // For efficiency, we store the size of the geometry in bytes in the first 32 bits
+            // This allows AsterixDB to skip over this attribute if needed.
+            out.writeInt(buffer.length);
+            out.write(buffer);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index aef4ca1..57f3449 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.ADuration;
 import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AInt16;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
@@ -120,6 +121,8 @@
                 return AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
             case MULTISET:
                 return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
+            case GEOMETRY:
+                return AGeometrySerializerDeserializer.INSTANCE.deserialize(in);
             default:
                 throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
         }
@@ -213,6 +216,9 @@
             case TYPE:
                 ATypeSerializerDeserializer.INSTANCE.serialize((IAType) instance, out);
                 break;
+            case GEOMETRY:
+                AGeometrySerializerDeserializer.INSTANCE.serialize((AGeometry) instance, out);
+                break;
             default:
                 throw new HyracksDataException(
                         "No serializer/deserializer implemented for type " + t.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
index 00291cf..b44aa5b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
@@ -20,11 +20,9 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
 import org.apache.asterix.om.base.APoint3D;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 97c49e0..2685e48 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -305,8 +305,8 @@
         return offset + AInt32SerializerDeserializer.getInt(serRecord, pointer + nullBitmapSize + (4 * fieldId));
     }
 
-    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart)
-            throws HyracksDataException {
+    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart,
+            IBinaryHashFunction nameHashFunction, IBinaryComparator nameComparator) throws HyracksDataException {
         // 5 is the index of the byte that determines whether the record is expanded or not, i.e. it has an open part.
         if (hasNoFields(serRecord, start, len) || serRecord[start + 5] != 1) {
             return -1;
@@ -316,14 +316,7 @@
         int numberOfOpenField = AInt32SerializerDeserializer.getInt(serRecord, openPartOffset);
         int fieldUtflength = UTF8StringUtil.getUTFLength(fieldName, nstart + 1);
         int fieldUtfMetaLen = UTF8StringUtil.getNumBytesToStoreLength(fieldUtflength);
-
-        IBinaryHashFunction utf8HashFunction =
-                BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
-
-        IBinaryComparator utf8BinaryComparator =
-                BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
-
-        int fieldNameHashCode = utf8HashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
+        int fieldNameHashCode = nameHashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
 
         int offset = openPartOffset + 4;
         int fieldOffset = -1;
@@ -337,7 +330,7 @@
             if (h == fieldNameHashCode) {
                 fieldOffset = start + AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * mid) + 4);
                 // the utf8 comparator do not require to put the precise length, we can just pass a estimated limit.
-                if (utf8BinaryComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
+                if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                         fieldUtflength + fieldUtfMetaLen) == 0) {
                     // since they are equal, we can directly use the meta length and the utf length.
                     return fieldOffset + fieldUtfMetaLen + fieldUtflength;
@@ -346,7 +339,7 @@
                         h = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j));
                         if (h == fieldNameHashCode) {
                             fieldOffset = start + AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j) + 4);
-                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
+                            if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                                     fieldUtflength) == 0) {
                                 return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                             }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
index cf1e573..1a8cc5e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
@@ -20,10 +20,8 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.om.base.APoint;
 import org.apache.asterix.om.base.ARectangle;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
index 888b34c..4bb9f08 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
@@ -69,4 +69,8 @@
             throw HyracksDataException.create(e);
         }
     }
-}
+
+    public void serialize(char[] buffer, int start, int length, DataOutput out) throws IOException {
+        UTF8StringUtil.writeUTF8(buffer, start, length, out, utf8StringWriter);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index 438a9bf..751146a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -24,7 +24,6 @@
 import java.util.ArrayList;
 
 import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AUnorderedList;
 import org.apache.asterix.om.base.IACursor;
@@ -36,7 +35,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeTagUtil;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/valueproviders/PrimitiveValueProviderFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/valueproviders/PrimitiveValueProviderFactory.java
index abc235d..91fccda 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/valueproviders/PrimitiveValueProviderFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/valueproviders/PrimitiveValueProviderFactory.java
@@ -21,12 +21,17 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.impls.FloatPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.impls.IntegerPrimitiveValueProviderFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class PrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
 
     private static final long serialVersionUID = 1L;
@@ -62,4 +67,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 678a864..6906ae4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IDataFormat {
     public ISerializerDeserializerProvider getSerdeProvider();
@@ -66,16 +67,17 @@
     public IMissingWriterFactory getMissingWriterFactory();
 
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException;
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName, SourceLocation sourceLoc)
+            throws AlgebricksException;
 
     public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn) throws AlgebricksException;
+            List<String> fldName, int recordColumn, SourceLocation sourceLoc) throws AlgebricksException;
 
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException;
 
     public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
-            throws AlgebricksException;
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR,
+            SourceLocation sourceLoc) throws AlgebricksException;
 
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AnyBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AnyBinaryComparatorFactory.java
new file mode 100644
index 0000000..d74efa6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AnyBinaryComparatorFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.asterix.formats.nontagged;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+public class AnyBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return 0;
+            }
+        };
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new AnyBinaryComparatorFactory();
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
index fd4df0d..1dfa05b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
@@ -18,11 +18,14 @@
  */
 package org.apache.asterix.formats.nontagged;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class BinaryBooleanInspector implements IBinaryBooleanInspector {
     private static final BinaryBooleanInspector INSTANCE = new BinaryBooleanInspector();
@@ -36,11 +39,13 @@
         }
     };
 
+    private static final String NAME = "boolean-inspector";
+
     private BinaryBooleanInspector() {
     }
 
     @Override
-    public boolean getBooleanValue(byte[] bytes, int offset, int length) {
+    public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException {
         byte serializedTypeTag = bytes[offset];
         if (serializedTypeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                 || serializedTypeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
@@ -49,8 +54,9 @@
         /** check if the runtime type is boolean */
         ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serializedTypeTag);
         if (typeTag != ATypeTag.BOOLEAN) {
-            throw new IllegalStateException("Runtime error: the select condition should be of the boolean type!");
+            throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH_FUNCTION, NAME, 0, ATypeTag.BOOLEAN, typeTag);
         }
+
         return bytes[offset + 1] == 1;
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
index 603d34e..297b4c6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
@@ -20,7 +20,6 @@
 
 import java.io.Serializable;
 
-import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
 import org.apache.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalAscPartialBinaryComparatorFactory;
@@ -38,9 +37,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.data.std.primitive.BytePointable;
@@ -114,21 +111,7 @@
                 return anyBinaryComparatorFactory(ascending);
             case NULL:
             case MISSING:
-                return new IBinaryComparatorFactory() {
-
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public IBinaryComparator createBinaryComparator() {
-                        return new IBinaryComparator() {
-
-                            @Override
-                            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-                                return 0;
-                            }
-                        };
-                    }
-                };
+                return new AnyBinaryComparatorFactory();
             case BOOLEAN:
                 return addOffset(BooleanBinaryComparatorFactory.INSTANCE, ascending);
             case TINYINT:
@@ -176,33 +159,7 @@
     }
 
     private IBinaryComparatorFactory addOffset(final IBinaryComparatorFactory inst, final boolean ascending) {
-        return new IBinaryComparatorFactory() {
-
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IBinaryComparator createBinaryComparator() {
-                final IBinaryComparator bc = inst.createBinaryComparator();
-                if (ascending) {
-                    return new ABinaryComparator() {
-
-                        @Override
-                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
-                                throws HyracksDataException {
-                            return bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
-                        }
-                    };
-                } else {
-                    return new ABinaryComparator() {
-                        @Override
-                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
-                                throws HyracksDataException {
-                            return -bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
-                        }
-                    };
-                }
-            }
-        };
+        return new OrderedBinaryComparatorFactory(inst, ascending);
     }
 
     private IBinaryComparatorFactory anyBinaryComparatorFactory(boolean ascending) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LinearizeComparatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LinearizeComparatorFactoryProvider.java
index 91fb96a..57c5048 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LinearizeComparatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LinearizeComparatorFactoryProvider.java
@@ -24,9 +24,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.data.ILinearizeComparatorFactoryProvider;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.rtree.linearize.HilbertDoubleComparatorFactory;
 import org.apache.hyracks.storage.am.rtree.linearize.ZCurveDoubleComparatorFactory;
 import org.apache.hyracks.storage.am.rtree.linearize.ZCurveIntComparatorFactory;
@@ -58,46 +56,6 @@
     }
 
     private ILinearizeComparatorFactory addOffset(final IBinaryComparatorFactory inst, final boolean ascending) {
-        return new ILinearizeComparatorFactory() {
-
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ILinearizeComparator createBinaryComparator() {
-                final ILinearizeComparator bc = (ILinearizeComparator) inst.createBinaryComparator();
-                final int dimension = bc.getDimensions();
-                if (ascending) {
-                    return new ILinearizeComparator() {
-
-                        @Override
-                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
-                                throws HyracksDataException {
-                            return bc.compare(b1, s1 + 1, l1, b2, s2 + 1, l2);
-                        }
-
-                        @Override
-                        public int getDimensions() {
-                            // TODO Auto-generated method stub
-                            return dimension;
-                        }
-                    };
-                } else {
-                    return new ILinearizeComparator() {
-
-                        @Override
-                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
-                                throws HyracksDataException {
-                            return -bc.compare(b1, s1 + 1, l1, b2, s2 + 1, l2);
-                        }
-
-                        @Override
-                        public int getDimensions() {
-                            // TODO Auto-generated method stub
-                            return dimension;
-                        }
-                    };
-                }
-            }
-        };
+        return new OrderedLinearizeComparatorFactory(inst, ascending);
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedBinaryComparatorFactory.java
new file mode 100644
index 0000000..c76ec42
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedBinaryComparatorFactory.java
@@ -0,0 +1,75 @@
+/*
+ * 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.asterix.formats.nontagged;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+public class OrderedBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final boolean ascending;
+    private final IBinaryComparatorFactory factory;
+
+    public OrderedBinaryComparatorFactory(IBinaryComparatorFactory factory, boolean ascending) {
+        this.factory = factory;
+        this.ascending = ascending;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final IBinaryComparator bc = factory.createBinaryComparator();
+        if (ascending) {
+            return new ABinaryComparator() {
+
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException {
+                    return bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                }
+            };
+        } else {
+            return new ABinaryComparator() {
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException {
+                    return -bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                }
+            };
+        }
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("factory", factory.toJson(registry));
+        json.put("ascending", ascending);
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final IBinaryComparatorFactory factory = (IBinaryComparatorFactory) registry.deserialize(json.get("factory"));
+        return new OrderedBinaryComparatorFactory(factory, json.get("ascending").asBoolean());
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedLinearizeComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedLinearizeComparatorFactory.java
new file mode 100644
index 0000000..58ba0ad
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/OrderedLinearizeComparatorFactory.java
@@ -0,0 +1,87 @@
+/*
+ * 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.asterix.formats.nontagged;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
+import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+public class OrderedLinearizeComparatorFactory implements ILinearizeComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final boolean ascending;
+    private final IBinaryComparatorFactory factory;
+
+    public OrderedLinearizeComparatorFactory(IBinaryComparatorFactory factory, boolean ascending) {
+        this.factory = factory;
+        this.ascending = ascending;
+    }
+
+    @Override
+    public ILinearizeComparator createBinaryComparator() {
+        final ILinearizeComparator bc = (ILinearizeComparator) factory.createBinaryComparator();
+        final int dimension = bc.getDimensions();
+        if (ascending) {
+            return new ILinearizeComparator() {
+
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException {
+                    return bc.compare(b1, s1 + 1, l1, b2, s2 + 1, l2);
+                }
+
+                @Override
+                public int getDimensions() {
+                    return dimension;
+                }
+            };
+        } else {
+            return new ILinearizeComparator() {
+
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException {
+                    return -bc.compare(b1, s1 + 1, l1, b2, s2 + 1, l2);
+                }
+
+                @Override
+                public int getDimensions() {
+                    return dimension;
+                }
+            };
+        }
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("factory", factory.toJson(registry));
+        json.put("ascending", ascending);
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final IBinaryComparatorFactory factory = (IBinaryComparatorFactory) registry.deserialize(json.get("factory"));
+        return new OrderedLinearizeComparatorFactory(factory, json.get("ascending").asBoolean());
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
index 0bc1d2f..ab9f4e0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
@@ -48,36 +48,25 @@
                     @Override
                     public boolean evaluate(IFrameTupleAccessor fta0, int tupId0, IFrameTupleAccessor fta1,
                             int tupId1) {
-
-                        int tStart0 = fta0.getTupleStartOffset(tupId0);
-                        int fStartOffset0 = fta0.getFieldSlotsLength() + tStart0;
-
-                        for (int k0 : keys0) {
-                            int fieldStartIx = fta0.getFieldStartOffset(tupId0, k0);
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(fta0.getBuffer().array()[fieldStartIx + fStartOffset0]);
-                            if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
-                                return false;
-                            }
-                        }
-
-                        int tStart1 = fta1.getTupleStartOffset(tupId1);
-                        int fStartOffset1 = fta1.getFieldSlotsLength() + tStart1;
-
-                        for (int k1 : keys1) {
-                            int fieldStartIx = fta1.getFieldStartOffset(tupId1, k1);
-                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                                    .deserialize(fta1.getBuffer().array()[fieldStartIx + fStartOffset1]);
-                            if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
-                                return false;
-                            }
-                        }
-
-                        return true; //none of the fields (from both sides) is NULL
+                        return noNullOrMissingInKeys(fta0, tupId0, keys0) && noNullOrMissingInKeys(fta1, tupId1, keys1);
                     }
                 };
             }
         };
     }
 
+    private static boolean noNullOrMissingInKeys(IFrameTupleAccessor fta, int tupId, int[] keys) {
+        int tStart = fta.getTupleStartOffset(tupId);
+        int fStartOffset = fta.getFieldSlotsLength() + tStart;
+        for (int i = 0; i < keys.length; ++i) {
+            int key = keys[i];
+            int fieldStartIx = fta.getFieldStartOffset(tupId, key);
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                    .deserialize(fta.getBuffer().array()[fieldStartIx + fStartOffset]);
+            if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
index 81dc0c2..0cba8b6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
@@ -80,8 +81,7 @@
     // Can't be shared among threads <Stateful>
     @SuppressWarnings("rawtypes")
     public ISerializerDeserializer getAStringSerializerDeserializer() {
-        return addTag(new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader()),
-                ATypeTag.STRING);
+        return addTag(new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader()));
     }
 
     @SuppressWarnings("rawtypes")
@@ -97,7 +97,7 @@
                 // we could do smth better for nullable fields
                 return AObjectSerializerDeserializer.INSTANCE;
             default:
-                return addTag(getNonTaggedSerializerDeserializer(aqlType), aqlType.getTypeTag());
+                return addTag(getNonTaggedSerializerDeserializer(aqlType));
         }
     }
 
@@ -162,6 +162,8 @@
                 return AUUIDSerializerDeserializer.INSTANCE;
             case SHORTWITHOUTTYPEINFO:
                 return ShortSerializerDeserializer.INSTANCE;
+            case GEOMETRY:
+                return AGeometrySerializerDeserializer.INSTANCE;
             default:
                 throw new NotImplementedException(
                         "No serializer/deserializer implemented for type " + aqlType.getTypeTag() + " .");
@@ -169,7 +171,7 @@
     }
 
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer addTag(final ISerializerDeserializer nonTaggedSerde, final ATypeTag typeTag) {
+    private ISerializerDeserializer addTag(final ISerializerDeserializer nonTaggedSerde) {
         return new ISerializerDeserializer<IAObject>() {
 
             private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
index 9ec5a08..0fd2cf8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
@@ -21,22 +21,24 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.data.std.primitive.FixedLengthTypeTrait;
+import org.apache.hyracks.data.std.primitive.VarLengthTypeTrait;
 
 public class TypeTraitProvider implements ITypeTraitProvider {
 
     // WARNING: the byte sizes depend on the serializer!
     // currently assuming a serializer that adds a 1-byte type indicator before
     // the data
-    private static final ITypeTraits ONEBYTETYPETRAIT = new TypeTrait(1 + 1);
-    private static final ITypeTraits TWOBYTETYPETRAIT = new TypeTrait(2 + 1);
-    private static final ITypeTraits FOURBYTETYPETRAIT = new TypeTrait(4 + 1);
-    private static final ITypeTraits EIGHTBYTETYPETRAIT = new TypeTrait(8 + 1);
-    private static final ITypeTraits SIXTEENBYTETYPETRAIT = new TypeTrait(16 + 1);
-    private static final ITypeTraits SEVENTEENBYTETYPETRAIT = new TypeTrait(17 + 1);
-    private static final ITypeTraits THIRTYTWOBYTETYPETRAIT = new TypeTrait(32 + 1);
-    private static final ITypeTraits TWENTYFOURBYTETYPETRAIT = new TypeTrait(24 + 1);
+    private static final ITypeTraits ONEBYTETYPETRAIT = new FixedLengthTypeTrait(1 + 1);
+    private static final ITypeTraits TWOBYTETYPETRAIT = new FixedLengthTypeTrait(2 + 1);
+    private static final ITypeTraits FOURBYTETYPETRAIT = new FixedLengthTypeTrait(4 + 1);
+    private static final ITypeTraits EIGHTBYTETYPETRAIT = new FixedLengthTypeTrait(8 + 1);
+    private static final ITypeTraits SIXTEENBYTETYPETRAIT = new FixedLengthTypeTrait(16 + 1);
+    private static final ITypeTraits SEVENTEENBYTETYPETRAIT = new FixedLengthTypeTrait(17 + 1);
+    private static final ITypeTraits THIRTYTWOBYTETYPETRAIT = new FixedLengthTypeTrait(32 + 1);
+    private static final ITypeTraits TWENTYFOURBYTETYPETRAIT = new FixedLengthTypeTrait(24 + 1);
 
-    private static final ITypeTraits VARLENTYPETRAIT = new TypeTrait(false, -1);
+    private static final ITypeTraits VARLENTYPETRAIT = VarLengthTypeTrait.INSTANCE;
 
     public static final TypeTraitProvider INSTANCE = new TypeTraitProvider();
 
@@ -76,31 +78,4 @@
             }
         }
     }
-}
-
-class TypeTrait implements ITypeTraits {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public boolean isFixedLength() {
-        return isFixedLength;
-    }
-
-    @Override
-    public int getFixedLength() {
-        return fixedLength;
-    }
-
-    private boolean isFixedLength;
-    private int fixedLength;
-
-    public TypeTrait(boolean isFixedLength, int fixedLength) {
-        this.isFixedLength = isFixedLength;
-        this.fixedLength = fixedLength;
-    }
-
-    public TypeTrait(int fixedLength) {
-        this.isFixedLength = true;
-        this.fixedLength = fixedLength;
-    }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmArrayNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmArrayNode.java
index e489cd5..60ef41a4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmArrayNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmArrayNode.java
@@ -18,12 +18,19 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
-public class AdmArrayNode implements IAdmNode {
+public class AdmArrayNode implements IAdmNode, Iterable<IAdmNode> {
+    private static final long serialVersionUID = 1L;
     private final List<IAdmNode> children;
 
     public AdmArrayNode() {
@@ -42,6 +49,10 @@
         add(AdmBooleanNode.get(value));
     }
 
+    public int size() {
+        return children.size();
+    }
+
     public void add(IAdmNode value) {
         if (value == null) {
             value = AdmNullNode.INSTANCE; // NOSONAR
@@ -74,4 +85,23 @@
     public String toString() {
         return children.toString();
     }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        OrderedListBuilder listBuilder = new OrderedListBuilder();
+        listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+        for (int i = 0; i < children.size(); i++) {
+            itemValue.reset();
+            IAdmNode next = children.get(i);
+            next.serialize(itemValue.getDataOutput());
+            listBuilder.addItem(itemValue);
+        }
+        listBuilder.write(dataOutput, false);
+    }
+
+    @Override
+    public Iterator<IAdmNode> iterator() {
+        return children.iterator();
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBigIntNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBigIntNode.java
index 0006668..61afc74 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBigIntNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBigIntNode.java
@@ -18,9 +18,14 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AdmBigIntNode implements IAdmNode {
+
+    private static final long serialVersionUID = 1L;
     private long value;
 
     public AdmBigIntNode() {
@@ -52,4 +57,9 @@
     public String toString() {
         return Long.toString(value);
     }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        dataOutput.writeLong(value);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBooleanNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBooleanNode.java
index aeebd86..721ce4c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBooleanNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmBooleanNode.java
@@ -18,10 +18,14 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AdmBooleanNode implements IAdmNode {
 
+    private static final long serialVersionUID = 1L;
     public static final AdmBooleanNode TRUE = new AdmBooleanNode(true);
     public static final AdmBooleanNode FALSE = new AdmBooleanNode(false);
     private final boolean value;
@@ -56,4 +60,9 @@
     public String toString() {
         return Boolean.toString(value);
     }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        dataOutput.writeBoolean(value);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmDoubleNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmDoubleNode.java
index 0150b4e..0601e63 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmDoubleNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmDoubleNode.java
@@ -18,9 +18,13 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AdmDoubleNode implements IAdmNode {
+    private static final long serialVersionUID = 1L;
     private double value;
 
     public AdmDoubleNode() {
@@ -53,4 +57,9 @@
     public String toString() {
         return Double.toString(value);
     }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        dataOutput.writeDouble(value);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmNullNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmNullNode.java
index 2804ca3..6fe8357 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmNullNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmNullNode.java
@@ -18,10 +18,14 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.asterix.om.types.ATypeTag;
 
 public class AdmNullNode implements IAdmNode {
 
+    private static final long serialVersionUID = 1L;
     public static final AdmNullNode INSTANCE = new AdmNullNode();
 
     private AdmNullNode() {
@@ -41,4 +45,9 @@
     public String toString() {
         return "null";
     }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        // Nothing to write
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
index e35b28f..99b7176 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
@@ -18,23 +18,43 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 /**
  * An adm object instance
  */
 public class AdmObjectNode implements IAdmNode {
+
+    private static final long serialVersionUID = 1L;
+    public static final AdmObjectNode EMPTY = new AdmObjectNode(Collections.emptyMap());
     private final Map<String, IAdmNode> children;
 
     public AdmObjectNode() {
         children = new HashMap<>();
     }
 
+    public AdmObjectNode(Map<String, IAdmNode> children) {
+        this.children = children;
+    }
+
     @Override
     public ATypeTag getType() {
         return ATypeTag.OBJECT;
@@ -56,10 +76,13 @@
         return children.entrySet();
     }
 
-    public AdmObjectNode set(String fieldName, IAdmNode value) {
+    public AdmObjectNode set(String fieldName, IAdmNode value) throws CompilationException {
         if (value == null) {
             value = AdmNullNode.INSTANCE; // NOSONAR
         }
+        if (children.containsKey(fieldName)) {
+            throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, fieldName);
+        }
         children.put(fieldName, value);
         return this;
     }
@@ -92,4 +115,51 @@
         sb.append("}");
         return sb.toString();
     }
+
+    public static AdmObjectNode from(Map<String, IAdmNode> fields) {
+        return fields.isEmpty() ? EMPTY : new AdmObjectNode(fields);
+    }
+
+    public boolean isEmpty() {
+        return children.isEmpty();
+    }
+
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        ISerializerDeserializer<AString> stringSerde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+        RecordBuilder confRecordBuilder = new RecordBuilder();
+        confRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+        confRecordBuilder.init();
+        ArrayBackedValueStorage fieldNameBytes = new ArrayBackedValueStorage();
+        ArrayBackedValueStorage fieldValueBytes = new ArrayBackedValueStorage();
+        for (Entry<String, IAdmNode> field : getFields()) {
+            String fieldName = field.getKey();
+            fieldValueBytes.reset();
+            fieldNameBytes.reset();
+            stringSerde.serialize(new AString(fieldName), fieldNameBytes.getDataOutput());
+            IAdmNode value = field.getValue();
+            value.serialize(fieldValueBytes.getDataOutput());
+            confRecordBuilder.addField(fieldNameBytes, fieldValueBytes);
+        }
+        confRecordBuilder.write(dataOutput, false);
+    }
+
+    public boolean contains(String fieldName) {
+        return children.containsKey(fieldName);
+    }
+
+    public String getString(String field) throws HyracksDataException {
+        return getString(this, field);
+    }
+
+    public static String getString(AdmObjectNode openFields, String field) throws HyracksDataException {
+        IAdmNode node = openFields.get(field);
+        if (node == null) {
+            throw HyracksDataException.create(ErrorCode.FIELD_NOT_FOUND, field);
+        } else if (node.getType() != ATypeTag.STRING) {
+            throw HyracksDataException.create(ErrorCode.FIELD_NOT_OF_TYPE, field, ATypeTag.STRING, node.getType());
+        }
+        return ((AdmStringNode) node).get();
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmStringNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmStringNode.java
index 4c92f09..3dc70d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmStringNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmStringNode.java
@@ -18,9 +18,15 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class AdmStringNode implements IAdmNode {
+
+    private static final long serialVersionUID = 1L;
     private String value;
 
     public AdmStringNode() {
@@ -49,4 +55,9 @@
         return "\"" + value + "\"";
     }
 
+    @Override
+    public void serializeValue(DataOutput dataOutput) throws IOException {
+        UTF8StringUtil.writeUTF8(value, dataOutput, null);
+    }
+
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/IAdmNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/IAdmNode.java
index c382fd2..98d7fe4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/IAdmNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/IAdmNode.java
@@ -18,12 +18,16 @@
  */
 package org.apache.asterix.object.base;
 
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
 import org.apache.asterix.om.types.ATypeTag;
 
 /**
  * An interface representing an adm node
  */
-public interface IAdmNode {
+public interface IAdmNode extends Serializable {
 
     /**
      * @return true if the object is a value, false if the object is a container
@@ -62,4 +66,23 @@
      * reset the node
      */
     void reset();
+
+    /**
+     * Serialize the field with a type tag
+     *
+     * @param dataOutput
+     * @throws IOException
+     */
+    default void serialize(DataOutput dataOutput) throws IOException {
+        dataOutput.writeByte(getType().serialize());
+        serializeValue(dataOutput);
+    }
+
+    /**
+     * Serialize the value without a type tag
+     *
+     * @param dataOutput
+     * @throws IOException
+     */
+    void serializeValue(DataOutput dataOutput) throws IOException;
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
new file mode 100644
index 0000000..3b9c55d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
@@ -0,0 +1,77 @@
+/*
+ * 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.asterix.om.base;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+import java.io.IOException;
+
+public class AGeometry implements IAObject {
+
+    protected OGCGeometry geometry;
+
+    public AGeometry(OGCGeometry geometry) {
+        this.geometry = geometry;
+    }
+
+    public OGCGeometry getGeometry() {
+        return geometry;
+    }
+
+    @Override
+    public IAType getType() {
+        return BuiltinType.AGEOMETRY;
+    }
+
+    @Override
+    public boolean deepEqual(IAObject obj) {
+        if (!(obj instanceof AGeometry)) {
+            return false;
+        } else {
+            AGeometry p = (AGeometry) obj;
+            return p.geometry.equals(geometry);
+        }
+    }
+
+    @Override
+    public int hash() {
+        return geometry.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return geometry.toString();
+    }
+
+    @Override
+    public ObjectNode toJSON() {
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode json = null;
+        try {
+            json = (ObjectNode) om.readTree(geometry.asGeoJson());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return json;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
index 150575f..4fb0b67 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
@@ -71,7 +71,7 @@
 
     @Override
     public int hashCode() {
-        return (int) (((int) (this.intervalStart ^ (this.intervalStart >>> 32))) * 31
+        return (((int) (this.intervalStart ^ (this.intervalStart >>> 32))) * 31
                 + (int) (this.intervalEnd ^ (this.intervalEnd >>> 32))) * 31 + (int) this.typetag;
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
new file mode 100644
index 0000000..346d68a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
@@ -0,0 +1,46 @@
+/*
+ * 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.asterix.om.base;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AMutableGeometry extends AGeometry {
+
+    private OperatorImportFromWkt wktImporter;
+
+    public AMutableGeometry(OGCGeometry geom) {
+        super(geom);
+        wktImporter = OperatorImportFromWkt.local();
+    }
+
+    public void setValue(OGCGeometry geom) {
+        this.geometry = geom;
+    }
+
+    public void parseWKT(String wkt) {
+        OGCStructure structure;
+
+        structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, wkt, null);
+        this.geometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
index 2fb69ab..9a097dc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
@@ -23,15 +23,16 @@
 
 public class AMutableUUID extends AUUID {
 
-    public void parseUUIDString(String tokenImage) throws HyracksDataException {
-        if (tokenImage.length() != UUID_CHARS) {
-            throw new HyracksDataException("This is not a correct UUID value: " + tokenImage);
+    private final byte[] hexBytesBuffer = new byte[UUID_CHARS];
+
+    public void parseUUIDString(char[] buffer, int begin, int len) throws HyracksDataException {
+        if (len != UUID_CHARS) {
+            throw new HyracksDataException("This is not a correct UUID value: " + new String(buffer, begin, len));
         }
-        byte[] hexBytes = new byte[UUID_CHARS];
-        for (int i = 0; i < tokenImage.length(); i++) {
-            hexBytes[i] = (byte) tokenImage.charAt(i);
+        for (int i = 0; i < len; i++) {
+            hexBytesBuffer[i] = (byte) buffer[begin + i];
         }
-        parseUUIDHexBytes(hexBytes, 0);
+        parseUUIDHexBytes(hexBytesBuffer, 0);
     }
 
     public void parseUUIDHexBytes(byte[] serString, int offset) throws HyracksDataException {
@@ -122,4 +123,4 @@
         }
     }
 
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
index e6b8f7c8..678ab16 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AString.java
@@ -27,6 +27,7 @@
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class AString implements IAObject, Serializable {
+    private static final long serialVersionUID = -5482850888016554079L;
 
     protected String value;
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
index 995a1ec..66fb4c3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
@@ -73,8 +73,8 @@
         HOUR,
         MIN,
         MILLISEC,
-        SEC;
-    };
+        SEC
+    }
 
     public enum ADurationParseOption {
         YEAR_MONTH,
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
index 291d963..39f5b3a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
@@ -63,7 +63,6 @@
      * @throws HyracksDataException
      */
     public static int parseTimePart(String timeString, int start, int length) throws HyracksDataException {
-
         int offset = 0;
 
         int hour = 0, min = 0, sec = 0, millis = 0;
@@ -518,4 +517,4 @@
         return timezone;
     }
 
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
index f837e36..98abb97 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -148,20 +148,20 @@
     };
 
     private static final byte[][] TIMEZONE_IDS;
-    private static final int[] TIMEZONE_OFFSETS;
+    private static final TimeZone[] TIMEZONE_VALUES;
 
     static {
         String[] tzIds = TimeZone.getAvailableIDs();
         int tzCount = tzIds.length;
         TIMEZONE_IDS = new byte[tzCount][];
-        TIMEZONE_OFFSETS = new int[tzCount];
+        TIMEZONE_VALUES = new TimeZone[tzCount];
 
         for (int i = 0; i < tzCount; i++) {
             TIMEZONE_IDS[i] = tzIds[i].getBytes(ENCODING);
         }
         Arrays.sort(TIMEZONE_IDS, byteArrayComparator);
         for (int i = 0; i < tzCount; i++) {
-            TIMEZONE_OFFSETS[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING)).getRawOffset();
+            TIMEZONE_VALUES[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING));
         }
     }
 
@@ -241,9 +241,10 @@
         return -1;
     }
 
-    private int binaryTimezoneIDSearch(byte[] barray, int start, int length) {
-        return Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
+    public static TimeZone findTimeZone(byte[] barray, int start, int length) {
+        int idx = Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
                 Arrays.copyOfRange(barray, start, start + length), byteArrayComparator);
+        return idx >= 0 ? TIMEZONE_VALUES[idx] : null;
     }
 
     private int indexOf(byte[] barray, int start, int length, char c) {
@@ -683,10 +684,10 @@
                                 || data[dataStart + timezoneEndField] == '_')) {
                             timezoneEndField++;
                         }
-                        int searchIdx = binaryTimezoneIDSearch(data, dataStart + dataStringPointer,
-                                timezoneEndField - dataStringPointer);
-                        if (searchIdx >= 0) {
-                            timezone = TIMEZONE_OFFSETS[searchIdx];
+                        TimeZone tz =
+                                findTimeZone(data, dataStart + dataStringPointer, timezoneEndField - dataStringPointer);
+                        if (tz != null) {
+                            timezone = tz.getRawOffset();
                         } else {
                             if (raiseParseDataError) {
                                 throw new AsterixTemporalTypeParseException("Unexpected timezone string: " + new String(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
index ab07620..deb15c8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
@@ -55,7 +55,7 @@
         MINUTE,
         SECOND,
         MILLISECOND
-    };
+    }
 
     //public static final int YEAR = 0, MONTH = 1, DAY = 2, HOUR = 3, MINUTE = 4, SECOND = 5, MILLISECOND = 6;
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
index 71a5649..3035bd5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/IncompatibleTypeException.java
@@ -23,8 +23,10 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IncompatibleTypeException extends CompilationException {
+    private static final long serialVersionUID = 8381231750372706059L;
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
     public IncompatibleTypeException(FunctionIdentifier fid, ATypeTag typeTagLeft, ATypeTag typeTagRight) {
@@ -32,7 +34,19 @@
     }
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag typeTagLeft,
+            ATypeTag typeTagRight) {
+        super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, sourceLoc, fid.getName(), typeTagLeft, typeTagRight);
+    }
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
     public IncompatibleTypeException(String functionName, ATypeTag typeTagLeft, ATypeTag typeTagRight) {
         super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, functionName, typeTagLeft, typeTagRight);
     }
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, String functionName, ATypeTag typeTagLeft,
+            ATypeTag typeTagRight) {
+        super(ErrorCode.COMPILATION_TYPE_INCOMPATIBLE, sourceLoc, functionName, typeTagLeft, typeTagRight);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
index 699629b..c3c8fd2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/InvalidExpressionException.java
@@ -27,8 +27,10 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class InvalidExpressionException extends CompilationException {
+    private static final long serialVersionUID = -3601791480135148643L;
 
     public InvalidExpressionException(FunctionIdentifier fid, int index, ILogicalExpression actualExpr,
             LogicalExpressionTag... exprKinds) {
@@ -36,9 +38,21 @@
                 toExpectedTypeString(exprKinds));
     }
 
+    public InvalidExpressionException(SourceLocation sourceLoc, FunctionIdentifier fid, int index,
+            ILogicalExpression actualExpr, LogicalExpressionTag... exprKinds) {
+        super(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, fid.getName(), indexToPosition(index),
+                actualExpr.toString(), toExpectedTypeString(exprKinds));
+    }
+
     public InvalidExpressionException(String functionName, int index, ILogicalExpression actualExpr,
             LogicalExpressionTag... exprKinds) {
         super(ErrorCode.COMPILATION_INVALID_EXPRESSION, functionName, indexToPosition(index), actualExpr.toString(),
                 toExpectedTypeString(exprKinds));
     }
+
+    public InvalidExpressionException(SourceLocation sourceLoc, String functionName, int index,
+            ILogicalExpression actualExpr, LogicalExpressionTag... exprKinds) {
+        super(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, functionName, indexToPosition(index),
+                actualExpr.toString(), toExpectedTypeString(exprKinds));
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
index 7218a3a..bcf8b84 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
@@ -26,20 +26,46 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeMismatchException extends CompilationException {
+    private static final long serialVersionUID = -3069967719104299912L;
 
-    // Parameter type mistmatch.
+    // Function parameter type mismatch.
     public TypeMismatchException(FunctionIdentifier fid, Integer i, ATypeTag actualTypeTag,
             ATypeTag... expectedTypeTags) {
-        super(ErrorCode.COMPILATION_TYPE_MISMATCH, fid.getName(), indexToPosition(i),
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, fid.getName(), indexToPosition(i),
                 toExpectedTypeString(expectedTypeTags), actualTypeTag);
     }
 
-    // Parameter type mistmatch.
+    // Function parameter type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, ATypeTag actualTypeTag,
+            ATypeTag... expectedTypeTags) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags), actualTypeTag);
+    }
+
+    // Function parameter type mismatch.
     public TypeMismatchException(String functionName, Integer i, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
-        super(ErrorCode.COMPILATION_TYPE_MISMATCH, functionName, indexToPosition(i),
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, functionName, indexToPosition(i),
                 toExpectedTypeString(expectedTypeTags), actualTypeTag);
     }
 
+    // Function parameter type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, ATypeTag actualTypeTag,
+            ATypeTag... expectedTypeTags) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, sourceLoc, functionName, indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags), actualTypeTag);
+    }
+
+    // Generic type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_GENERIC, sourceLoc, toExpectedTypeString(expectedTypeTags),
+                actualTypeTag);
+    }
+
+    // Generic type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, ATypeTag actualTypeTag, String expectedType) {
+        super(ErrorCode.COMPILATION_TYPE_MISMATCH_GENERIC, sourceLoc, expectedType, actualTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
index 0393887..d0f0bf2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedItemTypeException.java
@@ -23,8 +23,10 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedItemTypeException extends CompilationException {
+    private static final long serialVersionUID = -8192292060995420548L;
 
     // Unsupported item type.
     public UnsupportedItemTypeException(FunctionIdentifier fid, ATypeTag itemTypeTag) {
@@ -32,7 +34,17 @@
     }
 
     // Unsupported item type.
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag itemTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_ITEM, sourceLoc, fid.getName(), itemTypeTag);
+    }
+
+    // Unsupported item type.
     public UnsupportedItemTypeException(String functionName, ATypeTag itemTypeTag) {
         super(ErrorCode.COMPILATION_TYPE_ITEM, functionName, itemTypeTag);
     }
+
+    // Unsupported item type.
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, String functionName, ATypeTag itemTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_ITEM, sourceLoc, functionName, itemTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
index bc69a48..c4a8372 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/UnsupportedTypeException.java
@@ -23,8 +23,10 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedTypeException extends CompilationException {
+    private static final long serialVersionUID = -641723317638101625L;
 
     // Unsupported input type.
     public UnsupportedTypeException(FunctionIdentifier fid, ATypeTag actualTypeTag) {
@@ -32,7 +34,17 @@
     }
 
     // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, ATypeTag actualTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, fid.getName(), actualTypeTag);
+    }
+
+    // Unsupported input type.
     public UnsupportedTypeException(String funcName, ATypeTag actualTypeTag) {
         super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, funcName, actualTypeTag);
     }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, String funcName, ATypeTag actualTypeTag) {
+        super(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, funcName, actualTypeTag);
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
index 35440e4..3aea6dc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AbstractFunctionDescriptor.java
@@ -18,25 +18,29 @@
  */
 package org.apache.asterix.om.functions;
 
-import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractFunctionDescriptor implements IFunctionDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    @Override
-    public abstract FunctionIdentifier getIdentifier();
+    protected SourceLocation sourceLoc;
 
     @Override
-    public abstract FunctionDescriptorTag getFunctionDescriptorTag();
+    public void setImmutableStates(Object... states) {
+    }
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
@@ -66,5 +70,4 @@
             throws AlgebricksException {
         throw new NotImplementedException("Not Implemented: " + getIdentifier());
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 8cd18fc..49db062 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -36,12 +36,16 @@
 import org.apache.asterix.om.typecomputer.impl.ADoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ADurationTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AFloatTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AGeometryTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt16TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt32TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt64TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AInt8TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AIntervalTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ALineTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AListFirstTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AListMultiListArgsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AListTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.APoint3DTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.APointTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.APolygonTypeComputer;
@@ -52,6 +56,9 @@
 import org.apache.asterix.om.typecomputer.impl.AUUIDTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AYearMonthDurationTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.AnyTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ArrayIfNullTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ArrayRangeTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ArrayRepeatTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.BooleanFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.BooleanOnlyTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.BooleanOrMissingTypeComputer;
@@ -61,23 +68,28 @@
 import org.apache.asterix.om.typecomputer.impl.CollectionMemberResultType;
 import org.apache.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ConcatTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.DoubleIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessByNameResultType;
 import org.apache.asterix.om.typecomputer.impl.FieldAccessNestedResultType;
 import org.apache.asterix.om.typecomputer.impl.FullTextContainsResultTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.GetOverlappingInvervalTypeComputer;
-import org.apache.asterix.om.typecomputer.impl.IfNanOrInfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfMissingOrNullTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfMissingTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.IfNanOrInfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.IfNullTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.LocalAvgTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.MinMaxAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.MissingIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NonTaggedGetItemResultType;
 import org.apache.asterix.om.typecomputer.impl.NotUnknownTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NullIfTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericAddSubMulDivTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NumericDivideTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericDoubleOutputFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericInt8OutputFunctionTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.NumericRound2TypeComputer;
@@ -93,7 +105,6 @@
 import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
-import org.apache.asterix.om.typecomputer.impl.RecordPairsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordRemoveFieldsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ScalarVersionOfAggregateResultType;
 import org.apache.asterix.om.typecomputer.impl.SleepTypeComputer;
@@ -111,6 +122,7 @@
 import org.apache.asterix.om.typecomputer.impl.ToDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToNumberTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToObjectTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.TreatAsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryBinaryInt64TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryMinusTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryStringInt64TypeComputer;
@@ -130,15 +142,12 @@
     }
 
     private static final FunctionInfoRepository registeredFunctions = new FunctionInfoRepository();
-
     private static final Map<IFunctionInfo, ATypeHierarchy.Domain> registeredFunctionsDomain = new HashMap<>();
 
     // it is supposed to be an identity mapping
     private static final Map<IFunctionInfo, IFunctionInfo> builtinPublicFunctionsSet = new HashMap<>();
     private static final Map<IFunctionInfo, IFunctionInfo> builtinPrivateFunctionsSet = new HashMap<>();
-
     private static final Map<IFunctionInfo, IResultTypeComputer> funTypeComputer = new HashMap<>();
-
     private static final Set<IFunctionInfo> builtinAggregateFunctions = new HashSet<>();
     private static final Map<IFunctionInfo, IFunctionToDataSourceRewriter> datasourceFunctions = new HashMap<>();
     private static final Set<IFunctionInfo> similarityFunctions = new HashSet<>();
@@ -159,14 +168,12 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-handle", 2);
     public static final FunctionIdentifier GET_DATA =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-data", 2);
-
     public static final FunctionIdentifier GET_ITEM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item", 2);
     public static final FunctionIdentifier ANY_COLLECTION_MEMBER =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "any-collection-member", 1);
     public static final FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1);
     public static final FunctionIdentifier LEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1);
-
     public static final FunctionIdentifier CONCAT_NON_NULL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "concat-non-null", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier EMPTY_STREAM =
@@ -181,6 +188,50 @@
     public static final FunctionIdentifier DEEP_EQUAL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "deep-equal", 2);
 
+    // array functions
+    public static final FunctionIdentifier ARRAY_REMOVE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-remove", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_PUT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-put", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_PREPEND =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-prepend", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_APPEND =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-append", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_INSERT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-insert", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_POSITION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-position", 2);
+    public static final FunctionIdentifier ARRAY_REPEAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-repeat", 2);
+    public static final FunctionIdentifier ARRAY_REVERSE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-reverse", 1);
+    public static final FunctionIdentifier ARRAY_CONTAINS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-contains", 2);
+    public static final FunctionIdentifier ARRAY_DISTINCT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-distinct", 1);
+    public static final FunctionIdentifier ARRAY_SORT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-sort", 1);
+    public static final FunctionIdentifier ARRAY_UNION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-union", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_INTERSECT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-intersect", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_IFNULL =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-ifnull", 1);
+    public static final FunctionIdentifier ARRAY_CONCAT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-concat", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_RANGE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-range", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_FLATTEN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-flatten", 2);
+    public static final FunctionIdentifier ARRAY_REPLACE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-replace", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_SYMDIFF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-symdiff", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_SYMDIFFN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-symdiffn", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier ARRAY_STAR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-star", 1);
+
     // objects
     public static final FunctionIdentifier RECORD_MERGE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-merge", 2);
@@ -213,6 +264,23 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-names", 1);
     public static final FunctionIdentifier RECORD_PAIRS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-pairs", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier GEOMETRY_CONSTRUCTOR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-geojson", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RECORD_REMOVE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-remove", 2);
+    public static final FunctionIdentifier RECORD_RENAME =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-rename", 3);
+    public static final FunctionIdentifier RECORD_UNWRAP =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-unwrap", 1);
+    public static final FunctionIdentifier RECORD_REPLACE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-replace", 3);
+    public static final FunctionIdentifier RECORD_ADD =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-add", 3);
+    public static final FunctionIdentifier RECORD_PUT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-put", 3);
+    public static final FunctionIdentifier RECORD_VALUES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-values", 1);
+    public static final FunctionIdentifier PAIRS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "pairs", 1);
 
     // numeric
     public static final FunctionIdentifier NUMERIC_UNARY_MINUS =
@@ -225,9 +293,10 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-divide", 2);
     public static final FunctionIdentifier NUMERIC_MOD =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-mod", 2);
-    public static final FunctionIdentifier NUMERIC_IDIV =
-            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-idiv", 2);
-    public static final FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
+    public static final FunctionIdentifier NUMERIC_DIV =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-div", 2);
+    public static final FunctionIdentifier NUMERIC_POWER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "power", 2);
     public static final FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "abs", 1);
     public static final FunctionIdentifier NUMERIC_ACOS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "acos", 1);
@@ -237,6 +306,10 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan", 1);
     public static final FunctionIdentifier NUMERIC_ATAN2 =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan2", 2);
+    public static final FunctionIdentifier NUMERIC_DEGREES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "degrees", 1);
+    public static final FunctionIdentifier NUMERIC_RADIANS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "radians", 1);
     public static final FunctionIdentifier NUMERIC_COS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cos", 1);
     public static final FunctionIdentifier NUMERIC_SIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sin", 1);
     public static final FunctionIdentifier NUMERIC_TAN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tan", 1);
@@ -247,6 +320,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sqrt", 1);
     public static final FunctionIdentifier NUMERIC_SIGN =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sign", 1);
+    public static final FunctionIdentifier NUMERIC_E = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "e", 0);
+    public static final FunctionIdentifier NUMERIC_PI = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "pi", 0);
 
     public static final FunctionIdentifier NUMERIC_CEILING =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ceiling", 1);
@@ -278,7 +353,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "find-binary", 2);
     public static final FunctionIdentifier FIND_BINARY_FROM =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "find-binary", 3);
-    // String funcitons
+
+    // String functions
     public static final FunctionIdentifier STRING_EQUAL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-equal", 2);
     public static final FunctionIdentifier STRING_MATCHES =
@@ -321,6 +397,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 3);
     public static final FunctionIdentifier STRING_REPLACE_WITH_LIMIT =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 4);
+    public static final FunctionIdentifier STRING_REVERSE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reverse", 1);
     public static final FunctionIdentifier STRING_LENGTH =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-length", 1);
     public static final FunctionIdentifier STRING_LIKE =
@@ -368,6 +446,7 @@
     public static final FunctionIdentifier MAKE_FIELD_NAME_HANDLE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "make-field-name-handle", 1);
 
+    // aggregate functions
     public static final FunctionIdentifier AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1);
     public static final FunctionIdentifier COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1);
     public static final FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
@@ -420,7 +499,6 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "intermediate-avg-serial", 1);
 
     // distinct aggregate functions
-
     public static final FunctionIdentifier COUNT_DISTINCT =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count-distinct", 1);
     public static final FunctionIdentifier SCALAR_COUNT_DISTINCT =
@@ -519,6 +597,7 @@
     public static final FunctionIdentifier SCALAR_SQL_MIN_DISTINCT =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-min-distinct", 1);
 
+    // unnesting functions
     public static final FunctionIdentifier SCAN_COLLECTION =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "scan-collection", 1);
     public static final FunctionIdentifier SUBSET_COLLECTION =
@@ -526,7 +605,7 @@
 
     public static final FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
 
-    // fuzzy functions:
+    // fuzzy functions
     public static final FunctionIdentifier FUZZY_EQ =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq", 2);
 
@@ -733,6 +812,129 @@
     public static final FunctionIdentifier UUID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "uuid", 0);
     public static final FunctionIdentifier CREATE_QUERY_UID =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-query-uid", 0);
+    public static final FunctionIdentifier RANDOM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 0);
+    public static final FunctionIdentifier RANDOM_WITH_SEED =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 1);
+
+    //Geo
+    public static final FunctionIdentifier ST_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-area", 1);
+    public static final FunctionIdentifier ST_MAKE_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 2);
+    public static final FunctionIdentifier ST_MAKE_POINT3D =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 3);
+    public static final FunctionIdentifier ST_MAKE_POINT3D_M =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 4);
+    public static final FunctionIdentifier ST_INTERSECTS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersects", 2);
+    public static final FunctionIdentifier ST_UNION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 2);
+    public static final FunctionIdentifier ST_IS_COLLECTION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-collection", 1);
+    public static final FunctionIdentifier ST_CONTAINS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-contains", 2);
+    public static final FunctionIdentifier ST_CROSSES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-crosses", 2);
+    public static final FunctionIdentifier ST_DISJOINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-disjoint", 2);
+    public static final FunctionIdentifier ST_EQUALS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-equals", 2);
+    public static final FunctionIdentifier ST_OVERLAPS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-overlaps", 2);
+    public static final FunctionIdentifier ST_TOUCHES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-touches", 2);
+    public static final FunctionIdentifier ST_WITHIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-within", 2);
+    public static final FunctionIdentifier ST_IS_EMPTY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-empty", 1);
+    public static final FunctionIdentifier ST_IS_SIMPLE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-simple", 1);
+    public static final FunctionIdentifier ST_COORD_DIM =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-coord-dim", 1);
+    public static final FunctionIdentifier ST_DIMENSION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-dimension", 1);
+    public static final FunctionIdentifier GEOMETRY_TYPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "geometry-type", 1);
+    public static final FunctionIdentifier ST_M = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-m", 1);
+    public static final FunctionIdentifier ST_N_RINGS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-rings", 1);
+    public static final FunctionIdentifier ST_N_POINTS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-points", 1);
+    public static final FunctionIdentifier ST_NUM_GEOMETRIIES =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-geometries", 1);
+    public static final FunctionIdentifier ST_NUM_INTERIOR_RINGS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-interior-rings", 1);
+    public static final FunctionIdentifier ST_SRID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-srid", 1);
+    public static final FunctionIdentifier ST_X = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x", 1);
+    public static final FunctionIdentifier ST_Y = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y", 1);
+    public static final FunctionIdentifier ST_X_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-max", 1);
+    public static final FunctionIdentifier ST_X_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-min", 1);
+    public static final FunctionIdentifier ST_Y_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-max", 1);
+    public static final FunctionIdentifier ST_Y_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-min", 1);
+    public static final FunctionIdentifier ST_Z = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z", 1);
+    public static final FunctionIdentifier ST_Z_MIN =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-min", 1);
+    public static final FunctionIdentifier ST_Z_MAX =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-max", 1);
+    public static final FunctionIdentifier ST_AS_BINARY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-binary", 1);
+    public static final FunctionIdentifier ST_AS_TEXT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-text", 1);
+    public static final FunctionIdentifier ST_AS_GEOJSON =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-geojson", 1);
+    public static final FunctionIdentifier ST_DISTANCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-distance", 2);
+    public static final FunctionIdentifier ST_LENGTH =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-length", 1);
+    public static final FunctionIdentifier SCALAR_ST_UNION_AGG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 1);
+    public static final FunctionIdentifier ST_UNION_AGG =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union-agg", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_TEXT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_TEXT_SRID =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 2);
+    public static final FunctionIdentifier ST_GEOM_FROM_WKB =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 1);
+    public static final FunctionIdentifier ST_GEOM_FROM_WKB_SRID =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 2);
+    public static final FunctionIdentifier ST_LINE_FROM_MULTIPOINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-line-from-multipoint", 1);
+    public static final FunctionIdentifier ST_MAKE_ENVELOPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-envelope", 5);
+    public static final FunctionIdentifier ST_IS_CLOSED =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-closed", 1);
+    public static final FunctionIdentifier ST_IS_RING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-ring", 1);
+    public static final FunctionIdentifier ST_RELATE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-relate", 3);
+    public static final FunctionIdentifier ST_BOUNDARY =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-boundary", 1);
+    public static final FunctionIdentifier ST_END_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-end-point", 1);
+    public static final FunctionIdentifier ST_ENVELOPE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-envelope", 1);
+    public static final FunctionIdentifier ST_EXTERIOR_RING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-exterior-ring", 1);
+    public static final FunctionIdentifier ST_GEOMETRY_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geometry-n", 2);
+    public static final FunctionIdentifier ST_INTERIOR_RING_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-interior-ring-n", 2);
+    public static final FunctionIdentifier ST_POINT_N =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-point-n", 2);
+    public static final FunctionIdentifier ST_START_POINT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-start-point", 1);
+    public static final FunctionIdentifier ST_DIFFERENCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-difference", 2);
+    public static final FunctionIdentifier ST_INTERSECTION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersection", 2);
+    public static final FunctionIdentifier ST_SYM_DIFFERENCE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-sym-difference", 2);
+    public static final FunctionIdentifier ST_POLYGONIZE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-polygonize", 1);
 
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR =
@@ -876,6 +1078,15 @@
     public static final FunctionIdentifier IF_NAN_OR_INF =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "if-nan-or-inf", FunctionIdentifier.VARARGS);
 
+    public static final FunctionIdentifier MISSING_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "missing-if", 2);
+    public static final FunctionIdentifier NULL_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "null-if", 2);
+    public static final FunctionIdentifier NAN_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nan-if", 2);
+    public static final FunctionIdentifier POSINF_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "posinf-if", 2);
+    public static final FunctionIdentifier NEGINF_IF =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "neginf-if", 2);
+
     public static final FunctionIdentifier TO_ATOMIC =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-atomic", 1);
     public static final FunctionIdentifier TO_ARRAY =
@@ -893,6 +1104,9 @@
     public static final FunctionIdentifier TO_STRING =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-string", 1);
 
+    public static final FunctionIdentifier TREAT_AS_INTEGER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "treat-as-integer", 1);
+
     public static final FunctionIdentifier EXTERNAL_LOOKUP =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "external-lookup", FunctionIdentifier.VARARGS);
 
@@ -937,14 +1151,13 @@
         addPrivateFunction(OR, BooleanFunctionTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_ADD, NumericAddSubMulDivTypeComputer.INSTANCE, true);
 
-        // Deep equality
+        // deep equality
         addFunction(DEEP_EQUAL, BooleanFunctionTypeComputer.INSTANCE, true);
 
         // and then, Asterix builtin functions
         addPrivateFunction(CHECK_UNKNOWN, NotUnknownTypeComputer.INSTANCE, true);
         addPrivateFunction(ANY_COLLECTION_MEMBER, CollectionMemberResultType.INSTANCE, true);
         addFunction(BOOLEAN_CONSTRUCTOR, ABooleanTypeComputer.INSTANCE, true);
-        addFunction(CARET, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addFunction(CIRCLE_CONSTRUCTOR, ACircleTypeComputer.INSTANCE, true);
         addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE, true);
 
@@ -960,6 +1173,8 @@
         addFunction(UUID, AUUIDTypeComputer.INSTANCE, false);
         addFunction(CREATE_QUERY_UID, ABinaryTypeComputer.INSTANCE, false);
         addFunction(UUID_CONSTRUCTOR, AUUIDTypeComputer.INSTANCE, true);
+        addFunction(RANDOM, ADoubleTypeComputer.INSTANCE, false);
+        addFunction(RANDOM_WITH_SEED, ADoubleTypeComputer.INSTANCE, false);
 
         addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE, true);
         addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE, true);
@@ -981,9 +1196,6 @@
         addPrivateFunction(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
         addPrivateFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
         addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
-        addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
-        addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
-        addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
         addPrivateFunction(INDEX_SEARCH, AnyTypeComputer.INSTANCE, true);
         addFunction(INT8_CONSTRUCTOR, AInt8TypeComputer.INSTANCE, true);
         addFunction(INT16_CONSTRUCTOR, AInt16TypeComputer.INSTANCE, true);
@@ -998,20 +1210,25 @@
         addPrivateFunction(NUMERIC_UNARY_MINUS, UnaryMinusTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_MULTIPLY, NumericAddSubMulDivTypeComputer.INSTANCE, true);
-        addPrivateFunction(NUMERIC_DIVIDE, NumericAddSubMulDivTypeComputer.INSTANCE, true);
-        addPrivateFunction(NUMERIC_MOD, NumericAddSubMulDivTypeComputer.INSTANCE, true);
-        addPrivateFunction(NUMERIC_IDIV, AInt64TypeComputer.INSTANCE, true);
+        addPrivateFunction(NUMERIC_DIVIDE, NumericDivideTypeComputer.INSTANCE, true);
+        addPrivateFunction(NUMERIC_MOD, NumericAddSubMulDivTypeComputer.INSTANCE_NULLABLE, true);
+        addPrivateFunction(NUMERIC_DIV, NumericAddSubMulDivTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(NUMERIC_ABS, NumericUnaryFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ACOS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ASIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ATAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ATAN2, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_DEGREES, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_RADIANS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_COS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_TAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_E, ADoubleTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_EXP, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_LN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_LOG, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_PI, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(NUMERIC_POWER, NumericAddSubMulDivTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SQRT, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_SIGN, NumericInt8OutputFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_CEILING, NumericUnaryFunctionTypeComputer.INSTANCE, true);
@@ -1020,14 +1237,11 @@
         addFunction(NUMERIC_ROUND_HALF_TO_EVEN, NumericUnaryFunctionTypeComputer.INSTANCE, true);
         addFunction(NUMERIC_ROUND_HALF_TO_EVEN2, NumericRound2TypeComputer.INSTANCE, true);
         addFunction(NUMERIC_TRUNC, NumericRound2TypeComputer.INSTANCE, true);
-        addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
-        addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
-        addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
 
         addFunction(BINARY_LENGTH, UnaryBinaryInt64TypeComputer.INSTANCE, true);
         addFunction(PARSE_BINARY, ABinaryTypeComputer.INSTANCE, true);
         addFunction(PRINT_BINARY, AStringTypeComputer.INSTANCE, true);
-        addFunction(BINARY_CONCAT, ABinaryTypeComputer.INSTANCE, true);
+        addFunction(BINARY_CONCAT, ConcatTypeComputer.INSTANCE_BINARY, true);
         addFunction(SUBBINARY_FROM, ABinaryTypeComputer.INSTANCE, true);
         addFunction(SUBBINARY_FROM_TO, ABinaryTypeComputer.INSTANCE, true);
         addFunction(FIND_BINARY, AInt64TypeComputer.INSTANCE, true);
@@ -1038,8 +1252,8 @@
         addFunction(STRING_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
         addFunction(STRING_TO_CODEPOINT, StringToInt64ListTypeComputer.INSTANCE, true);
         addFunction(CODEPOINT_TO_STRING, AStringTypeComputer.INSTANCE, true);
-        addFunction(STRING_CONCAT, AStringTypeComputer.INSTANCE, true);
-        addFunction(SUBSTRING2, StringIntToStringTypeComputer.INSTANCE, true);
+        addFunction(STRING_CONCAT, ConcatTypeComputer.INSTANCE_STRING, true);
+        addFunction(SUBSTRING2, StringIntToStringTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(STRING_LENGTH, UnaryStringInt64TypeComputer.INSTANCE, true);
         addFunction(STRING_LOWERCASE, StringStringTypeComputer.INSTANCE, true);
         addFunction(STRING_UPPERCASE, StringStringTypeComputer.INSTANCE, true);
@@ -1060,9 +1274,11 @@
         addFunction(STRING_REGEXP_POSITION, StringInt32TypeComputer.INSTANCE, true);
         addFunction(STRING_REGEXP_POSITION_WITH_FLAG, StringInt32TypeComputer.INSTANCE, true);
         addFunction(STRING_REGEXP_REPLACE, StringStringTypeComputer.INSTANCE, true);
-        addFunction(STRING_REGEXP_REPLACE_WITH_FLAG, StringStringTypeComputer.INSTANCE, true);
+        addFunction(STRING_REGEXP_REPLACE_WITH_FLAG,
+                StringIntToStringTypeComputer.INSTANCE_STRING_REGEXP_REPLACE_WITH_FLAG, true);
         addFunction(STRING_REPLACE, StringStringTypeComputer.INSTANCE, true);
         addFunction(STRING_REPLACE_WITH_LIMIT, StringIntToStringTypeComputer.INSTANCE_TRIPLE_STRING, true);
+        addFunction(STRING_REVERSE, StringStringTypeComputer.INSTANCE, true);
         addFunction(SUBSTRING_BEFORE, StringStringTypeComputer.INSTANCE, true);
         addFunction(SUBSTRING_AFTER, StringStringTypeComputer.INSTANCE, true);
         addPrivateFunction(STRING_EQUAL, StringBooleanTypeComputer.INSTANCE, true);
@@ -1087,6 +1303,21 @@
         addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
         addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
 
+        addPrivateFunction(TREAT_AS_INTEGER, TreatAsTypeComputer.INSTANCE_INTEGER, true);
+
+        addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+        addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
+        addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
+        addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
+        addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
+        addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+
+        addFunction(MISSING_IF, MissingIfTypeComputer.INSTANCE, true);
+        addFunction(NULL_IF, NullIfTypeComputer.INSTANCE, true);
+        addFunction(NAN_IF, DoubleIfTypeComputer.INSTANCE, true);
+        addFunction(POSINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+        addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+
         // Aggregate Functions
         addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
         addPrivateFunction(LOCAL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1198,6 +1429,71 @@
         addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE, true);
         addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE, true);
 
+        //geo functions
+        addFunction(ST_AREA, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT3D, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_POINT3D_M, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERSECTS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_UNION, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_CROSSES, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_DISJOINT, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_EQUALS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_OVERLAPS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_TOUCHES, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_WITHIN, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_EMPTY, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_SIMPLE, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_COORD_DIM, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_DIMENSION, AInt32TypeComputer.INSTANCE, true);
+        addFunction(GEOMETRY_TYPE, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_M, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_N_RINGS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_N_POINTS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_NUM_GEOMETRIIES, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_NUM_INTERIOR_RINGS, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_SRID, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ST_X, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_X_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_X_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Y_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z_MIN, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_Z_MAX, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_BINARY, ABinaryTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_TEXT, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_AS_GEOJSON, AStringTypeComputer.INSTANCE, true);
+        addFunction(ST_DISTANCE, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_LENGTH, ADoubleTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_TEXT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_TEXT_SRID, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_WKB, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOM_FROM_WKB_SRID, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_LINE_FROM_MULTIPOINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_MAKE_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_CLOSED, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_IS_RING, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_RELATE, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ST_BOUNDARY, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_END_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_EXTERIOR_RING, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_GEOMETRY_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERIOR_RING_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_POINT_N, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_START_POINT, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_INTERSECTION, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_SYM_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+        addPrivateFunction(ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(ST_POLYGONIZE, AGeometryTypeComputer.INSTANCE, true);
+
         // Binary functions
         addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
         addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
@@ -1216,6 +1512,29 @@
         addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(WORD_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
 
+        // array functions
+        addFunction(ARRAY_REMOVE, AListTypeComputer.INSTANCE_REMOVE, true);
+        addFunction(ARRAY_PUT, AListTypeComputer.INSTANCE_PUT, true);
+        addFunction(ARRAY_PREPEND, AListTypeComputer.INSTANCE_PREPEND, true);
+        addFunction(ARRAY_APPEND, AListTypeComputer.INSTANCE_APPEND, true);
+        addFunction(ARRAY_INSERT, AListTypeComputer.INSTANCE_INSERT, true);
+        addFunction(ARRAY_POSITION, AInt32TypeComputer.INSTANCE, true);
+        addFunction(ARRAY_REPEAT, ArrayRepeatTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_REVERSE, AListFirstTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_SORT, AListFirstTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_DISTINCT, AListFirstTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_UNION, AListMultiListArgsTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_INTERSECT, AListMultiListArgsTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_IFNULL, ArrayIfNullTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_CONCAT, AListMultiListArgsTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_RANGE, ArrayRangeTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_FLATTEN, AListFirstTypeComputer.INSTANCE_FLATTEN, true);
+        addFunction(ARRAY_REPLACE, AListTypeComputer.INSTANCE_REPLACE, true);
+        addFunction(ARRAY_SYMDIFF, AListMultiListArgsTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_SYMDIFFN, AListMultiListArgsTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_STAR, OpenARecordTypeComputer.INSTANCE, true);
+
         // objects
         addFunction(RECORD_MERGE, RecordMergeTypeComputer.INSTANCE, true);
         addFunction(RECORD_CONCAT, OpenARecordTypeComputer.INSTANCE, true);
@@ -1231,7 +1550,16 @@
         addFunction(GET_RECORD_FIELD_VALUE, FieldAccessNestedResultType.INSTANCE, true);
         addFunction(RECORD_LENGTH, AInt64TypeComputer.INSTANCE_NULLABLE, true);
         addFunction(RECORD_NAMES, OrderedListOfAStringTypeComputer.INSTANCE_NULLABLE, true);
-        addFunction(RECORD_PAIRS, RecordPairsTypeComputer.INSTANCE, true);
+        addFunction(RECORD_PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
+        addFunction(RECORD_REMOVE, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_RENAME, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_UNWRAP, AnyTypeComputer.INSTANCE, true);
+        addFunction(RECORD_REPLACE, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_ADD, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_PUT, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_VALUES, OrderedListOfAnyTypeComputer.INSTANCE, true);
 
         // temporal type accessors
         addFunction(ACCESSOR_TEMPORAL_YEAR, AInt64TypeComputer.INSTANCE, true);
@@ -1563,6 +1891,13 @@
 
         addDistinctAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM);
         addScalarAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM_DISTINCT);
+
+        // SPATIAL AGGREGATES
+
+        addAgg(ST_UNION_AGG);
+        addLocalAgg(ST_UNION_AGG, ST_UNION_AGG);
+        addIntermediateAgg(ST_UNION_AGG, ST_UNION_AGG);
+        addGlobalAgg(ST_UNION_AGG, ST_UNION_AGG);
     }
 
     static {
@@ -1712,19 +2047,19 @@
         registeredFunctions.put(fi, functionInfo);
     }
 
-    private static void addAgg(FunctionIdentifier fi) {
+    public static void addAgg(FunctionIdentifier fi) {
         builtinAggregateFunctions.add(getAsterixFunctionInfo(fi));
     }
 
-    private static void addLocalAgg(FunctionIdentifier fi, FunctionIdentifier localfi) {
+    public static void addLocalAgg(FunctionIdentifier fi, FunctionIdentifier localfi) {
         aggregateToLocalAggregate.put(getAsterixFunctionInfo(fi), getAsterixFunctionInfo(localfi));
     }
 
-    private static void addIntermediateAgg(FunctionIdentifier fi, FunctionIdentifier globalfi) {
+    public static void addIntermediateAgg(FunctionIdentifier fi, FunctionIdentifier globalfi) {
         aggregateToIntermediateAggregate.put(getAsterixFunctionInfo(fi), getAsterixFunctionInfo(globalfi));
     }
 
-    private static void addGlobalAgg(FunctionIdentifier fi, FunctionIdentifier globalfi) {
+    public static void addGlobalAgg(FunctionIdentifier fi, FunctionIdentifier globalfi) {
         aggregateToGlobalAggregate.put(getAsterixFunctionInfo(fi), getAsterixFunctionInfo(globalfi));
         globalAggregateFunctions.add(getAsterixFunctionInfo(globalfi));
     }
@@ -1733,15 +2068,15 @@
         builtinUnnestingFunctions.put(getAsterixFunctionInfo(fi), returnsUniqueValues);
     }
 
-    private static void addSerialAgg(FunctionIdentifier fi, FunctionIdentifier serialfi) {
+    public static void addSerialAgg(FunctionIdentifier fi, FunctionIdentifier serialfi) {
         aggregateToSerializableAggregate.put(getAsterixFunctionInfo(fi), getAsterixFunctionInfo(serialfi));
     }
 
-    private static void addScalarAgg(FunctionIdentifier fi, FunctionIdentifier scalarfi) {
+    public static void addScalarAgg(FunctionIdentifier fi, FunctionIdentifier scalarfi) {
         scalarToAggregateFunctionMap.put(getAsterixFunctionInfo(scalarfi), getAsterixFunctionInfo(fi));
     }
 
-    private static void addDistinctAgg(FunctionIdentifier distinctfi, FunctionIdentifier regularscalarfi) {
+    public static void addDistinctAgg(FunctionIdentifier distinctfi, FunctionIdentifier regularscalarfi) {
         distinctToRegularScalarAggregateFunctionMap.put(getAsterixFunctionInfo(distinctfi),
                 getAsterixFunctionInfo(regularscalarfi));
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/FunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/FunctionInfo.java
index 994b2a9..484f374 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/FunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/FunctionInfo.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 
 public class FunctionInfo implements IFunctionInfo {
+    private static final long serialVersionUID = 5460606629941107898L;
 
     private final FunctionIdentifier functionIdentifier;
     private final boolean isFunctional;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
index d9bb73a..1fc9ef9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionDescriptor.java
@@ -28,11 +28,13 @@
 import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IFunctionDescriptor extends Serializable {
 
-    default void setImmutableStates(Object... states) {
-    }
+    void setImmutableStates(Object... states);
+
+    void setSourceLocation(SourceLocation sourceLoc);
 
     FunctionIdentifier getIdentifier();
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AFlatValuePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AFlatValuePointable.java
index 86a8a29..4108aca 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AFlatValuePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AFlatValuePointable.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.om.pointables;
 
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.pointables.visitor.IVisitablePointableVisitor;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.container.IObjectFactory;
@@ -37,11 +36,7 @@
      * use object pool based allocator. The factory is not public so that it
      * cannot called in other places than PointableAllocator.
      */
-    static IObjectFactory<IVisitablePointable, IAType> FACTORY = new IObjectFactory<IVisitablePointable, IAType>() {
-        public AFlatValuePointable create(IAType type) {
-            return new AFlatValuePointable();
-        }
-    };
+    static IObjectFactory<AFlatValuePointable, IAType> FACTORY = type -> new AFlatValuePointable();
 
     /**
      * private constructor, to prevent arbitrary creation
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
index 5b85d00..15766b3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
@@ -47,11 +47,8 @@
      * DO NOT allow to create AListPointable object arbitrarily, force to use
      * object pool based allocator, in order to have object reuse.
      */
-    static IObjectFactory<IVisitablePointable, IAType> FACTORY = new IObjectFactory<IVisitablePointable, IAType>() {
-        public IVisitablePointable create(IAType type) {
-            return new AListVisitablePointable((AbstractCollectionType) type);
-        }
-    };
+    static IObjectFactory<AListVisitablePointable, IAType> FACTORY =
+            type -> new AListVisitablePointable((AbstractCollectionType) type);
 
     private final List<IVisitablePointable> items = new ArrayList<IVisitablePointable>();
     private final List<IVisitablePointable> itemTags = new ArrayList<IVisitablePointable>();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
index b6ba848..7089fd6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
@@ -49,12 +49,8 @@
      * DO NOT allow to create ARecordPointable object arbitrarily, force to use
      * object pool based allocator, in order to have object reuse
      */
-    static IObjectFactory<IVisitablePointable, IAType> FACTORY = new IObjectFactory<IVisitablePointable, IAType>() {
-        @Override
-        public IVisitablePointable create(IAType type) {
-            return new ARecordVisitablePointable((ARecordType) type);
-        }
-    };
+    static IObjectFactory<ARecordVisitablePointable, IAType> FACTORY =
+            type -> new ARecordVisitablePointable((ARecordType) type);
 
     // access results: field names, field types, and field values
     private final List<IVisitablePointable> fieldNames = new ArrayList<>();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/PointableAllocator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/PointableAllocator.java
index abd4841..91c47fd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/PointableAllocator.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/PointableAllocator.java
@@ -37,21 +37,21 @@
  */
 public class PointableAllocator {
 
-    private IObjectPool<IVisitablePointable, IAType> flatValueAllocator =
-            new ListObjectPool<IVisitablePointable, IAType>(AFlatValuePointable.FACTORY);
-    private IObjectPool<IVisitablePointable, IAType> recordValueAllocator =
-            new ListObjectPool<IVisitablePointable, IAType>(ARecordVisitablePointable.FACTORY);
-    private IObjectPool<IVisitablePointable, IAType> listValueAllocator =
-            new ListObjectPool<IVisitablePointable, IAType>(AListVisitablePointable.FACTORY);
+    private IObjectPool<AFlatValuePointable, IAType> flatValueAllocator =
+            new ListObjectPool<>(AFlatValuePointable.FACTORY);
+    private IObjectPool<ARecordVisitablePointable, IAType> recordValueAllocator =
+            new ListObjectPool<>(ARecordVisitablePointable.FACTORY);
+    private IObjectPool<AListVisitablePointable, IAType> listValueAllocator =
+            new ListObjectPool<>(AListVisitablePointable.FACTORY);
     private IObjectPool<AOrderedListType, IAType> orederedListTypeAllocator =
-            new ListObjectPool<AOrderedListType, IAType>(new IObjectFactory<AOrderedListType, IAType>() {
+            new ListObjectPool<>(new IObjectFactory<AOrderedListType, IAType>() {
                 @Override
                 public AOrderedListType create(IAType type) {
                     return new AOrderedListType(type, type.getTypeName() + "OrderedList");
                 }
             });
     private IObjectPool<AOrderedListType, IAType> unorederedListTypeAllocator =
-            new ListObjectPool<AOrderedListType, IAType>(new IObjectFactory<AOrderedListType, IAType>() {
+            new ListObjectPool<>(new IObjectFactory<AOrderedListType, IAType>() {
                 @Override
                 public AOrderedListType create(IAType type) {
                     return new AOrderedListType(type, type.getTypeName() + "UnorderedList");
@@ -126,11 +126,11 @@
             return flatValueAllocator.allocate(null);
     }
 
-    public IVisitablePointable allocateListValue(IAType type) {
+    public AListVisitablePointable allocateListValue(IAType type) {
         return listValueAllocator.allocate(type);
     }
 
-    public IVisitablePointable allocateRecordValue(IAType type) {
+    public ARecordVisitablePointable allocateRecordValue(IAType type) {
         return recordValueAllocator.allocate(type);
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
index 6eb7d4c..04ee28b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
@@ -75,7 +75,8 @@
             laccessorToCaster.put(accessor, caster);
         }
         if (arg.second.getTypeTag().equals(ATypeTag.ANY)) {
-            arg.second = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+            arg.second = accessor.ordered() ? DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE
+                    : DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
         }
         caster.castList(accessor, arg.first, (AbstractCollectionType) arg.second, this);
         return null;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
index ace6769..492a9d3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
@@ -21,7 +21,6 @@
 
 import java.io.DataOutput;
 import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.List;
 
 import org.apache.asterix.builders.OrderedListBuilder;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
index aa6e26b..671360a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
@@ -123,7 +123,7 @@
             openFields = new boolean[numInputFields];
             fieldNamesSortedIndex = new int[numInputFields];
         }
-        if (cachedReqType == null || !reqType.equals(cachedReqType)) {
+        if (!reqType.equals(cachedReqType)) {
             try {
                 loadRequiredType(reqType);
             } catch (IOException e) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
index ce04f6d..0063c96 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
@@ -24,12 +24,17 @@
 import org.apache.asterix.om.util.container.IObjectFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.data.std.primitive.BytePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.VarLengthTypeTrait;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 /*
  * This class interprets the binary data representation of an interval.
@@ -42,21 +47,9 @@
  */
 public class AIntervalPointable extends AbstractPointable {
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
+    public static final AIntervalPointableFactory FACTORY = new AIntervalPointableFactory();
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class AIntervalPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -66,9 +59,19 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static final IObjectFactory<IPointable, ATypeTag> ALLOCATOR = new IObjectFactory<IPointable, ATypeTag>() {
         @Override
@@ -99,6 +102,7 @@
         start.set(bytes, getStartOffset(), getStartSize());
     }
 
+    @Override
     public int getStartOffset() {
         return getTypeOffset() + getTypeSize();
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
index 7c74a69..1e3ba12 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
@@ -22,7 +22,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AbstractCollectionType;
@@ -30,16 +29,22 @@
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.data.std.primitive.BytePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.VarLengthTypeTrait;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 /*
  * This class interprets the binary data representation of a list.
  *
  * List {
+ *   byte tag;
  *   byte type;
  *   int length;
  *   int numberOfItems;
@@ -48,33 +53,34 @@
  */
 public class AListPointable extends AbstractPointable {
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+    public static final AListPointableFactory FACTORY = new AListPointableFactory();
+
+    public static final class AListPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
+        private AListPointableFactory() {
         }
 
         @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public IPointable createPointable() {
+        public AListPointable createPointable() {
             return new AListPointable();
         }
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static final IObjectFactory<IPointable, ATypeTag> ALLOCATOR = new IObjectFactory<IPointable, ATypeTag>() {
         @Override
@@ -169,7 +175,7 @@
             return getItemCountOffset() + getItemCountSize() + index * getFixedLength(inputType);
         } else {
             int offset = getItemCountOffset() + getItemCountSize() + index * ITEM_OFFSET_SIZE;
-            return IntegerPointable.getInteger(bytes, offset);
+            return start + IntegerPointable.getInteger(bytes, offset);
         }
     }
 
@@ -195,5 +201,4 @@
         }
         dOut.write(bytes, getItemOffset(inputType, index), getItemSize(inputType, index));
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
index 77491e8..4ffbf47 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
@@ -22,7 +22,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -33,15 +32,20 @@
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.data.std.primitive.BooleanPointable;
 import org.apache.hyracks.data.std.primitive.BytePointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.VarLengthTypeTrait;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 import org.apache.hyracks.util.string.UTF8StringWriter;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /*
  * This class interprets the binary data representation of a record.
  *
@@ -71,24 +75,10 @@
  */
 public class ARecordPointable extends AbstractPointable {
 
-    private final UTF8StringWriter utf8Writer = new UTF8StringWriter();
     public static final ARecordPointableFactory FACTORY = new ARecordPointableFactory();
+    private final UTF8StringWriter utf8Writer = new UTF8StringWriter();
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static class ARecordPointableFactory implements IPointableFactory {
+    public static final class ARecordPointableFactory implements IPointableFactory {
 
         private static final long serialVersionUID = 1L;
 
@@ -102,7 +92,17 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
+        }
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
         }
 
     }
@@ -230,6 +230,21 @@
         }
     }
 
+    /**
+     * This is always untagged
+     *
+     * @param recordType
+     * @param fieldId
+     * @param pointable
+     * @throws IOException
+     */
+    public void getClosedFieldValue(ARecordType recordType, int fieldId, IPointable pointable) throws IOException {
+        if (isClosedFieldNull(recordType, fieldId) || isClosedFieldMissing(recordType, fieldId)) {
+            throw new IllegalStateException("Can't read a null or missing field");
+        }
+        pointable.set(bytes, getClosedFieldOffset(recordType, fieldId), getClosedFieldSize(recordType, fieldId));
+    }
+
     public String getClosedFieldName(ARecordType recordType, int fieldId) {
         return recordType.getFieldNames()[fieldId];
     }
@@ -304,6 +319,14 @@
         dOut.write(bytes, getOpenFieldNameOffset(recordType, fieldId), getOpenFieldNameSize(recordType, fieldId));
     }
 
+    public String getOpenFieldName(ARecordType recordType, int fieldId) throws IOException {
+        StringBuilder str = new StringBuilder();
+        int offset = getOpenFieldNameOffset(recordType, fieldId);
+        UTF8StringUtil.toString(str, bytes, offset);
+        String fieldName = str.toString();
+        return fieldName;
+    }
+
     public int getOpenFieldNameSize(ARecordType recordType, int fieldId) {
         int utfleng = UTF8StringUtil.getUTFLength(bytes, getOpenFieldNameOffset(recordType, fieldId));
         return utfleng + UTF8StringUtil.getNumBytesToStoreLength(utfleng);
@@ -340,5 +363,4 @@
     public int getOpenFieldOffsetSize(ARecordType recordType, int fieldId) {
         return OPEN_FIELD_HASH_SIZE;
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
index 5352f3c..d1f6ab7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
@@ -19,11 +19,9 @@
 
 package org.apache.asterix.om.pointables.printer;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.List;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
index ef8c6da..3858bd6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.om.pointables.printer;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.List;
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
index 3dba6ef..3f22374 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/csv/APrintVisitor.java
@@ -21,7 +21,6 @@
 
 import java.io.PrintStream;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.printers.csv.AObjectPrinterFactory;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
index 4d6b7e6..60a4985 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/AbstractResultTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * This abstract class takes care of the handling of optional types.
@@ -39,23 +40,26 @@
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
         AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
         String funcName = functionCallExpression.getFunctionIdentifier().getName();
-        return TypeComputeUtils.resolveResultType(expression, env, (index, type) -> checkArgType(funcName, index, type),
-                this::getResultType, propagateNullAndMissing());
+        return TypeComputeUtils.resolveResultType(expression, env,
+                (index, type, sourceLoc) -> checkArgType(funcName, index, type, sourceLoc), this::getResultType,
+                propagateNullAndMissing());
     }
 
     /**
      * Checks whether an input type violates the requirement.
      *
-     * @param funcName
-     *            the function name.
      * @param argIndex,
      *            the index of the argument to consider.
      * @param type,
      *            the type of the input argument.
+     * @param funcName
+     *            the function name.
+     * @param sourceLoc
+     *            the source location
      * @throws AlgebricksException
      */
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
-
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
     }
 
     /**
@@ -64,7 +68,8 @@
      * @param expr
      *            the expression under consideration.
      * @param strippedInputTypes,
-     *            the stripped input types.
+     *            the stripped input types. When the function propagates null & missing, they can be any type except
+     *            null, missing, and union.
      * @return the result type without considering optional types.
      * @throws AlgebricksException
      */
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/IResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/IResultTypeComputer.java
index 04f9c96..e9dfe90 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/IResultTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/IResultTypeComputer.java
@@ -24,7 +24,8 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
+@FunctionalInterface
 public interface IResultTypeComputer {
-    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+    IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
index 8c40903..8aad9d7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/base/TypeCastUtils.java
@@ -19,12 +19,12 @@
 
 package org.apache.asterix.om.typecomputer.base;
 
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.om.exceptions.IncompatibleTypeException;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 
 public class TypeCastUtils {
@@ -33,7 +33,7 @@
     }
 
     public static boolean setRequiredAndInputTypes(AbstractFunctionCallExpression expr, IAType requiredType,
-            IAType inputType) throws AlgebricksException {
+            IAType inputType) throws CompilationException {
         boolean changed = false;
         Object[] opaqueParameters = expr.getOpaqueParameters();
         if (opaqueParameters == null) {
@@ -44,7 +44,7 @@
             ATypeTag actualTypeTag = TypeComputeUtils.getActualType(inputType).getTypeTag();
             if (!ATypeHierarchy.isCompatible(requiredTypeTag, actualTypeTag)) {
                 String funcName = expr.getFunctionIdentifier().getName();
-                throw new IncompatibleTypeException(funcName, actualTypeTag, requiredTypeTag);
+                throw new IncompatibleTypeException(expr.getSourceLocation(), funcName, actualTypeTag, requiredTypeTag);
             }
             expr.setOpaqueParameters(opaqueParameters);
             changed = true;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java
new file mode 100644
index 0000000..e85410ab5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class AGeometryTypeComputer extends AbstractResultTypeComputer {
+
+    public static final AGeometryTypeComputer INSTANCE = new AGeometryTypeComputer();
+
+    private AGeometryTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return BuiltinType.AGEOMETRY;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListFirstTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListFirstTypeComputer.java
new file mode 100644
index 0000000..660f4ff
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListFirstTypeComputer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+/**
+ * A type computer that returns the same list type as the presumably input list at argument 0. If the argument is not a
+ * list, it returns "ANY".
+ */
+public class AListFirstTypeComputer extends AbstractResultTypeComputer {
+    public static final AListFirstTypeComputer INSTANCE = new AListFirstTypeComputer(false, false);
+    public static final AListFirstTypeComputer INSTANCE_FLATTEN = new AListFirstTypeComputer(true, true);
+
+    private final boolean makeOpen;
+    private final boolean makeNullable;
+
+    private AListFirstTypeComputer(boolean makeOpen, boolean makeNullable) {
+        this.makeOpen = makeOpen;
+        this.makeNullable = makeNullable;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType argType = strippedInputTypes[0];
+        switch (argType.getTypeTag()) {
+            case ARRAY:
+            case MULTISET:
+                if (makeOpen) {
+                    argType = DefaultOpenFieldType.getDefaultOpenFieldType(argType.getTypeTag());
+                }
+                if (makeNullable) {
+                    return AUnionType.createNullableType(argType);
+                } else {
+                    return argType;
+                }
+            default:
+                return BuiltinType.ANY;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListMultiListArgsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListMultiListArgsTypeComputer.java
new file mode 100755
index 0000000..125281e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListMultiListArgsTypeComputer.java
@@ -0,0 +1,59 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+/**
+ * Returns a list that is missable/nullable. All input lists should have the same type. This is checked during runtime.
+ * List type is taken from one of the input args, [0]
+ */
+public class AListMultiListArgsTypeComputer extends AbstractResultTypeComputer {
+    public static final AListMultiListArgsTypeComputer INSTANCE = new AListMultiListArgsTypeComputer();
+
+    private AListMultiListArgsTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        if (strippedInputTypes.length < 2) {
+            String functionName = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier().getName();
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
+                    functionName);
+        }
+
+        IAType listType = strippedInputTypes[0];
+        if (listType.getTypeTag().isListType()) {
+            listType = DefaultOpenFieldType.getDefaultOpenFieldType(listType.getTypeTag());
+            return AUnionType.createUnknownableType(listType);
+        } else {
+            return BuiltinType.ANY;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
new file mode 100755
index 0000000..d1c9c48
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
@@ -0,0 +1,88 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+/**
+ * Returns a list that is missable/nullable. The list type is taken from one of the input args which is the input list.
+ */
+public class AListTypeComputer extends AbstractResultTypeComputer {
+    public static final AListTypeComputer INSTANCE_REMOVE = new AListTypeComputer(2, -1, false, false, true);
+    public static final AListTypeComputer INSTANCE_PUT = new AListTypeComputer(2, -1, false, true, true);
+    public static final AListTypeComputer INSTANCE_PREPEND = new AListTypeComputer(2, -1, true, true, false);
+    public static final AListTypeComputer INSTANCE_APPEND = new AListTypeComputer(2, -1, false, true, false);
+    public static final AListTypeComputer INSTANCE_INSERT = new AListTypeComputer(3, -1, false, true, false);
+    public static final AListTypeComputer INSTANCE_REPLACE = new AListTypeComputer(3, 4, false, true, false);
+
+    private final int minNumArgs;
+    private final int maxNumArgs;
+    private final boolean listIsLast;
+    private final boolean makeOpen;
+    private final boolean nullInNullOut;
+
+    private AListTypeComputer(int minNumArgs, int maxNumArgs, boolean listIsLast, boolean makeOpen,
+            boolean nullInNullOut) {
+        this.minNumArgs = minNumArgs;
+        this.maxNumArgs = maxNumArgs;
+        this.listIsLast = listIsLast;
+        this.makeOpen = makeOpen;
+        this.nullInNullOut = nullInNullOut;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        if (strippedInputTypes.length < minNumArgs || (maxNumArgs > 0 && strippedInputTypes.length > maxNumArgs)) {
+            String functionName = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier().getName();
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
+                    functionName);
+        }
+        // output type should be the same as as the type tag at [list index]. The output type is nullable/missable
+        // since the output could be null due to other invalid arguments or the tag at [list index] itself is not list
+        int listIndex = 0;
+        if (listIsLast) {
+            listIndex = strippedInputTypes.length - 1;
+        }
+
+        IAType listType = strippedInputTypes[listIndex];
+        if (listType.getTypeTag().isListType()) {
+            if (makeOpen) {
+                listType = DefaultOpenFieldType.getDefaultOpenFieldType(listType.getTypeTag());
+            }
+            return AUnionType.createUnknownableType(listType);
+        } else {
+            return BuiltinType.ANY;
+        }
+    }
+
+    @Override
+    protected boolean propagateNullAndMissing() {
+        return nullInNullOut;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
index 2626c6b..1d2cb21 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractIfMissingOrNullTypeComputer.java
@@ -33,12 +33,14 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 abstract class AbstractIfMissingOrNullTypeComputer implements IResultTypeComputer {
     @Override
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
         AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expression;
+        SourceLocation sourceLoc = fce.getSourceLocation();
         IAType outPrimeType = null;
         ATypeTag outQuantifier = null; // could be 'missing' or 'null'
 
@@ -66,22 +68,23 @@
                     } else {
                         IAType primeType = getOutputPrimeType(unionType);
                         ATypeTag quantifier = outQuantifier != null ? outQuantifier : getOutputQuantifier(unionType);
-                        return createOutputType(TypeResolverUtil.resolve(outPrimeType, primeType), quantifier);
+                        return createOutputType(TypeResolverUtil.resolve(outPrimeType, primeType), quantifier,
+                                sourceLoc);
                     }
                 }
             } else {
                 // ANY or no intersection
                 return outPrimeType == null ? argType
-                        : createOutputType(TypeResolverUtil.resolve(outPrimeType, argType), outQuantifier);
+                        : createOutputType(TypeResolverUtil.resolve(outPrimeType, argType), outQuantifier, sourceLoc);
             }
         }
 
         if (outPrimeType == null) {
             return BuiltinType.ANULL;
         }
-        IAType outType = createOutputType(outPrimeType, ATypeTag.NULL);
+        IAType outType = createOutputType(outPrimeType, ATypeTag.NULL, sourceLoc);
         if (outQuantifier == ATypeTag.MISSING) {
-            outType = createOutputType(outType, ATypeTag.MISSING);
+            outType = createOutputType(outType, ATypeTag.MISSING, sourceLoc);
         }
         return outType;
     }
@@ -96,7 +99,8 @@
         return type.getActualType();
     }
 
-    private IAType createOutputType(IAType primeType, ATypeTag quantifier) throws AlgebricksException {
+    private IAType createOutputType(IAType primeType, ATypeTag quantifier, SourceLocation sourceLoc)
+            throws CompilationException {
         if (quantifier == null || primeType.getTypeTag() == ATypeTag.ANY) {
             return primeType;
         }
@@ -106,7 +110,8 @@
             case NULL:
                 return AUnionType.createNullableType(primeType, null);
             default:
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, String.valueOf(quantifier));
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                        "Unexpected quantifier: " + quantifier);
         }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
index 27f364a..08d2ca5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractStringTypeComputer.java
@@ -24,14 +24,16 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 abstract public class AbstractStringTypeComputer extends AbstractResultTypeComputer {
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (actualTypeTag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayIfNullTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayIfNullTypeComputer.java
new file mode 100644
index 0000000..de32758
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayIfNullTypeComputer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class ArrayIfNullTypeComputer extends AbstractResultTypeComputer {
+    public static final ArrayIfNullTypeComputer INSTANCE = new ArrayIfNullTypeComputer();
+
+    private ArrayIfNullTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType type = strippedInputTypes[0];
+        if (type.getTypeTag().isListType()) {
+            return AUnionType.createNullableType(((AbstractCollectionType) type).getItemType());
+        }
+        return BuiltinType.ANY;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRangeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRangeTypeComputer.java
new file mode 100644
index 0000000..4ff4cea5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRangeTypeComputer.java
@@ -0,0 +1,65 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+public class ArrayRangeTypeComputer extends AbstractResultTypeComputer {
+
+    public static final ArrayRangeTypeComputer INSTANCE = new ArrayRangeTypeComputer();
+    public static final AOrderedListType LONG_LIST = new AOrderedListType(BuiltinType.AINT64, null);
+    public static final AOrderedListType DOUBLE_LIST = new AOrderedListType(BuiltinType.ADOUBLE, null);
+
+    private ArrayRangeTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        if (strippedInputTypes.length != 2 && strippedInputTypes.length != 3) {
+            String functionName = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier().getName();
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
+                    functionName);
+        }
+        IAType startNum = strippedInputTypes[0];
+        IAType endNum = strippedInputTypes[1];
+        IAType step = strippedInputTypes.length == 3 ? strippedInputTypes[2] : null;
+        if (ATypeHierarchy.canPromote(startNum.getTypeTag(), ATypeTag.BIGINT)
+                && ATypeHierarchy.canPromote(endNum.getTypeTag(), ATypeTag.BIGINT)
+                && (step == null || ATypeHierarchy.canPromote(step.getTypeTag(), ATypeTag.BIGINT))) {
+            return LONG_LIST;
+        } else if (ATypeHierarchy.canPromote(startNum.getTypeTag(), ATypeTag.DOUBLE)
+                && ATypeHierarchy.canPromote(endNum.getTypeTag(), ATypeTag.DOUBLE)
+                && (step == null || ATypeHierarchy.canPromote(step.getTypeTag(), ATypeTag.DOUBLE))) {
+            return DOUBLE_LIST;
+        } else {
+            return BuiltinType.ANY;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRepeatTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRepeatTypeComputer.java
new file mode 100644
index 0000000..28c61bb
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ArrayRepeatTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class ArrayRepeatTypeComputer extends AbstractResultTypeComputer {
+
+    public static final ArrayRepeatTypeComputer INSTANCE = new ArrayRepeatTypeComputer();
+
+    private ArrayRepeatTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return new AOrderedListType(strippedInputTypes[0], null);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
index 80ac8ef..c6c744c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/BooleanFunctionTypeComputer.java
@@ -41,8 +41,8 @@
         AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
         String funcName = functionCallExpression.getFunctionIdentifier().getName();
         // Boolean type computer doesn't follow the null/missing-in/out semantics.
-        return TypeComputeUtils.resolveResultType(expression, env, (index, type) -> checkArgType(funcName, index, type),
-                this::getResultType, false);
+        return TypeComputeUtils.resolveResultType(expression, env,
+                (index, type, sourceLoc) -> checkArgType(funcName, index, type, sourceLoc), this::getResultType, false);
     }
 
     @Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
index b328cf4..3500435 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
@@ -21,6 +21,8 @@
 
 import java.util.Iterator;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.exceptions.InvalidExpressionException;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
@@ -68,11 +70,18 @@
                 AUnionType unionType = (AUnionType) e2Type;
                 e2Type = AUnionType.createUnknownableType(unionType.getActualType());
             }
-            fieldTypes[i] = e2Type;
-            fieldNames[i] = ConstantExpressionUtil.getStringConstant(e1);
-            if (fieldNames[i] == null) {
-                throw new InvalidExpressionException(funcName, 2 * i, e1, LogicalExpressionTag.CONSTANT);
+            String fieldName = ConstantExpressionUtil.getStringConstant(e1);
+            if (fieldName == null) {
+                throw new InvalidExpressionException(f.getSourceLocation(), funcName, 2 * i, e1,
+                        LogicalExpressionTag.CONSTANT);
             }
+            for (int j = 0; j < i; j++) {
+                if (fieldName.equals(fieldNames[j])) {
+                    throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, f.getSourceLocation(), fieldName);
+                }
+            }
+            fieldTypes[i] = e2Type;
+            fieldNames[i] = fieldName;
             i++;
         }
         return new ARecordType(null, fieldNames, fieldTypes, false);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
index cc2f9b6..1db946d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/CollectionMemberResultType.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CollectionMemberResultType extends AbstractResultTypeComputer {
     public static final CollectionMemberResultType INSTANCE = new CollectionMemberResultType();
@@ -34,10 +35,11 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (type.getTypeTag() != ATypeTag.MULTISET && type.getTypeTag() != ATypeTag.ARRAY) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ConcatTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ConcatTypeComputer.java
new file mode 100644
index 0000000..db59877
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ConcatTypeComputer.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class ConcatTypeComputer extends AbstractResultTypeComputer {
+
+    public static final ConcatTypeComputer INSTANCE_STRING = new ConcatTypeComputer(BuiltinType.ASTRING);
+
+    public static final ConcatTypeComputer INSTANCE_BINARY = new ConcatTypeComputer(BuiltinType.ABINARY);
+
+    private final IAType resultType;
+
+    private ConcatTypeComputer(IAType resultType) {
+        this.resultType = resultType;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType argType = strippedInputTypes[0];
+        IAType outputType = resultType;
+        if (!argType.getTypeTag().isListType() || isUnknownable(((AbstractCollectionType) argType).getItemType())) {
+            outputType = AUnionType.createUnknownableType(outputType);
+        }
+        return outputType;
+    }
+
+    private boolean isUnknownable(IAType type) {
+        switch (type.getTypeTag()) {
+            case ANY:
+            case MISSING:
+            case NULL:
+                return true;
+            case UNION:
+                return ((AUnionType) type).isUnknownableType();
+            default:
+                return false;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java
new file mode 100644
index 0000000..105ea52
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class DoubleIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final DoubleIfTypeComputer INSTANCE = new DoubleIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.DOUBLE ? inputType : BuiltinType.ANY;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
index 73c019c..cf40cbf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessByIndexResultType extends AbstractResultTypeComputer {
 
@@ -37,13 +38,14 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1 && actualTypeTag != ATypeTag.INTEGER) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.INTEGER);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
index e9af1ab..91cd887 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessByNameResultType extends AbstractResultTypeComputer {
 
@@ -37,13 +38,14 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1 && actualTypeTag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
index 3eae173..c80e1ab 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessNestedResultType.java
@@ -26,7 +26,6 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.exceptions.TypeMismatchException;
-import org.apache.asterix.om.exceptions.UnsupportedItemTypeException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
@@ -38,6 +37,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FieldAccessNestedResultType extends AbstractResultTypeComputer {
     public static final FieldAccessNestedResultType INSTANCE = new FieldAccessNestedResultType();
@@ -46,29 +46,30 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0 && actualTypeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.OBJECT);
         }
         if (argIndex == 1) {
             switch (actualTypeTag) {
                 case STRING:
                     break;
                 case ARRAY:
-                    checkOrderedList(funcName, type);
+                    checkOrderedList(type, sourceLoc);
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.ARRAY);
+                    throw new TypeMismatchException(sourceLoc, actualTypeTag, ATypeTag.STRING, ATypeTag.ARRAY);
             }
         }
     }
 
-    private void checkOrderedList(String funcName, IAType type) throws AlgebricksException {
+    private void checkOrderedList(IAType type, SourceLocation sourceLoc) throws AlgebricksException {
         AOrderedListType listType = (AOrderedListType) type;
         ATypeTag itemTypeTag = listType.getItemType().getTypeTag();
         if (itemTypeTag != ATypeTag.STRING && itemTypeTag != ATypeTag.ANY) {
-            throw new UnsupportedItemTypeException(funcName, itemTypeTag);
+            throw new TypeMismatchException(sourceLoc, itemTypeTag, ATypeTag.STRING, ATypeTag.ANY);
         }
     }
 
@@ -97,5 +98,4 @@
         IAType fieldType = recType.getSubFieldType(fieldPath);
         return fieldType == null ? BuiltinType.ANY : fieldType;
     }
-
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
index 0748745..606a68e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FullTextContainsResultTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FullTextContainsResultTypeComputer extends AbstractResultTypeComputer {
 
@@ -34,24 +35,26 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         // Expression1 should be a string.
         if (argIndex == 0 && actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.ANY) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING);
         }
         // Expression2 should be a string, or an (un)ordered list.
         if (argIndex == 1 && (actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.MULTISET
                 && actualTypeTag != ATypeTag.ARRAY && actualTypeTag != ATypeTag.ANY)) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.MULTISET,
-                    ATypeTag.ARRAY);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                    ATypeTag.MULTISET, ATypeTag.ARRAY);
         }
         // Each option name should be a string if it is already processed by FullTextContainsParameterCheckRule.
         // Before, the third argument should be a record if exists.
         // The structure is: arg2 = optionName1, arg3 = optionValue1, arg4 = optionName1, arg5 = optionValue2, ...
         if (argIndex > 1 && argIndex % 2 == 0 && (actualTypeTag != ATypeTag.STRING && actualTypeTag != ATypeTag.OBJECT
                 && actualTypeTag != ATypeTag.ANY)) {
-            throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.OBJECT);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                    ATypeTag.OBJECT);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
index cc19ac4..51fd6fc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/InjectFailureTypeComputer.java
@@ -24,12 +24,13 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * The first argument of INJECT_FAILURE can be any data model instance and will be passed verbatim to the
  * caller. The second argument is a boolean that determines if the invocation throws an exception.
  *
- * Consequently {@link #checkArgType(String, int, IAType)} validates that the second argument is a
+ * Consequently {@link AbstractResultTypeComputer#checkArgType(String, int, IAType, SourceLocation)} validates that the second argument is a
  * boolean and {@link #getResultType(ILogicalExpression, IAType...)} returns the type of the first
  * argument.
  */
@@ -38,9 +39,10 @@
     public static final InjectFailureTypeComputer INSTANCE = new InjectFailureTypeComputer();
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (argIndex == 1 && type.getTypeTag() != ATypeTag.BOOLEAN) {
-            throw new TypeMismatchException(funcName, argIndex, type.getTypeTag(), ATypeTag.BOOLEAN);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, type.getTypeTag(), ATypeTag.BOOLEAN);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListConstructorTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListConstructorTypeComputer.java
index 2528697..4916fc1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListConstructorTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListConstructorTypeComputer.java
@@ -47,26 +47,21 @@
         if (reqType != null) {
             return reqType;
         }
-        return computeTypeFromItems(env, f);
+        final IAType currentType = computeContentType(env, f);
+        return getListType(currentType == null ? BuiltinType.ANY : currentType);
     }
 
-    private IAType computeTypeFromItems(IVariableTypeEnvironment env, AbstractFunctionCallExpression f)
+    private IAType computeContentType(IVariableTypeEnvironment env, AbstractFunctionCallExpression f)
             throws AlgebricksException {
         IAType currentType = null;
-        boolean any = false;
         for (int k = 0; k < f.getArguments().size(); k++) {
             IAType type = (IAType) env.getType(f.getArguments().get(k).getValue());
             if (type.getTypeTag() == ATypeTag.UNION || (currentType != null && !currentType.equals(type))) {
-                any = true;
-                break;
+                return null;
             }
             currentType = type;
         }
-        if (any || currentType == null) {
-            return getListType(BuiltinType.ANY);
-        } else {
-            return getListType(currentType);
-        }
+        return currentType;
     }
 
     protected abstract IAType getListType(IAType itemType);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
index 0526e7d..abdff7e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class MinMaxAggTypeComputer extends AbstractResultTypeComputer {
 
@@ -34,7 +35,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case DOUBLE:
@@ -52,7 +54,7 @@
             case ANY:
                 return;
             default:
-                throw new UnsupportedTypeException(funcName, tag);
+                throw new UnsupportedTypeException(sourceLoc, funcName, tag);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java
new file mode 100644
index 0000000..bb5203e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class MissingIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final MissingIfTypeComputer INSTANCE = new MissingIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createMissableType(inputType);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
index 575ec52..e896ae7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NonTaggedGetItemResultType extends AbstractResultTypeComputer {
 
@@ -37,15 +38,17 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag actualTypeTag = type.getTypeTag();
         if (argIndex == 0) {
             if (type.getTypeTag() != ATypeTag.MULTISET && type.getTypeTag() != ATypeTag.ARRAY) {
-                throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.STRING, ATypeTag.ARRAY);
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.STRING,
+                        ATypeTag.ARRAY);
             }
         } else {
             if (!ATypeHierarchy.isCompatible(type.getTypeTag(), ATypeTag.INTEGER)) {
-                throw new TypeMismatchException(funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, actualTypeTag, ATypeTag.INTEGER);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java
new file mode 100644
index 0000000..590d8f6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class NullIfTypeComputer extends AbstractResultTypeComputer {
+
+    public static final NullIfTypeComputer INSTANCE = new NullIfTypeComputer();
+
+    @Override
+    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createNullableType(inputType);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
index 034309b..47ad411 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAddSubMulDivTypeComputer.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.om.exceptions.IncompatibleTypeException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -28,16 +29,26 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 
 public class NumericAddSubMulDivTypeComputer extends AbstractResultTypeComputer {
+    /**
+     * For those functions that do not return NULL if both arguments are not NULL
+     */
+    public static final NumericAddSubMulDivTypeComputer INSTANCE = new NumericAddSubMulDivTypeComputer(false);
 
-    public static final NumericAddSubMulDivTypeComputer INSTANCE = new NumericAddSubMulDivTypeComputer();
+    /**
+     * For those functions that may return NULL even if both arguments are not NULL (e.g. division by zero)
+     */
+    public static final NumericAddSubMulDivTypeComputer INSTANCE_NULLABLE = new NumericAddSubMulDivTypeComputer(true);
 
-    private NumericAddSubMulDivTypeComputer() {
+    private final boolean nullable;
+
+    private NumericAddSubMulDivTypeComputer(boolean nullable) {
+        this.nullable = nullable;
     }
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expr;
-        String funcName = functionCallExpression.getFunctionIdentifier().getName();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        String funcName = funcExpr.getFunctionIdentifier().getName();
         IAType t1 = strippedInputTypes[0];
         IAType t2 = strippedInputTypes[1];
         ATypeTag tag1 = t1.getTypeTag();
@@ -59,7 +70,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case FLOAT:
@@ -78,7 +89,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case BIGINT:
@@ -99,7 +110,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case INTEGER:
@@ -122,7 +133,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case SMALLINT:
@@ -147,7 +158,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case TINYINT:
@@ -174,7 +185,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case ANY:
@@ -189,7 +200,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DATE:
@@ -206,7 +217,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case TIME:
@@ -223,7 +234,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DATETIME:
@@ -237,7 +248,7 @@
                         type = BuiltinType.ADATETIME;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DURATION:
@@ -255,7 +266,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case YEARMONTHDURATION:
@@ -276,7 +287,7 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             case DAYTIMEDURATION:
@@ -297,12 +308,17 @@
                         type = BuiltinType.ANY;
                         break;
                     default:
-                        throw new IncompatibleTypeException(funcName, tag1, tag2);
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
                 }
                 break;
             default:
-                throw new IncompatibleTypeException(funcName, tag1, tag2);
+                throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
         }
+
+        if (nullable && type.getTypeTag() != ATypeTag.ANY) {
+            type = AUnionType.createNullableType(type);
+        }
+
         return type;
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
index ec9c59f..32a6487 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericAggTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericAggTypeComputer extends AbstractResultTypeComputer {
     public static final NumericAggTypeComputer INSTANCE = new NumericAggTypeComputer();
@@ -34,7 +35,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case DOUBLE:
@@ -46,7 +48,7 @@
             case ANY:
                 break;
             default:
-                throw new UnsupportedTypeException(funcName, tag);
+                throw new UnsupportedTypeException(sourceLoc, funcName, tag);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
new file mode 100644
index 0000000..c94a22b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
@@ -0,0 +1,244 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.IncompatibleTypeException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+/**
+ * Returns double if both operands are integers
+ */
+public class NumericDivideTypeComputer extends AbstractResultTypeComputer {
+    public static final NumericDivideTypeComputer INSTANCE = new NumericDivideTypeComputer();
+
+    private NumericDivideTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        String funcName = funcExpr.getFunctionIdentifier().getName();
+        IAType t1 = strippedInputTypes[0];
+        IAType t2 = strippedInputTypes[1];
+        ATypeTag tag1 = t1.getTypeTag();
+        ATypeTag tag2 = t2.getTypeTag();
+
+        IAType type;
+        switch (tag1) {
+            case DOUBLE:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case FLOAT:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                        type = BuiltinType.AFLOAT;
+                        break;
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case BIGINT:
+            case INTEGER:
+            case SMALLINT:
+            case TINYINT:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case DOUBLE:
+                        type = BuiltinType.ADOUBLE;
+                        break;
+                    case FLOAT:
+                        type = BuiltinType.AFLOAT;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case ANY:
+                switch (tag2) {
+                    case TINYINT:
+                    case SMALLINT:
+                    case INTEGER:
+                    case BIGINT:
+                    case FLOAT:
+                    case ANY:
+                    case DOUBLE:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DATE:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case TIME:
+                switch (tag2) {
+                    case TIME:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DATETIME:
+                switch (tag2) {
+                    case DATETIME:
+                        type = BuiltinType.ADURATION;
+                        break;
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                    case DURATION:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case YEARMONTHDURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case YEARMONTHDURATION:
+                        type = BuiltinType.AYEARMONTHDURATION;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            case DAYTIMEDURATION:
+                switch (tag2) {
+                    case DATE:
+                        type = BuiltinType.ADATE;
+                        break;
+                    case TIME:
+                        type = BuiltinType.ATIME;
+                        break;
+                    case DATETIME:
+                        type = BuiltinType.ADATETIME;
+                        break;
+                    case DAYTIMEDURATION:
+                        type = BuiltinType.ADAYTIMEDURATION;
+                        break;
+                    case ANY:
+                        type = BuiltinType.ANY;
+                        break;
+                    default:
+                        throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+                }
+                break;
+            default:
+                throw new IncompatibleTypeException(funcExpr.getSourceLocation(), funcName, tag1, tag2);
+        }
+
+        if (type.getTypeTag() != ATypeTag.ANY) {
+            // returns NULL if division by 0
+            type = AUnionType.createNullableType(type);
+        }
+
+        return type;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
index 1e9678a..54e1bcd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDoubleOutputFunctionTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericDoubleOutputFunctionTypeComputer extends AbstractResultTypeComputer {
 
@@ -36,7 +37,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -47,7 +49,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
index 2081b9b..8d84ce0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericInt8OutputFunctionTypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericInt8OutputFunctionTypeComputer extends AbstractResultTypeComputer {
 
@@ -35,7 +36,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -46,7 +48,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
index dbec30e..b537ffc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericRound2TypeComputer.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericRound2TypeComputer extends AbstractResultTypeComputer {
 
@@ -39,7 +40,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (argIndex == 0) {
             switch (tag) {
@@ -51,8 +53,8 @@
                 case DOUBLE:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
             }
         }
         if (argIndex == 1) {
@@ -63,8 +65,8 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
index ea5c2bb..ef9791f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericUnaryFunctionTypeComputer.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class NumericUnaryFunctionTypeComputer extends AbstractResultTypeComputer {
     public static final NumericUnaryFunctionTypeComputer INSTANCE = new NumericUnaryFunctionTypeComputer();
@@ -37,7 +38,8 @@
     }
 
     @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    protected void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -49,7 +51,7 @@
             case ANY:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
index 1442dfe..6deb17c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
@@ -25,6 +25,8 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.types.ARecordType;
@@ -72,6 +74,9 @@
             IAType t2 = (IAType) env.getType(e2);
             String fieldName = ConstantExpressionUtil.getStringConstant(e1);
             if (fieldName != null && t2 != null && TypeHelper.isClosed(t2)) {
+                if (namesList.contains(fieldName)) {
+                    throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, f.getSourceLocation(), fieldName);
+                }
                 namesList.add(fieldName);
                 if (t2.getTypeTag() == ATypeTag.UNION) {
                     AUnionType unionType = (AUnionType) t2;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OrderedListOfAnyTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OrderedListOfAnyTypeComputer.java
index 19f0e70..8c23f51 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OrderedListOfAnyTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OrderedListOfAnyTypeComputer.java
@@ -20,20 +20,26 @@
 
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 
 public class OrderedListOfAnyTypeComputer extends AbstractResultTypeComputer {
 
-    public static final OrderedListOfAnyTypeComputer INSTANCE = new OrderedListOfAnyTypeComputer();
+    public static final OrderedListOfAnyTypeComputer INSTANCE = new OrderedListOfAnyTypeComputer(false);
 
-    private OrderedListOfAnyTypeComputer() {
+    public static final OrderedListOfAnyTypeComputer INSTANCE_NULLABLE = new OrderedListOfAnyTypeComputer(true);
+
+    private final IAType type;
+
+    private OrderedListOfAnyTypeComputer(boolean nullable) {
+        IAType t = AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+        type = nullable ? AUnionType.createNullableType(t) : t;
     }
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        return new AOrderedListType(BuiltinType.ANY, null);
+        return type;
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
index a1fb5b0..5c59af6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
@@ -63,7 +63,8 @@
         IAType type0 = (IAType) env.getType(funcExpr.getArguments().get(0).getValue());
         ARecordType inputRecordType = TypeComputeUtils.extractRecordType(type0);
         if (inputRecordType == null) {
-            throw new TypeMismatchException(funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(funcExpr.getSourceLocation(), funcName, 0, type0.getTypeTag(),
+                    ATypeTag.OBJECT);
         }
 
         ILogicalExpression arg1 = funcExpr.getArguments().get(1).getValue();
@@ -109,8 +110,8 @@
                         if (fn[j].equals(FIELD_NAME_NAME)) {
                             ILogicalExpression fieldNameExpr = recConsExpr.getArguments().get(j).getValue();
                             if (ConstantExpressionUtil.getStringConstant(fieldNameExpr) == null) {
-                                throw new InvalidExpressionException(funcName, 1, fieldNameExpr,
-                                        LogicalExpressionTag.CONSTANT);
+                                throw new InvalidExpressionException(funcExpr.getSourceLocation(), funcName, 1,
+                                        fieldNameExpr, LogicalExpressionTag.CONSTANT);
                             }
                             // Get the actual "field-name" string
                             fieldName = ConstantExpressionUtil.getStringArgument(recConsExpr, j + 1);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
index db4d1c4..98ec9e7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class RecordMergeTypeComputer implements IResultTypeComputer {
     public static final RecordMergeTypeComputer INSTANCE = new RecordMergeTypeComputer();
@@ -56,12 +57,12 @@
         boolean unknownable = TypeHelper.canBeUnknown(t0) || TypeHelper.canBeUnknown(t1);
         ARecordType recType0 = TypeComputeUtils.extractRecordType(t0);
         if (recType0 == null) {
-            throw new TypeMismatchException(funcName, 0, t0.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(f.getSourceLocation(), funcName, 0, t0.getTypeTag(), ATypeTag.OBJECT);
         }
 
         ARecordType recType1 = TypeComputeUtils.extractRecordType(t1);
         if (recType1 == null) {
-            throw new TypeMismatchException(funcName, 1, t1.getTypeTag(), ATypeTag.OBJECT);
+            throw new TypeMismatchException(f.getSourceLocation(), funcName, 1, t1.getTypeTag(), ATypeTag.OBJECT);
         }
 
         List<String> resultFieldNames = new ArrayList<>();
@@ -90,11 +91,13 @@
             if (pos >= 0) {
                 IAType resultFieldType = resultFieldTypes.get(pos);
                 if (resultFieldType.getTypeTag() != fieldTypes[i].getTypeTag()) {
-                    throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldNames[i]);
+                    throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, f.getSourceLocation(),
+                            fieldNames[i]);
                 }
                 // Assuming fieldTypes[i].getTypeTag() = resultFieldType.getTypeTag()
                 if (fieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
-                    resultFieldTypes.set(pos, mergedNestedType(fieldNames[i], fieldTypes[i], resultFieldType));
+                    resultFieldTypes.set(pos,
+                            mergedNestedType(fieldNames[i], fieldTypes[i], resultFieldType, f.getSourceLocation()));
                 }
             } else {
                 additionalFieldNames.add(fieldNames[i]);
@@ -116,9 +119,10 @@
         return resultType;
     }
 
-    private IAType mergedNestedType(String fieldName, IAType fieldType1, IAType fieldType0) throws AlgebricksException {
+    private IAType mergedNestedType(String fieldName, IAType fieldType1, IAType fieldType0, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (fieldType1.getTypeTag() != ATypeTag.OBJECT || fieldType0.getTypeTag() != ATypeTag.OBJECT) {
-            throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldName);
+            throw new CompilationException(ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, sourceLoc, fieldName);
         }
 
         ARecordType resultType = (ARecordType) fieldType0;
@@ -131,8 +135,8 @@
                 // If a sub-record do merge, else ignore and let the values decide what to do
                 if (fieldType1Copy.getFieldTypes()[i].getTypeTag() == ATypeTag.OBJECT) {
                     IAType[] oldTypes = resultType.getFieldTypes();
-                    oldTypes[pos] =
-                            mergedNestedType(fname, fieldType1Copy.getFieldTypes()[i], resultType.getFieldTypes()[pos]);
+                    oldTypes[pos] = mergedNestedType(fname, fieldType1Copy.getFieldTypes()[i],
+                            resultType.getFieldTypes()[pos], sourceLoc);
                     resultType = new ARecordType(resultType.getTypeName(), resultType.getFieldNames(), oldTypes,
                             resultType.isOpen());
                 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java
deleted file mode 100644
index ceba3fb..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPairsTypeComputer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- *  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.asterix.om.typecomputer.impl;
-
-import org.apache.asterix.om.exceptions.TypeMismatchException;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-
-public class RecordPairsTypeComputer extends AbstractResultTypeComputer {
-
-    public static final RecordPairsTypeComputer INSTANCE = new RecordPairsTypeComputer();
-
-    private RecordPairsTypeComputer() {
-    }
-
-    @Override
-    protected void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
-        ATypeTag typeTag = type.getTypeTag();
-        if (typeTag != ATypeTag.OBJECT) {
-            throw new TypeMismatchException(funcName, argIndex, typeTag, ATypeTag.OBJECT);
-        }
-    }
-
-    @Override
-    public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        return AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
-    }
-
-}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
index e0b10d1..4e9a563 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Cases to support:
@@ -65,10 +66,10 @@
     }
 
     private void getPathFromConstantExpression(String funcName, ILogicalExpression expression, Set<String> fieldNameSet,
-            List<List<String>> pathList) throws AlgebricksException {
+            List<List<String>> pathList, SourceLocation sourceLoc) throws AlgebricksException {
         ConstantExpression ce = (ConstantExpression) expression;
         if (!(ce.getValue() instanceof AsterixConstantValue)) {
-            throw new InvalidExpressionException(funcName, 1, ce, LogicalExpressionTag.CONSTANT);
+            throw new InvalidExpressionException(sourceLoc, funcName, 1, ce, LogicalExpressionTag.CONSTANT);
         }
         IAObject item = ((AsterixConstantValue) ce.getValue()).getObject();
         ATypeTag type = item.getType().getTypeTag();
@@ -89,11 +90,11 @@
                 pathList.add(path);
                 break;
             default:
-                throw new UnsupportedTypeException(funcName, type);
+                throw new UnsupportedTypeException(sourceLoc, funcName, type);
         }
     }
 
-    private List<String> getListFromExpression(String funcName, ILogicalExpression expression)
+    private List<String> getListFromExpression(String funcName, ILogicalExpression expression, SourceLocation sourceLoc)
             throws AlgebricksException {
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) expression;
         List<Mutable<ILogicalExpression>> args = funcExp.getArguments();
@@ -104,14 +105,14 @@
             // Input list has only one level of nesting (list of list or list of strings)
             ConstantExpression ce = (ConstantExpression) arg.getValue();
             if (!(ce.getValue() instanceof AsterixConstantValue)) {
-                throw new InvalidExpressionException(funcName, 1, ce, LogicalExpressionTag.CONSTANT);
+                throw new InvalidExpressionException(sourceLoc, funcName, 1, ce, LogicalExpressionTag.CONSTANT);
             }
             IAObject item = ((AsterixConstantValue) ce.getValue()).getObject();
             ATypeTag type = item.getType().getTypeTag();
             if (type == ATypeTag.STRING) {
                 list.add(((AString) item).getStringValue());
             } else {
-                throw new UnsupportedTypeException(funcName, type);
+                throw new UnsupportedTypeException(sourceLoc, funcName, type);
             }
         }
 
@@ -119,8 +120,8 @@
     }
 
     private void getPathFromFunctionExpression(String funcName, ILogicalExpression expression, Set<String> fieldNameSet,
-            List<List<String>> pathList) throws AlgebricksException {
-        List<String> path = getListFromExpression(funcName, expression);
+            List<List<String>> pathList, SourceLocation sourceLoc) throws AlgebricksException {
+        List<String> path = getListFromExpression(funcName, expression, sourceLoc);
         // Add the path head to remove set
         fieldNameSet.add(path.get(0));
         pathList.add(path);
@@ -136,14 +137,14 @@
             ILogicalExpression le = arg.getValue();
             switch (le.getExpressionTag()) {
                 case CONSTANT:
-                    getPathFromConstantExpression(funcName, le, fieldNameSet, pathList);
+                    getPathFromConstantExpression(funcName, le, fieldNameSet, pathList, funcExp.getSourceLocation());
                     break;
                 case FUNCTION_CALL:
-                    getPathFromFunctionExpression(funcName, le, fieldNameSet, pathList);
+                    getPathFromFunctionExpression(funcName, le, fieldNameSet, pathList, funcExp.getSourceLocation());
                     break;
                 default:
-                    throw new InvalidExpressionException(funcName, 1, le, LogicalExpressionTag.CONSTANT,
-                            LogicalExpressionTag.FUNCTION_CALL);
+                    throw new InvalidExpressionException(funcExp.getSourceLocation(), funcName, 1, le,
+                            LogicalExpressionTag.CONSTANT, LogicalExpressionTag.FUNCTION_CALL);
             }
         }
     }
@@ -159,7 +160,7 @@
         Set<String> fieldNameSet = new HashSet<>();
         Deque<String> fieldPathStack = new ArrayDeque<>();
 
-        ARecordType inputRecordType = getRecordTypeFromType(funcName, type0);
+        ARecordType inputRecordType = getRecordTypeFromType(funcName, type0, funcExpr.getSourceLocation());
         if (inputRecordType == null) {
             return BuiltinType.ANY;
         }
@@ -168,7 +169,8 @@
         IAType inputListType = (IAType) env.getType(arg1);
         AOrderedListType inputOrderedListType = TypeComputeUtils.extractOrderedListType(inputListType);
         if (inputOrderedListType == null) {
-            throw new TypeMismatchException(funcName, 1, inputListType.getTypeTag(), ATypeTag.ARRAY);
+            throw new TypeMismatchException(funcExpr.getSourceLocation(), funcName, 1, inputListType.getTypeTag(),
+                    ATypeTag.ARRAY);
         }
 
         ATypeTag tt = inputOrderedListType.getItemType().getTypeTag();
@@ -319,7 +321,8 @@
                 destFieldTypes.toArray(new IAType[n]), isOpen);
     }
 
-    private static ARecordType getRecordTypeFromType(String funcName, IAType type0) throws AlgebricksException {
+    private static ARecordType getRecordTypeFromType(String funcName, IAType type0, SourceLocation sourceLoc)
+            throws AlgebricksException {
         switch (type0.getTypeTag()) {
             case OBJECT:
                 return (ARecordType) type0;
@@ -334,7 +337,7 @@
                 }
                 // Falls through for other cases.
             default:
-                throw new TypeMismatchException(funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
+                throw new TypeMismatchException(sourceLoc, funcName, 0, type0.getTypeTag(), ATypeTag.OBJECT);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index fae3d90..07c79ad 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ScalarVersionOfAggregateResultType extends AbstractResultTypeComputer {
 
@@ -36,10 +37,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.ANY && tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
index b93f215..c05b085 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SleepTypeComputer.java
@@ -24,12 +24,14 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SleepTypeComputer extends AbstractResultTypeComputer {
     public static final SleepTypeComputer INSTANCE = new SleepTypeComputer();
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (argIndex == 1) {
             switch (type.getTypeTag()) {
                 case TINYINT:
@@ -38,7 +40,7 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, type.getTypeTag(), ATypeTag.TINYINT,
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, type.getTypeTag(), ATypeTag.TINYINT,
                             ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
index e29b6ec..b2c95c5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/StringIntToStringTypeComputer.java
@@ -18,48 +18,98 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
+import java.util.EnumSet;
+import java.util.Set;
+
 import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class StringIntToStringTypeComputer extends AbstractResultTypeComputer {
-    public static final StringIntToStringTypeComputer INSTANCE = new StringIntToStringTypeComputer(1);
+    public static final StringIntToStringTypeComputer INSTANCE = new StringIntToStringTypeComputer(0, 0, 1, 1, false);
 
-    public static final StringIntToStringTypeComputer INSTANCE_TRIPLE_STRING = new StringIntToStringTypeComputer(3);
+    public static final StringIntToStringTypeComputer INSTANCE_NULLABLE =
+            new StringIntToStringTypeComputer(0, 0, 1, 1, true);
 
-    private final int stringArgCount;
+    public static final StringIntToStringTypeComputer INSTANCE_TRIPLE_STRING =
+            new StringIntToStringTypeComputer(0, 2, 3, 3, false);
 
-    public StringIntToStringTypeComputer(int stringArgCount) {
-        this.stringArgCount = stringArgCount;
+    public static final StringIntToStringTypeComputer INSTANCE_STRING_REGEXP_REPLACE_WITH_FLAG =
+            new StringIntToStringTypeComputer(0, 3, 3, 3, false);
+
+    private final int stringArgIdxMin;
+
+    private final int stringArgIdxMax;
+
+    private final int intArgIdxMin;
+
+    private final int intArgIdxMax;
+
+    private final boolean nullable;
+
+    public StringIntToStringTypeComputer(int stringArgIdxMin, int stringArgIdxMax, int intArgIdxMin, int intArgIdxMax,
+            boolean nullable) {
+        this.stringArgIdxMin = stringArgIdxMin;
+        this.stringArgIdxMax = stringArgIdxMax;
+        this.intArgIdxMin = intArgIdxMin;
+        this.intArgIdxMax = intArgIdxMax;
+        this.nullable = nullable;
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
-        if (argIndex < stringArgCount) {
-            if (tag != ATypeTag.STRING) {
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+        boolean expectedStringType = false;
+        if (stringArgIdxMin <= argIndex && argIndex <= stringArgIdxMax) {
+            if (tag == ATypeTag.STRING) {
+                return;
             }
-        } else {
+            expectedStringType = true;
+        }
+
+        boolean expectedIntType = false;
+        if (intArgIdxMin <= argIndex && argIndex <= intArgIdxMax) {
             switch (tag) {
                 case TINYINT:
                 case SMALLINT:
                 case INTEGER:
                 case BIGINT:
-                    break;
-                default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    return;
             }
+            expectedIntType = true;
         }
+
+        throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag,
+                getExpectedTypes(expectedStringType, expectedIntType));
     }
 
     @Override
     public IAType getResultType(ILogicalExpression expr, IAType... types) throws AlgebricksException {
-        return BuiltinType.ASTRING;
+        IAType resultType = BuiltinType.ASTRING;
+        if (nullable) {
+            resultType = AUnionType.createNullableType(resultType);
+        }
+        return resultType;
+    }
+
+    private ATypeTag[] getExpectedTypes(boolean expectedStringType, boolean expectedIntType) {
+        Set<ATypeTag> expectedTypes = EnumSet.noneOf(ATypeTag.class);
+        if (expectedStringType) {
+            expectedTypes.add(ATypeTag.STRING);
+        }
+        if (expectedIntType) {
+            expectedTypes.add(ATypeTag.TINYINT);
+            expectedTypes.add(ATypeTag.SMALLINT);
+            expectedTypes.add(ATypeTag.INTEGER);
+            expectedTypes.add(ATypeTag.BIGINT);
+        }
+        return expectedTypes.toArray(new ATypeTag[0]);
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
index 0ab49d0..5655454 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubsetCollectionTypeComputer.java
@@ -55,7 +55,8 @@
             case UNION: {
                 AUnionType ut = (AUnionType) t;
                 if (!ut.isUnknownableType()) {
-                    throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                    throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag,
+                            ATypeTag.MULTISET, ATypeTag.ARRAY);
                 }
                 IAType t2 = ut.getActualType();
                 ATypeTag tag2 = t2.getTypeTag();
@@ -63,12 +64,14 @@
                     AbstractCollectionType act = (AbstractCollectionType) t2;
                     return act.getItemType();
                 }
-                throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag, ATypeTag.MULTISET,
+                        ATypeTag.ARRAY);
             }
             case ANY:
                 return BuiltinType.ANY;
             default:
-                throw new TypeMismatchException(funcName, 0, actualTypeTag, ATypeTag.MULTISET, ATypeTag.ARRAY);
+                throw new TypeMismatchException(fun.getSourceLocation(), funcName, 0, actualTypeTag, ATypeTag.MULTISET,
+                        ATypeTag.ARRAY);
         }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
index 74a9530..aa31942 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SubstringTypeComputer.java
@@ -21,19 +21,22 @@
 import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SubstringTypeComputer extends AbstractResultTypeComputer {
     public static final SubstringTypeComputer INSTANCE = new SubstringTypeComputer();
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (argIndex == 0 && tag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.STRING);
         }
         if (argIndex > 0 && argIndex <= 2) {
             switch (tag) {
@@ -43,14 +46,14 @@
                 case BIGINT:
                     break;
                 default:
-                    throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
-                            ATypeTag.INTEGER, ATypeTag.BIGINT);
+                    throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT,
+                            ATypeTag.SMALLINT, ATypeTag.INTEGER, ATypeTag.BIGINT);
             }
         }
     }
 
     @Override
     public IAType getResultType(ILogicalExpression expr, IAType... types) throws AlgebricksException {
-        return BuiltinType.ASTRING;
+        return AUnionType.createNullableType(BuiltinType.ASTRING);
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
index fa2bf5b..59bfddf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/SwitchCaseComputer.java
@@ -47,7 +47,8 @@
 
         int argNumber = fce.getArguments().size();
         if (argNumber < 3) {
-            throw new CompilationException(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, funcName, argNumber);
+            throw new CompilationException(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER, fce.getSourceLocation(),
+                    funcName, argNumber);
         }
         int argSize = fce.getArguments().size();
         List<IAType> types = new ArrayList<>();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
new file mode 100644
index 0000000..1a5861b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class TreatAsTypeComputer extends AbstractResultTypeComputer {
+    public static final TreatAsTypeComputer INSTANCE_INTEGER = new TreatAsTypeComputer(BuiltinType.AINT32);
+
+    private final IAType type;
+
+    private TreatAsTypeComputer(IAType type) {
+        this.type = type;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        if (ATypeHierarchy.isCompatible(inputType.getTypeTag(), type.getTypeTag())) {
+            return type;
+        } else {
+            throw new TypeMismatchException(expr.getSourceLocation(), inputType.getTypeTag(), type.getTypeTag());
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
index d6825e3..806235c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.om.typecomputer.impl;
 
+import java.util.List;
+
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -30,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeComputeUtils {
 
@@ -40,13 +43,13 @@
     private static final byte NULL = 16;
 
     @FunctionalInterface
-    public static interface ArgTypeChecker {
-        public void checkArgTypes(int argIndex, IAType argType) throws AlgebricksException;
+    public interface ArgTypeChecker {
+        void checkArgTypes(int argIndex, IAType argType, SourceLocation argSrcLoc) throws AlgebricksException;
     }
 
     @FunctionalInterface
-    public static interface ResultTypeGenerator {
-        public IAType getResultType(ILogicalExpression expr, IAType... knownInputTypes) throws AlgebricksException;
+    public interface ResultTypeGenerator {
+        IAType getResultType(ILogicalExpression expr, IAType... knownInputTypes) throws AlgebricksException;
     }
 
     private TypeComputeUtils() {
@@ -73,9 +76,10 @@
             throws AlgebricksException {
         AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
 
-        IAType[] inputTypes = new IAType[fce.getArguments().size()];
+        List<Mutable<ILogicalExpression>> arguments = fce.getArguments();
+        IAType[] inputTypes = new IAType[arguments.size()];
         int index = 0;
-        for (Mutable<ILogicalExpression> argRef : fce.getArguments()) {
+        for (Mutable<ILogicalExpression> argRef : arguments) {
             ILogicalExpression arg = argRef.getValue();
             inputTypes[index++] = (IAType) env.getType(arg);
         }
@@ -89,7 +93,7 @@
                     || argTypeTag == ATypeTag.MISSING) {
                 continue;
             }
-            checker.checkArgTypes(argIndex, knownInputTypes[argIndex]);
+            checker.checkArgTypes(argIndex, knownInputTypes[argIndex], fce.getSourceLocation());
         }
 
         // Computes the result type.
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
index d52ac88..911e4d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryBinaryInt64TypeComputer.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryBinaryInt64TypeComputer extends AbstractResultTypeComputer {
     public static final UnaryBinaryInt64TypeComputer INSTANCE = new UnaryBinaryInt64TypeComputer();
@@ -34,10 +35,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.BINARY) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.BINARY);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.BINARY);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
index f815f33..3066a22 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryMinusTypeComputer.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryMinusTypeComputer extends AbstractResultTypeComputer {
 
@@ -33,7 +34,8 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         switch (tag) {
             case TINYINT:
@@ -44,7 +46,7 @@
             case DOUBLE:
                 break;
             default:
-                throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
+                throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.TINYINT, ATypeTag.SMALLINT,
                         ATypeTag.INTEGER, ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE);
         }
     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
index e744ef1..5535890 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnaryStringInt64TypeComputer.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnaryStringInt64TypeComputer extends AbstractResultTypeComputer {
 
@@ -34,10 +35,11 @@
     }
 
     @Override
-    public void checkArgType(String funcName, int argIndex, IAType type) throws AlgebricksException {
+    public void checkArgType(String funcName, int argIndex, IAType type, SourceLocation sourceLoc)
+            throws AlgebricksException {
         ATypeTag tag = type.getTypeTag();
         if (tag != ATypeTag.STRING) {
-            throw new TypeMismatchException(funcName, argIndex, tag, ATypeTag.STRING);
+            throw new TypeMismatchException(sourceLoc, funcName, argIndex, tag, ATypeTag.STRING);
         }
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index baaed59..7d5e4c0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -28,7 +28,10 @@
 
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -134,25 +137,21 @@
 
     @Override
     public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(typeName + ": ");
-        if (isOpen) {
-            sb.append("open ");
-        } else {
-            sb.append("closed ");
+        return append(new StringBuilder()).toString();
+    }
+
+    private StringBuilder append(StringBuilder sb) {
+        if (typeName != null) {
+            sb.append(typeName).append(": ");
         }
-        sb.append("{\n");
+        sb.append(isOpen ? "open" : "closed");
+        sb.append(" {\n");
         int n = fieldNames.length;
         for (int i = 0; i < n; i++) {
-            sb.append("  " + fieldNames[i] + ": " + fieldTypes[i].toString());
-            if (i < (n - 1)) {
-                sb.append(",\n");
-            } else {
-                sb.append("\n");
-            }
+            sb.append("  ").append(fieldNames[i]).append(": ").append(fieldTypes[i]);
+            sb.append(i < (n - 1) ? ",\n" : "\n");
         }
-        sb.append("}\n");
-        return sb.toString();
+        return sb.append("}\n");
     }
 
     @Override
@@ -226,6 +225,39 @@
     }
 
     /**
+     *
+     * @param subFieldName
+     *            The full pathname of the field
+     * @return The nullability of the field
+     * @throws AlgebricksException
+     */
+    public boolean isSubFieldNullable(List<String> subFieldName) throws AlgebricksException {
+        IAType subRecordType = getFieldType(subFieldName.get(0));
+        for (int i = 1; i < subFieldName.size(); i++) {
+            if (subRecordType == null) {
+                // open field is nullable
+                return true;
+            }
+            if (subRecordType.getTypeTag().equals(ATypeTag.UNION)) {
+                if (NonTaggedFormatUtil.isOptional(subRecordType)) {
+                    return true;
+                }
+                subRecordType = ((AUnionType) subRecordType).getActualType();
+                if (subRecordType.getTypeTag() != ATypeTag.OBJECT) {
+                    throw new AsterixException(
+                            "Field accessor is not defined for values of type " + subRecordType.getTypeTag());
+                }
+            }
+            if (!(subRecordType instanceof ARecordType)) {
+                throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+                        "Illegal field type " + subRecordType.getTypeTag() + " when checking field nullability");
+            }
+            subRecordType = ((ARecordType) subRecordType).getFieldType(subFieldName.get(i));
+        }
+        return subRecordType == null || NonTaggedFormatUtil.isOptional(subRecordType);
+    }
+
+    /**
      * Returns the field type of the field name if it exists, otherwise null.
      *
      * @param fieldName
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
index 825ed70..f2b004f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
@@ -66,7 +66,8 @@
     DAYTIMEDURATION(37),
     UUID(38),
     SHORTWITHOUTTYPEINFO(40),
-    NULL(41);
+    NULL(41),
+    GEOMETRY(42);
 
     /*
      * Serialized Tags begin
@@ -100,6 +101,7 @@
     public static final byte SERIALIZED_FLOAT_TYPE_TAG = FLOAT.serialize();
     public static final byte SERIALIZED_BINARY_TYPE_TAG = BINARY.serialize();
     public static final byte SERIALIZED_UUID_TYPE_TAG = UUID.serialize();
+    public static final byte SERIALIZED_GEOMETRY_TYPE_TAG = GEOMETRY.serialize();
 
     /*
      * Serialized Tags end
@@ -133,6 +135,10 @@
         return this == ATypeTag.OBJECT || this == ATypeTag.ARRAY || this == ATypeTag.MULTISET || this == ATypeTag.UNION;
     }
 
+    public final boolean isListType() {
+        return this == ATypeTag.ARRAY || this == ATypeTag.MULTISET;
+    }
+
     @Override
     public String toString() {
         return this.name().toLowerCase();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index 3037e7c..a36e0e4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -682,6 +682,33 @@
         }
     };
 
+    public final static BuiltinType AGEOMETRY = new LowerCaseConstructorType() {
+
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public String getDisplayName() {
+            return "AGEOMETRY";
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.GEOMETRY;
+        }
+
+        @Override
+        public String getTypeName() {
+            return "geometry";
+        }
+
+        @Override
+        public ObjectNode toJSON() {
+            ObjectNode type = new ObjectMapper().createObjectNode();
+            type.put("type", "AGEOMETRY");
+            return type;
+        }
+    };
+
     public final static BuiltinType ACIRCLE = new LowerCaseConstructorType() {
 
         private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
index 05f265c..f2d1f1a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/EnumDeserializer.java
@@ -18,24 +18,25 @@
  */
 package org.apache.asterix.om.types;
 
-import java.util.HashMap;
-import java.util.Map;
+import java.lang.reflect.Array;
 
 public class EnumDeserializer<E extends Enum<E> & IEnumSerializer> {
 
     public static final EnumDeserializer<ATypeTag> ATYPETAGDESERIALIZER =
             new EnumDeserializer<ATypeTag>(ATypeTag.class);
 
-    private Map<Byte, E> enumvalMap = new HashMap<Byte, E>();
+    private final E[] enumvalMap;
 
+    @SuppressWarnings("unchecked")
     private EnumDeserializer(Class<E> enumClass) {
+        enumvalMap = (E[]) Array.newInstance(enumClass, Byte.MAX_VALUE);
         for (E constant : enumClass.getEnumConstants()) {
-            enumvalMap.put(constant.serialize(), constant);
+            enumvalMap[constant.serialize()] = constant;
         }
     }
 
     public E deserialize(byte value) {
-        return enumvalMap.get(value);
+        return enumvalMap[value];
     }
 
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
index cfc1b55..254dbee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
@@ -88,6 +88,8 @@
                 return AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE;
             case ARRAY:
                 return AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+            case GEOMETRY:
+                return BuiltinType.AGEOMETRY;
             default:
                 // TODO(tillw) should be an internal error
                 throw new HyracksDataException("Typetag " + typeTag + " is not a built-in type");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
index 67514a3..f182370 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/ATypeHierarchy.java
@@ -315,7 +315,7 @@
             case DOUBLE:
                 return DoubleToInt32TypeConvertComputer.getInstance(strictDemote).convertType(bytes, offset);
             default:
-                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH, name, argIndex,
+                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH_FUNCTION, name, argIndex,
                         Arrays.toString(new Object[] { ATypeTag.TINYINT, ATypeTag.SMALLINT, ATypeTag.INTEGER,
                                 ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE }),
                         sourceTypeTag);
@@ -354,7 +354,7 @@
             case DOUBLE:
                 return DoubleToInt64TypeConvertComputer.getInstance(strictDemote).convertType(bytes, offset);
             default:
-                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH, name, argIndex,
+                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH_FUNCTION, name, argIndex,
                         Arrays.toString(new Object[] { ATypeTag.TINYINT, ATypeTag.SMALLINT, ATypeTag.INTEGER,
                                 ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE }),
                         sourceTypeTag);
@@ -385,7 +385,7 @@
             case BIGINT:
                 return IntegerToDoubleTypeConvertComputer.getInstance().convertType(bytes, offset, sourceTypeTag);
             default:
-                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH, name, argIndex,
+                throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH_FUNCTION, name, argIndex,
                         Arrays.toString(new ATypeTag[] { ATypeTag.TINYINT, ATypeTag.SMALLINT, ATypeTag.INTEGER,
                                 ATypeTag.BIGINT, ATypeTag.FLOAT, ATypeTag.DOUBLE }),
                         sourceTypeTag);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
index 55108a1..be6cf62 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
@@ -99,8 +101,7 @@
                     int j = getFieldIndex(baaos.getByteArray(), serializedFieldNameOffsets[i],
                             UTF8StringUtil.getStringLength(baaos.getByteArray(), serializedFieldNameOffsets[i]));
                     if (j != i) {
-                        throw new IllegalStateException("Closed fields " + j + " and " + i
-                                + " have the same field name \"" + fieldNames[i] + "\"");
+                        throw new RuntimeDataException(ErrorCode.DUPLICATE_FIELD_NAME, fieldNames[i]);
                     }
                 }
             } catch (IOException e) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/AdmNodeUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/AdmNodeUtils.java
new file mode 100644
index 0000000..6f4ee17
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/AdmNodeUtils.java
@@ -0,0 +1,163 @@
+/*
+ * 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.asterix.om.utils;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmBooleanNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmNullNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.pointables.nonvisitor.AListPointable;
+import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.primitive.BooleanPointable;
+import org.apache.hyracks.data.std.primitive.DoublePointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class AdmNodeUtils {
+
+    private AdmNodeUtils() {
+    }
+
+    public static Map<String, IAdmNode> getOpenFields(ARecordPointable recPointable, ARecordType recordType)
+            throws IOException {
+        int openFieldCount = recPointable.getOpenFieldCount(recordType);
+        Map<String, IAdmNode> map = (openFieldCount == 0) ? Collections.emptyMap() : new HashMap<>();
+        for (int i = 0; i < openFieldCount; i++) {
+            map.put(recPointable.getOpenFieldName(recordType, i), getOpenField(recPointable, recordType, i));
+        }
+        return map;
+    }
+
+    private static IAdmNode getOpenField(ARecordPointable recPointable, ARecordType type, int i) throws IOException {
+        byte tagByte = recPointable.getOpenFieldTag(type, i);
+        ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[tagByte];
+        switch (tag) {
+            case ARRAY:
+                return getOpenFieldAsArray(recPointable, type, i);
+            case BIGINT:
+                return new AdmBigIntNode(LongPointable.getLong(recPointable.getByteArray(),
+                        recPointable.getOpenFieldValueOffset(type, i) + 1));
+            case BOOLEAN:
+                return AdmBooleanNode.get(BooleanPointable.getBoolean(recPointable.getByteArray(),
+                        recPointable.getOpenFieldValueOffset(type, i) + 1));
+            case DOUBLE:
+                return new AdmDoubleNode(DoublePointable.getDouble(recPointable.getByteArray(),
+                        recPointable.getOpenFieldValueOffset(type, i) + 1));
+            case NULL:
+                return AdmNullNode.INSTANCE;
+            case OBJECT:
+                return getOpenFieldAsObject(recPointable, type, i);
+            case STRING:
+                UTF8StringPointable str = UTF8StringPointable.FACTORY.createPointable();
+                str.set(recPointable.getByteArray(), recPointable.getOpenFieldValueOffset(type, i) + 1,
+                        recPointable.getOpenFieldValueSize(type, i) - 1);
+                return new AdmStringNode(str.toString());
+            default:
+                throw new UnsupportedOperationException("Unsupported item type: " + tag);
+        }
+    }
+
+    private static AdmObjectNode getOpenFieldAsObject(ARecordPointable recPointable, ARecordType type, int i)
+            throws IOException {
+        ARecordPointable pointable = ARecordPointable.FACTORY.createPointable();
+        int offset = recPointable.getOpenFieldValueOffset(type, i);
+        int len = recPointable.getOpenFieldValueSize(type, i);
+        pointable.set(recPointable.getByteArray(), offset, len);
+        return new AdmObjectNode(getOpenFields(pointable, RecordUtil.FULLY_OPEN_RECORD_TYPE));
+    }
+
+    private static AdmArrayNode getOpenFieldAsArray(ARecordPointable recPointable, ARecordType type, int i)
+            throws IOException {
+        AListPointable pointable = AListPointable.FACTORY.createPointable();
+        int offset = recPointable.getOpenFieldValueOffset(type, i);
+        int len = recPointable.getOpenFieldValueSize(type, i);
+        pointable.set(recPointable.getByteArray(), offset, len);
+        return getAsAdmNode(pointable);
+    }
+
+    public static AdmArrayNode getAsAdmNode(AListPointable listPointable) throws IOException {
+        int count = listPointable.getItemCount();
+        AdmArrayNode node = new AdmArrayNode(count);
+        for (int i = 0; i < count; i++) {
+            byte tagByte = listPointable.getItemTag(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i);
+            ATypeTag itemTag = ATypeTag.VALUE_TYPE_MAPPING[tagByte];
+            switch (itemTag) {
+                case ARRAY:
+                    node.add(getOpenFieldAsArray(listPointable, i));
+                    break;
+                case BIGINT:
+                    node.add(new AdmBigIntNode(LongPointable.getLong(listPointable.getByteArray(),
+                            listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i) + 1)));
+                    break;
+                case BOOLEAN:
+                    node.add(AdmBooleanNode.get(BooleanPointable.getBoolean(listPointable.getByteArray(),
+                            listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i) + 1)));
+                    break;
+                case DOUBLE:
+                    node.add(new AdmDoubleNode(DoublePointable.getDouble(listPointable.getByteArray(),
+                            listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i) + 1)));
+                    break;
+                case NULL:
+                    node.add(AdmNullNode.INSTANCE);
+                    break;
+                case OBJECT:
+                    node.add(getOpenFieldAsObject(listPointable, i));
+                    break;
+                case STRING:
+                    UTF8StringPointable str = UTF8StringPointable.FACTORY.createPointable();
+                    str.set(listPointable.getByteArray(),
+                            listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i) + 1,
+                            listPointable.getItemSize(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i));
+                    node.add(new AdmStringNode(str.toString()));
+                    break;
+                default:
+                    throw new UnsupportedOperationException("Unsupported item type: " + itemTag);
+            }
+        }
+        return node;
+    }
+
+    private static IAdmNode getOpenFieldAsObject(AListPointable listPointable, int i) throws IOException {
+        ARecordPointable pointable = ARecordPointable.FACTORY.createPointable();
+        int offset = listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i);
+        int len = listPointable.getItemSize(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i);
+        pointable.set(listPointable.getByteArray(), offset, len);
+        return new AdmObjectNode(AdmNodeUtils.getOpenFields(pointable, RecordUtil.FULLY_OPEN_RECORD_TYPE));
+    }
+
+    private static AdmArrayNode getOpenFieldAsArray(AListPointable listPointable, int i) throws IOException {
+        AListPointable pointable = AListPointable.FACTORY.createPointable();
+        int offset = listPointable.getItemOffset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i);
+        int len = listPointable.getItemSize(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE, i);
+        pointable.set(listPointable.getByteArray(), offset, len);
+        return getAsAdmNode(pointable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
index 069e47b..5b5249d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
@@ -24,8 +24,6 @@
 import java.util.List;
 
 import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 
 import com.google.common.collect.Lists;
 import org.apache.asterix.om.types.AOrderedListType;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
index 8558538..7da0263 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
@@ -68,6 +69,7 @@
             case ARRAY:
             case MULTISET:
             case POLYGON:
+            case GEOMETRY:
             case ANY:
                 return false;
             default:
@@ -193,6 +195,14 @@
                 } else {
                     return AUnorderedListSerializerDeserializer.getUnorderedListLength(serNonTaggedAObject, offset) - 1;
                 }
+            case GEOMETRY:
+                // Since Geometry is variable size, we store its size at the first 32 bits for efficiency
+                // @see: STGeomFromTextDescriptor#createEvaluatorFactory, AGeometrySerializerDeserializer#serialize
+                if (tagged) {
+                    return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset + 1) + 4;
+                } else {
+                    return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset) + 4;
+                }
             default:
                 throw new NotImplementedException(
                         "No getLength implemented for a value of this type " + typeTag + " .");
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/ExceptionTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/ExceptionTest.java
index fc4646f..29cb9a7 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/ExceptionTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/ExceptionTest.java
@@ -22,6 +22,7 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.List;
@@ -105,8 +106,14 @@
                 when(mockExpr.getOpaqueParameters()).thenReturn(opaqueParameters);
 
                 // Invokes a type computer.
-                IResultTypeComputer instance = (IResultTypeComputer) c.getField("INSTANCE").get(null);
-                instance.computeType(mockExpr, mockTypeEnv, mockMetadataProvider);
+                IResultTypeComputer instance;
+                Field[] fields = c.getFields();
+                for (Field field : fields) {
+                    if (field.getName().startsWith("INSTANCE")) {
+                        instance = (IResultTypeComputer) field.get(null);
+                        instance.computeType(mockExpr, mockTypeEnv, mockMetadataProvider);
+                    }
+                }
             } catch (AlgebricksException ae) {
                 String msg = ae.getMessage();
                 if (msg.startsWith("ASX")) {
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
index 1ff62b9..9f6a8eb 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
@@ -22,8 +22,10 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -50,6 +52,37 @@
 
     @Test
     public void test() throws Exception {
+        // Mocks the type environment.
+        IVariableTypeEnvironment mockTypeEnv = mock(IVariableTypeEnvironment.class);
+
+        // Mocks the metadata provider.
+        IMetadataProvider<?, ?> mockMetadataProvider = mock(IMetadataProvider.class);
+
+        // Mocks function expression.
+        AbstractFunctionCallExpression mockExpr = mock(AbstractFunctionCallExpression.class);
+        FunctionIdentifier fid = mock(FunctionIdentifier.class);
+        when(mockExpr.getFunctionIdentifier()).thenReturn(fid);
+        when(fid.getName()).thenReturn("testFunction");
+
+        // A function at most has six argument.
+        List<Mutable<ILogicalExpression>> sixArgs = createArgs(6, mockTypeEnv);
+
+        // Sets up arguments for the mocked expression.
+        when(mockExpr.getArguments()).thenReturn(sixArgs);
+
+        // Sets up required/actual types of the mocked expression.
+        Object[] opaqueParameters = new Object[2];
+        opaqueParameters[0] = BuiltinType.ANY;
+        opaqueParameters[1] = BuiltinType.ANY;
+        when(mockExpr.getOpaqueParameters()).thenReturn(opaqueParameters);
+
+        // functions that check the number of args inside the type computer
+        List<Mutable<ILogicalExpression>> replaceArgs = createArgs(4, mockTypeEnv);
+        List<Mutable<ILogicalExpression>> rangeArgs = createArgs(3, mockTypeEnv);
+        HashMap<String, List<Mutable<ILogicalExpression>>> map = new HashMap<>();
+        map.put("INSTANCE_REPLACE", replaceArgs);
+        map.put("ArrayRangeTypeComputer", rangeArgs);
+
         // Several exceptional type computers.
         Set<String> exceptionalTypeComputers = new HashSet<>();
         exceptionalTypeComputers.add("InjectFailureTypeComputer");
@@ -72,50 +105,58 @@
                 continue;
             }
             System.out.println("Test type computer: " + c.getName());
-            Assert.assertTrue(testTypeComputer(c));
+            Assert.assertTrue(testTypeComputer(c, mockTypeEnv, mockMetadataProvider, mockExpr, map, sixArgs));
         }
     }
 
-    private boolean testTypeComputer(Class<? extends IResultTypeComputer> c) throws Exception {
-        // Mocks the type environment.
-        IVariableTypeEnvironment mockTypeEnv = mock(IVariableTypeEnvironment.class);
-        // Mocks the metadata provider.
-        IMetadataProvider<?, ?> mockMetadataProvider = mock(IMetadataProvider.class);
+    private boolean testTypeComputer(Class<? extends IResultTypeComputer> c, IVariableTypeEnvironment mockTypeEnv,
+            IMetadataProvider<?, ?> mockMetadataProvider, AbstractFunctionCallExpression mockExpr,
+            HashMap<String, List<Mutable<ILogicalExpression>>> map, List<Mutable<ILogicalExpression>> sixArgs)
+            throws Exception {
+        // Tests the return type. It should be either ANY or NULLABLE/MISSABLE.
+        IResultTypeComputer instance;
+        IAType resultType;
+        Field[] fields = c.getFields();
+        List<Mutable<ILogicalExpression>> args;
+        for (Field field : fields) {
+            if (field.getName().startsWith("INSTANCE")) {
+                System.out.println("Test type computer INSTANCE: " + field.getName());
+                args = getArgs(field.getName(), c, map);
+                if (args != null) {
+                    when(mockExpr.getArguments()).thenReturn(args);
+                } else {
+                    when(mockExpr.getArguments()).thenReturn(sixArgs);
+                }
+                instance = (IResultTypeComputer) field.get(null);
+                resultType = instance.computeType(mockExpr, mockTypeEnv, mockMetadataProvider);
+                ATypeTag typeTag = resultType.getTypeTag();
+                if (typeTag != ATypeTag.ANY && !(typeTag == ATypeTag.UNION && ((AUnionType) resultType).isNullableType()
+                        && ((AUnionType) resultType).isMissableType())) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
 
-        // Mocks function expression.
-        AbstractFunctionCallExpression mockExpr = mock(AbstractFunctionCallExpression.class);
-        FunctionIdentifier fid = mock(FunctionIdentifier.class);
-        when(mockExpr.getFunctionIdentifier()).thenReturn(fid);
-        when(fid.getName()).thenReturn("testFunction");
-
-        // A function at most has six argument.
+    private List<Mutable<ILogicalExpression>> createArgs(int numArgs, IVariableTypeEnvironment mockTypeEnv)
+            throws Exception {
         List<Mutable<ILogicalExpression>> argRefs = new ArrayList<>();
-        for (int argIndex = 0; argIndex < 6; ++argIndex) {
+        for (int argIndex = 0; argIndex < numArgs; ++argIndex) {
             ILogicalExpression mockArg = mock(ILogicalExpression.class);
             argRefs.add(new MutableObject<>(mockArg));
             when(mockTypeEnv.getType(mockArg)).thenReturn(BuiltinType.ANY);
         }
 
-        // Sets up arguments for the mocked expression.
-        when(mockExpr.getArguments()).thenReturn(argRefs);
+        return argRefs;
+    }
 
-        // Sets up required/actual types of the mocked expression.
-        Object[] opaqueParameters = new Object[2];
-        opaqueParameters[0] = BuiltinType.ANY;
-        opaqueParameters[1] = BuiltinType.ANY;
-        when(mockExpr.getOpaqueParameters()).thenReturn(opaqueParameters);
-
-        // Tests the return type. It should be either ANY or NULLABLE/MISSABLE.
-        IResultTypeComputer instance = (IResultTypeComputer) c.getField("INSTANCE").get(null);
-        IAType resultType = instance.computeType(mockExpr, mockTypeEnv, mockMetadataProvider);
-        ATypeTag typeTag = resultType.getTypeTag();
-        if (typeTag == ATypeTag.ANY) {
-            return true;
+    private List<Mutable<ILogicalExpression>> getArgs(String instanceName, Class<? extends IResultTypeComputer> c,
+            HashMap<String, List<Mutable<ILogicalExpression>>> map) {
+        if (instanceName.equals("INSTANCE")) {
+            return map.get(c.getSimpleName());
+        } else {
+            return map.get(instanceName);
         }
-        if (typeTag == ATypeTag.UNION) {
-            AUnionType unionType = (AUnionType) resultType;
-            return unionType.isMissableType() && unionType.isNullableType();
-        }
-        return false;
     }
 }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
index 8840c3f..c8abe8f 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
@@ -21,8 +21,6 @@
 import java.nio.ByteBuffer;
 import java.nio.channels.SocketChannel;
 
-import org.apache.asterix.common.transactions.LogRecord;
-
 public interface IReplicationWorker extends Runnable {
 
     /**
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
index 5c324b1..8847e7e 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
@@ -169,7 +169,7 @@
     private void sendGoodBye() {
         try {
             ReplicationProtocol.sendGoodbye(sc);
-        } catch (IOException e) {
+        } catch (Exception e) {
             LOGGER.warn("Failed to send good bye to {}", this, e);
         }
     }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
index a94f073..2ed2ac9 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -57,7 +57,7 @@
         }
     }
 
-    public synchronized void isFull(boolean full) {
+    public synchronized void setFull(boolean full) {
         this.full.set(full);
         this.notify();
     }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
index 7a9d75f..6c8e372 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -167,7 +167,7 @@
 
     private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
         if (!currentTxnLogBuffer.hasSpace(logRecord)) {
-            currentTxnLogBuffer.isFull(true);
+            currentTxnLogBuffer.setFull(true);
             if (logRecord.getLogSize() > logPageSize) {
                 getAndInitNewLargePage(logRecord.getLogSize());
             } else {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index bc93294..1f6efa8 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -114,7 +114,7 @@
                     requestType = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
                 }
             } catch (Exception e) {
-                LOGGER.warn("Unexpectedly error during replication.", e);
+                LOGGER.warn("Unexpected error during replication.", e);
             } finally {
                 if (socketChannel.isOpen()) {
                     try {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 84922cd..e778cce 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -22,16 +22,21 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.file.Path;
 import java.util.Collection;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.storage.DatasetResourceReference;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.common.LocalResource;
 
 /**
@@ -40,9 +45,11 @@
 public class CheckpointPartitionIndexesTask implements IReplicaTask {
 
     private final int partition;
+    private final long maxComponentId;
 
-    public CheckpointPartitionIndexesTask(int partition) {
+    public CheckpointPartitionIndexesTask(int partition, long maxComponentId) {
         this.partition = partition;
+        this.maxComponentId = maxComponentId;
     }
 
     @Override
@@ -51,13 +58,26 @@
                 appCtx.getIndexCheckpointManagerProvider();
         PersistentLocalResourceRepository resRepo =
                 (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        final IIOManager ioManager = appCtx.getIoManager();
         final Collection<LocalResource> partitionResources = resRepo.getPartitionResources(partition).values();
         final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
         for (LocalResource ls : partitionResources) {
-            final IIndexCheckpointManager indexCheckpointManager =
-                    indexCheckpointManagerProvider.get(DatasetResourceReference.of(ls));
+            DatasetResourceReference ref = DatasetResourceReference.of(ls);
+            final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(ref);
             indexCheckpointManager.delete();
-            indexCheckpointManager.init(currentLSN);
+            // Get most recent sequence of existing files to avoid deletion
+            Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
+            String[] files = indexPath.toFile().list(AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER);
+            if (files == null) {
+                throw HyracksDataException
+                        .create(new IOException(indexPath + " is not a directory or an IO Error occurred"));
+            }
+            long maxComponentSequence = Long.MIN_VALUE;
+            for (String file : files) {
+                maxComponentSequence =
+                        Math.max(maxComponentSequence, IndexComponentFileReference.of(file).getSequenceEnd());
+            }
+            indexCheckpointManager.init(maxComponentSequence, currentLSN, maxComponentId);
         }
         ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
     }
@@ -72,6 +92,7 @@
         try {
             DataOutputStream dos = new DataOutputStream(out);
             dos.writeInt(partition);
+            dos.writeLong(maxComponentId);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
@@ -80,7 +101,8 @@
     public static CheckpointPartitionIndexesTask create(DataInput input) throws HyracksDataException {
         try {
             int partition = input.readInt();
-            return new CheckpointPartitionIndexesTask(partition);
+            long maxComponentId = input.readLong();
+            return new CheckpointPartitionIndexesTask(partition, maxComponentId);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
index d5dc51d..55dd5d4 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
@@ -63,8 +63,8 @@
         final IIOManager ioManager = appCtx.getIoManager();
         final FileReference localPath = ioManager.resolve(componentFile);
         final Path resourceDir = Files.createDirectories(localPath.getFile().getParentFile().toPath());
-        final String componentId = PersistentLocalResourceRepository.getComponentId(componentFile);
-        return Paths.get(resourceDir.toString(), StorageConstants.COMPONENT_MASK_FILE_PREFIX + componentId);
+        final String componentSequence = PersistentLocalResourceRepository.getComponentSequence(componentFile);
+        return Paths.get(resourceDir.toString(), StorageConstants.COMPONENT_MASK_FILE_PREFIX + componentSequence);
     }
 
     @Override
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
index 57474ef..b360a09 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -34,8 +34,10 @@
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.replication.api.IReplicaTask;
 import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 
 /**
  * A task to mark a replicated LSM component as valid
@@ -43,17 +45,21 @@
 public class MarkComponentValidTask implements IReplicaTask {
 
     private final long masterLsn;
+    private final long lastComponentId;
     private final String file;
 
-    public MarkComponentValidTask(String file, long masterLsn) {
+    public MarkComponentValidTask(String file, long masterLsn, long lastComponentId) {
         this.file = file;
+        this.lastComponentId = lastComponentId;
         this.masterLsn = masterLsn;
     }
 
     @Override
     public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
         try {
-            if (masterLsn > 0) {
+            if (masterLsn == IndexSynchronizer.BULKLOAD_LSN) {
+                updateBulkLoadedLastComponentSequence(appCtx);
+            } else if (masterLsn != IndexSynchronizer.MERGE_LSN) {
                 ensureComponentLsnFlushed(appCtx);
             }
             // delete mask
@@ -65,6 +71,14 @@
         }
     }
 
+    private void updateBulkLoadedLastComponentSequence(INcApplicationContext appCtx) throws HyracksDataException {
+        final ResourceReference indexRef = ResourceReference.of(file);
+        final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+        final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+        final long componentSequence = IndexComponentFileReference.of(indexRef.getName()).getSequenceEnd();
+        indexCheckpointManager.advanceValidComponentSequence(componentSequence);
+    }
+
     private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
             throws HyracksDataException, InterruptedException {
         final ResourceReference indexRef = ResourceReference.of(file);
@@ -81,8 +95,8 @@
                 indexCheckpointManager.wait(replicationTimeOut);
                 replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
             }
-            final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
-            indexCheckpointManager.replicated(componentEndTime, masterLsn);
+            final long componentSequence = IndexComponentFileReference.of(indexRef.getName()).getSequenceEnd();
+            indexCheckpointManager.replicated(componentSequence, masterLsn, lastComponentId);
         }
     }
 
@@ -97,6 +111,7 @@
             final DataOutputStream dos = new DataOutputStream(out);
             dos.writeUTF(file);
             dos.writeLong(masterLsn);
+            dos.writeLong(lastComponentId);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
@@ -105,6 +120,7 @@
     public static MarkComponentValidTask create(DataInput input) throws IOException {
         final String indexFile = input.readUTF();
         final long lsn = input.readLong();
-        return new MarkComponentValidTask(indexFile, lsn);
+        final long lastComponentId = input.readLong();
+        return new MarkComponentValidTask(indexFile, lsn, lastComponentId);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index ca0fcca..ae36c13 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -30,16 +30,17 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
 import org.apache.asterix.replication.management.NetworkingUtil;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -98,7 +99,8 @@
         final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
         final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
         indexCheckpointManager.delete();
-        indexCheckpointManager.init(currentLSN);
+        indexCheckpointManager.init(Long.MIN_VALUE, currentLSN,
+                LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId());
         LOGGER.info(() -> "Checkpoint index: " + indexRef);
     }
 
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
index 95ae690..30a5595 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -24,7 +24,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.replication.messaging.ComponentMaskTask;
@@ -37,12 +37,15 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 public class IndexSynchronizer {
 
     private static final Logger LOGGER = LogManager.getLogger();
+    public static final long MERGE_LSN = -1;
+    public static final long BULKLOAD_LSN = -2;
     private final IReplicationJob job;
     private final INcApplicationContext appCtx;
 
@@ -91,7 +94,8 @@
         final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
         job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
         // send mark component valid
-        MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn());
+        MarkComponentValidTask markValidTask =
+                new MarkComponentValidTask(indexFile, getReplicatedComponentLsn(), getReplicatedComponentId());
         ReplicationProtocol.sendTo(replica, markValidTask);
         ReplicationProtocol.waitForAck(replica);
         LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
@@ -118,12 +122,28 @@
 
     private long getReplicatedComponentLsn() throws HyracksDataException {
         final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+        if (indexReplJob.getLSMOpType() == LSMOperationType.MERGE) {
+            return MERGE_LSN;
+        } else if (indexReplJob.getLSMOpType() == LSMOperationType.LOAD) {
+            return BULKLOAD_LSN;
+        }
+
         if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
-            return AbstractLSMIOOperationCallback.INVALID;
+            return LSMIOOperationCallback.INVALID_LSN;
         }
         final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
         final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
-        return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+        return ((LSMIOOperationCallback) lsmIndex.getIOOperationCallback())
                 .getComponentLSN(ctx.getComponentsToBeReplicated());
     }
+
+    private long getReplicatedComponentId() throws HyracksDataException {
+        final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+        if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+            return -1L;
+        }
+        final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
+        LSMComponentId id = (LSMComponentId) ctx.getComponentsToBeReplicated().get(0).getId();
+        return id.getMinId();
+    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
index ef85977..09f1205 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -25,6 +25,8 @@
 import org.apache.asterix.replication.api.PartitionReplica;
 import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
 import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * Performs the steps required to ensure any newly added replica
@@ -60,9 +62,17 @@
     }
 
     private void checkpointReplicaIndexes() throws IOException {
+        final int partition = replica.getIdentifier().getPartition();
         CheckpointPartitionIndexesTask task =
-                new CheckpointPartitionIndexesTask(replica.getIdentifier().getPartition());
+                new CheckpointPartitionIndexesTask(partition, getPartitionMaxComponentId(partition));
         ReplicationProtocol.sendTo(replica, task);
         ReplicationProtocol.waitForAck(replica);
     }
+
+    private long getPartitionMaxComponentId(int partition) throws HyracksDataException {
+        final IReplicationStrategy replStrategy = appCtx.getReplicationManager().getReplicationStrategy();
+        final PersistentLocalResourceRepository localResourceRepository =
+                (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+        return localResourceRepository.getReplicatedIndexesMaxComponentId(partition, replStrategy);
+    }
 }
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 22d1bfc..0d0eb02 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -105,10 +105,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-control-nc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-common</artifactId>
     </dependency>
     <dependency>
@@ -189,12 +185,16 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>1.10.19</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+      <version>8.2.2</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
index 34f961c..6566a04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementAggregateDescriptor.java
@@ -38,7 +38,7 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new FirstElementEvalFactory(args, false);
+        return new FirstElementEvalFactory(args, false, sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
index bea8d5f..86785e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
@@ -19,12 +19,14 @@
 package org.apache.asterix.runtime.aggregates.collections;
 
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -35,15 +37,17 @@
     private static final long serialVersionUID = 1L;
     private final IScalarEvaluatorFactory[] args;
     private final boolean isLocal;
+    private final SourceLocation sourceLoc;
 
-    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal) {
+    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
         this.args = args;
         this.isLocal = isLocal;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-        return new IAggregateEvaluator() {
+        return new AbstractAggregateFunction(sourceLoc) {
 
             private boolean first = true;
             // Needs to copy the bytes from inputVal to outputVal because the byte space of inputVal could be re-used
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
index 4e35950..6cc119e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateDescriptor.java
@@ -58,6 +58,6 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new ListifyAggregateFunctionEvalFactory(args, oltype);
+        return new ListifyAggregateFunctionEvalFactory(args, oltype, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
index 2bb9eb3..e2e2859 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/ListifyAggregateFunctionEvalFactory.java
@@ -22,12 +22,14 @@
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -38,15 +40,18 @@
     private static final long serialVersionUID = 1L;
     private IScalarEvaluatorFactory[] args;
     private final AOrderedListType orderedlistType;
+    private final SourceLocation sourceLoc;
 
-    public ListifyAggregateFunctionEvalFactory(IScalarEvaluatorFactory[] args, AOrderedListType type) {
+    public ListifyAggregateFunctionEvalFactory(IScalarEvaluatorFactory[] args, AOrderedListType type,
+            SourceLocation sourceLoc) {
         this.args = args;
         this.orderedlistType = type;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-        return new IAggregateEvaluator() {
+        return new AbstractAggregateFunction(sourceLoc) {
 
             private IPointable inputVal = new VoidPointable();
             private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
index 5c73390..baf89e0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/LocalFirstElementAggregateDescriptor.java
@@ -37,7 +37,7 @@
 
     @Override
     public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new FirstElementEvalFactory(args, true);
+        return new FirstElementEvalFactory(args, true, sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index e929f0f..c8ebff6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -56,7 +56,7 @@
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 // Use ScanCollection to iterate over list items.
                 ScanCollectionUnnestingFunctionFactory scanCollectionFactory =
-                        new ScanCollectionUnnestingFunctionFactory(args[0]);
+                        new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
                 return new GenericScalarAggregateFunction(aggFuncFactory.createAggregateEvaluator(ctx),
                         scanCollectionFactory, ctx);
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java
new file mode 100644
index 0000000..3054323
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * 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.asterix.runtime.aggregates.serializable.std;
+
+import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractSerializableAggregateFunction implements ISerializedAggregateEvaluator {
+    protected final SourceLocation sourceLoc;
+
+    protected AbstractSerializableAggregateFunction(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
index 9d4712b7..563ff00 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
@@ -49,16 +49,16 @@
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSerializableAvgAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableAvgAggregateFunction extends AbstractSerializableAggregateFunction {
     private static final int SUM_FIELD_ID = 0;
     private static final int COUNT_FIELD_ID = 1;
 
@@ -90,8 +90,9 @@
     private ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -136,7 +137,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.AVG, bytes[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset], aggType.serialize());
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -173,7 +174,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, bytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset]);
         }
         BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
         BufferSerDeUtil.writeLong(count, state, start + COUNT_OFFSET);
@@ -252,7 +253,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, serBytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, serBytes[offset]);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
index a2a2d1f..c37a524 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableCountAggregateFunction.java
@@ -31,9 +31,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -41,7 +41,7 @@
 /**
  * count(NULL) returns NULL.
  */
-public abstract class AbstractSerializableCountAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableCountAggregateFunction extends AbstractSerializableAggregateFunction {
     private static final int MET_NULL_OFFSET = 0;
     private static final int COUNT_OFFSET = 1;
 
@@ -55,8 +55,9 @@
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
 
-    public AbstractSerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
index 8ba3a05..a4e1090 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableSumAggregateFunction.java
@@ -44,15 +44,15 @@
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluator;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSerializableSumAggregateFunction implements ISerializedAggregateEvaluator {
+public abstract class AbstractSerializableSumAggregateFunction extends AbstractSerializableAggregateFunction {
     protected static final int AGG_TYPE_OFFSET = 0;
     private static final int SUM_OFFSET = 1;
 
@@ -67,8 +67,9 @@
     @SuppressWarnings("rawtypes")
     public ISerializerDeserializer serde;
 
-    public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -100,7 +101,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.SUM, bytes[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset], aggType.serialize());
         }
 
         if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -147,7 +148,7 @@
                 break;
             }
             default:
-                throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, bytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, bytes[offset]);
         }
         state[start + AGG_TYPE_OFFSET] = aggType.serialize();
         BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
@@ -206,7 +207,7 @@
                     break;
                 }
                 default:
-                    throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
index ab4aa63..6601243 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableAvgAggregateFunction(args, ctx);
+                return new SerializableAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
index 4c32fe0..611538f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
index ec86cf8..506a9f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
@@ -56,7 +56,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableCountAggregateFunction(args, ctx);
+                return new SerializableCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
index fb8d8d7..0815dfd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateFunction.java
@@ -21,13 +21,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * count(NULL) returns NULL.
  */
 public class SerializableCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
-    public SerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
index 8f40f8a..f9a8834 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableGlobalAvgAggregateFunction(args, ctx);
+                return new SerializableGlobalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
index a916355..9bb4312 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableGlobalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableGlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableGlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
index 19149d5..c67da27 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableGlobalSqlAvgAggregateFunction(args, ctx);
+                return new SerializableGlobalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
index 8ddccdc..5fe1d7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableGlobalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableGlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableGlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
index 28b3aff..676eabb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableIntermediateAvgAggregateFunction(args, ctx);
+                return new SerializableIntermediateAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
index 3d56133..d25f603 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableIntermediateAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableIntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableIntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
index 43e63cd..86bb15d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
@@ -55,7 +55,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableIntermediateSqlAvgAggregateFunction(args, ctx);
+                return new SerializableIntermediateSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
index f8449a5..d3f9fea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableIntermediateSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableIntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableIntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
index 9c94ad2..39ae425 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableLocalAvgAggregateFunction(args, ctx);
+                return new SerializableLocalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
index 0e7ba07..f93bbd7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateFunction.java
@@ -26,13 +26,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableLocalAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableLocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableLocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
index d0819e0..c96eb2e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableLocalSqlAvgAggregateFunction(args, ctx);
+                return new SerializableLocalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
index 8925c31..c6195aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateFunction.java
@@ -24,13 +24,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableLocalSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableLocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableLocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
index 2cd75f1..3798b49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, true, ctx);
+                return new SerializableSqlSumAggregateFunction(args, true, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
index 3e0546c..54d24f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, true, ctx);
+                return new SerializableSumAggregateFunction(args, true, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
index af057ee..f34827a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlAvgAggregateFunction(args, ctx);
+                return new SerializableSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
index 3ef7aeb..608f2a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateFunction.java
@@ -23,13 +23,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SerializableSqlAvgAggregateFunction extends AbstractSerializableAvgAggregateFunction {
 
-    public SerializableSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
index 3beb59f..b44c114 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
@@ -56,7 +56,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlCountAggregateFunction(args, ctx);
+                return new SerializableSqlCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
index 257735f..241180c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateFunction.java
@@ -21,14 +21,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * count(NULL) returns NULL.
  */
 public class SerializableSqlCountAggregateFunction extends AbstractSerializableCountAggregateFunction {
-    public SerializableSqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SerializableSqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
index 4e8d515..d51a6fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSqlSumAggregateFunction(args, false, ctx);
+                return new SerializableSqlSumAggregateFunction(args, false, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
index c42c350..38033f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSumAggregateFunction.java
@@ -30,13 +30,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SerializableSqlSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
     private final boolean isLocalAgg;
 
     public SerializableSqlSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context) throws HyracksDataException {
-        super(args, context);
+            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -50,7 +51,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SQL_SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
index 8d2bb6c..43eea5b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public ISerializedAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SerializableSumAggregateFunction(args, false, ctx);
+                return new SerializableSumAggregateFunction(args, false, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
index 60e34fa..278914f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSumAggregateFunction.java
@@ -31,13 +31,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SerializableSumAggregateFunction extends AbstractSerializableSumAggregateFunction {
     private final boolean isLocalAgg;
 
     public SerializableSumAggregateFunction(IScalarEvaluatorFactory[] args, boolean isLocalAgg,
-            IHyracksTaskContext context) throws HyracksDataException {
-        super(args, context);
+            IHyracksTaskContext context, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -58,7 +59,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java
new file mode 100644
index 0000000..1ea28cc
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * 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.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractAggregateFunction implements IAggregateEvaluator {
+    protected final SourceLocation sourceLoc;
+
+    public AbstractAggregateFunction(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
index 37cf766..1dde27b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
@@ -47,19 +47,19 @@
 import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractAvgAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractAvgAggregateFunction extends AbstractAggregateFunction {
     private static final int SUM_FIELD_ID = 0;
     private static final int COUNT_FIELD_ID = 1;
 
@@ -93,8 +93,9 @@
     private ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
         recType = new ARecordType(null, new String[] { "sum", "count" },
                 new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
@@ -134,7 +135,7 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.AVG, data[offset], aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset], aggType.serialize());
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -171,7 +172,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.AVG, data[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset]);
             }
         }
     }
@@ -234,7 +235,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException("intermediate/global-Avg", serBytes[offset]);
+                throw new UnsupportedItemTypeException(sourceLoc, "intermediate/global-Avg", serBytes[offset]);
             }
         }
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
index 6dc4759..d156574 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractCountAggregateFunction.java
@@ -27,11 +27,11 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -40,7 +40,7 @@
 /**
  * COUNT returns the number of items in the given list. Note that COUNT(NULL) is not allowed.
  */
-public abstract class AbstractCountAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractCountAggregateFunction extends AbstractAggregateFunction {
     private AMutableInt64 result = new AMutableInt64(-1);
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<AInt64> int64Serde =
@@ -51,8 +51,9 @@
 
     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
 
-    public AbstractCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
index 0ba7c71..1aa609f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractMinMaxAggregateFunction.java
@@ -26,19 +26,19 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.om.types.hierachy.ITypeConvertComputer;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractMinMaxAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractMinMaxAggregateFunction extends AbstractAggregateFunction {
     private IPointable inputVal = new VoidPointable();
     private ArrayBackedValueStorage outputVal = new ArrayBackedValueStorage();
     private ArrayBackedValueStorage tempValForCasting = new ArrayBackedValueStorage();
@@ -50,8 +50,9 @@
     private ITypeConvertComputer tpc;
     private final boolean isMin;
 
-    public AbstractMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin)
-            throws HyracksDataException {
+    public AbstractMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
         this.isMin = isMin;
     }
@@ -87,7 +88,7 @@
             // Initialize min value.
             outputVal.assign(inputVal);
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException("min/max", typeTag.serialize(), aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, "min/max", typeTag.serialize(), aggType.serialize());
         } else {
 
             // If a system_null is encountered locally, it would be an error; otherwise if it is seen
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
index fd841e0..a8fa552 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractSumAggregateFunction.java
@@ -41,18 +41,18 @@
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
-import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public abstract class AbstractSumAggregateFunction implements IAggregateEvaluator {
+public abstract class AbstractSumAggregateFunction extends AbstractAggregateFunction {
     protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private IPointable inputVal = new VoidPointable();
     private IScalarEvaluator eval;
@@ -67,8 +67,9 @@
     @SuppressWarnings("rawtypes")
     protected ISerializerDeserializer serde;
 
-    public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
+    public AbstractSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
         eval = args[0].createScalarEvaluator(context);
     }
 
@@ -94,7 +95,8 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new IncompatibleTypeException(BuiltinFunctions.SUM, typeTag.serialize(), aggType.serialize());
+            throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.SUM, typeTag.serialize(),
+                    aggType.serialize());
         }
 
         if (ATypeHierarchy.canPromote(aggType, typeTag)) {
@@ -137,7 +139,7 @@
                 break;
             }
             default: {
-                throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         }
     }
@@ -194,7 +196,7 @@
                     break;
                 }
                 default:
-                    throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, aggType.serialize());
+                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM, aggType.serialize());
             }
         } catch (IOException e) {
             throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
index 93023e8..46fe3e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new AvgAggregateFunction(args, ctx);
+                return new AvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
index 6408e56..3572c4f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class AvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public AvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public AvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
-        super(args, context);
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
index f981ece..9f64f19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
@@ -54,7 +54,7 @@
 
             @Override
             public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new CountAggregateFunction(args, ctx);
+                return new CountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
index 94b653d..829e8ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateFunction.java
@@ -21,15 +21,16 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * COUNT returns the number of items in the given list. Note that COUNT(NULL) is not allowed.
  */
 public class CountAggregateFunction extends AbstractCountAggregateFunction {
 
-    public CountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+    public CountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, SourceLocation sourceLoc)
             throws HyracksDataException {
-        super(args, context);
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
index 199e5e5..a624d97 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new GlobalAvgAggregateFunction(args, ctx);
+                return new GlobalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
index 30a2746..425d928 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class GlobalAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public GlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public GlobalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
index 90124d1..4e3e8fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new GlobalSqlAvgAggregateFunction(args, ctx);
+                return new GlobalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
index 893613b..fb59615 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class GlobalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public GlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public GlobalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
index 9faccd8..391a5cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new IntermediateAvgAggregateFunction(args, ctx);
+                return new IntermediateAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
index ad64ef3..c182fcd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class IntermediateAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public IntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public IntermediateAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
index 144614a..25ca512 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new IntermediateSqlAvgAggregateFunction(args, ctx);
+                return new IntermediateSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
index bc260e6..eee736d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class IntermediateSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public IntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public IntermediateSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
index 2598da1..2e23c03 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new LocalAvgAggregateFunction(args, ctx);
+                return new LocalAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
index 685af83..0eab063 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateFunction.java
@@ -23,14 +23,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class LocalAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public LocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public LocalAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
index b2fc3b2..ea9b2ae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, false, true);
+                return new MinMaxAggregateFunction(args, ctx, false, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
index c29149d..d4355df 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, true, true);
+                return new MinMaxAggregateFunction(args, ctx, true, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
index dc8fbc5..777d50f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new LocalSqlAvgAggregateFunction(args, ctx);
+                return new LocalSqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
index 8c80c42..66a75e0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class LocalSqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public LocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public LocalSqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
index ab999d0..57da62b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMaxAggregateDescriptor.java
@@ -54,7 +54,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, false, true);
+                return new SqlMinMaxAggregateFunction(args, ctx, false, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
index ee45d48..ca2f65d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlMinAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, true, true);
+                return new SqlMinMaxAggregateFunction(args, ctx, true, true, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
index aac1927..1262399 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSumAggregateDescriptor.java
@@ -53,8 +53,8 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, true);
-            };
+                return new SqlSumAggregateFunction(args, ctx, true, sourceLoc);
+            }
         };
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
index dd6c974..0e11541 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
@@ -53,8 +53,8 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, true);
-            };
+                return new SumAggregateFunction(args, ctx, true, sourceLoc);
+            }
         };
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
index 3afa4fb..2575440 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MaxAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, false, false);
+                return new MinMaxAggregateFunction(args, ctx, false, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
index 4328618..dfff5f7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new MinMaxAggregateFunction(args, ctx, true, false);
+                return new MinMaxAggregateFunction(args, ctx, true, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
index 5d28018..3aeb58a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/MinMaxAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class MinMaxAggregateFunction extends AbstractMinMaxAggregateFunction {
     private final boolean isLocalAgg;
 
     public MinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
-            boolean isLocalAgg) throws HyracksDataException {
-        super(args, context, isMin);
+            boolean isLocalAgg, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, isMin, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -48,7 +49,7 @@
     @Override
     protected void processSystemNull() throws HyracksDataException {
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, "min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
index 0cc1b90..5f9cc46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
@@ -53,7 +53,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlAvgAggregateFunction(args, ctx);
+                return new SqlAvgAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
index e6c1233..8e8f60c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateFunction.java
@@ -22,14 +22,15 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class SqlAvgAggregateFunction extends AbstractAvgAggregateFunction {
 
-    public SqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlAvgAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
index a825df4..91d31c3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
@@ -55,7 +55,7 @@
 
             @Override
             public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new SqlCountAggregateFunction(args, ctx);
+                return new SqlCountAggregateFunction(args, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
index 7396d39..68cb37b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateFunction.java
@@ -21,15 +21,16 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * COUNT returns the number of non-null items in the given list. Note that COUNT(NULL) is not allowed.
  */
 public class SqlCountAggregateFunction extends AbstractCountAggregateFunction {
 
-    public SqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlCountAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
index d670a66..d5b86f9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMaxAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, false, false);
+                return new SqlMinMaxAggregateFunction(args, ctx, false, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
index 648dd2d..3abaa2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinAggregateDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlMinMaxAggregateFunction(args, ctx, true, false);
+                return new SqlMinMaxAggregateFunction(args, ctx, true, false, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
index 2ad0e55..f6415d6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlMinMaxAggregateFunction.java
@@ -25,13 +25,14 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SqlMinMaxAggregateFunction extends AbstractMinMaxAggregateFunction {
     private final boolean isLocalAgg;
 
     public SqlMinMaxAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isMin,
-            boolean isLocalAgg) throws HyracksDataException {
-        super(args, context, isMin);
+            boolean isLocalAgg, SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, isMin, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -42,7 +43,7 @@
     @Override
     protected void processSystemNull() throws HyracksDataException {
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException("min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, "min/max", ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
index e3b0187..b7fb755 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateDescriptor.java
@@ -52,8 +52,8 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SqlSumAggregateFunction(args, ctx, false);
-            };
+                return new SqlSumAggregateFunction(args, ctx, false, sourceLoc);
+            }
         };
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
index 663ed69..8b4bf7c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSumAggregateFunction.java
@@ -29,13 +29,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SqlSumAggregateFunction extends AbstractSumAggregateFunction {
     private final boolean isLocalAgg;
 
-    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
-            throws HyracksDataException {
-        super(args, context);
+    public SqlSumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -49,7 +50,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SQL_SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SQL_SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
index 34014a1..df429b0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateDescriptor.java
@@ -52,8 +52,8 @@
             @Override
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
-                return new SumAggregateFunction(args, ctx, false);
-            };
+                return new SumAggregateFunction(args, ctx, false, sourceLoc);
+            }
         };
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
index 2b3beef..abcac77 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SumAggregateFunction.java
@@ -29,13 +29,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class SumAggregateFunction extends AbstractSumAggregateFunction {
     private final boolean isLocalAgg;
 
-    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg)
-            throws HyracksDataException {
-        super(args, context);
+    public SumAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context, boolean isLocalAgg,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
         this.isLocalAgg = isLocalAgg;
     }
 
@@ -55,7 +56,8 @@
         // but if all input value are system null, then we should return
         // null in finish().
         if (isLocalAgg) {
-            throw new UnsupportedItemTypeException(BuiltinFunctions.SUM, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+            throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.SUM,
+                    ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
index dc2b355..fd7ccea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -58,7 +59,7 @@
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
 
-                return new IAggregateEvaluator() {
+                return new AbstractAggregateFunction(sourceLoc) {
 
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     @SuppressWarnings("rawtypes")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
index e9330d3..def848d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.aggregates.std.AbstractAggregateFunction;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -56,7 +57,7 @@
             public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
                     throws HyracksDataException {
 
-                return new IAggregateEvaluator() {
+                return new AbstractAggregateFunction(sourceLoc) {
 
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     @SuppressWarnings("rawtypes")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
index 2712a47..96ce843 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
@@ -41,7 +41,7 @@
     }
 
     @Override
-    public boolean accept(IFrameTupleReference tuple) throws Exception {
+    public boolean accept(IFrameTupleReference tuple) throws HyracksDataException {
         eval.evaluate(tuple, p);
         return boolInspector.getBooleanValue(p.getByteArray(), p.getStartOffset(), p.getLength());
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/ListAccessorFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/ListAccessorFactory.java
new file mode 100644
index 0000000..472d9ca
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/ListAccessorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.asterix.runtime.base;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+
+public class ListAccessorFactory implements IObjectFactory<ListAccessor, ATypeTag> {
+
+    @Override
+    public ListAccessor create(ATypeTag arg) {
+        return new ListAccessor();
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
index 98943a4..23581e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
@@ -93,7 +93,7 @@
                                 aPoint.setValue(cX, cY);
                                 pointSerde.serialize(aPoint, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
index e7a3f02..222ae76 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
@@ -88,7 +88,7 @@
                                 aDouble.setValue(radius);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
index d3069ee..60ee118 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
@@ -137,7 +137,7 @@
                                         startOffset + APolygonSerializerDeserializer.getNumberOfPointsOffset());
 
                                 if (numOfPoints < 3) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
                                 listBuilder.reset(pointListType);
@@ -153,7 +153,7 @@
                                 }
                                 listBuilder.write(out, true);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_LINE_TYPE_TAG, ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG,
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
index ad9dcda..487a632 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
@@ -88,7 +88,7 @@
                                 aDouble.setValue(x);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
index 6b96ddd..9409837 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
@@ -92,7 +92,7 @@
                                 aDouble.setValue(y);
                                 doubleSerde.serialize(aDouble, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
index e9d7339..50f8f68 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
index d18864b..aa684bb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
@@ -114,7 +114,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
index c5ce56b..98d6a9d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
@@ -109,11 +109,11 @@
                                     aDateTime.setValue(endTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
index 777eb76..909b657 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
@@ -91,11 +91,11 @@
                                     aDate.setValue((int) (endTime));
                                     dateSerde.serialize(aDate, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
index 34161da..a62a8a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
@@ -93,11 +93,11 @@
                                     aDateTime.setValue(endTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
index 10a3612..4d545e8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
@@ -90,11 +90,11 @@
                                     aTime.setValue((int) (endTime));
                                     timeSerde.serialize(aTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
index e858c68..b209139 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
@@ -110,11 +110,11 @@
                                     aDateTime.setValue(startTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
index 84ed55f..901d685 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
@@ -94,11 +94,11 @@
                                     aDate.setValue((int) (startTime));
                                     dateSerde.serialize(aDate, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
index d847b52..63cd481 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
@@ -91,11 +91,11 @@
                                     aDateTime.setValue(startTime);
                                     datetimeSerde.serialize(aDateTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
index e59a3d2..9a1b86f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
@@ -92,11 +92,11 @@
                                     aTime.setValue((int) (startTime));
                                     timeSerde.serialize(aTime, out);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
                         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
index fcfbed3..a5101b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
@@ -112,7 +112,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
index 2a84320..6b9e573 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
@@ -115,7 +115,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
index 4709171..3b93969 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
index fe2bb39..e757e12 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
@@ -113,7 +113,7 @@
                             } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
                                 chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
index 21dcc67..6327a36 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
@@ -132,7 +132,7 @@
                                 result.set(resultStorage);
                                 return;
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
index c763d3c..03ddd73 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/CreateMBREvalFactory.java
@@ -85,6 +85,7 @@
                 int startOffset2 = inputArg2.getStartOffset();
 
                 try {
+                    resultStorage.reset();
                     if (data0[startOffset0] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                             || data1[startOffset1] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                             || data2[startOffset2] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
@@ -99,7 +100,6 @@
                         result.set(resultStorage);
                         return;
                     }
-                    resultStorage.reset();
                     if (data1[startOffset1] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
                         throw new TypeMismatchException(BuiltinFunctions.CREATE_MBR, 1, data1[startOffset1],
                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
index dafc034..fd429d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
@@ -57,10 +57,8 @@
 
     protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     protected final DataOutput out = resultStorage.getDataOutput();
-    protected final TaggedValuePointable argLeft =
-            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-    protected final TaggedValuePointable argRight =
-            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
+    protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
     protected TaggedValuePointable[] argOptions;
     protected final IScalarEvaluator evalLeft;
     protected final IScalarEvaluator evalRight;
@@ -122,7 +120,7 @@
         for (int i = 0; i < optionArgsLength; i++) {
             this.evalOptions[i] = args[i + 2].createScalarEvaluator(context);
             this.outOptions[i] = VoidPointable.FACTORY.createPointable();
-            this.argOptions[i] = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+            this.argOptions[i] = TaggedValuePointable.FACTORY.createPointable();
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
index 3c97fc9..e834522 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
@@ -29,6 +29,8 @@
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 /**
  * Utility class for accessing serialized unordered and ordered lists.
@@ -107,6 +109,30 @@
         dos.write(listBytes, itemOffset, itemLength);
     }
 
+    /**
+     * @param itemIndex the index of the item requested
+     * @param pointable a pointable that will be set to point to the item requested
+     * @param storage if list is strongly typed, the item tag will be written followed by the item value to this storage
+     * @return true when the item requested has been written to the storage. false when a pointer to the item was set
+     * @throws IOException
+     */
+    public boolean getOrWriteItem(int itemIndex, IPointable pointable, ArrayBackedValueStorage storage)
+            throws IOException {
+        int itemOffset = getItemOffset(itemIndex);
+        int itemLength = getItemLength(itemOffset);
+        if (itemsAreSelfDescribing()) {
+            // +1 to account for the already included tag
+            pointable.set(listBytes, itemOffset, itemLength + 1);
+            return false;
+        } else {
+            storage.reset();
+            storage.getDataOutput().writeByte(itemType.serialize());
+            storage.getDataOutput().write(listBytes, itemOffset, itemLength);
+            pointable.set(storage);
+            return true;
+        }
+    }
+
     public byte[] getByteArray() {
         return listBytes;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index 620c543..50e3932 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -20,19 +20,15 @@
 
 import java.io.DataOutput;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -41,85 +37,52 @@
 
 public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
 
-    protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    protected DataOutput out = resultStorage.getDataOutput();
-    protected TaggedValuePointable argLeft = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-    protected TaggedValuePointable argRight = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-    protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
-    protected IPointable outRight = VoidPointable.FACTORY.createPointable();
-    protected IScalarEvaluator evalLeft;
-    protected IScalarEvaluator evalRight;
-    private ComparisonHelper ch = new ComparisonHelper();
-
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ABoolean> serde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ANull> nullSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput out = resultStorage.getDataOutput();
+    protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
+    protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
+    protected final IPointable outLeft = VoidPointable.FACTORY.createPointable();
+    protected final IPointable outRight = VoidPointable.FACTORY.createPointable();
+    protected final IScalarEvaluator evalLeft;
+    protected final IScalarEvaluator evalRight;
+    protected final SourceLocation sourceLoc;
+    private final ComparisonHelper ch;
+    private Number leftValue;
+    private Number rightValue;
 
     public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws HyracksDataException {
-        this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
-        this.evalRight = evalRightFactory.createScalarEvaluator(context);
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        this.evalLeft = evalLeftFactory.createScalarEvaluator(ctx);
+        this.evalRight = evalRightFactory.createScalarEvaluator(ctx);
+        this.sourceLoc = sourceLoc;
+        ch = new ComparisonHelper(sourceLoc);
+        leftValue = getValueOfConstantEval(evalLeftFactory);
+        rightValue = getValueOfConstantEval(evalRightFactory);
+    }
+
+    private Number getValueOfConstantEval(IScalarEvaluatorFactory factory) throws HyracksDataException {
+        if (factory instanceof ConstantEvalFactory) {
+            return ch.getNumberValue(((ConstantEvalFactory) factory).getValue());
+        }
+        return null;
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        resultStorage.reset();
-
         // Evaluates input args.
         evalLeft.evaluate(tuple, argLeft);
         evalRight.evaluate(tuple, argRight);
         argLeft.getValue(outLeft);
         argRight.getValue(outRight);
 
-        // checks whether we can apply >, >=, <, and <= to the given type since
-        // these operations cannot be defined for certain types.
-        if (isTotallyOrderable()) {
-            checkTotallyOrderable();
-        }
-
-        // Checks whether two types are comparable
-        if (comparabilityCheck()) {
-            // Two types can be compared
-            int r = compareResults();
-            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
-            serde.serialize(b, out);
-        } else {
-            // result:NULL - two types cannot be compared.
-            nullSerde.serialize(ANull.NULL, out);
-        }
-        result.set(resultStorage);
+        evaluateImpl(result);
     }
 
-    protected abstract boolean isTotallyOrderable();
-
-    protected abstract boolean getComparisonResult(int r);
-
-    // checks whether we can apply >, >=, <, and <= operations to the given type since
-    // these operations can not be defined for certain types.
-    protected void checkTotallyOrderable() throws HyracksDataException {
-        if (argLeft.getLength() != 0) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-            switch (typeTag) {
-                case DURATION:
-                case INTERVAL:
-                case LINE:
-                case POINT:
-                case POINT3D:
-                case POLYGON:
-                case CIRCLE:
-                case RECTANGLE:
-                    throw new UnsupportedTypeException(ComparisonHelper.COMPARISON, argLeft.getTag());
-                default:
-                    return;
-            }
-        }
-    }
+    protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
     // checks whether two types are comparable
-    protected boolean comparabilityCheck() {
+    boolean comparabilityCheck() {
         // Checks whether two types are comparable or not
         ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
         ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
@@ -128,10 +91,9 @@
         return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
     }
 
-    protected int compareResults() throws HyracksDataException {
-        int result = ch.compare(EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag()),
-                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag()), outLeft, outRight);
-        return result;
+    int compare() throws HyracksDataException {
+        ATypeTag leftTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+        ATypeTag rightTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
+        return ch.compare(leftTypeTag, rightTypeTag, outLeft, outRight, leftValue, rightValue);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
new file mode 100644
index 0000000..27ae471
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
@@ -0,0 +1,47 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
+
+    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IScalarEvaluatorFactory evalRightFactory,
+            IHyracksTaskContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        if (comparabilityCheck() && compare() == 0) {
+            resultStorage.reset();
+            writeEqualsResult();
+            result.set(resultStorage);
+        } else {
+            result.set(argLeft);
+        }
+    }
+
+    protected abstract void writeEqualsResult() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
new file mode 100644
index 0000000..2d8c499
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractValueComparisonEvaluator extends AbstractComparisonEvaluator {
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<ABoolean> serde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+    @SuppressWarnings("unchecked")
+    protected ISerializerDeserializer<ANull> nullSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    }
+
+    @Override
+    protected void evaluateImpl(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+
+        // checks whether we can apply >, >=, <, and <= to the given type since
+        // these operations cannot be defined for certain types.
+        if (isTotallyOrderable()) {
+            checkTotallyOrderable();
+        }
+
+        // Checks whether two types are comparable
+        if (comparabilityCheck()) {
+            // Two types can be compared
+            int r = compare();
+            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
+            serde.serialize(b, out);
+        } else {
+            // result:NULL - two types cannot be compared.
+            nullSerde.serialize(ANull.NULL, out);
+        }
+        result.set(resultStorage);
+    }
+
+    protected abstract boolean isTotallyOrderable();
+
+    protected abstract boolean getComparisonResult(int r);
+
+    // checks whether we can apply >, >=, <, and <= operations to the given type since
+    // these operations can not be defined for certain types.
+    protected void checkTotallyOrderable() throws HyracksDataException {
+        if (argLeft.getLength() != 0) {
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+            switch (typeTag) {
+                case DURATION:
+                case INTERVAL:
+                case LINE:
+                case POINT:
+                case POINT3D:
+                case POLYGON:
+                case CIRCLE:
+                case RECTANGLE:
+                    throw new UnsupportedTypeException(sourceLoc, ComparisonHelper.COMPARISON, argLeft.getTag());
+                default:
+                    return;
+            }
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
index 163bd9f..96c5250 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -37,15 +37,17 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
+
+import static org.apache.asterix.om.types.ATypeTag.TINYINT;
 
 public class ComparisonHelper implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -74,21 +76,27 @@
     private final IBinaryComparator byteArrayComparator =
             new PointableBinaryComparatorFactory(ByteArrayPointable.FACTORY).createBinaryComparator();
 
-    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+    private final SourceLocation sourceLoc;
+
+    public ComparisonHelper(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
             throws HyracksDataException {
         switch (typeTag1) {
             case TINYINT:
-                return compareInt8WithArg(typeTag2, arg1, arg2);
+                return compareInt8WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case SMALLINT:
-                return compareInt16WithArg(typeTag2, arg1, arg2);
+                return compareInt16WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case INTEGER:
-                return compareInt32WithArg(typeTag2, arg1, arg2);
+                return compareInt32WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case BIGINT:
-                return compareInt64WithArg(typeTag2, arg1, arg2);
+                return compareInt64WithArg(typeTag2, arg1, arg2, obj1, obj2);
             case FLOAT:
-                return compareFloatWithArg(typeTag2, arg1, arg2);
+                return compareFloatWithArg(typeTag2, arg1, arg2, obj1, obj2);
             case DOUBLE:
-                return compareDoubleWithArg(typeTag2, arg1, arg2);
+                return compareDoubleWithArg(typeTag2, arg1, arg2, obj1, obj2);
             case STRING:
                 return compareStringWithArg(typeTag2, arg1, arg2);
             case BOOLEAN:
@@ -98,10 +106,16 @@
         }
     }
 
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+            throws HyracksDataException {
+        return compare(typeTag1, typeTag2, arg1, arg2, null, null);
+    }
+
     private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
             IPointable arg2) throws HyracksDataException {
         if (expectedTypeTag != actualTypeTag) {
-            throw new IncompatibleTypeException(COMPARISON, actualTypeTag.serialize(), expectedTypeTag.serialize());
+            throw new IncompatibleTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize(),
+                    expectedTypeTag.serialize());
         }
         int result;
         byte[] leftBytes = arg1.getByteArray();
@@ -160,7 +174,7 @@
                         rightLen);
                 break;
             default:
-                throw new UnsupportedTypeException(COMPARISON, actualTypeTag.serialize());
+                throw new UnsupportedTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize());
         }
         return result;
     }
@@ -171,7 +185,8 @@
             byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
             return compareByte(b0, b1);
         }
-        throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG, typeTag2.serialize());
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG,
+                typeTag2.serialize());
     }
 
     private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
@@ -179,191 +194,197 @@
             return strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
                     arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength() - 1);
         }
-        throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG, typeTag2.serialize());
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                typeTag2.serialize());
     }
 
-    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        double s = ADoubleSerializerDeserializer.getDouble(leftBytes, leftOffset);
+        double s = getOrDeserializeDouble(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareDouble(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareDouble(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareDouble(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareDouble(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareDouble(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default: {
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
                         typeTag2.serialize());
             }
         }
     }
 
-    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        float s = FloatPointable.getFloat(leftBytes, leftOffset);
+        float s = getOrDeserializeFloat(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareFloat(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareFloat(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareFloat(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareFloat(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        long s = AInt64SerializerDeserializer.getLong(leftBytes, leftOffset);
+        long s = getOrDeserializeBigInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareLong(s, AInt8SerializerDeserializer.getByte(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
             case SMALLINT:
-                return compareLong(s, AInt16SerializerDeserializer.getShort(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
             case INTEGER:
-                return compareLong(s, AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
             case BIGINT:
-                return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        int s = IntegerPointable.getInteger(leftBytes, leftOffset);
+        int s = getOrDeserializeInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
-            case TINYINT: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case SMALLINT: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INTEGER: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case BIGINT: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
+            case TINYINT:
+                return compareInt(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareInt(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
-    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightOffset = arg2.getStartOffset();
 
-        short s = AInt16SerializerDeserializer.getShort(leftBytes, leftOffset);
+        short s = getOrDeserializeSmallInt(leftBytes, leftOffset, obj1);
         switch (typeTag2) {
-            case TINYINT: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case SMALLINT: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case INTEGER: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case BIGINT: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+            case TINYINT:
+                return compareShort(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                         typeTag2.serialize());
-            }
         }
     }
 
-    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
         byte[] leftBytes = arg1.getByteArray();
         int leftStart = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
         int rightStart = arg2.getStartOffset();
 
-        byte s = AInt8SerializerDeserializer.getByte(leftBytes, leftStart);
+        byte s = getOrDeserializeTinyInt(leftBytes, leftStart, obj1);
         switch (typeTag2) {
             case TINYINT:
-                return compareByte(s, AInt8SerializerDeserializer.getByte(rightBytes, rightStart));
+                return compareByte(s, getOrDeserializeTinyInt(rightBytes, rightStart, obj2));
             case SMALLINT:
-                return compareShort(s, AInt16SerializerDeserializer.getShort(rightBytes, rightStart));
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightStart, obj2));
             case INTEGER:
-                return compareInt(s, AInt32SerializerDeserializer.getInt(rightBytes, rightStart));
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightStart, obj2));
             case BIGINT:
-                return compareLong(s, AInt64SerializerDeserializer.getLong(rightBytes, rightStart));
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightStart, obj2));
             case FLOAT:
-                return compareFloat(s, AFloatSerializerDeserializer.getFloat(rightBytes, rightStart));
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightStart, obj2));
             case DOUBLE:
-                return compareDouble(s, ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart));
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightStart, obj2));
             default:
-                throw new IncompatibleTypeException(COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                         typeTag2.serialize());
         }
     }
 
+    private final byte getOrDeserializeTinyInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt8SerializerDeserializer.getByte(bytes, offset) : obj.byteValue();
+    }
+
+    private final short getOrDeserializeSmallInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt16SerializerDeserializer.getShort(bytes, offset) : obj.shortValue();
+    }
+
+    private final int getOrDeserializeInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt32SerializerDeserializer.getInt(bytes, offset) : obj.intValue();
+    }
+
+    private final long getOrDeserializeBigInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt64SerializerDeserializer.getLong(bytes, offset) : obj.longValue();
+    }
+
+    private final float getOrDeserializeFloat(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AFloatSerializerDeserializer.getFloat(bytes, offset) : obj.floatValue();
+    }
+
+    private final double getOrDeserializeDouble(byte[] bytes, int offset, Number obj) {
+        return obj == null ? ADoubleSerializerDeserializer.getDouble(bytes, offset) : obj.doubleValue();
+    }
+
     private final int compareByte(int v1, int v2) {
         if (v1 == v2) {
             return 0;
@@ -405,23 +426,41 @@
     }
 
     private final int compareFloat(float v1, float v2) {
-        if (v1 == v2) {
-            return 0;
-        } else if (v1 < v2) {
-            return -1;
-        } else {
-            return 1;
-        }
+        return Float.compare(v1, v2);
     }
 
     private final int compareDouble(double v1, double v2) {
-        if (v1 == v2) {
-            return 0;
-        } else if (v1 < v2) {
-            return -1;
-        } else {
-            return 1;
-        }
+        return Double.compare(v1, v2);
     }
 
+    /**
+     * When field value falls into the primitive type groups, we consider to cache its value instead of deserialize it
+     * every time.
+     *
+     * @param bytes
+     * @return primitive value as Number
+     */
+    public Number getNumberValue(byte[] bytes) {
+        ATypeTag aTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[0]);
+        int offset = 1;
+        if (aTypeTag == null) {
+            return null;
+        }
+        switch (aTypeTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(bytes, offset);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(bytes, offset);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(bytes, offset);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(bytes, offset);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(bytes, offset);
+            case DOUBLE:
+                return ADoubleSerializerDeserializer.getDouble(bytes, offset);
+            default:
+                return null;
+        }
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
index 21b19aa..abc796d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/DeepEqualAssessor.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index b277885..f38c3e5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 2826279..3815666 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index e12dd71..1598165 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index 7e97dfa..4198934 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 947dea1..478c652 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
new file mode 100644
index 0000000..7ed194c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MissingIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new MissingIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.MISSING_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<AMissing> missingSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        missingSerde.serialize(AMissing.MISSING, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
new file mode 100644
index 0000000..af50678
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NanIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NanIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NAN_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.NaN);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..a70bc7b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NegInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NegInfIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NEGINF_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.NEGATIVE_INFINITY);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index bfd5c98..6e03ef8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -50,7 +50,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
new file mode 100644
index 0000000..5df5142
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NullIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NullIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NULL_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ANull> nullSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        nullSerde.serialize(ANull.NULL, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..a1f4f13
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PosInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new PosInfIfEqualsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.POSINF_IF;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+
+                    final AMutableDouble equalsResult = new AMutableDouble(Double.POSITIVE_INFINITY);
+
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ADouble> doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+                    @Override
+                    protected void writeEqualsResult() throws HyracksDataException {
+                        doubleSerde.serialize(equalsResult, out);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
index e5e19ca..4337f28 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
@@ -46,7 +46,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new ABinaryHexStringConstructorDescriptor.ABinaryConstructorEvaluator(args[0],
-                        ByteArrayBase64ParserFactory.INSTANCE, ctx);
+                        ByteArrayBase64ParserFactory.INSTANCE, ctx, sourceLoc);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
index 720e9bc..dd25e49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -59,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx);
+                return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx, sourceLoc);
             }
         };
     }
@@ -70,6 +71,7 @@
     }
 
     static class ABinaryConstructorEvaluator implements IScalarEvaluator {
+        private final SourceLocation sourceLoc;
         private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
         private final DataOutput out = resultStorage.getDataOutput();
         private final IPointable inputArg = new VoidPointable();
@@ -78,7 +80,9 @@
         private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
 
         public ABinaryConstructorEvaluator(IScalarEvaluatorFactory copyEvaluatorFactory,
-                IValueParserFactory valueParserFactory, IHyracksTaskContext context) throws HyracksDataException {
+                IValueParserFactory valueParserFactory, IHyracksTaskContext context, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            this.sourceLoc = sourceLoc;
             eval = copyEvaluatorFactory.createScalarEvaluator(context);
             byteArrayParser = valueParserFactory.createValueParser();
         }
@@ -102,15 +106,14 @@
                     byteArrayParser.parse(buffer, 0, buffer.length, out);
                     result.set(resultStorage);
                 } else {
-                    throw new TypeMismatchException(BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
                             ATypeTag.SERIALIZED_BINARY_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                 }
             } catch (IOException e) {
-                throw new InvalidDataFormatException(BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
+                throw new InvalidDataFormatException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
                         ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
             }
         }
     }
 
-    ;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
index 2f981b9..7ddb7e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ABooleanConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
index 5b0d250..0649fc8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
@@ -98,11 +98,11 @@
                                 circleSerde.serialize(aCircle, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index eb9b40f..9849cef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -93,7 +93,7 @@
 
                                 // the string to be parsed should be at least 8 characters: YYYYMMDD
                                 if (stringLength < 8) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                                 }
 
@@ -117,11 +117,12 @@
                                 dateSerde.serialize(aDate, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                                    ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index f202dbe..9b71d4b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -91,7 +91,7 @@
                                 int startOffset = utf8Ptr.getCharStartOffset();
                                 // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
                                 if (stringLength < 14) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                                 }
                                 // +1 if it is negative (-)
@@ -102,7 +102,7 @@
                                 if (serString[startOffset + timeOffset] != 'T') {
                                     timeOffset += 2;
                                     if (serString[startOffset + timeOffset] != 'T') {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                                     }
                                 }
@@ -117,11 +117,11 @@
                                 datetimeSerde.serialize(aDateTime, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
index 0e7cc34..f6c6f09 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
@@ -99,11 +99,11 @@
                                 dayTimeDurationSerde.serialize(aDayTimeDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (Exception e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
index 87df400..f9991db 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return ADoubleConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index df39306..f3d1ba8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -93,11 +93,11 @@
                                 durationSerde.serialize(aDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
index b6f045e..09f2594 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
@@ -100,11 +100,11 @@
                                 floatSerde.serialize(aFloat, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
index 9413cb4..ae91c46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
@@ -112,17 +112,17 @@
                                             && serString[offset + 2] == '6' && offset + 3 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                     }
                                     value = (short) (value - digit);
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -133,11 +133,11 @@
                                 int16Serde.serialize(aInt16, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INT16_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
index 985d966..b4f1ae8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
@@ -111,17 +111,17 @@
                                             && serString[offset + 2] == '2' && offset + 3 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                     }
                                     value -= digit;
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -132,11 +132,11 @@
                                 int32Serde.serialize(aInt32, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
index 61a35b6..a41ec77 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
@@ -44,7 +44,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     public FunctionIdentifier getIdentifier() {
                         return AInt64ConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
index e4a9d32..3132640 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
@@ -112,17 +112,17 @@
                                             && offset + 2 == end) {
                                         break;
                                     } else {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                     }
                                     if (value < limit + digit) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                     }
                                     value = (byte) (value - digit);
                                 }
                                 if (value > 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                                 }
                                 if (value < 0 && positive) {
@@ -133,11 +133,11 @@
                                 int8Serde.serialize(aInt8, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e1) {
-                            throw new InvalidDataFormatException(getIdentifier(), e1,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e1,
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
index 9309591..f103192 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
@@ -90,7 +90,8 @@
 
                         try {
                             if (bytes0[offset0] != bytes1[offset1]) {
-                                throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
+                                throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
+                                        bytes1[offset1]);
                             }
 
                             long intervalStart, intervalEnd;
@@ -110,18 +111,18 @@
                                     intervalEnd = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                     break;
                                 default:
-                                    throw new UnsupportedItemTypeException(getIdentifier(), bytes0[offset0]);
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), bytes0[offset0]);
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, intervalType.serialize());
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index e2a3614..b1a6d0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -114,7 +114,7 @@
                                 int startOffset = utf8Ptr.getCharStartOffset();
                                 intervalStart = ADateParserFactory.parseDatePart(bytes0, startOffset, stringLength);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -139,7 +139,7 @@
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -147,14 +147,14 @@
                             intervalEnd = cal.getChrononInDays(intervalEnd);
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index 3e61405..8d15bff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -116,7 +116,7 @@
                                 if (bytes0[startOffset + timeOffset] != 'T') {
                                     timeOffset += 2;
                                     if (argPtr0.getByteArray()[startOffset + timeOffset] != 'T') {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                     }
                                 }
@@ -125,7 +125,7 @@
                                 intervalStart += ATimeParserFactory.parseTimePart(bytes0, startOffset + timeOffset + 1,
                                         stringLength - timeOffset - 1);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -149,19 +149,19 @@
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index fa402bd..e74e77a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -111,7 +111,7 @@
                                 intervalStart = ATimeParserFactory.parseTimePart(bytes0, utf8Ptr.getCharStartOffset(),
                                         stringLength);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -121,7 +121,7 @@
 
                             if (bytes1[offset1] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
                                 if (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -137,14 +137,14 @@
                                 ADurationParserFactory.parseDuration(bytes1, utf8Ptr.getCharStartOffset(), stringLength,
                                         aDuration, ADurationParseOption.All);
                                 if (aDuration.getMonths() != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart, 0,
                                         aDuration.getMilliseconds(), false);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
@@ -155,14 +155,14 @@
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
index 0ef585c..39bb515 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
@@ -101,11 +101,11 @@
                                 lineSerde.serialize(aLine, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
index 3415f38..4b8b0bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
@@ -96,11 +96,11 @@
                                 point3DSerde.serialize(aPoint3D, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POINT3D_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
index 24045b9..0a7c381 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
@@ -92,11 +92,11 @@
                                 pointSerde.serialize(aPoint, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
index eb139b0..f88713c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
@@ -86,7 +86,7 @@
                                 String s = utf8Ptr.toString();
                                 String[] points = WS.split(s.trim());
                                 if (points.length <= 2) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
                                 out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
@@ -98,11 +98,11 @@
                                 }
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
index ccaf7b8..69cd147 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
@@ -101,17 +101,17 @@
                                 } else if (aPoint[0].getX() < aPoint[1].getX() && aPoint[0].getY() < aPoint[1].getY()) {
                                     aRectangle.setValue(aPoint[0], aPoint[1]);
                                 } else {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                                 }
                                 rectangle2DSerde.serialize(aRectangle, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
index 00b5475..8f0182c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
@@ -45,7 +45,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected FunctionIdentifier getIdentifier() {
                         return AStringConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index c338a35..38d8e05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -93,7 +93,7 @@
 
                                 // the string to be parsed should be at least 6 characters: hhmmss
                                 if (stringLength < 6) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                 }
 
@@ -108,11 +108,11 @@
                                 timeSerde.serialize(aTime, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
index 64b9278..53c36bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
@@ -99,11 +99,12 @@
                                 uuidSerde.serialize(uuid, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_UUID_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
+                                    ATypeTag.SERIALIZED_UUID_TYPE_TAG);
                         }
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
index b95d512..60344fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
@@ -96,11 +96,11 @@
                                 yearMonthDurationSerde.serialize(aYearMonthDuration, out);
                                 result.set(resultStorage);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, tt,
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                         } catch (IOException e) {
-                            throw new InvalidDataFormatException(getIdentifier(), e,
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
index 9b9bf71..b78eb54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -52,12 +53,14 @@
     protected static final byte[] FALSE = UTF8StringUtil.writeStringToBytes("false");
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
 
-    protected AbstractBooleanConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractBooleanConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         inputArg = new VoidPointable();
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
@@ -70,7 +73,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
         }
     }
 
@@ -87,10 +90,10 @@
             } else if (UTF8_BINARY_CMP.compare(bytes, startOffset + 1, len - 1, FALSE, 0, FALSE.length) == 0) {
                 setBoolean(result, false);
             } else {
-                throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
index 456735f..5354fc0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -46,14 +47,16 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final IPointable inputArg;
     protected final AMutableDouble aDouble;
     protected final UTF8StringPointable utf8Ptr;
 
-    protected AbstractDoubleConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractDoubleConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
         inputArg = new VoidPointable();
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
         }
     }
 
@@ -87,12 +90,12 @@
                 handleUparseableString(result);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
     protected void handleUparseableString(IPointable result) throws HyracksDataException {
-        throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+        throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
     }
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
index 2f9470d..a49c1ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -46,14 +47,16 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final AMutableInt64 aInt64;
     protected final UTF8StringPointable utf8Ptr;
 
-    protected AbstractInt64ConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractInt64ConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
         this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         inputArg = new VoidPointable();
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
         }
     }
 
@@ -88,12 +91,12 @@
                 handleUnparseableString(result);
             }
         } else {
-            throw new TypeMismatchException(getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
     protected void handleUnparseableString(IPointable result) throws HyracksDataException {
-        throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+        throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_INT64_TYPE_TAG);
     }
 
     protected abstract FunctionIdentifier getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
index 52d3bf6..9f2dc2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -46,17 +47,19 @@
 public abstract class AbstractStringConstructorEvaluator implements IScalarEvaluator {
 
     protected final IScalarEvaluator inputEval;
+    protected final SourceLocation sourceLoc;
     protected final IPointable inputArg;
     protected final ArrayBackedValueStorage resultStorage;
     protected final DataOutput out;
     protected final UTF8StringBuilder builder;
     protected final GrowableArray baaos;
 
-    protected AbstractStringConstructorEvaluator(IScalarEvaluator inputEval) {
+    protected AbstractStringConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
+        this.inputEval = inputEval;
+        this.sourceLoc = sourceLoc;
         resultStorage = new ArrayBackedValueStorage();
         out = resultStorage.getDataOutput();
         inputArg = new VoidPointable();
-        this.inputEval = inputEval;
         builder = new UTF8StringBuilder();
         baaos = new GrowableArray();
     }
@@ -68,7 +71,7 @@
             resultStorage.reset();
             evaluateImpl(result);
         } catch (IOException e) {
-            throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
     }
 
@@ -156,7 +159,7 @@
                 case MULTISET:
                 case UUID:
                 default:
-                    throw new UnsupportedTypeException(getIdentifier(), serString[offset]);
+                    throw new UnsupportedTypeException(sourceLoc, getIdentifier(), serString[offset]);
             }
             builder.finish();
             out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayAddRemoveEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayAddRemoveEval.java
new file mode 100755
index 0000000..1ed467c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayAddRemoveEval.java
@@ -0,0 +1,221 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractArrayAddRemoveEval implements IScalarEvaluator {
+    protected static final int RETURN_MISSING = -1;
+    protected static final int RETURN_NULL = -2;
+
+    private final IAType[] argTypes;
+    private final ArrayBackedValueStorage storage;
+    private final IPointable listArg;
+    private final IPointable tempList;
+    private final IPointable tempItem;
+    private final IPointable[] valuesArgs;
+    private final IScalarEvaluator listArgEval;
+    private final IScalarEvaluator[] valuesEval;
+    private final SourceLocation sourceLocation;
+    private final CastTypeEvaluator caster;
+    private final ListAccessor listAccessor;
+    private final int listOffset;
+    private final int valuesOffset;
+    private final boolean comparesValues;
+    private final boolean makeOpen;
+    private final boolean acceptNullValues;
+    private IAsterixListBuilder orderedListBuilder;
+    private IAsterixListBuilder unorderedListBuilder;
+
+    public AbstractArrayAddRemoveEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, int listOffset,
+            int valuesOffset, int numValues, IAType[] argTypes, boolean comparesValues, SourceLocation sourceLocation,
+            boolean makeOpen, boolean acceptNullValues) throws HyracksDataException {
+        this.listOffset = listOffset;
+        this.valuesOffset = valuesOffset;
+        this.argTypes = argTypes;
+        this.comparesValues = comparesValues;
+        this.sourceLocation = sourceLocation;
+        this.makeOpen = makeOpen;
+        this.acceptNullValues = acceptNullValues;
+        orderedListBuilder = null;
+        unorderedListBuilder = null;
+        listAccessor = new ListAccessor();
+        caster = new CastTypeEvaluator();
+        storage = new ArrayBackedValueStorage();
+        listArg = new VoidPointable();
+        tempList = new VoidPointable();
+        tempItem = new VoidPointable();
+        listArgEval = args[listOffset].createScalarEvaluator(ctx);
+        valuesArgs = new IPointable[numValues];
+        valuesEval = new IScalarEvaluator[numValues];
+        for (int i = 0; i < numValues; i++) {
+            valuesArgs[i] = new VoidPointable();
+            valuesEval[i] = args[i + valuesOffset].createScalarEvaluator(ctx);
+        }
+    }
+
+    /**
+     * @param listType the type of the list, ordered or unordered.
+     * @param listArg the list into which to insert the items at the calculated returned position
+     * @param tuple the tuple that contains the arguments including position argument
+     * @return -1 if position value is missing, -2 if null, otherwise should return the adjusted position value, >= 0
+     */
+    protected abstract int getPosition(IFrameTupleReference tuple, IPointable listArg, ATypeTag listType)
+            throws HyracksDataException;
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        // get the list argument, 1st or last argument, make sure it's a list
+        listArgEval.evaluate(tuple, tempList);
+        ATypeTag listArgTag = ATYPETAGDESERIALIZER.deserialize(tempList.getByteArray()[tempList.getStartOffset()]);
+
+        // evaluate the position argument if provided by some functions
+        int adjustedPosition = getPosition(tuple, tempList, listArgTag);
+
+        if (listArgTag == ATypeTag.MISSING || adjustedPosition == RETURN_MISSING) {
+            PointableHelper.setMissing(result);
+            return;
+        }
+
+        boolean returnNull = false;
+        if (!listArgTag.isListType() || adjustedPosition == RETURN_NULL) {
+            returnNull = true;
+        }
+
+        // evaluate values to be added/removed
+        ATypeTag valueTag;
+        IAType defaultOpenType;
+        boolean encounteredNonPrimitive = false;
+        try {
+            for (int i = 0; i < valuesEval.length; i++) {
+                // cast val to open if needed. don't cast if function will return null anyway, e.g. list arg not list
+                defaultOpenType = DefaultOpenFieldType.getDefaultOpenFieldType(argTypes[i + valuesOffset].getTypeTag());
+                if (defaultOpenType != null && !returnNull) {
+                    caster.resetAndAllocate(defaultOpenType, argTypes[i + valuesOffset], valuesEval[i]);
+                    caster.evaluate(tuple, valuesArgs[i]);
+                } else {
+                    valuesEval[i].evaluate(tuple, valuesArgs[i]);
+                }
+                valueTag =
+                        ATYPETAGDESERIALIZER.deserialize(valuesArgs[i].getByteArray()[valuesArgs[i].getStartOffset()]);
+                // for now, we don't support deep equality of object/lists. Throw an error if value is of these types
+                if (comparesValues && valueTag.isDerivedType()) {
+                    encounteredNonPrimitive = true;
+                }
+                if (valueTag == ATypeTag.MISSING) {
+                    PointableHelper.setMissing(result);
+                    return;
+                }
+                if (!acceptNullValues && valueTag == ATypeTag.NULL) {
+                    returnNull = true;
+                }
+            }
+
+            if (returnNull) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            if (encounteredNonPrimitive) {
+                throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLocation);
+            }
+            // all arguments are valid
+            AbstractCollectionType listType;
+            IAsterixListBuilder listBuilder;
+            // create the new list to be returned. cast the input list and make it open if required
+            if (listArgTag == ATypeTag.ARRAY) {
+                if (orderedListBuilder == null) {
+                    orderedListBuilder = new OrderedListBuilder();
+                }
+                listBuilder = orderedListBuilder;
+                if (makeOpen || argTypes[listOffset].getTypeTag() != ATypeTag.ARRAY) {
+                    listType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                    caster.resetAndAllocate(listType, argTypes[listOffset], listArgEval);
+                    caster.cast(tempList, listArg);
+                } else {
+                    listType = (AbstractCollectionType) argTypes[listOffset];
+                    listArg.set(tempList);
+                }
+            } else {
+                if (unorderedListBuilder == null) {
+                    unorderedListBuilder = new UnorderedListBuilder();
+                }
+                listBuilder = unorderedListBuilder;
+                if (makeOpen || argTypes[listOffset].getTypeTag() != ATypeTag.MULTISET) {
+                    listType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                    caster.resetAndAllocate(listType, argTypes[listOffset], listArgEval);
+                    caster.cast(tempList, listArg);
+                } else {
+                    listType = (AbstractCollectionType) argTypes[listOffset];
+                    listArg.set(tempList);
+                }
+            }
+
+            listBuilder.reset(listType);
+            listAccessor.reset(listArg.getByteArray(), listArg.getStartOffset());
+            processList(listAccessor, listBuilder, valuesArgs, adjustedPosition);
+            storage.reset();
+            listBuilder.write(storage.getDataOutput(), true);
+            result.set(storage);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        } finally {
+            caster.deallocatePointables();
+        }
+    }
+
+    protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder, IPointable[] values,
+            int position) throws IOException {
+        int i;
+        for (i = 0; i < position; i++) {
+            listAccessor.getOrWriteItem(i, tempItem, storage);
+            listBuilder.addItem(tempItem);
+        }
+        // insert the values arguments
+        for (int j = 0; j < values.length; j++) {
+            listBuilder.addItem(values[j]);
+        }
+        for (; i < listAccessor.size(); i++) {
+            listAccessor.getOrWriteItem(i, tempItem, storage);
+            listBuilder.addItem(tempItem);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessArraysEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessArraysEval.java
new file mode 100755
index 0000000..1cf75bc
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessArraysEval.java
@@ -0,0 +1,186 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.AbvsBuilderFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractArrayProcessArraysEval implements IScalarEvaluator {
+    private final ArrayBackedValueStorage finalResult;
+    private final ListAccessor listAccessor;
+    private final IPointable tempList;
+    private final IPointable[] listsArgs;
+    private final IScalarEvaluator[] listsEval;
+    private final SourceLocation sourceLocation;
+    private final boolean isComparingElements;
+    private final PointableAllocator pointableAllocator;
+    private final IObjectPool<IMutableValueStorage, ATypeTag> storageAllocator;
+    private final IAType[] argTypes;
+    private final CastTypeEvaluator caster;
+    private OrderedListBuilder orderedListBuilder;
+    private UnorderedListBuilder unorderedListBuilder;
+
+    public AbstractArrayProcessArraysEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx,
+            boolean isComparingElements, SourceLocation sourceLoc, IAType[] argTypes) throws HyracksDataException {
+        orderedListBuilder = null;
+        unorderedListBuilder = null;
+        pointableAllocator = new PointableAllocator();
+        storageAllocator = new ListObjectPool<>(new AbvsBuilderFactory());
+        finalResult = new ArrayBackedValueStorage();
+        listAccessor = new ListAccessor();
+        caster = new CastTypeEvaluator();
+        tempList = new VoidPointable();
+        listsArgs = new IPointable[args.length];
+        listsEval = new IScalarEvaluator[args.length];
+        for (int i = 0; i < args.length; i++) {
+            listsArgs[i] = new VoidPointable();
+            listsEval[i] = args[i].createScalarEvaluator(ctx);
+        }
+        sourceLocation = sourceLoc;
+        this.isComparingElements = isComparingElements;
+        this.argTypes = argTypes;
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        byte listArgType;
+        boolean returnNull = false;
+        AbstractCollectionType outList = null;
+        ATypeTag listTag;
+        try {
+            for (int i = 0; i < listsEval.length; i++) {
+                listsEval[i].evaluate(tuple, tempList);
+                if (!returnNull) {
+                    listArgType = tempList.getByteArray()[tempList.getStartOffset()];
+                    listTag = ATYPETAGDESERIALIZER.deserialize(listArgType);
+                    if (!listTag.isListType()) {
+                        returnNull = true;
+                    } else if (outList != null && outList.getTypeTag() != listTag) {
+                        throw new RuntimeDataException(ErrorCode.DIFFERENT_LIST_TYPE_ARGS, sourceLocation);
+                    } else {
+                        if (outList == null) {
+                            outList = (AbstractCollectionType) DefaultOpenFieldType.getDefaultOpenFieldType(listTag);
+                        }
+
+                        caster.resetAndAllocate(outList, argTypes[i], listsEval[i]);
+                        caster.cast(tempList, listsArgs[i]);
+                    }
+                }
+            }
+
+            if (returnNull) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            IAsterixListBuilder listBuilder;
+            if (outList.getTypeTag() == ATypeTag.ARRAY) {
+                if (orderedListBuilder == null) {
+                    orderedListBuilder = new OrderedListBuilder();
+                }
+                listBuilder = orderedListBuilder;
+            } else {
+                if (unorderedListBuilder == null) {
+                    unorderedListBuilder = new UnorderedListBuilder();
+                }
+                listBuilder = unorderedListBuilder;
+            }
+
+            listBuilder.reset(outList);
+            init();
+            processLists(listsArgs, listBuilder);
+            finish(listBuilder);
+
+            finalResult.reset();
+            listBuilder.write(finalResult.getDataOutput(), true);
+            result.set(finalResult);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        } finally {
+            release();
+            storageAllocator.reset();
+            pointableAllocator.reset();
+            caster.deallocatePointables();
+        }
+    }
+
+    private void processLists(IPointable[] listsArgs, IAsterixListBuilder listBuilder) throws IOException {
+        boolean itemInStorage;
+        boolean isUsingItem;
+        IPointable item = pointableAllocator.allocateEmpty();
+        ArrayBackedValueStorage storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+        storage.reset();
+
+        // process each list one by one
+        for (int listIndex = 0; listIndex < listsArgs.length; listIndex++) {
+            listAccessor.reset(listsArgs[listIndex].getByteArray(), listsArgs[listIndex].getStartOffset());
+            // process the items of the current list
+            for (int j = 0; j < listAccessor.size(); j++) {
+                itemInStorage = listAccessor.getOrWriteItem(j, item, storage);
+                if (ATYPETAGDESERIALIZER.deserialize(item.getByteArray()[item.getStartOffset()]).isDerivedType()
+                        && isComparingElements) {
+                    throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLocation);
+                }
+                isUsingItem = processItem(item, listIndex, listBuilder);
+                if (isUsingItem) {
+                    item = pointableAllocator.allocateEmpty();
+                    if (itemInStorage) {
+                        storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                        storage.reset();
+                    }
+                }
+            }
+        }
+    }
+
+    protected abstract void init();
+
+    protected abstract void finish(IAsterixListBuilder listBuilder) throws HyracksDataException;
+
+    protected abstract void release();
+
+    protected abstract boolean processItem(IPointable item, int listIndex, IAsterixListBuilder listBuilder)
+            throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessEval.java
new file mode 100755
index 0000000..9cdd32c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayProcessEval.java
@@ -0,0 +1,136 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.AbvsBuilderFactory;
+import org.apache.asterix.builders.ArrayListFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractArrayProcessEval implements IScalarEvaluator {
+    private final AOrderedListType orderedListType;
+    private final AUnorderedListType unorderedListType;
+    private final ArrayBackedValueStorage storage;
+    private final IScalarEvaluator listArgEval;
+    private final ListAccessor listAccessor;
+    private final IPointable listArg;
+    private IAType inputListType;
+    private IAsterixListBuilder orderedListBuilder;
+    private IAsterixListBuilder unorderedListBuilder;
+
+    protected final PointableAllocator pointableAllocator;
+    protected final IObjectPool<IMutableValueStorage, ATypeTag> storageAllocator;
+    protected final IObjectPool<List<IPointable>, ATypeTag> arrayListAllocator;
+
+    public AbstractArrayProcessEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, IAType inputListType)
+            throws HyracksDataException {
+        orderedListBuilder = null;
+        unorderedListBuilder = null;
+        orderedListType = new AOrderedListType(BuiltinType.ANY, null);
+        unorderedListType = new AUnorderedListType(BuiltinType.ANY, null);
+        storage = new ArrayBackedValueStorage();
+        listArg = new VoidPointable();
+        pointableAllocator = new PointableAllocator();
+        storageAllocator = new ListObjectPool<>(new AbvsBuilderFactory());
+        arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
+        listArgEval = args[0].createScalarEvaluator(ctx);
+        listAccessor = new ListAccessor();
+        this.inputListType = inputListType;
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        // get the list argument and make sure it's a list
+        listArgEval.evaluate(tuple, listArg);
+        byte listArgType = listArg.getByteArray()[listArg.getStartOffset()];
+
+        // create the new list with the same type as the input list
+        IAsterixListBuilder listBuilder;
+        if (listArgType == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+            if (orderedListBuilder == null) {
+                orderedListBuilder = new OrderedListBuilder();
+            }
+            listBuilder = orderedListBuilder;
+        } else if (listArgType == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+            if (unorderedListBuilder == null) {
+                unorderedListBuilder = new UnorderedListBuilder();
+            }
+            listBuilder = unorderedListBuilder;
+        } else {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        listAccessor.reset(listArg.getByteArray(), listArg.getStartOffset());
+        AbstractCollectionType outputListType;
+        if (!inputListType.getTypeTag().isListType()) {
+            ATypeTag itemType = listAccessor.getItemType();
+            if (listAccessor.getListType() == ATypeTag.ARRAY) {
+                // TODO(ali): check the case when the item type from the runtime is a derived type
+                orderedListType.setItemType(TypeTagUtil.getBuiltinTypeByTag(itemType));
+                outputListType = orderedListType;
+            } else {
+                unorderedListType.setItemType(TypeTagUtil.getBuiltinTypeByTag(itemType));
+                outputListType = unorderedListType;
+            }
+        } else {
+            outputListType = (AbstractCollectionType) inputListType;
+        }
+
+        listBuilder.reset(outputListType);
+        try {
+            processList(listAccessor, listBuilder);
+            storage.reset();
+            listBuilder.write(storage.getDataOutput(), true);
+            result.set(storage);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        } finally {
+            pointableAllocator.reset();
+            storageAllocator.reset();
+            arrayListAllocator.reset();
+        }
+    }
+
+    protected abstract void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder) throws IOException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java
new file mode 100755
index 0000000..8b5ba72
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java
@@ -0,0 +1,111 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractArraySearchEval implements IScalarEvaluator {
+    private final IPointable listArg;
+    private final IPointable searchedValueArg;
+    private final IPointable tempVal;
+    private final IScalarEvaluator listEval;
+    private final IScalarEvaluator searchedValueEval;
+    private final IBinaryComparator comp;
+    private final ListAccessor listAccessor;
+    private final SourceLocation sourceLocation;
+    private final AMutableInt32 intValue;
+    protected final ArrayBackedValueStorage storage;
+
+    public AbstractArraySearchEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        storage = new ArrayBackedValueStorage();
+        listArg = new VoidPointable();
+        searchedValueArg = new VoidPointable();
+        tempVal = new VoidPointable();
+        listEval = args[0].createScalarEvaluator(ctx);
+        searchedValueEval = args[1].createScalarEvaluator(ctx);
+        comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        listAccessor = new ListAccessor();
+        intValue = new AMutableInt32(-1);
+        sourceLocation = sourceLoc;
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        // 1st arg: list
+        listEval.evaluate(tuple, listArg);
+        byte[] listBytes = listArg.getByteArray();
+        int listOffset = listArg.getStartOffset();
+
+        // 2nd arg: value to search for
+        searchedValueEval.evaluate(tuple, searchedValueArg);
+        byte[] valueBytes = searchedValueArg.getByteArray();
+        int valueOffset = searchedValueArg.getStartOffset();
+        int valueLength = searchedValueArg.getLength();
+
+        // for now, we don't support deep equality of object/lists. Throw an error if the value is of these types
+        if (ATYPETAGDESERIALIZER.deserialize(valueBytes[valueOffset]).isDerivedType()) {
+            throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLocation);
+        }
+
+        if (!ATYPETAGDESERIALIZER.deserialize(listBytes[listOffset]).isListType()) {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        // initialize variables; -1 = value not found
+        intValue.setValue(-1);
+        listAccessor.reset(listBytes, listOffset);
+        int numItems = listAccessor.size();
+
+        try {
+            for (int i = 0; i < numItems; i++) {
+                listAccessor.getOrWriteItem(i, tempVal, storage);
+                if (comp.compare(tempVal.getByteArray(), tempVal.getStartOffset(), tempVal.getLength(), valueBytes,
+                        valueOffset, valueLength) == 0) {
+                    intValue.setValue(i);
+                    break;
+                }
+            }
+            processResult(intValue, result);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    protected abstract void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
index 9c38fbc..07dae0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringBoolEval.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -39,8 +40,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
     public AbstractBinaryStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
index af0a819..2181aa6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -53,9 +54,12 @@
 
     // Function ID, for error reporting.
     private final FunctionIdentifier funcID;
+    private final SourceLocation sourceLoc;
 
     public AbstractBinaryStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        this.sourceLoc = sourceLoc;
         this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
         this.evalRight = evalRightFactory.createScalarEvaluator(context);
         this.funcID = funcID;
@@ -80,10 +84,10 @@
 
         // Type check.
         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[offset0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[offset0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
 
         // Sets StringUTF8Pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
index ebe1273..a562a05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringIntEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -41,8 +42,9 @@
     private AMutableInt32 resultValue = new AMutableInt32(0);
 
     public AbstractBinaryStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
index 7cdf673..04a0349 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractBinaryStringStringEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.GrowableArray;
@@ -39,8 +40,9 @@
     protected final UTF8StringBuilder resultBuilder = new UTF8StringBuilder();
 
     public AbstractBinaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, evalLeftFactory, evalRightFactory, funcID);
+            IScalarEvaluatorFactory evalRightFactory, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        super(context, evalLeftFactory, evalRightFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
index 936415d..fac7847 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -43,6 +43,7 @@
 import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.AMutableInt8;
 import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
@@ -53,7 +54,6 @@
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.runtime.exceptions.UnderflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -64,39 +64,56 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-@SuppressWarnings("serial")
 public abstract class AbstractNumericArithmeticEval extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 527445160961348706L;
 
-    abstract protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException;
+    /**
+     * abstract method for arithmetic operation between two integer values
+     *
+     * @param lhs first operand
+     * @param rhs second operand
+     * @param result result holder
+     * @return {@code false} if the result is {@code NULL}, otherwise {@code true}
+     */
+    abstract protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException;
 
-    abstract protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException;
+    /**
+     * abstract method for arithmetic operation between two floating point values
+     *
+     * @param lhs first operand
+     * @param rhs second operand
+     * @param result result holder
+     * @return {@code false} if the result is {@code NULL}, otherwise {@code true}
+     */
+    abstract protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result)
+            throws HyracksDataException;
 
     /**
      * abstract method for arithmetic operation between a time instance (date/time/datetime)
      * and a duration (duration/year-month-duration/day-time-duration)
      *
-     * @param chronon
-     * @param yearMonth
-     * @param dayTime
-     * @return
-     * @throws HyracksDataException
+     * @param chronon first operand
+     * @param yearMonth year-month component of the second operand
+     * @param dayTime day-time component of the second operand
+     * @param result result holder
+     * @return {@code false} if the result is {@code NULL}, otherwise {@code true}
      */
-    abstract protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime,
-            boolean isTimeOnly) throws HyracksDataException;
+    abstract protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime,
+            boolean isTimeOnly, AMutableInt64 result) throws HyracksDataException;
 
     /**
      * abstract method for arithmetic operation between two time instances (date/time/datetime)
      *
-     * @param chronon0
-     * @param chronon1
-     * @return
-     * @throws HyracksDataException
+     * @param chronon0 first operand
+     * @param chronon1 second operand
+     * @param result result holder
+     * @return {@code false} if the result is {@code NULL}, otherwise {@code true}
      */
-    abstract protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException;
+    abstract protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
+            throws HyracksDataException;
 
     @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
-            throws AlgebricksException {
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
@@ -104,80 +121,105 @@
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
                 return new IScalarEvaluator() {
-                    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-                    private DataOutput out = resultStorage.getDataOutput();
-                    private IPointable argPtr0 = new VoidPointable();
-                    private IPointable argPtr1 = new VoidPointable();
-                    private IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
-                    private IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
-                    private double[] operandsFloating = new double[args.length];
-                    private long[] operandsInteger = new long[args.length];
-                    private int resultType;
-                    static protected final int typeInt8 = 1;
-                    static protected final int typeInt16 = 2;
-                    static protected final int typeInt32 = 3;
-                    static protected final int typeInt64 = 4;
-                    static protected final int typeFloat = 5;
-                    static protected final int typeDouble = 6;
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private final DataOutput out = resultStorage.getDataOutput();
+                    private final IPointable argPtr0 = new VoidPointable();
+                    private final IPointable argPtr1 = new VoidPointable();
+                    private final IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
+                    private final double[] operandsFloating = new double[args.length];
+                    private final long[] operandsInteger = new long[args.length];
 
-                    protected AMutableFloat aFloat = new AMutableFloat(0);
-                    protected AMutableDouble aDouble = new AMutableDouble(0);
-                    protected AMutableInt64 aInt64 = new AMutableInt64(0);
-                    protected AMutableInt32 aInt32 = new AMutableInt32(0);
-                    protected AMutableInt16 aInt16 = new AMutableInt16((short) 0);
-                    protected AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+                    private final AMutableFloat aFloat = new AMutableFloat(0);
+                    private final AMutableDouble aDouble = new AMutableDouble(0);
+                    private final AMutableInt64 aInt64 = new AMutableInt64(0);
+                    private final AMutableInt32 aInt32 = new AMutableInt32(0);
+                    private final AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+                    private final AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
 
-                    protected AMutableDuration aDuration = new AMutableDuration(0, 0);
-                    protected AMutableDate aDate = new AMutableDate(0);
-                    protected AMutableTime aTime = new AMutableTime(0);
-                    protected AMutableDateTime aDatetime = new AMutableDateTime(0);
+                    private final AMutableDuration aDuration = new AMutableDuration(0, 0);
+                    private final AMutableDate aDate = new AMutableDate(0);
+                    private final AMutableTime aTime = new AMutableTime(0);
+                    private final AMutableDateTime aDatetime = new AMutableDateTime(0);
 
-                    private ATypeTag typeTag;
                     @SuppressWarnings("rawtypes")
-                    private ISerializerDeserializer serde;
+                    private final ISerializerDeserializer int8Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int16Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int32Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer int64Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer floatSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer doubleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer dateSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer timeSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer dateTimeSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer durationSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
+                    @SuppressWarnings("rawtypes")
+                    private final ISerializerDeserializer nullSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-                    @SuppressWarnings("unchecked")
                     @Override
+                    @SuppressWarnings("unchecked")
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-                        resultStorage.reset();
-                        resultType = 0;
-                        int currentType;
                         evalLeft.evaluate(tuple, argPtr0);
                         evalRight.evaluate(tuple, argPtr1);
 
-                        for (int i = 0; i < args.length; i++) {
+                        resultStorage.reset();
+
+                        ATypeTag argTypeMax = null;
+
+                        for (int i = 0; i < 2; i++) {
                             IPointable argPtr = i == 0 ? argPtr0 : argPtr1;
                             byte[] bytes = argPtr.getByteArray();
                             int offset = argPtr.getStartOffset();
 
-                            typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+                            ATypeTag currentType;
+                            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
                             switch (typeTag) {
                                 case TINYINT:
-                                    currentType = typeInt8;
+                                    currentType = ATypeTag.TINYINT;
                                     operandsInteger[i] = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case SMALLINT:
-                                    currentType = typeInt16;
+                                    currentType = ATypeTag.SMALLINT;
                                     operandsInteger[i] = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case INTEGER:
-                                    currentType = typeInt32;
+                                    currentType = ATypeTag.INTEGER;
                                     operandsInteger[i] = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case BIGINT:
-                                    currentType = typeInt64;
+                                    currentType = ATypeTag.BIGINT;
                                     operandsInteger[i] = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
                                     operandsFloating[i] = operandsInteger[i];
                                     break;
                                 case FLOAT:
-                                    currentType = typeFloat;
+                                    currentType = ATypeTag.FLOAT;
                                     operandsFloating[i] = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
                                     break;
                                 case DOUBLE:
-                                    currentType = typeDouble;
+                                    currentType = ATypeTag.DOUBLE;
                                     operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
                                     break;
                                 case DATE:
@@ -186,11 +228,11 @@
                                 case DURATION:
                                 case YEARMONTHDURATION:
                                 case DAYTIMEDURATION:
-                                    evaluateTemporalArthmeticOperation(typeTag);
+                                    evaluateTemporalArithmeticOperation(typeTag);
                                     result.set(resultStorage);
                                     return;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), i, bytes[offset],
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                             ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
@@ -201,85 +243,96 @@
                                             ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                             }
 
-                            if (resultType < currentType) {
-                                resultType = currentType;
+                            if (i == 0 || currentType.ordinal() > argTypeMax.ordinal()) {
+                                argTypeMax = currentType;
                             }
                         }
 
+                        ATypeTag resultType = getNumericResultType(argTypeMax);
+
                         long lres;
                         double dres;
                         switch (resultType) {
-                            case typeInt8:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT8);
-                                lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
-                                if (lres > Byte.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                            case TINYINT:
+                                if (evaluateInteger(operandsInteger[0], operandsInteger[1], aInt64)) {
+                                    lres = aInt64.getLongValue();
+                                    if (lres > Byte.MAX_VALUE) {
+                                        throw new OverflowException(sourceLoc, getIdentifier());
+                                    }
+                                    if (lres < Byte.MIN_VALUE) {
+                                        throw new UnderflowException(sourceLoc, getIdentifier());
+                                    }
+                                    aInt8.setValue((byte) lres);
+                                    int8Serde.serialize(aInt8, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
                                 }
-                                if (lres < Byte.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
-                                }
-                                aInt8.setValue((byte) lres);
-                                serde.serialize(aInt8, out);
                                 break;
-                            case typeInt16:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT16);
-                                lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
-                                if (lres > Short.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                            case SMALLINT:
+                                if (evaluateInteger(operandsInteger[0], operandsInteger[1], aInt64)) {
+                                    lres = aInt64.getLongValue();
+                                    if (lres > Short.MAX_VALUE) {
+                                        throw new OverflowException(sourceLoc, getIdentifier());
+                                    }
+                                    if (lres < Short.MIN_VALUE) {
+                                        throw new UnderflowException(sourceLoc, getIdentifier());
+                                    }
+                                    aInt16.setValue((short) lres);
+                                    int16Serde.serialize(aInt16, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
                                 }
-                                if (lres < Short.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
-                                }
-                                aInt16.setValue((short) lres);
-                                serde.serialize(aInt16, out);
                                 break;
-                            case typeInt32:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT32);
-                                lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
-                                if (lres > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                            case INTEGER:
+                                if (evaluateInteger(operandsInteger[0], operandsInteger[1], aInt64)) {
+                                    lres = aInt64.getLongValue();
+                                    if (lres > Integer.MAX_VALUE) {
+                                        throw new OverflowException(sourceLoc, getIdentifier());
+                                    }
+                                    if (lres < Integer.MIN_VALUE) {
+                                        throw new UnderflowException(sourceLoc, getIdentifier());
+                                    }
+                                    aInt32.setValue((int) lres);
+                                    int32Serde.serialize(aInt32, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
                                 }
-                                if (lres < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
-                                }
-                                aInt32.setValue((int) lres);
-                                serde.serialize(aInt32, out);
                                 break;
-                            case typeInt64:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AINT64);
-                                lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
-                                aInt64.setValue(lres);
-                                serde.serialize(aInt64, out);
-                                break;
-                            case typeFloat:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.AFLOAT);
-                                dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
-                                if (dres > Float.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                            case BIGINT:
+                                if (evaluateInteger(operandsInteger[0], operandsInteger[1], aInt64)) {
+                                    int64Serde.serialize(aInt64, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
                                 }
-                                if (dres < -Float.MAX_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
-                                }
-                                aFloat.setValue((float) dres);
-                                serde.serialize(aFloat, out);
                                 break;
-                            case typeDouble:
-                                serde = SerializerDeserializerProvider.INSTANCE
-                                        .getSerializerDeserializer(BuiltinType.ADOUBLE);
-                                aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
-                                serde.serialize(aDouble, out);
+                            case FLOAT:
+                                if (evaluateDouble(operandsFloating[0], operandsFloating[1], aDouble)) {
+                                    dres = aDouble.getDoubleValue();
+                                    if (dres > Float.MAX_VALUE) {
+                                        throw new OverflowException(sourceLoc, getIdentifier());
+                                    }
+                                    if (dres < -Float.MAX_VALUE) {
+                                        throw new UnderflowException(sourceLoc, getIdentifier());
+                                    }
+                                    aFloat.setValue((float) dres);
+                                    floatSerde.serialize(aFloat, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
+                                }
+                                break;
+                            case DOUBLE:
+                                if (evaluateDouble(operandsFloating[0], operandsFloating[1], aDouble)) {
+                                    doubleSerde.serialize(aDouble, out);
+                                } else {
+                                    nullSerde.serialize(ANull.NULL, out);
+                                }
                                 break;
                         }
                         result.set(resultStorage);
                     }
 
                     @SuppressWarnings("unchecked")
-                    private void evaluateTemporalArthmeticOperation(ATypeTag leftType) throws HyracksDataException {
+                    private void evaluateTemporalArithmeticOperation(ATypeTag leftType) throws HyracksDataException {
                         byte[] bytes1 = argPtr1.getByteArray();
                         int offset1 = argPtr1.getStartOffset();
                         ATypeTag rightType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
@@ -287,13 +340,9 @@
                         int offset0 = argPtr0.getStartOffset();
 
                         if (rightType == leftType) {
-
-                            serde = SerializerDeserializerProvider.INSTANCE
-                                    .getSerializerDeserializer(BuiltinType.ADURATION);
-
                             long leftChronon = 0, rightChronon = 0, dayTime = 0;
-
                             int yearMonth = 0;
+                            boolean yearMonthIsNull = false, dayTimeIsNull = false;
 
                             switch (leftType) {
                                 case DATE:
@@ -301,7 +350,6 @@
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
                                     rightChronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
-
                                     break;
                                 case TIME:
                                     leftChronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -312,9 +360,14 @@
                                     rightChronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                     break;
                                 case YEARMONTHDURATION:
-                                    yearMonth = (int) evaluateTimeInstanceArithmetic(
+                                    if (evaluateTimeInstanceArithmetic(
                                             AYearMonthDurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1),
-                                            AYearMonthDurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1));
+                                            AYearMonthDurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1),
+                                            aInt64)) {
+                                        yearMonth = (int) aInt64.getLongValue();
+                                    } else {
+                                        yearMonthIsNull = true;
+                                    }
                                     break;
                                 case DAYTIMEDURATION:
                                     leftChronon =
@@ -323,26 +376,33 @@
                                             ADayTimeDurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1);
                                     break;
                                 default:
-                                    throw new UnsupportedTypeException(getIdentifier(), bytes1[offset1]);
+                                    throw new UnsupportedTypeException(sourceLoc, getIdentifier(), bytes1[offset1]);
                             }
 
-                            dayTime = evaluateTimeInstanceArithmetic(leftChronon, rightChronon);
+                            if (evaluateTimeInstanceArithmetic(leftChronon, rightChronon, aInt64)) {
+                                dayTime = aInt64.getLongValue();
+                            } else {
+                                dayTimeIsNull = true;
+                            }
 
-                            aDuration.setValue(yearMonth, dayTime);
-
-                            serde.serialize(aDuration, out);
+                            if (yearMonthIsNull || dayTimeIsNull) {
+                                nullSerde.serialize(ANull.NULL, out);
+                            } else {
+                                aDuration.setValue(yearMonth, dayTime);
+                                durationSerde.serialize(aDuration, out);
+                            }
 
                         } else {
                             long chronon = 0, dayTime = 0;
                             int yearMonth = 0;
                             ATypeTag resultType = null;
+                            ISerializerDeserializer serde = null;
 
                             boolean isTimeOnly = false;
 
                             switch (leftType) {
                                 case TIME:
-                                    serde = SerializerDeserializerProvider.INSTANCE
-                                            .getSerializerDeserializer(BuiltinType.ATIME);
+                                    serde = timeSerde;
                                     chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                     isTimeOnly = true;
                                     resultType = ATypeTag.TIME;
@@ -357,20 +417,18 @@
                                                     ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1);
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case DATE:
-                                    serde = SerializerDeserializerProvider.INSTANCE
-                                            .getSerializerDeserializer(BuiltinType.ADATE);
+                                    serde = dateSerde;
                                     resultType = ATypeTag.DATE;
                                     chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
                                             * GregorianCalendarSystem.CHRONON_OF_DAY;
                                 case DATETIME:
                                     if (leftType == ATypeTag.DATETIME) {
-                                        serde = SerializerDeserializerProvider.INSTANCE
-                                                .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                        serde = dateTimeSerde;
                                         resultType = ATypeTag.DATETIME;
                                         chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                     }
@@ -389,8 +447,8 @@
                                                     offset1 + 1);
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case YEARMONTHDURATION:
@@ -398,21 +456,19 @@
                                             AYearMonthDurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1);
                                     switch (rightType) {
                                         case DATETIME:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                            serde = dateTimeSerde;
                                             resultType = ATypeTag.DATETIME;
                                             chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                             break;
                                         case DATE:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATE);
+                                            serde = dateSerde;
                                             resultType = ATypeTag.DATE;
                                             chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                                             break;
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 case DURATION:
@@ -425,60 +481,60 @@
                                     }
                                     switch (rightType) {
                                         case DATETIME:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATETIME);
+                                            serde = dateTimeSerde;
                                             resultType = ATypeTag.DATETIME;
                                             chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                             break;
                                         case DATE:
-                                            serde = SerializerDeserializerProvider.INSTANCE
-                                                    .getSerializerDeserializer(BuiltinType.ADATE);
+                                            serde = dateSerde;
                                             resultType = ATypeTag.DATE;
                                             chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
                                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                                             break;
                                         case TIME:
                                             if (yearMonth == 0) {
-                                                serde = SerializerDeserializerProvider.INSTANCE
-                                                        .getSerializerDeserializer(BuiltinType.ATIME);
+                                                serde = timeSerde;
                                                 resultType = ATypeTag.TIME;
                                                 chronon = ATimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                                 isTimeOnly = true;
                                                 break;
                                             }
                                         default:
-                                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                                    bytes1[offset1]);
+                                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(),
+                                                    bytes0[offset0], bytes1[offset1]);
                                     }
                                     break;
                                 default:
-                                    throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
+                                    throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
                                             bytes1[offset1]);
                             }
 
-                            chronon = evaluateTimeDurationArithmetic(chronon, yearMonth, dayTime, isTimeOnly);
-
-                            switch (resultType) {
-                                case DATE:
-                                    if (chronon < 0 && chronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
-                                        chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY - 1;
-                                    } else {
-                                        chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY;
-                                    }
-                                    aDate.setValue((int) chronon);
-                                    serde.serialize(aDate, out);
-                                    break;
-                                case TIME:
-                                    aTime.setValue((int) chronon);
-                                    serde.serialize(aTime, out);
-                                    break;
-                                case DATETIME:
-                                    aDatetime.setValue(chronon);
-                                    serde.serialize(aDatetime, out);
-                                    break;
-                                default:
-                                    throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0],
-                                            bytes1[offset1]);
+                            if (evaluateTimeDurationArithmetic(chronon, yearMonth, dayTime, isTimeOnly, aInt64)) {
+                                chronon = aInt64.getLongValue();
+                                switch (resultType) {
+                                    case DATE:
+                                        if (chronon < 0 && chronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
+                                            chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY - 1;
+                                        } else {
+                                            chronon = chronon / GregorianCalendarSystem.CHRONON_OF_DAY;
+                                        }
+                                        aDate.setValue((int) chronon);
+                                        serde.serialize(aDate, out);
+                                        break;
+                                    case TIME:
+                                        aTime.setValue((int) chronon);
+                                        serde.serialize(aTime, out);
+                                        break;
+                                    case DATETIME:
+                                        aDatetime.setValue(chronon);
+                                        serde.serialize(aDatetime, out);
+                                        break;
+                                    default:
+                                        throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
+                                                bytes1[offset1]);
+                                }
+                            } else {
+                                nullSerde.serialize(ANull.NULL, out);
                             }
                         }
                     }
@@ -486,4 +542,8 @@
             }
         };
     }
+
+    protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+        return argTypeMax;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
index 47c8d34..8a66e46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractQuadStringStringEval.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -46,82 +47,55 @@
 
     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private DataOutput dout = resultStorage.getDataOutput();
-    private IPointable array0 = new VoidPointable();
-    private IPointable array1 = new VoidPointable();
-    private IPointable array2 = new VoidPointable();
-    private IPointable array3 = new VoidPointable();
+    private IPointable ptr0 = new VoidPointable();
+    private IPointable ptr1 = new VoidPointable();
+    private IPointable ptr2 = new VoidPointable();
+    private IPointable ptr3 = new VoidPointable();
     private IScalarEvaluator eval0;
     private IScalarEvaluator eval1;
     private IScalarEvaluator eval2;
     private IScalarEvaluator eval3;
 
-    private final FunctionIdentifier funcID;
+    protected final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     private AMutableString resultBuffer = new AMutableString("");
     @SuppressWarnings("rawtypes")
     private ISerializerDeserializer strSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
 
-    private final UTF8StringPointable strPtr1st = new UTF8StringPointable();
-    private final UTF8StringPointable strPtr2nd = new UTF8StringPointable();
-    private final UTF8StringPointable strPtr3rd = new UTF8StringPointable();
-    private final UTF8StringPointable strPtr4th = new UTF8StringPointable();
+    private final UTF8StringPointable strPtr0 = new UTF8StringPointable();
+    private final UTF8StringPointable strPtr1 = new UTF8StringPointable();
+    private final UTF8StringPointable strPtr2 = new UTF8StringPointable();
+    private final UTF8StringPointable strPtr3 = new UTF8StringPointable();
 
     public AbstractQuadStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
             IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, IScalarEvaluatorFactory eval3,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
         this.eval3 = eval3.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-        eval0.evaluate(tuple, array0);
-        eval1.evaluate(tuple, array1);
-        eval2.evaluate(tuple, array2);
-        eval3.evaluate(tuple, array3);
+        eval0.evaluate(tuple, ptr0);
+        eval1.evaluate(tuple, ptr1);
+        eval2.evaluate(tuple, ptr2);
+        eval3.evaluate(tuple, ptr3);
 
-        byte[] bytes0 = array0.getByteArray();
-        byte[] bytes1 = array1.getByteArray();
-        byte[] bytes2 = array2.getByteArray();
-        byte[] bytes3 = array3.getByteArray();
-
-        int start0 = array0.getStartOffset();
-        int start1 = array1.getStartOffset();
-        int start2 = array2.getStartOffset();
-        int start3 = array3.getStartOffset();
-
-        int len0 = array0.getLength();
-        int len1 = array1.getLength();
-        int len2 = array2.getLength();
-        int len3 = array3.getLength();
+        processArgument(0, ptr0, strPtr0);
+        processArgument(1, ptr1, strPtr1);
+        processArgument(2, ptr2, strPtr2);
+        processArgument(3, ptr3, strPtr3);
 
         resultStorage.reset();
-        // Type check.
-        if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-        }
-        if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-        }
-        if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-        }
-        if (bytes3[start3] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 3, bytes1[start3], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-        }
-
-        strPtr1st.set(bytes0, start0 + 1, len0);
-        strPtr2nd.set(bytes1, start1 + 1, len1);
-        strPtr3rd.set(bytes2, start2 + 1, len2);
-        strPtr4th.set(bytes3, start3 + 1, len3);
-
         try {
-            String res = compute(strPtr1st, strPtr2nd, strPtr3rd, strPtr4th);
+            String res = compute(strPtr0, strPtr1, strPtr2, strPtr3);
             resultBuffer.setValue(res);
             strSerde.serialize(resultBuffer, dout);
         } catch (IOException e) {
@@ -130,7 +104,19 @@
         result.set(resultStorage);
     }
 
+    protected void processArgument(int argIdx, IPointable argPtr, UTF8StringPointable outStrPtr)
+            throws HyracksDataException {
+        byte[] bytes = argPtr.getByteArray();
+        int start = argPtr.getStartOffset();
+        // Type check.
+        if (bytes[start] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            throw new TypeMismatchException(sourceLoc, funcID, argIdx, bytes[start],
+                    ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+        }
+        int len = argPtr.getLength();
+        outStrPtr.set(bytes, start + 1, len);
+    }
+
     protected abstract String compute(UTF8StringPointable strPtr1st, UTF8StringPointable strPtr2nd,
             UTF8StringPointable strPtr3rd, UTF8StringPointable strPtr4th) throws IOException;
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
index 89bba67..c4bd804 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringStringStringIntEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -57,10 +58,12 @@
 
     // Function ID, for error reporting.
     protected final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractStringStringStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
             IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, IScalarEvaluatorFactory eval3,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
+        this.sourceLoc = sourceLoc;
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
@@ -96,18 +99,18 @@
 
         // Type check.
         if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes3[start3] != ATypeTag.SERIALIZED_INT8_TYPE_TAG && bytes3[start3] != ATypeTag.SERIALIZED_INT16_TYPE_TAG
                 && bytes3[start3] != ATypeTag.SERIALIZED_INT32_TYPE_TAG
                 && bytes3[start3] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 3, bytes3[start3], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+            throw new TypeMismatchException(sourceLoc, funcID, 3, bytes3[start3], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT64_TYPE_TAG);
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
index b2da86d..5624bcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringBoolEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -36,9 +37,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
 
     public AbstractTripleStringBoolEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
index 26563af..51d85f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringEval.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -55,13 +56,16 @@
 
     // Function ID, for error reporting.
     protected final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractTripleStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0, IScalarEvaluatorFactory eval1,
-            IScalarEvaluatorFactory eval2, FunctionIdentifier funcID) throws HyracksDataException {
+            IScalarEvaluatorFactory eval2, FunctionIdentifier funcID, SourceLocation sourceLoc)
+            throws HyracksDataException {
         this.eval0 = eval0.createScalarEvaluator(context);
         this.eval1 = eval1.createScalarEvaluator(context);
         this.eval2 = eval2.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
@@ -87,13 +91,13 @@
 
         // Type check.
         if (bytes0[start0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, bytes0[start0], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes1[start1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 1, bytes1[start1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         if (bytes2[start2] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 2, bytes2[start2], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
 
         // Sets argument UTF8Pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
index 1ee6c8b..5636a3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringIntEval.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -38,9 +39,9 @@
     private final AMutableInt32 resultValue = new AMutableInt32(0);
 
     public AbstractTripleStringIntEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
index 3c73b36..031be7d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractTripleStringStringEval.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
@@ -37,9 +38,9 @@
     private final AMutableString resultValue = new AMutableString("");
 
     public AbstractTripleStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory eval0,
-            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID)
-            throws HyracksDataException {
-        super(context, eval0, eval1, eval2, funcID);
+            IScalarEvaluatorFactory eval1, IScalarEvaluatorFactory eval2, FunctionIdentifier funcID,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, eval0, eval1, eval2, funcID, sourceLoc);
     }
 
     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
index 241452a..00586d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericDoubleFunctionEval.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 
 /**
@@ -31,8 +32,8 @@
 abstract class AbstractUnaryNumericDoubleFunctionEval extends AbstractUnaryNumericFunctionEval {
 
     public AbstractUnaryNumericDoubleFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
-        super(context, argEvalFactory, funcID);
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, argEvalFactory, funcID, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
index 61cd6f6..d2a4164 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryNumericFunctionEval.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -92,11 +93,13 @@
 
     // The function identifier, used for error messages.
     private final FunctionIdentifier funcID;
+    private final SourceLocation sourceLoc;
 
     public AbstractUnaryNumericFunctionEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.argEval = argEvalFactory.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @SuppressWarnings("unchecked")
@@ -126,7 +129,7 @@
             double val = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
             processDouble(val, result);
         } else {
-            throw new TypeMismatchException(funcID, 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+            throw new TypeMismatchException(sourceLoc, funcID, 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
                     ATypeTag.SERIALIZED_INT64_TYPE_TAG, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
index a8853f1..fca1068 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractUnaryStringStringEval.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -50,11 +51,13 @@
     private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     private final DataOutput dataOutput = resultStorage.getDataOutput();
     private final FunctionIdentifier funcID;
+    protected final SourceLocation sourceLoc;
 
     AbstractUnaryStringStringEval(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory,
-            FunctionIdentifier funcID) throws HyracksDataException {
+            FunctionIdentifier funcID, SourceLocation sourceLoc) throws HyracksDataException {
         this.argEval = argEvalFactory.createScalarEvaluator(context);
         this.funcID = funcID;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -65,7 +68,8 @@
         int offset = argPtr.getStartOffset();
         byte inputTypeTag = argBytes[offset];
         if (inputTypeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            throw new TypeMismatchException(funcID, 0, argBytes[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+            throw new TypeMismatchException(sourceLoc, funcID, 0, argBytes[offset],
+                    ATypeTag.SERIALIZED_STRING_TYPE_TAG);
         }
         stringPtr.set(argBytes, offset + 1, argPtr.getLength() - 1);
         resultArray.reset();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
index e9f9c9e..e0d9bd3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -107,7 +107,7 @@
                                 continue;
                             }
                             if (bytes[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), i, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), i, bytes[offset],
                                         ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                             }
                             boolean argResult = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index b78bc5c..b03dafa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -61,7 +61,7 @@
         return BuiltinFunctions.ANY_COLLECTION_MEMBER;
     }
 
-    private static class AnyCollectionMemberEvalFactory implements IScalarEvaluatorFactory {
+    private class AnyCollectionMemberEvalFactory implements IScalarEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
 
@@ -95,7 +95,7 @@
 
                     if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                             && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.ANY_COLLECTION_MEMBER, 0, serList[offset],
+                        throw new TypeMismatchException(sourceLoc, serList[offset],
                                 ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayAppendDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayAppendDescriptor.java
new file mode 100755
index 0000000..009d7b3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayAppendDescriptor.java
@@ -0,0 +1,112 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_append(list, val1, val2, ...) returns a new list with all the values appended to the input list items.
+ * Values can be null (i.e., one can append nulls)
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not a list.
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayAppendDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayAppendDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_APPEND;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayAppendEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayAppendEval extends AbstractArrayAddRemoveEval {
+
+        public ArrayAppendEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, 0, 1, args.length - 1, argTypes, false, sourceLoc, true, true);
+        }
+
+        @Override
+        protected int getPosition(IFrameTupleReference tuple, IPointable l, ATypeTag listTag)
+                throws HyracksDataException {
+            // l = list
+            if (listTag == ATypeTag.ARRAY) {
+                return AOrderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+            } else if (listTag == ATypeTag.MULTISET) {
+                return AUnorderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+            } else {
+                return RETURN_NULL;
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayConcatDescriptor.java
new file mode 100755
index 0000000..c808ec5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayConcatDescriptor.java
@@ -0,0 +1,119 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * <pre>
+ * array_concat(list1, list2, ...) returns a new list with all the values of all lists appended in order into the
+ * new list. Items of the lists can be null or missing (both are added as a null value).
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. an error if the input lists are not of the same type (one is an ordered list while the other is unordered).
+ * 3. null, if any input list is null or is not a list.
+ * 4. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayConcatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayConcatDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_CONCAT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayConcatEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayConcatEval extends AbstractArrayProcessArraysEval {
+        public ArrayConcatEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, false, sourceLoc, argTypes);
+        }
+
+        @Override
+        protected void init() {
+            // do nothing
+        }
+
+        @Override
+        protected void finish(IAsterixListBuilder listBuilder) {
+            // do nothing
+        }
+
+        @Override
+        protected void release() {
+            // do nothing
+        }
+
+        @Override
+        protected boolean processItem(IPointable item, int listIndex, IAsterixListBuilder listBuilder)
+                throws HyracksDataException {
+            listBuilder.addItem(item);
+            // listBuilder always copies the item, so we're not using the supplied item
+            return false;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java
new file mode 100755
index 0000000..5443834
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java
@@ -0,0 +1,98 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * <pre>
+ * array_contains(list, val) returns true if the the input list contains the value argument. It's case-sensitive to
+ * string value argument.
+ *
+ * It throws an error at compile time if the number of arguments != 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if any argument is null.
+ * 3. an error if the value is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. null, if the input list is not a list.
+ * 5. otherwise, returns true or false.
+ *
+ * </pre>
+ */
+public class ArrayContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayContainsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_CONTAINS;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayContainsEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayContainsEval extends AbstractArraySearchEval {
+        private final ISerializerDeserializer booleanSerde;
+
+        public ArrayContainsEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, sourceLoc);
+            // TODO(ali): should we get the nontagged serde?
+            booleanSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+        }
+
+        @Override
+        public void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException {
+            storage.reset();
+            booleanSerde.serialize(ABoolean.valueOf(intValue.getIntegerValue() != -1), storage.getDataOutput());
+            result.set(storage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
new file mode 100755
index 0000000..0179d34
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
@@ -0,0 +1,187 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+import java.util.List;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * <pre>
+ * array_distinct(list) returns a new list with distinct items of the input list. The returned list has the same type as
+ * the input list. The list can contain null and missing items. Null and missing are considered to be the same.
+ * It's case-sensitive to string items.
+ *
+ * array_distinct([1,2,null,4,missing,2,1]) will output [1,2,null,4]
+ *
+ * It throws an error at compile time if the number of arguments != 1
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not a list.
+ * 3. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayDistinctDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayDistinctDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            // the type of the input list is needed in order to use the same type for the new returned list
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_DISTINCT;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayDistinctFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+    public class ArrayDistinctFunction extends AbstractArrayProcessEval {
+        private final SourceLocation sourceLoc;
+        private final IBinaryHashFunction binaryHashFunction;
+        private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
+        private IPointable item;
+        private ArrayBackedValueStorage storage;
+
+        public ArrayDistinctFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            super(args, ctx, inputListType);
+            this.sourceLoc = sourceLoc;
+            hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
+                    .createBinaryHashFunction();
+        }
+
+        @Override
+        protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder) throws IOException {
+            int hash;
+            boolean itemInStorage;
+            boolean nullMissingWasAdded = false;
+            List<IPointable> sameHashes;
+            hashes.clear();
+            item = pointableAllocator.allocateEmpty();
+            storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+            for (int i = 0; i < listAccessor.size(); i++) {
+                // get the item and compute its hash
+                itemInStorage = listAccessor.getOrWriteItem(i, item, storage);
+                if (ATYPETAGDESERIALIZER.deserialize(item.getByteArray()[item.getStartOffset()]).isDerivedType()) {
+                    throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLoc);
+                }
+                if (isNullOrMissing(item)) {
+                    if (!nullMissingWasAdded) {
+                        listBuilder.addItem(item);
+                        nullMissingWasAdded = true;
+                    }
+                } else {
+                    // look up if it already exists
+                    hash = binaryHashFunction.hash(item.getByteArray(), item.getStartOffset(), item.getLength());
+                    hashes.get(hash);
+                    sameHashes = hashes.get(hash);
+                    if (sameHashes == null) {
+                        // new item
+                        sameHashes = arrayListAllocator.allocate(null);
+                        sameHashes.clear();
+                        addItem(item, listBuilder, itemInStorage, sameHashes);
+                        hashes.put(hash, sameHashes);
+                        item = pointableAllocator.allocateEmpty();
+                    } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
+                        // new item, it could happen that two hashes are the same but they are for different items
+                        addItem(item, listBuilder, itemInStorage, sameHashes);
+                        item = pointableAllocator.allocateEmpty();
+                    }
+                }
+            }
+        }
+
+        private boolean isNullOrMissing(IPointable item) {
+            byte tag = item.getByteArray()[item.getStartOffset()];
+            return tag == ATypeTag.SERIALIZED_NULL_TYPE_TAG || tag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+        }
+
+        private void addItem(IPointable item, IAsterixListBuilder listBuilder, boolean itemInStorage,
+                List<IPointable> sameHashes) throws HyracksDataException {
+            sameHashes.add(item);
+            listBuilder.addItem(item);
+            if (itemInStorage) {
+                // create new storage since the added item is using it now
+                storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayFlattenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayFlattenDescriptor.java
new file mode 100644
index 0000000..fc8fc84
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayFlattenDescriptor.java
@@ -0,0 +1,228 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.AbvsBuilderFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.base.ListAccessorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_flatten(list, depth) returns a new list with any nested list (all types) flattened up to the specified
+ * depth. The returned list type is the same as the input list type. Null and missing items are preserved.
+ * If the depth < 0, then it flattens the input list all the way deep.
+ *
+ * array_flatten([2, null, [5,6], 3, missing], 1) will result in [2, null, 5, 6, 3, null]
+ * array_flatten([2, [5,6], 3], 0) will result in [2, [5,6], 3] (0 depth does nothing)
+ *
+ * It throws an error at compile time if the number of arguments != 2
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if:
+ * - any argument is null.
+ * - the input list is not a list.
+ * - the depth arg is not numeric or
+ * - it's a floating-point number with decimals (e.g. 1.2 will produce null, 1.0 is OK).
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayFlattenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayFlattenDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_FLATTEN;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayFlattenEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayFlattenEval implements IScalarEvaluator {
+        private final IScalarEvaluator listEval;
+        private final IScalarEvaluator depthEval;
+        private final IPointable list;
+        private final AbstractPointable pointable;
+        private final TaggedValuePointable depthArg;
+        private final IObjectPool<IMutableValueStorage, ATypeTag> storageAllocator;
+        private final IObjectPool<ListAccessor, ATypeTag> listAccessorAllocator;
+        private final CastTypeEvaluator caster;
+        private final ArrayBackedValueStorage finalStorage;
+        private ArrayBackedValueStorage storage;
+        private IAsterixListBuilder orderedListBuilder;
+        private IAsterixListBuilder unorderedListBuilder;
+
+        public ArrayFlattenEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storageAllocator = new ListObjectPool<>(new AbvsBuilderFactory());
+            listAccessorAllocator = new ListObjectPool<>(new ListAccessorFactory());
+            finalStorage = new ArrayBackedValueStorage();
+            listEval = args[0].createScalarEvaluator(ctx);
+            depthEval = args[1].createScalarEvaluator(ctx);
+            list = new VoidPointable();
+            pointable = new VoidPointable();
+            caster = new CastTypeEvaluator();
+            depthArg = new TaggedValuePointable();
+            orderedListBuilder = null;
+            unorderedListBuilder = null;
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            // 1st arg: list to flatten
+            listEval.evaluate(tuple, pointable);
+            // 2nd arg: depthArg
+            depthEval.evaluate(tuple, depthArg);
+
+            ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(pointable.getByteArray()[pointable.getStartOffset()]);
+            if (!ATypeHierarchy.isCompatible(ATYPETAGDESERIALIZER.deserialize(depthArg.getTag()), ATypeTag.DOUBLE)
+                    || !listType.isListType()) {
+                PointableHelper.setNull(result);
+                return;
+            }
+            String name = getIdentifier().getName();
+            double depth = ATypeHierarchy.getDoubleValue(name, 1, depthArg.getByteArray(), depthArg.getStartOffset());
+            if (Double.isNaN(depth) || Double.isInfinite(depth) || Math.floor(depth) < depth) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            try {
+                caster.resetAndAllocate(DefaultOpenFieldType.getDefaultOpenFieldType(listType), inputListType,
+                        listEval);
+                caster.cast(pointable, list);
+
+                int depthInt = (int) depth;
+                // create list
+                IAsterixListBuilder listBuilder;
+                if (listType == ATypeTag.ARRAY) {
+                    if (orderedListBuilder == null) {
+                        orderedListBuilder = new OrderedListBuilder();
+                    }
+                    listBuilder = orderedListBuilder;
+                } else {
+                    if (unorderedListBuilder == null) {
+                        unorderedListBuilder = new UnorderedListBuilder();
+                    }
+                    listBuilder = unorderedListBuilder;
+                }
+
+                ListAccessor mainListAccessor = listAccessorAllocator.allocate(null);
+                listBuilder.reset((AbstractCollectionType) DefaultOpenFieldType.getDefaultOpenFieldType(listType));
+                mainListAccessor.reset(list.getByteArray(), list.getStartOffset());
+
+                storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                process(mainListAccessor, listBuilder, 0, depthInt);
+                finalStorage.reset();
+                listBuilder.write(finalStorage.getDataOutput(), true);
+                result.set(finalStorage);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            } finally {
+                storageAllocator.reset();
+                listAccessorAllocator.reset();
+                caster.deallocatePointables();
+            }
+        }
+
+        private void process(ListAccessor listAccessor, IAsterixListBuilder listBuilder, int currentDepth, int depth)
+                throws IOException {
+            boolean itemInStorage;
+            for (int i = 0; i < listAccessor.size(); i++) {
+                itemInStorage = listAccessor.getOrWriteItem(i, pointable, storage);
+                // if item is not a list or depth is reached, write it
+                if (!ATYPETAGDESERIALIZER.deserialize(pointable.getByteArray()[pointable.getStartOffset()]).isListType()
+                        || currentDepth == depth) {
+                    listBuilder.addItem(pointable);
+                } else {
+                    // recurse on the sublist
+                    ListAccessor newListAccessor = listAccessorAllocator.allocate(null);
+                    newListAccessor.reset(pointable.getByteArray(), pointable.getStartOffset());
+                    if (itemInStorage) {
+                        // create a new storage since the item is using it
+                        storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                        storage.reset();
+                    }
+                    process(newListAccessor, listBuilder, currentDepth + 1, depth);
+                }
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java
new file mode 100755
index 0000000..53853e2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_ifnull(list) returns the first item it encounters that is not a null or missing. Otherwise, it returns null.
+ *
+ * It throws an error at compile time if the number of arguments != 1
+ *
+ * It returns in order:
+ * 1. missing if the input list is missing
+ * 2. null if the input list is null or is not a list.
+ * 3. otherwise, the first non-null non-missing item in the list. Otherwise, null.
+ *
+ * </pre>
+ */
+public class ArrayIfNullDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayIfNullDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_IFNULL;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayIfNullEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayIfNullEval implements IScalarEvaluator {
+        private final ArrayBackedValueStorage storage;
+        private final IScalarEvaluator listArgEval;
+        private final IPointable listArg;
+        private final ListAccessor listAccessor;
+        private final AbstractPointable item;
+
+        public ArrayIfNullEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            listArg = new VoidPointable();
+            item = new VoidPointable();
+            listAccessor = new ListAccessor();
+            listArgEval = args[0].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            // get the list argument and make sure it's a list
+            listArgEval.evaluate(tuple, listArg);
+            byte[] listBytes = listArg.getByteArray();
+            int offset = listArg.getStartOffset();
+            ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(listBytes[offset]);
+            if (!listType.isListType()) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            listAccessor.reset(listBytes, offset);
+            ATypeTag itemTypeTag = listAccessor.getItemType();
+            try {
+                if (itemTypeTag == ATypeTag.NULL || itemTypeTag == ATypeTag.MISSING) {
+                    // list of nulls or list of missings
+                    PointableHelper.setNull(result);
+                    return;
+                }
+
+                int numItems = listAccessor.size();
+                for (int i = 0; i < numItems; i++) {
+                    listAccessor.getOrWriteItem(i, item, storage);
+                    itemTypeTag = ATYPETAGDESERIALIZER.deserialize(item.getByteArray()[item.getStartOffset()]);
+                    if (itemTypeTag != ATypeTag.NULL && itemTypeTag != ATypeTag.MISSING) {
+                        result.set(item);
+                        return;
+                    }
+                }
+                PointableHelper.setNull(result);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayInsertDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayInsertDescriptor.java
new file mode 100755
index 0000000..969e1d7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayInsertDescriptor.java
@@ -0,0 +1,149 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_insert(list, pos, val1, val2, ...) returns a new list with all values inserted at the specified position.
+ * Values can be null (i.e., one can insert nulls). Position can be negative where the last position = -1. When position
+ * is positive then the first position = 0. Input list can be empty where the only valid position is 0.
+ * For the list [5,6], the valid positions are 0, 1, 2, -1, -2. If position is floating-point, it's casted to integer.
+ *
+ * It throws an error at compile time if the number of arguments < 3
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if
+ * - the list arg is null or it's not a list
+ * - the position is not numeric or the position is out of bound or it's a floating-point with decimals or NaN or +-INF.
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayInsertDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayInsertDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_INSERT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayInsertEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayInsertEval extends AbstractArrayAddRemoveEval {
+        private final TaggedValuePointable positionArg;
+        private final IScalarEvaluator positionArgEval;
+
+        public ArrayInsertEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, 0, 2, args.length - 2, argTypes, false, sourceLoc, true, true);
+            positionArg = new TaggedValuePointable();
+            positionArgEval = args[1].createScalarEvaluator(ctx);
+        }
+
+        @Override
+        protected int getPosition(IFrameTupleReference tuple, IPointable l, ATypeTag listTag)
+                throws HyracksDataException {
+            positionArgEval.evaluate(tuple, positionArg);
+            if (positionArg.getTag() == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+                return RETURN_MISSING;
+            }
+
+            double position;
+            if (!ATypeHierarchy.isCompatible(ATypeTag.DOUBLE, ATYPETAGDESERIALIZER.deserialize(positionArg.getTag()))
+                    || !listTag.isListType()) {
+                return RETURN_NULL;
+            } else {
+                String name = getIdentifier().getName();
+                position = ATypeHierarchy.getDoubleValue(name, 1, positionArg.getByteArray(),
+                        positionArg.getStartOffset());
+                if (Double.isNaN(position) || Double.isInfinite(position) || Math.floor(position) < position) {
+                    return RETURN_NULL;
+                }
+                // list size
+                int size;
+                if (listTag == ATypeTag.ARRAY) {
+                    size = AOrderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+                } else {
+                    size = AUnorderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+                }
+                // adjust position for negative positions
+                if (position < 0) {
+                    position = size + position;
+                }
+                // position should always be positive now and should be within [0-list_size]
+                if (position < 0 || position > size) {
+                    return RETURN_NULL;
+                }
+                return (int) position;
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
new file mode 100755
index 0000000..85ba01f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
@@ -0,0 +1,409 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.AbvsBuilderFactory;
+import org.apache.asterix.builders.ArrayListFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+/**
+ * <pre>
+ * array_intersect(list1, list2, ...) returns a new list containing items that are present in all of the input
+ * lists. Null and missing items are ignored. It's case-sensitive to string items.
+ *
+ * array_intersect([null, 2, missing], [3,missing,2,null]) will result in [2].
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. an error if the input lists are not of the same type (one is an ordered list while the other is unordered).
+ * 3. null, if any input list is null or is not a list.
+ * 4. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 5. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayIntersectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayIntersectDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_INTERSECT;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayIntersectEval(args, ctx);
+            }
+        };
+    }
+
+    protected class ValueListIndex implements IValueReference {
+        private IPointable value;
+        private int listIndex;
+
+        protected ValueListIndex() {
+        }
+
+        protected void set(IPointable value, int listIndex) {
+            this.value = value;
+            this.listIndex = listIndex;
+        }
+
+        @Override
+        public byte[] getByteArray() {
+            return value.getByteArray();
+        }
+
+        @Override
+        public int getStartOffset() {
+            return value.getStartOffset();
+        }
+
+        @Override
+        public int getLength() {
+            return value.getLength();
+        }
+    }
+
+    protected class ValueListIndexAllocator implements IObjectFactory<ValueListIndex, ATypeTag> {
+
+        protected ValueListIndexAllocator() {
+        }
+
+        @Override
+        public ValueListIndex create(ATypeTag arg) {
+            return new ValueListIndex();
+        }
+    }
+
+    public class ArrayIntersectEval implements IScalarEvaluator {
+        private final ListAccessor listAccessor;
+        private final IPointable pointable;
+        private final ArrayBackedValueStorage currentItemStorage;
+        private final IPointable[] listsArgs;
+        private final IScalarEvaluator[] listsEval;
+        private final IBinaryHashFunction binaryHashFunction;
+        private final Int2ObjectMap<List<ValueListIndex>> hashes;
+        private final PointableAllocator pointableAllocator;
+        private final IObjectPool<IMutableValueStorage, ATypeTag> storageAllocator;
+        private final IObjectPool<List<ValueListIndex>, ATypeTag> arrayListAllocator;
+        private final IObjectPool<ValueListIndex, ATypeTag> valueListIndexAllocator;
+        private final ArrayBackedValueStorage finalResult;
+        private final CastTypeEvaluator caster;
+        private final IBinaryComparator comp;
+        private IAsterixListBuilder orderedListBuilder;
+        private IAsterixListBuilder unorderedListBuilder;
+
+        public ArrayIntersectEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            orderedListBuilder = null;
+            unorderedListBuilder = null;
+            pointableAllocator = new PointableAllocator();
+            storageAllocator = new ListObjectPool<>(new AbvsBuilderFactory());
+            arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
+            valueListIndexAllocator = new ListObjectPool<>(new ValueListIndexAllocator());
+            hashes = new Int2ObjectOpenHashMap<>();
+            finalResult = new ArrayBackedValueStorage();
+            listAccessor = new ListAccessor();
+            caster = new CastTypeEvaluator();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            listsArgs = new IPointable[args.length];
+            listsEval = new IScalarEvaluator[args.length];
+            pointable = new VoidPointable();
+            currentItemStorage = new ArrayBackedValueStorage();
+            for (int i = 0; i < args.length; i++) {
+                listsArgs[i] = new VoidPointable();
+                listsEval[i] = args[i].createScalarEvaluator(ctx);
+            }
+            binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
+                    .createBinaryHashFunction();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            byte listArgType;
+            boolean returnNull = false;
+            AbstractCollectionType outList = null;
+            ATypeTag listTag;
+            int minListIndex = 0;
+            int minSize = -1;
+            int nextSize;
+
+            // evaluate all the lists first to make sure they're all actually lists and of the same list type
+            try {
+                for (int i = 0; i < listsEval.length; i++) {
+                    listsEval[i].evaluate(tuple, pointable);
+                    if (!returnNull) {
+                        listArgType = pointable.getByteArray()[pointable.getStartOffset()];
+                        listTag = ATYPETAGDESERIALIZER.deserialize(listArgType);
+                        if (!listTag.isListType()) {
+                            returnNull = true;
+                        } else if (outList != null && outList.getTypeTag() != listTag) {
+                            throw new RuntimeDataException(ErrorCode.DIFFERENT_LIST_TYPE_ARGS, sourceLoc);
+                        } else {
+                            if (outList == null) {
+                                outList =
+                                        (AbstractCollectionType) DefaultOpenFieldType.getDefaultOpenFieldType(listTag);
+                            }
+
+                            caster.resetAndAllocate(outList, argTypes[i], listsEval[i]);
+                            caster.cast(pointable, listsArgs[i]);
+                            nextSize = getNumItems(outList, listsArgs[i].getByteArray(), listsArgs[i].getStartOffset());
+                            if (nextSize < minSize || minSize == -1) {
+                                minSize = nextSize;
+                                minListIndex = i;
+                            }
+                        }
+                    }
+                }
+
+                if (returnNull) {
+                    PointableHelper.setNull(result);
+                    return;
+                }
+
+                IAsterixListBuilder listBuilder;
+                if (outList.getTypeTag() == ATypeTag.ARRAY) {
+                    if (orderedListBuilder == null) {
+                        orderedListBuilder = new OrderedListBuilder();
+                    }
+                    listBuilder = orderedListBuilder;
+                } else {
+                    if (unorderedListBuilder == null) {
+                        unorderedListBuilder = new UnorderedListBuilder();
+                    }
+                    listBuilder = unorderedListBuilder;
+                }
+
+                IPointable listArg;
+                hashes.clear();
+
+                // first, get distinct items of the most restrictive (smallest) list.
+                // values will be added to listBuilder after inspecting all input lists
+                listArg = listsArgs[minListIndex];
+                listAccessor.reset(listArg.getByteArray(), listArg.getStartOffset());
+                buildRestrictiveList(listAccessor);
+                listBuilder.reset(outList);
+
+                if (!hashes.isEmpty()) {
+                    // process each list one by one
+                    for (int listIndex = 0; listIndex < listsArgs.length; listIndex++) {
+                        // TODO(ali): find a way to avoid comparing the smallest list
+                        listArg = listsArgs[listIndex];
+                        listAccessor.reset(listArg.getByteArray(), listArg.getStartOffset());
+                        processList(listAccessor, listIndex, listBuilder);
+                    }
+                }
+
+                finalResult.reset();
+                listBuilder.write(finalResult.getDataOutput(), true);
+                result.set(finalResult);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            } finally {
+                caster.deallocatePointables();
+                valueListIndexAllocator.reset();
+                storageAllocator.reset();
+                arrayListAllocator.reset();
+                pointableAllocator.reset();
+            }
+        }
+
+        private int getNumItems(AbstractCollectionType listType, byte[] listBytes, int offset) {
+            if (listType.getTypeTag() == ATypeTag.ARRAY) {
+                return AOrderedListSerializerDeserializer.getNumberOfItems(listBytes, offset);
+            } else {
+                return AUnorderedListSerializerDeserializer.getNumberOfItems(listBytes, offset);
+            }
+        }
+
+        // puts all the items of the smallest list in "hashes"
+        private void buildRestrictiveList(ListAccessor listAccessor) throws IOException {
+            if (listAccessor.size() > 0) {
+                int hash;
+                List<ValueListIndex> sameHashes;
+                boolean itemInStorage;
+                IPointable item = pointableAllocator.allocateEmpty();
+                ArrayBackedValueStorage storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                storage.reset();
+                for (int j = 0; j < listAccessor.size(); j++) {
+                    itemInStorage = listAccessor.getOrWriteItem(j, item, storage);
+                    validateItem(item);
+                    if (notNullAndMissing(item)) {
+                        hash = binaryHashFunction.hash(item.getByteArray(), item.getStartOffset(), item.getLength());
+                        sameHashes = hashes.get(hash);
+                        if (addToSmallestList(item, hash, sameHashes)) {
+                            // item has been added to intersect list and is being used, allocate new pointable
+                            item = pointableAllocator.allocateEmpty();
+                            if (itemInStorage) {
+                                storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                                storage.reset();
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        private void processList(ListAccessor listAccessor, int listIndex, IAsterixListBuilder listBuilder)
+                throws IOException {
+            int hash;
+            List<ValueListIndex> sameHashes;
+            for (int j = 0; j < listAccessor.size(); j++) {
+                listAccessor.getOrWriteItem(j, pointable, currentItemStorage);
+                validateItem(pointable);
+                if (notNullAndMissing(pointable)) {
+                    // hash the item and look up to see if it is common
+                    hash = binaryHashFunction.hash(pointable.getByteArray(), pointable.getStartOffset(),
+                            pointable.getLength());
+                    sameHashes = hashes.get(hash);
+                    incrementIfCommonValue(pointable, sameHashes, listIndex, listBuilder);
+                }
+            }
+        }
+
+        // collects the items of the most restrictive list, it initializes the list index as -1. each successive list
+        // should stamp the value with its list index if the list has the item. It starts with list index = 0
+        private boolean addToSmallestList(IPointable item, int hash, List<ValueListIndex> sameHashes)
+                throws IOException {
+            // add if new item
+            if (sameHashes == null) {
+                List<ValueListIndex> newHashes = arrayListAllocator.allocate(null);
+                newHashes.clear();
+                ValueListIndex valueListIndex = valueListIndexAllocator.allocate(null);
+                valueListIndex.set(item, -1);
+                newHashes.add(valueListIndex);
+                hashes.put(hash, newHashes);
+                return true;
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
+                ValueListIndex valueListIndex = valueListIndexAllocator.allocate(null);
+                valueListIndex.set(item, -1);
+                sameHashes.add(valueListIndex);
+                return true;
+            }
+            // else ignore for duplicate values in the same list
+            return false;
+        }
+
+        private void incrementIfCommonValue(IPointable item, List<ValueListIndex> sameHashes, int listIndex,
+                IAsterixListBuilder listBuilder) throws IOException {
+            if (sameHashes != null) {
+                // look for the same equal item, add to list builder when all lists have seen this item
+                incrementIfExists(sameHashes, item, listIndex, listBuilder);
+            }
+        }
+
+        private boolean notNullAndMissing(IPointable item) {
+            byte tag = item.getByteArray()[item.getStartOffset()];
+            return tag != ATypeTag.SERIALIZED_NULL_TYPE_TAG && tag != ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+        }
+
+        private void incrementIfExists(List<ValueListIndex> sameHashes, IPointable item, int listIndex,
+                IAsterixListBuilder listBuilder) throws HyracksDataException {
+            ValueListIndex sameValue = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
+            if (sameValue != null && listIndex - sameValue.listIndex == 1) {
+                // found the item, its stamp is OK (stamp saves the index of the last list that has seen this item)
+                // increment stamp of this item
+                sameValue.listIndex = listIndex;
+                if (listIndex == listsArgs.length - 1) {
+                    // if this list is the last to stamp, then add to the final result
+                    listBuilder.addItem(item);
+                }
+            }
+        }
+
+        // validates that the item is not derived, multisets, objects and arrays are not yet supported
+        private void validateItem(IPointable item) throws RuntimeDataException {
+            if (ATYPETAGDESERIALIZER.deserialize(item.getByteArray()[item.getStartOffset()]).isDerivedType()) {
+                throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLoc);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
new file mode 100755
index 0000000..411e846
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * <pre>
+ * array_position(list, val) returns the 0-based position (as integer) of the value argument in the input list. If the
+ * value does not exists, it returns -1. It's case-sensitive to string value argument.
+ *
+ * It throws an error at compile time if the number of arguments != 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if any argument is null.
+ * 3. an error if the value is of a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. null, if the input list is not a list.
+ * 5. otherwise, returns the position of the value in the list or -1 if not found.
+ *
+ * </pre>
+ */
+public class ArrayPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayPositionDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_POSITION;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayPositionEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayPositionEval extends AbstractArraySearchEval {
+        private final ISerializerDeserializer intSerde;
+
+        public ArrayPositionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, sourceLoc);
+            intSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+        }
+
+        @Override
+        public void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException {
+            storage.reset();
+            intSerde.serialize(intValue, storage.getDataOutput());
+            result.set(storage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPrependDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPrependDescriptor.java
new file mode 100755
index 0000000..fa56fb1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPrependDescriptor.java
@@ -0,0 +1,103 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_prepend(val1, val2, ..., list) returns a new list with all the values prepended to the input list items.
+ * Values can be null (i.e., one can append nulls)
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not a list.
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayPrependDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayPrependDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_PREPEND;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayPrependEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayPrependEval extends AbstractArrayAddRemoveEval {
+
+        public ArrayPrependEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, args.length - 1, 0, args.length - 1, argTypes, false, sourceLoc, true, true);
+        }
+
+        @Override
+        protected int getPosition(IFrameTupleReference tuple, IPointable listArg, ATypeTag listTag)
+                throws HyracksDataException {
+            return 0;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPutDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPutDescriptor.java
new file mode 100755
index 0000000..571fb4c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPutDescriptor.java
@@ -0,0 +1,161 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_put(list, val1, val2, ...) returns a new list with all the values appended to the input list items only if
+ * the list does not already have the value. Values cannot be null (i.e., one cannot append nulls).
+ * array_put([2, 3], 2, 2, 9, 9) will result in [2, 3, 9, 9].
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if any argument is null.
+ * 3. an error if any value arg is of a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayPutDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayPutDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_PUT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayPutEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayPutEval extends AbstractArrayAddRemoveEval {
+        private final ArrayBackedValueStorage storage;
+        private final IPointable item;
+        private final IBinaryComparator comp;
+        private final boolean[] add;
+
+        public ArrayPutEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, 0, 1, args.length - 1, argTypes, true, sourceLoc, true, false);
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            storage = new ArrayBackedValueStorage();
+            item = new VoidPointable();
+            add = new boolean[args.length - 1];
+        }
+
+        @Override
+        protected int getPosition(IFrameTupleReference tuple, IPointable l, ATypeTag listTag)
+                throws HyracksDataException {
+            // l = list
+            if (listTag == ATypeTag.ARRAY) {
+                return AOrderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+            } else if (listTag == ATypeTag.MULTISET) {
+                return AUnorderedListSerializerDeserializer.getNumberOfItems(l.getByteArray(), l.getStartOffset());
+            } else {
+                return RETURN_NULL;
+            }
+        }
+
+        @Override
+        protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder, IPointable[] values,
+                int position) throws IOException {
+            markAllToBeAdded();
+            // get the list items one by one and append to the new list
+            for (int i = 0; i < listAccessor.size(); i++) {
+                listAccessor.getOrWriteItem(i, item, storage);
+                listBuilder.addItem(item);
+                // mark the equal values to skip adding them
+                for (int j = 0; j < values.length; j++) {
+                    if (add[j] && comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(),
+                            values[j].getByteArray(), values[j].getStartOffset(), values[j].getLength()) == 0) {
+                        add[j] = false;
+                    }
+                    // skip comparison if the value is already marked
+                }
+            }
+            // append the values arguments only if they are not already present in the list, i.e. not marked
+            for (int i = 0; i < values.length; i++) {
+                if (add[i]) {
+                    listBuilder.addItem(values[i]);
+                }
+            }
+        }
+
+        private void markAllToBeAdded() {
+            for (int i = 0; i < add.length; i++) {
+                add[i] = true;
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRangeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRangeDescriptor.java
new file mode 100755
index 0000000..902ec7a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRangeDescriptor.java
@@ -0,0 +1,184 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.typecomputer.impl.ArrayRangeTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_range(start_num, end_num, step_num?) returns a new ordered list, list of long items or double items
+ * depending on the supplied arguments. One floating-point arg will make it a list of double items. step_num is optional
+ * where the default is 1. It returns an empty list for arguments like:
+ * array_range(2, 20, -2), array_range(10, 3, 4) and array_range(1,6,0) where it cannot determine a proper sequence.
+ *
+ * It throws an error at compile time if the number of arguments < 2 or > 3
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if any argument is null or they are not numeric or they are NaN +-INF.
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayRangeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayRangeDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_RANGE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayRangeEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayRangeEval implements IScalarEvaluator {
+        private final OrderedListBuilder listBuilder;
+        private final ArrayBackedValueStorage storage;
+        private final IScalarEvaluator startNumEval;
+        private final TaggedValuePointable start;
+        private final IScalarEvaluator endNumEval;
+        private final TaggedValuePointable end;
+        private final AMutableDouble aDouble;
+        private final AMutableInt64 aLong;
+        private IScalarEvaluator stepNumEval;
+        private TaggedValuePointable step;
+
+        public ArrayRangeEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            start = new TaggedValuePointable();
+            end = new TaggedValuePointable();
+            startNumEval = args[0].createScalarEvaluator(ctx);
+            endNumEval = args[1].createScalarEvaluator(ctx);
+            listBuilder = new OrderedListBuilder();
+            aDouble = new AMutableDouble(0);
+            aLong = new AMutableInt64(0);
+            if (args.length == 3) {
+                stepNumEval = args[2].createScalarEvaluator(ctx);
+                step = new TaggedValuePointable();
+            }
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            startNumEval.evaluate(tuple, start);
+            endNumEval.evaluate(tuple, end);
+            String n = getIdentifier().getName();
+            ATypeTag startTag = ATYPETAGDESERIALIZER.deserialize(start.getTag());
+            ATypeTag endTag = ATYPETAGDESERIALIZER.deserialize(end.getTag());
+            ATypeTag stepTag = ATypeTag.INTEGER;
+            double stepNum = 1;
+            if (stepNumEval != null) {
+                stepNumEval.evaluate(tuple, step);
+                stepTag = ATYPETAGDESERIALIZER.deserialize(step.getTag());
+                if (!ATypeHierarchy.isCompatible(ATypeTag.DOUBLE, stepTag)) {
+                    PointableHelper.setNull(result);
+                    return;
+                }
+                stepNum = ATypeHierarchy.getDoubleValue(n, 2, step.getByteArray(), step.getStartOffset());
+            }
+
+            if (!ATypeHierarchy.isCompatible(ATypeTag.DOUBLE, startTag) || Double.isNaN(stepNum)
+                    || !ATypeHierarchy.isCompatible(ATypeTag.DOUBLE, endTag) || Double.isInfinite(stepNum)) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            ISerializerDeserializer serde;
+            if (ATypeHierarchy.canPromote(startTag, ATypeTag.BIGINT)
+                    && ATypeHierarchy.canPromote(endTag, ATypeTag.BIGINT)
+                    && ATypeHierarchy.canPromote(stepTag, ATypeTag.BIGINT)) {
+                // all 3 numbers are whole numbers
+                serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+                long startNum = ATypeHierarchy.getLongValue(n, 0, start.getByteArray(), start.getStartOffset());
+                long endNum = ATypeHierarchy.getLongValue(n, 1, end.getByteArray(), end.getStartOffset());
+                listBuilder.reset(ArrayRangeTypeComputer.LONG_LIST);
+                while ((startNum < endNum && stepNum > 0) || (startNum > endNum && stepNum < 0)) {
+                    aLong.setValue(startNum);
+                    storage.reset();
+                    serde.serialize(aLong, storage.getDataOutput());
+                    listBuilder.addItem(storage);
+                    startNum += stepNum;
+                }
+            } else {
+                // one number is a floating-point number
+                serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+                double startNum = ATypeHierarchy.getDoubleValue(n, 0, start.getByteArray(), start.getStartOffset());
+                double endNum = ATypeHierarchy.getDoubleValue(n, 1, end.getByteArray(), end.getStartOffset());
+                if (Double.isNaN(startNum) || Double.isInfinite(startNum) || Double.isNaN(endNum)
+                        || Double.isInfinite(endNum)) {
+                    PointableHelper.setNull(result);
+                    return;
+                }
+                listBuilder.reset(ArrayRangeTypeComputer.DOUBLE_LIST);
+                while ((startNum < endNum && stepNum > 0) || (startNum > endNum && stepNum < 0)) {
+                    aDouble.setValue(startNum);
+                    storage.reset();
+                    serde.serialize(aDouble, storage.getDataOutput());
+                    listBuilder.addItem(storage);
+                    startNum += stepNum;
+                }
+            }
+
+            storage.reset();
+            listBuilder.write(storage.getDataOutput(), true);
+            result.set(storage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRemoveDescriptor.java
new file mode 100755
index 0000000..79288e1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRemoveDescriptor.java
@@ -0,0 +1,139 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_remove(list, val1, val2, ...) returns a new list with all the values removed from the input
+ * list. Values cannot be null (i.e., one cannot remove nulls). It's case-sensitive to string value arguments.
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if any argument is null.
+ * 3. an error if any value arg is of a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. otherwise, a new list that has the same type as the input list.
+ *
+ * </pre>
+ */
+public class ArrayRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayRemoveDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_REMOVE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayRemoveEval(args, ctx);
+            }
+        };
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    public class ArrayRemoveEval extends AbstractArrayAddRemoveEval {
+        private final ArrayBackedValueStorage storage;
+        private final IPointable item;
+        private final IBinaryComparator comp;
+
+        public ArrayRemoveEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, 0, 1, args.length - 1, argTypes, true, sourceLoc, false, false);
+            storage = new ArrayBackedValueStorage();
+            item = new VoidPointable();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        }
+
+        @Override
+        protected int getPosition(IFrameTupleReference tuple, IPointable listArg, ATypeTag listTag)
+                throws HyracksDataException {
+            return 0;
+        }
+
+        @Override
+        protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder, IPointable[] removed,
+                int position) throws IOException {
+            // get the list items one by one and append to the new list only if the list item is not in removed list
+            boolean addItem;
+            for (int i = 0; i < listAccessor.size(); i++) {
+                listAccessor.getOrWriteItem(i, item, storage);
+                addItem = true;
+                for (int j = 0; j < removed.length; j++) {
+                    if (comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(),
+                            removed[j].getByteArray(), removed[j].getStartOffset(), removed[j].getLength()) == 0) {
+                        addItem = false;
+                        break;
+                    }
+                }
+                if (addItem) {
+                    listBuilder.addItem(item);
+                }
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRepeatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRepeatDescriptor.java
new file mode 100644
index 0000000..5e50fe0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayRepeatDescriptor.java
@@ -0,0 +1,151 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_repeat(val, num_times) returns a new ordered list with the same item type as the input value.
+ *
+ * It throws an error at compile time if the number of arguments != 2
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if:
+ * - any argument is null
+ * - num_times is not numeric or it's a floating-point with decimals (3.2) or it's NaN/+-INF or it's negative.
+ * 3. otherwise, a new ordered list.
+ *
+ * </pre>
+ */
+public class ArrayRepeatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private AbstractCollectionType repeatedValueListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayRepeatDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        repeatedValueListType = new AOrderedListType((IAType) states[0], null);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_REPEAT;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayRepeatEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayRepeatEval implements IScalarEvaluator {
+        private final ArrayBackedValueStorage storage;
+        private final IScalarEvaluator repeatedValueEval;
+        private final IScalarEvaluator repeatEval;
+        private final IPointable repeatedValueArg;
+        private final IPointable repeatArg;
+        private final TaggedValuePointable repeatArgValue;
+        private final IAsterixListBuilder listBuilder;
+
+        public ArrayRepeatEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            repeatedValueEval = args[0].createScalarEvaluator(ctx);
+            repeatEval = args[1].createScalarEvaluator(ctx);
+            repeatedValueArg = new VoidPointable();
+            repeatArg = new VoidPointable();
+            repeatArgValue = new TaggedValuePointable();
+            listBuilder = new OrderedListBuilder();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            // 1st arg: value to repeat
+            repeatedValueEval.evaluate(tuple, repeatedValueArg);
+
+            // 2nd arg: number of repetitions
+            repeatEval.evaluate(tuple, repeatArg);
+            repeatArgValue.set(repeatArg);
+            if (!ATypeHierarchy.isCompatible(ATypeTag.DOUBLE, ATypeTag.VALUE_TYPE_MAPPING[repeatArgValue.getTag()])) {
+                PointableHelper.setNull(result);
+                return;
+            }
+            final String name = getIdentifier().getName();
+            final double repetitions =
+                    ATypeHierarchy.getDoubleValue(name, 1, repeatArg.getByteArray(), repeatArg.getStartOffset());
+
+            if (Double.isNaN(repetitions) || Double.isInfinite(repetitions) || Math.floor(repetitions) < repetitions
+                    || repetitions < 0) {
+                PointableHelper.setNull(result);
+                return;
+            }
+            // create list
+            listBuilder.reset(repeatedValueListType);
+            for (int i = 0; i < repetitions; ++i) {
+                listBuilder.addItem(repeatedValueArg);
+            }
+            storage.reset();
+            listBuilder.write(storage.getDataOutput(), true);
+            result.set(storage);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReplaceDescriptor.java
new file mode 100644
index 0000000..f591b54
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReplaceDescriptor.java
@@ -0,0 +1,254 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_replace(list, val1, val2, max_num_times?) returns a new list with the occurrences of val1 replaced with
+ * val2. max_num_times arg is optional. If supplied, it replaces val1 as many as max_num_times. Any negative number for
+ * max_num_times means "replace all occurrences". val2 can be null meaning you can replace existing items with nulls.
+ *
+ * array_replace([2,3,3,3,1], 3, 8, 0) will do nothing and result in [2,3,3,3,1].
+ *
+ * It throws an error at compile time if the number of arguments < 3 or > 4
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if:
+ * - any argument is null (except for val2).
+ * - input list is not a list.
+ * - num_times is not numeric or it's a floating-point number with decimals, e.g, 3.2 (3.0 is OK).
+ * 3. an error if val1 is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+    private IAType newValueType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayReplaceDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_REPLACE;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+        newValueType = (IAType) states[2];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayReplaceEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayReplaceEval implements IScalarEvaluator {
+        private final IScalarEvaluator listEval;
+        private final IScalarEvaluator targetValEval;
+        private final IScalarEvaluator newValEval;
+        private IScalarEvaluator maxEval;
+        private final IPointable list;
+        private final IPointable tempList;
+        private final IPointable target;
+        private final IPointable newVal;
+        private final IPointable tempVal;
+        private TaggedValuePointable maxArg;
+        private final AbstractPointable item;
+        private final ListAccessor listAccessor;
+        private final IBinaryComparator comp;
+        private final ArrayBackedValueStorage storage;
+        private final CastTypeEvaluator caster;
+        private IAsterixListBuilder orderedListBuilder;
+        private IAsterixListBuilder unorderedListBuilder;
+
+        public ArrayReplaceEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            listEval = args[0].createScalarEvaluator(ctx);
+            targetValEval = args[1].createScalarEvaluator(ctx);
+            newValEval = args[2].createScalarEvaluator(ctx);
+            if (args.length == 4) {
+                maxEval = args[3].createScalarEvaluator(ctx);
+                maxArg = new TaggedValuePointable();
+            }
+            list = new VoidPointable();
+            tempList = new VoidPointable();
+            target = new VoidPointable();
+            newVal = new VoidPointable();
+            tempVal = new VoidPointable();
+            item = new VoidPointable();
+            listAccessor = new ListAccessor();
+            caster = new CastTypeEvaluator();
+            orderedListBuilder = null;
+            unorderedListBuilder = null;
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            storage.reset();
+            listEval.evaluate(tuple, tempList);
+            targetValEval.evaluate(tuple, target);
+            newValEval.evaluate(tuple, tempVal);
+            ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(tempList.getByteArray()[tempList.getStartOffset()]);
+            ATypeTag targetTag = ATYPETAGDESERIALIZER.deserialize(target.getByteArray()[target.getStartOffset()]);
+            ATypeTag newValTag = ATYPETAGDESERIALIZER.deserialize(tempVal.getByteArray()[tempVal.getStartOffset()]);
+            if (listType == ATypeTag.MISSING || targetTag == ATypeTag.MISSING || newValTag == ATypeTag.MISSING) {
+                PointableHelper.setMissing(result);
+                return;
+            }
+
+            double maxDouble = -1;
+            String name = getIdentifier().getName();
+            if (maxEval != null) {
+                maxEval.evaluate(tuple, maxArg);
+                ATypeTag maxTag = ATYPETAGDESERIALIZER.deserialize(maxArg.getTag());
+                if (maxTag == ATypeTag.MISSING) {
+                    PointableHelper.setMissing(result);
+                    return;
+                } else if (!ATypeHierarchy.isCompatible(maxTag, ATypeTag.DOUBLE)) {
+                    PointableHelper.setNull(result);
+                    return;
+                }
+                maxDouble = ATypeHierarchy.getDoubleValue(name, 3, maxArg.getByteArray(), maxArg.getStartOffset());
+            }
+
+            if (!listType.isListType() || Math.floor(maxDouble) < maxDouble || targetTag == ATypeTag.NULL
+                    || Double.isInfinite(maxDouble) || Double.isNaN(maxDouble)) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            if (targetTag.isDerivedType()) {
+                throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLoc);
+            }
+
+            try {
+                IAType defaultOpenType = DefaultOpenFieldType.getDefaultOpenFieldType(listType);
+                caster.resetAndAllocate(defaultOpenType, inputListType, listEval);
+                caster.cast(tempList, list);
+
+                defaultOpenType = DefaultOpenFieldType.getDefaultOpenFieldType(newValTag);
+                if (defaultOpenType != null) {
+                    caster.resetAndAllocate(defaultOpenType, newValueType, newValEval);
+                    caster.cast(tempVal, newVal);
+                } else {
+                    newVal.set(tempVal);
+                }
+
+                int max = (int) maxDouble;
+                // create list
+                IAsterixListBuilder listBuilder;
+                if (listType == ATypeTag.ARRAY) {
+                    if (orderedListBuilder == null) {
+                        orderedListBuilder = new OrderedListBuilder();
+                    }
+                    listBuilder = orderedListBuilder;
+                } else {
+                    if (unorderedListBuilder == null) {
+                        unorderedListBuilder = new UnorderedListBuilder();
+                    }
+                    listBuilder = unorderedListBuilder;
+                }
+
+                listBuilder.reset((AbstractCollectionType) DefaultOpenFieldType.getDefaultOpenFieldType(listType));
+                listAccessor.reset(list.getByteArray(), list.getStartOffset());
+
+                int counter = 0;
+                byte[] targetBytes = target.getByteArray();
+                int offset = target.getStartOffset();
+                int length = target.getLength();
+                for (int i = 0; i < listAccessor.size(); i++) {
+                    listAccessor.getOrWriteItem(i, item, storage);
+                    if (counter != max && comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(),
+                            targetBytes, offset, length) == 0) {
+                        listBuilder.addItem(newVal);
+                        counter++;
+                    } else {
+                        listBuilder.addItem(item);
+                    }
+                }
+                storage.reset();
+                listBuilder.write(storage.getDataOutput(), true);
+                result.set(storage);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            } finally {
+                caster.deallocatePointables();
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReverseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReverseDescriptor.java
new file mode 100755
index 0000000..05a5c39
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayReverseDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * <pre>
+ * array_reverse(list) returns a new list with the entries of the original input list in reverse order.
+ * The returned list has the same type as the input list. The list can contain null/missing items. Both are preserved.
+ *
+ * It throws an error at compile time if the number of arguments != 1
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not a list.
+ * 3. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayReverseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayReverseDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            // the type of the input list is needed in order to use the same type for the new returned list
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_REVERSE;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayReverseEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayReverseEval extends AbstractArrayProcessEval {
+        private final ArrayBackedValueStorage storage;
+        private final AbstractPointable item;
+
+        public ArrayReverseEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, inputListType);
+            storage = new ArrayBackedValueStorage();
+            item = new VoidPointable();
+        }
+
+        @Override
+        protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder) throws IOException {
+            // get the list items in reverse and append to the new list
+            for (int i = listAccessor.size() - 1; i >= 0; i--) {
+                listAccessor.getOrWriteItem(i, item, storage);
+                listBuilder.addItem(item);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
new file mode 100755
index 0000000..b2cc7ba
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
@@ -0,0 +1,155 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * <pre>
+ * array_sort(list) returns a new list with the items sorted in ascending order. The returned list has the same type as
+ * the input list. The list can contain null and missing items, and both are preserved. It's case-sensitive to string
+ * items.
+ *
+ * It throws an error at compile time if the number of arguments != 1
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not a list.
+ * 3. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 4. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArraySortDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArraySortDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            // the type of the input list is needed in order to use the same type for the new returned list
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SORT;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArraySortEval(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+    protected class ArraySortComparator implements Comparator<IPointable> {
+        private final IBinaryComparator comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+        @Override
+        public int compare(IPointable val1, IPointable val2) {
+            try {
+                return comp.compare(val1.getByteArray(), val1.getStartOffset(), val1.getLength(), val2.getByteArray(),
+                        val2.getStartOffset(), val2.getLength());
+            } catch (HyracksDataException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+    }
+
+    public class ArraySortEval extends AbstractArrayProcessEval {
+        private final SourceLocation sourceLoc;
+        private final PriorityQueue<IPointable> sortedList;
+        private IPointable item;
+        private ArrayBackedValueStorage storage;
+
+        public ArraySortEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            super(args, ctx, inputListType);
+            this.sourceLoc = sourceLoc;
+            sortedList = new PriorityQueue<>(new ArraySortComparator());
+        }
+
+        @Override
+        protected void processList(ListAccessor listAccessor, IAsterixListBuilder listBuilder) throws IOException {
+            sortedList.clear();
+            boolean itemInStorage;
+            item = pointableAllocator.allocateEmpty();
+            storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+            for (int i = 0; i < listAccessor.size(); i++) {
+                itemInStorage = listAccessor.getOrWriteItem(i, item, storage);
+                if (ATYPETAGDESERIALIZER.deserialize(item.getByteArray()[item.getStartOffset()]).isDerivedType()) {
+                    throw new RuntimeDataException(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLoc);
+                }
+                sortedList.add(item);
+                if (itemInStorage) {
+                    storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
+                }
+                item = pointableAllocator.allocateEmpty();
+            }
+            while (!sortedList.isEmpty()) {
+                listBuilder.addItem(sortedList.poll());
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayStarDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayStarDescriptor.java
new file mode 100755
index 0000000..a247639
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayStarDescriptor.java
@@ -0,0 +1,323 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.builders.ArrayListFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * <pre>
+ * array_star(ordered_list) returns a new object. The input ordered list is supposed to be a list of objects:
+ * [{"id":1, "dept":"CS"}, {"id":2, "dept":"FIN"}, {"id":3, "dept":"CS"}]
+ * For the returned object, each field has a value = list of values of that specific field taken from each object in the
+ * input list.
+ *
+ * Ex1: array_star([{"a":1, "b":2}, {"a":9, "b":4}]) will produce: {"a":[1, 9], "b":[2, 4]}
+ * Ex2: array_star([{"a":1}, {"a":9, "b":4}]) will produce: {"a":[1, 9], "b":[null, 4]}
+ * Ex3: array_star([{"a":1, "c":5}, {"a":9, "b":4}]) will produce: {"a":[1, 9], "b":[null, 4], "c":[5,null]}
+ * Ex4: array_star([{"c":5, "a":1}, "non_object"]) will produce: {"a":[1, null], "c":[5,null]}
+ * Ex5: array_star(["non_object1", "non_object2"]) will produce: {} (i.e., missing)
+ *
+ * Note that in the final object result, the fields are ordered by their names regardless of their original order in the
+ * object items in the input list. "a" comes before "c". However, for every field, all the items in each list must not
+ * be ordered. They should appear in the sequence they appear in the input list.
+ * For Ex1, "a":[1,9] in the final result, item at index 0 comes from object at index 0 (which is 1).
+ *
+ * It throws an error at compile time if the number of arguments != 1
+ *
+ * It returns in order:
+ * 1. missing, if any argument is missing.
+ * 2. null, if the list arg is null or it's not an ordered list.
+ * 3. missing, if input list is missing the notion of fields.
+ *    E.g., the input list contains no object items (e.g., list of int), or all objects have no fields.
+ * 4. otherwise, a new object.
+ *
+ * </pre>
+ */
+public class ArrayStarDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayStarDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    private IAType inputListType;
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_STAR;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayStarEval(args, ctx);
+            }
+        };
+    }
+
+    public class UTF8StringComparator implements Comparator<IValueReference> {
+        private final IBinaryComparator comp = PointableHelper.createStringBinaryComparator();
+
+        @Override
+        public int compare(IValueReference val1, IValueReference val2) {
+            try {
+                return PointableHelper.compareStringBinValues(val1, val2, comp);
+            } catch (HyracksDataException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+    }
+
+    protected class FieldNameToValues implements IValueReference {
+        private IVisitablePointable fieldName;
+        private List<IVisitablePointable> values;
+
+        @Override
+        public byte[] getByteArray() {
+            return fieldName.getByteArray();
+        }
+
+        @Override
+        public int getStartOffset() {
+            return fieldName.getStartOffset();
+        }
+
+        @Override
+        public int getLength() {
+            return fieldName.getLength();
+        }
+    }
+
+    protected class FieldNameToValuesAllocator implements IObjectFactory<FieldNameToValues, ATypeTag> {
+
+        @Override
+        public FieldNameToValues create(ATypeTag arg) {
+            return new FieldNameToValues();
+        }
+    }
+
+    public class ArrayStarEval implements IScalarEvaluator {
+        private final IBinaryComparator binaryStrComp = PointableHelper.createStringBinaryComparator();
+        private final UTF8StringComparator comp = new UTF8StringComparator();
+        private final ArrayBackedValueStorage storage;
+        private final IScalarEvaluator listEval;
+        private final IPointable list;
+        private final IPointable tempList;
+        private final IPointable object;
+        private final CastTypeEvaluator caster;
+        private final ListAccessor listAccessor;
+        private final RecordBuilder recordBuilder;
+        private final IAsterixListBuilder listBuilder;
+        private final PointableAllocator pointableAllocator;
+        private final List<FieldNameToValues> fieldNameToValuesList;
+        private final PriorityQueue<FieldNameToValues> tempMinHeap;
+        private final IObjectPool<List<IVisitablePointable>, ATypeTag> arrayListAllocator;
+        private final IObjectPool<FieldNameToValues, ATypeTag> fieldNameToValuesAllocator;
+
+        public ArrayStarEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            object = new VoidPointable();
+            list = new VoidPointable();
+            tempList = new VoidPointable();
+            listEval = args[0].createScalarEvaluator(ctx);
+            caster = new CastTypeEvaluator();
+            listAccessor = new ListAccessor();
+            recordBuilder = new RecordBuilder();
+            listBuilder = new OrderedListBuilder();
+            pointableAllocator = new PointableAllocator();
+            fieldNameToValuesList = new ArrayList<>();
+            tempMinHeap = new PriorityQueue<>(comp);
+            arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
+            fieldNameToValuesAllocator = new ListObjectPool<>(new FieldNameToValuesAllocator());
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+            storage.reset();
+            listEval.evaluate(tuple, tempList);
+            ATypeTag listTag = ATYPETAGDESERIALIZER.deserialize(tempList.getByteArray()[tempList.getStartOffset()]);
+            if (listTag != ATypeTag.ARRAY) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            try {
+                caster.resetAndAllocate(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE, inputListType, listEval);
+                caster.cast(tempList, list);
+
+                tempMinHeap.clear();
+                fieldNameToValuesList.clear();
+                listAccessor.reset(list.getByteArray(), list.getStartOffset());
+                int numObjects = listAccessor.size();
+
+                for (int objectIndex = 0; objectIndex < numObjects; objectIndex++) {
+                    listAccessor.getOrWriteItem(objectIndex, object, storage);
+                    processObject(object, objectIndex, numObjects);
+                }
+
+                if (fieldNameToValuesList.isEmpty()) {
+                    PointableHelper.setMissing(result);
+                    return;
+                }
+                for (int i = 0; i < fieldNameToValuesList.size(); i++) {
+                    tempMinHeap.add(fieldNameToValuesList.get(i));
+                }
+                recordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+                recordBuilder.init();
+
+                FieldNameToValues fieldNameToValues;
+                IVisitablePointable oneValue;
+                while (!tempMinHeap.isEmpty()) {
+                    fieldNameToValues = tempMinHeap.poll();
+                    listBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE);
+                    for (int k = 0; k < fieldNameToValues.values.size(); k++) {
+                        oneValue = fieldNameToValues.values.get(k);
+                        if (oneValue == null) {
+                            listBuilder.addItem(PointableHelper.NULL_REF);
+                        } else {
+                            listBuilder.addItem(oneValue);
+                        }
+                    }
+                    storage.reset();
+                    listBuilder.write(storage.getDataOutput(), true);
+                    recordBuilder.addField(fieldNameToValues.fieldName, storage);
+                }
+
+                storage.reset();
+                recordBuilder.write(storage.getDataOutput(), true);
+                result.set(storage);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            } finally {
+                pointableAllocator.reset();
+                arrayListAllocator.reset();
+                fieldNameToValuesAllocator.reset();
+                caster.deallocatePointables();
+            }
+        }
+
+        private void processObject(IPointable object, int objectIndex, int numObjects) throws HyracksDataException {
+            ARecordVisitablePointable record;
+            // process only objects (records)
+            if (object.getByteArray()[object.getStartOffset()] == ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+                record = pointableAllocator.allocateRecordValue(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+                record.set(object.getByteArray(), object.getStartOffset(), object.getLength());
+
+                List<IVisitablePointable> fieldNames = record.getFieldNames();
+                List<IVisitablePointable> fieldValues = record.getFieldValues();
+                List<IVisitablePointable> values;
+                IVisitablePointable fieldName;
+                for (int j = 0; j < fieldNames.size(); j++) {
+                    fieldName = fieldNames.get(j);
+                    FieldNameToValues fieldNameToValues = findField(fieldName, fieldNameToValuesList, binaryStrComp);
+
+                    if (fieldNameToValues == null) {
+                        // new field name
+                        fieldNameToValues = fieldNameToValuesAllocator.allocate(null);
+                        values = arrayListAllocator.allocate(null);
+                        clear(values, numObjects);
+                        fieldNameToValues.fieldName = fieldName;
+                        fieldNameToValues.values = values;
+                        fieldNameToValuesList.add(fieldNameToValues);
+                    } else {
+                        // field name already exists, get the values vector
+                        values = fieldNameToValues.values;
+                    }
+                    values.set(objectIndex, fieldValues.get(j));
+                }
+            }
+        }
+
+        private void clear(List<IVisitablePointable> values, int numObjects) {
+            values.clear();
+            for (int i = 1; i <= numObjects; i++) {
+                values.add(null);
+            }
+        }
+
+        private FieldNameToValues findField(IVisitablePointable fieldName, List<FieldNameToValues> fieldNamesList,
+                IBinaryComparator strComp) throws HyracksDataException {
+            FieldNameToValues anotherFieldName;
+            for (int i = 0; i < fieldNamesList.size(); i++) {
+                anotherFieldName = fieldNamesList.get(i);
+                if (PointableHelper.isEqual(fieldName, anotherFieldName.fieldName, strComp)) {
+                    return anotherFieldName;
+                }
+            }
+            return null;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffDescriptor.java
new file mode 100755
index 0000000..f9e68df
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffDescriptor.java
@@ -0,0 +1,91 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * <pre>
+ * array_symdiff(list1, list2, ...) returns a new list based on the set symmetric difference, or disjunctive union,
+ * of the input. The new list contains only those items that appear in exactly one of the input lists.
+ * array_symdiff([null, 2,3], [missing, 3]) will result in [2, null, null] where one null is for the missing item
+ * and the second null for the null item.
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. an error if the input lists are not of the same type (one is an ordered list while the other is unordered).
+ * 3. null, if any input list is null or is not a list.
+ * 4. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 5. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArraySymDiffDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArraySymDiffDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SYMDIFF;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArraySymDiffEval(args, ctx, sourceLoc, argTypes);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
new file mode 100755
index 0000000..d649c48
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
@@ -0,0 +1,173 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.util.List;
+
+import org.apache.asterix.builders.ArrayListFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.common.arraylist.IntArrayList;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
+public class ArraySymDiffEval extends AbstractArrayProcessArraysEval {
+    private final IBinaryHashFunction binaryHashFunction;
+    private final Int2ObjectMap<List<ValueCounter>> hashes;
+    private final IObjectPool<List<ValueCounter>, ATypeTag> arrayListAllocator;
+    private final IObjectPool<ValueCounter, ATypeTag> valueCounterAllocator;
+    private final IBinaryComparator comp;
+    private final IntArrayList intHashes;
+
+    public ArraySymDiffEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLocation,
+            IAType[] argTypes) throws HyracksDataException {
+        super(args, ctx, true, sourceLocation, argTypes);
+        arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
+        valueCounterAllocator = new ListObjectPool<>(new ValueCounterFactory());
+        hashes = new Int2ObjectOpenHashMap<>();
+        comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        intHashes = new IntArrayList(50, 10);
+        binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
+                .createBinaryHashFunction();
+    }
+
+    protected class ValueCounter implements IValueReference {
+        private IPointable value;
+        private int listIndex;
+        private int counter;
+
+        protected ValueCounter() {
+        }
+
+        protected void reset(IPointable value, int listIndex, int counter) {
+            this.value = value;
+            this.listIndex = listIndex;
+            this.counter = counter;
+        }
+
+        @Override
+        public byte[] getByteArray() {
+            return value.getByteArray();
+        }
+
+        @Override
+        public int getStartOffset() {
+            return value.getStartOffset();
+        }
+
+        @Override
+        public int getLength() {
+            return value.getLength();
+        }
+    }
+
+    public class ValueCounterFactory implements IObjectFactory<ValueCounter, ATypeTag> {
+
+        @Override
+        public ValueCounter create(ATypeTag arg) {
+            return new ValueCounter();
+        }
+    }
+
+    @Override
+    protected void init() {
+        hashes.clear();
+        intHashes.clear();
+    }
+
+    @Override
+    protected void finish(IAsterixListBuilder listBuilder) throws HyracksDataException {
+        ValueCounter item;
+        List<ValueCounter> items;
+        // TODO(ali): temp solution to avoid iterator object creation, find a better way
+        for (int i = 0; i < intHashes.size(); i++) {
+            items = hashes.get(intHashes.get(i));
+            for (int k = 0; k < items.size(); k++) {
+                item = items.get(k);
+                if (checkCounter(item.counter)) {
+                    listBuilder.addItem(item.value);
+                }
+            }
+        }
+    }
+
+    @Override
+    protected void release() {
+        arrayListAllocator.reset();
+        valueCounterAllocator.reset();
+    }
+
+    protected boolean checkCounter(int counter) {
+        return counter == 1;
+    }
+
+    @Override
+    protected boolean processItem(IPointable item, int listIndex, IAsterixListBuilder listBuilder)
+            throws HyracksDataException {
+        // lookup the item
+        int hash = binaryHashFunction.hash(item.getByteArray(), item.getStartOffset(), item.getLength());
+        List<ValueCounter> sameHashes = hashes.get(hash);
+        if (sameHashes == null) {
+            // new item
+            sameHashes = arrayListAllocator.allocate(null);
+            sameHashes.clear();
+            addItem(item, listIndex, sameHashes);
+            hashes.put(hash, sameHashes);
+            intHashes.add(hash);
+            return true;
+        } else {
+            // potentially, item already exists
+            ValueCounter itemListIdxCounter = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
+            if (itemListIdxCounter == null) {
+                // new item having the same hash as a different item
+                addItem(item, listIndex, sameHashes);
+                return true;
+            }
+            // the item already exists, increment the counter (don't increment the counter for the same listIndex)
+            if (itemListIdxCounter.listIndex != listIndex) {
+                itemListIdxCounter.listIndex = listIndex;
+                itemListIdxCounter.counter++;
+            }
+            // false, since we didn't add (use) the item
+            return false;
+        }
+    }
+
+    private void addItem(IPointable item, int listIndex, List<ValueCounter> sameHashes) {
+        ValueCounter valueCounter = valueCounterAllocator.allocate(null);
+        valueCounter.reset(item, listIndex, 1);
+        sameHashes.add(valueCounter);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffnDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffnDescriptor.java
new file mode 100755
index 0000000..0cafb1b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffnDescriptor.java
@@ -0,0 +1,103 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * <pre>
+ * array_symdiffn(list1, list2, ...) returns a new list based on the set symmetric difference, or disjunctive
+ * union, of the input lists. The new list contains only those items that appear in an odd number of input lists.
+ * array_symdiffn([null, 2,3], [missing, 3]) will result in [2, null, null] where one null is for the missing item
+ * and the second null for the null item.
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. an error if the input lists are not of the same type (one is an ordered list while the other is unordered).
+ * 3. null, if any input list is null or is not a list.
+ * 4. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 5. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArraySymDiffnDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArraySymDiffnDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SYMDIFFN;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArraySymDiffnEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArraySymDiffnEval extends ArraySymDiffEval {
+
+        public ArraySymDiffnEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, sourceLoc, argTypes);
+        }
+
+        @Override
+        protected boolean checkCounter(int counter) {
+            return counter % 2 != 0;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
new file mode 100755
index 0000000..d6a2f1f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
@@ -0,0 +1,164 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.util.List;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.asterix.builders.ArrayListFactory;
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * <pre>
+ * array_union(list1, list2, ...) returns a new list with the set union of the input lists (no duplicates).
+ * Items of the lists can be null or missing (both are added as a null value).
+ * array_union([null, 2], [missing, 3, null]) will result in [null, 2, null, 3] where one null is for the missing item
+ * and the second null for the null item.
+ *
+ * It throws an error at compile time if the number of arguments < 2
+ *
+ * It returns (or throws an error at runtime) in order:
+ * 1. missing, if any argument is missing.
+ * 2. an error if the input lists are not of the same type (one is an ordered list while the other is unordered).
+ * 3. null, if any input list is null or is not a list.
+ * 4. an error if any list item is a list/object type (i.e. derived type) since deep equality is not yet supported.
+ * 5. otherwise, a new list.
+ *
+ * </pre>
+ */
+public class ArrayUnionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayUnionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_UNION;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayUnionEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayUnionEval extends AbstractArrayProcessArraysEval {
+        private final IObjectPool<List<IPointable>, ATypeTag> pointableListAllocator;
+        private final IBinaryHashFunction binaryHashFunction;
+        private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
+
+        public ArrayUnionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+            super(args, ctx, true, sourceLoc, argTypes);
+            pointableListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
+            hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
+                    .createBinaryHashFunction();
+        }
+
+        @Override
+        protected void init() {
+            hashes.clear();
+        }
+
+        @Override
+        protected void finish(IAsterixListBuilder listBuilder) {
+            // do nothing
+        }
+
+        @Override
+        protected void release() {
+            pointableListAllocator.reset();
+        }
+
+        @Override
+        protected boolean processItem(IPointable item, int listIndex, IAsterixListBuilder listBuilder)
+                throws HyracksDataException {
+            int hash = binaryHashFunction.hash(item.getByteArray(), item.getStartOffset(), item.getLength());
+            List<IPointable> sameHashes = hashes.get(hash);
+            if (sameHashes == null) {
+                // new item
+                sameHashes = pointableListAllocator.allocate(null);
+                sameHashes.clear();
+                addItem(listBuilder, item, sameHashes);
+                hashes.put(hash, sameHashes);
+                return true;
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
+                // new item, it could happen that two hashes are the same but they are for different items
+                addItem(listBuilder, item, sameHashes);
+                return true;
+            }
+            // else ignore since the item already exists
+            return false;
+        }
+
+        private void addItem(IAsterixListBuilder listBuilder, IPointable item, List<IPointable> sameHashes)
+                throws HyracksDataException {
+            listBuilder.addItem(item);
+            sameHashes.add(item);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
index 524b2ed..b9f693b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeEvaluator.java
@@ -33,24 +33,31 @@
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class CastTypeEvaluator implements IScalarEvaluator {
+public class CastTypeEvaluator implements IScalarEvaluator {
 
-    private final IScalarEvaluator argEvaluator;
+    private IScalarEvaluator argEvaluator;
     private final IPointable argPointable = new VoidPointable();
-
     private final PointableAllocator allocator = new PointableAllocator();
-    private final IVisitablePointable inputPointable;
-    private final IVisitablePointable resultPointable;
+    private IVisitablePointable inputPointable;
+    private IVisitablePointable resultPointable;
+    private final ACastVisitor castVisitor = createCastVisitor();
+    private final Triple<IVisitablePointable, IAType, Boolean> arg = new Triple<>(null, null, null);
 
-    private final ACastVisitor castVisitor;
-    private final Triple<IVisitablePointable, IAType, Boolean> arg;
+    public CastTypeEvaluator() {
+        // reset() should be called after using this constructor before calling any method
+    }
 
     public CastTypeEvaluator(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator) {
+        resetAndAllocate(reqType, inputType, argEvaluator);
+    }
+
+    public void resetAndAllocate(IAType reqType, IAType inputType, IScalarEvaluator argEvaluator) {
         this.argEvaluator = argEvaluator;
         this.inputPointable = allocatePointable(inputType, reqType);
         this.resultPointable = allocatePointable(reqType, inputType);
-        this.arg = new Triple<>(resultPointable, reqType, Boolean.FALSE);
-        this.castVisitor = createCastVisitor();
+        this.arg.first = resultPointable;
+        this.arg.second = reqType;
+        this.arg.third = Boolean.FALSE;
     }
 
     protected ACastVisitor createCastVisitor() {
@@ -69,8 +76,14 @@
         result.set(resultPointable);
     }
 
+    // TODO: refactor in a better way
+    protected void cast(IPointable argPointable, IPointable result) throws HyracksDataException {
+        inputPointable.set(argPointable);
+        cast(result);
+    }
+
     // Allocates the result pointable.
-    private final IVisitablePointable allocatePointable(IAType typeForPointable, IAType typeForOtherSide) {
+    private IVisitablePointable allocatePointable(IAType typeForPointable, IAType typeForOtherSide) {
         if (!typeForPointable.equals(BuiltinType.ANY)) {
             return allocator.allocateFieldValue(typeForPointable);
         }
@@ -91,4 +104,8 @@
                 return allocator.allocateFieldValue(null);
         }
     }
+
+    public void deallocatePointables() {
+        allocator.reset();
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
index 1e65dc0..7f90973 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
@@ -68,7 +68,7 @@
 
                         if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
                                 || data[offset] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
-                            throw new UnsupportedTypeException(getIdentifier(), data[offset]);
+                            throw new UnsupportedTypeException(sourceLoc, getIdentifier(), data[offset]);
                         }
                         result.set(inputArg);
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
index a59d03d..dbe2e05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CodePointToStringDescriptor.java
@@ -95,7 +95,8 @@
                                                 offset);
                                         break;
                                     default:
-                                        throw new UnsupportedTypeException(getIdentifier(), serOrderedList[offset]);
+                                        throw new UnsupportedTypeException(sourceLoc, getIdentifier(),
+                                                serOrderedList[offset]);
                                 }
                             }
                             // calculate length first
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
index 132c99e..8d49ae8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
@@ -91,11 +91,11 @@
 
                         // Type check: (point, double)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
index 6f79f97..9d49466 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
@@ -90,11 +90,11 @@
 
                         // type-check: (point, point)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index d61d6f2..71901c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -84,11 +84,11 @@
 
                         // type-check: (double, double)
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
index 74f8757..bdb8b94 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
@@ -89,7 +89,7 @@
 
                             if (listBytes[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && listBytes[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, listBytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[offset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
@@ -107,16 +107,16 @@
                                         missingSerde.serialize(AMissing.MISSING, out);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(BuiltinFunctions.CREATE_POLYGON,
+                                    throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.CREATE_POLYGON,
                                             itemType.serialize());
                                 }
 
                             }
                             if (listAccessor.size() < 6) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             } else if (listAccessor.size() % 2 != 0) {
-                                throw new InvalidDataFormatException(getIdentifier(),
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                         ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                             }
                             out.writeByte(ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
index ad874f5..b758845 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
@@ -90,11 +90,11 @@
 
                         resultStorage.reset();
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index 0c55ef1..70b0891 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -44,6 +45,7 @@
 public class GetItemDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -61,14 +63,11 @@
         return BuiltinFunctions.GET_ITEM;
     }
 
-    private static class GetItemEvalFactory implements IScalarEvaluatorFactory {
+    private class GetItemEvalFactory implements IScalarEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory listEvalFactory;
         private IScalarEvaluatorFactory indexEvalFactory;
-        private byte serItemTypeTag;
-        private ATypeTag itemTag;
-        private boolean selfDescList = false;
 
         public GetItemEvalFactory(IScalarEvaluatorFactory[] args) {
             this.listEvalFactory = args[0];
@@ -86,9 +85,6 @@
                 private IScalarEvaluator evalList = listEvalFactory.createScalarEvaluator(ctx);
                 private IScalarEvaluator evalIdx = indexEvalFactory.createScalarEvaluator(ctx);
                 private byte[] missingBytes = new byte[] { ATypeTag.SERIALIZED_MISSING_TYPE_TAG };
-                private int itemIndex;
-                private int itemOffset;
-                private int itemLength;
 
                 @Override
                 public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -96,48 +92,51 @@
                         evalList.evaluate(tuple, inputArgList);
                         evalIdx.evaluate(tuple, inputArgIdx);
 
-                        byte[] serOrderedList = inputArgList.getByteArray();
+                        byte[] serList = inputArgList.getByteArray();
                         int offset = inputArgList.getStartOffset();
                         byte[] indexBytes = inputArgIdx.getByteArray();
                         int indexOffset = inputArgIdx.getStartOffset();
 
-                        if (serOrderedList[offset] == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                            itemIndex = ATypeHierarchy.getIntegerValue(BuiltinFunctions.GET_ITEM.getName(), 0,
-                                    indexBytes, indexOffset);
+                        int itemCount;
+                        byte serListTypeTag = serList[offset];
+                        if (serListTypeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+                            itemCount = AOrderedListSerializerDeserializer.getNumberOfItems(serList, offset);
+                        } else if (serListTypeTag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+                            itemCount = AUnorderedListSerializerDeserializer.getNumberOfItems(serList, offset);
                         } else {
-                            throw new TypeMismatchException(BuiltinFunctions.GET_ITEM, 0, serOrderedList[offset],
-                                    ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
+                            throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_ITEM, 0, serListTypeTag,
+                                    ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+                                    ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                         }
 
-                        if (itemIndex < 0 || itemIndex >= AOrderedListSerializerDeserializer
-                                .getNumberOfItems(serOrderedList, offset)) {
+                        int itemIndex = ATypeHierarchy.getIntegerValue(BuiltinFunctions.GET_ITEM.getName(), 0,
+                                indexBytes, indexOffset);
+
+                        if (itemIndex < 0 || itemIndex >= itemCount) {
                             // Out-of-bound index access should return MISSING.
                             result.set(missingBytes, 0, 1);
                             return;
                         }
 
-                        itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + 1]);
-                        if (itemTag == ATypeTag.ANY) {
-                            selfDescList = true;
-                        } else {
-                            serItemTypeTag = serOrderedList[offset + 1];
-                        }
+                        byte serItemTypeTag = serList[offset + 1];
+                        ATypeTag itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serItemTypeTag);
+                        boolean selfDescList = itemTag == ATypeTag.ANY;
 
-                        itemOffset =
-                                AOrderedListSerializerDeserializer.getItemOffset(serOrderedList, offset, itemIndex);
+                        int itemOffset = serListTypeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
+                                ? AOrderedListSerializerDeserializer.getItemOffset(serList, offset, itemIndex)
+                                : AUnorderedListSerializerDeserializer.getItemOffset(serList, offset, 0);
 
                         if (selfDescList) {
-                            itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[itemOffset]);
-                            itemLength =
-                                    NonTaggedFormatUtil.getFieldValueLength(serOrderedList, itemOffset, itemTag, true)
-                                            + 1;
-                            result.set(serOrderedList, itemOffset, itemLength);
+                            itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serList[itemOffset]);
+                            int itemLength =
+                                    NonTaggedFormatUtil.getFieldValueLength(serList, itemOffset, itemTag, true) + 1;
+                            result.set(serList, itemOffset, itemLength);
                         } else {
-                            itemLength =
-                                    NonTaggedFormatUtil.getFieldValueLength(serOrderedList, itemOffset, itemTag, false);
+                            int itemLength =
+                                    NonTaggedFormatUtil.getFieldValueLength(serList, itemOffset, itemTag, false);
                             resultStorage.reset();
                             output.writeByte(serItemTypeTag);
-                            output.write(serOrderedList, itemOffset, itemLength);
+                            output.write(serList, itemOffset, itemLength);
                             result.set(resultStorage);
                         }
                     } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
index 17f7a96..f4ba646 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
@@ -50,7 +50,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractUnaryStringStringEval(ctx, args[0],
-                        GetJobParameterByNameDescriptor.this.getIdentifier()) {
+                        GetJobParameterByNameDescriptor.this.getIdentifier(), sourceLoc) {
                     private byte[] result;
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
index 25b6c5d9..df875a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
@@ -82,7 +82,7 @@
                                     argPtr.getStartOffset() + 1);
                             if (argResult) {
                                 LOGGER.log(Level.ERROR, ctx.getTaskAttemptId() + " injecting failure");
-                                throw new RuntimeDataException(ErrorCode.INJECTED_FAILURE, getIdentifier());
+                                throw new RuntimeDataException(ErrorCode.INJECTED_FAILURE, sourceLoc, getIdentifier());
                             }
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
index 7bae18b..cbae79e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -83,7 +83,7 @@
 
                         if (serList[offset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && serList[offset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, serList[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serList[offset],
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
index 13037a9..7c9736d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -86,7 +86,7 @@
                             ABoolean aResult = argRes ? ABoolean.FALSE : ABoolean.TRUE;
                             booleanSerde.serialize(aResult, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                         }
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
index a7ba7b0..a0c9ae6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
@@ -59,7 +59,7 @@
 
         NumericACosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericACosDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericACosDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
index 02c5896..f20736c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericASinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericASinDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericASinDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
index d5a5c21..2939f49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericATanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericATanDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericATanDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
index d153a69..67d3e4d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
@@ -64,7 +64,7 @@
 
         NumericAbsEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericAbsDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericAbsDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index 3f52a47..9b8ffeb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -18,11 +18,14 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -42,23 +45,34 @@
     }
 
     @Override
-    protected long evaluateInteger(long x, long y) throws HyracksDataException {
-        return Math.addExact(x, y);
+    protected boolean evaluateInteger(long x, long y, AMutableInt64 result) throws HyracksDataException {
+        try {
+            long res = Math.addExact(x, y);
+            result.setValue(res);
+            return true;
+        } catch (ArithmeticException e) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
     }
 
     @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return lhs + rhs;
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) throws HyracksDataException {
+        double res = lhs + rhs;
+        result.setValue(res);
+        return true;
     }
 
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) throws HyracksDataException {
+        long res = DurationArithmeticOperations.addDuration(chronon, yearMonth, dayTime, isTimeOnly);
+        result.setValue(res);
+        return true;
+    }
+
+    @Override
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
             throws HyracksDataException {
-        return DurationArithmeticOperations.addDuration(chronon, yearMonth, dayTime, isTimeOnly);
-    }
-
-    @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
deleted file mode 100644
index 0079a8a..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.asterix.runtime.evaluators.functions;
-
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.exceptions.OverflowException;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.google.common.math.LongMath;
-
-public class NumericCaretDescriptor extends AbstractNumericArithmeticEval {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new NumericCaretDescriptor();
-        }
-    };
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
-     */
-    @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        if (rhs > Integer.MAX_VALUE) {
-            throw new OverflowException(getIdentifier());
-        }
-        return LongMath.checkedPow(lhs, (int) rhs);
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
-     */
-    @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return Math.pow(lhs, rhs);
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
-     */
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return BuiltinFunctions.CARET;
-    }
-
-    @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
-            throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
-    }
-
-    @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
-    }
-
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
index 5d0eebb..407eb4d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericCeilingEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericCeilingDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericCeilingDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
index 7ef6bd4..1122bbe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericCosEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericCosDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericCosDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
new file mode 100644
index 0000000..6539e20
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericDegreesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericDegreesDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+                        aDouble.setValue(Math.toDegrees(arg));
+                        serialize(aDouble, doubleSerde, resultPointable);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_DEGREES;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
new file mode 100644
index 0000000..2eeefe8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
@@ -0,0 +1,82 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NumericDivDescriptor extends AbstractNumericArithmeticEval {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericDivDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_DIV;
+    }
+
+    @Override
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) {
+        if (rhs == 0) {
+            return false; // result = NULL
+        }
+        double res = lhs / rhs;
+        result.setValue(res);
+        return true;
+    }
+
+    @Override
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException {
+        if (rhs == 0) {
+            return false; // result = NULL
+        }
+        if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
+        long res = lhs / rhs;
+        result.setValue(res);
+        return true;
+    }
+
+    @Override
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) {
+        throw new NotImplementedException("Divide operation is not defined for temporal types");
+    }
+
+    @Override
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
+            throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 77c94bf..6e6b5a8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -18,21 +18,20 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class NumericDivideDescriptor extends AbstractNumericArithmeticEval {
-
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NumericDivideDescriptor();
@@ -45,29 +44,34 @@
     }
 
     @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+    protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+        return argTypeMax.ordinal() < ATypeTag.FLOAT.ordinal() ? ATypeTag.DOUBLE : argTypeMax;
+    }
+
+    @Override
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) {
         if (rhs == 0) {
-            throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
+            return false; // result = NULL
         }
-        if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
-            throw new OverflowException(getIdentifier());
-        }
-        return lhs / rhs;
+        double res = lhs / rhs;
+        result.setValue(res);
+        return true;
     }
 
     @Override
-    protected double evaluateDouble(double lhs, double rhs) {
-        return lhs / rhs;
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) {
+        throw new IllegalStateException();
     }
 
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
-            throws HyracksDataException {
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) {
         throw new NotImplementedException("Divide operation is not defined for temporal types");
     }
 
     @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
+            throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
index c5aeaa1..3a51e48 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericExpEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericExpDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericExpDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
index fd001d8..7f1b9bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
@@ -64,7 +64,7 @@
 
         NumericFloorEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericFloorDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericFloorDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
index 1e9d2be..8cc3327 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericLnEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericLnDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericLnDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
index f5a35f6..8867ea7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericLogEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericLogDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericLogDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
index 777e447..e737841 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -43,23 +45,34 @@
     }
 
     @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        return lhs % rhs;
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException {
+        if (rhs == 0) {
+            return false; // result = NULL
+        }
+        long res = lhs % rhs;
+        result.setValue(res);
+        return true;
     }
 
     @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return lhs % rhs;
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) throws HyracksDataException {
+        if (rhs == 0) {
+            return false; // result = NULL
+        }
+        double res = lhs % rhs;
+        result.setValue(res);
+        return true;
     }
 
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+    }
+
+    @Override
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
             throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
-    }
-
-    @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index d907bf3..64fa2b2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -41,23 +44,32 @@
     }
 
     @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        return Math.multiplyExact(lhs, rhs);
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException {
+        try {
+            long res = Math.multiplyExact(lhs, rhs);
+            result.setValue(res);
+            return true;
+        } catch (ArithmeticException e) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
     }
 
     @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return lhs * rhs;
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) throws HyracksDataException {
+        double res = lhs * rhs;
+        result.setValue(res);
+        return true;
     }
 
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+    }
+
+    @Override
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
             throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
-    }
-
-    @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
new file mode 100644
index 0000000..68ed6e8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
@@ -0,0 +1,90 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.google.common.math.LongMath;
+
+public class NumericPowerDescriptor extends AbstractNumericArithmeticEval {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericPowerDescriptor();
+        }
+    };
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
+     */
+    @Override
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException {
+        if (rhs > Integer.MAX_VALUE) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
+        try {
+            long res = LongMath.checkedPow(lhs, (int) rhs);
+            result.setValue(res);
+            return true;
+        } catch (ArithmeticException e) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
+     */
+    @Override
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) throws HyracksDataException {
+        double res = Math.pow(lhs, rhs);
+        result.setValue(res);
+        return true;
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+     */
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_POWER;
+    }
+
+    @Override
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+    }
+
+    @Override
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
+            throws HyracksDataException {
+        throw new UnsupportedTypeException(sourceLoc, getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
new file mode 100644
index 0000000..a0c1a33
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericRadiansDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new NumericRadiansDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+                        aDouble.setValue(Math.toRadians(arg));
+                        serialize(aDouble, doubleSerde, resultPointable);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.NUMERIC_RADIANS;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
index 0ad0dbe..e355a05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundDescriptor.java
@@ -140,7 +140,7 @@
                             aDouble.setValue(val);
                             serde.serialize(aDouble, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
index 926ef49..00e7415 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEven2Descriptor.java
@@ -110,7 +110,7 @@
                         } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
                             return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
@@ -176,7 +176,7 @@
                                 serde.serialize(aDouble, out);
                             }
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
index 1d45b3c..ecb62e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
@@ -138,7 +138,7 @@
                             aDouble.setValue(Math.rint(val));
                             serde.serialize(aDouble, out);
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
                                     ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                     ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                     ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
index 392dc0a..fcffdb8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSignEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSignDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSignDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
index 9b55844..08ddf71 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSinEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSinDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSinDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
index 2715639..cd48626 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericSqrtEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericSqrtDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericSqrtDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
index f694226..30f6dda 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -43,33 +46,43 @@
      * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
      */
     @Override
-    protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
-        return Math.subtractExact(lhs, rhs);
+    protected boolean evaluateInteger(long lhs, long rhs, AMutableInt64 result) throws HyracksDataException {
+        try {
+            long res = Math.subtractExact(lhs, rhs);
+            result.setValue(res);
+            return true;
+        } catch (ArithmeticException e) {
+            throw new OverflowException(sourceLoc, getIdentifier());
+        }
     }
 
     /* (non-Javadoc)
      * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
      */
     @Override
-    protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
-        return lhs - rhs;
+    protected boolean evaluateDouble(double lhs, double rhs, AMutableDouble result) throws HyracksDataException {
+        double res = lhs - rhs;
+        result.setValue(res);
+        return true;
     }
 
     /* (non-Javadoc)
      * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateTimeDurationArithmetic(long, int, long, boolean)
      */
     @Override
-    protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
-            throws HyracksDataException {
-        return DurationArithmeticOperations.addDuration(chronon, -1 * yearMonth, -1 * dayTime, isTimeOnly);
+    protected boolean evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly,
+            AMutableInt64 result) throws HyracksDataException {
+        long res = DurationArithmeticOperations.addDuration(chronon, -1 * yearMonth, -1 * dayTime, isTimeOnly);
+        result.setValue(res);
+        return true;
     }
 
     /* (non-Javadoc)
      * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateTimeInstanceArithmetic(long, long)
      */
     @Override
-    protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
-        return evaluateInteger(chronon0, chronon1);
+    protected boolean evaluateTimeInstanceArithmetic(long chronon0, long chronon1, AMutableInt64 result)
+            throws HyracksDataException {
+        return evaluateInteger(chronon0, chronon1, result);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
index 427e6eb..10b97e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
@@ -134,7 +134,7 @@
                                     operands[i] = ADoubleSerializerDeserializer.getDouble(data, offset + 1);
                                     break;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), i, data[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), i, data[offset],
                                             ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                             ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                             ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
index 3b1772d..ade44a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericTanEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericTanDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericTanDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
index fb52641..2e52d1e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTruncDescriptor.java
@@ -118,9 +118,9 @@
             } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
                 return (int) AInt64SerializerDeserializer.getLong(bytes, offset + 1);
             } else {
-                throw new TypeMismatchException(getIdentifier(), 1, bytes[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes[offset],
+                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
             }
         }
 
@@ -178,10 +178,10 @@
                     serde.serialize(aDouble, out);
                 }
             } else {
-                throw new TypeMismatchException(getIdentifier(), 0, data[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
-                        ATypeTag.SERIALIZED_INT64_TYPE_TAG, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
-                        ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data[offset],
+                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                        ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
             }
             result.set(resultStorage);
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
index 7f0b962..f60df2a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
@@ -60,7 +60,7 @@
 
         NumericUnaryMinusEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory argEvalFactory)
                 throws HyracksDataException {
-            super(context, argEvalFactory, NumericUnaryMinusDescriptor.this.getIdentifier());
+            super(context, argEvalFactory, NumericUnaryMinusDescriptor.this.getIdentifier(), sourceLoc);
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
index 7aea25c..b828c25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
@@ -103,7 +103,7 @@
                                 continue;
                             }
                             if (data[offset] != ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), i, data[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), i, data[offset],
                                         ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
                             }
                             boolean argResult = ABooleanSerializerDeserializer.getBoolean(data, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
index 004e50a..1f086b8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.util.string.UTF8StringWriter;
 
 /**
@@ -43,9 +44,14 @@
 public class PointableHelper {
 
     private static final byte[] NULL_BYTES = new byte[] { ATypeTag.SERIALIZED_NULL_TYPE_TAG };
-
+    private static final byte[] MISSING_BYTES = new byte[] { ATypeTag.SERIALIZED_MISSING_TYPE_TAG };
     private final UTF8StringWriter utf8Writer;
 
+    public static final IPointable NULL_REF = new VoidPointable();
+    static {
+        NULL_REF.set(NULL_BYTES, 0, NULL_BYTES.length);
+    }
+
     public PointableHelper() {
         utf8Writer = new UTF8StringWriter();
     }
@@ -135,4 +141,8 @@
     public static void setNull(IPointable pointable) {
         pointable.set(NULL_BYTES, 0, NULL_BYTES.length);
     }
+
+    public static void setMissing(IPointable pointable) {
+        pointable.set(MISSING_BYTES, 0, MISSING_BYTES.length);
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
new file mode 100644
index 0000000..fb99438
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RandomDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private final RandomHelper randomHelper = new RandomHelper(false);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        randomHelper.nextDouble(result);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANDOM;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
new file mode 100644
index 0000000..6dd5eac
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomWithSeedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RandomWithSeedDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final IPointable arg0 = new VoidPointable();
+
+                    private final RandomHelper randomHelper = new RandomHelper(true);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable resultPointable)
+                            throws HyracksDataException {
+                        eval0.evaluate(tuple, arg0);
+
+                        byte[] bytes = arg0.getByteArray();
+                        int offset = arg0.getStartOffset();
+                        ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+                        switch (tt) {
+                            case TINYINT:
+                            case SMALLINT:
+                            case INTEGER:
+                            case BIGINT:
+                            case FLOAT:
+                            case DOUBLE:
+                                randomHelper.setSeed(bytes, offset + 1, arg0.getLength() - 1);
+                                randomHelper.nextDouble(resultPointable);
+                                break;
+                            default:
+                                PointableHelper.setNull(resultPointable);
+                                break;
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANDOM_WITH_SEED;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
index f3d0bd8..1aec610 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
@@ -85,7 +85,7 @@
                                     int numOfPoints = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
 
                                     if (numOfPoints < 3) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
                                     }
                                     area = Math.abs(SpatialUtils.polygonArea(bytes, offset, numOfPoints));
@@ -118,7 +118,7 @@
                                     out.writeDouble(area);
                                     break;
                                 default:
-                                    throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                             ATypeTag.SERIALIZED_POLYGON_TYPE_TAG, ATypeTag.SERIALIZED_CIRCLE_TYPE_TAG,
                                             ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index 9b47b92..ce9ab54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -128,19 +128,19 @@
                                 rectangleSerde.serialize(aRectangle, out);
                             } else {
                                 if (tag0 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 0, tag0,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tag0,
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                                 if (tag1 != ATypeTag.SERIALIZED_POINT_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 1, tag1,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, tag1,
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                                 if (tag2 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 2, tag2,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, tag2,
                                             ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                                 }
                                 if (tag3 != ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
-                                    throw new TypeMismatchException(getIdentifier(), 3, tag3,
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 3, tag3,
                                             ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
                                 }
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
index ba47d3d..0fe73a7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
@@ -93,11 +93,11 @@
                                             offset1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
                                     distance = Math.sqrt(Math.pow(x2 - x1, 2) + Math.pow(y2 - y1, 2));
                                 } else {
-                                    throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                    throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                             ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                                 }
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_POINT_TYPE_TAG);
                             }
                             out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
index 851e1a4..a2872bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -87,13 +87,14 @@
 
                             if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                     && listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[listOffset],
                                         ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                         ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                             }
                             listAccessor.reset(listBytes, listOffset);
                             // calculate length first
                             int utf8Len = 0;
+                            boolean itemIsNull = false;
                             for (int i = 0; i < listAccessor.size(); i++) {
                                 int itemOffset = listAccessor.getItemOffset(i);
                                 ATypeTag itemType = listAccessor.getItemType(itemOffset);
@@ -104,19 +105,24 @@
                                 }
                                 if (itemType != ATypeTag.STRING) {
                                     if (itemType == ATypeTag.NULL) {
-                                        nullSerde.serialize(ANull.NULL, out);
-                                        result.set(resultStorage);
-                                        return;
+                                        itemIsNull = true;
+                                        continue;
                                     }
                                     if (itemType == ATypeTag.MISSING) {
                                         missingSerde.serialize(AMissing.MISSING, out);
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 utf8Len += UTF8StringUtil.getUTFLength(listBytes, itemOffset);
                             }
+                            if (itemIsNull) {
+                                nullSerde.serialize(ANull.NULL, out);
+                                result.set(resultStorage);
+                                return;
+                            }
                             out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             int cbytes = UTF8StringUtil.encodeUTF8Length(utf8Len, tempLengthArray, 0);
                             out.write(tempLengthArray, 0, cbytes);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
index 46865cf..4f098fd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_CONTAINS) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_CONTAINS,
+                        sourceLoc) {
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
                         return UTF8StringPointable.contains(left, right, false);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
index 6422158..090c7d1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_ENDS_WITH) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_ENDS_WITH,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index 71a5742..1eb7a88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_EQUAL) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_EQUAL,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
index 10a67b6..2b3f43f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
@@ -50,7 +50,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringInitCapDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringInitCapDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
                             throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
index 70b3687..2c00ee5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -89,14 +89,14 @@
                         int listOffset = inputArgList.getStartOffset();
                         if (listBytes[listOffset] != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                                 && listBytes[listOffset] != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, listBytes[listOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, listBytes[listOffset],
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                         }
                         byte[] sepBytes = inputArgSep.getByteArray();
                         int sepOffset = inputArgSep.getStartOffset();
                         if (sepBytes[sepOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, sepBytes[sepOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, sepBytes[sepOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         int sepLen = UTF8StringUtil.getUTFLength(sepBytes, sepOffset + 1);
@@ -126,7 +126,8 @@
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 int currentSize = UTF8StringUtil.getUTFLength(listBytes, itemOffset);
                                 if (i != size - 1 && currentSize != 0) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
index bcf3d63..2595cd4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringLTrim2Descriptor.this.getIdentifier()) {
+                        StringLTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
index 7ec5a40..de635ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringLTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringLTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index b435025..9fd173a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -83,7 +83,7 @@
                                 result.setValue(len);
                                 int64Serde.serialize(result, out);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serString[offset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             resultPointable.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
index 4481328..36bf7d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
@@ -61,7 +61,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringLikeDescriptor.this.getIdentifier()) {
+                        StringLikeDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
                     private final RegExpMatcher.IRegExpPatternGenerator patternGenerator = new LikePatternGenerator();
 
@@ -84,4 +84,4 @@
         }
     }
 
-};
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index a9957f7..98c9a69 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -49,7 +49,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringLowerCaseDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringLowerCaseDescriptor.this.getIdentifier(),
+                        sourceLoc) {
 
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index e9ec211..b4311ea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -58,7 +58,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
-                        StringPositionDescriptor.this.getIdentifier()) {
+                        StringPositionDescriptor.this.getIdentifier(), sourceLoc) {
 
                     @Override
                     protected int compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
index 5d32bca..7f886ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringRTrim2Descriptor.this.getIdentifier()) {
+                        StringRTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
index 6ec708d..7c89832 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringRTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringRTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
index 7a23578..91e1ffa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsDescriptor.java
@@ -50,7 +50,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringRegExpContainsDescriptor.this.getIdentifier()) {
+                        StringRegExpContainsDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
index a5a79d9..21d038a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpContainsWithFlagDescriptor.java
@@ -52,7 +52,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
-                        StringRegExpContainsWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpContainsWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
index 77eded9..83a6f39 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
@@ -51,7 +51,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringBoolEval(ctx, args[0], args[1],
-                        StringRegExpLikeDescriptor.this.getIdentifier()) {
+                        StringRegExpLikeDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
index 635b4a6..317545a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
@@ -49,7 +49,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringBoolEval(ctx, args[0], args[1], args[2],
-                        StringRegExpLikeWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpLikeWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
index b39c473..62cf32d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
@@ -19,8 +19,6 @@
 
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.IOException;
-
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -59,7 +57,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
-                        StringRegExpPositionDescriptor.this.getIdentifier()) {
+                        StringRegExpPositionDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
index 0d910be..c85bd3d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
@@ -57,7 +57,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringIntEval(ctx, args[0], args[1], args[2],
-                        StringRegExpPositionWithFlagDescriptor.this.getIdentifier()) {
+                        StringRegExpPositionWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
index a1ea169..fc42f04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceDescriptor.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.IOException;
-
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,7 +48,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractTripleStringStringEval(ctx, args[0], args[1], args[2],
-                        StringRegExpReplaceDescriptor.this.getIdentifier()) {
+                        StringRegExpReplaceDescriptor.this.getIdentifier(), sourceLoc) {
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagDescriptor.java
new file mode 100644
index 0000000..707865c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagDescriptor.java
@@ -0,0 +1,100 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringRegExpReplaceWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringRegExpReplaceWithFlagDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractQuadStringStringEval(ctx, args[0], args[1], args[2], args[3],
+                        StringRegExpReplaceWithFlagDescriptor.this.getIdentifier(), sourceLoc) {
+                    private final UTF8StringPointable emptyFlags = UTF8StringPointable.generateUTF8Pointable("");
+                    private final RegExpMatcher matcher = new RegExpMatcher();
+                    private int limit;
+
+                    @Override
+                    protected void processArgument(int argIdx, IPointable argPtr, UTF8StringPointable outStrPtr)
+                            throws HyracksDataException {
+                        if (argIdx == 3) {
+                            byte[] bytes = argPtr.getByteArray();
+                            int start = argPtr.getStartOffset();
+                            ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[start]];
+                            switch (tt) {
+                                case TINYINT:
+                                case SMALLINT:
+                                case INTEGER:
+                                case BIGINT:
+                                    limit = ATypeHierarchy.getIntegerValue(funcID.getName(), argIdx, bytes, start,
+                                            true);
+                                    outStrPtr.set(emptyFlags);
+                                    return;
+                                default:
+                                    limit = Integer.MAX_VALUE;
+                                    break;
+                            }
+                        }
+                        super.processArgument(argIdx, argPtr, outStrPtr);
+                    }
+
+                    @Override
+                    protected String compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr,
+                            UTF8StringPointable replacePtr, UTF8StringPointable flagsPtr) throws IOException {
+                        matcher.build(srcPtr, patternPtr, flagsPtr);
+                        return matcher.replace(replacePtr, limit);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.STRING_REGEXP_REPLACE_WITH_FLAG;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
deleted file mode 100644
index f938c24..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpReplaceWithFlagsDescriptor.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.asterix.runtime.evaluators.functions;
-
-import java.io.IOException;
-
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-
-public class StringRegExpReplaceWithFlagsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        @Override
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new StringRegExpReplaceWithFlagsDescriptor();
-        }
-    };
-
-    @Override
-    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new IScalarEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractQuadStringStringEval(ctx, args[0], args[1], args[2], args[3],
-                        StringRegExpReplaceWithFlagsDescriptor.this.getIdentifier()) {
-                    private final RegExpMatcher matcher = new RegExpMatcher();
-
-                    @Override
-                    protected String compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr,
-                            UTF8StringPointable replacePtr, UTF8StringPointable flagsPtr) throws IOException {
-                        matcher.build(srcPtr, patternPtr, flagsPtr);
-                        return matcher.replace(replacePtr);
-                    }
-                };
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return BuiltinFunctions.STRING_REGEXP_REPLACE_WITH_FLAG;
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
index 4672c75..90f4e35 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRepeatDescriptor.java
@@ -87,7 +87,7 @@
                                 ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
                         // Checks repeatingTimes. It should be a non-negative value.
                         if (repeatingTimes < 0) {
-                            throw new RuntimeDataException(ErrorCode.NEGATIVE_VALUE, getIdentifier(), 1,
+                            throw new RuntimeDataException(ErrorCode.NEGATIVE_VALUE, sourceLoc, getIdentifier(), 1,
                                     repeatingTimes);
                         }
 
@@ -96,7 +96,7 @@
                         offset = argString.getStartOffset();
                         // Checks the type of the string argument.
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
index 098d98f..472cc25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -48,7 +48,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractTripleStringEval(ctx, args[0], args[1], args[2], getIdentifier()) {
+                return new AbstractTripleStringEval(ctx, args[0], args[1], args[2], getIdentifier(), sourceLoc) {
 
                     final StringReplacer replacer = new StringReplacer();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
index fcd8218..489255e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceWithLimitDescriptor.java
@@ -48,7 +48,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringStringStringIntEval(ctx, args[0], args[1], args[2], args[3], getIdentifier()) {
+                return new AbstractStringStringStringIntEval(ctx, args[0], args[1], args[2], args[3], getIdentifier(),
+                        sourceLoc) {
 
                     final StringReplacer replacer = new StringReplacer();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
new file mode 100644
index 0000000..bcb8161
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringReverseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringReverseDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new AbstractUnaryStringStringEval(ctx, args[0], getIdentifier(), sourceLoc) {
+                    @Override
+                    void process(UTF8StringPointable inputString, IPointable resultPointable) throws IOException {
+                        UTF8StringPointable.reverse(inputString, resultBuilder, resultArray);
+                        resultPointable.set(resultArray.getByteArray(), 0, resultArray.getLength());
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.STRING_REVERSE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
index 4cfb92d..5bdcb54 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
@@ -97,7 +97,7 @@
                             int srcLen = argString.getLength();
                             // Type check for the first argument.
                             if (srcString[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, srcString[srcOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, srcString[srcOffset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -107,8 +107,8 @@
                             int patternLen = argPattern.getLength();
                             // Type check for the second argument.
                             if (patternString[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, patternString[patternOffset],
-                                        ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1,
+                                        patternString[patternOffset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
                             // Sets the UTF8 String pointables.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
index aa43e66..360f156 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
@@ -49,7 +49,8 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_STARTS_WITH) {
+                return new AbstractBinaryStringBoolEval(ctx, args[0], args[1], BuiltinFunctions.STRING_STARTS_WITH,
+                        sourceLoc) {
 
                     @Override
                     protected boolean compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index 63d474d..8b0568b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -103,7 +103,7 @@
                                 }
                                 listBuilder.write(out, true);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, serString[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, serString[offset],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
index 0136994..941f11f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
@@ -56,7 +56,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringStringEval(ctx, args[0], args[1],
-                        StringTrim2Descriptor.this.getIdentifier()) {
+                        StringTrim2Descriptor.this.getIdentifier(), sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray);
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
index bc4d150..391c8ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
@@ -56,7 +56,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringTrimDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringTrimDescriptor.this.getIdentifier(),
+                        sourceLoc) {
                     private StringTrimmer stringTrimmer = new StringTrimmer(resultBuilder, resultArray, " ");
 
                     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
index 53c7e1a..50f1d62 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
@@ -49,7 +49,8 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractUnaryStringStringEval(ctx, args[0], StringUpperCaseDescriptor.this.getIdentifier()) {
+                return new AbstractUnaryStringStringEval(ctx, args[0], StringUpperCaseDescriptor.this.getIdentifier(),
+                        sourceLoc) {
 
                     @Override
                     protected void process(UTF8StringPointable inputString, IPointable resultPointable)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index 53f73ae..52ca6cd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -86,31 +86,30 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset)
-                                - baseOffset;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
                         bytes = argString.getByteArray();
                         offset = argString.getStartOffset();
                         int len = argString.getLength();
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         string.set(bytes, offset + 1, len - 1);
                         array.reset();
                         try {
-                            UTF8StringPointable.substr(string, start, Integer.MAX_VALUE, builder, array);
-                        } catch (StringIndexOutOfBoundsException e) {
-                            throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start);
+                            int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+                            boolean success =
+                                    UTF8StringPointable.substr(string, actualStart, Integer.MAX_VALUE, builder, array);
+                            if (success) {
+                                out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                                out.write(array.getByteArray(), 0, array.getLength());
+                                result.set(resultStorage);
+                            } else {
+                                PointableHelper.setNull(result);
+                            }
                         } catch (IOException e) {
                             throw HyracksDataException.create(e);
                         }
-                        try {
-                            out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-                            out.write(array.getByteArray(), 0, array.getLength());
-                        } catch (IOException e) {
-                            throw HyracksDataException.create(e);
-                        }
-                        result.set(resultStorage);
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
index 92e0c52..eac2649 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
@@ -81,11 +81,11 @@
                         int patternLen = array1.getLength();
 
                         if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, src[srcOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, pattern[patternOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
index fe6631c..5ed13bc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -83,11 +83,11 @@
                         int patternLen = array1.getLength();
 
                         if (src[srcOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, src[srcOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, src[srcOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (pattern[patternOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, pattern[patternOffset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, pattern[patternOffset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index 148d0a4..ab0f520 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -93,8 +93,7 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset)
-                                - baseOffset;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset);
 
                         bytes = argLen.getByteArray();
                         offset = argLen.getStartOffset();
@@ -104,26 +103,24 @@
                         offset = argString.getStartOffset();
                         int length = argString.getLength();
                         if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         string.set(bytes, offset + 1, length - 1);
                         array.reset();
                         try {
-                            UTF8StringPointable.substr(string, start, len, builder, array);
-                        } catch (StringIndexOutOfBoundsException e) {
-                            throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start + len - 1);
+                            int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+                            boolean success = UTF8StringPointable.substr(string, actualStart, len, builder, array);
+                            if (success) {
+                                out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+                                out.write(array.getByteArray(), 0, array.getLength());
+                                result.set(resultStorage);
+                            } else {
+                                PointableHelper.setNull(result);
+                            }
                         } catch (IOException e) {
                             throw HyracksDataException.create(e);
                         }
-
-                        try {
-                            out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
-                            out.write(array.getByteArray(), 0, array.getLength());
-                        } catch (IOException e) {
-                            throw HyracksDataException.create(e);
-                        }
-                        result.set(resultStorage);
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
index 2762d61..8101bb5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
@@ -53,7 +53,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws IOException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
index fccd6d1..4a1bb70 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws HyracksDataException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
index 4fa33de..fecf528 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
@@ -53,7 +53,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @Override
                     protected void evaluateImpl(IPointable result) throws IOException {
                         byte[] bytes = inputArg.getByteArray();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
index d044744..770f200 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
@@ -127,7 +127,7 @@
                                 break;
 
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[startOffset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
                                         ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
index 82dbd95..b981fcf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
@@ -38,6 +38,8 @@
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class ToObjectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = -4146417699529927812L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
index 9559da4..595bb05 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
@@ -54,7 +54,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx)) {
+                return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
                     @SuppressWarnings("unchecked")
                     private final ISerializerDeserializer<ANull> nullSerde =
                             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
new file mode 100644
index 0000000..2c3e148
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * 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.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class TreatAsIntegerDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new TreatAsIntegerDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+
+                final IScalarEvaluator inputEval = args[0].createScalarEvaluator(ctx);
+                final IPointable inputArg = new VoidPointable();
+                final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                final DataOutput out = resultStorage.getDataOutput();
+                final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+                @SuppressWarnings("unchecked")
+                final ISerializerDeserializer<AInt32> int32Ser =
+                        SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+                return new IScalarEvaluator() {
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        inputEval.evaluate(tuple, inputArg);
+
+                        int intValue;
+                        byte[] bytes = inputArg.getByteArray();
+                        int startOffset = inputArg.getStartOffset();
+                        ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[startOffset]];
+                        switch (tt) {
+                            case TINYINT:
+                            case SMALLINT:
+                            case INTEGER:
+                            case BIGINT:
+                                intValue = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes,
+                                        startOffset, true);
+                                break;
+                            case FLOAT:
+                            case DOUBLE:
+                                double doubleValue =
+                                        ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, bytes, startOffset);
+                                intValue = asInt(doubleValue);
+                                break;
+                            default:
+                                throw new TypeMismatchException(sourceLoc, bytes[startOffset],
+                                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                        }
+
+                        resultStorage.reset();
+                        aInt32.setValue(intValue);
+                        int32Ser.serialize(aInt32, out);
+                        result.set(resultStorage);
+                    }
+
+                    private int asInt(double d) throws HyracksDataException {
+                        if (Double.isFinite(d)) {
+                            long v = (long) d;
+                            if (v == d && Integer.MIN_VALUE <= v && v <= Integer.MAX_VALUE) {
+                                return (int) v;
+                            }
+                        }
+                        throw new RuntimeDataException(ErrorCode.INTEGER_VALUE_EXPECTED, sourceLoc, d);
+                    }
+                };
+            }
+        };
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TREAT_AS_INTEGER;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
index bcd8094..81a6adb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractBinaryScalarEvaluator.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -38,37 +39,17 @@
     protected DataOutput dataOutput = resultStorage.getDataOutput();
     protected IPointable[] pointables;
     protected IScalarEvaluator[] evaluators;
+    protected SourceLocation sourceLoc;
 
-    public AbstractBinaryScalarEvaluator(final IHyracksTaskContext context,
-            final IScalarEvaluatorFactory[] evaluatorFactories) throws HyracksDataException {
+    public AbstractBinaryScalarEvaluator(IHyracksTaskContext context,
+            final IScalarEvaluatorFactory[] evaluatorFactories, SourceLocation sourceLoc) throws HyracksDataException {
         pointables = new IPointable[evaluatorFactories.length];
         evaluators = new IScalarEvaluator[evaluatorFactories.length];
         for (int i = 0; i < evaluators.length; ++i) {
             pointables[i] = new VoidPointable();
             evaluators[i] = evaluatorFactories[i].createScalarEvaluator(context);
         }
-    }
-
-    private static final String FIRST = "1st";
-    private static final String SECOND = "2nd";
-    private static final String THIRD = "3rd";
-    private static final String TH = "th";
-
-    protected String rankToString(int i) {
-        String prefix = "";
-        if (i >= 10) {
-            prefix = String.valueOf(i / 10);
-        }
-        switch (i % 10) {
-            case 1:
-                return prefix + FIRST;
-            case 2:
-                return prefix + SECOND;
-            case 3:
-                return prefix + THIRD;
-            default:
-                return String.valueOf(i) + TH;
-        }
+        this.sourceLoc = sourceLoc;
     }
 
     protected void checkTypeMachingThrowsIfNot(String title, ATypeTag[] expected, ATypeTag... actual)
@@ -77,7 +58,8 @@
             if (expected[i] != actual[i]) {
                 if (!ATypeHierarchy.canPromote(actual[i], expected[i])
                         && !ATypeHierarchy.canPromote(expected[i], actual[i])) {
-                    throw new TypeMismatchException(title, i, actual[i].serialize(), expected[i].serialize());
+                    throw new TypeMismatchException(sourceLoc, title, i, actual[i].serialize(),
+                            expected[i].serialize());
                 }
             }
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
index c88fe25..1637130 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -46,8 +47,8 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     public AbstractFindBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            int baseOffset, String functionName) throws HyracksDataException {
-        super(context, copyEvaluatorFactories);
+            int baseOffset, String functionName, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, copyEvaluatorFactories, sourceLoc);
         this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
index 6c84893..391df5e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -42,8 +43,8 @@
     private static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.INTEGER };
 
     public AbstractSubBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            int baseOffset, String functionName) throws HyracksDataException {
-        super(context, copyEvaluatorFactories);
+            int baseOffset, String functionName, SourceLocation sourceLoc) throws HyracksDataException {
+        super(context, copyEvaluatorFactories, sourceLoc);
         this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
index 907dfd3..4eaa1d1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
@@ -66,7 +66,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private final ListAccessor listAccessor = new ListAccessor();
                     private final byte[] metaBuffer = new byte[5];
@@ -87,25 +87,37 @@
                         byte typeTag = data[offset];
                         if (typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG
                                 && typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag,
                                     ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG,
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                         }
                         try {
                             listAccessor.reset(data, offset);
                             int concatLength = 0;
+                            boolean itemIsNull = false;
                             for (int i = 0; i < listAccessor.size(); i++) {
                                 int itemOffset = listAccessor.getItemOffset(i);
                                 ATypeTag itemType = listAccessor.getItemType(itemOffset);
                                 if (itemType != ATypeTag.BINARY) {
-                                    if (serializeUnknownIfAnyUnknown(itemType)) {
+                                    if (itemType == ATypeTag.NULL) {
+                                        itemIsNull = true;
+                                        continue;
+                                    }
+                                    if (itemType == ATypeTag.MISSING) {
+                                        missingSerde.serialize(AMissing.MISSING, dataOutput);
                                         result.set(resultStorage);
                                         return;
                                     }
-                                    throw new UnsupportedItemTypeException(getIdentifier(), itemType.serialize());
+                                    throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
+                                            itemType.serialize());
                                 }
                                 concatLength += ByteArrayPointable.getContentLength(data, itemOffset);
                             }
+                            if (itemIsNull) {
+                                nullSerde.serialize(ANull.NULL, dataOutput);
+                                result.set(resultStorage);
+                                return;
+                            }
                             dataOutput.writeByte(ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
                             int metaLen = VarLenIntEncoderDecoder.encode(concatLength, metaBuffer, 0);
                             dataOutput.write(metaBuffer, 0, metaLen);
@@ -121,20 +133,6 @@
                         }
                         result.set(resultStorage);
                     }
-
-                    private boolean serializeUnknownIfAnyUnknown(ATypeTag... tags) throws HyracksDataException {
-                        for (ATypeTag typeTag : tags) {
-                            if (typeTag == ATypeTag.NULL) {
-                                nullSerde.serialize(ANull.NULL, dataOutput);
-                                return true;
-                            }
-                            if (typeTag == ATypeTag.MISSING) {
-                                missingSerde.serialize(AMissing.MISSING, dataOutput);
-                                return true;
-                            }
-                        }
-                        return false;
-                    }
                 };
             }
         };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
index 2c53225..dc8edbe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
@@ -56,7 +56,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private AMutableInt64 result = new AMutableInt64(0);
                     @SuppressWarnings("unchecked")
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
index 307585d..f83f092 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) {
                         return 0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
index 243d344..fff7ca5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
@@ -62,7 +62,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
index 67d7b0f..35e7fba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
@@ -67,7 +67,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ABinary> binarySerde =
@@ -108,7 +108,7 @@
 
                             aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
                         } else {
-                            throw new UnsupportedItemTypeException(getIdentifier(), formatTag.serialize());
+                            throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), formatTag.serialize());
                         }
                         binarySerde.serialize(aBinary, dataOutput);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
index 11b68d3..c141e88 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
@@ -66,7 +66,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractBinaryScalarEvaluator(ctx, args) {
+                return new AbstractBinaryScalarEvaluator(ctx, args, sourceLoc) {
 
                     private StringBuilder stringBuilder = new StringBuilder();
                     private final ByteArrayPointable byteArrayPtr = new ByteArrayPointable();
@@ -101,7 +101,7 @@
                                 Base64Printer.printBase64Binary(byteArrayPtr.getByteArray(),
                                         byteArrayPtr.getContentStartOffset(), lengthBinary, stringBuilder);
                             } else {
-                                throw new UnsupportedItemTypeException(getIdentifier(), arg1Tag.serialize());
+                                throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(), arg1Tag.serialize());
                             }
                             dataOutput.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             writer.writeUTF8(stringBuilder.toString(), dataOutput);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
index 3b07c6c..c2fcdab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
@@ -60,7 +60,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) {
                         return Integer.MAX_VALUE;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
index 961edbf..a9c4c46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
@@ -62,7 +62,7 @@
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName(), sourceLoc) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
new file mode 100644
index 0000000..0fba990
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractRecordAddPutEvaluator implements IScalarEvaluator {
+
+    private final CastTypeEvaluator inputRecordCaster;
+    private final CastTypeEvaluator argRecordCaster;
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final IScalarEvaluator eval2;
+    final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    final DataOutput resultOutput = resultStorage.getDataOutput();
+    final IPointable inputRecordPointable = new VoidPointable();
+    final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
+    final IPointable newFieldValuePointable = new VoidPointable();
+    final IBinaryComparator stringBinaryComparator = PointableHelper.createStringBinaryComparator();
+    final RecordBuilder outRecordBuilder = new RecordBuilder();
+    final ARecordVisitablePointable inputOpenRecordPointable;
+    boolean newFieldValueIsMissing = false;
+
+    AbstractRecordAddPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
+            IAType[] argTypes) {
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        this.eval2 = eval2;
+        inputOpenRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
+        argRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        eval0.evaluate(tuple, inputRecordPointable);
+        eval1.evaluate(tuple, newFieldNamePointable);
+        eval2.evaluate(tuple, newFieldValuePointable);
+        if (containsMissing(inputRecordPointable, newFieldNamePointable)) {
+            writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+            result.set(resultStorage);
+            return;
+        }
+        final ATypeTag inputObjectType = PointableHelper.getTypeTag(inputRecordPointable);
+        final ATypeTag newFieldNameValueType = PointableHelper.getTypeTag(newFieldNamePointable);
+        if (inputObjectType != ATypeTag.OBJECT || newFieldNameValueType != ATypeTag.STRING) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        inputRecordCaster.evaluate(tuple, inputRecordPointable);
+        final ATypeTag newFieldValueTag = PointableHelper.getTypeTag(newFieldValuePointable);
+        if (newFieldValueTag.isDerivedType()) {
+            argRecordCaster.evaluate(tuple, newFieldValuePointable);
+        }
+        newFieldValueIsMissing = newFieldValueTag == ATypeTag.MISSING;
+        buildOutputRecord();
+        result.set(resultStorage);
+    }
+
+    protected abstract void buildOutputRecord() throws HyracksDataException;
+
+    private boolean containsMissing(IPointable... pointables) {
+        for (int i = 0; i < pointables.length; i++) {
+            if (PointableHelper.getTypeTag(pointables[i]) == ATypeTag.MISSING) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private void writeTypeTag(byte typeTag) throws HyracksDataException {
+        try {
+            resultOutput.writeByte(typeTag);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordPairsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordPairsEvaluator.java
new file mode 100644
index 0000000..82c45af
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordPairsEvaluator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractRecordPairsEvaluator implements IScalarEvaluator {
+    protected final IScalarEvaluator eval0;
+    protected final IPointable inputPointable = new VoidPointable();
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput resultOutput = resultStorage.getDataOutput();
+    private CastTypeEvaluator inputCaster;
+
+    AbstractRecordPairsEvaluator(IScalarEvaluator eval0, IAType inputType) {
+        this.eval0 = eval0;
+        if (inputType != null) {
+            inputCaster = new CastTypeEvaluator(BuiltinType.ANY, inputType, eval0);
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        eval0.evaluate(tuple, inputPointable);
+        final ATypeTag inputTypeTag = PointableHelper.getTypeTag(inputPointable);
+        if (!validateInputType(inputTypeTag)) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        inputCaster.evaluate(tuple, inputPointable);
+        resultStorage.reset();
+        buildOutput();
+        result.set(resultStorage);
+    }
+
+    protected abstract boolean validateInputType(ATypeTag inputTypeTag);
+
+    protected abstract void buildOutput() throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
index d641ea9..d624133 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessByIndexEvalFactory(args[0], args[1], recType);
+        return new FieldAccessByIndexEvalFactory(args[0], args[1], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
index d5918cb..3fcf87d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -35,6 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -49,13 +49,15 @@
     private IScalarEvaluatorFactory fieldIndexEvalFactory;
     private int nullBitmapSize;
     private ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessByIndexEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fieldIndexEvalFactory, ARecordType recordType) {
+            IScalarEvaluatorFactory fieldIndexEvalFactory, ARecordType recordType, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fieldIndexEvalFactory = fieldIndexEvalFactory;
         this.recordType = recordType;
         this.nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -89,14 +91,14 @@
                     int offset = inputArg0.getStartOffset();
 
                     if (serRecord[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 0, serRecord[offset],
+                        throw new TypeMismatchException(sourceLoc, serRecord[offset],
                                 ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
                     eval1.evaluate(tuple, inputArg1);
                     byte[] indexBytes = inputArg1.getByteArray();
                     int indexOffset = inputArg1.getStartOffset();
                     if (indexBytes[indexOffset] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, 1, indexBytes[offset],
+                        throw new TypeMismatchException(sourceLoc, indexBytes[offset],
                                 ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                     }
                     fieldIndex = IntegerPointable.getInteger(indexBytes, indexOffset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
index 4312cac..acd11c7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
@@ -41,7 +41,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessByNameEvalFactory(args[0], args[1]);
+        return new FieldAccessByNameEvalFactory(args[0], args[1], sourceLoc);
     }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 2078921..e66cc52 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -22,7 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
@@ -30,7 +31,10 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -42,17 +46,23 @@
 
     private IScalarEvaluatorFactory recordEvalFactory;
     private IScalarEvaluatorFactory fldNameEvalFactory;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessByNameEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fldNameEvalFactory) {
+            IScalarEvaluatorFactory fldNameEvalFactory, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fldNameEvalFactory = fldNameEvalFactory;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private DataOutput out = resultStorage.getDataOutput();
 
@@ -75,13 +85,13 @@
                     int serRecordLen = inputArg0.getLength();
 
                     if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_BY_NAME, 0,
-                                serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, serRecord[serRecordOffset],
+                                ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
                     byte[] serFldName = inputArg1.getByteArray();
                     int serFldNameOffset = inputArg1.getStartOffset();
                     fieldValueOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (fieldValueOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
index 5dd3e19..91b36cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedDescriptor.java
@@ -61,6 +61,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new FieldAccessNestedEvalFactory(args[0], recType, fldName);
+        return new FieldAccessNestedEvalFactory(args[0], recType, fldName, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 067a458..f885f5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -23,11 +23,12 @@
 import java.util.List;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -38,12 +39,14 @@
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -54,22 +57,27 @@
 
     private static final long serialVersionUID = 1L;
 
-    private IScalarEvaluatorFactory recordEvalFactory;
-    private ARecordType recordType;
-    private List<String> fieldPath;
+    private final IScalarEvaluatorFactory recordEvalFactory;
+    private final ARecordType recordType;
+    private final List<String> fieldPath;
+    private final SourceLocation sourceLoc;
 
     public FieldAccessNestedEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType,
-            List<String> fldName) {
+            List<String> fldName, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.recordType = recordType;
         this.fieldPath = fldName;
-
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
             private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
@@ -116,7 +124,7 @@
                     int len = inputArg0.getLength();
 
                     if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.FIELD_ACCESS_NESTED, 0, serRecord[start],
+                        throw new TypeMismatchException(sourceLoc, serRecord[start],
                                 ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
 
@@ -139,8 +147,8 @@
                             subType = ((AUnionType) subType).getActualType();
                             byte serializedTypeTag = subType.getTypeTag().serialize();
                             if (serializedTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                                throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED.getName(),
-                                        serializedTypeTag);
+                                throw new TypeMismatchException(sourceLoc, serializedTypeTag,
+                                        ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                             }
                             if (subType.getTypeTag() == ATypeTag.OBJECT) {
                                 recTypeInfos[pathIndex].reset((ARecordType) subType);
@@ -195,7 +203,8 @@
                         // type check
                         if (pathIndex < fieldPointables.length - 1
                                 && serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED, serRecord[start]);
+                            throw new TypeMismatchException(sourceLoc, serRecord[start],
+                                    ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                         }
                     }
 
@@ -203,7 +212,8 @@
                     for (; pathIndex < fieldPointables.length; pathIndex++) {
                         openField = true;
                         subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, start, len,
-                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset());
+                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset(),
+                                fieldNameHashFunction, fieldNameComparator);
                         if (subFieldOffset < 0) {
                             out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                             result.set(resultStorage);
@@ -229,8 +239,8 @@
                             return;
                         }
                         if (serRecord[start] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new UnsupportedTypeException(BuiltinFunctions.FIELD_ACCESS_NESTED.getName(),
-                                    serRecord[start]);
+                            throw new TypeMismatchException(sourceLoc, serRecord[start],
+                                    ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                         }
                     }
                     // emit the final result.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
index 4c07e2d..b25ba1e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new GetRecordFieldValueEvalFactory(args[0], args[1], recType);
+        return new GetRecordFieldValueEvalFactory(args[0], args[1], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 95fb91a..7cc3aa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -33,7 +35,10 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -46,18 +51,24 @@
     private IScalarEvaluatorFactory recordEvalFactory;
     private IScalarEvaluatorFactory fldNameEvalFactory;
     private final ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
     public GetRecordFieldValueEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
-            IScalarEvaluatorFactory fldNameEvalFactory, ARecordType recordType) {
+            IScalarEvaluatorFactory fldNameEvalFactory, ARecordType recordType, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fldNameEvalFactory = fldNameEvalFactory;
         this.recordType = recordType;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
 
@@ -86,7 +97,7 @@
                     int serRecordLen = inputArg0.getLength();
 
                     if (serRecord[serRecordOffset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.GET_RECORD_FIELD_VALUE, 0,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_RECORD_FIELD_VALUE, 0,
                                 serRecord[serRecordOffset], ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
 
@@ -117,7 +128,7 @@
 
                     // Look at open fields.
                     subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (subFieldOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
index 57e98d0..03c26e3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsDescriptor.java
@@ -57,6 +57,6 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
-        return new GetRecordFieldsEvalFactory(args[0], recType);
+        return new GetRecordFieldsEvalFactory(args[0], recType, sourceLoc);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
index 69ea1bc..7723719 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -41,18 +42,20 @@
 
     private IScalarEvaluatorFactory recordEvalFactory;
     private final ARecordType recordType;
+    private final SourceLocation sourceLoc;
 
-    public GetRecordFieldsEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType) {
+    public GetRecordFieldsEvalFactory(IScalarEvaluatorFactory recordEvalFactory, ARecordType recordType,
+            SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.recordType = recordType;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
-            private final ARecordPointable recordPointable =
-                    (ARecordPointable) ARecordPointable.FACTORY.createPointable();
+            private final ARecordPointable recordPointable = ARecordPointable.FACTORY.createPointable();
             private IPointable inputArg0 = new VoidPointable();
             private IScalarEvaluator eval0 = recordEvalFactory.createScalarEvaluator(ctx);
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -68,7 +71,7 @@
                 int len = inputArg0.getLength();
 
                 if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.GET_RECORD_FIELDS, 0, data[offset],
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.GET_RECORD_FIELDS, 0, data[offset],
                             ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                 }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsDescriptor.java
new file mode 100644
index 0000000..38295b0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PairsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new PairsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private IAType inputType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new PairsEvaluator(args[0].createScalarEvaluator(ctx), inputType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.PAIRS;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsEvaluator.java
new file mode 100644
index 0000000..4cc54e1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/PairsEvaluator.java
@@ -0,0 +1,133 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.util.ArrayDeque;
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+class PairsEvaluator extends AbstractRecordPairsEvaluator {
+
+    // For writing output list
+    private final OrderedListBuilder outerListBuilder = new OrderedListBuilder();
+
+    // For writing each individual inner list.
+    private final ArrayBackedValueStorage innerListStorage = new ArrayBackedValueStorage();
+    private final DataOutput innerListOutput = innerListStorage.getDataOutput();
+    private final OrderedListBuilder innerListBuilder = new OrderedListBuilder();
+
+    private final PointableAllocator pAlloc = new PointableAllocator();
+    private final ArrayDeque<IPointable> pNameQueue = new ArrayDeque<>();
+    private final ArrayDeque<IPointable> pValueQueue = new ArrayDeque<>();
+
+    private final VoidPointable nullPointable = new VoidPointable();
+
+    PairsEvaluator(IScalarEvaluator eval0, IAType inputType) {
+        super(eval0, inputType);
+        PointableHelper.setNull(nullPointable);
+    }
+
+    @Override
+    protected void buildOutput() throws HyracksDataException {
+        pAlloc.reset();
+        pNameQueue.clear();
+        pValueQueue.clear();
+
+        outerListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+
+        addMembersToQueue(nullPointable, inputPointable);
+        IPointable namePointable, valuePointable;
+        while ((valuePointable = pValueQueue.poll()) != null) {
+            namePointable = pNameQueue.remove();
+
+            addMembersToQueue(namePointable, valuePointable);
+
+            if (PointableHelper.getTypeTag(namePointable) != ATypeTag.NULL) {
+                innerListStorage.reset();
+                innerListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+                innerListBuilder.addItem(namePointable);
+                innerListBuilder.addItem(valuePointable);
+                innerListBuilder.write(innerListOutput, true);
+
+                outerListBuilder.addItem(innerListStorage);
+            }
+        }
+
+        // Writes the result and sets the result pointable.
+        outerListBuilder.write(resultOutput, true);
+    }
+
+    private void addMembersToQueue(IPointable namePointable, IPointable valuePointable) {
+        ATypeTag valueTypeTag = PointableHelper.getTypeTag(valuePointable);
+        switch (valueTypeTag) {
+            case OBJECT:
+                addRecordFieldsToQueue(valuePointable);
+                break;
+            case ARRAY:
+            case MULTISET:
+                addListItemsToQueue(valuePointable, DefaultOpenFieldType.getDefaultOpenFieldType(valueTypeTag),
+                        namePointable);
+                break;
+        }
+    }
+
+    private void addRecordFieldsToQueue(IPointable recordPointable) {
+        ARecordVisitablePointable visitablePointable =
+                pAlloc.allocateRecordValue(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        visitablePointable.set(recordPointable);
+        List<IVisitablePointable> fieldNames = visitablePointable.getFieldNames();
+        List<IVisitablePointable> fieldValues = visitablePointable.getFieldValues();
+        for (int i = fieldNames.size() - 1; i >= 0; i--) {
+            pNameQueue.push(fieldNames.get(i));
+            pValueQueue.push(fieldValues.get(i));
+        }
+    }
+
+    private void addListItemsToQueue(IPointable listPointable, IAType listType, IPointable fieldNamePointable) {
+        AListVisitablePointable visitablePointable = pAlloc.allocateListValue(listType);
+        visitablePointable.set(listPointable);
+        List<IVisitablePointable> items = visitablePointable.getItems();
+        for (int i = items.size() - 1; i >= 0; i--) {
+            pNameQueue.push(fieldNamePointable);
+            pValueQueue.push(items.get(i));
+        }
+    }
+
+    @Override
+    protected boolean validateInputType(ATypeTag inputTypeTag) {
+        return inputTypeTag == ATypeTag.OBJECT || inputTypeTag == ATypeTag.ARRAY || inputTypeTag == ATypeTag.MULTISET;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
new file mode 100644
index 0000000..7788bc1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordAddDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = new IAType[states.length];
+        for (int i = 0; i < states.length; i++) {
+            argTypes[i] = (IAType) states[i];
+        }
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_ADD;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
new file mode 100644
index 0000000..bdadfb6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
@@ -0,0 +1,58 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.util.List;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+class RecordAddEvaluator extends AbstractRecordAddPutEvaluator {
+
+    RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
+        super(eval0, eval1, eval2, argTypes);
+    }
+
+    @Override
+    protected void buildOutputRecord() throws HyracksDataException {
+        resultStorage.reset();
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        inputOpenRecordPointable.set(inputRecordPointable);
+        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        boolean newFieldFound = false;
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+                newFieldFound = true;
+            }
+            outRecordBuilder.addField(fieldName, fieldValues.get(i));
+        }
+        if (!newFieldValueIsMissing && !newFieldFound) {
+            outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
index 71a8291..0edd74b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddFieldsDescriptor.java
@@ -147,14 +147,14 @@
                         // Make sure we get a valid record
                         byte typeTag0 = argPtr0.getByteArray()[argPtr0.getStartOffset()];
                         if (typeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag0,
                                     ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                         }
 
                         // Make sure we get a valid list
                         byte typeTag1 = argPtr1.getByteArray()[argPtr1.getStartOffset()];
                         if (typeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, typeTag1,
                                     ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                         }
 
@@ -229,7 +229,8 @@
                                 }
 
                                 if (namePointable == null || valuePointable == null) {
-                                    throw new InvalidDataFormatException(getIdentifier(), "fields to be added");
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                            "fields to be added");
                                 }
 
                                 // Check that the field being added is a valid field
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
index 0a3ba36..401335c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
@@ -57,7 +57,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordConcatEvalFactory(args, argTypes, false);
+        return new RecordConcatEvalFactory(args, argTypes, false, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
index 82cc876..989f217 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,10 +57,14 @@
 
     private final boolean failOnArgTypeMismatch;
 
-    RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, boolean failOnArgTypeMismatch) {
+    private final SourceLocation sourceLoc;
+
+    RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, boolean failOnArgTypeMismatch,
+            SourceLocation sourceLoc) {
         this.args = args;
         this.argTypes = argTypes;
         this.failOnArgTypeMismatch = failOnArgTypeMismatch;
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -161,7 +166,7 @@
                     returnNull = true;
                 } else if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
                     if (failOnArgTypeMismatch) {
-                        throw new TypeMismatchException(BuiltinFunctions.RECORD_CONCAT, i, typeTag,
+                        throw new TypeMismatchException(sourceLoc, BuiltinFunctions.RECORD_CONCAT, i, typeTag,
                                 ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     } else {
                         returnNull = true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
index 77fe301..b51f66a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
@@ -56,7 +56,7 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordConcatEvalFactory(args, argTypes, true);
+        return new RecordConcatEvalFactory(args, argTypes, true, sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
index d93d572..0d85807 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
@@ -27,7 +27,6 @@
 import org.apache.asterix.builders.ListBuilderFactory;
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.builders.RecordBuilderFactory;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AMutableString;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordLengthDescriptor.java
index cda069d..cd24ef7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordLengthDescriptor.java
@@ -79,8 +79,7 @@
                     private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
 
                     private final IPointable argPtr = new VoidPointable();
-                    private final ARecordPointable recordPointable =
-                            (ARecordPointable) ARecordPointable.FACTORY.createPointable();
+                    private final ARecordPointable recordPointable = ARecordPointable.FACTORY.createPointable();
                     private final AMutableInt64 aInt64 = new AMutableInt64(0);
                     @SuppressWarnings("unchecked")
                     private final ISerializerDeserializer<AInt64> int64Serde =
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordNamesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordNamesDescriptor.java
index 1719980..0ba2aa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordNamesDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordNamesDescriptor.java
@@ -77,8 +77,7 @@
 
                     private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
                     private final IPointable argPtr = new VoidPointable();
-                    private final ARecordPointable recordPointable =
-                            (ARecordPointable) ARecordPointable.FACTORY.createPointable();
+                    private final ARecordPointable recordPointable = ARecordPointable.FACTORY.createPointable();
                     private final AOrderedListType listType = new AOrderedListType(BuiltinType.ASTRING, null);
                     private final OrderedListBuilder listBuilder = new OrderedListBuilder();
                     private final ArrayBackedValueStorage itemStorage = new ArrayBackedValueStorage();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
index 6f8ba29..166ce96 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsDescriptor.java
@@ -18,25 +18,11 @@
  */
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.functions.IFunctionTypeInferer;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.utils.RecordUtil;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,10 +30,6 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class RecordPairsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -59,7 +41,7 @@
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_STRICT;
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
         }
     };
 
@@ -83,67 +65,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                // For writing each individual output record.
-                final ArrayBackedValueStorage itemStorage = new ArrayBackedValueStorage();
-                final DataOutput itemOutput = itemStorage.getDataOutput();
-                final RecordBuilder recBuilder = new RecordBuilder();
-                recBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
-
-                // For writing the resulting list of records.
-                final OrderedListBuilder listBuilder = new OrderedListBuilder();
-                final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-                final DataOutput resultOut = resultStorage.getDataOutput();
-
-                // Sets up the constant field names, "name" for the key field, "value" for the value field.
-                final ArrayBackedValueStorage nameStorage = new ArrayBackedValueStorage();
-                final ArrayBackedValueStorage valueStorage = new ArrayBackedValueStorage();
-                AObjectSerializerDeserializer serde = AObjectSerializerDeserializer.INSTANCE;
-                try {
-                    serde.serialize(new AString("name"), nameStorage.getDataOutput());
-                    serde.serialize(new AString("value"), valueStorage.getDataOutput());
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
-                }
-
-                return new IScalarEvaluator() {
-                    private final IScalarEvaluator argEvaluator = args[0].createScalarEvaluator(ctx);
-                    private final IPointable argPtr = new VoidPointable();
-                    private final ARecordVisitablePointable recordVisitablePointable =
-                            new ARecordVisitablePointable(recType);
-
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-                        // Resets the result storage.
-                        resultStorage.reset();
-
-                        // Gets the input record.
-                        argEvaluator.evaluate(tuple, argPtr);
-                        byte inputTypeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
-                        if (inputTypeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, inputTypeTag,
-                                    ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
-                        }
-                        recordVisitablePointable.set(argPtr);
-
-                        listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
-                        List<IVisitablePointable> fieldNames = recordVisitablePointable.getFieldNames();
-                        List<IVisitablePointable> fieldValues = recordVisitablePointable.getFieldValues();
-                        // Adds each field of the input record as a key-value pair into the result.
-                        int numFields = recordVisitablePointable.getFieldNames().size();
-                        for (int fieldIndex = 0; fieldIndex < numFields; ++fieldIndex) {
-                            itemStorage.reset();
-                            recBuilder.init();
-                            recBuilder.addField(nameStorage, fieldNames.get(fieldIndex));
-                            recBuilder.addField(valueStorage, fieldValues.get(fieldIndex));
-                            recBuilder.write(itemOutput, true);
-                            listBuilder.addItem(itemStorage);
-                        }
-
-                        // Writes the result and sets the result pointable.
-                        listBuilder.write(resultOut, true);
-                        result.set(resultStorage);
-                    }
-                };
+                return new RecordPairsEvaluator(args[0].createScalarEvaluator(ctx), recType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsEvaluator.java
new file mode 100644
index 0000000..03273d9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPairsEvaluator.java
@@ -0,0 +1,96 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+class RecordPairsEvaluator extends AbstractRecordPairsEvaluator {
+
+    // For writing output list
+    private final OrderedListBuilder listBuilder = new OrderedListBuilder();
+
+    // For writing each individual output record.
+    private final ArrayBackedValueStorage itemStorage = new ArrayBackedValueStorage();
+    private final DataOutput itemOutput = itemStorage.getDataOutput();
+    private final RecordBuilder recBuilder = new RecordBuilder();
+
+    // Sets up the constant field names, "name" for the key field, "value" for the value field.
+    private final ArrayBackedValueStorage nameStorage = new ArrayBackedValueStorage();
+    private final ArrayBackedValueStorage valueStorage = new ArrayBackedValueStorage();
+
+    private ARecordVisitablePointable recordVisitablePointable;
+
+    RecordPairsEvaluator(IScalarEvaluator eval0, ARecordType recordType) throws HyracksDataException {
+        super(eval0, recordType);
+        recBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        recordVisitablePointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+
+        try {
+            AObjectSerializerDeserializer serde = AObjectSerializerDeserializer.INSTANCE;
+            serde.serialize(new AString("name"), nameStorage.getDataOutput());
+            serde.serialize(new AString("value"), valueStorage.getDataOutput());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    @Override
+    protected void buildOutput() throws HyracksDataException {
+        listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+
+        recordVisitablePointable.set(inputPointable);
+        List<IVisitablePointable> fieldNames = recordVisitablePointable.getFieldNames();
+        List<IVisitablePointable> fieldValues = recordVisitablePointable.getFieldValues();
+        // Adds each field of the input record as a key-value pair into the result.
+        int numFields = recordVisitablePointable.getFieldNames().size();
+        for (int fieldIndex = 0; fieldIndex < numFields; ++fieldIndex) {
+            itemStorage.reset();
+            recBuilder.init();
+            recBuilder.addField(nameStorage, fieldNames.get(fieldIndex));
+            recBuilder.addField(valueStorage, fieldValues.get(fieldIndex));
+            recBuilder.write(itemOutput, true);
+            listBuilder.addItem(itemStorage);
+        }
+
+        // Writes the result and sets the result pointable.
+        listBuilder.write(resultOutput, true);
+    }
+
+    @Override
+    protected boolean validateInputType(ATypeTag inputTypeTag) {
+        return inputTypeTag == ATypeTag.OBJECT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
new file mode 100644
index 0000000..a312795
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordPutDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = new IAType[states.length];
+        for (int i = 0; i < states.length; i++) {
+            argTypes[i] = (IAType) states[i];
+        }
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_PUT;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
new file mode 100644
index 0000000..83ee0c4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.util.List;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+class RecordPutEvaluator extends AbstractRecordAddPutEvaluator {
+
+    RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
+        super(eval0, eval1, eval2, argTypes);
+    }
+
+    @Override
+    protected void buildOutputRecord() throws HyracksDataException {
+        resultStorage.reset();
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        inputOpenRecordPointable.set(inputRecordPointable);
+        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        boolean newFieldFound = false;
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (!PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            } else {
+                newFieldFound = true;
+                if (!newFieldValueIsMissing) {
+                    putNewField();
+                }
+            }
+        }
+        if (!newFieldFound) {
+            putNewField();
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private void putNewField() throws HyracksDataException {
+        outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
new file mode 100644
index 0000000..b57f01e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
@@ -0,0 +1,76 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordRemoveDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_REMOVE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
new file mode 100644
index 0000000..d92c9a1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
@@ -0,0 +1,147 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordRemoveEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final UTF8StringPointable fieldToRemovePointable = new UTF8StringPointable();
+    private final IBinaryComparator stringBinaryComparator = PointableHelper.createStringBinaryComparator();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final ARecordVisitablePointable openRecordPointable;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+
+    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType recordType) {
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg =
+                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        boolean returnNull = false;
+        eval0.evaluate(tuple, inputRecordPointable);
+        byte[] data = inputRecordPointable.getByteArray();
+        int offset = inputRecordPointable.getStartOffset();
+        byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval1.evaluate(tuple, fieldToRemovePointable);
+        data = fieldToRemovePointable.getByteArray();
+        offset = fieldToRemovePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        if (returnNull) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        evaluate();
+        result.set(resultStorage);
+    }
+
+    private void evaluate() throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
+            buildOutputRecord(inputRecord);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
+                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            }
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordPointable;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
index 909c74b..54db1e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsDescriptor.java
@@ -60,7 +60,8 @@
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new RecordRemoveFieldsEvalFactory(args[0], args[1], outputRecordType, inputRecType, inputListType);
+        return new RecordRemoveFieldsEvalFactory(args[0], args[1], outputRecordType, inputRecType, inputListType,
+                sourceLoc);
     }
 
     @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
index aeee36f..2d372f3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveFieldsEvalFactory.java
@@ -44,6 +44,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -56,16 +57,17 @@
     private ARecordType requiredRecType;
     private ARecordType inputRecType;
     private AOrderedListType inputListType;
+    private final SourceLocation sourceLoc;
 
     public RecordRemoveFieldsEvalFactory(IScalarEvaluatorFactory inputRecordEvalFactory,
             IScalarEvaluatorFactory removeFieldPathsFactory, ARecordType requiredRecType, ARecordType inputRecType,
-            AOrderedListType inputListType) {
+            AOrderedListType inputListType, SourceLocation sourceLoc) {
         this.inputRecordEvalFactory = inputRecordEvalFactory;
         this.removeFieldPathsFactory = removeFieldPathsFactory;
         this.requiredRecType = requiredRecType;
         this.inputRecType = inputRecType;
         this.inputListType = inputListType;
-
+        this.sourceLoc = sourceLoc;
     }
 
     @Override
@@ -98,13 +100,13 @@
 
                 byte inputTypeTag0 = inputArg0.getByteArray()[inputArg0.getStartOffset()];
                 if (inputTypeTag0 != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.REMOVE_FIELDS, 0, inputTypeTag0,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.REMOVE_FIELDS, 0, inputTypeTag0,
                             ATypeTag.SERIALIZED_INT32_TYPE_TAG);
                 }
 
                 byte inputTypeTag1 = inputArg1.getByteArray()[inputArg1.getStartOffset()];
                 if (inputTypeTag1 != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
-                    throw new TypeMismatchException(BuiltinFunctions.REMOVE_FIELDS, 1, inputTypeTag1,
+                    throw new TypeMismatchException(sourceLoc, BuiltinFunctions.REMOVE_FIELDS, 1, inputTypeTag1,
                             ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
                 }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
new file mode 100644
index 0000000..ac1b8e2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
@@ -0,0 +1,76 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordRenameDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_RENAME;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
new file mode 100644
index 0000000..3d46b69
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
@@ -0,0 +1,161 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordRenameEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final UTF8StringPointable oldFieldNamePointable = new UTF8StringPointable();
+    private final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
+    private final IBinaryComparator stringBinaryComparator = PointableHelper.createStringBinaryComparator();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final IScalarEvaluator eval2;
+    private final ARecordVisitablePointable openRecordPointable;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+
+    RecordRenameEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
+            ARecordType recordType) {
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        this.eval2 = eval2;
+        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg =
+                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        boolean returnNull = false;
+        eval0.evaluate(tuple, inputRecordPointable);
+        byte[] data = inputRecordPointable.getByteArray();
+        int offset = inputRecordPointable.getStartOffset();
+        byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval1.evaluate(tuple, oldFieldNamePointable);
+        data = oldFieldNamePointable.getByteArray();
+        offset = oldFieldNamePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        eval2.evaluate(tuple, newFieldNamePointable);
+        data = newFieldNamePointable.getByteArray();
+        offset = newFieldNamePointable.getStartOffset();
+        typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+            returnNull = true;
+        }
+        if (returnNull) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        evaluate();
+        result.set(resultStorage);
+    }
+
+    private void evaluate() throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
+            buildOutputRecord(inputRecord);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
+                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            } else {
+                outRecordBuilder.addField(newFieldNamePointable, fieldValues.get(i));
+            }
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordPointable;
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceDescriptor.java
new file mode 100644
index 0000000..03d0088
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordReplaceDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private IAType[] argTypes;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = new IAType[states.length];
+        for (int i = 0; i < states.length; i++) {
+            argTypes[i] = (IAType) states[i];
+        }
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator[] argEvals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    argEvals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new RecordReplaceEvaluator(sourceLoc, argEvals[0], argEvals[1], argEvals[2], argTypes);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_REPLACE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
new file mode 100644
index 0000000..5f004ef
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
@@ -0,0 +1,159 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
+import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordReplaceEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final IPointable oldValuePointable = new VoidPointable();
+    private final IPointable newValuePointable = new VoidPointable();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+    private final VoidPointable existingValuePtr = new VoidPointable();
+    private final VoidPointable oldValuePtr = new VoidPointable();
+    private final IScalarEvaluator eval0;
+    private final IScalarEvaluator eval1;
+    private final IScalarEvaluator eval2;
+    private final ARecordVisitablePointable openRecordPointable;
+    private final CastTypeEvaluator inputRecordCaster;
+    private final CastTypeEvaluator newValueRecordCaster;
+    private final SourceLocation sourceLoc;
+    private final ComparisonHelper comparisonHelper;
+
+    RecordReplaceEvaluator(SourceLocation sourceLoc, IScalarEvaluator eval0, IScalarEvaluator eval1,
+            IScalarEvaluator eval2, IAType[] argTypes) {
+        this.sourceLoc = sourceLoc;
+        this.eval0 = eval0;
+        this.eval1 = eval1;
+        this.eval2 = eval2;
+        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
+        newValueRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
+        comparisonHelper = new ComparisonHelper(sourceLoc);
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        eval0.evaluate(tuple, inputRecordPointable);
+        eval1.evaluate(tuple, oldValuePointable);
+        eval2.evaluate(tuple, newValuePointable);
+        if (containsMissing(inputRecordPointable, oldValuePointable, newValuePointable)) {
+            writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+            result.set(resultStorage);
+            return;
+        }
+        final ATypeTag inputObjectType = PointableHelper.getTypeTag(inputRecordPointable);
+        final ATypeTag oldValueType = PointableHelper.getTypeTag(oldValuePointable);
+        if (inputObjectType != ATypeTag.OBJECT || oldValueType == ATypeTag.NULL) {
+            writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+            result.set(resultStorage);
+            return;
+        }
+        if (oldValueType.isDerivedType()) {
+            throw new TypeMismatchException(sourceLoc, BuiltinFunctions.RECORD_REPLACE, 1, oldValueType.serialize(),
+                    "primitive");
+        }
+        inputRecordCaster.evaluate(tuple, inputRecordPointable);
+        final ATypeTag newValueType = PointableHelper.getTypeTag(newValuePointable);
+        if (newValueType.isDerivedType()) {
+            newValueRecordCaster.evaluate(tuple, newValuePointable);
+        }
+        resultStorage.reset();
+        buildOutputRecord(oldValueType);
+        result.set(resultStorage);
+    }
+
+    private void buildOutputRecord(ATypeTag oldValueTypeTag) throws HyracksDataException {
+        openRecordPointable.set(inputRecordPointable);
+        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        outRecordBuilder.init();
+        final List<IVisitablePointable> fieldNames = openRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = openRecordPointable.getFieldValues();
+        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+            final IVisitablePointable fieldName = fieldNames.get(i);
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            final ATypeTag existingValueTypeTag = PointableHelper.getTypeTag(fieldValue);
+            if (isEqual(existingValueTypeTag, fieldValue, oldValueTypeTag, oldValuePointable)) {
+                outRecordBuilder.addField(fieldName, newValuePointable);
+            } else {
+                outRecordBuilder.addField(fieldName, fieldValue);
+            }
+        }
+        outRecordBuilder.write(resultOutput, true);
+    }
+
+    private boolean containsMissing(IPointable... pointables) {
+        for (int i = 0; i < pointables.length; i++) {
+            if (PointableHelper.getTypeTag(pointables[i]) == ATypeTag.MISSING) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private void writeTypeTag(byte typeTag) throws HyracksDataException {
+        try {
+            resultOutput.writeByte(typeTag);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private boolean isEqual(ATypeTag typeTag1, IPointable value1, ATypeTag typeTag2, IPointable value2)
+            throws HyracksDataException {
+        if (!ATypeHierarchy.isCompatible(typeTag1, typeTag2)) {
+            return false;
+        }
+        setValuePointer(value1, existingValuePtr);
+        setValuePointer(value2, oldValuePtr);
+        return comparisonHelper.compare(typeTag1, typeTag2, existingValuePtr, oldValuePtr) == 0;
+    }
+
+    private static void setValuePointer(IPointable src, IPointable value) {
+        value.set(src.getByteArray(), src.getStartOffset() + 1, src.getLength() - 1);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java
new file mode 100644
index 0000000..5890d7a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapDescriptor.java
@@ -0,0 +1,73 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordUnwrapDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordUnwrapDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                return new RecordUnwrapEvaluator(eval0, recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_UNWRAP;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java
new file mode 100644
index 0000000..97bc01a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordUnwrapEvaluator.java
@@ -0,0 +1,118 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordUnwrapEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final IScalarEvaluator eval0;
+    private ARecordVisitablePointable inputRecordVisitable;
+    private ARecordVisitablePointable openRecordVisitablePointable;
+    private boolean requiresCast = false;
+    private ACastVisitor castVisitor;
+    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+
+    RecordUnwrapEvaluator(IScalarEvaluator eval0, ARecordType recordType) {
+        this.eval0 = eval0;
+        openRecordVisitablePointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        if (recordType != null) {
+            inputRecordVisitable = new ARecordVisitablePointable(recordType);
+            if (hasDerivedType(recordType.getFieldTypes())) {
+                requiresCast = true;
+                castVisitor = new ACastVisitor();
+                castVisitorArg = new Triple<>(openRecordVisitablePointable,
+                        openRecordVisitablePointable.getInputRecordType(), Boolean.FALSE);
+            }
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        eval0.evaluate(tuple, inputRecordPointable);
+        final byte[] data = inputRecordPointable.getByteArray();
+        final int offset = inputRecordPointable.getStartOffset();
+        final byte typeTag = data[offset];
+        if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        final ARecordVisitablePointable inputRecordVisitablePointable = getInputRecordVisitablePointable();
+        final List<IVisitablePointable> recValues = inputRecordVisitablePointable.getFieldValues();
+        if (recValues.size() != 1) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        writeValue(recValues.get(0));
+        result.set(resultStorage);
+    }
+
+    private boolean hasDerivedType(IAType[] types) {
+        for (IAType type : types) {
+            if (type.getTypeTag().isDerivedType()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
+        inputRecordVisitable.set(inputRecordPointable);
+        if (requiresCast) {
+            return castToOpenRecord();
+        }
+        return inputRecordVisitable;
+    }
+
+    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
+        inputRecordVisitable.accept(castVisitor, castVisitorArg);
+        return openRecordVisitablePointable;
+    }
+
+    private void writeValue(IVisitablePointable value) throws HyracksDataException {
+        try {
+            resultOutput.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesDescriptor.java
new file mode 100644
index 0000000..6e7bf03b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class RecordValuesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RecordValuesDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private ARecordType recordType;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        recordType = (ARecordType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+                return new RecordValuesEvaluator(args[0].createScalarEvaluator(ctx), recordType);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RECORD_VALUES;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesEvaluator.java
new file mode 100644
index 0000000..83de818
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordValuesEvaluator.java
@@ -0,0 +1,85 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class RecordValuesEvaluator implements IScalarEvaluator {
+
+    private final IPointable inputRecordPointable = new VoidPointable();
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput resultOutput = resultStorage.getDataOutput();
+    private final IScalarEvaluator eval0;
+    private OrderedListBuilder listBuilder;
+    private ARecordVisitablePointable openRecordPointable;
+    private CastTypeEvaluator inputRecordCaster;
+
+    RecordValuesEvaluator(IScalarEvaluator eval0, ARecordType recordType) {
+        this.eval0 = eval0;
+        if (recordType != null) {
+            openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+            inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, recordType, eval0);
+            listBuilder = new OrderedListBuilder();
+        }
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        eval0.evaluate(tuple, inputRecordPointable);
+        final ATypeTag inputTypeTag = PointableHelper.getTypeTag(inputRecordPointable);
+        if (inputTypeTag != ATypeTag.OBJECT) {
+            PointableHelper.setNull(result);
+            return;
+        }
+        inputRecordCaster.evaluate(tuple, inputRecordPointable);
+        resultStorage.reset();
+        buildOutputList();
+        result.set(resultStorage);
+    }
+
+    private void buildOutputList() throws HyracksDataException {
+        listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+        openRecordPointable.set(inputRecordPointable);
+        final List<IVisitablePointable> fieldValues = openRecordPointable.getFieldValues();
+        for (int i = 0, valuesCount = fieldValues.size(); i < valuesCount; i++) {
+            listBuilder.addItem(fieldValues.get(i));
+        }
+        listBuilder.write(resultOutput, true);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
index 0a1bcf5..5e396a3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
@@ -51,13 +51,11 @@
 
                 return new IScalarEvaluator() {
 
-                    protected final IntervalLogic il = new IntervalLogic();
+                    protected final IntervalLogic il = new IntervalLogic(sourceLoc);
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     private DataOutput out = resultStorage.getDataOutput();
-                    private TaggedValuePointable argPtr0 =
-                            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-                    private TaggedValuePointable argPtr1 =
-                            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+                    private TaggedValuePointable argPtr0 = TaggedValuePointable.FACTORY.createPointable();
+                    private TaggedValuePointable argPtr1 = TaggedValuePointable.FACTORY.createPointable();
                     private AIntervalPointable interval0 =
                             (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
                     private AIntervalPointable interval1 =
@@ -78,12 +76,12 @@
 
                         byte typeTag0 = argPtr0.getTag();
                         if (typeTag0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, typeTag0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, typeTag0,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         byte typeTag1 = argPtr0.getTag();
                         if (typeTag1 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, typeTag1,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, typeTag1,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
 
@@ -91,7 +89,7 @@
                         argPtr1.getValue(interval1);
 
                         if (typeTag0 != typeTag1) {
-                            throw new IncompatibleTypeException(getIdentifier(), typeTag0, typeTag1);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), typeTag0, typeTag1);
                         }
 
                         ABoolean res = compareIntervals(il, interval0, interval1) ? ABoolean.TRUE : ABoolean.FALSE;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
index 38d890c..89865ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
@@ -95,12 +95,12 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
 
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -109,7 +109,7 @@
                                     utf8Ptr.getCharStartOffset());
 
                             if (!calInstance.validateTimeZone(timezone)) {
-                                throw new InvalidDataFormatException(getIdentifier(), "timezone");
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), "timezone");
                             }
 
                             long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
index 8c09e47..b57961a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
@@ -93,12 +93,12 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             }
 
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
@@ -106,7 +106,7 @@
                             int timezone = ATimeParserFactory.parseTimezonePart(bytes1, utf8Ptr.getCharStartOffset());
 
                             if (!calInstance.validateTimeZone(timezone)) {
-                                throw new InvalidDataFormatException(getIdentifier(), "timezone");
+                                throw new InvalidDataFormatException(sourceLoc, getIdentifier(), "timezone");
                             }
 
                             int chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
index 3dab641..6244087 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDuartionFromDateDescriptor.java
@@ -94,12 +94,12 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
 
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index 867d030..5ee87fa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -111,12 +111,12 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
 
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
index cfac2e9..cc449b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
@@ -84,7 +84,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
index f70a6e2..43764ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
@@ -93,11 +93,11 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
index 87e83d7..9e16fc3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
@@ -104,7 +104,7 @@
                                 aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1));
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
index abf4016..47705a2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
@@ -105,7 +105,7 @@
                                 aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1) * 1000l);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
index 7bb4265..59006ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
@@ -94,7 +94,7 @@
                             chronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1)
                                     * GregorianCalendarSystem.CHRONON_OF_DAY;
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
index cb26c58..172e2e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationComparatorDescriptor.java
@@ -83,17 +83,17 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
                         if ((ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1) != 0)
                                 || (ADurationSerializerDeserializer.getYearMonth(bytes1, offset1 + 1) != 0)) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
index 9b29ca8..8739fe5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
@@ -84,11 +84,11 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
index d8be601..7761fb8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
@@ -85,7 +85,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
                         long chrononStart = AIntervalSerializerDeserializer.getIntervalStart(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
index f1987ca..b81ae06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
@@ -100,7 +100,7 @@
                                 aDuration.setValue(0, AInt64SerializerDeserializer.getLong(bytes, offset + 1));
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                         ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
                                         ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
index 9410e6d..733c349 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
@@ -84,7 +84,7 @@
                         int offset = argPtr0.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index 4bd3ede..9374aba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -62,13 +62,11 @@
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
                 return new IScalarEvaluator() {
 
-                    protected final IntervalLogic il = new IntervalLogic();
+                    protected final IntervalLogic il = new IntervalLogic(sourceLoc);
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     private DataOutput out = resultStorage.getDataOutput();
-                    private TaggedValuePointable argPtr0 =
-                            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
-                    private TaggedValuePointable argPtr1 =
-                            (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+                    private TaggedValuePointable argPtr0 = TaggedValuePointable.FACTORY.createPointable();
+                    private TaggedValuePointable argPtr1 = TaggedValuePointable.FACTORY.createPointable();
                     private AIntervalPointable interval0 =
                             (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
                     private AIntervalPointable interval1 =
@@ -100,7 +98,8 @@
                             byte intervalType1 = interval1.getType();
 
                             if (intervalType0 != intervalType1) {
-                                throw new IncompatibleTypeException(getIdentifier(), intervalType0, intervalType1);
+                                throw new IncompatibleTypeException(sourceLoc, getIdentifier(), intervalType0,
+                                        intervalType1);
                             }
 
                             if (il.overlaps(interval0, interval1) || il.overlappedBy(interval0, interval1)
@@ -115,10 +114,10 @@
                             result.set(resultStorage);
                             return;
                         } else if (type0 != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, type0,
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, type0,
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         } else {
-                            throw new IncompatibleTypeException(getIdentifier(), type0, type1);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), type0, type1);
                         }
                     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
index fd58d6c..03faa96 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
@@ -81,7 +81,7 @@
                         byte[] bytes0 = argPtr0.getByteArray();
                         int offset0 = argPtr0.getStartOffset();
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         aYearMonthDuration.setMonths(ADurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
index 5694c27..ac2e06a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
@@ -117,7 +117,7 @@
                                 chrononToBin = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -127,7 +127,8 @@
                         ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
 
                         if (type0 != type1) {
-                            throw new IncompatibleTypeException(getIdentifier(), bytes0[offset0], bytes1[offset1]);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), bytes0[offset0],
+                                    bytes1[offset1]);
                         }
 
                         long chrononToStart = 0;
@@ -143,7 +144,7 @@
                                 chrononToStart = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -169,11 +170,11 @@
                                         + ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
 
                                 if (binIndex > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
 
                                 if (binIndex < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
 
                                 break;
@@ -209,7 +210,7 @@
                                 break;
                             case TIME:
                                 if (yearMonth != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
                                 binStartChronon = DurationArithmeticOperations.addDuration(chrononToStart,
@@ -224,7 +225,7 @@
                                         yearMonth * ((int) binIndex + 1), dayTime * (binIndex + 1), false);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
index 8f447d0..db0046c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
@@ -23,18 +23,23 @@
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 
 public class IntervalLogic implements Serializable {
 
     private static final long serialVersionUID = 1L;
-    private final ComparisonHelper ch = new ComparisonHelper();
+    private final ComparisonHelper ch;
     private final transient IPointable s1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable s2 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e2 = VoidPointable.FACTORY.createPointable();
 
+    public IntervalLogic(SourceLocation sourceLoc) {
+        ch = new ComparisonHelper(sourceLoc);
+    }
+
     public boolean validateInterval(AIntervalPointable ip1) throws HyracksDataException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
index c3bf58d..8e01e11 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
@@ -80,7 +80,7 @@
                         byte[] bytes = argPtr0.getByteArray();
                         int offset = argPtr0.getStartOffset();
                         if (bytes[offset] != ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
                         aInt64.setValue(ADayTimeDurationSerializerDeserializer.getDayTime(bytes, offset + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 3d595c3..0c78c7c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -75,13 +75,13 @@
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-                        resultStorage.reset();;
+                        resultStorage.reset();
                         eval0.evaluate(tuple, argPtr0);
                         byte[] bytes = argPtr0.getByteArray();
                         int offset = argPtr0.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
                         aInt64.setValue(AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, offset + 1));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
index 9710e3f..c8f73b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
@@ -125,7 +125,7 @@
                                 intervalStart = intervalStart * GregorianCalendarSystem.CHRONON_OF_DAY;
                             }
                         } else {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                         }
 
@@ -134,7 +134,8 @@
                         int offset1 = argPtr1.getStartOffset();
                         ATypeTag type1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
                         if (intervalTypeTag != bytes1[offset1]) {
-                            throw new IncompatibleTypeException(getIdentifier(), intervalTypeTag, bytes1[offset1]);
+                            throw new IncompatibleTypeException(sourceLoc, getIdentifier(), intervalTypeTag,
+                                    bytes1[offset1]);
                         }
 
                         long anchorTime;
@@ -150,7 +151,7 @@
                                 anchorTime = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
@@ -178,11 +179,11 @@
                                         + ((totalMonths < 0 && totalMonths % yearMonth != 0) ? -1 : 0);
 
                                 if (firstBinIndex > Integer.MAX_VALUE) {
-                                    throw new OverflowException(getIdentifier());
+                                    throw new OverflowException(sourceLoc, getIdentifier());
                                 }
 
                                 if (firstBinIndex < Integer.MIN_VALUE) {
-                                    throw new UnderflowException(getIdentifier());
+                                    throw new UnderflowException(sourceLoc, getIdentifier());
                                 }
                                 break;
 
@@ -195,7 +196,7 @@
                                         + ((totalChronon < 0 && totalChronon % dayTime != 0) ? -1 : 0);
                                 break;
                             default:
-                                throw new TypeMismatchException(getIdentifier(), 2, bytes2[offset2],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 2, bytes2[offset2],
                                         ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
                         }
@@ -233,7 +234,7 @@
 
                             } else if (intervalTypeTag == ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
                                 if (yearMonth != 0) {
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -248,7 +249,7 @@
 
                                 if (binStartChronon < 0 || binStartChronon >= GregorianCalendarSystem.CHRONON_OF_DAY) {
                                     // avoid the case where a time bin is before 00:00:00 or no early than 24:00:00
-                                    throw new InvalidDataFormatException(getIdentifier(),
+                                    throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                             ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                 }
 
@@ -272,7 +273,7 @@
                                     }
 
                                     if (binEndChronon < binStartChronon) {
-                                        throw new InvalidDataFormatException(getIdentifier(),
+                                        throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                                 ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG);
                                     }
                                 }
@@ -292,7 +293,7 @@
                                     binOffset++;
                                 } while (binEndChronon < intervalEnd);
                             } else {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_TIME_TYPE_TAG,
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
index f8d5cf7..da8d732 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -104,11 +104,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
 
@@ -136,7 +136,8 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         aDate.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
                         dateSerde.serialize(aDate, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
index 8bb1a20..ab19f9e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -96,11 +96,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
@@ -127,7 +127,7 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         aDateTime.setValue(aInt64.getLongValue());
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
index 6b9488d..af85b5b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -96,11 +96,11 @@
                         int len1 = argPtr1.getLength();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                         }
                         utf8Ptr.set(bytes0, offset0 + 1, len0 - 1);
@@ -127,7 +127,8 @@
                             formatStart += formatLength + 1;
                         }
                         if (!processSuccessfully) {
-                            throw new InvalidDataFormatException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
+                                    ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
                         aTime.setValue((int) aInt64.getLongValue());
                         timeSerde.serialize(aTime, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
index 98fe013..c0b155d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -89,11 +89,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             long chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
index 5d2f088..066bef0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -89,11 +89,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
                             long chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
index 48a5382..2ad3642 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -88,11 +88,11 @@
 
                         try {
                             if (bytes0[offset0] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                         ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                             }
                             if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-                                throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                                throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                         ATypeTag.SERIALIZED_STRING_TYPE_TAG);
                             }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
index 273cba4..3010540 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
@@ -89,7 +89,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
index 3dc449d..a069347 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
@@ -82,7 +82,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATE_TYPE_TAG);
                         }
                         long dateChronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
index 2dc57c5..4003195 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
@@ -80,7 +80,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
index 34fedb6..b8adc06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
@@ -86,7 +86,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != SERIALIZED_DATETIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
                         }
                         long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
index 0909764..c1932f6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
@@ -82,7 +82,7 @@
                         int offset = argPtr.getStartOffset();
 
                         if (bytes[offset] != ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes[offset],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
                                     ATypeTag.SERIALIZED_TIME_TYPE_TAG);
                         }
                         long timeChronon = ATimeSerializerDeserializer.getChronon(bytes, offset + 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
index 5748956..f7407c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationComparatorDescriptor.java
@@ -83,17 +83,17 @@
                         int offset1 = argPtr1.getStartOffset();
 
                         if (bytes0[offset0] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
                         if (bytes1[offset1] != ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
-                            throw new TypeMismatchException(getIdentifier(), 1, bytes1[offset1],
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
                                     ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
                         }
 
                         if ((ADurationSerializerDeserializer.getDayTime(bytes0, offset0 + 1) != 0)
                                 || (ADurationSerializerDeserializer.getDayTime(bytes1, offset1 + 1) != 0)) {
-                            throw new InvalidDataFormatException(getIdentifier(),
+                            throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
                                     ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
                         }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
new file mode 100644
index 0000000..7ed1ce5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
@@ -0,0 +1,80 @@
+/*
+ * 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.asterix.runtime.evaluators.functions.utils;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.security.SecureRandom;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.DataUtils;
+import org.apache.hyracks.data.std.util.GrowableArray;
+
+public final class RandomHelper {
+
+    private final SecureRandom random = new SecureRandom();
+
+    private final GrowableArray seed;
+
+    private final AMutableDouble aDouble = new AMutableDouble(0);
+
+    @SuppressWarnings("rawtypes")
+    private ISerializerDeserializer doubleSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private final DataOutput dataOutput = resultStorage.getDataOutput();
+
+    public RandomHelper(boolean withSeed) {
+        seed = withSeed ? new GrowableArray(8) : null;
+    }
+
+    public void setSeed(byte[] bytes, int offset, int length) throws HyracksDataException {
+        if (seed == null) {
+            throw new IllegalStateException();
+        }
+
+        boolean sameSeed =
+                seed.getLength() == length && DataUtils.equalsInRange(seed.getByteArray(), 0, bytes, offset, length);
+
+        if (!sameSeed) {
+            try {
+                seed.reset();
+                seed.append(bytes, offset, length);
+                random.setSeed(seed.getByteArray());
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+
+    public void nextDouble(IPointable resultPointable) throws HyracksDataException {
+        aDouble.setValue(random.nextDouble());
+        resultStorage.reset();
+        doubleSerde.serialize(aDouble, dataOutput);
+        resultPointable.set(resultStorage);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RegExpMatcher.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RegExpMatcher.java
index 8d238df..77622d3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RegExpMatcher.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RegExpMatcher.java
@@ -166,6 +166,23 @@
      * @return a new string with contained regular expressions replaced.
      */
     public String replace(UTF8StringPointable replaceStrPtr) {
+        return replace(replaceStrPtr, Integer.MAX_VALUE);
+    }
+
+    /**
+     * Replaces the appearances of a regular expression defined pattern in a source string with a given
+     * replacement string.
+     *
+     * @param replaceStrPtr
+     *            , the string for replacing the regular expression.
+     * @param replaceLimit
+     *            , the maximum number of replacements to make
+     * @return a new string with contained regular expressions replaced.
+     */
+    public String replace(UTF8StringPointable replaceStrPtr, int replaceLimit) {
+        if (replaceLimit < 0) {
+            replaceLimit = Integer.MAX_VALUE;
+        }
         // Sets up a new replacement string if necessary.
         final boolean newReplace =
                 replaceStrPtr != null && (replaceStr == null || lastReplaceStrPtr.compareTo(replaceStrPtr) != 0);
@@ -173,10 +190,9 @@
             StringEvaluatorUtils.copyResetUTF8Pointable(replaceStrPtr, lastReplaceStorage, lastReplaceStrPtr);
             replaceStr = replaceStrPtr.toString();
         }
-
         // Does the actual replacement.
         resultBuf.setLength(0);
-        while (matcher.find()) {
+        for (int i = 0; i < replaceLimit && matcher.find(); i++) {
             matcher.appendReplacement(resultBuf, replaceStr);
         }
         matcher.appendTail(resultBuf);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/ListDeepEqualityChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/ListDeepEqualityChecker.java
index 6341b79..e510441 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/ListDeepEqualityChecker.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/visitors/ListDeepEqualityChecker.java
@@ -18,10 +18,8 @@
  */
 package org.apache.asterix.runtime.evaluators.visitors;
 
-import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.pointables.AListVisitablePointable;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
 import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
index 5feb87f..ebb7222 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/IncompatibleTypeException.java
@@ -23,19 +23,24 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class IncompatibleTypeException extends RuntimeDataException {
+    private static final long serialVersionUID = 3967461820552150509L;
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
-    public IncompatibleTypeException(FunctionIdentifier fid, byte typeTagLeft, byte typeTagRight) {
-        super(ErrorCode.TYPE_INCOMPATIBLE, fid.getName(),
+    public IncompatibleTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
     }
 
     // Incompatible input parameters, e.g., "1.0" > 1.0
-    public IncompatibleTypeException(String functionName, byte typeTagLeft, byte typeTagRight) {
-        super(ErrorCode.TYPE_INCOMPATIBLE, functionName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+    public IncompatibleTypeException(SourceLocation sourceLoc, String functionName, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, functionName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
index acb0495..bb94ca6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
@@ -23,20 +23,23 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class InvalidDataFormatException extends RuntimeDataException {
+    private static final long serialVersionUID = 7927137063741221011L;
 
-    public InvalidDataFormatException(FunctionIdentifier fid, byte expectedTypeTag) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(),
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, byte expectedTypeTag) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag));
     }
 
-    public InvalidDataFormatException(FunctionIdentifier fid, String expectedType) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(), expectedType);
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, String expectedType) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), expectedType);
     }
 
-    public InvalidDataFormatException(FunctionIdentifier fid, Throwable cause, byte expectedTypeTag) {
-        super(ErrorCode.INVALID_FORMAT, fid.getName(), cause, expectedTypeTag);
+    public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, Throwable cause,
+            byte expectedTypeTag) {
+        super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), cause, expectedTypeTag);
         addSuppressed(cause);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
index 3690236..1d84a60 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/OverflowException.java
@@ -22,17 +22,13 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class OverflowException extends RuntimeDataException {
+    private static final long serialVersionUID = -9108000688790364894L;
 
     // Overflow.
-    public OverflowException(FunctionIdentifier fid) {
-        super(ErrorCode.OVERFLOW, fid.getName());
+    public OverflowException(SourceLocation sourceLoc, FunctionIdentifier fid) {
+        super(ErrorCode.OVERFLOW, sourceLoc, fid.getName());
     }
-
-    // Overflow.
-    public OverflowException(String functionName) {
-        super(ErrorCode.OVERFLOW, functionName);
-    }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
index f6cf1c7..d5472d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
@@ -26,19 +26,57 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class TypeMismatchException extends RuntimeDataException {
+    private static final long serialVersionUID = -668005043013338591L;
 
-    // Parameter type mistmatch.
+    // Function parameter type mismatch.
     public TypeMismatchException(FunctionIdentifier fid, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
-        super(ErrorCode.TYPE_MISMATCH, fid.getName(), indexToPosition(i), toExpectedTypeString(expectedTypeTags),
+        super(ErrorCode.TYPE_MISMATCH_FUNCTION, fid.getName(), indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
 
-    // Parameter type mistmatch.
+    // Function parameter type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, byte actualTypeTag,
+            byte... expectedTypeTags) {
+        super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Function parameter type mismatch.
     public TypeMismatchException(String functionName, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
-        super(ErrorCode.TYPE_MISMATCH, functionName, indexToPosition(i), toExpectedTypeString(expectedTypeTags),
+        super(ErrorCode.TYPE_MISMATCH_FUNCTION, functionName, indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
 
+    // Function parameter type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, byte actualTypeTag,
+            byte... expectedTypeTags) {
+        super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, functionName, indexToPosition(i),
+                toExpectedTypeString(expectedTypeTags),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Function parameter type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, byte actualTypeTag,
+            String expectedType) {
+        super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i), expectedType,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Generic type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, byte actualTypeTag, byte... expectedTypeTags) {
+        super(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, toExpectedTypeString(expectedTypeTags),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Generic type mismatch.
+    public TypeMismatchException(SourceLocation sourceLoc, byte actualTypeTag, String expectedType) {
+        super(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, expectedType,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
index 6471cbd..926367c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnderflowException.java
@@ -22,17 +22,13 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnderflowException extends RuntimeDataException {
+    private static final long serialVersionUID = -8808136503804136973L;
 
     // Underflow.
-    public UnderflowException(FunctionIdentifier fid) {
-        super(ErrorCode.UNDERFLOW, fid.getName());
+    public UnderflowException(SourceLocation sourceLoc, FunctionIdentifier fid) {
+        super(ErrorCode.UNDERFLOW, sourceLoc, fid.getName());
     }
-
-    // Underflow.
-    public UnderflowException(String functionName) {
-        super(ErrorCode.UNDERFLOW, functionName);
-    }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
index f634649..680f20e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedItemTypeException.java
@@ -23,16 +23,20 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedItemTypeException extends RuntimeDataException {
+    private static final long serialVersionUID = -3443141044058396191L;
 
     // Unsupported item type.
-    public UnsupportedItemTypeException(FunctionIdentifier fid, byte itemTypeTag) {
-        super(ErrorCode.TYPE_ITEM, fid.getName(), EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte itemTypeTag) {
+        super(ErrorCode.TYPE_ITEM, sourceLoc, fid.getName(),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
     }
 
     // Unsupported item type.
-    public UnsupportedItemTypeException(String functionName, byte itemTypeTag) {
-        super(ErrorCode.TYPE_ITEM, functionName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
+    public UnsupportedItemTypeException(SourceLocation sourceLoc, String functionName, byte itemTypeTag) {
+        super(ErrorCode.TYPE_ITEM, sourceLoc, functionName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTypeTag));
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
index 29b3819..91a571e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/UnsupportedTypeException.java
@@ -23,12 +23,14 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class UnsupportedTypeException extends RuntimeDataException {
+    private static final long serialVersionUID = 1627870063930883067L;
 
     // Unsupported input type.
-    public UnsupportedTypeException(FunctionIdentifier fid, byte actualTypeTag) {
-        super(ErrorCode.TYPE_UNSUPPORTED, fid.getName(),
+    public UnsupportedTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, fid.getName(),
                 EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
 
@@ -36,4 +38,10 @@
     public UnsupportedTypeException(String funcName, byte actualTypeTag) {
         super(ErrorCode.TYPE_UNSUPPORTED, funcName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
     }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, String funcName, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, funcName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index e237319..8b25e06 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -79,6 +79,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
 public class NonTaggedDataFormat implements IDataFormat {
@@ -120,7 +121,7 @@
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory getFieldAccessEvaluatorFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn) throws AlgebricksException {
+            List<String> fldName, int recordColumn, SourceLocation sourceLoc) throws AlgebricksException {
         IScalarEvaluatorFactory recordEvalFactory = new ColumnAccessEvalFactory(recordColumn);
 
         if (fldName.size() == 1) {
@@ -141,6 +142,7 @@
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                     IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     return fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -157,6 +159,7 @@
                 IScalarEvaluatorFactory fldNameEvalFactory =
                         new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                 IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                fDesc.setSourceLocation(sourceLoc);
                 return fDesc.createEvaluatorFactory(
                         new IScalarEvaluatorFactory[] { recordEvalFactory, fldNameEvalFactory });
             }
@@ -164,6 +167,7 @@
 
         if (fldName.size() > 1) {
             IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             return fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
         }
@@ -174,10 +178,10 @@
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory[] createMBRFactory(IFunctionManager functionManager, ARecordType recType,
-            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR)
-            throws AlgebricksException {
+            List<String> fldName, int recordColumn, int dimension, List<String> filterFieldName, boolean isPointMBR,
+            SourceLocation sourceLoc) throws AlgebricksException {
         IScalarEvaluatorFactory evalFactory =
-                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn);
+                getFieldAccessEvaluatorFactory(functionManager, recType, fldName, recordColumn, sourceLoc);
         int numOfFields = isPointMBR ? dimension : dimension * 2;
         IScalarEvaluatorFactory[] evalFactories =
                 new IScalarEvaluatorFactory[numOfFields + (filterFieldName == null ? 0 : 1)];
@@ -209,7 +213,7 @@
         }
         if (filterFieldName != null) {
             evalFactories[numOfFields] =
-                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn);
+                    getFieldAccessEvaluatorFactory(functionManager, recType, filterFieldName, recordColumn, sourceLoc);
         }
         return evalFactories;
     }
@@ -217,7 +221,8 @@
     @SuppressWarnings("unchecked")
     @Override
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
-            IFunctionManager functionManager, ARecordType recType, List<String> fldName) throws AlgebricksException {
+            IFunctionManager functionManager, ARecordType recType, List<String> fldName, SourceLocation sourceLoc)
+            throws AlgebricksException {
         String[] names = recType.getFieldNames();
         int n = names.length;
         if (fldName.size() > 1) {
@@ -237,6 +242,7 @@
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
                     IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     IScalarEvaluatorFactory evalFactory = fDesc.createEvaluatorFactory(
                             new IScalarEvaluatorFactory[] { recordEvalFactory, fldIndexEvalFactory });
@@ -246,6 +252,7 @@
                     ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
                             new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
                             new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    partitionFun.setSourceLocation(sourceLoc);
                     return new Triple<>(evalFactory, partitionFun, recType.getFieldTypes()[i]);
                 }
             }
@@ -261,6 +268,7 @@
                 throw new AlgebricksException(e);
             }
             IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             IScalarEvaluatorFactory evalFactory =
                     fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
@@ -269,6 +277,7 @@
             ScalarFunctionCallExpression partitionFun = new ScalarFunctionCallExpression(finfoAccess,
                     new MutableObject<>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
                     new MutableObject<>(new ConstantExpression(new AsterixConstantValue(as))));
+            partitionFun.setSourceLocation(sourceLoc);
             return new Triple<>(evalFactory, partitionFun, recType.getSubFieldType(fldName));
         }
         throw new AlgebricksException("Could not find field " + fldName + " in the schema.");
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index b05dbed..d885ce5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -104,7 +104,12 @@
 import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanOrEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.LessThanDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.LessThanOrEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.MissingIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NanIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NegInfIfEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.NotEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NullIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.PosInfIfEqualsDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
@@ -138,6 +143,27 @@
 import org.apache.asterix.runtime.evaluators.constructors.UnorderedListConstructorDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayAppendDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayConcatDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayContainsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayDistinctDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayFlattenDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayIfNullDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayInsertDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayIntersectDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayPositionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayPrependDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayPutDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayRangeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayRemoveDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayRepeatDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayReplaceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayReverseDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySortDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayStarDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffnDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayUnionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastTypeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastTypeLaxDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CheckUnknownDescriptor;
@@ -180,9 +206,10 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericATanDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericAddDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericCosDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDegreesDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDivDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericExpDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
@@ -190,6 +217,8 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericLogDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericPowerDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRadiansDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
@@ -201,6 +230,8 @@
 import org.apache.asterix.runtime.evaluators.functions.NumericTruncDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
@@ -226,10 +257,11 @@
 import org.apache.asterix.runtime.evaluators.functions.StringRegExpPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringRegExpPositionWithFlagDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringRegExpReplaceDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.StringRegExpReplaceWithFlagsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringRegExpReplaceWithFlagDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringRepeatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithLimitDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringReverseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringSplitDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
@@ -249,6 +281,7 @@
 import org.apache.asterix.runtime.evaluators.functions.ToNumberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToObjectDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToStringDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.TreatAsIntegerDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.UUIDDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryLengthDescriptor;
@@ -263,6 +296,8 @@
 import org.apache.asterix.runtime.evaluators.functions.records.FieldAccessNestedDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.GetRecordFieldValueDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.GetRecordFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.PairsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordAddDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordAddFieldsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordConcatStrictDescriptor;
@@ -270,7 +305,13 @@
 import org.apache.asterix.runtime.evaluators.functions.records.RecordMergeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordNamesDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordPairsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordPutDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.records.RecordRemoveFieldsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordRenameDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordReplaceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordUnwrapDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.records.RecordValuesDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustDateTimeForTimeZoneDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.AdjustTimeForTimeZoneDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDuartionFromDateDescriptor;
@@ -333,6 +374,7 @@
  * This class holds a list of function descriptor factories.
  */
 public final class FunctionCollection implements IFunctionCollection {
+    private static final long serialVersionUID = -8308873930697425307L;
 
     private static final String FACTORY = "FACTORY";
 
@@ -351,7 +393,30 @@
     public static FunctionCollection createDefaultFunctionCollection() {
         FunctionCollection fc = new FunctionCollection();
 
-        // unnesting function
+        // array functions
+        fc.add(ArrayRemoveDescriptor.FACTORY);
+        fc.add(ArrayPutDescriptor.FACTORY);
+        fc.add(ArrayPrependDescriptor.FACTORY);
+        fc.add(ArrayAppendDescriptor.FACTORY);
+        fc.add(ArrayInsertDescriptor.FACTORY);
+        fc.addGenerated(ArrayPositionDescriptor.FACTORY);
+        fc.addGenerated(ArrayRepeatDescriptor.FACTORY);
+        fc.addGenerated(ArrayContainsDescriptor.FACTORY);
+        fc.addGenerated(ArrayReverseDescriptor.FACTORY);
+        fc.addGenerated(ArraySortDescriptor.FACTORY);
+        fc.addGenerated(ArrayDistinctDescriptor.FACTORY);
+        fc.addGenerated(ArrayUnionDescriptor.FACTORY);
+        fc.addGenerated(ArrayIntersectDescriptor.FACTORY);
+        fc.addGenerated(ArrayIfNullDescriptor.FACTORY);
+        fc.addGenerated(ArrayConcatDescriptor.FACTORY);
+        fc.addGenerated(ArrayRangeDescriptor.FACTORY);
+        fc.addGenerated(ArrayFlattenDescriptor.FACTORY);
+        fc.add(ArrayReplaceDescriptor.FACTORY);
+        fc.addGenerated(ArraySymDiffDescriptor.FACTORY);
+        fc.addGenerated(ArraySymDiffnDescriptor.FACTORY);
+        fc.addGenerated(ArrayStarDescriptor.FACTORY);
+
+        // unnesting functions
         fc.add(TidRunningAggregateDescriptor.FACTORY);
         fc.add(ScanCollectionDescriptor.FACTORY);
         fc.add(RangeDescriptor.FACTORY);
@@ -460,6 +525,7 @@
         fc.add(CreateUUIDDescriptor.FACTORY);
         fc.add(UUIDDescriptor.FACTORY);
         fc.add(CreateQueryUIDDescriptor.FACTORY);
+        fc.add(RandomDescriptor.FACTORY);
         fc.add(CurrentDateDescriptor.FACTORY);
         fc.add(CurrentTimeDescriptor.FACTORY);
         fc.add(CurrentDateTimeDescriptor.FACTORY);
@@ -481,10 +547,11 @@
         fc.addGenerated(NumericUnaryMinusDescriptor.FACTORY);
         fc.addGenerated(NumericAddDescriptor.FACTORY);
         fc.addGenerated(NumericDivideDescriptor.FACTORY);
+        fc.addGenerated(NumericDivDescriptor.FACTORY);
         fc.addGenerated(NumericMultiplyDescriptor.FACTORY);
         fc.addGenerated(NumericSubDescriptor.FACTORY);
         fc.addGenerated(NumericModuloDescriptor.FACTORY);
-        fc.addGenerated(NumericCaretDescriptor.FACTORY);
+        fc.addGenerated(NumericPowerDescriptor.FACTORY);
         fc.addGenerated(NotDescriptor.FACTORY);
         fc.addGenerated(LenDescriptor.FACTORY);
         fc.addGenerated(NumericAbsDescriptor.FACTORY);
@@ -496,6 +563,8 @@
         fc.addGenerated(NumericACosDescriptor.FACTORY);
         fc.addGenerated(NumericASinDescriptor.FACTORY);
         fc.addGenerated(NumericATanDescriptor.FACTORY);
+        fc.addGenerated(NumericDegreesDescriptor.FACTORY);
+        fc.addGenerated(NumericRadiansDescriptor.FACTORY);
         fc.addGenerated(NumericCosDescriptor.FACTORY);
         fc.addGenerated(NumericSinDescriptor.FACTORY);
         fc.addGenerated(NumericTanDescriptor.FACTORY);
@@ -515,6 +584,13 @@
         fc.addGenerated(LessThanOrEqualsDescriptor.FACTORY);
         fc.addGenerated(NotEqualsDescriptor.FACTORY);
 
+        // If-Equals functions
+        fc.addGenerated(MissingIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NullIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NanIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(PosInfIfEqualsDescriptor.FACTORY);
+        fc.addGenerated(NegInfIfEqualsDescriptor.FACTORY);
+
         // Binary functions
         fc.addGenerated(BinaryLengthDescriptor.FACTORY);
         fc.addGenerated(ParseBinaryDescriptor.FACTORY);
@@ -549,7 +625,7 @@
         fc.addGenerated(StringRegExpPositionDescriptor.FACTORY);
         fc.addGenerated(StringRegExpPositionWithFlagDescriptor.FACTORY);
         fc.addGenerated(StringRegExpReplaceDescriptor.FACTORY);
-        fc.addGenerated(StringRegExpReplaceWithFlagsDescriptor.FACTORY);
+        fc.addGenerated(StringRegExpReplaceWithFlagDescriptor.FACTORY);
         fc.addGenerated(StringInitCapDescriptor.FACTORY);
         fc.addGenerated(StringTrimDescriptor.FACTORY);
         fc.addGenerated(StringLTrimDescriptor.FACTORY);
@@ -561,6 +637,7 @@
         fc.addGenerated(StringRepeatDescriptor.FACTORY);
         fc.addGenerated(StringReplaceDescriptor.FACTORY);
         fc.addGenerated(StringReplaceWithLimitDescriptor.FACTORY);
+        fc.addGenerated(StringReverseDescriptor.FACTORY);
         fc.addGenerated(StringSplitDescriptor.FACTORY);
 
         // Constructors
@@ -621,6 +698,14 @@
         fc.addGenerated(RecordRemoveFieldsDescriptor.FACTORY);
         fc.addGenerated(RecordLengthDescriptor.FACTORY);
         fc.addGenerated(RecordNamesDescriptor.FACTORY);
+        fc.addGenerated(RecordRemoveDescriptor.FACTORY);
+        fc.addGenerated(RecordRenameDescriptor.FACTORY);
+        fc.addGenerated(RecordUnwrapDescriptor.FACTORY);
+        fc.add(RecordReplaceDescriptor.FACTORY);
+        fc.add(RecordAddDescriptor.FACTORY);
+        fc.add(RecordPutDescriptor.FACTORY);
+        fc.addGenerated(RecordValuesDescriptor.FACTORY);
+        fc.addGenerated(PairsDescriptor.FACTORY);
 
         // Spatial and temporal type accessors
         fc.addGenerated(TemporalYearAccessor.FACTORY);
@@ -707,6 +792,8 @@
         fc.addGenerated(ToObjectDescriptor.FACTORY);
         fc.addGenerated(ToStringDescriptor.FACTORY);
 
+        fc.addGenerated(TreatAsIntegerDescriptor.FACTORY);
+
         // Cast function
         fc.addGenerated(CastTypeDescriptor.FACTORY);
         fc.addGenerated(CastTypeLaxDescriptor.FACTORY);
@@ -714,6 +801,9 @@
         // Record function
         fc.addGenerated(RecordPairsDescriptor.FACTORY);
 
+        // Other functions
+        fc.addGenerated(RandomWithSeedDescriptor.FACTORY);
+
         ServiceLoader.load(IFunctionRegistrant.class).iterator().forEachRemaining(c -> c.register(fc));
         return fc;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index e5a4301..b9c58c7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -79,6 +79,22 @@
         }
     };
 
+    /** Sets the types of the function arguments */
+    public static final IFunctionTypeInferer SET_ARGUMENTS_TYPE = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+            IAType[] argsTypes = new IAType[fce.getArguments().size()];
+            int i = 0;
+            for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
+                argsTypes[i] = TypeComputeUtils.getActualType((IAType) context.getType(arg.getValue()));
+                i++;
+            }
+            fd.setImmutableStates((Object[]) argsTypes);
+        }
+    };
+
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
         public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
index 8ea1fa7..b123a5e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
@@ -26,12 +26,15 @@
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
 import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 // To avoid the computation cost for checking the capacity constraint for each node,
 // currently the admit/allocation decisions are based on the aggregated resource information.
 // TODO(buyingyi): investigate partition-aware resource control.
 public class JobCapacityController implements IJobCapacityController {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     private final IResourceManager resourceManager;
 
     public JobCapacityController(IResourceManager resourceManager) {
@@ -71,6 +74,16 @@
         int aggregatedNumCores = currentCapacity.getAggregatedCores();
         currentCapacity.setAggregatedMemoryByteSize(aggregatedMemoryByteSize + reqAggregatedMemoryByteSize);
         currentCapacity.setAggregatedCores(aggregatedNumCores + reqAggregatedNumCores);
+        ensureMaxCapacity();
     }
 
+    private void ensureMaxCapacity() {
+        final IClusterCapacity currentCapacity = resourceManager.getCurrentCapacity();
+        final IReadOnlyClusterCapacity maximumCapacity = resourceManager.getMaximumCapacity();
+        if (currentCapacity.getAggregatedCores() > maximumCapacity.getAggregatedCores()
+                || currentCapacity.getAggregatedMemoryByteSize() > maximumCapacity.getAggregatedMemoryByteSize()) {
+            LOGGER.warn("Current cluster available capacity {} is more than its maximum capacity {}", currentCapacity,
+                    maximumCapacity);
+        }
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersMessage.java
deleted file mode 100644
index fe9a5b8..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersMessage.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.asterix.runtime.message;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
-import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.control.CcId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ReportLocalCountersMessage implements ICcAddressedMessage {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOGGER = LogManager.getLogger();
-    private final long maxResourceId;
-    private final long maxTxnId;
-    private final long maxJobId;
-    private final String src;
-
-    public ReportLocalCountersMessage(String src, long maxResourceId, long maxTxnId, long maxJobId) {
-        this.src = src;
-        this.maxResourceId = maxResourceId;
-        this.maxTxnId = maxTxnId;
-        this.maxJobId = maxJobId;
-    }
-
-    @Override
-    public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
-        try {
-            appCtx.getTxnIdFactory().ensureMinimumId(maxTxnId);
-        } catch (AlgebricksException e) {
-            throw HyracksDataException.create(e);
-        }
-        resourceIdManager.report(src, maxResourceId);
-        ((ClusterControllerService) appCtx.getServiceContext().getControllerService()).getJobIdFactory()
-                .setMaxJobId(maxJobId);
-    }
-
-    public static void send(CcId ccId, NodeControllerService ncs) throws HyracksDataException {
-        INcApplicationContext appContext = (INcApplicationContext) ncs.getApplicationContext();
-        long maxResourceId = Math.max(appContext.getLocalResourceRepository().maxId(),
-                MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID);
-        long maxTxnId = appContext.getMaxTxnId();
-        long maxJobId = ncs.getMaxJobId(ccId);
-        ReportLocalCountersMessage countersMessage =
-                new ReportLocalCountersMessage(ncs.getId(), maxResourceId, maxTxnId, maxJobId);
-        try {
-            ((INCMessageBroker) ncs.getContext().getMessageBroker()).sendMessageToCC(ccId, countersMessage);
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Unable to report local counters", e);
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    @Override
-    public String toString() {
-        return ReportLocalCountersMessage.class.getSimpleName();
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersRequestMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersRequestMessage.java
deleted file mode 100644
index 51f53e7..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReportLocalCountersRequestMessage.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.asterix.runtime.message;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.CcIdentifiedMessage;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.nc.NodeControllerService;
-
-public class ReportLocalCountersRequestMessage extends CcIdentifiedMessage implements INcAddressedMessage {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public void handle(INcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ReportLocalCountersMessage.send(getCcId(),
-                (NodeControllerService) appCtx.getServiceContext().getControllerService());
-    }
-
-    @Override
-    public String toString() {
-        return ReportLocalCountersRequestMessage.class.getSimpleName();
-    }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
index 087913f..fbfca55 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ResourceIdRequestMessage.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.runtime.message;
 
-import java.util.Set;
-
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
@@ -49,7 +47,6 @@
                             new Exception("Cannot generate global resource id when cluster is not active."));
                 } else {
                     response.setException(new Exception("One or more nodes has not reported max resource id."));
-                    requestMaxResourceID(clusterStateManager, resourceIdManager, broker);
                 }
             }
             broker.sendApplicationMessageToNC(response, src);
@@ -58,17 +55,6 @@
         }
     }
 
-    private void requestMaxResourceID(IClusterStateManager clusterStateManager, IResourceIdManager resourceIdManager,
-            ICCMessageBroker broker) throws Exception {
-        Set<String> getParticipantNodes = clusterStateManager.getParticipantNodes(true);
-        ReportLocalCountersRequestMessage msg = new ReportLocalCountersRequestMessage();
-        for (String nodeId : getParticipantNodes) {
-            if (!resourceIdManager.reported(nodeId)) {
-                broker.sendApplicationMessageToNC(msg, nodeId);
-            }
-        }
-    }
-
     @Override
     public String toString() {
         return ResourceIdRequestMessage.class.getSimpleName();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
index 2415556..4130490 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.operators;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -33,7 +36,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 
 public class LSMIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
@@ -71,29 +73,26 @@
     @Override
     protected void initializeBulkLoader() throws HyracksDataException {
         ILSMIndex targetIndex = (ILSMIndex) index;
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
         if (usage.equals(BulkLoadUsage.LOAD)) {
-            // for a loaded dataset, we use the default Id 0 which is guaranteed to be smaller
-            // than Ids of all memory components
-
-            // TODO handle component Id for datasets loaded multiple times
-            // TODO move this piece of code to io operation callback
-            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
-            ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
-            LSMComponentIdUtils.persist(LSMComponentId.DEFAULT_COMPONENT_ID, diskComponent.getMetadata());
+            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    parameters);
         } else {
             primaryIndexHelper.open();
             primaryIndex = (ILSMIndex) primaryIndexHelper.getIndexInstance();
             List<ILSMDiskComponent> primaryComponents = primaryIndex.getDiskComponents();
-            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
             if (!primaryComponents.isEmpty()) {
-                // TODO move this piece of code to io operation callback
-                // Ideally, this should be done in io operation callback when a bulk load operation is finished
-                // However, currently we don't have an extensible callback mechanism to support this
                 ILSMComponentId bulkloadId = LSMComponentIdUtils.union(primaryComponents.get(0).getId(),
                         primaryComponents.get(primaryComponents.size() - 1).getId());
-                ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
-                LSMComponentIdUtils.persist(bulkloadId, diskComponent.getMetadata());
+                parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, bulkloadId);
+            } else {
+                parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+                        LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
             }
+            bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                    parameters);
+
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index dba6760..b855981 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.ILogMarkerCallback;
 import org.apache.asterix.common.transactions.PrimaryIndexLogMarkerCallback;
+import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -158,11 +159,13 @@
                             if (cursor.hasNext()) {
                                 cursor.next();
                                 prevTuple = cursor.getTuple();
+                                appendUpsertIndicator(!isDelete);
                                 appendFilterToPrevTuple();
                                 appendPrevRecord();
                                 appendPreviousMeta();
                                 appendFilterToOutput();
                             } else {
+                                appendUpsertIndicator(!isDelete);
                                 appendPreviousTupleAsMissing();
                             }
                         } finally {
@@ -170,6 +173,7 @@
                         }
                     } else {
                         searchCallback.before(key); // lock
+                        appendUpsertIndicator(!isDelete);
                         appendPreviousTupleAsMissing();
                     }
                     if (isDelete && prevTuple != null) {
@@ -202,6 +206,12 @@
             public void finish() throws HyracksDataException {
                 lsmAccessor.getCtx().setOperation(IndexOperation.UPSERT);
             }
+
+            @Override
+            public void fail(Throwable th) {
+                // We must fail before we exit the components
+                frameOpCallback.fail(th);
+            }
         };
         tracer = ctx.getJobletContext().getServiceContext().getTracer();
         traceCategory = tracer.getRegistry().get(TraceUtils.LATENCY);
@@ -256,14 +266,19 @@
 
                 @Override
                 public void frameCompleted() throws HyracksDataException {
-                    callback.frameCompleted();
                     appender.write(writer, true);
+                    callback.frameCompleted();
                 }
 
                 @Override
                 public void close() throws IOException {
                     callback.close();
                 }
+
+                @Override
+                public void fail(Throwable th) {
+                    callback.fail(th);
+                }
             };
         } catch (Throwable e) { // NOSONAR: Re-thrown
             throw HyracksDataException.create(e);
@@ -320,6 +335,11 @@
         }
     }
 
+    private void appendUpsertIndicator(boolean isUpsert) throws IOException {
+        recordDesc.getFields()[0].serialize(isUpsert ? ABoolean.TRUE : ABoolean.FALSE, dos);
+        tb.addFieldEndOffset();
+    }
+
     private void appendPrevRecord() throws IOException {
         dos.write(prevTuple.getFieldData(numOfPrimaryKeys), prevTuple.getFieldStart(numOfPrimaryKeys),
                 prevTuple.getFieldLength(numOfPrimaryKeys));
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
index 6d9ec47..1029d6f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
@@ -19,7 +19,10 @@
 package org.apache.asterix.runtime.operators;
 
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
 import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorNodePushable.DeletedTupleCounter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -32,9 +35,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 
 /**
+ * Note: only used with correlated merge policy
  * This operator node is used to bulk load incoming tuples (scanned from the primary index)
  * into multiple disk components of the secondary index.
  * Incoming tuple format:
@@ -182,14 +185,12 @@
 
     private void loadNewComponent(int componentPos) throws HyracksDataException {
         endCurrentComponent();
-
         int numTuples = getNumDeletedTuples(componentPos);
         ILSMDiskComponent primaryComponent = primaryIndex.getDiskComponents().get(componentPos);
-        componentBulkLoader =
-                (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples, false);
-        ILSMDiskComponent diskComponent = componentBulkLoader.getComponent();
-        // TODO move this piece of code to io operation callback
-        LSMComponentIdUtils.persist(primaryComponent.getId(), diskComponent.getMetadata());
+        Map<String, Object> parameters = new HashMap<>();
+        parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, primaryComponent.getId());
+        componentBulkLoader = (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples,
+                false, parameters);
     }
 
     private void addAntiMatterTuple(ITupleReference tuple) throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
index 9376d1b..ac7fc89 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
@@ -276,7 +276,7 @@
 
     private boolean equalPrimaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
         for (int i = numTagFields + numSecondaryKeys; i < numTagFields + numPrimaryKeys + numSecondaryKeys; i++) {
-            if (!equalField(tuple1, tuple2, i)) {
+            if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
                 return false;
             }
         }
@@ -285,16 +285,10 @@
 
     private boolean equalSecondaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
         for (int i = numTagFields; i < numTagFields + numSecondaryKeys; i++) {
-            if (!equalField(tuple1, tuple2, i)) {
+            if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
                 return false;
             }
         }
         return true;
     }
-
-    private boolean equalField(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
-        return LSMSecondaryUpsertOperatorNodePushable.equals(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx),
-                tuple1.getFieldLength(fIdx), tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx),
-                tuple2.getFieldLength(fIdx));
-    }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
index 958288a..df658b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.runtime.operators;
 
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -34,14 +35,19 @@
 
     private static final long serialVersionUID = 1L;
     private final int[] prevValuePermutation;
+    private final int upsertIndiatorFieldIndex;
+    private final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
 
     public LSMSecondaryUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
             ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackFactory,
+            int upsertIndicatorFieldIndex, IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
             int[] prevValuePermutation) {
         super(spec, outRecDesc, fieldPermutation, IndexOperation.UPSERT, indexHelperFactory, tupleFilterFactory, false,
                 modificationOpCallbackFactory);
         this.prevValuePermutation = prevValuePermutation;
+        this.upsertIndiatorFieldIndex = upsertIndicatorFieldIndex;
+        this.upsertIndicatorInspectorFactory = upsertIndicatorInspectorFactory;
     }
 
     @Override
@@ -49,6 +55,7 @@
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         RecordDescriptor intputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
         return new LSMSecondaryUpsertOperatorNodePushable(ctx, partition, indexHelperFactory, modCallbackFactory,
-                tupleFilterFactory, fieldPermutation, intputRecDesc, prevValuePermutation);
+                tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndiatorFieldIndex,
+                upsertIndicatorInspectorFactory, prevValuePermutation);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index a22e5e7..2dc7f5e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -19,15 +19,20 @@
 package org.apache.asterix.runtime.operators;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.TypeTagUtil;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -55,50 +60,34 @@
 public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
 
     private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
-    private int numberOfFields;
+    private final int upsertIndicatorFieldIndex;
+    private final IBinaryBooleanInspector upsertIndicatorInspector;
+    private final int numberOfFields;
     private AbstractIndexModificationOperationCallback abstractModCallback;
+    private final boolean isPrimaryKeyIndex;
 
     public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
             IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
             ITupleFilterFactory tupleFilterFactory, int[] fieldPermutation, RecordDescriptor inputRecDesc,
+            int upsertIndicatorFieldIndex, IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
             int[] prevValuePermutation) throws HyracksDataException {
         super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, IndexOperation.UPSERT,
                 modCallbackFactory, tupleFilterFactory);
         this.prevValueTuple.setFieldPermutation(prevValuePermutation);
+        this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
+        this.upsertIndicatorInspector = upsertIndicatorInspectorFactory.createBinaryBooleanInspector(ctx);
         this.numberOfFields = prevValuePermutation.length;
+        // a primary key index only has primary keys, and thus these two permutations are the same
+        this.isPrimaryKeyIndex = Arrays.equals(fieldPermutation, prevValuePermutation);
     }
 
     @Override
     public void open() throws HyracksDataException {
         super.open();
+        frameTuple = new FrameTupleReference();
         abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
     }
 
-    public static boolean equals(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
-        if (aLength != bLength) {
-            return false;
-        }
-        for (int i = 0; i < aLength; i++) {
-            if (a[aOffset + i] != b[bOffset + i]) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    public static boolean equalTuples(PermutingFrameTupleReference t1, PermutingFrameTupleReference t2, int numOfFields)
-            throws HyracksDataException {
-        byte[] t1Data = t1.getFieldData(0);
-        byte[] t2Data = t2.getFieldData(0);
-        for (int i = 0; i < numOfFields; i++) {
-            if (!equals(t1Data, t1.getFieldStart(i), t1.getFieldLength(i), t2Data, t2.getFieldStart(i),
-                    t2.getFieldLength(i))) {
-                return false;
-            }
-        }
-        return true;
-    }
-
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
         accessor.reset(buffer);
@@ -106,9 +95,15 @@
         int tupleCount = accessor.getTupleCount();
         for (int i = 0; i < tupleCount; i++) {
             try {
+                frameTuple.reset(accessor, i);
+                boolean isUpsert =
+                        upsertIndicatorInspector.getBooleanValue(frameTuple.getFieldData(upsertIndicatorFieldIndex),
+                                frameTuple.getFieldStart(upsertIndicatorFieldIndex),
+                                frameTuple.getFieldLength(upsertIndicatorFieldIndex));
                 // if both previous value and new value are null, then we skip
                 tuple.reset(accessor, i);
                 prevValueTuple.reset(accessor, i);
+
                 boolean isNewValueMissing = isMissing(tuple, 0);
                 boolean isOldValueMissing = isMissing(prevValueTuple, 0);
                 if (isNewValueMissing && isOldValueMissing) {
@@ -116,8 +111,10 @@
                     continue;
                 }
                 // At least, one is not null
-                // If they are equal, then we skip
-                if (equalTuples(tuple, prevValueTuple, numberOfFields)) {
+                if (!isPrimaryKeyIndex && TupleUtils.equalTuples(tuple, prevValueTuple, numberOfFields)) {
+                    // For a secondary index, if the secondary key values do not change, we can skip upserting it.
+                    // However, for a primary key index, we cannot do this because it only contains primary keys
+                    // which are always the same
                     continue;
                 }
                 if (!isOldValueMissing) {
@@ -125,7 +122,7 @@
                     abstractModCallback.setOp(Operation.DELETE);
                     lsmAccessor.forceDelete(prevValueTuple);
                 }
-                if (!isNewValueMissing) {
+                if (isUpsert && !isNewValueMissing) {
                     // we need to insert the new value
                     abstractModCallback.setOp(Operation.INSERT);
                     lsmAccessor.forceInsert(tuple);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
index d61e9a0..7e42d14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
@@ -22,6 +22,7 @@
 
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.context.DatasetInfo;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ILockManager;
@@ -82,7 +83,13 @@
                     // lock the dataset granule
                     lockManager.lock(datasetId, -1, LockMode.S, txnCtx);
                     // flush the dataset synchronously
-                    datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+                    DatasetInfo datasetInfo = datasetLifeCycleManager.getDatasetInfo(datasetId.getId());
+                    // TODO: Remove the isOpen check and let it fail if flush is requested for a dataset that is closed
+                    synchronized (datasetLifeCycleManager) {
+                        if (datasetInfo.isOpen()) {
+                            datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+                        }
+                    }
                 } catch (ACIDException e) {
                     throw HyracksDataException.create(e);
                 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/transaction/ResourceIdManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/transaction/ResourceIdManager.java
index 6d3077e..5bcd5aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/transaction/ResourceIdManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/transaction/ResourceIdManager.java
@@ -24,7 +24,6 @@
 
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.transactions.IResourceIdManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ResourceIdManager implements IResourceIdManager {
 
@@ -48,11 +47,8 @@
     }
 
     @Override
-    public void report(String nodeId, long maxResourceId) throws HyracksDataException {
+    public void report(String nodeId, long maxResourceId) {
         globalResourceId.updateAndGet(prev -> Math.max(maxResourceId, prev));
-        if (reportedNodes.add(nodeId)) {
-            csm.refreshState();
-        }
+        reportedNodes.add(nodeId);
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index 3409e61..a1a4e42 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -57,16 +58,18 @@
 
     @Override
     public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
-        return new ScanCollectionUnnestingFunctionFactory(args[0]);
+        return new ScanCollectionUnnestingFunctionFactory(args[0], sourceLoc);
     }
 
     public static class ScanCollectionUnnestingFunctionFactory implements IUnnestingEvaluatorFactory {
 
         private static final long serialVersionUID = 1L;
         private IScalarEvaluatorFactory listEvalFactory;
+        private final SourceLocation sourceLoc;
 
-        public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg) {
+        public ScanCollectionUnnestingFunctionFactory(IScalarEvaluatorFactory arg, SourceLocation sourceLoc) {
             this.listEvalFactory = arg;
+            this.sourceLoc = sourceLoc;
         }
 
         @Override
@@ -91,8 +94,8 @@
                     }
                     if (typeTag != ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
                             && typeTag != ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
-                        throw new TypeMismatchException(BuiltinFunctions.SCAN_COLLECTION, 0, typeTag,
-                                ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+                        throw new TypeMismatchException(sourceLoc, typeTag, ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG,
+                                ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
                     }
                     listAccessor.reset(inputVal.getByteArray(), inputVal.getStartOffset());
                     itemIndex = 0;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
new file mode 100644
index 0000000..14f102c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.runtime.utils;
+
+import java.util.List;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class ArrayFunctionsUtil {
+
+    private ArrayFunctionsUtil() {
+    }
+
+    public static <T extends IValueReference> T findItem(IValueReference item, List<T> sameHashes,
+            IBinaryComparator comp) throws HyracksDataException {
+        T sameItem;
+        for (int k = 0; k < sameHashes.size(); k++) {
+            sameItem = sameHashes.get(k);
+            if (comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(), sameItem.getByteArray(),
+                    sameItem.getStartOffset(), sameItem.getLength()) == 0) {
+                return sameItem;
+            }
+        }
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/BulkTxnIdFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/BulkTxnIdFactory.java
index 542bc17..296ce9c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/BulkTxnIdFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/BulkTxnIdFactory.java
@@ -24,7 +24,7 @@
 import org.apache.asterix.common.transactions.ITxnIdFactory;
 import org.apache.asterix.common.transactions.TxnId;
 
-class BulkTxnIdFactory implements ITxnIdFactory {
+public class BulkTxnIdFactory implements ITxnIdFactory {
 
     private final AtomicLong maxId = new AtomicLong();
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 8539fa4..8d3187b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -37,12 +37,14 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.replication.INcLifecycleCoordinator;
 import org.apache.asterix.common.transactions.IResourceIdManager;
+import org.apache.asterix.common.utils.NcLocalCounters;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.Section;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.application.ConfigManagerApplicationConfig;
 import org.apache.hyracks.control.common.config.ConfigManager;
 import org.apache.hyracks.control.common.controllers.NCConfig;
@@ -133,8 +135,9 @@
     }
 
     @Override
-    public synchronized void updateNodePartitions(String nodeId, boolean active) {
+    public synchronized void updateNodeState(String nodeId, boolean active, NcLocalCounters localCounters) {
         if (active) {
+            updateClusterCounters(nodeId, localCounters);
             participantNodes.add(nodeId);
         } else {
             participantNodes.remove(nodeId);
@@ -182,15 +185,6 @@
             setState(ClusterState.UNUSABLE);
             return;
         }
-
-        IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
-        for (String node : participantNodes) {
-            if (!resourceIdManager.reported(node)) {
-                LOGGER.info("Partitions are ready but {} has not yet registered its max resource id...", node);
-                setState(ClusterState.UNUSABLE);
-                return;
-            }
-        }
         // the metadata bootstrap & global recovery must be complete before the cluster can be active
         if (metadataNodeActive) {
             if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
@@ -452,6 +446,14 @@
         return metadataPartition;
     }
 
+    private void updateClusterCounters(String nodeId, NcLocalCounters localCounters) {
+        final IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
+        resourceIdManager.report(nodeId, localCounters.getMaxResourceId());
+        appCtx.getTxnIdFactory().ensureMinimumId(localCounters.getMaxTxnId());
+        ((ClusterControllerService) appCtx.getServiceContext().getControllerService()).getJobIdFactory()
+                .setMaxJobId(localCounters.getMaxJobId());
+    }
+
     private void updateNodeConfig(String nodeId, Map<IOption, Object> configuration) {
         ConfigManager configManager =
                 ((ConfigManagerApplicationConfig) appCtx.getServiceContext().getAppConfig()).getConfigManager();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
index 177bed2..0ffbe87 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/RuntimeComponentsProvider.java
@@ -21,6 +21,9 @@
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
@@ -28,6 +31,8 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class RuntimeComponentsProvider implements IStorageManager, ILSMIOOperationSchedulerProvider {
 
     private static final long serialVersionUID = 1L;
@@ -62,4 +67,13 @@
         return ((INcApplicationContext) ctx.getApplicationContext()).getResourceIdFactory();
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return RUNTIME_PROVIDER;
+    }
 }
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 22e0fad..7049475 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -169,8 +169,8 @@
               <url>https://raw.githubusercontent.com/mojohaus/appassembler/appassembler-2.0.0/LICENSE.txt</url>
             </override>
             <override>
-              <gav>io.netty:netty-all:4.1.6.Final</gav>
-              <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.16.Final/NOTICE.txt</noticeUrl>
+              <gav>io.netty:netty-all:4.1.25.Final</gav>
+              <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.25.Final/NOTICE.txt</noticeUrl>
             </override>
           </overrides>
           <licenses>
@@ -250,7 +250,10 @@
             </license>
             <license>
               <url>https://opensource.org/licenses/mit-license.php</url>
-              <aliasUrls>http://www.opensource.org/licenses/mit-license.php</aliasUrls>
+              <aliasUrls>
+                <aliasUrl>http://www.opensource.org/licenses/mit-license.php</aliasUrl>
+                <aliasUrl>http://opensource.org/licenses/MIT</aliasUrl>
+              </aliasUrls>
             </license>
             <license>
               <url>https://opensource.org/licenses/bsd-license.php</url>
@@ -407,6 +410,7 @@
             <usedDependency>org.codehaus.mojo.appassembler:appassembler-booter</usedDependency>
             <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
             <usedDependency>org.apache.asterix:asterix-dashboard</usedDependency>
+            <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
           </usedDependencies>
         </configuration>
       </plugin>
@@ -622,7 +626,7 @@
     <dependency>
       <groupId>org.codehaus.plexus</groupId>
       <artifactId>plexus-utils</artifactId>
-      <version>3.0.24</version>
+      <version>3.1.0</version>
     </dependency>
     <dependency>
       <groupId>commons-io</groupId>
@@ -665,5 +669,14 @@
       <artifactId>asterix-dashboard</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.asterix</groupId>
+      <artifactId>asterix-geo</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
index 51d25cb..e337cb0 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
@@ -39,6 +39,53 @@
   export JAVA_HOME
 fi
 
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ]; then
+              if [ -x "/usr/libexec/java_home" ]; then
+                  JAVA_HOME=`/usr/libexec/java_home`
+              else
+                  JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+              fi
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+
 export JAVA_VERSION=$(java -version 2>&1 | head -1 | awk '{ print $3 }' | tr -d '"')
 case $JAVA_VERSION in
   1.8*|1.9*|10*|11*)
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
index dbce377..522fb7c 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
@@ -39,13 +39,69 @@
   export JAVA_HOME
 fi
 
-[ -z "$JAVA_HOME" ] && {
-  echo "JAVA_HOME not set"
-  exit 1
-}
-"$JAVA_HOME/bin/java" -version 2>&1 | grep -q '1\.[89]' || {
+while [ -n "$1" ]; do
+  case $1 in
+    -f|-force) force=1;;
+    -help|--help|-usage|--usage) usage; exit 0;;
+    *) echo "ERROR: unknown argument '$1'"; usage; exit 1;;
+  esac
+  shift
+done
+
+if [ -z "$JAVA_HOME" -a -x /usr/libexec/java_home ]; then
+  JAVA_HOME=$(/usr/libexec/java_home)
+  export JAVA_HOME
+fi
+
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ]; then
+              if [ -x "/usr/libexec/java_home" ]; then
+                  JAVA_HOME=`/usr/libexec/java_home`
+              else
+                  JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+              fi
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+"$JAVACMD" -version 2>&1 | grep -q '1\.[89]' || {
   echo "JAVA_HOME must be at version 1.8 or later:"
-  "$JAVA_HOME/bin/java" -version
+  "$JAVACMD" -version
   exit 2
 }
 DIRNAME=$(dirname "$0")
diff --git a/asterixdb/asterix-server/src/main/opt/local/conf/cc.conf b/asterixdb/asterix-server/src/main/opt/local/conf/cc.conf
index 6fc0579..bd5f403 100644
--- a/asterixdb/asterix-server/src/main/opt/local/conf/cc.conf
+++ b/asterixdb/asterix-server/src/main/opt/local/conf/cc.conf
@@ -37,4 +37,5 @@
 address = 127.0.0.1
 
 [common]
-log.level=INFO
+log.dir = logs/
+log.level = INFO
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
index 042120a..0479bb2 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
@@ -23,11 +23,8 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.StringWriter;
-import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.utils.Servlets;
@@ -36,7 +33,6 @@
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -46,6 +42,10 @@
 import org.junit.rules.TestRule;
 import org.junit.runners.MethodSorters;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
 public class SampleLocalClusterIT {
 
@@ -120,11 +120,13 @@
     @Test
     public void test1_sanityQuery() throws Exception {
         TestExecutor testExecutor = new TestExecutor();
-        InputStream resultStream = testExecutor.executeQuery("1+1", OutputFormat.ADM,
-                new URI("http", null, "127.0.0.1", 19002, Servlets.AQL_QUERY, null, null), Collections.emptyList());
-        StringWriter sw = new StringWriter();
-        IOUtils.copy(resultStream, sw);
-        Assert.assertEquals("2", sw.toString().trim());
+        InputStream resultStream = testExecutor.executeQueryService("1+1;",
+                testExecutor.getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.ADM);
+        final ObjectMapper objectMapper = new ObjectMapper();
+        final ObjectNode response = objectMapper.readValue(resultStream, ObjectNode.class);
+        final JsonNode result = response.get("results");
+        Assert.assertEquals(1, result.size());
+        Assert.assertEquals(2, result.get(0).asInt());
     }
 
     @Test
diff --git a/asterixdb/asterix-test-framework/pom.xml b/asterixdb/asterix-test-framework/pom.xml
index 2025d5a..188e4d9 100644
--- a/asterixdb/asterix-test-framework/pom.xml
+++ b/asterixdb/asterix-test-framework/pom.xml
@@ -78,6 +78,14 @@
       <groupId>com.sun.activation</groupId>
       <artifactId>javax.activation</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
index 7a34648..7d0e3bf 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
@@ -22,7 +22,9 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Pattern;
 
 import org.apache.asterix.testframework.template.TemplateHelper;
@@ -92,7 +94,7 @@
                     return OutputFormat.NONE;
             }
         }
-    };
+    }
 
     public static final String DEFAULT_TESTSUITE_XML_NAME = "testsuite.xml";
     public static final String ONLY_TESTSUITE_XML_NAME = "only.xml";
@@ -105,6 +107,7 @@
     private TestGroup[] testGroups;
 
     private TestCase testCase;
+    private Map<String, Object> kv;
 
     public TestCaseContext(File tsRoot, TestSuite testSuite, TestGroup[] testGroups, TestCase testCase) {
         this.tsRoot = tsRoot;
@@ -133,6 +136,18 @@
         return testCase.getRepeat().intValue();
     }
 
+    public void put(String key, Object object) {
+        if (kv == null) {
+            kv = new HashMap<>();
+        }
+        kv.put(key, object);
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> T get(String key) {
+        return (T) kv.get(key);
+    }
+
     public List<TestFileContext> getFilesInDir(String basePath, String dirName, boolean withType) {
         List<TestFileContext> testFileCtxs = new ArrayList<>();
 
@@ -202,6 +217,11 @@
         return filePath;
     }
 
+    public boolean isSourceLocationExpected(CompilationUnit cUnit) {
+        Boolean v = cUnit.isSourceLocation();
+        return v != null ? v : testSuite.isSourceLocation();
+    }
+
     @Override
     public String toString() {
         StringBuilder sb = new StringBuilder(testCase.getFilePath());
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestFileContext.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestFileContext.java
index 7a53a9e..3ae6dcd 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestFileContext.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestFileContext.java
@@ -67,6 +67,13 @@
         return 0;
     }
 
+    public TestFileContext copy() {
+        TestFileContext copy = new TestFileContext(file);
+        copy.setSeqNum(Integer.toString(seqNum));
+        copy.setType(type);
+        return copy;
+    }
+
     @Override
     public String toString() {
         return String.valueOf(seqNum) + ":" + type + ":" + file;
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/xml/TestSuiteParser.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/xml/TestSuiteParser.java
index 201945c..8939bd1 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/xml/TestSuiteParser.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/xml/TestSuiteParser.java
@@ -29,17 +29,27 @@
 import javax.xml.parsers.SAXParserFactory;
 import javax.xml.transform.sax.SAXSource;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
+import org.xml.sax.SAXNotRecognizedException;
+import org.xml.sax.SAXNotSupportedException;
 
 public class TestSuiteParser {
 
+    private static final Logger LOGGER = LogManager.getLogger();
+
     public TestSuite parse(File testSuiteCatalog) throws SAXException, JAXBException, ParserConfigurationException {
         SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
         saxParserFactory.setNamespaceAware(true);
         saxParserFactory.setXIncludeAware(true);
         SAXParser saxParser = saxParserFactory.newSAXParser();
-        saxParser.setProperty(XMLConstants.ACCESS_EXTERNAL_DTD, "file");
+        try {
+            saxParser.setProperty(XMLConstants.ACCESS_EXTERNAL_DTD, "file");
+        } catch (SAXNotRecognizedException | SAXNotSupportedException e) {
+            LOGGER.warn("ignoring exception setting sax parser property", e);
+        }
 
         JAXBContext ctx = JAXBContext.newInstance(TestSuite.class);
         Unmarshaller um = ctx.createUnmarshaller();
diff --git a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
index 7a5387c..b6bc7bf 100644
--- a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
+++ b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
@@ -63,6 +63,14 @@
             </xs:annotation>
          </xs:attribute>
 
+         <xs:attribute name="SourceLocation" type="xs:boolean" default="false">
+            <xs:annotation>
+               <xs:documentation>
+                  whether to expect a source location in error messages
+               </xs:documentation>
+            </xs:annotation>
+         </xs:attribute>
+
       </xs:complexType>
 
       <xs:unique name="unique-test-group">
@@ -133,6 +141,7 @@
                      <xs:complexType>
                         <xs:attribute name="name" type="xs:string" />
                         <xs:attribute name="value" type="xs:string" />
+                        <xs:attribute name="type" type="test:parameter-type-enum" />
                      </xs:complexType>
                   </xs:element>
                   <xs:element name="output-dir" minOccurs="0">
@@ -160,6 +169,17 @@
                      </xs:annotation>
                   </xs:element>
 
+                  <!-- Whether the source location is expected in the error message -->
+
+                  <xs:element name="source-location" type="xs:boolean" minOccurs="0">
+                     <xs:annotation>
+                        <xs:documentation>
+                           Whether to expect a source location in the error message
+                           (default is defined by the test-suite element)
+                        </xs:documentation>
+                     </xs:annotation>
+                  </xs:element>
+
                </xs:sequence>
 
                <!-- This name is always equal to the name of the test case -->
@@ -250,4 +270,20 @@
       </xs:restriction>
    </xs:simpleType>
 
+   <!-- parameter-type-enum type                                              -->
+   <!--    Identify the type of a parameter value                             -->
+
+   <xs:simpleType name="parameter-type-enum">
+      <xs:annotation>
+         <xs:documentation>
+            Identify the type of a parameter value
+         </xs:documentation>
+      </xs:annotation>
+
+      <xs:restriction base="xs:string">
+         <xs:enumeration value="string"/>
+         <xs:enumeration value="json"/>
+      </xs:restriction>
+   </xs:simpleType>
+
 </xs:schema>
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 07dcf5d..4796f69 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -111,7 +111,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>1.10.19</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -163,5 +162,13 @@
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
+    <dependency>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
index 8746fba..d57d234 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -32,7 +32,6 @@
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 
 public abstract class AbstractIndexModificationOperationCallback extends AbstractOperationCallback
         implements IExtendedModificationOperationCallback {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
index eef1cb0..ab84211 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexOperationTrackerFactory.java
@@ -24,10 +24,15 @@
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class PrimaryIndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -44,7 +49,18 @@
         IDatasetLifecycleManager dslcManager =
                 ((INcApplicationContext) ctx.getApplicationContext()).getDatasetLifecycleManager();
         int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
-        return dslcManager.getOperationTracker(datasetId, partition);
+        return dslcManager.getOperationTracker(datasetId, partition, resource.getPath());
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new PrimaryIndexOperationTrackerFactory(json.get("datasetId").asInt());
+    }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
index 7586980..a104ae3 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerFactory.java
@@ -22,25 +22,43 @@
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.context.BaseOperationTracker;
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class SecondaryIndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
 
     private static final long serialVersionUID = 1L;
 
-    private final int datasetID;
+    private final int datasetId;
 
-    public SecondaryIndexOperationTrackerFactory(int datasetID) {
-        this.datasetID = datasetID;
+    public SecondaryIndexOperationTrackerFactory(int datasetId) {
+        this.datasetId = datasetId;
     }
 
     @Override
     public ILSMOperationTracker getOperationTracker(INCServiceContext ctx, IResource resource) {
         IDatasetLifecycleManager dslcManager =
                 ((INcApplicationContext) ctx.getApplicationContext()).getDatasetLifecycleManager();
-        return new BaseOperationTracker(datasetID, dslcManager.getDatasetInfo(datasetID));
+        return new BaseOperationTracker(datasetId, dslcManager.getDatasetInfo(datasetId));
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("datasetId", datasetId);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new SecondaryIndexOperationTrackerFactory(json.get("datasetId").asInt());
     }
 
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 0375c30..8f870c0 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -19,26 +19,19 @@
 package org.apache.asterix.transaction.management.resource;
 
 import static org.apache.asterix.common.utils.StorageConstants.INDEX_CHECKPOINT_FILE_PREFIX;
+import static org.apache.asterix.common.utils.StorageConstants.METADATA_FILE_NAME;
 import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_CREATE_FILE;
 import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER;
-import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.COMPONENT_TIMESTAMP_FORMAT;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.text.Format;
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -67,23 +60,31 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.LocalResource;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 
 public class PersistentLocalResourceRepository implements ILocalResourceRepository {
 
     private static final Logger LOGGER = LogManager.getLogger();
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final String METADATA_FILE_MASK_NAME =
+            StorageConstants.MASK_FILE_PREFIX + StorageConstants.METADATA_FILE_NAME;
     private static final FilenameFilter LSM_INDEX_FILES_FILTER =
             (dir, name) -> !name.startsWith(INDEX_CHECKPOINT_FILE_PREFIX);
     private static final FilenameFilter MASK_FILES_FILTER =
@@ -101,6 +102,18 @@
         }
     };
 
+    private static final IOFileFilter METADATA_MASK_FILES_FILTER = new IOFileFilter() {
+        @Override
+        public boolean accept(File file) {
+            return file.getName().equals(METADATA_FILE_MASK_NAME);
+        }
+
+        @Override
+        public boolean accept(File dir, String name) {
+            return false;
+        }
+    };
+
     private static final IOFileFilter ALL_DIR_FILTER = new IOFileFilter() {
         @Override
         public boolean accept(File file) {
@@ -113,9 +126,6 @@
         }
     };
 
-    private static final ThreadLocal<SimpleDateFormat> THREAD_LOCAL_FORMATTER =
-            ThreadLocal.withInitial(() -> new SimpleDateFormat(COMPONENT_TIMESTAMP_FORMAT));
-
     // Finals
     private final IIOManager ioManager;
     private final Cache<String, LocalResource> resourceCache;
@@ -125,11 +135,14 @@
     private IReplicationManager replicationManager;
     private final Path[] storageRoots;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    private final IPersistedResourceRegistry persistedResourceRegistry;
 
     public PersistentLocalResourceRepository(IIOManager ioManager,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider,
+            IPersistedResourceRegistry persistedResourceRegistry) {
         this.ioManager = ioManager;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+        this.persistedResourceRegistry = persistedResourceRegistry;
         storageRoots = new Path[ioManager.getIODevices().size()];
         final List<IODeviceHandle> ioDevices = ioManager.getIODevices();
         for (int i = 0; i < ioDevices.size(); i++) {
@@ -164,6 +177,7 @@
         return resource;
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
     public synchronized void insert(LocalResource resource) throws HyracksDataException {
         String relativePath = getFileName(resource.getPath());
@@ -176,23 +190,41 @@
         if (!parent.exists() && !parent.mkdirs()) {
             throw HyracksDataException.create(CANNOT_CREATE_FILE, parent.getAbsolutePath());
         }
-
-        try (FileOutputStream fos = new FileOutputStream(resourceFile.getFile());
-                ObjectOutputStream oosToFos = new ObjectOutputStream(fos)) {
-            oosToFos.writeObject(resource);
-            oosToFos.flush();
-        } catch (IOException e) {
+        // The next block should be all or nothing
+        try {
+            createResourceFileMask(resourceFile);
+            byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(resource.toJson(persistedResourceRegistry));
+            final Path path = Paths.get(resourceFile.getAbsolutePath());
+            Files.write(path, bytes);
+            indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(Long.MIN_VALUE, 0,
+                    LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId());
+            deleteResourceFileMask(resourceFile);
+        } catch (Exception e) {
+            cleanup(resourceFile);
             throw HyracksDataException.create(e);
+        } catch (Throwable th) {
+            LOGGER.error("Error creating resource {}", resourceFile, th);
+            ExitUtil.halt(ExitUtil.EC_ERROR_CREATING_RESOURCES);
         }
-
         resourceCache.put(resource.getPath(), resource);
-        indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(0);
         //if replication enabled, send resource metadata info to remote nodes
         if (isReplicationEnabled) {
             createReplicationJob(ReplicationOperation.REPLICATE, resourceFile);
         }
     }
 
+    @SuppressWarnings("squid:S1181")
+    private void cleanup(FileReference resourceFile) {
+        if (resourceFile.getFile().exists()) {
+            try {
+                IoUtil.delete(resourceFile);
+            } catch (Throwable th) {
+                LOGGER.error("Error cleaning up corrupted resource {}", resourceFile, th);
+                ExitUtil.halt(ExitUtil.EC_FAILED_TO_DELETE_CORRUPTED_RESOURCES);
+            }
+        }
+    }
+
     @Override
     public synchronized void delete(String relativePath) throws HyracksDataException {
         FileReference resourceFile = getLocalResourceFileByName(ioManager, relativePath);
@@ -200,13 +232,13 @@
             if (isReplicationEnabled) {
                 createReplicationJob(ReplicationOperation.DELETE, resourceFile);
             }
-            // delete all checkpoints
             final LocalResource localResource = readLocalResource(resourceFile.getFile());
-            indexCheckpointManagerProvider.get(DatasetResourceReference.of(localResource)).delete();
             // Invalidate before deleting the file just in case file deletion throws some exception.
             // Since it's just a cache invalidation, it should not affect correctness.
             resourceCache.invalidate(relativePath);
             IoUtil.delete(resourceFile);
+            // delete all checkpoints
+            indexCheckpointManagerProvider.get(DatasetResourceReference.of(localResource)).delete();
         } else {
             throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.RESOURCE_DOES_NOT_EXIST,
                     relativePath);
@@ -226,7 +258,7 @@
             final Collection<File> files = FileUtils.listFiles(root.toFile(), METADATA_FILES_FILTER, ALL_DIR_FILTER);
             try {
                 for (File file : files) {
-                    final LocalResource localResource = PersistentLocalResourceRepository.readLocalResource(file);
+                    final LocalResource localResource = readLocalResource(file);
                     if (filter.test(localResource)) {
                         resourcesMap.put(localResource.getId(), localResource);
                     }
@@ -254,10 +286,11 @@
                 : (path + File.separator + StorageConstants.METADATA_FILE_NAME);
     }
 
-    public static LocalResource readLocalResource(File file) throws HyracksDataException {
-        try (FileInputStream fis = new FileInputStream(file);
-                ObjectInputStream oisFromFis = new ObjectInputStream(fis)) {
-            LocalResource resource = (LocalResource) oisFromFis.readObject();
+    private LocalResource readLocalResource(File file) throws HyracksDataException {
+        final Path path = Paths.get(file.getAbsolutePath());
+        try {
+            final JsonNode jsonNode = OBJECT_MAPPER.readValue(Files.readAllBytes(path), JsonNode.class);
+            LocalResource resource = (LocalResource) persistedResourceRegistry.deserialize(jsonNode);
             if (resource.getVersion() == ITreeIndexFrame.Constants.VERSION) {
                 return resource;
             } else {
@@ -361,6 +394,21 @@
         return partitionReplicatedFiles;
     }
 
+    public long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
+            throws HyracksDataException {
+        long maxComponentId = LSMComponentId.MIN_VALID_COMPONENT_ID;
+        final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
+        for (LocalResource lr : partitionResources.values()) {
+            DatasetLocalResource datasetLocalResource = (DatasetLocalResource) lr.getResource();
+            if (strategy.isMatch(datasetLocalResource.getDatasetId())) {
+                final IIndexCheckpointManager indexCheckpointManager =
+                        indexCheckpointManagerProvider.get(DatasetResourceReference.of(lr));
+                maxComponentId = Math.max(maxComponentId, indexCheckpointManager.getLatest().getLastComponentId());
+            }
+        }
+        return maxComponentId;
+    }
+
     private List<String> getIndexFiles(File indexDir) {
         final List<String> indexFiles = new ArrayList<>();
         if (indexDir.isDirectory()) {
@@ -409,6 +457,20 @@
         return resourcesStats;
     }
 
+    public void deleteCorruptedResources() throws HyracksDataException {
+        for (Path root : storageRoots) {
+            final Collection<File> metadataMaskFiles =
+                    FileUtils.listFiles(root.toFile(), METADATA_MASK_FILES_FILTER, ALL_DIR_FILTER);
+            for (File metadataMaskFile : metadataMaskFiles) {
+                final File resourceFile = new File(metadataMaskFile.getParent(), METADATA_FILE_NAME);
+                if (resourceFile.exists()) {
+                    IoUtil.delete(resourceFile);
+                }
+                IoUtil.delete(metadataMaskFile);
+            }
+        }
+    }
+
     private void deleteIndexMaskedFiles(File index) throws IOException {
         File[] masks = index.listFiles(MASK_FILES_FILTER);
         if (masks != null) {
@@ -429,24 +491,17 @@
     }
 
     private void deleteIndexInvalidComponents(File index) throws IOException, ParseException {
-        final Optional<String> validComponentTimestamp = getIndexCheckpointManager(index).getValidComponentTimestamp();
-        if (!validComponentTimestamp.isPresent()) {
-            // index doesn't have any components
-            return;
-        }
-        final Format formatter = THREAD_LOCAL_FORMATTER.get();
-        final Date validTimestamp = (Date) formatter.parseObject(validComponentTimestamp.get());
         final File[] indexComponentFiles = index.listFiles(COMPONENT_FILES_FILTER);
-        if (indexComponentFiles != null) {
-            for (File componentFile : indexComponentFiles) {
-                // delete any file with startTime > validTimestamp
-                final String fileStartTimeStr =
-                        AbstractLSMIndexFileManager.getComponentStartTime(componentFile.getName());
-                final Date fileStartTime = (Date) formatter.parseObject(fileStartTimeStr);
-                if (fileStartTime.after(validTimestamp)) {
-                    LOGGER.info(() -> "Deleting invalid component file: " + componentFile.getAbsolutePath());
-                    Files.delete(componentFile.toPath());
-                }
+        if (indexComponentFiles == null) {
+            throw new IOException(index + " doesn't exist or an IO error occurred");
+        }
+        final long validComponentSequence = getIndexCheckpointManager(index).getValidComponentSequence();
+        for (File componentFile : indexComponentFiles) {
+            // delete any file with start sequence > valid component sequence
+            final long fileStart = IndexComponentFileReference.of(componentFile.getName()).getSequenceStart();
+            if (fileStart > validComponentSequence) {
+                LOGGER.info(() -> "Deleting invalid component file: " + componentFile.getAbsolutePath());
+                Files.delete(componentFile.toPath());
             }
         }
     }
@@ -488,8 +543,8 @@
                     long fileSize = file.length();
                     totalSize += fileSize;
                     if (isComponentFile(resolvedPath.getFile(), file.getName())) {
-                        String componentId = getComponentId(file.getAbsolutePath());
-                        componentsStats.put(componentId, componentsStats.getOrDefault(componentId, 0L) + fileSize);
+                        String componentSeq = getComponentSequence(file.getAbsolutePath());
+                        componentsStats.put(componentSeq, componentsStats.getOrDefault(componentSeq, 0L) + fileSize);
                     }
                 }
             }
@@ -500,17 +555,35 @@
         return null;
     }
 
+    private void createResourceFileMask(FileReference resourceFile) throws HyracksDataException {
+        Path maskFile = getResourceMaskFilePath(resourceFile);
+        try {
+            Files.createFile(maskFile);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void deleteResourceFileMask(FileReference resourceFile) throws HyracksDataException {
+        Path maskFile = getResourceMaskFilePath(resourceFile);
+        IoUtil.delete(maskFile);
+    }
+
+    private Path getResourceMaskFilePath(FileReference resourceFile) {
+        return Paths.get(resourceFile.getFile().getParentFile().getAbsolutePath(), METADATA_FILE_MASK_NAME);
+    }
+
     /**
-     * Gets a component id based on its unique timestamp.
-     * e.g. a component file 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439_b
-     * will return a component id 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439
+     * Gets a component sequence based on its unique timestamp.
+     * e.g. a component file 1_3_b
+     * will return a component sequence 1_3
      *
      * @param componentFile any component file
-     * @return The component id
+     * @return The component sequence
      */
-    public static String getComponentId(String componentFile) {
+    public static String getComponentSequence(String componentFile) {
         final ResourceReference ref = ResourceReference.of(componentFile);
-        return ref.getName().substring(0, ref.getName().lastIndexOf(AbstractLSMIndexFileManager.DELIMITER));
+        return IndexComponentFileReference.of(ref.getName()).getSequence();
     }
 
     private static boolean isComponentMask(File mask) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index d15e6ff..d1de086 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -20,21 +20,26 @@
 
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.common.ILocalResourceRepository;
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 
 public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
     private final IIOManager ioManager;
     private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+    private final IPersistedResourceRegistry persistedResourceRegistry;
 
     public PersistentLocalResourceRepositoryFactory(IIOManager ioManager,
-            IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+            IIndexCheckpointManagerProvider indexCheckpointManagerProvider,
+            IPersistedResourceRegistry persistedResourceRegistry) {
         this.ioManager = ioManager;
         this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+        this.persistedResourceRegistry = persistedResourceRegistry;
     }
 
     @Override
     public ILocalResourceRepository createRepository() {
-        return new PersistentLocalResourceRepository(ioManager, indexCheckpointManagerProvider);
+        return new PersistentLocalResourceRepository(ioManager, indexCheckpointManagerProvider,
+                persistedResourceRegistry);
     }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
index 74ba139..2692cc7 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntime.java
@@ -89,7 +89,7 @@
                 return;
             }
             initAccessAppend(ctx);
-            writer.open();
+            super.open();
         } catch (ACIDException e) {
             throw HyracksDataException.create(e);
         }
@@ -142,31 +142,6 @@
     }
 
     @Override
-    public void fail() throws HyracksDataException {
-        failed = true;
-        if (isSink) {
-            return;
-        }
-        writer.fail();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (isSink) {
-            return;
-        }
-        try {
-            flushIfNotFailed();
-        } catch (Exception e) {
-            writer.fail();
-            throw e;
-        } finally {
-            writer.close();
-        }
-        appender.reset(frame, true);
-    }
-
-    @Override
     public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
         this.inputRecordDesc = recordDescriptor;
         this.tAccess = new FrameTupleAccessor(inputRecordDesc);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
index bbfde38..708e8dc 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/runtime/CommitRuntimeFactory.java
@@ -21,12 +21,12 @@
 
 import org.apache.asterix.common.api.IJobEventListenerFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 
-public class CommitRuntimeFactory implements IPushRuntimeFactory {
+public class CommitRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index c91d233..3f2412f 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.common.exceptions.ACIDException;
@@ -36,30 +35,37 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMaps;
+import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
 /**
  * A concurrent implementation of the ILockManager interface.
  *
  * @see ResourceGroupTable
  * @see ResourceGroup
  */
+@SuppressWarnings("squid:RedundantThrowsDeclarationCheck") // throws ACIDException
 public class ConcurrentLockManager implements ILockManager, ILifeCycleComponent {
 
     static final Logger LOGGER = LogManager.getLogger();
     static final Level LVL = Level.TRACE;
-    public static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
+    private static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
 
-    public static final int NIL = -1;
-    public static final long NILL = -1L;
+    private static final int NIL = -1;
+    private static final long NILL = -1L;
 
-    public static final boolean DEBUG_MODE = false;//true
-    public static final boolean CHECK_CONSISTENCY = false;
+    private static final boolean DEBUG_MODE = false;//true
+    private static final boolean CHECK_CONSISTENCY = false;
 
-    private ResourceGroupTable table;
-    private ResourceArenaManager resArenaMgr;
-    private RequestArenaManager reqArenaMgr;
-    private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Long, Long> txnId2TxnSlotMap;
-    private LockManagerStats stats = new LockManagerStats(10000);
+    private final ResourceGroupTable table;
+    private final ResourceArenaManager resArenaMgr;
+    private final RequestArenaManager reqArenaMgr;
+    private final JobArenaManager jobArenaMgr;
+    private final Long2LongMap txnId2TxnSlotMap;
+    private final LockManagerStats stats = new LockManagerStats(10000);
 
     enum LockAction {
         ERR(false, false),
@@ -77,7 +83,7 @@
         }
     }
 
-    static LockAction[][] ACTION_MATRIX = {
+    private static final LockAction[][] ACTION_MATRIX = {
             // new    NL              IS               IX                S                X
             { LockAction.ERR, LockAction.UPD, LockAction.UPD, LockAction.UPD, LockAction.UPD }, // NL
             { LockAction.ERR, LockAction.GET, LockAction.UPD, LockAction.UPD, LockAction.WAIT }, // IS
@@ -97,7 +103,7 @@
         resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
         reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
         jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
-        txnId2TxnSlotMap = new ConcurrentHashMap<>();
+        txnId2TxnSlotMap = Long2LongMaps.synchronize(new Long2LongOpenHashMap());
     }
 
     @Override
@@ -129,10 +135,10 @@
                             enqueueWaiter(group, reqSlot, resSlot, jobSlot, act, txnContext);
                             break;
                         }
-                        //no break
+                        //fall-through
                     case UPD:
                         resArenaMgr.setMaxMode(resSlot, lockMode);
-                        // no break
+                        //fall-through
                     case GET:
                         addHolder(reqSlot, resSlot, jobSlot);
                         locked = true;
@@ -185,31 +191,35 @@
         void pop();
     }
 
-    static class NOPTracker implements DeadlockTracker {
+    private static class NOPTracker implements DeadlockTracker {
         static final DeadlockTracker INSTANCE = new NOPTracker();
 
         @Override
         public void pushResource(long resSlot) {
+            // no-op
         }
 
         @Override
         public void pushRequest(long reqSlot) {
+            // no-op
         }
 
         @Override
         public void pushJob(long jobSlot) {
+            // no-op
         }
 
         @Override
         public void pop() {
+            // no-op
         }
     }
 
-    static class CollectingTracker implements DeadlockTracker {
+    private static class CollectingTracker implements DeadlockTracker {
 
         static final boolean DEBUG = false;
 
-        ArrayList<Long> slots = new ArrayList<>();
+        LongList slots = new LongArrayList();
         ArrayList<String> types = new ArrayList<>();
 
         @Override
@@ -217,7 +227,7 @@
             types.add("Resource");
             slots.add(resSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
@@ -226,7 +236,7 @@
             types.add("Request");
             slots.add(reqSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
@@ -235,24 +245,24 @@
             types.add("Job");
             slots.add(jobSlot);
             if (DEBUG) {
-                System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
         }
 
         @Override
         public void pop() {
             if (DEBUG) {
-                System.err.println("pop " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+                LOGGER.info("pop " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
             }
             types.remove(types.size() - 1);
-            slots.remove(slots.size() - 1);
+            slots.removeLong(slots.size() - 1);
         }
 
         @Override
         public String toString() {
             StringBuilder sb = new StringBuilder();
             for (int i = 0; i < slots.size(); ++i) {
-                sb.append(types.get(i) + " " + TypeUtil.Global.toString(slots.get(i)) + "\n");
+                sb.append(types.get(i)).append(" ").append(TypeUtil.Global.toString(slots.getLong(i))).append("\n");
             }
             return sb.toString();
         }
@@ -270,15 +280,11 @@
      * @return true if a cycle would be introduced, false otherwise
      */
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
-        if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
-            /**
-             * Due to the deadlock-free locking protocol, deadlock is not possible.
-             * So, this method always returns false.
-             */
-            return false;
-        } else {
-            return introducesDeadlock(resSlot, jobSlot, tracker, 0);
-        }
+        /*
+         * Due to the deadlock-free locking protocol, deadlock is not possible.
+         * So, this method always returns false in that case
+         */
+        return !ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL && introducesDeadlock(resSlot, jobSlot, tracker, 0);
     }
 
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker,
@@ -298,20 +304,20 @@
                 // The scanWaiters flag indicates if we are currently scanning the waiters (true) or the upgraders
                 // (false).
                 boolean scanWaiters = true;
-                long waiter = jobArenaMgr.getLastWaiter(holderJobSlot);
-                if (waiter < 0 && scanWaiters) {
+                long jobWaiter = jobArenaMgr.getLastWaiter(holderJobSlot);
+                if (jobWaiter < 0) {
                     scanWaiters = false;
-                    waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+                    jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
                 }
-                while (waiter >= 0) {
-                    long waitingOnResSlot = reqArenaMgr.getResourceId(waiter);
+                while (jobWaiter >= 0) {
+                    long waitingOnResSlot = reqArenaMgr.getResourceId(jobWaiter);
                     if (introducesDeadlock(waitingOnResSlot, jobSlot, tracker, depth + 1)) {
                         return true;
                     }
-                    waiter = reqArenaMgr.getNextJobRequest(waiter);
-                    if (waiter < 0 && scanWaiters) {
+                    jobWaiter = reqArenaMgr.getNextJobRequest(jobWaiter);
+                    if (jobWaiter < 0 && scanWaiters) {
                         scanWaiters = false;
-                        waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+                        jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
                     }
                 }
 
@@ -407,7 +413,7 @@
             switch (act) {
                 case UPD:
                     resArenaMgr.setMaxMode(resSlot, lockMode);
-                    // no break
+                    //fall-through
                 case GET:
                     addHolder(reqSlot, resSlot, jobSlot);
                     return true;
@@ -532,8 +538,8 @@
         stats.releaseLocks();
 
         long txnId = txnContext.getTxnId().getId();
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
             // we don't know the job, so there are no locks for it - we're done
             return;
         }
@@ -565,15 +571,15 @@
     }
 
     private long findOrAllocJobSlot(long txnId) {
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
-            jobSlot = new Long(jobArenaMgr.allocate());
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
+            jobSlot = jobArenaMgr.allocate();
             if (DEBUG_MODE) {
                 LOGGER.trace("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + txnId + ")");
             }
             jobArenaMgr.setTxnId(jobSlot, txnId);
-            Long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
-            if (oldSlot != null) {
+            long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
+            if (oldSlot != 0) {
                 // if another thread allocated a slot for this jobThreadId between
                 // get(..) and putIfAbsent(..), we'll use that slot and
                 // deallocate the one we allocated
@@ -584,7 +590,7 @@
                 jobSlot = oldSlot;
             }
         }
-        assert (jobSlot >= 0);
+        assert jobSlot > 0;
         return jobSlot;
     }
 
@@ -670,7 +676,7 @@
     }
 
     private long findResourceInGroup(ResourceGroup group, int dsId, int entityHashValue) {
-        stats.logCounters(LOGGER, Level.DEBUG, false);
+        stats.logCounters(LOGGER, LVL, false);
         long resSlot = group.firstResourceIndex.get();
         while (resSlot != NILL) {
             // either we already have a lock on this resource or we have a
@@ -754,7 +760,7 @@
         void remove(long request, long resource, long job);
     }
 
-    final Queue waiter = new Queue() {
+    private final Queue waiter = new Queue() {
         @Override
         public void add(long request, long resource, long job) {
             long waiter = resArenaMgr.getFirstWaiter(resource);
@@ -788,7 +794,7 @@
         }
     };
 
-    final Queue upgrader = new Queue() {
+    private final Queue upgrader = new Queue() {
         @Override
         public void add(long request, long resource, long job) {
             long upgrader = resArenaMgr.getFirstUpgrader(resource);
@@ -987,6 +993,7 @@
                 }
             }
         } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
             throw new IllegalStateException("interrupted", e);
         }
     }
@@ -1015,8 +1022,8 @@
      * @return the slot of the request, if the lock request is found, NILL otherwise
      */
     private long findLockInJobQueue(final int dsId, final int entityHashValue, final long txnId, byte lockMode) {
-        Long jobSlot = txnId2TxnSlotMap.get(txnId);
-        if (jobSlot == null) {
+        long jobSlot = txnId2TxnSlotMap.get(txnId);
+        if (jobSlot == 0) {
             return NILL;
         }
 
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
index 26261c2..6a5372e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
@@ -19,17 +19,17 @@
 
 package org.apache.asterix.transaction.management.service.locking;
 
-import java.util.concurrent.ConcurrentHashMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
 
 public class DumpTablePrinter implements TablePrinter {
     private ResourceGroupTable table;
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Long, Long> txnIdToJobSlotMap;
+    private Long2LongMap txnIdToJobSlotMap;
 
     DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
-            JobArenaManager jobArenaMgr, ConcurrentHashMap<Long, Long> txnIdToJobSlotMap) {
+            JobArenaManager jobArenaMgr, Long2LongMap txnIdToJobSlotMap) {
         this.table = table;
         this.resArenaMgr = resArenaMgr;
         this.reqArenaMgr = reqArenaMgr;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
deleted file mode 100644
index 5bf5ad6..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
+++ /dev/null
@@ -1,595 +0,0 @@
-/*
- * 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.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-
-/**
- * PrimitiveIntHashMap supports primitive int type as key and value.
- * The hash map grows when the available slots in a bucket are overflowed.
- * Also, the hash map shrinks according to the following shrink policy.
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class PrimitiveIntHashMap {
-    private final int CHILD_BUCKETS; //INIT_NUM_OF_BUCKETS;
-    private final int NUM_OF_SLOTS; //NUM_OF_SLOTS_IN_A_BUCKET;
-    private final int SHRINK_TIMER_THRESHOLD;
-
-    private int occupiedSlots;
-    private ArrayList<ChildIntArrayManager> pArray; //parent array
-    private int hashMod;
-    private long shrinkTimer;
-    private boolean isShrinkTimerOn;
-    private int iterBucketIndex;
-    private int iterSlotIndex;
-    private int iterChildIndex;
-    private KeyValuePair iterPair;
-
-    //    ////////////////////////////////////////////////
-    //    // begin of unit test
-    //    ////////////////////////////////////////////////
-    //
-    //    /**
-    //     * @param args
-    //     */
-    //    public static void main(String[] args) {
-    //        int i, j;
-    //        int k = 0;
-    //        int num = 5;
-    //        int key[] = new int[500];
-    //        int val[] = new int[500];
-    //        KeyValuePair pair;
-    //        PrimitiveIntHashMap map = new PrimitiveIntHashMap(1<<4, 1<<3, 5);
-    //
-    //        for (j=0; j < num; j++) {
-    //
-    //            k += 100;
-    //            //generate data
-    //            for (i=0; i < k; i++) {
-    //                key[i] = i;
-    //                val[i] = i;
-    //            }
-    //
-    //            //put data to map
-    //            for (i=0; i < k-30; i++) {
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            //put data to map
-    //            for (i=0; i < k-30; i++) {
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            map.beginIterate();
-    //            pair = map.getNextKeyValue();
-    //            i = 0;
-    //            while (pair != null) {
-    //                i++;
-    //                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //                pair = map.getNextKeyValue();
-    //            }
-    //
-    //            //System.out.println(map.prettyPrint());
-    //
-    //            for (i=k-20; i< k; i++) { //skip X70~X79
-    //                map.put(key[i], val[i]);
-    //            }
-    //
-    //            System.out.println(map.prettyPrint());
-    //
-    //            //remove data to map
-    //            for (i=0; i < k-10; i++) {
-    //                map.remove(key[i]);
-    //                try {
-    //                    Thread.currentThread().sleep(1);
-    //                } catch (InterruptedException e) {
-    //                    e.printStackTrace();
-    //                }
-    //            }
-    //
-    //            map.beginIterate();
-    //            pair = map.getNextKeyValue();
-    //            i = 0;
-    //            while (pair != null) {
-    //                i++;
-    //                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //                pair = map.getNextKeyValue();
-    //            }
-    //
-    //            //remove data to map
-    //            for (i=0; i < k-10; i++) {
-    //                map.remove(key[i]);
-    //                try {
-    //                    Thread.currentThread().sleep(1);
-    //                } catch (InterruptedException e) {
-    //                    // TODO Auto-generated catch block
-    //                    e.printStackTrace();
-    //                }
-    //            }
-    //
-    //            System.out.println(map.prettyPrint());
-    //
-    //            //get data from map
-    //            for (i=0; i < k; i++) {
-    //                System.out.println(""+i+"=> key:"+ key[i] + ", val:"+val[i] +", result: " + map.get(key[i]));
-    //            }
-    //        }
-    //
-    //        map.beginIterate();
-    //        pair = map.getNextKeyValue();
-    //        i = 0;
-    //        while (pair != null) {
-    //            i++;
-    //            System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
-    //            pair = map.getNextKeyValue();
-    //        }
-    //    }
-    //
-    //    ////////////////////////////////////////////////
-    //    // end of unit test
-    //    ////////////////////////////////////////////////
-
-    public PrimitiveIntHashMap() {
-        CHILD_BUCKETS = 1 << 9; //INIT_NUM_OF_BUCKETS;
-        NUM_OF_SLOTS = 1 << 3; //NUM_OF_SLOTS_IN_A_BUCKET;
-        SHRINK_TIMER_THRESHOLD = 120000; //2min
-        pArray = new ArrayList<ChildIntArrayManager>();
-        pArray.add(new ChildIntArrayManager(this));
-        hashMod = CHILD_BUCKETS;
-        occupiedSlots = 0;
-        iterPair = new KeyValuePair();
-    }
-
-    public PrimitiveIntHashMap(int childBuckets, int numOfSlots, int shrinkTimerThreshold) {
-        CHILD_BUCKETS = childBuckets;
-        NUM_OF_SLOTS = numOfSlots;
-        SHRINK_TIMER_THRESHOLD = shrinkTimerThreshold;
-        pArray = new ArrayList<ChildIntArrayManager>();
-        pArray.add(new ChildIntArrayManager(this));
-        hashMod = CHILD_BUCKETS;
-        occupiedSlots = 0;
-        iterPair = new KeyValuePair();
-    }
-
-    public void put(int key, int value) {
-        int growCount = 0;
-        int bucketNum = hash(key);
-        ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
-        while (child.isFull(bucketNum % CHILD_BUCKETS)) {
-            growHashMap();
-            bucketNum = hash(key);
-            child = pArray.get(bucketNum / CHILD_BUCKETS);
-            if (growCount > 2) {
-                //changeHashFunc();
-            }
-            growCount++;
-        }
-        occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, false);
-    }
-
-    public void upsert(int key, int value) {
-        int growCount = 0;
-        int bucketNum = hash(key);
-        ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
-        while (child.isFull(bucketNum % CHILD_BUCKETS)) {
-            growHashMap();
-            bucketNum = hash(key);
-            child = pArray.get(bucketNum / CHILD_BUCKETS);
-            if (growCount > 2) {
-                //changeHashFunc();
-            }
-            growCount++;
-        }
-        occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, true);
-    }
-
-    private int hash(int key) {
-        return key % hashMod;
-    }
-
-    private void growHashMap() {
-        int size = pArray.size();
-        int i;
-
-        //grow buckets by adding more child
-        for (i = 0; i < size; i++) {
-            pArray.add(new ChildIntArrayManager(this));
-        }
-
-        //increase hashMod
-        hashMod *= 2;
-
-        //re-hash
-        rehash(0, size, hashMod / 2);
-    }
-
-    private void shrinkHashMap() {
-        int size = pArray.size();
-        int i;
-
-        //decrease hashMod
-        hashMod /= 2;
-
-        //re-hash
-        rehash(size / 2, size, hashMod * 2);
-
-        //shrink buckets by removing child(s)
-        for (i = size - 1; i >= size / 2; i--) {
-            pArray.remove(i);
-        }
-    }
-
-    private void rehash(int begin, int end, int oldHashMod) {
-        int i, j, k;
-        int key, value;
-        ChildIntArrayManager child;
-
-        //re-hash
-        for (i = begin; i < end; i++) {
-            child = pArray.get(i);
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                if (child.cArray[j][0] == 0) {
-                    continue;
-                }
-                for (k = 1; k < NUM_OF_SLOTS; k++) {
-                    //if the hashValue of the key is different, then re-hash it.
-                    key = child.cArray[j][k * 2];
-                    if (hash(key) != key % oldHashMod) {
-                        value = child.cArray[j][k * 2 + 1];
-                        //remove existing key and value
-                        //Notice! To avoid bucket iteration, child.remove() is not used.
-                        child.cArray[j][k * 2] = -1;
-                        child.cArray[j][0]--;
-                        //re-hash it
-                        pArray.get(hash(key) / CHILD_BUCKETS).put(hash(key) % CHILD_BUCKETS, key, value, false);
-                    }
-                }
-            }
-        }
-    }
-
-    //    private void changeHashFunc() {
-    //        //TODO need to implement.
-    //        throw new UnsupportedOperationException("changeHashFunc() not implemented");
-    //    }
-
-    public int get(int key) {
-        int bucketNum = hash(key);
-        return pArray.get(bucketNum / CHILD_BUCKETS).get(bucketNum % CHILD_BUCKETS, key);
-    }
-
-    public void remove(int key) {
-        int bucketNum = hash(key);
-        occupiedSlots -= pArray.get(bucketNum / CHILD_BUCKETS).remove(bucketNum % CHILD_BUCKETS, key);
-
-        if (needShrink()) {
-            shrinkHashMap();
-        }
-    }
-
-    /**
-     * Shrink policy:
-     * Shrink when the resource under-utilization lasts for a certain amount of time.
-     *
-     * @return
-     */
-    private boolean needShrink() {
-        int size = pArray.size();
-        int usedSlots = occupiedSlots;
-        if (usedSlots == 0) {
-            usedSlots = 1;
-        }
-        if (size > 1 && size * CHILD_BUCKETS * NUM_OF_SLOTS / usedSlots >= 3 && isSafeToShrink()) {
-            if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
-                    isShrinkTimerOn = false;
-                    return true;
-                }
-            } else {
-                //turn on timer
-                isShrinkTimerOn = true;
-                shrinkTimer = System.currentTimeMillis();
-            }
-        } else {
-            //turn off timer
-            isShrinkTimerOn = false;
-        }
-        return false;
-    }
-
-    private boolean isSafeToShrink() {
-        int i, j;
-        int size = pArray.size();
-        //Child: 0, 1, 2, 3, 4, 5, 6, 7
-        //[HChild(Head Child):0 and TChild(Tail Child): 4], [1(H),5(T)], [2(H),6(T)] and so on.
-        //When the map shrinks, the sum of occupied slots in H/TChild should not exceed the NUM_OF_SLOTS-1.
-        //Then it is safe to shrink. Otherwise, unsafe.
-        ChildIntArrayManager HChild, TChild;
-
-        for (i = 0; i < size / 2; i++) {
-            HChild = pArray.get(i);
-            TChild = pArray.get(size / 2 + i);
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                if (HChild.cArray[j][0] + TChild.cArray[j][0] > NUM_OF_SLOTS - 1) {
-                    return false;
-                }
-            }
-        }
-        return true;
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder("\n########### PrimitiveIntHashMap Status #############\n");
-        ChildIntArrayManager child;
-        int i, j, k;
-        int size = pArray.size();
-        for (i = 0; i < size; i++) {
-            child = pArray.get(i);
-            s.append("child[").append(i).append("]\n");
-            for (j = 0; j < CHILD_BUCKETS; j++) {
-                s.append(j).append(" ");
-                for (k = 0; k < NUM_OF_SLOTS; k++) {
-                    s.append("[").append(child.cArray[j][k * 2]).append(",").append(child.cArray[j][k * 2 + 1])
-                            .append("] ");
-                }
-                s.append("\n");
-            }
-        }
-        return s.toString();
-    }
-
-    public int getNumOfSlots() {
-        return NUM_OF_SLOTS;
-    }
-
-    public int getNumOfChildBuckets() {
-        return CHILD_BUCKETS;
-    }
-
-    public void clear(boolean needShrink) {
-        int size = pArray.size();
-        for (int i = size - 1; i >= 0; i--) {
-            if (needShrink && i != 0) {
-                pArray.remove(i);
-            } else {
-                pArray.get(i).clear();
-            }
-        }
-        occupiedSlots = 0;
-    }
-
-    ///////////////////////////////////////
-    // iterate method
-    ///////////////////////////////////////
-
-    public void beginIterate() {
-        iterChildIndex = 0;
-        iterBucketIndex = 0;
-        iterSlotIndex = 1;
-    }
-
-    public KeyValuePair getNextKeyValue() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
-                    iterSlotIndex++;
-                    return iterPair;
-                }
-            }
-        }
-        return null;
-    }
-
-    public int getNextKey() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterSlotIndex++;
-                    return iterPair.key;
-                }
-            }
-        }
-        return -1;
-    }
-
-    public int getNextValue() {
-        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
-            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
-                if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
-                    continue;
-                }
-                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
-                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
-                    if (iterPair.key == -1) {
-                        continue;
-                    }
-                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
-                    iterSlotIndex++;
-                    return iterPair.value;
-                }
-            }
-        }
-        return -1;
-    }
-
-    public static class KeyValuePair {
-        public int key;
-        public int value;
-    }
-}
-
-class ChildIntArrayManager {
-    private final int DIM1_SIZE;
-    private final int DIM2_SIZE;
-    private final int NUM_OF_SLOTS;
-    public int[][] cArray; //child array
-
-    public ChildIntArrayManager(PrimitiveIntHashMap parentHashMap) {
-        DIM1_SIZE = parentHashMap.getNumOfChildBuckets();
-        DIM2_SIZE = parentHashMap.getNumOfSlots() * 2; //2: Array of [key, value] pair
-        NUM_OF_SLOTS = parentHashMap.getNumOfSlots();
-        initialize();
-    }
-
-    private void initialize() {
-        cArray = new int[DIM1_SIZE][DIM2_SIZE];
-        int i, j;
-        for (i = 0; i < DIM1_SIZE; i++) {
-            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
-            //cArray[i][1] is not used.
-            cArray[i][0] = 0;
-            for (j = 1; j < NUM_OF_SLOTS; j++) {
-                cArray[i][j * 2] = -1; // -1 represent that the slot is empty
-            }
-        }
-    }
-
-    public void clear() {
-        int i, j;
-        for (i = 0; i < DIM1_SIZE; i++) {
-            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
-            //cArray[i][1] is not used.
-            if (cArray[i][0] == 0) {
-                continue;
-            }
-            cArray[i][0] = 0;
-            for (j = 1; j < NUM_OF_SLOTS; j++) {
-                cArray[i][j * 2] = -1; // -1 represent that the slot is empty
-            }
-        }
-    }
-
-    public void deinitialize() {
-        cArray = null;
-    }
-
-    public void allocate() {
-        initialize();
-    }
-
-    public boolean isFull(int bucketNum) {
-        return cArray[bucketNum][0] == NUM_OF_SLOTS - 1;
-    }
-
-    public boolean isEmpty(int bucketNum) {
-        return cArray[bucketNum][0] == 0;
-    }
-
-    /**
-     * Put key,value into a slot in the bucket if the key doesn't exist.
-     * Update value if the key exists and if isUpsert is true
-     * No need to call get() to check the existence of the key before calling put().
-     * Notice! Caller should make sure that there is an available slot.
-     *
-     * @param bucketNum
-     * @param key
-     * @param value
-     * @param isUpsert
-     * @return 1 for new insertion, 0 for key duplication
-     */
-    public int put(int bucketNum, int key, int value, boolean isUpsert) {
-        int i;
-        int emptySlot = -1;
-
-        if (cArray[bucketNum][0] == 0) {
-            cArray[bucketNum][2] = key;
-            cArray[bucketNum][3] = value;
-            cArray[bucketNum][0]++;
-            return 1;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                if (isUpsert) {
-                    cArray[bucketNum][i * 2 + 1] = value;
-                }
-                return 0;
-            } else if (cArray[bucketNum][i * 2] == -1) {
-                emptySlot = i;
-            }
-        }
-
-        if (emptySlot == -1) {
-            throw new UnsupportedOperationException("error");
-        }
-
-        cArray[bucketNum][emptySlot * 2] = key;
-        cArray[bucketNum][emptySlot * 2 + 1] = value;
-        cArray[bucketNum][0]++;
-        return 1;
-    }
-
-    public int get(int bucketNum, int key) {
-        int i;
-
-        if (cArray[bucketNum][0] == 0) {
-            return -1;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                return cArray[bucketNum][i * 2 + 1];
-            }
-        }
-        return -1;
-    }
-
-    /**
-     * remove key if it exists. Otherwise, ignore it.
-     *
-     * @param bucketNum
-     * @param key
-     * @return 1 for success, 0 if the key doesn't exist
-     */
-    public int remove(int bucketNum, int key) {
-        int i;
-
-        if (cArray[bucketNum][0] == 0) {
-            return 0;
-        }
-
-        for (i = 1; i < NUM_OF_SLOTS; i++) {
-            if (cArray[bucketNum][i * 2] == key) {
-                cArray[bucketNum][i * 2] = -1;
-                cArray[bucketNum][0]--;
-                return 1;
-            }
-        }
-
-        return 0;
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 21268e5..28290bb 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -89,8 +89,7 @@
     public void append(ILogRecord logRecord, long appendLsn) {
         logRecord.writeLogRecord(appendBuffer);
 
-        if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
-                && logRecord.getLogType() != LogType.WAIT) {
+        if (isLocalTransactionLog(logRecord)) {
             logRecord.getTxnCtx().setLastLSN(appendLsn);
         }
 
@@ -100,13 +99,10 @@
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
             if (logRecord.getLogSource() == LogSource.LOCAL) {
-                if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
-                        || logRecord.getLogType() == LogType.WAIT) {
+                if (syncPendingNonFlushLog(logRecord)) {
                     logRecord.isFlushed(false);
                     syncCommitQ.add(logRecord);
-                }
-                if (logRecord.getLogType() == LogType.FLUSH) {
-                    logRecord.isFlushed(false);
+                } else if (logRecord.getLogType() == LogType.FLUSH) {
                     flushQ.add(logRecord);
                 }
             } else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
@@ -117,6 +113,16 @@
         }
     }
 
+    private boolean syncPendingNonFlushLog(ILogRecord logRecord) {
+        return logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                || logRecord.getLogType() == LogType.WAIT || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES;
+    }
+
+    private boolean isLocalTransactionLog(ILogRecord logRecord) {
+        return logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
+                && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
+    }
+
     @Override
     public void setFileChannel(FileChannel fileChannel) {
         this.fileChannel = fileChannel;
@@ -231,7 +237,8 @@
                         notifyJobTermination();
                     } else if (logRecord.getLogType() == LogType.FLUSH) {
                         notifyFlushTermination();
-                    } else if (logRecord.getLogType() == LogType.WAIT) {
+                    } else if (logRecord.getLogType() == LogType.WAIT
+                            || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
                         notifyWaitTermination();
                     }
                 } else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
@@ -317,6 +324,7 @@
     }
 
     private class MutableTxnId extends TxnId {
+        private static final long serialVersionUID = 579540092176284383L;
 
         public MutableTxnId(long id) {
             super(id);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index 736de07..0a6dda9 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -18,8 +18,6 @@
  */
 package org.apache.asterix.transaction.management.service.logging;
 
-import static org.apache.hyracks.util.ExitUtil.EC_IMMEDIATE_HALT;
-
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
@@ -31,8 +29,6 @@
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -106,7 +102,8 @@
         nodeId = txnSubsystem.getId();
         flushLogsQ = new LinkedBlockingQueue<>();
         txnSubsystem.getApplicationContext().getThreadExecutor().execute(new FlushLogsLogger());
-        initializeLogManager(SMALLEST_LOG_FILE_ID);
+        final long onDiskMaxLogFileId = getOnDiskMaxLogFileId();
+        initializeLogManager(onDiskMaxLogFileId);
     }
 
     private void initializeLogManager(long nextLogFileId) {
@@ -134,13 +131,33 @@
 
     @Override
     public void log(ILogRecord logRecord) {
-        if (logRecord.getLogType() == LogType.FLUSH) {
-            flushLogsQ.add(logRecord);
-            return;
+        if (!logToFlushQueue(logRecord)) {
+            appendToLogTail(logRecord);
         }
-        appendToLogTail(logRecord);
     }
 
+    @SuppressWarnings("squid:S2445")
+    protected boolean logToFlushQueue(ILogRecord logRecord) {
+        //Remote flush logs do not need to be flushed separately since they may not trigger local flush
+        if ((logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL)
+                || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+            logRecord.isFlushed(false);
+            flushLogsQ.add(logRecord);
+            if (logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+                InvokeUtil.doUninterruptibly(() -> {
+                    synchronized (logRecord) {
+                        while (!logRecord.isFlushed()) {
+                            logRecord.wait();
+                        }
+                    }
+                });
+            }
+            return true;
+        }
+        return false;
+    }
+
+    @SuppressWarnings("squid:S2445")
     protected void appendToLogTail(ILogRecord logRecord) {
         syncAppendToLogTail(logRecord);
         if (waitForFlush(logRecord) && !logRecord.isFlushed()) {
@@ -161,7 +178,7 @@
 
     synchronized void syncAppendToLogTail(ILogRecord logRecord) {
         if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
-                && logRecord.getLogType() != LogType.WAIT) {
+                && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES) {
             ITransactionContext txnCtx = logRecord.getTxnCtx();
             if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
                 throw new ACIDException(
@@ -348,56 +365,32 @@
         }
     }
 
-    private long initializeLogAnchor(long nextLogFileId) {
-        long fileId = 0;
-        long offset = 0;
-        File fileLogDir = new File(logDir);
-        try {
-            if (fileLogDir.exists()) {
-                List<Long> logFileIds = getLogFileIds();
-                if (logFileIds.isEmpty()) {
-                    fileId = nextLogFileId;
-                    createFileIfNotExists(getLogFilePath(fileId));
-                    if (LOGGER.isInfoEnabled()) {
-                        LOGGER.info("created a log file: " + getLogFilePath(fileId));
-                    }
-                } else {
-                    fileId = logFileIds.get(logFileIds.size() - 1);
-                    File logFile = new File(getLogFilePath(fileId));
-                    offset = logFile.length();
-                }
-            } else {
-                fileId = nextLogFileId;
-                createNewDirectory(logDir);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
-                }
-                createFileIfNotExists(getLogFilePath(fileId));
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("created a log file: " + getLogFilePath(fileId));
-                }
-            }
-        } catch (IOException ioe) {
-            throw new IllegalStateException("Failed to initialize the log anchor", ioe);
-        }
+    private long initializeLogAnchor(long fileId) {
+        final String logFilePath = getLogFilePath(fileId);
+        createFileIfNotExists(logFilePath);
+        final File logFile = new File(logFilePath);
+        long offset = logFile.length();
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("log file Id: " + fileId + ", offset: " + offset);
+            LOGGER.info("initializing log anchor with log file Id: {} at offset: {}", fileId, offset);
         }
-        return logFileSize * fileId + offset;
+        return getLogFileFirstLsn(fileId) + offset;
     }
 
     @Override
     public void renewLogFiles() {
         terminateLogFlusher();
         closeCurrentLogFile();
-        long lastMaxLogFileId = deleteAllLogFiles();
-        initializeLogManager(lastMaxLogFileId + 1);
+        long nextLogFileId = getNextLogFileId();
+        createFileIfNotExists(getLogFilePath(nextLogFileId));
+        final long logFileFirstLsn = getLogFileFirstLsn(nextLogFileId);
+        deleteOldLogFiles(logFileFirstLsn);
+        initializeLogManager(nextLogFileId);
     }
 
     @Override
     public void deleteOldLogFiles(long checkpointLSN) {
         Long checkpointLSNLogFileID = getLogFileId(checkpointLSN);
-        List<Long> logFileIds = getLogFileIds();
+        List<Long> logFileIds = getOrderedLogFileIds();
         if (!logFileIds.isEmpty()) {
             //sort log files from oldest to newest
             Collections.sort(logFileIds);
@@ -444,24 +437,7 @@
         }
     }
 
-    private long deleteAllLogFiles() {
-        List<Long> logFileIds = getLogFileIds();
-        if (!logFileIds.isEmpty()) {
-            for (Long id : logFileIds) {
-                File file = new File(getLogFilePath(id));
-                LOGGER.info("Deleting log file: " + file.getAbsolutePath());
-                if (!file.delete()) {
-                    throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
-                }
-                LOGGER.info("log file: " + file.getAbsolutePath() + " was deleted successfully");
-            }
-            return logFileIds.get(logFileIds.size() - 1);
-        } else {
-            throw new IllegalStateException("Couldn't find any log files.");
-        }
-    }
-
-    public List<Long> getLogFileIds() {
+    public List<Long> getOrderedLogFileIds() {
         File fileLogDir = new File(logDir);
         String[] logFileNames = null;
         List<Long> logFileIds = null;
@@ -493,12 +469,7 @@
         for (String fileName : logFileNames) {
             logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
         }
-        Collections.sort(logFileIds, new Comparator<Long>() {
-            @Override
-            public int compare(Long arg0, Long arg1) {
-                return arg0.compareTo(arg1);
-            }
-        });
+        logFileIds.sort(Long::compareTo);
         return logFileIds;
     }
 
@@ -514,17 +485,21 @@
         return lsn / logFileSize;
     }
 
-    private static boolean createFileIfNotExists(String path) throws IOException {
-        File file = new File(path);
-        File parentFile = file.getParentFile();
-        if (parentFile != null) {
-            parentFile.mkdirs();
+    private static void createFileIfNotExists(String path) {
+        try {
+            File file = new File(path);
+            if (file.exists()) {
+                return;
+            }
+            File parentFile = file.getParentFile();
+            if (parentFile != null) {
+                parentFile.mkdirs();
+            }
+            Files.createFile(file.toPath());
+            LOGGER.info("Created log file {}", path);
+        } catch (IOException e) {
+            throw new IllegalStateException("Failed to create file in " + path, e);
         }
-        return file.createNewFile();
-    }
-
-    private static boolean createNewDirectory(String path) {
-        return (new File(path)).mkdir();
     }
 
     private void createNextLogFile() throws IOException {
@@ -562,7 +537,7 @@
 
     @Override
     public long getReadableSmallestLSN() {
-        List<Long> logFileIds = getLogFileIds();
+        List<Long> logFileIds = getOrderedLogFileIds();
         if (!logFileIds.isEmpty()) {
             return logFileIds.get(0) * logFileSize;
         } else {
@@ -612,6 +587,22 @@
         fileChannel.close();
     }
 
+    private long getNextLogFileId() {
+        return getOnDiskMaxLogFileId() + 1;
+    }
+
+    private long getLogFileFirstLsn(long logFileId) {
+        return logFileId * logFileSize;
+    }
+
+    private long getOnDiskMaxLogFileId() {
+        final List<Long> logFileIds = getOrderedLogFileIds();
+        if (logFileIds.isEmpty()) {
+            return SMALLEST_LOG_FILE_ID;
+        }
+        return logFileIds.get(logFileIds.size() - 1);
+    }
+
     /**
      * This class is used to log FLUSH logs.
      * FLUSH logs are flushed on a different thread to avoid a possible deadlock in {@link LogBuffer} batchUnlock
@@ -686,7 +677,7 @@
             }
         } catch (Exception e) {
             LOGGER.log(Level.ERROR, "LogFlusher is terminating abnormally. System is in unusable state; halting", e);
-            ExitUtil.halt(EC_IMMEDIATE_HALT);
+            ExitUtil.halt(ExitUtil.EC_TXN_LOG_FLUSHER_FAILURE);
             throw new AssertionError("not reachable");
         } finally {
             if (interrupted) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 1e13883..d2e9629 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -40,9 +40,11 @@
         super(txnSubsystem);
     }
 
+    @SuppressWarnings("squid:S2445")
     @Override
     public void log(ILogRecord logRecord) {
-        boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT;
+        boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT
+                && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
         if (shouldReplicate) {
             switch (logRecord.getLogType()) {
                 case LogType.ENTITY_COMMIT:
@@ -63,16 +65,12 @@
             }
         }
         logRecord.setReplicate(shouldReplicate);
-
-        //Remote flush logs do not need to be flushed separately since they may not trigger local flush
-        if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
-            flushLogsQ.add(logRecord);
-            return;
+        if (!logToFlushQueue(logRecord)) {
+            appendToLogTail(logRecord);
         }
-
-        appendToLogTail(logRecord);
     }
 
+    @SuppressWarnings("squid:S2445")
     @Override
     protected void appendToLogTail(ILogRecord logRecord) {
         syncAppendToLogTail(logRecord);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
index f8ab952..e221da8 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/AbstractCheckpointManager.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.transaction.management.service.recovery;
 
-import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
@@ -41,10 +40,14 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 /**
  * An abstract implementation of {@link ICheckpointManager}.
  * The AbstractCheckpointManager contains the implementation of
@@ -54,12 +57,15 @@
 
     private static final Logger LOGGER = LogManager.getLogger();
     private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
     public static final long SHARP_CHECKPOINT_LSN = -1;
     private static final FilenameFilter filter = (File dir, String name) -> name.startsWith(CHECKPOINT_FILENAME_PREFIX);
+    private static final long FIRST_CHECKPOINT_ID = 0;
     private final File checkpointDir;
     private final int historyToKeep;
     private final int lsnThreshold;
     private final int pollFrequency;
+    private final IPersistedResourceRegistry persistedResourceRegistry;
     protected final ITransactionSubsystem txnSubsystem;
     private CheckpointThread checkpointer;
 
@@ -83,64 +89,27 @@
         lsnThreshold = checkpointProperties.getLsnThreshold();
         pollFrequency = checkpointProperties.getPollFrequency();
         // We must keep at least the latest checkpoint
-        historyToKeep = checkpointProperties.getHistoryToKeep() == 0 ? 1 : checkpointProperties.getHistoryToKeep();
+        historyToKeep = checkpointProperties.getHistoryToKeep() + 1;
+        persistedResourceRegistry = txnSubsystem.getApplicationContext().getPersistedResourceRegistry();
     }
 
     @Override
-    public Checkpoint getLatest() throws ACIDException {
-        // Read all checkpointObjects from the existing checkpoint files
+    public Checkpoint getLatest() {
         LOGGER.log(Level.INFO, "Getting latest checkpoint");
-        File[] checkpoints = checkpointDir.listFiles(filter);
-        if (checkpoints == null || checkpoints.length == 0) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.log(Level.INFO,
-                        "Listing of files in the checkpoint dir returned " + (checkpoints == null ? "null" : "empty"));
-            }
+        final List<File> checkpointFiles = getCheckpointFiles();
+        if (checkpointFiles.isEmpty()) {
             return null;
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO, "Listing of files in the checkpoint dir returned " + Arrays.toString(checkpoints));
+        final List<Checkpoint> orderedCheckpoints = getOrderedCheckpoints(checkpointFiles);
+        if (orderedCheckpoints.isEmpty()) {
+            /*
+             * If all checkpoint files are corrupted, we have no option but to try to perform recovery.
+             * We will forge a checkpoint that forces recovery to start from the beginning of the log.
+             * This shouldn't happen unless a hardware corruption happens.
+             */
+            return forgeForceRecoveryCheckpoint();
         }
-        List<Checkpoint> checkpointObjectList = new ArrayList<>();
-        for (File file : checkpoints) {
-            try {
-                if (LOGGER.isWarnEnabled()) {
-                    LOGGER.log(Level.WARN, "Reading checkpoint file: " + file.getAbsolutePath());
-                }
-                String jsonString = new String(Files.readAllBytes(Paths.get(file.getAbsolutePath())));
-                checkpointObjectList.add(Checkpoint.fromJson(jsonString));
-            } catch (ClosedByInterruptException e) {
-                Thread.currentThread().interrupt();
-                if (LOGGER.isWarnEnabled()) {
-                    LOGGER.log(Level.WARN, "Interrupted while reading checkpoint file: " + file.getAbsolutePath(), e);
-                }
-                throw new ACIDException(e);
-            } catch (IOException e) {
-                // ignore corrupted checkpoint file
-                if (LOGGER.isWarnEnabled()) {
-                    LOGGER.log(Level.WARN, "Failed to read checkpoint file: " + file.getAbsolutePath(), e);
-                }
-                file.delete();
-                if (LOGGER.isWarnEnabled()) {
-                    LOGGER.log(Level.WARN, "Deleted corrupted checkpoint file: " + file.getAbsolutePath());
-                }
-            }
-        }
-        /**
-         * If all checkpoint files are corrupted, we have no option but to try to perform recovery.
-         * We will forge a checkpoint that forces recovery to start from the beginning of the log.
-         * This shouldn't happen unless a hardware corruption happens.
-         */
-        if (checkpointObjectList.isEmpty()) {
-            LOGGER.error("All checkpoint files are corrupted. Forcing recovery from the beginning of the log");
-            checkpointObjectList.add(forgeForceRecoveryCheckpoint());
-        }
-
-        // Sort checkpointObjects in descending order by timeStamp to find out the most recent one.
-        Collections.sort(checkpointObjectList);
-
-        // Return the most recent one (the first one in sorted list)
-        return checkpointObjectList.get(0);
+        return orderedCheckpoints.get(orderedCheckpoints.size() - 1);
     }
 
     @Override
@@ -166,34 +135,35 @@
         // Nothing to dump
     }
 
-    public Path getCheckpointPath(long checkpointTimestamp) {
+    public Path getCheckpointPath(long checkpointId) {
         return Paths.get(checkpointDir.getAbsolutePath() + File.separator + CHECKPOINT_FILENAME_PREFIX
-                + Long.toString(checkpointTimestamp));
+                + Long.toString(checkpointId));
     }
 
     protected void capture(long minMCTFirstLSN, boolean sharp) throws HyracksDataException {
         ILogManager logMgr = txnSubsystem.getLogManager();
         ITransactionManager txnMgr = txnSubsystem.getTransactionManager();
-        Checkpoint checkpointObject = new Checkpoint(logMgr.getAppendLSN(), minMCTFirstLSN, txnMgr.getMaxTxnId(),
-                System.currentTimeMillis(), sharp, StorageConstants.VERSION);
+        final long nextCheckpointId = getNextCheckpointId();
+        final Checkpoint checkpointObject = new Checkpoint(nextCheckpointId, logMgr.getAppendLSN(), minMCTFirstLSN,
+                txnMgr.getMaxTxnId(), sharp, StorageConstants.VERSION);
         persist(checkpointObject);
         cleanup();
     }
 
-    protected Checkpoint forgeForceRecoveryCheckpoint() {
-        /**
+    private Checkpoint forgeForceRecoveryCheckpoint() {
+        /*
          * By setting the checkpoint first LSN (low watermark) to Long.MIN_VALUE, the recovery manager will start from
          * the first available log.
          * We set the storage version to the current version. If there is a version mismatch, it will be detected
          * during recovery.
          */
-        return new Checkpoint(Long.MIN_VALUE, Long.MIN_VALUE, Integer.MIN_VALUE, System.currentTimeMillis(), false,
+        return new Checkpoint(Long.MIN_VALUE, Long.MIN_VALUE, Integer.MIN_VALUE, FIRST_CHECKPOINT_ID, false,
                 StorageConstants.VERSION);
     }
 
     private void persist(Checkpoint checkpoint) throws HyracksDataException {
         // Get checkpoint file path
-        Path path = getCheckpointPath(checkpoint.getTimeStamp());
+        Path path = getCheckpointPath(checkpoint.getId());
 
         if (LOGGER.isInfoEnabled()) {
             File file = path.toFile();
@@ -201,9 +171,9 @@
                     + (file.exists() ? "already exists" : "doesn't exist yet"));
         }
         // Write checkpoint file to disk
-        try (BufferedWriter writer = Files.newBufferedWriter(path)) {
-            writer.write(checkpoint.asJson());
-            writer.flush();
+        try {
+            byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(checkpoint.toJson(persistedResourceRegistry));
+            Files.write(path, bytes);
         } catch (IOException e) {
             LOGGER.log(Level.ERROR, "Failed to write checkpoint to disk", e);
             throw HyracksDataException.create(e);
@@ -215,17 +185,73 @@
         }
     }
 
-    private void cleanup() {
-        File[] checkpointFiles = checkpointDir.listFiles(filter);
-        // Sort the filenames lexicographically to keep the latest checkpoint history files.
-        Arrays.sort(checkpointFiles);
-        for (int i = 0; i < checkpointFiles.length - historyToKeep; i++) {
-            if (LOGGER.isWarnEnabled()) {
-                LOGGER.warn("Deleting checkpoint file at: " + checkpointFiles[i].getAbsolutePath());
+    private List<File> getCheckpointFiles() {
+        File[] checkpoints = checkpointDir.listFiles(filter);
+        if (checkpoints == null || checkpoints.length == 0) {
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.log(Level.INFO,
+                        "Listing of files in the checkpoint dir returned " + (checkpoints == null ? "null" : "empty"));
             }
-            if (!checkpointFiles[i].delete() && LOGGER.isWarnEnabled()) {
-                LOGGER.warn("Could not delete checkpoint file at: " + checkpointFiles[i].getAbsolutePath());
+            return Collections.emptyList();
+        }
+        if (LOGGER.isInfoEnabled()) {
+            LOGGER.log(Level.INFO, "Listing of files in the checkpoint dir returned " + Arrays.toString(checkpoints));
+        }
+        return Arrays.asList(checkpoints);
+    }
+
+    private List<Checkpoint> getOrderedCheckpoints(List<File> checkpoints) {
+        List<Checkpoint> checkpointObjectList = new ArrayList<>();
+        for (File file : checkpoints) {
+            try {
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.log(Level.WARN, "Reading checkpoint file: " + file.getAbsolutePath());
+                }
+                final JsonNode jsonNode =
+                        OBJECT_MAPPER.readValue(Files.readAllBytes(Paths.get(file.getAbsolutePath())), JsonNode.class);
+                Checkpoint cp = (Checkpoint) persistedResourceRegistry.deserialize(jsonNode);
+                checkpointObjectList.add(cp);
+            } catch (ClosedByInterruptException e) {
+                Thread.currentThread().interrupt();
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.log(Level.WARN, "Interrupted while reading checkpoint file: " + file.getAbsolutePath(), e);
+                }
+                throw new ACIDException(e);
+            } catch (IOException e) {
+                // ignore corrupted checkpoint file
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.log(Level.WARN, "Failed to read checkpoint file: " + file.getAbsolutePath(), e);
+                }
+                file.delete();
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.log(Level.WARN, "Deleted corrupted checkpoint file: " + file.getAbsolutePath());
+                }
+            }
+        }
+        Collections.sort(checkpointObjectList);
+        return checkpointObjectList;
+    }
+
+    private void cleanup() {
+        final List<File> checkpointFiles = getCheckpointFiles();
+        final List<Checkpoint> orderedCheckpoints = getOrderedCheckpoints(checkpointFiles);
+        final int deleteCount = orderedCheckpoints.size() - historyToKeep;
+        for (int i = 0; i < deleteCount; i++) {
+            final Checkpoint checkpoint = orderedCheckpoints.get(i);
+            final Path checkpointPath = getCheckpointPath(checkpoint.getId());
+            LOGGER.warn("Deleting checkpoint file at: {}", checkpointPath);
+            if (!checkpointPath.toFile().delete()) {
+                LOGGER.warn("Could not delete checkpoint file at: {}", checkpointPath);
             }
         }
     }
+
+    private long getNextCheckpointId() {
+        final Checkpoint latest = getLatest();
+        if (latest == null) {
+            return FIRST_CHECKPOINT_ID;
+        }
+        return latest.getId() + 1;
+    }
+
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
index 6efd0e5..ce523db 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
@@ -38,7 +38,7 @@
 public class CheckpointManager extends AbstractCheckpointManager {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final long NO_SECURED_LSN = -1l;
+    private static final long NO_SECURED_LSN = -1L;
     private final Map<TxnId, Long> securedLSNs;
 
     public CheckpointManager(ITransactionSubsystem txnSubsystem, CheckpointProperties checkpointProperties) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AbstractTransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AbstractTransactionContext.java
index 95cabf9..a0944ea 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AbstractTransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AbstractTransactionContext.java
@@ -36,7 +36,7 @@
 public abstract class AbstractTransactionContext implements ITransactionContext {
 
     protected final TxnId txnId;
-    protected final Map<Long, ITransactionOperationTracker> txnOpTrackers;
+    private final Map<Long, ITransactionOperationTracker> txnOpTrackers;
     private final AtomicLong firstLSN;
     private final AtomicLong lastLSN;
     private final AtomicInteger txnState;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AtomicTransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AtomicTransactionContext.java
index 079e99a..083c26b 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AtomicTransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/AtomicTransactionContext.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.transaction.management.service.transaction;
 
-import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.common.exceptions.ACIDException;
@@ -35,9 +35,9 @@
 @ThreadSafe
 public class AtomicTransactionContext extends AbstractTransactionContext {
 
-    private final Map<Long, ILSMOperationTracker> opTrackers = new HashMap<>();
-    private final Map<Long, AtomicInteger> indexPendingOps = new HashMap<>();
-    private final Map<Long, IModificationOperationCallback> callbacks = new HashMap<>();
+    private final Map<Long, ILSMOperationTracker> opTrackers = new ConcurrentHashMap<>();
+    private final Map<Long, AtomicInteger> indexPendingOps = new ConcurrentHashMap<>();
+    private final Map<Long, IModificationOperationCallback> callbacks = new ConcurrentHashMap<>();
 
     public AtomicTransactionContext(TxnId txnId) {
         super(txnId);
@@ -47,7 +47,7 @@
     public void register(long resourceId, int partition, ILSMIndex index, IModificationOperationCallback callback,
             boolean primaryIndex) {
         super.register(resourceId, partition, index, callback, primaryIndex);
-        synchronized (txnOpTrackers) {
+        synchronized (opTrackers) {
             if (primaryIndex && !opTrackers.containsKey(resourceId)) {
                 opTrackers.put(resourceId, index.getOperationTracker());
                 callbacks.put(resourceId, callback);
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/EntityLevelTransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/EntityLevelTransactionContext.java
index 9fcb08b..188bb1b3 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/EntityLevelTransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/EntityLevelTransactionContext.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.transaction.management.service.transaction;
 
-import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
@@ -42,23 +42,21 @@
 
     public EntityLevelTransactionContext(TxnId txnId) {
         super(txnId);
-        this.primaryIndexTrackers = new HashMap<>();
-        this.resourcePendingOps = new HashMap<>();
-        this.partitionPendingOps = new HashMap<>();
+        this.primaryIndexTrackers = new ConcurrentHashMap<>();
+        this.resourcePendingOps = new ConcurrentHashMap<>();
+        this.partitionPendingOps = new ConcurrentHashMap<>();
     }
 
     @Override
     public void register(long resourceId, int partition, ILSMIndex index, IModificationOperationCallback callback,
             boolean primaryIndex) {
         super.register(resourceId, partition, index, callback, primaryIndex);
-        synchronized (txnOpTrackers) {
-            AtomicInteger pendingOps = partitionPendingOps.computeIfAbsent(partition, p -> new AtomicInteger(0));
-            resourcePendingOps.put(resourceId, pendingOps);
-            if (primaryIndex) {
-                Pair<PrimaryIndexOperationTracker, IModificationOperationCallback> pair =
-                        new Pair<>((PrimaryIndexOperationTracker) index.getOperationTracker(), callback);
-                primaryIndexTrackers.put(partition, pair);
-            }
+        AtomicInteger pendingOps = partitionPendingOps.computeIfAbsent(partition, p -> new AtomicInteger(0));
+        resourcePendingOps.put(resourceId, pendingOps);
+        if (primaryIndex) {
+            Pair<PrimaryIndexOperationTracker, IModificationOperationCallback> pair =
+                    new Pair<>((PrimaryIndexOperationTracker) index.getOperationTracker(), callback);
+            primaryIndexTrackers.put(partition, pair);
         }
     }
 
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index d8aab3d..fbf17c1 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -46,7 +46,7 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
     <jvm.extraargs />
-    <jdk.version>1.8</jdk.version>
+    <source.jdk.version>1.8</source.jdk.version>
     <javac.xlint.value>all</javac.xlint.value>
     <source-format.goal>format</source-format.goal>
     <source-format.skip>false</source-format.skip>
@@ -75,7 +75,7 @@
     <!-- Versions under dependencymanagement or used in many projects via properties -->
     <algebricks.version>0.3.4-SNAPSHOT</algebricks.version>
     <hyracks.version>0.3.4-SNAPSHOT</hyracks.version>
-    <hadoop.version>2.2.0</hadoop.version>
+    <hadoop.version>2.8.5</hadoop.version>
     <jacoco.version>0.7.6.201602180812</jacoco.version>
 
     <implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
@@ -197,8 +197,8 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>${jdk.version}</source>
-          <target>${jdk.version}</target>
+          <source>${source.jdk.version}</source>
+          <target>${target.jdk.version}</target>
           <compilerArgument>-Xlint:${javac.xlint.value}</compilerArgument>
         </configuration>
       </plugin>
@@ -321,7 +321,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
-          <version>3.7.0</version>
+          <version>3.8.0</version>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
         <plugin>
@@ -523,7 +523,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.0.2</version>
+          <version>3.1.1</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -533,7 +533,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-plugin-plugin</artifactId>
-          <version>3.5.1</version>
+          <version>3.5.2</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -765,6 +765,68 @@
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
+    <profile>
+      <id>java8</id>
+      <activation>
+        <jdk>1.8</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>8</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java9</id>
+      <activation>
+        <jdk>9</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>9</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java10</id>
+      <activation>
+        <jdk>10</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>10</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java11</id>
+      <activation>
+        <jdk>11</jdk>
+      </activation>
+      <properties>
+        <!-- several plugins do not yet support Java 11 classes (maven-plugin-plugin, maven-dependency-plugin) -->
+        <target.jdk.version>10</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java8-bootclasspath</id>
+      <activation>
+        <property>
+          <name>java8-bootclasspath</name>
+        </property>
+      </activation>
+      <properties>
+        <target.jdk.version>8</target.jdk.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <compilerArgs combine.children="append">
+                <arg>-bootclasspath</arg>
+                <arg>${java8-bootclasspath}</arg>
+              </compilerArgs>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
 
   <modules>
@@ -795,6 +857,7 @@
     <module>asterix-active</module>
     <module>asterix-client-helper</module>
     <module>asterix-license</module>
+    <module>asterix-geo</module>
   </modules>
 
   <dependencyManagement>
@@ -809,17 +872,17 @@
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-simple</artifactId>
-        <version>1.7.5</version>
+        <version>1.7.25</version>
       </dependency>
       <dependency>
         <groupId>org.apache.maven</groupId>
         <artifactId>maven-plugin-api</artifactId>
-        <version>3.3.9</version>
+        <version>3.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.maven</groupId>
         <artifactId>maven-model</artifactId>
-        <version>3.3.9</version>
+        <version>3.6.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.maven</groupId>
@@ -867,10 +930,19 @@
             <groupId>jdk.tools</groupId>
             <artifactId>jdk.tools</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>jdk.tools</groupId>
+            <artifactId>jdk.tools</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-hdfs-client</artifactId>
+        <version>${hadoop.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-common</artifactId>
         <version>${hadoop.version}</version>
         <classifier>tests</classifier>
@@ -922,7 +994,7 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-compress</artifactId>
-        <version>1.15</version>
+        <version>1.18</version>
       </dependency>
       <dependency>
         <groupId>commons-logging</groupId>
@@ -1172,7 +1244,7 @@
       <dependency>
         <groupId>org.apache.zookeeper</groupId>
         <artifactId>zookeeper</artifactId>
-        <version>3.4.5</version>
+        <version>3.4.13</version>
         <exclusions>
           <exclusion>
             <groupId>com.sun.jmx</groupId>
@@ -1207,12 +1279,17 @@
       <dependency>
         <groupId>commons-codec</groupId>
         <artifactId>commons-codec</artifactId>
-        <version>1.9</version>
+        <version>1.11</version>
+      </dependency>
+      <dependency>
+        <groupId>it.unimi.dsi</groupId>
+        <artifactId>fastutil</artifactId>
+        <version>8.2.2</version>
       </dependency>
       <dependency>
         <groupId>com.sun.xml.bind</groupId>
         <artifactId>jaxb-core</artifactId>
-        <version>2.3.0</version>
+        <version>2.3.0.1</version>
       </dependency>
       <dependency>
         <groupId>com.sun.xml.bind</groupId>
@@ -1234,6 +1311,21 @@
         <artifactId>maven-core</artifactId>
         <version>3.3.9</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.maven</groupId>
+        <artifactId>maven-compat</artifactId>
+        <version>3.3.9</version>
+      </dependency>
+      <dependency>
+        <groupId>com.esri.geometry</groupId>
+        <artifactId>esri-geometry-api</artifactId>
+        <version>2.0.0</version>
+      </dependency>
+      <dependency>
+        <groupId>org.reflections</groupId>
+        <artifactId>reflections</artifactId>
+        <version>0.9.10</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index c4bdd06..18ad7ef 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -34,13 +34,13 @@
       <licenses>
         <license>
           <name>an MIT-style license</name>
-          <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.8.0-alpha2/LICENSE.txt</url>
+          <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.7.25/slf4j-api/LICENSE.txt</url>
         </license>
       </licenses>
       <properties>
-        <license.ignoreMissingEmbeddedNotice>1.8.0-alpha2</license.ignoreMissingEmbeddedNotice>
-        <license.ignoreMissingEmbeddedLicense>1.8.0-alpha2</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreLicenseOverride>1.8.0-alpha2</license.ignoreLicenseOverride>
+        <license.ignoreMissingEmbeddedNotice>1.7.25</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.7.25</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.7.25</license.ignoreLicenseOverride>
       </properties>
     </project>
   </supplement>
@@ -160,9 +160,9 @@
       <artifactId>netty-all</artifactId>
       <properties>
         <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
-        <license.ignoreMissingEmbeddedLicense>4.1.6.Final</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>4.1.6.Final</license.ignoreMissingEmbeddedNotice>
-        <license.ignoreNoticeOverride>4.1.6.Final</license.ignoreNoticeOverride>
+        <license.ignoreMissingEmbeddedLicense>4.1.25.Final,4.1.32.Final</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>4.1.25.Final,4.1.32.Final</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreNoticeOverride>4.1.25.Final,4.1.32.Final</license.ignoreNoticeOverride>
       </properties>
     </project>
   </supplement>
@@ -172,6 +172,7 @@
       <artifactId>guava</artifactId>
       <properties>
         <!-- guava is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+        <!-- see https://github.com/google/guava/blob/v18.0/COPYING -->
         <license.ignoreMissingEmbeddedNotice>18.0</license.ignoreMissingEmbeddedNotice>
         <license.ignoreMissingEmbeddedLicense>18.0</license.ignoreMissingEmbeddedLicense>
       </properties>
@@ -204,7 +205,31 @@
       <artifactId>jackson-annotations</artifactId>
       <properties>
         <!-- jackson-annotation does not provide an embedded NOTICE file -->
-        <license.ignoreMissingEmbeddedNotice>2.8.4</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedNotice>2.8.4,2.9.7</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>com.esri.geometry</groupId>
+      <artifactId>esri-geometry-api</artifactId>
+      <properties>
+        <!-- esri is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+        <license.ignoreMissingEmbeddedLicense>2.0.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.0.0</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>it.unimi.dsi</groupId>
+      <artifactId>fastutil</artifactId>
+      <properties>
+        <!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+        <!-- see https://github.com/vigna/fastutil/blob/8.1.1/LICENSE-2.0 -->
+        <!-- see https://github.com/vigna/fastutil/blob/8.2.2/LICENSE-2.0 -->
+        <license.ignoreMissingEmbeddedLicense>8.1.1,8.2.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>8.1.1,8.2.2</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
diff --git a/asterixdb/src/main/licenses/content/opensource.org_licenses_mit-license.php.txt b/asterixdb/src/main/licenses/content/opensource.org_licenses_mit-license.php.txt
new file mode 100644
index 0000000..ece42d0
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/opensource.org_licenses_mit-license.php.txt
@@ -0,0 +1,7 @@
+The MIT License
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt
deleted file mode 100644
index 92dfd5b..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.16.Final_NOTICE.txt
+++ /dev/null
@@ -1,214 +0,0 @@
-
-                            The Netty Project
-                            =================
-
-Please visit the Netty web site for more information:
-
-  * http://netty.io/
-
-Copyright 2014 The Netty Project
-
-The Netty Project 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.
-
-Also, please refer to each LICENSE.<component>.txt file, which is located in
-the 'license' directory of the distribution file, for the license terms of the
-components that this product depends on.
-
--------------------------------------------------------------------------------
-This product contains the extensions to Java Collections Framework which has
-been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
-  * LICENSE:
-    * license/LICENSE.jsr166y.txt (Public Domain)
-  * HOMEPAGE:
-    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
-    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-This product contains a modified version of Robert Harder's Public Domain
-Base64 Encoder and Decoder, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.base64.txt (Public Domain)
-  * HOMEPAGE:
-    * http://iharder.sourceforge.net/current/java/base64/
-
-This product contains a modified portion of 'Webbit', an event based  
-WebSocket and HTTP server, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.webbit.txt (BSD License)
-  * HOMEPAGE:
-    * https://github.com/joewalnes/webbit
-
-This product contains a modified portion of 'SLF4J', a simple logging
-facade for Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.slf4j.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.slf4j.org/
-
-This product contains a modified portion of 'Apache Harmony', an open source
-Java SE, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.harmony.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://archive.apache.org/dist/harmony/
-
-This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-and decompression library written by Matthew J. Francis. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jbzip2.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jbzip2/
-
-This product contains a modified portion of 'libdivsufsort', a C API library to construct
-the suffix array and the Burrows-Wheeler transformed string for any input string of
-a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.libdivsufsort.txt (MIT License)
-  * HOMEPAGE:
-    * https://github.com/y-256/libdivsufsort
-
-This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
- which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jctools.txt (ASL2 License)
-  * HOMEPAGE:
-    * https://github.com/JCTools/JCTools
-
-This product optionally depends on 'JZlib', a re-implementation of zlib in
-pure Java, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jzlib.txt (BSD style License)
-  * HOMEPAGE:
-    * http://www.jcraft.com/jzlib/
-
-This product optionally depends on 'Compress-LZF', a Java library for encoding and
-decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/ning/compress
-
-This product optionally depends on 'lz4', a LZ4 Java compression
-and decompression library written by Adrien Grand. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lz4.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jpountz/lz4-java
-
-This product optionally depends on 'lzma-java', a LZMA Java compression
-and decompression library, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.lzma-java.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/jponge/lzma-java
-
-This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-and decompression library written by William Kinney. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jfastlz.txt (MIT License)
-  * HOMEPAGE:
-    * https://code.google.com/p/jfastlz/
-
-This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-interchange format, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.protobuf.txt (New BSD License)
-  * HOMEPAGE:
-    * https://github.com/google/protobuf
-
-This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
-a temporary self-signed X.509 certificate when the JVM does not provide the
-equivalent functionality.  It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.bouncycastle.txt (MIT License)
-  * HOMEPAGE:
-    * http://www.bouncycastle.org/
-
-This product optionally depends on 'Snappy', a compression library produced
-by Google Inc, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.snappy.txt (New BSD License)
-  * HOMEPAGE:
-    * https://github.com/google/snappy
-
-This product optionally depends on 'JBoss Marshalling', an alternative Java
-serialization API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
-  * HOMEPAGE:
-    * http://www.jboss.org/jbossmarshalling
-
-This product optionally depends on 'Caliper', Google's micro-
-benchmarking framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.caliper.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/google/caliper
-
-This product optionally depends on 'Apache Commons Logging', a logging
-framework, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-logging.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://commons.apache.org/logging/
-
-This product optionally depends on 'Apache Log4J', a logging framework, which
-can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.log4j.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://logging.apache.org/log4j/
-
-This product optionally depends on 'Aalto XML', an ultra-high performance
-non-blocking XML processor, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * http://wiki.fasterxml.com/AaltoHome
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.hpack.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://github.com/twitter/hpack
-
-This product contains a modified portion of 'Apache Commons Lang', a Java library
-provides utilities for the java.lang API, which can be obtained at:
-
-  * LICENSE:
-    * license/LICENSE.commons-lang.txt (Apache License 2.0)
-  * HOMEPAGE:
-    * https://commons.apache.org/proper/commons-lang/
-
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.32.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.32.Final_NOTICE.txt
new file mode 100644
index 0000000..f973663
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.32.Final_NOTICE.txt
@@ -0,0 +1,223 @@
+
+                            The Netty Project
+                            =================
+
+Please visit the Netty web site for more information:
+
+  * http://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+  * LICENSE:
+    * license/LICENSE.jsr166y.txt (Public Domain)
+  * HOMEPAGE:
+    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.base64.txt (Public Domain)
+  * HOMEPAGE:
+    * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.webbit.txt (BSD License)
+  * HOMEPAGE:
+    * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.slf4j.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.slf4j.org/
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+  * NOTICE:
+    * license/NOTICE.harmony.txt
+  * LICENSE:
+    * license/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://archive.apache.org/dist/harmony/
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jbzip2.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.libdivsufsort.txt (MIT License)
+  * HOMEPAGE:
+    * https://github.com/y-256/libdivsufsort
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jctools.txt (ASL2 License)
+  * HOMEPAGE:
+    * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jzlib.txt (BSD style License)
+  * HOMEPAGE:
+    * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lz4.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/jponge/lzma-java
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jfastlz.txt (MIT License)
+  * HOMEPAGE:
+    * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.protobuf.txt (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/google/protobuf
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality.  It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.bouncycastle.txt (MIT License)
+  * HOMEPAGE:
+    * http://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.snappy.txt (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/google/snappy
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+  * HOMEPAGE:
+    * http://www.jboss.org/jbossmarshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.caliper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/google/caliper
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.log4j.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.hpack.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/twitter/hpack
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+  * LICENSE:
+    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://commons.apache.org/proper/commons-lang/
+
+
+This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
+
+  * LICENSE:
+    * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/takari/maven-wrapper
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt
new file mode 100644
index 0000000..5a11c0c
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt
@@ -0,0 +1,21 @@
+Copyright (c) 2004-2007 QOS.ch
+All rights reserved.
+
+Permission is hereby granted, free  of charge, to any person obtaining
+a  copy  of this  software  and  associated  documentation files  (the
+"Software"), to  deal in  the Software without  restriction, including
+without limitation  the rights to  use, copy, modify,  merge, publish,
+distribute,  sublicense, and/or sell  copies of  the Software,  and to
+permit persons to whom the Software  is furnished to do so, subject to
+the following conditions:
+
+The  above  copyright  notice  and  this permission  notice  shall  be
+included in all copies or substantial portions of the Software.
+
+THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.8.0-alpha2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.8.0-alpha2_LICENSE.txt
deleted file mode 100644
index 744377c..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.8.0-alpha2_LICENSE.txt
+++ /dev/null
@@ -1,21 +0,0 @@
-Copyright (c) 2004-2017 QOS.ch
-All rights reserved.
-
-Permission is hereby granted, free  of charge, to any person obtaining
-a  copy  of this  software  and  associated  documentation files  (the
-"Software"), to  deal in  the Software without  restriction, including
-without limitation  the rights to  use, copy, modify,  merge, publish,
-distribute,  sublicense, and/or sell  copies of  the Software,  and to
-permit persons to whom the Software  is furnished to do so, subject to
-the following conditions:
-
-The  above  copyright  notice  and  this permission  notice  shall  be
-included in all copies or substantial portions of the Software.
-
-THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
-EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
-MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
-NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
-LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
-OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
-WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/templates/asterix-license.ftl b/asterixdb/src/main/licenses/templates/asterix-license.ftl
index bf2c7ea..381d400 100644
--- a/asterixdb/src/main/licenses/templates/asterix-license.ftl
+++ b/asterixdb/src/main/licenses/templates/asterix-license.ftl
@@ -46,7 +46,7 @@
    <#list projects as p>
        * ${p.name} (${p.groupId}:${p.artifactId}:${p.version})
      <#list p.locations as loc>
-         - ${loc}${p.artifactId}-${p.version}.jar
+         - ${loc}${p.jarName}
        </#list>
    </#list>
 
diff --git a/asterixdb/src/main/licenses/templates/asterix-notice.ftl b/asterixdb/src/main/licenses/templates/asterix-notice.ftl
index d31e408..41cd8a0 100644
--- a/asterixdb/src/main/licenses/templates/asterix-notice.ftl
+++ b/asterixdb/src/main/licenses/templates/asterix-notice.ftl
@@ -36,7 +36,7 @@
    <#list projects as p>
 ${p.name} (${p.groupId}:${p.artifactId}:${p.version})
        <#list p.locations as loc>
-- ${loc}${p.artifactId}-${p.version}.jar
+- ${loc}${p.jarName}
        </#list>
    </#list>
 
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/constraints/AlgebricksAbsolutePartitionConstraint.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/constraints/AlgebricksAbsolutePartitionConstraint.java
index b6443c4..fa4a707 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/constraints/AlgebricksAbsolutePartitionConstraint.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/constraints/AlgebricksAbsolutePartitionConstraint.java
@@ -25,10 +25,12 @@
 
 public class AlgebricksAbsolutePartitionConstraint extends AlgebricksPartitionConstraint {
     private final String[] locations;
+    private final String[] sortedLocations;
 
     public AlgebricksAbsolutePartitionConstraint(String[] locations) {
-        this.locations = locations.clone();
-        Arrays.sort(locations);
+        this.locations = locations;
+        sortedLocations = locations.clone();
+        Arrays.sort(sortedLocations);
     }
 
     @Override
@@ -56,7 +58,7 @@
                 break;
             case ABSOLUTE:
                 AlgebricksAbsolutePartitionConstraint thatAbsolute = (AlgebricksAbsolutePartitionConstraint) that;
-                if (Arrays.equals(locations, thatAbsolute.locations)) {
+                if (Arrays.equals(sortedLocations, thatAbsolute.sortedLocations)) {
                     return this;
                 }
                 break;
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index adb6c79..46e80be 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -21,9 +21,11 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
-public class AlgebricksException extends Exception {
+public class AlgebricksException extends Exception implements IFormattedException {
     private static final long serialVersionUID = 1L;
 
     public static final int UNKNOWN = 0;
@@ -31,15 +33,17 @@
     private final int errorCode;
     private final Serializable[] params;
     private final String nodeId;
+    private final SourceLocation sourceLoc;
 
     @SuppressWarnings("squid:S1165") // exception class not final
     private transient CachedMessage msgCache;
 
-    public AlgebricksException(String component, int errorCode, String message, Throwable cause, String nodeId,
-            Serializable... params) {
+    public AlgebricksException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, String nodeId, Serializable... params) {
         super(message, cause);
         this.component = component;
         this.errorCode = errorCode;
+        this.sourceLoc = sourceLoc;
         this.nodeId = nodeId;
         this.params = params;
     }
@@ -49,7 +53,7 @@
      */
     @Deprecated
     public AlgebricksException(String message) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, (Serializable[]) null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
     }
 
     /**
@@ -65,38 +69,68 @@
      */
     @Deprecated
     public AlgebricksException(String message, Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, null, (Serializable[]) null);
+    }
+
+    public AlgebricksException(String component, int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        this(component, errorCode, null, null, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, Serializable... params) {
-        this(component, errorCode, null, null, null, params);
+        this(component, errorCode, null, null, null, null, params);
+    }
+
+    public AlgebricksException(Throwable cause, int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(Throwable cause, int errorCode, Serializable... params) {
-        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
+        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, String message, SourceLocation sourceLoc,
+            Serializable... params) {
+        this(component, errorCode, message, null, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, String message, Serializable... params) {
-        this(component, errorCode, message, null, null, params);
+        this(component, errorCode, message, null, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, Throwable cause, SourceLocation sourceLoc,
+            Serializable... params) {
+        this(component, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, Throwable cause, Serializable... params) {
-        this(component, errorCode, cause.getMessage(), cause, null, params);
+        this(component, errorCode, cause.getMessage(), cause, null, null, params);
+    }
+
+    public AlgebricksException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, Serializable... params) {
+        this(component, errorCode, message, cause, sourceLoc, null, params);
     }
 
     public AlgebricksException(String component, int errorCode, String message, Throwable cause,
             Serializable... params) {
-        this(component, errorCode, message, cause, null, params);
+        this(component, errorCode, message, cause, null, null, params);
+    }
+
+    public static AlgebricksException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
+        return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc,
+                params);
     }
 
     public static AlgebricksException create(int errorCode, Serializable... params) {
-        return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+        return create(errorCode, null, params);
     }
 
+    @Override
     public String getComponent() {
         return component;
     }
 
+    @Override
     public int getErrorCode() {
         return errorCode;
     }
@@ -109,11 +143,15 @@
         return nodeId;
     }
 
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
     @Override
     public String getMessage() {
         if (msgCache == null) {
-            msgCache =
-                    new CachedMessage(ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params));
+            msgCache = new CachedMessage(
+                    ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params));
         }
         return msgCache.message;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
index 16bc73e..d308969 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/NotImplementedException.java
@@ -22,18 +22,10 @@
     private static final long serialVersionUID = 2L;
 
     public NotImplementedException() {
-        System.err.println("Not implemented.");
+        super("Not implemented.");
     }
 
     public NotImplementedException(String message) {
         super(message);
     }
-
-    public NotImplementedException(Throwable cause) {
-        super(cause);
-    }
-
-    public NotImplementedException(String message, Throwable cause) {
-        super(message, cause);
-    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 2d6123e..a25e2a3 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -93,7 +93,7 @@
                     @Override
                     public JobSpecification createJob(Object appContext,
                             IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting Job Generation.\n");
+                        AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
                         JobGenContext context = new JobGenContext(null, metadata, appContext,
                                 serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
                                 comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 339cb15..18aa104 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -36,10 +36,12 @@
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
 @PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
 public class SequentialFirstRuleCheckFixpointRuleControllerTest {
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index 669988c..2a97b6f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -78,6 +78,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
index 7e10203..1de7e3c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalExpression.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILogicalExpression {
 
@@ -73,4 +74,6 @@
     public abstract ILogicalExpression cloneExpression();
 
     public boolean isFunctional();
+
+    SourceLocation getSourceLocation();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
index dd7e065..6bd0d02 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalOperator.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface ILogicalOperator {
 
@@ -103,4 +104,6 @@
      * Indicates whether the expressions used by this operator must be variable reference expressions.
      */
     public boolean requiresVariableReferenceExpressions();
+
+    SourceLocation getSourceLocation();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
deleted file mode 100644
index 3ecdb35..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.hyracks.algebricks.core.algebra.base;
-
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public interface ILogicalPlanAndMetadata {
-    public ILogicalPlan getPlan();
-
-    public IMetadataProvider<?, ?> getMetadataProvider();
-
-    public AlgebricksPartitionConstraint getClusterLocations();
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
index 0717c0e..de4d5ac 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractLogicalExpression.java
@@ -25,9 +25,12 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractLogicalExpression implements ILogicalExpression {
 
+    protected SourceLocation sourceLoc;
+
     @Override
     public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
             Map<LogicalVariable, EquivalenceClass> equivClasses) {
@@ -45,4 +48,12 @@
         return true;
     }
 
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
index 8fca47c..bdd820e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
@@ -59,6 +59,7 @@
         AggregateFunctionCallExpression fun = new AggregateFunctionCallExpression(finfo, twoStep, clonedArgs);
         fun.setStepTwoAggregate(stepTwoAggregate);
         fun.setStepOneAggregate(stepOneAggregate);
+        fun.setSourceLocation(sourceLoc);
         return fun;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index 689d51c..42ff3c0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -174,6 +174,7 @@
         annotationMap.forEach((key, value1) -> m.put(key, value1.copy()));
         ConstantExpression c = new ConstantExpression(value);
         c.annotationMap = m;
+        c.setSourceLocation(sourceLoc);
         return c;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
index 6308636..f8b25e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ScalarFunctionCallExpression.java
@@ -47,6 +47,7 @@
         ScalarFunctionCallExpression funcExpr = new ScalarFunctionCallExpression(finfo, clonedArgs);
         funcExpr.getAnnotations().putAll(cloneAnnotations());
         funcExpr.setOpaqueParameters(this.getOpaqueParameters());
+        funcExpr.setSourceLocation(sourceLoc);
         return funcExpr;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
index 661c389..0d310f7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/StatefulFunctionCallExpression.java
@@ -52,7 +52,10 @@
     public StatefulFunctionCallExpression cloneExpression() {
         cloneAnnotations();
         List<Mutable<ILogicalExpression>> clonedArgs = cloneArguments();
-        return new StatefulFunctionCallExpression(finfo, propertiesComputer, clonedArgs);
+        StatefulFunctionCallExpression clonedExpr =
+                new StatefulFunctionCallExpression(finfo, propertiesComputer, clonedArgs);
+        clonedExpr.setSourceLocation(sourceLoc);
+        return clonedExpr;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
index ac539ec..3dad669 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/UnnestingFunctionCallExpression.java
@@ -50,6 +50,7 @@
         UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(finfo, clonedArgs);
         ufce.setReturnsUniqueValues(returnsUniqueValues);
         ufce.setOpaqueParameters(this.getOpaqueParameters());
+        ufce.setSourceLocation(sourceLoc);
         return ufce;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
index c1342cc..1c7da34 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/VariableReferenceExpression.java
@@ -95,7 +95,9 @@
 
     @Override
     public AbstractLogicalExpression cloneExpression() {
-        return new VariableReferenceExpression(variable);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(variable);
+        varRef.setSourceLocation(sourceLoc);
+        return varRef;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 6860147..3d004a2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 public interface IMetadataProvider<S, I> {
     public IDataSource<S> findDataSource(S id) throws AlgebricksException;
@@ -47,7 +48,8 @@
      */
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
-            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException;
 
@@ -203,8 +205,14 @@
             IDataSourceIndex<I, S> dataSourceIndex, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
             List<LogicalVariable> additionalFilteringKeys, ILogicalExpression filterExpr,
-            List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKeys,
-            RecordDescriptor inputDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException;
+            LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
+            LogicalVariable prevAdditionalFilteringKeys, RecordDescriptor inputDesc, JobGenContext context,
+            JobSpecification spec) throws AlgebricksException;
 
-    public Map<String, String> getConfig();
+    public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+            IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
+            throws AlgebricksException;
+
+    public Map<String, Object> getConfig();
+
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
index 4686f32..1dbf15e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
 import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractLogicalOperator implements ILogicalOperator {
 
@@ -69,6 +70,8 @@
     protected final List<Mutable<ILogicalOperator>> inputs;
     protected List<LogicalVariable> schema;
 
+    private SourceLocation sourceLoc;
+
     public AbstractLogicalOperator() {
         inputs = new ArrayList<>();
     }
@@ -195,4 +198,13 @@
     public boolean requiresVariableReferenceExpressions() {
         return true;
     }
+
+    @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index 2511fa9..6f11dc1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -35,7 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
 public class DataSourceScanOperator extends AbstractDataSourceOperator {
-    private List<LogicalVariable> projectVars;
+    private final List<LogicalVariable> projectVars;
 
     private boolean projectPushed = false;
 
@@ -43,9 +43,22 @@
     private List<LogicalVariable> minFilterVars;
     private List<LogicalVariable> maxFilterVars;
 
+    // the select condition in the SELECT operator. Only results satisfying this selectCondition
+    // would be returned by this operator
+    private Mutable<ILogicalExpression> selectCondition;
+    // the maximum of number of results output by this operator
+    private long outputLimit = -1;
+
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
+        this(variables, dataSource, null, -1);
+    }
+
+    public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
+            Mutable<ILogicalExpression> selectCondition, long outputLimit) {
         super(variables, dataSource);
         projectVars = new ArrayList<LogicalVariable>();
+        this.selectCondition = selectCondition;
+        this.outputLimit = outputLimit;
     }
 
     @Override
@@ -133,4 +146,20 @@
     public List<Mutable<ILogicalExpression>> getAdditionalFilteringExpressions() {
         return additionalFilteringExpressions;
     }
+
+    public Mutable<ILogicalExpression> getSelectCondition() {
+        return selectCondition;
+    }
+
+    public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+        this.selectCondition = selectCondition;
+    }
+
+    public long getOutputLimit() {
+        return outputLimit;
+    }
+
+    public void setOutputLimit(long outputLimit) {
+        this.outputLimit = outputLimit;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
index 31a1294..154fb13 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
@@ -49,6 +49,7 @@
     // used for upsert operations
     private List<Mutable<ILogicalExpression>> prevSecondaryKeyExprs;
     private Mutable<ILogicalExpression> prevAdditionalFilteringExpression;
+    private Mutable<ILogicalExpression> upsertIndicatorExpr;
     private final int numberOfAdditionalNonFilteringFields;
 
     public IndexInsertDeleteUpsertOperator(IDataSourceIndex<?, ?> dataSourceIndex,
@@ -93,6 +94,12 @@
                 }
             }
         }
+
+        // Upsert indicator var <For upsert>
+        if (upsertIndicatorExpr != null && visitor.transform(upsertIndicatorExpr)) {
+            b = true;
+        }
+
         // Old secondary <For upsert>
         if (prevSecondaryKeyExprs != null) {
             for (int i = 0; i < prevSecondaryKeyExprs.size(); i++) {
@@ -189,4 +196,12 @@
     public int getNumberOfAdditionalNonFilteringFields() {
         return numberOfAdditionalNonFilteringFields;
     }
+
+    public Mutable<ILogicalExpression> getUpsertIndicatorExpr() {
+        return upsertIndicatorExpr;
+    }
+
+    public void setUpsertIndicatorExpr(Mutable<ILogicalExpression> upsertIndicatorExpr) {
+        this.upsertIndicatorExpr = upsertIndicatorExpr;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
index 9838c12..ae90462 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
@@ -59,6 +59,9 @@
     // previous additional fields (for UPSERT)
     private List<LogicalVariable> prevAdditionalNonFilteringVars;
     private List<Object> prevAdditionalNonFilteringTypes;
+    // a boolean variable that indicates whether it's a delete operation (false) or upsert operation (true)
+    private LogicalVariable upsertIndicatorVar;
+    private Object upsertIndicatorVarType;
 
     public InsertDeleteUpsertOperator(IDataSource<?> dataSource, Mutable<ILogicalExpression> payloadExpr,
             List<Mutable<ILogicalExpression>> primaryKeyExprs,
@@ -85,6 +88,7 @@
     public void recomputeSchema() throws AlgebricksException {
         schema = new ArrayList<LogicalVariable>();
         if (operation == Kind.UPSERT) {
+            schema.add(upsertIndicatorVar);
             // The upsert case also produces the previous record
             schema.add(prevRecordVar);
             if (additionalNonFilteringExpressions != null) {
@@ -98,6 +102,9 @@
     }
 
     public void getProducedVariables(Collection<LogicalVariable> producedVariables) {
+        if (upsertIndicatorVar != null) {
+            producedVariables.add(upsertIndicatorVar);
+        }
         if (prevRecordVar != null) {
             producedVariables.add(prevRecordVar);
         }
@@ -147,6 +154,7 @@
             public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
                     throws AlgebricksException {
                 if (operation == Kind.UPSERT) {
+                    target.addVariable(upsertIndicatorVar);
                     target.addVariable(prevRecordVar);
                     if (prevAdditionalNonFilteringVars != null) {
                         for (LogicalVariable var : prevAdditionalNonFilteringVars) {
@@ -171,6 +179,7 @@
     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
         PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
         if (operation == Kind.UPSERT) {
+            env.setVarType(upsertIndicatorVar, upsertIndicatorVarType);
             env.setVarType(prevRecordVar, prevRecordType);
             if (prevAdditionalNonFilteringVars != null) {
                 for (int i = 0; i < prevAdditionalNonFilteringVars.size(); i++) {
@@ -224,6 +233,22 @@
         this.prevRecordVar = prevRecordVar;
     }
 
+    public LogicalVariable getUpsertIndicatorVar() {
+        return upsertIndicatorVar;
+    }
+
+    public void setUpsertIndicatorVar(LogicalVariable upsertIndicatorVar) {
+        this.upsertIndicatorVar = upsertIndicatorVar;
+    }
+
+    public Object getUpsertIndicatorVarType() {
+        return upsertIndicatorVarType;
+    }
+
+    public void setUpsertIndicatorVarType(Object upsertIndicatorVarType) {
+        this.upsertIndicatorVarType = upsertIndicatorVarType;
+    }
+
     public void setPrevRecordType(Object recordType) {
         prevRecordType = recordType;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index e7fb6c0..c4bcc52 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -32,9 +32,23 @@
 
 public class UnnestMapOperator extends AbstractUnnestMapOperator {
 
+    // the select condition in the SELECT operator. Only results satisfying this selectCondition
+    // would be returned by this operator
+    private Mutable<ILogicalExpression> selectCondition;
+    // the maximum of number of results output by this operator
+    private long outputLimit = -1;
+
     public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
+        this(variables, expression, variableTypes, propagateInput, null, -1);
+    }
+
+    public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
+            List<Object> variableTypes, boolean propagateInput, Mutable<ILogicalExpression> selectCondition,
+            long outputLimit) {
         super(variables, expression, variableTypes, propagateInput);
+        this.selectCondition = selectCondition;
+        this.outputLimit = outputLimit;
     }
 
     @Override
@@ -64,4 +78,20 @@
         return env;
     }
 
+    public Mutable<ILogicalExpression> getSelectCondition() {
+        return selectCondition;
+    }
+
+    public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+        this.selectCondition = selectCondition;
+    }
+
+    public long getOutputLimit() {
+        return outputLimit;
+    }
+
+    public void setOutputLimit(long outputLimit) {
+        this.outputLimit = outputLimit;
+    }
+
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 2d5780d..d0d121f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -313,12 +313,10 @@
                 newGbyList.add(p);
             }
         }
-        if (changed) {
-            if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-                AlgebricksConfig.ALGEBRICKS_LOGGER
-                        .debug(">>>> Group-by list changed from " + GroupByOperator.veListToString(gByList) + " to "
-                                + GroupByOperator.veListToString(newGbyList) + ".\n");
-            }
+        if (changed && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .trace(">>>> Group-by list changed from " + GroupByOperator.veListToString(gByList) + " to "
+                            + GroupByOperator.veListToString(newGbyList) + ".\n");
         }
         gByList.clear();
         gByList.addAll(newGbyList);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index fb1bcec..90c0067 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -504,7 +504,7 @@
                 columnsArg.set(i, leftVar);
             }
         }
-        return VariableUtilities.varListEqualUnordered(columns, columnsArg);
+        return columns.equals(columnsArg);
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
index 24c5162..6be8ee0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IVariableContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
@@ -80,6 +81,10 @@
         dest.setOpaqueParameters(newOpaqueParameters);
     }
 
+    private void copySourceLocation(ILogicalExpression src, AbstractLogicalExpression dest) {
+        dest.setSourceLocation(src.getSourceLocation());
+    }
+
     public MutableObject<ILogicalExpression> deepCopyExpressionReference(Mutable<ILogicalExpression> exprRef)
             throws AlgebricksException {
         return new MutableObject<>(deepCopy(exprRef.getValue()));
@@ -104,12 +109,15 @@
         exprCopy.setStepTwoAggregate(expr.getStepTwoAggregate());
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
     @Override
     public ILogicalExpression visitConstantExpression(ConstantExpression expr, Void arg) throws AlgebricksException {
-        return new ConstantExpression(expr.getValue());
+        ConstantExpression exprCopy = new ConstantExpression(expr.getValue());
+        copySourceLocation(expr, exprCopy);
+        return exprCopy;
     }
 
     @Override
@@ -119,6 +127,7 @@
                 deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
 
     }
@@ -130,6 +139,7 @@
                 expr.getPropertiesComputer(), deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
@@ -140,6 +150,7 @@
                 deepCopyExpressionReferenceList(expr.getArguments()));
         deepCopyAnnotations(expr, exprCopy);
         deepCopyOpaqueParameters(expr, exprCopy);
+        copySourceLocation(expr, exprCopy);
         return exprCopy;
     }
 
@@ -153,13 +164,17 @@
         LogicalVariable givenVarReplacement = inVarMapping.get(var);
         if (givenVarReplacement != null) {
             outVarMapping.put(var, givenVarReplacement);
-            return new VariableReferenceExpression(givenVarReplacement);
+            VariableReferenceExpression varRef = new VariableReferenceExpression(givenVarReplacement);
+            copySourceLocation(expr, varRef);
+            return varRef;
         }
         LogicalVariable varCopy = outVarMapping.get(var);
         if (varCopy == null) {
             varCopy = varContext.newVar();
             outVarMapping.put(var, varCopy);
         }
-        return new VariableReferenceExpression(varCopy);
+        VariableReferenceExpression varRef = new VariableReferenceExpression(varCopy);
+        copySourceLocation(expr, varRef);
+        return varRef;
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 74afdf5..e0210cc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -146,6 +146,10 @@
         dest.getAnnotations().putAll(src.getAnnotations());
     }
 
+    private void copySourceLocation(ILogicalOperator src, AbstractLogicalOperator dest) {
+        dest.setSourceLocation(src.getSourceLocation());
+    }
+
     public ILogicalOperator deepCopy(ILogicalOperator op) throws AlgebricksException {
         // The deep copy call outside this visitor always has a null argument.
         return deepCopy(op, null);
@@ -269,6 +273,7 @@
             AbstractLogicalOperator opCopy) throws AlgebricksException {
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
     }
 
@@ -317,8 +322,10 @@
     @Override
     public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg)
             throws AlgebricksException {
-        DataSourceScanOperator opCopy =
-                new DataSourceScanOperator(deepCopyVariableList(op.getVariables()), op.getDataSource());
+        Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+                ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
+        DataSourceScanOperator opCopy = new DataSourceScanOperator(deepCopyVariableList(op.getVariables()),
+                op.getDataSource(), newSelectCondition, op.getOutputLimit());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
@@ -335,6 +342,7 @@
     @Override
     public ILogicalOperator visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg) {
         EmptyTupleSourceOperator opCopy = new EmptyTupleSourceOperator();
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -369,6 +377,7 @@
                         deepCopyOperatorReference(op.getInputs().get(0), arg),
                         deepCopyOperatorReference(op.getInputs().get(1), arg));
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -381,6 +390,7 @@
                         deepCopyOperatorReference(op.getInputs().get(0), arg),
                         deepCopyOperatorReference(op.getInputs().get(1), arg));
         copyAnnotations(op, opCopy);
+        copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
@@ -535,9 +545,11 @@
     @Override
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg)
             throws AlgebricksException {
+        Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+                ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
         UnnestMapOperator opCopy = new UnnestMapOperator(deepCopyVariableList(op.getVariables()),
                 exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
-                op.propagatesInput());
+                op.propagatesInput(), newSelectCondition, op.getOutputLimit());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index ae93386..6dfe254 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -80,7 +80,7 @@
             computeLogicalPropertiesRec(ref.getValue(), visitor, context);
         }
         op.accept(visitor, context);
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace(
                     "Logical properties visitor for " + op + ": " + context.getLogicalPropertiesVector(op) + "\n");
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 600714b..0db0f74 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -35,11 +35,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -225,8 +225,10 @@
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
         ArrayList<LogicalVariable> newInputList = new ArrayList<>();
         newInputList.addAll(op.getVariables());
+        Mutable<ILogicalExpression> newSelectCondition =
+                op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
         return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
-                new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+                new ArrayList<>(op.getVariableTypes()), op.propagatesInput(), newSelectCondition, op.getOutputLimit());
     }
 
     @Override
@@ -242,7 +244,11 @@
     public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
         ArrayList<LogicalVariable> newInputList = new ArrayList<>();
         newInputList.addAll(op.getVariables());
-        return new DataSourceScanOperator(newInputList, op.getDataSource());
+        Mutable<ILogicalExpression> newSelectCondition =
+                op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
+        DataSourceScanOperator newOp =
+                new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit());
+        return newOp;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index cf24ee7..3587e29 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -34,11 +34,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -134,6 +134,9 @@
             }
         }
         substVarTypes(op, pair);
+        if (op.getSelectCondition() != null) {
+            op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
+        }
         return null;
     }
 
@@ -316,6 +319,9 @@
     public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         substituteVarsForAbstractUnnestMapOp(op, pair);
+        if (op.getSelectCondition() != null) {
+            op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index b8cb4ff..e66809e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -19,7 +19,9 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
 
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -34,11 +36,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -76,7 +78,7 @@
 
 public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
 
-    private Collection<LogicalVariable> usedVariables;
+    private final Collection<LogicalVariable> usedVariables;
 
     public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
         this.usedVariables = usedVariables;
@@ -105,6 +107,12 @@
                 e.getValue().getUsedVariables(usedVariables);
             }
         }
+        if (op.getSelectCondition() != null) {
+            Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+            op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+            usedVariablesBySelect.removeAll(op.getVariables());
+            usedVariables.addAll(usedVariablesBySelect);
+        }
         return null;
     }
 
@@ -305,6 +313,12 @@
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {
         getUsedVarsFromExprAndFilterExpr(op);
+        if (op.getSelectCondition() != null) {
+            Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+            op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+            usedVariablesBySelect.removeAll(op.getVariables());
+            usedVariables.addAll(usedVariablesBySelect);
+        }
         return null;
     }
 
@@ -407,6 +421,9 @@
                 e.getValue().getUsedVariables(usedVariables);
             }
         }
+        if (op.getUpsertIndicatorExpr() != null) {
+            op.getUpsertIndicatorExpr().getValue().getUsedVariables(usedVariables);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java
index 64e50ed..c18d76c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPreclusteredGroupByPOperator.java
@@ -89,6 +89,10 @@
         IPartitioningProperty pp = childProp.getPartitioningProperty();
         Map<LogicalVariable, LogicalVariable> ppSubstMap = computePartitioningPropertySubstitutionMap(gby, pp);
         if (ppSubstMap != null) {
+            // We cannot modify pp directly, since it is owned by the input operator.
+            // Otherwise, the partitioning property would be modified even before this group by operator,
+            // which will be undesirable.
+            pp = pp.clonePartitioningProperty();
             pp.substituteColumnVars(ppSubstMap);
         }
         List<ILocalStructuralProperty> childLocals = childProp.getLocalProperties();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 4a177f7..0602258 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -81,6 +81,7 @@
         int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
         UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
                 unnest.getPositionWriter(), leftOuter, context.getMissingWriterFactory());
+        unnestRuntime.setSourceLocation(unnest.getSourceLocation());
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
         ILogicalOperator src = unnest.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
index 147d5cc..2736b6f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
@@ -102,6 +102,7 @@
         }
 
         AggregateRuntimeFactory runtime = new AggregateRuntimeFactory(aggFactories);
+        runtime.setSourceLocation(aggOp.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
index ccd27f4..2df5a4e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
@@ -90,6 +90,7 @@
 
         AssignRuntimeFactory runtime =
                 new AssignRuntimeFactory(outColumns, evalFactories, projectionList, flushFramesRapidly);
+        runtime.setSourceLocation(assign.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
@@ -103,7 +104,6 @@
         // and contribute one edge from its child
         ILogicalOperator src = assign.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, assign, 0);
-
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
index 2204637..bb0f08b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
@@ -107,8 +107,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, primaryKeys, payload,
                         additionalFilteringKeys, additionalNonFilterVars, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 1421cef..3ddb233 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -43,11 +43,12 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 @SuppressWarnings("rawtypes")
 public class DataSourceScanPOperator extends AbstractScanPOperator {
 
-    private IDataSource<?> dataSource;
+    private final IDataSource<?> dataSource;
     private Object implConfig;
 
     public DataSourceScanPOperator(IDataSource<?> dataSource) {
@@ -109,12 +110,20 @@
         List<LogicalVariable> vars = scan.getVariables();
         List<LogicalVariable> projectVars = scan.getProjectVariables();
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p =
-                mp.getScannerRuntime(dataSource, vars, projectVars, scan.isProjectPushed(), scan.getMinFilterVars(),
-                        scan.getMaxFilterVars(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
-        builder.contributeHyracksOperator(scan, p.first);
+        ITupleFilterFactory tupleFilterFactory = null;
+        if (scan.getSelectCondition() != null) {
+            tupleFilterFactory = context.getMetadataProvider().createTupleFilterFactory(
+                    new IOperatorSchema[] { opSchema }, typeEnv, scan.getSelectCondition().getValue(), context);
+        }
+
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars, projectVars,
+                scan.isProjectPushed(), scan.getMinFilterVars(), scan.getMaxFilterVars(), tupleFilterFactory,
+                scan.getOutputLimit(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
+        IOperatorDescriptor opDesc = p.first;
+        opDesc.setSourceLocation(scan.getSourceLocation());
+        builder.contributeHyracksOperator(scan, opDesc);
         if (p.second != null) {
-            builder.contributeAlgebricksPartitionConstraint(p.first, p.second);
+            builder.contributeAlgebricksPartitionConstraint(opDesc, p.second);
         }
 
         ILogicalOperator srcExchange = scan.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
index 178f2a1..34dc0d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -105,8 +105,10 @@
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getResultHandleRuntime(resultOp.getDataSink(), columns, pf, inputDesc, true, spec);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(resultOp.getSourceLocation());
+        builder.contributeHyracksOperator(resultOp, opDesc);
 
-        builder.contributeHyracksOperator(resultOp, runtimeAndConstraints.first);
         ILogicalOperator src = resultOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, resultOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
index 0718d13..502d022 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/EmptyTupleSourcePOperator.java
@@ -63,6 +63,7 @@
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         EmptyTupleSourceRuntimeFactory runtime = new EmptyTupleSourceRuntimeFactory();
+        runtime.setSourceLocation(op.getSourceLocation());
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {});
         builder.contributeMicroOperator(op, runtime, recDesc);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
index 5ee967d..652196d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -62,8 +62,8 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 
 public class ExternalGroupByPOperator extends AbstractPhysicalOperator {
@@ -253,8 +253,10 @@
             merges[i] = expressionRuntimeProvider.createSerializableAggregateFunctionFactory(mergeFun, aggOpInputEnv,
                     localInputSchemas, context);
         }
-        IAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
-        IAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
+        AbstractAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
+        AbstractAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
+        mergeFactory.setSourceLocation(gby.getSourceLocation());
 
         INormalizedKeyComputerFactory normalizedKeyFactory =
                 JobGenHelper.variablesToAscNormalizedKeyComputerFactory(gbyCols, aggOpInputEnv, context);
@@ -268,6 +270,7 @@
         ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, hashTableSize, inputSize,
                 keyAndDecFields, frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
                 recordDescriptor, recordDescriptor, new HashSpillableTableFactory(hashFunctionFactories));
+        gbyOpDesc.setSourceLocation(gby.getSourceLocation());
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index 301b8f1..45ec44b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -78,12 +78,13 @@
         this.maxInputBuildSizeInFrames = maxInputSizeInFrames;
         this.aveRecordsPerFrame = aveRecordsPerFrame;
         this.fudgeFactor = fudgeFactor;
-
-        LOGGER.debug("HybridHashJoinPOperator constructed with: JoinKind=" + kind + ", JoinPartitioningType="
-                + partitioningType + ", List<LogicalVariable>=" + sideLeftOfEqualities + ", List<LogicalVariable>="
-                + sideRightOfEqualities + ", int memSizeInFrames=" + memSizeInFrames + ", int maxInputSize0InFrames="
-                + maxInputSizeInFrames + ", int aveRecordsPerFrame=" + aveRecordsPerFrame + ", double fudgeFactor="
-                + fudgeFactor + ".");
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("HybridHashJoinPOperator constructed with: JoinKind=" + kind + ", JoinPartitioningType="
+                    + partitioningType + ", List<LogicalVariable>=" + sideLeftOfEqualities + ", List<LogicalVariable>="
+                    + sideRightOfEqualities + ", int memSizeInFrames=" + memSizeInFrames
+                    + ", int maxInputSize0InFrames=" + maxInputSizeInFrames + ", int aveRecordsPerFrame="
+                    + aveRecordsPerFrame + ", double fudgeFactor=" + fudgeFactor + ".");
+        }
     }
 
     @Override
@@ -152,6 +153,7 @@
             opDesc = generateHashJoinRuntime(context, inputSchemas, keysLeft, keysRight, hashFunFactories,
                     comparatorFactories, predEvaluatorFactory, recDescriptor, spec);
         }
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
@@ -164,68 +166,49 @@
             int[] keysLeft, int[] keysRight, IBinaryHashFunctionFactory[] hashFunFactories,
             IBinaryComparatorFactory[] comparatorFactories, IPredicateEvaluatorFactory predEvaluatorFactory,
             RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) throws AlgebricksException {
-        IOperatorDescriptor opDesc;
-        try {
-            switch (kind) {
-                case INNER:
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
-                            aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
-                            comparatorFactories, recDescriptor, predEvaluatorFactory, false, null);
-                    break;
-                case LEFT_OUTER:
-                    IMissingWriterFactory[] nonMatchWriterFactories =
-                            new IMissingWriterFactory[inputSchemas[1].getSize()];
-                    for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                        nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                    }
-                    opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
-                            aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
-                            comparatorFactories, recDescriptor, predEvaluatorFactory, true, nonMatchWriterFactories);
-                    break;
-                default:
-                    throw new NotImplementedException();
-            }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        switch (kind) {
+            case INNER:
+                return new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
+                        aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
+                        comparatorFactories, recDescriptor, predEvaluatorFactory, false, null);
+            case LEFT_OUTER:
+                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
+                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
+                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
+                }
+                return new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(), maxInputBuildSizeInFrames,
+                        aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight, hashFunFactories,
+                        comparatorFactories, recDescriptor, predEvaluatorFactory, true, nonMatchWriterFactories);
+            default:
+                throw new NotImplementedException();
         }
-        return opDesc;
     }
 
     private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context, IOperatorSchema[] inputSchemas,
             int[] keysLeft, int[] keysRight, IBinaryHashFunctionFamily[] hashFunFamilies,
             IBinaryComparatorFactory[] comparatorFactories, IPredicateEvaluatorFactory predEvaluatorFactory,
             RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) throws AlgebricksException {
-        IOperatorDescriptor opDesc;
-        try {
-            switch (kind) {
-                case INNER:
-                    opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
-                            comparatorFactories, recDescriptor,
-                            new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
-                            new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft),
-                            predEvaluatorFactory);
-                    break;
-                case LEFT_OUTER:
-                    IMissingWriterFactory[] nonMatchWriterFactories =
-                            new IMissingWriterFactory[inputSchemas[1].getSize()];
-                    for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                        nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                    }
-                    opDesc = new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
-                            maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
-                            comparatorFactories, recDescriptor,
-                            new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
-                            new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft),
-                            predEvaluatorFactory, true, nonMatchWriterFactories);
-                    break;
-                default:
-                    throw new NotImplementedException();
-            }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        switch (kind) {
+            case INNER:
+                return new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                        maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                        comparatorFactories, recDescriptor,
+                        new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
+                        new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft), predEvaluatorFactory);
+            case LEFT_OUTER:
+                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
+                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
+                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
+                }
+                return new OptimizedHybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
+                        maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
+                        comparatorFactories, recDescriptor,
+                        new JoinMultiComparatorFactory(comparatorFactories, keysLeft, keysRight),
+                        new JoinMultiComparatorFactory(comparatorFactories, keysRight, keysLeft), predEvaluatorFactory,
+                        true, nonMatchWriterFactories);
+            default:
+                throw new NotImplementedException();
         }
-        return opDesc;
     }
 
     @Override
@@ -264,7 +247,7 @@
 }
 
 /**
- * {@ ITuplePairComparatorFactory} implementation for optimized hybrid hash join.
+ * {@code ITuplePairComparatorFactory} implementation for optimized hybrid hash join.
  */
 class JoinMultiComparatorFactory implements ITuplePairComparatorFactory {
     private static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 9c29c53..580d8e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -105,15 +105,14 @@
         RecordDescriptor recDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
-        IOperatorDescriptor opDesc = null;
+        IOperatorDescriptor opDesc;
 
         switch (kind) {
-            case INNER: {
+            case INNER:
                 opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
                         comparatorFactories, recDescriptor, tableSize, predEvaluatorFactory, memSizeInFrames);
                 break;
-            }
-            case LEFT_OUTER: {
+            case LEFT_OUTER:
                 IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
                 for (int j = 0; j < nonMatchWriterFactories.length; j++) {
                     nonMatchWriterFactories[j] = context.getMissingWriterFactory();
@@ -122,11 +121,11 @@
                         comparatorFactories, predEvaluatorFactory, recDescriptor, true, nonMatchWriterFactories,
                         tableSize, memSizeInFrames);
                 break;
-            }
-            default: {
+            default:
                 throw new NotImplementedException();
-            }
         }
+
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index fa0fb1a..6512700 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -135,8 +135,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
                         secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(indexInsertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(indexInsertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(indexInsertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = indexInsertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, indexInsertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index a66db35..228ca52 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -54,14 +54,16 @@
     private final ILogicalExpression filterExpr;
     private final IDataSourceIndex<?, ?> dataSourceIndex;
     private final List<LogicalVariable> additionalFilteringKeys;
+    private final LogicalVariable upsertIndicatorVar;
     private final List<LogicalVariable> prevSecondaryKeys;
     private final LogicalVariable prevAdditionalFilteringKey;
     private final int numOfAdditionalNonFilteringFields;
 
     public IndexInsertDeleteUpsertPOperator(List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
             List<LogicalVariable> additionalFilteringKeys, Mutable<ILogicalExpression> filterExpr,
-            IDataSourceIndex<?, ?> dataSourceIndex, List<LogicalVariable> prevSecondaryKeys,
-            LogicalVariable prevAdditionalFilteringKey, int numOfAdditionalNonFilteringFields) {
+            IDataSourceIndex<?, ?> dataSourceIndex, LogicalVariable upsertIndicatorVar,
+            List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKey,
+            int numOfAdditionalNonFilteringFields) {
         this.primaryKeys = primaryKeys;
         this.secondaryKeys = secondaryKeys;
         if (filterExpr != null) {
@@ -71,6 +73,7 @@
         }
         this.dataSourceIndex = dataSourceIndex;
         this.additionalFilteringKeys = additionalFilteringKeys;
+        this.upsertIndicatorVar = upsertIndicatorVar;
         this.prevSecondaryKeys = prevSecondaryKeys;
         this.prevAdditionalFilteringKey = prevAdditionalFilteringKey;
         this.numOfAdditionalNonFilteringFields = numOfAdditionalNonFilteringFields;
@@ -118,19 +121,30 @@
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(insertDeleteUpsertOp);
-        if (insertDeleteUpsertOp.getOperation() == Kind.INSERT) {
-            runtimeAndConstraints = mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
-        } else if (insertDeleteUpsertOp.getOperation() == Kind.DELETE) {
-            runtimeAndConstraints = mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
-        } else if (insertDeleteUpsertOp.getOperation() == Kind.UPSERT) {
-            runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
-                    primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, prevSecondaryKeys,
-                    prevAdditionalFilteringKey, inputDesc, context, spec);
+        Kind operation = insertDeleteUpsertOp.getOperation();
+        switch (operation) {
+            case INSERT:
+                runtimeAndConstraints =
+                        mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
+                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
+                break;
+            case DELETE:
+                runtimeAndConstraints =
+                        mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
+                                secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
+                break;
+            case UPSERT:
+                runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+                        typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, upsertIndicatorVar,
+                        prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec);
+                break;
+            default:
+                throw new AlgebricksException("Unsupported Operation " + operation);
         }
-        builder.contributeHyracksOperator(insertDeleteUpsertOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteUpsertOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteUpsertOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteUpsertOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteUpsertOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
index 6ded4a3..927fb66 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
@@ -106,22 +106,28 @@
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
-        if (operation == Kind.INSERT) {
-            runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
-                    additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec, false);
-        } else if (operation == Kind.DELETE) {
-            runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
-                    additionalFilteringKeys, inputDesc, context, spec);
-        } else if (operation == Kind.UPSERT) {
-            runtimeAndConstraints = mp.getUpsertRuntime(dataSource, inputSchemas[0], typeEnv, keys, payload,
-                    additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec);
-        } else {
-            throw new AlgebricksException("Unsupported Operation " + operation);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints;
+        switch (operation) {
+            case INSERT:
+                runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec, false);
+                break;
+            case DELETE:
+                runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, inputDesc, context, spec);
+                break;
+            case UPSERT:
+                runtimeAndConstraints = mp.getUpsertRuntime(dataSource, inputSchemas[0], typeEnv, keys, payload,
+                        additionalFilteringKeys, additionalNonFilteringFields, inputDesc, context, spec);
+                break;
+            default:
+                throw new AlgebricksException("Unsupported Operation " + operation);
         }
 
-        builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(insertDeleteOp.getSourceLocation());
+        builder.contributeHyracksOperator(insertDeleteOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
index 1d36cc0..544c546 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IntersectPOperator.java
@@ -151,7 +151,7 @@
         } catch (HyracksException e) {
             throw new AlgebricksException(e);
         }
-
+        opDescriptor.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDescriptor);
         for (int i = 0; i < op.getInputs().size(); i++) {
             builder.contributeGraphEdge(op.getInputs().get(i).getValue(), 0, op, i);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
index a48e3c2..fe8985a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MaterializePOperator.java
@@ -74,6 +74,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         MaterializingOperatorDescriptor materializationOpDesc =
                 new MaterializingOperatorDescriptor(builder.getJobSpec(), recDescriptor, isSingleActivity);
+        materializationOpDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
index 94d5fd4..fda879c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreSortedDistinctByPOperator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.algebricks.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 
 public class MicroPreSortedDistinctByPOperator extends AbstractPreSortedDistinctByPOperator {
 
@@ -62,8 +62,9 @@
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
                 .variablesToAscBinaryComparatorFactories(columnList, context.getTypeEnvironment(op), context);
         IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, keysAndDecs);
+        aggregatorFactory.setSourceLocation(op.getSourceLocation());
 
         RecordDescriptor recordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
@@ -73,6 +74,7 @@
         /* make fd columns part of the key but the comparator only compares the distinct key columns */
         MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keysAndDecs,
                 comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recordDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
index 629afa3..13308a1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
@@ -74,6 +74,7 @@
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
         MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keys,
                 comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
+        runtime.setSourceLocation(gby.getSourceLocation());
         builder.contributeMicroOperator(gby, runtime, recordDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
index f5e992e..da48049 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroUnionAllPOperator.java
@@ -49,6 +49,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
 
         MicroUnionAllRuntimeFactory runtime = new MicroUnionAllRuntimeFactory(op.getInputs().size());
+        runtime.setSourceLocation(op.getSourceLocation());
         builder.contributeMicroOperator(op, runtime, recordDescriptor);
 
         super.contributeRuntimeOperator(builder, context, op, opSchema, inputSchemas, outerPlanSchema);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 4d7bd7e1..83cea4a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -139,15 +139,14 @@
         ITuplePairComparatorFactory comparatorFactory =
                 new TuplePairEvaluatorFactory(cond, context.getBinaryBooleanInspectorFactory());
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
-        IOperatorDescriptor opDesc = null;
+        IOperatorDescriptor opDesc;
 
         switch (kind) {
-            case INNER: {
+            case INNER:
                 opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, false,
                         null);
                 break;
-            }
-            case LEFT_OUTER: {
+            case LEFT_OUTER:
                 IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
                 for (int j = 0; j < nonMatchWriterFactories.length; j++) {
                     nonMatchWriterFactories[j] = context.getMissingWriterFactory();
@@ -155,12 +154,12 @@
                 opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
                         nonMatchWriterFactories);
                 break;
-            }
-            default: {
+            default:
                 throw new NotImplementedException();
-            }
         }
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+        opDesc.setSourceLocation(join.getSourceLocation());
+        contributeOpDesc(builder, join, opDesc);
 
         ILogicalOperator src1 = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src1, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
index 179bb73..7971e78 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
@@ -98,6 +98,7 @@
             throws AlgebricksException {
         propagatedSchema.addAllVariables(outerPlanSchema);
         NestedTupleSourceRuntimeFactory runtime = new NestedTupleSourceRuntimeFactory();
+        runtime.setSourceLocation(op.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(op, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
index 54e577f..dd4c65f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 
 public class PreSortedDistinctByPOperator extends AbstractPreSortedDistinctByPOperator {
@@ -64,14 +64,16 @@
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
                 .variablesToAscBinaryComparatorFactories(columnList, context.getTypeEnvironment(op), context);
         IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, keysAndDecs);
+        aggregatorFactory.setSourceLocation(op.getSourceLocation());
 
         RecordDescriptor recordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         /* make fd columns part of the key but the comparator only compares the distinct key columns */
         PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keysAndDecs,
                 comparatorFactories, aggregatorFactory, recordDescriptor);
+        opDesc.setSourceLocation(op.getSourceLocation());
 
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
index 0e0953c..e5076ce 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -38,7 +37,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 
 public class PreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
@@ -71,7 +70,7 @@
         int fdColumns[] = getFdColumns(gby, inputSchemas[0]);
         // compile subplans and set the gby op. schema accordingly
         AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
-        IAggregatorDescriptorFactory aggregatorFactory;
+        AbstractAggregatorDescriptorFactory aggregatorFactory;
 
         List<ILogicalPlan> nestedPlans = gby.getNestedPlans();
         if (!nestedPlans.isEmpty() && nestedPlans.get(0).getRoots().get(0).getValue()
@@ -80,6 +79,7 @@
         } else {
             aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys, fdColumns);
         }
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
 
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper
@@ -89,8 +89,9 @@
 
         PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
                 comparatorFactories, aggregatorFactory, recordDescriptor, groupAll, framesLimit);
+        opDesc.setSourceLocation(gby.getSourceLocation());
 
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+        contributeOpDesc(builder, gby, opDesc);
 
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
index 25d31d2..f34cecc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
@@ -22,7 +22,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -50,9 +49,10 @@
         int outputArity = rop.getOutputArity();
         boolean[] outputMaterializationFlags = rop.getOutputMaterializationFlags();
 
-        ReplicateOperatorDescriptor splitOpDesc =
+        ReplicateOperatorDescriptor ropDesc =
                 new ReplicateOperatorDescriptor(spec, recDescriptor, outputArity, outputMaterializationFlags);
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, splitOpDesc);
+        ropDesc.setSourceLocation(rop.getSourceLocation());
+        contributeOpDesc(builder, rop, ropDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index 3a6ba74..d68be20 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -91,6 +91,7 @@
 
         RunningAggregateRuntimeFactory runtime =
                 new RunningAggregateRuntimeFactory(outColumns, runningAggFuns, projectionList);
+        runtime.setSourceLocation(ragg.getSourceLocation());
 
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
index 5084c18..6c9f6e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
@@ -79,6 +79,7 @@
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
 
         SinkOperatorDescriptor opDesc = new SinkOperatorDescriptor(spec, op.getInputs().size());
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         for (int i = 0; i < op.getInputs().size(); i++) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index f76b69b..6e37b3a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -101,10 +101,12 @@
 
         IMetadataProvider<?, ?> mp = context.getMetadataProvider();
 
-        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtime =
+        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+        IPushRuntimeFactory runtime = runtimeAndConstraints.first;
+        runtime.setSourceLocation(write.getSourceLocation());
 
-        builder.contributeMicroOperator(write, runtime.first, recDesc, runtime.second);
+        builder.contributeMicroOperator(write, runtime, recDesc, runtimeAndConstraints.second);
         ILogicalOperator src = write.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, write, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
index 1aeeca9..967e7e6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
@@ -63,7 +63,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
 
 public class SortGroupByPOperator extends AbstractPhysicalOperator {
@@ -258,10 +258,12 @@
         RecordDescriptor partialAggRecordDescriptor =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), localInputSchemas[0], context);
 
-        IAggregatorDescriptorFactory aggregatorFactory =
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(aff, keyAndDecFields);
-        IAggregatorDescriptorFactory mergeFactory =
+        aggregatorFactory.setSourceLocation(gby.getSourceLocation());
+        AbstractAggregatorDescriptorFactory mergeFactory =
                 new SimpleAlgebricksAccumulatingAggregatorFactory(merges, keyAndDecFields);
+        mergeFactory.setSourceLocation(gby.getSourceLocation());
 
         INormalizedKeyComputerFactory normalizedKeyFactory = null;
         INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
@@ -275,6 +277,7 @@
         SortGroupByOperatorDescriptor gbyOpDesc = new SortGroupByOperatorDescriptor(spec, frameLimit, keys,
                 keyAndDecFields, normalizedKeyFactory, compFactories, aggregatorFactory, mergeFactory,
                 partialAggRecordDescriptor, recordDescriptor, false);
+        gbyOpDesc.setSourceLocation(gby.getSourceLocation());
 
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
index c9fde4b..1a1eff6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SplitPOperator.java
@@ -23,7 +23,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -60,10 +59,11 @@
 
         IBinaryIntegerInspectorFactory intInsepctorFactory = context.getBinaryIntegerInspectorFactory();
 
-        SplitOperatorDescriptor splitOpDesc = new SplitOperatorDescriptor(spec, recDescriptor, outputArity,
+        SplitOperatorDescriptor sopDesc = new SplitOperatorDescriptor(spec, recDescriptor, outputArity,
                 brachingExprEvalFactory, intInsepctorFactory, defaultBranch, propageToAllBranchAsDefault);
+        sopDesc.setSourceLocation(sop.getSourceLocation());
 
-        contributeOpDesc(builder, (AbstractLogicalOperator) op, splitOpDesc);
+        contributeOpDesc(builder, sop, sopDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
index 3a4249b..269a809 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
 
@@ -98,22 +99,21 @@
             i++;
         }
 
+        AbstractSorterOperatorDescriptor sortOpDesc;
         // topK == -1 means that a topK value is not provided.
         if (topK == -1) {
-            ExternalSortOperatorDescriptor sortOpDesc =
+            sortOpDesc =
                     new ExternalSortOperatorDescriptor(spec, maxNumberOfFrames, sortFields, nkcf, comps, recDescriptor);
-            contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
-            ILogicalOperator src = op.getInputs().get(0).getValue();
-            builder.contributeGraphEdge(src, 0, op, 0);
         } else {
             // Since topK value is provided, topK optimization is possible.
             // We call topKSorter instead of calling ExternalSortOperator.
-            TopKSorterOperatorDescriptor sortOpDesc = new TopKSorterOperatorDescriptor(spec, maxNumberOfFrames, topK,
-                    sortFields, nkcf, comps, recDescriptor);
-            contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
-            ILogicalOperator src = op.getInputs().get(0).getValue();
-            builder.contributeGraphEdge(src, 0, op, 0);
+            sortOpDesc = new TopKSorterOperatorDescriptor(spec, maxNumberOfFrames, topK, sortFields, nkcf, comps,
+                    recDescriptor);
         }
+        sortOpDesc.setSourceLocation(op.getSourceLocation());
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, sortOpDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
index da75da8..90732ce 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
@@ -98,6 +98,7 @@
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         StreamLimitRuntimeFactory runtime = new StreamLimitRuntimeFactory(maxObjectsFact, offsetFact, null,
                 context.getBinaryIntegerInspectorFactory());
+        runtime.setSourceLocation(limit.getSourceLocation());
         builder.contributeMicroOperator(limit, runtime, recDesc);
         // and contribute one edge from its child
         ILogicalOperator src = limit.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
index 3ff7dc1..0789cee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
@@ -68,6 +68,7 @@
             projectionList[i++] = pos;
         }
         StreamProjectRuntimeFactory runtime = new StreamProjectRuntimeFactory(projectionList, flushFramesRapidly);
+        runtime.setSourceLocation(project.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(project, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index ddde5f3..a519275 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -70,6 +70,7 @@
                 new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(),
                         select.getRetainMissing(), inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()),
                         context.getMissingWriterFactory());
+        runtime.setSourceLocation(select.getSourceLocation());
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(select, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
index 01e9a0c..0e8005d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
@@ -65,6 +65,7 @@
         StringStreamingScriptDescription sssd = (StringStreamingScriptDescription) scriptDesc;
         StringStreamingRuntimeFactory runtime = new StringStreamingRuntimeFactory(sssd.getCommand(),
                 sssd.getPrinterFactories(), sssd.getFieldDelimiter(), sssd.getParserFactory());
+        runtime.setSourceLocation(scriptOp.getSourceLocation());
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         builder.contributeMicroOperator(scriptOp, runtime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
index 95efbac..5ec6d0a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -101,6 +101,7 @@
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         SubplanRuntimeFactory runtime =
                 new SubplanRuntimeFactory(np, missingWriterFactories, inputRecordDesc, recDesc, null);
+        runtime.setSourceLocation(subplan.getSourceLocation());
         builder.contributeMicroOperator(subplan, runtime, recDesc);
 
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
index cd696bc..43a1695 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -94,8 +93,10 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
                 mp.getTokenizerRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
                         secondaryKeys, null, inputDesc, context, spec, true);
-        builder.contributeHyracksOperator(tokenizeOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(tokenizeOp.getSourceLocation());
+        builder.contributeHyracksOperator(tokenizeOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = tokenizeOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, tokenizeOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
index 4ccce92..fcd9fa7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -50,6 +50,7 @@
 
         UnionAllOperatorDescriptor opDesc =
                 new UnionAllOperatorDescriptor(builder.getJobSpec(), op.getInputs().size(), recordDescriptor);
+        opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
         super.contributeRuntimeOperator(builder, context, op, opSchema, inputSchemas, outerPlanSchema);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
index 7ec1914..70e596a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/WriteResultPOperator.java
@@ -101,9 +101,10 @@
         JobSpecification spec = builder.getJobSpec();
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getWriteResultRuntime(
                 dataSource, propagatedSchema, keys, payload, additionalFilteringKeys, context, spec);
-
-        builder.contributeHyracksOperator(writeResultOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        IOperatorDescriptor opDesc = runtimeAndConstraints.first;
+        opDesc.setSourceLocation(writeResultOp.getSourceLocation());
+        builder.contributeHyracksOperator(writeResultOp, opDesc);
+        builder.contributeAlgebricksPartitionConstraint(opDesc, runtimeAndConstraints.second);
         ILogicalOperator src = writeResultOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, writeResultOp, 0);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 70f19c1..99ed738 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -299,21 +299,25 @@
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit());
+        return null;
     }
 
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
             throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        return null;
     }
 
-    private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
-            throws AlgebricksException {
+    private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+            String opSignature) throws AlgebricksException {
         AlgebricksAppendable plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
                 + op.getExpressionRef().getValue().accept(exprVisitor, indent));
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
-        return null;
+        return plan;
     }
 
     @Override
@@ -321,6 +325,24 @@
         AlgebricksAppendable plan = addIndent(indent).append(
                 "data-scan " + op.getProjectVariables() + "<-" + op.getVariables() + " <- " + op.getDataSource());
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit());
+        return null;
+    }
+
+    private Void appendSelectConditionInformation(AlgebricksAppendable plan,
+            Mutable<ILogicalExpression> selectCondition, Integer indent) throws AlgebricksException {
+        if (selectCondition != null) {
+            plan.append(" condition (").append(selectCondition.getValue().accept(exprVisitor, indent)).append(")");
+        }
+
+        return null;
+    }
+
+    private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit) throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(" limit ").append(String.valueOf(outputLimit));
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 8acf08b..f1f1f3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -83,7 +83,7 @@
 
     public class IdCounter {
         private int id;
-        private Deque<Integer> prefix;
+        private final Deque<Integer> prefix;
 
         public IdCounter() {
             prefix = new LinkedList<Integer>();
@@ -400,24 +400,28 @@
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit(), indent);
+        return null;
     }
 
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
             throws AlgebricksException {
-        return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        return null;
     }
 
-    private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
-            throws AlgebricksException {
+    private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+            String opSignature) throws AlgebricksException {
         AlgebricksAppendable plan = addIndent(indent).append("\"operator\": \"" + opSignature + "\"");
         variablePrintHelper(op.getVariables(), indent);
         buffer.append(",\n");
         addIndent(indent).append("\"expressions\": \""
                 + op.getExpressionRef().getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
-        return null;
+        return plan;
     }
 
     @Override
@@ -435,6 +439,8 @@
             addIndent(indent).append("\"data-source\": \"" + op.getDataSource() + "\"");
         }
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
+        appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+        appendLimitInformation(plan, op.getOutputLimit(), indent);
         return null;
     }
 
@@ -467,6 +473,25 @@
         return null;
     }
 
+    private Void appendSelectConditionInformation(AlgebricksAppendable plan, Mutable<ILogicalExpression> condition,
+            Integer indent) throws AlgebricksException {
+        if (condition != null) {
+            plan.append(",\n");
+            addIndent(indent).append(
+                    "\"condition\": \"" + condition.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+        }
+        return null;
+    }
+
+    private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit, Integer indent)
+            throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(",\n");
+            addIndent(indent).append("\"limit\": \"" + outputLimit + "\"");
+        }
+        return null;
+    }
+
     private void appendVars(List<LogicalVariable> minFilterVars) throws AlgebricksException {
         boolean first = true;
         for (LogicalVariable v : minFilterVars) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
index bc6a45d..3e78fd2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
@@ -60,6 +60,12 @@
 
     @Override
     public void substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
+
+    }
+
+    @Override
+    public IPartitioningProperty clonePartitioningProperty() {
+        return new BroadcastPartitioningProperty(domain);
     }
 
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
index f41d197..5164192 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
@@ -77,6 +77,8 @@
     void setNodeDomain(INodeDomain domain);
 
     void substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap);
+
+    IPartitioningProperty clonePartitioningProperty();
 }
 
 class UnpartitionedProperty implements IPartitioningProperty {
@@ -116,4 +118,9 @@
     public void substituteColumnVars(Map<LogicalVariable, LogicalVariable> variableMap) {
         // No partition columns are maintained for UNPARTITIONED.
     }
+
+    @Override
+    public IPartitioningProperty clonePartitioningProperty() {
+        return new UnpartitionedProperty();
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
index 23c8273..b5a2bb5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
@@ -28,7 +28,7 @@
 
 public class OrderedPartitionedProperty implements IPartitioningProperty {
 
-    private List<OrderColumn> orderColumns;
+    private final List<OrderColumn> orderColumns;
     private INodeDomain domain;
 
     public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain) {
@@ -92,4 +92,9 @@
         }
     }
 
+    @Override
+    public IPartitioningProperty clonePartitioningProperty() {
+        return new OrderedPartitionedProperty(new ArrayList<>(orderColumns), domain);
+    }
+
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java
index bbd835c..951a031 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/RandomPartitioningProperty.java
@@ -65,6 +65,12 @@
 
     @Override
     public void substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
+
+    }
+
+    @Override
+    public IPartitioningProperty clonePartitioningProperty() {
+        return new RandomPartitioningProperty(domain);
     }
 
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
index f59638c..5966407 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.hyracks.algebricks.common.utils.ListSet;
 import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 
@@ -77,4 +78,9 @@
         });
     }
 
+    @Override
+    public IPartitioningProperty clonePartitioningProperty() {
+        return new UnorderedPartitionedProperty(new ListSet<>(columnSet), domain);
+    }
+
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index c574cd8..67199b9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -262,6 +262,7 @@
     public static ILogicalOperator deepCopy(ILogicalOperator op) throws AlgebricksException {
         OperatorDeepCopyVisitor visitor = new OperatorDeepCopyVisitor();
         AbstractLogicalOperator copiedOperator = (AbstractLogicalOperator) op.accept(visitor, null);
+        copiedOperator.setSourceLocation(op.getSourceLocation());
         copiedOperator.setExecutionMode(op.getExecutionMode());
         copiedOperator.getAnnotations().putAll(op.getAnnotations());
         copiedOperator.setSchema(op.getSchema());
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index c63e8a1..8779777 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -22,7 +22,7 @@
 import org.apache.logging.log4j.Logger;
 
 public class AlgebricksConfig {
-    public static final boolean DEBUG = true;
+
     public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
index cad62c4..370ec6d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
@@ -19,7 +19,6 @@
 package org.apache.hyracks.algebricks.core.rewriter.base;
 
 import java.util.Collection;
-import java.util.logging.Level;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -66,7 +65,7 @@
     }
 
     private String getPlanString(Mutable<ILogicalOperator> opRef) throws AlgebricksException {
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled() && context != null) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled() && context != null) {
             LogicalOperatorPrettyPrintVisitor pvisitor = context.getPrettyPrintVisitor();
             pvisitor.reset(new AlgebricksAppendable());
             PlanPrettyPrinter.printOperator((AbstractLogicalOperator) opRef.getValue(), pvisitor, 0);
@@ -77,10 +76,10 @@
 
     private void printRuleApplication(IAlgebraicRewriteRule rule, String beforePlan, String afterPlan)
             throws AlgebricksException {
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Rule " + rule.getClass() + " fired.\n");
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Before plan\n" + beforePlan + "\n");
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> After plan\n" + afterPlan + "\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Rule " + rule.getClass() + " fired.\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Before plan\n" + beforePlan + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> After plan\n" + afterPlan + "\n");
         }
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index 9119d6c..8eb9b90 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -74,15 +74,15 @@
         if (plan == null) {
             return;
         }
-        if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting logical optimizations.\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting logical optimizations.\n");
         }
 
-        logPlanAt("Logical Plan", Level.DEBUG);
+        logPlanAt("Logical Plan", Level.TRACE);
         runOptimizationSets(plan, logicalRewrites);
         computeSchemaBottomUpForPlan(plan);
         runPhysicalOptimizations(plan, physicalRewrites);
-        logPlanAt("Optimized Plan", Level.DEBUG);
+        logPlanAt("Optimized Plan", Level.TRACE);
     }
 
     private void logPlanAt(String name, Level lvl) throws AlgebricksException {
@@ -126,8 +126,8 @@
     private void runPhysicalOptimizations(ILogicalPlan plan,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites)
             throws AlgebricksException {
-        if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Starting physical optimizations.\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting physical optimizations.\n");
         }
         // PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(plan);
         runOptimizationSets(plan, physicalRewrites);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 4649d6d..2cb2d35 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -289,21 +289,24 @@
     @Override
     public String visitUnnestMapOperator(UnnestMapOperator op, Void noArgs) throws AlgebricksException {
         stringBuilder.setLength(0);
-        return printAbstractUnnestMapOperator(op, "unnest-map");
+        printAbstractUnnestMapOperator(op, "unnest-map");
+        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        return stringBuilder.toString();
     }
 
     @Override
     public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void noArgs)
             throws AlgebricksException {
         stringBuilder.setLength(0);
-        return printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        return stringBuilder.toString();
     }
 
-    private String printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
+    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
         stringBuilder.append(opSignature).append(" ").append(op.getVariables()).append(" <- ")
                 .append(op.getExpressionRef().getValue().toString());
         appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
-        return stringBuilder.toString();
     }
 
     @Override
@@ -312,10 +315,12 @@
         stringBuilder.append("data-scan ").append(op.getProjectVariables()).append("<-").append(op.getVariables())
                 .append(" <- ").append(op.getDataSource());
         appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+        appendLimitInformation(stringBuilder, op.getOutputLimit());
         return stringBuilder.toString();
     }
 
-    private String appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
+    private void appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
             List<LogicalVariable> maxFilterVars) {
         if (minFilterVars != null || maxFilterVars != null) {
             plan.append(" with filter on");
@@ -326,7 +331,21 @@
         if (maxFilterVars != null) {
             plan.append(" max:").append(maxFilterVars);
         }
-        return stringBuilder.toString();
+    }
+
+    private Void appendSelectConditionInformation(StringBuilder plan, Mutable<ILogicalExpression> condition)
+            throws AlgebricksException {
+        if (condition != null) {
+            plan.append(" condition:").append(condition.getValue().toString());
+        }
+        return null;
+    }
+
+    private Void appendLimitInformation(StringBuilder plan, long outputLimit) throws AlgebricksException {
+        if (outputLimit >= 0) {
+            plan.append(" limit:").append(String.valueOf(outputLimit));
+        }
+        return null;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
index 24a5e21..d507e5a 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.algebricks.data;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
 public interface IBinaryBooleanInspector {
-    public boolean getBooleanValue(byte[] bytes, int offset, int length);
+    public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/utils/WriteValueTools.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/utils/WriteValueTools.java
index ba27c4e..e60bc75 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/utils/WriteValueTools.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/utils/WriteValueTools.java
@@ -20,7 +20,6 @@
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.io.PrintStream;
 
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
index d9dccce..dd1cbe6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
@@ -49,6 +49,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
index 7802b29..7fc2efe 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractDecorrelationRule.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractDecorrelationRule implements IAlgebraicRewriteRule {
 
@@ -91,9 +92,11 @@
 
     protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g,
             List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+        SourceLocation sourceLoc = g.getSourceLocation();
         for (LogicalVariable ov : vars) {
             LogicalVariable newVar = context.newVar();
-            ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+            VariableReferenceExpression varExpr = new VariableReferenceExpression(newVar);
+            varExpr.setSourceLocation(sourceLoc);
             outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
                     new MutableObject<ILogicalExpression>(varExpr)));
             for (ILogicalPlan p : g.getNestedPlans()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
index bce72c1..036b3e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractExtractExprRule.java
@@ -36,6 +36,7 @@
             IOptimizationContext context) throws AlgebricksException {
         LogicalVariable v = context.newVar();
         AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(gExpr));
+        a.setSourceLocation(gExpr.getSourceLocation());
         a.getInputs().add(new MutableObject<ILogicalOperator>(opRef2.getValue()));
         opRef2.setValue(a);
         if (gExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
index 3f61cc0..ade2402 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractIntroduceCombinerRule implements IAlgebraicRewriteRule {
 
@@ -67,7 +68,7 @@
 
     protected Pair<Boolean, Mutable<ILogicalOperator>> tryToPushAgg(AggregateOperator initAgg, GroupByOperator newGbyOp,
             Set<SimilarAggregatesInfo> toReplaceSet, IOptimizationContext context) throws AlgebricksException {
-
+        SourceLocation sourceLoc = initAgg.getSourceLocation();
         List<LogicalVariable> initVars = initAgg.getVariables();
         List<Mutable<ILogicalExpression>> initExprs = initAgg.getExpressions();
         int numExprs = initVars.size();
@@ -87,6 +88,7 @@
         for (int i = 0; i < numExprs; i++) {
             Mutable<ILogicalExpression> expRef = initExprs.get(i);
             AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) expRef.getValue();
+            SourceLocation aggFunSourceLoc = aggFun.getSourceLocation();
             IFunctionInfo fi1 = aggFun.getStepOneAggregate();
             // Clone the aggregate's args.
             List<Mutable<ILogicalExpression>> newArgs = new ArrayList<>(aggFun.getArguments().size());
@@ -98,10 +100,13 @@
             SimilarAggregatesInfo inf = new SimilarAggregatesInfo();
             LogicalVariable newAggVar = context.newVar();
             pushedVars.add(newAggVar);
-            inf.stepOneResult = new VariableReferenceExpression(newAggVar);
+            VariableReferenceExpression newAggVarRef = new VariableReferenceExpression(newAggVar);
+            newAggVarRef.setSourceLocation(aggFunSourceLoc);
+            inf.stepOneResult = newAggVarRef;
             inf.simAggs = new ArrayList<>();
             toReplaceSet.add(inf);
             AggregateFunctionCallExpression aggLocal = new AggregateFunctionCallExpression(fi1, false, newArgs);
+            aggLocal.setSourceLocation(aggFunSourceLoc);
             pushedExprs.add(new MutableObject<>(aggLocal));
             AggregateExprInfo aei = new AggregateExprInfo();
             aei.aggExprRef = expRef;
@@ -112,6 +117,7 @@
 
         if (!pushedVars.isEmpty()) {
             AggregateOperator pushedAgg = new AggregateOperator(pushedVars, pushedExprs);
+            pushedAgg.setSourceLocation(sourceLoc);
             pushedAgg.setExecutionMode(ExecutionMode.LOCAL);
             // If newGbyOp is null, then we optimizing an aggregate without group by.
             if (newGbyOp != null) {
@@ -133,6 +139,7 @@
 
                 // Hook up the nested aggregate op with the outer group by.
                 NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<>(newGbyOp));
+                nts.setSourceLocation(sourceLoc);
                 nts.setExecutionMode(ExecutionMode.LOCAL);
                 bottomRef.setValue(nts);
                 pushedAgg.getInputs().add(inputRef);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
index a921301..dc48d96 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public abstract class AbstractIntroduceGroupByCombinerRule extends AbstractIntroduceCombinerRule {
 
@@ -105,7 +106,9 @@
         for (LogicalVariable var : freeVars) {
             if (!propagatedVars.contains(var)) {
                 LogicalVariable newDecorVar = context.newVar();
-                newGbyOp.addDecorExpression(newDecorVar, new VariableReferenceExpression(var));
+                VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+                varRef.setSourceLocation(gbyOp.getSourceLocation());
+                newGbyOp.addDecorExpression(newDecorVar, varRef);
                 VariableUtilities.substituteVariables(gbyOp.getNestedPlans().get(0).getRoots().get(0).getValue(), var,
                         newDecorVar, context);
             }
@@ -128,10 +131,12 @@
 
     private GroupByOperator opToPush(GroupByOperator gbyOp, BookkeepingInfo bi, IOptimizationContext context)
             throws AlgebricksException {
+        SourceLocation sourceLoc = gbyOp.getSourceLocation();
         // Hook up input to new group-by.
         Mutable<ILogicalOperator> opRef3 = gbyOp.getInputs().get(0);
         ILogicalOperator op3 = opRef3.getValue();
         GroupByOperator newGbyOp = new GroupByOperator();
+        newGbyOp.setSourceLocation(sourceLoc);
         newGbyOp.getInputs().add(new MutableObject<ILogicalOperator>(op3));
         // Copy annotations.
         Map<String, Object> annotations = newGbyOp.getAnnotations();
@@ -199,7 +204,9 @@
         // set the vars in the new op
         int n = newOpGbyList.size();
         for (int i = 0; i < n; i++) {
-            newGbyOp.addGbyExpression(replGbyList.get(i), new VariableReferenceExpression(newOpGbyList.get(i)));
+            VariableReferenceExpression varRef = new VariableReferenceExpression(newOpGbyList.get(i));
+            varRef.setSourceLocation(sourceLoc);
+            newGbyOp.addGbyExpression(replGbyList.get(i), varRef);
             VariableUtilities.substituteVariables(gbyOp, newOpGbyList.get(i), replGbyList.get(i), false, context);
         }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index d975cce..ab665b3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class BreakSelectIntoConjunctsRule implements IAlgebraicRewriteRule {
 
@@ -58,6 +59,8 @@
             return false;
         }
 
+        SourceLocation sourceLoc = select.getSourceLocation();
+
         Mutable<ILogicalOperator> childOfSelect = select.getInputs().get(0);
         boolean fst = true;
         ILogicalOperator botOp = select;
@@ -70,6 +73,7 @@
             } else {
                 SelectOperator newSelect = new SelectOperator(new MutableObject<ILogicalExpression>(e),
                         select.getRetainMissing(), select.getMissingPlaceholderVariable());
+                newSelect.setSourceLocation(sourceLoc);
                 List<Mutable<ILogicalOperator>> botInpList = botOp.getInputs();
                 botInpList.clear();
                 botInpList.add(new MutableObject<ILogicalOperator>(newSelect));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
index 0b94fdd..28139d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexJoinInferenceRule.java
@@ -83,6 +83,7 @@
         ntsToEtsInSubplan(subplan, context);
         cleanupJoins(subplan);
         InnerJoinOperator join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        join.setSourceLocation(op.getSourceLocation());
         join.getInputs().add(opRef3);
         opRef2.setValue(OperatorManipulationUtil.eliminateSingleSubplanOverEts(subplan));
         join.getInputs().add(new MutableObject<ILogicalOperator>(op));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
index fa35a98..54fe09d 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -117,6 +117,7 @@
 
         InnerJoinOperator product =
                 new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        product.setSourceLocation(op.getSourceLocation());
         // Outer branch.
         product.getInputs().add(new MutableObject<ILogicalOperator>(outerRoot));
         // Inner branch.
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
index ae52c35..d9849b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -79,6 +79,7 @@
             // Initialize the new conjuncts, if necessary.
             if (conj == null) {
                 conj = new ScalarFunctionCallExpression(andFn);
+                conj.setSourceLocation(firstSelect.getSourceLocation());
                 // Add the first select's condition.
                 conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
index 53548e4..7e36748 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/CopyLimitDownRule.java
@@ -38,16 +38,17 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamLimitPOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class CopyLimitDownRule implements IAlgebraicRewriteRule {
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
     }
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
@@ -66,13 +67,17 @@
 
         List<LogicalVariable> candidateProducedVars = new ArrayList<>();
         while (true) {
-            candidateProducedVars.clear();
             ILogicalOperator candidateOp = candidateOpRef.getValue();
             LogicalOperatorTag candidateOpTag = candidateOp.getOperatorTag();
             if (candidateOp.getInputs().size() > 1 || !candidateOp.isMap()
                     || candidateOpTag == LogicalOperatorTag.SELECT || candidateOpTag == LogicalOperatorTag.LIMIT
-                    || candidateOpTag == LogicalOperatorTag.UNNEST_MAP
-                    || !OperatorPropertiesUtil.disjoint(limitUsedVars, candidateProducedVars)) {
+                    || candidateOpTag == LogicalOperatorTag.UNNEST_MAP) {
+                break;
+            }
+
+            candidateProducedVars.clear();
+            VariableUtilities.getProducedVariables(candidateOp, candidateProducedVars);
+            if (!OperatorPropertiesUtil.disjoint(limitUsedVars, candidateProducedVars)) {
                 break;
             }
 
@@ -84,9 +89,11 @@
             ILogicalOperator safeOp = safeOpRef.getValue();
             Mutable<ILogicalOperator> unsafeOpRef = safeOp.getInputs().get(0);
             ILogicalOperator unsafeOp = unsafeOpRef.getValue();
+            SourceLocation sourceLoc = limitOp.getSourceLocation();
             LimitOperator limitCloneOp = null;
             if (limitOp.getOffset().getValue() == null) {
                 limitCloneOp = new LimitOperator(limitOp.getMaxObjects().getValue(), false);
+                limitCloneOp.setSourceLocation(sourceLoc);
             } else {
                 // Need to add an offset to the given limit value
                 // since the original topmost limit will use the offset value.
@@ -98,7 +105,9 @@
                         new MutableObject<ILogicalExpression>(limitOp.getMaxObjects().getValue().cloneExpression()));
                 addArgs.add(new MutableObject<ILogicalExpression>(limitOp.getOffset().getValue().cloneExpression()));
                 ScalarFunctionCallExpression maxPlusOffset = new ScalarFunctionCallExpression(finfoAdd, addArgs);
+                maxPlusOffset.setSourceLocation(sourceLoc);
                 limitCloneOp = new LimitOperator(maxPlusOffset, false);
+                limitCloneOp.setSourceLocation(sourceLoc);
             }
             limitCloneOp.setPhysicalOperator(new StreamLimitPOperator());
             limitCloneOp.getInputs().add(new MutableObject<ILogicalOperator>(unsafeOp));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
index baad59b..096520e 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
@@ -141,6 +141,7 @@
             List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprs =
                     deepCopyOrderAndExpression(sourceOrderOp.getOrderExpressions());
             OrderOperator newOrderOp = new OrderOperator(orderExprs);
+            newOrderOp.setSourceLocation(sourceOrderOp.getSourceLocation());
             context.addToDontApplySet(this, newOrderOp);
             inputs.set(i, new MutableObject<ILogicalOperator>(newOrderOp));
             newOrderOp.getInputs().add(inputOpRef);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index f06933a..6f7f86a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -89,6 +89,7 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 
 public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
@@ -123,8 +124,8 @@
         // somewhere else, too.
 
         physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Optimizing operator " + op.getPhysicalOperator() + ".\n");
         }
 
         PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(op, context);
@@ -319,7 +320,7 @@
         }
 
         if (opIsRedundantSort) {
-            if (AlgebricksConfig.DEBUG && loggerTraceEnabled) {
+            if (loggerTraceEnabled) {
                 AlgebricksConfig.ALGEBRICKS_LOGGER
                         .trace(">>>> Removing redundant SORT operator " + op.getPhysicalOperator() + "\n");
                 printOp(op);
@@ -476,7 +477,7 @@
 
     private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
             boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER
                     .trace(">>>> Adding local enforcers for local props = " + localProperties + "\n");
         }
@@ -526,16 +527,20 @@
     private Mutable<ILogicalOperator> enforceOrderProperties(List<LocalOrderProperty> oList,
             Mutable<ILogicalOperator> topOp, boolean isMicroOp, IOptimizationContext context)
             throws AlgebricksException {
+        SourceLocation sourceLoc = topOp.getValue().getSourceLocation();
         List<Pair<IOrder, Mutable<ILogicalExpression>>> oe = new LinkedList<>();
         for (LocalOrderProperty orderProperty : oList) {
             for (OrderColumn oc : orderProperty.getOrderColumns()) {
                 IOrder ordType = (oc.getOrder() == OrderKind.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
-                Pair<IOrder, Mutable<ILogicalExpression>> pair = new Pair<>(ordType,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oc.getColumn())));
+                VariableReferenceExpression ocColumnRef = new VariableReferenceExpression(oc.getColumn());
+                ocColumnRef.setSourceLocation(sourceLoc);
+                Pair<IOrder, Mutable<ILogicalExpression>> pair =
+                        new Pair<>(ordType, new MutableObject<ILogicalExpression>(ocColumnRef));
                 oe.add(pair);
             }
         }
         OrderOperator oo = new OrderOperator(oe);
+        oo.setSourceLocation(sourceLoc);
         oo.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
         if (isMicroOp) {
             oo.setPhysicalOperator(new InMemoryStableSortPOperator());
@@ -544,7 +549,7 @@
         }
         oo.getInputs().add(topOp);
         context.computeAndSetTypeEnvironmentForOperator(oo);
-        if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace(">>>> Added sort enforcer " + oo.getPhysicalOperator() + ".\n");
         }
         return new MutableObject<ILogicalOperator>(oo);
@@ -575,7 +580,7 @@
                 }
                 case UNORDERED_PARTITIONED: {
                     List<LogicalVariable> varList = new ArrayList<>(((UnorderedPartitionedProperty) pp).getColumnSet());
-                    String hashMergeHint = context.getMetadataProvider().getConfig().get(HASH_MERGE);
+                    String hashMergeHint = (String) context.getMetadataProvider().getConfig().get(HASH_MERGE);
                     if (hashMergeHint == null || !hashMergeHint.equalsIgnoreCase(TRUE_CONSTANT)) {
                         pop = new HashPartitionExchangePOperator(varList, domain);
                         break;
@@ -627,9 +632,9 @@
             exchg.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
             OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
             context.computeAndSetTypeEnvironmentForOperator(exchg);
-            if (AlgebricksConfig.DEBUG && AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
+            if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
                 AlgebricksConfig.ALGEBRICKS_LOGGER
-                        .debug(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
+                        .trace(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
                 printOp((AbstractLogicalOperator) op);
             }
         }
@@ -647,8 +652,8 @@
     private void printOp(AbstractLogicalOperator op) throws AlgebricksException {
         LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
         PlanPrettyPrinter.printOperator(op, pvisitor, 0);
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug(pvisitor.get().toString());
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace(pvisitor.get().toString());
         }
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index b95d6e4..ad0a9da 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -41,13 +41,12 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Factors out common sub-expressions by assigning them to a variables, and replacing the common sub-expressions with references to those variables.
@@ -74,15 +73,6 @@
  * distinct ([$$5])
  * assign [$$5] <- [field-access($$0, 1)]
  * unnest $$0 <- [scan-dataset]
- * Example 3 - Pulling Common Expressions Above Joins (simplified)
- * Before plan:
- * assign [$$9] <- funcZ(funcY($$8))
- * join (funcX(funcY($$8)))
- * After plan:
- * assign [$$9] <- funcZ($$10))
- * select (funcX($$10))
- * assign [$$10] <- [funcY($$8)]
- * join (TRUE)
  */
 public class ExtractCommonExpressionsRule implements IAlgebraicRewriteRule {
 
@@ -255,7 +245,10 @@
                         // Also just replace the expr if we are replacing common exprs from within the same operator.
                         if (liveVars.contains(exprEqClass.getVariable()) || !liveVars.containsAll(usedVars)
                                 || op == exprEqClass.getFirstOperator()) {
-                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(exprEqClass.getVariable());
+                            varRef.setSourceLocation(expr.getSourceLocation());
+                            exprRef.setValue(varRef);
                             // Do not descend into children since this expr has been completely replaced.
                             return true;
                         }
@@ -267,7 +260,10 @@
                         VariableUtilities.getLiveVariables(op, liveVars);
                         //rewrite only when the variable is live
                         if (liveVars.contains(exprEqClass.getVariable())) {
-                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            VariableReferenceExpression varRef =
+                                    new VariableReferenceExpression(exprEqClass.getVariable());
+                            varRef.setSourceLocation(expr.getSourceLocation());
+                            exprRef.setValue(varRef);
                             // Do not descend into children since this expr has been completely replaced.
                             return true;
                         }
@@ -295,41 +291,27 @@
 
         private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr)
                 throws AlgebricksException {
+            SourceLocation sourceLoc = expr.getSourceLocation();
             AbstractLogicalOperator firstOp = (AbstractLogicalOperator) exprEqClass.getFirstOperator();
             Mutable<ILogicalExpression> firstExprRef = exprEqClass.getFirstExpression();
-            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN
-                    || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
-                // Do not extract common expressions from within the same join operator.
-                if (firstOp == op) {
-                    return false;
-                }
-                AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) firstOp;
-                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();
-                ILogicalExpression enclosingExpr = getEnclosingExpression(joinCond, firstExprRef.getValue());
-                if (enclosingExpr == null) {
-                    // No viable enclosing expression that we can pull out from the join.
-                    return false;
-                }
-                // Place a Select operator beneath op that contains the enclosing expression.
-                SelectOperator selectOp =
-                        new SelectOperator(new MutableObject<ILogicalExpression>(enclosingExpr), false, null);
-                selectOp.getInputs().add(new MutableObject<ILogicalOperator>(op.getInputs().get(0).getValue()));
-                op.getInputs().get(0).setValue(selectOp);
-                // Set firstOp to be the select below op, since we want to assign the common subexpr there.
-                firstOp = selectOp;
-            } else if (firstOp.getInputs().size() > 1) {
+            // We don't consider to eliminate common exprs in join operators by doing a cartesian production
+            // and pulling the condition in to a select. This will negatively impact the performance.
+            if (firstOp.getInputs().size() > 1) {
                 // Bail for any non-join operator with multiple inputs.
                 return false;
             }
             LogicalVariable newVar = context.newVar();
             AssignOperator newAssign = new AssignOperator(newVar,
                     new MutableObject<ILogicalExpression>(firstExprRef.getValue().cloneExpression()));
+            newAssign.setSourceLocation(sourceLoc);
             // Place assign below firstOp.
             newAssign.getInputs().add(new MutableObject<ILogicalOperator>(firstOp.getInputs().get(0).getValue()));
             newAssign.setExecutionMode(firstOp.getExecutionMode());
             firstOp.getInputs().get(0).setValue(newAssign);
             // Replace original expr with variable reference, and set var in expression equivalence class.
-            firstExprRef.setValue(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(sourceLoc);
+            firstExprRef.setValue(newVarRef);
             exprEqClass.setVariable(newVar);
             context.computeAndSetTypeEnvironmentForOperator(newAssign);
             context.computeAndSetTypeEnvironmentForOperator(firstOp);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 4c2d910..3335d71 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
 
@@ -162,7 +163,9 @@
                 continue;
             }
             candidate = group.get(0);
+            SourceLocation candidateSourceLoc = candidate.getValue().getSourceLocation();
             ReplicateOperator rop = new ReplicateOperator(group.size(), materializationFlags);
+            rop.setSourceLocation(candidateSourceLoc);
             rop.setPhysicalOperator(new ReplicatePOperator());
             Mutable<ILogicalOperator> ropRef = new MutableObject<ILogicalOperator>(rop);
             AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
@@ -204,7 +207,9 @@
             VariableUtilities.getLiveVariables(candidate.getValue(), liveVarsNew);
             ArrayList<Mutable<ILogicalExpression>> assignExprs = new ArrayList<Mutable<ILogicalExpression>>();
             for (LogicalVariable liveVar : liveVarsNew) {
-                assignExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(liveVar)));
+                VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
+                liveVarRef.setSourceLocation(candidateSourceLoc);
+                assignExprs.add(new MutableObject<ILogicalExpression>(liveVarRef));
             }
             for (Mutable<ILogicalOperator> ref : group) {
                 if (ref.equals(candidate)) {
@@ -218,10 +223,14 @@
                     liveVars.add(variableMappingBack.get(liveVarsNew.get(i)));
                 }
 
+                SourceLocation refSourceLoc = ref.getValue().getSourceLocation();
+
                 AbstractLogicalOperator assignOperator = new AssignOperator(liveVars, assignExprs);
+                assignOperator.setSourceLocation(refSourceLoc);
                 assignOperator.setExecutionMode(rop.getExecutionMode());
                 assignOperator.setPhysicalOperator(new AssignPOperator());
                 AbstractLogicalOperator projectOperator = new ProjectOperator(liveVars);
+                projectOperator.setSourceLocation(refSourceLoc);
                 projectOperator.setPhysicalOperator(new StreamProjectPOperator());
                 projectOperator.setExecutionMode(rop.getExecutionMode());
                 AbstractLogicalOperator exchOp = new ExchangeOperator();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
index 198510a..d937eb1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Factors out function expressions from each comparison function or similarity function in join condition by
@@ -104,9 +105,11 @@
         } else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi) || isComparisonFunction(fi)) {
             for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
                 if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    SourceLocation exprRefSourceLoc = exprRef.getValue().getSourceLocation();
                     LogicalVariable newVar = context.newVar();
                     AssignOperator newAssign = new AssignOperator(newVar,
                             new MutableObject<ILogicalExpression>(exprRef.getValue().cloneExpression()));
+                    newAssign.setSourceLocation(exprRefSourceLoc);
                     newAssign.setExecutionMode(joinOp.getExecutionMode());
 
                     // Place assign below joinOp.
@@ -137,7 +140,9 @@
 
                     if (modified) {
                         // Replace original expr with variable reference.
-                        exprRef.setValue(new VariableReferenceExpression(newVar));
+                        VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+                        newVarRef.setSourceLocation(exprRefSourceLoc);
+                        exprRef.setValue(newVarRef);
                         context.computeAndSetTypeEnvironmentForOperator(newAssign);
                         context.computeAndSetTypeEnvironmentForOperator(joinOp);
                     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
index 7a0012a..eb2bee6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGbyExpressionsRule.java
@@ -73,7 +73,9 @@
             ILogicalExpression expr = gbyPair.second.getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                gbyPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                gbyPair.second.setValue(vRef);
             }
         }
         return true;
@@ -88,7 +90,9 @@
             ILogicalExpression expr = decorPair.second.getValue();
             if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                 LogicalVariable v = extractExprIntoAssignOpRef(expr, opRef2, context);
-                decorPair.second.setValue(new VariableReferenceExpression(v));
+                VariableReferenceExpression vRef = new VariableReferenceExpression(v);
+                vRef.setSourceLocation(expr.getSourceLocation());
+                decorPair.second.setValue(vRef);
             }
         }
         return true;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
index 05cc7b6..5bbc80d 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractGroupByDecorVariablesRule.java
@@ -76,7 +76,9 @@
             exprs.add(exprRef);
 
             // Normalizes the decor entry -- expression be a variable reference
-            decorVarExpr.second = new MutableObject<>(new VariableReferenceExpression(newVar));
+            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+            newVarRef.setSourceLocation(expr.getSourceLocation());
+            decorVarExpr.second = new MutableObject<>(newVarRef);
         }
         if (!changed) {
             return false;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
index 2f28a84..d549b90 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/FactorRedundantGroupAndDecorVarsRule.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class FactorRedundantGroupAndDecorVarsRule implements IAlgebraicRewriteRule {
 
@@ -78,8 +79,11 @@
             LogicalVariable lhs = varRhsToLhs.get(v);
             if (lhs != null) {
                 if (p.first != null) {
-                    AssignOperator assign = new AssignOperator(p.first,
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lhs)));
+                    VariableReferenceExpression lhsRef = new VariableReferenceExpression(lhs);
+                    SourceLocation sourceLoc = p.second.getValue().getSourceLocation();
+                    lhsRef.setSourceLocation(sourceLoc);
+                    AssignOperator assign = new AssignOperator(p.first, new MutableObject<ILogicalExpression>(lhsRef));
+                    assign.setSourceLocation(sourceLoc);
                     ILogicalOperator op = opRef.getValue();
                     assign.getInputs().add(new MutableObject<ILogicalOperator>(op));
                     opRef.setValue(assign);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index cd42407..6967271 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -187,7 +187,7 @@
         return modified;
     }
 
-    protected class InlineVariablesVisitor implements ILogicalExpressionReferenceTransform {
+    public static class InlineVariablesVisitor implements ILogicalExpressionReferenceTransform {
 
         private final Map<LogicalVariable, ILogicalExpression> varAssignRhs;
         private final Set<LogicalVariable> liveVars = new HashSet<>();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
index 2772d8d..c4ae57d 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InsertProjectBeforeUnionRule.java
@@ -76,12 +76,13 @@
     private void insertProjectOperator(UnionAllOperator opUnion, int branch, ArrayList<LogicalVariable> usedVariables,
             IOptimizationContext context) throws AlgebricksException {
         ProjectOperator projectOp = new ProjectOperator(usedVariables);
-        ILogicalOperator parentOp = opUnion.getInputs().get(branch).getValue();
-        projectOp.getInputs().add(new MutableObject<ILogicalOperator>(parentOp));
+        ILogicalOperator inputOp = opUnion.getInputs().get(branch).getValue();
+        projectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         opUnion.getInputs().get(branch).setValue(projectOp);
         projectOp.setPhysicalOperator(new StreamProjectPOperator());
+        projectOp.setExecutionMode(inputOp.getExecutionMode());
         context.computeAndSetTypeEnvironmentForOperator(projectOp);
-        context.computeAndSetTypeEnvironmentForOperator(parentOp);
+        context.computeAndSetTypeEnvironmentForOperator(inputOp);
     }
 
     private boolean isIdentical(List<LogicalVariable> finalSchema, List<LogicalVariable> inputSchema)
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
index 43c58e2..f432c9f 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
@@ -163,6 +163,7 @@
                     assignOp.getExpressions().get(i).getValue().cloneExpression()));
         }
         AssignOperator assignCloneOp = new AssignOperator(vars, exprs);
+        assignCloneOp.setSourceLocation(assignOp.getSourceLocation());
         assignCloneOp.setExecutionMode(assignOp.getExecutionMode());
         return assignCloneOp;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
index d7090d2..e8f701c 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushFunctionsBelowJoin.java
@@ -195,7 +195,9 @@
                 LogicalVariable replacementVar = context.newVar();
                 assignVars.add(replacementVar);
                 assignExprs.add(new MutableObject<ILogicalExpression>(funcExpr));
-                funcExprRef.setValue(new VariableReferenceExpression(replacementVar));
+                VariableReferenceExpression replacementVarRef = new VariableReferenceExpression(replacementVar);
+                replacementVarRef.setSourceLocation(funcExpr.getSourceLocation());
+                funcExprRef.setValue(replacementVarRef);
                 funcIter.remove();
             }
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
index 5675a03..1ce6329 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
@@ -208,6 +208,7 @@
         }
 
         ProjectOperator pi2 = new ProjectOperator(new ArrayList<LogicalVariable>(toPush));
+        pi2.setSourceLocation(op.getSourceLocation());
         pi2.getInputs().add(new MutableObject<ILogicalOperator>(op));
         opRef.setValue(pi2);
         pi2.setExecutionMode(op.getExecutionMode());
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index d277043..4869761 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -356,9 +356,11 @@
                                 new IndexBulkloadPOperator(primaryKeys, secondaryKeys, additionalFilteringKeys,
                                         opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
                     } else {
+                        LogicalVariable upsertIndicatorVar = null;
                         List<LogicalVariable> prevSecondaryKeys = null;
                         LogicalVariable prevAdditionalFilteringKey = null;
                         if (opInsDel.getOperation() == Kind.UPSERT) {
+                            upsertIndicatorVar = getKey(opInsDel.getUpsertIndicatorExpr().getValue());
                             prevSecondaryKeys = new ArrayList<LogicalVariable>();
                             getKeys(opInsDel.getPrevSecondaryKeyExprs(), prevSecondaryKeys);
                             if (opInsDel.getPrevAdditionalFilteringExpression() != null) {
@@ -369,7 +371,7 @@
                         }
                         op.setPhysicalOperator(new IndexInsertDeleteUpsertPOperator(primaryKeys, secondaryKeys,
                                 additionalFilteringKeys, opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex(),
-                                prevSecondaryKeys, prevAdditionalFilteringKey,
+                                upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey,
                                 opInsDel.getNumberOfAdditionalNonFilteringFields()));
                     }
                     break;
@@ -407,14 +409,17 @@
 
     private static void getKeys(List<Mutable<ILogicalExpression>> keyExpressions, List<LogicalVariable> keys) {
         for (Mutable<ILogicalExpression> kExpr : keyExpressions) {
-            ILogicalExpression e = kExpr.getValue();
-            if (e.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-                throw new NotImplementedException();
-            }
-            keys.add(((VariableReferenceExpression) e).getVariableReference());
+            keys.add(getKey(kExpr.getValue()));
         }
     }
 
+    private static LogicalVariable getKey(ILogicalExpression keyExpression) {
+        if (keyExpression.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            throw new NotImplementedException();
+        }
+        return ((VariableReferenceExpression) keyExpression).getVariableReference();
+    }
+
     private static LogicalVariable getKeysAndLoad(Mutable<ILogicalExpression> payloadExpr,
             List<Mutable<ILogicalExpression>> keyExpressions, List<LogicalVariable> keys) {
         LogicalVariable payload;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
index 4cd15a0..2ce0e63 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateSubplanRule.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class EliminateSubplanRule implements IAlgebraicRewriteRule {
 
@@ -101,6 +102,7 @@
     private void elimSubplanOverEts(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx)
             throws AlgebricksException {
         SubplanOperator subplan = (SubplanOperator) opRef.getValue();
+        SourceLocation sourceLoc = subplan.getSourceLocation();
         for (ILogicalPlan p : subplan.getNestedPlans()) {
             for (Mutable<ILogicalOperator> r : p.getRoots()) {
                 OperatorManipulationUtil.ntsToEts(r, ctx);
@@ -117,6 +119,7 @@
                 } else {
                     InnerJoinOperator j =
                             new InnerJoinOperator(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+                    j.setSourceLocation(sourceLoc);
                     j.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
                     j.getInputs().add(r);
                     ctx.setOutputTypeEnvironment(j, j.computeOutputTypeEnvironment(ctx));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index 5118bf3..c1e613b 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -62,6 +62,7 @@
 import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * The rule searches for SUBPLAN operator with a optional PROJECT operator and
@@ -201,8 +202,8 @@
             //retain the intersection
             pkVars.retainAll(producedVars);
         }
-        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Found FD for introducing group-by: " + pkVars);
+        if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Found FD for introducing group-by: " + pkVars);
         }
 
         Mutable<ILogicalOperator> rightRef = join.getInputs().get(1);
@@ -236,26 +237,35 @@
             default:
                 break;
         }
+
+        SourceLocation sourceLoc = subplan.getSourceLocation();
+
         if (testForNull == null) {
             testForNull = context.newVar();
             AssignOperator tmpAsgn =
                     new AssignOperator(testForNull, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+            tmpAsgn.setSourceLocation(sourceLoc);
             tmpAsgn.getInputs().add(new MutableObject<ILogicalOperator>(rightRef.getValue()));
             rightRef.setValue(tmpAsgn);
             context.computeAndSetTypeEnvironmentForOperator(tmpAsgn);
         }
 
         IFunctionInfo finfoEq = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.IS_MISSING);
-        ILogicalExpression isNullTest = new ScalarFunctionCallExpression(finfoEq,
+        ScalarFunctionCallExpression isNullTest = new ScalarFunctionCallExpression(finfoEq,
                 new MutableObject<ILogicalExpression>(new VariableReferenceExpression(testForNull)));
+        isNullTest.setSourceLocation(sourceLoc);
         IFunctionInfo finfoNot = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.NOT);
         ScalarFunctionCallExpression nonNullTest =
                 new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isNullTest));
+        nonNullTest.setSourceLocation(sourceLoc);
         SelectOperator selectNonNull =
                 new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
+        selectNonNull.setSourceLocation(sourceLoc);
         GroupByOperator g = new GroupByOperator();
+        g.setSourceLocation(sourceLoc);
         Mutable<ILogicalOperator> newSubplanRef = new MutableObject<ILogicalOperator>(subplan);
         NestedTupleSourceOperator nts = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(g));
+        nts.setSourceLocation(sourceLoc);
         opRef.setValue(g);
         selectNonNull.getInputs().add(new MutableObject<ILogicalOperator>(nts));
 
@@ -318,10 +328,12 @@
     private Map<LogicalVariable, LogicalVariable> buildVarExprList(Collection<LogicalVariable> vars,
             IOptimizationContext context, GroupByOperator g,
             List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException {
+        SourceLocation sourceLoc = g.getSourceLocation();
         Map<LogicalVariable, LogicalVariable> m = new HashMap<LogicalVariable, LogicalVariable>();
         for (LogicalVariable ov : vars) {
             LogicalVariable newVar = context.newVar();
             ILogicalExpression varExpr = new VariableReferenceExpression(newVar);
+            ((VariableReferenceExpression) varExpr).setSourceLocation(sourceLoc);
             outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov,
                     new MutableObject<ILogicalExpression>(varExpr)));
             for (ILogicalPlan p : g.getNestedPlans()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
index f809e96..e37be1b 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
@@ -88,6 +88,7 @@
                 }
                 ntsRef.setValue(opUnder.getValue());
                 LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+                loj.setSourceLocation(join.getSourceLocation());
                 loj.getInputs().add(leftRef);
                 loj.getInputs().add(rightRef);
                 opRef.setValue(loj);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
index d9acf53..53b8eff 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/NestedSubplanToJoinRule.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
@@ -103,20 +104,24 @@
                 continue;
             }
 
+            SourceLocation sourceLoc = subplan.getSourceLocation();
+
             /**
              * Expends the input and roots into a DAG of nested loop joins.
              * Though joins should be left-outer joins, a left-outer join with condition TRUE is equivalent to an inner join.
              **/
             Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
             Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
-            ILogicalOperator join =
+            InnerJoinOperator join =
                     new InnerJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput), nestedRootRef);
+            join.setSourceLocation(sourceLoc);
 
             /** rewrite the nested tuple source to be empty tuple source */
             rewriteNestedTupleSource(nestedRootRef, context);
 
             for (int i = 1; i < nestedRoots.size(); i++) {
                 join = new InnerJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
+                join.setSourceLocation(sourceLoc);
             }
             op1.getInputs().get(index).setValue(join);
             context.computeAndSetTypeEnvironmentForOperator(join);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
index 555d468..6306338 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -70,15 +70,20 @@
                         setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideLeft, sideRight, context);
                         break;
                     case LEFT:
-                        Mutable<ILogicalOperator> opRef0 = op.getInputs().get(0);
-                        Mutable<ILogicalOperator> opRef1 = op.getInputs().get(1);
-                        ILogicalOperator tmp = opRef0.getValue();
-                        opRef0.setValue(opRef1.getValue());
-                        opRef1.setValue(tmp);
-                        setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideRight, sideLeft, context);
+                        if (op.getJoinKind() == AbstractBinaryJoinOperator.JoinKind.INNER) {
+                            Mutable<ILogicalOperator> opRef0 = op.getInputs().get(0);
+                            Mutable<ILogicalOperator> opRef1 = op.getInputs().get(1);
+                            ILogicalOperator tmp = opRef0.getValue();
+                            opRef0.setValue(opRef1.getValue());
+                            opRef1.setValue(tmp);
+                            setHashJoinOp(op, JoinPartitioningType.BROADCAST, sideRight, sideLeft, context);
+                        } else {
+                            setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+                        }
                         break;
                     default:
-                        setHashJoinOp(op, JoinPartitioningType.PAIRWISE, sideLeft, sideRight, context);
+                        // This should never happen
+                        throw new IllegalStateException(side.toString());
                 }
             }
         } else {
@@ -109,18 +114,18 @@
         ILogicalOperator opBuild = op.getInputs().get(1).getValue();
         LogicalPropertiesVisitor.computeLogicalPropertiesDFS(opBuild, context);
         ILogicalPropertiesVector v = context.getLogicalPropertiesVector(opBuild);
-        boolean loggerDebugEnabled = AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled();
-        if (loggerDebugEnabled) {
+        boolean loggerTraceEnabled = AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled();
+        if (loggerTraceEnabled) {
             AlgebricksConfig.ALGEBRICKS_LOGGER
-                    .debug("// HybridHashJoin inner branch -- Logical properties for " + opBuild + ": " + v + "\n");
+                    .trace("// HybridHashJoin inner branch -- Logical properties for " + opBuild + ": " + v + "\n");
         }
         if (v != null) {
             int size2 = v.getMaxOutputFrames();
             HybridHashJoinPOperator hhj = (HybridHashJoinPOperator) op.getPhysicalOperator();
             if (size2 > 0 && size2 * hhj.getFudgeFactor() <= hhj.getMemSizeInFrames()) {
-                if (loggerDebugEnabled) {
+                if (loggerTraceEnabled) {
                     AlgebricksConfig.ALGEBRICKS_LOGGER
-                            .debug("// HybridHashJoin inner branch " + opBuild + " fits in memory\n");
+                            .trace("// HybridHashJoin inner branch " + opBuild + " fits in memory\n");
                 }
                 // maintains the local properties on the probe side
                 op.setPhysicalOperator(
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index f90de81..82b6f9c 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -22,7 +22,11 @@
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IPushRuntimeFactory extends Serializable {
     IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException;
+
+    default void setSourceLocation(SourceLocation sourceLoc) {
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
index 26cc5cb..c55d0f3 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/evaluators/ConstantEvalFactory.java
@@ -34,6 +34,10 @@
         this.value = value;
     }
 
+    public byte[] getValue() {
+        return value;
+    }
+
     @Override
     public String toString() {
         return "Constant";
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
index e99b61b..1f9cb91 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/AggregateRuntimeFactory.java
@@ -67,7 +67,6 @@
             private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(aggregs.length);
 
             private boolean first = true;
-            private boolean isOpen = false;
 
             @Override
             public void open() throws HyracksDataException {
@@ -81,8 +80,7 @@
                 for (int i = 0; i < aggregFactories.length; i++) {
                     aggregs[i].init();
                 }
-                isOpen = true;
-                writer.open();
+                super.open();
             }
 
             @Override
@@ -121,14 +119,6 @@
                     aggregs[f].step(tupleRef);
                 }
             }
-
-            @Override
-            public void fail() throws HyracksDataException {
-                failed = true;
-                if (isOpen) {
-                    writer.fail();
-                }
-            }
         };
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 0a578f6..b260a8a 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -150,8 +150,8 @@
                     ArrayTupleBuilder tb = outputWriter.getTupleBuilder();
                     byte[] data = tb.getByteArray();
                     if (data.length > memoryBudget) {
-                        throw HyracksDataException.create(ErrorCode.GROUP_BY_MEMORY_BUDGET_EXCEEDS, data.length,
-                                memoryBudget);
+                        throw HyracksDataException.create(ErrorCode.GROUP_BY_MEMORY_BUDGET_EXCEEDS, sourceLoc,
+                                data.length, memoryBudget);
                     }
                 }
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
index 75b2fb2..f057515 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -37,11 +37,11 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class NestedPlansRunningAggregatorFactory implements IAggregatorDescriptorFactory {
+public class NestedPlansRunningAggregatorFactory extends AbstractAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private final AlgebricksPipeline[] subplans;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index a7468a7..71b44d3 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -51,25 +52,20 @@
 
     @Override
     public void close() throws HyracksDataException {
-        HyracksDataException closeException = null;
+        if (!isOpen) {
+            return;
+        }
+        Throwable closeException = null;
         try {
             flushIfNotFailed();
         } catch (Exception e) {
-            closeException = HyracksDataException.create(e);
-            writer.fail();
+            closeException = e;
+            fail(closeException);
         } finally {
-            try {
-                writer.close();
-            } catch (Exception e) {
-                if (closeException == null) {
-                    closeException = HyracksDataException.create(e);
-                } else {
-                    closeException.addSuppressed(e);
-                }
-            }
+            closeException = CleanupUtils.close(writer, closeException);
         }
         if (closeException != null) {
-            throw closeException;
+            throw HyracksDataException.create(closeException);
         }
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 7b3fb46..525e452 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -19,11 +19,10 @@
 package org.apache.hyracks.algebricks.runtime.operators.base;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRuntimeFactory {
+public abstract class AbstractOneInputOneOutputRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputPushRuntime.java
index 5cced8d..c7d2d94 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputPushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputPushRuntime.java
@@ -27,6 +27,7 @@
     protected IFrameWriter writer;
     protected RecordDescriptor outputRecordDesc;
     protected boolean failed;
+    protected boolean isOpen;
 
     @Override
     public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
@@ -35,8 +36,24 @@
     }
 
     @Override
+    public void open() throws HyracksDataException {
+        isOpen = true;
+        writer.open();
+    }
+
+    @Override
     public void fail() throws HyracksDataException {
         failed = true;
-        writer.fail();
+        if (isOpen) {
+            writer.fail();
+        }
+    }
+
+    protected void fail(Throwable failure) {
+        try {
+            fail();
+        } catch (Throwable th) {
+            failure.addSuppressed(th);
+        }
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputSourcePushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputSourcePushRuntime.java
index 35563e0..cccfd62 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputSourcePushRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractOneInputSourcePushRuntime.java
@@ -32,16 +32,6 @@
     }
 
     @Override
-    public void close() throws HyracksDataException {
-        // close is a no op since this operator completes operating in open()
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        // fail is a no op since if a failure happened, the operator would've already called fail() on downstream
-    }
-
-    @Override
     public void flush() throws HyracksDataException {
         // flush will never be called on this runtime
         throw new UnsupportedOperationException();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java
new file mode 100644
index 0000000..89f3696
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/AbstractPushRuntimeFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.hyracks.algebricks.runtime.operators.base;
+
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractPushRuntimeFactory implements IPushRuntimeFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    protected SourceLocation sourceLoc;
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
index f0e9406..1aceadc 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/base/SinkRuntimeFactory.java
@@ -21,11 +21,10 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class SinkRuntimeFactory implements IPushRuntimeFactory {
+public class SinkRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 159fde7..3cee12d 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -151,7 +151,8 @@
 
         @Override
         public void open() throws HyracksDataException {
-            writer.open();
+            // writer opened many times?
+            super.open();
             if (first) {
                 first = false;
                 initAccessAppendRef(ctx);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index f251bb7..2453029 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
 import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
@@ -69,7 +70,7 @@
 
             @Override
             public void open() throws HyracksDataException {
-                writer.open();
+                super.open();
                 if (frameSorter == null) {
                     IFrameBufferManager manager = new VariableFrameMemoryManager(
                             new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY),
@@ -87,11 +88,22 @@
 
             @Override
             public void close() throws HyracksDataException {
-                try {
-                    frameSorter.sort();
-                    frameSorter.flush(writer);
-                } finally {
-                    writer.close();
+                Throwable failure = null;
+                if (isOpen) {
+                    try {
+                        if (!failed) {
+                            frameSorter.sort();
+                            frameSorter.flush(writer);
+                        }
+                    } catch (Throwable th) {
+                        failure = th;
+                        fail(th);
+                    } finally {
+                        failure = CleanupUtils.close(writer, failure);
+                    }
+                }
+                if (failure != null) {
+                    throw HyracksDataException.create(failure);
                 }
             }
         };
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
index aefc99d..5b36c5f 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
@@ -96,7 +96,6 @@
             private IScalarEvaluator[] eval = new IScalarEvaluator[evalFactories.length];
             private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
             private boolean first = true;
-            private boolean isOpen = false;
             private int tupleIndex = 0;
 
             @Override
@@ -109,15 +108,7 @@
                         eval[i] = evalFactories[i].createScalarEvaluator(ctx);
                     }
                 }
-                isOpen = true;
-                writer.open();
-            }
-
-            @Override
-            public void close() throws HyracksDataException {
-                if (isOpen) {
-                    super.close();
-                }
+                super.open();
             }
 
             @Override
@@ -172,14 +163,7 @@
                         }
                     }
                 } catch (HyracksDataException e) {
-                    throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, tupleIndex);
-                }
-            }
-
-            @Override
-            public void fail() throws HyracksDataException {
-                if (isOpen) {
-                    super.fail();
+                    throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, tupleIndex);
                 }
             }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
index 67f4a77..9ca3cd6 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -19,15 +19,15 @@
 package org.apache.hyracks.algebricks.runtime.operators.std;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputSourcePushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 
-public class EmptyTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+public class EmptyTupleSourceRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -48,7 +48,7 @@
 
             @Override
             public void open() throws HyracksDataException {
-                writer.open();
+                super.open();
                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                     throw new IllegalStateException();
                 }
@@ -56,13 +56,10 @@
             }
 
             @Override
-            public void fail() throws HyracksDataException {
-                writer.fail();
-            }
-
-            @Override
             public void close() throws HyracksDataException {
-                writer.close();
+                if (isOpen) {
+                    writer.close();
+                }
             }
 
             @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index 8e64092..832cb22 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -21,12 +21,12 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
+public class NestedTupleSourceRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
@@ -49,11 +49,6 @@
             initAccessAppend(ctx);
         }
 
-        @Override
-        public void open() throws HyracksDataException {
-            writer.open();
-        }
-
         public void writeTuple(ByteBuffer inputBuffer, int tIndex) throws HyracksDataException {
             tAccess.reset(inputBuffer);
             appendTupleToFrame(tIndex);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
index 8a06ecf..7d6f851 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/PrinterRuntimeFactory.java
@@ -21,12 +21,12 @@
 import org.apache.hyracks.algebricks.data.IAWriter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 
-public class PrinterRuntimeFactory implements IPushRuntimeFactory {
+public class PrinterRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
index 33b7725..ca58d4d 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
@@ -89,7 +89,6 @@
             private final IRunningAggregateEvaluator[] raggs = new IRunningAggregateEvaluator[runningAggregates.length];
             private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
             private boolean first = true;
-            private boolean isOpen = false;
 
             @Override
             public void open() throws HyracksDataException {
@@ -104,22 +103,7 @@
                 for (int i = 0; i < runningAggregates.length; i++) {
                     raggs[i].init();
                 }
-                isOpen = true;
-                writer.open();
-            }
-
-            @Override
-            public void close() throws HyracksDataException {
-                if (isOpen) {
-                    super.close();
-                }
-            }
-
-            @Override
-            public void fail() throws HyracksDataException {
-                if (isOpen) {
-                    writer.fail();
-                }
+                super.open();
             }
 
             @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
index d41b464..eae8178 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SinkWriterRuntimeFactory.java
@@ -28,12 +28,12 @@
 import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class SinkWriterRuntimeFactory implements IPushRuntimeFactory {
+public class SinkWriterRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
index 59df402..aca5bf1 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
@@ -72,7 +72,7 @@
 
             @Override
             public void open() throws HyracksDataException {
-                writer.open();
+                super.open();
                 if (evalMaxObjects == null) {
                     initAccessAppendRef(ctx);
                     evalMaxObjects = maxObjectsEvalFactory.createScalarEvaluator(ctx);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
index a8ca082..713a99c 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamProjectRuntimeFactory.java
@@ -53,7 +53,7 @@
 
             @Override
             public void open() throws HyracksDataException {
-                writer.open();
+                super.open();
                 if (first) {
                     first = false;
                     initAccessAppend(ctx);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
index 171544d..933e640 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -92,8 +92,7 @@
                     initAccessAppendFieldRef(ctx);
                     eval = cond.createScalarEvaluator(ctx);
                 }
-                writer.open();
-
+                super.open();
                 //prepare nullTupleBuilder
                 if (retainMissing && missingWriter == null) {
                     missingWriter = missingWriterFactory.createMissingWriter();
@@ -105,15 +104,6 @@
             }
 
             @Override
-            public void close() throws HyracksDataException {
-                try {
-                    flushIfNotFailed();
-                } finally {
-                    writer.close();
-                }
-            }
-
-            @Override
             public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
                 tAccess.reset(buffer);
                 int nTuple = tAccess.getTupleCount();
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
index 53974b2..7e5c346 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StringStreamingRuntimeFactory.java
@@ -43,6 +43,9 @@
     private char fieldDelimiter;
     private ITupleParserFactory parserFactory;
 
+    /*
+     * NOTE: This operator doesn't follow the IFrameWriter protocol
+     */
     public StringStreamingRuntimeFactory(String command, IPrinterFactory[] printerFactories, char fieldDelimiter,
             ITupleParserFactory parserFactory) {
         super(null);
@@ -129,7 +132,6 @@
                     first = false;
                     initAccessAppendRef(ctx);
                 }
-
                 try {
                     ITupleParser parser = parserFactory.createTupleParser(ctx);
                     process = Runtime.getRuntime().exec(command);
@@ -141,6 +143,7 @@
                             new DumpInStreamToPrintStream(process.getErrorStream(), System.err);
                     dumpStderr = new Thread(disps);
                     dumpStderr.start();
+                    super.open();
                 } catch (IOException e) {
                     throw HyracksDataException.create(e);
                 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 914f4a0..22189ac 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -94,7 +94,7 @@
 
             @Override
             public void open() throws HyracksDataException {
-                writer.open();
+                super.open();
                 if (tRef == null) {
                     initAccessAppendRef(ctx);
                 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
index 1706e59..e727551 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/union/MicroUnionAllRuntimeFactory.java
@@ -24,13 +24,13 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
-import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractPushRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class MicroUnionAllRuntimeFactory implements IPushRuntimeFactory {
+public class MicroUnionAllRuntimeFactory extends AbstractPushRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl b/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
index 7ce76de..077e7bc 100644
--- a/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
+++ b/hyracks-fullstack/hyracks-fullstack-license/src/main/licenses/templates/hyracks-notice.ftl
@@ -32,7 +32,7 @@
    <#list projects as p>
 ${p.name} (${p.groupId}:${p.artifactId}:${p.version})
        <#list p.locations as loc>
-- ${loc}${p.artifactId}-${p.version}.jar
+- ${loc}${p.jarName}
        </#list>
    </#list>
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index 3484e23..453bb7b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -80,7 +80,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplication.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplication.java
index 6bcdd8a..aa1021b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplication.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplication.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.api.application;
 
 import org.apache.hyracks.api.config.IConfigManager;
+import org.apache.hyracks.api.control.IGatekeeper;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
 
 public interface ICCApplication extends IApplication {
@@ -26,4 +27,7 @@
     IJobCapacityController getJobCapacityController();
 
     IConfigManager getConfigManager();
+
+    IGatekeeper getGatekeeper();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCServiceContext.java
index 94ebcfe..c0e9834 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCServiceContext.java
@@ -22,7 +22,6 @@
 
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
-import org.apache.hyracks.api.service.IControllerService;
 
 /**
  * Service Context at the Cluster Controller for an application.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplication.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplication.java
index af6cb92..919722d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplication.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplication.java
@@ -34,5 +34,5 @@
      */
     IFileDeviceResolver getFileDeviceResolver();
 
-    void onRegisterNode(CcId ccId) throws Exception;
+    void tasksCompleted(CcId ccId) throws Exception;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
index bc3d7a1..6effee3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
@@ -22,6 +22,7 @@
 import java.util.concurrent.ThreadFactory;
 
 import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
 import org.apache.hyracks.api.messages.IMessageBroker;
 import org.apache.hyracks.api.service.IControllerService;
@@ -55,4 +56,17 @@
 
     Object getApplicationContext();
 
+    /**
+     * Sets the IPersistedResourceRegistry that contains the mapping between classes and type ids used
+     * for serialization.
+     *
+     * @param persistedResourceRegistry
+     */
+    default void setPersistedResourceRegistry(IPersistedResourceRegistry persistedResourceRegistry) {
+        throw new UnsupportedOperationException();
+    }
+
+    default IPersistedResourceRegistry getPersistedResourceRegistry() {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannelMonitor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannelMonitor.java
index 52509d3..559f49a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannelMonitor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannelMonitor.java
@@ -19,9 +19,9 @@
 package org.apache.hyracks.api.channels;
 
 public interface IInputChannelMonitor {
-    public void notifyFailure(IInputChannel channel);
+    void notifyFailure(IInputChannel channel, int errorCode);
 
-    public void notifyDataAvailability(IInputChannel channel, int nFrames);
+    void notifyDataAvailability(IInputChannel channel, int nFrames);
 
-    public void notifyEndOfStream(IInputChannel channel);
+    void notifyEndOfStream(IInputChannel channel);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 23c41fe..36ce18f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -25,12 +25,12 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultSetId;
 
 public class HyracksClientInterfaceFunctions {
     public enum FunctionId {
@@ -39,12 +39,13 @@
         GET_JOB_STATUS,
         GET_JOB_INFO,
         START_JOB,
-        DISTRIBUTE_JOB,
-        DESTROY_JOB,
+        DEPLOY_JOB,
+        UNDEPLOY_JOB,
+        REDEPLOY_JOB,
         CANCEL_JOB,
-        GET_DATASET_DIRECTORY_SERIVICE_INFO,
-        GET_DATASET_RESULT_STATUS,
-        GET_DATASET_RESULT_LOCATIONS,
+        GET_RESULT_DIRECTORY_ADDRESS,
+        GET_RESULT_STATUS,
+        GET_RESULT_LOCATIONS,
         WAIT_FOR_COMPLETION,
         GET_NODE_CONTROLLERS_INFO,
         CLI_DEPLOY_BINARY,
@@ -107,6 +108,32 @@
         }
     }
 
+    public static class redeployJobSpecFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final byte[] acggfBytes;
+
+        private final DeployedJobSpecId deployedJobSpecId;
+
+        public redeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) {
+            this.deployedJobSpecId = deployedJobSpecId;
+            this.acggfBytes = acggfBytes;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REDEPLOY_JOB;
+        }
+
+        public byte[] getACGGFBytes() {
+            return acggfBytes;
+        }
+
+        public DeployedJobSpecId getDeployedJobSpecId() {
+            return deployedJobSpecId;
+        }
+    }
+
     public static class DeployJobSpecFunction extends Function {
         private static final long serialVersionUID = 1L;
 
@@ -118,7 +145,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DISTRIBUTE_JOB;
+            return FunctionId.DEPLOY_JOB;
         }
 
         public byte[] getACGGFBytes() {
@@ -159,7 +186,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_JOB;
+            return FunctionId.UNDEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
@@ -223,30 +250,30 @@
         }
     }
 
-    public static class GetDatasetDirectoryServiceInfoFunction extends Function {
+    public static class GetResultDirectoryAddressFunction extends Function {
         private static final long serialVersionUID = 1L;
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.GET_DATASET_DIRECTORY_SERIVICE_INFO;
+            return FunctionId.GET_RESULT_DIRECTORY_ADDRESS;
         }
     }
 
-    public static class GetDatasetResultStatusFunction extends Function {
+    public static class GetResultStatusFunction extends Function {
         private static final long serialVersionUID = 1L;
 
         private final JobId jobId;
 
         private final ResultSetId rsId;
 
-        public GetDatasetResultStatusFunction(JobId jobId, ResultSetId rsId) {
+        public GetResultStatusFunction(JobId jobId, ResultSetId rsId) {
             this.jobId = jobId;
             this.rsId = rsId;
         }
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.GET_DATASET_RESULT_STATUS;
+            return FunctionId.GET_RESULT_STATUS;
         }
 
         public JobId getJobId() {
@@ -258,16 +285,16 @@
         }
     }
 
-    public static class GetDatasetResultLocationsFunction extends Function {
+    public static class GetResultLocationsFunction extends Function {
         private static final long serialVersionUID = 1L;
 
         private final JobId jobId;
 
         private final ResultSetId rsId;
 
-        private final DatasetDirectoryRecord[] knownRecords;
+        private final ResultDirectoryRecord[] knownRecords;
 
-        public GetDatasetResultLocationsFunction(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords) {
+        public GetResultLocationsFunction(JobId jobId, ResultSetId rsId, ResultDirectoryRecord[] knownRecords) {
             this.jobId = jobId;
             this.rsId = rsId;
             this.knownRecords = knownRecords;
@@ -275,7 +302,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.GET_DATASET_RESULT_LOCATIONS;
+            return FunctionId.GET_RESULT_LOCATIONS;
         }
 
         public JobId getJobId() {
@@ -286,7 +313,7 @@
             return rsId;
         }
 
-        public DatasetDirectoryRecord[] getKnownRecords() {
+        public ResultDirectoryRecord[] getKnownRecords() {
             return knownRecords;
         }
     }
@@ -419,6 +446,7 @@
     }
 
     public static class ThreadDumpFunction extends Function {
+        private static final long serialVersionUID = 2956155746070390274L;
         private final String node;
 
         public ThreadDumpFunction(String node) {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index eddcaa5..63a32f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -98,16 +98,23 @@
     }
 
     @Override
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
-        HyracksClientInterfaceFunctions.UndeployJobSpecFunction sjf =
-                new HyracksClientInterfaceFunctions.UndeployJobSpecFunction(deployedJobSpecId);
-        return (DeployedJobSpecId) rpci.call(ipcHandle, sjf);
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception {
+        HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+                new HyracksClientInterfaceFunctions.redeployJobSpecFunction(deployedJobSpecId, acggfBytes);
+        rpci.call(ipcHandle, udjsf);
     }
 
     @Override
-    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
-        HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf =
-                new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+        HyracksClientInterfaceFunctions.UndeployJobSpecFunction sjf =
+                new HyracksClientInterfaceFunctions.UndeployJobSpecFunction(deployedJobSpecId);
+        rpci.call(ipcHandle, sjf);
+    }
+
+    @Override
+    public NetworkAddress getResultDirectoryAddress() throws Exception {
+        HyracksClientInterfaceFunctions.GetResultDirectoryAddressFunction gddsf =
+                new HyracksClientInterfaceFunctions.GetResultDirectoryAddressFunction();
         return (NetworkAddress) rpci.call(ipcHandle, gddsf);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
index f676d27..48c656f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
@@ -26,6 +26,11 @@
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpPut;
@@ -44,10 +49,16 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.topology.ClusterTopology;
+import org.apache.hyracks.api.util.InvokeUtil;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.ipc.api.RPCInterface;
 import org.apache.hyracks.ipc.impl.IPCSystem;
 import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.hyracks.util.InterruptibleAction;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 /**
  * Connection Class used by a Hyracks Client to interact with a Hyracks Cluster
@@ -56,6 +67,9 @@
  * @author vinayakb
  */
 public final class HyracksConnection implements IHyracksClientConnection {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private final String ccHost;
 
     private final int ccPort;
@@ -66,6 +80,15 @@
 
     private final ClusterControllerInfo ccInfo;
 
+    private volatile boolean running = false;
+
+    private volatile long reqId = 0L;
+
+    private final ExecutorService uninterruptibleExecutor =
+            Executors.newFixedThreadPool(2, r -> new Thread(r, "HyracksConnection Uninterrubtible thread: "));
+
+    private final BlockingQueue<UnInterruptibleRequest<?>> uninterruptibles = new ArrayBlockingQueue<>(1);
+
     /**
      * Constructor to create a connection to the Hyracks Cluster Controller.
      *
@@ -86,6 +109,8 @@
         hci = new HyracksClientInterfaceRemoteProxy(ipc.getReconnectingHandle(new InetSocketAddress(ccHost, ccPort)),
                 rpci);
         ccInfo = hci.getClusterControllerInfo();
+        uninterruptibleExecutor.execute(new UninterrubtileRequestHandler());
+        uninterruptibleExecutor.execute(new UninterrubtileHandlerWatcher());
     }
 
     @Override
@@ -95,7 +120,8 @@
 
     @Override
     public void cancelJob(JobId jobId) throws Exception {
-        hci.cancelJob(jobId);
+        CancelJobRequest request = new CancelJobRequest(jobId);
+        uninterruptiblySubmitAndExecute(request);
     }
 
     @Override
@@ -111,6 +137,13 @@
     }
 
     @Override
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception {
+        JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
+                new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
+        hci.redeployJobSpec(deployedJobSpecId, JavaSerializationUtils.serialize(jsacggf));
+    }
+
+    @Override
     public DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception {
         JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
                 new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
@@ -118,18 +151,19 @@
     }
 
     @Override
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
-        return hci.undeployJobSpec(deployedJobSpecId);
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+        hci.undeployJobSpec(deployedJobSpecId);
     }
 
     @Override
     public JobId startJob(DeployedJobSpecId deployedJobSpecId, Map<byte[], byte[]> jobParameters) throws Exception {
-        return hci.startJob(deployedJobSpecId, jobParameters);
+        StartDeployedJobRequest request = new StartDeployedJobRequest(deployedJobSpecId, jobParameters);
+        return interruptiblySubmitAndExecute(request);
     }
 
     @Override
     public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception {
-        return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
+        return startJob(null, acggf, jobFlags);
     }
 
     public DeployedJobSpecId deployJobSpec(IActivityClusterGraphGeneratorFactory acggf) throws Exception {
@@ -137,8 +171,8 @@
     }
 
     @Override
-    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
-        return hci.getDatasetDirectoryServiceInfo();
+    public NetworkAddress getResultDirectoryAddress() throws Exception {
+        return hci.getResultDirectoryAddress();
     }
 
     @Override
@@ -147,7 +181,7 @@
             hci.waitForCompletion(jobId);
         } catch (InterruptedException e) {
             // Cancels an on-going job if the current thread gets interrupted.
-            hci.cancelJob(jobId);
+            cancelJob(jobId);
             throw e;
         }
     }
@@ -225,7 +259,8 @@
     @Override
     public JobId startJob(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
             EnumSet<JobFlag> jobFlags) throws Exception {
-        return hci.startJob(deploymentId, JavaSerializationUtils.serialize(acggf), jobFlags);
+        StartJobRequest request = new StartJobRequest(deploymentId, acggf, jobFlags);
+        return interruptiblySubmitAndExecute(request);
     }
 
     @Override
@@ -262,4 +297,184 @@
     public boolean isConnected() {
         return hci.isConnected();
     }
+
+    private <T> T uninterruptiblySubmitAndExecute(UnInterruptibleRequest<T> request) throws Exception {
+        InvokeUtil.doUninterruptibly(() -> uninterruptibles.put(request));
+        return uninterruptiblyExecute(request);
+    }
+
+    private <T> T uninterruptiblyExecute(UnInterruptibleRequest<T> request) throws Exception {
+        InvokeUtil.doUninterruptibly(request);
+        return request.result();
+    }
+
+    private <T> T interruptiblySubmitAndExecute(UnInterruptibleRequest<T> request) throws Exception {
+        uninterruptibles.put(request);
+        return uninterruptiblyExecute(request);
+    }
+
+    private abstract class UnInterruptibleRequest<T> implements InterruptibleAction {
+        boolean completed = false;
+        boolean failed = false;
+        Throwable failure = null;
+        T response = null;
+
+        @SuppressWarnings("squid:S1181")
+        private final void handle() {
+            try {
+                response = doHandle();
+            } catch (Throwable th) {
+                failed = true;
+                failure = th;
+            } finally {
+                synchronized (this) {
+                    completed = true;
+                    notifyAll();
+                }
+            }
+        }
+
+        protected abstract T doHandle() throws Exception;
+
+        @Override
+        public final synchronized void run() throws InterruptedException {
+            while (!completed) {
+                wait();
+            }
+        }
+
+        public T result() throws Exception {
+            if (failed) {
+                if (failure instanceof Error) {
+                    throw (Error) failure;
+                }
+                throw (Exception) failure;
+            }
+            return response;
+        }
+    }
+
+    private class CancelJobRequest extends UnInterruptibleRequest<Void> {
+        final JobId jobId;
+
+        public CancelJobRequest(JobId jobId) {
+            this.jobId = jobId;
+        }
+
+        @Override
+        protected Void doHandle() throws Exception {
+            hci.cancelJob(jobId);
+            return null;
+        }
+
+        @Override
+        public String toString() {
+            return "CancelJobRequest: " + jobId.toString();
+        }
+
+    }
+
+    private class StartDeployedJobRequest extends UnInterruptibleRequest<JobId> {
+
+        private final DeployedJobSpecId deployedJobSpecId;
+        private final Map<byte[], byte[]> jobParameters;
+
+        public StartDeployedJobRequest(DeployedJobSpecId deployedJobSpecId, Map<byte[], byte[]> jobParameters) {
+            this.deployedJobSpecId = deployedJobSpecId;
+            this.jobParameters = jobParameters;
+        }
+
+        @Override
+        protected JobId doHandle() throws Exception {
+            return hci.startJob(deployedJobSpecId, jobParameters);
+        }
+
+    }
+
+    private class StartJobRequest extends UnInterruptibleRequest<JobId> {
+        private final DeploymentId deploymentId;
+        private final IActivityClusterGraphGeneratorFactory acggf;
+        private final EnumSet<JobFlag> jobFlags;
+
+        public StartJobRequest(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
+                EnumSet<JobFlag> jobFlags) {
+            this.deploymentId = deploymentId;
+            this.acggf = acggf;
+            this.jobFlags = jobFlags;
+        }
+
+        @Override
+        protected JobId doHandle() throws Exception {
+            if (deploymentId == null) {
+                return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
+            } else {
+                return hci.startJob(deploymentId, JavaSerializationUtils.serialize(acggf), jobFlags);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "StartJobRequest";
+        }
+
+    }
+
+    private class UninterrubtileRequestHandler implements Runnable {
+        @SuppressWarnings({ "squid:S2189", "squid:S2142" })
+        @Override
+        public void run() {
+            String nameBefore = Thread.currentThread().getName();
+            Thread.currentThread().setName(nameBefore + getClass().getSimpleName());
+            try {
+                while (true) {
+                    try {
+                        UnInterruptibleRequest<?> current = uninterruptibles.take();
+                        reqId++;
+                        running = true;
+                        current.handle();
+                    } catch (InterruptedException e) {
+                        LOGGER.log(Level.WARN, "Ignoring interrupt. This thread should never be interrupted.");
+                        continue;
+                    } finally {
+                        running = false;
+                    }
+                }
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+    }
+
+    public class UninterrubtileHandlerWatcher implements Runnable {
+        @Override
+        @SuppressWarnings({ "squid:S2189", "squid:S2142" })
+        public void run() {
+            String nameBefore = Thread.currentThread().getName();
+            Thread.currentThread().setName(nameBefore + getClass().getSimpleName());
+            try {
+                long currentReqId = 0L;
+                long currentTime = System.nanoTime();
+                while (true) {
+                    try {
+                        TimeUnit.MINUTES.sleep(1);
+                    } catch (InterruptedException e) {
+                        LOGGER.log(Level.WARN, "Ignoring interrupt. This thread should never be interrupted.");
+                        continue;
+                    }
+                    if (running) {
+                        if (reqId == currentReqId) {
+                            if (TimeUnit.NANOSECONDS.toMinutes(System.nanoTime() - currentTime) > 0) {
+                                ExitUtil.halt(ExitUtil.EC_FAILED_TO_PROCESS_UN_INTERRUPTIBLE_REQUEST);
+                            }
+                        } else {
+                            currentReqId = reqId;
+                            currentTime = System.nanoTime();
+                        }
+                    }
+                }
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 510a6b6..89f2ad4 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -99,13 +99,24 @@
     DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception;
 
     /**
+     * Update the JobSpec for a deployed job.
+     *
+     * @param deployedJobSpecId
+     *            The id of the deployed job spec
+     * @param jobSpec
+     *            Job Specification
+     * @throws Exception
+     */
+    void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception;
+
+    /**
      * Remove the deployed Job Spec
      *
      * @param deployedJobSpecId
      *            The id of the deployed job spec
      * @throws Exception
      */
-    DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+    void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
 
     /**
      * Used to run a deployed Job Spec by id
@@ -130,12 +141,12 @@
     JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception;
 
     /**
-     * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
+     * Gets the IP Address and port for the ResultDirectoryService wrapped in NetworkAddress
      *
      * @return {@link NetworkAddress}
      * @throws Exception
      */
-    NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+    NetworkAddress getResultDirectoryAddress() throws Exception;
 
     /**
      * Waits until the specified job has completed, either successfully or has
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index f0c7872..4cc47d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -45,9 +45,11 @@
 
     public DeployedJobSpecId deployJobSpec(byte[] acggfBytes) throws Exception;
 
-    public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+    public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception;
 
-    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+    public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+
+    public NetworkAddress getResultDirectoryAddress() throws Exception;
 
     public void waitForCompletion(JobId jobId) throws Exception;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
index 57f389f..aab6b2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
@@ -31,18 +31,18 @@
 
     private final NetworkAddress netAddress;
 
-    private final NetworkAddress datasetNetworkAddress;
+    private final NetworkAddress resultNetworkAddress;
 
     private final NetworkAddress messagingNetworkAddress;
 
     private final int numAvailableCores;
 
     public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
-            NetworkAddress datasetNetworkAddress, NetworkAddress messagingNetworkAddress, int numAvailableCores) {
+            NetworkAddress resultNetworkAddress, NetworkAddress messagingNetworkAddress, int numAvailableCores) {
         this.nodeId = nodeId;
         this.status = status;
         this.netAddress = netAddress;
-        this.datasetNetworkAddress = datasetNetworkAddress;
+        this.resultNetworkAddress = resultNetworkAddress;
         this.messagingNetworkAddress = messagingNetworkAddress;
         this.numAvailableCores = numAvailableCores;
     }
@@ -59,8 +59,8 @@
         return netAddress;
     }
 
-    public NetworkAddress getDatasetNetworkAddress() {
-        return datasetNetworkAddress;
+    public NetworkAddress getResultNetworkAddress() {
+        return resultNetworkAddress;
     }
 
     public NetworkAddress getMessagingNetworkAddress() {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/ActivityClusterGraphBuilder.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/ActivityClusterGraphBuilder.java
index e2cd923..e4a3388 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/ActivityClusterGraphBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/ActivityClusterGraphBuilder.java
@@ -50,7 +50,7 @@
                     for (IConnectorDescriptor conn : inputList) {
                         ActivityId inTask = jag.getProducerActivity(conn.getConnectorId());
                         if (!eqSet.contains(inTask)) {
-                            return Pair.<ActivityId, ActivityId> of(t, inTask);
+                            return Pair.of(t, inTask);
                         }
                     }
                 }
@@ -59,7 +59,7 @@
                     for (IConnectorDescriptor conn : outputList) {
                         ActivityId outTask = jag.getConsumerActivity(conn.getConnectorId());
                         if (!eqSet.contains(outTask)) {
-                            return Pair.<ActivityId, ActivityId> of(t, outTask);
+                            return Pair.of(t, outTask);
                         }
                     }
                 }
@@ -146,8 +146,8 @@
         }
         acg.addActivityClusters(acList);
 
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(acg.toJSON().asText());
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace(acg.toJSON().asText());
         }
         return acg;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
index 752a6b3..b70e0c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
@@ -24,7 +24,6 @@
 import java.io.Serializable;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.Arrays;
 
 import org.apache.hyracks.api.io.IWritable;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index bf42d0c..09b6506 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -23,7 +23,7 @@
 import java.util.concurrent.ExecutorService;
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.io.IWorkspaceFileFactory;
@@ -43,7 +43,7 @@
 
     ExecutorService getExecutorService();
 
-    IDatasetPartitionManager getDatasetPartitionManager();
+    IResultPartitionManager getResultPartitionManager();
 
     void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) throws Exception;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/control/IGatekeeper.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/control/IGatekeeper.java
new file mode 100644
index 0000000..43c8143
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/control/IGatekeeper.java
@@ -0,0 +1,30 @@
+/*
+ * 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.hyracks.api.control;
+
+@FunctionalInterface
+public interface IGatekeeper {
+    /**
+     * Indicates whether the supplied node is authorized to join this cluster
+     * @param nodeId
+     *          the node to consider
+     * @return <code>true</code> if the supplied node is authorized
+     */
+    boolean isAuthorized(String nodeId);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
index 9148d6b..801dc37 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -20,11 +20,11 @@
 
 import java.io.Serializable;
 
-import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.constraints.IConstraintAcceptor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Descriptor for operators in Hyracks.
@@ -37,7 +37,7 @@
      *
      * @return operator id
      */
-    public OperatorDescriptorId getOperatorId();
+    OperatorDescriptorId getOperatorId();
 
     /**
      * Sets the id of the operator.
@@ -51,21 +51,21 @@
      *
      * @return Number of inputs.
      */
-    public int getInputArity();
+    int getInputArity();
 
     /**
      * Returns the number of outputs out of this operator.
      *
      * @return Number of outputs.
      */
-    public int getOutputArity();
+    int getOutputArity();
 
     /**
      * Gets the output record descriptor
      *
      * @return Array of RecordDescriptor, one per output.
      */
-    public RecordDescriptor[] getOutputRecordDescriptors();
+    RecordDescriptor[] getOutputRecordDescriptors();
 
     /**
      * Contributes the activity graph that describes the behavior of this
@@ -74,7 +74,7 @@
      * @param builder
      *            - graph builder
      */
-    public void contributeActivities(IActivityGraphBuilder builder);
+    void contributeActivities(IActivityGraphBuilder builder);
 
     /**
      * Contributes any scheduling constraints imposed by this operator.
@@ -84,20 +84,30 @@
      * @param plan
      *            - Job Plan
      */
-    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext ccServiceCtx);
+    void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext ccServiceCtx);
 
     /**
      * Gets the display name.
      */
-    public String getDisplayName();
+    String getDisplayName();
 
     /**
      * Sets the display name.
      */
-    public void setDisplayName(String displayName);
+    void setDisplayName(String displayName);
+
+    /**
+     * Gets the source location.
+     */
+    SourceLocation getSourceLocation();
+
+    /**
+     * Sets the source location.
+     */
+    void setSourceLocation(SourceLocation sourceLoc);
 
     /**
      * Translates this operator descriptor to JSON.
      */
-    public ObjectNode toJSON();
+    ObjectNode toJSON();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
deleted file mode 100644
index c6d8f71..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.hyracks.api.dataflow;
-
-public interface IOperatorNodePullable extends IOpenableDataReader<Object[]> {
-    public void setDataReader(int index, IOpenableDataWriter<Object[]> writer);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryComparatorFactory.java
index 4cfa113..94d4eab 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryComparatorFactory.java
@@ -20,6 +20,8 @@
 
 import java.io.Serializable;
 
-public interface IBinaryComparatorFactory extends Serializable {
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+public interface IBinaryComparatorFactory extends Serializable, IJsonSerializable {
     public IBinaryComparator createBinaryComparator();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
deleted file mode 100644
index 4ec4c12..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.hyracks.api.dataflow.value;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IHashFunction<T> {
-    public int hash(T o) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
deleted file mode 100644
index b22dbe7..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.hyracks.api.dataflow.value;
-
-import java.io.Serializable;
-
-public interface IHashFunctionFactory<T> extends Serializable {
-    public IHashFunction<T> createHashFunction();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITypeTraits.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITypeTraits.java
index ec3ad3f..b1efee2 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITypeTraits.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITypeTraits.java
@@ -20,10 +20,12 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.io.IJsonSerializable;
+
 /**
  * Specify whether a type has fixed length and if so, what is the length
  */
-public interface ITypeTraits extends Serializable {
+public interface ITypeTraits extends Serializable, IJsonSerializable {
     /**
      * @return true if the type has a fixed length, false otherwise
      */
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetDirectoryRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetDirectoryRecord.java
deleted file mode 100644
index e47b1e2..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetDirectoryRecord.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.comm.NetworkAddress;
-
-public class DatasetDirectoryRecord implements Serializable {
-    public enum Status {
-        IDLE,
-        RUNNING,
-        SUCCESS,
-        FAILED
-    }
-
-    private static final long serialVersionUID = 1L;
-
-    private NetworkAddress address;
-
-    private boolean readEOS;
-
-    private Status status;
-
-    private boolean empty;
-
-    public DatasetDirectoryRecord() {
-        this.address = null;
-        this.readEOS = false;
-        this.status = Status.IDLE;
-    }
-
-    public void setNetworkAddress(NetworkAddress address) {
-        this.address = address;
-    }
-
-    public NetworkAddress getNetworkAddress() {
-        return address;
-    }
-
-    public void setEmpty(boolean empty) {
-        this.empty = empty;
-    }
-
-    public boolean isEmpty() {
-        return empty;
-    }
-
-    public void readEOS() {
-        this.readEOS = true;
-    }
-
-    public boolean hasReachedReadEOS() {
-        return readEOS;
-    }
-
-    public void start() {
-        updateStatus(Status.RUNNING);
-    }
-
-    public void writeEOS() {
-        updateStatus(Status.SUCCESS);
-    }
-
-    public void fail() {
-        status = Status.FAILED;
-    }
-
-    private void updateStatus(final DatasetDirectoryRecord.Status newStatus) {
-        // FAILED is a stable status
-        if (status != Status.FAILED) {
-            status = newStatus;
-        }
-    }
-
-    public Status getStatus() {
-        return status;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == this) {
-            return true;
-        }
-        if (!(o instanceof DatasetDirectoryRecord)) {
-            return false;
-        }
-        return address.equals(((DatasetDirectoryRecord) o).address);
-    }
-
-    @Override
-    public String toString() {
-        return String.valueOf(address) + " " + status + (empty ? " (empty)" : "") + (readEOS ? " (EOS)" : "");
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetJobRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetJobRecord.java
deleted file mode 100644
index 4e7ddda..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/DatasetJobRecord.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class DatasetJobRecord implements IDatasetStateRecord {
-    public enum State {
-        IDLE,
-        RUNNING,
-        SUCCESS,
-        FAILED
-    }
-
-    public static class Status implements Serializable {
-
-        private static final long serialVersionUID = 1L;
-
-        State state = State.IDLE;
-
-        private List<Exception> exceptions;
-
-        public State getState() {
-            return state;
-        }
-
-        void setState(State state) {
-            this.state = state;
-        }
-
-        public List<Exception> getExceptions() {
-            return exceptions;
-        }
-
-        void setExceptions(List<Exception> exceptions) {
-            this.exceptions = exceptions;
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder sb = new StringBuilder();
-            sb.append("{ \"state\": \"").append(state.name()).append("\"");
-            if (exceptions != null && !exceptions.isEmpty()) {
-                sb.append(", \"exceptions\": ");
-                List<String> msgs = new ArrayList<>();
-                exceptions.forEach(e -> msgs.add("\"" + e.getMessage() + "\""));
-                sb.append(Arrays.toString(msgs.toArray()));
-            }
-            sb.append(" }");
-            return sb.toString();
-        }
-    }
-
-    private static final long serialVersionUID = 1L;
-
-    private final long timestamp;
-
-    private Status status;
-
-    private Map<ResultSetId, ResultSetMetaData> resultSetMetadataMap = new HashMap<>();
-
-    public DatasetJobRecord() {
-        this.timestamp = System.nanoTime();
-        this.status = new Status();
-    }
-
-    private void updateState(State newStatus) {
-        // FAILED is a stable status
-        if (status.state != State.FAILED) {
-            status.setState(newStatus);
-        }
-    }
-
-    public void start() {
-        updateState(State.RUNNING);
-    }
-
-    public void success() {
-        updateState(State.SUCCESS);
-    }
-
-    public void fail(ResultSetId rsId, int partition) {
-        getOrCreateDirectoryRecord(rsId, partition).fail();
-    }
-
-    public void fail(List<Exception> exceptions) {
-        updateState(State.FAILED);
-        status.setExceptions(exceptions);
-    }
-
-    @Override
-    public long getTimestamp() {
-        return timestamp;
-    }
-
-    public Status getStatus() {
-        return status;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("{ \"status\": ").append(status.toString()).append(", ");
-        sb.append("\"timestamp\": ").append(timestamp).append(", ");
-        sb.append("\"resultsets\": ").append(Arrays.toString(resultSetMetadataMap.entrySet().toArray())).append(" }");
-        return sb.toString();
-    }
-
-    public void setResultSetMetaData(ResultSetId rsId, boolean orderedResult, int nPartitions)
-            throws HyracksDataException {
-        ResultSetMetaData rsMd = resultSetMetadataMap.get(rsId);
-        if (rsMd == null) {
-            resultSetMetadataMap.put(rsId, new ResultSetMetaData(nPartitions, orderedResult));
-        } else if (rsMd.getOrderedResult() != orderedResult || rsMd.getRecords().length != nPartitions) {
-            throw HyracksDataException.create(ErrorCode.INCONSISTENT_RESULT_METADATA, rsId.toString());
-        }
-        //TODO(tillw) throwing a HyracksDataException here hangs the execution tests
-    }
-
-    public ResultSetMetaData getResultSetMetaData(ResultSetId rsId) {
-        return resultSetMetadataMap.get(rsId);
-    }
-
-    public synchronized DatasetDirectoryRecord getOrCreateDirectoryRecord(ResultSetId rsId, int partition) {
-        DatasetDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
-        if (records[partition] == null) {
-            records[partition] = new DatasetDirectoryRecord();
-        }
-        return records[partition];
-    }
-
-    public synchronized DatasetDirectoryRecord getDirectoryRecord(ResultSetId rsId, int partition)
-            throws HyracksDataException {
-        DatasetDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
-        if (records[partition] == null) {
-            throw HyracksDataException.create(ErrorCode.RESULT_NO_RECORD, partition, rsId);
-        }
-        return records[partition];
-    }
-
-    public synchronized void updateState(ResultSetId rsId) {
-        int successCount = 0;
-        DatasetDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
-        for (DatasetDirectoryRecord record : records) {
-            if ((record != null) && (record.getStatus() == DatasetDirectoryRecord.Status.SUCCESS)) {
-                successCount++;
-            }
-        }
-        if (successCount == records.length) {
-            success();
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetManager.java
deleted file mode 100644
index a0c1f78..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetManager.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import java.util.Set;
-
-import org.apache.hyracks.api.job.JobId;
-
-public interface IDatasetManager {
-
-    Set<JobId> getJobIds();
-
-    IDatasetStateRecord getState(JobId jobId);
-
-    void sweep(JobId jobId);
-
-    /**
-     * Removes all references and deletes persisted files for
-     * all expired datasets.
-     */
-    void sweepExpiredDatasets();
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java
deleted file mode 100644
index b1e203f..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetPartitionManager.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.job.JobId;
-
-public interface IDatasetPartitionManager extends IDatasetManager {
-    IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
-            boolean asyncMode, int partition, int nPartitions, long maxReads) throws HyracksException;
-
-    void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
-            boolean orderedResult, boolean emptyResult) throws HyracksException;
-
-    void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
-
-    void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition, IFrameWriter noc)
-            throws HyracksException;
-
-    void removePartition(JobId jobId, ResultSetId resultSetId, int partition);
-
-    void abortReader(JobId jobId);
-
-    void close();
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetStateRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetStateRecord.java
deleted file mode 100644
index d18e6cf..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IDatasetStateRecord.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-public interface IDatasetStateRecord {
-    public long getTimestamp();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDataset.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDataset.java
deleted file mode 100644
index 0fa5c75..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDataset.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-
-public interface IHyracksDataset {
-    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
deleted file mode 100644
index 200f5e5..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.job.JobId;
-
-public interface IHyracksDatasetDirectoryServiceConnection {
-    /**
-     * Gets the result status for the given result set.
-     *
-     * @param jobId
-     *            ID of the job
-     * @param rsId
-     *            ID of the result set
-     * @return {@link Status}
-     * @throws Exception
-     */
-    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
-
-    /**
-     * Gets the IP Addresses and ports for the partition generating the result for each location.
-     *
-     * @param jobId
-     *            ID of the job
-     * @param rsId
-     *            ID of the result set
-     * @param knownRecords
-     *            Locations that are already known to the client
-     * @return {@link NetworkAddress[]}
-     * @throws Exception
-     */
-    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords) throws Exception;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
deleted file mode 100644
index 56c0af9..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.job.JobId;
-
-public interface IHyracksDatasetDirectoryServiceInterface {
-    /**
-     * Gets the result status for the given result set.
-     *
-     * @param jobId
-     *            ID of the job
-     * @param rsId
-     *            ID of the result set
-     * @return {@link Status}
-     * @throws Exception
-     */
-    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
-
-    /**
-     * Gets the IP Addresses and ports for the partition generating the result for each location.
-     *
-     * @param jobId
-     *            ID of the job
-     * @param rsId
-     *            ID of the result set
-     * @param knownRecords
-     *            Locations from the dataset directory that are already known to the client
-     * @return {@link NetworkAddress[]}
-     * @throws Exception
-     */
-    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords) throws Exception;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetReader.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetReader.java
deleted file mode 100644
index 858faaa..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/IHyracksDatasetReader.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IHyracksDatasetReader {
-    public Status getResultStatus();
-
-    public int read(IFrame frame) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetId.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetId.java
deleted file mode 100644
index 1e21e19..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetId.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import java.io.Serializable;
-
-public class ResultSetId implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    private final long id;
-
-    public ResultSetId(long id) {
-        this.id = id;
-    }
-
-    public long getId() {
-        return id;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) id;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == this) {
-            return true;
-        }
-        if (!(o instanceof ResultSetId)) {
-            return false;
-        }
-        return ((ResultSetId) o).id == id;
-    }
-
-    @Override
-    public String toString() {
-        return "RSID:" + id;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetMetaData.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetMetaData.java
deleted file mode 100644
index 8e9e3dc..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataset/ResultSetMetaData.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.hyracks.api.dataset;
-
-import java.util.Arrays;
-
-public class ResultSetMetaData {
-    private final DatasetDirectoryRecord[] records;
-    private final boolean ordered;
-
-    ResultSetMetaData(int len, boolean ordered) {
-        this.records = new DatasetDirectoryRecord[len];
-        this.ordered = ordered;
-    }
-
-    public boolean getOrderedResult() {
-        return ordered;
-    }
-
-    public DatasetDirectoryRecord[] getRecords() {
-        return records;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("{ordered: ").append(ordered).append(", records: ").append(Arrays.toString(records));
-        return sb.toString();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 51afac1..09193d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -121,7 +121,7 @@
     public static final int FOUND_MULTIPLE_TRANSACTIONS = 85;
     public static final int UNRECOGNIZED_INDEX_COMPONENT_FILE = 86;
     public static final int UNEQUAL_NUM_FILTERS_TREES = 87;
-    public static final int CANNOT_MODIFY_INDEX_DISK_IS_FULL = 88;
+    public static final int INDEX_NOT_MODIFIABLE = 88;
     public static final int GROUP_BY_MEMORY_BUDGET_EXCEEDS = 89;
     public static final int ILLEGAL_MEMORY_BUDGET = 90;
     public static final int TIMEOUT = 91;
@@ -148,6 +148,7 @@
     public static final int CANNOT_ADD_ELEMENT_TO_INVERTED_INDEX_SEARCH_RESULT = 112;
     public static final int UNDEFINED_INVERTED_LIST_MERGE_TYPE = 113;
     public static final int NODE_IS_NOT_ACTIVE = 114;
+    public static final int LOCAL_NETWORK_ERROR = 115;
 
     // Compilation error codes.
     public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index b9f84e8..54e4eaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -51,10 +51,21 @@
         return new HyracksDataException(cause);
     }
 
+    public static HyracksDataException create(int code, SourceLocation sourceLoc, Serializable... params) {
+        return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), null, sourceLoc,
+                params);
+    }
+
     public static HyracksDataException create(int code, Serializable... params) {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
     }
 
+    public static HyracksDataException create(int code, Throwable cause, SourceLocation sourceLoc,
+            Serializable... params) {
+        return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, sourceLoc,
+                params);
+    }
+
     public static HyracksDataException create(int code, Throwable cause, Serializable... params) {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
@@ -111,6 +122,11 @@
         super(component, errorCode, message, cause, null, params);
     }
 
+    public HyracksDataException(String component, int errorCode, String message, Throwable cause,
+            SourceLocation sourceLoc, Serializable... params) {
+        super(component, errorCode, message, cause, sourceLoc, null, params);
+    }
+
     public static HyracksDataException create(HyracksDataException e, String nodeId) {
         return new HyracksDataException(e.getComponent(), e.getErrorCode(), e.getMessage(), e.getCause(), nodeId,
                 e.getStackTrace(), e.getParams());
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index 210779e..69601cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -20,10 +20,11 @@
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Objects;
 
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
-public class HyracksException extends IOException {
+public class HyracksException extends IOException implements IFormattedException {
     private static final long serialVersionUID = 1L;
 
     public static final int UNKNOWN = 0;
@@ -31,6 +32,7 @@
     private final int errorCode;
     private final Serializable[] params;
     private final String nodeId;
+    private final SourceLocation sourceLoc;
     private transient volatile String msgCache;
 
     public static HyracksException create(Throwable cause) {
@@ -59,9 +61,10 @@
         return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
 
-    public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
-            Serializable... params) {
+    public HyracksException(String component, int errorCode, String message, Throwable cause, SourceLocation sourceLoc,
+            String nodeId, Serializable... params) {
         super(message, cause);
+        this.sourceLoc = sourceLoc;
         this.component = component;
         this.errorCode = errorCode;
         this.nodeId = nodeId;
@@ -73,7 +76,7 @@
      */
     @Deprecated
     public HyracksException(String message) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
     }
 
     /**
@@ -81,7 +84,7 @@
      */
     @Deprecated
     protected HyracksException(Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, String.valueOf(cause), cause, null);
+        this(ErrorMessageUtil.NONE, UNKNOWN, ErrorMessageUtil.getCauseMessage(cause), cause, (Serializable[]) null);
     }
 
     /**
@@ -97,7 +100,7 @@
     }
 
     public HyracksException(Throwable cause, int errorCode, Serializable... params) {
-        this(ErrorMessageUtil.NONE, errorCode, String.valueOf(cause), cause, null, params);
+        this(ErrorMessageUtil.NONE, errorCode, ErrorMessageUtil.getCauseMessage(cause), cause, null, params);
     }
 
     public HyracksException(String component, int errorCode, String message, Serializable... params) {
@@ -105,17 +108,24 @@
     }
 
     public HyracksException(String component, int errorCode, Throwable cause, Serializable... params) {
-        this(component, errorCode, String.valueOf(cause), cause, null, params);
+        this(component, errorCode, ErrorMessageUtil.getCauseMessage(cause), cause, null, params);
     }
 
     public HyracksException(String component, int errorCode, String message, Throwable cause, Serializable... params) {
         this(component, errorCode, message, cause, null, params);
     }
 
+    public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
+            Serializable... params) {
+        this(component, errorCode, message, cause, null, nodeId, params);
+    }
+
+    @Override
     public String getComponent() {
         return component;
     }
 
+    @Override
     public int getErrorCode() {
         return errorCode;
     }
@@ -128,11 +138,25 @@
         return nodeId;
     }
 
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
     @Override
     public String getMessage() {
         if (msgCache == null) {
-            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params);
+            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params);
         }
         return msgCache;
     }
+
+    public boolean matches(String component, int errorCode) {
+        Objects.requireNonNull(component, "component");
+        return component.equals(this.component) && errorCode == this.errorCode;
+    }
+
+    @Override
+    public String toString() {
+        return getLocalizedMessage();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
new file mode 100644
index 0000000..994915f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hyracks.api.exceptions;
+
+public interface IFormattedException {
+
+    /**
+     * Gets the component of this {@link IFormattedException}
+     *
+     * @return the exception component
+     */
+    String getComponent();
+
+    /**
+     * Gets the error code of this {@link IFormattedException}
+     *
+     * @return the error code
+     */
+    int getErrorCode();
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
new file mode 100644
index 0000000..fd3992c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hyracks.api.exceptions;
+
+import java.io.Serializable;
+
+public final class SourceLocation implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int line;
+
+    private final int column;
+
+    public SourceLocation(int line, int column) {
+        this.line = line;
+        this.column = column;
+    }
+
+    public int getLine() {
+        return line;
+    }
+
+    public int getColumn() {
+        return column;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
index 0375e9e..4ded855 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/FileReference.java
@@ -20,6 +20,7 @@
 
 import java.io.File;
 import java.io.Serializable;
+import java.util.Date;
 
 /**
  * A device handle and a relative path.
@@ -31,6 +32,7 @@
     private final File file;
     private final IODeviceHandle dev;
     private final String path;
+    private long registrationTime = 0L;
 
     public FileReference(IODeviceHandle dev, String path) {
         file = new File(dev.getMount(), path);
@@ -90,4 +92,23 @@
     public FileReference getChild(String name) {
         return new FileReference(dev, path + File.separator + name);
     }
+
+    public void register() {
+        if (registrationTime != 0) {
+            throw new IllegalStateException(
+                    "File " + toString() + " was already registered at " + new Date(registrationTime));
+        }
+        registrationTime = System.currentTimeMillis();
+    }
+
+    public long registrationTime() {
+        return registrationTime;
+    }
+
+    public void unregister() {
+        if (registrationTime == 0) {
+            throw new IllegalStateException("File " + toString() + " wasn't registered before");
+        }
+        registrationTime = 0;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
index ff1e47f..962f826 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IIOManager.java
@@ -97,4 +97,11 @@
      * @throws HyracksDataException
      */
     FileReference createWorkspaceFile(String prefix) throws HyracksDataException;
+
+    /**
+     * Gets the total disk usage in bytes of this {@link IIOManager} io devices handles.
+     *
+     * @return the total disk usage in bytes
+     */
+    long getTotalDiskUsage();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IJsonSerializable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IJsonSerializable.java
new file mode 100644
index 0000000..c717835
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IJsonSerializable.java
@@ -0,0 +1,34 @@
+/*
+ * 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.hyracks.api.io;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public interface IJsonSerializable {
+    /**
+     * @param registry
+     * @return A JSON representation of an object as a JSON Node
+     * @throws HyracksDataException
+     */
+    default JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
new file mode 100644
index 0000000..38162c6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/IPersistedResourceRegistry.java
@@ -0,0 +1,49 @@
+/*
+ * 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.hyracks.api.io;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+/**
+ * A IPersistedResourceRegistry is responsible for maintaining a mapping between classes and type ids
+ * that are used for serialization.
+ */
+public interface IPersistedResourceRegistry {
+
+    String TYPE_FIELD_ID = "@type";
+    String VERSION_FIELD_ID = "@version";
+    String CLASS_FIELD_ID = "@class";
+
+    /**
+     * @param clazz
+     * @param version
+     * @return A JsonObject with the registered type id in IPersistedResourceRegistry.
+     */
+    ObjectNode getClassIdentifier(Class<? extends IJsonSerializable> clazz, long version);
+
+    /**
+     * @param json
+     * @return A class object of the type id in {@code json}
+     * @throws HyracksDataException
+     */
+    IJsonSerializable deserialize(JsonNode json) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/ActivityCluster.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/ActivityCluster.java
index 94e9c74..22240cb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/ActivityCluster.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/ActivityCluster.java
@@ -110,9 +110,7 @@
         insertIntoIndexedMap(activityInputMap, consumerActivity.getActivityId(), consumerPort, connector);
         insertIntoIndexedMap(activityOutputMap, producerActivity.getActivityId(), producerPort, connector);
         connectorActivityMap.put(connector.getConnectorId(),
-                Pair.<Pair<IActivity, Integer>, Pair<IActivity, Integer>> of(
-                        Pair.<IActivity, Integer> of(producerActivity, producerPort),
-                        Pair.<IActivity, Integer> of(consumerActivity, consumerPort)));
+                Pair.of(Pair.of(producerActivity, producerPort), Pair.of(consumerActivity, consumerPort)));
         connectorRecordDescriptorMap.put(connector.getConnectorId(), recordDescriptor);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index c4c7320..829c550 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
@@ -40,7 +40,7 @@
 import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.job.resource.ClusterCapacity;
 import org.apache.hyracks.api.job.resource.IClusterCapacity;
 
@@ -144,9 +144,7 @@
         insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
         insertIntoIndexedMap(opOutputMap, producerOp.getOperatorId(), producerPort, conn);
         connectorOpMap.put(conn.getConnectorId(),
-                Pair.<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> of(
-                        Pair.<IOperatorDescriptor, Integer> of(producerOp, producerPort),
-                        Pair.<IOperatorDescriptor, Integer> of(consumerOp, consumerPort)));
+                Pair.of(Pair.of(producerOp, producerPort), Pair.of(consumerOp, consumerPort)));
     }
 
     public void setProperty(String name, Serializable value) {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java
index 712d575..6b03968 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.util.StorageUtil;
 
 public class ClusterCapacity implements IClusterCapacity {
+    private static final long serialVersionUID = 3487998182013966747L;
 
     private long aggregatedMemoryByteSize = 0;
     private int aggregatedCores = 0;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java
index 7902e7d..87988fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java
@@ -25,6 +25,7 @@
  * Specifies the capacity for computation on a particular node, i.e., a NCDriver process.
  */
 public class NodeCapacity implements Serializable {
+    private static final long serialVersionUID = -6124502740160006465L;
 
     // All memory for computations -- this is not changed during the lifetime of a running instance.
     private final long memoryByteSize;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/partitions/ResultSetPartitionId.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/partitions/ResultSetPartitionId.java
index 8a3e15a..8343fe0 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/partitions/ResultSetPartitionId.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/partitions/ResultSetPartitionId.java
@@ -20,7 +20,7 @@
 
 import java.io.Serializable;
 
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.job.JobId;
 
 public final class ResultSetPartitionId implements Serializable {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultDirectory.java
new file mode 100644
index 0000000..41b9d1a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultDirectory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hyracks.api.result;
+
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.job.JobId;
+
+public interface IResultDirectory {
+    /**
+     * Gets the result status for the given result set.
+     *
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @return {@link Status}
+     * @throws Exception
+     */
+    Status getResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+    /**
+     * Gets the IP Addresses and ports for the partition generating the result for each location.
+     *
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @param knownRecords
+     *            Locations that are already known to the client
+     * @return {@link ResultDirectoryRecord[]}
+     * @throws Exception
+     */
+    ResultDirectoryRecord[] getResultLocations(JobId jobId, ResultSetId rsId, ResultDirectoryRecord[] knownRecords)
+            throws Exception;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultManager.java
new file mode 100644
index 0000000..d45cf44
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultManager.java
@@ -0,0 +1,38 @@
+/*
+ * 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.hyracks.api.result;
+
+import java.util.Set;
+
+import org.apache.hyracks.api.job.JobId;
+
+public interface IResultManager {
+
+    Set<JobId> getJobIds();
+
+    IResultStateRecord getState(JobId jobId);
+
+    void sweep(JobId jobId);
+
+    /**
+     * Removes all references and deletes persisted files for
+     * all expired results.
+     */
+    void sweepExpiredResultSets();
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultPartitionManager.java
new file mode 100644
index 0000000..a539d37
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultPartitionManager.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hyracks.api.result;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+
+public interface IResultPartitionManager extends IResultManager {
+    IFrameWriter createResultPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            boolean asyncMode, int partition, int nPartitions, long maxReads) throws HyracksException;
+
+    void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
+            boolean orderedResult, boolean emptyResult) throws HyracksException;
+
+    void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
+
+    void initializeResultPartitionReader(JobId jobId, ResultSetId resultSetId, int partition, IFrameWriter noc)
+            throws HyracksException;
+
+    void removePartition(JobId jobId, ResultSetId resultSetId, int partition);
+
+    void abortReader(JobId jobId);
+
+    void close();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSet.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSet.java
new file mode 100644
index 0000000..ff2e48a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSet.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hyracks.api.result;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+
+public interface IResultSet {
+    IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSetReader.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSetReader.java
new file mode 100644
index 0000000..0884c53
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultSetReader.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hyracks.api.result;
+
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResultSetReader {
+    Status getResultStatus();
+
+    int read(IFrame frame) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultStateRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultStateRecord.java
new file mode 100644
index 0000000..fe6bc15
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultStateRecord.java
@@ -0,0 +1,23 @@
+/*
+ * 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.hyracks.api.result;
+
+public interface IResultStateRecord {
+    long getTimestamp();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultDirectoryRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultDirectoryRecord.java
new file mode 100644
index 0000000..71792be
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultDirectoryRecord.java
@@ -0,0 +1,111 @@
+/*
+ * 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.hyracks.api.result;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.comm.NetworkAddress;
+
+public class ResultDirectoryRecord implements Serializable {
+    public enum Status {
+        IDLE,
+        RUNNING,
+        SUCCESS,
+        FAILED
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private NetworkAddress address;
+
+    private boolean readEOS;
+
+    private Status status;
+
+    private boolean empty;
+
+    public ResultDirectoryRecord() {
+        this.address = null;
+        this.readEOS = false;
+        this.status = Status.IDLE;
+    }
+
+    public void setNetworkAddress(NetworkAddress address) {
+        this.address = address;
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return address;
+    }
+
+    public void setEmpty(boolean empty) {
+        this.empty = empty;
+    }
+
+    public boolean isEmpty() {
+        return empty;
+    }
+
+    public void readEOS() {
+        this.readEOS = true;
+    }
+
+    public boolean hasReachedReadEOS() {
+        return readEOS;
+    }
+
+    public void start() {
+        updateStatus(Status.RUNNING);
+    }
+
+    public void writeEOS() {
+        updateStatus(Status.SUCCESS);
+    }
+
+    public void fail() {
+        status = Status.FAILED;
+    }
+
+    private void updateStatus(final ResultDirectoryRecord.Status newStatus) {
+        // FAILED is a stable status
+        if (status != Status.FAILED) {
+            status = newStatus;
+        }
+    }
+
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof ResultDirectoryRecord)) {
+            return false;
+        }
+        return address.equals(((ResultDirectoryRecord) o).address);
+    }
+
+    @Override
+    public String toString() {
+        return String.valueOf(address) + " " + status + (empty ? " (empty)" : "") + (readEOS ? " (EOS)" : "");
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultJobRecord.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultJobRecord.java
new file mode 100644
index 0000000..b8ddbd2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultJobRecord.java
@@ -0,0 +1,177 @@
+/*
+ * 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.hyracks.api.result;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultJobRecord implements IResultStateRecord {
+    public enum State {
+        IDLE,
+        RUNNING,
+        SUCCESS,
+        FAILED
+    }
+
+    public static class Status implements Serializable {
+
+        private static final long serialVersionUID = 1L;
+
+        State state = State.IDLE;
+
+        private List<Exception> exceptions;
+
+        public State getState() {
+            return state;
+        }
+
+        void setState(State state) {
+            this.state = state;
+        }
+
+        public List<Exception> getExceptions() {
+            return exceptions;
+        }
+
+        void setExceptions(List<Exception> exceptions) {
+            this.exceptions = exceptions;
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append("{ \"state\": \"").append(state.name()).append("\"");
+            if (exceptions != null && !exceptions.isEmpty()) {
+                sb.append(", \"exceptions\": ");
+                List<String> msgs = new ArrayList<>();
+                exceptions.forEach(e -> msgs.add("\"" + e.getMessage() + "\""));
+                sb.append(Arrays.toString(msgs.toArray()));
+            }
+            sb.append(" }");
+            return sb.toString();
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final long timestamp;
+
+    private Status status;
+
+    private Map<ResultSetId, ResultSetMetaData> resultSetMetadataMap = new HashMap<>();
+
+    public ResultJobRecord() {
+        this.timestamp = System.nanoTime();
+        this.status = new Status();
+    }
+
+    private void updateState(State newStatus) {
+        // FAILED is a stable status
+        if (status.state != State.FAILED) {
+            status.setState(newStatus);
+        }
+    }
+
+    public void start() {
+        updateState(State.RUNNING);
+    }
+
+    public void success() {
+        updateState(State.SUCCESS);
+    }
+
+    public void fail(ResultSetId rsId, int partition) {
+        getOrCreateDirectoryRecord(rsId, partition).fail();
+    }
+
+    public void fail(List<Exception> exceptions) {
+        updateState(State.FAILED);
+        status.setExceptions(exceptions);
+    }
+
+    @Override
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("{ \"status\": ").append(status.toString()).append(", ");
+        sb.append("\"timestamp\": ").append(timestamp).append(", ");
+        sb.append("\"resultsets\": ").append(Arrays.toString(resultSetMetadataMap.entrySet().toArray())).append(" }");
+        return sb.toString();
+    }
+
+    public void setResultSetMetaData(ResultSetId rsId, boolean orderedResult, int nPartitions)
+            throws HyracksDataException {
+        ResultSetMetaData rsMd = resultSetMetadataMap.get(rsId);
+        if (rsMd == null) {
+            resultSetMetadataMap.put(rsId, new ResultSetMetaData(nPartitions, orderedResult));
+        } else if (rsMd.getOrderedResult() != orderedResult || rsMd.getRecords().length != nPartitions) {
+            throw HyracksDataException.create(ErrorCode.INCONSISTENT_RESULT_METADATA, rsId.toString());
+        }
+        //TODO(tillw) throwing a HyracksDataException here hangs the execution tests
+    }
+
+    public ResultSetMetaData getResultSetMetaData(ResultSetId rsId) {
+        return resultSetMetadataMap.get(rsId);
+    }
+
+    public synchronized ResultDirectoryRecord getOrCreateDirectoryRecord(ResultSetId rsId, int partition) {
+        ResultDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
+        if (records[partition] == null) {
+            records[partition] = new ResultDirectoryRecord();
+        }
+        return records[partition];
+    }
+
+    public synchronized ResultDirectoryRecord getDirectoryRecord(ResultSetId rsId, int partition)
+            throws HyracksDataException {
+        ResultDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
+        if (records[partition] == null) {
+            throw HyracksDataException.create(ErrorCode.RESULT_NO_RECORD, partition, rsId);
+        }
+        return records[partition];
+    }
+
+    public synchronized void updateState(ResultSetId rsId) {
+        int successCount = 0;
+        ResultDirectoryRecord[] records = getResultSetMetaData(rsId).getRecords();
+        for (ResultDirectoryRecord record : records) {
+            if ((record != null) && (record.getStatus() == ResultDirectoryRecord.Status.SUCCESS)) {
+                successCount++;
+            }
+        }
+        if (successCount == records.length) {
+            success();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetId.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetId.java
new file mode 100644
index 0000000..ffd4ced
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetId.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hyracks.api.result;
+
+import java.io.Serializable;
+
+public class ResultSetId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final long id;
+
+    public ResultSetId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof ResultSetId)) {
+            return false;
+        }
+        return ((ResultSetId) o).id == id;
+    }
+
+    @Override
+    public String toString() {
+        return "RSID:" + id;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetMetaData.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetMetaData.java
new file mode 100644
index 0000000..b7b8f1c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/ResultSetMetaData.java
@@ -0,0 +1,46 @@
+/*
+ * 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.hyracks.api.result;
+
+import java.util.Arrays;
+
+public class ResultSetMetaData {
+    private final ResultDirectoryRecord[] records;
+    private final boolean ordered;
+
+    ResultSetMetaData(int len, boolean ordered) {
+        this.records = new ResultDirectoryRecord[len];
+        this.ordered = ordered;
+    }
+
+    public boolean getOrderedResult() {
+        return ordered;
+    }
+
+    public ResultDirectoryRecord[] getRecords() {
+        return records;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("{ordered: ").append(ordered).append(", records: ").append(Arrays.toString(records));
+        return sb.toString();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index d499554..46a1dec 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -205,6 +205,10 @@
                 tasks.add(ctx.getExecutorService().submit(() -> {
                     startSemaphore.release();
                     try {
+                        Thread.currentThread()
+                                .setName(Thread.currentThread().getName() + ":" + ctx.getJobletContext().getJobId()
+                                        + ":" + ctx.getTaskAttemptId() + ":"
+                                        + SuperActivityOperatorNodePushable.class.getSimpleName());
                         action.run(op);
                     } catch (Throwable th) { // NOSONAR: Must catch all causes of failure
                         failures.offer(th);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
index c5a6de2..6e6d342 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
@@ -38,7 +38,8 @@
                     destroyable.destroy();
                 } catch (Throwable th) { // NOSONAR. Had to be done to satisfy contracts
                     try {
-                        LOGGER.log(Level.WARN, "Failure destroying a destroyable resource", th);
+                        LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
+                                "Failure destroying a destroyable resource", th);
                     } catch (Throwable ignore) { // NOSONAR: Ignore catching Throwable
                         // NOSONAR Ignore logging failure
                     }
@@ -65,7 +66,8 @@
                 writer.close();
             } catch (Throwable th) { // NOSONAR Will be suppressed
                 try {
-                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                    LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
+                            "Failure closing a closeable resource of class {}", writer.getClass().getSimpleName(), th);
                 } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                     // NOSONAR: Ignore logging failure
                 }
@@ -89,7 +91,8 @@
             writer.fail();
         } catch (Throwable th) { // NOSONAR Will be suppressed
             try {
-                LOGGER.log(Level.WARN, "Failure failing " + writer.getClass().getSimpleName(), th);
+                LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
+                        "Failure failing " + writer.getClass().getSimpleName(), th);
             } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                 // NOSONAR ignore logging failure
             }
@@ -113,7 +116,8 @@
                 closable.close();
             } catch (Throwable th) { // NOSONAR Will be suppressed
                 try {
-                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                    LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
+                            "Failure closing a closeable resource", th);
                 } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                     // NOSONAR ignore logging failure
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index 26ce2c1..8758ef7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -27,6 +27,8 @@
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -83,12 +85,15 @@
      * @param errorCode
      *            the error code itself
      * @param message
-     *            the user provided error message (a format string as specified in {@link java.util.Formatter})
+     *            the user provided error message (a format string as specified in {@link Formatter})
+     * @param sourceLoc
+     *            the source location where the error originated
      * @param params
-     *            an array of objects taht will be provided to the {@link java.util.Formatter}
+     *            an array of objects taht will be provided to the {@link Formatter}
      * @return the formatted string
      */
-    public static String formatMessage(String component, int errorCode, String message, Serializable... params) {
+    public static String formatMessage(String component, int errorCode, String message, SourceLocation sourceLoc,
+            Serializable... params) {
         try (Formatter fmt = new Formatter()) {
             if (!NONE.equals(component)) {
                 fmt.format("%1$s%2$04d: ", component, errorCode);
@@ -98,6 +103,10 @@
                 return message;
             }
             fmt.format(message == null ? "null" : message, (Object[]) params);
+            if (sourceLoc != null) {
+                fmt.out().append(" (in line ").append(String.valueOf(sourceLoc.getLine())).append(", at column ")
+                        .append(String.valueOf(sourceLoc.getColumn())).append(')');
+            }
             return fmt.out().toString();
         } catch (Exception e) {
             // Do not throw further exceptions during exception processing.
@@ -106,4 +115,10 @@
         }
     }
 
+    public static String getCauseMessage(Throwable t) {
+        if (t instanceof IFormattedException) {
+            return t.getMessage();
+        }
+        return String.valueOf(t);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 9302f46..21f393c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -94,7 +94,7 @@
      * @return the root exception, or null if both parameters are null
      */
     public static Throwable suppress(Throwable first, Throwable second) {
-        if (second != null && second instanceof InterruptedException) {
+        if (second instanceof InterruptedException) {
             Thread.currentThread().interrupt();
         }
         if (first == null) {
@@ -119,4 +119,18 @@
         stackThrowable.setStackTrace(thread.getStackTrace());
         return stackThrowable;
     }
+
+    public static Throwable getRootCause(Throwable e) {
+        Throwable current = e;
+        Throwable cause = e.getCause();
+        while (cause != null && cause != current) {
+            current = cause;
+            cause = current.getCause();
+        }
+        return current;
+    }
+
+    public static boolean causedByInterrupt(Throwable th) {
+        return getRootCause(th) instanceof InterruptedException;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
index b2dd680..9413d1b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/InvokeUtil.java
@@ -26,8 +26,12 @@
 import java.util.function.BooleanSupplier;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ComputingAction;
+import org.apache.hyracks.util.IDelay;
 import org.apache.hyracks.util.IOInterruptibleAction;
+import org.apache.hyracks.util.IRetryPolicy;
 import org.apache.hyracks.util.InterruptibleAction;
+import org.apache.hyracks.util.Span;
 import org.apache.hyracks.util.ThrowingAction;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -217,7 +221,7 @@
     }
 
     /**
-     * Runs the supplied action, after suspending any pending interruption.  An error will be logged if
+     * Runs the supplied action, after suspending any pending interruption. An error will be logged if
      * the action is itself interrupted.
      */
     public static void runUninterruptible(ThrowingAction action) throws Exception {
@@ -251,4 +255,30 @@
             }
         }
     }
+
+    public static <T> T retryUntilSuccessOrExhausted(Span span, ComputingAction<T> action, IRetryPolicy policy,
+            IDelay delay) throws HyracksDataException {
+        Throwable failure;
+        int attempt = 0;
+        do {
+            attempt++;
+            try {
+                return action.compute();
+            } catch (Throwable th) {
+                failure = th;
+                if (!policy.retry(th)) {
+                    break;
+                }
+                try {
+                    LOGGER.log(Level.WARN, "Failure executing action {} for the {} time", action, attempt, failure);
+                    span.sleep(delay.calculate(attempt), TimeUnit.MILLISECONDS);
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
+        } while (!span.elapsed());
+        LOGGER.log(Level.WARN, "Final Failure executing action {} after {} attempts", action, attempt, failure);
+        throw HyracksDataException.create(failure);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
index 03227ee..09ecb15 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
@@ -23,6 +23,7 @@
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -44,6 +45,16 @@
     }
 
     /**
+     * Deletes a file
+     *
+     * @param filePath the file path to be deleted
+     * @throws HyracksDataException if the file couldn't be deleted
+     */
+    public static void delete(Path filePath) throws HyracksDataException {
+        delete(filePath.toFile());
+    }
+
+    /**
      * Delete a file
      *
      * @param fileRef the file to be deleted
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
index ccf0163..85019c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
@@ -65,20 +65,20 @@
         }
     }
 
-    public void stop() throws HyracksDataException, InterruptedException {
+    public void stop() throws HyracksDataException {
         stopped = true;
         executorThread.interrupt();
-        executorThread.join(1000);
+        InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         int attempt = 0;
         while (executorThread.isAlive()) {
             attempt++;
-            LOGGER.log(Level.WARN,
-                    "Failed to stop event processor after " + attempt + " attempts. Interrupted exception swallowed?");
+            LOGGER.log(Level.WARN, "Failed to stop event processor after {} attempts. Interrupted exception swallowed?",
+                    attempt, ExceptionUtils.fromThreadStack(executorThread));
             if (attempt == 10) {
                 throw HyracksDataException.create(ErrorCode.FAILED_TO_SHUTDOWN_EVENT_PROCESSOR, name);
             }
             executorThread.interrupt();
-            executorThread.join(1000);
+            InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 452d379..c704d7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -97,14 +97,14 @@
 78 = Failed to purge the bloom filter since it is active
 79 = Cannot bulk-load a non-empty tree
 80 = Cannot create index because it already exists
-81 = File %1$s is already mapped
+81 = File %1$s is already mapped as %2$s registered at %3$s
 82 = Failed to create the file %1$s because it already exists
 83 = No index found with resourceID %1$s
 84 = Files with overlapping non-contained timestamp intervals were found in %1$s
 85 = Found more than one transaction file in %1$s
 86 = Found an unrecognized index file %1$s
 87 = Unequal number of trees and filters found in %1$s
-88 = Cannot modify index (Disk is full)
+88 = Cannot modify index (%1$s)
 89 = The byte size of a single group (%1$s bytes) exceeds the budget for a group by operator (%2$s bytes)
 90 = Memory budget for the %1$s operator (%2$s bytes) is lower than the minimum (%3$s bytes)
 91 = Operation timed out
@@ -131,6 +131,7 @@
 112 = Cannot add an element to an inverted-index search result.
 113 = Undefined inverted-list merge type: %1$s
 114 = Node (%1$s) is not active
+115 = Local network error
 
 10000 = The given rule collection %1$s is not an instance of the List class.
 10001 = Cannot compose partition constraint %1$s with %2$s
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
index 23a1caa..c8c1790 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
@@ -36,7 +36,7 @@
     @Test
     public void returnedMessageWithNoComponentTest() {
         HyracksDataException cause = new HyracksDataException(ErrorMessageUtil.NONE, ErrorCode.ERROR_PROCESSING_TUPLE,
-                ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), null, null, 2);
+                ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), 2);
         HyracksDataException causeWithNodeId = HyracksDataException.create(cause, "nc1");
         Assert.assertEquals(cause.getMessage(), causeWithNodeId.getMessage());
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDataset.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDataset.java
deleted file mode 100644
index 7f549ca..0000000
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDataset.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.hyracks.client.dataset;
-
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
-import org.apache.hyracks.api.dataset.IHyracksDatasetReader;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.client.net.ClientNetworkManager;
-import org.apache.hyracks.control.nc.resources.memory.FrameManager;
-
-public class HyracksDataset implements IHyracksDataset {
-    private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
-
-    private final ClientNetworkManager netManager;
-
-    private final IHyracksCommonContext datasetClientCtx;
-
-    public HyracksDataset(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
-        NetworkAddress ddsAddress = hcc.getDatasetDirectoryServiceInfo();
-        datasetDirectoryServiceConnection =
-                new HyracksDatasetDirectoryServiceConnection(ddsAddress.getAddress(), ddsAddress.getPort());
-
-        netManager = new ClientNetworkManager(nReaders);
-        netManager.start();
-
-        datasetClientCtx = new DatasetClientContext(frameSize);
-    }
-
-    @Override
-    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
-        IHyracksDatasetReader reader = null;
-        try {
-            reader = new HyracksDatasetReader(datasetDirectoryServiceConnection, netManager, datasetClientCtx, jobId,
-                    resultSetId);
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-        return reader;
-    }
-
-    static class DatasetClientContext extends FrameManager implements IHyracksCommonContext {
-
-        DatasetClientContext(int frameSize) {
-            super(frameSize);
-        }
-
-        @Override
-        public IIOManager getIoManager() {
-            return null;
-        }
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
deleted file mode 100644
index 63139d9..0000000
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.hyracks.client.dataset;
-
-import java.net.InetSocketAddress;
-
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
-import org.apache.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.ipc.api.IIPCHandle;
-import org.apache.hyracks.ipc.api.RPCInterface;
-import org.apache.hyracks.ipc.impl.IPCSystem;
-import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
-
-//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class HyracksDatasetDirectoryServiceConnection implements IHyracksDatasetDirectoryServiceConnection {
-    private final IPCSystem ipc;
-    private final IHyracksDatasetDirectoryServiceInterface ddsi;
-
-    public HyracksDatasetDirectoryServiceConnection(String ddsHost, int ddsPort) throws Exception {
-        RPCInterface rpci = new RPCInterface();
-        ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
-        ipc.start();
-        IIPCHandle ddsIpchandle = ipc.getReconnectingHandle(new InetSocketAddress(ddsHost, ddsPort));
-        this.ddsi = new HyracksDatasetDirectoryServiceInterfaceRemoteProxy(ddsIpchandle, rpci);
-    }
-
-    @Override
-    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
-        return ddsi.getDatasetResultStatus(jobId, rsId);
-    }
-
-    @Override
-    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords) throws Exception {
-        return ddsi.getDatasetResultLocations(jobId, rsId, knownRecords);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
deleted file mode 100644
index 7eeb913..0000000
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.hyracks.client.dataset;
-
-import org.apache.hyracks.api.client.HyracksClientInterfaceFunctions;
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.ipc.api.IIPCHandle;
-import org.apache.hyracks.ipc.api.RPCInterface;
-
-//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class HyracksDatasetDirectoryServiceInterfaceRemoteProxy implements IHyracksDatasetDirectoryServiceInterface {
-    private final IIPCHandle ipcHandle;
-
-    private final RPCInterface rpci;
-
-    public HyracksDatasetDirectoryServiceInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
-        this.ipcHandle = ipcHandle;
-        this.rpci = rpci;
-    }
-
-    @Override
-    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
-        HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf =
-                new HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction(jobId, rsId);
-        return (Status) rpci.call(ipcHandle, gdrlf);
-    }
-
-    @Override
-    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords) throws Exception {
-        HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf =
-                new HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction(jobId, rsId, knownRecords);
-        return (DatasetDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetReader.java
deleted file mode 100644
index 36c77ce..0000000
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/dataset/HyracksDatasetReader.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * 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.hyracks.client.dataset;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.channels.IInputChannel;
-import org.apache.hyracks.api.channels.IInputChannelMonitor;
-import org.apache.hyracks.api.comm.FrameHelper;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
-import org.apache.hyracks.api.dataset.IHyracksDatasetReader;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.client.net.ClientNetworkManager;
-import org.apache.hyracks.comm.channels.DatasetNetworkInputChannel;
-import org.apache.hyracks.util.annotations.NotThreadSafe;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-@NotThreadSafe
-public class HyracksDatasetReader implements IHyracksDatasetReader {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-    private static final int NUM_READ_BUFFERS = 1;
-    private final IHyracksDatasetDirectoryServiceConnection datasetDirectory;
-    private final ClientNetworkManager netManager;
-    private final IHyracksCommonContext datasetClientCtx;
-    private final JobId jobId;
-    private final ResultSetId resultSetId;
-    private DatasetDirectoryRecord[] knownRecords;
-    private DatasetInputChannelMonitor[] monitors;
-    private DatasetInputChannelMonitor currentRecordMonitor;
-    private DatasetNetworkInputChannel currentRecordChannel;
-    private int currentRecord;
-
-    public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectory,
-            ClientNetworkManager netManager, IHyracksCommonContext datasetClientCtx, JobId jobId,
-            ResultSetId resultSetId) {
-        this.datasetDirectory = datasetDirectory;
-        this.netManager = netManager;
-        this.datasetClientCtx = datasetClientCtx;
-        this.jobId = jobId;
-        this.resultSetId = resultSetId;
-        currentRecord = -1;
-    }
-
-    @Override
-    public Status getResultStatus() {
-        try {
-            return datasetDirectory.getDatasetResultStatus(jobId, resultSetId);
-        } catch (HyracksDataException e) {
-            if (e.getErrorCode() != ErrorCode.NO_RESULT_SET) {
-                LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
-            }
-        } catch (Exception e) {
-            LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
-        }
-        return null;
-    }
-
-    @Override
-    public int read(IFrame frame) throws HyracksDataException {
-        frame.reset();
-        int readSize = 0;
-        if (isFirstRead() && !hasNextRecord()) {
-            return readSize;
-        }
-        // read until frame is full or all dataset records have been read
-        while (readSize < frame.getFrameSize()) {
-            if (currentRecordMonitor.hasMoreFrames()) {
-                final ByteBuffer readBuffer = currentRecordChannel.getNextBuffer();
-                if (readBuffer == null) {
-                    throw new IllegalStateException("Unexpected empty frame");
-                }
-                currentRecordMonitor.notifyFrameRead();
-                if (readSize == 0) {
-                    final int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
-                    frame.ensureFrameSize(frame.getMinSize() * nBlocks);
-                    frame.getBuffer().clear();
-                }
-                frame.getBuffer().put(readBuffer);
-                currentRecordChannel.recycleBuffer(readBuffer);
-                readSize = frame.getBuffer().position();
-            } else {
-                currentRecordChannel.close();
-                if (currentRecordMonitor.failed()) {
-                    throw HyracksDataException.create(ErrorCode.FAILED_TO_READ_RESULT, jobId);
-                }
-                if (isLastRecord() || !hasNextRecord()) {
-                    break;
-                }
-            }
-        }
-        frame.getBuffer().flip();
-        return readSize;
-    }
-
-    private SocketAddress getSocketAddress(DatasetDirectoryRecord record) throws HyracksDataException {
-        try {
-            final NetworkAddress netAddr = record.getNetworkAddress();
-            return new InetSocketAddress(InetAddress.getByAddress(netAddr.lookupIpAddress()), netAddr.getPort());
-        } catch (UnknownHostException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    private DatasetInputChannelMonitor getMonitor(int partition) {
-        if (knownRecords == null || knownRecords[partition] == null) {
-            throw new IllegalStateException("Accessing monitors before obtaining the corresponding addresses");
-        }
-        if (monitors == null) {
-            monitors = new DatasetInputChannelMonitor[knownRecords.length];
-        }
-        if (monitors[partition] == null) {
-            monitors[partition] = new DatasetInputChannelMonitor();
-        }
-        return monitors[partition];
-    }
-
-    private boolean hasNextRecord() throws HyracksDataException {
-        currentRecord++;
-        DatasetDirectoryRecord record = getRecord(currentRecord);
-        // skip empty records
-        while (record.isEmpty() && ++currentRecord < knownRecords.length) {
-            record = getRecord(currentRecord);
-        }
-        if (currentRecord == knownRecords.length) {
-            // exhausted all known records
-            return false;
-        }
-        requestRecordData(record);
-        return true;
-    }
-
-    private DatasetDirectoryRecord getRecord(int recordNum) throws HyracksDataException {
-        try {
-            while (knownRecords == null || knownRecords[recordNum] == null) {
-                knownRecords = datasetDirectory.getDatasetResultLocations(jobId, resultSetId, knownRecords);
-            }
-            return knownRecords[recordNum];
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    private void requestRecordData(DatasetDirectoryRecord record) throws HyracksDataException {
-        currentRecordChannel = new DatasetNetworkInputChannel(netManager, getSocketAddress(record), jobId, resultSetId,
-                currentRecord, NUM_READ_BUFFERS);
-        currentRecordMonitor = getMonitor(currentRecord);
-        currentRecordChannel.registerMonitor(currentRecordMonitor);
-        currentRecordChannel.open(datasetClientCtx);
-    }
-
-    private boolean isFirstRead() {
-        return currentRecord == -1;
-    }
-
-    private boolean isLastRecord() {
-        return knownRecords != null && currentRecord == knownRecords.length - 1;
-    }
-
-    private static class DatasetInputChannelMonitor implements IInputChannelMonitor {
-
-        private int availableFrames;
-        private boolean eos;
-        private boolean failed;
-
-        DatasetInputChannelMonitor() {
-            eos = false;
-            failed = false;
-        }
-
-        @Override
-        public synchronized void notifyFailure(IInputChannel channel) {
-            failed = true;
-            notifyAll();
-        }
-
-        @Override
-        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
-            availableFrames += nFrames;
-            notifyAll();
-        }
-
-        @Override
-        public synchronized void notifyEndOfStream(IInputChannel channel) {
-            eos = true;
-            notifyAll();
-        }
-
-        synchronized boolean failed() {
-            return failed;
-        }
-
-        synchronized void notifyFrameRead() {
-            availableFrames--;
-            notifyAll();
-        }
-
-        synchronized boolean hasMoreFrames() throws HyracksDataException {
-            while (!failed && !eos && availableFrames == 0) {
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                    throw HyracksDataException.create(e);
-                }
-            }
-            return !failed && !isFullyConsumed();
-        }
-
-        private synchronized boolean isFullyConsumed() {
-            return availableFrames == 0 && eos;
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
new file mode 100644
index 0000000..6f8c4d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hyracks.client.result;
+
+import java.net.InetSocketAddress;
+
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.IResultDirectory;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.ipc.api.IIPCHandle;
+import org.apache.hyracks.ipc.api.RPCInterface;
+import org.apache.hyracks.ipc.impl.IPCSystem;
+import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
+public class ResultDirectory implements IResultDirectory {
+    private final IPCSystem ipc;
+    private final IResultDirectory remoteResultDirectory;
+
+    public ResultDirectory(String resultHost, int resultPort) throws Exception {
+        RPCInterface rpci = new RPCInterface();
+        ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
+        ipc.start();
+        IIPCHandle ddsIpchandle = ipc.getReconnectingHandle(new InetSocketAddress(resultHost, resultPort));
+        this.remoteResultDirectory = new ResultDirectoryRemoteProxy(ddsIpchandle, rpci);
+    }
+
+    @Override
+    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        return remoteResultDirectory.getResultStatus(jobId, rsId);
+    }
+
+    @Override
+    public ResultDirectoryRecord[] getResultLocations(JobId jobId, ResultSetId rsId,
+            ResultDirectoryRecord[] knownRecords) throws Exception {
+        return remoteResultDirectory.getResultLocations(jobId, rsId, knownRecords);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectoryRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectoryRemoteProxy.java
new file mode 100644
index 0000000..77c6e4b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectoryRemoteProxy.java
@@ -0,0 +1,55 @@
+/*
+ * 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.hyracks.client.result;
+
+import org.apache.hyracks.api.client.HyracksClientInterfaceFunctions;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.IResultDirectory;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.ipc.api.IIPCHandle;
+import org.apache.hyracks.ipc.api.RPCInterface;
+
+//TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
+public class ResultDirectoryRemoteProxy implements IResultDirectory {
+    private final IIPCHandle ipcHandle;
+
+    private final RPCInterface rpci;
+
+    public ResultDirectoryRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
+        this.ipcHandle = ipcHandle;
+        this.rpci = rpci;
+    }
+
+    @Override
+    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        HyracksClientInterfaceFunctions.GetResultStatusFunction gdrlf =
+                new HyracksClientInterfaceFunctions.GetResultStatusFunction(jobId, rsId);
+        return (Status) rpci.call(ipcHandle, gdrlf);
+    }
+
+    @Override
+    public ResultDirectoryRecord[] getResultLocations(JobId jobId, ResultSetId rsId,
+            ResultDirectoryRecord[] knownRecords) throws Exception {
+        HyracksClientInterfaceFunctions.GetResultLocationsFunction gdrlf =
+                new HyracksClientInterfaceFunctions.GetResultLocationsFunction(jobId, rsId, knownRecords);
+        return (ResultDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
new file mode 100644
index 0000000..ef93cce
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hyracks.client.result;
+
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.IResultDirectory;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.client.net.ClientNetworkManager;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+
+public class ResultSet implements IResultSet {
+    private final IResultDirectory resultDirectory;
+
+    private final ClientNetworkManager netManager;
+
+    private final IHyracksCommonContext resultClientCtx;
+
+    public ResultSet(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
+        NetworkAddress ddsAddress = hcc.getResultDirectoryAddress();
+        resultDirectory = new ResultDirectory(ddsAddress.getAddress(), ddsAddress.getPort());
+
+        netManager = new ClientNetworkManager(nReaders);
+        netManager.start();
+
+        resultClientCtx = new ResultClientContext(frameSize);
+    }
+
+    @Override
+    public IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+        IResultSetReader reader = null;
+        try {
+            reader = new ResultSetReader(resultDirectory, netManager, resultClientCtx, jobId, resultSetId);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        return reader;
+    }
+
+    static class ResultClientContext extends FrameManager implements IHyracksCommonContext {
+
+        ResultClientContext(int frameSize) {
+            super(frameSize);
+        }
+
+        @Override
+        public IIOManager getIoManager() {
+            return null;
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
new file mode 100644
index 0000000..092d860
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
@@ -0,0 +1,243 @@
+/*
+ * 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.hyracks.client.result;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.channels.IInputChannel;
+import org.apache.hyracks.api.channels.IInputChannelMonitor;
+import org.apache.hyracks.api.comm.FrameHelper;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.IResultDirectory;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.client.net.ClientNetworkManager;
+import org.apache.hyracks.comm.channels.ResultNetworkInputChannel;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+@NotThreadSafe
+public class ResultSetReader implements IResultSetReader {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final int NUM_READ_BUFFERS = 1;
+    private final IResultDirectory resultDirectory;
+    private final ClientNetworkManager netManager;
+    private final IHyracksCommonContext resultClientCtx;
+    private final JobId jobId;
+    private final ResultSetId resultSetId;
+    private ResultDirectoryRecord[] knownRecords;
+    private ResultInputChannelMonitor[] monitors;
+    private ResultInputChannelMonitor currentRecordMonitor;
+    private ResultNetworkInputChannel currentRecordChannel;
+    private int currentRecord;
+
+    public ResultSetReader(IResultDirectory resultDirectory, ClientNetworkManager netManager,
+            IHyracksCommonContext resultClientCtx, JobId jobId, ResultSetId resultSetId) {
+        this.resultDirectory = resultDirectory;
+        this.netManager = netManager;
+        this.resultClientCtx = resultClientCtx;
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        currentRecord = -1;
+    }
+
+    @Override
+    public Status getResultStatus() {
+        try {
+            return resultDirectory.getResultStatus(jobId, resultSetId);
+        } catch (HyracksDataException e) {
+            if (e.getErrorCode() != ErrorCode.NO_RESULT_SET) {
+                LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
+            }
+        } catch (Exception e) {
+            LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
+        }
+        return null;
+    }
+
+    @Override
+    public int read(IFrame frame) throws HyracksDataException {
+        frame.reset();
+        int readSize = 0;
+        if (isFirstRead() && !hasNextRecord()) {
+            return readSize;
+        }
+        // read until frame is full or all result records have been read
+        while (readSize < frame.getFrameSize()) {
+            if (currentRecordMonitor.hasMoreFrames()) {
+                final ByteBuffer readBuffer = currentRecordChannel.getNextBuffer();
+                if (readBuffer == null) {
+                    throw new IllegalStateException("Unexpected empty frame");
+                }
+                currentRecordMonitor.notifyFrameRead();
+                if (readSize == 0) {
+                    final int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
+                    frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+                    frame.getBuffer().clear();
+                }
+                frame.getBuffer().put(readBuffer);
+                currentRecordChannel.recycleBuffer(readBuffer);
+                readSize = frame.getBuffer().position();
+            } else {
+                currentRecordChannel.close();
+                if (currentRecordMonitor.failed()) {
+                    throw HyracksDataException.create(ErrorCode.FAILED_TO_READ_RESULT, jobId);
+                }
+                if (isLastRecord() || !hasNextRecord()) {
+                    break;
+                }
+            }
+        }
+        frame.getBuffer().flip();
+        return readSize;
+    }
+
+    private SocketAddress getSocketAddress(ResultDirectoryRecord record) throws HyracksDataException {
+        try {
+            final NetworkAddress netAddr = record.getNetworkAddress();
+            return new InetSocketAddress(InetAddress.getByAddress(netAddr.lookupIpAddress()), netAddr.getPort());
+        } catch (UnknownHostException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private ResultInputChannelMonitor getMonitor(int partition) {
+        if (knownRecords == null || knownRecords[partition] == null) {
+            throw new IllegalStateException("Accessing monitors before obtaining the corresponding addresses");
+        }
+        if (monitors == null) {
+            monitors = new ResultInputChannelMonitor[knownRecords.length];
+        }
+        if (monitors[partition] == null) {
+            monitors[partition] = new ResultInputChannelMonitor();
+        }
+        return monitors[partition];
+    }
+
+    private boolean hasNextRecord() throws HyracksDataException {
+        currentRecord++;
+        ResultDirectoryRecord record = getRecord(currentRecord);
+        // skip empty records
+        while (record.isEmpty() && ++currentRecord < knownRecords.length) {
+            record = getRecord(currentRecord);
+        }
+        if (currentRecord == knownRecords.length) {
+            // exhausted all known records
+            return false;
+        }
+        requestRecordData(record);
+        return true;
+    }
+
+    private ResultDirectoryRecord getRecord(int recordNum) throws HyracksDataException {
+        try {
+            while (knownRecords == null || knownRecords[recordNum] == null) {
+                knownRecords = resultDirectory.getResultLocations(jobId, resultSetId, knownRecords);
+            }
+            return knownRecords[recordNum];
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void requestRecordData(ResultDirectoryRecord record) throws HyracksDataException {
+        currentRecordChannel = new ResultNetworkInputChannel(netManager, getSocketAddress(record), jobId, resultSetId,
+                currentRecord, NUM_READ_BUFFERS);
+        currentRecordMonitor = getMonitor(currentRecord);
+        currentRecordChannel.registerMonitor(currentRecordMonitor);
+        currentRecordChannel.open(resultClientCtx);
+    }
+
+    private boolean isFirstRead() {
+        return currentRecord == -1;
+    }
+
+    private boolean isLastRecord() {
+        return knownRecords != null && currentRecord == knownRecords.length - 1;
+    }
+
+    private static class ResultInputChannelMonitor implements IInputChannelMonitor {
+
+        private int availableFrames;
+        private boolean eos;
+        private boolean failed;
+
+        ResultInputChannelMonitor() {
+            eos = false;
+            failed = false;
+        }
+
+        @Override
+        public synchronized void notifyFailure(IInputChannel channel, int errorCode) {
+            failed = true;
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            availableFrames += nFrames;
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyEndOfStream(IInputChannel channel) {
+            eos = true;
+            notifyAll();
+        }
+
+        synchronized boolean failed() {
+            return failed;
+        }
+
+        synchronized void notifyFrameRead() {
+            availableFrames--;
+            notifyAll();
+        }
+
+        synchronized boolean hasMoreFrames() throws HyracksDataException {
+            while (!failed && !eos && availableFrames == 0) {
+                try {
+                    wait();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
+            return !failed && !isFullyConsumed();
+        }
+
+        private synchronized boolean isFullyConsumed() {
+            return availableFrames == 0 && eos;
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/AggregateCounter.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/AggregateCounter.java
index 5d51340..6cb90e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/AggregateCounter.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/AggregateCounter.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.control.common.job.profiling.counters.Counter;
 
 public class AggregateCounter extends Counter {
+    private static final long serialVersionUID = 9140555872026977436L;
 
     private long sum = 0;
     private long numOfItems = 0;
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/impl/ClientCounterContext.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/impl/ClientCounterContext.java
index 9e87f52..1e562ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/impl/ClientCounterContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/stats/impl/ClientCounterContext.java
@@ -27,8 +27,7 @@
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
+
 import org.apache.hyracks.api.job.profiling.counters.ICounter;
 import org.apache.hyracks.client.stats.AggregateCounter;
 import org.apache.hyracks.client.stats.Counters;
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/DatasetNetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/DatasetNetworkInputChannel.java
deleted file mode 100644
index 44c3d36..0000000
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/DatasetNetworkInputChannel.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * 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.hyracks.comm.channels;
-
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-import org.apache.hyracks.api.channels.IInputChannel;
-import org.apache.hyracks.api.channels.IInputChannelMonitor;
-import org.apache.hyracks.api.comm.IBufferAcceptor;
-import org.apache.hyracks.api.comm.IChannelControlBlock;
-import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
-import org.apache.hyracks.api.context.IHyracksCommonContext;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class DatasetNetworkInputChannel implements IInputChannel {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    static final int INITIAL_MESSAGE_SIZE = 20;
-
-    private final IChannelConnectionFactory netManager;
-
-    private final SocketAddress remoteAddress;
-
-    private final JobId jobId;
-
-    private final ResultSetId resultSetId;
-
-    private final int partition;
-
-    private final Queue<ByteBuffer> fullQueue;
-
-    private final int nBuffers;
-
-    private IChannelControlBlock ccb;
-
-    private IInputChannelMonitor monitor;
-
-    private Object attachment;
-
-    public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
-            ResultSetId resultSetId, int partition, int nBuffers) {
-        this.netManager = netManager;
-        this.remoteAddress = remoteAddress;
-        this.jobId = jobId;
-        this.resultSetId = resultSetId;
-        this.partition = partition;
-        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
-        this.nBuffers = nBuffers;
-    }
-
-    @Override
-    public void registerMonitor(IInputChannelMonitor monitor) {
-        this.monitor = monitor;
-    }
-
-    @Override
-    public void setAttachment(Object attachment) {
-        this.attachment = attachment;
-    }
-
-    @Override
-    public Object getAttachment() {
-        return attachment;
-    }
-
-    @Override
-    public synchronized ByteBuffer getNextBuffer() {
-        return fullQueue.poll();
-    }
-
-    @Override
-    public void recycleBuffer(ByteBuffer buffer) {
-        buffer.clear();
-        ccb.getReadInterface().getEmptyBufferAcceptor().accept(buffer);
-    }
-
-    @Override
-    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
-        try {
-            ccb = netManager.connect(remoteAddress);
-        } catch (Exception e) {
-            throw HyracksDataException.create(e);
-        }
-        ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
-        ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
-        for (int i = 0; i < nBuffers; ++i) {
-            ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
-        }
-        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
-        writeBuffer.putLong(jobId.getId());
-        writeBuffer.putLong(resultSetId.getId());
-        writeBuffer.putInt(partition);
-        writeBuffer.flip();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
-                    + ccb);
-        }
-        ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
-        ccb.getWriteInterface().getFullBufferAcceptor().close();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-
-    }
-
-    private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
-        @Override
-        public void accept(ByteBuffer buffer) {
-            fullQueue.add(buffer);
-            monitor.notifyDataAvailability(DatasetNetworkInputChannel.this, 1);
-        }
-
-        @Override
-        public void close() {
-            monitor.notifyEndOfStream(DatasetNetworkInputChannel.this);
-        }
-
-        @Override
-        public void error(int ecode) {
-            monitor.notifyFailure(DatasetNetworkInputChannel.this);
-        }
-    }
-
-    private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
-        @Override
-        public void accept(ByteBuffer buffer) {
-            // do nothing
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
index a831492..58664c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
@@ -107,8 +107,8 @@
         writeBuffer.putInt(partitionId.getSenderIndex());
         writeBuffer.putInt(partitionId.getReceiverIndex());
         writeBuffer.flip();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Sending partition request: " + partitionId + " on channel: " + ccb);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Sending partition request: " + partitionId + " on channel: " + ccb);
         }
         ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
         ccb.getWriteInterface().getFullBufferAcceptor().close();
@@ -133,7 +133,7 @@
 
         @Override
         public void error(int ecode) {
-            monitor.notifyFailure(NetworkInputChannel.this);
+            monitor.notifyFailure(NetworkInputChannel.this, ecode);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkOutputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkOutputChannel.java
index 3016a7a..261e7c4 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkOutputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkOutputChannel.java
@@ -18,9 +18,11 @@
  */
 package org.apache.hyracks.comm.channels;
 
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.Deque;
+import java.util.Objects;
 
 import org.apache.hyracks.api.comm.IBufferAcceptor;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -75,7 +77,15 @@
                         break;
                     }
                     try {
-                        wait();
+                        InetSocketAddress remoteAddress = ccb.getRemoteAddress();
+                        String nameBefore = Thread.currentThread().getName();
+                        try {
+                            Thread.currentThread()
+                                    .setName(nameBefore + ":SendingTo(" + Objects.toString(remoteAddress) + ")");
+                            wait();
+                        } finally {
+                            Thread.currentThread().setName(nameBefore);
+                        }
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw HyracksDataException.create(e);
@@ -96,7 +106,7 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        ccb.getWriteInterface().getFullBufferAcceptor().error(AbstractChannelWriteInterface.REMOTE_WRITE_ERROR_CODE);
+        ccb.getWriteInterface().getFullBufferAcceptor().error(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
     }
 
     @Override
@@ -104,8 +114,8 @@
         ccb.getWriteInterface().getFullBufferAcceptor().close();
     }
 
-    public void abort() {
-        ccb.getWriteInterface().getFullBufferAcceptor().error(AbstractChannelWriteInterface.REMOTE_WRITE_ERROR_CODE);
+    public void abort(int ecode) {
+        ccb.getWriteInterface().getFullBufferAcceptor().error(ecode);
         synchronized (NetworkOutputChannel.this) {
             aborted = true;
             NetworkOutputChannel.this.notifyAll();
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java
new file mode 100644
index 0000000..1df39e9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java
@@ -0,0 +1,154 @@
+/*
+ * 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.hyracks.comm.channels;
+
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import org.apache.hyracks.api.channels.IInputChannel;
+import org.apache.hyracks.api.channels.IInputChannelMonitor;
+import org.apache.hyracks.api.comm.IBufferAcceptor;
+import org.apache.hyracks.api.comm.IChannelControlBlock;
+import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
+import org.apache.hyracks.api.context.IHyracksCommonContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResultNetworkInputChannel implements IInputChannel {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IChannelConnectionFactory netManager;
+
+    private final SocketAddress remoteAddress;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+
+    private final int partition;
+
+    private final Queue<ByteBuffer> fullQueue;
+
+    private final int nBuffers;
+
+    private IChannelControlBlock ccb;
+
+    private IInputChannelMonitor monitor;
+
+    private Object attachment;
+
+    public ResultNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
+            ResultSetId resultSetId, int partition, int nBuffers) {
+        this.netManager = netManager;
+        this.remoteAddress = remoteAddress;
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        this.partition = partition;
+        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        this.nBuffers = nBuffers;
+    }
+
+    @Override
+    public void registerMonitor(IInputChannelMonitor monitor) {
+        this.monitor = monitor;
+    }
+
+    @Override
+    public void setAttachment(Object attachment) {
+        this.attachment = attachment;
+    }
+
+    @Override
+    public Object getAttachment() {
+        return attachment;
+    }
+
+    @Override
+    public synchronized ByteBuffer getNextBuffer() {
+        return fullQueue.poll();
+    }
+
+    @Override
+    public void recycleBuffer(ByteBuffer buffer) {
+        buffer.clear();
+        ccb.getReadInterface().getEmptyBufferAcceptor().accept(buffer);
+    }
+
+    @Override
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
+        try {
+            ccb = netManager.connect(remoteAddress);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
+        ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
+        for (int i = 0; i < nBuffers; ++i) {
+            ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
+        }
+        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+        writeBuffer.putLong(jobId.getId());
+        writeBuffer.putLong(resultSetId.getId());
+        writeBuffer.putInt(partition);
+        writeBuffer.flip();
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
+                    + ccb);
+        }
+        ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
+        ccb.getWriteInterface().getFullBufferAcceptor().close();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+
+    }
+
+    private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+        @Override
+        public void accept(ByteBuffer buffer) {
+            fullQueue.add(buffer);
+            monitor.notifyDataAvailability(ResultNetworkInputChannel.this, 1);
+        }
+
+        @Override
+        public void close() {
+            monitor.notifyEndOfStream(ResultNetworkInputChannel.this);
+        }
+
+        @Override
+        public void error(int ecode) {
+            monitor.notifyFailure(ResultNetworkInputChannel.this, ecode);
+        }
+    }
+
+    private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+        @Override
+        public void accept(ByteBuffer buffer) {
+            // do nothing
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index 6862582..4d8b3f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -90,12 +90,15 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+      <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-core</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/BaseCCApplication.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/BaseCCApplication.java
index dc63481..46adda3 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/BaseCCApplication.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/BaseCCApplication.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.config.IConfigManager;
 import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.api.control.IGatekeeper;
 import org.apache.hyracks.api.job.resource.DefaultJobCapacityController;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.util.HyracksConstants;
@@ -91,4 +92,8 @@
         return configManager;
     }
 
+    @Override
+    public IGatekeeper getGatekeeper() {
+        return node -> true;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCDriver.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCDriver.java
index a188594..c4ad139 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCDriver.java
@@ -30,6 +30,10 @@
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
 import org.kohsuke.args4j.CmdLineException;
 
 public class CCDriver {
@@ -44,6 +48,12 @@
             ICCApplication application = getApplication(args);
             application.registerConfig(configManager);
             CCConfig ccConfig = new CCConfig(configManager);
+            LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
+            Configuration cfg = ctx.getConfiguration();
+            CCLogConfigurationFactory logCfgFactory = new CCLogConfigurationFactory(ccConfig);
+            ConfigurationFactory.setConfigurationFactory(logCfgFactory);
+            cfg.removeLogger("Console");
+            ctx.start(logCfgFactory.getConfiguration(ctx, ConfigurationSource.NULL_SOURCE));
             ClusterControllerService ccService = new ClusterControllerService(ccConfig, application);
             ccService.start();
             while (true) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCLogConfigurationFactory.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCLogConfigurationFactory.java
new file mode 100644
index 0000000..22dea9f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/CCLogConfigurationFactory.java
@@ -0,0 +1,90 @@
+/*
+ * 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.hyracks.control.cc;
+
+import java.io.File;
+import java.net.URI;
+
+import org.apache.hyracks.control.common.controllers.CCConfig;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
+import org.apache.logging.log4j.core.config.builder.api.AppenderComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.api.ComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.api.ConfigurationBuilder;
+import org.apache.logging.log4j.core.config.builder.api.LayoutComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.impl.BuiltConfiguration;
+
+public class CCLogConfigurationFactory extends ConfigurationFactory {
+    private CCConfig config;
+
+    public CCLogConfigurationFactory(CCConfig config) {
+        this.config = config;
+    }
+
+    public Configuration createConfiguration(ConfigurationBuilder<BuiltConfiguration> builder) {
+        File logDir = new File(config.getLogDir());
+        builder.setStatusLevel(Level.WARN);
+        builder.setConfigurationName("RollingBuilder");
+        // create a rolling file appender
+        LayoutComponentBuilder defaultLayout = builder.newLayout("PatternLayout").addAttribute("pattern",
+                "%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n");
+        ComponentBuilder triggeringPolicy = builder.newComponent("Policies")
+                .addComponent(builder.newComponent("CronTriggeringPolicy").addAttribute("schedule", "0 0 0 * * ?"))
+                .addComponent(builder.newComponent("SizeBasedTriggeringPolicy").addAttribute("size", "50M"));
+        AppenderComponentBuilder defaultRoll =
+                builder.newAppender("default", "RollingFile").addAttribute("fileName", new File(logDir, "cc.log"))
+                        .addAttribute("filePattern", new File(logDir, "cc-%d{MM-dd-yy}.log.gz")).add(defaultLayout)
+                        .addComponent(triggeringPolicy);
+        builder.add(defaultRoll);
+
+        // create the new logger
+        builder.add(builder.newRootLogger(Level.INFO).add(builder.newAppenderRef("default")));
+
+        LayoutComponentBuilder accessLayout = builder.newLayout("PatternLayout").addAttribute("pattern", "%m%n");
+        AppenderComponentBuilder accessRoll =
+                builder.newAppender("access", "RollingFile").addAttribute("fileName", new File(logDir, "access.log"))
+                        .addAttribute("filePattern", new File(logDir, "access-%d{MM-dd-yy}.log.gz")).add(accessLayout)
+                        .addComponent(triggeringPolicy);
+        builder.add(accessRoll);
+        builder.add(builder.newLogger("org.apache.hyracks.http.server.CLFLogger", Level.forName("ACCESS", 550))
+                .add(builder.newAppenderRef("access")).addAttribute("additivity", false));
+
+        return builder.build();
+    }
+
+    @Override
+    public Configuration getConfiguration(final LoggerContext loggerContext, final ConfigurationSource source) {
+        return getConfiguration(loggerContext, source.toString(), null);
+    }
+
+    @Override
+    public Configuration getConfiguration(final LoggerContext loggerContext, final String name,
+            final URI configLocation) {
+        ConfigurationBuilder<BuiltConfiguration> builder = newConfigurationBuilder();
+        return createConfiguration(builder);
+    }
+
+    @Override
+    protected String[] getSupportedTypes() {
+        return new String[] { "*" };
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e46aa7f..2edbab8 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -20,21 +20,21 @@
 
 import org.apache.hyracks.api.client.HyracksClientInterfaceFunctions;
 import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.DeployedJobSpecIdFactory;
 import org.apache.hyracks.api.job.JobIdFactory;
 import org.apache.hyracks.api.job.JobInfo;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
 import org.apache.hyracks.control.cc.work.CancelJobWork;
 import org.apache.hyracks.control.cc.work.CliDeployBinaryWork;
 import org.apache.hyracks.control.cc.work.CliUnDeployBinaryWork;
 import org.apache.hyracks.control.cc.work.ClusterShutdownWork;
 import org.apache.hyracks.control.cc.work.DeployJobSpecWork;
-import org.apache.hyracks.control.cc.work.GetDatasetDirectoryServiceInfoWork;
 import org.apache.hyracks.control.cc.work.GetJobInfoWork;
 import org.apache.hyracks.control.cc.work.GetJobStatusWork;
 import org.apache.hyracks.control.cc.work.GetNodeControllersInfoWork;
 import org.apache.hyracks.control.cc.work.GetNodeDetailsJSONWork;
+import org.apache.hyracks.control.cc.work.GetResultDirectoryAddressWork;
 import org.apache.hyracks.control.cc.work.GetResultPartitionLocationsWork;
 import org.apache.hyracks.control.cc.work.GetResultStatusWork;
 import org.apache.hyracks.control.cc.work.GetThreadDumpWork;
@@ -85,13 +85,19 @@
                 ccs.getWorkQueue().schedule(new GetJobInfoWork(ccs.getJobManager(), gjif.getJobId(),
                         new IPCResponder<JobInfo>(handle, mid)));
                 break;
-            case DISTRIBUTE_JOB:
+            case DEPLOY_JOB:
                 HyracksClientInterfaceFunctions.DeployJobSpecFunction djf =
                         (HyracksClientInterfaceFunctions.DeployJobSpecFunction) fn;
                 ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, djf.getACGGFBytes(),
-                        deployedJobSpecIdFactory.create(), new IPCResponder<>(handle, mid)));
+                        deployedJobSpecIdFactory.create(), false, new IPCResponder<>(handle, mid)));
                 break;
-            case DESTROY_JOB:
+            case REDEPLOY_JOB:
+                HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+                        (HyracksClientInterfaceFunctions.redeployJobSpecFunction) fn;
+                ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, udjsf.getACGGFBytes(),
+                        udjsf.getDeployedJobSpecId(), true, new IPCResponder<>(handle, mid)));
+                break;
+            case UNDEPLOY_JOB:
                 HyracksClientInterfaceFunctions.UndeployJobSpecFunction dsjf =
                         (HyracksClientInterfaceFunctions.UndeployJobSpecFunction) fn;
                 ccs.getWorkQueue().schedule(
@@ -115,19 +121,19 @@
                 ccs.getWorkQueue().schedule(new JobStartWork(ccs, sjf.getDeploymentId(), acggfBytes, sjf.getJobFlags(),
                         jobIdFactory, sjf.getJobParameters(), new IPCResponder<>(handle, mid), id));
                 break;
-            case GET_DATASET_DIRECTORY_SERIVICE_INFO:
+            case GET_RESULT_DIRECTORY_ADDRESS:
                 ccs.getWorkQueue().schedule(
-                        new GetDatasetDirectoryServiceInfoWork(ccs, new IPCResponder<NetworkAddress>(handle, mid)));
+                        new GetResultDirectoryAddressWork(ccs, new IPCResponder<NetworkAddress>(handle, mid)));
                 break;
-            case GET_DATASET_RESULT_STATUS:
-                HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrsf =
-                        (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+            case GET_RESULT_STATUS:
+                HyracksClientInterfaceFunctions.GetResultStatusFunction gdrsf =
+                        (HyracksClientInterfaceFunctions.GetResultStatusFunction) fn;
                 ccs.getWorkQueue().schedule(new GetResultStatusWork(ccs, gdrsf.getJobId(), gdrsf.getResultSetId(),
                         new IPCResponder<Status>(handle, mid)));
                 break;
-            case GET_DATASET_RESULT_LOCATIONS:
-                HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf =
-                        (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+            case GET_RESULT_LOCATIONS:
+                HyracksClientInterfaceFunctions.GetResultLocationsFunction gdrlf =
+                        (HyracksClientInterfaceFunctions.GetResultLocationsFunction) fn;
                 ccs.getWorkQueue().schedule(new GetResultPartitionLocationsWork(ccs, gdrlf.getJobId(),
                         gdrlf.getResultSetId(), gdrlf.getKnownRecords(), new IPCResponder<>(handle, mid)));
                 break;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
index 84cb4bd..85aa8ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
@@ -74,7 +74,8 @@
                 break;
             case NODE_HEARTBEAT:
                 CCNCFunctions.NodeHeartbeatFunction nhf = (CCNCFunctions.NodeHeartbeatFunction) fn;
-                ccs.getExecutor().execute(new NodeHeartbeatWork(ccs, nhf.getNodeId(), nhf.getHeartbeatData()));
+                ccs.getExecutor().execute(
+                        new NodeHeartbeatWork(ccs, nhf.getNodeId(), nhf.getHeartbeatData(), nhf.getNcAddress()));
                 break;
             case NOTIFY_JOBLET_CLEANUP:
                 CCNCFunctions.NotifyJobletCleanupFunction njcf = (CCNCFunctions.NotifyJobletCleanupFunction) fn;
@@ -161,6 +162,10 @@
                 ccs.getWorkQueue()
                         .schedule(new NotifyThreadDumpResponse(ccs, tdrf.getRequestId(), tdrf.getThreadDumpJSON()));
                 break;
+            case PING_RESPONSE:
+                CCNCFunctions.PingResponseFunction prf = (CCNCFunctions.PingResponseFunction) fn;
+                LOGGER.debug("Received ping response from node {}", prf.getNodeId());
+                break;
             default:
                 LOGGER.warn("Unknown function: " + fn.getFunctionId());
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index 153a32d..b5dacfb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -55,10 +55,10 @@
 import org.apache.hyracks.control.cc.application.CCServiceContext;
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.cc.cluster.NodeManager;
-import org.apache.hyracks.control.cc.dataset.DatasetDirectoryService;
-import org.apache.hyracks.control.cc.dataset.IDatasetDirectoryService;
 import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobManager;
+import org.apache.hyracks.control.cc.result.IResultDirectoryService;
+import org.apache.hyracks.control.cc.result.ResultDirectoryService;
 import org.apache.hyracks.control.cc.scheduler.IResourceManager;
 import org.apache.hyracks.control.cc.scheduler.ResourceManager;
 import org.apache.hyracks.control.cc.web.WebServer;
@@ -122,7 +122,7 @@
 
     private final DeadNodeSweeper sweeper;
 
-    private final IDatasetDirectoryService datasetDirectoryService;
+    private final IResultDirectoryService resultDirectoryService;
 
     private final Map<DeploymentId, DeploymentRun> deploymentRunMap;
 
@@ -171,15 +171,15 @@
         final ClusterTopology topology = computeClusterTopology(ccConfig);
         ccContext = new ClusterControllerContext(topology);
         sweeper = new DeadNodeSweeper();
-        datasetDirectoryService =
-                new DatasetDirectoryService(ccConfig.getResultTTL(), ccConfig.getResultSweepThreshold());
+        resultDirectoryService =
+                new ResultDirectoryService(ccConfig.getResultTTL(), ccConfig.getResultSweepThreshold());
 
         deploymentRunMap = new HashMap<>();
         stateDumpRunMap = new HashMap<>();
         threadDumpRunMap = Collections.synchronizedMap(new HashMap<>());
 
         // Node manager is in charge of cluster membership management.
-        nodeManager = new NodeManager(this, ccConfig, resourceManager);
+        nodeManager = new NodeManager(this, ccConfig, resourceManager, application.getGatekeeper());
 
         ccId = ccConfig.getCcId();
         jobIdFactory = new JobIdFactory(ccId);
@@ -217,11 +217,11 @@
         webServer.start();
         info = new ClusterControllerInfo(ccId, ccConfig.getClientPublicAddress(), ccConfig.getClientPublicPort(),
                 ccConfig.getConsolePublicPort());
-        timer.schedule(sweeper, 0, ccConfig.getHeartbeatPeriodMillis());
+        timer.schedule(sweeper, 0, ccConfig.getDeadNodeSweepThreshold());
         jobLog.open();
         startApplication();
 
-        datasetDirectoryService.init(executor);
+        resultDirectoryService.init(executor);
         workQueue.start();
         connectNCs();
         LOGGER.log(Level.INFO, "Started ClusterControllerService");
@@ -230,7 +230,7 @@
 
     private void startApplication() throws Exception {
         serviceCtx = new CCServiceContext(this, serverCtx, ccContext, ccConfig.getAppConfig());
-        serviceCtx.addJobLifecycleListener(datasetDirectoryService);
+        serviceCtx.addJobLifecycleListener(resultDirectoryService);
         application.init(serviceCtx);
         executor = MaintainedThreadNameExecutorService.newCachedThreadPool(serviceCtx.getThreadFactory());
         application.start(ccConfig.getAppArgsArray());
@@ -408,7 +408,7 @@
         return clusterIPC;
     }
 
-    public NetworkAddress getDatasetDirectoryServiceInfo() {
+    public NetworkAddress getResultDirectoryAddress() {
         return new NetworkAddress(ccConfig.getClientPublicAddress(), ccConfig.getClientPublicPort());
     }
 
@@ -461,8 +461,8 @@
         }
     }
 
-    public IDatasetDirectoryService getDatasetDirectoryService() {
-        return datasetDirectoryService;
+    public IResultDirectoryService getResultDirectoryService() {
+        return resultDirectoryService;
     }
 
     public synchronized void addStateDumpRun(String id, StateDumpRun sdr) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
index 0e22c25..041e224 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
@@ -40,9 +40,6 @@
     public void addDeployedJobSpecDescriptor(DeployedJobSpecId deployedJobSpecId,
             ActivityClusterGraph activityClusterGraph, JobSpecification jobSpecification,
             Set<Constraint> activityClusterGraphConstraints) throws HyracksException {
-        if (deployedJobSpecDescriptorMap.get(deployedJobSpecId) != null) {
-            throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
-        }
         DeployedJobSpecDescriptor descriptor =
                 new DeployedJobSpecDescriptor(activityClusterGraph, jobSpecification, activityClusterGraphConstraints);
         deployedJobSpecDescriptorMap.put(deployedJobSpecId, descriptor);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
index b44a6bb8..4f76ced 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -34,6 +34,7 @@
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.client.NodeStatus;
+import org.apache.hyracks.api.control.IGatekeeper;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -45,8 +46,6 @@
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.scheduler.IResourceManager;
 import org.apache.hyracks.control.common.controllers.CCConfig;
-import org.apache.hyracks.control.common.ipc.CCNCFunctions.AbortCCJobsFunction;
-import org.apache.hyracks.ipc.api.IIPCHandle;
 import org.apache.hyracks.ipc.exceptions.IPCException;
 import org.apache.hyracks.util.annotations.Idempotent;
 import org.apache.hyracks.util.annotations.NotThreadSafe;
@@ -63,14 +62,17 @@
     private final Map<String, NodeControllerState> nodeRegistry;
     private final Map<InetAddress, Set<String>> ipAddressNodeNameMap;
     private final int nodeCoresMultiplier;
+    private final IGatekeeper gatekeeper;
 
-    public NodeManager(ClusterControllerService ccs, CCConfig ccConfig, IResourceManager resourceManager) {
+    public NodeManager(ClusterControllerService ccs, CCConfig ccConfig, IResourceManager resourceManager,
+            IGatekeeper gatekeeper) {
         this.ccs = ccs;
         this.ccConfig = ccConfig;
         this.resourceManager = resourceManager;
         this.nodeRegistry = new LinkedHashMap<>();
         this.ipAddressNodeNameMap = new HashMap<>();
         this.nodeCoresMultiplier = ccConfig.getCoresMultiplier();
+        this.gatekeeper = gatekeeper;
     }
 
     @Override
@@ -95,23 +97,24 @@
 
     @Override
     public synchronized void addNode(String nodeId, NodeControllerState ncState) throws HyracksException {
-        LOGGER.warn("addNode(" + nodeId + ") called");
+        LOGGER.warn("+addNode: " + nodeId);
         if (nodeId == null || ncState == null) {
             throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
+        if (!gatekeeper.isAuthorized(nodeId)) {
+            throw HyracksException.create(ErrorCode.NO_SUCH_NODE, nodeId);
+        }
         // Updates the node registry.
         if (nodeRegistry.containsKey(nodeId)) {
-            LOGGER.warn("Node with name " + nodeId + " has already registered; failing the node then re-registering.");
+            LOGGER.warn("Node '" + nodeId + "' is already registered; failing the node then re-registering.");
             failNode(nodeId);
         }
         try {
-            // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
-            IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(ncState.getNodeController().getAddress());
-            ncIPCHandle.send(-1, new AbortCCJobsFunction(ccConfig.getCcId()), null);
+            ncState.getNodeController().abortJobs(ccs.getCcId());
         } catch (IPCException e) {
             throw HyracksDataException.create(e);
         }
-        LOGGER.warn("adding node to registry");
+        LOGGER.info("adding node to registry");
         nodeRegistry.put(nodeId, ncState);
         // Updates the IP address to node names map.
         try {
@@ -123,8 +126,7 @@
             nodeRegistry.remove(nodeId);
             throw e;
         }
-        // Updates the cluster capacity.
-        LOGGER.warn("updating cluster capacity");
+        LOGGER.info("updating cluster capacity");
         resourceManager.update(nodeId, getAdjustedNodeCapacity(ncState.getCapacity()));
     }
 
@@ -146,7 +148,7 @@
         Map<String, NodeControllerInfo> result = new LinkedHashMap<>();
         nodeRegistry.forEach(
                 (key, ncState) -> result.put(key, new NodeControllerInfo(key, NodeStatus.ACTIVE, ncState.getDataPort(),
-                        ncState.getDatasetPort(), ncState.getMessagingPort(), ncState.getCapacity().getCores())));
+                        ncState.getResultPort(), ncState.getMessagingPort(), ncState.getCapacity().getCores())));
         return result;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java
deleted file mode 100644
index 04aaddd..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/DatasetDirectoryService.java
+++ /dev/null
@@ -1,316 +0,0 @@
-/*
- * 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.hyracks.control.cc.dataset;
-
-import java.io.PrintWriter;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.State;
-import org.apache.hyracks.api.dataset.IDatasetStateRecord;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.dataset.ResultSetMetaData;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.control.common.dataset.AbstractDatasetManager;
-import org.apache.hyracks.control.common.dataset.ResultStateSweeper;
-import org.apache.hyracks.control.common.work.IResultCallback;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * TODO(madhusudancs): The potential perils of this global dataset directory service implementation is that, the jobs
- * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
- * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
- * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
- * job.
- */
-public class DatasetDirectoryService extends AbstractDatasetManager implements IDatasetDirectoryService {
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private final long resultSweepThreshold;
-
-    private final Map<JobId, JobResultInfo> jobResultLocations;
-
-    public DatasetDirectoryService(long resultTTL, long resultSweepThreshold) {
-        super(resultTTL);
-        this.resultSweepThreshold = resultSweepThreshold;
-        jobResultLocations = new LinkedHashMap<>();
-    }
-
-    @Override
-    public void init(ExecutorService executor) {
-        executor.execute(new ResultStateSweeper(this, resultSweepThreshold, LOGGER));
-    }
-
-    @Override
-    public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info(getClass().getSimpleName() + " notified of new job " + jobId);
-        }
-        if (jobResultLocations.get(jobId) != null) {
-            throw HyracksDataException.create(ErrorCode.MORE_THAN_ONE_RESULT, jobId);
-        }
-        jobResultLocations.put(jobId, new JobResultInfo(new DatasetJobRecord(), null));
-    }
-
-    @Override
-    public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
-        jobResultLocations.get(jobId).getRecord().start();
-    }
-
-    @Override
-    public void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
-        // Auto-generated method stub
-    }
-
-    private DatasetJobRecord getDatasetJobRecord(JobId jobId) {
-        final JobResultInfo jri = jobResultLocations.get(jobId);
-        return jri == null ? null : jri.getRecord();
-    }
-
-    private DatasetJobRecord getNonNullDatasetJobRecord(JobId jobId) throws HyracksDataException {
-        final DatasetJobRecord djr = getDatasetJobRecord(jobId);
-        if (djr == null) {
-            throw HyracksDataException.create(ErrorCode.NO_RESULT_SET, jobId);
-        }
-        return djr;
-    }
-
-    @Override
-    public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
-            boolean emptyResult, int partition, int nPartitions, NetworkAddress networkAddress)
-            throws HyracksDataException {
-        DatasetJobRecord djr = getNonNullDatasetJobRecord(jobId);
-        djr.setResultSetMetaData(rsId, orderedResult, nPartitions);
-        DatasetDirectoryRecord record = djr.getOrCreateDirectoryRecord(rsId, partition);
-
-        record.setNetworkAddress(networkAddress);
-        record.setEmpty(emptyResult);
-        record.start();
-
-        final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
-        Waiter waiter = jobResultInfo.getWaiter(rsId);
-        if (waiter != null) {
-            try {
-                DatasetDirectoryRecord[] updatedRecords = updatedRecords(jobId, rsId, waiter.knownRecords);
-                if (updatedRecords != null) {
-                    jobResultInfo.removeWaiter(rsId);
-                    waiter.callback.setValue(updatedRecords);
-                }
-            } catch (Exception e) {
-                waiter.callback.setException(e);
-            }
-        }
-        notifyAll();
-    }
-
-    @Override
-    public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition)
-            throws HyracksDataException {
-        DatasetJobRecord djr = getNonNullDatasetJobRecord(jobId);
-        djr.getDirectoryRecord(rsId, partition).writeEOS();
-        djr.updateState(rsId);
-        notifyAll();
-    }
-
-    @Override
-    public synchronized void reportJobFailure(JobId jobId, List<Exception> exceptions) {
-        LOGGER.log(Level.INFO, "job " + jobId + " failed and is being reported to " + getClass().getSimpleName(),
-                exceptions.get(0));
-        DatasetJobRecord djr = getDatasetJobRecord(jobId);
-        LOGGER.log(Level.INFO, "Dataset job record is " + djr);
-        if (djr != null) {
-            LOGGER.log(Level.INFO, "Setting exceptions in Dataset job record");
-            djr.fail(exceptions);
-        }
-        final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
-        LOGGER.log(Level.INFO, "Job result info is " + jobResultInfo);
-        if (jobResultInfo != null) {
-            LOGGER.log(Level.INFO, "Setting exceptions in Job result info");
-            jobResultInfo.setException(exceptions.isEmpty() ? null : exceptions.get(0));
-        }
-        notifyAll();
-    }
-
-    @Override
-    public synchronized DatasetJobRecord.Status getResultStatus(JobId jobId, ResultSetId rsId)
-            throws HyracksDataException {
-        return getNonNullDatasetJobRecord(jobId).getStatus();
-    }
-
-    @Override
-    public synchronized Set<JobId> getJobIds() {
-        return jobResultLocations.keySet();
-    }
-
-    @Override
-    public IDatasetStateRecord getState(JobId jobId) {
-        return getDatasetJobRecord(jobId);
-    }
-
-    @Override
-    public synchronized void sweep(JobId jobId) {
-        jobResultLocations.remove(jobId);
-    }
-
-    @Override
-    public synchronized void getResultPartitionLocations(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback)
-            throws HyracksDataException {
-        DatasetDirectoryRecord[] updatedRecords = updatedRecords(jobId, rsId, knownRecords);
-        if (updatedRecords == null) {
-            jobResultLocations.get(jobId).addWaiter(rsId, knownRecords, callback);
-        } else {
-            callback.setValue(updatedRecords);
-        }
-    }
-
-    /**
-     * Compares the records already known by the client for the given job's result set id with the records that the
-     * dataset directory service knows and if there are any newly discovered records returns a whole array with the
-     * new records filled in.
-     *
-     * @param jobId
-     *            - Id of the job for which the directory records should be retrieved.
-     * @param rsId
-     *            - Id of the result set for which the directory records should be retrieved.
-     * @param knownRecords
-     *            - An array of directory records that the client is already aware of.
-     * @return
-     *         Returns the updated records if new record were discovered, null otherwise
-     * @throws HyracksDataException
-     *             TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n) iterations for
-     *             every check. This already looks very expensive.
-     */
-    private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords) throws HyracksDataException {
-        DatasetJobRecord djr = getNonNullDatasetJobRecord(jobId);
-
-        if (djr.getStatus().getState() == State.FAILED) {
-            List<Exception> caughtExceptions = djr.getStatus().getExceptions();
-            if (caughtExceptions != null && !caughtExceptions.isEmpty()) {
-                final Exception cause = caughtExceptions.get(caughtExceptions.size() - 1);
-                if (cause instanceof HyracksDataException) {
-                    throw (HyracksDataException) cause;
-                }
-                throw HyracksDataException.create(ErrorCode.RESULT_FAILURE_EXCEPTION, cause, rsId, jobId);
-            } else {
-                throw HyracksDataException.create(ErrorCode.RESULT_FAILURE_NO_EXCEPTION, rsId, jobId);
-            }
-        }
-
-        final ResultSetMetaData resultSetMetaData = djr.getResultSetMetaData(rsId);
-        if (resultSetMetaData == null) {
-            return null;
-        }
-        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
-
-        return Arrays.equals(records, knownRecords) ? null : records;
-    }
-
-    public PrintWriter print(PrintWriter pw) {
-        for (JobId jId : getJobIds()) {
-            pw.print(jId.toString());
-            pw.print(" - ");
-            pw.println(String.valueOf(getDatasetJobRecord(jId)));
-        }
-        pw.flush();
-        return pw;
-    }
-}
-
-class JobResultInfo {
-
-    private DatasetJobRecord record;
-    private Waiters waiters;
-    private Exception exception;
-
-    JobResultInfo(DatasetJobRecord record, Waiters waiters) {
-        this.record = record;
-        this.waiters = waiters;
-    }
-
-    DatasetJobRecord getRecord() {
-        return record;
-    }
-
-    void addWaiter(ResultSetId rsId, DatasetDirectoryRecord[] knownRecords,
-            IResultCallback<DatasetDirectoryRecord[]> callback) {
-        if (waiters == null) {
-            waiters = new Waiters();
-        }
-        waiters.put(rsId, new Waiter(knownRecords, callback));
-        if (exception != null) {
-            // Exception was set before the waiter is added.
-            setException(exception);
-        }
-    }
-
-    Waiter removeWaiter(ResultSetId rsId) {
-        return waiters.remove(rsId);
-    }
-
-    Waiter getWaiter(ResultSetId rsId) {
-        return waiters != null ? waiters.get(rsId) : null;
-    }
-
-    void setException(Exception exception) {
-        if (waiters != null) {
-            for (ResultSetId rsId : waiters.keySet()) {
-                waiters.remove(rsId).callback.setException(exception);
-            }
-        }
-        // Caches the exception anyway for future added waiters.
-        this.exception = exception;
-    }
-
-    @Override
-    public String toString() {
-        return record.toString();
-    }
-}
-
-class Waiters extends HashMap<ResultSetId, Waiter> {
-    private static final long serialVersionUID = 1L;
-}
-
-class Waiter {
-    DatasetDirectoryRecord[] knownRecords;
-    IResultCallback<DatasetDirectoryRecord[]> callback;
-
-    Waiter(DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
-        this.knownRecords = knownRecords;
-        this.callback = callback;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/IDatasetDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/IDatasetDirectoryService.java
deleted file mode 100644
index 68d6c16..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/dataset/IDatasetDirectoryService.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.hyracks.control.cc.dataset;
-
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.IDatasetManager;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.IJobLifecycleListener;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.control.common.work.IResultCallback;
-
-public interface IDatasetDirectoryService extends IJobLifecycleListener, IDatasetManager {
-    public void init(ExecutorService executor);
-
-    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
-            boolean emptyResult, int partition, int nPartitions, NetworkAddress networkAddress)
-            throws HyracksDataException;
-
-    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition)
-            throws HyracksDataException;
-
-    public void reportJobFailure(JobId jobId, List<Exception> exceptions);
-
-    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
-
-    public void getResultPartitionLocations(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownLocations,
-            IResultCallback<DatasetDirectoryRecord[]> callback) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
index 3fe88bf..a88f9ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
@@ -74,11 +74,11 @@
 
         TaskCluster[] taskClusters = computeTaskClusters(ac, jobRun, activityPlanMap);
 
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Plan for " + ac);
-            LOGGER.info("Built " + taskClusters.length + " Task Clusters");
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Plan for " + ac);
+            LOGGER.trace("Built " + taskClusters.length + " Task Clusters");
             for (TaskCluster tc : taskClusters) {
-                LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+                LOGGER.trace("Tasks: " + Arrays.toString(tc.getTasks()));
             }
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
index 06af2d8..bdc73d5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
@@ -197,8 +197,8 @@
         Set<TaskCluster> taskClusterRoots = new HashSet<>();
         findRunnableTaskClusterRoots(taskClusterRoots,
                 jobRun.getActivityClusterGraph().getActivityClusterMap().values());
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO,
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.log(Level.TRACE,
                     "Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: " + inProgressTaskClusters);
         }
         if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
@@ -227,19 +227,19 @@
                 queue.add(new RankedRunnableTaskCluster(priority, tc));
             }
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Ranked TCs: " + queue);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Ranked TCs: " + queue);
         }
 
         Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<>();
         for (RankedRunnableTaskCluster rrtc : queue) {
             TaskCluster tc = rrtc.getTaskCluster();
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Found runnable TC: " + tc);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Found runnable TC: " + tc);
                 List<TaskClusterAttempt> attempts = tc.getAttempts();
-                LOGGER.debug("Attempts so far:" + attempts.size());
+                LOGGER.trace("Attempts so far:" + attempts.size());
                 for (TaskClusterAttempt tcAttempt : attempts) {
-                    LOGGER.debug("Status: " + tcAttempt.getStatus());
+                    LOGGER.trace("Status: " + tcAttempt.getStatus());
                 }
             }
             assignTaskLocations(tc, taskAttemptMap);
@@ -259,16 +259,16 @@
      * Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _}
      */
     private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Computing runnability: " + goal);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Computing runnability: " + goal);
         }
         if (runnabilityMap.containsKey(goal)) {
             return runnabilityMap.get(goal);
         }
         TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
         if (lastAttempt != null) {
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Last Attempt Status: " + lastAttempt.getStatus());
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Last Attempt Status: " + lastAttempt.getStatus());
             }
             if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
                 Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
@@ -285,15 +285,15 @@
         PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
         Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
         for (PartitionId pid : goal.getRequiredPartitions()) {
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Inspecting required partition: " + pid);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Inspecting required partition: " + pid);
             }
             Runnability runnability;
             ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
             IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
             PartitionState maxState = pmm.getMaximumAvailableState(pid);
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Policy: " + cPolicy + " maxState: " + maxState);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Policy: " + cPolicy + " maxState: " + maxState);
             }
             if (PartitionState.COMMITTED.equals(maxState)) {
                 runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
@@ -329,8 +329,8 @@
                 // already not runnable -- cannot get better. bail.
                 break;
             }
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("aggregateRunnability: " + aggregateRunnability);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("aggregateRunnability: " + aggregateRunnability);
             }
         }
         runnabilityMap.put(goal, aggregateRunnability);
@@ -470,7 +470,7 @@
             throw new HyracksException("No satisfiable location found for " + taskAttempt.getTaskAttemptId());
         }
         if (!liveNodes.contains(nodeId)) {
-            throw new HyracksException("Node " + nodeId + " not live");
+            throw HyracksException.create(ErrorCode.NO_SUCH_NODE, nodeId);
         }
         return nodeId;
     }
@@ -512,8 +512,8 @@
                 if (node != null) {
                     node.getActiveJobIds().add(jobRun.getJobId());
                     boolean changed = jobRun.getParticipatingNodeIds().add(nodeId);
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("Starting: " + taskDescriptors + " at " + entry.getKey());
+                    if (LOGGER.isTraceEnabled()) {
+                        LOGGER.trace("Starting: " + taskDescriptors + " at " + entry.getKey());
                     }
                     byte[] jagBytes = changed ? acgBytes : null;
                     node.getNodeController().startTasks(deploymentId, jobId, jagBytes, taskDescriptors,
@@ -539,14 +539,14 @@
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt,
             TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
-        LOGGER.debug("Aborting task cluster: " + tcAttempt.getAttempt());
+        LOGGER.trace(() -> "Aborting task cluster: " + tcAttempt.getAttempt());
         Set<TaskAttemptId> abortTaskIds = new HashSet<>();
         Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<>();
         for (TaskAttempt ta : tcAttempt.getTaskAttempts().values()) {
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskAttempt.TaskStatus status = ta.getStatus();
             abortTaskIds.add(taId);
-            LOGGER.debug("Checking " + taId + ": " + ta.getStatus());
+            LOGGER.trace(() -> "Checking " + taId + ": " + ta.getStatus());
             if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
                 ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
                 ta.setEndTime(System.currentTimeMillis());
@@ -561,13 +561,13 @@
             }
         }
         final JobId jobId = jobRun.getJobId();
-        LOGGER.info("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+        LOGGER.trace(() -> "Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
         INodeManager nodeManager = ccs.getNodeManager();
         abortTaskAttemptMap.forEach((key, abortTaskAttempts) -> {
             final NodeControllerState node = nodeManager.getNodeControllerState(key);
             if (node != null) {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("Aborting: " + abortTaskAttempts + " at " + key);
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Aborting: " + abortTaskAttempts + " at " + key);
                 }
                 try {
                     node.getNodeController().abortTasks(jobId, abortTaskAttempts);
@@ -587,7 +587,7 @@
     }
 
     private void abortDoomedTaskClusters() throws HyracksException {
-        LOGGER.log(Level.INFO, "aborting doomed task clusters");
+        LOGGER.trace("aborting doomed task clusters");
         Set<TaskCluster> doomedTaskClusters = new HashSet<>();
         for (TaskCluster tc : inProgressTaskClusters) {
             // Start search at TCs that produce no outputs (sinks)
@@ -596,7 +596,7 @@
             }
         }
 
-        LOGGER.log(Level.INFO, "number of doomed task clusters found = " + doomedTaskClusters.size());
+        LOGGER.trace(() -> "number of doomed task clusters found = " + doomedTaskClusters.size());
         for (TaskCluster tc : doomedTaskClusters) {
             TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
             if (tca != null) {
@@ -683,28 +683,28 @@
      */
     public void notifyTaskFailure(TaskAttempt ta, List<Exception> exceptions) {
         try {
-            LOGGER.log(Level.INFO, "Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
+            LOGGER.debug("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskCluster tc = ta.getTask().getTaskCluster();
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
-                LOGGER.log(Level.INFO, "Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
+                LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
                 ta.setStatus(TaskAttempt.TaskStatus.FAILED, exceptions);
                 abortTaskCluster(lastAttempt, TaskClusterAttempt.TaskClusterStatus.FAILED);
                 abortDoomedTaskClusters();
                 int maxReattempts = jobRun.getActivityClusterGraph().getMaxReattempts();
-                LOGGER.log(Level.INFO, "Marking TaskAttempt " + ta.getTaskAttemptId()
+                LOGGER.trace(() -> "Marking TaskAttempt " + ta.getTaskAttemptId()
                         + " as failed and the number of max re-attempts = " + maxReattempts);
                 if (lastAttempt.getAttempt() >= maxReattempts || isCancelled()) {
-                    LOGGER.log(Level.INFO, "Aborting the job of " + ta.getTaskAttemptId());
+                    LOGGER.debug(() -> "Aborting the job of " + ta.getTaskAttemptId());
                     abortJob(exceptions, NoOpCallback.INSTANCE);
                     return;
                 }
-                LOGGER.log(Level.INFO, "We will try to start runnable activity clusters of " + ta.getTaskAttemptId());
+                LOGGER.debug(() -> "We will try to start runnable activity clusters of " + ta.getTaskAttemptId());
                 startRunnableActivityClusters();
             } else {
-                LOGGER.warn(
-                        "Ignoring task failure notification: " + taId + " -- Current last attempt = " + lastAttempt);
+                LOGGER.warn(() -> "Ignoring task failure notification: " + taId + " -- Current last attempt = "
+                        + lastAttempt);
             }
         } catch (Exception e) {
             abortJob(Collections.singletonList(e), NoOpCallback.INSTANCE);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
index 3ba25f5..7e1ca61 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -157,8 +157,8 @@
 
     @Override
     public void prepareComplete(JobRun run, JobStatus status, List<Exception> exceptions) throws HyracksException {
-        ccs.removeJobParameterByteStore(run.getJobId());
         checkJob(run);
+        ccs.removeJobParameterByteStore(run.getJobId());
         if (status == JobStatus.FAILURE_BEFORE_EXECUTION) {
             run.setPendingStatus(JobStatus.FAILURE, exceptions);
             finalComplete(run);
@@ -221,17 +221,18 @@
         JobId jobId = run.getJobId();
         Throwable caughtException = null;
         CCServiceContext serviceCtx = ccs.getContext();
-        if (serviceCtx != null) {
-            try {
-                serviceCtx.notifyJobFinish(jobId, run.getPendingStatus(), run.getPendingExceptions());
-            } catch (Exception e) {
-                LOGGER.error("Exception notifying job finish {}", jobId, e);
-                caughtException = e;
-            }
+        try {
+            serviceCtx.notifyJobFinish(jobId, run.getPendingStatus(), run.getPendingExceptions());
+        } catch (Exception e) {
+            LOGGER.error("Exception notifying job finish {}", jobId, e);
+            caughtException = e;
         }
         run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
         run.setEndTime(System.currentTimeMillis());
-        activeRunMap.remove(jobId);
+        if (activeRunMap.remove(jobId) != null) {
+            // non-active jobs have zero capacity
+            releaseJobCapacity(run);
+        }
         runMapArchive.put(jobId, run);
         runMapHistory.put(jobId, run.getExceptions());
 
@@ -247,10 +248,6 @@
             }
         }
 
-        // Releases cluster capacitys occupied by the job.
-        JobSpecification job = run.getJobSpecification();
-        jobCapacityController.release(job);
-
         // Picks the next job to execute.
         pickJobsToRun();
 
@@ -347,4 +344,9 @@
             throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
         }
     }
+
+    private void releaseJobCapacity(JobRun jobRun) {
+        final JobSpecification job = jobRun.getJobSpecification();
+        jobCapacityController.release(job);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
index 6f5c5ad..ac29b53 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -59,7 +59,7 @@
             while (i.hasNext()) {
                 PartitionRequest req = i.next();
                 if (partitionDescriptor.getState().isAtLeast(req.getMinimumState())) {
-                    matches.add(Pair.<PartitionDescriptor, PartitionRequest> of(partitionDescriptor, req));
+                    matches.add(Pair.of(partitionDescriptor, req));
                     i.remove();
                     matched = true;
                     if (!partitionDescriptor.isReusable()) {
@@ -95,7 +95,7 @@
             while (i.hasNext()) {
                 PartitionDescriptor descriptor = i.next();
                 if (descriptor.getState().isAtLeast(partitionRequest.getMinimumState())) {
-                    match = Pair.<PartitionDescriptor, PartitionRequest> of(descriptor, partitionRequest);
+                    match = Pair.of(descriptor, partitionRequest);
                     if (!descriptor.isReusable()) {
                         i.remove();
                     }
@@ -174,7 +174,9 @@
     }
 
     public void removeUncommittedPartitions(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
-        LOGGER.info("Removing uncommitted partitions: " + partitionIds);
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Removing uncommitted partitions: " + partitionIds);
+        }
         IEntryFilter<PartitionDescriptor> filter = new IEntryFilter<PartitionDescriptor>() {
             @Override
             public boolean matches(PartitionDescriptor o) {
@@ -193,7 +195,9 @@
     }
 
     public void removePartitionRequests(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
-        LOGGER.info("Removing partition requests: " + partitionIds);
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Removing partition requests: " + partitionIds);
+        }
         IEntryFilter<PartitionRequest> filter = new IEntryFilter<PartitionRequest>() {
             @Override
             public boolean matches(PartitionRequest o) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/IResultDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/IResultDirectoryService.java
new file mode 100644
index 0000000..c8f576c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/IResultDirectoryService.java
@@ -0,0 +1,50 @@
+/*
+ * 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.hyracks.control.cc.result;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.result.IResultManager;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.control.common.work.IResultCallback;
+
+public interface IResultDirectoryService extends IJobLifecycleListener, IResultManager {
+    public void init(ExecutorService executor);
+
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+            boolean emptyResult, int partition, int nPartitions, NetworkAddress networkAddress)
+            throws HyracksDataException;
+
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition)
+            throws HyracksDataException;
+
+    public void reportJobFailure(JobId jobId, List<Exception> exceptions);
+
+    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
+
+    public void getResultPartitionLocations(JobId jobId, ResultSetId rsId, ResultDirectoryRecord[] knownLocations,
+            IResultCallback<ResultDirectoryRecord[]> callback) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
new file mode 100644
index 0000000..cdfa4d3
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/result/ResultDirectoryService.java
@@ -0,0 +1,315 @@
+/*
+ * 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.hyracks.control.cc.result;
+
+import java.io.PrintWriter;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.result.IResultStateRecord;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultJobRecord;
+import org.apache.hyracks.api.result.ResultJobRecord.State;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetMetaData;
+import org.apache.hyracks.control.common.result.AbstractResultManager;
+import org.apache.hyracks.control.common.result.ResultStateSweeper;
+import org.apache.hyracks.control.common.work.IResultCallback;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * TODO(madhusudancs): The potential perils of this global result directory service implementation is that, the jobs
+ * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
+ * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
+ * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
+ * job.
+ */
+public class ResultDirectoryService extends AbstractResultManager implements IResultDirectoryService {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final long resultSweepThreshold;
+
+    private final Map<JobId, JobResultInfo> jobResultLocations;
+
+    public ResultDirectoryService(long resultTTL, long resultSweepThreshold) {
+        super(resultTTL);
+        this.resultSweepThreshold = resultSweepThreshold;
+        jobResultLocations = new LinkedHashMap<>();
+    }
+
+    @Override
+    public void init(ExecutorService executor) {
+        executor.execute(new ResultStateSweeper(this, resultSweepThreshold, LOGGER));
+    }
+
+    @Override
+    public synchronized void notifyJobCreation(JobId jobId, JobSpecification spec) throws HyracksException {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug(getClass().getSimpleName() + " notified of new job " + jobId);
+        }
+        if (jobResultLocations.get(jobId) != null) {
+            throw HyracksDataException.create(ErrorCode.MORE_THAN_ONE_RESULT, jobId);
+        }
+        jobResultLocations.put(jobId, new JobResultInfo(new ResultJobRecord(), null));
+    }
+
+    @Override
+    public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
+        jobResultLocations.get(jobId).getRecord().start();
+    }
+
+    @Override
+    public void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions) throws HyracksException {
+        // Auto-generated method stub
+    }
+
+    private ResultJobRecord getResultJobRecord(JobId jobId) {
+        final JobResultInfo jri = jobResultLocations.get(jobId);
+        return jri == null ? null : jri.getRecord();
+    }
+
+    private ResultJobRecord getNonNullResultJobRecord(JobId jobId) throws HyracksDataException {
+        final ResultJobRecord djr = getResultJobRecord(jobId);
+        if (djr == null) {
+            throw HyracksDataException.create(ErrorCode.NO_RESULT_SET, jobId);
+        }
+        return djr;
+    }
+
+    @Override
+    public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+            boolean emptyResult, int partition, int nPartitions, NetworkAddress networkAddress)
+            throws HyracksDataException {
+        ResultJobRecord djr = getNonNullResultJobRecord(jobId);
+        djr.setResultSetMetaData(rsId, orderedResult, nPartitions);
+        ResultDirectoryRecord record = djr.getOrCreateDirectoryRecord(rsId, partition);
+
+        record.setNetworkAddress(networkAddress);
+        record.setEmpty(emptyResult);
+        record.start();
+
+        final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
+        Waiter waiter = jobResultInfo.getWaiter(rsId);
+        if (waiter != null) {
+            try {
+                ResultDirectoryRecord[] updatedRecords = updatedRecords(jobId, rsId, waiter.knownRecords);
+                if (updatedRecords != null) {
+                    jobResultInfo.removeWaiter(rsId);
+                    waiter.callback.setValue(updatedRecords);
+                }
+            } catch (Exception e) {
+                waiter.callback.setException(e);
+            }
+        }
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition)
+            throws HyracksDataException {
+        ResultJobRecord djr = getNonNullResultJobRecord(jobId);
+        djr.getDirectoryRecord(rsId, partition).writeEOS();
+        djr.updateState(rsId);
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void reportJobFailure(JobId jobId, List<Exception> exceptions) {
+        Exception ex = exceptions.isEmpty() ? null : exceptions.get(0);
+        Level logLevel = Level.DEBUG;
+        if (LOGGER.isEnabled(logLevel)) {
+            LOGGER.log(logLevel, "job " + jobId + " failed and is being reported to " + getClass().getSimpleName(), ex);
+        }
+        ResultJobRecord rjr = getResultJobRecord(jobId);
+        if (rjr != null) {
+            rjr.fail(exceptions);
+        }
+        final JobResultInfo jobResultInfo = jobResultLocations.get(jobId);
+        if (jobResultInfo != null) {
+            jobResultInfo.setException(ex);
+        }
+        notifyAll();
+    }
+
+    @Override
+    public synchronized ResultJobRecord.Status getResultStatus(JobId jobId, ResultSetId rsId)
+            throws HyracksDataException {
+        return getNonNullResultJobRecord(jobId).getStatus();
+    }
+
+    @Override
+    public synchronized Set<JobId> getJobIds() {
+        return jobResultLocations.keySet();
+    }
+
+    @Override
+    public IResultStateRecord getState(JobId jobId) {
+        return getResultJobRecord(jobId);
+    }
+
+    @Override
+    public synchronized void sweep(JobId jobId) {
+        jobResultLocations.remove(jobId);
+    }
+
+    @Override
+    public synchronized void getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+            ResultDirectoryRecord[] knownRecords, IResultCallback<ResultDirectoryRecord[]> callback)
+            throws HyracksDataException {
+        ResultDirectoryRecord[] updatedRecords = updatedRecords(jobId, rsId, knownRecords);
+        if (updatedRecords == null) {
+            jobResultLocations.get(jobId).addWaiter(rsId, knownRecords, callback);
+        } else {
+            callback.setValue(updatedRecords);
+        }
+    }
+
+    /**
+     * Compares the records already known by the client for the given job's result set id with the records that the
+     * result directory service knows and if there are any newly discovered records returns a whole array with the
+     * new records filled in.
+     *
+     * @param jobId
+     *            - Id of the job for which the directory records should be retrieved.
+     * @param rsId
+     *            - Id of the result set for which the directory records should be retrieved.
+     * @param knownRecords
+     *            - An array of directory records that the client is already aware of.
+     * @return
+     *         Returns the updated records if new record were discovered, null otherwise
+     * @throws HyracksDataException
+     *             TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n)
+     *             iterations for every check. This already looks very expensive.
+     */
+    private ResultDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, ResultDirectoryRecord[] knownRecords)
+            throws HyracksDataException {
+        ResultJobRecord djr = getNonNullResultJobRecord(jobId);
+
+        if (djr.getStatus().getState() == State.FAILED) {
+            List<Exception> caughtExceptions = djr.getStatus().getExceptions();
+            if (caughtExceptions != null && !caughtExceptions.isEmpty()) {
+                final Exception cause = caughtExceptions.get(caughtExceptions.size() - 1);
+                if (cause instanceof HyracksDataException) {
+                    throw (HyracksDataException) cause;
+                }
+                throw HyracksDataException.create(ErrorCode.RESULT_FAILURE_EXCEPTION, cause, rsId, jobId);
+            } else {
+                throw HyracksDataException.create(ErrorCode.RESULT_FAILURE_NO_EXCEPTION, rsId, jobId);
+            }
+        }
+
+        final ResultSetMetaData resultSetMetaData = djr.getResultSetMetaData(rsId);
+        if (resultSetMetaData == null) {
+            return null;
+        }
+        ResultDirectoryRecord[] records = resultSetMetaData.getRecords();
+
+        return Arrays.equals(records, knownRecords) ? null : records;
+    }
+
+    public PrintWriter print(PrintWriter pw) {
+        for (JobId jId : getJobIds()) {
+            pw.print(jId.toString());
+            pw.print(" - ");
+            pw.println(String.valueOf(getResultJobRecord(jId)));
+        }
+        pw.flush();
+        return pw;
+    }
+}
+
+class JobResultInfo {
+
+    private ResultJobRecord record;
+    private Waiters waiters;
+    private Exception exception;
+
+    JobResultInfo(ResultJobRecord record, Waiters waiters) {
+        this.record = record;
+        this.waiters = waiters;
+    }
+
+    ResultJobRecord getRecord() {
+        return record;
+    }
+
+    void addWaiter(ResultSetId rsId, ResultDirectoryRecord[] knownRecords,
+            IResultCallback<ResultDirectoryRecord[]> callback) {
+        if (waiters == null) {
+            waiters = new Waiters();
+        }
+        waiters.put(rsId, new Waiter(knownRecords, callback));
+        if (exception != null) {
+            // Exception was set before the waiter is added.
+            setException(exception);
+        }
+    }
+
+    Waiter removeWaiter(ResultSetId rsId) {
+        return waiters.remove(rsId);
+    }
+
+    Waiter getWaiter(ResultSetId rsId) {
+        return waiters != null ? waiters.get(rsId) : null;
+    }
+
+    void setException(Exception exception) {
+        if (waiters != null) {
+            for (ResultSetId rsId : waiters.keySet()) {
+                waiters.remove(rsId).callback.setException(exception);
+            }
+        }
+        // Caches the exception anyway for future added waiters.
+        this.exception = exception;
+    }
+
+    @Override
+    public String toString() {
+        return record.toString();
+    }
+}
+
+class Waiters extends HashMap<ResultSetId, Waiter> {
+    private static final long serialVersionUID = 1L;
+}
+
+class Waiter {
+    ResultDirectoryRecord[] knownRecords;
+    IResultCallback<ResultDirectoryRecord[]> callback;
+
+    Waiter(ResultDirectoryRecord[] knownRecords, IResultCallback<ResultDirectoryRecord[]> callback) {
+        this.knownRecords = knownRecords;
+        this.callback = callback;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
index f413fe5..2938bd2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.control.cc.web.util.IJSONOutputFunction;
 import org.apache.hyracks.control.cc.web.util.JSONOutputRequestHandler;
 import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.StaticResourceServlet;
 import org.apache.hyracks.http.server.WebManager;
 
@@ -40,7 +41,8 @@
         listeningPort = port;
         ctx = new ConcurrentHashMap<>();
         webMgr = new WebManager();
-        server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), listeningPort);
+        server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), listeningPort,
+                HttpServerConfigBuilder.createDefault());
         addHandlers();
         webMgr.add(server);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
index 8e7faff..401360c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
@@ -27,9 +27,9 @@
 
 public abstract class AbstractHeartbeatWork extends SynchronizableWork {
 
-    private final ClusterControllerService ccs;
-    private final String nodeId;
-    private final HeartbeatData hbData;
+    protected final ClusterControllerService ccs;
+    protected final String nodeId;
+    protected final HeartbeatData hbData;
 
     public AbstractHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
         this.ccs = ccs;
@@ -38,7 +38,7 @@
     }
 
     @Override
-    public void doRun() {
+    public void doRun() throws Exception {
         INodeManager nodeManager = ccs.getNodeManager();
         NodeControllerState state = nodeManager.getNodeControllerState(nodeId);
         if (state != null) {
@@ -51,6 +51,6 @@
         runWork();
     }
 
-    public abstract void runWork();
+    public abstract void runWork() throws Exception;
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
index d0c6567..b9053ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
@@ -36,17 +36,13 @@
 import org.apache.hyracks.control.cc.job.TaskClusterAttempt;
 
 public abstract class AbstractTaskLifecycleWork extends AbstractHeartbeatWork {
-    protected final ClusterControllerService ccs;
     protected final JobId jobId;
     protected final TaskAttemptId taId;
-    protected final String nodeId;
 
     public AbstractTaskLifecycleWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
         super(ccs, nodeId, null);
-        this.ccs = ccs;
         this.jobId = jobId;
         this.taId = taId;
-        this.nodeId = nodeId;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
index 392046d..f2aa1f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -35,15 +35,11 @@
     private static final Logger LOGGER = LogManager.getLogger();
     private byte[] message;
     private DeploymentId deploymentId;
-    private String nodeId;
-    private ClusterControllerService ccs;
 
     public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, DeploymentId deploymentId,
             String nodeId) {
         super(ccs, nodeId, null);
-        this.ccs = ccs;
         this.deploymentId = deploymentId;
-        this.nodeId = nodeId;
         this.message = message;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
index c51f3c5..60c88c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
@@ -39,20 +39,24 @@
     private final byte[] acggfBytes;
     private final DeployedJobSpecId deployedJobSpecId;
     private final IResultCallback<DeployedJobSpecId> callback;
+    private final boolean upsert;
 
     public DeployJobSpecWork(ClusterControllerService ccs, byte[] acggfBytes, DeployedJobSpecId deployedJobSpecId,
-            IResultCallback<DeployedJobSpecId> callback) {
+            boolean upsert, IResultCallback<DeployedJobSpecId> callback) {
         this.deployedJobSpecId = deployedJobSpecId;
         this.ccs = ccs;
         this.acggfBytes = acggfBytes;
         this.callback = callback;
+        this.upsert = upsert;
     }
 
     @Override
     protected void doRun() throws Exception {
         try {
             final CCServiceContext ccServiceCtx = ccs.getContext();
-            ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+            if (!upsert) {
+                ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+            }
             IActivityClusterGraphGeneratorFactory acggf =
                     (IActivityClusterGraphGeneratorFactory) DeploymentUtils.deserialize(acggfBytes, null, ccServiceCtx);
             IActivityClusterGraphGenerator acgg =
@@ -65,7 +69,7 @@
 
             INodeManager nodeManager = ccs.getNodeManager();
             for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
-                node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes);
+                node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes, upsert);
             }
             callback.setValue(deployedJobSpecId);
         } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
deleted file mode 100644
index fb1febf..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.hyracks.control.cc.work;
-
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.common.work.IResultCallback;
-import org.apache.hyracks.control.common.work.SynchronizableWork;
-
-public class GetDatasetDirectoryServiceInfoWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
-
-    private final IResultCallback<NetworkAddress> callback;
-
-    public GetDatasetDirectoryServiceInfoWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
-        this.ccs = ccs;
-        this.callback = callback;
-    }
-
-    @Override
-    public void doRun() {
-        try {
-            NetworkAddress addr = ccs.getDatasetDirectoryServiceInfo();
-            callback.setValue(addr);
-        } catch (Exception e) {
-            callback.setException(e);
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
index 872fb9c..bed437f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
@@ -22,7 +22,6 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
new file mode 100644
index 0000000..bf95ff2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
@@ -0,0 +1,45 @@
+/*
+ * 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.hyracks.control.cc.work;
+
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.common.work.IResultCallback;
+import org.apache.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultDirectoryAddressWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final IResultCallback<NetworkAddress> callback;
+
+    public GetResultDirectoryAddressWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
+        this.ccs = ccs;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            NetworkAddress addr = ccs.getResultDirectoryAddress();
+            callback.setValue(addr);
+        } catch (Exception e) {
+            callback.setException(e);
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
index ce82ec8..68382f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -20,12 +20,12 @@
 
 import java.util.Arrays;
 
-import org.apache.hyracks.api.dataset.DatasetDirectoryRecord;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultDirectoryRecord;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.dataset.IDatasetDirectoryService;
+import org.apache.hyracks.control.cc.result.IResultDirectoryService;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
@@ -36,12 +36,12 @@
 
     private final ResultSetId rsId;
 
-    private final DatasetDirectoryRecord[] knownRecords;
+    private final ResultDirectoryRecord[] knownRecords;
 
-    private final IResultCallback<DatasetDirectoryRecord[]> callback;
+    private final IResultCallback<ResultDirectoryRecord[]> callback;
 
     public GetResultPartitionLocationsWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
-            DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
+            ResultDirectoryRecord[] knownRecords, IResultCallback<ResultDirectoryRecord[]> callback) {
         this.ccs = ccs;
         this.jobId = jobId;
         this.rsId = rsId;
@@ -51,7 +51,7 @@
 
     @Override
     public void doRun() {
-        final IDatasetDirectoryService dds = ccs.getDatasetDirectoryService();
+        final IResultDirectoryService dds = ccs.getResultDirectoryService();
         ccs.getExecutor().execute(new Runnable() {
             @Override
             public void run() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultStatusWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultStatusWork.java
index 676276a..560f505 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultStatusWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultStatusWork.java
@@ -18,8 +18,8 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import org.apache.hyracks.api.dataset.DatasetJobRecord.Status;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultJobRecord.Status;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -46,7 +46,7 @@
     @Override
     public void doRun() {
         try {
-            Status status = ccs.getDatasetDirectoryService().getResultStatus(jobId, rsId);
+            Status status = ccs.getResultDirectoryService().getResultStatus(jobId, rsId);
             callback.setValue(status);
         } catch (HyracksDataException e) {
             callback.setException(e);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
index f847cdb..77d2f82 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
@@ -52,10 +52,14 @@
     @Override
     public void run() {
         if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Cleanup for JobRun with id: " + jobId);
+            LOGGER.info("Cleanup for job: {}", jobId);
+        }
+        final JobRun jobRun = jobManager.get(jobId);
+        if (jobRun == null) {
+            LOGGER.debug("Ignoring cleanup for unknown job: {}", jobId);
+            return;
         }
         try {
-            JobRun jobRun = jobManager.get(jobId);
             jobManager.prepareComplete(jobRun, status, exceptions);
             callback.setValue(null);
         } catch (HyracksException e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index cc37f9c..727793b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -37,15 +37,11 @@
 public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
     private static final Logger LOGGER = LogManager.getLogger();
 
-    private ClusterControllerService ccs;
     private JobId jobId;
-    private String nodeId;
 
     public JobletCleanupNotificationWork(ClusterControllerService ccs, JobId jobId, String nodeId) {
         super(ccs, nodeId, null);
-        this.ccs = ccs;
         this.jobId = jobId;
-        this.nodeId = nodeId;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
index 5c98035..b772ef9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NodeHeartbeatWork.java
@@ -18,19 +18,39 @@
  */
 package org.apache.hyracks.control.cc.work;
 
+import java.net.InetSocketAddress;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatData;
+import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import org.apache.logging.log4j.Level;
 
 public class NodeHeartbeatWork extends AbstractHeartbeatWork {
 
-    public NodeHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
+    private final InetSocketAddress ncAddress;
+
+    public NodeHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData,
+            InetSocketAddress ncAddress) {
         super(ccs, nodeId, hbData);
+        this.ncAddress = ncAddress;
     }
 
     @Override
-    public void runWork() {
-
+    public void runWork() throws Exception {
+        INodeManager nodeManager = ccs.getNodeManager();
+        final NodeControllerState ncState = nodeManager.getNodeControllerState(nodeId);
+        if (ncState != null) {
+            ncState.getNodeController().heartbeatAck(ccs.getCcId(), null);
+        } else {
+            // unregistered nc- let him know
+            NodeControllerRemoteProxy nc =
+                    new NodeControllerRemoteProxy(ccs.getCcId(), ccs.getClusterIPC().getReconnectingHandle(ncAddress));
+            nc.heartbeatAck(ccs.getCcId(), HyracksDataException.create(ErrorCode.NO_SUCH_NODE, nodeId));
+        }
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
index 62c19bb..80aae39 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
@@ -31,16 +31,12 @@
  */
 public class NotifyDeployBinaryWork extends AbstractHeartbeatWork {
 
-    private final ClusterControllerService ccs;
-    private final String nodeId;
     private final DeploymentId deploymentId;
     private DeploymentStatus deploymentStatus;
 
     public NotifyDeployBinaryWork(ClusterControllerService ccs, DeploymentId deploymentId, String nodeId,
             DeploymentStatus deploymentStatus) {
         super(ccs, nodeId, null);
-        this.ccs = ccs;
-        this.nodeId = nodeId;
         this.deploymentId = deploymentId;
         this.deploymentStatus = deploymentStatus;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index de7d941..fe33bc9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -28,11 +28,8 @@
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
-import org.apache.hyracks.control.common.ipc.CCNCFunctions;
 import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
-import org.apache.hyracks.ipc.api.IIPCHandle;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -52,36 +49,31 @@
     @Override
     protected void doRun() throws Exception {
         String id = reg.getNodeId();
-        // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
-        IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
-        CCNCFunctions.NodeRegistrationResult result;
-        Map<IOption, Object> ncConfiguration = new HashMap<>();
+        LOGGER.info("registering node: {}", id);
+        NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
+                ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
+        INodeManager nodeManager = ccs.getNodeManager();
+        NodeParameters params = new NodeParameters();
+        params.setClusterControllerInfo(ccs.getClusterControllerInfo());
+        params.setDistributedState(ccs.getContext().getDistributedState());
+        params.setHeartbeatPeriod(ccs.getCCConfig().getHeartbeatPeriodMillis());
+        params.setProfileDumpPeriod(ccs.getCCConfig().getProfileDumpPeriod());
+        params.setRegistrationId(registrationId);
         try {
-            LOGGER.log(Level.WARN, "Registering INodeController: id = " + id);
-            NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
-                    ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
             NodeControllerState state = new NodeControllerState(nc, reg);
-            INodeManager nodeManager = ccs.getNodeManager();
             nodeManager.addNode(id, state);
             IApplicationConfig cfg = state.getNCConfig().getConfigManager().getNodeEffectiveConfig(id);
+            final Map<IOption, Object> ncConfiguration = new HashMap<>();
             for (IOption option : cfg.getOptions()) {
                 ncConfiguration.put(option, cfg.get(option));
             }
-            LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
-            NodeParameters params = new NodeParameters();
-            params.setClusterControllerInfo(ccs.getClusterControllerInfo());
-            params.setDistributedState(ccs.getContext().getDistributedState());
-            params.setHeartbeatPeriod(ccs.getCCConfig().getHeartbeatPeriodMillis());
-            params.setProfileDumpPeriod(ccs.getCCConfig().getProfileDumpPeriod());
-            params.setRegistrationId(registrationId);
-            result = new CCNCFunctions.NodeRegistrationResult(params, null);
+            LOGGER.info("registered node: {}", id);
+            nc.sendRegistrationResult(params, null);
+            ccs.getContext().notifyNodeJoin(id, ncConfiguration);
         } catch (Exception e) {
-            LOGGER.log(Level.WARN, "Node registration failed", e);
-            result = new CCNCFunctions.NodeRegistrationResult(null, e);
+            LOGGER.error("node {} registration failed", id, e);
+            nodeManager.removeNode(id);
+            nc.sendRegistrationResult(params, e);
         }
-        LOGGER.warn("sending registration response to node");
-        ncIPCHandle.send(-1, result, null);
-        LOGGER.warn("notifying node join");
-        ccs.getContext().notifyNodeJoin(id, ncConfiguration);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
index ad36701..98fe722 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
@@ -22,7 +22,7 @@
 import java.util.List;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -68,7 +68,7 @@
     @Override
     public void run() {
         try {
-            ccs.getDatasetDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, emptyResult,
+            ccs.getResultDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, emptyResult,
                     partition, nPartitions, networkAddress);
         } catch (HyracksDataException e) {
             LOGGER.log(Level.WARN, "Failed to register partition location", e);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
index d63bc8a..b36c917 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -44,7 +44,7 @@
     @Override
     public void run() {
         try {
-            ccs.getDatasetDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+            ccs.getResultDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
         } catch (HyracksDataException e) {
             throw new RuntimeException(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
index e2f8b0d..0c53142 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
@@ -43,6 +43,9 @@
     protected void performEvent(TaskAttempt ta) {
         IJobManager jobManager = ccs.getJobManager();
         JobRun run = jobManager.get(jobId);
+        if (run == null) {
+            return;
+        }
         if (statistics != null) {
             JobProfile jobProfile = run.getJobProfile();
             Map<String, JobletProfile> jobletProfiles = jobProfile.getJobletProfiles();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
index a2be15c..833066e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
@@ -22,6 +22,7 @@
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
@@ -42,10 +43,15 @@
 
     @Override
     protected void performEvent(TaskAttempt ta) {
-        LOGGER.log(Level.WARN, "Executing task failure work for " + this, exceptions.get(0));
+        Exception ex = exceptions.get(0);
+        LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN,
+                "Executing task failure work for " + this, ex);
         IJobManager jobManager = ccs.getJobManager();
         JobRun run = jobManager.get(jobId);
-        ccs.getDatasetDirectoryService().reportJobFailure(jobId, exceptions);
+        if (run == null) {
+            return;
+        }
+        ccs.getResultDirectoryService().reportJobFailure(jobId, exceptions);
         run.getExecutor().notifyTaskFailure(ta, exceptions);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
index 143c8c1..69b55ed 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
@@ -28,10 +28,10 @@
 public class UndeployJobSpecWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final DeployedJobSpecId deployedJobSpecId;
-    private final IResultCallback<DeployedJobSpecId> callback;
+    private final IResultCallback<Void> callback;
 
     public UndeployJobSpecWork(ClusterControllerService ccs, DeployedJobSpecId deployedJobSpecId,
-            IResultCallback<DeployedJobSpecId> callback) {
+            IResultCallback<Void> callback) {
         this.deployedJobSpecId = deployedJobSpecId;
         this.ccs = ccs;
         this.callback = callback;
@@ -45,7 +45,7 @@
             for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
                 node.getNodeController().undeployJobSpec(deployedJobSpecId);
             }
-            callback.setValue(deployedJobSpecId);
+            callback.setValue(null);
         } catch (Exception e) {
             callback.setException(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 53998aa..911bedf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -50,6 +50,8 @@
                 @Override
                 public void run() {
                     try {
+                        Thread.currentThread()
+                                .setName(Thread.currentThread().getName() + " : WaitForCompletionForJobId: " + jobId);
                         jobRun.waitForCompletion();
                         callback.setValue(null);
                     } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
index 9959e34..9d755a0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -56,7 +56,7 @@
         final CCConfig ccConfig = makeCCConfig();
         final int coresMultiplier = 1;
         ccConfig.setCoresMultiplier(coresMultiplier);
-        INodeManager nodeManager = new NodeManager(mockCcs(), ccConfig, resourceManager);
+        INodeManager nodeManager = new NodeManager(mockCcs(), ccConfig, resourceManager, nodeId -> true);
         NodeControllerState ncState1 = mockNodeControllerState(NODE1, false);
         NodeControllerState ncState2 = mockNodeControllerState(NODE2, false);
 
@@ -84,7 +84,7 @@
         final CCConfig ccConfig = makeCCConfig();
         final int coresMultiplier = 3;
         ccConfig.setCoresMultiplier(coresMultiplier);
-        INodeManager nodeManager = new NodeManager(mockCcs(), ccConfig, resourceManager);
+        INodeManager nodeManager = new NodeManager(mockCcs(), ccConfig, resourceManager, nodeId -> true);
         NodeControllerState ncState1 = mockNodeControllerState(NODE1, false);
         NodeControllerState ncState2 = mockNodeControllerState(NODE2, false);
 
@@ -113,7 +113,7 @@
     @Test
     public void testException() throws HyracksException, IPCException {
         IResourceManager resourceManager = new ResourceManager();
-        INodeManager nodeManager = new NodeManager(mockCcs(), makeCCConfig(), resourceManager);
+        INodeManager nodeManager = new NodeManager(mockCcs(), makeCCConfig(), resourceManager, nodeId -> true);
         NodeControllerState ncState1 = mockNodeControllerState(NODE1, true);
 
         boolean invalidNetworkAddress = false;
@@ -134,7 +134,6 @@
         IPCSystem ipcSystem = Mockito.mock(IPCSystem.class);
         IIPCHandle ipcHandle = Mockito.mock(IIPCHandle.class);
         Mockito.when(ccs.getClusterIPC()).thenReturn(ipcSystem);
-        Mockito.when(ipcSystem.getHandle(Mockito.any())).thenReturn(ipcHandle);
         Mockito.when(ipcSystem.getHandle(Mockito.any(), Mockito.anyInt())).thenReturn(ipcHandle);
         Mockito.when(ccs.getExecutor()).thenReturn(Executors.newCachedThreadPool());
         return ccs;
@@ -143,7 +142,7 @@
     @Test
     public void testNullNode() throws HyracksException {
         IResourceManager resourceManager = new ResourceManager();
-        INodeManager nodeManager = new NodeManager(null, makeCCConfig(), resourceManager);
+        INodeManager nodeManager = new NodeManager(null, makeCCConfig(), resourceManager, nodeId -> true);
 
         boolean invalidParameter = false;
         // Verifies states after a failure during adding nodes.
@@ -173,7 +172,7 @@
         NetworkAddress msgAddr = new NetworkAddress(ipAddr, 1003);
         when(ncState.getCapacity()).thenReturn(new NodeCapacity(NODE_MEMORY_SIZE, NODE_CORES));
         when(ncState.getDataPort()).thenReturn(dataAddr);
-        when(ncState.getDatasetPort()).thenReturn(resultAddr);
+        when(ncState.getResultPort()).thenReturn(resultAddr);
         when(ncState.getMessagingPort()).thenReturn(msgAddr);
         NCConfig ncConfig = new NCConfig(nodeId);
         ncConfig.setDataPublicAddress(ipAddr);
@@ -218,4 +217,26 @@
         Assert.assertTrue(nodeNotExist);
     }
 
+    @Test
+    public void testUnauthorized() throws HyracksException, IPCException {
+        IResourceManager resourceManager = new ResourceManager();
+        final CCConfig ccConfig = makeCCConfig();
+        INodeManager nodeManager = new NodeManager(mockCcs(), ccConfig, resourceManager, NODE1::equals);
+        NodeControllerState ncState1 = mockNodeControllerState(NODE1, false);
+        NodeControllerState ncState2 = mockNodeControllerState(NODE2, false);
+
+        nodeManager.addNode(NODE1, ncState1);
+        boolean nodeNotExist = false;
+        try {
+            nodeManager.addNode(NODE2, ncState2);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+        Assert.assertTrue(nodeManager.getIpAddressNodeNameMap().size() == 1);
+        Assert.assertTrue(nodeManager.getAllNodeIds().size() == 1);
+        Assert.assertTrue(nodeManager.getAllNodeControllerStates().size() == 1);
+        Assert.assertTrue(nodeManager.getNodeControllerState(NODE1) == ncState1);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
index ddab504..1926da6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/NodeControllerData.java
@@ -46,7 +46,7 @@
 
     private final NetworkAddress dataPort;
 
-    private final NetworkAddress datasetPort;
+    private final NetworkAddress resultPort;
 
     private final NetworkAddress messagingPort;
 
@@ -118,13 +118,13 @@
 
     private final long[] netSignalingBytesWritten;
 
-    private final long[] datasetNetPayloadBytesRead;
+    private final long[] resultNetPayloadBytesRead;
 
-    private final long[] datasetNetPayloadBytesWritten;
+    private final long[] resultNetPayloadBytesWritten;
 
-    private final long[] datasetNetSignalingBytesRead;
+    private final long[] resultNetSignalingBytesRead;
 
-    private final long[] datasetNetSignalingBytesWritten;
+    private final long[] resultNetSignalingBytesWritten;
 
     private final long[] ipcMessagesSent;
 
@@ -147,7 +147,7 @@
     public NodeControllerData(NodeRegistration reg) {
         ncConfig = reg.getNCConfig();
         dataPort = reg.getDataPort();
-        datasetPort = reg.getDatasetPort();
+        resultPort = reg.getResultPort();
         messagingPort = reg.getMessagingPort();
         activeJobIds = new HashSet<>();
 
@@ -191,10 +191,10 @@
         netPayloadBytesWritten = new long[RRD_SIZE];
         netSignalingBytesRead = new long[RRD_SIZE];
         netSignalingBytesWritten = new long[RRD_SIZE];
-        datasetNetPayloadBytesRead = new long[RRD_SIZE];
-        datasetNetPayloadBytesWritten = new long[RRD_SIZE];
-        datasetNetSignalingBytesRead = new long[RRD_SIZE];
-        datasetNetSignalingBytesWritten = new long[RRD_SIZE];
+        resultNetPayloadBytesRead = new long[RRD_SIZE];
+        resultNetPayloadBytesWritten = new long[RRD_SIZE];
+        resultNetSignalingBytesRead = new long[RRD_SIZE];
+        resultNetSignalingBytesWritten = new long[RRD_SIZE];
         ipcMessagesSent = new long[RRD_SIZE];
         ipcMessageBytesSent = new long[RRD_SIZE];
         ipcMessagesReceived = new long[RRD_SIZE];
@@ -231,10 +231,10 @@
         netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
         netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
         netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
-        datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
-        datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
-        datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
-        datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
+        resultNetPayloadBytesRead[rrdPtr] = hbData.resultNetPayloadBytesRead;
+        resultNetPayloadBytesWritten[rrdPtr] = hbData.resultNetPayloadBytesWritten;
+        resultNetSignalingBytesRead[rrdPtr] = hbData.resultNetSignalingBytesRead;
+        resultNetSignalingBytesWritten[rrdPtr] = hbData.resultNetSignalingBytesWritten;
         ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
         ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
         ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
@@ -264,8 +264,8 @@
         return dataPort;
     }
 
-    public NetworkAddress getDatasetPort() {
-        return datasetPort;
+    public NetworkAddress getResultPort() {
+        return resultPort;
     }
 
     public NetworkAddress getMessagingPort() {
@@ -332,10 +332,10 @@
             put(o, "net-payload-bytes-written", netPayloadBytesWritten);
             put(o, "net-signaling-bytes-read", netSignalingBytesRead);
             put(o, "net-signaling-bytes-written", netSignalingBytesWritten);
-            put(o, "dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
-            put(o, "dataset-net-payload-bytes-written", datasetNetPayloadBytesWritten);
-            put(o, "dataset-net-signaling-bytes-read", datasetNetSignalingBytesRead);
-            put(o, "dataset-net-signaling-bytes-written", datasetNetSignalingBytesWritten);
+            put(o, "result-net-payload-bytes-read", resultNetPayloadBytesRead);
+            put(o, "result-net-payload-bytes-written", resultNetPayloadBytesWritten);
+            put(o, "result-net-signaling-bytes-read", resultNetSignalingBytesRead);
+            put(o, "result-net-signaling-bytes-written", resultNetSignalingBytesWritten);
             put(o, "ipc-messages-sent", ipcMessagesSent);
             put(o, "ipc-message-bytes-sent", ipcMessageBytesSent);
             put(o, "ipc-messages-received", ipcMessagesReceived);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
index 1ee2315..5da62f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.config.IApplicationConfig;
 import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
 import org.apache.hyracks.api.job.JobSerializerDeserializerContainer;
 import org.apache.hyracks.api.messages.IMessageBroker;
@@ -35,6 +36,7 @@
     protected Serializable distributedState;
     protected IMessageBroker messageBroker;
     protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
+    protected IPersistedResourceRegistry persistedResourceRegistry;
 
     public ServiceContext(ServerContext serverCtx, IApplicationConfig appConfig, ThreadFactory threadFactory) {
         this.serverCtx = serverCtx;
@@ -76,4 +78,14 @@
     public IApplicationConfig getAppConfig() {
         return appConfig;
     }
+
+    @Override
+    public void setPersistedResourceRegistry(IPersistedResourceRegistry persistedResourceRegistry) {
+        this.persistedResourceRegistry = persistedResourceRegistry;
+    }
+
+    @Override
+    public IPersistedResourceRegistry getPersistedResourceRegistry() {
+        return persistedResourceRegistry;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/IClusterController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/IClusterController.java
index 6230f1d..c811169 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/IClusterController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/IClusterController.java
@@ -18,14 +18,15 @@
  */
 package org.apache.hyracks.control.common.base;
 
+import java.net.InetSocketAddress;
 import java.util.List;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
 import org.apache.hyracks.control.common.deployment.DeploymentStatus;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatData;
@@ -54,7 +55,7 @@
 
     void notifyShutdown(String nodeId) throws Exception;
 
-    void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception;
+    void nodeHeartbeat(String id, HeartbeatData hbData, InetSocketAddress ncAddress) throws Exception;
 
     void reportProfile(String id, List<JobProfile> profiles) throws Exception;
 
@@ -72,4 +73,6 @@
     void getNodeControllerInfos() throws Exception;
 
     void notifyThreadDump(String nodeId, String requestId, String threadDumpJSON) throws Exception;
+
+    void notifyPingResponse(String nodeId) throws Exception;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
index 92764a7..42a0d66 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
@@ -24,16 +24,20 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import org.apache.hyracks.api.deployment.DeploymentId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
+import org.apache.hyracks.ipc.exceptions.IPCException;
 
 public interface INodeController {
     void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
@@ -51,7 +55,8 @@
 
     void undeployBinary(DeploymentId deploymentId) throws Exception;
 
-    void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception;
+    void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean checkForDuplicate)
+            throws Exception;
 
     void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
 
@@ -62,4 +67,40 @@
     void sendApplicationMessageToNC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception;
 
     void takeThreadDump(String requestId) throws Exception;
+
+    /**
+     * Sends a request to this {@link INodeController} to abort all jobs
+     * started by cluster controller with id {@code ccId}
+     *
+     * @param ccId
+     * @throws IPCException
+     */
+    void abortJobs(CcId ccId) throws IPCException;
+
+    /**
+     * Sends node registration result to this {@link INodeController}.
+     *
+     * @param parameters
+     * @param regFailure
+     * @throws IPCException
+     */
+    void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException;
+
+    /**
+     * Sends a request to this {@link INodeController} to ping the
+     * cluster controller with id {@code ccId}
+     *
+     * @param ccId
+     * @throws IPCException
+     */
+    void ping(CcId ccId) throws IPCException;
+
+    /**
+     * Delivers a response to a heartbeat delivered to this {@link CcId}
+     *
+     * @param ccId
+     * @param e
+     * @throws IPCException
+     */
+    void heartbeatAck(CcId ccId, HyracksDataException e) throws IPCException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
index b49dc20..4b3934b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/ConfigManager.java
@@ -176,7 +176,7 @@
     }
 
     public synchronized void ensureNode(String nodeId) {
-        LOGGER.debug("ensureNode: " + nodeId);
+        LOGGER.trace("+ensureNode: {}", nodeId);
         Map<IOption, Object> nodeDefinedMap =
                 nodeSpecificDefinedMap.computeIfAbsent(nodeId, this::createNodeSpecificMap);
         Map<IOption, Object> nodeDefaultMap =
@@ -186,14 +186,14 @@
     }
 
     public synchronized void forgetNode(String nodeId) {
-        LOGGER.debug("forgetNode: " + nodeId);
+        LOGGER.trace("+forgetNode: {}", nodeId);
         nodeSpecificDefinedMap.remove(nodeId);
         nodeSpecificDefaultMap.remove(nodeId);
         nodeEffectiveMaps.remove(nodeId);
     }
 
     private Map<IOption, Object> createNodeSpecificMap(String nodeId) {
-        LOGGER.debug("createNodeSpecificMap: " + nodeId);
+        LOGGER.trace("+createNodeSpecificMap: {}", nodeId);
         return Collections.synchronizedMap(new HashMap<>());
     }
 
@@ -384,6 +384,8 @@
             return map.get(option);
         } else {
             Object value = resolveDefault(option, new ConfigManagerApplicationConfig(this) {
+                private static final long serialVersionUID = -5505664489371709335L;
+
                 @Override
                 public Object getStatic(IOption option) {
                     return getOrDefault(map, option, nodeId);
@@ -453,6 +455,8 @@
         final Map<IOption, Object> nodeMap = nodeSpecificDefaultMap.get(nodeId);
         Map<IOption, Object> nodeEffectiveMap = getNodeEffectiveMap(nodeId);
         return new ConfigManagerApplicationConfig(this) {
+            private static final long serialVersionUID = 3166949269001016392L;
+
             @Override
             public Object getStatic(IOption option) {
                 if (!nodeEffectiveMap.containsKey(option)) {
@@ -582,6 +586,8 @@
     }
 
     private static class NoOpMapMutator implements CompositeMap.MapMutator<IOption, Object> {
+        private static final long serialVersionUID = -7239155627408457440L;
+
         @Override
         public Object put(CompositeMap<IOption, Object> compositeMap, Map<IOption, Object>[] maps, IOption iOption,
                 Object o) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/IConfigSetter.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/IConfigSetter.java
index 2234cca..72265f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/IConfigSetter.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/IConfigSetter.java
@@ -23,6 +23,8 @@
     void set(String nodeId, Object value, boolean isDefault) throws SetException;
 
     class SetException extends RuntimeException {
+        private static final long serialVersionUID = 700327181143546634L;
+
         public SetException(Exception e) {
             super(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
index f83df6e..57cf339 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
@@ -41,6 +41,7 @@
 
 @SuppressWarnings("SameParameterValue")
 public class CCConfig extends ControllerConfig {
+    private static final long serialVersionUID = 4118822454622201176L;
 
     public enum Option implements IOption {
         APP_CLASS(STRING, (String) null),
@@ -60,6 +61,7 @@
         CONSOLE_PUBLIC_PORT(INTEGER, CONSOLE_LISTEN_PORT),
         HEARTBEAT_PERIOD(LONG, 10000L), // TODO (mblow): add time unit
         HEARTBEAT_MAX_MISSES(INTEGER, 5),
+        DEAD_NODE_SWEEP_THRESHOLD(LONG, HEARTBEAT_PERIOD),
         PROFILE_DUMP_PERIOD(INTEGER, 0),
         JOB_HISTORY_SIZE(INTEGER, 10),
         RESULT_TTL(LONG, 86400000L), // TODO(mblow): add time unit
@@ -154,7 +156,9 @@
                 case HEARTBEAT_PERIOD:
                     return "Sets the time duration between two heartbeats from each node controller in milliseconds";
                 case HEARTBEAT_MAX_MISSES:
-                    return "Sets the maximum number of missed heartbeats before a node is marked as dead";
+                    return "Sets the maximum number of missed heartbeats before a node can be considered dead";
+                case DEAD_NODE_SWEEP_THRESHOLD:
+                    return "Sets the frequency (in milliseconds) to process nodes that can be considered dead";
                 case PROFILE_DUMP_PERIOD:
                     return "Sets the time duration between two profile dumps from each node controller in "
                             + "milliseconds; 0 to disable";
@@ -326,6 +330,14 @@
         configManager.set(Option.HEARTBEAT_MAX_MISSES, heartbeatMaxMisses);
     }
 
+    public long getDeadNodeSweepThreshold() {
+        return getAppConfig().getLong(Option.DEAD_NODE_SWEEP_THRESHOLD);
+    }
+
+    public void setDeadNodeSweepThreshold(long deadNodeSweepThreshold) {
+        configManager.set(Option.DEAD_NODE_SWEEP_THRESHOLD, deadNodeSweepThreshold);
+    }
+
     public int getProfileDumpPeriod() {
         return getAppConfig().getInt(Option.PROFILE_DUMP_PERIOD);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/ControllerConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/ControllerConfig.java
index 8ecd312..07e61ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/ControllerConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/ControllerConfig.java
@@ -20,6 +20,7 @@
 
 import java.io.Serializable;
 import java.net.URL;
+import java.util.function.Function;
 
 import org.apache.hyracks.api.config.IApplicationConfig;
 import org.apache.hyracks.api.config.IOption;
@@ -35,21 +36,32 @@
     private static final long serialVersionUID = 1L;
 
     public enum Option implements IOption {
-        CONFIG_FILE(OptionTypes.STRING, "Specify path to master configuration file", null),
-        CONFIG_FILE_URL(OptionTypes.URL, "Specify URL to master configuration file", null),
+        CONFIG_FILE(OptionTypes.STRING, (String) null, "Specify path to master configuration file"),
+        CONFIG_FILE_URL(OptionTypes.URL, (URL) null, "Specify URL to master configuration file"),
         DEFAULT_DIR(
                 OptionTypes.STRING,
                 "Directory where files are written to by default",
-                FileUtil.joinPath(System.getProperty(ConfigurationUtil.JAVA_IO_TMPDIR), "hyracks")),;
+                FileUtil.joinPath(System.getProperty(ConfigurationUtil.JAVA_IO_TMPDIR), "hyracks")),
+        LOG_DIR(
+                OptionTypes.STRING,
+                (Function<IApplicationConfig, String>) appConfig -> FileUtil
+                        .joinPath(appConfig.getString(ControllerConfig.Option.DEFAULT_DIR), "logs"),
+                "The directory where logs for this node are written");
 
         private final IOptionType type;
         private final String description;
-        private String defaultValue;
+        private Object defaultValue;
 
-        Option(IOptionType type, String description, String defaultValue) {
+        <T> Option(IOptionType<T> type, T defaultValue, String description) {
             this.type = type;
-            this.description = description;
             this.defaultValue = defaultValue;
+            this.description = description;
+        }
+
+        <T> Option(IOptionType<T> type, Function<IApplicationConfig, T> defaultValue, String description) {
+            this.type = type;
+            this.defaultValue = defaultValue;
+            this.description = description;
         }
 
         @Override
@@ -106,4 +118,8 @@
     public void setConfigFileUrl(URL configFileUrl) {
         configManager.set(ControllerConfig.Option.CONFIG_FILE_URL, configFileUrl);
     }
+
+    public String getLogDir() {
+        return configManager.getAppConfig().getString(ControllerConfig.Option.LOG_DIR);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index aa3260d..95c67df 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -157,13 +157,13 @@
                 case DATA_PUBLIC_PORT:
                     return "Public IP port to announce data listener";
                 case RESULT_LISTEN_ADDRESS:
-                    return "IP Address to bind dataset result distribution listener";
+                    return "IP Address to bind result distribution listener";
                 case RESULT_LISTEN_PORT:
-                    return "IP port to bind dataset result distribution listener";
+                    return "IP port to bind result distribution listener";
                 case RESULT_PUBLIC_ADDRESS:
-                    return "Public IP Address to announce dataset result distribution listener";
+                    return "Public IP Address to announce result distribution listener";
                 case RESULT_PUBLIC_PORT:
-                    return "Public IP port to announce dataset result distribution listener";
+                    return "Public IP port to announce result distribution listener";
                 case MESSAGING_LISTEN_ADDRESS:
                     return "IP Address to bind messaging listener";
                 case MESSAGING_LISTEN_PORT:
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
index c2f2e1a..b4d835d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
@@ -43,7 +43,7 @@
 
     private final NetworkAddress dataPort;
 
-    private final NetworkAddress datasetPort;
+    private final NetworkAddress resultPort;
 
     private final String osName;
 
@@ -78,12 +78,12 @@
     private final NodeCapacity capacity;
 
     public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
-            NetworkAddress datasetPort, HeartbeatSchema hbSchema, NetworkAddress messagingPort, NodeCapacity capacity) {
+            NetworkAddress resultPort, HeartbeatSchema hbSchema, NetworkAddress messagingPort, NodeCapacity capacity) {
         this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
         this.dataPort = dataPort;
-        this.datasetPort = datasetPort;
+        this.resultPort = resultPort;
         this.hbSchema = hbSchema;
         this.messagingPort = messagingPort;
         this.capacity = capacity;
@@ -122,8 +122,8 @@
         return dataPort;
     }
 
-    public NetworkAddress getDatasetPort() {
-        return datasetPort;
+    public NetworkAddress getResultPort() {
+        return resultPort;
     }
 
     public String getOSName() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/AbstractDatasetManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/AbstractDatasetManager.java
deleted file mode 100644
index f95229e..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/AbstractDatasetManager.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.hyracks.control.common.dataset;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hyracks.api.dataset.IDatasetManager;
-import org.apache.hyracks.api.dataset.IDatasetStateRecord;
-import org.apache.hyracks.api.job.JobId;
-
-public abstract class AbstractDatasetManager implements IDatasetManager {
-
-    private final long nanoResultTTL;
-
-    protected AbstractDatasetManager(long resultTTL) {
-        this.nanoResultTTL = TimeUnit.MILLISECONDS.toNanos(resultTTL);
-    }
-
-    @Override
-    public synchronized void sweepExpiredDatasets() {
-        final List<JobId> expiredDatasets = new ArrayList<>();
-        final long sweepTime = System.nanoTime();
-        for (JobId jobId : getJobIds()) {
-            final IDatasetStateRecord state = getState(jobId);
-            if (state != null && hasExpired(state, sweepTime, nanoResultTTL)) {
-                expiredDatasets.add(jobId);
-            }
-        }
-        for (JobId jobId : expiredDatasets) {
-            sweep(jobId);
-        }
-    }
-
-    private static boolean hasExpired(IDatasetStateRecord dataset, long currentTime, long ttl) {
-        return currentTime - dataset.getTimestamp() - ttl > 0;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/ResultStateSweeper.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/ResultStateSweeper.java
deleted file mode 100644
index 901ec67..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/dataset/ResultStateSweeper.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.hyracks.control.common.dataset;
-
-import org.apache.hyracks.api.dataset.IDatasetManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * Sweeper to clean up the stale result distribution files and result states.
- */
-public class ResultStateSweeper implements Runnable {
-
-    private final IDatasetManager datasetManager;
-    private final long resultSweepThreshold;
-    private final Logger logger;
-
-    public ResultStateSweeper(IDatasetManager datasetManager, long resultSweepThreshold, Logger logger) {
-        this.datasetManager = datasetManager;
-        this.resultSweepThreshold = resultSweepThreshold;
-        this.logger = logger;
-    }
-
-    @Override
-    public void run() {
-        while (!Thread.currentThread().isInterrupted()) {
-            try {
-                Thread.sleep(resultSweepThreshold);
-                datasetManager.sweepExpiredDatasets();
-                logger.trace("Result state cleanup instance successfully completed.");
-            } catch (InterruptedException e) {
-                logger.warn("Result cleaner thread interrupted, shutting down.");
-                Thread.currentThread().interrupt();
-            }
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
index 76b68bb..b8ba173 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -44,10 +44,10 @@
     public long netPayloadBytesWritten;
     public long netSignalingBytesRead;
     public long netSignalingBytesWritten;
-    public long datasetNetPayloadBytesRead;
-    public long datasetNetPayloadBytesWritten;
-    public long datasetNetSignalingBytesRead;
-    public long datasetNetSignalingBytesWritten;
+    public long resultNetPayloadBytesRead;
+    public long resultNetPayloadBytesWritten;
+    public long resultNetSignalingBytesRead;
+    public long resultNetSignalingBytesWritten;
     public long ipcMessagesSent;
     public long ipcMessageBytesSent;
     public long ipcMessagesReceived;
@@ -79,9 +79,9 @@
         netSignalingBytesRead = dis.readLong();
         netSignalingBytesWritten = dis.readLong();
         netSignalingBytesWritten = dis.readLong();
-        datasetNetPayloadBytesWritten = dis.readLong();
-        datasetNetSignalingBytesRead = dis.readLong();
-        datasetNetSignalingBytesWritten = dis.readLong();
+        resultNetPayloadBytesWritten = dis.readLong();
+        resultNetSignalingBytesRead = dis.readLong();
+        resultNetSignalingBytesWritten = dis.readLong();
         ipcMessagesSent = dis.readLong();
         ipcMessageBytesSent = dis.readLong();
         ipcMessagesReceived = dis.readLong();
@@ -119,10 +119,10 @@
         dos.writeLong(netPayloadBytesWritten);
         dos.writeLong(netSignalingBytesRead);
         dos.writeLong(netSignalingBytesWritten);
-        dos.writeLong(datasetNetPayloadBytesRead);
-        dos.writeLong(datasetNetPayloadBytesWritten);
-        dos.writeLong(datasetNetSignalingBytesRead);
-        dos.writeLong(datasetNetSignalingBytesWritten);
+        dos.writeLong(resultNetPayloadBytesRead);
+        dos.writeLong(resultNetPayloadBytesWritten);
+        dos.writeLong(resultNetSignalingBytesRead);
+        dos.writeLong(resultNetSignalingBytesWritten);
         dos.writeLong(ipcMessagesSent);
         dos.writeLong(ipcMessageBytesSent);
         dos.writeLong(ipcMessagesReceived);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
index 8e02936..1616343 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
@@ -24,8 +24,11 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 import java.net.URL;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -46,13 +49,14 @@
 import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.dataflow.connectors.ConnectorPolicyFactory;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.deployment.DeploymentId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
 import org.apache.hyracks.control.common.deployment.DeploymentStatus;
@@ -81,6 +85,7 @@
         NOTIFY_TASK_COMPLETE,
         NOTIFY_TASK_FAILURE,
         NODE_HEARTBEAT,
+        NODE_HEARTBEAT_ACK,
         REPORT_PROFILE,
         REGISTER_PARTITION_PROVIDER,
         REGISTER_PARTITION_REQUEST,
@@ -103,8 +108,8 @@
         SHUTDOWN_REQUEST,
         SHUTDOWN_RESPONSE,
 
-        DISTRIBUTE_JOB,
-        DESTROY_JOB,
+        DEPLOY_JOB,
+        UNDEPLOY_JOB,
         DEPLOYED_JOB_FAILURE,
 
         STATE_DUMP_REQUEST,
@@ -113,6 +118,9 @@
         THREAD_DUMP_REQUEST,
         THREAD_DUMP_RESPONSE,
 
+        PING_REQUEST,
+        PING_RESPONSE,
+
         OTHER
     }
 
@@ -380,10 +388,12 @@
 
         private final String nodeId;
         private final HeartbeatData hbData;
+        private final InetSocketAddress ncAddress;
 
-        public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData) {
+        public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData, InetSocketAddress ncAddress) {
             this.nodeId = nodeId;
             this.hbData = hbData;
+            this.ncAddress = ncAddress;
         }
 
         @Override
@@ -399,21 +409,27 @@
             return hbData;
         }
 
+        public InetSocketAddress getNcAddress() {
+            return ncAddress;
+        }
+
         public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
             ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
-            DataInputStream dis = new DataInputStream(bais);
+            ObjectInputStream dis = new ObjectInputStream(bais);
 
-            String nodeId = dis.readUTF();
             HeartbeatData hbData = new HeartbeatData();
             hbData.readFields(dis);
-            return new NodeHeartbeatFunction(nodeId, hbData);
+            String nodeId = dis.readUTF();
+            InetSocketAddress ncAddress = (InetSocketAddress) dis.readObject();
+            return new NodeHeartbeatFunction(nodeId, hbData, ncAddress);
         }
 
         public static void serialize(OutputStream out, Object object) throws Exception {
             NodeHeartbeatFunction fn = (NodeHeartbeatFunction) object;
-            DataOutputStream dos = new DataOutputStream(out);
-            dos.writeUTF(fn.nodeId);
+            ObjectOutputStream dos = new ObjectOutputStream(out);
             fn.hbData.write(dos);
+            dos.writeUTF(fn.nodeId);
+            dos.writeObject(fn.ncAddress);
         }
     }
 
@@ -713,15 +729,18 @@
 
         private final byte[] acgBytes;
 
-        public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, CcId ccId) {
+        private final boolean upsert;
+
+        public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, boolean upsert, CcId ccId) {
             super(ccId);
             this.deployedJobSpecId = deployedJobSpecId;
             this.acgBytes = acgBytes;
+            this.upsert = upsert;
         }
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DISTRIBUTE_JOB;
+            return FunctionId.DEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
@@ -731,6 +750,10 @@
         public byte[] getacgBytes() {
             return acgBytes;
         }
+
+        public boolean getUpsert() {
+            return upsert;
+        }
     }
 
     public static class UndeployJobSpecFunction extends CCIdentifiedFunction {
@@ -745,7 +768,7 @@
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_JOB;
+            return FunctionId.UNDEPLOY_JOB;
         }
 
         public DeployedJobSpecId getDeployedJobSpecId() {
@@ -1309,6 +1332,38 @@
         }
     }
 
+    public static class PingFunction extends CCIdentifiedFunction {
+        private static final long serialVersionUID = 1L;
+
+        public PingFunction(CcId ccId) {
+            super(ccId);
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.PING_REQUEST;
+        }
+    }
+
+    public static class NodeHeartbeatAckFunction extends CCIdentifiedFunction {
+        private static final long serialVersionUID = 1L;
+        private final HyracksDataException exception;
+
+        public NodeHeartbeatAckFunction(CcId ccId, HyracksDataException e) {
+            super(ccId);
+            exception = e;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NODE_HEARTBEAT_ACK;
+        }
+
+        public HyracksDataException getException() {
+            return exception;
+        }
+    }
+
     public static class ShutdownRequestFunction extends CCIdentifiedFunction {
         private static final long serialVersionUID = 1L;
 
@@ -1348,6 +1403,25 @@
         }
     }
 
+    public static class PingResponseFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+
+        public PingResponseFunction(String nodeId) {
+            this.nodeId = nodeId;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.PING_RESPONSE;
+        }
+    }
+
     public static class SerializerDeserializer implements IPayloadSerializerDeserializer {
         private final JavaSerializationBasedPayloadSerializerDeserializer javaSerde;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bf35e6b..13a08b2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -18,14 +18,15 @@
  */
 package org.apache.hyracks.control.common.ipc;
 
+import java.net.InetSocketAddress;
 import java.util.List;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.common.base.IClusterController;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
 import org.apache.hyracks.control.common.deployment.DeploymentStatus;
@@ -101,8 +102,8 @@
     }
 
     @Override
-    public void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
-        NodeHeartbeatFunction fn = new NodeHeartbeatFunction(id, hbData);
+    public void nodeHeartbeat(String id, HeartbeatData hbData, InetSocketAddress ncAddress) throws Exception {
+        NodeHeartbeatFunction fn = new NodeHeartbeatFunction(id, hbData, ncAddress);
         ipcHandle.send(-1, fn, null);
     }
 
@@ -175,6 +176,12 @@
     }
 
     @Override
+    public void notifyPingResponse(String nodeId) throws Exception {
+        CCNCFunctions.PingResponseFunction fn = new CCNCFunctions.PingResponseFunction(nodeId);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
     public String toString() {
         return getClass().getSimpleName() + " [" + ipcHandle.getRemoteAddress() + "]";
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index b78e53f..d32ee32 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -30,12 +30,14 @@
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import org.apache.hyracks.api.deployment.DeploymentId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.DeployedJobSpecId;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.common.base.INodeController;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.AbortTasksFunction;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.CleanupJobletFunction;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.DeployBinaryFunction;
@@ -50,6 +52,7 @@
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.UndeployJobSpecFunction;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
 import org.apache.hyracks.ipc.api.IIPCHandle;
+import org.apache.hyracks.ipc.exceptions.IPCException;
 
 public class NodeControllerRemoteProxy implements INodeController {
     private final CcId ccId;
@@ -101,8 +104,8 @@
     }
 
     @Override
-    public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception {
-        DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, ccId);
+    public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean upsert) throws Exception {
+        DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, upsert, ccId);
         ipcHandle.send(-1, fn, null);
     }
 
@@ -136,6 +139,26 @@
         ipcHandle.send(-1, fn, null);
     }
 
+    @Override
+    public void abortJobs(CcId ccId) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.AbortCCJobsFunction(ccId), null);
+    }
+
+    @Override
+    public void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.NodeRegistrationResult(parameters, regFailure), null);
+    }
+
+    @Override
+    public void ping(CcId ccId) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.PingFunction(ccId), null);
+    }
+
+    @Override
+    public void heartbeatAck(CcId ccId, HyracksDataException e) throws IPCException {
+        ipcHandle.send(-1, new CCNCFunctions.NodeHeartbeatAckFunction(ccId, e), null);
+    }
+
     public InetSocketAddress getAddress() {
         return ipcHandle.getRemoteAddress();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/OperatorStats.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/OperatorStats.java
index 718ac3d..185e197 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/OperatorStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/OperatorStats.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.control.common.job.profiling.counters.Counter;
 
 public class OperatorStats implements IOperatorStats {
+    private static final long serialVersionUID = 6401830963367567167L;
 
     public final String operatorName;
     public final ICounter tupleCounter;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
index 90cdc72..e95c107 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.job.profiling.IStatsCollector;
 
 public class StatsCollector implements IStatsCollector {
+    private static final long serialVersionUID = 6858817639895434577L;
 
     private final Map<String, IOperatorStats> operatorStatsMap = new HashMap<>();
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/counters/Counter.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/counters/Counter.java
index e914878..0bcb246 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/counters/Counter.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/counters/Counter.java
@@ -23,6 +23,8 @@
 import org.apache.hyracks.api.job.profiling.counters.ICounter;
 
 public class Counter implements ICounter {
+    private static final long serialVersionUID = -3935601595055562080L;
+
     private final String name;
     private final AtomicLong counter;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/AbstractResultManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/AbstractResultManager.java
new file mode 100644
index 0000000..e314a6d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/AbstractResultManager.java
@@ -0,0 +1,55 @@
+/*
+ * 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.hyracks.control.common.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hyracks.api.result.IResultManager;
+import org.apache.hyracks.api.result.IResultStateRecord;
+import org.apache.hyracks.api.job.JobId;
+
+public abstract class AbstractResultManager implements IResultManager {
+
+    private final long nanoResultTTL;
+
+    protected AbstractResultManager(long resultTTL) {
+        this.nanoResultTTL = TimeUnit.MILLISECONDS.toNanos(resultTTL);
+    }
+
+    @Override
+    public synchronized void sweepExpiredResultSets() {
+        final List<JobId> expiredResultSets = new ArrayList<>();
+        final long sweepTime = System.nanoTime();
+        for (JobId jobId : getJobIds()) {
+            final IResultStateRecord state = getState(jobId);
+            if (state != null && hasExpired(state, sweepTime, nanoResultTTL)) {
+                expiredResultSets.add(jobId);
+            }
+        }
+        for (JobId jobId : expiredResultSets) {
+            sweep(jobId);
+        }
+    }
+
+    private static boolean hasExpired(IResultStateRecord state, long currentTime, long ttl) {
+        return currentTime - state.getTimestamp() - ttl > 0;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/ResultStateSweeper.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/ResultStateSweeper.java
new file mode 100644
index 0000000..76bbabb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/result/ResultStateSweeper.java
@@ -0,0 +1,53 @@
+/*
+ * 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.hyracks.control.common.result;
+
+import org.apache.hyracks.api.result.IResultManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * Sweeper to clean up the stale result distribution files and result states.
+ */
+public class ResultStateSweeper implements Runnable {
+
+    private final IResultManager resultManager;
+    private final long resultSweepThreshold;
+    private final Logger logger;
+
+    public ResultStateSweeper(IResultManager resultManager, long resultSweepThreshold, Logger logger) {
+        this.resultManager = resultManager;
+        this.resultSweepThreshold = resultSweepThreshold;
+        this.logger = logger;
+    }
+
+    @Override
+    public void run() {
+        while (!Thread.currentThread().isInterrupted()) {
+            try {
+                Thread.sleep(resultSweepThreshold);
+                resultManager.sweepExpiredResultSets();
+                logger.trace("Result state cleanup instance successfully completed.");
+            } catch (InterruptedException e) {
+                logger.warn("Result cleaner thread interrupted, shutting down.");
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
index b7f3332..9ec3214 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/AbstractWork.java
@@ -23,7 +23,7 @@
 public abstract class AbstractWork implements Runnable {
 
     public Level logLevel() {
-        return Level.INFO;
+        return Level.DEBUG;
     }
 
     public String getName() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
index fb812e1..2e20aa1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/work/WorkQueue.java
@@ -83,8 +83,8 @@
         if (DEBUG) {
             LOGGER.log(Level.DEBUG, "Enqueue (" + hashCode() + "): " + enqueueCount.incrementAndGet());
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Scheduling: " + event);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Scheduling: " + event);
         }
         queue.offer(event);
     }
@@ -128,19 +128,21 @@
                 } catch (Exception e) {
                     LOGGER.log(Level.WARN, "Exception while executing " + r, e);
                 } finally {
-                    auditWaitsAndBlocks(r, before);
+                    if (LOGGER.isTraceEnabled()) {
+                        traceWaitsAndBlocks(r, before);
+                    }
                 }
             }
         }
 
-        protected void auditWaitsAndBlocks(AbstractWork r, ThreadInfo before) {
+        protected void traceWaitsAndBlocks(AbstractWork r, ThreadInfo before) {
             ThreadInfo after = threadMXBean.getThreadInfo(thread.getId());
             final long waitedDelta = after.getWaitedCount() - before.getWaitedCount();
             final long blockedDelta = after.getBlockedCount() - before.getBlockedCount();
             if (waitedDelta > 0 || blockedDelta > 0) {
-                LOGGER.warn("Work " + r + " waited " + waitedDelta + " times (~"
-                        + (after.getWaitedTime() - before.getWaitedTime()) + "ms), blocked " + blockedDelta
-                        + " times (~" + (after.getBlockedTime() - before.getBlockedTime()) + "ms)");
+                LOGGER.trace("Work {} waited {} times (~{}ms), blocked {} times (~{}ms)", r, waitedDelta,
+                        after.getWaitedTime() - before.getWaitedTime(), blockedDelta,
+                        after.getBlockedTime() - before.getBlockedTime());
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index c962029..d99b5ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -88,6 +88,10 @@
       <artifactId>log4j-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-core</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/BaseNCApplication.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/BaseNCApplication.java
index ea16032..800721c9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/BaseNCApplication.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/BaseNCApplication.java
@@ -60,7 +60,7 @@
     }
 
     @Override
-    public void onRegisterNode(CcId ccId) throws Exception {
+    public void tasksCompleted(CcId ccId) throws Exception {
         // no-op
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
index 627e972..d1f7d5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/CcConnection.java
@@ -36,13 +36,15 @@
     private static final long REGISTRATION_RESPONSE_POLL_PERIOD = TimeUnit.SECONDS.toMillis(1);
 
     private final IClusterController ccs;
+    private final InetSocketAddress ccAddress;
     private boolean registrationPending;
     private boolean registrationCompleted;
     private Exception registrationException;
     private NodeParameters nodeParameters;
 
-    CcConnection(IClusterController ccs) {
+    CcConnection(IClusterController ccs, InetSocketAddress ccAddress) {
         this.ccs = ccs;
+        this.ccAddress = ccAddress;
     }
 
     @Override
@@ -73,7 +75,7 @@
         }
         if (registrationException != null) {
             LOGGER.fatal("Registering with {} failed with exception", this, registrationException);
-            ExitUtil.halt(ExitUtil.EC_IMMEDIATE_HALT);
+            ExitUtil.halt(ExitUtil.EC_NODE_REGISTRATION_FAILURE);
         }
         return getCcId();
     }
@@ -86,19 +88,17 @@
         return nodeParameters;
     }
 
-    public synchronized void notifyConnectionRestored(NodeControllerService ncs, InetSocketAddress ccAddress)
-            throws InterruptedException {
-        if (registrationCompleted) {
-            registrationCompleted = false;
-            ncs.getExecutor().submit(() -> {
-                try {
-                    return ncs.registerNode(this, ccAddress);
-                } catch (Exception e) {
-                    LOGGER.log(Level.ERROR, "Failed registering with cc", e);
-                    throw new IllegalStateException(e);
-                }
-            });
-        }
+    public synchronized void forceReregister(NodeControllerService ncs) throws InterruptedException {
+        registrationCompleted = false;
+        ncs.getExecutor().submit(() -> {
+            try {
+                return ncs.registerNode(this);
+            } catch (Exception e) {
+                LOGGER.log(Level.ERROR, "Failed registering with cc", e);
+                throw new IllegalStateException(e);
+            }
+        });
+
         while (!registrationCompleted) {
             wait();
         }
@@ -108,4 +108,8 @@
         registrationCompleted = true;
         notifyAll();
     }
+
+    public InetSocketAddress getCcAddress() {
+        return ccAddress;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
index 55bc192..1d2b77a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
@@ -230,7 +230,7 @@
     public void close() {
         long stillAllocated = memoryAllocation.get();
         if (stillAllocated > 0) {
-            LOGGER.info(() -> "Freeing leaked " + stillAllocated + " bytes");
+            LOGGER.trace(() -> "Freeing leaked " + stillAllocated + " bytes");
             serviceCtx.getMemoryManager().deallocate(stillAllocated);
         }
         nodeController.getExecutor().execute(() -> deallocatableRegistry.close());
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCDriver.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCDriver.java
index a03e0ce..fdd271c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCDriver.java
@@ -28,6 +28,10 @@
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
 import org.kohsuke.args4j.CmdLineException;
 
 @SuppressWarnings("InfiniteLoopStatement")
@@ -44,6 +48,12 @@
             INCApplication application = getApplication(args);
             application.registerConfig(configManager);
             NCConfig ncConfig = new NCConfig(nodeId, configManager);
+            LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
+            Configuration cfg = ctx.getConfiguration();
+            NCLogConfigurationFactory logCfgFactory = new NCLogConfigurationFactory(ncConfig);
+            ConfigurationFactory.setConfigurationFactory(logCfgFactory);
+            cfg.removeLogger("Console");
+            ctx.start(logCfgFactory.getConfiguration(ctx, ConfigurationSource.NULL_SOURCE));
             final NodeControllerService ncService = new NodeControllerService(ncConfig, application);
             ncService.start();
             while (true) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCLogConfigurationFactory.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCLogConfigurationFactory.java
new file mode 100644
index 0000000..990d6c9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NCLogConfigurationFactory.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.control.nc;
+
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
+import org.apache.logging.log4j.core.config.builder.api.AppenderComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.api.ComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.api.ConfigurationBuilder;
+import org.apache.logging.log4j.core.config.builder.api.LayoutComponentBuilder;
+import org.apache.logging.log4j.core.config.builder.impl.BuiltConfiguration;
+
+import java.net.URI;
+
+public class NCLogConfigurationFactory extends ConfigurationFactory {
+    private NCConfig config;
+
+    public NCLogConfigurationFactory(NCConfig config) {
+        this.config = config;
+    }
+
+    public Configuration createConfiguration(ConfigurationBuilder<BuiltConfiguration> builder) {
+        String nodeId = config.getNodeId();
+        String logDir = config.getLogDir();
+        builder.setStatusLevel(Level.WARN);
+        builder.setConfigurationName("RollingBuilder");
+        // create a rolling file appender
+        LayoutComponentBuilder defaultLayout = builder.newLayout("PatternLayout").addAttribute("pattern",
+                "%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n");
+        ComponentBuilder triggeringPolicy = builder.newComponent("Policies")
+                .addComponent(builder.newComponent("CronTriggeringPolicy").addAttribute("schedule", "0 0 0 * * ?"))
+                .addComponent(builder.newComponent("SizeBasedTriggeringPolicy").addAttribute("size", "50M"));
+        AppenderComponentBuilder defaultRoll =
+                builder.newAppender("default", "RollingFile").addAttribute("fileName", logDir + "nc-" + nodeId + ".log")
+                        .addAttribute("filePattern", logDir + "nc-" + nodeId + "-%d{MM-dd-yy}.log.gz")
+                        .add(defaultLayout).addComponent(triggeringPolicy);
+        builder.add(defaultRoll);
+
+        // create the new logger
+        builder.add(builder.newRootLogger(Level.INFO).add(builder.newAppenderRef("default")));
+
+        LayoutComponentBuilder accessLayout = builder.newLayout("PatternLayout").addAttribute("pattern", "%m%n");
+        AppenderComponentBuilder accessRoll = builder.newAppender("access", "RollingFile")
+                .addAttribute("fileName", logDir + "access-" + nodeId + ".log")
+                .addAttribute("filePattern", logDir + "access-" + nodeId + "-%d{MM-dd-yy}.log.gz").add(accessLayout)
+                .addComponent(triggeringPolicy);
+        builder.add(accessRoll);
+        builder.add(builder.newLogger("org.apache.hyracks.http.server.CLFLogger", Level.forName("ACCESS", 550))
+                .add(builder.newAppenderRef("access")).addAttribute("additivity", false));
+
+        return builder.build();
+    }
+
+    @Override
+    public Configuration getConfiguration(final LoggerContext loggerContext, final ConfigurationSource source) {
+        return getConfiguration(loggerContext, source.toString(), null);
+    }
+
+    @Override
+    public Configuration getConfiguration(final LoggerContext loggerContext, final String name,
+            final URI configLocation) {
+        ConfigurationBuilder<BuiltConfiguration> builder = newConfigurationBuilder();
+        return createConfiguration(builder);
+    }
+
+    @Override
+    protected String[] getSupportedTypes() {
+        return new String[] { "*" };
+    }
+
+    @Override
+    public String toString() {
+        return "NCLogConfiguration";
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
index 735f7cf..3bc9710 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
@@ -20,6 +20,8 @@
 
 import org.apache.hyracks.control.common.ipc.CCNCFunctions;
 import org.apache.hyracks.control.common.ipc.CCNCFunctions.StateDumpRequestFunction;
+import org.apache.hyracks.control.nc.task.HeartbeatAckTask;
+import org.apache.hyracks.control.nc.task.PingTask;
 import org.apache.hyracks.control.nc.task.ShutdownTask;
 import org.apache.hyracks.control.nc.task.ThreadDumpTask;
 import org.apache.hyracks.control.nc.work.AbortAllJobsWork;
@@ -107,13 +109,13 @@
                 ncs.getWorkQueue().schedule(new UnDeployBinaryWork(ncs, ndbf.getDeploymentId(), ndbf.getCcId()));
                 return;
 
-            case DISTRIBUTE_JOB:
+            case DEPLOY_JOB:
                 CCNCFunctions.DeployJobSpecFunction djf = (CCNCFunctions.DeployJobSpecFunction) fn;
-                ncs.getWorkQueue().schedule(
-                        new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(), djf.getCcId()));
+                ncs.getWorkQueue().schedule(new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(),
+                        djf.getUpsert(), djf.getCcId()));
                 return;
 
-            case DESTROY_JOB:
+            case UNDEPLOY_JOB:
                 CCNCFunctions.UndeployJobSpecFunction dsjf = (CCNCFunctions.UndeployJobSpecFunction) fn;
                 ncs.getWorkQueue().schedule(new UndeployJobSpecWork(ncs, dsjf.getDeployedJobSpecId(), dsjf.getCcId()));
                 return;
@@ -133,6 +135,16 @@
                 ncs.getExecutor().submit(new ThreadDumpTask(ncs, tdrf.getRequestId(), tdrf.getCcId()));
                 return;
 
+            case PING_REQUEST:
+                final CCNCFunctions.PingFunction pcf = (CCNCFunctions.PingFunction) fn;
+                ncs.getExecutor().submit(new PingTask(ncs, pcf.getCcId()));
+                return;
+
+            case NODE_HEARTBEAT_ACK:
+                final CCNCFunctions.NodeHeartbeatAckFunction nbaf = (CCNCFunctions.NodeHeartbeatAckFunction) fn;
+                ncs.getExecutor().submit(new HeartbeatAckTask(ncs, nbaf.getCcId(), nbaf.getException()));
+                return;
+
             default:
                 throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index 47ed342..653d6e01 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -46,7 +46,6 @@
 import org.apache.hyracks.api.client.NodeStatus;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.control.CcId;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -57,9 +56,9 @@
 import org.apache.hyracks.api.job.JobParameterByteStore;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.api.util.CleanupUtils;
-import org.apache.hyracks.api.util.InvokeUtil;
 import org.apache.hyracks.control.common.NodeControllerData;
 import org.apache.hyracks.control.common.base.IClusterController;
 import org.apache.hyracks.control.common.config.ConfigManager;
@@ -74,15 +73,15 @@
 import org.apache.hyracks.control.common.work.FutureValue;
 import org.apache.hyracks.control.common.work.WorkQueue;
 import org.apache.hyracks.control.nc.application.NCServiceContext;
-import org.apache.hyracks.control.nc.dataset.DatasetPartitionManager;
 import org.apache.hyracks.control.nc.heartbeat.HeartbeatComputeTask;
-import org.apache.hyracks.control.nc.heartbeat.HeartbeatTask;
+import org.apache.hyracks.control.nc.heartbeat.HeartbeatManager;
 import org.apache.hyracks.control.nc.io.IOManager;
-import org.apache.hyracks.control.nc.net.DatasetNetworkManager;
 import org.apache.hyracks.control.nc.net.MessagingNetworkManager;
 import org.apache.hyracks.control.nc.net.NetworkManager;
+import org.apache.hyracks.control.nc.net.ResultNetworkManager;
 import org.apache.hyracks.control.nc.partitions.PartitionManager;
 import org.apache.hyracks.control.nc.resources.memory.MemoryManager;
+import org.apache.hyracks.control.nc.result.ResultPartitionManager;
 import org.apache.hyracks.control.nc.work.AbortAllJobsWork;
 import org.apache.hyracks.control.nc.work.BuildJobProfilesWork;
 import org.apache.hyracks.ipc.api.IIPCEventListener;
@@ -118,9 +117,9 @@
 
     private NetworkManager netManager;
 
-    private IDatasetPartitionManager datasetPartitionManager;
+    private IResultPartitionManager resultPartitionManager;
 
-    private DatasetNetworkManager datasetNetworkManager;
+    private ResultNetworkManager resultNetworkManager;
 
     private final WorkQueue workQueue;
 
@@ -144,7 +143,7 @@
 
     private ExecutorService executor;
 
-    private Map<CcId, Thread> heartbeatThreads = new ConcurrentHashMap<>();
+    private Map<CcId, HeartbeatManager> heartbeatManagers = new ConcurrentHashMap<>();
 
     private Map<CcId, Timer> ccTimers = new ConcurrentHashMap<>();
 
@@ -262,10 +261,10 @@
     }
 
     private void init() {
-        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.getResultManagerMemory(),
+        resultPartitionManager = new ResultPartitionManager(this, executor, ncConfig.getResultManagerMemory(),
                 ncConfig.getResultTTL(), ncConfig.getResultSweepThreshold());
-        datasetNetworkManager = new DatasetNetworkManager(ncConfig.getResultListenAddress(),
-                ncConfig.getResultListenPort(), datasetPartitionManager, ncConfig.getNetThreadCount(),
+        resultNetworkManager = new ResultNetworkManager(ncConfig.getResultListenAddress(),
+                ncConfig.getResultListenPort(), resultPartitionManager, ncConfig.getNetThreadCount(),
                 ncConfig.getNetBufferCount(), ncConfig.getResultPublicAddress(), ncConfig.getResultPublicPort(),
                 FullFrameChannelInterfaceFactory.INSTANCE);
         if (ncConfig.getMessagingListenAddress() != null && serviceCtx.getMessagingChannelInterfaceFactory() != null) {
@@ -289,7 +288,7 @@
         netManager.start();
         startApplication();
         init();
-        datasetNetworkManager.start();
+        resultNetworkManager.start();
         if (messagingNetManager != null) {
             messagingNetManager.start();
         }
@@ -323,11 +322,11 @@
         }
         HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
 
-        NetworkAddress datasetAddress = datasetNetworkManager.getPublicNetworkAddress();
+        NetworkAddress resultAddress = resultNetworkManager.getPublicNetworkAddress();
         NetworkAddress netAddress = netManager.getPublicNetworkAddress();
         NetworkAddress messagingAddress =
                 messagingNetManager != null ? messagingNetManager.getPublicNetworkAddress() : null;
-        nodeRegistration = new NodeRegistration(ncAddress, id, ncConfig, netAddress, datasetAddress, hbSchema,
+        nodeRegistration = new NodeRegistration(ncAddress, id, ncConfig, netAddress, resultAddress, hbSchema,
                 messagingAddress, application.getCapacity());
 
         ncData = new NodeControllerData(nodeRegistration);
@@ -348,7 +347,7 @@
                     // we need to re-register in case of NC -> CC connection reset
                     final CcConnection ccConnection = getCcConnection(ccAddressMap.get(ccAddress));
                     try {
-                        ccConnection.notifyConnectionRestored(NodeControllerService.this, ccAddress);
+                        ccConnection.forceReregister(NodeControllerService.this);
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new IPCException(e);
@@ -357,7 +356,7 @@
             };
             ClusterControllerRemoteProxy ccProxy = new ClusterControllerRemoteProxy(
                     ipc.getHandle(ccAddress, ncConfig.getClusterConnectRetries(), 1, ipcEventListener));
-            return registerNode(new CcConnection(ccProxy), ccAddress);
+            return registerNode(new CcConnection(ccProxy, ccAddress));
         }
     }
 
@@ -395,8 +394,10 @@
                         () -> String.valueOf(e));
             }
             getWorkQueue().scheduleAndSync(new AbortAllJobsWork(this, ccId));
-            Thread hbThread = heartbeatThreads.remove(ccId);
-            hbThread.interrupt();
+            HeartbeatManager hbMgr = heartbeatManagers.remove(ccId);
+            if (hbMgr != null) {
+                hbMgr.shutdown();
+            }
             Timer ccTimer = ccTimers.remove(ccId);
             if (ccTimer != null) {
                 ccTimer.cancel();
@@ -406,13 +407,13 @@
         }
     }
 
-    public CcId registerNode(CcConnection ccc, InetSocketAddress ccAddress) throws Exception {
+    public CcId registerNode(CcConnection ccc) throws Exception {
         LOGGER.info("Registering with Cluster Controller {}", ccc);
         int registrationId = nextRegistrationId.incrementAndGet();
         pendingRegistrations.put(registrationId, ccc);
         CcId ccId = ccc.registerNode(nodeRegistration, registrationId);
         ccMap.put(ccId, ccc);
-        ccAddressMap.put(ccAddress, ccId);
+        ccAddressMap.put(ccc.getCcAddress(), ccId);
         Serializable distributedState = ccc.getNodeParameters().getDistributedState();
         if (distributedState != null) {
             getDistributedState().put(ccId, distributedState);
@@ -420,15 +421,8 @@
         IClusterController ccs = ccc.getClusterControllerService();
         NodeParameters nodeParameters = ccc.getNodeParameters();
         // Start heartbeat generator.
-        if (!heartbeatThreads.containsKey(ccId)) {
-            Thread heartbeatThread = new Thread(
-                    new HeartbeatTask(getId(), hbTask.getHeartbeatData(), ccs, nodeParameters.getHeartbeatPeriod()),
-                    id + "-Heartbeat");
-            heartbeatThread.setPriority(Thread.MAX_PRIORITY);
-            heartbeatThread.setDaemon(true);
-            heartbeatThread.start();
-            heartbeatThreads.put(ccId, heartbeatThread);
-        }
+        heartbeatManagers.computeIfAbsent(ccId, newCcId -> HeartbeatManager.init(this, ccc, hbTask.getHeartbeatData(),
+                nodeRegistration.getNodeControllerAddress()));
         if (!ccTimers.containsKey(ccId) && nodeParameters.getProfileDumpPeriod() > 0) {
             Timer ccTimer = new Timer("Timer-" + ccId, true);
             // Schedule profile dump generator.
@@ -463,7 +457,6 @@
     }
 
     private ConcurrentHashMap<CcId, Serializable> getDistributedState() {
-        //noinspection unchecked
         return (ConcurrentHashMap<CcId, Serializable>) serviceCtx.getDistributedState();
     }
 
@@ -495,9 +488,9 @@
                 LOGGER.log(Level.ERROR, "Some jobs failed to exit, continuing with abnormal shutdown");
             }
             partitionManager.close();
-            datasetPartitionManager.close();
+            resultPartitionManager.close();
             netManager.stop();
-            datasetNetworkManager.stop();
+            resultNetworkManager.stop();
             if (messagingNetManager != null) {
                 messagingNetManager.stop();
             }
@@ -507,10 +500,7 @@
              * Stop heartbeats only after NC has stopped to avoid false node failure detection
              * on CC if an NC takes a long time to stop.
              */
-            heartbeatThreads.values().parallelStream().forEach(t -> {
-                t.interrupt();
-                InvokeUtil.doUninterruptibly(() -> t.join(1000));
-            });
+            heartbeatManagers.values().parallelStream().forEach(HeartbeatManager::shutdown);
             synchronized (ccLock) {
                 ccMap.values().parallelStream().forEach(cc -> {
                     try {
@@ -559,9 +549,6 @@
 
     public void storeActivityClusterGraph(DeployedJobSpecId deployedJobSpecId, ActivityClusterGraph acg)
             throws HyracksException {
-        if (deployedJobSpecActivityClusterGraphMap.get(deployedJobSpecId.getId()) != null) {
-            throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
-        }
         deployedJobSpecActivityClusterGraphMap.put(deployedJobSpecId.getId(), acg);
     }
 
@@ -586,8 +573,8 @@
         return netManager;
     }
 
-    public DatasetNetworkManager getDatasetNetworkManager() {
-        return datasetNetworkManager;
+    public ResultNetworkManager getResultNetworkManager() {
+        return resultNetworkManager;
     }
 
     public PartitionManager getPartitionManager() {
@@ -649,8 +636,8 @@
         getClusterController(ccId).sendApplicationMessageToCC(data, deploymentId, id);
     }
 
-    public IDatasetPartitionManager getDatasetPartitionManager() {
-        return datasetPartitionManager;
+    public IResultPartitionManager getResultPartitionManager() {
+        return resultPartitionManager;
     }
 
     public MessagingNetworkManager getMessagingNetworkManager() {
@@ -659,7 +646,7 @@
 
     public void notifyTasksCompleted(CcId ccId) throws Exception {
         partitionManager.jobsCompleted(ccId);
-        application.onRegisterNode(ccId);
+        application.tasksCompleted(ccId);
     }
 
     private static INCApplication getApplication(NCConfig config)
@@ -677,6 +664,14 @@
         return application.getApplicationContext();
     }
 
+    public HeartbeatManager getHeartbeatManager(CcId ccId) {
+        return heartbeatManagers.get(ccId);
+    }
+
+    public NodeRegistration getNodeRegistration() {
+        return nodeRegistration;
+    }
+
     private class ProfileDumpTask extends TimerTask {
         private final IClusterController cc;
         private final CcId ccId;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index d7b930c..252fe97 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -43,7 +43,6 @@
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.state.IStateObject;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -57,6 +56,7 @@
 import org.apache.hyracks.api.job.profiling.counters.ICounterContext;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.common.job.PartitionState;
@@ -290,7 +290,7 @@
                     .schedule(new NotifyTaskFailureWork(ncs, this, exceptions, joblet.getJobId(), taskAttemptId));
             return;
         }
-        ct.setName(displayName + ":" + taskAttemptId + ":" + 0);
+        ct.setName(displayName + ":" + joblet.getJobId() + ":" + taskAttemptId + ":" + 0);
         try {
             Throwable operatorException = null;
             try {
@@ -309,7 +309,8 @@
                                 if (!addPendingThread(thread)) {
                                     return;
                                 }
-                                thread.setName(displayName + ":" + taskAttemptId + ":" + cIdx);
+                                thread.setName(
+                                        displayName + ":" + joblet.getJobId() + ":" + taskAttemptId + ":" + cIdx);
                                 thread.setPriority(Thread.MIN_PRIORITY);
                                 try {
                                     pushFrames(collector, inputChannelsFromConnectors.get(cIdx), writer);
@@ -354,10 +355,11 @@
         if (!exceptions.isEmpty()) {
             if (LOGGER.isWarnEnabled()) {
                 for (int i = 0; i < exceptions.size(); i++) {
-                    LOGGER.log(Level.WARN,
-                            "Task " + taskAttemptId + " failed with exception"
+                    Exception e = exceptions.get(i);
+                    LOGGER.log(ExceptionUtils.causedByInterrupt(e) ? Level.DEBUG : Level.WARN,
+                            "Task failed with exception"
                                     + (exceptions.size() > 1 ? "s (" + (i + 1) + "/" + exceptions.size() + ")" : ""),
-                            exceptions.get(i));
+                            e);
                 }
             }
             ExceptionUtils.setNodeIds(exceptions, ncs.getId());
@@ -426,8 +428,8 @@
     }
 
     @Override
-    public IDatasetPartitionManager getDatasetPartitionManager() {
-        return ncs.getDatasetPartitionManager();
+    public IResultPartitionManager getResultPartitionManager() {
+        return ncs.getResultPartitionManager();
     }
 
     @Override
@@ -469,4 +471,10 @@
     public boolean isCompleted() {
         return completed;
     }
+
+    @Override
+    public String toString() {
+        return "{ \"class\" : \"" + getClass().getSimpleName() + "\", \"node\" : \"" + ncs.getId() + "\" \"jobId\" : \""
+                + joblet.getJobId() + "\", \"taskId\" : \"" + taskAttemptId + "\" }";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCServiceContext.java
index 87330226..3eaa0cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/application/NCServiceContext.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.control.nc.application;
 
-import java.io.IOException;
 import java.io.Serializable;
 import java.util.concurrent.ConcurrentHashMap;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetMemoryManager.java
deleted file mode 100644
index 37cddb8..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetMemoryManager.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.partitions.ResultSetPartitionId;
-
-public class DatasetMemoryManager {
-    private int availableMemory;
-
-    private final Set<Page> availPages;
-
-    private final LeastRecentlyUsedList leastRecentlyUsedList;
-
-    private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
-
-    private final static int FRAME_SIZE = 32768;
-
-    public DatasetMemoryManager(int availableMemory) {
-        this.availableMemory = availableMemory;
-
-        availPages = new HashSet<Page>();
-
-        // Atleast have one page for temporarily storing the results.
-        if (this.availableMemory <= FRAME_SIZE)
-            this.availableMemory = FRAME_SIZE;
-
-        leastRecentlyUsedList = new LeastRecentlyUsedList();
-        resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
-    }
-
-    public synchronized Page requestPage(ResultSetPartitionId resultSetPartitionId, ResultState resultState)
-            throws HyracksDataException {
-        Page page;
-        if (availPages.isEmpty()) {
-            if (availableMemory >= FRAME_SIZE) {
-                /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
-                 * instead of direct ByteBuffer.allocate()?
-                 */
-                availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
-                availableMemory -= FRAME_SIZE;
-                page = getAvailablePage();
-            } else {
-                page = evictPage();
-            }
-        } else {
-            page = getAvailablePage();
-        }
-
-        page.clear();
-
-        /*
-         * It is extremely important to update the reference after obtaining the page because, in the cases where memory
-         * manager is allocated only one page of memory, the front of the LRU list should not be created by the
-         * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
-         * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
-         */
-        PartitionNode pn = updateReference(resultSetPartitionId, resultState);
-        pn.add(page);
-        return page;
-    }
-
-    public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
-        // When a page is referenced the dataset partition writer should already be known, so we pass null.
-        updateReference(resultSetPartitionId, null);
-    }
-
-    public static int getPageSize() {
-        return FRAME_SIZE;
-    }
-
-    protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
-        leastRecentlyUsedList.add(pn);
-        resultPartitionNodesMap.put(resultSetPartitionId, pn);
-    }
-
-    protected PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
-        PartitionNode pn = null;
-
-        if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
-            if (resultState != null) {
-                pn = new PartitionNode(resultSetPartitionId, resultState);
-                insertPartitionNode(resultSetPartitionId, pn);
-            }
-            return pn;
-        }
-        synchronized (this) {
-            pn = resultPartitionNodesMap.get(resultSetPartitionId);
-            leastRecentlyUsedList.remove(pn);
-            insertPartitionNode(resultSetPartitionId, pn);
-        }
-
-        return pn;
-    }
-
-    protected Page evictPage() throws HyracksDataException {
-        PartitionNode pn = leastRecentlyUsedList.getFirst();
-        ResultState resultState = pn.getResultState();
-        Page page = resultState.returnPage();
-
-        /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
-         * away all the pages allocated to it and add to the available pages set.
-         */
-        if (page == null) {
-            availPages.addAll(pn);
-            pn.clear();
-            resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
-            leastRecentlyUsedList.remove(pn);
-
-            /* Based on the assumption that if the dataset partition writer returned a null page, it should be lying about
-             * the number of pages it holds in which case we just evict all the pages it holds and should thus be able to
-             * add all those pages to available set and we have at least one page to allocate back.
-             */
-            page = getAvailablePage();
-        } else {
-            pn.remove(page);
-
-            // If the partition no more holds any pages, remove it from the linked list and the hash map.
-            if (pn.isEmpty()) {
-                resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
-                leastRecentlyUsedList.remove(pn);
-            }
-        }
-
-        return page;
-    }
-
-    protected Page getAvailablePage() {
-        Iterator<Page> iter = availPages.iterator();
-        Page page = iter.next();
-        iter.remove();
-        return page;
-    }
-
-    private class LeastRecentlyUsedList {
-        private PartitionNode head;
-
-        private PartitionNode tail;
-
-        public LeastRecentlyUsedList() {
-            head = null;
-            tail = null;
-        }
-
-        public void add(PartitionNode node) {
-            if (head == null) {
-                head = tail = node;
-                return;
-            }
-            tail.setNext(node);
-            node.setPrev(tail);
-            tail = node;
-        }
-
-        public void remove(PartitionNode node) {
-            if ((node == head) && (node == tail)) {
-                head = tail = null;
-                return;
-            } else if (node == head) {
-                head = head.getNext();
-                head.setPrev(null);
-                return;
-            } else if (node == tail) {
-                tail = tail.getPrev();
-                tail.setNext(null);
-                return;
-            } else {
-                PartitionNode prev = node.getPrev();
-                PartitionNode next = node.getNext();
-                prev.setNext(next);
-                next.setPrev(prev);
-            }
-        }
-
-        public PartitionNode getFirst() {
-            return head;
-        }
-    }
-
-    private class PartitionNode extends HashSet<Page> {
-        private static final long serialVersionUID = 1L;
-
-        private final ResultSetPartitionId resultSetPartitionId;
-
-        private final ResultState resultState;
-
-        private PartitionNode prev;
-
-        private PartitionNode next;
-
-        public PartitionNode(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
-            this.resultSetPartitionId = resultSetPartitionId;
-            this.resultState = resultState;
-            prev = null;
-            next = null;
-        }
-
-        public ResultSetPartitionId getResultSetPartitionId() {
-            return resultSetPartitionId;
-        }
-
-        public ResultState getResultState() {
-            return resultState;
-        }
-
-        public void setPrev(PartitionNode node) {
-            prev = node;
-        }
-
-        public PartitionNode getPrev() {
-            return prev;
-        }
-
-        public void setNext(PartitionNode node) {
-            next = node;
-        }
-
-        public PartitionNode getNext() {
-            return next;
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java
deleted file mode 100644
index b7cf9a4..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionManager.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Executor;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.io.IWorkspaceFileFactory;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.control.common.dataset.AbstractDatasetManager;
-import org.apache.hyracks.control.common.dataset.ResultStateSweeper;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.control.nc.io.WorkspaceFileFactory;
-import org.apache.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class DatasetPartitionManager extends AbstractDatasetManager implements IDatasetPartitionManager {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private final NodeControllerService ncs;
-
-    private final Executor executor;
-
-    private final Map<JobId, ResultSetMap> partitionResultStateMap;
-
-    private final DefaultDeallocatableRegistry deallocatableRegistry;
-
-    private final IWorkspaceFileFactory fileFactory;
-
-    private final DatasetMemoryManager datasetMemoryManager;
-
-    public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory, long resultTTL,
-            long resultSweepThreshold) {
-        super(resultTTL);
-        this.ncs = ncs;
-        this.executor = executor;
-        deallocatableRegistry = new DefaultDeallocatableRegistry();
-        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, ncs.getIoManager());
-        if (availableMemory >= DatasetMemoryManager.getPageSize()) {
-            datasetMemoryManager = new DatasetMemoryManager(availableMemory);
-        } else {
-            datasetMemoryManager = null;
-        }
-        partitionResultStateMap = new HashMap<>();
-        executor.execute(new ResultStateSweeper(this, resultSweepThreshold, LOGGER));
-    }
-
-    @Override
-    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
-            boolean asyncMode, int partition, int nPartitions, long maxReads) {
-        DatasetPartitionWriter dpw;
-        JobId jobId = ctx.getJobletContext().getJobId();
-        synchronized (this) {
-            dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, asyncMode, orderedResult, partition, nPartitions,
-                    datasetMemoryManager, fileFactory, maxReads);
-            ResultSetMap rsIdMap = partitionResultStateMap.computeIfAbsent(jobId, k -> new ResultSetMap());
-            ResultState[] resultStates = rsIdMap.createOrGetResultStates(rsId, nPartitions);
-            resultStates[partition] = dpw.getResultState();
-        }
-        LOGGER.debug("Initialized partition writer: JobId: {}:partition: {}", jobId, partition);
-        return dpw;
-    }
-
-    @Override
-    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
-            boolean orderedResult, boolean emptyResult) throws HyracksException {
-        try {
-            // Be sure to send the *public* network address to the CC
-            ncs.getClusterController(jobId.getCcId()).registerResultPartitionLocation(jobId, rsId, orderedResult,
-                    emptyResult, partition, nPartitions, ncs.getDatasetNetworkManager().getPublicNetworkAddress());
-        } catch (Exception e) {
-            throw HyracksException.create(e);
-        }
-    }
-
-    @Override
-    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
-        try {
-            LOGGER.debug("Reporting partition write completion: JobId: {}:ResultSetId: {}:partition: {}", jobId, rsId,
-                    partition);
-            ncs.getClusterController(jobId.getCcId()).reportResultPartitionWriteCompletion(jobId, rsId, partition);
-        } catch (Exception e) {
-            throw HyracksException.create(e);
-        }
-    }
-
-    @Override
-    public void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition,
-            IFrameWriter writer) throws HyracksException {
-        ResultState resultState = getResultState(jobId, resultSetId, partition);
-        DatasetPartitionReader dpr = new DatasetPartitionReader(this, datasetMemoryManager, executor, resultState);
-        dpr.writeTo(writer);
-        LOGGER.debug("Initialized partition reader: JobId: {}:ResultSetId: {}:partition: {}", jobId, resultSetId,
-                partition);
-    }
-
-    private synchronized ResultState getResultState(JobId jobId, ResultSetId resultSetId, int partition)
-            throws HyracksException {
-        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
-        if (rsIdMap == null) {
-            throw new HyracksException("Unknown JobId " + jobId);
-        }
-        ResultState[] resultStates = rsIdMap.getResultStates(resultSetId);
-        if (resultStates == null) {
-            throw new HyracksException("Unknown JobId: " + jobId + " ResultSetId: " + resultSetId);
-        }
-        ResultState resultState = resultStates[partition];
-        if (resultState == null) {
-            throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
-        }
-        return resultState;
-    }
-
-    @Override
-    public synchronized void removePartition(JobId jobId, ResultSetId resultSetId, int partition) {
-        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
-        if (rsIdMap != null && rsIdMap.removePartition(jobId, resultSetId, partition)) {
-            partitionResultStateMap.remove(jobId);
-        }
-    }
-
-    @Override
-    public synchronized void abortReader(JobId jobId) {
-        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
-        if (rsIdMap != null) {
-            rsIdMap.abortAll();
-        }
-    }
-
-    @Override
-    public synchronized void close() {
-        for (JobId jobId : getJobIds()) {
-            deinit(jobId);
-        }
-        deallocatableRegistry.close();
-    }
-
-    @Override
-    public synchronized Set<JobId> getJobIds() {
-        return partitionResultStateMap.keySet();
-    }
-
-    @Override
-    public synchronized ResultSetMap getState(JobId jobId) {
-        return partitionResultStateMap.get(jobId);
-    }
-
-    @Override
-    public synchronized void sweep(JobId jobId) {
-        deinit(jobId);
-        partitionResultStateMap.remove(jobId);
-    }
-
-    private synchronized void deinit(JobId jobId) {
-        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
-        if (rsIdMap != null) {
-            rsIdMap.closeAndDeleteAll();
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionReader.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionReader.java
deleted file mode 100644
index 8c4fcb0..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionReader.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.Executor;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.partitions.ResultSetPartitionId;
-import org.apache.hyracks.comm.channels.NetworkOutputChannel;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class DatasetPartitionReader {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private final DatasetPartitionManager datasetPartitionManager;
-    private final DatasetMemoryManager datasetMemoryManager;
-    private final Executor executor;
-    private final ResultState resultState;
-
-    public DatasetPartitionReader(DatasetPartitionManager datasetPartitionManager,
-            DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
-        this.datasetPartitionManager = datasetPartitionManager;
-        this.datasetMemoryManager = datasetMemoryManager;
-        this.executor = executor;
-        this.resultState = resultState;
-    }
-
-    public void writeTo(final IFrameWriter writer) {
-        executor.execute(new ResultPartitionSender((NetworkOutputChannel) writer));
-    }
-
-    private class ResultPartitionSender implements Runnable {
-
-        private final NetworkOutputChannel channel;
-
-        ResultPartitionSender(final NetworkOutputChannel channel) {
-            this.channel = channel;
-        }
-
-        @Override
-        public void run() {
-            channel.setFrameSize(resultState.getFrameSize());
-            channel.open();
-            try {
-                resultState.readOpen();
-                long offset = 0;
-                final ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
-                while (true) {
-                    buffer.clear();
-                    final long size = read(offset, buffer);
-                    if (size <= 0) {
-                        break;
-                    } else if (size < buffer.limit()) {
-                        throw new IllegalStateException(
-                                "Premature end of file - readSize: " + size + " buffer limit: " + buffer.limit());
-                    }
-                    offset += size;
-                    buffer.flip();
-                    channel.nextFrame(buffer);
-                }
-                if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("result reading successful(" + resultState.getResultSetPartitionId() + ")");
-                }
-            } catch (Exception e) {
-                LOGGER.error(() -> "failed to send result partition " + resultState.getResultSetPartitionId(), e);
-                channel.abort();
-            } finally {
-                close();
-            }
-        }
-
-        private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
-            return datasetMemoryManager != null ? resultState.read(datasetMemoryManager, offset, buffer)
-                    : resultState.read(offset, buffer);
-        }
-
-        private void close() {
-            try {
-                channel.close();
-                resultState.readClose();
-                if (resultState.isExhausted()) {
-                    final ResultSetPartitionId partitionId = resultState.getResultSetPartitionId();
-                    datasetPartitionManager.removePartition(partitionId.getJobId(), partitionId.getResultSetId(),
-                            partitionId.getPartition());
-                }
-            } catch (HyracksDataException e) {
-                LOGGER.error("unexpected failure in partition reader clean up", e);
-            }
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionWriter.java
deleted file mode 100644
index b593bb5..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/DatasetPartitionWriter.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.io.IWorkspaceFileFactory;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.partitions.ResultSetPartitionId;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class DatasetPartitionWriter implements IFrameWriter {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private final IDatasetPartitionManager manager;
-
-    private final JobId jobId;
-
-    private final ResultSetId resultSetId;
-
-    private final boolean orderedResult;
-
-    private final int partition;
-
-    private final int nPartitions;
-
-    private final DatasetMemoryManager datasetMemoryManager;
-
-    private final ResultSetPartitionId resultSetPartitionId;
-
-    private final ResultState resultState;
-
-    private boolean partitionRegistered;
-
-    private boolean failed = false;
-
-    public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
-            ResultSetId rsId, boolean asyncMode, boolean orderedResult, int partition, int nPartitions,
-            DatasetMemoryManager datasetMemoryManager, IWorkspaceFileFactory fileFactory, long maxReads) {
-        this.manager = manager;
-        this.jobId = jobId;
-        this.resultSetId = rsId;
-        this.orderedResult = orderedResult;
-        this.partition = partition;
-        this.nPartitions = nPartitions;
-        this.datasetMemoryManager = datasetMemoryManager;
-
-        resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
-        resultState = new ResultState(resultSetPartitionId, asyncMode, ctx.getIoManager(), fileFactory,
-                ctx.getInitialFrameSize(), maxReads);
-    }
-
-    public ResultState getResultState() {
-        return resultState;
-    }
-
-    @Override
-    public void open() {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("open(" + partition + ")");
-        }
-        partitionRegistered = false;
-        resultState.open();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        registerResultPartitionLocation(false);
-        if (datasetMemoryManager == null) {
-            resultState.write(buffer);
-        } else {
-            resultState.write(datasetMemoryManager, buffer);
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        failed = true;
-        resultState.closeAndDelete();
-        resultState.abort();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("close(" + partition + ")");
-        }
-        try {
-            if (!failed) {
-                registerResultPartitionLocation(true);
-            }
-        } finally {
-            resultState.close();
-        }
-        try {
-            if (partitionRegistered) {
-                manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
-            }
-        } catch (HyracksException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    void registerResultPartitionLocation(boolean empty) throws HyracksDataException {
-        try {
-            if (!partitionRegistered) {
-                manager.registerResultPartitionLocation(jobId, resultSetId, partition, nPartitions, orderedResult,
-                        empty);
-                partitionRegistered = true;
-            }
-        } catch (HyracksException e) {
-            if (e instanceof HyracksDataException) {
-                throw (HyracksDataException) e;
-            } else {
-                throw HyracksDataException.create(e);
-            }
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/Page.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/Page.java
deleted file mode 100644
index 2eb33fd..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/Page.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.nio.ByteBuffer;
-
-public class Page {
-    private final ByteBuffer buffer;
-
-    public Page(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public ByteBuffer getBuffer() {
-        return buffer;
-    }
-
-    public ByteBuffer clear() {
-        return (ByteBuffer) buffer.clear();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultSetMap.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultSetMap.java
deleted file mode 100644
index 1a64a5a..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultSetMap.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hyracks.api.dataset.IDatasetStateRecord;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-class ResultSetMap implements IDatasetStateRecord, Serializable {
-    private static final long serialVersionUID = 1L;
-
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private final long timestamp;
-    private final HashMap<ResultSetId, ResultState[]> resultStateMap;
-
-    ResultSetMap() {
-        timestamp = System.nanoTime();
-        resultStateMap = new HashMap<>();
-    }
-
-    @Override
-    public long getTimestamp() {
-        return timestamp;
-    }
-
-    ResultState[] getResultStates(ResultSetId rsId) {
-        return resultStateMap.get(rsId);
-    }
-
-    ResultState[] createOrGetResultStates(ResultSetId rsId, int nPartitions) {
-        return resultStateMap.computeIfAbsent(rsId, (k) -> new ResultState[nPartitions]);
-    }
-
-    /**
-     * removes a result partition for a result set
-     *
-     * @param jobId
-     *            the id of the job that produced the result set
-     * @param resultSetId
-     *            the id of the result set
-     * @param partition
-     *            the partition number
-     * @return true, if all partitions for the resultSetId have been removed
-     */
-    boolean removePartition(JobId jobId, ResultSetId resultSetId, int partition) {
-        final ResultState[] resultStates = resultStateMap.get(resultSetId);
-        if (resultStates != null) {
-            final ResultState state = resultStates[partition];
-            if (state != null) {
-                state.closeAndDelete();
-                LOGGER.debug("Removing partition: " + partition + " for JobId: " + jobId);
-            }
-            resultStates[partition] = null;
-            boolean stateEmpty = true;
-            for (ResultState resState : resultStates) {
-                if (resState != null) {
-                    stateEmpty = false;
-                    break;
-                }
-            }
-            if (stateEmpty) {
-                resultStateMap.remove(resultSetId);
-            }
-            return resultStateMap.isEmpty();
-        }
-        return true;
-    }
-
-    void abortAll() {
-        applyToAllStates((rsId, state, i) -> state.abort());
-    }
-
-    void closeAndDeleteAll() {
-        applyToAllStates((rsId, state, i) -> {
-            state.closeAndDelete();
-            LOGGER.debug("Removing partition: " + i + " for result set " + rsId);
-        });
-    }
-
-    @FunctionalInterface
-    private interface StateModifier {
-        void modify(ResultSetId rsId, ResultState entry, int partition);
-    }
-
-    private void applyToAllStates(StateModifier modifier) {
-        for (Map.Entry<ResultSetId, ResultState[]> entry : resultStateMap.entrySet()) {
-            final ResultSetId rsId = entry.getKey();
-            final ResultState[] resultStates = entry.getValue();
-            if (resultStates == null) {
-                continue;
-            }
-            for (int i = 0; i < resultStates.length; i++) {
-                final ResultState state = resultStates[i];
-                if (state != null) {
-                    modifier.modify(rsId, state, i);
-                }
-            }
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
deleted file mode 100644
index b832b20..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * 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.hyracks.control.nc.dataset;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hyracks.api.dataflow.state.IStateObject;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.IFileHandle;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.io.IWorkspaceFileFactory;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.partitions.ResultSetPartitionId;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-public class ResultState implements IStateObject {
-    private static final String FILE_PREFIX = "result_";
-
-    private final ResultSetPartitionId resultSetPartitionId;
-
-    private final boolean asyncMode;
-
-    private final int frameSize;
-
-    private final IIOManager ioManager;
-
-    private final IWorkspaceFileFactory fileFactory;
-
-    private final AtomicBoolean eos;
-
-    private final AtomicBoolean failed;
-
-    private final List<Page> localPageList;
-
-    private FileReference fileRef;
-
-    private IFileHandle writeFileHandle;
-
-    private IFileHandle readFileHandle;
-
-    private long size;
-
-    private long persistentSize;
-    private long remainingReads;
-
-    ResultState(ResultSetPartitionId resultSetPartitionId, boolean asyncMode, IIOManager ioManager,
-            IWorkspaceFileFactory fileFactory, int frameSize, long maxReads) {
-        if (maxReads <= 0) {
-            throw new IllegalArgumentException("maxReads must be > 0");
-        }
-        this.resultSetPartitionId = resultSetPartitionId;
-        this.asyncMode = asyncMode;
-        this.ioManager = ioManager;
-        this.fileFactory = fileFactory;
-        this.frameSize = frameSize;
-        remainingReads = maxReads;
-        eos = new AtomicBoolean(false);
-        failed = new AtomicBoolean(false);
-        localPageList = new ArrayList<>();
-
-        fileRef = null;
-        writeFileHandle = null;
-    }
-
-    public synchronized void open() {
-        size = 0;
-        persistentSize = 0;
-    }
-
-    public synchronized void close() {
-        eos.set(true);
-        closeWriteFileHandle();
-        notifyAll();
-    }
-
-    public synchronized void closeAndDelete() {
-        // Deleting a job is equivalent to aborting the job for all practical purposes, so the same action, needs
-        // to be taken when there are more requests to these result states.
-        failed.set(true);
-        closeWriteFileHandle();
-        if (fileRef != null) {
-            fileRef.delete();
-            fileRef = null;
-        }
-    }
-
-    private void closeWriteFileHandle() {
-        if (writeFileHandle != null) {
-            try {
-                ioManager.close(writeFileHandle);
-            } catch (IOException e) {
-                // Since file handle could not be closed, just ignore.
-            }
-            writeFileHandle = null;
-        }
-    }
-
-    public synchronized void write(ByteBuffer buffer) throws HyracksDataException {
-        if (fileRef == null) {
-            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
-            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
-            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_SYNC);
-        }
-
-        size += ioManager.syncWrite(writeFileHandle, size, buffer);
-        notifyAll();
-    }
-
-    public synchronized void write(DatasetMemoryManager datasetMemoryManager, ByteBuffer buffer)
-            throws HyracksDataException {
-        int srcOffset = 0;
-        Page destPage = null;
-
-        if (!localPageList.isEmpty()) {
-            destPage = localPageList.get(localPageList.size() - 1);
-        }
-
-        while (srcOffset < buffer.limit()) {
-            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
-                destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
-                localPageList.add(destPage);
-            }
-            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
-            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
-            srcOffset += srcLength;
-            size += srcLength;
-        }
-
-        notifyAll();
-    }
-
-    public synchronized void readOpen() {
-        if (isExhausted()) {
-            throw new IllegalStateException("Result reads exhausted");
-        }
-        remainingReads--;
-    }
-
-    public synchronized void readClose() throws HyracksDataException {
-        if (readFileHandle != null) {
-            ioManager.close(readFileHandle);
-            readFileHandle = null;
-        }
-    }
-
-    public synchronized long read(long offset, ByteBuffer buffer) throws HyracksDataException {
-        long readSize = 0;
-
-        while (offset >= size && !eos.get() && !failed.get()) {
-            try {
-                wait();
-            } catch (InterruptedException e) {
-                throw HyracksDataException.create(e);
-            }
-        }
-        if ((offset >= size && eos.get()) || failed.get()) {
-            return readSize;
-        }
-
-        if (readFileHandle == null) {
-            initReadFileHandle();
-        }
-        readSize = ioManager.syncRead(readFileHandle, offset, buffer);
-
-        return readSize;
-    }
-
-    public long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
-            throws HyracksDataException {
-        long readSize = 0;
-        synchronized (this) {
-            while (offset >= size && !eos.get() && !failed.get()) {
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                    throw HyracksDataException.create(e);
-                }
-            }
-
-            if ((offset >= size && eos.get()) || failed.get()) {
-                return readSize;
-            }
-
-            if (offset < persistentSize) {
-                if (readFileHandle == null) {
-                    initReadFileHandle();
-                }
-                readSize = ioManager.syncRead(readFileHandle, offset, buffer);
-                if (readSize < 0) {
-                    throw new HyracksDataException("Premature end of file");
-                }
-            }
-
-            if (readSize < buffer.capacity()) {
-                long localPageOffset = offset - persistentSize;
-                int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
-                int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
-                Page page = getPage(localPageIndex);
-                if (page == null) {
-                    return readSize;
-                }
-                readSize += buffer.remaining();
-                buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
-            }
-        }
-        datasetMemoryManager.pageReferenced(resultSetPartitionId);
-        return readSize;
-    }
-
-    public synchronized void abort() {
-        failed.set(true);
-        notifyAll();
-    }
-
-    public synchronized Page returnPage() throws HyracksDataException {
-        Page page = removePage();
-
-        // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
-        if (page == null) {
-            ioManager.close(writeFileHandle);
-            return null;
-        }
-
-        page.getBuffer().flip();
-
-        if (fileRef == null) {
-            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
-            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
-            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-            notifyAll();
-        }
-
-        long delta = ioManager.syncWrite(writeFileHandle, persistentSize, page.getBuffer());
-        persistentSize += delta;
-        return page;
-    }
-
-    public synchronized void setEOS(boolean eos) {
-        this.eos.set(eos);
-    }
-
-    public ResultSetPartitionId getResultSetPartitionId() {
-        return resultSetPartitionId;
-    }
-
-    public int getFrameSize() {
-        return frameSize;
-    }
-
-    public IIOManager getIOManager() {
-        return ioManager;
-    }
-
-    public boolean getAsyncMode() {
-        return asyncMode;
-    }
-
-    @Override
-    public JobId getJobId() {
-        return resultSetPartitionId.getJobId();
-    }
-
-    @Override
-    public Object getId() {
-        return resultSetPartitionId;
-    }
-
-    @Override
-    public long getMemoryOccupancy() {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void toBytes(DataOutput out) throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void fromBytes(DataInput in) throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    private Page getPage(int index) {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.get(index);
-        }
-        return page;
-    }
-
-    private Page removePage() {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.remove(localPageList.size() - 1);
-        }
-        return page;
-    }
-
-    private void initReadFileHandle() throws HyracksDataException {
-        while (fileRef == null && !failed.get()) {
-            try {
-                wait();
-            } catch (InterruptedException e) {
-                throw HyracksDataException.create(e);
-            }
-        }
-        if (failed.get()) {
-            return;
-        }
-
-        readFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
-                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-    }
-
-    @Override
-    public String toString() {
-        try {
-            ObjectMapper om = new ObjectMapper();
-            ObjectNode on = om.createObjectNode();
-            on.put("rspid", resultSetPartitionId.toString());
-            on.put("async", asyncMode);
-            on.put("remainingReads", remainingReads);
-            on.put("eos", eos.get());
-            on.put("failed", failed.get());
-            on.put("fileRef", String.valueOf(fileRef));
-            return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
-        } catch (JsonProcessingException e) { // NOSONAR
-            return e.getMessage();
-        }
-    }
-
-    public synchronized boolean isExhausted() {
-        return remainingReads == 0;
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatComputeTask.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatComputeTask.java
index dfa5ff3..cd051e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatComputeTask.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatComputeTask.java
@@ -33,7 +33,6 @@
 import org.apache.hyracks.control.nc.io.profiling.IOCounterFactory;
 import org.apache.hyracks.ipc.api.IPCPerformanceCounters;
 import org.apache.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -85,11 +84,11 @@
             hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
             hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
 
-            MuxDemuxPerformanceCounters datasetNetPC = ncs.getDatasetNetworkManager().getPerformanceCounters();
-            hbData.datasetNetPayloadBytesRead = datasetNetPC.getPayloadBytesRead();
-            hbData.datasetNetPayloadBytesWritten = datasetNetPC.getPayloadBytesWritten();
-            hbData.datasetNetSignalingBytesRead = datasetNetPC.getSignalingBytesRead();
-            hbData.datasetNetSignalingBytesWritten = datasetNetPC.getSignalingBytesWritten();
+            MuxDemuxPerformanceCounters resultNetPC = ncs.getResultNetworkManager().getPerformanceCounters();
+            hbData.resultNetPayloadBytesRead = resultNetPC.getPayloadBytesRead();
+            hbData.resultNetPayloadBytesWritten = resultNetPC.getPayloadBytesWritten();
+            hbData.resultNetSignalingBytesRead = resultNetPC.getSignalingBytesRead();
+            hbData.resultNetSignalingBytesWritten = resultNetPC.getSignalingBytesWritten();
 
             IPCPerformanceCounters ipcPC = ncs.getIpcSystem().getPerformanceCounters();
             hbData.ipcMessagesSent = ipcPC.getMessageSentCount();
@@ -103,6 +102,6 @@
 
             ncs.getNodeControllerData().notifyHeartbeat(hbData);
         }
-        LOGGER.log(Level.DEBUG, "Successfully refreshed heartbeat data");
+        LOGGER.trace("Successfully refreshed heartbeat data");
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java
new file mode 100644
index 0000000..08b8c11
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java
@@ -0,0 +1,76 @@
+/*
+ * 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.hyracks.control.nc.heartbeat;
+
+import java.net.InetSocketAddress;
+
+import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.common.heartbeat.HeartbeatData;
+import org.apache.hyracks.control.nc.CcConnection;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HeartbeatManager {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final NodeControllerService ncs;
+    private final CcConnection ccc;
+    private final Thread hbThread;
+    private final CcId ccId;
+
+    private HeartbeatManager(NodeControllerService ncs, CcConnection ccc, HeartbeatData hbData,
+            InetSocketAddress ncAddress) {
+        this.ncs = ncs;
+        this.ccc = ccc;
+        hbThread = new Thread(new HeartbeatTask(ncs.getId(), hbData, ccc.getClusterControllerService(),
+                ccc.getNodeParameters().getHeartbeatPeriod(), ncAddress), ncs.getId() + "-Heartbeat");
+        hbThread.setPriority(Thread.MAX_PRIORITY);
+        hbThread.setDaemon(true);
+        ccId = ccc.getCcId();
+    }
+
+    public static HeartbeatManager init(NodeControllerService ncs, CcConnection ccc, HeartbeatData hbData,
+            InetSocketAddress ncAddress) {
+        HeartbeatManager hbMgr = new HeartbeatManager(ncs, ccc, hbData, ncAddress);
+        hbMgr.start();
+        return hbMgr;
+    }
+
+    public void shutdown() {
+        hbThread.interrupt();
+    }
+
+    public void start() {
+        hbThread.start();
+    }
+
+    public void notifyAck(HyracksDataException exception) {
+        LOGGER.debug("ack rec'd from {} w/ exception: {}", ccId::toString, () -> String.valueOf(exception));
+        if (exception != null && exception.matches(ErrorCode.HYRACKS, ErrorCode.NO_SUCH_NODE)) {
+            LOGGER.info("{} indicates it does not recognize us; force a reconnect", ccId);
+            try {
+                ccc.forceReregister(ncs);
+            } catch (Exception e) {
+                LOGGER.warn("ignoring exception attempting to reregister with {}", ccId, e);
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatTask.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatTask.java
index 9160e46..4af7e4e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatTask.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatTask.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.control.nc.heartbeat;
 
+import java.net.InetSocketAddress;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
@@ -36,12 +37,15 @@
     private final Semaphore delayBlock = new Semaphore(0);
     private final IClusterController cc;
     private final long heartbeatPeriodNanos;
+    private final InetSocketAddress ncAddress;
 
-    public HeartbeatTask(String ncId, HeartbeatData hbData, IClusterController cc, long heartbeatPeriod) {
+    public HeartbeatTask(String ncId, HeartbeatData hbData, IClusterController cc, long heartbeatPeriod,
+            InetSocketAddress ncAddress) {
         this.ncId = ncId;
         this.hbData = hbData;
         this.cc = cc;
         this.heartbeatPeriodNanos = TimeUnit.MILLISECONDS.toNanos(heartbeatPeriod);
+        this.ncAddress = ncAddress;
     }
 
     @Override
@@ -67,18 +71,15 @@
     private boolean execute() throws InterruptedException {
         try {
             synchronized (hbData) {
-                cc.nodeHeartbeat(ncId, hbData);
+                cc.nodeHeartbeat(ncId, hbData, ncAddress);
             }
-            LOGGER.log(Level.DEBUG, "Successfully sent heartbeat");
+            LOGGER.trace("Successfully sent heartbeat");
             return true;
         } catch (InterruptedException e) {
             throw e;
         } catch (Exception e) {
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.log(Level.DEBUG, "Exception sending heartbeat; will retry after 1s", e);
-            } else {
-                LOGGER.log(Level.ERROR, "Exception sending heartbeat; will retry after 1s: " + e.toString());
-            }
+            LOGGER.log(Level.DEBUG, "Exception sending heartbeat; will retry after 1s", e);
+            LOGGER.log(Level.WARN, "Exception sending heartbeat; will retry after 1s: " + e.toString());
             return false;
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 7de716a..b5cb21a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -36,6 +36,7 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -55,8 +56,8 @@
     /*
      * Constants
      */
+    public static final int IO_REQUEST_QUEUE_SIZE = 100; // TODO: Make configurable
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final int IO_REQUEST_QUEUE_SIZE = 100; // TODO: Make configurable
     private static final String WORKSPACE_FILE_SUFFIX = ".waf";
     private static final FilenameFilter WORKSPACE_FILES_FILTER = (dir, name) -> name.endsWith(WORKSPACE_FILE_SUFFIX);
     /*
@@ -71,7 +72,7 @@
      * Mutables
      */
     private int workspaceIndex;
-    private IFileDeviceResolver deviceComputer;
+    private final IFileDeviceResolver deviceComputer;
 
     public IOManager(List<IODeviceHandle> devices, IFileDeviceResolver deviceComputer) throws HyracksDataException {
         this.ioDevices = Collections.unmodifiableList(devices);
@@ -416,4 +417,13 @@
             Thread.currentThread().interrupt();
         }
     }
+
+    @Override
+    public long getTotalDiskUsage() {
+        long totalSize = 0;
+        for (IODeviceHandle handle : ioDevices) {
+            totalSize += FileUtils.sizeOfDirectory(handle.getMount());
+        }
+        return totalSize;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IoRequest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IoRequest.java
index 8c81d41..93e38f5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IoRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IoRequest.java
@@ -44,7 +44,7 @@
     private long offset;
     private ByteBuffer data;
     private ByteBuffer[] dataArray;
-    private HyracksDataException failure;
+    private Throwable failure;
     private int read;
     private int write;
     private long writes;
@@ -133,15 +133,17 @@
             state = State.OPERATION_SUCCEEDED;
         } catch (Throwable th) { // NOSONAR: This method must never throw anything
             state = State.OPERATION_FAILED;
-            failure = HyracksDataException.create(th);
+            failure = th;
+        } finally {
+            notifyAll();
         }
-        notifyAll();
     }
 
     public State getState() {
         return state;
     }
 
+    @SuppressWarnings("squid:S899") // Offer failing means we're over capacity and this should be garbage collected
     void recycle() {
         reset();
         freeRequests.offer(this);
@@ -165,6 +167,6 @@
     }
 
     public HyracksDataException getFailure() {
-        return failure;
+        return HyracksDataException.create(failure);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/DatasetNetworkManager.java
deleted file mode 100644
index 5eba281..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/DatasetNetworkManager.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * 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.hyracks.control.nc.net;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.IChannelInterfaceFactory;
-import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
-import org.apache.hyracks.api.comm.NetworkAddress;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.exceptions.NetException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.comm.channels.IChannelConnectionFactory;
-import org.apache.hyracks.comm.channels.NetworkOutputChannel;
-import org.apache.hyracks.net.protocols.muxdemux.ChannelControlBlock;
-import org.apache.hyracks.net.protocols.muxdemux.IChannelOpenListener;
-import org.apache.hyracks.net.protocols.muxdemux.MultiplexedConnection;
-import org.apache.hyracks.net.protocols.muxdemux.MuxDemux;
-import org.apache.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class DatasetNetworkManager implements IChannelConnectionFactory {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private static final int MAX_CONNECTION_ATTEMPTS = 5;
-
-    static final int INITIAL_MESSAGE_SIZE = 20;
-
-    private final IDatasetPartitionManager partitionManager;
-
-    private final MuxDemux md;
-
-    private final int nBuffers;
-
-    private NetworkAddress localNetworkAddress;
-
-    private NetworkAddress publicNetworkAddress;
-
-    /**
-     * @param inetAddress
-     *            - Internet address to bind the listen port to
-     * @param inetPort
-     *            - Port to bind on inetAddress
-     * @param publicInetAddress
-     *            - Internet address to report to consumers;
-     *            useful when behind NAT. null = same as inetAddress
-     * @param publicInetPort
-     *            - Port to report to consumers; useful when
-     *            behind NAT. Ignored if publicInetAddress is null. 0 = same as inetPort
-     */
-    public DatasetNetworkManager(String inetAddress, int inetPort, IDatasetPartitionManager partitionManager,
-            int nThreads, int nBuffers, String publicInetAddress, int publicInetPort,
-            IChannelInterfaceFactory channelInterfaceFactory) {
-        this.partitionManager = partitionManager;
-        this.nBuffers = nBuffers;
-        md = new MuxDemux(new InetSocketAddress(inetAddress, inetPort), new ChannelOpenListener(), nThreads,
-                MAX_CONNECTION_ATTEMPTS, channelInterfaceFactory);
-        // Just save these values for the moment; may be reset in start()
-        publicNetworkAddress = new NetworkAddress(publicInetAddress, publicInetPort);
-    }
-
-    public void start() throws IOException {
-        md.start();
-        InetSocketAddress sockAddr = md.getLocalAddress();
-        localNetworkAddress = new NetworkAddress(sockAddr.getHostString(), sockAddr.getPort());
-
-        // See if the public address was explicitly specified, and if not,
-        // make it a copy of localNetworkAddress
-        if (publicNetworkAddress.getAddress() == null) {
-            publicNetworkAddress = localNetworkAddress;
-        } else {
-            // Likewise for public port
-            if (publicNetworkAddress.getPort() == 0) {
-                publicNetworkAddress = new NetworkAddress(publicNetworkAddress.getAddress(), sockAddr.getPort());
-            }
-        }
-    }
-
-    public NetworkAddress getLocalNetworkAddress() {
-        return localNetworkAddress;
-    }
-
-    public NetworkAddress getPublicNetworkAddress() {
-        return publicNetworkAddress;
-    }
-
-    public void stop() {
-
-    }
-
-    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
-        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
-        return mConn.openChannel();
-    }
-
-    private class ChannelOpenListener implements IChannelOpenListener {
-        @Override
-        public void channelOpened(ChannelControlBlock channel) {
-            channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
-            channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
-        }
-    }
-
-    private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
-        private final ChannelControlBlock ccb;
-
-        private NetworkOutputChannel noc;
-
-        public InitialBufferAcceptor(ChannelControlBlock ccb) {
-            this.ccb = ccb;
-        }
-
-        @Override
-        public void accept(ByteBuffer buffer) {
-            JobId jobId = new JobId(buffer.getLong());
-            ResultSetId rsId = new ResultSetId(buffer.getLong());
-            int partition = buffer.getInt();
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Received initial dataset partition read request for JobId: " + jobId + " partition: "
-                        + partition + " on channel: " + ccb);
-            }
-            noc = new NetworkOutputChannel(ccb, nBuffers);
-            try {
-                partitionManager.initializeDatasetPartitionReader(jobId, rsId, partition, noc);
-            } catch (HyracksException e) {
-                noc.abort();
-            }
-        }
-
-        @Override
-        public void close() {
-
-        }
-
-        @Override
-        public void error(int ecode) {
-            if (noc != null) {
-                noc.abort();
-            }
-        }
-    }
-
-    public MuxDemuxPerformanceCounters getPerformanceCounters() {
-        return md.getPerformanceCounters();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
index cfe0991..8b02f9c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/NetworkManager.java
@@ -124,8 +124,8 @@
         @Override
         public void accept(ByteBuffer buffer) {
             PartitionId pid = readInitialMessage(buffer);
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Received initial partition request: " + pid + " on channel: " + ccb);
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Received initial partition request: " + pid + " on channel: " + ccb);
             }
             noc = new NetworkOutputChannel(ccb, nBuffers);
             partitionManager.registerPartitionRequest(pid, noc);
@@ -139,7 +139,7 @@
         @Override
         public void error(int ecode) {
             if (noc != null) {
-                noc.abort();
+                noc.abort(ecode);
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/ResultNetworkManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/ResultNetworkManager.java
new file mode 100644
index 0000000..ee821d6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/net/ResultNetworkManager.java
@@ -0,0 +1,170 @@
+/*
+ * 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.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IChannelInterfaceFactory;
+import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.exceptions.NetException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.comm.channels.IChannelConnectionFactory;
+import org.apache.hyracks.comm.channels.NetworkOutputChannel;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
+import org.apache.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import org.apache.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import org.apache.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import org.apache.hyracks.net.protocols.muxdemux.MuxDemux;
+import org.apache.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResultNetworkManager implements IChannelConnectionFactory {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IResultPartitionManager partitionManager;
+
+    private final MuxDemux md;
+
+    private final int nBuffers;
+
+    private NetworkAddress localNetworkAddress;
+
+    private NetworkAddress publicNetworkAddress;
+
+    /**
+     * @param inetAddress
+     *            - Internet address to bind the listen port to
+     * @param inetPort
+     *            - Port to bind on inetAddress
+     * @param publicInetAddress
+     *            - Internet address to report to consumers;
+     *            useful when behind NAT. null = same as inetAddress
+     * @param publicInetPort
+     *            - Port to report to consumers; useful when
+     *            behind NAT. Ignored if publicInetAddress is null. 0 = same as inetPort
+     */
+    public ResultNetworkManager(String inetAddress, int inetPort, IResultPartitionManager partitionManager,
+            int nThreads, int nBuffers, String publicInetAddress, int publicInetPort,
+            IChannelInterfaceFactory channelInterfaceFactory) {
+        this.partitionManager = partitionManager;
+        this.nBuffers = nBuffers;
+        md = new MuxDemux(new InetSocketAddress(inetAddress, inetPort), new ChannelOpenListener(), nThreads,
+                MAX_CONNECTION_ATTEMPTS, channelInterfaceFactory);
+        // Just save these values for the moment; may be reset in start()
+        publicNetworkAddress = new NetworkAddress(publicInetAddress, publicInetPort);
+    }
+
+    public void start() throws IOException {
+        md.start();
+        InetSocketAddress sockAddr = md.getLocalAddress();
+        localNetworkAddress = new NetworkAddress(sockAddr.getHostString(), sockAddr.getPort());
+
+        // See if the public address was explicitly specified, and if not,
+        // make it a copy of localNetworkAddress
+        if (publicNetworkAddress.getAddress() == null) {
+            publicNetworkAddress = localNetworkAddress;
+        } else {
+            // Likewise for public port
+            if (publicNetworkAddress.getPort() == 0) {
+                publicNetworkAddress = new NetworkAddress(publicNetworkAddress.getAddress(), sockAddr.getPort());
+            }
+        }
+    }
+
+    public NetworkAddress getLocalNetworkAddress() {
+        return localNetworkAddress;
+    }
+
+    public NetworkAddress getPublicNetworkAddress() {
+        return publicNetworkAddress;
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    private class ChannelOpenListener implements IChannelOpenListener {
+        @Override
+        public void channelOpened(ChannelControlBlock channel) {
+            channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+            channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+        }
+    }
+
+    private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+        private final ChannelControlBlock ccb;
+
+        private NetworkOutputChannel noc;
+
+        public InitialBufferAcceptor(ChannelControlBlock ccb) {
+            this.ccb = ccb;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            JobId jobId = new JobId(buffer.getLong());
+            ResultSetId rsId = new ResultSetId(buffer.getLong());
+            int partition = buffer.getInt();
+            if (LOGGER.isTraceEnabled()) {
+                LOGGER.trace("Received initial result partition read request for JobId: " + jobId + " partition: "
+                        + partition + " on channel: " + ccb);
+            }
+            noc = new NetworkOutputChannel(ccb, nBuffers);
+            try {
+                partitionManager.initializeResultPartitionReader(jobId, rsId, partition, noc);
+            } catch (HyracksException e) {
+                LOGGER.warn("Failed to initialize result partition reader", e);
+                noc.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+            }
+        }
+
+        @Override
+        public void close() {
+
+        }
+
+        @Override
+        public void error(int ecode) {
+            if (noc != null) {
+                noc.abort(ecode);
+            }
+        }
+    }
+
+    public MuxDemuxPerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
index bcfe517..3e36946 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartition.java
@@ -28,7 +28,6 @@
 import org.apache.hyracks.api.io.IFileHandle;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.partitions.IPartition;
-import org.apache.hyracks.control.nc.io.IOManager;
 
 public class MaterializedPartition implements IPartition {
     private final IHyracksTaskContext ctx;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
index 29c2ff2..96cbc35 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.partitions.IPartition;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.common.job.PartitionState;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -89,7 +90,7 @@
                 Thread thread = Thread.currentThread();
                 setDataConsumerThread(thread); // Sets the data consumer thread to the current thread.
                 try {
-                    thread.setName(MaterializingPipelinedPartition.class.getName() + pid);
+                    thread.setName(MaterializingPipelinedPartition.this.getClass().getSimpleName() + " " + pid);
                     FileReference fRefCopy;
                     synchronized (MaterializingPipelinedPartition.this) {
                         while (fRef == null && !eos && !failed) {
@@ -164,7 +165,8 @@
                         }
                     }
                 } catch (Exception e) {
-                    LOGGER.log(Level.ERROR, e.getMessage(), e);
+                    LOGGER.log(ExceptionUtils.causedByInterrupt(e) ? Level.DEBUG : Level.WARN,
+                            "Failure writing to a frame", e);
                 } finally {
                     setDataConsumerThread(null); // Sets back the data consumer thread to null.
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
index d023ce9..7c8fb34 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/PartitionManager.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.control.nc.io.WorkspaceFileFactory;
 import org.apache.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
 
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
@@ -105,7 +106,8 @@
 
     public synchronized void registerPartitionRequest(PartitionId partitionId, NetworkOutputChannel writer) {
         if (failedJobsCache.getIfPresent(partitionId.getJobId()) != null) {
-            writer.abort();
+            writer.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+            return;
         }
         List<IPartition> pList = availablePartitionMap.get(partitionId);
         if (pList != null && !pList.isEmpty()) {
@@ -137,7 +139,8 @@
         if (!jobPartitions.isEmpty() || !pendingRequests.isEmpty()) {
             ncs.getExecutor().execute(() -> {
                 jobPartitions.forEach(IDeallocatable::deallocate);
-                pendingRequests.forEach(NetworkOutputChannel::abort);
+                pendingRequests.forEach(networkOutputChannel -> networkOutputChannel
+                        .abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE));
             });
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
index c902ad8..5ffed7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
@@ -103,7 +103,7 @@
         }
 
         @Override
-        public synchronized void notifyFailure(IInputChannel channel) {
+        public synchronized void notifyFailure(IInputChannel channel, int errorCode) {
             failed.set(true);
             notifyAll();
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/Page.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/Page.java
new file mode 100644
index 0000000..47230a6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/Page.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.nio.ByteBuffer;
+
+public class Page {
+    private final ByteBuffer buffer;
+
+    public Page(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    public ByteBuffer clear() {
+        return (ByteBuffer) buffer.clear();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultMemoryManager.java
new file mode 100644
index 0000000..34c9eb5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultMemoryManager.java
@@ -0,0 +1,247 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.partitions.ResultSetPartitionId;
+
+public class ResultMemoryManager {
+    private int availableMemory;
+
+    private final Set<Page> availPages;
+
+    private final LeastRecentlyUsedList leastRecentlyUsedList;
+
+    private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
+
+    private final static int FRAME_SIZE = 32768;
+
+    public ResultMemoryManager(int availableMemory) {
+        this.availableMemory = availableMemory;
+
+        availPages = new HashSet<Page>();
+
+        // Atleast have one page for temporarily storing the results.
+        if (this.availableMemory <= FRAME_SIZE) {
+            this.availableMemory = FRAME_SIZE;
+        }
+
+        leastRecentlyUsedList = new LeastRecentlyUsedList();
+        resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
+    }
+
+    public synchronized Page requestPage(ResultSetPartitionId resultSetPartitionId, ResultState resultState)
+            throws HyracksDataException {
+        Page page;
+        if (availPages.isEmpty()) {
+            if (availableMemory >= FRAME_SIZE) {
+                /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's
+                 * allocateFrame() instead of direct ByteBuffer.allocate()?
+                 */
+                availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+                availableMemory -= FRAME_SIZE;
+                page = getAvailablePage();
+            } else {
+                page = evictPage();
+            }
+        } else {
+            page = getAvailablePage();
+        }
+
+        page.clear();
+
+        /*
+         * It is extremely important to update the reference after obtaining the page because, in the cases where
+         * memory manager is allocated only one page of memory, the front of the LRU list should not be created by the
+         * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the
+         * page, then make a updateReference call which in turn creates a new node in the LRU list and then add the
+         * page to it.
+         */
+        PartitionNode pn = updateReference(resultSetPartitionId, resultState);
+        pn.add(page);
+        return page;
+    }
+
+    public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
+        // When a page is referenced the result partition writer should already be known, so we pass null.
+        updateReference(resultSetPartitionId, null);
+    }
+
+    public static int getPageSize() {
+        return FRAME_SIZE;
+    }
+
+    protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
+        leastRecentlyUsedList.add(pn);
+        resultPartitionNodesMap.put(resultSetPartitionId, pn);
+    }
+
+    protected PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
+        PartitionNode pn = null;
+
+        if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
+            if (resultState != null) {
+                pn = new PartitionNode(resultSetPartitionId, resultState);
+                insertPartitionNode(resultSetPartitionId, pn);
+            }
+            return pn;
+        }
+        synchronized (this) {
+            pn = resultPartitionNodesMap.get(resultSetPartitionId);
+            leastRecentlyUsedList.remove(pn);
+            insertPartitionNode(resultSetPartitionId, pn);
+        }
+
+        return pn;
+    }
+
+    protected Page evictPage() throws HyracksDataException {
+        PartitionNode pn = leastRecentlyUsedList.getFirst();
+        ResultState resultState = pn.getResultState();
+        Page page = resultState.returnPage();
+
+        /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just
+         * take away all the pages allocated to it and add to the available pages set.
+         */
+        if (page == null) {
+            availPages.addAll(pn);
+            pn.clear();
+            resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+            leastRecentlyUsedList.remove(pn);
+
+            /* Based on the assumption that if the result partition writer returned a null page, it should be lying
+             * about the number of pages it holds in which case we just evict all the pages it holds and should thus be
+             * able to add all those pages to available set and we have at least one page to allocate back.
+             */
+            page = getAvailablePage();
+        } else {
+            pn.remove(page);
+
+            // If the partition no more holds any pages, remove it from the linked list and the hash map.
+            if (pn.isEmpty()) {
+                resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+                leastRecentlyUsedList.remove(pn);
+            }
+        }
+
+        return page;
+    }
+
+    protected Page getAvailablePage() {
+        Iterator<Page> iter = availPages.iterator();
+        Page page = iter.next();
+        iter.remove();
+        return page;
+    }
+
+    private class LeastRecentlyUsedList {
+        private PartitionNode head;
+
+        private PartitionNode tail;
+
+        public LeastRecentlyUsedList() {
+            head = null;
+            tail = null;
+        }
+
+        public void add(PartitionNode node) {
+            if (head == null) {
+                head = tail = node;
+                return;
+            }
+            tail.setNext(node);
+            node.setPrev(tail);
+            tail = node;
+        }
+
+        public void remove(PartitionNode node) {
+            if ((node == head) && (node == tail)) {
+                head = tail = null;
+                return;
+            } else if (node == head) {
+                head = head.getNext();
+                head.setPrev(null);
+                return;
+            } else if (node == tail) {
+                tail = tail.getPrev();
+                tail.setNext(null);
+                return;
+            } else {
+                PartitionNode prev = node.getPrev();
+                PartitionNode next = node.getNext();
+                prev.setNext(next);
+                next.setPrev(prev);
+            }
+        }
+
+        public PartitionNode getFirst() {
+            return head;
+        }
+    }
+
+    private class PartitionNode extends HashSet<Page> {
+        private static final long serialVersionUID = 1L;
+
+        private final ResultSetPartitionId resultSetPartitionId;
+
+        private final ResultState resultState;
+
+        private PartitionNode prev;
+
+        private PartitionNode next;
+
+        public PartitionNode(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
+            this.resultSetPartitionId = resultSetPartitionId;
+            this.resultState = resultState;
+            prev = null;
+            next = null;
+        }
+
+        public ResultSetPartitionId getResultSetPartitionId() {
+            return resultSetPartitionId;
+        }
+
+        public ResultState getResultState() {
+            return resultState;
+        }
+
+        public void setPrev(PartitionNode node) {
+            prev = node;
+        }
+
+        public PartitionNode getPrev() {
+            return prev;
+        }
+
+        public void setNext(PartitionNode node) {
+            next = node;
+        }
+
+        public PartitionNode getNext() {
+            return next;
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionManager.java
new file mode 100644
index 0000000..835b59b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionManager.java
@@ -0,0 +1,184 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executor;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.io.IWorkspaceFileFactory;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.control.common.result.AbstractResultManager;
+import org.apache.hyracks.control.common.result.ResultStateSweeper;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.control.nc.io.WorkspaceFileFactory;
+import org.apache.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResultPartitionManager extends AbstractResultManager implements IResultPartitionManager {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final NodeControllerService ncs;
+
+    private final Executor executor;
+
+    private final Map<JobId, ResultSetMap> partitionResultStateMap;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    private final ResultMemoryManager resultMemoryManager;
+
+    public ResultPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory, long resultTTL,
+            long resultSweepThreshold) {
+        super(resultTTL);
+        this.ncs = ncs;
+        this.executor = executor;
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, ncs.getIoManager());
+        if (availableMemory >= ResultMemoryManager.getPageSize()) {
+            resultMemoryManager = new ResultMemoryManager(availableMemory);
+        } else {
+            resultMemoryManager = null;
+        }
+        partitionResultStateMap = new HashMap<>();
+        executor.execute(new ResultStateSweeper(this, resultSweepThreshold, LOGGER));
+    }
+
+    @Override
+    public IFrameWriter createResultPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            boolean asyncMode, int partition, int nPartitions, long maxReads) {
+        ResultPartitionWriter dpw;
+        JobId jobId = ctx.getJobletContext().getJobId();
+        synchronized (this) {
+            dpw = new ResultPartitionWriter(ctx, this, jobId, rsId, asyncMode, orderedResult, partition, nPartitions,
+                    resultMemoryManager, fileFactory, maxReads);
+            ResultSetMap rsIdMap = partitionResultStateMap.computeIfAbsent(jobId, k -> new ResultSetMap());
+            ResultState[] resultStates = rsIdMap.createOrGetResultStates(rsId, nPartitions);
+            resultStates[partition] = dpw.getResultState();
+        }
+        LOGGER.trace("Initialized partition writer: JobId: {}:partition: {}", jobId, partition);
+        return dpw;
+    }
+
+    @Override
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, int partition, int nPartitions,
+            boolean orderedResult, boolean emptyResult) throws HyracksException {
+        try {
+            // Be sure to send the *public* network address to the CC
+            ncs.getClusterController(jobId.getCcId()).registerResultPartitionLocation(jobId, rsId, orderedResult,
+                    emptyResult, partition, nPartitions, ncs.getResultNetworkManager().getPublicNetworkAddress());
+        } catch (Exception e) {
+            throw HyracksException.create(e);
+        }
+    }
+
+    @Override
+    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+        try {
+            LOGGER.trace("Reporting partition write completion: JobId: {}:ResultSetId: {}:partition: {}", jobId, rsId,
+                    partition);
+            ncs.getClusterController(jobId.getCcId()).reportResultPartitionWriteCompletion(jobId, rsId, partition);
+        } catch (Exception e) {
+            throw HyracksException.create(e);
+        }
+    }
+
+    @Override
+    public void initializeResultPartitionReader(JobId jobId, ResultSetId resultSetId, int partition,
+            IFrameWriter writer) throws HyracksException {
+        ResultState resultState = getResultState(jobId, resultSetId, partition);
+        ResultPartitionReader dpr = new ResultPartitionReader(this, resultMemoryManager, executor, resultState);
+        dpr.writeTo(writer);
+        LOGGER.trace("Initialized partition reader: JobId: {}:ResultSetId: {}:partition: {}", jobId, resultSetId,
+                partition);
+    }
+
+    private synchronized ResultState getResultState(JobId jobId, ResultSetId resultSetId, int partition)
+            throws HyracksException {
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
+        if (rsIdMap == null) {
+            throw new HyracksException("Unknown JobId " + jobId);
+        }
+        ResultState[] resultStates = rsIdMap.getResultStates(resultSetId);
+        if (resultStates == null) {
+            throw new HyracksException("Unknown JobId: " + jobId + " ResultSetId: " + resultSetId);
+        }
+        ResultState resultState = resultStates[partition];
+        if (resultState == null) {
+            throw new HyracksException("No ResultPartitionWriter for partition " + partition);
+        }
+        return resultState;
+    }
+
+    @Override
+    public synchronized void removePartition(JobId jobId, ResultSetId resultSetId, int partition) {
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
+        if (rsIdMap != null && rsIdMap.removePartition(jobId, resultSetId, partition)) {
+            partitionResultStateMap.remove(jobId);
+        }
+    }
+
+    @Override
+    public synchronized void abortReader(JobId jobId) {
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
+        if (rsIdMap != null) {
+            rsIdMap.abortAll();
+        }
+    }
+
+    @Override
+    public synchronized void close() {
+        for (JobId jobId : getJobIds()) {
+            deinit(jobId);
+        }
+        deallocatableRegistry.close();
+    }
+
+    @Override
+    public synchronized Set<JobId> getJobIds() {
+        return partitionResultStateMap.keySet();
+    }
+
+    @Override
+    public synchronized ResultSetMap getState(JobId jobId) {
+        return partitionResultStateMap.get(jobId);
+    }
+
+    @Override
+    public synchronized void sweep(JobId jobId) {
+        deinit(jobId);
+        partitionResultStateMap.remove(jobId);
+    }
+
+    private synchronized void deinit(JobId jobId) {
+        ResultSetMap rsIdMap = partitionResultStateMap.get(jobId);
+        if (rsIdMap != null) {
+            rsIdMap.closeAndDeleteAll();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionReader.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionReader.java
new file mode 100644
index 0000000..3774530
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionReader.java
@@ -0,0 +1,111 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.partitions.ResultSetPartitionId;
+import org.apache.hyracks.comm.channels.NetworkOutputChannel;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResultPartitionReader {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final ResultPartitionManager resultPartitionManager;
+    private final ResultMemoryManager resultMemoryManager;
+    private final Executor executor;
+    private final ResultState resultState;
+
+    public ResultPartitionReader(ResultPartitionManager resultPartitionManager, ResultMemoryManager resultMemoryManager,
+            Executor executor, ResultState resultState) {
+        this.resultPartitionManager = resultPartitionManager;
+        this.resultMemoryManager = resultMemoryManager;
+        this.executor = executor;
+        this.resultState = resultState;
+    }
+
+    public void writeTo(final IFrameWriter writer) {
+        executor.execute(new ResultPartitionSender((NetworkOutputChannel) writer));
+    }
+
+    private class ResultPartitionSender implements Runnable {
+
+        private final NetworkOutputChannel channel;
+
+        ResultPartitionSender(final NetworkOutputChannel channel) {
+            this.channel = channel;
+        }
+
+        @Override
+        public void run() {
+            channel.setFrameSize(resultState.getFrameSize());
+            channel.open();
+            try {
+                resultState.readOpen();
+                long offset = 0;
+                final ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
+                while (true) {
+                    buffer.clear();
+                    final long size = read(offset, buffer);
+                    if (size <= 0) {
+                        break;
+                    } else if (size < buffer.limit()) {
+                        throw new IllegalStateException(
+                                "Premature end of file - readSize: " + size + " buffer limit: " + buffer.limit());
+                    }
+                    offset += size;
+                    buffer.flip();
+                    channel.nextFrame(buffer);
+                }
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("result reading successful(" + resultState.getResultSetPartitionId() + ")");
+                }
+            } catch (Exception e) {
+                LOGGER.error(() -> "failed to send result partition " + resultState.getResultSetPartitionId(), e);
+                channel.abort(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+            } finally {
+                close();
+            }
+        }
+
+        private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+            return resultMemoryManager != null ? resultState.read(resultMemoryManager, offset, buffer)
+                    : resultState.read(offset, buffer);
+        }
+
+        private void close() {
+            try {
+                channel.close();
+                resultState.readClose();
+                if (resultState.isExhausted()) {
+                    final ResultSetPartitionId partitionId = resultState.getResultSetPartitionId();
+                    resultPartitionManager.removePartition(partitionId.getJobId(), partitionId.getResultSetId(),
+                            partitionId.getPartition());
+                }
+            } catch (HyracksDataException e) {
+                LOGGER.error("unexpected failure in partition reader clean up", e);
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionWriter.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionWriter.java
new file mode 100644
index 0000000..53f66e7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultPartitionWriter.java
@@ -0,0 +1,138 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.io.IWorkspaceFileFactory;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.partitions.ResultSetPartitionId;
+import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResultPartitionWriter implements IFrameWriter {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final IResultPartitionManager manager;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+
+    private final boolean orderedResult;
+
+    private final int partition;
+
+    private final int nPartitions;
+
+    private final ResultMemoryManager resultMemoryManager;
+
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final ResultState resultState;
+
+    private boolean partitionRegistered;
+
+    private boolean failed = false;
+
+    public ResultPartitionWriter(IHyracksTaskContext ctx, IResultPartitionManager manager, JobId jobId,
+            ResultSetId rsId, boolean asyncMode, boolean orderedResult, int partition, int nPartitions,
+            ResultMemoryManager resultMemoryManager, IWorkspaceFileFactory fileFactory, long maxReads) {
+        this.manager = manager;
+        this.jobId = jobId;
+        this.resultSetId = rsId;
+        this.orderedResult = orderedResult;
+        this.partition = partition;
+        this.nPartitions = nPartitions;
+        this.resultMemoryManager = resultMemoryManager;
+
+        resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
+        resultState = new ResultState(resultSetPartitionId, asyncMode, ctx.getIoManager(), fileFactory,
+                ctx.getInitialFrameSize(), maxReads);
+    }
+
+    public ResultState getResultState() {
+        return resultState;
+    }
+
+    @Override
+    public void open() {
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("open(" + partition + ")");
+        }
+        partitionRegistered = false;
+        resultState.open();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        registerResultPartitionLocation(false);
+        if (resultMemoryManager == null) {
+            resultState.write(buffer);
+        } else {
+            resultState.write(resultMemoryManager, buffer);
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        failed = true;
+        resultState.closeAndDelete();
+        resultState.abort();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("close(" + partition + ")");
+        }
+        try {
+            if (!failed) {
+                registerResultPartitionLocation(true);
+            }
+        } finally {
+            resultState.close();
+        }
+        try {
+            if (partitionRegistered) {
+                manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
+            }
+        } catch (HyracksException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    void registerResultPartitionLocation(boolean empty) throws HyracksDataException {
+        try {
+            if (!partitionRegistered) {
+                manager.registerResultPartitionLocation(jobId, resultSetId, partition, nPartitions, orderedResult,
+                        empty);
+                partitionRegistered = true;
+            }
+        } catch (HyracksException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultSetMap.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultSetMap.java
new file mode 100644
index 0000000..518dc48
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultSetMap.java
@@ -0,0 +1,123 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultStateRecord;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+class ResultSetMap implements IResultStateRecord, Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private final long timestamp;
+    private final HashMap<ResultSetId, ResultState[]> resultStateMap;
+
+    ResultSetMap() {
+        timestamp = System.nanoTime();
+        resultStateMap = new HashMap<>();
+    }
+
+    @Override
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    ResultState[] getResultStates(ResultSetId rsId) {
+        return resultStateMap.get(rsId);
+    }
+
+    ResultState[] createOrGetResultStates(ResultSetId rsId, int nPartitions) {
+        return resultStateMap.computeIfAbsent(rsId, (k) -> new ResultState[nPartitions]);
+    }
+
+    /**
+     * removes a result partition for a result set
+     *
+     * @param jobId
+     *            the id of the job that produced the result set
+     * @param resultSetId
+     *            the id of the result set
+     * @param partition
+     *            the partition number
+     * @return true, if all partitions for the resultSetId have been removed
+     */
+    boolean removePartition(JobId jobId, ResultSetId resultSetId, int partition) {
+        final ResultState[] resultStates = resultStateMap.get(resultSetId);
+        if (resultStates != null) {
+            final ResultState state = resultStates[partition];
+            if (state != null) {
+                state.closeAndDelete();
+                LOGGER.trace("Removing partition: {} for JobId: {}", partition, jobId);
+            }
+            resultStates[partition] = null;
+            boolean stateEmpty = true;
+            for (ResultState resState : resultStates) {
+                if (resState != null) {
+                    stateEmpty = false;
+                    break;
+                }
+            }
+            if (stateEmpty) {
+                resultStateMap.remove(resultSetId);
+            }
+            return resultStateMap.isEmpty();
+        }
+        return true;
+    }
+
+    void abortAll() {
+        applyToAllStates((rsId, state, i) -> state.abort());
+    }
+
+    void closeAndDeleteAll() {
+        applyToAllStates((rsId, state, i) -> {
+            state.closeAndDelete();
+            LOGGER.trace("Removing partition: {} for result set {}", i, rsId);
+        });
+    }
+
+    @FunctionalInterface
+    private interface StateModifier {
+        void modify(ResultSetId rsId, ResultState entry, int partition);
+    }
+
+    private void applyToAllStates(StateModifier modifier) {
+        for (Map.Entry<ResultSetId, ResultState[]> entry : resultStateMap.entrySet()) {
+            final ResultSetId rsId = entry.getKey();
+            final ResultState[] resultStates = entry.getValue();
+            if (resultStates == null) {
+                continue;
+            }
+            for (int i = 0; i < resultStates.length; i++) {
+                final ResultState state = resultStates[i];
+                if (state != null) {
+                    modifier.modify(rsId, state, i);
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultState.java
new file mode 100644
index 0000000..25d3f00
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/result/ResultState.java
@@ -0,0 +1,381 @@
+/*
+ * 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.hyracks.control.nc.result;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hyracks.api.dataflow.state.IStateObject;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IFileHandle;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IWorkspaceFileFactory;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.partitions.ResultSetPartitionId;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class ResultState implements IStateObject {
+    private static final String FILE_PREFIX = "result_";
+
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final boolean asyncMode;
+
+    private final int frameSize;
+
+    private final IIOManager ioManager;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    private final AtomicBoolean eos;
+
+    private final AtomicBoolean failed;
+
+    private final List<Page> localPageList;
+
+    private FileReference fileRef;
+
+    private IFileHandle fileHandle;
+
+    private volatile int referenceCount = 0;
+
+    private long size;
+
+    private long persistentSize;
+    private long remainingReads;
+
+    ResultState(ResultSetPartitionId resultSetPartitionId, boolean asyncMode, IIOManager ioManager,
+            IWorkspaceFileFactory fileFactory, int frameSize, long maxReads) {
+        if (maxReads <= 0) {
+            throw new IllegalArgumentException("maxReads must be > 0");
+        }
+        this.resultSetPartitionId = resultSetPartitionId;
+        this.asyncMode = asyncMode;
+        this.ioManager = ioManager;
+        this.fileFactory = fileFactory;
+        this.frameSize = frameSize;
+        remainingReads = maxReads;
+        eos = new AtomicBoolean(false);
+        failed = new AtomicBoolean(false);
+        localPageList = new ArrayList<>();
+
+        fileRef = null;
+        fileHandle = null;
+    }
+
+    public synchronized void open() {
+        size = 0;
+        persistentSize = 0;
+        referenceCount = 0;
+    }
+
+    public synchronized void close() {
+        eos.set(true);
+        closeWriteFileHandle();
+        notifyAll();
+    }
+
+    public synchronized void closeAndDelete() {
+        // Deleting a job is equivalent to aborting the job for all practical purposes, so the same action, needs
+        // to be taken when there are more requests to these result states.
+        failed.set(true);
+        closeWriteFileHandle();
+        if (fileRef != null) {
+            fileRef.delete();
+            fileRef = null;
+        }
+    }
+
+    private void closeWriteFileHandle() {
+        if (fileHandle != null) {
+            doCloseFileHandle();
+        }
+    }
+
+    private void doCloseFileHandle() {
+        if (--referenceCount == 0) {
+            // close the file if there is no more reference
+            try {
+                ioManager.close(fileHandle);
+            } catch (IOException e) {
+                // Since file handle could not be closed, just ignore.
+            }
+            fileHandle = null;
+        }
+    }
+
+    public synchronized void write(ByteBuffer buffer) throws HyracksDataException {
+        if (fileRef == null) {
+            initWriteFileHandle();
+        }
+
+        size += ioManager.syncWrite(fileHandle, size, buffer);
+        notifyAll();
+    }
+
+    public synchronized void write(ResultMemoryManager resultMemoryManager, ByteBuffer buffer)
+            throws HyracksDataException {
+        int srcOffset = 0;
+        Page destPage = null;
+
+        if (!localPageList.isEmpty()) {
+            destPage = localPageList.get(localPageList.size() - 1);
+        }
+
+        while (srcOffset < buffer.limit()) {
+            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+                destPage = resultMemoryManager.requestPage(resultSetPartitionId, this);
+                localPageList.add(destPage);
+            }
+            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+            srcOffset += srcLength;
+            size += srcLength;
+        }
+
+        notifyAll();
+    }
+
+    public synchronized void readOpen() {
+        if (isExhausted()) {
+            throw new IllegalStateException("Result reads exhausted");
+        }
+        remainingReads--;
+    }
+
+    public synchronized void readClose() throws HyracksDataException {
+        if (fileHandle != null) {
+            doCloseFileHandle();
+        }
+    }
+
+    public synchronized long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+        long readSize = 0;
+
+        while (offset >= size && !eos.get() && !failed.get()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+        if ((offset >= size && eos.get()) || failed.get()) {
+            return readSize;
+        }
+
+        if (fileHandle == null) {
+            initReadFileHandle();
+        }
+        readSize = ioManager.syncRead(fileHandle, offset, buffer);
+
+        return readSize;
+    }
+
+    public synchronized long read(ResultMemoryManager resultMemoryManager, long offset, ByteBuffer buffer)
+            throws HyracksDataException {
+        long readSize = 0;
+        while (offset >= size && !eos.get() && !failed.get()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+
+        if ((offset >= size && eos.get()) || failed.get()) {
+            return readSize;
+        }
+
+        if (offset < persistentSize) {
+            if (fileHandle == null) {
+                initReadFileHandle();
+            }
+            readSize = ioManager.syncRead(fileHandle, offset, buffer);
+            if (readSize < 0) {
+                throw new HyracksDataException("Premature end of file");
+            }
+        }
+
+        if (readSize < buffer.capacity()) {
+            long localPageOffset = offset - persistentSize;
+            int localPageIndex = (int) (localPageOffset / ResultMemoryManager.getPageSize());
+            int pageOffset = (int) (localPageOffset % ResultMemoryManager.getPageSize());
+            Page page = getPage(localPageIndex);
+            if (page == null) {
+                return readSize;
+            }
+            readSize += buffer.remaining();
+            buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+        }
+        resultMemoryManager.pageReferenced(resultSetPartitionId);
+        return readSize;
+    }
+
+    public synchronized void abort() {
+        failed.set(true);
+        notifyAll();
+    }
+
+    public synchronized Page returnPage() throws HyracksDataException {
+        Page page = removePage();
+
+        // If we do not have any pages to be given back close the write channel since we don't write any more, return
+        // null.
+        if (page == null) {
+            ioManager.close(fileHandle);
+            return null;
+        }
+
+        page.getBuffer().flip();
+
+        if (fileRef == null) {
+            initWriteFileHandle();
+        }
+
+        long delta = ioManager.syncWrite(fileHandle, persistentSize, page.getBuffer());
+        persistentSize += delta;
+        return page;
+    }
+
+    public synchronized void setEOS(boolean eos) {
+        this.eos.set(eos);
+    }
+
+    public ResultSetPartitionId getResultSetPartitionId() {
+        return resultSetPartitionId;
+    }
+
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    public IIOManager getIOManager() {
+        return ioManager;
+    }
+
+    public boolean getAsyncMode() {
+        return asyncMode;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return resultSetPartitionId.getJobId();
+    }
+
+    @Override
+    public Object getId() {
+        return resultSetPartitionId;
+    }
+
+    @Override
+    public long getMemoryOccupancy() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    private Page getPage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(index);
+        }
+        return page;
+    }
+
+    private Page removePage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.remove(localPageList.size() - 1);
+        }
+        return page;
+    }
+
+    private void initWriteFileHandle() throws HyracksDataException {
+        if (fileHandle == null) {
+            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
+            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
+            fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+            if (referenceCount != 0) {
+                throw new IllegalStateException("Illegal reference count " + referenceCount);
+            }
+            referenceCount = 1;
+            notifyAll(); // NOSONAR: always called from a synchronized block
+        }
+    }
+
+    private void initReadFileHandle() throws HyracksDataException {
+        while (fileRef == null && !failed.get()) {
+            // wait for writer to create the file
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+        if (failed.get()) {
+            return;
+        }
+        if (fileHandle == null) {
+            // fileHandle has been closed by the writer, create it again
+            fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        }
+        referenceCount++;
+    }
+
+    @Override
+    public String toString() {
+        try {
+            ObjectMapper om = new ObjectMapper();
+            ObjectNode on = om.createObjectNode();
+            on.put("rspid", resultSetPartitionId.toString());
+            on.put("async", asyncMode);
+            on.put("remainingReads", remainingReads);
+            on.put("eos", eos.get());
+            on.put("failed", failed.get());
+            on.put("fileRef", String.valueOf(fileRef));
+            return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
+        } catch (JsonProcessingException e) { // NOSONAR
+            return e.getMessage();
+        }
+    }
+
+    public synchronized boolean isExhausted() {
+        return remainingReads == 0;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/HeartbeatAckTask.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/HeartbeatAckTask.java
new file mode 100644
index 0000000..f43c029
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/HeartbeatAckTask.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hyracks.control.nc.task;
+
+import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HeartbeatAckTask implements Runnable {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final NodeControllerService ncs;
+    private final CcId ccId;
+    private final HyracksDataException exception;
+
+    public HeartbeatAckTask(NodeControllerService ncs, CcId ccId, HyracksDataException exception) {
+        this.ncs = ncs;
+        this.ccId = ccId;
+        this.exception = exception;
+    }
+
+    @Override
+    public void run() {
+        try {
+            ncs.getHeartbeatManager(ccId).notifyAck(exception);
+        } catch (Exception e) {
+            LOGGER.info("failure processing heartbeat ack from {}", ccId, e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/PingTask.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/PingTask.java
new file mode 100644
index 0000000..15c62bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/task/PingTask.java
@@ -0,0 +1,45 @@
+/*
+ * 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.hyracks.control.nc.task;
+
+import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class PingTask implements Runnable {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final NodeControllerService ncs;
+    private final CcId ccId;
+
+    public PingTask(NodeControllerService ncs, CcId ccId) {
+        this.ncs = ncs;
+        this.ccId = ccId;
+    }
+
+    @Override
+    public void run() {
+        try {
+            ncs.getClusterController(ccId).notifyPingResponse(ncs.getId());
+        } catch (Exception e) {
+            LOGGER.info("failed to respond to ping from cc {}", ccId, e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllJobsWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllJobsWork.java
index c6696fd..b11dada 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllJobsWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortAllJobsWork.java
@@ -23,9 +23,9 @@
 import java.util.Deque;
 
 import org.apache.hyracks.api.control.CcId;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 import org.apache.hyracks.control.nc.Joblet;
 import org.apache.hyracks.control.nc.NodeControllerService;
@@ -48,9 +48,9 @@
     @Override
     protected void doRun() throws Exception {
         LOGGER.info("Aborting all tasks for controller {}", ccId);
-        IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
-        if (dpm == null) {
-            LOGGER.log(Level.WARN, "DatasetPartitionManager is null on " + ncs.getId());
+        IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager();
+        if (resultPartitionManager == null) {
+            LOGGER.log(Level.WARN, "ResultPartitionManager is null on " + ncs.getId());
         }
         Deque<Task> abortedTasks = new ArrayDeque<>();
         Collection<Joblet> joblets = ncs.getJobletMap().values();
@@ -61,9 +61,9 @@
                 abortedTasks.add(task);
             });
             final JobId jobId = joblet.getJobId();
-            if (dpm != null) {
-                dpm.abortReader(jobId);
-                dpm.sweep(jobId);
+            if (resultPartitionManager != null) {
+                resultPartitionManager.abortReader(jobId);
+                resultPartitionManager.sweep(jobId);
             }
             ncs.getWorkQueue().schedule(new CleanupJobletWork(ncs, jobId, JobStatus.FAILURE));
         });
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
index 80f3e98..f47e1ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/AbortTasksWork.java
@@ -22,8 +22,8 @@
 import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.Joblet;
 import org.apache.hyracks.control.nc.NodeControllerService;
@@ -52,9 +52,9 @@
         if (LOGGER.isInfoEnabled()) {
             LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
         }
-        IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
-        if (dpm != null) {
-            ncs.getDatasetPartitionManager().abortReader(jobId);
+        IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager();
+        if (resultPartitionManager != null) {
+            ncs.getResultPartitionManager().abortReader(jobId);
         }
         Joblet ji = ncs.getJobletMap().get(jobId);
         if (ji != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index 33d1d60..6d4f173 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -46,7 +46,7 @@
     public void run() {
         NCServiceContext ctx = ncs.getContext();
         try {
-            IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);;
+            IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);
             if (ctx.getMessageBroker() != null) {
                 ctx.getMessageBroker().receivedMessage(data, nodeId);
             } else {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
index c5a9d73..ae2cfa0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
@@ -45,11 +45,9 @@
 
     @Override
     public void run() {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Cleaning up after job: " + jobId);
-        }
+        LOGGER.debug("cleaning up after job: {}", jobId);
         ncs.removeJobParameterByteStore(jobId);
-        ncs.getPartitionManager().jobCompleted(jobId, status);;
+        ncs.getPartitionManager().jobCompleted(jobId, status);
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
         Joblet joblet = jobletMap.remove(jobId);
         if (joblet != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
index 92612dd..bcdb97f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
@@ -28,7 +28,7 @@
 import org.apache.hyracks.control.nc.NodeControllerService;
 
 /**
- * pre-distribute a job that can be executed later
+ * Deploy a job that can be executed later
  *
  */
 public class DeployJobSpecWork extends AbstractWork {
@@ -37,19 +37,23 @@
     private final byte[] acgBytes;
     private final CcId ccId;
     private final DeployedJobSpecId deployedJobSpecId;
+    private final boolean upsert;
 
     public DeployJobSpecWork(NodeControllerService ncs, DeployedJobSpecId deployedJobSpecId, byte[] acgBytes,
-            CcId ccId) {
+            boolean upsert, CcId ccId) {
         this.ncs = ncs;
         this.deployedJobSpecId = deployedJobSpecId;
         this.acgBytes = acgBytes;
         this.ccId = ccId;
+        this.upsert = upsert;
     }
 
     @Override
     public void run() {
         try {
-            ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+            if (!upsert) {
+                ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+            }
             ActivityClusterGraph acg =
                     (ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, null, ncs.getContext());
             ncs.storeActivityClusterGraph(deployedJobSpecId, acg);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
index 5964c04..0f36c80 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/EnsureAllCcTasksCompleted.java
@@ -66,11 +66,11 @@
                 LOGGER.error("{} tasks associated with CC {} failed to complete after {}ms. Giving up",
                         runningTasks.size(), ccId, TIMEOUT);
                 logPendingTasks();
-                ExitUtil.halt(ExitUtil.NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
+                ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
             }
         } catch (Throwable th) {
             LOGGER.error("Failed to abort all previous tasks associated with CC {}", ccId, th);
-            ExitUtil.halt(ExitUtil.NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
+            ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
index 614a9e0..554c660 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
@@ -51,6 +51,7 @@
 
     @Override
     public String toString() {
-        return getClass().getSimpleName() + ":" + task.getTaskAttemptId();
+        return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId()
+                + "]";
     }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
index f0b68a0..b0c60aa 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskFailureWork.java
@@ -21,8 +21,9 @@
 import java.util.List;
 
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.control.nc.Task;
@@ -49,12 +50,13 @@
 
     @Override
     public void run() {
-        LOGGER.log(Level.WARN, ncs.getId() + " is sending a notification to cc that task " + taskId + " has failed",
-                exceptions.get(0));
+        Exception ex = exceptions.get(0);
+        LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN, "task " + taskId + " has failed",
+                ex);
         try {
-            IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
-            if (dpm != null) {
-                dpm.abortReader(jobId);
+            IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager();
+            if (resultPartitionManager != null) {
+                resultPartitionManager.abortReader(jobId);
             }
             ncs.getClusterController(jobId.getCcId()).notifyTaskFailure(jobId, taskId, ncs.getId(), exceptions);
         } catch (Exception e) {
@@ -64,4 +66,9 @@
             task.getJoblet().removeTask(task);
         }
     }
+
+    @Override
+    public String toString() {
+        return getName() + ": [" + ncs.getId() + "[" + jobId + ":" + taskId + "]";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 660621e..09a4c18 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -147,7 +147,7 @@
                 ActivityId aid = tid.getActivityId();
                 ActivityCluster ac = acg.getActivityMap().get(aid);
                 IActivity han = ac.getActivityMap().get(aid);
-                LOGGER.info("Initializing {} -> {} for {}", taId, han, jobId);
+                LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
                 final int partition = tid.getPartition();
                 List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
                 task = null;
@@ -159,7 +159,7 @@
                     for (int i = 0; i < inputs.size(); ++i) {
                         IConnectorDescriptor conn = inputs.get(i);
                         IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
-                        LOGGER.info("input: {}: {}", i, conn.getConnectorId());
+                        LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         RecordDescriptor recordDesc = ac.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
                         IPartitionCollector collector =
                                 createPartitionCollector(td, partition, task, i, conn, recordDesc, cPolicy);
@@ -176,7 +176,7 @@
 
                         IPartitionWriterFactory pwFactory =
                                 createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
-                        LOGGER.info("input: {}: {}", i, conn.getConnectorId());
+                        LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
                                 td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
                         writer = enforce ? EnforceFrameWriter.enforce(writer) : writer;
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
index 2a06010..5076260 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
@@ -52,5 +52,9 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
index d1fc8a2..ff9758c 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
@@ -20,10 +20,16 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class PointableBinaryComparatorFactory implements IBinaryComparatorFactory {
     private static final long serialVersionUID = 1L;
 
@@ -52,4 +58,18 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode classIdentifier = registry.getClassIdentifier(getClass(), serialVersionUID);
+        classIdentifier.set("pointableFactory", pf.toJson(registry));
+        return classIdentifier;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final IPointableFactory pointableFactory =
+                (IPointableFactory) registry.deserialize(json.get("pointableFactory"));
+        return of(pointableFactory);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/AbstractPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/AbstractPointable.java
index 05417a8..2e21048 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/AbstractPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/AbstractPointable.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.data.std.api;
 
 public abstract class AbstractPointable implements IPointable {
+
     protected byte[] bytes;
 
     protected int start;
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/IPointableFactory.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/IPointableFactory.java
index 5294d17..bd71697 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/IPointableFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/api/IPointableFactory.java
@@ -21,9 +21,10 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.io.IJsonSerializable;
 
-public interface IPointableFactory extends Serializable {
-    public IPointable createPointable();
+public interface IPointableFactory extends Serializable, IJsonSerializable {
+    IPointable createPointable();
 
-    public ITypeTraits getTypeTraits();
+    ITypeTraits getTypeTraits();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BooleanPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BooleanPointable.java
index bd18ea0..ca26ef8 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BooleanPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BooleanPointable.java
@@ -19,39 +19,31 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class BooleanPointable extends AbstractPointable implements IHashable, IComparable {
 
     public static final BooleanPointableFactory FACTORY = new BooleanPointableFactory();
-
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 1;
-        }
-    };
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(1);
 
     public static class BooleanPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
-        public IPointable createPointable() {
+        public BooleanPointable createPointable() {
             return new BooleanPointable();
         }
 
-        public IPointable createPointable(boolean value) {
+        public BooleanPointable createPointable(boolean value) {
             BooleanPointable pointable = new BooleanPointable();
             pointable.setBoolean(value);
             return pointable;
@@ -61,6 +53,16 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
     }
 
     public static boolean getBoolean(byte[] bytes, int start) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ByteArrayPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ByteArrayPointable.java
index 98700a3..9a4c158 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ByteArrayPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ByteArrayPointable.java
@@ -23,6 +23,9 @@
 import java.util.Arrays;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -30,7 +33,11 @@
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class ByteArrayPointable extends AbstractPointable implements IHashable, IComparable, Serializable {
+
+    public static final ByteArrayPointableFactory FACTORY = new ByteArrayPointableFactory();
     private static final long serialVersionUID = 1L;
 
     // These three values are cached to speed up the length data access.
@@ -47,21 +54,7 @@
         hash = 0;
     }
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class ByteArrayPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -71,9 +64,19 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     @Override
     public int compareTo(IPointable pointer) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BytePointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BytePointable.java
index 5b623e7..ebfa903 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BytePointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/BytePointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,22 +29,14 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class BytePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
+    public static final BytePointableFactory FACTORY = new BytePointableFactory();
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(1);
 
-        @Override
-        public int getFixedLength() {
-            return 1;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class BytePointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -53,7 +48,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static byte getByte(byte[] bytes, int start) {
         return bytes[start];
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/DoublePointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/DoublePointable.java
index 3464a9f..f34bb13 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/DoublePointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/DoublePointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,32 +29,31 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
-public final class DoublePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
-    private final static double machineEpsilon;
-    static {
-        float epsilon = 1.0f;
+import com.fasterxml.jackson.databind.JsonNode;
 
+public final class DoublePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+
+    public static final DoublePointableFactory FACTORY = new DoublePointableFactory();
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(8) {
+        private static final long serialVersionUID = 7348262203696059687L;
+
+        //TODO fix RTREE logic based on class comparision in LSMRTreeUtils#proposeBestLinearizer
+        @Override
+        public boolean isFixedLength() {
+            return super.isFixedLength();
+        }
+    };
+    private static final double MACHINE_EPSILON = getMachineEpsilon();
+
+    private static double getMachineEpsilon() {
+        float epsilon = 1.0f;
         do {
             epsilon /= 2.0f;
         } while ((float) (1.0 + (epsilon / 2.0)) != 1.0);
-        machineEpsilon = epsilon;
+        return epsilon;
     }
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 8;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static class DoublePointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -63,7 +65,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static long getLongBits(byte[] bytes, int start) {
         return LongPointable.getLong(bytes, start);
@@ -150,6 +162,6 @@
     }
 
     public static double getEpsilon() {
-        return machineEpsilon;
+        return MACHINE_EPSILON;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FixedLengthTypeTrait.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FixedLengthTypeTrait.java
new file mode 100644
index 0000000..3658ddb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FixedLengthTypeTrait.java
@@ -0,0 +1,59 @@
+/*
+ * 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.hyracks.data.std.primitive;
+
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class FixedLengthTypeTrait implements ITypeTraits {
+
+    private static final long serialVersionUID = 1L;
+    private final int fixedLength;
+
+    public FixedLengthTypeTrait(int fixedLength) {
+        this.fixedLength = fixedLength;
+    }
+
+    @Override
+    public boolean isFixedLength() {
+        return true;
+    }
+
+    @Override
+    public int getFixedLength() {
+        return fixedLength;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("fixedLength", fixedLength);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new FixedLengthTypeTrait(json.get("fixedLength").asInt());
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
index b0f686f..926ee32 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,22 +29,14 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class FloatPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(4);
+    public static final FloatPointableFactory FACTORY = new FloatPointableFactory();
 
-        @Override
-        public int getFixedLength() {
-            return 4;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class FloatPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -53,7 +48,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     private static int getIntBits(byte[] bytes, int start) {
         return IntegerPointable.getInteger(bytes, start);
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/IntegerPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/IntegerPointable.java
index a3b9f44..9c29ddd 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/IntegerPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/IntegerPointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,22 +29,22 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
-public final class IntegerPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
+import com.fasterxml.jackson.databind.JsonNode;
 
+public final class IntegerPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+
+    public static final IntegerPointableFactory FACTORY = new IntegerPointableFactory();
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(4) {
+        private static final long serialVersionUID = -7178318032449879790L;
+
+        //TODO fix RTREE logic based on class comparision in LSMRTreeUtils#proposeBestLinearizer
         @Override
         public boolean isFixedLength() {
-            return true;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 4;
+            return super.isFixedLength();
         }
     };
 
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class IntegerPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -53,7 +56,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static int getInteger(byte[] bytes, int start) {
         return ((bytes[start] & 0xff) << 24) + ((bytes[start + 1] & 0xff) << 16) + ((bytes[start + 2] & 0xff) << 8)
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/LongPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/LongPointable.java
index 0f27e63..007d0c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/LongPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/LongPointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,21 +29,12 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class LongPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+
     public static final LongPointableFactory FACTORY = new LongPointableFactory();
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 8;
-        }
-    };
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(8);
 
     public static class LongPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
@@ -63,6 +57,16 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
     }
 
     public static long getLong(byte[] bytes, int start) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/RawUTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/RawUTF8StringPointable.java
index 70bac4d..7af12f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/RawUTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/RawUTF8StringPointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,27 +29,18 @@
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /**
  * This class provides the raw bytes-based comparison and hash function for UTF8 strings.
  * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
  * But it works for single-byte character languages.
  */
 public final class RawUTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
+    public static final RawUTF8StringPointableFactory FACTORY = new RawUTF8StringPointableFactory();
 
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class RawUTF8StringPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -56,9 +50,19 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     @Override
     public int compareTo(IPointable pointer) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ShortPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ShortPointable.java
index 70ad44c..7332ba8 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ShortPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/ShortPointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,22 +29,14 @@
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class ShortPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
+    public static final ITypeTraits TYPE_TRAITS = new FixedLengthTypeTrait(2);
+    public static final IPointableFactory FACTORY = new ShortPointableFactory();
 
-        @Override
-        public int getFixedLength() {
-            return 2;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class ShortPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -53,7 +48,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static short getShort(byte[] bytes, int start) {
         return (short) (((bytes[start] & 0xff) << 8) + (bytes[start + 1] & 0xff));
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/TaggedValuePointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/TaggedValuePointable.java
index 7acd32c..c67a5ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/TaggedValuePointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/TaggedValuePointable.java
@@ -17,26 +17,17 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
-import org.apache.hyracks.data.std.primitive.BytePointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+import com.fasterxml.jackson.databind.JsonNode;
 
 public class TaggedValuePointable extends AbstractPointable {
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public ITypeTraits getTypeTraits() {
-            return VoidPointable.TYPE_TRAITS;
-        }
-
-        @Override
-        public IPointable createPointable() {
-            return new TaggedValuePointable();
-        }
-    };
+    public static final TaggedValuePointableFactory FACTORY = new TaggedValuePointableFactory();
 
     public byte getTag() {
         return BytePointable.getByte(bytes, start);
@@ -45,4 +36,31 @@
     public void getValue(IPointable value) {
         value.set(bytes, start + 1, length - 1);
     }
+
+    public static final class TaggedValuePointableFactory implements IPointableFactory {
+        private static final long serialVersionUID = 1L;
+
+        private TaggedValuePointableFactory() {
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return VoidPointable.TYPE_TRAITS;
+        }
+
+        @Override
+        public TaggedValuePointable createPointable() {
+            return new TaggedValuePointable();
+        }
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercasePointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercasePointable.java
index 6e4810c..da5696a 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercasePointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercasePointable.java
@@ -19,6 +19,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -26,22 +29,13 @@
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class UTF8StringLowercasePointable extends AbstractPointable implements IHashable, IComparable {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
+    public static final UTF8StringLowercasePointableFactory FACTORY = new UTF8StringLowercasePointableFactory();
 
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class UTF8StringLowercasePointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -51,9 +45,19 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     @Override
     public int compareTo(IPointable pointer) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercaseTokenPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercaseTokenPointable.java
index 66c1ab9..6aa0c00 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercaseTokenPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringLowercaseTokenPointable.java
@@ -15,6 +15,9 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -22,6 +25,8 @@
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /**
  * This lowercase string token pointable is for the UTF8 string that doesn't have length bytes in the beginning.
  * This pointable exists to represent a string token.
@@ -29,21 +34,11 @@
  * Instead, the length of this string is provided as a parameter.
  */
 public final class UTF8StringLowercaseTokenPointable extends AbstractPointable implements IHashable, IComparable {
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
+    public static final UTF8StringLowercaseTokenPointableFactory FACTORY =
+            new UTF8StringLowercaseTokenPointableFactory();
 
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
+    public static final class UTF8StringLowercaseTokenPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
         @Override
@@ -53,9 +48,19 @@
 
         @Override
         public ITypeTraits getTypeTraits() {
-            return TYPE_TRAITS;
+            return VarLengthTypeTrait.INSTANCE;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     // Set the length of this pointable
     public void setLength(int length) {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 8fdcd83..f683615 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -24,6 +24,9 @@
 
 import org.apache.commons.lang3.CharSet;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IComparable;
 import org.apache.hyracks.data.std.api.IHashable;
@@ -33,8 +36,12 @@
 import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class UTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
 
+    public static final UTF8StringPointableFactory FACTORY = new UTF8StringPointableFactory();
+    public static final ITypeTraits TYPE_TRAITS = VarLengthTypeTrait.INSTANCE;
     // These values are cached to speed up the length data access.
     // Since we are using the variable-length encoding, we can save the repeated decoding efforts.
     // WARNING: must call the resetConstants() method after each reset().
@@ -56,25 +63,14 @@
         stringLength = -1;
     }
 
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+    public static class UTF8StringPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public boolean isFixedLength() {
-            return false;
+        private UTF8StringPointableFactory() {
         }
 
         @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
-
-    public static final IPointableFactory FACTORY = new IPointableFactory() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public IPointable createPointable() {
+        public UTF8StringPointable createPointable() {
             return new UTF8StringPointable();
         }
 
@@ -82,7 +78,17 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
-    };
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
+    }
 
     public static UTF8StringPointable generateUTF8Pointable(String string) {
         byte[] bytes;
@@ -322,21 +328,25 @@
         builder.finish();
     }
 
-    public void substr(int charOffset, int charLength, UTF8StringBuilder builder, GrowableArray out)
+    /**
+     * @return {@code true} if substring was successfully written into given {@code out}, or
+     *         {@code false} if substring could not be obtained ({@code charOffset} or {@code charLength}
+     *         are less than 0 or starting position is greater than the input length)
+     */
+    public boolean substr(int charOffset, int charLength, UTF8StringBuilder builder, GrowableArray out)
             throws IOException {
-        substr(this, charOffset, charLength, builder, out);
+        return substr(this, charOffset, charLength, builder, out);
     }
 
-    public static void substr(UTF8StringPointable src, int charOffset, int charLength, UTF8StringBuilder builder,
+    /**
+     * @return {@code true} if substring was successfully written into given {@code out}, or
+     *         {@code false} if substring could not be obtained ({@code charOffset} or {@code charLength}
+     *         are less than 0 or starting position is greater than the input length)
+     */
+    public static boolean substr(UTF8StringPointable src, int charOffset, int charLength, UTF8StringBuilder builder,
             GrowableArray out) throws IOException {
-        // Really don't understand why we need to support the charOffset < 0 case.
-        // At this time, usually there is mistake on user side, we'd better give him a warning.
-        // assert charOffset >= 0;
-        if (charOffset < 0) {
-            charOffset = 0;
-        }
-        if (charLength < 0) {
-            charLength = 0;
+        if (charOffset < 0 || charLength < 0) {
+            return false;
         }
 
         int utfLen = src.getUTF8Length();
@@ -347,11 +357,7 @@
             chIdx++;
         }
         if (byteIdx >= utfLen) {
-            // Again, why do we tolerant this kind of mistakes?
-            // throw new StringIndexOutOfBoundsException(charOffset);
-            builder.reset(out, 0);
-            builder.finish();
-            return;
+            return false;
         }
 
         builder.reset(out, Math.min(utfLen - byteIdx, (int) (charLength * 1.0 * byteIdx / chIdx)));
@@ -362,6 +368,7 @@
             byteIdx += src.charSize(src.getMetaDataLength() + byteIdx);
         }
         builder.finish();
+        return true;
     }
 
     public void substrBefore(UTF8StringPointable match, UTF8StringBuilder builder, GrowableArray out)
@@ -587,6 +594,31 @@
         builder.finish();
     }
 
+    /**
+     * Generates a reversed string from an input source string
+     *
+     * @param srcPtr
+     *            , the input source string.
+     * @param builder
+     *            , a builder for the resulting string.
+     * @param out
+     *            , the storage for a result string.
+     * @throws IOException
+     */
+    public static void reverse(UTF8StringPointable srcPtr, UTF8StringBuilder builder, GrowableArray out)
+            throws IOException {
+        builder.reset(out, srcPtr.getUTF8Length());
+        int srcStart = srcPtr.getCharStartOffset();
+        int srcEnd = srcPtr.getStartOffset() + srcPtr.getLength() - 1;
+        for (int cursorIndex = srcEnd; cursorIndex >= srcStart; cursorIndex--) {
+            if (UTF8StringUtil.isCharStart(srcPtr.bytes, cursorIndex)) {
+                int charSize = UTF8StringUtil.charSize(srcPtr.bytes, cursorIndex);
+                builder.appendUtf8StringPointable(srcPtr, cursorIndex, charSize);
+            }
+        }
+        builder.finish();
+    }
+
     public boolean findAndReplace(UTF8StringPointable searchPtr, UTF8StringPointable replacePtr, int replaceLimit,
             UTF8StringBuilder builder, GrowableArray out) throws IOException {
         return findAndReplace(this, searchPtr, replacePtr, replaceLimit, builder, out);
@@ -595,9 +627,12 @@
     public static boolean findAndReplace(UTF8StringPointable srcPtr, UTF8StringPointable searchPtr,
             UTF8StringPointable replacePtr, int replaceLimit, UTF8StringBuilder builder, GrowableArray out)
             throws IOException {
-        if (replaceLimit < 1) {
+        if (replaceLimit == 0) {
             return false;
         }
+        if (replaceLimit < 0) {
+            replaceLimit = Integer.MAX_VALUE;
+        }
         int curIdx = find(srcPtr, searchPtr, false);
         if (curIdx < 0) {
             return false;
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VarLengthTypeTrait.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VarLengthTypeTrait.java
new file mode 100644
index 0000000..3c217a4
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VarLengthTypeTrait.java
@@ -0,0 +1,53 @@
+/*
+ * 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.hyracks.data.std.primitive;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+public class VarLengthTypeTrait implements ITypeTraits {
+    public static final VarLengthTypeTrait INSTANCE = new VarLengthTypeTrait();
+    private static final long serialVersionUID = 1L;
+
+    private VarLengthTypeTrait() {
+    }
+
+    @Override
+    public boolean isFixedLength() {
+        return false;
+    }
+
+    @Override
+    public int getFixedLength() {
+        return 0;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VoidPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VoidPointable.java
index 51c155e..802fd89 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VoidPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/VoidPointable.java
@@ -19,24 +19,18 @@
 package org.apache.hyracks.data.std.primitive;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.AbstractPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public final class VoidPointable extends AbstractPointable {
+
+    public static final ITypeTraits TYPE_TRAITS = VarLengthTypeTrait.INSTANCE;
     public static final VoidPointableFactory FACTORY = new VoidPointableFactory();
-    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
-        private static final long serialVersionUID = 1L;
-
-        @Override
-        public boolean isFixedLength() {
-            return false;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return 0;
-        }
-    };
 
     public static class VoidPointableFactory implements IPointableFactory {
         private static final long serialVersionUID = 1L;
@@ -53,5 +47,15 @@
         public ITypeTraits getTypeTraits() {
             return TYPE_TRAITS;
         }
+
+        @Override
+        public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+            return registry.getClassIdentifier(getClass(), serialVersionUID);
+        }
+
+        @SuppressWarnings("squid:S1172") // unused parameter
+        public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+            return FACTORY;
+        }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/BinaryHashSet.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/BinaryHashSet.java
index c5b8e8a..0f29514 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/BinaryHashSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/BinaryHashSet.java
@@ -232,11 +232,11 @@
     }
 
     private int getFrameIndex(int ptr) {
-        return (int) (ptr >> 16);
+        return ptr >> 16;
     }
 
     private int getFrameOffset(int ptr) {
-        return (int) (ptr & 0xffff);
+        return ptr & 0xffff;
     }
 
     public int size() {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
index 61235ca..406f566 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
@@ -83,6 +83,11 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
index 98acbc0..0495cfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
@@ -28,6 +28,8 @@
  * The first split value for each field followed by the second split value for each field, etc.
  */
 public class RangeMap implements IRangeMap, Serializable {
+    private static final long serialVersionUID = -7523433293419648234L;
+
     private final int fields;
     private final byte[] bytes;
     private final int[] offsets;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
index e99a2ff..ca3f021 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/SerdeUtils.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.BooleanPointable;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
+import org.apache.hyracks.data.std.primitive.FixedLengthTypeTrait;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.LongPointable;
@@ -41,24 +42,6 @@
 
 @SuppressWarnings("rawtypes")
 public class SerdeUtils {
-    public static class PayloadTypeTraits implements ITypeTraits {
-        private static final long serialVersionUID = 1L;
-        final int payloadSize;
-
-        public PayloadTypeTraits(int payloadSize) {
-            this.payloadSize = payloadSize;
-        }
-
-        @Override
-        public boolean isFixedLength() {
-            return true;
-        }
-
-        @Override
-        public int getFixedLength() {
-            return payloadSize;
-        }
-    }
 
     public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes) {
         ITypeTraits[] typeTraits = new ITypeTraits[serdes.length];
@@ -73,7 +56,7 @@
         for (int i = 0; i < serdes.length; i++) {
             typeTraits[i] = serdeToTypeTrait(serdes[i]);
         }
-        typeTraits[serdes.length] = new PayloadTypeTraits(payloadSize);
+        typeTraits[serdes.length] = new FixedLengthTypeTrait(payloadSize);
         return typeTraits;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
index 08ed922..49b5309 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
@@ -164,4 +164,30 @@
             tupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
         }
     }
+
+    public static boolean equalTuples(ITupleReference tuple1, ITupleReference tuple2, int numCmpFields) {
+        for (int i = 0; i < numCmpFields; i++) {
+            if (!equalFields(tuple1, tuple2, i)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static boolean equalFields(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
+        return equalFields(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx), tuple1.getFieldLength(fIdx),
+                tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx), tuple2.getFieldLength(fIdx));
+    }
+
+    public static boolean equalFields(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
+        if (aLength != bLength) {
+            return false;
+        }
+        for (int i = 0; i < aLength; i++) {
+            if (a[aOffset + i] != b[bOffset + i]) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactoryTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactoryTest.java
index 9122721..cd0cbeb 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactoryTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactoryTest.java
@@ -51,7 +51,7 @@
     }
 
     private IPointable getIntPointable(int value) {
-        IntegerPointable pointable = (IntegerPointable) IntegerPointable.FACTORY.createPointable();
+        IntegerPointable pointable = IntegerPointable.FACTORY.createPointable();
         pointable.set(new byte[Integer.BYTES], 0, Integer.BYTES);
         pointable.setInteger(value);
         return pointable;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index c78c98c..0ff425f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -76,6 +76,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-net</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>com.e-movimento.tinytools</groupId>
       <artifactId>privilegedaccessor</artifactId>
       <version>1.2.2</version>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index c6512929..205d1ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -43,6 +44,8 @@
 
     protected String displayName;
 
+    protected SourceLocation sourceLoc;
+
     public AbstractOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity) {
         odId = spec.createOperatorDescriptorId(this);
         this.inputArity = inputArity;
@@ -87,6 +90,16 @@
     }
 
     @Override
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    @Override
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
     public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCServiceContext serviceCtx) {
         // do nothing
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
index 0efed6f..f32adcc 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
@@ -25,40 +25,53 @@
 import org.apache.hyracks.api.comm.FrameHelper;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class InputChannelFrameReader implements IFrameReader, IInputChannelMonitor {
+    private static final Logger LOGGER = LogManager.getLogger();
     private final IInputChannel channel;
 
     private int availableFrames;
 
     private boolean eos;
 
-    private boolean failed;
+    private int errorCode;
 
     public InputChannelFrameReader(IInputChannel channel) {
         this.channel = channel;
         availableFrames = 0;
+        errorCode = AbstractChannelWriteInterface.NO_ERROR_CODE;
         eos = false;
-        failed = false;
     }
 
     @Override
     public void open() throws HyracksDataException {
     }
 
+    private boolean hasFailed() {
+        return errorCode != AbstractChannelWriteInterface.NO_ERROR_CODE;
+    }
+
     private synchronized boolean canGetNextBuffer() throws HyracksDataException {
-        while (!failed && !eos && availableFrames <= 0) {
+        while (!hasFailed() && !eos && availableFrames <= 0) {
             try {
                 wait();
             } catch (InterruptedException e) {
                 throw HyracksDataException.create(e);
             }
         }
-        if (failed) {
+        if (hasFailed()) {
+            if (errorCode == AbstractChannelWriteInterface.CONNECTION_LOST_ERROR_CODE) {
+                throw HyracksDataException.create(ErrorCode.LOCAL_NETWORK_ERROR);
+            }
             // Do not throw exception here to allow the root cause exception gets propagated to the master first.
             // Return false to allow the nextFrame(...) call to be a non-op.
+            LOGGER.warn("Sender failed.. returning silently");
             return false;
         }
         if (availableFrames <= 0 && eos) {
@@ -92,8 +105,7 @@
 
         for (int i = 1; i < nBlocks; ++i) {
             if (!canGetNextBuffer()) {
-                throw new HyracksDataException(
-                        "InputChannelReader is waiting for the new frames, but the input stream is finished");
+                return false;
             }
             srcFrame = channel.getNextBuffer();
             frame.getBuffer().put(srcFrame);
@@ -112,8 +124,11 @@
     }
 
     @Override
-    public synchronized void notifyFailure(IInputChannel channel) {
-        failed = true;
+    public synchronized void notifyFailure(IInputChannel channel, int errorCode) {
+        // Note: if a remote failure overwrites the value of localFailure, then we rely on
+        // the fact that the remote task will notify the cc of the failure.
+        // Otherwise, the local task must fail
+        this.errorCode = errorCode;
         notifyAll();
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
index b6f7cad..becbb00 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -23,8 +23,10 @@
 
 import org.apache.hyracks.api.channels.IInputChannel;
 import org.apache.hyracks.api.channels.IInputChannelMonitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -47,6 +49,8 @@
 
     private int lastReadSender;
 
+    private boolean localFailure;
+
     public NonDeterministicChannelReader(int nSenderPartitions, BitSet expectedPartitions) {
         this.nSenderPartitions = nSenderPartitions;
         channels = new IInputChannel[nSenderPartitions];
@@ -106,6 +110,10 @@
                 return lastReadSender;
             }
             if (!failSenders.isEmpty()) {
+                LOGGER.warn("Sender failed.. returning silently");
+                if (localFailure) {
+                    throw HyracksDataException.create(ErrorCode.LOCAL_NETWORK_ERROR);
+                }
                 // Do not throw exception here to allow the root cause exception gets propagated to the master first.
                 // Return a negative value to allow the nextFrame(...) call to be a non-op.
                 return -1;
@@ -140,13 +148,17 @@
     }
 
     @Override
-    public synchronized void notifyFailure(IInputChannel channel) {
+    public synchronized void notifyFailure(IInputChannel channel, int errorCode) {
         PartitionId pid = (PartitionId) channel.getAttachment();
         int senderIndex = pid.getSenderIndex();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Failure: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Failure: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
                     + pid.getReceiverIndex());
         }
+        // Note: if a remote failure overwrites the value of localFailure, then we rely on
+        // the fact that the remote task will notify the cc of the failure.
+        // Otherwise, the local task must fail
+        localFailure = errorCode == AbstractChannelWriteInterface.CONNECTION_LOST_ERROR_CODE;
         failSenders.set(senderIndex);
         eosSenders.set(senderIndex);
         notifyAll();
@@ -156,8 +168,8 @@
     public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
         PartitionId pid = (PartitionId) channel.getAttachment();
         int senderIndex = pid.getSenderIndex();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Data available: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Data available: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
                     + pid.getReceiverIndex());
         }
         availableFrameCounts[senderIndex] += nFrames;
@@ -169,8 +181,8 @@
     public synchronized void notifyEndOfStream(IInputChannel channel) {
         PartitionId pid = (PartitionId) channel.getAttachment();
         int senderIndex = pid.getSenderIndex();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("EOS: " + pid);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("EOS: " + pid);
         }
         eosSenders.set(senderIndex);
         notifyAll();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
index d546e5e..4426acb 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
@@ -23,7 +23,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public abstract class AbstractAccumulatingAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+public abstract class AbstractAccumulatingAggregatorDescriptorFactory extends AbstractAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..12ad3ec
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractAggregatorDescriptorFactory.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hyracks.dataflow.std.group;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+    private static final long serialVersionUID = 1L;
+
+    protected SourceLocation sourceLoc;
+
+    public SourceLocation getSourceLocation() {
+        return sourceLoc;
+    }
+
+    public void setSourceLocation(SourceLocation sourceLoc) {
+        this.sourceLoc = sourceLoc;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
deleted file mode 100644
index fd16c52..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.hyracks.dataflow.std.group;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.dataflow.IDataReader;
-import org.apache.hyracks.api.dataflow.IDataWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IGroupAggregator extends Serializable {
-    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
-
-    public void close() throws Exception;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
index 2b9ad54..664a3ea8 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
@@ -21,7 +21,6 @@
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public interface ISpillableTable {
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
index cb32c4a..4725911 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
index 290cc58..92cb679 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
index bba9900..a1856c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
index fc8d956..f0b8dca 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
index 90e1474..179297c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.std.group.AggregateState;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java
index 95994f3..999dbac 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java
@@ -125,7 +125,7 @@
                 int groupByColumnsCount = mergeGroupFields.length;
                 int hashTableCardinality = ExternalGroupOperatorDescriptor.calculateGroupByTableCardinality(
                         memoryBudgetInBytes, groupByColumnsCount, ctx.getInitialFrameSize());
-                hashTableCardinality = (int) Math.min(hashTableCardinality, numOfTuples[i]);
+                hashTableCardinality = Math.min(hashTableCardinality, numOfTuples[i]);
                 ISpillableTable partitionTable = spillableTableFactory.buildSpillableTable(ctx, hashTableCardinality,
                         runs[i].getFileSize(), mergeGroupFields, groupByComparators, nmkComputer,
                         mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, frameLimit, level);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index bee0590..dc250e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -91,13 +91,12 @@
      * @param hashFunctionFactories
      * @param comparatorFactories
      * @param recordDescriptor
-     * @throws HyracksDataException
      */
     public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
-            IMissingWriterFactory[] nullWriterFactories1) throws HyracksDataException {
+            IMissingWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.memsize = memsize;
         this.inputsize0 = inputsize0;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 10c6227..89c370f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -110,8 +110,10 @@
         } else {
             isTableCapacityNotZero = false;
         }
-        LOGGER.debug("InMemoryHashJoin has been created for a table size of " + table.getTableSize() + " for Thread ID "
-                + Thread.currentThread().getId() + ".");
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("InMemoryHashJoin has been created for a table size of " + table.getTableSize()
+                    + " for Thread ID " + Thread.currentThread().getId() + ".");
+        }
     }
 
     public void build(ByteBuffer buffer) throws HyracksDataException {
@@ -205,8 +207,8 @@
             }
         }
         buffers.clear();
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("InMemoryHashJoin has finished using " + nFrames + " frames for Thread ID "
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("InMemoryHashJoin has finished using " + nFrames + " frames for Thread ID "
                     + Thread.currentThread().getId() + ".");
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 9eeb363..e7984d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -144,7 +144,7 @@
             IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, ITuplePairComparatorFactory tupPaircomparatorFactory01,
             ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory,
-            boolean isLeftOuter, IMissingWriterFactory[] nonMatchWriterFactories) throws HyracksDataException {
+            boolean isLeftOuter, IMissingWriterFactory[] nonMatchWriterFactories) {
         super(spec, 2, 1);
         this.memSizeInFrames = memSizeInFrames;
         this.inputsize0 = inputsize0;
@@ -165,8 +165,7 @@
             int inputsize0, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, ITuplePairComparatorFactory tupPaircomparatorFactory01,
-            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory)
-            throws HyracksDataException {
+            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory) {
         this(spec, memSizeInFrames, inputsize0, factor, keys0, keys1, hashFunctionGeneratorFactories,
                 comparatorFactories, recordDescriptor, tupPaircomparatorFactory01, tupPaircomparatorFactory10,
                 predEvaluatorFactory, false, null);
@@ -296,8 +295,8 @@
                             buildHpc, predEvaluator, isLeftOuter, nonMatchWriterFactories);
 
                     state.hybridHJ.initBuild();
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("OptimizedHybridHashJoin is starting the build phase with " + state.numOfPartitions
+                    if (LOGGER.isTraceEnabled()) {
+                        LOGGER.trace("OptimizedHybridHashJoin is starting the build phase with " + state.numOfPartitions
                                 + " partitions using " + state.memForJoin + " frames for memory.");
                     }
                 }
@@ -314,8 +313,8 @@
                         state.hybridHJ.clearBuildTempFiles();
                     } else {
                         ctx.setStateObject(state);
-                        if (LOGGER.isDebugEnabled()) {
-                            LOGGER.debug("OptimizedHybridHashJoin closed its build phase");
+                        if (LOGGER.isTraceEnabled()) {
+                            LOGGER.trace("OptimizedHybridHashJoin closed its build phase");
                         }
                     }
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
index b0cc40c..9fb9815 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -31,11 +31,11 @@
 import org.apache.hyracks.api.dataflow.value.IResultSerializer;
 import org.apache.hyracks.api.dataflow.value.IResultSerializerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.result.IResultPartitionManager;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.dataflow.common.comm.io.FrameOutputStream;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -67,7 +67,7 @@
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
             throws HyracksDataException {
-        final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
+        final IResultPartitionManager resultPartitionManager = ctx.getResultPartitionManager();
 
         final IFrame frame = new VSizeFrame(ctx);
 
@@ -82,15 +82,15 @@
         final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(outRecordDesc);
 
         return new AbstractUnaryInputSinkOperatorNodePushable() {
-            private IFrameWriter datasetPartitionWriter;
+            private IFrameWriter resultPartitionWriter;
             private boolean failed = false;
 
             @Override
             public void open() throws HyracksDataException {
                 try {
-                    datasetPartitionWriter = dpm.createDatasetPartitionWriter(ctx, rsId, ordered, asyncMode, partition,
-                            nPartitions, maxReads);
-                    datasetPartitionWriter.open();
+                    resultPartitionWriter = resultPartitionManager.createResultPartitionWriter(ctx, rsId, ordered,
+                            asyncMode, partition, nPartitions, maxReads);
+                    resultPartitionWriter.open();
                     resultSerializer.init();
                 } catch (HyracksException e) {
                     throw HyracksDataException.create(e);
@@ -103,7 +103,7 @@
                 for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
                     resultSerializer.appendTuple(frameTupleAccessor, tIndex);
                     if (!frameOutputStream.appendTuple()) {
-                        frameOutputStream.flush(datasetPartitionWriter);
+                        frameOutputStream.flush(resultPartitionWriter);
 
                         resultSerializer.appendTuple(frameTupleAccessor, tIndex);
                         frameOutputStream.appendTuple();
@@ -114,23 +114,23 @@
             @Override
             public void fail() throws HyracksDataException {
                 failed = true;
-                if (datasetPartitionWriter != null) {
-                    datasetPartitionWriter.fail();
+                if (resultPartitionWriter != null) {
+                    resultPartitionWriter.fail();
                 }
             }
 
             @Override
             public void close() throws HyracksDataException {
-                if (datasetPartitionWriter != null) {
+                if (resultPartitionWriter != null) {
                     try {
                         if (!failed && frameOutputStream.getTupleCount() > 0) {
-                            frameOutputStream.flush(datasetPartitionWriter);
+                            frameOutputStream.flush(resultPartitionWriter);
                         }
                     } catch (Exception e) {
-                        datasetPartitionWriter.fail();
+                        resultPartitionWriter.fail();
                         throw e;
                     } finally {
-                        datasetPartitionWriter.close();
+                        resultPartitionWriter.close();
                     }
                 }
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
index f51271e..980ad9b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
@@ -235,8 +235,8 @@
         }
         maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
         outputAppender.write(writer, true);
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace(
                     "Flushed records:" + limit + " out of " + tupleCount + "; Flushed through " + (io + 1) + " frames");
         }
         return maxFrameSize;
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
index f274ca1..406703e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
@@ -132,8 +132,8 @@
                     runGen.close();
                     state.generatedRunFileReaders = runGen.getRuns();
                     state.sorter = runGen.getSorter();
-                    if (LOGGER.isInfoEnabled()) {
-                        LOGGER.info("InitialNumberOfRuns:" + runGen.getRuns().size());
+                    if (LOGGER.isTraceEnabled()) {
+                        LOGGER.trace("InitialNumberOfRuns:" + runGen.getRuns().size());
                     }
                     ctx.setStateObject(state);
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
index bfdf431..58b1891 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
@@ -31,7 +31,6 @@
 
     // @Test commented out due to ASTERIXDB-1881
     public void test() {
-        ;
         FileInputStream in = null;
         BufferedReader reader = null;
         try {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/TestNCApplication.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/TestNCApplication.java
index 15248e7..b13feda 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/TestNCApplication.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/TestNCApplication.java
@@ -46,7 +46,7 @@
     }
 
     @Override
-    public void onRegisterNode(CcId ccs) throws Exception {
+    public void tasksCompleted(CcId ccs) throws Exception {
         // No-op
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 840c6dd..4aaf5e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -117,7 +117,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index c44544a..d0a3531 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -85,7 +85,7 @@
         LSMRTREE,
         LSMRTREE_WITH_ANTIMATTER,
         RTREE
-    };
+    }
 
     protected RTreeType rTreeType;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractIntegrationTest.java
index 0931501..5dcc99a 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -36,16 +36,16 @@
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.IHyracksDatasetReader;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.control.common.controllers.NCConfig;
@@ -154,8 +154,8 @@
 
         IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
 
-        IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
-        IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
+        IResultSet resultSet = new ResultSet(hcc, spec.getFrameSize(), nReaders);
+        IResultSetReader reader = resultSet.createReader(jobId, resultSetId);
 
         List<String> resultRecords = new ArrayList<>();
         ByteBufferInputStream bbis = new ByteBufferInputStream();
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index 58da8a2..55fd9a0 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -29,8 +29,6 @@
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.IHyracksDatasetReader;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobFlag;
@@ -38,7 +36,9 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
-import org.apache.hyracks.client.dataset.HyracksDataset;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.control.cc.BaseCCApplication;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.application.CCServiceContext;
@@ -160,8 +160,8 @@
         IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
 
         if (!spec.getResultSetIds().isEmpty()) {
-            IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
-            IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
+            IResultSet resultSet = new ResultSet(hcc, spec.getFrameSize(), nReaders);
+            IResultSetReader reader = resultSet.createReader(jobId, spec.getResultSetIds().get(0));
 
             ObjectMapper om = new ObjectMapper();
             ArrayNode resultRecords = om.createArrayNode();
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
index 752c643..39e7a45 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
@@ -28,10 +28,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
index c28a5aa..c542fe9 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
@@ -27,10 +27,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
index 40b6b27..d8f4064 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
@@ -199,6 +199,16 @@
         for (int i = 0; i < 100; i++) {
             hcc.startJob(distributedId2, new HashMap<>());
         }
+
+        //Change the second job into the first job and see whether it runs
+        hcc.redeployJobSpec(distributedId2, spec1);
+        JobId jobRunId4 = hcc.startJob(distributedId2, new HashMap<>());
+        hcc.waitForCompletion(jobRunId4);
+
+        //Run it one more time
+        JobId jobRunId5 = hcc.startJob(distributedId2, new HashMap<>());
+        hcc.waitForCompletion(jobRunId5);
+
     }
 
     @AfterClass
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
index 9e795bf..1dce88d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
@@ -27,10 +27,10 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 49dee84..8d3b0dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -29,10 +29,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/NodesAPIIntegrationTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/NodesAPIIntegrationTest.java
index 672d2c4..9d79d80 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/NodesAPIIntegrationTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/NodesAPIIntegrationTest.java
@@ -45,8 +45,8 @@
             "nonheap-committed-sizes", "nonheap-max-sizes", "application-memory-budget", "application-cpu-core-budget",
             "thread-counts", "peak-thread-counts", "system-load-averages", "gc-names", "gc-collection-counts",
             "gc-collection-times", "net-payload-bytes-read", "net-payload-bytes-written", "net-signaling-bytes-read",
-            "net-signaling-bytes-written", "dataset-net-payload-bytes-read", "dataset-net-payload-bytes-written",
-            "dataset-net-signaling-bytes-read", "dataset-net-signaling-bytes-written", "ipc-messages-sent",
+            "net-signaling-bytes-written", "result-net-payload-bytes-read", "result-net-payload-bytes-written",
+            "result-net-signaling-bytes-read", "result-net-signaling-bytes-written", "ipc-messages-sent",
             "ipc-message-bytes-sent", "ipc-messages-received", "ipc-message-bytes-received", "disk-reads",
             "disk-writes", "config" };
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ReplicateOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ReplicateOperatorTest.java
index 09629b2..bc5de11 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ReplicateOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ReplicateOperatorTest.java
@@ -27,10 +27,10 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ScanPrintTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ScanPrintTest.java
index 75ba33f..7838a34 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ScanPrintTest.java
@@ -26,10 +26,10 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SortMergeTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SortMergeTest.java
index 315b74c..06c0eaa 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SortMergeTest.java
@@ -26,7 +26,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 816f3fa..6154e28 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -28,10 +28,10 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index c2b3263..0eb6810 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -32,11 +32,11 @@
 import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
 import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/UnionTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/UnionTest.java
index 81a71eb..48f7837 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/UnionTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/UnionTest.java
@@ -24,10 +24,10 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.io.FileSplit;
 import org.apache.hyracks.api.io.ManagedFileSplit;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java
index b55b64e..4c0d89f 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java
@@ -144,7 +144,7 @@
                             resultAccessor.getAbsoluteFieldStartOffset(tid, fid));
                     outRecord[fid] = outputRec.getFields()[fid].deserialize(di);
                 }
-                Result result = answer.remove((String) outRecord[0]);
+                Result result = answer.remove(outRecord[0]);
                 assertNotNull(result);
                 assertEquals(result.sum, (int) outRecord[1]);
                 assertEquals(result.count, (int) outRecord[2]);
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index 409abbd..bcc6c55 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -192,5 +192,11 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>xml-apis</groupId>
+      <artifactId>xml-apis</artifactId>
+      <version>2.0.2</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
index 0babc5f..7dda474 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -78,7 +78,7 @@
         return new AbstractUnaryInputSinkOperatorNodePushable() {
 
             private FSDataOutputStream dos;
-            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
             private FrameTupleReference tuple = new FrameTupleReference();
             private ITupleWriter tupleWriter;
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
index c691896..ec5c638 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -77,7 +77,7 @@
         return new AbstractUnaryInputSinkOperatorNodePushable() {
 
             private FSDataOutputStream dos;
-            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
             private FrameTupleReference tuple = new FrameTupleReference();
             private ITupleWriter tupleWriter;
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletResponse.java
index 38f2d23..95f8f27 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletResponse.java
@@ -86,4 +86,9 @@
      * Notifies the response that the channel has become inactive.
      */
     void notifyChannelInactive();
+
+    /**
+     * Called on a created request that is cancelled before it is started
+     */
+    void cancel();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
new file mode 100644
index 0000000..4732d71
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
@@ -0,0 +1,138 @@
+/*
+ * 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.hyracks.http.server;
+
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.channel.ChannelDuplexHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelPromise;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.DefaultHttpContent;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.HttpContent;
+import io.netty.handler.codec.http.HttpHeaderNames;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.LastHttpContent;
+
+//Based in part on LoggingHandler from Netty
+public class CLFLogger extends ChannelDuplexHandler {
+
+    private static final Logger accessLogger = LogManager.getLogger();
+    private static final Level ACCESS_LOG_LEVEL = Level.forName("ACCESS", 550);
+    private static final DateTimeFormatter DATE_TIME_FORMATTER =
+            DateTimeFormatter.ofPattern("dd/MMM/yyyy:HH:mm:ss Z").withZone(ZoneId.systemDefault());
+    private StringBuilder logLineBuilder;
+
+    private String clientIp;
+    private Instant requestTime;
+    private String reqLine;
+    private int statusCode;
+    private long respSize;
+    private String userAgentRef;
+    private boolean lastChunk = false;
+
+    public CLFLogger() {
+        this.logLineBuilder = new StringBuilder();
+        respSize = 0;
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg) {
+        if (msg instanceof HttpRequest) {
+            HttpRequest req = (HttpRequest) msg;
+            clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+            requestTime = Instant.now();
+            reqLine = req.method().toString() + " " + req.getUri() + " " + req.getProtocolVersion().toString();
+            userAgentRef = headerValueOrDash("Referer", req) + " " + headerValueOrDash("User-Agent", req);
+            lastChunk = false;
+        }
+        ctx.fireChannelRead(msg);
+    }
+
+    private String headerValueOrDash(String headerKey, HttpRequest req) {
+        String value = req.headers().get(headerKey);
+        if (value == null) {
+            value = "-";
+        } else {
+            value = "\"" + value + "\"";
+        }
+        return value;
+
+    }
+
+    @Override
+    public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
+        if (msg instanceof DefaultHttpResponse) {
+            HttpResponse resp = (DefaultHttpResponse) msg;
+            statusCode = resp.status().code();
+            if (msg instanceof DefaultFullHttpResponse) {
+                lastChunk = true;
+                respSize = resp.headers().getInt(HttpHeaderNames.CONTENT_LENGTH, 0);
+            }
+        } else if (msg instanceof DefaultHttpContent) {
+            HttpContent content = (DefaultHttpContent) msg;
+
+            respSize += content.content().readableBytes();
+        } else if (msg instanceof LastHttpContent) {
+            lastChunk = true;
+        }
+
+        ctx.write(msg, promise);
+    }
+
+    @Override
+    public void flush(ChannelHandlerContext ctx) throws Exception {
+        if (lastChunk) {
+            printAndPrepare();
+            lastChunk = false;
+        }
+        ctx.flush();
+    }
+
+    private void printAndPrepare() {
+        if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
+            return;
+        }
+        logLineBuilder.append(clientIp);
+        //identd value - not relevant here
+        logLineBuilder.append(" - ");
+        //no http auth or any auth either for that matter
+        logLineBuilder.append(" - [");
+        logLineBuilder.append(DATE_TIME_FORMATTER.format(requestTime));
+        logLineBuilder.append("] \"");
+        logLineBuilder.append(reqLine);
+        logLineBuilder.append("\"");
+        logLineBuilder.append(" ").append(statusCode);
+        logLineBuilder.append(" ").append(respSize);
+        logLineBuilder.append(" ").append(userAgentRef);
+        accessLogger.log(ACCESS_LOG_LEVEL, logLineBuilder);
+        respSize = 0;
+        logLineBuilder.setLength(0);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
index d4f1b3d..adea133 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
@@ -29,6 +29,7 @@
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.http.DefaultHttpContent;
 import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.util.ReferenceCountUtil;
 import io.netty.util.internal.OutOfDirectMemoryError;
 
 public class ChunkedNettyOutputStream extends OutputStream {
@@ -105,7 +106,7 @@
                 response.beforeFlush();
                 DefaultHttpContent content = new DefaultHttpContent(buffer);
                 ctx.writeAndFlush(content, ctx.channel().voidPromise());
-                // The responisbility of releasing the buffer is now with the netty pipeline since it is forwarded
+                // The responsibility of releasing the buffer is now with the netty pipeline since it is forwarded
                 // within the http content. We must nullify buffer before we allocate the next one to avoid
                 // releasing the buffer twice in case the allocation call fails.
                 buffer = null;
@@ -128,13 +129,17 @@
                 wait();
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
-                LOGGER.log(Level.WARN, "Interupted while waiting for channel to be writable", e);
+                LOGGER.log(Level.WARN, "Interrupted while waiting for channel to be writable", e);
                 throw new IOException(e);
             }
         }
     }
 
-    public synchronized void resume() {
+    public synchronized void channelWritabilityChanged() {
         notifyAll();
     }
+
+    public void cancel() {
+        ReferenceCountUtil.release(buffer);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
index 5a43d25..a67b40e 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
@@ -79,9 +79,8 @@
         response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR);
         response.headers().set(HttpHeaderNames.TRANSFER_ENCODING, HttpHeaderValues.CHUNKED);
         keepAlive = HttpUtil.isKeepAlive(request);
-        if (keepAlive) {
-            response.headers().set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE);
-        }
+        response.headers().set(HttpHeaderNames.CONNECTION,
+                keepAlive ? HttpHeaderValues.KEEP_ALIVE : HttpHeaderValues.CLOSE);
     }
 
     @Override
@@ -119,17 +118,10 @@
                 }
                 future = ctx.channel().close();
             } else {
-                if (keepAlive && response.status() != HttpResponseStatus.UNAUTHORIZED) {
-                    response.headers().remove(HttpHeaderNames.CONNECTION);
-                }
-                // we didn't send anything to the user, we need to send an unchunked error response
+                // we didn't send anything to the user, we need to send an non-chunked error response
                 fullResponse(response.protocolVersion(), response.status(),
                         error == null ? ctx.alloc().buffer(0, 0) : error, response.headers());
             }
-            if (response.status() != HttpResponseStatus.UNAUTHORIZED) {
-                // since the request failed, we need to close the channel on complete
-                future.addListener(ChannelFutureListener.CLOSE);
-            }
         }
         done = true;
     }
@@ -185,11 +177,16 @@
 
     @Override
     public void notifyChannelWritable() {
-        outputStream.resume();
+        outputStream.channelWritabilityChanged();
     }
 
     @Override
     public void notifyChannelInactive() {
-        outputStream.resume();
+        outputStream.channelWritabilityChanged();
+    }
+
+    @Override
+    public void cancel() {
+        outputStream.cancel();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
index 90e33b6..1d28472 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FullResponse.java
@@ -111,4 +111,9 @@
         // Do nothing.
         // This response is sent as a single piece
     }
+
+    @Override
+    public void cancel() {
+        // Do nothing, as this response doesn't allocate buffers in constructor
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java
new file mode 100644
index 0000000..6e93523
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java
@@ -0,0 +1,80 @@
+/*
+ * 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.hyracks.http.server;
+
+import java.util.List;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpMessage;
+import io.netty.handler.codec.http.HttpObject;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+
+@SuppressWarnings("squid:MaximumInheritanceDepth")
+public class HttpRequestAggregator extends HttpObjectAggregator {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+    private static final FullHttpResponse TOO_LARGE_CLOSE = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1,
+            HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE, Unpooled.EMPTY_BUFFER);
+    private boolean failed = false;
+
+    public HttpRequestAggregator(int maxContentLength) {
+        super(maxContentLength);
+    }
+
+    public HttpRequestAggregator(int maxContentLength, boolean closeOnExpectationFailed) {
+        super(maxContentLength, closeOnExpectationFailed);
+    }
+
+    @Override
+    protected void decode(final ChannelHandlerContext ctx, HttpObject msg, List<Object> out) throws Exception {
+        if (!failed) {
+            super.decode(ctx, msg, out);
+        }
+    }
+
+    @Override
+    protected void handleOversizedMessage(final ChannelHandlerContext ctx, HttpMessage oversized) throws Exception {
+        failed = true;
+        LOGGER.warn("A large request encountered. Closing the channel");
+        if (oversized instanceof HttpRequest) {
+            // send back a 413 and close the connection
+            final ChannelFuture future = ctx.writeAndFlush(TOO_LARGE_CLOSE.retainedDuplicate());
+            future.addListener((ChannelFutureListener) channelFuture -> {
+                if (!channelFuture.isSuccess()) {
+                    LOGGER.debug("Failed to send a 413 Request Entity Too Large.", channelFuture.cause());
+                }
+                ctx.close();
+            });
+        } else {
+            throw new IllegalStateException("Unknown large message of class " + oversized.getClass());
+        }
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestHandler.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestHandler.java
index 65a082c..1c0801c 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestHandler.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestHandler.java
@@ -40,6 +40,8 @@
     private final IServlet servlet;
     private final IServletRequest request;
     private final IServletResponse response;
+    private boolean started = false;
+    private boolean cancelled = false;
 
     public HttpRequestHandler(ChannelHandlerContext ctx, IServlet servlet, IServletRequest request, int chunkSize) {
         this.ctx = ctx;
@@ -52,6 +54,13 @@
 
     @Override
     public Void call() throws Exception {
+        synchronized (this) {
+            if (cancelled) {
+                LOGGER.warn("Request cancelled before it is started");
+                return null;
+            }
+            started = true;
+        }
         try {
             ChannelFuture lastContentFuture = handle();
             if (!HttpUtil.isKeepAlive(request.getHttpRequest())) {
@@ -83,7 +92,18 @@
     }
 
     public void notifyChannelInactive() {
-        response.notifyChannelInactive();
+        synchronized (this) {
+            if (!started) {
+                cancelled = true;
+            }
+        }
+        if (cancelled) {
+            // release request and response
+            response.cancel();
+            request.getHttpRequest().release();
+        } else {
+            response.notifyChannelInactive();
+        }
     }
 
     public void reject() throws IOException {
@@ -98,4 +118,8 @@
     public IServlet getServlet() {
         return servlet;
     }
+
+    public IServletRequest getRequest() {
+        return request;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
index 41e0088..1a5c2fa 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
@@ -28,6 +28,9 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.netty.util.internal.logging.InternalLoggerFactory;
+import io.netty.util.internal.logging.Log4J2LoggerFactory;
+import io.netty.util.internal.logging.Log4JLoggerFactory;
 import org.apache.hyracks.http.api.IChannelClosedHandler;
 import org.apache.hyracks.http.api.IServlet;
 import org.apache.hyracks.util.MXHelper;
@@ -39,14 +42,18 @@
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.EventLoopGroup;
 import io.netty.channel.FixedRecvByteBufAllocator;
 import io.netty.channel.WriteBufferWaterMark;
+import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
+import io.netty.util.internal.logging.InternalLoggerFactory;
+import io.netty.util.internal.logging.Log4J2LoggerFactory;
 
 public class HttpServer {
     // Constants
@@ -55,8 +62,6 @@
     protected static final WriteBufferWaterMark WRITE_BUFFER_WATER_MARK =
             new WriteBufferWaterMark(LOW_WRITE_BUFFER_WATER_MARK, HIGH_WRITE_BUFFER_WATER_MARK);
     protected static final int RECEIVE_BUFFER_SIZE = 4096;
-    protected static final int DEFAULT_NUM_EXECUTOR_THREADS = 16;
-    protected static final int DEFAULT_REQUEST_QUEUE_SIZE = 256;
     private static final Logger LOGGER = LogManager.getLogger();
     private static final int FAILED = -1;
     private static final int STOPPED = 0;
@@ -79,39 +84,37 @@
     private volatile Thread recoveryThread;
     private volatile Channel channel;
     private Throwable cause;
+    private HttpServerConfig config;
 
-    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port) {
-        this(bossGroup, workerGroup, port, DEFAULT_NUM_EXECUTOR_THREADS, DEFAULT_REQUEST_QUEUE_SIZE, null);
+    static {
+        InternalLoggerFactory.setDefaultFactory(Log4J2LoggerFactory.INSTANCE);
     }
 
-    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port,
+    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, HttpServerConfig config) {
+        this(bossGroup, workerGroup, port, config, null);
+    }
+
+    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, HttpServerConfig config,
             IChannelClosedHandler closeHandler) {
-        this(bossGroup, workerGroup, port, DEFAULT_NUM_EXECUTOR_THREADS, DEFAULT_REQUEST_QUEUE_SIZE, closeHandler);
-    }
-
-    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, int numExecutorThreads,
-            int requestQueueSize) {
-        this(bossGroup, workerGroup, port, numExecutorThreads, requestQueueSize, null);
-    }
-
-    public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, int numExecutorThreads,
-            int requestQueueSize, IChannelClosedHandler closeHandler) {
         this.bossGroup = bossGroup;
         this.workerGroup = workerGroup;
         this.port = port;
         this.closedHandler = closeHandler;
+        this.config = config;
         ctx = new ConcurrentHashMap<>();
         servlets = new ArrayList<>();
-        workQueue = new LinkedBlockingQueue<>(requestQueueSize);
+        workQueue = new LinkedBlockingQueue<>(config.getRequestQueueSize());
+        int numExecutorThreads = config.getThreadCount();
         executor = new ThreadPoolExecutor(numExecutorThreads, numExecutorThreads, 0L, TimeUnit.MILLISECONDS, workQueue,
                 runnable -> new Thread(runnable, "HttpExecutor(port:" + port + ")-" + threadId.getAndIncrement()));
         long directMemoryBudget = numExecutorThreads * (long) HIGH_WRITE_BUFFER_WATER_MARK
-                + numExecutorThreads * HttpServerInitializer.RESPONSE_CHUNK_SIZE;
-        LOGGER.log(Level.INFO, "The output direct memory budget for this server is " + directMemoryBudget + " bytes");
+                + numExecutorThreads * config.getMaxResponseChunkSize();
+        LOGGER.log(Level.DEBUG,
+                "The output direct memory budget for this server " + "is " + directMemoryBudget + " bytes");
         long inputBudgetEstimate =
-                (long) HttpServerInitializer.MAX_REQUEST_INITIAL_LINE_LENGTH * (requestQueueSize + numExecutorThreads);
+                (long) config.getMaxRequestInitialLineLength() * (config.getRequestQueueSize() + numExecutorThreads);
         inputBudgetEstimate = inputBudgetEstimate * 2;
-        LOGGER.log(Level.INFO,
+        LOGGER.log(Level.DEBUG,
                 "The \"estimated\" input direct memory budget for this server is " + inputBudgetEstimate + " bytes");
         // Having multiple arenas, memory fragments, and local thread cached buffers
         // can cause the input memory usage to exceed estimate and custom buffer allocator must be used to avoid this
@@ -242,7 +245,7 @@
                 .childOption(ChannelOption.AUTO_READ, Boolean.FALSE)
                 .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
                 .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, WRITE_BUFFER_WATER_MARK)
-                .handler(new LoggingHandler(LogLevel.DEBUG)).childHandler(new HttpServerInitializer(this));
+                .handler(new LoggingHandler(LogLevel.DEBUG)).childHandler(getChannelInitializer());
         Channel newChannel = b.bind(port).sync().channel();
         newChannel.closeFuture().addListener(f -> {
             // This listener is invoked from within a netty IO thread. Hence, we can never block it
@@ -381,6 +384,10 @@
         return new HttpServerHandler<>(this, chunkSize);
     }
 
+    protected ChannelInitializer<SocketChannel> getChannelInitializer() {
+        return new HttpServerInitializer(this);
+    }
+
     public ThreadPoolExecutor getExecutor(HttpRequestHandler handler) {
         return executor;
     }
@@ -402,4 +409,8 @@
         return "{\"class\":\"" + getClass().getSimpleName() + "\",\"port\":" + port + ",\"state\":\"" + getState()
                 + "\"}";
     }
+
+    public HttpServerConfig getConfig() {
+        return config;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java
new file mode 100644
index 0000000..443cc3c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java
@@ -0,0 +1,78 @@
+/*
+ * 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.hyracks.http.server;
+
+import org.apache.hyracks.util.annotations.ThreadSafe;
+
+@ThreadSafe
+public class HttpServerConfig {
+
+    private int maxRequestSize;
+    private int threadCount;
+    private int requestQueueSize;
+    private int maxRequestChunkSize;
+    private int maxResponseChunkSize;
+    private int maxRequestHeaderSize;
+    private int maxRequestInitialLineLength;
+
+    private HttpServerConfig() {
+    }
+
+    public static HttpServerConfig of(int threadCount, int requestQueueSize, int maxRequestSize,
+            int maxRequestInitialLineLength, int maxRequestHeaderSize, int maxRequestChunkSize,
+            int maxResponseChunkSize) {
+        final HttpServerConfig config = new HttpServerConfig();
+        config.threadCount = threadCount;
+        config.requestQueueSize = requestQueueSize;
+        config.maxRequestSize = maxRequestSize;
+        config.maxRequestInitialLineLength = maxRequestInitialLineLength;
+        config.maxRequestHeaderSize = maxRequestHeaderSize;
+        config.maxRequestChunkSize = maxRequestChunkSize;
+        config.maxResponseChunkSize = maxResponseChunkSize;
+        return config;
+    }
+
+    public int getMaxRequestSize() {
+        return maxRequestSize;
+    }
+
+    public int getThreadCount() {
+        return threadCount;
+    }
+
+    public int getRequestQueueSize() {
+        return requestQueueSize;
+    }
+
+    public int getMaxRequestChunkSize() {
+        return maxRequestChunkSize;
+    }
+
+    public int getMaxResponseChunkSize() {
+        return maxResponseChunkSize;
+    }
+
+    public int getMaxRequestHeaderSize() {
+        return maxRequestHeaderSize;
+    }
+
+    public int getMaxRequestInitialLineLength() {
+        return maxRequestInitialLineLength;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java
new file mode 100644
index 0000000..359d760
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java
@@ -0,0 +1,91 @@
+/*
+ * 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.hyracks.http.server;
+
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+@NotThreadSafe
+public class HttpServerConfigBuilder {
+
+    private static final int MAX_REQUEST_CHUNK_SIZE = 262144;
+    private static final int MAX_REQUEST_HEADER_SIZE = 262144;
+    private static final int MAX_REQUEST_INITIAL_LINE_LENGTH = 131072;
+    private static final int RESPONSE_CHUNK_SIZE = 4096;
+    private static final int DEFAULT_THREAD_COUNT = 16;
+    private static final int DEFAULT_MAX_QUEUE_SIZE = 256;
+
+    private int maxRequestSize = Integer.MAX_VALUE;
+    private int threadCount = DEFAULT_THREAD_COUNT;
+    private int requestQueueSize = DEFAULT_MAX_QUEUE_SIZE;
+    private int maxRequestChunkSize = MAX_REQUEST_CHUNK_SIZE;
+    private int maxResponseChunkSize = RESPONSE_CHUNK_SIZE;
+    private int maxRequestHeaderSize = MAX_REQUEST_HEADER_SIZE;
+    private int maxRequestInitialLineLength = MAX_REQUEST_INITIAL_LINE_LENGTH;
+
+    private HttpServerConfigBuilder() {
+    }
+
+    public static HttpServerConfig createDefault() {
+        return new HttpServerConfigBuilder().build();
+    }
+
+    public static HttpServerConfigBuilder custom() {
+        return new HttpServerConfigBuilder();
+    }
+
+    public HttpServerConfigBuilder setMaxRequestSize(int maxRequestSize) {
+        this.maxRequestSize = maxRequestSize;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setThreadCount(int threadCount) {
+        this.threadCount = threadCount;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setRequestQueueSize(int requestQueueSize) {
+        this.requestQueueSize = requestQueueSize;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setMaxRequestChunkSize(int maxRequestChunkSize) {
+        this.maxRequestChunkSize = maxRequestChunkSize;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setMaxResponseChunkSize(int maxResponseChunkSize) {
+        this.maxResponseChunkSize = maxResponseChunkSize;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setMaxRequestHeaderSize(int maxRequestHeaderSize) {
+        this.maxRequestHeaderSize = maxRequestHeaderSize;
+        return this;
+    }
+
+    public HttpServerConfigBuilder setMaxRequestInitialLineLength(int maxRequestInitialLineLength) {
+        this.maxRequestInitialLineLength = maxRequestInitialLineLength;
+        return this;
+    }
+
+    public HttpServerConfig build() {
+        return HttpServerConfig.of(threadCount, requestQueueSize, maxRequestSize, maxRequestInitialLineLength,
+                maxRequestHeaderSize, maxRequestChunkSize, maxResponseChunkSize);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
index 7b3d18a..baa664a 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
@@ -91,7 +91,7 @@
                 submit(ctx, servlet, request);
             }
         } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Failure Submitting HTTP Request", e);
+            LOGGER.log(Level.WARN, "Failure Submitting HTTP Request", e);
             respond(ctx, request.protocolVersion(), new HttpResponseStatus(500, e.getMessage()));
         }
     }
@@ -125,15 +125,15 @@
     }
 
     protected void handleServletNotFound(ChannelHandlerContext ctx, FullHttpRequest request) {
-        if (LOGGER.isWarnEnabled()) {
-            LOGGER.warn("No servlet for " + request.uri());
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("No servlet for " + request.uri());
         }
         respond(ctx, request.protocolVersion(), HttpResponseStatus.NOT_FOUND);
     }
 
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-        LOGGER.log(Level.ERROR, "Failure handling HTTP Request", cause);
+        LOGGER.log(Level.WARN, "Failure handling HTTP Request", cause);
         ctx.close();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
index bc173fd..eafee2d 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
@@ -24,27 +24,36 @@
 import io.netty.handler.codec.http.HttpObjectAggregator;
 import io.netty.handler.codec.http.HttpRequestDecoder;
 import io.netty.handler.codec.http.HttpResponseEncoder;
+import io.netty.handler.logging.LogLevel;
+import org.apache.logging.log4j.Logger;
 
 public class HttpServerInitializer extends ChannelInitializer<SocketChannel> {
 
-    public static final int MAX_REQUEST_CHUNK_SIZE = 262144;
-    public static final int MAX_REQUEST_HEADER_SIZE = 262144;
-    public static final int MAX_REQUEST_INITIAL_LINE_LENGTH = 131072;
-    public static final int RESPONSE_CHUNK_SIZE = 4096;
     private final HttpServer server;
+    private final int maxRequestSize;
+    private final int maxRequestInitialLineLength;
+    private final int maxRequestHeaderSize;
+    private final int maxRequestChunkSize;
+    private final int maxResponseChunkSize;
 
     public HttpServerInitializer(HttpServer server) {
         this.server = server;
+        final HttpServerConfig config = server.getConfig();
+        maxRequestSize = config.getMaxRequestSize();
+        maxRequestInitialLineLength = config.getMaxRequestInitialLineLength();
+        maxRequestHeaderSize = config.getMaxRequestHeaderSize();
+        maxRequestChunkSize = config.getMaxRequestChunkSize();
+        maxResponseChunkSize = config.getMaxResponseChunkSize();
     }
 
     @Override
     public void initChannel(SocketChannel ch) {
         ChannelPipeline p = ch.pipeline();
         p.addLast(new HttpRequestCapacityController(server));
-        p.addLast(new HttpRequestDecoder(MAX_REQUEST_INITIAL_LINE_LENGTH, MAX_REQUEST_HEADER_SIZE,
-                MAX_REQUEST_CHUNK_SIZE));
+        p.addLast(new HttpRequestDecoder(maxRequestInitialLineLength, maxRequestHeaderSize, maxRequestChunkSize));
         p.addLast(new HttpResponseEncoder());
-        p.addLast(new HttpObjectAggregator(Integer.MAX_VALUE));
-        p.addLast(server.createHttpHandler(RESPONSE_CHUNK_SIZE));
+        p.addLast(new CLFLogger());
+        p.addLast(new HttpRequestAggregator(maxRequestSize));
+        p.addLast(server.createHttpHandler(maxResponseChunkSize));
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index 515c57a..8d6dfbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -50,6 +50,7 @@
     public static class ContentType {
         public static final String APPLICATION_ADM = "application/x-adm";
         public static final String APPLICATION_JSON = "application/json";
+        public static final String JSON = "json";
         public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded";
         public static final String CSV = "text/csv";
         public static final String IMG_PNG = "image/png";
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
index 17f6f9a..78226ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
@@ -32,8 +32,8 @@
 import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.client.methods.RequestBuilder;
 import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.http.impl.client.HttpClients;
-import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
 
 import io.netty.handler.codec.http.HttpResponseStatus;
 
@@ -41,8 +41,8 @@
 
     protected final HttpUriRequest request;
 
-    protected HttpRequestTask() throws URISyntaxException {
-        request = post(null);
+    protected HttpRequestTask(int entitySize) throws URISyntaxException {
+        request = post(null, entitySize);
     }
 
     @Override
@@ -53,6 +53,8 @@
                 HttpServerTest.SUCCESS_COUNT.incrementAndGet();
             } else if (response.getStatusLine().getStatusCode() == HttpResponseStatus.SERVICE_UNAVAILABLE.code()) {
                 HttpServerTest.UNAVAILABLE_COUNT.incrementAndGet();
+            } else if (response.getStatusLine().getStatusCode() == HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE.code()) {
+                throw new Exception(HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE.reasonPhrase());
             } else {
                 HttpServerTest.OTHER_COUNT.incrementAndGet();
             }
@@ -73,7 +75,7 @@
     }
 
     protected HttpResponse executeHttpRequest(HttpUriRequest method) throws Exception {
-        HttpClient client = HttpClients.custom().setRetryHandler(StandardHttpRequestRetryHandler.INSTANCE).build();
+        HttpClient client = HttpClients.custom().setRetryHandler(new DefaultHttpRequestRetryHandler(0, false)).build();
         try {
             return client.execute(method);
         } catch (Exception e) {
@@ -90,7 +92,7 @@
         return builder.build();
     }
 
-    protected HttpUriRequest post(String query) throws URISyntaxException {
+    protected HttpUriRequest post(String query, int entitySize) throws URISyntaxException {
         URI uri = new URI(HttpServerTest.PROTOCOL, null, HttpServerTest.HOST, HttpServerTest.PORT, HttpServerTest.PATH,
                 query, null);
         RequestBuilder builder = RequestBuilder.post(uri);
@@ -102,7 +104,13 @@
         String statement = str.toString();
         builder.setHeader("Content-type", "application/x-www-form-urlencoded");
         builder.addParameter("statement", statement);
-        builder.setEntity(new StringEntity(statement, StandardCharsets.UTF_8));
+        if (entitySize > 0) {
+            str.setLength(0);
+            for (int i = 0; i < entitySize; i++) {
+                str.append("x");
+            }
+            builder.setEntity(new StringEntity(str.toString(), StandardCharsets.UTF_8));
+        }
         builder.setCharset(StandardCharsets.UTF_8);
         return builder.build();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
index b39a141..b5683ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
@@ -34,10 +34,13 @@
 
 import org.apache.hyracks.http.HttpTestUtil;
 import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
 import org.apache.hyracks.http.server.InterruptOnCloseHandler;
 import org.apache.hyracks.http.server.WebManager;
 import org.apache.hyracks.http.servlet.ChattyServlet;
 import org.apache.hyracks.http.servlet.SleepyServlet;
+import org.apache.hyracks.util.StorageUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -79,8 +82,9 @@
         int numExecutors = 16;
         int serverQueueSize = 16;
         int numRequests = 128;
-        HttpServer server =
-                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+                .setRequestQueueSize(serverQueueSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
         SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -135,8 +139,9 @@
         int numExecutors = 2;
         int serverQueueSize = 2;
         int numPatches = 60;
-        HttpServer server =
-                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+                .setRequestQueueSize(serverQueueSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
         SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -182,8 +187,9 @@
         int serverQueueSize = 24;
         HttpTestUtil.printMemUsage();
         WebManager webMgr = new WebManager();
-        HttpServer server =
-                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+                .setRequestQueueSize(serverQueueSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
         ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -208,8 +214,9 @@
         int numExecutors = 16;
         int serverQueueSize = 16;
         WebManager webMgr = new WebManager();
-        HttpServer server =
-                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+                .setRequestQueueSize(serverQueueSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
         SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -245,8 +252,9 @@
         int serverQueueSize = 16;
         int numRequests = 1;
         WebManager webMgr = new WebManager();
-        HttpServer server =
-                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+                .setRequestQueueSize(serverQueueSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
         ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -301,8 +309,10 @@
         WebManager webMgr = new WebManager();
         int numExecutors = 1;
         int queueSize = 1;
-        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, queueSize,
-                InterruptOnCloseHandler.INSTANCE);
+        final HttpServerConfig config =
+                HttpServerConfigBuilder.custom().setThreadCount(numExecutors).setRequestQueueSize(queueSize).build();
+        HttpServer server =
+                new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config, InterruptOnCloseHandler.INSTANCE);
         SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
         server.addServlet(servlet);
         webMgr.add(server);
@@ -335,6 +345,31 @@
         }
     }
 
+    @Test
+    public void testLargeRequest() throws Exception {
+        WebManager webMgr = new WebManager();
+        // Server with max allowed request size = 512K
+        final int maxRequestSize = StorageUtil.getIntSizeInBytes(512, StorageUtil.StorageUnit.KILOBYTE);
+        final HttpServerConfig config = HttpServerConfigBuilder.custom().setMaxRequestSize(maxRequestSize).build();
+        HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
+        ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
+        server.addServlet(servlet);
+        webMgr.add(server);
+        webMgr.start();
+        Exception failure = null;
+        try {
+            request(1, maxRequestSize + 1);
+            for (Future<Void> thread : FUTURES) {
+                thread.get();
+            }
+        } catch (Exception e) {
+            failure = e;
+        } finally {
+            webMgr.stop();
+        }
+        Assert.assertNotNull(failure);
+    }
+
     public static void setPrivateField(Object obj, String filedName, Object value) throws Exception {
         Field f = obj.getClass().getDeclaredField(filedName);
         f.setAccessible(true);
@@ -342,8 +377,12 @@
     }
 
     private void request(int count) throws URISyntaxException {
+        request(count, 0);
+    }
+
+    private void request(int count, int entitySize) throws URISyntaxException {
         for (int i = 0; i < count; i++) {
-            HttpRequestTask requestTask = new HttpRequestTask();
+            HttpRequestTask requestTask = new HttpRequestTask(entitySize);
             Future<Void> next = executor.submit(requestTask);
             FUTURES.add(next);
             TASKS.add(requestTask);
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
index 7f59db1..1f9ec37 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
@@ -41,10 +41,6 @@
   </properties>
   <dependencies>
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
deleted file mode 100644
index 4328fc3..0000000
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.hyracks.ipc.api;
-
-public interface IResponseCallback {
-    public void callback(IIPCHandle handle, Object response, Exception exception);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
index 912c267..39a29af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
@@ -23,6 +23,5 @@
     CONNECT_SENT,
     CONNECT_RECEIVED,
     CONNECTED,
-    CONNECT_FAILED,
     CLOSED,
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index 3e6c64b..b4828e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -23,7 +23,6 @@
 import java.net.ServerSocket;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channel;
-import java.nio.channels.ClosedChannelException;
 import java.nio.channels.SelectableChannel;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.Selector;
@@ -39,7 +38,7 @@
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.NetworkUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -100,7 +99,7 @@
 
     void stop() {
         stopped = true;
-        IOUtils.closeQuietly(serverSocketChannel);
+        NetworkUtil.closeQuietly(serverSocketChannel);
         networkThread.selector.wakeup();
     }
 
@@ -121,8 +120,10 @@
                 return handle;
             }
             if (maxRetries < 0 || retries++ < maxRetries) {
-                LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
-                        : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
+                            : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+                }
                 Thread.sleep(delay);
                 delay = Math.min(MAX_RETRY_DELAY_MILLIS, (int) (delay * 1.5));
             } else {
@@ -137,31 +138,13 @@
     }
 
     synchronized void write(Message msg) {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Enqueued message: " + msg);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Enqueued message: " + msg);
         }
         sendList.add(msg);
         networkThread.selector.wakeup();
     }
 
-    private synchronized void collectOutstandingWork() {
-        if (!pendingConnections.isEmpty()) {
-            moveAll(pendingConnections, workingPendingConnections);
-        }
-        if (!sendList.isEmpty()) {
-            moveAll(sendList, workingSendList);
-        }
-    }
-
-    private Message createInitialReqMessage(IPCHandle handle) {
-        Message msg = new Message(handle);
-        msg.setMessageId(system.createMessageId());
-        msg.setRequestMessageId(-1);
-        msg.setFlag(Message.INITIAL_REQ);
-        msg.setPayload(address);
-        return msg;
-    }
-
     private Message createInitialAckMessage(IPCHandle handle, Message req) {
         Message msg = new Message(handle);
         msg.setMessageId(system.createMessageId());
@@ -177,16 +160,18 @@
 
     private class NetworkThread extends Thread {
         private final Selector selector;
-
         private final Set<SocketChannel> openChannels = new HashSet<>();
+        private final BitSet unsentMessagesBitmap = new BitSet();
+        private final List<Message> tempUnsentMessages = new ArrayList<>();
 
-        public NetworkThread() {
+        NetworkThread() {
             super("IPC Network Listener Thread [" + address + "]");
             setDaemon(true);
             try {
                 selector = Selector.open();
+                serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
             } catch (IOException e) {
-                throw new RuntimeException(e);
+                throw new IllegalStateException(e);
             }
         }
 
@@ -200,139 +185,166 @@
         }
 
         private void doRun() {
-            try {
-                serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
-            } catch (ClosedChannelException e) {
-                throw new RuntimeException(e);
-            }
-            BitSet unsentMessagesBitmap = new BitSet();
-            List<Message> tempUnsentMessages = new ArrayList<>();
-            int failingLoops = 0;
             while (!stopped) {
                 try {
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("Starting Select");
-                    }
                     int n = selector.select();
                     collectOutstandingWork();
                     if (!workingPendingConnections.isEmpty()) {
-                        for (IPCHandle handle : workingPendingConnections) {
-                            SocketChannel channel = SocketChannel.open();
-                            register(channel);
-                            SelectionKey cKey;
-                            if (channel.connect(handle.getRemoteAddress())) {
-                                cKey = channel.register(selector, SelectionKey.OP_READ);
-                                handle.setState(HandleState.CONNECT_SENT);
-                                IPCConnectionManager.this.write(createInitialReqMessage(handle));
-                            } else {
-                                cKey = channel.register(selector, SelectionKey.OP_CONNECT);
-                            }
-                            handle.setKey(cKey);
-                            cKey.attach(handle);
-                        }
-                        workingPendingConnections.clear();
+                        establishPendingConnections();
                     }
                     if (!workingSendList.isEmpty()) {
-                        unsentMessagesBitmap.clear();
-                        int len = workingSendList.size();
-                        for (int i = 0; i < len; ++i) {
-                            Message msg = workingSendList.get(i);
-                            LOGGER.debug(() -> "Processing send of message: " + msg);
-                            IPCHandle handle = msg.getIPCHandle();
-                            if (handle.getState() != HandleState.CLOSED) {
-                                if (!handle.full()) {
-                                    while (true) {
-                                        ByteBuffer buffer = handle.getOutBuffer();
-                                        buffer.compact();
-                                        boolean success = msg.write(buffer);
-                                        buffer.flip();
-                                        if (success) {
-                                            system.getPerformanceCounters().addMessageSentCount(1);
-                                            SelectionKey key = handle.getKey();
-                                            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
-                                        } else {
-                                            if (!buffer.hasRemaining()) {
-                                                handle.resizeOutBuffer();
-                                                continue;
-                                            }
-                                            handle.markFull();
-                                            unsentMessagesBitmap.set(i);
-                                        }
-                                        break;
-                                    }
-                                } else {
-                                    unsentMessagesBitmap.set(i);
-                                }
-                            }
-                        }
-                        copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+                        sendPendingMessages();
                     }
                     if (n > 0) {
-                        for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
-                            SelectionKey key = i.next();
-                            i.remove();
-                            final SelectableChannel sc = key.channel();
-                            if (key.isReadable()) {
-                                read(key);
-                            } else if (key.isWritable()) {
-                                write(key);
-                            } else if (key.isAcceptable()) {
-                                assert sc == serverSocketChannel;
-                                SocketChannel channel = serverSocketChannel.accept();
-                                register(channel);
-                                IPCHandle handle = new IPCHandle(system, null);
-                                SelectionKey cKey = channel.register(selector, SelectionKey.OP_READ);
-                                handle.setKey(cKey);
-                                cKey.attach(handle);
-                                handle.setState(HandleState.CONNECT_RECEIVED);
-                            } else if (key.isConnectable()) {
-                                SocketChannel channel = (SocketChannel) sc;
-                                IPCHandle handle = (IPCHandle) key.attachment();
-                                if (!finishConnect(channel)) {
-                                    handle.setState(HandleState.CONNECT_FAILED);
-                                    continue;
-                                }
-
-                                handle.setState(HandleState.CONNECT_SENT);
-                                registerHandle(handle);
-                                key.interestOps(SelectionKey.OP_READ);
-                                IPCConnectionManager.this.write(createInitialReqMessage(handle));
-                            }
-                        }
+                        processSelectedKeys();
                     }
-                    // reset failingLoops on a good loop
-                    failingLoops = 0;
                 } catch (Exception e) {
-                    int sleepSecs = (int) Math.pow(2, Math.min(11, failingLoops++));
-                    LOGGER.log(Level.ERROR, "Exception processing message; sleeping " + sleepSecs + " seconds", e);
-                    try {
-                        Thread.sleep(TimeUnit.SECONDS.toMillis(sleepSecs));
-                    } catch (InterruptedException e1) {
-                        Thread.currentThread().interrupt();
-                    }
+                    LOGGER.log(Level.ERROR, "Exception processing message", e);
                 }
             }
         }
 
+        private void processSelectedKeys() {
+            for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
+                SelectionKey key = i.next();
+                i.remove();
+                final SelectableChannel sc = key.channel();
+                if (key.isReadable()) {
+                    read(key);
+                } else if (key.isWritable()) {
+                    write(key);
+                } else if (key.isAcceptable()) {
+                    assert sc == serverSocketChannel;
+                    accept();
+                } else if (key.isConnectable()) {
+                    finishConnect(key);
+                }
+            }
+        }
+
+        private void finishConnect(SelectionKey connectableKey) {
+            SocketChannel channel = (SocketChannel) connectableKey.channel();
+            IPCHandle handle = (IPCHandle) connectableKey.attachment();
+            boolean connected = false;
+            try {
+                connected = channel.finishConnect();
+                if (connected) {
+                    connectableKey.interestOps(SelectionKey.OP_READ);
+                    connectionEstablished(handle);
+                }
+            } catch (Exception e) {
+                LOGGER.warn("Exception finishing connect", e);
+            } finally {
+                if (!connected) {
+                    LOGGER.warn("Failed to finish connect to {}", handle.getRemoteAddress());
+                    close(connectableKey, channel);
+                }
+            }
+        }
+
+        private void accept() {
+            SocketChannel channel = null;
+            SelectionKey channelKey = null;
+            try {
+                channel = serverSocketChannel.accept();
+                register(channel);
+                channelKey = channel.register(selector, SelectionKey.OP_READ);
+                IPCHandle handle = new IPCHandle(system, null);
+                handle.setKey(channelKey);
+                channelKey.attach(handle);
+                handle.setState(HandleState.CONNECT_RECEIVED);
+            } catch (Exception e) {
+                LOGGER.error("Failed to accept channel ", e);
+                close(channelKey, channel);
+            }
+        }
+
+        private void establishPendingConnections() {
+            for (IPCHandle handle : workingPendingConnections) {
+                SocketChannel channel = null;
+                SelectionKey channelKey = null;
+                try {
+                    channel = SocketChannel.open();
+                    register(channel);
+                    if (channel.connect(handle.getRemoteAddress())) {
+                        channelKey = channel.register(selector, SelectionKey.OP_READ);
+                        connectionEstablished(handle);
+                    } else {
+                        channelKey = channel.register(selector, SelectionKey.OP_CONNECT);
+                    }
+                    handle.setKey(channelKey);
+                    channelKey.attach(handle);
+                } catch (Exception e) {
+                    LOGGER.error("Failed to accept channel ", e);
+                    close(channelKey, channel);
+                    handle.setState(HandleState.CLOSED);
+                }
+            }
+            workingPendingConnections.clear();
+        }
+
+        private void connectionEstablished(IPCHandle handle) {
+            handle.setState(HandleState.CONNECT_SENT);
+            registerHandle(handle);
+            IPCConnectionManager.this.write(createInitialReqMessage(handle));
+        }
+
+        private void sendPendingMessages() {
+            unsentMessagesBitmap.clear();
+            int len = workingSendList.size();
+            for (int i = 0; i < len; ++i) {
+                Message msg = workingSendList.get(i);
+                final boolean sent = sendMessage(msg);
+                if (!sent) {
+                    unsentMessagesBitmap.set(i);
+                }
+            }
+            copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+        }
+
+        private boolean sendMessage(Message msg) {
+            LOGGER.trace("Processing send of message: {}", msg);
+            IPCHandle handle = msg.getIPCHandle();
+            if (handle.getState() == HandleState.CLOSED) {
+                // message will never be sent
+                return true;
+            }
+            if (handle.full()) {
+                return false;
+            }
+            try {
+                while (true) {
+                    ByteBuffer buffer = handle.getOutBuffer();
+                    buffer.compact();
+                    boolean success = msg.write(buffer);
+                    buffer.flip();
+                    if (success) {
+                        system.getPerformanceCounters().addMessageSentCount(1);
+                        SelectionKey key = handle.getKey();
+                        key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                        return true;
+                    } else {
+                        if (!buffer.hasRemaining()) {
+                            handle.resizeOutBuffer();
+                            continue;
+                        }
+                        handle.markFull();
+                        return false;
+                    }
+                }
+            } catch (Exception e) {
+                LOGGER.fatal("Unrecoverable networking failure; Halting...", e);
+                ExitUtil.halt(ExitUtil.EC_NETWORK_FAILURE);
+            }
+            return false;
+        }
+
         private void cleanup() {
             for (Channel channel : openChannels) {
-                IOUtils.closeQuietly(channel);
+                NetworkUtil.closeQuietly(channel);
             }
             openChannels.clear();
-            IOUtils.closeQuietly(selector);
-        }
-
-        private boolean finishConnect(SocketChannel channel) {
-            boolean connectFinished = false;
-            try {
-                connectFinished = channel.finishConnect();
-                if (!connectFinished) {
-                    LOGGER.log(Level.WARN, "Channel connect did not finish");
-                }
-            } catch (IOException e) {
-                LOGGER.log(Level.WARN, "Exception finishing channel connect", e);
-            }
-            return connectFinished;
+            NetworkUtil.closeQuietly(selector);
         }
 
         private void copyUnsentMessages(BitSet unsentMessagesBitmap, List<Message> tempUnsentMessages) {
@@ -365,7 +377,7 @@
                 if (!readBuffer.hasRemaining()) {
                     handle.resizeInBuffer();
                 }
-            } catch (IOException e) {
+            } catch (Exception e) {
                 LOGGER.error("TCP read error from {}", handle.getRemoteAddress(), e);
                 close(readableKey, channel);
             }
@@ -389,26 +401,49 @@
                     handle.clearFull();
                     selector.wakeup();
                 }
-            } catch (IOException e) {
+            } catch (Exception e) {
                 LOGGER.error("TCP write error to {}", handle.getRemoteAddress(), e);
                 close(writableKey, channel);
             }
         }
 
         private void close(SelectionKey key, SocketChannel sc) {
-            key.cancel();
-            NetworkUtil.closeQuietly(sc);
-            openChannels.remove(sc);
-            final IPCHandle handle = (IPCHandle) key.attachment();
-            handle.close();
+            if (key != null) {
+                final Object attachment = key.attachment();
+                if (attachment != null) {
+                    ((IPCHandle) attachment).close();
+                }
+                key.cancel();
+            }
+            if (sc != null) {
+                NetworkUtil.closeQuietly(sc);
+                openChannels.remove(sc);
+            }
         }
-    }
 
-    private <T> void moveAll(List<T> source, List<T> target) {
-        int len = source.size();
-        for (int i = 0; i < len; ++i) {
-            target.add(source.get(i));
+        private void collectOutstandingWork() {
+            synchronized (IPCConnectionManager.this) {
+                if (!pendingConnections.isEmpty()) {
+                    moveAll(pendingConnections, workingPendingConnections);
+                }
+                if (!sendList.isEmpty()) {
+                    moveAll(sendList, workingSendList);
+                }
+            }
         }
-        source.clear();
+
+        private Message createInitialReqMessage(IPCHandle handle) {
+            Message msg = new Message(handle);
+            msg.setMessageId(system.createMessageId());
+            msg.setRequestMessageId(-1);
+            msg.setFlag(Message.INITIAL_REQ);
+            msg.setPayload(address);
+            return msg;
+        }
+
+        private <T> void moveAll(List<T> source, List<T> target) {
+            target.addAll(source);
+            source.clear();
+        }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
index d63bfbd..09c7c97 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
@@ -62,10 +62,6 @@
         return system;
     }
 
-    void setRemoteAddress(InetSocketAddress remoteAddress) {
-        this.remoteAddress = remoteAddress;
-    }
-
     @Override
     public long send(long requestId, Object req, Exception exception) throws IPCException {
         if (!isConnected()) {
@@ -127,7 +123,6 @@
                     wait();
                     break;
                 case CONNECTED:
-                case CONNECT_FAILED:
                 case CLOSED:
                     return state == HandleState.CONNECTED;
                 default:
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
index b36e645..b7dcf05 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
@@ -66,10 +66,6 @@
         cMgr.stop();
     }
 
-    public IIPCHandle getHandle(InetSocketAddress remoteAddress) throws IPCException {
-        return getHandle(remoteAddress, 0);
-    }
-
     public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries) throws IPCException {
         return getHandle(remoteAddress, maxRetries, 0);
     }
@@ -78,16 +74,6 @@
         return getReconnectingHandle(remoteAddress, 1);
     }
 
-    public IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
-            throws IPCException {
-        return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
-    }
-
-    public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
-            throws IPCException {
-        return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
-    }
-
     public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts,
             IIPCEventListener eventListener) throws IPCException {
         if (reconnectAttempts > 0) {
@@ -132,4 +118,14 @@
     public IPCPerformanceCounters getPerformanceCounters() {
         return perfCounters;
     }
+
+    private IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
+            throws IPCException {
+        return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+    }
+
+    private IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
+            throws IPCException {
+        return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
index cca3abe..70a0e18 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
@@ -43,7 +43,7 @@
         IPCSystem client = createClientIPCSystem(rpci);
         client.start();
 
-        IIPCHandle handle = client.getHandle(serverAddr);
+        IIPCHandle handle = client.getHandle(serverAddr, 0);
 
         for (int i = 0; i < 100; ++i) {
             Assert.assertEquals(rpci.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
index bc7e612..6f8d176 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
@@ -36,17 +36,17 @@
     <dependency>
       <groupId>org.apache.maven.plugin-tools</groupId>
       <artifactId>maven-plugin-annotations</artifactId>
-      <version>3.5</version>
+      <version>3.6.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-core</artifactId>
-      <version>3.3.9</version>
+      <version>3.6.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven.plugins</groupId>
       <artifactId>maven-remote-resources-plugin</artifactId>
-      <version>1.5</version>
+      <version>1.6.0</version>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
@@ -55,7 +55,7 @@
     <dependency>
       <groupId>org.freemarker</groupId>
       <artifactId>freemarker</artifactId>
-      <version>2.3.23</version>
+      <version>2.3.28</version>
     </dependency>
     <dependency>
       <groupId>commons-io</groupId>
@@ -64,7 +64,6 @@
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-annotations</artifactId>
-      <version>2.8.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
@@ -74,7 +73,7 @@
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-model</artifactId>
-      <version>3.3.9</version>
+      <version>3.6.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
@@ -83,17 +82,17 @@
     <dependency>
       <groupId>org.codehaus.plexus</groupId>
       <artifactId>plexus-utils</artifactId>
-      <version>3.0.22</version>
+      <version>3.1.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-artifact</artifactId>
-      <version>3.0</version>
+      <version>3.6.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-compat</artifactId>
-      <version>3.3.9</version>
+      <version>3.6.0</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -102,4 +101,4 @@
     </dependency>
   </dependencies>
 
-</project>
\ No newline at end of file
+</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
index 475ed50..1107473 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
@@ -103,7 +103,7 @@
     protected Map<String, String> templateProperties = new HashMap<>();
 
     @Parameter
-    private boolean stripFoundationAssertionFromNotices = true;
+    private boolean stripFoundationAssertionFromNotices = false;
 
     private SortedMap<String, SortedSet<Project>> noticeMap;
 
@@ -147,7 +147,12 @@
             getLog().debug("Resolving content for " + artifact.getUrl() + " (" + artifact.getContentFile() + ")");
             File cFile = new File(artifact.getContentFile());
             if (!cFile.isAbsolute()) {
-                cFile = new File(licenseDirectory, artifact.getContentFile());
+                for (File directory : licenseDirectories) {
+                    cFile = new File(directory, artifact.getContentFile());
+                    if (cFile.exists()) {
+                        break;
+                    }
+                }
             }
             if (!cFile.exists()) {
                 if (!bestEffort) {
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
index a94727d..e72404c 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
@@ -74,6 +74,9 @@
     protected List<LicenseSpec> licenses = new ArrayList<>();
 
     @Parameter
+    protected List<NoticeSpec> notices = new ArrayList<>();
+
+    @Parameter
     protected Set<String> excludedScopes = new HashSet<>();
 
     @Parameter
@@ -106,7 +109,10 @@
     @Parameter(required = true)
     private String location;
 
-    @Parameter(required = true)
+    @Parameter
+    protected List<File> licenseDirectories = new ArrayList<>();
+
+    @Parameter
     protected File licenseDirectory;
 
     @Parameter
@@ -122,6 +128,7 @@
     private List<Pattern> excludePatterns;
 
     Map<String, LicenseSpec> urlToLicenseMap = new HashMap<>();
+    Map<String, NoticeSpec> urlToNoticeMap = new HashMap<>();
     Map<String, LicensedProjects> licenseMap = new TreeMap<>();
     private Map<Pair<String, ProjectFlag>, Object> projectFlags = new HashMap<>();
     Map<String, String> noticeOverrides = new HashMap<String, String>();
@@ -133,6 +140,9 @@
     }
 
     protected void init() throws MojoExecutionException {
+        if (licenseDirectory != null) {
+            licenseDirectories.add(0, licenseDirectory);
+        }
         if (warningTouchFile != null) {
             warningTouchFile.getParentFile().mkdirs();
         }
@@ -140,7 +150,7 @@
         excludedScopes.add("system");
         excludePatterns = compileExcludePatterns();
         supplementModels = SupplementalModelHelper.loadSupplements(getLog(), models);
-        buildUrlLicenseMap();
+        buildUrlMaps();
     }
 
     private void interceptLogs() {
@@ -321,7 +331,7 @@
         }
     }
 
-    private void buildUrlLicenseMap() throws MojoExecutionException {
+    private void buildUrlMaps() throws MojoExecutionException {
         for (LicenseSpec license : licenses) {
             if (urlToLicenseMap.put(license.getUrl(), license) != null) {
                 throw new MojoExecutionException("Duplicate URL mapping: " + license.getUrl());
@@ -332,6 +342,16 @@
                 }
             }
         }
+        for (NoticeSpec notice : notices) {
+            if (urlToNoticeMap.put(notice.getUrl(), notice) != null) {
+                throw new MojoExecutionException("Duplicate URL mapping: " + notice.getUrl());
+            }
+            for (String alias : notice.getAliasUrls()) {
+                if (urlToNoticeMap.put(alias, notice) != null) {
+                    throw new MojoExecutionException("Duplicate URL mapping: " + alias);
+                }
+            }
+        }
     }
 
     protected Map<MavenProject, List<Pair<String, String>>> gatherDependencies() throws ProjectBuildingException {
@@ -344,13 +364,16 @@
             MavenProject dep = dependencyGavMap.get(gav);
             if (dep == null) {
                 getLog().warn("Unused override dependency " + gav + "; ignoring...");
-            } else if (override.getUrl() != null) {
+                continue;
+            }
+            if (override.getUrl() != null) {
                 final List<Pair<String, String>> newLicense =
                         Collections.singletonList(new ImmutablePair<>(override.getUrl(), override.getName()));
                 List<Pair<String, String>> prevLicense = dependencyLicenseMap.put(dep, newLicense);
                 warnUnlessFlag(dep, IGNORE_LICENSE_OVERRIDE, "license list for " + toGav(dep)
                         + " changed with <override>; was: " + prevLicense + ", now: " + newLicense);
-            } else if (override.getNoticeUrl() != null) {
+            }
+            if (override.getNoticeUrl() != null) {
                 noticeOverrides.put(gav, override.getNoticeUrl());
                 warnUnlessFlag(dep, IGNORE_NOTICE_OVERRIDE,
                         "notice for " + toGav(dep) + " changed with <override>; now: " + override.getNoticeUrl());
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseUtil.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseUtil.java
index 2a34fc7..1476326 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseUtil.java
@@ -109,7 +109,7 @@
                 while (trimmed.length() > wrapLength) {
                     int cut = trimmed.lastIndexOf(' ', wrapLength);
                     cut = Math.max(cut, trimmed.lastIndexOf('\t', wrapLength));
-                    if (cut != -1) {
+                    if (cut != -1 && cut > leadingWS) {
                         out.append(trimmed.substring(0, cut));
                         out.append('\n');
                         trimmed = trimmed.substring(cut + 1);
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
index 0a24a76..5c4a081 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
@@ -269,5 +269,15 @@
         public Proxy getProxy() {
             return null;
         }
+
+        @java.lang.Override
+        public List<ArtifactRepository> getMirroredRepositories() {
+            return Collections.emptyList();
+        }
+
+        @java.lang.Override
+        public void setMirroredRepositories(List<ArtifactRepository> list) {
+            // unused
+        }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
index ff35162..e44914d 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/project/Project.java
@@ -40,6 +40,7 @@
     private String noticeText;
     private String licenseText;
     private String sourcePointer;
+    private String classifier;
 
     @JsonIgnore
     private MavenProject mavenProject;
@@ -54,6 +55,7 @@
         artifactId = project.getArtifactId();
         version = project.getVersion();
         url = project.getUrl();
+        classifier = project.getArtifact().getClassifier();
         this.artifactPath = artifactPath.getPath();
         setLocation(location);
     }
@@ -63,7 +65,7 @@
             @JsonProperty("artifactId") String artifactId, @JsonProperty("url") String url,
             @JsonProperty("version") String version, @JsonProperty("location") String location,
             @JsonProperty("artifactPath") String artifactPath, @JsonProperty("noticeText") String noticeText,
-            @JsonProperty("licenseText") String licenseText) {
+            @JsonProperty("licenseText") String licenseText, @JsonProperty("classifier") String classifier) {
         this.name = name;
         this.groupId = groupId;
         this.artifactId = artifactId;
@@ -73,6 +75,7 @@
         this.artifactPath = artifactPath;
         this.noticeText = noticeText;
         this.licenseText = licenseText;
+        this.classifier = classifier;
     }
 
     public String getName() {
@@ -91,6 +94,10 @@
         return url;
     }
 
+    public String getClassifier() {
+        return classifier;
+    }
+
     public String getVersion() {
         return version;
     }
@@ -108,6 +115,11 @@
     }
 
     @JsonIgnore
+    public String getJarName() {
+        return artifactId + "-" + version + (classifier != null ? "-" + classifier : "") + ".jar";
+    }
+
+    @JsonIgnore
     public List<String> getLocations() {
         // TODO(mblow): store locations as an set instead of string
         return Arrays.asList(getLocation().split(","));
@@ -129,6 +141,10 @@
         this.url = url;
     }
 
+    public void setClassifier(String classifier) {
+        this.classifier = classifier;
+    }
+
     public void setVersion(String version) {
         this.version = version;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
index b06d9e3..dc6e9e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
@@ -46,7 +46,7 @@
     <dependency>
       <groupId>org.apache.maven</groupId>
       <artifactId>maven-plugin-api</artifactId>
-      <version>3.3.9</version>
+      <version>3.6.0</version>
     </dependency>
   </dependencies>
 
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 1040e81..3b7b20d 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -60,5 +60,10 @@
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelReadInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelReadInterface.java
index ff8d451..31cb69f 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelReadInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelReadInterface.java
@@ -42,10 +42,6 @@
         }
     }
 
-    public void reportError(int ecode) {
-        fba.error(ecode);
-    }
-
     @Override
     public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor) {
         fba = fullBufferAcceptor;
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
index 28c1a71..5c927f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
@@ -21,6 +21,7 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.Queue;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hyracks.api.comm.IBufferAcceptor;
 import org.apache.hyracks.api.comm.IChannelControlBlock;
@@ -31,17 +32,19 @@
 
 public abstract class AbstractChannelWriteInterface implements IChannelWriteInterface {
 
-    public static final int REMOTE_WRITE_ERROR_CODE = 1;
+    public static final int NO_ERROR_CODE = 0;
+    public static final int CONNECTION_LOST_ERROR_CODE = -1;
+    public static final int REMOTE_ERROR_CODE = 1;
     private static final Logger LOGGER = LogManager.getLogger();
     protected final IChannelControlBlock ccb;
     protected final Queue<ByteBuffer> wiFullQueue;
+    protected final AtomicInteger ecode = new AtomicInteger(NO_ERROR_CODE);
     protected boolean channelWritabilityState;
     protected final int channelId;
     protected IBufferAcceptor eba;
     protected int credits;
     protected boolean eos;
     protected boolean eosSent;
-    protected int ecode;
     protected boolean ecodeSent;
     protected ByteBuffer currentWriteBuffer;
     private final ICloseableBufferAcceptor fba;
@@ -54,7 +57,6 @@
         credits = 0;
         eos = false;
         eosSent = false;
-        ecode = -1;
         ecodeSent = false;
     }
 
@@ -73,13 +75,10 @@
         if (writableDataPresent) {
             return credits > 0;
         }
-        if (eos && !eosSent) {
+        if (isPendingCloseWrite()) {
             return true;
         }
-        if (ecode >= 0 && !ecodeSent) {
-            return true;
-        }
-        return false;
+        return ecode.get() == REMOTE_ERROR_CODE && !ecodeSent;
     }
 
     @Override
@@ -117,6 +116,10 @@
         return credits;
     }
 
+    protected boolean isPendingCloseWrite() {
+        return eos && !eosSent && !ecodeSent;
+    }
+
     private class CloseableBufferAcceptor implements ICloseableBufferAcceptor {
         @Override
         public void accept(ByteBuffer buffer) {
@@ -136,7 +139,7 @@
                     return;
                 }
                 eos = true;
-                if (ecode != REMOTE_WRITE_ERROR_CODE) {
+                if (ecode.get() != REMOTE_ERROR_CODE) {
                     adjustChannelWritability();
                 }
             }
@@ -144,8 +147,11 @@
 
         @Override
         public void error(int ecode) {
+            AbstractChannelWriteInterface.this.ecode.set(ecode);
+            if (ecode == CONNECTION_LOST_ERROR_CODE) {
+                return;
+            }
             synchronized (ccb) {
-                AbstractChannelWriteInterface.this.ecode = ecode;
                 adjustChannelWritability();
             }
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelControlBlock.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
index ba463d3..f7ef2aa 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.net.protocols.muxdemux;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.nio.channels.SocketChannel;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -122,7 +123,7 @@
         localCloseAck.set(true);
     }
 
-    synchronized void reportRemoteError(int ecode) {
+    void reportRemoteError(int ecode) {
         ri.flush();
         ri.getFullBufferAcceptor().error(ecode);
         remoteClose.set(true);
@@ -163,4 +164,8 @@
                 + " remoteClose: " + remoteClose + " remoteCloseAck:" + remoteCloseAck + " readCredits: "
                 + ri.getCredits() + " writeCredits: " + wi.getCredits() + "]";
     }
+
+    public InetSocketAddress getRemoteAddress() {
+        return cSet.getMultiplexedConnection().getRemoteAddress();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelSet.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelSet.java
index bd42560..f5cdf2c 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/ChannelSet.java
@@ -82,8 +82,8 @@
             ChannelControlBlock ccb = ccbArray[i];
             if (ccb != null) {
                 if (ccb.completelyClosed()) {
-                    if (LOGGER.isDebugEnabled()) {
-                        LOGGER.debug("Cleaning free channel: " + ccb);
+                    if (LOGGER.isTraceEnabled()) {
+                        LOGGER.trace("Cleaning free channel: " + ccb);
                     }
                     freeChannel(ccb);
                 }
@@ -199,8 +199,8 @@
         synchronized (mConn) {
             for (int i = 0; i < ccbArray.length; ++i) {
                 ChannelControlBlock ccb = ccbArray[i];
-                if (ccb != null && !ccb.getRemoteEOS()) {
-                    ccb.reportRemoteError(-1);
+                if (ccb != null) {
+                    ccb.reportRemoteError(AbstractChannelWriteInterface.CONNECTION_LOST_ERROR_CODE);
                     markEOSAck(i);
                     unmarkPendingCredits(i);
                 }
@@ -218,8 +218,8 @@
         if (ccbArray[idx] != null) {
             assert ccbArray[idx].completelyClosed() : ccbArray[idx].toString();
             if (ccbArray[idx].completelyClosed()) {
-                if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("Cleaning free channel: " + ccbArray[idx]);
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Cleaning free channel: " + ccbArray[idx]);
                 }
                 freeChannel(ccbArray[idx]);
             }
@@ -239,4 +239,8 @@
             ccbArray = Arrays.copyOf(ccbArray, ccbArray.length * 2);
         }
     }
+
+    public MultiplexedConnection getMultiplexedConnection() {
+        return mConn;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
index 432382a..3ba8627 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelReadInterface.java
@@ -21,31 +21,32 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.SocketChannel;
-import java.util.ArrayDeque;
-import java.util.Deque;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.LinkedBlockingDeque;
 
 import org.apache.hyracks.api.comm.IBufferFactory;
 import org.apache.hyracks.api.comm.IChannelControlBlock;
 import org.apache.hyracks.api.exceptions.NetException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class FullFrameChannelReadInterface extends AbstractChannelReadInterface {
 
-    private final Deque<ByteBuffer> riEmptyStack;
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final BlockingDeque<ByteBuffer> riEmptyStack;
     private final IChannelControlBlock ccb;
 
-    FullFrameChannelReadInterface(IChannelControlBlock ccb) {
+    public FullFrameChannelReadInterface(IChannelControlBlock ccb) {
         this.ccb = ccb;
-        riEmptyStack = new ArrayDeque<>();
+        riEmptyStack = new LinkedBlockingDeque<>();
         credits = 0;
 
         emptyBufferAcceptor = buffer -> {
-            int delta = buffer.remaining();
-            synchronized (ccb) {
-                if (ccb.isRemotelyClosed()) {
-                    return;
-                }
-                riEmptyStack.push(buffer);
+            if (ccb.isRemotelyClosed()) {
+                return;
             }
+            final int delta = buffer.remaining();
+            riEmptyStack.push(buffer);
             ccb.addPendingCredits(delta);
         };
     }
@@ -64,6 +65,12 @@
                     currentReadBuffer = bufferFactory.createBuffer();
                 }
             }
+            if (currentReadBuffer == null) {
+                if (LOGGER.isWarnEnabled()) {
+                    LOGGER.warn("{} read buffers exceeded. Current empty buffers: {}", ccb, riEmptyStack.size());
+                }
+                throw new IllegalStateException(ccb + " read buffers exceeded");
+            }
             int rSize = Math.min(size, currentReadBuffer.remaining());
             if (rSize > 0) {
                 currentReadBuffer.limit(currentReadBuffer.position() + rSize);
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
index 17b70a8..3f4618b 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
@@ -50,15 +50,15 @@
             } else {
                 adjustChannelWritability();
             }
-        } else if (ecode >= 0 && !ecodeSent) {
+        } else if (ecode.get() == REMOTE_ERROR_CODE && !ecodeSent) {
             writerState.getCommand().setChannelId(channelId);
             writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.ERROR);
-            writerState.getCommand().setData(ecode);
+            writerState.getCommand().setData(REMOTE_ERROR_CODE);
             writerState.reset(null, 0, null);
             ecodeSent = true;
             ccb.reportLocalEOS();
             adjustChannelWritability();
-        } else if (eos && !eosSent) {
+        } else if (isPendingCloseWrite()) {
             writerState.getCommand().setChannelId(channelId);
             writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL);
             writerState.getCommand().setData(0);
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/MultiplexedConnection.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
index a7fa49e..4c3836a 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.net.protocols.muxdemux;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SocketChannel;
@@ -346,8 +347,8 @@
                 }
                 readerState.readBuffer.flip();
                 readerState.command.read(readerState.readBuffer);
-                if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("Received command: " + readerState.command);
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Received command: " + readerState.command);
                 }
                 ChannelControlBlock ccb = null;
                 switch (readerState.command.getCommandType()) {
@@ -389,8 +390,8 @@
                         muxDemux.getChannelOpenListener().channelOpened(ccb);
                     }
                 }
-                if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("Applied command: " + readerState.command + " on " + ccb);
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Applied command: " + readerState.command + " on " + ccb);
                 }
             }
             if (readerState.pendingReadSize > 0) {
@@ -437,4 +438,8 @@
             }
         }
     }
+
+    public InetSocketAddress getRemoteAddress() {
+        return tcpConnection == null ? null : tcpConnection.getRemoteAddress();
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
new file mode 100644
index 0000000..f9a610c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.hyracks.net.tests;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.comm.IBufferFactory;
+import org.apache.hyracks.api.comm.IChannelControlBlock;
+import org.apache.hyracks.api.comm.ICloseableBufferAcceptor;
+import org.apache.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import org.apache.hyracks.net.protocols.muxdemux.FullFrameChannelReadInterface;
+import org.apache.hyracks.util.StorageUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+@RunWith(Parameterized.class)
+public class FullFrameChannelReadInterfaceTest {
+
+    private static final int TEST_RUNS = 100;
+    private static final int RECEIVER_BUFFER_COUNT = 50;
+    private static int FRAMES_TO_READ_COUNT = 10000;
+    private static final int FRAME_SIZE = StorageUtil.getIntSizeInBytes(32, StorageUtil.StorageUnit.KILOBYTE);
+    private static final int EXPECTED_CHANNEL_CREDIT = FRAME_SIZE * RECEIVER_BUFFER_COUNT;
+
+    @Parameterized.Parameters
+    public static Object[][] data() {
+        return new Object[TEST_RUNS][0];
+    }
+
+    @Test
+    public void bufferRecycleTest() throws Exception {
+        final AtomicInteger channelCredit = new AtomicInteger();
+        final IChannelControlBlock ccb = mockChannelControlBlock(channelCredit);
+        final ReadBufferFactory bufferFactory = new ReadBufferFactory(RECEIVER_BUFFER_COUNT, FRAME_SIZE);
+        final FullFrameChannelReadInterface readInterface = new FullFrameChannelReadInterface(ccb);
+        final LinkedBlockingDeque<ByteBuffer> fullBufferQ = new LinkedBlockingDeque<>();
+        readInterface.setFullBufferAcceptor(new ReadFullBufferAcceptor(fullBufferQ));
+        readInterface.setBufferFactory(bufferFactory, RECEIVER_BUFFER_COUNT, FRAME_SIZE);
+        Assert.assertEquals(EXPECTED_CHANNEL_CREDIT, channelCredit.get());
+        final SocketChannel socketChannel = mockSocketChannel(ccb);
+        final Thread networkFrameReader = new Thread(() -> {
+            try {
+                int framesRead = FRAMES_TO_READ_COUNT;
+                while (framesRead > 0) {
+                    while (channelCredit.get() == 0) {
+                        synchronized (channelCredit) {
+                            channelCredit.wait(10000);
+                            if (channelCredit.get() == 0) {
+                                System.err.println("Sender doesn't have any write credit");
+                                System.exit(1);
+                            }
+                        }
+                    }
+                    readInterface.read(socketChannel, FRAME_SIZE);
+                    framesRead--;
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        });
+
+        final Thread frameProcessor = new Thread(() -> {
+            int framesProcessed = 0;
+            try {
+                while (true) {
+                    final ByteBuffer fullFrame = fullBufferQ.take();
+                    fullFrame.clear();
+                    readInterface.getEmptyBufferAcceptor().accept(fullFrame);
+                    framesProcessed++;
+                    if (framesProcessed == FRAMES_TO_READ_COUNT) {
+                        return;
+                    }
+                }
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        });
+        networkFrameReader.start();
+        frameProcessor.start();
+        networkFrameReader.join();
+        frameProcessor.join();
+        if (channelCredit.get() != EXPECTED_CHANNEL_CREDIT) {
+            System.err
+                    .println("Expected channel credit " + EXPECTED_CHANNEL_CREDIT + " , found " + channelCredit.get());
+            System.exit(1);
+        }
+    }
+
+    private IChannelControlBlock mockChannelControlBlock(AtomicInteger credit) {
+        final ChannelControlBlock ccb = Mockito.mock(ChannelControlBlock.class);
+        Mockito.when(ccb.isRemotelyClosed()).thenReturn(false);
+        Mockito.doAnswer(invocation -> {
+            final Integer delta = invocation.getArgumentAt(0, Integer.class);
+            credit.addAndGet(delta);
+            synchronized (credit) {
+                credit.notifyAll();
+            }
+            return null;
+        }).when(ccb).addPendingCredits(Mockito.anyInt());
+        return ccb;
+    }
+
+    private SocketChannel mockSocketChannel(IChannelControlBlock ccb) throws IOException {
+        final SocketChannel sc = Mockito.mock(SocketChannel.class);
+        Mockito.when(sc.read(Mockito.any(ByteBuffer.class))).thenAnswer(invocation -> {
+            ccb.addPendingCredits(-FRAME_SIZE);
+            final ByteBuffer buffer = invocation.getArgumentAt(0, ByteBuffer.class);
+            while (buffer.hasRemaining()) {
+                buffer.put((byte) 0);
+            }
+            return FRAME_SIZE;
+        });
+        return sc;
+    }
+
+    private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+        private final BlockingQueue<ByteBuffer> fullBufferQ;
+
+        ReadFullBufferAcceptor(BlockingQueue<ByteBuffer> fullBuffer) {
+            this.fullBufferQ = fullBuffer;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            fullBufferQ.add(buffer);
+        }
+
+        @Override
+        public void close() {
+        }
+
+        @Override
+        public void error(int ecode) {
+        }
+    }
+
+    public class ReadBufferFactory implements IBufferFactory {
+        private final int limit;
+        private final int frameSize;
+        private int counter = 0;
+
+        ReadBufferFactory(int limit, int frameSize) {
+            this.limit = limit;
+            this.frameSize = frameSize;
+        }
+
+        @Override
+        public ByteBuffer createBuffer() {
+            if (counter >= limit) {
+                throw new IllegalStateException("Buffer limit exceeded");
+            }
+            counter++;
+            return ByteBuffer.allocate(frameSize);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index 6c16bd1..2dce1cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/src/main/java/org/apache/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public class BloomFilter {
@@ -275,7 +276,7 @@
         return new BloomFilterBuilder(numElements, numHashes, numBitsPerElement);
     }
 
-    public class BloomFilterBuilder implements IIndexBulkLoader {
+    public class BloomFilterBuilder extends PageWriteFailureCallback implements IIndexBulkLoader {
         private final long[] hashes = BloomFilter.createHashArray();
         private final long estimatedNumElements;
         private final int numHashes;
@@ -286,6 +287,7 @@
         private final ICachedPage[] pages;
         private ICachedPage metaDataPage = null;
 
+        @SuppressWarnings("squid:S1181") // Catch Throwable Must return all confiscated pages
         public BloomFilterBuilder(long estimatedNumElemenets, int numHashes, int numBitsPerElement)
                 throws HyracksDataException {
             if (!isActivated) {
@@ -303,11 +305,22 @@
             actualNumElements = 0;
             pages = new ICachedPage[numPages];
             int currentPageId = 1;
-            while (currentPageId <= numPages) {
-                ICachedPage page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
-                initPage(page.getBuffer().array());
-                pages[currentPageId - 1] = page;
-                ++currentPageId;
+            try {
+                while (currentPageId <= numPages) {
+                    ICachedPage page =
+                            bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
+                    initPage(page.getBuffer().array());
+                    pages[currentPageId - 1] = page;
+                    ++currentPageId;
+                }
+            } catch (Throwable th) {
+                // return confiscated pages
+                for (int i = 0; i < currentPageId; i++) {
+                    if (pages[i] != null) {
+                        bufferCache.returnPage(pages[i]);
+                    }
+                }
+                throw th;
             }
         }
 
@@ -364,11 +377,14 @@
         @Override
         public void end() throws HyracksDataException {
             allocateAndInitMetaDataPage();
-            queue.put(metaDataPage);
+            queue.put(metaDataPage, this);
             for (ICachedPage p : pages) {
-                queue.put(p);
+                queue.put(p, this);
             }
             bufferCache.finishQueue();
+            if (hasFailed()) {
+                throw HyracksDataException.create(getFailure());
+            }
             BloomFilter.this.numBits = numBits;
             BloomFilter.this.numHashes = numHashes;
             BloomFilter.this.numElements = actualNumElements;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 782af54..2a522f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -68,7 +68,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 86497a8..8c327f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -77,7 +77,7 @@
 
         ByteBuffer buf = frame.getBuffer();
         byte[] pageArray = buf.array();
-        IPrefixSlotManager slotManager = (IPrefixSlotManager) frame.getSlotManager();
+        IPrefixSlotManager slotManager = frame.getSlotManager();
 
         // perform analysis pass
         ArrayList<KeyPartition> keyPartitions = getKeyPartitions(frame, cmp, occurrenceThreshold);
@@ -334,7 +334,7 @@
         int maxCmps = cmps.length - 1;
         ByteBuffer buf = frame.getBuffer();
         byte[] pageArray = buf.array();
-        IPrefixSlotManager slotManager = (IPrefixSlotManager) frame.getSlotManager();
+        IPrefixSlotManager slotManager = frame.getSlotManager();
 
         ArrayList<KeyPartition> keyPartitions = new ArrayList<KeyPartition>();
         KeyPartition kp = new KeyPartition(maxCmps);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 49eca6f..566d8e2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 
 public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -48,6 +49,8 @@
     protected boolean appendOpCallbackProceedResult;
     protected byte[] searchCallbackProceedResultFalseValue;
     protected byte[] searchCallbackProceedResultTrueValue;
+    protected final ITupleFilterFactory tupleFilterFactory;
+    protected final long outputLimit;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -56,7 +59,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
         this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
     }
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -64,8 +67,8 @@
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
-            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -80,6 +83,8 @@
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
         this.outRecDescs[0] = outRecDesc;
+        this.tupleFilterFactory = tupleFilterFactory;
+        this.outputLimit = outputLimit;
         this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
         this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
         this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
@@ -92,7 +97,7 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 6bbf437..70b88fe 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -50,7 +51,7 @@
             throws HyracksDataException {
         this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
-                missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null, null);
+                missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
     }
 
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -58,11 +59,12 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index fb8770e..4fc8af9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -1012,9 +1012,7 @@
             try {
                 int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
                         interiorFrame.getBytesRequiredToWriteTuple(tuple));
-
                 NodeFrontier leafFrontier = nodeFrontiers.get(0);
-
                 int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
                 int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
 
@@ -1045,12 +1043,11 @@
 
                         ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
 
-                        queue.put(leafFrontier.page);
+                        queue.put(leafFrontier.page, this);
                         for (ICachedPage c : pagesToWrite) {
-                            queue.put(c);
+                            queue.put(c, this);
                         }
                         pagesToWrite.clear();
-
                         splitKey.setRightPage(leafFrontier.pageId);
                     }
                     if (tupleSize > maxTupleSize) {
@@ -1155,7 +1152,7 @@
                 ICachedPage lastLeaf = nodeFrontiers.get(level).page;
                 int lastLeafPage = nodeFrontiers.get(level).pageId;
                 lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
-                queue.put(lastLeaf);
+                queue.put(lastLeaf, this);
                 nodeFrontiers.get(level).page = null;
                 persistFrontiers(level + 1, lastLeafPage);
                 return;
@@ -1170,9 +1167,8 @@
             ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
             int finalPageId = freePageManager.takePage(metaFrame);
             frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-            queue.put(frontier.page);
+            queue.put(frontier.page, this);
             frontier.pageId = finalPageId;
-
             persistFrontiers(level + 1, finalPageId);
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
index f7e0ce0..1491424 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -78,7 +78,7 @@
     @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
 
-        page = ((BTreeCursorInitialState) initialState).getPage();
+        page = initialState.getPage();
         isPageDirty = false;
         frame.setPage(page);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index 5f05ce9..e8c7b05 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -110,5 +110,9 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
index 0c4167c..333f6d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IMetadataPageManagerFactory.java
@@ -18,13 +18,14 @@
  */
 package org.apache.hyracks.storage.am.common.api;
 
+import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 /**
  * A factory class for {@link org.apache.hyracks.storage.am.common.api.IMetadataPageManager}
  */
 @FunctionalInterface
-public interface IMetadataPageManagerFactory extends IPageManagerFactory {
+public interface IMetadataPageManagerFactory extends IPageManagerFactory, IJsonSerializable {
 
     @Override
     IMetadataPageManager createPageManager(IBufferCache bufferCache);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
index b7987f8..f8a929d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPageManager.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.common.api;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public interface IPageManager {
 
@@ -42,12 +43,14 @@
      * 3. When we need to have a persisted state.
      *
      * Note: This method will not force indexes to disk driver using fsync
+     *
      * @throws HyracksDataException
      */
-    void close() throws HyracksDataException;
+    void close(IPageWriteFailureCallback callback) throws HyracksDataException;
 
     /**
      * Create a metadata frame to be used for reading and writing to metadata pages
+     *
      * @return a new metadata frame
      */
     ITreeIndexMetadataFrame createMetadataFrame();
@@ -87,6 +90,7 @@
     /**
      * Get the location of a block of free pages to use for index operations
      * This is used for records that are larger than a normal page
+     *
      * @param frame
      *            A metadata frame to use to wrap metadata pages
      * @return The starting page location, or -1 if a block of free pages could be found or allocated
@@ -107,6 +111,7 @@
 
     /**
      * Add a page back to the pool of free pages within an index file
+     *
      * @param frame
      *            A metadata frame to use to wrap metadata pages
      * @param page
@@ -129,6 +134,7 @@
 
     /**
      * Check whether the index is empty or not.
+     *
      * @param frame
      *            interior frame
      * @param rootPage
@@ -140,6 +146,7 @@
 
     /**
      * Get the root page of the id
+     *
      * @return the root page
      * @throws HyracksDataException
      */
@@ -147,6 +154,7 @@
 
     /**
      * Get the first page to start the bulk load
+     *
      * @return
      * @throws HyracksDataException
      */
@@ -154,6 +162,7 @@
 
     /**
      * Set the root page id and finalize the bulk load operation
+     *
      * @param rootPage
      * @throws HyracksDataException
      */
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
index 942517c..b3db003 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IPrimitiveValueProviderFactory.java
@@ -21,11 +21,13 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.io.IJsonSerializable;
+
 /**
  * Provides {@link org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider} for R-Tree operations
  */
 @FunctionalInterface
-public interface IPrimitiveValueProviderFactory extends Serializable {
+public interface IPrimitiveValueProviderFactory extends Serializable, IJsonSerializable {
     /**
      * @return {@link org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider} instance
      */
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
index 7ac49c6..dc59612 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -36,7 +36,7 @@
          * Storage version #. Change this if you alter any tree frame formats to stop
          * possible corruption from old versions reading new formats.
          */
-        public static final int VERSION = 6;
+        public static final int VERSION = 7;
         public static final int TUPLE_COUNT_OFFSET = 0;
         public static final int FREE_SPACE_OFFSET = TUPLE_COUNT_OFFSET + 4;
         public static final int LEVEL_OFFSET = FREE_SPACE_OFFSET + 4;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
index 50673cf..b5dfe00 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
@@ -19,8 +19,9 @@
 
 package org.apache.hyracks.storage.am.common.api;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public interface ITupleFilter {
-    public boolean accept(IFrameTupleReference tuple) throws Exception;
+    public boolean accept(IFrameTupleReference tuple) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
index 16c0afa..c7be7eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
@@ -39,6 +39,6 @@
     /**
      * Abort the bulk modify op
      */
-    public void abort();
+    public void abort() throws HyracksDataException;
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java
index 7a0aa6d..094b090 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java
@@ -19,12 +19,18 @@
 package org.apache.hyracks.storage.am.common.data;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.api.INumeric;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class PointablePrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
     private static final long serialVersionUID = 1L;
 
@@ -48,4 +54,17 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("pf", pf.toJson(registry));
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final IPointableFactory pf = (IPointableFactory) registry.deserialize(json.get("pf"));
+        return new PointablePrimitiveValueProviderFactory(pf);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index 2c691f1..fb287cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IMissingWriter;
@@ -37,11 +38,14 @@
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilter;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -95,22 +99,43 @@
     protected byte[] searchCallbackProceedResultFalseValue;
     protected byte[] searchCallbackProceedResultTrueValue;
 
+    protected final ITupleFilterFactory tupleFilterFactory;
+    protected ReferenceFrameTupleReference referenceFilterTuple;
+    // filter out tuples based on the query-provided select condition
+    // only results satisfying the filter condition would be returned to downstream operators
+    protected ITupleFilter tupleFilter;
+    protected final long outputLimit;
+    protected long outputCount = 0;
+    protected boolean finished;
+
+    // no filter and limit pushdown
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
             throws HyracksDataException {
         this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
-                null);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
+                false, null, null);
     }
 
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+        this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                tupleFilterFactory, outputLimit, false, null, null);
+    }
+
+    public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
+            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+            throws HyracksDataException {
         this.ctx = ctx;
         this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
         this.retainInput = retainInput;
@@ -138,6 +163,12 @@
         if (ctx.getStatsCollector() != null) {
             ctx.getStatsCollector().add(stats);
         }
+        this.tupleFilterFactory = tupleFactoryFactory;
+        this.outputLimit = outputLimit;
+
+        if (this.tupleFilterFactory != null && this.retainMissing) {
+            throw new IllegalStateException("RetainMissing with tuple filter is not supported");
+        }
     }
 
     protected abstract ISearchPredicate createSearchPredicate();
@@ -179,6 +210,13 @@
             buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
         }
 
+        if (tupleFilterFactory != null) {
+            tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
+            referenceFilterTuple = new ReferenceFrameTupleReference();
+        }
+        finished = false;
+        outputCount = 0;
+
         try {
             searchPred = createSearchPredicate();
             tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
@@ -201,9 +239,14 @@
     protected void writeSearchResults(int tupleIndex) throws Exception {
         long matchingTupleCount = 0;
         while (cursor.hasNext()) {
-            matchingTupleCount++;
-            tb.reset();
             cursor.next();
+            matchingTupleCount++;
+            ITupleReference tuple = cursor.getTuple();
+            if (tupleFilter != null && !tupleFilter.accept(referenceFilterTuple.reset(tuple))) {
+                continue;
+            }
+            tb.reset();
+
             if (retainInput) {
                 frameTuple.reset(accessor, tupleIndex);
                 for (int i = 0; i < frameTuple.getFieldCount(); i++) {
@@ -211,7 +254,6 @@
                     tb.addFieldEndOffset();
                 }
             }
-            ITupleReference tuple = cursor.getTuple();
             writeTupleToOutput(tuple);
             if (appendSearchCallbackProceedResult) {
                 writeSearchCallbackProceedResult(tb,
@@ -222,6 +264,10 @@
                 writeFilterTupleToOutput(((ILSMIndexCursor) cursor).getFilterMaxTuple());
             }
             FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+            if (outputLimit >= 0 && ++outputCount >= outputLimit) {
+                finished = true;
+                break;
+            }
         }
         stats.getTupleCounter().update(matchingTupleCount);
 
@@ -237,7 +283,7 @@
         accessor.reset(buffer);
         int tupleCount = accessor.getTupleCount();
         try {
-            for (int i = 0; i < tupleCount; i++) {
+            for (int i = 0; i < tupleCount && !finished; i++) {
                 resetSearchPredicate(i);
                 cursor.close();
                 indexAccessor.search(cursor, searchPred);
@@ -344,4 +390,50 @@
         }
     }
 
+    /**
+     * A wrapper class to wrap ITupleReference into IFrameTupleReference, as the latter
+     * is used by ITupleFilter
+     *
+     */
+    private static class ReferenceFrameTupleReference implements IFrameTupleReference {
+        private ITupleReference tuple;
+
+        public IFrameTupleReference reset(ITupleReference tuple) {
+            this.tuple = tuple;
+            return this;
+        }
+
+        @Override
+        public int getFieldCount() {
+            return tuple.getFieldCount();
+        }
+
+        @Override
+        public byte[] getFieldData(int fIdx) {
+            return tuple.getFieldData(fIdx);
+        }
+
+        @Override
+        public int getFieldStart(int fIdx) {
+            return tuple.getFieldStart(fIdx);
+        }
+
+        @Override
+        public int getFieldLength(int fIdx) {
+            return tuple.getFieldLength(fIdx);
+        }
+
+        @Override
+        public IFrameTupleAccessor getFrameTupleAccessor() {
+            throw new UnsupportedOperationException(
+                    "getFrameTupleAccessor is not supported by ReferenceFrameTupleReference");
+        }
+
+        @Override
+        public int getTupleIndex() {
+            throw new UnsupportedOperationException("getTupleIndex is not supported by ReferenceFrameTupleReference");
+        }
+
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
index 5c389d2..97e7ed7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManager.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public class AppendOnlyLinkedMetadataPageManager implements IMetadataPageManager {
@@ -207,7 +208,7 @@
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void close(IPageWriteFailureCallback callback) throws HyracksDataException {
         if (ready) {
             IFIFOPageQueue queue = bufferCache.createFIFOQueue();
             ITreeIndexMetadataFrame metaFrame = frameFactory.createFrame();
@@ -220,7 +221,9 @@
             }
             int finalMetaPage = getMaxPageId(metaFrame) + 1;
             confiscatedPage.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalMetaPage));
-            queue.put(confiscatedPage);
+            // WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page
+            // won't be flushed to disk because it won't be dirty until the write latch has been released.
+            queue.put(confiscatedPage, callback);
             bufferCache.finishQueue();
             metadataPage = getMetadataPageId();
             ready = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
index 2c581b0..947bfa0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/AppendOnlyLinkedMetadataPageManagerFactory.java
@@ -18,11 +18,16 @@
  */
 package org.apache.hyracks.storage.am.common.freepage;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class AppendOnlyLinkedMetadataPageManagerFactory implements IMetadataPageManagerFactory {
     private static final long serialVersionUID = 1L;
     public static final AppendOnlyLinkedMetadataPageManagerFactory INSTANCE =
@@ -36,4 +41,13 @@
         return new AppendOnlyLinkedMetadataPageManager(bufferCache, new LIFOMetaDataFrameFactory());
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/LinkedMetaDataPageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/LinkedMetaDataPageManager.java
index 951d824..e348e24 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/LinkedMetaDataPageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/freepage/LinkedMetaDataPageManager.java
@@ -28,8 +28,14 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrameFactory;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
+/**
+ * @deprecated
+ *             This class must not be used. Instead, use {@link AppendOnlyLinkedMetadataPageManager}
+ */
+@Deprecated
 public class LinkedMetaDataPageManager implements IMetadataPageManager {
     private final IBufferCache bufferCache;
     private int fileId = -1;
@@ -238,7 +244,7 @@
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void close(IPageWriteFailureCallback callback) throws HyracksDataException {
         if (ready) {
             ICachedPage metaNode =
                     bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, getMetadataPageId()), false);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 905c99d..b77f14f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -26,19 +26,19 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -95,7 +95,7 @@
             freePageManager.open(fileId);
             freePageManager.init(interiorFrameFactory, leafFrameFactory);
             setRootPage();
-            freePageManager.close();
+            freePageManager.close(HaltOnFailureCallback.INSTANCE);
             failed = false;
         } finally {
             bufferCache.closeFile(fileId);
@@ -132,7 +132,7 @@
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_INDEX);
         }
-        freePageManager.close();
+        freePageManager.close(HaltOnFailureCallback.INSTANCE);
         bufferCache.closeFile(fileId);
         isActive = false;
     }
@@ -227,7 +227,7 @@
         return fieldCount;
     }
 
-    public abstract class AbstractTreeIndexBulkLoader implements IIndexBulkLoader {
+    public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
         protected final MultiComparator cmp;
         protected final int slotSize;
         protected final int leafMaxBytes;
@@ -297,6 +297,9 @@
         @Override
         public void end() throws HyracksDataException {
             bufferCache.finishQueue();
+            if (hasFailed()) {
+                throw HyracksDataException.create(getFailure());
+            }
             freePageManager.setRootPageId(rootPage);
         }
 
@@ -317,31 +320,6 @@
         public void setLeafFrame(ITreeIndexFrame leafFrame) {
             this.leafFrame = leafFrame;
         }
-
-    }
-
-    public class TreeIndexInsertBulkLoader implements IIndexBulkLoader {
-        ITreeIndexAccessor accessor;
-
-        public TreeIndexInsertBulkLoader() throws HyracksDataException {
-            accessor = (ITreeIndexAccessor) createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            accessor.insert(tuple);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            // do nothing
-        }
-
-        @Override
-        public void abort() {
-
-        }
-
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/ExtendedIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/ExtendedIndexAccessParameters.java
index dbefe74..51729d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/ExtendedIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/ExtendedIndexAccessParameters.java
@@ -19,7 +19,6 @@
 package org.apache.hyracks.storage.am.common.impls;
 
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
-import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
index 0068f4f..e1c1202 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
@@ -23,7 +23,6 @@
 
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
 public class NoOpIndexAccessParameters implements IIndexAccessParameters {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
index 636e4f5..56600b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -35,6 +35,5 @@
     FLUSH,
     REPLICATE,
     DISK_COMPONENT_SCAN,
-    DELETE_MEMORY_COMPONENT,
-    DELETE_DISK_COMPONENTS
+    DELETE_COMPONENTS
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
index 18de540..c635411 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
@@ -19,8 +19,6 @@
 
 package org.apache.hyracks.storage.am.common.tuples;
 
-import java.util.Arrays;
-
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 /**
@@ -96,14 +94,17 @@
         return tuples[tupleIndex].getFieldLength(fieldIndex);
     }
 
+    /**
+     * Right now this class is only used by inverted index, and only contains 2 tuples.
+     * As a result, sequential search would be more efficient than binary search
+     */
     private int getTupleIndex(int fIdx) {
-        int tupleIndex = Arrays.binarySearch(fieldCounts, 0, numTuples - 1, fIdx);
-        if (tupleIndex < 0) {
-            tupleIndex = -tupleIndex - 1;
-        } else {
-            ++tupleIndex;
+        for (int i = 0; i < numTuples; i++) {
+            if (fIdx < fieldCounts[i]) {
+                return i;
+            }
         }
-        return tupleIndex;
+        throw new IllegalArgumentException("Illegal field index " + fIdx);
     }
 
     private int getFieldIndex(int tupleIndex, int fIdx) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
index 934ab65..80b6baa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -87,5 +87,9 @@
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index 445f363..76f7e61 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -26,6 +26,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -35,6 +37,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class ExternalBTreeLocalResource extends LSMBTreeLocalResource {
 
     private static final long serialVersionUID = 1L;
@@ -53,6 +58,11 @@
                 ioSchedulerProvider, durable);
     }
 
+    private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+    }
+
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = serviceCtx.getIoManager();
@@ -64,4 +74,18 @@
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, durable, metadataPageManagerFactory, serviceCtx.getTracer());
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
+        return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
+                lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index acdb09e..2a57e74 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -26,6 +26,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -35,6 +37,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
  * The local resource for disk only lsm btree with buddy tree
  */
@@ -56,6 +61,12 @@
                 ioSchedulerProvider, durable);
     }
 
+    private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
+            int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields)
+            throws HyracksDataException {
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+    }
+
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = serviceCtx.getIoManager();
@@ -67,4 +78,18 @@
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, bloomFilterKeyFields, durable, metadataPageManagerFactory, serviceCtx.getTracer());
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
+        return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
+                lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index f0b86d2..40278d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -27,6 +27,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -39,6 +41,9 @@
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class LSMBTreeLocalResource extends LsmResource {
 
     private static final long serialVersionUID = 1L;
@@ -65,6 +70,15 @@
         this.btreeFields = btreeFields;
     }
 
+    protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
+        super(registry, json);
+        this.bloomFilterKeyFields = bloomFilterKeyFields;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
+        this.isPrimary = isPrimary;
+        this.btreeFields = btreeFields;
+    }
+
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = serviceCtx.getIoManager();
@@ -80,4 +94,31 @@
                 ioOpCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer());
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final int[] bloomFilterKeyFields = OBJECT_MAPPER.convertValue(json.get("bloomFilterKeyFields"), int[].class);
+        final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
+        final boolean isPrimary = json.get("isPrimary").asBoolean();
+        final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+        return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
+                btreeFields);
+    }
+
+    @Override
+    protected void appendToJson(final ObjectNode json, IPersistedResourceRegistry registry)
+            throws HyracksDataException {
+        super.appendToJson(json, registry);
+        json.putPOJO("bloomFilterKeyFields", bloomFilterKeyFields);
+        json.put("bloomFilterFalsePositiveRate", bloomFilterFalsePositiveRate);
+        json.put("isPrimary", isPrimary);
+        json.putPOJO("btreeFields", btreeFields);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
index c0f7571..2be5125 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -22,6 +22,7 @@
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -40,8 +41,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -54,12 +53,14 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.util.trace.ITracer;
 
 /**
@@ -170,8 +171,7 @@
     // The only reason to override the following method is that it uses a different context object
     // in addition, determining whether or not to keep deleted tuples is different here
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public LSMBTreeMergeOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         opCtx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -188,16 +188,18 @@
             }
         }
         LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
-        BTree firstBTree = ((LSMBTreeDiskComponent) mergingComponents.get(0)).getIndex();
-        BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getIndex();
+        BTree lastBTree = ((LSMBTreeDiskComponent) mergingComponents.get(0)).getIndex();
+        BTree firstBTree = ((LSMBTreeDiskComponent) mergingComponents.get(mergingComponents.size() - 1)).getIndex();
         FileReference firstFile = firstBTree.getFileReference();
         FileReference lastFile = lastBTree.getFileReference();
         LSMComponentFileReferences relMergeFileRefs =
                 fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
-        ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, cursor,
+        LSMBTreeMergeOperation mergeOp = new LSMBTreeMergeOperation(accessor, cursor,
                 relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath()));
+                ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     // This function should only be used when a transaction fail. it doesn't
@@ -286,11 +288,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -353,8 +350,7 @@
 
     // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
     }
 
@@ -401,16 +397,16 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     // The bulk loader used for both initial loading and transaction
@@ -418,21 +414,37 @@
     public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
         private final ILSMDiskComponentBulkLoader componentBulkLoader;
+        private final LoadOperation loadOp;
 
         private final boolean isTransaction;
 
         public LSMTwoPCBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component = createDiskComponent(transactionComponentFactory,
+                        componentFileRefs.getInsertIndexFileReference(), null,
+                        componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
-
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
             componentBulkLoader =
-                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, true);
         }
 
         // It is expected that the mode was set to insert operation before
@@ -444,16 +456,26 @@
 
         @Override
         public void end() throws HyracksDataException {
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        try {
+                            component.markAsValid(durable, loadOp);
+                        } finally {
+                            ioOpCallback.afterFinalize(loadOp);
+                        }
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(loadOp);
+                    }
                 }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
@@ -465,25 +487,27 @@
         }
 
         @Override
-        public void abort() {
+        public void abort() throws HyracksDataException {
             try {
                 componentBulkLoader.abort();
-            } catch (Exception e) {
-                // Do nothing
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    null, componentFileRefs.getBloomFilterFileReference(), true);
+        @Override
+        public void writeFailed(ICachedPage page, Throwable failure) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasFailed() {
+            return componentBulkLoader.hasFailed();
+        }
+
+        @Override
+        public Throwable getFailure() {
+            return componentBulkLoader.getFailure();
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index 1ba55f7..be91244 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -35,7 +36,6 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
@@ -44,7 +44,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -59,12 +58,14 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.util.trace.ITracer;
 
 public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeIndex, ITwoPCIndex {
@@ -217,16 +218,16 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     @Override
@@ -244,8 +245,7 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
     }
 
@@ -256,8 +256,8 @@
 
     protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
             throws HyracksDataException {
-        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getIndex();
-        BTree lastTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
+        BTree lastTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(0)).getIndex();
+        BTree firstTree = ((LSMBTreeWithBuddyDiskComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
                 .getIndex();
         FileReference firstFile = firstTree.getFileReference();
         FileReference lastFile = lastTree.getFileReference();
@@ -267,7 +267,7 @@
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+    public LSMBTreeWithBuddyMergeOperation createMergeOperation(ILSMIndexOperationContext ctx)
             throws HyracksDataException {
         ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
         bctx.setOperation(IndexOperation.MERGE);
@@ -289,10 +289,13 @@
                     .get(secondDiskComponents.size() - 1);
         }
 
-        ioScheduler.scheduleOperation(
+        LSMBTreeWithBuddyMergeOperation mergeOp =
                 new LSMBTreeWithBuddyMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
                         relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
-                        callback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples));
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples);
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+
     }
 
     // This method creates the appropriate opContext for the targeted version
@@ -331,7 +334,8 @@
                 numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
                         .getBloomFilter().getNumElements();
             }
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+            componentBulkLoader =
+                    mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             try {
                 while (buddyBtreeCursor.hasNext()) {
                     buddyBtreeCursor.next();
@@ -342,7 +346,7 @@
                 buddyBtreeCursor.close();
             }
         } else {
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+            componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
 
         try {
@@ -430,11 +434,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
         // Even though, we deactivate the index, we don't exit components or
         // modify any of the lists to make sure they
         // are there if the index was opened again
@@ -497,21 +496,39 @@
     // modifications
     public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
+        private final LoadOperation loadOp;
         private final ILSMDiskComponentBulkLoader componentBulkLoader;
         private final boolean isTransaction;
 
         public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
 
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
             componentBulkLoader =
-                    component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, false);
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
@@ -521,16 +538,26 @@
 
         @Override
         public void end() throws HyracksDataException {
-            componentBulkLoader.end();
-            if (component.getComponentSize() > 0) {
-                if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        try {
+                            component.markAsValid(durable, loadOp);
+                        } finally {
+                            ioOpCallback.afterFinalize(loadOp);
+                        }
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(loadOp);
+                    }
                 }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
@@ -540,25 +567,31 @@
         }
 
         @Override
-        public void abort() {
+        public void abort() throws HyracksDataException {
             try {
-                componentBulkLoader.abort();
-            } catch (Exception e) {
+                try {
+                    componentBulkLoader.abort();
+                } finally {
+                    ioOpCallback.afterFinalize(loadOp);
+                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then eventually be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                    true);
+        @Override
+        public void writeFailed(ICachedPage page, Throwable failure) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasFailed() {
+            return componentBulkLoader.hasFailed();
+        }
+
+        @Override
+        public Throwable getFailure() {
+            return componentBulkLoader.getFailure();
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
index 7f53ed5..1309e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
@@ -26,7 +26,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 41a11e6..5332b1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -74,7 +74,7 @@
 
 public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
 
-    private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
+    private static final ICursorFactory cursorFactory = LSMBTreeSearchCursor::new;
     // Common for in-memory and on-disk components.
     protected final ITreeIndexFrameFactory insertLeafFrameFactory;
     protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
@@ -109,8 +109,7 @@
                     new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), interiorFrameFactory,
                             insertLeafFrameFactory, cmpFactories, fieldCount,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_" + i)),
-                    virtualBufferCache, i == 0 ? true : false,
-                    filterHelper == null ? null : filterHelper.createFilter());
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -277,7 +276,7 @@
             }
             component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
                     true);
-            componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
             IIndexCursor scanCursor = accessor.createSearchCursor(false);
             accessor.search(scanCursor, nullPred);
             try {
@@ -337,7 +336,7 @@
                     mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
                     componentBulkLoader =
-                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                            mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
                     while (cursor.hasNext()) {
                         cursor.next();
                         ITupleReference frameTuple = cursor.getTuple();
@@ -391,7 +390,7 @@
             LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback) {
         ILSMIndexAccessor accessor = createAccessor(opCtx);
         return new LSMBTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
@@ -476,6 +475,6 @@
         }
         LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
         return new LSMBTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
-                mergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index 54bc1fe..2240fd9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
-import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -35,6 +34,7 @@
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 
@@ -54,21 +54,14 @@
 
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
-        String ts = getCurrentTimestamp();
-        String baseName = ts + DELIMITER + ts;
-        // Begin timestamp and end timestamp are identical since it is a flush
+        String baseName = getNextComponentSequence(btreeFilter);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX), null,
                 hasBloomFilter ? baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
     }
 
     @Override
-    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
-            throws HyracksDataException {
-        String[] firstTimestampRange = firstFileName.split(DELIMITER);
-        String[] lastTimestampRange = lastFileName.split(DELIMITER);
-
-        String baseName = firstTimestampRange[0] + DELIMITER + lastTimestampRange[1];
-        // Get the range of timestamps by taking the earliest and the latest timestamps
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
+        final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX), null,
                 hasBloomFilter ? baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX) : null);
     }
@@ -76,21 +69,22 @@
     @Override
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException {
         List<LSMComponentFileReferences> validFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBloomFilterFiles = new ArrayList<>();
         // create transaction filter <to hide transaction files>
         FilenameFilter transactionFilter = getTransactionFileFilter(false);
         // List of valid BTree files.
-        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles);
+        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles,
+                btreeFactory.getBufferCache());
         HashSet<String> btreeFilesSet = new HashSet<>();
-        for (ComparableFileName cmpFileName : allBTreeFiles) {
-            int index = cmpFileName.fileName.lastIndexOf(DELIMITER);
-            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+        for (IndexComponentFileReference cmpFileName : allBTreeFiles) {
+            int index = cmpFileName.getFileName().lastIndexOf(DELIMITER);
+            btreeFilesSet.add(cmpFileName.getFileName().substring(0, index));
         }
 
         if (hasBloomFilter) {
             validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(transactionFilter, bloomFilterFilter),
-                    null);
+                    null, btreeFactory.getBufferCache());
             // Sanity check.
             if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
                 throw HyracksDataException.create(ErrorCode.UNEQUAL_NUM_FILTERS_TREES, baseDir);
@@ -104,53 +98,51 @@
 
         // Special case: sorting is not required
         if (allBTreeFiles.size() == 1 && (!hasBloomFilter || allBloomFilterFiles.size() == 1)) {
-            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).fileRef, null,
-                    hasBloomFilter ? allBloomFilterFiles.get(0).fileRef : null));
+            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).getFileRef(), null,
+                    hasBloomFilter ? allBloomFilterFiles.get(0).getFileRef() : null));
             return validFiles;
         }
 
-        // Sorts files names from earliest to latest timestamp.
+        // Sorts files names from earliest to latest sequence.
         Collections.sort(allBTreeFiles);
         if (hasBloomFilter) {
             Collections.sort(allBloomFilterFiles);
         }
 
-        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<>();
-        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastBTree = allBTreeFiles.get(0);
         validComparableBTreeFiles.add(lastBTree);
 
-        List<ComparableFileName> validComparableBloomFilterFiles = null;
-        ComparableFileName lastBloomFilter = null;
+        List<IndexComponentFileReference> validComparableBloomFilterFiles = null;
+        IndexComponentFileReference lastBloomFilter = null;
         if (hasBloomFilter) {
             validComparableBloomFilterFiles = new ArrayList<>();
             lastBloomFilter = allBloomFilterFiles.get(0);
             validComparableBloomFilterFiles.add(lastBloomFilter);
         }
 
-        ComparableFileName currentBTree;
-        ComparableFileName currentBloomFilter = null;
+        IndexComponentFileReference currentBTree;
+        IndexComponentFileReference currentBloomFilter = null;
         for (int i = 1; i < allBTreeFiles.size(); i++) {
             currentBTree = allBTreeFiles.get(i);
             if (hasBloomFilter) {
                 currentBloomFilter = allBloomFilterFiles.get(i);
             }
-            // Current start timestamp is greater than last stop timestamp.
-            if (currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
-                    && (!hasBloomFilter || currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0)) {
+            // Current start sequence is greater than last stop sequence.
+            if (currentBTree.isMoreRecentThan(lastBTree)
+                    && (!hasBloomFilter || currentBloomFilter.isMoreRecentThan(lastBloomFilter))) {
                 validComparableBTreeFiles.add(currentBTree);
                 lastBTree = currentBTree;
                 if (hasBloomFilter) {
                     validComparableBloomFilterFiles.add(currentBloomFilter);
                     lastBloomFilter = currentBloomFilter;
                 }
-            } else if (currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
-                    && currentBTree.interval[1].compareTo(lastBTree.interval[1]) <= 0
-                    && (!hasBloomFilter || (currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
-                            && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0))) {
+            } else if (currentBTree.isWithin(lastBTree)
+                    && (!hasBloomFilter || currentBloomFilter.isWithin(lastBloomFilter))) {
                 // Invalid files are completely contained in last interval.
-                IoUtil.delete(new File(currentBTree.fullPath));
+                delete(btreeFactory.getBufferCache(), currentBTree.getFullPath());
                 if (hasBloomFilter) {
-                    IoUtil.delete(new File(currentBloomFilter.fullPath));
+                    delete(btreeFactory.getBufferCache(), currentBloomFilter.getFullPath());
                 }
             } else {
                 // This scenario should not be possible.
@@ -161,21 +153,21 @@
         // Sort valid files in reverse lexicographical order, such that newer
         // files come first.
         Collections.sort(validComparableBTreeFiles, recencyCmp);
-        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
-        Iterator<ComparableFileName> bloomFilterFileIter = null;
+        Iterator<IndexComponentFileReference> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> bloomFilterFileIter = null;
         if (hasBloomFilter) {
             Collections.sort(validComparableBloomFilterFiles, recencyCmp);
             bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
         }
-        ComparableFileName cmpBTreeFileName = null;
-        ComparableFileName cmpBloomFilterFileName = null;
+        IndexComponentFileReference cmpBTreeFileName = null;
+        IndexComponentFileReference cmpBloomFilterFileName = null;
         while (btreeFileIter.hasNext() && (hasBloomFilter ? bloomFilterFileIter.hasNext() : true)) {
             cmpBTreeFileName = btreeFileIter.next();
             if (hasBloomFilter) {
                 cmpBloomFilterFileName = bloomFilterFileIter.next();
             }
-            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.fileRef, null,
-                    hasBloomFilter ? cmpBloomFilterFileName.fileRef : null));
+            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.getFileRef(), null,
+                    hasBloomFilter ? cmpBloomFilterFileName.getFileRef() : null));
         }
 
         return validFiles;
@@ -183,11 +175,10 @@
 
     @Override
     public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
-        String ts = getCurrentTimestamp();
+        String sequence = getNextComponentSequence(btreeFilter);
         // Create transaction lock file
-        IoUtil.create(baseDir.getChild(TXN_PREFIX + ts));
-        String baseName = ts + DELIMITER + ts;
-        // Begin timestamp and end timestamp are identical since it is a transaction
+        IoUtil.create(baseDir.getChild(TXN_PREFIX + sequence));
+        String baseName = getNextComponentSequence(btreeFilter);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX), null,
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
index 7acc59f..2ad4fee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -26,11 +26,13 @@
 
 public class LSMBTreeFlushOperation extends FlushOperation {
     private final FileReference bloomFilterFlushTarget;
+    private final LSMComponentFileReferences fileReferences;
 
     public LSMBTreeFlushOperation(ILSMIndexAccessor accessor, FileReference flushTarget,
             FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
         super(accessor, flushTarget, callback, indexIdentifier);
         this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+        fileReferences = new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
     }
 
     public FileReference getBloomFilterTarget() {
@@ -39,6 +41,6 @@
 
     @Override
     public LSMComponentFileReferences getComponentFiles() {
-        return new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
+        return fileReferences;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
index c7d555d..831562c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
@@ -28,9 +28,9 @@
 
     private final BTree btree;
 
-    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc, boolean isActive,
+    public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc,
             ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1cfc414..1312e30 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -32,7 +32,6 @@
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -40,7 +39,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 361612e..fb0b9ac0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -105,49 +105,55 @@
         while (!outputPriorityQueue.isEmpty() || needPushElementIntoQueue) {
             if (!outputPriorityQueue.isEmpty()) {
                 PriorityQueueElement queueHead = outputPriorityQueue.peek();
-                if (canCallProceed && includeMutableComponent) {
-                    resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
-                    if (!resultOfSearchCallbackProceed) {
-                        // In case proceed() fails and there is an in-memory component,
-                        // we can't simply use this element since there might be a change.
-                        PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
-                        if (mutableElement != null) {
-                            // Copies the current queue head
-                            if (tupleBuilder == null) {
-                                tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                if (canCallProceed) {
+                    if (includeMutableComponent) {
+                        resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
+                        if (!resultOfSearchCallbackProceed) {
+                            // In case proceed() fails and there is an in-memory component,
+                            // we can't simply use this element since there might be a change.
+                            PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
+                            if (mutableElement != null) {
+                                // Copies the current queue head
+                                if (tupleBuilder == null) {
+                                    tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                                }
+                                TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
+                                copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+                                // Unlatches/unpins the leaf page of the index.
+                                rangeCursors[0].close();
+                                // Reconcile.
+                                searchCallback.reconcile(copyTuple);
+                                // Re-traverses the index.
+                                reusablePred.setLowKey(copyTuple, true);
+                                btreeAccessors[0].search(rangeCursors[0], reusablePred);
+                                pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
+                                // now that we have completed the search and we have latches over the pages,
+                                // it is safe to complete the operation.. but as per the API of the callback
+                                // we only complete if we're producing this tuple
+                                // get head again
+                                queueHead = outputPriorityQueue.peek();
+                                /*
+                                 * We need to restart in one of two cases:
+                                 * 1. no more elements in the priority queue.
+                                 * 2. the key of the head has changed (which means we need to call proceed)
+                                 */
+                                if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
+                                    // cancel since we're not continuing
+                                    searchCallback.cancel(copyTuple);
+                                    continue;
+                                }
+                                searchCallback.complete(copyTuple);
+                                // it is safe to proceed now
+                            } else {
+                                // There are no more elements in the memory component.. can safely skip locking for the
+                                // remaining operations
+                                includeMutableComponent = false;
                             }
-                            TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
-                            copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-                            // Unlatches/unpins the leaf page of the index.
-                            rangeCursors[0].close();
-                            // Reconcile.
-                            searchCallback.reconcile(copyTuple);
-                            // Re-traverses the index.
-                            reusablePred.setLowKey(copyTuple, true);
-                            btreeAccessors[0].search(rangeCursors[0], reusablePred);
-                            pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
-                            // now that we have completed the search and we have latches over the pages,
-                            // it is safe to complete the operation.. but as per the API of the callback
-                            // we only complete if we're producing this tuple
-                            // get head again
-                            queueHead = outputPriorityQueue.peek();
-                            /*
-                             * We need to restart in one of two cases:
-                             * 1. no more elements in the priority queue.
-                             * 2. the key of the head has changed (which means we need to call proceed)
-                             */
-                            if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
-                                // cancel since we're not continuing
-                                searchCallback.cancel(copyTuple);
-                                continue;
-                            }
-                            searchCallback.complete(copyTuple);
-                            // it is safe to proceed now
-                        } else {
-                            // There are no more elements in the memory component.. can safely skip locking for the
-                            // remaining operations
-                            includeMutableComponent = false;
                         }
+                    } else {
+                        // only perform locking for tuples from memory components.
+                        // all tuples from disk components have already been committed, and we're safe to proceed
+                        resultOfSearchCallbackProceed = true;
                     }
                 }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index baf0d4a7..efacad1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -107,6 +107,6 @@
 
     @Override
     public boolean getSearchOperationCallbackProceedResult() {
-        return false;
+        return currentCursor.getSearchOperationCallbackProceedResult();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java
index f61d783..8fb3751 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyFileManager.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
-import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -36,6 +35,7 @@
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 
@@ -59,23 +59,15 @@
 
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
-        String ts = getCurrentTimestamp();
-        String baseName = ts + DELIMITER + ts;
-        // Begin timestamp and end timestamp are identical since it is a flush
+        String baseName = getNextComponentSequence(btreeFilter);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + DELETE_TREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
     }
 
     @Override
-    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
-            throws HyracksDataException {
-        String[] firstTimestampRange = firstFileName.split(DELIMITER);
-        String[] lastTimestampRange = lastFileName.split(DELIMITER);
-
-        String baseName = firstTimestampRange[0] + DELIMITER + lastTimestampRange[1];
-        // Get the range of timestamps by taking the earliest and the latest
-        // timestamps
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
+        final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + DELETE_TREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
@@ -84,23 +76,22 @@
     @Override
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException {
         List<LSMComponentFileReferences> validFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBuddyBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBuddyBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBloomFilterFiles = new ArrayList<>();
         // Create transaction file filter
         FilenameFilter transactionFilefilter = getTransactionFileFilter(false);
         // Gather files.
         cleanupAndGetValidFilesInternal(getCompoundFilter(btreeFilter, transactionFilefilter), btreeFactory,
-                allBTreeFiles);
+                allBTreeFiles, btreeFactory.getBufferCache());
         HashSet<String> btreeFilesSet = new HashSet<>();
-        for (ComparableFileName cmpFileName : allBTreeFiles) {
-            int index = cmpFileName.fileName.lastIndexOf(DELIMITER);
-            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+        for (IndexComponentFileReference cmpFileName : allBTreeFiles) {
+            btreeFilesSet.add(cmpFileName.getSequence());
         }
         validateFiles(btreeFilesSet, allBuddyBTreeFiles, getCompoundFilter(buddyBtreeFilter, transactionFilefilter),
-                buddyBtreeFactory);
+                buddyBtreeFactory, btreeFactory.getBufferCache());
         validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(bloomFilterFilter, transactionFilefilter),
-                null);
+                null, btreeFactory.getBufferCache());
         // Sanity check.
         if (allBTreeFiles.size() != allBuddyBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
             throw HyracksDataException.create(ErrorCode.UNEQUAL_NUM_FILTERS_TREES, baseDir);
@@ -110,52 +101,47 @@
             return validFiles;
         }
         if (allBTreeFiles.size() == 1 && allBuddyBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
-            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).fileRef,
-                    allBuddyBTreeFiles.get(0).fileRef, allBloomFilterFiles.get(0).fileRef));
+            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).getFileRef(),
+                    allBuddyBTreeFiles.get(0).getFileRef(), allBloomFilterFiles.get(0).getFileRef()));
             return validFiles;
         }
 
-        // Sorts files names from earliest to latest timestamp.
+        // Sorts files names from earliest to latest sequence.
         Collections.sort(allBTreeFiles);
         Collections.sort(allBuddyBTreeFiles);
         Collections.sort(allBloomFilterFiles);
 
-        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<>();
-        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastBTree = allBTreeFiles.get(0);
         validComparableBTreeFiles.add(lastBTree);
 
-        List<ComparableFileName> validComparableBuddyBTreeFiles = new ArrayList<>();
-        ComparableFileName lastBuddyBTree = allBuddyBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableBuddyBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastBuddyBTree = allBuddyBTreeFiles.get(0);
         validComparableBuddyBTreeFiles.add(lastBuddyBTree);
 
-        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<>();
-        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        List<IndexComponentFileReference> validComparableBloomFilterFiles = new ArrayList<>();
+        IndexComponentFileReference lastBloomFilter = allBloomFilterFiles.get(0);
         validComparableBloomFilterFiles.add(lastBloomFilter);
 
         for (int i = 1; i < allBTreeFiles.size(); i++) {
-            ComparableFileName currentBTree = allBTreeFiles.get(i);
-            ComparableFileName currentBuddyBTree = allBuddyBTreeFiles.get(i);
-            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
-            // Current start timestamp is greater than last stop timestamp.
-            if (currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
-                    && currentBuddyBTree.interval[0].compareTo(lastBuddyBTree.interval[1]) > 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+            IndexComponentFileReference currentBTree = allBTreeFiles.get(i);
+            IndexComponentFileReference currentBuddyBTree = allBuddyBTreeFiles.get(i);
+            IndexComponentFileReference currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start sequence is greater than last stop sequence
+            if (currentBTree.isMoreRecentThan(lastBTree) && currentBuddyBTree.isMoreRecentThan(lastBuddyBTree)
+                    && currentBloomFilter.isMoreRecentThan(lastBloomFilter)) {
                 validComparableBTreeFiles.add(currentBTree);
                 validComparableBuddyBTreeFiles.add(currentBuddyBTree);
                 validComparableBloomFilterFiles.add(currentBloomFilter);
                 lastBTree = currentBTree;
                 lastBuddyBTree = currentBuddyBTree;
                 lastBloomFilter = currentBloomFilter;
-            } else if (currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
-                    && currentBTree.interval[1].compareTo(lastBTree.interval[1]) <= 0
-                    && currentBuddyBTree.interval[0].compareTo(lastBuddyBTree.interval[0]) >= 0
-                    && currentBuddyBTree.interval[1].compareTo(lastBuddyBTree.interval[1]) <= 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
-                    && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
-                // Invalid files are completely contained in last interval.
-                IoUtil.delete(new File(currentBTree.fullPath));
-                IoUtil.delete(new File(currentBuddyBTree.fullPath));
-                IoUtil.delete(new File(currentBloomFilter.fullPath));
+            } else if (currentBTree.isWithin(lastBTree) && currentBuddyBTree.isWithin(lastBuddyBTree)
+                    && currentBloomFilter.isWithin(lastBloomFilter)) {
+                // Invalid files are completely contained in last sequence.
+                delete(treeFactory.getBufferCache(), currentBTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentBuddyBTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentBloomFilter.getFullPath());
             } else {
                 // This scenario should not be possible.
                 throw HyracksDataException.create(ErrorCode.FOUND_OVERLAPPING_LSM_FILES, baseDir);
@@ -164,19 +150,19 @@
 
         // Sort valid files in reverse lexicographical order, such that newer
         // files come first.
-        Collections.sort(validComparableBTreeFiles, recencyCmp);
-        Collections.sort(validComparableBuddyBTreeFiles, recencyCmp);
-        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+        validComparableBTreeFiles.sort(recencyCmp);
+        validComparableBuddyBTreeFiles.sort(recencyCmp);
+        validComparableBloomFilterFiles.sort(recencyCmp);
 
-        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
-        Iterator<ComparableFileName> buddyBtreeFileIter = validComparableBuddyBTreeFiles.iterator();
-        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        Iterator<IndexComponentFileReference> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> buddyBtreeFileIter = validComparableBuddyBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
         while (btreeFileIter.hasNext() && buddyBtreeFileIter.hasNext()) {
-            ComparableFileName cmpBTreeFileName = btreeFileIter.next();
-            ComparableFileName cmpBuddyBTreeFileName = buddyBtreeFileIter.next();
-            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
-            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.fileRef, cmpBuddyBTreeFileName.fileRef,
-                    cmpBloomFilterFileName.fileRef));
+            IndexComponentFileReference cmpBTreeFileName = btreeFileIter.next();
+            IndexComponentFileReference cmpBuddyBTreeFileName = buddyBtreeFileIter.next();
+            IndexComponentFileReference cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.getFileRef(),
+                    cmpBuddyBTreeFileName.getFileRef(), cmpBloomFilterFileName.getFileRef()));
         }
 
         return validFiles;
@@ -184,10 +170,9 @@
 
     @Override
     public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
-        String ts = getCurrentTimestamp();
         // Create transaction lock file
-        Files.createFile(Paths.get(baseDir + TXN_PREFIX + ts));
-        String baseName = ts + DELIMITER + ts;
+        String baseName = getNextComponentSequence(btreeFilter);
+        Files.createFile(Paths.get(baseDir + TXN_PREFIX + baseName));
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + DELETE_TREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
index a1a1e45..ed2ee70 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
@@ -20,10 +20,10 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 
 /*
  * This class is also not needed at the moment but is implemented anyway
@@ -34,8 +34,8 @@
     private final BTree buddyBtree;
 
     public LSMBTreeWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, BTree btree, BTree buddyBtree,
-            IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.btree = btree;
         this.buddyBtree = buddyBtree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index cc10a98..d071bac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -153,29 +153,24 @@
         // can be used for both inserts and deletes
         ITreeIndexFrameFactory transactionLeafFrameFactory =
                 new BTreeNSMLeafFrameFactory(transactionTupleWriterFactory);
-
         TreeIndexFactory<DiskBTree> diskBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
         TreeIndexFactory<DiskBTree> bulkLoadBTreeFactory = new DiskBTreeFactory(ioManager, diskBufferCache,
                 freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-
         BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-
         // This is the component factory for transactions
         TreeIndexFactory<DiskBTree> transactionBTreeFactory =
                 new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
                         transactionLeafFrameFactory, cmpFactories, typeTraits.length);
         //TODO remove BloomFilter from external dataset's secondary LSMBTree index
         ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, true);
-
         ILSMDiskComponentFactory componentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory bulkLoadComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
         ILSMDiskComponentFactory transactionComponentFactory =
                 new LSMBTreeWithBloomFilterDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
-
         // the disk only index uses an empty ArrayList for virtual buffer caches
         return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
                 diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
index 7d37f00..2ea822f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
@@ -90,5 +90,9 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
index 107190d..6b6e5e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public abstract class AbstractLSMWithBloomFilterDiskComponent extends AbstractLSMDiskComponent {
     public AbstractLSMWithBloomFilterDiskComponent(AbstractLSMIndex lsmIndex, IMetadataPageManager mdPageManager,
@@ -42,11 +43,11 @@
     public abstract IBufferCache getBloomFilterBufferCache();
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
+    public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
         // The order of forcing the dirty page to be flushed is critical. The
         // bloom filter must be always done first.
         ComponentUtils.markAsValid(getBloomFilterBufferCache(), getBloomFilter(), persist);
-        super.markAsValid(persist);
+        super.markAsValid(persist, callback);
     }
 
     @Override
@@ -59,13 +60,6 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        super.deactivateAndDestroy();
-        getBloomFilter().deactivate();
-        getBloomFilter().destroy();
-    }
-
-    @Override
     public void destroy() throws HyracksDataException {
         super.destroy();
         getBloomFilter().destroy();
@@ -78,9 +72,8 @@
     }
 
     @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        super.deactivateAndPurge();
-        getBloomFilter().deactivate();
+    protected void purge() throws HyracksDataException {
+        super.purge();
         getBloomFilter().purge();
     }
 
@@ -92,10 +85,10 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
-        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(fillFactor, verifyInput,
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
                 numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
         if (numElementsHint > 0) {
             chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
index c2f52e0..f7feb78 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public abstract class AbstractLSMWithBuddyDiskComponent extends AbstractLSMWithBloomFilterDiskComponent {
 
@@ -37,9 +38,9 @@
     public abstract AbstractTreeIndex getBuddyIndex();
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
-        super.markAsValid(persist);
-        ComponentUtils.markAsValid(getBuddyIndex(), persist);
+    public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
+        super.markAsValid(persist, callback);
+        ComponentUtils.markAsValid(getBuddyIndex(), persist, callback);
     }
 
     @Override
@@ -52,13 +53,6 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        super.deactivateAndDestroy();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-    }
-
-    @Override
     public void destroy() throws HyracksDataException {
         super.destroy();
         getBuddyIndex().destroy();
@@ -71,9 +65,8 @@
     }
 
     @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        super.deactivateAndPurge();
-        getBuddyIndex().deactivate();
+    protected void purge() throws HyracksDataException {
+        super.purge();
         getBuddyIndex().purge();
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
deleted file mode 100644
index 551303f..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.api;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-
-public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
-
-    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
-    }
-
-    public abstract AbstractTreeIndex getBuddyIndex();
-
-    @Override
-    public void doReset() throws HyracksDataException {
-        super.doReset();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-        getBuddyIndex().create();
-        getBuddyIndex().activate();
-    }
-
-    @Override
-    public void doAllocate() throws HyracksDataException {
-        super.doAllocate();
-        getBuddyIndex().create();
-        getBuddyIndex().activate();
-    }
-
-    @Override
-    public void doDeallocate() throws HyracksDataException {
-        super.doDeallocate();
-        getBuddyIndex().deactivate();
-        getBuddyIndex().destroy();
-    }
-
-    @Override
-    public void validate() throws HyracksDataException {
-        super.validate();
-        getBuddyIndex().validate();
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
index df78c53..78e5862 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
@@ -33,4 +33,11 @@
      * @throws HyracksDataException
      */
     void frameCompleted() throws HyracksDataException;
+
+    /**
+     * Called when the task has failed.
+     *
+     * @param th
+     */
+    void fail(Throwable th);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameTupleProcessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameTupleProcessor.java
index 3fbe6cd..b6192c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameTupleProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameTupleProcessor.java
@@ -43,4 +43,11 @@
      * Called once per batch before ending the batch process
      */
     void finish() throws HyracksDataException;
+
+    /**
+     * Called when a failure is encountered processing a frame
+     *
+     * @param th
+     */
+    void fail(Throwable th);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
new file mode 100644
index 0000000..a9b9cc0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.api;
+
+/**
+ * A callback that will be called on failures of IO operations
+ */
+public interface IIoOperationFailedCallback {
+    /**
+     * Called on a failed IO operation
+     *
+     * @param operation
+     *            the failed operation
+     * @param failure
+     *            the failure
+     */
+    void operationFailed(ILSMIOOperation operation, Throwable failure);
+
+    /**
+     * Called on a scheduler failure.
+     *
+     * @param scheduler
+     *            the failed scheduler
+     * @param failure
+     *            the failure
+     */
+    void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index fc55ce5..fef59e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.common.IIndex;
 
 /**
@@ -139,4 +140,17 @@
      * @throws HyracksDataException
      */
     ILSMComponentId getId() throws HyracksDataException;
+
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException;
+
+    /**
+     * @return the number of readers inside a component
+     */
+    int getReaderCount();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
index 5662862..c3835eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
@@ -38,4 +38,14 @@
     boolean missing();
 
     IdCompareResult compareTo(ILSMComponentId id);
+
+    /**
+     * @return the min Id
+     */
+    long getMinId();
+
+    /**
+     * @return the max Id
+     */
+    long getMaxId();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
index 5dd3061..a5c6360 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
@@ -26,12 +26,16 @@
     /**
      * @return An Id for LSM component
      */
-    public ILSMComponentId getId();
+    ILSMComponentId getId();
 
     /**
      * Refresh the component Id generator to generate the next Id.
      * {@link #getId()} would always return the same Id before this method is called.
      */
-    public void refresh();
+    void refresh();
 
+    /**
+     * @return the index of the current memory component
+     */
+    int getCurrentComponentIndex();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
deleted file mode 100644
index 4ec82c1..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.api;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.common.IResource;
-
-@FunctionalInterface
-public interface ILSMComponentIdGeneratorFactory extends Serializable {
-    ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
-            throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
index bd2bb45..543779c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
 import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public interface ILSMDiskComponent extends ILSMComponent {
 
@@ -50,6 +51,7 @@
     /**
      * @return LsmIndex of the component
      */
+    @Override
     AbstractLSMIndex getLsmIndex();
 
     /**
@@ -67,9 +69,11 @@
      *
      * @param persist
      *            whether the call should force data to disk before returning
+     * @param callback
+     *            callback for when a page write operation fails
      * @throws HyracksDataException
      */
-    void markAsValid(boolean persist) throws HyracksDataException;
+    void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException;
 
     /**
      * Activates the component
@@ -142,6 +146,7 @@
      * Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
      * component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
      *
+     * @param operation
      * @param fillFactor
      * @param verifyInput
      * @param numElementsHint
@@ -151,6 +156,7 @@
      * @return
      * @throws HyracksDataException
      */
-    ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-            boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent) throws HyracksDataException;
+    ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+            throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
index 13ce971..8284cee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
@@ -41,4 +41,9 @@
      */
     void cleanupArtifacts() throws HyracksDataException;
 
+    /**
+     * @return The operation this bulkLoader is operating under
+     */
+    ILSMIOOperation getOperation();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index b32dd0f..9e8c568 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -108,7 +108,7 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Schedule full merge
@@ -118,17 +118,16 @@
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a merge operation
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void merge(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Schedule a flush
@@ -137,27 +136,26 @@
      * @param callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     /**
      * Perform a flush
      *
-     * @param ctx
      * @param operation
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+    void flush(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * Add bulk loaded component
      *
-     * @param index
-     *            the new component
+     * @param ioOperation
+     *            the io operation that added the new component
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void addBulkLoadedComponent(ILSMDiskComponent index) throws HyracksDataException;
+    void addBulkLoadedComponent(ILSMIOOperation ioOperation) throws HyracksDataException;
 
     /**
      * Get index operation tracker
@@ -171,13 +169,11 @@
      *            the operation context
      * @param diskComponents
      *            the disk component to be replicated
-     * @param bulkload
-     *            true if the components were bulk loaded, false otherwise
      * @param opType
      *            The operation type
      * @throws HyracksDataException
      */
-    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
             LSMOperationType opType) throws HyracksDataException;
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 65e7f64..0e13933 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -18,14 +18,17 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.api;
 
+import java.util.Map;
 import java.util.concurrent.Callable;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
-public interface ILSMIOOperation extends Callable<Boolean> {
+public interface ILSMIOOperation extends Callable<LSMIOOperationStatus>, IPageWriteFailureCallback {
 
     /**
      * Represents the io operation type
@@ -38,6 +41,20 @@
     }
 
     /**
+     * Represents the status of the IO operation
+     */
+    enum LSMIOOperationStatus {
+        /**
+         * Operation successful
+         */
+        SUCCESS,
+        /**
+         * Operation failed
+         */
+        FAILURE
+    }
+
+    /**
      * @return the device on which the operation is running
      */
     IODeviceHandle getDevice();
@@ -58,7 +75,7 @@
     LSMIOOperationType getIOOpertionType();
 
     @Override
-    Boolean call() throws HyracksDataException;
+    LSMIOOperationStatus call() throws HyracksDataException;
 
     /**
      * @return The target of the io operation
@@ -74,4 +91,63 @@
      * @return the component files produced by this operation
      */
     LSMComponentFileReferences getComponentFiles();
+
+    /**
+     * @return the failure in the io operation if any, null otherwise
+     */
+    @Override
+    Throwable getFailure();
+
+    /**
+     * @return set the failure in the io operation
+     */
+    void setFailure(Throwable failure);
+
+    /**
+     * @return the status of the IO operation
+     */
+    LSMIOOperationStatus getStatus();
+
+    /**
+     * Set the status of the IO operation
+     *
+     * @param status
+     */
+    void setStatus(LSMIOOperationStatus status);
+
+    /**
+     * @return the new component produced by this operation if any, null otherwise
+     */
+    ILSMDiskComponent getNewComponent();
+
+    /**
+     * Set the new component produced by this operation
+     *
+     * @param component
+     */
+    void setNewComponent(ILSMDiskComponent component);
+
+    /**
+     * Destroy the operation after the scheduler is done with it
+     */
+    void complete();
+
+    /**
+     * Wait for the operation to complete
+     *
+     * @throws InterruptedException
+     */
+    void sync() throws InterruptedException;
+
+    /**
+     * Add a listener for operation complete event
+     *
+     * @param listener
+     */
+    void addCompleteListener(IoOperationCompleteListener listener);
+
+    /**
+     * Get parameters passed when calling this IO operation
+     */
+    Map<String, Object> getParameters();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index acc9e89..e448ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -23,14 +23,23 @@
 public interface ILSMIOOperationCallback {
 
     /**
-     * This method is called on an IO operation before the operation starts.
-     * (i.e. IO operations could be flush, or merge operations.)
-     * For flush, this is called immediately before switching the current memory component pointer
+     * This method is called on an IO operation before the operation is scheduled
+     * For operations that are not scheduled(i,e. Bulk load), this call is skipped.
+     *
+     * @param operation
+     * @throws HyracksDataException
      */
-    void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void scheduled(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
-     * This method is called on an IO operation sometime after the operation was completed.
+     * This method is called on an IO operation before the operation starts.
+     * (i.e. IO operations could be flush, or merge operations.)
+     */
+    void beforeOperation(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called on an IO operation sometime after the operation is completed but before the new component
+     * is marked as valid.
      * (i.e. IO operations could be flush or merge operations.)
      *
      * Copying content of metadata page from memory component to disk component should be done in this call
@@ -38,31 +47,37 @@
      *
      * @throws HyracksDataException
      */
-    void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void afterOperation(ILSMIOOperation operation) throws HyracksDataException;
 
     /**
      * This method is called on an IO operation when the operation needs any cleanup works
      * regardless that the IO operation was executed or not. Once the IO operation is executed,
      * this method should be called after ILSMIOOperationCallback.afterOperation() was called.
      *
+     */
+    void afterFinalize(ILSMIOOperation operation) throws HyracksDataException;
+
+    /**
+     * This method is called after the schduler is done with the IO operation
+     * For operation that are not scheduled, this call is skipped
+     *
+     * @param operation
      * @throws HyracksDataException
      */
-    void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+    void completed(ILSMIOOperation operation);
 
     /**
      * This method is called when a memory component is recycled
      *
      * @param component
-     * @param componentSwitched
-     *            true if the component index was advanced for this recycle, false otherwise
      */
-    void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException;
+    void recycled(ILSMMemoryComponent component) throws HyracksDataException;
 
     /**
      * This method is called when a memory component is allocated
      *
      * @param component
+     *            the allocated component
      */
     void allocated(ILSMMemoryComponent component) throws HyracksDataException;
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index e8742b5..b758231 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -22,9 +22,10 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.storage.common.IResource;
 
-public interface ILSMIOOperationCallbackFactory extends Serializable {
+public interface ILSMIOOperationCallbackFactory extends Serializable, IJsonSerializable {
     /**
      * Initialize the callback factory with the given ncCtx and resource
      *
@@ -32,5 +33,18 @@
      */
     void initialize(INCServiceContext ncCtx, IResource resource);
 
+    /**
+     * Create the IO Operation Callback
+     *
+     * @param index
+     * @return
+     * @throws HyracksDataException
+     */
     ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException;
+
+    /**
+     * @return the current memory component index
+     * @throws HyracksDataException
+     */
+    int getCurrentMemoryComponentIndex() throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
index 486008d..79235cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
@@ -21,11 +21,12 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.io.IJsonSerializable;
 
 /**
  * Provides the {@link ILSMIOOperationScheduler} for the application
  */
 @FunctionalInterface
-public interface ILSMIOOperationSchedulerProvider extends Serializable {
+public interface ILSMIOOperationSchedulerProvider extends Serializable, IJsonSerializable {
     ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index a8467d3..5cb05a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
@@ -28,6 +29,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMHarness;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 
@@ -48,8 +50,6 @@
 
     ILSMOperationTracker getOperationTracker();
 
-    ILSMIOOperationScheduler getIOScheduler();
-
     ILSMIOOperationCallback getIOOperationCallback();
 
     /**
@@ -74,11 +74,33 @@
 
     public void scanDiskComponents(ILSMIndexOperationContext ctx, IIndexCursor cursor) throws HyracksDataException;
 
-    void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a flush operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @return the flush operation
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException;
 
-    void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+    /**
+     * Create a merge operation.
+     * This is an atomic operation. If an exception is thrown, no partial effect is left
+     *
+     * @param ctx
+     *            the operation context
+     * @param callback
+     *            the IO callback
+     * @throws HyracksDataException
+     */
+    ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
 
     ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException;
 
@@ -108,7 +130,7 @@
 
     boolean isCurrentMutableComponentEmpty() throws HyracksDataException;
 
-    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+    void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
             ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException;
 
     boolean isMemoryComponentsAllocated();
@@ -157,4 +179,36 @@
      * @return the {@link ILSMHarness} of the index
      */
     ILSMHarness getHarness();
+
+    /**
+     * Cleanup the files of the failed operation
+     *
+     * @param operation
+     */
+    void cleanUpFilesForFailedOperation(ILSMIOOperation operation);
+
+    /**
+     * @return the absolute path of the index
+     */
+    String getIndexIdentifier();
+
+    /**
+     * Create a bulk loader
+     *
+     * @param fillFactor
+     * @param verifyInput
+     * @param numElementsHint
+     * @param checkIfEmptyIndex
+     * @param parameters
+     * @return
+     * @throws HyracksDataException
+     */
+    IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException;
+
+    /**
+     * Reset the current memory component id to 0.
+     */
+    void resetCurrentComponentIndex();
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index 61ef6cf..42d3ab7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -44,34 +44,27 @@
     /**
      * Schedule a flush operation
      *
-     * @param callback
-     *            the IO operation callback
      * @throws HyracksDataException
      */
-    void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFlush() throws HyracksDataException;
 
     /**
      * Schedule a merge operation
      *
-     * @param callback
-     *            the merge operation callback
      * @param components
      *            the components to be merged
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException;
+    ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException;
 
     /**
      * Schedule a full merge
      *
-     * @param callback
-     *            the merge operation callback
      * @throws HyracksDataException
      * @throws IndexException
      */
-    void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException;
+    ILSMIOOperation scheduleFullMerge() throws HyracksDataException;
 
     /**
      * Delete the tuple from the memory component only. Don't replace with antimatter tuple
@@ -193,17 +186,15 @@
      *
      * @param diskComponents
      *            the components to be replicated
-     * @param bulkload
-     *            true if the components were bulkloaded, false otherwise
      * @param opType
      *            the operation type
      * @throws HyracksDataException
      */
-    void scheduleReplication(List<ILSMDiskComponent> diskComponents, boolean bulkload, LSMOperationType opType)
+    void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
             throws HyracksDataException;
 
     /**
-     * Force a flush of the in-memory component.
+     * Flush an in-memory component.
      *
      * @throws HyracksDataException
      * @throws TreeIndexException
@@ -265,9 +256,10 @@
     void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException;
 
     /**
-    * Update the filter of an LSM index
-    * @param tuple
-    * @throws HyracksDataException
-    */
+     * Update the filter of an LSM index
+     *
+     * @param tuple
+     * @throws HyracksDataException
+     */
     void updateFilter(ITupleReference tuple) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
index b34b403..238d5f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -19,12 +19,11 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -94,26 +93,26 @@
     void setRecovery(boolean recovery);
 
     /**
-     * @return the IO Operation type associated with this context
+     * @return the IO operation associated with this context
      */
-    LSMIOOperationType getIoOperationType();
+    ILSMIOOperation getIoOperation();
 
     /**
-     * Set the IO Operation type associated with this context
+     * Set the IO operation associated with this context
      *
-     * @param ioOpType
+     * @param ioOperation
      */
-    void setIoOperationType(LSMIOOperationType ioOpType);
+    void setIoOperation(ILSMIOOperation ioOperation);
 
     /**
-     * @return the new component produced by this operation if any, null otherwise
-     */
-    ILSMDiskComponent getNewComponent();
-
-    /**
-     * Set the new component produced by this operation
+     * Set a map in the context to pass pairs of keys and values
      *
-     * @param component
+     * @param map
      */
-    void setNewComponent(ILSMDiskComponent component);
+    void setParameters(Map<String, Object> map);
+
+    /**
+     * @return the key value map of the context
+     */
+    Map<String, Object> getParameters();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
index 4ff6377..c86f7b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
@@ -63,19 +63,6 @@
     void setModified();
 
     /**
-     * request the component to be active
-     */
-    void requestActivation();
-
-    /**
-     * Set the component state
-     *
-     * @param state
-     *            the new state
-     */
-    void setState(ComponentState state);
-
-    /**
      * Allocates memory to this component, create and activate it.
      * This method is atomic. If an exception is thrown, then the call had no effect.
      *
@@ -108,8 +95,14 @@
      *
      * @param newId
      * @param force
-     *      Whether to force reset the Id to skip sanity checks
+     *            Whether to force reset the Id to skip sanity checks
      * @throws HyracksDataException
      */
     void resetId(ILSMComponentId newId, boolean force) throws HyracksDataException;
+
+    /**
+     * Set the component state to be unwritable to prevent future writers from non-force
+     * entry to the component
+     */
+    void setUnwritable();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMergePolicyFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMergePolicyFactory.java
index 71e33c1..9424fdd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMergePolicyFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMergePolicyFactory.java
@@ -23,8 +23,9 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.io.IJsonSerializable;
 
-public interface ILSMMergePolicyFactory extends Serializable {
+public interface ILSMMergePolicyFactory extends Serializable, IJsonSerializable {
     ILSMMergePolicy createMergePolicy(Map<String, String> configuration, INCServiceContext ctx);
 
     String getName();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
index ef22620..9f85d6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
@@ -22,9 +22,10 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.storage.common.IResource;
 
 @FunctionalInterface
-public interface ILSMOperationTrackerFactory extends Serializable {
+public interface ILSMOperationTrackerFactory extends Serializable, IJsonSerializable {
     ILSMOperationTracker getOperationTracker(INCServiceContext ctx, IResource resource) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
index fc1d5f4..74a5d60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.common.api;
 
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -30,8 +31,8 @@
      * This function is used to create a BulkLoader for a transaction that is capable of insertions and deletions
      * and the bulk loaded component is hidden from the index
      */
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException;
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException;
 
     /**
      * This function is used to commit the previous transaction if it was resulted in creating any components
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
index 8510e8b..802d44d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IVirtualBufferCacheProvider.java
@@ -24,9 +24,10 @@
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IJsonSerializable;
 
 @FunctionalInterface
-public interface IVirtualBufferCacheProvider extends Serializable {
+public interface IVirtualBufferCacheProvider extends Serializable, IJsonSerializable {
     List<IVirtualBufferCache> getVirtualBufferCaches(INCServiceContext ctx, FileReference fileRef)
             throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
new file mode 100644
index 0000000..e5ba81e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.api;
+
+public interface IoOperationCompleteListener {
+
+    /**
+     * Called when an IO operation completes
+     *
+     * @param operation
+     */
+    void completed(ILSMIOOperation operation);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
index 63d2697..e200bfd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
@@ -22,6 +22,7 @@
     SEARCH,
     MODIFICATION,
     FORCE_MODIFICATION,
+    LOAD,
     FLUSH,
     MERGE,
     REPLICATE,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
index 4e8cf71..682ffef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
@@ -57,7 +57,7 @@
         indexHelper.open();
         ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
         ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFullMerge(index.getIOOperationCallback());
+        accessor.scheduleFullMerge();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
index b541750..7126a38 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
@@ -18,10 +18,14 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.dataflow;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -32,14 +36,19 @@
 import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.LocalResource;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
- * TODO(amoudi): Change this class and its subclasses to use json serialization instead of Java serialization
  * The base resource that will be written to disk. it will go in the serializable resource
  * member in {@link LocalResource}
  */
 public abstract class LsmResource implements IResource {
 
     private static final long serialVersionUID = 1L;
+    protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
     protected String path;
     protected final IStorageManager storageManager;
     protected final ITypeTraits[] typeTraits;
@@ -80,6 +89,111 @@
         this.durable = durable;
     }
 
+    protected LsmResource(IPersistedResourceRegistry registry, JsonNode json) throws HyracksDataException {
+        path = json.get("path").asText();
+        storageManager = (IStorageManager) registry.deserialize(json.get("storageManager"));
+        final List<ITypeTraits> typeTraitsList = new ArrayList<>();
+        final ArrayNode jsonTypeTraits = (ArrayNode) json.get("typeTraits");
+        for (JsonNode tt : jsonTypeTraits) {
+            typeTraitsList.add((ITypeTraits) registry.deserialize(tt));
+        }
+        typeTraits = typeTraitsList.toArray(new ITypeTraits[0]);
+
+        final List<IBinaryComparatorFactory> cmpFactoriesList = new ArrayList<>();
+        final ArrayNode jsonCmpFactories = (ArrayNode) json.get("cmpFactories");
+        for (JsonNode cf : jsonCmpFactories) {
+            cmpFactoriesList.add((IBinaryComparatorFactory) registry.deserialize(cf));
+        }
+        cmpFactories = cmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
+
+        if (json.hasNonNull("filterTypeTraits")) {
+            final List<ITypeTraits> filterTypeTraitsList = new ArrayList<>();
+            final ArrayNode jsonFilterTypeTraits = (ArrayNode) json.get("filterTypeTraits");
+            for (JsonNode tt : jsonFilterTypeTraits) {
+                filterTypeTraitsList.add((ITypeTraits) registry.deserialize(tt));
+            }
+            filterTypeTraits = filterTypeTraitsList.toArray(new ITypeTraits[0]);
+        } else {
+            filterTypeTraits = null;
+        }
+
+        if (json.hasNonNull("filterCmpFactories")) {
+            final List<IBinaryComparatorFactory> filterCmpFactoriesList = new ArrayList<>();
+            final ArrayNode jsonFilterCmpFactories = (ArrayNode) json.get("filterCmpFactories");
+            for (JsonNode cf : jsonFilterCmpFactories) {
+                filterCmpFactoriesList.add((IBinaryComparatorFactory) registry.deserialize(cf));
+            }
+            filterCmpFactories = filterCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
+        } else {
+            filterCmpFactories = null;
+        }
+
+        filterFields = OBJECT_MAPPER.convertValue(json.get("filterFields"), int[].class);
+        opTrackerProvider = (ILSMOperationTrackerFactory) registry.deserialize(json.get("opTrackerProvider"));
+        ioOpCallbackFactory = (ILSMIOOperationCallbackFactory) registry.deserialize(json.get("ioOpCallbackFactory"));
+
+        metadataPageManagerFactory =
+                (IMetadataPageManagerFactory) registry.deserialize(json.get("metadataPageManagerFactory"));
+        if (json.hasNonNull("vbcProvider")) {
+            vbcProvider = (IVirtualBufferCacheProvider) registry.deserialize(json.get("vbcProvider"));
+        } else {
+            vbcProvider = null;
+        }
+        ioSchedulerProvider = (ILSMIOOperationSchedulerProvider) registry.deserialize(json.get("ioSchedulerProvider"));
+        mergePolicyFactory = (ILSMMergePolicyFactory) registry.deserialize(json.get("mergePolicyFactory"));
+        mergePolicyProperties = OBJECT_MAPPER.convertValue(json.get("mergePolicyProperties"), Map.class);
+        durable = json.get("durable").asBoolean();
+    }
+
+    protected void appendToJson(final ObjectNode json, IPersistedResourceRegistry registry)
+            throws HyracksDataException {
+        json.put("path", path);
+        json.set("storageManager", storageManager.toJson(registry));
+        ArrayNode ttArray = OBJECT_MAPPER.createArrayNode();
+        for (ITypeTraits tt : typeTraits) {
+            ttArray.add(tt.toJson(registry));
+        }
+        json.set("typeTraits", ttArray);
+
+        ArrayNode cmpArray = OBJECT_MAPPER.createArrayNode();
+        for (IBinaryComparatorFactory factory : cmpFactories) {
+            cmpArray.add(factory.toJson(registry));
+        }
+        json.set("cmpFactories", cmpArray);
+
+        if (filterTypeTraits != null) {
+            ArrayNode fttArray = OBJECT_MAPPER.createArrayNode();
+            for (ITypeTraits tt : filterTypeTraits) {
+                fttArray.add(tt.toJson(registry));
+            }
+            json.set("filterTypeTraits", fttArray);
+        } else {
+            json.set("filterTypeTraits", null);
+        }
+
+        if (filterCmpFactories != null) {
+            ArrayNode filterCmpArray = OBJECT_MAPPER.createArrayNode();
+            for (IBinaryComparatorFactory factory : filterCmpFactories) {
+                filterCmpArray.add(factory.toJson(registry));
+            }
+            json.set("filterCmpFactories", filterCmpArray);
+        } else {
+            json.set("filterCmpFactories", null);
+        }
+
+        json.putPOJO("filterFields", filterFields);
+        json.set("opTrackerProvider", opTrackerProvider.toJson(registry));
+        json.set("ioOpCallbackFactory", ioOpCallbackFactory.toJson(registry));
+        json.set("metadataPageManagerFactory", metadataPageManagerFactory.toJson(registry));
+        if (vbcProvider != null) {
+            json.set("vbcProvider", vbcProvider.toJson(registry));
+        }
+        json.set("ioSchedulerProvider", ioSchedulerProvider.toJson(registry));
+        json.set("mergePolicyFactory", mergePolicyFactory.toJson(registry));
+        json.putPOJO("mergePolicyProperties", mergePolicyProperties);
+        json.put("durable", durable);
+    }
+
     @Override
     public String getPath() {
         return path;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
index 08c75dc..9d62c0d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/freepage/VirtualFreePageManager.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public class VirtualFreePageManager implements IPageManager {
@@ -88,7 +89,7 @@
     }
 
     @Override
-    public void close() {
+    public void close(IPageWriteFailureCallback callback) {
         // Method doesn't make sense for this free page manager.
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
index aee46f0..3d76755 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
@@ -18,11 +18,19 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public abstract class AbstractIoOperation implements ILSMIOOperation {
 
@@ -30,6 +38,11 @@
     protected final FileReference target;
     protected final ILSMIOOperationCallback callback;
     protected final String indexIdentifier;
+    private volatile Throwable failure;
+    private LSMIOOperationStatus status = LSMIOOperationStatus.SUCCESS;
+    private ILSMDiskComponent newComponent;
+    private boolean completed = false;
+    private List<IoOperationCompleteListener> completeListeners;
 
     public AbstractIoOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
             String indexIdentifier) {
@@ -63,4 +76,85 @@
     public String getIndexIdentifier() {
         return indexIdentifier;
     }
+
+    @Override
+    public Throwable getFailure() {
+        return failure;
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        status = LSMIOOperationStatus.FAILURE;
+        this.failure = ExceptionUtils.suppress(this.failure, failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return status;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        this.status = status;
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return newComponent;
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        this.newComponent = component;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (completed) {
+            throw new IllegalStateException("Multiple destroy calls");
+        }
+        callback.completed(this);
+        completed = true;
+        if (completeListeners != null) {
+            for (IoOperationCompleteListener listener : completeListeners) {
+                listener.completed(this);
+            }
+            completeListeners = null;
+        }
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void sync() throws InterruptedException {
+        while (!completed) {
+            wait();
+        }
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return accessor.getOpContext().getParameters();
+    }
+
+    @Override
+    public synchronized void addCompleteListener(IoOperationCompleteListener listener) {
+        if (completed) {
+            listener.completed(this);
+        } else {
+            if (completeListeners == null) {
+                completeListeners = new LinkedList<>();
+            }
+            completeListeners.add(listener);
+        }
+    }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        setFailure(failure);
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return status == LSMIOOperationStatus.FAILURE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
index 84d2fe5..574a371 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
@@ -51,4 +51,9 @@
     public final AbstractLSMIndex getLsmIndex() {
         return lsmIndex;
     }
+
+    @Override
+    public int getReaderCount() {
+        return readerCount;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index c4616d3..1cad00f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -24,11 +24,13 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 import org.apache.hyracks.storage.common.MultiComparator;
-import org.apache.logging.log4j.Level;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -52,27 +54,34 @@
     }
 
     @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        if (ioOperationType != LSMIOOperationType.MERGE) {
+            throw new IllegalStateException("Unsupported operation type: " + ioOperationType);
+        }
+        if (state == ComponentState.INACTIVE) {
+            throw new IllegalStateException("Trying to schedule a merge of an inactive disk component");
+        }
+        if (state == ComponentState.READABLE_MERGING) {
+            // This should never happen unless there are two concurrent merges that were scheduled
+            // concurrently and they have interleaving components to be merged.
+            // This should be handled properly by the merge policy, but we guard against that here anyway.
+            throw new IllegalStateException("The disk component has already been scheduled for a merge");
+        }
+        state = ComponentState.READABLE_MERGING;
+    }
+
+    @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
         if (state == ComponentState.INACTIVE) {
             throw new IllegalStateException("Trying to enter an inactive disk component");
         }
-
         switch (opType) {
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
-                readerCount++;
-                break;
             case MERGE:
-                if (state == ComponentState.READABLE_MERGING) {
-                    // This should never happen unless there are two concurrent merges that were scheduled
-                    // concurrently and they have interleaving components to be merged.
-                    // This should be handled properly by the merge policy, but we guard against that here anyway.
-                    return false;
-                }
-                state = ComponentState.READABLE_MERGING;
                 readerCount++;
                 break;
             default:
@@ -86,19 +95,22 @@
             throws HyracksDataException {
         switch (opType) {
             case MERGE:
+                readerCount--;
                 // In case two merge operations were scheduled to merge an overlapping set of components,
                 // the second merge will fail and it must reset those components back to their previous state.
                 if (failedOperation) {
                     state = ComponentState.READABLE_UNWRITABLE;
+                } else {
+                    state = (readerCount == 0) ? ComponentState.INACTIVE : ComponentState.UNREADABLE_UNWRITABLE;
                 }
-                // Fallthrough
+                break;
             case FORCE_MODIFICATION:
             case MODIFICATION:
             case REPLICATE:
             case SEARCH:
             case DISK_COMPONENT_SCAN:
                 readerCount--;
-                if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+                if (readerCount == 0 && state == ComponentState.UNREADABLE_UNWRITABLE) {
                     state = ComponentState.INACTIVE;
                 }
                 break;
@@ -144,10 +156,10 @@
      * @throws HyracksDataException
      */
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
-        ComponentUtils.markAsValid(getMetadataHolder(), persist);
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO, "Marked as valid component with id: " + getId());
+    public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
+        ComponentUtils.markAsValid(getMetadataHolder(), persist, callback);
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Marked as valid component with id: " + getId());
         }
     }
 
@@ -163,9 +175,15 @@
     }
 
     @Override
-    public void deactivateAndDestroy() throws HyracksDataException {
-        getIndex().deactivate();
-        getIndex().destroy();
+    public final void deactivateAndDestroy() throws HyracksDataException {
+        deactivateAndPurge();
+        destroy();
+    }
+
+    @Override
+    public final void deactivateAndPurge() throws HyracksDataException {
+        deactivate();
+        purge();
     }
 
     @Override
@@ -178,9 +196,7 @@
         getIndex().deactivate();
     }
 
-    @Override
-    public void deactivateAndPurge() throws HyracksDataException {
-        getIndex().deactivate();
+    protected void purge() throws HyracksDataException {
         getIndex().purge();
     }
 
@@ -203,17 +219,27 @@
                 getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
     }
 
+    /**
+     * Allows sub-class extend this method to use specialized bulkloader for merge
+     */
+    protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        return this.createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+    }
+
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
         ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
-                new ChainedLSMDiskComponentBulkLoader(this, cleanupEmptyComponent);
+                new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
         if (withFilter && getLsmIndex().getFilterFields() != null) {
             chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
         }
-        chainedBulkLoader
-                .addBulkLoader(createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+        IChainedComponentBulkLoader indexBulkloader = operation.getIOOpertionType() == LSMIOOperationType.MERGE
+                ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex)
+                : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        chainedBulkLoader.addBulkLoader(indexBulkloader);
         return chainedBulkLoader;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 2db2f79..9199fbb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -21,9 +21,11 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -41,6 +43,7 @@
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
@@ -74,7 +77,6 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final ILSMHarness lsmHarness;
     protected final IIOManager ioManager;
-    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ILSMIOOperationCallback ioOpCallback;
 
     // In-memory components.
@@ -95,6 +97,7 @@
     protected final int[] filterFields;
     protected final boolean durable;
     protected boolean isActive;
+    protected volatile boolean isDeactivating = false;
     protected final AtomicBoolean[] flushRequests;
     protected volatile boolean memoryComponentsAllocated = false;
     protected ITracer tracer;
@@ -102,6 +105,7 @@
     protected final ILSMDiskComponentFactory componentFactory;
     // Factory for creating on-disk index components during bulkload.
     protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
+    private int numScheduledFlushes = 0;
 
     public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
@@ -115,7 +119,6 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
@@ -127,11 +130,12 @@
         this.inactiveDiskComponents = new LinkedList<>();
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new LSMHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(), tracer);
+        lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
+                tracer);
         isActive = false;
         diskComponents = new ArrayList<>();
         memoryComponents = new ArrayList<>();
-        currentMutableComponentId = new AtomicInteger();
+        currentMutableComponentId = new AtomicInteger(ioOpCallbackFactory.getCurrentMemoryComponentIndex());
         flushRequests = new AtomicBoolean[virtualBufferCaches.size()];
         for (int i = 0; i < virtualBufferCaches.size(); i++) {
             flushRequests[i] = new AtomicBoolean();
@@ -148,13 +152,13 @@
         this.diskBufferCache = diskBufferCache;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
-        this.ioScheduler = ioScheduler;
         this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
         this.componentFactory = componentFactory;
         this.bulkLoadComponentFactory = bulkLoadComponentFactory;
         this.durable = durable;
         this.tracer = tracer;
-        lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled());
+        lsmHarness = new ExternalIndexHarness(this, ioScheduler, mergePolicy, opTracker,
+                diskBufferCache.isReplicationEnabled());
         isActive = false;
         diskComponents = new LinkedList<>();
         this.inactiveDiskComponents = new LinkedList<>();
@@ -205,28 +209,35 @@
         deactivate(true);
     }
 
+    @SuppressWarnings({ "squid:S1181", "squid:S2142" })
     @Override
     public synchronized void deactivate(boolean flush) throws HyracksDataException {
         if (!isActive) {
             throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_INDEX);
         }
-        if (flush) {
-            flushMemoryComponent();
-        }
-        deactivateDiskComponents();
-        deallocateMemoryComponents();
-        isActive = false;
-    }
-
-    private void flushMemoryComponent() throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
-        ILSMIndexAccessor accessor = createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleFlush(cb);
+        // The following member is used to prevent scheduling of new merges as memory components
+        // get flushed. This now works only if the caller of deactivate waited for all IO
+        // operations to complete. Otherwise, disk components can be evicted while background
+        // merges are ongoing.
+        isDeactivating = true;
         try {
-            cb.waitForIO();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. STARTED", this);
+            if (flush && memoryComponentsAllocated) {
+                try {
+                    createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFlush().sync();
+                } catch (InterruptedException e) {
+                    throw HyracksDataException.create(e);
+                }
+                LOGGER.log(Level.INFO, "Deactivating the index: {}. Flushed", this);
+            }
+            LOGGER.log(Level.INFO, "Deactivating the disk components of: {}", this);
+            deactivateDiskComponents();
+            LOGGER.log(Level.INFO, "Deallocating memory components of: {}", this);
+            deallocateMemoryComponents();
+            isActive = false;
+            LOGGER.log(Level.INFO, "Deactivating the index: {}. COMPLETED", this);
+        } finally {
+            isDeactivating = false;
         }
     }
 
@@ -282,6 +293,8 @@
                 c.reset();
             }
         }
+        numScheduledFlushes = 0;
+        currentMutableComponentId.set(0);
     }
 
     @Override
@@ -297,19 +310,18 @@
         switch (ctx.getOperation()) {
             case UPDATE:
             case PHYSICALDELETE:
-            case FLUSH:
-            case DELETE_MEMORY_COMPONENT:
+            case DELETE_COMPONENTS:
             case DELETE:
             case UPSERT:
                 operationalComponents.add(memoryComponents.get(cmc));
                 break;
             case INSERT:
-                addOperationalMutableComponents(operationalComponents, true);
+                addOperationalMemoryComponents(operationalComponents, true);
                 operationalComponents.addAll(diskComponents);
                 break;
             case SEARCH:
                 if (memoryComponentsAllocated) {
-                    addOperationalMutableComponents(operationalComponents, false);
+                    addOperationalMemoryComponents(operationalComponents, false);
                 }
                 if (filterManager != null) {
                     for (int i = 0; i < diskComponents.size(); i++) {
@@ -326,13 +338,6 @@
                 }
 
                 break;
-            case MERGE:
-            case DELETE_DISK_COMPONENTS:
-                operationalComponents.addAll(ctx.getComponentsToBeMerged());
-                break;
-            case FULL_MERGE:
-                operationalComponents.addAll(diskComponents);
-                break;
             case REPLICATE:
                 operationalComponents.addAll(ctx.getComponentsToBeReplicated());
                 break;
@@ -350,65 +355,169 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(ctx.getComponentHolder());
-        ILSMIOOperation flushOp = createFlushOperation(opCtx, componentFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(flushOp, tracer));
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMMemoryComponent flushingComponent = getCurrentMemoryComponent();
+        if (flushingComponent.getWriterCount() > 0) {
+            throw new IllegalStateException(
+                    "createFlushOperation is called on a component with writers: " + flushingComponent);
+        }
+        // take care of the flush cycling
+        ILSMIOOperation flushOp =
+                TracedIOOperation.wrap(createFlushOperation(createOpContext(NoOpIndexAccessParameters.INSTANCE),
+                        fileManager.getRelFlushFileReference(), ioOpCallback), tracer);
+        // Changing the flush status should *always* precede changing the mutable component.
+        flushingComponent.schedule(LSMIOOperationType.FLUSH);
+        numScheduledFlushes++;
+        changeFlushStatusForCurrentMutableCompoent(false);
+        changeMutableComponent();
+        ILSMIndexAccessor accessor = flushOp.getAccessor();
+        ILSMIndexOperationContext flushCtx = accessor.getOpContext();
+        flushCtx.setOperation(ctx.getOperation()); // Could be component delete
+        flushCtx.getComponentHolder().add(flushingComponent);
+        flushCtx.setIoOperation(flushOp);
+        propagateMap(ctx, flushCtx);
+        ioOpCallback.scheduled(flushOp);
+        return flushOp;
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
-        // merge must create a different op ctx
-        AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setOperation(ctx.getOperation());
-        opCtx.getComponentHolder().addAll(mergingComponents);
-        mergingComponents.stream().map(ILSMDiskComponent.class::cast).forEach(opCtx.getComponentsToBeMerged()::add);
-        ILSMDiskComponent firstComponent = (ILSMDiskComponent) mergingComponents.get(0);
-        ILSMDiskComponent lastComponent = (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1);
-        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
-        ILSMIOOperation mergeOp = createMergeOperation(opCtx, mergeFileRefs, callback);
-        ioScheduler.scheduleOperation(TracedIOOperation.wrap(mergeOp, tracer));
-    }
-
-    private void addOperationalMutableComponents(List<ILSMComponent> operationalComponents, boolean modification) {
-        int cmc = currentMutableComponentId.get();
-        int numMutableComponents = memoryComponents.size();
-        for (int i = 0; i < numMutableComponents - 1; i++) {
-            ILSMMemoryComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
-            if (c.isReadable()) {
-                // Make sure newest components are added first if readable
-                operationalComponents.add(0, c);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        List<ILSMDiskComponent> mergingComponents = ctx.getComponentsToBeMerged();
+        // Merge operation can fail if another merge is already scheduled on those components
+        // This should be guarded against by the merge policy but we still protect against here
+        if (isDeactivating
+                || (mergingComponents.size() < 2 && ctx.getOperation() != IndexOperation.DELETE_COMPONENTS)) {
+            return NoOpIoOperation.INSTANCE;
+        }
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            if (mergingComponents.get(i).getState() == ComponentState.READABLE_MERGING) {
+                return NoOpIoOperation.INSTANCE;
             }
         }
-        // The current mutable component is added if modification operation or if readable
-        // This ensures that activation of new component only happens in case of modifications
-        // and allow for controlling that without stopping search operations
-        ILSMMemoryComponent c = memoryComponents.get(cmc);
-        if (modification || c.isReadable()) {
-            operationalComponents.add(0, c);
+        // merge must create a different op ctx
+        AbstractLSMIndexOperationContext mergeCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
+        mergeCtx.setOperation(ctx.getOperation());
+        mergeCtx.getComponentHolder().addAll(mergingComponents);
+        propagateMap(ctx, mergeCtx);
+        mergingComponents.stream().forEach(mergeCtx.getComponentsToBeMerged()::add);
+        ILSMDiskComponent lastComponent = mergingComponents.get(0);
+        ILSMDiskComponent firstComponent = mergingComponents.get(mergingComponents.size() - 1);
+        LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
+        ILSMIOOperation mergeOp =
+                TracedIOOperation.wrap(createMergeOperation(mergeCtx, mergeFileRefs, ioOpCallback), tracer);
+        mergeCtx.setIoOperation(mergeOp);
+        for (int i = 0; i < mergingComponents.size(); i++) {
+            mergingComponents.get(i).schedule(LSMIOOperationType.MERGE);
         }
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
+    }
+
+    private static void propagateMap(ILSMIndexOperationContext src, ILSMIndexOperationContext destination) {
+        Map<String, Object> map = src.getParameters();
+        if (map != null && !map.isEmpty()) {
+            destination.setParameters(new HashMap<>(map));
+        }
+    }
+
+    private void addOperationalMemoryComponents(List<ILSMComponent> operationalComponents, boolean modification) {
+        // add current memory component first if needed
+        if (numScheduledFlushes < memoryComponents.size()) {
+            ILSMMemoryComponent c = memoryComponents.get(currentMutableComponentId.get());
+            // The current mutable component is added if modification or readable
+            // This ensures that activation of new component only happens in case of modifications
+            // and allow for controlling that without stopping search operations
+            if (modification || c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+        if (modification && numScheduledFlushes >= memoryComponents.size()) {
+            // will fail the enterComponent call and retry
+            operationalComponents.add(memoryComponents.get(0));
+            return;
+        }
+        addImmutableMemoryComponents(operationalComponents);
+    }
+
+    private void addImmutableMemoryComponents(List<ILSMComponent> operationalComponents) {
+        int cmc = currentMutableComponentId.get();
+        int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+        int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+        for (int i = 0; i < numImmutableMemoryComponents; i++) {
+            next--;
+            if (next < 0) {
+                next = memoryComponents.size() - 1;
+            }
+            //newer components first
+            ILSMMemoryComponent c = memoryComponents.get(next);
+            if (c.isReadable()) {
+                operationalComponents.add(c);
+            }
+        }
+    }
+
+    private ILSMMemoryComponent getOldestReadableMemoryComponent() {
+        synchronized (getOperationTracker()) {
+            int cmc = currentMutableComponentId.get();
+            int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+            int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                next--;
+                if (next < 0) {
+                    next = memoryComponents.size() - 1;
+                }
+            }
+
+            // start going forward
+            for (int i = 0; i < numImmutableMemoryComponents; i++) {
+                if (memoryComponents.get(next).isReadable()) {
+                    return memoryComponents.get(next);
+                }
+                next++;
+                if (next == memoryComponents.size()) {
+                    next = 0;
+                }
+            }
+            throw new IllegalStateException("Couldn't find any readable component");
+        }
+    }
+
+    private int getNextToBeFlushed() {
+        // we have:
+        // 1. currentMemeoryComponent
+        // 2. numMemoryComponents
+        // 3. numScheduledFlushes
+        int diff = numScheduledFlushes % memoryComponents.size();
+        int cmc = currentMutableComponentId.get() - diff;
+        return cmc < 0 ? memoryComponents.size() + cmc : cmc;
     }
 
     @Override
     public final IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex) throws HyracksDataException {
+        return createBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, null);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException {
         if (checkIfEmptyIndex && !isEmptyIndex()) {
             throw HyracksDataException.create(ErrorCode.LOAD_NON_EMPTY_INDEX);
         }
-        return createBulkLoader(fillLevel, verifyInput, numElementsHint);
+        return createBulkLoader(fillFactor, verifyInput, numElementsHint, parameters);
     }
 
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
         AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
-        opCtx.setIoOperationType(LSMIOOperationType.LOAD);
-        ioOpCallback.beforeOperation(opCtx);
+        opCtx.setParameters(parameters);
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        LoadOperation loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+        loadOp.setNewComponent(createDiskComponent(bulkLoadComponentFactory,
+                componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
+                componentFileRefs.getBloomFilterFileReference(), true));
+        ioOpCallback.scheduled(loadOp);
+        opCtx.setIoOperation(loadOp);
         return new LSMIndexDiskComponentBulkLoader(this, opCtx, fillLevel, verifyInput, numElementsHint);
     }
 
@@ -468,7 +577,7 @@
         if (c != EmptyComponent.INSTANCE) {
             diskComponents.add(0, c);
         }
-        assert checkComponentIds();
+        validateComponentIds();
     }
 
     @Override
@@ -479,7 +588,7 @@
         if (newComponent != EmptyComponent.INSTANCE) {
             diskComponents.add(swapIndex, newComponent);
         }
-        assert checkComponentIds();
+        validateComponentIds();
     }
 
     /**
@@ -488,22 +597,21 @@
      *
      * @throws HyracksDataException
      */
-    private boolean checkComponentIds() throws HyracksDataException {
+    private void validateComponentIds() throws HyracksDataException {
         for (int i = 0; i < diskComponents.size() - 1; i++) {
             ILSMComponentId id1 = diskComponents.get(i).getId();
             ILSMComponentId id2 = diskComponents.get(i + 1).getId();
             IdCompareResult cmp = id1.compareTo(id2);
             if (cmp != IdCompareResult.UNKNOWN && cmp != IdCompareResult.GREATER_THAN) {
-                return false;
+                throw new IllegalStateException(
+                        "found non-decreasing component ids (" + id1 + " -> " + id2 + ") on index " + this);
             }
         }
-        return true;
     }
 
     @Override
     public void changeMutableComponent() {
         currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
-        memoryComponents.get(currentMutableComponentId.get()).requestActivation();
     }
 
     @Override
@@ -527,11 +635,6 @@
     }
 
     @Override
-    public ILSMIOOperationScheduler getIOScheduler() {
-        return ioScheduler;
-    }
-
-    @Override
     public ILSMIOOperationCallback getIOOperationCallback() {
         return ioOpCallback;
     }
@@ -556,8 +659,9 @@
     @Override
     public final String toString() {
         return "{\"class\" : \"" + getClass().getSimpleName() + "\", \"dir\" : \"" + fileManager.getBaseDir()
-                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents.size()) + ", \"disk\" : "
-                + diskComponents.size() + "}";
+                + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents) + ", \"disk\" : "
+                + diskComponents.size() + ", \"num-scheduled-flushes\":" + numScheduledFlushes
+                + ", \"current-memory-component\":" + currentMutableComponentId.get() + "}";
     }
 
     @Override
@@ -567,8 +671,12 @@
 
     @Override
     public boolean isCurrentMutableComponentEmpty() throws HyracksDataException {
-        //check if the current memory component has been modified
-        return !memoryComponents.get(currentMutableComponentId.get()).isModified();
+        synchronized (getOperationTracker()) {
+            ILSMMemoryComponent cmc = getCurrentMemoryComponent();
+            ComponentState state = cmc.getState();
+            return state == ComponentState.READABLE_UNWRITABLE_FLUSHING || state == ComponentState.INACTIVE
+                    || state == ComponentState.UNREADABLE_UNWRITABLE || !cmc.isModified();
+        }
     }
 
     @Override
@@ -583,7 +691,7 @@
 
     @Override
     public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
-            boolean bulkload, ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
+            ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
         //get set of files to be replicated for this component
         Set<String> componentFiles = new HashSet<>();
 
@@ -593,7 +701,7 @@
         }
 
         ReplicationExecutionType executionType;
-        if (bulkload) {
+        if (opType == LSMOperationType.LOAD) {
             executionType = ReplicationExecutionType.SYNC;
         } else {
             executionType = ReplicationExecutionType.ASYNC;
@@ -700,40 +808,57 @@
     }
 
     @Override
+    public void resetCurrentComponentIndex() {
+        synchronized (lsmHarness.getOperationTracker()) {
+            // validate no reader in any of the memory components and that all of them are INVALID
+            for (ILSMMemoryComponent c : memoryComponents) {
+                if (c.getReaderCount() > 0) {
+                    throw new IllegalStateException(
+                            "Attempt to reset current component index while readers are inside the components. " + c);
+                }
+                if (c.getState() != ComponentState.INACTIVE) {
+                    throw new IllegalStateException(
+                            "Attempt to reset current component index while a component is not INACTIVE. " + c);
+                }
+            }
+            currentMutableComponentId.set(0);
+            memoryComponents.get(0);
+            try {
+                memoryComponents.get(0).resetId(null, true);
+            } catch (HyracksDataException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+    }
+
+    @Override
     public final ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        if (opCtx.getOperation() == IndexOperation.DELETE_MEMORY_COMPONENT) {
+        ILSMMemoryComponent memoryComponent = (ILSMMemoryComponent) opCtx.getComponentHolder().get(0);
+        if (memoryComponent != getOldestReadableMemoryComponent()) {
+            throw new IllegalStateException("An attempt to flush a memory component that is not the oldest");
+        }
+        if (!memoryComponent.isModified() || opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS) {
             return EmptyComponent.INSTANCE;
         }
         if (LOGGER.isInfoEnabled()) {
             FlushOperation flushOp = (FlushOperation) operation;
-            LOGGER.log(Level.INFO, "Flushing component with id: " + flushOp.getFlushingComponent().getId());
+            LOGGER.log(Level.INFO,
+                    "Flushing component with id: " + flushOp.getFlushingComponent().getId() + " in the index " + this);
         }
-        try {
-            return doFlush(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute flush " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return doFlush(operation);
     }
 
     @Override
     public final ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException {
         ILSMIndexAccessor accessor = operation.getAccessor();
         ILSMIndexOperationContext opCtx = accessor.getOpContext();
-        try {
-            return opCtx.getOperation() == IndexOperation.DELETE_DISK_COMPONENTS ? EmptyComponent.INSTANCE
-                    : doMerge(operation);
-        } catch (Exception e) {
-            LOGGER.error("Fail to execute merge " + this, e);
-            cleanUpFiles(operation, e);
-            throw HyracksDataException.create(e);
-        }
+        return opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS ? EmptyComponent.INSTANCE : doMerge(operation);
     }
 
-    protected void cleanUpFiles(ILSMIOOperation operation, Exception e) {
+    @Override
+    public void cleanUpFilesForFailedOperation(ILSMIOOperation operation) {
         LSMComponentFileReferences componentFiles = operation.getComponentFiles();
         if (componentFiles == null) {
             return;
@@ -742,14 +867,25 @@
         for (FileReference file : files) {
             try {
                 if (file != null) {
+                    diskBufferCache.closeFileIfOpen(file);
                     diskBufferCache.deleteFile(file);
                 }
-            } catch (HyracksDataException hde) {
-                e.addSuppressed(hde);
+            } catch (Throwable th) { // NOSONAR Must catch all failures
+                operation.getFailure().addSuppressed(th);
             }
         }
     }
 
+    @Override
+    public String getIndexIdentifier() {
+        return fileManager.getBaseDir().getAbsolutePath();
+    }
+
+    //Called when a memory component is reset
+    public void memoryComponentsReset() {
+        numScheduledFlushes = Integer.max(0, numScheduledFlushes - 1);
+    }
+
     protected abstract LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
             ILSMDiskComponent lastComponent) throws HyracksDataException;
 
@@ -765,5 +901,4 @@
     protected abstract ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException;
 
     protected abstract ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException;
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index 59a919b..904029b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -19,15 +19,11 @@
 
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.text.Format;
-import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 
@@ -43,7 +39,9 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
 
+@NotThreadSafe
 public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManager {
 
     public enum TreeIndexState {
@@ -77,22 +75,19 @@
      */
     public static final String TXN_PREFIX = ".T";
 
-    public static final String COMPONENT_TIMESTAMP_FORMAT = "yyyy-MM-dd-HH-mm-ss-SSS";
-
     public static final FilenameFilter COMPONENT_FILES_FILTER = (dir, name) -> !name.startsWith(".");
     protected static final FilenameFilter txnFileNameFilter = (dir, name) -> name.startsWith(TXN_PREFIX);
     protected static FilenameFilter bloomFilterFilter =
             (dir, name) -> !name.startsWith(".") && name.endsWith(BLOOM_FILTER_SUFFIX);
-    protected static final FilenameFilter dummyFilter = (dir, name) -> true;
     protected static final Comparator<String> cmp = new FileNameComparator();
-
+    private static final FilenameFilter dummyFilter = (dir, name) -> true;
+    private static final long UNINITALIZED_COMPONENT_SEQ = -1;
     protected final IIOManager ioManager;
     // baseDir should reflect dataset name and partition name and be absolute
     protected final FileReference baseDir;
-    protected final Format formatter = new SimpleDateFormat(COMPONENT_TIMESTAMP_FORMAT);
-    protected final Comparator<ComparableFileName> recencyCmp = new RecencyComparator();
+    protected final Comparator<IndexComponentFileReference> recencyCmp = new RecencyComparator();
     protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
-    private String prevTimestamp = null;
+    private long lastUsedComponentSeq = UNINITALIZED_COMPONENT_SEQ;
 
     public AbstractLSMIndexFileManager(IIOManager ioManager, FileReference file,
             TreeIndexFactory<? extends ITreeIndex> treeFactory) {
@@ -132,20 +127,20 @@
     }
 
     protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
-            throws HyracksDataException {
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<IndexComponentFileReference> allFiles,
+            IBufferCache bufferCache) throws HyracksDataException {
         String[] files = listDirFiles(baseDir, filter);
         for (String fileName : files) {
             FileReference fileRef = baseDir.getChild(fileName);
             if (treeFactory == null) {
-                allFiles.add(new ComparableFileName(fileRef));
+                allFiles.add(IndexComponentFileReference.of(fileRef));
                 continue;
             }
             TreeIndexState idxState = isValidTreeIndex(treeFactory.createIndexInstance(fileRef));
             if (idxState == TreeIndexState.VALID) {
-                allFiles.add(new ComparableFileName(fileRef));
+                allFiles.add(IndexComponentFileReference.of(fileRef));
             } else if (idxState == TreeIndexState.INVALID) {
-                fileRef.delete();
+                bufferCache.deleteFile(fileRef);
             }
         }
     }
@@ -168,18 +163,16 @@
         return files;
     }
 
-    protected void validateFiles(HashSet<String> groundTruth, ArrayList<ComparableFileName> validFiles,
-            FilenameFilter filter, TreeIndexFactory<? extends ITreeIndex> treeFactory) throws HyracksDataException {
-        ArrayList<ComparableFileName> tmpAllInvListsFiles = new ArrayList<>();
-        cleanupAndGetValidFilesInternal(filter, treeFactory, tmpAllInvListsFiles);
-        for (ComparableFileName cmpFileName : tmpAllInvListsFiles) {
-            int index = cmpFileName.fileName.lastIndexOf(DELIMITER);
-            String file = cmpFileName.fileName.substring(0, index);
-            if (groundTruth.contains(file)) {
+    protected void validateFiles(HashSet<String> groundTruth, ArrayList<IndexComponentFileReference> validFiles,
+            FilenameFilter filter, TreeIndexFactory<? extends ITreeIndex> treeFactory, IBufferCache bufferCache)
+            throws HyracksDataException {
+        ArrayList<IndexComponentFileReference> tmpAllInvListsFiles = new ArrayList<>();
+        cleanupAndGetValidFilesInternal(filter, treeFactory, tmpAllInvListsFiles, bufferCache);
+        for (IndexComponentFileReference cmpFileName : tmpAllInvListsFiles) {
+            if (groundTruth.contains(cmpFileName.getSequence())) {
                 validFiles.add(cmpFileName);
             } else {
-                File invalidFile = new File(cmpFileName.fullPath);
-                IoUtil.delete(invalidFile);
+                delete(bufferCache, cmpFileName.getFullPath());
             }
         }
     }
@@ -199,72 +192,64 @@
 
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
-        String ts = getCurrentTimestamp();
-        // Begin timestamp and end timestamp are identical since it is a flush
-        return new LSMComponentFileReferences(baseDir.getChild(ts + DELIMITER + ts), null, null);
+        final String sequence = getNextComponentSequence(COMPONENT_FILES_FILTER);
+        return new LSMComponentFileReferences(baseDir.getChild(sequence), null, null);
     }
 
     @Override
-    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
-            throws HyracksDataException {
-        String[] firstTimestampRange = firstFileName.split(DELIMITER);
-        String[] lastTimestampRange = lastFileName.split(DELIMITER);
-        // Get the range of timestamps by taking the earliest and the latest timestamps
-        return new LSMComponentFileReferences(
-                baseDir.getChild(firstTimestampRange[0] + DELIMITER + lastTimestampRange[1]), null, null);
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
+        final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
+        return new LSMComponentFileReferences(baseDir.getChild(baseName), null, null);
     }
 
     @Override
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException {
         List<LSMComponentFileReferences> validFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allFiles = new ArrayList<>();
 
         // Gather files and delete invalid files
         // There are two types of invalid files:
         // (1) The isValid flag is not set
         // (2) The file's interval is contained by some other file
         // Here, we only filter out (1).
-        cleanupAndGetValidFilesInternal(COMPONENT_FILES_FILTER, treeFactory, allFiles);
+        cleanupAndGetValidFilesInternal(COMPONENT_FILES_FILTER, treeFactory, allFiles, treeFactory.getBufferCache());
 
         if (allFiles.isEmpty()) {
             return validFiles;
         }
 
         if (allFiles.size() == 1) {
-            validFiles.add(new LSMComponentFileReferences(allFiles.get(0).fileRef, null, null));
+            validFiles.add(new LSMComponentFileReferences(allFiles.get(0).getFileRef(), null, null));
             return validFiles;
         }
 
-        // Sorts files names from earliest to latest timestamp.
+        // Sorts files names from earliest to latest
         Collections.sort(allFiles);
 
-        List<ComparableFileName> validComparableFiles = new ArrayList<>();
-        ComparableFileName last = allFiles.get(0);
+        List<IndexComponentFileReference> validComparableFiles = new ArrayList<>();
+        IndexComponentFileReference last = allFiles.get(0);
         validComparableFiles.add(last);
         for (int i = 1; i < allFiles.size(); i++) {
-            ComparableFileName current = allFiles.get(i);
-            // The current start timestamp is greater than last stop timestamp so current is valid.
-            if (current.interval[0].compareTo(last.interval[1]) > 0) {
+            IndexComponentFileReference current = allFiles.get(i);
+            if (current.isMoreRecentThan(last)) {
+                // The current start sequence is greater than last stop sequence so current is valid.
                 validComparableFiles.add(current);
                 last = current;
-            } else if (current.interval[0].compareTo(last.interval[0]) >= 0
-                    && current.interval[1].compareTo(last.interval[1]) <= 0) {
+            } else if (current.isWithin(last)) {
                 // The current file is completely contained in the interval of the
                 // last file. Thus the last file must contain at least as much information
                 // as the current file, so delete the current file.
-                current.fileRef.delete();
+                delete(treeFactory.getBufferCache(), current.getFullPath());
             } else {
                 // This scenario should not be possible since timestamps are monotonically increasing.
                 throw HyracksDataException.create(ErrorCode.FOUND_OVERLAPPING_LSM_FILES, baseDir);
             }
         }
-
         // Sort valid files in reverse lexicographical order, such that newer files come first.
-        Collections.sort(validComparableFiles, recencyCmp);
-        for (ComparableFileName cmpFileName : validComparableFiles) {
-            validFiles.add(new LSMComponentFileReferences(cmpFileName.fileRef, null, null));
+        validComparableFiles.sort(recencyCmp);
+        for (IndexComponentFileReference cmpFileName : validComparableFiles) {
+            validFiles.add(new LSMComponentFileReferences(cmpFileName.getFileRef(), null, null));
         }
-
         return validFiles;
     }
 
@@ -283,8 +268,7 @@
     private static class FileNameComparator implements Comparator<String> {
         @Override
         public int compare(String a, String b) {
-            // Consciously ignoring locale.
-            return -a.compareTo(b);
+            return IndexComponentFileReference.of(b).compareTo(IndexComponentFileReference.of(a));
         }
     }
 
@@ -305,39 +289,14 @@
         }
     }
 
-    protected class ComparableFileName implements Comparable<ComparableFileName> {
-        public final FileReference fileRef;
-        public final String fullPath;
-        public final String fileName;
-
-        // Timestamp interval.
-        public final String[] interval;
-
-        public ComparableFileName(FileReference fileRef) {
-            this.fileRef = fileRef;
-            this.fullPath = fileRef.getFile().getAbsolutePath();
-            this.fileName = fileRef.getFile().getName();
-            interval = fileName.split(DELIMITER);
-        }
-
+    private class RecencyComparator implements Comparator<IndexComponentFileReference> {
         @Override
-        public int compareTo(ComparableFileName b) {
-            int startCmp = interval[0].compareTo(b.interval[0]);
+        public int compare(IndexComponentFileReference a, IndexComponentFileReference b) {
+            int startCmp = -Long.compare(a.getSequenceStart(), b.getSequenceStart());
             if (startCmp != 0) {
                 return startCmp;
             }
-            return b.interval[1].compareTo(interval[1]);
-        }
-    }
-
-    private class RecencyComparator implements Comparator<ComparableFileName> {
-        @Override
-        public int compare(ComparableFileName a, ComparableFileName b) {
-            int cmp = -a.interval[0].compareTo(b.interval[0]);
-            if (cmp != 0) {
-                return cmp;
-            }
-            return -a.interval[1].compareTo(b.interval[1]);
+            return -Long.compare(a.getSequenceEnd(), b.getSequenceEnd());
         }
     }
 
@@ -372,10 +331,10 @@
         return null;
     }
 
-    protected static FilenameFilter createTransactionFilter(String transactionFileName, final boolean inclusive) {
+    private static FilenameFilter createTransactionFilter(String transactionFileName, final boolean inclusive) {
         final String timeStamp =
                 transactionFileName.substring(transactionFileName.indexOf(TXN_PREFIX) + TXN_PREFIX.length());
-        return (dir, name) -> inclusive ? name.startsWith(timeStamp) : !name.startsWith(timeStamp);
+        return (dir, name) -> inclusive == name.startsWith(timeStamp);
     }
 
     protected FilenameFilter getTransactionFileFilter(boolean inclusive) throws HyracksDataException {
@@ -387,38 +346,28 @@
         }
     }
 
+    protected void delete(IBufferCache bufferCache, String fullPath) throws HyracksDataException {
+        FileReference fileRef = ioManager.resolveAbsolutePath(fullPath);
+        bufferCache.deleteFile(fileRef);
+    }
+
     protected FilenameFilter getCompoundFilter(final FilenameFilter filter1, final FilenameFilter filter2) {
         return (dir, name) -> filter1.accept(dir, name) && filter2.accept(dir, name);
     }
 
-    /**
-     * @return The string format of the current timestamp.
-     *         The returned results of this method are guaranteed to not have duplicates.
-     */
-    protected String getCurrentTimestamp() {
-        Date date = new Date();
-        String ts = formatter.format(date);
-        /**
-         * prevent a corner case where the same timestamp can be given.
-         */
-        while (prevTimestamp != null && ts.compareTo(prevTimestamp) == 0) {
-            try {
-                Thread.sleep(1);
-                date = new Date();
-                ts = formatter.format(date);
-            } catch (InterruptedException e) {
-                //ignore
-            }
+    protected String getNextComponentSequence(FilenameFilter filenameFilter) throws HyracksDataException {
+        if (lastUsedComponentSeq == UNINITALIZED_COMPONENT_SEQ) {
+            lastUsedComponentSeq = getOnDiskLastUsedComponentSequence(filenameFilter);
         }
-        prevTimestamp = ts;
-        return ts;
+        return IndexComponentFileReference.getFlushSequence(++lastUsedComponentSeq);
     }
 
-    public static String getComponentStartTime(String fileName) {
-        return fileName.split(DELIMITER)[0];
-    }
-
-    public static String getComponentEndTime(String fileName) {
-        return fileName.split(DELIMITER)[1];
+    private long getOnDiskLastUsedComponentSequence(FilenameFilter filenameFilter) throws HyracksDataException {
+        long maxComponentSeq = -1;
+        final String[] files = listDirFiles(baseDir, filenameFilter);
+        for (String fileName : files) {
+            maxComponentSeq = Math.max(maxComponentSeq, IndexComponentFileReference.of(fileName).getSequenceEnd());
+        }
+        return maxComponentSeq;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
index c993874..2e133a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -28,7 +29,7 @@
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -57,8 +58,8 @@
     private long enterExitTime = 0L;
     protected boolean skipFilter = false;
     protected boolean recovery = false;
-    private LSMIOOperationType ioOpType = LSMIOOperationType.NOOP;
-    private ILSMDiskComponent newDiskComponent;
+    private ILSMIOOperation ioOperation;
+    private Map<String, Object> parametersMap;
 
     public AbstractLSMIndexOperationContext(ILSMIndex index, int[] treeFields, int[] filterFields,
             IBinaryComparatorFactory[] filterCmpFactories, ISearchOperationCallback searchCallback,
@@ -210,22 +211,23 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
-        return ioOpType;
+    public ILSMIOOperation getIoOperation() {
+        return ioOperation;
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
-        this.ioOpType = ioOpType;
+    public void setIoOperation(ILSMIOOperation ioOperation) {
+        this.ioOperation = ioOperation;
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
-        return newDiskComponent;
+    public void setParameters(Map<String, Object> map) {
+        this.parametersMap = map;
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
-        this.newDiskComponent = component;
+    public Map<String, Object> getParameters() {
+        return parametersMap;
     }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 9596495..9440648 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -24,12 +24,12 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -39,31 +39,56 @@
     private final IVirtualBufferCache vbc;
     private final AtomicBoolean isModified;
     private int writerCount;
-    private boolean requestedToBeActive;
+    private int pendingFlushes = 0;
     private final MemoryComponentMetadata metadata;
     private ILSMComponentId componentId;
 
-    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
-            ILSMComponentFilter filter) {
+    public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
         super(lsmIndex, filter);
         this.vbc = vbc;
         writerCount = 0;
-        if (isActive) {
-            state = ComponentState.READABLE_WRITABLE;
-        } else {
-            state = ComponentState.INACTIVE;
-        }
+        state = ComponentState.INACTIVE;
         isModified = new AtomicBoolean();
         metadata = new MemoryComponentMetadata();
     }
 
+    /**
+     * Prepare the component to be scheduled for an IO operation
+     *
+     * @param ioOperationType
+     * @throws HyracksDataException
+     */
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        activeate();
+        if (ioOperationType == LSMIOOperationType.FLUSH) {
+            if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to schedule a flush when writerCount != 0");
+                }
+                state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+            } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
+                    || state == ComponentState.UNREADABLE_UNWRITABLE) {
+                // There is an ongoing flush. Increase pending flush count
+                pendingFlushes++;
+            } else {
+                throw new IllegalStateException("Trying to schedule a flush when the component state = " + state);
+            }
+        } else {
+            throw new UnsupportedOperationException("Unsupported operation " + ioOperationType);
+        }
+    }
+
+    private void activeate() throws HyracksDataException {
+        if (state == ComponentState.INACTIVE) {
+            state = ComponentState.READABLE_WRITABLE;
+            lsmIndex.getIOOperationCallback().recycled(this);
+        }
+    }
+
     @Override
     public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException {
-        if (state == ComponentState.INACTIVE && requestedToBeActive) {
-            state = ComponentState.READABLE_WRITABLE;
-            requestedToBeActive = false;
-            lsmIndex.getIOOperationCallback().recycled(this, true);
-        }
+        activeate();
         switch (opType) {
             case FORCE_MODIFICATION:
                 if (isMutableComponent) {
@@ -97,7 +122,6 @@
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE
                         || state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
@@ -107,16 +131,18 @@
                 }
                 break;
             case FLUSH:
-                if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
-                    if (writerCount != 0) {
-                        throw new IllegalStateException("Trying to flush when writerCount != 0");
-                    }
-                    state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
-                    readerCount++;
-                } else {
+                if (state == ComponentState.UNREADABLE_UNWRITABLE) {
                     return false;
                 }
-                break;
+                if (state != ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+                    throw new IllegalStateException("Trying to flush when component state = " + state);
+                }
+                if (writerCount != 0) {
+                    throw new IllegalStateException("Trying to flush when writerCount = " + writerCount);
+                }
+                readerCount++;
+                return true;
+
             default:
                 throw new UnsupportedOperationException("Unsupported operation " + opType);
         }
@@ -139,15 +165,14 @@
                 } else {
                     readerCount--;
                     if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                        state = ComponentState.INACTIVE;
+                        reset();
                     }
                 }
                 break;
-            case REPLICATE:
             case SEARCH:
                 readerCount--;
                 if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 }
                 break;
             case FLUSH:
@@ -156,12 +181,12 @@
                 }
                 readerCount--;
                 if (failedOperation) {
-                    // if flush failed, return the component state to READABLE_UNWRITABLE
-                    state = ComponentState.READABLE_UNWRITABLE;
+                    // If flush failed, keep the component state to READABLE_UNWRITABLE_FLUSHING
                     return;
                 }
+                // operation succeeded
                 if (readerCount == 0) {
-                    state = ComponentState.INACTIVE;
+                    reset();
                 } else {
                     state = ComponentState.UNREADABLE_UNWRITABLE;
                 }
@@ -177,20 +202,15 @@
 
     @Override
     public boolean isReadable() {
-        if (state == ComponentState.INACTIVE || state == ComponentState.UNREADABLE_UNWRITABLE) {
-            return false;
+        return state != ComponentState.INACTIVE && state != ComponentState.UNREADABLE_UNWRITABLE;
+    }
+
+    @Override
+    public void setUnwritable() {
+        if (state != ComponentState.READABLE_WRITABLE) {
+            throw new IllegalStateException("Attempt to set unwritable a component that is " + state);
         }
-        return true;
-    }
-
-    @Override
-    public void setState(ComponentState state) {
-        this.state = state;
-    }
-
-    @Override
-    public void requestActivation() {
-        requestedToBeActive = true;
+        this.state = ComponentState.READABLE_UNWRITABLE;
     }
 
     @Override
@@ -210,12 +230,23 @@
 
     @Override
     public final void reset() throws HyracksDataException {
+        state = ComponentState.INACTIVE;
         isModified.set(false);
         metadata.reset();
         if (filter != null) {
             filter.reset();
         }
         doReset();
+        lsmIndex.memoryComponentsReset();
+        // a flush can be pending on a component that just completed its flush... here is when this can happen:
+        // primary index has 2 components, secondary index has 2 components.
+        // 2 flushes are scheduled on each p1, p2, s1, and s2.
+        // p1 and p2 both finish. primary component 1 gets full and secondary doesn't have any entries (optional field).
+        // then flush is scheduled on p1, s1 will have a pending flush in that case.
+        if (pendingFlushes > 0) {
+            schedule(LSMIOOperationType.FLUSH);
+            pendingFlushes--;
+        }
     }
 
     protected void doReset() throws HyracksDataException {
@@ -244,7 +275,7 @@
             allocated = true;
         } finally {
             if (!allocated) {
-                ((IVirtualBufferCache) getIndex().getBufferCache()).close();
+                getIndex().getBufferCache().close();
             }
         }
     }
@@ -267,6 +298,7 @@
     @Override
     public final void deallocate() throws HyracksDataException {
         try {
+            state = ComponentState.INACTIVE;
             doDeallocate();
         } finally {
             getIndex().getBufferCache().close();
@@ -297,15 +329,24 @@
 
     @Override
     public void resetId(ILSMComponentId componentId, boolean force) throws HyracksDataException {
-        if (!force && this.componentId != null && !componentId.missing() // for backward compatibility
+        if (!force && this.componentId != null
                 && this.componentId.compareTo(componentId) != IdCompareResult.LESS_THAN) {
             throw new IllegalStateException(
                     this + " receives illegal id. Old id " + this.componentId + ", new id " + componentId);
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO, "Component Id was reset from " + this.componentId + " to " + componentId);
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Component Id was reset from " + this.componentId + " to " + componentId);
         }
         this.componentId = componentId;
-        LSMComponentIdUtils.persist(this.componentId, metadata);
+        if (componentId != null) {
+            LSMComponentIdUtils.persist(this.componentId, metadata);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "{\"class\":\"" + getClass().getSimpleName() + "\", \"state\":\"" + state + "\", \"writers\":"
+                + writerCount + ", \"readers\":" + readerCount + ", \"pendingFlushes\":" + pendingFlushes
+                + ", \"id\":\"" + componentId + "\"}";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
new file mode 100644
index 0000000..373d7e7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
@@ -0,0 +1,63 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+
+public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
+
+    public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
+    }
+
+    public abstract AbstractTreeIndex getBuddyIndex();
+
+    @Override
+    public void doReset() throws HyracksDataException {
+        super.doReset();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doAllocate() throws HyracksDataException {
+        super.doAllocate();
+        getBuddyIndex().create();
+        getBuddyIndex().activate();
+    }
+
+    @Override
+    public void doDeallocate() throws HyracksDataException {
+        super.doDeallocate();
+        getBuddyIndex().deactivate();
+        getBuddyIndex().destroy();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        super.validate();
+        getBuddyIndex().validate();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index a439ace..e4b845a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -18,94 +18,80 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 
-public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+public class AsynchronousScheduler implements ILSMIOOperationScheduler, Closeable {
     // Since this is a asynchronous scheduler, we make sure that flush operations coming from the same lsm index
     // will be executed serially in same order of scheduling the operations. Look at asterix issue 630.
 
-    public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
-    private ExecutorService executor;
-    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<String, ILSMIOOperation>();
-    private final Map<String, PriorityQueue<ILSMIOOperation>> waitingFlushOperations =
-            new HashMap<String, PriorityQueue<ILSMIOOperation>>();
+    private final ExecutorService executor;
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations = new HashMap<>();
+    private final Map<String, Throwable> failedGroups = new HashMap<>();
 
-    public void init(ThreadFactory threadFactory) {
-        // Creating an executor with the same configuration of Executors.newCachedThreadPool.
-        executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
-                threadFactory) {
-
-            @Override
-            protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
-                return new LSMIOOperationTask<T>(callable);
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            protected void afterExecute(Runnable r, Throwable t) {
-                super.afterExecute(r, t);
-                LSMIOOperationTask<Boolean> task = (LSMIOOperationTask<Boolean>) r;
-                ILSMIOOperation executedOp = task.getOperation();
-                if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-                    String id = executedOp.getIndexIdentifier();
-                    synchronized (this) {
-                        runningFlushOperations.remove(id);
-                        if (waitingFlushOperations.containsKey(id)) {
-                            try {
-                                ILSMIOOperation op = waitingFlushOperations.get(id).poll();
-                                if (op != null) {
-                                    scheduleOperation(op);
-                                } else {
-                                    waitingFlushOperations.remove(id);
-                                }
-                            } catch (HyracksDataException e) {
-                                t = e.getCause();
-                            }
-                        }
-                    }
-                }
-            }
-        };
+    public AsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback) {
+        executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations,
+                waitingFlushOperations, failedGroups);
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-        if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
-            executor.submit(operation);
-        } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-            String id = operation.getIndexIdentifier();
-            synchronized (executor) {
-                if (runningFlushOperations.containsKey(id)) {
-                    if (waitingFlushOperations.containsKey(id)) {
-                        waitingFlushOperations.get(id).offer(operation);
-                    } else {
-                        PriorityQueue<ILSMIOOperation> q = new PriorityQueue<ILSMIOOperation>();
-                        q.offer(operation);
-                        waitingFlushOperations.put(id, q);
-                    }
-                } else {
-                    runningFlushOperations.put(id, operation);
-                    executor.submit(operation);
-                }
-            }
-        } else {
-            // this should never happen
-            // just guard here to avoid silient failures in case of future extensions
-            throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
+    public void scheduleOperation(ILSMIOOperation operation) {
+        switch (operation.getIOOpertionType()) {
+            case FLUSH:
+                scheduleFlush(operation);
+                break;
+            case MERGE:
+                executor.submit(operation);
+                break;
+            case NOOP:
+                return;
+            default:
+                // this should never happen
+                // just guard here to avoid silent failures in case of future extensions
+                throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
         }
     }
+
+    private void scheduleFlush(ILSMIOOperation operation) {
+        String id = operation.getIndexIdentifier();
+        synchronized (executor) {
+            if (failedGroups.containsKey(id)) {
+                // Group failure. Fail the operation right away
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                        failedGroups.get(id)));
+                operation.complete();
+                return;
+            }
+            if (runningFlushOperations.containsKey(id)) {
+                if (waitingFlushOperations.containsKey(id)) {
+                    waitingFlushOperations.get(id).offer(operation);
+                } else {
+                    Deque<ILSMIOOperation> q = new ArrayDeque<>();
+                    q.offer(operation);
+                    waitingFlushOperations.put(id, q);
+                }
+            } else {
+                runningFlushOperations.put(id, operation);
+                executor.submit(operation);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        executor.shutdown();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
deleted file mode 100644
index a8ee286..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-public class BlockingIOOperationCallbackWrapper implements ILSMIOOperationCallback {
-
-    private boolean notified = false;
-
-    private final ILSMIOOperationCallback wrappedCallback;
-
-    public BlockingIOOperationCallbackWrapper(ILSMIOOperationCallback callback) {
-        this.wrappedCallback = callback;
-    }
-
-    public synchronized void waitForIO() throws InterruptedException {
-        while (!notified) {
-            wait();
-        }
-        notified = false;
-    }
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.beforeOperation(opCtx);
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterOperation(opCtx);
-    }
-
-    @Override
-    public synchronized void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        wrappedCallback.afterFinalize(opCtx);
-        notifyAll();
-        notified = true;
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
-        wrappedCallback.recycled(component, componentSwitched);
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
-        wrappedCallback.allocated(component);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
index 29ca388..a9c70e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BloomFilterBulkLoader.java
@@ -21,6 +21,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public class BloomFilterBulkLoader implements IChainedComponentBulkLoader {
 
@@ -65,4 +66,19 @@
             endedBloomFilterLoad = true;
         }
     }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return bulkLoader.hasFailed();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        return bulkLoader.getFailure();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
index f38614c..3fa45c9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -18,26 +18,33 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import java.util.LinkedList;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.util.annotations.CriticalPath;
 
 /**
  * Class encapsulates a chain of operations, happening during an LSM disk component bulkload
  */
 public class ChainedLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
 
-    private List<IChainedComponentBulkLoader> bulkloaderChain = new LinkedList<>();
-    private boolean isEmptyComponent = true;
-    private boolean cleanedUpArtifacts = false;
+    private List<IChainedComponentBulkLoader> bulkloaderChain = new ArrayList<>();
+    private final ILSMIOOperation operation;
     private final ILSMDiskComponent diskComponent;
     private final boolean cleanupEmptyComponent;
+    private boolean isEmptyComponent = true;
+    private boolean cleanedUpArtifacts = false;
 
-    public ChainedLSMDiskComponentBulkLoader(ILSMDiskComponent diskComponent, boolean cleanupEmptyComponent) {
+    public ChainedLSMDiskComponentBulkLoader(ILSMIOOperation operation, ILSMDiskComponent diskComponent,
+            boolean cleanupEmptyComponent) {
+        this.operation = operation;
         this.diskComponent = diskComponent;
         this.cleanupEmptyComponent = cleanupEmptyComponent;
     }
@@ -46,14 +53,18 @@
         bulkloaderChain.add(bulkloader);
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
+    @CriticalPath
     public void add(ITupleReference tuple) throws HyracksDataException {
         try {
             ITupleReference t = tuple;
-            for (IChainedComponentBulkLoader lsmBulkloader : bulkloaderChain) {
-                t = lsmBulkloader.add(t);
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                t = bulkloaderChain.get(i).add(t);
             }
-        } catch (Exception e) {
+        } catch (Throwable e) {
+            operation.setFailure(e);
             cleanupArtifacts();
             throw e;
         }
@@ -62,14 +73,18 @@
         }
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
+    @CriticalPath
     public void delete(ITupleReference tuple) throws HyracksDataException {
         try {
             ITupleReference t = tuple;
-            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
-                t = lsmOperation.delete(t);
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                t = bulkloaderChain.get(i).delete(t);
             }
-        } catch (Exception e) {
+        } catch (Throwable e) {
+            operation.setFailure(e);
             cleanupArtifacts();
             throw e;
         }
@@ -82,18 +97,20 @@
     public void cleanupArtifacts() throws HyracksDataException {
         if (!cleanedUpArtifacts) {
             cleanedUpArtifacts = true;
-            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
-                lsmOperation.cleanupArtifacts();
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                bulkloaderChain.get(i).cleanupArtifacts();;
             }
+            diskComponent.deactivateAndDestroy();
         }
-        diskComponent.deactivateAndDestroy();
     }
 
     @Override
     public void end() throws HyracksDataException {
         if (!cleanedUpArtifacts) {
-            for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
-                lsmOperation.end();
+            final int bulkloadersCount = bulkloaderChain.size();
+            for (int i = 0; i < bulkloadersCount; i++) {
+                bulkloaderChain.get(i).end();
             }
             if (isEmptyComponent && cleanupEmptyComponent) {
                 cleanupArtifacts();
@@ -103,8 +120,42 @@
 
     @Override
     public void abort() throws HyracksDataException {
-        for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
-            lsmOperation.abort();
+        operation.setStatus(LSMIOOperationStatus.FAILURE);
+        final int bulkloadersCount = bulkloaderChain.size();
+        for (int i = 0; i < bulkloadersCount; i++) {
+            bulkloaderChain.get(i).abort();
         }
     }
+
+    @Override
+    public ILSMIOOperation getOperation() {
+        return operation;
+    }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        final int bulkloadersCount = bulkloaderChain.size();
+        for (int i = 0; i < bulkloadersCount; i++) {
+            if (bulkloaderChain.get(i).hasFailed()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public Throwable getFailure() {
+        final int bulkloadersCount = bulkloaderChain.size();
+        for (int i = 0; i < bulkloadersCount; i++) {
+            if (bulkloaderChain.get(i).hasFailed()) {
+                return bulkloaderChain.get(i).getFailure();
+            }
+        }
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 7bb24dc..35edcc5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -30,14 +31,12 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -48,7 +47,7 @@
     private final List<ILSMComponentId> replacedComponentIds;
     private final int[] swapIndexes;
     private int count = 0;
-    boolean accessingComponent = true;
+    private boolean accessingComponent = true;
 
     public ComponentReplacementContext(ILSMIndex lsmIndex) {
         components = new ArrayList<>(lsmIndex.getNumberOfAllMemoryComponents());
@@ -156,12 +155,12 @@
             replacedComponentIds.add(components.get(i).getId());
             // ensure that disk component exists
             boolean found = false;
-            LOGGER.log(Level.INFO, "Looking for a component with the id: " + replacedComponentIds.get(i));
-            for (int j = 0; j < allDiskComponents.size(); j++) {
-                ILSMDiskComponent dc = allDiskComponents.get(j);
+            final ILSMComponentId replacedComponentId = replacedComponentIds.get(i);
+            LOGGER.trace("looking for a component with the id: {}", replacedComponentId);
+            for (ILSMDiskComponent dc : allDiskComponents) {
                 ILSMComponentId diskComponentId = dc.getId();
-                LOGGER.log(Level.INFO, "Next disk component id: " + diskComponentId);
-                if (diskComponentId.equals(replacedComponentIds.get(i))) {
+                LOGGER.trace("next disk component id: {}", diskComponentId);
+                if (diskComponentId.equals(replacedComponentId)) {
                     found = true;
                     diskComponents.add(dc);
                     break;
@@ -169,8 +168,8 @@
             }
             if (!found) {
                 // component has been merged?
-                LOGGER.log(Level.WARN, "Memory Component with id = " + replacedComponentIds.get(i)
-                        + " was flushed and merged before search cursor replaces it");
+                LOGGER.warn("memory component {} was flushed and merged before search cursor replaces it",
+                        replacedComponentId);
                 return false;
             }
         }
@@ -194,14 +193,14 @@
             for (int i = 0; i < count; i++) {
                 ILSMComponent removed = ctx.getComponentHolder().remove(swapIndexes[i]);
                 if (removed.getType() == LSMComponentType.MEMORY) {
-                    LOGGER.log(Level.INFO, "Removed a memory component from the search operation");
+                    LOGGER.info("Removed a memory component from the search operation");
                 } else {
                     throw new IllegalStateException("Disk components can't be removed from the search operation");
                 }
                 ctx.getComponentHolder().add(swapIndexes[i], diskComponents.get(i));
             }
         } catch (Exception e) {
-            LOGGER.log(Level.WARN, "Failure replacing memory components with disk components", e);
+            LOGGER.warn("Failure replacing memory components with disk components", e);
             throw e;
         }
     }
@@ -242,22 +241,22 @@
     }
 
     @Override
-    public LSMIOOperationType getIoOperationType() {
+    public ILSMIOOperation getIoOperation() {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setIoOperationType(LSMIOOperationType ioOpType) {
+    public void setIoOperation(ILSMIOOperation ioOperation) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public ILSMDiskComponent getNewComponent() {
+    public void setParameters(Map<String, Object> map) {
         throw new UnsupportedOperationException();
     }
 
     @Override
-    public void setNewComponent(ILSMDiskComponent component) {
+    public Map<String, Object> getParameters() {
         throw new UnsupportedOperationException();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 13911ef..c642d82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -49,10 +49,10 @@
             IIndexAccessParameters iap =
                     new IndexAccessParameters(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             ILSMIndexAccessor accessor = index.createAccessor(iap);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            accessor.scheduleFullMerge();
         } else if (immutableComponents.size() >= numComponents) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
         }
     }
 
@@ -108,7 +108,7 @@
                 throw new IllegalStateException();
             }
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+            accessor.scheduleMerge(immutableComponents);
             return true;
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
index 52bf896..75eaf83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyFactory.java
@@ -24,9 +24,14 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class ConstantMergePolicyFactory implements ILSMMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
@@ -49,4 +54,14 @@
         policy.configure(configuration);
         return policy;
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new ConstantMergePolicyFactory();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index e3ca9f1..a2d9fdd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -27,8 +27,11 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
 import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public class EmptyComponent implements ILSMDiskComponent {
     public static final EmptyComponent INSTANCE = new EmptyComponent();
@@ -84,7 +87,7 @@
 
     @Override
     public ILSMComponentId getId() {
-        return LSMComponentId.MISSING_COMPONENT_ID;
+        return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
     }
 
     @Override
@@ -103,7 +106,7 @@
     }
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
+    public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
         // No Op
     }
 
@@ -144,9 +147,19 @@
     }
 
     @Override
-    public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
-            long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
-            throws HyracksDataException {
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent) throws HyracksDataException {
         return null;
     }
+
+    @Override
+    public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public int getReaderCount() {
+        return 0;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
index 404c8e9..854e541 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -28,8 +28,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -39,15 +38,11 @@
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.util.trace.ITracer;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 public class ExternalIndexHarness extends LSMHarness {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled) {
-        super(lsmIndex, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
+    public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled) {
+        super(lsmIndex, ioScheduler, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
     }
 
     @Override
@@ -104,14 +99,6 @@
             }
             ctx.setAccessingComponents(true);
         }
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
@@ -135,7 +122,7 @@
                             if (replicationEnabled) {
                                 componentsToBeReplicated.clear();
                                 componentsToBeReplicated.add((ILSMDiskComponent) c);
-                                lsmIndex.scheduleReplication(null, componentsToBeReplicated, false,
+                                lsmIndex.scheduleReplication(null, componentsToBeReplicated,
                                         ReplicationOperation.DELETE, opType);
                             }
                             ((ILSMDiskComponent) c).deactivateAndDestroy();
@@ -155,7 +142,7 @@
                             if (replicationEnabled) {
                                 componentsToBeReplicated.clear();
                                 componentsToBeReplicated.add(newComponent);
-                                triggerReplication(componentsToBeReplicated, false, opType);
+                                triggerReplication(componentsToBeReplicated, opType);
                             }
                             mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
                         }
@@ -205,63 +192,24 @@
         }
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            // If the merge cannot be scheduled because there is already an ongoing merge on subset/all of the components, then
-            // whenever the current merge has finished, it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
-        }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
-    }
-
-    @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
-
-        ILSMDiskComponent newComponent = null;
+    public void addBulkLoadedComponent(ILSMIOOperation ioOperation) throws HyracksDataException {
+        ILSMDiskComponent c = ioOperation.getNewComponent();
         try {
-            newComponent = lsmIndex.merge(operation);
-            ctx.setNewComponent(newComponent);
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            operation.getCallback().afterOperation(ctx);
-            newComponent.markAsValid(lsmIndex.isDurable());
-        } finally {
-            exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
-            operation.getCallback().afterFinalize(ctx);
+            c.markAsValid(lsmIndex.isDurable(), ioOperation);
+        } catch (Throwable th) {
+            ioOperation.setFailure(th);
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
+        if (ioOperation.hasFailed()) {
+            throw HyracksDataException.create(ioOperation.getFailure());
         }
-    }
-
-    @Override
-    public void addBulkLoadedComponent(ILSMDiskComponent c) throws HyracksDataException {
-        c.markAsValid(lsmIndex.isDurable());
         synchronized (opTracker) {
             lsmIndex.addDiskComponent(c);
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(c);
-                triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+                triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
             }
             // Enter the component
             enterComponent(c);
@@ -304,14 +252,13 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-        callback.afterFinalize(ctx);
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        return NoOpIoOperation.INSTANCE;
     }
 
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        throw new UnsupportedOperationException();
     }
 
     @Override
@@ -354,7 +301,7 @@
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(diskComponent);
-                lsmIndex.scheduleReplication(null, componentsToBeReplicated, false, ReplicationOperation.DELETE, null);
+                lsmIndex.scheduleReplication(null, componentsToBeReplicated, ReplicationOperation.DELETE, null);
             }
             diskComponent.deactivateAndDestroy();
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
index 43d2b0d..880f5be 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FilterBulkLoader.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterManager;
 import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public class FilterBulkLoader implements IChainedComponentBulkLoader {
 
@@ -79,4 +80,19 @@
         filterTuple.reset(tuple);
         filter.update(filterTuple, filterCmp, NoOpOperationCallback.INSTANCE);
     }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return false;
+    }
+
+    @Override
+    public Throwable getFailure() {
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
index d835021..b2a2e48 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
@@ -23,11 +23,10 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 
-public abstract class FlushOperation extends AbstractIoOperation implements Comparable<ILSMIOOperation> {
+public abstract class FlushOperation extends AbstractIoOperation {
 
     public FlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
             String indexIdentifier) {
@@ -35,9 +34,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.flush(this);
-        return true;
+        return getStatus();
     }
 
     @Override
@@ -65,14 +64,6 @@
     }
 
     @Override
-    public int compareTo(ILSMIOOperation o) {
-        if (o instanceof FlushOperation) {
-            return target.getFile().getName().compareTo(((FlushOperation) o).getTarget().getFile().getName());
-        }
-        return -1;
-    }
-
-    @Override
     public boolean equals(Object o) {
         return (o instanceof FlushOperation)
                 && Objects.equals(target.getFile().getName(), ((FlushOperation) o).target.getFile().getName());
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
index 90ef127..1361c79 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IChainedComponentBulkLoader.java
@@ -20,8 +20,9 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
-public interface IChainedComponentBulkLoader {
+public interface IChainedComponentBulkLoader extends IPageWriteFailureCallback {
     /**
      * Adds a tuple to the bulkloaded component
      *
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexComponentFileReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexComponentFileReference.java
new file mode 100644
index 0000000..bbadf60
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexComponentFileReference.java
@@ -0,0 +1,131 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.DELIMITER;
+
+import java.util.Objects;
+
+import org.apache.hyracks.api.io.FileReference;
+
+public class IndexComponentFileReference implements Comparable<IndexComponentFileReference> {
+
+    private FileReference fileRef;
+    private String fullPath;
+    private String fileName;
+    private long sequenceStart;
+    private long sequenceEnd;
+
+    private IndexComponentFileReference() {
+    }
+
+    public static IndexComponentFileReference of(String file) {
+        final IndexComponentFileReference ref = new IndexComponentFileReference();
+        ref.fileName = file;
+        final String[] splits = file.split(DELIMITER);
+        ref.sequenceStart = Long.parseLong(splits[0]);
+        ref.sequenceEnd = Long.parseLong(splits[1]);
+        return ref;
+    }
+
+    public static IndexComponentFileReference of(FileReference fileRef) {
+        final IndexComponentFileReference ref = of(fileRef.getFile().getName());
+        ref.fileRef = fileRef;
+        ref.fullPath = fileRef.getFile().getAbsolutePath();
+        return ref;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        IndexComponentFileReference that = (IndexComponentFileReference) o;
+        return Objects.equals(fileName, that.fileName);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(fileName);
+    }
+
+    @Override
+    public int compareTo(IndexComponentFileReference o) {
+        int startCmp = Long.compare(sequenceStart, o.sequenceStart);
+        if (startCmp != 0) {
+            return startCmp;
+        }
+        return Long.compare(o.sequenceEnd, sequenceEnd);
+    }
+
+    public String getFileName() {
+        return fileName;
+    }
+
+    public long getSequenceStart() {
+        return sequenceStart;
+    }
+
+    public long getSequenceEnd() {
+        return sequenceEnd;
+    }
+
+    public String getFullPath() {
+        return fullPath;
+    }
+
+    public FileReference getFileRef() {
+        return fileRef;
+    }
+
+    public String getSequence() {
+        return sequenceStart + DELIMITER + sequenceEnd;
+    }
+
+    public boolean isMoreRecentThan(IndexComponentFileReference other) {
+        return sequenceStart > other.getSequenceEnd();
+    }
+
+    public boolean isWithin(IndexComponentFileReference other) {
+        return sequenceStart >= other.getSequenceStart() && sequenceEnd <= other.getSequenceEnd();
+    }
+
+    @Override
+    public String toString() {
+        return "{\"type\" : \"" + (isFlush() ? "flush" : "merge") + "\", \"start\" : \"" + sequenceStart
+                + "\", \"end\" : \"" + sequenceEnd + "\"}";
+    }
+
+    private boolean isFlush() {
+        return sequenceStart == sequenceEnd;
+    }
+
+    public static String getFlushSequence(long componentSequence) {
+        return componentSequence + DELIMITER + componentSequence;
+    }
+
+    public static String getMergeSequence(String firstComponentName, String lastComponentName) {
+        long mergeSequenceStart = IndexComponentFileReference.of(firstComponentName).getSequenceStart();
+        long mergeSequenceEnd = IndexComponentFileReference.of(lastComponentName).getSequenceEnd();
+        return mergeSequenceStart + DELIMITER + mergeSequenceEnd;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
index 4fb2919..394126d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public class IndexWithBuddyBulkLoader implements IChainedComponentBulkLoader {
 
@@ -69,4 +70,24 @@
         bulkLoader.abort();
         buddyBTreeBulkLoader.abort();
     }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return bulkLoader.hasFailed() || buddyBTreeBulkLoader.hasFailed();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        if (bulkLoader.hasFailed()) {
+            return bulkLoader.getFailure();
+        } else if (buddyBTreeBulkLoader.hasFailed()) {
+            return buddyBTreeBulkLoader.getFailure();
+        }
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
new file mode 100644
index 0000000..354b1af
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
@@ -0,0 +1,121 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public class IoOperationExecutor extends ThreadPoolExecutor {
+
+    private final ILSMIOOperationScheduler scheduler;
+    private final IIoOperationFailedCallback callback;
+    private final Map<String, ILSMIOOperation> runningFlushOperations;
+    private final Map<String, Throwable> failedGroups;
+    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations;
+
+    public IoOperationExecutor(ThreadFactory threadFactory, ILSMIOOperationScheduler scheduler,
+            IIoOperationFailedCallback callback, Map<String, ILSMIOOperation> runningFlushOperations,
+            Map<String, Deque<ILSMIOOperation>> waitingFlushOperations, Map<String, Throwable> failedGroups) {
+        super(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(), threadFactory);
+        this.scheduler = scheduler;
+        this.callback = callback;
+        this.runningFlushOperations = runningFlushOperations;
+        this.waitingFlushOperations = waitingFlushOperations;
+        this.failedGroups = failedGroups;
+    }
+
+    @Override
+    protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
+        return new LSMIOOperationTask<>(callable);
+    }
+
+    @Override
+    protected void afterExecute(Runnable r, Throwable t) {
+        super.afterExecute(r, t);
+        LSMIOOperationTask<?> task = (LSMIOOperationTask<?>) r;
+        ILSMIOOperation executedOp = task.getOperation();
+        try {
+            doAfterExecute(executedOp, t);
+        } catch (Throwable th) { // NOSONAR must catch all
+            callback.schedulerFailed(scheduler, th);
+            shutdown();
+        }
+    }
+
+    private void doAfterExecute(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        final boolean failed = (t != null) || (executedOp.getStatus() == LSMIOOperationStatus.FAILURE);
+        if (failed) {
+            fail(executedOp, t != null ? t : executedOp.getFailure());
+        }
+        if (!failed || executedOp.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+            executedOp.complete(); // destroy if merge or successful flush
+        }
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            String id = executedOp.getIndexIdentifier();
+            synchronized (this) {
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    ILSMIOOperation op = waitingFlushOperations.get(id).poll();
+                    if (op != null) {
+                        scheduler.scheduleOperation(op);
+                    } else {
+                        waitingFlushOperations.remove(id);
+                    }
+                }
+            }
+        }
+    }
+
+    private void fail(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+        callback.operationFailed(executedOp, t);
+        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            executedOp.complete();
+            // Doesn't make sense to process further flush requests... Mark the operation group permanently failed
+            // Fail other scheduled operations
+            synchronized (this) {
+                String id = executedOp.getIndexIdentifier();
+                failedGroups.put(id, t);
+                runningFlushOperations.remove(id);
+                if (waitingFlushOperations.containsKey(id)) {
+                    Deque<ILSMIOOperation> ops = waitingFlushOperations.remove(id);
+                    ILSMIOOperation next = ops.poll();
+                    while (next != null) {
+                        next.setFailure(new RuntimeException("Operation group " + id + " has permanently failed", t));
+                        next.setStatus(LSMIOOperationStatus.FAILURE);
+                        next.complete();
+                        next = ops.poll();
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java
index 4dd57eb..9346b56 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java
@@ -53,4 +53,10 @@
     public FileReference[] getFileReferences() {
         return new FileReference[] { insertIndexFileReference, deleteIndexFileReference, bloomFilterFileReference };
     }
+
+    @Override
+    public String toString() {
+        return "{ \"insert\" : \"" + insertIndexFileReference + "\", \"delete\" : \"" + deleteIndexFileReference
+                + "\", \"bloom\" : \"" + bloomFilterFileReference + "\"}";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
index 442af56..cf6c4a2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
@@ -24,12 +24,14 @@
 public class LSMComponentId implements ILSMComponentId {
 
     public static final long NOT_FOUND = -1;
+    public static final long MIN_VALID_COMPONENT_ID = 0;
 
-    // Use to handle legacy datasets which do not have the component Id
-    public static final ILSMComponentId MISSING_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
+    // Used to represent an empty index with no components
+    public static final LSMComponentId EMPTY_INDEX_LAST_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
 
     // A default component id used for bulk loaded component
-    public static final ILSMComponentId DEFAULT_COMPONENT_ID = new LSMComponentId(0, 0);
+    public static final LSMComponentId DEFAULT_COMPONENT_ID =
+            new LSMComponentId(MIN_VALID_COMPONENT_ID, MIN_VALID_COMPONENT_ID);
 
     private long minId;
 
@@ -46,10 +48,12 @@
         this.maxId = maxId;
     }
 
+    @Override
     public long getMinId() {
         return this.minId;
     }
 
+    @Override
     public long getMaxId() {
         return this.maxId;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
index e174153..21a27a9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
@@ -21,46 +21,40 @@
 
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.util.annotations.ThreadSafe;
 
 /**
  * A default implementation of {@link ILSMComponentIdGenerator}.
- *
  */
+@ThreadSafe
 public class LSMComponentIdGenerator implements ILSMComponentIdGenerator {
 
-    protected long previousTimestamp = -1L;
-
+    private final int numComponents;
+    private int currentComponentIndex;
+    private long lastUsedId;
     private ILSMComponentId componentId;
 
-    public LSMComponentIdGenerator() {
+    public LSMComponentIdGenerator(int numComponents, long lastUsedId) {
+        this.numComponents = numComponents;
+        this.lastUsedId = lastUsedId;
         refresh();
+        currentComponentIndex = 0;
     }
 
     @Override
-    public void refresh() {
-        long ts = getCurrentTimestamp();
-        componentId = new LSMComponentId(ts, ts);
+    public synchronized void refresh() {
+        final long nextId = ++lastUsedId;
+        componentId = new LSMComponentId(nextId, nextId);
+        currentComponentIndex = (currentComponentIndex + 1) % numComponents;
     }
 
     @Override
-    public ILSMComponentId getId() {
+    public synchronized ILSMComponentId getId() {
         return componentId;
     }
 
-    protected long getCurrentTimestamp() {
-        long timestamp = System.currentTimeMillis();
-        while (timestamp <= previousTimestamp) {
-            // make sure timestamp is strictly increasing
-            try {
-                Thread.sleep(1);
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-            }
-            timestamp = System.currentTimeMillis();
-        }
-        previousTimestamp = timestamp;
-        return timestamp;
-
+    @Override
+    public synchronized int getCurrentComponentIndex() {
+        return currentComponentIndex;
     }
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
deleted file mode 100644
index d288ec8..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.common.IResource;
-
-/**
- * A default implementation of {@link ILSMComponentIdGeneratorFactory}.
- *
- */
-public class LSMComponentIdGeneratorFactory implements ILSMComponentIdGeneratorFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
-        return new LSMComponentIdGenerator();
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 644508e..e9f6f20 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -42,8 +42,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -51,9 +52,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.util.IOOperationUtils;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.util.annotations.CriticalPath;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.ITracer.Scope;
 import org.apache.logging.log4j.Level;
@@ -64,6 +65,7 @@
     private static final Logger LOGGER = LogManager.getLogger();
 
     protected final ILSMIndex lsmIndex;
+    protected final ILSMIOOperationScheduler ioScheduler;
     protected final ComponentReplacementContext componentReplacementCtx;
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMOperationTracker opTracker;
@@ -73,9 +75,10 @@
     protected ITracer tracer;
     protected long traceCategory;
 
-    public LSMHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            boolean replicationEnabled, ITracer tracer) {
+    public LSMHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+            ILSMOperationTracker opTracker, boolean replicationEnabled, ITracer tracer) {
         this.lsmIndex = lsmIndex;
+        this.ioScheduler = ioScheduler;
         this.opTracker = opTracker;
         this.mergePolicy = mergePolicy;
         this.tracer = tracer;
@@ -100,58 +103,12 @@
             synchronized (opTracker) {
                 while (true) {
                     lsmIndex.getOperationalComponents(ctx);
-                    // Before entering the components, prune those corner cases that indeed should not proceed.
-                    switch (opType) {
-                        case FLUSH:
-                            // if the lsm index does not have memory components allocated, then nothing to flush
-                            if (!lsmIndex.isMemoryComponentsAllocated()) {
-                                return false;
-                            }
-                            ILSMMemoryComponent flushingComponent =
-                                    (ILSMMemoryComponent) ctx.getComponentHolder().get(0);
-                            if (!flushingComponent.isModified()) {
-                                recycle(flushingComponent);
-                                return false;
-                            }
-                            if (flushingComponent.getWriterCount() > 0) {
-                                /*
-                                 * This case is a case where even though FLUSH log was flushed to disk
-                                 * and scheduleFlush is triggered, the current in-memory component (whose state was
-                                 * changed to READABLE_WRITABLE (RW) from READABLE_UNWRITABLE(RU) before FLUSH log
-                                 * was written to log tail (which is memory buffer of log file) and then the state was
-                                 * changed back to RW (as shown in the following scenario)) can have writers based on
-                                 * the current code base/design. Thus, the writer count of the component may be greater
-                                 * than 0. if this happens, intead of throwing exception, scheduleFlush() deal with
-                                 * this situation by not flushing the component.
-                                 * for more detailed information: ASTERIXDB-1027
-                                 */
-                                return false;
-                            }
-                            break;
-                        case MERGE:
-                            if (ctx.getComponentHolder().size() < 2
-                                    && ctx.getOperation() != IndexOperation.DELETE_DISK_COMPONENTS) {
-                                // There is only a single component. There is nothing to merge.
-                                return false;
-                            }
-                            break;
-                        default:
-                            break;
-                    }
                     if (enterComponents(ctx, opType)) {
                         return true;
                     } else if (isTryOperation) {
                         return false;
                     }
                     try {
-                        // Flush and merge operations should never reach this wait call,
-                        // because they are always try operations. If they fail to enter the components,
-                        // then it means that there are an ongoing flush/merge operation on
-                        // the same components, so they should not proceed.
-                        if (opType == LSMOperationType.MODIFICATION) {
-                            // before waiting, make sure the index is in a modifiable state to avoid waiting forever.
-                            ensureIndexModifiable();
-                        }
                         opTracker.wait();
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
@@ -166,20 +123,7 @@
         }
     }
 
-    private void recycle(ILSMMemoryComponent flushingComponent) throws HyracksDataException {
-        if (flushingComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
-            //The mutable component has not been modified by any writer.
-            // There is nothing to flush. Since the component is empty, set its state back
-            // to READABLE_WRITABLE only when it's state has been set to READABLE_UNWRITABLE
-            flushingComponent.setState(ComponentState.READABLE_WRITABLE);
-            opTracker.notifyAll(); // NOSONAR: Always synchronized from caller
-            // Call recycled only when we change it's state is reset back to READABLE_WRITABLE
-            // Otherwise, if the component is in other state, e.g., INACTIVE, or
-            // READABLE_UNWRITABLE_FLUSHING, it's not considered as being recycled here.
-            lsmIndex.getIOOperationCallback().recycled(flushingComponent, false);
-        }
-    }
-
+    @CriticalPath
     protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType)
             throws HyracksDataException {
         validateOperationEnterComponentsState(ctx);
@@ -187,29 +131,29 @@
         int numEntered = 0;
         boolean entranceSuccessful = false;
         try {
-            for (ILSMComponent c : components) {
-                boolean isMutableComponent = numEntered == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
-                if (!c.threadEnter(opType, isMutableComponent)) {
+            final int componentsCount = components.size();
+            for (int i = 0; i < componentsCount; i++) {
+                final ILSMComponent component = components.get(i);
+                boolean isMutableComponent = numEntered == 0 && component.getType() == LSMComponentType.MEMORY;
+                if (!component.threadEnter(opType, isMutableComponent)) {
                     break;
                 }
                 numEntered++;
             }
             entranceSuccessful = numEntered == components.size();
         } catch (Throwable e) { // NOSONAR: Log and re-throw
-            if (LOGGER.isErrorEnabled()) {
-                LOGGER.log(Level.ERROR, opType.name() + " failed to enter components on " + lsmIndex, e);
-            }
+            LOGGER.warn("{} failed to enter components on {}", opType.name(), lsmIndex, e);
             throw e;
         } finally {
             if (!entranceSuccessful) {
-                int i = 0;
-                for (ILSMComponent c : components) {
+                final int componentsCount = components.size();
+                for (int i = 0; i < componentsCount; i++) {
+                    final ILSMComponent component = components.get(i);
                     if (numEntered == 0) {
                         break;
                     }
-                    boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
-                    c.threadExit(opType, true, isMutableComponent);
-                    i++;
+                    boolean isMutableComponent = i == 0 && component.getType() == LSMComponentType.MEMORY;
+                    component.threadExit(opType, true, isMutableComponent);
                     numEntered--;
                 }
             }
@@ -218,44 +162,26 @@
             return false;
         }
         ctx.setAccessingComponents(true);
-        // Check if there is any action that is needed to be taken based on the operation type
-        switch (opType) {
-            case FLUSH:
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                // Changing the flush status should *always* precede changing the mutable component.
-                lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
-                lsmIndex.changeMutableComponent();
-                // Notify all waiting threads whenever a flush has been scheduled since they will check
-                // again if they can grab and enter the mutable component.
-                opTracker.notifyAll(); // NOSONAR: Always called from a synchronized block
-                break;
-            case MERGE:
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                lsmIndex.getIOOperationCallback().beforeOperation(ctx);
-                break;
-            default:
-                break;
-        }
         opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
         return true;
     }
 
+    @CriticalPath
     private void doExitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
             ILSMDiskComponent newComponent, boolean failedOperation) throws HyracksDataException {
-        /**
+        /*
          * FLUSH and MERGE operations should always exit the components
          * to notify waiting threads.
          */
         if (!ctx.isAccessingComponents() && opType != LSMOperationType.FLUSH && opType != LSMOperationType.MERGE) {
             return;
         }
-        List<ILSMDiskComponent> inactiveDiskComponents = null;
+        List<ILSMDiskComponent> inactiveDiskComponents;
         List<ILSMDiskComponent> inactiveDiskComponentsToBeDeleted = null;
         try {
             synchronized (opTracker) {
                 try {
-                    /**
+                    /*
                      * [flow control]
                      * If merge operations are lagged according to the merge policy,
                      * flushing in-memory components are hold until the merge operation catches up.
@@ -273,9 +199,7 @@
                     ctx.setAccessingComponents(false);
                     exitOperation(ctx, opType, newComponent, failedOperation);
                 } catch (Throwable e) { // NOSONAR: Log and re-throw
-                    if (LOGGER.isErrorEnabled()) {
-                        LOGGER.log(Level.ERROR, e.getMessage(), e);
-                    }
+                    LOGGER.warn("Failure exiting components", e);
                     throw e;
                 } finally {
                     if (failedOperation && (opType == LSMOperationType.MODIFICATION
@@ -319,7 +243,7 @@
                 try {
                     //schedule a replication job to delete these inactive disk components from replicas
                     if (replicationEnabled) {
-                        lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted, false,
+                        lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted,
                                 ReplicationOperation.DELETE, opType);
                     }
                     for (ILSMDiskComponent c : inactiveDiskComponentsToBeDeleted) {
@@ -343,10 +267,12 @@
                 // newComponent is null if the flush op. was not performed.
                 if (!failedOperation && newComponent != null) {
                     lsmIndex.addDiskComponent(newComponent);
+                    // TODO: The following should also replicate component Id
+                    // even if empty component
                     if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
                         componentsToBeReplicated.clear();
                         componentsToBeReplicated.add(newComponent);
-                        triggerReplication(componentsToBeReplicated, false, opType);
+                        triggerReplication(componentsToBeReplicated, opType);
                     }
                     mergePolicy.diskComponentAdded(lsmIndex, false);
                 }
@@ -358,7 +284,7 @@
                     if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
                         componentsToBeReplicated.clear();
                         componentsToBeReplicated.add(newComponent);
-                        triggerReplication(componentsToBeReplicated, false, opType);
+                        triggerReplication(componentsToBeReplicated, opType);
                     }
                     mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
                 }
@@ -368,13 +294,16 @@
         }
     }
 
+    @CriticalPath
     private void exitOperationalComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
             boolean failedOperation) throws HyracksDataException {
         // First check if there is any action that is needed to be taken
         // based on the state of each component.
-        for (int i = 0; i < ctx.getComponentHolder().size(); i++) {
-            ILSMComponent c = ctx.getComponentHolder().get(i);
-            boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+        final List<ILSMComponent> componentHolder = ctx.getComponentHolder();
+        final int componentsCount = componentHolder.size();
+        for (int i = 0; i < componentsCount; i++) {
+            final ILSMComponent c = componentHolder.get(i);
+            boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY;
             c.threadExit(opType, failedOperation, isMutableComponent);
             if (c.getType() == LSMComponentType.MEMORY) {
                 switch (c.getState()) {
@@ -386,7 +315,6 @@
                         break;
                     case INACTIVE:
                         tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex.toString());
-                        ((AbstractLSMMemoryComponent) c).reset();
                         // Notify all waiting threads whenever the mutable component's state
                         // has changed to inactive. This is important because even though we switched
                         // the mutable components, it is possible that the component that we just
@@ -548,140 +476,155 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
-            ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-            callback.afterFinalize(ctx);
-            return;
+    public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation flush;
+        LOGGER.debug("Flush is being scheduled on {}", lsmIndex);
+        if (!lsmIndex.isMemoryComponentsAllocated()) {
+            lsmIndex.allocateMemoryComponents();
         }
-        lsmIndex.scheduleFlush(ctx, callback);
+        synchronized (opTracker) {
+            try {
+                flush = lsmIndex.createFlushOperation(ctx);
+            } finally {
+                // Notify all waiting threads whenever a flush has been scheduled since they will check
+                // again if they can grab and enter the mutable component.
+                opTracker.notifyAll();
+            }
+        }
+        ioScheduler.scheduleOperation(flush);
+        return flush;
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
-    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a flush operation for index: " + lsmIndex + " ...");
+    public void flush(ILSMIOOperation operation) throws HyracksDataException {
+        LOGGER.debug("Started a flush operation for index: {}", lsmIndex);
+        synchronized (opTracker) {
+            while (!enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH)) {
+                try {
+                    opTracker.wait();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    throw HyracksDataException.create(e);
+                }
+            }
         }
         try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.flush(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Flush failed on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.FLUSH, newComponent, failedOperation);
-                ctx.setIoOperationType(LSMIOOperationType.FLUSH);
-                operation.getCallback().afterFinalize(ctx);
+            doIo(operation);
+        } finally {
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
+        }
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Finished the flush operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
+        }
+    }
 
+    public void doIo(ILSMIOOperation operation) {
+        try {
+            operation.getCallback().beforeOperation(operation);
+            ILSMDiskComponent newComponent = operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+                    ? lsmIndex.flush(operation) : lsmIndex.merge(operation);
+            operation.setNewComponent(newComponent);
+            operation.getCallback().afterOperation(operation);
+            if (newComponent != null) {
+                newComponent.markAsValid(lsmIndex.isDurable(), operation);
+            }
+        } catch (Throwable e) { // NOSONAR Must catch all
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(e);
+            if (LOGGER.isErrorEnabled()) {
+                LOGGER.log(Level.ERROR, "{} operation failed on {}", operation.getIOOpertionType(), lsmIndex, e);
             }
         } finally {
-            /*
-             * Completion of flush/merge operations is done explicitly here to make sure all generated files during
-             * io operations is completed before the io operation is declared complete
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
+            try {
+                operation.getCallback().afterFinalize(operation);
+            } catch (Throwable th) {// NOSONAR Must catch all
+                operation.setStatus(LSMIOOperationStatus.FAILURE);
+                operation.setFailure(th);
+                if (LOGGER.isErrorEnabled()) {
+                    LOGGER.log(Level.ERROR, "{} operation.afterFinalize failed on {}", operation.getIOOpertionType(),
+                            lsmIndex, th);
+                }
+            }
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the flush operation for index: " + lsmIndex);
+        // if the operation failed, we need to cleanup files
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            lsmIndex.cleanUpFilesForFailedOperation(operation);
         }
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            LOGGER.info("Failed to enter components for merge operation. Calling finalize");
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
+    public void merge(ILSMIOOperation operation) throws HyracksDataException {
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Started a merge operation for index: {}", lsmIndex);
         }
-        lsmIndex.scheduleMerge(ctx, callback);
+        synchronized (opTracker) {
+            enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE);
+        }
+        try {
+            doIo(operation);
+        } finally {
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, operation.getNewComponent(),
+                    operation.getStatus() == LSMIOOperationStatus.FAILURE);
+            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE,
+                    operation.getAccessor().getOpContext().getSearchOperationCallback(),
+                    operation.getAccessor().getOpContext().getModificationCallback());
+        }
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Finished the merge operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
+        }
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        fullMergeIsRequested.set(true);
-        if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+    public ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            operation = lsmIndex.createMergeOperation(ctx);
+        }
+        ioScheduler.scheduleOperation(operation);
+        return operation;
+    }
+
+    @Override
+    public ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        ILSMIOOperation operation;
+        synchronized (opTracker) {
+            fullMergeIsRequested.set(true);
+            ctx.getComponentsToBeMerged().addAll(lsmIndex.getDiskComponents());
+            operation = lsmIndex.createMergeOperation(ctx);
+            if (operation != NoOpIoOperation.INSTANCE) {
+                fullMergeIsRequested.set(false);
+            }
             // If the merge cannot be scheduled because there is already an ongoing merge on
             // subset/all of the components, then whenever the current merge has finished,
             // it will schedule the full merge again.
-            ctx.setIoOperationType(LSMIOOperationType.MERGE);
-            callback.afterFinalize(ctx);
-            return;
         }
-        fullMergeIsRequested.set(false);
-        lsmIndex.scheduleMerge(ctx, callback);
+        ioScheduler.scheduleOperation(operation);
+        return operation;
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
-    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
-        }
+    public void addBulkLoadedComponent(ILSMIOOperation ioOperation) throws HyracksDataException {
+        ILSMDiskComponent c = ioOperation.getNewComponent();
         try {
-            ILSMDiskComponent newComponent = null;
-            boolean failedOperation = false;
-            try {
-                newComponent = lsmIndex.merge(operation);
-                ctx.setNewComponent(newComponent);
-                ctx.setIoOperationType(LSMIOOperationType.MERGE);
-                operation.getCallback().afterOperation(ctx);
-                newComponent.markAsValid(lsmIndex.isDurable());
-            } catch (Throwable e) { // NOSONAR: Log and re-throw
-                failedOperation = true;
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.log(Level.ERROR, "Failed merge operation on " + lsmIndex, e);
-                }
-                throw e;
-            } finally {
-                exitComponents(ctx, LSMOperationType.MERGE, newComponent, failedOperation);
-                operation.getCallback().afterFinalize(ctx);
-            }
-        } finally {
-            /*
-             * Completion of the merge operation is called here to and not on afterOperation because
-             * deletion of old components comes after afterOperation is called and the number of
-             * io operation should not be decremented before the operation is complete to avoid
-             * index destroy from competing with the merge on deletion of the files.
-             * The order becomes:
-             * 1. scheduleMerge
-             * 2. enterComponents
-             * 3. beforeOperation (increment the numOfIoOperations)
-             * 4. merge
-             * 5. exitComponents
-             * 6. afterOperation (no op)
-             * 7. delete components
-             * 8. completeOperation (decrement the numOfIoOperations)
-             */
-            opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE, ctx.getSearchOperationCallback(),
-                    ctx.getModificationCallback());
+            c.markAsValid(lsmIndex.isDurable(), ioOperation);
+        } catch (Throwable th) {
+            ioOperation.setFailure(th);
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Finished the merge operation for index: " + lsmIndex);
+        if (ioOperation.hasFailed()) {
+            throw HyracksDataException.create(ioOperation.getFailure());
         }
-    }
-
-    @Override
-    public void addBulkLoadedComponent(ILSMDiskComponent c) throws HyracksDataException {
-        c.markAsValid(lsmIndex.isDurable());
         synchronized (opTracker) {
             lsmIndex.addDiskComponent(c);
             if (replicationEnabled) {
                 componentsToBeReplicated.clear();
                 componentsToBeReplicated.add(c);
-                triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+                triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
             }
             mergePolicy.diskComponentAdded(lsmIndex, false);
         }
@@ -692,20 +635,20 @@
         return opTracker;
     }
 
-    protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleReplication(lsmComponents, bulkload, opType);
+        accessor.scheduleReplication(lsmComponents, opType);
     }
 
     @Override
     public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
-            boolean bulkload, LSMOperationType opType) throws HyracksDataException {
+            LSMOperationType opType) throws HyracksDataException {
         //enter the LSM components to be replicated to prevent them from being deleted until they are replicated
         if (!getAndEnterComponents(ctx, LSMOperationType.REPLICATE, false)) {
             return;
         }
-        lsmIndex.scheduleReplication(ctx, lsmComponents, bulkload, ReplicationOperation.REPLICATE, opType);
+        lsmIndex.scheduleReplication(ctx, lsmComponents, ReplicationOperation.REPLICATE, opType);
     }
 
     @Override
@@ -757,13 +700,14 @@
             try {
                 processFrame(accessor, tuple, processor);
                 frameOpCallback.frameCompleted();
+            } catch (Throwable th) {
+                processor.fail(th);
+                throw th;
             } finally {
                 processor.finish();
             }
         } catch (HyracksDataException e) {
-            if (LOGGER.isErrorEnabled()) {
-                LOGGER.log(Level.ERROR, "Failed to process frame", e);
-            }
+            LOGGER.warn("Failed to process frame", e);
             throw e;
         } finally {
             exit(ctx);
@@ -771,36 +715,6 @@
         }
     }
 
-    /***
-     * Ensures the index is in a modifiable state (no failed flushes)
-     *
-     * @throws HyracksDataException
-     *             if the index is not in a modifiable state
-     */
-    private void ensureIndexModifiable() throws HyracksDataException {
-        // if current memory component has a flush request, it means that flush didn't start for it
-        if (lsmIndex.hasFlushRequestForCurrentMutableComponent()) {
-            return;
-        }
-        // find if there is any memory component which is in a writable state or eventually will be in a writable state
-        for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
-            switch (memoryComponent.getState()) {
-                case INACTIVE:
-                    // will be activated on next modification
-                case UNREADABLE_UNWRITABLE:
-                    // flush completed successfully but readers are still inside
-                case READABLE_WRITABLE:
-                    // writable
-                case READABLE_UNWRITABLE_FLUSHING:
-                    // flush is ongoing
-                    return;
-                default:
-                    // continue to the next component
-            }
-        }
-        throw HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
-    }
-
     /**
      * Waits for any lagging merge operations to finish to avoid breaking
      * the merge policy (i.e. adding a new disk component can make the
@@ -824,51 +738,52 @@
         }
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
     public void deleteComponents(ILSMIndexOperationContext ctx, Predicate<ILSMComponent> predicate)
             throws HyracksDataException {
-        BlockingIOOperationCallbackWrapper ioCallback =
-                new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
-        boolean deleteMemoryComponent = false;
+        ILSMIOOperation ioOperation = null;
+        // We need to always start the component delete from current memory component.
+        // This will ensure Primary and secondary component id still matches after component delete
+        if (!lsmIndex.isMemoryComponentsAllocated()) {
+            lsmIndex.allocateMemoryComponents();
+        }
         synchronized (opTracker) {
             waitForFlushesAndMerges();
-            ensureNoFailedFlush();
-            if (lsmIndex.isMemoryComponentsAllocated()) {
-                // We always start with the memory component
-                ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
-                deleteMemoryComponent = predicate.test(memComponent);
-                if (deleteMemoryComponent) {
-                    // schedule a delete for flushed component
-                    ctx.reset();
-                    ctx.setOperation(IndexOperation.DELETE_MEMORY_COMPONENT);
-                    // ScheduleFlush is actually a try operation
-                    scheduleFlush(ctx, ioCallback);
-                } else {
-                    // shouldn't try to delete disk components while memory component is still there
-                    return;
-                }
+            // We always start with the memory component
+            ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+            if (predicate.test(memComponent)) {
+                // schedule a delete for flushed component
+                ctx.reset();
+                ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
+                ioOperation = scheduleFlush(ctx);
+            } else {
+                // since we're not deleting the memory component, we can't delete any previous component
+                return;
             }
         }
         // Here, we are releasing the opTracker to allow other operations:
         // (searches, delete flush we will schedule, delete merge we will schedule).
-        if (deleteMemoryComponent) {
-            IOOperationUtils.waitForIoOperation(ioCallback);
+        try {
+            ioOperation.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(ioOperation.getFailure());
         }
         ctx.reset();
-        ioCallback = new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
-        ctx.setOperation(IndexOperation.DELETE_DISK_COMPONENTS);
+        ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
         List<ILSMDiskComponent> toBeDeleted;
         synchronized (opTracker) {
             waitForFlushesAndMerges();
-            // Ensure that current memory component is empty and that no failed flushes happened so far
-            // This is a workaround until ASTERIXDB-2106 is fixed
-            ensureNoFailedFlush();
             List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
             for (ILSMDiskComponent component : diskComponents) {
                 if (predicate.test(component)) {
                     ctx.getComponentsToBeMerged().add(component);
                 } else {
-                    // can't delete components while newer ones are still there
+                    // Can't delete older components when newer one is still there
                     break;
                 }
             }
@@ -877,9 +792,17 @@
             }
             toBeDeleted = new ArrayList<>(ctx.getComponentsToBeMerged());
             // ScheduleMerge is actually a try operation
-            scheduleMerge(ctx, ioCallback);
+            ioOperation = scheduleMerge(ctx);
         }
-        IOOperationUtils.waitForIoOperation(ioCallback);
+        try {
+            ioOperation.sync();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw HyracksDataException.create(e);
+        }
+        if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(ioOperation.getFailure());
+        }
         synchronized (opTracker) {
             // ensure that merge has succeeded
             for (ILSMDiskComponent component : toBeDeleted) {
@@ -890,21 +813,6 @@
         }
     }
 
-    /**
-     * This can only be called in the steady state where:
-     * 1. no scheduled flushes
-     * 2. no incoming data
-     *
-     * @throws HyracksDataException
-     */
-    private void ensureNoFailedFlush() throws HyracksDataException {
-        for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
-            if (memoryComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
-                throw HyracksDataException.create(ErrorCode.A_FLUSH_OPERATION_HAS_FAILED);
-            }
-        }
-    }
-
     private void waitForFlushesAndMerges() throws HyracksDataException {
         while (flushingOrMerging()) {
             try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 84857f4..977697b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public class LSMIndexBulkLoader implements IChainedComponentBulkLoader {
     private final IIndexBulkLoader bulkLoader;
@@ -64,4 +65,19 @@
     public void abort() throws HyracksDataException {
         bulkLoader.abort();
     }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return bulkLoader.hasFailed();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        return bulkLoader.getFailure();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
index 5e105a4..c739ad0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -22,63 +22,114 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 
 public class LSMIndexDiskComponentBulkLoader implements IIndexBulkLoader {
     private final AbstractLSMIndex lsmIndex;
     private final ILSMDiskComponentBulkLoader componentBulkLoader;
-    private ILSMIndexOperationContext opCtx;
+    private final ILSMIndexOperationContext opCtx;
+    private boolean failed = false;
 
     public LSMIndexDiskComponentBulkLoader(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext opCtx, float fillFactor,
             boolean verifyInput, long numElementsHint) throws HyracksDataException {
         this.lsmIndex = lsmIndex;
         this.opCtx = opCtx;
-        // Note that by using a flush target file name, we state that the
-        // new bulk loaded component is "newer" than any other merged component.
-        opCtx.setNewComponent(lsmIndex.createBulkLoadTarget());
-        this.componentBulkLoader =
-                opCtx.getNewComponent().createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+        this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
+                fillFactor, verifyInput, numElementsHint, false, true, true);
     }
 
     public ILSMDiskComponent getComponent() {
-        return opCtx.getNewComponent();
+        return opCtx.getIoOperation().getNewComponent();
     }
 
+    @SuppressWarnings("squid:S1181")
     @Override
     public void add(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.add(tuple);
+        try {
+            componentBulkLoader.add(tuple);
+        } catch (Throwable th) {
+            opCtx.getIoOperation().setFailure(th);
+            throw th;
+        }
     }
 
+    @SuppressWarnings("squid:S1181")
     public void delete(ITupleReference tuple) throws HyracksDataException {
-        componentBulkLoader.delete(tuple);
+        try {
+            componentBulkLoader.delete(tuple);
+        } catch (Throwable th) {
+            opCtx.getIoOperation().setFailure(th);
+            throw th;
+        }
     }
 
     @Override
     public void end() throws HyracksDataException {
         try {
-            componentBulkLoader.end();
-            if (opCtx.getNewComponent().getComponentSize() > 0) {
-                //TODO(amoudi): Ensure Bulk load follow the same lifecycle Other Operations (Flush, Merge, etc).
-                //then after operation should be called from harness as well
-                //https://issues.apache.org/jira/browse/ASTERIXDB-1764
-                lsmIndex.getIOOperationCallback().afterOperation(opCtx);
-                lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getNewComponent());
-            }
+            presistComponentToDisk();
+        } catch (Throwable th) { // NOSONAR must cleanup in case of any failure
+            fail(th);
+            throw th;
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
     @Override
     public void abort() throws HyracksDataException {
+        opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
         try {
-            componentBulkLoader.abort();
-            opCtx.setNewComponent(null);
-            lsmIndex.getIOOperationCallback().afterOperation(opCtx);
+            try {
+                componentBulkLoader.abort();
+            } finally {
+                lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+            }
         } finally {
-            lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+            lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
         }
     }
 
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return opCtx.getIoOperation().hasFailed();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        return opCtx.getIoOperation().getFailure();
+    }
+
+    private void presistComponentToDisk() throws HyracksDataException {
+        try {
+            lsmIndex.getIOOperationCallback().afterOperation(opCtx.getIoOperation());
+            componentBulkLoader.end();
+        } catch (Throwable th) { // NOSONAR Must not call afterFinalize without setting failure
+            fail(th);
+            throw th;
+        } finally {
+            lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+        }
+        if (opCtx.getIoOperation().getStatus() == LSMIOOperationStatus.SUCCESS
+                && opCtx.getIoOperation().getNewComponent().getComponentSize() > 0) {
+            lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getIoOperation());
+        }
+    }
+
+    private void fail(Throwable th) {
+        if (!failed) {
+            failed = true;
+            final ILSMIOOperation loadOp = opCtx.getIoOperation();
+            loadOp.setFailure(th);
+            lsmIndex.cleanUpFilesForFailedOperation(loadOp);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 445a005..12caec4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -125,7 +125,6 @@
                     rangeCursors[i].close();
                 }
             }
-            rangeCursors = null;
         } finally {
             if (lsmHarness != null) {
                 lsmHarness.endSearch(opCtx);
@@ -247,9 +246,9 @@
         }
     }
 
-    public class PriorityQueueElement {
+    public static class PriorityQueueElement {
         private ITupleReference tuple;
-        private int cursorIndex;
+        private final int cursorIndex;
 
         public PriorityQueueElement(int cursorIndex) {
             tuple = null;
@@ -269,7 +268,7 @@
         }
     }
 
-    public class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
+    public static class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
 
         protected MultiComparator cmp;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 1548f86..8412b8c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -120,12 +119,12 @@
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
@@ -135,33 +134,32 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
-    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ctx.setOperation(IndexOperation.REPLICATE);
         ctx.getComponentsToBeReplicated().clear();
         ctx.getComponentsToBeReplicated().addAll(lsmComponents);
-        lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+        lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
new file mode 100644
index 0000000..21c52d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
@@ -0,0 +1,62 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+
+public class LoadOperation extends AbstractIoOperation {
+
+    private final LSMComponentFileReferences fileReferences;
+    private final Map<String, Object> parameters;
+
+    public LoadOperation(LSMComponentFileReferences fileReferences, ILSMIOOperationCallback callback,
+            String indexIdentifier, Map<String, Object> parameters) {
+        super(null, fileReferences.getInsertIndexFileReference(), callback, indexIdentifier);
+        this.fileReferences = fileReferences;
+        this.parameters = parameters;
+    }
+
+    @Override
+    public final LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.LOAD;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        return fileReferences;
+    }
+
+    @Override
+    public void sync() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return parameters;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MemoryComponentMetadata.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MemoryComponentMetadata.java
index e73fa0a..0c2167f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MemoryComponentMetadata.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MemoryComponentMetadata.java
@@ -29,7 +29,6 @@
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -95,11 +94,11 @@
     public void copy(IMetadataPageManager mdpManager) throws HyracksDataException {
         lock.readLock().lock();
         try {
-            LOGGER.log(Level.INFO, "Copying Metadata into a different component");
+            LOGGER.trace("Copying Metadata into a different component");
             ITreeIndexMetadataFrame frame = mdpManager.createMetadataFrame();
             for (Pair<IValueReference, ArrayBackedValueStorage> pair : store) {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.log(Level.INFO, "Copying " + pair.getKey() + " : " + pair.getValue().getLength() + " bytes");
+                if (LOGGER.isTraceEnabled()) {
+                    LOGGER.trace("Copying " + pair.getKey() + " : " + pair.getValue().getLength() + " bytes");
                 }
                 mdpManager.put(frame, pair.getKey(), pair.getValue());
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
index ec2305d..9d7c449 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
@@ -41,9 +41,9 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         accessor.merge(this);
-        return true;
+        return getStatus();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index 9b25471..9bd5007 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -220,4 +220,8 @@
         vbc.deleteFile(file);
     }
 
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
index 8ce636b..3d07a67 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoMergePolicyFactory.java
@@ -25,9 +25,14 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class NoMergePolicyFactory implements ILSMMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
@@ -51,4 +56,14 @@
         policy.configure(configuration);
         return policy;
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new NoMergePolicyFactory();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
index 3432624..a05838e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
@@ -20,15 +20,24 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.IResource;
 
-public enum NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-    INSTANCE;
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final NoOpIOOperationCallbackFactory INSTANCE = new NoOpIOOperationCallbackFactory();
+
+    private NoOpIOOperationCallbackFactory() {
+    }
 
     @Override
     public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) {
@@ -40,6 +49,16 @@
         // No op
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
+
     public static class NoOpIOOperationCallback implements ILSMIOOperationCallback {
         private static final NoOpIOOperationCallback INSTANCE = new NoOpIOOperationCallback();
 
@@ -47,22 +66,7 @@
         }
 
         @Override
-        public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-            // Do nothing.
-        }
-
-        @Override
-        public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
+        public void recycled(ILSMMemoryComponent component) {
             // Do nothing.
         }
 
@@ -70,5 +74,35 @@
         public void allocated(ILSMMemoryComponent component) {
             // Do nothing.
         }
+
+        @Override
+        public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+            // Do nothing.
+        }
+
+        @Override
+        public void completed(ILSMIOOperation operation) {
+            // Do nothing.
+        }
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return 0;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
new file mode 100644
index 0000000..5ee1503
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
@@ -0,0 +1,140 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+public class NoOpIoOperation implements ILSMIOOperation {
+    public static final NoOpIoOperation INSTANCE = new NoOpIoOperation();
+
+    private NoOpIoOperation() {
+    }
+
+    @Override
+    public IODeviceHandle getDevice() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getIndexIdentifier() {
+        return NoOpIoOperation.class.getSimpleName();
+    }
+
+    @Override
+    public LSMIOOperationType getIOOpertionType() {
+        return LSMIOOperationType.NOOP;
+    }
+
+    @Override
+    public LSMIOOperationStatus call() throws HyracksDataException {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public FileReference getTarget() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ILSMIndexAccessor getAccessor() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public LSMComponentFileReferences getComponentFiles() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Throwable getFailure() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        // No Op
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return LSMIOOperationStatus.SUCCESS;
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        // No Op
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        // No Op
+    }
+
+    @Override
+    public void complete() {
+        // No Op
+    }
+
+    @Override
+    public void sync() {
+        // No Op
+    }
+
+    @Override
+    public void addCompleteListener(IoOperationCompleteListener listener) {
+        listener.completed(this);
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return null;
+    }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
new file mode 100644
index 0000000..3b2b2c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class NoOpIoOperationFailedCallback implements IIoOperationFailedCallback {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    public static final NoOpIoOperationFailedCallback INSTANCE = new NoOpIoOperationFailedCallback();
+
+    private NoOpIoOperationFailedCallback() {
+    }
+
+    @Override
+    public void operationFailed(ILSMIOOperation operation, Throwable t) {
+        LOGGER.error("Operation {} failed", operation, t);
+    }
+
+    @Override
+    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+        LOGGER.error("IO Scheduler failed", failure);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
index 73b5d20..b571e58 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
@@ -31,7 +31,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 
 public class PrefixMergePolicy implements ILSMMergePolicy {
@@ -59,8 +58,7 @@
         }
 
         if (fullMergeIsRequested) {
-            ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFullMerge(index.getIOOperationCallback());
+            index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFullMerge();
             return;
         }
 
@@ -244,8 +242,7 @@
 
         // Reverse the components order back to its original order
         Collections.reverse(mergableComponents);
-        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        accessor.scheduleMerge(index.getIOOperationCallback(), mergableComponents);
+        index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleMerge(mergableComponents);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
index bae551b..3480eaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicyFactory.java
@@ -25,9 +25,14 @@
 import java.util.Set;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class PrefixMergePolicyFactory implements ILSMMergePolicyFactory {
 
     private static final long serialVersionUID = 1L;
@@ -52,4 +57,14 @@
         policy.configure(configuration);
         return policy;
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return new PrefixMergePolicyFactory();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
deleted file mode 100644
index 2c16be0..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-/**
- * This class is for testing. It's basically a way to get the new/old component info from the
- * harness callback simply.
- */
-
-public class StubIOOperationCallback implements ILSMIOOperationCallback {
-
-    private ILSMIndexOperationContext opCtx = null;
-
-    @Override
-    public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Not interested in this
-    }
-
-    @Override
-    public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        this.opCtx = opCtx;
-    }
-
-    @Override
-    public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
-        // Redundant info from after
-    }
-
-    public List<ILSMDiskComponent> getLastOldComponents() {
-        return opCtx.getComponentsToBeMerged();
-    }
-
-    public ILSMDiskComponent getLastNewComponent() {
-        return opCtx.getNewComponent();
-    }
-
-    @Override
-    public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
-        // Not interested in this
-    }
-
-    @Override
-    public void allocated(ILSMMemoryComponent component) {
-        // Not interested in this
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
index 506dcea..ae3b4e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -18,8 +18,13 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -27,18 +32,76 @@
 
 public class SynchronousScheduler implements ILSMIOOperationScheduler {
     private static final Logger LOGGER = LogManager.getLogger();
-    public static final SynchronousScheduler INSTANCE = new SynchronousScheduler();
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new ConcurrentHashMap<>();
+    private final Map<String, Throwable> failedGroups = new ConcurrentHashMap<>();
+    private final IIoOperationFailedCallback failureCallback;
 
-    private SynchronousScheduler() {
+    public SynchronousScheduler(IIoOperationFailedCallback failureCallback) {
+        this.failureCallback = failureCallback;
     }
 
     @Override
-    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+    public void scheduleOperation(ILSMIOOperation operation) {
+        try {
+            before(operation);
+            if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                return;
+            }
+            run(operation);
+        } catch (Throwable e) { // NOSONAR: Must catch them all
+            throw new IllegalStateException(e);
+        } finally {
+            after(operation);
+        }
+    }
+
+    private void run(ILSMIOOperation operation) {
         try {
             operation.call();
-        } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "IO Operation failed", e);
-            throw HyracksDataException.create(e);
+        } catch (Throwable th) { // NOSONAR Must catch all
+            LOGGER.log(Level.ERROR, "IO Operation failed", th);
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(th);
+        }
+        if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+            failureCallback.operationFailed(operation, operation.getFailure());
+        }
+    }
+
+    private void after(ILSMIOOperation operation) {
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                runningFlushOperations.remove(operation.getIndexIdentifier());
+                if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    failedGroups.putIfAbsent(operation.getIndexIdentifier(), operation.getFailure());
+                }
+                operation.complete();
+                runningFlushOperations.notifyAll();
+            }
+        } else {
+            operation.complete();
+        }
+    }
+
+    private void before(ILSMIOOperation operation) throws InterruptedException {
+        String id = operation.getIndexIdentifier();
+        if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+            synchronized (runningFlushOperations) {
+                while (true) {
+                    if (failedGroups.containsKey(id)) {
+                        operation.setStatus(LSMIOOperationStatus.FAILURE);
+                        operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+                                failedGroups.get(id)));
+                        return;
+                    }
+                    if (runningFlushOperations.containsKey(id)) {
+                        runningFlushOperations.wait();
+                    } else {
+                        runningFlushOperations.put(id, operation);
+                        break;
+                    }
+                }
+            }
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
index 823db04..132d5cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -22,12 +22,16 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 
-public enum SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
-    INSTANCE;
+public class SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
+    private static final long serialVersionUID = 1L;
+    public static final SynchronousSchedulerProvider INSTANCE = new SynchronousSchedulerProvider();
+
+    private SynchronousSchedulerProvider() {
+    }
 
     @Override
     public ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx) {
-        return SynchronousScheduler.INSTANCE;
+        return new SynchronousScheduler(NoOpIoOperationFailedCallback.INSTANCE);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index a7fbe06..07eeb28 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -58,7 +58,7 @@
         if (opType == LSMOperationType.MODIFICATION && threadRefCount.decrementAndGet() == 0
                 && index.hasFlushRequestForCurrentMutableComponent()) {
             ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            accessor.scheduleFlush(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback(index));
+            accessor.scheduleFlush();
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 7238f8e..3345e3a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -19,12 +19,17 @@
 
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
+import java.util.Map;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.util.trace.ITracer;
 import org.apache.hyracks.util.trace.ITracer.Scope;
 import org.apache.hyracks.util.trace.TraceUtils;
@@ -53,10 +58,7 @@
         if (tracer.isEnabled(traceCategory)) {
             tracer.instant("schedule-" + ioOpName, traceCategory, Scope.p,
                     "{\"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
-        }
-        if (tracer.isEnabled(traceCategory)) {
-            return ioOp instanceof Comparable ? new ComparableTracedIOOperation(ioOp, tracer, traceCategory)
-                    : new TracedIOOperation(ioOp, tracer, traceCategory);
+            return new TracedIOOperation(ioOp, tracer, traceCategory);
         }
         return ioOp;
     }
@@ -86,7 +88,7 @@
     }
 
     @Override
-    public Boolean call() throws HyracksDataException {
+    public LSMIOOperationStatus call() throws HyracksDataException {
         final String name = getTarget().getRelativePath();
         final long tid = tracer.durationB(name, traceCategory, null);
         try {
@@ -111,33 +113,64 @@
     public LSMComponentFileReferences getComponentFiles() {
         return ioOp.getComponentFiles();
     }
+
+    @Override
+    public Throwable getFailure() {
+        return ioOp.getFailure();
+    }
+
+    @Override
+    public void setFailure(Throwable failure) {
+        ioOp.setFailure(failure);
+    }
+
+    @Override
+    public LSMIOOperationStatus getStatus() {
+        return ioOp.getStatus();
+    }
+
+    @Override
+    public void setStatus(LSMIOOperationStatus status) {
+        ioOp.setStatus(status);
+    }
+
+    @Override
+    public ILSMDiskComponent getNewComponent() {
+        return ioOp.getNewComponent();
+    }
+
+    @Override
+    public void setNewComponent(ILSMDiskComponent component) {
+        ioOp.setNewComponent(component);
+    }
+
+    @Override
+    public void complete() {
+        ioOp.complete();
+    }
+
+    @Override
+    public void sync() throws InterruptedException {
+        ioOp.sync();
+    }
+
+    @Override
+    public void addCompleteListener(IoOperationCompleteListener listener) {
+        ioOp.addCompleteListener(listener);
+    }
+
+    @Override
+    public Map<String, Object> getParameters() {
+        return ioOp.getParameters();
+    }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        ioOp.writeFailed(page, failure);
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return ioOp.hasFailed();
+    }
 }
-
-class ComparableTracedIOOperation extends TracedIOOperation implements Comparable<ILSMIOOperation> {
-
-    protected ComparableTracedIOOperation(ILSMIOOperation ioOp, ITracer trace, long traceCategory) {
-        super(ioOp, trace, traceCategory);
-    }
-
-    @Override
-    public int hashCode() {
-        return this.ioOp.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        return other instanceof ILSMIOOperation && compareTo((ILSMIOOperation) other) == 0;
-    }
-
-    @Override
-    public int compareTo(ILSMIOOperation other) {
-        final ILSMIOOperation myIoOp = this.ioOp;
-        if (myIoOp instanceof Comparable && other instanceof ComparableTracedIOOperation) {
-            return ((Comparable) myIoOp).compareTo(((ComparableTracedIOOperation) other).getIoOp());
-        }
-        LOGGER.warn("Comparing ioOps of type " + myIoOp.getClass().getSimpleName() + " and "
-                + other.getClass().getSimpleName() + " in " + getClass().getSimpleName());
-        return Integer.signum(hashCode() - other.hashCode());
-    }
-
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index ef9852d..0a7c842 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -40,7 +40,6 @@
 import org.apache.hyracks.storage.common.file.FileMapManager;
 import org.apache.hyracks.storage.common.file.IFileMapManager;
 import org.apache.hyracks.util.JSONUtil;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -126,8 +125,8 @@
     }
 
     private void logStats() {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO, "Free (allocated) pages = " + freePages.size() + ". Budget = " + pageBudget
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Free (allocated) pages = " + freePages.size() + ". Budget = " + pageBudget
                     + ". Large pages = " + largePages.get() + ". Overall usage = " + used.get());
         }
     }
@@ -188,8 +187,8 @@
                 bucket.bucketLock.unlock();
             }
         }
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.log(Level.INFO, "Reclaimed pages = " + reclaimedPages);
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("Reclaimed pages = " + reclaimedPages);
         }
         logStats();
     }
@@ -462,4 +461,10 @@
         map.put("open", open);
         return map;
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
index 4b7f338..1ff9fa8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -190,12 +191,14 @@
         }
     }
 
-    public static void markAsValid(ITreeIndex treeIndex, boolean forceToDisk) throws HyracksDataException {
+    public static void markAsValid(ITreeIndex treeIndex, boolean forceToDisk, IPageWriteFailureCallback callback)
+            throws HyracksDataException {
         int fileId = treeIndex.getFileId();
         IBufferCache bufferCache = treeIndex.getBufferCache();
-        treeIndex.getPageManager().close();
-        // WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page
-        // won't be flushed to disk because it won't be dirty until the write latch has been released.
+        treeIndex.getPageManager().close(callback);
+        if (callback.hasFailed()) {
+            throw HyracksDataException.create(callback.getFailure());
+        }
         // Force modified metadata page to disk.
         // If the index is not durable, then the flush is not necessary.
         if (forceToDisk) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
deleted file mode 100644
index 9bd873d..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.util;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class IOOperationUtils {
-    private static final Logger LOGGER = LogManager.getLogger();
-
-    private IOOperationUtils() {
-    }
-
-    public static void waitForIoOperation(BlockingIOOperationCallbackWrapper ioCallback) throws HyracksDataException {
-        // Note that the following call assumes that the io operation has succeeded.
-        try {
-            ioCallback.waitForIO();
-        } catch (InterruptedException e) {
-            LOGGER.log(Level.WARN, "Operation has been interrupted. returning");
-            Thread.currentThread().interrupt();
-            throw HyracksDataException.create(e);
-        }
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
index 6d4b0a7..1779527 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
@@ -25,8 +25,11 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class LSMComponentIdUtils {
+    private static final Logger LOGGER = LogManager.getLogger();
 
     private static final MutableArrayValueReference COMPONENT_ID_MIN_KEY =
             new MutableArrayValueReference("Component_Id_Min".getBytes());
@@ -43,7 +46,9 @@
         long minId = ComponentUtils.getLong(metadata, COMPONENT_ID_MIN_KEY, LSMComponentId.NOT_FOUND, buffer);
         long maxId = ComponentUtils.getLong(metadata, COMPONENT_ID_MAX_KEY, LSMComponentId.NOT_FOUND, buffer);
         if (minId == LSMComponentId.NOT_FOUND || maxId == LSMComponentId.NOT_FOUND) {
-            return LSMComponentId.MISSING_COMPONENT_ID;
+            LOGGER.warn("Invalid component id {} was persisted to a component metadata",
+                    LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
+            return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
         } else {
             return new LSMComponentId(minId, maxId);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
index 5259a7c..f880e66 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -92,5 +92,9 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/InvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/InvertedListCursor.java
index 9db7dc8..b6d5902 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/InvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/InvertedListCursor.java
@@ -43,7 +43,7 @@
         int invListEndPageId = LSMInvertedIndexSearchCursorInitialState.INVALID_VALUE;
         int invListStartOffset = LSMInvertedIndexSearchCursorInitialState.INVALID_VALUE;
         int invListNumElements = LSMInvertedIndexSearchCursorInitialState.INVALID_VALUE;
-        if (initialState != null && initialState instanceof LSMInvertedIndexSearchCursorInitialState) {
+        if (initialState instanceof LSMInvertedIndexSearchCursorInitialState) {
             LSMInvertedIndexSearchCursorInitialState invIndexInitialState =
                     (LSMInvertedIndexSearchCursorInitialState) initialState;
             invListStartPageId = invIndexInitialState.getInvListStartPageId();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 4eb7728..5614ca5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.storage.am.lsm.invertedindex.dataflow;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -27,6 +28,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -43,6 +46,10 @@
 import org.apache.hyracks.storage.common.IStorageManager;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class LSMInvertedIndexLocalResource extends LsmResource {
 
     private static final long serialVersionUID = 1L;
@@ -79,6 +86,22 @@
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
     }
 
+    private LSMInvertedIndexLocalResource(IPersistedResourceRegistry registry, JsonNode json,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, boolean isPartitioned, int[] invertedIndexFields,
+            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
+            double bloomFilterFalsePositiveRate) throws HyracksDataException {
+        super(registry, json);
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.tokenizerFactory = tokenizerFactory;
+        this.isPartitioned = isPartitioned;
+        this.invertedIndexFields = invertedIndexFields;
+        this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
+        this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
+    }
+
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = serviceCtx.getIoManager();
@@ -95,14 +118,67 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory);
+                    metadataPageManagerFactory, serviceCtx.getTracer());
         } else {
             return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
                     tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache, file.getAbsolutePath(),
                     bloomFilterFalsePositiveRate, mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this),
                     ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories,
                     filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory);
+                    metadataPageManagerFactory, serviceCtx.getTracer());
         }
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        final ArrayNode tokenTypeTraitsArray = OBJECT_MAPPER.createArrayNode();
+        for (ITypeTraits tt : tokenTypeTraits) {
+            tokenTypeTraitsArray.add(tt.toJson(registry));
+        }
+        jsonObject.set("tokenTypeTraits", tokenTypeTraitsArray);
+        final ArrayNode tokenCmpFactoriesArray = OBJECT_MAPPER.createArrayNode();
+        for (IBinaryComparatorFactory factory : tokenCmpFactories) {
+            tokenCmpFactoriesArray.add(factory.toJson(registry));
+        }
+        jsonObject.set("tokenCmpFactories", tokenCmpFactoriesArray);
+        jsonObject.set("tokenizerFactory", tokenizerFactory.toJson(registry));
+        jsonObject.put("isPartitioned", isPartitioned);
+        jsonObject.putPOJO("invertedIndexFields", invertedIndexFields);
+        jsonObject.putPOJO("filterFieldsForNonBulkLoadOps", filterFieldsForNonBulkLoadOps);
+        jsonObject.putPOJO("invertedIndexFieldsForNonBulkLoadOps", invertedIndexFieldsForNonBulkLoadOps);
+        jsonObject.putPOJO("bloomFilterFalsePositiveRate", bloomFilterFalsePositiveRate);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final List<ITypeTraits> tokenTypeTraitsList = new ArrayList<>();
+        final ArrayNode jsonTokenTypeTraits = (ArrayNode) json.get("tokenTypeTraits");
+        for (JsonNode tt : jsonTokenTypeTraits) {
+            tokenTypeTraitsList.add((ITypeTraits) registry.deserialize(tt));
+        }
+        final ITypeTraits[] tokenTypeTraits = tokenTypeTraitsList.toArray(new ITypeTraits[0]);
+
+        final List<IBinaryComparatorFactory> tokenCmpFactoriesList = new ArrayList<>();
+        final ArrayNode jsontokenCmpFactories = (ArrayNode) json.get("tokenCmpFactories");
+        for (JsonNode cf : jsontokenCmpFactories) {
+            tokenCmpFactoriesList.add((IBinaryComparatorFactory) registry.deserialize(cf));
+        }
+        final IBinaryComparatorFactory[] tokenCmpFactories =
+                tokenCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
+        final IBinaryTokenizerFactory tokenizerFactory =
+                (IBinaryTokenizerFactory) registry.deserialize(json.get("tokenizerFactory"));
+        final boolean isPartitioned = json.get("isPartitioned").asBoolean();
+        final int[] invertedIndexFields = OBJECT_MAPPER.convertValue(json.get("invertedIndexFields"), int[].class);
+        final int[] filterFieldsForNonBulkLoadOps =
+                OBJECT_MAPPER.convertValue(json.get("filterFieldsForNonBulkLoadOps"), int[].class);
+        final int[] invertedIndexFieldsForNonBulkLoadOps =
+                OBJECT_MAPPER.convertValue(json.get("invertedIndexFieldsForNonBulkLoadOps"), int[].class);
+        final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
+        return new LSMInvertedIndexLocalResource(registry, json, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+                bloomFilterFalsePositiveRate);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 5fda514..22556af 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -59,7 +59,6 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
@@ -100,10 +99,10 @@
             IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, componentFactory, filterFrameFactory,
-                filterManager, filterFields, durable, filterHelper, invertedIndexFields, ITracer.NONE);
+                filterManager, filterFields, durable, filterHelper, invertedIndexFields, tracer);
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
@@ -111,7 +110,6 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
-
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             InMemoryInvertedIndex memInvIndex =
@@ -120,9 +118,8 @@
                     BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                             invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
                             ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false);
-            LSMInvertedIndexMemoryComponent mutableComponent =
-                    new LSMInvertedIndexMemoryComponent(this, memInvIndex, deleteKeysBTree, virtualBufferCache,
-                            i == 0 ? true : false, filterHelper == null ? null : filterHelper.createFilter());
+            LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
+                    deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
@@ -296,7 +293,7 @@
         }
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, numBTreeTuples, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, numBTreeTuples, false, false, false);
 
         // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
         IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -349,65 +346,66 @@
         LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
         RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
         IIndexCursor cursor = mergeOp.getCursor();
-        try {
-            ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
-            // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
-            // Create an inverted index instance.
-            ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                    mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
-            ILSMDiskComponentBulkLoader componentBulkLoader;
-            // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
-            // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
-            if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
-                    .get(diskComponents.size() - 1)) {
-                // Keep the deleted tuples since the oldest disk component is not included in the merge operation
-                LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
-                        new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
-                try {
-                    long numElements = 0L;
-                    for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                        numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
-                                .getBloomFilter().getNumElements();
-                    }
-                    componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
-                    loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
-                } finally {
-                    btreeCursor.destroy();
-                }
-            } else {
-                componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
-            }
-            search(opCtx, cursor, mergePred);
+        ILSMIndexOperationContext opCtx = ((LSMInvertedIndexMergeCursor) cursor).getOpCtx();
+        // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+        // Create an inverted index instance.
+        ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
+                mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
+        ILSMDiskComponentBulkLoader componentBulkLoader;
+        // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
+        // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
+        if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+                .get(diskComponents.size() - 1)) {
+            // Keep the deleted tuples since the oldest disk component is not included in the merge operation
+            LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
+                    new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
             try {
-                while (cursor.hasNext()) {
-                    cursor.next();
-                    ITupleReference tuple = cursor.getTuple();
-                    componentBulkLoader.add(tuple);
-                }
-            } finally {
-                cursor.close();
-            }
-            if (component.getLSMComponentFilter() != null) {
-                List<ITupleReference> filterTuples = new ArrayList<>();
+                long numElements = 0L;
                 for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                    ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
-                    ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
-                    if (min != null) {
-                        filterTuples.add(min);
-                    }
-                    if (max != null) {
-                        filterTuples.add(max);
-                    }
+                    numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
+                            .getBloomFilter().getNumElements();
                 }
-                getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
-                        NoOpOperationCallback.INSTANCE);
-                getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+                componentBulkLoader =
+                        component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+                loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
+            } finally {
+                btreeCursor.destroy();
             }
-            componentBulkLoader.end();
-            return component;
-        } finally {
-            cursor.close();
+        } else {
+            componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         }
+        search(opCtx, cursor, mergePred);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                componentBulkLoader.add(cursor.getTuple());
+            }
+        } finally {
+            try {
+                cursor.close();
+            } finally {
+                cursor.destroy();
+            }
+        }
+        if (component.getLSMComponentFilter() != null) {
+            List<ITupleReference> filterTuples = new ArrayList<>();
+            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
+                ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
+                if (min != null) {
+                    filterTuples.add(min);
+                }
+                if (max != null) {
+                    filterTuples.add(max);
+                }
+            }
+            getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
+                    NoOpOperationCallback.INSTANCE);
+            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+        }
+        componentBulkLoader.end();
+
+        return component;
     }
 
     private void loadDeleteTuples(ILSMIndexOperationContext opCtx,
@@ -488,16 +486,16 @@
             throws HyracksDataException {
         return new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(getHarness(), opCtx),
                 componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+                componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
 
     @Override
     protected ILSMIOOperation createMergeOperation(AbstractLSMIndexOperationContext opCtx,
             LSMComponentFileReferences mergeFileRefs, ILSMIOOperationCallback callback) throws HyracksDataException {
         ILSMIndexAccessor accessor = new LSMInvertedIndexAccessor(getHarness(), opCtx);
-        IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(opCtx);
+        IIndexCursor cursor = new LSMInvertedIndexMergeCursor(opCtx);
         return new LSMInvertedIndexMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 247e44c..8e39b62 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -86,43 +85,42 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFlush() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FLUSH);
-        lsmHarness.scheduleFlush(ctx, callback);
+        return lsmHarness.scheduleFlush(ctx);
     }
 
     @Override
     public void flush(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.flush(ctx, operation);
+        lsmHarness.flush(operation);
     }
 
     @Override
-    public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
-            throws HyracksDataException {
+    public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
         ctx.setOperation(IndexOperation.MERGE);
         ctx.getComponentsToBeMerged().clear();
         ctx.getComponentsToBeMerged().addAll(components);
-        lsmHarness.scheduleMerge(ctx, callback);
+        return lsmHarness.scheduleMerge(ctx);
     }
 
     @Override
-    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+    public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
             throws HyracksDataException {
         ctx.setOperation(IndexOperation.REPLICATE);
         ctx.getComponentsToBeReplicated().clear();
         ctx.getComponentsToBeReplicated().addAll(lsmComponents);
-        lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+        lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
     }
 
     @Override
-    public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+    public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
         ctx.setOperation(IndexOperation.FULL_MERGE);
-        lsmHarness.scheduleFullMerge(ctx, callback);
+        return lsmHarness.scheduleFullMerge(ctx);
     }
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        lsmHarness.merge(ctx, operation);
+        lsmHarness.merge(operation);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
index f1f5241..faa90eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
@@ -60,7 +60,6 @@
         ArrayList<IIndexAccessor> btreeAccessors = lsmInitialState.getDeletedKeysBTreeAccessors();
         for (int i = 0; i < numBTrees; i++) {
             rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(false);
-            btreeAccessors.get(i).search(rangeCursors[i], btreePredicate);
         }
         IndexCursorUtils.open(btreeAccessors, rangeCursors, btreePredicate);
         try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index 279a518..41e72cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -25,12 +25,16 @@
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
 public class LSMInvertedIndexDiskComponent extends AbstractLSMWithBuddyDiskComponent {
 
@@ -99,14 +103,28 @@
     }
 
     @Override
-    public void markAsValid(boolean persist) throws HyracksDataException {
+    public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
         ComponentUtils.markAsValid(getBloomFilterBufferCache(), getBloomFilter(), persist);
 
         // Flush inverted index second.
         invIndex.getBufferCache().force((invIndex).getInvListsFileId(), true);
-        ComponentUtils.markAsValid(getMetadataHolder(), persist);
+        ComponentUtils.markAsValid(getMetadataHolder(), persist, callback);
+        if (!callback.hasFailed()) {
+            // Flush deleted keys BTree.
+            ComponentUtils.markAsValid(getBuddyIndex(), persist, callback);
+        }
+        if (callback.hasFailed()) {
+            throw HyracksDataException.create(callback.getFailure());
+        }
+    }
 
-        // Flush deleted keys BTree.
-        ComponentUtils.markAsValid(getBuddyIndex(), persist);
+    @Override
+    protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+            long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+        IIndexBulkLoader indexBulkLoader =
+                invIndex.createMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        IIndexBulkLoader buddyBulkLoader =
+                getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+        return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
index c4a2f73..4471102 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
-import java.io.File;
 import java.io.FilenameFilter;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -31,9 +30,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
 
@@ -59,21 +58,15 @@
 
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
-        String ts = getCurrentTimestamp();
-        String baseName = ts + DELIMITER + ts;
-        // Begin timestamp and end timestamp are identical since it is a flush
+        String baseName = getNextComponentSequence(deletedKeysBTreeFilter);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + DICT_BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + DELETED_KEYS_BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
     }
 
     @Override
-    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
-            throws HyracksDataException {
-        String[] firstTimestampRange = firstFileName.split(DELIMITER);
-        String[] lastTimestampRange = lastFileName.split(DELIMITER);
-        String baseName = firstTimestampRange[0] + DELIMITER + lastTimestampRange[1];
-        // Get the range of timestamps by taking the earliest and the latest timestamps
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
+        final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + DICT_BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + DELETED_KEYS_BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
@@ -82,23 +75,25 @@
     @Override
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException {
         List<LSMComponentFileReferences> validFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allDictBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allInvListsFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allDictBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allInvListsFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allDeletedKeysBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBloomFilterFiles = new ArrayList<>();
 
         // Gather files.
-        cleanupAndGetValidFilesInternal(deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
+        cleanupAndGetValidFilesInternal(deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles,
+                btreeFactory.getBufferCache());
         HashSet<String> deletedKeysBTreeFilesSet = new HashSet<>();
-        for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
-            int index = cmpFileName.fileName.lastIndexOf(DELIMITER);
-            deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+        for (IndexComponentFileReference cmpFileName : allDeletedKeysBTreeFiles) {
+            deletedKeysBTreeFilesSet.add(cmpFileName.getSequence());
         }
 
         // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
-        validateFiles(deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
-        validateFiles(deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
-        validateFiles(deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        validateFiles(deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null, btreeFactory.getBufferCache());
+        validateFiles(deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory,
+                btreeFactory.getBufferCache());
+        validateFiles(deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null,
+                btreeFactory.getBufferCache());
 
         // Sanity check.
         if (allDictBTreeFiles.size() != allInvListsFiles.size()
@@ -115,52 +110,48 @@
 
         if (allDictBTreeFiles.size() == 1 && allInvListsFiles.size() == 1 && allDeletedKeysBTreeFiles.size() == 1
                 && allBloomFilterFiles.size() == 1) {
-            validFiles.add(new LSMComponentFileReferences(allDictBTreeFiles.get(0).fileRef,
-                    allDeletedKeysBTreeFiles.get(0).fileRef, allBloomFilterFiles.get(0).fileRef));
+            validFiles.add(new LSMComponentFileReferences(allDictBTreeFiles.get(0).getFileRef(),
+                    allDeletedKeysBTreeFiles.get(0).getFileRef(), allBloomFilterFiles.get(0).getFileRef()));
             return validFiles;
         }
 
-        // Sorts files names from earliest to latest timestamp.
+        // Sorts files names from earliest to latest sequence.
         Collections.sort(allDeletedKeysBTreeFiles);
         Collections.sort(allDictBTreeFiles);
         Collections.sort(allBloomFilterFiles);
 
-        List<ComparableFileName> validComparableDictBTreeFiles = new ArrayList<>();
-        ComparableFileName lastDictBTree = allDictBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableDictBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastDictBTree = allDictBTreeFiles.get(0);
         validComparableDictBTreeFiles.add(lastDictBTree);
 
-        List<ComparableFileName> validComparableDeletedKeysBTreeFiles = new ArrayList<>();
-        ComparableFileName lastDeletedKeysBTree = allDeletedKeysBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableDeletedKeysBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastDeletedKeysBTree = allDeletedKeysBTreeFiles.get(0);
         validComparableDeletedKeysBTreeFiles.add(lastDeletedKeysBTree);
 
-        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<>();
-        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        List<IndexComponentFileReference> validComparableBloomFilterFiles = new ArrayList<>();
+        IndexComponentFileReference lastBloomFilter = allBloomFilterFiles.get(0);
         validComparableBloomFilterFiles.add(lastBloomFilter);
 
         for (int i = 1; i < allDictBTreeFiles.size(); i++) {
-            ComparableFileName currentDeletedKeysBTree = allDeletedKeysBTreeFiles.get(i);
-            ComparableFileName currentDictBTree = allDictBTreeFiles.get(i);
-            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
-            // Current start timestamp is greater than last stop timestamp.
-            if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[1]) > 0
-                    && currentDictBTree.interval[0].compareTo(lastDictBTree.interval[1]) > 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+            IndexComponentFileReference currentDeletedKeysBTree = allDeletedKeysBTreeFiles.get(i);
+            IndexComponentFileReference currentDictBTree = allDictBTreeFiles.get(i);
+            IndexComponentFileReference currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start sequence is greater than last stop sequence.
+            if (currentDeletedKeysBTree.isMoreRecentThan(lastDeletedKeysBTree)
+                    && currentDictBTree.isMoreRecentThan(lastDictBTree)
+                    && currentBloomFilter.isMoreRecentThan(lastBloomFilter)) {
                 validComparableDictBTreeFiles.add(currentDictBTree);
                 validComparableDeletedKeysBTreeFiles.add(currentDeletedKeysBTree);
                 validComparableBloomFilterFiles.add(currentBloomFilter);
                 lastDictBTree = currentDictBTree;
                 lastDeletedKeysBTree = currentDeletedKeysBTree;
                 lastBloomFilter = currentBloomFilter;
-            } else if (currentDeletedKeysBTree.interval[0].compareTo(lastDeletedKeysBTree.interval[0]) >= 0
-                    && currentDeletedKeysBTree.interval[1].compareTo(lastDeletedKeysBTree.interval[1]) <= 0
-                    && currentDictBTree.interval[0].compareTo(lastDictBTree.interval[0]) >= 0
-                    && currentDictBTree.interval[1].compareTo(lastDictBTree.interval[1]) <= 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
-                    && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
-                // Invalid files are completely contained in last interval.
-                IoUtil.delete(new File(currentDeletedKeysBTree.fullPath));
-                IoUtil.delete(new File(currentDictBTree.fullPath));
-                IoUtil.delete(new File(currentBloomFilter.fullPath));
+            } else if (currentDeletedKeysBTree.isWithin(lastDeletedKeysBTree)
+                    && currentDictBTree.isWithin(lastDictBTree) && currentBloomFilter.isWithin(lastBloomFilter)) {
+                // Invalid files are completely contained in last sequence.
+                delete(treeFactory.getBufferCache(), currentDeletedKeysBTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentDictBTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentBloomFilter.getFullPath());
             } else {
                 // This scenario should not be possible.
                 throw HyracksDataException.create(ErrorCode.FOUND_OVERLAPPING_LSM_FILES, baseDir);
@@ -169,21 +160,20 @@
 
         // Sort valid files in reverse lexicographical order, such that newer
         // files come first.
-        Collections.sort(validComparableDictBTreeFiles, recencyCmp);
-        Collections.sort(validComparableDeletedKeysBTreeFiles, recencyCmp);
-        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+        validComparableDictBTreeFiles.sort(recencyCmp);
+        validComparableDeletedKeysBTreeFiles.sort(recencyCmp);
+        validComparableBloomFilterFiles.sort(recencyCmp);
 
-        Iterator<ComparableFileName> dictBTreeFileIter = validComparableDictBTreeFiles.iterator();
-        Iterator<ComparableFileName> deletedKeysBTreeIter = validComparableDeletedKeysBTreeFiles.iterator();
-        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        Iterator<IndexComponentFileReference> dictBTreeFileIter = validComparableDictBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> deletedKeysBTreeIter = validComparableDeletedKeysBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
         while (dictBTreeFileIter.hasNext() && deletedKeysBTreeIter.hasNext()) {
-            ComparableFileName cmpDictBTreeFile = dictBTreeFileIter.next();
-            ComparableFileName cmpDeletedKeysBTreeFile = deletedKeysBTreeIter.next();
-            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
-            validFiles.add(new LSMComponentFileReferences(cmpDictBTreeFile.fileRef, cmpDeletedKeysBTreeFile.fileRef,
-                    cmpBloomFilterFileName.fileRef));
+            IndexComponentFileReference cmpDictBTreeFile = dictBTreeFileIter.next();
+            IndexComponentFileReference cmpDeletedKeysBTreeFile = deletedKeysBTreeIter.next();
+            IndexComponentFileReference cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpDictBTreeFile.getFileRef(),
+                    cmpDeletedKeysBTreeFile.getFileRef(), cmpBloomFilterFileName.getFileRef()));
         }
-
         return validFiles;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
index 77c62b4..a8005bc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
@@ -20,9 +20,9 @@
 package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
 
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 
 public class LSMInvertedIndexMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -31,8 +31,8 @@
     private final BTree deletedKeysBTree;
 
     public LSMInvertedIndexMemoryComponent(LSMInvertedIndex lsmIndex, InMemoryInvertedIndex invIndex,
-            BTree deletedKeysBTree, IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            BTree deletedKeysBTree, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.invIndex = invIndex;
         this.deletedKeysBTree = deletedKeysBTree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
new file mode 100644
index 0000000..c80455d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
@@ -0,0 +1,369 @@
+/*
+ * 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.hyracks.storage.am.lsm.invertedindex.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueComparator;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueElement;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+/**
+ * This cursor is specially designed and optimized for merging inverted index.
+ * For simplicity, it assumes all components are disk components, and the cursor is not reused.
+ *
+ */
+public class LSMInvertedIndexMergeCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
+    protected final LSMInvertedIndexOpContext opCtx;
+    protected PriorityQueueElement outputTokenElement;
+    protected OnDiskInvertedIndexRangeSearchCursor[] rangeCursors;
+    protected PriorityQueueElement[] tokenQueueElements;
+    protected PriorityQueue<PriorityQueueElement> tokenQueue;
+    protected PriorityQueueComparator tokenQueueCmp;
+
+    protected PriorityQueueElement outputKeyElement;
+    protected PriorityQueueElement[] keyQueueElements;
+    protected PriorityQueue<PriorityQueueElement> keyQueue;
+    protected PriorityQueueComparator keyQueueCmp;
+
+    protected boolean needPushElementIntoKeyQueue;
+
+    protected ILSMHarness lsmHarness;
+
+    protected MultiComparator tokenCmp;
+    protected MultiComparator keyCmp;
+
+    protected List<ILSMComponent> operationalComponents;
+
+    // Assuming the cursor for all deleted-keys indexes are of the same type.
+    protected IIndexCursor[] deletedKeysBTreeCursors;
+    protected BloomFilter[] bloomFilters;
+    protected final long[] hashes = BloomFilter.createHashArray();
+    protected ArrayList<IIndexAccessor> deletedKeysBTreeAccessors;
+    protected RangePredicate deletedKeyBTreeSearchPred;
+
+    protected final TokenKeyPairTuple outputTuple;
+
+    public LSMInvertedIndexMergeCursor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = (LSMInvertedIndexOpContext) opCtx;
+        outputTokenElement = null;
+        outputKeyElement = null;
+        needPushElementIntoKeyQueue = false;
+
+        IInvertedIndex invertedIndex = (IInvertedIndex) this.opCtx.getIndex();
+        this.outputTuple = new TokenKeyPairTuple(invertedIndex.getTokenTypeTraits().length,
+                invertedIndex.getInvListTypeTraits().length);
+
+        this.tokenCmp = MultiComparator.create(invertedIndex.getTokenCmpFactories());
+        this.keyCmp = MultiComparator.create(invertedIndex.getInvListCmpFactories());
+        this.tokenQueueCmp = new PriorityQueueComparator(tokenCmp);
+        this.keyQueueCmp = new PriorityQueueComparator(keyCmp);
+    }
+
+    public LSMInvertedIndexOpContext getOpCtx() {
+        return opCtx;
+    }
+
+    @Override
+    public void doOpen(ICursorInitialState initState, ISearchPredicate searchPred) throws HyracksDataException {
+        LSMInvertedIndexRangeSearchCursorInitialState lsmInitState =
+                (LSMInvertedIndexRangeSearchCursorInitialState) initState;
+        int numComponents = lsmInitState.getNumComponents();
+        rangeCursors = new OnDiskInvertedIndexRangeSearchCursor[numComponents];
+        for (int i = 0; i < numComponents; i++) {
+            IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) lsmInitState.getIndexAccessors().get(i);
+            rangeCursors[i] = (OnDiskInvertedIndexRangeSearchCursor) invIndexAccessor.createRangeSearchCursor();
+            invIndexAccessor.rangeSearch(rangeCursors[i], lsmInitState.getSearchPredicate());
+        }
+        lsmHarness = lsmInitState.getLSMHarness();
+        operationalComponents = lsmInitState.getOperationalComponents();
+        deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
+        bloomFilters = new BloomFilter[deletedKeysBTreeAccessors.size()];
+        if (!deletedKeysBTreeAccessors.isEmpty()) {
+            deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
+            for (int i = 0; i < operationalComponents.size(); i++) {
+                ILSMComponent component = operationalComponents.get(i);
+                deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false);
+                if (component.getType() == LSMComponentType.MEMORY) {
+                    // No need for a bloom filter for the in-memory BTree.
+                    bloomFilters[i] = null;
+                } else {
+                    bloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
+                }
+            }
+        }
+        deletedKeyBTreeSearchPred = new RangePredicate(null, null, true, true, keyCmp, keyCmp);
+        initPriorityQueues();
+    }
+
+    private void initPriorityQueues() throws HyracksDataException {
+        int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
+        tokenQueue = new PriorityQueue<>(pqInitSize, tokenQueueCmp);
+        keyQueue = new PriorityQueue<>(pqInitSize, keyQueueCmp);
+        tokenQueueElements = new PriorityQueueElement[pqInitSize];
+        keyQueueElements = new PriorityQueueElement[pqInitSize];
+        for (int i = 0; i < pqInitSize; i++) {
+            tokenQueueElements[i] = new PriorityQueueElement(i);
+            keyQueueElements[i] = new PriorityQueueElement(i);
+        }
+        for (int i = 0; i < rangeCursors.length; i++) {
+            if (rangeCursors[i].hasNext()) {
+                rangeCursors[i].next();
+                tokenQueueElements[i].reset(rangeCursors[i].getTuple());
+                tokenQueue.offer(tokenQueueElements[i]);
+            } else {
+                rangeCursors[i].close();
+            }
+        }
+        searchNextToken();
+    }
+
+    private void searchNextToken() throws HyracksDataException {
+        if (tokenQueue.isEmpty()) {
+            return;
+        }
+        if (!keyQueue.isEmpty()) {
+            throw new IllegalStateException("Illegal call of initializing key queue");
+        }
+        outputTokenElement = tokenQueue.poll();
+        initPushIntoKeyQueue(outputTokenElement);
+        ITupleReference tokenTuple = getTokenTuple(outputTokenElement);
+        outputTuple.setTokenTuple(tokenTuple);
+        // pop all same tokens
+        while (!tokenQueue.isEmpty()) {
+            PriorityQueueElement tokenElement = tokenQueue.peek();
+            if (TupleUtils.equalTuples(tokenTuple, getTokenTuple(tokenElement), tokenCmp.getKeyFieldCount())) {
+                initPushIntoKeyQueue(tokenElement);
+                tokenQueue.poll();
+            } else {
+                break;
+            }
+        }
+    }
+
+    private ITupleReference getKeyTuple(PriorityQueueElement tokenElement) {
+        return ((TokenKeyPairTuple) tokenElement.getTuple()).getKeyTuple();
+    }
+
+    private ITupleReference getTokenTuple(PriorityQueueElement tokenElement) {
+        return ((TokenKeyPairTuple) tokenElement.getTuple()).getTokenTuple();
+    }
+
+    private void initPushIntoKeyQueue(PriorityQueueElement tokenElement) {
+        PriorityQueueElement keyElement = keyQueueElements[tokenElement.getCursorIndex()];
+        keyElement.reset(getKeyTuple(tokenElement));
+        keyQueue.add(keyElement);
+    }
+
+    private void pushIntoKeyQueueAndReplace(PriorityQueueElement keyElement) throws HyracksDataException {
+        int cursorIndex = keyElement.getCursorIndex();
+        if (rangeCursors[cursorIndex].hasNext()) {
+            rangeCursors[cursorIndex].next();
+            TokenKeyPairTuple tuple = (TokenKeyPairTuple) rangeCursors[cursorIndex].getTuple();
+            if (tuple.isNewToken()) {
+                // if this element is a new token, then the current inverted list has exuasted
+                PriorityQueueElement tokenElement = tokenQueueElements[cursorIndex];
+                tokenElement.reset(tuple);
+                tokenQueue.offer(tokenElement);
+            } else {
+                keyElement.reset(tuple.getKeyTuple());
+                keyQueue.offer(keyElement);
+            }
+        } else {
+            rangeCursors[cursorIndex].close();
+        }
+    }
+
+    @Override
+    public boolean doHasNext() throws HyracksDataException {
+        checkPriorityQueue();
+        return !keyQueue.isEmpty();
+    }
+
+    @Override
+    public void doNext() throws HyracksDataException {
+        outputKeyElement = keyQueue.poll();
+        outputTuple.setKeyTuple(outputKeyElement.getTuple());
+        needPushElementIntoKeyQueue = true;
+    }
+
+    @Override
+    public ITupleReference doGetTuple() {
+        return outputTuple;
+    }
+
+    protected void checkPriorityQueue() throws HyracksDataException {
+        checkKeyQueue();
+        if (keyQueue.isEmpty()) {
+            // if key queue is empty, we search the next token and check again
+            searchNextToken();
+            checkKeyQueue();
+        }
+    }
+
+    protected void checkKeyQueue() throws HyracksDataException {
+        while (!keyQueue.isEmpty() || needPushElementIntoKeyQueue) {
+            if (!keyQueue.isEmpty()) {
+                PriorityQueueElement checkElement = keyQueue.peek();
+                // If there is no previous tuple or the previous tuple can be ignored
+                if (outputKeyElement == null) {
+                    if (isDeleted(checkElement)) {
+                        // If the key has been deleted then pop it and set needPush to true.
+                        // We cannot push immediately because the tuple may be
+                        // modified if hasNext() is called
+                        outputKeyElement = checkElement;
+                        needPushElementIntoKeyQueue = true;
+                    } else {
+                        // we have found the next record
+                        return;
+                    }
+                } else {
+                    // Compare the previous tuple and the head tuple in the PQ
+                    if (keyCmp.compare(outputKeyElement.getTuple(), checkElement.getTuple()) == 0) {
+                        // If the previous tuple and the head tuple are
+                        // identical
+                        // then pop the head tuple and push the next tuple from
+                        // the tree of head tuple
+
+                        // the head element of PQ is useless now
+                        PriorityQueueElement e = keyQueue.poll();
+                        pushIntoKeyQueueAndReplace(e);
+                    } else {
+                        // If the previous tuple and the head tuple are different
+                        // the info of previous tuple is useless
+                        if (needPushElementIntoKeyQueue) {
+                            pushIntoKeyQueueAndReplace(outputKeyElement);
+                            needPushElementIntoKeyQueue = false;
+                        }
+                        outputKeyElement = null;
+                    }
+                }
+            } else {
+                // the priority queue is empty and needPush
+                // NOSONAR: outputKeyElement is not null when needPushElementIntoKeyQueue = true
+                pushIntoKeyQueueAndReplace(outputKeyElement);
+                needPushElementIntoKeyQueue = false;
+                outputKeyElement = null;
+            }
+        }
+    }
+
+    /**
+     * Check deleted-keys BTrees whether they contain the key in the checkElement's tuple.
+     */
+    protected boolean isDeleted(PriorityQueueElement keyElement) throws HyracksDataException {
+        ITupleReference keyTuple = keyElement.getTuple();
+        int end = keyElement.getCursorIndex();
+        for (int i = 0; i < end; i++) {
+            if (bloomFilters[i] != null && !bloomFilters[i].contains(keyTuple, hashes)) {
+                continue;
+            }
+            deletedKeysBTreeCursors[i].close();
+            deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], deletedKeyBTreeSearchPred);
+            try {
+                if (deletedKeysBTreeCursors[i].hasNext()) {
+                    return true;
+                }
+            } finally {
+                deletedKeysBTreeCursors[i].close();
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        outputTokenElement = null;
+        outputKeyElement = null;
+        needPushElementIntoKeyQueue = false;
+        try {
+            if (rangeCursors != null) {
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    rangeCursors[i].close();
+                }
+            }
+        } finally {
+            if (lsmHarness != null) {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        try {
+            if (tokenQueue != null) {
+                tokenQueue.clear();
+            }
+            if (keyQueue != null) {
+                keyQueue.clear();
+            }
+            if (rangeCursors != null) {
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    if (rangeCursors[i] != null) {
+                        rangeCursors[i].destroy();
+                    }
+                }
+                rangeCursors = null;
+            }
+        } finally {
+            if (lsmHarness != null) {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public ITupleReference getFilterMinTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMaxTuple() {
+        return null;
+    }
+
+    @Override
+    public boolean getSearchOperationCallbackProceedResult() {
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index 12dc23f..020b788 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -98,10 +98,9 @@
         keysOnlyTuple.reset(checkElement.getTuple());
         int end = checkElement.getCursorIndex();
         for (int i = 0; i < end; i++) {
-            if (bloomFilters[i] != null && bloomFilters[i].contains(keysOnlyTuple, hashes)) {
+            if (bloomFilters[i] != null && !bloomFilters[i].contains(keysOnlyTuple, hashes)) {
                 continue;
             }
-            deletedKeysBTreeCursors[i].close();
             deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
             try {
                 if (deletedKeysBTreeCursors[i].hasNext()) {
@@ -114,4 +113,31 @@
         return false;
     }
 
+    @Override
+    public void doClose() throws HyracksDataException {
+        try {
+            super.doClose();
+        } finally {
+            if (deletedKeysBTreeCursors != null) {
+                for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+                    deletedKeysBTreeCursors[i].close();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        try {
+            super.doDestroy();
+        } finally {
+            if (deletedKeysBTreeCursors != null) {
+                for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+                    deletedKeysBTreeCursors[i].destroy();
+                }
+                deletedKeysBTreeCursors = null;
+            }
+        }
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 4d444b9..d39c601 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -59,6 +59,7 @@
     private List<IIndexAccessor> deletedKeysBTreeAccessors;
     private RangePredicate keySearchPred;
     private ILSMIndexOperationContext opCtx;
+    private boolean includeMemoryComponents;
 
     private List<ILSMComponent> operationalComponents;
     private ITupleReference currentTuple = null;
@@ -76,7 +77,7 @@
         accessorIndex = 0;
         this.searchPred = searchPred;
         this.searchCallback = lsmInitState.getSearchOperationCallback();
-
+        includeMemoryComponents = false;
         // For searching the deleted-keys BTrees.
         deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
         deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
@@ -87,6 +88,7 @@
             if (component.getType() == LSMComponentType.MEMORY) {
                 // No need for a bloom filter for the in-memory BTree.
                 deletedKeysBTreeBloomFilters[i] = null;
+                includeMemoryComponents = true;
             } else {
                 deletedKeysBTreeBloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
             }
@@ -121,7 +123,8 @@
         while (currentCursor.hasNext()) {
             currentCursor.next();
             currentTuple = currentCursor.getTuple();
-            resultOfSearchCallBackProceed = searchCallback.proceed(currentTuple);
+            resultOfSearchCallBackProceed =
+                    includeMemoryComponents && accessorIndex == 0 ? searchCallback.proceed(currentTuple) : true;
 
             if (!resultOfSearchCallBackProceed) {
                 // We assume that the underlying cursors materialize their results such that
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index d7482a8..604d374 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
 
 public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
 
@@ -52,12 +53,12 @@
             IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
                 bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
                 tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
                 ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index e48f16f..0dd6b2c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
index da3f079..7f3d12f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
@@ -220,9 +220,11 @@
             // Assumption: processing inverted list takes time; so, we don't want to keep them on the buffer cache.
             // Rather, we utilize the assigned working memory (buffers).
             tmpBuffer = page.getBuffer();
-            tmpBuffer.rewind();
-            buffers.get(currentBufferIdx).rewind();
-            buffers.get(currentBufferIdx).put(tmpBuffer);
+
+            // Copies the entire content of the page to the current buffer in the working memory.
+            System.arraycopy(tmpBuffer.array(), 0, buffers.get(currentBufferIdx).array(), 0,
+                    buffers.get(currentBufferIdx).capacity());
+            buffers.get(currentBufferIdx).position(buffers.get(currentBufferIdx).capacity());
 
             currentBufferIdx++;
             bufferCache.unpin(page);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
index b033b99..2401c67 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
@@ -53,6 +53,7 @@
     protected ICachedPage page;
 
     protected boolean pinned;
+    protected int pinnedPageId = -1;
 
     public FixedSizeElementInvertedListScanCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields)
             throws HyracksDataException {
@@ -103,14 +104,18 @@
      */
     @Override
     public void loadPages() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
         if (currentPageId == endPageId) {
+            // inverted list exhausted, return
             return;
         }
         currentPageId++;
+        if (pinned && pinnedPageId == currentPageId) {
+            // already pinned, return
+            return;
+        }
+        unloadPages();
         page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
+        pinnedPageId = currentPageId;
         pinned = true;
     }
 
@@ -134,7 +139,6 @@
         // Deducts 1 since the startPage would be set to bufferCurrentPageId + 1 in loadPages().
         this.currentPageId = startPageId - 1;
         this.numPages = endPageId - startPageId + 1;
-        this.pinned = false;
     }
 
     @Override
@@ -158,16 +162,14 @@
 
     @Override
     public void doClose() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
+        // No op
+        // We allow the inverted list cursor to hold at most one page to avoid
+        // unnecessary pins
     }
 
     @Override
     public void doDestroy() throws HyracksDataException {
-        if (pinned) {
-            unloadPages();
-        }
+        unloadPages();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 2f4f1d6..0b504a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexSearchCursorInitialState;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -60,6 +61,7 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IFIFOPageQueue;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 /**
@@ -204,9 +206,9 @@
         try {
             if (ctx.getBtreeCursor().hasNext()) {
                 ctx.getBtreeCursor().next();
-                openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor);
+                openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor, ctx);
             } else {
-                LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+                LSMInvertedIndexSearchCursorInitialState initState = ctx.getCursorInitialState();
                 initState.setInvertedListInfo(0, 0, 0, 0);
                 listCursor.open(initState, null);
             }
@@ -215,8 +217,8 @@
         }
     }
 
-    public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor)
-            throws HyracksDataException {
+    public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor,
+            OnDiskInvertedIndexOpContext opCtx) throws HyracksDataException {
         int startPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListStartPageIdField),
                 btreeTuple.getFieldStart(invListStartPageIdField));
         int endPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListEndPageIdField),
@@ -225,35 +227,34 @@
                 btreeTuple.getFieldStart(invListStartOffField));
         int numElements = IntegerPointable.getInteger(btreeTuple.getFieldData(invListNumElementsField),
                 btreeTuple.getFieldStart(invListNumElementsField));
-        LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+        LSMInvertedIndexSearchCursorInitialState initState = opCtx.getCursorInitialState();
         initState.setInvertedListInfo(startPageId, endPageId, startOff, numElements);
         listCursor.open(initState, null);
     }
 
-    public final class OnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
-        private final ArrayTupleBuilder btreeTupleBuilder;
-        private final ArrayTupleReference btreeTupleReference;
-        private final IIndexBulkLoader btreeBulkloader;
+    public abstract class AbstractOnDiskInvertedIndexBulkLoader extends PageWriteFailureCallback
+            implements IIndexBulkLoader {
+        protected final ArrayTupleBuilder btreeTupleBuilder;
+        protected final ArrayTupleReference btreeTupleReference;
+        protected final IIndexBulkLoader btreeBulkloader;
 
-        private int currentInvListStartPageId;
-        private int currentInvListStartOffset;
-        private final ArrayTupleBuilder lastTupleBuilder;
-        private final ArrayTupleReference lastTuple;
+        protected int currentInvListStartPageId;
+        protected int currentInvListStartOffset;
+        protected final ArrayTupleBuilder lastTupleBuilder;
+        protected final ArrayTupleReference lastTuple;
 
-        private int currentPageId;
-        private ICachedPage currentPage;
-        private final MultiComparator tokenCmp;
-        private final MultiComparator invListCmp;
+        protected int currentPageId;
+        protected ICachedPage currentPage;
+        protected final MultiComparator invListCmp;
 
-        private final boolean verifyInput;
-        private final MultiComparator allCmp;
+        protected final boolean verifyInput;
+        protected final MultiComparator allCmp;
 
-        private final IFIFOPageQueue queue;
+        protected final IFIFOPageQueue queue;
 
-        public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+        public AbstractOnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
                 boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
             this.verifyInput = verifyInput;
-            this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
             this.invListCmp = MultiComparator.create(invListCmpFactories);
             if (verifyInput) {
                 allCmp = MultiComparator.create(btree.getComparatorFactories(), invListCmpFactories);
@@ -272,22 +273,15 @@
             queue = bufferCache.createFIFOQueue();
         }
 
-        public void pinNextPage() throws HyracksDataException {
-            queue.put(currentPage);
+        protected void pinNextPage() throws HyracksDataException {
+            queue.put(currentPage, this);
             currentPageId++;
             currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
         }
 
-        private void createAndInsertBTreeTuple() throws HyracksDataException {
+        protected void insertBTreeTuple() throws HyracksDataException {
             // Build tuple.
-            btreeTupleBuilder.reset();
             DataOutput output = btreeTupleBuilder.getDataOutput();
-            // Add key fields.
-            lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
-            for (int i = 0; i < numTokenFields; i++) {
-                btreeTupleBuilder.addField(lastTuple.getFieldData(i), lastTuple.getFieldStart(i),
-                        lastTuple.getFieldLength(i));
-            }
             // Add inverted-list 'pointer' value fields.
             try {
                 output.writeInt(currentInvListStartPageId);
@@ -304,84 +298,69 @@
             // Reset tuple reference and add it into the BTree load.
             btreeTupleReference.reset(btreeTupleBuilder.getFieldEndOffsets(), btreeTupleBuilder.getByteArray());
             btreeBulkloader.add(btreeTupleReference);
+            btreeTupleBuilder.reset();
         }
 
-        /**
-         * Assumptions:
-         * The first btree.getMultiComparator().getKeyFieldCount() fields in tuple
-         * are btree keys (e.g., a string token).
-         * The next invListCmp.getKeyFieldCount() fields in tuple are keys of the
-         * inverted list (e.g., primary key).
-         * Key fields of inverted list are fixed size.
-         */
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            boolean firstElement = lastTupleBuilder.getSize() == 0;
-            boolean startNewList = firstElement;
-            if (!firstElement) {
-                // If the current and the last token don't match, we start a new list.
-                lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
-                startNewList = tokenCmp.compare(tuple, lastTuple) != 0;
-            }
-            if (startNewList) {
-                if (!firstElement) {
-                    // Create entry in btree for last inverted list.
-                    createAndInsertBTreeTuple();
-                }
-                if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                    pinNextPage();
-                    invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
-                    if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                        throw new IllegalStateException("Failed to create first inverted list.");
-                    }
-                }
-                currentInvListStartPageId = currentPageId;
-                currentInvListStartOffset = invListBuilder.getPos();
-            } else {
-                if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
-                    // Duplicate inverted-list element.
-                    return;
-                }
-            }
-
-            // Append to current inverted list.
-            if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+        protected void startNewList(ITupleReference tokenTuple) throws HyracksDataException {
+            if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
                 pinNextPage();
                 invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
-                if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+                if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
+                    throw new IllegalStateException("Failed to create first inverted list.");
+                }
+            }
+            currentInvListStartPageId = currentPageId;
+            currentInvListStartOffset = invListBuilder.getPos();
+        }
+
+        protected void appendInvertedList(ITupleReference keyTuple, int startField) throws HyracksDataException {
+            if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
+                pinNextPage();
+                invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+                if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
                     throw new IllegalStateException(
                             "Failed to append element to inverted list after switching to a new page.");
                 }
             }
+        }
 
-            if (verifyInput && lastTupleBuilder.getSize() != 0) {
-                if (allCmp.compare(tuple, lastTuple) <= 0) {
-                    throw new HyracksDataException(
-                            "Input stream given to OnDiskInvertedIndex bulk load is not sorted.");
-                }
+        protected void verifyTuple(ITupleReference tuple) throws HyracksDataException {
+            if (lastTupleBuilder.getSize() > 0 && allCmp.compare(tuple, lastTuple) <= 0) {
+                HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
             }
+        }
 
-            // Remember last tuple by creating a copy.
-            // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
+        protected void saveLastTuple(ITupleReference tuple) throws HyracksDataException {
             lastTupleBuilder.reset();
             for (int i = 0; i < tuple.getFieldCount(); i++) {
                 lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
             }
+            lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
+        }
+
+        protected void copyTokenToBTreeTuple(ITupleReference tokenTuple) throws HyracksDataException {
+            for (int i = 0; i < numTokenFields; i++) {
+                btreeTupleBuilder.addField(tokenTuple.getFieldData(i), tokenTuple.getFieldStart(i),
+                        tokenTuple.getFieldLength(i));
+            }
         }
 
         @Override
         public void end() throws HyracksDataException {
-            // The last tuple builder is empty if add() was never called.
-            if (lastTupleBuilder.getSize() != 0) {
-                createAndInsertBTreeTuple();
+            if (btreeTupleBuilder.getSize() != 0) {
+                insertBTreeTuple();
             }
+
             btreeBulkloader.end();
 
             if (currentPage != null) {
-                queue.put(currentPage);
+                queue.put(currentPage, this);
             }
             invListsMaxPageId = currentPageId;
             bufferCache.finishQueue();
+            if (hasFailed()) {
+                throw HyracksDataException.create(getFailure());
+            }
         }
 
         @Override
@@ -392,6 +371,72 @@
         }
     }
 
+    public class OnDiskInvertedIndexMergeBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+        public OnDiskInvertedIndexMergeBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+            super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException {
+            TokenKeyPairTuple pairTuple = (TokenKeyPairTuple) tuple;
+            ITupleReference tokenTuple = pairTuple.getTokenTuple();
+            ITupleReference keyTuple = pairTuple.getKeyTuple();
+            boolean startNewList = pairTuple.isNewToken();
+            if (startNewList) {
+                if (btreeTupleBuilder.getSize() > 0) {
+                    insertBTreeTuple();
+                }
+                startNewList(tokenTuple);
+                copyTokenToBTreeTuple(tokenTuple);
+            }
+            appendInvertedList(keyTuple, 0);
+            if (verifyInput) {
+                verifyTuple(tuple);
+                saveLastTuple(tuple);
+            }
+        }
+    }
+
+    public class OnDiskInvertedIndexBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+        public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+                boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+            super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException {
+            boolean firstElement = btreeTupleBuilder.getSize() == 0;
+            boolean startNewList = firstElement;
+            if (!firstElement) {
+                // If the current and the last token don't match, we start a new list.
+                startNewList = !TupleUtils.equalTuples(tuple, lastTuple, numTokenFields);
+            }
+            if (startNewList) {
+                if (!firstElement) {
+                    // Create entry in btree for last inverted list.
+                    insertBTreeTuple();
+                }
+                startNewList(tuple);
+                copyTokenToBTreeTuple(tuple);
+            } else {
+                if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
+                    // Duplicate inverted-list element.
+                    return;
+                }
+            }
+            appendInvertedList(tuple, numTokenFields);
+            if (verifyInput) {
+                verifyTuple(tuple);
+            }
+
+            saveLastTuple(tuple);
+        }
+
+    }
+
     @Override
     public IBufferCache getBufferCache() {
         return bufferCache;
@@ -518,6 +563,12 @@
                 rootPageId);
     }
 
+    public IIndexBulkLoader createMergeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex) throws HyracksDataException {
+        return new OnDiskInvertedIndexMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+                rootPageId);
+    }
+
     @Override
     public void validate() throws HyracksDataException {
         btree.validate();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
index 267cc79..81bc46b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexSearchCursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -32,12 +33,13 @@
 public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
 
     private final RangePredicate btreePred = new RangePredicate(null, null, true, true, null, null);
-    private IIndexAccessor btreeAccessor;
-    private IIndexCursor btreeCursor;
-    private MultiComparator searchCmp;
+    private final IIndexAccessor btreeAccessor;
+    private final IIndexCursor btreeCursor;
+    private final MultiComparator searchCmp;
     // For prefix search on partitioned indexes.
     private MultiComparator prefixSearchCmp;
     private boolean destroyed = false;
+    private LSMInvertedIndexSearchCursorInitialState cursorInitialState;
 
     public OnDiskInvertedIndexOpContext(BTree btree) throws HyracksDataException {
         // TODO: Ignore opcallbacks for now.
@@ -96,4 +98,11 @@
             btreeCursor.destroy();
         }
     }
+
+    public LSMInvertedIndexSearchCursorInitialState getCursorInitialState() {
+        if (cursorInitialState == null) {
+            cursorInitialState = new LSMInvertedIndexSearchCursorInitialState();
+        }
+        return cursorInitialState;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index d9e7d34..11b483e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -24,10 +24,9 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
 import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -41,7 +40,7 @@
 
     private final BTree btree;
     private final IIndexAccessor btreeAccessor;
-    private final IInPlaceInvertedIndex invIndex;
+    private final OnDiskInvertedIndex invIndex;
     private final IIndexOperationContext opCtx;
     private final InvertedListCursor invListRangeSearchCursor;
     private boolean isInvListCursorOpen;
@@ -50,11 +49,11 @@
     private RangePredicate btreePred;
 
     private final PermutingTupleReference tokenTuple;
-    private ConcatenatingTupleReference concatTuple;
+    private final TokenKeyPairTuple resultTuple;
 
-    public OnDiskInvertedIndexRangeSearchCursor(IInPlaceInvertedIndex invIndex, IIndexOperationContext opCtx)
+    public OnDiskInvertedIndexRangeSearchCursor(OnDiskInvertedIndex invIndex, IIndexOperationContext opCtx)
             throws HyracksDataException {
-        this.btree = ((OnDiskInvertedIndex) invIndex).getBTree();
+        this.btree = invIndex.getBTree();
         this.btreeAccessor = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
         this.invIndex = invIndex;
         this.opCtx = opCtx;
@@ -65,7 +64,7 @@
         }
         tokenTuple = new PermutingTupleReference(fieldPermutation);
         btreeCursor = btreeAccessor.createSearchCursor(false);
-        concatTuple = new ConcatenatingTupleReference(2);
+        resultTuple = new TokenKeyPairTuple(invIndex.getTokenTypeTraits().length, btree.getCmpFactories().length);
         invListRangeSearchCursor = invIndex.createInvertedListRangeSearchCursor();
         isInvListCursorOpen = false;
     }
@@ -87,11 +86,7 @@
             return true;
         }
         // The current inverted-list-range-search cursor is exhausted.
-        try {
-            invListRangeSearchCursor.unloadPages();
-        } finally {
-            invListRangeSearchCursor.close();
-        }
+        invListRangeSearchCursor.close();
         isInvListCursorOpen = false;
         openInvListRangeSearchCursor();
         return isInvListCursorOpen;
@@ -100,23 +95,14 @@
     @Override
     public void doNext() throws HyracksDataException {
         invListRangeSearchCursor.next();
-        if (concatTuple.hasMaxTuples()) {
-            concatTuple.removeLastTuple();
-        }
-        concatTuple.addTuple(invListRangeSearchCursor.getTuple());
+        resultTuple.setKeyTuple(invListRangeSearchCursor.getTuple());
     }
 
     @Override
     public void doDestroy() throws HyracksDataException {
         try {
-            if (isInvListCursorOpen) {
-                try {
-                    invListRangeSearchCursor.unloadPages();
-                } finally {
-                    isInvListCursorOpen = false;
-                    invListRangeSearchCursor.destroy();
-                }
-            }
+            invListRangeSearchCursor.destroy();
+            isInvListCursorOpen = false;
         } finally {
             btreeCursor.destroy();
         }
@@ -125,14 +111,8 @@
     @Override
     public void doClose() throws HyracksDataException {
         try {
-            if (isInvListCursorOpen) {
-                try {
-                    invListRangeSearchCursor.unloadPages();
-                } finally {
-                    invListRangeSearchCursor.close();
-                }
-                isInvListCursorOpen = false;
-            }
+            invListRangeSearchCursor.close();
+            isInvListCursorOpen = false;
         } finally {
             btreeCursor.close();
         }
@@ -140,7 +120,7 @@
 
     @Override
     public ITupleReference doGetTuple() {
-        return concatTuple;
+        return resultTuple;
     }
 
     // Opens an inverted-list-scan cursor for the given tuple.
@@ -148,11 +128,11 @@
         if (btreeCursor.hasNext()) {
             btreeCursor.next();
             tokenTuple.reset(btreeCursor.getTuple());
-            invIndex.openInvertedListCursor(invListRangeSearchCursor, tokenTuple, opCtx);
+            invIndex.openInvertedListCursor(btreeCursor.getTuple(), invListRangeSearchCursor,
+                    (OnDiskInvertedIndexOpContext) opCtx);
             invListRangeSearchCursor.prepareLoadPages();
             invListRangeSearchCursor.loadPages();
-            concatTuple.reset();
-            concatTuple.addTuple(tokenTuple);
+            resultTuple.setTokenTuple(tokenTuple);
             isInvListCursorOpen = true;
         } else {
             isInvListCursorOpen = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index eff4f5a..8c6b386 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -117,7 +117,7 @@
                 short numTokens = ShortPointable.getShort(btreeTuple.getFieldData(PARTITIONING_NUM_TOKENS_FIELD),
                         btreeTuple.getFieldStart(PARTITIONING_NUM_TOKENS_FIELD));
                 InvertedListCursor invListCursor = partSearcher.getCachedInvertedListCursor();
-                openInvertedListCursor(btreeTuple, invListCursor);
+                openInvertedListCursor(btreeTuple, invListCursor, ctx);
                 invListPartitions.addInvertedListCursor(invListCursor, numTokens);
                 tokenExists = true;
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index 5da4702..f3ca8b0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -165,7 +165,7 @@
             // Needs to return the calculation result for the final list only.
             // Otherwise, the process needs to be continued until this method traverses the final inverted list
             // and either generates some output in the output buffer or finishes traversing it.
-            if (isFinalList && doneMerge) {
+            if (isFinalList) {
                 return doneMerge;
             }
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
index e583c7d..1a0f387 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class DelimitedUTF8StringBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -37,4 +44,21 @@
     public IBinaryTokenizer createTokenizer() {
         return new DelimitedUTF8StringBinaryTokenizer(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("tokenFactory", tokenFactory.toJson(registry));
+        json.put("ignoreTokenCount", ignoreTokenCount);
+        json.put("sourceHasTypeTag", sourceHasTypeTag);
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ITokenFactory tokenFactory = (ITokenFactory) registry.deserialize(json.get("tokenFactory"));
+        final boolean ignoreTokenCount = json.get("ignoreTokenCount").asBoolean();
+        final boolean sourceHasTypeTag = json.get("sourceHasTypeTag").asBoolean();
+        return new DelimitedUTF8StringBinaryTokenizerFactory(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
index 0bc01b6..2daa09d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class HashedUTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
 
     private static final long serialVersionUID = 1L;
@@ -35,4 +42,19 @@
     public IToken createToken() {
         return new HashedUTF8NGramToken(tokenTypeTag, countTypeTag);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("tokenTypeTag", tokenTypeTag);
+        json.put("countTypeTag", countTypeTag);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        final byte tokenTypeTag = (byte) json.get("tokenTypeTag").asInt();
+        final byte countTypeTag = (byte) json.get("countTypeTag").asInt();
+        return new HashedUTF8NGramTokenFactory(tokenTypeTag, countTypeTag);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
index 7819e16..e8a83d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class HashedUTF8WordTokenFactory extends AbstractUTF8TokenFactory {
 
     private static final long serialVersionUID = 1L;
@@ -35,4 +42,19 @@
     public IToken createToken() {
         return new HashedUTF8WordToken(tokenTypeTag, countTypeTag);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("tokenTypeTag", tokenTypeTag);
+        json.put("countTypeTag", countTypeTag);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        final byte tokenTypeTag = (byte) json.get("tokenTypeTag").asInt();
+        final byte countTypeTag = (byte) json.get("countTypeTag").asInt();
+        return new HashedUTF8WordTokenFactory(tokenTypeTag, countTypeTag);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java
index 01e83a3..77c604c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java
@@ -21,6 +21,8 @@
 
 import java.io.Serializable;
 
-public interface IBinaryTokenizerFactory extends Serializable {
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+public interface IBinaryTokenizerFactory extends Serializable, IJsonSerializable {
     public IBinaryTokenizer createTokenizer();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java
index 0439be4..21cadbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java
@@ -21,6 +21,8 @@
 
 import java.io.Serializable;
 
-public interface ITokenFactory extends Serializable {
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+public interface ITokenFactory extends Serializable, IJsonSerializable {
     public IToken createToken();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java
index a95072d..fbd9f8c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class NGramUTF8StringBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
     private static final long serialVersionUID = 1L;
@@ -43,4 +50,26 @@
                 tokenFactory);
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.set("tokenFactory", tokenFactory.toJson(registry));
+        json.put("gramLength", gramLength);
+        json.put("usePrePost", usePrePost);
+        json.put("ignoreTokenCount", ignoreTokenCount);
+        json.put("sourceHasTypeTag", sourceHasTypeTag);
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final ITokenFactory tokenFactory = (ITokenFactory) registry.deserialize(json.get("tokenFactory"));
+        final int gramLength = json.get("gramLength").asInt();
+        final boolean usePrePost = json.get("usePrePost").asBoolean();
+        final boolean ignoreTokenCount = json.get("ignoreTokenCount").asBoolean();
+        final boolean sourceHasTypeTag = json.get("sourceHasTypeTag").asBoolean();
+        return new NGramUTF8StringBinaryTokenizerFactory(gramLength, usePrePost, ignoreTokenCount, sourceHasTypeTag,
+                tokenFactory);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java
index 64f1e99..aa4d30f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class UTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
 
     private static final long serialVersionUID = 1L;
@@ -36,4 +43,18 @@
         return new UTF8NGramToken(tokenTypeTag, countTypeTag);
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("tokenTypeTag", tokenTypeTag);
+        json.put("countTypeTag", countTypeTag);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        final byte tokenTypeTag = (byte) json.get("tokenTypeTag").asInt();
+        final byte countTypeTag = (byte) json.get("countTypeTag").asInt();
+        return new UTF8NGramTokenFactory(tokenTypeTag, countTypeTag);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java
index 3a8a0483..d50e83b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java
@@ -19,6 +19,13 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class UTF8WordTokenFactory extends AbstractUTF8TokenFactory {
 
     private static final long serialVersionUID = 1L;
@@ -36,4 +43,19 @@
         return new UTF8WordToken(tokenTypeTag, countTypeTag);
     }
 
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("tokenTypeTag", tokenTypeTag);
+        json.put("countTypeTag", countTypeTag);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        final byte tokenTypeTag = (byte) json.get("tokenTypeTag").asInt();
+        final byte countTypeTag = (byte) json.get("countTypeTag").asInt();
+        return new UTF8NGramTokenFactory(tokenTypeTag, countTypeTag);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
new file mode 100644
index 0000000..102fe96
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.tuples;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class TokenKeyPairTuple implements ITupleReference {
+
+    private ITupleReference tokenTuple;
+    private ITupleReference keyTuple;
+
+    private final int tokenFieldCount;
+    private final int keyFieldCount;
+
+    private boolean newToken;
+
+    public TokenKeyPairTuple(int tokenFieldCount, int keyFieldCount) {
+        this.tokenFieldCount = tokenFieldCount;
+        this.keyFieldCount = keyFieldCount;
+
+    }
+
+    public void setTokenTuple(ITupleReference token) {
+        this.tokenTuple = token;
+        this.keyTuple = null;
+    }
+
+    public void setKeyTuple(ITupleReference key) {
+        newToken = this.keyTuple == null;
+        this.keyTuple = key;
+    }
+
+    public ITupleReference getTokenTuple() {
+        return tokenTuple;
+    }
+
+    public ITupleReference getKeyTuple() {
+        return keyTuple;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return tokenFieldCount + keyFieldCount;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldData(fieldIndex);
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldStart(fieldIndex);
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        ITupleReference tuple = getTuple(fIdx);
+        int fieldIndex = getFieldIndex(fIdx);
+        return tuple.getFieldLength(fieldIndex);
+    }
+
+    private ITupleReference getTuple(int fIdx) {
+        return fIdx < tokenFieldCount ? tokenTuple : keyTuple;
+    }
+
+    private int getFieldIndex(int fIdx) {
+        return fIdx < tokenFieldCount ? fIdx : fIdx - tokenFieldCount;
+    }
+
+    public boolean isNewToken() {
+        return newToken;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index ba54ed4..1e812b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -62,6 +62,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
 
 public class InvertedIndexUtils {
 
@@ -129,7 +130,7 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            IMetadataPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
                 invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -166,7 +167,7 @@
                 filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
                 ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -178,7 +179,7 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
             ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
             int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
                 invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -215,6 +216,6 @@
                 filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
index 1c118de..7ed450c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -91,5 +91,9 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
index f72e17c..f74a23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
@@ -27,6 +27,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -38,6 +40,9 @@
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
  * The local resource class for disk only lsm r-tree
  */
@@ -62,6 +67,15 @@
                 bloomFilterFalsePositiveRate);
     }
 
+    private ExternalRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields,
+            int[] buddyBTreeFields, boolean isPointMBR, double bloomFilterFalsePositiveRate)
+            throws HyracksDataException {
+        super(registry, json, btreeCmpFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory,
+                rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+    }
+
     @Override
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         IIOManager ioManager = ncServiceCtx.getIoManager();
@@ -75,4 +89,20 @@
                 buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory, ncServiceCtx.getTracer());
 
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        LSMRTreeLocalResource lsmRTree = (LSMRTreeLocalResource) LSMRTreeLocalResource.fromJson(registry, json);
+        return new ExternalRTreeLocalResource(registry, json, lsmRTree.btreeCmpFactories,
+                lsmRTree.valueProviderFactories, lsmRTree.rtreePolicyType, lsmRTree.linearizeCmpFactory,
+                lsmRTree.rtreeFields, lsmRTree.buddyBTreeFields, lsmRTree.isPointMBR,
+                lsmRTree.bloomFilterFalsePositiveRate);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index 634504b..d744ab3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.storage.am.lsm.rtree.dataflow;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -28,6 +29,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -42,6 +45,10 @@
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class LSMRTreeLocalResource extends LsmResource {
 
     private static final long serialVersionUID = 1L;
@@ -78,6 +85,22 @@
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
     }
 
+    protected LSMRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields,
+            int[] buddyBTreeFields, boolean isPointMBR, double bloomFilterFalsePositiveRate)
+            throws HyracksDataException {
+        super(registry, json);
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+        this.rtreeFields = rtreeFields;
+        this.buddyBTreeFields = buddyBTreeFields;
+        this.isPointMBR = isPointMBR;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
+    }
+
     @Override
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         IIOManager ioManager = ncServiceCtx.getIoManager();
@@ -93,4 +116,60 @@
                 buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields, durable, isPointMBR,
                 metadataPageManagerFactory);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    @Override
+    protected void appendToJson(final ObjectNode json, IPersistedResourceRegistry registry)
+            throws HyracksDataException {
+        super.appendToJson(json, registry);
+        final ArrayNode btreeCmpFactoriesArray = OBJECT_MAPPER.createArrayNode();
+        for (IBinaryComparatorFactory factory : btreeCmpFactories) {
+            btreeCmpFactoriesArray.add(factory.toJson(registry));
+        }
+        json.set("btreeCmpFactories", btreeCmpFactoriesArray);
+        json.set("linearizeCmpFactory", linearizeCmpFactory.toJson(registry));
+        final ArrayNode valueProviderFactoriesArray = OBJECT_MAPPER.createArrayNode();
+        for (IPrimitiveValueProviderFactory factory : valueProviderFactories) {
+            valueProviderFactoriesArray.add(factory.toJson(registry));
+        }
+        json.set("valueProviderFactories", valueProviderFactoriesArray);
+        json.set("rtreePolicyType", rtreePolicyType.toJson(registry));
+        json.putPOJO("rtreeFields", rtreeFields);
+        json.putPOJO("buddyBTreeFields", buddyBTreeFields);
+        json.put("isPointMBR", isPointMBR);
+        json.put("bloomFilterFalsePositiveRate", bloomFilterFalsePositiveRate);
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final int[] buddyBTreeFields = OBJECT_MAPPER.convertValue(json.get("buddyBTreeFields"), int[].class);
+        final int[] rtreeFields = OBJECT_MAPPER.convertValue(json.get("rtreeFields"), int[].class);
+        final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
+        final boolean isPointMBR = json.get("isPointMBR").asBoolean();
+        final RTreePolicyType rtreePolicyType = (RTreePolicyType) registry.deserialize(json.get("rtreePolicyType"));
+        final ILinearizeComparatorFactory linearizeCmpFactory =
+                (ILinearizeComparatorFactory) registry.deserialize(json.get("linearizeCmpFactory"));
+        final List<IBinaryComparatorFactory> btreeCmpFactoriesList = new ArrayList<>();
+        final ArrayNode jsonBtreeCmpFactories = (ArrayNode) json.get("btreeCmpFactories");
+        for (JsonNode cf : jsonBtreeCmpFactories) {
+            btreeCmpFactoriesList.add((IBinaryComparatorFactory) registry.deserialize(cf));
+        }
+        final IBinaryComparatorFactory[] btreeCmpFactories =
+                btreeCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
+        final List<IPrimitiveValueProviderFactory> valueProviderFactoriesList = new ArrayList<>();
+        final ArrayNode jsonValueProviderFactories = (ArrayNode) json.get("valueProviderFactories");
+        for (JsonNode cf : jsonValueProviderFactories) {
+            valueProviderFactoriesList.add((IPrimitiveValueProviderFactory) registry.deserialize(cf));
+        }
+        final IPrimitiveValueProviderFactory[] valueProviderFactories =
+                valueProviderFactoriesList.toArray(new IPrimitiveValueProviderFactory[0]);
+        return new LSMRTreeLocalResource(registry, json, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index f91a5f7..4b6f1de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.storage.am.lsm.rtree.dataflow;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -28,6 +29,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -42,11 +45,15 @@
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class LSMRTreeWithAntiMatterLocalResource extends LsmResource {
 
     private static final long serialVersionUID = 1L;
 
-    protected final IBinaryComparatorFactory[] btreeComparatorFactories;
+    protected final IBinaryComparatorFactory[] btreeCmpFactories;
     protected final IPrimitiveValueProviderFactory[] valueProviderFactories;
     protected final RTreePolicyType rtreePolicyType;
     protected final ILinearizeComparatorFactory linearizeCmpFactory;
@@ -59,13 +66,26 @@
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeComparatorFactories,
+            Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable);
-        this.btreeComparatorFactories = btreeComparatorFactories;
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+        this.rtreeFields = rtreeFields;
+        this.isPointMBR = isPointMBR;
+    }
+
+    private LSMRTreeWithAntiMatterLocalResource(IPersistedResourceRegistry registry, JsonNode json,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields,
+            boolean isPointMBR) throws HyracksDataException {
+        super(registry, json);
+        this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
         this.linearizeCmpFactory = linearizeCmpFactory;
@@ -80,11 +100,59 @@
         List<IVirtualBufferCache> virtualBufferCaches = vbcProvider.getVirtualBufferCaches(serviceCtx, file);
         ioOpCallbackFactory.initialize(serviceCtx, this);
         return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(ioManager, virtualBufferCaches, file,
-                storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, btreeComparatorFactories,
+                storageManager.getBufferCache(serviceCtx), typeTraits, cmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType,
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits, filterCmpFactories,
                 filterFields, durable, isPointMBR, metadataPageManagerFactory);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        super.appendToJson(jsonObject, registry);
+        ArrayNode btreeCmpFactoriesArray = OBJECT_MAPPER.createArrayNode();
+        for (IBinaryComparatorFactory factory : btreeCmpFactories) {
+            btreeCmpFactoriesArray.add(factory.toJson(registry));
+        }
+        jsonObject.set("btreeCmpFactories", btreeCmpFactoriesArray);
+        jsonObject.set("linearizeCmpFactory", linearizeCmpFactory.toJson(registry));
+
+        final ArrayNode valueProviderFactoriesArray = OBJECT_MAPPER.createArrayNode();
+        for (IPrimitiveValueProviderFactory factory : valueProviderFactories) {
+            valueProviderFactoriesArray.add(factory.toJson(registry));
+        }
+        jsonObject.set("valueProviderFactories", valueProviderFactoriesArray);
+        jsonObject.set("rtreePolicyType", rtreePolicyType.toJson(registry));
+        jsonObject.putPOJO("rtreeFields", rtreeFields);
+        jsonObject.put("isPointMBR", isPointMBR);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final int[] rtreeFields = OBJECT_MAPPER.convertValue(json.get("rtreeFields"), int[].class);
+        final boolean isPointMBR = json.get("isPointMBR").asBoolean();
+        final RTreePolicyType rtreePolicyType = (RTreePolicyType) registry.deserialize(json.get("rtreePolicyType"));
+        final ILinearizeComparatorFactory linearizeCmpFactory =
+                (ILinearizeComparatorFactory) registry.deserialize(json.get("linearizeCmpFactory"));
+
+        final List<IBinaryComparatorFactory> btreeCmpFactoriesList = new ArrayList<>();
+        final ArrayNode jsonBtreeCmpFactories = (ArrayNode) json.get("btreeCmpFactories");
+        for (JsonNode cf : jsonBtreeCmpFactories) {
+            btreeCmpFactoriesList.add((IBinaryComparatorFactory) registry.deserialize(cf));
+        }
+        final IBinaryComparatorFactory[] btreeCmpFactories =
+                btreeCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
+        final List<IPrimitiveValueProviderFactory> valueProviderFactoriesList = new ArrayList<>();
+        final ArrayNode jsonValueProviderFactories = (ArrayNode) json.get("valueProviderFactories");
+        for (JsonNode cf : jsonValueProviderFactories) {
+            valueProviderFactoriesList.add((IPrimitiveValueProviderFactory) registry.deserialize(cf));
+        }
+        final IPrimitiveValueProviderFactory[] valueProviderFactories =
+                valueProviderFactoriesList.toArray(new IPrimitiveValueProviderFactory[0]);
+        return new LSMRTreeWithAntiMatterLocalResource(registry, json, btreeCmpFactories, valueProviderFactories,
+                rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index e36abb4..c695a5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -97,9 +97,8 @@
             BTree memBTree = new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
                     btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, btreeCmpFactories.length,
                     ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_b_" + i));
-            LSMRTreeMemoryComponent mutableComponent =
-                    new LSMRTreeMemoryComponent(this, memRTree, memBTree, virtualBufferCache, i == 0 ? true : false,
-                            filterHelper == null ? null : filterHelper.createFilter());
+            LSMRTreeMemoryComponent mutableComponent = new LSMRTreeMemoryComponent(this, memRTree, memBTree,
+                    virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
             ++i;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
index 7b12250..015f34c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -30,7 +31,6 @@
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
 import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
@@ -40,8 +40,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -50,9 +48,10 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -60,6 +59,7 @@
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.util.trace.ITracer;
 
 /**
@@ -323,12 +323,6 @@
         if (!isActive) {
             throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
         }
-        if (flushOnExit) {
-            AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
-            opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
-            ioOpCallback.afterFinalize(opCtx);
-        }
-
         for (ILSMDiskComponent c : diskComponents) {
             c.deactivateAndPurge();
         }
@@ -383,16 +377,13 @@
         version = 0;
     }
 
-    // Not supported
     @Override
     public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
         throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree");
     }
 
-    // Not supported
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
     }
 
@@ -445,152 +436,139 @@
 
     // For initial load
     @Override
-    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false);
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false, parameters);
     }
 
     // For transaction bulk load <- could consolidate with the above method ->
     @Override
-    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
-            throws HyracksDataException {
-        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+    public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+            Map<String, Object> parameters) throws HyracksDataException {
+        return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
     }
 
     // The bulk loader used for both initial loading and transaction
     // modifications
     public class LSMTwoPCRTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
         private final ILSMDiskComponent component;
-        private final IIndexBulkLoader rtreeBulkLoader;
-        private final BTreeBulkLoader btreeBulkLoader;
-        private final IIndexBulkLoader builder;
-        private boolean cleanedUpArtifacts = false;
-        private boolean isEmptyComponent = true;
-        private boolean endedBloomFilterLoad = false;
         private final boolean isTransaction;
+        private final LoadOperation loadOp;
+        private final ChainedLSMDiskComponentBulkLoader componentBulkLoader;
 
         public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
-                boolean isTransaction) throws HyracksDataException {
+                boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
             this.isTransaction = isTransaction;
             // Create the appropriate target
+            LSMComponentFileReferences componentFileRefs;
             if (isTransaction) {
-                component = createTransactionTarget();
+                try {
+                    componentFileRefs = fileManager.getNewTransactionFileReference();
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+                component = createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+                        componentFileRefs.getDeleteIndexFileReference(),
+                        componentFileRefs.getBloomFilterFileReference(), true);
             } else {
-                component = createBulkLoadTarget();
+                componentFileRefs = fileManager.getRelFlushFileReference();
+                component =
+                        createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                                componentFileRefs.getDeleteIndexFileReference(),
+                                componentFileRefs.getBloomFilterFileReference(), true);
             }
 
-            // Create the three loaders
-            rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getIndex().createBulkLoader(fillFactor, verifyInput,
-                    numElementsHint, false);
-            btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBuddyIndex()
-                    .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
-            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
-            BloomFilterSpecification bloomFilterSpec =
-                    BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
-            builder = ((LSMRTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
-                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+            loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+            loadOp.setNewComponent(component);
+            ioOpCallback.scheduled(loadOp);
+            ioOpCallback.beforeOperation(loadOp);
+            componentBulkLoader =
+                    component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
         }
 
         @Override
         public void add(ITupleReference tuple) throws HyracksDataException {
-            try {
-                rtreeBulkLoader.add(tuple);
-            } catch (Exception e) {
-                cleanupArtifacts();
-                throw e;
-            }
-            if (isEmptyComponent) {
-                isEmptyComponent = false;
-            }
-        }
-
-        // This is made public in case of a failure, it is better to delete all
-        // created artifacts.
-        public void cleanupArtifacts() throws HyracksDataException {
-            if (!cleanedUpArtifacts) {
-                cleanedUpArtifacts = true;
-                ((LSMRTreeDiskComponent) component).deactivateAndDestroy();
-            }
+            componentBulkLoader.add(tuple);
         }
 
         @Override
         public void end() throws HyracksDataException {
-            if (!cleanedUpArtifacts) {
-                if (!endedBloomFilterLoad) {
-                    builder.end();
-                    endedBloomFilterLoad = true;
+            try {
+                ioOpCallback.afterOperation(loadOp);
+                componentBulkLoader.end();
+                if (component.getComponentSize() > 0) {
+                    if (isTransaction) {
+                        // Since this is a transaction component, validate and
+                        // deactivate. it could later be added or deleted
+                        try {
+                            component.markAsValid(durable, loadOp);
+                        } finally {
+                            ioOpCallback.afterFinalize(loadOp);
+                        }
+                        component.deactivate();
+                    } else {
+                        ioOpCallback.afterFinalize(loadOp);
+                        getHarness().addBulkLoadedComponent(loadOp);
+                    }
                 }
-                rtreeBulkLoader.end();
-                btreeBulkLoader.end();
-                if (isEmptyComponent) {
-                    cleanupArtifacts();
-                } else if (isTransaction) {
-                    // Since this is a transaction component, validate and
-                    // deactivate. it could later be added or deleted
-                    component.markAsValid(durable);
-                    component.deactivate();
-                } else {
-                    getHarness().addBulkLoadedComponent(component);
-                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
         @Override
         public void delete(ITupleReference tuple) throws HyracksDataException {
+            componentBulkLoader.delete(tuple);
+        }
+
+        @Override
+        public void abort() throws HyracksDataException {
             try {
-                btreeBulkLoader.add(tuple);
-                builder.add(tuple);
-            } catch (Exception e) {
-                cleanupArtifacts();
-                throw e;
-            }
-            if (isEmptyComponent) {
-                isEmptyComponent = false;
+                try {
+                    componentBulkLoader.abort();
+                } finally {
+                    ioOpCallback.afterFinalize(loadOp);
+                }
+            } finally {
+                ioOpCallback.completed(loadOp);
             }
         }
 
         @Override
-        public void abort() {
-            try {
-                cleanupArtifacts();
-            } catch (Exception e) {
-
-            }
+        public void writeFailed(ICachedPage page, Throwable failure) {
+            throw new UnsupportedOperationException();
         }
 
-        // This method is used to create a target for a bulk modify operation. This
-        // component must then eventually be either committed or deleted
-        private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
-            LSMComponentFileReferences componentFileRefs;
-            try {
-                componentFileRefs = fileManager.getNewTransactionFileReference();
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-            return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
-                    componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                    true);
+        @Override
+        public boolean hasFailed() {
+            return loadOp.hasFailed();
+        }
+
+        @Override
+        public Throwable getFailure() {
+            return loadOp.getFailure();
         }
     }
 
     // The only change the the schedule merge is the method used to create the
     // opCtx. first line <- in schedule merge, we->
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
         rctx.setOperation(IndexOperation.MERGE);
         List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
         LSMRTreeSortedCursor cursor = new LSMRTreeSortedCursor(rctx, linearizer, buddyBTreeFields);
         LSMComponentFileReferences relMergeFileRefs =
-                getMergeFileReferences((ILSMDiskComponent) mergingComponents.get(0),
-                        (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1));
+                getMergeFileReferences((ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1),
+                        (ILSMDiskComponent) mergingComponents.get(0));
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), rctx, buddyBTreeFields);
         // create the merge operation.
-        LSMRTreeMergeOperation mergeOp = new LSMRTreeMergeOperation(accessor, cursor,
-                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
-                relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
-        ioScheduler.scheduleOperation(mergeOp);
+        LSMRTreeMergeOperation mergeOp =
+                new LSMRTreeMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
+                        relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
+                        ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+        ioOpCallback.scheduled(mergeOp);
+        return mergeOp;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 4510618..4566d32 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -134,8 +134,8 @@
                 rTreeTupleSorter.sort();
                 component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
                         flushOp.getBloomFilterTarget(), true);
-                componentBulkLoader =
-                        component.createBulkLoader(1.0f, false, numBTreeTuples.longValue(), false, false, false);
+                componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
+                        false, false, false);
                 flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
                 // scan the memory BTree and bulk load delete tuples
                 flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -332,11 +332,12 @@
                                 .getNumElements();
                     }
                     componentBulkLoader =
-                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false, false);
                     mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
                 } else {
                     //no buddy-btree needed
-                    componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false);
                 }
                 //search old rtree components
                 while (cursor.hasNext()) {
@@ -423,7 +424,7 @@
         LSMRTreeAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
                 componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -433,6 +434,6 @@
         ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
-                fileManager.getBaseDir().getAbsolutePath());
+                getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
index 001228c..3348407 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.storage.am.lsm.rtree.impls;
 
-import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -37,6 +36,7 @@
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
 
@@ -59,22 +59,15 @@
 
     @Override
     public LSMComponentFileReferences getRelFlushFileReference() throws HyracksDataException {
-        String ts = getCurrentTimestamp();
-        String baseName = ts + DELIMITER + ts;
-        // Begin timestamp and end timestamp are identical since it is a flush
+        String baseName = getNextComponentSequence(btreeFilter);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + RTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
     }
 
     @Override
-    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
-            throws HyracksDataException {
-        String[] firstTimestampRange = firstFileName.split(DELIMITER);
-        String[] lastTimestampRange = lastFileName.split(DELIMITER);
-        String baseName = firstTimestampRange[0] + DELIMITER + lastTimestampRange[1];
-        // Get the range of timestamps by taking the earliest and the latest
-        // timestamps
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName) {
+        final String baseName = IndexComponentFileReference.getMergeSequence(firstFileName, lastFileName);
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + RTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
@@ -83,23 +76,24 @@
     @Override
     public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException {
         List<LSMComponentFileReferences> validFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allRTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<>();
-        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allRTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBTreeFiles = new ArrayList<>();
+        ArrayList<IndexComponentFileReference> allBloomFilterFiles = new ArrayList<>();
 
         // Create a transaction filter <- to hide transaction components->
         FilenameFilter transactionFilter = getTransactionFileFilter(false);
 
         // Gather files.
-        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles);
+        cleanupAndGetValidFilesInternal(getCompoundFilter(transactionFilter, btreeFilter), btreeFactory, allBTreeFiles,
+                btreeFactory.getBufferCache());
         HashSet<String> btreeFilesSet = new HashSet<>();
-        for (ComparableFileName cmpFileName : allBTreeFiles) {
-            int index = cmpFileName.fileName.lastIndexOf(DELIMITER);
-            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+        for (IndexComponentFileReference cmpFileName : allBTreeFiles) {
+            btreeFilesSet.add(cmpFileName.getSequence());
         }
-        validateFiles(btreeFilesSet, allRTreeFiles, getCompoundFilter(transactionFilter, rtreeFilter), rtreeFactory);
-        validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(transactionFilter, bloomFilterFilter),
-                null);
+        validateFiles(btreeFilesSet, allRTreeFiles, getCompoundFilter(transactionFilter, rtreeFilter), rtreeFactory,
+                btreeFactory.getBufferCache());
+        validateFiles(btreeFilesSet, allBloomFilterFiles, getCompoundFilter(transactionFilter, bloomFilterFilter), null,
+                btreeFactory.getBufferCache());
 
         // Sanity check.
         if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
@@ -112,52 +106,47 @@
         }
 
         if (allRTreeFiles.size() == 1 && allBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
-            validFiles.add(new LSMComponentFileReferences(allRTreeFiles.get(0).fileRef, allBTreeFiles.get(0).fileRef,
-                    allBloomFilterFiles.get(0).fileRef));
+            validFiles.add(new LSMComponentFileReferences(allRTreeFiles.get(0).getFileRef(),
+                    allBTreeFiles.get(0).getFileRef(), allBloomFilterFiles.get(0).getFileRef()));
             return validFiles;
         }
 
-        // Sorts files names from earliest to latest timestamp.
+        // Sorts files names from earliest to latest sequence.
         Collections.sort(allRTreeFiles);
         Collections.sort(allBTreeFiles);
         Collections.sort(allBloomFilterFiles);
 
-        List<ComparableFileName> validComparableRTreeFiles = new ArrayList<>();
-        ComparableFileName lastRTree = allRTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableRTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastRTree = allRTreeFiles.get(0);
         validComparableRTreeFiles.add(lastRTree);
 
-        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<>();
-        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        List<IndexComponentFileReference> validComparableBTreeFiles = new ArrayList<>();
+        IndexComponentFileReference lastBTree = allBTreeFiles.get(0);
         validComparableBTreeFiles.add(lastBTree);
 
-        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<>();
-        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        List<IndexComponentFileReference> validComparableBloomFilterFiles = new ArrayList<>();
+        IndexComponentFileReference lastBloomFilter = allBloomFilterFiles.get(0);
         validComparableBloomFilterFiles.add(lastBloomFilter);
 
         for (int i = 1; i < allRTreeFiles.size(); i++) {
-            ComparableFileName currentRTree = allRTreeFiles.get(i);
-            ComparableFileName currentBTree = allBTreeFiles.get(i);
-            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
-            // Current start timestamp is greater than last stop timestamp.
-            if (currentRTree.interval[0].compareTo(lastRTree.interval[1]) > 0
-                    && currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+            IndexComponentFileReference currentRTree = allRTreeFiles.get(i);
+            IndexComponentFileReference currentBTree = allBTreeFiles.get(i);
+            IndexComponentFileReference currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start sequence is greater than last stop sequence.
+            if (currentRTree.isMoreRecentThan(lastRTree) && currentBTree.isMoreRecentThan(lastBTree)
+                    && currentBloomFilter.isMoreRecentThan(lastBloomFilter)) {
                 validComparableRTreeFiles.add(currentRTree);
                 validComparableBTreeFiles.add(currentBTree);
                 validComparableBloomFilterFiles.add(currentBloomFilter);
                 lastRTree = currentRTree;
                 lastBTree = currentBTree;
                 lastBloomFilter = currentBloomFilter;
-            } else if (currentRTree.interval[0].compareTo(lastRTree.interval[0]) >= 0
-                    && currentRTree.interval[1].compareTo(lastRTree.interval[1]) <= 0
-                    && currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
-                    && currentBTree.interval[1].compareTo(lastBTree.interval[1]) <= 0
-                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
-                    && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
-                // Invalid files are completely contained in last interval.
-                IoUtil.delete(new File(currentRTree.fullPath));
-                IoUtil.delete(new File(currentBTree.fullPath));
-                IoUtil.delete(new File(currentBloomFilter.fullPath));
+            } else if (currentRTree.isWithin(lastRTree) && currentBTree.isWithin(lastBTree)
+                    && currentBloomFilter.isWithin(lastBloomFilter)) {
+                // Invalid files are completely contained in last sequence.
+                delete(treeFactory.getBufferCache(), currentRTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentBTree.getFullPath());
+                delete(treeFactory.getBufferCache(), currentBloomFilter.getFullPath());
             } else {
                 // This scenario should not be possible.
                 throw HyracksDataException.create(ErrorCode.FOUND_OVERLAPPING_LSM_FILES, baseDir);
@@ -166,29 +155,28 @@
 
         // Sort valid files in reverse lexicographical order, such that newer
         // files come first.
-        Collections.sort(validComparableRTreeFiles, recencyCmp);
-        Collections.sort(validComparableBTreeFiles, recencyCmp);
-        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+        validComparableRTreeFiles.sort(recencyCmp);
+        validComparableBTreeFiles.sort(recencyCmp);
+        validComparableBloomFilterFiles.sort(recencyCmp);
 
-        Iterator<ComparableFileName> rtreeFileIter = validComparableRTreeFiles.iterator();
-        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
-        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        Iterator<IndexComponentFileReference> rtreeFileIter = validComparableRTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<IndexComponentFileReference> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
         while (rtreeFileIter.hasNext() && btreeFileIter.hasNext()) {
-            ComparableFileName cmpRTreeFileName = rtreeFileIter.next();
-            ComparableFileName cmpBTreeFileName = btreeFileIter.next();
-            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
-            validFiles.add(new LSMComponentFileReferences(cmpRTreeFileName.fileRef, cmpBTreeFileName.fileRef,
-                    cmpBloomFilterFileName.fileRef));
+            IndexComponentFileReference cmpRTreeFileName = rtreeFileIter.next();
+            IndexComponentFileReference cmpBTreeFileName = btreeFileIter.next();
+            IndexComponentFileReference cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpRTreeFileName.getFileRef(), cmpBTreeFileName.getFileRef(),
+                    cmpBloomFilterFileName.getFileRef()));
         }
         return validFiles;
     }
 
     @Override
     public LSMComponentFileReferences getNewTransactionFileReference() throws IOException {
-        String ts = getCurrentTimestamp();
+        String baseName = getNextComponentSequence(btreeFilter);
         // Create transaction lock file
-        Files.createFile(Paths.get(baseDir + TXN_PREFIX + ts));
-        String baseName = ts + DELIMITER + ts;
+        Files.createFile(Paths.get(baseDir + TXN_PREFIX + baseName));
         return new LSMComponentFileReferences(baseDir.getChild(baseName + DELIMITER + RTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BTREE_SUFFIX),
                 baseDir.getChild(baseName + DELIMITER + BLOOM_FILTER_SUFFIX));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
index 2e1ba68..ef7b815 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
@@ -21,9 +21,9 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
 
 public class LSMRTreeMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -32,8 +32,8 @@
     private final BTree btree;
 
     public LSMRTreeMemoryComponent(AbstractLSMRTree lsmIndex, RTree rtree, BTree btree, IVirtualBufferCache vbc,
-            boolean isActive, ILSMComponentFilter filter) {
-        super(lsmIndex, vbc, isActive, filter);
+            ILSMComponentFilter filter) {
+        super(lsmIndex, vbc, filter);
         this.rtree = rtree;
         this.btree = btree;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
index eb79960..63fd47f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
@@ -29,7 +29,6 @@
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
@@ -37,7 +36,6 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
 import org.apache.hyracks.storage.am.rtree.impls.RTree;
 import org.apache.hyracks.storage.am.rtree.impls.RTreeOpContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index c79735f..b9e174f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -98,7 +98,8 @@
                 rtreeCursors[currentCursor].next();
                 ITupleReference currentTuple = rtreeCursors[currentCursor].getTuple();
                 // Call proceed() to do necessary operations before returning this tuple.
-                resultOfsearchCallbackProceed = searchCallback.proceed(currentTuple);
+                resultOfsearchCallbackProceed =
+                        currentCursor == 0 && includeMutableComponent ? searchCallback.proceed(currentTuple) : true;
                 btreeTuple.reset(rtreeCursors[currentCursor].getTuple());
                 boolean killerTupleFound = false;
                 for (int i = 0; i < currentCursor && !killerTupleFound; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index a3ba4b1..2757595 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -109,7 +109,7 @@
                 try {
                     memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
                     component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
-                    componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
+                    componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
                     // Since the LSM-RTree is used as a secondary assumption, the
                     // primary key will be the last comparator in the BTree comparators
                     rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
@@ -235,7 +235,7 @@
         ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
 
         ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, 0L, false, false, false);
+                component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
         try {
             while (cursor.hasNext()) {
                 cursor.next();
@@ -273,7 +273,7 @@
             throws HyracksDataException {
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 
     @Override
@@ -288,6 +288,6 @@
                 new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx, returnDeletedTuples);
         ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
         return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(), null, null,
-                callback, fileManager.getBaseDir().getAbsolutePath());
+                callback, getIndexIdentifier());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 7db65bd..05dc3ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -177,7 +177,7 @@
                 // reconcile() and complete() can be added later after considering the semantics.
 
                 // Call proceed() to do necessary operations before returning this tuple.
-                resultOfsearchCallBackProceed = searchCallback.proceed(diskRTreeTuple);
+                resultOfsearchCallBackProceed = true;
                 if (searchMemBTrees(diskRTreeTuple, numMemoryComponents)) {
                     // anti-matter tuple is NOT found
                     foundNext = true;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
index 1432aba..e28d5a2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
@@ -128,7 +128,9 @@
 
     @Override
     protected int getNullFlagsBytes() {
-        return BitOperationUtils.getFlagBytes(inputTotalFieldCount + (antimatterAware ? 1 : 0));
+        // stored key field count + value field count
+        return BitOperationUtils.getFlagBytes(
+                storedKeyFieldCount + inputTotalFieldCount - inputKeyFieldCount + (antimatterAware ? 1 : 0));
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
index 0d08f23..e604492 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -77,5 +77,9 @@
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
deleted file mode 100644
index 1ca75cb..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.hyracks.storage.am.rtree.api;
-
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-
-public interface IGenericPrimitiveSerializerDeserializer<T> extends ISerializerDeserializer<T> {
-    public double getValue(byte[] bytes, int offset);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 18ced6d..4dcfa94 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -55,8 +55,9 @@
             boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
             byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
             throws HyracksDataException {
+        // TODO: predicate & limit pushdown not enabled for RTree yet
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
                 appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
         if (keyFields != null && keyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
index 5550e1f..2bd8c59 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
@@ -26,6 +26,8 @@
 
 public class RTreeNSMInteriorFrameFactory extends RTreeFrameFactory {
 
+    private static final long serialVersionUID = -8880568594091620229L;
+
     public RTreeNSMInteriorFrameFactory(RTreeTypeAwareTupleWriterFactory tupleWriterFactory,
             IPrimitiveValueProviderFactory[] keyValueProviderFactories, RTreePolicyType rtreePolicyType,
             boolean isPointMBR) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
index 16b22c0..166ece9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
 public class RTreeNSMLeafFrameFactory extends RTreeFrameFactory {
+    private static final long serialVersionUID = 1360338463029768516L;
 
     public RTreeNSMLeafFrameFactory(RTreeTypeAwareTupleWriterFactory tupleWriterFactory,
             IPrimitiveValueProviderFactory[] keyValueProviderFactories, RTreePolicyType rtreePolicyType,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreePolicyType.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreePolicyType.java
index d9dbd81..ed4656c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreePolicyType.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreePolicyType.java
@@ -19,7 +19,47 @@
 
 package org.apache.hyracks.storage.am.rtree.frames;
 
-public enum RTreePolicyType {
-    RTREE,
-    RSTARTREE
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public enum RTreePolicyType implements IJsonSerializable {
+    RTREE("RTREE"),
+    RSTARTREE("RSTARTREE");
+
+    private static final Map<String, RTreePolicyType> namesMap = new HashMap<>(2);
+
+    static {
+        namesMap.put("RTREE", RTREE);
+        namesMap.put("RSTARTREE", RSTARTREE);
+    }
+
+    private static final long serialVersionUID = 1L;
+    private final String type;
+
+    RTreePolicyType(String type) {
+        this.type = type;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getDeclaringClass(), serialVersionUID);
+        json.put("type", getType());
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return namesMap.get(json.get("type").asText());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
index 9842e23..14c5eac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
@@ -18,10 +18,15 @@
  */
 package org.apache.hyracks.storage.am.rtree.impls;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class DoublePrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
     private static final long serialVersionUID = 1L;
 
@@ -39,4 +44,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
index 7423cfb..f3c4b62 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
@@ -18,10 +18,15 @@
  */
 package org.apache.hyracks.storage.am.rtree.impls;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.primitive.FloatPointable;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class FloatPrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
     private static final long serialVersionUID = 1L;
 
@@ -39,4 +44,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
index 13eddd4..3e26d82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
@@ -18,10 +18,15 @@
  */
 package org.apache.hyracks.storage.am.rtree.impls;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public class IntegerPrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
     private static final long serialVersionUID = 1L;
 
@@ -39,4 +44,14 @@
             }
         };
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        return registry.getClassIdentifier(getClass(), serialVersionUID);
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return INSTANCE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 0e455c5..635fe7a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -941,11 +941,10 @@
                     propagateBulk(1, false, pagesToWrite);
 
                     leafFrontier.pageId = freePageManager.takePage(metaFrame);
-                    queue.put(leafFrontier.page);
+                    queue.put(leafFrontier.page, this);
                     for (ICachedPage c : pagesToWrite) {
-                        queue.put(c);
+                        queue.put(c, this);
                     }
-
                     pagesToWrite.clear();
                     leafFrontier.page = bufferCache
                             .confiscatePage(BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId));
@@ -975,7 +974,7 @@
             }
 
             for (ICachedPage c : pagesToWrite) {
-                queue.put(c);
+                queue.put(c, this);
             }
             finish();
             super.end();
@@ -1007,11 +1006,12 @@
                 //set next guide MBR
                 //if propagateBulk didnt have to do anything this may be un-necessary
                 if (nodeFrontiers.size() > 1 && nodeFrontiers.indexOf(n) < nodeFrontiers.size() - 1) {
+                    lowerFrame = nodeFrontiers.indexOf(n) != 0 ? prevInteriorFrame : leafFrame;
                     lowerFrame.setPage(n.page);
                     ((RTreeNSMFrame) lowerFrame).adjustMBR();
                     interiorFrameTupleWriter.writeTupleFields(((RTreeNSMFrame) lowerFrame).getMBRTuples(), 0, mbr, 0);
                 }
-                queue.put(n.page);
+                queue.put(n.page, this);
                 n.page = null;
                 prevPageId = n.pageId;
             }
@@ -1021,7 +1021,6 @@
 
         protected void propagateBulk(int level, boolean toRoot, List<ICachedPage> pagesToWrite)
                 throws HyracksDataException {
-            boolean propagated = false;
 
             if (level == 1) {
                 lowerFrame = leafFrame;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java
index 4a1637a..05ba749 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java
@@ -20,6 +20,12 @@
 
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class HilbertDoubleComparatorFactory implements ILinearizeComparatorFactory {
     private static final long serialVersionUID = 1L;
@@ -38,4 +44,16 @@
     public ILinearizeComparator createBinaryComparator() {
         return new HilbertDoubleComparator(dim);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("dim", dim);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return get(json.get("dim").asInt());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java
index f5c6046..c35983c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java
@@ -20,6 +20,12 @@
 
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ZCurveDoubleComparatorFactory implements ILinearizeComparatorFactory {
     private static final long serialVersionUID = 1L;
@@ -38,4 +44,16 @@
     public ILinearizeComparator createBinaryComparator() {
         return new ZCurveDoubleComparator(dim);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("dim", dim);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return get(json.get("dim").asInt());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java
index 6a25995..f257c5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java
@@ -20,6 +20,12 @@
 
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ZCurveIntComparatorFactory implements ILinearizeComparatorFactory {
     private static final long serialVersionUID = 1L;
@@ -38,4 +44,16 @@
     public ILinearizeComparator createBinaryComparator() {
         return new ZCurveIntComparator(dim);
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("dim", dim);
+        return json;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        return get(json.get("dim").asInt());
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index 281960d..a76fe48 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -49,8 +49,22 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index fc54903..138705f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -20,8 +20,9 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
 
-public interface IIndexBulkLoader {
+public interface IIndexBulkLoader extends IPageWriteFailureCallback {
     /**
      * Append a tuple to the index in the context of a bulk load.
      *
@@ -35,7 +36,8 @@
     public void add(ITupleReference tuple) throws HyracksDataException;
 
     /**
-     * Finalize the bulk loading operation in the given context.
+     * Finalize the bulk loading operation in the given context and release all resources.
+     * After this method is called, caller can't add more tuples nor abort
      *
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResource.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResource.java
index 7b9166d..ec8f7ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IResource.java
@@ -22,8 +22,9 @@
 
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
 
-public interface IResource extends Serializable {
+public interface IResource extends Serializable, IJsonSerializable {
 
     IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java
index dfc2f41..d985f31 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IStorageManager.java
@@ -21,13 +21,14 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 
 /**
  * Provides storage components during hyracks tasks execution
  */
-public interface IStorageManager extends Serializable {
+public interface IStorageManager extends Serializable, IJsonSerializable {
     /**
      * @param ctx
      *            the nc service context
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/LocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/LocalResource.java
index 93c4dea..f5e2945 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/LocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/LocalResource.java
@@ -20,7 +20,14 @@
 
 import java.io.Serializable;
 
-public class LocalResource implements Serializable {
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class LocalResource implements Serializable, IJsonSerializable {
     private static final long serialVersionUID = 2L;
     /*
      * object members
@@ -102,4 +109,23 @@
                 .append(id).append(", \"resource\" : ").append(resource).append(", \"version\" : ").append(version)
                 .append(" } ").toString();
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+        json.put("id", id);
+        json.put("version", version);
+        json.put("durable", durable);
+        json.set("resource", resource.toJson(registry));
+        return json;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final long id = json.get("id").asLong();
+        final int version = json.get("version").asInt();
+        final boolean durable = json.get("durable").asBoolean();
+        final IResource resource = (IResource) registry.deserialize(json.get("resource"));
+        return new LocalResource(id, version, durable, resource);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
index dbead1e..589d697 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
@@ -24,15 +24,19 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class AsyncFIFOPageQueueManager implements Runnable {
-    private final static boolean DEBUG = false;
+    private static final boolean DEBUG = false;
+    private static final Logger LOGGER = LogManager.getLogger();
 
-    protected LinkedBlockingQueue<ICachedPage> queue = new LinkedBlockingQueue<ICachedPage>();
+    protected LinkedBlockingQueue<ICachedPage> queue = new LinkedBlockingQueue<>();
     volatile Thread writerThread;
     protected AtomicBoolean poisoned = new AtomicBoolean(false);
     protected BufferCache bufferCache;
-    volatile protected PageQueue pageQueue;
+    protected volatile PageQueue pageQueue;
 
     public AsyncFIFOPageQueueManager(BufferCache bufferCache) {
         this.bufferCache = bufferCache;
@@ -57,17 +61,27 @@
             return writer;
         }
 
+        @SuppressWarnings("squid:S2142")
         @Override
-        public void put(ICachedPage page) throws HyracksDataException {
+        public void put(ICachedPage page, IPageWriteFailureCallback callback) throws HyracksDataException {
+            failIfPreviousPageFailed(callback);
+            page.setFailureCallback(callback);
             try {
                 if (!poisoned.get()) {
                     queue.put(page);
                 } else {
-                    throw new HyracksDataException("Queue is closing");
+                    LOGGER.error("An attempt to write a page found buffer cache closed");
+                    ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
                 }
             } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw HyracksDataException.create(e);
+                LOGGER.error("IO Operation interrupted", e);
+                ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+            }
+        }
+
+        private void failIfPreviousPageFailed(IPageWriteFailureCallback callback) throws HyracksDataException {
+            if (callback.hasFailed()) {
+                throw HyracksDataException.create(callback.getFailure());
             }
         }
     }
@@ -136,18 +150,21 @@
         }
     }
 
+    @SuppressWarnings("squid:S2142")
     @Override
     public void run() {
-        if (DEBUG)
-            System.out.println("[FIFO] Writer started");
+        if (DEBUG) {
+            LOGGER.info("[FIFO] Writer started");
+        }
         boolean die = false;
         while (!die) {
             ICachedPage entry;
             try {
                 entry = queue.take();
             } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                break;
+                LOGGER.error("BufferCache Write Queue was interrupted", e);
+                ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+                return; // Keep compiler happy
             }
             if (entry.getQueueInfo() != null && entry.getQueueInfo().hasWaiters()) {
                 synchronized (entry) {
@@ -158,17 +175,11 @@
                     continue;
                 }
             }
-
-            if (DEBUG)
-                System.out.println("[FIFO] Write " + BufferedFileHandle.getFileId(((CachedPage) entry).dpid) + ","
+            if (DEBUG) {
+                LOGGER.info("[FIFO] Write " + BufferedFileHandle.getFileId(((CachedPage) entry).dpid) + ","
                         + BufferedFileHandle.getPageId(((CachedPage) entry).dpid));
-
-            try {
-                pageQueue.getWriter().write(entry, bufferCache);
-            } catch (HyracksDataException e) {
-                //TODO: What do we do, if we could not write the page?
-                e.printStackTrace();
             }
+            pageQueue.getWriter().write(entry, bufferCache);
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 1443bbc..1e3f85b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.storage.common.buffercache;
 
+import static org.apache.hyracks.control.nc.io.IOManager.IO_REQUEST_QUEUE_SIZE;
+
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
@@ -27,9 +29,9 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
@@ -44,6 +46,7 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.replication.IIOReplicationManager;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 import org.apache.hyracks.storage.common.file.IFileMapManager;
@@ -74,10 +77,11 @@
     private final CleanerThread cleanerThread;
     private final Map<Integer, BufferedFileHandle> fileInfoMap;
     private final AsyncFIFOPageQueueManager fifoWriter;
-    private final Queue<BufferCacheHeaderHelper> headerPageCache = new ConcurrentLinkedQueue<>();
+    private final BlockingQueue<BufferCacheHeaderHelper> headerPageCache =
+            new ArrayBlockingQueue<>(IO_REQUEST_QUEUE_SIZE);
 
     //DEBUG
-    private Level fileOpsLevel = Level.DEBUG;
+    private static final Level fileOpsLevel = Level.TRACE;
     private ArrayList<CachedPage> confiscatedPages;
     private Lock confiscateLock;
     private HashMap<CachedPage, StackTraceElement[]> confiscatedPagesOwner;
@@ -190,7 +194,8 @@
                         tryRead(cPage);
                         cPage.valid = true;
                     } catch (Exception e) {
-                        LOGGER.log(Level.WARN, "Failure while trying to read a page from disk", e);
+                        LOGGER.log(ExceptionUtils.causedByInterrupt(e) ? Level.DEBUG : Level.WARN,
+                                "Failure while trying to read a page from disk", e);
                         throw e;
                     } finally {
                         if (!cPage.valid) {
@@ -588,7 +593,7 @@
             fInfo = fileInfoMap.get(fileId);
         }
         if (fInfo == null) {
-            throw new HyracksDataException("No such file mapped");
+            throw HyracksDataException.create(ErrorCode.FILE_DOES_NOT_EXIST, fileId);
         }
         return fInfo;
     }
@@ -609,31 +614,34 @@
         BufferedFileHandle fInfo = getFileInfo(cPage);
         // synchronize on fInfo to prevent the file handle from being deleted until the page is written.
         synchronized (fInfo) {
-            if (!fInfo.fileHasBeenDeleted()) {
-                ByteBuffer buf = cPage.buffer.duplicate();
-                final int totalPages = cPage.getFrameSizeMultiplier();
-                final int extraBlockPageId = cPage.getExtraBlockPageId();
-                final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
-                BufferCacheHeaderHelper header = checkoutHeaderHelper();
-                try {
-                    buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
-                    buf.position(0);
-                    long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
-                            getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)),
-                            header.prepareWrite(cPage, buf));
+            if (fInfo.fileHasBeenDeleted()) {
+                return;
+            }
+            ByteBuffer buf = cPage.buffer.duplicate();
+            final int totalPages = cPage.getFrameSizeMultiplier();
+            final int extraBlockPageId = cPage.getExtraBlockPageId();
+            final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
+            BufferCacheHeaderHelper header = checkoutHeaderHelper();
+            try {
+                buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
+                buf.position(0);
+                long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
+                        getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)), header.prepareWrite(cPage, buf));
 
-                    if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
-                            + getPageSizeWithHeader()) {
-                        throw new HyracksDataException("Failed to write completely: " + bytesWritten);
-                    }
-                } finally {
-                    returnHeaderHelper(header);
+                if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
+                        + getPageSizeWithHeader()) {
+                    throw new HyracksDataException("Failed to write completely: " + bytesWritten);
                 }
-                if (totalPages > 1 && !contiguousLargePages) {
-                    buf.limit(totalPages * pageSize);
-                    ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
-                }
-                assert buf.capacity() == (pageSize * totalPages);
+            } finally {
+                returnHeaderHelper(header);
+            }
+            if (totalPages > 1 && !contiguousLargePages) {
+                buf.limit(totalPages * pageSize);
+                ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
+            }
+            if (buf.capacity() != pageSize * totalPages) {
+                throw new IllegalStateException("Illegal number of bytes written, expected bytes written: "
+                        + pageSize * totalPages + " actual bytes writte: " + buf.capacity());
             }
         }
     }
@@ -786,11 +794,8 @@
         synchronized (fileInfoMap) {
             fileInfoMap.forEach((key, value) -> {
                 try {
-                    boolean fileHasBeenDeleted = value.fileHasBeenDeleted();
-                    sweepAndFlush(key, !fileHasBeenDeleted);
-                    if (!fileHasBeenDeleted) {
-                        ioManager.close(value.getFileHandle());
-                    }
+                    sweepAndFlush(key, true);
+                    ioManager.close(value.getFileHandle());
                 } catch (HyracksDataException e) {
                     if (LOGGER.isWarnEnabled()) {
                         LOGGER.log(Level.WARN, "Error flushing file id: " + key, e);
@@ -827,15 +832,15 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Opening file: " + fileRef + " in cache: " + this);
         }
-        int fileId;
+        int fileId = -1;
         synchronized (fileInfoMap) {
             if (fileMapManager.isMapped(fileRef)) {
                 fileId = fileMapManager.lookupFileId(fileRef);
             } else {
                 fileId = fileMapManager.registerFile(fileRef);
             }
-            openFile(fileId);
         }
+        openFile(fileId);
         return fileId;
     }
 
@@ -844,42 +849,62 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Opening file: " + fileId + " in cache: " + this);
         }
-        synchronized (fileInfoMap) {
-            BufferedFileHandle fInfo;
-            fInfo = fileInfoMap.get(fileId);
-            if (fInfo == null) {
-                boolean unreferencedFileFound = true;
-                while (fileInfoMap.size() >= maxOpenFiles && unreferencedFileFound) {
-                    // map is full, make room by cleaning up unreferenced files
-                    unreferencedFileFound = false;
-                    for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
-                        if (entry.getValue().getReferenceCount() <= 0) {
-                            int entryFileId = entry.getKey();
-                            boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
-                            sweepAndFlush(entryFileId, !fileHasBeenDeleted);
-                            if (!fileHasBeenDeleted) {
-                                ioManager.close(entry.getValue().getFileHandle());
-                            }
-                            fileInfoMap.remove(entryFileId);
-                            unreferencedFileFound = true;
-                            // for-each iterator is invalid because we changed
-                            // fileInfoMap
-                            break;
+        try {
+            final BufferedFileHandle fInfo = getOrCreateFileHandle(fileId);
+            if (fInfo.getFileHandle() == null) {
+                // a new file
+                synchronized (fInfo) {
+                    // prevent concurrent opening of the same file
+                    if (fInfo.getFileHandle() == null) {
+                        if (fileInfoMap.size() > maxOpenFiles) {
+                            closeOpeningFiles(fInfo);
                         }
+                        // create, open, and map new file reference
+                        FileReference fileRef;
+                        synchronized (fileInfoMap) {
+                            fileRef = fileMapManager.lookupFileName(fileId);
+                        }
+                        IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                        fInfo.setFileHandle(fh);
                     }
                 }
-                if (fileInfoMap.size() >= maxOpenFiles) {
-                    throw new HyracksDataException("Could not open fileId " + fileId + ". Max number of files "
-                            + maxOpenFiles + " already opened and referenced.");
-                }
-                // create, open, and map new file reference
-                FileReference fileRef = fileMapManager.lookupFileName(fileId);
-                IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                        IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-                fInfo = new BufferedFileHandle(fileId, fh);
-                fileInfoMap.put(fileId, fInfo);
             }
             fInfo.incReferenceCount();
+        } catch (Exception e) {
+            removeFileInfo(fileId);
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    private void closeOpeningFiles(BufferedFileHandle newFileHandle) throws HyracksDataException {
+        synchronized (fileInfoMap) {
+            boolean unreferencedFileFound = true;
+            while (fileInfoMap.size() > maxOpenFiles && unreferencedFileFound) {
+                // map is full, make room by cleaning up unreferenced files
+                unreferencedFileFound = false;
+                for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
+                    BufferedFileHandle fh = entry.getValue();
+                    if (fh != newFileHandle && fh.getReferenceCount() <= 0) {
+                        if (fh.getReferenceCount() < 0) {
+                            throw new IllegalStateException("Illegal reference count " + fh.getReferenceCount()
+                                    + " of file " + fh.getFileHandle().getFileReference());
+                        }
+                        int entryFileId = entry.getKey();
+                        sweepAndFlush(entryFileId, true);
+                        ioManager.close(entry.getValue().getFileHandle());
+                        fileInfoMap.remove(entryFileId);
+                        unreferencedFileFound = true;
+                        // for-each iterator is invalid because we changed
+                        // fileInfoMap
+                        break;
+                    }
+                }
+            }
+            if (fileInfoMap.size() > maxOpenFiles) {
+                throw new HyracksDataException("Could not open fileId " + newFileHandle.getFileId()
+                        + ". Max number of files " + maxOpenFiles + " already opened and referenced.");
+            }
         }
     }
 
@@ -942,8 +967,8 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Closing file: " + fileId + " in cache: " + this);
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug(dumpState());
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace(dumpState());
         }
 
         synchronized (fileInfoMap) {
@@ -977,15 +1002,19 @@
 
     @Override
     public void deleteFile(FileReference fileRef) throws HyracksDataException {
+        boolean mapped = false;
+        int fileId = -1;
         synchronized (fileInfoMap) {
             if (fileMapManager.isMapped(fileRef)) {
-                int fileId = fileMapManager.lookupFileId(fileRef);
-                deleteFile(fileId);
-                return;
-            } else {
-                IoUtil.delete(fileRef);
+                mapped = true;
+                fileId = fileMapManager.lookupFileId(fileRef);
             }
         }
+        if (mapped) {
+            deleteFile(fileId);
+        } else {
+            IoUtil.delete(fileRef);
+        }
     }
 
     @Override
@@ -993,29 +1022,26 @@
         if (LOGGER.isEnabled(fileOpsLevel)) {
             LOGGER.log(fileOpsLevel, "Deleting file: " + fileId + " in cache: " + this);
         }
-        synchronized (fileInfoMap) {
-            sweepAndFlush(fileId, false);
-            BufferedFileHandle fInfo = null;
+        BufferedFileHandle fInfo = removeFileInfo(fileId);
+        if (fInfo == null) {
+            return;
+        }
+        sweepAndFlush(fileId, false);
+        try {
+            if (fInfo.getReferenceCount() > 0) {
+                throw new HyracksDataException("Deleting open file");
+            }
+        } finally {
+            FileReference fileRef = null;
             try {
-                fInfo = fileInfoMap.get(fileId);
-                if (fInfo != null && fInfo.getReferenceCount() > 0) {
-                    throw new HyracksDataException("Deleting open file");
+                synchronized (fileInfoMap) {
+                    fileRef = fileMapManager.unregisterFile(fileId);
                 }
-            } catch (Exception e) {
-                throw HyracksDataException.create(e);
             } finally {
-                FileReference fileRef = fileMapManager.unregisterFile(fileId);
                 try {
-                    if (fInfo != null) {
-                        // Mark the fInfo as deleted,
-                        // such that when its pages are reclaimed in openFile(),
-                        // the pages are not flushed to disk but only invalidated.
-                        synchronized (fInfo) {
-                            if (!fInfo.fileHasBeenDeleted()) {
-                                ioManager.close(fInfo.getFileHandle());
-                                fInfo.markAsDeleted();
-                            }
-                        }
+                    synchronized (fInfo) {
+                        ioManager.close(fInfo.getFileHandle());
+                        fInfo.markAsDeleted();
                     }
                 } finally {
                     IoUtil.delete(fileRef);
@@ -1264,6 +1290,18 @@
         return null;
     }
 
+    private BufferedFileHandle getOrCreateFileHandle(int fileId) {
+        synchronized (fileInfoMap) {
+            return fileInfoMap.computeIfAbsent(fileId, id -> new BufferedFileHandle(fileId, null));
+        }
+    }
+
+    private BufferedFileHandle removeFileInfo(int fileId) {
+        synchronized (fileInfoMap) {
+            return fileInfoMap.remove(fileId);
+        }
+    }
+
     private ICachedPage getPageLoop(long dpid, int multiplier, boolean confiscate) throws HyracksDataException {
         final long startingPinCount = DEBUG ? masterPinCount.get() : -1;
         int cycleCount = 0;
@@ -1404,14 +1442,14 @@
 
     @Override
     public void purgeHandle(int fileId) throws HyracksDataException {
-        synchronized (fileInfoMap) {
-            BufferedFileHandle fh = fileInfoMap.get(fileId);
-            if (fh != null) {
-                ioManager.close(fh.getFileHandle());
-                fileInfoMap.remove(fileId);
+        BufferedFileHandle fh = removeFileInfo(fileId);
+        if (fh != null) {
+            synchronized (fileInfoMap) {
                 fileMapManager.unregisterFile(fileId);
             }
+            ioManager.close(fh.getFileHandle());
         }
+
     }
 
     static class BufferCacheHeaderHelper {
@@ -1451,4 +1489,23 @@
             return multiplier;
         }
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        synchronized (fileInfoMap) {
+            if (fileMapManager.isMapped(fileRef)) {
+                int fileId;
+                try {
+                    fileId = fileMapManager.lookupFileId(fileRef);
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+                BufferedFileHandle fInfo = fileInfoMap.get(fileId);
+                if (fInfo != null && fInfo.getReferenceCount() > 0) {
+                    fInfo.decReferenceCount();
+                }
+            }
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
index 02eb8bf..6ec12aa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/CachedPage.java
@@ -23,10 +23,14 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 /**
  * @author yingyib
  */
 public class CachedPage implements ICachedPageInternal {
+    private static final Logger LOGGER = LogManager.getLogger();
     final int cpid;
     ByteBuffer buffer;
     public final AtomicInteger pinCount;
@@ -44,6 +48,7 @@
     // DEBUG
     private static final boolean DEBUG = false;
     private final StackTraceElement[] ctorStack;
+    private IPageWriteFailureCallback failureCallback;
 
     //Constructor for making dummy entry for FIFO queue
     public CachedPage() {
@@ -85,6 +90,7 @@
         confiscated.set(false);
         pageReplacementStrategy.notifyCachePageReset(this);
         queueInfo = null;
+        failureCallback = null;
     }
 
     public void invalidate() {
@@ -103,11 +109,7 @@
 
     @Override
     public boolean isGoodVictim() {
-        if (confiscated.get()) {
-            return false; // i am not a good victim because i cant flush!
-        } else {
-            return pinCount.get() == 0;
-        }
+        return !confiscated.get() && pinCount.get() == 0;
     }
 
     @Override
@@ -205,4 +207,21 @@
     public boolean isLargePage() {
         return multiplier > 1;
     }
+
+    @Override
+    public void setFailureCallback(IPageWriteFailureCallback failureCallback) {
+        if (this.failureCallback != null) {
+            throw new IllegalStateException("failureCallback is already set");
+        }
+        this.failureCallback = failureCallback;
+    }
+
+    @Override
+    public void writeFailed(Exception e) {
+        if (failureCallback != null) {
+            failureCallback.writeFailed(this, e);
+        } else {
+            LOGGER.error("An IO Failure took place but the failure callback is not set", e);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 1cbe404..9e92f21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -246,4 +246,10 @@
         deleteFileCount.incrementAndGet();
         bufferCache.deleteFile(file);
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        bufferCache.closeFileIfOpen(fileRef);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
index 9d0b728..3d3ce3c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/FIFOLocalWriter.java
@@ -15,30 +15,37 @@
 
 package org.apache.hyracks.storage.common.buffercache;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class FIFOLocalWriter implements IFIFOPageWriter {
+    private static final Logger LOGGER = LogManager.getLogger();
     public static final FIFOLocalWriter INSTANCE = new FIFOLocalWriter();
-    private static boolean DEBUG = false;
+    private static final boolean DEBUG = false;
 
     private FIFOLocalWriter() {
     }
 
+    @SuppressWarnings("squid:S1181") // System must halt on all IO errors
     @Override
-    public void write(ICachedPage page, BufferCache bufferCache) throws HyracksDataException {
+    public void write(ICachedPage page, BufferCache bufferCache) {
         CachedPage cPage = (CachedPage) page;
         try {
             bufferCache.write(cPage);
+        } catch (Exception e) {
+            page.writeFailed(e);
+            LOGGER.warn("Failed to write page {}", cPage, e);
+        } catch (Throwable th) {
+            // Halt
+            LOGGER.error("FIFOLocalWriter has encountered a fatal error", th);
+            ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
         } finally {
             bufferCache.returnPage(cPage);
             if (DEBUG) {
-                System.out.println("[FIFO] Return page: " + cPage.cpid + "," + cPage.dpid);
+                LOGGER.error("[FIFO] Return page: {}, {}", cPage.cpid, cPage.dpid);
             }
         }
     }
 
-    @Override
-    public void sync(int fileId, BufferCache bufferCache) throws HyracksDataException {
-        bufferCache.force(fileId, true);
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/HaltOnFailureCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/HaltOnFailureCallback.java
new file mode 100644
index 0000000..0b748e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/HaltOnFailureCallback.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hyracks.storage.common.buffercache;
+
+import org.apache.hyracks.util.ExitUtil;
+
+public class HaltOnFailureCallback implements IPageWriteFailureCallback {
+    public static final HaltOnFailureCallback INSTANCE = new HaltOnFailureCallback();
+
+    private HaltOnFailureCallback() {
+    }
+
+    @Override
+    public void writeFailed(ICachedPage page, Throwable failure) {
+        ExitUtil.halt(ExitUtil.EC_ABNORMAL_TERMINATION);
+    }
+
+    @Override
+    public boolean hasFailed() {
+        return false;
+    }
+
+    @Override
+    public Throwable getFailure() {
+        return null;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 8dccc4a..21d3677 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -269,4 +269,13 @@
      */
     void resizePage(ICachedPage page, int multiplier, IExtraPageBlockHelper extraPageBlockHelper)
             throws HyracksDataException;
+
+    /**
+     * Close the file if open.
+     *
+     * @param fileRef
+     * @throws HyracksDataException
+     */
+    void closeFileIfOpen(FileReference fileRef);
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
index 16837b9..cfbb145 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPage.java
@@ -44,10 +44,14 @@
 
     void setDiskPageId(long dpid);
 
+    void setFailureCallback(IPageWriteFailureCallback callback);
+
     /**
      * Check if a page is a large page
      *
      * @return true if the page is large, false otherwise
      */
     boolean isLargePage();
+
+    void writeFailed(Exception e);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
index c500286..d900852 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ICachedPageInternal.java
@@ -19,13 +19,16 @@
 package org.apache.hyracks.storage.common.buffercache;
 
 public interface ICachedPageInternal extends ICachedPage {
-    public int getCachedPageId();
+    int getCachedPageId();
 
-    public long getDiskPageId();
+    long getDiskPageId();
 
-    public Object getReplacementStrategyObject();
+    Object getReplacementStrategyObject();
 
-    public boolean isGoodVictim();
+    /**
+     * @return true if can be evicted, false otherwise
+     */
+    boolean isGoodVictim();
 
     void setFrameSizeMultiplier(int multiplier);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java
index 6c03671..189c402 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageQueue.java
@@ -19,5 +19,20 @@
 
 @FunctionalInterface
 public interface IFIFOPageQueue {
-    void put(ICachedPage page) throws HyracksDataException;
+
+    /**
+     * Put a page in the write queue
+     *
+     * @param page
+     *            the page to be written
+     * @param callback
+     *            callback in case of a failure
+     * @throws HyracksDataException
+     *             if the callback has already failed. This indicates a failure writing a previous page
+     *             in the same operation.
+     *             Note: having this failure at this place removes the need to check for failures with
+     *             every add() call in the bulk loader and so, we check per page given to disk rather
+     *             than per tuple given to loader. At the same time, it allows the bulk load to fail early.
+     */
+    void put(ICachedPage page, IPageWriteFailureCallback callback) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
index 567c01e..26fd414 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IFIFOPageWriter.java
@@ -15,10 +15,7 @@
 
 package org.apache.hyracks.storage.common.buffercache;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
+@FunctionalInterface
 public interface IFIFOPageWriter {
-    public void write(ICachedPage page, BufferCache bufferCache) throws HyracksDataException;
-
-    void sync(int fileId, BufferCache bufferCache) throws HyracksDataException;
+    void write(ICachedPage page, BufferCache bufferCache);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteFailureCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteFailureCallback.java
new file mode 100644
index 0000000..da9cb6a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IPageWriteFailureCallback.java
@@ -0,0 +1,43 @@
+/*
+ * 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.hyracks.storage.common.buffercache;
+
+public interface IPageWriteFailureCallback {
+
+    /**
+     * Notify that an async write operation has failed
+     *
+     * @param page
+     * @param failure
+     */
+    void writeFailed(ICachedPage page, Throwable failure);
+
+    /**
+     * @return true if the callback has received any failure
+     */
+    boolean hasFailed();
+
+    /**
+     * @return a failure writing to disk or null if no failure has been seen
+     *         This doesn't guarantee which failure is returned but that if one or more failures occurred
+     *         while trying to write to disk, one of those failures is returned. All other failures are expected
+     *         to be logged.
+     */
+    Throwable getFailure();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/PageWriteFailureCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/PageWriteFailureCallback.java
new file mode 100644
index 0000000..c11e596
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/PageWriteFailureCallback.java
@@ -0,0 +1,42 @@
+/*
+ * 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.hyracks.storage.common.buffercache;
+
+public class PageWriteFailureCallback implements IPageWriteFailureCallback {
+
+    private volatile Throwable failure;
+
+    @Override
+    public final void writeFailed(ICachedPage page, Throwable failure) {
+        if (this.failure == null) {
+            this.failure = failure;
+        }
+    }
+
+    @Override
+    public final boolean hasFailed() {
+        return failure != null;
+    }
+
+    @Override
+    public final Throwable getFailure() {
+        return failure;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
index d7ec4e9..b2a1ff2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
@@ -147,4 +147,14 @@
         str.append("}");
         return str.toString();
     }
+
+    @Override
+    public void setFailureCallback(IPageWriteFailureCallback callback) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeFailed(Exception e) {
+        throw new UnsupportedOperationException();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
index 177128e..4f15588 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
@@ -24,7 +24,7 @@
 
 public class BufferedFileHandle {
     private final int fileId;
-    private IFileHandle handle;
+    private volatile IFileHandle handle;
     private final AtomicInteger refCount;
 
     public BufferedFileHandle(int fileId, IFileHandle handle) {
@@ -37,6 +37,10 @@
         return fileId;
     }
 
+    public void setFileHandle(IFileHandle fileHandle) {
+        this.handle = fileHandle;
+    }
+
     public IFileHandle getFileHandle() {
         return handle;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/FileMapManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/FileMapManager.java
index 26df884..0f2703b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/FileMapManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/FileMapManager.java
@@ -18,13 +18,16 @@
  */
 package org.apache.hyracks.storage.common.file;
 
+import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
 
+@NotThreadSafe
 public class FileMapManager implements IFileMapManager {
     private static final long serialVersionUID = 1L;
 
@@ -67,15 +70,20 @@
             throw HyracksDataException.create(ErrorCode.NO_MAPPING_FOR_FILE_ID, fileId);
         }
         name2IdMap.remove(fileRef);
+        fileRef.unregister();
         return fileRef;
     }
 
     @Override
     public int registerFile(FileReference fileRef) throws HyracksDataException {
-        if (isMapped(fileRef)) {
-            throw HyracksDataException.create(ErrorCode.FILE_ALREADY_MAPPED, fileRef);
+        Integer existingKey = name2IdMap.get(fileRef);
+        if (existingKey != null) {
+            FileReference prevFile = id2nameMap.get(existingKey);
+            throw HyracksDataException.create(ErrorCode.FILE_ALREADY_MAPPED, fileRef, prevFile,
+                    new Date(prevFile.registrationTime()).toString());
         }
         int fileId = idCounter++;
+        fileRef.register();
         id2nameMap.put(fileId, fileRef);
         name2IdMap.put(fileRef, fileId);
         return fileId;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
deleted file mode 100644
index d303cca..0000000
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.hyracks.storage.am.common;
-
-import java.util.Collection;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.common.IIndexAccessor;
-
-@SuppressWarnings("rawtypes")
-public interface ITreeIndexTestContext<T extends CheckTuple> {
-    public int getFieldCount();
-
-    public int getKeyFieldCount();
-
-    public ISerializerDeserializer[] getFieldSerdes();
-
-    public IBinaryComparatorFactory[] getComparatorFactories();
-
-    public IIndexAccessor getIndexAccessor();
-
-    public ITreeIndex getIndex();
-
-    public ArrayTupleReference getTuple();
-
-    public ArrayTupleBuilder getTupleBuilder();
-
-    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples);
-
-    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples);
-
-    public Collection<T> getCheckTuples();
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TestOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TestOperationCallback.java
index 3a5c8fe..18781f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TestOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TestOperationCallback.java
@@ -23,7 +23,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 
 public enum TestOperationCallback implements ISearchOperationCallback, IExtendedModificationOperationCallback {
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/datagen/ProbabilityHelper.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/datagen/ProbabilityHelper.java
index c7fbd3f..ccfa00b 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/datagen/ProbabilityHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/datagen/ProbabilityHelper.java
@@ -34,10 +34,10 @@
         double[] probDist = new double[numChoices];
         double divisor = 0;
         for (int i = 1; i <= numChoices; i++) {
-            divisor += 1.0 / (double) Math.pow((double) i, (double) zipfSkew);
+            divisor += 1.0 / Math.pow((double) i, (double) zipfSkew);
         }
         for (int i = 1; i <= numChoices; i++) {
-            probDist[i - 1] = (1.0 / (double) Math.pow((double) i, (double) zipfSkew)) / divisor;
+            probDist[i - 1] = (1.0 / Math.pow((double) i, (double) zipfSkew)) / divisor;
         }
         return probDist;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
index 67b24a2..2d6f97c 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
@@ -30,10 +30,16 @@
 public abstract class AbstractRTreeBulkLoadTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
+    private final boolean isPoint;
 
     public AbstractRTreeBulkLoadTest(boolean testRstarPolicy) {
+        this(testRstarPolicy, false);
+    }
+
+    public AbstractRTreeBulkLoadTest(boolean testRstarPolicy, boolean isPoint) {
         super(testRstarPolicy);
         this.rTreeTestUtils = new RTreeTestUtils();
+        this.isPoint = isPoint;
     }
 
     @Override
@@ -47,9 +53,9 @@
         // We assume all fieldSerdes are of the same type. Check the first
         // one to determine which field types to generate.
         if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
-            rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+            rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom(), isPoint);
         } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
-            rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
+            rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom(), isPoint);
         }
 
         rTreeTestUtils.checkScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 6505675..e73b771 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -62,7 +62,7 @@
         LSMRTREE,
         LSMRTREE_WITH_ANTIMATTER,
         RTREE
-    };
+    }
 
     protected static final Logger LOGGER = LogManager.getLogger();
     protected final Random rnd = new Random(50);
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
index 165bf43..92f5055 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -93,8 +93,13 @@
         }
     }
 
+    public void insertDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws HyracksDataException {
+        insertDoubleTuples(ctx, numTuples, rnd, false);
+    }
+
     @SuppressWarnings("unchecked")
-    public void insertDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd, boolean isPoint)
+            throws HyracksDataException {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -104,7 +109,7 @@
         double maxValue = Math.ceil(Math.pow(numTuples, 1.0 / numKeyFields));
         for (int i = 0; i < numTuples; i++) {
             // Set keys.
-            setDoubleKeyFields(fieldValues, numKeyFields, maxValue, rnd);
+            setDoubleKeyFields(fieldValues, numKeyFields, maxValue, rnd, isPoint);
             // Set values.
             setDoublePayloadFields(fieldValues, numKeyFields, fieldCount);
             TupleUtils.createDoubleTuple(ctx.getTupleBuilder(), ctx.getTuple(), fieldValues);
@@ -126,15 +131,20 @@
         }
     }
 
-    private void setDoubleKeyFields(double[] fieldValues, int numKeyFields, double maxValue, Random rnd) {
+    private void setDoubleKeyFields(double[] fieldValues, int numKeyFields, double maxValue, Random rnd,
+            boolean isPoint) {
         int maxFieldPos = numKeyFields / 2;
         for (int j = 0; j < maxFieldPos; j++) {
             int k = maxFieldPos + j;
             double firstValue = rnd.nextDouble() % maxValue;
             double secondValue;
-            do {
-                secondValue = rnd.nextDouble() % maxValue;
-            } while (secondValue < firstValue);
+            if (isPoint) {
+                secondValue = firstValue;
+            } else {
+                do {
+                    secondValue = rnd.nextDouble() % maxValue;
+                } while (secondValue < firstValue);
+            }
             fieldValues[j] = firstValue;
             fieldValues[k] = secondValue;
         }
@@ -155,8 +165,13 @@
         return checkTuple;
     }
 
-    @SuppressWarnings("unchecked")
     public void bulkLoadDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+        bulkLoadDoubleTuples(ctx, numTuples, rnd, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void bulkLoadDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd, boolean isPoint)
+            throws HyracksDataException {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -164,7 +179,7 @@
         Collection<CheckTuple> tmpCheckTuples = createCheckTuplesCollection();
         for (int i = 0; i < numTuples; i++) {
             // Set keys.
-            setDoubleKeyFields(fieldValues, numKeyFields, maxValue, rnd);
+            setDoubleKeyFields(fieldValues, numKeyFields, maxValue, rnd, isPoint);
             // Set values.
             setDoublePayloadFields(fieldValues, numKeyFields, fieldCount);
 
@@ -222,22 +237,56 @@
         return checkTuple;
     }
 
-    @Override
-    protected void setIntKeyFields(int[] fieldValues, int numKeyFields, int maxValue, Random rnd) {
+    @SuppressWarnings("unchecked")
+    public void bulkLoadIntTuples(IIndexTestContext ctx, int numTuples, Random rnd, boolean isPoint)
+            throws HyracksDataException {
+        int fieldCount = ctx.getFieldCount();
+        int numKeyFields = ctx.getKeyFieldCount();
+        int[] fieldValues = new int[ctx.getFieldCount()];
+        int maxValue = (int) Math.ceil(Math.pow(numTuples, 1.0 / numKeyFields));
+        Collection<CheckTuple> tmpCheckTuples = createCheckTuplesCollection();
+        for (int i = 0; i < numTuples; i++) {
+            // Set keys.
+            setIntKeyFields(fieldValues, numKeyFields, maxValue, rnd, isPoint);
+            // Set values.
+            setIntPayloadFields(fieldValues, numKeyFields, fieldCount);
+
+            // Set expected values. (We also use these as the pre-sorted stream
+            // for ordered indexes bulk loading).
+            ctx.insertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), tmpCheckTuples);
+        }
+        bulkLoadCheckTuples(ctx, tmpCheckTuples, false);
+
+        // Add tmpCheckTuples to ctx check tuples for comparing searches.
+        for (CheckTuple checkTuple : tmpCheckTuples) {
+            ctx.insertCheckTuple(checkTuple, ctx.getCheckTuples());
+        }
+    }
+
+    protected void setIntKeyFields(int[] fieldValues, int numKeyFields, int maxValue, Random rnd, boolean isPoint) {
         int maxFieldPos = numKeyFields / 2;
         for (int j = 0; j < maxFieldPos; j++) {
             int k = maxFieldPos + j;
             int firstValue = rnd.nextInt() % maxValue;
             int secondValue;
-            do {
-                secondValue = rnd.nextInt() % maxValue;
-            } while (secondValue < firstValue);
+            if (isPoint) {
+                secondValue = firstValue;
+            } else {
+                do {
+                    secondValue = rnd.nextInt() % maxValue;
+                } while (secondValue < firstValue);
+            }
             fieldValues[j] = firstValue;
             fieldValues[k] = secondValue;
         }
     }
 
     @Override
+    protected void setIntKeyFields(int[] fieldValues, int numKeyFields, int maxValue, Random rnd) {
+        setIntKeyFields(fieldValues, numKeyFields, maxValue, rnd, false);
+    }
+
+    @Override
     protected void setIntPayloadFields(int[] fieldValues, int numKeyFields, int numFields) {
         for (int j = numKeyFields; j < numFields; j++) {
             fieldValues[j] = intPayloadValue++;
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
index 159df9e..9e52785 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
@@ -38,8 +38,13 @@
             final String targetDir = getTargetDir();
             final String[] list = new File(targetDir).list((dir, name) -> name.matches(pattern));
             final String topLevelPattern = getTopLevelDirPattern();
-            final String[] topLevel =
-                    new File(FileUtil.joinPath(targetDir, list[0])).list((dir, name) -> name.matches(topLevelPattern));
+            String[] topLevel;
+            if (topLevelPattern == null) {
+                topLevel = new String[] { "" };
+            } else {
+                topLevel = new File(FileUtil.joinPath(targetDir, list[0]))
+                        .list((dir, name) -> name.matches(topLevelPattern));
+            }
             installerDir = FileUtil.joinPath(targetDir, list[0], topLevel[0]);
             Assert.assertNotNull("installerDir", list);
             Assert.assertFalse("Ambiguous install dir (" + pattern + "): " + Arrays.toString(topLevel),
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
index b022b15..2809343 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
@@ -45,7 +45,7 @@
         this.frameSize = frameSize;
         this.serviceContext = serviceContext;
         this.jobId = jobId;
-        fileFactory = new WorkspaceFileFactory(this, (IIOManager) getIOManager());
+        fileFactory = new WorkspaceFileFactory(this, getIOManager());
         this.frameManger = new FrameManager(frameSize);
         this.jobStartTime = System.currentTimeMillis();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
index 673a27f..7b7e850 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.control.nc.io.DefaultDeviceResolver;
 import org.apache.hyracks.control.nc.io.IOManager;
@@ -88,11 +89,8 @@
 
     public synchronized static IBufferCache getBufferCache(INCServiceContext ctx) {
         if (bufferCache == null) {
-            ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-            IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, pageSize, numPages);
-            IFileMapProvider fileMapProvider = getFileMapProvider();
-            bufferCache = new BufferCache(ctx.getIoManager(), prs, new DelayPageCleanerPolicy(1000),
-                    (IFileMapManager) fileMapProvider, maxOpenFiles, threadFactory);
+            IIOManager ioManager = ctx.getIoManager();
+            return getBufferCache(ioManager);
         }
         return bufferCache;
     }
@@ -145,4 +143,16 @@
         }
         return resourceIdFactory;
     }
+
+    public static IBufferCache getBufferCache(IIOManager ioManager) {
+        if (bufferCache != null) {
+            return bufferCache;
+        }
+        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
+        IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, pageSize, numPages);
+        IFileMapProvider fileMapProvider = getFileMapProvider();
+        bufferCache = new BufferCache(ioManager, prs, new DelayPageCleanerPolicy(1000),
+                (IFileMapManager) fileMapProvider, maxOpenFiles, threadFactory);
+        return bufferCache;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index ac52573..174d5cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.dataflow.state.IStateObject;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
@@ -39,6 +38,7 @@
 import org.apache.hyracks.api.job.profiling.IStatsCollector;
 import org.apache.hyracks.api.job.profiling.counters.ICounterContext;
 import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.control.common.job.profiling.StatsCollector;
 import org.apache.hyracks.control.nc.io.WorkspaceFileFactory;
 
@@ -134,7 +134,7 @@
     }
 
     @Override
-    public IDatasetPartitionManager getDatasetPartitionManager() {
+    public IResultPartitionManager getResultPartitionManager() {
         return null;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
index b7b4639e..4834a03 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/org/apache/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
@@ -277,7 +277,7 @@
                 k1 = MurmurHash128Bit.rotl64(k1, 31);
                 k1 *= c2;
                 h1 ^= k1;
-        };
+        }
 
         //----------
         // finalization
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
index c77bea0..75437c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
@@ -160,7 +160,7 @@
                 }
             }
         } finally {
-            pointCursor.destroy();;
+            pointCursor.destroy();
         }
         return true;
     }
@@ -183,6 +183,9 @@
             bulkloader.add(tuple);
         }
         bulkloader.end();
+        if (bulkloader.hasFailed()) {
+            throw HyracksDataException.create(bulkloader.getFailure());
+        }
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index 46a8347..7803d2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -144,5 +144,9 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
new file mode 100644
index 0000000..cb791c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.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.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class EncapsulatingIoCallback implements ILSMIOOperationCallback {
+
+    private final ILSMIOOperationCallback encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallback(ILSMIOOperationCallback inner, ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        this.encapsulated = inner;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        scheduledCallback.before(operation);
+        encapsulated.scheduled(operation);
+        scheduledCallback.after(operation);
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperationCallback.before(operation);
+        encapsulated.beforeOperation(operation);
+        beforeOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperationCallback.before(operation);
+        encapsulated.afterOperation(operation);
+        afterOperationCallback.after(operation);
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalizeCallback.before(operation);
+        encapsulated.afterFinalize(operation);
+        afterFinalizeCallback.after(operation);
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        try {
+            completedCallback.before(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+        encapsulated.completed(operation);
+        try {
+            completedCallback.after(operation);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.recycled(component);
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        encapsulated.allocated(component);
+    }
+
+    public ILSMIOOperationCallback getEncapsulated() {
+        return encapsulated;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
new file mode 100644
index 0000000..bb10236
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
@@ -0,0 +1,71 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class EncapsulatingIoCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ILSMIOOperationCallbackFactory encapsulated;
+    private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+    private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+    private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+    private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+    public EncapsulatingIoCallbackFactory(ILSMIOOperationCallbackFactory factory,
+            ITestOpCallback<ILSMIOOperation> scheduledCallback,
+            ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+            ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+            ITestOpCallback<ILSMIOOperation> completedCallback) {
+        encapsulated = factory;
+        this.scheduledCallback = scheduledCallback;
+        this.beforeOperationCallback = beforeOperationCallback;
+        this.afterOperationCallback = afterOperationCallback;
+        this.afterFinalizeCallback = afterFinalizeCallback;
+        this.completedCallback = completedCallback;
+    }
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        encapsulated.initialize(ncCtx, resource);
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        ILSMIOOperationCallback inner = encapsulated.createIoOpCallback(index);
+        return new EncapsulatingIoCallback(inner, scheduledCallback, beforeOperationCallback, afterOperationCallback,
+                afterFinalizeCallback, completedCallback);
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return encapsulated.getCurrentMemoryComponentIndex();
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
new file mode 100644
index 0000000..251f57a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -0,0 +1,470 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.NoOpTestCallback;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LSMBTreeComponentLifecycleTest {
+    private static final Logger LOGGER = LogManager.getLogger();
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private final OrderedIndexTestUtils testUtils = new OrderedIndexTestUtils();
+    private final ISerializerDeserializer[] fieldSerdes =
+            { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final int numKeys = 1;
+    private static final int numTuplesToInsert = 100;
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            ILSMIOOperationScheduler scheduler, ILSMIOOperationCallbackFactory ioCallbackFactory) throws Exception {
+        return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
+                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+                scheduler, ioCallbackFactory, harness.getMetadataPageManagerFactory(), false, true, false);
+    }
+
+    private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys)
+            throws Exception {
+        return createTestContext(fieldSerdes, numKeys, harness.getIOScheduler(),
+                harness.getIOOperationCallbackFactory());
+    }
+
+    @Test
+    public void testFlushUnallocatedIndex() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnallocatedIndexStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnallocatedIndex();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testNormalFlushOperation() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, after were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert into the index
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testNormalFlushOperationStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testNormalFlushOperation();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    @Test
+    public void testFlushUnModifiedComponent() throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(2, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // insert more
+        testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+        // flush
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(3, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Test
+    public void testFlushUnModifiedComponentStartFromSecondComponent() throws Exception {
+        CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+        try {
+            testFlushUnModifiedComponent();
+        } finally {
+            CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+        }
+    }
+
+    public int getExpectedMemoryComponentIndex(int expectedIndex) {
+        return (CountingIoOperationCallbackFactory.STARTING_INDEX + expectedIndex) % 2;
+    }
+
+    @Test
+    public void testScheduleMoreFlushesThanComponents() throws Exception {
+        final AtomicInteger counter = new AtomicInteger();
+        final Semaphore flushSemaphore = new Semaphore(0);
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, new AsynchronousScheduler(
+                r -> new Thread(r, "LsmIoThread-" + counter.getAndIncrement()), new IIoOperationFailedCallback() {
+                    @Override
+                    public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Scheduler failed", failure);
+                    }
+
+                    @Override
+                    public void operationFailed(ILSMIOOperation operation, Throwable failure) {
+                        LOGGER.log(Level.ERROR, "Operation {} failed", operation, failure);
+                    }
+                }), new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
+                        NoOpTestCallback.get(), new ITestOpCallback<ILSMIOOperation>() {
+                            @Override
+                            public void before(ILSMIOOperation t) throws HyracksDataException {
+                                try {
+                                    flushSemaphore.acquire();
+                                } catch (InterruptedException e) {
+                                    throw new IllegalStateException(e);
+                                }
+                            }
+
+                            @Override
+                            public void after(ILSMIOOperation t) throws HyracksDataException {
+                            }
+                        }, NoOpTestCallback.get(), NoOpTestCallback.get()));
+        ILSMIndex index = (ILSMIndex) ctx.getIndex();
+        index.create();
+        index.activate();
+        Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+        int numMemoryComponents = index.getNumberOfAllMemoryComponents();
+        // create a flusher that will schedule 13 flushes.
+        // wait for all flushes to be scheduled.
+        // create an inserter that will insert some records.
+        // one by one allow flushes until one flush remains, and ensure no record went in.
+        // allow the last flush, then wait for the inserts to succeed, and ensure they went to
+        // the expected memory component
+        final int numFlushes = 13;
+        User firstUser = new User("FirstUser");
+        User secondUser = new User("SecondUser");
+        Request flushRequest = new Request(Request.Statement.FLUSH, ctx, numFlushes);
+        firstUser.add(flushRequest);
+        firstUser.step();
+        // wait until all flushes have been scheduled.. Not yet performed
+        flushRequest.await(1);
+        // create an inserter and allow it to go all the way
+        Request insertRequest = new Request(Request.Statement.INSERT, ctx, 1);
+        secondUser.add(insertRequest);
+        secondUser.step();
+        secondUser.step();
+        ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        ILSMIndexOperationContext opCtx = accessor.getOpContext();
+        assertCorrectSearchComponents(opCtx, index, 0);
+        // Allow one flush at a time and ensure that inserter didn't succeed
+        for (int i = 0; i < numFlushes - index.getNumberOfAllMemoryComponents(); i++) {
+            flushSemaphore.release();
+            firstUser.step();
+            flushRequest.await(2 + i);
+            Assert.assertEquals(0, insertRequest.getSteps());
+            // also ensure that you get the correct components when searching
+            assertCorrectSearchComponents(opCtx, index, i + 1);
+        }
+        flushSemaphore.release();
+        firstUser.step();
+        // wait for the insert to complete
+        insertRequest.await();
+        // Allow last flush to proceed
+        flushSemaphore.release();
+        firstUser.step();
+        firstUser.step();
+        flushRequest.await();
+        firstUser.stop();
+        secondUser.stop();
+        int expectedMemoryComponent = numFlushes % numMemoryComponents;
+        Assert.assertEquals(getExpectedMemoryComponentIndex(expectedMemoryComponent),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(0, index.getDiskComponents().size());
+        EncapsulatingIoCallback encapsulating = (EncapsulatingIoCallback) index.getIOOperationCallback();
+        CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) encapsulating.getEncapsulated();
+        // assert equal before, after, finalize were called
+        Assert.assertEquals(numFlushes, ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // flush, there was no insert before
+        flushSemaphore.release();
+        flush(ctx);
+        Assert.assertEquals(getExpectedMemoryComponentIndex((expectedMemoryComponent + 1) % numMemoryComponents),
+                index.getCurrentMemoryComponentIndex());
+        Assert.assertEquals(1, index.getDiskComponents().size());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+        Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+        // deactivate will cause a flush
+        flushSemaphore.release();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    private void assertCorrectSearchComponents(ILSMIndexOperationContext opCtx, ILSMIndex index,
+            int numSuccesfullyCompletedFlushes) throws HyracksDataException {
+        opCtx.reset();
+        opCtx.setOperation(IndexOperation.SEARCH);
+        index.getOperationalComponents(opCtx);
+        List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
+        int first = numSuccesfullyCompletedFlushes % memComponents.size();
+        Assert.assertEquals(memComponents.get(first), getFirstMemoryComponent(opCtx));
+    }
+
+    private ILSMComponent getFirstMemoryComponent(ILSMIndexOperationContext opCtx) {
+        List<ILSMComponent> components = opCtx.getComponentHolder();
+        // backward
+        for (int i = components.size() - 1; i >= 0; i--) {
+            ILSMComponent next = components.get(i);
+            if (next.getType() == LSMComponentType.MEMORY) {
+                return next;
+            }
+        }
+        return null;
+    }
+
+    private void flush(OrderedIndexTestContext ctx) throws HyracksDataException, InterruptedException {
+        ILSMIOOperation flush = scheduleFlush(ctx);
+        flush.sync();
+        if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+            throw HyracksDataException.create(flush.getFailure());
+        }
+    }
+
+    private ILSMIOOperation scheduleFlush(OrderedIndexTestContext ctx)
+            throws HyracksDataException, InterruptedException {
+        ILSMIndexAccessor accessor =
+                (ILSMIndexAccessor) ctx.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        return accessor.scheduleFlush();
+    }
+
+    private static class Request {
+        private enum Statement {
+            FLUSH,
+            INSERT
+        }
+
+        private final Statement statement;
+        private final OrderedIndexTestContext ctx;
+        private final int repeats;
+        private boolean done = false;
+        private int step = 0;
+
+        public Request(Statement statement, OrderedIndexTestContext ctx, int repeats) {
+            this.statement = statement;
+            this.ctx = ctx;
+            this.repeats = repeats;
+        }
+
+        Statement statement() {
+            return statement;
+        }
+
+        synchronized void complete() {
+            done = true;
+            notifyAll();
+        }
+
+        synchronized void await() throws InterruptedException {
+            while (!done) {
+                wait();
+            }
+        }
+
+        synchronized void step() {
+            step++;
+            notifyAll();
+        }
+
+        synchronized int getSteps() {
+            return step;
+        }
+
+        synchronized void await(int step) throws InterruptedException {
+            while (this.step < step) {
+                wait();
+            }
+        }
+    }
+
+    private class User extends SingleThreadEventProcessor<Request> {
+
+        private Semaphore step = new Semaphore(0);
+
+        public User(String username) {
+            super(username);
+        }
+
+        public void step() {
+            step.release();
+        }
+
+        @Override
+        protected void handle(Request req) throws Exception {
+            try {
+                step.acquire();
+                switch (req.statement()) {
+                    case FLUSH:
+                        List<ILSMIOOperation> flushes = new ArrayList<>(req.repeats);
+                        for (int i = 0; i < req.repeats; i++) {
+                            flushes.add(scheduleFlush(req.ctx));
+                        }
+                        req.step();
+                        for (ILSMIOOperation op : flushes) {
+                            step.acquire();
+                            op.sync();
+                            if (op.getStatus() == LSMIOOperationStatus.FAILURE) {
+                                throw HyracksDataException.create(op.getFailure());
+                            }
+                            req.step(); // report after completion of each flush
+                        }
+                        break;
+                    case INSERT:
+                        testUtils.insertIntTuples(req.ctx, numTuplesToInsert, harness.getRandom());
+                        break;
+                    default:
+                        break;
+                }
+                req.step();
+                step.acquire();
+            } finally {
+                req.step();
+                req.complete();
+            }
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
index 9d768aa..03b221e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
@@ -29,7 +29,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.junit.After;
 import org.junit.Assert;
@@ -71,8 +70,8 @@
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
         accessor.insert(tuple);
 
-        // Flush to generate a disk component
-        accessor.scheduleFlush(((ILSMIndex) ctx.getIndex()).getIOOperationCallback());
+        // Flush to generate a disk component. This uses synchronous scheduler
+        accessor.scheduleFlush();
 
         // Make sure the disk component was generated
         LSMBTree btree = (LSMBTree) ctx.getIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
index 815c8bc..1b9a7a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
@@ -40,9 +40,9 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.hyracks.storage.am.lsm.common.impls.StubIOOperationCallback;
 import org.junit.Assert;
 
 /**
@@ -113,13 +113,14 @@
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getRight(), minMax.getRight(), comp));
                 }
 
-                StubIOOperationCallback stub = new StubIOOperationCallback();
-                BlockingIOOperationCallbackWrapper waiter = new BlockingIOOperationCallbackWrapper(stub);
-                accessor.scheduleFlush(waiter);
-                waiter.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 if (minMax != null) {
                     Pair<ITupleReference, ITupleReference> obsMinMax =
-                            filterToMinMax(stub.getLastNewComponent().getLSMComponentFilter());
+                            filterToMinMax(flush.getNewComponent().getLSMComponentFilter());
                     Assert.assertEquals(0,
                             TreeIndexTestUtils.compareFilterTuples(obsMinMax.getLeft(), minMax.getLeft(), comp));
                     Assert.assertEquals(0,
@@ -143,8 +144,7 @@
                     expectedMergeMinMax.setRight(componentMinMax.getRight());
                 }
             }
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             flushedComponents = ((LSMBTree) ctx.getIndex()).getDiskComponents();
             Pair<ITupleReference, ITupleReference> mergedMinMax =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index 475ab9c..a8639c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -76,32 +77,24 @@
         ITupleReference tuple1 = TupleUtils.createIntegerTuple(1, 1, 1);
         accessor.insert(tuple1);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple2 = TupleUtils.createIntegerTuple(2, 2, 2);
         accessor.insert(tuple2);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
+        accessor.scheduleFlush();
 
         ITupleReference tuple3 = TupleUtils.createIntegerTuple(3, 3, 3);
         accessor.insert(tuple3);
         // flush component
-        accessor.scheduleFlush(btree.getIOOperationCallback());
-
+        accessor.scheduleFlush();
         scheduler.modify = true;
-
-        boolean exceptionThrown = false;
-        try {
-            accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
-        } catch (HyracksDataException e) {
-            exceptionThrown = true;
-        }
-        Assert.assertTrue(exceptionThrown);
-
+        ILSMIOOperation merge = accessor.scheduleMerge(btree.getDiskComponents());
+        merge.sync();
+        Assert.assertEquals(LSMIOOperationStatus.FAILURE, merge.getStatus());
         scheduler.modify = false;
-        accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
+        accessor.scheduleMerge(btree.getDiskComponents());
         Assert.assertEquals(1, btree.getDiskComponents().size());
-
         btree.deactivate();
         btree.destroy();
     }
@@ -120,14 +113,18 @@
 
         @Override
         public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-            if (modify) {
-                try {
-                    modifyOperation(operation);
-                } catch (Exception e) {
-                    throw HyracksDataException.create(e);
+            try {
+                if (modify) {
+                    try {
+                        modifyOperation(operation);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
                 }
+                operation.call();
+            } finally {
+                operation.complete();
             }
-            operation.call();
         }
 
         private void modifyOperation(ILSMIOOperation operation) throws Exception {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 87ddf33..7c59671 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -74,8 +74,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((LSMBTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
 
             orderedIndexTestUtils.checkPointSearches(ctx);
             orderedIndexTestUtils.checkScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 9f17efa..4fafb38 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.btree;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
@@ -26,9 +27,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.util.trace.ITracer;
 import org.junit.Test;
@@ -72,8 +73,6 @@
         IndexAccessParameters actx = new IndexAccessParameters(cb, NoOpOperationCallback.INSTANCE);
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(actx);
 
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             isFoundNull = true;
             for (int i = 0; i < NUM_TUPLES; i++) {
@@ -82,8 +81,11 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = accessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -95,8 +97,7 @@
             }
 
             if (j == 1) {
-                accessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                accessor.scheduleFlush().sync();
                 isFoundNull = true;
             } else {
                 isFoundNull = false;
@@ -106,9 +107,7 @@
                 TupleUtils.createIntegerTuple(builder, tuple, i);
                 accessor.delete(tuple);
             }
-
-            accessor.scheduleFlush(ioOpCallback);
-            ioOpCallback.waitForIO();
+            accessor.scheduleFlush().sync();
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index 48812b4..2574c4d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -44,8 +44,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -113,21 +111,20 @@
     protected void test(OrderedIndexTestContext ctx, ISerializerDeserializer[] fieldSerdes)
             throws HyracksDataException {
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         //component 2 contains 1 and 2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 1 contains 1 and -2
         upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
         deleteTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         //component 0 contains 2 and 3
         upsertTuple(ctx, fieldSerdes, getValue(3, fieldSerdes));
         upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
 
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 3, btree.getDiskComponents().size());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index 3a48160..afcb3c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -52,8 +52,6 @@
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.junit.After;
@@ -352,11 +350,10 @@
             throws HyracksDataException {
 
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
         op1.performOperation(ctx, AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT);
         op2.performOperation(ctx,
                 AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT / AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS);
-        accessor.scheduleFlush(ioCallback);
+        accessor.scheduleFlush();
         LSMBTree btree = (LSMBTree) ctx.getIndex();
         Assert.assertEquals("Check disk components", 1, btree.getDiskComponents().size());
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index fc852cd..0914541 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -35,9 +35,9 @@
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
@@ -99,8 +99,6 @@
 
     private void test(IndexModification op1, IndexModification op2) throws Exception {
         ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) accessor;
-        BlockingIOOperationCallbackWrapper ioOpCallback =
-                new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
         for (int j = 0; j < 2; j++) {
             index.clear();
             isFoundNull = true;
@@ -111,8 +109,11 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                ILSMIOOperation flush = lsmAccessor.scheduleFlush();
+                flush.sync();
+                if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+                    throw HyracksDataException.create(flush.getFailure());
+                }
                 isFoundNull = true;
                 isUpdated = false;
             } else {
@@ -126,8 +127,7 @@
             }
 
             if (j == 1) {
-                lsmAccessor.scheduleFlush(ioOpCallback);
-                ioOpCallback.waitForIO();
+                lsmAccessor.scheduleFlush().sync();
             } else {
                 isFoundNull = false;
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
index 19a9872..2e43198 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
@@ -35,7 +35,7 @@
     }
 
     @Override
-    public void after() throws HyracksDataException {
+    public void after(Semaphore t) throws HyracksDataException {
 
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
new file mode 100644
index 0000000..84b530a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
@@ -0,0 +1,101 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class CountingIoOperationCallback implements ILSMIOOperationCallback {
+    private int beforeOperation;
+    private int afterOperation;
+    private int afterFinalize;
+    private int recycled;
+    private int allocated;
+    private int beforeSchedule;
+    private int destroy;
+
+    public CountingIoOperationCallback() {
+    }
+
+    public int getAfterFinalizeCount() {
+        return afterFinalize;
+    }
+
+    @Override
+    public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+        recycled++;
+    }
+
+    public int getRecycledCount() {
+        return recycled;
+    }
+
+    @Override
+    public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+        allocated++;
+    }
+
+    public int getAllocatedCount() {
+        return allocated;
+    }
+
+    @Override
+    public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+        beforeSchedule++;
+    }
+
+    public int getBeforeScheduleCount() {
+        return beforeSchedule;
+    }
+
+    @Override
+    public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        beforeOperation++;
+    }
+
+    public int getBeforeOperationCount() {
+        return beforeOperation;
+    }
+
+    @Override
+    public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+        afterOperation++;
+    }
+
+    public int getAfterOperationCount() {
+        return afterOperation;
+    }
+
+    @Override
+    public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+        afterFinalize++;
+    }
+
+    @Override
+    public void completed(ILSMIOOperation operation) {
+        destroy++;
+    }
+
+    public int getDestroyCount() {
+        return destroy;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
new file mode 100644
index 0000000..b64b9b1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class CountingIoOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static int STARTING_INDEX = 0;
+
+    @Override
+    public void initialize(INCServiceContext ncCtx, IResource resource) {
+        // No op
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+        return new CountingIoOperationCallback();
+    }
+
+    @Override
+    public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+        return STARTING_INDEX;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
index e888238..fa3097d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
@@ -23,5 +23,5 @@
 public interface ITestOpCallback<T> {
     void before(T t) throws HyracksDataException;
 
-    void after() throws HyracksDataException;
+    void after(T t) throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
new file mode 100644
index 0000000..2370663
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
@@ -0,0 +1,45 @@
+/*
+ * 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.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class NoOpTestCallback implements ITestOpCallback {
+    public static final NoOpTestCallback INSTANCE = new NoOpTestCallback();
+
+    private NoOpTestCallback() {
+    }
+
+    @SuppressWarnings("unchecked")
+    public static final <T> ITestOpCallback<T> get() {
+        return INSTANCE;
+    }
+
+    @Override
+    public void before(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+    @Override
+    public void after(Object t) throws HyracksDataException {
+        // NoOp
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
index 1d4b7d6..4867c71 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -65,9 +64,11 @@
 
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioAllocateCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<ILSMMemoryComponent>> ioRecycleCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpScheduledCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioBeforeCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterOpCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> ioAfterFinalizeCallbacks = new ArrayList<>();
+    private final List<ITestOpCallback<Void>> ioOpCompletedCallbacks = new ArrayList<>();
     private final List<ITestOpCallback<Void>> allocateComponentCallbacks = new ArrayList<>();
 
     private volatile int numScheduledFlushes;
@@ -110,7 +111,7 @@
         super.modify(ictx, tuple);
         synchronized (modifyCallbacks) {
             for (ITestOpCallback<Semaphore> callback : modifyCallbacks) {
-                callback.after();
+                callback.after(modifySemaphore);
             }
         }
     }
@@ -122,17 +123,15 @@
     }
 
     @Override
-    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleFlush(ctx, callback);
+    public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledFlushes++;
+        return super.createFlushOperation(ctx);
     }
 
     @Override
-    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
-            throws HyracksDataException {
-        super.scheduleMerge(ctx, callback);
+    public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
         numScheduledMerges++;
+        return super.createMergeOperation(ctx);
     }
 
     @Override
@@ -148,7 +147,7 @@
         numFinishedFlushes++;
         synchronized (flushCallbacks) {
             for (ITestOpCallback<Semaphore> callback : flushCallbacks) {
-                callback.after();
+                callback.after(flushSemaphore);
             }
         }
         return c;
@@ -167,7 +166,7 @@
         numFinishedMerges++;
         synchronized (mergeCallbacks) {
             for (ITestOpCallback<Semaphore> callback : mergeCallbacks) {
-                callback.after();
+                callback.after(mergeSemaphore);
             }
         }
         return c;
@@ -347,6 +346,30 @@
         }
     }
 
+    public void addIoScheduledCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoScheduledCallbacks() {
+        synchronized (ioOpScheduledCallbacks) {
+            ioOpScheduledCallbacks.clear();
+        }
+    }
+
+    public void addIoCompletedCallback(ITestOpCallback<Void> callback) {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.add(callback);
+        }
+    }
+
+    public void clearIoCompletedCallbacks() {
+        synchronized (ioOpCompletedCallbacks) {
+            ioOpCompletedCallbacks.clear();
+        }
+    }
+
     @Override
     public void allocateMemoryComponents() throws HyracksDataException {
         synchronized (allocateComponentCallbacks) {
@@ -357,7 +380,7 @@
         super.allocateMemoryComponents();
         synchronized (allocateComponentCallbacks) {
             for (ITestOpCallback<Void> callback : allocateComponentCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -373,7 +396,7 @@
     public void beforeIoOperationReturned() throws HyracksDataException {
         synchronized (ioBeforeCallbacks) {
             for (ITestOpCallback<Void> callback : ioBeforeCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -389,7 +412,7 @@
     public void afterIoOperationReturned() throws HyracksDataException {
         synchronized (ioAfterOpCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterOpCallbacks) {
-                callback.after();
+                callback.after(null);
             }
         }
     }
@@ -405,7 +428,39 @@
     public void afterIoFinalizeReturned() throws HyracksDataException {
         synchronized (ioAfterFinalizeCallbacks) {
             for (ITestOpCallback<Void> callback : ioAfterFinalizeCallbacks) {
-                callback.after();
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioScheduledCalled() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioScheduledReturned() throws HyracksDataException {
+        synchronized (ioOpScheduledCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+                callback.after(null);
+            }
+        }
+    }
+
+    public void ioCompletedCalled() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.before(null);
+            }
+        }
+    }
+
+    public void ioCompletedReturned() throws HyracksDataException {
+        synchronized (ioOpCompletedCallbacks) {
+            for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+                callback.after(null);
             }
         }
     }
@@ -421,7 +476,7 @@
     public void recycledReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioRecycleCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioRecycleCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
@@ -437,7 +492,7 @@
     public void allocatedReturned(ILSMMemoryComponent component) throws HyracksDataException {
         synchronized (ioAllocateCallbacks) {
             for (ITestOpCallback<ILSMMemoryComponent> callback : ioAllocateCallbacks) {
-                callback.after();
+                callback.after(component);
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 3ce24b0..b25a229 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -27,6 +27,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResource;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -38,6 +40,9 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
 import org.apache.hyracks.storage.common.IStorageManager;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class TestLsmBtreeLocalResource extends LSMBTreeLocalResource {
     private static final long serialVersionUID = 1L;
 
@@ -55,6 +60,11 @@
                 vbcProvider, ioSchedulerProvider, durable);
     }
 
+    protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields) throws HyracksDataException {
+        super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields);
+    }
+
     @Override
     public ILSMIndex createInstance(INCServiceContext serviceCtx) throws HyracksDataException {
         IIOManager ioManager = serviceCtx.getIoManager();
@@ -75,4 +85,21 @@
                 ioOpCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 durable, metadataPageManagerFactory, false, serviceCtx.getTracer());
     }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
+        appendToJson(jsonObject, registry);
+        return jsonObject;
+    }
+
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+            throws HyracksDataException {
+        final int[] bloomFilterKeyFields = OBJECT_MAPPER.convertValue(json.get("bloomFilterKeyFields"), int[].class);
+        final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
+        final boolean isPrimary = json.get("isPrimary").asBoolean();
+        final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+        return new TestLsmBtreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
+                isPrimary, btreeFields);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index c7e064f..2e7a9f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -212,4 +212,10 @@
     public IFileMapManager getFileMapProvider() {
         return vbc.getFileMapProvider();
     }
+
+    @Override
+    public void closeFileIfOpen(FileReference fileRef) {
+        throw new UnsupportedOperationException();
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 4c07dea..583fd5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -116,7 +116,7 @@
                 consumeCursorTuples(searchCursor);
                 break;
             case MERGE:
-                accessor.scheduleMerge(lsmBTree.getIOOperationCallback(), lsmBTree.getDiskComponents());
+                accessor.scheduleMerge(lsmBTree.getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
index 62aeb3e..4501485 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
@@ -72,12 +72,12 @@
         this.typeTraits = typeTraits;
         tupleCmp = new TupleComparator(cmp);
         tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        tupleWriter = (TypeAwareTupleWriter) tupleWriterFactory.createTupleWriter();
+        tupleWriter = tupleWriterFactory.createTupleWriter();
         int numTuples = numBatches * batchSize;
         tuples = new ArrayList<TypeAwareTupleReference>();
         tupleBuf = ByteBuffer.allocate(numTuples * tupleSize);
         for (int i = 0; i < numTuples; i++) {
-            tuples.add((TypeAwareTupleReference) tupleWriter.createTupleReference());
+            tuples.add(tupleWriter.createTupleReference());
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index a8e0aee..2462c85 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -36,7 +36,9 @@
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
@@ -48,10 +50,14 @@
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
 import org.apache.hyracks.test.support.TestUtils;
+import org.apache.hyracks.util.ExitUtil;
 import org.apache.hyracks.util.trace.ITracer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class LSMTreeRunner implements IExperimentRunner {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final int MAX_OPEN_FILES = Integer.MAX_VALUE;
     private static final int HYRACKS_FRAME_SIZE = 131072;
 
@@ -103,8 +109,17 @@
             virtualBufferCaches.add(virtualBufferCache);
         }
 
-        this.ioScheduler = AsynchronousScheduler.INSTANCE;
-        AsynchronousScheduler.INSTANCE.init(threadFactory);
+        this.ioScheduler = new AsynchronousScheduler(threadFactory, new IIoOperationFailedCallback() {
+            @Override
+            public void operationFailed(ILSMIOOperation operation, Throwable t) {
+                LOGGER.error("Operation {} failed", operation, t);
+            }
+
+            @Override
+            public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+                ExitUtil.exit(ExitUtil.EC_IO_SCHEDULER_FAILED);
+            }
+        });
 
         lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
index c0d9bb8..8510229 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
@@ -73,8 +73,7 @@
                 ITupleReference maxTuple = TupleUtils.createTuple(maxFieldSerdes, (Object[]) maxFields);
                 ByteBuffer maxMatterBuf = writeTuple(maxTuple, maxMatterTupleWriter);
                 // Tuple reference should work for both matter and antimatter tuples (doesn't matter which factory creates it).
-                LSMBTreeTupleReference maxLsmBTreeTuple =
-                        (LSMBTreeTupleReference) maxMatterTupleWriter.createTupleReference();
+                LSMBTreeTupleReference maxLsmBTreeTuple = maxMatterTupleWriter.createTupleReference();
 
                 ISerializerDeserializer[] fieldSerdes = Arrays.copyOfRange(maxFieldSerdes, 0, numFields);
                 ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
@@ -99,8 +98,7 @@
                 }
 
                 // Tuple reference should work for both matter and antimatter tuples (doesn't matter which factory creates it).
-                LSMBTreeTupleReference lsmBTreeTuple =
-                        (LSMBTreeTupleReference) matterTupleWriter.createTupleReference();
+                LSMBTreeTupleReference lsmBTreeTuple = matterTupleWriter.createTupleReference();
 
                 // Use LSMBTree tuple reference to interpret the written tuples.
                 // Repeat the block inside to test that repeated resetting to matter/antimatter tuples works.
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 803c5cb..707628b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -36,14 +36,14 @@
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -95,12 +95,12 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
         this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
-        this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+        this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
     }
 
     public void setUp() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index 56a1fb3..8694824 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -91,7 +91,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>2.0.2-beta</version>
       <scope>test</scope>
     </dependency>
   </dependencies>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/component/TestLsmIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/component/TestLsmIndexFileManager.java
index 87fb11a..c255ee5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/component/TestLsmIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/component/TestLsmIndexFileManager.java
@@ -22,27 +22,38 @@
 import java.io.FilenameFilter;
 import java.util.ArrayList;
 
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
 import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
 public class TestLsmIndexFileManager extends AbstractLSMIndexFileManager {
 
-    public TestLsmIndexFileManager(IIOManager ioManager, FileReference file) {
-        super(ioManager, file, null);
+    private long componentSeq = 0;
+
+    public TestLsmIndexFileManager(IIOManager ioManager, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeIndexFactory) {
+        super(ioManager, file, treeIndexFactory);
     }
 
     @Override
     protected void cleanupAndGetValidFilesInternal(FilenameFilter filter,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
-            throws HyracksDataException {
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<IndexComponentFileReference> allFiles,
+            IBufferCache bufferCache) {
         String[] files = baseDir.getFile().list(filter);
         for (String fileName : files) {
             FileReference fileRef = baseDir.getChild(fileName);
-            allFiles.add(new ComparableFileName(fileRef));
+            allFiles.add(IndexComponentFileReference.of(fileRef));
         }
     }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        String sequence = IndexComponentFileReference.getFlushSequence(componentSeq++);
+        return new LSMComponentFileReferences(baseDir.getChild(sequence), null, null);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMIndexFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMIndexFileManagerTest.java
index 8c1124f..22456e8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMIndexFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMIndexFileManagerTest.java
@@ -38,13 +38,17 @@
 import org.apache.hyracks.api.util.IoUtil;
 import org.apache.hyracks.control.nc.io.DefaultDeviceResolver;
 import org.apache.hyracks.control.nc.io.IOManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.component.TestLsmIndexFileManager;
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class LSMIndexFileManagerTest {
     private static final int DEFAULT_PAGE_SIZE = 256;
@@ -54,6 +58,7 @@
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected final static String sep = System.getProperty("file.separator");
     protected IOManager ioManager;
+    protected IBufferCache bufferCache;
     protected String baseDir;
     protected FileReference file;
 
@@ -61,6 +66,7 @@
     public void setUp() throws HyracksDataException {
         TestStorageManagerComponentHolder.init(DEFAULT_PAGE_SIZE, DEFAULT_NUM_PAGES, DEFAULT_MAX_OPEN_FILES);
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        bufferCache = TestStorageManagerComponentHolder.getBufferCache(ioManager);
         baseDir = ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getMount() + sep + "lsm_tree"
                 + simpleDateFormat.format(new Date()) + sep;
         File f = new File(baseDir);
@@ -75,7 +81,9 @@
     }
 
     public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
-        ILSMIndexFileManager fileManager = new TestLsmIndexFileManager(ioManager, file);
+        TreeIndexFactory<? extends ITreeIndex> treeIndexFactory = Mockito.mock(TreeIndexFactory.class);
+        Mockito.when(treeIndexFactory.getBufferCache()).thenReturn(bufferCache);
+        ILSMIndexFileManager fileManager = new TestLsmIndexFileManager(ioManager, file, treeIndexFactory);
         LinkedList<String> fileNames = new LinkedList<>();
 
         int numFileNames = 100;
@@ -122,7 +130,9 @@
             IoUtil.delete(f);
         }
         FileReference file = ioManager.resolveAbsolutePath(f.getAbsolutePath());
-        ILSMIndexFileManager fileManager = new TestLsmIndexFileManager(ioManager, file);
+        TreeIndexFactory<? extends ITreeIndex> treeIndexFactory = Mockito.mock(TreeIndexFactory.class);
+        Mockito.when(treeIndexFactory.getBufferCache()).thenReturn(bufferCache);
+        ILSMIndexFileManager fileManager = new TestLsmIndexFileManager(ioManager, file, treeIndexFactory);
         fileManager.createDirs();
 
         List<FileReference> flushFiles = new ArrayList<>();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 2fd1dbc..54fd3bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -30,7 +30,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -197,7 +196,7 @@
         Mockito.doAnswer(new Answer<Void>() {
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
                 if (mergedSizes != null) {
                     mergedComponents.forEach(component -> {
                         mergedSizes.add(component.getComponentSize());
@@ -219,8 +218,7 @@
                 }
                 return null;
             }
-        }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
-                Mockito.anyListOf(ILSMDiskComponent.class));
+        }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
 
         Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 4657109..1a882b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -56,8 +56,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index 584de0c..f5a7c9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -58,8 +58,7 @@
                 invIndex.activate();
             }
             // Perform merge.
-            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
-                    ((LSMInvertedIndex) invIndex).getDiskComponents());
+            invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
             validateAndCheckIndex(testCtx);
             runTinySearchWorkload(testCtx, tupleGen);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
index a420ba9..da87b27 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
@@ -84,6 +85,12 @@
             LSMInvertedIndexTestUtils.compareActualAndExpectedIndexes(testCtx);
         }
         LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesRangeSearch(testCtx);
+        if (invIndexType == InvertedIndexType.LSM || invIndexType == InvertedIndexType.PARTITIONED_LSM) {
+            LSMInvertedIndex lsmIndex = (LSMInvertedIndex) invIndex;
+            if (!lsmIndex.isMemoryComponentsAllocated() || lsmIndex.isCurrentMutableComponentEmpty()) {
+                LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesMergeSearch(testCtx);
+            }
+        }
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 8e41c86..c1e1b4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -43,7 +43,7 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -92,7 +92,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 78b1658..2fcd96f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -70,8 +70,8 @@
         IHyracksTaskContext ctx = new HyracksTaskTestContext();
         // Intermediate and final search result will use this buffer manager to get frames.
         IDeallocatableFramePool framePool = new DeallocatableFramePool(ctx,
-                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());;
-        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);;
+                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());
+        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);
         // Keep the buffer manager in the hyracks context so that the search process can get it via the context.
         TaskUtil.put(HyracksConstants.INVERTED_INDEX_SEARCH_FRAME_MANAGER, bufferManagerForSearch, ctx);
         IIndexAccessParameters iap =
@@ -138,7 +138,7 @@
             }
 
             case MERGE: {
-                accessor.scheduleMerge(invIndex.getIOOperationCallback(), invIndex.getDiskComponents());
+                accessor.scheduleMerge(invIndex.getDiskComponents());
                 break;
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index ae8713c..164efc2 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -55,6 +55,9 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils.HyracksTaskTestContext;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.util.trace.ITraceCategoryRegistry;
+import org.apache.hyracks.util.trace.TraceCategoryRegistry;
+import org.apache.hyracks.util.trace.Tracer;
 
 @SuppressWarnings("rawtypes")
 public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
@@ -66,7 +69,7 @@
         PARTITIONED_INMEMORY,
         PARTITIONED_ONDISK,
         PARTITIONED_LSM
-    };
+    }
 
     protected IInvertedIndex invIndex;
     protected IBinaryComparatorFactory[] allCmpFactories;
@@ -88,8 +91,8 @@
         IHyracksTaskContext ctx = new HyracksTaskTestContext();
         // Intermediate and final search result will use this buffer manager to get frames.
         IDeallocatableFramePool framePool = new DeallocatableFramePool(ctx,
-                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());;
-        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);;
+                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());
+        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);
         // Keep the buffer manager in the hyracks context so that the search process can get it via the context.
         TaskUtil.put(HyracksConstants.INVERTED_INDEX_SEARCH_FRAME_MANAGER, bufferManagerForSearch, ctx);
         IIndexAccessParameters iap =
@@ -184,7 +187,9 @@
                         harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
                         filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
                 break;
             }
             case PARTITIONED_LSM: {
@@ -195,7 +200,9 @@
                         harness.getOperationTracker(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
                         filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
-                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+                        invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
                 break;
             }
             default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index db3bb50..82014ea 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -44,7 +44,6 @@
 import org.apache.hyracks.api.dataflow.state.IStateObject;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataset.IDatasetPartitionManager;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -55,6 +54,7 @@
 import org.apache.hyracks.api.job.profiling.IStatsCollector;
 import org.apache.hyracks.api.job.profiling.counters.ICounterContext;
 import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.util.GrowableArray;
@@ -87,6 +87,8 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexMergeCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
@@ -276,17 +278,33 @@
         }
     }
 
+    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        IInvertedIndexAccessor invIndexAccessor =
+                (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        compareActualAndExpectedIndexesRangeSearch(testCtx, invIndexAccessor.createRangeSearchCursor());
+    }
+
+    public static void compareActualAndExpectedIndexesMergeSearch(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        LSMInvertedIndexAccessor invIndexAccessor =
+                (LSMInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        compareActualAndExpectedIndexesRangeSearch(testCtx,
+                new LSMInvertedIndexMergeCursor(invIndexAccessor.getOpContext()));
+    }
+
     /**
      * Compares actual and expected indexes using the rangeSearch() method of the inverted-index accessor.
      */
-    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
-            throws HyracksDataException {
+    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx,
+            IIndexCursor invIndexCursor) throws HyracksDataException {
         IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
         int tokenFieldCount = invIndex.getTokenTypeTraits().length;
         int invListFieldCount = invIndex.getInvListTypeTraits().length;
         IInvertedIndexAccessor invIndexAccessor =
                 (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        IIndexCursor invIndexCursor = invIndexAccessor.createRangeSearchCursor();
         try {
             MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
             IBinaryComparatorFactory[] tupleCmpFactories =
@@ -526,8 +544,8 @@
         IHyracksTaskContext ctx = new HyracksTaskTestContext();
         // Intermediate and final search result will use this buffer manager to get frames.
         IDeallocatableFramePool framePool = new DeallocatableFramePool(ctx,
-                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());;
-        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);;
+                AccessMethodTestsConfig.LSM_INVINDEX_SEARCH_FRAME_LIMIT * ctx.getInitialFrameSize());
+        ISimpleFrameBufferManager bufferManagerForSearch = new FramePoolBackedFrameBufferManager(framePool);
         // Keep the buffer manager in the hyracks context so that the search process can get it via the context.
         TaskUtil.put(HyracksConstants.INVERTED_INDEX_SEARCH_FRAME_MANAGER, bufferManagerForSearch, ctx);
         IIndexAccessParameters iap =
@@ -708,7 +726,7 @@
         }
 
         @Override
-        public IDatasetPartitionManager getDatasetPartitionManager() {
+        public IResultPartitionManager getResultPartitionManager() {
             return null;
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 0e0f7e5..ee044bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -76,8 +76,7 @@
             }
 
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getIOOperationCallback(),
-                    ((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
+            accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
 
             rTreeTestUtils.checkScan(ctx);
             rTreeTestUtils.checkDiskOrderScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
new file mode 100644
index 0000000..0df7caf
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreePointMBRBulkLoadTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.hyracks.storage.am.lsm.rtree;
+
+import java.util.Random;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
+import org.apache.hyracks.storage.am.rtree.AbstractRTreeBulkLoadTest;
+import org.apache.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import org.junit.After;
+import org.junit.Before;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreePointMBRBulkLoadTest extends AbstractRTreeBulkLoadTest {
+
+    public LSMRTreePointMBRBulkLoadTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY, true);
+    }
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception {
+        return LSMRTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
+                harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, valueProviderFactories, numKeys,
+                rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
+                harness.getMetadataPageManagerFactory(), true);
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 581dbf7..1fa9d95 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -25,7 +25,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
@@ -75,9 +74,8 @@
     }
 
     protected void flush(AbstractRTreeTestContext ctx) throws HyracksDataException {
-        ILSMIndex lsmIndex = (ILSMIndex) ctx.getIndex();
         ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
-        accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+        accessor.scheduleFlush();
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 2a14780..810c3c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -78,7 +78,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(lsmRTree.getIOOperationCallback(), lsmRTree.getDiskComponents());
+                accessor.scheduleMerge(lsmRTree.getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index 653677c..595d675 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -66,8 +66,7 @@
                 break;
 
             case MERGE:
-                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getIOOperationCallback(),
-                        ((AbstractLSMRTree) lsmRTree).getDiskComponents());
+                accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getDiskComponents());
                 break;
 
             default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 2bd74af..acdb268 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -79,6 +79,17 @@
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory) throws Exception {
+        return create(ioManager, virtualBufferCaches, file, diskBufferCache, fieldSerdes, valueProviderFactories,
+                numKeyFields, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+                ioOpCallbackFactory, metadataPageManagerFactory, false);
+    }
+
+    public static LSMRTreeTestContext create(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+            FileReference file, IBufferCache diskBufferCache, ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
+            IMetadataPageManagerFactory metadataPageManagerFactory, boolean isPointMBR) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories =
                 SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -95,7 +106,7 @@
                 typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
-                null, null, true, false, metadataPageManagerFactory);
+                null, null, true, isPointMBR, metadataPageManagerFactory);
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 8a5d0c5..598cf18 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -42,7 +42,7 @@
 import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -89,7 +89,7 @@
         this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
         this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
         this.mergePolicy = new NoMergePolicy();
         this.opTracker = new ThreadCountingTracker();
         this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index 56c4b53..bc1a2c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -163,7 +163,7 @@
             int p2x = RND.nextInt();
             int p2y = RND.nextInt();
 
-            int pk = RND.nextInt();;
+            int pk = RND.nextInt();
 
             TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                     Math.max(p1y, p2y), pk);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
index f94914c..d0d02a9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/org/apache/hyracks/storage/common/BufferCacheTest.java
@@ -33,7 +33,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -41,6 +40,7 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -96,7 +96,7 @@
             long dpid = BufferedFileHandle.getDiskPageId(fileId, i);
             ICachedPage page = bufferCache.confiscatePage(dpid);
             page.getBuffer().putInt(0, i);
-            bufferCache.createFIFOQueue().put(page);
+            bufferCache.createFIFOQueue().put(page, HaltOnFailureCallback.INSTANCE);
         }
         bufferCache.finishQueue();
         bufferCache.closeFile(fileId);
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index 212673e..ff5f622 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -79,10 +79,22 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-core</artifactId>
     </dependency>
-      <dependency>
-          <groupId>commons-io</groupId>
-          <artifactId>commons-io</artifactId>
-      </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-text</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+    </dependency>
   </dependencies>
 
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java
new file mode 100644
index 0000000..147e871
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ComputingAction.java
@@ -0,0 +1,24 @@
+/*
+ * 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.hyracks.util;
+
+public interface ComputingAction<T> {
+    @SuppressWarnings("squid:S00112")
+    T compute() throws Throwable;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java
new file mode 100644
index 0000000..d2023e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/CountRetryPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.hyracks.util;
+
+public class CountRetryPolicy implements IRetryPolicy {
+
+    private final int count;
+    private int attempted = 0;
+
+    public CountRetryPolicy(int count) {
+        this.count = count;
+    }
+
+    @Override
+    public boolean retry(Throwable failure) {
+        if (attempted < count) {
+            attempted++;
+            return true;
+        }
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 1a17012..e2ae73a 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hyracks.util;
 
+import java.lang.reflect.Field;
+import java.util.IdentityHashMap;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.mutable.MutableLong;
@@ -33,22 +36,34 @@
     public static final int EC_ABNORMAL_TERMINATION = 1;
     public static final int EC_FAILED_TO_STARTUP = 2;
     public static final int EC_FAILED_TO_RECOVER = 3;
-    public static final int NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS = 4;
+    public static final int EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS = 4;
+    public static final int EC_FAILED_TO_PROCESS_UN_INTERRUPTIBLE_REQUEST = 5;
+    public static final int EC_FAILED_TO_COMMIT_METADATA_TXN = 6;
+    public static final int EC_FAILED_TO_ABORT_METADATA_TXN = 7;
+    public static final int EC_INCONSISTENT_METADATA = 8;
+    public static final int EC_UNCAUGHT_THROWABLE = 9;
     public static final int EC_UNHANDLED_EXCEPTION = 11;
+    public static final int EC_FAILED_TO_DELETE_CORRUPTED_RESOURCES = 12;
+    public static final int EC_ERROR_CREATING_RESOURCES = 13;
+    public static final int EC_TXN_LOG_FLUSHER_FAILURE = 14;
+    public static final int EC_NODE_REGISTRATION_FAILURE = 15;
+    public static final int EC_NETWORK_FAILURE = 16;
+    public static final int EC_ACTIVE_SUSPEND_FAILURE = 17;
+    public static final int EC_ACTIVE_RESUME_FAILURE = 18;
+    public static final int EC_FAILED_TO_CANCEL_ACTIVE_START_STOP = 22;
     public static final int EC_IMMEDIATE_HALT = 33;
     public static final int EC_HALT_ABNORMAL_RESERVED_44 = 44;
-    public static final int EC_HALT_ABNORMAL_RESERVED_55 = 55;
+    public static final int EC_IO_SCHEDULER_FAILED = 55;
     public static final int EC_HALT_SHUTDOWN_TIMED_OUT = 66;
     public static final int EC_HALT_WATCHDOG_FAILED = 77;
-    public static final int EC_HALT_ABNORMAL_RESERVED_88 = 88;
+    public static final int EC_FLUSH_FAILED = 88;
     public static final int EC_TERMINATE_NC_SERVICE_DIRECTIVE = 99;
-
     private static final ExitThread exitThread = new ExitThread();
     private static final ShutdownWatchdog watchdogThread = new ShutdownWatchdog();
     private static final MutableLong shutdownHaltDelay = new MutableLong(10 * 60 * 1000L); // 10 minutes default
 
     static {
-        Runtime.getRuntime().addShutdownHook(new Thread(watchdogThread::start));
+        watchdogThread.start();
     }
 
     private ExitUtil() {
@@ -64,7 +79,7 @@
                 LOGGER.warn("ignoring duplicate request to exit with status " + status
                         + "; already exiting with status " + exitThread.status + "...");
             } else {
-                exitThread.setStatus(status);
+                exitThread.setStatus(status, new Throwable("exit callstack"));
                 exitThread.start();
             }
         }
@@ -75,8 +90,8 @@
         exit(status);
     }
 
-    public static void halt(int status) {
-        LOGGER.fatal("JVM halting with status " + status + "; bye!", new Throwable("halt stacktrace"));
+    public static synchronized void halt(int status) {
+        LOGGER.fatal("JVM halting with status {}; thread dump at halt: {}", status, ThreadDumpUtil.takeDumpString());
         // try to give time for the log to be emitted...
         LogManager.shutdown();
         Runtime.getRuntime().halt(status);
@@ -84,6 +99,8 @@
 
     private static class ShutdownWatchdog extends Thread {
 
+        private final Semaphore startSemaphore = new Semaphore(0);
+
         private ShutdownWatchdog() {
             super("ShutdownWatchdog");
             setDaemon(true);
@@ -91,11 +108,14 @@
 
         @Override
         public void run() {
+            startSemaphore.acquireUninterruptibly();
+            LOGGER.info("starting shutdown watchdog- system will halt if shutdown is not completed within {} seconds",
+                    TimeUnit.MILLISECONDS.toSeconds(shutdownHaltDelay.getValue()));
             try {
-                exitThread.join(shutdownHaltDelay.getValue()); // 10 min
+                exitThread.join(shutdownHaltDelay.getValue());
                 if (exitThread.isAlive()) {
                     try {
-                        LOGGER.info("Watchdog is angry. Killing shutdown hook");
+                        LOGGER.fatal("shutdown did not complete within configured delay; halting");
                     } finally {
                         ExitUtil.halt(EC_HALT_SHUTDOWN_TIMED_OUT);
                     }
@@ -104,10 +124,15 @@
                 ExitUtil.halt(EC_HALT_WATCHDOG_FAILED);
             }
         }
+
+        public void beginWatch() {
+            startSemaphore.release();
+        }
     }
 
     private static class ExitThread extends Thread {
-        private int status;
+        private volatile int status;
+        private volatile Throwable callstack;
 
         ExitThread() {
             super("JVM exit thread");
@@ -116,15 +141,33 @@
 
         @Override
         public void run() {
+            watchdogThread.beginWatch();
             try {
-                LOGGER.info("JVM exiting with status " + status + "; bye!");
+                LOGGER.warn("JVM exiting with status " + status + "; bye!", callstack);
+                logShutdownHooks();
             } finally {
                 Runtime.getRuntime().exit(status);
             }
         }
 
-        public void setStatus(int status) {
+        public void setStatus(int status, Throwable callstack) {
             this.status = status;
+            this.callstack = callstack;
+        }
+
+        private static void logShutdownHooks() {
+            try {
+                Class clazz = Class.forName("java.lang.ApplicationShutdownHooks");
+                Field hooksField = clazz.getDeclaredField("hooks");
+                hooksField.setAccessible(true);
+                IdentityHashMap hooks = (IdentityHashMap) hooksField.get(null);
+                if (hooks != null) {
+                    LOGGER.info("the following ({}) shutdown hooks have been registered: {}", hooks::size,
+                            hooks::toString);
+                }
+            } catch (Exception e) {
+                LOGGER.debug("ignoring exception trying to log shutdown hooks", e);
+            }
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java
new file mode 100644
index 0000000..b266f01
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IDelay.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hyracks.util;
+
+@FunctionalInterface
+public interface IDelay {
+
+    long calculate(long attempt);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
new file mode 100644
index 0000000..29469d5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hyracks.util;
+
+@FunctionalInterface
+public interface IRetryPolicy {
+    /**
+     * @param failure
+     *            the cause of the failure
+     * @return true if one more attempt should be done
+     */
+    boolean retry(Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
index f9f45c1..c6b76fc 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
@@ -18,10 +18,19 @@
  */
 package org.apache.hyracks.util;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.StandardSocketOptions;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.http.HttpHost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.conn.util.InetAddressUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -37,13 +46,65 @@
         sc.setOption(StandardSocketOptions.SO_KEEPALIVE, true);
     }
 
-    public static void closeQuietly(SocketChannel sc) {
-        if (sc.isOpen()) {
+    public static void closeQuietly(Closeable closeable) {
+        if (closeable != null) {
             try {
-                sc.close();
+                closeable.close();
             } catch (IOException e) {
-                LOGGER.warn("Failed to close socket", e);
+                LOGGER.warn("Failed to close", e);
             }
         }
     }
+
+    public static URI toUri(HttpHost host) throws URISyntaxException {
+        return builderFrom(host).build();
+    }
+
+    public static URI toUri(HttpHost host, String path) throws URISyntaxException {
+        return builderFrom(host).setPath(path).build();
+    }
+
+    public static URIBuilder builderFrom(HttpHost host) {
+        return new URIBuilder().setHost(host.getHostName()).setPort(host.getPort()).setScheme(host.getSchemeName());
+    }
+
+    public static String toHostPort(String host, String port) {
+        return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+    }
+
+    public static String toHostPort(String host, int port) {
+        return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+    }
+
+    public static String toHostPort(InetSocketAddress address) {
+        return toHostPort(address.getHostString(), address.getPort());
+    }
+
+    public static InetSocketAddress parseInetSocketAddress(String hostPortString) {
+        int lastColon = hostPortString.lastIndexOf(':');
+        String host = decodeIPv6LiteralHost(lastColon < 0 ? hostPortString : hostPortString.substring(0, lastColon));
+        int port = lastColon < 0 ? 0 : Integer.parseInt(hostPortString.substring(lastColon + 1));
+        return new InetSocketAddress(host, port);
+    }
+
+    public static InetSocketAddress toInetSocketAddress(String maybeLiteralHost, int port) {
+        return new InetSocketAddress(decodeIPv6LiteralHost(maybeLiteralHost), port);
+    }
+
+    public static List<InetSocketAddress> parseInetSocketAddresses(String... hostPortStrings) {
+        List<InetSocketAddress> hosts = new ArrayList<>();
+        for (String node : hostPortStrings) {
+            hosts.add(parseInetSocketAddress(node));
+        }
+        return hosts;
+    }
+
+    public static String encodeIPv6LiteralHost(String hostname) {
+        return InetAddressUtils.isIPv6Address(hostname) ? "[" + hostname + "]" : hostname;
+    }
+
+    public static String decodeIPv6LiteralHost(String hostname) {
+        return hostname.length() > 0 && hostname.charAt(0) == '[' ? hostname.substring(1, hostname.length() - 1)
+                : hostname;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java
new file mode 100644
index 0000000..3040fb1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NoRetryPolicy.java
@@ -0,0 +1,32 @@
+/*
+ * 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.hyracks.util;
+
+public class NoRetryPolicy implements IRetryPolicy {
+    public static final NoRetryPolicy INSTANCE = new NoRetryPolicy();
+
+    private NoRetryPolicy() {
+    }
+
+    @Override
+    public boolean retry(Throwable failure) {
+        return false;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
index d8d6bb1..183cb6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
@@ -29,6 +29,10 @@
         spanNanos = unit.toNanos(span);
     }
 
+    public long getSpanNanos() {
+        return spanNanos;
+    }
+
     public static Span start(long span, TimeUnit unit) {
         return new Span(span, unit);
     }
@@ -41,14 +45,27 @@
         return unit.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS);
     }
 
+    /**
+     * Sleep for the minimum of the duration or the remaining of this span
+     *
+     * @param sleep
+     *            the amount to sleep
+     * @param unit
+     *            the unit of the amount
+     * @throws InterruptedException
+     */
     public void sleep(long sleep, TimeUnit unit) throws InterruptedException {
-        TimeUnit.NANOSECONDS.sleep(Math.min(elapsed(TimeUnit.NANOSECONDS), unit.toNanos(sleep)));
+        TimeUnit.NANOSECONDS.sleep(Math.min(remaining(TimeUnit.NANOSECONDS), unit.toNanos(sleep)));
     }
 
     public long remaining(TimeUnit unit) {
         return unit.convert(Long.max(spanNanos - elapsed(TimeUnit.NANOSECONDS), 0L), TimeUnit.NANOSECONDS);
     }
 
+    public void wait(Object monitor) throws InterruptedException {
+        TimeUnit.NANOSECONDS.timedWait(monitor, remaining(TimeUnit.NANOSECONDS));
+    }
+
     public void loopUntilExhausted(ThrowingAction action) throws Exception {
         loopUntilExhausted(action, 0, TimeUnit.NANOSECONDS);
     }
@@ -59,7 +76,41 @@
             if (elapsed(delayUnit) < delay) {
                 break;
             }
-            delayUnit.sleep(delay);
+            sleep(delay, delayUnit);
         }
     }
+
+    @Override
+    public String toString() {
+        long nanos = spanNanos % 1000;
+        long micros = TimeUnit.MICROSECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 1000;
+        long millis = TimeUnit.MILLISECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 1000;
+        long seconds = TimeUnit.SECONDS.convert(spanNanos, TimeUnit.NANOSECONDS) % 60;
+        long minutes = TimeUnit.MINUTES.convert(spanNanos, TimeUnit.NANOSECONDS) % 60;
+        long hours = TimeUnit.HOURS.convert(spanNanos, TimeUnit.NANOSECONDS) % 24;
+        long days = TimeUnit.DAYS.convert(spanNanos, TimeUnit.NANOSECONDS);
+        StringBuilder builder = new StringBuilder();
+        if (days > 0) {
+            builder.append(days).append("d");
+        }
+        if (hours > 0) {
+            builder.append(hours).append("hr");
+        }
+        if (minutes > 0) {
+            builder.append(minutes).append("m");
+        }
+        if (seconds > 0) {
+            builder.append(seconds).append("s");
+        }
+        if (millis > 0) {
+            builder.append(millis).append("ms");
+        }
+        if (micros > 0) {
+            builder.append(micros).append("us");
+        }
+        if (nanos > 0 || builder.length() == 0) {
+            builder.append(nanos).append("ns");
+        }
+        return builder.toString();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
index b457c79..11be0ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
@@ -23,7 +23,7 @@
 import java.util.regex.Pattern;
 
 import org.apache.commons.collections4.map.LRUMap;
-import org.apache.commons.lang3.text.WordUtils;
+import org.apache.commons.text.WordUtils;
 
 public class StringUtil {
     private static final Map<String, String> CAMEL_CACHE = Collections.synchronizedMap(new LRUMap<>(1024));
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
new file mode 100644
index 0000000..e092666
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hyracks.util.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * The method on which this annotation is applied has a very high call frequency during normal execution.
+ * Careful consideration should be taken with regard to the number of objects created and any expensive operations in
+ * there.
+ */
+@Documented
+@Target({ ElementType.METHOD })
+@Retention(RetentionPolicy.SOURCE)
+public @interface CriticalPath {
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index 11fb6c0..3b3e7b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -84,6 +84,11 @@
         throw new IllegalStateException();
     }
 
+    public static boolean isCharStart(byte[] b, int s) {
+        int c = b[s] & 0xff;
+        return (c >> 6) != 2;
+    }
+
     public static int getModifiedUTF8Len(char c) {
         if (c >= 0x0001 && c <= 0x007F) {
             return 1;
@@ -289,6 +294,11 @@
         return hash(bytes, start, length, true, false, 31, Integer.MAX_VALUE);
     }
 
+    public static String toString(byte[] bytes, int start) {
+        StringBuilder builder = new StringBuilder();
+        return toString(builder, bytes, start).toString();
+    }
+
     public static StringBuilder toString(StringBuilder builder, byte[] bytes, int start) {
         int utfLen = getUTFLength(bytes, start);
         int offset = getNumBytesToStoreLength(utfLen);
@@ -371,7 +381,7 @@
 
     private static int compareTo(byte[] thisBytes, int thisStart, byte[] thatBytes, int thatStart, boolean useLowerCase,
             boolean useRawByte) {
-        int thisLength = getUTFLength(thisBytes, thisStart);;
+        int thisLength = getUTFLength(thisBytes, thisStart);
         int thatLength = getUTFLength(thatBytes, thatStart);
         int thisActualStart = thisStart + getNumBytesToStoreLength(thisLength);
         int thatActualStart = thatStart + getNumBytesToStoreLength(thatLength);
@@ -595,7 +605,7 @@
         out.write(tempBytes, 0, count);
     }
 
-    static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
+    public static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
             throws IOException {
         int utflen = 0;
         int count = 0;
@@ -645,10 +655,10 @@
         if (writer == null) {
             tempBytes = new byte[utflen + 5];
         } else {
-            if (writer.tempBytes == null || writer.tempBytes.length < utflen + 5) {
-                writer.tempBytes = new byte[utflen + 5];
+            if (writer.getTempBytes() == null || writer.getTempBytes().length < utflen + 5) {
+                writer.setTempBytes(new byte[utflen + 5]);
             }
-            tempBytes = writer.tempBytes;
+            tempBytes = writer.getTempBytes();
         }
         return tempBytes;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
index 485bd7d..3d9a7c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringWriter.java
@@ -20,9 +20,12 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.io.Serializable;
 
-public class UTF8StringWriter {
-    byte[] tempBytes;
+public class UTF8StringWriter implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private byte[] tempBytes;
 
     public final void writeUTF8(CharSequence str, DataOutput out) throws IOException {
         UTF8StringUtil.writeUTF8(str, out, this);
@@ -32,4 +35,12 @@
         UTF8StringUtil.writeUTF8(buffer, start, length, out, this);
     }
 
+    public byte[] getTempBytes() {
+        return tempBytes;
+    }
+
+    public void setTempBytes(byte[] tempBytes) {
+        this.tempBytes = tempBytes;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index 7bef19e..6255d85 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -35,7 +35,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-plugin-plugin</artifactId>
-          <version>3.5.1</version>
+          <version>3.5.2</version>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
         <plugin>
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index e72fa97..5987022 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -48,7 +48,7 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
-    <jdk.version>1.8</jdk.version>
+    <source.jdk.version>1.8</source.jdk.version>
     <javac.xlint.value>all</javac.xlint.value>
     <jvm.extraargs />
     <sonar.jacoco.reportPath>${env.PWD}/target/jacoco-merged.exec</sonar.jacoco.reportPath>
@@ -65,7 +65,7 @@
     <test.includes>${global.test.includes}</test.includes>
     <test.excludes>${global.test.excludes}</test.excludes>
     <!-- Versions under dependencymanagement or used in many projects via properties -->
-    <hadoop.version>2.2.0</hadoop.version>
+    <hadoop.version>2.8.5</hadoop.version>
     <jacoco.version>0.7.6.201602180812</jacoco.version>
 
     <implementation.title>Apache Hyracks and Algebricks - ${project.name}</implementation.title>
@@ -78,7 +78,7 @@
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty-all</artifactId>
-        <version>4.1.6.Final</version>
+        <version>4.1.32.Final</version>
       </dependency>
       <dependency>
         <groupId>junit</groupId>
@@ -142,19 +142,24 @@
         <version>2.6</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-text</artifactId>
+        <version>1.6</version>
+      </dependency>
+      <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-databind</artifactId>
-        <version>2.8.4</version>
+        <version>2.9.7</version>
       </dependency>
       <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-core</artifactId>
-        <version>2.8.4</version>
+        <version>2.9.7</version>
       </dependency>
       <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-annotations</artifactId>
-        <version>2.8.4</version>
+        <version>2.9.7</version>
       </dependency>
       <dependency>
         <groupId>com.google.guava</groupId>
@@ -164,27 +169,27 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-lang3</artifactId>
-        <version>3.7</version>
+        <version>3.8.1</version>
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-collections4</artifactId>
-        <version>4.1</version>
+        <version>4.2</version>
       </dependency>
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
         <artifactId>httpcore</artifactId>
-        <version>4.4.5</version>
+        <version>4.4.10</version>
       </dependency>
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
         <artifactId>httpclient</artifactId>
-        <version>4.5.2</version>
+        <version>4.5.6</version>
       </dependency>
       <dependency>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
-        <version>0.12</version>
+        <version>0.13</version>
       </dependency>
       <dependency>
         <groupId>net.revelc.code.formatter</groupId>
@@ -199,18 +204,23 @@
       <dependency>
         <groupId>org.apache.logging.log4j</groupId>
         <artifactId>log4j-api</artifactId>
-        <version>2.10.0</version>
+        <version>2.11.1</version>
       </dependency>
       <dependency>
         <groupId>org.apache.logging.log4j</groupId>
         <artifactId>log4j-core</artifactId>
-        <version>2.10.0</version>
+        <version>2.11.1</version>
       </dependency>
       <dependency>
         <groupId>org.mockito</groupId>
         <artifactId>mockito-all</artifactId>
         <version>2.0.2-beta</version>
       </dependency>
+      <dependency>
+        <groupId>javax.xml.bind</groupId>
+        <artifactId>jaxb-api</artifactId>
+        <version>2.3.0</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <build>
@@ -286,7 +296,7 @@
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
           <argLine>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8 -Xdebug
-            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${coverageArgLine} 
+            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${coverageArgLine}
           </argLine>
           <systemPropertyVariables>
             <log4j.configurationFile>${testLog4jConfigFile}</log4j.configurationFile>
@@ -313,8 +323,8 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>${jdk.version}</source>
-          <target>${jdk.version}</target>
+          <source>${source.jdk.version}</source>
+          <target>${target.jdk.version}</target>
           <compilerArgument>-Xlint:${javac.xlint.value}</compilerArgument>
         </configuration>
       </plugin>
@@ -420,7 +430,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
-          <version>3.7.0</version>
+          <version>3.8.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.rat</groupId>
@@ -554,7 +564,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.0.2</version>
+          <version>3.1.1</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.doxia</groupId>
@@ -692,10 +702,72 @@
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
+    <profile>
+      <id>java8</id>
+      <activation>
+        <jdk>1.8</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>8</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java9</id>
+      <activation>
+        <jdk>9</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>9</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java10</id>
+      <activation>
+        <jdk>10</jdk>
+      </activation>
+      <properties>
+        <target.jdk.version>10</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java11</id>
+      <activation>
+        <jdk>11</jdk>
+      </activation>
+      <properties>
+        <!-- several plugins do not yet support Java 11 classes (maven-plugin-plugin, maven-dependency-plugin) -->
+        <target.jdk.version>10</target.jdk.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>java8-bootclasspath</id>
+      <activation>
+        <property>
+          <name>java8-bootclasspath</name>
+        </property>
+      </activation>
+      <properties>
+        <target.jdk.version>8</target.jdk.version>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <compilerArgs combine.children="append">
+                <arg>-bootclasspath</arg>
+                <arg>${java8-bootclasspath}</arg>
+              </compilerArgs>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
   </profiles>
   <modules>
     <module>hyracks</module>
     <module>algebricks</module>
     <module>hyracks-fullstack-license</module>
   </modules>
-</project>
\ No newline at end of file
+</project>